From d3c6404e311a36a64b3666bc36f8255a0eef2466 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 6 Nov 2013 11:13:54 -0800 Subject: [PATCH 001/778] initial commit --- build.sbt | 7 +++ project/plugins.sbt | 26 +++++++++ src/main/scala/expressions/Expression.scala | 6 +++ src/main/scala/expressions/literals.scala | 13 +++++ src/main/scala/frontend/Hive.scala | 54 +++++++++++++++++++ .../scala/plans/logical/LogicalPlan.scala | 7 +++ .../scala/plans/logical/basicOperators.scala | 0 src/main/scala/trees/TreeNode.scala | 4 ++ src/main/scala/types/dataTypes.scala | 7 +++ 9 files changed, 124 insertions(+) create mode 100644 build.sbt create mode 100755 project/plugins.sbt create mode 100644 src/main/scala/expressions/Expression.scala create mode 100644 src/main/scala/expressions/literals.scala create mode 100644 src/main/scala/frontend/Hive.scala create mode 100644 src/main/scala/plans/logical/LogicalPlan.scala create mode 100644 src/main/scala/plans/logical/basicOperators.scala create mode 100644 src/main/scala/trees/TreeNode.scala create mode 100644 src/main/scala/types/dataTypes.scala diff --git a/build.sbt b/build.sbt new file mode 100644 index 0000000000000..316ccef9dcbb6 --- /dev/null +++ b/build.sbt @@ -0,0 +1,7 @@ +// Should this dep be the other way around? libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.8.0" + +libraryDependencies += "org.apache.hive" % "hive-cli" % "0.12.0" + +libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "1.0.4" + +scalaVersion := "2.10.0" \ No newline at end of file diff --git a/project/plugins.sbt b/project/plugins.sbt new file mode 100755 index 0000000000000..d06b220d154c3 --- /dev/null +++ b/project/plugins.sbt @@ -0,0 +1,26 @@ +// Copyright (C) 2012 The Regents of The University California. +// All rights reserved. +// +// Licensed 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. + +addSbtPlugin("org.ensime" % "ensime-sbt-cmd" % "0.1.1") + +addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") + +addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.4.0") + +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2") + +resolvers += Resolver.url( + "sbt-plugin-releases", + new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) diff --git a/src/main/scala/expressions/Expression.scala b/src/main/scala/expressions/Expression.scala new file mode 100644 index 0000000000000..441ed5b1160d9 --- /dev/null +++ b/src/main/scala/expressions/Expression.scala @@ -0,0 +1,6 @@ +package catalyst +package expressions + +import trees._ + +class Expression extends TreeNode diff --git a/src/main/scala/expressions/literals.scala b/src/main/scala/expressions/literals.scala new file mode 100644 index 0000000000000..65a58d1174895 --- /dev/null +++ b/src/main/scala/expressions/literals.scala @@ -0,0 +1,13 @@ +package catalyst +package expressions + +import types._ + +object Literal { + def apply(v: Any): Literal = v match { + // TODO(marmbrus): Use bigInt type? + case i: Int => Literal(v, IntegerType) + } +} + +case class Literal(value: Any, dataType: DataType) extends Expression \ No newline at end of file diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala new file mode 100644 index 0000000000000..9f3acafa03e47 --- /dev/null +++ b/src/main/scala/frontend/Hive.scala @@ -0,0 +1,54 @@ +package catalyst +package frontend + +import org.apache.hadoop.hive.ql.lib.Node +import org.apache.hadoop.hive.ql.parse._ + +import expressions._ +import plans.logical._ + +import collection.JavaConversions._ + +object Hive { + def parseSql(sql: String): LogicalPlan = { + val tree = + ParseUtils.findRootNonNullToken( + (new ParseDriver()).parse(sql, null /* no context required for parsing alone */)) + + nodeToPlan(tree) + } + + /** Extractor for matching Hive's AST Tokens */ + protected object Token { + def unapply(t: Any) = t match { + case t: ASTNode => + Some((t.getText, Option(t.getChildren).map(_.toList).getOrElse(Nil))) + case _ => None + } + } + + protected def nodeToPlan(node: Node): LogicalPlan = node match { + case Token("TOK_QUERY", + fromClause :: + insertClause :: Nil) => + dumpTree(fromClause) + nodeToPlan(fromClause) + case a: ASTNode => + println("FAILURE") + dumpTree(a) + ??? + } + + protected def nodeToExpr(node: Node): Expression = node match { + case other => Literal(1) // WRONG + } + + protected def dumpTree(node: Node, indent: Int = 0) { + node match { + case a: ASTNode => println((" " * indent) + a.getText + " " + a.getClass.getName) + case other => sys.error(s"Non ASTNode encountered: $other") + } + + Option(node.getChildren).map(_.toList).getOrElse(Nil).foreach(dumpTree(_, indent + 1)) + } +} \ No newline at end of file diff --git a/src/main/scala/plans/logical/LogicalPlan.scala b/src/main/scala/plans/logical/LogicalPlan.scala new file mode 100644 index 0000000000000..e5e193c0775e2 --- /dev/null +++ b/src/main/scala/plans/logical/LogicalPlan.scala @@ -0,0 +1,7 @@ +package catalyst +package plans +package logical + +import trees._ + +class LogicalPlan extends TreeNode \ No newline at end of file diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala new file mode 100644 index 0000000000000..1c80dcb2d78c0 --- /dev/null +++ b/src/main/scala/trees/TreeNode.scala @@ -0,0 +1,4 @@ +package catalyst +package trees + +class TreeNode diff --git a/src/main/scala/types/dataTypes.scala b/src/main/scala/types/dataTypes.scala new file mode 100644 index 0000000000000..b07dd7b213add --- /dev/null +++ b/src/main/scala/types/dataTypes.scala @@ -0,0 +1,7 @@ +package catalyst +package types + +sealed class DataType + +case object IntegerType extends DataType +case object StringType extends DataType \ No newline at end of file From 08e4f573daa3751527f114f78aa1e5d30529704c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 6 Nov 2013 19:47:05 -0800 Subject: [PATCH 002/778] upgrade scala include shark. --- build.sbt | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/build.sbt b/build.sbt index 316ccef9dcbb6..e1fa804778432 100644 --- a/build.sbt +++ b/build.sbt @@ -1,7 +1,7 @@ -// Should this dep be the other way around? libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.8.0" +libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.8.0" -libraryDependencies += "org.apache.hive" % "hive-cli" % "0.12.0" +libraryDependencies ++= Seq( + "org.apache.hadoop" % "hadoop-client" % "1.0.4", + "org.scalatest" %% "scalatest" % "1.9.1" % "test") -libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "1.0.4" - -scalaVersion := "2.10.0" \ No newline at end of file +scalaVersion := "2.10.3" \ No newline at end of file From f74c4ee8d166368892bb14d8b15726bd819ce6f4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 6 Nov 2013 19:47:25 -0800 Subject: [PATCH 003/778] parsing a simple query. --- src/main/scala/frontend/Hive.scala | 26 ++++++++++++++++++++------ 1 file changed, 20 insertions(+), 6 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 9f3acafa03e47..07345734580bd 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -1,9 +1,11 @@ package catalyst package frontend +import catalyst.analysis.UnresolvedRelation import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ +import analysis._ import expressions._ import plans.logical._ @@ -30,22 +32,34 @@ object Hive { protected def nodeToPlan(node: Node): LogicalPlan = node match { case Token("TOK_QUERY", fromClause :: - insertClause :: Nil) => - dumpTree(fromClause) - nodeToPlan(fromClause) + Token("TOK_INSERT", + Token("TOK_DESTINATION", + Token("TOK_DIR", // For now only support queries with no INSERT clause. + Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) :: + Token("TOK_SELECT", + selectExprs) :: Nil) :: Nil) => + Project(selectExprs.map(nodeToExpr), nodeToPlan(fromClause)) + case Token("TOK_FROM", + Token("TOK_TABREF", + Token("TOK_TABNAME", + Token(name, Nil) :: Nil) :: Nil) :: Nil) => + UnresolvedRelation(name, None) case a: ASTNode => println("FAILURE") dumpTree(a) ??? } - protected def nodeToExpr(node: Node): Expression = node match { - case other => Literal(1) // WRONG + protected def nodeToExpr(node: Node): NamedExpression = node match { + case Token("TOK_SELEXPR", + Token("TOK_TABLE_OR_COL", + Token(name, Nil) :: Nil) :: Nil) => + UnresolvedAttribute(name) } protected def dumpTree(node: Node, indent: Int = 0) { node match { - case a: ASTNode => println((" " * indent) + a.getText + " " + a.getClass.getName) + case a: ASTNode => println((" " * indent) + a.getText) case other => sys.error(s"Non ASTNode encountered: $other") } From db0299ff7f3c63b650772f73bbb331f9df2a32f9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 6 Nov 2013 19:48:34 -0800 Subject: [PATCH 004/778] basic analysis of relations minus transform function. --- src/main/scala/analysis/Analyzer.scala | 18 +++++++++++ src/main/scala/analysis/Catalog.scala | 8 +++++ src/main/scala/analysis/unresolved.scala | 9 ++++++ .../scala/expressions/namedExpressions.scala | 6 ++++ src/main/scala/plans/QueryPlan.scala | 6 ++++ .../scala/plans/logical/LogicalPlan.scala | 2 +- .../scala/plans/logical/basicOperators.scala | 7 +++++ src/main/scala/rules/Rule.scala | 16 ++++++++++ src/main/scala/rules/RuleExecutor.scala | 16 ++++++++++ src/main/scala/trees/TreeNode.scala | 4 ++- src/test/scala/HiveParsingSuite.scala | 19 ++++++++++++ src/test/scala/TestUtil.scala | 30 +++++++++++++++++++ 12 files changed, 139 insertions(+), 2 deletions(-) create mode 100644 src/main/scala/analysis/Analyzer.scala create mode 100644 src/main/scala/analysis/Catalog.scala create mode 100644 src/main/scala/analysis/unresolved.scala create mode 100644 src/main/scala/expressions/namedExpressions.scala create mode 100644 src/main/scala/plans/QueryPlan.scala create mode 100644 src/main/scala/rules/Rule.scala create mode 100644 src/main/scala/rules/RuleExecutor.scala create mode 100644 src/test/scala/HiveParsingSuite.scala create mode 100644 src/test/scala/TestUtil.scala diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala new file mode 100644 index 0000000000000..d44ada3f37e34 --- /dev/null +++ b/src/main/scala/analysis/Analyzer.scala @@ -0,0 +1,18 @@ +package catalyst +package analysis + +import plans.logical._ +import rules._ + +class Analyzer(catalog: Catalog) extends RuleExecutor[LogicalPlan] { + val batches = Seq( + Batch("Resolution", + ResolveRelations) + ) + + object ResolveRelations extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case UnresolvedRelation(name, alias) => catalog.lookupRelation(name, alias) + } + } +} \ No newline at end of file diff --git a/src/main/scala/analysis/Catalog.scala b/src/main/scala/analysis/Catalog.scala new file mode 100644 index 0000000000000..c647473a9b8b6 --- /dev/null +++ b/src/main/scala/analysis/Catalog.scala @@ -0,0 +1,8 @@ +package catalyst +package analysis + +import plans.logical.LogicalPlan + +abstract class Catalog { + def lookupRelation(name: String, alias: Option[String]): LogicalPlan +} \ No newline at end of file diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala new file mode 100644 index 0000000000000..338c7322a5b6c --- /dev/null +++ b/src/main/scala/analysis/unresolved.scala @@ -0,0 +1,9 @@ +package catalyst +package analysis + +import expressions._ +import plans.logical._ + +case class UnresolvedRelation(name: String, alias: Option[String]) extends LogicalPlan + +case class UnresolvedAttribute(name: String) extends Attribute \ No newline at end of file diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala new file mode 100644 index 0000000000000..755da6f099380 --- /dev/null +++ b/src/main/scala/expressions/namedExpressions.scala @@ -0,0 +1,6 @@ +package catalyst +package expressions + +class NamedExpression extends Expression + +class Attribute extends NamedExpression \ No newline at end of file diff --git a/src/main/scala/plans/QueryPlan.scala b/src/main/scala/plans/QueryPlan.scala new file mode 100644 index 0000000000000..0c20fd1a62c32 --- /dev/null +++ b/src/main/scala/plans/QueryPlan.scala @@ -0,0 +1,6 @@ +package catalyst +package plans + +import trees._ + +class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] \ No newline at end of file diff --git a/src/main/scala/plans/logical/LogicalPlan.scala b/src/main/scala/plans/logical/LogicalPlan.scala index e5e193c0775e2..ff089539ac5bc 100644 --- a/src/main/scala/plans/logical/LogicalPlan.scala +++ b/src/main/scala/plans/logical/LogicalPlan.scala @@ -4,4 +4,4 @@ package logical import trees._ -class LogicalPlan extends TreeNode \ No newline at end of file +class LogicalPlan extends QueryPlan[LogicalPlan] \ No newline at end of file diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index e69de29bb2d1d..b05a33fbbc1a4 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -0,0 +1,7 @@ +package catalyst +package plans +package logical + +import expressions._ + +case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends LogicalPlan \ No newline at end of file diff --git a/src/main/scala/rules/Rule.scala b/src/main/scala/rules/Rule.scala new file mode 100644 index 0000000000000..5f1129c98cebb --- /dev/null +++ b/src/main/scala/rules/Rule.scala @@ -0,0 +1,16 @@ +package catalyst +package rules + +import plans._ + +abstract class Rule[PlanType <: QueryPlan[_]] { + val name = { + val className = getClass.getName + if(className endsWith "$") + className.dropRight(1) + else + className + } + + def apply(plan: PlanType): PlanType +} \ No newline at end of file diff --git a/src/main/scala/rules/RuleExecutor.scala b/src/main/scala/rules/RuleExecutor.scala new file mode 100644 index 0000000000000..0bf18fae45908 --- /dev/null +++ b/src/main/scala/rules/RuleExecutor.scala @@ -0,0 +1,16 @@ +package catalyst +package rules + +import plans._ + +abstract class RuleExecutor[PlanType <: QueryPlan[_]] { + case class Batch(name: String, rules: Rule[PlanType]*) + + val batches: Seq[Batch] + + def apply(plan: PlanType): PlanType = { + batches.foldLeft(plan) { + case (curPlan, batch) => batch.rules.foldRight(curPlan)(_.apply(_)) + } + } +} \ No newline at end of file diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index 1c80dcb2d78c0..e750253fe43ae 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -1,4 +1,6 @@ package catalyst package trees -class TreeNode +class TreeNode[BaseType <: TreeNode[BaseType]] { + def transform(rule: PartialFunction[BaseType, BaseType]): BaseType = ??? +} diff --git a/src/test/scala/HiveParsingSuite.scala b/src/test/scala/HiveParsingSuite.scala new file mode 100644 index 0000000000000..975cb9a06e163 --- /dev/null +++ b/src/test/scala/HiveParsingSuite.scala @@ -0,0 +1,19 @@ +package catalyst +package frontend + +import shark.SharkContext +import shark.SharkEnv + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +class SQLSuite extends FunSuite with BeforeAndAfterAll { + var sc: SharkContext = _ + + override def beforeAll() { + sc = TestUtils.getTestContext + } + + test("trivial select query") { + sc.sql("SELECT key, val FROM test").foreach(println) + } +} \ No newline at end of file diff --git a/src/test/scala/TestUtil.scala b/src/test/scala/TestUtil.scala new file mode 100644 index 0000000000000..5a69b3ab124ee --- /dev/null +++ b/src/test/scala/TestUtil.scala @@ -0,0 +1,30 @@ +package catalyst + +import java.io.File +import shark.SharkEnv + +object TestUtils { + def getTempFilePath(prefix: String, suffix: String = ""): File = { + val tempFile = File.createTempFile(prefix, suffix) + tempFile.delete() + tempFile + } + + def getTestContext = { + val WAREHOUSE_PATH = TestUtils.getTempFilePath("sharkWarehouse") + val METASTORE_PATH = TestUtils.getTempFilePath("sharkMetastore") + val MASTER = "local" + + val sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) + + sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + METASTORE_PATH + ";create=true") + sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) + + // test + //sc.runSql("DROP TABLE IF EXISTS test") + sc.runSql("CREATE TABLE test (key INT, val STRING)") + sc.runSql("""LOAD DATA LOCAL INPATH '/Users/marmbrus/workspace/hive/data/files/kv1.txt' INTO TABLE test""") + + sc + } +} \ No newline at end of file From 6a158cbcd7bc5f2bf5ebc4da23137a0c6430a372 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 6 Nov 2013 22:28:33 -0800 Subject: [PATCH 005/778] simple transform working. --- src/main/scala/analysis/unresolved.scala | 4 +- src/main/scala/expressions/Expression.scala | 12 ++++- src/main/scala/expressions/arithmetic.scala | 4 ++ src/main/scala/expressions/literals.scala | 4 +- .../scala/expressions/namedExpressions.scala | 8 ++- src/main/scala/expressions/package.scala | 4 ++ src/main/scala/plans/QueryPlan.scala | 4 +- .../scala/plans/logical/LogicalPlan.scala | 12 ++++- .../scala/plans/logical/basicOperators.scala | 2 +- src/main/scala/trees/TreeNode.scala | 49 ++++++++++++++++++- src/test/scala/TreeNodeSuite.scala | 22 +++++++++ 11 files changed, 113 insertions(+), 12 deletions(-) create mode 100644 src/main/scala/expressions/arithmetic.scala create mode 100644 src/main/scala/expressions/package.scala create mode 100644 src/test/scala/TreeNodeSuite.scala diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala index 338c7322a5b6c..4cd2bd467c6ac 100644 --- a/src/main/scala/analysis/unresolved.scala +++ b/src/main/scala/analysis/unresolved.scala @@ -4,6 +4,6 @@ package analysis import expressions._ import plans.logical._ -case class UnresolvedRelation(name: String, alias: Option[String]) extends LogicalPlan +case class UnresolvedRelation(name: String, alias: Option[String]) extends LeafNode -case class UnresolvedAttribute(name: String) extends Attribute \ No newline at end of file +case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] \ No newline at end of file diff --git a/src/main/scala/expressions/Expression.scala b/src/main/scala/expressions/Expression.scala index 441ed5b1160d9..26f67f03ccb56 100644 --- a/src/main/scala/expressions/Expression.scala +++ b/src/main/scala/expressions/Expression.scala @@ -3,4 +3,14 @@ package expressions import trees._ -class Expression extends TreeNode +abstract class Expression extends TreeNode[Expression] { + self: Product => +} + +abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] { + self: Product => +} + +abstract class LeafExpression extends Expression with trees.LeafNode[Expression] { + self: Product => +} diff --git a/src/main/scala/expressions/arithmetic.scala b/src/main/scala/expressions/arithmetic.scala new file mode 100644 index 0000000000000..51b5bece2f559 --- /dev/null +++ b/src/main/scala/expressions/arithmetic.scala @@ -0,0 +1,4 @@ +package catalyst +package expressions + +case class Add(left: Expression, right: Expression) extends BinaryExpression \ No newline at end of file diff --git a/src/main/scala/expressions/literals.scala b/src/main/scala/expressions/literals.scala index 65a58d1174895..ead70a666eb38 100644 --- a/src/main/scala/expressions/literals.scala +++ b/src/main/scala/expressions/literals.scala @@ -5,9 +5,9 @@ import types._ object Literal { def apply(v: Any): Literal = v match { - // TODO(marmbrus): Use bigInt type? + // TODO(marmbrus): Use bigInt type for value? case i: Int => Literal(v, IntegerType) } } -case class Literal(value: Any, dataType: DataType) extends Expression \ No newline at end of file +case class Literal(value: Any, dataType: DataType) extends LeafExpression \ No newline at end of file diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index 755da6f099380..c99c38012fc70 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -1,6 +1,10 @@ package catalyst package expressions -class NamedExpression extends Expression +abstract class NamedExpression extends Expression { + self: Product => +} -class Attribute extends NamedExpression \ No newline at end of file +abstract class Attribute extends NamedExpression { + self: Product => +} \ No newline at end of file diff --git a/src/main/scala/expressions/package.scala b/src/main/scala/expressions/package.scala new file mode 100644 index 0000000000000..aaddda3c9518a --- /dev/null +++ b/src/main/scala/expressions/package.scala @@ -0,0 +1,4 @@ +package catalyst + +package object expressions { +} \ No newline at end of file diff --git a/src/main/scala/plans/QueryPlan.scala b/src/main/scala/plans/QueryPlan.scala index 0c20fd1a62c32..a937d4b3a01ba 100644 --- a/src/main/scala/plans/QueryPlan.scala +++ b/src/main/scala/plans/QueryPlan.scala @@ -3,4 +3,6 @@ package plans import trees._ -class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] \ No newline at end of file +abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { + self: PlanType with Product => +} \ No newline at end of file diff --git a/src/main/scala/plans/logical/LogicalPlan.scala b/src/main/scala/plans/logical/LogicalPlan.scala index ff089539ac5bc..1abece2d7c620 100644 --- a/src/main/scala/plans/logical/LogicalPlan.scala +++ b/src/main/scala/plans/logical/LogicalPlan.scala @@ -4,4 +4,14 @@ package logical import trees._ -class LogicalPlan extends QueryPlan[LogicalPlan] \ No newline at end of file +abstract class LogicalPlan extends QueryPlan[LogicalPlan] { + self: Product => +} + +abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { + self: Product => +} + +abstract class UnaryNode extends LogicalPlan with trees.UnaryNode[LogicalPlan] { + self: Product => +} \ No newline at end of file diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index b05a33fbbc1a4..a21569f519dc1 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -4,4 +4,4 @@ package logical import expressions._ -case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends LogicalPlan \ No newline at end of file +case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode \ No newline at end of file diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index e750253fe43ae..4830528ed7cb3 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -1,6 +1,51 @@ package catalyst package trees -class TreeNode[BaseType <: TreeNode[BaseType]] { - def transform(rule: PartialFunction[BaseType, BaseType]): BaseType = ??? +abstract class TreeNode[BaseType <: TreeNode[BaseType]] { + self: BaseType with Product => + + def children: Seq[BaseType] + + def transform(rule: PartialFunction[BaseType, BaseType]): BaseType = { + val afterRule = rule.applyOrElse(this, identity[BaseType]) + if(this == afterRule) + transformChildren(rule) + else + afterRule.transformChildren(rule) + } + + def transformChildren(rule: PartialFunction[BaseType, BaseType]): this.type = { + var changed = false + val newArgs = productIterator.map { + case arg: AnyRef if(children contains arg) => + val newChild = arg.asInstanceOf[BaseType].transform(rule) + if(newChild != arg) { + changed = true + newChild + } else { + arg + } + case nonChild: AnyRef => nonChild + }.toArray + if(changed) makeCopy(newArgs) else this + } + + def makeCopy(newArgs: Array[AnyRef]): this.type = + getClass.getConstructors.head.newInstance(newArgs: _*).asInstanceOf[this.type] +} + +trait BinaryNode[BaseType <: TreeNode[BaseType]] { + def left: BaseType + def right: BaseType + + def children = Seq(left, right) +} + +trait LeafNode[BaseType <: TreeNode[BaseType]] { + def children = Nil +} + +trait UnaryNode[BaseType <: TreeNode[BaseType]] { + def child: BaseType + def children = child :: Nil } diff --git a/src/test/scala/TreeNodeSuite.scala b/src/test/scala/TreeNodeSuite.scala new file mode 100644 index 0000000000000..9748b1fa1ab75 --- /dev/null +++ b/src/test/scala/TreeNodeSuite.scala @@ -0,0 +1,22 @@ +package catalyst +package trees + +import catalyst.types.IntegerType +import expressions._ + +import org.scalatest.{FunSuite} + +class TransformSuite extends FunSuite { + + test("top node changed") { + val after = Literal(1) transform { case Literal(1, _) => Literal(2) } + assert(after === Literal(2,IntegerType)) + } + + test("one child changed") { + val before = Add(Literal(1), Literal(2)) + val after = before transform { case Literal(2, _) => Literal(1) } + + assert(after === Add(Literal(1), Literal(1))) + } +} \ No newline at end of file From daa71caffe053fffb19e2b47f365a194f57cfc45 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 6 Nov 2013 23:28:35 -0800 Subject: [PATCH 006/778] foreach, maps, and scaladoc --- src/main/scala/trees/TreeNode.scala | 62 ++++++++++++++++++++++++++++- 1 file changed, 61 insertions(+), 1 deletion(-) diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index 4830528ed7cb3..76310ea0a598c 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -1,11 +1,50 @@ package catalyst package trees +object TreeNode { + private val currentId = new java.util.concurrent.atomic.AtomicLong + protected def nextId() = currentId.getAndIncrement() +} + abstract class TreeNode[BaseType <: TreeNode[BaseType]] { self: BaseType with Product => + /** Returns a Seq of the children of this node */ def children: Seq[BaseType] + /** + * A globally unique id for this specific instance. Not preserved across copies. + * Unlike [[equals]] [[id]] be used to differentiate distinct but stucturally + * identical branches of a tree. + */ + val id = TreeNode.nextId() + + /** + * Runs [[f]] on this node and then recursively on [[children]]. + * @param f the function to be applied. + */ + def foreach(f: BaseType => Unit): Unit = { + f(this) + children.foreach(f) + } + + /** + * Returns a Seq containing the result of applying [[f]] to each + * node in this tree in a preorder traversal. + * @param f the function to be applied. + */ + def map[A](f: BaseType => A): Seq[A] = { + val ret = new collection.mutable.ArrayBuffer[A]() + foreach(ret += f(_)) + ret + } + + /** + * Returns a copy of this node where [[rule]] has been recursively + * applied to it and all of its children. When [[rule]] does not + * apply to a given node it is left unchanged. + * @param rule the function use to transform this nodes children + */ def transform(rule: PartialFunction[BaseType, BaseType]): BaseType = { val afterRule = rule.applyOrElse(this, identity[BaseType]) if(this == afterRule) @@ -14,6 +53,12 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { afterRule.transformChildren(rule) } + /** + * Returns a copy of this node where [[rule]] has been recursively + * applied to all the children of this node. When [[rule]] does not + * apply to a given node it is left unchanged. + * @param rule the function use to transform this nodes children + */ def transformChildren(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { @@ -30,10 +75,19 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { if(changed) makeCopy(newArgs) else this } - def makeCopy(newArgs: Array[AnyRef]): this.type = + /** + * Creates a copy of this type of tree node after a transformation. + * Must be overridden by child classes that have constructor arguments + * that are not present in the [[productIterator]]. + * @param newArgs the new product arguments. + */ + protected def makeCopy(newArgs: Array[AnyRef]): this.type = getClass.getConstructors.head.newInstance(newArgs: _*).asInstanceOf[this.type] } +/** + * A [[TreeNode]] that has two children, [[left]] and [[right]]. + */ trait BinaryNode[BaseType <: TreeNode[BaseType]] { def left: BaseType def right: BaseType @@ -41,10 +95,16 @@ trait BinaryNode[BaseType <: TreeNode[BaseType]] { def children = Seq(left, right) } +/** + * A [[TreeNode]] with no children. + */ trait LeafNode[BaseType <: TreeNode[BaseType]] { def children = Nil } +/** + * A [[TreeNode]] with a single [[child]]. + */ trait UnaryNode[BaseType <: TreeNode[BaseType]] { def child: BaseType def children = child :: Nil From a1a8b381ffe7d06f74c60b6a55b7ca67bc0f7f0f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 6 Nov 2013 23:29:06 -0800 Subject: [PATCH 007/778] test that ids don't change for no-op transforms. --- src/test/scala/TreeNodeSuite.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/test/scala/TreeNodeSuite.scala b/src/test/scala/TreeNodeSuite.scala index 9748b1fa1ab75..85579e461a62c 100644 --- a/src/test/scala/TreeNodeSuite.scala +++ b/src/test/scala/TreeNodeSuite.scala @@ -19,4 +19,12 @@ class TransformSuite extends FunSuite { assert(after === Add(Literal(1), Literal(1))) } + + test("no change") { + val before = Add(Literal(1), Add(Literal(2), Add(Literal(3), Literal(4)))) + val after = before transform { case Literal(5, _) => Literal(1)} + + assert(before === after) + assert(before.map(_.id) === after.map(_.id)) + } } \ No newline at end of file From 0608a0084e1631778e4c7fa0e311cf18b681da14 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 00:04:45 -0800 Subject: [PATCH 008/778] tighten public interface --- src/main/scala/rules/RuleExecutor.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/rules/RuleExecutor.scala b/src/main/scala/rules/RuleExecutor.scala index 0bf18fae45908..3e732da259195 100644 --- a/src/main/scala/rules/RuleExecutor.scala +++ b/src/main/scala/rules/RuleExecutor.scala @@ -4,9 +4,9 @@ package rules import plans._ abstract class RuleExecutor[PlanType <: QueryPlan[_]] { - case class Batch(name: String, rules: Rule[PlanType]*) + protected case class Batch(name: String, rules: Rule[PlanType]*) - val batches: Seq[Batch] + protected val batches: Seq[Batch] def apply(plan: PlanType): PlanType = { batches.foldLeft(plan) { From be25003914e149cf2418bd570b31a64834e77ba5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 18:12:56 -0800 Subject: [PATCH 009/778] add repl initialization to sbt. --- build.sbt | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index e1fa804778432..0fe13548b53fa 100644 --- a/build.sbt +++ b/build.sbt @@ -4,4 +4,17 @@ libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client" % "1.0.4", "org.scalatest" %% "scalatest" % "1.9.1" % "test") -scalaVersion := "2.10.3" \ No newline at end of file +scalaVersion := "2.10.3" + +initialCommands in console := """ +import catalyst.analysis._ +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.frontend._ +import catalyst.plans.logical._ +import catalyst.plans.physical +import catalyst.rules._ +import catalyst.types._ +import catalyst.util._ +lazy val testShark = new catalyst.util.TestShark +import testShark._""" \ No newline at end of file From 002d4d471e8a33d1066de32ce4fc175a4d86bdf6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 18:13:12 -0800 Subject: [PATCH 010/778] default to no alias. --- src/main/scala/analysis/Catalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/analysis/Catalog.scala b/src/main/scala/analysis/Catalog.scala index c647473a9b8b6..57f5e5fd3d62f 100644 --- a/src/main/scala/analysis/Catalog.scala +++ b/src/main/scala/analysis/Catalog.scala @@ -4,5 +4,5 @@ package analysis import plans.logical.LogicalPlan abstract class Catalog { - def lookupRelation(name: String, alias: Option[String]): LogicalPlan + def lookupRelation(name: String, alias: Option[String] = None): LogicalPlan } \ No newline at end of file From 45da47b5c7e42d413d1529b84fcfc775eb769b0d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 18:14:20 -0800 Subject: [PATCH 011/778] flesh out plans and expressions a little. first cut at named expressions. --- src/main/scala/analysis/unresolved.scala | 18 +++++++--- src/main/scala/expressions/Expression.scala | 4 +++ src/main/scala/expressions/arithmetic.scala | 10 +++++- src/main/scala/expressions/literals.scala | 4 ++- .../scala/expressions/namedExpressions.scala | 36 +++++++++++++++++++ src/main/scala/plans/QueryPlan.scala | 3 ++ .../scala/plans/logical/basicOperators.scala | 4 ++- 7 files changed, 72 insertions(+), 7 deletions(-) diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala index 4cd2bd467c6ac..856a49b192d31 100644 --- a/src/main/scala/analysis/unresolved.scala +++ b/src/main/scala/analysis/unresolved.scala @@ -1,9 +1,19 @@ package catalyst package analysis -import expressions._ -import plans.logical._ +import expressions.{Attribute, Expression} +import plans.logical.LeafNode +import trees.TreeNode -case class UnresolvedRelation(name: String, alias: Option[String]) extends LeafNode +class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: String) extends + errors.OptimizationException(tree, "Invalid call to $function on unresolved object") -case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] \ No newline at end of file +case class UnresolvedRelation(name: String, alias: Option[String]) extends LeafNode { + def output = throw new UnresolvedException(this, "output") +} + +case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { + def exprId = throw new UnresolvedException(this, "exprId") + def dataType = throw new UnresolvedException(this, "dataType") + def nullable = throw new UnresolvedException(this, "nullable") +} \ No newline at end of file diff --git a/src/main/scala/expressions/Expression.scala b/src/main/scala/expressions/Expression.scala index 26f67f03ccb56..8c2a2a2fd2165 100644 --- a/src/main/scala/expressions/Expression.scala +++ b/src/main/scala/expressions/Expression.scala @@ -2,9 +2,13 @@ package catalyst package expressions import trees._ +import types._ abstract class Expression extends TreeNode[Expression] { self: Product => + + def dataType: DataType + def nullable: Boolean } abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] { diff --git a/src/main/scala/expressions/arithmetic.scala b/src/main/scala/expressions/arithmetic.scala index 51b5bece2f559..70e47300fe378 100644 --- a/src/main/scala/expressions/arithmetic.scala +++ b/src/main/scala/expressions/arithmetic.scala @@ -1,4 +1,12 @@ package catalyst package expressions -case class Add(left: Expression, right: Expression) extends BinaryExpression \ No newline at end of file +case class Add(left: Expression, right: Expression) extends BinaryExpression { + def dataType = { + require(left.dataType == right.dataType) // TODO(marmbrus): Figure out rules for coersions. + left.dataType + } + + def nullable = left.nullable || right.nullable + +} \ No newline at end of file diff --git a/src/main/scala/expressions/literals.scala b/src/main/scala/expressions/literals.scala index ead70a666eb38..24cee7e95bb88 100644 --- a/src/main/scala/expressions/literals.scala +++ b/src/main/scala/expressions/literals.scala @@ -10,4 +10,6 @@ object Literal { } } -case class Literal(value: Any, dataType: DataType) extends LeafExpression \ No newline at end of file +case class Literal(value: Any, dataType: DataType) extends LeafExpression { + final def nullable = false +} \ No newline at end of file diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index c99c38012fc70..1d58cfed8baf5 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -1,10 +1,46 @@ package catalyst package expressions +import types._ + +object NamedExpression { + private val curId = new java.util.concurrent.atomic.AtomicLong() + def newExprId = ExprId(curId.getAndIncrement()) +} + +/** + * A globally (within this JVM) id for a given named expression. + * Used to identify with attribute output by a relation is being + * referenced in a subsuqent computation. + */ +case class ExprId(id: Long) + abstract class NamedExpression extends Expression { self: Product => + + def name: String + def exprId: ExprId + def toAttribute: Attribute } abstract class Attribute extends NamedExpression { self: Product => + + def toAttribute = this +} + +case class Alias(child: Expression, name: String) + (val exprId: ExprId = NamedExpression.newExprId) + extends NamedExpression with trees.UnaryNode[Expression] { + + def dataType = child.dataType + def nullable = child.nullable + + def toAttribute = AttributeReference(name, child.dataType, child.nullable)(exprId) +} + +case class AttributeReference(name: String, dataType: DataType, nullable: Boolean = true) + (val exprId: ExprId = NamedExpression.newExprId) + extends Attribute with trees.LeafNode[Expression] { + } \ No newline at end of file diff --git a/src/main/scala/plans/QueryPlan.scala b/src/main/scala/plans/QueryPlan.scala index a937d4b3a01ba..8434a4cea61f1 100644 --- a/src/main/scala/plans/QueryPlan.scala +++ b/src/main/scala/plans/QueryPlan.scala @@ -1,8 +1,11 @@ package catalyst package plans +import expressions.Attribute import trees._ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => + + def output: Seq[Attribute] } \ No newline at end of file diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index a21569f519dc1..5712af8ca9c1d 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -4,4 +4,6 @@ package logical import expressions._ -case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode \ No newline at end of file +case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { + def output = projectList.map(_.toAttribute) +} \ No newline at end of file From 7fb3d5eefca2e7a1f2ee15875c717e8ac30875e6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 18:14:33 -0800 Subject: [PATCH 012/778] docs and equality improvements. --- src/main/scala/trees/TreeNode.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index 76310ea0a598c..4a684a110922d 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -21,7 +21,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** * Runs [[f]] on this node and then recursively on [[children]]. - * @param f the function to be applied. + * @param f the function to be applied to each node in the tree. */ def foreach(f: BaseType => Unit): Unit = { f(this) @@ -47,7 +47,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { */ def transform(rule: PartialFunction[BaseType, BaseType]): BaseType = { val afterRule = rule.applyOrElse(this, identity[BaseType]) - if(this == afterRule) + // Check if unchanged and then possibly return old copy to avoid gc churn. + if(this.id == afterRule.id || this == afterRule) transformChildren(rule) else afterRule.transformChildren(rule) @@ -62,9 +63,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def transformChildren(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { - case arg: AnyRef if(children contains arg) => + case arg: TreeNode[_] if(children contains arg) => val newChild = arg.asInstanceOf[BaseType].transform(rule) - if(newChild != arg) { + if(newChild.id != arg.id && newChild != arg) { changed = true newChild } else { From 3766a41ecd12cce4d520278d2681a58386e09a29 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 18:15:49 -0800 Subject: [PATCH 013/778] rearrange utility functions. --- src/main/scala/util/TestShark.scala | 45 +++++++++++++++++++++++++++ src/main/scala/util/package.scala | 11 +++++++ src/test/scala/HiveParsingSuite.scala | 5 ++- src/test/scala/TestUtil.scala | 30 ------------------ 4 files changed, 60 insertions(+), 31 deletions(-) create mode 100644 src/main/scala/util/TestShark.scala create mode 100644 src/main/scala/util/package.scala delete mode 100644 src/test/scala/TestUtil.scala diff --git a/src/main/scala/util/TestShark.scala b/src/main/scala/util/TestShark.scala new file mode 100644 index 0000000000000..26140595d33be --- /dev/null +++ b/src/main/scala/util/TestShark.scala @@ -0,0 +1,45 @@ +package catalyst +package util + +import catalyst.analysis.{Analyzer, HiveMetastoreCatalog} +import catalyst.frontend.Hive +import shark.{SharkContext, SharkEnv} + +import util._ + +class TestShark { + val WAREHOUSE_PATH = getTempFilePath("sharkWarehouse") + val METASTORE_PATH = getTempFilePath("sharkMetastore") + val MASTER = "local" + + val sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) + + sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + METASTORE_PATH + ";create=true") + sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) + + def loadKv1 { + //sc.runSql("DROP TABLE IF EXISTS test") + sc.runSql("CREATE TABLE test (key INT, val STRING)") + sc.runSql("""LOAD DATA LOCAL INPATH '/Users/marmbrus/workspace/hive/data/files/kv1.txt' INTO TABLE test""") + } + + val catalog = new HiveMetastoreCatalog(SharkContext.hiveconf) + val analyze = new Analyzer(new HiveMetastoreCatalog(SharkContext.hiveconf)) + + class SharkQuery(sql: String) { + lazy val parsed = Hive.parseSql(sql) + lazy val analyzed = analyze(parsed) + + override def toString(): String = + s""" + |$sql + | + |== Logical Plan == + |$analyzed + """.stripMargin + } + + implicit class stringToQuery(str: String) { + def q = new SharkQuery(str) + } +} \ No newline at end of file diff --git a/src/main/scala/util/package.scala b/src/main/scala/util/package.scala new file mode 100644 index 0000000000000..31a36e29a41cd --- /dev/null +++ b/src/main/scala/util/package.scala @@ -0,0 +1,11 @@ +package catalyst + +import java.io.File + +package object util { + def getTempFilePath(prefix: String, suffix: String = ""): File = { + val tempFile = File.createTempFile(prefix, suffix) + tempFile.delete() + tempFile + } +} \ No newline at end of file diff --git a/src/test/scala/HiveParsingSuite.scala b/src/test/scala/HiveParsingSuite.scala index 975cb9a06e163..3a2da9415ed96 100644 --- a/src/test/scala/HiveParsingSuite.scala +++ b/src/test/scala/HiveParsingSuite.scala @@ -5,12 +5,15 @@ import shark.SharkContext import shark.SharkEnv import org.scalatest.{BeforeAndAfterAll, FunSuite} +import util.TestShark class SQLSuite extends FunSuite with BeforeAndAfterAll { var sc: SharkContext = _ override def beforeAll() { - sc = TestUtils.getTestContext + val testShark = new TestShark + testShark.loadKv1 + sc = testShark.sc } test("trivial select query") { diff --git a/src/test/scala/TestUtil.scala b/src/test/scala/TestUtil.scala deleted file mode 100644 index 5a69b3ab124ee..0000000000000 --- a/src/test/scala/TestUtil.scala +++ /dev/null @@ -1,30 +0,0 @@ -package catalyst - -import java.io.File -import shark.SharkEnv - -object TestUtils { - def getTempFilePath(prefix: String, suffix: String = ""): File = { - val tempFile = File.createTempFile(prefix, suffix) - tempFile.delete() - tempFile - } - - def getTestContext = { - val WAREHOUSE_PATH = TestUtils.getTempFilePath("sharkWarehouse") - val METASTORE_PATH = TestUtils.getTempFilePath("sharkMetastore") - val MASTER = "local" - - val sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) - - sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + METASTORE_PATH + ";create=true") - sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - - // test - //sc.runSql("DROP TABLE IF EXISTS test") - sc.runSql("CREATE TABLE test (key INT, val STRING)") - sc.runSql("""LOAD DATA LOCAL INPATH '/Users/marmbrus/workspace/hive/data/files/kv1.txt' INTO TABLE test""") - - sc - } -} \ No newline at end of file From 1c111361acfef22e9ec3a5f9e6f4d06696cb0b48 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 18:16:07 -0800 Subject: [PATCH 014/778] first draft of error handling / plans for debugging. --- src/main/scala/errors/package.scala | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 src/main/scala/errors/package.scala diff --git a/src/main/scala/errors/package.scala b/src/main/scala/errors/package.scala new file mode 100644 index 0000000000000..97b975c0e39ab --- /dev/null +++ b/src/main/scala/errors/package.scala @@ -0,0 +1,18 @@ +package catalyst + +import trees._ + +package object errors { + + class OptimizationException[TreeType <: TreeNode[_]] + (tree: TreeType, msg: String, cause: Throwable = null) extends Exception(msg, cause) { + + } + + def attachTree[TreeType <: TreeNode[_], A](tree: TreeType, msg: String = "")(f: => A): A = { + try f catch { + case e: Exception => throw new OptimizationException(tree, msg, e) + } + } + +} \ No newline at end of file From 7e7acf063d4d752355e47f67fe8dd4ffa2b2366f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 18:16:19 -0800 Subject: [PATCH 015/778] physical placeholder. --- src/main/scala/plans/physical/package.scala | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 src/main/scala/plans/physical/package.scala diff --git a/src/main/scala/plans/physical/package.scala b/src/main/scala/plans/physical/package.scala new file mode 100644 index 0000000000000..c0b314d03ad85 --- /dev/null +++ b/src/main/scala/plans/physical/package.scala @@ -0,0 +1,4 @@ +package catalyst +package plans + +package object physical \ No newline at end of file From 7ba19bed33fcca5882b24acd5f10737cb09a52f6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 18:16:32 -0800 Subject: [PATCH 016/778] First draft of interface to hive metastore. --- .../scala/analysis/MetastoreCatalog.scala | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) create mode 100644 src/main/scala/analysis/MetastoreCatalog.scala diff --git a/src/main/scala/analysis/MetastoreCatalog.scala b/src/main/scala/analysis/MetastoreCatalog.scala new file mode 100644 index 0000000000000..53da5b664704b --- /dev/null +++ b/src/main/scala/analysis/MetastoreCatalog.scala @@ -0,0 +1,39 @@ +package catalyst +package analysis + +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.Table +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient + +import expressions.AttributeReference +import plans.logical.{LogicalPlan, LeafNode} +import types._ + +import collection.JavaConversions._ + +class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { + protected val client = new HiveMetaStoreClient(hiveConf) + + def lookupRelation(name: String, alias: Option[String]): LogicalPlan = + MetastoreRelation(name)(client.getTable("default", name)) +} + +object HiveMetatoreTypes { + def toDataType(metastoreType: String): DataType = + metastoreType match { + case "string" => StringType + case "int" => IntegerType + } +} + +case class MetastoreRelation(tableName: String)(val table: Table) extends LeafNode { + + // Must be a stable value since new attributes are born here. + val output = table.getSd.getCols.map { col => + AttributeReference( + col.getName, + HiveMetatoreTypes.toDataType(col.getType), + true // AHHH, who makes a metastore with no concept of nullalbility? + )() + } +} \ No newline at end of file From 78e34bff2a0764896aee460fd05fab1dfeb83992 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 18:16:43 -0800 Subject: [PATCH 017/778] simple git ignore. --- .gitignore | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 .gitignore diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000000000..5e0013d24255d --- /dev/null +++ b/.gitignore @@ -0,0 +1,4 @@ +target/ +.idea/ +metastore_db/ + From 7c45dd7af709a621c097c832cb1633656c308e43 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 Nov 2013 18:36:46 -0800 Subject: [PATCH 018/778] pretty printing of trees. --- src/main/scala/analysis/unresolved.scala | 2 ++ .../scala/expressions/namedExpressions.scala | 1 + src/main/scala/trees/TreeNode.scala | 23 +++++++++++++++++++ 3 files changed, 26 insertions(+) diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala index 856a49b192d31..3000dcc10e019 100644 --- a/src/main/scala/analysis/unresolved.scala +++ b/src/main/scala/analysis/unresolved.scala @@ -16,4 +16,6 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo def exprId = throw new UnresolvedException(this, "exprId") def dataType = throw new UnresolvedException(this, "dataType") def nullable = throw new UnresolvedException(this, "nullable") + + override def toString(): String = s"'$name" } \ No newline at end of file diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index 1d58cfed8baf5..c3abc286113e3 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -43,4 +43,5 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea (val exprId: ExprId = NamedExpression.newExprId) extends Attribute with trees.LeafNode[Expression] { + override def toString(): String = s"$name#${exprId.id}" } \ No newline at end of file diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index 4a684a110922d..308726d656f81 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -84,6 +84,29 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { */ protected def makeCopy(newArgs: Array[AnyRef]): this.type = getClass.getConstructors.head.newInstance(newArgs: _*).asInstanceOf[this.type] + + /** Returns the name of this type of TreeNode. Defaults to the class name */ + def nodeName = getClass.getSimpleName + + /** Returns a string representing the arguments to this node, minus any children */ + def argString = productIterator.flatMap { + case tn: TreeNode[_] if children contains tn => Nil + case seq: Seq[_] => seq.mkString("{", ",", "}") :: Nil + case other => other :: Nil + }.mkString(", ") + + /** String representation of this node without any children */ + def simpleString = s"$nodeName $argString" + + override def toString(): String = generateTreeString(0, new StringBuilder).toString + + protected def generateTreeString(depth: Int, builder: StringBuilder): StringBuilder = { + builder.append(" " * depth) + builder.append(simpleString) + builder.append("\n") + children.foreach(_.generateTreeString(depth + 1, builder)) + builder + } } /** From d49dc023fde5716778155db44f04d0780b1d97e8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 8 Nov 2013 21:25:16 -0800 Subject: [PATCH 019/778] scaladoc for named exprs --- .../scala/expressions/namedExpressions.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index c3abc286113e3..6857efbf5a9e3 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -29,6 +29,15 @@ abstract class Attribute extends NamedExpression { def toAttribute = this } +/** + * Used to assign a new name to a computation. + * For example the SQL expression "1 + 1 AS a" could be represented as follows: + * Alias(Add(Literal(1), Literal(1), "a")() + * + * @param child the computation being performed + * @param name + * @param exprId + */ case class Alias(child: Expression, name: String) (val exprId: ExprId = NamedExpression.newExprId) extends NamedExpression with trees.UnaryNode[Expression] { @@ -39,6 +48,14 @@ case class Alias(child: Expression, name: String) def toAttribute = AttributeReference(name, child.dataType, child.nullable)(exprId) } +/** + * A reference to an attribute produced by another operator in the tree. + * + * @param name The name of this attribute, should only be used during analysis or for debugging. + * @param dataType The [[DataType]] of this attribute. + * @param nullable True if null is a valid value for this attribute. + * @param exprId A globally unique id used to check if different AttributeReferences + */ case class AttributeReference(name: String, dataType: DataType, nullable: Boolean = true) (val exprId: ExprId = NamedExpression.newExprId) extends Attribute with trees.LeafNode[Expression] { From ae7d776ca0cf3982f8da224fa2958854fbce78c0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 8 Nov 2013 21:25:37 -0800 Subject: [PATCH 020/778] add nullability changing function --- src/main/scala/expressions/namedExpressions.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index 6857efbf5a9e3..9036e67c29a2a 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -60,5 +60,14 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea (val exprId: ExprId = NamedExpression.newExprId) extends Attribute with trees.LeafNode[Expression] { + /** + * Returns a copy of this [[AttributeReference]] with changed nullability. + */ + def withNullability(newNullability: Boolean) = + if(nullable == newNullability) + this + else + AttributeReference(name, dataType, newNullability)(exprId) + override def toString(): String = s"$name#${exprId.id}" } \ No newline at end of file From 24ef6fb3c4ee1b567ba6025680157d1849198913 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 8 Nov 2013 21:25:52 -0800 Subject: [PATCH 021/778] toString for alias. --- src/main/scala/expressions/namedExpressions.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index 9036e67c29a2a..d9fab10c764fd 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -46,6 +46,8 @@ case class Alias(child: Expression, name: String) def nullable = child.nullable def toAttribute = AttributeReference(name, child.dataType, child.nullable)(exprId) + + override def toString(): String = s"$child AS $name" } /** From 1e423ebc12b81275139cea0150973d84077114c5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 8 Nov 2013 21:26:36 -0800 Subject: [PATCH 022/778] placeholders in LogicalPlan, docs --- .../scala/plans/logical/LogicalPlan.scala | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/src/main/scala/plans/logical/LogicalPlan.scala b/src/main/scala/plans/logical/LogicalPlan.scala index 1abece2d7c620..72464ee46b984 100644 --- a/src/main/scala/plans/logical/LogicalPlan.scala +++ b/src/main/scala/plans/logical/LogicalPlan.scala @@ -2,16 +2,47 @@ package catalyst package plans package logical +import expressions.Attribute import trees._ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => + + /** + * Returns the set of attributes that are referenced by this node + * during evaluation. + */ + def references: Set[Attribute] = ??? + + /** + * Returns the set of attributes that are output by this node. + */ + def outputSet: Set[Attribute] = ??? + + /** + * Returns the set of attributes that this node takes as + * input from its children. + */ + def inputSet: Set[Attribute] = ??? } +/** + * A logical plan node with no children. + */ abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { self: Product => } +/** + * A logical plan node with single child. + */ abstract class UnaryNode extends LogicalPlan with trees.UnaryNode[LogicalPlan] { self: Product => +} + +/** + * A logical plan node with a left and right child. + */ +abstract class BinaryNode extends LogicalPlan with trees.BinaryNode[LogicalPlan] { + self: Product => } \ No newline at end of file From d9161ce422f5760565336fb29a40eec9a97aa585 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 8 Nov 2013 21:26:49 -0800 Subject: [PATCH 023/778] add join operator --- src/main/scala/plans/logical/basicOperators.scala | 9 +++++++++ src/main/scala/plans/shared.scala | 8 ++++++++ 2 files changed, 17 insertions(+) create mode 100644 src/main/scala/plans/shared.scala diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index 5712af8ca9c1d..26707edf4c2f7 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -6,4 +6,13 @@ import expressions._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) +} + +case class Join( + left: LogicalPlan, + right: LogicalPlan, + joinType: JoinType, + condition: Option[Expression]) extends BinaryNode { + + def output = left.output ++ right.output } \ No newline at end of file diff --git a/src/main/scala/plans/shared.scala b/src/main/scala/plans/shared.scala new file mode 100644 index 0000000000000..75730aa139390 --- /dev/null +++ b/src/main/scala/plans/shared.scala @@ -0,0 +1,8 @@ +package catalyst +package plans + +sealed abstract class JoinType +case object Inner extends JoinType +case object LeftOuter extends JoinType +case object RightOuter extends JoinType +case object FullOuter extends JoinType \ No newline at end of file From 54f3460654b4944a6b4b3a21073743459f04e8d8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 8 Nov 2013 21:27:00 -0800 Subject: [PATCH 024/778] initial optiq parsing. --- src/main/scala/frontend/Optiq.scala | 67 ++++++++++++++++++++++++++ src/test/scala/OptiqParsingSuite.scala | 12 +++++ 2 files changed, 79 insertions(+) create mode 100644 src/main/scala/frontend/Optiq.scala create mode 100644 src/test/scala/OptiqParsingSuite.scala diff --git a/src/main/scala/frontend/Optiq.scala b/src/main/scala/frontend/Optiq.scala new file mode 100644 index 0000000000000..238fa31d28608 --- /dev/null +++ b/src/main/scala/frontend/Optiq.scala @@ -0,0 +1,67 @@ +package catalyst +package frontend + +import org.eigenbase.sql._ +import org.eigenbase.sql.SqlKind._ + +import analysis.{UnresolvedRelation, UnresolvedAttribute} +import expressions._ +import plans._ +import plans.logical._ + +import collection.JavaConversions._ + +object Optiq { + def parseSql(sql: String): LogicalPlan = { + val sqlNode = (new parser.SqlParser(sql)).parseQuery + sqlNodeToPlan(sqlNode) + } + + protected object Call { + def unapply(other: Any) = other match { + case call: SqlCall => + Some((call.getKind, call.getOperands.toSeq)) + case other => None + } + } + + def sqlNodeToPlan(node: SqlNode): LogicalPlan = node match { + case select: SqlSelect => + val selectList = select.getSelectList.map(sqlNodeToNamedExpr) + // Lots of things not supported yet. + require(select.getGroup == null) + require(select.getHaving == null) + require(select.getWhere == null) + require(select.getOffset == null) + require(select.getOrderList == null) + // require(select.getWindowList == null) + + Project(selectList.toSeq, sqlNodeToPlan(select.getFrom)) + + case join: SqlJoin => + Join( + sqlNodeToPlan(join.getLeft), + sqlNodeToPlan(join.getRight), + Inner, + None) + + case ident: SqlIdentifier => + UnresolvedRelation(ident.toString, None) + + case unsupported => + val nodeString = try unsupported.toString catch { case e: Exception => "[FAILED toString]" } + sys.error(s"no rule for sqlNode $nodeString, class: ${unsupported.getClass.getSimpleName}") + } + + def sqlNodeToNamedExpr(node: SqlNode): NamedExpression = node match { + case Call(AS, Seq(child, alias)) => + Alias(sqlNodeToExpr(child), alias.toString)() + case ident: SqlIdentifier => + UnresolvedAttribute(ident.toString) + case unsupported => sys.error(s"no rule for sqlExpr $unsupported, class: ${unsupported.getClass.getSimpleName}") + } + + def sqlNodeToExpr(node: SqlNode): Expression = node match { + case named => sqlNodeToNamedExpr(named) + } +} \ No newline at end of file diff --git a/src/test/scala/OptiqParsingSuite.scala b/src/test/scala/OptiqParsingSuite.scala new file mode 100644 index 0000000000000..fee901b489457 --- /dev/null +++ b/src/test/scala/OptiqParsingSuite.scala @@ -0,0 +1,12 @@ +package catalyst +package frontend + +import org.scalatest.{FunSuite} + +class OptiqParsingSuite extends FunSuite { + test("simple select") { + println(Optiq.parseSql("SELECT a FROM foo")) + println(Optiq.parseSql("SELECT a AS b FROM foo")) + println(Optiq.parseSql("SELECT a AS b FROM foo JOIN bar")) + } +} From 3f5bc98dc7d8805262138cc88afee2e0f3f0402e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 8 Nov 2013 21:27:24 -0800 Subject: [PATCH 025/778] add optiq to sbt. --- build.sbt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 0fe13548b53fa..c70aef6445d42 100644 --- a/build.sbt +++ b/build.sbt @@ -1,8 +1,11 @@ +resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" + libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.8.0" libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client" % "1.0.4", - "org.scalatest" %% "scalatest" % "1.9.1" % "test") + "org.scalatest" %% "scalatest" % "1.9.1" % "test", + "net.hydromatic" % "optiq-core" % "0.4.16-SNAPSHOT") scalaVersion := "2.10.3" From a1cf7540e1b36de7a59f68d9046ef5f6808f57e9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 8 Nov 2013 23:06:48 -0800 Subject: [PATCH 026/778] add joins and equality. --- src/main/scala/frontend/Optiq.scala | 38 +++++++++++++------ .../scala/plans/logical/basicOperators.scala | 4 ++ src/main/scala/types/dataTypes.scala | 3 +- src/test/scala/OptiqParsingSuite.scala | 1 + 4 files changed, 34 insertions(+), 12 deletions(-) diff --git a/src/main/scala/frontend/Optiq.scala b/src/main/scala/frontend/Optiq.scala index 238fa31d28608..7278fd8769c24 100644 --- a/src/main/scala/frontend/Optiq.scala +++ b/src/main/scala/frontend/Optiq.scala @@ -10,6 +10,7 @@ import plans._ import plans.logical._ import collection.JavaConversions._ +import org.eigenbase.sql.SqlJoinOperator.JoinType object Optiq { def parseSql(sql: String): LogicalPlan = { @@ -28,22 +29,35 @@ object Optiq { def sqlNodeToPlan(node: SqlNode): LogicalPlan = node match { case select: SqlSelect => val selectList = select.getSelectList.map(sqlNodeToNamedExpr) - // Lots of things not supported yet. - require(select.getGroup == null) - require(select.getHaving == null) - require(select.getWhere == null) - require(select.getOffset == null) - require(select.getOrderList == null) - // require(select.getWindowList == null) + val from = sqlNodeToPlan(select.getFrom) + val where = Option(select.getWhere) + .map(sqlNodeToExpr) + .map(e => Filter(e, _: LogicalPlan)) + .getOrElse(identity[LogicalPlan](_)) - Project(selectList.toSeq, sqlNodeToPlan(select.getFrom)) + // Not implemented yet. + val group = Option(select.getGroup).map(sqlNodeToExpr).isEmpty || ??? + val having = Option(select.getHaving).map(sqlNodeToExpr).isEmpty || ??? + val offset = Option(select.getOffset).map(sqlNodeToExpr).isEmpty || ??? + val order = Option(select.getOrderList).map(sqlNodeToExpr).isEmpty || ??? + // val windowList = Option(select.getWindowList).map(sqlNodeToExpr) + + Project(selectList.toSeq, + where( + from)) case join: SqlJoin => + require(join.getCondition == null) Join( sqlNodeToPlan(join.getLeft), sqlNodeToPlan(join.getRight), - Inner, - None) + join.getJoinType match { + case JoinType.Full => FullOuter + case JoinType.Inner => Inner + case JoinType.Right => RightOuter + case JoinType.Left => LeftOuter + }, + Option(join.getCondition).map(sqlNodeToExpr)) case ident: SqlIdentifier => UnresolvedRelation(ident.toString, None) @@ -58,10 +72,12 @@ object Optiq { Alias(sqlNodeToExpr(child), alias.toString)() case ident: SqlIdentifier => UnresolvedAttribute(ident.toString) - case unsupported => sys.error(s"no rule for sqlExpr $unsupported, class: ${unsupported.getClass.getSimpleName}") + case unsupported => + sys.error(s"no rule for sqlExpr $unsupported, class: ${unsupported.getClass.getSimpleName}") } def sqlNodeToExpr(node: SqlNode): Expression = node match { + case Call(EQUALS, Seq(left,right)) => Equals(sqlNodeToExpr(left), sqlNodeToExpr(right)) case named => sqlNodeToNamedExpr(named) } } \ No newline at end of file diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index 26707edf4c2f7..8f43c75613173 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -8,6 +8,10 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend def output = projectList.map(_.toAttribute) } +case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { + def output = child.output +} + case class Join( left: LogicalPlan, right: LogicalPlan, diff --git a/src/main/scala/types/dataTypes.scala b/src/main/scala/types/dataTypes.scala index b07dd7b213add..8a55ff98bd0ba 100644 --- a/src/main/scala/types/dataTypes.scala +++ b/src/main/scala/types/dataTypes.scala @@ -4,4 +4,5 @@ package types sealed class DataType case object IntegerType extends DataType -case object StringType extends DataType \ No newline at end of file +case object StringType extends DataType +case object BooleanType extends DataType \ No newline at end of file diff --git a/src/test/scala/OptiqParsingSuite.scala b/src/test/scala/OptiqParsingSuite.scala index fee901b489457..971e341c51c47 100644 --- a/src/test/scala/OptiqParsingSuite.scala +++ b/src/test/scala/OptiqParsingSuite.scala @@ -6,6 +6,7 @@ import org.scalatest.{FunSuite} class OptiqParsingSuite extends FunSuite { test("simple select") { println(Optiq.parseSql("SELECT a FROM foo")) + println(Optiq.parseSql("SELECT a FROM foo WHERE a = b")) println(Optiq.parseSql("SELECT a AS b FROM foo")) println(Optiq.parseSql("SELECT a AS b FROM foo JOIN bar")) } From 069df02cbf9f2ebb170a9bff5a0a20d1567041d5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 8 Nov 2013 23:27:03 -0800 Subject: [PATCH 027/778] parsing binary predicates --- src/main/scala/frontend/Optiq.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/scala/frontend/Optiq.scala b/src/main/scala/frontend/Optiq.scala index 7278fd8769c24..473d56bc32cf3 100644 --- a/src/main/scala/frontend/Optiq.scala +++ b/src/main/scala/frontend/Optiq.scala @@ -78,6 +78,10 @@ object Optiq { def sqlNodeToExpr(node: SqlNode): Expression = node match { case Call(EQUALS, Seq(left,right)) => Equals(sqlNodeToExpr(left), sqlNodeToExpr(right)) + case Call(GREATER_THAN, Seq(left,right)) => GreaterThan(sqlNodeToExpr(left), sqlNodeToExpr(right)) + case Call(GREATER_THAN_OR_EQUAL, Seq(left,right)) => GreaterThanOrEqual(sqlNodeToExpr(left), sqlNodeToExpr(right)) + case Call(LESS_THAN, Seq(left,right)) => LessThan(sqlNodeToExpr(left), sqlNodeToExpr(right)) + case Call(LESS_THAN_OR_EQUAL, Seq(left,right)) => LessThanOrEqual(sqlNodeToExpr(left), sqlNodeToExpr(right)) case named => sqlNodeToNamedExpr(named) } } \ No newline at end of file From ec1d7c09c1a2a8217f7ca88241a0438788fd44d2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 9 Nov 2013 00:38:37 -0800 Subject: [PATCH 028/778] Simple attribute resolution. --- src/main/scala/analysis/Analyzer.scala | 15 +++++++++- src/main/scala/analysis/Catalog.scala | 4 +++ src/main/scala/expressions/predicates.scala | 23 +++++++++++++++ src/main/scala/plans/QueryPlan.scala | 28 ++++++++++++++++++- .../scala/plans/logical/LogicalPlan.scala | 12 ++++++++ .../scala/plans/logical/TestRelation.scala | 7 +++++ src/test/scala/AnalysisSuite.scala | 20 +++++++++++++ 7 files changed, 107 insertions(+), 2 deletions(-) create mode 100644 src/main/scala/expressions/predicates.scala create mode 100644 src/main/scala/plans/logical/TestRelation.scala create mode 100644 src/test/scala/AnalysisSuite.scala diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala index d44ada3f37e34..a4e44b65fc2ff 100644 --- a/src/main/scala/analysis/Analyzer.scala +++ b/src/main/scala/analysis/Analyzer.scala @@ -7,7 +7,8 @@ import rules._ class Analyzer(catalog: Catalog) extends RuleExecutor[LogicalPlan] { val batches = Seq( Batch("Resolution", - ResolveRelations) + ResolveRelations, + ResolveReferences) ) object ResolveRelations extends Rule[LogicalPlan] { @@ -15,4 +16,16 @@ class Analyzer(catalog: Catalog) extends RuleExecutor[LogicalPlan] { case UnresolvedRelation(name, alias) => catalog.lookupRelation(name, alias) } } + + object ResolveReferences extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => + // logger.fine(s"resolving ${plan.simpleString}") + q transformExpressions { + case u @ UnresolvedAttribute(name) => + // Leave unchanged if resolution fails. Hopefully will be resolved next round. + q.resolve(name).getOrElse(u) + } + } + } } \ No newline at end of file diff --git a/src/main/scala/analysis/Catalog.scala b/src/main/scala/analysis/Catalog.scala index 57f5e5fd3d62f..43dead9c58330 100644 --- a/src/main/scala/analysis/Catalog.scala +++ b/src/main/scala/analysis/Catalog.scala @@ -5,4 +5,8 @@ import plans.logical.LogicalPlan abstract class Catalog { def lookupRelation(name: String, alias: Option[String] = None): LogicalPlan +} + +object EmptyCatalog extends Catalog { + def lookupRelation(name: String, alias: Option[String] = None) = ??? } \ No newline at end of file diff --git a/src/main/scala/expressions/predicates.scala b/src/main/scala/expressions/predicates.scala new file mode 100644 index 0000000000000..21e9426bc5d2f --- /dev/null +++ b/src/main/scala/expressions/predicates.scala @@ -0,0 +1,23 @@ +package catalyst +package expressions + +import types._ + +abstract class Predicate extends Expression { + self: Product => + + def dataType = BooleanType +} + +abstract class BinaryPredicate(op: String) extends Predicate with trees.BinaryNode[Expression] { + self: Product => + + def nullable = left.nullable || right.nullable + override def toString(): String = s"$left $op $right" +} + +case class Equals(left: Expression, right: Expression) extends BinaryPredicate("=") +case class LessThan(left: Expression, right: Expression) extends BinaryPredicate("<") +case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryPredicate("<=") +case class GreaterThan(left: Expression, right: Expression) extends BinaryPredicate(">") +case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryPredicate(">=") diff --git a/src/main/scala/plans/QueryPlan.scala b/src/main/scala/plans/QueryPlan.scala index 8434a4cea61f1..b4ee043c1728a 100644 --- a/src/main/scala/plans/QueryPlan.scala +++ b/src/main/scala/plans/QueryPlan.scala @@ -1,11 +1,37 @@ package catalyst package plans -import expressions.Attribute +import expressions.{Attribute, Expression} import trees._ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => def output: Seq[Attribute] + + def transformExpressions(rule: PartialFunction[Expression, Expression]): this.type = { + var changed = false + + + @inline def transformExpression(e: Expression) = { + val newE = e.transform(rule) + println("te") + if(newE.id != e.id && newE != e) { + changed = true + newE + } else { + e + } + } + + val newArgs = productIterator.map { + case e: Expression => transformExpression(e) + case Some(e: Expression) => Some(transformExpression(e)) + case seqE: Seq[Expression] if !seqE.isEmpty && classOf[Expression].isAssignableFrom(seqE.head.getClass) => + seqE.map(transformExpression) + case other: AnyRef => other + }.toArray + + if(changed) makeCopy(newArgs) else this + } } \ No newline at end of file diff --git a/src/main/scala/plans/logical/LogicalPlan.scala b/src/main/scala/plans/logical/LogicalPlan.scala index 72464ee46b984..e281b7deba32e 100644 --- a/src/main/scala/plans/logical/LogicalPlan.scala +++ b/src/main/scala/plans/logical/LogicalPlan.scala @@ -3,6 +3,7 @@ package plans package logical import expressions.Attribute +import errors._ import trees._ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { @@ -24,6 +25,17 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { * input from its children. */ def inputSet: Set[Attribute] = ??? + + def resolve(name: String): Option[Attribute] = { + val options = children.flatMap(_.output).filter(_.name == name) + options match { + case a :: Nil => Some(a) // One match, use it. + case Nil => None // No matches. + case ambiguousReferences => + throw new OptimizationException( + this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") + } + } } /** diff --git a/src/main/scala/plans/logical/TestRelation.scala b/src/main/scala/plans/logical/TestRelation.scala new file mode 100644 index 0000000000000..c5493ff4ac065 --- /dev/null +++ b/src/main/scala/plans/logical/TestRelation.scala @@ -0,0 +1,7 @@ +package catalyst +package plans +package logical + +import expressions._ + +case class TestRelation(output: Seq[Attribute]) extends LeafNode \ No newline at end of file diff --git a/src/test/scala/AnalysisSuite.scala b/src/test/scala/AnalysisSuite.scala new file mode 100644 index 0000000000000..07a0489232e33 --- /dev/null +++ b/src/test/scala/AnalysisSuite.scala @@ -0,0 +1,20 @@ +package catalyst +package frontend + +import org.scalatest.FunSuite + +import analysis._ +import expressions._ +import plans.logical._ +import types._ + +class AnalysisSuite extends FunSuite { + val analyze = new Analyzer(EmptyCatalog) + + val testRelation = new TestRelation(AttributeReference("a", IntegerType)() :: Nil) + + test("analyze project") { + assert(analyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === Project(testRelation.output, testRelation)) + + } +} \ No newline at end of file From a703c4960a1954c53d827c5ecb6956ec0b917b28 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 9 Nov 2013 01:06:49 -0800 Subject: [PATCH 029/778] this order works better until fixed point is implemented. --- src/main/scala/analysis/Analyzer.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala index a4e44b65fc2ff..9a825993d9f2f 100644 --- a/src/main/scala/analysis/Analyzer.scala +++ b/src/main/scala/analysis/Analyzer.scala @@ -7,8 +7,8 @@ import rules._ class Analyzer(catalog: Catalog) extends RuleExecutor[LogicalPlan] { val batches = Seq( Batch("Resolution", - ResolveRelations, - ResolveReferences) + ResolveReferences, + ResolveRelations) ) object ResolveRelations extends Rule[LogicalPlan] { From 6f9bafd501b28cbe56eb8e70cbe9a8f4bf9a8b85 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 9 Nov 2013 01:07:11 -0800 Subject: [PATCH 030/778] empty output for unresolved relation to avoid exception in resolution. --- src/main/scala/analysis/unresolved.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala index 3000dcc10e019..2eb8d80d89e44 100644 --- a/src/main/scala/analysis/unresolved.scala +++ b/src/main/scala/analysis/unresolved.scala @@ -9,7 +9,7 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str errors.OptimizationException(tree, "Invalid call to $function on unresolved object") case class UnresolvedRelation(name: String, alias: Option[String]) extends LeafNode { - def output = throw new UnresolvedException(this, "output") + def output = Nil } case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { From 4b765d58f2e06725acd6d0d40b1cb9318f6cc2b6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 9 Nov 2013 01:07:22 -0800 Subject: [PATCH 031/778] docs, drop println --- src/main/scala/plans/QueryPlan.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/main/scala/plans/QueryPlan.scala b/src/main/scala/plans/QueryPlan.scala index b4ee043c1728a..90df4c33b6229 100644 --- a/src/main/scala/plans/QueryPlan.scala +++ b/src/main/scala/plans/QueryPlan.scala @@ -9,13 +9,16 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy def output: Seq[Attribute] + /** + * Runs [[transform]] with [[rule]] on all expressions present in this query operator. + * @param rule the rule to be applied to every expression in this operator. + * @return + */ def transformExpressions(rule: PartialFunction[Expression, Expression]): this.type = { var changed = false - @inline def transformExpression(e: Expression) = { val newE = e.transform(rule) - println("te") if(newE.id != e.id && newE != e) { changed = true newE From ef21a0bc8e11a2085b7538955b39f663e6c0cd90 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 9 Nov 2013 01:07:33 -0800 Subject: [PATCH 032/778] line up comments. --- src/main/scala/plans/logical/LogicalPlan.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/plans/logical/LogicalPlan.scala b/src/main/scala/plans/logical/LogicalPlan.scala index e281b7deba32e..66ecc6eacf22a 100644 --- a/src/main/scala/plans/logical/LogicalPlan.scala +++ b/src/main/scala/plans/logical/LogicalPlan.scala @@ -30,7 +30,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { val options = children.flatMap(_.output).filter(_.name == name) options match { case a :: Nil => Some(a) // One match, use it. - case Nil => None // No matches. + case Nil => None // No matches. case ambiguousReferences => throw new OptimizationException( this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") From 910811c700472bf56e2dd975d492269aefeb2605 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 9 Nov 2013 10:42:33 -0800 Subject: [PATCH 033/778] check each item of the sequence --- src/main/scala/plans/QueryPlan.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/scala/plans/QueryPlan.scala b/src/main/scala/plans/QueryPlan.scala index 90df4c33b6229..50c135ec6b948 100644 --- a/src/main/scala/plans/QueryPlan.scala +++ b/src/main/scala/plans/QueryPlan.scala @@ -30,8 +30,10 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy val newArgs = productIterator.map { case e: Expression => transformExpression(e) case Some(e: Expression) => Some(transformExpression(e)) - case seqE: Seq[Expression] if !seqE.isEmpty && classOf[Expression].isAssignableFrom(seqE.head.getClass) => - seqE.map(transformExpression) + case seq: Seq[_] => seq.map { + case e: Expression => transformExpression(e) + case other => other + } case other: AnyRef => other }.toArray From c7cd57f28cf3acb9a268d1630a3f1de8077bdd37 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 11 Nov 2013 16:42:59 -0800 Subject: [PATCH 034/778] docs for util function. --- src/main/scala/util/package.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/scala/util/package.scala b/src/main/scala/util/package.scala index 31a36e29a41cd..61e46f1b74a62 100644 --- a/src/main/scala/util/package.scala +++ b/src/main/scala/util/package.scala @@ -3,6 +3,12 @@ package catalyst import java.io.File package object util { + /** + * Returns a path to a temporary file that probably does not exist. + * Note, there is always the race condition that someone created this + * file since the last time we checked. Thus, this shouldn't be used + * for anything security conscious. + */ def getTempFilePath(prefix: String, suffix: String = ""): File = { val tempFile = File.createTempFile(prefix, suffix) tempFile.delete() From 7bff274ee7b1695efcf9a4e7ab08dabc467a7efa Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 12 Nov 2013 19:15:06 -0800 Subject: [PATCH 035/778] point at new shark. --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index c70aef6445d42..4b7341a5871ae 100644 --- a/build.sbt +++ b/build.sbt @@ -1,6 +1,6 @@ resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" -libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.8.0" +libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.9.0-SNAPSHOT" libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client" % "1.0.4", From 5c7d24437090708217990a3acbb74bac22819083 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 12 Nov 2013 19:15:44 -0800 Subject: [PATCH 036/778] first draft of references implementation. --- src/main/scala/expressions/Expression.scala | 2 ++ src/main/scala/expressions/literals.scala | 3 ++- src/main/scala/expressions/namedExpressions.scala | 2 ++ src/main/scala/expressions/predicates.scala | 1 + src/main/scala/plans/logical/LogicalPlan.scala | 5 ++++- src/main/scala/plans/logical/basicOperators.scala | 3 +++ 6 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/main/scala/expressions/Expression.scala b/src/main/scala/expressions/Expression.scala index 8c2a2a2fd2165..afe2349b06663 100644 --- a/src/main/scala/expressions/Expression.scala +++ b/src/main/scala/expressions/Expression.scala @@ -9,10 +9,12 @@ abstract class Expression extends TreeNode[Expression] { def dataType: DataType def nullable: Boolean + def references: Set[Attribute] } abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] { self: Product => + def references = left.references ++ right.references } abstract class LeafExpression extends Expression with trees.LeafNode[Expression] { diff --git a/src/main/scala/expressions/literals.scala b/src/main/scala/expressions/literals.scala index 24cee7e95bb88..e60f49d363b86 100644 --- a/src/main/scala/expressions/literals.scala +++ b/src/main/scala/expressions/literals.scala @@ -11,5 +11,6 @@ object Literal { } case class Literal(value: Any, dataType: DataType) extends LeafExpression { - final def nullable = false + def nullable = false + def references = Set.empty } \ No newline at end of file diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index d9fab10c764fd..99fc6540ec9bf 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -26,6 +26,7 @@ abstract class NamedExpression extends Expression { abstract class Attribute extends NamedExpression { self: Product => + def references = Set(this) def toAttribute = this } @@ -44,6 +45,7 @@ case class Alias(child: Expression, name: String) def dataType = child.dataType def nullable = child.nullable + def references = child.references def toAttribute = AttributeReference(name, child.dataType, child.nullable)(exprId) diff --git a/src/main/scala/expressions/predicates.scala b/src/main/scala/expressions/predicates.scala index 21e9426bc5d2f..afba100347b50 100644 --- a/src/main/scala/expressions/predicates.scala +++ b/src/main/scala/expressions/predicates.scala @@ -13,6 +13,7 @@ abstract class BinaryPredicate(op: String) extends Predicate with trees.BinaryNo self: Product => def nullable = left.nullable || right.nullable + def references = left.references ++ right.references override def toString(): String = s"$left $op $right" } diff --git a/src/main/scala/plans/logical/LogicalPlan.scala b/src/main/scala/plans/logical/LogicalPlan.scala index 66ecc6eacf22a..d49fd61e31de4 100644 --- a/src/main/scala/plans/logical/LogicalPlan.scala +++ b/src/main/scala/plans/logical/LogicalPlan.scala @@ -13,7 +13,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { * Returns the set of attributes that are referenced by this node * during evaluation. */ - def references: Set[Attribute] = ??? + def references: Set[Attribute] /** * Returns the set of attributes that are output by this node. @@ -43,6 +43,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { */ abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { self: Product => + + // Leaf nodes by definition cannot reference any input attributes. + def references = Set.empty } /** diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index 8f43c75613173..4f3b26f0f4f60 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -6,10 +6,12 @@ import expressions._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) + def references = projectList.flatMap(_.references).toSet } case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { def output = child.output + def references = condition.references } case class Join( @@ -18,5 +20,6 @@ case class Join( joinType: JoinType, condition: Option[Expression]) extends BinaryNode { + def references = condition.map(_.references).getOrElse(Set.empty) def output = left.output ++ right.output } \ No newline at end of file From f72b7ce3a24a9716ca61e21cc98e16da4bbc6cc0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 12 Nov 2013 19:16:22 -0800 Subject: [PATCH 037/778] first trivial end to end query execution. --- src/main/scala/planning/QueryPlanner.scala | 18 +++++++ src/main/scala/planning/Strategy.scala | 33 ++++++++++++ .../scala/plans/physical/PhysicalPlan.scala | 28 +++++++++++ .../scala/plans/physical/basicOperators.scala | 50 +++++++++++++++++++ src/main/scala/util/TestShark.scala | 19 ++++--- src/test/scala/HiveParsingSuite.scala | 11 ++-- 6 files changed, 147 insertions(+), 12 deletions(-) create mode 100644 src/main/scala/planning/QueryPlanner.scala create mode 100644 src/main/scala/planning/Strategy.scala create mode 100644 src/main/scala/plans/physical/PhysicalPlan.scala create mode 100644 src/main/scala/plans/physical/basicOperators.scala diff --git a/src/main/scala/planning/QueryPlanner.scala b/src/main/scala/planning/QueryPlanner.scala new file mode 100644 index 0000000000000..14fbff435a265 --- /dev/null +++ b/src/main/scala/planning/QueryPlanner.scala @@ -0,0 +1,18 @@ +package catalyst +package planning + +import catalyst.plans.logical.LogicalPlan +import catalyst.plans.physical.PhysicalPlan + +abstract class QueryPlanner { + def strategies: Seq[Strategy] + + def apply(plan: LogicalPlan): Iterator[PhysicalPlan] = { + // Obviously a lot to do here still... + strategies.head(plan).toIterator + } +} + +object TrivalPlanner extends QueryPlanner { + val strategies = HiveTableScans :: Nil +} \ No newline at end of file diff --git a/src/main/scala/planning/Strategy.scala b/src/main/scala/planning/Strategy.scala new file mode 100644 index 0000000000000..e4642b5718ee7 --- /dev/null +++ b/src/main/scala/planning/Strategy.scala @@ -0,0 +1,33 @@ +package catalyst +package planning + +import catalyst.analysis.MetastoreRelation +import expressions._ +import plans.logical._ +import plans.physical +import plans.physical.PhysicalPlan + +abstract class Strategy { + def apply(plan: LogicalPlan): Seq[PhysicalPlan] +} + +object HiveTableScans extends Strategy { + def apply(plan: LogicalPlan): Seq[PhysicalPlan] = plan match { + case p @ Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => + physical.HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m) :: Nil + case m: MetastoreRelation => + physical.HiveTableScan(m.output, m) :: Nil + case _ => Nil + } + + /** + * Returns true if [[projectList]] only performs column pruning and + * does not evaluate other complex expressions. + */ + def isSimpleProject(projectList: Seq[NamedExpression]) = { + projectList.map { + case a: Attribute => true + case _ => false + }.reduceLeft(_ && _) + } +} \ No newline at end of file diff --git a/src/main/scala/plans/physical/PhysicalPlan.scala b/src/main/scala/plans/physical/PhysicalPlan.scala new file mode 100644 index 0000000000000..e707e45be6958 --- /dev/null +++ b/src/main/scala/plans/physical/PhysicalPlan.scala @@ -0,0 +1,28 @@ +package catalyst +package plans +package physical + +import org.apache.spark.rdd.RDD + +abstract class PhysicalPlan extends QueryPlan[PhysicalPlan] { + self: Product => + + /** + * Runs this query returning the result as an RDD. + * This fact that this returns an RDD should probably be + * abstracted away from the rest of the planning code. + */ + def execute(): RDD[_] +} + +abstract trait LeafNode extends trees.LeafNode[PhysicalPlan] { + self: Product => +} + +abstract trait UnaryNode extends trees.UnaryNode[PhysicalPlan] { + self: Product => +} + +abstract trait BinaryNode extends trees.BinaryNode[PhysicalPlan] { + self: Product => +} \ No newline at end of file diff --git a/src/main/scala/plans/physical/basicOperators.scala b/src/main/scala/plans/physical/basicOperators.scala new file mode 100644 index 0000000000000..704cc00211e2f --- /dev/null +++ b/src/main/scala/plans/physical/basicOperators.scala @@ -0,0 +1,50 @@ +package catalyst +package plans +package physical + +import expressions._ +import plans.logical +import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, StructObjectInspector} +import shark.execution.HadoopTableReader +import shark.{SharkContext, SharkEnv} + +import collection.JavaConversions._ +import org.apache.hadoop.hive.serde2.`lazy`.LazyPrimitive + +case class HiveTableScan(attributes: Seq[Attribute], relation: analysis.MetastoreRelation) extends PhysicalPlan with LeafNode { + val hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(relation.table) + val tableDesc = new TableDesc( + Class.forName(relation.table.getSd.getSerdeInfo.getSerializationLib).asInstanceOf[Class[org.apache.hadoop.hive.serde2.Deserializer]], + Class.forName(relation.table.getSd.getInputFormat).asInstanceOf[Class[org.apache.hadoop.mapred.InputFormat[_,_]]], + Class.forName(relation.table.getSd.getOutputFormat), + hiveQlTable.getSchema + ) + + @transient + val hadoopReader = new HadoopTableReader(tableDesc, SharkContext.hiveconf) + + @transient + lazy val objectInspector = + tableDesc.getDeserializer.getObjectInspector.asInstanceOf[StructObjectInspector] + + @transient + lazy val refs = attributes.map { a => + objectInspector.getAllStructFieldRefs + .find(_.getFieldName == a.name) + .getOrElse(sys.error(s"Invalid attribute ${a.name} referenced in table $relation")) + } + + def execute() = { + hadoopReader.makeRDDForTable(hiveQlTable).map { + case struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct => + refs.map { ref => + val data = objectInspector.getStructFieldData(struct, ref) + ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector].getPrimitiveJavaObject(data) + }.toSeq + } + } + + def output = attributes +} + diff --git a/src/main/scala/util/TestShark.scala b/src/main/scala/util/TestShark.scala index 26140595d33be..0087abc926ebe 100644 --- a/src/main/scala/util/TestShark.scala +++ b/src/main/scala/util/TestShark.scala @@ -2,7 +2,8 @@ package catalyst package util import catalyst.analysis.{Analyzer, HiveMetastoreCatalog} -import catalyst.frontend.Hive +import catalyst.frontend._ +import catalyst.planning.TrivalPlanner import shark.{SharkContext, SharkEnv} import util._ @@ -29,14 +30,18 @@ class TestShark { class SharkQuery(sql: String) { lazy val parsed = Hive.parseSql(sql) lazy val analyzed = analyze(parsed) + // TODO: Don't just pick the first one... + lazy val physicalPlan = TrivalPlanner(analyzed).next() + + def execute() = physicalPlan.execute() override def toString(): String = - s""" - |$sql - | - |== Logical Plan == - |$analyzed - """.stripMargin + s"""$sql + |== Logical Plan == + |$analyzed + |== Physical Plan == + |$physicalPlan + """.stripMargin.trim } implicit class stringToQuery(str: String) { diff --git a/src/test/scala/HiveParsingSuite.scala b/src/test/scala/HiveParsingSuite.scala index 3a2da9415ed96..3bc154a739411 100644 --- a/src/test/scala/HiveParsingSuite.scala +++ b/src/test/scala/HiveParsingSuite.scala @@ -8,15 +8,16 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite} import util.TestShark class SQLSuite extends FunSuite with BeforeAndAfterAll { - var sc: SharkContext = _ override def beforeAll() { - val testShark = new TestShark - testShark.loadKv1 - sc = testShark.sc } test("trivial select query") { - sc.sql("SELECT key, val FROM test").foreach(println) + val x = new TestShark + import x._ + loadKv1 + //sc.sql("SELECT key, val FROM test").foreach(println) + + "SELECT key FROM test".q.execute().collect.foreach(println) } } \ No newline at end of file From 64d2923b80be03b6a403b88109432569806d84c0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 19:16:48 -0800 Subject: [PATCH 038/778] Add classes for representing sorts. --- src/main/scala/expressions/SortOrder.scala | 18 ++++++++++++++++++ .../scala/plans/logical/basicOperators.scala | 5 +++++ 2 files changed, 23 insertions(+) create mode 100644 src/main/scala/expressions/SortOrder.scala diff --git a/src/main/scala/expressions/SortOrder.scala b/src/main/scala/expressions/SortOrder.scala new file mode 100644 index 0000000000000..d895d8e1d7d28 --- /dev/null +++ b/src/main/scala/expressions/SortOrder.scala @@ -0,0 +1,18 @@ +package catalyst +package expressions + +import types._ + +abstract sealed class SortDirection +case object Ascending extends SortDirection +case object Descending extends SortDirection + +/** + * An expression that can be used to sort a tuple. This class extends expression primarily so that + * transformations over expression will descend into its child. + */ +case class SortOrder(child: Expression, direction: SortDirection) extends UnaryExpression { + // This is a little sloppy... + def dataType = IntegerType + def nullable = child.nullable +} \ No newline at end of file diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index 4f3b26f0f4f60..468fc5b69fcfc 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -22,4 +22,9 @@ case class Join( def references = condition.map(_.references).getOrElse(Set.empty) def output = left.output ++ right.output +} + +case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { + def output = child.output + def references = child.references } \ No newline at end of file From 665f7d0a9ba94a6cb522dd2ef888a5efea1aa7fc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 19:17:13 -0800 Subject: [PATCH 039/778] add logical nodes for hive data sinks. --- src/main/scala/plans/logical/hiveOperations.scala | 8 ++++++++ 1 file changed, 8 insertions(+) create mode 100644 src/main/scala/plans/logical/hiveOperations.scala diff --git a/src/main/scala/plans/logical/hiveOperations.scala b/src/main/scala/plans/logical/hiveOperations.scala new file mode 100644 index 0000000000000..84c5259a54481 --- /dev/null +++ b/src/main/scala/plans/logical/hiveOperations.scala @@ -0,0 +1,8 @@ +package catalyst +package plans +package logical + +case class InsertIntoHiveTable(hiveTable: String, child: LogicalPlan) extends UnaryNode { + def references = Set.empty + def output = Seq.empty +} \ No newline at end of file From 3aa1b281fdcefe0f39d3d5d63801a077e02fec83 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 19:17:48 -0800 Subject: [PATCH 040/778] support for table names in the form 'database.tableName' --- src/main/scala/analysis/MetastoreCatalog.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/main/scala/analysis/MetastoreCatalog.scala b/src/main/scala/analysis/MetastoreCatalog.scala index 53da5b664704b..3f71a6148701b 100644 --- a/src/main/scala/analysis/MetastoreCatalog.scala +++ b/src/main/scala/analysis/MetastoreCatalog.scala @@ -14,8 +14,13 @@ import collection.JavaConversions._ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { protected val client = new HiveMetaStoreClient(hiveConf) - def lookupRelation(name: String, alias: Option[String]): LogicalPlan = - MetastoreRelation(name)(client.getTable("default", name)) + def lookupRelation(name: String, alias: Option[String]): LogicalPlan = { + val (databaseName, tableName) = name.split("\\.") match { + case Array(tableOnly) => ("default", tableOnly) + case Array(db, table) => (db, table) + } + MetastoreRelation(name)(client.getTable(databaseName, tableName)) + } } object HiveMetatoreTypes { From 59ac4443cfdfb8d07eb81319ffd302ac2f63edee Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 19:18:01 -0800 Subject: [PATCH 041/778] add unary expression --- src/main/scala/expressions/Expression.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/main/scala/expressions/Expression.scala b/src/main/scala/expressions/Expression.scala index afe2349b06663..5d3358118b25c 100644 --- a/src/main/scala/expressions/Expression.scala +++ b/src/main/scala/expressions/Expression.scala @@ -20,3 +20,8 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express abstract class LeafExpression extends Expression with trees.LeafNode[Expression] { self: Product => } + +abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { + self: Product => + def references = child.references +} From 633cebc7eee40a384698dbc346bc6ca658472a88 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 19:18:29 -0800 Subject: [PATCH 042/778] better error message when there is no appropriate planning strategy. --- src/main/scala/planning/QueryPlanner.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/scala/planning/QueryPlanner.scala b/src/main/scala/planning/QueryPlanner.scala index 14fbff435a265..415aad9f78d72 100644 --- a/src/main/scala/planning/QueryPlanner.scala +++ b/src/main/scala/planning/QueryPlanner.scala @@ -9,7 +9,9 @@ abstract class QueryPlanner { def apply(plan: LogicalPlan): Iterator[PhysicalPlan] = { // Obviously a lot to do here still... - strategies.head(plan).toIterator + val iter = strategies.head(plan).toIterator + assert(iter.hasNext, s"No plan for $plan") + iter } } From 29a7163e6919bb6a04f7bbfe920d8f7ed5522d6c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 19:18:55 -0800 Subject: [PATCH 043/778] Insert into hive table physical operator. --- src/main/scala/plans/physical/basicOperators.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/scala/plans/physical/basicOperators.scala b/src/main/scala/plans/physical/basicOperators.scala index 704cc00211e2f..f30152ea376aa 100644 --- a/src/main/scala/plans/physical/basicOperators.scala +++ b/src/main/scala/plans/physical/basicOperators.scala @@ -48,3 +48,7 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: analysis.Metastor def output = attributes } +case class InsertIntoHiveTable(tableName: String, child: PhysicalPlan) extends UnaryNode { + def output = Seq.empty +} + From d8a924f585c47dccd2ff3642c52c3f06372ffaa7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 19:19:11 -0800 Subject: [PATCH 044/778] scaladoc --- src/main/scala/plans/physical/basicOperators.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/main/scala/plans/physical/basicOperators.scala b/src/main/scala/plans/physical/basicOperators.scala index f30152ea376aa..7d7a9ef12495f 100644 --- a/src/main/scala/plans/physical/basicOperators.scala +++ b/src/main/scala/plans/physical/basicOperators.scala @@ -24,10 +24,17 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: analysis.Metastor @transient val hadoopReader = new HadoopTableReader(tableDesc, SharkContext.hiveconf) + /** + * The hive object inspector for this table, which can be used to extract values from the + * serialized row representation. + */ @transient lazy val objectInspector = tableDesc.getDeserializer.getObjectInspector.asInstanceOf[StructObjectInspector] + /** + * The hive struct field references that correspond to the attributes to be read from this table. + */ @transient lazy val refs = attributes.map { a => objectInspector.getAllStructFieldRefs From 0763490930dbacfa277813006a1d3da74404a162 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 19:19:30 -0800 Subject: [PATCH 045/778] support for hive native command pass-through. --- src/main/scala/frontend/Hive.scala | 191 ++++++++++++++++++++++++++-- src/main/scala/util/TestShark.scala | 5 +- 2 files changed, 181 insertions(+), 15 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 07345734580bd..891e0ec0337e9 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -11,13 +11,62 @@ import plans.logical._ import collection.JavaConversions._ +/** + * A logical node that represent a non-query command to be executed by the system. For example, + * commands can be used by parsers to represent DDL operations. + */ +abstract class Command extends LeafNode { + self: Product => + def output = Seq.empty +} + +/** + * Returned for commands supported by the parser, but not catalyst. In general these are DDL + * commands that are passed directly to Hive. + */ +case class NativeCommand(cmd: String) extends Command + +case class DfsCommand(cmd: String) extends Command + +case class ShellCommand(cmd: String) extends Command + +case class SourceCommand(filePath: String) extends Command + +case class ConfigurationAssignment(cmd: String) extends Command + +case class AddJar(jarPath: String) extends Command + +case class AddFile(filePath: String) extends Command + object Hive { def parseSql(sql: String): LogicalPlan = { - val tree = - ParseUtils.findRootNonNullToken( - (new ParseDriver()).parse(sql, null /* no context required for parsing alone */)) + if(sql.toLowerCase.startsWith("set")) + ConfigurationAssignment(sql) + else if(sql.toLowerCase.startsWith("add jar")) + AddJar(sql.drop(8)) + else if(sql.toLowerCase.startsWith("add file")) + AddFile(sql.drop(9)) + else if(sql.startsWith("dfs")) + DfsCommand(sql) + else if(sql.startsWith("source")) + SourceCommand(sql.split(" ").toSeq match { case Seq("source", filePath) => filePath }) + else if(sql.startsWith("!")) + ShellCommand(sql.drop(1)) + else { + val tree = + try { + ParseUtils.findRootNonNullToken( + (new ParseDriver()).parse(sql, null /* no context required for parsing alone */)) + } catch { + case pe: org.apache.hadoop.hive.ql.parse.ParseException => + sys.error(s"Failed to parse '$sql'") + } - nodeToPlan(tree) + nodeToPlan(tree) match { + case NativePlaceholder(_) => NativeCommand(sql) + case other => other + } + } } /** Extractor for matching Hive's AST Tokens */ @@ -29,25 +78,135 @@ object Hive { } } + // Kinda gross hack... + case class NativePlaceholder(ast: Node) extends Command + protected def nodeToPlan(node: Node): LogicalPlan = node match { + case a @ Token("TOK_EXPLAIN", _) => NativePlaceholder(a) + + case a @ Token("TOK_DESCFUNCTION", _) => NativePlaceholder(a) + case a @ Token("TOK_DESCTABLE", _) => NativePlaceholder(a) + case a @ Token("TOK_DESCDATABASE", _) => NativePlaceholder(a) + case a @ Token("TOK_SHOW_TABLESTATUS", _) => NativePlaceholder(a) + case a @ Token("TOK_SHOWDATABASES", _) => NativePlaceholder(a) + case a @ Token("TOK_SHOWFUNCTIONS", _) => NativePlaceholder(a) + case a @ Token("TOK_SHOWINDEXES", _) => NativePlaceholder(a) + case a @ Token("TOK_SHOWINDEXES", _) => NativePlaceholder(a) + case a @ Token("TOK_SHOWPARTITIONS", _) => NativePlaceholder(a) + case a @ Token("TOK_SHOWTABLES", _) => NativePlaceholder(a) + + case a @ Token("TOK_LOCKTABLE", _) => NativePlaceholder(a) + case a @ Token("TOK_SHOWLOCKS", _) => NativePlaceholder(a) + case a @ Token("TOK_UNLOCKTABLE", _) => NativePlaceholder(a) + + case a @ Token("TOK_CREATEROLE", _) => NativePlaceholder(a) + case a @ Token("TOK_DROPROLE", _) => NativePlaceholder(a) + case a @ Token("TOK_GRANT", _) => NativePlaceholder(a) + case a @ Token("TOK_GRANT_ROLE", _) => NativePlaceholder(a) + case a @ Token("TOK_REVOKE", _) => NativePlaceholder(a) + case a @ Token("TOK_SHOW_GRANT", _) => NativePlaceholder(a) + case a @ Token("TOK_SHOW_ROLE_GRANT", _) => NativePlaceholder(a) + + case a @ Token("TOK_CREATEFUNCTION", _) => NativePlaceholder(a) + case a @ Token("TOK_DROPFUNCTION", _) => NativePlaceholder(a) + + case a @ Token("TOK_ALTERDATABASE_PROPERTIES", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERINDEX_PROPERTIES", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERINDEX_REBUILD", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_ADDCOLS", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_ADDPARTS", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_ARCHIVE", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_DROPPARTS", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_PARTITION", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_PROPERTIES", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_RENAME", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_RENAMECOL", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_REPLACECOLS", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_TOUCH", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERTABLE_UNARCHIVE", _) => NativePlaceholder(a) + case a @ Token("TOK_ANALYZE", _) => NativePlaceholder(a) + case a @ Token("TOK_CREATEDATABASE", _) => NativePlaceholder(a) + case a @ Token("TOK_CREATEINDEX", _) => NativePlaceholder(a) + case a @ Token("TOK_CREATETABLE", _) => NativePlaceholder(a) + case a @ Token("TOK_DROPDATABASE", _) => NativePlaceholder(a) + case a @ Token("TOK_DROPINDEX", _) => NativePlaceholder(a) + case a @ Token("TOK_DROPTABLE", _) => NativePlaceholder(a) + case a @ Token("TOK_MSCK", _) => NativePlaceholder(a) + + // TODO(marmbrus): Figure out how view are expanded by hive, as we might need to handle this. + case a @ Token("TOK_ALTERVIEW_ADDPARTS", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERVIEW_DROPPARTS", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERVIEW_PROPERTIES", _) => NativePlaceholder(a) + case a @ Token("TOK_ALTERVIEW_RENAME", _) => NativePlaceholder(a) + case a @ Token("TOK_CREATEVIEW", _) => NativePlaceholder(a) + case a @ Token("TOK_DROPVIEW", _) => NativePlaceholder(a) + + + case a @ Token("TOK_EXPORT", _) => NativePlaceholder(a) + case a @ Token("TOK_IMPORT", _) => NativePlaceholder(a) + case a @ Token("TOK_LOAD", _) => NativePlaceholder(a) + + case a @ Token("TOK_SWITCHDATABASE", _) => NativePlaceholder(a) + case a @ Token("TOK_EXPLAIN", _) => NativePlaceholder(a) + case Token("TOK_QUERY", fromClause :: Token("TOK_INSERT", - Token("TOK_DESTINATION", - Token("TOK_DIR", // For now only support queries with no INSERT clause. - Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) :: + destClause :: Token("TOK_SELECT", selectExprs) :: Nil) :: Nil) => - Project(selectExprs.map(nodeToExpr), nodeToPlan(fromClause)) + nodeToDest( + destClause, + Project(selectExprs.map(nodeToExpr), + nodeToPlan(fromClause))) + + // TODO: find a less redundant way to do this. + case Token("TOK_QUERY", + fromClause :: + Token("TOK_INSERT", + destClause :: + Token("TOK_SELECT", + selectExprs) :: + Token("TOK_ORDERBY", + orderByExprs) :: Nil) :: Nil) => + nodeToDest( + destClause, + Sort(orderByExprs.map(nodeToSortOrder), + Project(selectExprs.map(nodeToExpr), + nodeToPlan(fromClause)))) + case Token("TOK_FROM", Token("TOK_TABREF", Token("TOK_TABNAME", Token(name, Nil) :: Nil) :: Nil) :: Nil) => UnresolvedRelation(name, None) case a: ASTNode => - println("FAILURE") - dumpTree(a) - ??? + throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") + } + + def nodeToSortOrder(node: Node): SortOrder = node match { + case Token("TOK_TABSORTCOLNAMEASC", + Token("TOK_TABLE_OR_COL", + Token(name, Nil) :: Nil) :: Nil) => + SortOrder(UnresolvedAttribute(name), Ascending) + + case a: ASTNode => + throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") + } + + protected def nodeToDest(node: Node, query: LogicalPlan): LogicalPlan = node match { + case Token("TOK_DESTINATION", + Token("TOK_DIR", + Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) => + query + case Token("TOK_DESTINATION", + Token("TOK_TAB", + Token("TOK_TABNAME", + Token(tableName, Nil) :: Nil) :: Nil) :: Nil) => + InsertIntoHiveTable(tableName, query) + case a: ASTNode => + throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } protected def nodeToExpr(node: Node): NamedExpression = node match { @@ -55,14 +214,18 @@ object Hive { Token("TOK_TABLE_OR_COL", Token(name, Nil) :: Nil) :: Nil) => UnresolvedAttribute(name) + case a: ASTNode => + throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } - protected def dumpTree(node: Node, indent: Int = 0) { + protected def dumpTree(node: Node, builder: StringBuilder = new StringBuilder, indent: Int = 0) + : StringBuilder = { node match { - case a: ASTNode => println((" " * indent) + a.getText) + case a: ASTNode => builder.append((" " * indent) + a.getText + "\n") case other => sys.error(s"Non ASTNode encountered: $other") } - Option(node.getChildren).map(_.toList).getOrElse(Nil).foreach(dumpTree(_, indent + 1)) + Option(node.getChildren).map(_.toList).getOrElse(Nil).foreach(dumpTree(_, builder, indent + 1)) + builder } } \ No newline at end of file diff --git a/src/main/scala/util/TestShark.scala b/src/main/scala/util/TestShark.scala index 0087abc926ebe..1a3538faca50d 100644 --- a/src/main/scala/util/TestShark.scala +++ b/src/main/scala/util/TestShark.scala @@ -33,7 +33,10 @@ class TestShark { // TODO: Don't just pick the first one... lazy val physicalPlan = TrivalPlanner(analyzed).next() - def execute() = physicalPlan.execute() + def execute() = analyzed match { + case NativeCommand(cmd) => sc.runSql(cmd); null + case _ => physicalPlan.execute() + } override def toString(): String = s"""$sql From dc72469638b68d1081fe14224322facbee436c0d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 19:19:56 -0800 Subject: [PATCH 046/778] beginning of hive compatibility testing framework. --- src/test/scala/HiveCompatability.scala | 74 ++++++++++++++++++++++++++ 1 file changed, 74 insertions(+) create mode 100644 src/test/scala/HiveCompatability.scala diff --git a/src/test/scala/HiveCompatability.scala b/src/test/scala/HiveCompatability.scala new file mode 100644 index 0000000000000..793d89e760ac0 --- /dev/null +++ b/src/test/scala/HiveCompatability.scala @@ -0,0 +1,74 @@ +package catalyst + +import shark.SharkContext +import shark.SharkEnv + +import java.io._ +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import frontend.Hive +import util.TestShark + +class HiveCompatability extends FunSuite with BeforeAndAfterAll { + /** A list of tests currently deemed out of scope and thus completely ignored */ + val blackList = Seq( + "set_processor_namespaces" // Unclear how we want to handle the + ) + + /** The set of tests that are believed to be working in catalyst. Tests not in white */ + val whiteList = Seq( + "tablename_with_select" + ) + + override def beforeAll() { + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests + // without restarting the JVM. + System.clearProperty("spark.driver.port") + + testShark.sc.runSql("CREATE TABLE src (key INT, val STRING)") + testShark.sc.runSql("""LOAD DATA LOCAL INPATH '/Users/marmbrus/workspace/hive/data/files/kv1.txt' INTO TABLE src""") + } + + val testShark = new TestShark + + // TODO: bundle in jar files... + val hiveQueryDir = new File("/Users/marmbrus/workspace/hive/ql/src/test/queries/clientpositive") + val testCases = hiveQueryDir.listFiles + + testCases.foreach { testCase => + val testCaseName = testCase.getName.stripSuffix(".q") + if(blackList contains testCaseName) { + // Do nothing + } else if(whiteList contains testCaseName) { + // Build a test case and submit it to scala test framework... + test(testCaseName) { + val queriesString = fileToString(testCase) + queriesString.split("(?<=[^\\\\]);").map(_.trim).filterNot(_ == "").foreach { queryString => + val query = new testShark.SharkQuery(queryString) + query.execute() + } + } + } else { + ignore(testCaseName) {} + } + } + + def fileToString(file: File, encoding: String = "UTF-8") = { + val inStream = new FileInputStream(file) + val outStream = new ByteArrayOutputStream + try { + var reading = true + while ( reading ) { + inStream.read() match { + case -1 => reading = false + case c => outStream.write(c) + } + } + outStream.flush() + } + finally { + inStream.close() + } + new String(outStream.toByteArray(), encoding) + } +} From 26f410afaefff80c0cc7fda915c8fd5fdb8a388e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 21:12:04 -0800 Subject: [PATCH 047/778] physical traits should extend PhysicalPlan. --- src/main/scala/plans/physical/PhysicalPlan.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/plans/physical/PhysicalPlan.scala b/src/main/scala/plans/physical/PhysicalPlan.scala index e707e45be6958..57f5baa6dae5a 100644 --- a/src/main/scala/plans/physical/PhysicalPlan.scala +++ b/src/main/scala/plans/physical/PhysicalPlan.scala @@ -15,14 +15,14 @@ abstract class PhysicalPlan extends QueryPlan[PhysicalPlan] { def execute(): RDD[_] } -abstract trait LeafNode extends trees.LeafNode[PhysicalPlan] { +abstract trait LeafNode extends PhysicalPlan with trees.LeafNode[PhysicalPlan] { self: Product => } -abstract trait UnaryNode extends trees.UnaryNode[PhysicalPlan] { +abstract trait UnaryNode extends PhysicalPlan with trees.UnaryNode[PhysicalPlan] { self: Product => } -abstract trait BinaryNode extends trees.BinaryNode[PhysicalPlan] { +abstract trait BinaryNode extends PhysicalPlan with trees.BinaryNode[PhysicalPlan] { self: Product => } \ No newline at end of file From e7174ec25e3630922a5fd5c44515158167269813 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 21:12:36 -0800 Subject: [PATCH 048/778] fix schema, add docs, make helper method protected. --- src/test/scala/HiveCompatability.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/test/scala/HiveCompatability.scala b/src/test/scala/HiveCompatability.scala index 793d89e760ac0..252ff9384e429 100644 --- a/src/test/scala/HiveCompatability.scala +++ b/src/test/scala/HiveCompatability.scala @@ -25,16 +25,17 @@ class HiveCompatability extends FunSuite with BeforeAndAfterAll { // without restarting the JVM. System.clearProperty("spark.driver.port") - testShark.sc.runSql("CREATE TABLE src (key INT, val STRING)") + testShark.sc.runSql("CREATE TABLE src (key INT, value STRING)") testShark.sc.runSql("""LOAD DATA LOCAL INPATH '/Users/marmbrus/workspace/hive/data/files/kv1.txt' INTO TABLE src""") } val testShark = new TestShark - // TODO: bundle in jar files... + // TODO: bundle in jar files... get from classpath val hiveQueryDir = new File("/Users/marmbrus/workspace/hive/ql/src/test/queries/clientpositive") val testCases = hiveQueryDir.listFiles + // Go through all the test cases and add them to scala test. testCases.foreach { testCase => val testCaseName = testCase.getName.stripSuffix(".q") if(blackList contains testCaseName) { @@ -53,7 +54,7 @@ class HiveCompatability extends FunSuite with BeforeAndAfterAll { } } - def fileToString(file: File, encoding: String = "UTF-8") = { + protected def fileToString(file: File, encoding: String = "UTF-8") = { val inStream = new FileInputStream(file) val outStream = new ByteArrayOutputStream try { From b34a7709e3b3b0b845e5a7f69937d161c71364c1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 21:14:16 -0800 Subject: [PATCH 049/778] Add data sink strategy, make strategy application a little more robust. --- src/main/scala/planning/QueryPlanner.scala | 6 ++++-- src/main/scala/planning/Strategy.scala | 11 +++++++++++ src/main/scala/plans/physical/basicOperators.scala | 1 + 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/main/scala/planning/QueryPlanner.scala b/src/main/scala/planning/QueryPlanner.scala index 415aad9f78d72..7df6a30e40ea4 100644 --- a/src/main/scala/planning/QueryPlanner.scala +++ b/src/main/scala/planning/QueryPlanner.scala @@ -9,12 +9,14 @@ abstract class QueryPlanner { def apply(plan: LogicalPlan): Iterator[PhysicalPlan] = { // Obviously a lot to do here still... - val iter = strategies.head(plan).toIterator + val iter = strategies.flatMap(_(plan)).toIterator assert(iter.hasNext, s"No plan for $plan") iter } } object TrivalPlanner extends QueryPlanner { - val strategies = HiveTableScans :: Nil + val strategies = + HiveTableScans :: + DataSinks :: Nil } \ No newline at end of file diff --git a/src/main/scala/planning/Strategy.scala b/src/main/scala/planning/Strategy.scala index e4642b5718ee7..a323569eec616 100644 --- a/src/main/scala/planning/Strategy.scala +++ b/src/main/scala/planning/Strategy.scala @@ -9,6 +9,17 @@ import plans.physical.PhysicalPlan abstract class Strategy { def apply(plan: LogicalPlan): Seq[PhysicalPlan] + + // TODO: Actually plan later. + def planLater(plan: LogicalPlan): PhysicalPlan = TrivalPlanner(plan).next +} + +object DataSinks extends Strategy { + def apply(plan: LogicalPlan): Seq[PhysicalPlan] = plan match { + case InsertIntoHiveTable(tableName, child) => + physical.InsertIntoHiveTable(tableName, planLater(child)) :: Nil + case _ => Nil + } } object HiveTableScans extends Strategy { diff --git a/src/main/scala/plans/physical/basicOperators.scala b/src/main/scala/plans/physical/basicOperators.scala index 7d7a9ef12495f..89cc9d543c5c1 100644 --- a/src/main/scala/plans/physical/basicOperators.scala +++ b/src/main/scala/plans/physical/basicOperators.scala @@ -57,5 +57,6 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: analysis.Metastor case class InsertIntoHiveTable(tableName: String, child: PhysicalPlan) extends UnaryNode { def output = Seq.empty + def execute() = ??? } From ed3633ead1fe104ff5bafb444a412d066cc90724 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Nov 2013 23:07:59 -0800 Subject: [PATCH 050/778] start consolidating Hive/Shark specific code. first hive compatibility test case passing! --- src/main/scala/planning/QueryPlanner.scala | 6 -- src/main/scala/planning/Strategy.scala | 30 +--------- .../scala/plans/physical/PhysicalPlan.scala | 4 +- .../scala/plans/physical/basicOperators.scala | 28 ++++++++-- src/main/scala/util/TestShark.scala | 56 ++++++++++++++++++- src/test/scala/HiveCompatability.scala | 2 +- 6 files changed, 81 insertions(+), 45 deletions(-) diff --git a/src/main/scala/planning/QueryPlanner.scala b/src/main/scala/planning/QueryPlanner.scala index 7df6a30e40ea4..10841dce3ada5 100644 --- a/src/main/scala/planning/QueryPlanner.scala +++ b/src/main/scala/planning/QueryPlanner.scala @@ -13,10 +13,4 @@ abstract class QueryPlanner { assert(iter.hasNext, s"No plan for $plan") iter } -} - -object TrivalPlanner extends QueryPlanner { - val strategies = - HiveTableScans :: - DataSinks :: Nil } \ No newline at end of file diff --git a/src/main/scala/planning/Strategy.scala b/src/main/scala/planning/Strategy.scala index a323569eec616..70725ed65a875 100644 --- a/src/main/scala/planning/Strategy.scala +++ b/src/main/scala/planning/Strategy.scala @@ -11,34 +11,6 @@ abstract class Strategy { def apply(plan: LogicalPlan): Seq[PhysicalPlan] // TODO: Actually plan later. - def planLater(plan: LogicalPlan): PhysicalPlan = TrivalPlanner(plan).next -} -object DataSinks extends Strategy { - def apply(plan: LogicalPlan): Seq[PhysicalPlan] = plan match { - case InsertIntoHiveTable(tableName, child) => - physical.InsertIntoHiveTable(tableName, planLater(child)) :: Nil - case _ => Nil - } -} - -object HiveTableScans extends Strategy { - def apply(plan: LogicalPlan): Seq[PhysicalPlan] = plan match { - case p @ Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => - physical.HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m) :: Nil - case m: MetastoreRelation => - physical.HiveTableScan(m.output, m) :: Nil - case _ => Nil - } - - /** - * Returns true if [[projectList]] only performs column pruning and - * does not evaluate other complex expressions. - */ - def isSimpleProject(projectList: Seq[NamedExpression]) = { - projectList.map { - case a: Attribute => true - case _ => false - }.reduceLeft(_ && _) - } + //def planLater(plan: LogicalPlan): PhysicalPlan = TrivalPlanner(plan).next } \ No newline at end of file diff --git a/src/main/scala/plans/physical/PhysicalPlan.scala b/src/main/scala/plans/physical/PhysicalPlan.scala index 57f5baa6dae5a..6b0007169da1c 100644 --- a/src/main/scala/plans/physical/PhysicalPlan.scala +++ b/src/main/scala/plans/physical/PhysicalPlan.scala @@ -9,10 +9,10 @@ abstract class PhysicalPlan extends QueryPlan[PhysicalPlan] { /** * Runs this query returning the result as an RDD. - * This fact that this returns an RDD should probably be + * TODO: This fact that this returns an RDD should probably be * abstracted away from the rest of the planning code. */ - def execute(): RDD[_] + def execute(): RDD[IndexedSeq[Any]] } abstract trait LeafNode extends PhysicalPlan with trees.LeafNode[PhysicalPlan] { diff --git a/src/main/scala/plans/physical/basicOperators.scala b/src/main/scala/plans/physical/basicOperators.scala index 89cc9d543c5c1..91e3f4bdf1187 100644 --- a/src/main/scala/plans/physical/basicOperators.scala +++ b/src/main/scala/plans/physical/basicOperators.scala @@ -12,7 +12,14 @@ import shark.{SharkContext, SharkEnv} import collection.JavaConversions._ import org.apache.hadoop.hive.serde2.`lazy`.LazyPrimitive -case class HiveTableScan(attributes: Seq[Attribute], relation: analysis.MetastoreRelation) extends PhysicalPlan with LeafNode { +case class Sort(sortExprs: Seq[SortOrder], child: PhysicalPlan) extends UnaryNode { + // TODO: actually sort + def execute() = child.execute() + + def output = child.output +} + +case class HiveTableScan(attributes: Seq[Attribute], relation: analysis.MetastoreRelation) extends LeafNode { val hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(relation.table) val tableDesc = new TableDesc( Class.forName(relation.table.getSd.getSerdeInfo.getSerializationLib).asInstanceOf[Class[org.apache.hadoop.hive.serde2.Deserializer]], @@ -48,15 +55,26 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: analysis.Metastor refs.map { ref => val data = objectInspector.getStructFieldData(struct, ref) ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector].getPrimitiveJavaObject(data) - }.toSeq + }.toIndexedSeq } } def output = attributes } -case class InsertIntoHiveTable(tableName: String, child: PhysicalPlan) extends UnaryNode { - def output = Seq.empty - def execute() = ??? +case class InsertIntoHiveTable(tableName: String, child: PhysicalPlan) + (sc: SharkContext) extends UnaryNode { + def output = child.output + def execute() = { + val childRdd = child.execute() + // TODO: write directly to hive + val tempDir = java.io.File.createTempFile("data", "tsv") + tempDir.delete() + tempDir.mkdir() + childRdd.map(_.map(_.toString).mkString("\001")).saveAsTextFile(tempDir.getCanonicalPath) + sc.runSql(s"LOAD DATA LOCAL INPATH '${tempDir.getCanonicalPath}/*' INTO TABLE $tableName") + + childRdd + } } diff --git a/src/main/scala/util/TestShark.scala b/src/main/scala/util/TestShark.scala index 1a3538faca50d..b3bebe59a0361 100644 --- a/src/main/scala/util/TestShark.scala +++ b/src/main/scala/util/TestShark.scala @@ -1,9 +1,13 @@ package catalyst package util -import catalyst.analysis.{Analyzer, HiveMetastoreCatalog} +import catalyst.analysis.{MetastoreRelation, Analyzer, HiveMetastoreCatalog} +import catalyst.expressions.{NamedExpression, Attribute} import catalyst.frontend._ -import catalyst.planning.TrivalPlanner +import catalyst.planning.{QueryPlanner, Strategy} +import catalyst.plans.logical._ +import catalyst.plans.physical +import catalyst.plans.physical.PhysicalPlan import shark.{SharkContext, SharkEnv} import util._ @@ -21,12 +25,60 @@ class TestShark { def loadKv1 { //sc.runSql("DROP TABLE IF EXISTS test") sc.runSql("CREATE TABLE test (key INT, val STRING)") + // USE ENV VARS sc.runSql("""LOAD DATA LOCAL INPATH '/Users/marmbrus/workspace/hive/data/files/kv1.txt' INTO TABLE test""") } val catalog = new HiveMetastoreCatalog(SharkContext.hiveconf) val analyze = new Analyzer(new HiveMetastoreCatalog(SharkContext.hiveconf)) + def planLater(plan: LogicalPlan): PhysicalPlan = TrivalPlanner(plan).next + + object DataSinks extends Strategy { + def apply(plan: LogicalPlan): Seq[PhysicalPlan] = plan match { + case InsertIntoHiveTable(tableName, child) => + physical.InsertIntoHiveTable(tableName, planLater(child))(sc) :: Nil + case _ => Nil + } + } + + object HiveTableScans extends Strategy { + def apply(plan: LogicalPlan): Seq[PhysicalPlan] = plan match { + case p @ Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => + physical.HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m) :: Nil + case m: MetastoreRelation => + physical.HiveTableScan(m.output, m) :: Nil + case _ => Nil + } + + /** + * Returns true if [[projectList]] only performs column pruning and + * does not evaluate other complex expressions. + */ + def isSimpleProject(projectList: Seq[NamedExpression]) = { + projectList.map { + case a: Attribute => true + case _ => false + }.reduceLeft(_ && _) + } + } + + // Can we automate these 'pass through' operations? + object BasicOperators extends Strategy { + def apply(plan: LogicalPlan): Seq[PhysicalPlan] = plan match { + case Sort(sortExprs, child) => + physical.Sort(sortExprs, planLater(child)) :: Nil + case _ => Nil + } + } + + object TrivalPlanner extends QueryPlanner { + val strategies = + HiveTableScans :: + DataSinks :: + BasicOperators :: Nil + } + class SharkQuery(sql: String) { lazy val parsed = Hive.parseSql(sql) lazy val analyzed = analyze(parsed) diff --git a/src/test/scala/HiveCompatability.scala b/src/test/scala/HiveCompatability.scala index 252ff9384e429..618a46279f276 100644 --- a/src/test/scala/HiveCompatability.scala +++ b/src/test/scala/HiveCompatability.scala @@ -46,7 +46,7 @@ class HiveCompatability extends FunSuite with BeforeAndAfterAll { val queriesString = fileToString(testCase) queriesString.split("(?<=[^\\\\]);").map(_.trim).filterNot(_ == "").foreach { queryString => val query = new testShark.SharkQuery(queryString) - query.execute() + Option(query.execute()).foreach(_.foreach(println)) } } } else { From e1f7f4c37b495683bba96227f51650a3149c7b1e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 18 Nov 2013 13:45:22 -0800 Subject: [PATCH 051/778] Remove "NativePlaceholder" hack. --- src/main/scala/frontend/Hive.scala | 146 ++++++++++++++--------------- 1 file changed, 73 insertions(+), 73 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 891e0ec0337e9..7e4261c203967 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -39,6 +39,74 @@ case class AddJar(jarPath: String) extends Command case class AddFile(filePath: String) extends Command object Hive { + protected val nativeCommands = Seq( + "TOK_EXPLAIN", + + "TOK_DESCFUNCTION", + "TOK_DESCTABLE", + "TOK_DESCDATABASE", + "TOK_SHOW_TABLESTATUS", + "TOK_SHOWDATABASES", + "TOK_SHOWFUNCTIONS", + "TOK_SHOWINDEXES", + "TOK_SHOWINDEXES", + "TOK_SHOWPARTITIONS", + "TOK_SHOWTABLES", + + "TOK_LOCKTABLE", + "TOK_SHOWLOCKS", + "TOK_UNLOCKTABLE", + + "TOK_CREATEROLE", + "TOK_DROPROLE", + "TOK_GRANT", + "TOK_GRANT_ROLE", + "TOK_REVOKE", + "TOK_SHOW_GRANT", + "TOK_SHOW_ROLE_GRANT", + + "TOK_CREATEFUNCTION", + "TOK_DROPFUNCTION", + + "TOK_ALTERDATABASE_PROPERTIES", + "TOK_ALTERINDEX_PROPERTIES", + "TOK_ALTERINDEX_REBUILD", + "TOK_ALTERTABLE_ADDCOLS", + "TOK_ALTERTABLE_ADDPARTS", + "TOK_ALTERTABLE_ARCHIVE", + "TOK_ALTERTABLE_CLUSTER_SORT", + "TOK_ALTERTABLE_DROPPARTS", + "TOK_ALTERTABLE_PARTITION", + "TOK_ALTERTABLE_PROPERTIES", + "TOK_ALTERTABLE_RENAME", + "TOK_ALTERTABLE_RENAMECOL", + "TOK_ALTERTABLE_REPLACECOLS", + "TOK_ALTERTABLE_TOUCH", + "TOK_ALTERTABLE_UNARCHIVE", + "TOK_ANALYZE", + "TOK_CREATEDATABASE", + "TOK_CREATEINDEX", + "TOK_CREATETABLE", + "TOK_DROPDATABASE", + "TOK_DROPINDEX", + "TOK_DROPTABLE", + "TOK_MSCK", + + // TODO(marmbrus): Figure out how view are expanded by hive, as we might need to handle this. + "TOK_ALTERVIEW_ADDPARTS", + "TOK_ALTERVIEW_DROPPARTS", + "TOK_ALTERVIEW_PROPERTIES", + "TOK_ALTERVIEW_RENAME", + "TOK_CREATEVIEW", + "TOK_DROPVIEW", + + "TOK_EXPORT", + "TOK_IMPORT", + "TOK_LOAD", + + "TOK_SWITCHDATABASE" + ) + def parseSql(sql: String): LogicalPlan = { if(sql.toLowerCase.startsWith("set")) ConfigurationAssignment(sql) @@ -59,13 +127,13 @@ object Hive { (new ParseDriver()).parse(sql, null /* no context required for parsing alone */)) } catch { case pe: org.apache.hadoop.hive.ql.parse.ParseException => - sys.error(s"Failed to parse '$sql'") + throw new RuntimeException(s"Failed to parse sql: '$sql'", pe) } - nodeToPlan(tree) match { - case NativePlaceholder(_) => NativeCommand(sql) - case other => other - } + if(nativeCommands contains tree.getText) + NativeCommand(sql) + else + nodeToPlan(tree) } } @@ -78,77 +146,9 @@ object Hive { } } - // Kinda gross hack... - case class NativePlaceholder(ast: Node) extends Command - - protected def nodeToPlan(node: Node): LogicalPlan = node match { - case a @ Token("TOK_EXPLAIN", _) => NativePlaceholder(a) - - case a @ Token("TOK_DESCFUNCTION", _) => NativePlaceholder(a) - case a @ Token("TOK_DESCTABLE", _) => NativePlaceholder(a) - case a @ Token("TOK_DESCDATABASE", _) => NativePlaceholder(a) - case a @ Token("TOK_SHOW_TABLESTATUS", _) => NativePlaceholder(a) - case a @ Token("TOK_SHOWDATABASES", _) => NativePlaceholder(a) - case a @ Token("TOK_SHOWFUNCTIONS", _) => NativePlaceholder(a) - case a @ Token("TOK_SHOWINDEXES", _) => NativePlaceholder(a) - case a @ Token("TOK_SHOWINDEXES", _) => NativePlaceholder(a) - case a @ Token("TOK_SHOWPARTITIONS", _) => NativePlaceholder(a) - case a @ Token("TOK_SHOWTABLES", _) => NativePlaceholder(a) - - case a @ Token("TOK_LOCKTABLE", _) => NativePlaceholder(a) - case a @ Token("TOK_SHOWLOCKS", _) => NativePlaceholder(a) - case a @ Token("TOK_UNLOCKTABLE", _) => NativePlaceholder(a) - - case a @ Token("TOK_CREATEROLE", _) => NativePlaceholder(a) - case a @ Token("TOK_DROPROLE", _) => NativePlaceholder(a) - case a @ Token("TOK_GRANT", _) => NativePlaceholder(a) - case a @ Token("TOK_GRANT_ROLE", _) => NativePlaceholder(a) - case a @ Token("TOK_REVOKE", _) => NativePlaceholder(a) - case a @ Token("TOK_SHOW_GRANT", _) => NativePlaceholder(a) - case a @ Token("TOK_SHOW_ROLE_GRANT", _) => NativePlaceholder(a) - - case a @ Token("TOK_CREATEFUNCTION", _) => NativePlaceholder(a) - case a @ Token("TOK_DROPFUNCTION", _) => NativePlaceholder(a) - - case a @ Token("TOK_ALTERDATABASE_PROPERTIES", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERINDEX_PROPERTIES", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERINDEX_REBUILD", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_ADDCOLS", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_ADDPARTS", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_ARCHIVE", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_DROPPARTS", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_PARTITION", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_PROPERTIES", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_RENAME", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_RENAMECOL", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_REPLACECOLS", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_TOUCH", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERTABLE_UNARCHIVE", _) => NativePlaceholder(a) - case a @ Token("TOK_ANALYZE", _) => NativePlaceholder(a) - case a @ Token("TOK_CREATEDATABASE", _) => NativePlaceholder(a) - case a @ Token("TOK_CREATEINDEX", _) => NativePlaceholder(a) - case a @ Token("TOK_CREATETABLE", _) => NativePlaceholder(a) - case a @ Token("TOK_DROPDATABASE", _) => NativePlaceholder(a) - case a @ Token("TOK_DROPINDEX", _) => NativePlaceholder(a) - case a @ Token("TOK_DROPTABLE", _) => NativePlaceholder(a) - case a @ Token("TOK_MSCK", _) => NativePlaceholder(a) - - // TODO(marmbrus): Figure out how view are expanded by hive, as we might need to handle this. - case a @ Token("TOK_ALTERVIEW_ADDPARTS", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERVIEW_DROPPARTS", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERVIEW_PROPERTIES", _) => NativePlaceholder(a) - case a @ Token("TOK_ALTERVIEW_RENAME", _) => NativePlaceholder(a) - case a @ Token("TOK_CREATEVIEW", _) => NativePlaceholder(a) - case a @ Token("TOK_DROPVIEW", _) => NativePlaceholder(a) - case a @ Token("TOK_EXPORT", _) => NativePlaceholder(a) - case a @ Token("TOK_IMPORT", _) => NativePlaceholder(a) - case a @ Token("TOK_LOAD", _) => NativePlaceholder(a) - case a @ Token("TOK_SWITCHDATABASE", _) => NativePlaceholder(a) - case a @ Token("TOK_EXPLAIN", _) => NativePlaceholder(a) case Token("TOK_QUERY", fromClause :: From 063bf44cc319a50b29874b9deae14cf6f9822d91 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 18 Nov 2013 13:45:55 -0800 Subject: [PATCH 052/778] Periods should end all comments. --- src/main/scala/trees/TreeNode.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index 308726d656f81..14c8fd5b56f04 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -85,7 +85,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { protected def makeCopy(newArgs: Array[AnyRef]): this.type = getClass.getConstructors.head.newInstance(newArgs: _*).asInstanceOf[this.type] - /** Returns the name of this type of TreeNode. Defaults to the class name */ + /** Returns the name of this type of TreeNode. Defaults to the class name. */ def nodeName = getClass.getSimpleName /** Returns a string representing the arguments to this node, minus any children */ From 15371a8fdedf85f790fddcc8e5819ccec23352e6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 18 Nov 2013 13:48:33 -0800 Subject: [PATCH 053/778] First draft of simple Hive DDL parser. --- src/main/scala/frontend/Hive.scala | 62 ++++++++++++++++++++++++++++ src/main/scala/types/dataTypes.scala | 10 ++++- 2 files changed, 71 insertions(+), 1 deletion(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 7e4261c203967..134ee75619675 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -8,6 +8,7 @@ import org.apache.hadoop.hive.ql.parse._ import analysis._ import expressions._ import plans.logical._ +import types._ import collection.JavaConversions._ @@ -137,6 +138,25 @@ object Hive { } } + def parseDdl(ddl: String): Seq[Attribute] = { + val tree = + try { + ParseUtils.findRootNonNullToken( + (new ParseDriver()).parse(ddl, null /* no context required for parsing alone */)) + } catch { + case pe: org.apache.hadoop.hive.ql.parse.ParseException => + throw new RuntimeException(s"Failed to parse ddl: '$ddl'", pe) + } + assert(tree.asInstanceOf[ASTNode].getText == "TOK_CREATETABLE", "Only CREATE TABLE supported.") + val tableOps = tree.getChildren + val colList = + tableOps + .find(_.asInstanceOf[ASTNode].getText == "TOK_TABCOLLIST") + .getOrElse(sys.error("No columnList!")).getChildren + + colList.map(nodeToAttribute) + } + /** Extractor for matching Hive's AST Tokens */ protected object Token { def unapply(t: Any) = t match { @@ -146,10 +166,52 @@ object Hive { } } + protected def nodeToAttribute(node: Node): Attribute = node match { + case Token("TOK_TABCOL", + Token(colName, Nil) :: + dataType :: Nil) => + AttributeReference(colName, nodeToDataType(dataType), true)() + case a: ASTNode => + throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") + } + protected def nodeToDataType(node: Node): DataType = node match { + case Token("TOK_BIGINT", Nil) => IntegerType + case Token("TOK_INT", Nil) => IntegerType + case Token("TOK_TINYINT", Nil) => IntegerType + case Token("TOK_SMALLINT", Nil) => IntegerType + case Token("TOK_BOOLEAN", Nil) => BooleanType + case Token("TOK_STRING", Nil) => StringType + case Token("TOK_FLOAT", Nil) => FloatType + case Token("TOK_DOUBLE", Nil) => FloatType + case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType)) + case Token("TOK_STRUCT", + Token("TOK_TABCOLLIST", fields) :: Nil) => + StructType(fields.map(nodeToStructField)) + case Token("TOK_MAP", + keyType :: + valueType :: Nil) => + MapType(nodeToDataType(keyType), nodeToDataType(valueType)) + case a: ASTNode => + throw new NotImplementedError(s"No parse rules for DataType:\n ${dumpTree(a).toString} ") + } + protected def nodeToStructField(node: Node): StructField = node match { + case Token("TOK_TABCOL", + Token(fieldName, Nil) :: + dataType :: Nil) => + StructField(fieldName, nodeToDataType(dataType)) + case Token("TOK_TABCOL", + Token(fieldName, Nil) :: + dataType :: + _ /* comment */:: Nil) => + StructField(fieldName, nodeToDataType(dataType) ) + case a: ASTNode => + throw new NotImplementedError(s"No parse rules for StructField:\n ${dumpTree(a).toString} ") + } + protected def nodeToPlan(node: Node): LogicalPlan = node match { case Token("TOK_QUERY", fromClause :: Token("TOK_INSERT", diff --git a/src/main/scala/types/dataTypes.scala b/src/main/scala/types/dataTypes.scala index 8a55ff98bd0ba..908221c841238 100644 --- a/src/main/scala/types/dataTypes.scala +++ b/src/main/scala/types/dataTypes.scala @@ -5,4 +5,12 @@ sealed class DataType case object IntegerType extends DataType case object StringType extends DataType -case object BooleanType extends DataType \ No newline at end of file +case object BooleanType extends DataType +case object FloatType extends DataType + +case class ArrayType(elementType: DataType) extends DataType + +case class StructField(name: String, dataType: DataType) +case class StructType(fields: Seq[StructField]) extends DataType + +case class MapType(keyType: DataType, valueType: DataType) extends DataType \ No newline at end of file From 84082f95f793d0de62405254f979e5c71214da10 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 18 Nov 2013 14:07:04 -0800 Subject: [PATCH 054/778] add sbt binaries and scripts --- sbt/jansi-license.txt | 202 +++++++++++++++++++++++++ sbt/jansi.jar | Bin 0 -> 112811 bytes sbt/sbt | 107 +++++++++++++ sbt/sbt-launch-lib.bash | 159 +++++++++++++++++++ sbt/sbt-launch.jar | Bin 0 -> 1105722 bytes sbt/sbt.bat | 54 +++++++ sbt/win-sbt | 35 +++++ src/test/scala/HiveCompatability.scala | 17 ++- 8 files changed, 570 insertions(+), 4 deletions(-) create mode 100755 sbt/jansi-license.txt create mode 100755 sbt/jansi.jar create mode 100755 sbt/sbt create mode 100755 sbt/sbt-launch-lib.bash create mode 100755 sbt/sbt-launch.jar create mode 100755 sbt/sbt.bat create mode 100755 sbt/win-sbt diff --git a/sbt/jansi-license.txt b/sbt/jansi-license.txt new file mode 100755 index 0000000000000..d645695673349 --- /dev/null +++ b/sbt/jansi-license.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/sbt/jansi.jar b/sbt/jansi.jar new file mode 100755 index 0000000000000000000000000000000000000000..287310fef87faf79be968a34c53b184ed316c2b7 GIT binary patch literal 112811 zcmeFXb980Vx+j{7ZQB*wHY&F5ifvYQY+DuEwr$%<#Wp&1-t9Y1kG}W5d-{#{U++EE zUiv5_sJYi@@8IF(pB&1Q2CukEm6NVnNPB9!Dqoj5}#- z70d`6A{~}Z)4(cF=z_j&UCY$i8mc_}^O>H<9f&Y-dB9-A+g+YJwTC1*Hr#tYQyN-P zMX;caU%2Ac+@>-SQ5p>V=*!h8XZ*mmM@Qi}LnrVFr%{XCnj2rbbn-LHiRY5d9+y{V zMRt9aX6;KnR95u{t<6){5c329=%WzCAr!Eqd_9S*`47*oSITE5Cf(?9)FM6ZWer23 z@Z&}|pCgU{JkN~^>&bGh<6bRr{zv1Oe2dSY4ASv;TEUvQHub`fZSx4#xBz?WCzmBK zq$Gx|X&4ek9-=;4O4c}N)Et7aQb;8*3;IcYOhj{|-zIz{S>afO=pSr39O^Ij8AA{6 z_#e+mYZPJfKejKh-zSadR$FH(IyRiUQWxKLmfG85f5SQI3>h#8`EGr*Y~EjxT476V zA{!urSN#P6oQs2Z-!~wj77(B>Q2b{IV*6(hSR1&Q*wEWLn9-R!JDND!Iy)Gd(3u;! zI{mb}f5j`p|L*l)F&FvY&HtNKx3jgTvvaVuGjVXT zFmY6Jk(-dB{XrusqaabLd?iC8HL<4%4{;Nq1cMp@iv}B69wA*3F^;CKBlJbX!nKSl zhKDiV-de%kOSjC1-$=pmC|TWx&)(1e2v-Wfl?snjx@STjld8R8YC#VF!8C^)FXcy- zBh)Hd;7*p>QlkBjHlFnjHP+#)25w*1fBtxR|3B5y&Dtta+1fUp0r4YC&tCX-`UhB? zO?+dNqKUV{%4$BLQ-%MAW77YVjK>osk&??R>*yqJ%xylGKeyW zdb<`gL*Hg-jnp5NmKwydYqMffvzr7C^hAq};-A(OigAFV{~y9;}^9 zV7Jp*Z&C~sPc!d6-#=NdIl9~#F5|lK)z%xp zR7Xs8=*kD=e+y`?Ayo2P>dWr z>AXy%9V4lbsDN##Tjd&_l)o6A0p@dLJ0tS5kuX&3muRzUZ};W1W_8EK&hRVZ(?$>C z6*u^=Pm-&qo7&wpG^e&V>Ve4M_<*2f%nxVe9&3zE^+@gls@QTCW@xj+Ona1_T|~G> z1#NN@^vbcEq`cAVT@_egNwM+ij_%7Mi>jZ~0_AKbx+Wp82JaS^zDH0Hncl*LZJ_63 z%eY$1t655I6w&*k`NZ~XDH{g&nEGzSPXQ*0q=P$e;JvkF?%xXS z@p0k2i?C#%bG7x`u&|u=w{PFcYh&PK;qtF_y`=xm0RKwQ`F9TT?=y%0aM1V<3jM!!ns_6;Fuek>emlHf#M3K% z>oWz@GO_To(33SXW1RaxEavoT!PXuFF0_WF9Au2mHD$3u+#oc9_dvqqfUrg^)wigM@+auT|@P-Xgnxkrq6> zD_uBQ+_sNfubvwiQFX!7lv1jcnUkO|7k8d_FIQJp5Xa;ua|w1);8w)w9Atip zyNr^tDM<5Dc0iwu+z`hEnm%`{ydqnYf!=iupYPl`)rMGMSI`5}IT0v?f}FsUps^+b z$o}RL!02RzKX4K2a|{cYF^TD7o~$IQW4rw#zw!Go^bZQFOSm6wNr&t&-=c>D!V}Bs zq>6NlubW+G_<7r~GdKyWc~JUJSQ#y zwmAl}H*(5hD%8$*N(c=raaa*B!V%PgTfKyn_pOfpSj4%5fDqVI@_o9!-$3*}+1Pxj z40^Bo*4imrw4_Gx+V^pfRz9h4dBXhS`FXPUjaZ*TUpOh$0myx7Hsod#5gSnvNx!#V zfQRT?<}}b6o>`Wx?UE9pB(9_2lndwFynH1)^QHQ#csx-0bDyL5G7xAg<@upNmufMxAH z#?h_ z()@|`COnbt?%*7o^>gaLt*o)5ItGOR1|o}#uP#2-(86BI2l-5VZ9JWz?bMxgv!&I0 z#2?B~ZC-0%s6_d7f0#eHvim9ws9F!V7kNw8o6u=H7zwT{dLW%mxgsnGZO8t^7?Et{ z6?=4x>WLSllvNu=C?X|i67(PQPZtz`iVHxX7P$O5EiM4@eY*@XHU^G}*O@oHF%~ND zTQpkCc*Kt118PL6iBqvc(<$rr`LQheQ`r(=J-bM#D<+p zwjFaON+_tq{+`^Y(6F{Om~4od>x3|)Nwe+xtJ7a`{hj*530bpJTX`@NI#rUAjww5U z=I9b7-ScRjD){+I6~;nP6N}^YWB2CN8zaEah1L7^(+m*U@$OhqtCE-*mV(Ye_re<<(472~rz{+=#fkAe7pRzULI)ZZ zflJf7_8$I#T+k~WXXazMnW-4JfG9MerH>>GU&nr$vKNH0&BEit)wGN>Mw+Gd+&#Mi zMbwMpNsLH;*uex+*xF1urXQNb7~#a~ME02NegljtiumJop>IRCE@PyN_A&^Xa(AHx zdWb=Bz$=>Mc4)#w?$(Jj(EWqPLClqRY&9Gl9ZpeEQ8891^@YZ=LaAo5QgrlPY;jA! zq#tF*twRMUDdGv=%jO(W#>l zH{5({hP?Odr74%#`OL$kiR@=uv5OldcC2r0$`ij1JRXJKz$%3<(v~KA4I?>}|0XN^ zsBA7!1vH!G3#Af`<-2x6*(l7_QwH~yr_6&$0!Y((8;KK}KG>w8WZ^}CXb-SgFH#kGO5$1HQq3E&oq*W+Wo zlRgLbxIrZP6Yo&d;GJYS{%LYfZx`WPviuK09a|6WleZt|xsyuY#WQU&@rVYVM&^D` zGDk#naZg3pPv2Z3k&t>HYZOXk9$}I=XW2*1yJrb1nyryS@z%|GHb}g_yXrkOy4Nqn zb+7sb7Z#-M(hxp6ZY$lqv15>B6&M3=Hd^X%{Vb7AztzFe)f z-t!06R*M<51TbmE`G_+y+&S7s`ovGCbr-)rb(QS}WWZ-7*tlHW=a!Xq;=8AAJv0JA zkOmtth(5L4cI3pY$;sq~TQCg_E)gcs2ec#7&c;dZxXU~}-4*^s ze-76b-VMDi#!6TM_A*DhRzCImDX^r493Fv7r)n^dF0V;<@OxrTW?${j&uM;NOQ#)F z{4>YhC0oiaO%>3SmZTDBh5RLm%Nwg@#}~DuO_R$~UmrHi*5o4x53`x#l)UX$$71xG z^A-Kd?8?L3)}oV)_BIDLJ@+P?@N2^-v>MMwvf<0*lHV0wmVw+P#MN`yhd38z&=rTM z*iK5tWA~pD9?1>n;{iGqSBLf|>4p|U*U(uO>zByLc&c=LDD7ag zpaM}mU-)778$?AK=Z|uqH*|DpftAR@PdMZ$;*82h12mVF2vdai#+@_-Bw2};<)E^{ zlc+O16O>lCH6GEXpZFr@6rA-QP`YEZ>fPd~>9!Nm1LBkb zigIMKCB#$*Xv1jL1^s)mz8OQYUs%L78T@um7)m2B!R8`H%!8pb_dxHOK3R;YFb2iS z)Wu%FKqklSNj7Z}uU8D==FPKDhptOTNnI>*tB!`JvZe4ET}6XB=PabwT@^au1t
3QEK}=U8>>;A=e{!6 zh^%+4{<8kXRXdzIQCudT%KnT9PucsUXiY4Y?a>q=#Tp@Eaw&;6yB?_Qn!(?)Q?*my zymO{!D8q@LTg(CAhmftb%Pd1158^zOvhSz0yL1&45;RWA4)h~p@@*N~J7i_R~ep-E;K% z^CRDAOvs;I9Ut@9v@GtqES7#4k)difuNQdLG=N4_nLV6+aH^79J#Z-bVDe#i=DJt?_$W8ir-&L8 zOJ39aID^Ao)p?(l0b5MaXH263cj) z64x>StuV+*E8XZg!g3*xlK_I((JyT=G6<*?Y#4>3uEbuh@K?2kd^do}dJK9u!0~QB zkM`;CVGd~OSgP{;rSRzmxm2|?aKCNhJKu%By##g#e!27bv2B7w0SquWCT;>{c(oE> zc?D<4uTK&n9t~8MB}uA=Jl%~Ckfx9Zy@@vBj8RHuG-*`CI?PELGK95Z>hg;ZQC*UrVm;=h9aR=Z_SL}Nuc1hiEsYv-f>c}ni zq-pZ9k*$P!F(hu-EQt~V+|xL)uYNufQ6hbNfr>$^vv8%II`rRP{tMTA3?su%Uv12m)X zUO+(^42xVi6dS_9(rD9+_L9ayqdx4%Li9|u? zEmLni(-NUsP^eD&7TLz>G05m`sC3kW@Vaj(d_`wa*gF=TbWXa*vCZAddc`LGejA)R zj{y|}L)2Ei!v1)Y?CQRY6(Je=1j546$Vxw+XY54OSLMKa`;`m=){NH|z)xWgF`=Ee z3*!Av3c?#T_o_e*Lo4or>^Cs@TqZ0m1B+z=!NG#I5mh4Vzl67(HiJkH<|jqT?=xW* z`zC-hL9L`lWw1Vj$^CA-ux*~UoC24o*i2bCKN^gcj5(PkFUTJ=*Uys-RiMLO+qf!X z^_7$7E-r)7KVn``(rO2EoukjG^|weiuA@8sbZa@B448X03~g=D*&{NFfB4G@9jr_S zmg?+k_7tvrDeqeSG~nSyd61EweK24*4>T{tT+iIY3#2aMyFFRSKLB5-W_((o6E$WT z5M#5tnEh#aPBuNSXGA7h;Ms0pPsiz3S~NE5X|djIF`ss6 zMCBmyYUlx9PUXj*8%T3pYtd*Ez*7=;+YZXa?F!-VKt^|sJM|MB2&f$6zZ&i_|9QCow>?HH3ma!QW~P5> za7X+P{QN8XiX{Jm{r@b7m4)FyHWld{ZO^@7yp#tv9#TJ7Heds#1d!zs0|+qyq}s~L zrDBjFLE%s$?Um`q!fd1~%`0}-iZgyfAdq1d*50&w9g9+BhG?f9If|BAW#$d5KRZ|{ z8kT1}=abpCK2sgD&lDTpH@9BjH$@PRc+9d=KgWIP%uFZw+SHN~r;vp(C$mRBzg3Jf zMwEC}b_})hFHo=u4RRQvX7e`QH|1qJ2Y$POMXS7kYl;%|GGZY>;?8)g5A9?BqIH6t z33>X5RtWMfBVm1IpMdxr&MY&`vanh`7Mg#1;MFmkR>D{4hI__E>w4|q%mhbwCRyw; za90REaArsMc){-M+3`hf8&MFL3_YYjUQ_#?H94LRa^r7YS#+_}Y4DuYJXUsjZEsah zc)tS%?Un$K84tIXde13Y{5Rh(BU?T8B5KthlO{HK-U=)C%MXIP9$pFH8Tp4AM(E;q5x_iiX!bRpMna@;#7s@a0KVg(f287d_QBIvU%N*{?>ds* zvfnqHnB>f8CGyWt?ZO&QWegvoyUeG$xIqnZW7_u?T!Rnk@%Q$+4;JYVFm$d2AKg9| zBFSd%;}}HwrcG@oQb4E)0U|GtZVNZZk`S{TJtGa zaIIyA6#tmdhb)q%p?KAqiozz>4E7h;lXx7~nu~f#=CR_*O_)bCG!AM-zAWE32|E&F zOBETh-p@FBQ{k7fPL`*g2F(wG5ZxY%9I+>Ve|C^HK{Z!&s_pR~Ik;V$y*)h~Ophx` zwApV}=E1j?Bu`s0W=Z-npo-mm5X3%!jT6s`-PdZ7fu)5WPOc=C=WLxZXRK99j&B{p z+C-igHVz1zq>xiCGAv2-`X2kigEYCT)NVhzPmiCQpKp9Kk9B*bGD%M#E?b(RprFb| z&-Ld{w9Yh9MdbTLD!b)JvaF-ez&JgY>ts7vIJ($(m4u0>8;cAT7aJ!<4V9I4U)J2w zmV$!=sz^XKH&_+0Rts1q3pUCEt7yeY-d?16J6lqeJxZmGbwe8K*3U{X;Q~f`#Tisn z?7Fd|3_cRBmXKUe9m8n0{Et4Rz1vUmLAu&Oqql}AiM!tTBj#6KUE&#xKVs6wYkh8fQW^5M)w#Y3q>RcKz52Ac;Qj@V^(hk~Z8lk2HVyW{Pmr+nsH z)ZA(gvzcyu&U^#xzO{YFt3gO5r>Yi5Ri~;vf+zMZN11dVZD=j!;#XP8sDpF)`0S57 z6%1MA_fKY1@6ZS!gE5_E)WTZ~(LBnHmvGLzFK=Zo?R`^uQ8*7w787HC?2Z<^YM$P* z&zAQ;7`sYqZ~0918ukRqn=U7`?w8H`ZMchmzY`MkWnMJpcEnUI0>oS^cW;9ph@KvU z1m7;7lCwAka5P)FR%0_!vb9`9Ugr!mfpxT2d+dage4n(rk$dbMx9pVDGeAfUlFr7%0{%A1-m;^8KYjy@)?gHS*8G!yE?W&@FRBdDS!dH_ zKZrJeNZs0axV7;Yp0&dU0D`NCLTQe}L%gT8nc}d`upFz$d>YcrLXzf}Cwo!#BdE%r{&J3;|xbSK*zsbKfk?yjIgHKgura zG8vnqrw9&weC>YyTx&Y4`g0CH0oSPAMpO@;Nl?IjLC7`NdNe0q^9j!Vk=|DI>mB_i z=%CSiDj44%i5mrZQs;~c|=Hduk<5bXt-l`0SO7{dd zUUb{ceG%hIMWanm_`?lYk;dZ6b~(d`Kj_*`V8DYI4O36(bW z+AX>7%+zSIm)prBD|z0sTH?Z2FPEQ|+AGidX2nf-Am!H+UuR-I;JoGFtTnxFZw-Eh z+3|`mPiFJ-gvWxXuVZWn}MWIv}ZCv{8@Oe-COH)BBi7^ct={+((hNlvy5*^)TAh*F5Fr z31M_L$2F*6Wy~Hz!9Eo$zjqNYKV2hUY=vURDuIBJkv_5-$1ET#Bd`iG@MV5>O~RB zS6tS)E%fIb{eh;A2`7TaE^+5I)5G115s){WaLF?SevvTep$C2)fTgTOTp?=3Vq-ld z9qASg-pbutmuPp0pBFmSF~e?S<|%~JGIoXqYE*h++QWw4h7H&fW{vA& z1=-F>z8m#1Z3VgC2>bYnKwEK0bdK?bz&7q6rtS_Y8;8IJkMwA_4)3thBrJ=E!8Yue zKQq9>)$MS!VCRaj+6CbDW<^FI@(VBFsu1&w9!p$@1t=Ye58qO>iOlW4w%bHSH0(G% zTj%rWYL{+X+}I?z=5dJjq~2iqEQ*GSv;@@cvprbntGDMWX3^S$We`75xCOnz$EDFN zQaiM~VL_{t+bG!OuETMR1%)4^`_);y`Fmkp!Q$xBJOWK3iBqdn?^zK!BxN3XXn+el z4iF@6h>;-wJSfvGo{^9!u%qNZ5gw`({@VTS@U)i-;fVo3$n3K&liQD7cV?*BxJiwr z<`mmk7&FOTzwCyQQ$Hjvx^_-PR)zMKS0`P0#3^+b7g{nHoH^*?Ud$xR^61);hVfI~_Y;orcZ_)QX&@Ys*wB?Fq8h z_#xSe1MN+wVlqZ8$hb-M;-_Usu%3NSehcxXVxP7<&DMAs+qe{ry@F9nGG27JYF~Rh zDh5D!TYY|{HOt+T*%Ki#eYSvSsU9GBh}wQn63xqL0IM1Puof|n>z zx>D9&P0B#SjlSb!jC;AJ3kf^0-wqr2TsZ=Q4mKx#6+FN4LZcn#RiqKzkEwwuUN|MZ zBJ81ICv!tq97j8%rp0h0)9%VmKqu~__u~0#6iv^s!|1>BcNI<7Ymb{wbnzBU->v&d z+;;htOk1=dkepqjr=@B^pXy{jFG@Im$0mcS>&CWV`>iW%*cIdeU_@8memWtc@!HUv8UcVoWUJ703?YX@3+9lX}YlxC0gA zJeo8LZWS@?d8ZL*r)awL`!w6vyWarpQ0`zEe1h%eSLj69LKt+&pCwl^LKlJ_WQNO# z+v$K+PKKh8DvEZMVWsJoXp{x$E%N=H?(ZR%Xq{5SI>TF%yL5m{GzY%bPp4~Uz$JzQ zBK8>Nt{gN!>3$!DO$o=D-S)ZAg@6ZhKMQ~WB zJWv6BiF_l<66H`MwSxgz2mkP@5#LD;#O)(#cW*p)1--H%q8F@C3!{YYqT5&LRx!-& zwS{_yziCo0SAN1HvW4K2ThF_C71|w$ch7bEu_H?~CHA6l-P!92-~l;^4Am9(e}oDJ z6lhQnQnc$0qs*|K^jiTCAXi|Tb?2$Q9SPZDk!T&Ug$#+ zP<3rb%M=Lc!>=S%J_~t5-Y(I_soMtW~E zBoHM$vhT_QY?%*!9n;%}>p*V}c;!EGf5E|1CKbp>?vC%rLKo$wh&w;x5Y<}@Wwty5 zfjNm|CT;(k9@+Q{YXYplL0Ey_+CAbOXA}W>j2|=bFJC0{t$VQ-+2|jSv!s14R1Fd}-nj_K(c<>?~P$RjmG|I%Eebg-yoxI=qG!qzGqyy^H(3cjh!5N_>Ng(= zI`NkSr^w#S41y;!{Ed>U&6qd%6SHsUif#nS5YJ^fywH~jl4qYOFeQ&1DxX9xIPVo< zQ7TXR5}n|mw1o};HNb)L(-F^4MK@=4=zQOA*2-?y>YISOyu0Gw)4m(6lV{;Jv`y5c zKKU%{7wH{Z19$$jffD`p($8uywD-C!a!N1eA#qR3>OEVyF+uaLNT){VE?OG)-`*@r z@x8796U+z2det4zJ|=_@tKYAvo%}RQJ(~_Msl5n*gKzaiiFcvBYkfD-+j_Dce^ZzQ zsQy&JuDF^rXhC>)MawL`nyfwp^VOC)qHL$8(Gz@DQ%dip4?3{gzrT zGQqlWCGl3-=`i5LKX?bEgx8<$l=yuJ5%ZxlyU_;yxAwEQ#;83{UCYRrjV7-&oCm zZro7=nnd;?pI&vja;{{MQXxJN%%4P5K9O2b-yqBjd$mYzh?m#I;GnnAW8F&3KVsM@ zz0^eTVLnK{d_0QnJDSr+&uT$$5ya{VJ{gIKmVS^pFz@mU?xAkt$J!FVS#YWC43RjX zzug`m46Cl0x+jS35`6XS8_Yy< zC;K{#7^&iBE3z5*)-L0Swuuq@Nb=eBTNUsqzP}XP_4o%IfFQD;t$!)(A+hrd=^5(D z^X#LQN<69fD0f3o?P$1fMwc&FkK$t$_t~|6%lj_3_tbG;vA&CcP7uI;Fg$W~q-U{A zs`ANsqp}U+6wv#{W5guj{|ybWos8wy?+Q;t-Apr@UZB{fmea4JsQ>kO-Vpvdk~%iS zXRHJ`3|Fk+w{$oID^`Y?*2KB-gTideO|b3bmvYceF}9AJ5Hu$~`YpM*xXe7fp~&qN z(_d6EUYM~NUr|vLAKR2KgKxU8H=K&7evPpmcFQhF`^9{NgKXP`qV87J{f7Pl21#Mg zi1rEId7)D5B7z)JU*Sn=g+~M#u4ivM4AnaRV_N~#fQ-gw{+A00WSxb5YI@EOO)`ga z(H9a4b=uc*U0X1?-L_|Su5hDY*XLJ;f$U}ypJIA1DUpXpk>@QX+x+D@ECHPuu!`5` zih99zG(7{pPdslRos5{_wma+Y=&oEF+_FwqRr^NXXHuO_CeJ8e@y^Fu&g*r(Tp5zK ziS0aDwnl~&*NtMDgX<8XTgzv?K=Rq^ViCLZ zx2{W;mgZe%_O^{AFU!=pt2~hc4?1+wx%)VA6QciARyDNT@4C6tNGyX)Yab_-;s)Mp zK+2Y+33w3VjLZ}>rJb-cZ%x>q{T0Ilo%9UIgmk5l&u~`X z3MLfR^yT)wr;Ztvi=5m~K08$ZJAZQDO-*?I<9YY?671rGa1HJ+t!}T!80oc}czr{D zsyUz|%?nV=n_Ts%10U(puI;}P7KWO-g2y+%gP+wGyr^R}?dao6@FVrK{E*}i!*~9K z`p>l0)s9Jt2kPYH2lO1IbB#RTq|X}v>r}!5_i|8?Po^0-?ORyf7eD zzagtHPmGN^(0rner~Z||q$Wh;33$&#s`H_g(*_%ICVds`=5P3>?ta}oz7SBFOHdf@ z2n~&>gXr!I7l}e!!;dP?!EGsT>H{xyj z+ga6|Y)rv-LF_C$EO3Y7eJBMQ!Km7fx;za*4keXuZl7JP$J+G{UrhAE^P}%9*Dg>` zQSw8;4Wd_0_hv{QDoLTi{J$`g|H2nvbEm$ubS8Lj=@;&mVe{AOVy%>x`~qp6 z^gdxzS|I|X|3Z~MATaVF;>)Cn(BN@nX#u1VnuDf`3Z{qA8CW9Ogk+_V4hRb8u~UIW z|AY!LYvij~-Wd4^e|=3kKIxx5hr^ma5ZgHg2@HwQ!{cZteL>Q995ce#i0RT>|4o=f z7wH&p7V8r!N)6nD-eJ%;a1SkD^Dgt6V8^W>r%BE1O`JoQ@i=n)A1d%3V|Ig|HJG16 zE>*Js5J?_8ZzuTDd@%(fzz2Q*J^{P0n~MoC#~weVClgY}H34Y*;j!3XZq&q=@Y5di zlld<@uuvM*>bVwsud_ym5tZC^o;eR)vt3oo;914_-<+mpi7tj_`)q{ zh;wm7-P@4tW5M_6o_IvTw)-3-#K2_!U2EQASV1=gQoh+=bYh6QP@!GNg2_aXw!wi& zvjLq^gzXf_EgsV#kiFa_HCq+e-@OxWhsdL8x7XPV9A04&Mpn>(QhPEt!@_3U|~2-i(l4j*x+}6)t_U_ zvj`o2?_9l^hUpYD)H-0u3y;2e_uR2xXbP2GHfVG4x6QxFrf!&uqecL zUmLVUEYe&{k$!r$8mysEZ9_kL$mPkQ3$iE|UfXg}lyt4v+r034AsF~ebfw*GI!s^y zx923pmZ-OX*9u+~vaXOYVB!1g&B1nfSl#mjF|`mavHHd>sz`GsM|yfy9r1byq^g_@ zy3u7_^J_lXV}|J)&6Cl*Wn;y*?{sFgY&2mz@xo$P$_t^nIRj+OFR`TUEr^BmK}bD> zK`G@=r3x*SY)Dm0)kRg|8miQoXpG7eqy0^#=|4`?v7^GeR8-Om=R&?HP}`AZm{_x6 zwZ+~P!#^cD?xAn-5H z0au->PA4r|zjV?ad$?YKFY1+VFD4DASGki1b=Ixwov z5Q0C)(k2n0+v*NJ3g%7>#{Rl;W-wx`%bm#Gf$<;AwW@E)QWCBa4;j0{SM*-T)9akE z@clL{AClWo_YkF?U%s6#7B zl26rfTOzRmGh!DcSDbbbVz6xHWlLldCO02hCZtJY+fwvOg-pI({hEygzmv#BnFkh{ z%a^y0gs2EDU9FgSRKfYF`Pp>emAdmD9Eu>5p+9mOmM8Mkq$NP9VcQl9Khx$XWB(E( zI)UQEen%vhBNW*H z6`GeJ6v+|@VS@Q5_7~0LkLV6ebOS21|IhmWkH7+$C_o?t*B_A+n8@Kv12S+J2{M)h zkyxBiBuXH}>mPOr8iXRjh>$b|*gASd^F(+E!3k10;x!b}ReZNSpGzAO0fLyu!IK~9lGGSS7T$JKyqq7uYJ<;nZg%iJtTErjOP(X(M zjI&j@SbvtDF+p4XcUVcU+QztFHG^lb>YrG7a(&Z-hVEE+OxfY0pIcaXdS|cX<@EbPA^Sco&eRd)1YT5%P_go+;*& zw{^>)#k4E;iNW08;DMX+aetjtUMSKuCfyUd2xV8wLMP}Irv;IB7gTl9KnNEcKd)w> zQx$rLGg#&{@=xJ6MceN>lQw!wppf36;t&Ow5~k?PxmyO(DUp#aO{v1Z_97PZQ;zKXCgu;96kc*L7V!7;L;|-1?)vLkaW!@?Dip$|n`FNC8 z>`~RHXl3@s7KQ*NkVdhShL*Q+n;9(iH9mrIW+by1&w(h%^R@^Kmt7>&Qqd7IuYm=a*qju%z#7roA+ z>~CKe9L9>qkFP#{((r2%C3|((T(;Rf-wchTrxA_|I@RYuojowAAl;Es?}n?+cl{=w zw=HX3*k1HkA^7~ss_$+CkF=D1=gq~PJ*oVeooUM_T06{R(wf>-b~iAKVU`BsnhKcH zqhamxe5T_(f#7+9vF9~K1TW;?zx{ z9d8}XSirCK3C8w)QjFu-fw4ubq7UDthicst=~FVy%KQp#s@GV=2GnpRc26#4Bf?_UDRk9}jOQ;86%#Fp8mtB3w1&Ozb2BORP0w ziF(vHjZo{w(nB1m7Fz65-mYe>GY>mg5!`5v&8&;i+*Y4kS-?xS7oS ztr>MF(W*n5dt##PjFVB3++G%2cGz|oZ3WQbEm|UB;w=AY4 z{$TrKBciRxdA7UoUCDwMEV01-`rts)1^xSP@d~Cq*83U}<{zd9li3!(twd$o$tDUi zuxu+N$x2l;ZDt~Z^22)n(Cza@Bdazx^eeGqL1PZv)KiZ1L8L3YKX5Z!=2Inla3z}S z)|&5>-?-hzIdxRU9kR5E+L|k3IGUE~v*(~fjTc83l#jw=-_!(rEBYR60@Lo zz}X`YJ1ov?VLf8IvNA38;uL2*&tO-d$j?eqWQfR%u4EuCOWF?x9(YQ|5;!h%7=i^kQW)cSg#iD4`1SZr)bt( zV;SqXR_a*V&UMhM$vV23I@eCq!ommfToS&}|9cZsUt!yQ@^;poYkH6e#RJS!g` zPr%Fu&ZDc&A&!$qzEL_<1&nH_az$(qaFQK(J~X~qsys5|{+sp>Dt2@gHm<9g$5qP$ zXICM(k>5=%^F^u8u#u1h=XdjYR{HIHv%yp33HwtEhOeVCz5j!=w+xFT`1(W#2_(T1 zf&_=)9^74ny9Rf6mw^!6J?H?z-QC^YHMqO`ZT|1O&+h%O`|Q0RIDPt5)v-F&O#hfF zu>8&4mDwAz4K)TnkSZedDWBchiqcrEKwN(gxca5Y6(u{{AsA6lBpVuNDpr+I+TAGd z=E0^@SEV_F*3@dk)|buCwW)8PHgHUiCW>ZWO_B#ECRF+WW$9UQoZZPsq_lBTA| z+dgElc0M$Q9`O2pcBzezG8#y;s~`9H+BEZ>Y1Il%<#P#5;ZgkTZ0^R*7YPr~IQO(3 zn?j(PrrluoWx2g_Hz0p$#B1|K znW=kXztFp+t5QAF*$4QY{B;n*EJ2QjgQ^vf(fp3T2j(I zGPSU_2%nSLVK|Oqm7(x}K58|$;q+Y~Q*i7nXc*WL=1ijMqA(JU4X>Fn)Y7uxwxTh$ zD7kHTr(PN{Lt|x8!N|jq^_*Pc~#{5 zu4_xzEQ1+d+igy+o;NUS7UGm_o)!VKR)Trb`orty6r$`=MSA@yb;auDJ6G35JDp2F zw7K)Gz~n)aExn4`3GU~rsDE2uqKjPFPk4VBrUexEW_c(#wsh8^Zg_-jq!^nVUCI#H z*}(43y#&e@uw~i>(ypOZF;0DF@mlqR4iauM57L%&PikC4bEL@l58ubbmo@x1njGyR`WNcx+j1L#>Dm+dcmb!)>#!8=WBtO8CN6}b^v&^ZGtD+S> zcL$G{f1;LgK;GQfN02VImQ5$?%R;OgdPjY<)R-aGlsY361}uvEgJt>SKb9M)!{Xa5eb#3Pc26_V}DF2d&FH} ztADqX=KCU-brLA3#wPO) z!X}f1OG`k9a1k!NDC7^0Zdd+2YEOy=;z4X4nYlh* zRpgdIVVdT5wyV1ha}z4o<0Xq}N}Az~CTccih>Vl29>v<^=QZ8;Qcf$zV3Srontz*? zDWMKIBRy{Zy;?ne?kS;{qQFlJHs+eegAX818BsQwNP8^(xK=#+DtBM@AO?3}bG{+| zj=p{0r;+>7X9TT~;sC{_2=4t5?`WX<9IDB6nr*9ahZYIVBRii7PR~8k;|oq;1m;E0 z;b!Q6Xr!v4|1IT3XIi3TuaV~|opNeOo91XzF-;XS$;v7PRoz|sQ3U2}g#S~= zqzyOwD~?nlVs?~{HpB2Q!_bNbk11s1dV;e?OGT-x$VY`$Hx1jIok&jUTS0auu7$*O zd5xjQvjM5rNYO}EvSF+{y1WujnYAWrc#i&vyc)duVztM6 zcGq3LiDS|LUz_6IXcPCF-(I|>nG;#qlp*y9b9znb)1 zqgI~}DJBMeja||s>$vpsHZ7Nj`id&1c6B#(yE)+=pLXV3`ulL=4Dc{5xN$!a*eFLh z6-T%>b3dkDoeMq}8~jPYe_6C#HRS#HsNN??F!q8QYUjTmrhC-nJ^OhuYPE(H#aoLZ zHPSnQz;RWSzy|$N#b|wHZ?nW`b&q2y4E&T)t9H^@6EO0d}kV-Y^g(-EUN-C zN6de@PNl=yynOVY(i<;%XI=l+mqS>ccAxWlaU5&(u8>@I{Y9eKVsLu-7_Zi#u*=H* z9#c;PHWSxb%es5H+#c?(iSWnF@-}O(Mw`+O8PT@ZZ3=r2z3Ofk^7&s7%dGyLQtYIq zXY!`Du0OT(Am#slwvW>b7aOggm6R+S>k5b-jB1ktqb!(A8MAzZ*GwS)gIr6OD~4-{ zCEhe&SibbrQtQasE`pQ5v0{b^ew)2Mk+aO+&c>Bg z^1+_x8Ml(qlJv^e9++vj*iiUFriTK5l&Ysq{i5Ac7rWN=SEg`iB$U~+Pe=r%>%`35 zN?l#dp4kC`7OuYw_4?YnU-14RpkP6qiA4Iqzn%ka?6kKxUJ6G_ob1x&jn%eMC{xQ+ z%u+t1Abgu9qk zX1kZZ?I?Gg=9zjUdQV^Z#wvSmZb3JvOPz-VFb!Z&!sapbZ!PCrUZSY3WNr{lSZ!+* zI+T1{@6koqO1*WPnA;?oGjRJI8N{ntmi7MV*}r+r9@?Pa_CXB)u?{%q=&1Y{iLGT! z@fY9P6|dL_&!T-V{4ZaB&8B$mx9GV;K8vtl?TzkA#Xrl9d&~4cdXjcx&Kr<;r@|SS z8}#Eq;PgEUgIQRCkKX4rEi8DFIpN7ljVvoFs;Wu`!he}CM$3W{#tMFZ4QEWo;x`$! zh1%Jj5Izpc+LU3p3+ecBig7eH3%yZdY{l3pd^{yOBg~6!p{bXjQec*2kY~9n z6@OM7GTC56_X}%xQ8K?*y8rP{Zk6(v<=lxn(=1@y4MbZ5KaI~q#b}{(uXA#0Xh>N+`m4!;#8PjqJRV6B49C1?!f8LXv{ZNF&tKQ{X0cb8i~o*KFd z$+6t7RdgMA?a^Ux_hhYHl{8+MY_u7`Uqh_5z1O`YuO)Jq$oUyC?{R8TFh?ZKl})+ouHeOE#_i!`^Pa5(}V9d2-tISHPO9hV7hF@Dm}DH(+OuwS1R<}zAuu~S1kEAcFeK*m-OPF zpQqV|O{m0LjN%^04LP|NqT6NkV4dXlQqI{!y$p*8fihDuN7JS><;I;+0`X&{Rl@C` ztcbiIPcyITvDb+kWVvf$uc@B3zf1~@C7lykDb$$rs<}TKIi+)8Ae1k1;g;ufvfDi- zKAgyCV83pi^t&NU;FvO-^1C8VZkL;-u~x|AUO!hcpY$Mb%=h4Z9VQv2ff7IGBGYn|SdsT3`OsLkjKcHZKmqW6B8`Pt zLS3o>4Ll!tS9GMx{%QZ!>UCSX+$=oL(u13YQgYGO3a12fl&_VmUT1zC z7{`yLQV#uj43`vl)D8T~LtRok8_=5-VLBnNLNSRJ5!iEPvA1U%kS%U*11*@q&{}Q! zi(g~PSO%i-nTncaE=BZ!#}IV0QE+AbxgPj_js6kC-Soq z+fXvsn&4~pI3H19(kEw7v+<;gnrrsj_aYg?BJE3&>qX)^1mUq%RQb6B7L*=%RUJLm zGPfFct5xJ^?|^(C3eN(Hr?B5;=adlD7FLs3pZ8nfV4?fveG6U{|IudsDQ|`w_gSal z(%&r)% z&!NYzlwG?I^$@nUa8{rCp~|mn>Yg*6SNO>aQ##(Fc5oRLH;PyO>G;b&XPFG}Fue)A z%51$NOAUG$tWj~ZehnR$_m?}x>q4R zwD8d;I9y_GX?@u$N~egpR2F!^F6YOdg8X~MYhWPj1kyC zj*i}k5s}2|u{z%wERrK)Y&m!!s?%Q~r~h8MxLzZpq`^~kpe#`5}|(^otN z_27M3>f`NrHYYvq>IGq;)%8i+O^D>a94hv~$)pYF^&q|q8~@5C#F6z!+eH9DB+D+* zG5wRBiT{}GBzii{#wirV5v0Y4b34W8VwF|vS=RU=w@#+*)p@(og+~2LtH=vUo!nHL zG>1T)d?Gld(|tvP=VnCWd`)J(%K8@<(iVU1O!@4TUg`V1BTKuB-)g6*)>3zuu|7+r zbo|^at#Ut2zSgM*4MF8BHHqj;G8(lps3G{I`ZZmG2Rfz|xrs!H zHbkl|mqlKKN_KNj=&6*3utj9rvZbEaDG%cuIXW((3p>j0caKZ(paXi#WCZSBg?;vwI2LE&|lV+G=Rhgfk2~QfZvx?jPKm%-@dr zNz=P^1m)qWE0S$KpGe?jb#pwSng&wY@xBVP(_4LgL}jw;hEAu8Fbib-RVG?}MlQem zxlly%o+a(l(72)S!bZs!Wgcrea>SoPagpA`&X1F=oucG}r!74~H6^!tP zo5rpQQ*}ud!@uKVutt5WXZQD{5|W}GG}S%28Oe?l_s`By^~`M-y^=69j5YBfnn~~= z+qO;nbT_)@yF<8mu5i8p4F?*Ws&&d~x?M(&lPUItL@mQy81ufG>;ab5>%P+FDwJ)ce*gqv5A`ejC&O15Ie0zxc{nYGH` zra_IXOYc|X&4=maD$Fd^{rjbe#=|IF{Af*gp@_ouDt2`y;$UxAuY*!Pmg}NmYMXJZKs>z3Tn`@!8M@l6+^E+;0 z68pP<`?ExJN62)yg=)`G<}>O1?S6&0;%}Ryk~8WORD)S>OQ>Fa8;muq(4M&rp3cEf z6J9g>H#UsBfAcYC3xDAdIvonE!tIY0{u?%mdf5>7V*>xm(s}40#2REQlK0o*%4R0J zw4V6>c+Hlh6yuiiUi8L#FxHOd67yTJ91%J zw_~xBPs5gBK`1uSlfQ<$)+1!SX>$^W4RoJR=_Sp~jG?l}cw;6*hNCGiL#I8-wT-+{ z2Eptb<_YS^k32#b%FK;en=$hf_f*(vp{>$-H2|{`svcEp_&J=6AJU<5sm#0>Vs1pK zF8r5X&J$TPwk&I6?O%vD)vI%Fng3s8xf^H37p;Hr72yge#!tNU+z=kJZLP4x_7~k+ z*+CC1FpSVEp^6&FjbJnH?^EUxJLRbfH@c$rS1zGJt?p0;%fKo+$}%!uwJ=T(v?lez z!=OI-RlOrS$};kZK++qvo7yG}jgq(1GM3-LDEp0@n zPd8~`k8r^|#Rj@HAUx^HFnA1G4W0LiHIFCwcQ{jfQ1vTk=GBa=>_$8sxIl{(T{I6y<01XvK?gj8yfJ^0e|EZMrU)HtIs# zx9`r>7*oOxGSc?zRdb$JVb$hR;A~gFR%=j;@<~2!ZIn0e;s8w>I(#?sg4dvN`X^CRnsh}B^ z$b{HK1Mu#C=&aAmvMr7+`+`7^*U~r;G4@z?+jY-mtpi9QX*|78vwu%^GfRA3R@KT z;>D^`#LA~f24~;6+nkF}4+dOKKL#Ha-QX5q(H=|UnkMh4uEvmW3gcAs&Qavn^3)^a z=5o$^D8S@6&iR&LY(C8Ab1~me)_)Exn@R4VuL>gs<_+fPdI+h{r1< z-xO9;de;J<$eN@EA1Cjc{>a)?EM+(Ih3^u|wT{$EHc+SOAb?x*Hg!1L(5~ zn-Mp6iP3fD@-7|wA^CTWL}_b5=S`3n(xIieQPnO2NY(P^%IEzj=6;>Z4+^WvqqiNe zJeH+#bvc*t-&?a)YPCU44tjsE*d!;V&88>c z6G!q%^o8@NO@}LiR%1F&#XJfrNVZF`JW)1=m>c;Aml*F@ z|K&|dE3Bq>jEUScnWeCF3NVAT;IFNclet48xq}z5%dP=0A7bf`O z=0(=_LmN!tQfvt(Bk;Llo)o6E8QikaqZf3@ZYGUg3%{Qi49#g~i3PHxnSEh1v_;V0 z!>dKKeVp`!9hT?ZC_Tu8YC@F3FH`Gnn%ol05rPV!gw@zc6fa=E3`SqeM=?|1zPvW=N8D{0dC1JOA9U&_q zZ#|*9ryhaCv^C-=qj&`#6$7wa68>}KASxeVSE zQ%D?};icIgWW=%Eb>YSGh4f+9nJV@IgXphUTgE-$+(=+%v(6;IC5+^Ct!h=v4=z>R zD+cx5jw>pxCXJwo+}}>{Qo5L2z6##oDtbp-f>|M;XBcMnL60g9YvdyxML)p3%d~Nu zzo$Rkz^wHCZTPv5KRcqQGI=6J%YzBO}~8Pv3Q#iGLnXUCx~yx{VpTzft0J^Mx(buyd~IjI<2Ysh~fXfpeQq zkBB+Gb?kz)wQr@Vx_CGYHq@t+bfltN&O-D2@0x1iY;BWlf?&y*i&M72wy%WC4&=*m zuTg&&VP}T41Nzndjr|!p%MKN|uPb^dkOW>cxgl~l64s~UA zBGt${Y*%lJm@<2yo2t@5eZ6nALQkr4@usiFrI=ePISQrqvn@SS7VL9+>7h#yIiBG{I0uR}-8&0H}hs9y6e-!qwoRYdy*TnXNDdFw|8O=P#Q3 z1b+B9B`^DVS529Hy+z;{3a>)BZ9U>yzPNaAw5$0)*C^~cWxw!oYF_j*uRiZp*d3m7 zuXtxtwTseS?q7Yo$^$hgAZAz-;KPG7TUtQz$p554q9ib>C!qRqbR{wvVcd2#(6<55 zfwdkSD3KHfcN2sGW2?#7t^jXK3R;SVqO%9Nyw}2a4tlQBa5v>c0@a4aLS6NH6J&f4 zAY>_r59cSuUB`03SL_3oGlDXt(~!r*1qh7}pnb~etq^r03&KIopMes=hk4&L_d{TC zdJWW>P|NUzJNMy(mbNsg9qAd=^H$8#c2i7BVPFf z!B=d8`ubLDlfQXwM1d;j#3elHz)@T=Fh>R*LW35kIY294U=?y(Q9jk4hUc#{T!TP5 zu%POKL{KLgSRsnB=XWcJ>y;)rX+kbNO}Ic>31ln4=tI?N#=Nn%x5Y&m1RmcM@C6GV z+@bK`N4x@pU*NsNHv!FEUs1$sAkhoKtVh)Tn}qvLEZ{#~ez$QsJq6;UkUe>2 zEP=mnyy|JgyKnG;%1B}M?odGZMpw|lwLQoXg!kG<_(J|Bo9%rN{NXMtU7PI-QciEN@0CA*55ac`k&a!K?Iax6kwC`zVj83Hw&0yXV))@F=;y@7ns9ue!dQF~8*5Q`kI5c-pnf*_NS! zjGjJIZm{B%0Q-ZWK(7*ykh`5>C+Ro!Z8O4>YettRrxyPx`|<~vvqme8&xPOe&&mT^ z4ppTCKaP3Ld$$#SpyGzscMg$tf>LtD?zVp}BoOe27dql^JHXFB3kwRrDsmR|?-MDW zFLTGC;^xrbdnb0?I6-M981I65b!UHditTQ)>&7fM!q3u_{|q;X<;vvSh?-BW6=&Hj zC_+L7H=V}6vY%y-q!cY&0$+%eZ;k1c;mawS~i&QcD4eqp;xkd2ZM*6Q6L$nzt=Iy503(Q_8)ouKM>aPu6>T!YRa00DCX zJk|%a}Q&f?GCCc@f#MSef^!WPH+I_}i$GjbE?o z>6H|yLpCP8Tem^<+U$7Gbp9D5u}z7WMT_a1HQK<%bFDtpHz${ZEWS8iLGY=XKh;tSOyJu>l@_P`~(E7Lez2vj?AY!Y+kmc8`R z7YI@GjgcS<1v{xw;59rhLb@{%KI$)aIICQX9o-Kk%M4+(-BVKco378Ys+LM(8lwmu zG5X+7JbgPqsvaQgcV5sAdzGYup{(U@UEqMpoy;W5QWkv-dxAftt({V@egw&%9l|x& zzf)age-cyqN4|^nhw^WGI+l}Oqfw6byILrNzbYS<4rU_0W50&IAA+~@F`9o^L*zjG zN$Z?nED=!hDuQY;DQs&G_l2F~pFE_nJ~7#zjDy1g?p2JPV^v#5=jIh&h+V<1`#*_rLMGL5uMUW5#Mxd0;~aB+;|u3DhI{Q@+5r)p4kP1sKUY&SFN-~m+tm9P z&f)?y{T_^DYdx48k6?xRWWK*$0#On1B)&URbc|;XSp38E%M|Kc^FaeH8&U)` zQoZ%fxj=iQbBy4AClsiEfd*)Rvv|grl1IdY77A2JHsKAj7Ah|zM2dLkIDli^dUj(25mHThBLIsd_Ehq!L zXob!nL_E$g9upu!n#W%kuX3>iw6J+taAb22l0Y2AQ7uF0erw4x&i!2 z9=-thwDt%VEKa4-(eD85Xapq66_8{^yh7v16rfMXDk~NzbL=(Xu2Po-5LH^|0YGOe zP65M(CVX$L5FSHD$34f|KERkdjTWOZnoywN(R;w0o^uFbnrzmZ(J?lPoYyg+FhHiiXjp;~05Y_3@j98qGFctvsNZd3DbRGFmLU9%(fs|Pr znEl6UbrPN)pch#M@Wd41@NEcY(+tqL#WD}j`?#tL70t zkWQPbT|i$;-X^4LJQGF9QyCHOLA*y5uqgYVo5isgAk%vetcJfW+kh_g>J4$g5_MPe zd%Jwqq63^GO0iA^e3rw0)_#j5l{fC~jtm_DiCn&N#tFq{7k}lPjvS473k=FJ`fViK z1o+2W)k10|fs5()|BBL&at`oJE?F1w_3H@WJ#(rGFbmi}*xsmgBru)Wk~VMknaW5) zED8+dMh7OBDl--cB3?cP$Z~Kskd5()tzi@(27^6hAjZClQ=l17_Vld+qU}ixfPwj` zy+Dd8G*aH`AtV7%nJj7n^peiw0w#YXI}hMTa&O;g<_nNg^MYwW?a#uu!j{DY1Upks z0F}w}cq{lHV}P=cpJUtpsRGxjPk_;I<{n^HevbmxJS|8K=%H1(2jX7b3j~HO?!V=p zVbKgQ9?SbyzNv2urzHFDn>NY3w+JDN0Pi%h4Roi=(gb)d!8SmdBbF0jOGBCfi2H$V z8wg25@@*xT<~F?1%(pe%Du(mcU-oUeJwUY>023I<+z$AU#Qe5;!yS`>L?KlV0z9pw zAHauSy**bn_l*FrQue6dsM#Mtvb0YE@NF4y>%1-dEs7Np7r~@t0JU4~=M4nEXXgI4?g-3$ zfYx2pW`JVmF9H%sJWw)Ms3L%s7Ku3^4EJ#Sx6*}s2C_Pi|M0d1HO*V8faa2&BA~21 zl-_EF;H?@Sf~bIDUJ}wkCT^zQ0hBlIEn5ef9>9a!y;khx&*lyZv7%>=4v#p4W53>B zRAQ!(b^x*);MVtNmw9er5tvJZHA9er-$m3VK-H7(1BX!tW_{!y7;O5X^h*z(@RiKu zvPpiG^{=(t82@el?6v{lcLE%!d%3N(4NH{Q561=ZTqO1MgOAxRWSO0)bSP|HNAYMS zja`)g5b|wJhSSfAE;?*GtQecRXyq_y5jcjR;-P;xnRFfejVQ#YT9&A)YQB$u!QHDP zp?v-D_sIg($>Y)~eLu!t=#lBZ1pJc~<)T**kM+O;Q$Qdeh`=NxdptkRG0MERfs*sl zS2yn;-2#Por9lHM)v_E8i>eG4mOrg5T5e2AG)&!A0KK7h4j;+`*MBewZ|Iq-7UfEu zGs(;n&=Cq~>(s@Y*0Evv_>n1t_#<=%aq1dU_&cFpfrj8X=-H1aN8zi=Ea~sjtuLNw z6=q6(@k6x@h8tS`df~p?!+h6*yI*_E2}(x*?bWWpjGG7$&JdC;%y%IYAJXp78K$;d>*8#+z(g>Yl^7hV;;ZyF^p%Ihv@C7tKBn~&8%lmF` zo-m5h>3coR{IxFLfOHxxzbY``Bh1y*n;eUO{yRf%df)UI$dwCtlRO5!U!r<@V}u9k zmHA6cT2m1)#{&=us}nQ-+6QP#MHA@XR@lzo_KKL$XL4_r-(~Q;y?TD|Devl7z08hC;yG8$iU38W6;fZ0=23`!0+#cux>=Xis5L zAN2r8;ogC;LU9KL7JrDi!_4dyXc+_9djTUjkpCNmxEIa1c>^H`{u_k2=OL`{*hTq* zKjhb;Vzvj2fPl8*-;84rR-o)fV&f0ZL0>EAP*}N9N6oV#0!T+0Ofa4Z_|v)CDBNRC zD}aIggfmoyh~vTfcCTjh<+{@7z)g?7p%%yd-mBya>fZQII7jxCghj585*=}bziZiX z^Aq^vDphFzCkSy2(%YOS2urv1CcNa?tB5{8!IA)DUd^jr%Ub4uG2AVyAMzlRU|1ej z?dMqGefVDje&K4baw-oW#tjT+`ToVVBD-5oKLq+uy6{JaYl3(8Z~S;KuQIh&CBcOv zds?7o$Qurk4wLbEM&hFZfA#^CGXSDRCL}oevuNmp3~$Q}!ut;n!3&xrg^_XqQA5>} zWBv1lKO+XE%fhMcg786ga9rIlnzTICI?y1^-T{!i;VVqW1z{BSe3uyEr--vKj>rNt z&`*2-CeQ#i@&J%j$-DQDlf7rxe;U8|puhkqK@~{vmXlIEX%T=4s>751F|a7@U5tJN z)n^i9guazCG5Nv{n6IMzENbEQ%-^ZpwKoC-dugIK33kEBL;@x?pvYkcp(COV@ zujMS#$nwuV1q`6em37kdSKQu(D7)DRuBNSWo~qCSxPrM_P19%$;XW~oR66{76x%R2 zvh_h8o%S2~&7Vu9GMKjC|IoO0cHJ!GqGu)}qT}C`(?1m|W_CIdBqVGo_GEMt6FU3o z)L>9?Bk2o06-vvkcYNX+E^~*a;wIKN@}8lVTethnwOaOo_}_p;&DdU?fNt4v0KQJd z3xd$;5w|QX6|&N@$N&a)9MhQ${DBz%bH50lZU!ewAws?tzm~f78EOqozU3#Cv+ayQ#OZs~Ya%%#{`iLcO{8hgU~I&LK{K&>q3-LL^lfrbIn{q7 z5}!b9B%nzlJDBlz?S3D897iJ&+ebjnsUYz-OC}Hs%$YJYp;l!d=n36NZ&S4JUID-)uV-A#Y*6#9kPpx^L^0g(;sEF6DE}1gh%D81EWn{5RKNSjyS^onY{shkSL5 z*sD&74fAP`FZ6R__23xUzu1x7F{SkgARjw2dctF+a&VwvS<@n49GA!hEJb%Dr6&?~ zA-rJ{9Rp{KZcY2Me+(5es_OUm`cAVkFL&;{<8f~HGFZuA$@*$ZHFj6l)Mtk+D0zCWpf|NRB649I=l*7G| z-u-K!E|NTF2^2OLkS|E78&&r~dgt*@R{slvT(?K|hFh^6N4)d2yD{7qW}G`!;StTR zkCEnHTyTO;luEAVLuVdFk>)T;uGt6dO*7}0FydZIvB}^}fs%WeE7yyIIcgVA&?<^{ zcUQnV)S$0a75}PYpR=(#{15Fx-RT`OB77PMAyV%hMf;-R`_+N2hs^~=CW8m0I)VeX z6}nv^qnGzWP?oDij2^Dbs-o4U^E*DG|G za@Xv++98huvM2B>%dShvTqq3$CQ@y&&h>n^2I4oG1EHZBTW?kVj?N z6IGyV333x!gJRJBEvD@GZtcLZRqP@O;0GBZah^w@SXxnpOat<=Lm#v3!|2<};DU`ac(g|GD7IDzerK@uBTr#LWx)(hBp|egghfx@GXc+%Sh)CEgGisld7NTn#zi z30}DOuV0U=^l>4V1)jL1Zl1aK|GO66j`ePKBS*06q!^fO#z%}Twq|n5$laiLkwzv1 zHGTAFO7Q#8Ug``>jX*qSF4+M~Q3FH9{|GjQhK=5R@sJH{YMEMBeTC2KVqJ5bzb8mX z9(&B|T4~AcT1nqOEArX5O_y{STHJ>qe@yk%;;bR$eBAeS6f*oDYF-PpS(l|H?NE2^M+bcMopf@9BLQ z@(*H$VYQ&aVY?v_^8{96su5vletTr}e*NeeYT#sSb@qH6-@L7tEX2}7XZ{c|Hr=p( zCgI^&-~4bHyzIh+gCI9IL2eqHg-b=Fnvbby6XdS(d%mKQ!jZUfr?tXAZ9{*bbX(@l zQfz`)ebxiFg%nmaqCW-ZHX`}E_ILlLSmY`0pESxx71r0A%<|um`)f*sEPEZt(wQPX zXR;+M`%aX!`%BgO^n@|=Actj^MbD2rh}}8?<^YdZRNXGh&CSy%V#UJ2xcxAclX|FD zUXq#X5GG#e0Ks+Pzu|!ok5{iUk)#j3YZi?~l}@VLPi)0R+OrZD(urYS(J92OSgwyz zT-_Iq^;N_w%d?Ae&87=6`ptzNo@NvBBBVVf$Kh6K(w{TUPk){}QHVjLD6CDxT}UeF zEiGxZ8TRwX<-hdxeUqkLzjIahuJ9Bq(e&}=u#_h*46f^@Gov!^;Nps&|8@Ia)_Tv< zDm2o@iVq4iEW%aMLW!cgQ&4Rp?>t_T7;l4F89Mpf$i~ms3l6`lwqe~! z#bYQtkA1_Y*GBl`;@#t)FTJsR&#Rezc;G3!K1@ERez6UZruT zLQZ)5?i1&GZ71ShCe|p%q+>O>9?)$`5yM@tJ#Kx~_|`59m&eU0$6+YPM<~aPZuUbB z1PKa-XDd!>AJkOUe>C6Q-1u6fOIC-FYKVvy+2-B9nzy9hhHiT8BidxDo@Vr!1UcBg zwnYDpFV+@2ZB6|7x=ExlEB1^Mu;(&)cyYxC&nPf(P;=#b{gv`XMKc&&y@?(_cDv}F zM8MucEhOp{b#hX7H~9#kjR}{X2%8N-iz=<7zVcY4?zFN}ROuzI`gWdR8ap%5NQD}r z8Je7xhGZ3ep+?g@nT!#vT0`HJ$r!CMgQ1RUh6$r@&W>+g6w;H$U}0U&I=PHt7ABVL zXjjp@A)HYZ+HE4bggN+B6YXCz;e1Itw{2xL(VH9~@{P}OB)EpoEWD1+VOY7l+s&X} zdUmcLfYJANqpO;!d~K|nb+(LEXu#%2YAV8>f|BT3G7EPxE!Q8y>QX9FEh!zv_NMOA zrVaTrB-5Wxg4AlFYlGN31GW2tSlNwKtNpn+vwasPZj4GQTutVeVB_XLYh!#6Sxj+s zb2?YEc8A!3w z(#IzEN0%A+!zp!hJ&L7f` z!>08g4~?31vPQeCZ!h-e zGNRJ>79RSAXERB)^WRG0^lp@)vlNQ&1sT`(WzQP{}~V5QuiY1e*cRi<$H#P8utbJ z;g0c1S45c3J-Fi&Lsgc#S95F}hWMOBr@J@@NvY#Kd-;r+WnM8fEi5x@t4Q`t@5LiG ze{_F)A|*8y1;&D{y_jHs1M?o{GUUFrt8=l0uo^+hzj*e(B%*2MsODj{PIP8$y%G7^ z!?q#2Sq##VFn7KCM#4rWXyl^So%Etdasv5XAN>>jg{(^O&gHn9$D6&I-l^2~3PT6n z#EtG4f3ENDw-;(l=PGeexbMtmAliaG|2<(QBM?FAVz(X=d|0|f86T~ksch_6NKvG^ zI*k^ZKNNAAESLU=Gq*rQuWqLm@%o!^Lx;>y`D9;A%s2@K%Z+F=s|mPWeX7odLgk0X zjkd6ROwt{DLm9^cTm?k~eY}`_txjsE+)zi`Jv=aG4+F;M>=HQ#O;vlQrX}zqUjDGqK?65pcF~uYV71k+u`0E^TnEg+8is zjWcCMMiFyE&co23&j&~H-6ra3FkTi$@IB^7S|ql3&OpZ9?;RXcj>uxrr#HcW2IOCp z*M72)Cx~7gu$Rm!=mCHR^vnr~0Cl5OeJO3@&*3uvQCx=|issNIDwhvI=z%U)A0)58 zN!><}`byGaF}mLta1n(q5k!*_^-^)}d(KUDNuSX{MjB3Z@Z-oac&+)F5)z#T1P7VN zc#@Z}3KP;FGYl(S#KmieRiF?Jcnp#KwPP2+HIm531`CF?f}~-0Cs&aTt_XEC zto!~V${u1SG~a}@MO@A5D_B}5HF8CLe|7`An3hsf<3+{ch-lPcP7JTwk67emR|ykw zy8a*1-YGb<=xg_ltxm`6*tR>i?R0EgZ)~&Uq+_RJqhs5)ZN2Z#fA3T0t9@$MzBu2! zn6)le&4sEp=Nj`F${T2sZv-H*JD*1V8bcz4PSF3!Uh+X!L>lKV;~-SXQ)7(=wY4 ztk1S0bAH$slRH;cIJ*;sELUbDNzbKDEGFmz2X=uDsJV98%1& zemIbPr(;C=N*U$afmzcEsOV}?(t6`Ln2>PoE=@lGgX3sv7GEn=sGEK6{7trnBJVy= ze|2Bl;xW*F8Jn1C4vJcf6@IHfy`q0;cYSKJQ@OGy;=cO!EHJA8D&FRLdX#T-BD;c? zldZQ()I!@$V>%%!)6|dU``CGx#>qCbr>CH6-^yHO)L7iBp+$FDQY5sOA} zzo4W>@18kWEiBo{Mwu4NiWaW(H^vsVC6v~@nRDRZ-_(L76qb5T|780ARsN=3_CJf? zHmmM3*hGt))G=M4#h}^&?IFpP3W%v1i<*ZX9ftBE@c)P3r0mqQ$C5@;N%{__R5KKGw0Sj-3=AvDUv&zq0%rm8sE;_}c|znW}tt z+(#7-UY~vr3q~o2!7f@?+&K+%&}m(7fgY+Lnlp6&S0(ivy7dr|bXhS?T0+$yiNAOk zXfQeberjh5xZ zceL0UtG=96#$KC2i>9Af(Y5gr%d52J3Hri3h~fLRJ>&ri=z)Tlr;%qY0(oBXx@zr9 zE`~tmw0p~?#pGvhLqfaI*;m-geNWNw=^%{j+h5g6#77N)B8oh|jz{9$D}l2a3GiW> zy=%IL&(kfFg48mh2JaT)Lf%1Y2l4jj)+vkWZ6KTPE6+2L8&0?&nKUq>LNhao%IR8p z|AA_dis|MsufIzYWF~gJJnQ?H34r}f1iQZV0j%uh>Haux=p5dtEeAQL8b04wYi|__ zF|%ynLb%m@g3QvMSDrYYmf!C7JqC~d&T$!i?jOU__PEcdWsA7En)dX3ygED_ZkOCS zyobF4&c=IG54SUt39i$*-tRmrKQ_0SL4IlT&%7YUR6zPzEkN?mBX5uWGH2dr@!2Jd z>^slhZ?N~bm)}ev?^M7i*954M*Wp8QATRu0Kxcy?{Z{*(UbZ+pqT!ck%5TUmUt+2rU)NNuaY`s$graNRXzy}XyFYp{cXFu6+AK;o?8+mun)@1)QObqa6 zbI-_Qu(souHdGwkR@S&aSCk0&?9$EI=QG>~bqhBCoK{Tg8;aLVR$IdYG2FmfX5{YG zz;>RNU%72%ZnjCWmm%{OeGXK)D&cPV(y zyKJ7vb9LqU_}!K~d(`4@JwR@p2Ywkl;2(FLLTB={@+eCq>06@{1!hbbaPPY|rq%wz+3$HPvXW1I&Q3He7%%y$qEk3MXD za!Z#yUk%e{2o-weQ~&2%1NdnzbP*unZ3b2V=tNg*vfyKYlyv%8ecA2mwfXJv<#?Ps zZAG(JGJfZ~skeFUc1MkIqz}e@1LcoTDkQ&Thf|O1Q^z*xv~51%hu-QVDECP;^$O&7 zLG4_F`NpP$obRmSxqY3eNKq=t=eK`@6#5Kee5Rt^1ic@~hj8qEUA5Jz<=;C%vs1VD ztB>(emJxdQuk9RhAG>}^C06$YUH|D&YX?0qZnq>a-%}x6gZ%D@GI9^z=5Xw75dTx| zJZ%1p-kxJul_H=94LR83mc;s^=9)VMl3-8QK)%R}o=4U(50fOZ>cwS<_wP z>S05Prq)#Mt2C8-7LT8dcv)3h)yW9J>N~HgC!xDJJ>n^Y|7!xz+%KREANg}OkPv@) zdyhw$dZCpx1}|h?M^n6$5zbRhL+E__)|KQoYfh=GpimTf7KA^I(CwZD%-bqRS0wfM zY^QmJtWXlOX8^9})^Klw@c+Y34%4zZW&8;Sw$S;1voXNxewcyw)WfrFpmmtvF$oebM79bb34on1o1d(IN^!G~>7+YT~ zxKo!Kjair7Jy;)I!J3Gk&j2koBP-O5G3}Zdpe5E6@DrQMFY-83$1pT_z3A3g-ugSP zJNSC;8*&DyQ`fEpXMH@jJFcJq(?4Gse0+Alp3?P*z>T7dxHwu1k3=S`Gx|0Z%4rLv zAkVOZ|8Fgj35W6XIau*lvO7fRPMN0rXs@yvs(|bb1di5DXsFIA5;eEGx|Jvy$R>mg z;-)U(1&>0%Cyv0iHTQ#TNpj7xozXC;#YDa@`Xgo%yg3{6rZ5XqOL1--7+o(DECl2$ zo<76EekAh0O)2?B^XTN(0|aKZ$kEnR^8x#6hNY3_2@TTpLO9;+XlLpW1>KHBeso8P z4croqi}1*l7-I`9wdt$Z7Lp)O}Bz+jp}U1bqr3f{(A zJ_0ou{LIKvZ>oTSM8^^C=boGJEpwv-ZS85y8`*4TE)dKhW;e5pXsMwG0bcu5l^-sCZ@%w ze!@<$cE`!$d;qP)D)~JA+aav@?fQzO40jZ-+ur?_v^x>LBwwi((@7)te-fL57_qO< z>f45!d`0oNbxCJD9sJMg)IYwOQT}|T!R=6OTjyCU%XCr9cAbG6^C98=Mu1LjvjJV; zz^=Tjg7;$dDFdU!M^NQ?r^)IsQ3KuU<+dZ&Vh>H8G-Od|`F#JXgL4W~?1zf;p1=l+ z0>T;P+U1sgl$FjiNrTBf`HkliDBtU%b?Yuk3E1PEBU`?XseSa5&?W92`3-jki#Dl> zxob+E0KsR-E4wR{nX`uqV$O;@PU~X%wN-r2FZi^g(Qe5=6cvdw=u&;#US(UJD%{Cz z6ThgpGS{VadklRoXFO-u4_Cq*ODm7=hV{av1gmpx#0*U-i^-(G8AsE2ix79!m%)o; z+Kk`%Sm9apO5eViPGI&}YbgM&t^cKC>RJ8%o^8GHO=DSoq*U`_E?(u+u|q}yQa+Nx zUhl+Sv9_$=%DL8|rr@v_{CFsA8=Gh|3N{EyDz=A z9M3I6>M#q4TcgpaMm;HEV9MHzJ(XxZst8hH)GE(Jc5lqba=sRjz4SLtKuRMa``TBe zS%mI>#RKW4-|LL^(2yGv7@Q3pWZMx6t!+r@DM0JN>;~?oiR|l$4s*yn>M@nUDumq5 zQOU;X^q7!b_J8v__)e;+BNcbfR8<5deVN`F*M$Q4yj0Kz^&n}kM(sHCswWdqXA4o{ z-dp8`ZO5?fVf+7orU*hTdJp^A`M0eCBc|7HoqT9#kCHQKl1k_TdwNQ>P=C6$Y~fUK zl7eIsnzo-UZ5^od^Et9+YP104AJos+oUS6)wwjzx?6huU)(?>}ru0|mr<%-#+so0s z-k!2tY54COB9gVHd-J?MwRgXhywN(qQeb-BqlJG zuf@E6^!aGgS?Y;6Q=^IYq>dQbk6^Lu z#Jxf)z167c>UMY|zeR`0(xKJcTZ%J{#4_0B5Xx+7OouI1BS%ens&8Ga7A$x*d+Eq= z0Fu|UHs$5Lf5Fncp`msacQuU`E;Ko;oZU83yFGi@Z=9@FEO0d2+EU0}kKkfn?+V6h z?gYy9GC;TNuHQ1iwzc*o1O)-ytSunZ(R)gE*a)7 zJWl zpFL~3MxVq6@~#&uc{ zb0C!#&`a=`%siH1pHmG!@mJquv$3uW4!p8Np2wF!SVwPoZeJw`Wr>h8S>-G5KAyW& z9bHYuv{%*O-1Sqv9;nI+u^fb1W&Z@)T`bmpPL@=!teDS5Pml;-iz}nQmH!ewZ_N6b zq@E-|jKZFBailSjE(lh*&2&bQ7_3HiHX1kR-Vg~d*84E-C})f)zHQUOD}m(%Wwp%k zu|9W6S#TXgO8wj!kK6tZ0t~!^xU?fdLIhL2b~LruLdV0CzMry|v##C1j@yhI3OZ@T z*B818D}166e%>*{mkBx5E1XzciZDZvtKg?P${rKARhokn{C0f#=L;XfKsd1qQD^t5 z?{l6D8u87s9It0DPPGa+$or}t7vPOA|0Y!#bS)3W8VIT z$@ncR2)Tjf4Q$npXa*xkIerBnMLp^T9DIu|YnNw#mIZZ61TABG%wzK!no#`nCzp9u zZ#5%;#jVerlP1fs1vd5Hh~61QaH+CvZkd<1xbIkGq@T;}Giv=HpZ9|KJv*C4q4M%Mg@y)gfO2%mFHR+gK3wf5e@w-P$p-XoqTM+h~IgWx&bp5d$b-~+i`6~7)RWN z)`_0s)Mg^r^Nl#c2aapTY&f!eC+V>_j6oEUb$&kMpG;gZxEGI{ zDDnnW!DCrVFk((hzv-*eb3!%$riX-cH%Y0FuaxUWgoOqD#OVEI=UWn22<;fMVd&Q`m~NSL9v3iIhvm1Mw0UF z_l?qdS)(70_~GkN8!C?Y22l>%9*J4|SgJfg3P(*XdC`wfxLeMyU6p*Sy272qxIjiM1z z?49C7fKaFxQ0!HeZATI%gtp3ck{b!=9G7>vx)F$)+Ar~;)9ljy(bY)s@d4RLaC8ne z84Q$VM{TYy@_#nsBAyJE&n@u!vSL5_MgOeQ+7?{RfiUceyj@RjV6Wa&ko=}p<|VftWgOv7?`8o|1EUH`#*$^9#;0I z4jwLStpDu*^1nv1{$=lC#b|14d&!6B`h7Wdx*3UeMGKtI2?4y1cR{{a3QszXespC7 zg5V>OVs&Vj%+x2w)O0ja!boLBlzD3h$=2$hyL2X}Qw}?i{uE=DD5TZB6El1f1%7zR z07K(rvpp@Ag=lh0D}L0|78jZJ9TUr{i?=%$@9|Vt+jaN5&-0dtWE$O-fS3v#33`B3 zNJ6sM3r6xZFa~Tw`=OBMz;i7tT9Gofr)_2T3F^=e9JCZrslIh`_xh3D8JG$yBG@)e z#Z+BCuKOu(>n|}>rI$F&<+uQq|3!e?~A@O&E4(*Jx*U%fc`Rc zo1p6luW4Y*>5E+=5RT9;SAy#&8(j-hCLcW@sAow7b)aJSSCUlF<^utgg`d;XCJ3X zy^%>x&<3>XWuW(j0r+Px#=|SY`cz-*HHvaUXwcRzF|(hnfBxuT8=Q7B#hO1|Z;}{q zf^IfbLL&4?zf74!ggt@2gZ~d4OY%R}x?&lj-)|7>41~}GLxfi;=P+XnbvZ&K4gE`Q zcV%5D1b9$L`I6hW6im@t4oq34NlS|k2>kz;JU~92E=jqU2aPC<8da=KIZRAemmjum zRPj>&>)dXl(=>lUwphx72{g2%1kF354B+N{3zPJR0i1x1lr=9)Ox@%52UOT!(N3yM zQlZ7eEebo-K$w>E`LxmdqktxVMPy+a51NtXe6uHUAo2COV%brgy4&mvmpiYH8F?PnHn;ycIa zW*PO(sYv4_!byv8&_tWpeAt_+GixBAhy?^jx@^$z5*@1pLPsn5_;2dQOHs-0y$Sb|>69Y#LkjP+s$m z@p8+8EI{X9n}wxwqdY7ELZSvJLvRwbIPlIaf*6rW4*Sn>GFR9X<2_)Yz3I`sG%Vb^ zsbXD|rELuqgSveY&H{A~;B>_H&iizCs~i3j6-87z z91;(8wtQI*P%Tm1ulB>6LELrEtzA(Zm0j6IT$dBw<9f z?x&OqJv(%MR?yPE?%(VSLab;stgP4HgnN|ZmFKqZbyj_rEE;keG27U(kZ{Rxp~aL1 ztw(W>7UUg5Bza<>9B5MkEImG*q$#EYz9?lcq|r2TkGQ0f4aA)w-NE}yX0vKhVZ>L- zgc=I%AKH`&*)4a3QljP9cl;EIaO8s!qw61J%8bW_b2>#3k!}dWtbqdpp+DMw&M_hc zJqA^A3xAywI11hN+$+tfBp}#E&z1c^x_CQ!>1?7(rIeE;LfUK%q3%f=IY*r;#t|x} zog!0{DPruDGFFx;vX?6*mahZSMqdx;og5^;!fWwchDs7G+;flJ>=uG2R~$6G#7kYg z`^)Tw&S)H>KuS%DmW$6c(&i98Z;3kg zp@f5AFyr;*Nv}tjWJ?px0hH*dBYkkStd+sg#8i;4xEcuT#Otn ztHWqlWC@dr=z`<)ctE@ct{`n483>}h@s@7tEqr|Y3!NV#HExQ-HtaAnY{W#jV%PQr=Ejqu%Kcdo(EmMS5) z1{2PbALU11U?hB4f@mBb9E!Z$Pw|n&og18>TrziG@O858fXCPtSkzNjyc1R+?WOXO zIO#V*vWg2vUk~(5;x`2}ko&#m-9oJ)R?^Fpne-7G%D`<-$YsT}F&%EdJ4~L_h+vI6 zAH9lf;T{_DCB(xDtjF}8A?kKAg7XjL*BG+>6ueciNPUd$c0&2gc8V``?*fiNyAd}^ zJn_WAXJPl`;>*(oG_GZ#IlR>%vp>irc{{Ufk^5WV$*exETW)Q*Th16uFL5XH3R0xy z745ENP7v-Dd-xuZY-C}bCO4Nk^}CD%x?t`~6ka6J>v{L@UfHrF@7v5dN~Dq0bK7tp2YPWm_Y6FQ_Qq_rjs?tSG2;CEhsr2h7WVnLz2rW4B{M1&$VP|qM zpcND)OJ#dczXgTQg$;aMW1g+{4m={M3(&7P>NM`UPDsWVB3lfr7b*u?B<^frKn*A4 z1B`>PmED=pJK0#ktJf?Njf|>@nUsZ|;+Q=DbTt{LzKGPY z1^o;WnHN(2A@h3SK{*zuy$MO0Rmj5G$&)kPO76@OjHU(NQy34KxX=323z(mP5_J%VnJ z8e{Bm;stNYVg%|U^FYisErLk(c6nlFI>t>a=%eKy(Z0cOO~c6i5wS%k z8-lD!zc3_Yih89xNxyUu?iS0d6ocI|=_=JN@+&QSB-7ryiC;W+bH^}p`jkz9T1kti zi{Y>ex&h*G{QHf*7#5IrBPa2|A7tYwM0|Cgu{G)UWx!2bQ)BM^nD#J3;8#Il`dQ7A z6lgMh^*~-mLrsZHE2cRlVU)Z2iX!E($L7pz>@MSp9L_YB=6W8VW%MS4GVk+>thDVI z)-g$_AS}Kf2)6b6G7&MSt3XIE!)quh4u8E1`b0O0OB7z(MjOKbmf;e!kguwDM8yco z_S-$3FSgv`GUTS5(N$k>XNVf4BfP9rjcjnET#-Q!p8z?u^lFEZf`fJcH8pmt6C9I7 za-gIHIeBGlNKvA#m}J>tx&wI0PT+(B%?7)h>iQZ5;e{LN3e`#)PqFf{iG*7WxeKn5 z3SU&`hv)_a@)b%g4@@N~_F12PPFKb|Q4w3KSt8;GoP%^Yy!Nua)i5l2|JB1!w_%xo zei);Iam^D4jFJeX*zNRW5PxjUi_FE0@Y{Yc_q8GV4#IUZpU{~WsC9{t5nfj$xTzPJ z1hYCHW=K)%f7A;biS&oUpvg$f2t|-kA3J2r3}bozgcSUMCaQFlyU8kI zUNyrt-a4sh-6~GhQGRplCl+NDK+Z{M{Hn}i<@_5~sb@VLd!Xd+!E4&NtFEA1vWeztY zu;BzRN(Mf|>2|x9pI%0r9&YchaKJ<2&@f|KH_VGBZ%LiSvKpl)CrEf{xM3H?Jz{66 zKTX76@}CtBa#}$hf$4LOw73a4;Wua4WGDss?-RmGx0tNmlmfmeON$N=!^%mpVfO(Z z@qh5LQ~n~=u6lWPkzv>3@fgS>?yUL#@L?cijd5)y3r8u}c5PLa2tPb!<_E?u7~^;% zo+{5P&lLvW=8LbXMos4gFOz+<9IgH>EM0_8<+GqMQNiC@A}w{Dv0$b_($oI8*VPTB zH_0Bps|%_n@gf$7mAF&zI|Kix-yM`T%K}RfNz|wpPT|AY5|-5qT223FgaqG=ctZ&h z7BW#Ik`Ag)Sk(z&G8%%(W+gXh)`bQgFG??;ga{s4g0Yv54f$=Ls|$k>+U@P<}!=% zYM-jg@W=3&1DaBb8XK}w4T6!$%w0c92Y%n3LGpf(Ih;TqeIPa@J38Ab9PO%pl*`{r zCe;xo`BbwU+8#wF(fimPMZTLeAD^S7$++;=+T~4OD;zJE-~$Py_LpT<{eQi%Gux<3 zI^^z({FrzU@usT#{?fGJIQ}boLy?KBzQq26Evv~y3xNs@8Kc-N#0&Yuuc;iiw0(v7 z1^s#hto}xS=WpCRI&%xmgH9ZGC9_E}o62vNo58l)Fl@iMZI<8U_E3gSP33fus|&#@ zAJ={G>^#Z8? zV^?fYzUkK+If#(~^_7D$`Zr^5=thfC1|4`8V;0WpF@Fd{aOI zXFxbuE`%r14t*~Zcn$bOKzau>*f`h~cq?`b<&Eh)XNmUekzV@zwXMfEwcG@Un9te0<9h1K%Ic@| zwfcwjEh#B!>NQR^6&_l9wUim^38GS4POy7Mql=mDpv@V{KIIxbbb=5=X4IDhQXm=+2 z0*7>L2&S4SQi~7OJOLZG2%2Ykw(cbe~s8f_zl}Bu$cdTlt8xZ-Br^)P?H>dFaV!5MjsdDF*9V1kzL|z!y zM|Uz0ujWUwqSI|~r!NSQC2y={&?Qt_%dEc zxiQxnNTk}-RMe-6RKW^7H|b-nJqmj!S}S4sY5Sz|UIWrn^W1(+^JzE3c3(I2t|YjP z4aK(!vi;HeclHwA(-#;koL*brx80q{Yu+|J=+)eS&!cHXH*SZKo5mHwzg0qw!ar4( z2CKz%j!=)^l?(IB%wJbkw3F8HK;fm&1#{TnDMk?XE<{HQsu1}{ln_@~%^hPP;~PI3 z7vR>O^c^f;pO4l7tOrUB%~=2?9qG|C&lnvWNG#yNK5C;?!@9863saFf>2El5Z8q*} zM-nIDnNXFsqOCB%x8hzyo>^MNY1PDq;W}KajcILf4tZcXm${OOfcKu2tnm(6%~%tJ z45P2R$}1x0-pJ;f0@Iqd93|)`v>AJ~_K)X-9)<{VLFy9X~fyM?`pPb>m<)uE*Drchb)89DkkL zk=0ozJ%lt6AZmH8!MWr5{<+>;`EwbLqwn}2{{{kN-ZOt}Z5BLbSsnUY&MC|{|BmyH zAs?pWLy*?hr#{&jyNFv{4(-Cq!E~LwG^?E<-<|WVk3o<%TQBS*A=j?HQ8e|@&TP~L zy(8UhedO>(yP>CP64S^;tVnYYELlKlQOl-x8;%8myZyf4Cs_TDsE>@IA*vB^_QXAlKan6dOy_hRjVUvhS7Ah5@wnn9? z$m}LtFL>JC6=0>3%+`dtLw5gIL=B?nRgKh^#6~Ljn_wRlGKu+u3D(bU8tW;9XW|C= zoLNTUIXY(<7D6n0yf&n699kZzTHxOQnuFC40^pDqHBgv6`C<%7Oc8%F1=tFbgk$}* z3T-n6J7-{qb+lw8(F7mi_k^*8Jb}f5qMjl>kPcv24Y>g4hH^v#1N8eKML41vck^DC)4OQAj(ygY7!&fhHFspsL~u*uf1{#^qX?g; zAA+8om~Nw7>*U5n2$iv`sc6&`_9+k1_=e|E>a0F_wuqIbs;$x1eimI9YJYN+jwsNe zVp#z!U5TKm*K%X4s+(6usNTZXkUf!0jNLE!J&=PmD0BJ6F>pGsEj- z1^m@v={k(O6#IKtk}j$El`=Pcpx--5{oRJwULV}@O;x%h;xvrosePW2hD-H} z0e~a!_xk!}zR^#!R{pw0<=zl17@*+$8V?7QTCQfhg{Ds zIgk`_;%+yFdKJ>)nCjwp@*Q4Q8yL~>AePr$V4aH55%4mXNr)M}}uu*foUIfHwu4n6I-IGcpH)v=CnhULRtW8{H< zH@Zd)#AqJHyRi+&Zmzav=EN1UvkEWv{77wB*RI`&)|^Mztea@r?1@FDPYma$h<%~7 z)S0`ugIT2zAHqzgC&KJDc0L`#{Q3?w2V$^tvsO5lmIvPO3Y!O*^&(Hsm3fq}zc8(J zDf=zZB@}!&vV#=PAOCtAi%8E}K6Dxk5*?a((neY&{c5K*4=Pg-QfUoapg@J#$d9lx z5V7{$Gk9%EU^tEEjm+5AH{N((%Ld^N=0@X@gg)eaz6XtVahw_641-Biq&ELw=t58H z^OHAHo5-dj4A9PB|1Qt_!+UI>p=-jMo>W!L22AKEL3LBwblUS2LrtJ| z#vgC-#{!u_c4kH9Yu1)ZkBa!p_XX^QZ;!T&boNuDuTe^K>PKzZtV?vh3+pFfg%KqkY9uF$fvF zyG}%A5C62$rB^prZM3qs-t~@^YIfJ%U8e4)=G6=U(uVS288t9yX)!0Gx}qHDa@F_? zU$6(A1`(suF&$|9YhD;$lBq3d*;2K~)>9Ev=~D0xX%8jLvhIAwG*e1bh17v^Joz7~ zcP(S_ecN2U`>PeTWmx6T>N=^y>Q(h+XKMKBjOq>6;npo@zZM(n$gV2{l_rg@j5Jsj zN+MT>pfcRexf;@WCiusE(@%f-k9@k|I|04re>_eNWosfZp7f%a%|nP6fQE#h?^%|6e#5@ z_tEyydH%k+-VVO$;CkK12&j5jOaGIH?9(NcS87;e@j&@$?bqa=+$~Hzb3Jpt3#b@QklLlWO2fOlrWM1pHdm;e;*h1|43zDBmI|Dribi`KiEXH znzsK!E36Db3Y>b<sbn#WG>pWwpFqo*%*hR2WR^zsbtTR*Wi)6&zc)4#IGak&K>FE&ZqOg#WL zCnWib27R)PHYe~R0-`+>YB+DmyM7jB@f+{qL_gT+ZyhKWb{`*hfTsFc)o^b5d67>T z2J6(zv4!aR3Tv(*Y5!u266V25eVyA>LyOMYLl#~__AoYuV>RBn?Q2v=e`FMG&5sNG z_*|pUM-o{SL(to242DxLp^53px+7*IY1|<=B6VL722_k@jYHXIfjs~zl0;-rxl_VX z3gpdv({&xmTxSW^sAz+nm7~hqS;xX8*mfJ?zv&h<(fIb836E*f#HNlk%FPfH?+HlN zk19mQXGFx`L-WryUZx$?)A&kf`udR*zb`&Fk%kb4;g0xJAVd7b+qe`UC+-nFNw}+v zg#CSzBoVAPiR60H@Ebf&Fb@~)K^$LBLAA1cvN7<~XS+8_Xn7k>0AHxdI!s}8kS*qg zmd>Br-|AY3k$d@TOH+?~oU$8dwF3Ipn3t}i2h1)%>|0*s6=P5@lzOS9`g1Si9RAIq zPDUTBG@<=Fqt$yc5g+6{D3BQyr+Vm@gl}YBe1((6w-SJsmgzSOtpWYFoL;Lrq%Pa< zc_zQ0+RK`o2}qQ_&)i=+W==*yYwQbmy!fm|)4Tj+aPeQ6Y-e!3Yf)( zp?_1!dZ~xkZ9~+3&BZi2obAq3XGS^!ml!ejfu}6qJ*Gwi5#RvjDAgUwn*Hf!q1N(X z#3NZ-y{ghIBl%{=kb2Ua7NuXjaBAiI$Skp*S@#qpG>lHeoZ$!D9YY{)Ix< zOsW(G-P0#SnAVBRw_tstw1dz0fTV#QgMfD?N9cnZ$Y$50bsM8lVW=PFuj4-RXF1S2 zagt4?NtbxO2F!+wtVH9039mwpPs94VMnyR7qTIw%q7lo?^}?OYti#z88v_V`Wm+uv z%a_j(bGpvGHJHd`ff2Zh!WgeE=dXi7JJlS4^+~DWixLzp&rh)E?>);?Ic6k1sW#60 z$Bf_eWAkDC&0{`(;sTcQRvjM&RC?1{W7t62fGg=Q;Y(5<%&kfp^4e(b`v^fMh#&n? znkA*qC5DEUg==FBBaEuof$}gAUz?^Y0r{*5s6y3iuiV%z#=sOX=Df`+@VMZ*@^T2&Sq>NfL> zo#+Xh4Q<9FDZqXf_L4Nh!fC`4D_+6pNQ4&$y)k|`_-X!czPzSvr*m&aalY)59A0t* zYY%A>&PjU)0iaM;A%%7iJ}1ksIGMjE>g8XA%W5CLfE*ZKnL$00ED(wR4nF%`%wY@O zt3;g56(H&dWFfk1&NT!!YI9_E&x0pp~XjS+V^PvT7zFJq=oT?rjom07}AH)i0)t~e~F92fPqx-Qdo8-qd&vwd*h35MK7c$_gn+P zGLWfWM&$%*rVJKj+K8lDH0ogG+%?1E@>`$|St*bJ&8J1S0|@>c8}p8kmRvwwsz4=T zZlN$5es7V2g-SNEm}PQ^s_q%;j3TyYiW^Cmwg`pZ0PvH_yiT;%ggs4JYxJgB;oTEn z;k@wcJwzohbg5Y$q|wvX;znB!N7CpEQ#D>Ad$T4~QH+XC=h>cAW84-9t}=P3h*cOe zmPClsz-^jFs4jFqm;5-R6y+EH##Qpe@gf_IMe5roO+@3Usc$gp3WnH{X@nVua$=B@ zd>FUB2%0!eM9e5!PiX`bHWk@jYJ`iY2<1ckWaG1xbUnaS{`Ixerf>6U(UTtPt)c{k z>|7AUK);HNz2hmufJI)&1K~e~R8|QFhP&vAS2JHzw=>z)J|VK5g>;0e5|R2dH;_>9 zP*r#2PPv5aV&2sKF^arfCaOz~+m$?&*PW?hBtjkubKZ@oA!+c29`5lWV%eq5%HrxDMlHO>&Fsg~kApE)fE9%^ znnnVP7LNwyXXujLnEZlSeNHr@)-CwuKCD=%BB;ezfT3$RIRr%$K5t6H@LkF(p?JZ9 zJaV+ixVsjO?AMzm=8fqn&L1$T<+|X<)uf#-X>8M7Ub%HE{$?)X}Luo5=_m15PL*nXyXkS?(_%i9n<(wxXPx~HVdC3& z2%n5@*L5-1-1zmm{L9~eF3mS2C>XNT=Ikhoi%y-SalziN$dhhzUC*JC9Cfi8KWi8n z6@r8>gWa5Tx2t|jRq`kDWYl+aEy{Bz=3n!<$@tZZvG`elUvPv%6?fOAt6i(gl)m{d z9%+7y7jaeg;^^d|!SdtJgG!T+#`s%Su7DdY=XM}$s7p2c=NE!@8#M{#L|OD;h`8I| z-w7BF@77x#{4^N08T@A6qOvjB_p5QgyWKK)Pu>4CxD+#Zx?GFwLK-IA zW7AL{6KUl7^>L)*^_fv3qYa3|N(PU`qO)Zd>VxrGfMGM{93yMYS5`Kx)l9> zH0M?H2m^U+P6ie+X3r)x8cA`jm+M8>cJG5FW10}b=3ExllsgR z0=sf)ty0QS*f;BXo1~-jEvDjEZ@B4Y?8Vzx<2_&`o3KuM7~a| zMYq=jX5Zi8pSu2YyWL#Qs|RuD(OHp_9xdjxjCY?2^|=fYu$=1(q#e{^8?Ea zz}q!0mwf>&OUD7?{5<)&u<8V4=JYIR!w%DU;}#W92=_#hMJ74%SVm6!F-O3?&B|Z- z4zXsLBZ8W0*8%U{Y+sc6Z(65H2g~`Vk$lqZlZdrLEA!9~^0ZPLZuqAH)6#eb;V>O9 z(9xwl{~r7j%qqR}wnwU7HOaH1tiN$i;?bdRnTzo`Ueoj{)vb2C823i~Vfv}qAf?!n z(A){qrucjDkF-({wQ{w>X?I)u)qQj~-PHXE%lD>8d z!||%dRc6QdqoKV7OlWLeb5@dPu6$@>SHG7zoYfROQfu+IY*&>3X)P~qSctWdOYqyh z?BBoE&IQ7)V87>gN$;FtzH!ep3VvGRdF7%=HcFudrwcOR~ z>+K5c8=Q`)-AjM_2QhQopP@GFn<2mCM*m%6v0(X9vqvQkKn*n}V;Jy5M+q`X^Xb6N zPeraF(qdrQXdY=P!0`zKUhsVsJG02sMI=N?@`iq=)+f*Rl6XSbymu=#Io@e7l-ZB| zVLPrz3yJqu6H;TFh-EjHiyA>EfL4l`<(=Vek^Cp}UcBUtX|JZ+OGOH${;n20nT!w@j?(t8-c6|%zkX)+M+7~JidrtTuz9l=1>ixoplY1{ zqjp(#ZeZARC2ERvWl9+leW9_>j`z=Y6p5x-yc>!~>)yn4(nJZRK2PQ^awZyL>y57Q zC0^@kWrlj(Gt6p=1C7^>Ou?=)J%X`~nr;xwvOsL=;H?w00~EwYB>PU$UmOigI2T=ucmKaghaWUy?!mqX`*d|o!5z(t19w@!b$G8V6qoT zE)BeQWc8WxdptUXi{{_#A5$gtM$gXwtSMR~WR4evA-I0;HNe{xUZ0bKR0!`=4p)qq zK>g`go+3KG%oaQd&8>y9bO3>y6^O0pxZQa2t;E3rOZ5>36u5_|17ROCke0a*Z81!7 z@O^KD>?yWR*&vys2tWLpm8c~1f3WrrK(cjB+GyLhdD^yZ+qP}nwr$%ujnlSm+c?eJ z?|gG3{+Ju{{WB4_Dq>Yd)n2t%ttT^gYG+FD>Era;!hS_)u)oY@GjDo|Bpj#i#l7Ge z@|0TM&s})1`ur)ztfXqT3|(hhfo&`wg@Qj5yo=OsJ@??%Vm2xHj;-Mwu#%qpIcf2t z<`c{}X(~Z1IddPY!4TqT%rai2sLg)=C|f^&zE_0L-*6f3Kr7onq=VQ%4YGVl8qROx zI}A(eSQ^f_X+Ut32ga_pz9lPpRhTf6>Tn-~<8UZx{W({U(Dp8O zjvNCE6w$OAX;CBUGSf|C9pTw02@eY#uikpHj-5(mK01DJh~IrMt5SqaXiZ?3Z=(j; zsRBC^d^`n_9fgYce^fC220q>n~X#QeB-fSts}S>NAoNHuBZS;9U@TZD5PB%XD`dC8~vByS1AOeH&)Q%U4>pPWrXh1J4T zzc{rDEs$bXDS8w-{S-R!Q%`k}dUOlBh3is(<{upxo_#4sX#N_PWR@z#mw3ZVIt7g2 zrJVEn^`;Zyt5qlvp-ZD6TG)crqg=?Bc&gK&SLOTEaAtV+Qi+UnZ>Q@(`bX%jJj_u1 zkioj}l^Dq;;rt>CXBkXP$*O3mCDo>d@&+46)k>O6q-W4lqP>lUvnGaymCQFTWI1#4 zBIT3f8u?9WG3g8C1@bc^>+-o&6n&$0c}PMBLHs&V^_er;7f$eDZ{8;3P2 z%EED#m6e54=JL;#b=J~ebH>qevg^p<6&zJ_i@J^tYgCH`&|i8|bGD%V{B5+hvaLhs zwHg|lQn0G3?Kxw&7VWkDX34LW{Xe1Rwl19*N<|?tG0-b)Eu2bN_B~5`=8nJaHn29N zsqy$KT>53GEUX%Anpp$#s@rtWfIhKv8n;W2(OSp4x0i&=DjSw+yG3YB#gHe4tEyvW zo7OBH1jO2NT9&zVNSDtnUHfLM8}tB@JewB!NmtMVR!1PEJ!g$E@t2wukO}ZEbY->}|sqSB5}sfthqM^V(9K zpl5Fc;bdlZuX-C~IEx>y4%oEq)80LadfGv1ABb_IA8rP6Ae73qZ{wWq%a0yBCqZ+c z$PeV=vBl!%`s?ii_dabtqxI9+kB+z?=OaK6ZFhBm?>K{j5Y9q7EgD)xeBBk4klc8i z_Zpn8zeKwIt{Q3>a?%j93O^9S&Bdd;c>?DYwDnUn)7JvWOx~e!Z*#;A%-WfR#C^0C zFZk1a#C8%k>;wmHdGjf{3nJLe|t^u5}ZO5O|7A%6i9Pww-CA_T|6UnSF z7gvC2CLk*xl@@3(Ni!!s+w^_vV^i1qN8?AT4V#$dTlX2O8H1IXmCFxn1{-e^ipT8) zHNGQBswi>|86!p)cL}JZhd_1ZvzHYJp4Qb9@s($$#T-K}%yF0Jfj>Ru`|1u|8Y~TH zH_ML-L3jCcI?WD@TQ|=#+^EkXnp2I7is;DqiHZ;ul^5QQR_}f6g67Z2u99rSmE;JV z0#y%^VHNZ85*O&E3#`R74`70)DG#<1n|r-9QvPb+{{~9l|Ji60o$hHsm%pWBC*K#|;3PQTr+;N# zG=a&OL%p~-dMh??Zo`0hr;p)7L6N2eM>FOIAQgL(Hb~PC*<(1BpL*@f#UeDYudk2Y zAF)-0V2sEZYNb^0cfGa(bD%>uvpQhwuPNS2xYMXCCnc(a6nX+)pt2V`0eOn!XHn4j zn>#QKsqmztW{3FO9|Yn|Dn09F(ni4)mahWxdjU-hY?pD7s=4v~XWu^`O=C9-ai+mm zlZTF)-Q8m!76{irwgAH=pkfX*n<;3HQR_QT8=S#Ut|c!w0w#O;!;_MA>|B^NHH<_w z(q_VD>6khNFa+()9y@&wzxTY8=O&(>!CD;6{r57m9t{#YL>SYm9PID!|Fqf-7czzWLP5+=5=*Dv>5Jq+eLkUF$Y z)yPLZy0>)Ws=~~t4qD*OgZf?Naix;%jbXioL)X{afD(*UAq z&F69L|8DhYkCN0iKzwpN{#jCDQ6zWt6)|$M|M>uW(PWr}wCT(;V zx|X$%$HT)3dUCzD_3gqWr49Zf{*0s_v9mjVwf&fU<-r-l<4@aY7~$;6rBUR(i#9B7 zCdM{zxp;hbEJcWp8&o^BO@T)9k(1*l{m@*WF=k$Pawt$1EojAL zyaZ;Sm=0sXiY&lbmLWe#2U=iH_E!G`GphX7-Z<*2%bjNC%^$D0?ugaFY8oL9dxfmUMHa84tnS$e;vdKSF#BBDdd4L zEG3k@YfuWQk2fa94 zTAP}>Hco9So0un=lP$8t?vsd;jfxazyRWQA!mvLSZb412Dacl%rSCzQ;GN9;L zRx+SCS+X*qJmz(a`3qs2zh6C?IYPqtPLr-QH!lebk52G2_sIG0hHi^ z93$W5JVx`Z;q*$pdf$i4FJHfxFD?$gkNY@o(MJ9343Kty(%aaf*D)of2aXMw0F1iW)_!|Y5 zDK`nkL5gPwd0mN|%@R_j2!(-v>&Ap6aWf-mIsJ_$_@KYiC#Z1pj3DFt~6eakX? z3CX)R{+XiM^B)<%Kr^FCIdF^2R*x@1_W2T1OAo- zwF=KH{|1l@zV{}Oe9U7CCz%K?;|Duf>%CVbomH#*-8OU>duX3jjgj` z)wmVges(?Ezj*(BjpxVL|1Eaim!si!wDaT!&-AnT`Xk>!|HM5d*-K}1aDL*Rvi~mb ziT`teoue6@sf&||lbwsBkqMo-ft&MBJs3Ji7h7iw8<_eP#UGXJ|dc-o?S}_e5??XqA8kIEA2AyiyAWm zBRX{Ei+~Sn=6xO`{11N?_e`$S1xK0P%friGcz`JPgRt-uDAo9QdNRTaf?#wQRg-A) zL+>%7W#ecD*)a^WZd#*22hHJM2e6n*(M*h`n0Djo2Hhk^sdrS6mHwk($RAfk` z-nDd=95Zr07pKwwt~|VZRb1MN`ZrfqSoj3_j){s+RA`1Kepndk>FrDxQz&H~C4~Bq zc5yX3TBl~I>hM#0r!h0=eCk7-TK zH02l)IlhxU8drffb(<|)tFG42a+(eq_WDsS{F2ycz^(!Uvvse>S}VF7PLN4@gY7y^ zk1>s*^cb8ZB!f0#^Ei^U9kx(SY`bz#(Ugb5YqDt41+Fyyg}D_wz~TrC*X zsn^@t=Db$Dg3Q*_f1!eG74=9GeA=Y69ID|5aF;%DKr7eIeRiKbu|PZ1&OO17VK`A3 zN%WB}hT&W&A%ol`Wg+kiGy~e5Cjl9b^wv%%e#fxsc96EWxBRRdK38Ynk@rW6jX=bx$m}-fgG^_Dc$o|DA)RFO@q?r-Nsk$_Yx=jq9_3q zMqyDNx|TWHy$y{shKC-sj;O9>!N%wW%_G)df{eF_QC4x`VS@vdfw3JKu>u6Rf}m}a z@Y1ZZ5}LUjK_INVt&PqZC z*4BmwMppmgr5F_*Cu9>8-&TptI+_qcXfz@M!37x-luDo)aSHs_Va0XD0&jfGM zMK?AAPEJln&LVA(lNz?@903Zf@zH1|U5?`ui!XiJr07;G@}<8Svh!_7`#oL?e9NcDQd&UNmiJowv#og>E*j^ zvp8n!&u!Fkt<`ZY)t^|ZMKzB7bsBq&uoA~wsH9z_uI#{iII@hL&BkR3d9)m8+O!=URXwM>ucy*3n}v=qL8*9*(V{e% z+)^%G>*&%fB!<;b)(}%XmT41C*A6?AG~yo2LX5aw`Lf%#h8m^NfFvUDsnoZ35AJqT zVIbOuBTLz|T+x`VC0zNeHZ|Q(mB*ZHU!8`lvYB(bkmZi+c{n~MCo-q{YT4sM>p%1z z-y9g9csvjJL(?-vzNbXe zBXl@j4?uD04C%lxBoKu}7F=AM2WJzI#0#S=Na+%EalH=5eFK{N00VRbf`H6!Lt|I$ zrAK(3&)B0F91|AnU0B4(C|w!sI5^RMXdRr4g;~O?b6KM^G^VO77$r06XS-e%9c`4B zpT}kbt|Hx!*3mE}Zh;}2?+HcX@ii13c1wGQaGf9u#xq(Ldm;s(nopdsR{g;snT>-0vro@d z;0h;Zubhvp3}h~mkdTo~G6bR&{*6B%oG#$s9&8RTsy5rqIf`+_fm&SW+Kl&07apk& zJmOtISgNA=*CszrwrfW3Kr;;{6$C~FWuT}WDj~6gNK?Wa>K-MLs8{`XBA1}bpY@;} zezx^}mD4r{l7?RjWD4&R{R0N@_!LL%d58yinu~ivi>%}|S(;Vzu8JGw&90VN$!Kgy z@Ux;z1i-^`v&g~XUiHL}09jS5_rWm}A6~#v(Ohzrb&YBw*26%pMSspGvZ2E|+AnoM z(d}`W-0?NM0kHA!55=0jqpegr=Rt&@Ex= zRx))TGErwvnutq{zmK+ru8fJ*3^841y01ufBrSu#<=@BP?pnQ|nXVwj?nxPV0$_F& z9F$CDxT$n`;~&~N57CtgdT{_@7h=sIDXR#Ig)JHpF$N^8G*nO|BXCc?S0q>O>^eH; zW@>J?Qbh*ZOAL2>|ENntt+t_PKf?A9^4|$u+W&#T{YPi{KeQJm8@UB}6yNLR8XXEd zU@&O}J^5BbWfbp?U-2AB`}+`EsnUG$j1%N%oyAsnVXtDwzYLkNc;AbnPjs<>5-?5^ zA1*sit~kotet&y^gX%-pA;;I}8zOc_H%E_9CaUirM+73KWXpN{X$)kDrGiq#$)mtH zdDZR*89`^%Wh@`M?=-l?uw!e(a6F1#nSTkaUAh9%U6_BFlKTpsViX-Dvq#30*|3VQX?3e}1vaWHnf&S$PDGqZDDb{VO7A1R-so;aon$7 z*0C0VU?aj(K^%C8zytZrI9W_2r1lUBdTHb?u~GaM@MqKnCE?!)M(tBp&^ri2GrO`F zzXQYBudoA>-$Pd^e&r%`B-^J|yP}E0UPJTQ#9TJN))qYiA8vjuLhg%wOQhY!Ta9Fg zkMRqnUS=fDFhlxj@)58kIzEDoGav<)CldGsnllKzfntV3Gm?+SQ;*i)C#g4!r8N>h z&6+VA@;qY4xWfmD3SAqlvV(DP>Pck=2c~Yxea|;%a%5U#LaURVCCFRCJpkc8Y3q-}RwXt0_n*YTnyONuAajhi2Z3Qny}2VT?`ZRt~o}-DY@RHdkJJKfj)l z`_NjA;_FL^!n&iQqXQ!`tDPPM1r$pT!Nja7(~or3;*Y;dqG`ZdG^-YO@UU(y-(f=D z`Yz5!`e(7=fa|^#+-sVS6yR#mv&LbL zcf3`2nEuf4^MBB=+T>1&SSRmk=5)Z>TDIrp4-N0^clfniWF}s%`(_yIg)X;z0IIx` zvC3dGUFKT71&B%S;mDDf)H2c1$}FmG=IeF#95Y79!v<8YYhO*e*d`@erXReJ6z!qv zG9B7VRTD=pSrcIwIi5v2FF$QHogR_r zSDqtgT!$++Vmd0hJ0*$}Rut);)eBd#t0tVDUe<@&r z5BTocnHHAdmaUIg)aS5YBKQo-yEVtcurbcJ8BQvhU_b%Nkuz&vT?fy!*R;c31w@og zyZRS!zb0IXlVDK3XeMr}gfilIL=oqj5Q#}rFP)i^eILH=^gVmvKnaZxVX9G3C5IH@$kI5@uck;#zaK9F)gN! z#}!3DjCL-hgwafyGdePeQOWJ3Q^g;VE4B!z_(I4J4jkJN3q-6siWUaaM2;leEV!FN zkjjg|K2;;F4sEmjBQ@RHq`w3T3VsdZtM)(QS@tQza}4=E%IemxL~h}b_SRI zJSG&nW;mKQ#znQl5hD{!N8j>16Cj5n`ATJMAKt^-eNOug)PYhQrzS6k)(OM)WHqVPMe4Q^XdmcQ{fkYA0Q5ES-u_g}|5->q6yGP3K z?4PbxETMyIo2m7}CE@8u9Lc8T#e77C3!d!{wRRFu?$e@VLQE*7>n~fqw-!0zOjo5@ zFKshL%mXtC@1VU?ui7o&@IMgx#K_)r57nqE{sR&B{|^xThiw0UIY;Uri2TdAwLui*y%CI<@ettF-Jgwtvh;5e z`RTI98LLT3tXg$o{HB8V`tVJ%hX_LB?N6QY^vp_`^mg#`0PJB+z~OM%op-|p#*v2^ z$yKas64D`RgP%P{LKWZHU?STz!By2UPdVX4iXlM3F!wynr4blMJp%AXF%TjJJj;Xa zFtk;zdQ=hKyfHAD1yDRk^@h4|Eyf1LwKzntC6$5G+$)o=CNb0~jpKhWIHGk&av93o ze)qa3h(4pzY8DETegD+b1KdU}h)YE(BNG%!e(+{lk!KgR=~`fmHfUQjU&73;3EeP{ zAc>hB$2z2tpU*-%ZO+5*oO%Zfieh<4|IQRJNO*7H6O0S>?QGmML_rm`l9-j44^^UA zrz##>C!n(EBNQ`TYv{Nf6|}uRO%U%}mOeJfx;C13CD%R$6e1s>y&OUlgna&&F@M8^ zK&$uzmd3vY>wj<+{^2JX6K8Wf7*ImwR4`CQZok>E&_T4>9j?ZOqp=W! zim+8nk_*(em=@-4Eor?0@Wcmkwq&S2lycYNC=M<1i!1hSf)M8jqD--3uW>2bs|YhZ|xr7=KEwrSH{iTj;Kl zKXOS>8)XR#zUh4Jdy4*5FwFK$_Lb?ZwaJ?cIAC7oeZq7c$ZVAaZZ}Ow*1kYPhbL38 z^t^>%kSc;89bDU>+m*H$izo9ya7%9;s-MHSeW6sPp$YR*n4GR#{yP(je@mVmo6R(% zO=jH5ms@4|mplggO)hcQx&|wIMGo_Mcmehdq7=2EgfAYuD&-XUl;yfMC`vP>;N!&w z=FOBGX6m9Q7lpZNtHEP%f}A1q8iXh5vBIloc0VvG?>jO8|=ofB;w^kZ>J|sQyt=ed~aJh+lUIba(X8uHX#y z@E)#c!MGa@Mk6v?tTtE81rbIovV`#a{IFD(NL@0>o9z|{q_)Up*HjS;%r+0@8^v;6 z-x(RWR|A3$?{^L_+Z`vlUU=Ug+CBHr&~!+1F(2Z=qNg(^uTc;_LA{s-U1u&Ky(RN) z#3+SqmJJVSj2|g5el*N}$OXSkZpa^XQhLz~*v{S&EsK69?SWH%_I!NFm-xw_;j6rn zrT7-l$EmzUcz;z0e&ZBiK6Yxqq)vW`n0`|><&E7srF_Yh_{Gio&H9bIe3NW^*UWEF z_7u^Jm=pHYGE!+5IRbd$BoqD;QU?7#+B)2NQOhw5ELtrl6|CLmz&MY>b#$VNR{g`!(?iz{6auzeli zB0aY)^oGcju-r5l0l4uKqL|1C)Zj6qrx1govO8{+b{T8Voq0KaOg3PUq2X%s^achW zw8$t)cSlE~nyj{{Gq$wu`A7`eaF{yRhFXz6XK51kBV;mG*Lo^wCKNnM^z`hq&1ih) z1-l9|&8=u*GlM0AL%A2_RjW@F-Ou-3aJ29@|p?4W*Q;NKm zT7rMU^1%8kmKf5VGhsb{6=W~RQiz%sMPi%d)`0{}EIQ8}zqt=Mcf`>o9Z(6f#ILc0 zavp9MCZSKMT`d?9wYmuWrpaXU$aTlrkJ0j2yp z-J2mbdWE1WktI}%kf~BotEu@G))<VSy%G9`Otlu#F=hrVYfDSAwNN!VSeWkO>?8QPNEz9iwVZ3HfhM5RonT1UrvQ!+<|ENKqgsZ+#chw$m5P%f9ZSQ7jQ z6#oAD=bmk)RKH1FYxb!AP<`_?{DN6T9cxyrBL+4(UhHOGAS`3<4CBpL2=GL>GfQf!sZzgGfs^G8wL_oEZsBz0bj74q z=Kwc0_^)8AP+KB6mttp)oAIV6s?0GX?y-F7fq1FWX!B9k02d41p$=BeJVT7HkaSro z53teto||@3l{<2zOFLxcmm(&iUzezv0oWUgk7g0sDHUUP<(wEH^Cv4$(VUp^bMh$H z!}+aAW)G*Dcvp)rX3f?W&!-i#x%VWoDA4>b zl=*PQ?-Xyy16`rr}O?&+r7z z6gX#(Y0*fB-JbwS3_Lw^o@)s;nKnG7wdEra{`@O_8OwC|7%&^1PUT$MSxRCK6>`O} z#%3Rf!wbYtO#_K07~EOvu!qJ*#7N^&(cWRn#Pql@vrR9KdCYiCa}SgV#?w_K3ajD$x!sftwtNpoc%Yl7b*)WEPR^QW@J z%yu2Erd^tU;V+1ZMbVy0J~Od6=sb3JpK5s2HdL>)DUv6mmy=pz4DN!VexJZn*X0v- zCJa~RI?}Wc-3%OSYh2znpSEMbWF{`{NnSE%dhPKIdjP0sC76;}$_nzWUNF%|IyHZ8Y0B;=4-%~F~nV5{(b z-cLuM5_-i}LM@;RUgA8re5_fe3MrUt(KuiV_)J&Y45=tMH}pqlXZhQ;njEE(q;^oY zSRyd%je3B`*{zW5O|M5Y%7kw8l&<3I6;r&UYKV~@N@ARVC zH$KJkd|PpAC~2{H`7cNuOdVmkw(y2>?XUq_0Kbuq?phy|7gUS=N0{V?otR|uY2ao| z)7DQo<_;*D!cKkieKDM-zV9J=yLS?96;8JGZkyF_wxDU&D=iveHLdBo?i^{1NuMhn zUVZxx-Vw@!UUQJKi}XKCGVKEdgXV5ADCmavPwaIzI4WaW@`^@ZW0)M!=VY_7L1uL7 zzwt69z0N#4p|{sh#1Wl3-V`VAO41J`2$}A*0I>4q9xtncM4!(MMJlNhT|3j{xJFR4 zS^Amog~f(`^%j?>!o5yc4c!<$O2PT$80k_n+5L?7lkXGxqIt&|dw}*RnI>})Cl-nc z=TVFGglI8n7WWQNVnt;}a~Itn#K%hAf0ZZ8M6x28Kb!4xI1r*v5omd&W0JwJn(8HE zHU0Np3EUB>9e$G~htkoeT9pXh2TQP{5oCG>n8L))J)N9|5>0W-HAOCJ7q2;@a;b&j zX2k8$ze(TR<^!u2V$N~Q8Vg;TRK`rZn4<4ZJuEs_9w*aXIM)+Q4#blK5ibIttn^FE z&}&Hy(>J?sCOEKmM#FooJaV!NUJgyqcORf|1Ak1$fzHfzJ7h}>OhyW!v#wZer^+YWT?smk_-Y~RD z+0>3xx$mAM&h?{kKxN!K+dQD1Ry-u4iEJ6#K*Wj~F^`bir?#uqrhQ69FL$apPRZ)W zT02jjhn>*9cUnfEm%P2Ae$}Vto|D7}ljIY`p6i64O}gD4L7XgmF4A$llj5F2_kk5LnXJ+`p%wN>nIYulG8!3-!P@Mvqbria6{O^&< z^l#^o2RY9ZH$lOOxzt1#HTUVLZOYw;-M!2=#ce(EDi#`FU+aLFAvD&@4>U$|;-^ zWK{*4@)|k^Y|ziZ;tDCW*X^8KPN9gI(bK}zlvbf@X-UxRpjar3oHraYgFE(vwSIhW zo?U+)ySP!6i|M$Kdlx8^uQ@~g>>J+Y=8#B0&cl|u$%4rpmqfIUTHtR zdEP*JBz{Znj6lDI#g;txNe+hRnIWP>wRht;B!tEAp8EyWc+;WdW;ZcWsq`0yxo70` zQA1?#%8)rIl~=o?=8)^xx%(h-V9mJARM724Z#D(K?|)v=xq<=b6b#E7-C{)pzroaYo!h>ObKerON>(H6w76K7>9tj_#2X?xQM-9N8lO?lyeZ^>g}L-XnlRee9+-M zjJ${TLyc{5;bf8a25klS`tYP9?o6E|J+923WQZqVk9J4kG$9S7fu3*sk&|igT3+ zA9FPASTL>FEK)2d?hGsZz1S(3d5OHg3X~Jgk|QolL#Qx!fM$H(0e3{IBhS`=g62e$ z1puTEnOs2+cZyC0=7Ke50#~eF@ z>_mLDCA$8ywi~AyEvLxWK{mL~Mf9rLdq?ZK^7qcLZSsz2Ury$_7}mn}eG`w^v)s4Q z%mBmi&j3C1_JZS3vG9hkP@-Nr>mSbXXWfenh>oxFz=$=@SjN zKjp@S%RP@9?6hD5=+@MM_5vG52DBpzVFNzbA|C)|&S?8fB+UPg@fkrZSVSjS1m7h3 zrX&{Z@XD;(cTyE=U*|c%a|ww)R7I~O(ufePNU=r>b#Du@k0;)aZejVcRh%q=_Cyl( zmMB7~(JR}QjzJFr z0AQQ&-+6ZF{_pPgf4QOlkAP6}yDYXC%JA=@;jUi)G%ZC43)){p$EMe5nesSZ64H~I zjnEYmnvEDnXqZ~}Bv;6fkZ&L^U6U54Jcc?f5!|g^{qbSs`Ozqo|@#TCKp6kGTqAF#2aaph2mNN(?qO8E^SG^mh ze!b{%{fknzCA&V;rA<%YJB{SmR1 z*?|{xm&g9RM^LI(E$5VK4&`4%?)(}wD#!aCC#M+83+pGOEZriVVCsa{y?fW9Zx~sJ zH3;4lHd4%fhH88p0$;Qcs$tLQ0>PG4Y z>ZWS_Xxoh;r3DshRZA64K|gBW>xe^mIjJF1T&KMcCRw#?QR2T1p?cMs+Ia=&%%a~! zJQqRUApY#M9E2Pc93&a>*{RtH*(upc*=gGG+R569+Ns(G=R%uM>k;?@)%$oT)d^iP zhjODq_e`AI{V#zyI%f0!7z-4BxD*+`{oxQ{U;6o;!E$bPPatv8NjJ%2nit|lMWdCF zC0736SH|`FbF%V&UAW>Yg;Cr0Lw^UX=PS84qo&^uM}sVnFaluwV> zORuL-kTvKGK7)fmMio^HIYTT}LoSs|_wC7HBiKaEa93%Udvra{w-@r1Z?_tNy&&AH zg?_2PL%R;OXL|~_Zwv0Kaf4(d+QjYAYVR8GDHRLBUV7RYedqQJfcAwqsG2=1n4kpV z7On|AFBqXT$9t;|lq~=R8mdJiXBUU?Y!}Vn)+Gz(*(4C=-XwDNz#w$?YL^Q4#AwkK z@c3Ks7yFbDmmfWpAs8`Qk@)ybvh>t5!#N;VF-x|BVJG1okJY!2*Y^)oH$SGGLjR5V zF1)2_nDsXCAxPWTe}HI zu~z6)me-sp(cIAcSqSs{q?*-aBgjF+v-bO;>&f}IuirbMJ|rlpkoqJ+1j!sX!qgyh zD(X3Y)UgDX+=9A+c$e;!t9wvD3MLYqohD324YBV)44B)w=+KLGnQp%1f(=*i5kv8^ z^~iG+Op<0tTVwVPaekr}aee~LMnmhyROQjty`_*~7pdK)!wlMkKEodVI!)*$>&q3g zRnt{^?-eJE<(dO^-QsRds>pi?%}NzExJ_oe%}jH&(APAdqxB%Mf$IVz=Lr?>ix$&K zgScO)J2uH)fI<0E?HeAOh58~Ge~K!+`EdRs&F@FHg7(xh<`h+9ih?oydF56bA-skA zPeU|zF25a#Vdt!o%_i3^fsn-PVzUGS3fWQCtpj+Ni(>m^e_2Xu*xU9<%g@c3`C%XWSd zhy|vNCFsYF&6#LMEu2Etf=y;IqkC!id_yBSJuXf=P&&R~5?YJrM##F1CJmv)oYFz9 z(BtZ3Il}#N34F&8EIHhFf-G`{0>TKrXh+1MRcy%@kHGK@?#^eTBNkLFh);%5Mc64u zpQZZsRUtK6)hlTG;(~bL+VE3tXRCr1JyKlxkNkybxv(~Q0W?YU1Fb1=wV3jtG0`1w zL^tUd8zM`E{1lIQL8kEurMH-)-YTL8go>HC%g8-zAgXrR9IRt1eEl-88K%!QS)yLT zHESylLn>s9$!X~HoG;Ep;}#~)*gvB>Zo;=YrE>|lsws|3 zl5ESKqeGaTRQshB@rk{>Uumew+p%VDpfPEi@=nJ2+KR!Z++-;B>_$Klqo`W0lHKar z<>FJQigMcJ?%RbhfPU~X(BBUPkS*9J7>^joJ)78c7$&TjbQHr#dL1-%?*sQAT_?SC zqw(=aE+zd0YX18uHvRufa{X&Zr_?;$QI0UbWg5Gfx-k1d&~&0$MQ49yrSBuk3=kT| zFCxUrBlM5QTPH?L*mi`ts=@PpDOkBHp2v-^9j9+Ejr0K6Yi zk+h?vrR+L-tKjJ^+&uwE+GoU40j2v>g`89(Y15=zNy$fEIBV0Lm=f;D2adHXJ(O~rJtBfzYL{e7U8T0o&9aVfe}nB%bD5SMo2P4a zc*93mX%_Y~@oh`Ak+rNhh--O1Rf-!nv`|V)Ygly;M$$GQwRU%j%Ts2mSD`Ep~$uqz;nm7htKALFDzFJ&RzjhK|0B_IO*wlEf#v^Ofvx!Zc*%YEoi)#})lUY|ZJ?03r$X==ps%57&Xn{zjZ zq);ueOgG86o7mJTf0jEomrQq%n@o4uP1Y-*uX!PQYlMnEXj<$7!9T5C~G0tnya z_ypDBd7U|R9sM2l$MT<>TQA+(a)vwb0LjrptT^g{JF(;J^37FJPhiyLa9WFH$FHJt zzOgMsq`%a3d}%oMZlMu;Q9FM9dP9GIpS^ZP#NEP=<22kAtMpu!P+JybtlsfbohAQ) z@l}oKVW{-lT-wtz5?Q#{Y0@)Us^#S?MHz5%Ub{|B9MV;B_ezNwar}5xA8Q?5o5thO z8IJVKNJLGvV_YiVnxnO9b56!86wjrHVmQw+5eDxfu0STn=>-cQ*{8-%-j#@uHM^V# zgBy@o^PWmWB&~Q+ZiOK13*mzzb7HrsD@$MNY5^Or`!uX-N)qa9rJ6K zn5z^I4Cy4L7($jCy=fnBm{M;60l!nbqD3?2Bdu%5k!fIXhtHL6&2| zlT>xS8{@%~Qn>^?wpV_p(2 zVDBxVOb{V$hvyp&#RpyC?5UdU~Up?peYOS;Y^xRNr1;oLoe#MQ7Oq+YZIFr_Fx0 zYl!%Pe)5GW{B*v)9WKBDrAkj=a;T1Mj+-y6gyv4}P&WA^MPucJ*}mZQFLuvTfV8ZQHh8HOsbbomy+{9sA*){olB8Uq%c>jEArHo;_R3mMwue0c{?T zm(LZ-iv#~gy)*f3w7v(pkLL6}Bt5!+-+(Px`|@hB5C#Z_n}kfj9-=SW^fB(%k3*OD z3zq7z_}ZeIm{&0k=wTo#X{wBfg651Jr;01ZnsI6w8ig5&%4L^q*-qYud{nbXfY?t# z1OhP~p-jSJj3*KfCAtR$?3HHTxXa5n7YM)7UrjhtD7wR*;&!Xnr7)8fMF|B>qzB&I zM#s^&W{&|~8UKtm?>iC2hRTC@w)@zeIr8uocZ-Es_5VR_kTJ17l%xwREEd5Vz^*9k zb-)3x=rMQuKA|~JF}DU^b{|l(_82@xZa0l8Atp_PH(19D_iI|6xy<{*R`@pJWmbl1 zG{@oZ{uJ%TYj{Bha)0e0QF}29p{8ZAA--heHA7o)&UIAyAgE&(fO%SAdBje9>l}|s} zD|kWp*-v9EOiW8ldPNW|Ez2ic984T~erf0C(&}}u^$tmSgCJgx7-o;We_VW9%pTr< zWBw}(etmLIDuo09Fhc+TnFY()nEeY^Nb5P8{x`1ZlR=b6;;zYZHszd0Ayb=GU@ayN z3~O)H^9u``$|ZtBx!P)sd178bzk3V61s3}Nak)7O1@&V`y@# ze97kMh&MbI?EGN8p`g}e#jlIAg{iQEumY`thgqxX->G_wD9(4)l2@SkN4ik*__=R*pb!IR?? z{#b3L!$6_ZVv|p)E>B)5R8yeDL^Z}Ege6y8kfHBPhr;a*Vc6;uVv)`?oaRpOxRbW; z?&jqI%-rwos}A4>dxx$HX&fb~uR$1_4W*@>HrB^SgDnuiWKJPWs<0!_u2Lk7lyI81 zMOuND4a~N`ZP}?CEuPn+Uqm4w2VbV1aSO|owAwq&Xx#0-25EtQ^h4baGE5g_wLjez zhQ7q>v)u+HM&u~ zwKmqcy=VcJX`*xdyJE|gfH+{Q^|sJnfdecUbOm}*mfJp$amFM&#@=Ybl}9lrG%;SB zL`}JFx?VI0%kggkJFpC-G3L($Z~sS>iTwY3;Q#ig{{PhSjsL&>3IESfSR(rWESHzU zlr|$V++}pJH99wm)z2i5P~7Ez-lqEv0!rzBZA4+9p(mu1Sd(l z_sU@(OKNu{(hc;J8Lr?3AAT3>?a)4TRB87ZeAysvOd*u}n`h;i0xsc>Tb3?vj@1ss z?`DUQrGSw_OJ2=2@(wblGs*JV`ykcKKX&djvQlSn%w@S>n z+4eF|4oIOkC3H&JI>2eYho?Ct^7w1!{7IVD^TH8)yFpd3@@*7bCy?w~BUqmYa1c7d zG#=3_hR5iS?UaAn9<;aH>X*1}mZ+ClU(^a#cleLU*n=!d`0-~Gru>k#|2y6B&wJ>< zkGy=F4DgRopJaq&BAk$~^;*!+pO^^=k4OfBoDWcf|CorxuF{^VtKs#7SuW2%t z>&C)AuZJj!Ey9L^p!EPbCRS^##=d3}xa=C64xQO>6ae&Ch#Vfvl?xgKe}R~{v}2iG z`-80%vSmz}Y?B*OGGkq6IYvTzCZcRwOs{G2pk?E+wY=lK`x>G;l;s12w}wfWnSjj$ z|2$mOj&Ht7HV|IKAUpMt{bdhuXQdN}&K^m`u^33m|@|^I} z-DRpt_S#rGx)>0s2-xJ>i$4GX&-hNpP@Zu=hy_$KSQJB^q9rNAx#nFaPQ*FNP?- zeT{>Guc>cdlWT|d1l}*VFHT>mm*L;6kbsqcmO?;q!vQl#FhZQX9byEm1z7&=DWK5z+O)?QmQ*($K zhE#EB&E2w!KYy*X3dI03Xk3LHrG##AJbiXHNy=oZF=_JXt|4tPGx#cH*fTSm*)?I+ zl_D+oG{mjMG}L*V%H>E4QBz6!)S7DK3Hype3>Fx}1K+y3hB1dxN={93Iy8$CT1(`Y zSyajrkK{0MEJ+DtEDh8P0_V>Xjm)xT_AM{5SG^Qfs*sN&QD=`qx^aYOOU3iGyjW%~ zoWdZ$$ryCHdr@z0z3ena`!HW0sE(d~v|cX~ z17X}si+0$$=e0LtBV5&It-(YSl`=lms7{f#t$`6t3qGIef9c1AyQ@sbF!$s3v1lAp zcj(4;Aw$Sv7qE=k2H7X=#cWge|Jct}XO#%F)M-?!gm=iYHq^^&RW0ZR>a>Z%5UyYA ze8xyMS~O%a%G?;WoTiQ5O@h32aL9hQbur@eVEMA`-NX;bhZw+)VCy)u?b^=oB8Mo# zmS9h_MRI03rP{I`&l>jcbSM^{10UdDfz0c?#Pw&)#*#Q?j`sM%T#`C=`ksR+U@({y z^++zfoV!k-^ZLUcfJ+^P(!C|}U@1)N9X?3n2hG)F0v=+;d7rXIlO31ZgBtNYVFZk`7iaD>kiAFlkF)1x;+IVT z569ZeP?)GWj|U9b+RKoWQgs-Pf>V;_XO>8Y{jMCnkHT4L4O$5+Dnbq@Su00Z~N6opva8Ug3W zBn5AeN(hv}ECJVMAqcd_EPMVOgi0uev$+wgkLDpgA$ zk)6j$=d0xoj!HZTp~&Ws67D_XGF9r^)Fr>Mu1BR!MhA%vRPb7}H3|HZ9<{8M`KXT| zgPN=Wjp}OC5NB~rRO~A-(IE{Z&?UgFct>q+`W39%8`M{FMJFn#vVA5)FD2zTn zjq`#B{#Z@2M;T{5fFhvUK8RedAfCt)aXPW2iYZ(JzbC?z#mzlaMmV+H<@47Red``O zS9p&J;q9IxYOvS1u|4;xOfDG1YtqnkYy{HL%G5>r#k_9GqPs@hFnf~Po&%eM7TkH^3hc0RlJRiGE`{GT0cg!zK01 ze4y_GkMJOlTLVcA*l@Wc1SgD=#XB+iHIK4GI`97iFS_SG9#Q!oJ|-Chnn2Z{eWU_{sPmeq~r$tsjNAc z@9aCSR5nYa|EQ3tEE+mdkuyPfw~p#K`sY0r@=7vFea#D2_f5D2qAgwSO!dTMqRS}2#pWvWa3*Kx-0G_z8XLHr8MH|}{> z~?q=Dz=py3bP- zoeqEUZkiw0t^Ymu{^z(MY_I3=-ys$@@fHirBEm{YT9(IgCOc!p`|)+m zHh^*LlfW=FuCblwP-!?P&%-_q>+{Q$mo<(l+D%-{Enc-=>U#*r$Hq*B>tHSobxo&O zH5CJy*>a^CwEM6f6{|7>C2~bST^x1C(1}Qtzx@=o+MM)4*~%PIS}xxpT4RvmnRtM( zgmJ@QlU1_$;1C0Jy@B;yMcgJ+%arMuWB8obQd1TI;L;zSG_UMwdZ}(MLF|}ZV);I~ z7phnL@=lqe_xff7?kcM)^(#h)Dk0KxJdLeCsZgIal%AmGgqk#|MV+T>nRVjlVe8S_ zStj5vf2IsMl*SWA&nz-D6f8|P*Ys|VF7gdjENE_D1xf!jnzzBzLkIO|kx&NpGkH+! zkTDa_0Ro!D9294*seCM|9EVM&Af`yN*vwDRmas=fLw!tOF2(UF3;_)kDXut5;S~{v z3S%GJ>WBgma&_h#00?pfx7`|q<~_H{q|4?hIafY;r!2En2_zAL%2aMN8{F!Ra8Q4` z4x!=tgir_HhoQfDyq%Ih5CLhi@OihMer^oxY(mJw0DTgADqsQ94%@;rk%yCv-H)h( zy$4qdis_Q+qoJmKRExYK62u(Oh7Elm#Z4AlKpd z?++3#7Wuss7I`Tc_3OaPw1b)?U=4T+VV`r)RhHTtu5v%{utilk;^Pv7nY?K&5gq|c zuN+hX)!In@bbrwhX=g?MF`A8l3XE*OTKP6EYYvt+!(w6eYxs-L_b*rip`U0yr=q>l^@X3`H#j{B>#BD%Gmsi@%ZmMc19MF zAL+|@)ygcwKa>~ll{gn9&VLwJZcu-zP+TyQ?Ac0vTz`|fL9_Q;LY5>QjaH}C^>%M4 z0+m*$)0KXhwIM?yx1&M2+2(laz598K)#iBTy5lzhN^QC@YI6N-U7`qmlu__v{pHFR zq@b#|djLOeM+SrG4R)D4vL*|PWbW>GAQtMdI-Xl~?@ zI@dNc_iW)WN#zt)NKI;uR%)FVNKTRsXkP+HLdSxRju8d{ui@-{qq@n{{K8uc{*p}?AN`(pSBZcpzJ6;jyVRRWw`_2s z!ONZEp;P|^_RowCL*xN+s}|h*pta>Y^y;6--c5(Ki1H?3PWFh5>?WP#WMR#LSGK=Q zS8+GLLHO+a8AR|izc!p%_J7mm;zt?9>1&Aps+rz2Rt(K`aESq>!*umOdgFFjrkDW5 z?a01dr(p+cJ$$9%J!2WQdNoT#>!m|ko)wVyWL^8>GDE@?Ijod?Ce-xgMBmL`r+1mc zK@*clSsLl5Lqit;E=Y9_4*NxjHi4LZEhm+NR)8rI@{$!yf-$)TlOs_ZD8T1sa z!{oT&Bf+svDEQ6I4^Ynri2p903E9^&Y7WGB{|0_nAe0T`ELiAzUh+V&%Lf!K3mMJG zR&&4>x`Q?dGDaU5fiD_nsVW*h6K_DZ!aYbI3I8JuP26Ilk3~~ec_7Zummd>JaR7?0 zfyWxa9}xnIwh8ATLA3Q$U8E11z#p+>Gah00#!9EJ;xIJLx2(BkOy?8GV7+GC2j;hUQE~ zL-O}kA2OX~O&XJE!3v^-a33`c#}O~M^W7e4S+co?NlXm;cvJ^JgVaNCujBjQApbIR z&goxgCGH>7(f^CFAxxopi<+tA8e;3E?zZ`gQ>L ze*rP!_w!)Ft24w;j#32@O_?&TLD#1*@nOW%86ni8T8V&JtrdC@OJ^`?t6MR?an($Z zsJADK;(A|qxp?N>c3$vQg&=~F?!*=#X z+1yRgYP}G94-M65FFs*{c~8vj!+uN$d46p9@wUo^?r*$^WQ zMDCIM>?XUok|FbaECm^NPFP*vo^NZhAzpLdxZvFUruj;XD0)eW2!DQk+-_sq0j~BM z9TJWFN{x_heff?0eYoeN@8!Dinj&v(Yk1D90FOx;82&zuT{mg*vK^U=1k@ z3?!jRRw&ZR@559sf}O8`*TK9OQw}Rn9J`JbAq;f>uK&Q{@7}rlhmC;Fkbm8NJKU*? z=bGcXzxWWSpEC9sCM2KUhpc%X_d>D3l$`R{ombUXk)*~$-bSZN@7(;6g}Jp2GMV}I z$@YZ}_?zG+3{O1rSvKL$g_41}Xr!<*>D#-$-ycCyw#7Nru+luWfMs0hw`DM8jLdT= z$B>V@?obLk>d%ulP`#;LTe-)Oqe()c>0sMhMPjo8Xy7#g%tbN0;d>R!(9*BzC)*HS zu>uogEJNavx^CK7IHhELXNUX<5#$%J%_>A$kmuX_x-81iIA132IW>+} zJI!mpkxA(K$qN{Lt-h5ARzOKj$(#YA(_#x*{3Xo~z?Rwvg>Gg{xP-rb zKetdx7(L5BmG{DwE<;~lm*eXs0eL91uu`m1hP)KGkarY z3k`WX3tbxt5nw^p?L3Hn^e!R7%|s8h;cu)JI%SXbwoJXp;(ttqq7-Vytfz~Olb|ic zHY3xTM~u5MU0JKg8nwGl!4?;*AB{B_*EQOc?fbR071k^9I(pKNigK72Op{y!c=r@p zAItJ!on%Pd2C7KncO4l;)F6_Ut=o%2rF6{3Y9%#YXZ@iAP;PRC(Lt#tG`5S*SryQx zbN|Sn1rk|+>8q%!4#zTYm}6)c;yX#oS)XTC+ZK-CCh!v(F;;)pzj3;ghnL2|>Hdzh z<{%s*0Y{LS)7062N^MUvQ^`1R3iNRQ=p-(; z=t6f^m1$+Q`@pd1(%q`P;iHhBMZSoz&Lt}i;-9AZn`S|J;_Wzn32x4sJ3-n91>{+D z%;1V(!Cw}LzYPw$WbIQ+KEJsv$eQWRO4$mBO3ns$%puf7AQFz@*Pke<3tvzhmT7I@ zMtNhC1o=54?BM$#N%4lDduE7Um}$?snmad? zyj7;J>80IYb<|G3Jau@;k;d4TJ|RM}HFapnlIF;kJ|#l2Ep=qblSZZ>F)L=Guv8)~ zBuAiKkcm8B-4#1uxDPRZ2_J>bRmD~P$O|%i1PW4zxM}corSK0JrXnQ@tOg@>lM
    9kkbwkb zLwlQxVf$ddJZxocw+kEPql+_3osrS7V^$^t5r3wx!;`d`E#~?>OWy%yKzo>89+|AQ zxlKujty@Kn6Uan@yy%~NGaS87KPC2+gtE;o$IT^;qd(mF5=Y>FKmRVfxL++YEqik~ zXw19Nifw196iG}2S9Xx58v~G<>CG$W90e@THc?|JdXSkEQ+H7zTqzUMOSF$~0&I(zF?d^@s|?+|fb$0w1zLQIgwN=d)r?7e6lrKQoiUJ)6%QWYR z?w>F$ZF9;EwqPzryjU-+v;L@#;d@2Ob7ZMa<`RIIA-P9(eq^)lxXmTpU~x@37!7cR z_TVpI*%`Sz#z;FT53?EZsYJ>E0s$J+1ttP&sX~wXh>x|U&hB@q`M{&@(9V7)vDl*1 zTSHG>^x>?!v)fH7i0E_K8I{uh12+bKv4G) z#CzaU%rhJGQkqpHWH#}kFGV5hxI*M4Y>MolHJ$t^nGupkQI$*K2#GqF=yeSoE-RD} z6~>*VB`QdWWhKf^Mr;wCEPBnhAq=ylCkJ4)it$~{N0Tt>7Z+j}`jcy%p9=Qqx|Xqr zXAv(5Fu>@8pJX5)2h|co?Lg6s+9MU(9)q;XCYaF`nMsfDFyb=zyf{>7DpUX5SXVnc zquBB;~v_-T&NtfTgekXS;IhZk3>^&ec&4FSkOg>;N zI-N*bgG9jovBcV! zvgH|Qo?a-Ju39PLKe9M$g-|ewRx_F%t+F^6wnExfFW_{H!e|n^xgC%P)6M%rm1 zkhNrNCDwJ38@6Hi9!1-DP%}ni<|5Hp`lT-J-U3TWdT**p3}@=6oS(AZk!vyDq+h zZ8(yZ?95fUTBi$UGDax^787G2LGgkEv@N;wPSpp-bi+>>B|1XSjNF;g>jD0eOZ*ZG z+D$O^H!~U1JGD0k?nPem#t7r3Hi@!)KQ$YYM{Gy%7mp=Gw8hRBQv@p^^5iyX4B|^E z@k=PEHx-PlF~y4+5U=2l33XqLJHDBU{JDoiZ`)t6=EsD-LkZg*eB*w7U%O-?yU*uA1rk`J)JZ#cn0Q8XGJ zk*Jr!os7A5^n%-X=p(9!RFVe-cLRg-1&FG|kp&@l_RVa#ocM*iG=9V`W5IQO)s;;}zCTZ$ho3Yn>C zJ&n&6d6{V~g1ZKFoM_J^E;`5%kE+@0D+YNEiPx?cdzE38IWP5}QEM`)c8ToKECd zGT%rLM-(6HE}HRaOIsvVVvIKc3{OhoDVSHZFC<#uT|XYnsbgIFxeX(z2X`9SetOIQ(lRUqR=tEi(oEp`}hzc4n3vR1cW;rgA`DA;vw;I~hnZ6}p`P|#Nw>T3Wo z!>;ou-2id{`hNM{IJJvTYO0DU#fni%8;P~u2b*zHyF}81jX~d0xq|(wa5&0R&oZN; zzi7(2=DTRgYrR--MW;X=7u{$~D6g*Ts91X?V*Y3B?OV8KE2U@zsHmuRaY;%dXCWF@ zmcQ4;8LO6zQ|@n>I_D=)>->0fBs(bOaz<9Oa`r5zc1{uEm*@?MyFQ4{9i$+iaAj8{ z-q?tkM4x^l9+R_=NSA8wjAT(U7`KhFRu}pNy@pm%EIHOUUo|0qHH!3TSnezXBh z|EL7{M~dygw#t7gz*N)~F-76NPz+TxXpeXdL#4r25SryPSJ(kKhxcmzV{Ka_9d=8v;WDE=hGgwf|9_4{PaETKTSjy~TmhMwB0c&W)Ixoq zCm5x+H0xkVzfTO6d>ylDfv7V#6$P~$wwpln;~ohT-E*|Up8a;3Cz+)?UH@5WY0jnl z_YgdcGCJc}LX^rIl=hKWM1dGV?PT6_%L$fxH*QHF4Z%Pq$)P{4$(l=LnjgZNAhuz!xJryEiyQAI~he&ukNm6?StN^MRSV<3?u7<+&Q`u2}E6GT?-D zi5+vJ(yr)8a9ByJc|Jh9)|*eV44cw^sMm9G^La-ZAjV|k`I>;0-aE+_e@&P-dG)BF zmMckfp{wTF+pI_Gm-v$u(G88}v2;d!dBI=|Ujy?{*a-8&{x*!W= zgt5j30;)djbCcuY`g?~7#Igrh5=tAYu!OQe{j%1NjTh(Gq@3buwIxh~7-Fwcy3^Ll z7#dqxd2tv~`LV2dFQd(xM*OtGzv3*kAcp`Z5a!auzdM7U!UHh03Mv(cb+k&DBw<_q z_|HNOTahY*Pl0HC%o_)noRCfv4)Kqg5ik4qq0X8qP9UsB-( zCvR}NNZH-_dXPlb38W0dIS6%eCd=+O56!?9fuC^XL*!;kE3l-=*QulH%^focvypLk z1kYe2_Xw{Bm;FI-DN-6Vmmt9}^Nqmnbg-2fWoN$3v#UzDy^eUb@=T>JCqlIwgR(F3 zq|dhJda0l0rf<9)JInDJ6UcWoRSr1+9(gqGf^|ppB3Kprr&19*ix+A^+y?!+Q1YFl z@a2WSG12O!H65yH4c%tzQ3UiXT?1&Xi-_hN;F zUmARSkmkPUbR+2AGjemnM@>-PJm8~Be868Fps*=(X~Rc{4fSa95&rU-AfnWx?`4vj zb^}EC0s#gAh5w2~{DVxV(`yGr0$vM|N|vj}7gn0i6fM~%)q6mY-G7KuO-8`!Mb)7a zu9W0k1{95I>+g#QB85n&?U+f_fDU8~nM(YpnXX*;5X*JUId2I(H3ILJ3u=ZopJTN+ z+5#k#>WZfJMgWSIf`F73WmS>Nvpgt;4!N04)9VsuOIe1*%7I%^xM9E2*~B8ipsJy3 zsjYhI)XcWIBP*L^+nmIiKFTIE(7?`zt$Pbs;+GU#j!f}KRakX3U*gyzW*EIXK3tSX zMk_}ru^llEC?cOe}X zU79Um9D5Tg`r1M3=i2*sOiirrnroQa)Hfi_Ev%K!jiE+^N@j_sPAsn66%0bDX7Tca z`&<*tSEgnN*~3ii25mLMYL_xI6nYu;D>N}Pqbw$MUG)_$ZL4_=_w|;HwINIglD+jp ze>pU;rurM{uv5%3<)BjU;6B*)TsLl9l^CQ);RB}3>X-R=Fh|s{9R(HR$%?z14{+Tvpyzzu>sQ~0D6t~J&o<`CBE@W!LNo2mPvc3++ zIuH@1Tud$=!!b!1IM%qfe0FCTgzo|^8K@dXW^cCCJ+!KD;UumtLPs#e<=RR%4$&kU zk5z4Rbj8FB;wB!T+?*wT4vW#=!~9F4j&pwZch$3KfIn&2(MvPl*_fpZ(VF&pQYeUi zoKPd7T*oGEG>8Q@06J!FtGG7BL>D*JCNj)8r$M!I7&Sqi~ofj%k3>0DDIzFwZ6(SF$A?IHn=Ltp$toXX_1#!`Q zR7a6cFCM@!aoIRNt&G?WmFcJH0mMhPe_Xg0h7Iik2l~(z*5)m=Z}}E+V?u-6u;YZ# zpzh_J2o)Sm(6AwbfHuEk3D>etAOiQvX=|tF=?#_DkMb@~u{l!R*r%q{yOW?46LdU0 zhByLSoMfqRPQ=@IiCf<5nP?BTF~I4ku-}vGD=FFxB=Lexe_;&lW+{Mw6VQ?RiRP-vXZ$xbOD#@Ll`K^j?Gg@X{zKZ{&4jV*h2xfqQ_muO3L z4-;Pl+_cmEHD0cLHbf^(*a47-flMd~$$`jy6X;SRz3gVy)h?JuNo32B6>yq+#^sVr znGKsEV*%l86JY{%HkT9IgiT=>Kaa8n9~|o7!sVNe=ZACarSXlNfO03dF z@buyV6m5GP{HK+%egZXef>K_CyEAz7Y(BFh?hgF{TA7hy6XUdqyu$f#i=k#2_wm_N z9L~0HVj$RWQ76OsZZNS-wj-wc>a*Dv3TT ziQ>X)S1rGV6(I_YBWpKWjFgL2rPlutJI|1j!d#iUfWCB^tixy$HAv;{_nJ*nQ{%A2 zc`3SE)_^%Yhh>qrhIt{}|H&hRXivttn`R&}=~PY@~GE?u=( zDUl6tg0*af8%6pHh%|13208nbu-+FmXl zBe5J8;aCIgCJd6nsWdfxK#D|u7_8*W%pg)eCMGKE;@u=?404;f9IB5;?#tjcJ;Jw2 z_$S~XA144HeFs&1;lr65EkS#`N~LQ@F>Z^Oz%Yu|w1O3kHu%k!7|j>MP*`^(wz!mI zYK5V9(Vzo@PC)H(mc`a>tVw&;$rex!RIWTi_rS-*A9<+{H3B*&3UrpUf}X2W{v4ucZ{GvMu!F}{MP2f zEoGwJ^kWewRrwYQhKnot!}{bDu@;Jrk7_M_wO3R-m@qFSI~g!9C_5&YHx9lnKQDgo zFfS@Q*r1(=uD!l4fY&TL>i_}c(~xV{xz_+V3_IMt1E5?$ovf1AZlGKe%19G84dHv} z-YW7012YtBKl3#yCB0peKeDO5Ue!OK81__zzvJOsDBd(gHp)-V(d{boZ46&jg}>AP z^(8Ctm-G%Q#!Xi*EyK>cug`?9F7!uyq^s&&5$FfQPKwFUemIrSH}}LgWy8m(b8?TW z(W{43a?keIjSI$+X{PTsh!I#CA2G+64x^Z5ixZE+4W60vLV#gCF3J5c|F}o&9Z1VY z1(K(1krL02=XgO7Q#Q)3GFwn_d_b`xwKgGcU%Nzma&^MC8(l}Xb#E}0HY%58rYa=1 zvA|<3FJCfeVyYST={MvJ)$l^J2`@ZuZv*&@2^~OzGiA0xQuBC*8HI|)Q`GNRvxHZz z6)m@<8SCIGFA|`VTzg;0_r@M*)bd8F?zG|6dm$x{qIu`$KOm4E1)x0|t7B2q%?4OylCOyXvt2m|{Q!XRdxHG~LO z*7{19WSPk+mcW&7mwi&xNG%x@V40?Q%QwxpF$mV}h$v6*qO_SIVp|{q(l7}pl@ZQ7 zGbbD3@JefE{SuyCft~2>r18+WwpHFdb6p_#=Ip2&eMz-))mS&g_v{`kkyP(ng&?_p z$&zH-$P;o!-R5{4#qW}L6&gA*PuL>oXOI7Qcc-8XvEhg@oOEJiar;g6(C9EX#W6d8 z&`ZO2-454ug78pTF%`SgBh&sWE#+hoVZ8j%U{1%fUE)`oz`Hkh(3S<9rXa8J?Hu)sgSXse>qNV_W z+|s_;zPA{#lc-#+f=Gk_I@i-s<$Qk(=mkrOOE%%8$8OFIzA?p=q?_mB1zKG7B3)^y zIuCc&$2cuO7LN%k)nF@T4JKJ&Um~CQ~;o96BbZmOez;)oJk$P-lSH0v+;J$qjv^LZ! z){!VXM{`WPhV0o+L9DGb$U2L3R?;y#nKGyKhQAs0=8D9Uor=br`zlCX{6`LcEJ~KoG3>s0R^D5m}PHG z0u^suRB@%j`?``f~k>)wrn^Whaw)bs% z);zrJ^}vK=9VV&us0!mBXhS7q#mLG@Eeds7I99?hR9h8x+o-sOEo8~#{*tXhzw>!E zC5${hD2USASy?KgNK067q<%# zip15eUKbbvH)3tWP+kNyGIWr22Ib#iVvN`f^t(XA46sz{A&ue2$BWkkI)^SU4k(2L za2WVv0W0Zp+4*8zsXL@*V%90|8%PwUN|-7SXWZf>-CV;(ZkIG?2n9WOyaA8dWcma` zy^)ZXJ_;p=l%9HPIMfA(|^|79m#(G`WJ`fq^-qt4i5m}^waG2zZXXSkwa22 zvNklbSG70$>2LoJhg6iH`7Z_b$al%6bDlO}+oC<9T$`1jvK-)qumKUh5}p3+&H|96 zx_@Hf#-i{~;! z%tXRE(72+>HiBu*ijxH)+g8)Y@{x5EK^C&fOr=#u_0&0C=}Sm6DV>hpM!6lFYx420 zt862zG^ZJs>{vIWrDy8rD2w-GO%yRsKUz7lYTDUlp;cH(3ufSL^$L)!Jl_@glXfxK z*UTmO-;j>F5{aFYU^?IrfjJh&UFLH99Xd%wyLQVdJahawY*%&SUR1;vp387+8{G9z zA9sdIU&KW(aWq4Y6-ot^dJ(-Mqp+SG#-j}_@EM7MQl+za8YtkLO$K$464tbV*NwJBxknNl`W2TL8K|rixOeYvb_Pwo zJzMS*#(V=^rDkD@hYNU*gdPubUYN+qP42Zq9ek{klirKIh)RMlM)9qTm?K0LKa5~wK?(SRG{ndm^?~CEa-|?(4fcs)^zo&f z;_<&Fw{LLa!W(w_K)D(yn%2~dOHFHQOIP192s-U+KF3qip#&Bjyi=}S*PEVG-p5&g zuhK2Ac|NFq$Em=xW<2|Y)9whP%e3E+z@Od5r_hckS{iii<+`s1Rkz(R2Z6D2xhWS^ zj}O|CQK22C3+O1S5gs@bY%8cy9IOj$$*GYZa0+yl*JzFW4eThb5g7R!^vft> z64yu3i$__3_8!&;=*_06Kz&c`=oU{YQB!&pOuAEKe?O7)?N>SQX#ucWCP@YI@qXK{FY(&TgtH1k0=N;cjOvJ73U z=*0tYgDB|Mbdq49;#9h{Au9-B|#teq(2@Sh-H1L6cp2%OUZ7F{h$Bx9+o}hGEd#g4wWZ!`*$|lfPdX{+Gn-xbmma%G z5X?EO%D>Gg1j=&SmQbUO4wr_Sn{xb~f9*ONOvPfz!^f_iE!a(3gdX=89Sky`G@2Gi zjg9MOE}^dm1yxLGaqHXM>}jv=_fW2W4eryFmj`v>DTt}G<22sp^`4RPBR|;1phpDJ zpxK7GNe*aIY$@C(g*8ic9PUx0+XlH&4rr2ZN!+T0sZ*Oor8F8Wg{jk;1f)b6u!OBr znM9;i8Z?Ei(wKy#WE!}rZ<-IRQ*YVaK7}@mc69DtqS|J=y$!6>Zu#88g}U-}1nl9U zT*tbF3~*9y>EA|&x(apV@8O_cr??FdaMEnq-^zrthj~7)dE3s{OqW}p8@&D)x1NzP6VHEE%9>V%OS>3Z@FsGPY~%Wd>UYXqnn|_QHa(0yd27>U*idngJUo zcI~}5V6K1-W4q>FZm_?A4O6@BUJx*N0GH8CO)n8x41mkzrll7dOcshz5!WTkYwJgQ zaR=`Rx844a48I_ex*tL>?npIR{`(XPS;X}>_||D%P2u}^ef=iyW^4^XYi~$%cfs0$l-1?;gbS@lVyRXjJ5WSC z;~3Mq$W>=0A8e{sJh`^Dr=*H$w)0^J7E`B@kO{NHUP#lEk^KrZz24k$ArPN9a&Gd! zPKsLF+5Ogu#e%t~)7ySW6rtsZIwVkB18Yt9J2oj3=vc@@7!zu^K_#!J`J!em1$nhj zzGQoMzn@m#cds427vHuv-4{1ceQBiMS2Go&@KibRZ-n$+#t@FZtkZ1mW_ z7k1piWT2*Q{Q>3B5*WT8nb~k5oe{s6n&d1doe{pTN^_Nz%xOR3W}0>&nNhsgN&Bk; zxl!PbH~F@fF(V88(MZ>G6zQiivebv4uE-(gm@%wZ2~)eNC8RJqD^WE$Tp+11IzJ}I znF%dU!D?xoa(So_KW0~Xu#VG`QotZUNpu;lC9Qy2m>9EysU@S(DomMgoqld_VH~N# zW8u2z5kFR`@sg4``beButAT0idgKu=;itl5Sjw;!QQe& z?q=q>I0bSeQjI6ETmdwTICR2g1!#DqT{>FP3O6!@g?<^D#Dxyz9}E37G#>N6GJ0)j z0vcrXwi0^5Xk(3Z^H&Ky#SL<~LFz8spf<~WR@58%eHGlH6Y~NKQ!;v8XlQ1+OSpe}1gHlZ^0A(U&XJ%T7!q`U=C=bS$v7wRBUz>!x}*g>BZaQ$GN82z*v zGkR1o`SpAw32Vr02x^dr5L!N(ky?%oAvaEdfn8m>{4_cIdb}TyTilT$H)_Q}ujZ{m zucnlM{~7`OI1veJJ`X`vI{kYzrb%o15l{_Q zvG&pT+g3?^_a8fIp2zTEmL7t%M3d2IY+2R_d#l)FGv17uUvlTt>-^2QbnZ6e5%GNs z?C6K|5k*sRosIOX53_L{LG0NNV>m=In(=BY-%H_Vbq|EWd#^`9hX|FcE<=X8*$a;}Q3 zituG!HyT`{^fO;1w@Z#Fr=h8lng>c~!3^~0fa##LR-eGiz-Ck<^@7FadRO+!Q%080 z?f17vH?Np`IOb!9x9o`(TBri?baZqh)zkE7>fPk3^X)IM?*_;Xl~1?A> zL>M9Z(0zg6z>L__1OXXgI}U$F26q;JRbwvnK@AvDd)(mARI()^$^(A{#gWj8qrk{; zGe}Hido3}Yk-%8Ry`Ckx8My^JdA~F&EjrW_@^zL>qb)n_d1blvO-KAiPOcT#mSRNY6e z{Ds(y^<_q`6-Ng}{`oqVl zRx(^^4@(woB{s7Y6cFlJ?7h~2DJ8`+=Lv(Yku>C@%djeKDtjfy!Z^gA<+!ycM;&<4 z2DiLJUoHM-(}xx64^(|seu!OKBz+X=s_-1=s@%6mQfY;}1AQ!{pi=$zY3v8m>4H5- zTa2`l3VU;?uo{$V_ioGWURcR2wHEl%zLjE)A4w@;EHPi|nPn zi@Ui6`$#(T_L$Y@ZXmX2DFY~|q?}==95}YSP^m2XTT8cGZOmGW_87}f9RcJx%ePh0 ze{Z^?;RC%wu{d~oi#d3Z?^-9k{i>b40&Ii4iuXNhGl!&qo@a+v`WM10oQy}NsvFd~ z+mp!G?D|<#WV1Sobd8TD7W7j@-OYe8Vvx^JwY6>x4{6H({9ITm=jvA&^W+53AWf1p+!c22vw6wu(fO0~(iA#RbafJGanlPaRV6%ANRHy-?%)B2H zq{)xGuU-;i+1Y1LdPxkM{5V51DooaLRTbWo7n5evIx5h$^L%s9bdt1gZ8YpClS}+8 z;!t#4ZTalCGf`%vMYd$38&(B^+1KIfXUJRLYPaQw=uz6ZK@-zzERofL2UA99@f;`3 zXzM)g->Zyr*vE+CM2oTwau{T2_2tVP%i<`tY?Lls_j8?*D&2K8?L{U&KbTQZhbq-& z>A|+NQI;=GSX~I$t7Q3Z9teUc+QbUD8pxd`*MPYRUp}tcq0@NF=-LVNzon@uO^VU2qdQ&(85i;mmk0>r=S#h zcO!}yktKfRQk~)goIB@Zr6OGy`tLhu@5xc{$`FFUS^>p*`m&ZDz2movH~N5fLfHCa*c z)OD-#mjy`-!GPFbx<1tuMWn}-ki?T-z=bHXLf{p`qem^Yw-&odDV{lYF;if|yRjV< znv5Z~DSeQm7u3}KC##p#>**tLYi1y;&?L1RB5mz&MWJ)!DRX})juF4K<?fJj^pJ#odcLK-EsQA8gA2NL zUYb-5f;HrZtJ({$X8uC`Oe~`bd68Q<@mE~5g|Gtk$@Y-C z2V=o;UA3_)%KcMwvoq?F-farM5bwlMck2zhbDkasXzuVRY*j{V_8=^deb6s1o}*`y z;cAZkum6fPumB4{X#bWU_dx%rdhj3B`rmvbbDMvqjYlbKe^&#PO;%lG(s=<81<;?= zY61v~ycU&YXb8|OP~mgH)|bSX@wo&6_XKO7$X~x{y=R(&h1JWwFEqSQdC$(e67)k6 z&F~zKd@|~%dH!B_-Y@fZf57c<`6$^#-CLgw`}=!Re$&8#W)Z`aHRb-2bX9~`!8kJWkR4$-LttfxS zRsM+uWdR&6PpLtYPdt;LyPSQ}TBbvJTtwN{X~y-`Vjc-Sz*^oWemq66X%Dl`T$$!L z@AM#`3>J>+H^5wP(uNG|o!YM_k_2inKS45+#cDaEZJV#R0#6*u$6<^5couJy?tn`* zv=M#^kujy|!rpwu9&G6DGCNXv&OAIJJxA5nq)wTjzhG|-hC(@%)DzUtj;)ig*Jl`I zJwG-%In6~4NUu}48id+fOY7ll;eMQ8Cbib|Ww~xm&yoZLcM+JYWl_?3vF)PyHd?hX zdcvMeV)N6APX~#hBMI9`GssdgvNB9wr8LIh{z099EWoa=K_{^zWMp*(HQBItPG^(P zA9@HYlc#J)m0=_jl9-XLzI%>g&OY+e-z00>C!(xd>T9#w&`c-Roc%VHTohlnU=q(c zk9zG}BB@3#_j5~0C6q`1P;57rfNi*Le#wAYqKoVZW_*qf-xN*Xm_LlP)@%xfrz;@{ ziD81#m)kjri$Z3*?*@=MY66bzH2X<$Eo*ko>`3t7-F0F>7W$j6>N~C%wH*5)SV?w} z{i2r?XKP4yLrC6#wdCzm3^7aX1P99=Izi0?Yz##DsmOl&eYN00Wm&id<86&Bm?iQ3 z?Y%=_%Yr=o47MTuAPj8)%^5izVs6BmqSl72)nSBk=h2$wQUv=eaGGBGh z$rn8fyhr&P5|@j!J5gX7xXrj~9h!~k9{n`S8@DFxAlr zGUq}>6M#Tu#Jc+m!6wur?>h}XpdR_-^8_0bqDmovf2})Jg>b`@^N=R~y=ikbsLjg- ztF+xP-`ULh+sm@u(7EA$`4;$6)v#TR7B9v+KY>7d-1MBSDe5X&iY|2g&yeSjMXE1i zJ_hr-6Otf)t(_CXOOjXZ++*WNMRi7$qypctfgw!CEL?hVda45*IWgK!+Bw?3Rd%n5 zXEN2&$($DKZvSn!C)EMnVAG@<^k4798vO*n(G*6(K`ASH6~V#o1weaQPpIK}6D#*W zx^NzZZWN&htQ2Du^v%h&`ki?hX;pw+(v5|UIq)t9;G#umCCIMxGcBHj%m6(1vj=AJ z`4Gb$ML(_za|9AwIyp~}ZSA$pC=_AG4{+cSs=$q##7&fj!&H8UNDDUwHLTe|XTOb_ z3NoMrGl)Yf509!8B$85$#zN_K2JAO3fp|Fac_HcE>;P`neM6cXkUyMZxI&OT{nQ@N zg>+t0pWU39ul635fN25Fp}k`b`+5@b$&;A^k!D!3QLm*d}x8K$V1zix%h6OA?e8 ziF?6`2>4LnGT)`bN=eP;CK~I-CYs764a>$a_v^`tba8=Q%X@Ff>(=8f&ne$&_P-nV zU0g3Of}%g9R2?o$hNxLKEJUGf#`#cfl8-#+HuOy$O~HP-jNiP?UDWQ=Y)~(7yIk~j zvTT_4>u}cDX>wd8e}cPk{q`1kT#F0)sB4lX#)*XmSM;OBXMkUo5B1sQHtQE;$axjO zPHH8>bi)uecAJnn3+a1{qw*Fa#ckZD=b|CZj$OqYzxkDSRNUvOP*Tj$@@#IN6=%(h ze0D7GAy8sk=q|IwO}hXs@S%{VD}Hae+etPl$>614q8jL-oVG1?uZiVj+HVPEb@{`W zY)285%o(cA5&+Pa#KKZprZq2jQDsj~*euX%Nr>qC=B^FyN0cxD<=_uDG%X&wuvW;& zTJBYrXq=VVRw=|98*pfrpCK<+sH?1B5SFK{Au%7PsL}r<)zR_`Gbf|7hxwUbewJ>r zJ|PF#fQ>?_Q{W$KD>?zLFL?%zSl-5zqZ?Y4+g{jlX=G-{(yn@KI$LCBt{Q4udrbrT z8LiQ+3(Mi1B_{?;Zr29VTNOS90q!SOqg}ajavRcts^hB$`^w!Fet4m@J%LFgIG$0B z&xr8g;@-N2dGpgBCH+dZolF1P#C}B`YEB?)(qedxY*xa+Z}}RQcF$d66N@{Y8+D6M zSde%F8Im<9pv-jseUzFjhH-`;Qr)Inw=%|REe9-}8;cmiVbjkgrg{+hVQfWDqi<8E zEQ{qiY{~5d$B#OfVh+;7YzfFU3L85JxZTpSeGU1Ep}}p6hWTQFs8)s1*`jo3;^E=p zd8!&JBw!0RjxKkFTRtuQuJd!!f(|=9dTm5ujVTpwa%b&i?#~nrpdx7i42g)Tc1ny6 z1)>TjumjlN^coUuZBcFn^&xPf=qab{Ram7G2=hqsLwon?==b<`uOu?uL(Ae+4RCpV zZm#`kY!l|hy4LhOr$8ETGij0yGx5|;YJQoM*Fq&!PB1P`p|#`;794cLA7?JvCd`JG z1f3`-J5K?oGFjwRaugCUED+5G5)-h-BMb+p!lB(Y5JX=#8w~HN+4V&nnXMY)nxN1+ zB)4+Uc=k!BipB<&dNT{T-v!@5^?QM`5ZbDc`mB8gN*<+4NFVcpLO_YUdnvO_zU z5s{9~27Qiwqn`kBy~ho{HLL>8)1!;?jK!lw_h>{@0-`z^#?v><^4{+!Yjh)P&NJAJ z{VYZNg_X^z2C=Z>mW(PaZjV3WNLi|tqh_=cN(#;rPMs!Bu41D)S5QlS3P%FM4L4W? z<46X#3yZ$nC}0ew{A<#|e4}qXo>VjVQQ#0_5r0y`TqW0}WFHn04~i77&1zf_Ctdr>G?2w zq4U&Q0_Q`U#^nkY@UGqO+6>IUmryZWo2BVono*`v=M|bAI2_B8(8sA~lWVUg=#&#C zOJ?px6!Qa941$V98&q~U9a_@6(Z;yGAu0{EcKX8E z<^nS(8u4nLw29{lYB{Rys){N_kU3rB1EE(EB`d1!beY?zi|AS~5F=51@0T|Gs{<~bsRT~D@y8;JogvH)OTQK)%ea1!toVdwukxpn1=fb@A!X9zXiA%;>+6m7l`2>er$$wClk+x9e zR+|BhIr-6dM}_Eu=#eY{^%mQS!oknvV-jO9&QL@TMhXX?ztSDyn?=@a8?IVAiF2XpHakPLDb9^2>Q<9Ne{lpRNW3-|YIkafg*;H}*bc+c$D?bSzNuo_xS zW~_@CbJiO4#_$C~j_l>nC{rZyCOgYABzV8r^OAg7@R8E02ls@9OkA2(iTgG=42`At zrnfh8O*t4middQ!pBFBUY68@$4n9eRk7KrIjDvpq;`3FG--5xg<+g5|7*-yWpj@E5hCwCq|}_u9;D zEsRs(F*Xxv*vTBzaAoXgPISQSdpS@iJ9N?F%SvQeyD`3-8xWo)BXlNkuK-sA))^>~ z6t}w?q(7cI;t8AH44WEsm703IKSntony!9RWt7}7kw>qJs4j1aY$C2iNsn&Akd(z+ z*mJCC>#Y<7bi@w>J>8JG&MLXi@BGaJ{;QAn#PG5`E%XJXc*s+44yx zr^|Rli2G^*aN~?JI-G)_&Bt(~cGVG(s5-b>N!-9;Ce>!1Nzsi0B$QL9^-44dRT;`g zWtZpV;}u`#Ftcd`cqOZfE+N)K+>v7#KyVn1i{dfDk%FaU1x)I0SVWy|8Jv>SorY~T zjF3YgKgh742J)@&1j%ttC%E;2<#9pec_2c^Wq^dlvtO=6_@yvl1`@2Y2}6q^_fnC; zZXc0rK@)-?19zkO`MJiQ*!|tI@O9Yq5t8q<7x_0PYvP^Suh{S!`sSZ`FNT)jN z%=y()nL!1MA%>cbV&wX%kRGZs<>%e0qI(D z>-m*}a^&S_P0Z*NG?77s+<4aX5v#$Q_t0Ih_Pp{CEsJr0)6J7~Mi~WyOWmR>=o-i? z{!!=H)dWJtn6oJ5*)ab*h^7QO5~axZ^C`XN$v|U*#-ZiGA5LfC)Lk$0GLn|rj-9loteM_oorU?Jt@2)d@`tLApc<7d|(>q9m$aj zP%foQWui8jV_M>6wTwBH$&yMC-q&nsMfard#AAHwpN&ZEOlkBU_L;aE#EuU9mJtA4 zvub~1V`j}pjOk3E$K(BkW=C>}6S3GB90GPQax-yxjf-4bSV&S?%4XEmP`8_~FgNaH zYcEV(+{#E&3EtmGTC%D;swKLPO!`PM*itFAPOrAs%(2!ivDPfG)vU1POjsZLYrZU1 z!3f*GG9>_p?2JpT70CGfQ*P9SpxYRoJ{FzPIa_>OX`jWXxX3I~4eES=Ic@s@OAj5n z<9*r|eH`$@yG=vtANfyzJ91kfAf#`db{Y160+(?A-Nf}@85=hGR{x)Z0sbw!-Im^ z(d`FUWxIx@VHncyKt5#q7mq!p77__pQxl!3?)NT#kE?BYfvk4T0puIfdW?|VexBx1 zdXzyJSCr@cX%!)mTn1t(LlGjSwc~Ph+SSE)FkZXuAk{<fi!Sw59mOj(D4*0#sTnWc_IqG6Cq(Dk*pl7)@|Dz)(vx5K(<=o}nE4_CrBO=q}i7MRfi!+Ad1b9`1*Sz5d`>VORTE+00saj84}`*+v0pIso4@q(jZ*x3 z`kv_j_Pj_MJJ=XoF)|Pf%E~CpN(t*K3MvQ-%jgOyi--s-=!(mT$o`Akr>v>CE`suj z7HI|j6N12e{|^{a{Bt}ZFjFVrJV2jSCQ#ZMgv8sh@}WYGoS_D(|IuO@(rtL{kjHjSWy^k}2cqTt z83uFn5pFS{%J#)(*;3tN{p6=8oNDKYi1*$NVY8KEl>S8z3*0EFXPmT|9I-Kjgju3P zWftvn7?TF|_+Raq-K1&X`i+1F1eFJY{32!OAO}OkVl^rytU`y{c>PdF5{GhSibZL# z!UMLX;?-c;XiT*2sJ#(8uPTRMHC)3ri^UD)bG7W9G!nK#LT1I~Q zV;APvHAX}bTw51leW>O1k$>-s*%V?pi`I!c*Hm zHcW^7S&AhKrlxrUSiC%@L2}y%h5iUS!8orbp*CpGnzTmzA)55@o$R=zbxzW;F3a|212sO-Sn)V?k`c%eBpz1sBuy{BJ4ub*}1kCcUw=t5gMr7$}%7< zIOubtP^q(t=#5-`ZHvwV%~Z8dy-eSa#$=U-G|!RapLRN=Q`%9Nk?Q&Bb;WePYazmq z*2;D=^mp478>N7*e23Ho6Su>4oDB zVML{MJGe!O`wc5C#LQj8_;2F*|^_GfgkQ z9a-O=h%qCFH#M=!xV_Pp`ZPWrll?<#*9T)W1FyZ;jZ ziVl?GG+l}rpz%Ly%sL2osROnAnM!p=WkN6hWsT}E zs#@I{b|I8~bj4-ll$CT9ga!ZmAZ04AD}0;AaMwdXLSoYG z5VH*>(KnIA)nIrXni$otK!*#7`=g8oSj2*1)U8Qh(iGUe5Gf=c5WSy(>yDLryK#SXDuHMiI91CK9Sm2 zQe7TP%!{K&L05#CMK#PfsK6RaOB6$U>w6jp?}paE5SnOUPrT8 z+QTI zn@>+@8o6F%4bnUkq8Id?Ds`~TY*!eaoQ&-z;tsl2TY`aK8+6@>stH1gXDQ=HeHVY` zjHcLKaUX&mH@O6Zw^XISH(8xHC@BqxiEAra&h(Zr<8UXq7#|OBk4vy_9n;o9KXA?C zy>5!r9#Ym5RK6+&zT~V+5@XEpb6HE+NON~Ng8q|xwn$i*I(bkcq6~HY3l1JSL0CVk zjiCb2EMVVeQm#^uYEX#Z6R*VF<&<*wOtXD_sa=z!Y`R8fthWM+Lz5U`sqT+LZd(C} z0MmZ50oG0kXSYG@X08>sR+m)qVYydKC>GWr%b&9d8I{=q)d?23I1RwBbJx9dUjplHG11Ksow?kwtY%7I&}P1b z1)U=P2X6gY46Q$F%F7HU>AnfsMb$rU;jZckh~x4ynat6Za_BiOz~Ol*u%z{{+W@4| zOn)`fMu}h*)5s03e4t%9`1ntFa_({Z@DSuf)ql5yIQT!I_pNP*T71Pm;Ex>X|5k{* zE8!;|bdUUHkunq+q7D<+mF)z$y~A0Mb-*PG_4zNeXe$LvZ0ql;DEuGu zqJ{E#;D6GO7~ex6y$!Tn=ee*mOvaM+e)h-GeZ4_VCc*1EQrPXSgGaSdYR!GE4e{oM z%Xt}fJsD)AbpngY*g|zA2TL+Z8C!-Ci`o@{($+RP#oiKsquOnf>37rOc= zq=Jw@!VyA17FH@iPJk+r?k=A+NZju>Pl9r%5JH>n)36nkZs+|47Cmx)ssujrCgmN+ zbPgu%oZLokx^`4kh+7^`l(NA4)3~f6>)Pp7s<)yH#_%!Egq>Mw!jR>osN}zEF4U|N(dzd>F9x$i#Tb?ZIzf-uhGeu zEv;AX5Ti9>3)#T8)i+g?f?{47a=KOy*wznf`tVTR%rCD9mXw`aEgTd(IUJWOKEHAx z{;ga=uXTELtuH-aPJw7%_NLHfS^MDwBuWPyy34XG6+Ej}5^#eOW9SzWZO&IK7jbr= zGXSg%fNq48sWQI08x)XMN$^Jh>vqH{)TveyF@~bky9l%t_tXc~6TB}>{rgxA8h^CC z^)K&>h|#AhT>aNPO1l}X`Kq;w2rrZ zj`|utOl|tA<9n>an7=I3Do5iPDbJE2K->UsPY`Nkv6G2?^k^98x*O* z5QyEhsB_TPSuBSDe8x@!M$pz(EQbXA`c8s+(AGsPhY0-LtynsK!cGE2e<4r7JkZp0 z*i@ol)>_aE--|P~CCNSZ zh8h%Sh!2=u)~gmf?FaGjXi4ZCEG_`o0^$ka>oV_ZsdNk=7JhYnli5W)Z8)#Y_sywy zJ9?r$S$B{64D$o)Jo@DZxRH6*@-F%DHBeIy-omqYmam29_^1c6kBnUP3HK7PosjP* z$?Ky0W-8jaZqs8B3tlG{DGC+Gqe4VKIYGv;Cro90#stzLByo>KU{uWjTIO6hy@JJgwYN-*7YX%F-l*wQzafSj%?nX7;Oz zP362@YA@LCSPT-Mn;%SPlqt`+L#vjEz|1=#>8YEkFP9UxF4yDNve`c-cKm>U-gBSy zN|W1&+iQk%LK&#Z*`Y^3QD489ZLGR6}Z_IODRy%+F%$ z16Tyf3^kM+=W1H5St^iJE!T<^SpzMY9T!4^#UQJa4PmmvtCY!qc_~Gw$L8rtoXJ#} z+y}We@fE31*srrt2hTUT^-oIBbt6q1Ejf+~5e&=}-iHg49FI=4m=qg!5v#|}8HQOz zJ#n=bJ)ubt)3c7Ktudno5@ZNVKm;#A5a}ne3)+fcaILS9Ih5zfzd5-LQJKG{nLxJM zSi7(@^LLVaqSAxT1@~44R;?~j2tMd-J5SaGQNV;~1{ambh}m;Cw8e%QnclDw8_CCHCXE9y9+%t_ny-T5xAw$G+fT>| zAlYg)Kb^Ea3vsSZS{4SPXEU%?D3;S{53VHXl1#&t1u@QH zcZ$H#)z}U(nnb3A!ysON>x5Z9`gk4JdrD>aW)t-(1}&H@F*Cc>FgF%}|~dn(3Dt zZ(bjSKyY4^bG)!=cV3;F-C={*1bp^v_`PB``3WCVGhXz$ZiH-To{ocIVLTl;10o&7 zY+a~mRqqyJf&$VIQ3iGCu)+qOS}HVHN|l*${$l)zR0IW5K|?^O zoGv=e*fCE}C}J?aa^0j{C&X6Z@9P&A3XsVcgzq#{~LUPZJ-v57qi0FWT5 z51ZYx5(s-L9rd%iRtuJBULe4vE+b9KS>iC`Q+%rnAeZJ&k}%1G6mcU-!BB391hi8W zQR_dxnN#6VR~m^f@{o`aXo1iA2vCvhgM%=kOd5s!apK^}F?wrUkYnLanj#aNXd3G& zVC>VBFAtbC()DLJ|4GgqUs}v=6<$=>05DHT!;T^iw1guV4+?GgTcgf&=}A0tiIb1C zQLC_}CxT=$D>gMLgP|f~sw}<|OLq9sS7MWDpiR9MA8Q0HKtpzJDKxc}w zVFZC_Q(-i(SfdIL^BW@~It>%lH{U9w!N`z}gIDsGY{!K0{H?PN8A>Ww~0yB4^KTeVh!O8~(sdduJwL zB9uR|bkk|DrW73OU)Tf&D=` z98(}M4v`K$lwvE=A?B%(k)++t=WQfOc+{_5tz3_C z(0MpUS%m&Nli7lGVZ^E{_q~qZl6rDgila=BO7xY=`ypc65--oEk~FTF#aWj#B~9+q zx9DgDm)#?Tyc5`6XKXlvA6vOtnRl#`nHFVerjV>hPP0er8I%IeQBCTGO-7kkoce3? z*0%)|3mu4kf^4XL#dhJFN8G~OET!|q9f2Tt_>=?%aksk?5Vm{E1n*$ou)9n=IDOYO zoPo_Xntjye5j^Kpt2dp%%Qb>6Pj3jH_nQ&`aJk%|%LaDW^qHX^2h;=E z^xE7c_PlUJd=pY#Tw zgaGnfnT0$7c_DT!KRjFoAF_f^I$>wwS@~>t!Y7lrF>XPyNusm3lXpDAkkMbCVI4pt z<}N*upiIAk^;dczvC2H6MeCd;meBoB6hxfNJ7DXwyio=P47Bh87ratArn3EZzHYeqzhIQi|9;3lyYIGY z*u|rk;eA>=Gy8CkI!bx`8`)0nGbfhR?srqSouy}xp-Q?SE6TUL*Ef{AYMsb z=0JeV>}_y$ovkzL&F>8O%6LZq5Y@vaqDBwPxT+~sglph<_o4#y9phb)>lR)3rxpOrHye4e(Cttscc`y-8p$L#&SE9Z$#gdA0V=!SBGME|@R(K(u2&8`+im5loZ} z#AdIz`W3=%zarf3nR_NSEfKQ4ZhX2B8dvZ>ztFES_6@Y#5D;iHKzkswIsLW8Z7>Lv zu-e>2T>o1AkJI;d-|%l+!EqEYng`HiM1Na>;OrN*99>4YSxoesU`7>=ynV>Vy@ zr(NMaT4ddwXEf1Sf2J$m>9<4qrzW6rD+E)!B&H$Em|go{xz{_4LfTUo_0QPWPdnCJ ze$)Mv(J#VYekCCcw)UTF^$#Qt;lQ_Y`w5NjlksST4km9)*2%S1!_=;xgjcw>1D=KJ zzg*w?KWnbBn?yVGeN|=gk=%Zs(;D>O-()?CnSQ{W==NnD=LB*S66v(Ai|X?_&f&zi zf4gn6*}<_g67*JO6^YH~-2lq&Gatp{iXK++o!{B_I_*9;=NoUHaxh~xcav@T5;{;> z$JgzC?`z*hwTRjT5MKA>bR_6^Le}5WWbC{XxUbBE-;FuQkL z`@vyuBQd>;CDVgBrV&21px{)LmZyQV^l~e`rrw&;#K8&|7ZFRtW-Tt%doJf{mt1VA zy0ZBP%{T6+74~|KIZjb=-&Ngo^OSTOiF{zYOjB`v z^R>#5wXY00G%A?qmh{t15o4Xr=GcvBm?fH%?dre{8Tg-O`0#k<94y~69|`(@n*09u z8=!yj+ZCPd>}(yJ{@ImQs{B*VBZwr>H_}XK*6ZKs2VA>mfvOo4+Y{(zh9pM6#MT28 zpii19C3Ly0_~l1?spty?-xpw_oXnfUTUP!iV-P8yynwdUrt5gMb$sDC71aItdH?hG z^lb#*aPBkWU|x)%RK=bS@%s%vIm`0kEo`ksn6gF%gMk{-x-o5?kCPH&7iu{w{RYai z?`uK4B8k0!7ipRNe?W} zTwXCPCv|bti1<4fpKl5H>*$2ko9F3`CHURphyOa&Wniq{om;3ORgN_HBP%n!I6Pdpd&hqhC6aa?z^yS}<>ebQq6 zdnSL9{8wZjJ!q7)qkId;yY9GV*^zlY43`h=ZsKUInc?gQ_&0ecs#ec`cWj&WSIxPF z!A_~V#Z0v~1rXY3km2UIedUdphjlWX9oH0ls+@WoJ?)mW&$|0{%WIZd%rdM|*LZ{D(r#iji<;X)`^Q*yt?RrrwKa-wzK3eM z#;qr-tS{d+YkbqNyXEH7dGve&pI^VTEm zX~z`39aaTZk0~bbAMhe$EFx~~wTQU&SRMa^_~823Wh}0@ckPqT(RWqIoJfvkwtiDi zLeRS)ujvV$JIg4S7VppLaKTR*omFAkS`((96#u@$q|s$|*}?cj|Lh6)_jSy4v!MsF zGwb}4d^`@EG2EA`Xs0hE+@~pAh>_+lxE%(57UZHmwRnXFMo+n1bOb=c3N$!*i?n9hZ5 zTN^@ctBacII_d*Gs;@V92Un?_V^gL(hflpVc}&k1{i@HCmMvXSJi>U+9k+r@87H(Z z488E~uwR<>g3G(U>wO9!hD09k7}wJsWf)PZ6SuaGxBco=N?ZY@+xU1mVRg`c-&}Tw z+E#_afJhw`c6*J+%~Q#{{G)cYIx3k~RCmoh9d__|Ax68p9|3$}qTog1ORSH89- z?ZlABS^jfYT$--#*|E6`N7wAg8>M~YXVbAaHPn~v1>Z_z@YDQ5g2&|LT)&cK_nBo= z3NgK(l|nxo^jyBtx~r|tq_o!e{8!ejJ&)7Y48@x_cFd`FGT0_MG0X$_atRC za0WHsceSzomfH~8psA|+bkbb461T+@bU)Pp?WmjSdho%V5>wqPLA>;IzjUJ+T5)6b zmUhp#(aNDGy;n2cp-aiN;nv&aU2idLAewJZt}aeYG#m1&NVzM*t|~HhJm>m`DI`q= z>)6E1OloLiMy7N3#fM#YEu2mESl8}JrfeFXtj#SA_|RTpm6f8V#cZ2ll6by#hNb0g zt;u=MSo6vcc<*iF@yb1KJ}D(GdWGku=e?>~OLuI>`31&(%yElZOl7C*kvf8Q@Gq$b z?07Y`2LH`F1|R=yO6)|NG3`CX>$R_^oO=6W;vMg)Z$~w%jLp7Ru%Tyo%a`+&c+*26 zB}$Nfvmly2hMd^x#dkY+^;2&&KmUeLUK*@hp6rd@R2Zaz)4&ab&)uL|3*bQ?4wf~0 z%Wxb#7(^AKy@#zSYy>jKAT>A}oSG4-n|2U^2su$)XVaxNI_;$z}PH3I3!IP5^^y_UpGPi_73p>0~o$ zG6XO15SDDYWk_R=76ReuhbR+UIbA^`fd&7{gYRNg<@sR9|JLXrCY=z(Vg`{}94eWO zl^8R&<&YJy^`>%aNrKn zhVrN{B1}dffFMNig40xPDI#amXC?O@KcLazi`G8>x46=6L};1xH~=1+o`#7k+JhoA ze@A07XmRKUL z;Bb1V+Ks)HM$Yjk204(x{zYfsKTPINY;qShW;AB~#>7 z3q3$YC5Zx=Y@Ucpr&l|#KM7mIgXmb5N+l2m)Pb=9j-^>@0?PD7$3#^M=?mr+;{a>I z2A|j`04%~PHte`GPG1jhq)EMSq7`c_k_Z&jiB7XmK3)S}gmc9zX8E@?j$DdOL0PP( z3SL9QqL>K><^I9gOrF6Q&Tm99e>LZ!I=J|o3bAlB>#am<^povKG};D| zZ(!doI2PT>r#OUy(hs4H1QhyYy=*j%#AZuYTW@;GMSEZeim<~PWU?KoWZFi_?5_PK zsx~mEoCJJ90$O3wZj#A%Vo<@};tldeIe)DqfNcPbwsn59JYYM4O~j)=S*jA;0li8D z?a7x1-D?u@X#TLROsc@SOC>R+FyCAWQ$90c55iIWAO%^Mj6Rhmdk970>C|R0hcUp>6SkruGU2Yh)5_%> z5{IqdckIAr_;RRBhIkttI$4$926`t#AE1S4DgTzRUXe=}kS*Dd1UE0f0&l{at+PEi zDH~>oxqK@y-DB(NQt9x+xXeHpWNL7GV7_=~3xc2)M92=puP zHA3l5A19OUVQC?rDgc4Hy#*$Ji1K>82(@!HB}xP>u5VRQAQVeObS=C5?#*oGT1x(2;~Mqh*R3XQBal!mltk=PQn{FwHH zD?lbdPh path to global settings/plugins directory (default: ~/.sbt) + -sbt-boot path to shared boot directory (default: ~/.sbt/boot in 0.11 series) + -ivy path to local Ivy repository (default: ~/.ivy2) + -mem set memory options (default: $sbt_mem, which is $(get_mem_opts $sbt_mem)) + -no-share use all local caches; no sharing + -no-global uses global caches, but does not use global ~/.sbt directory. + -jvm-debug Turn on JVM debugging, open at the given port. + -batch Disable interactive mode + + # sbt version (default: from project/build.properties if present, else latest release) + -sbt-version use the specified version of sbt + -sbt-jar use the specified jar as the sbt launcher + -sbt-rc use an RC version of sbt + -sbt-snapshot use a snapshot version of sbt + + # java version (default: java from PATH, currently $(java -version 2>&1 | grep version)) + -java-home alternate JAVA_HOME + + # jvm options and output control + JAVA_OPTS environment variable, if unset uses "$java_opts" + SBT_OPTS environment variable, if unset uses "$default_sbt_opts" + .sbtopts if this file exists in the current directory, it is + prepended to the runner args + /etc/sbt/sbtopts if this file exists, it is prepended to the runner args + -Dkey=val pass -Dkey=val directly to the java runtime + -J-X pass option -X directly to the java runtime + (-J is stripped) + -S-X add -X to sbt's scalacOptions (-J is stripped) + +In the case of duplicated or conflicting options, the order above +shows precedence: JAVA_OPTS lowest, command line options highest. +EOM +} + +process_my_args () { + while [[ $# -gt 0 ]]; do + case "$1" in + -no-colors) addJava "-Dsbt.log.noformat=true" && shift ;; + -no-share) addJava "$noshare_opts" && shift ;; + -no-global) addJava "-Dsbt.global.base=$(pwd)/project/.sbtboot" && shift ;; + -sbt-boot) require_arg path "$1" "$2" && addJava "-Dsbt.boot.directory=$2" && shift 2 ;; + -sbt-dir) require_arg path "$1" "$2" && addJava "-Dsbt.global.base=$2" && shift 2 ;; + -debug-inc) addJava "-Dxsbt.inc.debug=true" && shift ;; + -batch) exec &2 "$@" +} +vlog () { + [[ $verbose || $debug ]] && echoerr "$@" +} +dlog () { + [[ $debug ]] && echoerr "$@" +} + +jar_file () { + echo "$(dirname $(realpath $0))/sbt-launch.jar" +} + +acquire_sbt_jar () { + sbt_jar="$(jar_file)" + + if [[ ! -f "$sbt_jar" ]]; then + echoerr "Could not find launcher jar: $sbt_jar" + exit 2 + fi +} + +execRunner () { + # print the arguments one to a line, quoting any containing spaces + [[ $verbose || $debug ]] && echo "# Executing command line:" && { + for arg; do + if printf "%s\n" "$arg" | grep -q ' '; then + printf "\"%s\"\n" "$arg" + else + printf "%s\n" "$arg" + fi + done + echo "" + } + + exec "$@" +} + +addJava () { + dlog "[addJava] arg = '$1'" + java_args=( "${java_args[@]}" "$1" ) +} +addSbt () { + dlog "[addSbt] arg = '$1'" + sbt_commands=( "${sbt_commands[@]}" "$1" ) +} +addResidual () { + dlog "[residual] arg = '$1'" + residual_args=( "${residual_args[@]}" "$1" ) +} +addDebugger () { + addJava "-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=$1" +} + +# a ham-fisted attempt to move some memory settings in concert +# so they need not be dicked around with individually. +get_mem_opts () { + local mem=${1:-2048} + local perm=$(( $mem / 4 )) + (( $perm > 256 )) || perm=256 + (( $perm < 1024 )) || perm=1024 + local codecache=$(( $perm / 2 )) + + echo "-Xms${mem}m -Xmx${mem}m -XX:MaxPermSize=${perm}m -XX:ReservedCodeCacheSize=${codecache}m" +} + +require_arg () { + local type="$1" + local opt="$2" + local arg="$3" + if [[ -z "$arg" ]] || [[ "${arg:0:1}" == "-" ]]; then + die "$opt requires <$type> argument" + fi +} + +is_function_defined() { + declare -f "$1" > /dev/null +} + +process_args () { + while [[ $# -gt 0 ]]; do + case "$1" in + -h|-help) usage; exit 1 ;; + -v|-verbose) verbose=1 && shift ;; + -d|-debug) debug=1 && shift ;; + + -ivy) require_arg path "$1" "$2" && addJava "-Dsbt.ivy.home=$2" && shift 2 ;; + -mem) require_arg integer "$1" "$2" && sbt_mem="$2" && shift 2 ;; + -jvm-debug) require_arg port "$1" "$2" && addDebugger $2 && shift 2 ;; + -batch) exec MN43JQAtxORf-@6S;D zJ>~!7gRuVIwR1G1H?TJ_GB=^OaP^>!A?a3fY56T zgRg!wZHo+wL=FhpB@&kM*6OdN3E4DZmcaZ#@#}+%O#Nw*9Lx8tkish=4{tbBm63Cj zv#`MHWBT?MJEI2zzbgU_Mo1G}o3F3XhXNsoMnmmhLXfX7(pM3{3ULO4+QU`G!leQ} zTSY{nCa{rGAQj<@M!1n&AU8z1p#-y4S%Al@cuYmNAA@RukoaScAy9#Gn03-nZ~$kR zg(o0?ml2Y6lDL2{!~g;A$2`)A55i3GIvEi>s}7e?6`n5DxJig9Z`KOCQ%O7X$6t#t|+7}5sFT#C4! zQz*@6#sx#*dQ6Qc6GzNhGnbc+xeT0M11J~_y9*H$3{AMaHaynj$-`$q1^le(Oj z>$BlaF4{1)T79>~UrQ46o%T2{!wR(Q*Ica`C*(1l+&F~7#?j+JepHiKht`;WZ*I$F zX<2A2w{6$pAHuLH2bY=IpGFko9dp@kD&xXrx*PbFUJ`8Spg`s?KwSKTCH<+0z8vN^ z6>!|d*qq=H9zZHIk7ml%hbwi?$fBxx{avXX3%<&a9&hg6lu!DqtU%UzKgS?s+K-fyeuMagi1d@g}*x!K}uUQc#oXya*K`W((Nzvw=I_D2Cp;QVNaJ(>g$g%H0&YI*U? z$4`(Jz7#C^YVSN8C6)uB9a9iBegfBkm_j%=uO5O{7im@1 zpUgw%R-yv}Aipy##p&}7!7qAH&Pz%l*&^uJV))pAlx*P|Y|$K2Czd6;AY%}P$Z+Z~_E&6}_e@6emV?SFR&K*Yu?bEJ)%$hw>ti@)H%qF8qBGVQx zv-xK}Pmzrxw)lKn(C`izS*Aq&g{fI~x(v2}2$Bc{fCgFw9JP!H)Miyi@<0KC>ME5U zRpBQ@?lbtd`}E``Sx0&m(0e>)Bq-%PZuas+in6KpP6cPd$!f+gk1Id2Ua2^R%$pUfYp#+Nf zL2n6n(z>6>aHMtO$i)fjG~uKP>Qv!O5=l~q=_Dyq1%`O&#FK(X2K7IT>OzM&A|o}t zLxv7%x_b@jBP-Rlr}i|_-*_l`ZND3PrwxtpQV7Nkk@HaeDMi*grm4Qu=c9Yj2XXb3 z9|tRb{t+p~V@x&LuMaS}kwDLV(ues@Gvq--XLQ4fu2$QcFvNslGAVBI?Y3;xl38$_-y0?I*Y_u4d{9FK4`J)~S{~-^t50qxG6t)%`qwd)n z#PSA!`F#hQ2CEpYOBI6ztqD~;t%5RK}tgvPI7rTVggAev$=W2=Y#-Vr}R8 zOWH%t-s=4q8Ck7gwKQBQ3{2DyAFvzzU6y7AjVgN9ZM6ml`f4SgLulNESY=?N`l|0q za$t1%v2AS{g;?Ql^K`Ju*cFGZ8uCH|2qV(`DwmMxT5P8frE;bO+@#6DEYD2IZB6M~ zf=~x)ADykUs@8CDvFcT8rV(@h^b?UIy*L*H`qEXJwXbp)Sal==<1#|BmtiMsq@Kdn z^t6u#sqt2MtVj~oo%D6P8URpxHwm~Sf{8kO@Rn)4PCMZ19iyu`#ED9%*_8v+MZcXF zwuOlL^dI*&taatFR3;GKc5z?(q*%Dd|h7TZ9z)&8S}@@P6Dpsms8gd2#<1y2y%27FtAMw|1xCz$vaw zqFZecH{y8ljCvbhC=_|{mt+lkCOKxJRD^^wQ^o^A6pa>fI;e*WFJ2QkxD$-ZZ26-f zHU}-kpIa6>mYvOol_yZ~_b&bdl-1*pwqdQeVL4WZWLHXXh`E9iPRXyz8k$)~auo>+ zkyWvXXQ5HxWQ~b|Xk_=ASs}*~6;3)EKG7Q$FFtjR8elC=g$)xHa4qs4OL>1Xc zS2_<2JEbNXJtf$#cj@wu{wY5y*3B)sDyXw{p)r3WiKY-OZMvxkEJRt~za`hM8jE#> zul6+}rHureMaif)K^xmnw7qR11M zG(Cfcwg4g|M5~a?CTE>3|EQYI!7HIH#K}&9Dj?g$RX0G9rq@(90*-#8!`@7$f_U0w zREO(C*p@M;&`#*mo!G3LK&0L>ydAP3th4Sj*$1-(6H$iFb0Vv{wKyR6XE~{@jBoeX zqE|15gaiQdkSlPc5~hWQ`^RQeltdH9iU-+lJY~v>53@hIg-p$;t(Fx#V?P3fnoOqI zgL^C5Hc>R9xVDf%$-DNsi6*#W*uizk?e=|<)OknNfv3;fSCGJDr&i2kvWeFeLkyHJ z(;;Zcgkll>&98x$GVRqRdwp+DMN_K8MClIr7nDPDHiijWks1=&pVfIheB61f)eKZF z0F_M>hc%QrBFX}E(RmLgq;M?a@;RbQmx>Ohhy)8*QZv-klPHRL<^(!yi*#BxCc7B# zfp$uB2P4xDSL|}JSVrPRTcrj#bQ>y{+0BlNHD4}H23bE`+`z!~=I9tK$cow@L*G79 zZb#resrIih7XMry!P>s`_*H%};WfP?xDQ*Z?Ne4(4WX!Zz{+X1re23F)$|*wej)H! zJy>10E=jt_Xnywt&tLvaxU9DRy?|5N!jvyVE5&^Av;9(li63; z!MYghGlcvBJ86JO*6a*7j1PQyFCWRFg})^aWvF@E;}~#bm*t2cVp0?l-NG1vDYB?% zJuEs2&ktw8j_H@mIQ1Qt+u8WRHoI`cR-IsPHe%0Xv5YT*X+cGUqqWzD;%*0jc5>2M zNKrh8Rz3%FKVpx=%es~_KC0*jA6=wjWt9?dDh6*QvERa4Db9=(n9FizJ>@umKU5n- zD{BDrK#^khqRqG{SI}_vlY8Rzt#8*o*fAM?Ue*vzm{7am2SfpQaew5YA;{?<(X$Ap zEQ-`D)n9>}R47Lyq^?XrrMiQ>H+xus^%lm>4w$hepT4f5ORgMP7d^Nz{no@!SB5<@heL z=Y4wzdV42Wpky*`suJMlu^gkM#G7T|0AX_@xz{_Rcs&x1Z;a|ii+Jz05&h~Dhm{tP zlp)c{ALB&x0OR0&&2S9u#)dZ>*V|N95Ncp*bRnt+CgNRgRer`G3ch!|`Lzs(u$=Y1 zXr^3Zy8LAOm@6RpcY%BP8OX1L7YB;BZBJj-0=hnvavkNL1tZ)R7*G^(PzrR23dn}7 z%7YG=O-t{^jCBo6mYk@ZZ7rp_JsXFpz}zAt`Rv}q1x(9{D0N%fTSv9IzE7u9eRd7A zkOanHWWyB;T{oPo3}Js`oKmx_hYd*0v)E@%!^_)2O%I$6Y4fnWnmtu0b0sns!P#x2 z9x!JgGG}^-@zC-`Qzx3IJD&&VerEQ!Z#Iw67QT@++-bJcEFT0iULbBHd1uHaUELB{ z&T4O{5fohHzegO#KT=>YZhkRTA?k{-UVJL!*^-hu7IS;S!o7Cgk=4E+ElcMo{j%n~ zBfyF&SjprMxA_R2t;K&Sy7@C$SJ(4lKvST?=JFiYRCUkoz_0DYS935GS$K%#&1YAg z`aCDthNd1XV@}v(!@sN`0zpUCh54_Kgbjzv*O>|F=#yvU4=~r&ScPvvoFccmB6#E>_cW!dU`* z4kxhQnl~npPV}!Z=?BtYQT-w$16g0umsk-@Ku8EAO7I_P5UXrj?wFYTWgXZ?LZa}J~*j}6n71V1MkRFRYzS|(V08&?zog_jU|U?=a*%ehl!1{ejo3`z|`d?(D5J(&Fr`HR9&nS+-A#OMmuU9dmW^+#Q2&?GO9_g#y1ssE4L`)R%XmryC+VQWOr=`do_;9;b=v& zIj^or-{t!i$;PRD=ducV1;hOi^Poya89RRCU7O%EcTxU{K8M9Qy{Y@yx@Gq!%}IfA zP_aX&_hX}@tY+!m;z?M?)n;W7vB>(kQ*o*7kb=?)jLB3~IprY(Qh8hRXxp_!xF0d% zr{B|yU)$8Md)@iUX-?MQx-6KNX_$mVmgATbCIAgism*#-0nBoo>WDXEMPThm0OTee zcU0`&vOW5N@IX6!-zd0hRAdWnW!Y~b?YI5~;pITwG^Q75_m9-%b$KRvu{GT{#A*(? zQH&~CnV)K3p7k^oHy+02pV821YCL#Y$f!$)azbzr6S267@lH@PDLgQ&e^?yUpcPV-ISE1hZt#w2 zHk7zw_%UBkk6T%(Jn%@Z=Z?^&_NEpfptHOCCz_U0gx&=+K#&RzGgRQ0?Xo&m$3m2F zNCg9--pEp0MP4Bl#nQVqMRGW;>_;10t766GrDpKrL5NABy_zz(EzsFef5J$ddtjC8 z#OOW#B8F4P50RHN|eJ5nbz2uXVk2BEb z;0fHjjTc{2GSyhy^`FcaF3AM87He&aYBN`Fe~Mr$&k^U z^+fLuu93O6c3b^(3?wByWa>p*(`}gS14y0-fp0IoA$VCusr;3mZ3}V)v&f#!Vb0Nc zW7}7#2)Z>`=Z5OH^n{R+`hT7VSF(x`yyhxocG~)sNj_GrUIl!3$6Y@d*WO#(XArlE zYRh+C(b8cb-664yWe^|UMv0{_Hg^7u#gnouG7ku_<3^(}Wf#hV>U@$yKUWvu_2#G!m$j+fizgC@Unno08Ryn=C4pQ+_4oX#pd zrT5r1EL$&FjE#RKddM9qHTvg^+^s5lrNc5WTC_J6TTk#{aWZ6Q5LQhc&*yaLKgClv zw{-Um^4m8Qr2keth5vK$ls9m8HgU8SH?jUZ;w6!Dv@o-`B5yny=RHi}ny!Zr;lW<3oi-x*kRcvx)qw!m{fCCIB!9%=T7Lv+HVIV#r(OK3P7nc;55I0;43EanGLauk$ z9H7Km)ZdE>S>Pzxg&$@)h?`{qDj?MCXduLFob7QUbZ2z|Q(3f=8rT{0c4=`)SQ^t+ zu?<8TjdmIp81^^^#)rjtM;hzBpflZ9UF#i)?TTc>TQ$-mO0C5PQG(0))^7)kMg-T7 z0Fue8fcO@jg@){MzOnCEDe-8Cc4t{;nbbV7RUPxA<_u;xujV5PuIw4d;z+6@DB&51 zZlY#Yh~#pSNi9dVeILGLD9`pS2@r#7%gAOmbidWVzSO*DCWoM+-?dde-B0+u+_gNW zuw+*(_(#0C_|$Wv!van+nv=zy&^rp?a8h+gn3+Y#v0W#BStR@DCA~BD->lsN<22-bs^YE`b7x+DO+<6|H#tG<4!()#s#_QomxuVgU`M7Fci|llGtX%W#s)q{G(P!DVh1NRvHV($|$D>ZC#ZT55 zTUBYbJIBBTbf>|V6*gZL4GJh>nKRuoxdovU$8j^902-;-jitjh&R3LmZ6&!vlDS08 zr6Y@SNq_w*on)*?@#KAEIq~X0l%LR}D$nVvHVww;1X^dNgK(R;YeNYJwXMzQiqvZw z`HrMvv%(Vjm_?!0o$ia7vuzSDV@~i*==D2ow{WLWQJM}lZ8NB5F>BNJnb|mU3-r)Lw4`XZd1#aZ5vIo#+iO zlFxs%ZD};|S{``2vU}DXh1QzX@l*H5czMaciVR;n=`IoEdpTgZ=P+1)0ASY9zsiu7 zr&pER0{UGzWxLT`;|k(Xo(Iil9ujTsD+F_NIAt~*f?J6}TbNwxTXrfKDa>m6z zVYV)r0Tg1dNb2d6`)E8Gl3WX#(VrSOs?aZQEO!Y10d^a9={0O+s*}A{jPogSN+HlF z&GQdsWO9rcv&O3Mq}+vFt8)g5K~@)<@Q6;*?$3=APZ|xeuzjE2$wy`zl61ePa_*1Y zTJk`{bMfC+=yk6>j@UzcBt+iv3&)Bc?B5VqsY$A zaLD!SfPO?s-D7VRZ{Og30KXs0X_xd|13sc-PwzjiTi zlTQQDycv@1!DT(+Xz1F|V|%;sN{1Yh{WJk{A_@D#f@|E>s6<6nXN|f=#eXl#S~7q_DIq=IMeHXk23Xp25>9=D~tT zl#6AvT)`QrTzdHrur?IKOdCY@m0&8ZeL#q3Y$)_SPar*aEwmgoJgm3qPHk^(Zz^~U z*AAL#&8VkCX72^48}tU{H4Km(f{(gaveKatj1O?PO9q8LVa~`$cTEYDgX1B-=7dVw zp#ygO?W5eKcO0Nn_V9qEIMyG;pt%^&0lgRyil9QEVmRtC-(fe-fAW56?ic`Lp>dEn zj(cG@4ubIUQJ%HPL7(GLSf6WqyFj1dcrJu#@lf3t!@0cY!=0}Edxb#1!{H!3m!TZ} zEeq@p(mVJNzI4~zz~2yCv)71Q%h$Q^#zRl6^}Cs%+sMyQKzu9Xkshm}9d<0vYm{h~ zUFwnFERMUaqDt4i(ItBj(Ho!(*J#lt`)63$q&0Xz%;ySGW)o*N-=aBd%qc*(q(?S& zjnj=A))b1*orB=c6@)LvwH%hmO@SzayKZ5!yGWtRbF-+GaGwZh=3v0oGfbY8U{hAr5XSEpen`JF#+j22R&wP=vCUtZ#b%z@(_!=RaZU_z>sc~HV zSG}AZS+wx41o?w1TQcbfRlBvqtEIC7&tIjs6V<$;9IJ71TPf1QhP=`_DWu;?i09f( zmy5{vVRFiTFxP2ZVT-MshHVu~N0NJ<-t+Nxu9rogZ|Y&hkVmy|x&Qu|wB2#amRYE3 zWLLzG9Lws)nwHX>=}y^qTD3JNoOWV^Ww+A~ytE}Y$XHd@Zx<-aG+D6qMa{_32+LWp zykpCpUpUYZ_dXpBYCDvP_g!_OO&n>+HOx2JQgD*#co<~c2LltOv1W-{&SCJ3@;N^x z;Z`i#*t!9tI$@v}U_^NAuta&NqHD(zHx_*JNkETMNPkfJL?^q~`$U($I?f1R6>D?1 zdL5NE*6T1HbZ<|mqK1{0cBICTm2^%Rzx=|d;#Nuu5Y2vv!)B$F%r``1)5As?6K}zj zwUrR|B&rMPyiI-5oCS&f=&|n&B6Bj#WzG*S-O}(PY*SkNpey9h3cGipA8Tu>-NT;t z-y0jbk2hUN1$}Ci~N68SiA0yU z)0B%DZjiv9ZaJAc$9ChAK3pdUV`y>t)=7hLiJqpghD4fBao z-`zn*cm^p?w$d48XOEl(3qh6$D7nY!iK??JY2E8?iD1SylX%{%7wKWZJ-1IMMMB-1 z7jYXp*VPAYr5y~ev&ON!dUWt)E)C-Izb~Wr%!f4%?{`{SJhEn&pSY|&KHYFp#jZUZ z`^T%s^EP{bT}cc=y{5UeY;7A}%I;2KzBSj{RT9^AS&OTYh88hx>0n-=pgfedvTKx+ zQ@`zflx(l)INiA?8j9h?@=q+}qC%9;aroi55oTMG+;t7#>=+HTiXGi_#v83qmNtmR zvXOa0l8h~1!D0JYsHMg*I5_M)Cm-o;kHppMW~;Wk+t-T4Xc-aZU!}^CPTv7eMZ5gi z)xfI(`#tOpe-8NSpQB>RiH+{>yp}vACK$C%V^R*$^!x8&Bn^P+(!iY+v4>CN;r%SE zI+3w)9mv>$IrpA9*2ltiJXS;AJ^o_$%GNUWe#!Hbtx<9^(-KvE>rT1SmJ0c?$r|6F zr9^{DNes2IKxc*cyHrj<(^XAXu4Z0u7qIrg=SiWvq}wk2&lVqDChUX_>w-pPY&lK( z?@BS<1-$U|JPAvdCIdL z1wW>Q1Ir;ZvjdY#RO4zr6JnAE#b^U2%1IWJprlP=rAQgeS7@$8hZmFpkMWf}V;(QI zDv5>-w2QN7Lzol;!f$Y4YUxQ7yj zVg}(;2mEBT8R#)y37nlw_u%t`RlBwO{zdeD=8`@1b%b)mAt{!JKYdc43CEJ@>6<>c zAB-UKPV@94lo{&4ivQiPT<1k`U8Y>mqzK)#NYckwX;=;j%0#I2N6!gcdII(BJven% z=26dVh5ZK7=3Sf5HV|l_Jz$EsuZgnyz=~ISo}%{|A+JBNS@K7XIpgD(2H-1_kmj_` z<6KyIRIST?6d!ycAO$2TFALMx1+{w?i4M^cZ-h~M*8x6|+33d`bRV+Z7{gu=SnE&G ztVulq8wz!r+^AxvFtqsN9Dgl`mRjS1x95`J~GybP)-GJ=r)c&4yHlu zhJ$CC@ZR__)8t9m!O&WemV8{d zJSaEA@0X%DPYe$Ot@uxS3LBvUcnG^7-eBMN!10Fp_ldJ4p~_I9Eb#F1J-w6Ib`|Cw z!m9g?oZ0b8WA}~jQFHQ8+!K5wN;%xPuOp=|AzP6~>vB)IBhU!HkPOA@(wrRD0nSI{ zR|_QvJtia=#w1eSpWz}oyWUs5Ru%xuBd}xrQ|j*h+PU2s!ego3h?D|PY6u%v%5x5X zx%Mh31=Fe)yx>$9Tm5#{;E4d&9`Fk;I6;BKAFhx*kEm;c4;6=uEI1-A6g#S5l+H!= zV@S;*0Ym*CAaGnU1yo52hZW+xL~;(W7$vnP&zqMD))Hp& z<-5;RbW4ZQ2H6v*hHWmCK^#Luwr3zoch0b>uF$re=h&-mpbbAlS2)F1pu{+PX9_@P z45)HOr1|8M-9hedh`2wkv`2fU1^7cmaYjTDE#SJtv)#kN`~nts*Kt0tb_l&OVs}h% z`l+A6yGO8kX4ZWDYxY7XUx;%)U_y&|v1bb~Rz^&;wL*$Z?0K|=GIbh4<7=-$a#8H3 z7(<~LlS4t$#Wk7C*yD_}@a(g*4JxN^?Fr2d93P>%tcJ5zqC&Mevd#jLrD4}1Pm>Eg zo65Rd%eEAhubjDgmydGt)Ob}}oL))^zK&raD_5CG06@!6ov|L=ckaHPiL{YZRm5Iy zuBZi$DBBpxhKYV+-P;0b9w$npd23azOHMi(MnTQRm~t+dfqh6x>Sqy?+- zE=udkgo(s-_A^Bh0Bpz>BeZ?m_&pgE=jNBcH@i^eoeZ|FUFe%GLoScTs zuX_SCi8dV0xIEFZ0NUTuFm=_N=~%152t6Qn2fL-;n!Ur2zkGRZq;ppNKpTdwSK#F=17_yDW+&U z;k%s`$PkE~m-fsuT}WgJ)h&3rXHuDqwE*c7jXZX_)r=@xQ zZ9znxe^#|a=B;RVV+c&PnYk9c3z2DS_~ny^*2xy4`d<=!I>(x|&C35&s7*S}m~u~e zn3|2@aay&v@uXdnvh`@gP3ju5dxcM(la7niD~LWJtF+#ENwnb*x;WFFTF=kf}ShweV_<~uU1geX& z6N$4M&~8SlwnJn)Q*R&H4m>z>+ZJ-$vu}obozU?JKkZnwe-90}L%jdS?X60udy&QN z7_56tz1{b_eO7W}|Gmq8g?!0Y#(asGB8T$~TDAHfZAyCg9q?v4;gh*?&CPV$%`~m$DOK&8*9wh* z(wu6Yt|*=*1ShIhS-$oF78a;YmzB&#v5jZ*RoVcBv(J$_L5*7tIO2V7S2bEtjK$3P zO*G5qVKyhCpoJ!oIZecNX9mXY{)?r~<7V(-^H}*OCmfk#oWvAabqk80yK zo0T!!M=Py7CS;^N&CF(+0{QKlt~=lZ7gK?Hc2XO%upFl$^=WVTaa(KNFh^ZKv~=Po zDFkoUY1La)$^uIT3cc~l)|1=0DLkN z{~v%R4yOd=K)!tw`MX^BpVw-c|HA}A*3Qw!z*^D7+Q8Yu)kMYd@1>KSwX2Ea|HMmP zMy~%Sl8;pWQpDa{`}({B1Vh) z8xnt}q_;X-(to#9`a0Vauw>+oJZ~OITA9Atm1zE zUE90sr^v3+YjB?%-?6rdoqv5DnHxS8ODlQx4+603NE3vA5!n2D{$~Ve{{sU5@xPB& z)RIFH_`$2xtfP(5rU6dBpa__&R)Z18 zq>JobsHc1^$fKE7o<*HYkT9@!RVKb2YG>(ImN6eEQOWW76<3ol;j;fxwAWJ7y-mJW zy%=1BIeaPCcmkn24Q4~(5-%W?bijZPWH@YSdaL|^@_oBWnUYANQ~Ox| z57dglE^>L`lPdP3S}XD3rVU5cyI_8FY1I&7)_Z>erN-G`dA}KywWSH*b&qCAu?S_q zZ{H#cMN{04cS_=c>1_Yknr~BQzHZ&$IOzSa*+};vV!`NNJMaI?Zuz(U^6x}UR?$(y zR{7zZ0$v{!9CtXNxQ0x<78f+UT%)?CDJfMf0|wg&5m_&hQcF5z=(@0HZq`{>$Q%wUa4)a|u(#B+7z@pP`|_w|bHZGMp6^_5HU`+Ks_-D7&}&LZ~ydrh{-;7?J{^p)bomCL!eeH8F|sN-R3C z30fME7BC>lLn_>Z6zIFgcOr(JqiT;h63kId0K^F5ARM%SkmxMZ$DyZ6>otL>A*|Mi z6+_cTJ4Mh|W(ZY7Gv^@dL&5uzI19iN-&nM@y2(nJ85VQQFhP)zr=AT;pFuwV^^N=@ zV7K?)X(_&;U#ls7C6YHLrquS-!mDg~sco-6KsRro+S8efUlFQCri@G+G;}6Qy~_+WjnldDgaN8`_#UV4ZX3ESC^q)bZd9JY^KM=G=h^iXRgY*PrZ4n zKZZ7O&FLFV;n(_y^~d9Vh}l;mw(e!ZA`aXMb%=>pDQV-7AP&5jv4R}^2_`5l_++Cg zwrP&$0wuBCiW$kh@*f;ec4KvzTSpQXX%q1UJvxhy zDe4?e37JbaN7VXD+GMU1rc0pq6J$;b4d_xQ6Co~rO;2rT4ZslAY`yVJw^5r)nvg>k zIY$WP3Ryw74>Jop8~xj(+c7qPlZdwvyJVn$}L?I|A_l!^OtB^)@HOz zpYr6mm4|BR2ChU3vmV~MAL@eiv zbJ7#oB3MkFTZ=@YH2AZB@b-R~cbKhrkk4}(??+I>D<#o0Joz=J*#{i_zEsSVT5C|v zYMRRAp4Ra*qQ*uk#^>y&qoZx&Ok|YW?p<}?Svu#^R*U-Sa`#Jpf09oQ>$6n$*+w6j zN#)C?bVoYzHE+){6wZ1mM+|G!4%B4~=l1Q3-@govDjOPZ6Yy`}U}64SF=P3!1Sa$k zjS1Vi*;?Bf82@kjFI!PZs$U<~SGROJHu=$@#FPMS5{>S-{&@IW^0wf(KT?01J@c1L zv&DUGJB4@v-7CmP*|D{+fTTYB^vHlmj+5*3+r!5V%+3!Hz&g>bLlD6$S;{vd<$fw; zWTus*xB)I95xU$??rMHr8TCTp4=I))gBdL?o}&e>ZL^ISZs$xNM=MXRQs;R#J1Xk|>C4U7$(4gPm% zlWgP`jayv{;tAO?#4@UV&K zrW68Qi@7AC_0lNLN+oYB#0y{?7XE{g|C^^ajbKx=c?HlH5)>e(*%`}zy zPEOf#*ARZIO$ZH$Zis&^3#<1awOHpVj<>RG7TL3q6LT{DW!zR5o0bm48ms z7IgGD6nZn%F)h~Yxu0_k51R}7LYvjs1CTsI`J2+A500Of+0w~+$<)E~nVVCJ`F}++ zJ%eOQl~PB=)sH>X5XA2)vGx$xq?ZAfaJ==_UQexOr_Ey&F$1S>lEtIujny453@WcR zfidzS4T-483|ZcxEVFAkTCDq3`M*2&eYP0n`3A`*tLfY%)S9cln-1(Xsm{@3Jhu;u ze|)VKKu-5q?gTC7%{VpdWm{gg&!l9-i6Gx>G7ZjX<~9bU_c7=jsx!h}FA+lqwirsf zQ-_4MP*t&>4Lt~jf`D-cZ%&|_seZ`%SlT5bpQ=f+QC8QJN%RW`g;&Ir4RsU5&FQR2 zB-W@2FH2(`IF*BqT;b^iKr z0X*Pv`3Mm&*o|P^XCJKO|2Ad`arKDBm3fc${fvXz^7ZULW5<^|8;`BG@F^ROiKer~L_& zh0r58e-X~q%2(+ymRVBI(qZJ=SCgt0GP;4%z7QM_=N#y8Iyb@={=6f`Ezhwp5VheW zyf;*P5vo39xE58s<50x2^K;iJG`bjt9QaJt^^<1JD+fEQJdW$Lgwz?6=S$%QcqwRw z^vMHuTP?|Q5_2DubUr27_+a;|lJw@#$U z4OiZ5W2Of`(3DT|L<011 zPSBk7;|H0!(mr_ z79>z&+cF0uhgiur^lle2862FFk#sx=+RDavSpESjkYp-+pX)*zJ921A#gspXbi6(@ zoIV|A(@*$%-Jl#1S{5AWqO?&OKUx?JcI$(w(W^CEO?N%9=T~edb&U{BX9IAD&zVbR z(!el_YI(tQnLDC5lk%synMd{IxYug8$qI;v;eRk2@3T0vP45IngTC5i4Nzhq#Wy*C z1hUa(93vyHw*nK{A~+^%N!RtV05ie+k$HO?3-K7ru<+RRAKxn<83&@RZ9+@4sB`Gy z&X~|tZ~(AR;m(~$cd`Y%+n8#*@5V(`@lpE)7*?30W!EeIhYb(T53s`tCl(d@#}G2toV3V#+sS{avZ~<4p}|`Uyg)GT3{Hr@++E;YxN<$agGp85U>xRw z=sKA~DVZ?aMw+W~Je)qeK5BCm4yG3~x{;k$`-5NmY|Zj{dhzC?L)EH`WTRAKbk=hk zj>iW`$XY#~kbhNkfyqSmgDX;(z>d6u6mt`UGIJF$3qNV8v z15cjO#vP&)P3CWL2?krXO?3+``rP!W#exDouAQpdBszaCxln7wGS4=9P*^TP+&DB@ zS5ayplA|Qd9OH_hlv4OB*qf{+=yIbM^!$4gBM$kx0L2cTqW51@h_Ze1UaNc|D3Qp| z#vYKlWke#@Z3TkuG+~~u37Mjk08)W`w)h*HJq3lwj^wu0-w!)JC41&qIgC;&zcy3K6Tnl zg==%{-6*;w75Xh#2qSmI_7hp-GvZhmWolI=qv0SuUZwCvqu8Qmq>}X;x%)pi zYj|P2l@D+3a5JCkJtRg2{UQ3$8TZ9ZHEV6$dvXR!YJ`(dyIyu@rj>B0MEwJ`wKP06B_1 z36Xb2K#$^J4O*QZ01EX6TjX68AV;}pA@VK@=usPFj&N56%uw!mh;-(KVW8bWiM%TS z{1gYJBAy$FJ`2K9(QYD$bX5lR&~7q_bkzp2Bixk$c9eTIBHmS_pApV$(;w5%dVGzw z)G!<3J4fJbD6QeWHsF^(doRE}0TiUOslDo;Bgkn9-;e4lp|yo}*1TKObZkN)tigcX zFg!nZWWYQ9f9B(&*Y~XKDmTM`4$MB0lu6*DA4G9c9M?h8XkxN zu$Lf+^_R@c`bi4hKSv^ZBz)unwI!Ap$QwsAKqsFS-SGhRBfaJa-6kYBV3=vy7u*!t zu?77OSP{@C2X2BtBDp8^8iD(Y=<@?hVcuhUuV8qjcD{puito&T>k?n11D{~Lg?HdU zzeIY0KN&$VfCk_(B=)GlIdHOd1GJ8QM1U&@kQVwGiF?p|p7D_YSc7!k=W1?}x@LgT zF^Ona9|@x!;QzgLVD(%Nx^)5ZsJ4f2=i|bjj*%nP$2RogToAMannz$q0Mk6PT?Q(LtIP3N4TxvRTe zuH3*2=;v^7%X53)wxx%28}N5~P$8J8#sLKH1YBEV_VwMP^H0#QdM~7FU+`hHUQ1s^ zkY7eUWcL2O)Zn_L*N8wjnD@Zm-_Wg!-^}X1P7kvG++%ttz;y|)L4e-@_Nw17 zZizrgFnUEgrli9-o zy`Xu-^P+o+!F{E6FhOUK-GROMte^D2U!TOcET4MuH89*;Fwy>jHRQFxUNrFP1>y3Y zjN^%(1!4YAB^izrhi^cjcO)625O9#dH;@d{sfP^lsMMlgq)RN%+%QZ(r9s2KWfPCx+sUj|AV!6imvqSwg)S&*iI@=c5JI++qP}nuGqG1 z+pgHQ?SFpfJEuoq^!JT%y8GhY`|7>eYpk{AdY(P!TyjDq`OkxC7?XH}H4?}M&u~(a z2S8Ghd#o1vTs|p1@o6bNUTJK>z9FpU`h>h1$nnQF-{o@?N*a9>W1-KmRUAhC?kFXB zFqxJj+&S+SL2CS`h4JP46~9&KeHZh*;1DJ|BpZ^N;GTyxwp3Z;8xdwX30J2MG35GE zD>!<#pzPeR)&cKd4~q)Jo>-nh-LXB>%(Z{2{|=P?UF!)*Qyu6}QcyX3` z59BQln}0T!%8p@Oa z9bGaf6xB?7F-ZG)^Tv!wDyAV?Tka}}jCdM#Ym^o`i&6P5DOGWJWUdeU6-MeUUf!0K zq|a@$G~njg?B~I3$Fo%w7Ip6mIs| zRL5}j;=mhI4E&2w+H;CdQJBon7llt>(yknGPqA&vE(K(tqc`Ro_%0RXt?PEC8=Jbu zuj?(d-R@(_U7vSUDbHb@MdE7-=F#WQw0+NfQZS&=X7jmV07J}|c082)h-a?p+|*aW zD6luB-`dahDlu3v{%dq8wA+m|DkK)E$3Cz>Bm-l;KBc3Zz)UM4*&^Haz)J`%Nj=$+ z_*Ud&WN%2v!#essx0pCtTPSeU&{Q^2&t6rbyjXBmk0P4meU$`weW4M(Ii7puHSU=f zYWT-4VEk32HuF&3E4=u;C{rtHv0F%xtl%G$Rm_wn?St51;QZsT87Nty7ku|~?}0ya zZ6^={R$I#>`C`i_&L3K_mh{RUaXoOfgXaoIu?fQ$OL(zu2C(>4YezV+xX~674FJY$ zfuLfr7%lyplpS&^^H4UXt$4#vF9Dwzvq1qo$*f4wRCg?cc`(K;z@%u-$}LzL1{}Df zu{m1EKf&#m7fmPPD0_QQRS$227wG|xJ0<-e^ghJ;amO33;2`JXA{={Tzk=TQV_KgqNCMr~s#5geH3|F}amMfEi$+2R*W6YknTih$JG z8sd`s|23?@U>G`K1^49f8F2yykz>gd4k61GH6k+3+|jzIgbOS;$&)ABRacYZOUQFK zXT!Po5Ln*itn^Z!!H&`|C#rN(W^2H~ek)*}iVy-;OFL5$eAnI@K*8kyRDF1Ft(R|Y z8UpS;J=w&7RE}FzJ+tafgdJ*{iiG(vx!e_O-gmo|TB$7_(ho_HP|suEF4Y_E-w4%qX-(I z3L44O^Ffr#lsj96Y%5AU#>2RG=A8B?+m4~_8}e@^Mi#qTj>0BVs66ef1R35kW{~R~ zSsh%Y!e5MCqnc-7nxX3mo^0dPq})H?ds^Pet&|bnIhrjYwUvh&H;izY_dJ{?Po2V4 zx7?<&6_Xf`7iD|+w=TL!(e4p^V!N>ShnD7DDXB9z_#f02i^Cm-m^Zp7*03Tcr3z%g zv2A`D$P*tOrjV@FA?bG4PJR}Q^D#mvSB0Z#;YakPp$D9-KpdC8(6WQFa09lneDEK& zf~$I-@Cpp7PKV8i?pgLV8v{2 zmkx(ZVkM)9uZqt*xAd+lD0w??mkdwx`r>O-siArP1TkXWuTrS2h(yOhTF48qdQ}4v zWd-3+4!LFY4%M@pd;0w0*|rtun(ZKJLu;R40FlUy$qCnsz36qt2QQ`9z!svb!M(v# zZ*=hVe%$}5>!GjYELY#1cDlQbe?||!LwgmtRr$ot&4p7OEBwo{fQqRrgJ~shK)}PP zuzU9HS1Vv>WT`D@x|8o6foVfp7w#j7qzWXScM7x~kL3Ngka2R>LO?PzG?8D5o#SBM z7yQD^UdETlH>v`tI3e}~!u_%WEV$dj$^A85N(Oi@dnXfl>r~k(Qtp?kOAj2k>V#Hw z5)zACh*@T5|8DNptEm;CGwFkRF*?EUzY>!pVE=M*)8g6dN#$`Zvt{;JVt&=KIMyir z^Nm0I0)fK3otk;x;2lty0Hx1HKY-V9f(~k^^4zc}AGo^G40d7tQ&FNfo)ZbL{8%+Z zmj)q|G$G8BDjqip&w6-D($Q_T6(n(Y3&kUGmoWzmnwb#dt1L8}e;XVX^s|@o-S3ze zJ9Yoqhrv`H6L=n$T!wp)OAId+p)JeQk6c?aJAFUnGmu&WHAJL-3{jexPe^R%XDXS8 z)-C_1HiOiEo(uEkY;JY&SFPbp0_NplxhO}Q)c4~HgH zQPEsqzf-$h$Xv;ip0EM~Dk=q@bIzpRoG_aH94)bc#kWzQ8eX3~c7YCBmMfwv($`36 z4Jgv0TX`~OFp%8e=WO6^@e|nW)aU(9Woc;Eu&@062182@-YPB3MbkVtAL$lq#OoOu z($yz-N7AL$sah_p8RQQSjE9Rm;f^nH1>x)95LxVn#*2|B07-h@Bhm<&QaDP|7Rq-N-K?2mPbhim{E4 z0cL1>6&=}w$i_?^_q(Tf+)Qd>dj>+aA@>={7qT>vXK13g$US5YU*MM11GAbMc~;Qcb-eyTAvTh^2ErP3m*PMS@{5v7)UZdSPHyHPeh)FT z2rK|clTBNzZpy5z28)ATfg|yvQlYRtb^Ofxwz^A1$!Ak&ukJY=NEgp?f~1GDF~07E z`2B;_V_`5EI7l3BY9hwvVJj!KQbCTPrGUb>bHiV`IBP+y8c7R|I#^IC+#(1g1F?!G z#cuqE=^5%%LtbeD86QDAkDW=#SX&DTDS&ZX9c;S!Oa=5YR`?KI$un%2d0st0Z=QW- zyQGrI1HiP3ghkbNq?WQGRnO2KcMqN^SG!_NVl>pVcmx`}|5_6qi;@x}?5>({o3NkM zr2cJ?S2_B9#>1b~-m9mE`SB`n7=Bubc_eY%x@@jmY`l45Lq?xIFuY|#Ie9A%mcO;Q6;Ro5$Yli zQx6>pwGYiyXZKq-#X0z75;nvDJmTC4chc>*xJ~h_N3o;2VDPd7a(cvrAa?zOq2AmC zIXrzq`jHSzYP1004%p-s_DCeYx#q0@ZZWr$skj51rRL1ef~CUCo7gn6r3V`l0}|(i z_hVd#!P_1jM>mMItxuE_9-b+`C=W;+kuKzsrP6$mBEVa;c5ZCDeJJNQ2OTb$h_unR z^p#aF+oolj^s9_+sFF8+;3cY=Rqum6D1W(KZ>=L8QMLt}`G9$YJiQmnIR@76O*J1* zWICT^{`U*N5>(4c2uEeFB`OxYG~}vP-h88F)oz!Gx>#i4_!K$Dk=T^QkN1)-bt!&T z^jj8)G}#TmV;F~S1>o6Ky$zmg>m)h+Jd`BL6URFNN&$@h`)DvGr43-PDPRHjp4W%4 z=|shx+#~t(^0~J%{aW3`={OreqXu^2f@WbcvC`=1DhPS4l!Vmh>}%z4+6v<$%ZoJ=t2IFgGchIkE84 zHuF|tgu?zMfKJ0+E5f@;Lht~?`SnEv)J}33cOx`5)5I0tN~z(6PPWQX@kemyn^Nk4 z{5g-azB~1FWyxCk50u{5vru};zNq;#>3Hc561^GPVF-UU_c&uj$+fXt|IZow@WaCW z3uP{BmN{Pcg6Shp@OlaJs}3pC-UeuK?Nk{~w`NZIrb9YXu9C+bFL0PV3!1&gvr6mCZ~iKD$zC!y#S*Hf~H@hnQz?NV*X6@A4Ppn9l9? z)up?fnJRTKEhtAn(y|~PU(*C9YV!&o)HRIZ4Ic01J!@5g^EqEkZ^d!mcz8>|v7=g} z#}3$GpPH2aQ&e#y{GOS5$Q9i`+2|2s*XpJo>a+uDe!wR??~^pge(HGR2M7=gxm8|y z02Dw5J(A+$$IgY>*%69=6GsEcxhFQtNP|lQM^sdyFz@J{55TEs;n% z7EzY2VW1eHl8pdaJuBh=Tef*N&h!Xbkh6&dZUkpm+9c>CDl9SsixV$Xs2=D0!OJX_ zyE~p}VUC#3GaXt{=3~0}OOfVmzCf17g(A*95nlA`bY`Bbo0WguikoCAC~t9A#HK?+ zakgJLI*ZPEg+YJ;*@^COK326DUL3O!o=GI{G29kN$ z|Ik}~&RlUVj2Ds~r@huHO3weu0@kS9YDiy6|K5-t)Iur?7*JkXk=hw~`5rrDn;;3> z4B%+h5m3WYrm@J6sy!XDWmD$v9t1qR8MEXj>QR6a<$_5=z*+171AwJ{-a&t`eJJ67 zv}u*RI>l_iK49IQGE^5HI{CUeVX3?d0A3qKm+v15{7;r}`FEXud}I=KDF4Xs_6WSz z@GJuZ3Umr=hs|>29J=z;*oZcG>QR2uW^2xkg53gHTepN?Z$gg6_`;+2=L)po^M^l2 z2-HUK^FshZub5Zc(yN0bu;{~ap(=acQO8t%zx_>or&d(TLl_1$+59J;typ zlM6EyEY1co)YmG@?;|sWb=waYlZz&ZzXiwbuYLBY0YOtd#rQ`1&LapND6Pv?2qCN1 zp5^PWmW|^P7tWznc+PKfCNsbmc6Fd}JMhfbe`#;u2aaS{ql zO1yCUBOt{I?a*9Im}L+bc?tOzLUpL%t|2zI6j06&67u*_3`1$?g)MG6I3Meh4fw}! zws&L3<$=d=XIDdyHjE0WTUF+?;Ix`&qDiAMmvLv)HIOGm(J8FoMiH{Nv+o4)`&56? zTmM>)B7O0fo0dfM3#99D!F^mBvtbFjnIK2@jB9x0mgd5GtNaQv!Hs6)LJv7w zTalTWiPc5kXFrG@i(D|_%dldEG0{sIQ|=X`0&c8OLN`n%A2ACR>GFKfk&@MmbSt1g z++H`hJSKK3DqGs?IpQPX#Dl=@cS#A*3i4u7ZRyCZ3IM~j^q(7pa|XXu4jI<_B9+}| z`J<1FV;wEg6_77a3#%0k+||^-A=`{?E$mXlw4Pfh?-F9>v3)4sNz^?@zvxK^UhihnEV6CZ#}k@V80W+Y_K9)iG{UCbgaMJ#!-PL(j^@dkw~Ke^sGV5vi{P{p1w#HEYijyzlRD|-*WVk+!hRy69w9vsvvfc>18RoR@vpw{N( zlCoFat&dWU{dN6%=zT0@6?eIJ-4X_}wR5+2Q6u6bJR5(=32M8gD|X4hLsrvX}2HZp;2Vn!4yu1`5&yT3Ca30*d05gE{7K({AAjz zJ7H`cSBj4Uw4RDE;C8OlsaLhw1&_E{zb2#;7QR-{ zWy*%Gf8k;anjVJ48_i}3MBd6XJqCJrfT_s@wsq^Px1-{|&W?2FO5W3+6J0~3{Jpd( zgI&(&&>^-rGjVJsF5V$Ao>`ZP%RG)_B06RFbE(DFq5D8!bp_{d=DhMwLKcz?t&Ckj zaY+>UuVHOdD_bTjPXu;g<@WC&vE)= z#;K$J=KnNqFHRQ$(LzKp0h$N(dpigrT7r%o@?(N)R7J)*KrAF(SM zTzT%a^#&Qz;2hJL8H#=~eRXGV#7)v1Ryh5b0fNF~i7SY-L>r0k%Vme$-m`YS!C zKwVR-h{&;^2FqNJ98FoM4apI+nJ>3^=1-F?p79Tsk1QHrY36OMB3n9Pso;MGg~Gb|C^LbnO$$yzUd#dMdOsNKh<3^+=RCE zyRcd_A(sq3+k&z|n(rKTP`iHd93J&{zknO>9WioljjX@(JHvE`;ttaq6L@2`r|FEc z+?6j=zneQlbSG`}`wVICF4e=oM4fANVFs*9am8#+K);}E#@HP6zf=Moa7elWs&}7u zJe_aPo!+3aQO1@7N3$wvWtXa@mv z?s);Y3;~DgC(H7A$~PKi5_iK}u7W3D`lpxdphmLTJ&%*Jjj9FG!wl*5)Xp|0*8^gz zFzivV72<0vJ6B8W66yPw;SPzRqXV60)M)Dv^w+~3siL%)k)lJC zM!oI%tG`|?e64gMU!awq9D)y&#$vBd@E&U2SUfX7KW5c8f8)6#du>zS`?=VKxZy<~ z;Pp6uB4!Nm2DiQ=tM>2)0lr|S_H}x|KS?wPy8Ss{c(-{znYr-U&iTqc0>9$9d%gyL z9enB0-hI`v)^>S9yon!A&!=e|H3Bdbt(}Ur;6|*IIbq|=9VP^{8v1P>_E+hf`^4)W z+%5+w1hT6OPPc8@;Ev2E$D=Soe}XCI_UXpLhd{$dqXf~3qXdZD`|kyU6oqZl;Z~E} zt5~(mWk>YS3?|*oOZ|-vvn*nJu-l)9ka;-bYHRk zVM|V_g}BT~ima1>0TIH1gp-Rt#hH3aN&on*XiTg5yCLBLwj=_vrF8eQ9{ZC02S~=C za9FXemUQ=^B~jN~=W>E}(jvfT7raG~2P9Kqx=>~?3m z#qpr3pX?8Pw)_`D|cSD0jN$B_ZvDp34MiztdHJ%7lV1)1vh=>O*z_ zlKRm5#sf_aoo5H!q4BT*@8_bmkxHQlmKkgTQizmlnLq3PnCi*ZhG$ahpF`EAZ2%T) zD)p(`RVoZP2U9DwG6u6yj%tbrTF>2TvpOkg467qxF$|_m@WP|cDeZvXvRhK1?I3B< zl5aM5p&S!jo+FG`Jg_;$>9r-uvC|?8%{dhwI%Z1=Bibj=mv5Z{Ga4SZ^Rm}j>PP7$fwr4BnJ9DY%CIPlH_TN7F zx)~3oHKK)qb>c8c6l^1CsSU(+DT@Z7)C)5A)g7S%-2@rCkQ4J9q0L>xP4L2fWC+Ed?Os^ zUF~&4xU!fdfV3ii64kG`LblkPxr=~wWX=`?MfE37g}auSkJ4jGOM^~^G+F&%C;UT> zH#OKI2+KNaqV(TBDeB(nl1;7(YQ}{>`>7H!lt6Pyp&(88kFBm7)0yNOFwK!yS6~$C z>>P{9-GFK{2p{@@C4vejROyb}79`a)Fd}Ki$2u|1U(LNDs@CGX{?x4p6i^JUSctHk zBA)yuKhv@l|Gm`;P^>B=W-5v($EsPrI1@~UB4~#n#o>wRFCKY_XY<^XQwZUsmHZhD zr{?GPY|S>?my&-4Z>gP^#E0@B*>BCGJ9;0PPt*T4;bd*%#6g`Nr5wuFr=a)=s@VzI za67zn%EmcZGs>l}a6oknbBl9`d5k%;Z6{yJrCOZ`!3H6h>)GN^b4OPkZ$9Ct2*CSg z5|WFF=Nnd72Gcn>-WJ5g- zlbe=E*OP^tj@@%ld|U;^4a6RLAV=K8DOA1Sb>>{Zq;gJ5;1G7wy#USH)Yr_OutD~WvrEU{!CMcVhfKkvxj^t!T+Z)0)ZuzxW z748iDQI#g{5}GFOR!$)2Z-L^aLl9%e;`93Lc?+9dUkma*{n( z@WoSew{(D|0ZWyDZH0US^~-hC8SWD1sz>DY0lO)&%90b^@qpwV>_gnqb>ZaNyu7w@ z?&iTOXyb?XIVWUb23T4}kh~fMsy&P1XzOfs2uDn5FE4o#5p3$iMaOn^io zx^;Y1OEpvP-W&w281K81H_+9{!F4Pb2+#1FjbSJ~rmd=0zRkX`NZ0Cd&gbD=61Rvi;hyTBN<6eW{q0K!?YFQm ztS?3H#h&A&`)f`7FI5*mU%D#!y-m{ryvQNf*W?P zE|L@eMu`?p^>W-%8{v8H}5x~m3d3Xq8`Qp9<-n{cAxuf zm|s5p*qns%P4IIx2WwKRfVtCyp# z+8wPQ$eb)NHp<7iluj50vb$p_x@^*(+%~LWt_bz)?gBm_(X*1AR|l8kSfOw$pTbJm zS|<~Hm%$oXTy7BfKIoQP;IMyhQp2uO!)g+nr0$p=Kr3G1dP4tx7<%uPRLhJ3Wy%rC zIk!cF1g8;kE~|6sH;rECdjy{w$Ufe^uqZsXwiw>)Tccd&=;Q(%d*z+l+jEtxjUbd~ zFsH>FhZI6K5{D!R_tW~8=)&XqYA5~mHE>Jb&j!X1IrgQYIhX0%a6elRD0kzXEO7U% z19Yw&?kz%+eSlam{@UHjMvN<5pq*q()A(@GquWJF+voH35u}Gr zSt+u-rEE7h+zoYFk*Y*hjxHa+dOOyi3@VEvMTu?+3>TdkMhqi~E{|`}H<%`4K4;KY zyAKss#8yp!l}5aUSWkACIZ_;B-VPlT_sLdbn0St(h*(m>HFAvu3JHybil}&xeGX=L zJ|EiVqRmRls#(7hyw8pj8Rp_-HW|k2&6Z4 zbSCB1bc7Kmhy4{xIZUb8u#^9rQ0vmEC}I<1TWZHBcA3A8G1NUGR6K%)k2Ggv21wL^mw!`M%6X`RV%hM2};vF zu8>@@rg_%S!(G`C-@j-VQmYR$)?$j5OE*Mb!Slpss)7sQS7 zj_(!<7(?()TqDFTEv$myOh1Jt6qU0iQxwMb%M0_q@^g1dLF`#w*Tiz>@=0CXAv*V> z;N^wp7E72TL3zgXrL^gQ)Xd_-7rj~d415@XE%WuRuv|>oFBLO@v)>MYV}^7X19pXl zeKareP^*C_dJ5`f_BU~dbTI_6NkbM>v)wwAyv8wO)D$&F(QIwdri|mDZF~)0W+0|L zonU}7XeXUmQq;%<_zKtR2}FhrDdX@Wl*aK(zZ*C?e|eAt4oHG5UQ${|0E!`YCAouT zOedNc8F?M~T(^v5t#?*jf}JKkZ@$er^wf+?T35X==SM;{o6<9!Q8sNExWBt8+V6{wD_J-h*hIXW53 ztA}|F>zvQy;Euvecw)_*G1$b{dH_nQ#!^B*NibpM$U|KAEsNizpWBkTW> zsU*gn$ z{Drf^JnikCdia`pnm$7R0kebepZW6cNP)@gCqE8ryNpp3~XO`)|i34Vi%yTsC)OqhhjMJZ!jD^ zIbIuCWWhAPOJ2)RZLr)rdI&GwARjvaggw@=3K8>LL&WCWo^~?gdhT{@_i-1tMoT}ILg3TY&U+xF2Sr1QV4Wyx{lkb zZowLD?IJcq^x20Hjl&hzoLjC!X8uK<7P6FJ73W_;^ahAnH7bu^3;h|9E2mfJn;uDW z=*x<;pntAk`vP(KiuCF}WI(H38-j zTiOyU#7%DFF#>^Xag`q+z=tSDm7vEZqWZE*{_lGwJp{%UnHkZ3eH; zeY%MPoTeg-RKtyr#sw~0;bZ~@qQ~RQf2%4<=gBF(03r;_EW!4hgv!+OX5s7&e~+bX z(VJ{$=a^PW<)p%s0lad#`Wbc@!UL~LZV`++M8NQD!SHH*(@|m)o5aM-sw<*%({*yA zMk$YsBH!{yMIMW+awFNtc*9R*eSFJ}} zM_CS6D|b)b-5+qepv`;FY|+?uR6{qZpfq}7{7QXwpaOm2fl81`pc?#IeNvFp=!O|+ zbg{o-0w6^RG7xegU1|NS4M8(dY?XUjeD8oaW&T{2_&PwI_y6&A0nHHCx+zJs2VNN6%P&r z!({fv?-!B|4pGD8p&2**nGc(L!%Q=x+AoB81{iAz5)1A;>BZ^Kz280$-<)AqXf9jz zGz&`Zym>tEKf+K>Op39le#_D`k8d)YbM9{v{ri*xIIYv@Sh#P~);>&|Rb3iu>Tk5C>n!7>ae-RD3tgLBOCmU@cK%zkg1WV@9@K6$ zTr$r?jaMPbt&V|y&dVLpY5l6yZ{O(td*iSdxkSKZ?WBssxxd6V&r8k+)81;6rYo>= z?<7<0GUa*4#1f{uPsqnC47%Nf)6xgAXc5)}YJNw0jl6iPz$G?`t9oVXwcrmdD4Uhr z)ay8L>0;_9yJNO(yCDb=7R>C`R%qYcj$U5A6lFHZ*AG9=cN@7@GF2>jdn)%YOc1<>1Zm*k=A`UtF=~nG%PH4mT4n z16&3>I}3;NDNLca%|`Jx@lOZkvx(YkbNTai{6M>DxOowQ8-Odqt}2~l=|XI&4$vb$ zc^eQG$zV)Y7b4xP27-3+iXa_Cv=NIf-FyV+(`@>Rpi68pHk&)%s0XppGjR<9!QY=E zg3Tac`NxHy21Q0^c!H%)KrD;`{VT4@6vhVKc~2gq=QNi9CTR+Jz2 z2^=@giSU(Uon@J+A{3;~v?z*MomDu_U_3iC-aVXOx*2d9?vBgk92J5(PRQJYJFqZ+ z+zkN>1L-TO#-d?G3987|xhG6JMj8(jO$UByb>h6?rjL zLc9;5W~vF&>C*1VD%&%c(*YKDCqDn+u5&XDd);?iWcrUb+HC);)+lRl<7{U5KVwg! zf`-EUZa*@@eB}!G0EU(_YzDX2J=^(beWx@$fp)rQJ&kemkM1 zFkSQ5W=bq2YYHuYm8Y}vZiz*a2?v$wkQ-#U)@7}UT=%d)y79D9!{Q35&9%zsh zA{bf(ZE_Er`x2-Egt^JbXw#tQ=(l;XxPM#G(aggpTKztd&2z@jB4njHtJFhI{$FQn zW{{u}Us!dCB}5ls`aKsZ_t#Voa>&x>KDPeh*+Y_;ca;$15#tin8cG;&Ap*A2;RV_I z=XnKaH2F>P0_@r}4=T|rlzgOe!z7InDhZ2<=@@THr+D;5U7iN;LEISi+26Z5pT_uY zl}0l-axW;^15OBB#0JA#2Vcn|m$2I@dirQ?@!-UaP;pjS4I8gUEO_a1KNOR&#SrVe z{|kwKH2bPTevf#p{}G8<{#WUyVsG}nI^$m;Y$e0@kIugWQcO7Ce_QAp#J$#GS_zDR zATQ_0u1Sq7l2hX99E*1+;9K91N5X-OqBa+DS(X=m`T6et^@DEEZzy`GYDmyPX6M53 zR{^mC_S?XIn@X9NMq3%d4(W4r(CRVQhP7&}F=dj;M+$7;Jx=%{BkH=U-8Ta|qC)c> zvi@bvF<3K)@~Cxq>JnA@H4MTo_J9i$mDI(}!=4i7Be*&_*<#$uF;|{#qKs8ZYJ5Vg^kW_Z!w2M_ieSc_3Ryt>=le09nGvw9HeZl|1nPZzq}HX)EtmS5Qe{q0#FNTdv|Tw z`t&SCngT`O4w;IB*pWtaEu&S2%K4}&T$I}DfzjdJCeVL`)An$dibG|Da2@)=3(eMR z6RnjOiYJ(R6nng~a*JFw@OXVb@%})iGPEO#c%ia3wxf(#Q(IpN%EXxT+xZR0NoP3_ z<7Q|}7~_q@wEG7!B#KRcKt>pcc{c@dI0}P)mv|V4l#NC|%>WHGT7qrR4pS(aGG(^~ zs@D$I0B4DQiaul>N9ldyhLV)sTOceym_Y>h2t#}@EbWLj$WrVq49VF;Nj0l_AkV&_ zY3;P@imI%~VRbCqC`Ih1iICYIrtzXxDSxSuG9A&G15D!vD>q*-rw|UL7NwKpuVS1z zyWYGr$j%rl>L0bRpeQsTufZ?~n4^fs;yUUeqK%?wub0xgd&@b7SSI>NOX1`OoAyaq zku_vit6@P*8JAT{{gkWIlsdw32RzQELLp{r$H&~&aW+c%_)&D1!o@L{vone)`^sdw zwyP`68|)Atl~=#pZVK=(x zb{B!z8Ai=V_{G@t;LqFh041DS4`+9^7TJ5WDHzL z8Q=!IkiSAum3z)m2oWhIO@9I!O@Qb{)cX1#C7(RKq-~~SvUR$FMDdY(kWtA|87YjF zCQ4h%ceB1P{h%p+mAIDd#)c2KBMi`sy#|yBsK)K_0ql!&B;xYpj`^Mor~--j&V`9V zdG)`QD6+G-{6FNx-1(P-HW8Ta%~}Ha#HK7-q7~vYW^F;lV39HA>B8c%is&lzupnVs zNm`=_U|nqa`@Kf(GZz00m4JhQz);U-jTU>Rp0Q_5pR&%&1np#0q7>tdPnd|UH<3Hz zG8XH&1P*5U3qeR3L68$z{T1;zC4l^^L9ncaltKejg7p^#3-Es z9IjWSHb3M+RL>CN1#n*Ty}&A#eyq(#J05SoA4%G~`|)8?8-fb-7o?)kU*0v;7`t*D ziBXI%c5h%g{++1TSvC4u1XX1eT3+OcwjWGj$r&d~+>`<|`s`KusU>|O0-UD;cgS8CK<^QpJ;Fu;{bBl z>((=^IKO3>btjtBQE9g!Z>da+a_|T{9h%Yn@L@Qop`_{GWZ+<(GBi0f^o{>() z6Y}H0;&%F1WRr~P^Y0@jrO>9?bBN&4A!PR-()oxcGhxJOw$SLTR9yrf6qVEZc(bzn zWCD;O7)84z4U8RqFjgEGT|{)eT75vRv?#;#SH)P$&Iq**Gw^)93X~x#E!0RQ7s6u5q{^0xFiJ)7W|^A@MruI(WC9%v8ER-=86Wh+K2|7b7|nE(;yw z;AClE)0>d1Kw@(cDa{Ezd)k&s^|=o1*$nKv_UT)NYMIlEhe6iW#-`2(cd|%l-7P!8 zTY&6dt8qL;Nfw+E3bRUgA7>X5Wsflkq&Cg9OH*tCsc|#VAF$xYn;~fo-QT+;n$wu) z9X@xAZWUVaI>=MD^ZRxL+o3iwm?AWF&UB|!%;rZX;G@d8FBo$V5h_DDqbY-6Nt_X6 zh5FcY5DkO5bnG{*W0k9dm#Pu8wrDTUzt1HcGTCSCA_Zc};vq&aDVLz;)4(?+-A=3; zt;#rv%;pS+r^Lpak;trASxi@tqA(%)Os3L?e~gMAjqWonf5`@8$MgiFkAf%n?rvMM zkZI=FA>;G|9DG8~^s;I-o7uW=lCF0^qE-SIiG4{!x7nX_vnEAX>qAK*Mk`Y)$bjz@!X>)C@`d2(rwR?b z8`krrLIqyyTCXir2d?@4;k`&IhNk{u^OTEwj@*KL4ky^l<6o(Z*Vn20q|3Q%V!LE= zJ^Jy6w*CeEPl~MsZVP>VPjY1cpHl4K|084-LoW?Nk08_?|0n1-Uj)Duz#pJ6Q^*${ z*dR|RYY$w{i6IhUa`2!&wg6!J5yHQm4){qz!S`n!_)mk(l%BvVMlXYk4vXK z$FQ&>iSjCPVZMKK!`V=kCpEkXIA)LiTP0K(?h;38Y1kXR50Pwz`X*1)wiM@Y2D8?| z71^RK->-B=lvS7zR3{$aF2mqA8?K=uI|sFHe#_WoBit-aB47eb-(`GsSE;j2G3Hz_ z(NFqqje5xIVW$~?7o66)duwxcw8V_6t!r;NoM#c^;|-Vz_|5U**Fz5eyTYVBEf+7W zhi;QQ;~n^)m_nf$?_Ym|YV?1ckZJ!7>>3(bej75|(+d9UTgdu*6W@RD-BXethwGI^ z;8rms=YRLhRB!pEQG25b39av=5#J_Xxjr*sq9#V==^KMIubwIY`YRWJ;A;`l|`%hd9y=!UOzkl`d_w(;h4pjdpE=JBq){eA7 z|9t+px!S*)SfY}K4YCNz=g_7hu8D)4se3M4kUqp#-C7%pDGa%8oP8}U5~92yrISI- zB%#R?RyBy*RrfZowg<_KXoF26h0-Z)9yeoqB`rO$_S6cu!y)I^i}BHUHm#2r&^GdV z_>MWulB4FL+@hV(a9cDi?vm5`jNGhTCO-_Gn1g1TBaWOW&M+uX0x9Znf#)1Ze8Qn& z#%)poJK``(v=mYFftlo9f}c!m>=2VJ96JcfK5WMD5jc%#eK0DB&e*b$A*?2UZ;uL_ zK@KUpb)SByah(eW8skcP;@e}DRijn2(Z+h$^qF^<9tk4?b@PRFGYY7C*L>5WOY50R zM%A{RB{lRdi-+v!DHRt`Kh9f ztYy&1>umKhi!kq}qE7q^@7gTWg6zE6o2DxE7_PAYp(l&1O*>l!%JxkDq zbGqieS%z1-l>n4?I%$=P(cYb_Ne}DV^-l|$vwW^lu2{uk)LWn`- zR&(3;<;5R{%Cw@IM(lEB3nFEYpXEqY^!I!7i`wpNs;CY#(kQe>QX;is5pj~aPU8xmHnG_D~#qti|U)KHE_GRT~j|sN~ z=C?$ASY)IKM_jAu)FoyOxZeXxJYGQaPWVTtM{V1DStHf6@+BU$2Z(mCQo`<{roPwR zugemlhQhS?yeUX6;<}KHnWbQr!%a&wf5e#g#kMIh)-fe}XjJa1uLpNcgrDA*VFZqR z3+W8G#{X3N`9^yU5~LyF&d{~XGWNK6Sivk>$0&{wH+3CBN*V%`U45Ou7}itJt(?&( z*UoQ(G-18GYVo1pdHcogWP6Y)<=NlRxofTie$i;MItaSsGw`%cMpE%|q$Qi>kqj)z z-1IGLbG0H{pTCP->@9OcBGd8qbCJ6lCQ}5vV~~UhckeO@d*l;E*ezy+_8bmIWWx>_ zCeqNB#?yc#X53O!-#Yq)-~b%DH;&K!PIl*GK@>ntVVQ+C{)?jC7pYF^2^_`3Th#{= zm;&`goo`B$!(e(W1nIG2CANDO2J3fH0IHpA(q0RUap-$#Mm?1DVnqzIda)C9xhi_g zzM75b8X>FF*n@5-f&y`N%bh9$Ri7$77quauZ8bYEoK)-bQK^&`60--- z)gIH)WbCUg_`VAU=i`J2?z=K&%^HDv7!wO;-WE^=KAhFc?Kf{tJBR>7pUEk_4;#06 zHs6x8AL&U7-#f|czK+U9aA1TNZV%;+%Zk7jsrBq`Nk?nJM+gjV7V%&3`7;37S}YeE zt%LdR>B*3veEEd4ALm{*c{-B>PenBjrT7dw${drX5T8unE!$XEGlWWn*8zSBE=z=s z#TXj~B0Cb8)(>&_wV3mNU`ry-!J~^FT)6@mBy7LaM)YQnq%lq{{R~Vn%Q{&n2IFcTuew;fSS7te=OVOpWk?Yb5A|+HS}%#wv|f#TQ~pu zZU6N2|41~|EY$zlfesR)lmRJq=h`%s1v5)Y;hd0a3{MA3f0>+?KAI*0cI&4|QN>Hn zwVr(?KC;^g%p%KKaN>WM^V=l8xLX-|Ljo;#L*Rb;*m3*m$0ys<=cDX(UpRXLLD0D) zyj_rfNDR{EHd@EqO+Fy1aU0dP50H59PNBbu^?m$B3z*5+n`XOA{J2{aA{iP?4Zj)m z*$EPa#g162s1UxOV6XKJ2%(M@6ig3}M@*Zf`y!u{tPGija~=?|x{qW3;RBK`=6T#} z`wJEtnI?3@5SNw_sTm;72CKT#=K|5BI%=R5OjAs+t6K9JAJVp@TEDnCga+$v9PK5p z`sP+zIAp&HCQHs1Lro3!Mn_D0q!iW}{LfF|(mVL7k~?S%to4n;a62*Z8B5s~965y~ z90hdlDj8rYcQx~r&!nYTzG6CzWpVl4`Z4$l6Dtlu2hk2nGARbhl|t_n%fjXKdzs2TB5&}4Vc6i zuW*fvpY-%V+SXqo`qUO!_Z4E-8D2FGed)S9j%#XzzCC-o>d*dm0%v4%N1pNGXC@=O z7ypbcy?<}Fr_k+B3snW!t(A>6u2S(*HiBw_WJ3VJ)oQv9XxgbTpULu@RDSyTS)>g` zq^%gL7{d*@gxBH5osNmMGO~Ct_!g0Te}~a4?Hhy*4B?VdP?5U5KCohmjDU9YoOP*T zfpy6@Cp-wV1`4GYI+I`<#go9x9`exvV6z2zlj%BR@!2Z=C!Lnjh3Yl5y;V=}me}zw z!-_;5m%{+pQLfH$P{RUk)JN(~^9Du1c_j8NtP^|_U`f^y41-z#Cub%YIy!MuwL8{5 z-(Q7_hJnNT6~Rj8OMYFwiQ2Wt zLP`cD;)q!R$UMT5mkp~?#{g4r4eKE#Of#%7UKRY+4PG&Xb+``2PU5m77ED)jVw+`Y zcTQ6Ddc~R)>c1dNm`3gGM7VI^TDBKmi@g5{0OCu{aBUJUH8_J!Ug&~yB z<2)tx7Wm`FmmlJod3%F^W+2F*VH`XbTwc%`KiPpXtO3|5aopPZDuVhmovw$!T81l% zwsu+AbBMV;}R86Q7Dtr8_zgf>8$heeR}#yCP?~Wr(3)>Y+PLHTpZfI!sRNn%J*Ul{ck@>d1 z_3|EDtLn0pzpX)yUIAj?XIpsB*$uRskTyQYC)K%_81SKA?AWmIBLxe4k{DoU!L6 zE73ef)*L*ERp#I4X?*$>B;4?T9vpH%|9Ev@m14-J?M|{e}AQTU+|qg%Yzh{-*-V ze{;7oRkdG5@6le~TC>IvNe;$+QOqkAGNoFSVcZCQgD0tAB*~1AwsmtnI`m`zc=Y>d zYQsjD-#0SVj{UqUl_5NzbGSP%@5W=cuODG*H4r#d1)g1Q4`z~;SN(oH?&-ZTRbX>i z>n);Glkqm)Y7DcJ^)}13?B+tpZ8q6Jf^tK)2Hs77Q-|%~JRf@lV$K^X5B`wRpl_rb z?}V+}oliNF=8he;Suf^lMo%yhL>Wa%A*&-hj16Mp3MDPHrr(N-l((Pir(LUaAUs>v zm|5X{U+n;Owsfb{gCa|dXS)>;nci$2XtXGfHE#`HvDzO%NzC$jvshWn0M|jfZ2gMz zR+wn6EnM1IMO#0G)FES*e{JI>>fHFMa&I8){E2{(Qrgx@uY#(Ul8lu}aDX2@l3Q_2 z>B$JQpTC&8veucT(<{PYR`bB)KB!KwRZpq4uX<&qR{h-#_n`154CxKVqJqOEdB}?V zB9D6`@1@@DB3Cy!fyN}G-$0psEzUfY8ZFk(Esu=X+4z_GbfscbJ$_5GG9}P!-kADw zRy?yJOYF3hnyiG84YRV}UmsU5D!?q0kxMMU@Wud?qY4abDA&arqO1Zr8@|A78FAl| zCK^@KIwai*7xn31z0*)hkm68=jufJ1>`CZCt_`9CN+f)DwdGa8u&j}`9lV59=R@w% zx>WfH3@1MlZ=H4FY?D~^qaO&|1PS1rIe%~ZcqdX6s&lhC2cA4PQ0vu<<+bm0#o^FUJ{xg5(QKAPD z=7hKWOt?SZ{xaV8AINmfu&>rAmu4jJXG?~jhN6oJc4d6PAL_2gdq@CK~ z-b0-+*Ppj)*TZL-Z;oNFjbFra`jWvx*_?l8l>5=ltY&V&ZWi6BTC`N^h_vm-czv=p zm(6^N{W$QTw?Txt?!8EgdrpB-ss2f~7DD zoq&%cW`HJ^V8R5&0d`jRX89$!lpc}MZL_ z+PqYt3`vw?k2jbrC88z>hvoaZ^p~{--VPNX3N{mE54^UT12g}zer~>TYabkL&*P%2 z5|vMhIx+0e{xTU(@}*id~7g!Agh>a|Y~InlHMM*N|VM$HapxW>1Grgo~O?6Gf2+5!#oP4Ee$)GyrO?6B4ENqo56(tRo-YST) zexYu7S3^BV(aFz5CCJj(K+jPnifW8Y2BmKAYoX{^s!b`GqvWgD>TCGH&{4e9*Hp8> z*U?w8)R$JWp~Fm*LIa;HkjGYxMgi%%MsslRox&UR@4+QT%*Oh#4;aw6{xO06D3Xql z9>YS4;_t>PDSDeT{Y}U;AaG)KAnB4}XcM5q&)+P{Ag}z{=B4DIHu$5aD)@S=e;))z z{%3}x>}=rZ{JSvtPl$!>x>y;I=m~*^6>F0tQ_E0)1w*ek6 z@u`IjWeV;T_q4~WoS$;y{^2`O8wkJsSOesLBbAYc&JW&!c1rIy_G~@yz zmD^n-vFM>-fk0MH)wskV6IgxsRc6f!Qz8GXN2{g!0*GUEl)W)G+_xm*uE#QA4{p)& zDD_1a^keb-bQb{NA-*(i8e)ty|1d_l5*&?1A1k9SRUiwXAwy>O-o^RDLuajr_TIf- zvbVowPyHTkIsPRx$@B`g{-HQ2YGQ9~hqCL!(nsxuEDMoMOUg6!V~d3F zH2xfCTu!p`60>4MB&bR;G8g>3J?K#3@>O(!z`$3b{y6TFI~{#dn-$iRkOtrT=uZX? z=-7}&nD_n-x?)MF5Tx?YnaAfr^(4b``*&-IR7yoHTI#FC9(V$g;C9o6vU$zCyq$ir z0i+6{pbqU<5OxiFyFxCs~$zqM_tU$NERn(kBiD@Yj22bUqF>!a?zqk<7B1z) zZQG#OxL+UgDup1pR!TjIxU}d|Mo3o{^+8z?ZH70o_N0HSk5jWmx)F()U2?nJ6kP5K z-r`mzj-gD3Z38=4c#}Q?Pg1jHiL_W@kEH#&b`iDMvL`R0CRoL_OtGAO#c)Jm2rrMx zoeKgQ@djgJVp;J@9`j`IjFskM3sOkV;9#Y;(X?2=x?ue`lpZh8YWkZfh41z|^+V9a zIn1NN{>)szWXOhqUqcr7w>lN?f9qradG&vH?+a9pl>T6=CTR<&Yn8ks4PT-02SAHK zi?(3G_$%5!9p#THeV2+)qa=KUec&F4fBSL{@wA7;lXViTMm_#HKDBP|@n^d0C5|SNO+z(L<%K3W|BGj5s%17r3Z2g54%94I|W1)r=7H< zTw?PIvDw~tJg?EGfNXeyM(ejh@tVf_#nb*w)TQA$+2Vude=H{k#(n z16y*f&FVSh_p<}G7v4)4Jg>`Wbp$DX9V}DZ0o~|*3j^XZBK3BQ8!0y7{ZeyG#u~Uq zoVB^VxdscHZ~J8yd+DF$U6D~Ml@vg0;kwH**v;GWLv29eN862pdr%C7WuZyM-E6zc z1_pSxhkuo;n=k7w!m5@Y^b`43U0_CvnAJI6msp}f9ND2o*?F{FvZ_UH?N(+}M<0)D zW=Ep7_*bIvPGxGrH%}=q!jwjCv=v@6oWj*Z1LVL+%W)GJ@R)E5Ch`>yzG}D?dnIX4 z_ng--vrCJY_{8I-`e)DM3)C=`0>K(QNsXs($kx)r@%e6Ab>p5Ib-W_U+CyK|{GuT6 z!Vt&iC?Cx26WQ^c>!-`UuyB7U`>>CbHYI&z=Cg%gc@QrVx*tYn?pZGTD|`^amW6<< z6k}3ulIDX1jPpnPc1YW#59KXIySQBW66F+%4%9tr0iQSq^9{=LIaN_8o|F(B(sl?) z$scLjAkq#4Lk;15a0@3Ts1p^*SwQNc+l0>CDWd2I-np~;9!b-g&wo-vQPZ_wYF>>) z`?tpNchZ4>VY>ff9RCL8DiS`a0~9eq>*xvS5+49`ot=aV1ayxI$33D*A`C2F_xt^M zrkm|S90b&l#LqFdv#tAZ*LhkeY@hYJv>4QN+A?l=9^2{`>$nB|e11Xbqn{b`C)Y5N zQ5b3W;{&}|r&)jcEL?6TqVl~zemGx>mJAF^i0MIkXyG0wlLs{C#l&W*TbMAUH(8B&Id|qNq$45(!hymv=O3b}UsJZz6a^Q_;cj<2nPFJEzZQf_7w0M0=hIm>664b|^{R9tUae}}FEN^{mw6El za9ia}2@)EvUywT>@5%5Hh^N|;U#6fM;gmIan9*@_=kUuLrSEuWNI~6dQ0_21mcTuC z6Zfg9BF_*C32j+UCxBEH_)~(;?nr!;nunLm=A-Z}1zlS9K|-97_t@r-*sNV=av{zx z#8K9iL=MeJm5KaSvPk1~{t}Q%x*Q(B^RTg&_{oV=#p$L~6sSg7$i*)Fy?J_2w2rI0 z7hG?AKTDVt6#QLGu7Iqx2cgzDN~;qlD7GJ}^eGTVQYmg%Oo7+fV5#5qx-) zS6>$(F&D&C&qRA|UhHH0LXcxFSs3Jx1U+F-hIbj)a*rYcV4U0rpvJz?^-36@Lqg0; z8r#mx;HPS1Dce~`);Q0SsZ0eIBdnmqT&&xJ9W$^@Jc*P4`)K(MVQ8H9pHV4Hr7HpoDrMi@J2}30 z6ux}9c!1qNE8}42r}xshP!ukR!Yf6d7&{FdgozG%>mk@69dc4xLxNKSe;QWi-MdYV zlqlC+&@0NLxt5bsPfiH-#5lmg!2RfLZj6ndoB*xXS8^}qTDYa7Cgm$o>6uU_3iP2^ z=5s@i9{Uwj>zP|GJ%KyCCk_zFD`Q_Z?txg;W(m#Mx|KDBYE|hoEXzhrmz(wtX&#g+ zg6m^rA()W<5WIP`<;^)^|G>q&=MB19P_o*x9y(x*s@{07GL>?~;clzrTxO%9;9A%D zwg2Z&Z6<2LNCDrR$S{<>W#{3zvX8}ne=>L+OPKm-UV(1eZRk||Uegm!Cd$NTk**r)rO7lJsj&-l)$F{u$0bwVJ`zJ}x z=6>)S8|<@-{2?Edi$BMo4;frdkzP-J{q_0#T=w@J0!Nd7oV?OMUjF^K)oiqvOfft1 z;7JPYR94KbGOL2$^f#6**ISv@*fLMaY0V$5L3;Gg##Hb4k3D}rxjjZ|^;tk&h+dkz zg0@eXw%Tp&4@q}A%}9Rap(1~Jp2+rt(11<8XaMiA$^d2hbC9*gWkF;;iHgY=dB{wV zWC4!QbMR55)uisgu9^TO4M2772I+gidCg~eOr359nB-W@7{C_2LZ0Fl>0;T6JzNB;9 zPsXx^XRhE(-XcWbcgAO*g5AqcJ)d4nN~iKsCwM<>g{s;+tQEPFjlL zQY+O%KC)I8tmKHV8ytoWDEzo`?tc5`lV)`K=;p`V4(K`NF&DY{5fnKD9M4bLzE3-Q zI(w66Vdy%q^CA)PV<5B8ip-hg<~DghC+Sq5Rm4%Ks8{4k#$ zJH1iK%_r+bw@B){!1wKbuhnvgO8On`&dJ@-bzShg05PzA@IMLM%dTI+ecRu?ZA_-hC9hB_30^ICLM$ zDtZ%mCFNbs`_qsk0rwFK31H3~_ALwNrwS=1KGs@p&}pbRwMk)c*f+(xPju31Z9Z); zfunJwukZzehg?xK{6*c|znF&V8x6|(04n^eu_&mn#$<9|)@Gkq|0a&|N@`d3lxAGAZ2SJ6*JG=bz88F(mi zXk%qg7$^u3A$w?lH6n@iZlJtO90UjvL5?!P`!lL9q%V}(!ALW|#sjIUZ-NOIFd*o& zRtay@M9XUCX@}Fc2V9~7K2 z+>E0$3pB1Lt|u@6`I)HMJ~}6T^Swwm9AH2iw>Svn6S4quJMst6dyp>>m4$Jeh0=t- zD-_HOGUo4t0-_hP;U=;~umGLfSfUMpNNqKP*FabkYklEw==++O#YormXC)z}aY{3M z!yt?OgUlGAla>s&n*`RpG`z5mkA^A&?`#t4!8LisaF&MZ%GW()6tOJqCW7&!1))@l z)dq*lmfbq@Z*R)8q?jmatnaD}rr3^@&{UsvzVMw(bk*EtB{7Rocoj44-sMvJpL}BV zx004W`oV63I8aykK_KIn_1hZcZmf(FMzQAinI&mk!AfAdCBIm>8Cg6VU@2p~x1fEr zq@-MEF3J=nJ4`f@isyiHv8Yao!fA(VX^#uErwl!a}PC!k{7%=2d?l!Eh$DV$Dh_%{M}b++Q=IaGP*T=x3~=$m^6yt_d_}eUa0*@97rJ_hiI}PdZ#Pj*7GB>*9SXI z($cJCi#xkf!I1$AseU?Q7KC_MhL>3{f*Shi!HPCBUF#|l#i1QrtFf@TnE1TW-RG;d zR`Ft5ZPQGG(6Z=Y^EQ(c%(Va{W zudRjqw^loryqwTB5a4VW8CC>l2Cr0OYQsRa2m1#Hzv(I`Hv6%@xKKV!?!lCYzyI}q zci~E$r>vH$PoIlSSsDf|S;L8n%VJCciU?Pe>vh}-uIDPF7f0NP=bFP%%a|Wq4)(Ru_ih0$8Oitr_jQPQ}r=*Rr zGrLG0JfU5<%VOQq;>C2C01k8_vLCiXS~8tE?L@IV4%;vnT>M+dK)n zvhm`%RPokA)V=>^8C#5njuOdZB%uVm^SOm-r2cx`CIYMr_A4D|vZQer*W}b3e zyTi$NarelKCf@6gU%!aPF-R?P0csHnENXvC{VT|?QOFM!(fEe25>Cx55Z58{U4x)K&XxRJ~F8p2b>c4>tj?NaQ zuW;cnm=LTeD-X$nnQ_?=nhRg-2k}PA>Jd znM^um7R9eLzqBD(R}hgY;Pr;x9E=B%E5Hvn;K;q3$j^hZ1b;}=+4H#qM z>G5>vxZLB)jR6cUb0n(Or>~N&OhYg%d)Mfl|-Dl;zI86ft-u$ z?J+Gl43)h;@DTg;gT0~EwaMBk;bTO`28>2LM8@|6cJuevN&Tq-}|xaWnDKb*ccH8pcq z9>|&YSe?e*I(VGM&jba1AyfL}$%!a0sKW*jondAWO)>cun2WwF_`fyIZ>H>v3Fn@s zU1Iq{>i+>NydG00c?%>>Vu50Wo3s@LBB;f?RFc_h1U)tv7+~MBk8V@;2yv>faU6nd z@uLdtpgZi#VL*-6wh<(m0#3KNwIgHtB)4e>tX@2OtfSJ+bv-_i%htftha_8f(0$TT zZbwym<-|>zHi}K9`hC^`5|h1;&zy^E5Y~hEI~_CckH=x*)ne71_Wr8~^fhvfX^r-x z)x9wq{G3~5wux-{<6W{t&Y-MX`@5@Zwau48#TpZhvLya?UL%ud6ALR3r^wKXPA!^b zHQ+1?*R|}6%wQ`*&sDQBMPMf?idS^b$W|8(bk9CbO`MI>NOiIgUAY65#-q8Te3J2g z`<=yJ)QdnU$;?A%v3isu8Ef6I3ms5F8Aq7s&V8;ebS=KFMoq93Fo!zg%|ORPsezsR_$cLGO!05ba%=dPzyrhZUDT;7gkhXwxDt(- z1AcB{???DySE%-qMv?0&)pZgj6zAt$QP5X%HpMCW4||hh!EDs}p+D&;Z%><5vf6Mh zIM#*SI?f4~_Dp@b&|%TRMp4Y+`|QlcSOl7{Svu{nM4pf&HH zs1ucP|Cg$Lur>DLp@kU77Aq_YhO<&A9{a_g?Nb^BjXT5F`yBY&_9@T*?oR*ZZpPG) zoG|9GepOI-nRs+H0^mmUC4$g)5;4(5sP(t>ttDp{K>Bcy`Lf@qmvn2_ILZ>cq?Yn& zWUo%iY?Sudu0Jfrbrd#kGgBNy$neeh?z8)tJmh53WR73kNj`q10}7N|gdK$VPPTYW z@Z7FAP26YZdHLO+9^t&P?k$6mfz^l6hdWnzqldUL!b%i-5dfYTynB@+G-)H>c4Czq z;%3;sV(oahO~u~t6jIRFgr_=RZxi$u(FAiTa8+D!no zg9I9iN3D8{SAfFn^a9pCQm&$R@Fc^aO~5gLoMP}128d})9ePxq2SHb(9J2E$)W0(d0Ml_tWdvXZc*CPo-!VJGM0cy4<3`cmg>DGE zRqv93;h`xc?jb|W5)$tkaFFbZpTy)3e#GKuD$MRRfXVLu0HcRV*Fn-TZm88o=dRGz zbY2}`x2+4a7Zbk`;aW(pgMUK6T`FrOrCiq^-&1L%vAqgN!&&5+XO0hC!u0q$SeJQ` z&sk7Fb)C;55HGVFm6cJ?#-l86dTqVLQmC^m^>gui3QLv~`+Ri;J}R!J(JV8H8%VtM zr4F4ep5VIU=h2LgsdkA>)7~LOG*Qwd;DldkY`Y?$6Zy6^cnA>aB@umS(WGsi7|ytA zu1Qk7Vl;(~dZUB&GAHg)(npKVr?Z+3S9(|1)|>=cJi#d|&Osb*U*9SkH50f zdyDUv*xoOyFK1W>Th16$CSqPZ+AewrYWH)`$-uS4n<+BY;X-m8DDm$mt$C_9rRU`Q zz)wW8FI@LG8ARVdojJ$Rk`#*<5g2JX-N+%_WbBB`l8v3U5`s*c1prSzuHwmPv~}Kf z)s#67->R>3R(6U{!RC2|3T5A$Ty=fqSMS#NO55!G4u#GJZL>_)U!@?GeE3hQGo7X% zh(wxOeT;>p!T2++SS2#P#A)8^J_VA^bAkuf#2`$Tl8l~;9(ne2^3cnk62K;;9X->+M3jQ5qK#H zFQ*uo43jo2V>r1X(Dl1Cy{_cQ<5Az6rW_w}zHd?=8vc=dlgwQ^ot}ILASdfYX)uuk z)k}fwIXX`Z8)Xf7YWDl`dC68dLc*CVdcxG12@RQ2;C=g{6VKn-Oi@+ZAN#a%8=4|)m9IEyn zU#!Y*oDs^TAN_B>WVOT>8OE|i?Pdl*P?rf~C-3$-WQyI6uwbLdg_?XtA7ek+CZ#JA zs5lt8pi8#Z5|5PHZ)qJ)us=!!N@~(W^kB)7?j8nRB<(hsAf(e_dv=YghgN4D&kxjR zXbM?5bW%!Wvo_TGPBODujG=S?G>Z&@zN|IDP5iVx!ZmfU6oH1&O0jSqFBQ>cNLihv z{^Lj}NJu=gIrS1nf$X$?Z4keL>r(D8vaP^#dmD!5tcoDr!(*=@SuM)pC$smtQ`-@L zg2+{~#pL`9(N}v|KEsnwI5n}?A@H3bltW0)xeZL-En|j_Bs>DJS9jxrRVdS!2yp!S zgX2zL>|aDld>C}?}47~Rh@*2$D%l%Qv)7|&$C;q9UGEGZshvGc#ja3$>g2v z(c@=&oLJXE5_cWV6WW}L8jTSE;(_$p1p>g+Dgnm(F_NL0#7<^Et2=G1><}%BGvD%C zf(@fweWnj#_N|(U`5OG_KbvMjNBOiLi`;XKp0c1i{D-at@_(JOJ6k0rP_kA z_7wf7RrsPC?1G*nW_VdtG3v~fduFZFdm8G@QRdr&6HCt|%u~ef=mD0^r^3El%K&Qx zM_1Si+6nRw?SM~C8iL-egJ1NNw2e;-K~QxX1)w9dn%4a@v~?T(!=vitm&}bGYghxy zjzIgc%YuXr5etlUspUJFLlO-oZ(dD`n0Z@Sx-=%8N!z$2fNMD69S5; zhHFbzmJ1i6E11&v%Bl?wN;mAP*}G96ReKUknUrr@O4XKWRrN@AMJmHBy$TN9EN*7* zgs|o~_Gnf1iXRY!9ZC)lrOT?~eXoYS+l7YREIzo<#ZoAI$9rTEE>b+=!|}o(VZ`{j zS6#V3BaQkZNMi0c-2KAS;@yBvZQl9u>AT}o6duL{lDi&C zL9f+@!k}REAbt&ubt1DnQRFe^6v;ZaUxc1lP$s`mN@K6p3T{Q}}yn zm`KZvxlEis5sr_dgdU1cx$iW;nNDdu_1;BTlGc3Irl=;xt}cMJ3rIo5aaa>EhUzpF z^^H_}f}P)PTKL>l1MD;!xEFxEfa8o7taW)p7`S-5{39D!)^q`Q2~fXK?|)QG3D%Hd zEp;LAY8MFyH8Y62;U=Bv{_JkJzGwfsDaJlvw1yt%)VztAAlls6dk`UZyTl*pXY|++?a9-h%N)enr>>G;sjUm>zw2!L9fRcm&NKcKqW$3ZN`w3h z0+@^oAa)C(fi~@)UWIIsE)WYH7+~no%b}6F`Yl(%N7AF9G0Lqudj8KLfOzR%3s@rE z6pwad`8RCWP}h(_K~vZ!o$oC3g4+y{}e$y^8SS|LI9 zW=KWuHULUwV}f4^K{=+sL_89G?5v}4)UueH^SboLlxkl!IQEGV$o}@qv<(VM#UkIJw)QvHV#`o`b z0z1Fy<=IZtiatOyr3-l7cZ_8TJZ{hLRlT9Cp*4`}&J8LKN(pKX3IQaMlauo&o#&%R zHd{w@h~F{qh!9rZO@7wZzJ64Uh7e@jEM)*apBqkMlQrVrh;(xRl~ZCIaM)N)ZP2@9 zfw$;!HJ_`&aPWf~;tKtUr-SZ6N33zU@v+_cV0K$(cgQYi5wmwGE2_6DV#-K7CLDDm zb*4c%w^fU*!G*mpvSeQ;k#-JUTL6??D#~1gQ$HNbo$#r|Iadd1EEUI2?Zu8y=rmV1 zq;=OD*(z>)&= zLcpcG!dSaQDplqocS`A)J&C>?r6qmEMJW-`ZtsmfC>W6`tjSqEu0#Swgo6Got5ma1 zv}$b)?aj@g?_$1(WOjh?C3a?ay%hiTO-Oh4BmnPMXyL8golV?WlvRxDFj^OK9?BJN z73H`BY2KO8K2+N9zHu@2CuF@1n!KX7>)+n~GVsXUN9g`ipxn1m5yzyW^c6U4ZX^jB zF*oD>hD$>Tw#UmV>d3o-f4MNKC(65j10a06{`}$reIJw;t?^mxnlv1}mpPK#rj25B z%G9V?qJvlFFUmA?m$hp3Yu?}l^E=NW_P@&;q-Y2}I}-^lUV%P=K?u!N zqOZrQ*Fq#41YagJ4mdKCr%;0LXxOVKXeKDAK%_LgJc}^mE6XLwlaju-DvM(r7pP zs9TVgyoSv1g0KWEJE5EJ$aUgb{8@l*EEL;WWQNT&+YqFzTcN`#eS|?9Z?UC4nbBzOM~CIH`_Vtkk#znMX4_?WQm5 zuC)wZvTzv%rHyG}3{LubK7@u9`DO9RlC~dmth*$7uI_Sq zfKFL^CvIhNO()*iJAMxJlWNPeqtX=bW>*ckJ}XWe`&f^bu?wLPn6aMCPJDo&Ai0*S zosa~M9@i&-05Zf*$(uiHfYvD&k6RxsR*}(0-z_l*PFZeYQn2zqk`hm;h*OD(9T?sr z>H%?%&Izr5!H=AYN29}@4y|sq0%Fy{PibiLovU4sP5HD zqJC>7f45}$|BkPsCZ+~1*8ipJ*QfT;X;}qVAe`9IW>=9aCuMpo4x3;xERG$8fhM?6 zD#C(5mTSHIr|cKo~(XBDqt9Q}<))4CH^NftZ0bg&C!3j?_lP zX0Fsn$3RT#ofCl2Y%6#d0)#<52L;?ZD0fR*dfzLzms%Cto|qr4u#{(dAc%z6MmX3w z*`(@Rh$N4oC~MF1knPd)*qBMojCgKh6awec)T#*!6OVJPrhOE@p4e~a=x$h*TdE*0 z4`WMlUdd>Ve?3HlQQAi3Ux{N*JFL{=UzH^6B=A$ujI;}@78IbfR&QC|C0yN3jGGy` zqDrf9>egC@x|^sMQ={P)PhFOy6QHPgKFg=VPtnL=lB2l|kbK!sh;y@5`qh!s-61g7 z%`1(t&Q2e=9*5Rugah<(!d?_v<+$H`NyP*p1df3E)K;cBoj9*;;zOZc>?Rd%am;^^ zpGi8ex@JOtwF;#N_{aye8ex`96ZR+nYz{*EWIV(o^Ieb{fz?C(8ep;Q+;C`{F8*Qf zT48q9y-ks>PP|QaAU{Z)VxU=fmWKoH3oC_=b+m0C$C&DJHMdjJPsDP1Va40bcIi+} zYc4C*W?QUY``%9tt_ylSHI~UsQHl{;lwxL5BR`~8*-t)+?@kb2+^kjFhd?$!>GGvb zHnjl?qrQp)D5IM^EQ(FOih30v>w!Ph{UBbTOHC;>QZ4M^4SnwbIjx$_O)u~O%uF}5`NcoKrtgM%=6AE(^t4(C!Tg$g+yhB5YVHA&j0bPFZ%0hg&4 z>Ou&06|4KG`TI0SDS^UFYM3Z9dR#uQxUY>xuI?=VL(T4iX?kl}K2kj4>10Wl54Fw3 z8^|EyxI41miyEAb2Gh^ZY!~`;LZmBXPO$;Z^4#;7a*Av4i|8CD2Y|~nNn?P)A;+mY zK5oBRmS<~8rJ@u`E!QmXcqs%e*E94p9Q^LKj|=ikqDR02ME5-}$kEc2_}S8vn8n2~ zo05HLah2RKh(c>aRtqtrwcg=v_BKBO*ndX2HAM)EZJ_}2r@h*fh-FeNIUP1x?h9za zh<{HnpD9e%ufVfz;(k#N{gF|&zN=%>I3#rStjJe()=M7gsV(~%r3NRfpXjEplk6v< zv1bDMm2xN5;d1?bYYCL^A*$$@@X-fV-=B(pkJl_ItiY`t0ORZXn#9AA}EhpHNM|NrrF);G`g@SV^p=+w>1j31vWU60?q2KzESn zH6wkGa}gZsMcRRRQH|`fziS^WA{avQHjkkHsaTXaKxPyC0kH=dw-&uQw zJ2667Zk@|K2m#+V5YjGpa1#%*Q?pK#C-R_1*CPS83G?WPMh`(q8LPARJ0qwxu&k&t z`JD|Qdytu#p{FYW{PpP$3)|RxEP8#T75vytYLu+U|E-9%qp$o8JX!!`TUV3?L_=3p zDvHfVWsJ0%LdJz8+j11d;x!!nQMKgCv6y00y`2S5;mkQDa(v2Ei;5C0>4-`;)%7LC zyu!ZOe4rCRCbGtPc3Qd7q6CA!4~)x}%ZCBi-KB#z4>JqHr?#5Qjn7Y-8Esu%EU=Gf zyBCHzkl~);P~~EKNC)|USbNKu%Az%E7k8J9ySo+c?(Xgmg}WB+PT^3vI~4Bj?!Iw{ z!mYTR(|yxPch0%r*ST5A{j;^^k)xr%NVw!=Q^)8Q`wT>M zl%eYZ&km$vF%DJa!4CPGDiBSLx%EuN4hu)#JRC$(l8$8yOeWk_v?Uz!d3XxDP&xyf zfY=tDzJ?q2jX79(VtHR@QDCwtD+{2+UWjKmEiIAWTqcfYCgOybGY;gg6-*jz&5%;E zYqy>qnyrj&59o8PqRyK4qAvaN+bf4m?k`c}vLy5jUFc)qN=&S1zN-cnVa{U`t76Yh zaWCY$puvJtD0?TBfH|WVU;HejI5{xiDAK7PzEIBU6vNL-Yzmtm;Y+CcVfH;SnWa7B zz(UOAsEK^SC7PG!XOCXPXirm3@ycCw1>@Bvi?V=jMdRCwNF;$hOzK%3LE+Eh%gGaK zR;}}8a$Eozmf_AAk6RT~VGDb7u0meZ*e7>5*zn)$-d3DbCf1~hjCC2sXRS(EDDB04 z9zi+6h*JBXMYP2kT}HO#4JOdwLh3T-9-dhCY?{h8x%O8rLzc&{XKtRVLktXibuV1( zj=Ti*OA*#8nkp<)7AN_B%OJ=$)-I`N#fM_>s{>f_aq~Rmb6a5Y0P0Jog2SdzjXJSb znNq>MP_cVuDVP)iQR;iJ@jq-$mMAcE^DZI0n1aS>Dhk*)H0EL=dk6~QJZx5QZsxQ2 zaL=q(2#_+YRm?6g`1*sY@kUCpkR(;}4`N+oi3xaW=Nl0i&#t>=Dp~ifJ=Ib9#h481bO6vv% z@_UBLpZqeNiv3K=f_USkt8LKgLV#L9?whh+NiKQ99Z?*M6TxA6?Lth$y5(^!F6GFc znSzHKb87|v)TaGbjaj552V`aL`;@4*a0TS=_!h@ATh&Uu8mH<}d`E`tSiUKpIw=TiC8=fmesD7lI^GzOcMI?Q9BbfCDCUojCTCA1Bc#;y*$mfR45Iy)P$Z|KHYNNdE6n z*Z+R1cJWvmsahtn%_QZ~3ug8PQVXl$Val-^Wx{iK))VwXk;O4! z2qNKw1&%!@!!F2EXVk^RzGc?s!MRx$6%px3bOSyQ@Z-Pob$n)bWMw|*th z({Jiv1u^Rl#A+kX#W#t8WTaQPtlO!4+{|(LqjeU)jMHabL+nGSY53pbx1(B{1IFD0pPP7c!pB z%KQc+*bV%G~wko7sEuZQh9w9gRx9XYrMRmg}qy$=pK!x;O5eZbK(4y1&cIo%jy zU*hHrhO2j%7*vjZK*|6c7CCDV!?Pcu(2GX{GCtiHKfYnt2kO&2EYdNeFK@~y91Gjm z8VcIi28znpj#B@TWp!p(8Hxab2d8szR~^cM*)zG17a@-MIib%G2cII)+9G{;T`_zB z!NBZ>G587FcY4>BZ8GqaLlmiVW|x<3Qe+OHac|5O8`2(O9fxmfR}kkt7=#qulqco~ z5%#+SPmCq-R*p0Glu`)m>-z163M#df z%i@{Sr;D&{;TX~|7U8cG%WfQ8yjY0!uzm=$&fh=aDgn}N2>dXul;4>g&_9N0*K0F;9j$h-~X~k)z*bFL_^#r(Nj}*e!$@If}d7~<}?18~D4b#IiNv4Hp?kd674sxRzgeFsT z;*LnQ<~VIQwTG}w`UE1 zB-IKpDeK7=mvRFj+#28Iz-QdMq(4NOOt?`X+#167xH}<)dTJ50g)b66nm{}xn!9mx2osPP zcE@txHxLO{(2U7<(Bc&vOXeAt$u|z1;yp;x5U2**V-nguY>wtX5ET&JW2!{E_tYxY zPvpo9wvKY~o3?Fz0VBzN&)Z>CgWzVx`#du!w~lkSWeMUSL6FTZnbSvfZw{tlipim2 zij1R3dw8{~IgY)wc1q_sy_MM3ewqAjJrW&jcHI+#W&u#v%no6b>C2jkJl+%v4%^R$ zs?tFDbJk>eaQ-3JLh<*^e(i)nvrU&CL2hzH4GeVGUmg+U*k4)w28Qm&y~kwENs_X` z`#)v+SuZhQ=KBL%Ls=C~(P~7F7;-}3AaCDLQyiQW}?B%zSJc?t+aS zs!MtFR74LR4JnvZCE#us4bYg+&_yp|M;T%$|7l(_bwf5S4G<1<1Aw zyZpTA?bv9XV*1xCK%NLUPl0z-^_9%jaITM?L{36P2iCoB!S&KwSK;Er@gkvDp`~=Y z(|yox3k;}0OzyD)F@%DOnjK7C8Jn9}&Z^&hVNbR8D_y5wylqOy?<&KGG>XU&GsbA~ zVXVUah>d3}o9Qz*>_zWP-#x2ugJONR9QeNJZ{9slOzwOV$8+aNRUN1zBRw*6tAO$+ zo~H}UkPYw3_Otxto0Hr5T7$04;NB-DRi(A7Gt=)_JMNzWT#WVgPoJ3rV;L^a^?DB* z-^)JJ)BQKIsHZcW-t8bKJ*3fgYWegP_@KN?4jKdmY~`KPP<7X3Pu*l6$L5O{`bZAU zPn$qSN-@__4mql{HjivHOu8~9Rb2D*uo=KBNxyCfKXcs* zhQN*g+=e&bd0yc+*o!M$-{K6t6c7iP|X-o_m{1%cp1SY`| zkdlueE?)kP@-LkBdyRBE?Xrlz9#y4|pnilodh>O)!zDitXQoFYS_vh&FCM}+R}-7$ zfDwhwFHtZj{=vPpFlyq30{liUsOIyY$psOqpxTwshr;M!@)5Jt9hf(3;`S?F*(oJ4 zKow4za*p$Yq*gWQCUH7?Eav3Yes44u<->1cxDCU`#ts2ufWzm2icY-BKABUGXh0#t z+rh1C3O(gTv);4_$C^-`ctt{3HQ7uxqc`&Um z^&nQBkjP2Kc@g-HK;!&jWiaEnl}q|#;VL=R${6{WOhR_{g@H*+nXR}JE4x_?m6xGJ z7|ls#W`B`$-YN}4e34gEIU^DT`sS9QcDyK}lvrYVU&w;GtLLT&Y1TP*pqPl`Fg*CKk3?G$hfgxkUN>7)>{{`?SnBq&_k-y!|ZqH>=ii3+l) zr?&$2G}skgB&;3yL{Kt^g+-)jmzm<&edrnIep2y2?SK7T6MHN1sI4Of@35>7$AX;* zT6`$Zrrle0d)#G>`Z`ZEYovJuYZ=o-K@A)AlIgSJYc#Cg_9Ut)l@WGumv-R7l|jRAP1gXlGyM*%S2GCw*PGCvxr5hB(06r; z!d`;VjV>8+#*bd8K5E3Nxn7W9NS+3XRA5x&Iy}zU74;;OT}%)e`$M)rx?qgIFu{E= zrGLjwE3MdYUKe+r6@Cprr>(TMHeWS0K6TfYXaZsRCujumQt5&e6XsXN+LC3Iq*oSy zZKCYj@N!UdzDCrp?%%0H#P-;%A#$8~S-KYxv{SvBtM$_jLS;@6xB|u}kXhs*mUJE%4FLaNkEN%fq;t z^ojeP`i?2(K{_F9$C!n@3 zDo)#Ey#ZmzwmEZlls}AcD+;ZuKRn2qL}PhZPV7R(zx9Ckzmp^j3|%M`#;+g(lhUOE z4Qk*uDH1_c-VmSmEFAo^?kZg9JA+jy*k497f)iaio7J)4Z`mub8<0k3+r}tPocHEA zUKL`BgtZ4FwFh)`FP*G{{^s2cta+L;)Fw|JJ|^sk$i0LOQ$uPP--Cr6IbSR|V}ins z`1ruM>*9T9W!%39qfiW3{JGE=&z#r=xd;q)MCSwb=uH;$gFzZly`kVk3<$Plfy^x= z&@hVpd6Gd)8=3pvgDZzj6(L1KnPO5662zeBS{6>VZTQ@)Gz@brS1yT@t3Q-}J1XwlRV-Vw^rU`U{7_*FCiw}D+E6G>ysNcQg8CPe2;d#8 z1EX=1QtF#!nF3RDuuo)z+PrBMfZh(CW&gaY50ceAGTLdP0oSMtz2_9#Ez1SxpwG;*+5uGD2dUWg(Z$H39rV+O_9A#R=>W=kU8g1q zp76=B)p~adjoN}B930pFSSc2kPIP}JB|bB9xK}dRcBoS8Ac@2e$fXHa_$o#6zsVxm zm1uYxXg}@JS<^Y_&ib#PMZ>xkK|EgAcJ2Ba^%aAY;a{zs_bc=Ymu8{V+sbLUO~S0? zi_yGVwlXtG6x@CfK-G-pIPx2EtIbuHD70z>s(0+sSI#9Z!>+(KG0PF`5IfdKCsj3M zVt1(r>emh0Ov*=pmpaK=^v$e3QfN&#=k;`jO`-@pF42`=b`4eBN*?Cb+MOfoi0^KW z1s& zVI}&sK(NSTQXUTV$D7EWWCI?0Ku%(%C&Jr^jLYp zv5ggn7gogbHy$-wqBFjsN@QKtav!5ZMzi>|#%2cV%es``3yQ%W&@ zMl_G1ZmUF{T!B}L@ODE#Qu;)VppilWP5lP=Py}ioxg)B(ZupQ6a|Avi&w~AsH{eHo zM9f5ggF+)(QQk~ZEsR(!_DHkCqU%9qXM<(yP-V+A=to=}ahB@fI$`N-hYzVj}_oNCb_jPMxaFT-n}V3zDckwE*25JH8Z*LlGU>j0_cWiwk9aXS)>(upNnv18izcxLh?D zObWY+KyhC2Ut2bI!EF1j9z=djCB+1y)wH|nM*Wq>)CS=DPS6Lt%}RkQeKMP{-#bAk z-vih>Qf2cWv5Zo4R32S9i_g~qS|XS;;W~Yg z-%W2tnm2cB7!r$p3TNSzE{k1*C z--T9~+=NYlza`w3aEp1iX-G!HkX}edA*e8pAwNDVM&pp2;vqGzkeyFO!K5&5CO@7a zMsuB=W9%u_y0Y&UUqzGdN3 z&!bBn-JQ2-C26nK_bT-j0sKvA`ox#m#Y!nOOiJwu!g*$&9C4t*=nkfM2Cp^@GG*`j zX_aPCqJR1zM`O0(#T~ouvK2h7bNT952*sRxbn2&y&0r1m4&SUB%m7*lS_?(p~9tX!qOQ> z)WO*W;WdOT6)nHC>TLi~>=T9$!`_uQAwRO;?SmV6cC%SK$T-NRURRGb&M<5c zV;1EzzW(EgK0Oq4{_d%xy_EBnNi99VW<_8G;vbJQ&+>j4K)5*~2Q`BKZVH^M^T_(~ zEU2)C7a$5tINn2xO+8wtP!`IQAGc|ss4veDItj%ii@x5}XlCmWL6W_x&m1Tyv$r_} zG=-r1=t3!3<4jL{u824g|4i=`a^q960s3SL&q~^buaA3b)d03K{!M_p*n?U* zl0eKKve~MArq7Y|L7|{KUvf`A@_-ZdC6*u7L7}KSwCwsU(%Xgffnd?v{<kZUiiCjt00efeoQ|AHL5k26 zEtF4IB$teYX?E{=&W}HhJolY~bSp&m+dNCk%>@M8=N~JNCQ&!i58FjF4V97V`3{Ue z5ODfag1Re~bW;}Eowmpx2f2K^!?b3E_pI~G|EP9=r1Pc#1Au^-e=$Y>b-}>@dC=@X zO}+mygBYu_@zuA1!e8F3YumMRE%2;VJI7g1x-DErN*fe^`dz1RV~b)c>ThbdmeN0& z-t$pqn74;=lo@PnRM{&nrC1;n=`yCXePzKc(ECV~tWv+WM8ZR-5p)uoXEVf!+8ORG zSb}otDd&b}^RgK9BJukCP*YyY=V+;qadey;j}hS8j#`fXT(+tuWlF8IiIh#~iS>7- z=xuE}aV{oliZDIWJU##9s36yK**y%j5Kh$$$)}r&4`oq-4v+n|gN=)KcqfW_ zc}yFLLtq<&)HL_CAtBgbIJr;?HA~jP&#Wm^`~$50jYPE z4GoLE$a{?}yA78WgD|oC(=MSRO&%R-Wdy17pWQ@L2mB-3|a%a;%mX3S17?GqoTZEXl)l z!A{WVNR$L}<_SWjp^WfbI87wB`MlAf_c|vtCJ27oa)sf>%$$aufR>;)(y|-2LUpAv zwx+`)ChVWi)ehB%9ZqfZ?1|r8BW4EqP;wMtt6r*jSz+e;YO5T_>S6V#|A8P<=ZFXS5jfFCE3&$U0)x#W)H-L>Px32F zbj{4r$T+?Z8pV#E+SBksqPhjhK;0Ijw;t58T=`J3tg+$Uq_Lsn^OEItefjW6LB6-( z?;l{+y)x;0{DmT#_BeV27(CCYzJVBZK7UpG*$vp*d*&e4w(KEtbk2owuH9JRUDfR| zaC9z(wXEGxalCI+hOG`NWu=>;>)HcZvph#B34cfGbJTB8=GH|Tt`D|w+~$x&Ntyg2 zk0NK0LJOvFiXq2OYSly2lmf~~>rq5jQaHtu`_o6-QaI(1*Q5<1CbcS~aY_R%q%RT% z8IoGn(ORT|R7tJ+XdY5P1?f$SNd6?NpJLJEIuvu1k?zub6p={Md<>CYNvl+mA4y!q zk-$jG^59*Pr!X3m43I*)lRi?D{I-JJAbD_x;9VE~1YZN)!jdlsTwR3TlLvE3WSrLBWVQvkx1}4Z#-y7rV+5#R$a=#TdoBE>ajh zn241tzV8Kr(X4I?sgQXDiHaF}$_PORC#FS|CC=oA3`%p=m>xN12owQ|wRIJV5&t_h zGP0+V$qhJf%B~S_it~;qlsN(=E0;95Db6C>>ZB2TaJ74S%rP^}#-15|Ke1Uj4gD+z zy#;feFk~<;l;wso?hz~36jG{Q<)D{)dK?Qc16`gl5!43`1}j&XFl;a^0_D0f<2XVI zbD2q9U>_Q9c90jsg?m}Q4)3I73xUFb3a?tDOhxm$8q@;zBD2=yE&`MR@8XS$?QCD~ z*sdIu<(e_5f{9}R0l>JB08luERJxAtdpJ#)vNX@`V{xlP%(SfSfQVF^9s)?jM{8|*5xI$9!&J?R%_JkEmwe0o#n=)QMfE=>Nf&qHm=oFPmR ze2QSfnm(|L0aeOcW>J^WM+#NSrZsMa6MTq};;R*8i=&r|EPGqv%Z1K4{!Cc7J->vbL|*R?gmHL zEc(r7MeUe@Cv)W}j|mgLZrW&AJ|#*sZMi{SGpB;O@q+9=>1R%DIY1J9#W!AJ2)L#(*+bwbMu^9^%QQ{DvuPC5&m#%>e(=bUKQ-C=7#WRgwVPM5fIaZOy zQwIR=)IzT?;$v11@icBiBS0B#zR4Hdml5Lu2-Z1j$Zo4Ls!Qsl-6V@{#JR@L{e$nT z2B>uEq3>KJjpm`OJjQ&v38cpuw(X@ds}F;I|GM8;b}|gK6}I~+j(wdbPUf=R>xT-Z zd=+h6Ne_(aL={XMqI_2YWVNjg8o@_9F|POrK8*vzpX=zS2Q1B;1b$Bd!8-K;SPf_X zA>rorvFa^3(7&TawDks?qjB`2c+H(S1*oHaLK23$VmycnH{-+2>XVh-fh`T5Ac9Fl z!K8|S_y~apPlTStq5aM@MPN>YrzplKQ2u*2g|qoB$Mp!)jktE_!0%her!J~L=C|D^ zbkWGI*P5H}WbiLkzWM;e`l5j0fH9!$dj@^?p7{r);Ljn$Tg`u1r>XRRr@=pa-^^|m zSv78%pgxBcQ9=67vNcY{Mm@aASOS7u1OPcQfSMPhGPleENcp-PDnq+%dC^Vy3V!MCka zyrC`b8LM<9SM?>>l87ZAx?w7*|Sjs2#`Rg}~g z_jO;gtad;~O%|EWA}Z|>c@l4LX0q~}>6D!4sJxHpwL;yQhu%<*3x3P~SE=dSNvyJF zk!vES#2Zn?T&5UXjZ0<%XPH>JE{%Rd8EW4cKdBb(ZCC-~o|ICKo%3;1tN~9psy?Di z(r~@hCFT67B>5ITLP{t`v{Ffz7FhAFYbbGNjq-=pv6l=VH7j+7%5h zYc)lba~sQA#}{>RUz6XA8FFW`mV3BPla61cJ+C^dRu|=;1}RxH++A2Cg4IGZw6gu# zsvabMQsV)e)X8l!-(sn4Q>Kc`2Xxf3tKqvNB(ff|%BNf)#N^`}fryAXGcDPdoE<$` zXl4-=4q2xP-tD337)wjs-jD3s&Jbqu*X_0B@3gA#+XTC$YzAhirjxJoe$#p`lY~Ab zjW|>0cuO=Etj|F>QbhcESr}u?f*@Hcw zWnA~mI9D?;InHjI!-2hu#pFIHVBZ77eDnv%hU3z1*Snc@S(6d zWwSLva1w&HgC`y3A8;r5CfElkqoFPiU8e8+B!bA9bz0A0rrH^R7^@f!=f%0ae+u9gu9ZZwsz0O$hrU8*=g2;6@T~f zwCV7ipn?Z?Wm}f}(+TG4LSx$6pY85ycb@o{4y&sI^Tvuw3$oR;GJSlyg}*NDcCQ93 z`B)ClC6$56a^w)vZ05DEG^}_p zbxj;!myA&y(c%HOK}g1AY{<$^l~{k1WNt#^PN=GMFozV4Hz7@#sOK}-V(91_2AOHF zslEytoS%+!!Bi>@WM+|Q5?#p|NRhkiWQ9osy(el*=W~c{LQ`vtuADTU)0_!fOT}2bo2QsViMW?(H9UY}r zzm-5NB$>0y;PSBJs{s--Phym#v6#~*F31R%ZYo9Q(m4ln`^tacUq8r{aG!PbMOQsa zaO2)`K4az$i+G^(44gT3b*(Af9+gb3oi{I=6M55FxL41_93=6Ywy zsmf2s|3J^Oo1#OKRSeDrx?6hJG|%YG=JV=$KBVTWzs9 zK?-^)R|>}TyF`j>TdK^qYYq}Ft0&gRwwZeAH;1(w`WO?PZ6~OupQ@jb(`!n)jr@Cb z;v|uGSC}_6?xrfqoX`K>PWc zBmXdi`Aeb3!9aXN3mqXi1-EP&UB&Sd^yCjbtUp|*Yob_~GMq7k?2&mZa(%=n#EE}`ew5~(v|6Z) zuiXsQ*qI4CyP&P@w2Qb}=B*yDKa$^@Lgv(FUAc>qWRmBVigCQkJ8$|y1gx>jLL?qt zBqZ581Ccc4o*J|0pgxO8{;24&xzl4u8zrjat!}YdtE6W@6nm>n@~sBBYpeslfAxD$ zJ8{b6jfck?hU6=4VBHN2a)}FPvJF5&;TXPvMnFH)v_^$NMAZ0Flq2&T_3S4#3nN10OQd!eT#Zc4C}NauFSbpIHBRfx zW!p?c{WhaAtM$x~BJ1d73;+d)X}zU=V5b&PZEh0?Us3HtRsD6uo z#JYO)n64LIx=7#_|M<;mGL)Xa6JmJTw8=YYp8b*2oS+>=j|UopdZBp&{`}B%@`9c$ zbHn7Rs&=%rmxq=sXSmpb<812XzY?vR?YeePcPPid+VN?fc!xG{KX)}hrpqOwwt%faI>(U z>0WfX3Cdlge+IU($Lsv_^2PyhUb7BKSn)}aPY~IPV?wd{*Rp|tCjTfP!N#fPBiJ@p z?+t5(L+P*<<3O9?kXxJ^iSE?m$A~e*IF0R(aCIciw#Y{^9bMWra9duZ@6}{0B&VXt zr-#ASH$uFfjJuQe+(i}ogCBIjTvM7EhHZM0`qakOv{(jQ2>EF|=?7@3^5@T!5sVgG~N1=Pu%Rbcc7v56raS(zRcv$aHHu zC1C#j8*`cte;y2Lb~tYb4C{BF3;Dh`@P$DHPj3;)Vb8*6T!_9 zK4YmGC49q(_dP_yA-uHA59BKBc$d%<{3=wj{2lBnyr);;nUJto_=Y$xEZh-{FXpbS zDlEs7hcE27cYhV6gBhgbR?icRDVX1r;@dvInKZ)<24NtQFKF&Cr!muCxW~O~V?lpd zXwa9Pe(SJkyk0Qh?jF}>k>pCm$Ch4DRz6K=PboJ+8z?6Ai860%wyJ_N&kBQi1iEsb z#ELZb0l2hK&Ju8&vJZ?)(F&b3lg4fJh$1yj!c(@Tb6NG3k|Sl6yKk+?qIpQz`np_Y zm>kznjS)s;DpoDr|Lta z`#{x}m5e345r{f8LGd)yfW0TeW+>7L(=t%>b%;5%>4zFpu|7Oun8@}t%!j>IFmEQw zXjeI#W^cuSG4Y-wc8G$>J@31ar7J$nt_))>aXQ-ebQc$#^9@A*PTVEWUq>M~Abw{5 zrxY508gG5NlHh=c+amWj2F>ir_QX(LgkZfZ*TTm6(Xhp$P~X8>(d3_jG5bd3aa3`} zojYcKhV{e7sL;-^#>S}5lab{JWBMWs6WI5pjfTdQ0x_9EN>S3vPDAiw&^6F(va=(? z&K8yJJJA_tEe3PNS9?YT&sUdVx6E>NFN@HE2L{8D?PL~7@L>{k8-Xh@i|}NaISeVzDz@Tux_AQKUrdYX%?$Xj2moc`6E;gZ(?tW z5G!%Zuw$2b@+wfdIWoE?kda4`_Z=!9(!IHNe~~@~wzVy3re&8fVoqGks+eahth&xA zU`#muH+YmRQg0KWmEq%vXji4y57}_kcQ2qjYg99-st3c>Ft)O~j_`!W%Z9gQ8lHfc?8zSCEqC zzCVE*m_1G*&uZXvEy1?LD!j|a9#&PYY3#uYiWj>;b#mJI0of|$`ko&g_R_e|q^S&V z=Qy7Pw%4W*pUn-83*oid4fXf%_>2)|vUuGHN$N3EUKn;_tr7=2HPn`TPjyv`^ zd&OVlGl|dg<@#R>oDX#3IvmU+5_kWWX7%NRY3sW_HY{T& zJSEA{izm}$|C?0N!VUY@i38~mYVwi%GpD&J2O0Y* zlJmb#q0Qvs(fA(*m7`O7QlSS4_r)>CnZJL+Ws!{-yPyPT>FjP2C`pkt$|EYUcWl3W z$PzGDFJF$@hw6H?!M`>4BA?)2G;WI`nQNNRejqs6bw^niYw@Av8!66A&Q-dbXe;TJgmVaD%2 z-WB)F?LWk#iT1*=_k@AJQFl!!y`1?p{LwAlGE1pk5DuAXSwW)cum+FW!Kyb5HX~fN z3Cc19T4wiSgRaUOJGrJFUH?o+cTZrwAlv;pp{0z-IuP(l$3cHm$Sxp5PCAo@54M)5 zS`fQHto)IAStG*vve&^^kzDz3qA+c@O%j)VFo{c*`r;5j*-2i%jMnchPUNGGCwnyk zVD;*489nZj7r4ecSljur=UN@#|MUxcCman7{CXi14e*_;7&M^3@srN;$M_0$RRDU=4`k}Ozqmcc_lMUhnty|n zZd7kDKYXwA@1{7V3bPRGEb%tA3eQb`%(X4F%dY0#wezoMC~MZ{X0XpB#ri<~b(#$NJFhS;1VzB97y^)N(!VDR6|CF?Cg*|KQ#?RU2%a5H zJ6UPY8auuJ`$M$HkNxNP9^oJBV7`F~q+YwTPdTCHSM0|OUWyswOgyoO7#4(JF*1C` zLoL*NSc`?Je0j%CLeRsGqiLVJVCvny zs#mtt{9VMm-2#_=EUFt_{@-Oj7A@113(|&&XxM-ZnsaZm^9h&-&ZDaH*s5C6WleN{ z{!l*HWzsS4twN~f6|^%dR~#xWTV%+#n#7cTJnbE+DhJZ{2iGx6Sprs>DbyXJlkd!l zJZ5GeAoczUH?q_gvp3h|!TTc{O9YN+IfoOx(I}SYViBCBJz8-p;6s4cSGhFZfQWZv zx=q=VfYe8%m#gV(UX`Sn{TB6lcsmnLt|9bYG{t_;dgzh}i*zy_qk5}Qwz4jjrU7ms zP5brmTx`V<1(}pa_jRt5Dxr(HeQ_(ejVqO!F@XY`feR-WzQ)rHpfxB5zP@8u?xDz6 zL>hsUXE($wNWCn~Hgn%PRQ^q$5Xqej6*n)ABjt|S-O`V%U|+gQAht(ggzvk;OtXTc zi>3$#w~bJ`3M@3~bm36@L#pz69CWOhz3weejR>KV z^aHyr71#AKMQ$^Few0}GiVLe7?f7TnC*Qu$-HLUST2r2Olw&I^k}fJgm&EaC^r*Zt zS8SNRLYQZOHJH;T<`~H=&*H4Pl4c2;(aA&FGQ)9X>3sa?0RaD__eBt>4X;B80>Y;D z@1hG*{})#F{|nmtSA*xN|7@#8@e|)ve+Lr;uHoS~ZOcokF|&|lMr8A)vQ|;ZcF^d9 z1xy7C_7a)ngOVsSmW*|dWqCPR%*W5Pcz~pFMJdNAM>Qy&G*;JWT*pY7Z;brN; zrUY2Y3cKg($Sq|!3nbF2zNQx8C>Fct9?U-cNimC;SkI#8V!}&Fyi8nL7)~+^PPVx; z->Pa<|7=Dssdd#lW$cCoEB-yvRl=K`Bt((#cy6~t-lo~d#f!#e{(VdDM+DuJG)e@w zmy&3v<;+6N(NLV?%kGj%{1&M7RnN#2nP7gGH9 zNWmBFN_CO>i*|LRuCJ7U={R)F)Xt?df|H}{U?${hY2@0A_dr{ek5`g)RkTL%SnIWi zd?U%78`l3lA4qD@r06m|mX++1^!NSY13J*s^TK#6_&Wc}IKz0Y)`6T7&NC~z1VOC@ zD`If;S87O3`mk}`X*cd3Uf8iHl*3q!jC#RVk#f)9RO9K|fQ{9CeZoOK$-dw0GLi>= zd25Pu2Wb+(cUD&E|k^BsPu9wKexM~(7!Cw-B!f5!+? zum8w;9R#dBh2L3W(K#`eo7fwqN+IepT;@HbZ>GqJv_Z&E`M%%gZ>@>5U zmS#|kn)9AwldP!ITNAfVS57lC0>#j&2E$L=EW$}fLeh!O7117d}_P9g4#p z3P~c27BTO=!1ZG=Xz(CL?PIb*=HuYKK<#7mJ!L*y?IUFocu+7Il=M#Om$n$R*qO4} zc_4$;`m33eA`=G(qdxMjq{NI8%)09zyj=;Xn%(#Lf&;ad$!s+VDA0s-e+3hR_U-@` z4I5`zGH@S8EqNZ*#;3o}zQBNHPt5KsxguQh~8VgNW@f|-dC=vS5t8fJD{U#;M4 z&k6k7UCH@>yet1lh|f=HOaYt)!L1$f&3Ttd2bMXTn?$WBF-quultzA&M4b}HJiFo@ z7g``NUL;*SP)H>4&xNJn8^sJ$ydsDrSO=sXW}SX-T+qOt6`E>j``l6PECbQ(pqQ3q zR@VVw2EC(=LvNi8vNT`=FOLpOr-~gu^+tkC6;6|=+zKhIuwzVwiQSukHrv@bAI;yP zlT|>sJym?f!3`2`Aw4-?_E@^lU)&dUU{jsj+MT5d?zKNv0~b<|bM)8s-by1R4j}Kd zb4+r~t6R?7D= z{F!Z$R~N!s6P@sJGo#V+ir1}2v9K<3qEq^RMnT2UTEGCN;pej{h*c%ua6L@Dm zIGY?6vaLWqOSKymC=6Nn)t%H3B(0~+9D4p$l$37-TM?uS_A^o45swVPiCxnOTgheU z|6%Mcqas_EWzojnoyOhW-Q8*2-QAr=3U_xc+}#^@hsNEZacCU+vDZ51-Fx4Cd+qaM zjyXovpIMTTnO{am;1`*nwCpDTj_Ga8cFr}*wOpqCB{4+Rz1N{u`HOsc|E0!8-EJ-G zWx%m{+9uVZU7>F>3@4Apba6xdn~XF)Tfm>+cQ+4+j1bQoaDdh`Ex25*ZF8Fb(z>z= z9TWmmF2Di)M14@FYo|}m7!F|hNw&xu>d^pdgZZ8PGL5eiwQVb!YjRev3Gbt3qKBYd=|yI9Ysw}fXdNxD>5N$dQFiLAaJBH zy4Y49R1{K`CeZLEOOo1 zSA48>>PRR9=IqOyAU%7D4=J8el`K6;xT6a{vqiS@Y9J^;u$&6pwCH)oBfH~4_Y zEApOIn0rKBBSh-PaQ^G*FC#v7zEBvSPkd$lN9nNtIgb2aQ*Pm9b&7uyx_a(s}nQ(%=aj5%X7!cI=WR%=vR(S>ld0%#*TWD9x z3>$;!AzlaOxXi-53#RZ33JlivK=4hV1B4dqD2C~YPmkCYt(T_lYi~F{*Hdmv)71jv6WJ_ zTDn4W0;i#4h5oid6xHBd-$%WV1Ov)ZzmFBSJ0I4s9h zPoM6m26!<|c5TH`24YB{6he#kd-ts9q(N&*xtSTA{Y$oVw5nzv(9Gq#5MiT=n>FJ7 z=|BFQl5=yL=zPwtvt>#Tv)@>qy_X}M!6d!nv4MbGVkcY7qBwZtY;p*bCAgd^dnmVC zjV9AMb!seiLm7J>p1wl{ynI&e!lDAY{6*cXk=n(#%Sg4 z#c1N-Y{vM1gQ}dBiJ862f62f4{Wp}R+Uv9k(GnS8fFv|*x<9ip3wt56r=v>=iL(jN zb&6^=Y}%D{l6AdO`YvMWv!jG3UW57G5SDwSLc(*HCbB;6PqurQUwyp2@8k8tW2#`w z@793o32O<9e=jiPK!ET=wYNZ)WC$LF*hrkEEf5SwJm5(X(;vcyRFgB(6h{$7mH3vJ z`;9pa1miB<^n_kP<8caX!BM`}+!nI`#ySfAX7AnxvinKV-p?XWr*H4iJ>~8t1D0!9 zE&lrjdxm<%SJRTU*>z~8fSJgvik>nY^MR&^wp~0t+*HOk<;ROz@@)=vQk!5xF8Ps1 z{S~`2l_5>9h#KAUZ`g@(cr!4g^mM%auzIR|hX|Le8;yt>ZRo z%l$8iSd3HnX(Z3?7%W--RCFI33DIp<>)*y~rHlaQHH>HGb>ms`jS^PC;b8Vg1@*)# zjefI}_KzC)_lxZ~Vl49hUZ)t0#`e~+i0tl2asU3P`w33I+^MWoN8Im>(G@06wbF0j zb_^B^c|-ceV=0&w_Hm2ET`;un=C(j27Bq;mi5znG5|^uUNQzQo6Xx$tihrnt?-Cg0 zQea_|;fdf5ob*FS*kD`2ND)(*H9JW8wezJ$2Nj9YHv>y*RVm9*UZ}Mf>t$)0-G0ge zq*U02L)c23O2z+(dBofG2#St8Q2RZzl38AD=>Obvp;T&>X$p{3gx84x2DyLWG(kxHgbKLu>I9Casbd3l6T%c2e_%aeS)aeKe`+IJLks#`(G_D_TOFV-|^k8 zZln5D4clMAJ_|tq{mYtCdARm~vvgFGO8L*xp%OrpZe()BrY)k8(<(3vO*{LB@kucI zCU`o%plSawi}fWBZ@W<%det<(oYjBj@%;7tZfpC~)#B>-2WT54ZKR~~iSj`a6Z(c) zUHQ)UPq0P8Ku33kSci?on*GiR^puTk1xiUX(+#Dg?=y=rVT+>Ijvyx$f;kL2qG&@g z7)nK=P)qRU?^U2*m9+(bRoO^1))vQ9r4bm4j>Kc3!DOm5Mo6MrVlLQ&n=F4)@qdDC zRXZoD?jMAraVLLZ7chq|0W`vQm+x-W_~)0>Z!(l2zTsMrhx0hq^b2*R@7XbUW_qnQ6I>lJ?I2!5e9mMKS-?doqd3A^yBLdiU zuD3Fysdis;eS2rqaJQ_}JPgjzJCe5#0qHfjI;->R`-@s$=$>PaQ+~T`&A#S}fv4&L z$HJzQS0BgeQ^9=f0}y`OvL~;sXx5z3e-fog?*7dfp`bts=i&O+DdW|j5WmBTZR7PxKm%w-Ii)pbpjjr;ASk;!N5zw1j8OWI z$oSC^5~tvneH2*Cgo;5FBYCivGPl}>-kYw1ho@78t& zYD*d}wHLt}n|ed_XCc@RWQxF|%qFzm<#Zf9F3bBKE7d%nb;Q`&B$A!<-j!lwOV;^h z0lfs|SPk6`JhmTXHfbUDUyFaQpl?Tn)Jqe5MI%+;HQ z*OThLM7RRtq=rXfJ>!fOywn%iD}Nx9@%GNeUROMT&P|basR)-2)12TwgJBsEHDo~J z6bzmm1Y3jjA)8kZB%2ljML2RyBLcrKn1rdC{LR3ROfN^g<2JK}}=8goDh!esT z%k`c&7bUG*=p0r?uEDH0swdKr++Zsbt6dk4L1zRu;Tb*?PiY5LS$FzzMd8<=zJi=1 zhgtI;eQ5&E^yZ1my63k_^g%!NK0V)j#M8aO}y{ov`TH=uZX zh}5}Wgj`Jzeo+Jf1m=xLc>kbp9#LeE=vYj}eN=r6?~t-l0_v9n<1G`*a&EbY%>V}Z zc>B#?{3TZ$KGG+zc9+J6A-bwnfTyF(FJLRz4!Ri}mw{CrRGK#a-5+1V#NP2ZZ5BUp z*^%G+MqX17D7O@iJ_!E>T)h|Zv1*^7RrrrWWdC!&$p8OUksS4P6*OVw3kWjE_*Z!$ z<(tI(I_O9$DsWUXOcdA(-}ut8OEUQ8Rr!nQlF{1df2|0m@^@#!N%w|Aaq(Jo_;^pd zdK}Jd8~gw)k~}OaPwNd z0ms$&jdsc1V?$-8G?9lZN0w-jGpMWjRxQ33e%XAy0-=1bN1;sTx*2J8&zXh>zcDzfW~zc{Sc%Ik$-v+GodDHX-D;d-4j}DD^Q*lw}zZQI@hs zV^N}7#oKkFS=e0qt;8Zi$}EQ8(M}mRy^b7SI-@cUuTd*O#*1G2zC=Nl`+|%I{Wbvf zCqsounYrafjTwegaF41r?r)_wrgZ$JILP;sf$kUUvQe|~25V)01WeEp)uE#{YHSki z7)W(_wTOd2lCKE>C$GbVdVn@HTbAv9Pa`VQjAa6kR9kMAl6mkTIQKlUi_ZcOd9Lpb$ zs%nz)qvfXaBT0LpVPfB(Q)N1rPFG2NjR|0RH>j41-XJs{vnXV4Wy=>rGpjtV1aNIa z0cb4R@kTamW4E3vB_A0nT_G<%oM%^?`2o5-t1@c8 z&1xYZ^Wi!QpuC9zf+R;iY)|l{U&60;L3T%wS+CQt>E%!^2k>?=2Vpq# zLzMfGhnqAdj37wGn*c1nXQ+RLXMFSf5B5(LR>gnxXH)tgSIdg-X3ox5 zre^<3hV_4)M*N?LV8Cym0;9sn9~5L-iB&?#VnTzQS>T#pPoQET1!6GC#!fpNtM~&- zWA+=ig-=LtFu8o*Lo~uSJIa-BpRo&r6pAjvXDirq`qAOzUJ>y2c8k&LCTG=Fy&DUF zLGWOgvu?u@hPIQ*jtkm#MF+`_Ac;mK=SewHDb$H@k+~n74Rj};K4Wr}9wAcUk-ILERIh)j;po@$#BwEUgf0w(}|E8z(HDHH$Aa2=|pM6@S5w|un&L3-*zyO zopA)w4}fv$t{Q{e&_*kvb4XA91;cu^2E==A$*9p(6xYH8{+e7*UNj_A$6kxsj-)eN z3O$UYRL6OVE*gp>iQJXfOZZLt>XKtvvOMg>-9n|EOZQY`A%G0AyV76JL2w>Cg7v1u z66|7oqmA&#qVGDywODWrIorvuZ?=mFzg~0PBIGgJ!a1aA$y+!A*l^8f6{X!|WhATM ztwhcumbRH_5y4%)(vK6zyM!w`I^G@&Iv9mIgMmY$wX1kV;2!VS>&rwREI}E?SIa|g zGdAHYiP^4L3cne`N6hl93+wZca&<$UhdZ6@EBwaHvuRRTGOB~3|I>>5&G%1Wg{bPO zuNeakzRTc*SLumr<3UW$8x)gV9-r4$CJw<*NtGEv6YRHED5*!4nk`a0Ao2yLw@lLx zs5Qu(2z=R3@y|g2PD_S%5QDrW)WQ7>lVEBMr`-?$7efebER|>r1XmJ??r3n`@~N>kR5iSn_y>X$@TGOG zE9OMUl{%BA3VGlb0nU5-T~a0GGbIM*VE<$?O<1T1%hvTgp1UGRHDf#KAnB)O!+ zlWpNx;Zf-$RpI1PB>F~jgSbaxRgLxX9I1nJP>ws0r%7B~#g>VpVd$FzYBOr&rI;+^ zJys{7azVjz1LCAbW#sHBKhfXcM5HkgPD}a-O zdY37;V-e=Fh12HO9tdpi{ExF`DwS?qKs&P#w_)+qteqYGS-6h&!2-Wkrr&7nW)ur( zwh&Oq*38v&gDv{A?i0@@Q(Q`ME^MgwEuHkslYN~AbYH3ML@iA1o4?J&TER9wnz4IG z=+yc#QBPSO@xb|_Z_pj}sLy$2k5*W$YJd8= z{J7m#s)j(+n+kx&NM4p+)v7ZEzOKKuM&U{5tfSEUq}xK@Wh?jWfFbA*3`Da#A5~o( zR_H!VYRPNu3RC!lPBE&W+9JHqaNB*$zILbyQHWRD%NVDD9Pg8j@zfKznT7+XM<13F zB}t_MjYkoN=MN_xWXPM+B2s{Qu3lmE6zD%}i+m5Uko&!FLI-S_A zCiI-iFf67D=BYm1-kT2)&OAr}pXCoB`W?W#Nj0v0n%8bpyP;ukXuziP2G=lk-P7&z_oS>NktSn>R0 zSpA3g_J0d26*Kow6G{jB{|>K|&-NBg1KU3q#uTP9BoM1fsjRh540I0&J@4FDa;O5{ zyrRxcmSt+y5Rrv&vit$JMmXyPK)R4aU}f2FGI%ZG@!Q_48%MQa$>2Non-Mtf&du@o z^ZYoa^M%y2U6#G>4P?+4apOs0w}@mdO@pyC25;x z!U04w10t9d8eCzW04M-uKsggnydkzpTc2apE+QCylwta=3!u};m8BMq6UUx;!yMa! z$Qgb4Th*Sx0LcLAKtm*@6pL@s9u%MY4QY>h{dX-4)W>hdnlh{>>8c4v)D!Uo(UG7% zs>*#qk#>`?LPGGff)vQ%2ow+vg^OF?nb`wmEXrMV^>lo{!l$d4_4|`1- zO${(wiRIb3Of3c!D&n$==#i%~x%aNdYYj2ca;sx(bIoz{SomYa1`hFWLfahU5_7JI z%8+qfRgb8eTO0GL4}&+^<}?}6_HzXbzfTl>&8^*@2w63L&Hlp9e}Xjbk$L%cbYK#r z(Xz_((}_~>X!@5Nsq~WKtD1>GEW}zBV}LgT=Iy9BYVbWFrezMB+(0fWmrpjwq(692 z<4_aBY`tOniwdx=T*tb(-F6 zJ?M17#&(>)bBeWEfEk5Tnnm;lv1~qe;DD@O1-)%ofi70%f{W5Uq7h*fIVCa;ELV1v zcHKnzm~o4G^<53_X>L9#s0y;Q7Z`ci?l*Pb!dVru%9+B-nbDLdk}iV@mOQg-u^j17 zspbQs^AZL5p-Z`7VoOM~^)+fvW>o%mgn5?yeSW?Vyq?i~B(^||HW7fO=FiwakzazE z1Qx*YapnQ~MY2&pBw8`;7Cyv0FzRU-tdxh;YItN;)FAPJDakc70&dCf8mAt~@s;-p zw_#Wz0H$4R2$x`%c4@6G3C3ZoKZw$3=>ufyj#!G!;x4HG3;J0ESYDxD2eaGZZ$YWO zTeAk?bPZ8$`z0P&V0aoE5f`@%hsr}kTD5#L7-Hvm~JN8z!XtaLBc4D-h$EYr$i3@Qb@UWv<9AG(blZh@-5 zUb&9fensS#?R9Ak-OXVtUfdP?JM5n`B$quQfU%R#_41JtyRBUI&^q0->Y z)&J4X`bP*ei^$@N zxs|jObTM5+jC2+oTnMJFie;TN+;?m#wAG#!SP|zC^!Z8Gwds2xqaKCgE>sgEpDRoO zHLG53`f1$Rz3!Op@T95rOt#OA$ISb5VNp@Q$NM|dm*81YPu{@#FN`GZb&3*(}=uwWP<$-}uAi39CnexDqZj1L8ZN0^{`w|mNp zfN6Yf_e6*k^VqfxjK;>oGVO|+(CwNSDvNgVr6LXe#WL^98DoUg%0#kHs+)AfTr-|N z0L60Tq%47%#FE-hWhfNn%Mlp84V7k6{j7>H&Go` zyD2uRxWW$g;&0D;z(rqVpLw^V!$r3*if25uWiMrFj_Mgkae~wJi6cwjRs$r7YSBTp zD>-qxKjjoN^F7OCk|z@?lnwAt?&S1VTWxp?Fv@xK*Oh7jIw6q3)rz<_E!GE8Hq3Z> zR|k|i04tA*4*MEb8lRa;_WUoad@4!?V+B=UEvG;L99s=&KGlvSFQEdPh|SZe)3U7> zFyf%b>M>if4{xhThg0gSDE&K#8B||1>ZXLn85Y}+{+=$v2MJZTtoG-lypebKrpNse zTLPt!TACY|Z0WTy$+&{?y{FVS`&uow7QZSnf+n=I(@%>(Du$OKqqtMrZR0PLu2eBhwm=V(vCF&pmhPa{qTWnFVIzdhz z7ikLd@#wv~+QN~{0tO3kwI@$6CQFa!g)`{u^(#kFmlSnJThu~ir4gr_(d1Xi&9}b>V)C|^godBURSNx21F`>L!TwoW{f%7z-^!|4)5jNG1LK2Y zbxMu{j7%2P_WL#T_bGt}p{V3*3VB&ug3vF_$goN0WLgcY2(QrJwSPs){G@47hZE4Y zV+N`mwo7Mf=WEWZ_gR!zt5<6`Thl&>(|#b&i(xk?982s?caTx6 zi)t^8#Gia`nM5D;<}1n%j2jD-0F;}G@MW<#wLveGfC(`H4i;PUmGRjz$1I3dOH*jA z_+UtY1f(6d`O5G$3Ls9~a7Y`Fg~P$@I~i23IkMACgw>2GXNd6?9j-=XHYmc<+|s0N zC`cMwlPSYbZWt4w4nQ7oheN3_(VVIKUXWW5(KoVV5zgeh5=3oTZ%88$AIjIndOq6E zRR)g_PiWf44!4hD_7opIurhTm4uHaWV0LBZ8?4LQHNugcqumGZ>Afb$p`0V#hw;@K z_PD{oQQXtVDZPOK_?X`BC$Z!X=dk4Vahm$Zg;~_e4r>7judgLP&7lcRYNPY&&P$)N zqb*R9oDf?lJP|!6o^X1o_q{fV_dTAHqtm?hkUYN)6W*xd1Q@2Ws4HDouFu%6m z1V+QjUEps}?(3C2c=+lMi=U%@0TKBg=<1`~9;oZ1JcJ1py?IIzYAo$Aidw_gN~f!d z2=V=HjvDtBS&~%XM=4(+AX$n@42@fqf+pp`r~#L~kP#W4-6z={STFwjiKoS9oVR6R z@O~F9rZ0ZH5!fdrc42BW0xe@F?69b4Qd_F>E5%28l|9H=Ge(RmlZjEU=n>0$yM6*GppC4wYI6d4t4}ecLAHzkKqD^f8zh=WPRizE;WV$7Ws5l@Y0m!6CTeGBJs-d{X#~5 zjv`f}5UQ2)mw}glagpw82tqwW1G7>t+QzyTPyO_`EcT#d(`jxg3|f?GclA0QXYuY& zf$XMPomI7$3uVk#7hlI+oW%w9DXLV?Sz)uDPmvsI7w8ChwMKd|7b>giiQCe#_gR)H zcW;esg(BRqOMy^bTv+$Y<^bYREp1VVI3U;8*mLHr^AkWd(V`cMdZk%)e?`Ts13K6% zo(AqQ=c**Yqe*lLh?@bHV&qm7F&Zau+GVvrmZ3*W`rNFOP=8Y%Rg=;DiVjJZ#Tq5q z6|x+RkWiEP>VzxWeWuiidL4?<(+Ms|Hb%3zXLDPwKi~$*K_-CfiB$Mx*+j8Y0SUQh zLebo(M2i4vH;75D3v}XCj(d{X7-Xe=0o8AA?1*?6z}m5(pL)6H6pv(ojxh|k>^ZlKA1ej0pjc^)fe^YB2#gpzns6MKfA{E5nsfC zm2qG0F6ifuU!;s8CsoQ(iuCM;Xn~a4*->8zbqLjGV7?iGvk0m`b#Bv<5sa7Vc_Yl~ z!6}1q-eT=;uzaCQ#?Bfcdv04|uWMnrDxF$OMz@K~KG8*-db&x)%G53I;w{9PBR|h( zO=(#o$sFvzkQ0loj?23R0~c_3w&and3Dp#lW$1&gj^Xu)*HJQ_ zr!t2@B0@sy2l%A}7|{2KC;R>Bh_uZ-Kc(f9*c9sGPn=*`@fG^Wm88N@Wwo2yUM<^4 z$~@9BaoCuLbz ze}0($SeRs4aZJhzWQ&#j_{<}^F8*?{Dtk<^M&e#u{}sBI!p z!l>blw)lnY@~7IP@ivs0CbDRQOc` zF-goVoxE70@Y%CireBl@t^~}fBP<0S%A_W_X)moQ>Qqz&ne9)jrQdzp%Ls}aGv`m3 zv5_t=7ony=6T=0!Dl6v62*s%gBZdOCXj#}-Y1T$KV?NMad6_Dv%#A#mwt&U`>X5E%5dN%O}yqh!86yh)sWgK-r< znC&)aJzX|Z(2K~(koMi1BsVYpDR*E?VV>KUk1z!`uvQa*?~YiYA5C|5B!YSyPa7?o zz(k+{$5Dk|L-L4LFP$F3VZ?HYUSinFlp^9gUN>1vc{Oa^0YEfrt>H1~tO>mFOBd?A z=^;%wXa8;JU(KH5bbS!Fq%eTT^XD!h#*FMe_JQo;mGfUi9IyOF)xggI4l>q1Rd|g5 z!`}2?s*5gvpGxO^9tohSq5qN3GA$WHm$CegzNk{yBrPL^UQaJe4FM}pgANOULOI5k zZD7RhgeKqbKxVHxrqu0BWT?x+<_W;H7`jm1_32>-uCmj?7s2{ zcVcf%Z~=;=GYj~%zqr*O@s1CVh|Ga^v$B+g%rVUeveDNKpTv2V+m3MSwj&rpxW ze}UE+V3uU{Yi-xLuQUPx#!V&_E1Rl!QRy+aReNA8Z7S)}TpvlqaemNp9cgH2_9} zXj~hPRi`sZ49}EcDuQ-{+SncaDvoEq2mKJvHQv7!OtL5X@SDt-M8YtfEA*X2p-5v8 zhm4EM8h3fov8>AB1TP*?98Uv8EvPvO#9PR^o7(i2kR~7HvzWI%iNrmdJ&Z24!Fge# zJr>oJQEYUZpmVVD6OE&ObS$fISkbY(=y*9@3L*+#c4{)L<$QTD1=Lr>^X_V@_3D!a zkuM`Qdzp|Q8|E}&Mf{FrWB={NrdYbkI|VF_Pk`M{cQ{{UiHJ{5HGX%N!oh73xTmK1 z&D}JI2fHwwZpn*lG?>KPI$n#+ix1qEFyM9PaUTIlg ziILg1%f<=kx!II5Uy-1svZ|yqKbCCtj;Lc@Fr}a~{Ijqa-49_C-A-Hgh{#o}!=p)J zZH;^K{F{pNpp?YnBqDXi_+;V_1S|}Y9MMoc=Jy6!Em&=ad5i+~jm^x2h_M6dQ#}Jb5jat;gditK5ISM1Y zxB@74^ObCN$bzxa?Z9jtII3D3^AW3GCz;LEC%rh-6|$$QE6=Pt-<*iP{uEV~Ul8S8 zaFR}|&(>!K?QZ5uuA}^7MRIx%O`O5JZ3rv|!T4aNyH?Ea@_>)(5&jLqDnZE|%ok|R zRhOvGPM;j6&**;7(~vsbM#T6@5;oDyP8yTl>c!<<(4H)P((;70;$64aj;KC_^OR*Q zU_LpNe)%EcM#L4H0$fSSv!>NODGnmJKl@eqEsV*sR9h+nyN)GbhvX=(aQE#`0Yl@K z-@ymTxAk|X_Ov<6C$OTn9Fok>mJWR5+jKvLMLbdTJ|g0NTbO9l#4+xOO02$CwH%z& z69GoI{2SM$xwYCp6{w{0FVX=TXDnaDJf_$ZR9LDS;?teZxW0>%>7?xt`<%qqtaT?|QnP|~oq zQBPl8STzp_i{M8h-q4Z{$qtJ>%5zSMrgYidpIySE*w&h=t+S1%n8&nxO{>?A>oi>; zad;`L81nAkMh^+wslT|HH!7$u&*WoTl%Ke0exE51VXT8OFot6wiLQlSSvYAF&LL4! zz;%OL%z>Bs4L)8r)H=J?>Ln``U3i2VghiLuX}htTAJfTgxnC2>gj8WH4wH$eK~}J| zeRe2y!!6a=5TpBHWWBwV8Fph)c!5RNU+Y@sFUspb!to|NvZMH-c#T!yPvB#>$j;V4 z0cYDMTQ=qJv3>B)<-WcZbumDvkl&Cd9&IG;KPX5RW<}R0`uCdL(mg@PpHJp?7{Wj0 z7MlN0C%DO49Jzr92hNLs(*Eq)@5zL>M8) z;x(goQVflqO@-+1kGZSl`{55>{-7#Iek(u!0&)PbW#C)V!H{=RJf9C!qTD04B9W>#h*3P}@W@-!&>%*$U} zF(l@jYbM0FfU;-uN6UWw8dg$i^PM)Lp#ZHaS&PLE_a|Z`V(DUuago&N*zhBjm#4<{ zq5Hi^4jLp7RI^}GBHNS+!(uwS=4Y|5gQ(JlTim;uvGfF3+dlaw*GcS%r9CqLq#rEly=j zq_T0w5=lz(nU9r>x*WND0bT1ds0C;Y`nZ>s*41lS4~lAlFB{(NxoOSlyJ}=ep0bbT zFVm43SVA~;Fx-2X40>QtR2}+pe*@+(^MmMv3MSHGR9{d=oq|{I_tNMbao$TeeNG_9>>B(Xu z3EV1JT!v84XNPA=e7#EZ4@jqt3wKdU_|V*TATc(IYBkf_FYpVd@b$Tk`g~CRFZWQh zy1rjnulEaWM#|{u;9C{GC%-afl4f=g(p86srr+UpNN`hN0AFye_h~ZwWt1J=+)^f# zv1W3*H1K{&w~2k7Lb4{0R&~mOTL3k7YLOg1hnHAmmW!R}6mqd8EC-qdHd+b6^BL=f z#*9GBtr>cmet0}X+|VCZ=NlGz9B5}OT5OPMWwlrAoo zqF-zqWC}C*j}6Pm+*7W%j>OmWqY?0H$^~eefkbD~!}SUW&n(eVzdw1VY2VH8g3i@z zQ>)oDW>V?RD(6tcHM+@Jo6h1$32BPG5!=4%v%*);F9S9J4+k*5X&!+E6A9L8Jux?DS75Q2Omv=PV9TqjepdXwPHrbj?n)F)- zOEz;k(qLq<(M`>2fD_MvJr4ZH>Ad(ClIM5N=3VM=Mu|3^)1P>v5IYH+ex3GQnS*w* ze~?%d%dpmJpVj~+a9gd?KY$jA$KG(I3V88OOL{GvJ%C-*fm$}4PbDXS;9NWj?i}Gg z={w$}=yskVcIE^4fI8;bmi#f5cY81Iz6c4;qk-&Ubvb&>VRaD@*xmxc zD@Yhp$NY__GCJH~9!P9b_+?;_fXi%wnEe9pf$zsxP=CoYtWs5r4yx9Tmf_1U# zhnQf`bhflXg$v|62{tUiSn{Rtt2`u0*KRPg8O@)c9Nxm{9eXOnVm+m8ktJcwBfVvM zOK24nls7r|Dc2%3raP?Y2lr>k?M*a({MaxWbG&bZ77zu;*FKk9Fp|>*c%0CotT2E% zO|FIFoV!ho5jJdzeVMsJCock7i3c+^~ zyLso^%JIJ_3ix}oPO6`adN`jng@3R@c;O^)CFA)`Hws* zOV(Hj85}q&I3~JgVs8+UWIqbD1{oBpl$4N>yN5gLn5lUVDVgVvpFwyTl_zbXrBzD1 zT*^xfjo`~zKySdn3O{d_4JT%1_1jVRrT>ph|4oOZ_m9V!A756YukIM-6%-(^Ex#y& zJL(LYp*Tdfk?i54Xoz>w?(w5sMYU1vfuRsawlVCPpyY)=efyV3Ib%Nw8%wq+*U&8S zr&zWu)u25L$r(pk$r*^a-swqLrKGT3Qj4+;&WkPtZaKAB|v}O?4 z5LI6={76(ivJW;Q3ma<_OA~VwR#EbW%xVof5=IgHD-I%acDxbscs|ShSX;P1PF#V; z@II3n-`JY0)VOD4v*JL?4g4NF49?76b~Kiw#IUTAX;<_ei@vzru+2&0XNS0D=m}?G zw$V$~y?*xZs>N{{vf9D4!D&+4eniM#+i!jwa9*8=JL5gD) z^-#fF>R;2gSN~5e^8dXmySst)QCQnIYp+X)sI8F6hGgTh(Zn=^j-|- zU_y_`NX50n#kOWcki<6yRNrC$&T`VwAZ;n)s*&HlDH3N+E*@WOzN>0b8!6zkeVDt9 z`_&zDa$Tp>o9L)I{Z2*1jZ+P0LF=-{IA$jG#_KZr2MK6^Qe$Yx;FWq3m?hye!W-hPsYaEZe$aR(-!>{zRI5Nu>wy}%TT(@G97@c9TqV)0a#^`m9?xbeE z7Tw5J2IjxhzG&qE8JN20^`5V4HA2sf7pvXsnV)wvDc&3FWT5CVVjGuDG+qzGRu(oQws`V=u)T zZbDZC{}TdaH)-?0%etM_(&LSW zZCJV9N7urb1L39ajd2}V3B@QQJ?^YW^x=b2-tIFmv}iSR!INX9zXx48W@Vp!Kc>Bx z2PR1h!ghB6NYeF5z2-?>XLw{DYIlsb{UP{Q*Fj?g?2kq)=aN*vzl%q$-XL(`kC=lj zE3tgD{ntxgpmYbzEv#oFqHAHKj2=W0cCSL?gS+^9@1oC4pBSw_JqiM(JkoO5}7Zi%V;@+bb|>+hRYJNZ{a2a&h&dn*ZH>n=3T zu{G7lr_Q+Ku~-Gqc+|#*cA^|3(|qIN=hp4PKhdZPn=~NcB#hjHtj(iFV)=`Hye(;DstdS~-9-P6JUQ~vmO zhjfl^#n@XauZasTT~qV1v+? zISbcj9;e_I(}BI9n@v5y6w078=9W}FS0jTkW8dCh)IR3ntQdNWXIeKOiz4sjATv4{ z;jr|_vapOLrb=q2R8M8dZ1;Y5yZmMbQ@9?LT47L5>XB=z@OhKm$HhSoJw@5|g%>iV zVDHoDIdDhF%~SlhN)x{X68SJSV|T;+PDjYgU&|vaPz7AepBWYX9~b2R-RiKjnWKZV z>;Fr&6?5>gw{Ju-21sp-F}*%IGUn_>Fp?gqJIo12orX{zlHH#8R}#=dP;c=2X5nFb z5FhX^G81$doPAG?VJna;n9-g1zyVNzSs53(2`Z(j>CiegUQ)p>Jq~vlanWaDB?mDB z!RVschuunWf}=F^r5SCVV+29fEvN(cX!98z96){Bz1(#f7bh<{trw;*y-cdB!5pU! zwY4QG#fB{Ug(a?fn5aI=vN2GnKR%pXn3$_#vNl9p-=wc%Vs%&l0i2pfg>K0nLFcc@honVI($}PqcW>gSUO5K!7MAyYVf=0KtvmOL}#s^ zCT$sM%3Fvwt}2DIZ}(Eq*I_8kN4U~5*T7*YNYwJhou@j_Sz2Hd zO-V0xPK5$DnZH7qm?0TACazp;++gK*cUGoME*=54%y12A1n#8yLi8wWV$!}!%>h&G zVO(ildRpyiLv8c9l@on>rjo;-q}mLn%qxZZWmG|1scIANZtcB6r4?RP!|Bfu`RJWa zK7GAGhSlaHK`Vs~(Uc60@srbax$4ep0errA%D3u_K)q&f6OG7ouTpfAiVio^sj|}w z*M0{D;IL6-o&(8}qm`?Q-@@WfUkw>&EI-U}p31p^J z2KiY>ZKMGse)!|x@i9XahKrL9ER(}JOO+JHw2T6TS1L%ldo;!~L@hvFtJUFnml z%Q36BEiCKQckRI*GgOI#%mNoi8$(Nk86yh``D_#Vu9z1{M3-gzgG8C(euLdrT}T%z zabIA=bzpT?Zbd|04p2ZJe_7F+U%ROY4z6RmWvNw(CquKzJWZZ_$)v_bUbe_FNO@51 zc*o*n?5p|I54&}K!%DqPI><8t0!1BKRaJ{I(~|t{%kQ0T5%9`nJJ06vA8i*^i~WN* zf6d!}pw+le7Yl?kM^$-1$8%y!BVnQ(8aCi{^Q6*a}z7iGw9CbDTwR zF1IZPEk~Hrtl9cYym z341HktPn3-tEZJv^*wfPSN!g_zF-eD7=lN>nZ?!(VZod5KM}n57odu3^W<6<28lD$ z@7SUWmEF@lWCo7XJ!F4KAnE(j^BGPBG#SX*!n(4IYNQxnRr z(KNbxHzhUi^EQ>_4(9ftP zF(4yl}D#U>``uUi1RRRKV1x9o>jO@OiJSmoF@kvC}nOCiCcLKcJk58o`TR0 zrMPz4#@0i+^hSdfzCqW5N>hH*y)r#Cz`RwXtO(VJ(a9-8NzhJWa)7;b=@0dQ@v2Qi z=~5ld5xK;4u8n_$dk&0`<6%a)axX;mfcDCZM}l3YKQIvyU^qY%5jY*xS|L3kRao2~ zP?+rXr!5F)QZyQL;JKyl6oMRWWS-!1Z>W>^?5hn^5Y!lEoj zvHSmU_D(^vwcFP2SGH}}EZeS{W!tuG&a!RWwr$(Ctt_9p*4}@_A1C(OC*oXW#JJ1I zF?wdcz4fQH&PJ4r{vbjWO2wdYG}EVp#)0yxyl3q#KKux%iPNiahpMe`$NEveC-~GF z_A_}Q_pLGnu0Wm4L#pTn;=) zRqifaEcq!~OvO{a0LoLSNb8%wK+03JV9ZmpkPc{(T)b0CE;)2azPO`Jo*7|RsTduk zn(kvvp6Pc}&WNt9WD$eUrY3&8E0e^>RU41Itz4p*IEAr5Zuo~^?OZ#Zc4$@EGFfTk z4k1~!q1?G#zXw)KSu=Q5c_WIT%$dw9ux0NTl@3{B$;t*o-uhNI~1_*Ybqzv zm>8~KR!)WtUGL6-d?MGZk*nDpG&9BAkai8lTS7)$k5a@P;Gm$HIDEAd+#2Q)ukM$K zzWllQRI;}xpU98AnE5&4SdAYWln{_7vXZT-ez|)584T6}Ck#9uw}U!&_^r+`p`MXp z6t16Zig``#sZ`o;foF1OIt$1Q1hP`{+B>=v4PLzn?B~!jv$mwEqbw04sTWqVGB)Et zod@2AqK%Jg{M-t7xM~vYF88r7qehNn$o4}XZ$4Q=zd;CbiIECuG~j`A=wF;3+Nxg? zZ-E2fmgbn{y`4~M^>xVCNqIZxDA}%C(SHHS3+=B|EOW*@+God${(HIU zilet#x`Iu#jufNnUQEQoR?9#hR5mm8JhXUm@m11C)6QBN^)IH{H11k$e0+n>4`)NK z3@C*T8ccX{I`TCwj*Os2E#mzieZK?(A?vPVC?(GxeKL+suGOl`^^N(R?ihPpEgJOj z<=d^1;%dH%;e)FyPrnnS=M=R|8Pew}M-wM6F)ix+j=;dJWZbi5qsyipM625ARXFl2 zfNiFIL0~1bdbVZ~rQ=MO!zlDOv7%5#pvGP6VH{P`m z?Iy<@mtVcq#2w<^9WpLZH~YGrJjl6HfM=E-f8it%3ihMqvsI$zSlZP=p$AhjJ?4Si z=Y?^BQtbV{MI|7yTF>lwSvu1z+O54uRLYt}8FCcah!u-k34vsM?_RJuU5eSiJ=~N@ z3`kbrb|slOD{*q7!{quz4lH26usVbv)WIr+y2RKqWTYcHnD+12VX@?@iEPQ_j%kaf zqm=EkQ<%e6$N~rU9ixPvhs(k_-NXT_k(NQ_+=7e3aYs`J#szD5+NA2?i(~~D`29A_ltX28Y%&pha%QO~a= zh{Kds4LVFYkp0Af6+6{^1o$`bsI1ekbURu21*qys5QY(}&EwTVQ%Bjq^R2Cq2mY?b zUzljEoal=oQ)C-IsmtsQDE7Vd^0%%WHAi2XJSSU4d)qbm0^4;cSAha z9eHXaZ=E3hO5TC=|8Slnpl&focz_~eI}H`&litRbZZ}&H8hg6H^)8WCmK={|whF|K za-O#-PX@}q%O%#R>g*jPhoyl9M~X5{wPRhuOmNRtkc^k{KrrbcPvoIXUR1sn%UezXSUzNF(MQ_T zA{_txEKlT;d8ktRX|syr=rWF|xJQp7zkyUTaz)jveluBA(KiFA@2O&y1l;j^&EA1) zYeM0QdDy(*jp|P+EIr|khKZ(dGCM{~;P7vWre0Za14b9!A)R>WK8U8C+i*kY-2WgD zeI}Pf$d;t`#}t>1h*%3Sn-)gDRn7%X)TkZf690b6X*#40X#*!j5#>2>?yLi7F=;|2MDSO^=wr7B!&3AQcj5`WUSqjF zX(*)ajQiLPux(Avb^~DSsa5x~uUTiUTBZK@6W@o-nUHLi8c`?jqMV@j(JPw(&9@TsF8u^n5v;kI7|hP%NlPOEH2 zJrdXKF9BsS1p#=iO_NsdrqQ#uT32aT_Uo}e&=~A(SBZEqO6uE()IO-o(KQPO)+l=U zba)Opp&JRBhC$w!+_+6{M@WMkBieN^SmbxUEzcm&OZ=qY>LxQAj`kyfaawCE|#!Sz$qj>qrYX=AT0HisbT?m0Z1 zIk}8Ezc6QKv1SvmX7u#2e1oO#P!An)`EsI5BTwC;PAx_rVGli0?s-Ptzx&+3$J}T3 zYRPkGaSi($g=-Q=|VM7wBgxAnR!}vN$^u=2QlMFbzLD9W4grb$J3BZC*8C0 z0N~p0iS*Z6=0lKldo+48w05kocP?AKwZz=6_Peri0olqS+~eaTGedT?Mn1TvL%Ig$UYG3&1A7K^tWHNdf-ilY$*V{{C zAs}}knj8&EcYwk*;bJH-Z&-k_$-}fLa#!jMW7C_#l&z7q- z#SE9mMs(6TBoAi_e?rpXY#KE29jJlX9P7XVa+)jqD^3$RT69Kl#K2-x2|h4!Uwm48 zINv|JoL0=`%`~~QPqxi2osK^g=zga8H07pu{&va;y-6G%UR(`}`leiP+LTpQv{<55 zLDUAEPH&W*Zbz$%^w90=Wf-UjM>u`X&_-+`^>H=FLt#dgWa<|#9w7(AKmYc^4SF4Z zhX3V36p8=;NSXiL@%>M7Ty~<>zmh6Lb}g(@TVQNxsdfZJCT$~WL}<*Tq53gpGz}gP|9Ev1w^nrz z_l(EwcZP`Zi%G>Ll59s3o9efWCV}(R+9B&;*5n%S+^iO7LA;RM*vR|C7KTEbmSwjI zfPZ0{aQ*G{|;rp{?}-<|0JyczpC;79;7ELX(=trBYw;JlfzM_I`(P`6v2th zgU+jB1~CmvL*P@*>}iS1Sux01X5{gGP=8{CNtzjp?tUr6co|N93F`HHxz66?@OruW zeSLp}^Z~CJZ^hSB9F|8_MX5kZA*Yb>q|^};p#%m90{sRK0gIvxGZgR-@&#iOMH>Q* zG!X7<6QTp5GG!pr*9;E%pgI9N1IE{v9R4j`;U2(=HXCQG&kWmL?)*K+!Q{MYKjGQ` zN>iF;U@lRt$MjT8mwk$o^uq{lc_Y&)_g{m4U++$J`@V(a48-C(N>_sF_hU5LfWR--@z2q9QvT7|YS%>zEJve1e-u4NK#gj?v?N9X)KyG;5>oYU2^ z()!pg6Ef3ve3zZ)*w>9Q;x7XfzWL7Zp=ToM>WY;#urnfBBfqvW2{R1GlW3=L5K^X4 z?N}-v)C&jODOzMa2c?meaH{&fGr{1YFxF!8*kDlWb(}YV$N#!>q_XRX7gBS{?=-w5 zz3lfbhx;IdjsrBVi5Z`jKaZjW z(?m=h61x_MIVFDqS01eBK6e*DB$6Sj8B|oTJYPKR`+L-E;Y<#adrpW$8i|DEt1xLF z>4xoqFCjUeCQINe0{$9xJC`ut(K`TO@;eLmIu-7E)GX{Km#9TgjCkHP0-x(bf+0ZZ zX=VQHvFp$ModnlG)6{m8`l0#|Y06lGv~{p`lVNngh>o-1v2kFuE>Lw!)ni3`~PMd%y==r}zwNMk!6ZgMG?eM=A zHUEF04gR~RWlWq6j18O({!iG*|MZjVqo6|%@~?1UIn`##Wp*fftpW2cL^7mMy4`y^sBj7mt=7Y4mHll8iZ&4qSTkP~hD z#g2m*&N$A-wR=Km!!_peL-Rhwh??5K{a|^T4xDB|QO3k>>0`bt+3%97jEAPhG6^Rl zOMIep-Cxwd7BFh=hGTc)du%*4+fEybqkEC&u&N?%FvY-`VWX-}FBbTjD{3+ZsIA1@ z_0&=<)ruI5w@oPEJ;R_(-u7x4uRfU{dzId^J9g!3c;`+b`}su`iq0^>u=7RAgSnw& zJhSYf6D>nzCy7ABt7^Ilzs0#HQYz32B8PMf%6Y2nzJZ?w*n8fC&c5h*kpjR9W(=C- zJa4e(b8TPyXY`*vUbpS1Szaq2Cl5ot1bpE7MBmBLUX2bMIChnQoQMwlkRe+Q!oi%csq<$+5r_f}-CkBhP1Cd~UuLk$XFjRmP;>|s`PZX#Dbx+ej zPCz>EcOEmup-2@XyE%aB1=?G6aCN?TU(j1<@b#V(j=wLJg(rv`l&^UIpPu6V(6!S2 zJaOECR#s=2Ysj0$Ig&eun|XUwg4H`nzN&o%Z^6OXdkeVUJ~39@(Q9nqa4lAz5FA#X z@J*IaXg4c&NmswbmEMTCn=ie?AN(B$rt7j6J9@0~pQr4yaqhTpHFshi_6_hm?y zBp!|goJi~eI~n#2;;tmP2P;EK)g5j4&%=WXwx~tcOE`@4xw%5t3&AaEP4-RGmyv+a zO_$pF`Sa({!j4v6TyanrJAv4@LeU_}?&bNm`HM%rpg5#EwzS4*&rqboVi*78lFLk) zt%|;D%Jr0)Wy$=~McieQeeaZosO895qp7Om4i;j zm`tfBSy zBgTF2B$u+K(f5^dzX^_s3ToyOdq%{zjMPWHu3YGllYf`FE>2~ul8mL?*DyjT)eSQz zAo!T9qP`BJ)E{e!X;isIxr0){ZdMrE?=ToljW63y*rq6SS;-f>6Z$!yk6IL=+ z$qkW&yL-3MeCa)AD5z8sS_LC99Z-8*E;EE{*Ou2**6LIX?3+>4?m(VLhrginc8sqg zAGr#4NO^3{@f^VsvjT{Vl5~W2@3NAJ^#~_W+cE~i%k<2!<`eEMdJ&{VQ*%5pK5WLG zz<^$xzM8B!9LJ%tg#p5l6y`Y~$~C4(u6afF&;qM;bckrd-513$ShHLY3}eY3SW zeRupMG#)Jc`Km)^Onmzq+S-=IZb1?_t7BJ;ujV0W*|Dy*Wo2J*FUtjEAO2hb;V^AF zWABkKgu=};plt5sjT+Jr-<&{h8h?7&Oh)-ZMWB!J7bgZR|@q%JwvKL|*!BbAR2{ zC1k3Mq2qWcSMDn zsU1W)1%Kpfp>WKKFY4iU+JpssDOspI4?{K^11%LTcThTlf?_{=*kV4i)qWo^Uphm7lcdt8dKhc33gM=?6YUYeY$1%G>ML@ zqf9-17kgml&DWQs2Cn~e(i@a*qcIgF(b75bVqi0L0*db~L4O+lQY3t+ju?u|-MZCz zTMid4ZmMyOICQNN<_0w?%iNO+^o^*(JfHMIfhbZWb|h7Yg2!Y;%?`5&!jn~w&~_jC zMMp-zWe9}U{xBQO2rJJuO^;!=>U3ikZmEeX*H%n}(QCiXi7O2qZS9$0Acz;9q}pw} zzdl-8Ni&1FuW1XIQoa&i?sG~9Z_0hN7-LL&Wf{Rp@Hxmy>inl09uP?v6Npxar(CMT zJ3iFf5x%liU&G7h7IXvMra}~=B+>|za2+oDTukY=8h46r1|HPLydbyhe9U%JVWR6* zKHjK9r->K@wH!w+E$<&nxaSnsXcNj6Wlm_k6y8RY6qd5B`RHMlnFyg%!a`Y9Wpz=R zx~PL2VlTmMNu0lCFq1*>7{#%iCCsH%4KwtlXT0_3V4dz$hz8zyn&{tYQ#rZegK2{~ zIdbRTpLn^gO|uOfq*7c>Ydfm33WJA7KS;I`57fz1kxw^R=Qf(k-k8OMCa3e?S1jyI zoGLL>lBJs(XZp$5!SJn44(vPI+R4k&oUeuo>k4g{TkT-!+i7;!beAJtz#=Qg8M|IVOfjWBJE(5jv7ha^UIwL)`ZlR z3^4$fqu$1oJ**v#Swkq0%{SBVw>_k;ose=)8gAuriTU~^7j*ADo!MP^-1p|1$*oVmfrkc%JO-9i%mWoz=-79ZFwz<`I zM@{sYh3K$WdZs6_K7-QVU7tF2zPaXFqIz5?U3;h0Xril6l|)4s<3UtL%trSF%-*f@}|LKP0?El3a z{I}!PMwS-=NC7eH^pZokFhB0hn-ss>JWP@z?0b6~MsB8>CkvoK6+VD)S2%>6ye31= z-I%!T!DHw4j}9u{5gw{GWD)cjA#I#BODb@BPxzEU-$NWQtvvobCA0)s<95G%)2+2~ z6{sW%%K5p&rA4Q_?wF{Xtk~scy+KRc#=Pp?rCr#mC)%U(Yhl|L=gT^VQP74THk-W+9F{l64zk}jQVDX&0|9tXJI?a-OOubcMMoa>!e#oW&?e~^C|6ceUD)tPe*PMD{=)~3wT->jMG86OX)ykpW+Gv4k^ z`BtYUr@FSLsHePh(r4KpQKr1J)A@}~W;5QLOuxI*e+H-Y7#=aEKhc2dvT4O$`_$X&T2^BXv#dcW7o9BH>Ky=V{o|hto|R(lxGFB6Upf|7cvXM?y>= z7^VABKGqsLwA0kY4JS9WNusMw9$;%|Q%7DJ*Xy8j8#{E2ZkjlR(%>fy18LwU4G%WB3!}$Q9N0AgI%)6|MKWpN#}6Dd zc!(l@ncOGQXng)gryn~I(dZ(LOf|WWpwUepwrT*B(&(lR<21M{ zq1%lg*l2W-MEZ>%=xB8PihN}@V6-Bx<3p)f=K{t+Zp-RZ0j(jg%kImC{7mn&gLGE| z=7oGB)+e)v^#=xa1r`AIX(ip9M~YY)eb+gNKnWxj?0oG3X`PD};<9X9(#7 zg^?yZD!<|Z)UkH?kXzZ;#yf7{p}%(@zkY8@Mf#!wC!&<`x- zZU;kS1O$^cb<@;Gd8G*4mZs!#N9->L^@#z*cw>61t`EkJG}e$uiUVa$T$kRb0osDt z7StCF`Yp6q3wlFXm)OS(dP9OpTo>E-8#L$F9h?6vl()bh46q(DAib{^k^@m66cmWr zUm6%4SO@Y58BabYOJ+|KconHFx$heEhUg9wm~S1p`HC2r zUPGS@PHWtjWKMeT78IXkPIM2^e-x@qc8>&@fMhT0Pr3jd5CqUKe-!P&z_;cK>Zqd_wy_ebk^kBzK(tKZJzs?(RhX66hS!HY9iKz_rNk(S6*Y zJ)(VH$e4!yTrXK*T~~C#2q^9$eVL%&B6}z>EO*{8plht4HNWgJfpSIlJHP_26ku2y za2oW%HwE-b{3D^X{r^mYq5=m(!jdy2^sUjPP5TP=eIR0{ffZlD0;?gmVF3}PB0EwN z5zN{O_BBGSLBbL;u)>@ybU9h%bE3uUkL)sA-bpc zX@T~T-RYn!*MEV+BLmFPX-wph0AlDgX53I$#Pu zA~0v@6@s!qQ1|z=(DsLtN!rAbg1Z!uNC4XrnB#JaL*~9Q#3OgCpmb_O>b{-CBS;VK zzUjo|f2#&vM7pR&qDvKtu15$d@l94ldf!iVNZB`nxCCkb+&hyv=J=Ju5ZpiglS>5k zaX2DPq`NVEe7!5=A4N(cMm(`@r?9|7k3ixR#pbBWh%&+?59RDxUQl_`Uw57&G=(~*= zTS9OJQGJqLd5Eh=;BQ#p-%z0wnFx-^mrv{OFyfi~9Ve(;9^Z1}7R7UK;7tI&h_*m6 z4Ae0(m5%ftGH~icaYS39dS|4a+R%0p=d1i^H>m4lU_@K2o$}C^a(C{J4w4t37qvc> zotjwqVr_t5VTi9s4XN5Yo|upDj>tb2fD-`-TDsc(v=;G>C?V$~5&4NqdUj$Vx(7p` zJdm>dh~O^{t9|o|0EFz8huKM#_a1hl-t+ns0DMJ0#fx(RqY;2~<2x9e+1jRppSZs7 z$z0_jyH9eFPgy43C;?y=hw%GeM0fdk4*opx0KYPEfXKCIC$71Nh68%?6A(eF4Q__vZYjGw6%)Md|amCG6Xu%uf1EY=hC?I3Ke z)f+l$rSCR9n${C@p(qifk}&>J9;$J1`y&8zsi&$!*bsW?iQJF@V9}^^=crx~c%nCHSK3KkXEMphM1LNb+!C z1Unr`UfQ1O%)Hb*yQnl35ox*p9EujP+vc1wnD+smz?X*XlpIy;%B+O{{z)(6`?(lp z9?-5_bGGfLw(*(NtjV2NS)0Ugp ze@DrN%(&Krr!tMvWA@Ob!(_`|`;8rpmKMb2i4AKK!oaTNa>!F+?ql|J<>?3c034yc zz=~0c9`565KuMd#(B}&Cns~XYy`%-Dz3!=6dyx;(9BXA`Tn>C)iaTm3s2V>5NSETO z^T#(5@6UOF3uXXQBO4n_kNdtd(hE7{5Drz?j*?VdW}0D0su0DZvEiY?Vtor>wwT}t zspP}_5dtN_ea;qSXD3r!ZqFB7a(~d}9h!tqm54==ShX3n4SbmbB9)rx$q;gic+2l+ zWMk5E?go%R&!;4EXX2x{upY-#IVpLkieThg#4Hq~l5yZ=B^U}enoP!}snKb}EQ^T9 zH`8ZsV4y7SfBwjNfydiF&!fmQ99f4FrM(pCu+KsKHlV=_9G&oMjRxVB)Yca?~|&m?Myh6a~Nc}O)ZCGl|ZxqbJk zdhM#cM6xq@qBLO?ALASBD9M(_KN2z(6++j>PhLu}<5Q}gF+~#yyky0lgsOX2ltQrt z6vAJ`yOG-KSqsc{fD#d|ozD@O#9-p`^cmt<$LPu`Ex&MiL)CUa_kekYdt5OPm2 zFDy8>Q$MS&vaXcD)Ok1_gcNUVC@qlf+>{Vj%l}=XI{tGGTdp~fsvv>trW*=7EY|x3 ze?J+MbAYs6iQcSx!3~r4Fi?t>*v(gP+YL8Gpx5XhFk!(EMcF_ zLQ43W!8nTc?5RDuC1m|RH_?AiSLM}p-QND$QY*1`p5TWnQ}Aagd&9~VJYuq>rtmQC z+moOY->%6rM^WeW-ZNkk(^#4jdYyUF6m^!Cx>{RZ(*gHDe3(KJ{EduxV#3sLmU8%! zN>cW38?ryJ4nq}JK=gz}`VY9yB#&-$LA`JAg`RjTDhI>x<}-ddu-mF@ZAUep_Bz4G z=q|Gy&wJ_+3AZ03Bk4AKK{nJfEHpPt()qBHnQp7=iOFSJq31H!%h?U+J4$Mc%FUnZ z{@dR!m3FUW;XHnV=Lc9YW`!ZMdMu4qO=J;X#V7?jXyWP5g?dJ<1C#mc`Lti?m9Ln8jOD2r-gY(X7E)t$I-6kg_#OnI8I8nG zn;99gNS`lZ0K5f#aF9>gjA{jG2e$E=lMwnh*B(D-C9m^?B@>d zGHp@C+-1LTNH#ZA+SJ-gdzOz{Eq#2o?2|36>><=F(Ac3}(0o#lRCG301?tmarI%eQ zHm!KRVBalf*SeaPB7y1Posu=KEA8w*mV@~Y&?Ykqp2FQ~<8Mo!YIEOPZf(r96;8`n zI#Y+;#8^O7`T)kzo6gnn)KFAKr7_wR3Xo+-1V(kDYrthEa2+_c7d$Q%#Uffn8~(vP z$Y{($TgFx3+xQ!|w#(7Jy%%tFPfL}@Ib2&>u;@>mke2o`aH`U*7ZIXN{n*Vb+v=g* zxmt%xAR$>KAGYO;d^idc261j~cyOv;z67=@8muo+NJbHuEqgWGFY4usLpg z{}PxfMFsn&CR@)_;AOmJP7%AsD~QT<<>;cRvwXvgec?q+R9&J#+$zx-ug0e6I(;SM z7++7aB9uB_#vy?T|xnks9)%}`ZB~H;Xm~o0WGe(3hZjf^$_qQs;g?d{zh~Qa%*DkBuvU$<$u3C`Aw&W_)9NXZ5{77$ z%PfBb$oPIEH%gRGF8{e9Z%#==I31wP7`GH!95!q^bBqw2Uy@XVFC?2nZ8-_mBH@eC zX2*tXPCTRa1WNsV5`v})r9kflF*fXF$(tdf7>NM)3kK%c{QdogLUj5LfL>jJhaD~H z9>2jb=kXpwe$M3v+(9qnNTJ#5C5qS>jutJr%-}G6@*F+QRPjE+ z)Qs)JR$A%uFv~FL)8Qwld|_QJH=87m>SK6ueVmE1Xhor@E3FhoZX%NTwXf-ZYi{S< zePNnZFz(+kI1#j;2JgulSXfRKeOBazQ8Vn$izgvR>f%d5h*ewyuClr(^39b!3_w_P z&KHpbF^2TjGXi<{S8@L%RI~9W^_NJZ-cf-crEv~vr6k|iJbPt5%zjv6lXw(Exu*r(f!JMQz(QR-6l#Hs8&W#=`Z=HcEfCJ<;#o% z&yLX-%oyxgx}}S1IGo2R_38A}>ff~EPoSp3%)h`U87+`q&aaRI{nwx21bqa8y4F!B zU`~~D{@50vBWxsZ%2}WgVeY0XePocUb+hqiXiSAhXyR899CrTV9A?JEw4ogqXkE}@ zAM96+7N{cI}el*0Pyg zh}a@ll!CrqOiw5kkV&h&&{mn$)*)JIHE^o#;56bBlTkF<79+| zP@qEgHy3$u4_W(L7Mji(psq2ja0W)djmE_{Y+-tWTT=wg8z)D)MEKs%bax|Bg z8xonV99{_*>^rtlg|&Po@B6jj6_MtHRk@FApG=NeDxS^=1aY|Usz4`9p+Ez7LYBbi zvQQD_P0T+~p~-sC?kgmLB?7md5*Xu*O%Q=6sg6sCsA1SYv zJGrJ)u}`?`^wHkAoLy|W;NEiH>XAp=FNxl{l?$Ti9pb$wGps-NYV|Lq;sAm^PcIm4 z>s^Fl67$#*77Hv+jh+J}a@VJ)&PKs3BTS>Ql zOm|%}tzjJo+vaN@_^%aHXmvxk8~?|b0S{L(n64fiGqX>Zpv63DA^HcU@M4oVh+qC_ zvi?SWaDxg3MuNfTd;ab65Rv^5r9vPnvY~{4d?;Y?W0UW)`kK)Ah(5EypZSig>MK+q zotbX?mhBu%)@X|wQ@9#a+Lo%oMSEI4_1F&=HoqZt88F3t0IsFmRWS2a4B}-Y#*CP+ zDzV+&w(eSxN%BPQRd5Lf0&icCJ;{Jf9Guy~XQlyU7TETg9m-h~vu`lX7zBe(13}5b zMa?)KQo!XE5#S$!b{#~HJ3YvjTch9g&)Il$Fs%oI#I|i1gM85-k(m~#>X!r zOQz=##wa@*@IG)I!sKelEPJ0x21S$lQ5zH}=O@{*6_f)z+k;LG$a3 zg|*@RG)HBBeOp(5*MDRe@&VSiCM-FW+owYHb=@rr%`Mi@0=|ga*jmlfb{J51k1As7 zHTM*i@#>kQOa@FV(-9qs0jt=Kjv;#lHB&xyMghYaLz8;F04%O{h^!50$Tg}2197YD?&wV{I^+vFKr4U@QK*kk`09CCR@Z8E%r;kHQSSu zC6?@csVhw=mN?sgKWS7Exk^LQZNeW8RwBP^+MoTSsi|62p#%q0m4`Lr>^Nqh4WwcO z$6lJ#(#q2kPl0)TI53p0uzvDFj7Jf%+o=5)(1N5M7y;p4OKlpm2f0)^p0G8Nx(+EC zGYuZpFi(DvS&2e-w}D8D3tBOpc0F$I=ycTC*5<6KS>aKn8EZMw=rn|aP@GH=pG4nc zveI3~Un^S{vX+O$---GbvD(76@dm|dU0D0eJ>@5(IT0eS;Fg2&4JgJmXR@wt;9(7X z+YkX(kq356uQ!0@G@OKJb}JYgZRnf#Hz2J`9Eet8Cn!dfn>xwbe>Z)nT}NM`{VKl3 zPFV;foimCm341@-iMfLPwzn{&>FCCDU1HPL*`V%+ufra!KC*E%@d+F(+emnCR$8O6 z!iZL4<_*Q!lq~o)i(k$gK%6?qIDEzRp0(Ab$`+y>vVB7X+b>0Xx5Ec;g5fFX{7O^(R|>nLZF zufl?_Y-bo^dOXh7Hys23EvlEDLVy*P?j=O#EEq3hdRl%R>IbaE%2gBOYVz|vWA=Ie zh8p=}!+=QlhtZyBq*zKHTpQYHk0}iybuio>Y-EmlAG;0uM#TM}n&8nrE_blwZ}6KL zzkNAt$gRGRe!ca3dlYUUThqvWbXVxDk(E9?YwE3W+fc9d?z+fs5U({m!~>H26_XeEcn?eXedUJ#9admmTt1LyNo8nPZC`q zdy&^3?N*Rm(brwh)89cuH#C8G?V#8~=w1C2oVOuU|7nTQ9?T*lV1Wg@M=1O=tnzz? z1*nHzCYV5aasSt%G$7Uj=+m?!U{@yEeyu5maZz+c#*EpZBQBi*K4#yK$?!@iE?;DF z*>!vGb+D*HJE{x=&$+-gZbq7XmgP^tp6eJ1(RNo8f_B))lrYrU(2o+J9x!hTl`>h(KD{I2FP8&s`*g>+KP>ec8~XUw*F zSkGDWmh~C)@|D^eZ8&sbm1$DmVF)lC!lA5{U>i*Z+kag~vz88PFpRzv2K+Y{0Z4u7 zfj5@F@9e0XgZ7*}72DVoV&Xscb`ZCWrr}#VA1buB=a*e~5uEI&Tbue4SFU{Se5tj9 zWpObqn$NUt$MvUsca2$_8#G`GHZOzu)U$#&xp7+E;Gay289=KoNHZfhwZKQ~9VyHn z4Hzpy2{c!Wt)aC0Q~IN=IW;%N&2?;>s?hAimix4%gAp|Fyk{=WhzW1+f+=PvWpI2CWi3wHMyTnnitw7}_E%l)jHWvxX4Bxn(5>6qf@zzO{GXHH#ogpLqu4a zEqsasckNh;9-5mN4XOmPT9_HPTHV9`b0xVnRGCNJiS1c$rH|N&F4}mh54;IKYz<*~ z@I2RN`zQKJ)IDo^O1?BbeS7A;kvlzbdw8Ti`%1VaQfK_eOs&ys?U`oC)|hs!F_yPa zQv=5fk>*M55*9AY+FXD3Lm#i^@us2yajO}R36CO^+iktDB27w6;e{cjM=Y*gj41aD z4{Dl|i6|VhURX$uF#lKloRat%)|seZ_Y~m~6XY{FC8u+)v`9iJUC$X-1xz-{I!xTK zE-1AZ)b$bdcaq>K{8eF^L&xEtCyX+uu=#cmqBckFw)pPW)d!HIkK!N0*197$l*l)+2fCWj-|fLN~)_P z;cTLmMuhK>V<(osjzdwUv%I%o<{FsEhfHXiGW$T&p}UmAwxd`)F{>U?_U^D_ z>YwQBhum9XxAC0>`b-~4l52AE6TaXuU;AQ#jY3974R|(jChV@{4)otJ$fj7(+)gqkQf)xcx9DOUH9x%E9 zxW3m%fV68Z3Qzc}_Frre3hN6R7bBe|K9*|7i!BDsHWEWtG9t5sG^ zC`PKL_Lb`&*s|QIc()7o+l4upy4y(vw*;%Vx@u|Nu(Y(KGIuqj#R%awzTJh!4B7C& zwK}jmpStYrJ(nrzrcLAYIJe}YuP-NO? z&IYxA(rjHg=`-#IX!+3PM9c&dZhVvb@>vg_7JmwDwfTCt)<49;Vz^~4?lUQDgyO$a zKUIY0ne$6W1?|MYf2}mP>loc&(d^rM?e_~gdT8$4QlTolLP5qh2alR6*3m6xH7PGn zl*FAy$?LUIGfh#Qx0AwKY{Qof=%lGj1*09CIAtQ_qWU5BF-x%hFdrFqWrWn!C+&j5 z0H)hkHuaf%@m9Scv`24)@VtN7ze{DsqeP){_Oo!_<)V;Gt$wxqt~mkK+@oYU{4`CI z%($GJ+0|q^n?G8JwdKO2m0r2hzWg=Qyrl(=&|>tMp60gb#LMUQi^2CFEI-Wd6B7Ny z*nWx~@WVHvrHfGmK0oA?2QvdOKX}=}K*G}ANGk?iOZP7#R4A=I&+Un)rV%6~Q&iah zhqZT#uEg86y{m$XZQHhO+qP}nwr$(CZQDs@#i*q6<$v}*?Y8gX?0X*W(^{>q*D>cB zW6a+B4;w-uw=X>O5fO87iaxwAaMKS{wmhk4)_$60w4geBo1cm*=5EMn(AOQZHuJu^ z&6QuT1~3YH{(D(c&WZzNX2<(gqRa@-NmV5U%e8&Je#jx>;K=Iu5(^q@&v5+}mh}GC z0QL)^<&}&*;44}icBrXgv;;l2s(wG5vXjoIwXz##dKQ?A*+{Ehs%`5h^3BJq>blst`J(y&Yd!GOTUq;m;uyBPeu(K7i-!C`fiC2 zWW2#lc<~R6gnkmEMp`UjdeD52#u$%QBXfF;62Y5c2{8k$0#y6E{*B2>c5_3TgZF|9 z0M`tl#6jkE68`AKJ0leNipba}cnX`ttsmbi)Zvo6jB6CNs1Rq6o{#rq6cytjXgGz$ zc4ip8@<$^bR+Od?iH(BP9l&a49V||*gRv#+;)b4^`sMGYK|uj=IWIP2 z{q=!Sy{q0%R;aOPZ3xr_y>{Wnf=37^66v9+(V;t~g*3IZiTN+-1*EwRqQt~j#EWQf z3(RVUZ%umP5zI<W}J~+xO2nHc=CnOUoH0n7~qkiJ~FD7B!X%wD()yH7Z1*p~V z-Fk_~04Nc?c9qAVokMhY`Nyyz!ynzGzlOXF!9NIpG0|pIOCdLUGX%n*id2h78<)Ih zhuK68v$rZoo7plX)F3Bqcrt|KP?a`DjWD~`q|g3F8@VQ6$m*ezTrV?({Uey%G*lR9 z7U99IH+|MK4B;KA-XAq0s7rW0vNHFfI@HdlzbiG`Gml}nKw=QZyFkM6Jj6)TMB0)) z=Z_2ZIL7^{P{AjwKMzE}gJ8kCod^IDQ5J|*k4)73^)MnKPa!KR>wBnno=wl_&aCfF zB2xQePrN3?C-ofBy83IzfF~MrO$FhIbAcp@Cf?Jc`g6o2mRe*rjqpk;;i+W&T@hf8 z>uAC$G1ts&*SV6dh2Eo?x)J4QBszN= z6KifXu;){^OuwM%0~?+dAy-<|2-nyOS+ubW_}0heFIZMQp0^TZ?U`ir9pw;D=cI^t zqszY)7QZSjyGL>oh+$#v(7Gc5C*|#9*RvpRy78X_2?g0bS?gpPTW2DyynZN5wPy-*kkd&I^ZKvT z_7XxIG$ASAi7YF)XPU=(iV|946{a2;6!Bc93NMo(xIolJXO}$s%r5EjSuXA%8UhR1 z4}CjS#Y8Q^obRY5UpU55zW5M;a*+^jpZsbw zasl#j0kU;8gA+YBQ1KJ0wBYWE*hJ|=>y+|?^9+9(Am#NpI20mB#51S51SeV0c4 z99J2}zx?%7^99K`vStnrJ%{t2?s!zr?tkBIHaXJlo7{iQ@#mo9EbGP?O$)@7IB!f{ z%#O1#JdYmt3tDHGZeHmFQkD7zt&f0f=#4}|x)GwnFQZ-sv1g)QWbs0Mrf}=#o0B<= zJG@3Z_LmT$3(L^$OigS%;%|?5>L_Fn?;q-cO%g26KYM`jLOAs}EJ?5i#I96EsWM)W zvO@do3c9pP9a1wsD%?$0HA_UVu?ueem3fci#f*4Ush~S)E|5-fp#f4D;4!Cp-Rdd9tWr-z`BSU(kgQTzWCBCgUQQ~XtEu%PeR70XqY+q;@L@g-yEzf;JCDRAVqlt(J-&i%76MmZo{7F31M$Mt9k{Ypo+7(^(-Ryc zeJc+`<#NX5mQK@yo>T%?Jn^iw4Ky>Z^ zTKFx4%ohUW3Allb7eA;I>0459f2w#Qvw!9#LhIJC{hmq9r_OYMB2N{uyN|qi=h=Rtl4cKV~zHb3ykiaIa*m_dig%i!df`Mdo14lV5Q0#SJUrhIK~%mp-q_j+aL#b{eI>3 zzD@>GmmE>ygX4^3XTwZ%N;H&N;+Mx~T9F-x^7zBY5=Dq4~Vv{VJ4D zPpTvrFVlm+w2bvD{oQ|O8BH|feE?m_58bYDp+}RL2s6JS+-mZs7tVDT?#gwH3393a znS+<(>+D}n{0lr{`ZTH^KeQbGliHW@f2Z~pwEku^e5Z^$|0^4_M@h#HQ30Mu>P=n0 zAjFs4a;qjXMqKy`hU@z1^ zgHD(pC4k%C?GO614Y~R-R>T;DWdcIB7DI=DZ zBg!lF(lSZ#3YXuBFqNr7qy*AC=kn2O7e2IUbUElaLTxZ*1!~-Dj+mejRmi8dLTs6# zZvhvoXUc3HQ_g8E?K=kV53?2-Yv>GRj&E#;t4>CgbX8>2kC12~Q8%7B9iVtV>5qu- zXWJ=}Z+pd2ef(?)yKUH0T*Xj)go(j#1U>G%xiSvYgU%C<>K&gIliO>eE2O5LibRcS z)A@>3&t{A}Ry>0)Zu5|XMCVTINN-X@w}*kTwNGFG{PUfR?|bFk@&DsEi} zDKrVx8E%#?j~Z4x6rTrAdw**LPZUi~PAq1M;%0nw$zUu7fVYKRMj4YR$?hapu&=5E3G<*USUDpRYQFz`yTLGBEWlkP6oRX0 zW(J@ffPwiczA6Fp(SG#z%f(BCL3pM=nNcI!K6)kmtB|?ljxH^}9k6=eL4yCYZ2f;u zdi|HA|KGB8Ap>h`6GtcFe|`R|noCr*{y|y&m9<@x4Q;&ujRq$~mM$wPNY<5>DoE98 zrr9Kfugx(nz369FG9^vP_upP7)AL5ab*9k+1c2#8p%O`<_5$|tzNwmc%RZzQi^T~l zlS0oNW_rCo&nL#ud0!vT(EYgHH2Z-+r2J|65#kZ*@%NQ}s)2gb?sxiQ0og*n3ij7% zE)z^jppkiTL^v?^PxvC>j%PbNEz%8!$s+j9LgniS)4U6dYg0N@1P2jj+`tO<^UCfX z8PkVjI@>c2PXyo#Ump~+h2!smE)X4n#5y`^4~lpb8$cc*B1?;>k9t5zU?=4tHiN)G zKrp?^3x2Oh+k+V(e#MHX3uzB>hmsgG5FTL8ioZgC=h%#v2XkSc;HLRTNI>Wv`24af zrkS0`w!rEsrY2Vqk|--oiY+Nq0){RyaUEUwe!D(}w+nYKO^Kv@kaSdyx`f&=UuNpww(z~s$1|^EL8EAWD%2^BB z&XDVTff1T2zO`Xl@H?`4A2+YJm?S5lb%h#MUL?W)PZeMd93!r}cR|yM*|5~?QF&Am zJMud#L8ZPDWEu%EwwnBRxCOd17VNzuhRC~;anF<2%3_afQO<$Eb#(UU9%g$%%w_A5 zX-%ds3a*g8yxi3+e30F+GD~RbtDtMPD-k5gL?c%RgwSVs?&>I$(^963Us@alSItl8usEY3aT6hDY_J?Evy!*Szp!#BbYPx8t8w?3F{>=gSb|NB zNb()9Qk-z0ft*AI`_qN;{xXUS)i*Cf*b zY2u-T{6Y2^MqAOBs8*Y%DhpysbeRQKHz${9D%6^Q$bl9r}VxMd#y zy+7rj9G8ej$LU;pgQI_tNe7(g?I7%S!uDRq|CEPsh==@~T|oEDCF~WiVG(&{CPfAO zfn>0OY_NOq&nm@aot~uKOszna&+&qZsmvY}m6QyhiwBLj330c)H`SsN$|SnkFnrvC z+9b3Hb97HDQ2c_tD5H3*YFbgR$Rtp~jEzDu9D!Wn>o>Jc5DGr{dHJyG%AoxH{+UdY zKgF&*VphI*=AzgWRWxFMj{!Zd>#0@(i?k!=9hJLpv)HHDeI$8)I;piylJK=!Yj zTxJ%L)#UpYpZ$+FK5sAzEr|Y)TeItZG1OT3T5x3rdhlV+G1v0?l7$K$|VKtW=t5 zn_F(=JZHV=GNjQB!at9EUUr@4I9{@y=DwXyV|!ize1DABHY5myKO)>DhZYEP;q0V` z?cru3++>Gwgt=&U-+*b6ZsNl_LS585HiWuxca>o)LfLS4onb86V?&rBO|;4S*aOPY zxQ+-{k|DG0dV>}$*XH&EdIO#z6zDwHp$TR*pcp%!R7w<7QyTVx@pB9N>d@{tv2 z-%!}l80ai?=TLN^Z5h;84Aypq*eP^P4Bkn2WAJY{v@m+D!D}&A5&RSXFly)(I`2sU z)(*5542+L0NFyMw=Tz1A@#UFW7JY7sS_K5901O z1-Bz*2@^C~RaUD(;6~MlfMxqYYo%4-VeAx+tZkTZz#c3?3@-ktL4UkW7%n87r##G} zlG~<|p;Tl!9k(E&usH%XW<^i<8|rbO)TQiKb_(VC^7`Q^{9v;Pd>4NTWv<{yye@Lz zTcVkU3TtlQH@7H2NFm>-TECcI46&B2!Hy1C5i=ZcUBR=0hX0dYy^+=OTR(ZwAY3I! zWtw?LKR1QMX%q6SCr(~C>axIgW^H|EsThp;u$mXjtZ9Q*#pu$Th-nqQU6xd> zu^{$BgSuC^1Uo{QEQ`gCPw6`wW~AMm>K$O4M~7O7(>J~;*sYu~ z$OR+SVAYFC%=34)m5(2M*=j>Y)vWpD`~%sRk+F>{W*B=y&FSQHW*AeWR#cN=izKno zU-SlwI&HS*M6@eUaiyV9f-9zMSPXL@RFzyK%2GXQZPZxM0egDagdbaojT-(v1->Fv zjNd#CUFkCu5aep`(3aBIQdXKqO&cf#KM7Gh$k60{$(AT`rN-@FZc(^#uPveYRBbi3 zz0uYgOpQbqp_eQ>&%cUx>v10{@6^k1A#7KGcGG{upj%w64DISuSB7k&(Cx5W;^XJ1 z&o47}*zUgBG@Uu>LuQnRZ3W>tc&f?UCe#cY)kp>?cd6(7sF;Bs!el`v>mcl#XlO6f zSTm{)0Qt0Z-$3;vpN-7%<-t}8GFUvoIM*$FipTh=6Q%hM*1;o=E*ny)N z?HF0_J6{@bSa?C`RH&h(^y`elh8_bXy0C#GcDW)xY`6 zqXYA%IFKV|b#P0|`~504Tc)bHxq(2L{+hA4c%%FYU8%Jm0av^g}qH-f$M zEniE75ViXz?yu(mV7W6fJUjxp5n9k3}G!I6U3AP zcCe!+GM(RMg*RGCpY;i`d?R4uv%ckdNPtR)92GI0nVpXzV8W-IT{yU5MS^;+0MA;k z>W~{~qfy!c)2Wdu0`APP<~msF2)eKNDeIK7(+p|%bn4qqzOSSPuuPSWT$;qBl6+!OjpweIDAlxb z6s%#9$PygTZ3k9tv6eCviA5ONRRXJ=2uMkdkQtJ$4$J~juu~xGatB$kheA;hqaA5L zo;o3ipEcmp?NfG(m-0p0q+TFZpp?5j;F2tmUKG1}v2>^> zNwRGjSl39BZG;);@1!T)sPb+o{-uK5Fc8@X`kg@_Pmac6L@RlChC?Aek>oh{l&Fg|)v4)q$oq zp8MD#rQ{T4+-7hgUHf91TW6;d32PLoO^${EtNh!a{LwI3TUd-gKs(+4_NR9_T5J!x zOxdDA^2992Xj$}JO_Sa&yr-NDWmA&5EuD2MXBdOp>3T_fbnZ2$@IVT^W;{hQO1fDy zl*6L=%n!DCRKf8eV&r^va^hV=WpI*o^5jBQa-s}JYGhHR$SHRX^eCi3_C&n6FGv)2 zl4M9%gF<;?X;K?L#^3(8UvU2fP)WH#Iw^ez7FePFr_r1Df7h^yxLO$fW2Yzn?=QmN z`D6cT-$s83qHhho!JXvwLu4w;Mit3uSlq+=Bz$O=Xwt?=jyE`rp_>Dna~i;HD7$ZXQvnr zV{=bWqjxZ~H5JpV!Q?uVJ1?ZK$neZqE50G>!0C8X7mzj3b^M{SN*)Frc#k<&Rnge$QA&^aS5CWx~EM?IC%qR0^hkr7OJPeZRjP?U!0;kvLo zL+$3Ci*aT965QA7+*~1cVx}Vaf}q*_phXOg6SUK7Bz+G1In6&>MP-1Mx%@kK zH(hU2aLFuughd>k6oK42T@YL$k1UhS&wsYPGla9H%OA6)%s+fn%fE2+19Z8vVG^kF zg)m1T-X6l2y8t8=VL)m52*c1lR`t9b32G_o?@!)Ff457t<-h#*Sb}k_@F;p&<#F$9 z3;eFZ)|{1X1Y#v|;U3lIB6ppcu`>+Km2Gu#_yT z3F8D^Dr|9EriklfrTm526OGtsfe@V>fr7|LNhn?wZYY^v9hn|(I8JI0xipw9zE6D* zkv$|06vdwFAItHpLcTiq8C9rNeuv(x^dQ}KNxjC-o{AjpHhOstc+WMxk>kKBkcIhLwG2g`{1T=z`>2E6|ox z+AvjUMCHxBW}D90Uzw)g@#mb&l83>b!=mHf%aY;Ngo48S= zjbClzBEG{B^9gn4U~COF)RlZjM@TQZg;u7-A~) zkS+TIgsDqN6YDPCiaGfc`jabJEnL#)hDiAn3)!(NeO)vb@=k!DH|m(RFrjz(ONbrU z(gy~^4t`7vs4UM2+?OL8I$dJMf&Fy8?ocCcFr@s=TmSLJL7bms4`e@Zey z68(+rdq+F*m zTazS86BpY(Q{@%JeU#|Y^qwR6t~j0fHtqzjDn(6%7#}+v6Z_vt!TDvR<5+J zR_?UvS$)iU*)dKSD*`S1{S#LBlKW?;Y5QeX>6Ygn-QVNY4VHFlz|09y8rp-Yy9|JW zp|fz;7@}kOkQ}1I)Lj8&hU!_p_Jp=Ee+Ue1VeU?d5U@gKQvnnLkqZ>|xdUW?3<79?babP~I=PPziGXH+&>+1z;W!C_wm^6@ z!i)e6Kc)1m`bU8BCWLygXVPrm8foy-&fBrpAg@thEBk4H+|Jj-=q_G4ZQ<7|js!Gb z;>U4O!;g4zdVv0bz5_&X_WDyeaYtBy+AiItL*?|-VsG`sVs8&0IAQlKallLty@2En zpg4Q!kofv@Y1ba%wM&ihiG23e>->bQlqP8McC8r< zzTa>;!_pB|nFnuCW3lGkvyRA3rL>I{1uAfjBqJSeFvnMK;l_q`^!U~WD^Ju{Y?fm% z6dr(+L59gJW91MT%amek2xzb`>Y6xX(qzq4@~gS2NUowXd2;ffB2|h92DnCWzt^v>D)+N^W!Gd=ecu zXFDlvKKI0$Qy{0!>tV_gVIWU4gQ*fHZtXf`o~lH`XH7AAHi;YJ;1Upj5$9sfNOTn~ zBcMcFfmefJ^x^KFHP|2?)j^4GIa>A!X^dG@b8PDDnn~dLn3``9Y%NI|#sNK7KEQs0 z=p;1_-dL0ovBBNeHhA5fNW?3OZPJ>h5JEAWsXyOgSk6N-+F`v203F=YG`JV%ph!!a zA`~q*X%a;EL$Q?uI%}yJ;gi-@ef0cMEd0S@9pM9SlbB4rD=!&^q`0=!QZQ=Ierd1X z!Gg(PS;)Lss;;bTte{Rcs-G3EiWx!Fw_v~-#wxrSBj3*`T;O!B(fkY_q(1vsbP4?Z zW}kf&cjK=$iVvvJDD4Y`kg(7=muGgt&=7&%`7?H`WBs55k#WGY`f%xmr6nUt!|=wY z>Ck$sL;76u)*wYj#g0{7wEP4ELEWq+3}~hdhcIAnAAH#0zR>#$CPYnT@gO zC^S$ZMJ8F(jqo)E^e=$FAjqY02MO{3oY6mHdF@0#G;mH7=J40glU z;BI%QohQF4A5deOM2$OBXfGhE5iCv`fVRrzqe86+5rPh~kzmLKsk)}XN2Q@2G+0cV zG-pLak-4u;#1o9#ZaZ!2wtl;9weO%d9Cu*3A>JOCgE4XdGb=wHwnzH9XL)@KdL@%~ zA|`Ad8nqIyWdS(D44>ncw3F8(#bZ?7a7XJt@spNt4S^;s0AlWHlCgwA;?+ndb*xz< z6?e?4{t{{+D~-Qz8lNbVH0Ow2i6h^(LY)yI@|&?&Q#}*9)aQdCjP2~JTQOWYzJg7r zvmd8(7ekC8m(`2ayw&pSAr*^S7441roL$RGnK)u9c$bXz1!nsvigUXHu1o4~z;lJc zD<{gRX-gVfw1yTMLz9bfXbq@z2`EuYX)9F5BwVhgCM(x`;oNB(yTn7vu2@OenN<=t zx-F;n$CFIs3BWTq77o+rLzW>s>7ld;fQ&kkjp4{X%k!PXk&djklR!}|`be%s9L+cd4G!t6G05~0yO zjh~!88fxOrP7j9`w>@CU>c`}aA$n0#T?=|U&ASTmXny?kdgSYV@tfG^@pS$XrGB zUlZ9NnUXJo6ZNnx++40`t&lYS$+3Oz*q2$0d{%cdWzU#`hc#BHGd{VkbP#oveEJ^b zx|d4#$aRy=Zo24Kf?KP;A3k138uS8C9c>8S;_+g9DVFEL#bp6~v!sMvhG0Hn&UEA? zt-@eem>K3~fhsF^RP;(^xRi%V%pBg3AS&PZOuA%5lY8cI#oD9VI|g;g?$kP^os#DP zw)03Vn-iw(a&mvy&ea`RIx{NN^k-~=N4X?cZtUaF89cia(7wv;jw51_1e43057P|c zU8Y)o({{4~bnGr(nuwHOiHSH++dkv<6R!2;D#mpgbUtE#F&5(d?bR7C$yI^m0j7I| z-p+VD0Xo0VmCcO8u#%E5Ni{6s4Ej|`kY@6D=J@V}vbk&tvYY{4$-=yib?$JX)>-Eh z5MPue`%)qrrFNQvcccvL#w!pyJKK>wHA}d5jN~0r^?ls8#DiEJZjah`ce=72P4P`h zo;O3W3~yV6^luGN10g5yc`i zz70MtXFq=5lQ?zAPWNW#ZEDqzuEk+bl17b(C(`yXVt2)uM&=?cCB;2rSRnZK} zI??CKr2yHT$*Z2k1~k0@jAtmdoc2HDtX#J9roMO`DrB$Stif}{Zx`}uL}EGv+#QJ? zjX8B!D(osZ`x$$q-#o!J`yBUk?w0`j%ns;|x0%1o*y*gyV5c3oqS#9ri{bYrRJmF#)Xy1Dgvi(uDyJzjhN zmA+`6!;L2b`QwKa#((Nj|Cc%f6{l}LJF&2dy@{>ye{>LhIiIyoY!OY-!6~~xs!RBvbq`Y`M zJ)OZc=OJe{`}g(jjmjT7)mS?|A+etPpf#8a@wa3g1{&(iJBYa1_UT?hOkz3OKhAgAa0kN$BSwJ! zazLYT&Adh27lAFCTbMnA(?44p5C1Vi8=7rgv?3`0r?C!x&T3OcrTB7O1%-rOEPbam?WV(3v4u z!#GYY1_j#Y+Qv!yK6jgij+)_GJY@}#=dyZ5EPyntxLyxHv+!;6>5D@%5j<86>KlZBTtlgsxeWEaZ3CWnbe7vL*3XMsUs64ytSm{th%7^KFh zB`r;{9!df}&%g`!=I?jQ{u^z6%0+mLdCn&o2%CHwu*iQ6{`0Hh_Lb0}t?Y>*=7v({ zT#2wu!a9LYJ~{y}I;NPXQv3E}^z0sF54|8L!Z zm%EY~#y30mMj~km)liGQWM{HHg_*8Gv+6i`RI_=z9g0p6H+;Dml8uNIUV`8htz7PJM)d zF*oThJF0@QX3R}@1jeXKE(YTYu}CzY1>zBfZ`U#u%F1XkB<++EVX%t3x&4fF+O7&&|76S9}2 z-6=48i~w|RBt4MIJuC2*pm_h!YkD9aVeNir{RfeNZ!|EZFQxP+Hk^b%NJaM74dU-c zE6tR`qGkuY;<;j5vgUW zx`>t8YJT&cI9dnUedlj~0F1KU7gUN8t3r>+~ z5E0b_rvX+~6mGyEo7DmfeXMMTnW@|>%VSyWK4*LucI%xK@ip0Rk z6p|%-jhuKv+i{`Bln!-~laB+7mXBt5wq7&xBrX>NLb4GhWIlu=ZZ#u|Fd4II*g=5^ zP7>(JGh~`D&ALQ9gGA0~w|Mj@kMfeUnT%R(HD{k@1{Wc!;}yGQonuh*PIGyy^7?3) z5eWmGkf`~S`S6laum!pFxfJCa=lp)a_GKSDb-} z6MJ>+Fy`)fJ$*0Te3=;nHs%Yn=kwF&v)h+)dSSQq?{4j`Z0yu;SP{VpFJkq!yJ*D9 zP9cOIHV)QmtRP=Zq{Kl)f?^L{k|-$ICLy?Qt4Rz!=>RXh4Cc=sB4XG2Kfph0wRbl+ zGqI{sfTQ@(iA5J%?bQbf2ZRH2JPoy&84)pM?s1JFM@ZJ$jedz&$3nzGkd4uwPFs~kUIvXC&6^qAITAG?|v=I&PO@nu6}c-oT+26Xaa$~>mO zQJ9iYXOK_<$jx5fVFo6_KJIise*#qzd;>#>?&JxXYvtB}lE{!TA{aXa@1DJ1CJWl# zs8jQ;Qsi@SW5O8w49*WTCUo#6*T{LaKMwPBtw_zX3|`5=+d9>=3ZNVhasIykEWM}B zxbQx^*3(3!gu03mqalwBY_KS~j^{OCOtnvbtwC7wTMfrxyIIu&^F!Z}E<5I7+FKo^ z^Wa7;ZMbVO9#CP3-~ zqKbVGbuF-bkfT1Jqj`s0J?8b6$ra*e_N*<`jOuHpb0+Tx5#>Gen}WHlz7}LokRz!@ zW0Vf?^Ric*%3>#Gx#eC zv((LXcKZ$8g*RzsLuMMtMJXV*?hF$(;-;*=do0_bOrYIq)iMIZ8(;O_=X@CHamIm= zUEoS@NPYAj=kS0O9tw-1TX#^JS&ng)uSjj_N)9n<@ODV}Fm-+_Q7}rl*@4lDn6PT0 zns()K)wo8QN@ePtrX^XJ>cxU}$rln!qhB-*6Ot9HLs={F>{H?|tX+Z6!Yc1)#u^kIG2>|T&Nw$1Dr=d`v^nlqiU zg_mt6f_@cZ&s?=dy5+PrtyRh{t!1(<>zQ71;GK4t+3he<+XRC53ui2_wFU4OB4;xL6e^ zPt_IW#B$?gEF?s*U+w)08t+}ik7cC=Dnkre=V05a;^6(&yI`hRuyRo+N1)nZewB)#9m15S~ zd|E2ekt!>z*BFG%ww}l_Aeo%76)Y(pps*2bX$GAt@iI4{irImZu^dCSz*74R z&RGE=xGCWhQRo${(h@dOr)m)D8AkAPa(i%U3&u@Tk~FY_0G}jkXspzpRg{8!(WHk6 z>0UX`Vw&XAk>#MWnBf}Uybwxm;?oJt;c)G0ItId2p^8`%bo?o`RGGw9+vW5;x!BcV zH~pQi@j^x2SvRXQ`lk?X*s$k?Q4XjWsr~T>Wa6 z+3H_s>Nl2Cwcs)?(kctLc&M35B_)X)gvZ<0QWh4#h$Nh(jO>K4QX{1khz!SvLyd>C{V?X^BB6Bx(`2+w&iCcPgvAdZUxI0vW)2oK@4y#FjGJpv9K9Y`Vo z3Y7P3c!6CFpTa&DsR=-W=IjiieJn`9OALSrkn56<&F>ch%0qY!ghE-Tghver*QN$g z{B(L^M59cAZ}0yG0CAz5i(R9m`0Gw9~Rg7b1Q8I$Gg!tA5e-r zAy_c(G|3nzXkeq$G`PIknQ9Zz>raOr?)+!f#g?fPk<27%jm)q>oNZ!h>G+@4t|mHk z$QVw9NV^idshKjKUObyMbUmBiFhH1jG$fJ%swQAp$ovYXp(wZ6ym|}m9LCv(#k-&v zr?=L88i3&=N_?P2vyK@K(g~w*i5J#E_T8z$=*@?_jR^KI_pezhs>Z;RbNd+{$j>8k zE}cV-TsS(#3rdY4L}K(Ts0aRxw1#jYyI=;LWjycM)>O}b>CowVl6puyHl4z4g~dt3?3T`0&*2cv_2;qVhZb2rz;{0rP4CX zsdTTQB}~zYkrg|XA*F*VQL8l_Lop>P$CY|QfDaL>S}>zO(Br5n%VG~PlB^L%rl}=P zs0_HuX`U==ws1@*tE<=K)?tmDt%P+8ETKK0bIa+&HQ{60%pcT(StQMwoRi+W7PWQ=jAT5#f}FCjYZQt@c~$DG&3ou%0eytgEGiF5(7T zk@%_{H3zGKcS{zThglVA$R(+q)-n=P=Z1SsI>MuH`CJo8W;93y1MjnG?3{qdpNRr- z@`h-+p%h}Om*2*Kb+&MK4#(FCM^{{g=C$OGZ8FM8TOEZgvCNK#5hSCqx?n^ub9#Cm@bzc=i5WI zE^d)JerGi*5UgaGy;{y8ZiKx`oLi_xmS@OE6u&+>@0cvvy4-7#CDIl3@Ce0GQpE!= z>jFo3?KGm1X)*1eKgVbrc@(X}IfQfErSj-@`#O-z$18zmz492C!&OZq~& zU)M}@83yU}(0;s;f8V{g0e%0wwma+owgz(S_=$NsjP7J{n!0z{0qc$(3rI`BE^)x2 zxQ;s4b^1I)(xScEv60hqILOS4UR))@ z9mDUmx{p5-AH1PpuQ{+GUiMUXXW2G^Fd`qc@#`pccYLv1?lYcFbytN^?cM*HXQ;k^iHx$`DRU zIYVy0$lV{Om_59})4qeKJ(dx(clj=WtyDLC^D?Nk*N(>W^t-SpU2Cr%&51G6REVgy zm`5ghj*?P4hC&^fP4;M0Q+Z!R+lV4>r0#G{RgTXsuDm{LcxmFD`xnis=q3TI^0F6L zeS47$I%`Xp!4E1|vyS@MLo@lux%NfG9l=iFu|7Q7_Zyo7Tc@yk2?Ow3RdXBk!3#(( zhNK`=kvX$g53C9g@EOqcbw2=xooO7#GOV>fu-QP6>#`9MQ7?ql`~Z(0`bN{}{dC6W z6?KU9%Pi^F)Oy;X8rssRr0*JdpBp$98?3{uud$x7oW!tuG+qP}n_Np%1c9*(r+qP|^ z3thUk&woeUdmqj|XFbe_6)_)Xbv%{t3Xb6PZ)c+adgKcK@qw=tWx2Ojo1!>)MK(Tp&E^8u_F&11igvW_5V5Yq- z#uL{+wcL&1tdY=4+*H$JiRtn@Fjq$)uRi3I@&vT#RMK4kGX$u1`(#UfhVCS5v)SlH z+=V6J96Pjwku9C!AXq7JKWHP>xzEnV6{D_LJyU9|hSU@%O6Gi9<_zV@G-p-B^}?w2 z7LsSh!OJ+#n5v^JhyrmSsTEY=z&TevEK}^n1y@Awi3xX?iFX9WKXoi?|5=mbG9S(b z+|l`os7LU6WiBtvXAMqklOge~>8@2_?ASkJGoY`<<#0XQmPMcFLZ+2H+<4|ks<}&# z&^1UaH^+0L?e$kdc3;=fe)p_(5}gUow4{&jBICT{`aIYO{n-{6$HMWTctbnHZU^+x zp{?4f0B5!v9VJdF4Ucs)c#X4Rn6pteb-`cd^$+SN&D6;j+6?lumS06z_Q|ZyQD8hq zH0=e9D6$WhYG9Yv^eRFF`ZAwCXk_UNCOWJ4+c?~y{hg|`8=EG?dR=+#5aIC5pM4SX zK>LWY`;uKTu0)C&lie;o*~bhQ-mtMUt{?REr)axz+Bc@w%tLuMruMjd2QX&4S(A`>{ zo#Ti3@dFd@KMC{y-r@S6#OS|;`TuJn*5wOjs48~v;r2&ndt!0|3Mf~%C^!@sm-^e% zrY!~_5-iwjBMFWvW!^tYirA^%tApys7YyBAIjrWov;>u)#@lDIV$)UgM^jEFK(eEx@r5Co83%GVqK3TyVSJyN*1Imdc{3Tt-79xGg# z4X4OJ1~z`BQ5ayv%Ci%Yf;~O&*bH!C&92;o!k%8Jl^%G-&i%Cq42NOeDmh>chhfty zI)K6ED>cw#`BD$4v3#ij)L6Z=0~{=0QUMM&FR_3d>lY2UorPN(xSns(kNsIJwOG7Z zTsTH6cWhaoM*e2)TuiR8czBk{tCD14bB`;xoj-=d)+Uyu!{(Jl9#A=^!}{7u{#7`p z!TMT3E--i4Lj9?d(xZITR-s!q{-IRrJ_?hrAPf}siIRnWu@X?U2VH`{QL5Ie5d>#UTwR4+^(WiCdH%j{<)%4 zBIUW_o?q=#DCN2G9!PCib**&#s=`*|D5PSuU|gW$o=AT5BzN4awgb>4U|+5RT2CF%txyJ_E=G0VUdY!6v)0&KWNpO#`N?%ZWC z3g&SHIiMPpa~e!-p9^F1WCW#cU(|_dfDAT`*_PC>8*E=2gVs3J+9?Ajb(avJM9&#v zMZuD7+^3C^JzTOdXpOPxT#o+6gnJfjW}F0*8YRH^9Kd^s?iRxk(#Feo2JQ)Chz{Y=TN>lA zMnqxKTfE>6Y{A;W#xePd28O{xu;7j}h8FxRm;|fT>RIwk|E2>qVE#5-9e1P{xWT$M z@(e7<*&_l-VCI^x4&BlYJiszAc8=XL4V=IRSiDdStiXOh;NUIGfIje~p(2(ptQV{Z z7Jw<*5Oc&a8D=>WL(j4FWgN^R!hmUW=9YPY2DXOzJ?aM^?10{(au()q^VZxwalj5E z?`Unvo)7Tl3}fIPBF4LPBeShZYZ}q;tpxx9lb=+uFQ3VjsTIo=&<9&W&#AE~ADBU} zIYQsApFQP>G2jQgf%Q3h3p!vAD8tM(^UU32fURNa?7L+j=zt9{dr3D!7!bhhHhqcR zg8+OX6Eaf5+JB=!eF4A#V?YsRC;e)mMC&fG6W)LvtOL`#WS|M=FbnP#U}&Eg@PPH3 zNN_BE%8)VIoYG^Ud|v=`bpwun(g<@(5KX4dm`Y>urw&wY9?m=M;*9XezBG7Zr?7AT zZ8^^QAd5X@WF;oP`RdqhD~|9!JpcoX-{b}GW2Ai#cdp+NVSoqrhUq- z(~`=W5o{Jt__xZZf&ZR85^B{>`>0b&192>;huFtz#|_mucV>)w-X&}2#bj7IBJvEQ zM;PL9TnkZ;hPYW+FNE=`*oFKNHJ#y^SN*kgr!DKLQjom}nOWk$XOEnJM1JX;~ zlh%vv<<^VsUvT5XzHsy6M7V$~jp;)i5Z6mR$Z5xo>RCAA>=aLva7xqf6ihqFxUsBS zIs(N%7X?VsK4QXNI5V%v8?CFtGH(`5Sb1jBm$6LlMrKq!)ah3AR=RrOUik7N)FRI; z9?||zk5M>vhNoXR(#79^lFhQ{yK1j8b~|yAfVs8Tqc_zYj9Y4zON-JvA)E9?wsAy$ zc4rpI?HA!pS>lAtn=3)kICG|osyNaXU_W#Irtr4m>bSK9QxNvnCYIH~bDY)Z5`@4K z0l-VE_mwekmdrwBlwf`pCdkOaR$?tPva(c}afa;CKAzokSIwEh=vPimd9n?_1Ik>z zShmY1c6O=SeX$K@k63uOW|-fhW>${!#=4GNTGi4R>c6^vcDTMGH(%VFbqb}`?76i2 z;t?zy;dmC&?+li?y`tw9O|)Qb_kG_)uU|FM<9xkLFFXbLr3Z!O$shEyZb8(8HX;Mvx*d;lW>GU3anq`-*!hZcqn+@@_k2knUg z=8c4g7v+>HEtj~UoxG=v9re*C5){^*vMEGTa)+CLOt$-CI4WhFaQG9@MpdSCJ5WivuD>`E=le>cLm;Q2IHK{g~Kg`%|~BmpuoPAD-x>?!yZ6JbH;KN zBO0_Yi2Q2|_^+35pSA+0=%e?1AS=|OxB46LqWB%H9eaQ*l)VDs{v8LHb>XLW{SN^$ z&?Pn=HS~uBRXkjDxiO{lJ25^te*@BUBcH{W^YeG3>qLOvI@7k4e1BD{<`9auxIhn~DI&D;fwIaKcxfm$2_Bm?KEX^ORplB1*+RWq_}oLj(H(fPl5p*a1_3lZ1t8ie z6}jHhGb~~R>Qs-l^WN1U$-y*-q1E>=yQJ*Rrtp9#sj_fS(2*jIjS6;aphOy01`p?})??t~(R$GvUyPgeEhyd_Vp- z8e2hao=q7bSm_HJ-OrRmmhJd}x7ml}f;T14;x)MY$p5;rHOu^yJb;ngncF1z_0~SU z8VXzMB5pAh1dcT`A<2%ROdMswo^?tKd}?7|6nR%pr5af3 z+bj3V#Js599y{yP;XR22knm-4#}WEm5#rQ)os-@{L`yP0E!EiYt;SND_|qd28c!MbX#YL`=2QBp8}K4ZceIyX+zz%}f~VcCd|A z64Q(U!&yugK@m}4sOWG8iPXc%EPe$3hgFm#dB+*16ILCpvaoBmHh0rj(d?|^bTP?= z(CXy@b5437+sVUb6Sk!X=~O+V#!#W2qMvuNG*Yv)b&HEPl>|F(rjve}N-Kt0%%7>( z;dM#D;qEi`(;#hUp3R=n^qCP(RHs(wYes_8f+|ziR@%{bKxiH_+in+J%w#Uq7!|N| zc?-SOo_*@YTU9oJC0FKJhF&CBnyJtuWpooygYdwYOBm0GYkcvk>Y%)(Bfxek$jj8S zWX3`qk3+cS%Cjo;FD5E3rA(XU=ErDQ#vh5lyr@J=IxgHU^i$5Dx4stZ`=;d#bVvMZ zJ64C!4DsYz)|fD|dCC3+YK9Nzp2x3-_$LrNNzL6DKczxZQ7d!Gw78g}x(Te>U!YT{ zYoE0HIn+;ANG7a4QJ>42ppADZF09hG5mgsnCn+;GfBS@HOB5e(N{EkF+sm}5jVCu! zARS^Hcc+|Duao_eYr~u3c66dLSuF)geV(wxoU!A!C$9TKNts`v?25O)@`w24S@Bay zzWX?UsulvI^f$wcz-jvh5xE*9w|8D<@hT(__X`uB-8P#z!&%VJTzj|U<-F2Id|R54 zueea{SC1uz_~nnM)Z%114saujN* z?2*yW-G*w-TFTSbP7B-C3zgQe#>Q}w>SiOND=%;GwKr(Lc^dOUYgj3r@$a?T_~di% z*(zqbTb%f?`eVGafae(5Tmu&@K6@2iHccaSHPoYd9 zEjEwU^5$Dk<6_OQs<63k@sLar{XskC2c07Uhp9^qT{GcT1&umeN+O<{G|8<@=I+sC3=r zd*d>Rw1hO&Kc2;W6As?B2GFgAmF#a`H?t2I$$T-bn(Rt~dyooZi@xQ!Pc_!Cp$9kf zuXq8mE8cpt$Iqf7L_qX46t%pV`>%-kK74Kkv(jZ^0W~WsHgQ=vr25-FUBEAH{+z#C zMzCG8R}W8C-tH*uxA*vZ(wfG2xe;MSr7R9-z~HlXrQTT_yqDg`9*#98bH|3bQeaAb zj;FI&+`QFN0z=4UPvxrug|;;4VZvU?zCuA>Bm&b(1v8`zganr#wOmd#^0e9f_~DF( z4io+Ecz0SFr)QFnnzwaK9QXi$Cou#sA1^-&xNKx+{ zwsQlmsY+6cxJb_|2ct0t_7vCh-~NnWcsnrA)Pzqyv)Ogm{ptAK9lCB1GW(ZU&RSia zHmf~&A2$Ft(E=$L^+$gU-0?F}yOi+_>@5Z!7TfolGW!Xa)l-na^EYPJp(j(3PYkiA zVNVo5`{#9-wwemVaFH|*J&B(OOI+N|om!#mNL3aWn}WPr*FbOerqZst`=Ji-3&~&+ zWA{eo-UfTX$a(5D$os2GP){$=607O6BnFk?Z)Q)daRi|_difuQ2e#z|yL;>M$eue# za$Uii^;y_}M)RS@;vp$N?gGGH*GPH7xZ1(v{2G2m5%yAa)|v;AWp?2QL8obr5$}Wq zk;R~E1#MZ@mb-X~!Zf$5HM9BmCbf67nC@udQkOA#rD^70CLHLAFT=yksk+lwo1wAo zi*oZxO$rOONtRG1r&5cO5~V+n;;V?1uAcQ%FUxbbqOHcJ3G)>!5)Zatb7&;ZgXCn! zEiJ`uiI+~ehhCQ#*Qm92b~LpWn%+rsh3AryQV`VDKUgP7E)z5swS{@6L+2ux2Qt$S zJ{pDJ$XCd=FkBNCCKGw?$$uMAn!BoeZ+ZpbVA$u!Tg=%uOl4jB)h{qTu43)5YY-xD zW$SQLqER9MR!D9|P#n42eF`qJkyzHW4M*gVwn`g(w&R_H9HwFLOZ>4KpeJF%N;a{j zH4(}u%^R~Ks~lX?7v@xAsM?I8CVI@TE#+&}s(@4-Ig3{kr1{BvyOYd$SfTXlFS8Go3l~$uVKK6n?Wp)%ziW2(hppEW^~P-JJ82OEt~Mt zYkN<_gn2Fkkx-g>|W!AxM2$M zq%iSH2N(c)jw75P8JJ5M6Kl~qO}}>uD^1uM?Ry7GjD%zJ2NZ=Pve5;F7ZlzmAi@PR6YT{$dtQV*w?W_=i|gF z!~Q8-t#-6vdFS^xA2$(2S5l&d6p*SMuO&K$E}EPSfQnv3Sj& zE9NQnd!OjTnVL~!d9|VJGaKu7xMf`YL{3ljS@iZz4Gyl(&bnM1>k7z6%iQ)- z>s`IZ!n2u6e#Nnb@i0!xF^oIyHamH^;Kz0ifDjEo0C{Ow2z>6{F@&NNsKxoThu!|8 zch_AVzu#39ILG(duHKst;;lqw=(&zzb#}-7=Bdd&L4+{OLADM8l`pX^T9+Doeb)=b7Wt?bmsE0buZxM6SE4wugW5u!VlFs)=U+jgHfq1KF=8XVxz+#zl-Zyc9Y~gcd-WP zlAnZq@q-`ugWQGlm)Sj6!4z*NsY+E5p$a$t2Mfv5Ty~f{-y6Evr2)6mVLF_YE!Ko-$F0|>Nnl@|i7Z%xV61?xvYqTHSIX}-S*PutNq&(vF zH#*%FMo1PxL=hiYV?_AD{Nt6uS{drK1~03} z=a%K61yO)GK(962|3PRs>xFXf?a`H2Qt{No(FsRarxQfJ+&*N7??)O&8t;#+NEw2_ ziO323K$>eZ^q>F|UPxvrg);G_co>=jj>RCLKa7yUi$hs2jdY%qFF`p~bl8*oJuJ-MOT8fxfT6pUc)c&*iW!{+G8C zn6_kUqwP$q!b=}If;5QL@k;SBY)q|Q9qD_w(uJ5<8++)+P1f24g%H89SL%wL;ziij zyoBH!ZdIeT)bP?3mKDcFEqyHdb`aTWdb;c20zM6}L!*&GRjtK~v+JQ77tE{!VtPmp z{;;3I0${}%LUJI%P>)L(N7%?*Jtupd-7VMhLF+iM^UPM z@)&g!B@H|uOuhV#_@j`I2GbM<891T}%;!eGJ)xGigY^n$fB)Uwn4tGNy|P1Xgo%XS z7H*&*_n)V)K*A4F+vaBvTf{yzH zhZFJWg<6`{jXpVR7<@OJvcu7BSfJd53ny*x%BUs;r<3CCGQR!QQX@27f#s(I^_Q_A zZ!QZGj6}P9X8Vd*aDXuyclnY?Cw(q06xFm<lmB0xGs|3x?w`YB?`&9jKu)=90a2 zJWFQE<*M1KhL$lKyoY0yD&UH;qMniOItn0OS)x!X0A7>O@@Nwz0s~LtTvt$sa22?; zcvMsb7v<xV_$V&g*8h@f;rY4mN> zqHKq8?DD-}Z)4cMC_aY74Hn$iQ37TL6K>nmfen5_zjUO-8AOQPHfh3tp%c9bWy1+X z!0h%Ilf2XZ4(i%BU#G`MrPMFHzaUoWWKg{en;rBZhEZx&M6;j7;C{kb;}kvIH_s3o z4`&>$OE-R$VVvEmj!W0_6KY&(A9IMIl#*tU^$3$V*(ZI9jt14&9L<^9php_z7uYLcm*inkeiEsSgNG~#VHalk{OC$**0HQ_oe8fjc0y zDb1IRSNk*iQ^x^7J3~9kipsp8EJ8KO+quBb7(yWy;qXLTFnDN#F@fm5fSwK4N398S|ZTq5-CXL&U7kXla6d zPkFpS=h=U6ii(oN3_3C?v0|^vQU05Pwz6~>L9^NZ420d%>+VdD-$+rWH;FPUe63JI zEwpDQ2fJ;(^|l6%;q}-ft6yUSk*#jmI{9iDVuy=V^~7zVX8xw6TfeU1)8Nr3NWufY zV+#=e{IF&*#jz*Z$b4Du$}ZJ2YP|`eF)@aY@!{DVKPI+0wF~CjGS@I4<>`a=THDzB z3=@Ci+tzJ}??6%9q`*g0Q1t&m-;UZikhy|bHyS+=5(0B(M1fWc6Z+NXI2k;58r+r+ zQf5eBMY0~I`9s5)!4nNGah`y`H>ef@qECYpvs)|<(X)2;R&fp|JhIO-J^iPcF**7I zqx7IHEm~BqY<4t>C`c@_#cqNzUM!l%+_WM+rcnKAWT zEvUGeqcgHPP|If<>X-?O8$|M%7o+rti1QS4LDCmCDiyVa>QL@7;3&67Wz;$7C?qpX17JXfw>|WR0FYqe^2(3V$Q70l&&s3W^(!M~=sI`a_nF%-#aC<^2`i>a!UW3wh7?4> z#JarQ$+cS%s3=3clyQG7Q(da=A@R#bcsG=cOgB}l6)z#Ft`;j$DAbUS_!iino!`lU zFPzdf8Y+#2Ga+!eajZKGDV5?(OYB3oQMuV37ESX!>rWR~$(^O8A%+L%=o}kAFj2Or zYNP#!?Eoq=lxqZMZez|K1sl8%e+n%Rsa;i6m)Tp#5S|)?(>zD)0VGwaCF&CeZw_Pc zbc}ia05u;gmiHU_bgSDMG{GrF3-H-FU1wlQGW4%)*fxN^^%YRH&kkvyw`_76HR`9% zGk?~hl>}!v@kZfVtk~F43pdtS4A=s)4eNs9SRvHN2JnF8c}w76b^e}Qd=$rk?>N?U z?}i~CSaQ^zM=jJ%pv|qk<&fPiDp}Qx5l$z7XBY3Cglk*mmW^LlzD4EN3nSTTTqLir z?BfsxRm3)#G*t$EltRSVBM-`#Xu$)QQrJus*-Nly7(*n>9a$Za0aDg2ob!0gDAf*{ z$G*Xk&P&kC4*^mLw}XS`5O`oPXQ{g;+c%L)iyHu#F1Izyr1Rzl$wQr2Ysz+cn)ihR;?G(5?!jOK6q|?CD#R+swjA;=oE6Z+0N(Ev$1^I za1INWy4TKhy{5&it0k*MHxi$YS*s4W1^!8%E2_(^9*WdEI!7K4rp0@RB}Kac%iY8? zgbON zu2cFXhFN_eyw1lU>t8(T6HK*u6nHr7+>>iO8qCB~P8fn3rM=}XyM zFX9l*BzkkOpJ2cB7IQn}esuz-HTmY{PSu6Lp6Q5tYMiCr-D!~FhFEuUTT9%EKBf!I z*EW(#6iZ&o7h{rc(h^94S3V&*q)Ls096wpwvIrF={YJ5D#%WpBmTJ_)UB6?(&KW1- zm{8YvY*nOdHJM2^rqAN4ajUN%nHv@}VwE#P_C9S2m8B8mGamwud)mn|0d!SW{ot9y%LLTsKusf<8{?STL zELT#GEN$k5S@&_uN7ArfiK%sE3@-;tQGmOv>+8x0vWZL2g_FDCo~8^hpDg3lLpJ8$ z`T+Ag?`iQcB6-7xfv-f3JkIDWv15ON3GB%A3O7SW^AkVG2xV&Mdmw z)-`^ql`_)u$(lAa$^2lf&ekm(+jz6nj0$^hxM8AlA#zSB1?{FZ<){VDN6*J5L;DaB zQME9eJn<7HAsIy;{>qK!h&!iy)PKbeAf{(K{`U^o{6T6udw`_Pbc)CUqf&uaM?3`m0kVm#$@O;%MvM+^gn2q|<<(lpsECEd4!RT3|d!8sIBBmJ&w|@Rh}) zCeRo^N*b@br#b?CZLAmMFRDtIp~4u$NR~N@8fUvFD^csN;-xfDav#%3NjtO|FT0mB zi}THDW(ex2{tyba3{X7NI6Cma*_4_n=e&kDD=BTT1q7KjQ~aiD4o*%X>FL4()}?M1 zE_}ra@i4jVW{Wtsn#~@R(yfrr1{a`WFFGX9Ih~d7TR4xEk!=~gtlMmGyAi2Ibo^^z z`8-2AF2dNOt!yQ#7KtCVa8~DbBU7D^Bc0KOnmlOV`2*T&gBDco45TXsXx4xWbC^oR zu$dWFUM+Ih)r^ysW~f%5D(oT|$+3$WSv!U4_9rvEzG`f63mU_v((nMsIOvOh2BZLW zQfQ$pz}JhqtlrEDB^QG!Sj~fSkj-&8o&$&QMd1*UCGU5aGlw)Oc0Xl z35KF^uHa-EBZ%}qsX!9`7JOV&R0-sBGU>2Mx3lUrkk~7lDfNZnPiK5tWqN?vxcK~; zq^|u1Yy1~M!#jQQO#sRiOU5rl?WXE((9|UZ3E9;8}_};tH_@vj8u-zYFh{;wj*#948;qW)Ptb zB_3xS2xC_;X{^sH1r{{4HK4nt^%i%@NF|H6E0#~Z^OuCwdP{cP8LUSn)?`T1cF|yf zkYgd7w+Jk@0HVjC7aPO5#!=1caZ^g{-$i$w&6Xf5AD!u11g&mLPWCaY(Gs&Nxo9wh zhBdiz3=K>rcZr9}4{%6?=Cle(gIr?WM+k#b8Lmx@Az3y0Sxpib_FQ8l{#?r5xgpQ6 zd>_J@p09o5gybbOAyM$#Kj~)b^|bZBwjqP21E}ok*Oaxv?Pi|)eMPetxPzb(1u8zH z7HziKMStMqD8hJx`T!vm;NKQk<^7(0fPdEw`e681L(qhfGlneCj~`~=P1*lF@a=y> zx&QlxDLDR%3;ORjRia^|i@JjLWuTdEhNXys(NRcwKUz;|!x*pn3rOgKofno6-7X8Z z10N$TPBO)RM~#!e)uG-f>z8OJ8iYW`R$iGM+T!yw?Q%>Ub)#>E^% z6A&S`h4AHavgz{qy!p*?=Xn?pFyKTmWcj=npNt9L^jwXR*z{bCap+*8+H-;uNOTnH zk0W~*?N=asm+hY+>m=PvL)K6IW~mrRy)=btkaea)Fl_jALJ5CgRk%-p<`rd#)@m$!ys+cUK@ zAt}7t5&j6irw@n^xxT1|0s{~ab+AT&XC~jkJ$nG{2dyA;uZTgYC%kOfr9~4H>t8%%9pYS^Nx0M)JeoQLi^h$wv+^>m+E~{Oj(LF$id{QYMJ;uo z+=+{$$wnh0_h9IGl;O2ZwX5DteNTtG`DlI(MXzjR<0qm*2`?buY>A2yV^rdsh?8-U zSewQRk%rFfMR$wJAt{+0kB#hpqY+fgkv|Juxz-D6ugo^fOxf`bZ#;(?4@l6L9mmAm zH;Uv~C-c8Ed{IDguW(&~>43sbbxkyhDR-g3#o_Ft?bED-c)caz2OA+{hRA_DR7gvQ zng-c}HcKCSmntOGg3PY8b+2S&VSkC&Fe8%0tWb4KU>F!pari5wFq}z`UR<`@ShRL| zNVEhtcJOSj%y)!*7tYT-Jvmki&h`5)I_D?H;qzIEsuCB_L7gWMj}%U*7b@)&+eQp| zGICc8OR4OaH0LJQXnuE+=i7d@qb7z9dA3MIz&& zDu5ezCY_ve;AAEmg&5~eSw1-GI~1R3Sx493u8z}HG9gR!iq0_aMomI3*Sdhhnryjh zE~YwIq^fwP8WzEk*VurUQeS8AKm~7J%3Pq7##B7WwxF#jNG=DixA$%muBHiAoGxIy zkF7o_{|EADs3^nEC4*{l^0%omtJHK+yHJBy=2|tpay7yNcIKK5rIU(`rAdWZ$B3#< zn|Awtol{h~LWEWMU`Nwq52adV*BtsUnQLu_scaw#;{-!RYS(?dt z2f?7M&l|RSnV6XSx$(cj#!1m@$`fphNI6loGoz>87_ay+sH1a zndN$lggFXlXQ8y#k@?Nm4E@8jg>V_5`j?MPajkcFTPg3QX40+;?R#f!ed&p^`TIYO z#Vc+y-5F0k>8TFICkmSNn85-+AO%DeMiN9w^2t2GIUw{H`WN!9rp`0^GxO7hxdztr zZJ^cqE~!-Se+3sL8|mG$+GqA#=TdWtkpkzDRTUv%8RF6}E~Lqz(5*%yU-o;u79w{+ z8WRY&;*@@uF05g|y>DP-#B!F4y9WIX{uMrRGSf*9naM zQPG^KPb`;IXT%v(CMRu+<&v^b?*~t<%*ZxfLmk&lrCog^d=Gt|g+D(#sEgqL(5?eK z>5Kc+6n3r5ZgVVbUs&D9H9n*Jq8e>R@i-CubrL}Jg~jHN72y=e;+4--*iG^bY$X^3 zjmaGp@x(?@(a#&!{0q5~ct|n{rR+*^B0ScVoJYJsbqZgKb6KhofA(y7wMZozFSk>L z=n;RGnK}I#o3^{g8gd0pF;h_c1HU;lokK^puhBK4bE6}Bd_UH1hp-qxA$VcEox;7`>lBE_-gME#UT3Kfsrhk1%S^5#xG1oSMo*3zsFw@42 zh7){pa^46`y{G3I`|bZaFn<&jS(R)(3Yh*aEV4@3dW6pSOvoI~JwUb(*ffFVA_SV@n#>@CzepQr4cS=L?0OR647}YaA zfmbvUkid`|Z{)VqM$884>QFdXZfKqA?k7Kt(4M%`T(2g=Q$FK5Tp+7u;*0@U+1?)` z&bHTY5s5RWhGaG5hs|JOnwdO_@K^IUG`;T+0_o23Lc;W+wz07hzU=3?6^B@QJQ{o}^dXI#%bCp;&c8z)aA3=TU$n8RnV-*CK)z6eu)oK$^9yi=xeLR!Z zLKEEx63khqpcroH)uMxBUV@Y&hL)-Mzd;M&ZSv|Ntc9@$1TAVzaDq@%T+7! zX&$zb=TbZF?_*OH!b*R{%o9_~HFZll@B)_w>Tt5J-zR@l`B3N<0$l3?mZ~*yYlaco zc5>2!v=Y_M*5+w;vLfUTlw5v`-?s}I7zu^bPi`k3IqI$nprs2DbRcocC)|KSF^o+?Hnp#14_`TbHy|BZT zqtM3p9Ik}aXFID}1zWU^M85Wf5tgc<;gH@e(zTH1Z#|w-&gvxW9TMl6HKwAsTD=R{ zxqLf`SIxA`uR&*oWs(4MVP}~!Z=JU34UhY61uuyiwQEP2PEKnzwH=e2XRoWWtmPj1 zOhnW+UZG+mQw|}(KwN|nfOLY#AWfG1ADaIrl5>k#f z;(<2`Yo9UBtX`~u@H!xycc|6feSOwEvouS55y7;KyU~gk5P27}jlIJ$=`6jKIzIWaMsnTu??o%fm4^oZ!gK_Q>FR((1} z^1z%CThD*RYN#>AmiRZ08}A$2@ZZPk|6)t+YUJ$t?~U5O=fZzNmFJJc3O_hFI0(4D zJGi|&I6ghNI5;{!I6JtyIJgpAUe#c|=tuTor|ZXVV&P<`tIZUnI5>OOWalI@PrmTU zd$OuHxPRVcr)tmqX7OTS{9rL|{!dj!YbPmVC8+rE87dh#%5epGI{IahjN#Eg9Rg|0@hY=St%$wwj2!n(|3m1mY6<;sfbrJFsA} z(gXEDfW!$w{~}1%gLp{`I*}7jm6mkn1bLBY=7LB%2Zk6`?fKdF`0y5|A zwzBd)i_!nbdG6a>^@oGA1%r{Jk%^@lgO$4%gNcK)8H2OgKV5a|-}9R5|Ak$|sMski zC}Q$?;Ls#9V=^kD8hBo{ORcHuXA>bs&xIBsf%G|SsY{IMO0CeX^~{SBN*ceLe|uZx z>Kv0Z%$cpZ_{{iSpK($6`G5ZX3)f5R42_OvifW3+t9mQhml%A+FUmweA%_M$KrLJ! zY8GKE8x|}oK#X}NfgoDSW|6aO&d9qwN%^SpB>i`I88rujbt^8Qy}?7zs5|3rAT zmX{%_1=?3vZPuDCxfLYn8tN%w6e}Vd{k@b71QulGFVuxf) z(`PG=#S~Ge>~~P7?4?WV<0_Zq#+9oE5mcyBxne-+pj~T}YjY0o`B2AbIYjo7aaJk+h^s^$=VHg7^4=9skZ62DjxcA}d@Y4?3&^Y$<=|8_A>iX`*)^x29P(Z;ak%sa^clN)OV4Y>jQMr&iUk z3r8osxXM^hEunsRei@6`PFid4>Rg1YV`5capJ{BWL&ih$EP>6@|GVI0?g~0up1(fi z{M&opxv+#Xqhb97uz_?j<}5CoausXFMcb2U4BR2U)*|6=*<)#hT0~3VkggKzDIp`< zo-RnS^`0o~wGd|Qwtu01A|fp8{QF{q`NALV-Z8yAFv1|z{2X}JU)wQF?&B)Hl#F$Ghs#Wjp`W$rd zvWkvYM~kk@myhC)5=$y~>=tg6Q`b&<57OVBahyMm!@bx>85ZrAozIV!_i5K-r%_Ho znl8j9_H-+vPHt|)V0_OKxHtGl$T~ObrWkgp>ht$19&%MC)!S)Pf10V^zFD#0h&tBj zEUmYtIx5-F7PMG+*sZAGG+jCQGuvwhYEUFO=_R%4=+RoBh-b~S;7)J$dUc)i*yH)H zIcJMFSTJXuFRu&@daMk`%HbxD^Y+;~#n*BqPX6s^67?I-le_rLH6^D~_~1#T>M{4v z(BHthxc(!NMKoOibLVP%(g&GE3mGC!=&T!8WO@mIjx4~LQY|HIl_ z2SxU6U4jLLyG!A2iMvy{ySuw2E`__hJB7QuTjDN-yHmJR6b!%b^%p%I)9+1B#AL*c z%s=zqefB>0tiASHp>?ZUN4B&SlVpNk&6dfHM*f zKKW~)j~N>S+aq8+z)0N3T|MgV?*Sn{>uW1gfpjji-|CTaeak#v()qotJLAaoMuLLj zouj_8tOY?WP7mvEHWF1F1-PdaZwFaZ{ti%`v#yazS-h6v)NBoVn+_Vm7Jj3@nKy%%VV=}yvUwnFx&<(mfUqaP_E6>(`m&w8Elt^3+D4yY@~s3CDC;C z6`S3Jg~DNtVy}iyE)$z4mvbNUYV}pYFi%a)kI_SII)r+)@?;L;yz=aPhwV}Zo(_6Q&jWbspq_XZn9+bf2n=J`V?*fMi?=mhvzvnmSd!wD;X_vX& zyu0Vs$UjvxnXX$sMnTg{7P9OL(pS6S$pU4PIsDVQG=;o$DlIgsDTK@~wDTJZj{Hfo z=}W(*M{1Jfr^%<(lgy;gf2mk)RZlK4;+>)`@N*}nJ?|YTI>V+Q1oGbHL=xS*GN;y*VHRZgtxv6s0i`%|#CJeT? zCIcDZxWnms(<4LRl}DW5a1b5o4AjaVM*DbgPE}f5n?#+$fN(%|@c1H3LK4h!h9yEd zefNeB>H8jbCe2ZE#}#gO6exU=GT<>1%&odAL(NrxLhjGTMvNN6y()#-pbmXG+Tj5Vouh!x>(Lzgc)C!l zJy>f%4>&T#T;`rk8MQ+ioHjXL)U_pxD_sFnYThj{rwF{KQ?ea!ahgR7T+(Ru%8qWK z{IRVAG3)92{eQ(o6&j;@f}|>MiwN+YP+8)nEOJ3rLd5gdq7|-!$vec>fq*>>+JnTw z{XDVq|K*%W>|;*`7i1 z8D@BesF5gN;BTCY&t+leiG-Eaz>WVM*-7}8sa;k8tIMAsRWM5~n?IHg0xK&m2k`ql z=*&CRXL|mLpV~_d=HZIa&^VmM(yAx2`O4i@lJ7x4Ctwzjyl(j8ma~@6|IA1svpAB>y8Hm16<@%`&?a|j zj)u2ku%5s#SQi<%qp?Wjv6({3-}Z12K9_Y=RiXpMyDSFfwUedKe0~DVA=!BpBDCnNzR_845f!2h;id~eBza5Pzo83b6l~E5;$=g_8ibJjA3HXOvmKO zqf*I^51i_02TbdvqXp%k^g{9jV>kkl9V1xP*->+gq6-YP81qw?oG)=NkCYRoQ*x;? z%|>ae16Ij@bj8Icncz))`=~Wd&^m5qriXoH?_noga|LR%_%=3K_i9TXvE#`j0K?obIi&^DAE_%RsPn z0Ws<#u@;z!wsp9nOJN7g%Xy1a**5L$veVxM0OKzmX);s;xej!@DEEXTf%t!Dbbf2M zv7x!)iS4XC5`)50n4|avTD9QKdRXY}7fj6{r_WWa`2=clGp6zQ*gTGC7@!VGG!`}1 zPF3XdOChL~@Dh^Qq?(sFUv<5ER>d(dgdr*7e zYxLlnq3HZx+l1)129LZs3NR5@sT;2micOVz!POX14%ViRF4Q1sa@-ilKlGT~OSfjV ziqQ+eXX@z!&*G+c+x^p)^%1jB2xDN3{qBo}gio~jis=^HsnrkvtgD3N%(BjXxl2no zL3G*(fCgnE9+rrxBd@MPtv0Q7ZRZ(@IT=!zReJt<)Niq9WHVba z0NGzaYCAD=IUMtXsSxwG%n(dPvyZp}YKD&m#^HOuV3=0O$#m*+z6jqh4s+pWXelgrt z&5haSSe&s~8nd>gzR~5314)Az$p@I}R{9PWp4nP~9$v{k?HgO13czhXtGc&IN)@Nz~_;V=fy)-f=nT0>?dx;M-l{O#uBpCjFNLW+VH;+HQ06#ogp z`S+e`E>2cvZf1<)R<>sHMs{X$MnK?yJ5~Ho=tkZRio*>G=^wym%JTmS*@#0KwN7TM z_6}{RZVv?pE|T)cqlrV+?7YqYJW!3tCky%_oaLviZtVn179?qJ3?^F`8BzZ|(@HrL zl`P23SE<9-_`9)_c!jUIRuTq7t676iwVuPxWYSs!4ZaNMQu8@C1NJ2nP0Pc{n zA!MQ~0^q+87(`{vp4?A-rT_E#_vjVtzeQmF$IUC7{p-iSl#R%l|8=fSQI>I75JGt` zsXyOK?@=Z^RMIaX7wSyL!-!XeM^jc){EpvNog=q0ej`4?6P`EVHyR?NgsLhQ4lfAD z?3Q0vfQ`h<`saePaXb2GWJck~7keY-zTq!v;JQda3L*Z!C6r)lQOFT^R@{LRRZPMX zvTRZM0j=zCU=0&|dJGVZHF#Cr*+l2Dcr-_TxWmOd1(@~Sgje{{^i~^o(~kKK#h~W^`Bv zQRxNh_`2NTnN=*_(juK+=i)LAq~5!BIiu^M-S5HqkW!N0Ec6TJ@j zI$+fTNzfF9OF8+s#EpB&!tVnl@J0}v#ToWq)7lh8YqUkTiA1?I_yk3^!wfY7mD1mj z4G+Wh77hD4Ak!Acb77Luvxnp}Nb6&{|F1dnzb=!1 zEs|y)D?_}cIo8ewcZymQ9RW_t@37qXdGWyz7_sCnU}0QHZb;d|A}!0)Xj*8AzlVx9 zXe8Qc(nJOmz>lma5=J=!g2CndmGhw`kT0E5I-82|<$5yossXDFCU-u0ev>_%4qIKG ztas)|{4c9MalgU+MMBVUkyCD%a4jadBtY#fgbe$9xW^M7!O;@qC#jAB(Gp`GP0=Hx z9#zrsX1e72#<-PcRSEU#pb4`sVGx~Jmo%vIQ%(h>W7?$x(#QQE-9Kh|*8$aSFC|6{5lwY!)wy7XUo@n>@Wx{_=@mE%+JC7 ztq@ELU?cs#_)XibINqQg8F0-be;)l?(dYDjL7ca&ehq{klWPNTOA|wj_z~i~eiDQd zZbKSYO*lqokBl#H2%^w{AO{%3%&*_li@*fT>hiuwBT#Z1a=$HuQT3lAY?1k&B2cnW zy9$wYNquR=RW<9=ef}CGh}H)VrNXiUGieM|gu;itLhr-H-3dj8F(e5J!fj(Q1e?+G zZ%2T`ku?$;8tFTRm`wnS+7*OAVa@;6GwQ!z&-FF6?-zukIa5GDQvV`C*~l2~EJUxy z^*7zY9a$(NC_G$SV<1V;6oPM&A+o(95I!iYUZ+2A&kM4Zr43WTV#f#w74#)Y2D1y% zuWILF(ig9V=9!gGzn|Bmp!1p!^2%KOX~z|AE6N753)!!1=kH3z&dd!PR7YTMA3DYx z1%uKxn62vdcU$#q65X1eL63^S^&4_XcSFg3>Kc{083=6Zl{iE=q@7hbfdOZC77vUp z4DVr+vob z6YD?rxY@q#yYq+*WCWgz-P1ku+W#fgpnHbiS|;-BWi(0)WORPQgsNqEMh_A+=?*m5 zVS)VUAAd>g7p&Lo_p2{5Fv%f3FpeTU*d2~k^cN7LChtaB$%heJDS$!0VG*MS<&)wE z|DqcpN7G1;wxAj4$5Dd7cB37@Si~@*eHTJSBAE-e^7u(AJ5n6Uggzts1)_<}bmc;S zs#MIyuM^&=W0P>62V8H%2Jv~_+skI?5FTPcd-M=i?4CoxW=PJ(%Iyca&^F-5xmUI8 z;P_6YPz=>if|z)#xEc882w{#_qg=MJOJtNr^|+a(l0R%|r4xmJ_~;j{;e##tvX*0m zU8-fAJ-_-+sRe%ecB^zD{AgR>mcAGtmxc(SY0$MvW3K|TFXuKj)YP{eVL*&ErRO}oh2%w!X|s4R@BSvDPhSy@rI z=|LKRf_OlV$Y#|wuQ~uJe4&<`ABLb??Cz@33?;0srl;ZV+DA0U!L0NGBU&x)t|Tji zNYbtMg#NOjG=|pd||!c{+0-K)>)L41N(z~9huuL!e3Jl&K#a~Eu46#JaNKfnHCP$C_5nbg~I z&^AdeH+~vA!ft==FQ_YI@EFo3$21a{vKw8H?!oQ|$-p@qquogu8{82we7_4oASy5s zz8w519$h>dr-2JZ6!smfYA(-qLE0awuJI(Bg0t!uqDE^qmNP8e#=iEzL%avaDr7iB zrIiO|?a^P_+OTpc>o%As&t)ebIFDx$hLT~bBK0E{NOIZANG%LK@61unvdD-ROIj73 z3Dy1Zq@E?5A28CFFlp5syd_9U)Ur*Sk%%OHpc>2Y7cax*@p*gPrJth4wa-hYq!awI zi$Zr;N?NK%lmIR$4qK9n@N{Ct`~x&9I!0CCEgO9E7a%VsV2xlE&HgRWw&7Z_SW{Nr zF&jUY=u-q!BCo~Rhq^9dKcOCkULHPz3k%8u1GO2C%cgvK$(w8rofy6JAtmH@Ca?6z zn-se(*K4y3kPzD0B9<0@D!@8trEm>~8%Q+~BNigA)2J2WQx)?PmsLED`_awJvlFgF%$%>{`kh6W*%ueo zzL!3_&jb76H<8r9vdVU;rc$S+tkG(kJrKVx@y?h^_|d>#o<_iLfy^i^PZTi~Uu^Ri zL%q2~(@>Va78gC*-!SF9YY1c{gHUcXmFXJqu57Byx|}sQE1ljoz`HkIRZ=Ifx(O;S zW?ig6e7a{&SrZ>_H&5OW>?WVv8l#Xl2Gi0@9AE_LksH!%mQS1}(q7wb?z)9=p7ayf z|IFTZ>qN8Pv!|34lzuu-FgcOeA-J}7K{t?A*I!@JgDVONMGdR3JjR)lM@A11$M3CgtPe zxTv)Q<#Br>h77&42qwqtVE3D5Ysr#up##myNg_;zzy0zKw%R~=UcY~or^O#dj|uxiN3-~1VhSlTHsCGQ zqIm<4uC-`;dk%q{!^2)eX@Y;=_1O;!ag0dKn&Bxq;S;is+@FUTNxFSmzeEmRsnp{A zl;KjB%n`)DdDf8<4)KzCailxK7Ek+aRT}DU8$C%-SMK?>sXM4_z0A{_ z{o9}BBZmt@U4Glhyw^F(OWG+C;lU=VFsOs|0!O}mR z*Y!(-a+~k!PUQTLDQw$>a8^eZs^ePB0C%Xby@yXgx{J_DUp7D4%^M-n&t}Y6-ty#yu%gSED~lfcL1Vt0%}Mn>}ES+YWD7eU%XH3 zLO)6u!0e9wDU@_$aE$nB2hp1dmrJH{oGsUP>FpXL6eK*b2%H6nr{ZrmM-8&JfmNRG z{xa(r_ovlfLmu$RoolX<`5=?#l=N}E93%Pg118Wu^YQRlphL=&XQjC5udMl$(AXoe;H;z2uBZ{I(ExN)Vz zLx%s*>$yQ#(>8RLi{R%JHj_M~ZC{pF)Q0F1>g|0|#{il9U8Q@Xp<6>a$ofNd@aEQV z9L!*g#uPWJINHJ+UE#YU6Gp!3>h7(Z(=E;$NmQf>oOi!G1Aa{2D;M+#-!EC01~Hi_ zU{{;+fE0}$(n!uA*0Nvz%}8Ad*~Thb>`8+BbZ2`|PVhkv0Y~Fi)1VF#|L;+5=c8Lw zR%i)u^>>*lh-g!D@3h1^^Zep6`Thz8+$IRLE|-2z5a*l6ufT z9HE3rZdc6>WTuHR|>vS3LzfA(B?X@NX|e(MuOSK zT?H3U`2hDD8mP0+TezvxBz7yPE%_DO(P^!UrnK-wSXFf}T)uVGK$}{qGZt|t)G1v* z40d-ba){6W0d%+wB-@8AeXWBgfy0}e^F2w=6KAbu3iFMbMb6qlGpdmt7%etC^Z`b! z=Mq;>7}GbxJo>^UVx)Q}iQ{+0;nc2ueR*4jjmn%*<*k1=!sw8hP>smMZYSt!p?whf z_2oBaUrxdJkM9Heqs&kT&;AYQM+>w@?j=j`4 zuJ>jaY)S7W;weO7KA4K5*D-?AL*W!bcByQ){@$+Riro&s^pl$HNhqp9eD~Xv=rMT; z#9T|C)U-&nI6WqDSa>segHxmkDMp0}xeai^rj`WI zIyKH1ON9#kNl@b$i|Jk7y$~h*IZ%1#F6D)Ofnok7_Ui&(;_Ouc%yw(vFt;Gehp&u|_~}_ z9w^*1xlVtE_>GnqKsB`8pTxn*XTpn=+e{yHY%6!&MgN6yrDI2I-SryH>e6go?i!9F zQ=qGF63GMR5HX`^MFOj*%X5`*v~tC)6Ym-3 z_~%i%(sk#8+rX_kdo{A}k^EIyUIoT`YX^nRvTQCzz3W}l(RQJ<_>eR@^o|vt*sK8w zfR0L{-Pe^-c57;4+S=Snfhc?VxBULmsc28<9@uy+`mXp2cw_X{RH}0ynsoFRzYsQ$ z81XHC+-pjFM2G&HYYw&n-UHgJfz@k5?(siBL~kKyK{5s+w6HisI&Xxh0jg^hZ#do^ zc;yv0KAP0H(px{mxnD-P4W%fA;iY>nCI<#PJ0CicbAD22w|)X!-()Z!ipC9QA89AU z5tC?*jXH+x=#FEmasB^r_x<*3yzfiyWuuu$qCt*R;pnszcZs8tb1d6^-Id?FzBlBr z$7z7|mD9gL#OQAY!gxcgT^pn${;{+bTwEZ`LNy|QbBPr_5ys2OU~9{rUYw#Hq_?CB z*$g?(fRp46ZSh0#e=gT@k5{YRgwMe2x{dn@9JF?0bl0}hf~0!Ak1U}0!ac*6$aj!h z9lB}FRV5CRx6B98+lytVW#pBYT&;y}m!H05G;X3Vvgs!T{Wc+OYovy=5HnhrPv4wW zdRB*9_CWz!RL&EY8BShY;ydQDZkHeZUV|F8=8zj5bDvjgn`)Xcr+U2flbGUQ%Z)!` z{3d&6Z(+=kpcXN2z%4e%rgr)-MY22O)c1 zKi^23~VTKzZaz>B%5yt&m*S zFjkk`47ES`u1`l@f4lz>HJp%_>}!KECaQmX92H%9Ml*zCUx^H9r2%4xXF*9?b^RzL zjX$({H2TI@#?BaV{K1eoGAO?j;RTv?CP|a2#X^hmL4AWU7v?elPH8Seax_3b8FZSC zd{>WywMSky7*v2-+NV7(p`Q%}mM!mSM)?L~(SN83rIT<=q29-@3A+6C(?4o9N zoZeV-gB=XXeEsE~F>8jdfGP2zl_7k#B|3}W;0|p<2Cp#+uoN)AU9%P7DqzSDViiD^ z!k*X5aUrjVOgMJeLAvhR^ND!iaKikUV|)V>8Tf&NDlj6CS|^k3k0*w0j}}+>uFNOF zhs}spJ0LSu?NPoGrV*~No3eN3k@8HdPv0YFLbT($C*i^@+BMdOzx6EmPfMZ`5`G@# z(*jK7bFlBbYb=Z`b8LFGe1o=PMk`y|J*><<^Wtr9{~8%j3|uw?>FMUJChN`>A$oWjnybR=(o= zIjOn%!e0aP+xa)~U&wzRF)^!F{o7|-!}UK>Pg(v?8z2A6Xl0J7oeG{A20tFSr;(JT zbom;#K^}AKwfWx=t z?-R(LQ;(V`{tyrFSL91xKs5yi%4c%7RnE&Qnk7(Z`FVB8rbpB zUQx5C>KtNMb!cgWh9wognwrffj@#IjC;LHEue|(zG1$e+ve&siUJGAajFk;Rx5aiM zYY8RCriKpj)8U}5%;(BP9BhU2>@WKA5XlkbiBz&z$1=H zJ5Q9;uKFWhcXFYdya4IoCtfuSq8~SSyloEAQ@38(mx^ zJg*ncZg1C1QDUeKgEb4p{#cY|YUPDN9Mnx)8*jhT%%y!r|CK>Y*Q(`ES?sD^zwxuZ zoOV5?!_2{{7Hp#0c3o}R?P+Wc^xc|lrDR+Ei6Zy)saAV1W6+!B8z|2!GOFT|7ss0L zH_Ptt)^taa(DO^=Up@ms*QdUspEZK1L#dV0v1*OBvUgiiw@`AVq?~vfsO`VrJeb3~ z9^Xc7rs6_~4PbMz12qwDyAW{~W)I%UAj+{L`Ra9Mo3e8jm4~ z6BmCk z)frJjM^eH19XP*ieYmy zfW;{5{p0!0zmUpc*Ghj_8BPR4n~J41m!8jFR={jcOX-j1lXV$uHoCTI+sgi$<;|QB zZ@q_nErEOG0^eMR+?Cz#e6-Q{wAJnR{{Duz!*Gd5jaP=F$qpRp%LJnfA*W=QwMv%@ zG9ry>3x%L$2`|Vuq6}h|&1wteM(1R5Fvf+awBPTL>PeHUDzw}q#KPGtPD%6e9#N!Q zLiSA4QP!F%rmJv05=-4kT86}?6szrzr@t+yRBOd_c-#_x}>ok(4|^tOiai{sQaHXm$~C)Tr( zX=gZ;TB@)ML2vNv=zVxp zje-8E2NLkHXFbSHFq$RLBwew8rfRefGe;O^eT$&^JJmSFDl5SbQpA~O^!Vei%{y`d zCp{C}G@}^tUcM_@H8(d)+=XQHlfGBkadp~ns?{nYvB@w0A?j4`Cw0kFmEU~D9JlyY zQ&6zR>0@NFcCW60c*h%j#M1~Q6Cs1QduRtcOBdFfD_;H0K?;xGu$<1u%_DC+#+ zy*~E{`HL)u5C1ul*xhw2d(B;*Qw`FGa>>76=eLK71AK$dVi&ke|I+cBnyi!pyBgL+ z1Y-Xh6}R_XlCXG)8dN4uR7<8ULh5pi9m2NrQMi>0E)4tqvk-Mkn?qBLV&2 zIfOc!89A9)GOGO7*ZNIJilGDzS_Z7q0P_?jpYpe~Vo0YuGiZ(BcYPu8hjI#LjBcX@9J+OzyezE4V zbi&}DZ2J~|VEztf{*Iv^=NZp*z%#J=U!L||E0n7L44I(*$DRJ)Nhmn~4|xklDL2pm zeuMgNenX5Bzs#T_2LDV^1ZYj$?k^5LL_$-b0##UKlp=~XBps4{?I~+dA|-_ntP8fE z95yy2w4>9tL4Uf^^L>6U0hS z+W{l$&Lwk&qCjZUdrzGa+WT{aP@QgzvNl#f&YawPm+sd-!<w+?Eo0c3q_1*Z24_0l(SfQU`nj3RDH)bF zrQgoXC|gMfv;>ZfkrYJokT20(B6dnA{WOEC13TwsSrE4d=6ho{5QLcj#zWlCT)#H53??K3KC({H2-4GbfN^N^vUg*!tCJ>O(i-OT7*cT;%n zxa4Qjw4haf&F=4t=1nxZu;#Lss*d308plB>E{QyldtFQ&%drfDaME_nd~PUc0HO6T z{>daElJrRkAHAB-^kjzOlm};c9k`ON2_?H6i8u%<$H+@CE2D?o6g1+}!|^6?pI3aJ z_&vp}QZ4*mCQOgMl}>)UMO=&PbdN0KGHQ1gsd^!QBvkW=be;m|#U@6F_icvSgdl|G z-rq@_fzY?#JLqprHa7>zIU76jFhcee`JVza0v1Zn^%;?u|No-G`rlEJlL1%4;IGBC zux6245Wu1j44=!#Zn89LLJbzOF>11W!p~bsauf$~2KrIHg4wTOG=;0vzbefUprUCo zZ2k2*-WcqDI)2(m@8d?YO}0(8`@Oc(Sbybjv>6Z)18Aws7)HVmP1Lb^9M-%R3KnoU z*9`HE=?BswBbd4GUezq5<<_VX;#SdN()vspE*I9AuJqt4tSQ^<#ZD_jiS(2UduAc< zu=HT(_zTJNN*gP1sAg&o+#M*gV)&GwE3LpMm+FvW^hB3$H_!&*4GUn4?D9+1fY2|+ z>JJLrDGg>VK6#-0J%(+%xEqljQp6u-v8m}1>R?ypLKEqn=CQGcv-7wtF7fPMO>W~( z7zER2LM5-C!2$o4S(SnnV}YS4>W0Kml9%XQYQVH@=tpKEA9IR!9Aaxkne;FHo@3_C zSi#T0fd5Ax&i|)c#`ZshGbn`OV>2&mVprYbi_iB@%KQr2DcBREhEeN}gmzeO1JN_n zoKntyEXtr_@P$($zf(IZ!>8tYt-P!~9bHU&e{}hM;TmZ9Oqa|UM7%7H7JMcE4^zGB z5L{}MX4s&jYqz?F>3=1R&S%1W`zK+T7dOf~&*e2a1s~cFg5Z(?R6tztokGv!fEZFgekur`b?M~nFSbkQ$s%snR{r4HRi1Zrgg^k(aIb+!Z$9F zfYf@SFmZXUh@Ux*1k{V$hB4In3Q5hkKbeIgZK&n*{~S0wo$jC?(fww`m@Ht<-tCI7 zjy47*>?3|~eg@_JKV}U7zd1_&Z#S5Kjcot-e)F#gH*#%xJBBDjniF~f(Shl`S85Sdh z(yq9l)@Q6{+WC7wygy(KzapC*m~A8|#XD+%y1AD3G~-0#OvnNt%%vC-CKGhkQAFgw zdsDNLYp4SDDk9AQWo z)Ntd-7Fko3ChwrOy^*Q!(Gg2JL+k~=hRXroQbV5AaZN32BGPmZ`Ia{P~t`d$%$>gJDW0aDVh0)K_xe28!_@RWP&cJiF&aZAR~ zb`e%*;+*9pr?@n5p-4(BpZa#C*<4$>$l5gXU6%D#A%3k40u7=TOjGQALp< zwBf{5zTf8vH7l~=iVDKGd`kNSok$e4LJ7K_{ug;~Iz2Ru{j*?n{l`~EhW`)i^Ir?H z+Q;1xedPA~_vM&-Dsr6|yBZjD^noxHbFeuvP8~@)S{R!Vd?B$+EcsXYtM0D*_$?`Y z4@RLr4o5U%$oU^v-A}K|FXlC6+47+2t3N@j-J2P!pP!6XyCPmPXzFjH8%$Y4{bURWxag zO3_!0Q$?`~r6Mwnu~NzJV%Ca9@uVS`2q=nYu(H-53pwD*b5+x9}a3*A&JUqSfPjtGrg$; z5GD*4HmuM`@i%zLV#Q7D!D{i*M3tJ}Gy#0$hvOQarLY7i_OP|yXrn&H_BgfP7@`DC zZ%P60iNo6sD|9l%GP0>{09dU=(=iemtp@UmL{gd6hJ)l{d@W_O@+^RBS}|cmGM$WP z!$D$kZo{HXs@H@vk=7ZV%wJO)gVfiF`7c`EsAYmot?B@X2^6h>3o02AGplp}=IH!; z1DkHD_^5ILASRh24&Z54E}0r>RxX+`tJ8q(65Fu@ze4ed>-z&CKzxlF z5C`imzheWwjq;q@=MK?Bb^SGf2=-IS%>uj^I{!XpDmGMo^dA9zQ4sp1 z*DL`VDA!{0nu9%}eSgq#c7mIAxe78E-3A~Mq6pnD{JPS6UFU*iUJ!?ei?k;IGS;{{wo zc%h6^#7p#vTGhX_TbCT0rk!vSZ7og`(-5`qk9he$=`O71g)h;&?$E2<@5GgpruCM=VLNDuDI5U2ZlG1?vU*D*d){2t?)MeSNb`CN zBHu-s;|;_N`2+A^ZHcdAX`$|}RCkZk%8jYFD94yx0=! zpf1L`6uVzqzUSeaMk~LgfbOA=Ni~0L73|GH-v>$l9P&n~raR7PYHf zIx2igd8P-c0!(PFua(it3@0vV5zeYDMrNl9H)pH19Z2 zAU%S90qYI_H8*EUXqXn<=Rh!(>7L8A5=+3wP7UJg&pB}e z+muckG;+Q%0vHluMt5p_M~y10W#&L5gB-yD!?w;jA}bit!T{hXf##__5@-9bS7By% z!0*SU<<~+bciWU+u&^k2QCYFwQ8>hDF-VNqY`RwDLKpnP>H51 zVDE|DP@!iP{XXPOWl;GMnuv5uIK`8ELnb00$yl z*3b8~5=r7^-3}%JgWs1HZk)s5{qwg#rf`!pJs;%`T4P_`BBqE_>C<$g@WgmenwXRx z5Ze(vnUSh1U?;AmK;utD2=@9w!mF|Y*{YJ2p`5do9oHVC-hl)!Y}E~81Ozs&jQ`3$ zi=mBpI9trF<(ES{hSDqP%~zi9p>J7|U|0?HxT7wOfEbW>eF=39A|`Cb4Q`n%9fO#& z?pYLsH(lWSEs**MKA5e8=F}yozvNYjTHE>(H1AZ%!rDVu8DnS4XrCQWPv%(vAT7M1 zlu>@4qs?PLmk33B=1;EM+2JA(NPeF(A? ziu3u$J|#p_3Iy}zm`AWH;H|^keA2A1nV`DiB^X-Q%}mPpgCJ1Z8YE>Knz_=|m;!Ih)au$J_JqqFUxLS5ZJ8oh;6$w8DeN-n0)vB?bk*5Z&@+!SQD ziQ|gMDEp+=a#?9f6rTF>&>~`xSwMh&`6cU7UYet+4n6W)0 zoF9__nT`EXc*07hDEpaoThLxZy@_4C@{>eW=N_*S&&=ysr$k&^Avs2i?2Me1G62>; z-MD(YNn_~pr)cPS+nQULP>3C^bb2RbX-NlfC9WY+-7`50m{{h@pC|N!BI3>U*rj=d ztnuZK*v9_1u+C-uWY3&|-+1`_aVBUp~*!?le1n6R}J{3T_KbD80EB`VV{ zO{0x_^u@aT&2o5xy|V(xS9QoD7xxDO(l?zaIpH7e9}h zn-IM?A5qTL+-WWWh%;%rW^J*Yzm(crkhe7>y@fnen;f8PQ&;IqZHi5Z_~^-cPC0uu3RcSGqu4RtsA84)bmkP()~(4rZ3t6S?uN}10z z;z?SQ2w~==XDS})f7{K4tiui$NieNr>+68+F}ZeYIMHlMQ~Ale!m{SD>HU;P)=5J*n{03Ud4ih*=TTnv6%XY&u=?mCyA|Ze_G|8A z2hRStV26L!n;&m_3G7+~vF`lR*V0nDu!(4NmYmHh*PgYNz)@1aZa+PfwQ?uj($R%c z&{Y&*5Gba{TW6kJv#(pwG{G6wzQn#1WuZo>#@_9u{xERb)SI((!esro$zg8EFmYqy z5odjH?-nQe?6sLxuq(Vj0aOVwO}SHBXc6hPnZDU$I7?s`3wlGm1bM7!i_c(BT{iE| zhNu4s*^Uv02sFL4H&Fe7sAZDw*EAH-39grUy|h%&2U7|ibaX_l5l){`tzSybNn`+w zJqakISel)SFfys2KQ9`Cin1QY61l?3I1yPt9HLj;4%TdAc>RLKz6%Solzmc0aB33R z1b0zV{*`V{gz!`6iG3ad*X6DNOXH^<_Ck#@aqYF(U)Havh3XRQt|6BNX|I8D82W5^ zVMW?P89lL&J~p>FP!25_XbEyzE;8S+!z^CI?G5cHi=i#GUM+r$vzuEv=s_a9 zo?fXe0Rz1%~&p@LAwbvj{1uDJ}$(BLwxhO>1CcVzgibt?ME<$H7qJ+9NG?*K9R#C>-p7{E>MhsbRgU5Vr`D4D_rhIQ~{G^Ya}t;Y){rY%rP}s$Cj)J(vlL*%wiN)&hx9 z2#fNX@`8lqP_Y5w#m9qsul z^6kKQTk0_%b+8{4ukqwPlz-8t)alNW7E_ERFGa(KQ_a)u*;u7@^D&Z>lUb&V-NA>O zpy{z_R%-aYPpC5~6b%axmmATyHF@CK1L zL8^u3047|tSJ8XwwwH3k&rV&>e!UL%*fU3y%^gmSP9g=8_(SnjkyQ5#1KqD`PBW2X z@^7RaXv&-onTu$0`6~KF=ij$`hB(dSMmtn)U405wZMbCg2d-)!LWGUzbp~9K#OAcvPBO4^Ur*|&Yj4FW zfxnF5+G(+D0Ce;ZA;uJ506;-eY(u+8l@)`PtoGcR(AO4;ugL)<5}V!a=Rv!e_GB~+ z&SdaOcuus;w*zQ}y$6P(@^TrQHq04h2^yhU!mb`$1ZKR_qrJ-nB2y`+f&o&Bvqf;q zQq)YHQ+azRs6P2`%EV19RhVGeejJ#GJ(O^B++Z$d59Zd(WGM6sQzp6rCzEVpjL<(D zzkM5POY#3C-Y?QluXPV-VB5Nl6!MI(U}zJ-)gl)EEg6+FDa`#mgmb>m57Gc!A;n36n?BnVIpJnVIdFnH@u}|M$Hst+d*A?~bO` z)9Oz>EmhA{J-@2QP1>6*4Gd`~rGuq(hsby`v->us2tb%^BXnR`UI|&b@93n8IEJWo z`$UlBl~SpdYU<;{mvU(E1Not~vdOvOBlP4q(rxS67%;h&`in66+Ft&)#6?8%HzD%1 z_`$HqKqUgT`v}F2s`L|bOa#QKtno`=N~9aZ`tF(^{d1p&ekskE1v zAzNbMQeN0nxWSD66@pyM0w?>YW#gO;jih0Imu)PX*6n0_7ZE>PSr^G}d3r_v;>j z?wa*x!9ysg7_E@>XNiXaOj4LKZS)jXW+RFli)+K*jJDUq-Y(s#@=x48b9mCYr$YsN ztIi_oFJcAvhzGq!YbT8zmkMYw$g?N}VIBr=>ERG&665HP2*LZiDiPdb3PY`8%M92K zIOr6+3b2K#yJF&2f$1@&hZo6&;5b?>LxLj8yjlnCS^A(1{^*c^ylmm(GbhbCzy(Y0 z%$!8Y-ylx;HfX97l*K~`1Lu@G+>I{RmX@%)aeTkpxS8;anev&4=YVFoyMYO}#aFHX zXV4sf72)Edw2O=sj}K3EjZVqs5OdGz9(#MYvjsXxH%(rlL8vud(#x=t4-o|0BUY$h zsnR@O>qTI?b3$j?cU(;ra{_<6^4X0W@HdYnVJ;hWd?V;KX2RY!O%DeR?Q~-rmVGpM zy$`6)5tm0{U9X|e1iP_ARB&t5QB$3eA=H}j<;0BG7i1SMpIG^zhmU={ydnjby^<`| z*FBroKv211)=c)xO$FbV%%XM0^V9!5rPTt36zPM4I zXi5$XFiFOwW_i$0Rx)L@%uOuKmU@`KAEQaOZp%p9@G4G~WUk-7C0oor)}+7ULy$^f zfta$r#?=j(q)oiiFxvx0bNhJC(Ur~%2xOsFa1x^1dd(uz%# zR5`0^<4u&B-qBvV&5d^$Z-YVEipF^x4djR)7MQHy+s)M-rRmp!T)VwNG;P~mgtLwlvir#*X-1Kl;T;upN z|EOKepBrsi=q^?r^rxs5LU?DjN6JWiFvkM|8~evLt>rS@eY*hIyOTqpQM*&;t#)G( zDctp~SCnkYtqi!; zEi-927CqZ<;XQ34HrqP4djcu*ThC`3!GQ8ToBP+c_F8?vK&I^V&2EJT9^89T^)>`) z@#MCur^%a~a-fJ}`(}K;|NgN$#Ox3epSW{G@f102d6B8fl`L(3-7lF}VzKD~@pzj0 zCbf4dR$9vq^|88I*FVF#UUa-x1Zl;LTs-)F4CI&`cE1QdCn(F3?w7D51k@xo7N3ui z)fTY)+Q(Qcp!O}#C?fSw?(!Zdt!_=!W9`_)9*)Wq%`;|YiF0mD=FuNInqGAinQxRs z2JpfkS%le&%Gkc!l<^dfux{(^xyiNeW>l)hSMfO>UtzJP=if&}$V?N3HR<3)ujqNs z){dnIO{S?%_KjH`w{SQYq!lenWLVAFD2Pd_1ABg)bPu7HdKPlbDD95M+O{r>st-u1 z2Z8J6$fMYZ&S6r1eZ#kBenY4ZwwXa1>^JydHo4*LbCl$dBgY`F9;c>lNRaFfgiKWVk@UJTD36`ilK+ zKS-Ds`KZV00_ODM9)`BjFzkY(wMdEMGciK@*U%EaDmr9pc|oZwj%NzR_c?HL zpVv3Ndx`ZO=JU2MUKF;0F=!Igd4TT7FfdfCX(D-)l_R-PDity{)-a%%zwa2CLw% zs!Cf(6?&@>ibRTS7YzhF69a>)U_=Y!6zkvdEFodFR8W132!h=O!Gid&fH0et2)1|L zYFry!a1(KTN*l=7j%OpxTCUPEq@$L%Oy;wzb7VoZCr~3X)g)Yw^e#)9ylySVuoCs? z7N#vZGPn5i=K9Q-pS?CV##ilsQCl~ptXymzjP=<2S>1rWJD5N%IjiQt{0ds#*A)+q zKC%WzbL~q!(74r^@c@mZxw&4Mje>P;w;9)GrK!rgeVDz8z_!Xkwg|+oCENzelgu{; z07G~bLid}HRkoquxcvE>d#9_cDKLAE2jw-a{YxHyAuk7TaI_rixm!Y-7)m3II}*(G z*Xx!_w8=akj8_IDuES}^{+d@8#OuILsf6Or*%~u5Y(lIsSHL)0A4=C+ZS%ksDxUPi z5loJm#Z8?zKO5+fR;@Sf4MfUX>SKRNxgvAgH$q z(+X7`vc4;~^5v;tcLv%G40-*-9rn_nuNEh}f9dE`X%10!%?};^>A-%2=)ewCPXk=l z5?(XlL0;7b?)o%;;Wv@~)NMlkXeii4a|L)0$pzavQ0UjP?n=9|F4+avtPotYdttun zgRWMsu1Q~Eefw752lxYXR+p~@9$DXpK0w=T0KaRU=ZL4ko)?sradD=dqR6{ZnQP^= zK>aZjU)u3QZB3-@L6r}{*_6H+-?KJdir%c&ZuFe$W9_~RbAGv#V%7Ar5Hgf{O8&3Pa%3j+* zFf}kpSB|b+ySx5ifB*Zz-kk?ueIIve?6IV)jPVgg%le)f@jZ>88M)jGD(L}&3_#N9pmE@t^4?z)a>t$PS2o0m|J zd+E7+1^86Zup$=^;g@_f?%#T%dvLr`#&FbEB+*CoDRJVQxr=iSncaw%h6-%kFk+3i z^gg-JB*|>=QR#HrK0n%Io0V5N5~?R+PfYOlbOR=BNKtG0#CjJXea%fsq++%Re)`br z;og0p=uNuXXOV6v`KO(uS;A%DeQQTvi+D|&n6Zh4p}6gr!TJ936b~Jkz3>7wvHA95 z=mH;7C4z|7g$QPVm-iuxv^}iJ`D07%*J^GDG@<;lpljn_gQtIHLUg~ruO=}C=|Tmc zTT&S>_p7)cbqrYZryAEzg@Y8bImEQ-T)4rd@Qqu1&lTpVtZ=4XSNxp?!%+qfT{zWlHk4~{nV_>^;LI^n*eh>?a9(u!Sk?8Gtv#k;-B*y&%@X0U zcF>=c=c|j0bFpZi>u#HCOqZgY*f4PfZEIH@psUGA0fW?8taEsIBhqdCaYv7pVW2Wk z(AJGcC!|33=2W$zJ>ozDUp1-WtIL7IYlkB$kDOI^a-Vn2uI3|a8MbE1q3Pb9^~!5< z@KMV8D*exC!cpG37d24y-7VQrS0`w^-HzuKIS|oHz|Hgw!cWHdQxLgEbD#YCJ9`eT z{kLfl`vx4=9e!&bI_3y+oUmMTSU^$);a7}2>VUk{Qy50!^Xlb@$r z1BAy(2qLxPm7jXq&E{TM2YyqlcT{+hVS6NeV-B>*4K~+%WDH{y!0u#C(CdB|rt*jF3x1&I6KF&h&du z1dU`ya;{bh!#5L^Kh&iu%Cqru;+yC6i3@8?!XRN+;th07>9Sbu zeNZsDEP8hl;2$Z%#Y>96us!r`PqncO0m_=kYNts*eOS#t!6+|X;%f{~2b-z9hrM~X zM(M-4T}L=D-|lUBHrBa%3@ljTBVUKU%Jc);6aL(46NPNUx4m(Y31DDkbSH0XEIUHx zAKq~S79Pa*8V5RH+MB?2?A2px$sOQ*8`JSok5Bs1bZ0STu>+yGpt_UC&4#3%jn}^4d>j_0`qdAj^Gy;PoD*>JW|2Ca;~)H{ zS$a6#XY|3Fz7gpzWA}?#lGy$F&OPee(EcVgV_u})If!K0S7mj|mJ(*gi4TxFt!sbj zDI08e9^_Q#UKSkHa9Plz=zo<9u^rs~#)tj~GIdCxFOT*G)A%?rstH@@XW8{7D7!Il zYsj!#aM(ZM78CBanS6FXCos_OYu9rM@LC^i@PUZyBba$YV$5$GyG8f&e5Rg3n^Kk(~gsyviTm}#fG45}9Fv^j@}&c*3Y=O8L~Wp_5g!#j0s>c5g)ajIImk4rr>P^!TGR*wy6GlB(t zw@oDj2^LZPPBQv-YT&8^qY6UeH6glEdX)7}z+hUMGg1w}ntl|WQAY<$CT~W zD8M!1tC}o(8DL%_WO`0|-wx=dz_9<$fSmSN6QMqHL$nS4*HlY7;h}bX=S}UA&8T_A zjeEom-5yc5gSu~Djb@OT3X72sIMV#ic}NN{=AYTW^=t}r@lxX4Gvr68ustMA=pl#G0R>ux+)*TklF7L!ia#Jmw+LTkQ6VyS32a}LW zo@!Mnt#iELwvPiUh$2f53-_;S1{H170ILJ6rP=p`P|Jl0RGqY+JYu47Vxno-1fa-d z(gJYYY#i5eu){L%fYJ15Qr|6o?hG&o$AkTDC@{@;=G zg*|ok+ZZN?1!enhmSIABBY-#Fbfohevg%ygox|Jrwq{MhYu75427|CZ5d({-QOUw z#Iv)M$8=rTa848O^2 zk(g$nbCBST`m|n3yb=E(-UQOf`2L;}70t;ya__ zI|G$EBmPD2{=1NfCnp;XDJJ_@Cv^)abxSRE%Pln*ArLDWkIX80%Pw_`D0M^sfp8;> znZnxZQpyn43FS*J$j=P5YKd`xg?^xZOyMgmMrr(5#)=Chi$V zV@J@+ICl?zV*&SyxO)xx!|e1zI_wBi`RIRmQc&cOybMLkgvUw3u~3Uj8oj{>j(*#> zwPKpJf^q6{L?F=7YH6Ug;AmP}jT998d)ERWEWr&>Ney@lC9w`i2tX9b-5%`q6Zjj_ z(AP75_~^DV!7AdNASai_(O%V zrG=WoNR`TZS)-j!NHZ7PACNW#=9VBfjj5hu9pA{p{C-q&1g`JFoGJ0Uj&?O%4m zD+WntREcLm=V)3sLN`3#F#zkCo$V#v@+@1`uxvs{- z-!EWw7W0#2tE=N0V*($LLmruh@{yNA)m&#dp2J6~8o+lQ&hRoxa$v=i-)5{GHI=B@ z!3{ikZxDf@KXIXk(xN^!4tz_rCp{)alD=Y2#Up6A2rO$7A8uqrsp55}ON(Qe?Kse9b zYY(%ot}rlXk(~M5_@j(Wu!s3&3FqLrokc#T5n;Q0^lPovpPpZam$K~fIM|dI<2xlY z>}NUlF;@e$O=#iyBYI3)>}#~3?1cMT3U)q~JPDaU<}6w8r1L zt|52gnZja`4jI=G|E9alggDHI+bjI&88Mq#eh_RD2YP-CW1i(6dle4b&rsUWaKB-6 z2W{RJRNy@=8a@KoPawhnF2u>0B~W}6(B&W#Pt*&&ve1Wy+Cm)~G1fUt=5J)zh? zaL1TA2?Lc5+`%-%+El~qRKvKwa=gW7v^4}akGTTCkh^1Yb4^-2`5u>EwF-zWaNqXu zpAT4^tbX|h=6^L+OOJeGtt(rZJxycJSYywe^}ama4MTX(FNDh?`=#@fR{S44g;7Sv zGzc(|t9@~U+OBjI&f%h%^|5h-Jg#&|$LJCGCV3i)09cP?Kk@7enYm!Ye_^bzO1p=5Gb z3!*iF0b?w8PeNrJuU&(L_>}%&2bTKi+XjGl%kEZ^D;EssDK_oXeBA5AGtPjqy<0vg zrr^6?-GM@MgP>z35bL`KIB>g+!^EP`;#}ym@it9!IR@05qh3x15>@;J%03L0-tP+k zmPkGfC!7jUEDA86h2gl{Bki=ps0>paAFTkOqUo8&92jL>t%JFvZE8O?_u@QS#K%BL z%J&*qKwSzB7Nnm1X+gqGfU0sE!cq-P6OgiUd+M@qxukF$@q(lG<`d^4^zV)qH>@@0 zjI`&4CLG=`9TMhoQgYyhlKYz@H;AmhF;cpgYCQku3KW??kCxO$e)rAZQVATruf$>! zOex9qyZbNfXh-Lh6rlU^CEoGBgX{ly1)Kh7xxSjAqlKZ7wW+Y3t+Szp?f=O7YZEhxGpwAM#$&83 zk|LF|u`deeto3U&g`7;gZ6=hHSn`r;JD}h;_twbR)BjnNqtBCGGJfG_vsOm zeu!GrMMP%ZA+jhwX3pGwgiIx|drq&>)BBYN+QniZS_XZ=i2XrSikhtm*~y{sK2OQ( z9-{En5pa$5AKC#nZ)Ly||0# zr5P)%b;-{(UazN(yT?o?N4t-Yqv@V68@)A#-Z(>9OqZY=S!_9MFZgJb z9(Vv$2RuA{IJ^v$8p4``a(`kt6Nn5TjjcKx5Co4wirO8^#9$~SDsn0s8-JY^B#D7L zq|!JRrADUOxZ4~~Wh-L;)dDU@y0yp<3Yly|T4Sj;tcLuvL}&0TZuf8{=*Ph&8XeF* z+yzl!FArq?EF}yb!}b>Bg)5ahPgEElL5^rT0eps<$E8^@FbD+r1TuFSIPs~6aqGwo zID!%uFER}ejT_f_vC)!s*oHz5v48Ff-*tFu%40dR=43M*N3)L3cu2J;uKv+{B@+ z*Hqp8@xw761vxe+#aa?8L$EwKOa6?UuLC?U))qgd#qs%*3%lk#=k#Z+-4@TYxVMB} z`4}Y04P77kw_Dc`eGb8MbUY1l9w3WTOnk`{R9uB`evEv}txkslD&iAf=Y_-EGy<(# zDTV033|jG^MGNXS>F|vIz@01BWK3#JGX{=f(j|P0v!+kJOI4Rh-^bycMxh`u2cz%+ z=r`yX*S*(Q3Wz721T%uRViVZu&m~*e}U<#!2_Nj;;H(z^g?zZBHWA|fpR$oZCP**MT&{cY#6jW1V97rBdD#^se*|yn+ zd~e6-&y~yd=Xb83=g`Oe?P55q8uu5zloqUfSsy;(Z!kX-YV5nd zzP7qW!kW(fu=&)^#&^$(1tjV)1<)9Yw;tk|)lMMnVYTz2Hgodn@Hb&o5uN7$i|R|nC{}v-m;5J^ z3Di$H2$3*y`-tjAhdlwo#dMORpM)<{J*3O^Vg(|l`}9VVFO~UMoX#)fTQvOK{eOaw zfKL@{E5A7vu#_05L^O+!;8i`NB&-$C+Y{cYFfXLx6BY_CIe|HT(X@EOz|~6iiZocV z^5@cU`LJPPe&4J_Cn89~J(>r$mwvoBrgvS1x7;qh*X-DH9LXG^tp~V#*9==vv4Mlu zs&|C{*e>AL6`=Ab1+)8EwDfr4342e1B@=oRclXi%*BYkF;0&@1ra zpcB=#$=xl9Dr#AnvVo2Y%IoAa6L!{`L}~v@J;2Cr%+|LSvEa6lxxG8<`6{3`+fQ0r z2wDgg13^3+H++z_vim>Fn_&5?6ZmO(vp(DZet8`KC)2E9WBotKGX+COCsW7&!8@BO z$bMqA=$TbE`4&x8Kf0(1c^heLq3kdOzsZXZmbz~iN^ho{jhiXoGlmL?$%n)E{T7h& zmW3DcS{d#%-0Ez(yL){=%#8v?v`467Sut?%GPM$hYgFb&7QtIyV`HXQ@qh}<+=zw( z#7QolwsA3@cW=?Y@qsK=8|8H|6yGV1Wf$%aX!snSSZeS_U{cOhuxYq&Tmn4nivC!!y!V=awm_HVdj)1@-MJ4>TYoycP@~kC<^F*$~ zL}8d7A&#ANjqOHWxsw;<;?1z@2(n^qDlXPY?<$zE>t?$*2q4bJ z(=)`L$l^PL@0mkv9kTCs%r3Lh5?if+p`2`TjWB z7)@zGdw^fKy(^3P@%Z8b`_T)ptGNP2G)}uppu&ab-feNIH%1~?WJ;d$;GT`!=|SQ z9AU-}s}F=zRNaOGD}YaOft6YZ%>cXWB?SxmzxgAXS8TEABYUsN-BE7ccGLD#i|X2M7!V(aazeVFS|{J~WZ@?UNx#1%tgMj1Zkr(W$J=_R(Mu*Wa`!AFL5 z9CD2gsUbo*-Ws9`FUHCX&mqDYI-8C}jpRV;8bE|WST)-&CwKoNG_#nrq%T3!l;n>m zz)#2%UCTRtBHrk4H4}QrgHAe?F)#we*v?BiUHUg1r_R!dch8r5uYHU+ zFu;m(8~ps0e@y_2L9o!4>U%4CGTMl~)CY7OVG5chA2H>kXUs;yQUZ(eSpdHJSw6n9 zUmuKsApfK~1vkcH*Zuzs+gmWavgUrKmFa-_?<_Xk|Fw1a{{T%bH*cN6n;e3qmY;kW z0$ASYPOXy3HcU=W?#T%FH%HY?aD-q&w(9C+kqw04V6R8TeoKDA zk-6#$!Ict{5JK!pQ@`Ajq)hSl^*erbH8(eNGylAv8uxpU`60gS-*8gfChc)hb4WUL zp!+56AyHo@?!i#|Chk#E`zG&UQGd`5+Mx!EBp%0eN$jD7c}Y4;Z)orFqdp~Z$?Scg z@}VNfa_Q_*g&C72OVXuq$r%+5z@Z{1aw+Z+qar69Cvj;SB@JkznoCzEbBP-Tiu4SK zg~_63kT*-{5Hr)&$?mye(cG|vrAT)Q?Ioes#Je{Q@Q`nj-^in0#ktoE z$b~&gdjHxp3xf;OCbLa?&K>|gw|UD&_4=fHs<QL%5*osW$1~PrE0T=MqO0~j$i}2Z%JJj$Wa(&7e zys~}s7`(!LtQfq~eGM4A;(bmSIt4pd7#q1eVo)`z*FT|Z)UTnT>{PBPpzPGHk>R)U zcdFpGig!MT>en&wTX{QvcRO?rT;T5jW$a=W8ROI`g+Zo3Lkh4WfEK&jRWJg(k10Ga zF)!JW5X=nt3bN8gF-T4raKx%a(Zl*SNCpyUB^qdCYH1EDz_G+m$I-&6cI%V+B-ej| z08HSkf$L;aK(9nBUSOqxAtc|S(NvX#HNNS$tq3t_t-_3wd zeJ3(Jx|I%nIQapFyx=&%9S$ZFouS5%!n8dEI1GRYhyoCUQ~?A4O4iK&YaWm?j+`-9 z)-C|ViZf-zm9dKta>DVQwr2*D15#LYrtG1>NdR@`O7qsZT{EmI865C8&?-O{z}dEA zoxQ_tREaS4XRa+jbC(HZ3E~DI;9xR8W3S;%nYV@-5`ftOoUI0kR+^-0tU5aZ+Yt^7 zT=Ek}t%FcU5mHz>RHxXO9Xm$M@w+ToIwVrqKxzh4sFW~aY^2GMMIm#IG-S$%(HHyh z)NvG(Q04)gr9<%5faHsPF&E~=pGth3~d}j(`Qz&6~Kix@ij+-SfhT7<{EcLh(~pB-O;HA~bT3+{M9wiTGE_rNUdf=WB!q z=AntsWpE$^0~2YI4XVS7bO>m(73v@wRPm?nQNVqG1Z*VJYZm+|0s{Z?aPeEF;!uGy zt}KiR11Q$PGaAA{-Kp<>iZ?)qm~i$z4-Bh81ez(+Ll2{f@Vh}>8it)Yp_}}07ad(P zJtc=acg;a^x@MJ)sy!BlMOHejO&UVdY~>q`oAB`Auo4wsy^WeZxDEUKW2M>Q_tQON z3`nDnaLO=w6#z?>>PCqXEC$9j8gi-o*C*G!XQja=lDxg2b((`#?!dD_bDAyskMh0u zeDOhs?0)c`F2;}V|I{2Zo;$*0Ztwq1Tw~pYhQ~OZIDjYz1XRwyZ|z9|n;TOVAK-2-=J zZ{ou3>R3CYxM{kF6Fl_K6>fxY%ENy|l&HSZ;+>;`ml5WH(zPYUEFmBfznZ6>$=$5Jrjez;&1%-H)(aE(Uj`LR8-5S8jX!+D&D{W z7Mif8*nImALQM>~f%`Rk(z;+_inM!AQcpV6ne{N-J5hD6rWo3r5MkLgam+>Ix?iQ- z=^UAwk$himC$;{$a0`PAJbdRBT4L{w{jAyMQ*t*Xpuw(eFWi=O2f#Wwo;j7k|X8Em9M_0R!$b8z4uA5AqsZU;D^6JD`$-jWk=QMDy|NEE}DPm93IqK{>GkH{NsseUhFuXTEn}5)TS1 z>n!FqyGN!w>sQROwKvuXo7~6U>M}j1AMVJUOaIMFU`C$&wLKYERMu!N`=K8nc1a|Q z91_^hD1s|?5zs($93h5$Wm*aF9wojAXFj3*(=nLZczXS@s*LBTCRQx7-XAKOgz(U8 z&(_N`0xQ6@0s5dXyQXakWuaQW&iWZD! zl_oiT!;^@#NDL^@IaRfIqGN_6uAI7~U@X+ydhVCgM(84`mCuKSJ7{ZR+w8Q<&gQRU z^oP&Mq7OvG;*7a8v+?Q_j=AgeL$d}EK1(X=#mR*K;ZBy*BbI{OtUs6}-`Yo?=&|Zc*fB z2Pr17O!@g;y6HU0d495lJI0R>3qcN1*{pHbF))o-(X1uD+&-_Y*Z2~4J*np`OHJ-l z#5mK`OHr3hT}a%YfAfPB$(Ju*jV}6S9IJMR++*kJQyDQ!M5dm(25DF#xaV7cR#;!l zud(}WH^xAf+<)IlxTCuRxOsTjtpxcFvgD|xRn7g|9e7Lfd42_b95lD^@g;m49y*`~ zd#@Ia%RUqrTo5wvWqJveWv1xB-h8xD-hhR(X=lx<|vvCdyC zK(5N7%vU}YR$In{!oPe-UAhse^0*AU@6(I%@w^PvxcvSfs!a1Y%R^#pjjDj^??x#M zlQ5cg1*Km(-+_Rn0p7xj+5Njjwz$FLbex;SmvVj?w$ZSy0@GH2K!fdrPf30pal~}wE z8*#}0WXBhnG0@~GVlK`M91dyY-H(twdc(7(Db4h6y%;-eL(#3jUr0!vGIT|!>bUH> z57ac6!(lgb4M5%x^cR1nFQ|O1#aXoj(ODs`@uj+TJMfC#(n$#x*3{LNlcN+i(d?1@ z5Vq;oK$rxPr5U+YaZ+?;6;C3HkOabb_mz%nS`8E}C{SPQfMzU@j;VD-jXph$Yg}aZ zEhlya8|bFrqU$U}G*zTIXuO#z1+E5gUoqi8MeKm)os=U?faYX*8M$#M33+|@o|?pPcaws@{BN{as{A?p4Fdxjm(jFfc#c)sC<1kZn-(D| z4eb1S`V@X;=g?2%ff?>-U`KA|k4c9(@r#MXMIM$!c$UW)Y)l7M&!|-krN; zXr=q#A%+09&m_fre^C}K5gNr$*ONzqiwl3Cmu*dgf7f3(dXC+{aAZY)SvyplgkNnE z3!;&tjCWK&<&*54Q7KZ35i5IrLH`j+QN(P0&MgK1@CAx~-gMLn5m`=j?(kO!;jIGt z0~wY8#vJ^pF0doc*Q(+W%}{LR2WN(^N`}RqV}cJkv*jqPRJY9Rj7Eo1gTf|$zJki1 z{auV>g2wNMqL`qYlf?`V%TGrOJyM> zLS7LQocQhn`S&?cv0ZIlPjhXl!1ff)G{cTWHX55s&AdJ9@SB4cmD8?r51D!4$%)Yd zN5bpksRKDCn#1-uf>I$`F}{plRp!%5@*L?nd%AOEm`Q)SRI77!P7Kx(hvHbURcSNO zCBwE_wj7oHK~vB&R`EQ^VOB!Rd$!FJ28Xyq^ojkJa>AD{cUn-qn-JDCvR#Tr2{VJ7 zzxE<9pGuU3(iFaq^OWEYSnJ^(t^TriPE!DB$ZraCuF6)WKOcT8A2BkUkfd>j$}e-L zhFm(Mc!4Ld-Z+JA2!Ak>_V4p3Bm3u|b_6nR=y$kvebtjTB=N{fO^HiyH;FC!+o?ZW z=(X8ry?X!3N-#lf>B3aX`5G`!GC2h8%!KhI_nnH@)Hf#oOV$LwrH|m+bCoA4K|7LZ z`5gnf!4LB$TlO|lae)bEM`ZSL=G$Z`(zkn@f;csq2odj!f@WR7XKbV~xLUfA@s(d% znMg|B7T8j#TR(T|f{*;TIy&B9I-k|^IiZzT!{Lm`G{4-)%FYS2xe>P$Oit}m%Sg%y3VJ{b#_Zo zYPbEwpcHyrv&Qa6C7>d=S^eYCzD=wO$H|cD;_mV${(U;9xhfX%ePRrP7&?Ch6JNy3 zbdI8xJxP)K={T6|xD>`}gk1e~OZ6>|n-Y=yVq>@EBg`V;1h+E2Y9OY$uOw7Ayei~- z2FUO5bu$`{9+uFZkP&zp9+vnmUzy3l8mCmLA?Q^a1@TBD*L%KCg>vcy-NYRb(D+1@ zCAa`}Qh0$@_@02V>rzVOe7>RLI@B_0Pu05C%0T$b0yj&rkcTtBW$u|uLu@X_kJNuL z62U3cJe%ZDJkz5nl1w>}w_}7jEOTX7YEY$0s>k{qYtuvRPxw#A;ysnYseuS3GTN^{ zmLlekEDtURcRI|&*-S8Zi|0yh^P;@vXcNWK*b^Uo!Fn^*&2O!2-7y4KOwVC1 zEt*@Yx;${NRkr`txu|XT6unzml(rGm^>L_Pk|_mB!xlAj8fG^q&J+>IrPd<27O| zXzxsC?1tA@PS)mGpPQ|$NN&PTkl;h`<~kAKsr9m?KRehnu=V0%|8|R*uJW%cQak+F zU5PhUbrs-e0~#ME{5{^WYhC`%`t@JY+R(R775xs?b8>@<74hDU@G7I+-w#aT@PXfW z;cd38lO4@dIvQHSi6sh`f_i0px)eE@5k9hU=YcGM-j`TptlxMH1yF7H6nz*>h}A)Z z_83u4^ifU(X*UvHkw$D$aVbE==_@V84$;;jTZ~uY1y}Ik!yjECf{foGsQCs6yxG4g zTa}W34^Q~}>25*ZjiJ?+IawSvw{6boZ6nfJSY~O7LJ$&dy*BaEDEvYCOJ?AthQ+IN zHA&H5AicS|-pO4^l!IX%a9(toX_Lj44BnAQ1yK4!pc$4wWI+?};}yhZ14{bRL~`IS z#5OZuZ)&l2q3fT-_BxTEdml^!@p>US`?*@fOs&E~ujWJ0th zw<$E>S`vnMN<%tIW4uaJIxI}D@{+n@JYRl!8>~$<#d_v$-6p#U!%RLhZe>Y */ -package object trees \ No newline at end of file +package object trees { + // Since we want tree nodes to be lightweight, we create one logger for all treenode instances. + protected val logger = Logger("catalyst.trees") +} \ No newline at end of file From c78b587a5fe1b369b5d55b60e22e9c6e22d2ac13 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 12 Dec 2013 00:10:20 -0800 Subject: [PATCH 201/778] casting. --- src/main/scala/expressions/Cast.scala | 9 +++++++++ src/main/scala/frontend/Hive.scala | 5 +++++ 2 files changed, 14 insertions(+) create mode 100644 src/main/scala/expressions/Cast.scala diff --git a/src/main/scala/expressions/Cast.scala b/src/main/scala/expressions/Cast.scala new file mode 100644 index 0000000000000..297933581671b --- /dev/null +++ b/src/main/scala/expressions/Cast.scala @@ -0,0 +1,9 @@ +package catalyst +package expressions + +import types.DataType + +case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { + def nullable = child.nullable + override def toString = s"CAST($child, $dataType)" +} \ No newline at end of file diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index d86df189982a6..6a14fb75d0dc0 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -480,6 +480,11 @@ object HiveQl { case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) => CountDistinct(args.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(SUM(), Nil) :: arg :: Nil) => Sum(nodeToExpr(arg)) + /* Casts */ + case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), IntegerType) + case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), FloatType) + case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) From dcc4fe1cb9076878e0b292ded52cb5f48b2ec246 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 12 Dec 2013 00:10:38 -0800 Subject: [PATCH 202/778] support for src1 test table. --- src/main/scala/shark2/TestShark.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index f18118aa23f44..f144de615e459 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -204,7 +204,10 @@ object TestShark extends Logging { val testTables = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)", - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' INTO TABLE src") + s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' INTO TABLE src"), + TestTable("src1", + "CREATE TABLE src1 (key INT, value STRING)", + s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv3.txt' INTO TABLE src") ) protected val testTableNames = testTables.map(_.name).toSet From e5c9d1ae1e8b5b50ce50e799406a51605a0bd06c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 12 Dec 2013 00:11:19 -0800 Subject: [PATCH 203/778] use nonEmpty --- src/test/scala/shark2/HiveComparisionTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/shark2/HiveComparisionTest.scala b/src/test/scala/shark2/HiveComparisionTest.scala index 1a05b7999c355..813957576880b 100644 --- a/src/test/scala/shark2/HiveComparisionTest.scala +++ b/src/test/scala/shark2/HiveComparisionTest.scala @@ -41,7 +41,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with val orderedAnswer = sharkQuery.parsed match { case _: Command => answer // Don't attempt to modify the result of Commands since they are run by hive. case _ => - val isOrdered = !sharkQuery.executedPlan.collect { case s: Sort => s}.isEmpty + val isOrdered = sharkQuery.executedPlan.collect { case s: Sort => s}.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. if(!isOrdered) answer.sorted else answer } From ad596d2d44d8efb76f001981ee00d1da715607bb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 12 Dec 2013 00:11:49 -0800 Subject: [PATCH 204/778] add sc to Union's otherCopyArgs --- src/main/scala/shark2/basicOperators.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/shark2/basicOperators.scala b/src/main/scala/shark2/basicOperators.scala index 4de1b98a59038..5013ca6b87507 100644 --- a/src/main/scala/shark2/basicOperators.scala +++ b/src/main/scala/shark2/basicOperators.scala @@ -39,6 +39,8 @@ case class Union(left: SharkPlan, right: SharkPlan)(@transient sc: SharkContext) def output = left.output // TODO: is it more efficient to union a bunch of rdds at once? should union be variadic? def execute() = sc.union(left.execute(), right.execute()) + + override def otherCopyArgs = sc :: Nil } case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { From 72e183b68e4ad706a7082762b465a8bc61f32e19 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 15:17:24 -0800 Subject: [PATCH 205/778] support for null values in tree node args. --- src/main/scala/trees/TreeNode.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index 7b1da18754c9b..c0b08cab7f709 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -113,6 +113,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case other => other } case nonChild: AnyRef => nonChild + case null => null }.toArray if(changed) makeCopy(newArgs) else this } From 7f4a1dc2c9d82235ba922ed1c793e2949673c49c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 15:17:53 -0800 Subject: [PATCH 206/778] add NoRelation logical operator --- src/main/scala/plans/logical/basicOperators.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index ef4079c6de323..b2e08e45e8657 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -59,4 +59,8 @@ case class StopAfter(limit: Expression, child: LogicalPlan) extends UnaryNode { case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { def output = child.output.map(_.withQualifiers(alias :: Nil)) def references = Set.empty -} \ No newline at end of file +} + +case object NoRelation extends LeafNode { + def output = Nil +} From c45b44085e992cd15536a2e247141eb89bd154bc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 15:19:11 -0800 Subject: [PATCH 207/778] support for is (not) null and boolean logic --- src/main/scala/expressions/Evaluate.scala | 5 +++++ src/main/scala/expressions/predicates.scala | 9 +++++++++ src/main/scala/frontend/Hive.scala | 7 +++++++ 3 files changed, 21 insertions(+) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index ffa34394a9e23..258f249824318 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -107,6 +107,11 @@ object Evaluate { case GreaterThanOrEqual(l, r) => n2(l, r, _.gteq(_, _)) case LessThan(l, r) => n2(l, r, _.lt(_, _)) case LessThanOrEqual(l, r) => n2(l, r, _.lteq(_, _)) + case IsNull(e) => eval(e) == null + case IsNotNull(e) => eval(e) != null + + /* Casts */ + case Cast(e, StringType) => eval(e).toString /* Boolean Logic */ case Not(c) => !eval(c).asInstanceOf[Boolean] diff --git a/src/main/scala/expressions/predicates.scala b/src/main/scala/expressions/predicates.scala index 11fd141c72f70..e48a18fbb0078 100644 --- a/src/main/scala/expressions/predicates.scala +++ b/src/main/scala/expressions/predicates.scala @@ -41,3 +41,12 @@ case class GreaterThan(left: Expression, right: Expression) extends BinaryPredic case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryPredicate { def symbol = ">=" } + +case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { + def references = child.references + def nullable = false +} +case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { + def references = child.references + def nullable = false +} diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 6a14fb75d0dc0..6fac159901976 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -500,6 +500,13 @@ object HiveQl { case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right)) + case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => IsNotNull(nodeToExpr(child)) + case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => IsNull(nodeToExpr(child)) + + /* Boolean Logic */ + case Token("AND", left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) + case Token("OR", left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right)) + case Token("NOT", child :: Nil) => Not(nodeToExpr(child)) /* Other functions */ case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand From 60ec19db1ecb7f7da713a8cb7b50a5a264a257a0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 15:21:24 -0800 Subject: [PATCH 208/778] initial support for udfs --- src/main/scala/analysis/Analyzer.scala | 17 +++++- src/main/scala/analysis/Catalog.scala | 2 +- .../scala/analysis/FunctionRegistry.scala | 12 ++++ src/main/scala/analysis/unresolved.scala | 14 ++++- .../scala/expressions/BoundAttribute.scala | 1 - src/main/scala/expressions/Evaluate.scala | 6 +- src/main/scala/expressions/Expression.scala | 6 ++ src/main/scala/expressions/functions.scala | 9 +++ .../scala/expressions/namedExpressions.scala | 13 ---- src/main/scala/frontend/Hive.scala | 8 +++ src/main/scala/shark2/FunctionRegistry.scala | 61 +++++++++++++++++++ src/main/scala/shark2/TestShark.scala | 2 +- src/main/scala/shark2/package.scala | 22 ++++++- src/test/scala/AnalysisSuite.scala | 2 +- src/test/scala/shark2/HiveQueryTests.scala | 4 ++ 15 files changed, 154 insertions(+), 25 deletions(-) create mode 100644 src/main/scala/analysis/FunctionRegistry.scala create mode 100644 src/main/scala/expressions/functions.scala create mode 100644 src/main/scala/shark2/FunctionRegistry.scala diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala index d2d21fdad20bd..28e540f3b0490 100644 --- a/src/main/scala/analysis/Analyzer.scala +++ b/src/main/scala/analysis/Analyzer.scala @@ -9,16 +9,17 @@ import rules._ * A trivial [[Analyzer]] with an [[EmptyCatalog]]. Used for testing when all relations are * already filled in and the analyser needs only to resolve attribute references. */ -object SimpleAnalyzer extends Analyzer(EmptyCatalog) +object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyRegistry) -class Analyzer(catalog: Catalog) extends RuleExecutor[LogicalPlan] { +class Analyzer(catalog: Catalog, registry: FunctionRegistry) extends RuleExecutor[LogicalPlan] { val fixedPoint = FixedPoint(100) val batches = Seq( Batch("Resolution", fixedPoint, ResolveReferences, ResolveRelations, - StarExpansion), + StarExpansion, + ResolveFunctions), Batch("Aggregation", Once, GlobalAggregates), Batch("Type Coersion", fixedPoint, @@ -49,6 +50,16 @@ class Analyzer(catalog: Catalog) extends RuleExecutor[LogicalPlan] { } } + object ResolveFunctions extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => + q transformExpressions { + case UnresolvedFunction(name, children) if children.map(_.resolved).reduceLeft(_&&_) => + registry.lookupFunction(name, children) + } + } + } + /** * Turns projections that contain aggregate expressions into aggregations. */ diff --git a/src/main/scala/analysis/Catalog.scala b/src/main/scala/analysis/Catalog.scala index 1842c90a2db1f..be931890661b7 100644 --- a/src/main/scala/analysis/Catalog.scala +++ b/src/main/scala/analysis/Catalog.scala @@ -6,7 +6,7 @@ import plans.logical.LogicalPlan /** * An interface for looking up relations by name. Used by an [[Analyzer]]. */ -abstract class Catalog { +abstract trait Catalog { def lookupRelation(name: String, alias: Option[String] = None): LogicalPlan } diff --git a/src/main/scala/analysis/FunctionRegistry.scala b/src/main/scala/analysis/FunctionRegistry.scala new file mode 100644 index 0000000000000..be166b0931353 --- /dev/null +++ b/src/main/scala/analysis/FunctionRegistry.scala @@ -0,0 +1,12 @@ +package catalyst +package analysis + +import expressions._ + +abstract trait FunctionRegistry { + def lookupFunction(name: String, children: Seq[Expression]): Expression +} + +object EmptyRegistry extends FunctionRegistry { + def lookupFunction(name: String, children: Seq[Expression]): Expression = ??? +} \ No newline at end of file diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala index 32f9b24f7ef47..fc135b7854db6 100644 --- a/src/main/scala/analysis/unresolved.scala +++ b/src/main/scala/analysis/unresolved.scala @@ -26,13 +26,23 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo def dataType = throw new UnresolvedException(this, "dataType") def nullable = throw new UnresolvedException(this, "nullable") def qualifiers = throw new UnresolvedException(this, "qualifiers") - def resolved = false + override lazy val resolved = false def withQualifiers(newQualifiers: Seq[String]) = this override def toString(): String = s"'$name" } +case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { + def exprId = throw new UnresolvedException(this, "exprId") + def dataType = throw new UnresolvedException(this, "dataType") + def nullable = throw new UnresolvedException(this, "nullable") + def qualifiers = throw new UnresolvedException(this, "qualifiers") + def references = children.flatMap(_.references).toSet + override lazy val resolved = false + override def toString = s"'$name(${children.mkString(",")})" +} + /** * Represents all of the input attributes to a given relational operator, for example in "SELECT * FROM ...". * @@ -44,7 +54,7 @@ case class Star(table: Option[String]) extends Attribute with trees.LeafNode[Exp def dataType = throw new UnresolvedException(this, "dataType") def nullable = throw new UnresolvedException(this, "nullable") def qualifiers = throw new UnresolvedException(this, "qualifiers") - def resolved = false + override lazy val resolved = false def withQualifiers(newQualifiers: Seq[String]) = this diff --git a/src/main/scala/expressions/BoundAttribute.scala b/src/main/scala/expressions/BoundAttribute.scala index 912f4bead0180..bc1d8e6b0532b 100644 --- a/src/main/scala/expressions/BoundAttribute.scala +++ b/src/main/scala/expressions/BoundAttribute.scala @@ -19,7 +19,6 @@ case class BoundReference(inputTuple: Int, ordinal: Int, baseReference: Attribut def exprId = baseReference.exprId def qualifiers = baseReference.qualifiers def name = baseReference.name - def resolved = true def withQualifiers(newQualifiers: Seq[String]) = BoundReference(inputTuple, ordinal, baseReference.withQualifiers(newQualifiers)) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index 258f249824318..f3b5fc934ff64 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -15,7 +15,6 @@ object Evaluate { * A set of helper functions that return the correct decendent of [[scala.math.Numeric]] type and do any casting * necessary of child evaluation. */ - @inline def n1(e: Expression, f: ((Numeric[Any], Any) => Any)): Any = e.dataType match { case IntegerType => @@ -124,7 +123,10 @@ object Evaluate { /* Functions */ case Rand => scala.util.Random.nextDouble - case other => throw new NotImplementedError(s"Evaluation for:\n $e") + /* UDFs */ + case implementedFunction: ImplementedUdf => implementedFunction.evaluate(implementedFunction.children.map(eval)) + + case other => throw new OptimizationException(other, "evaluation not implemented") } } } \ No newline at end of file diff --git a/src/main/scala/expressions/Expression.scala b/src/main/scala/expressions/Expression.scala index 7a0d7790c3923..f933779c62d13 100644 --- a/src/main/scala/expressions/Expression.scala +++ b/src/main/scala/expressions/Expression.scala @@ -10,6 +10,12 @@ abstract class Expression extends TreeNode[Expression] { def dataType: DataType def nullable: Boolean def references: Set[Attribute] + + /** + * Returns true if this expression and all its children have been resolved to a specific schema and false if it is + * still contains any unresolved placeholders. + */ + lazy val resolved: Boolean = children.map(_.resolved).reduceLeftOption(_&&_).getOrElse(true) } abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] { diff --git a/src/main/scala/expressions/functions.scala b/src/main/scala/expressions/functions.scala new file mode 100644 index 0000000000000..44b058f5eda2e --- /dev/null +++ b/src/main/scala/expressions/functions.scala @@ -0,0 +1,9 @@ +package catalyst +package expressions + +/** + * A UDF that has a native JVM implementation. + */ +abstract trait ImplementedUdf { + def evaluate(evaluatedChildren: Seq[Any]): Any +} \ No newline at end of file diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index 2964436153e2e..0faa5b15d6bdd 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -22,12 +22,6 @@ abstract class NamedExpression extends Expression { def name: String def exprId: ExprId def toAttribute: Attribute - - /** - * Returns true if this attribute has been resolved to a specific input value and false if it is still an unresolved - * placeholder - */ - def resolved: Boolean } abstract class Attribute extends NamedExpression { @@ -59,11 +53,6 @@ case class Alias(child: Expression, name: String) def nullable = child.nullable def references = child.references - // An alias is only resolved if all of its children are. - def resolved = try { child.dataType; true } catch { - case e: UnresolvedException[_] => false - } - def toAttribute = if(resolved) AttributeReference(name, child.dataType, child.nullable)(exprId) @@ -89,8 +78,6 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { - def resolved = true - override def equals(other: Any) = other match { case ar: AttributeReference => exprId == ar.exprId case _ => false diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 6fac159901976..9e62046ac7c8f 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -88,6 +88,7 @@ object HiveQl { "TOK_ALTERTABLE_UNARCHIVE", "TOK_ANALYZE", "TOK_CREATEDATABASE", + "TOK_CREATEFUNCTION", "TOK_CREATEINDEX", "TOK_CREATETABLE", "TOK_DROPDATABASE", @@ -323,6 +324,8 @@ object HiveQl { } protected def nodeToPlan(node: Node): LogicalPlan = node match { + // Just fake explain on create function... + case Token("TOK_EXPLAIN", Token("TOK_CREATEFUNCTION", _) :: Nil) => NoRelation case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. val Some(query) :: _ :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED"), explainArgs) @@ -511,7 +514,12 @@ object HiveQl { /* Other functions */ case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand + /* UDFs - Must be last otherwise will preempt built in functions */ + case Token("TOK_FUNCTION", Token(name, Nil) :: args) => + UnresolvedFunction(name, args.map(nodeToExpr)) + /* Literals */ + case Token("TOK_NULL", Nil) => Literal(null, IntegerType) // TODO: What type is null? case Token("TOK_STRINGLITERALSEQUENCE", strings) => Literal(strings.map(s => BaseSemanticAnalyzer.unescapeSQLString(s.asInstanceOf[ASTNode].getText)).mkString) diff --git a/src/main/scala/shark2/FunctionRegistry.scala b/src/main/scala/shark2/FunctionRegistry.scala new file mode 100644 index 0000000000000..69e7fd0557e54 --- /dev/null +++ b/src/main/scala/shark2/FunctionRegistry.scala @@ -0,0 +1,61 @@ +package catalyst +package shark2 + +import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} + +import expressions._ +import types._ +import org.apache.hadoop.io.{DoubleWritable, LongWritable, IntWritable, Text} + +import collection.JavaConversions._ + +object HiveFunctionRegistry extends analysis.FunctionRegistry { + def lookupFunction(name: String, children: Seq[Expression]): Expression = { + // We only look it up to see if it exists, but do not include it in the HiveUDF since it is not always serializable. + FunctionRegistry.getFunctionInfo(name) + // TODO: Check that the types match up. + HiveUdf(name, IntegerType, children) + } +} + +case class HiveUdf( + name: String, + dataType: DataType, + children: Seq[Expression]) extends Expression with ImplementedUdf { + def nullable = true + def references = children.flatMap(_.references).toSet + + // FunctionInfo is not serializable so we must look it up here again. + lazy val functionInfo = FunctionRegistry.getFunctionInfo(name) + lazy val function = functionInfo.getFunctionClass.newInstance.asInstanceOf[org.apache.hadoop.hive.ql.exec.UDF] + lazy val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) + + // TODO: Finish input output types. + def evaluate(evaluatedChildren: Seq[Any]): Any = { + // Wrap the function arguments in the expected types. + val args = evaluatedChildren.zip(method.getParameterTypes).map { + case (null, _) => null + case (arg: Double, argClass) if argClass.isAssignableFrom(classOf[DoubleWritable]) => + new DoubleWritable(arg) + case (arg: Double, argClass) if argClass.isAssignableFrom(classOf[org.apache.hadoop.hive.serde2.io.DoubleWritable]) => + new org.apache.hadoop.hive.serde2.io.DoubleWritable(arg) + case (arg: Int, argClass) if argClass.isAssignableFrom(classOf[org.apache.hadoop.hive.serde2.io.DoubleWritable]) => + new org.apache.hadoop.hive.serde2.io.DoubleWritable(arg) + case (arg: Int, argClass) if argClass.isAssignableFrom(classOf[IntWritable]) => + new IntWritable(arg) + case (arg, argClass) => + argClass.getConstructor(arg.getClass).newInstance(arg.asInstanceOf[AnyRef]).asInstanceOf[AnyRef] + }.toArray + + // Invoke the udf and unwrap the result. + method.invoke(function, args: _*) match { + case i: IntWritable => i.get + case t: Text => t.toString + case l: LongWritable => l.get + case null => null + case other => other + } + } + + override def toString = s"${functionInfo.getDisplayName}(${children.mkString(",")})" +} \ No newline at end of file diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index f144de615e459..c35d732c3d0e2 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -70,7 +70,7 @@ object TestShark extends Logging { /* A catalyst metadata catalog that points to the Shark/Hive Metastore. */ val catalog = new HiveMetastoreCatalog(SharkContext.hiveconf) /* An analyzer that uses the Shark/Hive metastore. */ - val analyze = new Analyzer(catalog) + val analyze = new Analyzer(catalog, HiveFunctionRegistry) /** Sets up the system initially or after a RESET command */ protected def configure() { diff --git a/src/main/scala/shark2/package.scala b/src/main/scala/shark2/package.scala index 52fe66d9c5015..3a3b2c1ab265f 100644 --- a/src/main/scala/shark2/package.scala +++ b/src/main/scala/shark2/package.scala @@ -1,5 +1,9 @@ package catalyst +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo + +import types._ + /** * A partial reimplementation of Shark, a Hive compatible SQL engine running on Spark, using Catalyst. * @@ -9,4 +13,20 @@ package catalyst * Currently functions that are not supported by this implementation are passed back to the original Shark * implementation for execution. */ -package object shark2 \ No newline at end of file +package object shark2 { + implicit class typeInfoConversions(dt: DataType) { + import org.apache.hadoop.hive.serde2.typeinfo._ + import TypeInfoFactory._ + + def toTypeInfo: TypeInfo = dt match { + case BooleanType => booleanTypeInfo + case ByteType => byteTypeInfo + case DoubleType => doubleTypeInfo + case FloatType => floatTypeInfo + case IntegerType => intTypeInfo + case LongType => longTypeInfo + case ShortType => shortTypeInfo + case StringType => stringTypeInfo + } + } +} \ No newline at end of file diff --git a/src/test/scala/AnalysisSuite.scala b/src/test/scala/AnalysisSuite.scala index be85a24d90a73..464a344ea93e6 100644 --- a/src/test/scala/AnalysisSuite.scala +++ b/src/test/scala/AnalysisSuite.scala @@ -11,7 +11,7 @@ import types._ import dsl._ class AnalysisSuite extends FunSuite { - val analyze = new Analyzer(EmptyCatalog) + val analyze = SimpleAnalyzer val testRelation = LocalRelation('a.int) diff --git a/src/test/scala/shark2/HiveQueryTests.scala b/src/test/scala/shark2/HiveQueryTests.scala index 9fbc3065a188a..e7484927f4617 100644 --- a/src/test/scala/shark2/HiveQueryTests.scala +++ b/src/test/scala/shark2/HiveQueryTests.scala @@ -37,4 +37,8 @@ class HiveQueryTests extends HiveComaparisionTest { createQueryTest("small.cartesian", "SELECT a.key, b.key FROM (SELECT key FROM src WHERE key < 1) a JOIN (SELECT key FROM src WHERE key = 2) b") + + createQueryTest("length.udf", + "SELECT length(\"test\") FROM src LIMIT 1") + } \ No newline at end of file From a5c0a1b20c10dd16cd2d2cf19d282ac0837f1ca4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 15:21:59 -0800 Subject: [PATCH 209/778] more test harness improvements: * regex whitelist * side by side answer comparison (still needs formatting work) --- src/main/scala/util/package.scala | 10 +++++ .../scala/shark2/HiveComparisionTest.scala | 40 +++++++++++-------- 2 files changed, 34 insertions(+), 16 deletions(-) diff --git a/src/main/scala/util/package.scala b/src/main/scala/util/package.scala index cd2f035d89450..9fec9ef2deb9c 100644 --- a/src/main/scala/util/package.scala +++ b/src/main/scala/util/package.scala @@ -40,4 +40,14 @@ package object util { out.close() file } + + def sideBySide(left: Seq[String], right: Seq[String]): Seq[String] = { + val maxLeftSize = left.map(_.size).max + val leftPadded = left ++ Seq.fill(if(left.size < right.size) right.size - left.size else 0)("") + val rightPadded = right ++ Seq.fill(if(right.size < left.size) left.size - right.size else 0)("") + + leftPadded.zip(rightPadded).map { + case (l,r) => (if(l == r) " " else "*") + l + (" " * ((maxLeftSize - left.size) + 3)) + r + } + } } \ No newline at end of file diff --git a/src/test/scala/shark2/HiveComparisionTest.scala b/src/test/scala/shark2/HiveComparisionTest.scala index 813957576880b..479d5019eaef8 100644 --- a/src/test/scala/shark2/HiveComparisionTest.scala +++ b/src/test/scala/shark2/HiveComparisionTest.scala @@ -6,16 +6,18 @@ import shark.{SharkContext, SharkEnv} import java.io._ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} -import catalyst.frontend.hive.{ExplainCommand, Command} +import frontend.hive.{ExplainCommand, Command} import util._ +import collection.JavaConversions._ + /** * Allows the creations of tests that execute the same query against both hive * and catalyst, comparing the results. * * The "golden" results from Hive are cached in [[answerCache]] to speed up testing. */ -abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with GivenWhenThen { +abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with GivenWhenThen with Logging { val testShark = TestShark protected val targetDir = new File("target") @@ -57,10 +59,12 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with def createQueryTest(testCaseName: String, sql: String) = { test(testCaseName) { - println( - s"""============================= - |===HIVE TEST: $testCaseName=== - |=============================""".stripMargin) + logger.info( + s""" + |============================= + |HIVE TEST: $testCaseName + |============================= + """.stripMargin) val queryList = sql.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq try { @@ -72,7 +76,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with val cachedAnswerFile = new File(answerCache, cachedAnswerName) if(cachedAnswerFile.exists) { - println(s"Using cached answer for: $queryString") + logger.info(s"Using cached answer for: $queryString") val cachedAnswer = fileToString(cachedAnswerFile) if(cachedAnswer == "") Nil @@ -116,26 +120,30 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with } }.toSeq - testShark.reset() (queryList, hiveResults, catalystResults).zipped.foreach { case (query, hive, (sharkQuery, catalyst)) => // Check that the results match unless its an EXPLAIN query. - if((!sharkQuery.parsed.isInstanceOf[ExplainCommand]) && prepareAnswer(sharkQuery,hive) != catalyst) { + val preparedHive = prepareAnswer(sharkQuery,hive) + + if((!sharkQuery.parsed.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) { + + val hivePrintOut = s"== HIVE - ${hive.size} row(s) ==" +: preparedHive + val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst + + val resultComparision = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") + fail( s""" |Results do not match for query: - |$sharkQuery\n${sharkQuery.analyzed.output.mkString("\t")} - |== HIVE - ${hive.size} row(s) == - |${hive.mkString("\n")} - |== CATALYST - ${catalyst.size} row(s) == - |${catalyst.mkString("\n")} + |$sharkQuery\n${sharkQuery.analyzed.output.map(_.name).mkString("\t")} + |$resultComparision """.stripMargin) } } - passedList.println(testCaseName) + passedList.println(s""""$testCaseName",""") } catch { case tf: org.scalatest.exceptions.TestFailedException => throw tf case originalException: Exception => @@ -147,7 +155,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with testShark.runSqlHive("SELECT key FROM src") } catch { case e: Exception => - println(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") + logger.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") // The testing setup traps exits so wait here for a long time so the developer can see when things started // to go wrong. Thread.sleep(1000000) From ad1f3b43420a6a12c43bb368681df0614f011a76 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 15:22:12 -0800 Subject: [PATCH 210/778] toString for null literals --- src/main/scala/expressions/literals.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/expressions/literals.scala b/src/main/scala/expressions/literals.scala index b87ea71042d65..35012adbf4ba1 100644 --- a/src/main/scala/expressions/literals.scala +++ b/src/main/scala/expressions/literals.scala @@ -29,5 +29,5 @@ case class Literal(value: Any, dataType: DataType) extends LeafExpression { def nullable = false def references = Set.empty - override def toString = value.toString + override def toString = if(value != null) value.toString else "null" } \ No newline at end of file From 8b2a2ee3e7063c894eeb3b1b769a6341ec4b3834 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 15:22:25 -0800 Subject: [PATCH 211/778] more tests passing! --- src/test/scala/shark2/HiveCompatability.scala | 59 +++++++++++++++++-- 1 file changed, 55 insertions(+), 4 deletions(-) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index 0e1c556d5d8cd..cb80a9cf93925 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -22,7 +22,10 @@ class HiveCompatability extends HiveComaparisionTest { // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 - "ops_comparison" + "ops_comparison", + + // The skewjoin test seems to never complete on hive... + "skewjoin" ) /** @@ -53,17 +56,30 @@ class HiveCompatability extends HiveComaparisionTest { "input0", "input11", "input11_limit", + "input4_limit", "insert1", - "join_view", "join0", + "join1", + "join10", + "join15", + "join19", + "join22", + "join3", + "join_casesensitive", + "join_view", "literal_double", "literal_ints", "literal_string", + "mergejoins", "nestedvirtual", "noalias_subq1", "nullgroup", "nullgroup2", "nullinput", + "ppd_gby_join", + "ppd_random", + "ppd_udf_col", + "progress_1", "quote2", "rename_column", "select_as_omitted", @@ -71,7 +87,9 @@ class HiveCompatability extends HiveComaparisionTest { "show_describe_func_quotes", "show_functions", "tablename_with_select", + "udf9", "udf_add", + "udf_ascii", "udf_avg", "udf_bigint", "udf_bitwise_and", @@ -91,6 +109,7 @@ class HiveCompatability extends HiveComaparisionTest { "udf_float", "udf_floor", "udf_from_unixtime", + "udf_hour", "udf_index", "udf_int", "udf_isnotnull", @@ -100,20 +119,29 @@ class HiveCompatability extends HiveComaparisionTest { "udf_log", "udf_log10", "udf_log2", + "udf_lower", + "udf_lpad", "udf_ltrim", + "udf_minute", "udf_modulo", "udf_month", "udf_not", "udf_or", + "udf_parse_url", + "udf_pmod", "udf_positive", "udf_pow", "udf_power", "udf_rand", "udf_regexp_extract", "udf_regexp_replace", + "udf_repeat", "udf_rlike", + "udf_rpad", "udf_rtrim", + "udf_second", "udf_smallint", + "udf_space", "udf_sqrt", "udf_std", "udf_stddev", @@ -127,11 +155,34 @@ class HiveCompatability extends HiveComaparisionTest { "udf_to_date", "udf_trim", "udf_ucase", + "udf_unhex", + "udf_unix_timestamp", "udf_upper", "udf_var_pop", "udf_var_samp", "udf_variance", - "union16" + "udf_weekofyear", + "udf_xpath_boolean", + "udf_xpath_double", + "udf_xpath_float", + "udf_xpath_int", + "udf_xpath_long", + "udf_xpath_short", + "union10", + "union11", + "union13", + "union15", + "union16", + "union2", + "union20", + "union28", + "union29", + "union30", + "union4", + "union5", + "union7", + "union8", + "union9" ) // TODO: bundle in jar files... get from classpath @@ -147,7 +198,7 @@ class HiveCompatability extends HiveComaparisionTest { val testCaseName = testCase.getName.stripSuffix(".q") if(blackList contains testCaseName) { // Do nothing - } else if(realWhiteList.contains(testCaseName) || runAll) { + } else if(realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCase) createQueryTest(testCaseName, queriesString) From 4d5eba7411a1f140c0299e23107e49b826386e7b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 16:46:11 -0800 Subject: [PATCH 212/778] add more dsl for expression arithmetic and boolean logic --- src/main/scala/dsl.scala | 6 ++++++ src/main/scala/expressions/Evaluate.scala | 2 ++ 2 files changed, 8 insertions(+) diff --git a/src/main/scala/dsl.scala b/src/main/scala/dsl.scala index 783b9253d011a..8ff22fea1d98b 100644 --- a/src/main/scala/dsl.scala +++ b/src/main/scala/dsl.scala @@ -36,6 +36,12 @@ package object dsl { def expr: Expression def +(other: Expression) = Add(expr, other) + def -(other: Expression) = Subtract(expr, other) + def *(other: Expression) = Multiply(expr, other) + def /(other: Expression) = Divide(expr, other) + + def &&(other: Expression) = And(expr, other) + def ||(other: Expression) = Or(expr, other) def <(other: Expression) = LessThan(expr, other) def <=(other: Expression) = LessThanOrEqual(expr, other) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index f3b5fc934ff64..a761ca825b1ef 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -114,6 +114,8 @@ object Evaluate { /* Boolean Logic */ case Not(c) => !eval(c).asInstanceOf[Boolean] + case And(l,r) => eval(l).asInstanceOf[Boolean] && eval(l).asInstanceOf[Boolean] + case Or(l,r) => eval(l).asInstanceOf[Boolean] || eval(l).asInstanceOf[Boolean] /* References to input tuples */ case br @ BoundReference(inputTuple, ordinal, _) => try input(inputTuple)(ordinal) catch { From 69024909787b9502c425e132d2f733606f6e6b99 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 16:46:37 -0800 Subject: [PATCH 213/778] fix boolean logic evaluation --- src/main/scala/expressions/Evaluate.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index a761ca825b1ef..7194acaabf877 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -7,7 +7,7 @@ import types._ /** * Performs evaluation of an expression tree, given a set of input tuples. */ -object Evaluate { +object Evaluate extends Logging { def apply(e: Expression, input: Seq[Seq[Any]]): Any = attachTree(e, "Expression Evaluation Failed") { def eval(e: Expression) = Evaluate(e, input) @@ -114,8 +114,8 @@ object Evaluate { /* Boolean Logic */ case Not(c) => !eval(c).asInstanceOf[Boolean] - case And(l,r) => eval(l).asInstanceOf[Boolean] && eval(l).asInstanceOf[Boolean] - case Or(l,r) => eval(l).asInstanceOf[Boolean] || eval(l).asInstanceOf[Boolean] + case And(l,r) => eval(l).asInstanceOf[Boolean] && eval(r).asInstanceOf[Boolean] + case Or(l,r) => eval(l).asInstanceOf[Boolean] || eval(r).asInstanceOf[Boolean] /* References to input tuples */ case br @ BoundReference(inputTuple, ordinal, _) => try input(inputTuple)(ordinal) catch { @@ -130,5 +130,8 @@ object Evaluate { case other => throw new OptimizationException(other, "evaluation not implemented") } + + logger.debug(s"Evaluated $e => $result") + result } } \ No newline at end of file From 3ea9b008be484e9dc6dead88d1ea656891b87190 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 16:47:18 -0800 Subject: [PATCH 214/778] don't use simpleString if there are no new lines. --- src/main/scala/trees/TreeNode.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index c0b08cab7f709..94291f95cf028 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -151,7 +151,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** Returns a string representing the arguments to this node, minus any children */ def argString: String = productIterator.flatMap { case tn: TreeNode[_] if children contains tn => Nil - case tn: TreeNode[_] => s"(${tn.simpleString})" :: Nil + case tn: TreeNode[_] if(tn.toString contains "\n") => s"(${tn.simpleString})" :: Nil case seq: Seq[_] => seq.mkString("{", ",", "}") :: Nil case other => other :: Nil }.mkString(", ") From 7fcf480fd82182ca9a256dda52cf6c9e963f023d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 16:48:02 -0800 Subject: [PATCH 215/778] test for and logic --- .../scala/ExpressionEvaluationSuite.scala | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/src/test/scala/ExpressionEvaluationSuite.scala b/src/test/scala/ExpressionEvaluationSuite.scala index 1495a610ab8dc..db182b6bbd66e 100644 --- a/src/test/scala/ExpressionEvaluationSuite.scala +++ b/src/test/scala/ExpressionEvaluationSuite.scala @@ -3,7 +3,7 @@ package expressions import org.scalatest.FunSuite -import types.IntegerType +import types._ import expressions._ import dsl._ @@ -13,4 +13,21 @@ class ExpressionEvaluationSuite extends FunSuite { test("literals") { assert(Evaluate(Literal(1) + Literal(1), Nil) === 2) } + + test("boolean logic") { + val andTruthTable = + (true, true, true) :: + (true, false, false) :: + (false, true, false) :: + (false, false, false) :: Nil + + andTruthTable.foreach { + case (l,r,answer) => + val result = Evaluate(Literal(l, BooleanType) && Literal(r, BooleanType), Nil) + if(result != answer) + fail(s"$l && $r != $result") + } + + + } } \ No newline at end of file From 7aee69c241c0fd4264111e70904b9cf57847ca8e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 16:48:21 -0800 Subject: [PATCH 216/778] parsing for joins, boolean logic --- src/main/scala/frontend/Hive.scala | 57 +++++++++++++++++------------- 1 file changed, 33 insertions(+), 24 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 9e62046ac7c8f..9d3c233fb3ead 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -187,6 +187,9 @@ object HiveQl { } } + class ParseException(sql: String, cause: Throwable) + extends Exception(s"Failed to parse: $sql", cause) + /** * Returns the AST for the given SQL string. */ @@ -195,8 +198,7 @@ object HiveQl { ParseUtils.findRootNonNullToken( (new ParseDriver()).parse(sql)) } catch { - case pe: org.apache.hadoop.hive.ql.parse.ParseException => - throw new RuntimeException(s"Failed to parse sql: '$sql'", pe) + case e: Exception => throw new ParseException(sql, e) } } @@ -348,7 +350,7 @@ object HiveQl { Filter(nodeToExpr(whereExpr), relations) }.getOrElse(relations) - val selectExpressions = nameExpressions(selectClause.getChildren.map(selExprNodeToExpr)) + val selectExpressions = nameExpressions(selectClause.getChildren.flatMap(selExprNodeToExpr)) val withProject = groupByClause match { case Some(groupBy) => Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withWhere) @@ -370,6 +372,7 @@ object HiveQl { throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } + val allJoinTokens = "(TOK_.*JOIN)".r def nodeToRelation(node: Node): LogicalPlan = node match { case Token("TOK_SUBQUERY", query :: Token(alias, Nil) :: Nil) => @@ -390,24 +393,20 @@ object HiveQl { val tableName = tableNameParts.map { case Token(part, Nil) => part }.mkString(".") UnresolvedRelation(tableName, Some(alias)) - /* Join no condition */ - case Token("TOK_JOIN", + case Token(allJoinTokens(joinToken), relation1 :: - relation2 :: Nil) => + relation2 :: other) => + assert(other.size <= 1, "Unhandled join child") + val joinType = joinToken match { + case "TOK_JOIN" => Inner + case "TOK_RIGHTOUTERJOIN" => RightOuter + case "TOK_LEFTOUTERJOIN" => LeftOuter + case "TOK_FULLOUTERJOIN" => FullOuter + } Join(nodeToRelation(relation1), nodeToRelation(relation2), - Inner, - None) - - /* Join with condition */ - case Token("TOK_JOIN", - relation1 :: - relation2 :: - condition :: Nil) => - Join(nodeToRelation(relation1), - nodeToRelation(relation2), - Inner, - Some(nodeToExpr(condition))) + joinType, + other.headOption.map(nodeToExpr)) case a: ASTNode => throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") @@ -435,13 +434,20 @@ object HiveQl { throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } - protected def selExprNodeToExpr(node: Node): Expression = node match { + protected def selExprNodeToExpr(node: Node): Option[Expression] = node match { case Token("TOK_SELEXPR", e :: Nil) => - nodeToExpr(e) + Some(nodeToExpr(e)) + case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) => - Alias(nodeToExpr(e), alias)() + Some(Alias(nodeToExpr(e), alias)()) + + /* Hints are ignored */ + case Token("TOK_HINTLIST", _) => None + + case a: ASTNode => + throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } @@ -461,6 +467,9 @@ object HiveQl { val AVG = "(?i)AVG".r val SUM = "(?i)SUM".r val RAND = "(?i)RAND".r + val AND = "(?i)AND".r + val OR = "(?i)OR".r + val NOT = "(?i)Not".r protected def nodeToExpr(node: Node): Expression = node match { /* Attribute References */ @@ -507,9 +516,9 @@ object HiveQl { case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => IsNull(nodeToExpr(child)) /* Boolean Logic */ - case Token("AND", left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) - case Token("OR", left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right)) - case Token("NOT", child :: Nil) => Not(nodeToExpr(child)) + case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) + case Token(OR(), left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right)) + case Token(NOT(), child :: Nil) => Not(nodeToExpr(child)) /* Other functions */ case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand From 8a9d21c18c0f5e83bfea894dd2ba63fc1a0812f0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 16:49:11 -0800 Subject: [PATCH 217/778] fix evaluation --- src/main/scala/expressions/Evaluate.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index 7194acaabf877..a79d70cfaa611 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -80,7 +80,7 @@ object Evaluate extends Logging { } } - e match { + val result = e match { case Literal(v, _) => v /* Alias operations do not effect evaluation */ From 480ade59ad1349274d6a0fd03b45a16f47c43b2d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 21:33:13 -0800 Subject: [PATCH 218/778] don't use partial cached results. --- .../scala/shark2/HiveComparisionTest.scala | 63 ++++++++++++------- 1 file changed, 39 insertions(+), 24 deletions(-) diff --git a/src/test/scala/shark2/HiveComparisionTest.scala b/src/test/scala/shark2/HiveComparisionTest.scala index 479d5019eaef8..0d5210a2f9490 100644 --- a/src/test/scala/shark2/HiveComparisionTest.scala +++ b/src/test/scala/shark2/HiveComparisionTest.scala @@ -59,7 +59,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with def createQueryTest(testCaseName: String, sql: String) = { test(testCaseName) { - logger.info( + logger.error( s""" |============================= |HIVE TEST: $testCaseName @@ -70,33 +70,50 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with try { testShark.reset() - val hiveResults: Seq[Seq[String]] = queryList.zipWithIndex.map { + val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" - val cachedAnswerFile = new File(answerCache, cachedAnswerName) + new File(answerCache, cachedAnswerName) + } - if(cachedAnswerFile.exists) { - logger.info(s"Using cached answer for: $queryString") - val cachedAnswer = fileToString(cachedAnswerFile) - if(cachedAnswer == "") + val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => + if(cachedAnswerFile.exists) { + val cachedString = fileToString(cachedAnswerFile) + val cachedAnswer = + if(cachedString == "") Nil else - cachedAnswer.split("\n").toSeq - } else { - // Analyze the query with catalyst to ensure test tables are loaded. - val sharkQuery = (new testShark.SharkSqlQuery(queryString)) - val answer = sharkQuery.analyzed match { - case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. - case _ => testShark.runSqlHive(queryString) - } - - stringToFile(cachedAnswerFile, answer.mkString("\n")) - - answer - } - }.toSeq + cachedString.split("\n").toSeq + Some(cachedAnswer) + } else { + logger.debug(s"File $cachedAnswerFile not found") + None + } + } - testShark.reset() + val hiveResults: Seq[Seq[String]] = + if(hiveCachedResults.size == queryList.size) { + logger.warn(s"Using answer cache for test: $testCaseName") + hiveCachedResults + } else { + val computedResults = queryList.zip(hiveCacheFiles).zipWithIndex.map { + case ((queryString, cachedAnswerFile), i)=> + logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") + // Analyze the query with catalyst to ensure test tables are loaded. + val sharkQuery = (new testShark.SharkSqlQuery(queryString)) + val answer = sharkQuery.analyzed match { + case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. + case _ => testShark.runSqlHive(queryString) + } + + stringToFile(cachedAnswerFile, answer.mkString("\n")) + + answer + }.toSeq + testShark.reset() + + computedResults + } // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => @@ -120,8 +137,6 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with } }.toSeq - testShark.reset() - (queryList, hiveResults, catalystResults).zipped.foreach { case (query, hive, (sharkQuery, catalyst)) => // Check that the results match unless its an EXPLAIN query. From 175c43eac364d9bc4637c1a615b67e5395bd9f9c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 21:34:05 -0800 Subject: [PATCH 219/778] better errors for parse exceptions --- src/main/scala/frontend/Hive.scala | 50 +++++++++++++++--------------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 9d3c233fb3ead..c11172b0633c7 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -194,34 +194,34 @@ object HiveQl { * Returns the AST for the given SQL string. */ def getAst(sql: String): ASTNode = { - try { - ParseUtils.findRootNonNullToken( - (new ParseDriver()).parse(sql)) - } catch { - case e: Exception => throw new ParseException(sql, e) - } + ParseUtils.findRootNonNullToken( + (new ParseDriver()).parse(sql)) } def parseSql(sql: String): LogicalPlan = { - if(sql.toLowerCase.startsWith("set")) - ConfigurationAssignment(sql) - else if(sql.toLowerCase.startsWith("add jar")) - AddJar(sql.drop(8)) - else if(sql.toLowerCase.startsWith("add file")) - AddFile(sql.drop(9)) - else if(sql.startsWith("dfs")) - DfsCommand(sql) - else if(sql.startsWith("source")) - SourceCommand(sql.split(" ").toSeq match { case Seq("source", filePath) => filePath }) - else if(sql.startsWith("!")) - ShellCommand(sql.drop(1)) - else { - val tree = getAst(sql) - - if(nativeCommands contains tree.getText) - NativeCommand(sql) - else - nodeToPlan(tree) + try { + if(sql.toLowerCase.startsWith("set")) + ConfigurationAssignment(sql) + else if(sql.toLowerCase.startsWith("add jar")) + AddJar(sql.drop(8)) + else if(sql.toLowerCase.startsWith("add file")) + AddFile(sql.drop(9)) + else if(sql.startsWith("dfs")) + DfsCommand(sql) + else if(sql.startsWith("source")) + SourceCommand(sql.split(" ").toSeq match { case Seq("source", filePath) => filePath }) + else if(sql.startsWith("!")) + ShellCommand(sql.drop(1)) + else { + val tree = getAst(sql) + + if(nativeCommands contains tree.getText) + NativeCommand(sql) + else + nodeToPlan(tree) + } + } catch { + case e: Exception => throw new ParseException(sql, e) } } From 5f14c35143dd8c565227b93eba3ade5b3ef99344 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 21:34:22 -0800 Subject: [PATCH 220/778] more test tables supported --- src/main/scala/shark2/TestShark.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index c35d732c3d0e2..941c399ee9c89 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -207,7 +207,13 @@ object TestShark extends Logging { s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' INTO TABLE src"), TestTable("src1", "CREATE TABLE src1 (key INT, value STRING)", - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv3.txt' INTO TABLE src") + s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv3.txt' INTO TABLE src1"), + TestTable("dest1", + "CREATE TABLE src (key INT, value STRING)"), + TestTable("dest2", + "CREATE TABLE src (key INT, value STRING)"), + TestTable("dest3", + "CREATE TABLE src (key INT, value STRING)") ) protected val testTableNames = testTables.map(_.name).toSet From 956e760d2bffff953e52b82e10307e9f155c3442 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 21:34:35 -0800 Subject: [PATCH 221/778] extended explain --- src/main/scala/frontend/Hive.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index c11172b0633c7..7ed6dc3b6f1a5 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -330,7 +330,8 @@ object HiveQl { case Token("TOK_EXPLAIN", Token("TOK_CREATEFUNCTION", _) :: Nil) => NoRelation case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. - val Some(query) :: _ :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED"), explainArgs) + val Some(query) :: _ :: _ :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) + // TODO: support EXTENDED? ExplainCommand(nodeToPlan(query)) case Token("TOK_QUERY", Token("TOK_FROM", fromClause :: Nil) :: From a1245f963aba78adf9f180da7696d90464e256e9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 21:34:44 -0800 Subject: [PATCH 222/778] more tests passing --- src/test/scala/shark2/HiveCompatability.scala | 39 +++++++++++++++---- 1 file changed, 31 insertions(+), 8 deletions(-) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index cb80a9cf93925..08b6164db22f9 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -25,7 +25,13 @@ class HiveCompatability extends HiveComaparisionTest { "ops_comparison", // The skewjoin test seems to never complete on hive... - "skewjoin" + "skewjoin", + + // This test fails and and exits the JVM. + "auto_join18_multi_distinct", + + // Uses a serde that isn't on the classpath... breaks other tests. + "bucketizedhiveinputformat" ) /** @@ -34,9 +40,13 @@ class HiveCompatability extends HiveComaparisionTest { */ val whiteList = Seq( "add_part_exist", + "auto_join23", + "auto_join26", + "auto_join28", "avro_change_schema", "avro_schema_error_message", "avro_schema_literal", + "bucketmapjoin6", "count", "ct_case_insensitive", "database_properties", @@ -50,21 +60,30 @@ class HiveCompatability extends HiveComaparisionTest { "drop_partitions_filter3", "drop_table", "drop_view", - "groupby4_map", - "groupby4_map_skew", "index_auth", + "index_auto_self_join", + "innerjoin", "input0", - "input11", - "input11_limit", "input4_limit", "insert1", "join0", "join1", "join10", + "join11", + "join12", + "join13", "join15", + "join16", "join19", "join22", - "join3", + "join23", + "join25", + "join29", + "join30", + "join31", + "join34", + "join36", + "join37", "join_casesensitive", "join_view", "literal_double", @@ -72,12 +91,15 @@ class HiveCompatability extends HiveComaparisionTest { "literal_string", "mergejoins", "nestedvirtual", + "no_hooks", "noalias_subq1", "nullgroup", "nullgroup2", "nullinput", "ppd_gby_join", + "ppd_outer_join5", "ppd_random", + "ppd_repeated_alias", "ppd_udf_col", "progress_1", "quote2", @@ -86,6 +108,7 @@ class HiveCompatability extends HiveComaparisionTest { "set_variable_sub", "show_describe_func_quotes", "show_functions", + "smb_mapjoin_10", "tablename_with_select", "udf9", "udf_add", @@ -115,6 +138,7 @@ class HiveCompatability extends HiveComaparisionTest { "udf_isnotnull", "udf_isnull", "udf_lcase", + "udf_length", "udf_ln", "udf_log", "udf_log10", @@ -171,7 +195,7 @@ class HiveCompatability extends HiveComaparisionTest { "union10", "union11", "union13", - "union15", + "union14", "union16", "union2", "union20", @@ -180,7 +204,6 @@ class HiveCompatability extends HiveComaparisionTest { "union30", "union4", "union5", - "union7", "union8", "union9" ) From 6916c6330ed4706de645752adf8f3425957e5a1b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 22:47:01 -0800 Subject: [PATCH 223/778] Reading from partitioned hive tables. --- src/main/scala/shark2/MetastoreCatalog.scala | 18 +++++++++--- src/main/scala/shark2/TestShark.scala | 29 +++++++++++++------ src/main/scala/shark2/hiveOperators.scala | 24 +++++++++++---- .../scala/shark2/HiveComparisionTest.scala | 1 + src/test/scala/shark2/HiveQueryTests.scala | 3 ++ 5 files changed, 56 insertions(+), 19 deletions(-) diff --git a/src/main/scala/shark2/MetastoreCatalog.scala b/src/main/scala/shark2/MetastoreCatalog.scala index 0483cef9cab05..5002de667267e 100644 --- a/src/main/scala/shark2/MetastoreCatalog.scala +++ b/src/main/scala/shark2/MetastoreCatalog.scala @@ -2,7 +2,7 @@ package catalyst package shark2 import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.api.Table +import org.apache.hadoop.hive.metastore.api.{Partition, Table} import org.apache.hadoop.hive.metastore.HiveMetaStoreClient import analysis.Catalog @@ -20,7 +20,15 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { case Array(tableOnly) => ("default", tableOnly) case Array(db, table) => (db, table) } - MetastoreRelation(databaseName, tableName, alias)(client.getTable(databaseName, tableName)) + val table = client.getTable(databaseName, tableName) + val hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) + val partitions = + if(hiveQlTable.isPartitioned) + client.listPartitions(databaseName, tableName, 255).toSeq + else + Nil + + MetastoreRelation(databaseName, tableName, alias)(table, partitions) } } @@ -32,9 +40,11 @@ object HiveMetatoreTypes { } } -case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String])(val table: Table) +case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String])(val table: Table, val partitions: Seq[Partition]) extends plans.logical.BaseRelation { - val hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) + + def hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) + def hiveQlPartitions = partitions.map(new org.apache.hadoop.hive.ql.metadata.Partition(hiveQlTable, _)) val tableDesc = new TableDesc( Class.forName(table.getSd.getSerdeInfo.getSerializationLib).asInstanceOf[Class[org.apache.hadoop.hive.serde2.Deserializer]], Class.forName(table.getSd.getInputFormat).asInstanceOf[Class[org.apache.hadoop.mapred.InputFormat[_,_]]], diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index 941c399ee9c89..ca2025850ee9c 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -195,25 +195,36 @@ object TestShark extends Logging { implicit def logicalToSharkQuery(plan: LogicalPlan) = new SharkQuery { val parsed = plan } - protected case class TestTable(name: String, commands: String*) + protected case class TestTable(name: String, commands: (()=>Unit)*) + + protected implicit class SqlCmd(sql: String) { def cmd = () => sql.q.stringResult(): Unit} /** * A list of test tables and the DDL required to initialize them. A test table is loaded on demand when a query * are run against it. */ val testTables = Seq( TestTable("src", - "CREATE TABLE src (key INT, value STRING)", - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' INTO TABLE src"), + "CREATE TABLE src (key INT, value STRING)".cmd, + s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' INTO TABLE src".cmd), TestTable("src1", - "CREATE TABLE src1 (key INT, value STRING)", - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv3.txt' INTO TABLE src1"), + "CREATE TABLE src1 (key INT, value STRING)".cmd, + s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv3.txt' INTO TABLE src1".cmd), TestTable("dest1", - "CREATE TABLE src (key INT, value STRING)"), + "CREATE TABLE src (key INT, value STRING)".cmd), TestTable("dest2", - "CREATE TABLE src (key INT, value STRING)"), + "CREATE TABLE src (key INT, value STRING)".cmd), TestTable("dest3", - "CREATE TABLE src (key INT, value STRING)") + "CREATE TABLE src (key INT, value STRING)".cmd), + TestTable("srcpart", () => { + runSqlHive("CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") + Seq("2008-04-08", "2008-04-09").foreach { ds => + Seq("11", "12").foreach { hr => + val partSpec = Map("ds" -> ds, "hr" -> hr) + runSqlHive(s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr')") + } + } + }) ) protected val testTableNames = testTables.map(_.name).toSet @@ -222,7 +233,7 @@ object TestShark extends Logging { if(!(loadedTables contains name)) { logger.info(s"Loading test table $name") val createCmds = testTables.find(_.name == name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) - createCmds.foreach(_.q.stringResult()) + createCmds.foreach(_()) loadedTables += name } } diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index 10401a39da896..5677488048021 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -39,12 +39,24 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation } def execute() = { - hadoopReader.makeRDDForTable(relation.hiveQlTable).map { - case struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct => - refs.map { ref => - val data = objectInspector.getStructFieldData(struct, ref) - ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector].getPrimitiveJavaObject(data) - }.toIndexedSeq + val rdd = if(!relation.hiveQlTable.isPartitioned) + hadoopReader.makeRDDForTable(relation.hiveQlTable) + else + hadoopReader.makeRDDForPartitionedTable(relation.hiveQlPartitions) + + def unpackStruct(struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct) = + refs.map { ref => + val data = objectInspector.getStructFieldData(struct, ref) + ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector].getPrimitiveJavaObject(data) + }.toIndexedSeq + + rdd.map { + case array: Array[Any] => + array.flatMap { + case struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct => unpackStruct(struct) + case array: Array[Any] => array + } + case struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct => unpackStruct(struct) } } diff --git a/src/test/scala/shark2/HiveComparisionTest.scala b/src/test/scala/shark2/HiveComparisionTest.scala index 0d5210a2f9490..edc12b9d1c970 100644 --- a/src/test/scala/shark2/HiveComparisionTest.scala +++ b/src/test/scala/shark2/HiveComparisionTest.scala @@ -99,6 +99,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with val computedResults = queryList.zip(hiveCacheFiles).zipWithIndex.map { case ((queryString, cachedAnswerFile), i)=> logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") + info(s"HIVE: $queryString") // Analyze the query with catalyst to ensure test tables are loaded. val sharkQuery = (new testShark.SharkSqlQuery(queryString)) val answer = sharkQuery.analyzed match { diff --git a/src/test/scala/shark2/HiveQueryTests.scala b/src/test/scala/shark2/HiveQueryTests.scala index e7484927f4617..0960ad502bd6d 100644 --- a/src/test/scala/shark2/HiveQueryTests.scala +++ b/src/test/scala/shark2/HiveQueryTests.scala @@ -41,4 +41,7 @@ class HiveQueryTests extends HiveComaparisionTest { createQueryTest("length.udf", "SELECT length(\"test\") FROM src LIMIT 1") + createQueryTest("partitioned table scan", + "SELECT * FROM srcpart") + } \ No newline at end of file From 420e05ba4c032b915ec8d31bcd030bf305712431 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 14 Dec 2013 23:20:23 -0800 Subject: [PATCH 224/778] more tests passing! --- src/test/scala/shark2/HiveCompatability.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index 08b6164db22f9..b935af0339f8a 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -61,6 +61,8 @@ class HiveCompatability extends HiveComaparisionTest { "drop_table", "drop_view", "index_auth", + "index_auto_mult_tables", + "index_auto_mult_tables_compact", "index_auto_self_join", "innerjoin", "input0", @@ -85,6 +87,7 @@ class HiveCompatability extends HiveComaparisionTest { "join36", "join37", "join_casesensitive", + "join_empty", "join_view", "literal_double", "literal_ints", @@ -95,6 +98,7 @@ class HiveCompatability extends HiveComaparisionTest { "noalias_subq1", "nullgroup", "nullgroup2", + "nullgroup3", "nullinput", "ppd_gby_join", "ppd_outer_join5", From cb7b5afe7ff4d0724a00edfd1a788ed2842b3a5c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 15 Dec 2013 02:40:25 -0800 Subject: [PATCH 225/778] views example --- src/main/scala/ViewsExample.scala | 39 +++++++++++++++++++++++++++++++ 1 file changed, 39 insertions(+) create mode 100644 src/main/scala/ViewsExample.scala diff --git a/src/main/scala/ViewsExample.scala b/src/main/scala/ViewsExample.scala new file mode 100644 index 0000000000000..a6049146eb692 --- /dev/null +++ b/src/main/scala/ViewsExample.scala @@ -0,0 +1,39 @@ +package catalyst + +import catalyst.analysis.UnresolvedRelation +import catalyst.plans.Inner +import catalyst.plans.logical._ + +/* Implicit Conversions */ +import dsl._ +import shark2.TestShark._ // For .toRdd execution using locally running test Shark. + +object ViewsExample { + def main(args: Array[String]): Unit = { + // Create a list of named views that can be substituted into logical plans. + // In this example the views read from local, in-memory relations with schema (a INT, b STRING) and (c INT, d STRING) + // respectively. loadData returns a copy of that relation with the specified tuples appended to the Rdd. + // The .select uses the DSL to add a projection on top of the relation that returns only the column "a". + val views = Map( + "view1" -> LocalRelation('a.int, 'b.string).loadData(("a", 1) :: ("b", 2) :: Nil).select('a), + "view2" -> LocalRelation('c.int, 'd.string).loadData(("c", 1) :: ("d", 2) :: Nil) + ) + + // Construct a plan that has UnresolvedRelations in it using the DSL. + val unresolvedPlan = + UnresolvedRelation("view1") + .join(UnresolvedRelation("view2"), Inner, Some('a === 'c)) + .where('c < 1) + .select('a, 'c) + println(s"Unresolved Plan:\n$unresolvedPlan") + + // Replace UnresolvedRelations with logical plans from the views map. + val withRelations = unresolvedPlan transform { + case UnresolvedRelation(name, _) => views(name) + } + + println(s"With relations:\n$withRelations ") + println(s"Analyzed:\n${withRelations.analyze}") // Print with all references resolved. + println(s"Answer: ${withRelations.toRdd.collect.toSeq}") + } +} \ No newline at end of file From be5ae2c6fe4cec5c2fa2f11aa0d94775ec438552 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 15 Dec 2013 03:06:31 -0800 Subject: [PATCH 226/778] better resolution logging --- src/main/scala/analysis/Analyzer.scala | 8 +++++--- src/main/scala/rules/Rule.scala | 2 +- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala index 28e540f3b0490..001ae89600dd5 100644 --- a/src/main/scala/analysis/Analyzer.scala +++ b/src/main/scala/analysis/Analyzer.scala @@ -41,12 +41,14 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry) extends RuleExecuto object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan if childIsFullyResolved(q) => - // logger.fine(s"resolving ${plan.simpleString}") + logger.trace(s"Attempting to resolve ${q.simpleString}") q transformExpressions { case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. - q.resolve(name).getOrElse(u) - } + val result = q.resolve(name).getOrElse(u) + logger.debug(s"Resolving $u to $result") + result + } } } diff --git a/src/main/scala/rules/Rule.scala b/src/main/scala/rules/Rule.scala index 5fa7fca57176d..c20eca0109d88 100644 --- a/src/main/scala/rules/Rule.scala +++ b/src/main/scala/rules/Rule.scala @@ -3,7 +3,7 @@ package rules import trees._ -abstract class Rule[TreeType <: TreeNode[_]] { +abstract class Rule[TreeType <: TreeNode[_]] extends Logging { val name = { val className = getClass.getName if(className endsWith "$") From 2469b004027622e7dd025553342b2335d4a41b0c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 15 Dec 2013 03:06:53 -0800 Subject: [PATCH 227/778] skip nodes with unresolved children when doing coersions --- src/main/scala/analysis/typeCoercion.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/scala/analysis/typeCoercion.scala b/src/main/scala/analysis/typeCoercion.scala index 41e3bedb2e0f8..82bb3b7d1b539 100644 --- a/src/main/scala/analysis/typeCoercion.scala +++ b/src/main/scala/analysis/typeCoercion.scala @@ -15,6 +15,9 @@ object ConvertNaNs extends Rule[LogicalPlan]{ def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + /* Double Conversions */ case b: BinaryExpression if b.left == stringNaN && b.right.dataType == DoubleType => b.makeCopy(Array(b.right, Literal(Double.NaN))) From 6d03ce9dcb9bfa44b8200379f59ac477395699fb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 15 Dec 2013 03:07:07 -0800 Subject: [PATCH 228/778] defaults for unresolved relation --- src/main/scala/analysis/unresolved.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala index fc135b7854db6..82a53c1926e14 100644 --- a/src/main/scala/analysis/unresolved.scala +++ b/src/main/scala/analysis/unresolved.scala @@ -14,7 +14,7 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str /** * Holds the name of a relation that has yet to be looked up in a [[Catalog]]. */ -case class UnresolvedRelation(name: String, alias: Option[String]) extends BaseRelation { +case class UnresolvedRelation(name: String, alias: Option[String] = None) extends BaseRelation { def output = Nil } From 9e74808a9e306db9fbde3710a1deaa117c981f2e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 15 Dec 2013 03:07:23 -0800 Subject: [PATCH 229/778] add children resolved. --- src/main/scala/expressions/Expression.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/scala/expressions/Expression.scala b/src/main/scala/expressions/Expression.scala index f933779c62d13..db2c95aecd673 100644 --- a/src/main/scala/expressions/Expression.scala +++ b/src/main/scala/expressions/Expression.scala @@ -15,7 +15,9 @@ abstract class Expression extends TreeNode[Expression] { * Returns true if this expression and all its children have been resolved to a specific schema and false if it is * still contains any unresolved placeholders. */ - lazy val resolved: Boolean = children.map(_.resolved).reduceLeftOption(_&&_).getOrElse(true) + lazy val resolved: Boolean = childrenResolved + + def childrenResolved = children.map(_.resolved).reduceLeftOption(_&&_).getOrElse(true) } abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] { From b67a225f6895b2b3c4fdd6086714516135e495ba Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 15 Dec 2013 03:07:39 -0800 Subject: [PATCH 230/778] better errors when types don't match up. --- src/main/scala/expressions/arithmetic.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/main/scala/expressions/arithmetic.scala b/src/main/scala/expressions/arithmetic.scala index fc4e9c43150e4..af30e4fb90637 100644 --- a/src/main/scala/expressions/arithmetic.scala +++ b/src/main/scala/expressions/arithmetic.scala @@ -1,6 +1,8 @@ package catalyst package expressions +import catalyst.analysis.UnresolvedException + case class UnaryMinus(child: Expression) extends UnaryExpression { def dataType = child.dataType @@ -11,8 +13,12 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { abstract class BinaryArithmetic extends BinaryExpression { self: Product => + override lazy val resolved = left.dataType == right.dataType + def dataType = { - require(left.dataType == right.dataType) // TODO(marmbrus): Figure out how to handle coersions. + if(!resolved) + throw new UnresolvedException( + this, s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") left.dataType } } From 12584f406fe0e2e7af23a2ebeabb638ff18d6d48 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 15 Dec 2013 03:08:16 -0800 Subject: [PATCH 231/778] better errors for missing clauses. support for matching multiple clauses with the same name. --- src/main/scala/frontend/Hive.scala | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 7ed6dc3b6f1a5..1005f1ed81816 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -13,6 +13,7 @@ import plans.logical._ import types._ import collection.JavaConversions._ +import scala.collection.mutable /** * A logical node that represent a non-query command to be executed by the system. For example, @@ -258,7 +259,16 @@ object HiveQl { val unhandledClauses = nodeList.filterNot(clauseNames contains _.getText) require(unhandledClauses.isEmpty, s"Unhandled parse clauses: $unhandledClauses") - clauseNames.map(getClauseOption(_, nodeList)) + var remainingNodes = nodeList + + val clauses = clauseNames.map { clauseName => + val (matches, nonMatches) = remainingNodes.partition(_.getText == clauseName) + remainingNodes = nonMatches ++ (if(matches.nonEmpty) matches.tail else Nil) + matches.headOption + } + + assert(remainingNodes.isEmpty, s"Unhandled clauses: ${remainingNodes.map(dumpTree(_)).mkString("\n")}") + clauses } def getClause(clauseName: String, nodeList: Seq[Node]) = @@ -397,7 +407,7 @@ object HiveQl { case Token(allJoinTokens(joinToken), relation1 :: relation2 :: other) => - assert(other.size <= 1, "Unhandled join child") + assert(other.size <= 1, s"Unhandled join child ${other}") val joinType = joinToken match { case "TOK_JOIN" => Inner case "TOK_RIGHTOUTERJOIN" => RightOuter From 389bc0bcd06a879083be6f9cc5b32ddb65b21ebb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 15 Dec 2013 03:08:46 -0800 Subject: [PATCH 232/778] support for partitioned columns in output. --- src/main/scala/shark2/MetastoreCatalog.scala | 21 ++++++++++++-------- src/main/scala/shark2/TestShark.scala | 6 +++--- src/main/scala/shark2/hiveOperators.scala | 15 ++++++++------ src/main/scala/util/package.scala | 2 ++ 4 files changed, 27 insertions(+), 17 deletions(-) diff --git a/src/main/scala/shark2/MetastoreCatalog.scala b/src/main/scala/shark2/MetastoreCatalog.scala index 5002de667267e..c5255c4904f11 100644 --- a/src/main/scala/shark2/MetastoreCatalog.scala +++ b/src/main/scala/shark2/MetastoreCatalog.scala @@ -2,7 +2,7 @@ package catalyst package shark2 import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.api.{Partition, Table} +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table} import org.apache.hadoop.hive.metastore.HiveMetaStoreClient import analysis.Catalog @@ -52,12 +52,17 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt hiveQlTable.getSchema ) + implicit class SchemaAttribute(f: FieldSchema) { + def toAttribute = + AttributeReference( + f.getName, + HiveMetatoreTypes.toDataType(f.getType), + true // Since data can be dumped in randomly with no validation, everything is nullable. + )(qualifiers = tableName +: alias.toSeq) + } + + val partitionKeys = hiveQlTable.getPartitionKeys.map(_.toAttribute) + // Must be a stable value since new attributes are born here. - val output = table.getSd.getCols.map { col => - AttributeReference( - col.getName, - HiveMetatoreTypes.toDataType(col.getType), - true // AHHH, who makes a metastore with no concept of nullalbility? - )(qualifiers = tableName +: alias.toSeq) - } + val output = partitionKeys ++ table.getSd.getCols.map(_.toAttribute) } \ No newline at end of file diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index ca2025850ee9c..efd1d91453f63 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -211,11 +211,11 @@ object TestShark extends Logging { "CREATE TABLE src1 (key INT, value STRING)".cmd, s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv3.txt' INTO TABLE src1".cmd), TestTable("dest1", - "CREATE TABLE src (key INT, value STRING)".cmd), + "CREATE TABLE IF NOT EXISTS dest1 (key INT, value STRING)".cmd), TestTable("dest2", - "CREATE TABLE src (key INT, value STRING)".cmd), + "CREATE TABLE IF NOT EXISTS dest2 (key INT, value STRING)".cmd), TestTable("dest3", - "CREATE TABLE src (key INT, value STRING)".cmd), + "CREATE TABLE IF NOT EXISTS dest3 (key INT, value STRING)".cmd), TestTable("srcpart", () => { runSqlHive("CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") Seq("2008-04-08", "2008-04-09").foreach { ds => diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index 5677488048021..9b2972418871d 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -12,6 +12,7 @@ import shark.execution.HadoopTableReader import shark.SharkContext import expressions.Attribute +import util._ /* Implicits */ import collection.JavaConversions._ @@ -32,25 +33,26 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation * The hive struct field references that correspond to the attributes to be read from this table. */ @transient - lazy val refs = attributes.map { a => + lazy val refs = attributes.flatMap { a => objectInspector.getAllStructFieldRefs .find(_.getFieldName == a.name) - .getOrElse(sys.error(s"Invalid attribute ${a.name} referenced in table $relation")) } - def execute() = { - val rdd = if(!relation.hiveQlTable.isPartitioned) + @transient + def inputRdd = + if(!relation.hiveQlTable.isPartitioned) hadoopReader.makeRDDForTable(relation.hiveQlTable) else hadoopReader.makeRDDForPartitionedTable(relation.hiveQlPartitions) + def execute() = { def unpackStruct(struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct) = refs.map { ref => val data = objectInspector.getStructFieldData(struct, ref) ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector].getPrimitiveJavaObject(data) }.toIndexedSeq - rdd.map { + inputRdd.map { case array: Array[Any] => array.flatMap { case struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct => unpackStruct(struct) @@ -90,12 +92,13 @@ case class InsertIntoHiveTable(table: MetastoreRelation, child: SharkPlan) def output = child.output def execute() = { val childRdd = child.execute() + assert(childRdd != null) // TODO: write directly to hive val tempDir = java.io.File.createTempFile("data", "tsv") tempDir.delete() tempDir.mkdir() - childRdd.map(_.map(_.toString).mkString("\001")).saveAsTextFile(tempDir.getCanonicalPath) + childRdd.map(_.map(a => stringOrNull(a.asInstanceOf[AnyRef])).mkString("\001")).saveAsTextFile(tempDir.getCanonicalPath) sc.runSql(s"LOAD DATA LOCAL INPATH '${tempDir.getCanonicalPath}/*' INTO TABLE ${table.tableName}") // It would be nice to just return the childRdd unchanged so insert operations could be chained, diff --git a/src/main/scala/util/package.scala b/src/main/scala/util/package.scala index 9fec9ef2deb9c..ba7161c2acf2a 100644 --- a/src/main/scala/util/package.scala +++ b/src/main/scala/util/package.scala @@ -50,4 +50,6 @@ package object util { case (l,r) => (if(l == r) " " else "*") + l + (" " * ((maxLeftSize - left.size) + 3)) + r } } + + def stringOrNull(a: AnyRef) = if(a == null) null else a.toString } \ No newline at end of file From 0f6279f46b1f5bbdd7fc3cc8293849d7a24bc876 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 15 Dec 2013 03:15:52 -0800 Subject: [PATCH 233/778] broken tests. --- src/test/scala/shark2/HiveCompatability.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index b935af0339f8a..0bdccc4b2e85d 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -61,8 +61,8 @@ class HiveCompatability extends HiveComaparisionTest { "drop_table", "drop_view", "index_auth", - "index_auto_mult_tables", - "index_auto_mult_tables_compact", + //"index_auto_mult_tables", + //"index_auto_mult_tables_compact", "index_auto_self_join", "innerjoin", "input0", From 2b27230dcca82264e8f91aa031fcdc2a78228cd3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 00:59:38 -0800 Subject: [PATCH 234/778] add depth based subtree access --- src/main/scala/trees/TreeNode.scala | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index 94291f95cf028..de68b7e85fb8f 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -8,6 +8,9 @@ object TreeNode { protected def nextId() = currentId.getAndIncrement() } +/** Used when traversing the tree for a node at a given depth */ +private class MutableInt(var i: Int) + abstract class TreeNode[BaseType <: TreeNode[BaseType]] { self: BaseType with Product => @@ -164,6 +167,26 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** Returns a string representation of the nodes in this tree */ def treeString = generateTreeString(0, new StringBuilder).toString + /** + * Returns a string representation of the nodes in this tree, where each operator is numbered. The numbers can be + * used with [[apply]] to easily access specific subtrees. + */ + def numberedTreeString = + treeString.split("\n").zipWithIndex.map { case (line,i) => f"$i%02d $line" }.mkString("\n") + + def apply(depth: Int): BaseType = getNodeAtDepth(new MutableInt(depth)) + + protected def getNodeAtDepth(depth: MutableInt): BaseType = { + if(depth.i < 0) + return null.asInstanceOf[BaseType] + else if(depth.i == 0) + this + else { + depth.i -= 1 + children.map(_.getNodeAtDepth(depth)).collectFirst { case n if n != null => n }.getOrElse(sys.error("Invalid depth")) + } + } + /** Appends the string represent of this node and its children to [[builder]]. */ protected def generateTreeString(depth: Int, builder: StringBuilder): StringBuilder = { builder.append(" " * depth) From f1f7e96693ba42f0e43b5d53e1b39bd960fa57e9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 01:00:11 -0800 Subject: [PATCH 235/778] fix incorrect generation of join keys --- src/main/scala/shark2/joins.scala | 30 ++++++++++++++++++++---------- 1 file changed, 20 insertions(+), 10 deletions(-) diff --git a/src/main/scala/shark2/joins.scala b/src/main/scala/shark2/joins.scala index 877dc9ac73055..8d42e12b6c295 100644 --- a/src/main/scala/shark2/joins.scala +++ b/src/main/scala/shark2/joins.scala @@ -1,9 +1,11 @@ package catalyst package shark2 -import expressions._ import org.apache.spark.rdd.RDD +import errors._ +import expressions._ + /* Implicits */ import org.apache.spark.SparkContext._ @@ -15,23 +17,31 @@ case class SparkEquiInnerJoin( def output = left.output ++ right.output - def execute() = { - val leftWithKeys = generateKeys(leftKeys, left.execute()) - val rightWithKeys = generateKeys(rightKeys, right.execute()) + def execute() = attachTree(this, "execute") { + val leftWithKeys = left.execute .map { row => + val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) + logger.debug(s"Generated left join keys ($leftKeys) => ($joinKeys) given row $row") + (joinKeys, row) + } + + val rightWithKeys = right.execute().map { row => + val joinKeys = rightKeys.map(Evaluate(_, Vector(Nil, row))) + logger.debug(s"Generated right join keys ($rightKeys) => ($joinKeys) given row $row") + (joinKeys, row) + } + // Do the join. - val joined = leftWithKeys.join(rightWithKeys) + val joined = filterNulls(leftWithKeys).join(filterNulls(rightWithKeys)) // Drop join keys and merge input tuples. joined.map { case (_, (leftTuple, rightTuple)) => leftTuple ++ rightTuple } } /** - * Turns row into (joinKeys, row). Filters any rows where the any of the join keys is null, ensuring three-valued + * Filters any rows where the any of the join keys is null, ensuring three-valued * logic for the equi-join conditions. */ - protected def generateKeys(keys: Seq[Expression], rdd: RDD[IndexedSeq[Any]]) = - rdd.map { - case row => (leftKeys.map(Evaluate(_, Vector(row))), row) - }.filter { + protected def filterNulls(rdd: RDD[(Seq[Any], IndexedSeq[Any])]) = + rdd.filter { case (key: Seq[_], _) => !key.map(_ == null).reduceLeft(_ || _) } } From e43dc1ee8bc5a608e53b489746ef71b21f6d68b4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 01:00:31 -0800 Subject: [PATCH 236/778] add string => int cast evaluation --- src/main/scala/expressions/Evaluate.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index a79d70cfaa611..5d31fda7f66c9 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -111,6 +111,7 @@ object Evaluate extends Logging { /* Casts */ case Cast(e, StringType) => eval(e).toString + case Cast(e, IntegerType) if e.dataType == StringType => eval(e).asInstanceOf[String].toInt /* Boolean Logic */ case Not(c) => !eval(c).asInstanceOf[Boolean] From e2ef4a5371b19dc7e5bc0c39ff785d4d8a47c97c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 01:00:42 -0800 Subject: [PATCH 237/778] logging --- src/main/scala/expressions/BoundAttribute.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/expressions/BoundAttribute.scala b/src/main/scala/expressions/BoundAttribute.scala index bc1d8e6b0532b..26a70cac6c4a2 100644 --- a/src/main/scala/expressions/BoundAttribute.scala +++ b/src/main/scala/expressions/BoundAttribute.scala @@ -32,10 +32,12 @@ object BindReferences extends Rule[SharkPlan] { plan.transform { case leafNode: SharkPlan if leafNode.children.isEmpty => leafNode case nonLeaf: SharkPlan => attachTree(nonLeaf, "Binding references in operator") { + logger.debug(s"Binding references in node ${nonLeaf.simpleString}") nonLeaf.transformExpressions { case a: AttributeReference => attachTree(a, "Binding attribute") { val inputTuple = nonLeaf.children.indexWhere(_.output contains a) val ordinal = nonLeaf.children(inputTuple).output.indexWhere(_ == a) + logger.debug(s"Binding $a to $inputTuple.$ordinal given input ${nonLeaf.children.map(_.output.mkString("{", ",", "}")).mkString(",")}") assert(ordinal != -1, "Reference not found in child plan") BoundReference(inputTuple, ordinal, a) } From 91573a44b3e9398976ac5acaf3d74e1fec309db2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 01:00:53 -0800 Subject: [PATCH 238/778] initial type promotion --- src/main/scala/analysis/Analyzer.scala | 1 + src/main/scala/analysis/typeCoercion.scala | 15 +++++++++++++++ 2 files changed, 16 insertions(+) diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala index 001ae89600dd5..34f1e075afa55 100644 --- a/src/main/scala/analysis/Analyzer.scala +++ b/src/main/scala/analysis/Analyzer.scala @@ -23,6 +23,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry) extends RuleExecuto Batch("Aggregation", Once, GlobalAggregates), Batch("Type Coersion", fixedPoint, + PromoteTypes, ConvertNaNs) ) diff --git a/src/main/scala/analysis/typeCoercion.scala b/src/main/scala/analysis/typeCoercion.scala index 82bb3b7d1b539..680e56582a746 100644 --- a/src/main/scala/analysis/typeCoercion.scala +++ b/src/main/scala/analysis/typeCoercion.scala @@ -35,4 +35,19 @@ object ConvertNaNs extends Rule[LogicalPlan]{ b.makeCopy(Array(Literal(Float.NaN), b.left)) } } +} + +object PromoteTypes extends Rule[LogicalPlan] { + // TODO: Do this generically given some list of type precedence. + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + case b: BinaryExpression if b.left.dataType == StringType && b.right.dataType == IntegerType => + b.makeCopy(Array(Cast(b.left, IntegerType), b.right)) + case b: BinaryExpression if b.left.dataType == IntegerType && b.right.dataType == StringType => + b.makeCopy(Array(b.left, Cast(b.right, IntegerType))) + } + } } \ No newline at end of file From e10df999267098d8f2b6cb12a9e0d1abed61f054 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 16:35:03 -0800 Subject: [PATCH 239/778] Create new expr ids for local relations that exist more than once in a query plan. --- src/main/scala/analysis/Analyzer.scala | 2 ++ src/main/scala/analysis/unresolved.scala | 2 ++ .../scala/expressions/BoundAttribute.scala | 1 + .../scala/expressions/namedExpressions.scala | 8 ++++-- .../scala/plans/logical/TestRelation.scala | 28 +++++++++++++++++++ src/main/scala/trees/TreeNode.scala | 3 ++ 6 files changed, 42 insertions(+), 2 deletions(-) diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala index 34f1e075afa55..025766e6886d5 100644 --- a/src/main/scala/analysis/Analyzer.scala +++ b/src/main/scala/analysis/Analyzer.scala @@ -15,6 +15,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry) extends RuleExecuto val fixedPoint = FixedPoint(100) val batches = Seq( + Batch("LocalRelations", Once, + NewLocalRelationInstances), Batch("Resolution", fixedPoint, ResolveReferences, ResolveRelations, diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala index 82a53c1926e14..2e5b587473b0c 100644 --- a/src/main/scala/analysis/unresolved.scala +++ b/src/main/scala/analysis/unresolved.scala @@ -28,6 +28,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo def qualifiers = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false + def newInstance = this def withQualifiers(newQualifiers: Seq[String]) = this override def toString(): String = s"'$name" @@ -56,6 +57,7 @@ case class Star(table: Option[String]) extends Attribute with trees.LeafNode[Exp def qualifiers = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false + def newInstance = this def withQualifiers(newQualifiers: Seq[String]) = this override def toString = table.map(_ + ".").getOrElse("") + "*" diff --git a/src/main/scala/expressions/BoundAttribute.scala b/src/main/scala/expressions/BoundAttribute.scala index 26a70cac6c4a2..84d7947c40fe0 100644 --- a/src/main/scala/expressions/BoundAttribute.scala +++ b/src/main/scala/expressions/BoundAttribute.scala @@ -20,6 +20,7 @@ case class BoundReference(inputTuple: Int, ordinal: Int, baseReference: Attribut def qualifiers = baseReference.qualifiers def name = baseReference.name + def newInstance = BoundReference(inputTuple, ordinal, baseReference.newInstance) def withQualifiers(newQualifiers: Seq[String]) = BoundReference(inputTuple, ordinal, baseReference.withQualifiers(newQualifiers)) diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index 0faa5b15d6bdd..a72e9a9c3b59e 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -33,6 +33,7 @@ abstract class Attribute extends NamedExpression { def references = Set(this) def toAttribute = this + def newInstance: Attribute } /** @@ -71,8 +72,8 @@ case class Alias(child: Expression, name: String) * @param dataType The [[DataType]] of this attribute. * @param nullable True if null is a valid value for this attribute. * @param exprId A globally unique id used to check if different AttributeReferences refer to the same attribute. - * @param qualifiers a list of strings that can be used to refered to this attribute in a fully qualified way. Consider - * the examples tableName.name, subQueryAlias.name. tableName and subQueryAlias are possible qualifers. + * @param qualifiers a list of strings that can be used to referred to this attribute in a fully qualified way. Consider + * the examples tableName.name, subQueryAlias.name. tableName and subQueryAlias are possible qualifiers. */ case class AttributeReference(name: String, dataType: DataType, nullable: Boolean = true) (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) @@ -83,6 +84,9 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea case _ => false } + def newInstance = + AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) + /** * Returns a copy of this [[AttributeReference]] with changed nullability. */ diff --git a/src/main/scala/plans/logical/TestRelation.scala b/src/main/scala/plans/logical/TestRelation.scala index e8ad4fa6582f8..ca49f585e7f8b 100644 --- a/src/main/scala/plans/logical/TestRelation.scala +++ b/src/main/scala/plans/logical/TestRelation.scala @@ -3,6 +3,7 @@ package plans package logical import expressions._ +import rules._ object LocalRelation { def apply(output: Attribute*) = @@ -12,4 +13,31 @@ object LocalRelation { case class LocalRelation(output: Seq[Attribute], data: Seq[Product] = Nil) extends LeafNode { // TODO: Validate schema compliance. def loadData(newData: Seq[Product]) = new LocalRelation(output, data ++ newData) + + /** + * Returns an identical copy of this relation with new exprIds for all attributes. Different attributes are required + * when a relation is going to be included multiple times in the same query. + * @return + */ + def newInstance: LocalRelation = { + LocalRelation(output.map(_.newInstance), data) + } + + override protected def stringArgs = Iterator(output) +} + +/** + * If any local relation appears more than once in the query plan then the plan is updated so that each instance has + * unique expression ids for the attributes produced. + */ +object NewLocalRelationInstances extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + val localRelations = plan collect { case l: LocalRelation => l} + val multiAppearance = + localRelations.groupBy(identity[LocalRelation]).filter { case (_, ls) => ls.size > 1 }.map(_._1).toSet + + plan transform { + case l: LocalRelation if multiAppearance contains l => l.newInstance + } + } } \ No newline at end of file diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index de68b7e85fb8f..b937f0564ac95 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -151,6 +151,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** Returns the name of this type of TreeNode. Defaults to the class name. */ def nodeName = getClass.getSimpleName + /** The arguments that should be included in the arg string. Defaults to the [[productIterator]] */ + protected def stringArgs = productIterator + /** Returns a string representing the arguments to this node, minus any children */ def argString: String = productIterator.flatMap { case tn: TreeNode[_] if children contains tn => Nil From 76ec650bae79aef1d3b7ad9ec49232b8bfa9f20e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 16:35:25 -0800 Subject: [PATCH 240/778] fix join conditions --- src/test/scala/shark2/DslQueryTests.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/test/scala/shark2/DslQueryTests.scala b/src/test/scala/shark2/DslQueryTests.scala index e05e799d67e6b..ef15c564df1cd 100644 --- a/src/test/scala/shark2/DslQueryTests.scala +++ b/src/test/scala/shark2/DslQueryTests.scala @@ -158,7 +158,7 @@ class DslQueryTests extends FunSuite with BeforeAndAfterAll { val x = testData2.where('a === 1).subquery('x) val y = testData2.where('a === 1).subquery('y) checkAnswer( - x.join(y).where("x.a" === "y.a"), + x.join(y).where("x.a".attr === "y.a".attr), (1,1,1,1) :: (1,1,1,2) :: (1,2,1,1) :: @@ -170,7 +170,7 @@ class DslQueryTests extends FunSuite with BeforeAndAfterAll { val x = testData2.where('a === 1).subquery('x) val y = testData2.where('a === 2).subquery('y) checkAnswer( - x.join(y).where("x.a" === "y.a"), + x.join(y).where("x.a".attr === "y.a".attr), Nil) } @@ -180,7 +180,7 @@ class DslQueryTests extends FunSuite with BeforeAndAfterAll { val bigDataY = bigData.subquery('y) checkAnswer( - bigDataX.join(bigDataY).where("x.key" === "y.key"), + bigDataX.join(bigDataY).where("x.key".attr === "y.key".attr), testData.data.flatMap(row => Seq.fill(16)((row.productIterator ++ row.productIterator).toSeq))) } From 15ff4488b45ffec839048c230597f6424fe81ed4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 16:35:43 -0800 Subject: [PATCH 241/778] better error message when a dsl test throws an exception --- src/test/scala/shark2/DslQueryTests.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/test/scala/shark2/DslQueryTests.scala b/src/test/scala/shark2/DslQueryTests.scala index ef15c564df1cd..3ab4b10ced79f 100644 --- a/src/test/scala/shark2/DslQueryTests.scala +++ b/src/test/scala/shark2/DslQueryTests.scala @@ -209,7 +209,18 @@ class DslQueryTests extends FunSuite with BeforeAndAfterAll { val isSorted = plan.collect { case s: logical.Sort => s}.nonEmpty def prepareAnswer(answer: Seq[Any]) = if(!isSorted) answer.sortBy(_.toString) else answer - val sharkAnswer = plan.toRdd.collect().toSeq + val sharkAnswer = try plan.toRdd.collect().toSeq catch { + case e: Exception => + fail( + s""" + |Exception thrown while executing query: + |$plan + |== Physical Plan == + |${plan.executedPlan} + |== Exception == + |$e + """.stripMargin) + } assert(prepareAnswer(convertedAnswer) === prepareAnswer(sharkAnswer)) } } \ No newline at end of file From c43a259ce39ffcf29bd460f54b41c982451921b2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 16:37:35 -0800 Subject: [PATCH 242/778] comments --- src/main/scala/analysis/typeCoercion.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/analysis/typeCoercion.scala b/src/main/scala/analysis/typeCoercion.scala index 680e56582a746..24f9587218041 100644 --- a/src/main/scala/analysis/typeCoercion.scala +++ b/src/main/scala/analysis/typeCoercion.scala @@ -44,6 +44,7 @@ object PromoteTypes extends Rule[LogicalPlan] { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e + // Int String or String Int => Int Int case b: BinaryExpression if b.left.dataType == StringType && b.right.dataType == IntegerType => b.makeCopy(Array(Cast(b.left, IntegerType), b.right)) case b: BinaryExpression if b.left.dataType == IntegerType && b.right.dataType == StringType => From cc5efe31b99e491c04147c2519c4f32a462b32fa Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 16:54:21 -0800 Subject: [PATCH 243/778] First draft of broadcast nested loop join with full outer support. --- src/main/scala/shark2/TestShark.scala | 3 +- src/main/scala/shark2/joins.scala | 60 +++++++++++++++++++ .../scala/shark2/planningStrategies.scala | 8 +++ src/test/scala/shark2/DslQueryTests.scala | 38 +++++++++++- 4 files changed, 106 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index efd1d91453f63..4c8bd372187fd 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -123,7 +123,8 @@ object TestShark extends Logging { HiveTableScans :: DataSinks :: BasicOperators :: - CartesianProduct :: Nil + CartesianProduct :: + BroadcastNestedLoopJoin :: Nil } object PrepareForExecution extends RuleExecutor[SharkPlan] { diff --git a/src/main/scala/shark2/joins.scala b/src/main/scala/shark2/joins.scala index 8d42e12b6c295..f9329774d2d5a 100644 --- a/src/main/scala/shark2/joins.scala +++ b/src/main/scala/shark2/joins.scala @@ -5,6 +5,11 @@ import org.apache.spark.rdd.RDD import errors._ import expressions._ +import plans._ +import shark.SharkContext +import org.apache.spark.util.collection.BitSet +import scala.collection +import scala.collection.mutable /* Implicits */ import org.apache.spark.SparkContext._ @@ -52,4 +57,59 @@ case class CartesianProduct(left: SharkPlan, right: SharkPlan) extends BinaryNod def execute() = left.execute().cartesian(right.execute()).map { case (l: IndexedSeq[Any], r: IndexedSeq[Any]) => l ++ r } +} + +case class BroadcastNestedLoopJoin(streamed: SharkPlan, broadcast: SharkPlan, joinType: JoinType, condition: Option[Expression]) + (@transient sc: SharkContext) extends BinaryNode { + override def otherCopyArgs = sc :: Nil + def output = left.output ++ right.output + + /** The Streamed Relation */ + def left = streamed + /** The Broadcast relation */ + def right = broadcast + + def execute() = { + val broadcastedRelation = sc.broadcast(broadcast.execute().collect().toIndexedSeq) + + val streamedPlusMatches = streamed.execute().map { streamedRow => + var i = 0 + val matchedRows = new mutable.ArrayBuffer[IndexedSeq[Any]] + val includedBroadcastTuples = new scala.collection.mutable.BitSet(broadcastedRelation.value.size) + + while(i < broadcastedRelation.value.size) { + // TODO: One bitset per partition instead of per row. + val broadcastedRow = broadcastedRelation.value(i) + val includeRow = condition match { + case None => true + case Some(c) => Evaluate(c, Vector(streamedRow, broadcastedRow)).asInstanceOf[Boolean] + } + if(includeRow) { + matchedRows += (streamedRow ++ broadcastedRow) + includedBroadcastTuples += i + } + i += 1 + } + val outputRows = if(matchedRows.size > 0) + matchedRows + else if(joinType == LeftOuter || joinType == FullOuter) + Vector(streamedRow ++ Array.fill(right.output.size)(null)) + else + Vector() + (outputRows, includedBroadcastTuples) + } + + val allIncludedBroadcastTupes = streamedPlusMatches.map(_._2).reduce(_ ++ _) + val rightOuterMatches: Seq[IndexedSeq[Any]] = + if(joinType == RightOuter || joinType == FullOuter) + broadcastedRelation.value.zipWithIndex.filter { + case (row, i) => !allIncludedBroadcastTupes.contains(i) + }.map { + case (row, _) => Vector.fill(left.output.size)(null) ++ row + } + else + Vector() + + sc.union(streamedPlusMatches.flatMap(_._1), sc.makeRDD(rightOuterMatches)) + } } \ No newline at end of file diff --git a/src/main/scala/shark2/planningStrategies.scala b/src/main/scala/shark2/planningStrategies.scala index 79e483d910de1..b6efad849989c 100644 --- a/src/main/scala/shark2/planningStrategies.scala +++ b/src/main/scala/shark2/planningStrategies.scala @@ -110,6 +110,14 @@ abstract trait PlanningStrategies { expr.references subsetOf plan.outputSet } + object BroadcastNestedLoopJoin extends Strategy { + def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { + case logical.Join(left, right, joinType, condition) => + shark2.BroadcastNestedLoopJoin(planLater(left), planLater(right), joinType, condition)(sc) :: Nil + case _ => Nil + } + } + object CartesianProduct extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Join(left, right, _, None) => shark2.CartesianProduct(planLater(left), planLater(right)) :: Nil diff --git a/src/test/scala/shark2/DslQueryTests.scala b/src/test/scala/shark2/DslQueryTests.scala index 3ab4b10ced79f..146745deca780 100644 --- a/src/test/scala/shark2/DslQueryTests.scala +++ b/src/test/scala/shark2/DslQueryTests.scala @@ -37,7 +37,6 @@ object TestData { val upperCaseData = logical.LocalRelation('N.int, 'L.string).loadData( - (null, "") :: (1, "A") :: (2, "B") :: (3, "C") :: @@ -48,7 +47,6 @@ object TestData { val lowerCaseData = logical.LocalRelation('n.int, 'l.string).loadData( - (null, "") :: (1, "a") :: (2, "b") :: (3, "c") :: @@ -193,6 +191,42 @@ class DslQueryTests extends FunSuite with BeforeAndAfterAll { (2, 2, 2, 2) :: Nil) } + test("left outer join") { + checkAnswer( + upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N)), + (1, "A", 1, "a") :: + (2, "B", 2, "b") :: + (3, "C", 3, "c") :: + (4, "D", 4, "d") :: + (5, "E", null, null) :: + (6, "F", null, null) :: Nil) + } + + test("right outer join") { + checkAnswer( + lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N)), + (1, "a", 1, "A") :: + (2, "b", 2, "B") :: + (3, "c", 3, "C") :: + (4, "d", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + } + + test("full outer join") { + val left = upperCaseData.where('N <= 4).subquery('left) + val right = upperCaseData.where('N >= 3).subquery('right) + + checkAnswer( + left.join(right, FullOuter, Some("left.N".attr === "right.N".attr)), + (1, "A", null, null) :: + (2, "B", null, null) :: + (3, "C", 3, "C") :: + (4, "D", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + } + /** * Runs the plan and makes sure the answer matches the expected result. * @param plan the query to be executed From 02e6deeef1d36cbc818ee406431deb2674e0dd0b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 16:54:35 -0800 Subject: [PATCH 244/778] add another test that breaks the harness to the blacklist. --- src/test/scala/shark2/HiveCompatability.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index 0bdccc4b2e85d..e666e30a46897 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -29,6 +29,7 @@ class HiveCompatability extends HiveComaparisionTest { // This test fails and and exits the JVM. "auto_join18_multi_distinct", + "join18_multi_distinct", // Uses a serde that isn't on the classpath... breaks other tests. "bucketizedhiveinputformat" From b82481fc423d7658f87ad76746d6c49a5c9fb53c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 17:08:16 -0800 Subject: [PATCH 245/778] add todo comment. --- src/main/scala/shark2/hiveOperators.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index 9b2972418871d..32bc19567aace 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -46,6 +46,7 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation hadoopReader.makeRDDForPartitionedTable(relation.hiveQlPartitions) def execute() = { + // TODO: THIS DOES NOT CORRECTLY RETURN PARTITION ATTRIBUTES. def unpackStruct(struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct) = refs.map { ref => val data = objectInspector.getStructFieldData(struct, ref) From b7d8be086d75f84ccf472e2910d13ae789d75f04 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 18 Dec 2013 17:42:46 -0800 Subject: [PATCH 246/778] more tests passing. --- src/test/scala/shark2/HiveCompatability.scala | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index e666e30a46897..df6a385ec6be8 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -48,12 +48,14 @@ class HiveCompatability extends HiveComaparisionTest { "avro_schema_error_message", "avro_schema_literal", "bucketmapjoin6", + "combine1", "count", "ct_case_insensitive", "database_properties", "default_partition_name", "delimiter", "describe_database_json", + "diff_part_input_formats", "drop_function", "drop_index", "drop_partitions_filter", @@ -61,12 +63,21 @@ class HiveCompatability extends HiveComaparisionTest { "drop_partitions_filter3", "drop_table", "drop_view", + "groupby4_map", + "groupby4_map_skew", "index_auth", //"index_auto_mult_tables", //"index_auto_mult_tables_compact", "index_auto_self_join", + "index_auto_update", + "index_stale", "innerjoin", "input0", + "input11", + "input11_limit", + "input24", + "input25", + "input41", "input4_limit", "insert1", "join0", @@ -77,11 +88,15 @@ class HiveCompatability extends HiveComaparisionTest { "join13", "join15", "join16", + "join17", "join19", + "join2", "join22", "join23", "join25", + "join27", "join29", + "join3", "join30", "join31", "join34", @@ -89,7 +104,10 @@ class HiveCompatability extends HiveComaparisionTest { "join37", "join_casesensitive", "join_empty", + "join_reorder2", + "join_reorder3", "join_view", + "lineage1", "literal_double", "literal_ints", "literal_string", @@ -101,7 +119,14 @@ class HiveCompatability extends HiveComaparisionTest { "nullgroup2", "nullgroup3", "nullinput", + "nullinput2", + "ppd1", "ppd_gby_join", + "ppd_join3", + "ppd_outer_join1", + "ppd_outer_join2", + "ppd_outer_join3", + "ppd_outer_join4", "ppd_outer_join5", "ppd_random", "ppd_repeated_alias", @@ -115,7 +140,9 @@ class HiveCompatability extends HiveComaparisionTest { "show_functions", "smb_mapjoin_10", "tablename_with_select", + "udf2", "udf9", + "udf_10_trims", "udf_add", "udf_ascii", "udf_avg", @@ -201,6 +228,7 @@ class HiveCompatability extends HiveComaparisionTest { "union11", "union13", "union14", + "union15", "union16", "union2", "union20", @@ -209,6 +237,7 @@ class HiveCompatability extends HiveComaparisionTest { "union30", "union4", "union5", + "union7", "union8", "union9" ) From 304d17dc410742d378c8d8b72d1d8a62aa7cba9b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 Dec 2013 12:03:06 -0800 Subject: [PATCH 247/778] Create README.md --- README.md | 87 +++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 87 insertions(+) create mode 100644 README.md diff --git a/README.md b/README.md new file mode 100644 index 0000000000000..0b347f6f0e0ec --- /dev/null +++ b/README.md @@ -0,0 +1,87 @@ +catalyst +======== +Catalyst is a functional framework for optimizing relational query plans. + +More documentation can be found in the project's [scaladoc](http://marmbrus.github.io/catalyst/api/current/#catalyst.package) + +Getting Started +=============== +SCADS is built using [SBT](https://github.com/harrah/xsbt). The SBT launcher is included in the distribution (bin/sbt) and is responsible for downloading all other required jars (scala compiler and other dependencies). + +SBT commands can be invoked from the command line. For example, to clean and build a jar, you would run the following command: + + catalyst/$ sbt clean package + +Additionally, if you are going to be running several commands, you can use SBT from an interactive console, which amortizes the cost of starting the JVM and JITing SBT and the scala compiler. For example: + +``` +$ sbt/sbt +[info] Loading project definition from /Users/marmbrus/workspace/catalyst.clean/project +[info] Set current project to default-1207ac (in build file:/Users/marmbrus/workspace/catalyst.clean/) +> clean +> test:compile +[info] Compiling 10 Scala sources to catalyst/target/scala-2.10/test-classes... +[success] Total time: 15 s, completed Dec 20, 2013 12:00:06 PM +> test-only catalyst.shark2.DslQueryTests +``` + +Any command that is prefixed with a `~` (e.g. `~compile`) will be run automatically in a loop each time any dependent files have changed. + +Other dependencies +------------------ +In order to run all of the test cases or interact with sample data, you will need to set several environmental variables. + +export HIVE_HOME="/hive/build/dist" +export HIVE_DEV_HOME="/hive/" +export HADOOP_HOME="/hadoop-1.0.4" + +Using the console +================= +An interactive scala console can be invoked by running `sbt/sbt console`. From here you can execute queries and inspect the various stages of query optimization. + +```scala +catalyst$ sbt/sbt console + +[info] Starting scala interpreter... +import catalyst.analysis._ +import catalyst.dsl._ +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.frontend._ +import catalyst.plans.logical._ +import catalyst.rules._ +import catalyst.types._ +import catalyst.util._ +import catalyst.shark2.TestShark._ +Welcome to Scala version 2.10.3 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45). +Type in expressions to have them evaluated. +Type :help for more information. + +scala> val query = "SELECT * FROM (SELECT * FROM src) a".q +query: catalyst.shark2.TestShark.SharkSqlQuery = +SELECT * FROM (SELECT * FROM src) a +== Logical Plan == +Project {key#0,value#1} + Subquery a + Project {key#0,value#1} + MetastoreRelation default, src, None + +== Physical Plan == +Project {key#0,value#1} + HiveTableScan {key#0,value#1}, (MetastoreRelation default, src, None) +``` + +From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`s](http://marmbrus.github.io/catalyst/api/current/index.html#catalyst.trees.TreeNode). +```scala +scala> query.optimizedPlan +res1: catalyst.plans.logical.LogicalPlan = +Project {key#0,value#1} + Project {key#0,value#1} + MetastoreRelation default, src, None + + +scala> query.optimizedPlan transform { case Project(projectList, child) if projectList == child.output => child } +res2: catalyst.plans.logical.LogicalPlan = +Project {key#0,value#1} + MetastoreRelation default, src, None +``` From 37763959aae9d886a56349cf8b3aaffabd05a0d7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 Dec 2013 12:05:31 -0800 Subject: [PATCH 248/778] Update README.md --- README.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 0b347f6f0e0ec..c8f8400d1dc3a 100644 --- a/README.md +++ b/README.md @@ -31,9 +31,11 @@ Other dependencies ------------------ In order to run all of the test cases or interact with sample data, you will need to set several environmental variables. +``` export HIVE_HOME="/hive/build/dist" export HIVE_DEV_HOME="/hive/" export HADOOP_HOME="/hadoop-1.0.4" +``` Using the console ================= @@ -71,7 +73,7 @@ Project {key#0,value#1} HiveTableScan {key#0,value#1}, (MetastoreRelation default, src, None) ``` -From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`s](http://marmbrus.github.io/catalyst/api/current/index.html#catalyst.trees.TreeNode). +From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](http://marmbrus.github.io/catalyst/api/current/index.html#catalyst.trees.TreeNode) objects. ```scala scala> query.optimizedPlan res1: catalyst.plans.logical.LogicalPlan = From 356b3211dc04cd84ac58a335d2c9211a2c7c8018 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 Dec 2013 12:07:23 -0800 Subject: [PATCH 249/778] Update README.md --- README.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index c8f8400d1dc3a..f7a6ea7fd983e 100644 --- a/README.md +++ b/README.md @@ -82,7 +82,9 @@ Project {key#0,value#1} MetastoreRelation default, src, None -scala> query.optimizedPlan transform { case Project(projectList, child) if projectList == child.output => child } +scala> res0.optimizedPlan transform { + | case Project(projectList, child) if projectList == child.output => child + | } res2: catalyst.plans.logical.LogicalPlan = Project {key#0,value#1} MetastoreRelation default, src, None From 4b0a888070d6eb018e1c9a1e27d8f40f48d71625 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 Dec 2013 15:01:06 -0800 Subject: [PATCH 250/778] implement string comparison and more casts. --- src/main/scala/expressions/Evaluate.scala | 25 ++++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index 5d31fda7f66c9..e109cf964677e 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -12,8 +12,11 @@ object Evaluate extends Logging { def eval(e: Expression) = Evaluate(e, input) /** - * A set of helper functions that return the correct decendent of [[scala.math.Numeric]] type and do any casting + * A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type and do any casting * necessary of child evaluation. + * + * Instead of matching here we could consider pushing the appropriate Fractional/Integral type into the type objects + * themselves. */ @inline def n1(e: Expression, f: ((Numeric[Any], Any) => Any)): Any = e.dataType match { @@ -102,6 +105,16 @@ object Evaluate extends Logging { /* Comparisons */ case Equals(l, r) => eval(l) == eval(r) + // Strings + case GreaterThan(l, r) if l.dataType == StringType && r.dataType == StringType => + eval(l).asInstanceOf[String] > eval(r).asInstanceOf[String] + case GreaterThanOrEqual(l, r) if l.dataType == StringType && r.dataType == StringType => + eval(l).asInstanceOf[String] >= eval(r).asInstanceOf[String] + case LessThan(l, r) if l.dataType == StringType && r.dataType == StringType => + eval(l).asInstanceOf[String] < eval(r).asInstanceOf[String] + case LessThanOrEqual(l, r) if l.dataType == StringType && r.dataType == StringType => + eval(l).asInstanceOf[String] <= eval(r).asInstanceOf[String] + // Numerics case GreaterThan(l, r) => n2(l, r, _.gt(_, _)) case GreaterThanOrEqual(l, r) => n2(l, r, _.gteq(_, _)) case LessThan(l, r) => n2(l, r, _.lt(_, _)) @@ -110,8 +123,18 @@ object Evaluate extends Logging { case IsNotNull(e) => eval(e) != null /* Casts */ + // toString case Cast(e, StringType) => eval(e).toString + // String => Numeric Types case Cast(e, IntegerType) if e.dataType == StringType => eval(e).asInstanceOf[String].toInt + case Cast(e, IntegerType) if e.dataType == StringType => eval(e).asInstanceOf[String].toDouble + // Numeric Type => Numeric Type + case Cast(e, IntegerType) => n1(e, _.toInt(_)) + case Cast(e, DoubleType) => n1(e, _.toDouble(_)) + case Cast(e, FloatType) => n1(e, _.toFloat(_)) + case Cast(e, LongType) => n1(e, _.toLong(_)) + case Cast(e, ShortType) => n1(e, _.toInt(_).toShort) + case Cast(e, ByteType) => n1(e, _.toInt(_).toByte) /* Boolean Logic */ case Not(c) => !eval(c).asInstanceOf[Boolean] From 06749d078d18e661f5da42c3a127ce74c0833897 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 Dec 2013 15:01:38 -0800 Subject: [PATCH 251/778] add expression enumerations for query plan operators and recursive version of transform expression. --- src/main/scala/plans/QueryPlan.scala | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/src/main/scala/plans/QueryPlan.scala b/src/main/scala/plans/QueryPlan.scala index 50c135ec6b948..d574954af2855 100644 --- a/src/main/scala/plans/QueryPlan.scala +++ b/src/main/scala/plans/QueryPlan.scala @@ -39,4 +39,24 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy if(changed) makeCopy(newArgs) else this } + + /** Returns the result of running [[transformExpressions]] on this node and all its children */ + def transformAllExpressions(rule: PartialFunction[Expression, Expression]): this.type = { + transform { + case q: QueryPlan[_] => q.transformExpressions(rule).asInstanceOf[PlanType] + }.asInstanceOf[this.type] + } + + /** Returns all of the expressions present in this query plan operator. */ + def expressions: Seq[Expression] = { + productIterator.flatMap { + case e: Expression => e :: Nil + case Some(e: Expression) => e :: Nil + case seq: Seq[_] => seq.flatMap { + case e: Expression => e :: Nil + case other => Nil + } + case other => Nil + }.toSeq + } } \ No newline at end of file From 15a53fcd32a3dd50110ceea27322e729664e833e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 Dec 2013 15:02:36 -0800 Subject: [PATCH 252/778] more generic sum calculation and better binding of grouping expressions. --- src/main/scala/shark2/aggregates.scala | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/src/main/scala/shark2/aggregates.scala b/src/main/scala/shark2/aggregates.scala index 3f2eb0abfb29f..44419748901e5 100644 --- a/src/main/scala/shark2/aggregates.scala +++ b/src/main/scala/shark2/aggregates.scala @@ -45,12 +45,10 @@ case class Aggregate(groupingExpressions: Seq[Expression], case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - // TODO: Support other types. - require(expr.dataType == IntegerType) - var sum: Int = _ + var sum = Evaluate(Cast(Literal(0), expr.dataType), Nil) def apply(input: Seq[Seq[Any]]): Unit = - sum += Evaluate(expr, input).asInstanceOf[Int] + sum = Evaluate(Add(Literal(sum), expr), input) def result: Any = sum } @@ -90,9 +88,16 @@ case class Aggregate(groupingExpressions: Seq[Expression], // If any references exist that are not inside agg functions then the must be grouping exprs in this case // we must rebind them to the grouping tuple. if(remainingAttributes.nonEmpty) { - val ordinal = groupingExpressions.indexOf(agg) - if(ordinal == -1) sys.error(s"$agg is not in grouping expressions: $groupingExpressions") - BoundReference(0, ordinal, Alias(impl, "AGGEXPR")().toAttribute) + val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } + + // An exact match with a grouping expression + val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { + case -1 => None + case ordinal => Some(BoundReference(0, ordinal, Alias(impl, "AGGEXPR")().toAttribute) + ) + } + + exactGroupingExpr.getOrElse(sys.error(s"$agg is not in grouping expressions: $groupingExpressions")) } else { impl } From 6cbe8d1638fb9e8bc551cba47ff8c52a4bc786ba Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 Dec 2013 15:02:57 -0800 Subject: [PATCH 253/778] fix bug in side by side, add support for working with unsplit strings --- src/main/scala/util/package.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/scala/util/package.scala b/src/main/scala/util/package.scala index ba7161c2acf2a..19667ea4c5f99 100644 --- a/src/main/scala/util/package.scala +++ b/src/main/scala/util/package.scala @@ -41,13 +41,15 @@ package object util { file } + def sideBySide(left: String, right: String) = sideBySide(left.split("\n"), right.split("\n")) + def sideBySide(left: Seq[String], right: Seq[String]): Seq[String] = { val maxLeftSize = left.map(_.size).max val leftPadded = left ++ Seq.fill(if(left.size < right.size) right.size - left.size else 0)("") val rightPadded = right ++ Seq.fill(if(right.size < left.size) left.size - right.size else 0)("") leftPadded.zip(rightPadded).map { - case (l,r) => (if(l == r) " " else "*") + l + (" " * ((maxLeftSize - left.size) + 3)) + r + case (l,r) => (if(l == r) " " else "!") + l + (" " * ((maxLeftSize - l.size) + 3)) + r } } From f392755ffc2561b8f74b03d20de47f7d13d63f8a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 Dec 2013 15:03:29 -0800 Subject: [PATCH 254/778] Add flatMap to TreeNode --- src/main/scala/trees/TreeNode.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index b937f0564ac95..cdec810c1b680 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -61,6 +61,16 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { ret } + /** + * Returns a Seq by applying a function to all nodes in this tree and using the elements of the resulting + * collections. + */ + def flatMap[A](f: BaseType => TraversableOnce[A]): Seq[A] = { + val ret = new collection.mutable.ArrayBuffer[A]() + foreach(ret ++= f(_)) + ret + } + /** * Returns a Seq containing the result of applying a partial function to all elements in this tree on which the * function is defined. From 03b952635b72c2cc916d614a681915eaaa6e0251 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 Dec 2013 15:11:10 -0800 Subject: [PATCH 255/778] First draft of optimizer tests. --- src/test/scala/OptimizerSuite.scala | 128 ++++++++++++++++++++++++++++ 1 file changed, 128 insertions(+) create mode 100644 src/test/scala/OptimizerSuite.scala diff --git a/src/test/scala/OptimizerSuite.scala b/src/test/scala/OptimizerSuite.scala new file mode 100644 index 0000000000000..ddbca2a606abf --- /dev/null +++ b/src/test/scala/OptimizerSuite.scala @@ -0,0 +1,128 @@ +package catalyst + +import org.scalatest.FunSuite + +import analysis._ +import expressions._ +import optimizer.Optimize +import plans.logical._ +import types._ +import util._ + +/* Implicit conversions for creating query plans */ +import dsl._ + +class OptimizerSuite extends FunSuite { + // Test relations. Feel free to create more. + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + + // Helper functions for comparing plans. + + /** + * Since attribute references are given globally unique ids during analysis we must normalize them to check if two + * different queries are identical. + */ + protected def normalizeExprIds(plan: LogicalPlan) = { + val minId = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)).min + plan transformAllExpressions { + case a: AttributeReference => + AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) + } + } + + /** Fails the test if the two plans do not match */ + protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { + val normalized1 = normalizeExprIds(plan1) + val normalized2 = normalizeExprIds(plan2) + if(normalized1 != normalized2) + fail( + s""" + |== FAIL: Plans do not match === + |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} + """.stripMargin) + } + + // This test already passes. + test("eliminate subqueries") { + val originalQuery = + testRelation + .subquery('y) + .select('a) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = + testRelation + .select('a.attr) + .analyze + + comparePlans(optimized, correctAnswer) + } + + // After this line is unimplemented. + test("simple push down") { + val originalQuery = + testRelation + .select('a) + .where('a === 1) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = + testRelation + .where('a === 1) + .select('a) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("can't push without rewrite") { + val originalQuery = + testRelation + .select('a + 'b as 'e) + .where('e === 1) + + /* Your code here */ + fail("not implemented") + } + + test("joins: push to either side") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = { + x.join(y) + .where("x.b".attr === 1) + .where("y.b".attr === 2) + .analyze + } + + fail("not implemented") + } + + test("joins: can't push down") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = { + x.join(y) + .where("x.b".attr === "y.b".attr) + .analyze + } + val optimized = Optimize(originalQuery.analyze) + + comparePlans(originalQuery, optimized) + } + + test("joins: conjunctive predicates") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = { + x.join(y) + .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("y.a".attr === 1)) + .analyze + } + + fail("not implemented") + } +} \ No newline at end of file From 8c60cc07012b4fdca29f2a4c76dc1d0d51056eff Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 21 Dec 2013 10:22:19 -0800 Subject: [PATCH 256/778] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index f7a6ea7fd983e..4445683d83a75 100644 --- a/README.md +++ b/README.md @@ -2,7 +2,7 @@ catalyst ======== Catalyst is a functional framework for optimizing relational query plans. -More documentation can be found in the project's [scaladoc](http://marmbrus.github.io/catalyst/api/current/#catalyst.package) +More documentation can be found in the project's [scaladoc](http://marmbrus.github.io/catalyst/latest/api/#catalyst.package) Getting Started =============== From 05679b74931dd0f7a65fc06f19269596e4aa714a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 21 Dec 2013 10:05:29 -0800 Subject: [PATCH 257/778] download assembly jar for easy compiling during interview. --- build.sbt | 3 ++- src/main/scala/shark2/hiveOperators.scala | 3 +-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/build.sbt b/build.sbt index 7b961cdc0a3f0..c18f4d008e040 100644 --- a/build.sbt +++ b/build.sbt @@ -1,6 +1,7 @@ resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" -libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.9.0-SNAPSHOT" +// Downloaded as giant jar for easy compiling during interview question. +libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.9.0-SNAPSHOT" from "http://www.eecs.berkeley.edu/~marmbrus/tmp/shark-assembly-0.9.0-SNAPSHOT-hadoop1.0.4.jar" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index 32bc19567aace..23262dd7ad839 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -85,8 +85,7 @@ case class InsertIntoHiveTable(table: MetastoreRelation, child: SharkPlan) table.tableDesc, outputClass, desc, - new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory(), "test.out"), - null) + new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory(), "test.out")) override def otherCopyArgs = sc :: Nil From b50dd0ee818db1376fb24613f3169f073c42efa5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 21 Dec 2013 10:05:45 -0800 Subject: [PATCH 258/778] fix return type of overloaded method --- src/main/scala/util/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/util/package.scala b/src/main/scala/util/package.scala index 19667ea4c5f99..d756c6186d225 100644 --- a/src/main/scala/util/package.scala +++ b/src/main/scala/util/package.scala @@ -41,7 +41,7 @@ package object util { file } - def sideBySide(left: String, right: String) = sideBySide(left.split("\n"), right.split("\n")) + def sideBySide(left: String, right: String): Seq[String] = sideBySide(left.split("\n"), right.split("\n")) def sideBySide(left: Seq[String], right: Seq[String]): Seq[String] = { val maxLeftSize = left.map(_.size).max From 5c518e4a7bbeee37cf8558c1957881ea25178088 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 21 Dec 2013 10:05:53 -0800 Subject: [PATCH 259/778] fix bug in test. --- src/test/scala/OptimizerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/OptimizerSuite.scala b/src/test/scala/OptimizerSuite.scala index ddbca2a606abf..04504451533dc 100644 --- a/src/test/scala/OptimizerSuite.scala +++ b/src/test/scala/OptimizerSuite.scala @@ -110,7 +110,7 @@ class OptimizerSuite extends FunSuite { } val optimized = Optimize(originalQuery.analyze) - comparePlans(originalQuery, optimized) + comparePlans(optimizer.EliminateSubqueries(originalQuery), optimized) } test("joins: conjunctive predicates") { From cbe1ca1fee91bee92f0d2a53d31b9d40a1f54d7e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 21 Dec 2013 22:58:42 -0500 Subject: [PATCH 260/778] add explicit result type to the overloaded sideBySide --- src/main/scala/util/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/util/package.scala b/src/main/scala/util/package.scala index 19667ea4c5f99..d756c6186d225 100644 --- a/src/main/scala/util/package.scala +++ b/src/main/scala/util/package.scala @@ -41,7 +41,7 @@ package object util { file } - def sideBySide(left: String, right: String) = sideBySide(left.split("\n"), right.split("\n")) + def sideBySide(left: String, right: String): Seq[String] = sideBySide(left.split("\n"), right.split("\n")) def sideBySide(left: Seq[String], right: Seq[String]): Seq[String] = { val maxLeftSize = left.map(_.size).max From 191eb7df7aadd17b051f3e5286c706d8fc6b818a Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 22 Dec 2013 14:33:43 -0500 Subject: [PATCH 261/778] First draft of literal evaluation in the optimization phase. TreeNode has been extended to support transform in the post order. So, for an expression, we can evaluate literal from the leaf nodes of this expression tree. For an attribute reference in the expression node, we just leave it as is. --- src/main/scala/optimizer/Optimizer.scala | 16 ++++++++- src/main/scala/plans/QueryPlan.scala | 31 +++++++++++++++++ src/main/scala/trees/TreeNode.scala | 43 ++++++++++++++++++++++++ 3 files changed, 89 insertions(+), 1 deletion(-) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index 91258b79e34d2..3f3ee1bd779e7 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -1,13 +1,16 @@ package catalyst package optimizer +import catalyst.expressions._ import catalyst.plans.logical._ import catalyst.rules._ object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Subqueries", Once, - EliminateSubqueries) :: Nil + EliminateSubqueries) :: + Batch("EvaluateLiterals", Once, + EvaluateLiterals) :: Nil } @@ -15,4 +18,15 @@ object EliminateSubqueries extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Subquery(_, child) => child } +} + +object EvaluateLiterals extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case project @ Project(projectList, child) => project transformExpressionsPostOrder { + case b: BinaryExpression + if b.left.isInstanceOf[Literal] && b.right.isInstanceOf[Literal] => { + Literal(Evaluate(b, Nil)) + } + } + } } \ No newline at end of file diff --git a/src/main/scala/plans/QueryPlan.scala b/src/main/scala/plans/QueryPlan.scala index d574954af2855..a3a4e97eb73ba 100644 --- a/src/main/scala/plans/QueryPlan.scala +++ b/src/main/scala/plans/QueryPlan.scala @@ -40,6 +40,37 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy if(changed) makeCopy(newArgs) else this } + /** + * Runs [[transformPostOrder]] with [[rule]] on all expressions present in this query operator. + * @param rule the rule to be applied to every expression in this operator. + * @return + */ + def transformExpressionsPostOrder(rule: PartialFunction[Expression, Expression]): this.type = { + var changed = false + + @inline def transformExpressionPostOrder(e: Expression) = { + val newE = e.transformPostOrder(rule) + if(newE.id != e.id && newE != e) { + changed = true + newE + } else { + e + } + } + + val newArgs = productIterator.map { + case e: Expression => transformExpressionPostOrder(e) + case Some(e: Expression) => Some(transformExpressionPostOrder(e)) + case seq: Seq[_] => seq.map { + case e: Expression => transformExpressionPostOrder(e) + case other => other + } + case other: AnyRef => other + }.toArray + + if(changed) makeCopy(newArgs) else this + } + /** Returns the result of running [[transformExpressions]] on this node and all its children */ def transformAllExpressions(rule: PartialFunction[Expression, Expression]): this.type = { transform { diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index cdec810c1b680..8f2dbb4e12386 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -131,6 +131,49 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { if(changed) makeCopy(newArgs) else this } + /** + * Returns a copy of this node where [[rule]] has been recursively + * applied first to all of its children and then itself. + * When [[rule]] does not apply to a given node, it is left unchanged. + * @param rule the function use to transform this nodes children + */ + def transformPostOrder(rule: PartialFunction[BaseType, BaseType]): BaseType = { + val afterRuleOnChildren = transformChildrenPostOrder(rule); + if (this fastEquals afterRuleOnChildren) { + rule.applyOrElse(this, identity[BaseType]) + } else { + rule.applyOrElse(afterRuleOnChildren, identity[BaseType]) + } + } + + def transformChildrenPostOrder(rule: PartialFunction[BaseType, BaseType]): this.type = { + var changed = false + val newArgs = productIterator.map { + case arg: TreeNode[_] if(children contains arg) => + val newChild = arg.asInstanceOf[BaseType].transformPostOrder(rule) + if(!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + case args: Seq[_] => args.map { + case arg: TreeNode[_] if(children contains arg) => + val newChild = arg.asInstanceOf[BaseType].transformPostOrder(rule) + if(!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + case other => other + } + case nonChild: AnyRef => nonChild + case null => null + }.toArray + if(changed) makeCopy(newArgs) else this + } + /** * Args to the constructor that should be copied, but not transformed. * These are appended to the transformed args automatically by makeCopy From 73a05fd958f50302a146acde23159d4958bf5b46 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 22 Dec 2013 16:10:16 -0500 Subject: [PATCH 262/778] add a comment to EvaluateLiterals. --- src/main/scala/optimizer/Optimizer.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index 3f3ee1bd779e7..d08ed0f7d5e77 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -20,9 +20,15 @@ object EliminateSubqueries extends Rule[LogicalPlan] { } } +/* +* An optimization rule to evaluate literals appearing in expressions. +* It traverses the expressions in a post order to visit BinaryExpression. +* When it finds both the left child and right child of a node are literals, +* it evaluates the current visiting BinaryExpression. +* */ object EvaluateLiterals extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case project @ Project(projectList, child) => project transformExpressionsPostOrder { + case q: LogicalPlan => q transformExpressionsPostOrder { case b: BinaryExpression if b.left.isInstanceOf[Literal] && b.right.isInstanceOf[Literal] => { Literal(Evaluate(b, Nil)) From ef3321effd0103d67fb48ccad8ab765976612fd4 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 22 Dec 2013 16:17:15 -0500 Subject: [PATCH 263/778] explain the limitation of the current EvaluateLiterals --- src/main/scala/optimizer/Optimizer.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index d08ed0f7d5e77..93f395596d3de 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -25,6 +25,8 @@ object EliminateSubqueries extends Rule[LogicalPlan] { * It traverses the expressions in a post order to visit BinaryExpression. * When it finds both the left child and right child of a node are literals, * it evaluates the current visiting BinaryExpression. +* Because, currently, we evaluate literals based on the structure of the expression +* tree, key+1+1 will not be transformed to key+2. * */ object EvaluateLiterals extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { From 669089cff68b192acd1ed3f0c1e4ee6cef8f3241 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 22 Dec 2013 16:17:34 -0500 Subject: [PATCH 264/778] support Boolean literals --- src/main/scala/expressions/literals.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/expressions/literals.scala b/src/main/scala/expressions/literals.scala index 35012adbf4ba1..c61071befa9e8 100644 --- a/src/main/scala/expressions/literals.scala +++ b/src/main/scala/expressions/literals.scala @@ -12,6 +12,7 @@ object Literal { case b: Byte => Literal(b, ByteType) case s: Short => Literal(s, ShortType) case s: String => Literal(s, StringType) + case b: Boolean => Literal(b, BooleanType) } } From d9d18b44480ea8284f5e27350f27375dff95fa45 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 13:54:03 -0800 Subject: [PATCH 265/778] debug logging implicit. --- src/main/scala/shark2/Transform.scala | 22 ++++ src/main/scala/util/package.scala | 6 ++ src/test/scala/shark2/BigDataBenchmark.scala | 103 +++++++++++++++++++ 3 files changed, 131 insertions(+) create mode 100644 src/main/scala/shark2/Transform.scala create mode 100644 src/test/scala/shark2/BigDataBenchmark.scala diff --git a/src/main/scala/shark2/Transform.scala b/src/main/scala/shark2/Transform.scala new file mode 100644 index 0000000000000..e1a691e178bc4 --- /dev/null +++ b/src/main/scala/shark2/Transform.scala @@ -0,0 +1,22 @@ +package catalyst +package shark2 + +import catalyst.expressions._ +import shark.SharkContext + +/** + * Transforms the input by forking and running the specified script. + * + * @param input the set of expression that should be passed to the script. + * @param script the command that should be executed. + * @param output the attributes that are produced by the script. + */ +case class Transform( + input: Seq[Expression], + script: String, + output: Seq[Attribute], + child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { + override def otherCopyArgs = sc :: Nil + + def execute() = ??? +} \ No newline at end of file diff --git a/src/main/scala/util/package.scala b/src/main/scala/util/package.scala index d756c6186d225..a0052997d9c9c 100644 --- a/src/main/scala/util/package.scala +++ b/src/main/scala/util/package.scala @@ -54,4 +54,10 @@ package object util { } def stringOrNull(a: AnyRef) = if(a == null) null else a.toString + + implicit class debugLogging(a: AnyRef) { + def debugLogging { + org.apache.log4j.Logger.getLogger(a.getClass.getName).setLevel(org.apache.log4j.Level.DEBUG) + } + } } \ No newline at end of file diff --git a/src/test/scala/shark2/BigDataBenchmark.scala b/src/test/scala/shark2/BigDataBenchmark.scala new file mode 100644 index 0000000000000..3a3eb58b5a637 --- /dev/null +++ b/src/test/scala/shark2/BigDataBenchmark.scala @@ -0,0 +1,103 @@ +package catalyst +package shark2 + +import java.io.File + +/** + * A set of test cases based on the big-data-benchmark. + * https://amplab.cs.berkeley.edu/benchmark/ + */ +class BigDataBenchmarkTests extends HiveComaparisionTest { + import TestShark._ + + val testDataDirectory = new File("target/big-data-benchmark-testdata") + + val testTables = Seq( + TestTable( + "rankings", + s""" + |CREATE EXTERNAL TABLE rankings ( + | pageURL STRING, + | pageRank INT, + | avgDuration INT) + | ROW FORMAT DELIMITED FIELDS TERMINATED BY "," + | STORED AS TEXTFILE LOCATION "${new File(testDataDirectory, "rankings").getCanonicalPath}" + """.stripMargin.cmd), + TestTable( + "scratch", + s""" + |CREATE EXTERNAL TABLE scratch ( + | pageURL STRING, + | pageRank INT, + | avgDuration INT) + | ROW FORMAT DELIMITED FIELDS TERMINATED BY "," + | STORED AS TEXTFILE LOCATION "${new File(testDataDirectory, "scratch").getCanonicalPath}" + """.stripMargin.cmd), + TestTable( + "uservisits", + s""" + |CREATE EXTERNAL TABLE uservisits ( + | sourceIP STRING, + | destURL STRING, + | visitDate STRING, + | adRevenue DOUBLE, + | userAgent STRING, + | countryCode STRING, + | languageCode STRING, + | searchWord STRING, + | duration INT) + | ROW FORMAT DELIMITED FIELDS TERMINATED BY "," + | STORED AS TEXTFILE LOCATION "${new File(testDataDirectory, "uservisits").getCanonicalPath}" + """.stripMargin.cmd), + TestTable( + "documents", + s""" + |CREATE EXTERNAL TABLE documents (line STRING) + |STORED AS TEXTFILE + |LOCATION "${new File(testDataDirectory, "crawl").getCanonicalPath}" + """.stripMargin.cmd)) + + testTables.foreach(registerTestTable) + + createQueryTest("query1", + "SELECT pageURL, pageRank FROM rankings WHERE pageRank > 1") + + createQueryTest("query2", + "SELECT SUBSTR(sourceIP, 1, 10), SUM(adRevenue) FROM uservisits GROUP BY SUBSTR(sourceIP, 1, 10)") + + createQueryTest("query3", + """ + |SELECT sourceIP, + | sum(adRevenue) as totalRevenue, + | avg(pageRank) as pageRank + |FROM + | rankings R JOIN + | (SELECT sourceIP, destURL, adRevenue + | FROM uservisits UV + | WHERE UV.visitDate > "1980-01-01" + | AND UV.visitDate < "1980-04-01") + | NUV ON (R.pageURL = NUV.destURL) + |GROUP BY sourceIP + |ORDER BY totalRevenue DESC + |LIMIT 1 + """.stripMargin) + + /* + createQueryTest("documents sanity check", + "SELECT line FROM documents ORDER BY line DESC LIMIT 1000") + + createQueryTest("query4", + """ + |DROP TABLE IF EXISTS url_counts_partial; + |CREATE TABLE url_counts_partial AS + | SELECT TRANSFORM (line) + | USING "python /root/url_count.py" as (sourcePage, + | destPage, count) from documents; + |DROP TABLE IF EXISTS url_counts_total; + |CREATE TABLE url_counts_total AS + | SELECT SUM(count) AS totalCount, destpage + | FROM url_counts_partial GROUP BY destpage; + |SELECT * FROM url_counts_partial; + |SELECT * FROM url_counts_total; + """.stripMargin) */ +} \ No newline at end of file From b6f083ec740cff77df8cb009a08a86116f679bda Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:43:00 -0800 Subject: [PATCH 266/778] support for publishing scala doc to github from sbt --- build.sbt | 22 +++++++++++++++++++--- project/plugins.sbt | 6 ++++++ 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/build.sbt b/build.sbt index 7b961cdc0a3f0..5af26f9b340d4 100644 --- a/build.sbt +++ b/build.sbt @@ -1,3 +1,11 @@ +name := "catalyst" + +organization := "com.databricks" + +version := "0.1-SNAPSHOT" + +scalaVersion := "2.10.3" + resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.9.0-SNAPSHOT" @@ -24,8 +32,6 @@ resolvers ++= Seq( "Datanucleus Repository" at "http://www.datanucleus.org/downloads/maven2") -scalaVersion := "2.10.3" - initialCommands in console := """ import catalyst.analysis._ import catalyst.dsl._ @@ -36,4 +42,14 @@ import catalyst.plans.logical._ import catalyst.rules._ import catalyst.types._ import catalyst.util._ -import catalyst.shark2.TestShark._""" \ No newline at end of file +import catalyst.shark2.TestShark._""" + +site.settings + +ghpages.settings + +git.remoteRepo := "git@github.com:marmbrus/catalyst.git" + +site.settings + +site.includeScaladoc() diff --git a/project/plugins.sbt b/project/plugins.sbt index d06b220d154c3..0b57a5bf6cf1d 100755 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -24,3 +24,9 @@ addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2") resolvers += Resolver.url( "sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) + +resolvers += "jgit-repo" at "http://download.eclipse.org/jgit/maven" + +addSbtPlugin("com.typesafe.sbt" % "sbt-ghpages" % "0.5.2") + +addSbtPlugin("com.typesafe.sbt" % "sbt-site" % "0.7.1") From 320df042a52654cb2c1dedd44244b662af8e2ee5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:43:26 -0800 Subject: [PATCH 267/778] add snapshot repo for databricks (has shark/spark snapshots) --- build.sbt | 1 + 1 file changed, 1 insertion(+) diff --git a/build.sbt b/build.sbt index 5af26f9b340d4..0da02611c54d1 100644 --- a/build.sbt +++ b/build.sbt @@ -31,6 +31,7 @@ resolvers ++= Seq( // For jdo-2 required by Hive < 0.12.0 "Datanucleus Repository" at "http://www.datanucleus.org/downloads/maven2") +resolvers += "Databees" at "http://repository-databricks.forge.cloudbees.com/snapshot/" initialCommands in console := """ import catalyst.analysis._ From 1031b650e84a5b532e3b851342118ad1ebd3e9a9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:45:04 -0800 Subject: [PATCH 268/778] support for case insensitive resolution. --- src/main/scala/analysis/Analyzer.scala | 23 +++++++++++++++++++++-- src/main/scala/shark2/TestShark.scala | 8 ++++++-- 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala index 025766e6886d5..5d40e5d0e8e56 100644 --- a/src/main/scala/analysis/Analyzer.scala +++ b/src/main/scala/analysis/Analyzer.scala @@ -8,15 +8,19 @@ import rules._ /** * A trivial [[Analyzer]] with an [[EmptyCatalog]]. Used for testing when all relations are * already filled in and the analyser needs only to resolve attribute references. + * */ -object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyRegistry) +object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyRegistry, true) -class Analyzer(catalog: Catalog, registry: FunctionRegistry) extends RuleExecutor[LogicalPlan] { +class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean) + extends RuleExecutor[LogicalPlan] { val fixedPoint = FixedPoint(100) val batches = Seq( Batch("LocalRelations", Once, NewLocalRelationInstances), + Batch("CaseInsensitiveAttributeReferences", Once, + (if(caseSensitive) Nil else LowercaseAttributeReferences :: Nil):_*), Batch("Resolution", fixedPoint, ResolveReferences, ResolveRelations, @@ -38,6 +42,21 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry) extends RuleExecuto } } + /** + * Makes attribute naming case insensitive by turning all UnresolvedAttributes to lowercase. + */ + object LowercaseAttributeReferences extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case UnresolvedRelation(name, alias) => UnresolvedRelation(name, alias.map(_.toLowerCase)) + case Subquery(alias, child) => Subquery(alias.toLowerCase, child) + case q: LogicalPlan => q transformExpressions { + case Star(name) => Star(name.map(_.toLowerCase)) + case UnresolvedAttribute(name) => UnresolvedAttribute(name.toLowerCase) + case Alias(c, name) => Alias(c, name.toLowerCase)() + } + } + } + /** * Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s from a logical plan node's children. */ diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index 4c8bd372187fd..b39a7c091fcc0 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -70,7 +70,7 @@ object TestShark extends Logging { /* A catalyst metadata catalog that points to the Shark/Hive Metastore. */ val catalog = new HiveMetastoreCatalog(SharkContext.hiveconf) /* An analyzer that uses the Shark/Hive metastore. */ - val analyze = new Analyzer(catalog, HiveFunctionRegistry) + val analyze = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) /** Sets up the system initially or after a RESET command */ protected def configure() { @@ -190,11 +190,15 @@ object TestShark extends Logging { """.stripMargin.trim } + abstract class CaseSensitiveSharkQuery extends SharkQuery { + override lazy val analyzed = SimpleAnalyzer(parsed) + } + implicit class stringToQuery(str: String) { def q = new SharkSqlQuery(str) } - implicit def logicalToSharkQuery(plan: LogicalPlan) = new SharkQuery { val parsed = plan } + implicit def logicalToSharkQuery(plan: LogicalPlan) = new CaseSensitiveSharkQuery { val parsed = plan } protected case class TestTable(name: String, commands: (()=>Unit)*) From 5cd76d68977f2d42a4ca4b9ec6cbb36da5858068 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:45:49 -0800 Subject: [PATCH 269/778] break up the file based test case code for reuse --- src/test/scala/shark2/HiveCompatability.scala | 48 +++++++++---------- src/test/scala/shark2/HiveQueryFileTest.scala | 44 +++++++++++++++++ 2 files changed, 66 insertions(+), 26 deletions(-) create mode 100644 src/test/scala/shark2/HiveQueryFileTest.scala diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index df6a385ec6be8..d32a2764aed1c 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -6,11 +6,11 @@ import java.io._ import util._ /** - * A framework for running the query tests that are included in hive distribution. + * Runs the test cases that are included in the hive distribution. */ -class HiveCompatability extends HiveComaparisionTest { +class HiveCompatability extends HiveQueryFileTest { /** A list of tests deemed out of scope and thus completely disregarded */ - val blackList = Seq( + override def blackList = Seq( "hook_order", // These tests use hooks that are not on the classpath and thus break all subsequent SQL execution. "hook_context", "mapjoin_hook", @@ -32,14 +32,16 @@ class HiveCompatability extends HiveComaparisionTest { "join18_multi_distinct", // Uses a serde that isn't on the classpath... breaks other tests. - "bucketizedhiveinputformat" + "bucketizedhiveinputformat"//, + // "udaf_covar_pop" + ) /** * The set of tests that are believed to be working in catalyst. Tests not in whiteList * blacklist are implicitly marked as ignored. */ - val whiteList = Seq( + override def whiteList = Seq( "add_part_exist", "auto_join23", "auto_join26", @@ -75,10 +77,13 @@ class HiveCompatability extends HiveComaparisionTest { "input0", "input11", "input11_limit", + "input22", "input24", "input25", "input41", + "input4_cb_delim", "input4_limit", + "input7", "insert1", "join0", "join1", @@ -89,6 +94,7 @@ class HiveCompatability extends HiveComaparisionTest { "join15", "join16", "join17", + "join18", "join19", "join2", "join22", @@ -100,6 +106,7 @@ class HiveCompatability extends HiveComaparisionTest { "join30", "join31", "join34", + "join35", "join36", "join37", "join_casesensitive", @@ -112,9 +119,12 @@ class HiveCompatability extends HiveComaparisionTest { "literal_ints", "literal_string", "mergejoins", - "nestedvirtual", + // "nestedvirtual", "no_hooks", "noalias_subq1", + "nomore_ambiguous_table_col", + "notable_alias1", + "notable_alias2", "nullgroup", "nullgroup2", "nullgroup3", @@ -122,6 +132,7 @@ class HiveCompatability extends HiveComaparisionTest { "nullinput2", "ppd1", "ppd_gby_join", + "ppd_join", "ppd_join3", "ppd_outer_join1", "ppd_outer_join2", @@ -131,6 +142,7 @@ class HiveCompatability extends HiveComaparisionTest { "ppd_random", "ppd_repeated_alias", "ppd_udf_col", + "ppd_union", "progress_1", "quote2", "rename_column", @@ -139,6 +151,7 @@ class HiveCompatability extends HiveComaparisionTest { "show_describe_func_quotes", "show_functions", "smb_mapjoin_10", + "subq2", "tablename_with_select", "udf2", "udf9", @@ -232,6 +245,7 @@ class HiveCompatability extends HiveComaparisionTest { "union16", "union2", "union20", + "union27", "union28", "union29", "union30", @@ -243,24 +257,6 @@ class HiveCompatability extends HiveComaparisionTest { ) // TODO: bundle in jar files... get from classpath - val hiveQueryDir = new File(testShark.hiveDevHome, "ql/src/test/queries/clientpositive") - val testCases = hiveQueryDir.listFiles - val runAll = !(System.getProperty("shark.hive.alltests") == null) - - // Allow the whitelist to be overriden by a system property - val realWhiteList = Option(System.getProperty("shark.hive.whitelist")).map(_.split(",").toSeq).getOrElse(whiteList) - - // Go through all the test cases and add them to scala test. - testCases.foreach { testCase => - val testCaseName = testCase.getName.stripSuffix(".q") - if(blackList contains testCaseName) { - // Do nothing - } else if(realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { - // Build a test case and submit it to scala test framework... - val queriesString = fileToString(testCase) - createQueryTest(testCaseName, queriesString) - } else { - ignore(testCaseName) {} - } - } + lazy val hiveQueryDir = new File(testShark.hiveDevHome, "ql/src/test/queries/clientpositive") + def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f).toMap } diff --git a/src/test/scala/shark2/HiveQueryFileTest.scala b/src/test/scala/shark2/HiveQueryFileTest.scala new file mode 100644 index 0000000000000..668a17be469bf --- /dev/null +++ b/src/test/scala/shark2/HiveQueryFileTest.scala @@ -0,0 +1,44 @@ +package catalyst +package shark2 + +import java.io._ + +import util._ + +/** + * A framework for running the query tests that are listed as a set of text files. + * + * TestSuites that derive from this class must provide a map of testCaseName -> testCaseFiles that should be included. + * Additionally, there is support for whitelisting and blacklisting tests as development progresses. + */ +abstract class HiveQueryFileTest extends HiveComaparisionTest { + /** A list of tests deemed out of scope and thus completely disregarded */ + def blackList: Seq[String] = Nil + + /** + * The set of tests that are believed to be working in catalyst. Tests not in whiteList + * blacklist are implicitly marked as ignored. + */ + def whiteList: Seq[String] = ".*" :: Nil + + def testCases: Map[String, File] + + val runAll = !(System.getProperty("shark.hive.alltests") == null) + + // Allow the whitelist to be overriden by a system property + val realWhiteList = Option(System.getProperty("shark.hive.whitelist")).map(_.split(",").toSeq).getOrElse(whiteList) + + // Go through all the test cases and add them to scala test. + testCases.foreach { + case (testCaseName, testCaseFile) => + if(blackList contains testCaseName) { + // Do nothing + } else if(realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { + // Build a test case and submit it to scala test framework... + val queriesString = fileToString(testCaseFile) + createQueryTest(testCaseName, queriesString) + } else { + ignore(testCaseName) {} + } + } +} \ No newline at end of file From 81659cb7a67d0fe3d7340c3520f688f60841a5a9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:46:44 -0800 Subject: [PATCH 270/778] implement transform operator. --- src/main/scala/frontend/Hive.scala | 31 ++++++++++++++--- src/main/scala/plans/logical/Transform.scala | 20 +++++++++++ src/main/scala/shark2/Transform.scala | 33 ++++++++++++++++++- .../scala/shark2/planningStrategies.scala | 2 ++ 4 files changed, 81 insertions(+), 5 deletions(-) create mode 100644 src/main/scala/plans/logical/Transform.scala diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 1005f1ed81816..a8ab8b3cd68d0 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -361,11 +361,34 @@ object HiveQl { Filter(nodeToExpr(whereExpr), relations) }.getOrElse(relations) - val selectExpressions = nameExpressions(selectClause.getChildren.flatMap(selExprNodeToExpr)) + // Script transformations are expressed as a select clause with a single expression of type TOK_TRANSFORM + val transformation = selectClause.getChildren.head match { + case Token("TOK_SELEXPR", + Token("TOK_TRANSFORM", + Token("TOK_EXPLIST", inputExprs) :: + Token("TOK_SERDE", Nil) :: + Token("TOK_RECORDWRITER", Nil) :: // TODO: Need to support other types of (in/out)put + Token(script, Nil):: + Token("TOK_SERDE", Nil) :: + Token("TOK_RECORDREADER", Nil) :: + Token("TOK_ALIASLIST", aliases) :: Nil) :: Nil) => + + val output = aliases.map { case Token(n, Nil) => AttributeReference(n, StringType)() } + val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) + Some(Transform(inputExprs.map(nodeToExpr), unescapedScript, output, withWhere)) + case _ => None + } + + // The projection of the query can either be a normal projection, an aggregation (if there is a group by) or + // a script transformation. + val withProject = transformation.getOrElse { + // Not a transformation so must be either project or aggregation. + val selectExpressions = nameExpressions(selectClause.getChildren.flatMap(selExprNodeToExpr)) - val withProject = groupByClause match { - case Some(groupBy) => Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withWhere) - case None => Project(selectExpressions, withWhere) + groupByClause match { + case Some(groupBy) => Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withWhere) + case None => Project(selectExpressions, withWhere) + } } require(!(orderByClause.isDefined && sortByClause.isDefined), "Can't have both a sort by and order by.") diff --git a/src/main/scala/plans/logical/Transform.scala b/src/main/scala/plans/logical/Transform.scala new file mode 100644 index 0000000000000..2192a58fe2991 --- /dev/null +++ b/src/main/scala/plans/logical/Transform.scala @@ -0,0 +1,20 @@ +package catalyst +package plans +package logical + +import expressions._ + +/** + * Transforms the input by forking and running the specified script. + * + * @param input the set of expression that should be passed to the script. + * @param script the command that should be executed. + * @param output the attributes that are produced by the script. + */ +case class Transform( + input: Seq[Expression], + script: String, + output: Seq[Attribute], + child: LogicalPlan) extends UnaryNode { + def references = input.flatMap(_.references).toSet +} \ No newline at end of file diff --git a/src/main/scala/shark2/Transform.scala b/src/main/scala/shark2/Transform.scala index e1a691e178bc4..f8f0142db7cc4 100644 --- a/src/main/scala/shark2/Transform.scala +++ b/src/main/scala/shark2/Transform.scala @@ -3,6 +3,9 @@ package shark2 import catalyst.expressions._ import shark.SharkContext +import java.io.{InputStreamReader, BufferedReader} + +import collection.JavaConversions._ /** * Transforms the input by forking and running the specified script. @@ -18,5 +21,33 @@ case class Transform( child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil - def execute() = ??? + def execute() = { + child.execute().mapPartitions { partition => + val cmd = List("/bin/bash", "-c", script) + val builder = new ProcessBuilder(cmd) + val proc = builder.start() + val inputStream = proc.getInputStream + val outputStream = proc.getOutputStream + val reader = new BufferedReader(new InputStreamReader(inputStream)) + + // TODO: This should be exposed as an iterator instead of reading in all the data at once for a partition. + val outputLines = collection.mutable.ArrayBuffer[IndexedSeq[Any]]() + val readerThread = new Thread("Transform OutoutReader") { + override def run() { + var curLine = reader.readLine() + while(curLine != null) { + outputLines += curLine.split("\t") + curLine = reader.readLine() + } + } + } + readerThread.start() + partition + .map(row => input.map(Evaluate(_, Vector(row)))) + .map(_.mkString("", "\t", "\n").getBytes).foreach(outputStream.write) + outputStream.close() + readerThread.join() + outputLines.toIterator + } + } } \ No newline at end of file diff --git a/src/main/scala/shark2/planningStrategies.scala b/src/main/scala/shark2/planningStrategies.scala index b6efad849989c..c2a75e3ee0449 100644 --- a/src/main/scala/shark2/planningStrategies.scala +++ b/src/main/scala/shark2/planningStrategies.scala @@ -145,6 +145,8 @@ abstract trait PlanningStrategies { shark2.StopAfter(Evaluate(limit, Nil).asInstanceOf[Int], planLater(child))(sc) :: Nil case logical.Union(left, right) => shark2.Union(planLater(left), planLater(right))(sc) :: Nil + case logical.Transform(input, script, output, child) => + shark2.Transform(input, script, output, planLater(child))(sc) :: Nil case _ => Nil } } From 34b30faaf3a95f9dff4bea3475d0c2f1d54cc56d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:47:09 -0800 Subject: [PATCH 271/778] not all attributes need to be bound (e.g. output attributes that are contained in non-leaf operators.) --- src/main/scala/expressions/BoundAttribute.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/main/scala/expressions/BoundAttribute.scala b/src/main/scala/expressions/BoundAttribute.scala index 84d7947c40fe0..f03c39141b1bd 100644 --- a/src/main/scala/expressions/BoundAttribute.scala +++ b/src/main/scala/expressions/BoundAttribute.scala @@ -37,10 +37,14 @@ object BindReferences extends Rule[SharkPlan] { nonLeaf.transformExpressions { case a: AttributeReference => attachTree(a, "Binding attribute") { val inputTuple = nonLeaf.children.indexWhere(_.output contains a) - val ordinal = nonLeaf.children(inputTuple).output.indexWhere(_ == a) - logger.debug(s"Binding $a to $inputTuple.$ordinal given input ${nonLeaf.children.map(_.output.mkString("{", ",", "}")).mkString(",")}") - assert(ordinal != -1, "Reference not found in child plan") - BoundReference(inputTuple, ordinal, a) + val ordinal = if(inputTuple == -1) -1 else nonLeaf.children(inputTuple).output.indexWhere(_ == a) + if(ordinal == -1) { + logger.debug(s"No binding found for $a given input ${nonLeaf.children.map(_.output.mkString("{", ",", "}")).mkString(",")}") + a + } else { + logger.debug(s"Binding $a to $inputTuple.$ordinal given input ${nonLeaf.children.map(_.output.mkString("{", ",", "}")).mkString(",")}") + BoundReference(inputTuple, ordinal, a) + } } } } From d42b725abaa08f159f0517ad80acc37dfc4a3239 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:47:19 -0800 Subject: [PATCH 272/778] Sum of strings requires cast --- src/main/scala/analysis/typeCoercion.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/scala/analysis/typeCoercion.scala b/src/main/scala/analysis/typeCoercion.scala index 24f9587218041..518908f27b345 100644 --- a/src/main/scala/analysis/typeCoercion.scala +++ b/src/main/scala/analysis/typeCoercion.scala @@ -49,6 +49,9 @@ object PromoteTypes extends Rule[LogicalPlan] { b.makeCopy(Array(Cast(b.left, IntegerType), b.right)) case b: BinaryExpression if b.left.dataType == IntegerType && b.right.dataType == StringType => b.makeCopy(Array(b.left, Cast(b.right, IntegerType))) + + case Sum(e) if e.dataType == StringType => + Sum(Cast(e, IntegerType)) } } } \ No newline at end of file From 5f54f03a84dc3560580c7e1e1aea2239ac0bcdf3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:47:32 -0800 Subject: [PATCH 273/778] parsing for ASC --- src/main/scala/frontend/Hive.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index a8ab8b3cd68d0..a7243fb40f618 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -449,6 +449,8 @@ object HiveQl { def nodeToSortOrder(node: Node): SortOrder = node match { case Token("TOK_TABSORTCOLNAMEASC", sortExpr :: Nil) => SortOrder(nodeToExpr(sortExpr), Ascending) + case Token("TOK_TABSORTCOLNAMEDESC", sortExpr :: Nil) => + SortOrder(nodeToExpr(sortExpr), Descending) case a: ASTNode => throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") From 7d7fa9f2c9755121f44842b16ebcd742d75f6363 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:52:22 -0800 Subject: [PATCH 274/778] support for create table as --- src/main/scala/frontend/Hive.scala | 22 +++++++- .../scala/plans/logical/basicOperators.scala | 9 ++- src/main/scala/shark2/MetastoreCatalog.scala | 55 +++++++++++++++++-- src/main/scala/shark2/TestShark.scala | 2 +- src/test/scala/shark2/HiveQueryTests.scala | 8 +++ 5 files changed, 85 insertions(+), 11 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index a7243fb40f618..9874c86433507 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -24,6 +24,12 @@ abstract class Command extends LeafNode { def output = Seq.empty } +/** + * Used when we need to start parsing the AST before deciding that we are going to pass the command back for Hive to + * execute natively. Will be replaced with a native command that contains the cmd string. + */ +case object NativePlaceholder extends Command + /** * Returned for commands supported by the parser, but not catalyst. In general these are DDL * commands that are passed directly to Hive. @@ -91,7 +97,6 @@ object HiveQl { "TOK_CREATEDATABASE", "TOK_CREATEFUNCTION", "TOK_CREATEINDEX", - "TOK_CREATETABLE", "TOK_DROPDATABASE", "TOK_DROPINDEX", "TOK_DROPTABLE", @@ -219,7 +224,10 @@ object HiveQl { if(nativeCommands contains tree.getText) NativeCommand(sql) else - nodeToPlan(tree) + nodeToPlan(tree) match { + case NativePlaceholder => NativeCommand(sql) + case other => other + } } } catch { case e: Exception => throw new ParseException(sql, e) @@ -343,6 +351,16 @@ object HiveQl { val Some(query) :: _ :: _ :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) // TODO: support EXTENDED? ExplainCommand(nodeToPlan(query)) + + case Token("TOK_CREATETABLE", children) if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty => + val (Some(Token("TOK_TABNAME", Token(tableName, Nil) :: Nil)) :: + _ /* likeTable */ :: + Some(query) :: Nil) = getClauses(Seq("TOK_TABNAME", "TOK_LIKETABLE", "TOK_QUERY"), children) + InsertIntoCreatedTable(tableName, nodeToPlan(query)) + + // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. + case Token("TOK_CREATETABLE", _) => NativePlaceholder + case Token("TOK_QUERY", Token("TOK_FROM", fromClause :: Nil) :: Token("TOK_INSERT", insertClauses) :: Nil) => diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index b2e08e45e8657..ec4a7a8da343e 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -30,11 +30,16 @@ case class Join( def output = left.output ++ right.output } -case class InsertIntoTable(table: BaseRelation, child: logical.LogicalPlan) extends LogicalPlan { +case class InsertIntoTable(table: BaseRelation, child: LogicalPlan) extends LogicalPlan { // The table being inserted into is a child for the purposes of transformations. def children = table :: child :: Nil def references = Set.empty - def output = Seq.empty + def output = child.output +} + +case class InsertIntoCreatedTable(tableName: String, child: LogicalPlan) extends UnaryNode { + def references = Set.empty + def output = child.output } case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { diff --git a/src/main/scala/shark2/MetastoreCatalog.scala b/src/main/scala/shark2/MetastoreCatalog.scala index c5255c4904f11..0ca367c465304 100644 --- a/src/main/scala/shark2/MetastoreCatalog.scala +++ b/src/main/scala/shark2/MetastoreCatalog.scala @@ -2,20 +2,24 @@ package catalyst package shark2 import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table} +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table, StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.metastore.HiveMetaStoreClient +import org.apache.hadoop.hive.ql.plan.TableDesc + import analysis.Catalog import expressions._ +import plans.logical._ +import rules._ import types._ import collection.JavaConversions._ -import org.apache.hadoop.hive.ql.plan.TableDesc + class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { val client = new HiveMetaStoreClient(hiveConf) - def lookupRelation(name: String, alias: Option[String]): plans.logical.LogicalPlan = { + def lookupRelation(name: String, alias: Option[String]): BaseRelation = { val (databaseName, tableName) = name.split("\\.") match { case Array(tableOnly) => ("default", tableOnly) case Array(db, table) => (db, table) @@ -28,7 +32,44 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { else Nil - MetastoreRelation(databaseName, tableName, alias)(table, partitions) + // Since HiveQL is case insensitive for table names we make them all lowercase. + MetastoreRelation(databaseName.toLowerCase, tableName.toLowerCase, alias)(table, partitions) + } + + /** + * Creates any tables required for query execution. For example, because of a CREATE TABLE X AS statement. + */ + object CreateTables extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case InsertIntoCreatedTable(name, child) => + val (databaseName, tableName) = name.split("\\.") match { + case Array(tableOnly) => ("default", tableOnly) + case Array(db, table) => (db, table) + } + + val table = new Table() + val schema = child.output.map(attr => new FieldSchema(attr.name, "string", "")) + + table.setDbName(databaseName) + table.setTableName(tableName) + val sd = new StorageDescriptor() + table.setSd(sd) + sd.setCols(schema) + + // TODO: THESE ARE ALL DEFAULTS, WE NEED TO PARSE / UNDERSTAND the output specs. + sd.setCompressed(false) + sd.setParameters(Map[String, String]()) + sd.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") + sd.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") + val serDeInfo = new SerDeInfo() + serDeInfo.setName(tableName) + serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") + serDeInfo.setParameters(Map[String, String]()) + sd.setSerdeInfo(serDeInfo) + client.createTable(table) + + InsertIntoTable(lookupRelation(tableName, None), child) + } } } @@ -37,11 +78,12 @@ object HiveMetatoreTypes { metastoreType match { case "string" => StringType case "int" => IntegerType + case "double" => DoubleType } } case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String])(val table: Table, val partitions: Seq[Partition]) - extends plans.logical.BaseRelation { + extends BaseRelation { def hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) def hiveQlPartitions = partitions.map(new org.apache.hadoop.hive.ql.metadata.Partition(hiveQlTable, _)) @@ -65,4 +107,5 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt // Must be a stable value since new attributes are born here. val output = partitionKeys ++ table.getSd.getCols.map(_.toAttribute) -} \ No newline at end of file +} + diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index b39a7c091fcc0..45243820e7b2d 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -151,7 +151,7 @@ object TestShark extends Logging { // Proceed with analysis. analyze(parsed) } - lazy val optimizedPlan = Optimize(analyzed) + lazy val optimizedPlan = Optimize(catalog.CreateTables(analyzed)) // TODO: Don't just pick the first one... lazy val sharkPlan = TrivalPlanner(optimizedPlan).next() lazy val executedPlan = PrepareForExecution(sharkPlan) diff --git a/src/test/scala/shark2/HiveQueryTests.scala b/src/test/scala/shark2/HiveQueryTests.scala index 0960ad502bd6d..bab8bdb43895a 100644 --- a/src/test/scala/shark2/HiveQueryTests.scala +++ b/src/test/scala/shark2/HiveQueryTests.scala @@ -44,4 +44,12 @@ class HiveQueryTests extends HiveComaparisionTest { createQueryTest("partitioned table scan", "SELECT * FROM srcpart") + createQueryTest("create table as", + """ + |CREATE TABLE createdtable AS SELECT * FROM src; + |SELECT * FROM createdtable + """.stripMargin) + + createQueryTest("transform", + "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src") } \ No newline at end of file From cc9a95789ec0c159b52ec7cd9ba73a64a0fc7eba Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:57:22 -0800 Subject: [PATCH 275/778] support for creating test tables outside of TestShark --- src/main/scala/shark2/TestShark.scala | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index 45243820e7b2d..07ff0be93b7f1 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -16,6 +16,7 @@ import org.apache.spark.rdd.RDD import collection.JavaConversions._ import org.apache.hadoop.hive.metastore.MetaStoreUtils +import scala.collection.mutable /** * A locally running test instance of spark. The lifecycle for a given query is managed by the inner class @@ -145,7 +146,7 @@ object TestShark extends Logging { lazy val analyzed = { // Make sure any test tables referenced are loaded. val referencedTables = parsed collect { case UnresolvedRelation(name, _) => name.split("\\.").last } - val referencedTestTables = referencedTables.filter(testTableNames.contains) + val referencedTestTables = referencedTables.filter(testTables.contains) logger.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) // Proceed with analysis. @@ -200,15 +201,20 @@ object TestShark extends Logging { implicit def logicalToSharkQuery(plan: LogicalPlan) = new CaseSensitiveSharkQuery { val parsed = plan } - protected case class TestTable(name: String, commands: (()=>Unit)*) + case class TestTable(name: String, commands: (()=>Unit)*) + implicit class SqlCmd(sql: String) { def cmd = () => sql.q.stringResult(): Unit} - protected implicit class SqlCmd(sql: String) { def cmd = () => sql.q.stringResult(): Unit} /** * A list of test tables and the DDL required to initialize them. A test table is loaded on demand when a query * are run against it. */ - val testTables = Seq( + val testTables = new mutable.HashMap[String, TestTable]() + def registerTestTable(testTable: TestTable) = testTables += (testTable.name -> testTable) + + // The test tables that are defined in the Hive QTestUtil. + // https://github.com/apache/hive/blob/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java + val hiveQTestUtilTables = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)".cmd, s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' INTO TABLE src".cmd), @@ -231,13 +237,14 @@ object TestShark extends Logging { } }) ) - protected val testTableNames = testTables.map(_.name).toSet + + hiveQTestUtilTables.foreach(registerTestTable) private val loadedTables = new collection.mutable.HashSet[String] def loadTestTable(name: String) { if(!(loadedTables contains name)) { logger.info(s"Loading test table $name") - val createCmds = testTables.find(_.name == name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) + val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) loadedTables += name } From b1d1843fd47fb00c2d7745ca00a15ddedd42bb9e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 22 Dec 2013 15:58:59 -0800 Subject: [PATCH 276/778] more work on big data benchmark tests. --- src/test/scala/shark2/BigDataBenchmark.scala | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/src/test/scala/shark2/BigDataBenchmark.scala b/src/test/scala/shark2/BigDataBenchmark.scala index 3a3eb58b5a637..f12b0d9f3e946 100644 --- a/src/test/scala/shark2/BigDataBenchmark.scala +++ b/src/test/scala/shark2/BigDataBenchmark.scala @@ -11,6 +11,7 @@ class BigDataBenchmarkTests extends HiveComaparisionTest { import TestShark._ val testDataDirectory = new File("target/big-data-benchmark-testdata") + require(testDataDirectory.exists()) val testTables = Seq( TestTable( @@ -82,22 +83,22 @@ class BigDataBenchmarkTests extends HiveComaparisionTest { |LIMIT 1 """.stripMargin) - /* - createQueryTest("documents sanity check", - "SELECT line FROM documents ORDER BY line DESC LIMIT 1000") - createQueryTest("query4", """ |DROP TABLE IF EXISTS url_counts_partial; |CREATE TABLE url_counts_partial AS | SELECT TRANSFORM (line) - | USING "python /root/url_count.py" as (sourcePage, + | USING 'python target/url_count.py' as (sourcePage, | destPage, count) from documents; |DROP TABLE IF EXISTS url_counts_total; |CREATE TABLE url_counts_total AS | SELECT SUM(count) AS totalCount, destpage - | FROM url_counts_partial GROUP BY destpage; - |SELECT * FROM url_counts_partial; - |SELECT * FROM url_counts_total; - """.stripMargin) */ + | FROM url_counts_partial GROUP BY destpage + |-- The following queries run, but generate different results in HIVE likely because the UDF is not deterministic + |-- given different input splits. + |-- SELECT CAST(SUM(count) AS INT) FROM url_counts_partial + |-- SELECT COUNT(*) FROM url_counts_partial + |-- SELECT * FROM url_counts_partial + |-- SELECT * FROM url_counts_total + """.stripMargin) } \ No newline at end of file From 468667f8a232f5257de8f5635ca022db2de5a03e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 22 Dec 2013 14:33:43 -0500 Subject: [PATCH 277/778] First draft of literal evaluation in the optimization phase. TreeNode has been extended to support transform in the post order. So, for an expression, we can evaluate literal from the leaf nodes of this expression tree. For an attribute reference in the expression node, we just leave it as is. --- src/main/scala/optimizer/Optimizer.scala | 16 ++++++++- src/main/scala/plans/QueryPlan.scala | 31 +++++++++++++++++ src/main/scala/trees/TreeNode.scala | 43 ++++++++++++++++++++++++ 3 files changed, 89 insertions(+), 1 deletion(-) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index 91258b79e34d2..3f3ee1bd779e7 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -1,13 +1,16 @@ package catalyst package optimizer +import catalyst.expressions._ import catalyst.plans.logical._ import catalyst.rules._ object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Subqueries", Once, - EliminateSubqueries) :: Nil + EliminateSubqueries) :: + Batch("EvaluateLiterals", Once, + EvaluateLiterals) :: Nil } @@ -15,4 +18,15 @@ object EliminateSubqueries extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Subquery(_, child) => child } +} + +object EvaluateLiterals extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case project @ Project(projectList, child) => project transformExpressionsPostOrder { + case b: BinaryExpression + if b.left.isInstanceOf[Literal] && b.right.isInstanceOf[Literal] => { + Literal(Evaluate(b, Nil)) + } + } + } } \ No newline at end of file diff --git a/src/main/scala/plans/QueryPlan.scala b/src/main/scala/plans/QueryPlan.scala index d574954af2855..a3a4e97eb73ba 100644 --- a/src/main/scala/plans/QueryPlan.scala +++ b/src/main/scala/plans/QueryPlan.scala @@ -40,6 +40,37 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy if(changed) makeCopy(newArgs) else this } + /** + * Runs [[transformPostOrder]] with [[rule]] on all expressions present in this query operator. + * @param rule the rule to be applied to every expression in this operator. + * @return + */ + def transformExpressionsPostOrder(rule: PartialFunction[Expression, Expression]): this.type = { + var changed = false + + @inline def transformExpressionPostOrder(e: Expression) = { + val newE = e.transformPostOrder(rule) + if(newE.id != e.id && newE != e) { + changed = true + newE + } else { + e + } + } + + val newArgs = productIterator.map { + case e: Expression => transformExpressionPostOrder(e) + case Some(e: Expression) => Some(transformExpressionPostOrder(e)) + case seq: Seq[_] => seq.map { + case e: Expression => transformExpressionPostOrder(e) + case other => other + } + case other: AnyRef => other + }.toArray + + if(changed) makeCopy(newArgs) else this + } + /** Returns the result of running [[transformExpressions]] on this node and all its children */ def transformAllExpressions(rule: PartialFunction[Expression, Expression]): this.type = { transform { diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/trees/TreeNode.scala index cdec810c1b680..8f2dbb4e12386 100644 --- a/src/main/scala/trees/TreeNode.scala +++ b/src/main/scala/trees/TreeNode.scala @@ -131,6 +131,49 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { if(changed) makeCopy(newArgs) else this } + /** + * Returns a copy of this node where [[rule]] has been recursively + * applied first to all of its children and then itself. + * When [[rule]] does not apply to a given node, it is left unchanged. + * @param rule the function use to transform this nodes children + */ + def transformPostOrder(rule: PartialFunction[BaseType, BaseType]): BaseType = { + val afterRuleOnChildren = transformChildrenPostOrder(rule); + if (this fastEquals afterRuleOnChildren) { + rule.applyOrElse(this, identity[BaseType]) + } else { + rule.applyOrElse(afterRuleOnChildren, identity[BaseType]) + } + } + + def transformChildrenPostOrder(rule: PartialFunction[BaseType, BaseType]): this.type = { + var changed = false + val newArgs = productIterator.map { + case arg: TreeNode[_] if(children contains arg) => + val newChild = arg.asInstanceOf[BaseType].transformPostOrder(rule) + if(!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + case args: Seq[_] => args.map { + case arg: TreeNode[_] if(children contains arg) => + val newChild = arg.asInstanceOf[BaseType].transformPostOrder(rule) + if(!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + case other => other + } + case nonChild: AnyRef => nonChild + case null => null + }.toArray + if(changed) makeCopy(newArgs) else this + } + /** * Args to the constructor that should be copied, but not transformed. * These are appended to the transformed args automatically by makeCopy From 37817b5d6c795f579a7a6d1e30489aa7c9f9f284 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 22 Dec 2013 16:10:16 -0500 Subject: [PATCH 278/778] add a comment to EvaluateLiterals. --- src/main/scala/optimizer/Optimizer.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index 3f3ee1bd779e7..d08ed0f7d5e77 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -20,9 +20,15 @@ object EliminateSubqueries extends Rule[LogicalPlan] { } } +/* +* An optimization rule to evaluate literals appearing in expressions. +* It traverses the expressions in a post order to visit BinaryExpression. +* When it finds both the left child and right child of a node are literals, +* it evaluates the current visiting BinaryExpression. +* */ object EvaluateLiterals extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case project @ Project(projectList, child) => project transformExpressionsPostOrder { + case q: LogicalPlan => q transformExpressionsPostOrder { case b: BinaryExpression if b.left.isInstanceOf[Literal] && b.right.isInstanceOf[Literal] => { Literal(Evaluate(b, Nil)) From eaac9e2a449fae6b96c80ecae9dd19a682b66a46 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 22 Dec 2013 16:17:15 -0500 Subject: [PATCH 279/778] explain the limitation of the current EvaluateLiterals --- src/main/scala/optimizer/Optimizer.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index d08ed0f7d5e77..93f395596d3de 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -25,6 +25,8 @@ object EliminateSubqueries extends Rule[LogicalPlan] { * It traverses the expressions in a post order to visit BinaryExpression. * When it finds both the left child and right child of a node are literals, * it evaluates the current visiting BinaryExpression. +* Because, currently, we evaluate literals based on the structure of the expression +* tree, key+1+1 will not be transformed to key+2. * */ object EvaluateLiterals extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { From dedbf0c3909d6a83a99a35b4e5a80cfc1fbd38e3 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 22 Dec 2013 16:17:34 -0500 Subject: [PATCH 280/778] support Boolean literals --- src/main/scala/expressions/literals.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/expressions/literals.scala b/src/main/scala/expressions/literals.scala index 35012adbf4ba1..c61071befa9e8 100644 --- a/src/main/scala/expressions/literals.scala +++ b/src/main/scala/expressions/literals.scala @@ -12,6 +12,7 @@ object Literal { case b: Byte => Literal(b, ByteType) case s: Short => Literal(s, ShortType) case s: String => Literal(s, StringType) + case b: Boolean => Literal(b, BooleanType) } } From 21976ae596e308dd2314e1ac1b6d09633bd0f97b Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 23 Dec 2013 15:40:09 -0500 Subject: [PATCH 281/778] update --- src/main/scala/optimizer/Optimizer.scala | 34 ++++++- src/main/scala/shark2/TestShark.scala | 4 +- src/main/scala/shark2/hiveOperators.scala | 3 +- src/test/scala/OptimizerSuite.scala | 94 +++++++++++--------- src/test/scala/shark2/BigDataBenchmark.scala | 2 +- 5 files changed, 91 insertions(+), 46 deletions(-) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index 93f395596d3de..1cb5c82466c07 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -4,13 +4,16 @@ package optimizer import catalyst.expressions._ import catalyst.plans.logical._ import catalyst.rules._ +import catalyst.types.BooleanType object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Subqueries", Once, EliminateSubqueries) :: Batch("EvaluateLiterals", Once, - EvaluateLiterals) :: Nil + EvaluateLiterals, + EvaluateLiteralsInAndOr + ) :: Nil } @@ -37,4 +40,33 @@ object EvaluateLiterals extends Rule[LogicalPlan] { } } } +} + +/* +* After EvaluateLiterals, for an And or an Or, if either side of +* this predicate is a Literal Boolean, we can further evaluate this predicate. +* */ +object EvaluateLiteralsInAndOr extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressionsPostOrder { + case b @ And(left, right) => { + (left, right) match { + case (Literal(true, BooleanType), r) => r + case (l, Literal(true, BooleanType)) => l + case (Literal(false, BooleanType), _) => Literal(false) + case (_, Literal(false, BooleanType)) => Literal(false) + case (_, _) => b + } + } + case b @ Or(left, right) => { + (left, right) match { + case (Literal(true, BooleanType), _) => Literal(true) + case (_, Literal(true, BooleanType)) => Literal(true) + case (Literal(false, BooleanType), r) => r + case (l, Literal(false, BooleanType)) => l + case (_, _) => b + } + } + } + } } \ No newline at end of file diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index 4c8bd372187fd..630deba4867df 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -196,10 +196,10 @@ object TestShark extends Logging { implicit def logicalToSharkQuery(plan: LogicalPlan) = new SharkQuery { val parsed = plan } - protected case class TestTable(name: String, commands: (()=>Unit)*) + case class TestTable(name: String, commands: (()=>Unit)*) - protected implicit class SqlCmd(sql: String) { def cmd = () => sql.q.stringResult(): Unit} + implicit class SqlCmd(sql: String) { def cmd = () => sql.q.stringResult(): Unit} /** * A list of test tables and the DDL required to initialize them. A test table is loaded on demand when a query * are run against it. diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index 23262dd7ad839..32bc19567aace 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -85,7 +85,8 @@ case class InsertIntoHiveTable(table: MetastoreRelation, child: SharkPlan) table.tableDesc, outputClass, desc, - new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory(), "test.out")) + new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory(), "test.out"), + null) override def otherCopyArgs = sc :: Nil diff --git a/src/test/scala/OptimizerSuite.scala b/src/test/scala/OptimizerSuite.scala index 04504451533dc..e57e10db1e11e 100644 --- a/src/test/scala/OptimizerSuite.scala +++ b/src/test/scala/OptimizerSuite.scala @@ -58,71 +58,83 @@ class OptimizerSuite extends FunSuite { comparePlans(optimized, correctAnswer) } - // After this line is unimplemented. - test("simple push down") { + /* + * Unit tests for evaluating literals in expressions. + * */ + test("Evaluating Literals Test 1: expressions only have literals") { val originalQuery = testRelation - .select('a) - .where('a === 1) + .select(Literal(2) + Literal(3) + Literal(4) as Symbol("2+3+4"), + Literal(2) * Literal(3) + Literal(4) as Symbol("2*3+4"), + Literal(2) * (Literal(3) + Literal(4)) as Symbol("2*(3+4)")) + .where(Literal(1) === Literal(1) && + Literal(2) > Literal(3) || + Literal(3) > Literal(2) ) + .groupBy(Literal(2) * Literal(3) - Literal(6) / (Literal(4) - Literal(2)))(Literal(9) / Literal(3) as Symbol("9/3")) val optimized = Optimize(originalQuery.analyze) + val correctAnswer = testRelation - .where('a === 1) - .select('a) + .select(Literal(9) as Symbol("2+3+4"), + Literal(10) as Symbol("2*3+4"), + Literal(14) as Symbol("2*(3+4)")) + .where(Literal(true)) + .groupBy(Literal(3))(Literal(3) as Symbol("9/3")) .analyze comparePlans(optimized, correctAnswer) } - test("can't push without rewrite") { + test("Evaluating Literals Test 2: expressions have attribute references and literals in" + + "arithmetic operations") { val originalQuery = testRelation - .select('a + 'b as 'e) - .where('e === 1) + .select(Literal(2) + Literal(3) + 'a as Symbol("c1"), + 'a + Literal(2) + Literal(3) as Symbol("c2"), + Literal(2) * 'a + Literal(4) as Symbol("c3"), + 'a * (Literal(3) + Literal(4)) as Symbol("c4")) - /* Your code here */ - fail("not implemented") - } - - test("joins: push to either side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val optimized = Optimize(originalQuery.analyze) - val originalQuery = { - x.join(y) - .where("x.b".attr === 1) - .where("y.b".attr === 2) + val correctAnswer = + testRelation + .select(Literal(5) + 'a as Symbol("c1"), + 'a + Literal(2) + Literal(3) as Symbol("c2"), + Literal(2) * 'a + Literal(4) as Symbol("c3"), + 'a * (Literal(7)) as Symbol("c4")) .analyze - } - fail("not implemented") + comparePlans(optimized, correctAnswer) } - test("joins: can't push down") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + test("Evaluating Literals Test 3: expressions have attribute references and literals in" + + "predicates") { + val originalQuery = + testRelation + .where((('a > 1 && Literal(1) === Literal(1)) || + ('a < 10 && Literal(1) === Literal(2)) || + (Literal(1) === Literal(1) && 'b > 1) || + (Literal(1) === Literal(2) && 'b < 10)) && + (('a > 1 || Literal(1) === Literal(1)) && + ('a < 10 || Literal(1) === Literal(2)) && + (Literal(1) === Literal(1) || 'b > 1) && + (Literal(1) === Literal(2) || 'b < 10))) - val originalQuery = { - x.join(y) - .where("x.b".attr === "y.b".attr) - .analyze - } - val optimized = Optimize(originalQuery.analyze) + println(originalQuery.analyze) - comparePlans(optimizer.EliminateSubqueries(originalQuery), optimized) - } + val optimized = Optimize(originalQuery.analyze) - test("joins: conjunctive predicates") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + println(optimized) - val originalQuery = { - x.join(y) - .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("y.a".attr === 1)) + val correctAnswer = + testRelation + .where(('a > 1 || + 'b > 1) && + ('a < 10 && + 'b < 10)) .analyze - } - fail("not implemented") + comparePlans(optimized, correctAnswer) } } \ No newline at end of file diff --git a/src/test/scala/shark2/BigDataBenchmark.scala b/src/test/scala/shark2/BigDataBenchmark.scala index 3a3eb58b5a637..43781d3d5ce99 100644 --- a/src/test/scala/shark2/BigDataBenchmark.scala +++ b/src/test/scala/shark2/BigDataBenchmark.scala @@ -57,7 +57,7 @@ class BigDataBenchmarkTests extends HiveComaparisionTest { |LOCATION "${new File(testDataDirectory, "crawl").getCanonicalPath}" """.stripMargin.cmd)) - testTables.foreach(registerTestTable) + // testTables.foreach(registerTestTable) createQueryTest("query1", "SELECT pageURL, pageRank FROM rankings WHERE pageRank > 1") From ae0024a976a45c6d235ee9d2ef56a7679578acaf Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 23 Dec 2013 15:42:19 -0500 Subject: [PATCH 282/778] update --- src/main/scala/optimizer/Optimizer.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index e6c9b4d1d66a9..9abc64e978d22 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -11,14 +11,9 @@ object Optimize extends RuleExecutor[LogicalPlan] { Batch("Subqueries", Once, EliminateSubqueries) :: Batch("EvaluateLiterals", Once, -<<<<<<< HEAD EvaluateLiterals, EvaluateLiteralsInAndOr ) :: Nil -======= - EvaluateLiterals) :: Nil ->>>>>>> origin/evalauteLiteralsInExpressions - } object EliminateSubqueries extends Rule[LogicalPlan] { From a045c9c5dc0bcb9303dc9a31885b387f33e969c6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 23 Dec 2013 16:00:29 -0800 Subject: [PATCH 283/778] SparkAggregate doesn't actually support sum right now. --- src/main/scala/shark2/planningStrategies.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/scala/shark2/planningStrategies.scala b/src/main/scala/shark2/planningStrategies.scala index c2a75e3ee0449..2ab5251da9495 100644 --- a/src/main/scala/shark2/planningStrategies.scala +++ b/src/main/scala/shark2/planningStrategies.scala @@ -49,8 +49,7 @@ abstract trait PlanningStrategies { object SparkAggregates extends Strategy { val allowedAggregates = Set[Class[_]]( classOf[Count], - classOf[Average], - classOf[Sum]) + classOf[Average]) /** Returns true if [[exprs]] contains only aggregates that can be computed using Accumulators. */ def onlyAllowedAggregates(exprs: Seq[Expression]): Boolean = { From 934c18c5dbd8571e6e0157f4648a463f64a4d91b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 23 Dec 2013 16:01:08 -0800 Subject: [PATCH 284/778] Filter out non-deterministic lines when comparing test answers. --- src/test/scala/shark2/HiveComparisionTest.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/test/scala/shark2/HiveComparisionTest.scala b/src/test/scala/shark2/HiveComparisionTest.scala index edc12b9d1c970..5ae42a9aa1c62 100644 --- a/src/test/scala/shark2/HiveComparisionTest.scala +++ b/src/test/scala/shark2/HiveComparisionTest.scala @@ -41,7 +41,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with protected def prepareAnswer(sharkQuery: TestShark.type#SharkSqlQuery, answer: Seq[String]): Seq[String] = { val orderedAnswer = sharkQuery.parsed match { - case _: Command => answer // Don't attempt to modify the result of Commands since they are run by hive. + case _: Command => answer.filterNot(nonDeterministicLine) // Clean out nondeterministic time schema info. case _ => val isOrdered = sharkQuery.executedPlan.collect { case s: Sort => s}.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. @@ -50,6 +50,9 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with orderedAnswer.map(cleanPaths) } + protected def nonDeterministicLine(line: String) = + Seq("CreateTime","transient_lastDdlTime", "grantTime").map(line contains _).reduceLeft(_||_) + /** * Removes non-deterministic paths from [[str]] so cached answers will still pass. */ From 9307ef958910659ae35f1cda4ffec27b38874add Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 23 Dec 2013 16:01:19 -0800 Subject: [PATCH 285/778] update list of passing tests. --- src/test/scala/shark2/HiveCompatability.scala | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index d32a2764aed1c..01d28fe40ae7b 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -47,14 +47,10 @@ class HiveCompatability extends HiveQueryFileTest { "auto_join26", "auto_join28", "avro_change_schema", - "avro_schema_error_message", - "avro_schema_literal", "bucketmapjoin6", - "combine1", "count", "ct_case_insensitive", "database_properties", - "default_partition_name", "delimiter", "describe_database_json", "diff_part_input_formats", @@ -68,8 +64,6 @@ class HiveCompatability extends HiveQueryFileTest { "groupby4_map", "groupby4_map_skew", "index_auth", - //"index_auto_mult_tables", - //"index_auto_mult_tables_compact", "index_auto_self_join", "index_auto_update", "index_stale", @@ -84,6 +78,7 @@ class HiveCompatability extends HiveQueryFileTest { "input4_cb_delim", "input4_limit", "input7", + "inputddl8", "insert1", "join0", "join1", @@ -114,12 +109,12 @@ class HiveCompatability extends HiveQueryFileTest { "join_reorder2", "join_reorder3", "join_view", + "keyword_1", "lineage1", "literal_double", "literal_ints", "literal_string", "mergejoins", - // "nestedvirtual", "no_hooks", "noalias_subq1", "nomore_ambiguous_table_col", @@ -130,6 +125,8 @@ class HiveCompatability extends HiveQueryFileTest { "nullgroup3", "nullinput", "nullinput2", + "nullscript", + "part_inherit_tbl_props_empty", "ppd1", "ppd_gby_join", "ppd_join", @@ -144,8 +141,8 @@ class HiveCompatability extends HiveQueryFileTest { "ppd_udf_col", "ppd_union", "progress_1", + "query_with_semi", "quote2", - "rename_column", "select_as_omitted", "set_variable_sub", "show_describe_func_quotes", @@ -253,7 +250,8 @@ class HiveCompatability extends HiveQueryFileTest { "union5", "union7", "union8", - "union9" + "union9", + "union_script" ) // TODO: bundle in jar files... get from classpath From c8b1553716ef19654372d7099d48a5b5c44c2282 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 23 Dec 2013 16:03:26 -0800 Subject: [PATCH 286/778] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 4445683d83a75..6082e6c92bd24 100644 --- a/README.md +++ b/README.md @@ -6,7 +6,7 @@ More documentation can be found in the project's [scaladoc](http://marmbrus.gith Getting Started =============== -SCADS is built using [SBT](https://github.com/harrah/xsbt). The SBT launcher is included in the distribution (bin/sbt) and is responsible for downloading all other required jars (scala compiler and other dependencies). +Catalyst is built using [SBT](https://github.com/harrah/xsbt). The SBT launcher is included in the distribution (bin/sbt) and is responsible for downloading all other required jars (scala compiler and other dependencies). SBT commands can be invoked from the command line. For example, to clean and build a jar, you would run the following command: From d1e7b8ee4f01f9def09617c2896351509dc3f9b6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 23 Dec 2013 16:03:53 -0800 Subject: [PATCH 287/778] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 6082e6c92bd24..004b229bb96fe 100644 --- a/README.md +++ b/README.md @@ -73,7 +73,7 @@ Project {key#0,value#1} HiveTableScan {key#0,value#1}, (MetastoreRelation default, src, None) ``` -From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](http://marmbrus.github.io/catalyst/api/current/index.html#catalyst.trees.TreeNode) objects. +From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](http://marmbrus.github.io/catalyst/latest/api/index.html#catalyst.trees.TreeNode) objects. ```scala scala> query.optimizedPlan res1: catalyst.plans.logical.LogicalPlan = From d873b2b7852f36e6d99f63e5d912d5770ec62fc8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 23 Dec 2013 22:32:19 -0500 Subject: [PATCH 288/778] Remove numbers associated with test cases. --- src/test/scala/OptimizerSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/test/scala/OptimizerSuite.scala b/src/test/scala/OptimizerSuite.scala index e57e10db1e11e..4da846caf1dc2 100644 --- a/src/test/scala/OptimizerSuite.scala +++ b/src/test/scala/OptimizerSuite.scala @@ -61,7 +61,7 @@ class OptimizerSuite extends FunSuite { /* * Unit tests for evaluating literals in expressions. * */ - test("Evaluating Literals Test 1: expressions only have literals") { + test("Evaluating Literals Test: expressions only have literals") { val originalQuery = testRelation .select(Literal(2) + Literal(3) + Literal(4) as Symbol("2+3+4"), @@ -86,7 +86,7 @@ class OptimizerSuite extends FunSuite { comparePlans(optimized, correctAnswer) } - test("Evaluating Literals Test 2: expressions have attribute references and literals in" + + test("Evaluating Literals Test: expressions have attribute references and literals in" + "arithmetic operations") { val originalQuery = testRelation @@ -108,7 +108,7 @@ class OptimizerSuite extends FunSuite { comparePlans(optimized, correctAnswer) } - test("Evaluating Literals Test 3: expressions have attribute references and literals in" + + test("Evaluating Literals Test: expressions have attribute references and literals in" + "predicates") { val originalQuery = testRelation From 1d0ae1ee59af897fcaa8383067b52fb4d362f349 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Dec 2013 12:24:28 -0800 Subject: [PATCH 289/778] Switch from IndexSeq[Any] to Row interface that will allow us unboxed access to primitive types. --- src/main/scala/expressions/Evaluate.scala | 2 +- src/main/scala/expressions/Row.scala | 79 +++++++++++++++++++++ src/main/scala/expressions/aggregates.scala | 2 +- src/main/scala/shark2/SharkPlan.scala | 4 +- src/main/scala/shark2/Transform.scala | 4 +- src/main/scala/shark2/aggregates.scala | 19 +++-- src/main/scala/shark2/basicOperators.scala | 4 +- src/main/scala/shark2/hiveOperators.scala | 5 +- src/main/scala/shark2/joins.scala | 18 ++--- src/main/scala/shark2/package.scala | 2 + 10 files changed, 111 insertions(+), 28 deletions(-) create mode 100644 src/main/scala/expressions/Row.scala diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index e109cf964677e..53db0ad3dc7fb 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -8,7 +8,7 @@ import types._ * Performs evaluation of an expression tree, given a set of input tuples. */ object Evaluate extends Logging { - def apply(e: Expression, input: Seq[Seq[Any]]): Any = attachTree(e, "Expression Evaluation Failed") { + def apply(e: Expression, input: Seq[Row]): Any = attachTree(e, "Expression Evaluation Failed") { def eval(e: Expression) = Evaluate(e, input) /** diff --git a/src/main/scala/expressions/Row.scala b/src/main/scala/expressions/Row.scala new file mode 100644 index 0000000000000..391a87b1f338a --- /dev/null +++ b/src/main/scala/expressions/Row.scala @@ -0,0 +1,79 @@ +package catalyst +package expressions + +/** + * Represents one row of output from a relational operator. Allows both generic access by ordinal, + * which will incur boxing overhead for primitives, as well as native primitive access. + * + * It is invalid to use the native primitive interface to retrieve a value that is null, instead a user + * must check [[isNullAt]] before attempting to retrieve a value that might be null. + */ +abstract class Row extends Seq[Any] with Serializable { + def apply(i: Int): Any + + def isNullAt(i: Int): Boolean + + def getInt(i: Int): Int + def getLong(i: Int): Long + def getDouble(i: Int): Double + def getBoolean(i: Int): Boolean + def getShort(i: Int): Short + def getByte(i: Int): Byte +} + +/** + * A row with no data. Calling any methods will result in an error. Can be used as a placeholder. + */ +object EmptyRow extends Row { + def apply(i: Int): Any = ??? + + def iterator = Iterator.empty + def length = 0 + def isNullAt(i: Int): Boolean = ??? + + def getInt(i: Int): Int = ??? + def getLong(i: Int): Long = ??? + def getDouble(i: Int): Double = ??? + def getBoolean(i: Int): Boolean = ??? + def getShort(i: Int): Short = ??? + def getByte(i: Int): Byte = ??? +} + +/** + * A row implementation that uses an array of objects as the underlying storage. + */ +class GenericRow(input: Seq[Any]) extends Row { + val values = input.toIndexedSeq + + def iterator = values.iterator + def length = values.length + + def apply(i: Int) = values(i) + + def isNullAt(i: Int) = values(i) == null + + def getInt(i: Int): Int = { + if (values(i) == null) sys.error("Failed to check null bit for primitive int value.") + values(i).asInstanceOf[Int] + } + def getLong(i: Int): Long = { + if (values(i) == null) sys.error("Failed to check null bit for primitive long value.") + values(i).asInstanceOf[Long] + } + def getDouble(i: Int): Double = { + if (values(i) == null) sys.error("Failed to check null bit for primitive double value.") + values(i).asInstanceOf[Double] + } + def getBoolean(i: Int): Boolean = { + if (values(i) == null) sys.error("Failed to check null bit for primitive boolean value.") + values(i).asInstanceOf[Boolean] + } + def getShort(i: Int): Short = { + if (values(i) == null) sys.error("Failed to check null bit for primitive short value.") + values(i).asInstanceOf[Short] + } + def getByte(i: Int): Byte = { + if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.") + values(i).asInstanceOf[Byte] + } +} \ No newline at end of file diff --git a/src/main/scala/expressions/aggregates.scala b/src/main/scala/expressions/aggregates.scala index 5e612e68d53cc..9cb1bf1ad994f 100644 --- a/src/main/scala/expressions/aggregates.scala +++ b/src/main/scala/expressions/aggregates.scala @@ -20,7 +20,7 @@ abstract class AggregateFunction extends AggregateExpression with Serializable w def nullable = base.nullable def dataType = base.dataType - def apply(input: Seq[Seq[Any]]): Unit + def apply(input: Seq[Row]): Unit def result: Any } diff --git a/src/main/scala/shark2/SharkPlan.scala b/src/main/scala/shark2/SharkPlan.scala index 352463740bc8c..51b706ead1df0 100644 --- a/src/main/scala/shark2/SharkPlan.scala +++ b/src/main/scala/shark2/SharkPlan.scala @@ -10,7 +10,9 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { /** * Runs this query returning the result as an RDD. */ - def execute(): RDD[IndexedSeq[Any]] + def execute(): RDD[Row] + + protected def buildRow(values: Seq[Any]): Row = new catalyst.expressions.GenericRow(values) } abstract trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { diff --git a/src/main/scala/shark2/Transform.scala b/src/main/scala/shark2/Transform.scala index f8f0142db7cc4..d271e8146abeb 100644 --- a/src/main/scala/shark2/Transform.scala +++ b/src/main/scala/shark2/Transform.scala @@ -31,12 +31,12 @@ case class Transform( val reader = new BufferedReader(new InputStreamReader(inputStream)) // TODO: This should be exposed as an iterator instead of reading in all the data at once for a partition. - val outputLines = collection.mutable.ArrayBuffer[IndexedSeq[Any]]() + val outputLines = collection.mutable.ArrayBuffer[Row]() val readerThread = new Thread("Transform OutoutReader") { override def run() { var curLine = reader.readLine() while(curLine != null) { - outputLines += curLine.split("\t") + outputLines += buildRow(curLine.split("\t")) curLine = reader.readLine() } } diff --git a/src/main/scala/shark2/aggregates.scala b/src/main/scala/shark2/aggregates.scala index 44419748901e5..0baffceb5b140 100644 --- a/src/main/scala/shark2/aggregates.scala +++ b/src/main/scala/shark2/aggregates.scala @@ -21,7 +21,7 @@ case class Aggregate(groupingExpressions: Seq[Expression], def result: Any = sum.toDouble / count.toDouble - def apply(input: Seq[Seq[Any]]): Unit = { + def apply(input: Seq[Row]): Unit = { count += 1 // TODO: Support all types here... sum += Evaluate(expr, input).asInstanceOf[Int] @@ -33,7 +33,7 @@ case class Aggregate(groupingExpressions: Seq[Expression], var count: Long = _ - def apply(input: Seq[Seq[Any]]): Unit = { + def apply(input: Seq[Row]): Unit = { val evaluatedExpr = expr.map(Evaluate(_, input)) if(evaluatedExpr.map(_ != null).reduceLeft(_ || _)) count += 1 @@ -47,7 +47,7 @@ case class Aggregate(groupingExpressions: Seq[Expression], var sum = Evaluate(Cast(Literal(0), expr.dataType), Nil) - def apply(input: Seq[Seq[Any]]): Unit = + def apply(input: Seq[Row]): Unit = sum = Evaluate(Add(Literal(sum), expr), input) def result: Any = sum @@ -58,7 +58,7 @@ case class Aggregate(groupingExpressions: Seq[Expression], val seen = new scala.collection.mutable.HashSet[Any]() - def apply(input: Seq[Seq[Any]]): Unit = { + def apply(input: Seq[Row]): Unit = { val evaluatedExpr = expr.map(Evaluate(_, input)) if(evaluatedExpr.map(_ != null).reduceLeft(_ && _)) seen += evaluatedExpr @@ -72,7 +72,7 @@ case class Aggregate(groupingExpressions: Seq[Expression], def execute() = attachTree(this, "execute") { - val grouped = child.execute().map(row => (groupingExpressions.map(Evaluate(_, Vector(row))), row)).groupByKey() + val grouped = child.execute().map(row => (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row)).groupByKey() grouped.map { case (group, rows) => // Replace all aggregate expressions with spark functions that will compute the result. @@ -111,8 +111,7 @@ case class Aggregate(groupingExpressions: Seq[Expression], val input = Vector(row) aggFunctions.foreach(_.apply(input)) } - // IS THIS RIGHT? - aggImplementations.map(Evaluate(_, Vector(group))).toIndexedSeq + buildRow(aggImplementations.map(Evaluate(_, Vector(group)))) } } } @@ -134,7 +133,7 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan val sum = sc.accumulable(0) def result: Any = sum.value.toDouble / count.value.toDouble - def apply(input: Seq[Seq[Any]]): Unit = { + def apply(input: Seq[Row]): Unit = { count += 1 // TODO: Support all types here... sum += Evaluate(expr, input).asInstanceOf[Int] @@ -146,7 +145,7 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan val count = sc.accumulable(0) - def apply(input: Seq[Seq[Any]]): Unit = + def apply(input: Seq[Row]): Unit = if(Evaluate(expr, input) != null) count += 1 @@ -169,6 +168,6 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan val input = Vector(row) aggFunctions.foreach(_.apply(input)) } - sc.makeRDD(Seq(aggImplementations.map(Evaluate(_, Nil)).toIndexedSeq), 1) + sc.makeRDD(Seq(buildRow(aggImplementations.map(Evaluate(_, Nil)))), 1) } } \ No newline at end of file diff --git a/src/main/scala/shark2/basicOperators.scala b/src/main/scala/shark2/basicOperators.scala index 5013ca6b87507..40d78c0a8cddf 100644 --- a/src/main/scala/shark2/basicOperators.scala +++ b/src/main/scala/shark2/basicOperators.scala @@ -23,7 +23,7 @@ case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends def output = projectList.map(_.toAttribute) def execute() = child.execute().map { row => - projectList.map(Evaluate(_, Vector(row))).toIndexedSeq + buildRow(projectList.map(Evaluate(_, Vector(row)))) } } @@ -109,6 +109,6 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { case class LocalRelation(output: Seq[Attribute], data: Seq[IndexedSeq[Any]]) (@transient sc: SharkContext) extends LeafNode { - def execute() = sc.makeRDD(data, 1) + def execute() = sc.makeRDD(data.map(buildRow), 1) } diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index 32bc19567aace..29360a284a39b 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -55,11 +55,12 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation inputRdd.map { case array: Array[Any] => - array.flatMap { + val res = array.flatMap { case struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct => unpackStruct(struct) case array: Array[Any] => array } - case struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct => unpackStruct(struct) + buildRow(res) + case struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct => buildRow(unpackStruct(struct)) } } diff --git a/src/main/scala/shark2/joins.scala b/src/main/scala/shark2/joins.scala index f9329774d2d5a..9705fdc788b70 100644 --- a/src/main/scala/shark2/joins.scala +++ b/src/main/scala/shark2/joins.scala @@ -30,7 +30,7 @@ case class SparkEquiInnerJoin( } val rightWithKeys = right.execute().map { row => - val joinKeys = rightKeys.map(Evaluate(_, Vector(Nil, row))) + val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) logger.debug(s"Generated right join keys ($rightKeys) => ($joinKeys) given row $row") (joinKeys, row) } @@ -38,14 +38,14 @@ case class SparkEquiInnerJoin( // Do the join. val joined = filterNulls(leftWithKeys).join(filterNulls(rightWithKeys)) // Drop join keys and merge input tuples. - joined.map { case (_, (leftTuple, rightTuple)) => leftTuple ++ rightTuple } + joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple ++ rightTuple) } } /** * Filters any rows where the any of the join keys is null, ensuring three-valued * logic for the equi-join conditions. */ - protected def filterNulls(rdd: RDD[(Seq[Any], IndexedSeq[Any])]) = + protected def filterNulls(rdd: RDD[(Seq[Any], Row)]) = rdd.filter { case (key: Seq[_], _) => !key.map(_ == null).reduceLeft(_ || _) } @@ -55,7 +55,7 @@ case class CartesianProduct(left: SharkPlan, right: SharkPlan) extends BinaryNod def output = left.output ++ right.output def execute() = left.execute().cartesian(right.execute()).map { - case (l: IndexedSeq[Any], r: IndexedSeq[Any]) => l ++ r + case (l: Row, r: Row) => buildRow(l ++ r) } } @@ -74,7 +74,7 @@ case class BroadcastNestedLoopJoin(streamed: SharkPlan, broadcast: SharkPlan, jo val streamedPlusMatches = streamed.execute().map { streamedRow => var i = 0 - val matchedRows = new mutable.ArrayBuffer[IndexedSeq[Any]] + val matchedRows = new mutable.ArrayBuffer[Row] val includedBroadcastTuples = new scala.collection.mutable.BitSet(broadcastedRelation.value.size) while(i < broadcastedRelation.value.size) { @@ -85,7 +85,7 @@ case class BroadcastNestedLoopJoin(streamed: SharkPlan, broadcast: SharkPlan, jo case Some(c) => Evaluate(c, Vector(streamedRow, broadcastedRow)).asInstanceOf[Boolean] } if(includeRow) { - matchedRows += (streamedRow ++ broadcastedRow) + matchedRows += buildRow(streamedRow ++ broadcastedRow) includedBroadcastTuples += i } i += 1 @@ -93,19 +93,19 @@ case class BroadcastNestedLoopJoin(streamed: SharkPlan, broadcast: SharkPlan, jo val outputRows = if(matchedRows.size > 0) matchedRows else if(joinType == LeftOuter || joinType == FullOuter) - Vector(streamedRow ++ Array.fill(right.output.size)(null)) + Vector(buildRow(streamedRow ++ Array.fill(right.output.size)(null))) else Vector() (outputRows, includedBroadcastTuples) } val allIncludedBroadcastTupes = streamedPlusMatches.map(_._2).reduce(_ ++ _) - val rightOuterMatches: Seq[IndexedSeq[Any]] = + val rightOuterMatches: Seq[Row] = if(joinType == RightOuter || joinType == FullOuter) broadcastedRelation.value.zipWithIndex.filter { case (row, i) => !allIncludedBroadcastTupes.contains(i) }.map { - case (row, _) => Vector.fill(left.output.size)(null) ++ row + case (row, _) => buildRow(Vector.fill(left.output.size)(null) ++ row) } else Vector() diff --git a/src/main/scala/shark2/package.scala b/src/main/scala/shark2/package.scala index 3a3b2c1ab265f..e0857cd74d4c5 100644 --- a/src/main/scala/shark2/package.scala +++ b/src/main/scala/shark2/package.scala @@ -14,6 +14,8 @@ import types._ * implementation for execution. */ package object shark2 { + type Row = catalyst.expressions.Row + implicit class typeInfoConversions(dt: DataType) { import org.apache.hadoop.hive.serde2.typeinfo._ import TypeInfoFactory._ From ac6376d977fbe5b073dda6623041033676ccc8a6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Dec 2013 12:51:56 -0800 Subject: [PATCH 290/778] Split out general shark query execution driver from test harness. --- src/main/scala/shark2/SharkInstance.scala | 158 ++++++++++++++++++ src/main/scala/shark2/TestShark.scala | 146 ++++------------ .../scala/shark2/HiveComparisionTest.scala | 16 +- src/test/scala/shark2/HiveCompatability.scala | 2 +- 4 files changed, 198 insertions(+), 124 deletions(-) create mode 100644 src/main/scala/shark2/SharkInstance.scala diff --git a/src/main/scala/shark2/SharkInstance.scala b/src/main/scala/shark2/SharkInstance.scala new file mode 100644 index 0000000000000..825eafd3fa6a7 --- /dev/null +++ b/src/main/scala/shark2/SharkInstance.scala @@ -0,0 +1,158 @@ +package catalyst +package shark2 + +import java.io.File + +import org.apache.spark.rdd.RDD +import shark.{SharkConfVars, SharkContext, SharkEnv} + +import analysis.{SimpleAnalyzer, Analyzer} +import frontend.hive._ +import optimizer.Optimize +import planning.QueryPlanner +import plans.logical.LogicalPlan +import rules.RuleExecutor + +/** + * Starts up an instance of shark where metadata is stored locally. An in-process medata data is created with data + * stored in ./metadata. Warehouse data is stored in in ./warehouse. + */ +class LocalSharkInstance(val master: String) extends SharkInstance { + def warehousePath = new File("warehouse").getCanonicalPath + def metastorePath = new File("metastore").getCanonicalPath +} + +/** + * An instance of the shark execution engine. This class is responsible for taking queries expressed either in SQl or + * as raw catalyst logical plans and optimizing them for execution using Spark. Additionally this class maintains + * the connection with the hive metadata store. + */ +abstract class SharkInstance extends Logging { + self => + + /** The URL of the shark master. */ + def master: String + /** The path to the hive warehouse. */ + def warehousePath: String + /** The path to the local metastore. */ + def metastorePath: String + + /** The SharkContext */ + lazy val sc = createContext() + + protected def createContext() = { + SharkEnv.initWithSharkContext("catalyst.shark2", master) + } + + /** Sets up the system initially or after a RESET command */ + protected def configure() { + // Use hive natively for queries that won't be executed by catalyst. This is because + // shark has dependencies on a custom version of hive that we are trying to avoid + // in catalyst. + SharkConfVars.setVar(SharkContext.hiveconf, SharkConfVars.EXEC_MODE, "hive") + + runSqlHive("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + metastorePath + ";create=true") + runSqlHive("set hive.metastore.warehouse.dir=" + warehousePath) + } + + configure() // Must be called before initializing the catalog below. + + /* A catalyst metadata catalog that points to the Shark/Hive Metastore. */ + val catalog = new HiveMetastoreCatalog(SharkContext.hiveconf) + /* An analyzer that uses the Shark/Hive metastore. */ + val analyze = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) + + /** + * Runs the specified SQL query using Hive. + */ + def runSqlHive(sql: String): Seq[String] = { + val maxResults = 100000 + val results = sc.sql(sql, 100000) + // It is very confusing when you only get back some of the results... + if(results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") + results + } + + object TrivalPlanner extends QueryPlanner[SharkPlan] with PlanningStrategies { + val sc = self.sc + val strategies = + SparkEquiInnerJoin :: + SparkAggregates :: + HiveTableScans :: + DataSinks :: + BasicOperators :: + CartesianProduct :: + BroadcastNestedLoopJoin :: Nil + } + + object PrepareForExecution extends RuleExecutor[SharkPlan] { + val batches = + Batch("Prepare Expressions", Once, + expressions.BindReferences) :: Nil + } + + class SharkSqlQuery(sql: String) extends SharkQuery { + lazy val parsed = HiveQl.parseSql(sql) + def hiveExec() = runSqlHive(sql) + override def toString = sql + "\n" + super.toString + } + + /** + * The primary workflow for executing queries using Shark. Designed to allow easy access to the intermediate phases + * of query execution. + */ + abstract class SharkQuery { + def parsed: LogicalPlan + + lazy val analyzed = analyze(parsed) + lazy val optimizedPlan = Optimize(catalog.CreateTables(analyzed)) + // TODO: Don't just pick the first one... + lazy val sharkPlan = TrivalPlanner(optimizedPlan).next() + lazy val executedPlan = PrepareForExecution(sharkPlan) + + lazy val toRdd = executedPlan.execute() + + /** + * Returns the result as a hive compatible sequence of strings. For native commands, the execution is simply + * passed back to Hive. + */ + def stringResult(): Seq[String] = analyzed match { + case NativeCommand(cmd) => runSqlHive(cmd) + case ConfigurationAssignment(cmd) => runSqlHive(cmd) + case ExplainCommand(plan) => new SharkQuery { val parsed = plan }.toString.split("\n") + case query => + val result: Seq[Seq[Any]] = toRdd.collect.toSeq + // Reformat to match hive tab delimited output. + val asString = result.map(_.map { + case null => "NULL" + case other => other + }).map(_.mkString("\t")).toSeq + + asString + } + + protected def stringOrError[A](f: => A): String = + try f.toString catch { case e: Throwable => e.toString } + + override def toString: String = + s"""== Logical Plan == + |${stringOrError(analyzed)} + |== Physical Plan == + |${stringOrError(sharkPlan)} + """.stripMargin.trim + } + + /** + * A shark query workflow for plans where all relations have already been resolved (likely because the query was + * built from raw RDDs). Additionally attribute resolution is case sensitive. + */ + abstract class LogicalSharkQuery extends SharkQuery { + override lazy val analyzed = SimpleAnalyzer(parsed) + } + + implicit class stringToQuery(str: String) { + def q = new SharkSqlQuery(str) + } + + implicit def logicalToSharkQuery(plan: LogicalPlan) = new LogicalSharkQuery { val parsed = plan } +} \ No newline at end of file diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index 07ff0be93b7f1..7d47bcca4f02b 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -1,22 +1,19 @@ package catalyst package shark2 -import catalyst.expressions.AttributeReference -import catalyst.optimizer.Optimize import java.io.File +import scala.collection.mutable + +import org.apache.hadoop.hive.metastore.MetaStoreUtils +import shark.SharkEnv import analysis._ -import catalyst.plans.logical.LogicalPlan +import plans.logical.LogicalPlan import frontend.hive._ -import planning._ -import rules._ -import shark.{SharkConfVars, SharkContext, SharkEnv} import util._ -import org.apache.spark.rdd.RDD import collection.JavaConversions._ -import org.apache.hadoop.hive.metastore.MetaStoreUtils -import scala.collection.mutable + /** * A locally running test instance of spark. The lifecycle for a given query is managed by the inner class @@ -44,59 +41,29 @@ import scala.collection.mutable * seems to lead to weird non-deterministic failures. Therefore, the execution of testcases that rely on TestShark * must be serialized. */ -object TestShark extends Logging { +object TestShark extends SharkInstance { self => - val WAREHOUSE_PATH = getTempFilePath("sharkWarehouse") - val METASTORE_PATH = getTempFilePath("sharkMetastore") - val MASTER = "local" + lazy val master = "local" + lazy val warehousePath = getTempFilePath("sharkWarehouse").getCanonicalPath + lazy val metastorePath = getTempFilePath("sharkMetastore").getCanonicalPath - /** A local shark context */ - protected val sc = { + override protected def createContext = { // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") - SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) + SharkEnv.initWithSharkContext("catalyst.shark2.TestShark", master) } - configure() - /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") /** The location of the hive source code. */ lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME") - /* A catalyst metadata catalog that points to the Shark/Hive Metastore. */ - val catalog = new HiveMetastoreCatalog(SharkContext.hiveconf) - /* An analyzer that uses the Shark/Hive metastore. */ - val analyze = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) - - /** Sets up the system initially or after a RESET command */ - protected def configure() { - // Use hive natively for queries that won't be executed by catalyst. This is because - // shark has dependencies on a custom version of hive that we are trying to avoid - // in catalyst. - SharkConfVars.setVar(SharkContext.hiveconf, SharkConfVars.EXEC_MODE, "hive") - - runSqlHive("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + METASTORE_PATH + ";create=true") - runSqlHive("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - - // HACK: Hive is too noisy by default. - org.apache.log4j.LogManager.getCurrentLoggers.foreach(_.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN)) - } - - /** - * Runs the specified SQL query using Hive. - */ - def runSqlHive(sql: String): Seq[String] = { - val maxResults = 100000 - val results = sc.sql(rewritePaths(sql), 100000) - // It is very confusing when you only get back some of the results... - if(results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") - results - } + // Override so we can intercept relative paths and rewrite them to point at hive. + override def runSqlHive(sql: String): Seq[String] = super.runSqlHive(rewritePaths(sql)) /** Returns the value of specified environmental variable as a [[java.io.File]] after checking to ensure it exists */ private def envVarToFile(envVar: String): File = { @@ -116,34 +83,11 @@ object TestShark extends Logging { else cmd - object TrivalPlanner extends QueryPlanner[SharkPlan] with PlanningStrategies { - val sc = self.sc - val strategies = - SparkEquiInnerJoin :: - SparkAggregates :: - HiveTableScans :: - DataSinks :: - BasicOperators :: - CartesianProduct :: - BroadcastNestedLoopJoin :: Nil - } - - object PrepareForExecution extends RuleExecutor[SharkPlan] { - val batches = - Batch("Prepare Expressions", Once, - expressions.BindReferences) :: Nil - } - - class SharkSqlQuery(sql: String) extends SharkQuery { - lazy val parsed = HiveQl.parseSql(sql) - def hiveExec() = runSqlHive(sql) - override def toString = sql + "\n" + super.toString - } - - abstract class SharkQuery { - def parsed: LogicalPlan - - lazy val analyzed = { + /** + * Override SharkQuery with special debug workflow. + */ + abstract class SharkQuery extends super.SharkQuery { + override lazy val analyzed = { // Make sure any test tables referenced are loaded. val referencedTables = parsed collect { case UnresolvedRelation(name, _) => name.split("\\.").last } val referencedTestTables = referencedTables.filter(testTables.contains) @@ -152,54 +96,25 @@ object TestShark extends Logging { // Proceed with analysis. analyze(parsed) } - lazy val optimizedPlan = Optimize(catalog.CreateTables(analyzed)) - // TODO: Don't just pick the first one... - lazy val sharkPlan = TrivalPlanner(optimizedPlan).next() - lazy val executedPlan = PrepareForExecution(sharkPlan) - - lazy val toRdd = executedPlan.execute() + /** Runs the query after interposing operators that print the result of each intermediate step. */ def debugExec() = DebugQuery(executedPlan).execute().collect - - /** - * Returns the result as a hive compatible sequence of strings. For native commands, the execution is simply - * passed back to Hive. - */ - def stringResult(): Seq[String] = analyzed match { - case NativeCommand(cmd) => runSqlHive(rewritePaths(cmd)) - case ConfigurationAssignment(cmd) => runSqlHive(cmd) - case ExplainCommand(plan) => (new SharkQuery { val parsed = plan }).toString.split("\n") - case query => - val result: Seq[Seq[Any]] = toRdd.collect.toSeq - // Reformat to match hive tab delimited output. - val asString = result.map(_.map { - case null => "NULL" - case other => other - }).map(_.mkString("\t")).toSeq - - asString - } - - protected def stringOrError[A](f: => A): String = - try f.toString catch { case e: Throwable => e.toString } - - override def toString: String = - s"""== Logical Plan == - |${stringOrError(analyzed)} - |== Physical Plan == - |${stringOrError(sharkPlan)} - """.stripMargin.trim } - abstract class CaseSensitiveSharkQuery extends SharkQuery { - override lazy val analyzed = SimpleAnalyzer(parsed) + class SharkSqlQuery(sql: String) extends SharkQuery { + lazy val parsed = HiveQl.parseSql(sql) + def hiveExec() = runSqlHive(sql) + override def toString = sql + "\n" + super.toString } - implicit class stringToQuery(str: String) { + + /* We must repeat the implicits so that we bind to the overriden versions */ + + implicit class stringToTestQuery(str: String) { def q = new SharkSqlQuery(str) } - implicit def logicalToSharkQuery(plan: LogicalPlan) = new CaseSensitiveSharkQuery { val parsed = plan } + implicit override def logicalToSharkQuery(plan: LogicalPlan) = new LogicalSharkQuery { val parsed = plan } case class TestTable(name: String, commands: (()=>Unit)*) @@ -256,6 +171,9 @@ object TestShark extends Logging { */ def reset() { try { + // HACK: Hive is too noisy by default. + org.apache.log4j.LogManager.getCurrentLoggers.foreach(_.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN)) + // It is important that we RESET first as broken hooks that might have been set could break other sql exec here. runSqlHive("RESET") // For some reason, RESET does not reset the following variables... diff --git a/src/test/scala/shark2/HiveComparisionTest.scala b/src/test/scala/shark2/HiveComparisionTest.scala index 5ae42a9aa1c62..fd223e0230bc4 100644 --- a/src/test/scala/shark2/HiveComparisionTest.scala +++ b/src/test/scala/shark2/HiveComparisionTest.scala @@ -18,8 +18,6 @@ import collection.JavaConversions._ * The "golden" results from Hive are cached in [[answerCache]] to speed up testing. */ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with GivenWhenThen with Logging { - val testShark = TestShark - protected val targetDir = new File("target") protected val answerCache = new File(targetDir, "comparison-test-cache") if(!answerCache.exists) @@ -71,7 +69,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with val queryList = sql.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq try { - testShark.reset() + TestShark.reset() val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => @@ -104,17 +102,17 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") info(s"HIVE: $queryString") // Analyze the query with catalyst to ensure test tables are loaded. - val sharkQuery = (new testShark.SharkSqlQuery(queryString)) + val sharkQuery = (new TestShark.SharkSqlQuery(queryString)) val answer = sharkQuery.analyzed match { case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. - case _ => testShark.runSqlHive(queryString) + case _ => TestShark.runSqlHive(queryString) } stringToFile(cachedAnswerFile, answer.mkString("\n")) answer }.toSeq - testShark.reset() + TestShark.reset() computedResults } @@ -122,7 +120,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => info(queryString) - val query = new testShark.SharkSqlQuery(queryString) + val query = new TestShark.SharkSqlQuery(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { case e: Exception => val out = new java.io.ByteArrayOutputStream @@ -170,8 +168,8 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with // When we encounter an error we check to see if the environment is still okay by running a simple query. // If this fails then we halt testing since something must have gone seriously wrong. try { - new testShark.SharkSqlQuery("SELECT key FROM src").stringResult() - testShark.runSqlHive("SELECT key FROM src") + new TestShark.SharkSqlQuery("SELECT key FROM src").stringResult() + TestShark.runSqlHive("SELECT key FROM src") } catch { case e: Exception => logger.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index 01d28fe40ae7b..8fd7320240aa2 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -255,6 +255,6 @@ class HiveCompatability extends HiveQueryFileTest { ) // TODO: bundle in jar files... get from classpath - lazy val hiveQueryDir = new File(testShark.hiveDevHome, "ql/src/test/queries/clientpositive") + lazy val hiveQueryDir = new File(TestShark.hiveDevHome, "ql/src/test/queries/clientpositive") def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f).toMap } From d5c05c6da11cbdcdbe8287348d3304a3c88796a4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Dec 2013 12:54:07 -0800 Subject: [PATCH 291/778] For now, ignore the big data benchmark tests when the data isn't there. --- src/test/scala/shark2/BigDataBenchmark.scala | 82 ++++++++++---------- 1 file changed, 43 insertions(+), 39 deletions(-) diff --git a/src/test/scala/shark2/BigDataBenchmark.scala b/src/test/scala/shark2/BigDataBenchmark.scala index f12b0d9f3e946..20322169995c5 100644 --- a/src/test/scala/shark2/BigDataBenchmark.scala +++ b/src/test/scala/shark2/BigDataBenchmark.scala @@ -11,7 +11,6 @@ class BigDataBenchmarkTests extends HiveComaparisionTest { import TestShark._ val testDataDirectory = new File("target/big-data-benchmark-testdata") - require(testDataDirectory.exists()) val testTables = Seq( TestTable( @@ -60,45 +59,50 @@ class BigDataBenchmarkTests extends HiveComaparisionTest { testTables.foreach(registerTestTable) - createQueryTest("query1", - "SELECT pageURL, pageRank FROM rankings WHERE pageRank > 1") + if(!testDataDirectory.exists()) { + // TODO: Auto download the files on demand. + ignore("No data files found for BigDataBenchmark tests.") {} + } else { + createQueryTest("query1", + "SELECT pageURL, pageRank FROM rankings WHERE pageRank > 1") - createQueryTest("query2", - "SELECT SUBSTR(sourceIP, 1, 10), SUM(adRevenue) FROM uservisits GROUP BY SUBSTR(sourceIP, 1, 10)") + createQueryTest("query2", + "SELECT SUBSTR(sourceIP, 1, 10), SUM(adRevenue) FROM uservisits GROUP BY SUBSTR(sourceIP, 1, 10)") - createQueryTest("query3", - """ - |SELECT sourceIP, - | sum(adRevenue) as totalRevenue, - | avg(pageRank) as pageRank - |FROM - | rankings R JOIN - | (SELECT sourceIP, destURL, adRevenue - | FROM uservisits UV - | WHERE UV.visitDate > "1980-01-01" - | AND UV.visitDate < "1980-04-01") - | NUV ON (R.pageURL = NUV.destURL) - |GROUP BY sourceIP - |ORDER BY totalRevenue DESC - |LIMIT 1 - """.stripMargin) + createQueryTest("query3", + """ + |SELECT sourceIP, + | sum(adRevenue) as totalRevenue, + | avg(pageRank) as pageRank + |FROM + | rankings R JOIN + | (SELECT sourceIP, destURL, adRevenue + | FROM uservisits UV + | WHERE UV.visitDate > "1980-01-01" + | AND UV.visitDate < "1980-04-01") + | NUV ON (R.pageURL = NUV.destURL) + |GROUP BY sourceIP + |ORDER BY totalRevenue DESC + |LIMIT 1 + """.stripMargin) - createQueryTest("query4", - """ - |DROP TABLE IF EXISTS url_counts_partial; - |CREATE TABLE url_counts_partial AS - | SELECT TRANSFORM (line) - | USING 'python target/url_count.py' as (sourcePage, - | destPage, count) from documents; - |DROP TABLE IF EXISTS url_counts_total; - |CREATE TABLE url_counts_total AS - | SELECT SUM(count) AS totalCount, destpage - | FROM url_counts_partial GROUP BY destpage - |-- The following queries run, but generate different results in HIVE likely because the UDF is not deterministic - |-- given different input splits. - |-- SELECT CAST(SUM(count) AS INT) FROM url_counts_partial - |-- SELECT COUNT(*) FROM url_counts_partial - |-- SELECT * FROM url_counts_partial - |-- SELECT * FROM url_counts_total - """.stripMargin) + createQueryTest("query4", + """ + |DROP TABLE IF EXISTS url_counts_partial; + |CREATE TABLE url_counts_partial AS + | SELECT TRANSFORM (line) + | USING 'python target/url_count.py' as (sourcePage, + | destPage, count) from documents; + |DROP TABLE IF EXISTS url_counts_total; + |CREATE TABLE url_counts_total AS + | SELECT SUM(count) AS totalCount, destpage + | FROM url_counts_partial GROUP BY destpage + |-- The following queries run, but generate different results in HIVE likely because the UDF is not deterministic + |-- given different input splits. + |-- SELECT CAST(SUM(count) AS INT) FROM url_counts_partial + |-- SELECT COUNT(*) FROM url_counts_partial + |-- SELECT * FROM url_counts_partial + |-- SELECT * FROM url_counts_total + """.stripMargin) + } } \ No newline at end of file From 58b111c9c16ae74d814c66ad905ef0ecdee4f2a4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Dec 2013 15:07:38 -0800 Subject: [PATCH 292/778] fix bad scaladoc tag. --- src/main/scala/frontend/Hive.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 9874c86433507..13ab2ece02ffd 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -255,7 +255,7 @@ object HiveQl { /** Extractor for matching Hive's AST Tokens. */ object Token { - /** @returns matches of the form (tokenName, children). */ + /** @return matches of the form (tokenName, children). */ def unapply(t: Any) = t match { case t: ASTNode => Some((t.getText, Option(t.getChildren).map(_.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]])) From 9087152652b0acbb34bb2cb813a83669179bc348 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Dec 2013 15:07:51 -0800 Subject: [PATCH 293/778] fix toString of Not. --- src/main/scala/expressions/predicates.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/expressions/predicates.scala b/src/main/scala/expressions/predicates.scala index e48a18fbb0078..347d995397053 100644 --- a/src/main/scala/expressions/predicates.scala +++ b/src/main/scala/expressions/predicates.scala @@ -18,6 +18,7 @@ abstract class BinaryPredicate extends BinaryExpression with Predicate { case class Not(child: Expression) extends Predicate with trees.UnaryNode[Expression]{ def references = child.references def nullable = child.nullable + override def toString = s"NOT $child" } case class And(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "&&" From 8000504d2b7379e87a8183566a836ffb8a89210d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 25 Dec 2013 14:43:48 -0800 Subject: [PATCH 294/778] Improve type coercion. --- src/main/scala/analysis/Analyzer.scala | 3 +- src/main/scala/analysis/typeCoercion.scala | 69 ++++++++++++++++--- src/main/scala/expressions/Evaluate.scala | 2 +- .../scala/shark2/HiveTypeCoersionSuite.scala | 17 +++++ 4 files changed, 78 insertions(+), 13 deletions(-) create mode 100644 src/test/scala/shark2/HiveTypeCoersionSuite.scala diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala index 5d40e5d0e8e56..187c1d96f70a9 100644 --- a/src/main/scala/analysis/Analyzer.scala +++ b/src/main/scala/analysis/Analyzer.scala @@ -29,7 +29,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool Batch("Aggregation", Once, GlobalAggregates), Batch("Type Coersion", fixedPoint, - PromoteTypes, + PromoteNumericTypes, + PromoteStrings, ConvertNaNs) ) diff --git a/src/main/scala/analysis/typeCoercion.scala b/src/main/scala/analysis/typeCoercion.scala index 518908f27b345..307c50eccb75d 100644 --- a/src/main/scala/analysis/typeCoercion.scala +++ b/src/main/scala/analysis/typeCoercion.scala @@ -1,14 +1,15 @@ package catalyst package analysis +import errors._ import expressions._ import plans.logical._ import rules._ import types._ /** - * Converts string "NaN"s that are in binary operators with a NaN-able types (Float / Double) to the appropriate numeric - * equivalent. + * Converts string "NaN"s that are in binary operators with a NaN-able types (Float / Double) to the + * appropriate numeric equivalent. */ object ConvertNaNs extends Rule[LogicalPlan]{ val stringNaN = Literal("NaN", StringType) @@ -37,21 +38,67 @@ object ConvertNaNs extends Rule[LogicalPlan]{ } } -object PromoteTypes extends Rule[LogicalPlan] { - // TODO: Do this generically given some list of type precedence. +/** + * Widens numeric types and converts strings to numbers when appropriate. + * + * Loosely based on rules from "Hadoop: The Definitive Guide" 2nd edition, by Tom White + * + * The implicit conversion rules can be summarized as follows. Any integral numeric type can be + * implicitly converted to a wider type. All the integral numeric types, FLOAT, and (perhaps + * surprisingly) STRING can be implicitly converted to DOUBLE. TINYINT, SMALLINT, and INT can all be + * converted to FLOAT. BOOLEAN types cannot be converted to any other type. + * + * String conversions are handled by the PromoteStrings rule. + */ +object PromoteNumericTypes extends Rule[LogicalPlan] { + val integralPrecedence = Seq(ByteType, ShortType, IntegerType, LongType) + val toDouble = integralPrecedence ++ Seq(FloatType, DoubleType) + val toFloat = Seq(ByteType, ShortType, IntegerType) :+ FloatType + val allPromotions = integralPrecedence :: toDouble :: toFloat :: Nil + def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e - // Int String or String Int => Int Int - case b: BinaryExpression if b.left.dataType == StringType && b.right.dataType == IntegerType => - b.makeCopy(Array(Cast(b.left, IntegerType), b.right)) - case b: BinaryExpression if b.left.dataType == IntegerType && b.right.dataType == StringType => - b.makeCopy(Array(b.left, Cast(b.right, IntegerType))) + case b: BinaryExpression if b.left.dataType != b.right.dataType => + // Try and find a promotion rule that contains both types in question. + val applicableConversion = + allPromotions.find(p => p.contains(b.left.dataType) && p.contains(b.right.dataType)) - case Sum(e) if e.dataType == StringType => - Sum(Cast(e, IntegerType)) + applicableConversion match { + case Some(promotionRule) => + val widestType = + promotionRule.filter(t => t == b.left.dataType || t == b.right.dataType).last + val newLeft = + if (b.left.dataType == widestType) b.left else Cast(b.left, widestType) + val newRight = + if (b.right.dataType == widestType) b.right else Cast(b.right, widestType) + b.makeCopy(Array(newLeft, newRight)) + + // If there is no applicable conversion, leave expression unchanged. + case None => b + } } } +} + +/** + * Promotes strings that appear in arithmetic expressions. + */ +object PromoteStrings extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + case a: BinaryArithmetic if a.left.dataType == StringType => + a.makeCopy(Array(Cast(a.left, DoubleType), a.right)) + case a: BinaryArithmetic if a.right.dataType == StringType => + a.makeCopy(Array(a.left, Cast(a.right, DoubleType))) + + case Sum(e) if e.dataType == StringType => + Sum(Cast(e, DoubleType)) + case Average(e) if e.dataType == StringType => + Sum(Cast(e, DoubleType)) + } } \ No newline at end of file diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index 53db0ad3dc7fb..465134699b66d 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -127,7 +127,7 @@ object Evaluate extends Logging { case Cast(e, StringType) => eval(e).toString // String => Numeric Types case Cast(e, IntegerType) if e.dataType == StringType => eval(e).asInstanceOf[String].toInt - case Cast(e, IntegerType) if e.dataType == StringType => eval(e).asInstanceOf[String].toDouble + case Cast(e, DoubleType) if e.dataType == StringType => eval(e).asInstanceOf[String].toDouble // Numeric Type => Numeric Type case Cast(e, IntegerType) => n1(e, _.toInt(_)) case Cast(e, DoubleType) => n1(e, _.toDouble(_)) diff --git a/src/test/scala/shark2/HiveTypeCoersionSuite.scala b/src/test/scala/shark2/HiveTypeCoersionSuite.scala new file mode 100644 index 0000000000000..0fb3d13bd20b7 --- /dev/null +++ b/src/test/scala/shark2/HiveTypeCoersionSuite.scala @@ -0,0 +1,17 @@ +package catalyst +package shark2 + +/** + * A set of tests that validate type promotion rules. + */ +class HiveTypeCoersionSuite extends HiveComaparisionTest { + import TestShark._ + + val baseTypes = Seq("1", "1.0", "1L", "1S", "1Y", "'1'") + + baseTypes.foreach { i => + baseTypes.foreach { j => + createQueryTest(s"$i + $j", s"SELECT $i + $j FROM src LIMIT 1") + } + } +} \ No newline at end of file From 6e6417a9bd460637afa3bb38e2bff6aa8d9f8cb5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 25 Dec 2013 14:57:04 -0800 Subject: [PATCH 295/778] correct handling of nulls in boolean logic and sorting. --- src/main/scala/expressions/Evaluate.scala | 36 +++++++- src/main/scala/shark2/basicOperators.scala | 8 +- .../scala/ExpressionEvaluationSuite.scala | 87 ++++++++++++++++--- 3 files changed, 114 insertions(+), 17 deletions(-) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index 465134699b66d..0a943508afd91 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -104,7 +104,14 @@ object Evaluate extends Logging { case UnaryMinus(child) => n1(child, _.negate(_)) /* Comparisons */ - case Equals(l, r) => eval(l) == eval(r) + case Equals(l, r) => + val left = eval(l) + val right = eval(r) + if(left == null || right == null) + null + else + left == right + // Strings case GreaterThan(l, r) if l.dataType == StringType && r.dataType == StringType => eval(l).asInstanceOf[String] > eval(r).asInstanceOf[String] @@ -137,9 +144,30 @@ object Evaluate extends Logging { case Cast(e, ByteType) => n1(e, _.toInt(_).toByte) /* Boolean Logic */ - case Not(c) => !eval(c).asInstanceOf[Boolean] - case And(l,r) => eval(l).asInstanceOf[Boolean] && eval(r).asInstanceOf[Boolean] - case Or(l,r) => eval(l).asInstanceOf[Boolean] || eval(r).asInstanceOf[Boolean] + case Not(c) => + val child = eval(c) + if(child == null) + null + else + !child.asInstanceOf[Boolean] + case And(l,r) => + val left = eval(l) + val right = eval(r) + if(left == false || right == false) + false + else if(left == null || right == null ) + null + else + true + case Or(l,r) => + val left = eval(l) + val right = eval(r) + if(left == true || right == true) + true + else if(left == null || right == null) + null + else + false /* References to input tuples */ case br @ BoundReference(inputTuple, ordinal, _) => try input(inputTuple)(ordinal) catch { diff --git a/src/main/scala/shark2/basicOperators.scala b/src/main/scala/shark2/basicOperators.scala index 40d78c0a8cddf..768b331782bf6 100644 --- a/src/main/scala/shark2/basicOperators.scala +++ b/src/main/scala/shark2/basicOperators.scala @@ -69,7 +69,13 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { logger.debug(s"Comparing $left, $right as $curDataType order $curDirection") // TODO: Use numeric here too? val comparison = - if(curDataType == IntegerType) + if(left == null && right == null) + 0 + else if(left == null) + -1 + else if(right == null) + 1 + else if(curDataType == IntegerType) if(curDirection == Ascending) left.asInstanceOf[Int] compare right.asInstanceOf[Int] else diff --git a/src/test/scala/ExpressionEvaluationSuite.scala b/src/test/scala/ExpressionEvaluationSuite.scala index db182b6bbd66e..1bcbb82acd94d 100644 --- a/src/test/scala/ExpressionEvaluationSuite.scala +++ b/src/test/scala/ExpressionEvaluationSuite.scala @@ -14,20 +14,83 @@ class ExpressionEvaluationSuite extends FunSuite { assert(Evaluate(Literal(1) + Literal(1), Nil) === 2) } - test("boolean logic") { - val andTruthTable = - (true, true, true) :: - (true, false, false) :: - (false, true, false) :: - (false, false, false) :: Nil - - andTruthTable.foreach { - case (l,r,answer) => - val result = Evaluate(Literal(l, BooleanType) && Literal(r, BooleanType), Nil) + /** + * Checks for three-valued-logic. Based on: + * http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29 + * + * p q p OR q p AND q p = q + * True True True True True + * True False True False False + * True Unknown True Unknown Unknown + * False True True False False + * False False False False True + * False Unknown Unknown False Unknown + * Unknown True True Unknown Unknown + * Unknown False Unknown False Unknown + * Unknown Unknown Unknown Unknown Unknown + * + * p NOT p + * True False + * False True + * Unknown Unknown + */ + + val notTrueTable = + (true, false) :: + (false, true) :: + (null, null) :: Nil + + test("3VL Not") { + notTrueTable.foreach { + case (v, answer) => + val expr = Not(Literal(v, BooleanType)) + val result = Evaluate(expr, Nil) if(result != answer) - fail(s"$l && $r != $result") - } + fail(s"$expr should not evaluate to $result, expected: $answer") } + } + booleanLogicTest("AND", _ && _, + (true, true, true) :: + (true, false, false) :: + (true, null, null) :: + (false, true, false) :: + (false, false, false) :: + (false, null, false) :: + (null, true, null) :: + (null, false, false) :: + (null, null, null) :: Nil) + booleanLogicTest("OR", _ || _, + (true, true, true) :: + (true, false, true) :: + (true, null, true) :: + (false, true, true) :: + (false, false, false) :: + (false, null, null) :: + (null, true, true) :: + (null, false, null) :: + (null, null, null) :: Nil) + + booleanLogicTest("=", _ === _, + (true, true, true) :: + (true, false, false) :: + (true, null, null) :: + (false, true, false) :: + (false, false, true) :: + (false, null, null) :: + (null, true, null) :: + (null, false, null) :: + (null, null, null) :: Nil) + + def booleanLogicTest(name: String, op: (Expression, Expression) => Expression, truthTable: Seq[(Any, Any, Any)]) { + test(s"3VL $name") { + truthTable.foreach { + case (l,r,answer) => + val expr = op(Literal(l, BooleanType), Literal(r, BooleanType)) + val result = Evaluate(expr, Nil) + if(result != answer) + fail(s"$expr should not evaluate to $result, expected: $answer") + } + } } } \ No newline at end of file From 90c453d9bd1a469705d5b765a49631922fbc4751 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 25 Dec 2013 14:57:15 -0800 Subject: [PATCH 296/778] more tests passing! --- src/test/scala/shark2/HiveCompatability.scala | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index 8fd7320240aa2..8179783d2e590 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -43,16 +43,24 @@ class HiveCompatability extends HiveQueryFileTest { */ override def whiteList = Seq( "add_part_exist", + "alter4", + "alter_partition_format_loc", + "alter_table_serde", + "authorization_3", + "authorization_5", "auto_join23", + "auto_join24", "auto_join26", "auto_join28", "avro_change_schema", "bucketmapjoin6", + "combine1", "count", "ct_case_insensitive", "database_properties", "delimiter", "describe_database_json", + "describe_table_json", "diff_part_input_formats", "drop_function", "drop_index", @@ -63,11 +71,14 @@ class HiveCompatability extends HiveQueryFileTest { "drop_view", "groupby4_map", "groupby4_map_skew", + "groupby5_map", + "groupby5_map_skew", "index_auth", "index_auto_self_join", "index_auto_update", "index_stale", "innerjoin", + "inoutdriver", "input0", "input11", "input11_limit", @@ -78,6 +89,8 @@ class HiveCompatability extends HiveQueryFileTest { "input4_cb_delim", "input4_limit", "input7", + "input9", + "inputddl4", "inputddl8", "insert1", "join0", @@ -92,8 +105,10 @@ class HiveCompatability extends HiveQueryFileTest { "join18", "join19", "join2", + "join20", "join22", "join23", + "join24", "join25", "join27", "join29", @@ -104,8 +119,10 @@ class HiveCompatability extends HiveQueryFileTest { "join35", "join36", "join37", + "join40", "join_casesensitive", "join_empty", + "join_nulls", "join_reorder2", "join_reorder3", "join_view", @@ -126,6 +143,7 @@ class HiveCompatability extends HiveQueryFileTest { "nullinput", "nullinput2", "nullscript", + "part_inherit_tbl_props", "part_inherit_tbl_props_empty", "ppd1", "ppd_gby_join", @@ -141,6 +159,7 @@ class HiveCompatability extends HiveQueryFileTest { "ppd_udf_col", "ppd_union", "progress_1", + "protectmode2", "query_with_semi", "quote2", "select_as_omitted", From 3946e31b6e58dbe90a6dc3863b9fc3727c82f39b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 25 Dec 2013 15:35:19 -0800 Subject: [PATCH 297/778] don't build strings unless assertion fails. --- src/main/scala/expressions/Evaluate.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index 0a943508afd91..c804cd6a11e81 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -37,7 +37,7 @@ object Evaluate extends Logging { @inline def n2(e1: Expression, e2: Expression, f: ((Numeric[Any], Any, Any) => Any)): Any = { - assert(e1.dataType == e2.dataType, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + if (e1.dataType != e2.dataType) throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") e1.dataType match { case IntegerType => f.asInstanceOf[(Numeric[Int], Int, Int) => Int](implicitly[Numeric[Int]], eval(e1).asInstanceOf[Int], eval(e2).asInstanceOf[Int]) @@ -57,7 +57,7 @@ object Evaluate extends Logging { @inline def f2(e1: Expression, e2: Expression, f: ((Fractional[Any], Any, Any) => Any)): Any = { - assert(e1.dataType == e2.dataType, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + if (e1.dataType != e2.dataType) throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") e1.dataType match { case DoubleType => f.asInstanceOf[(Fractional[Double], Double, Double) => Double](implicitly[Fractional[Double]], eval(e1).asInstanceOf[Double], eval(e2).asInstanceOf[Double]) @@ -69,7 +69,7 @@ object Evaluate extends Logging { @inline def i2(e1: Expression, e2: Expression, f: ((Integral[Any], Any, Any) => Any)): Any = { - assert(e1.dataType == e2.dataType, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + if (e1.dataType != e2.dataType) throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") e1.dataType match { case IntegerType => f.asInstanceOf[(Integral[Int], Int, Int) => Int](implicitly[Integral[Int]], eval(e1).asInstanceOf[Int], eval(e2).asInstanceOf[Int]) From 69a0bd4b919dd679ac23e99691e324584c1429ce Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 25 Dec 2013 15:35:33 -0800 Subject: [PATCH 298/778] promote strings in predicates with number too. --- src/main/scala/analysis/typeCoercion.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/main/scala/analysis/typeCoercion.scala b/src/main/scala/analysis/typeCoercion.scala index 307c50eccb75d..543bce5cf0c7a 100644 --- a/src/main/scala/analysis/typeCoercion.scala +++ b/src/main/scala/analysis/typeCoercion.scala @@ -96,6 +96,11 @@ object PromoteStrings extends Rule[LogicalPlan] { case a: BinaryArithmetic if a.right.dataType == StringType => a.makeCopy(Array(a.left, Cast(a.right, DoubleType))) + case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType != StringType => + p.makeCopy(Array(Cast(p.left, DoubleType), p.right)) + case p: BinaryPredicate if p.left.dataType != StringType && p.right.dataType == StringType => + p.makeCopy(Array(p.left, Cast(p.right, DoubleType))) + case Sum(e) if e.dataType == StringType => Sum(Cast(e, DoubleType)) case Average(e) if e.dataType == StringType => From 14d070fe4ba5109f3ae809da53123082f1291638 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 25 Dec 2013 19:30:30 -0800 Subject: [PATCH 299/778] add support for correctly extracting partition keys. --- src/main/scala/shark2/hiveOperators.scala | 40 ++++++++++++---------- src/test/scala/shark2/HiveQueryTests.scala | 2 +- 2 files changed, 23 insertions(+), 19 deletions(-) diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index 29360a284a39b..628f33b9fc1af 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -5,6 +5,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils import org.apache.hadoop.hive.ql.plan.FileSinkDesc import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, StructObjectInspector} +import org.apache.hadoop.hive.serde2.`lazy`.LazyStruct import org.apache.hadoop.hive.serde2.Serializer import org.apache.hadoop.mapred.JobConf @@ -30,12 +31,25 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation relation.tableDesc.getDeserializer.getObjectInspector.asInstanceOf[StructObjectInspector] /** - * The hive struct field references that correspond to the attributes to be read from this table. + * Functions that extract the requested attributes from the hive output. */ @transient - lazy val refs = attributes.flatMap { a => - objectInspector.getAllStructFieldRefs - .find(_.getFieldName == a.name) + protected lazy val attributeFunctions = attributes.map { a => + if(relation.partitionKeys.contains(a)) { + val ordinal = relation.partitionKeys.indexOf(a) + (struct: LazyStruct, partitionKeys: Array[String]) => partitionKeys(ordinal) + } else { + val ref = + objectInspector.getAllStructFieldRefs + .find(_.getFieldName == a.name) + .getOrElse(sys.error(s"Can't find attribute $a")) + + (struct: LazyStruct, _: Array[String]) => { + val data = objectInspector.getStructFieldData(struct, ref) + val inspector = ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector] + inspector.getPrimitiveJavaObject(data) + } + } } @transient @@ -46,21 +60,11 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation hadoopReader.makeRDDForPartitionedTable(relation.hiveQlPartitions) def execute() = { - // TODO: THIS DOES NOT CORRECTLY RETURN PARTITION ATTRIBUTES. - def unpackStruct(struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct) = - refs.map { ref => - val data = objectInspector.getStructFieldData(struct, ref) - ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector].getPrimitiveJavaObject(data) - }.toIndexedSeq - inputRdd.map { - case array: Array[Any] => - val res = array.flatMap { - case struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct => unpackStruct(struct) - case array: Array[Any] => array - } - buildRow(res) - case struct: org.apache.hadoop.hive.serde2.`lazy`.LazyStruct => buildRow(unpackStruct(struct)) + case Array(struct: LazyStruct, partitionKeys: Array[String]) => + buildRow(attributeFunctions.map(_(struct, partitionKeys))) + case struct: LazyStruct => + buildRow(attributeFunctions.map(_(struct, Array.empty))) } } diff --git a/src/test/scala/shark2/HiveQueryTests.scala b/src/test/scala/shark2/HiveQueryTests.scala index bab8bdb43895a..59a4ba4d7b4f8 100644 --- a/src/test/scala/shark2/HiveQueryTests.scala +++ b/src/test/scala/shark2/HiveQueryTests.scala @@ -42,7 +42,7 @@ class HiveQueryTests extends HiveComaparisionTest { "SELECT length(\"test\") FROM src LIMIT 1") createQueryTest("partitioned table scan", - "SELECT * FROM srcpart") + "SELECT ds, hr, key, value FROM srcpart") createQueryTest("create table as", """ From ef8b0a5b7e3b0acff9de7c1e05c79cb7efad15d0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 25 Dec 2013 19:31:56 -0800 Subject: [PATCH 300/778] more tests. --- src/test/scala/shark2/HiveCompatability.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index 8179783d2e590..5d296da3266a2 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -19,6 +19,8 @@ class HiveCompatability extends HiveQueryFileTest { "query_properties", "sample10", "updateAccessTime", + //"load_dyn_part8", + "index_compact_binary_search", // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 @@ -53,11 +55,13 @@ class HiveCompatability extends HiveQueryFileTest { "auto_join26", "auto_join28", "avro_change_schema", + "binarysortable_1", "bucketmapjoin6", "combine1", "count", "ct_case_insensitive", "database_properties", + "default_partition_name", "delimiter", "describe_database_json", "describe_table_json", @@ -145,6 +149,7 @@ class HiveCompatability extends HiveQueryFileTest { "nullscript", "part_inherit_tbl_props", "part_inherit_tbl_props_empty", + "part_inherit_tbl_props_with_star", "ppd1", "ppd_gby_join", "ppd_join", @@ -159,9 +164,11 @@ class HiveCompatability extends HiveQueryFileTest { "ppd_udf_col", "ppd_union", "progress_1", + "protectmode", "protectmode2", "query_with_semi", "quote2", + "rename_column", "select_as_omitted", "set_variable_sub", "show_describe_func_quotes", From 2b70abf5fc1ec4322becf4d08aa3b99cd7c1f945 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 26 Dec 2013 11:26:51 -0800 Subject: [PATCH 301/778] true and false literals. --- src/main/scala/frontend/Hive.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 13ab2ece02ffd..10bafde92d649 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -16,7 +16,7 @@ import collection.JavaConversions._ import scala.collection.mutable /** - * A logical node that represent a non-query command to be executed by the system. For example, + * A logical node that represents a non-query command to be executed by the system. For example, * commands can be used by parsers to represent DDL operations. */ abstract class Command extends LeafNode { @@ -523,7 +523,9 @@ object HiveQl { val RAND = "(?i)RAND".r val AND = "(?i)AND".r val OR = "(?i)OR".r - val NOT = "(?i)Not".r + val NOT = "(?i)NOT".r + val TRUE = "(?i)TRUE".r + val FALSE = "(?i)FALSE".r protected def nodeToExpr(node: Node): Expression = node match { /* Attribute References */ @@ -583,6 +585,8 @@ object HiveQl { /* Literals */ case Token("TOK_NULL", Nil) => Literal(null, IntegerType) // TODO: What type is null? + case Token(TRUE(), Nil) => Literal(true, BooleanType) // TODO: What type is null? + case Token(FALSE(), Nil) => Literal(false, BooleanType) // TODO: What type is null? case Token("TOK_STRINGLITERALSEQUENCE", strings) => Literal(strings.map(s => BaseSemanticAnalyzer.unescapeSQLString(s.asInstanceOf[ASTNode].getText)).mkString) From 0daaa0e193e7bc30815e3523eeec2466a7d7c0a9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 26 Dec 2013 16:45:30 -0800 Subject: [PATCH 302/778] Promote booleans that appear in comparisons. --- src/main/scala/analysis/Analyzer.scala | 3 ++- src/main/scala/analysis/typeCoercion.scala | 12 ++++++++++++ src/main/scala/expressions/predicates.scala | 15 ++++++++++----- 3 files changed, 24 insertions(+), 6 deletions(-) diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala index 187c1d96f70a9..6461fa26bcdd7 100644 --- a/src/main/scala/analysis/Analyzer.scala +++ b/src/main/scala/analysis/Analyzer.scala @@ -31,7 +31,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool Batch("Type Coersion", fixedPoint, PromoteNumericTypes, PromoteStrings, - ConvertNaNs) + ConvertNaNs, + BooleanComparisons) ) /** diff --git a/src/main/scala/analysis/typeCoercion.scala b/src/main/scala/analysis/typeCoercion.scala index 543bce5cf0c7a..f9c2aca5200bf 100644 --- a/src/main/scala/analysis/typeCoercion.scala +++ b/src/main/scala/analysis/typeCoercion.scala @@ -106,4 +106,16 @@ object PromoteStrings extends Rule[LogicalPlan] { case Average(e) if e.dataType == StringType => Sum(Cast(e, DoubleType)) } +} + +object BooleanComparisons extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + // No need to change Equals operators as that actually makes sense for boolean types. + case e: Equals => e + // Otherwise turn them to Byte types so that there exists and ordering. + case p: BinaryComparison if p.left.dataType == BooleanType && p.right.dataType == BooleanType => + p.makeCopy(Array(Cast(p.left, ByteType), Cast(p.right, ByteType))) + } } \ No newline at end of file diff --git a/src/main/scala/expressions/predicates.scala b/src/main/scala/expressions/predicates.scala index 347d995397053..cbb543a32d131 100644 --- a/src/main/scala/expressions/predicates.scala +++ b/src/main/scala/expressions/predicates.scala @@ -27,19 +27,24 @@ case class Or(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "||" } -case class Equals(left: Expression, right: Expression) extends BinaryPredicate { +abstract class BinaryComparison extends BinaryPredicate { + self: Product => +} + +case class Equals(left: Expression, right: Expression) extends BinaryComparison { def symbol = "=" } -case class LessThan(left: Expression, right: Expression) extends BinaryPredicate { + +case class LessThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<" } -case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryPredicate { +case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<=" } -case class GreaterThan(left: Expression, right: Expression) extends BinaryPredicate { +case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">" } -case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryPredicate { +case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">=" } From 21f0d91a312a86c18a9f5dcbd23eb255754d6528 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 10:38:02 -0800 Subject: [PATCH 303/778] Simple example of schemaRdd with scala filter function. --- src/main/scala/dsl.scala | 4 ++++ .../scala/examples/SchemaRddExample.scala | 20 +++++++++++++++++++ src/main/scala/expressions/ScalaUdf.scala | 18 +++++++++++++++++ 3 files changed, 42 insertions(+) create mode 100644 src/main/scala/examples/SchemaRddExample.scala create mode 100644 src/main/scala/expressions/ScalaUdf.scala diff --git a/src/main/scala/dsl.scala b/src/main/scala/dsl.scala index 8ff22fea1d98b..631c9515e47b2 100644 --- a/src/main/scala/dsl.scala +++ b/src/main/scala/dsl.scala @@ -1,5 +1,6 @@ package catalyst +import analysis.UnresolvedAttribute import expressions._ import plans._ import plans.logical._ @@ -105,6 +106,9 @@ package object dsl { def subquery(alias: Symbol) = Subquery(alias.name, plan) def unionAll(otherPlan: LogicalPlan) = Union(plan, otherPlan) + def filter[T1](arg1: Symbol)(udf: (T1) => Boolean) = + Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), plan) + def analyze = analysis.SimpleAnalyzer(plan) } } \ No newline at end of file diff --git a/src/main/scala/examples/SchemaRddExample.scala b/src/main/scala/examples/SchemaRddExample.scala new file mode 100644 index 0000000000000..f6318c1d23cdd --- /dev/null +++ b/src/main/scala/examples/SchemaRddExample.scala @@ -0,0 +1,20 @@ +package catalyst +package examples + +import plans.logical.LocalRelation + +import shark2.TestShark._ +import dsl._ + +object SchemaRddExample { + def main(args: Array[String]): Unit = { + val testLogs = LocalRelation('date.string, 'message.string).loadData( + ("12/1/2013", "INFO: blah blah") :: + ("12/2/2013", "WARN: blah blah") :: Nil + ) + + val filtered = testLogs.filter('date)((date: String) => new java.util.Date(date).getDay == 1) + + filtered.toRdd.collect.foreach(println) + } +} \ No newline at end of file diff --git a/src/main/scala/expressions/ScalaUdf.scala b/src/main/scala/expressions/ScalaUdf.scala new file mode 100644 index 0000000000000..6681e17c01a75 --- /dev/null +++ b/src/main/scala/expressions/ScalaUdf.scala @@ -0,0 +1,18 @@ +package catalyst +package expressions + +import types._ + +case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) + extends Expression with ImplementedUdf { + + def references = children.flatMap(_.references).toSet + def nullable = true + + def evaluate(evaluatedChildren: Seq[Any]): Any = { + children.size match { + case 1 => function.asInstanceOf[Function1[Any, Any]](evaluatedChildren(0)) + case 2 => function.asInstanceOf[Function2[Any, Any, Any]](evaluatedChildren(0), evaluatedChildren(1)) + } + } +} \ No newline at end of file From 8d5f5048f45744be3f7725edf0d5e11b32d96042 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 14:36:49 -0800 Subject: [PATCH 304/778] Example of schema RDD using scala's dynamic trait, resulting in a more standard ORM style of usage. --- src/main/scala/dsl.scala | 3 +++ .../scala/examples/SchemaRddExample.scala | 23 +++++++++++++++- src/main/scala/expressions/WrapDynamic.scala | 26 +++++++++++++++++++ src/main/scala/types/dataTypes.scala | 2 +- 4 files changed, 52 insertions(+), 2 deletions(-) create mode 100644 src/main/scala/expressions/WrapDynamic.scala diff --git a/src/main/scala/dsl.scala b/src/main/scala/dsl.scala index 631c9515e47b2..8517ff0dc8a4f 100644 --- a/src/main/scala/dsl.scala +++ b/src/main/scala/dsl.scala @@ -109,6 +109,9 @@ package object dsl { def filter[T1](arg1: Symbol)(udf: (T1) => Boolean) = Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), plan) + def filter(dynamicUdf: (DynamicRow) => Boolean) = + Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(plan.output))), plan) + def analyze = analysis.SimpleAnalyzer(plan) } } \ No newline at end of file diff --git a/src/main/scala/examples/SchemaRddExample.scala b/src/main/scala/examples/SchemaRddExample.scala index f6318c1d23cdd..89a64bd81b11f 100644 --- a/src/main/scala/examples/SchemaRddExample.scala +++ b/src/main/scala/examples/SchemaRddExample.scala @@ -8,13 +8,34 @@ import dsl._ object SchemaRddExample { def main(args: Array[String]): Unit = { + // Create an RDD with schema (data STRING, message STRING) and load some sample data into it. + // Acceptable base data includes Seq[Any] or Seq[TupleN]. val testLogs = LocalRelation('date.string, 'message.string).loadData( ("12/1/2013", "INFO: blah blah") :: ("12/2/2013", "WARN: blah blah") :: Nil ) + /** + * Example using the symbol based API. In this example, the attribute names that are passed to + * the first constructor are resolved during catalyst's analysis phase. Then at runtime only + * the requested attributes are passed to the UDF. Since this analysis occurs at runtime, + * the developer must manually annotate their function with the correct argument types. + */ val filtered = testLogs.filter('date)((date: String) => new java.util.Date(date).getDay == 1) - filtered.toRdd.collect.foreach(println) + + + /** + * Example using the dynamic, ORM-Style API. Using this API the developer is passed a dynamic + * row containing all of the attributes from the child operator. Method calls to the dynamic + * row result in run-time look-ups for the requested column. + * + * Essentially, this means that the call row.attrName is translated by the scala compiler to + * row.selectDynamic("attrName"). Note that, in this instance, the requested attribute is + * being resolved at runtime. Thus, we cannot return typed results. As such all dynamic calls + * always return strings. + */ + val filtered2 = testLogs.filter(row => new java.util.Date(row.date).getDay == 1) + filtered2.toRdd.collect.foreach(println) } } \ No newline at end of file diff --git a/src/main/scala/expressions/WrapDynamic.scala b/src/main/scala/expressions/WrapDynamic.scala new file mode 100644 index 0000000000000..128d3de9cb577 --- /dev/null +++ b/src/main/scala/expressions/WrapDynamic.scala @@ -0,0 +1,26 @@ +package catalyst +package expressions + +import scala.language.dynamics + +import types._ + +case object DynamicType extends DataType + +case class WrapDynamic(children: Seq[Attribute]) extends Expression with ImplementedUdf { + def nullable = false + def references = children.toSet + def dataType = DynamicType + + def evaluate(evaluatedChildren: Seq[Any]): Any = + new DynamicRow(children, evaluatedChildren) +} + +class DynamicRow(val schema: Seq[Attribute], values: Seq[Any]) + extends GenericRow(values) with Dynamic { + + def selectDynamic(attributeName: String): String = { + val ordinal = schema.indexWhere(_.name == attributeName) + values(ordinal).toString + } +} \ No newline at end of file diff --git a/src/main/scala/types/dataTypes.scala b/src/main/scala/types/dataTypes.scala index 034b031f36452..a52e667bcf416 100644 --- a/src/main/scala/types/dataTypes.scala +++ b/src/main/scala/types/dataTypes.scala @@ -1,7 +1,7 @@ package catalyst package types -sealed class DataType +class DataType case object IntegerType extends DataType case object StringType extends DataType From d4080213325bcdb8b6386d6757ecb403549622d0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 14:59:30 -0800 Subject: [PATCH 305/778] support for printing out arrays in the output in the same form as hive (e.g., [e1, e1]). --- src/main/scala/shark2/SharkInstance.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/src/main/scala/shark2/SharkInstance.scala b/src/main/scala/shark2/SharkInstance.scala index 825eafd3fa6a7..80cf715afc945 100644 --- a/src/main/scala/shark2/SharkInstance.scala +++ b/src/main/scala/shark2/SharkInstance.scala @@ -112,6 +112,12 @@ abstract class SharkInstance extends Logging { lazy val toRdd = executedPlan.execute() + def toHiveString(a: Any): String = a match { + case seq: Seq[_] => seq.map(toHiveString).map(s => "\"" + s + "\"").mkString("[", ",", "]") + case null => "NULL" + case other => other.toString + } + /** * Returns the result as a hive compatible sequence of strings. For native commands, the execution is simply * passed back to Hive. @@ -123,11 +129,7 @@ abstract class SharkInstance extends Logging { case query => val result: Seq[Seq[Any]] = toRdd.collect.toSeq // Reformat to match hive tab delimited output. - val asString = result.map(_.map { - case null => "NULL" - case other => other - }).map(_.mkString("\t")).toSeq - + val asString = result.map(_.map(toHiveString)).map(_.mkString("\t")).toSeq asString } From adab892751c2e10dc5bd1151fe13c52c114de09a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 14:59:57 -0800 Subject: [PATCH 306/778] Clear out functions that are created during tests when reset is called. --- src/main/scala/shark2/TestShark.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index 7d47bcca4f02b..36114fa78efae 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -13,6 +13,7 @@ import frontend.hive._ import util._ import collection.JavaConversions._ +import org.apache.hadoop.hive.ql.exec.FunctionRegistry /** @@ -165,6 +166,12 @@ object TestShark extends SharkInstance { } } + /** + * Records the UDFs present when the server starts, so we can delete ones that are created by + * tests. + */ + protected val originalUdfs = FunctionRegistry.getFunctionNames() + /** * Resets the test instance by deleting any tables that have been created. * TODO: also clear out UDFs, views, etc. @@ -197,6 +204,10 @@ object TestShark extends SharkInstance { catalog.client.dropDatabase(db, true, false, true) } + FunctionRegistry.getFunctionNames.filterNot(originalUdfs.contains(_)).foreach { udfName => + FunctionRegistry.unregisterTemporaryUDF(udfName) + } + configure() runSqlHive("USE default") From e5690a69da81fd9a3e8e3a00584c50d06df93d31 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 15:39:51 -0800 Subject: [PATCH 307/778] add BinaryType --- src/main/scala/types/dataTypes.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/scala/types/dataTypes.scala b/src/main/scala/types/dataTypes.scala index a52e667bcf416..4decfb607ac62 100644 --- a/src/main/scala/types/dataTypes.scala +++ b/src/main/scala/types/dataTypes.scala @@ -3,8 +3,10 @@ package types class DataType -case object IntegerType extends DataType case object StringType extends DataType +case object BinaryType extends DataType + +case object IntegerType extends DataType case object BooleanType extends DataType case object FloatType extends DataType case object DoubleType extends DataType From 3fe24ec8fe3895972a4db29aeff91bffd394f124 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 15:40:18 -0800 Subject: [PATCH 308/778] better implementation of 3vl in Evaluate, fix some > 100 char lines. --- src/main/scala/expressions/Evaluate.scala | 172 +++++++++++++++------- 1 file changed, 115 insertions(+), 57 deletions(-) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index c804cd6a11e81..1a2bf1814b84e 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -19,68 +19,111 @@ object Evaluate extends Logging { * themselves. */ @inline - def n1(e: Expression, f: ((Numeric[Any], Any) => Any)): Any = e.dataType match { - case IntegerType => - f.asInstanceOf[(Numeric[Int], Int) => Int](implicitly[Numeric[Int]], eval(e).asInstanceOf[Int]) - case DoubleType => - f.asInstanceOf[(Numeric[Double], Double) => Double](implicitly[Numeric[Double]], eval(e).asInstanceOf[Double]) - case LongType => - f.asInstanceOf[(Numeric[Long], Long) => Long](implicitly[Numeric[Long]], eval(e).asInstanceOf[Long]) - case FloatType => - f.asInstanceOf[(Numeric[Float], Float) => Float](implicitly[Numeric[Float]], eval(e).asInstanceOf[Float]) - case ByteType => - f.asInstanceOf[(Numeric[Byte], Byte) => Byte](implicitly[Numeric[Byte]], eval(e).asInstanceOf[Byte]) - case ShortType => - f.asInstanceOf[(Numeric[Short], Short) => Short](implicitly[Numeric[Short]], eval(e).asInstanceOf[Short]) - case other => sys.error(s"Type $other does not support numeric operations") + def n1(e: Expression, f: ((Numeric[Any], Any) => Any)): Any = { + val evalE = eval(e) + if(evalE == null) + null + else + e.dataType match { + case IntegerType => + f.asInstanceOf[(Numeric[Int], Int) => Int]( + implicitly[Numeric[Int]], eval(e).asInstanceOf[Int]) + case DoubleType => + f.asInstanceOf[(Numeric[Double], Double) => Double]( + implicitly[Numeric[Double]], eval(e).asInstanceOf[Double]) + case LongType => + f.asInstanceOf[(Numeric[Long], Long) => Long]( + implicitly[Numeric[Long]], eval(e).asInstanceOf[Long]) + case FloatType => + f.asInstanceOf[(Numeric[Float], Float) => Float]( + implicitly[Numeric[Float]], eval(e).asInstanceOf[Float]) + case ByteType => + f.asInstanceOf[(Numeric[Byte], Byte) => Byte]( + implicitly[Numeric[Byte]], eval(e).asInstanceOf[Byte]) + case ShortType => + f.asInstanceOf[(Numeric[Short], Short) => Short]( + implicitly[Numeric[Short]], eval(e).asInstanceOf[Short]) + case other => sys.error(s"Type $other does not support numeric operations") + } } @inline def n2(e1: Expression, e2: Expression, f: ((Numeric[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") - e1.dataType match { - case IntegerType => - f.asInstanceOf[(Numeric[Int], Int, Int) => Int](implicitly[Numeric[Int]], eval(e1).asInstanceOf[Int], eval(e2).asInstanceOf[Int]) - case DoubleType => - f.asInstanceOf[(Numeric[Double], Double, Double) => Double](implicitly[Numeric[Double]], eval(e1).asInstanceOf[Double], eval(e2).asInstanceOf[Double]) - case LongType => - f.asInstanceOf[(Numeric[Long], Long, Long) => Long](implicitly[Numeric[Long]], eval(e1).asInstanceOf[Long], eval(e2).asInstanceOf[Long]) - case FloatType => - f.asInstanceOf[(Numeric[Float], Float, Float) => Float](implicitly[Numeric[Float]], eval(e1).asInstanceOf[Float], eval(e2).asInstanceOf[Float]) - case ByteType => - f.asInstanceOf[(Numeric[Byte], Byte, Byte) => Byte](implicitly[Numeric[Byte]], eval(e1).asInstanceOf[Byte], eval(e2).asInstanceOf[Byte]) - case ShortType => - f.asInstanceOf[(Numeric[Short], Short, Short) => Short](implicitly[Numeric[Short]], eval(e1).asInstanceOf[Short], eval(e2).asInstanceOf[Short]) - case other => sys.error(s"Type $other does not support numeric operations") - } + if (e1.dataType != e2.dataType) + throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + + val evalE1 = eval(e1) + val evalE2 = eval(e2) + if(evalE1 == null || evalE2 == null) + null + else + e1.dataType match { + case IntegerType => + f.asInstanceOf[(Numeric[Int], Int, Int) => Int]( + implicitly[Numeric[Int]], evalE1.asInstanceOf[Int], evalE2.asInstanceOf[Int]) + case DoubleType => + f.asInstanceOf[(Numeric[Double], Double, Double) => Double]( + implicitly[Numeric[Double]], evalE1.asInstanceOf[Double], evalE2.asInstanceOf[Double]) + case LongType => + f.asInstanceOf[(Numeric[Long], Long, Long) => Long]( + implicitly[Numeric[Long]], evalE1.asInstanceOf[Long], evalE2.asInstanceOf[Long]) + case FloatType => + f.asInstanceOf[(Numeric[Float], Float, Float) => Float]( + implicitly[Numeric[Float]], evalE1.asInstanceOf[Float], evalE2.asInstanceOf[Float]) + case ByteType => + f.asInstanceOf[(Numeric[Byte], Byte, Byte) => Byte]( + implicitly[Numeric[Byte]], evalE1.asInstanceOf[Byte], evalE2.asInstanceOf[Byte]) + case ShortType => + f.asInstanceOf[(Numeric[Short], Short, Short) => Short]( + implicitly[Numeric[Short]], evalE1.asInstanceOf[Short], evalE2.asInstanceOf[Short]) + case other => sys.error(s"Type $other does not support numeric operations") + } } @inline def f2(e1: Expression, e2: Expression, f: ((Fractional[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") - e1.dataType match { - case DoubleType => - f.asInstanceOf[(Fractional[Double], Double, Double) => Double](implicitly[Fractional[Double]], eval(e1).asInstanceOf[Double], eval(e2).asInstanceOf[Double]) - case FloatType => - f.asInstanceOf[(Fractional[Float], Float, Float) => Float](implicitly[Fractional[Float]], eval(e1).asInstanceOf[Float], eval(e2).asInstanceOf[Float]) - case other => sys.error(s"Type $other does not support fractional operations") - } + if (e1.dataType != e2.dataType) + throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + + val evalE1 = eval(e1) + val evalE2 = eval(e2) + if(evalE1 == null || evalE2 == null) + null + else + e1.dataType match { + case DoubleType => + f.asInstanceOf[(Fractional[Double], Double, Double) => Double]( + implicitly[Fractional[Double]], evalE1.asInstanceOf[Double], evalE2.asInstanceOf[Double]) + case FloatType => + f.asInstanceOf[(Fractional[Float], Float, Float) => Float]( + implicitly[Fractional[Float]], evalE1.asInstanceOf[Float], evalE2.asInstanceOf[Float]) + case other => sys.error(s"Type $other does not support fractional operations") + } } @inline def i2(e1: Expression, e2: Expression, f: ((Integral[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") - e1.dataType match { - case IntegerType => - f.asInstanceOf[(Integral[Int], Int, Int) => Int](implicitly[Integral[Int]], eval(e1).asInstanceOf[Int], eval(e2).asInstanceOf[Int]) - case LongType => - f.asInstanceOf[(Integral[Long], Long, Long) => Long](implicitly[Integral[Long]], eval(e1).asInstanceOf[Long], eval(e2).asInstanceOf[Long]) - case ByteType => - f.asInstanceOf[(Integral[Byte], Byte, Byte) => Byte](implicitly[Integral[Byte]], eval(e1).asInstanceOf[Byte], eval(e2).asInstanceOf[Byte]) - case ShortType => - f.asInstanceOf[(Integral[Short], Short, Short) => Short](implicitly[Integral[Short]], eval(e1).asInstanceOf[Short], eval(e2).asInstanceOf[Short]) - case other => sys.error(s"Type $other does not support numeric operations") - } + val evalE1 = eval(e1) + val evalE2 = eval(e2) + if(evalE1 == null || evalE2 == null) + null + else + e1.dataType match { + case IntegerType => + f.asInstanceOf[(Integral[Int], Int, Int) => Int]( + implicitly[Integral[Int]], evalE1.asInstanceOf[Int], evalE2.asInstanceOf[Int]) + case LongType => + f.asInstanceOf[(Integral[Long], Long, Long) => Long]( + implicitly[Integral[Long]], evalE1.asInstanceOf[Long], evalE2.asInstanceOf[Long]) + case ByteType => + f.asInstanceOf[(Integral[Byte], Byte, Byte) => Byte]( + implicitly[Integral[Byte]], evalE1.asInstanceOf[Byte], evalE2.asInstanceOf[Byte]) + case ShortType => + f.asInstanceOf[(Integral[Short], Short, Short) => Short]( + implicitly[Integral[Short]], evalE1.asInstanceOf[Short], evalE2.asInstanceOf[Short]) + case other => sys.error(s"Type $other does not support numeric operations") + } } val result = e match { @@ -131,10 +174,23 @@ object Evaluate extends Logging { /* Casts */ // toString - case Cast(e, StringType) => eval(e).toString + case Cast(e, StringType) => + eval(e) match { + case null => null + case other => other.toString + } + // String => Numeric Types case Cast(e, IntegerType) if e.dataType == StringType => eval(e).asInstanceOf[String].toInt case Cast(e, DoubleType) if e.dataType == StringType => eval(e).asInstanceOf[String].toDouble + // Boolean conversions + case Cast(e, ByteType) if e.dataType == BooleanType => + eval(e) match { + case null => null + case true => 1.toByte + case false => 0.toByte + } + // Numeric Type => Numeric Type case Cast(e, IntegerType) => n1(e, _.toInt(_)) case Cast(e, DoubleType) => n1(e, _.toDouble(_)) @@ -145,11 +201,11 @@ object Evaluate extends Logging { /* Boolean Logic */ case Not(c) => - val child = eval(c) - if(child == null) - null - else - !child.asInstanceOf[Boolean] + eval(c) match { + case null => null + case b: Boolean => !b + } + case And(l,r) => val left = eval(l) val right = eval(r) @@ -171,14 +227,16 @@ object Evaluate extends Logging { /* References to input tuples */ case br @ BoundReference(inputTuple, ordinal, _) => try input(inputTuple)(ordinal) catch { - case iob: IndexOutOfBoundsException => throw new OptimizationException(br, s"Reference not in tuple: $input") + case iob: IndexOutOfBoundsException => + throw new OptimizationException(br, s"Reference not in tuple: $input") } /* Functions */ case Rand => scala.util.Random.nextDouble /* UDFs */ - case implementedFunction: ImplementedUdf => implementedFunction.evaluate(implementedFunction.children.map(eval)) + case implementedFunction: ImplementedUdf => + implementedFunction.evaluate(implementedFunction.children.map(eval)) case other => throw new OptimizationException(other, "evaluation not implemented") } From 43db061d35fcdfe2ea6e97e6b6b473802d0f6cc3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 15:40:45 -0800 Subject: [PATCH 309/778] significant generalization of hive udf functionality. --- src/main/scala/shark2/FunctionRegistry.scala | 182 ++++++++++++++++--- 1 file changed, 152 insertions(+), 30 deletions(-) diff --git a/src/main/scala/shark2/FunctionRegistry.scala b/src/main/scala/shark2/FunctionRegistry.scala index 69e7fd0557e54..f0f73e5985f9f 100644 --- a/src/main/scala/shark2/FunctionRegistry.scala +++ b/src/main/scala/shark2/FunctionRegistry.scala @@ -3,59 +3,181 @@ package shark2 import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory + +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF +import org.apache.hadoop.hive.ql.exec.UDF + import expressions._ import types._ -import org.apache.hadoop.io.{DoubleWritable, LongWritable, IntWritable, Text} +import org.apache.hadoop.io._ import collection.JavaConversions._ +import scala.Predef._ object HiveFunctionRegistry extends analysis.FunctionRegistry { def lookupFunction(name: String, children: Seq[Expression]): Expression = { // We only look it up to see if it exists, but do not include it in the HiveUDF since it is not always serializable. - FunctionRegistry.getFunctionInfo(name) - // TODO: Check that the types match up. - HiveUdf(name, IntegerType, children) + val functionInfo = + Option(FunctionRegistry.getFunctionInfo(name)) + .getOrElse(sys.error(s"Couldn't find function $name")) + + if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { + val functionInfo = FunctionRegistry.getFunctionInfo(name) + val function = functionInfo.getFunctionClass.newInstance.asInstanceOf[UDF] + val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) + + lazy val expectedDataTypes = method.getParameterTypes.map(javaClassToDataType) + + HiveSimpleUdf( + name, + children.zip(expectedDataTypes).map { case (e, t) => Cast(e,t) } + ) + } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveGenericUdf(name, IntegerType, children) + } else { + sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") + } + } + + def javaClassToDataType(clz: Class[_]): DataType = clz match { + case c: Class[_] if c == classOf[DoubleWritable] => DoubleType + case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.DoubleWritable] => DoubleType + case c: Class[_] if c == classOf[Text] => StringType + case c: Class[_] if c == classOf[org.apache.hadoop.io.IntWritable] => IntegerType + case c: Class[_] if c == classOf[org.apache.hadoop.io.LongWritable] => LongType + case c: Class[_] if c == classOf[java.lang.String] => StringType + case c: Class[_] if c == java.lang.Short.TYPE => ShortType + case c: Class[_] if c == java.lang.Long.TYPE => LongType + case c: Class[_] if c == java.lang.Double.TYPE => DoubleType + case c: Class[_] if c == java.lang.Byte.TYPE => ByteType + case c: Class[_] if c == java.lang.Float.TYPE => FloatType + case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + case c: Class[_] if c == classOf[java.lang.Short] => ShortType + case c: Class[_] if c == classOf[java.lang.Long] => LongType + case c: Class[_] if c == classOf[java.lang.Double] => DoubleType + case c: Class[_] if c == classOf[java.lang.Byte] => ByteType + case c: Class[_] if c == classOf[java.lang.Float] => FloatType + case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType } } -case class HiveUdf( - name: String, - dataType: DataType, - children: Seq[Expression]) extends Expression with ImplementedUdf { +abstract class HiveUdf extends Expression with ImplementedUdf with Logging { + self: Product => + + type UDFType + val name: String + def nullable = true def references = children.flatMap(_.references).toSet // FunctionInfo is not serializable so we must look it up here again. lazy val functionInfo = FunctionRegistry.getFunctionInfo(name) - lazy val function = functionInfo.getFunctionClass.newInstance.asInstanceOf[org.apache.hadoop.hive.ql.exec.UDF] + lazy val function = functionInfo.getFunctionClass.newInstance.asInstanceOf[UDFType] + + override def toString = s"${nodeName}#${functionInfo.getDisplayName}(${children.mkString(",")})" + + def unwrap(a: Any): Any = a match { + case null => null + case i: IntWritable => i.get + case t: Text => t.toString + case l: LongWritable => l.get + case d: DoubleWritable => d.get() + case d: org.apache.hadoop.hive.serde2.io.DoubleWritable => d.get + case b: BooleanWritable => b.get() + case list: java.util.List[_] => list.map(unwrap) + case s: java.lang.Short => s + case s: java.lang.Long => s + case s: java.lang.Integer => s + case s: java.lang.Double => s + case s: java.lang.Byte => s + case str: String => str + } +} + +case class HiveSimpleUdf( + name: String, + children: Seq[Expression]) extends HiveUdf { + import HiveFunctionRegistry._ + type UDFType = UDF + lazy val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) + lazy val dataType = javaClassToDataType(method.getReturnType) + + lazy val wrappers = method.getParameterTypes.map { argClass => + val primitiveClasses = Seq( + Integer.TYPE, classOf[java.lang.Integer], classOf[java.lang.String], java.lang.Double.TYPE, + classOf[java.lang.Double], java.lang.Long.TYPE, classOf[java.lang.Long] + ) + val matchingConstructor = + argClass.getConstructors.find(c => + c.getParameterTypes.size == 1 && + primitiveClasses.contains(c.getParameterTypes.head)) + + val constructor = + matchingConstructor + .getOrElse(sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}.")) + + (a: Any) => { + logger.debug(s"Wrapping $a of type ${if(a == null) "null" else a.getClass.getName} using $constructor.") + // We must make sure that primitives get boxed java style. + if(a == null) + null + else + constructor.newInstance(a match { + case i: Int => i: java.lang.Integer + case other: AnyRef => other + }).asInstanceOf[AnyRef] + } + } // TODO: Finish input output types. def evaluate(evaluatedChildren: Seq[Any]): Any = { // Wrap the function arguments in the expected types. - val args = evaluatedChildren.zip(method.getParameterTypes).map { - case (null, _) => null - case (arg: Double, argClass) if argClass.isAssignableFrom(classOf[DoubleWritable]) => - new DoubleWritable(arg) - case (arg: Double, argClass) if argClass.isAssignableFrom(classOf[org.apache.hadoop.hive.serde2.io.DoubleWritable]) => - new org.apache.hadoop.hive.serde2.io.DoubleWritable(arg) - case (arg: Int, argClass) if argClass.isAssignableFrom(classOf[org.apache.hadoop.hive.serde2.io.DoubleWritable]) => - new org.apache.hadoop.hive.serde2.io.DoubleWritable(arg) - case (arg: Int, argClass) if argClass.isAssignableFrom(classOf[IntWritable]) => - new IntWritable(arg) - case (arg, argClass) => - argClass.getConstructor(arg.getClass).newInstance(arg.asInstanceOf[AnyRef]).asInstanceOf[AnyRef] - }.toArray + val args = evaluatedChildren.zip(wrappers).map { + case (arg, wrapper) => wrapper(arg) + } // Invoke the udf and unwrap the result. - method.invoke(function, args: _*) match { - case i: IntWritable => i.get - case t: Text => t.toString - case l: LongWritable => l.get - case null => null - case other => other - } + unwrap(method.invoke(function, args: _*)) + } +} + +case class HiveGenericUdf( + name: String, + dataType: DataType, + children: Seq[Expression]) extends HiveUdf { + import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ + type UDFType = GenericUDF + + lazy val inspectors = children.map(_.dataType).map { + case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector + case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector + case DoubleType => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector + case BooleanType => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector + case LongType => PrimitiveObjectInspectorFactory.javaLongObjectInspector + case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector + case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector + } + lazy val instance = { + function.initialize(inspectors.toArray) + function } - override def toString = s"${functionInfo.getDisplayName}(${children.mkString(",")})" + def wrap(a: Any): Any = a match { + case s: String => new Text(s) + case i: Int => i: java.lang.Integer + case b: Boolean => b: java.lang.Boolean + case d: Double => d: java.lang.Double + case l: Long => l: java.lang.Long + case l: Short => l: java.lang.Short + case l: Byte => l: java.lang.Byte + case s: Seq[_] => seqAsJavaList(s.map(wrap)) + case null => null // NullWritable.get() + } + + def evaluate(evaluatedChildren: Seq[Any]): Any = { + val args = evaluatedChildren.map(wrap).map(v => new DeferredJavaObject(v): DeferredObject).toArray + unwrap(instance.evaluate(args)) + } } \ No newline at end of file From 6c27aa7557cf2ca1e4e0ecfcbe103ec612ac1790 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 15:40:58 -0800 Subject: [PATCH 310/778] more cast parsing. --- src/main/scala/frontend/Hive.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 10bafde92d649..7b11abf708492 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -552,7 +552,10 @@ object HiveQl { case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), IntegerType) case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), FloatType) - case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), DoubleType) + case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ShortType) + case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ByteType) + case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BinaryType) /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) From 0831a3ca128909979822c3e6a16ee6f7797b5ddc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 15:41:20 -0800 Subject: [PATCH 311/778] support for parsing some operator udfs. --- src/main/scala/frontend/Hive.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 7b11abf708492..4ddd4e0e3f1b9 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -563,6 +563,7 @@ object HiveQl { case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right)) case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) case Token("/", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) + case Token("DIV", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) /* Comparisons */ case Token("=", left :: right:: Nil) => Equals(nodeToExpr(left), nodeToExpr(right)) @@ -571,6 +572,9 @@ object HiveQl { case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right)) + case Token("LIKE", left :: right:: Nil) => UnresolvedFunction("LIKE", Seq(nodeToExpr(left), nodeToExpr(right))) + case Token("RLIKE", left :: right:: Nil) => UnresolvedFunction("RLIKE", Seq(nodeToExpr(left), nodeToExpr(right))) + case Token("REGEXP", left :: right:: Nil) => UnresolvedFunction("REGEXP", Seq(nodeToExpr(left), nodeToExpr(right))) case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => IsNotNull(nodeToExpr(child)) case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => IsNull(nodeToExpr(child)) From 65f4e6984b7e7417cbd0f1990bdc4bb8829d0979 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 15:41:33 -0800 Subject: [PATCH 312/778] remove incorrect comments --- src/main/scala/frontend/Hive.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 4ddd4e0e3f1b9..9d771c429c793 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -592,8 +592,8 @@ object HiveQl { /* Literals */ case Token("TOK_NULL", Nil) => Literal(null, IntegerType) // TODO: What type is null? - case Token(TRUE(), Nil) => Literal(true, BooleanType) // TODO: What type is null? - case Token(FALSE(), Nil) => Literal(false, BooleanType) // TODO: What type is null? + case Token(TRUE(), Nil) => Literal(true, BooleanType) + case Token(FALSE(), Nil) => Literal(false, BooleanType) case Token("TOK_STRINGLITERALSEQUENCE", strings) => Literal(strings.map(s => BaseSemanticAnalyzer.unescapeSQLString(s.asInstanceOf[ASTNode].getText)).mkString) From 495d9dc6972f2e4579e3f49d56a06a99b7591942 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 15:51:28 -0800 Subject: [PATCH 313/778] Add an expression for when we decide to support LIKE natively instead of using the HIVE udf. --- src/main/scala/expressions/stringOperations.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 src/main/scala/expressions/stringOperations.scala diff --git a/src/main/scala/expressions/stringOperations.scala b/src/main/scala/expressions/stringOperations.scala new file mode 100644 index 0000000000000..d836c80a334f3 --- /dev/null +++ b/src/main/scala/expressions/stringOperations.scala @@ -0,0 +1,11 @@ +package catalyst +package expressions + +import catalyst.types.BooleanType + +case class Like(left: Expression, right: Expression) extends BinaryExpression { + def dataType = BooleanType + def nullable = left.nullable // Right cannot be null. + def symbol = "LIKE" +} + From d55bb527167ec687c4d3668f54dfdbae31af6369 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 17:16:18 -0800 Subject: [PATCH 314/778] add local warehouse/metastore to gitignore. --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index ebb30f804756a..698e53f701343 100644 --- a/.gitignore +++ b/.gitignore @@ -2,6 +2,8 @@ target/ .idea/ metastore_db/ +metastore/ +warehouse/ TempStatsStore/ hs_err_*.log derby.log From 910e33e2248fb52a263e23ed4b1519c770380554 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 17:16:42 -0800 Subject: [PATCH 315/778] basic support for building an assembly jar. --- build.sbt | 15 +++++++++++++++ project/plugins.sbt | 2 ++ 2 files changed, 17 insertions(+) diff --git a/build.sbt b/build.sbt index 0da02611c54d1..20183c04d0e31 100644 --- a/build.sbt +++ b/build.sbt @@ -1,3 +1,5 @@ +import AssemblyKeys._ // put this at the top of the file + name := "catalyst" organization := "com.databricks" @@ -54,3 +56,16 @@ git.remoteRepo := "git@github.com:marmbrus/catalyst.git" site.settings site.includeScaladoc() + +assemblySettings + +test in assembly := {} + +mergeStrategy in assembly := { + case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard + case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard + case "log4j.properties" => MergeStrategy.discard + case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines + case "reference.conf" => MergeStrategy.concat + case _ => MergeStrategy.first +} \ No newline at end of file diff --git a/project/plugins.sbt b/project/plugins.sbt index 0b57a5bf6cf1d..4a0b5f82a729d 100755 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -30,3 +30,5 @@ resolvers += "jgit-repo" at "http://download.eclipse.org/jgit/maven" addSbtPlugin("com.typesafe.sbt" % "sbt-ghpages" % "0.5.2") addSbtPlugin("com.typesafe.sbt" % "sbt-site" % "0.7.1") + +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2") \ No newline at end of file From 1d50af6a884d6a4623127304e5aeab00692c8385 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 17:17:23 -0800 Subject: [PATCH 316/778] more datatypes, fix nonserializable instance variables in udfs --- src/main/scala/shark2/FunctionRegistry.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/src/main/scala/shark2/FunctionRegistry.scala b/src/main/scala/shark2/FunctionRegistry.scala index f0f73e5985f9f..cc35674a29d4e 100644 --- a/src/main/scala/shark2/FunctionRegistry.scala +++ b/src/main/scala/shark2/FunctionRegistry.scala @@ -48,12 +48,14 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry { case c: Class[_] if c == classOf[org.apache.hadoop.io.LongWritable] => LongType case c: Class[_] if c == classOf[java.lang.String] => StringType case c: Class[_] if c == java.lang.Short.TYPE => ShortType + case c: Class[_] if c == java.lang.Integer.TYPE => ShortType case c: Class[_] if c == java.lang.Long.TYPE => LongType case c: Class[_] if c == java.lang.Double.TYPE => DoubleType case c: Class[_] if c == java.lang.Byte.TYPE => ByteType case c: Class[_] if c == java.lang.Float.TYPE => FloatType case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType case c: Class[_] if c == classOf[java.lang.Short] => ShortType + case c: Class[_] if c == classOf[java.lang.Integer] => ShortType case c: Class[_] if c == classOf[java.lang.Long] => LongType case c: Class[_] if c == classOf[java.lang.Double] => DoubleType case c: Class[_] if c == classOf[java.lang.Byte] => ByteType @@ -86,11 +88,13 @@ abstract class HiveUdf extends Expression with ImplementedUdf with Logging { case d: org.apache.hadoop.hive.serde2.io.DoubleWritable => d.get case b: BooleanWritable => b.get() case list: java.util.List[_] => list.map(unwrap) - case s: java.lang.Short => s - case s: java.lang.Long => s - case s: java.lang.Integer => s - case s: java.lang.Double => s - case s: java.lang.Byte => s + case p: java.lang.Short => p + case p: java.lang.Long => p + case p: java.lang.Float => p + case p: java.lang.Integer => p + case p: java.lang.Double => p + case p: java.lang.Byte => p + case p: java.lang.Boolean => p case str: String => str } } @@ -101,7 +105,9 @@ case class HiveSimpleUdf( import HiveFunctionRegistry._ type UDFType = UDF + @transient lazy val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) + @transient lazy val dataType = javaClassToDataType(method.getReturnType) lazy val wrappers = method.getParameterTypes.map { argClass => From 9b2642bb2dc1f4f0a35316588324a76717c66551 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 17:17:39 -0800 Subject: [PATCH 317/778] make the blacklist support regexes --- src/test/scala/shark2/HiveQueryFileTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/test/scala/shark2/HiveQueryFileTest.scala b/src/test/scala/shark2/HiveQueryFileTest.scala index 668a17be469bf..5ebb5e3c5aed9 100644 --- a/src/test/scala/shark2/HiveQueryFileTest.scala +++ b/src/test/scala/shark2/HiveQueryFileTest.scala @@ -25,14 +25,14 @@ abstract class HiveQueryFileTest extends HiveComaparisionTest { val runAll = !(System.getProperty("shark.hive.alltests") == null) - // Allow the whitelist to be overriden by a system property + // Allow the whiteList to be overridden by a system property val realWhiteList = Option(System.getProperty("shark.hive.whitelist")).map(_.split(",").toSeq).getOrElse(whiteList) // Go through all the test cases and add them to scala test. testCases.foreach { case (testCaseName, testCaseFile) => - if(blackList contains testCaseName) { - // Do nothing + if(blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { + logger.warn(s"Blacklisted test skipped $testCaseName") } else if(realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) From cfd6bbc29e3f792cc68369d5b0d2e0e86fae754d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 17:18:03 -0800 Subject: [PATCH 318/778] Quick skipping of tests that we can't even parse. --- src/test/scala/shark2/HiveComparisionTest.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/test/scala/shark2/HiveComparisionTest.scala b/src/test/scala/shark2/HiveComparisionTest.scala index fd223e0230bc4..1cc9b0cae1bb1 100644 --- a/src/test/scala/shark2/HiveComparisionTest.scala +++ b/src/test/scala/shark2/HiveComparisionTest.scala @@ -97,12 +97,14 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with logger.warn(s"Using answer cache for test: $testCaseName") hiveCachedResults } else { - val computedResults = queryList.zip(hiveCacheFiles).zipWithIndex.map { - case ((queryString, cachedAnswerFile), i)=> + val sharkQueries = queryList.map(new TestShark.SharkSqlQuery(_)) + // Make sure we can at least parse everything before doing hive execution. + sharkQueries.foreach(_.parsed) + val computedResults = (queryList.zipWithIndex, sharkQueries,hiveCacheFiles).zipped.map { + case ((queryString, i), sharkQuery, cachedAnswerFile)=> logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") info(s"HIVE: $queryString") // Analyze the query with catalyst to ensure test tables are loaded. - val sharkQuery = (new TestShark.SharkSqlQuery(queryString)) val answer = sharkQuery.analyzed match { case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. case _ => TestShark.runSqlHive(queryString) From 1724c1662cfa73ce14993b14748706fab59d340f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 17:18:24 -0800 Subject: [PATCH 319/778] clear lines that contain last updated times. --- src/test/scala/shark2/HiveComparisionTest.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/test/scala/shark2/HiveComparisionTest.scala b/src/test/scala/shark2/HiveComparisionTest.scala index 1cc9b0cae1bb1..273a7207752e2 100644 --- a/src/test/scala/shark2/HiveComparisionTest.scala +++ b/src/test/scala/shark2/HiveComparisionTest.scala @@ -45,12 +45,15 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with // If the query results aren't sorted, then sort them to ensure deterministic answers. if(!isOrdered) answer.sorted else answer } - orderedAnswer.map(cleanPaths) + orderedAnswer.map(cleanPaths).map(clearTimes) } protected def nonDeterministicLine(line: String) = Seq("CreateTime","transient_lastDdlTime", "grantTime").map(line contains _).reduceLeft(_||_) + protected def clearTimes(line: String) = + line.replaceAll("\"lastUpdateTime\":\\d+", "") + /** * Removes non-deterministic paths from [[str]] so cached answers will still pass. */ From 47c98d60630818bf443c51a9746deab561dab53f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 17:18:44 -0800 Subject: [PATCH 320/778] add query tests for like and hash. --- src/test/scala/shark2/HiveQueryTests.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/test/scala/shark2/HiveQueryTests.scala b/src/test/scala/shark2/HiveQueryTests.scala index 59a4ba4d7b4f8..c2d68588d31d8 100644 --- a/src/test/scala/shark2/HiveQueryTests.scala +++ b/src/test/scala/shark2/HiveQueryTests.scala @@ -44,6 +44,9 @@ class HiveQueryTests extends HiveComaparisionTest { createQueryTest("partitioned table scan", "SELECT ds, hr, key, value FROM srcpart") + createQueryTest("hash", + "SELECT hash('test') FROM src LIMIT 1") + createQueryTest("create table as", """ |CREATE TABLE createdtable AS SELECT * FROM src; @@ -52,4 +55,7 @@ class HiveQueryTests extends HiveComaparisionTest { createQueryTest("transform", "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src") + + createQueryTest("LIKE", + "SELECT * FROM src WHERE value LIKE '%1%'") } \ No newline at end of file From 83833e8c8821f47eefef4571c0cd53687f41c46d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 17:19:01 -0800 Subject: [PATCH 321/778] more tests passing! --- src/test/scala/shark2/HiveCompatability.scala | 61 +++++++++++++++++-- 1 file changed, 56 insertions(+), 5 deletions(-) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index 5d296da3266a2..0a9d3c2a823b1 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -19,7 +19,6 @@ class HiveCompatability extends HiveQueryFileTest { "query_properties", "sample10", "updateAccessTime", - //"load_dyn_part8", "index_compact_binary_search", // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. @@ -29,14 +28,16 @@ class HiveCompatability extends HiveQueryFileTest { // The skewjoin test seems to never complete on hive... "skewjoin", - // This test fails and and exits the JVM. + // These tests fail and and exit the JVM. "auto_join18_multi_distinct", "join18_multi_distinct", // Uses a serde that isn't on the classpath... breaks other tests. - "bucketizedhiveinputformat"//, - // "udaf_covar_pop" + "bucketizedhiveinputformat", + // Avro tests seem to change the output format permenently thus breaking the answer cache, until + // we figure out what this is the case let just ignore all of them + ".*avro.*" ) /** @@ -45,6 +46,7 @@ class HiveCompatability extends HiveQueryFileTest { */ override def whiteList = Seq( "add_part_exist", + "alter2", "alter4", "alter_partition_format_loc", "alter_table_serde", @@ -54,12 +56,14 @@ class HiveCompatability extends HiveQueryFileTest { "auto_join24", "auto_join26", "auto_join28", - "avro_change_schema", + "auto_join_nulls", "binarysortable_1", "bucketmapjoin6", "combine1", "count", + "create_default_prop", "ct_case_insensitive", + "database_location", "database_properties", "default_partition_name", "delimiter", @@ -73,11 +77,20 @@ class HiveCompatability extends HiveQueryFileTest { "drop_partitions_filter3", "drop_table", "drop_view", + "groupby1", + "groupby1_map", + "groupby1_map_nomap", + "groupby1_map_skew", + "groupby1_noskew", "groupby4_map", "groupby4_map_skew", + "groupby5", "groupby5_map", "groupby5_map_skew", + "groupby5_noskew", "index_auth", + "index_auto_mult_tables", + "index_auto_mult_tables_compact", "index_auto_self_join", "index_auto_update", "index_stale", @@ -94,7 +107,9 @@ class HiveCompatability extends HiveQueryFileTest { "input4_limit", "input7", "input9", + "input_part1", "inputddl4", + "inputddl7", "inputddl8", "insert1", "join0", @@ -103,6 +118,8 @@ class HiveCompatability extends HiveQueryFileTest { "join11", "join12", "join13", + "join14", + "join14_hadoop20", "join15", "join16", "join17", @@ -114,16 +131,21 @@ class HiveCompatability extends HiveQueryFileTest { "join23", "join24", "join25", + "join26", "join27", + "join28", "join29", "join3", "join30", "join31", + "join32", + "join33", "join34", "join35", "join36", "join37", "join40", + "join9", "join_casesensitive", "join_empty", "join_nulls", @@ -135,7 +157,12 @@ class HiveCompatability extends HiveQueryFileTest { "literal_double", "literal_ints", "literal_string", + "louter_join_ppr", + "mapjoin1", + "mapjoin_mapjoin", + "mapjoin_subquery", "mergejoins", + "misc_json", "no_hooks", "noalias_subq1", "nomore_ambiguous_table_col", @@ -144,15 +171,18 @@ class HiveCompatability extends HiveQueryFileTest { "nullgroup", "nullgroup2", "nullgroup3", + "nullgroup5", "nullinput", "nullinput2", "nullscript", + "outer_join_ppr", "part_inherit_tbl_props", "part_inherit_tbl_props_empty", "part_inherit_tbl_props_with_star", "ppd1", "ppd_gby_join", "ppd_join", + "ppd_join2", "ppd_join3", "ppd_outer_join1", "ppd_outer_join2", @@ -169,6 +199,7 @@ class HiveCompatability extends HiveQueryFileTest { "query_with_semi", "quote2", "rename_column", + "router_join_ppr", "select_as_omitted", "set_variable_sub", "show_describe_func_quotes", @@ -179,10 +210,15 @@ class HiveCompatability extends HiveQueryFileTest { "udf2", "udf9", "udf_10_trims", + "udf_abs", + "udf_acos", "udf_add", "udf_ascii", + "udf_asin", + "udf_atan", "udf_avg", "udf_bigint", + "udf_bin", "udf_bitwise_and", "udf_bitwise_not", "udf_bitwise_or", @@ -190,6 +226,9 @@ class HiveCompatability extends HiveQueryFileTest { "udf_boolean", "udf_ceil", "udf_ceiling", + "udf_conv", + "udf_cos", + "udf_count", "udf_date_add", "udf_date_sub", "udf_datediff", @@ -197,9 +236,13 @@ class HiveCompatability extends HiveQueryFileTest { "udf_dayofmonth", "udf_double", "udf_exp", + "udf_field", "udf_float", "udf_floor", "udf_from_unixtime", + "udf_greaterthan", + "udf_greaterthanorequal", + "udf_hex", "udf_hour", "udf_index", "udf_int", @@ -207,6 +250,8 @@ class HiveCompatability extends HiveQueryFileTest { "udf_isnull", "udf_lcase", "udf_length", + "udf_lessthan", + "udf_lessthanorequal", "udf_ln", "udf_log", "udf_log10", @@ -229,9 +274,12 @@ class HiveCompatability extends HiveQueryFileTest { "udf_regexp_replace", "udf_repeat", "udf_rlike", + "udf_round", "udf_rpad", "udf_rtrim", "udf_second", + "udf_sign", + "udf_sin", "udf_smallint", "udf_space", "udf_sqrt", @@ -243,6 +291,7 @@ class HiveCompatability extends HiveQueryFileTest { "udf_substring", "udf_subtract", "udf_sum", + "udf_tan", "udf_tinyint", "udf_to_date", "udf_trim", @@ -254,6 +303,8 @@ class HiveCompatability extends HiveQueryFileTest { "udf_var_samp", "udf_variance", "udf_weekofyear", + "udf_when", + "udf_xpath", "udf_xpath_boolean", "udf_xpath_double", "udf_xpath_float", From fb5ddfddab715a73dfac6c36d129c9125c1af4bd Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Dec 2013 17:36:33 -0800 Subject: [PATCH 322/778] move ViewExamples to examples/ --- src/main/scala/{ => examples}/ViewsExample.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename src/main/scala/{ => examples}/ViewsExample.scala (100%) diff --git a/src/main/scala/ViewsExample.scala b/src/main/scala/examples/ViewsExample.scala similarity index 100% rename from src/main/scala/ViewsExample.scala rename to src/main/scala/examples/ViewsExample.scala From dac278643e6af25953067910c6d7450e4b11e448 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 11:06:03 -0800 Subject: [PATCH 323/778] correctly handle null values when going from string to numeric types. --- src/main/scala/expressions/Evaluate.scala | 12 ++++++++++-- src/main/scala/expressions/nullFunctions.scala | 0 2 files changed, 10 insertions(+), 2 deletions(-) create mode 100644 src/main/scala/expressions/nullFunctions.scala diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index 1a2bf1814b84e..2546c389cb8ab 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -181,8 +181,16 @@ object Evaluate extends Logging { } // String => Numeric Types - case Cast(e, IntegerType) if e.dataType == StringType => eval(e).asInstanceOf[String].toInt - case Cast(e, DoubleType) if e.dataType == StringType => eval(e).asInstanceOf[String].toDouble + case Cast(e, IntegerType) if e.dataType == StringType => + eval(e) match { + case null => null + case s: String => s.toInt + } + case Cast(e, DoubleType) if e.dataType == StringType => + eval(e) match { + case null => null + case s: String => s.toDouble + } // Boolean conversions case Cast(e, ByteType) if e.dataType == BooleanType => eval(e) match { diff --git a/src/main/scala/expressions/nullFunctions.scala b/src/main/scala/expressions/nullFunctions.scala new file mode 100644 index 0000000000000..e69de29bb2d1d From 2326be1a8f91f6c1d77d38f199b060da62fbf9e4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:15:34 -0800 Subject: [PATCH 324/778] make getClauses case insensitive. --- src/main/scala/frontend/Hive.scala | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 9d771c429c793..5eb779003cc19 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -264,19 +264,16 @@ object HiveQl { } protected def getClauses(clauseNames: Seq[String], nodeList: Seq[ASTNode]): Seq[Option[Node]] = { - val unhandledClauses = nodeList.filterNot(clauseNames contains _.getText) - require(unhandledClauses.isEmpty, s"Unhandled parse clauses: $unhandledClauses") - var remainingNodes = nodeList - val clauses = clauseNames.map { clauseName => - val (matches, nonMatches) = remainingNodes.partition(_.getText == clauseName) + val (matches, nonMatches) = remainingNodes.partition(_.getText.toUpperCase == clauseName) remainingNodes = nonMatches ++ (if(matches.nonEmpty) matches.tail else Nil) matches.headOption } - assert(remainingNodes.isEmpty, s"Unhandled clauses: ${remainingNodes.map(dumpTree(_)).mkString("\n")}") - clauses + assert(remainingNodes.isEmpty, + s"Unhandled clauses: ${remainingNodes.map(dumpTree(_)).mkString("\n")}") + clauses } def getClause(clauseName: String, nodeList: Seq[Node]) = From c680e0d2ef56e2d8eddde747fd8416de4d70345d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:15:55 -0800 Subject: [PATCH 325/778] Failed string => number conversion should return null. --- src/main/scala/expressions/Evaluate.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index 2546c389cb8ab..1348aa13f10f4 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -126,6 +126,9 @@ object Evaluate extends Logging { } } + @inline def castOrNull[A](f: => A) = + try f catch { case _: java.lang.NumberFormatException => null } + val result = e match { case Literal(v, _) => v @@ -184,12 +187,12 @@ object Evaluate extends Logging { case Cast(e, IntegerType) if e.dataType == StringType => eval(e) match { case null => null - case s: String => s.toInt + case s: String => castOrNull(s.toInt) } case Cast(e, DoubleType) if e.dataType == StringType => eval(e) match { case null => null - case s: String => s.toDouble + case s: String => castOrNull(s.toDouble) } // Boolean conversions case Cast(e, ByteType) if e.dataType == BooleanType => From 67b88cf70104e146fd9c9a9e2212a42d195a2c42 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:16:10 -0800 Subject: [PATCH 326/778] more verbose debugging of evaluation return types --- src/main/scala/expressions/Evaluate.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index 1348aa13f10f4..e343158c1f382 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -252,7 +252,7 @@ object Evaluate extends Logging { case other => throw new OptimizationException(other, "evaluation not implemented") } - logger.debug(s"Evaluated $e => $result") + logger.debug(s"Evaluated $e => $result of type ${if(result == null) "null" else result.getClass.getName}, expected: ${e.dataType}") result } } \ No newline at end of file From 463269589380f2a9cd86016c000ec14de29e99c6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:16:27 -0800 Subject: [PATCH 327/778] support for megastore bigint --- src/main/scala/shark2/MetastoreCatalog.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/shark2/MetastoreCatalog.scala b/src/main/scala/shark2/MetastoreCatalog.scala index 0ca367c465304..48e3bf4663ed6 100644 --- a/src/main/scala/shark2/MetastoreCatalog.scala +++ b/src/main/scala/shark2/MetastoreCatalog.scala @@ -79,6 +79,7 @@ object HiveMetatoreTypes { case "string" => StringType case "int" => IntegerType case "double" => DoubleType + case "bigint" => LongType } } From fd4b096067ecea63df305bc465e4555688e04806 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:16:44 -0800 Subject: [PATCH 328/778] fix casing of null strings as well. --- src/main/scala/shark2/SharkInstance.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/shark2/SharkInstance.scala b/src/main/scala/shark2/SharkInstance.scala index 80cf715afc945..78faba5170c11 100644 --- a/src/main/scala/shark2/SharkInstance.scala +++ b/src/main/scala/shark2/SharkInstance.scala @@ -114,6 +114,7 @@ abstract class SharkInstance extends Logging { def toHiveString(a: Any): String = a match { case seq: Seq[_] => seq.map(toHiveString).map(s => "\"" + s + "\"").mkString("[", ",", "]") + case "null" => "NULL" case null => "NULL" case other => other.toString } From cda43ab89edf2c3fbde076fb56756f07735e4684 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:17:17 -0800 Subject: [PATCH 329/778] don't throw an exception when one of the join tables is empty. --- src/main/scala/shark2/joins.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/main/scala/shark2/joins.scala b/src/main/scala/shark2/joins.scala index 9705fdc788b70..e5084b6c71284 100644 --- a/src/main/scala/shark2/joins.scala +++ b/src/main/scala/shark2/joins.scala @@ -99,11 +99,17 @@ case class BroadcastNestedLoopJoin(streamed: SharkPlan, broadcast: SharkPlan, jo (outputRows, includedBroadcastTuples) } - val allIncludedBroadcastTupes = streamedPlusMatches.map(_._2).reduce(_ ++ _) + val includedBroadcastTuples = streamedPlusMatches.map(_._2) + val allIncludedBroadcastTuples = + if(includedBroadcastTuples.count == 0) + new scala.collection.mutable.BitSet(broadcastedRelation.value.size) + else + streamedPlusMatches.map(_._2).reduce(_ ++ _) + val rightOuterMatches: Seq[Row] = if(joinType == RightOuter || joinType == FullOuter) broadcastedRelation.value.zipWithIndex.filter { - case (row, i) => !allIncludedBroadcastTupes.contains(i) + case (row, i) => !allIncludedBroadcastTuples.contains(i) }.map { case (row, _) => buildRow(Vector.fill(left.output.size)(null) ++ row) } From 6355d0ebfc318c9154dcad5f6745a1f46382edec Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:17:37 -0800 Subject: [PATCH 330/778] match actual return type of count with expected --- src/main/scala/shark2/aggregates.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark2/aggregates.scala b/src/main/scala/shark2/aggregates.scala index 0baffceb5b140..c7a234672fa45 100644 --- a/src/main/scala/shark2/aggregates.scala +++ b/src/main/scala/shark2/aggregates.scala @@ -31,7 +31,7 @@ case class Aggregate(groupingExpressions: Seq[Expression], case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - var count: Long = _ + var count: Int = _ def apply(input: Seq[Row]): Unit = { val evaluatedExpr = expr.map(Evaluate(_, input)) From 06b2abaffc830a00812c0312c4d7260e44e32ba2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:18:00 -0800 Subject: [PATCH 331/778] don't be case sensitive when fixing load paths --- src/main/scala/shark2/TestShark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index 36114fa78efae..0949e0b5cf54a 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -79,7 +79,7 @@ object TestShark extends SharkInstance { * assume the system is set up. */ private def rewritePaths(cmd: String): String = - if(cmd startsWith "LOAD") + if(cmd.toUpperCase startsWith "LOAD") cmd.replaceAll("\\.\\.", hiveDevHome.getCanonicalPath) else cmd From 93b64b01ff1c24751939635badddb867ece35147 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:18:28 -0800 Subject: [PATCH 332/778] load test tables that are args to "DESCRIBE" --- src/main/scala/shark2/TestShark.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index 0949e0b5cf54a..83fccdd686d75 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -84,13 +84,21 @@ object TestShark extends SharkInstance { else cmd + val describedTable = "DESCRIBE (\\w+)".r /** * Override SharkQuery with special debug workflow. */ abstract class SharkQuery extends super.SharkQuery { override lazy val analyzed = { + val describedTables = parsed match { + case NativeCommand(describedTable(tbl)) => tbl :: Nil + case _ => Nil + } + // Make sure any test tables referenced are loaded. - val referencedTables = parsed collect { case UnresolvedRelation(name, _) => name.split("\\.").last } + val referencedTables = + describedTables ++ + parsed.collect { case UnresolvedRelation(name, _) => name.split("\\.").last } val referencedTestTables = referencedTables.filter(testTables.contains) logger.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) From d3c9305a95f80bb48439b16f5e3646e882963662 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:18:46 -0800 Subject: [PATCH 333/778] fix > 100 char line --- src/main/scala/shark2/TestShark.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index 83fccdd686d75..f6f3febf21840 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -168,7 +168,8 @@ object TestShark extends SharkInstance { def loadTestTable(name: String) { if(!(loadedTables contains name)) { logger.info(s"Loading test table $name") - val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) + val createCmds = + testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) loadedTables += name } From 7349e7b57ecc6e318f6eab7bc916cb53cb3c30b5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:19:01 -0800 Subject: [PATCH 334/778] initial support for test thrift table --- src/main/scala/shark2/TestShark.scala | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/shark2/TestShark.scala index f6f3febf21840..d3d08f1345a5a 100644 --- a/src/main/scala/shark2/TestShark.scala +++ b/src/main/scala/shark2/TestShark.scala @@ -14,6 +14,7 @@ import util._ import collection.JavaConversions._ import org.apache.hadoop.hive.ql.exec.FunctionRegistry +import org.apache.hadoop.hive.metastore.api.{SerDeInfo, StorageDescriptor} /** @@ -159,6 +160,31 @@ object TestShark extends SharkInstance { runSqlHive(s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr')") } } + }), + TestTable("src_thrift", () => { + import org.apache.thrift.protocol.TBinaryProtocol + import org.apache.hadoop.hive.serde2.thrift.test.Complex + import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer + import org.apache.hadoop.mapred.SequenceFileInputFormat + import org.apache.hadoop.mapred.SequenceFileOutputFormat + + val srcThrift = new org.apache.hadoop.hive.metastore.api.Table() + srcThrift.setTableName("src_thrift") + srcThrift.setDbName("default") + srcThrift.setSd(new StorageDescriptor) + srcThrift.getSd.setCols(Nil) + srcThrift.getSd.setInputFormat(classOf[SequenceFileInputFormat[_,_]].getName) + srcThrift.getSd.setOutputFormat(classOf[SequenceFileOutputFormat[_,_]].getName) + srcThrift.getSd.setSerdeInfo(new SerDeInfo) + srcThrift.getSd.getSerdeInfo.setSerializationLib(classOf[ThriftDeserializer].getName) + srcThrift.getSd.getSerdeInfo.setParameters( + Map( + "serialization.class" -> classOf[Complex].getName, + "serialization.format" -> classOf[TBinaryProtocol].getName)) + + catalog.client.createTable(srcThrift) + + runSqlHive(s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/complex.seq' INTO TABLE src_thrift") }) ) From 70f253f688eb3c866b2cd0982704e7f0bada8a53 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Dec 2013 15:57:04 -0800 Subject: [PATCH 335/778] more tests passing! --- src/test/scala/shark2/HiveCompatability.scala | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index 0a9d3c2a823b1..81f6dae6cfb9b 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -37,7 +37,13 @@ class HiveCompatability extends HiveQueryFileTest { // Avro tests seem to change the output format permenently thus breaking the answer cache, until // we figure out what this is the case let just ignore all of them - ".*avro.*" + ".*avro.*", + + // Unique joins are weird and will require a lot of hacks (see comments in hive parser) + "uniquejoin", + + // Hive seems to get the wrong answer on some outer joins. MySQL agrees with catalyst. + "auto_join29" ) /** @@ -58,7 +64,10 @@ class HiveCompatability extends HiveQueryFileTest { "auto_join28", "auto_join_nulls", "binarysortable_1", + "bucket1", "bucketmapjoin6", + "bucketmapjoin_negative", + "bucketmapjoin_negative2", "combine1", "count", "create_default_prop", @@ -70,6 +79,7 @@ class HiveCompatability extends HiveQueryFileTest { "describe_database_json", "describe_table_json", "diff_part_input_formats", + "disable_file_format_check", "drop_function", "drop_index", "drop_partitions_filter", @@ -88,6 +98,7 @@ class HiveCompatability extends HiveQueryFileTest { "groupby5_map", "groupby5_map_skew", "groupby5_noskew", + "implicit_cast1", "index_auth", "index_auto_mult_tables", "index_auto_mult_tables_compact", @@ -100,11 +111,13 @@ class HiveCompatability extends HiveQueryFileTest { "input11", "input11_limit", "input22", + "input23", "input24", "input25", "input41", "input4_cb_delim", "input4_limit", + "input6", "input7", "input9", "input_part1", @@ -144,10 +157,16 @@ class HiveCompatability extends HiveQueryFileTest { "join35", "join36", "join37", + "join39", + "join4", "join40", + "join5", + "join6", + "join8", "join9", "join_casesensitive", "join_empty", + "join_hive_626", "join_nulls", "join_reorder2", "join_reorder3", @@ -161,6 +180,7 @@ class HiveCompatability extends HiveQueryFileTest { "mapjoin1", "mapjoin_mapjoin", "mapjoin_subquery", + "mapjoin_subquery2", "mergejoins", "misc_json", "no_hooks", @@ -179,7 +199,9 @@ class HiveCompatability extends HiveQueryFileTest { "part_inherit_tbl_props", "part_inherit_tbl_props_empty", "part_inherit_tbl_props_with_star", + "partitions_json", "ppd1", + "ppd_gby", "ppd_gby_join", "ppd_join", "ppd_join2", @@ -208,6 +230,8 @@ class HiveCompatability extends HiveQueryFileTest { "subq2", "tablename_with_select", "udf2", + "udf5", + "udf7", "udf9", "udf_10_trims", "udf_abs", From 4529594b08a11c196d7a5a68fc11a0cd7e7bf187 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 00:54:46 -0800 Subject: [PATCH 336/778] draft of coalesce --- src/main/scala/expressions/nullFunctions.scala | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/main/scala/expressions/nullFunctions.scala b/src/main/scala/expressions/nullFunctions.scala index e69de29bb2d1d..593ef1e9c6a71 100644 --- a/src/main/scala/expressions/nullFunctions.scala +++ b/src/main/scala/expressions/nullFunctions.scala @@ -0,0 +1,18 @@ +package catalyst +package expressions + +import catalyst.analysis.UnresolvedException + +case class Coalesce(children: Seq[Expression]) extends Expression { + def nullable = children.map(_.nullable).reduce(_&&_) + def references = children.flatMap(_.references).toSet + + // Only resolved if all the children are of the same type. + override lazy val resolved = childrenResolved && (children.map(_.dataType).distinct.size == 1) + + def dataType = + if(resolved) + children.head.dataType + else + throw new UnresolvedException(this, "Coalesce cannot have children of different types.") +} \ No newline at end of file From a6ab6c7450b66899c6ca0a856f8c9e48e4cda537 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 00:55:08 -0800 Subject: [PATCH 337/778] add != --- src/main/scala/frontend/Hive.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 5eb779003cc19..3194bb9c5d19d 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -564,6 +564,7 @@ object HiveQl { /* Comparisons */ case Token("=", left :: right:: Nil) => Equals(nodeToExpr(left), nodeToExpr(right)) + case Token("!=", left :: right:: Nil) => Not(Equals(nodeToExpr(left), nodeToExpr(right))) case Token("<>", left :: right:: Nil) => Not(Equals(nodeToExpr(left), nodeToExpr(right))) case Token(">", left :: right:: Nil) => GreaterThan(nodeToExpr(left), nodeToExpr(right)) case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) From 8492548d4432bab572d3ce21dded67551cb16a43 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 00:55:40 -0800 Subject: [PATCH 338/778] beginning of UNIQUEJOIN parsing. --- src/main/scala/frontend/Hive.scala | 43 ++++++++++++++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 3194bb9c5d19d..630548b563424 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -434,6 +434,49 @@ object HiveQl { val tableName = tableNameParts.map { case Token(part, Nil) => part }.mkString(".") UnresolvedRelation(tableName, None) + case Token("TOK_UNIQUEJOIN", joinArgs) => + val tableOrdinals = + joinArgs.zipWithIndex.filter { + case (arg, i) => arg.getText == "TOK_TABREF" + }.map(_._2) + + val isPreserved = tableOrdinals.map(i => (i - 1 < 0) || joinArgs(i - 1).getText == "PRESERVE") + val tables = tableOrdinals.map(i => nodeToRelation(joinArgs(i))) + val joinExpressions = tableOrdinals.map(i => joinArgs(i + 1).getChildren.map(nodeToExpr)) + + val joinConditions = joinExpressions.sliding(2).map { + case Seq(c1, c2) => + val predicates = (c1, c2).zipped.map { case (e1, e2) => Equals(e1, e2): Expression } + predicates.reduceLeft(And) + }.toBuffer + + val joinType = isPreserved.sliding(2).map { + case Seq(true, true) => FullOuter + case Seq(true, false) => LeftOuter + case Seq(false, true) => RightOuter + case Seq(false, false) => Inner + }.toBuffer + + val joinedTables = tables.reduceLeft(Join(_,_, Inner, None)) + + // Must be transform down. + val joinedResult = joinedTables transform { + case j: Join => + j.copy( + condition = Some(joinConditions.remove(joinConditions.length - 1)), + joinType = joinType.remove(joinType.length - 1)) + } + + // Unique join is not really the same as an outer join so we must group together results where + // the joinExpressions are the same, taking the First of each value is only okay because the + // user of a unique join is implicitly promising that there is only one result. + // TODO: This doesn't actually work since [[Star]] is not a valid aggregate expression. + // instead we should figure out how important supporting this feature is and whether it is + // worth the number of hacks that will be required to implement it. Namely, we need to add + // some sort of mapped star expansion that would expand all child output row to be similarly + // named output expressions where some aggregate expression has been applied (i.e. First). + ??? // Aggregate(joinExpressions.map(Coalesce(_)), Star(None) :: Nil, joinedResult) + /* Table with Alias */ case Token("TOK_TABREF", Token("TOK_TABNAME", From a2874810a6e7d478183f22c6394c184349370efd Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 00:56:34 -0800 Subject: [PATCH 339/778] spelling --- src/test/scala/shark2/HiveCompatability.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index 81f6dae6cfb9b..aa3e655866fae 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -35,7 +35,7 @@ class HiveCompatability extends HiveQueryFileTest { // Uses a serde that isn't on the classpath... breaks other tests. "bucketizedhiveinputformat", - // Avro tests seem to change the output format permenently thus breaking the answer cache, until + // Avro tests seem to change the output format permanently thus breaking the answer cache, until // we figure out what this is the case let just ignore all of them ".*avro.*", From 1e283112e65bb411dbf894fb90fffc8451aab2d5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 00:57:11 -0800 Subject: [PATCH 340/778] make tests execute in alpha order again --- src/test/scala/shark2/HiveCompatability.scala | 2 +- src/test/scala/shark2/HiveQueryFileTest.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index aa3e655866fae..f65d830ef37cf 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -357,5 +357,5 @@ class HiveCompatability extends HiveQueryFileTest { // TODO: bundle in jar files... get from classpath lazy val hiveQueryDir = new File(TestShark.hiveDevHome, "ql/src/test/queries/clientpositive") - def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f).toMap + def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) } diff --git a/src/test/scala/shark2/HiveQueryFileTest.scala b/src/test/scala/shark2/HiveQueryFileTest.scala index 5ebb5e3c5aed9..2cc4832a46391 100644 --- a/src/test/scala/shark2/HiveQueryFileTest.scala +++ b/src/test/scala/shark2/HiveQueryFileTest.scala @@ -21,7 +21,7 @@ abstract class HiveQueryFileTest extends HiveComaparisionTest { */ def whiteList: Seq[String] = ".*" :: Nil - def testCases: Map[String, File] + def testCases: Seq[(String, File)] val runAll = !(System.getProperty("shark.hive.alltests") == null) From c944a954af31b7509aec5097e072f98f20b090ff Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 00:57:35 -0800 Subject: [PATCH 341/778] First aggregate expression. --- src/main/scala/expressions/aggregates.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/main/scala/expressions/aggregates.scala b/src/main/scala/expressions/aggregates.scala index 9cb1bf1ad994f..1422d1bd7aec6 100644 --- a/src/main/scala/expressions/aggregates.scala +++ b/src/main/scala/expressions/aggregates.scala @@ -51,4 +51,11 @@ case class Sum(child: Expression) extends AggregateExpression with trees.UnaryNo def nullable = false def dataType = child.dataType override def toString = s"SUM($child)" +} + +case class First(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { + def references = child.references + def nullable = child.nullable + def dataType = child.dataType + override def toString = s"FIRST($child)" } \ No newline at end of file From bcfc8c56ca8c8497eab557736d68098d29e8c19f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 00:58:41 -0800 Subject: [PATCH 342/778] start supporting partition attributes when inserting data. --- src/main/scala/frontend/Hive.scala | 15 ++++++++++++--- src/main/scala/plans/logical/basicOperators.scala | 2 +- src/main/scala/shark2/MetastoreCatalog.scala | 2 +- src/main/scala/shark2/hiveOperators.scala | 9 +++++++-- src/main/scala/shark2/planningStrategies.scala | 4 ++-- 5 files changed, 23 insertions(+), 9 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 630548b563424..4317bdd08c1e4 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -519,11 +519,20 @@ object HiveQl { Token("TOK_DIR", Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) => query + case Token("TOK_DESTINATION", Token("TOK_TAB", - Token("TOK_TABNAME", - Token(tableName, Nil) :: Nil) :: Nil) :: Nil) => - InsertIntoTable(UnresolvedRelation(tableName, None), query) + tableArgs) :: Nil) => + val Some(nameClause) :: partitionClause :: Nil = + getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) + val Token("TOK_TABNAME", Token(tableName, Nil) :: Nil) = nameClause + + val partitionKeys = partitionClause.map(_.getChildren.map { + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => key -> value + }.toMap).getOrElse(Map.empty) + + InsertIntoTable(UnresolvedRelation(tableName, None), partitionKeys, query) + case a: ASTNode => throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index ec4a7a8da343e..5b62eddfc8cdf 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -30,7 +30,7 @@ case class Join( def output = left.output ++ right.output } -case class InsertIntoTable(table: BaseRelation, child: LogicalPlan) extends LogicalPlan { +case class InsertIntoTable(table: BaseRelation, partition: Map[String, String], child: LogicalPlan) extends LogicalPlan { // The table being inserted into is a child for the purposes of transformations. def children = table :: child :: Nil def references = Set.empty diff --git a/src/main/scala/shark2/MetastoreCatalog.scala b/src/main/scala/shark2/MetastoreCatalog.scala index 48e3bf4663ed6..ed87941a1bbbe 100644 --- a/src/main/scala/shark2/MetastoreCatalog.scala +++ b/src/main/scala/shark2/MetastoreCatalog.scala @@ -68,7 +68,7 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { sd.setSerdeInfo(serDeInfo) client.createTable(table) - InsertIntoTable(lookupRelation(tableName, None), child) + InsertIntoTable(lookupRelation(tableName, None), Map.empty, child) } } } diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index 628f33b9fc1af..b9c4e20064421 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -71,7 +71,7 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation def output = attributes } -case class InsertIntoHiveTable(table: MetastoreRelation, child: SharkPlan) +case class InsertIntoHiveTable(table: MetastoreRelation, partition: Map[String, String], child: SharkPlan) (@transient sc: SharkContext) extends UnaryNode { /** * This file sink / record writer code is only the first step towards implementing this operator correctly and is not @@ -105,7 +105,12 @@ case class InsertIntoHiveTable(table: MetastoreRelation, child: SharkPlan) tempDir.delete() tempDir.mkdir() childRdd.map(_.map(a => stringOrNull(a.asInstanceOf[AnyRef])).mkString("\001")).saveAsTextFile(tempDir.getCanonicalPath) - sc.runSql(s"LOAD DATA LOCAL INPATH '${tempDir.getCanonicalPath}/*' INTO TABLE ${table.tableName}") + val partitionSpec = + if(partition.nonEmpty) + s"PARTITION (${partition.map { case (k,v) => s"$k=$v" }.mkString(",")})" + else + "" + sc.runSql(s"LOAD DATA LOCAL INPATH '${tempDir.getCanonicalPath}/*' INTO TABLE ${table.tableName} $partitionSpec") // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which diff --git a/src/main/scala/shark2/planningStrategies.scala b/src/main/scala/shark2/planningStrategies.scala index 2ab5251da9495..c368eb90ef3a2 100644 --- a/src/main/scala/shark2/planningStrategies.scala +++ b/src/main/scala/shark2/planningStrategies.scala @@ -15,8 +15,8 @@ abstract trait PlanningStrategies { object DataSinks extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { - case logical.InsertIntoTable(table: MetastoreRelation, child) => - InsertIntoHiveTable(table, planLater(child))(sc) :: Nil + case logical.InsertIntoTable(table: MetastoreRelation, partition, child) => + InsertIntoHiveTable(table, partition, planLater(child))(sc) :: Nil case _ => Nil } } From 3b854626f92df204071dbd9efa061d70911968b1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 00:58:51 -0800 Subject: [PATCH 343/778] more tests passing --- src/test/scala/shark2/HiveCompatability.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index f65d830ef37cf..ff6276b83baa0 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -87,6 +87,7 @@ class HiveCompatability extends HiveQueryFileTest { "drop_partitions_filter3", "drop_table", "drop_view", + "filter_join_breaktask", "groupby1", "groupby1_map", "groupby1_map_nomap", @@ -114,6 +115,7 @@ class HiveCompatability extends HiveQueryFileTest { "input23", "input24", "input25", + "input28", "input41", "input4_cb_delim", "input4_limit", @@ -181,6 +183,8 @@ class HiveCompatability extends HiveQueryFileTest { "mapjoin_mapjoin", "mapjoin_subquery", "mapjoin_subquery2", + "merge1", + "merge2", "mergejoins", "misc_json", "no_hooks", @@ -199,6 +203,8 @@ class HiveCompatability extends HiveQueryFileTest { "part_inherit_tbl_props", "part_inherit_tbl_props_empty", "part_inherit_tbl_props_with_star", + "partition_schema1", + "partition_vs_table_metadata", "partitions_json", "ppd1", "ppd_gby", @@ -287,6 +293,7 @@ class HiveCompatability extends HiveQueryFileTest { "udf_modulo", "udf_month", "udf_not", + "udf_notequal", "udf_or", "udf_parse_url", "udf_pmod", From c4060e461de23fbcd0d2fce64cc629721080d52f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 01:05:41 -0800 Subject: [PATCH 344/778] cleanup --- src/test/scala/shark2/HiveCompatability.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/shark2/HiveCompatability.scala index ff6276b83baa0..9c044592a9d3f 100644 --- a/src/test/scala/shark2/HiveCompatability.scala +++ b/src/test/scala/shark2/HiveCompatability.scala @@ -9,7 +9,11 @@ import util._ * Runs the test cases that are included in the hive distribution. */ class HiveCompatability extends HiveQueryFileTest { - /** A list of tests deemed out of scope and thus completely disregarded */ + // TODO: bundle in jar files... get from classpath + lazy val hiveQueryDir = new File(TestShark.hiveDevHome, "ql/src/test/queries/clientpositive") + def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) + + /** A list of tests deemed out of scope currently and thus completely disregarded */ override def blackList = Seq( "hook_order", // These tests use hooks that are not on the classpath and thus break all subsequent SQL execution. "hook_context", @@ -36,10 +40,10 @@ class HiveCompatability extends HiveQueryFileTest { "bucketizedhiveinputformat", // Avro tests seem to change the output format permanently thus breaking the answer cache, until - // we figure out what this is the case let just ignore all of them + // we figure out why this is the case let just ignore all of avro related tests. ".*avro.*", - // Unique joins are weird and will require a lot of hacks (see comments in hive parser) + // Unique joins are weird and will require a lot of hacks (see comments in hive parser). "uniquejoin", // Hive seems to get the wrong answer on some outer joins. MySQL agrees with catalyst. @@ -361,8 +365,4 @@ class HiveCompatability extends HiveQueryFileTest { "union9", "union_script" ) - - // TODO: bundle in jar files... get from classpath - lazy val hiveQueryDir = new File(TestShark.hiveDevHome, "ql/src/test/queries/clientpositive") - def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) } From c086a35d09e4e015fa5f7cf6b5bfde8112f7e04d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 01:10:33 -0800 Subject: [PATCH 345/778] docs, spacing --- src/main/scala/analysis/typeCoercion.scala | 3 +++ src/main/scala/shark2/MetastoreCatalog.scala | 2 -- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/scala/analysis/typeCoercion.scala b/src/main/scala/analysis/typeCoercion.scala index f9c2aca5200bf..62563374f7e5f 100644 --- a/src/main/scala/analysis/typeCoercion.scala +++ b/src/main/scala/analysis/typeCoercion.scala @@ -108,6 +108,9 @@ object PromoteStrings extends Rule[LogicalPlan] { } } +/** + * Changes Boolean values to Bytes so that expressions like true < false can be Evaluated. + */ object BooleanComparisons extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { // Skip nodes who's children have not been resolved yet. diff --git a/src/main/scala/shark2/MetastoreCatalog.scala b/src/main/scala/shark2/MetastoreCatalog.scala index ed87941a1bbbe..6dcd2b9d696e9 100644 --- a/src/main/scala/shark2/MetastoreCatalog.scala +++ b/src/main/scala/shark2/MetastoreCatalog.scala @@ -6,7 +6,6 @@ import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table, Stor import org.apache.hadoop.hive.metastore.HiveMetaStoreClient import org.apache.hadoop.hive.ql.plan.TableDesc - import analysis.Catalog import expressions._ import plans.logical._ @@ -15,7 +14,6 @@ import types._ import collection.JavaConversions._ - class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { val client = new HiveMetaStoreClient(hiveConf) From 2ab5a32fe9cf33b5f5070c319534b6530c99ac88 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 11:06:40 -0800 Subject: [PATCH 346/778] stop using uberjar as it has its own set of issues. --- build.sbt | 3 +-- src/main/scala/shark2/hiveOperators.scala | 3 ++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/build.sbt b/build.sbt index 07c73ef01fd59..bbc104c0bc107 100644 --- a/build.sbt +++ b/build.sbt @@ -10,8 +10,7 @@ scalaVersion := "2.10.3" resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" -// Downloaded as giant jar for easy compiling during interview question. -libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.9.0-SNAPSHOT" from "http://www.eecs.berkeley.edu/~marmbrus/tmp/shark-assembly-0.9.0-SNAPSHOT-hadoop1.0.4.jar" +libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.9.0-SNAPSHOT" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index 703f3a446d270..628f33b9fc1af 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -90,7 +90,8 @@ case class InsertIntoHiveTable(table: MetastoreRelation, child: SharkPlan) table.tableDesc, outputClass, desc, - new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory(), "test.out")) + new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory(), "test.out"), + null) override def otherCopyArgs = sc :: Nil From 60b1526d0687bb909c2110b66b03c6a8bc1b437f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 11:06:57 -0800 Subject: [PATCH 347/778] don't call dataType until checking if children are resolved. --- src/main/scala/expressions/arithmetic.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/scala/expressions/arithmetic.scala b/src/main/scala/expressions/arithmetic.scala index af30e4fb90637..23342cb001f6a 100644 --- a/src/main/scala/expressions/arithmetic.scala +++ b/src/main/scala/expressions/arithmetic.scala @@ -13,7 +13,8 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { abstract class BinaryArithmetic extends BinaryExpression { self: Product => - override lazy val resolved = left.dataType == right.dataType + override lazy val resolved = + left.resolved && right.resolved && left.dataType == right.dataType def dataType = { if(!resolved) From 191ce3e9eaa65314129ee90a819ea4eba8e31b3c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 11:07:11 -0800 Subject: [PATCH 348/778] analyze rewrite test query. --- src/test/scala/OptimizerSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/scala/OptimizerSuite.scala b/src/test/scala/OptimizerSuite.scala index 04504451533dc..af21151243b05 100644 --- a/src/test/scala/OptimizerSuite.scala +++ b/src/test/scala/OptimizerSuite.scala @@ -80,6 +80,7 @@ class OptimizerSuite extends FunSuite { testRelation .select('a + 'b as 'e) .where('e === 1) + .analyze /* Your code here */ fail("not implemented") From 5377c04c3797a4a41ff4d497337ef25ddb4c10f0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Dec 2013 11:06:57 -0800 Subject: [PATCH 349/778] don't call dataType until checking if children are resolved. --- src/main/scala/expressions/arithmetic.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/scala/expressions/arithmetic.scala b/src/main/scala/expressions/arithmetic.scala index af30e4fb90637..23342cb001f6a 100644 --- a/src/main/scala/expressions/arithmetic.scala +++ b/src/main/scala/expressions/arithmetic.scala @@ -13,7 +13,8 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { abstract class BinaryArithmetic extends BinaryExpression { self: Product => - override lazy val resolved = left.dataType == right.dataType + override lazy val resolved = + left.resolved && right.resolved && left.dataType == right.dataType def dataType = { if(!resolved) From ab6f67d7824a917ac9b9270e5183b2c97baf6579 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 1 Jan 2014 18:37:08 -0800 Subject: [PATCH 350/778] import the string "null" as actual null. --- src/main/scala/shark2/basicOperators.scala | 10 ++++++++-- src/main/scala/shark2/hiveOperators.scala | 17 ++++++++++++----- 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/src/main/scala/shark2/basicOperators.scala b/src/main/scala/shark2/basicOperators.scala index 768b331782bf6..d61d5e1f2cc61 100644 --- a/src/main/scala/shark2/basicOperators.scala +++ b/src/main/scala/shark2/basicOperators.scala @@ -72,9 +72,15 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { if(left == null && right == null) 0 else if(left == null) - -1 + if(curDirection == Ascending) + -1 + else + 1 else if(right == null) - 1 + if(curDirection == Ascending) + 1 + else + -1 else if(curDataType == IntegerType) if(curDirection == Ascending) left.asInstanceOf[Int] compare right.asInstanceOf[Int] diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index b9c4e20064421..d39d7fc86ee43 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -60,11 +60,18 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation hadoopReader.makeRDDForPartitionedTable(relation.hiveQlPartitions) def execute() = { - inputRdd.map { - case Array(struct: LazyStruct, partitionKeys: Array[String]) => - buildRow(attributeFunctions.map(_(struct, partitionKeys))) - case struct: LazyStruct => - buildRow(attributeFunctions.map(_(struct, Array.empty))) + inputRdd.map { row => + val values = row match { + case Array(struct: LazyStruct, partitionKeys: Array[String]) => + attributeFunctions.map(_(struct, partitionKeys)) + case struct: LazyStruct => + attributeFunctions.map(_(struct, Array.empty)) + } + buildRow(values.map { + case "NULL" => null + case "null" => null + case other => other + }) } } From 3a018b6a059d955586c461278f2e1780c8850b5d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 1 Jan 2014 18:37:45 -0800 Subject: [PATCH 351/778] fix up docs. --- src/main/scala/analysis/unresolved.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala index 2e5b587473b0c..da4d7e390ab78 100644 --- a/src/main/scala/analysis/unresolved.scala +++ b/src/main/scala/analysis/unresolved.scala @@ -1,7 +1,7 @@ package catalyst package analysis -import expressions.{Attribute, Expression} +import expressions._ import plans.logical.BaseRelation import trees.TreeNode @@ -45,9 +45,11 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E } /** - * Represents all of the input attributes to a given relational operator, for example in "SELECT * FROM ...". + * Represents all of the input attributes to a given relational operator, for example in + * "SELECT * FROM ...". * - * @param table an optional table that should be the target of the expansion. If omitted all tables' columns are produced. + * @param table an optional table that should be the target of the expansion. If omitted all + * tables' columns are produced. */ case class Star(table: Option[String]) extends Attribute with trees.LeafNode[Expression] { def name = throw new UnresolvedException(this, "exprId") From a75c0233af385d29291998b28ab3129205938d25 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 1 Jan 2014 18:39:46 -0800 Subject: [PATCH 352/778] implement Coalesce --- src/main/scala/expressions/Evaluate.scala | 9 +++++++++ src/main/scala/expressions/nullFunctions.scala | 2 ++ 2 files changed, 11 insertions(+) diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/expressions/Evaluate.scala index e343158c1f382..2a7427cfea7a7 100644 --- a/src/main/scala/expressions/Evaluate.scala +++ b/src/main/scala/expressions/Evaluate.scala @@ -172,8 +172,17 @@ object Evaluate extends Logging { case GreaterThanOrEqual(l, r) => n2(l, r, _.gteq(_, _)) case LessThan(l, r) => n2(l, r, _.lt(_, _)) case LessThanOrEqual(l, r) => n2(l, r, _.lteq(_, _)) + case IsNull(e) => eval(e) == null case IsNotNull(e) => eval(e) != null + case Coalesce(exprs) => + var currentExpression: Any = null + var i = 0 + while (i < exprs.size && currentExpression == null) { + currentExpression = eval(exprs(i)) + i += 1 + } + currentExpression /* Casts */ // toString diff --git a/src/main/scala/expressions/nullFunctions.scala b/src/main/scala/expressions/nullFunctions.scala index 593ef1e9c6a71..f9308ab0c5c67 100644 --- a/src/main/scala/expressions/nullFunctions.scala +++ b/src/main/scala/expressions/nullFunctions.scala @@ -10,6 +10,8 @@ case class Coalesce(children: Seq[Expression]) extends Expression { // Only resolved if all the children are of the same type. override lazy val resolved = childrenResolved && (children.map(_.dataType).distinct.size == 1) + override def toString = s"Coalesce(${children.mkString(",")})" + def dataType = if(resolved) children.head.dataType From d078333476252e28d8697b76a1ece63b66a9fc8d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 1 Jan 2014 18:43:12 -0800 Subject: [PATCH 353/778] remove extra space --- src/main/scala/expressions/arithmetic.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/expressions/arithmetic.scala b/src/main/scala/expressions/arithmetic.scala index 23342cb001f6a..1262d305aba1d 100644 --- a/src/main/scala/expressions/arithmetic.scala +++ b/src/main/scala/expressions/arithmetic.scala @@ -19,7 +19,7 @@ abstract class BinaryArithmetic extends BinaryExpression { def dataType = { if(!resolved) throw new UnresolvedException( - this, s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") + this, s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") left.dataType } } From a26494ed21e0f0b57146422c115c353fb7837610 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 1 Jan 2014 18:46:57 -0800 Subject: [PATCH 354/778] allow aliases to have qualifiers --- src/main/scala/expressions/namedExpressions.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index a72e9a9c3b59e..b3498d7575f84 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -21,14 +21,14 @@ abstract class NamedExpression extends Expression { def name: String def exprId: ExprId + def qualifiers: Seq[String] + def toAttribute: Attribute } abstract class Attribute extends NamedExpression { self: Product => - def qualifiers: Seq[String] - def withQualifiers(newQualifiers: Seq[String]): Attribute def references = Set(this) @@ -47,7 +47,8 @@ abstract class Attribute extends NamedExpression { * Auto-assigned if left blank. */ case class Alias(child: Expression, name: String) - (val exprId: ExprId = NamedExpression.newExprId) + (val exprId: ExprId = NamedExpression.newExprId, + val qualifiers: Seq[String] = Nil) extends NamedExpression with trees.UnaryNode[Expression] { def dataType = child.dataType @@ -56,13 +57,13 @@ case class Alias(child: Expression, name: String) def toAttribute = if(resolved) - AttributeReference(name, child.dataType, child.nullable)(exprId) + AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers) else UnresolvedAttribute(name) override def toString(): String = s"$child AS $name#${exprId.id}" - override protected final def otherCopyArgs = exprId :: Nil + override protected final def otherCopyArgs = exprId :: qualifiers :: Nil } /** From c795e06c7113698aafe3efe3cb93da89894ebe85 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 1 Jan 2014 18:49:15 -0800 Subject: [PATCH 355/778] improve star expansion --- src/main/scala/analysis/Analyzer.scala | 18 +++++++++++++----- src/main/scala/analysis/unresolved.scala | 19 ++++++++++++++++++- 2 files changed, 31 insertions(+), 6 deletions(-) diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/analysis/Analyzer.scala index 6461fa26bcdd7..335fcab3f83f5 100644 --- a/src/main/scala/analysis/Analyzer.scala +++ b/src/main/scala/analysis/Analyzer.scala @@ -52,7 +52,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case UnresolvedRelation(name, alias) => UnresolvedRelation(name, alias.map(_.toLowerCase)) case Subquery(alias, child) => Subquery(alias.toLowerCase, child) case q: LogicalPlan => q transformExpressions { - case Star(name) => Star(name.map(_.toLowerCase)) + case s: Star => s.copy(table = s.table.map(_.toLowerCase)) case UnresolvedAttribute(name) => UnresolvedAttribute(name.toLowerCase) case Alias(c, name) => Alias(c, name.toLowerCase)() } @@ -109,21 +109,29 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool */ object StarExpansion extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p @ Project(projectList, child) if childIsFullyResolved(p) && containsStar(projectList) => + // Wait until children are resolved + case p: LogicalPlan if !childIsFullyResolved(p) => p + case p @ Project(projectList, child) if containsStar(projectList) => Project( projectList.flatMap { - case Star(None) => child.output - case Star(Some(table)) => child.output.filter(_.qualifiers contains table) + case s: Star => s.expand(child.output) case o => o :: Nil }, child) + case a: Aggregate if containsStar(a.aggregateExpressions) => + a.copy( + aggregateExpressions = a.aggregateExpressions.flatMap { + case s: Star => s.expand(a.child.output) + case o => o :: Nil + } + ) } /** * Returns true if [[exprs]] contains a star. */ protected def containsStar(exprs: Seq[NamedExpression]): Boolean = - exprs.collect { case Star(_) => true }.nonEmpty + exprs.collect { case _: Star => true }.nonEmpty } /** diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala index da4d7e390ab78..ac942390bd8c6 100644 --- a/src/main/scala/analysis/unresolved.scala +++ b/src/main/scala/analysis/unresolved.scala @@ -51,7 +51,11 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E * @param table an optional table that should be the target of the expansion. If omitted all * tables' columns are produced. */ -case class Star(table: Option[String]) extends Attribute with trees.LeafNode[Expression] { +case class Star( + table: Option[String], + mapFunction: Attribute => Expression = identity[Attribute]) + extends Attribute with trees.LeafNode[Expression] { + def name = throw new UnresolvedException(this, "exprId") def exprId = throw new UnresolvedException(this, "exprId") def dataType = throw new UnresolvedException(this, "dataType") @@ -62,5 +66,18 @@ case class Star(table: Option[String]) extends Attribute with trees.LeafNode[Exp def newInstance = this def withQualifiers(newQualifiers: Seq[String]) = this + def expand(input: Seq[Attribute]): Seq[NamedExpression] = { + val expandedAttributes = table match { + case None => input + case Some(table) => input.filter(_.qualifiers contains table) + } + val mappedAttributes = expandedAttributes.map(mapFunction).zip(input).map { + case (n: NamedExpression, _) => n + case (e, originalAttribute) => + Alias(e, originalAttribute.name)(qualifiers = originalAttribute.qualifiers) + } + mappedAttributes + } + override def toString = table.map(_ + ".").getOrElse("") + "*" } \ No newline at end of file From e77c9b67928c51f2f5e15e9b8327c5f31862f7f9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 1 Jan 2014 18:51:09 -0800 Subject: [PATCH 356/778] more work on unique join. --- src/main/scala/frontend/Hive.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index 4317bdd08c1e4..a089eaa8c4d41 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -467,6 +467,8 @@ object HiveQl { joinType = joinType.remove(joinType.length - 1)) } + val groups = (0 until joinExpressions.head.size).map(i => Coalesce(joinExpressions.map(_(i)))) + // Unique join is not really the same as an outer join so we must group together results where // the joinExpressions are the same, taking the First of each value is only okay because the // user of a unique join is implicitly promising that there is only one result. @@ -475,7 +477,7 @@ object HiveQl { // worth the number of hacks that will be required to implement it. Namely, we need to add // some sort of mapped star expansion that would expand all child output row to be similarly // named output expressions where some aggregate expression has been applied (i.e. First). - ??? // Aggregate(joinExpressions.map(Coalesce(_)), Star(None) :: Nil, joinedResult) + ??? /// Aggregate(groups, Star(None, First(_)) :: Nil, joinedResult) /* Table with Alias */ case Token("TOK_TABREF", @@ -564,7 +566,6 @@ object HiveQl { val numericAstTypes = Seq(HiveParser.Number, HiveParser.TinyintLiteral, HiveParser.SmallintLiteral, HiveParser.BigintLiteral) - /* Case insensitive matches */ val COUNT = "(?i)COUNT".r val AVG = "(?i)AVG".r From e547e50037c5505457a9d35d3ae2edf13fee9380 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 1 Jan 2014 18:51:27 -0800 Subject: [PATCH 357/778] implement First. --- src/main/scala/shark2/aggregates.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/main/scala/shark2/aggregates.scala b/src/main/scala/shark2/aggregates.scala index c7a234672fa45..33d6ce69b4a6d 100644 --- a/src/main/scala/shark2/aggregates.scala +++ b/src/main/scala/shark2/aggregates.scala @@ -67,9 +67,18 @@ case class Aggregate(groupingExpressions: Seq[Expression], def result: Any = seen.size } - def output = aggregateExpressions.map(_.toAttribute) + case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + var result: Any = null + def apply(input: Seq[Row]): Unit = { + if(result == null) + result = Evaluate(expr, input) + } + } + + def output = aggregateExpressions.map(_.toAttribute) def execute() = attachTree(this, "execute") { val grouped = child.execute().map(row => (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row)).groupByKey() @@ -82,6 +91,7 @@ case class Aggregate(groupingExpressions: Seq[Expression], case base @ Sum(expr) => new SumFunction(expr, base) case base @ Count(expr) => new CountFunction(expr, base) case base @ CountDistinct(expr) => new CountDistinctFunction(expr, base) + case base @ First(expr) => new FirstFunction(expr, base) } val remainingAttributes = impl.collect { case a: Attribute => a } From 4283400fa63389eb315381bcf40a4f8f510a5838 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Thu, 2 Jan 2014 00:20:05 -0800 Subject: [PATCH 358/778] Add limited predicate push down --- src/main/scala/optimizer/Optimizer.scala | 196 +++++++++++++++++++++- src/main/scala/shark2/hiveOperators.scala | 3 +- src/test/scala/OptimizerSuite.scala | 133 ++++++++++++++- 3 files changed, 322 insertions(+), 10 deletions(-) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index 91258b79e34d2..0d02932de7517 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -1,13 +1,24 @@ package catalyst package optimizer +import catalyst.expressions._ +import catalyst.expressions.Alias +import catalyst.plans.Inner import catalyst.plans.logical._ +import catalyst.plans.logical.Filter +import catalyst.plans.logical.Project +import catalyst.plans.logical.Subquery import catalyst.rules._ +import catalyst.types.DataType object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Subqueries", Once, - EliminateSubqueries) :: Nil + EliminateSubqueries, + CombineFilters, + PredicatePushDown, + PredicatePushDownWithAlias, + PushPredicateThroughInnerEqualJoin) :: Nil } @@ -15,4 +26,187 @@ object EliminateSubqueries extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Subquery(_, child) => child } +} + +object CombineFilters extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case ff@Filter(fc, nf@Filter(nc, grandChild)) => Filter(And(nc, fc), grandChild) + } +} + +object PredicatePushDown extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case filter@Filter(_, project@Project(_, grandChild)) + if filter.references subsetOf (grandChild.outputSet) => + project.copy(child = filter.copy(child = grandChild)) + } +} + +object EmptyExpression extends EmptyExpressionType(null, null) + +case class EmptyExpressionType(left: Expression, right: Expression) extends Expression { + self: Product => + + case object NullType extends DataType + + /** Returns a Seq of the children of this node */ + + def children: Seq[Expression] = Seq.empty + + def dataType: DataType = NullType + + def nullable: Boolean = true + + def references: Set[Attribute] = Set.empty +} + +object PushPredicateThroughInnerEqualJoin extends Rule[LogicalPlan] { + class BinaryConditionStatus { + var left = false + var right = false + } + + object Direction extends Enumeration { + type Direction = Value + val Left, Right = Value + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case filter@Filter(_, join@Join(l, r, joinType@Inner, cond)) => { + var joinLeftExpression: Option[Expression] = None + var joinRightExpression: Option[Expression] = None + + val newFilterCondition = rewriteCondition (filter.condition transform { + case and: And => { + val status = new BinaryConditionStatus + + val rightFilter = + combineExpression( + joinFilter(status, Direction.Left, and.left, r.outputSet), + joinFilter(status, Direction.Right, and.right, r.outputSet), + And + ) + val leftFilter = + combineExpression( + joinFilter(status, Direction.Left, and.left, l.outputSet), + joinFilter(status, Direction.Right, and.right, l.outputSet), + And + ) + + joinLeftExpression = combineExpression(joinLeftExpression, leftFilter, And) + joinRightExpression = combineExpression(joinRightExpression, rightFilter, And) + + (status.left, status.right) match { + case (true, true) => EmptyExpression + case (true, false) => and.right + case (false, true) => and.left + case (false, false) => and + } + } + + case u: BinaryExpression if u.references subsetOf (l.outputSet) => { + joinLeftExpression = combineExpression(joinLeftExpression, Option(u), And) + EmptyExpression + } + + case u: BinaryExpression if u.references subsetOf (r.outputSet) => { + joinRightExpression = combineExpression(joinRightExpression, Option(u), And) + EmptyExpression + } + }) + + (newFilterCondition, joinLeftExpression, joinRightExpression) match { + case (EmptyExpression, _, _) => + copyJoin(join, joinLeftExpression, joinRightExpression) + case (_, None, None) => filter + case _ => { + filter.copy(condition = newFilterCondition, + child = copyJoin( + join, joinLeftExpression, joinRightExpression + ) + ) + } + } + } + } + + def rewriteCondition(exp: Expression): Expression = { + exp match { + case b: And => b match { + case And(EmptyExpression, e:Expression) => rewriteCondition(e) + case And(e:Expression, EmptyExpression) => rewriteCondition(e) + case And(e:Expression, e2:Expression) => And(rewriteCondition(e), rewriteCondition(e2)) + case And(EmptyExpression, EmptyExpression) => EmptyExpression + } + case _ => exp + } + } + + def combineExpression[T1, T2](firstExp: Option[T1], + secondExp: Option[T2], + combineFunc: (T1, T2) => T2): Option[T2] = { + (firstExp, secondExp) match { + case (o1: Some[T1], o2: Some[T2]) => Option(combineFunc(o1.get, o2.get)) + case (None, o2: Some[T2]) => o2 + case (o1: Some[T1], None) => Option(o1.get.asInstanceOf[T2]) + case (None, None) => None + } + } + + def copyJoin(join: Join, + joinLeftExpression: Option[Expression], + joinRightExpression: Option[Expression]): Join = { + join.copy( + left = + combineExpression( + joinLeftExpression, + Option(join.left), + Filter + ).get, + right = + combineExpression( + joinRightExpression, + Option(join.right), + Filter + ).get + ) + } + + def joinFilter(status: BinaryConditionStatus, + dir: Direction.Value, + expression: Expression, + outputSet: Set[Attribute]): Option[Expression] = { + if (expression.references subsetOf (outputSet)) { + dir match { + case Direction.Left => status.left = true + case Direction.Right => status.right = true + } + Option(expression) + } else None + } +} + +object PredicatePushDownWithAlias extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case filter@Filter(condition, project@Project(fields, grandChild)) => + val refMissing = filter.references &~ (filter.references intersect grandChild.outputSet) + val sourceAliases = fields.filter(_.isInstanceOf[Alias]).map(_.asInstanceOf[Alias]).toSet + if (isAliasOf(refMissing, sourceAliases)) + project.copy(child = filter.copy( + replaceAlias(condition, sourceAliases), + grandChild)) + else filter + } + + def isAliasOf(refMissing: Set[Attribute], sourceAliases: Set[Alias]): Boolean = { + refMissing.forall(r => sourceAliases.exists(_.exprId == r.exprId)) + } + + // Assuming the expression cost is minimal, we can replace condition with the alias's child to push down predicate + def replaceAlias(condition: Expression, sourceAliases: Set[Alias]): Expression = { + condition transform { + case a: AttributeReference if sourceAliases.exists(_.exprId == a.exprId) => + sourceAliases.find(_.exprId == a.exprId).get.child + } + } } \ No newline at end of file diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/shark2/hiveOperators.scala index 628f33b9fc1af..703f3a446d270 100644 --- a/src/main/scala/shark2/hiveOperators.scala +++ b/src/main/scala/shark2/hiveOperators.scala @@ -90,8 +90,7 @@ case class InsertIntoHiveTable(table: MetastoreRelation, child: SharkPlan) table.tableDesc, outputClass, desc, - new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory(), "test.out"), - null) + new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory(), "test.out")) override def otherCopyArgs = sc :: Nil diff --git a/src/test/scala/OptimizerSuite.scala b/src/test/scala/OptimizerSuite.scala index af21151243b05..0603aa9cfc864 100644 --- a/src/test/scala/OptimizerSuite.scala +++ b/src/test/scala/OptimizerSuite.scala @@ -34,6 +34,8 @@ class OptimizerSuite extends FunSuite { protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { val normalized1 = normalizeExprIds(plan1) val normalized2 = normalizeExprIds(plan2) + println(normalized1.treeString) + println(normalized2.treeString) if(normalized1 != normalized2) fail( s""" @@ -82,10 +84,33 @@ class OptimizerSuite extends FunSuite { .where('e === 1) .analyze - /* Your code here */ - fail("not implemented") + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = + testRelation + .where('a + 'b === 1) + .select('a + 'b as 'e) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("filters: combines filters") { + val originalQuery = testRelation + .select('a) + .where('a === 1) + .where('a === 2) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = + testRelation + .where('a === 1 && 'a === 2) + .select('a).analyze + + + comparePlans(optimized, correctAnswer) } + test("joins: push to either side") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -94,10 +119,51 @@ class OptimizerSuite extends FunSuite { x.join(y) .where("x.b".attr === 1) .where("y.b".attr === 2) - .analyze } - fail("not implemented") + val optimized = Optimize(originalQuery.analyze) + val left = testRelation.where('b === 1) + val right = testRelation.where('b === 2) + val correctAnswer = + left.join(right).analyze + + comparePlans(optimized, correctAnswer) + } + + test("joins: push to one side") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = { + x.join(y) + .where("x.b".attr === 1) + } + + val optimized = Optimize(originalQuery.analyze) + val left = testRelation.where('b === 1) + val right = testRelation + val correctAnswer = + left.join(right).analyze + + comparePlans(optimized, correctAnswer) + } + + test("joins: rewrite filter to push to either side") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = { + x.join(y) + .where("x.b".attr === 1 && "y.b".attr === 2) + } + + val optimized = Optimize(originalQuery.analyze) + val left = testRelation.where('b === 1) + val right = testRelation.where('b === 2) + val correctAnswer = + left.join(right).analyze + + comparePlans(optimized, correctAnswer) } test("joins: can't push down") { @@ -107,11 +173,10 @@ class OptimizerSuite extends FunSuite { val originalQuery = { x.join(y) .where("x.b".attr === "y.b".attr) - .analyze } val optimized = Optimize(originalQuery.analyze) - comparePlans(optimizer.EliminateSubqueries(originalQuery), optimized) + comparePlans(optimizer.EliminateSubqueries(originalQuery.analyze), optimized) } test("joins: conjunctive predicates") { @@ -121,9 +186,63 @@ class OptimizerSuite extends FunSuite { val originalQuery = { x.join(y) .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("y.a".attr === 1)) + } + + val optimized = Optimize(originalQuery.analyze) + val left = testRelation.where('a === 1).subquery('x) + val right = testRelation.where('a === 1).subquery('y) + val correctAnswer = + left.join(right) + .where("x.b".attr === "y.b".attr) + .analyze + + comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) + } + + test("joins: conjunctive predicates #2") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = { + x.join(y) + .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1)) + } + + val optimized = Optimize(originalQuery.analyze) + val left = testRelation.where('a === 1).subquery('x) + val right = testRelation.subquery('y) + val correctAnswer = + left.join(right) + .where("x.b".attr === "y.b".attr) .analyze + + comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) + } + + test("joins: conjunctive predicates #3") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + val z = testRelation.subquery('z) + + val originalQuery = { + z.join(x.join(y)) + .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("z.a".attr >= 3) && ("z.a".attr === "x.b".attr)) } - fail("not implemented") + val optimized = Optimize(originalQuery.analyze) + val lleft = testRelation.where('a >= 3).subquery('z) + val left = testRelation.where('a === 1).subquery('x) + val right = testRelation.subquery('y) + val correctAnswer = + lleft + .join( + left + .join(right) + .where("x.b".attr === "y.b".attr) + ) + .where("z.a".attr === "x.b".attr) + .analyze + + comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) } } \ No newline at end of file From 75b5a0199acd997d41bce649339a207abb28f536 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 2 Jan 2014 12:31:16 -0800 Subject: [PATCH 359/778] remove space. --- src/main/scala/analysis/typeCoercion.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/analysis/typeCoercion.scala b/src/main/scala/analysis/typeCoercion.scala index 62563374f7e5f..cdb9aa6100350 100644 --- a/src/main/scala/analysis/typeCoercion.scala +++ b/src/main/scala/analysis/typeCoercion.scala @@ -96,7 +96,7 @@ object PromoteStrings extends Rule[LogicalPlan] { case a: BinaryArithmetic if a.right.dataType == StringType => a.makeCopy(Array(a.left, Cast(a.right, DoubleType))) - case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType != StringType => + case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType != StringType => p.makeCopy(Array(Cast(p.left, DoubleType), p.right)) case p: BinaryPredicate if p.left.dataType != StringType && p.right.dataType == StringType => p.makeCopy(Array(p.left, Cast(p.right, DoubleType))) From 00793926c85b3525acdb2dc7887d98c61e49fcb5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 2 Jan 2014 16:56:27 -0800 Subject: [PATCH 360/778] support for multiple insert commands in a single query --- src/main/scala/frontend/Hive.scala | 113 ++++++++++++++++------------- 1 file changed, 64 insertions(+), 49 deletions(-) diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/frontend/Hive.scala index a089eaa8c4d41..69b571a8bca3f 100644 --- a/src/main/scala/frontend/Hive.scala +++ b/src/main/scala/frontend/Hive.scala @@ -360,60 +360,75 @@ object HiveQl { case Token("TOK_QUERY", Token("TOK_FROM", fromClause :: Nil) :: - Token("TOK_INSERT", insertClauses) :: Nil) => - - val (Some(destClause) :: - Some(selectClause) :: - whereClause :: - groupByClause :: - orderByClause :: - sortByClause :: - limitClause :: Nil) = getClauses(Seq("TOK_DESTINATION", "TOK_SELECT", "TOK_WHERE", "TOK_GROUPBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_LIMIT"), insertClauses) - - val relations = nodeToRelation(fromClause) - val withWhere = whereClause.map { whereNode => - val Seq(whereExpr) = whereNode.getChildren().toSeq - Filter(nodeToExpr(whereExpr), relations) - }.getOrElse(relations) - - // Script transformations are expressed as a select clause with a single expression of type TOK_TRANSFORM - val transformation = selectClause.getChildren.head match { - case Token("TOK_SELEXPR", - Token("TOK_TRANSFORM", - Token("TOK_EXPLIST", inputExprs) :: - Token("TOK_SERDE", Nil) :: - Token("TOK_RECORDWRITER", Nil) :: // TODO: Need to support other types of (in/out)put - Token(script, Nil):: - Token("TOK_SERDE", Nil) :: - Token("TOK_RECORDREADER", Nil) :: - Token("TOK_ALIASLIST", aliases) :: Nil) :: Nil) => - - val output = aliases.map { case Token(n, Nil) => AttributeReference(n, StringType)() } - val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) - Some(Transform(inputExprs.map(nodeToExpr), unescapedScript, output, withWhere)) - case _ => None - } + insertClauses) => + + // Return one query for each insert clause. + val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => + val (Some(destClause) :: + Some(selectClause) :: + whereClause :: + groupByClause :: + orderByClause :: + sortByClause :: + limitClause :: Nil) = getClauses(Seq("TOK_DESTINATION", "TOK_SELECT", "TOK_WHERE", "TOK_GROUPBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_LIMIT"), singleInsert) + + val relations = nodeToRelation(fromClause) + val withWhere = whereClause.map { whereNode => + val Seq(whereExpr) = whereNode.getChildren().toSeq + Filter(nodeToExpr(whereExpr), relations) + }.getOrElse(relations) + + + // Script transformations are expressed as a select clause with a single expression of type + // TOK_TRANSFORM + val transformation = selectClause.getChildren.head match { + case Token("TOK_SELEXPR", + Token("TOK_TRANSFORM", + Token("TOK_EXPLIST", inputExprs) :: + Token("TOK_SERDE", Nil) :: + Token("TOK_RECORDWRITER", Nil) :: // TODO: Need to support other types of (in/out)put + Token(script, Nil):: + Token("TOK_SERDE", Nil) :: + Token("TOK_RECORDREADER", Nil) :: + Token("TOK_ALIASLIST", aliases) :: Nil) :: Nil) => + + val output = aliases.map { case Token(n, Nil) => AttributeReference(n, StringType)() } + val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) + Some(Transform(inputExprs.map(nodeToExpr), unescapedScript, output, withWhere)) + case _ => None + } - // The projection of the query can either be a normal projection, an aggregation (if there is a group by) or - // a script transformation. - val withProject = transformation.getOrElse { - // Not a transformation so must be either project or aggregation. - val selectExpressions = nameExpressions(selectClause.getChildren.flatMap(selExprNodeToExpr)) + // The projection of the query can either be a normal projection, an aggregation (if there is a group by) or + // a script transformation. + val withProject = transformation.getOrElse { + // Not a transformation so must be either project or aggregation. + val selectExpressions = nameExpressions(selectClause.getChildren.flatMap(selExprNodeToExpr)) - groupByClause match { - case Some(groupBy) => Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withWhere) - case None => Project(selectExpressions, withWhere) + groupByClause match { + case Some(groupBy) => Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withWhere) + case None => Project(selectExpressions, withWhere) + } } - } - require(!(orderByClause.isDefined && sortByClause.isDefined), "Can't have both a sort by and order by.") - // Right now we treat sorting and ordering as identical. - val withSort = (orderByClause orElse sortByClause).map(_.getChildren.map(nodeToSortOrder)).map(Sort(_, withProject)).getOrElse(withProject) - val withLimit = limitClause.map(l => nodeToExpr(l.getChildren.head)).map(StopAfter(_, withSort)).getOrElse(withSort) + require(!(orderByClause.isDefined && sortByClause.isDefined), "Can't have both a sort by and order by.") + // Right now we treat sorting and ordering as identical. + val withSort = + (orderByClause orElse sortByClause) + .map(_.getChildren.map(nodeToSortOrder)) + .map(Sort(_, withProject)) + .getOrElse(withProject) + val withLimit = + limitClause.map(l => nodeToExpr(l.getChildren.head)) + .map(StopAfter(_, withSort)) + .getOrElse(withSort) + + nodeToDest( + destClause, + withLimit) + } - nodeToDest( - destClause, - withLimit) + // If there are multiple INSERTS just UNION them together into on query. + queries.reduceLeft(Union) case Token("TOK_UNION", left :: right :: Nil) => Union(nodeToPlan(left), nodeToPlan(right)) From 80ca4be4a78839f0f16e456a40a6803e22a961ed Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Thu, 2 Jan 2014 18:07:13 -0800 Subject: [PATCH 361/778] Address comments --- src/main/scala/expressions/predicates.scala | 7 + src/main/scala/optimizer/Optimizer.scala | 152 ++++++++---------- src/main/scala/planning/patterns.scala | 7 +- .../scala/plans/logical/basicOperators.scala | 2 +- src/main/scala/types/dataTypes.scala | 1 + src/test/scala/OptimizerSuite.scala | 2 - 6 files changed, 76 insertions(+), 95 deletions(-) diff --git a/src/main/scala/expressions/predicates.scala b/src/main/scala/expressions/predicates.scala index cbb543a32d131..dc52495a588b5 100644 --- a/src/main/scala/expressions/predicates.scala +++ b/src/main/scala/expressions/predicates.scala @@ -9,6 +9,13 @@ abstract trait Predicate extends Expression { def dataType = BooleanType } +abstract trait PredicateHelper { + def splitConjunctivePredicates(condition: Expression): Seq[Expression] = condition match { + case And(cond1, cond2) => splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) + case other => other :: Nil + } +} + abstract class BinaryPredicate extends BinaryExpression with Predicate { self: Product => diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index 0d02932de7517..f96be8df483dc 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -3,21 +3,19 @@ package optimizer import catalyst.expressions._ import catalyst.expressions.Alias -import catalyst.plans.Inner import catalyst.plans.logical._ import catalyst.plans.logical.Filter import catalyst.plans.logical.Project import catalyst.plans.logical.Subquery import catalyst.rules._ -import catalyst.types.DataType +import catalyst.types.{NullType, DataType} object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Subqueries", Once, EliminateSubqueries, CombineFilters, - PredicatePushDown, - PredicatePushDownWithAlias, + PredicatePushDownThoughProject, PushPredicateThroughInnerEqualJoin) :: Nil } @@ -28,17 +26,32 @@ object EliminateSubqueries extends Rule[LogicalPlan] { } } +/** + * Combines two filter operators into one + */ object CombineFilters extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case ff@Filter(fc, nf@Filter(nc, grandChild)) => Filter(And(nc, fc), grandChild) } } -object PredicatePushDown extends Rule[LogicalPlan] { +/** + * Pushes predicate through project, also inlines project's aliases in filter + */ +object PredicatePushDownThoughProject extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case filter@Filter(_, project@Project(_, grandChild)) - if filter.references subsetOf (grandChild.outputSet) => - project.copy(child = filter.copy(child = grandChild)) + case filter@Filter(condition, project@Project(fields, grandChild)) => + val sourceAliases = fields.collect { case a@Alias(c, _) => a.toAttribute -> c }.toMap + project.copy(child = filter.copy( + replaceAlias(condition, sourceAliases), + grandChild)) + } + + // Assuming the expression cost is minimal, we can replace condition with the alias's child to push down predicate + def replaceAlias(condition: Expression, sourceAliases: Map[Attribute, Expression]): Expression = { + condition transform { + case a: AttributeReference => sourceAliases.getOrElse(a, a) + } } } @@ -47,8 +60,6 @@ object EmptyExpression extends EmptyExpressionType(null, null) case class EmptyExpressionType(left: Expression, right: Expression) extends Expression { self: Product => - case object NullType extends DataType - /** Returns a Seq of the children of this node */ def children: Seq[Expression] = Seq.empty @@ -60,60 +71,42 @@ case class EmptyExpressionType(left: Expression, right: Expression) extends Expr def references: Set[Attribute] = Set.empty } +/** + * Pushes predicate through inner join + */ object PushPredicateThroughInnerEqualJoin extends Rule[LogicalPlan] { class BinaryConditionStatus { var left = false var right = false } - object Direction extends Enumeration { - type Direction = Value - val Left, Right = Value - } + abstract class Direction + case object Left extends Direction + case object Right extends Direction def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case filter@Filter(_, join@Join(l, r, joinType@Inner, cond)) => { + case filter@Filter(_, join@Join(l, r, _, _)) => { var joinLeftExpression: Option[Expression] = None var joinRightExpression: Option[Expression] = None - val newFilterCondition = rewriteCondition (filter.condition transform { - case and: And => { - val status = new BinaryConditionStatus - - val rightFilter = - combineExpression( - joinFilter(status, Direction.Left, and.left, r.outputSet), - joinFilter(status, Direction.Right, and.right, r.outputSet), - And - ) - val leftFilter = - combineExpression( - joinFilter(status, Direction.Left, and.left, l.outputSet), - joinFilter(status, Direction.Right, and.right, l.outputSet), - And - ) - - joinLeftExpression = combineExpression(joinLeftExpression, leftFilter, And) - joinRightExpression = combineExpression(joinRightExpression, rightFilter, And) + val conditions = filter.splitConjunctivePredicates(filter.condition).filter( + e => { + var changed = false + if(e.references subsetOf l.outputSet) { + joinLeftExpression = combineExpression(joinLeftExpression, Option(e), And) + changed = true + } - (status.left, status.right) match { - case (true, true) => EmptyExpression - case (true, false) => and.right - case (false, true) => and.left - case (false, false) => and + if(e.references subsetOf r.outputSet) { + joinRightExpression = combineExpression(joinRightExpression, Option(e), And) + changed = true } - } - case u: BinaryExpression if u.references subsetOf (l.outputSet) => { - joinLeftExpression = combineExpression(joinLeftExpression, Option(u), And) - EmptyExpression + changed } + ).toSet - case u: BinaryExpression if u.references subsetOf (r.outputSet) => { - joinRightExpression = combineExpression(joinRightExpression, Option(u), And) - EmptyExpression - } - }) + val newFilterCondition = rewriteCondition (filter.condition, conditions) (newFilterCondition, joinLeftExpression, joinRightExpression) match { case (EmptyExpression, _, _) => @@ -130,15 +123,27 @@ object PushPredicateThroughInnerEqualJoin extends Rule[LogicalPlan] { } } - def rewriteCondition(exp: Expression): Expression = { + def rewriteCondition(exp: Expression, removedExps: Set[Expression]): Expression = { exp match { case b: And => b match { - case And(EmptyExpression, e:Expression) => rewriteCondition(e) - case And(e:Expression, EmptyExpression) => rewriteCondition(e) - case And(e:Expression, e2:Expression) => And(rewriteCondition(e), rewriteCondition(e2)) - case And(EmptyExpression, EmptyExpression) => EmptyExpression + case And(e1: Expression, e2:Expression) => { + val r1 = rewriteCondition(e1, removedExps) + val r2 = rewriteCondition(e2, removedExps) + + (r1, r2) match { + case (EmptyExpression, e: Expression) => e + case (e: Expression, EmptyExpression) => e + case (EmptyExpression, EmptyExpression) => EmptyExpression + case _ => And(r1, r2) + } + } + } + case e => { + if(removedExps.contains(e)) + EmptyExpression + else + e } - case _ => exp } } @@ -146,9 +151,9 @@ object PushPredicateThroughInnerEqualJoin extends Rule[LogicalPlan] { secondExp: Option[T2], combineFunc: (T1, T2) => T2): Option[T2] = { (firstExp, secondExp) match { - case (o1: Some[T1], o2: Some[T2]) => Option(combineFunc(o1.get, o2.get)) - case (None, o2: Some[T2]) => o2 - case (o1: Some[T1], None) => Option(o1.get.asInstanceOf[T2]) + case (Some(o1), Some(o2)) => Option(combineFunc(o1, o2)) + case (None, Some(o2)) => Option(o2) + case (Some(o1), None) => Option(o1.asInstanceOf[T2]) case (None, None) => None } } @@ -173,40 +178,15 @@ object PushPredicateThroughInnerEqualJoin extends Rule[LogicalPlan] { } def joinFilter(status: BinaryConditionStatus, - dir: Direction.Value, + dir: Direction, expression: Expression, outputSet: Set[Attribute]): Option[Expression] = { - if (expression.references subsetOf (outputSet)) { + if (expression.references subsetOf outputSet) { dir match { - case Direction.Left => status.left = true - case Direction.Right => status.right = true + case Left => status.left = true + case Right => status.right = true } Option(expression) } else None } -} - -object PredicatePushDownWithAlias extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case filter@Filter(condition, project@Project(fields, grandChild)) => - val refMissing = filter.references &~ (filter.references intersect grandChild.outputSet) - val sourceAliases = fields.filter(_.isInstanceOf[Alias]).map(_.asInstanceOf[Alias]).toSet - if (isAliasOf(refMissing, sourceAliases)) - project.copy(child = filter.copy( - replaceAlias(condition, sourceAliases), - grandChild)) - else filter - } - - def isAliasOf(refMissing: Set[Attribute], sourceAliases: Set[Alias]): Boolean = { - refMissing.forall(r => sourceAliases.exists(_.exprId == r.exprId)) - } - - // Assuming the expression cost is minimal, we can replace condition with the alias's child to push down predicate - def replaceAlias(condition: Expression, sourceAliases: Set[Alias]): Expression = { - condition transform { - case a: AttributeReference if sourceAliases.exists(_.exprId == a.exprId) => - sourceAliases.find(_.exprId == a.exprId).get.child - } - } } \ No newline at end of file diff --git a/src/main/scala/planning/patterns.scala b/src/main/scala/planning/patterns.scala index ba88167df7843..cba1eb64f016a 100644 --- a/src/main/scala/planning/patterns.scala +++ b/src/main/scala/planning/patterns.scala @@ -13,7 +13,7 @@ import plans.logical._ * @returns A tuple containing a sequence of conjunctive predicates that should be used to filter the output and a * relational operator. */ -object FilteredOperation { +object FilteredOperation extends PredicateHelper { type ReturnType = (Seq[Expression], LogicalPlan) def unapply(plan: LogicalPlan): Option[ReturnType] = Some(collectFilters(Nil, plan)) @@ -23,9 +23,4 @@ object FilteredOperation { case Filter(condition, child) => collectFilters(filters ++ splitConjunctivePredicates(condition), child) case other => (filters, other) } - - private def splitConjunctivePredicates(condition: Expression): Seq[Expression] = condition match { - case And(cond1, cond2) => splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) - case other => other :: Nil - } } \ No newline at end of file diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/plans/logical/basicOperators.scala index ec4a7a8da343e..3cc9bbf8b06a3 100644 --- a/src/main/scala/plans/logical/basicOperators.scala +++ b/src/main/scala/plans/logical/basicOperators.scala @@ -9,7 +9,7 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend def references = projectList.flatMap(_.references).toSet } -case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { +case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode with PredicateHelper { def output = child.output def references = condition.references } diff --git a/src/main/scala/types/dataTypes.scala b/src/main/scala/types/dataTypes.scala index 4decfb607ac62..92f683d5e3ab6 100644 --- a/src/main/scala/types/dataTypes.scala +++ b/src/main/scala/types/dataTypes.scala @@ -13,6 +13,7 @@ case object DoubleType extends DataType case object LongType extends DataType case object ByteType extends DataType case object ShortType extends DataType +case object NullType extends DataType case class ArrayType(elementType: DataType) extends DataType diff --git a/src/test/scala/OptimizerSuite.scala b/src/test/scala/OptimizerSuite.scala index 0603aa9cfc864..d732d2142a3ca 100644 --- a/src/test/scala/OptimizerSuite.scala +++ b/src/test/scala/OptimizerSuite.scala @@ -2,11 +2,9 @@ package catalyst import org.scalatest.FunSuite -import analysis._ import expressions._ import optimizer.Optimize import plans.logical._ -import types._ import util._ /* Implicit conversions for creating query plans */ From 019ea74ed1aa4e24e46b4f6b9459dba8e30d3e53 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 2 Jan 2014 18:44:15 -0800 Subject: [PATCH 362/778] Updated .gitignore to include IntelliJ files. --- .gitignore | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/.gitignore b/.gitignore index 698e53f701343..c47f11d10e68c 100644 --- a/.gitignore +++ b/.gitignore @@ -1,10 +1,19 @@ -.DS_Store + +# Scala compilation target/ + +# IntelliJ files +*.iml .idea/ +.idea_modules/ + +# Hive files metastore_db/ metastore/ warehouse/ -TempStatsStore/ -hs_err_*.log derby.log -.idea_modules/ +hs_err_*.log +TempStatsStore/ + +# Mac OS X +.DS_Store From c9777d8f4ecf130ef8e5de88fddf663fc17322fc Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 2 Jan 2014 18:53:37 -0800 Subject: [PATCH 363/778] Put all source files in a catalyst directory. --- .../{ => catalyst}/analysis/Analyzer.scala | 0 .../{ => catalyst}/analysis/Catalog.scala | 0 .../analysis/FunctionRegistry.scala | 0 .../{ => catalyst}/analysis/package.scala | 0 .../analysis/typeCoercion.scala | 0 .../{ => catalyst}/analysis/unresolved.scala | 0 src/main/scala/{ => catalyst}/dsl.scala | 0 .../scala/{ => catalyst}/errors/package.scala | 0 .../examples/SchemaRddExample.scala | 0 .../examples/ViewsExample.scala | 0 .../expressions/BoundAttribute.scala | 0 .../{ => catalyst}/expressions/Cast.scala | 0 .../{ => catalyst}/expressions/Evaluate.scala | 0 .../expressions/Expression.scala | 0 .../{ => catalyst}/expressions/Rand.scala | 0 .../{ => catalyst}/expressions/Row.scala | 0 .../{ => catalyst}/expressions/ScalaUdf.scala | 0 .../expressions/SortOrder.scala | 0 .../expressions/WrapDynamic.scala | 0 .../expressions/aggregates.scala | 0 .../expressions/arithmetic.scala | 0 .../expressions/functions.scala | 0 .../{ => catalyst}/expressions/literals.scala | 0 .../expressions/namedExpressions.scala | 0 .../expressions/nullFunctions.scala | 0 .../{ => catalyst}/expressions/package.scala | 0 .../expressions/predicates.scala | 0 .../expressions/stringOperations.scala | 0 .../scala/{ => catalyst}/frontend/Hive.scala | 0 .../{ => catalyst}/frontend/package.scala | 0 .../{ => catalyst}/optimizer/Optimizer.scala | 0 src/main/scala/{ => catalyst}/package.scala | 0 .../planning/QueryPlanner.scala | 0 .../{ => catalyst}/planning/package.scala | 0 .../{ => catalyst}/planning/patterns.scala | 0 .../{ => catalyst}/plans/QueryPlan.scala | 0 .../plans/logical/BaseRelation.scala | 0 .../plans/logical/LogicalPlan.scala | 0 .../plans/logical/TestRelation.scala | 0 .../plans/logical/Transform.scala | 0 .../plans/logical/basicOperators.scala | 0 .../scala/{ => catalyst}/plans/package.scala | 0 .../scala/{ => catalyst}/plans/shared.scala | 0 .../scala/{ => catalyst}/rules/Rule.scala | 0 .../{ => catalyst}/rules/RuleExecutor.scala | 0 .../scala/{ => catalyst}/rules/package.scala | 0 .../shark2/FunctionRegistry.scala | 0 .../shark2/MetastoreCatalog.scala | 0 .../{ => catalyst}/shark2/SharkInstance.scala | 42 +++++++++---------- .../{ => catalyst}/shark2/SharkPlan.scala | 0 .../{ => catalyst}/shark2/TestShark.scala | 0 .../{ => catalyst}/shark2/Transform.scala | 0 .../{ => catalyst}/shark2/aggregates.scala | 0 .../shark2/basicOperators.scala | 0 .../scala/{ => catalyst}/shark2/debug.scala | 0 .../{ => catalyst}/shark2/hiveOperators.scala | 0 .../scala/{ => catalyst}/shark2/joins.scala | 0 .../scala/{ => catalyst}/shark2/package.scala | 0 .../shark2/planningStrategies.scala | 0 .../scala/{ => catalyst}/trees/TreeNode.scala | 0 .../scala/{ => catalyst}/trees/package.scala | 0 .../{ => catalyst}/types/dataTypes.scala | 0 .../scala/{ => catalyst}/types/package.scala | 0 .../scala/{ => catalyst}/util/package.scala | 0 64 files changed, 20 insertions(+), 22 deletions(-) rename src/main/scala/{ => catalyst}/analysis/Analyzer.scala (100%) rename src/main/scala/{ => catalyst}/analysis/Catalog.scala (100%) rename src/main/scala/{ => catalyst}/analysis/FunctionRegistry.scala (100%) rename src/main/scala/{ => catalyst}/analysis/package.scala (100%) rename src/main/scala/{ => catalyst}/analysis/typeCoercion.scala (100%) rename src/main/scala/{ => catalyst}/analysis/unresolved.scala (100%) rename src/main/scala/{ => catalyst}/dsl.scala (100%) rename src/main/scala/{ => catalyst}/errors/package.scala (100%) rename src/main/scala/{ => catalyst}/examples/SchemaRddExample.scala (100%) rename src/main/scala/{ => catalyst}/examples/ViewsExample.scala (100%) rename src/main/scala/{ => catalyst}/expressions/BoundAttribute.scala (100%) rename src/main/scala/{ => catalyst}/expressions/Cast.scala (100%) rename src/main/scala/{ => catalyst}/expressions/Evaluate.scala (100%) rename src/main/scala/{ => catalyst}/expressions/Expression.scala (100%) rename src/main/scala/{ => catalyst}/expressions/Rand.scala (100%) rename src/main/scala/{ => catalyst}/expressions/Row.scala (100%) rename src/main/scala/{ => catalyst}/expressions/ScalaUdf.scala (100%) rename src/main/scala/{ => catalyst}/expressions/SortOrder.scala (100%) rename src/main/scala/{ => catalyst}/expressions/WrapDynamic.scala (100%) rename src/main/scala/{ => catalyst}/expressions/aggregates.scala (100%) rename src/main/scala/{ => catalyst}/expressions/arithmetic.scala (100%) rename src/main/scala/{ => catalyst}/expressions/functions.scala (100%) rename src/main/scala/{ => catalyst}/expressions/literals.scala (100%) rename src/main/scala/{ => catalyst}/expressions/namedExpressions.scala (100%) rename src/main/scala/{ => catalyst}/expressions/nullFunctions.scala (100%) rename src/main/scala/{ => catalyst}/expressions/package.scala (100%) rename src/main/scala/{ => catalyst}/expressions/predicates.scala (100%) rename src/main/scala/{ => catalyst}/expressions/stringOperations.scala (100%) rename src/main/scala/{ => catalyst}/frontend/Hive.scala (100%) rename src/main/scala/{ => catalyst}/frontend/package.scala (100%) rename src/main/scala/{ => catalyst}/optimizer/Optimizer.scala (100%) rename src/main/scala/{ => catalyst}/package.scala (100%) rename src/main/scala/{ => catalyst}/planning/QueryPlanner.scala (100%) rename src/main/scala/{ => catalyst}/planning/package.scala (100%) rename src/main/scala/{ => catalyst}/planning/patterns.scala (100%) rename src/main/scala/{ => catalyst}/plans/QueryPlan.scala (100%) rename src/main/scala/{ => catalyst}/plans/logical/BaseRelation.scala (100%) rename src/main/scala/{ => catalyst}/plans/logical/LogicalPlan.scala (100%) rename src/main/scala/{ => catalyst}/plans/logical/TestRelation.scala (100%) rename src/main/scala/{ => catalyst}/plans/logical/Transform.scala (100%) rename src/main/scala/{ => catalyst}/plans/logical/basicOperators.scala (100%) rename src/main/scala/{ => catalyst}/plans/package.scala (100%) rename src/main/scala/{ => catalyst}/plans/shared.scala (100%) rename src/main/scala/{ => catalyst}/rules/Rule.scala (100%) rename src/main/scala/{ => catalyst}/rules/RuleExecutor.scala (100%) rename src/main/scala/{ => catalyst}/rules/package.scala (100%) rename src/main/scala/{ => catalyst}/shark2/FunctionRegistry.scala (100%) rename src/main/scala/{ => catalyst}/shark2/MetastoreCatalog.scala (100%) rename src/main/scala/{ => catalyst}/shark2/SharkInstance.scala (78%) rename src/main/scala/{ => catalyst}/shark2/SharkPlan.scala (100%) rename src/main/scala/{ => catalyst}/shark2/TestShark.scala (100%) rename src/main/scala/{ => catalyst}/shark2/Transform.scala (100%) rename src/main/scala/{ => catalyst}/shark2/aggregates.scala (100%) rename src/main/scala/{ => catalyst}/shark2/basicOperators.scala (100%) rename src/main/scala/{ => catalyst}/shark2/debug.scala (100%) rename src/main/scala/{ => catalyst}/shark2/hiveOperators.scala (100%) rename src/main/scala/{ => catalyst}/shark2/joins.scala (100%) rename src/main/scala/{ => catalyst}/shark2/package.scala (100%) rename src/main/scala/{ => catalyst}/shark2/planningStrategies.scala (100%) rename src/main/scala/{ => catalyst}/trees/TreeNode.scala (100%) rename src/main/scala/{ => catalyst}/trees/package.scala (100%) rename src/main/scala/{ => catalyst}/types/dataTypes.scala (100%) rename src/main/scala/{ => catalyst}/types/package.scala (100%) rename src/main/scala/{ => catalyst}/util/package.scala (100%) diff --git a/src/main/scala/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala similarity index 100% rename from src/main/scala/analysis/Analyzer.scala rename to src/main/scala/catalyst/analysis/Analyzer.scala diff --git a/src/main/scala/analysis/Catalog.scala b/src/main/scala/catalyst/analysis/Catalog.scala similarity index 100% rename from src/main/scala/analysis/Catalog.scala rename to src/main/scala/catalyst/analysis/Catalog.scala diff --git a/src/main/scala/analysis/FunctionRegistry.scala b/src/main/scala/catalyst/analysis/FunctionRegistry.scala similarity index 100% rename from src/main/scala/analysis/FunctionRegistry.scala rename to src/main/scala/catalyst/analysis/FunctionRegistry.scala diff --git a/src/main/scala/analysis/package.scala b/src/main/scala/catalyst/analysis/package.scala similarity index 100% rename from src/main/scala/analysis/package.scala rename to src/main/scala/catalyst/analysis/package.scala diff --git a/src/main/scala/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala similarity index 100% rename from src/main/scala/analysis/typeCoercion.scala rename to src/main/scala/catalyst/analysis/typeCoercion.scala diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/catalyst/analysis/unresolved.scala similarity index 100% rename from src/main/scala/analysis/unresolved.scala rename to src/main/scala/catalyst/analysis/unresolved.scala diff --git a/src/main/scala/dsl.scala b/src/main/scala/catalyst/dsl.scala similarity index 100% rename from src/main/scala/dsl.scala rename to src/main/scala/catalyst/dsl.scala diff --git a/src/main/scala/errors/package.scala b/src/main/scala/catalyst/errors/package.scala similarity index 100% rename from src/main/scala/errors/package.scala rename to src/main/scala/catalyst/errors/package.scala diff --git a/src/main/scala/examples/SchemaRddExample.scala b/src/main/scala/catalyst/examples/SchemaRddExample.scala similarity index 100% rename from src/main/scala/examples/SchemaRddExample.scala rename to src/main/scala/catalyst/examples/SchemaRddExample.scala diff --git a/src/main/scala/examples/ViewsExample.scala b/src/main/scala/catalyst/examples/ViewsExample.scala similarity index 100% rename from src/main/scala/examples/ViewsExample.scala rename to src/main/scala/catalyst/examples/ViewsExample.scala diff --git a/src/main/scala/expressions/BoundAttribute.scala b/src/main/scala/catalyst/expressions/BoundAttribute.scala similarity index 100% rename from src/main/scala/expressions/BoundAttribute.scala rename to src/main/scala/catalyst/expressions/BoundAttribute.scala diff --git a/src/main/scala/expressions/Cast.scala b/src/main/scala/catalyst/expressions/Cast.scala similarity index 100% rename from src/main/scala/expressions/Cast.scala rename to src/main/scala/catalyst/expressions/Cast.scala diff --git a/src/main/scala/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala similarity index 100% rename from src/main/scala/expressions/Evaluate.scala rename to src/main/scala/catalyst/expressions/Evaluate.scala diff --git a/src/main/scala/expressions/Expression.scala b/src/main/scala/catalyst/expressions/Expression.scala similarity index 100% rename from src/main/scala/expressions/Expression.scala rename to src/main/scala/catalyst/expressions/Expression.scala diff --git a/src/main/scala/expressions/Rand.scala b/src/main/scala/catalyst/expressions/Rand.scala similarity index 100% rename from src/main/scala/expressions/Rand.scala rename to src/main/scala/catalyst/expressions/Rand.scala diff --git a/src/main/scala/expressions/Row.scala b/src/main/scala/catalyst/expressions/Row.scala similarity index 100% rename from src/main/scala/expressions/Row.scala rename to src/main/scala/catalyst/expressions/Row.scala diff --git a/src/main/scala/expressions/ScalaUdf.scala b/src/main/scala/catalyst/expressions/ScalaUdf.scala similarity index 100% rename from src/main/scala/expressions/ScalaUdf.scala rename to src/main/scala/catalyst/expressions/ScalaUdf.scala diff --git a/src/main/scala/expressions/SortOrder.scala b/src/main/scala/catalyst/expressions/SortOrder.scala similarity index 100% rename from src/main/scala/expressions/SortOrder.scala rename to src/main/scala/catalyst/expressions/SortOrder.scala diff --git a/src/main/scala/expressions/WrapDynamic.scala b/src/main/scala/catalyst/expressions/WrapDynamic.scala similarity index 100% rename from src/main/scala/expressions/WrapDynamic.scala rename to src/main/scala/catalyst/expressions/WrapDynamic.scala diff --git a/src/main/scala/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala similarity index 100% rename from src/main/scala/expressions/aggregates.scala rename to src/main/scala/catalyst/expressions/aggregates.scala diff --git a/src/main/scala/expressions/arithmetic.scala b/src/main/scala/catalyst/expressions/arithmetic.scala similarity index 100% rename from src/main/scala/expressions/arithmetic.scala rename to src/main/scala/catalyst/expressions/arithmetic.scala diff --git a/src/main/scala/expressions/functions.scala b/src/main/scala/catalyst/expressions/functions.scala similarity index 100% rename from src/main/scala/expressions/functions.scala rename to src/main/scala/catalyst/expressions/functions.scala diff --git a/src/main/scala/expressions/literals.scala b/src/main/scala/catalyst/expressions/literals.scala similarity index 100% rename from src/main/scala/expressions/literals.scala rename to src/main/scala/catalyst/expressions/literals.scala diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/catalyst/expressions/namedExpressions.scala similarity index 100% rename from src/main/scala/expressions/namedExpressions.scala rename to src/main/scala/catalyst/expressions/namedExpressions.scala diff --git a/src/main/scala/expressions/nullFunctions.scala b/src/main/scala/catalyst/expressions/nullFunctions.scala similarity index 100% rename from src/main/scala/expressions/nullFunctions.scala rename to src/main/scala/catalyst/expressions/nullFunctions.scala diff --git a/src/main/scala/expressions/package.scala b/src/main/scala/catalyst/expressions/package.scala similarity index 100% rename from src/main/scala/expressions/package.scala rename to src/main/scala/catalyst/expressions/package.scala diff --git a/src/main/scala/expressions/predicates.scala b/src/main/scala/catalyst/expressions/predicates.scala similarity index 100% rename from src/main/scala/expressions/predicates.scala rename to src/main/scala/catalyst/expressions/predicates.scala diff --git a/src/main/scala/expressions/stringOperations.scala b/src/main/scala/catalyst/expressions/stringOperations.scala similarity index 100% rename from src/main/scala/expressions/stringOperations.scala rename to src/main/scala/catalyst/expressions/stringOperations.scala diff --git a/src/main/scala/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala similarity index 100% rename from src/main/scala/frontend/Hive.scala rename to src/main/scala/catalyst/frontend/Hive.scala diff --git a/src/main/scala/frontend/package.scala b/src/main/scala/catalyst/frontend/package.scala similarity index 100% rename from src/main/scala/frontend/package.scala rename to src/main/scala/catalyst/frontend/package.scala diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala similarity index 100% rename from src/main/scala/optimizer/Optimizer.scala rename to src/main/scala/catalyst/optimizer/Optimizer.scala diff --git a/src/main/scala/package.scala b/src/main/scala/catalyst/package.scala similarity index 100% rename from src/main/scala/package.scala rename to src/main/scala/catalyst/package.scala diff --git a/src/main/scala/planning/QueryPlanner.scala b/src/main/scala/catalyst/planning/QueryPlanner.scala similarity index 100% rename from src/main/scala/planning/QueryPlanner.scala rename to src/main/scala/catalyst/planning/QueryPlanner.scala diff --git a/src/main/scala/planning/package.scala b/src/main/scala/catalyst/planning/package.scala similarity index 100% rename from src/main/scala/planning/package.scala rename to src/main/scala/catalyst/planning/package.scala diff --git a/src/main/scala/planning/patterns.scala b/src/main/scala/catalyst/planning/patterns.scala similarity index 100% rename from src/main/scala/planning/patterns.scala rename to src/main/scala/catalyst/planning/patterns.scala diff --git a/src/main/scala/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala similarity index 100% rename from src/main/scala/plans/QueryPlan.scala rename to src/main/scala/catalyst/plans/QueryPlan.scala diff --git a/src/main/scala/plans/logical/BaseRelation.scala b/src/main/scala/catalyst/plans/logical/BaseRelation.scala similarity index 100% rename from src/main/scala/plans/logical/BaseRelation.scala rename to src/main/scala/catalyst/plans/logical/BaseRelation.scala diff --git a/src/main/scala/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala similarity index 100% rename from src/main/scala/plans/logical/LogicalPlan.scala rename to src/main/scala/catalyst/plans/logical/LogicalPlan.scala diff --git a/src/main/scala/plans/logical/TestRelation.scala b/src/main/scala/catalyst/plans/logical/TestRelation.scala similarity index 100% rename from src/main/scala/plans/logical/TestRelation.scala rename to src/main/scala/catalyst/plans/logical/TestRelation.scala diff --git a/src/main/scala/plans/logical/Transform.scala b/src/main/scala/catalyst/plans/logical/Transform.scala similarity index 100% rename from src/main/scala/plans/logical/Transform.scala rename to src/main/scala/catalyst/plans/logical/Transform.scala diff --git a/src/main/scala/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala similarity index 100% rename from src/main/scala/plans/logical/basicOperators.scala rename to src/main/scala/catalyst/plans/logical/basicOperators.scala diff --git a/src/main/scala/plans/package.scala b/src/main/scala/catalyst/plans/package.scala similarity index 100% rename from src/main/scala/plans/package.scala rename to src/main/scala/catalyst/plans/package.scala diff --git a/src/main/scala/plans/shared.scala b/src/main/scala/catalyst/plans/shared.scala similarity index 100% rename from src/main/scala/plans/shared.scala rename to src/main/scala/catalyst/plans/shared.scala diff --git a/src/main/scala/rules/Rule.scala b/src/main/scala/catalyst/rules/Rule.scala similarity index 100% rename from src/main/scala/rules/Rule.scala rename to src/main/scala/catalyst/rules/Rule.scala diff --git a/src/main/scala/rules/RuleExecutor.scala b/src/main/scala/catalyst/rules/RuleExecutor.scala similarity index 100% rename from src/main/scala/rules/RuleExecutor.scala rename to src/main/scala/catalyst/rules/RuleExecutor.scala diff --git a/src/main/scala/rules/package.scala b/src/main/scala/catalyst/rules/package.scala similarity index 100% rename from src/main/scala/rules/package.scala rename to src/main/scala/catalyst/rules/package.scala diff --git a/src/main/scala/shark2/FunctionRegistry.scala b/src/main/scala/catalyst/shark2/FunctionRegistry.scala similarity index 100% rename from src/main/scala/shark2/FunctionRegistry.scala rename to src/main/scala/catalyst/shark2/FunctionRegistry.scala diff --git a/src/main/scala/shark2/MetastoreCatalog.scala b/src/main/scala/catalyst/shark2/MetastoreCatalog.scala similarity index 100% rename from src/main/scala/shark2/MetastoreCatalog.scala rename to src/main/scala/catalyst/shark2/MetastoreCatalog.scala diff --git a/src/main/scala/shark2/SharkInstance.scala b/src/main/scala/catalyst/shark2/SharkInstance.scala similarity index 78% rename from src/main/scala/shark2/SharkInstance.scala rename to src/main/scala/catalyst/shark2/SharkInstance.scala index 78faba5170c11..871d470210acc 100644 --- a/src/main/scala/shark2/SharkInstance.scala +++ b/src/main/scala/catalyst/shark2/SharkInstance.scala @@ -3,7 +3,6 @@ package shark2 import java.io.File -import org.apache.spark.rdd.RDD import shark.{SharkConfVars, SharkContext, SharkEnv} import analysis.{SimpleAnalyzer, Analyzer} @@ -14,18 +13,18 @@ import plans.logical.LogicalPlan import rules.RuleExecutor /** - * Starts up an instance of shark where metadata is stored locally. An in-process medata data is created with data - * stored in ./metadata. Warehouse data is stored in in ./warehouse. + * Starts up an instance of shark where metadata is stored locally. An in-process metadata data is + * created with data stored in ./metadata. Warehouse data is stored in in ./warehouse. */ class LocalSharkInstance(val master: String) extends SharkInstance { - def warehousePath = new File("warehouse").getCanonicalPath - def metastorePath = new File("metastore").getCanonicalPath + override def warehousePath = new File("warehouse").getCanonicalPath + override def metastorePath = new File("metastore").getCanonicalPath } /** - * An instance of the shark execution engine. This class is responsible for taking queries expressed either in SQl or - * as raw catalyst logical plans and optimizing them for execution using Spark. Additionally this class maintains - * the connection with the hive metadata store. + * An instance of the shark execution engine. This class is responsible for taking queries + * expressed either in SQl or as raw catalyst logical plans and optimizing them for execution + * using Spark. Additionally this class maintains the connection with the hive metadata store. */ abstract class SharkInstance extends Logging { self => @@ -47,10 +46,10 @@ abstract class SharkInstance extends Logging { /** Sets up the system initially or after a RESET command */ protected def configure() { // Use hive natively for queries that won't be executed by catalyst. This is because - // shark has dependencies on a custom version of hive that we are trying to avoid - // in catalyst. + // shark has dependencies on a custom version of hive that we are trying to avoid in catalyst. SharkConfVars.setVar(SharkContext.hiveconf, SharkConfVars.EXEC_MODE, "hive") + // TODO: refactor this so we can work with other databases. runSqlHive("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + metastorePath + ";create=true") runSqlHive("set hive.metastore.warehouse.dir=" + warehousePath) } @@ -59,6 +58,7 @@ abstract class SharkInstance extends Logging { /* A catalyst metadata catalog that points to the Shark/Hive Metastore. */ val catalog = new HiveMetastoreCatalog(SharkContext.hiveconf) + /* An analyzer that uses the Shark/Hive metastore. */ val analyze = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) @@ -69,7 +69,7 @@ abstract class SharkInstance extends Logging { val maxResults = 100000 val results = sc.sql(sql, 100000) // It is very confusing when you only get back some of the results... - if(results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") + if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") results } @@ -86,9 +86,7 @@ abstract class SharkInstance extends Logging { } object PrepareForExecution extends RuleExecutor[SharkPlan] { - val batches = - Batch("Prepare Expressions", Once, - expressions.BindReferences) :: Nil + val batches = Batch("Prepare Expressions", Once, expressions.BindReferences) :: Nil } class SharkSqlQuery(sql: String) extends SharkQuery { @@ -98,8 +96,8 @@ abstract class SharkInstance extends Logging { } /** - * The primary workflow for executing queries using Shark. Designed to allow easy access to the intermediate phases - * of query execution. + * The primary workflow for executing queries using Shark. Designed to allow easy access to the + * intermediate phases of query execution. */ abstract class SharkQuery { def parsed: LogicalPlan @@ -108,7 +106,7 @@ abstract class SharkInstance extends Logging { lazy val optimizedPlan = Optimize(catalog.CreateTables(analyzed)) // TODO: Don't just pick the first one... lazy val sharkPlan = TrivalPlanner(optimizedPlan).next() - lazy val executedPlan = PrepareForExecution(sharkPlan) + lazy val executedPlan: SharkPlan = PrepareForExecution(sharkPlan) lazy val toRdd = executedPlan.execute() @@ -120,15 +118,15 @@ abstract class SharkInstance extends Logging { } /** - * Returns the result as a hive compatible sequence of strings. For native commands, the execution is simply - * passed back to Hive. + * Returns the result as a hive compatible sequence of strings. For native commands, the + * execution is simply passed back to Hive. */ def stringResult(): Seq[String] = analyzed match { case NativeCommand(cmd) => runSqlHive(cmd) case ConfigurationAssignment(cmd) => runSqlHive(cmd) case ExplainCommand(plan) => new SharkQuery { val parsed = plan }.toString.split("\n") case query => - val result: Seq[Seq[Any]] = toRdd.collect.toSeq + val result: Seq[Seq[Any]] = toRdd.collect().toSeq // Reformat to match hive tab delimited output. val asString = result.map(_.map(toHiveString)).map(_.mkString("\t")).toSeq asString @@ -146,8 +144,8 @@ abstract class SharkInstance extends Logging { } /** - * A shark query workflow for plans where all relations have already been resolved (likely because the query was - * built from raw RDDs). Additionally attribute resolution is case sensitive. + * A shark query workflow for plans where all relations have already been resolved (likely because + * the query was built from raw RDDs). Additionally attribute resolution is case sensitive. */ abstract class LogicalSharkQuery extends SharkQuery { override lazy val analyzed = SimpleAnalyzer(parsed) diff --git a/src/main/scala/shark2/SharkPlan.scala b/src/main/scala/catalyst/shark2/SharkPlan.scala similarity index 100% rename from src/main/scala/shark2/SharkPlan.scala rename to src/main/scala/catalyst/shark2/SharkPlan.scala diff --git a/src/main/scala/shark2/TestShark.scala b/src/main/scala/catalyst/shark2/TestShark.scala similarity index 100% rename from src/main/scala/shark2/TestShark.scala rename to src/main/scala/catalyst/shark2/TestShark.scala diff --git a/src/main/scala/shark2/Transform.scala b/src/main/scala/catalyst/shark2/Transform.scala similarity index 100% rename from src/main/scala/shark2/Transform.scala rename to src/main/scala/catalyst/shark2/Transform.scala diff --git a/src/main/scala/shark2/aggregates.scala b/src/main/scala/catalyst/shark2/aggregates.scala similarity index 100% rename from src/main/scala/shark2/aggregates.scala rename to src/main/scala/catalyst/shark2/aggregates.scala diff --git a/src/main/scala/shark2/basicOperators.scala b/src/main/scala/catalyst/shark2/basicOperators.scala similarity index 100% rename from src/main/scala/shark2/basicOperators.scala rename to src/main/scala/catalyst/shark2/basicOperators.scala diff --git a/src/main/scala/shark2/debug.scala b/src/main/scala/catalyst/shark2/debug.scala similarity index 100% rename from src/main/scala/shark2/debug.scala rename to src/main/scala/catalyst/shark2/debug.scala diff --git a/src/main/scala/shark2/hiveOperators.scala b/src/main/scala/catalyst/shark2/hiveOperators.scala similarity index 100% rename from src/main/scala/shark2/hiveOperators.scala rename to src/main/scala/catalyst/shark2/hiveOperators.scala diff --git a/src/main/scala/shark2/joins.scala b/src/main/scala/catalyst/shark2/joins.scala similarity index 100% rename from src/main/scala/shark2/joins.scala rename to src/main/scala/catalyst/shark2/joins.scala diff --git a/src/main/scala/shark2/package.scala b/src/main/scala/catalyst/shark2/package.scala similarity index 100% rename from src/main/scala/shark2/package.scala rename to src/main/scala/catalyst/shark2/package.scala diff --git a/src/main/scala/shark2/planningStrategies.scala b/src/main/scala/catalyst/shark2/planningStrategies.scala similarity index 100% rename from src/main/scala/shark2/planningStrategies.scala rename to src/main/scala/catalyst/shark2/planningStrategies.scala diff --git a/src/main/scala/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala similarity index 100% rename from src/main/scala/trees/TreeNode.scala rename to src/main/scala/catalyst/trees/TreeNode.scala diff --git a/src/main/scala/trees/package.scala b/src/main/scala/catalyst/trees/package.scala similarity index 100% rename from src/main/scala/trees/package.scala rename to src/main/scala/catalyst/trees/package.scala diff --git a/src/main/scala/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala similarity index 100% rename from src/main/scala/types/dataTypes.scala rename to src/main/scala/catalyst/types/dataTypes.scala diff --git a/src/main/scala/types/package.scala b/src/main/scala/catalyst/types/package.scala similarity index 100% rename from src/main/scala/types/package.scala rename to src/main/scala/catalyst/types/package.scala diff --git a/src/main/scala/util/package.scala b/src/main/scala/catalyst/util/package.scala similarity index 100% rename from src/main/scala/util/package.scala rename to src/main/scala/catalyst/util/package.scala From c6527f5a05c49290ac6b0044294322f0ba4f8ff6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 2 Jan 2014 19:01:36 -0800 Subject: [PATCH 364/778] Moved the test src files into the catalyst directory. --- src/test/scala/{ => catalyst}/AnalysisSuite.scala | 0 src/test/scala/{ => catalyst}/ExpressionEvaluationSuite.scala | 0 src/test/scala/{ => catalyst}/RuleExecutorSuite.scala | 0 src/test/scala/{ => catalyst}/TreeNodeSuite.scala | 0 src/test/scala/{ => catalyst}/shark2/BigDataBenchmark.scala | 0 src/test/scala/{ => catalyst}/shark2/DslQueryTests.scala | 0 src/test/scala/{ => catalyst}/shark2/HiveAstTransformSuite.scala | 0 src/test/scala/{ => catalyst}/shark2/HiveComparisionTest.scala | 0 src/test/scala/{ => catalyst}/shark2/HiveCompatability.scala | 0 src/test/scala/{ => catalyst}/shark2/HiveQueryFileTest.scala | 0 src/test/scala/{ => catalyst}/shark2/HiveQueryTests.scala | 0 src/test/scala/{ => catalyst}/shark2/HiveResolutionSuite.scala | 0 src/test/scala/{ => catalyst}/shark2/HiveTypeCoersionSuite.scala | 0 13 files changed, 0 insertions(+), 0 deletions(-) rename src/test/scala/{ => catalyst}/AnalysisSuite.scala (100%) rename src/test/scala/{ => catalyst}/ExpressionEvaluationSuite.scala (100%) rename src/test/scala/{ => catalyst}/RuleExecutorSuite.scala (100%) rename src/test/scala/{ => catalyst}/TreeNodeSuite.scala (100%) rename src/test/scala/{ => catalyst}/shark2/BigDataBenchmark.scala (100%) rename src/test/scala/{ => catalyst}/shark2/DslQueryTests.scala (100%) rename src/test/scala/{ => catalyst}/shark2/HiveAstTransformSuite.scala (100%) rename src/test/scala/{ => catalyst}/shark2/HiveComparisionTest.scala (100%) rename src/test/scala/{ => catalyst}/shark2/HiveCompatability.scala (100%) rename src/test/scala/{ => catalyst}/shark2/HiveQueryFileTest.scala (100%) rename src/test/scala/{ => catalyst}/shark2/HiveQueryTests.scala (100%) rename src/test/scala/{ => catalyst}/shark2/HiveResolutionSuite.scala (100%) rename src/test/scala/{ => catalyst}/shark2/HiveTypeCoersionSuite.scala (100%) diff --git a/src/test/scala/AnalysisSuite.scala b/src/test/scala/catalyst/AnalysisSuite.scala similarity index 100% rename from src/test/scala/AnalysisSuite.scala rename to src/test/scala/catalyst/AnalysisSuite.scala diff --git a/src/test/scala/ExpressionEvaluationSuite.scala b/src/test/scala/catalyst/ExpressionEvaluationSuite.scala similarity index 100% rename from src/test/scala/ExpressionEvaluationSuite.scala rename to src/test/scala/catalyst/ExpressionEvaluationSuite.scala diff --git a/src/test/scala/RuleExecutorSuite.scala b/src/test/scala/catalyst/RuleExecutorSuite.scala similarity index 100% rename from src/test/scala/RuleExecutorSuite.scala rename to src/test/scala/catalyst/RuleExecutorSuite.scala diff --git a/src/test/scala/TreeNodeSuite.scala b/src/test/scala/catalyst/TreeNodeSuite.scala similarity index 100% rename from src/test/scala/TreeNodeSuite.scala rename to src/test/scala/catalyst/TreeNodeSuite.scala diff --git a/src/test/scala/shark2/BigDataBenchmark.scala b/src/test/scala/catalyst/shark2/BigDataBenchmark.scala similarity index 100% rename from src/test/scala/shark2/BigDataBenchmark.scala rename to src/test/scala/catalyst/shark2/BigDataBenchmark.scala diff --git a/src/test/scala/shark2/DslQueryTests.scala b/src/test/scala/catalyst/shark2/DslQueryTests.scala similarity index 100% rename from src/test/scala/shark2/DslQueryTests.scala rename to src/test/scala/catalyst/shark2/DslQueryTests.scala diff --git a/src/test/scala/shark2/HiveAstTransformSuite.scala b/src/test/scala/catalyst/shark2/HiveAstTransformSuite.scala similarity index 100% rename from src/test/scala/shark2/HiveAstTransformSuite.scala rename to src/test/scala/catalyst/shark2/HiveAstTransformSuite.scala diff --git a/src/test/scala/shark2/HiveComparisionTest.scala b/src/test/scala/catalyst/shark2/HiveComparisionTest.scala similarity index 100% rename from src/test/scala/shark2/HiveComparisionTest.scala rename to src/test/scala/catalyst/shark2/HiveComparisionTest.scala diff --git a/src/test/scala/shark2/HiveCompatability.scala b/src/test/scala/catalyst/shark2/HiveCompatability.scala similarity index 100% rename from src/test/scala/shark2/HiveCompatability.scala rename to src/test/scala/catalyst/shark2/HiveCompatability.scala diff --git a/src/test/scala/shark2/HiveQueryFileTest.scala b/src/test/scala/catalyst/shark2/HiveQueryFileTest.scala similarity index 100% rename from src/test/scala/shark2/HiveQueryFileTest.scala rename to src/test/scala/catalyst/shark2/HiveQueryFileTest.scala diff --git a/src/test/scala/shark2/HiveQueryTests.scala b/src/test/scala/catalyst/shark2/HiveQueryTests.scala similarity index 100% rename from src/test/scala/shark2/HiveQueryTests.scala rename to src/test/scala/catalyst/shark2/HiveQueryTests.scala diff --git a/src/test/scala/shark2/HiveResolutionSuite.scala b/src/test/scala/catalyst/shark2/HiveResolutionSuite.scala similarity index 100% rename from src/test/scala/shark2/HiveResolutionSuite.scala rename to src/test/scala/catalyst/shark2/HiveResolutionSuite.scala diff --git a/src/test/scala/shark2/HiveTypeCoersionSuite.scala b/src/test/scala/catalyst/shark2/HiveTypeCoersionSuite.scala similarity index 100% rename from src/test/scala/shark2/HiveTypeCoersionSuite.scala rename to src/test/scala/catalyst/shark2/HiveTypeCoersionSuite.scala From 3f9fee1fccdcd60dcd9c6da83c565e28a525bf25 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 2 Jan 2014 19:49:44 -0800 Subject: [PATCH 365/778] rewrite push filter through join optimization. --- src/main/scala/optimizer/Optimizer.scala | 154 ++++------------------- 1 file changed, 22 insertions(+), 132 deletions(-) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index f96be8df483dc..d4a693765af41 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -3,6 +3,7 @@ package optimizer import catalyst.expressions._ import catalyst.expressions.Alias +import catalyst.plans.Inner import catalyst.plans.logical._ import catalyst.plans.logical.Filter import catalyst.plans.logical.Project @@ -16,8 +17,7 @@ object Optimize extends RuleExecutor[LogicalPlan] { EliminateSubqueries, CombineFilters, PredicatePushDownThoughProject, - PushPredicateThroughInnerEqualJoin) :: Nil - + PushPredicateThroughInnerJoin) :: Nil } object EliminateSubqueries extends Rule[LogicalPlan] { @@ -55,138 +55,28 @@ object PredicatePushDownThoughProject extends Rule[LogicalPlan] { } } -object EmptyExpression extends EmptyExpressionType(null, null) - -case class EmptyExpressionType(left: Expression, right: Expression) extends Expression { - self: Product => - - /** Returns a Seq of the children of this node */ - - def children: Seq[Expression] = Seq.empty - - def dataType: DataType = NullType - - def nullable: Boolean = true - - def references: Set[Attribute] = Set.empty -} - /** - * Pushes predicate through inner join + * Pushes down predicates that can be evaluated using only the attributes of the left or right side + * of a join. Other predicates are left as a single filter on top of the join. */ -object PushPredicateThroughInnerEqualJoin extends Rule[LogicalPlan] { - class BinaryConditionStatus { - var left = false - var right = false - } - - abstract class Direction - case object Left extends Direction - case object Right extends Direction - +object PushPredicateThroughInnerJoin extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case filter@Filter(_, join@Join(l, r, _, _)) => { - var joinLeftExpression: Option[Expression] = None - var joinRightExpression: Option[Expression] = None - - val conditions = filter.splitConjunctivePredicates(filter.condition).filter( - e => { - var changed = false - if(e.references subsetOf l.outputSet) { - joinLeftExpression = combineExpression(joinLeftExpression, Option(e), And) - changed = true - } - - if(e.references subsetOf r.outputSet) { - joinRightExpression = combineExpression(joinRightExpression, Option(e), And) - changed = true - } - - changed - } - ).toSet - - val newFilterCondition = rewriteCondition (filter.condition, conditions) - - (newFilterCondition, joinLeftExpression, joinRightExpression) match { - case (EmptyExpression, _, _) => - copyJoin(join, joinLeftExpression, joinRightExpression) - case (_, None, None) => filter - case _ => { - filter.copy(condition = newFilterCondition, - child = copyJoin( - join, joinLeftExpression, joinRightExpression - ) - ) - } - } - } - } - - def rewriteCondition(exp: Expression, removedExps: Set[Expression]): Expression = { - exp match { - case b: And => b match { - case And(e1: Expression, e2:Expression) => { - val r1 = rewriteCondition(e1, removedExps) - val r2 = rewriteCondition(e2, removedExps) - - (r1, r2) match { - case (EmptyExpression, e: Expression) => e - case (e: Expression, EmptyExpression) => e - case (EmptyExpression, EmptyExpression) => EmptyExpression - case _ => And(r1, r2) - } - } - } - case e => { - if(removedExps.contains(e)) - EmptyExpression - else - e - } - } - } - - def combineExpression[T1, T2](firstExp: Option[T1], - secondExp: Option[T2], - combineFunc: (T1, T2) => T2): Option[T2] = { - (firstExp, secondExp) match { - case (Some(o1), Some(o2)) => Option(combineFunc(o1, o2)) - case (None, Some(o2)) => Option(o2) - case (Some(o1), None) => Option(o1.asInstanceOf[T2]) - case (None, None) => None - } - } - - def copyJoin(join: Join, - joinLeftExpression: Option[Expression], - joinRightExpression: Option[Expression]): Join = { - join.copy( - left = - combineExpression( - joinLeftExpression, - Option(join.left), - Filter - ).get, - right = - combineExpression( - joinRightExpression, - Option(join.right), - Filter - ).get - ) - } - - def joinFilter(status: BinaryConditionStatus, - dir: Direction, - expression: Expression, - outputSet: Set[Attribute]): Option[Expression] = { - if (expression.references subsetOf outputSet) { - dir match { - case Left => status.left = true - case Right => status.right = true - } - Option(expression) - } else None + case f @ Filter(filterCondition, Join(left, right, Inner, joinCondition)) => + val allConditions = + splitConjunctivePredicates(filterCondition) ++ + joinCondition.map(splitConjunctivePredicates).getOrElse(Nil) + + // Split the predicates into those that can be evaluated on the left, right, and those that + // must be evaluated after the join. + val (rightConditions, leftOrJoinConditions) = + allConditions.partition(_.references subsetOf right.outputSet) + val (leftConditions, joinConditions) = + leftOrJoinConditions.partition(_.references subsetOf left.outputSet) + + // Build the new left and right side, optionally with the pushed down filters. + val newLeft = leftConditions.reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) + val newRight = rightConditions.reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) + val newJoin = Join(newLeft, newRight, Inner, None) + joinConditions.reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin) } } \ No newline at end of file From 72426ed578e7b8b96227da9a10503a44707685f0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 2 Jan 2014 21:41:45 -0800 Subject: [PATCH 366/778] Rename shark2 package to execution. --- README.md | 6 ++-- build.sbt | 2 +- .../catalyst/examples/SchemaRddExample.scala | 2 +- .../catalyst/examples/ViewsExample.scala | 2 +- .../FunctionRegistry.scala | 2 +- .../MetastoreCatalog.scala | 2 +- .../{shark2 => execution}/SharkInstance.scala | 4 +-- .../{shark2 => execution}/SharkPlan.scala | 2 +- .../{shark2 => execution}/TestShark.scala | 4 +-- .../{shark2 => execution}/Transform.scala | 2 +- .../{shark2 => execution}/aggregates.scala | 2 +- .../basicOperators.scala | 2 +- .../{shark2 => execution}/debug.scala | 2 +- .../{shark2 => execution}/hiveOperators.scala | 2 +- .../{shark2 => execution}/joins.scala | 2 +- .../{shark2 => execution}/package.scala | 2 +- .../planningStrategies.scala | 36 +++++++++---------- .../catalyst/expressions/BoundAttribute.scala | 2 +- .../BigDataBenchmark.scala | 2 +- .../{shark2 => execution}/DslQueryTests.scala | 2 +- .../HiveAstTransformSuite.scala | 0 .../HiveComparisionTest.scala | 2 +- .../HiveCompatability.scala | 2 +- .../HiveQueryFileTest.scala | 2 +- .../HiveQueryTests.scala | 2 +- .../HiveResolutionSuite.scala | 2 +- .../HiveTypeCoersionSuite.scala | 2 +- 27 files changed, 47 insertions(+), 47 deletions(-) rename src/main/scala/catalyst/{shark2 => execution}/FunctionRegistry.scala (99%) rename src/main/scala/catalyst/{shark2 => execution}/MetastoreCatalog.scala (99%) rename src/main/scala/catalyst/{shark2 => execution}/SharkInstance.scala (98%) rename src/main/scala/catalyst/{shark2 => execution}/SharkPlan.scala (97%) rename src/main/scala/catalyst/{shark2 => execution}/TestShark.scala (99%) rename src/main/scala/catalyst/{shark2 => execution}/Transform.scala (98%) rename src/main/scala/catalyst/{shark2 => execution}/aggregates.scala (99%) rename src/main/scala/catalyst/{shark2 => execution}/basicOperators.scala (99%) rename src/main/scala/catalyst/{shark2 => execution}/debug.scala (97%) rename src/main/scala/catalyst/{shark2 => execution}/hiveOperators.scala (99%) rename src/main/scala/catalyst/{shark2 => execution}/joins.scala (99%) rename src/main/scala/catalyst/{shark2 => execution}/package.scala (97%) rename src/main/scala/catalyst/{shark2 => execution}/planningStrategies.scala (77%) rename src/test/scala/catalyst/{shark2 => execution}/BigDataBenchmark.scala (99%) rename src/test/scala/catalyst/{shark2 => execution}/DslQueryTests.scala (99%) rename src/test/scala/catalyst/{shark2 => execution}/HiveAstTransformSuite.scala (100%) rename src/test/scala/catalyst/{shark2 => execution}/HiveComparisionTest.scala (99%) rename src/test/scala/catalyst/{shark2 => execution}/HiveCompatability.scala (99%) rename src/test/scala/catalyst/{shark2 => execution}/HiveQueryFileTest.scala (98%) rename src/test/scala/catalyst/{shark2 => execution}/HiveQueryTests.scala (99%) rename src/test/scala/catalyst/{shark2 => execution}/HiveResolutionSuite.scala (98%) rename src/test/scala/catalyst/{shark2 => execution}/HiveTypeCoersionSuite.scala (95%) diff --git a/README.md b/README.md index 004b229bb96fe..fb7b0d5aeef6e 100644 --- a/README.md +++ b/README.md @@ -22,7 +22,7 @@ $ sbt/sbt > test:compile [info] Compiling 10 Scala sources to catalyst/target/scala-2.10/test-classes... [success] Total time: 15 s, completed Dec 20, 2013 12:00:06 PM -> test-only catalyst.shark2.DslQueryTests +> test-only catalyst.execution.DslQueryTests ``` Any command that is prefixed with a `~` (e.g. `~compile`) will be run automatically in a loop each time any dependent files have changed. @@ -54,13 +54,13 @@ import catalyst.plans.logical._ import catalyst.rules._ import catalyst.types._ import catalyst.util._ -import catalyst.shark2.TestShark._ +import catalyst.execution.TestShark._ Welcome to Scala version 2.10.3 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45). Type in expressions to have them evaluated. Type :help for more information. scala> val query = "SELECT * FROM (SELECT * FROM src) a".q -query: catalyst.shark2.TestShark.SharkSqlQuery = +query: catalyst.execution.TestShark.SharkSqlQuery = SELECT * FROM (SELECT * FROM src) a == Logical Plan == Project {key#0,value#1} diff --git a/build.sbt b/build.sbt index 20183c04d0e31..4a180507d74e1 100644 --- a/build.sbt +++ b/build.sbt @@ -45,7 +45,7 @@ import catalyst.plans.logical._ import catalyst.rules._ import catalyst.types._ import catalyst.util._ -import catalyst.shark2.TestShark._""" +import catalyst.execution.TestShark._""" site.settings diff --git a/src/main/scala/catalyst/examples/SchemaRddExample.scala b/src/main/scala/catalyst/examples/SchemaRddExample.scala index 89a64bd81b11f..f89aae50925bb 100644 --- a/src/main/scala/catalyst/examples/SchemaRddExample.scala +++ b/src/main/scala/catalyst/examples/SchemaRddExample.scala @@ -3,7 +3,7 @@ package examples import plans.logical.LocalRelation -import shark2.TestShark._ +import execution.TestShark._ import dsl._ object SchemaRddExample { diff --git a/src/main/scala/catalyst/examples/ViewsExample.scala b/src/main/scala/catalyst/examples/ViewsExample.scala index a6049146eb692..7eee3b868f1f8 100644 --- a/src/main/scala/catalyst/examples/ViewsExample.scala +++ b/src/main/scala/catalyst/examples/ViewsExample.scala @@ -6,7 +6,7 @@ import catalyst.plans.logical._ /* Implicit Conversions */ import dsl._ -import shark2.TestShark._ // For .toRdd execution using locally running test Shark. +import execution.TestShark._ // For .toRdd execution using locally running test Shark. object ViewsExample { def main(args: Array[String]): Unit = { diff --git a/src/main/scala/catalyst/shark2/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala similarity index 99% rename from src/main/scala/catalyst/shark2/FunctionRegistry.scala rename to src/main/scala/catalyst/execution/FunctionRegistry.scala index cc35674a29d4e..691fb0abc06d4 100644 --- a/src/main/scala/catalyst/shark2/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} diff --git a/src/main/scala/catalyst/shark2/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala similarity index 99% rename from src/main/scala/catalyst/shark2/MetastoreCatalog.scala rename to src/main/scala/catalyst/execution/MetastoreCatalog.scala index 6dcd2b9d696e9..06b62687e88d6 100644 --- a/src/main/scala/catalyst/shark2/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table, StorageDescriptor, SerDeInfo} diff --git a/src/main/scala/catalyst/shark2/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala similarity index 98% rename from src/main/scala/catalyst/shark2/SharkInstance.scala rename to src/main/scala/catalyst/execution/SharkInstance.scala index 871d470210acc..edd7eb93b4431 100644 --- a/src/main/scala/catalyst/shark2/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import java.io.File @@ -40,7 +40,7 @@ abstract class SharkInstance extends Logging { lazy val sc = createContext() protected def createContext() = { - SharkEnv.initWithSharkContext("catalyst.shark2", master) + SharkEnv.initWithSharkContext("catalyst.execution", master) } /** Sets up the system initially or after a RESET command */ diff --git a/src/main/scala/catalyst/shark2/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala similarity index 97% rename from src/main/scala/catalyst/shark2/SharkPlan.scala rename to src/main/scala/catalyst/execution/SharkPlan.scala index 51b706ead1df0..519336f217fb1 100644 --- a/src/main/scala/catalyst/shark2/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import catalyst.plans.QueryPlan import org.apache.spark.rdd.RDD diff --git a/src/main/scala/catalyst/shark2/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala similarity index 99% rename from src/main/scala/catalyst/shark2/TestShark.scala rename to src/main/scala/catalyst/execution/TestShark.scala index d3d08f1345a5a..e500d4c79c080 100644 --- a/src/main/scala/catalyst/shark2/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import java.io.File import scala.collection.mutable @@ -56,7 +56,7 @@ object TestShark extends SharkInstance { System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") - SharkEnv.initWithSharkContext("catalyst.shark2.TestShark", master) + SharkEnv.initWithSharkContext("catalyst.execution.TestShark", master) } /** The location of the compiled hive distribution */ diff --git a/src/main/scala/catalyst/shark2/Transform.scala b/src/main/scala/catalyst/execution/Transform.scala similarity index 98% rename from src/main/scala/catalyst/shark2/Transform.scala rename to src/main/scala/catalyst/execution/Transform.scala index d271e8146abeb..0178efbc8c9dd 100644 --- a/src/main/scala/catalyst/shark2/Transform.scala +++ b/src/main/scala/catalyst/execution/Transform.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import catalyst.expressions._ import shark.SharkContext diff --git a/src/main/scala/catalyst/shark2/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala similarity index 99% rename from src/main/scala/catalyst/shark2/aggregates.scala rename to src/main/scala/catalyst/execution/aggregates.scala index 33d6ce69b4a6d..dfdb6e68fee31 100644 --- a/src/main/scala/catalyst/shark2/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import catalyst.errors._ import catalyst.expressions._ diff --git a/src/main/scala/catalyst/shark2/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala similarity index 99% rename from src/main/scala/catalyst/shark2/basicOperators.scala rename to src/main/scala/catalyst/execution/basicOperators.scala index d61d5e1f2cc61..a4ea279240041 100644 --- a/src/main/scala/catalyst/shark2/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, StructObjectInspector} diff --git a/src/main/scala/catalyst/shark2/debug.scala b/src/main/scala/catalyst/execution/debug.scala similarity index 97% rename from src/main/scala/catalyst/shark2/debug.scala rename to src/main/scala/catalyst/execution/debug.scala index e0e91a840d66e..a1e262f3e3035 100644 --- a/src/main/scala/catalyst/shark2/debug.scala +++ b/src/main/scala/catalyst/execution/debug.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution object DebugQuery { def apply(plan: SharkPlan): SharkPlan = { diff --git a/src/main/scala/catalyst/shark2/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala similarity index 99% rename from src/main/scala/catalyst/shark2/hiveOperators.scala rename to src/main/scala/catalyst/execution/hiveOperators.scala index d39d7fc86ee43..fb9e94074dd43 100644 --- a/src/main/scala/catalyst/shark2/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils diff --git a/src/main/scala/catalyst/shark2/joins.scala b/src/main/scala/catalyst/execution/joins.scala similarity index 99% rename from src/main/scala/catalyst/shark2/joins.scala rename to src/main/scala/catalyst/execution/joins.scala index e5084b6c71284..706d52d1fea93 100644 --- a/src/main/scala/catalyst/shark2/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import org.apache.spark.rdd.RDD diff --git a/src/main/scala/catalyst/shark2/package.scala b/src/main/scala/catalyst/execution/package.scala similarity index 97% rename from src/main/scala/catalyst/shark2/package.scala rename to src/main/scala/catalyst/execution/package.scala index e0857cd74d4c5..6e366263d3511 100644 --- a/src/main/scala/catalyst/shark2/package.scala +++ b/src/main/scala/catalyst/execution/package.scala @@ -13,7 +13,7 @@ import types._ * Currently functions that are not supported by this implementation are passed back to the original Shark * implementation for execution. */ -package object shark2 { +package object execution { type Row = catalyst.expressions.Row implicit class typeInfoConversions(dt: DataType) { diff --git a/src/main/scala/catalyst/shark2/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala similarity index 77% rename from src/main/scala/catalyst/shark2/planningStrategies.scala rename to src/main/scala/catalyst/execution/planningStrategies.scala index c368eb90ef3a2..556503b6fbbdc 100644 --- a/src/main/scala/catalyst/shark2/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import shark.SharkContext @@ -25,9 +25,9 @@ abstract trait PlanningStrategies { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { // Push attributes into table scan when possible. case p @ logical.Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => - shark2.HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m) :: Nil + execution.HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m) :: Nil case m: MetastoreRelation => - shark2.HiveTableScan(m.output, m) :: Nil + execution.HiveTableScan(m.output, m) :: Nil case _ => Nil } @@ -59,7 +59,7 @@ abstract trait PlanningStrategies { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Aggregate(Nil, agg, child) if onlyAllowedAggregates(agg) => - shark2.SparkAggregate(agg, planLater(child))(sc) :: Nil + execution.SparkAggregate(agg, planLater(child))(sc) :: Nil case _ => Nil } } @@ -87,11 +87,11 @@ abstract trait PlanningStrategies { val leftKeys = joinKeys.map(_._1) val rightKeys = joinKeys.map(_._2) - val joinOp = shark2.SparkEquiInnerJoin(leftKeys, rightKeys, planLater(left), planLater(right)) + val joinOp = execution.SparkEquiInnerJoin(leftKeys, rightKeys, planLater(left), planLater(right)) // Make sure other conditions are met if present. if(otherPredicates.nonEmpty) - shark2.Filter(combineConjunctivePredicates(otherPredicates), joinOp) :: Nil + execution.Filter(combineConjunctivePredicates(otherPredicates), joinOp) :: Nil else joinOp :: Nil } else { @@ -112,17 +112,17 @@ abstract trait PlanningStrategies { object BroadcastNestedLoopJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Join(left, right, joinType, condition) => - shark2.BroadcastNestedLoopJoin(planLater(left), planLater(right), joinType, condition)(sc) :: Nil + execution.BroadcastNestedLoopJoin(planLater(left), planLater(right), joinType, condition)(sc) :: Nil case _ => Nil } } object CartesianProduct extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { - case logical.Join(left, right, _, None) => shark2.CartesianProduct(planLater(left), planLater(right)) :: Nil + case logical.Join(left, right, _, None) => execution.CartesianProduct(planLater(left), planLater(right)) :: Nil case logical.Join(left, right, Inner, Some(condition)) => - shark2.Filter(condition, - shark2.CartesianProduct(planLater(left), planLater(right))) :: Nil + execution.Filter(condition, + execution.CartesianProduct(planLater(left), planLater(right))) :: Nil case _ => Nil } } @@ -131,21 +131,21 @@ abstract trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - shark2.Sort(sortExprs, planLater(child)) :: Nil + execution.Sort(sortExprs, planLater(child)) :: Nil case logical.Project(projectList, child) => - shark2.Project(projectList, planLater(child)) :: Nil + execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => - shark2.Filter(condition, planLater(child)) :: Nil + execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - shark2.Aggregate(group, agg, planLater(child)) :: Nil + execution.Aggregate(group, agg, planLater(child)) :: Nil case logical.LocalRelation(output, data) => - shark2.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil + execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => - shark2.StopAfter(Evaluate(limit, Nil).asInstanceOf[Int], planLater(child))(sc) :: Nil + execution.StopAfter(Evaluate(limit, Nil).asInstanceOf[Int], planLater(child))(sc) :: Nil case logical.Union(left, right) => - shark2.Union(planLater(left), planLater(right))(sc) :: Nil + execution.Union(planLater(left), planLater(right))(sc) :: Nil case logical.Transform(input, script, output, child) => - shark2.Transform(input, script, output, planLater(child))(sc) :: Nil + execution.Transform(input, script, output, planLater(child))(sc) :: Nil case _ => Nil } } diff --git a/src/main/scala/catalyst/expressions/BoundAttribute.scala b/src/main/scala/catalyst/expressions/BoundAttribute.scala index f03c39141b1bd..fa2ef0475caae 100644 --- a/src/main/scala/catalyst/expressions/BoundAttribute.scala +++ b/src/main/scala/catalyst/expressions/BoundAttribute.scala @@ -4,7 +4,7 @@ package expressions import rules._ import errors._ -import shark2.SharkPlan +import execution.SharkPlan /** * A bound reference points to a specific slot in the input tuple, allowing the actual value to be retrieved more diff --git a/src/test/scala/catalyst/shark2/BigDataBenchmark.scala b/src/test/scala/catalyst/execution/BigDataBenchmark.scala similarity index 99% rename from src/test/scala/catalyst/shark2/BigDataBenchmark.scala rename to src/test/scala/catalyst/execution/BigDataBenchmark.scala index 20322169995c5..d5e05ba28bacc 100644 --- a/src/test/scala/catalyst/shark2/BigDataBenchmark.scala +++ b/src/test/scala/catalyst/execution/BigDataBenchmark.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import java.io.File diff --git a/src/test/scala/catalyst/shark2/DslQueryTests.scala b/src/test/scala/catalyst/execution/DslQueryTests.scala similarity index 99% rename from src/test/scala/catalyst/shark2/DslQueryTests.scala rename to src/test/scala/catalyst/execution/DslQueryTests.scala index 146745deca780..0e8463996ad54 100644 --- a/src/test/scala/catalyst/shark2/DslQueryTests.scala +++ b/src/test/scala/catalyst/execution/DslQueryTests.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import org.scalatest.{BeforeAndAfterAll, FunSuite} diff --git a/src/test/scala/catalyst/shark2/HiveAstTransformSuite.scala b/src/test/scala/catalyst/execution/HiveAstTransformSuite.scala similarity index 100% rename from src/test/scala/catalyst/shark2/HiveAstTransformSuite.scala rename to src/test/scala/catalyst/execution/HiveAstTransformSuite.scala diff --git a/src/test/scala/catalyst/shark2/HiveComparisionTest.scala b/src/test/scala/catalyst/execution/HiveComparisionTest.scala similarity index 99% rename from src/test/scala/catalyst/shark2/HiveComparisionTest.scala rename to src/test/scala/catalyst/execution/HiveComparisionTest.scala index 273a7207752e2..203a5c22b5919 100644 --- a/src/test/scala/catalyst/shark2/HiveComparisionTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisionTest.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import shark.{SharkContext, SharkEnv} diff --git a/src/test/scala/catalyst/shark2/HiveCompatability.scala b/src/test/scala/catalyst/execution/HiveCompatability.scala similarity index 99% rename from src/test/scala/catalyst/shark2/HiveCompatability.scala rename to src/test/scala/catalyst/execution/HiveCompatability.scala index 9c044592a9d3f..c72e655932ebc 100644 --- a/src/test/scala/catalyst/shark2/HiveCompatability.scala +++ b/src/test/scala/catalyst/execution/HiveCompatability.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import java.io._ diff --git a/src/test/scala/catalyst/shark2/HiveQueryFileTest.scala b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala similarity index 98% rename from src/test/scala/catalyst/shark2/HiveQueryFileTest.scala rename to src/test/scala/catalyst/execution/HiveQueryFileTest.scala index 2cc4832a46391..fa93af8daa3c6 100644 --- a/src/test/scala/catalyst/shark2/HiveQueryFileTest.scala +++ b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution import java.io._ diff --git a/src/test/scala/catalyst/shark2/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala similarity index 99% rename from src/test/scala/catalyst/shark2/HiveQueryTests.scala rename to src/test/scala/catalyst/execution/HiveQueryTests.scala index c2d68588d31d8..ae710ea939adc 100644 --- a/src/test/scala/catalyst/shark2/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. diff --git a/src/test/scala/catalyst/shark2/HiveResolutionSuite.scala b/src/test/scala/catalyst/execution/HiveResolutionSuite.scala similarity index 98% rename from src/test/scala/catalyst/shark2/HiveResolutionSuite.scala rename to src/test/scala/catalyst/execution/HiveResolutionSuite.scala index ce9f1aadb8771..13dfb951cb55a 100644 --- a/src/test/scala/catalyst/shark2/HiveResolutionSuite.scala +++ b/src/test/scala/catalyst/execution/HiveResolutionSuite.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. diff --git a/src/test/scala/catalyst/shark2/HiveTypeCoersionSuite.scala b/src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala similarity index 95% rename from src/test/scala/catalyst/shark2/HiveTypeCoersionSuite.scala rename to src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala index 0fb3d13bd20b7..095dce23aade9 100644 --- a/src/test/scala/catalyst/shark2/HiveTypeCoersionSuite.scala +++ b/src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala @@ -1,5 +1,5 @@ package catalyst -package shark2 +package execution /** * A set of tests that validate type promotion rules. From e36caf5a210578b49ae16353d983bf1ba0e7459c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 2 Jan 2014 23:28:05 -0800 Subject: [PATCH 367/778] Renamed Rule.name to Rule.ruleName since name is used too frequently in the code base and is shadowed often by local scope. --- src/main/scala/catalyst/rules/Rule.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/main/scala/catalyst/rules/Rule.scala b/src/main/scala/catalyst/rules/Rule.scala index c20eca0109d88..fb609fd527b77 100644 --- a/src/main/scala/catalyst/rules/Rule.scala +++ b/src/main/scala/catalyst/rules/Rule.scala @@ -4,13 +4,12 @@ package rules import trees._ abstract class Rule[TreeType <: TreeNode[_]] extends Logging { - val name = { + + /** Name for this rule, automatically inferred based on class name. */ + val ruleName: String = { val className = getClass.getName - if(className endsWith "$") - className.dropRight(1) - else - className + if (className endsWith "$") className.dropRight(1) else className } def apply(plan: TreeType): TreeType -} \ No newline at end of file +} From 479e0558f0e1ee294fd22bb8f1bc4ca3032001be Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 Jan 2014 01:02:33 -0800 Subject: [PATCH 368/778] A set of minor changes, including: - import order - limit some lines to 100 character wide - inline code comment - more scaladocs - minor spacing (i.e. add a space after if) --- .../scala/catalyst/analysis/Analyzer.scala | 41 ++++++---- .../scala/catalyst/analysis/Catalog.scala | 12 +-- .../catalyst/analysis/FunctionRegistry.scala | 15 +++- .../catalyst/analysis/typeCoercion.scala | 15 ++-- .../scala/catalyst/analysis/unresolved.scala | 6 +- .../catalyst/execution/FunctionRegistry.scala | 49 ++++++------ .../catalyst/execution/MetastoreCatalog.scala | 25 ++++--- .../scala/catalyst/execution/SharkPlan.scala | 8 +- .../scala/catalyst/expressions/Cast.scala | 3 +- .../catalyst/expressions/SortOrder.scala | 2 - .../catalyst/expressions/aggregates.scala | 7 +- .../catalyst/expressions/functions.scala | 2 +- .../expressions/namedExpressions.scala | 31 ++++---- .../catalyst/expressions/predicates.scala | 10 ++- src/main/scala/catalyst/frontend/Hive.scala | 13 ++-- src/main/scala/catalyst/plans/QueryPlan.scala | 4 +- .../catalyst/plans/logical/BaseRelation.scala | 3 +- .../catalyst/plans/logical/LogicalPlan.scala | 11 +-- .../catalyst/plans/logical/TestRelation.scala | 19 +++-- src/main/scala/catalyst/plans/package.scala | 5 +- .../scala/catalyst/rules/RuleExecutor.scala | 21 +++++- src/main/scala/catalyst/trees/TreeNode.scala | 74 ++++++++++--------- src/main/scala/catalyst/types/package.scala | 5 +- 23 files changed, 215 insertions(+), 166 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 335fcab3f83f5..490dcb3b29008 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -6,21 +6,28 @@ import plans.logical._ import rules._ /** - * A trivial [[Analyzer]] with an [[EmptyCatalog]]. Used for testing when all relations are - * already filled in and the analyser needs only to resolve attribute references. - * + * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing + * when all relations are already filled in and the analyser needs only to resolve attribute + * references. */ -object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyRegistry, true) +object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true) +/** + * Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and + * [[UnresolvedRelation]]s into fully typed objects using information in a schema [[Catalog]] and + * a [[FunctionRegistry]]. + */ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean) - extends RuleExecutor[LogicalPlan] { + extends RuleExecutor[LogicalPlan] { + + // TODO: pass this in as a parameter. val fixedPoint = FixedPoint(100) - val batches = Seq( + val batches: Seq[Batch] = Seq( Batch("LocalRelations", Once, NewLocalRelationInstances), Batch("CaseInsensitiveAttributeReferences", Once, - (if(caseSensitive) Nil else LowercaseAttributeReferences :: Nil):_*), + (if (caseSensitive) Nil else LowercaseAttributeReferences :: Nil) : _*), Batch("Resolution", fixedPoint, ResolveReferences, ResolveRelations, @@ -60,22 +67,26 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s from a logical plan node's children. + * Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s + * from a logical plan node's children. */ object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan if childIsFullyResolved(q) => logger.trace(s"Attempting to resolve ${q.simpleString}") q transformExpressions { - case u @ UnresolvedAttribute(name) => - // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val result = q.resolve(name).getOrElse(u) - logger.debug(s"Resolving $u to $result") - result + case u @ UnresolvedAttribute(name) => + // Leave unchanged if resolution fails. Hopefully will be resolved next round. + val result = q.resolve(name).getOrElse(u) + logger.debug(s"Resolving $u to $result") + result } } } + /** + * Replaces [[UnresolvedFunction]]s with concrete [[Expression]]s. + */ object ResolveFunctions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => @@ -111,6 +122,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Wait until children are resolved case p: LogicalPlan if !childIsFullyResolved(p) => p + // If the projection list contains Star's, expand it. case p @ Project(projectList, child) if containsStar(projectList) => Project( projectList.flatMap { @@ -118,6 +130,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case o => o :: Nil }, child) + // If the aggregate function argument contains Star's, expand it. case a: Aggregate if containsStar(a.aggregateExpressions) => a.copy( aggregateExpressions = a.aggregateExpressions.flatMap { @@ -139,4 +152,4 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool */ protected def childIsFullyResolved(plan: LogicalPlan): Boolean = (!plan.inputSet.isEmpty) && plan.inputSet.map(_.resolved).reduceLeft(_ && _) -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/analysis/Catalog.scala b/src/main/scala/catalyst/analysis/Catalog.scala index be931890661b7..2d4e307c8921b 100644 --- a/src/main/scala/catalyst/analysis/Catalog.scala +++ b/src/main/scala/catalyst/analysis/Catalog.scala @@ -6,14 +6,16 @@ import plans.logical.LogicalPlan /** * An interface for looking up relations by name. Used by an [[Analyzer]]. */ -abstract trait Catalog { +trait Catalog { def lookupRelation(name: String, alias: Option[String] = None): LogicalPlan } /** - * A trivial catalog that returns an error when a relation is requested. Used for testing when all relations are - * already filled in and the analyser needs only to resolve attribute references. + * A trivial catalog that returns an error when a relation is requested. Used for testing when all + * relations are already filled in and the analyser needs only to resolve attribute references. */ object EmptyCatalog extends Catalog { - def lookupRelation(name: String, alias: Option[String] = None) = ??? -} \ No newline at end of file + def lookupRelation(name: String, alias: Option[String] = None) = { + throw new UnsupportedOperationException + } +} diff --git a/src/main/scala/catalyst/analysis/FunctionRegistry.scala b/src/main/scala/catalyst/analysis/FunctionRegistry.scala index be166b0931353..9fef07b363b9f 100644 --- a/src/main/scala/catalyst/analysis/FunctionRegistry.scala +++ b/src/main/scala/catalyst/analysis/FunctionRegistry.scala @@ -3,10 +3,17 @@ package analysis import expressions._ -abstract trait FunctionRegistry { +/** A catalog for looking up user defined functions, used by an [[Analyzer]]. */ +trait FunctionRegistry { def lookupFunction(name: String, children: Seq[Expression]): Expression } -object EmptyRegistry extends FunctionRegistry { - def lookupFunction(name: String, children: Seq[Expression]): Expression = ??? -} \ No newline at end of file +/** + * A trivial catalog that returns an error when a function is requested. Used for testing when all + * functions are already filled in and the analyser needs only to resolve attribute references. + */ +object EmptyFunctionRegistry extends FunctionRegistry { + def lookupFunction(name: String, children: Seq[Expression]): Expression = { + throw new UnsupportedOperationException + } +} diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index cdb9aa6100350..0edb6ef12d32e 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -1,7 +1,6 @@ package catalyst package analysis -import errors._ import expressions._ import plans.logical._ import rules._ @@ -11,7 +10,7 @@ import types._ * Converts string "NaN"s that are in binary operators with a NaN-able types (Float / Double) to the * appropriate numeric equivalent. */ -object ConvertNaNs extends Rule[LogicalPlan]{ +object ConvertNaNs extends Rule[LogicalPlan] { val stringNaN = Literal("NaN", StringType) def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -43,10 +42,12 @@ object ConvertNaNs extends Rule[LogicalPlan]{ * * Loosely based on rules from "Hadoop: The Definitive Guide" 2nd edition, by Tom White * - * The implicit conversion rules can be summarized as follows. Any integral numeric type can be - * implicitly converted to a wider type. All the integral numeric types, FLOAT, and (perhaps - * surprisingly) STRING can be implicitly converted to DOUBLE. TINYINT, SMALLINT, and INT can all be - * converted to FLOAT. BOOLEAN types cannot be converted to any other type. + * The implicit conversion rules can be summarized as follows: + * - Any integral numeric type can be implicitly converted to a wider type. + * - All the integral numeric types, FLOAT, and (perhaps surprisingly) STRING can be implicitly + * converted to DOUBLE. + * - TINYINT, SMALLINT, and INT can all be converted to FLOAT. + * - BOOLEAN types cannot be converted to any other type. * * String conversions are handled by the PromoteStrings rule. */ @@ -121,4 +122,4 @@ object BooleanComparisons extends Rule[LogicalPlan] { case p: BinaryComparison if p.left.dataType == BooleanType && p.right.dataType == BooleanType => p.makeCopy(Array(Cast(p.left, ByteType), Cast(p.right, ByteType))) } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/analysis/unresolved.scala b/src/main/scala/catalyst/analysis/unresolved.scala index ac942390bd8c6..0d4c45dfaa3b2 100644 --- a/src/main/scala/catalyst/analysis/unresolved.scala +++ b/src/main/scala/catalyst/analysis/unresolved.scala @@ -31,7 +31,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo def newInstance = this def withQualifiers(newQualifiers: Seq[String]) = this - override def toString(): String = s"'$name" + override def toString: String = s"'$name" } case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { @@ -67,8 +67,10 @@ case class Star( def withQualifiers(newQualifiers: Seq[String]) = this def expand(input: Seq[Attribute]): Seq[NamedExpression] = { - val expandedAttributes = table match { + val expandedAttributes: Seq[Attribute] = table match { + // If there is no table specified, use all input attributes. case None => input + // If there is a table, pick out attributes that are part of this table. case Some(table) => input.filter(_.qualifiers contains table) } val mappedAttributes = expandedAttributes.map(mapFunction).zip(input).map { diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index 691fb0abc06d4..a86f8f0a80a14 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -1,26 +1,23 @@ package catalyst package execution -import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} - -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import scala.collection.JavaConversions._ +import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.generic.GenericUDF import org.apache.hadoop.hive.ql.exec.UDF +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.io._ import expressions._ import types._ -import org.apache.hadoop.io._ - -import collection.JavaConversions._ -import scala.Predef._ object HiveFunctionRegistry extends analysis.FunctionRegistry { def lookupFunction(name: String, children: Seq[Expression]): Expression = { - // We only look it up to see if it exists, but do not include it in the HiveUDF since it is not always serializable. - val functionInfo = - Option(FunctionRegistry.getFunctionInfo(name)) - .getOrElse(sys.error(s"Couldn't find function $name")) + // We only look it up to see if it exists, but do not include it in the HiveUDF since it is + // not always serializable. + val functionInfo: FunctionInfo = Option(FunctionRegistry.getFunctionInfo(name)).getOrElse( + sys.error(s"Couldn't find function $name")) if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { val functionInfo = FunctionRegistry.getFunctionInfo(name) @@ -31,7 +28,7 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry { HiveSimpleUdf( name, - children.zip(expectedDataTypes).map { case (e, t) => Cast(e,t) } + children.zip(expectedDataTypes).map { case (e, t) => Cast(e, t) } ) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdf(name, IntegerType, children) @@ -99,41 +96,38 @@ abstract class HiveUdf extends Expression with ImplementedUdf with Logging { } } -case class HiveSimpleUdf( - name: String, - children: Seq[Expression]) extends HiveUdf { +case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { import HiveFunctionRegistry._ type UDFType = UDF @transient lazy val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) @transient - lazy val dataType = javaClassToDataType(method.getReturnType) + lazy val dataType: DataType = javaClassToDataType(method.getReturnType) lazy val wrappers = method.getParameterTypes.map { argClass => val primitiveClasses = Seq( Integer.TYPE, classOf[java.lang.Integer], classOf[java.lang.String], java.lang.Double.TYPE, classOf[java.lang.Double], java.lang.Long.TYPE, classOf[java.lang.Long] ) - val matchingConstructor = - argClass.getConstructors.find(c => - c.getParameterTypes.size == 1 && - primitiveClasses.contains(c.getParameterTypes.head)) + val matchingConstructor = argClass.getConstructors.find { c => + c.getParameterTypes.size == 1 && primitiveClasses.contains(c.getParameterTypes.head) + } - val constructor = - matchingConstructor - .getOrElse(sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}.")) + val constructor = matchingConstructor.getOrElse( + sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}.")) (a: Any) => { - logger.debug(s"Wrapping $a of type ${if(a == null) "null" else a.getClass.getName} using $constructor.") + logger.debug(s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} using $constructor.") // We must make sure that primitives get boxed java style. - if(a == null) + if (a == null) { null - else + } else { constructor.newInstance(a match { case i: Int => i: java.lang.Integer case other: AnyRef => other }).asInstanceOf[AnyRef] + } } } @@ -156,7 +150,7 @@ case class HiveGenericUdf( import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ type UDFType = GenericUDF - lazy val inspectors = children.map(_.dataType).map { + lazy val inspectors = children.map(_.dataType).map { case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector case DoubleType => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector @@ -165,6 +159,7 @@ case class HiveGenericUdf( case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector } + lazy val instance = { function.initialize(inspectors.toArray) function diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 06b62687e88d6..46906149873ad 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -25,17 +25,19 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { val table = client.getTable(databaseName, tableName) val hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) val partitions = - if(hiveQlTable.isPartitioned) + if (hiveQlTable.isPartitioned) { client.listPartitions(databaseName, tableName, 255).toSeq - else + } else { Nil + } // Since HiveQL is case insensitive for table names we make them all lowercase. MetastoreRelation(databaseName.toLowerCase, tableName.toLowerCase, alias)(table, partitions) } /** - * Creates any tables required for query execution. For example, because of a CREATE TABLE X AS statement. + * Creates any tables required for query execution. + * For example, because of a CREATE TABLE X AS statement. */ object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -81,8 +83,9 @@ object HiveMetatoreTypes { } } -case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String])(val table: Table, val partitions: Seq[Partition]) - extends BaseRelation { +case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String]) + (val table: Table, val partitions: Seq[Partition]) + extends BaseRelation { def hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) def hiveQlPartitions = partitions.map(new org.apache.hadoop.hive.ql.metadata.Partition(hiveQlTable, _)) @@ -94,12 +97,12 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt ) implicit class SchemaAttribute(f: FieldSchema) { - def toAttribute = - AttributeReference( - f.getName, - HiveMetatoreTypes.toDataType(f.getType), - true // Since data can be dumped in randomly with no validation, everything is nullable. - )(qualifiers = tableName +: alias.toSeq) + def toAttribute = AttributeReference( + f.getName, + HiveMetatoreTypes.toDataType(f.getType), + // Since data can be dumped in randomly with no validation, everything is nullable. + nullable = true + )(qualifiers = tableName +: alias.toSeq) } val partitionKeys = hiveQlTable.getPartitionKeys.map(_.toAttribute) diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 519336f217fb1..0f74b2147104d 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -15,14 +15,14 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { protected def buildRow(values: Seq[Any]): Row = new catalyst.expressions.GenericRow(values) } -abstract trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { +trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { self: Product => } -abstract trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { +trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { self: Product => } -abstract trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { +trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { self: Product => -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/Cast.scala b/src/main/scala/catalyst/expressions/Cast.scala index 297933581671b..9c6eb77bbe827 100644 --- a/src/main/scala/catalyst/expressions/Cast.scala +++ b/src/main/scala/catalyst/expressions/Cast.scala @@ -3,7 +3,8 @@ package expressions import types.DataType +/** Cast the child expression to the target data type. */ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { def nullable = child.nullable override def toString = s"CAST($child, $dataType)" -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/SortOrder.scala b/src/main/scala/catalyst/expressions/SortOrder.scala index cfce25cf1475f..b33ef867519c9 100644 --- a/src/main/scala/catalyst/expressions/SortOrder.scala +++ b/src/main/scala/catalyst/expressions/SortOrder.scala @@ -1,8 +1,6 @@ package catalyst package expressions -import types._ - abstract sealed class SortDirection case object Ascending extends SortDirection case object Descending extends SortDirection diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala index 1422d1bd7aec6..621f6e7bd668e 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/src/main/scala/catalyst/expressions/aggregates.scala @@ -8,10 +8,11 @@ abstract class AggregateExpression extends Expression { } /** - * A specific implementation of an aggregate function. Used to wrap a generic [[AggregateExpression]] with an - * algorithm that will be used to compute the result. + * A specific implementation of an aggregate function. Used to wrap a generic + * [[AggregateExpression]] with an algorithm that will be used to compute the result. */ -abstract class AggregateFunction extends AggregateExpression with Serializable with trees.LeafNode[Expression] { +abstract class AggregateFunction + extends AggregateExpression with Serializable with trees.LeafNode[Expression] { self: Product => /** Base should return the generic aggregate expression that this function is computing */ diff --git a/src/main/scala/catalyst/expressions/functions.scala b/src/main/scala/catalyst/expressions/functions.scala index 44b058f5eda2e..c16e096c93345 100644 --- a/src/main/scala/catalyst/expressions/functions.scala +++ b/src/main/scala/catalyst/expressions/functions.scala @@ -4,6 +4,6 @@ package expressions /** * A UDF that has a native JVM implementation. */ -abstract trait ImplementedUdf { +trait ImplementedUdf { def evaluate(evaluatedChildren: Seq[Any]): Any } \ No newline at end of file diff --git a/src/main/scala/catalyst/expressions/namedExpressions.scala b/src/main/scala/catalyst/expressions/namedExpressions.scala index b3498d7575f84..1a1faf32e2c3d 100644 --- a/src/main/scala/catalyst/expressions/namedExpressions.scala +++ b/src/main/scala/catalyst/expressions/namedExpressions.scala @@ -1,7 +1,7 @@ package catalyst package expressions -import catalyst.analysis.{UnresolvedAttribute, UnresolvedException} +import catalyst.analysis.UnresolvedAttribute import types._ object NamedExpression { @@ -43,25 +43,26 @@ abstract class Attribute extends NamedExpression { * * @param child the computation being performed * @param name the name to be associated with the result of computing [[child]]. - * @param exprId A globally unique id used to check if an [[AttributeReference]] refers to this alias. - * Auto-assigned if left blank. + * @param exprId A globally unique id used to check if an [[AttributeReference]] refers to this + * alias. Auto-assigned if left blank. */ case class Alias(child: Expression, name: String) - (val exprId: ExprId = NamedExpression.newExprId, - val qualifiers: Seq[String] = Nil) + (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends NamedExpression with trees.UnaryNode[Expression] { def dataType = child.dataType def nullable = child.nullable def references = child.references - def toAttribute = - if(resolved) + def toAttribute = { + if (resolved) { AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers) - else + } else { UnresolvedAttribute(name) + } + } - override def toString(): String = s"$child AS $name#${exprId.id}" + override def toString: String = s"$child AS $name#${exprId.id}" override protected final def otherCopyArgs = exprId :: qualifiers :: Nil } @@ -72,12 +73,14 @@ case class Alias(child: Expression, name: String) * @param name The name of this attribute, should only be used during analysis or for debugging. * @param dataType The [[DataType]] of this attribute. * @param nullable True if null is a valid value for this attribute. - * @param exprId A globally unique id used to check if different AttributeReferences refer to the same attribute. - * @param qualifiers a list of strings that can be used to referred to this attribute in a fully qualified way. Consider - * the examples tableName.name, subQueryAlias.name. tableName and subQueryAlias are possible qualifiers. + * @param exprId A globally unique id used to check if different AttributeReferences refer to the + * same attribute. + * @param qualifiers a list of strings that can be used to referred to this attribute in a fully + * qualified way. Consider the examples tableName.name, subQueryAlias.name. + * tableName and subQueryAlias are possible qualifiers. */ case class AttributeReference(name: String, dataType: DataType, nullable: Boolean = true) - (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) + (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { override def equals(other: Any) = other match { @@ -106,5 +109,5 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea else AttributeReference(name, dataType, nullable)(exprId, newQualifiers) - override def toString(): String = s"$name#${exprId.id}" + override def toString: String = s"$name#${exprId.id}" } \ No newline at end of file diff --git a/src/main/scala/catalyst/expressions/predicates.scala b/src/main/scala/catalyst/expressions/predicates.scala index cbb543a32d131..0600d92e34446 100644 --- a/src/main/scala/catalyst/expressions/predicates.scala +++ b/src/main/scala/catalyst/expressions/predicates.scala @@ -3,7 +3,7 @@ package expressions import types._ -abstract trait Predicate extends Expression { +trait Predicate extends Expression { self: Product => def dataType = BooleanType @@ -15,14 +15,16 @@ abstract class BinaryPredicate extends BinaryExpression with Predicate { def nullable = left.nullable || right.nullable } -case class Not(child: Expression) extends Predicate with trees.UnaryNode[Expression]{ +case class Not(child: Expression) extends Predicate with trees.UnaryNode[Expression] { def references = child.references def nullable = child.nullable override def toString = s"NOT $child" } + case class And(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "&&" } + case class Or(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "||" } @@ -38,12 +40,15 @@ case class Equals(left: Expression, right: Expression) extends BinaryComparison case class LessThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<" } + case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<=" } + case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">" } + case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">=" } @@ -52,6 +57,7 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr def references = child.references def nullable = false } + case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { def references = child.references def nullable = false diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 69b571a8bca3f..2d907211711e0 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -2,7 +2,8 @@ package catalyst package frontend package hive -import catalyst.analysis.UnresolvedRelation +import scala.collection.JavaConversions._ + import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ @@ -12,9 +13,6 @@ import plans._ import plans.logical._ import types._ -import collection.JavaConversions._ -import scala.collection.mutable - /** * A logical node that represents a non-query command to be executed by the system. For example, * commands can be used by parsers to represent DDL operations. @@ -25,8 +23,9 @@ abstract class Command extends LeafNode { } /** - * Used when we need to start parsing the AST before deciding that we are going to pass the command back for Hive to - * execute natively. Will be replaced with a native command that contains the cmd string. + * Used when we need to start parsing the AST before deciding that we are going to pass the command + * back for Hive to execute natively. Will be replaced with a native command that contains the + * cmd string. */ case object NativePlaceholder extends Command @@ -119,7 +118,7 @@ object HiveQl { /** * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations similar to - * [[catalyst.tree.TreeNode]]. + * [[catalyst.trees.TreeNode]]. * * Note that this should be considered very experimental and is not indented as a replacement for TreeNode. Primarily * it should be noted ASTNodes are not immutable and do not appear to have clean copy semantics. Therefore, users of diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index d574954af2855..c099a777ef4aa 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -19,7 +19,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy @inline def transformExpression(e: Expression) = { val newE = e.transform(rule) - if(newE.id != e.id && newE != e) { + if (newE.id != e.id && newE != e) { changed = true newE } else { @@ -37,7 +37,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case other: AnyRef => other }.toArray - if(changed) makeCopy(newArgs) else this + if (changed) makeCopy(newArgs) else this } /** Returns the result of running [[transformExpressions]] on this node and all its children */ diff --git a/src/main/scala/catalyst/plans/logical/BaseRelation.scala b/src/main/scala/catalyst/plans/logical/BaseRelation.scala index 829f0b3ff245b..16788affb3cf6 100644 --- a/src/main/scala/catalyst/plans/logical/BaseRelation.scala +++ b/src/main/scala/catalyst/plans/logical/BaseRelation.scala @@ -4,5 +4,4 @@ package logical abstract class BaseRelation extends LeafNode { self: Product => - -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala index a4bb80ef240ba..fdd1428601cf6 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala @@ -28,14 +28,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { def resolve(name: String): Option[Attribute] = { val parts = name.split("\\.") - val options = children.flatMap(_.output).filter {option => - // If the first part of the desired name matches a qualifier for this possible match, drop it. - val remainingParts = - if(option.qualifiers contains parts.head) - parts.drop(1) - else - parts - + val options = children.flatMap(_.output).filter { option => + // If the first part of the desired name matches a qualifier for this possible match, drop it. + val remainingParts = if (option.qualifiers contains parts.head) parts.drop(1) else parts option.name == remainingParts.head } diff --git a/src/main/scala/catalyst/plans/logical/TestRelation.scala b/src/main/scala/catalyst/plans/logical/TestRelation.scala index ca49f585e7f8b..88535680fa075 100644 --- a/src/main/scala/catalyst/plans/logical/TestRelation.scala +++ b/src/main/scala/catalyst/plans/logical/TestRelation.scala @@ -15,9 +15,9 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[Product] = Nil) exten def loadData(newData: Seq[Product]) = new LocalRelation(output, data ++ newData) /** - * Returns an identical copy of this relation with new exprIds for all attributes. Different attributes are required - * when a relation is going to be included multiple times in the same query. - * @return + * Returns an identical copy of this relation with new exprIds for all attributes. Different + * attributes are required when a relation is going to be included multiple times in the same + * query. */ def newInstance: LocalRelation = { LocalRelation(output.map(_.newInstance), data) @@ -27,17 +27,20 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[Product] = Nil) exten } /** - * If any local relation appears more than once in the query plan then the plan is updated so that each instance has - * unique expression ids for the attributes produced. + * If any local relation appears more than once in the query plan then the plan is updated so that + * each instance has unique expression ids for the attributes produced. */ object NewLocalRelationInstances extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { val localRelations = plan collect { case l: LocalRelation => l} - val multiAppearance = - localRelations.groupBy(identity[LocalRelation]).filter { case (_, ls) => ls.size > 1 }.map(_._1).toSet + val multiAppearance = localRelations + .groupBy(identity[LocalRelation]) + .filter { case (_, ls) => ls.size > 1 } + .map(_._1) + .toSet plan transform { case l: LocalRelation if multiAppearance contains l => l.newInstance } } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/plans/package.scala b/src/main/scala/catalyst/plans/package.scala index 3b98dbb518ff8..6ac3859a2329c 100644 --- a/src/main/scala/catalyst/plans/package.scala +++ b/src/main/scala/catalyst/plans/package.scala @@ -1,6 +1,7 @@ package catalyst /** - * A a collection of common abstractions for query plans as well as a base [[logical]] plan representation. + * A a collection of common abstractions for query plans as well as + * a base logical plan representation. */ -package object plans \ No newline at end of file +package object plans diff --git a/src/main/scala/catalyst/rules/RuleExecutor.scala b/src/main/scala/catalyst/rules/RuleExecutor.scala index 7bb2497f41fa5..59c5f29c83f49 100644 --- a/src/main/scala/catalyst/rules/RuleExecutor.scala +++ b/src/main/scala/catalyst/rules/RuleExecutor.scala @@ -4,14 +4,29 @@ package rules import trees._ abstract class RuleExecutor[TreeType <: TreeNode[_]] { + + /** + * An execution strategy for rules that indicates the maximum number of executions. If the + * execution reaches fix point (i.e. converge) before maxIterations, it will stop. + */ abstract class Strategy { def maxIterations: Int } + + /** A strategy that only runs once. */ case object Once extends Strategy { val maxIterations = 1 } + + /** A strategy that runs until fix point or maxIterations times, whichever comes first. */ case class FixedPoint(maxIterations: Int) extends Strategy + /** A batch of rules. */ protected case class Batch(name: String, strategy: Strategy, rules: Rule[TreeType]*) + /** Defines a sequence of rule batches, to be overridden by the implementation. */ protected val batches: Seq[Batch] + /** + * Executes the batches of rules defined by the subclass. The batches are executed serially + * using the defined execution strategy. Within each batch, rules are also executed serially. + */ def apply(plan: TreeType): TreeType = { var curPlan = plan @@ -20,8 +35,8 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] { var lastPlan = curPlan curPlan = batch.rules.foldLeft(curPlan) { case (curPlan, rule) => rule(curPlan) } - while(iteration < batch.strategy.maxIterations && - !(curPlan fastEquals lastPlan)) { + // Run until fix point (or the max number of iterations as specified in the strategy. + while (iteration < batch.strategy.maxIterations && !curPlan.fastEquals(lastPlan)) { lastPlan = curPlan curPlan = batch.rules.foldLeft(curPlan) { case (curPlan, rule) => rule(curPlan) } iteration += 1 @@ -30,4 +45,4 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] { curPlan } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index cdec810c1b680..e503a270ef3b4 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -25,8 +25,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { val id = TreeNode.nextId() /** - * Returns true if other is the same [[TreeNode]] instance. Unlike [[equals]] this function will return false for - * different instances of structurally identical trees. + * Returns true if other is the same [[TreeNode]] instance. Unlike [[equals]] this function will + * return false for different instances of structurally identical trees. */ def sameInstance(other: TreeNode[_]): Boolean = { this.id == other.id @@ -34,8 +34,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** * Faster version of equality which short-circuits when two treeNodes are the same instance. - * We don't just override Object.Equals, as doing so prevents the scala compiler from from generating case class - * [[equals]] methods. + * We don't just override Object.Equals, as doing so prevents the scala compiler from from + * generating case class [[equals]] methods. */ def fastEquals(other: TreeNode[_]): Boolean = { sameInstance(other) || this == other @@ -62,8 +62,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } /** - * Returns a Seq by applying a function to all nodes in this tree and using the elements of the resulting - * collections. + * Returns a Seq by applying a function to all nodes in this tree and using the elements of the + * resulting collections. */ def flatMap[A](f: BaseType => TraversableOnce[A]): Seq[A] = { val ret = new collection.mutable.ArrayBuffer[A]() @@ -72,8 +72,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } /** - * Returns a Seq containing the result of applying a partial function to all elements in this tree on which the - * function is defined. + * Returns a Seq containing the result of applying a partial function to all elements in this + * tree on which the function is defined. */ def collect[B](pf: PartialFunction[BaseType, B]): Seq[B] = { val ret = new collection.mutable.ArrayBuffer[B]() @@ -91,10 +91,11 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def transform(rule: PartialFunction[BaseType, BaseType]): BaseType = { val afterRule = rule.applyOrElse(this, identity[BaseType]) // Check if unchanged and then possibly return old copy to avoid gc churn. - if(this fastEquals afterRule) + if (this fastEquals afterRule) { transformChildren(rule) - else + } else { afterRule.transformChildren(rule) + } } /** @@ -106,16 +107,16 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def transformChildren(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if(children contains arg) => - val newChild = arg.asInstanceOf[BaseType].transform(rule) - if(!(newChild fastEquals arg)) { - changed = true - newChild - } else { - arg - } + case arg: TreeNode[_] if (children contains arg) => + val newChild = arg.asInstanceOf[BaseType].transform(rule) + if (!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } case args: Seq[_] => args.map { - case arg: TreeNode[_] if(children contains arg) => + case arg: TreeNode[_] if (children contains arg) => val newChild = arg.asInstanceOf[BaseType].transform(rule) if(!(newChild fastEquals arg)) { changed = true @@ -146,10 +147,12 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { */ def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") { try { - if(otherCopyArgs.isEmpty) - getClass.getConstructors.head.newInstance(newArgs: _*).asInstanceOf[this.type] - else - getClass.getConstructors.head.newInstance((newArgs ++ otherCopyArgs).toArray :_*).asInstanceOf[this.type] + val defaultCtor = getClass.getConstructors.head + if (otherCopyArgs.isEmpty) { + defaultCtor.newInstance(newArgs: _*).asInstanceOf[this.type] + } else { + defaultCtor.newInstance((newArgs ++ otherCopyArgs).toArray: _*).asInstanceOf[this.type] + } } catch { case e: java.lang.IllegalArgumentException => throw new OptimizationException( @@ -157,17 +160,18 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } } - /** Returns the name of this type of TreeNode. Defaults to the class name. */ def nodeName = getClass.getSimpleName - /** The arguments that should be included in the arg string. Defaults to the [[productIterator]] */ + /** + * The arguments that should be included in the arg string. Defaults to the [[productIterator]]. + */ protected def stringArgs = productIterator /** Returns a string representing the arguments to this node, minus any children */ def argString: String = productIterator.flatMap { case tn: TreeNode[_] if children contains tn => Nil - case tn: TreeNode[_] if(tn.toString contains "\n") => s"(${tn.simpleString})" :: Nil + case tn: TreeNode[_] if tn.toString contains "\n" => s"(${tn.simpleString})" :: Nil case seq: Seq[_] => seq.mkString("{", ",", "}") :: Nil case other => other :: Nil }.mkString(", ") @@ -175,28 +179,28 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** String representation of this node without any children */ def simpleString = s"$nodeName $argString" - override def toString(): String = treeString + override def toString: String = treeString /** Returns a string representation of the nodes in this tree */ def treeString = generateTreeString(0, new StringBuilder).toString /** - * Returns a string representation of the nodes in this tree, where each operator is numbered. The numbers can be - * used with [[apply]] to easily access specific subtrees. + * Returns a string representation of the nodes in this tree, where each operator is numbered. + * The numbers can be used with [[apply]] to easily access specific subtrees. */ def numberedTreeString = - treeString.split("\n").zipWithIndex.map { case (line,i) => f"$i%02d $line" }.mkString("\n") + treeString.split("\n").zipWithIndex.map { case (line, i) => f"$i%02d $line" }.mkString("\n") def apply(depth: Int): BaseType = getNodeAtDepth(new MutableInt(depth)) protected def getNodeAtDepth(depth: MutableInt): BaseType = { - if(depth.i < 0) - return null.asInstanceOf[BaseType] - else if(depth.i == 0) + if (depth.i < 0) { + null.asInstanceOf[BaseType] + } else if (depth.i == 0) { this - else { + } else { depth.i -= 1 - children.map(_.getNodeAtDepth(depth)).collectFirst { case n if n != null => n }.getOrElse(sys.error("Invalid depth")) + children.map(_.getNodeAtDepth(depth)).find(_ != null).getOrElse(sys.error("Invalid depth")) } } diff --git a/src/main/scala/catalyst/types/package.scala b/src/main/scala/catalyst/types/package.scala index 0e26a04ccf911..872b5b1ab904a 100644 --- a/src/main/scala/catalyst/types/package.scala +++ b/src/main/scala/catalyst/types/package.scala @@ -1,6 +1,7 @@ package catalyst /** - * Contains a type system for attributes produced by relations, including complex types like structs, arrays and maps. + * Contains a type system for attributes produced by relations, including complex types like + * structs, arrays and maps. */ -package object types \ No newline at end of file +package object types From 5c421ac1421b3f04c7bcd658616b65deb3f3e31b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 Jan 2014 04:58:04 -0800 Subject: [PATCH 369/778] Imported SharkEnv, SharkContext, and HadoopTableReader to remove Shark dependency. --- build.sbt | 2 +- .../execution/QueryExecutionException.scala | 3 + .../catalyst/execution/SharkContext.scala | 82 +++++++ .../scala/catalyst/execution/SharkEnv.scala | 43 ++++ .../catalyst/execution/SharkInstance.scala | 8 +- .../catalyst/execution/TableReader.scala | 230 ++++++++++++++++++ .../scala/catalyst/execution/TestShark.scala | 10 +- .../scala/catalyst/execution/Transform.scala | 7 +- .../scala/catalyst/execution/aggregates.scala | 2 - .../catalyst/execution/basicOperators.scala | 11 - .../catalyst/execution/hiveOperators.scala | 19 +- src/main/scala/catalyst/execution/joins.scala | 6 +- .../scala/catalyst/execution/package.scala | 2 - .../execution/planningStrategies.scala | 4 +- src/main/scala/catalyst/expressions/Row.scala | 4 +- .../catalyst/planning/QueryPlanner.scala | 20 +- .../execution/HiveComparisionTest.scala | 5 +- 17 files changed, 393 insertions(+), 65 deletions(-) create mode 100644 src/main/scala/catalyst/execution/QueryExecutionException.scala create mode 100644 src/main/scala/catalyst/execution/SharkContext.scala create mode 100644 src/main/scala/catalyst/execution/SharkEnv.scala create mode 100644 src/main/scala/catalyst/execution/TableReader.scala diff --git a/build.sbt b/build.sbt index 4a180507d74e1..963f7de8a4735 100644 --- a/build.sbt +++ b/build.sbt @@ -10,7 +10,7 @@ scalaVersion := "2.10.3" resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" -libraryDependencies += "edu.berkeley.cs.amplab" %% "shark" % "0.9.0-SNAPSHOT" +libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating-SNAPSHOT" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" diff --git a/src/main/scala/catalyst/execution/QueryExecutionException.scala b/src/main/scala/catalyst/execution/QueryExecutionException.scala new file mode 100644 index 0000000000000..40ec17798c319 --- /dev/null +++ b/src/main/scala/catalyst/execution/QueryExecutionException.scala @@ -0,0 +1,3 @@ +package catalyst.execution + +class QueryExecutionException(message: String) extends Exception(message) diff --git a/src/main/scala/catalyst/execution/SharkContext.scala b/src/main/scala/catalyst/execution/SharkContext.scala new file mode 100644 index 0000000000000..c5378b39516fd --- /dev/null +++ b/src/main/scala/catalyst/execution/SharkContext.scala @@ -0,0 +1,82 @@ +package catalyst.execution + +import java.io.PrintStream +import java.util.{ArrayList => JArrayList} + +import scala.collection.Map +import scala.collection.JavaConversions._ + +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.processors.CommandProcessor +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse +import org.apache.hadoop.hive.ql.session.SessionState + +import org.apache.spark.{SparkContext, SparkEnv} + + +class SharkContext( + master: String, + jobName: String, + sparkHome: String, + jars: Seq[String], + environment: Map[String, String]) + extends SparkContext(master, jobName, sparkHome, jars, environment) { + + @transient val sparkEnv = SparkEnv.get + + SharkContext.init() + import SharkContext._ + + /** + * Execute the command and return the results as a sequence. Each element + * in the sequence is one row. + */ + def sql(cmd: String, maxRows: Int = 1000): Seq[String] = { + SparkEnv.set(sparkEnv) + val cmd_trimmed: String = cmd.trim() + val tokens: Array[String] = cmd_trimmed.split("\\s+") + val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() + val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) + + SessionState.start(sessionState) + + if (proc.isInstanceOf[Driver]) { + val driver: Driver = proc.asInstanceOf[Driver] + driver.init() + + val results = new JArrayList[String] + val response: CommandProcessorResponse = driver.run(cmd) + // Throw an exception if there is an error in query processing. + if (response.getResponseCode != 0) { + driver.destroy() + throw new QueryExecutionException(response.getErrorMessage) + } + driver.setMaxRows(maxRows) + driver.getResults(results) + driver.destroy() + results + } else { + sessionState.out.println(tokens(0) + " " + cmd_1) + Seq(proc.run(cmd_1).getResponseCode.toString) + } + } +} + + +object SharkContext { + // Since we can never properly shut down Hive, we put the Hive related initializations + // here in a global singleton. + + @transient val hiveconf = new HiveConf(classOf[SessionState]) + + @transient val sessionState = new SessionState(hiveconf) + sessionState.out = new PrintStream(System.out, true, "UTF-8") + sessionState.err = new PrintStream(System.out, true, "UTF-8") + + // A dummy init to make sure the object is properly initialized. + def init() {} +} + + diff --git a/src/main/scala/catalyst/execution/SharkEnv.scala b/src/main/scala/catalyst/execution/SharkEnv.scala new file mode 100644 index 0000000000000..d316d6f0c5d0f --- /dev/null +++ b/src/main/scala/catalyst/execution/SharkEnv.scala @@ -0,0 +1,43 @@ +package catalyst.execution + +import scala.collection.mutable.{HashMap, HashSet} + +import org.apache.spark.SparkContext +import org.apache.spark.scheduler.StatsReportListener + + +/** A singleton object for the master program. The slaves should not access this. */ +object SharkEnv { + + def initWithSharkContext(jobName: String, master: String): SharkContext = { + if (sc != null) { + sc.stop() + } + + sc = new SharkContext( + if (master == null) "local" else master, + jobName, + System.getenv("SPARK_HOME"), + Nil, + executorEnvVars) + sc.addSparkListener(new StatsReportListener()) + sc.asInstanceOf[SharkContext] + } + + val executorEnvVars = new HashMap[String, String] + executorEnvVars.put("SCALA_HOME", getEnv("SCALA_HOME")) + executorEnvVars.put("SPARK_MEM", getEnv("SPARK_MEM")) + executorEnvVars.put("SPARK_CLASSPATH", getEnv("SPARK_CLASSPATH")) + executorEnvVars.put("HADOOP_HOME", getEnv("HADOOP_HOME")) + executorEnvVars.put("JAVA_HOME", getEnv("JAVA_HOME")) + executorEnvVars.put("MESOS_NATIVE_LIBRARY", getEnv("MESOS_NATIVE_LIBRARY")) + executorEnvVars.put("TACHYON_MASTER", getEnv("TACHYON_MASTER")) + executorEnvVars.put("TACHYON_WAREHOUSE_PATH", getEnv("TACHYON_WAREHOUSE_PATH")) + + val activeSessions = new HashSet[String] + + var sc: SparkContext = _ + + /** Return the value of an environmental variable as a string. */ + def getEnv(varname: String) = if (System.getenv(varname) == null) "" else System.getenv(varname) +} diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index edd7eb93b4431..8de19de4b44be 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -3,8 +3,6 @@ package execution import java.io.File -import shark.{SharkConfVars, SharkContext, SharkEnv} - import analysis.{SimpleAnalyzer, Analyzer} import frontend.hive._ import optimizer.Optimize @@ -39,16 +37,12 @@ abstract class SharkInstance extends Logging { /** The SharkContext */ lazy val sc = createContext() - protected def createContext() = { + protected def createContext(): SharkContext = { SharkEnv.initWithSharkContext("catalyst.execution", master) } /** Sets up the system initially or after a RESET command */ protected def configure() { - // Use hive natively for queries that won't be executed by catalyst. This is because - // shark has dependencies on a custom version of hive that we are trying to avoid in catalyst. - SharkConfVars.setVar(SharkContext.hiveconf, SharkConfVars.EXEC_MODE, "hive") - // TODO: refactor this so we can work with other databases. runSqlHive("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + metastorePath + ";create=true") runSqlHive("set hive.metastore.warehouse.dir=" + warehousePath) diff --git a/src/main/scala/catalyst/execution/TableReader.scala b/src/main/scala/catalyst/execution/TableReader.scala new file mode 100644 index 0000000000000..58b0c8f502591 --- /dev/null +++ b/src/main/scala/catalyst/execution/TableReader.scala @@ -0,0 +1,230 @@ +package catalyst.execution + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} +import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.io.Writable +import org.apache.hadoop.fs.{Path, PathFilter} +import org.apache.hadoop.mapred.{FileInputFormat, JobConf, InputFormat} + +import org.apache.spark.SerializableWritable +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.{HadoopRDD, UnionRDD, EmptyRDD, RDD} + + +/** + * A trait for subclasses that handle table scans. In Shark, there is one subclass for each + * type of table storage: HeapTableReader for Shark tables in Spark's block manager, + * TachyonTableReader for tables in Tachyon, and HadoopTableReader for Hive tables in a filesystem. + */ +sealed trait TableReader { + + def makeRDDForTable(hiveTable: HiveTable): RDD[_] + + def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] + +} + + +/** + * Helper class for scanning tables stored in Hadoop - e.g., to read Hive tables that reside in the + * data warehouse directory. + */ +class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) + extends TableReader { + + // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless + // it is smaller than what Spark suggests. + private val _minSplitsPerRDD = math.max( + _localHConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) + + + // TODO: set aws s3 credentials. + + private val _broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(_localHConf)) + + def broadcastedHiveConf = _broadcastedHiveConf + + def hiveConf = _broadcastedHiveConf.value.value + + override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = + makeRDDForTable( + hiveTable, + _tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], + filterOpt = None) + + /** + * Creates a Hadoop RDD to read data from the target table's data directory. Returns a transformed + * RDD that contains deserialized rows. + * + * @param hiveTable Hive metadata for the table being scanned. + * @param deserializerClass Class of the SerDe used to deserialize Writables read from Hadoop. + * @param filterOpt If defined, then the filter is used to reject files contained in the data + * directory being read. If None, then all files are accepted. + */ + def makeRDDForTable( + hiveTable: HiveTable, + deserializerClass: Class[_ <: Deserializer], + filterOpt: Option[PathFilter]): RDD[_] = + { + assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, + since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""") + + // Create local references to member variables, so that the entire `this` object won't be + // serialized in the closure below. + val tableDesc = _tableDesc + val broadcastedHiveConf = _broadcastedHiveConf + + val tablePath = hiveTable.getPath + val inputPathStr = applyFilterIfNeeded(tablePath, filterOpt) + + //logDebug("Table input: %s".format(tablePath)) + val ifc = hiveTable.getInputFormatClass + .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] + val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) + + val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => + val hconf = broadcastedHiveConf.value.value + val deserializer = deserializerClass.newInstance().asInstanceOf[Deserializer] + deserializer.initialize(hconf, tableDesc.getProperties) + + // Deserialize each Writable to get the row value. + iter.map { value => + value match { + case v: Writable => deserializer.deserialize(v) + case _ => throw new RuntimeException("Failed to match " + value.toString) + } + } + } + deserializedHadoopRDD + } + + override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { + val partitionToDeserializer = partitions.map(part => + (part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap + makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None) + } + + /** + * Create a HadoopRDD for every partition key specified in the query. Note that for on-disk Hive + * tables, a data directory is created for each partition corresponding to keys specified using + * 'PARTITION BY'. + * + * @param partitionToDeserializer Mapping from a Hive Partition metadata object to the SerDe + * class to use to deserialize input Writables from the corresponding partition. + * @param filterOpt If defined, then the filter is used to reject files contained in the data + * subdirectory of each partition being read. If None, then all files are accepted. + */ + def makeRDDForPartitionedTable( + partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], + filterOpt: Option[PathFilter]): RDD[_] = + { + val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => + val partDesc = Utilities.getPartitionDesc(partition) + val partPath = partition.getPartitionPath + val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) + val ifc = partDesc.getInputFileFormatClass + .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] + // Get partition field info + val partSpec = partDesc.getPartSpec() + val partProps = partDesc.getProperties() + + // TODO: fix this. + //val partColsDelimited: String = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) + val partColsDelimited = ";" + // Partitioning columns are delimited by "/" + val partCols = partColsDelimited.trim().split("/").toSeq + // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. + val partValues = if (partSpec == null) { + Array.fill(partCols.size)(new String) + } else { + partCols.map(col => new String(partSpec.get(col))).toArray + } + + // Create local references so that the outer object isn't serialized. + val tableDesc = _tableDesc + val broadcastedHiveConf = _broadcastedHiveConf + val localDeserializer = partDeserializer + + val hivePartitionRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) + hivePartitionRDD.mapPartitions { iter => + val hconf = broadcastedHiveConf.value.value + val rowWithPartArr = new Array[Object](2) + // Map each tuple to a row object + iter.map { value => + val deserializer = localDeserializer.newInstance() + deserializer.initialize(hconf, partProps) + val deserializedRow = deserializer.deserialize(value) // LazyStruct + rowWithPartArr.update(0, deserializedRow) + rowWithPartArr.update(1, partValues) + rowWithPartArr.asInstanceOf[Object] + } + } + }.toSeq + // Even if we don't use any partitions, we still need an empty RDD + if (hivePartitionRDDs.size == 0) { + new EmptyRDD[Object](SharkEnv.sc) + } else { + new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs) + } + } + + /** + * If `filterOpt` is defined, then it will be used to filter files from `path`. These files are + * returned in a single, comma-separated string. + */ + private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { + filterOpt match { + case Some(filter) => { + val fs = path.getFileSystem(_localHConf) + val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString) + filteredFiles.mkString(",") + } + case None => path.toString + } + } + + /** + * Creates a HadoopRDD based on the broadcasted HiveConf and other job properties that will be + * applied locally on each slave. + */ + private def createHadoopRdd( + tableDesc: TableDesc, + path: String, + inputFormatClass: Class[InputFormat[Writable, Writable]]) + : RDD[Writable] = { + val initializeJobConfFunc = HadoopTableReader.initializeLocalJobConfFunc(path, tableDesc) _ + + val rdd = new HadoopRDD( + SharkEnv.sc, + _broadcastedHiveConf.asInstanceOf[Broadcast[SerializableWritable[Configuration]]], + Some(initializeJobConfFunc), + inputFormatClass, + classOf[Writable], + classOf[Writable], + _minSplitsPerRDD) + + // Only take the value (skip the key) because Hive works only with values. + rdd.map(_._2) + } + +} + +object HadoopTableReader { + + /** + * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to + * instantiate a HadoopRDD. + */ + def initializeLocalJobConfFunc(path: String, tableDesc: TableDesc)(jobConf: JobConf) { + FileInputFormat.setInputPaths(jobConf, path) + if (tableDesc != null) { + Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) + } + val bufferSize = System.getProperty("spark.buffer.size", "65536") + jobConf.set("io.file.buffer.size", bufferSize) + } +} diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index e500d4c79c080..f9b150a82aa52 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -2,21 +2,19 @@ package catalyst package execution import java.io.File + import scala.collection.mutable +import scala.collection.JavaConversions._ +import org.apache.hadoop.hive.metastore.api.{SerDeInfo, StorageDescriptor} import org.apache.hadoop.hive.metastore.MetaStoreUtils -import shark.SharkEnv +import org.apache.hadoop.hive.ql.exec.FunctionRegistry import analysis._ import plans.logical.LogicalPlan import frontend.hive._ import util._ -import collection.JavaConversions._ -import org.apache.hadoop.hive.ql.exec.FunctionRegistry -import org.apache.hadoop.hive.metastore.api.{SerDeInfo, StorageDescriptor} - - /** * A locally running test instance of spark. The lifecycle for a given query is managed by the inner class * [[SharkQuery]]. A [[SharkQuery]] can either be instantiated directly or using the implicit conversion '.q'. diff --git a/src/main/scala/catalyst/execution/Transform.scala b/src/main/scala/catalyst/execution/Transform.scala index 0178efbc8c9dd..3a00fa4560920 100644 --- a/src/main/scala/catalyst/execution/Transform.scala +++ b/src/main/scala/catalyst/execution/Transform.scala @@ -1,11 +1,12 @@ package catalyst package execution -import catalyst.expressions._ -import shark.SharkContext import java.io.{InputStreamReader, BufferedReader} -import collection.JavaConversions._ +import scala.collection.JavaConversions._ + +import catalyst.expressions._ + /** * Transforms the input by forking and running the specified script. diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index dfdb6e68fee31..8a507c81b6b72 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,8 +3,6 @@ package execution import catalyst.errors._ import catalyst.expressions._ -import catalyst.types.IntegerType -import shark.SharkContext /* Implicits */ import org.apache.spark.SparkContext._ diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index a4ea279240041..4535ccb708772 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -1,21 +1,10 @@ package catalyst package execution -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} -import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, StructObjectInspector} -import shark.execution.HadoopTableReader -import shark.{SharkContext, SharkEnv} - import errors._ import expressions._ import types._ -import collection.JavaConversions._ -import org.apache.hadoop.hive.ql.exec.OperatorFactory -import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils -import org.apache.hadoop.hive.serde2.Serializer -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext._ diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index fb9e94074dd43..47b66c1d1d71d 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -9,9 +9,6 @@ import org.apache.hadoop.hive.serde2.`lazy`.LazyStruct import org.apache.hadoop.hive.serde2.Serializer import org.apache.hadoop.mapred.JobConf -import shark.execution.HadoopTableReader -import shark.SharkContext - import expressions.Attribute import util._ @@ -34,8 +31,8 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation * Functions that extract the requested attributes from the hive output. */ @transient - protected lazy val attributeFunctions = attributes.map { a => - if(relation.partitionKeys.contains(a)) { + protected lazy val attributeFunctions: Seq[(LazyStruct, Array[String]) => AnyRef] = attributes.map { a => + if (relation.partitionKeys.contains(a)) { val ordinal = relation.partitionKeys.indexOf(a) (struct: LazyStruct, partitionKeys: Array[String]) => partitionKeys(ordinal) } else { @@ -53,11 +50,11 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation } @transient - def inputRdd = - if(!relation.hiveQlTable.isPartitioned) - hadoopReader.makeRDDForTable(relation.hiveQlTable) - else - hadoopReader.makeRDDForPartitionedTable(relation.hiveQlPartitions) + def inputRdd = if (!relation.hiveQlTable.isPartitioned) { + hadoopReader.makeRDDForTable(relation.hiveQlTable) + } else { + hadoopReader.makeRDDForPartitionedTable(relation.hiveQlPartitions) + } def execute() = { inputRdd.map { row => @@ -117,7 +114,7 @@ case class InsertIntoHiveTable(table: MetastoreRelation, partition: Map[String, s"PARTITION (${partition.map { case (k,v) => s"$k=$v" }.mkString(",")})" else "" - sc.runSql(s"LOAD DATA LOCAL INPATH '${tempDir.getCanonicalPath}/*' INTO TABLE ${table.tableName} $partitionSpec") + sc.sql(s"LOAD DATA LOCAL INPATH '${tempDir.getCanonicalPath}/*' INTO TABLE ${table.tableName} $partitionSpec") // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 706d52d1fea93..6fbb8a9893c4b 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -1,15 +1,13 @@ package catalyst package execution +import scala.collection.mutable + import org.apache.spark.rdd.RDD import errors._ import expressions._ import plans._ -import shark.SharkContext -import org.apache.spark.util.collection.BitSet -import scala.collection -import scala.collection.mutable /* Implicits */ import org.apache.spark.SparkContext._ diff --git a/src/main/scala/catalyst/execution/package.scala b/src/main/scala/catalyst/execution/package.scala index 6e366263d3511..2a073f3e49558 100644 --- a/src/main/scala/catalyst/execution/package.scala +++ b/src/main/scala/catalyst/execution/package.scala @@ -1,7 +1,5 @@ package catalyst -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo - import types._ /** diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 556503b6fbbdc..2615dfff5108b 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -1,14 +1,12 @@ package catalyst package execution -import shark.SharkContext - import expressions._ import planning._ import plans._ import plans.logical.LogicalPlan -abstract trait PlanningStrategies { +trait PlanningStrategies { self: QueryPlanner[SharkPlan] => val sc: SharkContext diff --git a/src/main/scala/catalyst/expressions/Row.scala b/src/main/scala/catalyst/expressions/Row.scala index 391a87b1f338a..7a39508d973be 100644 --- a/src/main/scala/catalyst/expressions/Row.scala +++ b/src/main/scala/catalyst/expressions/Row.scala @@ -5,8 +5,8 @@ package expressions * Represents one row of output from a relational operator. Allows both generic access by ordinal, * which will incur boxing overhead for primitives, as well as native primitive access. * - * It is invalid to use the native primitive interface to retrieve a value that is null, instead a user - * must check [[isNullAt]] before attempting to retrieve a value that might be null. + * It is invalid to use the native primitive interface to retrieve a value that is null, instead a + * user must check [[isNullAt]] before attempting to retrieve a value that might be null. */ abstract class Row extends Seq[Any] with Serializable { def apply(i: Int): Any diff --git a/src/main/scala/catalyst/planning/QueryPlanner.scala b/src/main/scala/catalyst/planning/QueryPlanner.scala index 8f793564e09ca..7024ca4b75bb3 100644 --- a/src/main/scala/catalyst/planning/QueryPlanner.scala +++ b/src/main/scala/catalyst/planning/QueryPlanner.scala @@ -6,10 +6,11 @@ import plans.logical.LogicalPlan import trees._ /** - * Extended by classes that transform [[plans.logical.LogicalPlan]]s into Physical plans. Child classes are - * responsible for specifying a list of [[Strategy]] objects that each of which can return a list of possible - * physical plan options. If a given strategy is unable to plan all of the remaining operators in the tree, it can - * call [[planLater]], which returns a placeholder object that will be filled in using other available strategies. + * Extended by classes that transform [[LogicalPlan]]s into physical plans. Child classes are + * responsible for specifying a list of [[Strategy]] objects that each of which can return a list + * of possible physical plan options. If a given strategy is unable to plan all of the remaining + * operators in the tree, it can call [[planLater]], which returns a placeholder object that will + * be filled in using other available strategies. * * NOTE: RIGHT NOW ONLY ONE PLAN IS RETURNED EVER... PLAN SPACE EXPLORATION WILL BE IMPLEMENTED LATER. * @@ -20,17 +21,18 @@ abstract class QueryPlanner[PhysicalPlan <: TreeNode[PhysicalPlan]] { def strategies: Seq[Strategy] /** - * Given a [[plan.logical.LogicalPlan]], returns a list of [[PhysicalPlans]] that can be used for execution. If this - * strategy does not apply to the give logical operation then an empty list should be returned. + * Given a [[LogicalPlan]], returns a list of [[PhysicalPlans]] that can be used for execution. + * If this strategy does not apply to the give logical operation then an empty list should be + * returned. */ abstract protected class Strategy extends Logging { def apply(plan: LogicalPlan): Seq[PhysicalPlan] - } /** - * Returns a placeholder for a physical plan that executes [[plan]]. This placeholder will be filled in automatically - * by the QueryPlanner using the other execution strategies that are available. + * Returns a placeholder for a physical plan that executes [[plan]]. This placeholder will be + * filled in automatically by the QueryPlanner using the other execution strategies that are + * available. */ protected def planLater(plan: LogicalPlan) = apply(plan).next() diff --git a/src/test/scala/catalyst/execution/HiveComparisionTest.scala b/src/test/scala/catalyst/execution/HiveComparisionTest.scala index 203a5c22b5919..e7183a94d419e 100644 --- a/src/test/scala/catalyst/execution/HiveComparisionTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisionTest.scala @@ -1,22 +1,19 @@ package catalyst package execution -import shark.{SharkContext, SharkEnv} - import java.io._ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} import frontend.hive.{ExplainCommand, Command} import util._ -import collection.JavaConversions._ - /** * Allows the creations of tests that execute the same query against both hive * and catalyst, comparing the results. * * The "golden" results from Hive are cached in [[answerCache]] to speed up testing. */ +// TODO: correct the mispelled name. abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with GivenWhenThen with Logging { protected val targetDir = new File("target") protected val answerCache = new File(targetDir, "comparison-test-cache") From d48d0e1c10a9e0a46409437ab018f8760e65857b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 Jan 2014 12:12:34 -0800 Subject: [PATCH 370/778] Code review feedback. --- src/main/scala/catalyst/analysis/Analyzer.scala | 4 ++-- src/main/scala/catalyst/analysis/typeCoercion.scala | 10 +++++----- .../scala/catalyst/execution/FunctionRegistry.scala | 2 +- src/main/scala/catalyst/execution/SharkContext.scala | 2 +- src/main/scala/catalyst/execution/SharkInstance.scala | 2 +- src/main/scala/catalyst/execution/hiveOperators.scala | 2 +- 6 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 490dcb3b29008..24d705ed4b5bb 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -122,7 +122,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Wait until children are resolved case p: LogicalPlan if !childIsFullyResolved(p) => p - // If the projection list contains Star's, expand it. + // If the projection list contains Stars, expand it. case p @ Project(projectList, child) if containsStar(projectList) => Project( projectList.flatMap { @@ -130,7 +130,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case o => o :: Nil }, child) - // If the aggregate function argument contains Star's, expand it. + // If the aggregate function argument contains Stars, expand it. case a: Aggregate if containsStar(a.aggregateExpressions) => a.copy( aggregateExpressions = a.aggregateExpressions.flatMap { diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index 0edb6ef12d32e..968d9135632dd 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -43,11 +43,11 @@ object ConvertNaNs extends Rule[LogicalPlan] { * Loosely based on rules from "Hadoop: The Definitive Guide" 2nd edition, by Tom White * * The implicit conversion rules can be summarized as follows: - * - Any integral numeric type can be implicitly converted to a wider type. - * - All the integral numeric types, FLOAT, and (perhaps surprisingly) STRING can be implicitly - * converted to DOUBLE. - * - TINYINT, SMALLINT, and INT can all be converted to FLOAT. - * - BOOLEAN types cannot be converted to any other type. + * $ - Any integral numeric type can be implicitly converted to a wider type. + * $ - All the integral numeric types, FLOAT, and (perhaps surprisingly) STRING can be implicitly + * converted to DOUBLE. + * $ - TINYINT, SMALLINT, and INT can all be converted to FLOAT. + * $ - BOOLEAN types cannot be converted to any other type. * * String conversions are handled by the PromoteStrings rule. */ diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index a86f8f0a80a14..e46bd1163c99a 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -103,7 +103,7 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd @transient lazy val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) @transient - lazy val dataType: DataType = javaClassToDataType(method.getReturnType) + lazy val dataType = javaClassToDataType(method.getReturnType) lazy val wrappers = method.getParameterTypes.map { argClass => val primitiveClasses = Seq( diff --git a/src/main/scala/catalyst/execution/SharkContext.scala b/src/main/scala/catalyst/execution/SharkContext.scala index c5378b39516fd..a56b545e666c2 100644 --- a/src/main/scala/catalyst/execution/SharkContext.scala +++ b/src/main/scala/catalyst/execution/SharkContext.scala @@ -33,7 +33,7 @@ class SharkContext( * Execute the command and return the results as a sequence. Each element * in the sequence is one row. */ - def sql(cmd: String, maxRows: Int = 1000): Seq[String] = { + def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { SparkEnv.set(sparkEnv) val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 8de19de4b44be..fb51587d602aa 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -61,7 +61,7 @@ abstract class SharkInstance extends Logging { */ def runSqlHive(sql: String): Seq[String] = { val maxResults = 100000 - val results = sc.sql(sql, 100000) + val results = sc.runHive(sql, 100000) // It is very confusing when you only get back some of the results... if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") results diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 47b66c1d1d71d..c3c490e611276 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -114,7 +114,7 @@ case class InsertIntoHiveTable(table: MetastoreRelation, partition: Map[String, s"PARTITION (${partition.map { case (k,v) => s"$k=$v" }.mkString(",")})" else "" - sc.sql(s"LOAD DATA LOCAL INPATH '${tempDir.getCanonicalPath}/*' INTO TABLE ${table.tableName} $partitionSpec") + sc.runHive(s"LOAD DATA LOCAL INPATH '${tempDir.getCanonicalPath}/*' INTO TABLE ${table.tableName} $partitionSpec") // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which From 555d8393c6c222635eebe370f516a3f8e194e9e7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 Jan 2014 12:17:14 -0800 Subject: [PATCH 371/778] More cleaning ... --- .../catalyst/execution/FunctionRegistry.scala | 13 ++- .../catalyst/execution/MetastoreCatalog.scala | 13 ++- .../catalyst/execution/SharkInstance.scala | 3 +- .../scala/catalyst/execution/TestShark.scala | 81 +++++++++------- .../scala/catalyst/execution/aggregates.scala | 95 ++++++++++--------- .../catalyst/execution/basicOperators.scala | 49 +++++----- .../catalyst/execution/hiveOperators.scala | 61 +++++++----- src/main/scala/catalyst/execution/joins.scala | 29 +++--- .../scala/catalyst/execution/package.scala | 10 +- .../execution/planningStrategies.scala | 26 +++-- .../scala/catalyst/expressions/Evaluate.scala | 2 +- .../scala/catalyst/frontend/package.scala | 6 +- .../plans/logical/basicOperators.scala | 6 +- 13 files changed, 227 insertions(+), 167 deletions(-) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index e46bd1163c99a..f89329ea60c0b 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -6,7 +6,7 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.generic.GenericUDF import org.apache.hadoop.hive.ql.exec.UDF -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{AbstractPrimitiveJavaObjectInspector, PrimitiveObjectInspectorFactory} import org.apache.hadoop.io._ import expressions._ @@ -105,7 +105,7 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd @transient lazy val dataType = javaClassToDataType(method.getReturnType) - lazy val wrappers = method.getParameterTypes.map { argClass => + lazy val wrappers: Array[(Any) => AnyRef] = method.getParameterTypes.map { argClass => val primitiveClasses = Seq( Integer.TYPE, classOf[java.lang.Integer], classOf[java.lang.String], java.lang.Double.TYPE, classOf[java.lang.Double], java.lang.Long.TYPE, classOf[java.lang.Long] @@ -118,7 +118,8 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}.")) (a: Any) => { - logger.debug(s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} using $constructor.") + logger.debug( + s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} using $constructor.") // We must make sure that primitives get boxed java style. if (a == null) { null @@ -150,7 +151,7 @@ case class HiveGenericUdf( import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ type UDFType = GenericUDF - lazy val inspectors = children.map(_.dataType).map { + lazy val inspectors: Seq[AbstractPrimitiveJavaObjectInspector] = children.map(_.dataType).map { case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector case DoubleType => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector @@ -178,7 +179,9 @@ case class HiveGenericUdf( } def evaluate(evaluatedChildren: Seq[Any]): Any = { - val args = evaluatedChildren.map(wrap).map(v => new DeferredJavaObject(v): DeferredObject).toArray + val args = evaluatedChildren.map(wrap).map { v => + new DeferredJavaObject(v): DeferredObject + }.toArray unwrap(instance.evaluate(args)) } } \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 46906149873ad..33f1abeaaed72 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -5,6 +5,8 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table, StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.metastore.HiveMetaStoreClient import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.mapred.InputFormat import analysis.Catalog import expressions._ @@ -88,10 +90,14 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt extends BaseRelation { def hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) - def hiveQlPartitions = partitions.map(new org.apache.hadoop.hive.ql.metadata.Partition(hiveQlTable, _)) + + def hiveQlPartitions = partitions.map { p => + new org.apache.hadoop.hive.ql.metadata.Partition(hiveQlTable, p) + } + val tableDesc = new TableDesc( - Class.forName(table.getSd.getSerdeInfo.getSerializationLib).asInstanceOf[Class[org.apache.hadoop.hive.serde2.Deserializer]], - Class.forName(table.getSd.getInputFormat).asInstanceOf[Class[org.apache.hadoop.mapred.InputFormat[_,_]]], + Class.forName(table.getSd.getSerdeInfo.getSerializationLib).asInstanceOf[Class[Deserializer]], + Class.forName(table.getSd.getInputFormat).asInstanceOf[Class[InputFormat[_,_]]], Class.forName(table.getSd.getOutputFormat), hiveQlTable.getSchema ) @@ -110,4 +116,3 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt // Must be a stable value since new attributes are born here. val output = partitionKeys ++ table.getSd.getCols.map(_.toAttribute) } - diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index fb51587d602aa..1a59488aaf171 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -44,7 +44,8 @@ abstract class SharkInstance extends Logging { /** Sets up the system initially or after a RESET command */ protected def configure() { // TODO: refactor this so we can work with other databases. - runSqlHive("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + metastorePath + ";create=true") + runSqlHive("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + metastorePath + + ";create=true") runSqlHive("set hive.metastore.warehouse.dir=" + warehousePath) } diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index f9b150a82aa52..3cbfb80d297c2 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -2,6 +2,7 @@ package catalyst package execution import java.io.File +import java.util.{Set => JavaSet} import scala.collection.mutable import scala.collection.JavaConversions._ @@ -16,8 +17,9 @@ import frontend.hive._ import util._ /** - * A locally running test instance of spark. The lifecycle for a given query is managed by the inner class - * [[SharkQuery]]. A [[SharkQuery]] can either be instantiated directly or using the implicit conversion '.q'. + * A locally running test instance of spark. The lifecycle for a given query is managed by the + * inner class [[SharkQuery]]. A [[SharkQuery]] can either be instantiated directly or using the + * implicit conversion '.q'. * * {{{ * scala> val query = "SELECT key FROM src".q @@ -34,12 +36,13 @@ import util._ * res0: Array[IndexedSeq[Any]] = Array(Vector(238), Vector(86), Vector(311), ... * }}} * - * Data from [[testTables]] will be automatically loaded whenever a query is run over those tables. Calling - * [[reset]] will delete all tables and other state in the database, leaving the database in a "clean" state. + * Data from [[testTables]] will be automatically loaded whenever a query is run over those tables. + * Calling [[reset]] will delete all tables and other state in the database, leaving the database + * in a "clean" state. * - * TestShark is implemented as a singleton object because instantiating multiple copies of the hive metastore - * seems to lead to weird non-deterministic failures. Therefore, the execution of testcases that rely on TestShark - * must be serialized. + * TestShark is implemented as a singleton object because instantiating multiple copies of the hive + * metastore seems to lead to weird non-deterministic failures. Therefore, the execution of + * testcases that rely on TestShark must be serialized. */ object TestShark extends SharkInstance { self => @@ -48,7 +51,7 @@ object TestShark extends SharkInstance { lazy val warehousePath = getTempFilePath("sharkWarehouse").getCanonicalPath lazy val metastorePath = getTempFilePath("sharkMetastore").getCanonicalPath - override protected def createContext = { + override protected def createContext() = { // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.driver.port") @@ -65,7 +68,10 @@ object TestShark extends SharkInstance { // Override so we can intercept relative paths and rewrite them to point at hive. override def runSqlHive(sql: String): Seq[String] = super.runSqlHive(rewritePaths(sql)) - /** Returns the value of specified environmental variable as a [[java.io.File]] after checking to ensure it exists */ + /** + * Returns the value of specified environmental variable as a [[java.io.File]] after checking + * to ensure it exists + */ private def envVarToFile(envVar: String): File = { assert(System.getenv(envVar) != null, s"$envVar not set") val ret = new File(System.getenv(envVar)) @@ -74,8 +80,8 @@ object TestShark extends SharkInstance { } /** - * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the hive test cases - * assume the system is set up. + * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the + * hive test cases assume the system is set up. */ private def rewritePaths(cmd: String): String = if(cmd.toUpperCase startsWith "LOAD") @@ -84,6 +90,7 @@ object TestShark extends SharkInstance { cmd val describedTable = "DESCRIBE (\\w+)".r + /** * Override SharkQuery with special debug workflow. */ @@ -105,8 +112,10 @@ object TestShark extends SharkInstance { analyze(parsed) } - /** Runs the query after interposing operators that print the result of each intermediate step. */ - def debugExec() = DebugQuery(executedPlan).execute().collect + /** + * Runs the query after interposing operators that print the result of each intermediate step. + */ + def debugExec() = DebugQuery(executedPlan).execute().collect() } class SharkSqlQuery(sql: String) extends SharkQuery { @@ -122,15 +131,17 @@ object TestShark extends SharkInstance { def q = new SharkSqlQuery(str) } - implicit override def logicalToSharkQuery(plan: LogicalPlan) = new LogicalSharkQuery { val parsed = plan } + implicit override def logicalToSharkQuery(plan: LogicalPlan) = new LogicalSharkQuery { + val parsed = plan + } case class TestTable(name: String, commands: (()=>Unit)*) implicit class SqlCmd(sql: String) { def cmd = () => sql.q.stringResult(): Unit} /** - * A list of test tables and the DDL required to initialize them. A test table is loaded on demand when a query - * are run against it. + * A list of test tables and the DDL required to initialize them. A test table is loaded on + * demand when a query are run against it. */ val testTables = new mutable.HashMap[String, TestTable]() def registerTestTable(testTable: TestTable) = testTables += (testTable.name -> testTable) @@ -175,10 +186,9 @@ object TestShark extends SharkInstance { srcThrift.getSd.setOutputFormat(classOf[SequenceFileOutputFormat[_,_]].getName) srcThrift.getSd.setSerdeInfo(new SerDeInfo) srcThrift.getSd.getSerdeInfo.setSerializationLib(classOf[ThriftDeserializer].getName) - srcThrift.getSd.getSerdeInfo.setParameters( - Map( - "serialization.class" -> classOf[Complex].getName, - "serialization.format" -> classOf[TBinaryProtocol].getName)) + srcThrift.getSd.getSerdeInfo.setParameters(Map( + "serialization.class" -> classOf[Complex].getName, + "serialization.format" -> classOf[TBinaryProtocol].getName)) catalog.client.createTable(srcThrift) @@ -189,8 +199,9 @@ object TestShark extends SharkInstance { hiveQTestUtilTables.foreach(registerTestTable) private val loadedTables = new collection.mutable.HashSet[String] + def loadTestTable(name: String) { - if(!(loadedTables contains name)) { + if (!(loadedTables contains name)) { logger.info(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) @@ -203,7 +214,7 @@ object TestShark extends SharkInstance { * Records the UDFs present when the server starts, so we can delete ones that are created by * tests. */ - protected val originalUdfs = FunctionRegistry.getFunctionNames() + protected val originalUdfs: JavaSet[String] = FunctionRegistry.getFunctionNames /** * Resets the test instance by deleting any tables that have been created. @@ -212,27 +223,32 @@ object TestShark extends SharkInstance { def reset() { try { // HACK: Hive is too noisy by default. - org.apache.log4j.LogManager.getCurrentLoggers.foreach(_.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN)) + org.apache.log4j.LogManager.getCurrentLoggers.foreach { logger => + logger.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) + } - // It is important that we RESET first as broken hooks that might have been set could break other sql exec here. + // It is important that we RESET first as broken hooks that might have been set could break + // other sql exec here. runSqlHive("RESET") // For some reason, RESET does not reset the following variables... runSqlHive("set datanucleus.cache.collections=true") runSqlHive("set datanucleus.cache.collections.lazy=true") loadedTables.clear() - catalog.client.getAllTables("default").foreach(t => { + catalog.client.getAllTables("default").foreach { t => logger.debug(s"Deleting table $t") val table = catalog.client.getTable("default", t) - catalog.client.listIndexes("default", t, 255) - .foreach(i => catalog.client.dropIndex("default", t, i.getIndexName, true)) + catalog.client.listIndexes("default", t, 255).foreach { index => + catalog.client.dropIndex("default", t, index.getIndexName, true) + } - if(!MetaStoreUtils.isIndexTable(table)) + if (!MetaStoreUtils.isIndexTable(table)) { catalog.client.dropTable("default", t) - }) + } + } - catalog.client.getAllDatabases.filterNot(_ == "default").foreach {db => + catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db => logger.debug(s"Dropping Database: $db") catalog.client.dropDatabase(db, true, false, true) } @@ -247,8 +263,9 @@ object TestShark extends SharkInstance { } catch { case e: Exception => logger.error(s"FATAL ERROR: Failed to reset TestDB state. $e") - // At this point there is really no reason to continue, but the test framework traps exits. So instead we just - // pause forever so that at least the developer can see where things started to go wrong. + // At this point there is really no reason to continue, but the test framework traps exits. + // So instead we just pause forever so that at least the developer can see where things + // started to go wrong. Thread.sleep(100000) } } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 8a507c81b6b72..bd2a38eb0903e 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -7,11 +7,15 @@ import catalyst.expressions._ /* Implicits */ import org.apache.spark.SparkContext._ -case class Aggregate(groupingExpressions: Seq[Expression], - aggregateExpressions: Seq[NamedExpression], - child: SharkPlan) extends UnaryNode { +case class Aggregate( + groupingExpressions: Seq[Expression], + aggregateExpressions: Seq[NamedExpression], + child: SharkPlan) + extends UnaryNode { + + case class AverageFunction(expr: Expression, base: AggregateExpression) + extends AggregateFunction { - case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. var count: Long = _ @@ -33,8 +37,9 @@ case class Aggregate(groupingExpressions: Seq[Expression], def apply(input: Seq[Row]): Unit = { val evaluatedExpr = expr.map(Evaluate(_, input)) - if(evaluatedExpr.map(_ != null).reduceLeft(_ || _)) - count += 1 + if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { + count += 1 + } } def result: Any = count @@ -51,7 +56,9 @@ case class Aggregate(groupingExpressions: Seq[Expression], def result: Any = sum } - case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) extends AggregateFunction { + case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) + extends AggregateFunction { + def this() = this(null, null) // Required for serialization. val seen = new scala.collection.mutable.HashSet[Any]() @@ -79,47 +86,49 @@ case class Aggregate(groupingExpressions: Seq[Expression], def output = aggregateExpressions.map(_.toAttribute) def execute() = attachTree(this, "execute") { - val grouped = child.execute().map(row => (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row)).groupByKey() - grouped.map { - case (group, rows) => - // Replace all aggregate expressions with spark functions that will compute the result. - val aggImplementations = aggregateExpressions.map { agg => - val impl = agg transform { - case base @ Average(expr) => new AverageFunction(expr, base) - case base @ Sum(expr) => new SumFunction(expr, base) - case base @ Count(expr) => new CountFunction(expr, base) - case base @ CountDistinct(expr) => new CountDistinctFunction(expr, base) - case base @ First(expr) => new FirstFunction(expr, base) - } + val grouped = child.execute().map { row => + (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row) + }.groupByKey() + + grouped.map { case (group, rows) => + // Replace all aggregate expressions with spark functions that will compute the result. + val aggImplementations = aggregateExpressions.map { agg => + val impl = agg transform { + case base @ Average(expr) => new AverageFunction(expr, base) + case base @ Sum(expr) => new SumFunction(expr, base) + case base @ Count(expr) => new CountFunction(expr, base) + case base @ CountDistinct(expr) => new CountDistinctFunction(expr, base) + case base @ First(expr) => new FirstFunction(expr, base) + } + + val remainingAttributes = impl.collect { case a: Attribute => a } + // If any references exist that are not inside agg functions then the must be grouping exprs + // in this case we must rebind them to the grouping tuple. + if (remainingAttributes.nonEmpty) { + val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } - val remainingAttributes = impl.collect { case a: Attribute => a } - // If any references exist that are not inside agg functions then the must be grouping exprs in this case - // we must rebind them to the grouping tuple. - if(remainingAttributes.nonEmpty) { - val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } - - // An exact match with a grouping expression - val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { - case -1 => None - case ordinal => Some(BoundReference(0, ordinal, Alias(impl, "AGGEXPR")().toAttribute) - ) - } - - exactGroupingExpr.getOrElse(sys.error(s"$agg is not in grouping expressions: $groupingExpressions")) - } else { - impl + // An exact match with a grouping expression + val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { + case -1 => None + case ordinal => Some(BoundReference(0, ordinal, Alias(impl, "AGGEXPR")().toAttribute)) } + + exactGroupingExpr.getOrElse( + sys.error(s"$agg is not in grouping expressions: $groupingExpressions")) + } else { + impl } + } - // Pull out all the functions so we can feed each row into them. - val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f }) - assert(aggFunctions.nonEmpty) + // Pull out all the functions so we can feed each row into them. + val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f }) + assert(aggFunctions.nonEmpty) - rows.foreach { row => - val input = Vector(row) - aggFunctions.foreach(_.apply(input)) - } - buildRow(aggImplementations.map(Evaluate(_, Vector(group)))) + rows.foreach { row => + val input = Vector(row) + aggFunctions.foreach(_.apply(input)) + } + buildRow(aggImplementations.map(Evaluate(_, Vector(group)))) } } } diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 4535ccb708772..e7146b4c9f48e 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -5,9 +5,9 @@ import errors._ import expressions._ import types._ - import org.apache.spark.SparkContext._ + case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -23,7 +23,8 @@ case class Filter(condition: Expression, child: SharkPlan) extends UnaryNode { } } -case class Union(left: SharkPlan, right: SharkPlan)(@transient sc: SharkContext) extends BinaryNode { +case class Union(left: SharkPlan, right: SharkPlan)(@transient sc: SharkContext) + extends BinaryNode { // TODO: attributes output by union should be distinct for nullability purposes def output = left.output // TODO: is it more efficient to union a bunch of rdds at once? should union be variadic? @@ -49,7 +50,7 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { private class SortKey(val keyValues: IndexedSeq[Any]) extends Ordered[SortKey] with Serializable { def compare(other: SortKey): Int = { var i = 0 - while(i < keyValues.size) { + while (i < keyValues.size) { val left = keyValues(i) val right = other.keyValues(i) val curDirection = directions(i) @@ -58,35 +59,33 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { logger.debug(s"Comparing $left, $right as $curDataType order $curDirection") // TODO: Use numeric here too? val comparison = - if(left == null && right == null) + if (left == null && right == null) { 0 - else if(left == null) - if(curDirection == Ascending) - -1 - else - 1 - else if(right == null) - if(curDirection == Ascending) - 1 - else - -1 - else if(curDataType == IntegerType) - if(curDirection == Ascending) + } else if (left == null) { + if (curDirection == Ascending) -1 else 1 + } else if (right == null) { + if (curDirection == Ascending) 1 else -1 + } else if (curDataType == IntegerType) { + if (curDirection == Ascending) { left.asInstanceOf[Int] compare right.asInstanceOf[Int] - else + } else { right.asInstanceOf[Int] compare left.asInstanceOf[Int] - else if(curDataType == DoubleType) - if(curDirection == Ascending) + } + } else if (curDataType == DoubleType) { + if (curDirection == Ascending) { left.asInstanceOf[Double] compare right.asInstanceOf[Double] - else + } else { right.asInstanceOf[Double] compare left.asInstanceOf[Double] - else if(curDataType == StringType) - if(curDirection == Ascending) + } + } else if (curDataType == StringType) { + if (curDirection == Ascending) { left.asInstanceOf[String] compare right.asInstanceOf[String] - else + } else { right.asInstanceOf[String] compare left.asInstanceOf[String] - else + } + } else { sys.error(s"Comparison not yet implemented for: $curDataType") + } if(comparison != 0) return comparison i += 1 @@ -102,7 +101,7 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { val sortKey = new SortKey(sortExprs.map(s => Evaluate(s.child, input)).toIndexedSeq) (sortKey, row) - }.sortByKey(true, numPartitions).map(_._2) + }.sortByKey(ascending = true, numPartitions).map(_._2) } def output = child.output diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index c3c490e611276..9827fae3bf4d7 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -4,16 +4,16 @@ package execution import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils import org.apache.hadoop.hive.ql.plan.FileSinkDesc +import org.apache.hadoop.hive.serde2.Serializer import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, StructObjectInspector} import org.apache.hadoop.hive.serde2.`lazy`.LazyStruct -import org.apache.hadoop.hive.serde2.Serializer import org.apache.hadoop.mapred.JobConf import expressions.Attribute import util._ /* Implicits */ -import collection.JavaConversions._ +import scala.collection.JavaConversions._ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation) extends LeafNode { @transient @@ -31,20 +31,21 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation * Functions that extract the requested attributes from the hive output. */ @transient - protected lazy val attributeFunctions: Seq[(LazyStruct, Array[String]) => AnyRef] = attributes.map { a => - if (relation.partitionKeys.contains(a)) { - val ordinal = relation.partitionKeys.indexOf(a) - (struct: LazyStruct, partitionKeys: Array[String]) => partitionKeys(ordinal) - } else { - val ref = - objectInspector.getAllStructFieldRefs - .find(_.getFieldName == a.name) - .getOrElse(sys.error(s"Can't find attribute $a")) - - (struct: LazyStruct, _: Array[String]) => { - val data = objectInspector.getStructFieldData(struct, ref) - val inspector = ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector] - inspector.getPrimitiveJavaObject(data) + protected lazy val attributeFunctions: Seq[(LazyStruct, Array[String]) => AnyRef] = { + attributes.map { a => + if (relation.partitionKeys.contains(a)) { + val ordinal = relation.partitionKeys.indexOf(a) + (struct: LazyStruct, partitionKeys: Array[String]) => partitionKeys(ordinal) + } else { + val ref = objectInspector.getAllStructFieldRefs + .find(_.getFieldName == a.name) + .getOrElse(sys.error(s"Can't find attribute $a")) + + (struct: LazyStruct, _: Array[String]) => { + val data = objectInspector.getStructFieldData(struct, ref) + val inspector = ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector] + inspector.getPrimitiveJavaObject(data) + } } } } @@ -75,31 +76,37 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation def output = attributes } -case class InsertIntoHiveTable(table: MetastoreRelation, partition: Map[String, String], child: SharkPlan) - (@transient sc: SharkContext) extends UnaryNode { +case class InsertIntoHiveTable( + table: MetastoreRelation, partition: Map[String, String], child: SharkPlan) + (@transient sc: SharkContext) + extends UnaryNode { + /** - * This file sink / record writer code is only the first step towards implementing this operator correctly and is not - * actually used yet. + * This file sink / record writer code is only the first step towards implementing this operator + * correctly and is not actually used yet. */ val desc = new FileSinkDesc("./", table.tableDesc, false) + val outputClass = { val serializer = table.tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] serializer.initialize(null, table.tableDesc.getProperties) serializer.getSerializedClass } - lazy val conf = new JobConf(); + lazy val conf = new JobConf() + lazy val writer = HiveFileFormatUtils.getHiveRecordWriter( conf, table.tableDesc, outputClass, desc, - new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory(), "test.out"), + new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory, "test.out"), null) override def otherCopyArgs = sc :: Nil def output = child.output + def execute() = { val childRdd = child.execute() assert(childRdd != null) @@ -108,12 +115,16 @@ case class InsertIntoHiveTable(table: MetastoreRelation, partition: Map[String, val tempDir = java.io.File.createTempFile("data", "tsv") tempDir.delete() tempDir.mkdir() - childRdd.map(_.map(a => stringOrNull(a.asInstanceOf[AnyRef])).mkString("\001")).saveAsTextFile(tempDir.getCanonicalPath) + childRdd.map(_.map(a => stringOrNull(a.asInstanceOf[AnyRef])).mkString("\001")) + .saveAsTextFile(tempDir.getCanonicalPath) + val partitionSpec = - if(partition.nonEmpty) + if (partition.nonEmpty) { s"PARTITION (${partition.map { case (k,v) => s"$k=$v" }.mkString(",")})" - else + } else { "" + } + sc.runHive(s"LOAD DATA LOCAL INPATH '${tempDir.getCanonicalPath}/*' INTO TABLE ${table.tableName} $partitionSpec") // It would be nice to just return the childRdd unchanged so insert operations could be chained, diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 6fbb8a9893c4b..609b2ea7646ca 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -16,7 +16,8 @@ case class SparkEquiInnerJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SharkPlan, - right: SharkPlan) extends BinaryNode { + right: SharkPlan) + extends BinaryNode { def output = left.output ++ right.output @@ -57,9 +58,13 @@ case class CartesianProduct(left: SharkPlan, right: SharkPlan) extends BinaryNod } } -case class BroadcastNestedLoopJoin(streamed: SharkPlan, broadcast: SharkPlan, joinType: JoinType, condition: Option[Expression]) - (@transient sc: SharkContext) extends BinaryNode { +case class BroadcastNestedLoopJoin( + streamed: SharkPlan, broadcast: SharkPlan, joinType: JoinType, condition: Option[Expression]) + (@transient sc: SharkContext) + extends BinaryNode { + override def otherCopyArgs = sc :: Nil + def output = left.output ++ right.output /** The Streamed Relation */ @@ -73,27 +78,28 @@ case class BroadcastNestedLoopJoin(streamed: SharkPlan, broadcast: SharkPlan, jo val streamedPlusMatches = streamed.execute().map { streamedRow => var i = 0 val matchedRows = new mutable.ArrayBuffer[Row] - val includedBroadcastTuples = new scala.collection.mutable.BitSet(broadcastedRelation.value.size) + val includedBroadcastTuples = new mutable.BitSet(broadcastedRelation.value.size) - while(i < broadcastedRelation.value.size) { + while (i < broadcastedRelation.value.size) { // TODO: One bitset per partition instead of per row. val broadcastedRow = broadcastedRelation.value(i) val includeRow = condition match { case None => true case Some(c) => Evaluate(c, Vector(streamedRow, broadcastedRow)).asInstanceOf[Boolean] } - if(includeRow) { + if (includeRow) { matchedRows += buildRow(streamedRow ++ broadcastedRow) includedBroadcastTuples += i } i += 1 } - val outputRows = if(matchedRows.size > 0) + val outputRows = if (matchedRows.size > 0) { matchedRows - else if(joinType == LeftOuter || joinType == FullOuter) + } else if(joinType == LeftOuter || joinType == FullOuter) { Vector(buildRow(streamedRow ++ Array.fill(right.output.size)(null))) - else + } else { Vector() + } (outputRows, includedBroadcastTuples) } @@ -105,14 +111,15 @@ case class BroadcastNestedLoopJoin(streamed: SharkPlan, broadcast: SharkPlan, jo streamedPlusMatches.map(_._2).reduce(_ ++ _) val rightOuterMatches: Seq[Row] = - if(joinType == RightOuter || joinType == FullOuter) + if (joinType == RightOuter || joinType == FullOuter) { broadcastedRelation.value.zipWithIndex.filter { case (row, i) => !allIncludedBroadcastTuples.contains(i) }.map { case (row, _) => buildRow(Vector.fill(left.output.size)(null) ++ row) } - else + } else { Vector() + } sc.union(streamedPlusMatches.flatMap(_._1), sc.makeRDD(rightOuterMatches)) } diff --git a/src/main/scala/catalyst/execution/package.scala b/src/main/scala/catalyst/execution/package.scala index 2a073f3e49558..c05bdac4881fd 100644 --- a/src/main/scala/catalyst/execution/package.scala +++ b/src/main/scala/catalyst/execution/package.scala @@ -3,13 +3,13 @@ package catalyst import types._ /** - * A partial reimplementation of Shark, a Hive compatible SQL engine running on Spark, using Catalyst. + * A partial reimplementation of Shark, a Hive compatible SQL engine running on Spark. * - * This implementation uses the hive parser, metadata catalog and serdes, but performs all optimization and execution - * using catalyst and spark. + * This implementation uses the hive parser, metadata catalog and serdes, but performs all + * optimization and execution using catalyst and spark. * - * Currently functions that are not supported by this implementation are passed back to the original Shark - * implementation for execution. + * Currently functions that are not supported by this implementation are passed back to the + * original Shark implementation for execution. */ package object execution { type Row = catalyst.expressions.Row diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 2615dfff5108b..5211feb45ee4a 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -49,7 +49,9 @@ trait PlanningStrategies { classOf[Count], classOf[Average]) - /** Returns true if [[exprs]] contains only aggregates that can be computed using Accumulators. */ + /** + * Returns true if [[exprs]] contains only aggregates that can be computed using Accumulators. + */ def onlyAllowedAggregates(exprs: Seq[Expression]): Boolean = { val aggs = exprs.flatMap(_.collect { case a: AggregateExpression => a}).map(_.getClass) aggs.map(allowedAggregates contains _).reduceLeft(_ && _) @@ -66,9 +68,9 @@ trait PlanningStrategies { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case FilteredOperation(predicates, logical.Join(left, right, Inner, condition)) => logger.debug(s"Considering join: ${predicates ++ condition}") - // Find equi-join predicates that can be evaluated before the join, and thus can be used as join keys. - // Note we can only mix in the conditions with other predicates because the match above ensures that this is - // and Inner join. + // Find equi-join predicates that can be evaluated before the join, and thus can be used + // as join keys. Note we can only mix in the conditions with other predicates because the + // match above ensures that this is and Inner join. val (joinPredicates, otherPredicates) = (predicates ++ condition).partition { case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || (canEvaluate(l, right) && canEvaluate(r, left)) => true @@ -81,17 +83,19 @@ trait PlanningStrategies { } // Do not consider this strategy if there are no join keys. - if(joinKeys.nonEmpty) { + if (joinKeys.nonEmpty) { val leftKeys = joinKeys.map(_._1) val rightKeys = joinKeys.map(_._2) - val joinOp = execution.SparkEquiInnerJoin(leftKeys, rightKeys, planLater(left), planLater(right)) + val joinOp = execution.SparkEquiInnerJoin( + leftKeys, rightKeys, planLater(left), planLater(right)) // Make sure other conditions are met if present. - if(otherPredicates.nonEmpty) + if (otherPredicates.nonEmpty) { execution.Filter(combineConjunctivePredicates(otherPredicates), joinOp) :: Nil - else + } else { joinOp :: Nil + } } else { logger.debug(s"Avoiding spark join with no join keys.") Nil @@ -110,14 +114,16 @@ trait PlanningStrategies { object BroadcastNestedLoopJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Join(left, right, joinType, condition) => - execution.BroadcastNestedLoopJoin(planLater(left), planLater(right), joinType, condition)(sc) :: Nil + execution.BroadcastNestedLoopJoin( + planLater(left), planLater(right), joinType, condition)(sc) :: Nil case _ => Nil } } object CartesianProduct extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { - case logical.Join(left, right, _, None) => execution.CartesianProduct(planLater(left), planLater(right)) :: Nil + case logical.Join(left, right, _, None) => + execution.CartesianProduct(planLater(left), planLater(right)) :: Nil case logical.Join(left, right, Inner, Some(condition)) => execution.Filter(condition, execution.CartesianProduct(planLater(left), planLater(right))) :: Nil diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 2a7427cfea7a7..314027fccaede 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -252,7 +252,7 @@ object Evaluate extends Logging { } /* Functions */ - case Rand => scala.util.Random.nextDouble + case Rand => scala.util.Random.nextDouble() /* UDFs */ case implementedFunction: ImplementedUdf => diff --git a/src/main/scala/catalyst/frontend/package.scala b/src/main/scala/catalyst/frontend/package.scala index 70952d1bedade..4d70c4b420d98 100644 --- a/src/main/scala/catalyst/frontend/package.scala +++ b/src/main/scala/catalyst/frontend/package.scala @@ -1,7 +1,7 @@ package catalyst /** - * A collection of languages that can be parsed into a [[plans.logical.LogicalPlan]]. Currently only [[hive]]'s parser is - * supported. + * A collection of languages that can be parsed into a [[catalyst.plans.logical.LogicalPlan]]. + * Currently only [[hive]]'s parser is supported. */ -package object frontend \ No newline at end of file +package object frontend diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index 5b62eddfc8cdf..224e44d857051 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -30,7 +30,8 @@ case class Join( def output = left.output ++ right.output } -case class InsertIntoTable(table: BaseRelation, partition: Map[String, String], child: LogicalPlan) extends LogicalPlan { +case class InsertIntoTable(table: BaseRelation, partition: Map[String, String], child: LogicalPlan) + extends LogicalPlan { // The table being inserted into is a child for the purposes of transformations. def children = table :: child :: Nil def references = Set.empty @@ -50,7 +51,8 @@ case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan) + extends UnaryNode { def output = aggregateExpressions.map(_.toAttribute) def references = child.references From 2924468ce1711fc9bcd40c1c74e7c023edcf8162 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 3 Jan 2014 15:25:50 -0500 Subject: [PATCH 372/778] add two tests for testing pre-order and post-order tree traversal, respectively --- src/test/scala/TreeNodeSuite.scala | 25 ++++++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/src/test/scala/TreeNodeSuite.scala b/src/test/scala/TreeNodeSuite.scala index f38b6c9caa807..d6fec46a8c5a6 100644 --- a/src/test/scala/TreeNodeSuite.scala +++ b/src/test/scala/TreeNodeSuite.scala @@ -1,7 +1,8 @@ package catalyst package trees -import catalyst.types.IntegerType +import collection.mutable.ArrayBuffer + import expressions._ import org.scalatest.{FunSuite} @@ -35,4 +36,26 @@ class TreeNodeSuite extends FunSuite { assert(literals.size === 4) (1 to 4).foreach(i => assert(literals contains Literal(i))) } + + test("pre-order transform") { + val actual = new ArrayBuffer[String]() + val expected = Seq("+", "1", "*", "2", "-", "3", "4") + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + expression transform { + case b: BinaryExpression => {actual.append(b.symbol); b} + case l: Literal => {actual.append(l.toString); l} + } + assert(expected === actual) + } + + test("post-order transform") { + val actual = new ArrayBuffer[String]() + val expected = Seq("1", "2", "3", "4", "-", "*", "+") + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + expression transformPostOrder { + case b: BinaryExpression => {actual.append(b.symbol); b} + case l: Literal => {actual.append(l.toString); l} + } + assert(expected === actual) + } } \ No newline at end of file From 38124bd54ca2c7677821a827afcdb6a2568fde6f Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 3 Jan 2014 15:28:19 -0500 Subject: [PATCH 373/778] formatting --- src/test/scala/TreeNodeSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/test/scala/TreeNodeSuite.scala b/src/test/scala/TreeNodeSuite.scala index d6fec46a8c5a6..46ddaeeaa683b 100644 --- a/src/test/scala/TreeNodeSuite.scala +++ b/src/test/scala/TreeNodeSuite.scala @@ -45,6 +45,7 @@ class TreeNodeSuite extends FunSuite { case b: BinaryExpression => {actual.append(b.symbol); b} case l: Literal => {actual.append(l.toString); l} } + assert(expected === actual) } @@ -56,6 +57,7 @@ class TreeNodeSuite extends FunSuite { case b: BinaryExpression => {actual.append(b.symbol); b} case l: Literal => {actual.append(l.toString); l} } + assert(expected === actual) } } \ No newline at end of file From b367e368544f7f4b03f228e120edc8bfbd9cdffe Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 Jan 2014 14:00:22 -0800 Subject: [PATCH 374/778] Replaced the use of ??? with UnsupportedOperationException. --- src/main/scala/catalyst/expressions/Row.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/main/scala/catalyst/expressions/Row.scala b/src/main/scala/catalyst/expressions/Row.scala index 7a39508d973be..edff359d7fca4 100644 --- a/src/main/scala/catalyst/expressions/Row.scala +++ b/src/main/scala/catalyst/expressions/Row.scala @@ -25,18 +25,18 @@ abstract class Row extends Seq[Any] with Serializable { * A row with no data. Calling any methods will result in an error. Can be used as a placeholder. */ object EmptyRow extends Row { - def apply(i: Int): Any = ??? + def apply(i: Int): Any = throw new UnsupportedOperationException def iterator = Iterator.empty def length = 0 - def isNullAt(i: Int): Boolean = ??? + def isNullAt(i: Int): Boolean = throw new UnsupportedOperationException - def getInt(i: Int): Int = ??? - def getLong(i: Int): Long = ??? - def getDouble(i: Int): Double = ??? - def getBoolean(i: Int): Boolean = ??? - def getShort(i: Int): Short = ??? - def getByte(i: Int): Byte = ??? + def getInt(i: Int): Int = throw new UnsupportedOperationException + def getLong(i: Int): Long = throw new UnsupportedOperationException + def getDouble(i: Int): Double = throw new UnsupportedOperationException + def getBoolean(i: Int): Boolean = throw new UnsupportedOperationException + def getShort(i: Int): Short = throw new UnsupportedOperationException + def getByte(i: Int): Byte = throw new UnsupportedOperationException } /** From f0e1cbf0d3a19f68a7443bcbecaae086bc884f29 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 Jan 2014 14:01:57 -0800 Subject: [PATCH 375/778] Added resolved lazy val to LogicalPlan. --- .../scala/catalyst/analysis/unresolved.scala | 6 ++++-- .../catalyst/execution/MetastoreCatalog.scala | 1 + .../catalyst/expressions/Expression.scala | 12 +++++++++--- .../catalyst/expressions/nullFunctions.scala | 17 ++++++++++------- src/main/scala/catalyst/plans/QueryPlan.scala | 6 +++++- .../catalyst/plans/logical/LogicalPlan.scala | 18 ++++++++++++------ 6 files changed, 41 insertions(+), 19 deletions(-) diff --git a/src/main/scala/catalyst/analysis/unresolved.scala b/src/main/scala/catalyst/analysis/unresolved.scala index 0d4c45dfaa3b2..80b18c281ec14 100644 --- a/src/main/scala/catalyst/analysis/unresolved.scala +++ b/src/main/scala/catalyst/analysis/unresolved.scala @@ -6,7 +6,8 @@ import plans.logical.BaseRelation import trees.TreeNode /** - * Thrown when an invalid attempt is made to access a property of a tree that has yet to be fully resolved. + * Thrown when an invalid attempt is made to access a property of a tree that has yet to be fully + * resolved. */ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: String) extends errors.OptimizationException(tree, s"Invalid call to $function on unresolved object") @@ -16,6 +17,7 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str */ case class UnresolvedRelation(name: String, alias: Option[String] = None) extends BaseRelation { def output = Nil + override lazy val resolved = false } /** @@ -82,4 +84,4 @@ case class Star( } override def toString = table.map(_ + ".").getOrElse("") + "*" -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 33f1abeaaed72..b18689614b242 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -28,6 +28,7 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { val hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) val partitions = if (hiveQlTable.isPartitioned) { + // TODO: Is 255 the right number to pick? client.listPartitions(databaseName, tableName, 255).toSeq } else { Nil diff --git a/src/main/scala/catalyst/expressions/Expression.scala b/src/main/scala/catalyst/expressions/Expression.scala index db2c95aecd673..eaed3c9a494d8 100644 --- a/src/main/scala/catalyst/expressions/Expression.scala +++ b/src/main/scala/catalyst/expressions/Expression.scala @@ -12,12 +12,17 @@ abstract class Expression extends TreeNode[Expression] { def references: Set[Attribute] /** - * Returns true if this expression and all its children have been resolved to a specific schema and false if it is - * still contains any unresolved placeholders. + * Returns true if this expression and all its children have been resolved to a specific schema + * and false if it is still contains any unresolved placeholders. Implementations of expressions + * should override this. */ lazy val resolved: Boolean = childrenResolved - def childrenResolved = children.map(_.resolved).reduceLeftOption(_&&_).getOrElse(true) + /** + * Returns true if all the children of this expression have been resolved to a specific schema + * and false if it is still contains any unresolved placeholders. + */ + def childrenResolved = !children.exists(!_.resolved) } abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] { @@ -36,5 +41,6 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression] abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { self: Product => + def references = child.references } diff --git a/src/main/scala/catalyst/expressions/nullFunctions.scala b/src/main/scala/catalyst/expressions/nullFunctions.scala index f9308ab0c5c67..cab75526539f4 100644 --- a/src/main/scala/catalyst/expressions/nullFunctions.scala +++ b/src/main/scala/catalyst/expressions/nullFunctions.scala @@ -4,7 +4,10 @@ package expressions import catalyst.analysis.UnresolvedException case class Coalesce(children: Seq[Expression]) extends Expression { - def nullable = children.map(_.nullable).reduce(_&&_) + + /** Coalesce is nullable if all of its children are nullable, or if it has no children. */ + def nullable = !children.exists(!_.nullable) + def references = children.flatMap(_.references).toSet // Only resolved if all the children are of the same type. @@ -12,9 +15,9 @@ case class Coalesce(children: Seq[Expression]) extends Expression { override def toString = s"Coalesce(${children.mkString(",")})" - def dataType = - if(resolved) - children.head.dataType - else - throw new UnresolvedException(this, "Coalesce cannot have children of different types.") -} \ No newline at end of file + def dataType = if (resolved) { + children.head.dataType + } else { + throw new UnresolvedException(this, "Coalesce cannot have children of different types.") + } +} diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index c099a777ef4aa..54d54e42164aa 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -9,10 +9,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy def output: Seq[Attribute] + /** + * Returns the set of attributes that are output by this node. + */ + def outputSet: Set[Attribute] = output.toSet + /** * Runs [[transform]] with [[rule]] on all expressions present in this query operator. * @param rule the rule to be applied to every expression in this operator. - * @return */ def transformExpressions(rule: PartialFunction[Expression, Expression]): this.type = { var changed = false diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala index fdd1428601cf6..8f6dca8a80a59 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala @@ -4,7 +4,6 @@ package logical import expressions.Attribute import errors._ -import trees._ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => @@ -15,17 +14,24 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { */ def references: Set[Attribute] - /** - * Returns the set of attributes that are output by this node. - */ - def outputSet: Set[Attribute] = output.toSet - /** * Returns the set of attributes that this node takes as * input from its children. */ def inputSet: Set[Attribute] = children.flatMap(_.output).toSet + /** + * Returns true if this expression and all its children have been resolved to a specific schema + * and false if it is still contains any unresolved placeholders. Implementations of LogicalPlan + * can override this (e.g. [[UnresolvedRelation]] can set this to false). + */ + lazy val resolved: Boolean = !expressions.exists(!_.resolved) && childrenResolved + + /** + * Returns true if all its children of this query plan have been resolved. + */ + def childrenResolved = !children.exists(!_.resolved) + def resolve(name: String): Option[Attribute] = { val parts = name.split("\\.") val options = children.flatMap(_.output).filter { option => From 52864da52a647599c8d2b6c6ba411675b3a295f4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 Jan 2014 14:08:35 -0800 Subject: [PATCH 376/778] Added executeCollect method to SharkPlan. --- src/main/scala/catalyst/execution/SharkPlan.scala | 8 +++++++- src/main/scala/catalyst/execution/aggregates.scala | 8 +++++--- src/main/scala/catalyst/execution/basicOperators.scala | 6 +++++- 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 0f74b2147104d..6116d8397b090 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -1,9 +1,10 @@ package catalyst package execution -import catalyst.plans.QueryPlan import org.apache.spark.rdd.RDD +import catalyst.plans.QueryPlan + abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => @@ -12,6 +13,11 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { */ def execute(): RDD[Row] + /** + * Runs this query returning the result as an array. + */ + def executeCollect(): Array[Row] = execute().collect() + protected def buildRow(values: Seq[Any]): Row = new catalyst.expressions.GenericRow(values) } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index bd2a38eb0903e..3e5e65f526c7c 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -169,7 +169,7 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan def result: Any = count.value.toLong } - def execute() = attachTree(this, "SparkAggregate") { + override def executeCollect() = attachTree(this, "SparkAggregate") { // Replace all aggregate expressions with spark functions that will compute the result. val aggImplementations = aggregateExprs.map { _ transform { case base @ Average(expr) => new AverageFunction(expr, base) @@ -185,6 +185,8 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan val input = Vector(row) aggFunctions.foreach(_.apply(input)) } - sc.makeRDD(Seq(buildRow(aggImplementations.map(Evaluate(_, Nil)))), 1) + Array(buildRow(aggImplementations.map(Evaluate(_, Nil)))) } -} \ No newline at end of file + + def execute() = sc.makeRDD(executeCollect(), 1) +} diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index e7146b4c9f48e..2dee6b01e2fc5 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -37,8 +37,12 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) override def otherCopyArgs = sc :: Nil def output = child.output + + override def executeCollect() = child.execute().take(limit) + + // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. - def execute() = sc.makeRDD(child.execute().take(limit),1) + def execute() = sc.makeRDD(executeCollect(), 1) } case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { From e942da1f2e77fc80087d5ae92d4f244713481609 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 2 Jan 2014 19:21:25 -0800 Subject: [PATCH 377/778] Begin upgrade to Hive 0.12.0. --- build.sbt | 6 +++--- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/build.sbt b/build.sbt index 963f7de8a4735..1434a4a56b6ce 100644 --- a/build.sbt +++ b/build.sbt @@ -19,9 +19,9 @@ libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client" % "1.0.4", "org.scalatest" %% "scalatest" % "1.9.1" % "test", //"net.hydromatic" % "optiq-core" % "0.4.16-SNAPSHOT", - "org.apache.hive" % "hive-metastore" % "0.10.0", - "org.apache.hive" % "hive-exec" % "0.10.0", - "org.apache.hive" % "hive-builtins" % "0.10.0", + "org.apache.hive" % "hive-metastore" % "0.12.0", + "org.apache.hive" % "hive-exec" % "0.12.0", + "org.apache.hive" % "hive-serde" % "0.12.0", "com.typesafe" %% "scalalogging-slf4j" % "1.0.1") // Multiple queries rely on the TestShark singleton. See comments there for more details. diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index b18689614b242..2ec45e8f4d021 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -100,7 +100,7 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt Class.forName(table.getSd.getSerdeInfo.getSerializationLib).asInstanceOf[Class[Deserializer]], Class.forName(table.getSd.getInputFormat).asInstanceOf[Class[InputFormat[_,_]]], Class.forName(table.getSd.getOutputFormat), - hiveQlTable.getSchema + hiveQlTable.getMetadata() ) implicit class SchemaAttribute(f: FieldSchema) { From 6a7bd75f2487ecaca152c390dda2821c07d6d782 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 3 Jan 2014 12:46:22 -0800 Subject: [PATCH 378/778] fix partition column delimiter configuration. --- src/main/scala/catalyst/execution/TableReader.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/main/scala/catalyst/execution/TableReader.scala b/src/main/scala/catalyst/execution/TableReader.scala index 58b0c8f502591..47ca07c884e27 100644 --- a/src/main/scala/catalyst/execution/TableReader.scala +++ b/src/main/scala/catalyst/execution/TableReader.scala @@ -1,6 +1,7 @@ package catalyst.execution import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.conf.HiveConf @@ -132,9 +133,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf val partSpec = partDesc.getPartSpec() val partProps = partDesc.getProperties() - // TODO: fix this. - //val partColsDelimited: String = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) - val partColsDelimited = ";" + val partColsDelimited: String = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) // Partitioning columns are delimited by "/" val partCols = partColsDelimited.trim().split("/").toSeq // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. From 0bd1688f63b55c62cf778de627ff332b75686f8c Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 3 Jan 2014 18:05:36 -0500 Subject: [PATCH 379/778] update --- src/main/scala/analysis/unresolved.scala | 1 + src/main/scala/expressions/Cast.scala | 1 + src/main/scala/expressions/Expression.scala | 3 ++ src/main/scala/expressions/Rand.scala | 1 + src/main/scala/expressions/ScalaUdf.scala | 1 + src/main/scala/expressions/SortOrder.scala | 3 +- src/main/scala/expressions/WrapDynamic.scala | 1 + src/main/scala/expressions/aggregates.scala | 2 + src/main/scala/expressions/arithmetic.scala | 7 +-- src/main/scala/expressions/literals.scala | 2 + .../scala/expressions/namedExpressions.scala | 2 + .../scala/expressions/nullFunctions.scala | 1 + src/main/scala/expressions/predicates.scala | 4 +- src/main/scala/optimizer/Optimizer.scala | 42 ++++------------- src/main/scala/shark2/FunctionRegistry.scala | 1 + src/main/scala/types/dataTypes.scala | 1 + src/test/scala/OptimizerSuite.scala | 47 +++++++++++++++---- 17 files changed, 71 insertions(+), 49 deletions(-) diff --git a/src/main/scala/analysis/unresolved.scala b/src/main/scala/analysis/unresolved.scala index ac942390bd8c6..674f083c1026e 100644 --- a/src/main/scala/analysis/unresolved.scala +++ b/src/main/scala/analysis/unresolved.scala @@ -37,6 +37,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { def exprId = throw new UnresolvedException(this, "exprId") def dataType = throw new UnresolvedException(this, "dataType") + def foldable = throw new UnresolvedException(this, "foldable") def nullable = throw new UnresolvedException(this, "nullable") def qualifiers = throw new UnresolvedException(this, "qualifiers") def references = children.flatMap(_.references).toSet diff --git a/src/main/scala/expressions/Cast.scala b/src/main/scala/expressions/Cast.scala index 297933581671b..0bc213e822a7c 100644 --- a/src/main/scala/expressions/Cast.scala +++ b/src/main/scala/expressions/Cast.scala @@ -4,6 +4,7 @@ package expressions import types.DataType case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { + def foldable = child.foldable def nullable = child.nullable override def toString = s"CAST($child, $dataType)" } \ No newline at end of file diff --git a/src/main/scala/expressions/Expression.scala b/src/main/scala/expressions/Expression.scala index db2c95aecd673..c236412f96f6f 100644 --- a/src/main/scala/expressions/Expression.scala +++ b/src/main/scala/expressions/Expression.scala @@ -8,6 +8,7 @@ abstract class Expression extends TreeNode[Expression] { self: Product => def dataType: DataType + def foldable: Boolean def nullable: Boolean def references: Set[Attribute] @@ -25,6 +26,8 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express def symbol: String + def foldable = left.foldable && right.foldable + def references = left.references ++ right.references override def toString = s"($left $symbol $right)" diff --git a/src/main/scala/expressions/Rand.scala b/src/main/scala/expressions/Rand.scala index b6a0a9cd080a5..8368e5923d5ad 100644 --- a/src/main/scala/expressions/Rand.scala +++ b/src/main/scala/expressions/Rand.scala @@ -5,6 +5,7 @@ import types.DoubleType case object Rand extends LeafExpression { def dataType = DoubleType + def foldable = false def nullable = false def references = Set.empty override def toString = "RAND()" diff --git a/src/main/scala/expressions/ScalaUdf.scala b/src/main/scala/expressions/ScalaUdf.scala index 6681e17c01a75..b0a9b7ec2a3a6 100644 --- a/src/main/scala/expressions/ScalaUdf.scala +++ b/src/main/scala/expressions/ScalaUdf.scala @@ -7,6 +7,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi extends Expression with ImplementedUdf { def references = children.flatMap(_.references).toSet + def foldable = false def nullable = true def evaluate(evaluatedChildren: Seq[Any]): Any = { diff --git a/src/main/scala/expressions/SortOrder.scala b/src/main/scala/expressions/SortOrder.scala index cfce25cf1475f..b1a56186db4d6 100644 --- a/src/main/scala/expressions/SortOrder.scala +++ b/src/main/scala/expressions/SortOrder.scala @@ -1,8 +1,6 @@ package catalyst package expressions -import types._ - abstract sealed class SortDirection case object Ascending extends SortDirection case object Descending extends SortDirection @@ -13,6 +11,7 @@ case object Descending extends SortDirection */ case class SortOrder(child: Expression, direction: SortDirection) extends UnaryExpression { def dataType = child.dataType + def foldable = false def nullable = child.nullable override def toString = s"$child ${if(direction == Ascending) "ASC" else "DESC"}" } \ No newline at end of file diff --git a/src/main/scala/expressions/WrapDynamic.scala b/src/main/scala/expressions/WrapDynamic.scala index 128d3de9cb577..977b3e4e25e9c 100644 --- a/src/main/scala/expressions/WrapDynamic.scala +++ b/src/main/scala/expressions/WrapDynamic.scala @@ -9,6 +9,7 @@ case object DynamicType extends DataType case class WrapDynamic(children: Seq[Attribute]) extends Expression with ImplementedUdf { def nullable = false + def foldable = false def references = children.toSet def dataType = DynamicType diff --git a/src/main/scala/expressions/aggregates.scala b/src/main/scala/expressions/aggregates.scala index 1422d1bd7aec6..5c7bc806da83a 100644 --- a/src/main/scala/expressions/aggregates.scala +++ b/src/main/scala/expressions/aggregates.scala @@ -5,6 +5,8 @@ import catalyst.types._ abstract class AggregateExpression extends Expression { self: Product => + + def foldable = false } /** diff --git a/src/main/scala/expressions/arithmetic.scala b/src/main/scala/expressions/arithmetic.scala index 1262d305aba1d..51acf6d4951ea 100644 --- a/src/main/scala/expressions/arithmetic.scala +++ b/src/main/scala/expressions/arithmetic.scala @@ -6,12 +6,14 @@ import catalyst.analysis.UnresolvedException case class UnaryMinus(child: Expression) extends UnaryExpression { def dataType = child.dataType + def foldable = child.foldable def nullable = child.nullable override def toString = s"-$child" } abstract class BinaryArithmetic extends BinaryExpression { self: Product => + def nullable = left.nullable || right.nullable override lazy val resolved = left.resolved && right.resolved && left.dataType == right.dataType @@ -26,25 +28,20 @@ abstract class BinaryArithmetic extends BinaryExpression { case class Add(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "+" - def nullable = left.nullable || right.nullable } case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "-" - def nullable = left.nullable || right.nullable } case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "*" - def nullable = left.nullable || right.nullable } case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "/" - def nullable = left.nullable || right.nullable } case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "%" - def nullable = left.nullable || right.nullable } \ No newline at end of file diff --git a/src/main/scala/expressions/literals.scala b/src/main/scala/expressions/literals.scala index c61071befa9e8..10362bb12e603 100644 --- a/src/main/scala/expressions/literals.scala +++ b/src/main/scala/expressions/literals.scala @@ -13,6 +13,7 @@ object Literal { case s: Short => Literal(s, ShortType) case s: String => Literal(s, StringType) case b: Boolean => Literal(b, BooleanType) + case null => Literal(null, NullType) } } @@ -27,6 +28,7 @@ object IntegerLiteral { } case class Literal(value: Any, dataType: DataType) extends LeafExpression { + def foldable = true def nullable = false def references = Set.empty diff --git a/src/main/scala/expressions/namedExpressions.scala b/src/main/scala/expressions/namedExpressions.scala index b3498d7575f84..d808ea0352f78 100644 --- a/src/main/scala/expressions/namedExpressions.scala +++ b/src/main/scala/expressions/namedExpressions.scala @@ -19,6 +19,8 @@ case class ExprId(id: Long) abstract class NamedExpression extends Expression { self: Product => + def foldable = false + def name: String def exprId: ExprId def qualifiers: Seq[String] diff --git a/src/main/scala/expressions/nullFunctions.scala b/src/main/scala/expressions/nullFunctions.scala index f9308ab0c5c67..e58bf912d435b 100644 --- a/src/main/scala/expressions/nullFunctions.scala +++ b/src/main/scala/expressions/nullFunctions.scala @@ -6,6 +6,7 @@ import catalyst.analysis.UnresolvedException case class Coalesce(children: Seq[Expression]) extends Expression { def nullable = children.map(_.nullable).reduce(_&&_) def references = children.flatMap(_.references).toSet + def foldable = children.map(_.foldable).reduce(_&&_) // Only resolved if all the children are of the same type. override lazy val resolved = childrenResolved && (children.map(_.dataType).distinct.size == 1) diff --git a/src/main/scala/expressions/predicates.scala b/src/main/scala/expressions/predicates.scala index cbb543a32d131..fea45d2af467c 100644 --- a/src/main/scala/expressions/predicates.scala +++ b/src/main/scala/expressions/predicates.scala @@ -11,12 +11,12 @@ abstract trait Predicate extends Expression { abstract class BinaryPredicate extends BinaryExpression with Predicate { self: Product => - def nullable = left.nullable || right.nullable } case class Not(child: Expression) extends Predicate with trees.UnaryNode[Expression]{ def references = child.references + def foldable = child.foldable def nullable = child.nullable override def toString = s"NOT $child" } @@ -50,9 +50,11 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { def references = child.references + def foldable = child.foldable def nullable = false } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { def references = child.references + def foldable = child.foldable def nullable = false } diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index 9abc64e978d22..413bec7e5a242 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -10,10 +10,9 @@ object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Subqueries", Once, EliminateSubqueries) :: - Batch("EvaluateLiterals", Once, - EvaluateLiterals, - EvaluateLiteralsInAndOr - ) :: Nil + Batch("ConstantFolding", Once, + ConstantFolding + ) :: Nil } object EliminateSubqueries extends Rule[LogicalPlan] { @@ -22,50 +21,29 @@ object EliminateSubqueries extends Rule[LogicalPlan] { } } -/* -* An optimization rule to evaluate literals appearing in expressions. -* It traverses the expressions in a post order to visit BinaryExpression. -* When it finds both the left child and right child of a node are literals, -* it evaluates the current visiting BinaryExpression. -* Because, currently, we evaluate literals based on the structure of the expression -* tree, key+1+1 will not be transformed to key+2. -* */ -object EvaluateLiterals extends Rule[LogicalPlan] { +// Do I really need post order traversal at here???? +object ConstantFolding extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsPostOrder { - case b: BinaryExpression - if b.left.isInstanceOf[Literal] && b.right.isInstanceOf[Literal] => { - Literal(Evaluate(b, Nil)) - } - } - } -} - -/* -* After EvaluateLiterals, for an And or an Or, if either side of -* this predicate is a Literal Boolean, we can further evaluate this predicate. -* */ -object EvaluateLiteralsInAndOr extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case q: LogicalPlan => q transformExpressionsPostOrder { - case b @ And(left, right) => { + case a @ And(left, right) if !a.foldable => { (left, right) match { case (Literal(true, BooleanType), r) => r case (l, Literal(true, BooleanType)) => l case (Literal(false, BooleanType), _) => Literal(false) case (_, Literal(false, BooleanType)) => Literal(false) - case (_, _) => b + case (_, _) => a } } - case b @ Or(left, right) => { + case o @ Or(left, right) if !o.foldable => { (left, right) match { case (Literal(true, BooleanType), _) => Literal(true) case (_, Literal(true, BooleanType)) => Literal(true) case (Literal(false, BooleanType), r) => r case (l, Literal(false, BooleanType)) => l - case (_, _) => b + case (_, _) => o } } + case e if e.foldable => {Literal(Evaluate(e, Nil))} } } } \ No newline at end of file diff --git a/src/main/scala/shark2/FunctionRegistry.scala b/src/main/scala/shark2/FunctionRegistry.scala index cc35674a29d4e..90a41644e00e8 100644 --- a/src/main/scala/shark2/FunctionRegistry.scala +++ b/src/main/scala/shark2/FunctionRegistry.scala @@ -70,6 +70,7 @@ abstract class HiveUdf extends Expression with ImplementedUdf with Logging { type UDFType val name: String + def foldable = false def nullable = true def references = children.flatMap(_.references).toSet diff --git a/src/main/scala/types/dataTypes.scala b/src/main/scala/types/dataTypes.scala index 4decfb607ac62..92f683d5e3ab6 100644 --- a/src/main/scala/types/dataTypes.scala +++ b/src/main/scala/types/dataTypes.scala @@ -13,6 +13,7 @@ case object DoubleType extends DataType case object LongType extends DataType case object ByteType extends DataType case object ShortType extends DataType +case object NullType extends DataType case class ArrayType(elementType: DataType) extends DataType diff --git a/src/test/scala/OptimizerSuite.scala b/src/test/scala/OptimizerSuite.scala index 4da846caf1dc2..a6188152dde27 100644 --- a/src/test/scala/OptimizerSuite.scala +++ b/src/test/scala/OptimizerSuite.scala @@ -2,11 +2,10 @@ package catalyst import org.scalatest.FunSuite -import analysis._ import expressions._ import optimizer.Optimize import plans.logical._ -import types._ +import catalyst.types.{DoubleType, IntegerType} import util._ /* Implicit conversions for creating query plans */ @@ -61,7 +60,7 @@ class OptimizerSuite extends FunSuite { /* * Unit tests for evaluating literals in expressions. * */ - test("Evaluating Literals Test: expressions only have literals") { + test("Constant folding test: expressions only have literals") { val originalQuery = testRelation .select(Literal(2) + Literal(3) + Literal(4) as Symbol("2+3+4"), @@ -86,7 +85,7 @@ class OptimizerSuite extends FunSuite { comparePlans(optimized, correctAnswer) } - test("Evaluating Literals Test: expressions have attribute references and literals in" + + test("Constant folding test: expressions have attribute references and literals in" + "arithmetic operations") { val originalQuery = testRelation @@ -108,7 +107,7 @@ class OptimizerSuite extends FunSuite { comparePlans(optimized, correctAnswer) } - test("Evaluating Literals Test: expressions have attribute references and literals in" + + test("Constant folding test: expressions have attribute references and literals in" + "predicates") { val originalQuery = testRelation @@ -121,12 +120,8 @@ class OptimizerSuite extends FunSuite { (Literal(1) === Literal(1) || 'b > 1) && (Literal(1) === Literal(2) || 'b < 10))) - println(originalQuery.analyze) - val optimized = Optimize(originalQuery.analyze) - println(optimized) - val correctAnswer = testRelation .where(('a > 1 || @@ -137,4 +132,38 @@ class OptimizerSuite extends FunSuite { comparePlans(optimized, correctAnswer) } + + test("Constant folding test: expressions have foldable functions") { + val originalQuery = + testRelation + .select(Cast(Literal("2"), IntegerType) + Literal(3) + 'a as Symbol("c1"), + Coalesce(Seq(Cast(Literal("abc"), IntegerType), Literal(3))) as Symbol("c2")) + + val optimized = Optimize(originalQuery.analyze) + + val correctAnswer = + testRelation + .select(Literal(5) + 'a as Symbol("c1"), + Literal(3) as Symbol("c2")) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("Constant folding test: expressions have nonfoldable functions") { + val originalQuery = + testRelation + .select(Rand + Literal(1) as Symbol("c1"), + Sum('a) as Symbol("c2")) + + val optimized = Optimize(originalQuery.analyze) + + val correctAnswer = + testRelation + .select(Rand + Literal(1.0) as Symbol("c1"), + Sum('a) as Symbol("c2")) + .analyze + + comparePlans(optimized, correctAnswer) + } } \ No newline at end of file From e00372835321a8901854777305709e7498e9625c Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 3 Jan 2014 19:31:24 -0500 Subject: [PATCH 380/778] move OptimizerSuite to the package of catalyst.optimizer --- .../{ => catalyst/optimizer}/OptimizerSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) rename src/test/scala/{ => catalyst/optimizer}/OptimizerSuite.scala (97%) diff --git a/src/test/scala/OptimizerSuite.scala b/src/test/scala/catalyst/optimizer/OptimizerSuite.scala similarity index 97% rename from src/test/scala/OptimizerSuite.scala rename to src/test/scala/catalyst/optimizer/OptimizerSuite.scala index a6188152dde27..5ac479ca2e381 100644 --- a/src/test/scala/OptimizerSuite.scala +++ b/src/test/scala/catalyst/optimizer/OptimizerSuite.scala @@ -1,15 +1,15 @@ package catalyst +package optimizer import org.scalatest.FunSuite -import expressions._ -import optimizer.Optimize -import plans.logical._ -import catalyst.types.{DoubleType, IntegerType} +import types.IntegerType import util._ +import plans.logical.{LogicalPlan, LocalRelation} +import expressions._ +import dsl._ /* Implicit conversions for creating query plans */ -import dsl._ class OptimizerSuite extends FunSuite { // Test relations. Feel free to create more. @@ -161,7 +161,7 @@ class OptimizerSuite extends FunSuite { val correctAnswer = testRelation .select(Rand + Literal(1.0) as Symbol("c1"), - Sum('a) as Symbol("c2")) + Sum('a) as Symbol("c2")) .analyze comparePlans(optimized, correctAnswer) From 2795f05a3da782b4df0532a97af5e98474e7c9b9 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 3 Jan 2014 19:38:00 -0500 Subject: [PATCH 381/778] comment --- src/main/scala/catalyst/optimizer/Optimizer.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala index 413bec7e5a242..db09b9b54c58b 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/src/main/scala/catalyst/optimizer/Optimizer.scala @@ -21,7 +21,6 @@ object EliminateSubqueries extends Rule[LogicalPlan] { } } -// Do I really need post order traversal at here???? object ConstantFolding extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsPostOrder { From 8d47ed4faef6d925fd7a3d75afde0bdd174c74f8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 3 Jan 2014 19:53:23 -0500 Subject: [PATCH 382/778] comment --- src/test/scala/catalyst/optimizer/OptimizerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/catalyst/optimizer/OptimizerSuite.scala b/src/test/scala/catalyst/optimizer/OptimizerSuite.scala index 5ac479ca2e381..5d202fa75b50c 100644 --- a/src/test/scala/catalyst/optimizer/OptimizerSuite.scala +++ b/src/test/scala/catalyst/optimizer/OptimizerSuite.scala @@ -58,7 +58,7 @@ class OptimizerSuite extends FunSuite { } /* - * Unit tests for evaluating literals in expressions. + * Unit tests for constant folding in expressions. * */ test("Constant folding test: expressions only have literals") { val originalQuery = From 46bf41b66ca6c6a0a6848ea0f33b7b0234cc2142 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 3 Jan 2014 17:35:51 -0800 Subject: [PATCH 383/778] add a makefile for priming the test answer cache in parallel. usage: "make -j 8 -i" --- Makefile | 54 ++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 54 insertions(+) create mode 100644 Makefile diff --git a/Makefile b/Makefile new file mode 100644 index 0000000000000..e763516432300 --- /dev/null +++ b/Makefile @@ -0,0 +1,54 @@ +all: a b c d e f g h i j k l m n o p q r s t u v w x y" z + +a: + sbt -Dshark.hive.whitelist=a.* "test-only catalyst.execution.HiveCompatability" +b: + sbt -Dshark.hive.whitelist=b.* "test-only catalyst.execution.HiveCompatability" +c: + sbt -Dshark.hive.whitelist=c.* "test-only catalyst.execution.HiveCompatability" +d: + sbt -Dshark.hive.whitelist=d.* "test-only catalyst.execution.HiveCompatability" +e: + sbt -Dshark.hive.whitelist=e.* "test-only catalyst.execution.HiveCompatability" +f: + sbt -Dshark.hive.whitelist=f.* "test-only catalyst.execution.HiveCompatability" +g: + sbt -Dshark.hive.whitelist=g.* "test-only catalyst.execution.HiveCompatability" +h: + sbt -Dshark.hive.whitelist=h.* "test-only catalyst.execution.HiveCompatability" +i: + sbt -Dshark.hive.whitelist=i.* "test-only catalyst.execution.HiveCompatability" +j: + sbt -Dshark.hive.whitelist=j.* "test-only catalyst.execution.HiveCompatability" +k: + sbt -Dshark.hive.whitelist=k.* "test-only catalyst.execution.HiveCompatability" +l: + sbt -Dshark.hive.whitelist=l.* "test-only catalyst.execution.HiveCompatability" +m: + sbt -Dshark.hive.whitelist=m.* "test-only catalyst.execution.HiveCompatability" +n: + sbt -Dshark.hive.whitelist=n.* "test-only catalyst.execution.HiveCompatability" +o: + sbt -Dshark.hive.whitelist=o.* "test-only catalyst.execution.HiveCompatability" +p: + sbt -Dshark.hive.whitelist=p.* "test-only catalyst.execution.HiveCompatability" +q: + sbt -Dshark.hive.whitelist=q.* "test-only catalyst.execution.HiveCompatability" +r: + sbt -Dshark.hive.whitelist=r.* "test-only catalyst.execution.HiveCompatability" +s: + sbt -Dshark.hive.whitelist=s.* "test-only catalyst.execution.HiveCompatability" +t: + sbt -Dshark.hive.whitelist=t.* "test-only catalyst.execution.HiveCompatability" +u: + sbt -Dshark.hive.whitelist=u.* "test-only catalyst.execution.HiveCompatability" +v: + sbt -Dshark.hive.whitelist=v.* "test-only catalyst.execution.HiveCompatability" +w: + sbt -Dshark.hive.whitelist=w.* "test-only catalyst.execution.HiveCompatability" +x: + sbt -Dshark.hive.whitelist=x.* "test-only catalyst.execution.HiveCompatability" +y: + sbt -Dshark.hive.whitelist=y.* "test-only catalyst.execution.HiveCompatability" +z: + sbt -Dshark.hive.whitelist=z.* "test-only catalyst.execution.HiveCompatability" From 6f954eebd46f641f464fbdfc795929d956a03be9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 3 Jan 2014 17:36:20 -0800 Subject: [PATCH 384/778] export the hadoop classpath when starting sbt, required to invoke hive during tests. --- sbt/sbt | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/sbt/sbt b/sbt/sbt index f15c3539d85b8..b89cdfb5748d2 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,9 +1,12 @@ #!/usr/bin/env bash -SHARK_CONF_DIR=`dirname $0`/../conf -if [ -e $SHARK_CONF_DIR/shark-env.sh ] ; then - . $SHARK_CONF_DIR/shark-env.sh -fi +HADOOP_CLASSPATH="" + +for i in $HIVE_HOME/lib/* +do HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$i +done + +export HADOOP_CLASSPATH realpath () { ( From 138f53df5c98f0c24806528b83d19492866c85e7 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 3 Jan 2014 23:03:05 -0500 Subject: [PATCH 385/778] addressed comments and added a space after a space after the defining keyword of every control structure. --- README.md | 4 +- .../scala/catalyst/analysis/Analyzer.scala | 18 ++++----- .../catalyst/analysis/typeCoercion.scala | 8 ++-- .../scala/catalyst/analysis/unresolved.scala | 2 +- src/main/scala/catalyst/errors/package.scala | 2 +- .../catalyst/examples/ViewsExample.scala | 2 +- .../catalyst/execution/FunctionRegistry.scala | 1 - .../catalyst/execution/MetastoreCatalog.scala | 2 +- .../scala/catalyst/execution/TestShark.scala | 2 +- .../scala/catalyst/execution/Transform.scala | 2 +- .../scala/catalyst/execution/aggregates.scala | 12 +++--- .../catalyst/execution/basicOperators.scala | 2 +- src/main/scala/catalyst/execution/debug.scala | 2 +- src/main/scala/catalyst/execution/joins.scala | 4 +- .../catalyst/expressions/BoundAttribute.scala | 8 ++-- .../scala/catalyst/expressions/Cast.scala | 2 +- .../scala/catalyst/expressions/Evaluate.scala | 22 +++++----- .../catalyst/expressions/Expression.scala | 14 ++++++- .../scala/catalyst/expressions/Rand.scala | 1 - .../scala/catalyst/expressions/ScalaUdf.scala | 1 - .../catalyst/expressions/SortOrder.scala | 3 +- .../catalyst/expressions/WrapDynamic.scala | 1 - .../catalyst/expressions/aggregates.scala | 1 - .../catalyst/expressions/arithmetic.scala | 4 +- .../scala/catalyst/expressions/literals.scala | 4 +- .../expressions/namedExpressions.scala | 6 +-- .../catalyst/expressions/nullFunctions.scala | 3 +- .../catalyst/expressions/predicates.scala | 6 +-- src/main/scala/catalyst/frontend/Hive.scala | 22 +++++----- .../scala/catalyst/optimizer/Optimizer.scala | 22 ++++++---- src/main/scala/catalyst/plans/QueryPlan.scala | 39 +++++++++--------- .../catalyst/plans/logical/TestRelation.scala | 2 +- src/main/scala/catalyst/trees/TreeNode.scala | 40 +++++++++---------- src/main/scala/catalyst/util/package.scala | 8 ++-- .../catalyst/ExpressionEvaluationSuite.scala | 4 +- .../scala/catalyst/RuleExecutorSuite.scala | 2 +- src/test/scala/catalyst/TreeNodeSuite.scala | 12 +++--- .../catalyst/execution/BigDataBenchmark.scala | 2 +- .../catalyst/execution/DslQueryTests.scala | 2 +- .../execution/HiveComparisionTest.scala | 14 +++---- .../execution/HiveQueryFileTest.scala | 4 +- .../catalyst/optimizer/OptimizerSuite.scala | 12 +++--- 42 files changed, 168 insertions(+), 156 deletions(-) diff --git a/README.md b/README.md index fb7b0d5aeef6e..b6a2354c0c4b1 100644 --- a/README.md +++ b/README.md @@ -73,7 +73,7 @@ Project {key#0,value#1} HiveTableScan {key#0,value#1}, (MetastoreRelation default, src, None) ``` -From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](http://marmbrus.github.io/catalyst/latest/api/index.html#catalyst.trees.TreeNode) objects. +From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transformDown` function that is available on all [`TreeNode`](http://marmbrus.github.io/catalyst/latest/api/index.html#catalyst.trees.TreeNode) objects. ```scala scala> query.optimizedPlan res1: catalyst.plans.logical.LogicalPlan = @@ -82,7 +82,7 @@ Project {key#0,value#1} MetastoreRelation default, src, None -scala> res0.optimizedPlan transform { +scala> res0.optimizedPlan transformDown { | case Project(projectList, child) if projectList == child.output => child | } res2: catalyst.plans.logical.LogicalPlan = diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 24d705ed4b5bb..92fbf3ffcbecd 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -46,7 +46,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { case UnresolvedRelation(name, alias) => catalog.lookupRelation(name, alias) } } @@ -55,10 +55,10 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * Makes attribute naming case insensitive by turning all UnresolvedAttributes to lowercase. */ object LowercaseAttributeReferences extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { case UnresolvedRelation(name, alias) => UnresolvedRelation(name, alias.map(_.toLowerCase)) case Subquery(alias, child) => Subquery(alias.toLowerCase, child) - case q: LogicalPlan => q transformExpressions { + case q: LogicalPlan => q transformExpressionsDown { case s: Star => s.copy(table = s.table.map(_.toLowerCase)) case UnresolvedAttribute(name) => UnresolvedAttribute(name.toLowerCase) case Alias(c, name) => Alias(c, name.toLowerCase)() @@ -71,10 +71,10 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * from a logical plan node's children. */ object ResolveReferences extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { case q: LogicalPlan if childIsFullyResolved(q) => logger.trace(s"Attempting to resolve ${q.simpleString}") - q transformExpressions { + q transformExpressionsDown { case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. val result = q.resolve(name).getOrElse(u) @@ -88,9 +88,9 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * Replaces [[UnresolvedFunction]]s with concrete [[Expression]]s. */ object ResolveFunctions extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { case q: LogicalPlan => - q transformExpressions { + q transformExpressionsDown { case UnresolvedFunction(name, children) if children.map(_.resolved).reduceLeft(_&&_) => registry.lookupFunction(name, children) } @@ -101,7 +101,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * Turns projections that contain aggregate expressions into aggregations. */ object GlobalAggregates extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { case Project(projectList, child) if containsAggregates(projectList) => Aggregate(Nil, projectList, child) } @@ -119,7 +119,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * Expands any references to [[Star]] (*) in project operators. */ object StarExpansion extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { // Wait until children are resolved case p: LogicalPlan if !childIsFullyResolved(p) => p // If the projection list contains Stars, expand it. diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index 968d9135632dd..42125be995222 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -13,8 +13,8 @@ import types._ object ConvertNaNs extends Rule[LogicalPlan] { val stringNaN = Literal("NaN", StringType) - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case q: LogicalPlan => q transformExpressions { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + case q: LogicalPlan => q transformExpressionsDown { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e @@ -57,8 +57,8 @@ object PromoteNumericTypes extends Rule[LogicalPlan] { val toFloat = Seq(ByteType, ShortType, IntegerType) :+ FloatType val allPromotions = integralPrecedence :: toDouble :: toFloat :: Nil - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case q: LogicalPlan => q transformExpressions { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + case q: LogicalPlan => q transformExpressionsDown { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e diff --git a/src/main/scala/catalyst/analysis/unresolved.scala b/src/main/scala/catalyst/analysis/unresolved.scala index 1f1f996c93638..54ed4d9802392 100644 --- a/src/main/scala/catalyst/analysis/unresolved.scala +++ b/src/main/scala/catalyst/analysis/unresolved.scala @@ -39,7 +39,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { def exprId = throw new UnresolvedException(this, "exprId") def dataType = throw new UnresolvedException(this, "dataType") - def foldable = throw new UnresolvedException(this, "foldable") + override def foldable = throw new UnresolvedException(this, "foldable") def nullable = throw new UnresolvedException(this, "nullable") def qualifiers = throw new UnresolvedException(this, "qualifiers") def references = children.flatMap(_.references).toSet diff --git a/src/main/scala/catalyst/errors/package.scala b/src/main/scala/catalyst/errors/package.scala index d9685a22b1235..7a472da9e3e53 100644 --- a/src/main/scala/catalyst/errors/package.scala +++ b/src/main/scala/catalyst/errors/package.scala @@ -12,7 +12,7 @@ package object errors { override def getMessage: String = { val treeString = tree.toString - s"${super.getMessage}, tree:${if(treeString contains "\n") "\n" else " "}$tree" + s"${super.getMessage}, tree:${if (treeString contains "\n") "\n" else " "}$tree" } } diff --git a/src/main/scala/catalyst/examples/ViewsExample.scala b/src/main/scala/catalyst/examples/ViewsExample.scala index 7eee3b868f1f8..2dc470de538f3 100644 --- a/src/main/scala/catalyst/examples/ViewsExample.scala +++ b/src/main/scala/catalyst/examples/ViewsExample.scala @@ -28,7 +28,7 @@ object ViewsExample { println(s"Unresolved Plan:\n$unresolvedPlan") // Replace UnresolvedRelations with logical plans from the views map. - val withRelations = unresolvedPlan transform { + val withRelations = unresolvedPlan transformDown { case UnresolvedRelation(name, _) => views(name) } diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index fd540ea4e5551..f89329ea60c0b 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -67,7 +67,6 @@ abstract class HiveUdf extends Expression with ImplementedUdf with Logging { type UDFType val name: String - def foldable = false def nullable = true def references = children.flatMap(_.references).toSet diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index b18689614b242..69f86e47f2eac 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -43,7 +43,7 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { * For example, because of a CREATE TABLE X AS statement. */ object CreateTables extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { case InsertIntoCreatedTable(name, child) => val (databaseName, tableName) = name.split("\\.") match { case Array(tableOnly) => ("default", tableOnly) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 3cbfb80d297c2..d44b446bb5d56 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -84,7 +84,7 @@ object TestShark extends SharkInstance { * hive test cases assume the system is set up. */ private def rewritePaths(cmd: String): String = - if(cmd.toUpperCase startsWith "LOAD") + if (cmd.toUpperCase startsWith "LOAD") cmd.replaceAll("\\.\\.", hiveDevHome.getCanonicalPath) else cmd diff --git a/src/main/scala/catalyst/execution/Transform.scala b/src/main/scala/catalyst/execution/Transform.scala index 3a00fa4560920..c5f8870979a3f 100644 --- a/src/main/scala/catalyst/execution/Transform.scala +++ b/src/main/scala/catalyst/execution/Transform.scala @@ -36,7 +36,7 @@ case class Transform( val readerThread = new Thread("Transform OutoutReader") { override def run() { var curLine = reader.readLine() - while(curLine != null) { + while (curLine != null) { outputLines += buildRow(curLine.split("\t")) curLine = reader.readLine() } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 3e5e65f526c7c..d911863e5339a 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -65,7 +65,7 @@ case class Aggregate( def apply(input: Seq[Row]): Unit = { val evaluatedExpr = expr.map(Evaluate(_, input)) - if(evaluatedExpr.map(_ != null).reduceLeft(_ && _)) + if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) seen += evaluatedExpr } @@ -78,7 +78,7 @@ case class Aggregate( var result: Any = null def apply(input: Seq[Row]): Unit = { - if(result == null) + if (result == null) result = Evaluate(expr, input) } } @@ -93,7 +93,7 @@ case class Aggregate( grouped.map { case (group, rows) => // Replace all aggregate expressions with spark functions that will compute the result. val aggImplementations = aggregateExpressions.map { agg => - val impl = agg transform { + val impl = agg transformDown { case base @ Average(expr) => new AverageFunction(expr, base) case base @ Sum(expr) => new SumFunction(expr, base) case base @ Count(expr) => new CountFunction(expr, base) @@ -105,7 +105,7 @@ case class Aggregate( // If any references exist that are not inside agg functions then the must be grouping exprs // in this case we must rebind them to the grouping tuple. if (remainingAttributes.nonEmpty) { - val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } + val unaliasedAggregateExpr = agg transformDown { case Alias(c, _) => c } // An exact match with a grouping expression val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { @@ -163,7 +163,7 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan val count = sc.accumulable(0) def apply(input: Seq[Row]): Unit = - if(Evaluate(expr, input) != null) + if (Evaluate(expr, input) != null) count += 1 def result: Any = count.value.toLong @@ -171,7 +171,7 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan override def executeCollect() = attachTree(this, "SparkAggregate") { // Replace all aggregate expressions with spark functions that will compute the result. - val aggImplementations = aggregateExprs.map { _ transform { + val aggImplementations = aggregateExprs.map { _ transformDown { case base @ Average(expr) => new AverageFunction(expr, base) case base @ Count(expr) => new CountFunction(expr, base) }} diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 2dee6b01e2fc5..b2e3fd2b9806e 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -91,7 +91,7 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { sys.error(s"Comparison not yet implemented for: $curDataType") } - if(comparison != 0) return comparison + if (comparison != 0) return comparison i += 1 } return 0 diff --git a/src/main/scala/catalyst/execution/debug.scala b/src/main/scala/catalyst/execution/debug.scala index a1e262f3e3035..d18f3089a852c 100644 --- a/src/main/scala/catalyst/execution/debug.scala +++ b/src/main/scala/catalyst/execution/debug.scala @@ -4,7 +4,7 @@ package execution object DebugQuery { def apply(plan: SharkPlan): SharkPlan = { val visited = new collection.mutable.HashSet[Long]() - plan transform { + plan transformDown { case s: SharkPlan if !visited.contains(s.id) => visited += s.id DebugNode(s) diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 609b2ea7646ca..fc5b745781642 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -95,7 +95,7 @@ case class BroadcastNestedLoopJoin( } val outputRows = if (matchedRows.size > 0) { matchedRows - } else if(joinType == LeftOuter || joinType == FullOuter) { + } else if (joinType == LeftOuter || joinType == FullOuter) { Vector(buildRow(streamedRow ++ Array.fill(right.output.size)(null))) } else { Vector() @@ -105,7 +105,7 @@ case class BroadcastNestedLoopJoin( val includedBroadcastTuples = streamedPlusMatches.map(_._2) val allIncludedBroadcastTuples = - if(includedBroadcastTuples.count == 0) + if (includedBroadcastTuples.count == 0) new scala.collection.mutable.BitSet(broadcastedRelation.value.size) else streamedPlusMatches.map(_._2).reduce(_ ++ _) diff --git a/src/main/scala/catalyst/expressions/BoundAttribute.scala b/src/main/scala/catalyst/expressions/BoundAttribute.scala index fa2ef0475caae..76c66fe42a45c 100644 --- a/src/main/scala/catalyst/expressions/BoundAttribute.scala +++ b/src/main/scala/catalyst/expressions/BoundAttribute.scala @@ -30,15 +30,15 @@ case class BoundReference(inputTuple: Int, ordinal: Int, baseReference: Attribut // TODO: Should run against any query plan, not just SharkPlans object BindReferences extends Rule[SharkPlan] { def apply(plan: SharkPlan): SharkPlan = { - plan.transform { + plan.transformDown { case leafNode: SharkPlan if leafNode.children.isEmpty => leafNode case nonLeaf: SharkPlan => attachTree(nonLeaf, "Binding references in operator") { logger.debug(s"Binding references in node ${nonLeaf.simpleString}") - nonLeaf.transformExpressions { + nonLeaf.transformExpressionsDown { case a: AttributeReference => attachTree(a, "Binding attribute") { val inputTuple = nonLeaf.children.indexWhere(_.output contains a) - val ordinal = if(inputTuple == -1) -1 else nonLeaf.children(inputTuple).output.indexWhere(_ == a) - if(ordinal == -1) { + val ordinal = if (inputTuple == -1) -1 else nonLeaf.children(inputTuple).output.indexWhere(_ == a) + if (ordinal == -1) { logger.debug(s"No binding found for $a given input ${nonLeaf.children.map(_.output.mkString("{", ",", "}")).mkString(",")}") a } else { diff --git a/src/main/scala/catalyst/expressions/Cast.scala b/src/main/scala/catalyst/expressions/Cast.scala index 7ebb1f5c450e1..0fd0aca8f44cf 100644 --- a/src/main/scala/catalyst/expressions/Cast.scala +++ b/src/main/scala/catalyst/expressions/Cast.scala @@ -5,7 +5,7 @@ import types.DataType /** Cast the child expression to the target data type. */ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { - def foldable = child.foldable + override def foldable = child.foldable def nullable = child.nullable override def toString = s"CAST($child, $dataType)" } diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 314027fccaede..5df97de9440dc 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -21,7 +21,7 @@ object Evaluate extends Logging { @inline def n1(e: Expression, f: ((Numeric[Any], Any) => Any)): Any = { val evalE = eval(e) - if(evalE == null) + if (evalE == null) null else e.dataType match { @@ -54,7 +54,7 @@ object Evaluate extends Logging { val evalE1 = eval(e1) val evalE2 = eval(e2) - if(evalE1 == null || evalE2 == null) + if (evalE1 == null || evalE2 == null) null else e1.dataType match { @@ -87,7 +87,7 @@ object Evaluate extends Logging { val evalE1 = eval(e1) val evalE2 = eval(e2) - if(evalE1 == null || evalE2 == null) + if (evalE1 == null || evalE2 == null) null else e1.dataType match { @@ -106,7 +106,7 @@ object Evaluate extends Logging { if (e1.dataType != e2.dataType) throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) - if(evalE1 == null || evalE2 == null) + if (evalE1 == null || evalE2 == null) null else e1.dataType match { @@ -143,7 +143,7 @@ object Evaluate extends Logging { case Subtract(l, r) => n2(l,r, _.minus(_, _)) case Multiply(l, r) => n2(l,r, _.times(_, _)) // Divide & remainder implementation are different for fractional and integral dataTypes. - case Divide(l, r) if(l.dataType == DoubleType || l.dataType == FloatType) => f2(l,r, _.div(_, _)) + case Divide(l, r) if (l.dataType == DoubleType || l.dataType == FloatType) => f2(l,r, _.div(_, _)) case Divide(l, r) => i2(l,r, _.quot(_, _)) // Remainder is only allowed on Integral types. case Remainder(l, r) => i2(l,r, _.rem(_, _)) @@ -153,7 +153,7 @@ object Evaluate extends Logging { case Equals(l, r) => val left = eval(l) val right = eval(r) - if(left == null || right == null) + if (left == null || right == null) null else left == right @@ -229,18 +229,18 @@ object Evaluate extends Logging { case And(l,r) => val left = eval(l) val right = eval(r) - if(left == false || right == false) + if (left == false || right == false) false - else if(left == null || right == null ) + else if (left == null || right == null ) null else true case Or(l,r) => val left = eval(l) val right = eval(r) - if(left == true || right == true) + if (left == true || right == true) true - else if(left == null || right == null) + else if (left == null || right == null) null else false @@ -261,7 +261,7 @@ object Evaluate extends Logging { case other => throw new OptimizationException(other, "evaluation not implemented") } - logger.debug(s"Evaluated $e => $result of type ${if(result == null) "null" else result.getClass.getName}, expected: ${e.dataType}") + logger.debug(s"Evaluated $e => $result of type ${if (result == null) "null" else result.getClass.getName}, expected: ${e.dataType}") result } } \ No newline at end of file diff --git a/src/main/scala/catalyst/expressions/Expression.scala b/src/main/scala/catalyst/expressions/Expression.scala index ce023f26c4551..dcc9630c9df0d 100644 --- a/src/main/scala/catalyst/expressions/Expression.scala +++ b/src/main/scala/catalyst/expressions/Expression.scala @@ -8,7 +8,17 @@ abstract class Expression extends TreeNode[Expression] { self: Product => def dataType: DataType - def foldable: Boolean + /** + * foldable is used to indicate if an expression can be folded. + * Right now, we consider expressions listed below as foldable expressions. + * - A Coalesce is foldable if all of its children are foldable + * - A BinaryExpression is foldable if its both left and right child are foldable. + * - A Not, isNull, or isNotNull is foldable if its child is foldable. + * - A Literal is foldable. + * - A Cast or UnaryMinus is foldable if its child is foldable. + */ + // TODO: Supporting more folded expressions. For example, deterministic Hive UDFs. + def foldable: Boolean = false def nullable: Boolean def references: Set[Attribute] @@ -31,7 +41,7 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express def symbol: String - def foldable = left.foldable && right.foldable + override def foldable = left.foldable && right.foldable def references = left.references ++ right.references diff --git a/src/main/scala/catalyst/expressions/Rand.scala b/src/main/scala/catalyst/expressions/Rand.scala index 8368e5923d5ad..b6a0a9cd080a5 100644 --- a/src/main/scala/catalyst/expressions/Rand.scala +++ b/src/main/scala/catalyst/expressions/Rand.scala @@ -5,7 +5,6 @@ import types.DoubleType case object Rand extends LeafExpression { def dataType = DoubleType - def foldable = false def nullable = false def references = Set.empty override def toString = "RAND()" diff --git a/src/main/scala/catalyst/expressions/ScalaUdf.scala b/src/main/scala/catalyst/expressions/ScalaUdf.scala index b0a9b7ec2a3a6..6681e17c01a75 100644 --- a/src/main/scala/catalyst/expressions/ScalaUdf.scala +++ b/src/main/scala/catalyst/expressions/ScalaUdf.scala @@ -7,7 +7,6 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi extends Expression with ImplementedUdf { def references = children.flatMap(_.references).toSet - def foldable = false def nullable = true def evaluate(evaluatedChildren: Seq[Any]): Any = { diff --git a/src/main/scala/catalyst/expressions/SortOrder.scala b/src/main/scala/catalyst/expressions/SortOrder.scala index b1a56186db4d6..967590c030359 100644 --- a/src/main/scala/catalyst/expressions/SortOrder.scala +++ b/src/main/scala/catalyst/expressions/SortOrder.scala @@ -11,7 +11,6 @@ case object Descending extends SortDirection */ case class SortOrder(child: Expression, direction: SortDirection) extends UnaryExpression { def dataType = child.dataType - def foldable = false def nullable = child.nullable - override def toString = s"$child ${if(direction == Ascending) "ASC" else "DESC"}" + override def toString = s"$child ${if (direction == Ascending) "ASC" else "DESC"}" } \ No newline at end of file diff --git a/src/main/scala/catalyst/expressions/WrapDynamic.scala b/src/main/scala/catalyst/expressions/WrapDynamic.scala index 977b3e4e25e9c..128d3de9cb577 100644 --- a/src/main/scala/catalyst/expressions/WrapDynamic.scala +++ b/src/main/scala/catalyst/expressions/WrapDynamic.scala @@ -9,7 +9,6 @@ case object DynamicType extends DataType case class WrapDynamic(children: Seq[Attribute]) extends Expression with ImplementedUdf { def nullable = false - def foldable = false def references = children.toSet def dataType = DynamicType diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala index 4aa99582f8f11..c5aeeccc54c88 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/src/main/scala/catalyst/expressions/aggregates.scala @@ -6,7 +6,6 @@ import catalyst.types._ abstract class AggregateExpression extends Expression { self: Product => - def foldable = false } /** diff --git a/src/main/scala/catalyst/expressions/arithmetic.scala b/src/main/scala/catalyst/expressions/arithmetic.scala index 51acf6d4951ea..3085f80faaba6 100644 --- a/src/main/scala/catalyst/expressions/arithmetic.scala +++ b/src/main/scala/catalyst/expressions/arithmetic.scala @@ -6,7 +6,7 @@ import catalyst.analysis.UnresolvedException case class UnaryMinus(child: Expression) extends UnaryExpression { def dataType = child.dataType - def foldable = child.foldable + override def foldable = child.foldable def nullable = child.nullable override def toString = s"-$child" } @@ -19,7 +19,7 @@ abstract class BinaryArithmetic extends BinaryExpression { left.resolved && right.resolved && left.dataType == right.dataType def dataType = { - if(!resolved) + if (!resolved) throw new UnresolvedException( this, s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") left.dataType diff --git a/src/main/scala/catalyst/expressions/literals.scala b/src/main/scala/catalyst/expressions/literals.scala index 10362bb12e603..796ac67e0571a 100644 --- a/src/main/scala/catalyst/expressions/literals.scala +++ b/src/main/scala/catalyst/expressions/literals.scala @@ -28,9 +28,9 @@ object IntegerLiteral { } case class Literal(value: Any, dataType: DataType) extends LeafExpression { - def foldable = true + override def foldable = true def nullable = false def references = Set.empty - override def toString = if(value != null) value.toString else "null" + override def toString = if (value != null) value.toString else "null" } \ No newline at end of file diff --git a/src/main/scala/catalyst/expressions/namedExpressions.scala b/src/main/scala/catalyst/expressions/namedExpressions.scala index 7385fb0f290d3..bd85009a8af30 100644 --- a/src/main/scala/catalyst/expressions/namedExpressions.scala +++ b/src/main/scala/catalyst/expressions/namedExpressions.scala @@ -19,8 +19,6 @@ case class ExprId(id: Long) abstract class NamedExpression extends Expression { self: Product => - def foldable = false - def name: String def exprId: ExprId def qualifiers: Seq[String] @@ -97,7 +95,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea * Returns a copy of this [[AttributeReference]] with changed nullability. */ def withNullability(newNullability: Boolean) = - if(nullable == newNullability) + if (nullable == newNullability) this else AttributeReference(name, dataType, newNullability)(exprId, qualifiers) @@ -106,7 +104,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea * Returns a copy of this [[AttributeReference]] with new qualifiers. */ def withQualifiers(newQualifiers: Seq[String]) = - if(newQualifiers == qualifiers) + if (newQualifiers == qualifiers) this else AttributeReference(name, dataType, nullable)(exprId, newQualifiers) diff --git a/src/main/scala/catalyst/expressions/nullFunctions.scala b/src/main/scala/catalyst/expressions/nullFunctions.scala index a0e6c631044d8..4732d8952248e 100644 --- a/src/main/scala/catalyst/expressions/nullFunctions.scala +++ b/src/main/scala/catalyst/expressions/nullFunctions.scala @@ -9,7 +9,8 @@ case class Coalesce(children: Seq[Expression]) extends Expression { def nullable = !children.exists(!_.nullable) def references = children.flatMap(_.references).toSet - def foldable = children.map(_.foldable).reduce(_&&_) + // Coalesce is foldable if all children are foldable. + override def foldable = !children.exists(!_.foldable) // Only resolved if all the children are of the same type. override lazy val resolved = childrenResolved && (children.map(_.dataType).distinct.size == 1) diff --git a/src/main/scala/catalyst/expressions/predicates.scala b/src/main/scala/catalyst/expressions/predicates.scala index 42691d54362d6..e6cf7256005d5 100644 --- a/src/main/scala/catalyst/expressions/predicates.scala +++ b/src/main/scala/catalyst/expressions/predicates.scala @@ -16,7 +16,7 @@ abstract class BinaryPredicate extends BinaryExpression with Predicate { case class Not(child: Expression) extends Predicate with trees.UnaryNode[Expression] { def references = child.references - def foldable = child.foldable + override def foldable = child.foldable def nullable = child.nullable override def toString = s"NOT $child" } @@ -55,12 +55,12 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { def references = child.references - def foldable = child.foldable + override def foldable = child.foldable def nullable = false } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { def references = child.references - def foldable = child.foldable + override def foldable = child.foldable def nullable = false } diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 2d907211711e0..baed86957b602 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -176,7 +176,7 @@ object HiveQl { */ def checkEquals(other: ASTNode) { def check(field: String, f: ASTNode => Any) = - if(f(n) != f(other)) + if (f(n) != f(other)) sys.error(s"$field does not match for trees. '${f(n)}' != '${f(other)}' left: ${dumpTree(n)}, right: ${dumpTree(other)}") check("name", _.getName) @@ -205,22 +205,22 @@ object HiveQl { def parseSql(sql: String): LogicalPlan = { try { - if(sql.toLowerCase.startsWith("set")) + if (sql.toLowerCase.startsWith("set")) ConfigurationAssignment(sql) - else if(sql.toLowerCase.startsWith("add jar")) + else if (sql.toLowerCase.startsWith("add jar")) AddJar(sql.drop(8)) - else if(sql.toLowerCase.startsWith("add file")) + else if (sql.toLowerCase.startsWith("add file")) AddFile(sql.drop(9)) - else if(sql.startsWith("dfs")) + else if (sql.startsWith("dfs")) DfsCommand(sql) - else if(sql.startsWith("source")) + else if (sql.startsWith("source")) SourceCommand(sql.split(" ").toSeq match { case Seq("source", filePath) => filePath }) - else if(sql.startsWith("!")) + else if (sql.startsWith("!")) ShellCommand(sql.drop(1)) else { val tree = getAst(sql) - if(nativeCommands contains tree.getText) + if (nativeCommands contains tree.getText) NativeCommand(sql) else nodeToPlan(tree) match { @@ -266,7 +266,7 @@ object HiveQl { var remainingNodes = nodeList val clauses = clauseNames.map { clauseName => val (matches, nonMatches) = remainingNodes.partition(_.getText.toUpperCase == clauseName) - remainingNodes = nonMatches ++ (if(matches.nonEmpty) matches.tail else Nil) + remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil) matches.headOption } @@ -474,7 +474,7 @@ object HiveQl { val joinedTables = tables.reduceLeft(Join(_,_, Inner, None)) // Must be transform down. - val joinedResult = joinedTables transform { + val joinedResult = joinedTables transformDown { case j: Join => j.copy( condition = Some(joinConditions.remove(joinConditions.length - 1)), @@ -698,7 +698,7 @@ object HiveQl { case nfe: NumberFormatException => // Do nothing } - if(v == null) + if (v == null) sys.error(s"Failed to parse number ${ast.getText}") else v diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala index db09b9b54c58b..a2453059e7ae9 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/src/main/scala/catalyst/optimizer/Optimizer.scala @@ -11,20 +11,29 @@ object Optimize extends RuleExecutor[LogicalPlan] { Batch("Subqueries", Once, EliminateSubqueries) :: Batch("ConstantFolding", Once, - ConstantFolding + ConstantFolding, + BooleanSimpliï¬cation ) :: Nil } object EliminateSubqueries extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { case Subquery(_, child) => child } } object ConstantFolding extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case q: LogicalPlan => q transformExpressionsPostOrder { - case a @ And(left, right) if !a.foldable => { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + case q: LogicalPlan => q transformExpressionsUp { + case e if e.foldable => {Literal(Evaluate(e, Nil))} + } + } +} + +object BooleanSimpliï¬cation extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + case q: LogicalPlan => q transformExpressionsUp { + case a @ And(left, right) => { (left, right) match { case (Literal(true, BooleanType), r) => r case (l, Literal(true, BooleanType)) => l @@ -33,7 +42,7 @@ object ConstantFolding extends Rule[LogicalPlan] { case (_, _) => a } } - case o @ Or(left, right) if !o.foldable => { + case o @ Or(left, right) => { (left, right) match { case (Literal(true, BooleanType), _) => Literal(true) case (_, Literal(true, BooleanType)) => Literal(true) @@ -42,7 +51,6 @@ object ConstantFolding extends Rule[LogicalPlan] { case (_, _) => o } } - case e if e.foldable => {Literal(Evaluate(e, Nil))} } } } \ No newline at end of file diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 664dc239decc9..205de19dc3053 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -15,14 +15,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy def outputSet: Set[Attribute] = output.toSet /** - * Runs [[transform]] with [[rule]] on all expressions present in this query operator. + * Runs [[transformDown]] with [[rule]] on all expressions present in this query operator. * @param rule the rule to be applied to every expression in this operator. */ - def transformExpressions(rule: PartialFunction[Expression, Expression]): this.type = { + def transformExpressionsDown(rule: PartialFunction[Expression, Expression]): this.type = { var changed = false - @inline def transformExpression(e: Expression) = { - val newE = e.transform(rule) + @inline def transformExpressionDown(e: Expression) = { + val newE = e.transformDown(rule) if (newE.id != e.id && newE != e) { changed = true newE @@ -32,10 +32,10 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy } val newArgs = productIterator.map { - case e: Expression => transformExpression(e) - case Some(e: Expression) => Some(transformExpression(e)) + case e: Expression => transformExpressionDown(e) + case Some(e: Expression) => Some(transformExpressionDown(e)) case seq: Seq[_] => seq.map { - case e: Expression => transformExpression(e) + case e: Expression => transformExpressionDown(e) case other => other } case other: AnyRef => other @@ -45,16 +45,16 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy } /** - * Runs [[transformPostOrder]] with [[rule]] on all expressions present in this query operator. + * Runs [[transformUp]] with [[rule]] on all expressions present in this query operator. * @param rule the rule to be applied to every expression in this operator. * @return */ - def transformExpressionsPostOrder(rule: PartialFunction[Expression, Expression]): this.type = { + def transformExpressionsUp(rule: PartialFunction[Expression, Expression]): this.type = { var changed = false - @inline def transformExpressionPostOrder(e: Expression) = { - val newE = e.transformPostOrder(rule) - if(newE.id != e.id && newE != e) { + @inline def transformExpressionUp(e: Expression) = { + val newE = e.transformUp(rule) + if (newE.id != e.id && newE != e) { changed = true newE } else { @@ -63,22 +63,23 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy } val newArgs = productIterator.map { - case e: Expression => transformExpressionPostOrder(e) - case Some(e: Expression) => Some(transformExpressionPostOrder(e)) + case e: Expression => transformExpressionUp(e) + case Some(e: Expression) => Some(transformExpressionUp(e)) case seq: Seq[_] => seq.map { - case e: Expression => transformExpressionPostOrder(e) + case e: Expression => transformExpressionUp(e) case other => other } case other: AnyRef => other }.toArray - if(changed) makeCopy(newArgs) else this + if (changed) makeCopy(newArgs) else this } - /** Returns the result of running [[transformExpressions]] on this node and all its children */ + /** Returns the result of running [[transformExpressionsDown]] on this node + * and all its children. */ def transformAllExpressions(rule: PartialFunction[Expression, Expression]): this.type = { - transform { - case q: QueryPlan[_] => q.transformExpressions(rule).asInstanceOf[PlanType] + transformDown { + case q: QueryPlan[_] => q.transformExpressionsDown(rule).asInstanceOf[PlanType] }.asInstanceOf[this.type] } diff --git a/src/main/scala/catalyst/plans/logical/TestRelation.scala b/src/main/scala/catalyst/plans/logical/TestRelation.scala index 88535680fa075..b4379705a7c8d 100644 --- a/src/main/scala/catalyst/plans/logical/TestRelation.scala +++ b/src/main/scala/catalyst/plans/logical/TestRelation.scala @@ -39,7 +39,7 @@ object NewLocalRelationInstances extends Rule[LogicalPlan] { .map(_._1) .toSet - plan transform { + plan transformDown { case l: LocalRelation if multiAppearance contains l => l.newInstance } } diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index add0c98cf3582..bdb97faa2ebd4 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -84,17 +84,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** * Returns a copy of this node where [[rule]] has been recursively - * applied to it and all of its children. When [[rule]] does not + * applied to it and all of its children (pre-order). When [[rule]] does not * apply to a given node it is left unchanged. * @param rule the function use to transform this nodes children */ - def transform(rule: PartialFunction[BaseType, BaseType]): BaseType = { + def transformDown(rule: PartialFunction[BaseType, BaseType]): BaseType = { val afterRule = rule.applyOrElse(this, identity[BaseType]) // Check if unchanged and then possibly return old copy to avoid gc churn. if (this fastEquals afterRule) { - transformChildren(rule) + transformChildrenDown(rule) } else { - afterRule.transformChildren(rule) + afterRule.transformChildrenDown(rule) } } @@ -104,11 +104,11 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { * apply to a given node it is left unchanged. * @param rule the function use to transform this nodes children */ - def transformChildren(rule: PartialFunction[BaseType, BaseType]): this.type = { + def transformChildrenDown(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { case arg: TreeNode[_] if (children contains arg) => - val newChild = arg.asInstanceOf[BaseType].transform(rule) + val newChild = arg.asInstanceOf[BaseType].transformDown(rule) if (!(newChild fastEquals arg)) { changed = true newChild @@ -117,8 +117,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } case args: Seq[_] => args.map { case arg: TreeNode[_] if (children contains arg) => - val newChild = arg.asInstanceOf[BaseType].transform(rule) - if(!(newChild fastEquals arg)) { + val newChild = arg.asInstanceOf[BaseType].transformDown(rule) + if (!(newChild fastEquals arg)) { changed = true newChild } else { @@ -129,17 +129,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case nonChild: AnyRef => nonChild case null => null }.toArray - if(changed) makeCopy(newArgs) else this + if (changed) makeCopy(newArgs) else this } /** * Returns a copy of this node where [[rule]] has been recursively - * applied first to all of its children and then itself. + * applied first to all of its children and then itself (post-order). * When [[rule]] does not apply to a given node, it is left unchanged. * @param rule the function use to transform this nodes children */ - def transformPostOrder(rule: PartialFunction[BaseType, BaseType]): BaseType = { - val afterRuleOnChildren = transformChildrenPostOrder(rule); + def transformUp(rule: PartialFunction[BaseType, BaseType]): BaseType = { + val afterRuleOnChildren = transformChildrenUp(rule); if (this fastEquals afterRuleOnChildren) { rule.applyOrElse(this, identity[BaseType]) } else { @@ -147,21 +147,21 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } } - def transformChildrenPostOrder(rule: PartialFunction[BaseType, BaseType]): this.type = { + def transformChildrenUp(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if(children contains arg) => - val newChild = arg.asInstanceOf[BaseType].transformPostOrder(rule) - if(!(newChild fastEquals arg)) { + case arg: TreeNode[_] if (children contains arg) => + val newChild = arg.asInstanceOf[BaseType].transformUp(rule) + if (!(newChild fastEquals arg)) { changed = true newChild } else { arg } case args: Seq[_] => args.map { - case arg: TreeNode[_] if(children contains arg) => - val newChild = arg.asInstanceOf[BaseType].transformPostOrder(rule) - if(!(newChild fastEquals arg)) { + case arg: TreeNode[_] if (children contains arg) => + val newChild = arg.asInstanceOf[BaseType].transformUp(rule) + if (!(newChild fastEquals arg)) { changed = true newChild } else { @@ -172,7 +172,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case nonChild: AnyRef => nonChild case null => null }.toArray - if(changed) makeCopy(newArgs) else this + if (changed) makeCopy(newArgs) else this } /** diff --git a/src/main/scala/catalyst/util/package.scala b/src/main/scala/catalyst/util/package.scala index a0052997d9c9c..d147401e2a414 100644 --- a/src/main/scala/catalyst/util/package.scala +++ b/src/main/scala/catalyst/util/package.scala @@ -45,15 +45,15 @@ package object util { def sideBySide(left: Seq[String], right: Seq[String]): Seq[String] = { val maxLeftSize = left.map(_.size).max - val leftPadded = left ++ Seq.fill(if(left.size < right.size) right.size - left.size else 0)("") - val rightPadded = right ++ Seq.fill(if(right.size < left.size) left.size - right.size else 0)("") + val leftPadded = left ++ Seq.fill(if (left.size < right.size) right.size - left.size else 0)("") + val rightPadded = right ++ Seq.fill(if (right.size < left.size) left.size - right.size else 0)("") leftPadded.zip(rightPadded).map { - case (l,r) => (if(l == r) " " else "!") + l + (" " * ((maxLeftSize - l.size) + 3)) + r + case (l,r) => (if (l == r) " " else "!") + l + (" " * ((maxLeftSize - l.size) + 3)) + r } } - def stringOrNull(a: AnyRef) = if(a == null) null else a.toString + def stringOrNull(a: AnyRef) = if (a == null) null else a.toString implicit class debugLogging(a: AnyRef) { def debugLogging { diff --git a/src/test/scala/catalyst/ExpressionEvaluationSuite.scala b/src/test/scala/catalyst/ExpressionEvaluationSuite.scala index 1bcbb82acd94d..26836aa7b5623 100644 --- a/src/test/scala/catalyst/ExpressionEvaluationSuite.scala +++ b/src/test/scala/catalyst/ExpressionEvaluationSuite.scala @@ -45,7 +45,7 @@ class ExpressionEvaluationSuite extends FunSuite { case (v, answer) => val expr = Not(Literal(v, BooleanType)) val result = Evaluate(expr, Nil) - if(result != answer) + if (result != answer) fail(s"$expr should not evaluate to $result, expected: $answer") } } @@ -88,7 +88,7 @@ class ExpressionEvaluationSuite extends FunSuite { case (l,r,answer) => val expr = op(Literal(l, BooleanType), Literal(r, BooleanType)) val result = Evaluate(expr, Nil) - if(result != answer) + if (result != answer) fail(s"$expr should not evaluate to $result, expected: $answer") } } diff --git a/src/test/scala/catalyst/RuleExecutorSuite.scala b/src/test/scala/catalyst/RuleExecutorSuite.scala index 29434379ce7af..fcc2994930f00 100644 --- a/src/test/scala/catalyst/RuleExecutorSuite.scala +++ b/src/test/scala/catalyst/RuleExecutorSuite.scala @@ -8,7 +8,7 @@ import rules._ class RuleExecutorSuite extends FunSuite { object DecrementLiterals extends Rule[Expression] { - def apply(e: Expression): Expression = e transform { + def apply(e: Expression): Expression = e transformDown { case IntegerLiteral(i) if i > 0 => Literal(i - 1) } } diff --git a/src/test/scala/catalyst/TreeNodeSuite.scala b/src/test/scala/catalyst/TreeNodeSuite.scala index 46ddaeeaa683b..9167d44d91427 100644 --- a/src/test/scala/catalyst/TreeNodeSuite.scala +++ b/src/test/scala/catalyst/TreeNodeSuite.scala @@ -1,7 +1,7 @@ package catalyst package trees -import collection.mutable.ArrayBuffer +import scala.collection.mutable.ArrayBuffer import expressions._ @@ -10,20 +10,20 @@ import org.scalatest.{FunSuite} class TreeNodeSuite extends FunSuite { test("top node changed") { - val after = Literal(1) transform { case Literal(1, _) => Literal(2) } + val after = Literal(1) transformDown { case Literal(1, _) => Literal(2) } assert(after === Literal(2)) } test("one child changed") { val before = Add(Literal(1), Literal(2)) - val after = before transform { case Literal(2, _) => Literal(1) } + val after = before transformDown { case Literal(2, _) => Literal(1) } assert(after === Add(Literal(1), Literal(1))) } test("no change") { val before = Add(Literal(1), Add(Literal(2), Add(Literal(3), Literal(4)))) - val after = before transform { case Literal(5, _) => Literal(1)} + val after = before transformDown { case Literal(5, _) => Literal(1)} assert(before === after) assert(before.map(_.id) === after.map(_.id)) @@ -41,7 +41,7 @@ class TreeNodeSuite extends FunSuite { val actual = new ArrayBuffer[String]() val expected = Seq("+", "1", "*", "2", "-", "3", "4") val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) - expression transform { + expression transformDown { case b: BinaryExpression => {actual.append(b.symbol); b} case l: Literal => {actual.append(l.toString); l} } @@ -53,7 +53,7 @@ class TreeNodeSuite extends FunSuite { val actual = new ArrayBuffer[String]() val expected = Seq("1", "2", "3", "4", "-", "*", "+") val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) - expression transformPostOrder { + expression transformUp { case b: BinaryExpression => {actual.append(b.symbol); b} case l: Literal => {actual.append(l.toString); l} } diff --git a/src/test/scala/catalyst/execution/BigDataBenchmark.scala b/src/test/scala/catalyst/execution/BigDataBenchmark.scala index d5e05ba28bacc..52146d16f0cfe 100644 --- a/src/test/scala/catalyst/execution/BigDataBenchmark.scala +++ b/src/test/scala/catalyst/execution/BigDataBenchmark.scala @@ -59,7 +59,7 @@ class BigDataBenchmarkTests extends HiveComaparisionTest { testTables.foreach(registerTestTable) - if(!testDataDirectory.exists()) { + if (!testDataDirectory.exists()) { // TODO: Auto download the files on demand. ignore("No data files found for BigDataBenchmark tests.") {} } else { diff --git a/src/test/scala/catalyst/execution/DslQueryTests.scala b/src/test/scala/catalyst/execution/DslQueryTests.scala index 0e8463996ad54..3f4eae929ca85 100644 --- a/src/test/scala/catalyst/execution/DslQueryTests.scala +++ b/src/test/scala/catalyst/execution/DslQueryTests.scala @@ -242,7 +242,7 @@ class DslQueryTests extends FunSuite with BeforeAndAfterAll { } val isSorted = plan.collect { case s: logical.Sort => s}.nonEmpty - def prepareAnswer(answer: Seq[Any]) = if(!isSorted) answer.sortBy(_.toString) else answer + def prepareAnswer(answer: Seq[Any]) = if (!isSorted) answer.sortBy(_.toString) else answer val sharkAnswer = try plan.toRdd.collect().toSeq catch { case e: Exception => fail( diff --git a/src/test/scala/catalyst/execution/HiveComparisionTest.scala b/src/test/scala/catalyst/execution/HiveComparisionTest.scala index e7183a94d419e..d90fe2c88427e 100644 --- a/src/test/scala/catalyst/execution/HiveComparisionTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisionTest.scala @@ -17,7 +17,7 @@ import util._ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with GivenWhenThen with Logging { protected val targetDir = new File("target") protected val answerCache = new File(targetDir, "comparison-test-cache") - if(!answerCache.exists) + if (!answerCache.exists) answerCache.mkdir() val passedFile = new File(targetDir, s"$suiteName.passed") @@ -40,7 +40,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with case _ => val isOrdered = sharkQuery.executedPlan.collect { case s: Sort => s}.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. - if(!isOrdered) answer.sorted else answer + if (!isOrdered) answer.sorted else answer } orderedAnswer.map(cleanPaths).map(clearTimes) } @@ -78,10 +78,10 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with } val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => - if(cachedAnswerFile.exists) { + if (cachedAnswerFile.exists) { val cachedString = fileToString(cachedAnswerFile) val cachedAnswer = - if(cachedString == "") + if (cachedString == "") Nil else cachedString.split("\n").toSeq @@ -93,7 +93,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with } val hiveResults: Seq[Seq[String]] = - if(hiveCachedResults.size == queryList.size) { + if (hiveCachedResults.size == queryList.size) { logger.warn(s"Using answer cache for test: $testCaseName") hiveCachedResults } else { @@ -146,7 +146,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with // Check that the results match unless its an EXPLAIN query. val preparedHive = prepareAnswer(sharkQuery,hive) - if((!sharkQuery.parsed.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) { + if ((!sharkQuery.parsed.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${hive.size} row(s) ==" +: preparedHive val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst @@ -166,7 +166,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with } catch { case tf: org.scalatest.exceptions.TestFailedException => throw tf case originalException: Exception => - if(System.getProperty("shark.hive.canarytest") != null) { + if (System.getProperty("shark.hive.canarytest") != null) { // When we encounter an error we check to see if the environment is still okay by running a simple query. // If this fails then we halt testing since something must have gone seriously wrong. try { diff --git a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala index fa93af8daa3c6..99299bc31473e 100644 --- a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala +++ b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala @@ -31,9 +31,9 @@ abstract class HiveQueryFileTest extends HiveComaparisionTest { // Go through all the test cases and add them to scala test. testCases.foreach { case (testCaseName, testCaseFile) => - if(blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { + if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { logger.warn(s"Blacklisted test skipped $testCaseName") - } else if(realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { + } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) createQueryTest(testCaseName, queriesString) diff --git a/src/test/scala/catalyst/optimizer/OptimizerSuite.scala b/src/test/scala/catalyst/optimizer/OptimizerSuite.scala index 5d202fa75b50c..042cda1aa9e40 100644 --- a/src/test/scala/catalyst/optimizer/OptimizerSuite.scala +++ b/src/test/scala/catalyst/optimizer/OptimizerSuite.scala @@ -12,14 +12,14 @@ import dsl._ /* Implicit conversions for creating query plans */ class OptimizerSuite extends FunSuite { - // Test relations. Feel free to create more. + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) // Helper functions for comparing plans. /** - * Since attribute references are given globally unique ids during analysis we must normalize them to check if two - * different queries are identical. + * Since attribute references are given globally unique ids during analysis, + * we must normalize them to check if two different queries are identical. */ protected def normalizeExprIds(plan: LogicalPlan) = { val minId = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)).min @@ -33,7 +33,7 @@ class OptimizerSuite extends FunSuite { protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { val normalized1 = normalizeExprIds(plan1) val normalized2 = normalizeExprIds(plan2) - if(normalized1 != normalized2) + if (normalized1 != normalized2) fail( s""" |== FAIL: Plans do not match === @@ -85,7 +85,7 @@ class OptimizerSuite extends FunSuite { comparePlans(optimized, correctAnswer) } - test("Constant folding test: expressions have attribute references and literals in" + + test("Constant folding test: expressions have attribute references and literals in " + "arithmetic operations") { val originalQuery = testRelation @@ -107,7 +107,7 @@ class OptimizerSuite extends FunSuite { comparePlans(optimized, correctAnswer) } - test("Constant folding test: expressions have attribute references and literals in" + + test("Constant folding test: expressions have attribute references and literals in " + "predicates") { val originalQuery = testRelation From dbc367804a538aee40134c9165878a8e415cf164 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 16:43:12 -0800 Subject: [PATCH 386/778] update ghpages repo --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 1434a4a56b6ce..38492be908829 100644 --- a/build.sbt +++ b/build.sbt @@ -51,7 +51,7 @@ site.settings ghpages.settings -git.remoteRepo := "git@github.com:marmbrus/catalyst.git" +git.remoteRepo := "git@github.com:databricks/catalyst.git" site.settings From db00ebfb080b7f3377f8909e0172542140f5daf9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 16:43:27 -0800 Subject: [PATCH 387/778] more types for hive udfs --- src/main/scala/catalyst/execution/FunctionRegistry.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index f89329ea60c0b..9d1a8f8520106 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -40,9 +40,12 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry { def javaClassToDataType(clz: Class[_]): DataType = clz match { case c: Class[_] if c == classOf[DoubleWritable] => DoubleType case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.DoubleWritable] => DoubleType + case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.ByteWritable] => ByteType + case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.ShortWritable] => ShortType case c: Class[_] if c == classOf[Text] => StringType case c: Class[_] if c == classOf[org.apache.hadoop.io.IntWritable] => IntegerType case c: Class[_] if c == classOf[org.apache.hadoop.io.LongWritable] => LongType + case c: Class[_] if c == classOf[org.apache.hadoop.io.FloatWritable] => FloatType case c: Class[_] if c == classOf[java.lang.String] => StringType case c: Class[_] if c == java.lang.Short.TYPE => ShortType case c: Class[_] if c == java.lang.Integer.TYPE => ShortType From c99012cb4e6928bb26aade6e5e363108ea600290 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 16:44:01 -0800 Subject: [PATCH 388/778] skip tests with hooks --- src/test/scala/catalyst/execution/HiveComparisionTest.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/test/scala/catalyst/execution/HiveComparisionTest.scala b/src/test/scala/catalyst/execution/HiveComparisionTest.scala index e7183a94d419e..efcfdda818799 100644 --- a/src/test/scala/catalyst/execution/HiveComparisionTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisionTest.scala @@ -58,6 +58,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with str.replaceAll("file:\\/.*\\/", "") } + val installHooksCommand = "SET.*hooks".r def createQueryTest(testCaseName: String, sql: String) = { test(testCaseName) { logger.error( @@ -71,6 +72,9 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with try { TestShark.reset() + if (queryList.filter(installHooksCommand.findAllMatchIn(_).nonEmpty).nonEmpty) + sys.error("hive exec hooks not supported for tests.") + val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" From b0aa400ff25557720e10858d015342a1e2489acc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 16:44:21 -0800 Subject: [PATCH 389/778] update whitelist. --- .../execution/HiveCompatability.scala | 124 ++++++++++-------- 1 file changed, 69 insertions(+), 55 deletions(-) diff --git a/src/test/scala/catalyst/execution/HiveCompatability.scala b/src/test/scala/catalyst/execution/HiveCompatability.scala index c72e655932ebc..e77bc55789647 100644 --- a/src/test/scala/catalyst/execution/HiveCompatability.scala +++ b/src/test/scala/catalyst/execution/HiveCompatability.scala @@ -24,6 +24,7 @@ class HiveCompatability extends HiveQueryFileTest { "sample10", "updateAccessTime", "index_compact_binary_search", + "bucket_num_reducers", // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 @@ -56,42 +57,54 @@ class HiveCompatability extends HiveQueryFileTest { */ override def whiteList = Seq( "add_part_exist", - "alter2", + "add_partition_no_whitelist", + "add_partition_with_whitelist", + "alias_casted_column", "alter4", - "alter_partition_format_loc", - "alter_table_serde", + "alter_partition_with_whitelist", + "ambiguous_col", "authorization_3", "authorization_5", + "auto_join21", "auto_join23", "auto_join24", "auto_join26", "auto_join28", "auto_join_nulls", + "auto_sortmerge_join_10", + "auto_sortmerge_join_15", + "auto_sortmerge_join_5", + "auto_sortmerge_join_6", + "auto_sortmerge_join_9", "binarysortable_1", "bucket1", + "bucket_map_join_1", + "bucket_map_join_2", + "bucketcontext_5", "bucketmapjoin6", - "bucketmapjoin_negative", - "bucketmapjoin_negative2", + "bucketmapjoin_negative3", "combine1", + "convert_enum_to_string", + "correlationoptimizer15", + "correlationoptimizer4", + "correlationoptimizer6", + "correlationoptimizer7", "count", - "create_default_prop", + "create_like2", + "create_skewed_table1", "ct_case_insensitive", - "database_location", "database_properties", - "default_partition_name", "delimiter", + "desc_non_existent_tbl", "describe_database_json", "describe_table_json", - "diff_part_input_formats", "disable_file_format_check", "drop_function", "drop_index", - "drop_partitions_filter", - "drop_partitions_filter2", - "drop_partitions_filter3", "drop_table", "drop_view", - "filter_join_breaktask", + "escape_orderby1", + "escape_sortby1", "groupby1", "groupby1_map", "groupby1_map_nomap", @@ -103,10 +116,10 @@ class HiveCompatability extends HiveQueryFileTest { "groupby5_map", "groupby5_map_skew", "groupby5_noskew", + "groupby_multi_single_reducer2", + "groupby_mutli_insert_common_distinct", + "groupby_sort_test_1", "implicit_cast1", - "index_auth", - "index_auto_mult_tables", - "index_auto_mult_tables_compact", "index_auto_self_join", "index_auto_update", "index_stale", @@ -115,30 +128,22 @@ class HiveCompatability extends HiveQueryFileTest { "input0", "input11", "input11_limit", + "input1_limit", "input22", - "input23", - "input24", - "input25", - "input28", - "input41", "input4_cb_delim", "input4_limit", "input6", "input7", + "input8", "input9", - "input_part1", "inputddl4", - "inputddl7", "inputddl8", - "insert1", "join0", "join1", "join10", "join11", "join12", "join13", - "join14", - "join14_hadoop20", "join15", "join16", "join17", @@ -146,51 +151,50 @@ class HiveCompatability extends HiveQueryFileTest { "join19", "join2", "join20", + "join21", "join22", "join23", "join24", "join25", - "join26", "join27", - "join28", "join29", "join3", "join30", "join31", - "join32", - "join33", "join34", "join35", "join36", "join37", + "join38", "join39", "join4", "join40", + "join41", "join5", "join6", + "join7", "join8", - "join9", "join_casesensitive", "join_empty", "join_hive_626", "join_nulls", "join_reorder2", "join_reorder3", + "join_reorder4", + "join_star", "join_view", "keyword_1", "lineage1", "literal_double", "literal_ints", "literal_string", - "louter_join_ppr", - "mapjoin1", - "mapjoin_mapjoin", - "mapjoin_subquery", + "load_file_with_space_in_the_name", "mapjoin_subquery2", - "merge1", - "merge2", "mergejoins", + "mergejoins_mixed", "misc_json", + "multi_join_union", + "multigroupby_singlemr", "no_hooks", "noalias_subq1", "nomore_ambiguous_table_col", @@ -198,18 +202,11 @@ class HiveCompatability extends HiveQueryFileTest { "notable_alias2", "nullgroup", "nullgroup2", - "nullgroup3", - "nullgroup5", "nullinput", "nullinput2", "nullscript", - "outer_join_ppr", - "part_inherit_tbl_props", - "part_inherit_tbl_props_empty", - "part_inherit_tbl_props_with_star", - "partition_schema1", - "partition_vs_table_metadata", - "partitions_json", + "optional_outer", + "plan_json", "ppd1", "ppd_gby", "ppd_gby_join", @@ -226,22 +223,29 @@ class HiveCompatability extends HiveQueryFileTest { "ppd_udf_col", "ppd_union", "progress_1", - "protectmode", - "protectmode2", "query_with_semi", "quote2", "rename_column", - "router_join_ppr", "select_as_omitted", "set_variable_sub", "show_describe_func_quotes", "show_functions", - "smb_mapjoin_10", + "skewjoinopt13", + "skewjoinopt18", + "skewjoinopt9", + "smb_mapjoin_13", + "smb_mapjoin_14", + "smb_mapjoin_15", + "smb_mapjoin_16", + "smb_mapjoin_17", + "sort_merge_join_desc_1", + "sort_merge_join_desc_2", + "sort_merge_join_desc_3", + "sort_merge_join_desc_4", "subq2", "tablename_with_select", + "type_cast_1", "udf2", - "udf5", - "udf7", "udf9", "udf_10_trims", "udf_abs", @@ -260,6 +264,8 @@ class HiveCompatability extends HiveQueryFileTest { "udf_boolean", "udf_ceil", "udf_ceiling", + "udf_concat", + "udf_concat_insert2", "udf_conv", "udf_cos", "udf_count", @@ -271,13 +277,14 @@ class HiveCompatability extends HiveQueryFileTest { "udf_double", "udf_exp", "udf_field", + "udf_find_in_set", "udf_float", "udf_floor", + "udf_format_number", "udf_from_unixtime", "udf_greaterthan", "udf_greaterthanorequal", "udf_hex", - "udf_hour", "udf_index", "udf_int", "udf_isnotnull", @@ -296,11 +303,12 @@ class HiveCompatability extends HiveQueryFileTest { "udf_minute", "udf_modulo", "udf_month", + "udf_negative", "udf_not", "udf_notequal", + "udf_nvl", "udf_or", "udf_parse_url", - "udf_pmod", "udf_positive", "udf_pow", "udf_power", @@ -329,9 +337,9 @@ class HiveCompatability extends HiveQueryFileTest { "udf_tan", "udf_tinyint", "udf_to_date", + "udf_to_unix_timestamp", "udf_trim", "udf_ucase", - "udf_unhex", "udf_unix_timestamp", "udf_upper", "udf_var_pop", @@ -352,14 +360,20 @@ class HiveCompatability extends HiveQueryFileTest { "union14", "union15", "union16", + "union17", + "union18", + "union19", "union2", "union20", "union27", "union28", "union29", "union30", + "union31", + "union34", "union4", "union5", + "union6", "union7", "union8", "union9", From 68b58c1f683c1a58ee1c371e30d92c8558b7d234 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 16:55:35 -0800 Subject: [PATCH 390/778] drop a few more tests. --- src/test/scala/catalyst/execution/HiveCompatability.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/test/scala/catalyst/execution/HiveCompatability.scala b/src/test/scala/catalyst/execution/HiveCompatability.scala index e77bc55789647..76503e11eb86e 100644 --- a/src/test/scala/catalyst/execution/HiveCompatability.scala +++ b/src/test/scala/catalyst/execution/HiveCompatability.scala @@ -90,8 +90,6 @@ class HiveCompatability extends HiveQueryFileTest { "correlationoptimizer6", "correlationoptimizer7", "count", - "create_like2", - "create_skewed_table1", "ct_case_insensitive", "database_properties", "delimiter", @@ -303,7 +301,6 @@ class HiveCompatability extends HiveQueryFileTest { "udf_minute", "udf_modulo", "udf_month", - "udf_negative", "udf_not", "udf_notequal", "udf_nvl", From 9e190f54f0c98c5b9801a7e40eb318497cac441c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 16:59:02 -0800 Subject: [PATCH 391/778] drop unneeded () --- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 2ec45e8f4d021..96e61ae0cc6a7 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -100,7 +100,7 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt Class.forName(table.getSd.getSerdeInfo.getSerializationLib).asInstanceOf[Class[Deserializer]], Class.forName(table.getSd.getInputFormat).asInstanceOf[Class[InputFormat[_,_]]], Class.forName(table.getSd.getOutputFormat), - hiveQlTable.getMetadata() + hiveQlTable.getMetadata ) implicit class SchemaAttribute(f: FieldSchema) { From 5fabbecf3b285aace60a67ebe3ea7fcbd9d939f7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 17:08:08 -0800 Subject: [PATCH 392/778] ignore partitioned scan test. scan seems to be working but there is some error about the table already existing? --- src/test/scala/catalyst/execution/HiveQueryTests.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala index ae710ea939adc..5a39c35640684 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -41,9 +41,11 @@ class HiveQueryTests extends HiveComaparisionTest { createQueryTest("length.udf", "SELECT length(\"test\") FROM src LIMIT 1") - createQueryTest("partitioned table scan", - "SELECT ds, hr, key, value FROM srcpart") + ignore("partitioned table scan") { + createQueryTest("partitioned table scan", + "SELECT ds, hr, key, value FROM srcpart") + } createQueryTest("hash", "SELECT hash('test') FROM src LIMIT 1") From b6e48998b322652ffa90e2c092b43f904c1d1552 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 4 Jan 2014 21:30:39 -0500 Subject: [PATCH 393/778] formatting --- .../catalyst/optimizer/OptimizerSuite.scala | 81 ++++++++++--------- 1 file changed, 45 insertions(+), 36 deletions(-) diff --git a/src/test/scala/catalyst/optimizer/OptimizerSuite.scala b/src/test/scala/catalyst/optimizer/OptimizerSuite.scala index 042cda1aa9e40..6e787e8acf3ba 100644 --- a/src/test/scala/catalyst/optimizer/OptimizerSuite.scala +++ b/src/test/scala/catalyst/optimizer/OptimizerSuite.scala @@ -63,19 +63,24 @@ class OptimizerSuite extends FunSuite { test("Constant folding test: expressions only have literals") { val originalQuery = testRelation - .select(Literal(2) + Literal(3) + Literal(4) as Symbol("2+3+4"), - Literal(2) * Literal(3) + Literal(4) as Symbol("2*3+4"), - Literal(2) * (Literal(3) + Literal(4)) as Symbol("2*(3+4)")) - .where(Literal(1) === Literal(1) && - Literal(2) > Literal(3) || - Literal(3) > Literal(2) ) - .groupBy(Literal(2) * Literal(3) - Literal(6) / (Literal(4) - Literal(2)))(Literal(9) / Literal(3) as Symbol("9/3")) + .select( + Literal(2) + Literal(3) + Literal(4) as Symbol("2+3+4"), + Literal(2) * Literal(3) + Literal(4) as Symbol("2*3+4"), + Literal(2) * (Literal(3) + Literal(4)) as Symbol("2*(3+4)")) + .where( + Literal(1) === Literal(1) && + Literal(2) > Literal(3) || + Literal(3) > Literal(2) ) + .groupBy( + Literal(2) * Literal(3) - Literal(6) / (Literal(4) - Literal(2)) + )(Literal(9) / Literal(3) as Symbol("9/3")) val optimized = Optimize(originalQuery.analyze) val correctAnswer = testRelation - .select(Literal(9) as Symbol("2+3+4"), + .select( + Literal(9) as Symbol("2+3+4"), Literal(10) as Symbol("2*3+4"), Literal(14) as Symbol("2*(3+4)")) .where(Literal(true)) @@ -89,19 +94,21 @@ class OptimizerSuite extends FunSuite { "arithmetic operations") { val originalQuery = testRelation - .select(Literal(2) + Literal(3) + 'a as Symbol("c1"), - 'a + Literal(2) + Literal(3) as Symbol("c2"), - Literal(2) * 'a + Literal(4) as Symbol("c3"), - 'a * (Literal(3) + Literal(4)) as Symbol("c4")) + .select( + Literal(2) + Literal(3) + 'a as Symbol("c1"), + 'a + Literal(2) + Literal(3) as Symbol("c2"), + Literal(2) * 'a + Literal(4) as Symbol("c3"), + 'a * (Literal(3) + Literal(4)) as Symbol("c4")) val optimized = Optimize(originalQuery.analyze) val correctAnswer = testRelation - .select(Literal(5) + 'a as Symbol("c1"), - 'a + Literal(2) + Literal(3) as Symbol("c2"), - Literal(2) * 'a + Literal(4) as Symbol("c3"), - 'a * (Literal(7)) as Symbol("c4")) + .select( + Literal(5) + 'a as Symbol("c1"), + 'a + Literal(2) + Literal(3) as Symbol("c2"), + Literal(2) * 'a + Literal(4) as Symbol("c3"), + 'a * (Literal(7)) as Symbol("c4")) .analyze comparePlans(optimized, correctAnswer) @@ -111,23 +118,21 @@ class OptimizerSuite extends FunSuite { "predicates") { val originalQuery = testRelation - .where((('a > 1 && Literal(1) === Literal(1)) || - ('a < 10 && Literal(1) === Literal(2)) || - (Literal(1) === Literal(1) && 'b > 1) || - (Literal(1) === Literal(2) && 'b < 10)) && - (('a > 1 || Literal(1) === Literal(1)) && - ('a < 10 || Literal(1) === Literal(2)) && - (Literal(1) === Literal(1) || 'b > 1) && - (Literal(1) === Literal(2) || 'b < 10))) + .where( + (('a > 1 && Literal(1) === Literal(1)) || + ('a < 10 && Literal(1) === Literal(2)) || + (Literal(1) === Literal(1) && 'b > 1) || + (Literal(1) === Literal(2) && 'b < 10)) && + (('a > 1 || Literal(1) === Literal(1)) && + ('a < 10 || Literal(1) === Literal(2)) && + (Literal(1) === Literal(1) || 'b > 1) && + (Literal(1) === Literal(2) || 'b < 10))) val optimized = Optimize(originalQuery.analyze) val correctAnswer = testRelation - .where(('a > 1 || - 'b > 1) && - ('a < 10 && - 'b < 10)) + .where(('a > 1 || 'b > 1) && ('a < 10 && 'b < 10)) .analyze comparePlans(optimized, correctAnswer) @@ -136,15 +141,17 @@ class OptimizerSuite extends FunSuite { test("Constant folding test: expressions have foldable functions") { val originalQuery = testRelation - .select(Cast(Literal("2"), IntegerType) + Literal(3) + 'a as Symbol("c1"), - Coalesce(Seq(Cast(Literal("abc"), IntegerType), Literal(3))) as Symbol("c2")) + .select( + Cast(Literal("2"), IntegerType) + Literal(3) + 'a as Symbol("c1"), + Coalesce(Seq(Cast(Literal("abc"), IntegerType), Literal(3))) as Symbol("c2")) val optimized = Optimize(originalQuery.analyze) val correctAnswer = testRelation - .select(Literal(5) + 'a as Symbol("c1"), - Literal(3) as Symbol("c2")) + .select( + Literal(5) + 'a as Symbol("c1"), + Literal(3) as Symbol("c2")) .analyze comparePlans(optimized, correctAnswer) @@ -153,15 +160,17 @@ class OptimizerSuite extends FunSuite { test("Constant folding test: expressions have nonfoldable functions") { val originalQuery = testRelation - .select(Rand + Literal(1) as Symbol("c1"), - Sum('a) as Symbol("c2")) + .select( + Rand + Literal(1) as Symbol("c1"), + Sum('a) as Symbol("c2")) val optimized = Optimize(originalQuery.analyze) val correctAnswer = testRelation - .select(Rand + Literal(1.0) as Symbol("c1"), - Sum('a) as Symbol("c2")) + .select( + Rand + Literal(1.0) as Symbol("c1"), + Sum('a) as Symbol("c2")) .analyze comparePlans(optimized, correctAnswer) From b22b2207fafb98b6f88fd069878ae5cf0eee41b2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 18:34:16 -0800 Subject: [PATCH 394/778] also look for cached hive test answers on the classpath. --- build.sbt | 2 ++ src/main/scala/catalyst/util/package.scala | 22 +++++++++++++++++++ .../execution/HiveComparisionTest.scala | 16 +++++++------- 3 files changed, 32 insertions(+), 8 deletions(-) diff --git a/build.sbt b/build.sbt index 38492be908829..5eaba7523d81e 100644 --- a/build.sbt +++ b/build.sbt @@ -12,6 +12,8 @@ resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/ libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating-SNAPSHOT" +libraryDependencies += "catalyst" % "hive-golden" % "0" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden.jar" + // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" diff --git a/src/main/scala/catalyst/util/package.scala b/src/main/scala/catalyst/util/package.scala index a0052997d9c9c..96e9ffd1fcbf6 100644 --- a/src/main/scala/catalyst/util/package.scala +++ b/src/main/scala/catalyst/util/package.scala @@ -34,6 +34,28 @@ package object util { new String(outStream.toByteArray(), encoding) } + def resourceToString( + resource:String, + encoding: String = "UTF-8", + classLoader: ClassLoader = this.getClass.getClassLoader) = { + val inStream = classLoader.getResourceAsStream(resource) + val outStream = new ByteArrayOutputStream + try { + var reading = true + while ( reading ) { + inStream.read() match { + case -1 => reading = false + case c => outStream.write(c) + } + } + outStream.flush() + } + finally { + inStream.close() + } + new String(outStream.toByteArray(), encoding) + } + def stringToFile(file: File, str: String): File = { val out = new PrintWriter(file) out.write(str) diff --git a/src/test/scala/catalyst/execution/HiveComparisionTest.scala b/src/test/scala/catalyst/execution/HiveComparisionTest.scala index efcfdda818799..e313b33e471fb 100644 --- a/src/test/scala/catalyst/execution/HiveComparisionTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisionTest.scala @@ -82,18 +82,18 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with } val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => - if(cachedAnswerFile.exists) { - val cachedString = fileToString(cachedAnswerFile) - val cachedAnswer = - if(cachedString == "") - Nil - else - cachedString.split("\n").toSeq - Some(cachedAnswer) + logger.debug(s"Looking for cached answer file $cachedAnswerFile.") + if (cachedAnswerFile.exists) { + Some(fileToString(cachedAnswerFile)) + } else if (getClass.getClassLoader.getResourceAsStream(cachedAnswerFile.toString) != null) { + Some(resourceToString(cachedAnswerFile.toString)) } else { logger.debug(s"File $cachedAnswerFile not found") None } + }.map { + case "" => Nil + case other => other.split("\n").toSeq } val hiveResults: Seq[Seq[String]] = From 6250cbd841db0c0876292ddab47f71445b2142a4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 18:34:36 -0800 Subject: [PATCH 395/778] Do not exit on test failure --- src/test/scala/catalyst/execution/HiveComparisionTest.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/src/test/scala/catalyst/execution/HiveComparisionTest.scala b/src/test/scala/catalyst/execution/HiveComparisionTest.scala index e313b33e471fb..47cacd83f02b3 100644 --- a/src/test/scala/catalyst/execution/HiveComparisionTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisionTest.scala @@ -182,7 +182,6 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with // The testing setup traps exits so wait here for a long time so the developer can see when things started // to go wrong. Thread.sleep(1000000) - System.exit(1) } } From 8e7ce30781f6c7de5bfb20a2b76b3bf9cbc0d0ff Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 19:48:57 -0800 Subject: [PATCH 396/778] blacklist some env specific tests. --- .../scala/catalyst/execution/HiveCompatability.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/test/scala/catalyst/execution/HiveCompatability.scala b/src/test/scala/catalyst/execution/HiveCompatability.scala index 76503e11eb86e..020c30c1c6019 100644 --- a/src/test/scala/catalyst/execution/HiveCompatability.scala +++ b/src/test/scala/catalyst/execution/HiveCompatability.scala @@ -26,6 +26,16 @@ class HiveCompatability extends HiveQueryFileTest { "index_compact_binary_search", "bucket_num_reducers", + // User specific test answers, breaks the caching mechanism. + "authorization_3", + "authorization_5", + "keyword_1", + "misc_json", + + // Timezone specific test answers. + "udf_unix_timestamp", + "udf_to_unix_timestamp", + // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 "ops_comparison", From e06fd38a0d1145060929ce075db9fe72649ef7e7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 20:14:39 -0800 Subject: [PATCH 397/778] black list map reduce tests. --- src/test/scala/catalyst/execution/HiveCompatability.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/test/scala/catalyst/execution/HiveCompatability.scala b/src/test/scala/catalyst/execution/HiveCompatability.scala index 020c30c1c6019..ce90753f6a53f 100644 --- a/src/test/scala/catalyst/execution/HiveCompatability.scala +++ b/src/test/scala/catalyst/execution/HiveCompatability.scala @@ -36,6 +36,11 @@ class HiveCompatability extends HiveQueryFileTest { "udf_unix_timestamp", "udf_to_unix_timestamp", + // Cant run without local map/reduce. + "index_auto_update", + "index_auto_self_join", + "index_stale", + // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 "ops_comparison", From b3bd15fa17a60847df874e4f6c769f27fa91eb4f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 20:40:49 -0800 Subject: [PATCH 398/778] blacklist more mr requiring tests. --- src/test/scala/catalyst/execution/HiveCompatability.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/scala/catalyst/execution/HiveCompatability.scala b/src/test/scala/catalyst/execution/HiveCompatability.scala index ce90753f6a53f..dd1e6c426810d 100644 --- a/src/test/scala/catalyst/execution/HiveCompatability.scala +++ b/src/test/scala/catalyst/execution/HiveCompatability.scala @@ -40,6 +40,7 @@ class HiveCompatability extends HiveQueryFileTest { "index_auto_update", "index_auto_self_join", "index_stale", + "type_cast_1", // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 From d72a5a2ca17b940ce9ef6f5a549c0f36cf78c260 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 22:18:44 -0800 Subject: [PATCH 399/778] add long to literal factory object. --- src/main/scala/catalyst/expressions/literals.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/expressions/literals.scala b/src/main/scala/catalyst/expressions/literals.scala index 35012adbf4ba1..527153f88fd16 100644 --- a/src/main/scala/catalyst/expressions/literals.scala +++ b/src/main/scala/catalyst/expressions/literals.scala @@ -5,8 +5,8 @@ import types._ object Literal { def apply(v: Any): Literal = v match { - // TODO(marmbrus): Use bigInt type for value? case i: Int => Literal(i, IntegerType) + case l: Long => Literal(l, LongType) case d: Double => Literal(d, DoubleType) case f: Float => Literal(f, FloatType) case b: Byte => Literal(b, ByteType) From 61b266fb5b12a326173671bd51aeedf081114f2e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 22:20:03 -0800 Subject: [PATCH 400/778] comment for eliminate subqueries. --- src/main/scala/catalyst/optimizer/Optimizer.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala index 91258b79e34d2..34601902005f8 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/src/main/scala/catalyst/optimizer/Optimizer.scala @@ -11,6 +11,10 @@ object Optimize extends RuleExecutor[LogicalPlan] { } +/** + * Removes subqueries from the plan. Subqueries are only required to provide scoping information + * for attributes and can be removed once analysis is complete. + */ object EliminateSubqueries extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Subquery(_, child) => child From 0cd5cc6ae199d6a3a426cfd3b8035213e47dd708 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 4 Jan 2014 23:52:01 -0800 Subject: [PATCH 401/778] add BIGINT cast parsing --- src/main/scala/catalyst/frontend/Hive.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 2d907211711e0..3125550eca48b 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -615,6 +615,7 @@ object HiveQl { /* Casts */ case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), IntegerType) + case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), LongType) case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), FloatType) case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), DoubleType) case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ShortType) From 367fb9e56b1a92e85bacd0a290a87d153f429984 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 5 Jan 2014 13:01:42 -0500 Subject: [PATCH 402/778] update --- src/main/scala/catalyst/expressions/literals.scala | 1 - src/main/scala/catalyst/optimizer/Optimizer.scala | 10 +++++----- src/main/scala/catalyst/types/dataTypes.scala | 1 - 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/main/scala/catalyst/expressions/literals.scala b/src/main/scala/catalyst/expressions/literals.scala index 796ac67e0571a..66498a989ecad 100644 --- a/src/main/scala/catalyst/expressions/literals.scala +++ b/src/main/scala/catalyst/expressions/literals.scala @@ -13,7 +13,6 @@ object Literal { case s: Short => Literal(s, ShortType) case s: String => Literal(s, StringType) case b: Boolean => Literal(b, BooleanType) - case null => Literal(null, NullType) } } diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala index a2453059e7ae9..1e234d3034f8f 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/src/main/scala/catalyst/optimizer/Optimizer.scala @@ -25,7 +25,7 @@ object EliminateSubqueries extends Rule[LogicalPlan] { object ConstantFolding extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { case q: LogicalPlan => q transformExpressionsUp { - case e if e.foldable => {Literal(Evaluate(e, Nil))} + case e if e.foldable => Literal(Evaluate(e, Nil), e.dataType) } } } @@ -33,22 +33,22 @@ object ConstantFolding extends Rule[LogicalPlan] { object BooleanSimpliï¬cation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { case q: LogicalPlan => q transformExpressionsUp { - case a @ And(left, right) => { + case and @ And(left, right) => { (left, right) match { case (Literal(true, BooleanType), r) => r case (l, Literal(true, BooleanType)) => l case (Literal(false, BooleanType), _) => Literal(false) case (_, Literal(false, BooleanType)) => Literal(false) - case (_, _) => a + case (_, _) => and } } - case o @ Or(left, right) => { + case or @ Or(left, right) => { (left, right) match { case (Literal(true, BooleanType), _) => Literal(true) case (_, Literal(true, BooleanType)) => Literal(true) case (Literal(false, BooleanType), r) => r case (l, Literal(false, BooleanType)) => l - case (_, _) => o + case (_, _) => or } } } diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 92f683d5e3ab6..4decfb607ac62 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -13,7 +13,6 @@ case object DoubleType extends DataType case object LongType extends DataType case object ByteType extends DataType case object ShortType extends DataType -case object NullType extends DataType case class ArrayType(elementType: DataType) extends DataType From 3a90a5ffc244d606dbccd678f4232498f3032335 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 5 Jan 2014 11:04:39 -0800 Subject: [PATCH 403/778] simpler output when running a single test from the commandline. --- src/test/scala/catalyst/execution/HiveQueryFileTest.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala index fa93af8daa3c6..a11977a557ad8 100644 --- a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala +++ b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala @@ -25,8 +25,9 @@ abstract class HiveQueryFileTest extends HiveComaparisionTest { val runAll = !(System.getProperty("shark.hive.alltests") == null) + val whiteListProperty = "shark.hive.whitelist" // Allow the whiteList to be overridden by a system property - val realWhiteList = Option(System.getProperty("shark.hive.whitelist")).map(_.split(",").toSeq).getOrElse(whiteList) + val realWhiteList = Option(System.getProperty(whiteListProperty)).map(_.split(",").toSeq).getOrElse(whiteList) // Go through all the test cases and add them to scala test. testCases.foreach { @@ -38,7 +39,10 @@ abstract class HiveQueryFileTest extends HiveComaparisionTest { val queriesString = fileToString(testCaseFile) createQueryTest(testCaseName, queriesString) } else { - ignore(testCaseName) {} + // Only output warnings for the built in whitelist as this clutters the output when the user + // trying to execute a single test from the commandline. + if(System.getProperty(whiteListProperty) == null) + ignore(testCaseName) {} } } } \ No newline at end of file From c16c8b5cc4395d5fa24b7ac84da16d34023bf829 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 5 Jan 2014 11:04:56 -0800 Subject: [PATCH 404/778] case insensitive checking for hooks in tests. --- src/test/scala/catalyst/execution/HiveComparisionTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/catalyst/execution/HiveComparisionTest.scala b/src/test/scala/catalyst/execution/HiveComparisionTest.scala index efcfdda818799..4d21905b0c1a3 100644 --- a/src/test/scala/catalyst/execution/HiveComparisionTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisionTest.scala @@ -58,7 +58,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with str.replaceAll("file:\\/.*\\/", "") } - val installHooksCommand = "SET.*hooks".r + val installHooksCommand = "(?i)SET.*hooks".r def createQueryTest(testCaseName: String, sql: String) = { test(testCaseName) { logger.error( From ab8bbd1221f9c7c799cf49dc5dd0b5851d03d51e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 5 Jan 2014 11:05:11 -0800 Subject: [PATCH 405/778] parsing % operator --- src/main/scala/catalyst/frontend/Hive.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 3125550eca48b..97ecff6b23d3c 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -629,6 +629,7 @@ object HiveQl { case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) case Token("/", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) case Token("DIV", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) + case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) /* Comparisons */ case Token("=", left :: right:: Nil) => Equals(nodeToExpr(left), nodeToExpr(right)) From 8ca38d0786a95b577f9841ac094713315a3789b9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 5 Jan 2014 11:05:38 -0800 Subject: [PATCH 406/778] begin support for varchar / binary types. --- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 3 +++ src/main/scala/catalyst/execution/hiveOperators.scala | 1 + 2 files changed, 4 insertions(+) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 96e61ae0cc6a7..1809a96e9dedf 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -77,12 +77,15 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { } object HiveMetatoreTypes { + val VARCHAR = "(?i)VARCHAR\\((\\d+)\\)".r def toDataType(metastoreType: String): DataType = metastoreType match { case "string" => StringType case "int" => IntegerType case "double" => DoubleType case "bigint" => LongType + case "binary" => BinaryType + case VARCHAR(_) => StringType } } diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 9827fae3bf4d7..a4a806e7aa032 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -68,6 +68,7 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation buildRow(values.map { case "NULL" => null case "null" => null + case varchar: org.apache.hadoop.hive.common.`type`.HiveVarchar => varchar.getValue case other => other }) } From a43d41cc92c7051c652cb10307ec4eff5fee1a13 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 5 Jan 2014 11:05:49 -0800 Subject: [PATCH 407/778] more tests passing! --- src/test/scala/catalyst/execution/HiveCompatability.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/test/scala/catalyst/execution/HiveCompatability.scala b/src/test/scala/catalyst/execution/HiveCompatability.scala index 76503e11eb86e..2c7f017f0a93b 100644 --- a/src/test/scala/catalyst/execution/HiveCompatability.scala +++ b/src/test/scala/catalyst/execution/HiveCompatability.scala @@ -188,12 +188,12 @@ class HiveCompatability extends HiveQueryFileTest { "literal_string", "load_file_with_space_in_the_name", "mapjoin_subquery2", + "mapreduce3", "mergejoins", "mergejoins_mixed", "misc_json", "multi_join_union", "multigroupby_singlemr", - "no_hooks", "noalias_subq1", "nomore_ambiguous_table_col", "notable_alias1", @@ -301,6 +301,7 @@ class HiveCompatability extends HiveQueryFileTest { "udf_minute", "udf_modulo", "udf_month", + "udf_negative", "udf_not", "udf_notequal", "udf_nvl", @@ -374,6 +375,9 @@ class HiveCompatability extends HiveQueryFileTest { "union7", "union8", "union9", - "union_script" + "union_script", + "varchar_2", + "varchar_join1", + "varchar_union1" ) } From 1e964ea548ccd461ec74f900991f67af9eee0b89 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 5 Jan 2014 15:05:08 -0500 Subject: [PATCH 408/778] update --- README.md | 4 ++-- .../scala/catalyst/analysis/Analyzer.scala | 18 +++++++++--------- .../scala/catalyst/analysis/typeCoercion.scala | 8 ++++---- .../scala/catalyst/examples/ViewsExample.scala | 2 +- .../catalyst/execution/MetastoreCatalog.scala | 2 +- .../scala/catalyst/execution/aggregates.scala | 6 +++--- src/main/scala/catalyst/execution/debug.scala | 2 +- .../catalyst/expressions/BoundAttribute.scala | 4 ++-- .../scala/catalyst/expressions/literals.scala | 1 + src/main/scala/catalyst/frontend/Hive.scala | 2 +- .../scala/catalyst/optimizer/Optimizer.scala | 8 ++++---- src/main/scala/catalyst/plans/QueryPlan.scala | 16 +++++++++++++--- .../catalyst/plans/logical/TestRelation.scala | 2 +- src/main/scala/catalyst/trees/TreeNode.scala | 11 +++++++++++ src/main/scala/catalyst/types/dataTypes.scala | 2 ++ .../scala/catalyst/RuleExecutorSuite.scala | 2 +- src/test/scala/catalyst/TreeNodeSuite.scala | 6 +++--- 17 files changed, 60 insertions(+), 36 deletions(-) diff --git a/README.md b/README.md index b6a2354c0c4b1..fb7b0d5aeef6e 100644 --- a/README.md +++ b/README.md @@ -73,7 +73,7 @@ Project {key#0,value#1} HiveTableScan {key#0,value#1}, (MetastoreRelation default, src, None) ``` -From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transformDown` function that is available on all [`TreeNode`](http://marmbrus.github.io/catalyst/latest/api/index.html#catalyst.trees.TreeNode) objects. +From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](http://marmbrus.github.io/catalyst/latest/api/index.html#catalyst.trees.TreeNode) objects. ```scala scala> query.optimizedPlan res1: catalyst.plans.logical.LogicalPlan = @@ -82,7 +82,7 @@ Project {key#0,value#1} MetastoreRelation default, src, None -scala> res0.optimizedPlan transformDown { +scala> res0.optimizedPlan transform { | case Project(projectList, child) if projectList == child.output => child | } res2: catalyst.plans.logical.LogicalPlan = diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 92fbf3ffcbecd..0624d51451bd9 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -46,7 +46,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { case UnresolvedRelation(name, alias) => catalog.lookupRelation(name, alias) } } @@ -55,10 +55,10 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * Makes attribute naming case insensitive by turning all UnresolvedAttributes to lowercase. */ object LowercaseAttributeReferences extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { case UnresolvedRelation(name, alias) => UnresolvedRelation(name, alias.map(_.toLowerCase)) case Subquery(alias, child) => Subquery(alias.toLowerCase, child) - case q: LogicalPlan => q transformExpressionsDown { + case q: LogicalPlan => q transformExpressions { case s: Star => s.copy(table = s.table.map(_.toLowerCase)) case UnresolvedAttribute(name) => UnresolvedAttribute(name.toLowerCase) case Alias(c, name) => Alias(c, name.toLowerCase)() @@ -71,10 +71,10 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * from a logical plan node's children. */ object ResolveReferences extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case q: LogicalPlan if childIsFullyResolved(q) => logger.trace(s"Attempting to resolve ${q.simpleString}") - q transformExpressionsDown { + q transformExpressions { case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. val result = q.resolve(name).getOrElse(u) @@ -88,9 +88,9 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * Replaces [[UnresolvedFunction]]s with concrete [[Expression]]s. */ object ResolveFunctions extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => - q transformExpressionsDown { + q transformExpressions { case UnresolvedFunction(name, children) if children.map(_.resolved).reduceLeft(_&&_) => registry.lookupFunction(name, children) } @@ -101,7 +101,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * Turns projections that contain aggregate expressions into aggregations. */ object GlobalAggregates extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Project(projectList, child) if containsAggregates(projectList) => Aggregate(Nil, projectList, child) } @@ -119,7 +119,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool * Expands any references to [[Star]] (*) in project operators. */ object StarExpansion extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Wait until children are resolved case p: LogicalPlan if !childIsFullyResolved(p) => p // If the projection list contains Stars, expand it. diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index 42125be995222..968d9135632dd 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -13,8 +13,8 @@ import types._ object ConvertNaNs extends Rule[LogicalPlan] { val stringNaN = Literal("NaN", StringType) - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { - case q: LogicalPlan => q transformExpressionsDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e @@ -57,8 +57,8 @@ object PromoteNumericTypes extends Rule[LogicalPlan] { val toFloat = Seq(ByteType, ShortType, IntegerType) :+ FloatType val allPromotions = integralPrecedence :: toDouble :: toFloat :: Nil - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { - case q: LogicalPlan => q transformExpressionsDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e diff --git a/src/main/scala/catalyst/examples/ViewsExample.scala b/src/main/scala/catalyst/examples/ViewsExample.scala index 2dc470de538f3..7eee3b868f1f8 100644 --- a/src/main/scala/catalyst/examples/ViewsExample.scala +++ b/src/main/scala/catalyst/examples/ViewsExample.scala @@ -28,7 +28,7 @@ object ViewsExample { println(s"Unresolved Plan:\n$unresolvedPlan") // Replace UnresolvedRelations with logical plans from the views map. - val withRelations = unresolvedPlan transformDown { + val withRelations = unresolvedPlan transform { case UnresolvedRelation(name, _) => views(name) } diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 3baf442b0c8db..96e61ae0cc6a7 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -43,7 +43,7 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { * For example, because of a CREATE TABLE X AS statement. */ object CreateTables extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { case InsertIntoCreatedTable(name, child) => val (databaseName, tableName) = name.split("\\.") match { case Array(tableOnly) => ("default", tableOnly) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index d911863e5339a..81ccaafd10d24 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -93,7 +93,7 @@ case class Aggregate( grouped.map { case (group, rows) => // Replace all aggregate expressions with spark functions that will compute the result. val aggImplementations = aggregateExpressions.map { agg => - val impl = agg transformDown { + val impl = agg transform { case base @ Average(expr) => new AverageFunction(expr, base) case base @ Sum(expr) => new SumFunction(expr, base) case base @ Count(expr) => new CountFunction(expr, base) @@ -105,7 +105,7 @@ case class Aggregate( // If any references exist that are not inside agg functions then the must be grouping exprs // in this case we must rebind them to the grouping tuple. if (remainingAttributes.nonEmpty) { - val unaliasedAggregateExpr = agg transformDown { case Alias(c, _) => c } + val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } // An exact match with a grouping expression val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { @@ -171,7 +171,7 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan override def executeCollect() = attachTree(this, "SparkAggregate") { // Replace all aggregate expressions with spark functions that will compute the result. - val aggImplementations = aggregateExprs.map { _ transformDown { + val aggImplementations = aggregateExprs.map { _ transform { case base @ Average(expr) => new AverageFunction(expr, base) case base @ Count(expr) => new CountFunction(expr, base) }} diff --git a/src/main/scala/catalyst/execution/debug.scala b/src/main/scala/catalyst/execution/debug.scala index d18f3089a852c..a1e262f3e3035 100644 --- a/src/main/scala/catalyst/execution/debug.scala +++ b/src/main/scala/catalyst/execution/debug.scala @@ -4,7 +4,7 @@ package execution object DebugQuery { def apply(plan: SharkPlan): SharkPlan = { val visited = new collection.mutable.HashSet[Long]() - plan transformDown { + plan transform { case s: SharkPlan if !visited.contains(s.id) => visited += s.id DebugNode(s) diff --git a/src/main/scala/catalyst/expressions/BoundAttribute.scala b/src/main/scala/catalyst/expressions/BoundAttribute.scala index 76c66fe42a45c..edb17a9b5a6e4 100644 --- a/src/main/scala/catalyst/expressions/BoundAttribute.scala +++ b/src/main/scala/catalyst/expressions/BoundAttribute.scala @@ -30,11 +30,11 @@ case class BoundReference(inputTuple: Int, ordinal: Int, baseReference: Attribut // TODO: Should run against any query plan, not just SharkPlans object BindReferences extends Rule[SharkPlan] { def apply(plan: SharkPlan): SharkPlan = { - plan.transformDown { + plan.transform { case leafNode: SharkPlan if leafNode.children.isEmpty => leafNode case nonLeaf: SharkPlan => attachTree(nonLeaf, "Binding references in operator") { logger.debug(s"Binding references in node ${nonLeaf.simpleString}") - nonLeaf.transformExpressionsDown { + nonLeaf.transformExpressions { case a: AttributeReference => attachTree(a, "Binding attribute") { val inputTuple = nonLeaf.children.indexWhere(_.output contains a) val ordinal = if (inputTuple == -1) -1 else nonLeaf.children(inputTuple).output.indexWhere(_ == a) diff --git a/src/main/scala/catalyst/expressions/literals.scala b/src/main/scala/catalyst/expressions/literals.scala index ade4a91fa8786..56062e22e3227 100644 --- a/src/main/scala/catalyst/expressions/literals.scala +++ b/src/main/scala/catalyst/expressions/literals.scala @@ -13,6 +13,7 @@ object Literal { case s: Short => Literal(s, ShortType) case s: String => Literal(s, StringType) case b: Boolean => Literal(b, BooleanType) + case null => Literal(null, NullType) } } diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index baed86957b602..f911b992ffe37 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -474,7 +474,7 @@ object HiveQl { val joinedTables = tables.reduceLeft(Join(_,_, Inner, None)) // Must be transform down. - val joinedResult = joinedTables transformDown { + val joinedResult = joinedTables transform { case j: Join => j.copy( condition = Some(joinConditions.remove(joinConditions.length - 1)), diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala index 52c2017455833..d53c75fb9f364 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/src/main/scala/catalyst/optimizer/Optimizer.scala @@ -21,21 +21,21 @@ object Optimize extends RuleExecutor[LogicalPlan] { * for attributes and can be removed once analysis is complete. */ object EliminateSubqueries extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Subquery(_, child) => child } } object ConstantFolding extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { - case q: LogicalPlan => q transformExpressionsUp { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressionsDown { case e if e.foldable => Literal(Evaluate(e, Nil), e.dataType) } } } object BooleanSimpliï¬cation extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { case and @ And(left, right) => { (left, right) match { diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 205de19dc3053..d91a129c2ca70 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -14,6 +14,16 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy */ def outputSet: Set[Attribute] = output.toSet + /** + * Runs [[transform]] with [[rule]] on all expressions present in this query operator. + * Users should not expect a specific directionality. If a specific directionality is needed, + * transformExpressionsDown or transformExpressionsUp should be used. + * @param rule the rule to be applied to every expression in this operator. + */ + def transformExpressions(rule: PartialFunction[Expression, Expression]): this.type = { + transformExpressionsDown(rule) + } + /** * Runs [[transformDown]] with [[rule]] on all expressions present in this query operator. * @param rule the rule to be applied to every expression in this operator. @@ -75,11 +85,11 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy if (changed) makeCopy(newArgs) else this } - /** Returns the result of running [[transformExpressionsDown]] on this node + /** Returns the result of running [[transformExpressions]] on this node * and all its children. */ def transformAllExpressions(rule: PartialFunction[Expression, Expression]): this.type = { - transformDown { - case q: QueryPlan[_] => q.transformExpressionsDown(rule).asInstanceOf[PlanType] + transform { + case q: QueryPlan[_] => q.transformExpressions(rule).asInstanceOf[PlanType] }.asInstanceOf[this.type] } diff --git a/src/main/scala/catalyst/plans/logical/TestRelation.scala b/src/main/scala/catalyst/plans/logical/TestRelation.scala index b4379705a7c8d..88535680fa075 100644 --- a/src/main/scala/catalyst/plans/logical/TestRelation.scala +++ b/src/main/scala/catalyst/plans/logical/TestRelation.scala @@ -39,7 +39,7 @@ object NewLocalRelationInstances extends Rule[LogicalPlan] { .map(_._1) .toSet - plan transformDown { + plan transform { case l: LocalRelation if multiAppearance contains l => l.newInstance } } diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index bdb97faa2ebd4..337e5204746c2 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -82,6 +82,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { ret } + /** + * Returns a copy of this node where [[rule]] has been recursively applied to the tree. + * When [[rule]] does not apply to a given node it is left unchanged. + * Users should not expect a specific directionality. If a specific directionality is needed, + * transformDown or transformUp should be used. + * @param rule the function use to transform this nodes children + */ + def transform(rule: PartialFunction[BaseType, BaseType]): BaseType = { + transformDown(rule) + } + /** * Returns a copy of this node where [[rule]] has been recursively * applied to it and all of its children (pre-order). When [[rule]] does not diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 4decfb607ac62..8627a016efdb1 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -14,6 +14,8 @@ case object LongType extends DataType case object ByteType extends DataType case object ShortType extends DataType +case object NullType extends DataType + case class ArrayType(elementType: DataType) extends DataType case class StructField(name: String, dataType: DataType) diff --git a/src/test/scala/catalyst/RuleExecutorSuite.scala b/src/test/scala/catalyst/RuleExecutorSuite.scala index fcc2994930f00..29434379ce7af 100644 --- a/src/test/scala/catalyst/RuleExecutorSuite.scala +++ b/src/test/scala/catalyst/RuleExecutorSuite.scala @@ -8,7 +8,7 @@ import rules._ class RuleExecutorSuite extends FunSuite { object DecrementLiterals extends Rule[Expression] { - def apply(e: Expression): Expression = e transformDown { + def apply(e: Expression): Expression = e transform { case IntegerLiteral(i) if i > 0 => Literal(i - 1) } } diff --git a/src/test/scala/catalyst/TreeNodeSuite.scala b/src/test/scala/catalyst/TreeNodeSuite.scala index 9167d44d91427..45ed71f03f80e 100644 --- a/src/test/scala/catalyst/TreeNodeSuite.scala +++ b/src/test/scala/catalyst/TreeNodeSuite.scala @@ -10,20 +10,20 @@ import org.scalatest.{FunSuite} class TreeNodeSuite extends FunSuite { test("top node changed") { - val after = Literal(1) transformDown { case Literal(1, _) => Literal(2) } + val after = Literal(1) transform { case Literal(1, _) => Literal(2) } assert(after === Literal(2)) } test("one child changed") { val before = Add(Literal(1), Literal(2)) - val after = before transformDown { case Literal(2, _) => Literal(1) } + val after = before transform { case Literal(2, _) => Literal(1) } assert(after === Add(Literal(1), Literal(1))) } test("no change") { val before = Add(Literal(1), Add(Literal(2), Add(Literal(3), Literal(4)))) - val after = before transformDown { case Literal(5, _) => Literal(1)} + val after = before transform { case Literal(5, _) => Literal(1)} assert(before === after) assert(before.map(_.id) === after.map(_.id)) From 1a393da5f064fc3275795655880f6cb1adca29d2 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 5 Jan 2014 15:06:47 -0500 Subject: [PATCH 409/778] folded -> foldable --- src/main/scala/catalyst/expressions/Expression.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/expressions/Expression.scala b/src/main/scala/catalyst/expressions/Expression.scala index dcc9630c9df0d..1bca22cdac824 100644 --- a/src/main/scala/catalyst/expressions/Expression.scala +++ b/src/main/scala/catalyst/expressions/Expression.scala @@ -17,7 +17,7 @@ abstract class Expression extends TreeNode[Expression] { * - A Literal is foldable. * - A Cast or UnaryMinus is foldable if its child is foldable. */ - // TODO: Supporting more folded expressions. For example, deterministic Hive UDFs. + // TODO: Supporting more foldable expressions. For example, deterministic Hive UDFs. def foldable: Boolean = false def nullable: Boolean def references: Set[Attribute] From 01c00c27f1d5a7f6932720d19267375d7019adc7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 6 Jan 2014 00:44:52 -0800 Subject: [PATCH 410/778] new golden --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 5eaba7523d81e..dca4fd53f5334 100644 --- a/build.sbt +++ b/build.sbt @@ -12,7 +12,7 @@ resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/ libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating-SNAPSHOT" -libraryDependencies += "catalyst" % "hive-golden" % "0" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden.jar" +libraryDependencies += "catalyst" % "hive-golden" % "2" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden2.jar" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" From 0b1960aef23a6e908627b82ff996bfcfab35ee69 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 6 Jan 2014 14:46:46 -0800 Subject: [PATCH 411/778] Fix broken scala doc links / warnings. --- .../scala/catalyst/analysis/Analyzer.scala | 8 +-- src/main/scala/catalyst/errors/package.scala | 8 +-- .../catalyst/examples/ViewsExample.scala | 1 + .../catalyst/execution/SharkContext.scala | 2 +- .../execution/planningStrategies.scala | 8 +-- .../expressions/namedExpressions.scala | 2 +- src/main/scala/catalyst/frontend/Hive.scala | 9 ++- src/main/scala/catalyst/package.scala | 5 +- .../catalyst/planning/QueryPlanner.scala | 20 +++--- .../scala/catalyst/planning/patterns.scala | 12 ++-- src/main/scala/catalyst/plans/QueryPlan.scala | 6 +- .../catalyst/plans/logical/LogicalPlan.scala | 3 +- src/main/scala/catalyst/trees/TreeNode.scala | 63 ++++++++++--------- 13 files changed, 77 insertions(+), 70 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 0624d51451bd9..a0a03933f85ae 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -67,8 +67,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s - * from a logical plan node's children. + * Replaces [[UnresolvedAttribute]]s with concrete + * [[expressions.AttributeReference AttributeReferences]] from a logical plan node's children. */ object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { @@ -85,7 +85,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Replaces [[UnresolvedFunction]]s with concrete [[Expression]]s. + * Replaces [[UnresolvedFunction]]s with concrete [[expressions.Expression Expressions]]. */ object ResolveFunctions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -141,7 +141,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Returns true if [[exprs]] contains a star. + * Returns true if `exprs` contains a [[Star]]. */ protected def containsStar(exprs: Seq[NamedExpression]): Boolean = exprs.collect { case _: Star => true }.nonEmpty diff --git a/src/main/scala/catalyst/errors/package.scala b/src/main/scala/catalyst/errors/package.scala index 7a472da9e3e53..41bd9c1d8055e 100644 --- a/src/main/scala/catalyst/errors/package.scala +++ b/src/main/scala/catalyst/errors/package.scala @@ -17,8 +17,8 @@ package object errors { } /** - * Wraps any exceptions that are thrown while executing [[f]] in an [[OptimizationException]], attaching the provided - * [[tree]]. + * Wraps any exceptions that are thrown while executing `f` in an [[OptimizationException]], + * attaching the provided `tree`. */ def attachTree[TreeType <: TreeNode[_], A](tree: TreeType, msg: String = "")(f: => A): A = { try f catch { @@ -27,8 +27,8 @@ package object errors { } /** - * Executes [[f]] which is expected to throw an OptimizationException. The first tree encountered in the stack - * of exceptions of type [[TreeType]] is returned. + * Executes `f` which is expected to throw an OptimizationException. The first tree encountered in + * the stack of exceptions of type `TreeType` is returned. */ def getTree[TreeType <: TreeNode[_]](f: => Unit): TreeType = ??? // TODO: Implement } \ No newline at end of file diff --git a/src/main/scala/catalyst/examples/ViewsExample.scala b/src/main/scala/catalyst/examples/ViewsExample.scala index 7eee3b868f1f8..5bc1bc1262454 100644 --- a/src/main/scala/catalyst/examples/ViewsExample.scala +++ b/src/main/scala/catalyst/examples/ViewsExample.scala @@ -1,4 +1,5 @@ package catalyst +package examples import catalyst.analysis.UnresolvedRelation import catalyst.plans.Inner diff --git a/src/main/scala/catalyst/execution/SharkContext.scala b/src/main/scala/catalyst/execution/SharkContext.scala index a56b545e666c2..cdd53bcb90d8a 100644 --- a/src/main/scala/catalyst/execution/SharkContext.scala +++ b/src/main/scala/catalyst/execution/SharkContext.scala @@ -30,7 +30,7 @@ class SharkContext( import SharkContext._ /** - * Execute the command and return the results as a sequence. Each element + * Execute the command using Hive and return the results as a sequence. Each element * in the sequence is one row. */ def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 5211feb45ee4a..92a34102fa495 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -30,8 +30,8 @@ trait PlanningStrategies { } /** - * Returns true if [[projectList]] only performs column pruning and - * does not evaluate other complex expressions. + * Returns true if `projectList` only performs column pruning and does not evaluate other + * complex expressions. */ def isSimpleProject(projectList: Seq[NamedExpression]) = { projectList.map { @@ -50,7 +50,7 @@ trait PlanningStrategies { classOf[Average]) /** - * Returns true if [[exprs]] contains only aggregates that can be computed using Accumulators. + * Returns true if `exprs` only contains aggregates that can be computed using Accumulators. */ def onlyAllowedAggregates(exprs: Seq[Expression]): Boolean = { val aggs = exprs.flatMap(_.collect { case a: AggregateExpression => a}).map(_.getClass) @@ -106,7 +106,7 @@ trait PlanningStrategies { private def combineConjunctivePredicates(predicates: Seq[Expression]) = predicates.reduceLeft(And(_, _)) - /** Returns true if [[expr]] can be evaluated using only the output of [[plan]]. */ + /** Returns true if `expr` can be evaluated using only the output of `plan`. */ protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = expr.references subsetOf plan.outputSet } diff --git a/src/main/scala/catalyst/expressions/namedExpressions.scala b/src/main/scala/catalyst/expressions/namedExpressions.scala index bd85009a8af30..73c727ff3a5c8 100644 --- a/src/main/scala/catalyst/expressions/namedExpressions.scala +++ b/src/main/scala/catalyst/expressions/namedExpressions.scala @@ -71,7 +71,7 @@ case class Alias(child: Expression, name: String) * A reference to an attribute produced by another operator in the tree. * * @param name The name of this attribute, should only be used during analysis or for debugging. - * @param dataType The [[DataType]] of this attribute. + * @param dataType The [[types.DataType DataType]] of this attribute. * @param nullable True if null is a valid value for this attribute. * @param exprId A globally unique id used to check if different AttributeReferences refer to the * same attribute. diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index c766372a5f750..7161690c29cdd 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -126,9 +126,8 @@ object HiveQl { */ implicit class TransformableNode(n: ASTNode) { /** - * Returns a copy of this node where [[rule]] has been recursively - * applied to it and all of its children. When [[rule]] does not - * apply to a given node it is left unchanged. + * Returns a copy of this node where `rule` has been recursively applied to it and all of its + * children. When `rule` does not apply to a given node it is left unchanged. * @param rule the function use to transform this nodes children */ def transform(rule: PartialFunction[ASTNode, ASTNode]): ASTNode = { @@ -152,7 +151,7 @@ object HiveQl { Option(s).map(_.toSeq).getOrElse(Nil) /** - * Returns this ASTNode with the text changed to [[newText]]. + * Returns this ASTNode with the text changed to `newText``. */ def withText(newText: String): ASTNode = { n.token.asInstanceOf[org.antlr.runtime.CommonToken].setText(newText) @@ -160,7 +159,7 @@ object HiveQl { } /** - * Returns this ASTNode with the children changed to [[newChildren]]. + * Returns this ASTNode with the children changed to `newChildren`. */ def withChildren(newChildren: Seq[ASTNode]): ASTNode = { (1 to n.getChildCount).foreach(_ => n.deleteChild(0)) diff --git a/src/main/scala/catalyst/package.scala b/src/main/scala/catalyst/package.scala index 78a72ac65356f..3355b6402a834 100644 --- a/src/main/scala/catalyst/package.scala +++ b/src/main/scala/catalyst/package.scala @@ -3,6 +3,7 @@ * Catalyst is a framework for performing optimization on trees of dataflow operators. */ package object catalyst { - def Logger(name: String) = com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger(name)) - type Logging = com.typesafe.scalalogging.slf4j.Logging + protected[catalyst] def Logger(name: String) = + com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger(name)) + protected[catalyst] type Logging = com.typesafe.scalalogging.slf4j.Logging } \ No newline at end of file diff --git a/src/main/scala/catalyst/planning/QueryPlanner.scala b/src/main/scala/catalyst/planning/QueryPlanner.scala index 7024ca4b75bb3..cc4c89aab7f0b 100644 --- a/src/main/scala/catalyst/planning/QueryPlanner.scala +++ b/src/main/scala/catalyst/planning/QueryPlanner.scala @@ -6,13 +6,13 @@ import plans.logical.LogicalPlan import trees._ /** - * Extended by classes that transform [[LogicalPlan]]s into physical plans. Child classes are - * responsible for specifying a list of [[Strategy]] objects that each of which can return a list - * of possible physical plan options. If a given strategy is unable to plan all of the remaining - * operators in the tree, it can call [[planLater]], which returns a placeholder object that will - * be filled in using other available strategies. + * Extended by classes that transform [[plans.logical.LogicalPlan LogicalPlan]]s into physical plans. + * Child classes are responsible for specifying a list of [[Strategy]] objects that each of which + * can return a list of possible physical plan options. If a given strategy is unable to plan all + * of the remaining operators in the tree, it can call [[planLater]], which returns a placeholder + * object that will be filled in using other available strategies. * - * NOTE: RIGHT NOW ONLY ONE PLAN IS RETURNED EVER... PLAN SPACE EXPLORATION WILL BE IMPLEMENTED LATER. + * TODO: RIGHT NOW ONLY ONE PLAN IS RETURNED EVER... PLAN SPACE EXPLORATION WILL BE IMPLEMENTED LATER. * * @tparam PhysicalPlan The type of physical plan produced by this [[QueryPlanner]] */ @@ -21,16 +21,16 @@ abstract class QueryPlanner[PhysicalPlan <: TreeNode[PhysicalPlan]] { def strategies: Seq[Strategy] /** - * Given a [[LogicalPlan]], returns a list of [[PhysicalPlans]] that can be used for execution. - * If this strategy does not apply to the give logical operation then an empty list should be - * returned. + * Given a [[plans.logical.LogicalPlan LogicalPlan]], returns a list of `PhysicalPlan`s that can + * be used for execution. If this strategy does not apply to the give logical operation then an + * empty list should be returned. */ abstract protected class Strategy extends Logging { def apply(plan: LogicalPlan): Seq[PhysicalPlan] } /** - * Returns a placeholder for a physical plan that executes [[plan]]. This placeholder will be + * Returns a placeholder for a physical plan that executes `plan`. This placeholder will be * filled in automatically by the QueryPlanner using the other execution strategies that are * available. */ diff --git a/src/main/scala/catalyst/planning/patterns.scala b/src/main/scala/catalyst/planning/patterns.scala index ba88167df7843..796ccc37dd2c6 100644 --- a/src/main/scala/catalyst/planning/patterns.scala +++ b/src/main/scala/catalyst/planning/patterns.scala @@ -7,11 +7,12 @@ import expressions._ import plans.logical._ /** - * A pattern that matches any number of filter operations on top of another relational operator. Adjacent filter - * operators are collected and their conditions are broken up and returned as a sequence of conjunctive predicates. + * A pattern that matches any number of filter operations on top of another relational operator. + * Adjacent filter operators are collected and their conditions are broken up and returned as a + * sequence of conjunctive predicates. * - * @returns A tuple containing a sequence of conjunctive predicates that should be used to filter the output and a - * relational operator. + * @return A tuple containing a sequence of conjunctive predicates that should be used to filter the + * output and a relational operator. */ object FilteredOperation { type ReturnType = (Seq[Expression], LogicalPlan) @@ -20,7 +21,8 @@ object FilteredOperation { @tailrec private def collectFilters(filters: Seq[Expression], plan: LogicalPlan): ReturnType = plan match { - case Filter(condition, child) => collectFilters(filters ++ splitConjunctivePredicates(condition), child) + case Filter(condition, child) => + collectFilters(filters ++ splitConjunctivePredicates(condition), child) case other => (filters, other) } diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index d91a129c2ca70..3546416402f11 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -15,7 +15,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy def outputSet: Set[Attribute] = output.toSet /** - * Runs [[transform]] with [[rule]] on all expressions present in this query operator. + * Runs [[transform]] with `rule` on all expressions present in this query operator. * Users should not expect a specific directionality. If a specific directionality is needed, * transformExpressionsDown or transformExpressionsUp should be used. * @param rule the rule to be applied to every expression in this operator. @@ -25,7 +25,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy } /** - * Runs [[transformDown]] with [[rule]] on all expressions present in this query operator. + * Runs [[transformDown]] with `rule` on all expressions present in this query operator. * @param rule the rule to be applied to every expression in this operator. */ def transformExpressionsDown(rule: PartialFunction[Expression, Expression]): this.type = { @@ -55,7 +55,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy } /** - * Runs [[transformUp]] with [[rule]] on all expressions present in this query operator. + * Runs [[transformUp]] with `rule` on all expressions present in this query operator. * @param rule the rule to be applied to every expression in this operator. * @return */ diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala index 8f6dca8a80a59..27b96b5219085 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala @@ -23,7 +23,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { /** * Returns true if this expression and all its children have been resolved to a specific schema * and false if it is still contains any unresolved placeholders. Implementations of LogicalPlan - * can override this (e.g. [[UnresolvedRelation]] can set this to false). + * can override this (e.g. [[catalyst.analysis.UnresolvedRelation UnresolvedRelation]] should + * return `false`). */ lazy val resolved: Boolean = !expressions.exists(!_.resolved) && childrenResolved diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index 337e5204746c2..c40df65d3bafb 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -8,7 +8,7 @@ object TreeNode { protected def nextId() = currentId.getAndIncrement() } -/** Used when traversing the tree for a node at a given depth */ +/** Used by [[TreeNode.getNodeNumbered]] when traversing the tree for a given number */ private class MutableInt(var i: Int) abstract class TreeNode[BaseType <: TreeNode[BaseType]] { @@ -19,14 +19,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** * A globally unique id for this specific instance. Not preserved across copies. - * Unlike [[equals]] [[id]] be used to differentiate distinct but structurally + * Unlike `equals`, `id` can be used to differentiate distinct but structurally * identical branches of a tree. */ val id = TreeNode.nextId() /** - * Returns true if other is the same [[TreeNode]] instance. Unlike [[equals]] this function will - * return false for different instances of structurally identical trees. + * Returns true if other is the same [[catalyst.trees.TreeNode TreeNode]] instance. Unlike + * `equals` this function will return false for different instances of structurally identical + * trees. */ def sameInstance(other: TreeNode[_]): Boolean = { this.id == other.id @@ -35,14 +36,14 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** * Faster version of equality which short-circuits when two treeNodes are the same instance. * We don't just override Object.Equals, as doing so prevents the scala compiler from from - * generating case class [[equals]] methods. + * generating case class `equals` methods */ def fastEquals(other: TreeNode[_]): Boolean = { sameInstance(other) || this == other } /** - * Runs [[f]] on this node and then recursively on [[children]]. + * Runs the given function on this node and then recursively on [[children]]. * @param f the function to be applied to each node in the tree. */ def foreach(f: BaseType => Unit): Unit = { @@ -51,7 +52,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } /** - * Returns a Seq containing the result of applying [[f]] to each + * Returns a Seq containing the result of applying the given function to each * node in this tree in a preorder traversal. * @param f the function to be applied. */ @@ -83,8 +84,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } /** - * Returns a copy of this node where [[rule]] has been recursively applied to the tree. - * When [[rule]] does not apply to a given node it is left unchanged. + * Returns a copy of this node where `rule` has been recursively applied to the tree. + * When `rule` does not apply to a given node it is left unchanged. * Users should not expect a specific directionality. If a specific directionality is needed, * transformDown or transformUp should be used. * @param rule the function use to transform this nodes children @@ -94,10 +95,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } /** - * Returns a copy of this node where [[rule]] has been recursively - * applied to it and all of its children (pre-order). When [[rule]] does not - * apply to a given node it is left unchanged. - * @param rule the function use to transform this nodes children + * Returns a copy of this node where `rule` has been recursively applied to it and all of its + * children (pre-order). When `rule` does not apply to a given node it is left unchanged. + * @param rule the function used to transform this nodes children */ def transformDown(rule: PartialFunction[BaseType, BaseType]): BaseType = { val afterRule = rule.applyOrElse(this, identity[BaseType]) @@ -110,10 +110,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } /** - * Returns a copy of this node where [[rule]] has been recursively - * applied to all the children of this node. When [[rule]] does not - * apply to a given node it is left unchanged. - * @param rule the function use to transform this nodes children + * Returns a copy of this node where `rule` has been recursively applied to all the children of + * this node. When `rule` does not apply to a given node it is left unchanged. + * @param rule the function used to transform this nodes children */ def transformChildrenDown(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false @@ -144,9 +143,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } /** - * Returns a copy of this node where [[rule]] has been recursively - * applied first to all of its children and then itself (post-order). - * When [[rule]] does not apply to a given node, it is left unchanged. + * Returns a copy of this node where `rule` has been recursively applied first to all of its + * children and then itself (post-order). When `rule` does not apply to a given node, it is left + * unchanged. * @param rule the function use to transform this nodes children */ def transformUp(rule: PartialFunction[BaseType, BaseType]): BaseType = { @@ -196,7 +195,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** * Creates a copy of this type of tree node after a transformation. * Must be overridden by child classes that have constructor arguments - * that are not present in the [[productIterator]]. + * that are not present in the productIterator. * @param newArgs the new product arguments. */ def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") { @@ -218,7 +217,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def nodeName = getClass.getSimpleName /** - * The arguments that should be included in the arg string. Defaults to the [[productIterator]]. + * The arguments that should be included in the arg string. Defaults to the `productIterator`. */ protected def stringArgs = productIterator @@ -240,25 +239,29 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** * Returns a string representation of the nodes in this tree, where each operator is numbered. - * The numbers can be used with [[apply]] to easily access specific subtrees. + * The numbers can be used with [[trees.TreeNode.apply apply]] to easily access specific subtrees. */ def numberedTreeString = treeString.split("\n").zipWithIndex.map { case (line, i) => f"$i%02d $line" }.mkString("\n") - def apply(depth: Int): BaseType = getNodeAtDepth(new MutableInt(depth)) + /** + * Returns the tree node at the specified number. + * Numbers for each node can be found in the [[numberedTreeString]]. + */ + def apply(number: Int): BaseType = getNodeNumbered(new MutableInt(number)) - protected def getNodeAtDepth(depth: MutableInt): BaseType = { - if (depth.i < 0) { + protected def getNodeNumbered(number: MutableInt): BaseType = { + if (number.i < 0) { null.asInstanceOf[BaseType] - } else if (depth.i == 0) { + } else if (number.i == 0) { this } else { - depth.i -= 1 - children.map(_.getNodeAtDepth(depth)).find(_ != null).getOrElse(sys.error("Invalid depth")) + number.i -= 1 + children.map(_.getNodeNumbered(number)).find(_ != null).getOrElse(sys.error("No such node.")) } } - /** Appends the string represent of this node and its children to [[builder]]. */ + /** Appends the string represent of this node and its children to the given StringBuilder. */ protected def generateTreeString(depth: Int, builder: StringBuilder): StringBuilder = { builder.append(" " * depth) builder.append(simpleString) From 9de6b74c1eb4261650d2363b2a5a4a7cbf291449 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 6 Jan 2014 15:14:34 -0800 Subject: [PATCH 412/778] fix language feature and deprecation warnings. --- build.sbt | 2 ++ src/main/scala/catalyst/dsl.scala | 2 ++ .../scala/catalyst/examples/SchemaRddExample.scala | 5 +++-- .../scala/catalyst/execution/MetastoreCatalog.scala | 4 ++-- .../scala/catalyst/execution/SharkInstance.scala | 1 + src/main/scala/catalyst/execution/TableReader.scala | 12 ++++++------ src/main/scala/catalyst/execution/TestShark.scala | 1 + .../scala/catalyst/execution/hiveOperators.scala | 5 +++-- 8 files changed, 20 insertions(+), 12 deletions(-) diff --git a/build.sbt b/build.sbt index dca4fd53f5334..afae82f22288c 100644 --- a/build.sbt +++ b/build.sbt @@ -8,6 +8,8 @@ version := "0.1-SNAPSHOT" scalaVersion := "2.10.3" +scalacOptions ++= Seq("-deprecation", "-feature", "-unchecked") + resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating-SNAPSHOT" diff --git a/src/main/scala/catalyst/dsl.scala b/src/main/scala/catalyst/dsl.scala index 8517ff0dc8a4f..fb73630876ace 100644 --- a/src/main/scala/catalyst/dsl.scala +++ b/src/main/scala/catalyst/dsl.scala @@ -1,5 +1,7 @@ package catalyst +import scala.language.implicitConversions + import analysis.UnresolvedAttribute import expressions._ import plans._ diff --git a/src/main/scala/catalyst/examples/SchemaRddExample.scala b/src/main/scala/catalyst/examples/SchemaRddExample.scala index f89aae50925bb..32b61d20130c9 100644 --- a/src/main/scala/catalyst/examples/SchemaRddExample.scala +++ b/src/main/scala/catalyst/examples/SchemaRddExample.scala @@ -15,13 +15,14 @@ object SchemaRddExample { ("12/2/2013", "WARN: blah blah") :: Nil ) + val dateRegEx = "(\\d+)\\/(\\d+)\\/(\\d+)".r /** * Example using the symbol based API. In this example, the attribute names that are passed to * the first constructor are resolved during catalyst's analysis phase. Then at runtime only * the requested attributes are passed to the UDF. Since this analysis occurs at runtime, * the developer must manually annotate their function with the correct argument types. */ - val filtered = testLogs.filter('date)((date: String) => new java.util.Date(date).getDay == 1) + val filtered = testLogs.filter('date) { case dateRegEx(_,day,_) => day.toInt == 1 } filtered.toRdd.collect.foreach(println) @@ -35,7 +36,7 @@ object SchemaRddExample { * being resolved at runtime. Thus, we cannot return typed results. As such all dynamic calls * always return strings. */ - val filtered2 = testLogs.filter(row => new java.util.Date(row.date).getDay == 1) + val filtered2 = testLogs.filter( _.date match { case dateRegEx(_,day,_) => day.toInt == 1 } ) filtered2.toRdd.collect.foreach(println) } } \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 1809a96e9dedf..590bccfe7c8f4 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -5,7 +5,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table, StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.metastore.HiveMetaStoreClient import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.AbstractDeserializer import org.apache.hadoop.mapred.InputFormat import analysis.Catalog @@ -100,7 +100,7 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt } val tableDesc = new TableDesc( - Class.forName(table.getSd.getSerdeInfo.getSerializationLib).asInstanceOf[Class[Deserializer]], + Class.forName(table.getSd.getSerdeInfo.getSerializationLib).asInstanceOf[Class[AbstractDeserializer]], Class.forName(table.getSd.getInputFormat).asInstanceOf[Class[InputFormat[_,_]]], Class.forName(table.getSd.getOutputFormat), hiveQlTable.getMetadata diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 1a59488aaf171..dd1ec84d1618f 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -2,6 +2,7 @@ package catalyst package execution import java.io.File +import scala.language.implicitConversions import analysis.{SimpleAnalyzer, Analyzer} import frontend.hive._ diff --git a/src/main/scala/catalyst/execution/TableReader.scala b/src/main/scala/catalyst/execution/TableReader.scala index 47ca07c884e27..80d7f9014271a 100644 --- a/src/main/scala/catalyst/execution/TableReader.scala +++ b/src/main/scala/catalyst/execution/TableReader.scala @@ -5,7 +5,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.AbstractDeserializer import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.io.Writable import org.apache.hadoop.fs.{Path, PathFilter} @@ -54,7 +54,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = makeRDDForTable( hiveTable, - _tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], + _tableDesc.getDeserializerClass.asInstanceOf[Class[AbstractDeserializer]], filterOpt = None) /** @@ -68,7 +68,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf */ def makeRDDForTable( hiveTable: HiveTable, - deserializerClass: Class[_ <: Deserializer], + deserializerClass: Class[_ <: AbstractDeserializer], filterOpt: Option[PathFilter]): RDD[_] = { assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, @@ -89,7 +89,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value - val deserializer = deserializerClass.newInstance().asInstanceOf[Deserializer] + val deserializer = deserializerClass.newInstance().asInstanceOf[AbstractDeserializer] deserializer.initialize(hconf, tableDesc.getProperties) // Deserialize each Writable to get the row value. @@ -105,7 +105,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { val partitionToDeserializer = partitions.map(part => - (part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap + (part, part.getDeserializer.getClass.asInstanceOf[Class[AbstractDeserializer]])).toMap makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None) } @@ -120,7 +120,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf * subdirectory of each partition being read. If None, then all files are accepted. */ def makeRDDForPartitionedTable( - partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], + partitionToDeserializer: Map[HivePartition, Class[_ <: AbstractDeserializer]], filterOpt: Option[PathFilter]): RDD[_] = { val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index d44b446bb5d56..a27be94c107c4 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -6,6 +6,7 @@ import java.util.{Set => JavaSet} import scala.collection.mutable import scala.collection.JavaConversions._ +import scala.language.implicitConversions import org.apache.hadoop.hive.metastore.api.{SerDeInfo, StorageDescriptor} import org.apache.hadoop.hive.metastore.MetaStoreUtils diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index a4a806e7aa032..0b822c0a0dc9f 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -4,7 +4,7 @@ package execution import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils import org.apache.hadoop.hive.ql.plan.FileSinkDesc -import org.apache.hadoop.hive.serde2.Serializer +import org.apache.hadoop.hive.serde2.AbstractSerializer import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, StructObjectInspector} import org.apache.hadoop.hive.serde2.`lazy`.LazyStruct import org.apache.hadoop.mapred.JobConf @@ -89,7 +89,8 @@ case class InsertIntoHiveTable( val desc = new FileSinkDesc("./", table.tableDesc, false) val outputClass = { - val serializer = table.tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] + val serializer = + table.tableDesc.getDeserializerClass.newInstance().asInstanceOf[AbstractSerializer] serializer.initialize(null, table.tableDesc.getProperties) serializer.getSerializedClass } From 83227e49da76d6075ddde4f66adc557c1144cc71 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 6 Jan 2014 16:00:28 -0800 Subject: [PATCH 413/778] fix scaladoc list syntax, add docs for some rules --- .../scala/catalyst/analysis/typeCoercion.scala | 8 ++++---- .../scala/catalyst/expressions/Expression.scala | 15 ++++++++------- src/main/scala/catalyst/optimizer/Optimizer.scala | 13 +++++++++++-- 3 files changed, 23 insertions(+), 13 deletions(-) diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index 968d9135632dd..66296102616e4 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -43,11 +43,11 @@ object ConvertNaNs extends Rule[LogicalPlan] { * Loosely based on rules from "Hadoop: The Definitive Guide" 2nd edition, by Tom White * * The implicit conversion rules can be summarized as follows: - * $ - Any integral numeric type can be implicitly converted to a wider type. - * $ - All the integral numeric types, FLOAT, and (perhaps surprisingly) STRING can be implicitly + * - Any integral numeric type can be implicitly converted to a wider type. + * - All the integral numeric types, FLOAT, and (perhaps surprisingly) STRING can be implicitly * converted to DOUBLE. - * $ - TINYINT, SMALLINT, and INT can all be converted to FLOAT. - * $ - BOOLEAN types cannot be converted to any other type. + * - TINYINT, SMALLINT, and INT can all be converted to FLOAT. + * - BOOLEAN types cannot be converted to any other type. * * String conversions are handled by the PromoteStrings rule. */ diff --git a/src/main/scala/catalyst/expressions/Expression.scala b/src/main/scala/catalyst/expressions/Expression.scala index 1bca22cdac824..6a01833b6ca0c 100644 --- a/src/main/scala/catalyst/expressions/Expression.scala +++ b/src/main/scala/catalyst/expressions/Expression.scala @@ -9,13 +9,14 @@ abstract class Expression extends TreeNode[Expression] { def dataType: DataType /** - * foldable is used to indicate if an expression can be folded. - * Right now, we consider expressions listed below as foldable expressions. - * - A Coalesce is foldable if all of its children are foldable - * - A BinaryExpression is foldable if its both left and right child are foldable. - * - A Not, isNull, or isNotNull is foldable if its child is foldable. - * - A Literal is foldable. - * - A Cast or UnaryMinus is foldable if its child is foldable. + * Returns true when an expression is a candidate for static evaluation before the query is + * executed. + * The following conditions are used to determine suitability for constant folding: + * - A Coalesce is foldable if all of its children are foldable + * - A BinaryExpression is foldable if its both left and right child are foldable. + * - A Not, isNull, or isNotNull is foldable if its child is foldable. + * - A Literal is foldable. + * - A Cast or UnaryMinus is foldable if its child is foldable. */ // TODO: Supporting more foldable expressions. For example, deterministic Hive UDFs. def foldable: Boolean = false diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala index d53c75fb9f364..8126f1e949660 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/src/main/scala/catalyst/optimizer/Optimizer.scala @@ -12,7 +12,7 @@ object Optimize extends RuleExecutor[LogicalPlan] { EliminateSubqueries) :: Batch("ConstantFolding", Once, ConstantFolding, - BooleanSimpliï¬cation + BooleanSimplification ) :: Nil } @@ -26,6 +26,10 @@ object EliminateSubqueries extends Rule[LogicalPlan] { } } +/** + * Replaces expressions that can be statically evaluated with equivalent [[expressions.Literal]] + * values. + */ object ConstantFolding extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsDown { @@ -34,7 +38,12 @@ object ConstantFolding extends Rule[LogicalPlan] { } } -object BooleanSimpliï¬cation extends Rule[LogicalPlan] { +/** + * Simplifies boolean expressions where the answer can be determined without evaluating both sides. + * Note that this rule can eliminate expressions that might otherwise have been evaluated and thus + * is only safe when evaluations of expressions does not result in side effects. + */ +object BooleanSimplification extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { case and @ And(left, right) => { From 7eff191a63e9648e977b1545cd4ba9ff92529201 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 7 Jan 2014 18:29:49 -0800 Subject: [PATCH 414/778] link all the expression names. --- .../scala/catalyst/expressions/Expression.scala | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/src/main/scala/catalyst/expressions/Expression.scala b/src/main/scala/catalyst/expressions/Expression.scala index 6a01833b6ca0c..97f9f0320dbac 100644 --- a/src/main/scala/catalyst/expressions/Expression.scala +++ b/src/main/scala/catalyst/expressions/Expression.scala @@ -12,11 +12,14 @@ abstract class Expression extends TreeNode[Expression] { * Returns true when an expression is a candidate for static evaluation before the query is * executed. * The following conditions are used to determine suitability for constant folding: - * - A Coalesce is foldable if all of its children are foldable - * - A BinaryExpression is foldable if its both left and right child are foldable. - * - A Not, isNull, or isNotNull is foldable if its child is foldable. - * - A Literal is foldable. - * - A Cast or UnaryMinus is foldable if its child is foldable. + * - A [[expressions.Coalesce Coalesce]] is foldable if all of its children are foldable + * - A [[expressions.BinaryExpression BinaryExpression]] is foldable if its both left and right + * child are foldable + * - A [[expressions.Not Not]], [[expressions.IsNull IsNull]], or [[expressions.IsNotNull IsNotNull]] + * is foldable if its child is foldable. + * - A [[expressions.Literal]] is foldable. + * - A [[expressions.Cast Cast]] or [[expressions.UnaryMinus UnaryMinus]] is foldable if its + * child is foldable. */ // TODO: Supporting more foldable expressions. For example, deterministic Hive UDFs. def foldable: Boolean = false From 73b29cd22a8d8990460a3ee91d5616325c01b1f7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 8 Jan 2014 11:46:26 -0800 Subject: [PATCH 415/778] fix bad casting --- src/main/scala/catalyst/execution/TableReader.scala | 12 ++++++------ .../scala/catalyst/execution/hiveOperators.scala | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/main/scala/catalyst/execution/TableReader.scala b/src/main/scala/catalyst/execution/TableReader.scala index 80d7f9014271a..0f48788812301 100644 --- a/src/main/scala/catalyst/execution/TableReader.scala +++ b/src/main/scala/catalyst/execution/TableReader.scala @@ -5,7 +5,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.serde2.AbstractDeserializer +import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.io.Writable import org.apache.hadoop.fs.{Path, PathFilter} @@ -54,7 +54,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = makeRDDForTable( hiveTable, - _tableDesc.getDeserializerClass.asInstanceOf[Class[AbstractDeserializer]], + _tableDesc.getDeserializerClass.asInstanceOf[Class[AbstractSerDe]], filterOpt = None) /** @@ -68,7 +68,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf */ def makeRDDForTable( hiveTable: HiveTable, - deserializerClass: Class[_ <: AbstractDeserializer], + deserializerClass: Class[_ <: AbstractSerDe], filterOpt: Option[PathFilter]): RDD[_] = { assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, @@ -89,7 +89,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value - val deserializer = deserializerClass.newInstance().asInstanceOf[AbstractDeserializer] + val deserializer = deserializerClass.newInstance() deserializer.initialize(hconf, tableDesc.getProperties) // Deserialize each Writable to get the row value. @@ -105,7 +105,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { val partitionToDeserializer = partitions.map(part => - (part, part.getDeserializer.getClass.asInstanceOf[Class[AbstractDeserializer]])).toMap + (part, part.getDeserializer.getClass.asInstanceOf[Class[AbstractSerDe]])).toMap makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None) } @@ -120,7 +120,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf * subdirectory of each partition being read. If None, then all files are accepted. */ def makeRDDForPartitionedTable( - partitionToDeserializer: Map[HivePartition, Class[_ <: AbstractDeserializer]], + partitionToDeserializer: Map[HivePartition, Class[_ <: AbstractSerDe]], filterOpt: Option[PathFilter]): RDD[_] = { val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 0b822c0a0dc9f..42c8ef1775a83 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -4,7 +4,7 @@ package execution import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils import org.apache.hadoop.hive.ql.plan.FileSinkDesc -import org.apache.hadoop.hive.serde2.AbstractSerializer +import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, StructObjectInspector} import org.apache.hadoop.hive.serde2.`lazy`.LazyStruct import org.apache.hadoop.mapred.JobConf @@ -90,7 +90,7 @@ case class InsertIntoHiveTable( val outputClass = { val serializer = - table.tableDesc.getDeserializerClass.newInstance().asInstanceOf[AbstractSerializer] + table.tableDesc.getDeserializerClass.newInstance().asInstanceOf[AbstractSerDe] serializer.initialize(null, table.tableDesc.getProperties) serializer.getSerializedClass } From 0d2388b1f31ab8788120f8a8679525479a794d25 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 8 Jan 2014 11:57:24 -0800 Subject: [PATCH 416/778] Point at databricks hosted scaladoc. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index fb7b0d5aeef6e..b7987f0140f23 100644 --- a/README.md +++ b/README.md @@ -2,7 +2,7 @@ catalyst ======== Catalyst is a functional framework for optimizing relational query plans. -More documentation can be found in the project's [scaladoc](http://marmbrus.github.io/catalyst/latest/api/#catalyst.package) +More documentation can be found in the project's [scaladoc](http://databricks.github.io/catalyst/latest/api/#catalyst.package) Getting Started =============== From 5362365b1e4cad5db75cb432c719a08a3e04e073 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 8 Jan 2014 13:47:30 -0800 Subject: [PATCH 417/778] push conditions into join --- src/main/scala/optimizer/Optimizer.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/scala/optimizer/Optimizer.scala b/src/main/scala/optimizer/Optimizer.scala index d4a693765af41..c87ffed4e1474 100644 --- a/src/main/scala/optimizer/Optimizer.scala +++ b/src/main/scala/optimizer/Optimizer.scala @@ -76,7 +76,6 @@ object PushPredicateThroughInnerJoin extends Rule[LogicalPlan] with PredicateHel // Build the new left and right side, optionally with the pushed down filters. val newLeft = leftConditions.reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) val newRight = rightConditions.reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) - val newJoin = Join(newLeft, newRight, Inner, None) - joinConditions.reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin) + Join(newLeft, newRight, Inner, joinConditions.reduceLeftOption(And)) } } \ No newline at end of file From f54c94c36af64ee9c01190dbcf4d5a049c27db4a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Jan 2014 13:17:09 -0800 Subject: [PATCH 418/778] make golden answers file a test dependency --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index afae82f22288c..401fc04fd3cdd 100644 --- a/build.sbt +++ b/build.sbt @@ -14,7 +14,7 @@ resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/ libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating-SNAPSHOT" -libraryDependencies += "catalyst" % "hive-golden" % "2" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden2.jar" +libraryDependencies += "catalyst" % "hive-golden" % "2" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden2.jar" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" From eafaeed4d154bd3fec44fb94adf7ccd3eeca7b11 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Jan 2014 13:17:31 -0800 Subject: [PATCH 419/778] add type documentation --- src/main/scala/catalyst/execution/SharkInstance.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index dd1ec84d1618f..130578e87fbe2 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -36,7 +36,7 @@ abstract class SharkInstance extends Logging { def metastorePath: String /** The SharkContext */ - lazy val sc = createContext() + lazy val sc: SharkContext = createContext() protected def createContext(): SharkContext = { SharkEnv.initWithSharkContext("catalyst.execution", master) From 6f64cee0bd69dac4fbd63c987f47bde1931ecb32 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Jan 2014 13:17:46 -0800 Subject: [PATCH 420/778] don't line wrap string literal --- src/main/scala/catalyst/execution/SharkInstance.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 130578e87fbe2..c7d4eba52751f 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -45,8 +45,8 @@ abstract class SharkInstance extends Logging { /** Sets up the system initially or after a RESET command */ protected def configure() { // TODO: refactor this so we can work with other databases. - runSqlHive("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + metastorePath + - ";create=true") + runSqlHive( + s"set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$metastorePath;create=true") runSqlHive("set hive.metastore.warehouse.dir=" + warehousePath) } From 9b02b44a38ebf55ff66749c4fc262be7c8c5c655 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Jan 2014 14:33:49 -0800 Subject: [PATCH 421/778] Fix spelling error. Add failFast mode. --- Makefile | 55 ++++++++++--------- .../catalyst/execution/BigDataBenchmark.scala | 2 +- ...ionTest.scala => HiveComparisonTest.scala} | 34 +++++++++--- ...tability.scala => HiveCompatibility.scala} | 2 +- .../execution/HiveQueryFileTest.scala | 2 +- .../catalyst/execution/HiveQueryTests.scala | 2 +- .../execution/HiveResolutionSuite.scala | 2 +- .../execution/HiveTypeCoersionSuite.scala | 2 +- 8 files changed, 61 insertions(+), 40 deletions(-) rename src/test/scala/catalyst/execution/{HiveComparisionTest.scala => HiveComparisonTest.scala} (84%) rename src/test/scala/catalyst/execution/{HiveCompatability.scala => HiveCompatibility.scala} (99%) diff --git a/Makefile b/Makefile index e763516432300..bbcf31f353131 100644 --- a/Makefile +++ b/Makefile @@ -1,54 +1,57 @@ all: a b c d e f g h i j k l m n o p q r s t u v w x y" z +findBroken: + sbt -Dshark.hive.alltests -Dshark.hive.failFast "test-only catalyst.execution.HiveCompatibility" + a: - sbt -Dshark.hive.whitelist=a.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=a.* "test-only catalyst.execution.HiveCompatibility" b: - sbt -Dshark.hive.whitelist=b.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=b.* "test-only catalyst.execution.HiveCompatibility" c: - sbt -Dshark.hive.whitelist=c.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=c.* "test-only catalyst.execution.HiveCompatibility" d: - sbt -Dshark.hive.whitelist=d.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=d.* "test-only catalyst.execution.HiveCompatibility" e: - sbt -Dshark.hive.whitelist=e.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=e.* "test-only catalyst.execution.HiveCompatibility" f: - sbt -Dshark.hive.whitelist=f.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=f.* "test-only catalyst.execution.HiveCompatibility" g: - sbt -Dshark.hive.whitelist=g.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=g.* "test-only catalyst.execution.HiveCompatibility" h: - sbt -Dshark.hive.whitelist=h.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=h.* "test-only catalyst.execution.HiveCompatibility" i: - sbt -Dshark.hive.whitelist=i.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=i.* "test-only catalyst.execution.HiveCompatibility" j: - sbt -Dshark.hive.whitelist=j.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=j.* "test-only catalyst.execution.HiveCompatibility" k: - sbt -Dshark.hive.whitelist=k.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=k.* "test-only catalyst.execution.HiveCompatibility" l: - sbt -Dshark.hive.whitelist=l.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=l.* "test-only catalyst.execution.HiveCompatibility" m: - sbt -Dshark.hive.whitelist=m.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=m.* "test-only catalyst.execution.HiveCompatibility" n: - sbt -Dshark.hive.whitelist=n.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=n.* "test-only catalyst.execution.HiveCompatibility" o: - sbt -Dshark.hive.whitelist=o.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=o.* "test-only catalyst.execution.HiveCompatibility" p: - sbt -Dshark.hive.whitelist=p.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=p.* "test-only catalyst.execution.HiveCompatibility" q: - sbt -Dshark.hive.whitelist=q.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=q.* "test-only catalyst.execution.HiveCompatibility" r: - sbt -Dshark.hive.whitelist=r.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=r.* "test-only catalyst.execution.HiveCompatibility" s: - sbt -Dshark.hive.whitelist=s.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=s.* "test-only catalyst.execution.HiveCompatibility" t: - sbt -Dshark.hive.whitelist=t.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=t.* "test-only catalyst.execution.HiveCompatibility" u: - sbt -Dshark.hive.whitelist=u.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=u.* "test-only catalyst.execution.HiveCompatibility" v: - sbt -Dshark.hive.whitelist=v.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=v.* "test-only catalyst.execution.HiveCompatibility" w: - sbt -Dshark.hive.whitelist=w.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=w.* "test-only catalyst.execution.HiveCompatibility" x: - sbt -Dshark.hive.whitelist=x.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=x.* "test-only catalyst.execution.HiveCompatibility" y: - sbt -Dshark.hive.whitelist=y.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=y.* "test-only catalyst.execution.HiveCompatibility" z: - sbt -Dshark.hive.whitelist=z.* "test-only catalyst.execution.HiveCompatability" + sbt -Dshark.hive.whitelist=z.* "test-only catalyst.execution.HiveCompatibility" diff --git a/src/test/scala/catalyst/execution/BigDataBenchmark.scala b/src/test/scala/catalyst/execution/BigDataBenchmark.scala index 52146d16f0cfe..1ff9cb6491746 100644 --- a/src/test/scala/catalyst/execution/BigDataBenchmark.scala +++ b/src/test/scala/catalyst/execution/BigDataBenchmark.scala @@ -7,7 +7,7 @@ import java.io.File * A set of test cases based on the big-data-benchmark. * https://amplab.cs.berkeley.edu/benchmark/ */ -class BigDataBenchmarkTests extends HiveComaparisionTest { +class BigDataBenchmarkTests extends HiveComparisonTest { import TestShark._ val testDataDirectory = new File("target/big-data-benchmark-testdata") diff --git a/src/test/scala/catalyst/execution/HiveComparisionTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala similarity index 84% rename from src/test/scala/catalyst/execution/HiveComparisionTest.scala rename to src/test/scala/catalyst/execution/HiveComparisonTest.scala index bf73e8a153817..996c643125a26 100644 --- a/src/test/scala/catalyst/execution/HiveComparisionTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -11,22 +11,35 @@ import util._ * Allows the creations of tests that execute the same query against both hive * and catalyst, comparing the results. * - * The "golden" results from Hive are cached in [[answerCache]] to speed up testing. + * The "golden" results from Hive are cached in an retrieved both from the classpath and + * [[answerCache]] to speed up testing. */ -// TODO: correct the mispelled name. -abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with GivenWhenThen with Logging { +abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with GivenWhenThen with Logging { protected val targetDir = new File("target") + + /** The local directory with cached golden answer will be stored */ protected val answerCache = new File(targetDir, "comparison-test-cache") if (!answerCache.exists) answerCache.mkdir() + /** The [[ClassLoader]] that contains test dependencies. Used to look for golden answers. */ + protected val testClassLoader = this.getClass.getClassLoader + + /** A file where all the test cases that pass are written. Can be used to update the whiteList. */ val passedFile = new File(targetDir, s"$suiteName.passed") - val passedList = new PrintWriter(passedFile) + protected val passedList = new PrintWriter(passedFile) override def afterAll() { passedList.close() } + /** + * When `-Dshark.hive.failFast` is set the first test to fail will cause all subsequent tests to + * also fail. + */ + val failFast = System.getProperty("shark.hive.failFast") != null + private var testFailed = false + protected val cacheDigest = java.security.MessageDigest.getInstance("MD5") protected def getMd5(str: String): String = { val digest = java.security.MessageDigest.getInstance("MD5") @@ -36,7 +49,8 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with protected def prepareAnswer(sharkQuery: TestShark.type#SharkSqlQuery, answer: Seq[String]): Seq[String] = { val orderedAnswer = sharkQuery.parsed match { - case _: Command => answer.filterNot(nonDeterministicLine) // Clean out nondeterministic time schema info. + // Clean out non-deterministic time schema info. + case _: Command => answer.filterNot(nonDeterministicLine) case _ => val isOrdered = sharkQuery.executedPlan.collect { case s: Sort => s}.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. @@ -52,7 +66,7 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with line.replaceAll("\"lastUpdateTime\":\\d+", "") /** - * Removes non-deterministic paths from [[str]] so cached answers will still pass. + * Removes non-deterministic paths from str` so cached answers will still pass. */ protected def cleanPaths(str: String): String = { str.replaceAll("file:\\/.*\\/", "") @@ -61,6 +75,8 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with val installHooksCommand = "(?i)SET.*hooks".r def createQueryTest(testCaseName: String, sql: String) = { test(testCaseName) { + if(failFast && testFailed) sys.error("Failing fast due to previous failure") + testFailed = true logger.error( s""" |============================= @@ -82,11 +98,11 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with } val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => - logger.debug(s"Looking for cached answer file $cachedAnswerFile.") + logger.warn(s"Looking for cached answer file $cachedAnswerFile.") if (cachedAnswerFile.exists) { Some(fileToString(cachedAnswerFile)) } else if (getClass.getClassLoader.getResourceAsStream(cachedAnswerFile.toString) != null) { - Some(resourceToString(cachedAnswerFile.toString)) + Some(resourceToString(cachedAnswerFile.toString, classLoader = testClassLoader)) } else { logger.debug(s"File $cachedAnswerFile not found") None @@ -123,6 +139,8 @@ abstract class HiveComaparisionTest extends FunSuite with BeforeAndAfterAll with computedResults } + testFailed = false + // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => info(queryString) diff --git a/src/test/scala/catalyst/execution/HiveCompatability.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala similarity index 99% rename from src/test/scala/catalyst/execution/HiveCompatability.scala rename to src/test/scala/catalyst/execution/HiveCompatibility.scala index 53eecffce4a52..32daa17156b92 100644 --- a/src/test/scala/catalyst/execution/HiveCompatability.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -8,7 +8,7 @@ import util._ /** * Runs the test cases that are included in the hive distribution. */ -class HiveCompatability extends HiveQueryFileTest { +class HiveCompatibility extends HiveQueryFileTest { // TODO: bundle in jar files... get from classpath lazy val hiveQueryDir = new File(TestShark.hiveDevHome, "ql/src/test/queries/clientpositive") def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) diff --git a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala index afda9f7b5270b..4edcacc1131c3 100644 --- a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala +++ b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala @@ -11,7 +11,7 @@ import util._ * TestSuites that derive from this class must provide a map of testCaseName -> testCaseFiles that should be included. * Additionally, there is support for whitelisting and blacklisting tests as development progresses. */ -abstract class HiveQueryFileTest extends HiveComaparisionTest { +abstract class HiveQueryFileTest extends HiveComparisonTest { /** A list of tests deemed out of scope and thus completely disregarded */ def blackList: Seq[String] = Nil diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala index 5a39c35640684..a694a1d6194c6 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -4,7 +4,7 @@ package execution /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ -class HiveQueryTests extends HiveComaparisionTest { +class HiveQueryTests extends HiveComparisonTest { import TestShark._ createQueryTest("Simple Average", diff --git a/src/test/scala/catalyst/execution/HiveResolutionSuite.scala b/src/test/scala/catalyst/execution/HiveResolutionSuite.scala index 13dfb951cb55a..4ab6c0dc80a1f 100644 --- a/src/test/scala/catalyst/execution/HiveResolutionSuite.scala +++ b/src/test/scala/catalyst/execution/HiveResolutionSuite.scala @@ -4,7 +4,7 @@ package execution /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ -class HiveResolutionSuite extends HiveComaparisionTest { +class HiveResolutionSuite extends HiveComparisonTest { import TestShark._ createQueryTest("table.attr", diff --git a/src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala b/src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala index 095dce23aade9..f94b9951bacab 100644 --- a/src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala +++ b/src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala @@ -4,7 +4,7 @@ package execution /** * A set of tests that validate type promotion rules. */ -class HiveTypeCoersionSuite extends HiveComaparisionTest { +class HiveTypeCoersionSuite extends HiveComparisonTest { import TestShark._ val baseTypes = Seq("1", "1.0", "1L", "1S", "1Y", "'1'") From 962761679ba3d5b7533cc8ca6db8c33aba15fb66 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Jan 2014 18:42:40 -0800 Subject: [PATCH 422/778] Use current database as default database. --- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 590bccfe7c8f4..d83242583f541 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -5,6 +5,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table, StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.metastore.HiveMetaStoreClient import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.AbstractDeserializer import org.apache.hadoop.mapred.InputFormat @@ -21,7 +22,7 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { def lookupRelation(name: String, alias: Option[String]): BaseRelation = { val (databaseName, tableName) = name.split("\\.") match { - case Array(tableOnly) => ("default", tableOnly) + case Array(tableOnly) => (SessionState.get.getCurrentDatabase(), tableOnly) case Array(db, table) => (db, table) } val table = client.getTable(databaseName, tableName) @@ -46,7 +47,7 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case InsertIntoCreatedTable(name, child) => val (databaseName, tableName) = name.split("\\.") match { - case Array(tableOnly) => ("default", tableOnly) + case Array(tableOnly) => (SessionState.get.getCurrentDatabase(), tableOnly) case Array(db, table) => (db, table) } From 1aafea35c32ebb242d12e9ff7b24f04100050ecb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Jan 2014 18:42:59 -0800 Subject: [PATCH 423/778] Configure partition whitelist in TestShark reset. --- src/main/scala/catalyst/execution/TestShark.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index a27be94c107c4..e5cba37be9b1b 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -234,6 +234,8 @@ object TestShark extends SharkInstance { // For some reason, RESET does not reset the following variables... runSqlHive("set datanucleus.cache.collections=true") runSqlHive("set datanucleus.cache.collections.lazy=true") + // Lots of tests fail if we do not change the partition whitelist from the default. + runSqlHive("set hive.metastore.partition.name.whitelist.pattern=[\\-A-Za-z0-9:_]*") loadedTables.clear() catalog.client.getAllTables("default").foreach { t => From ca4ea2636e5666fd0442790b623095247d7e5254 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Jan 2014 18:43:29 -0800 Subject: [PATCH 424/778] Support for parsing UDF(*). --- src/main/scala/catalyst/frontend/Hive.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 7161690c29cdd..fb6c1d47b78ec 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -655,6 +655,8 @@ object HiveQl { /* UDFs - Must be last otherwise will preempt built in functions */ case Token("TOK_FUNCTION", Token(name, Nil) :: args) => UnresolvedFunction(name, args.map(nodeToExpr)) + case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => + UnresolvedFunction(name, Star(None) :: Nil) /* Literals */ case Token("TOK_NULL", Nil) => Literal(null, IntegerType) // TODO: What type is null? From 68aa2e6f6e85d6e06424676de53f00794e64b468 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Jan 2014 18:43:52 -0800 Subject: [PATCH 425/778] Stronger type for Token extractor. --- src/main/scala/catalyst/frontend/Hive.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index fb6c1d47b78ec..d72b521a3e6b5 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -254,7 +254,7 @@ object HiveQl { /** Extractor for matching Hive's AST Tokens. */ object Token { /** @return matches of the form (tokenName, children). */ - def unapply(t: Any) = t match { + def unapply(t: Any): Option[(String, Seq[ASTNode])] = t match { case t: ASTNode => Some((t.getText, Option(t.getChildren).map(_.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]])) case _ => None From 516481ca197807e91309780c81a719866cef336f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Jan 2014 18:44:21 -0800 Subject: [PATCH 426/778] Ignore requests to explain native commands. --- src/main/scala/catalyst/frontend/Hive.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index d72b521a3e6b5..031bd48739952 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -339,8 +339,8 @@ object HiveQl { } protected def nodeToPlan(node: Node): LogicalPlan = node match { - // Just fake explain on create function... - case Token("TOK_EXPLAIN", Token("TOK_CREATEFUNCTION", _) :: Nil) => NoRelation + // Just fake explain for any of the native commands. + case Token("TOK_EXPLAIN", Token(explainType, _) :: Nil) if nativeCommands contains explainType => NoRelation case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. val Some(query) :: _ :: _ :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) From 4b6fed8a47e00675171faafac0e582df39f3ff58 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Jan 2014 18:44:54 -0800 Subject: [PATCH 427/778] support for parsing both DESTINATION and INSERT_INTO. --- src/main/scala/catalyst/frontend/Hive.scala | 52 ++++++++++++++------- 1 file changed, 34 insertions(+), 18 deletions(-) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 031bd48739952..9b2b9997d2d69 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -362,13 +362,15 @@ object HiveQl { // Return one query for each insert clause. val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => - val (Some(destClause) :: + val ( + intoClause :: + destClause :: Some(selectClause) :: whereClause :: groupByClause :: orderByClause :: sortByClause :: - limitClause :: Nil) = getClauses(Seq("TOK_DESTINATION", "TOK_SELECT", "TOK_WHERE", "TOK_GROUPBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_LIMIT"), singleInsert) + limitClause :: Nil) = getClauses(Seq("TOK_INSERT_INTO", "TOK_DESTINATION", "TOK_SELECT", "TOK_WHERE", "TOK_GROUPBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_LIMIT"), singleInsert) val relations = nodeToRelation(fromClause) val withWhere = whereClause.map { whereNode => @@ -420,8 +422,13 @@ object HiveQl { .map(StopAfter(_, withSort)) .getOrElse(withSort) + // There are two tokens for specifying where to sent the result that seem to be used almost + // interchangeably. + val resultDestination = + (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) + nodeToDest( - destClause, + resultDestination, withLimit) } @@ -441,11 +448,27 @@ object HiveQl { Subquery(alias, nodeToPlan(query)) /* Table, No Alias */ - case Token("TOK_TABREF", - Token("TOK_TABNAME", - tableNameParts) :: Nil) => - val tableName = tableNameParts.map { case Token(part, Nil) => part }.mkString(".") - UnresolvedRelation(tableName, None) + case Token("TOK_TABREF", clauses) => + // If the last clause is not a token then it's the alias of the table. + val (nonAliasClauses, aliasClause) = + if(clauses.last.getText.startsWith("TOK")) + (clauses, None) + else + (clauses.dropRight(1), Some(clauses.last)) + + val (Some(tableNameParts) :: + sampleClause :: Nil) = getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE"), nonAliasClauses) + + val tableName = tableNameParts.getChildren.map { case Token(part, Nil) => part }.mkString(".") + val alias = aliasClause.map { case Token(a, Nil) => a } + val relation = UnresolvedRelation(tableName, alias) + // Apply sampling if requested. + sampleClause.map { + case Token("TOK_TABLESPLITSAMPLE", + Token("TOK_ROWCOUNT", Nil) :: + Token(count, Nil) :: Nil) => + StopAfter(Literal(count.toInt), relation) + }.getOrElse(relation) case Token("TOK_UNIQUEJOIN", joinArgs) => val tableOrdinals = @@ -492,14 +515,6 @@ object HiveQl { // named output expressions where some aggregate expression has been applied (i.e. First). ??? /// Aggregate(groups, Star(None, First(_)) :: Nil, joinedResult) - /* Table with Alias */ - case Token("TOK_TABREF", - Token("TOK_TABNAME", - tableNameParts) :: - Token(alias, Nil) :: Nil) => - val tableName = tableNameParts.map { case Token(part, Nil) => part }.mkString(".") - UnresolvedRelation(tableName, Some(alias)) - case Token(allJoinTokens(joinToken), relation1 :: relation2 :: other) => @@ -529,13 +544,14 @@ object HiveQl { throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } + val destinationToken = "TOK_DESTINATION|TOK_INSERT_INTO".r protected def nodeToDest(node: Node, query: LogicalPlan): LogicalPlan = node match { - case Token("TOK_DESTINATION", + case Token(destinationToken(), Token("TOK_DIR", Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) => query - case Token("TOK_DESTINATION", + case Token(destinationToken(), Token("TOK_TAB", tableArgs) :: Nil) => val Some(nameClause) :: partitionClause :: Nil = From 4c5fb0f4d804e8767ced60e30dcb01db9a86117c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Jan 2014 12:01:40 -0800 Subject: [PATCH 428/778] makefile target for building new whitelist. --- Makefile | 3 +++ 1 file changed, 3 insertions(+) diff --git a/Makefile b/Makefile index bbcf31f353131..8c66f1833f588 100644 --- a/Makefile +++ b/Makefile @@ -1,5 +1,8 @@ all: a b c d e f g h i j k l m n o p q r s t u v w x y" z +buildWhiteList: + sbt -Dshark.hive.alltests "test-only catalyst.execution.HiveCompatibility" + findBroken: sbt -Dshark.hive.alltests -Dshark.hive.failFast "test-only catalyst.execution.HiveCompatibility" From 4c6b454116025439dc6a5c3562aa203d30a1ffe3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Jan 2014 12:02:31 -0800 Subject: [PATCH 429/778] add option for recomputing the cached golden answer when tests fail. --- .../execution/HiveComparisonTest.scala | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index 996c643125a26..a6a3c3b172f28 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -13,11 +13,13 @@ import util._ * * The "golden" results from Hive are cached in an retrieved both from the classpath and * [[answerCache]] to speed up testing. + * + * TODO(marmbrus): Document system properties. */ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with GivenWhenThen with Logging { protected val targetDir = new File("target") - /** The local directory with cached golden answer will be stored */ + /** The local directory with cached golden answer will be stored. */ protected val answerCache = new File(targetDir, "comparison-test-cache") if (!answerCache.exists) answerCache.mkdir() @@ -40,6 +42,12 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G val failFast = System.getProperty("shark.hive.failFast") != null private var testFailed = false + /** + * Delete any cache files that result in test failures. Used when the test harness has been + * updated thus requiring new golden answers to be computed for some tests. + */ + val recomputeCache = System.getProperty("shark.hive.recomputeCache") != null + protected val cacheDigest = java.security.MessageDigest.getInstance("MD5") protected def getMd5(str: String): String = { val digest = java.security.MessageDigest.getInstance("MD5") @@ -66,7 +74,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G line.replaceAll("\"lastUpdateTime\":\\d+", "") /** - * Removes non-deterministic paths from str` so cached answers will still pass. + * Removes non-deterministic paths from `str` so cached answers will still pass. */ protected def cleanPaths(str: String): String = { str.replaceAll("file:\\/.*\\/", "") @@ -98,7 +106,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G } val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => - logger.warn(s"Looking for cached answer file $cachedAnswerFile.") + logger.debug(s"Looking for cached answer file $cachedAnswerFile.") if (cachedAnswerFile.exists) { Some(fileToString(cachedAnswerFile)) } else if (getClass.getClassLoader.getResourceAsStream(cachedAnswerFile.toString) != null) { @@ -173,13 +181,18 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G val hivePrintOut = s"== HIVE - ${hive.size} row(s) ==" +: preparedHive val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst - val resultComparision = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") + val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") + + if(recomputeCache) { + logger.warn(s"Clearing cache files for failed test $testCaseName") + hiveCacheFiles.foreach(_.delete()) + } fail( s""" |Results do not match for query: |$sharkQuery\n${sharkQuery.analyzed.output.map(_.name).mkString("\t")} - |$resultComparision + |$resultComparison """.stripMargin) } } From b01468dfdd3d99c2159534fbff152d9ff46576fb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Jan 2014 12:03:09 -0800 Subject: [PATCH 430/778] support path rewrites when the query begins with a comment. --- src/main/scala/catalyst/execution/TestShark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index e5cba37be9b1b..722baa3d8d594 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -85,7 +85,7 @@ object TestShark extends SharkInstance { * hive test cases assume the system is set up. */ private def rewritePaths(cmd: String): String = - if (cmd.toUpperCase startsWith "LOAD") + if (cmd.toUpperCase contains "LOAD DATA") cmd.replaceAll("\\.\\.", hiveDevHome.getCanonicalPath) else cmd From 8364ec2cc2a34461f15a7e9b19e426d7d9ce79b7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Jan 2014 12:03:28 -0800 Subject: [PATCH 431/778] whitelist all possible partition values. --- src/main/scala/catalyst/execution/TestShark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 722baa3d8d594..506f43d8e312c 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -235,7 +235,7 @@ object TestShark extends SharkInstance { runSqlHive("set datanucleus.cache.collections=true") runSqlHive("set datanucleus.cache.collections.lazy=true") // Lots of tests fail if we do not change the partition whitelist from the default. - runSqlHive("set hive.metastore.partition.name.whitelist.pattern=[\\-A-Za-z0-9:_]*") + runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") loadedTables.clear() catalog.client.getAllTables("default").foreach { t => From 78d730d657d0904b5a3c28b49737594aa40ce9cb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Jan 2014 12:06:01 -0800 Subject: [PATCH 432/778] Load src test table on RESET. --- src/main/scala/catalyst/execution/TestShark.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 506f43d8e312c..736166326656b 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -263,6 +263,12 @@ object TestShark extends SharkInstance { configure() runSqlHive("USE default") + + // Just loading src makes a lot of tests pass. This is because some tests do something like + // drop an index on src at the beginning. Since we just pass DDL to hive this bypasses our + // Analyzer and thus the test table auto-loading mechanism. + // Remove after we handle more DDL operations natively. + loadTestTable("src") } catch { case e: Exception => logger.error(s"FATAL ERROR: Failed to reset TestDB state. $e") From 0d9d56aca705b856191a636e27723c57b74d7d93 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Jan 2014 12:07:20 -0800 Subject: [PATCH 433/778] add more native commands to parser --- src/main/scala/catalyst/frontend/Hive.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 9b2b9997d2d69..c0a4de505aef9 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -82,6 +82,7 @@ object HiveQl { "TOK_ALTERINDEX_REBUILD", "TOK_ALTERTABLE_ADDCOLS", "TOK_ALTERTABLE_ADDPARTS", + "TOK_ALTERTABLE_ALTERPARTS", "TOK_ALTERTABLE_ARCHIVE", "TOK_ALTERTABLE_CLUSTER_SORT", "TOK_ALTERTABLE_DROPPARTS", @@ -90,6 +91,7 @@ object HiveQl { "TOK_ALTERTABLE_RENAME", "TOK_ALTERTABLE_RENAMECOL", "TOK_ALTERTABLE_REPLACECOLS", + "TOK_ALTERTABLE_SKEWED", "TOK_ALTERTABLE_TOUCH", "TOK_ALTERTABLE_UNARCHIVE", "TOK_ANALYZE", From 4cfc11a6c9dfc16cb216f2a18da4e4dd09bd246e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Jan 2014 12:07:36 -0800 Subject: [PATCH 434/778] more test coverage. --- build.sbt | 2 +- .../execution/HiveCompatibility.scala | 117 +++++++++++++++++- 2 files changed, 116 insertions(+), 3 deletions(-) diff --git a/build.sbt b/build.sbt index 401fc04fd3cdd..e660fb5bbf584 100644 --- a/build.sbt +++ b/build.sbt @@ -14,7 +14,7 @@ resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/ libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating-SNAPSHOT" -libraryDependencies += "catalyst" % "hive-golden" % "2" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden2.jar" +libraryDependencies += "catalyst" % "hive-golden" % "3" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden3.jar" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index 32daa17156b92..5c1dcf78ba6fd 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -31,6 +31,7 @@ class HiveCompatibility extends HiveQueryFileTest { "authorization_5", "keyword_1", "misc_json", + "create_like_tbl_props", // Timezone specific test answers. "udf_unix_timestamp", @@ -52,6 +53,11 @@ class HiveCompatibility extends HiveQueryFileTest { // These tests fail and and exit the JVM. "auto_join18_multi_distinct", "join18_multi_distinct", + "input44", + "input42", + "input_dfs", + "metadata_export_drop", + "repair", // Uses a serde that isn't on the classpath... breaks other tests. "bucketizedhiveinputformat", @@ -64,7 +70,23 @@ class HiveCompatibility extends HiveQueryFileTest { "uniquejoin", // Hive seems to get the wrong answer on some outer joins. MySQL agrees with catalyst. - "auto_join29" + "auto_join29", + + // No support for multi-alias i.e. udf as (e1, e2, e3). + "allcolref_in_udf", + + // No support for TestSerDe (not published afaik) + "alter1", + "input16", + + // Shark does not support buckets. + ".*bucket.*", + + // No window support yet + ".* window.*", + + // Fails in hive with authorization errors. + "alter_rename_partition_authorization" ) /** @@ -77,7 +99,10 @@ class HiveCompatibility extends HiveQueryFileTest { "add_partition_with_whitelist", "alias_casted_column", "alter4", + "alter_index", + "alter_partition_format_loc", "alter_partition_with_whitelist", + "alter_table_serde", "ambiguous_col", "authorization_3", "authorization_5", @@ -87,10 +112,18 @@ class HiveCompatibility extends HiveQueryFileTest { "auto_join26", "auto_join28", "auto_join_nulls", + "auto_sortmerge_join_1", "auto_sortmerge_join_10", + "auto_sortmerge_join_11", + "auto_sortmerge_join_12", "auto_sortmerge_join_15", + "auto_sortmerge_join_2", + "auto_sortmerge_join_3", + "auto_sortmerge_join_4", "auto_sortmerge_join_5", "auto_sortmerge_join_6", + "auto_sortmerge_join_7", + "auto_sortmerge_join_8", "auto_sortmerge_join_9", "binarysortable_1", "bucket1", @@ -106,19 +139,34 @@ class HiveCompatibility extends HiveQueryFileTest { "correlationoptimizer6", "correlationoptimizer7", "count", + "create_like2", + "create_like_tbl_props", + "create_view_translate", "ct_case_insensitive", "database_properties", + "default_partition_name", "delimiter", "desc_non_existent_tbl", "describe_database_json", "describe_table_json", + "describe_formatted_view_partitioned", + "describe_formatted_view_partitioned_json", + "describe_pretty", + "describe_syntax", + "diff_part_input_formats", "disable_file_format_check", "drop_function", "drop_index", + "drop_partitions_filter", + "drop_partitions_filter2", + "drop_partitions_filter3", + "drop_partitions_ignore_protection", "drop_table", + "drop_table2", "drop_view", "escape_orderby1", "escape_sortby1", + "filter_join_breaktask", "groupby1", "groupby1_map", "groupby1_map_nomap", @@ -130,34 +178,65 @@ class HiveCompatibility extends HiveQueryFileTest { "groupby5_map", "groupby5_map_skew", "groupby5_noskew", + "groupby7", + "groupby7_map", + "groupby7_map_multi_single_reducer", + "groupby7_map_skew", + "groupby7_noskew", + "groupby8_map", + "groupby8_map_skew", + "groupby8_noskew", "groupby_multi_single_reducer2", "groupby_mutli_insert_common_distinct", + "groupby_sort_6", + "groupby_sort_8", "groupby_sort_test_1", "implicit_cast1", "index_auto_self_join", "index_auto_update", "index_stale", + "index_auth", + "index_auto_file_format", + "index_auto_mult_tables", + "index_auto_mult_tables_compact", + "index_auto_multiple", + "index_bitmap_compression", + "index_compression", "innerjoin", "inoutdriver", + "input", "input0", "input11", "input11_limit", "input1_limit", "input22", + "input23", + "input24", + "input25", + "input28", + "input2_limit", + "input41", "input4_cb_delim", "input4_limit", "input6", "input7", "input8", "input9", + "input_limit", + "input_part1", + "input_part2", "inputddl4", + "inputddl7", "inputddl8", + "insert_compressed", "join0", "join1", "join10", "join11", "join12", "join13", + "join14", + "join14_hadoop20", "join15", "join16", "join17", @@ -170,11 +249,15 @@ class HiveCompatibility extends HiveQueryFileTest { "join23", "join24", "join25", + "join26", "join27", + "join28", "join29", "join3", "join30", "join31", + "join32", + "join33", "join34", "join35", "join36", @@ -188,6 +271,7 @@ class HiveCompatibility extends HiveQueryFileTest { "join6", "join7", "join8", + "join9", "join_casesensitive", "join_empty", "join_hive_626", @@ -203,8 +287,14 @@ class HiveCompatibility extends HiveQueryFileTest { "literal_ints", "literal_string", "load_file_with_space_in_the_name", + "louter_join_ppr", + "mapjoin_mapjoin", + "mapjoin_subquery", "mapjoin_subquery2", + "mapjoin_test_outer", "mapreduce3", + "merge1", + "merge2", "mergejoins", "mergejoins_mixed", "misc_json", @@ -216,12 +306,22 @@ class HiveCompatibility extends HiveQueryFileTest { "notable_alias2", "nullgroup", "nullgroup2", + "nullgroup3", + "nullgroup5", "nullinput", "nullinput2", "nullscript", "optional_outer", + "order", + "order2", + "outer_join_ppr", + "part_inherit_tbl_props", + "part_inherit_tbl_props_empty", + "partition_schema1", + "partitions_json", "plan_json", "ppd1", + "ppd_constant_where", "ppd_gby", "ppd_gby_join", "ppd_join", @@ -237,28 +337,40 @@ class HiveCompatibility extends HiveQueryFileTest { "ppd_udf_col", "ppd_union", "progress_1", + "protectmode", + "push_or", "query_with_semi", "quote2", "rename_column", + "router_join_ppr", "select_as_omitted", + "select_unquote_and", + "select_unquote_not", + "select_unquote_or", + "serde_reported_schema", "set_variable_sub", "show_describe_func_quotes", "show_functions", "skewjoinopt13", "skewjoinopt18", "skewjoinopt9", + "smb_mapjoin_10", "smb_mapjoin_13", "smb_mapjoin_14", "smb_mapjoin_15", "smb_mapjoin_16", "smb_mapjoin_17", + "smb_mapjoin_21", + "sort", "sort_merge_join_desc_1", "sort_merge_join_desc_2", "sort_merge_join_desc_3", "sort_merge_join_desc_4", + "sort_merge_join_desc_5", + "sort_merge_join_desc_6", + "sort_merge_join_desc_7", "subq2", "tablename_with_select", - "type_cast_1", "udf2", "udf9", "udf_10_trims", @@ -288,6 +400,7 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_datediff", "udf_day", "udf_dayofmonth", + "udf_div", "udf_double", "udf_exp", "udf_field", From 9ae740a589fb24aaf7635f3533abb570ca877281 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Jan 2014 14:11:09 -0800 Subject: [PATCH 435/778] blacklist more tests that require MR. --- src/test/scala/catalyst/execution/HiveCompatibility.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index 5c1dcf78ba6fd..ebd10286b4bd5 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -42,6 +42,13 @@ class HiveCompatibility extends HiveQueryFileTest { "index_auto_self_join", "index_stale", "type_cast_1", + "index_compression", + "index_bitmap_compression", + "index_auto_multiple", + "index_auto_mult_tables_compact", + "index_auto_mult_tables", + "index_auto_file_format", + "index_auth", // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 From 755b2292e980e285d5f7e5bdd4b66f65aa8ee211 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Jan 2014 14:35:07 -0800 Subject: [PATCH 436/778] blacklist some ddl tests. --- src/test/scala/catalyst/execution/HiveCompatibility.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index ebd10286b4bd5..fd1595e86d102 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -33,6 +33,11 @@ class HiveCompatibility extends HiveQueryFileTest { "misc_json", "create_like_tbl_props", + // Weird DDL differences result in failures on jenkins. + "create_like2", + "create_view_translate", + "partitions_json", + // Timezone specific test answers. "udf_unix_timestamp", "udf_to_unix_timestamp", From e9f45889903455fc2e8f298af4f8aefec40d10d8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Jan 2014 14:55:17 -0800 Subject: [PATCH 437/778] fix > 100 char. --- src/test/scala/catalyst/execution/HiveCompatibility.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index fd1595e86d102..f2eeebd6c886a 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -15,7 +15,8 @@ class HiveCompatibility extends HiveQueryFileTest { /** A list of tests deemed out of scope currently and thus completely disregarded */ override def blackList = Seq( - "hook_order", // These tests use hooks that are not on the classpath and thus break all subsequent SQL execution. + // These tests use hooks that are not on the classpath and thus break all subsequent execution. + "hook_order", "hook_context", "mapjoin_hook", "multi_sahooks", From ef7b9435dfa3a17ac66e8f54a6fe0d648610c07e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 11 Jan 2014 14:26:22 -0800 Subject: [PATCH 438/778] add metastore support for float --- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index d83242583f541..12b06996bd66e 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -82,6 +82,7 @@ object HiveMetatoreTypes { def toDataType(metastoreType: String): DataType = metastoreType match { case "string" => StringType + case "float" => FloatType case "int" => IntegerType case "double" => DoubleType case "bigint" => LongType From f0faa264b84e670e0139bd1496aef895b6f705ee Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 11 Jan 2014 14:31:02 -0800 Subject: [PATCH 439/778] add sample and distinct operators. --- .../scala/catalyst/plans/logical/basicOperators.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index 224e44d857051..0e34e7629a16a 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -68,6 +68,16 @@ case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { def references = Set.empty } +case class Sample(percentage: Double, child: LogicalPlan) extends UnaryNode { + def output = child.output + def references = Set.empty +} + +case class Distinct(child: LogicalPlan) extends UnaryNode { + def output = child.output + def references = child.outputSet +} + case object NoRelation extends LeafNode { def output = Nil } From f58d5a56ca93367658a4c34930c49225c7323952 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 11 Jan 2014 14:27:02 -0800 Subject: [PATCH 440/778] support for parsing SELECT DISTINCT --- src/main/scala/catalyst/frontend/Hive.scala | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index c0a4de505aef9..0692c83543c1d 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -367,12 +367,13 @@ object HiveQl { val ( intoClause :: destClause :: - Some(selectClause) :: + selectClause :: + selectDistinctClause :: whereClause :: groupByClause :: orderByClause :: sortByClause :: - limitClause :: Nil) = getClauses(Seq("TOK_INSERT_INTO", "TOK_DESTINATION", "TOK_SELECT", "TOK_WHERE", "TOK_GROUPBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_LIMIT"), singleInsert) + limitClause :: Nil) = getClauses(Seq("TOK_INSERT_INTO", "TOK_DESTINATION", "TOK_SELECT", "TOK_SELECTDI", "TOK_WHERE", "TOK_GROUPBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_LIMIT"), singleInsert) val relations = nodeToRelation(fromClause) val withWhere = whereClause.map { whereNode => @@ -380,10 +381,12 @@ object HiveQl { Filter(nodeToExpr(whereExpr), relations) }.getOrElse(relations) + val select = + (selectClause orElse selectDistinctClause).getOrElse(sys.error("No select clause.")) // Script transformations are expressed as a select clause with a single expression of type // TOK_TRANSFORM - val transformation = selectClause.getChildren.head match { + val transformation = select.getChildren.head match { case Token("TOK_SELEXPR", Token("TOK_TRANSFORM", Token("TOK_EXPLIST", inputExprs) :: @@ -404,7 +407,7 @@ object HiveQl { // a script transformation. val withProject = transformation.getOrElse { // Not a transformation so must be either project or aggregation. - val selectExpressions = nameExpressions(selectClause.getChildren.flatMap(selExprNodeToExpr)) + val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr)) groupByClause match { case Some(groupBy) => Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withWhere) @@ -412,13 +415,19 @@ object HiveQl { } } + val withDistinct = + if(selectDistinctClause.isDefined) + Distinct(withProject) + else + withProject + require(!(orderByClause.isDefined && sortByClause.isDefined), "Can't have both a sort by and order by.") // Right now we treat sorting and ordering as identical. val withSort = (orderByClause orElse sortByClause) .map(_.getChildren.map(nodeToSortOrder)) - .map(Sort(_, withProject)) - .getOrElse(withProject) + .map(Sort(_, withDistinct)) + .getOrElse(withDistinct) val withLimit = limitClause.map(l => nodeToExpr(l.getChildren.head)) .map(StopAfter(_, withSort)) From a92919d8ee1c1d3ce10082d0023b28dd63efd98b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 11 Jan 2014 14:27:15 -0800 Subject: [PATCH 441/778] add alter view as to native commands --- src/main/scala/catalyst/frontend/Hive.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 0692c83543c1d..1f203288543f2 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -105,6 +105,7 @@ object HiveQl { // TODO(marmbrus): Figure out how view are expanded by hive, as we might need to handle this. "TOK_ALTERVIEW_ADDPARTS", + "TOK_ALTERVIEW_AS", "TOK_ALTERVIEW_DROPPARTS", "TOK_ALTERVIEW_PROPERTIES", "TOK_ALTERVIEW_RENAME", From 0e975eafb5c8e1a1b9fd1c5155beb51716f2668e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 11 Jan 2014 14:29:18 -0800 Subject: [PATCH 442/778] parse bucket sampling as percentage sampling --- src/main/scala/catalyst/frontend/Hive.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 1f203288543f2..e094a921bbe40 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -459,7 +459,7 @@ object HiveQl { query :: Token(alias, Nil) :: Nil) => Subquery(alias, nodeToPlan(query)) - /* Table, No Alias */ + /* All relations, possibly with aliases or sampling clauses. */ case Token("TOK_TABREF", clauses) => // If the last clause is not a token then it's the alias of the table. val (nonAliasClauses, aliasClause) = @@ -469,17 +469,23 @@ object HiveQl { (clauses.dropRight(1), Some(clauses.last)) val (Some(tableNameParts) :: - sampleClause :: Nil) = getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE"), nonAliasClauses) + splitSampleClause :: + bucketSampleClause :: Nil) = getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"), nonAliasClauses) - val tableName = tableNameParts.getChildren.map { case Token(part, Nil) => part }.mkString(".") - val alias = aliasClause.map { case Token(a, Nil) => a } + val tableName = tableNameParts.getChildren.map { case Token(part, Nil) => cleanIdentifier(part) }.mkString(".") + val alias = aliasClause.map { case Token(a, Nil) => cleanIdentifier(a) } val relation = UnresolvedRelation(tableName, alias) + // Apply sampling if requested. - sampleClause.map { + (bucketSampleClause orElse splitSampleClause).map { case Token("TOK_TABLESPLITSAMPLE", Token("TOK_ROWCOUNT", Nil) :: Token(count, Nil) :: Nil) => StopAfter(Literal(count.toInt), relation) + case Token("TOK_TABLEBUCKETSAMPLE", + Token(numerator, Nil) :: + Token(denominator, Nil) :: Nil) => + Sample(numerator.toDouble / denominator.toDouble, relation) }.getOrElse(relation) case Token("TOK_UNIQUEJOIN", joinArgs) => From c5842d274355ebe859e5cf4cd4c61ca0e4e409af Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 11 Jan 2014 14:30:44 -0800 Subject: [PATCH 443/778] don't throw an error when a select clause outputs multiple copies of the same attribute. --- src/main/scala/catalyst/plans/logical/LogicalPlan.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala index 27b96b5219085..c3eaecde3b221 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala @@ -41,7 +41,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { option.name == remainingParts.head } - options match { + options.distinct match { case a :: Nil => Some(a) // One match, use it. case Nil => None // No matches. case ambiguousReferences => From 86355a65bf5036dbf05b0c9d0c75d7dc188d09c8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 11 Jan 2014 14:31:19 -0800 Subject: [PATCH 444/778] throw error if there are unexpected join clauses. --- src/main/scala/catalyst/frontend/Hive.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index e094a921bbe40..ddd0af49a97f6 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -543,6 +543,7 @@ object HiveQl { case "TOK_LEFTOUTERJOIN" => LeftOuter case "TOK_FULLOUTERJOIN" => FullOuter } + assert(other.size <= 1, "Unhandled join clauses.") Join(nodeToRelation(relation1), nodeToRelation(relation2), joinType, From 71e902d84541e294784c482d0b3f3b28c61c085c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 12 Jan 2014 19:42:58 -0800 Subject: [PATCH 445/778] fix test cases. --- ...Suite.scala => ConstantFoldingSuite.scala} | 65 +++++++----------- .../optimizer/FilterPushdownSuite.scala} | 66 ++++++------------- .../catalyst/optimizer/OptimizerTest.scala | 43 ++++++++++++ 3 files changed, 85 insertions(+), 89 deletions(-) rename src/test/scala/catalyst/optimizer/{OptimizerSuite.scala => ConstantFoldingSuite.scala} (69%) rename src/test/scala/{OptimizerSuite.scala => catalyst/optimizer/FilterPushdownSuite.scala} (75%) create mode 100644 src/test/scala/catalyst/optimizer/OptimizerTest.scala diff --git a/src/test/scala/catalyst/optimizer/OptimizerSuite.scala b/src/test/scala/catalyst/optimizer/ConstantFoldingSuite.scala similarity index 69% rename from src/test/scala/catalyst/optimizer/OptimizerSuite.scala rename to src/test/scala/catalyst/optimizer/ConstantFoldingSuite.scala index 6e787e8acf3ba..ae5dd48668566 100644 --- a/src/test/scala/catalyst/optimizer/OptimizerSuite.scala +++ b/src/test/scala/catalyst/optimizer/ConstantFoldingSuite.scala @@ -1,47 +1,26 @@ package catalyst package optimizer -import org.scalatest.FunSuite - import types.IntegerType import util._ import plans.logical.{LogicalPlan, LocalRelation} +import rules._ import expressions._ import dsl._ -/* Implicit conversions for creating query plans */ - -class OptimizerSuite extends FunSuite { - - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) +class ConstantFoldingSuite extends OptimizerTest { - // Helper functions for comparing plans. - - /** - * Since attribute references are given globally unique ids during analysis, - * we must normalize them to check if two different queries are identical. - */ - protected def normalizeExprIds(plan: LogicalPlan) = { - val minId = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)).min - plan transformAllExpressions { - case a: AttributeReference => - AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) - } + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Subqueries", Once, + EliminateSubqueries) :: + Batch("ConstantFolding", Once, + ConstantFolding, + BooleanSimplification) :: Nil } - /** Fails the test if the two plans do not match */ - protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { - val normalized1 = normalizeExprIds(plan1) - val normalized2 = normalizeExprIds(plan2) - if (normalized1 != normalized2) - fail( - s""" - |== FAIL: Plans do not match === - |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} - """.stripMargin) - } + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - // This test already passes. test("eliminate subqueries") { val originalQuery = testRelation @@ -57,9 +36,9 @@ class OptimizerSuite extends FunSuite { comparePlans(optimized, correctAnswer) } - /* - * Unit tests for constant folding in expressions. - * */ + /* + * Unit tests for constant folding in expressions. + * */ test("Constant folding test: expressions only have literals") { val originalQuery = testRelation @@ -69,8 +48,8 @@ class OptimizerSuite extends FunSuite { Literal(2) * (Literal(3) + Literal(4)) as Symbol("2*(3+4)")) .where( Literal(1) === Literal(1) && - Literal(2) > Literal(3) || - Literal(3) > Literal(2) ) + Literal(2) > Literal(3) || + Literal(3) > Literal(2) ) .groupBy( Literal(2) * Literal(3) - Literal(6) / (Literal(4) - Literal(2)) )(Literal(9) / Literal(3) as Symbol("9/3")) @@ -120,13 +99,13 @@ class OptimizerSuite extends FunSuite { testRelation .where( (('a > 1 && Literal(1) === Literal(1)) || - ('a < 10 && Literal(1) === Literal(2)) || - (Literal(1) === Literal(1) && 'b > 1) || - (Literal(1) === Literal(2) && 'b < 10)) && - (('a > 1 || Literal(1) === Literal(1)) && - ('a < 10 || Literal(1) === Literal(2)) && - (Literal(1) === Literal(1) || 'b > 1) && - (Literal(1) === Literal(2) || 'b < 10))) + ('a < 10 && Literal(1) === Literal(2)) || + (Literal(1) === Literal(1) && 'b > 1) || + (Literal(1) === Literal(2) && 'b < 10)) && + (('a > 1 || Literal(1) === Literal(1)) && + ('a < 10 || Literal(1) === Literal(2)) && + (Literal(1) === Literal(1) || 'b > 1) && + (Literal(1) === Literal(2) || 'b < 10))) val optimized = Optimize(originalQuery.analyze) diff --git a/src/test/scala/OptimizerSuite.scala b/src/test/scala/catalyst/optimizer/FilterPushdownSuite.scala similarity index 75% rename from src/test/scala/OptimizerSuite.scala rename to src/test/scala/catalyst/optimizer/FilterPushdownSuite.scala index d732d2142a3ca..e22b5088d66f7 100644 --- a/src/test/scala/OptimizerSuite.scala +++ b/src/test/scala/catalyst/optimizer/FilterPushdownSuite.scala @@ -1,46 +1,27 @@ package catalyst - -import org.scalatest.FunSuite +package optimizer import expressions._ -import optimizer.Optimize import plans.logical._ +import rules._ import util._ -/* Implicit conversions for creating query plans */ import dsl._ -class OptimizerSuite extends FunSuite { - // Test relations. Feel free to create more. - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - - // Helper functions for comparing plans. - - /** - * Since attribute references are given globally unique ids during analysis we must normalize them to check if two - * different queries are identical. - */ - protected def normalizeExprIds(plan: LogicalPlan) = { - val minId = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)).min - plan transformAllExpressions { - case a: AttributeReference => - AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) - } +class FilterPushdownSuite extends OptimizerTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Subqueries", Once, + EliminateSubqueries) :: + Batch("Filter Pushdown", Once, + EliminateSubqueries, + CombineFilters, + PredicatePushDownThoughProject, + PushPredicateThroughInnerJoin) :: Nil } - /** Fails the test if the two plans do not match */ - protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { - val normalized1 = normalizeExprIds(plan1) - val normalized2 = normalizeExprIds(plan2) - println(normalized1.treeString) - println(normalized2.treeString) - if(normalized1 != normalized2) - fail( - s""" - |== FAIL: Plans do not match === - |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} - """.stripMargin) - } + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) // This test already passes. test("eliminate subqueries") { @@ -169,8 +150,7 @@ class OptimizerSuite extends FunSuite { val y = testRelation.subquery('y) val originalQuery = { - x.join(y) - .where("x.b".attr === "y.b".attr) + x.join(y, condition = Some("x.b".attr === "y.b".attr)) } val optimized = Optimize(originalQuery.analyze) @@ -190,8 +170,7 @@ class OptimizerSuite extends FunSuite { val left = testRelation.where('a === 1).subquery('x) val right = testRelation.where('a === 1).subquery('y) val correctAnswer = - left.join(right) - .where("x.b".attr === "y.b".attr) + left.join(right, condition = Some("x.b".attr === "y.b".attr)) .analyze comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) @@ -210,8 +189,7 @@ class OptimizerSuite extends FunSuite { val left = testRelation.where('a === 1).subquery('x) val right = testRelation.subquery('y) val correctAnswer = - left.join(right) - .where("x.b".attr === "y.b".attr) + left.join(right, condition = Some("x.b".attr === "y.b".attr)) .analyze comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) @@ -232,13 +210,9 @@ class OptimizerSuite extends FunSuite { val left = testRelation.where('a === 1).subquery('x) val right = testRelation.subquery('y) val correctAnswer = - lleft - .join( - left - .join(right) - .where("x.b".attr === "y.b".attr) - ) - .where("z.a".attr === "x.b".attr) + lleft.join( + left.join(right, condition = Some("x.b".attr === "y.b".attr)), + condition = Some("z.a".attr === "x.b".attr)) .analyze comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) diff --git a/src/test/scala/catalyst/optimizer/OptimizerTest.scala b/src/test/scala/catalyst/optimizer/OptimizerTest.scala new file mode 100644 index 0000000000000..e885fc61a6937 --- /dev/null +++ b/src/test/scala/catalyst/optimizer/OptimizerTest.scala @@ -0,0 +1,43 @@ +package catalyst +package optimizer + +import org.scalatest.FunSuite + +import types.IntegerType +import util._ +import plans.logical.{LogicalPlan, LocalRelation} +import expressions._ +import dsl._ + +/* Implicit conversions for creating query plans */ + +/** + * Provides helper methods for comparing plans produced by optimization rules with the expected + * result + */ +class OptimizerTest extends FunSuite { + + /** + * Since attribute references are given globally unique ids during analysis, + * we must normalize them to check if two different queries are identical. + */ + protected def normalizeExprIds(plan: LogicalPlan) = { + val minId = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)).min + plan transformAllExpressions { + case a: AttributeReference => + AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) + } + } + + /** Fails the test if the two plans do not match */ + protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { + val normalized1 = normalizeExprIds(plan1) + val normalized2 = normalizeExprIds(plan2) + if (normalized1 != normalized2) + fail( + s""" + |== FAIL: Plans do not match === + |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} + """.stripMargin) + } +} \ No newline at end of file From 1fedd186f1fa6c0a25e5f9910294dc25390ce9f5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 14 Jan 2014 22:57:56 -0800 Subject: [PATCH 446/778] coercion from null to numeric types --- src/main/scala/catalyst/analysis/typeCoercion.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index 66296102616e4..f883a3f9c141e 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -52,9 +52,9 @@ object ConvertNaNs extends Rule[LogicalPlan] { * String conversions are handled by the PromoteStrings rule. */ object PromoteNumericTypes extends Rule[LogicalPlan] { - val integralPrecedence = Seq(ByteType, ShortType, IntegerType, LongType) - val toDouble = integralPrecedence ++ Seq(FloatType, DoubleType) - val toFloat = Seq(ByteType, ShortType, IntegerType) :+ FloatType + val integralPrecedence = Seq(NullType, ByteType, ShortType, IntegerType, LongType) + val toDouble = integralPrecedence ++ Seq(NullType, FloatType, DoubleType) + val toFloat = Seq(NullType, ByteType, ShortType, IntegerType) :+ FloatType val allPromotions = integralPrecedence :: toDouble :: toFloat :: Nil def apply(plan: LogicalPlan): LogicalPlan = plan transform { From 859200a37f9597ed13a24711db00eb7a25cb3e72 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 12:36:54 -0800 Subject: [PATCH 447/778] support for reading more types from the metastore. --- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 12b06996bd66e..3509d834706b4 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -79,6 +79,9 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { object HiveMetatoreTypes { val VARCHAR = "(?i)VARCHAR\\((\\d+)\\)".r + // TODO: this will not work for nested arrays or maps. + val ARRAY = "(?i)array<([^>]+)>".r + val MAP = "(?i)map<([^,]+),([^>]*)>".r def toDataType(metastoreType: String): DataType = metastoreType match { case "string" => StringType @@ -87,7 +90,11 @@ object HiveMetatoreTypes { case "double" => DoubleType case "bigint" => LongType case "binary" => BinaryType + case "boolean" => BooleanType case VARCHAR(_) => StringType + case ARRAY(elemType) => ArrayType(toDataType(elemType)) + case MAP(keyType, valueType) => MapType(toDataType(keyType), toDataType(valueType)) + case _ => sys.error(s"Unsupported dataType: $metastoreType") } } From 0606520fda729b75d016203805a4b185edde796c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 13:44:34 -0800 Subject: [PATCH 448/778] drop old comment. --- src/main/scala/catalyst/execution/FunctionRegistry.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index 9d1a8f8520106..ca595267fdf28 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -178,7 +178,7 @@ case class HiveGenericUdf( case l: Short => l: java.lang.Short case l: Byte => l: java.lang.Byte case s: Seq[_] => seqAsJavaList(s.map(wrap)) - case null => null // NullWritable.get() + case null => null } def evaluate(evaluatedChildren: Seq[Any]): Any = { From 12fd52d209b0ce930de18e52581761aa198aa1ba Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 13:45:11 -0800 Subject: [PATCH 449/778] support for sorting longs. --- src/main/scala/catalyst/execution/basicOperators.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index b2e3fd2b9806e..785cea6eeace4 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -81,6 +81,12 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { } else { right.asInstanceOf[Double] compare left.asInstanceOf[Double] } + } else if (curDataType == LongType) { + if (curDirection == Ascending) { + left.asInstanceOf[Long] compare right.asInstanceOf[Long] + } else { + right.asInstanceOf[Long] compare left.asInstanceOf[Long] + } } else if (curDataType == StringType) { if (curDirection == Ascending) { left.asInstanceOf[String] compare right.asInstanceOf[String] From 35d2948716378156e73717abfbf991637d79028c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 13:45:36 -0800 Subject: [PATCH 450/778] correctly order partition and normal attributes in hive relation output. --- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 3509d834706b4..c46b3c3d4655d 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -127,5 +127,5 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt val partitionKeys = hiveQlTable.getPartitionKeys.map(_.toAttribute) // Must be a stable value since new attributes are born here. - val output = partitionKeys ++ table.getSd.getCols.map(_.toAttribute) + val output = table.getSd.getCols.map(_.toAttribute) ++ partitionKeys } From 04f885bb5aa3909de1c572542f26a4ef03e294f0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 13:45:49 -0800 Subject: [PATCH 451/778] literals are only non-nullable if they are not null. --- src/main/scala/catalyst/expressions/literals.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/expressions/literals.scala b/src/main/scala/catalyst/expressions/literals.scala index 56062e22e3227..16d7f52b124c5 100644 --- a/src/main/scala/catalyst/expressions/literals.scala +++ b/src/main/scala/catalyst/expressions/literals.scala @@ -29,7 +29,7 @@ object IntegerLiteral { case class Literal(value: Any, dataType: DataType) extends LeafExpression { override def foldable = true - def nullable = false + def nullable = value == null def references = Set.empty override def toString = if (value != null) value.toString else "null" From a72e54080b1cf2c6e900ebfbf2ede0fbbf91187e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 13:46:21 -0800 Subject: [PATCH 452/778] Add IN operator. --- src/main/scala/catalyst/expressions/Evaluate.scala | 4 ++++ src/main/scala/catalyst/expressions/predicates.scala | 10 ++++++++++ src/main/scala/catalyst/frontend/Hive.scala | 1 + 3 files changed, 15 insertions(+) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 5df97de9440dc..e15a6e557cf3b 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -158,6 +158,10 @@ object Evaluate extends Logging { else left == right + case In(value, list) => + val evaluatedValue = eval(value) + list.exists(e => eval(e) == evaluatedValue) + // Strings case GreaterThan(l, r) if l.dataType == StringType && r.dataType == StringType => eval(l).asInstanceOf[String] > eval(r).asInstanceOf[String] diff --git a/src/main/scala/catalyst/expressions/predicates.scala b/src/main/scala/catalyst/expressions/predicates.scala index e6cf7256005d5..dc4b4a69b017e 100644 --- a/src/main/scala/catalyst/expressions/predicates.scala +++ b/src/main/scala/catalyst/expressions/predicates.scala @@ -21,6 +21,16 @@ case class Not(child: Expression) extends Predicate with trees.UnaryNode[Express override def toString = s"NOT $child" } +/** + * Evaluates to `true` if `list` contains `value`. + */ +case class In(value: Expression, list: Seq[Expression]) extends Predicate { + def children = value +: list + def references = children.flatMap(_.references).toSet + def nullable = true // TODO: Figure out correct nullability semantics of IN. + override def toString = s"$value IN ${list.mkString("(", ",", ")")}" +} + case class And(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "&&" } diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index ddd0af49a97f6..494e7fffe8ead 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -678,6 +678,7 @@ object HiveQl { case Token("REGEXP", left :: right:: Nil) => UnresolvedFunction("REGEXP", Seq(nodeToExpr(left), nodeToExpr(right))) case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => IsNotNull(nodeToExpr(child)) case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => IsNull(nodeToExpr(child)) + case Token("TOK_FUNCTION", Token("IN", Nil) :: value :: list) => In(nodeToExpr(value), list.map(nodeToExpr)) /* Boolean Logic */ case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) From 6aefa46a3a3995779202a201f4ad84aafda0820b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 13:46:36 -0800 Subject: [PATCH 453/778] Skip folding literals. --- src/main/scala/catalyst/optimizer/Optimizer.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala index 8126f1e949660..819c662a5185e 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/src/main/scala/catalyst/optimizer/Optimizer.scala @@ -33,6 +33,8 @@ object EliminateSubqueries extends Rule[LogicalPlan] { object ConstantFolding extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsDown { + // Skip redundant folding of literals. + case l: Literal => l case e if e.foldable => Literal(Evaluate(e, Nil), e.dataType) } } From c8e89d5d3af39d18ab0baf2f0709c5ec0815499e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 13:46:51 -0800 Subject: [PATCH 454/778] memoize inputSet calculation. --- src/main/scala/catalyst/plans/logical/LogicalPlan.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala index c3eaecde3b221..792eb88e77a28 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala @@ -18,7 +18,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { * Returns the set of attributes that this node takes as * input from its children. */ - def inputSet: Set[Attribute] = children.flatMap(_.output).toSet + lazy val inputSet: Set[Attribute] = children.flatMap(_.output).toSet /** * Returns true if this expression and all its children have been resolved to a specific schema From bc282c78765671ac3d42fba7243fa5e9fd35e3cd Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 13:47:10 -0800 Subject: [PATCH 455/778] fix bug in getNodeNumbered --- src/main/scala/catalyst/trees/TreeNode.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index c40df65d3bafb..a4af9b84c8fef 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -257,7 +257,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { this } else { number.i -= 1 - children.map(_.getNodeNumbered(number)).find(_ != null).getOrElse(sys.error("No such node.")) + children.map(_.getNodeNumbered(number)).find(_ != null).getOrElse(null.asInstanceOf[BaseType]) } } From 3ccaef7886b62b89cc3cc5e9e427e143f1ec3e13 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 14:02:36 -0800 Subject: [PATCH 456/778] add renaming TODO. --- src/main/scala/catalyst/execution/basicOperators.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 785cea6eeace4..75f5808618df7 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -117,6 +117,7 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { def output = child.output } +// TODO: Rename: SchemaRDD case class LocalRelation(output: Seq[Attribute], data: Seq[IndexedSeq[Any]]) (@transient sc: SharkContext) extends LeafNode { def execute() = sc.makeRDD(data.map(buildRow), 1) From 7102f33bb090ec27796b70287a2a2ae33e2b0d57 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 14:03:02 -0800 Subject: [PATCH 457/778] methods with side-effects should use (). --- src/main/scala/catalyst/util/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/util/package.scala b/src/main/scala/catalyst/util/package.scala index 17709cac2d370..895e39765084b 100644 --- a/src/main/scala/catalyst/util/package.scala +++ b/src/main/scala/catalyst/util/package.scala @@ -78,7 +78,7 @@ package object util { def stringOrNull(a: AnyRef) = if (a == null) null else a.toString implicit class debugLogging(a: AnyRef) { - def debugLogging { + def debugLogging() { org.apache.log4j.Logger.getLogger(a.getClass.getName).setLevel(org.apache.log4j.Level.DEBUG) } } From a90efda68f185844d482c1ffab80c058ff580334 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 14:03:22 -0800 Subject: [PATCH 458/778] utility function for outputing string stacktraces. --- src/main/scala/catalyst/util/package.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/main/scala/catalyst/util/package.scala b/src/main/scala/catalyst/util/package.scala index 895e39765084b..a7cf951a9c590 100644 --- a/src/main/scala/catalyst/util/package.scala +++ b/src/main/scala/catalyst/util/package.scala @@ -75,6 +75,14 @@ package object util { } } + def stackTraceToString(t: Throwable): String = { + val out = new java.io.ByteArrayOutputStream + val writer = new PrintWriter(out) + t.printStackTrace(writer) + writer.flush() + new String(out.toByteArray) + } + def stringOrNull(a: AnyRef) = if (a == null) null else a.toString implicit class debugLogging(a: AnyRef) { From b65626e27ef1601eb8ced6d9d24f0716f8caa820 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 14:04:04 -0800 Subject: [PATCH 459/778] Initial support for parsing BigDecimal. --- src/main/scala/catalyst/frontend/Hive.scala | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 494e7fffe8ead..92a7860519008 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -715,19 +715,9 @@ object HiveQl { // Literal tinyint. v = Literal(ast.getText().substring(0, ast.getText().length() - 1).toByte, ByteType) } else if (ast.getText().endsWith("BD")) { - throw new NotImplementedError("Hive Decimal not implemented yet") - /* TODO: Implement! // Literal decimal val strVal = ast.getText().substring(0, ast.getText().length() - 2); - HiveDecimal hd = HiveDecimal.create(strVal); - int prec = 1; - int scale = 0; - if (hd != null) { - prec = hd.precision(); - scale = hd.scale(); - } - DecimalTypeInfo typeInfo = TypeInfoFactory.getDecimalTypeInfo(prec, scale); - return new ExprNodeConstantDesc(typeInfo, strVal); */ + BigDecimal(strVal) } else { v = Literal(ast.getText().toDouble, DoubleType) v = Literal(ast.getText().toLong, LongType) From d0124f3118c7a1617f161b4d22bdec8a48cda260 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 14:04:18 -0800 Subject: [PATCH 460/778] Correctly type null literals. --- src/main/scala/catalyst/frontend/Hive.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 92a7860519008..2465b61752c4c 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -695,7 +695,7 @@ object HiveQl { UnresolvedFunction(name, Star(None) :: Nil) /* Literals */ - case Token("TOK_NULL", Nil) => Literal(null, IntegerType) // TODO: What type is null? + case Token("TOK_NULL", Nil) => Literal(null, NullType) case Token(TRUE(), Nil) => Literal(true, BooleanType) case Token(FALSE(), Nil) => Literal(false, BooleanType) case Token("TOK_STRINGLITERALSEQUENCE", strings) => From 7fcfa8a034eec369562a3e449630f3ed65c57259 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 14:05:24 -0800 Subject: [PATCH 461/778] Initial support for parsing unspecified partition parameters. --- src/main/scala/catalyst/execution/hiveOperators.scala | 5 +++-- src/main/scala/catalyst/frontend/Hive.scala | 3 ++- src/main/scala/catalyst/plans/logical/basicOperators.scala | 2 +- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 42c8ef1775a83..efdaa94950ac7 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -78,7 +78,7 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation } case class InsertIntoHiveTable( - table: MetastoreRelation, partition: Map[String, String], child: SharkPlan) + table: MetastoreRelation, partition: Map[String, Option[String]], child: SharkPlan) (@transient sc: SharkContext) extends UnaryNode { @@ -110,6 +110,7 @@ case class InsertIntoHiveTable( def output = child.output def execute() = { + require(partition.isEmpty, "Inserting into partitioned table not supported.") val childRdd = child.execute() assert(childRdd != null) @@ -122,7 +123,7 @@ case class InsertIntoHiveTable( val partitionSpec = if (partition.nonEmpty) { - s"PARTITION (${partition.map { case (k,v) => s"$k=$v" }.mkString(",")})" + s"PARTITION (${partition.map { case (k,v) => s"$k=${v.get}" }.mkString(",")})" } else { "" } diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 2465b61752c4c..4019f1b73b35b 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -578,7 +578,8 @@ object HiveQl { val Token("TOK_TABNAME", Token(tableName, Nil) :: Nil) = nameClause val partitionKeys = partitionClause.map(_.getChildren.map { - case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => key -> value + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => key -> Some(value) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => key -> None }.toMap).getOrElse(Map.empty) InsertIntoTable(UnresolvedRelation(tableName, None), partitionKeys, query) diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index 0e34e7629a16a..1f0f5d15c21e6 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -30,7 +30,7 @@ case class Join( def output = left.output ++ right.output } -case class InsertIntoTable(table: BaseRelation, partition: Map[String, String], child: LogicalPlan) +case class InsertIntoTable(table: BaseRelation, partition: Map[String, Option[String]], child: LogicalPlan) extends LogicalPlan { // The table being inserted into is a child for the purposes of transformations. def children = table :: child :: Nil From 6924e6eb9201b225760b4b42a864652b5a735671 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:44:33 -0800 Subject: [PATCH 462/778] Handle NullTypes when resolving HiveUDFs --- src/main/scala/catalyst/execution/FunctionRegistry.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index ca595267fdf28..d3b4318fb79d9 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -162,6 +162,7 @@ case class HiveGenericUdf( case LongType => PrimitiveObjectInspectorFactory.javaLongObjectInspector case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector + case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector } lazy val instance = { From f2ec57e441713141dceb09736d0441f53e36ac3a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:45:09 -0800 Subject: [PATCH 463/778] Begin supporting decimal. --- src/main/scala/catalyst/execution/FunctionRegistry.scala | 1 + src/main/scala/catalyst/types/dataTypes.scala | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index d3b4318fb79d9..7efc08629a99b 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -40,6 +40,7 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry { def javaClassToDataType(clz: Class[_]): DataType = clz match { case c: Class[_] if c == classOf[DoubleWritable] => DoubleType case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.DoubleWritable] => DoubleType + case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.HiveDecimalWritable] => DecimalType case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.ByteWritable] => ByteType case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.ShortWritable] => ShortType case c: Class[_] if c == classOf[Text] => StringType diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 8627a016efdb1..23cb7b20b11a0 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -14,6 +14,8 @@ case object LongType extends DataType case object ByteType extends DataType case object ShortType extends DataType +case object DecimalType extends DataType + case object NullType extends DataType case class ArrayType(elementType: DataType) extends DataType From 1814ed308074c7fd8e2f29a20e78da9fba31380b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:45:30 -0800 Subject: [PATCH 464/778] use childrenResolved function. --- src/main/scala/catalyst/analysis/Analyzer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index a0a03933f85ae..c959e97c09473 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -91,7 +91,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { - case UnresolvedFunction(name, children) if children.map(_.resolved).reduceLeft(_&&_) => + case u @ UnresolvedFunction(name, children) if u.childrenResolved => registry.lookupFunction(name, children) } } From e1440edac0de4df1af83081887a41d37fda143ce Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:45:53 -0800 Subject: [PATCH 465/778] Initial support for function specific type conversions. --- .../scala/catalyst/analysis/Analyzer.scala | 4 +- .../catalyst/analysis/typeCoercion.scala | 45 +++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index c959e97c09473..211b8281a0bb7 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -39,7 +39,9 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool PromoteNumericTypes, PromoteStrings, ConvertNaNs, - BooleanComparisons) + BooleanComparisons, + FunctionArgumentConversion, + PropagateTypes) ) /** diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index f883a3f9c141e..612008b31ba68 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -5,6 +5,33 @@ import expressions._ import plans.logical._ import rules._ import types._ +import catalyst.execution.{HiveUdf, HiveGenericUdf} + +/** + * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] dataTypes + * that are made by other rules to instances higher in the query tree. + */ +object PropagateTypes extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // No propagation required for leaf nodes. + case q: LogicalPlan if q.children.isEmpty => q + + case q: LogicalPlan => q transformExpressions { + case a: AttributeReference => + q.inputSet.find(_.exprId == a.exprId) match { + // This can happen when a Attribute reference is born in a non-leaf node, for example + // due to a call to an external script like in the Transform operator. + // TODO: Perhaps those should actually be aliases? + case None => a + // Leave the same if the dataTypes match. + case Some(newType) if a.dataType == newType.dataType => a + case Some(newType) => + logger.debug(s"Promoting $a to ${newType} in ${q.simpleString}}") + newType + } + } + } +} /** * Converts string "NaN"s that are in binary operators with a NaN-able types (Float / Double) to the @@ -123,3 +150,21 @@ object BooleanComparisons extends Rule[LogicalPlan] { p.makeCopy(Array(Cast(p.left, ByteType), Cast(p.right, ByteType))) } } + +/** + * This ensure that the types for various functions are as expected. Most of these rules are + * actually Hive specific. + * TODO: Move this to the hive specific package once we make that separation. + */ +object FunctionArgumentConversion extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + // Promote SUM to largest types to prevent overflows. + // TODO: This is enough to make most of the tests pass, but we really need a full set of our own + // to really ensure compatibility. + case Sum(e) if e.dataType == IntegerType => Sum(Cast(e, LongType)) + + } +} From e57f97ae7e7a80169ff5ea469e89a87813cc3ed5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:46:13 -0800 Subject: [PATCH 466/778] more decimal/null support. --- src/main/scala/catalyst/execution/package.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/catalyst/execution/package.scala b/src/main/scala/catalyst/execution/package.scala index c05bdac4881fd..d27a4721f6cc9 100644 --- a/src/main/scala/catalyst/execution/package.scala +++ b/src/main/scala/catalyst/execution/package.scala @@ -27,6 +27,8 @@ package object execution { case LongType => longTypeInfo case ShortType => shortTypeInfo case StringType => stringTypeInfo + case DecimalType => decimalTypeInfo + case NullType => voidTypeInfo } } } \ No newline at end of file From 87774899fe8b4d6778bcad37c61217300c6272df Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:47:13 -0800 Subject: [PATCH 467/778] Initial support for operators that allow the user to specify partitioning. --- .../execution/planningStrategies.scala | 3 +++ .../catalyst/plans/logical/partitioning.scala | 27 +++++++++++++++++++ 2 files changed, 30 insertions(+) create mode 100644 src/main/scala/catalyst/plans/logical/partitioning.scala diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 92a34102fa495..2d00f3ccb7972 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -136,6 +136,9 @@ trait PlanningStrategies { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => execution.Sort(sortExprs, planLater(child)) :: Nil + // TODO: It is correct, but overkill to do a global sorting here. + case logical.SortPartitions(sortExprs, child) => + execution.Sort(sortExprs, planLater(child)) :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => diff --git a/src/main/scala/catalyst/plans/logical/partitioning.scala b/src/main/scala/catalyst/plans/logical/partitioning.scala new file mode 100644 index 0000000000000..e655f1fcb115b --- /dev/null +++ b/src/main/scala/catalyst/plans/logical/partitioning.scala @@ -0,0 +1,27 @@ +package catalyst +package plans +package logical + +import expressions._ + +/** + * Performs a physical redistribution of the data. Used when the consumer of the query + * result have expectations about the distribution and ordering of partitioned input data. + */ +abstract class RedistributeData extends UnaryNode { + self: Product => + + def output = child.output +} + +case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan) + extends RedistributeData { + + def references = sortExpressions.flatMap(_.references).toSet +} + +case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan) + extends RedistributeData { + + def references = partitionExpressions.flatMap(_.references).toSet +} From 3ba53c98f5a35a883168e2ec2a841507472e8297 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:47:40 -0800 Subject: [PATCH 468/778] Output type suffixes on AttributeReferences. --- .../catalyst/expressions/namedExpressions.scala | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/main/scala/catalyst/expressions/namedExpressions.scala b/src/main/scala/catalyst/expressions/namedExpressions.scala index 73c727ff3a5c8..64f21cb66b210 100644 --- a/src/main/scala/catalyst/expressions/namedExpressions.scala +++ b/src/main/scala/catalyst/expressions/namedExpressions.scala @@ -24,6 +24,15 @@ abstract class NamedExpression extends Expression { def qualifiers: Seq[String] def toAttribute: Attribute + + protected def typeSuffix = + if(resolved) + dataType match { + case LongType => "L" + case _ => "" + } + else + "" } abstract class Attribute extends NamedExpression { @@ -62,7 +71,7 @@ case class Alias(child: Expression, name: String) } } - override def toString: String = s"$child AS $name#${exprId.id}" + override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" override protected final def otherCopyArgs = exprId :: qualifiers :: Nil } @@ -84,7 +93,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea extends Attribute with trees.LeafNode[Expression] { override def equals(other: Any) = other match { - case ar: AttributeReference => exprId == ar.exprId + case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType case _ => false } @@ -109,5 +118,5 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea else AttributeReference(name, dataType, nullable)(exprId, newQualifiers) - override def toString: String = s"$name#${exprId.id}" + override def toString: String = s"$name#${exprId.id}$typeSuffix" } \ No newline at end of file From 965ac9abef1fb8fbf89d31506f5c8e7f139484a3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:48:02 -0800 Subject: [PATCH 469/778] Add expressions that allow access into complex types. --- .../catalyst/expressions/complexTypes.scala | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 src/main/scala/catalyst/expressions/complexTypes.scala diff --git a/src/main/scala/catalyst/expressions/complexTypes.scala b/src/main/scala/catalyst/expressions/complexTypes.scala new file mode 100644 index 0000000000000..241f056b91c05 --- /dev/null +++ b/src/main/scala/catalyst/expressions/complexTypes.scala @@ -0,0 +1,35 @@ +package catalyst +package expressions + +import types._ + +/** + * Returns the item at `ordinal` in the Array `child`. + */ +case class GetItem(child: Expression, ordinal: Expression) extends Expression { + val children = child :: ordinal :: Nil + /** `Null` is returned for invalid ordinals. */ + override def nullable = true + override def references = children.flatMap(_.references).toSet + def dataType = child.dataType match { + case ArrayType(dt) => dt + } + override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[ArrayType] + override def toString = s"$child[$ordinal]" +} + +/** + * Returns the value of fields in the Struct `child`. + */ +case class GetField(child: Expression, fieldName: String) extends UnaryExpression { + def dataType = field.dataType + def nullable = field.nullable + lazy val field = child.dataType match { + case s: StructType => + s.fields + .find(_.name == fieldName) + .getOrElse(sys.error(s"No such field $fieldName in ${child.dataType}")) + } + override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType] + override def toString = s"$child.$fieldName" +} \ No newline at end of file From 3695141ddfaa816c2812d184b575fff0c936e29e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:48:24 -0800 Subject: [PATCH 470/778] Lots of parser improvements. --- .../scala/catalyst/expressions/Evaluate.scala | 1 + src/main/scala/catalyst/frontend/Hive.scala | 63 ++++++++++++++----- 2 files changed, 48 insertions(+), 16 deletions(-) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index e15a6e557cf3b..cb340004f0a84 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -222,6 +222,7 @@ object Evaluate extends Logging { case Cast(e, LongType) => n1(e, _.toLong(_)) case Cast(e, ShortType) => n1(e, _.toInt(_).toShort) case Cast(e, ByteType) => n1(e, _.toInt(_).toByte) + case Cast(e, DecimalType) => n1(e, (n,v) => BigDecimal(n.toDouble(v))) /* Boolean Logic */ case Not(c) => diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 4019f1b73b35b..2e24b50a56a40 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -77,6 +77,7 @@ object HiveQl { "TOK_CREATEFUNCTION", "TOK_DROPFUNCTION", + "TOK_ANALYZE", "TOK_ALTERDATABASE_PROPERTIES", "TOK_ALTERINDEX_PROPERTIES", "TOK_ALTERINDEX_REBUILD", @@ -324,12 +325,12 @@ object HiveQl { case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: Nil) => - StructField(fieldName, nodeToDataType(dataType)) + StructField(fieldName, nodeToDataType(dataType), nullable = true) case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: _ /* comment */:: Nil) => - StructField(fieldName, nodeToDataType(dataType) ) + StructField(fieldName, nodeToDataType(dataType), nullable = true) case a: ASTNode => throw new NotImplementedError(s"No parse rules for StructField:\n ${dumpTree(a).toString} ") } @@ -343,7 +344,8 @@ object HiveQl { protected def nodeToPlan(node: Node): LogicalPlan = node match { // Just fake explain for any of the native commands. - case Token("TOK_EXPLAIN", Token(explainType, _) :: Nil) if nativeCommands contains explainType => NoRelation + case Token("TOK_EXPLAIN", explainArgs) if nativeCommands contains explainArgs.head.getText => + NoRelation case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. val Some(query) :: _ :: _ :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) @@ -374,7 +376,9 @@ object HiveQl { groupByClause :: orderByClause :: sortByClause :: - limitClause :: Nil) = getClauses(Seq("TOK_INSERT_INTO", "TOK_DESTINATION", "TOK_SELECT", "TOK_SELECTDI", "TOK_WHERE", "TOK_GROUPBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_LIMIT"), singleInsert) + clusterByClause :: + distributeByClause :: + limitClause :: Nil) = getClauses(Seq("TOK_INSERT_INTO", "TOK_DESTINATION", "TOK_SELECT", "TOK_SELECTDI", "TOK_WHERE", "TOK_GROUPBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY", "TOK_DISTRIBUTEBY", "TOK_LIMIT"), singleInsert) val relations = nodeToRelation(fromClause) val withWhere = whereClause.map { whereNode => @@ -392,14 +396,21 @@ object HiveQl { Token("TOK_TRANSFORM", Token("TOK_EXPLIST", inputExprs) :: Token("TOK_SERDE", Nil) :: - Token("TOK_RECORDWRITER", Nil) :: // TODO: Need to support other types of (in/out)put + Token("TOK_RECORDWRITER", writerClause) :: // TODO: Need to support other types of (in/out)put Token(script, Nil):: - Token("TOK_SERDE", Nil) :: - Token("TOK_RECORDREADER", Nil) :: - Token("TOK_ALIASLIST", aliases) :: Nil) :: Nil) => - - val output = aliases.map { case Token(n, Nil) => AttributeReference(n, StringType)() } + Token("TOK_SERDE", serdeClause) :: + Token("TOK_RECORDREADER", readerClause) :: + outputClause :: Nil) :: Nil) => + + val output = outputClause match { + case Token("TOK_ALIASLIST", aliases) => + aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() } + case Token("TOK_TABCOLLIST", attributes) => + attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => + AttributeReference(name, nodeToDataType(dataType))() } + } val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) + Some(Transform(inputExprs.map(nodeToExpr), unescapedScript, output, withWhere)) case _ => None } @@ -422,13 +433,24 @@ object HiveQl { else withProject - require(!(orderByClause.isDefined && sortByClause.isDefined), "Can't have both a sort by and order by.") - // Right now we treat sorting and ordering as identical. val withSort = - (orderByClause orElse sortByClause) - .map(_.getChildren.map(nodeToSortOrder)) - .map(Sort(_, withDistinct)) - .getOrElse(withDistinct) + (orderByClause, sortByClause, distributeByClause, clusterByClause) match { + case (Some(totalOrdering), None, None, None) => + Sort(totalOrdering.getChildren.map(nodeToSortOrder), withDistinct) + case (None, Some(perPartitionOrdering), None, None) => + SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), withDistinct) + case (None, None, Some(partitionExprs), None) => + Repartition(partitionExprs.getChildren.map(nodeToExpr), withDistinct) + case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => + SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), + Repartition(partitionExprs.getChildren.map(nodeToExpr), withDistinct)) + case (None, None, None, Some(clusterExprs)) => + SortPartitions(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), + Repartition(clusterExprs.getChildren.map(nodeToExpr), withDistinct)) + case (None, None, None, None) => withDistinct + case _ => sys.error("Unsupported set of ordering / distribution clauses.") + } + val withLimit = limitClause.map(l => nodeToExpr(l.getChildren.head)) .map(StopAfter(_, withSort)) @@ -634,6 +656,10 @@ object HiveQl { case Token(".", qualifier :: Token(attr, Nil) :: Nil) => nodeToExpr(qualifier) match { case UnresolvedAttribute(qualifierName) => UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)) + // The precidence for . seems to be wrong, so [] binds tighter an we need to go inside to find + // the underlying attribute references. + case GetItem(UnresolvedAttribute(qualifierName), ordinal) => + GetItem(UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)), ordinal) } /* Stars (*) */ @@ -656,6 +682,7 @@ object HiveQl { case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ShortType) case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ByteType) case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BinaryType) + case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BooleanType) /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) @@ -686,6 +713,10 @@ object HiveQl { case Token(OR(), left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right)) case Token(NOT(), child :: Nil) => Not(nodeToExpr(child)) + /* Complex datatype manipulation */ + case Token("[", child :: ordinal :: Nil) => + GetItem(nodeToExpr(child), nodeToExpr(ordinal)) + /* Other functions */ case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand From 5460b2df6e9f970fabafa12fbeed545390585872 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:48:36 -0800 Subject: [PATCH 471/778] load srcpart by default. --- src/main/scala/catalyst/execution/TestShark.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 736166326656b..f297c14ca805e 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -144,7 +144,7 @@ object TestShark extends SharkInstance { * A list of test tables and the DDL required to initialize them. A test table is loaded on * demand when a query are run against it. */ - val testTables = new mutable.HashMap[String, TestTable]() + lazy val testTables = new mutable.HashMap[String, TestTable]() def registerTestTable(testTable: TestTable) = testTables += (testTable.name -> testTable) // The test tables that are defined in the Hive QTestUtil. @@ -269,6 +269,7 @@ object TestShark extends SharkInstance { // Analyzer and thus the test table auto-loading mechanism. // Remove after we handle more DDL operations natively. loadTestTable("src") + loadTestTable("srcpart") } catch { case e: Exception => logger.error(s"FATAL ERROR: Failed to reset TestDB state. $e") From 09c6300029e443b1d0168fb80451bb4982cbe3fa Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:49:06 -0800 Subject: [PATCH 472/778] Add nullability information to StructFields. --- src/main/scala/catalyst/types/dataTypes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 23cb7b20b11a0..4b23a523df01b 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -20,7 +20,7 @@ case object NullType extends DataType case class ArrayType(elementType: DataType) extends DataType -case class StructField(name: String, dataType: DataType) +case class StructField(name: String, dataType: DataType, nullable: Boolean) case class StructType(fields: Seq[StructField]) extends DataType case class MapType(keyType: DataType, valueType: DataType) extends DataType \ No newline at end of file From c4104ec08c5150db7078acd012aecaa7444d9bcd Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:49:44 -0800 Subject: [PATCH 473/778] Numerous improvements to testing infrastructure. See comments for details. --- .../execution/HiveComparisonTest.scala | 245 +++++++++++++----- .../execution/HiveQueryFileTest.scala | 10 +- 2 files changed, 194 insertions(+), 61 deletions(-) diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index a6a3c3b172f28..61a9f98c960a9 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -14,11 +14,49 @@ import util._ * The "golden" results from Hive are cached in an retrieved both from the classpath and * [[answerCache]] to speed up testing. * - * TODO(marmbrus): Document system properties. + * See the documentation of public vals in this class for information on how test execution can be + * configured using system properties. */ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with GivenWhenThen with Logging { + + /** + * When set, any cache files that result in test failures will be deleted. Used when the test + * harness or hive have been updated thus requiring new golden answers to be computed for some + * tests. Also prevents the classpath being used when looking for golden answers as these are + * usually stale. + */ + val recomputeCache = System.getProperty("shark.hive.recomputeCache") != null + + protected val shardRegEx = "(\\d+):(\\d+)".r + /** + * Allows multiple JVMs to be run in parallel, each responsible for portion of all test cases. + * Format `shardId:numShards`. Shard ids should be zero indexed. E.g. -Dshark.hive.testshard=0:4. + */ + val shardInfo = Option(System.getProperty("shark.hive.shard")).map { + case shardRegEx(id, total) => (id.toInt, total.toInt) + } + protected val targetDir = new File("target") + /** + * When set, this comma separated list is defines directories that contain the names of test cases + * that should be skipped. + * + * For example when `-Dshark.hive.skiptests=passed,hiveFailed` is specified and test cases listed + * in [[passedDirectory]] or [[hiveFailedDirectory]] will be skipped. + */ + val skipDirectories = + Option(System.getProperty("shark.hive.skiptests")) + .toSeq + .flatMap(_.split(",")) + .map(name => new File(targetDir, s"$suiteName.$name")) + + val runOnlyDirectories = + Option(System.getProperty("shark.hive.runonlytests")) + .toSeq + .flatMap(_.split(",")) + .map(name => new File(targetDir, s"$suiteName.$name")) + /** The local directory with cached golden answer will be stored. */ protected val answerCache = new File(targetDir, "comparison-test-cache") if (!answerCache.exists) @@ -27,26 +65,28 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G /** The [[ClassLoader]] that contains test dependencies. Used to look for golden answers. */ protected val testClassLoader = this.getClass.getClassLoader - /** A file where all the test cases that pass are written. Can be used to update the whiteList. */ - val passedFile = new File(targetDir, s"$suiteName.passed") - protected val passedList = new PrintWriter(passedFile) + /** Directory containing a file for each test case that passes. */ + val passedDirectory = new File(targetDir, s"$suiteName.passed") + if(!passedDirectory.exists()) + passedDirectory.mkdir() // Not atomic! - override def afterAll() { - passedList.close() - } + /** Directory containing output of tests that fail to execute with Catalyst. */ + val failedDirectory = new File(targetDir, s"$suiteName.failed") + if(!failedDirectory.exists()) + failedDirectory.mkdir() // Not atomic! - /** - * When `-Dshark.hive.failFast` is set the first test to fail will cause all subsequent tests to - * also fail. - */ - val failFast = System.getProperty("shark.hive.failFast") != null - private var testFailed = false + /** Directory containing output of tests where catalyst produces the wrong answer. */ + val wrongDirectory = new File(targetDir, s"$suiteName.wrong") + if(!wrongDirectory.exists()) + wrongDirectory.mkdir() // Not atomic! - /** - * Delete any cache files that result in test failures. Used when the test harness has been - * updated thus requiring new golden answers to be computed for some tests. - */ - val recomputeCache = System.getProperty("shark.hive.recomputeCache") != null + /** Directory containing output of tests where we fail to generate golden output with Hive. */ + val hiveFailedDirectory = new File(targetDir, s"$suiteName.hiveFailed") + if(!hiveFailedDirectory.exists()) + hiveFailedDirectory.mkdir() // Not atomic! + + /** All directories that contain per-query output files */ + val outputDirectories = Seq(passedDirectory, failedDirectory, wrongDirectory, hiveFailedDirectory) protected val cacheDigest = java.security.MessageDigest.getInstance("MD5") protected def getMd5(str: String): String = { @@ -58,47 +98,105 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G protected def prepareAnswer(sharkQuery: TestShark.type#SharkSqlQuery, answer: Seq[String]): Seq[String] = { val orderedAnswer = sharkQuery.parsed match { // Clean out non-deterministic time schema info. - case _: Command => answer.filterNot(nonDeterministicLine) + case _: Command => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _ => val isOrdered = sharkQuery.executedPlan.collect { case s: Sort => s}.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. if (!isOrdered) answer.sorted else answer } - orderedAnswer.map(cleanPaths).map(clearTimes) + orderedAnswer.map(cleanPaths) } + // TODO: Instead of filtering we should clean to avoid accidentally ignoring actual results. + lazy val nonDeterministicLineIndicators = Seq( + "CreateTime", + "transient_lastDdlTime", + "grantTime", + "lastUpdateTime", + "last_modified_time", + // The following are hive specific schema parameters which we do not need to match exactly. + "numFiles", + "numRows", + "rawDataSize", + "totalSize", + "totalNumberFiles", + "maxFileSize", + "minFileSize" + ) protected def nonDeterministicLine(line: String) = - Seq("CreateTime","transient_lastDdlTime", "grantTime").map(line contains _).reduceLeft(_||_) - - protected def clearTimes(line: String) = - line.replaceAll("\"lastUpdateTime\":\\d+", "") + nonDeterministicLineIndicators.map(line contains _).reduceLeft(_||_) /** - * Removes non-deterministic paths from `str` so cached answers will still pass. + * Removes non-deterministic paths from `str` so cached answers will compare correctly. */ protected def cleanPaths(str: String): String = { str.replaceAll("file:\\/.*\\/", "") } val installHooksCommand = "(?i)SET.*hooks".r - def createQueryTest(testCaseName: String, sql: String) = { + def createQueryTest(testCaseName: String, sql: String) { + // If test sharding is enable, skip tests that are not in the correct shard. + shardInfo.foreach { + case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return + case (shardId, _) => logger.debug(s"Shard $shardId includes test '$testCaseName'") + } + + // Skip tests found in directories specified by user. + skipDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + .foreach(_ => return) + + // If runonlytests is set, skip this test unless we find a file in one of the specified + // directories. + val runIndicators = + runOnlyDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + if(runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { + logger.debug( + s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") + return + } + test(testCaseName) { - if(failFast && testFailed) sys.error("Failing fast due to previous failure") - testFailed = true logger.error( s""" |============================= |HIVE TEST: $testCaseName |============================= """.stripMargin) - val queryList = sql.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq + + // Clear old output for this testcase. + outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) + + val allQueries = sql.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq + + // TODO: DOCUMENT UNSUPPORTED + val queryList = + allQueries + // In hive, setting the hive.outerjoin.supports.filters flag to "false" essentially tells + // 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") + + if(allQueries != queryList) + logger.warn(s"Simplifications made on unsupported operations for test $testCaseName") + + lazy val consoleTestCase = { + val quotes = "\"\"\"" + queryList.zipWithIndex.map { + case (query, i) => + s""" + |val q$i = $quotes$query$quotes.q + |q$i.stringResult() + """.stripMargin + }.mkString("\n== Console version of this test ==\n", "\n", "\n") + } try { TestShark.reset() - if (queryList.filter(installHooksCommand.findAllMatchIn(_).nonEmpty).nonEmpty) - sys.error("hive exec hooks not supported for tests.") - val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" @@ -109,14 +207,19 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G logger.debug(s"Looking for cached answer file $cachedAnswerFile.") if (cachedAnswerFile.exists) { Some(fileToString(cachedAnswerFile)) - } else if (getClass.getClassLoader.getResourceAsStream(cachedAnswerFile.toString) != null) { - Some(resourceToString(cachedAnswerFile.toString, classLoader = testClassLoader)) + } else if ( + getClass.getClassLoader.getResourceAsStream(cachedAnswerFile.toString) != null && + !recomputeCache) { + val answer = resourceToString(cachedAnswerFile.toString, classLoader = testClassLoader) + stringToFile(cachedAnswerFile, answer) + Some(answer) } else { logger.debug(s"File $cachedAnswerFile not found") None } }.map { case "" => Nil + case "\n" => Seq("") case other => other.split("\n").toSeq } @@ -125,49 +228,71 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G logger.warn(s"Using answer cache for test: $testCaseName") hiveCachedResults } else { + val sharkQueries = queryList.map(new TestShark.SharkSqlQuery(_)) - // Make sure we can at least parse everything before doing hive execution. + // Make sure we can at least parse everything before attempting hive execution. sharkQueries.foreach(_.parsed) val computedResults = (queryList.zipWithIndex, sharkQueries,hiveCacheFiles).zipped.map { case ((queryString, i), sharkQuery, cachedAnswerFile)=> - logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") - info(s"HIVE: $queryString") - // Analyze the query with catalyst to ensure test tables are loaded. - val answer = sharkQuery.analyzed match { - case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. - case _ => TestShark.runSqlHive(queryString) + try { + // Hooks often break the harness and don't really affect our test anyway, don't + // even try running them. + if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) + sys.error("hive exec hooks not supported for tests.") + + logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") + info(s"HIVE: $queryString") + // Analyze the query with catalyst to ensure test tables are loaded. + val answer = sharkQuery.analyzed match { + case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. + case _ => TestShark.runSqlHive(queryString) + } + + // We need to add a new line to non-empty answers so we can differentiate Seq() + // from Seq(""). + stringToFile( + cachedAnswerFile, answer.mkString("\n") + (if (answer.nonEmpty) "\n" else "")) + answer + } catch { + case e: Exception => + val errorMessage = + s""" + |Failed to generate golden answer for query: + |Error: ${e.getMessage} + |${stackTraceToString(e)} + |$queryString + |$consoleTestCase + """.stripMargin + stringToFile( + new File(hiveFailedDirectory, testCaseName), + errorMessage + consoleTestCase) + fail(errorMessage) } - - stringToFile(cachedAnswerFile, answer.mkString("\n")) - - answer }.toSeq TestShark.reset() computedResults } - testFailed = false - // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => info(queryString) val query = new TestShark.SharkSqlQuery(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { case e: Exception => - val out = new java.io.ByteArrayOutputStream - val writer = new PrintWriter(out) - e.printStackTrace(writer) - writer.flush() - fail( + val errorMessage = s""" |Failed to execute query using catalyst: |Error: ${e.getMessage} - |${new String(out.toByteArray)} + |${stackTraceToString(e)} |$query |== HIVE - ${hive.size} row(s) == |${hive.mkString("\n")} - """.stripMargin) + | + |$consoleTestCase + """.stripMargin + stringToFile(new File(failedDirectory, testCaseName), errorMessage + consoleTestCase) + fail(errorMessage) } }.toSeq @@ -188,16 +313,20 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G hiveCacheFiles.foreach(_.delete()) } - fail( + val errorMessage = s""" - |Results do not match for query: + |Results do not match for $testCaseName: |$sharkQuery\n${sharkQuery.analyzed.output.map(_.name).mkString("\t")} |$resultComparison - """.stripMargin) + """.stripMargin + + stringToFile(new File(wrongDirectory, testCaseName), errorMessage + consoleTestCase) + fail(errorMessage) } } - passedList.println(s""""$testCaseName",""") + // Touch passed file. + new FileOutputStream(new File(passedDirectory, testCaseName)).close() } catch { case tf: org.scalatest.exceptions.TestFailedException => throw tf case originalException: Exception => diff --git a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala index 4edcacc1131c3..10bf469dadda6 100644 --- a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala +++ b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala @@ -23,11 +23,15 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { def testCases: Seq[(String, File)] - val runAll = !(System.getProperty("shark.hive.alltests") == null) + val runAll = + !(System.getProperty("shark.hive.alltests") == null) || + runOnlyDirectories.nonEmpty || + skipDirectories.nonEmpty val whiteListProperty = "shark.hive.whitelist" // Allow the whiteList to be overridden by a system property - val realWhiteList = Option(System.getProperty(whiteListProperty)).map(_.split(",").toSeq).getOrElse(whiteList) + val realWhiteList = + Option(System.getProperty(whiteListProperty)).map(_.split(",").toSeq).getOrElse(whiteList) // Go through all the test cases and add them to scala test. testCases.foreach { @@ -41,7 +45,7 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { } else { // Only output warnings for the built in whitelist as this clutters the output when the user // trying to execute a single test from the commandline. - if(System.getProperty(whiteListProperty) == null) + if(System.getProperty(whiteListProperty) == null && !runAll) ignore(testCaseName) {} } } From 8351f251e68c6e9e9ceabb423ac8b914a879915f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:50:01 -0800 Subject: [PATCH 474/778] Add an ignored test to remind us we don't do empty aggregations right. --- src/test/scala/catalyst/execution/HiveQueryTests.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala index a694a1d6194c6..b6e3b54eefe51 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -60,4 +60,9 @@ class HiveQueryTests extends HiveComparisonTest { createQueryTest("LIKE", "SELECT * FROM src WHERE value LIKE '%1%'") + + ignore("empty aggregate input") { + createQueryTest("empty aggregate input", + "SELECT SUM(key) FROM (SELECT * FROM src LIMIT 0) a") + } } \ No newline at end of file From 568d15038348f3d37d5e80ec1f22e3b023a5cb16 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:51:32 -0800 Subject: [PATCH 475/778] Updates to white/blacklist. --- .../execution/HiveCompatibility.scala | 57 ++++++++++++------- 1 file changed, 38 insertions(+), 19 deletions(-) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index f2eeebd6c886a..64546fd5cff51 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -13,7 +13,7 @@ class HiveCompatibility extends HiveQueryFileTest { lazy val hiveQueryDir = new File(TestShark.hiveDevHome, "ql/src/test/queries/clientpositive") def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) - /** A list of tests deemed out of scope currently and thus completely disregarded */ + /** A list of tests deemed out of scope currently and thus completely disregarded. */ override def blackList = Seq( // These tests use hooks that are not on the classpath and thus break all subsequent execution. "hook_order", @@ -26,6 +26,8 @@ class HiveCompatibility extends HiveQueryFileTest { "updateAccessTime", "index_compact_binary_search", "bucket_num_reducers", + "column_access_stats", + "concatenate_inherit_table_location", // User specific test answers, breaks the caching mechanism. "authorization_3", @@ -92,18 +94,31 @@ class HiveCompatibility extends HiveQueryFileTest { "alter1", "input16", + // No support for unpublished test udfs. + "autogen_colalias", + // Shark does not support buckets. ".*bucket.*", // No window support yet - ".* window.*", + ".*window.*", + + // Views are not supported + ".*view.*", // Fails in hive with authorization errors. - "alter_rename_partition_authorization" + "alter_rename_partition_authorization", + "authorization.*", + + // Hadoop version specific tests + "archive_corrupt", + + // No support for case sensitivity is resolution using hive properties atm. + "case_sensitivity" ) /** - * The set of tests that are believed to be working in catalyst. Tests not in whiteList + * The set of tests that are believed to be working in catalyst. Tests not on whiteList or * blacklist are implicitly marked as ignored. */ override def whiteList = Seq( @@ -124,7 +139,9 @@ class HiveCompatibility extends HiveQueryFileTest { "auto_join24", "auto_join26", "auto_join28", + "auto_join32", "auto_join_nulls", + "auto_join_reordering_values", "auto_sortmerge_join_1", "auto_sortmerge_join_10", "auto_sortmerge_join_11", @@ -155,11 +172,14 @@ class HiveCompatibility extends HiveQueryFileTest { "create_like2", "create_like_tbl_props", "create_view_translate", + "create_skewed_table1", "ct_case_insensitive", + "database_location", "database_properties", "default_partition_name", "delimiter", "desc_non_existent_tbl", + "describe_comment_indent", "describe_database_json", "describe_table_json", "describe_formatted_view_partitioned", @@ -177,9 +197,10 @@ class HiveCompatibility extends HiveQueryFileTest { "drop_table", "drop_table2", "drop_view", + "escape_clusterby1", + "escape_distributeby1", "escape_orderby1", "escape_sortby1", - "filter_join_breaktask", "groupby1", "groupby1_map", "groupby1_map_nomap", @@ -202,7 +223,6 @@ class HiveCompatibility extends HiveQueryFileTest { "groupby_multi_single_reducer2", "groupby_mutli_insert_common_distinct", "groupby_sort_6", - "groupby_sort_8", "groupby_sort_test_1", "implicit_cast1", "index_auto_self_join", @@ -215,6 +235,9 @@ class HiveCompatibility extends HiveQueryFileTest { "index_auto_multiple", "index_bitmap_compression", "index_compression", + "index_auto_empty", + "index_auto_partitioned", + "index_bitmap_auto_partitioned", "innerjoin", "inoutdriver", "input", @@ -226,7 +249,6 @@ class HiveCompatibility extends HiveQueryFileTest { "input23", "input24", "input25", - "input28", "input2_limit", "input41", "input4_cb_delim", @@ -238,6 +260,8 @@ class HiveCompatibility extends HiveQueryFileTest { "input_limit", "input_part1", "input_part2", + "input_part4", + "input_part6", "inputddl4", "inputddl7", "inputddl8", @@ -306,8 +330,6 @@ class HiveCompatibility extends HiveQueryFileTest { "mapjoin_subquery2", "mapjoin_test_outer", "mapreduce3", - "merge1", - "merge2", "mergejoins", "mergejoins_mixed", "misc_json", @@ -330,7 +352,7 @@ class HiveCompatibility extends HiveQueryFileTest { "outer_join_ppr", "part_inherit_tbl_props", "part_inherit_tbl_props_empty", - "partition_schema1", + "part_inherit_tbl_props_with_star", "partitions_json", "plan_json", "ppd1", @@ -351,19 +373,17 @@ class HiveCompatibility extends HiveQueryFileTest { "ppd_union", "progress_1", "protectmode", - "push_or", "query_with_semi", "quote2", "rename_column", + "reduce_deduplicate_exclude_join", "router_join_ppr", "select_as_omitted", - "select_unquote_and", - "select_unquote_not", - "select_unquote_or", "serde_reported_schema", "set_variable_sub", "show_describe_func_quotes", "show_functions", + "show_partitions", "skewjoinopt13", "skewjoinopt18", "skewjoinopt9", @@ -373,15 +393,11 @@ class HiveCompatibility extends HiveQueryFileTest { "smb_mapjoin_15", "smb_mapjoin_16", "smb_mapjoin_17", - "smb_mapjoin_21", "sort", "sort_merge_join_desc_1", "sort_merge_join_desc_2", "sort_merge_join_desc_3", "sort_merge_join_desc_4", - "sort_merge_join_desc_5", - "sort_merge_join_desc_6", - "sort_merge_join_desc_7", "subq2", "tablename_with_select", "udf2", @@ -429,6 +445,7 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_int", "udf_isnotnull", "udf_isnull", + "udf_java_method", "udf_lcase", "udf_length", "udf_lessthan", @@ -452,12 +469,12 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_positive", "udf_pow", "udf_power", + "udf_radians", "udf_rand", "udf_regexp_extract", "udf_regexp_replace", "udf_repeat", "udf_rlike", - "udf_round", "udf_rpad", "udf_rtrim", "udf_second", @@ -494,6 +511,8 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_xpath_int", "udf_xpath_long", "udf_xpath_short", + "udf_xpath_string", + "unicode_notation", "union10", "union11", "union13", From a3003ae6bea3b4d7c12fb04b97737956a28d9d8b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:51:49 -0800 Subject: [PATCH 476/778] Update makefile to use better sharding support. --- Makefile | 77 ++++++++++++++------------------------------------------ 1 file changed, 19 insertions(+), 58 deletions(-) diff --git a/Makefile b/Makefile index 8c66f1833f588..b0cf880fef001 100644 --- a/Makefile +++ b/Makefile @@ -1,60 +1,21 @@ -all: a b c d e f g h i j k l m n o p q r s t u v w x y" z +all: s1 s2 s3 s4 s5 s6 s7 s8 -buildWhiteList: - sbt -Dshark.hive.alltests "test-only catalyst.execution.HiveCompatibility" +compile: + sbt test:compile -findBroken: - sbt -Dshark.hive.alltests -Dshark.hive.failFast "test-only catalyst.execution.HiveCompatibility" - -a: - sbt -Dshark.hive.whitelist=a.* "test-only catalyst.execution.HiveCompatibility" -b: - sbt -Dshark.hive.whitelist=b.* "test-only catalyst.execution.HiveCompatibility" -c: - sbt -Dshark.hive.whitelist=c.* "test-only catalyst.execution.HiveCompatibility" -d: - sbt -Dshark.hive.whitelist=d.* "test-only catalyst.execution.HiveCompatibility" -e: - sbt -Dshark.hive.whitelist=e.* "test-only catalyst.execution.HiveCompatibility" -f: - sbt -Dshark.hive.whitelist=f.* "test-only catalyst.execution.HiveCompatibility" -g: - sbt -Dshark.hive.whitelist=g.* "test-only catalyst.execution.HiveCompatibility" -h: - sbt -Dshark.hive.whitelist=h.* "test-only catalyst.execution.HiveCompatibility" -i: - sbt -Dshark.hive.whitelist=i.* "test-only catalyst.execution.HiveCompatibility" -j: - sbt -Dshark.hive.whitelist=j.* "test-only catalyst.execution.HiveCompatibility" -k: - sbt -Dshark.hive.whitelist=k.* "test-only catalyst.execution.HiveCompatibility" -l: - sbt -Dshark.hive.whitelist=l.* "test-only catalyst.execution.HiveCompatibility" -m: - sbt -Dshark.hive.whitelist=m.* "test-only catalyst.execution.HiveCompatibility" -n: - sbt -Dshark.hive.whitelist=n.* "test-only catalyst.execution.HiveCompatibility" -o: - sbt -Dshark.hive.whitelist=o.* "test-only catalyst.execution.HiveCompatibility" -p: - sbt -Dshark.hive.whitelist=p.* "test-only catalyst.execution.HiveCompatibility" -q: - sbt -Dshark.hive.whitelist=q.* "test-only catalyst.execution.HiveCompatibility" -r: - sbt -Dshark.hive.whitelist=r.* "test-only catalyst.execution.HiveCompatibility" -s: - sbt -Dshark.hive.whitelist=s.* "test-only catalyst.execution.HiveCompatibility" -t: - sbt -Dshark.hive.whitelist=t.* "test-only catalyst.execution.HiveCompatibility" -u: - sbt -Dshark.hive.whitelist=u.* "test-only catalyst.execution.HiveCompatibility" -v: - sbt -Dshark.hive.whitelist=v.* "test-only catalyst.execution.HiveCompatibility" -w: - sbt -Dshark.hive.whitelist=w.* "test-only catalyst.execution.HiveCompatibility" -x: - sbt -Dshark.hive.whitelist=x.* "test-only catalyst.execution.HiveCompatibility" -y: - sbt -Dshark.hive.whitelist=y.* "test-only catalyst.execution.HiveCompatibility" -z: - sbt -Dshark.hive.whitelist=z.* "test-only catalyst.execution.HiveCompatibility" +s1: compile + sbt ${ARGS} -Dshark.hive.shard=0:8 "test-only catalyst.execution.HiveCompatibility" +s2: compile + sbt ${ARGS} -Dshark.hive.shard=1:8 "test-only catalyst.execution.HiveCompatibility" +s3: compile + sbt ${ARGS} -Dshark.hive.shard=2:8 "test-only catalyst.execution.HiveCompatibility" +s4: compile + sbt ${ARGS} -Dshark.hive.shard=3:8 "test-only catalyst.execution.HiveCompatibility" +s5: compile + sbt ${ARGS} -Dshark.hive.shard=4:8 "test-only catalyst.execution.HiveCompatibility" +s6: compile + sbt ${ARGS} -Dshark.hive.shard=5:8 "test-only catalyst.execution.HiveCompatibility" +s7: compile + sbt ${ARGS} -Dshark.hive.shard=6:8 "test-only catalyst.execution.HiveCompatibility" +s8: compile + sbt ${ARGS} -Dshark.hive.shard=7:8 "test-only catalyst.execution.HiveCompatibility" From f71a3252be1fccad3d9c902eadd69e661961119c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 15:54:22 -0800 Subject: [PATCH 477/778] Update golden jar. --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index e660fb5bbf584..2fe38ceb72ac8 100644 --- a/build.sbt +++ b/build.sbt @@ -14,7 +14,7 @@ resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/ libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating-SNAPSHOT" -libraryDependencies += "catalyst" % "hive-golden" % "3" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden3.jar" +libraryDependencies += "catalyst" % "hive-golden" % "4" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden4.jar" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" From aa2239c8ae53a5aeadf2ab087dc84015554e218b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 17:01:27 -0800 Subject: [PATCH 478/778] filter test lines containing Owner: --- src/test/scala/catalyst/execution/HiveComparisonTest.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index 61a9f98c960a9..e829de47ff31c 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -114,6 +114,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G "grantTime", "lastUpdateTime", "last_modified_time", + "Owner:", // The following are hive specific schema parameters which we do not need to match exactly. "numFiles", "numRows", From ffa9f25f752addb689143060c90a5fbdbb836482 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 15 Jan 2014 17:01:44 -0800 Subject: [PATCH 479/778] blacklist some more MR tests. --- src/test/scala/catalyst/execution/HiveCompatibility.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index 64546fd5cff51..cbb2706c8dee3 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -57,6 +57,9 @@ class HiveCompatibility extends HiveQueryFileTest { "index_auto_mult_tables", "index_auto_file_format", "index_auth", + "index_auto_empty", + "index_auto_partitioned", + "index_bitmap_auto_partitioned", // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 From 058ec150ca43e72f10f96848bf72ed66bd8ee14b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 10:05:02 -0800 Subject: [PATCH 480/778] handle more writeables. --- src/main/scala/catalyst/execution/FunctionRegistry.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index 7efc08629a99b..fe76314a7dcf2 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -87,7 +87,9 @@ abstract class HiveUdf extends Expression with ImplementedUdf with Logging { case l: LongWritable => l.get case d: DoubleWritable => d.get() case d: org.apache.hadoop.hive.serde2.io.DoubleWritable => d.get + case s: org.apache.hadoop.hive.serde2.io.ShortWritable => s.get case b: BooleanWritable => b.get() + case b: org.apache.hadoop.hive.serde2.io.ByteWritable => b.get case list: java.util.List[_] => list.map(unwrap) case p: java.lang.Short => p case p: java.lang.Long => p From ccc4dbf58d56677d2b2a6e5ba915576fa2556170 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 10:05:22 -0800 Subject: [PATCH 481/778] Add optimization rule to simplify casts. --- src/main/scala/catalyst/optimizer/Optimizer.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala index 819c662a5185e..2c04db435f8d5 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/src/main/scala/catalyst/optimizer/Optimizer.scala @@ -12,8 +12,8 @@ object Optimize extends RuleExecutor[LogicalPlan] { EliminateSubqueries) :: Batch("ConstantFolding", Once, ConstantFolding, - BooleanSimplification - ) :: Nil + BooleanSimplification, + SimplifyCasts) :: Nil } /** @@ -68,4 +68,13 @@ object BooleanSimplification extends Rule[LogicalPlan] { } } } +} + +/** + * Removes casts that are unnecessary because the input is already the correct type. + */ +object SimplifyCasts extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case Cast(e, dataType) if e.dataType == dataType => e + } } \ No newline at end of file From 6d2924d56ecef7618aaf0374672e29a79ff1cb0d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 10:06:05 -0800 Subject: [PATCH 482/778] add support for If. Not integrated in HiveQL yet. --- src/main/scala/catalyst/expressions/Evaluate.scala | 3 +++ .../scala/catalyst/expressions/predicates.scala | 14 ++++++++++++++ 2 files changed, 17 insertions(+) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index cb340004f0a84..0b3737578b56c 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -149,6 +149,9 @@ object Evaluate extends Logging { case Remainder(l, r) => i2(l,r, _.rem(_, _)) case UnaryMinus(child) => n1(child, _.negate(_)) + /* Control Flow */ + case If(e, t, f) => if(eval(e).asInstanceOf[Boolean]) eval(t) else eval(f) + /* Comparisons */ case Equals(l, r) => val left = eval(l) diff --git a/src/main/scala/catalyst/expressions/predicates.scala b/src/main/scala/catalyst/expressions/predicates.scala index dc4b4a69b017e..c5723e07062a5 100644 --- a/src/main/scala/catalyst/expressions/predicates.scala +++ b/src/main/scala/catalyst/expressions/predicates.scala @@ -2,6 +2,7 @@ package catalyst package expressions import types._ +import catalyst.analysis.UnresolvedException trait Predicate extends Expression { self: Product => @@ -74,3 +75,16 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E override def foldable = child.foldable def nullable = false } + +case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) extends Expression { + def children = predicate :: trueValue :: falseValue :: Nil + def nullable = children.exists(_.nullable) + def references = children.flatMap(_.references).toSet + override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType + def dataType = { + if (!resolved) + throw new UnresolvedException( + this, s"datatype. Can not resolve due to differing types ${trueValue.dataType}, ${falseValue.dataType}") + trueValue.dataType + } +} From f0e01619d488c5c3a729cd6cadd9ee389f31c95d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 11:46:39 -0800 Subject: [PATCH 483/778] Move numeric types into DataTypes simplifying evaluator. This can probably also be use for codegen... --- .../scala/catalyst/expressions/Evaluate.scala | 74 ++++---------- src/main/scala/catalyst/types/dataTypes.scala | 99 ++++++++++++++++--- 2 files changed, 105 insertions(+), 68 deletions(-) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 0b3737578b56c..ff8ef3605f5e1 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -25,24 +25,8 @@ object Evaluate extends Logging { null else e.dataType match { - case IntegerType => - f.asInstanceOf[(Numeric[Int], Int) => Int]( - implicitly[Numeric[Int]], eval(e).asInstanceOf[Int]) - case DoubleType => - f.asInstanceOf[(Numeric[Double], Double) => Double]( - implicitly[Numeric[Double]], eval(e).asInstanceOf[Double]) - case LongType => - f.asInstanceOf[(Numeric[Long], Long) => Long]( - implicitly[Numeric[Long]], eval(e).asInstanceOf[Long]) - case FloatType => - f.asInstanceOf[(Numeric[Float], Float) => Float]( - implicitly[Numeric[Float]], eval(e).asInstanceOf[Float]) - case ByteType => - f.asInstanceOf[(Numeric[Byte], Byte) => Byte]( - implicitly[Numeric[Byte]], eval(e).asInstanceOf[Byte]) - case ShortType => - f.asInstanceOf[(Numeric[Short], Short) => Short]( - implicitly[Numeric[Short]], eval(e).asInstanceOf[Short]) + case n: NumericType => + f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType](n.numeric, eval(e).asInstanceOf[n.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } } @@ -58,24 +42,9 @@ object Evaluate extends Logging { null else e1.dataType match { - case IntegerType => - f.asInstanceOf[(Numeric[Int], Int, Int) => Int]( - implicitly[Numeric[Int]], evalE1.asInstanceOf[Int], evalE2.asInstanceOf[Int]) - case DoubleType => - f.asInstanceOf[(Numeric[Double], Double, Double) => Double]( - implicitly[Numeric[Double]], evalE1.asInstanceOf[Double], evalE2.asInstanceOf[Double]) - case LongType => - f.asInstanceOf[(Numeric[Long], Long, Long) => Long]( - implicitly[Numeric[Long]], evalE1.asInstanceOf[Long], evalE2.asInstanceOf[Long]) - case FloatType => - f.asInstanceOf[(Numeric[Float], Float, Float) => Float]( - implicitly[Numeric[Float]], evalE1.asInstanceOf[Float], evalE2.asInstanceOf[Float]) - case ByteType => - f.asInstanceOf[(Numeric[Byte], Byte, Byte) => Byte]( - implicitly[Numeric[Byte]], evalE1.asInstanceOf[Byte], evalE2.asInstanceOf[Byte]) - case ShortType => - f.asInstanceOf[(Numeric[Short], Short, Short) => Short]( - implicitly[Numeric[Short]], evalE1.asInstanceOf[Short], evalE2.asInstanceOf[Short]) + case n: NumericType => + f.asInstanceOf[(Numeric[n.JvmType], n.JvmType, n.JvmType) => Int]( + n.numeric, evalE1.asInstanceOf[n.JvmType], evalE2.asInstanceOf[n.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } } @@ -91,12 +60,9 @@ object Evaluate extends Logging { null else e1.dataType match { - case DoubleType => - f.asInstanceOf[(Fractional[Double], Double, Double) => Double]( - implicitly[Fractional[Double]], evalE1.asInstanceOf[Double], evalE2.asInstanceOf[Double]) - case FloatType => - f.asInstanceOf[(Fractional[Float], Float, Float) => Float]( - implicitly[Fractional[Float]], evalE1.asInstanceOf[Float], evalE2.asInstanceOf[Float]) + case f: FractionalType => + f.asInstanceOf[(Fractional[f.JvmType], f.JvmType, f.JvmType) => f.JvmType]( + f.fractional, evalE1.asInstanceOf[f.JvmType], evalE2.asInstanceOf[f.JvmType]) case other => sys.error(s"Type $other does not support fractional operations") } } @@ -110,24 +76,20 @@ object Evaluate extends Logging { null else e1.dataType match { - case IntegerType => - f.asInstanceOf[(Integral[Int], Int, Int) => Int]( - implicitly[Integral[Int]], evalE1.asInstanceOf[Int], evalE2.asInstanceOf[Int]) - case LongType => - f.asInstanceOf[(Integral[Long], Long, Long) => Long]( - implicitly[Integral[Long]], evalE1.asInstanceOf[Long], evalE2.asInstanceOf[Long]) - case ByteType => - f.asInstanceOf[(Integral[Byte], Byte, Byte) => Byte]( - implicitly[Integral[Byte]], evalE1.asInstanceOf[Byte], evalE2.asInstanceOf[Byte]) - case ShortType => - f.asInstanceOf[(Integral[Short], Short, Short) => Short]( - implicitly[Integral[Short]], evalE1.asInstanceOf[Short], evalE2.asInstanceOf[Short]) + case i: IntegralType => + f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( + i.integral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } } - @inline def castOrNull[A](f: => A) = - try f catch { case _: java.lang.NumberFormatException => null } + @inline def castOrNull[A](e: Expression, f: String => A) = + try { + eval(e) match { + case null => null + case s: String => f(s) + } + } catch { case _: java.lang.NumberFormatException => null } val result = e match { case Literal(v, _) => v diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 4b23a523df01b..49bbf88bec7c8 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -1,22 +1,97 @@ package catalyst package types -class DataType +import expressions.Expression -case object StringType extends DataType -case object BinaryType extends DataType +abstract class DataType { + /** Matches any expression that evaluates to this [[DataType]] */ + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType == this => true + case _ => false + } +} -case object IntegerType extends DataType -case object BooleanType extends DataType -case object FloatType extends DataType -case object DoubleType extends DataType -case object LongType extends DataType -case object ByteType extends DataType -case object ShortType extends DataType +case object NullType extends DataType -case object DecimalType extends DataType +abstract class NativeType extends DataType { type JvmType } +case object StringType extends NativeType { + type JvmType = String +} +case object BinaryType extends NativeType { + type JvmType = Array[Byte] +} +case object BooleanType extends NativeType { + type JvmType = Boolean +} -case object NullType extends DataType +abstract class NumericType extends NativeType { + // Unfortunately we can't get this implicitly as that breaks Spark Serialization. + val numeric: Numeric[JvmType] +} + +/** Matcher for any expressions that evaluate to [[IntegralType]]s */ +object IntegralType { + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType.isInstanceOf[IntegralType] => true + case _ => false + } +} +abstract class IntegralType extends NumericType { + val integral: Integral[JvmType] +} + +case object LongType extends IntegralType { + type JvmType = Long + val numeric = implicitly[Numeric[Long]] + val integral = implicitly[Integral[Long]] +} + +case object IntegerType extends IntegralType { + type JvmType = Int + val numeric = implicitly[Numeric[Int]] + val integral = implicitly[Integral[Int]] +} + +case object ShortType extends IntegralType { + type JvmType = Short + val numeric = implicitly[Numeric[Short]] + val integral = implicitly[Integral[Short]] +} + +case object ByteType extends IntegralType { + type JvmType = Byte + val numeric = implicitly[Numeric[Byte]] + val integral = implicitly[Integral[Byte]] +} + +/** Matcher for any expressions that evaluate to [[FractionalType]]s */ +object FractionalType { + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType.isInstanceOf[FractionalType] => true + case _ => false + } +} +abstract class FractionalType extends NumericType { + val fractional: Fractional[JvmType] +} + +case object DecimalType extends FractionalType { + type JvmType = BigDecimal + val numeric = implicitly[Numeric[BigDecimal]] + val fractional = implicitly[Fractional[BigDecimal]] +} + +case object DoubleType extends FractionalType { + type JvmType = Double + val numeric = implicitly[Numeric[Double]] + val fractional = implicitly[Fractional[Double]] +} + +case object FloatType extends FractionalType { + type JvmType = Float + val numeric = implicitly[Numeric[Float]] + val fractional = implicitly[Fractional[Float]] +} case class ArrayType(elementType: DataType) extends DataType From 5a262927f243fea71d9fdab081b3554526e1a16e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 11:48:37 -0800 Subject: [PATCH 484/778] Rules to bring casting more inline with Hive semantics. --- .../scala/catalyst/analysis/Analyzer.scala | 2 ++ .../catalyst/analysis/typeCoercion.scala | 30 +++++++++++++++++-- .../scala/catalyst/expressions/Evaluate.scala | 24 +++++++-------- 3 files changed, 40 insertions(+), 16 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 211b8281a0bb7..ae403e6bcf5db 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -36,6 +36,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool Batch("Aggregation", Once, GlobalAggregates), Batch("Type Coersion", fixedPoint, + StringToIntegralCasts, + BooleanCasts, PromoteNumericTypes, PromoteStrings, ConvertNaNs, diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index 612008b31ba68..780b83e535ba1 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -151,6 +151,30 @@ object BooleanComparisons extends Rule[LogicalPlan] { } } +/** + * Casts to/from [[BooleanType]] are transformed into comparisons since the JVM does not consider + * Booleans to be numeric types. + */ +object BooleanCasts extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case Cast(e, BooleanType) => Not(Equals(e, Literal(0))) + case Cast(e, dataType) if e.dataType == BooleanType => + Cast(If(e, Literal(1), Literal(0)), dataType) + } +} + +/** + * When encountering a cast from a string representing a valid fractional number to an integral type + * the jvm will throw a [[java.lang.NumberFormatException]]. Hive, in contrast, returns the + * truncated version of this number. + */ +object StringToIntegralCasts extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case Cast(e @ StringType(), t: IntegralType) => + Cast(Cast(e, DecimalType), t) + } +} + /** * This ensure that the types for various functions are as expected. Most of these rules are * actually Hive specific. @@ -162,9 +186,9 @@ object FunctionArgumentConversion extends Rule[LogicalPlan] { case e if !e.childrenResolved => e // Promote SUM to largest types to prevent overflows. - // TODO: This is enough to make most of the tests pass, but we really need a full set of our own - // to really ensure compatibility. - case Sum(e) if e.dataType == IntegerType => Sum(Cast(e, LongType)) + case s @ Sum(e @ DecimalType()) => s // Decimal is already the biggest. + case Sum(e @ IntegralType()) if e.dataType != LongType => Sum(Cast(e, LongType)) + case Sum(e @ FractionalType()) if e.dataType != DoubleType => Sum(Cast(e, DoubleType)) } } diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index ff8ef3605f5e1..4943bdb395444 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -104,9 +104,9 @@ object Evaluate extends Logging { case Add(l, r) => n2(l,r, _.plus(_, _)) case Subtract(l, r) => n2(l,r, _.minus(_, _)) case Multiply(l, r) => n2(l,r, _.times(_, _)) - // Divide & remainder implementation are different for fractional and integral dataTypes. - case Divide(l, r) if (l.dataType == DoubleType || l.dataType == FloatType) => f2(l,r, _.div(_, _)) - case Divide(l, r) => i2(l,r, _.quot(_, _)) + // Divide implementation are different for fractional and integral dataTypes. + case Divide(l @ FractionalType(), r) => f2(l,r, _.div(_, _)) + case Divide(l @ IntegralType(), r) => i2(l,r, _.quot(_, _)) // Remainder is only allowed on Integral types. case Remainder(l, r) => i2(l,r, _.rem(_, _)) case UnaryMinus(child) => n1(child, _.negate(_)) @@ -162,16 +162,14 @@ object Evaluate extends Logging { } // String => Numeric Types - case Cast(e, IntegerType) if e.dataType == StringType => - eval(e) match { - case null => null - case s: String => castOrNull(s.toInt) - } - case Cast(e, DoubleType) if e.dataType == StringType => - eval(e) match { - case null => null - case s: String => castOrNull(s.toDouble) - } + case Cast(e @ StringType(), IntegerType) => castOrNull(e, _.toInt) + case Cast(e @ StringType(), DoubleType) => castOrNull(e, _.toDouble) + case Cast(e @ StringType(), FloatType) => castOrNull(e, _.toFloat) + case Cast(e @ StringType(), LongType) => castOrNull(e, _.toLong) + case Cast(e @ StringType(), ShortType) => castOrNull(e, _.toShort) + case Cast(e @ StringType(), ByteType) => castOrNull(e, _.toByte) + case Cast(e @ StringType(), DecimalType) => castOrNull(e, BigDecimal(_)) + // Boolean conversions case Cast(e, ByteType) if e.dataType == BooleanType => eval(e) match { From 83adb9d53245a7fec37f3cf7e2d6bb335e71b442 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 9 Jan 2014 23:28:55 -0500 Subject: [PATCH 485/778] add DataProperty Author: Yin Huai --- .../catalyst/execution/DataProperty.scala | 106 ++++++++++++ .../scala/catalyst/execution/Exchange.scala | 160 ++++++++++++++++++ .../catalyst/execution/SharkInstance.scala | 11 +- .../scala/catalyst/execution/SharkPlan.scala | 21 +++ .../scala/catalyst/execution/aggregates.scala | 13 +- .../catalyst/execution/basicOperators.scala | 40 +++-- src/main/scala/catalyst/execution/joins.scala | 17 +- .../execution/planningStrategies.scala | 8 +- src/main/scala/catalyst/plans/QueryPlan.scala | 30 +++- .../spark/rdd/CoGroupedLocallyRDD.scala | 85 ++++++++++ .../spark/rdd/SharkOrderedRDDFunctions.scala | 41 +++++ .../spark/rdd/SharkPairRDDFunctions.scala | 67 ++++++++ .../execution/DataPropertyTests.scala | 154 +++++++++++++++++ .../catalyst/execution/DslQueryTests1.scala | 74 ++++++++ 14 files changed, 798 insertions(+), 29 deletions(-) create mode 100644 src/main/scala/catalyst/execution/DataProperty.scala create mode 100644 src/main/scala/catalyst/execution/Exchange.scala create mode 100644 src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala create mode 100644 src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala create mode 100644 src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala create mode 100644 src/test/scala/catalyst/execution/DataPropertyTests.scala create mode 100644 src/test/scala/catalyst/execution/DslQueryTests1.scala diff --git a/src/main/scala/catalyst/execution/DataProperty.scala b/src/main/scala/catalyst/execution/DataProperty.scala new file mode 100644 index 0000000000000..b8179b97f832c --- /dev/null +++ b/src/main/scala/catalyst/execution/DataProperty.scala @@ -0,0 +1,106 @@ +package catalyst +package execution + +import catalyst.expressions.{SortOrder, Expression} + +trait DataProperty { + def expressions: Seq[Expression] + + /** + * If we need an [[catalyst.execution.Exchange]] to re-partition data for + * the given [[catalyst.execution.DataProperty]] other. + * @param other The given [[catalyst.execution.DataProperty]]. + *@return + */ + // TODO: We should also consider functional dependencies between expressions of + // two data properties. For example, if we have a GroupProperty(a) and a + // GroupProperty(a+1), we will not need an exchange to re-partition the data. + def needExchange(other: DataProperty): Boolean +} + +/** + * An implementation of [[catalyst.execution.DataProperty]] represents that + * the data property of a dataset is not specified. + * If it is used as a required data property for a physical operator + * (a [[catalyst.execution.SharkPlan]]) (e.g. [[catalyst.execution.Project]]), + * it means that this operator does not require its input datasets to be + * organized in a certain way. + */ +case class NotSpecifiedProperty() extends DataProperty { + def expressions = Nil + + def needExchange(other: DataProperty): Boolean = { + other match { + case NotSpecifiedProperty() => false + case GroupProperty(groupingExpressions) => true + case SortProperty(_) => true + } + } +} + +/** + * An implementation of [[catalyst.execution.DataProperty]] represents that + * a dataset is grouped by groupingExpressions. + * @param groupingExpressions The expressions used to specify the way how rows should be grouped. + * If it is a Nil, the entire dataset is considered as a single group. + * In this case, a single reducer will be used. + */ +case class GroupProperty(groupingExpressions: Seq[Expression]) extends DataProperty { + override val expressions = groupingExpressions + + def needExchange(other: DataProperty): Boolean = { + other match { + case NotSpecifiedProperty() => false + // We do not need an Exchange operator if another GroupProperty only + // needs to group rows within a partition. + case g @ GroupProperty(otherExpressions) => { + if (expressions.toSet.subsetOf(otherExpressions.toSet)) false else true + } + // Because we use [[org.apache.spark.HashPartitioner]] for GroupProperty, + // we need to use an Exchange operator to sort data with a + // [[org.apache.spark.RangePartitioner]]. But, if the groupingExpressions is + // a Nil, we will have a single partition. So, we do not need an Exchange operator + // to sort this single partition. + case SortProperty(otherExpressions) => if (expressions == Nil) false else true + } + } +} + +/** + * An implementation of [[catalyst.execution.DataProperty]] represents that + * a dataset is sorted by sortingExpressions. A SortProperty also implies that + * the dataset is grouped by sortingExpressions. + * @param sortingExpressions The expressions used to specify the way how rows should be sorted. + * sortingExpressions should not be empty. + */ +case class SortProperty(sortingExpressions: Seq[SortOrder]) extends DataProperty { + + { + if (sortingExpressions == Nil) { + throw new IllegalArgumentException("Sorting expressions of a SortProperty " + + "are not specified.") + } + } + + def expressions = sortingExpressions + + def needExchange(other: DataProperty): Boolean = { + other match { + case NotSpecifiedProperty() => false + // A SortProperty implies a GroupProperty. We do not need an Exchange operator + // if the GroupProperty only needs to group rows within a partition. + case g @ GroupProperty(otherExpressions) => { + if (expressions.map(expression => expression.child). + toSet.subsetOf(otherExpressions.toSet)) false else true + } + // We do not need an Exchange operator if another SortProperty only needs to + // sort rows within a partition (cases satisfying otherExpressions.startsWith(expressions)) + // or we do not need to sort again (cases satisfying + // expressions.startsWith(otherExpressions)). + case s @ SortProperty(otherExpressions) => { + if (otherExpressions.startsWith(expressions) || + expressions.startsWith(otherExpressions)) false else true + } + } + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala new file mode 100644 index 0000000000000..373684eb61ee8 --- /dev/null +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -0,0 +1,160 @@ +package catalyst +package execution + +import catalyst.rules.Rule +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.types._ + +import org.apache.spark.{RangePartitioner, HashPartitioner} +import org.apache.spark.rdd.ShuffledRDD + +case class Exchange( + dataProperty: DataProperty, + child: SharkPlan, + numPartitions: Int = 8) extends UnaryNode { + + def output = child.output + + def execute() = attachTree(this , "execute") { + dataProperty match { + case NotSpecifiedProperty() => child.execute() + case g @ GroupProperty(groupingExpressions) => { + val rdd = child.execute().map { row => + (buildRow(groupingExpressions.toSeq.map(Evaluate(_, Vector(row)))), row) + } + val part = new HashPartitioner(numPartitions) + val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) + + shuffled.map(_._2) + } + case s @ SortProperty(sortingExpressions) => { + val directions = sortingExpressions.map(_.direction).toIndexedSeq + val dataTypes = sortingExpressions.map(_.dataType).toIndexedSeq + + class SortKey(val keyValues: IndexedSeq[Any]) + extends Ordered[SortKey] + with Serializable { + def compare(other: SortKey): Int = { + var i = 0 + while (i < keyValues.size) { + val left = keyValues(i) + val right = other.keyValues(i) + val curDirection = directions(i) + val curDataType = dataTypes(i) + + logger.debug(s"Comparing $left, $right as $curDataType order $curDirection") + // TODO: Use numeric here too? + val comparison = + if (left == null && right == null) { + 0 + } else if (left == null) { + if (curDirection == Ascending) -1 else 1 + } else if (right == null) { + if (curDirection == Ascending) 1 else -1 + } else if (curDataType == IntegerType) { + if (curDirection == Ascending) { + left.asInstanceOf[Int] compare right.asInstanceOf[Int] + } else { + right.asInstanceOf[Int] compare left.asInstanceOf[Int] + } + } else if (curDataType == DoubleType) { + if (curDirection == Ascending) { + left.asInstanceOf[Double] compare right.asInstanceOf[Double] + } else { + right.asInstanceOf[Double] compare left.asInstanceOf[Double] + } + } else if (curDataType == LongType) { + if (curDirection == Ascending) { + left.asInstanceOf[Long] compare right.asInstanceOf[Long] + } else { + right.asInstanceOf[Long] compare left.asInstanceOf[Long] + } + } else if (curDataType == StringType) { + if (curDirection == Ascending) { + left.asInstanceOf[String] compare right.asInstanceOf[String] + } else { + right.asInstanceOf[String] compare left.asInstanceOf[String] + } + } else { + sys.error(s"Comparison not yet implemented for: $curDataType") + } + + if (comparison != 0) return comparison + i += 1 + } + return 0 + } + } + + val rdd = child.execute().map { row => + val input = Vector(row) + val sortKey = new SortKey( + sortingExpressions.map(s => Evaluate(s.child, input)).toIndexedSeq) + + (sortKey, row) + } + val part = new RangePartitioner(numPartitions, rdd, ascending = true) + val shuffled = new ShuffledRDD[SortKey, Row, (SortKey, Row)](rdd, part) + + shuffled.map(_._2) + } + } + } +} + +object AddExchange extends Rule[SharkPlan] { + def apply(plan: SharkPlan): SharkPlan = { + // TODO: determine the number of partitions. + // TODO: We need to consider the number of partitions to determine if we + // will add an Exchange operator. If a dataset only has a single partition, + // even if needExchange returns true, we do not need to shuffle the data again. + val numPartitions = 8 + plan.transformUp { + case aggregate @ Aggregate( + groupingExpressions, + aggregateExpressions, + child) => { + if (child.outputDataProperty.needExchange(aggregate.requiredDataProperty)) { + val exchange = new Exchange(aggregate.requiredDataProperty, child, numPartitions) + + Aggregate(groupingExpressions, aggregateExpressions, exchange)() + } else { + aggregate + } + } + case equiInnerJoin @ SparkEquiInnerJoin( + leftKeys, + rightKeys, + left, + right) => { + val newLeft = { + if (left.outputDataProperty.needExchange(equiInnerJoin.leftRequiredDataProperty)) { + new Exchange(equiInnerJoin.leftRequiredDataProperty, left, numPartitions) + } else { + left + } + } + + val newRight = { + if (right.outputDataProperty.needExchange(equiInnerJoin.rightRequiredDataProperty)) { + new Exchange(equiInnerJoin.rightRequiredDataProperty, right, numPartitions) + } else { + right + } + } + + SparkEquiInnerJoin(leftKeys, rightKeys, newLeft, newRight)() + } + case sort @ Sort(sortExprs, child) => { + if (child.outputDataProperty.needExchange(sort.requiredDataProperty)) { + val exchange = new Exchange(sort.requiredDataProperty, child, numPartitions) + + Sort(sortExprs, exchange)() + } else { + sort + } + } + } + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index c7d4eba52751f..1576d81cb6f56 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -82,7 +82,16 @@ abstract class SharkInstance extends Logging { } object PrepareForExecution extends RuleExecutor[SharkPlan] { - val batches = Batch("Prepare Expressions", Once, expressions.BindReferences) :: Nil + // TODO: [[catalyst.execution.AddExchange]] should be in a phase before + // PrepareForExecution. We may also need a phase of physical optimizations. + // In this phase, we can alter the data property in an Exchange operator to + // reduce the number of shuffling phases. Or, this physical optimization phase can be + // a part of QueryPlanner when a planner can accept a physical operator (a SharkPlan). + val batches = + Batch("Add exchange", Once, + AddExchange) :: + Batch("Prepare Expressions", Once, + expressions.BindReferences) :: Nil } class SharkSqlQuery(sql: String) extends SharkQuery { diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 6116d8397b090..8566354b57f65 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -4,10 +4,14 @@ package execution import org.apache.spark.rdd.RDD import catalyst.plans.QueryPlan +import scala.reflect.ClassTag abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => + def requiredDataProperties: Seq[DataProperty] + def outputDataProperty: DataProperty + /** * Runs this query returning the result as an RDD. */ @@ -23,12 +27,29 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { self: Product => + + def requiredDataProperty: DataProperty = NotSpecifiedProperty() + def requiredDataProperties: Seq[DataProperty] = Seq(requiredDataProperty) + // TODO: We should get the output data properties of a leaf node from metadata. + def outputDataProperty: DataProperty = NotSpecifiedProperty() } trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { self: Product => + + def requiredDataProperty: DataProperty = NotSpecifiedProperty() + def requiredDataProperties: Seq[DataProperty] = Seq(requiredDataProperty) + + def outputDataProperty: DataProperty = child.outputDataProperty } trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { self: Product => + + def leftRequiredDataProperty: DataProperty = NotSpecifiedProperty() + def rightRequiredDataProperty: DataProperty = NotSpecifiedProperty() + def requiredDataProperties: Seq[DataProperty] = + Seq(leftRequiredDataProperty, rightRequiredDataProperty) + + def outputDataProperty: DataProperty = NotSpecifiedProperty() } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 81ccaafd10d24..73920b0f9b243 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,16 +3,23 @@ package execution import catalyst.errors._ import catalyst.expressions._ +import org.apache.spark.rdd.SharkPairRDDFunctions /* Implicits */ import org.apache.spark.SparkContext._ +import SharkPairRDDFunctions._ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], child: SharkPlan) + (override val outputDataProperty: DataProperty = + GroupProperty(groupingExpressions)) extends UnaryNode { + override val requiredDataProperty: DataProperty = GroupProperty(groupingExpressions) + override def otherCopyArgs = outputDataProperty :: Nil + case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -86,9 +93,12 @@ case class Aggregate( def output = aggregateExpressions.map(_.toAttribute) def execute() = attachTree(this, "execute") { + // TODO: If the child of it is an [[catalyst.execution.Exchange]], + // do not evaluate the groupingExpressions again since we have evaluated it + // in the [[catalyst.execution.Exchange]]. val grouped = child.execute().map { row => (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row) - }.groupByKey() + }.groupByKeyLocally() grouped.map { case (group, rows) => // Replace all aggregate expressions with spark functions that will compute the result. @@ -142,6 +152,7 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan (@transient sc: SharkContext) extends UnaryNode { def output = aggregateExprs.map(_.toAttribute) override def otherCopyArgs = Seq(sc) + override val requiredDataProperty: DataProperty = GroupProperty(Nil) case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 75f5808618df7..a0b1874e03991 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -1,12 +1,12 @@ package catalyst package execution -import errors._ -import expressions._ -import types._ - -import org.apache.spark.SparkContext._ +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.types._ +/* Implicits */ +import org.apache.spark.rdd.SharkOrderedRDDFunctions._ case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -45,12 +45,21 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { +case class Sort( + sortExprs: Seq[SortOrder], + child: SharkPlan) + (override val outputDataProperty: DataProperty = + SortProperty(sortExprs)) extends UnaryNode { val numPartitions = 1 // TODO: Set with input cardinality + override val requiredDataProperty: DataProperty = SortProperty(sortExprs) + override def otherCopyArgs = outputDataProperty :: Nil + private final val directions = sortExprs.map(_.direction).toIndexedSeq private final val dataTypes = sortExprs.map(_.dataType).toIndexedSeq + // TODO: This SortKey and the one in [[catalyst.execute.Exchange]] are the same. + // We should just define it only once. private class SortKey(val keyValues: IndexedSeq[Any]) extends Ordered[SortKey] with Serializable { def compare(other: SortKey): Int = { var i = 0 @@ -81,12 +90,6 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { } else { right.asInstanceOf[Double] compare left.asInstanceOf[Double] } - } else if (curDataType == LongType) { - if (curDirection == Ascending) { - left.asInstanceOf[Long] compare right.asInstanceOf[Long] - } else { - right.asInstanceOf[Long] compare left.asInstanceOf[Long] - } } else if (curDataType == StringType) { if (curDirection == Ascending) { left.asInstanceOf[String] compare right.asInstanceOf[String] @@ -106,12 +109,16 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { // TODO: Don't include redundant expressions in both sortKey and row. def execute() = attachTree(this, "sort") { + // TODO: If the child of it is an [[catalyst.execution.Exchange]], + // do not evaluate the sortKey again since we have evaluated it + // in the [[catalyst.execution.Exchange]]. child.execute().map { row => val input = Vector(row) - val sortKey = new SortKey(sortExprs.map(s => Evaluate(s.child, input)).toIndexedSeq) + val sortKey = new SortKey( + sortExprs.map(s => Evaluate(s.child, input)).toIndexedSeq) (sortKey, row) - }.sortByKey(ascending = true, numPartitions).map(_._2) + }.sortByKeyLocally(ascending = true).map(_._2) } def output = child.output @@ -120,6 +127,9 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { // TODO: Rename: SchemaRDD case class LocalRelation(output: Seq[Attribute], data: Seq[IndexedSeq[Any]]) (@transient sc: SharkContext) extends LeafNode { - def execute() = sc.makeRDD(data.map(buildRow), 1) + + // Since LocalRelation is used for unit tests, set the defaultParallelism to 2 + // to make sure we can cover bugs appearing in a distributed environment. + def execute() = sc.makeRDD(data.map(buildRow), 2) } diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index fc5b745781642..bf81ff580222e 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -9,33 +9,38 @@ import errors._ import expressions._ import plans._ -/* Implicits */ -import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.SharkPairRDDFunctions._ case class SparkEquiInnerJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SharkPlan, right: SharkPlan) + (override val outputDataProperty: DataProperty = + GroupProperty(leftKeys)) extends BinaryNode { def output = left.output ++ right.output + override val leftRequiredDataProperty: DataProperty = GroupProperty(leftKeys) + override val rightRequiredDataProperty: DataProperty = GroupProperty(rightKeys) + override def otherCopyArgs = outputDataProperty :: Nil + def execute() = attachTree(this, "execute") { - val leftWithKeys = left.execute .map { row => + val leftWithKeys = left.execute.map { row => val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) logger.debug(s"Generated left join keys ($leftKeys) => ($joinKeys) given row $row") (joinKeys, row) } - val rightWithKeys = right.execute().map { row => + val rightWithKeys = right.execute.map { row => val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) logger.debug(s"Generated right join keys ($rightKeys) => ($joinKeys) given row $row") (joinKeys, row) } // Do the join. - val joined = filterNulls(leftWithKeys).join(filterNulls(rightWithKeys)) + val joined = filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys)) // Drop join keys and merge input tuples. joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple ++ rightTuple) } } @@ -63,7 +68,7 @@ case class BroadcastNestedLoopJoin( (@transient sc: SharkContext) extends BinaryNode { - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sc :: Nil def output = left.output ++ right.output diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 2d00f3ccb7972..6b859c5e0f48d 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -88,7 +88,7 @@ trait PlanningStrategies { val rightKeys = joinKeys.map(_._2) val joinOp = execution.SparkEquiInnerJoin( - leftKeys, rightKeys, planLater(left), planLater(right)) + leftKeys, rightKeys, planLater(left), planLater(right))() // Make sure other conditions are met if present. if (otherPredicates.nonEmpty) { @@ -135,16 +135,16 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - execution.Sort(sortExprs, planLater(child)) :: Nil + execution.Sort(sortExprs, planLater(child))() :: Nil // TODO: It is correct, but overkill to do a global sorting here. case logical.SortPartitions(sortExprs, child) => - execution.Sort(sortExprs, planLater(child)) :: Nil + execution.Sort(sortExprs, planLater(child))() :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(group, agg, planLater(child)) :: Nil + execution.Aggregate(group, agg, planLater(child))() :: Nil case logical.LocalRelation(output, data) => execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 3546416402f11..6f6212e0cea94 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -1,8 +1,10 @@ package catalyst package plans -import expressions.{Attribute, Expression} -import trees._ +import catalyst.expressions.{SortOrder, Attribute, Expression} +import catalyst.trees._ +import catalyst.execution.GroupProperty +import catalyst.execution.SortProperty abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => @@ -48,6 +50,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionDown(e) case other => other } + case g @ GroupProperty(groupingExpressions) => GroupProperty( + groupingExpressions.map { + case e: Expression => transformExpressionDown(e) + case other => other}) + case g @ SortProperty(sortingExpressions) => SortProperty( + sortingExpressions.map { + case e: Expression => transformExpressionDown(e) + case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -79,6 +89,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionUp(e) case other => other } + case g @ GroupProperty(groupingExpressions) => GroupProperty( + groupingExpressions.map { + case e: Expression => transformExpressionUp(e) + case other => other}) + case g @ SortProperty(sortingExpressions) => SortProperty( + sortingExpressions.map { + case e: Expression => transformExpressionUp(e) + case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -102,6 +120,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => e :: Nil case other => Nil } + case g @ GroupProperty(groupingExpressions) => groupingExpressions.flatMap { + case e: Expression => e :: Nil + case other => Nil + } + case g @ SortProperty(sortingExpressions) => sortingExpressions.flatMap { + case e: Expression => e :: Nil + case other => Nil + } case other => Nil }.toSeq } diff --git a/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala b/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala new file mode 100644 index 0000000000000..7a7a6b61a05b0 --- /dev/null +++ b/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala @@ -0,0 +1,85 @@ +package org.apache.spark.rdd + +import org.apache.spark._ +import org.apache.spark.util.AppendOnlyMap +import scala.collection.mutable.ArrayBuffer + +/** + * A RDD that cogroups its already co-partitioned parents. This RDD works the same as + * a [[org.apache.spark.rdd.CoGroupedRDD]] except that its parents should have the + * same number of partitions. Like a [[org.apache.spark.rdd.CoGroupedRDD]], + * for each key k in parent RDDs, the resulting RDD contains a tuple with the list of + * values for that key. + * + * @param rdds parent RDDs. + */ +class CoGroupedLocallyRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]]) + extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { + + { + // Check if all parents have the same number of partitions. + // It is possible that a parent RDD does not preserve the partitioner, + // so we do not check if all of parent RDDs have the same partitioner. + if (!rdds.forall(rdd => rdd.partitions.size == rdds(0).partitions.size)) { + throw new IllegalArgumentException( + "All parent RDDs should have the same number of partitions.") + } + } + + // All dependencies of a CoGroupedLocallyRDD should be narrow dependencies. + override def getDependencies: Seq[Dependency[_]] = { + rdds.map { rdd: RDD[_ <: Product2[K, _]] => + logDebug("Adding one-to-one dependency with " + rdd) + new OneToOneDependency(rdd) + } + } + + override def getPartitions: Array[Partition] = { + val numPartitions = firstParent[(K, _)].partitions.size + val array = new Array[Partition](numPartitions) + for (i <- 0 until array.size) { + // Each CoGroupPartition will have a dependency per contributing RDD + array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) => + new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) + }.toArray) + } + array + } + + // Take the first not None partitioner. It is possible that all parent partitioners + // are None. + override val partitioner = rdds.find(rdd => rdd.partitioner != None) match { + case Some(rdd) => rdd.partitioner + case None => None + } + + override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { + val split = s.asInstanceOf[CoGroupPartition] + val numRdds = split.deps.size + // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) + val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] + + val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { + if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) + } + + val getSeq = (k: K) => { + map.changeValue(k, update) + } + + for ((dep, depNum) <- split.deps.zipWithIndex) dep match { + case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { + // Read them from the parent + rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { kv => + getSeq(kv._1)(depNum) += kv._2 + } + } + case _ => { + // We should not reach here. It is a sanity check. + throw new RuntimeException("A dependency of this CoGroupedLocallyRDD is not " + + "a narrow dependency.") + } + } + new InterruptibleIterator(context, map.iterator) + } +} \ No newline at end of file diff --git a/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala new file mode 100644 index 0000000000000..bd33b6bcd00eb --- /dev/null +++ b/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala @@ -0,0 +1,41 @@ +package org.apache.spark.rdd + +import scala.reflect.ClassTag +import org.apache.spark.Logging + +import scala.language.implicitConversions + +/** + * Extra functions for Shark available on RDDs of (key, value) pairs where the key is + * sortable through an implicit conversion. + * Import `org.apache.spark.SharkOrderedRDDFunctions._` at the top of your program to + * use these functions. They will work with any key type that has a `scala.math.Ordered` + * implementation. + */ +class SharkOrderedRDDFunctions[K <% Ordered[K]: ClassTag, + V: ClassTag, + P <: Product2[K, V] : ClassTag]( + self: RDD[P]) + extends Logging with Serializable { + + /** + * Sort data within a partition. This function will not introduce a + * shuffling operation. + */ + def sortByKeyLocally(ascending: Boolean = true): RDD[P] = { + self.mapPartitions(iter => { + val buf = iter.toArray + if (ascending) { + buf.sortWith((x, y) => x._1 < y._1).iterator + } else { + buf.sortWith((x, y) => x._1 > y._1).iterator + } + }, preservesPartitioning = true) + } +} + +object SharkOrderedRDDFunctions { + implicit def rddToSharkOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( + rdd: RDD[(K, V)]) = + new SharkOrderedRDDFunctions[K, V, (K, V)](rdd) +} \ No newline at end of file diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala new file mode 100644 index 0000000000000..d26a43ac5720b --- /dev/null +++ b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -0,0 +1,67 @@ +package org.apache.spark.rdd + +import scala.reflect._ +import org.apache.spark._ +import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Aggregator +import org.apache.spark.SparkContext._ + +import scala.language.implicitConversions + +/** + * Extra functions for Shark available on RDDs of (key, value) pairs through an implicit conversion. + * Import `org.apache.spark.SharkPairRDDFunctions._` at the top of your program to use these functions. + */ +class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) + extends Logging + with Serializable { + + /** + * Cogroup corresponding partitions of `this` and `other`. These two RDDs should have + * the same number of partitions. Partitions of these two RDDs are cogrouped + * according to the indexes of partitions. If we have two RDDs and + * each of them has n partitions, we will cogroup the partition i from `this` + * with the partition i from `other`. + * This function will not introduce a shuffling operation. + */ + def cogroupLocally[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + val cg = new CoGroupedLocallyRDD[K](Seq(self, other)) + val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classTag[K], ClassTags.seqSeqClassTag) + prfs.mapValues { case Seq(vs, ws) => + (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) + } + } + + /** + * Group the values for each key within a partition of the RDD into a single sequence. + * This function will not introduce a shuffling operation. + */ + def groupByKeyLocally(): RDD[(K, Seq[V])] = { + def createCombiner(v: V) = ArrayBuffer(v) + def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v + val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner _, mergeValue _, null) + val bufs = self.mapPartitionsWithContext((context, iter) => { + new InterruptibleIterator(context, aggregator.combineValuesByKey(iter)) + }, preservesPartitioning = true) + bufs.asInstanceOf[RDD[(K, Seq[V])]] + } + + /** + * Join corresponding partitions of `this` and `other`. + * If we have two RDDs and each of them has n partitions, + * we will join the partition i from `this` with the partition i from `other`. + * This function will not introduce a shuffling operation. + */ + def joinLocally[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + cogroupLocally(other).flatMapValues { case (vs, ws) => + for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + } + } +} + +object SharkPairRDDFunctions { + implicit def rddToSharkPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = + new SharkPairRDDFunctions(rdd) +} + + diff --git a/src/test/scala/catalyst/execution/DataPropertyTests.scala b/src/test/scala/catalyst/execution/DataPropertyTests.scala new file mode 100644 index 0000000000000..a459dc3e78aac --- /dev/null +++ b/src/test/scala/catalyst/execution/DataPropertyTests.scala @@ -0,0 +1,154 @@ +package catalyst.execution + +import catalyst.dsl._ + +import org.scalatest.FunSuite + +class DataPropertyTests extends FunSuite { + + protected def checkNeedExchange( + outputDataProperty: DataProperty, + inputDataProperty: DataProperty, + expected: Boolean) { + assert( + outputDataProperty.needExchange(inputDataProperty) === expected, + s""" + |== Output data property == + |$outputDataProperty + |== Input data property == + |$inputDataProperty + |== Expected result of needExchange == + |$expected + """.stripMargin) + } + + test("needExchange test: GroupProperty is the output DataProperty") { + // Cases which do not need an exchange between two data properties. + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + NotSpecifiedProperty(), + false) + + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + GroupProperty(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + GroupProperty(Seq('b, 'c)), + GroupProperty(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + GroupProperty(Nil), + GroupProperty(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + GroupProperty(Nil), + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + false) + + // Cases which need an exchange between two data properties. + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + GroupProperty(Seq('b, 'c)), + true) + + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + GroupProperty(Seq('d, 'e)), + true) + + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + GroupProperty(Nil), + true) + + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + true) + + checkNeedExchange( + GroupProperty(Seq('b, 'c)), + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + true) + + // TODO: We should check functional dependencies + /* + checkNeedExchange( + GroupProperty(Seq('b)), + GroupProperty(Seq('b + 1)), + false) + */ + } + + test("needExchange test: SortProperty is the output DataProperty") { + // Cases which do not need an exchange between two data properties. + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + NotSpecifiedProperty(), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + SortProperty(Seq('a.asc, 'b.asc)), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + SortProperty(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Seq('c, 'b, 'a)), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Seq('b, 'c, 'a, 'd)), + false) + + // Cases which need an exchange between two data properties. + // TODO: We can have an optimization to first sort the dataset + // by a.asc and then sort b, and c in a partition. This optimization + // should tradeoff the benefit of a less number of Exchange operators + // and the parallelism. + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + SortProperty(Seq('a.asc, 'b.desc, 'c.asc)), + true) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + SortProperty(Seq('b.asc, 'a.asc)), + true) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Seq('a, 'b)), + true) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Seq('c, 'd)), + true) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Nil), + true) + } +} \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/DslQueryTests1.scala b/src/test/scala/catalyst/execution/DslQueryTests1.scala new file mode 100644 index 0000000000000..dbce6a9743ba4 --- /dev/null +++ b/src/test/scala/catalyst/execution/DslQueryTests1.scala @@ -0,0 +1,74 @@ +package catalyst +package execution + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import analysis._ +import expressions._ +import plans._ +import plans.logical.LogicalPlan +import types._ + +/* Implicits */ +import dsl._ + +class DslQueryTests1 extends FunSuite with BeforeAndAfterAll { + override def beforeAll() { + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests + // without restarting the JVM. + System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") + } + + import TestShark._ + import TestData._ + + test("inner join and then group by using the same column") { + val x = testData2.subquery('x) + val y = testData3.subquery('y) + checkAnswer( + x.join(y).where("x.a".attr === "y.a".attr) + .groupBy("x.a".attr)("x.a".attr, Count("x.b".attr)), + (1,2) :: + (2,2) :: Nil + ) + } + + /** + * Runs the plan and makes sure the answer matches the expected result. + * @param plan the query to be executed + * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ]. + */ + protected def checkAnswer(plan: LogicalPlan, expectedAnswer: Any): Unit = { + val convertedAnswer = expectedAnswer match { + case s: Seq[_] if s.isEmpty => s + case s: Seq[_] if s.head.isInstanceOf[Product] && + !s.head.isInstanceOf[Seq[_]] => s.map(_.asInstanceOf[Product].productIterator.toIndexedSeq) + case s: Seq[_] => s + case singleItem => Seq(Seq(singleItem)) + } + + val isSorted = plan.collect { case s: logical.Sort => s}.nonEmpty + def prepareAnswer(answer: Seq[Any]) = if (!isSorted) answer.sortBy(_.toString) else answer + val sharkAnswer = try plan.toRdd.collect().toSeq catch { + case e: Exception => + fail( + s""" + |Exception thrown while executing query: + |$plan + |== Physical Plan == + |${plan.executedPlan} + |== Exception == + |$e + """.stripMargin) + } + println( + s""" + |Logical plan: + |$plan + |== Physical Plan == + |${plan.executedPlan} + """.stripMargin) + assert(prepareAnswer(convertedAnswer) === prepareAnswer(sharkAnswer)) + } +} \ No newline at end of file From ec906200862e8a3b1a9c7f07e86707f90e81d2be Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:20:35 -0800 Subject: [PATCH 486/778] Support for Sets as arguments to TreeNode classes. --- src/main/scala/catalyst/plans/QueryPlan.scala | 7 +++++-- src/main/scala/catalyst/trees/TreeNode.scala | 9 ++++++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 6f6212e0cea94..6f91f3ff795ee 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -46,7 +46,8 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy val newArgs = productIterator.map { case e: Expression => transformExpressionDown(e) case Some(e: Expression) => Some(transformExpressionDown(e)) - case seq: Seq[_] => seq.map { + case m: Map[_,_] => m + case seq: Traversable[_] => seq.map { case e: Expression => transformExpressionDown(e) case other => other } @@ -85,7 +86,8 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy val newArgs = productIterator.map { case e: Expression => transformExpressionUp(e) case Some(e: Expression) => Some(transformExpressionUp(e)) - case seq: Seq[_] => seq.map { + case m: Map[_,_] => m + case seq: Traversable[_] => seq.map { case e: Expression => transformExpressionUp(e) case other => other } @@ -125,6 +127,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case other => Nil } case g @ SortProperty(sortingExpressions) => sortingExpressions.flatMap { + case seq: Traversable[_] => seq.flatMap { case e: Expression => e :: Nil case other => Nil } diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index a4af9b84c8fef..5f0bb342edcdf 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -125,7 +125,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } else { arg } - case args: Seq[_] => args.map { + case m: Map[_,_] => m + case args: Traversable[_] => args.map { case arg: TreeNode[_] if (children contains arg) => val newChild = arg.asInstanceOf[BaseType].transformDown(rule) if (!(newChild fastEquals arg)) { @@ -168,7 +169,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } else { arg } - case args: Seq[_] => args.map { + case m: Map[_,_] => m + case args: Traversable[_] => args.map { case arg: TreeNode[_] if (children contains arg) => val newChild = arg.asInstanceOf[BaseType].transformUp(rule) if (!(newChild fastEquals arg)) { @@ -225,7 +227,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def argString: String = productIterator.flatMap { case tn: TreeNode[_] if children contains tn => Nil case tn: TreeNode[_] if tn.toString contains "\n" => s"(${tn.simpleString})" :: Nil - case seq: Seq[_] => seq.mkString("{", ",", "}") :: Nil + case seq: Seq[_] => seq.mkString("[", ",", "]") :: Nil + case seq: Set[_] => seq.mkString("{", ",", "}") :: Nil case other => other :: Nil }.mkString(", ") From 037a2ed8afb229718433cc8d075f6d05084e5a1a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:20:53 -0800 Subject: [PATCH 487/778] Better exception when unbound attributes make it to evaluation. --- src/main/scala/catalyst/expressions/Evaluate.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 4943bdb395444..b563041590c02 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -226,6 +226,7 @@ object Evaluate extends Logging { case implementedFunction: ImplementedUdf => implementedFunction.evaluate(implementedFunction.children.map(eval)) + case a: Attribute => throw new OptimizationException(a, "Unable to evaluate unbound reference without access to the input schema.") case other => throw new OptimizationException(other, "evaluation not implemented") } From dfce4268b075622c3553c349c33e99a4b0235d21 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:21:22 -0800 Subject: [PATCH 488/778] Add mapChildren and withNewChildren methods to TreeNode. --- src/main/scala/catalyst/trees/TreeNode.scala | 46 ++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index 5f0bb342edcdf..dd2085950fc3d 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -83,6 +83,52 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { ret } + /** + * Returns a copy of this node where `f` has been applied to all the nodes children. + */ + def mapChildren(f: BaseType => BaseType): this.type = { + var changed = false + val newArgs = productIterator.map { + case arg: TreeNode[_] if (children contains arg) => + val newChild = f(arg.asInstanceOf[BaseType]) + if(newChild fastEquals arg) + arg + else { + changed = true + newChild + } + case nonChild: AnyRef => nonChild + case null => null + }.toArray + if(changed) makeCopy(newArgs) else this + } + + /** + * Returns a copy of this node with the children replaced. + * TODO: Validate somewhere (in debug mode?) that children are ordered correctly. + */ + def withNewChildren(newChildren: Seq[BaseType]): this.type = { + assert(newChildren.size == children.size, "Incorrect number of children") + var changed = false + val remainingNewChildren = newChildren.toBuffer + val remainingOldChildren = children.toBuffer + val newArgs = productIterator.map { + case arg: TreeNode[_] if (children contains arg) => + val newChild = remainingNewChildren.remove(0) + val oldChild = remainingOldChildren.remove(0) + if(newChild fastEquals oldChild) + oldChild + else { + changed = true + newChild + } + case nonChild: AnyRef => nonChild + case null => null + }.toArray + + if(changed) makeCopy(newArgs) else this + } + /** * Returns a copy of this node where `rule` has been recursively applied to the tree. * When `rule` does not apply to a given node it is left unchanged. From a1ef62e3d24a9d3cac072e0ab53a0017cc185c94 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:21:45 -0800 Subject: [PATCH 489/778] Print the executed plan in SharkQuery toString. --- src/main/scala/catalyst/execution/SharkInstance.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 1576d81cb6f56..122f2031c8def 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -144,7 +144,7 @@ abstract class SharkInstance extends Logging { s"""== Logical Plan == |${stringOrError(analyzed)} |== Physical Plan == - |${stringOrError(sharkPlan)} + |${stringOrError(executedPlan)} """.stripMargin.trim } From 60f3a9a0b2026ff159186e8d9921e4f2d9700cc6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:22:46 -0800 Subject: [PATCH 490/778] More aggregate functions out of the aggregate class to make things more readable. --- .../scala/catalyst/execution/aggregates.scala | 138 +++++++++--------- 1 file changed, 70 insertions(+), 68 deletions(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 73920b0f9b243..7c0c8fa4858fe 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -20,75 +20,7 @@ case class Aggregate( override val requiredDataProperty: DataProperty = GroupProperty(groupingExpressions) override def otherCopyArgs = outputDataProperty :: Nil - case class AverageFunction(expr: Expression, base: AggregateExpression) - extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var count: Long = _ - var sum: Long = _ - - def result: Any = sum.toDouble / count.toDouble - - def apply(input: Seq[Row]): Unit = { - count += 1 - // TODO: Support all types here... - sum += Evaluate(expr, input).asInstanceOf[Int] - } - } - - case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var count: Int = _ - - def apply(input: Seq[Row]): Unit = { - val evaluatedExpr = expr.map(Evaluate(_, input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { - count += 1 - } - } - - def result: Any = count - } - - case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var sum = Evaluate(Cast(Literal(0), expr.dataType), Nil) - - def apply(input: Seq[Row]): Unit = - sum = Evaluate(Add(Literal(sum), expr), input) - - def result: Any = sum - } - - case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) - extends AggregateFunction { - - def this() = this(null, null) // Required for serialization. - - val seen = new scala.collection.mutable.HashSet[Any]() - - def apply(input: Seq[Row]): Unit = { - val evaluatedExpr = expr.map(Evaluate(_, input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) - seen += evaluatedExpr - } - - def result: Any = seen.size - } - - case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var result: Any = null - - def apply(input: Seq[Row]): Unit = { - if (result == null) - result = Evaluate(expr, input) - } - } def output = aggregateExpressions.map(_.toAttribute) @@ -143,6 +75,76 @@ case class Aggregate( } } +case class AverageFunction(expr: Expression, base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + var count: Long = _ + var sum: Long = _ + + def result: Any = sum.toDouble / count.toDouble + + def apply(input: Seq[Row]): Unit = { + count += 1 + // TODO: Support all types here... + sum += Evaluate(expr, input).asInstanceOf[Int] + } +} + +case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var count: Int = _ + + def apply(input: Seq[Row]): Unit = { + val evaluatedExpr = expr.map(Evaluate(_, input)) + if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { + count += 1 + } + } + + def result: Any = count +} + +case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var sum = Evaluate(Cast(Literal(0), expr.dataType), Nil) + + def apply(input: Seq[Row]): Unit = + sum = Evaluate(Add(Literal(sum), expr), input) + + def result: Any = sum +} + +case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new scala.collection.mutable.HashSet[Any]() + + def apply(input: Seq[Row]): Unit = { + val evaluatedExpr = expr.map(Evaluate(_, input)) + if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) + seen += evaluatedExpr + } + + def result: Any = seen.size +} + +case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var result: Any = null + + def apply(input: Seq[Row]): Unit = { + if (result == null) + result = Evaluate(expr, input) + } +} + /** * Uses spark Accumulators to perform global aggregation. * From 098dfc42642438e9c97f16fcc072fad765cbc161 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:24:14 -0800 Subject: [PATCH 491/778] Implement Long sorting again. --- src/main/scala/catalyst/execution/basicOperators.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index a0b1874e03991..746c81090aeaf 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -90,6 +90,12 @@ case class Sort( } else { right.asInstanceOf[Double] compare left.asInstanceOf[Double] } + } else if (curDataType == LongType) { + if (curDirection == Ascending) { + left.asInstanceOf[Long] compare right.asInstanceOf[Long] + } else { + right.asInstanceOf[Long] compare left.asInstanceOf[Long] + } } else if (curDataType == StringType) { if (curDirection == Ascending) { left.asInstanceOf[String] compare right.asInstanceOf[String] From 2abb0bc0042ccafaff50c045f23b46be6ae76f89 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 18:53:01 -0800 Subject: [PATCH 492/778] better debug messages, use exists. --- src/main/scala/catalyst/execution/joins.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index bf81ff580222e..192149292f1aa 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -29,13 +29,13 @@ case class SparkEquiInnerJoin( def execute() = attachTree(this, "execute") { val leftWithKeys = left.execute.map { row => val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) - logger.debug(s"Generated left join keys ($leftKeys) => ($joinKeys) given row $row") + logger.debug(s"Generated left join keys [${leftKeys.mkString(",")}] => [${joinKeys.mkString(",")}] given row $row") (joinKeys, row) } val rightWithKeys = right.execute.map { row => val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) - logger.debug(s"Generated right join keys ($rightKeys) => ($joinKeys) given row $row") + logger.debug(s"Generated right join keys [${rightKeys.mkString(",")}] => [${joinKeys.mkString(",")}] given row $row") (joinKeys, row) } @@ -51,7 +51,7 @@ case class SparkEquiInnerJoin( */ protected def filterNulls(rdd: RDD[(Seq[Any], Row)]) = rdd.filter { - case (key: Seq[_], _) => !key.map(_ == null).reduceLeft(_ || _) + case (key: Seq[_], _) => !key.exists(_ == null) } } From fb84ae40f85a62b4f01ca31c6c0a945223df68ba Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 18:53:41 -0800 Subject: [PATCH 493/778] Refactor DataProperty into Distribution. --- .../catalyst/execution/DataProperty.scala | 106 --------- .../scala/catalyst/execution/Exchange.scala | 89 +++---- .../catalyst/execution/SharkInstance.scala | 5 - .../scala/catalyst/execution/SharkPlan.scala | 26 +-- .../scala/catalyst/execution/aggregates.scala | 10 +- .../catalyst/execution/basicOperators.scala | 11 +- .../catalyst/execution/distribution.scala | 88 +++++++ src/main/scala/catalyst/execution/joins.scala | 13 +- .../execution/planningStrategies.scala | 11 +- src/main/scala/catalyst/plans/QueryPlan.scala | 27 --- .../execution/DataPropertyTests.scala | 219 +++++++++--------- .../execution/HiveComparisonTest.scala | 3 +- 12 files changed, 245 insertions(+), 363 deletions(-) delete mode 100644 src/main/scala/catalyst/execution/DataProperty.scala create mode 100644 src/main/scala/catalyst/execution/distribution.scala diff --git a/src/main/scala/catalyst/execution/DataProperty.scala b/src/main/scala/catalyst/execution/DataProperty.scala deleted file mode 100644 index b8179b97f832c..0000000000000 --- a/src/main/scala/catalyst/execution/DataProperty.scala +++ /dev/null @@ -1,106 +0,0 @@ -package catalyst -package execution - -import catalyst.expressions.{SortOrder, Expression} - -trait DataProperty { - def expressions: Seq[Expression] - - /** - * If we need an [[catalyst.execution.Exchange]] to re-partition data for - * the given [[catalyst.execution.DataProperty]] other. - * @param other The given [[catalyst.execution.DataProperty]]. - *@return - */ - // TODO: We should also consider functional dependencies between expressions of - // two data properties. For example, if we have a GroupProperty(a) and a - // GroupProperty(a+1), we will not need an exchange to re-partition the data. - def needExchange(other: DataProperty): Boolean -} - -/** - * An implementation of [[catalyst.execution.DataProperty]] represents that - * the data property of a dataset is not specified. - * If it is used as a required data property for a physical operator - * (a [[catalyst.execution.SharkPlan]]) (e.g. [[catalyst.execution.Project]]), - * it means that this operator does not require its input datasets to be - * organized in a certain way. - */ -case class NotSpecifiedProperty() extends DataProperty { - def expressions = Nil - - def needExchange(other: DataProperty): Boolean = { - other match { - case NotSpecifiedProperty() => false - case GroupProperty(groupingExpressions) => true - case SortProperty(_) => true - } - } -} - -/** - * An implementation of [[catalyst.execution.DataProperty]] represents that - * a dataset is grouped by groupingExpressions. - * @param groupingExpressions The expressions used to specify the way how rows should be grouped. - * If it is a Nil, the entire dataset is considered as a single group. - * In this case, a single reducer will be used. - */ -case class GroupProperty(groupingExpressions: Seq[Expression]) extends DataProperty { - override val expressions = groupingExpressions - - def needExchange(other: DataProperty): Boolean = { - other match { - case NotSpecifiedProperty() => false - // We do not need an Exchange operator if another GroupProperty only - // needs to group rows within a partition. - case g @ GroupProperty(otherExpressions) => { - if (expressions.toSet.subsetOf(otherExpressions.toSet)) false else true - } - // Because we use [[org.apache.spark.HashPartitioner]] for GroupProperty, - // we need to use an Exchange operator to sort data with a - // [[org.apache.spark.RangePartitioner]]. But, if the groupingExpressions is - // a Nil, we will have a single partition. So, we do not need an Exchange operator - // to sort this single partition. - case SortProperty(otherExpressions) => if (expressions == Nil) false else true - } - } -} - -/** - * An implementation of [[catalyst.execution.DataProperty]] represents that - * a dataset is sorted by sortingExpressions. A SortProperty also implies that - * the dataset is grouped by sortingExpressions. - * @param sortingExpressions The expressions used to specify the way how rows should be sorted. - * sortingExpressions should not be empty. - */ -case class SortProperty(sortingExpressions: Seq[SortOrder]) extends DataProperty { - - { - if (sortingExpressions == Nil) { - throw new IllegalArgumentException("Sorting expressions of a SortProperty " + - "are not specified.") - } - } - - def expressions = sortingExpressions - - def needExchange(other: DataProperty): Boolean = { - other match { - case NotSpecifiedProperty() => false - // A SortProperty implies a GroupProperty. We do not need an Exchange operator - // if the GroupProperty only needs to group rows within a partition. - case g @ GroupProperty(otherExpressions) => { - if (expressions.map(expression => expression.child). - toSet.subsetOf(otherExpressions.toSet)) false else true - } - // We do not need an Exchange operator if another SortProperty only needs to - // sort rows within a partition (cases satisfying otherExpressions.startsWith(expressions)) - // or we do not need to sort again (cases satisfying - // expressions.startsWith(otherExpressions)). - case s @ SortProperty(otherExpressions) => { - if (otherExpressions.startsWith(expressions) || - expressions.startsWith(otherExpressions)) false else true - } - } - } -} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 373684eb61ee8..14c4019cdc30e 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -9,29 +9,29 @@ import catalyst.types._ import org.apache.spark.{RangePartitioner, HashPartitioner} import org.apache.spark.rdd.ShuffledRDD -case class Exchange( - dataProperty: DataProperty, - child: SharkPlan, - numPartitions: Int = 8) extends UnaryNode { +case class Exchange(newPartitioning: Distribution, numPartitions: Int, child: SharkPlan) + extends UnaryNode { + override def outputPartitioning = newPartitioning def output = child.output def execute() = attachTree(this , "execute") { - dataProperty match { - case NotSpecifiedProperty() => child.execute() - case g @ GroupProperty(groupingExpressions) => { + newPartitioning match { + case ClusteredDistribution(expressions) => + // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().map { row => - (buildRow(groupingExpressions.toSeq.map(Evaluate(_, Vector(row)))), row) + (buildRow(expressions.toSeq.map(Evaluate(_, Vector(row)))), row) } val part = new HashPartitioner(numPartitions) val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) shuffled.map(_._2) - } - case s @ SortProperty(sortingExpressions) => { + + case OrderedDistribution(sortingExpressions) => val directions = sortingExpressions.map(_.direction).toIndexedSeq val dataTypes = sortingExpressions.map(_.dataType).toIndexedSeq + // TODO: MOVE THIS! class SortKey(val keyValues: IndexedSeq[Any]) extends Ordered[SortKey] with Serializable { @@ -98,63 +98,26 @@ case class Exchange( val shuffled = new ShuffledRDD[SortKey, Row, (SortKey, Row)](rdd, part) shuffled.map(_._2) - } + case UnknownDistribution => + logger.warn("Worthless repartitioning requested.") + child.execute() } } } object AddExchange extends Rule[SharkPlan] { - def apply(plan: SharkPlan): SharkPlan = { - // TODO: determine the number of partitions. - // TODO: We need to consider the number of partitions to determine if we - // will add an Exchange operator. If a dataset only has a single partition, - // even if needExchange returns true, we do not need to shuffle the data again. - val numPartitions = 8 - plan.transformUp { - case aggregate @ Aggregate( - groupingExpressions, - aggregateExpressions, - child) => { - if (child.outputDataProperty.needExchange(aggregate.requiredDataProperty)) { - val exchange = new Exchange(aggregate.requiredDataProperty, child, numPartitions) - - Aggregate(groupingExpressions, aggregateExpressions, exchange)() - } else { - aggregate - } - } - case equiInnerJoin @ SparkEquiInnerJoin( - leftKeys, - rightKeys, - left, - right) => { - val newLeft = { - if (left.outputDataProperty.needExchange(equiInnerJoin.leftRequiredDataProperty)) { - new Exchange(equiInnerJoin.leftRequiredDataProperty, left, numPartitions) - } else { - left - } - } - - val newRight = { - if (right.outputDataProperty.needExchange(equiInnerJoin.rightRequiredDataProperty)) { - new Exchange(equiInnerJoin.rightRequiredDataProperty, right, numPartitions) - } else { - right - } - } - - SparkEquiInnerJoin(leftKeys, rightKeys, newLeft, newRight)() - } - case sort @ Sort(sortExprs, child) => { - if (child.outputDataProperty.needExchange(sort.requiredDataProperty)) { - val exchange = new Exchange(sort.requiredDataProperty, child, numPartitions) - - Sort(sortExprs, exchange)() - } else { - sort - } - } - } + // TODO: determine the number of partitions. + // TODO: We need to consider the number of partitions to determine if we + // will add an Exchange operator. If a dataset only has a single partition, + // even if needExchange returns true, we do not need to shuffle the data again. + val numPartitions = 8 + + def apply(plan: SharkPlan): SharkPlan = plan.transformUp { + case operator: SharkPlan => + operator.withNewChildren(operator.requiredChildPartitioning.zip(operator.children).map { + case (required, child) if !child.outputPartitioning.satisfies(required) => + Exchange(required, numPartitions, child) + case (_, child) => child + }) } } \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 122f2031c8def..86ceb89ba0d49 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -82,11 +82,6 @@ abstract class SharkInstance extends Logging { } object PrepareForExecution extends RuleExecutor[SharkPlan] { - // TODO: [[catalyst.execution.AddExchange]] should be in a phase before - // PrepareForExecution. We may also need a phase of physical optimizations. - // In this phase, we can alter the data property in an Exchange operator to - // reduce the number of shuffling phases. Or, this physical optimization phase can be - // a part of QueryPlanner when a planner can accept a physical operator (a SharkPlan). val batches = Batch("Add exchange", Once, AddExchange) :: diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 8566354b57f65..fc46b1494d8a5 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -3,14 +3,15 @@ package execution import org.apache.spark.rdd.RDD -import catalyst.plans.QueryPlan -import scala.reflect.ClassTag +import plans.QueryPlan abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => - def requiredDataProperties: Seq[DataProperty] - def outputDataProperty: DataProperty + /** Specifies how data is partitioned across different nodes in the cluster. */ + def outputPartitioning: Distribution = UnknownDistribution + /** Specifies any partition requirements on the input data for this operator. */ + def requiredChildPartitioning: Seq[Distribution] = Seq.fill(children.size)(UnknownDistribution) /** * Runs this query returning the result as an RDD. @@ -27,29 +28,12 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { self: Product => - - def requiredDataProperty: DataProperty = NotSpecifiedProperty() - def requiredDataProperties: Seq[DataProperty] = Seq(requiredDataProperty) - // TODO: We should get the output data properties of a leaf node from metadata. - def outputDataProperty: DataProperty = NotSpecifiedProperty() } trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { self: Product => - - def requiredDataProperty: DataProperty = NotSpecifiedProperty() - def requiredDataProperties: Seq[DataProperty] = Seq(requiredDataProperty) - - def outputDataProperty: DataProperty = child.outputDataProperty } trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { self: Product => - - def leftRequiredDataProperty: DataProperty = NotSpecifiedProperty() - def rightRequiredDataProperty: DataProperty = NotSpecifiedProperty() - def requiredDataProperties: Seq[DataProperty] = - Seq(leftRequiredDataProperty, rightRequiredDataProperty) - - def outputDataProperty: DataProperty = NotSpecifiedProperty() } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 7c0c8fa4858fe..89fa0912bb147 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -13,14 +13,10 @@ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], child: SharkPlan) - (override val outputDataProperty: DataProperty = - GroupProperty(groupingExpressions)) extends UnaryNode { - override val requiredDataProperty: DataProperty = GroupProperty(groupingExpressions) - override def otherCopyArgs = outputDataProperty :: Nil - - + val requiredPartitioning = ClusteredDistribution(groupingExpressions) + override def requiredChildPartitioning = Seq(requiredPartitioning, requiredPartitioning) def output = aggregateExpressions.map(_.toAttribute) @@ -46,6 +42,7 @@ case class Aggregate( val remainingAttributes = impl.collect { case a: Attribute => a } // If any references exist that are not inside agg functions then the must be grouping exprs // in this case we must rebind them to the grouping tuple. + // TODO: Is this right still? Do we need this? if (remainingAttributes.nonEmpty) { val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } @@ -154,7 +151,6 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan (@transient sc: SharkContext) extends UnaryNode { def output = aggregateExprs.map(_.toAttribute) override def otherCopyArgs = Seq(sc) - override val requiredDataProperty: DataProperty = GroupProperty(Nil) case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 746c81090aeaf..4b253247994c3 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -45,16 +45,7 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class Sort( - sortExprs: Seq[SortOrder], - child: SharkPlan) - (override val outputDataProperty: DataProperty = - SortProperty(sortExprs)) extends UnaryNode { - val numPartitions = 1 // TODO: Set with input cardinality - - override val requiredDataProperty: DataProperty = SortProperty(sortExprs) - override def otherCopyArgs = outputDataProperty :: Nil - +case class SortPartitions(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { private final val directions = sortExprs.map(_.direction).toIndexedSeq private final val dataTypes = sortExprs.map(_.dataType).toIndexedSeq diff --git a/src/main/scala/catalyst/execution/distribution.scala b/src/main/scala/catalyst/execution/distribution.scala new file mode 100644 index 0000000000000..4ffd5a3f61cf7 --- /dev/null +++ b/src/main/scala/catalyst/execution/distribution.scala @@ -0,0 +1,88 @@ +package catalyst +package execution + +import expressions.{Expression, SortOrder} +import types._ + +/** + * Specifies how tuples that share common expressions will be distributed. Distribution can + * be used to refer to two distinct physical properties: + * - Inter-node partitioning of data: In this case the distribution describes how tuples are + * partitioned across physical machines in a cluster. Knowing this property allows some + * operators (e.g., [[Aggregate]]) to perform partition local operations instead of global ones. + * - Intra-partition ordering of data: In this case the distribution describes guarantees made + * about how tuples are distributed within a single partition. + * + * TOOD(marmbrus): Given the way that Spark does partitioning, I think the order of the grouping + * actually does matter, and thus our subset checking is probably not sufficient + * to ensure correct colocation for joins.? + * TODO(marmbrus): Similarly I'm not sure that satisfies is a sufficient check to see if an sort + * aggregation can be done. Maybe we need two checks? Separate ordering from + * partitioning? + */ +abstract sealed class Distribution extends Expression { + self: Product => + + /** + * Returns true iff the guarantees made by this [[Distribution]] are sufficient to satisfy all + * guarantees mandated by the `required` distribution. + */ + def satisfies(required: Distribution): Boolean +} + +/** + * Represents a distribution where no promises are made about co-location of data. + */ +case object UnknownDistribution extends Distribution with trees.LeafNode[Expression] { + def references = Set.empty + def nullable = false + def dataType = NullType + + def satisfies(required: Distribution): Boolean = required == UnknownDistribution +} + +/** + * Represents data where tuples that share the same values for the `clustering` [[Expression]]s will + * be co-located. Based on the context, this can mean such tuples are either co-located in the same + * partition or they will be contiguous within a single partition. + */ +case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution { + def children = clustering.toSeq + def references = clustering.flatMap(_.references).toSet + def nullable = false + def dataType = NullType + + def satisfies(required: Distribution): Boolean = required match { + case UnknownDistribution => true + // No clustering expressions means only one partition. + case _ if clustering.isEmpty => true + case ClusteredDistribution(requiredClustering) => + clustering.toSet.subsetOf(requiredClustering.toSet) + case _ => false + } +} + +/** + * Represents data where tuples have been ordered according to the `ordering` [[Expression]]s. This + * is a strictly stronger guarantee than [[ClusteredDistribution]] as an ordering will ensure that + * tuples that share the same value for the ordering expressions are contiguous and will never be + * split across partitions. + */ +case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { + def children = ordering + def references = ordering.flatMap(_.references).toSet + def nullable = false + def dataType = NullType + + def clustering = ordering.map(_.child).toSet + + def satisfies(required: Distribution): Boolean = required match { + case UnknownDistribution => true + case OrderedDistribution(requiredOrdering) => + val minSize = Seq(requiredOrdering.size, ordering.size).min + requiredOrdering.take(minSize) == ordering.take(minSize) + case ClusteredDistribution(requiredClustering) => + clustering.subsetOf(requiredClustering.toSet) + case _ => false + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 192149292f1aa..6f811f20a737d 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -15,16 +15,13 @@ case class SparkEquiInnerJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SharkPlan, - right: SharkPlan) - (override val outputDataProperty: DataProperty = - GroupProperty(leftKeys)) - extends BinaryNode { + right: SharkPlan) extends BinaryNode { - def output = left.output ++ right.output - override val leftRequiredDataProperty: DataProperty = GroupProperty(leftKeys) - override val rightRequiredDataProperty: DataProperty = GroupProperty(rightKeys) - override def otherCopyArgs = outputDataProperty :: Nil + override def requiredChildPartitioning = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + def output = left.output ++ right.output def execute() = attachTree(this, "execute") { val leftWithKeys = left.execute.map { row => diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 6b859c5e0f48d..bcce25697ac84 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -88,7 +88,7 @@ trait PlanningStrategies { val rightKeys = joinKeys.map(_._2) val joinOp = execution.SparkEquiInnerJoin( - leftKeys, rightKeys, planLater(left), planLater(right))() + leftKeys, rightKeys, planLater(left), planLater(right)) // Make sure other conditions are met if present. if (otherPredicates.nonEmpty) { @@ -135,16 +135,17 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - execution.Sort(sortExprs, planLater(child))() :: Nil - // TODO: It is correct, but overkill to do a global sorting here. + // First repartition then sort locally. + execution.SortPartitions(sortExprs, + execution.Exchange(OrderedDistribution(sortExprs), 8, planLater(child))) :: Nil case logical.SortPartitions(sortExprs, child) => - execution.Sort(sortExprs, planLater(child))() :: Nil + execution.SortPartitions(sortExprs, planLater(child)) :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(group, agg, planLater(child))() :: Nil + execution.Aggregate(group, agg, planLater(child)) :: Nil case logical.LocalRelation(output, data) => execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 6f91f3ff795ee..8a2ed4201602d 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -3,8 +3,6 @@ package plans import catalyst.expressions.{SortOrder, Attribute, Expression} import catalyst.trees._ -import catalyst.execution.GroupProperty -import catalyst.execution.SortProperty abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => @@ -51,14 +49,6 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionDown(e) case other => other } - case g @ GroupProperty(groupingExpressions) => GroupProperty( - groupingExpressions.map { - case e: Expression => transformExpressionDown(e) - case other => other}) - case g @ SortProperty(sortingExpressions) => SortProperty( - sortingExpressions.map { - case e: Expression => transformExpressionDown(e) - case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -91,14 +81,6 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionUp(e) case other => other } - case g @ GroupProperty(groupingExpressions) => GroupProperty( - groupingExpressions.map { - case e: Expression => transformExpressionUp(e) - case other => other}) - case g @ SortProperty(sortingExpressions) => SortProperty( - sortingExpressions.map { - case e: Expression => transformExpressionUp(e) - case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -118,15 +100,6 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy productIterator.flatMap { case e: Expression => e :: Nil case Some(e: Expression) => e :: Nil - case seq: Seq[_] => seq.flatMap { - case e: Expression => e :: Nil - case other => Nil - } - case g @ GroupProperty(groupingExpressions) => groupingExpressions.flatMap { - case e: Expression => e :: Nil - case other => Nil - } - case g @ SortProperty(sortingExpressions) => sortingExpressions.flatMap { case seq: Traversable[_] => seq.flatMap { case e: Expression => e :: Nil case other => Nil diff --git a/src/test/scala/catalyst/execution/DataPropertyTests.scala b/src/test/scala/catalyst/execution/DataPropertyTests.scala index a459dc3e78aac..933897cff3922 100644 --- a/src/test/scala/catalyst/execution/DataPropertyTests.scala +++ b/src/test/scala/catalyst/execution/DataPropertyTests.scala @@ -4,151 +4,150 @@ import catalyst.dsl._ import org.scalatest.FunSuite -class DataPropertyTests extends FunSuite { - - protected def checkNeedExchange( - outputDataProperty: DataProperty, - inputDataProperty: DataProperty, - expected: Boolean) { - assert( - outputDataProperty.needExchange(inputDataProperty) === expected, - s""" - |== Output data property == - |$outputDataProperty - |== Input data property == - |$inputDataProperty - |== Expected result of needExchange == - |$expected - """.stripMargin) +class DistributionTests extends FunSuite { + protected def checkSatisfied( + inputDistribution: Distribution, + requiredDistribution: Distribution, + satisfied: Boolean) { + if(inputDistribution.satisfies(requiredDistribution) != satisfied) + fail( + s""" + |== Input Distribution == + |$inputDistribution + |== Required Distribution == + |$requiredDistribution + |== Does input distribution satisfy requirements? == + |Expected $satisfied got ${inputDistribution.satisfies(requiredDistribution)} + """.stripMargin) } - test("needExchange test: GroupProperty is the output DataProperty") { + test("needExchange test: ClusteredDistribution is the output DataProperty") { // Cases which do not need an exchange between two data properties. - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - NotSpecifiedProperty(), - false) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + UnknownDistribution, + true) - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - GroupProperty(Seq('a, 'b, 'c)), - false) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) - checkNeedExchange( - GroupProperty(Seq('b, 'c)), - GroupProperty(Seq('a, 'b, 'c)), - false) + checkSatisfied( + ClusteredDistribution(Seq('b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) - checkNeedExchange( - GroupProperty(Nil), - GroupProperty(Seq('a, 'b, 'c)), - false) + checkSatisfied( + ClusteredDistribution(Nil), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) - checkNeedExchange( - GroupProperty(Nil), - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - false) + checkSatisfied( + ClusteredDistribution(Nil), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + true) // Cases which need an exchange between two data properties. - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - GroupProperty(Seq('b, 'c)), - true) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('b, 'c)), + false) - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - GroupProperty(Seq('d, 'e)), - true) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('d, 'e)), + false) - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - GroupProperty(Nil), - true) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Nil), + false) - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - true) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + false) - checkNeedExchange( - GroupProperty(Seq('b, 'c)), - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - true) + checkSatisfied( + ClusteredDistribution(Seq('b, 'c)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + false) // TODO: We should check functional dependencies /* - checkNeedExchange( - GroupProperty(Seq('b)), - GroupProperty(Seq('b + 1)), - false) + checkSatisfied( + ClusteredDistribution(Seq('b)), + ClusteredDistribution(Seq('b + 1)), + true) */ } - test("needExchange test: SortProperty is the output DataProperty") { + test("needExchange test: OrderedDistribution is the output DataProperty") { // Cases which do not need an exchange between two data properties. - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - NotSpecifiedProperty(), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + UnknownDistribution, + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - SortProperty(Seq('a.asc, 'b.asc)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc)), + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - SortProperty(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Seq('a, 'b, 'c)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Seq('c, 'b, 'a)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Seq('c, 'b, 'a)), + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Seq('b, 'c, 'a, 'd)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Seq('b, 'c, 'a, 'd)), + true) // Cases which need an exchange between two data properties. // TODO: We can have an optimization to first sort the dataset // by a.asc and then sort b, and c in a partition. This optimization // should tradeoff the benefit of a less number of Exchange operators // and the parallelism. - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - SortProperty(Seq('a.asc, 'b.desc, 'c.asc)), - true) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), + false) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - SortProperty(Seq('b.asc, 'a.asc)), - true) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('b.asc, 'a.asc)), + false) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Seq('a, 'b)), - true) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Seq('a, 'b)), + false) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Seq('c, 'd)), - true) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Seq('c, 'd)), + false) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Nil), - true) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Nil), + false) } } \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index e829de47ff31c..afd73c0eb0fd0 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -100,7 +100,8 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G // Clean out non-deterministic time schema info. case _: Command => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _ => - val isOrdered = sharkQuery.executedPlan.collect { case s: Sort => s}.nonEmpty + // TODO: Really we only care about the final total ordering here... + val isOrdered = sharkQuery.executedPlan.collect { case s @ Exchange(OrderedDistribution(_), _, _) => s}.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. if (!isOrdered) answer.sorted else answer } From 404d5522e264d1da355e98bad47c84b5dcc445b0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:20:53 -0800 Subject: [PATCH 494/778] Better exception when unbound attributes make it to evaluation. --- src/main/scala/catalyst/expressions/Evaluate.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 4943bdb395444..b563041590c02 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -226,6 +226,7 @@ object Evaluate extends Logging { case implementedFunction: ImplementedUdf => implementedFunction.evaluate(implementedFunction.children.map(eval)) + case a: Attribute => throw new OptimizationException(a, "Unable to evaluate unbound reference without access to the input schema.") case other => throw new OptimizationException(other, "evaluation not implemented") } From 3413e61b2c40ac89135b30c3fa27dd80c70fa1ad Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:21:22 -0800 Subject: [PATCH 495/778] Add mapChildren and withNewChildren methods to TreeNode. --- src/main/scala/catalyst/trees/TreeNode.scala | 46 ++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index a4af9b84c8fef..25d2ff2e13e7b 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -83,6 +83,52 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { ret } + /** + * Returns a copy of this node where `f` has been applied to all the nodes children. + */ + def mapChildren(f: BaseType => BaseType): this.type = { + var changed = false + val newArgs = productIterator.map { + case arg: TreeNode[_] if (children contains arg) => + val newChild = f(arg.asInstanceOf[BaseType]) + if(newChild fastEquals arg) + arg + else { + changed = true + newChild + } + case nonChild: AnyRef => nonChild + case null => null + }.toArray + if(changed) makeCopy(newArgs) else this + } + + /** + * Returns a copy of this node with the children replaced. + * TODO: Validate somewhere (in debug mode?) that children are ordered correctly. + */ + def withNewChildren(newChildren: Seq[BaseType]): this.type = { + assert(newChildren.size == children.size, "Incorrect number of children") + var changed = false + val remainingNewChildren = newChildren.toBuffer + val remainingOldChildren = children.toBuffer + val newArgs = productIterator.map { + case arg: TreeNode[_] if (children contains arg) => + val newChild = remainingNewChildren.remove(0) + val oldChild = remainingOldChildren.remove(0) + if(newChild fastEquals oldChild) + oldChild + else { + changed = true + newChild + } + case nonChild: AnyRef => nonChild + case null => null + }.toArray + + if(changed) makeCopy(newArgs) else this + } + /** * Returns a copy of this node where `rule` has been recursively applied to the tree. * When `rule` does not apply to a given node it is left unchanged. From de0c2485a8c32b54ba1247eef466defdde75aa67 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:21:45 -0800 Subject: [PATCH 496/778] Print the executed plan in SharkQuery toString. --- src/main/scala/catalyst/execution/SharkInstance.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index c7d4eba52751f..d5929d54d2418 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -135,7 +135,7 @@ abstract class SharkInstance extends Logging { s"""== Logical Plan == |${stringOrError(analyzed)} |== Physical Plan == - |${stringOrError(sharkPlan)} + |${stringOrError(executedPlan)} """.stripMargin.trim } From 80d0681c28fa873c9dd9f6a22158cecebe54bb5d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 17 Jan 2014 11:20:34 -0800 Subject: [PATCH 497/778] fix scaladoc links. --- src/main/scala/catalyst/analysis/typeCoercion.scala | 6 +++--- src/main/scala/catalyst/types/dataTypes.scala | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index 780b83e535ba1..74f10723a7cd0 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -152,8 +152,8 @@ object BooleanComparisons extends Rule[LogicalPlan] { } /** - * Casts to/from [[BooleanType]] are transformed into comparisons since the JVM does not consider - * Booleans to be numeric types. + * Casts to/from [[catalyst.types.BooleanType BooleanType]] are transformed into comparisons since + * the JVM does not consider Booleans to be numeric types. */ object BooleanCasts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { @@ -165,7 +165,7 @@ object BooleanCasts extends Rule[LogicalPlan] { /** * When encountering a cast from a string representing a valid fractional number to an integral type - * the jvm will throw a [[java.lang.NumberFormatException]]. Hive, in contrast, returns the + * the jvm will throw a `java.lang.NumberFormatException`. Hive, in contrast, returns the * truncated version of this number. */ object StringToIntegralCasts extends Rule[LogicalPlan] { diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 49bbf88bec7c8..08c9e64b4290f 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -4,7 +4,7 @@ package types import expressions.Expression abstract class DataType { - /** Matches any expression that evaluates to this [[DataType]] */ + /** Matches any expression that evaluates to this DataType */ def unapply(a: Expression): Boolean = a match { case e: Expression if e.dataType == this => true case _ => false From e286d20b88aebf4d183ae74cc4e02ca232860441 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 17 Jan 2014 11:20:51 -0800 Subject: [PATCH 498/778] address code review comments. --- .../scala/catalyst/expressions/Evaluate.scala | 16 ++++++++----- .../catalyst/expressions/predicates.scala | 11 +++++---- src/main/scala/catalyst/trees/TreeNode.scala | 24 +++++++++---------- src/main/scala/catalyst/types/dataTypes.scala | 1 + 4 files changed, 30 insertions(+), 22 deletions(-) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index b563041590c02..9de519bf9431f 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -26,7 +26,8 @@ object Evaluate extends Logging { else e.dataType match { case n: NumericType => - f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType](n.numeric, eval(e).asInstanceOf[n.JvmType]) + val castedFunction = f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType] + castedFunction(n.numeric, eval(e).asInstanceOf[n.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } } @@ -34,7 +35,7 @@ object Evaluate extends Logging { @inline def n2(e1: Expression, e2: Expression, f: ((Numeric[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new OptimizationException(e, s"Types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) @@ -52,7 +53,7 @@ object Evaluate extends Logging { @inline def f2(e1: Expression, e2: Expression, f: ((Fractional[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new OptimizationException(e, s"Types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) @@ -69,7 +70,8 @@ object Evaluate extends Logging { @inline def i2(e1: Expression, e2: Expression, f: ((Integral[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + if (e1.dataType != e2.dataType) + throw new OptimizationException(e, s"Types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) if (evalE1 == null || evalE2 == null) @@ -112,7 +114,7 @@ object Evaluate extends Logging { case UnaryMinus(child) => n1(child, _.negate(_)) /* Control Flow */ - case If(e, t, f) => if(eval(e).asInstanceOf[Boolean]) eval(t) else eval(f) + case If(e, t, f) => if (eval(e).asInstanceOf[Boolean]) eval(t) else eval(f) /* Comparisons */ case Equals(l, r) => @@ -226,7 +228,9 @@ object Evaluate extends Logging { case implementedFunction: ImplementedUdf => implementedFunction.evaluate(implementedFunction.children.map(eval)) - case a: Attribute => throw new OptimizationException(a, "Unable to evaluate unbound reference without access to the input schema.") + case a: Attribute => + throw new OptimizationException(a, + "Unable to evaluate unbound reference without access to the input schema.") case other => throw new OptimizationException(other, "evaluation not implemented") } diff --git a/src/main/scala/catalyst/expressions/predicates.scala b/src/main/scala/catalyst/expressions/predicates.scala index c5723e07062a5..fcdf7855988f5 100644 --- a/src/main/scala/catalyst/expressions/predicates.scala +++ b/src/main/scala/catalyst/expressions/predicates.scala @@ -76,15 +76,18 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E def nullable = false } -case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) extends Expression { +case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) + extends Expression { + def children = predicate :: trueValue :: falseValue :: Nil - def nullable = children.exists(_.nullable) + def nullable = trueValue.nullable || falseValue.nullable def references = children.flatMap(_.references).toSet override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType def dataType = { - if (!resolved) + if (!resolved) { throw new UnresolvedException( - this, s"datatype. Can not resolve due to differing types ${trueValue.dataType}, ${falseValue.dataType}") + this, s"Invalid types: ${trueValue.dataType}, ${falseValue.dataType}") + } trueValue.dataType } } diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index 25d2ff2e13e7b..bb6dfdd3110d5 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -89,18 +89,18 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def mapChildren(f: BaseType => BaseType): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = f(arg.asInstanceOf[BaseType]) - if(newChild fastEquals arg) + if (newChild fastEquals arg) { arg - else { + } else { changed = true newChild } case nonChild: AnyRef => nonChild case null => null }.toArray - if(changed) makeCopy(newArgs) else this + if (changed) makeCopy(newArgs) else this } /** @@ -113,12 +113,12 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { val remainingNewChildren = newChildren.toBuffer val remainingOldChildren = children.toBuffer val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = remainingNewChildren.remove(0) val oldChild = remainingOldChildren.remove(0) - if(newChild fastEquals oldChild) + if (newChild fastEquals oldChild) { oldChild - else { + } else { changed = true newChild } @@ -126,7 +126,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case null => null }.toArray - if(changed) makeCopy(newArgs) else this + if (changed) makeCopy(newArgs) else this } /** @@ -163,7 +163,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def transformChildrenDown(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformDown(rule) if (!(newChild fastEquals arg)) { changed = true @@ -172,7 +172,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { arg } case args: Seq[_] => args.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformDown(rule) if (!(newChild fastEquals arg)) { changed = true @@ -206,7 +206,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def transformChildrenUp(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformUp(rule) if (!(newChild fastEquals arg)) { changed = true @@ -215,7 +215,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { arg } case args: Seq[_] => args.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformUp(rule) if (!(newChild fastEquals arg)) { changed = true diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 08c9e64b4290f..73123e2b0444b 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -36,6 +36,7 @@ object IntegralType { case _ => false } } + abstract class IntegralType extends NumericType { val integral: Integral[JvmType] } From b2a1ec59e0a021cf4031cf61938959477bf8a088 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 17 Jan 2014 11:28:02 -0800 Subject: [PATCH 499/778] add comment on how using numeric implicitly complicates spark serialization. --- src/main/scala/catalyst/types/dataTypes.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 73123e2b0444b..88323fff13d9c 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -25,7 +25,11 @@ case object BooleanType extends NativeType { } abstract class NumericType extends NativeType { - // Unfortunately we can't get this implicitly as that breaks Spark Serialization. + // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for + // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a + // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets + // desugared by the compiler into an argument to the objects constructor. This means there is no + // longer an no argument constructor and thus the JVM cannot serialize the object anymore. val numeric: Numeric[JvmType] } From 7b9080ce9b370fbcc1ec8e2c942d689318a69bb3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 17 Jan 2014 15:24:23 -0800 Subject: [PATCH 500/778] Create OrderedRow class to allow ordering to be used by multiple operators. --- .../catalyst/execution/basicOperators.scala | 64 ++----------------- src/main/scala/catalyst/expressions/Row.scala | 49 ++++++++++++++ src/main/scala/catalyst/types/dataTypes.scala | 17 ++++- 3 files changed, 68 insertions(+), 62 deletions(-) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 75f5808618df7..2bd91b9cf17f4 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -46,72 +46,16 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) } case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { - val numPartitions = 1 // TODO: Set with input cardinality + val numPartitions = 8 // TODO: Set with input cardinality private final val directions = sortExprs.map(_.direction).toIndexedSeq private final val dataTypes = sortExprs.map(_.dataType).toIndexedSeq - private class SortKey(val keyValues: IndexedSeq[Any]) extends Ordered[SortKey] with Serializable { - def compare(other: SortKey): Int = { - var i = 0 - while (i < keyValues.size) { - val left = keyValues(i) - val right = other.keyValues(i) - val curDirection = directions(i) - val curDataType = dataTypes(i) - - logger.debug(s"Comparing $left, $right as $curDataType order $curDirection") - // TODO: Use numeric here too? - val comparison = - if (left == null && right == null) { - 0 - } else if (left == null) { - if (curDirection == Ascending) -1 else 1 - } else if (right == null) { - if (curDirection == Ascending) 1 else -1 - } else if (curDataType == IntegerType) { - if (curDirection == Ascending) { - left.asInstanceOf[Int] compare right.asInstanceOf[Int] - } else { - right.asInstanceOf[Int] compare left.asInstanceOf[Int] - } - } else if (curDataType == DoubleType) { - if (curDirection == Ascending) { - left.asInstanceOf[Double] compare right.asInstanceOf[Double] - } else { - right.asInstanceOf[Double] compare left.asInstanceOf[Double] - } - } else if (curDataType == LongType) { - if (curDirection == Ascending) { - left.asInstanceOf[Long] compare right.asInstanceOf[Long] - } else { - right.asInstanceOf[Long] compare left.asInstanceOf[Long] - } - } else if (curDataType == StringType) { - if (curDirection == Ascending) { - left.asInstanceOf[String] compare right.asInstanceOf[String] - } else { - right.asInstanceOf[String] compare left.asInstanceOf[String] - } - } else { - sys.error(s"Comparison not yet implemented for: $curDataType") - } - - if (comparison != 0) return comparison - i += 1 - } - return 0 - } - } - // TODO: Don't include redundant expressions in both sortKey and row. def execute() = attachTree(this, "sort") { - child.execute().map { row => - val input = Vector(row) - val sortKey = new SortKey(sortExprs.map(s => Evaluate(s.child, input)).toIndexedSeq) - - (sortKey, row) - }.sortByKey(ascending = true, numPartitions).map(_._2) + child.execute() + .mapPartitions(OrderedRow(sortExprs, _)) + .sortByKey(ascending = true, numPartitions).map(_._2) } def output = child.output diff --git a/src/main/scala/catalyst/expressions/Row.scala b/src/main/scala/catalyst/expressions/Row.scala index edff359d7fca4..0aaafa2a35366 100644 --- a/src/main/scala/catalyst/expressions/Row.scala +++ b/src/main/scala/catalyst/expressions/Row.scala @@ -1,6 +1,8 @@ package catalyst package expressions +import types._ + /** * Represents one row of output from a relational operator. Allows both generic access by ordinal, * which will incur boxing overhead for primitives, as well as native primitive access. @@ -76,4 +78,51 @@ class GenericRow(input: Seq[Any]) extends Row { if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.") values(i).asInstanceOf[Byte] } +} + +object OrderedRow { + def apply(ordering: Seq[SortOrder], input: Iterator[Row]): Iterator[(OrderedRow, Row)] = { + val expressions = ordering.map(_.child) + val orderingObjects = ordering.map { o => + o.dataType match { + case nativeType: NativeType => + if(o.direction == Ascending) + nativeType.ordering.asInstanceOf[Ordering[Any]] + else + nativeType.ordering.asInstanceOf[Ordering[Any]].reverse + case _ => sys.error(s"No ordering available for ${o.dataType}") + } + } + val directions = ordering.map(_.direction) + + input.map { row => + (new OrderedRow(orderingObjects, directions, expressions.map(Evaluate(_, Vector(row)))), row) + } + } +} + +class OrderedRow(ordering: Seq[Ordering[Any]], directions: Seq[SortDirection], input: Seq[Any]) + extends GenericRow(input) with Ordered[OrderedRow] { + + def compare(other: OrderedRow): Int = { + var i = 0 + while (i < values.size) { + val left = values(i) + val right = other.values(i) + + val comparison = + if (left == null && right == null) { + 0 + } else if (left == null) { + if (directions(i) == Ascending) -1 else 1 + } else if (right == null) { + if (directions(i) == Ascending) 1 else -1 + } else { + ordering(i).compare(left, right) + } + if (comparison != 0) return comparison + i += 1 + } + return 0 + } } \ No newline at end of file diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 88323fff13d9c..6e864d047ff4a 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -13,15 +13,21 @@ abstract class DataType { case object NullType extends DataType -abstract class NativeType extends DataType { type JvmType } +abstract class NativeType extends DataType { + type JvmType + val ordering: Ordering[JvmType] +} + case object StringType extends NativeType { type JvmType = String + val ordering = implicitly[Ordering[JvmType]] } -case object BinaryType extends NativeType { +case object BinaryType extends DataType { type JvmType = Array[Byte] } case object BooleanType extends NativeType { type JvmType = Boolean + val ordering = implicitly[Ordering[JvmType]] } abstract class NumericType extends NativeType { @@ -49,24 +55,28 @@ case object LongType extends IntegralType { type JvmType = Long val numeric = implicitly[Numeric[Long]] val integral = implicitly[Integral[Long]] + val ordering = implicitly[Ordering[JvmType]] } case object IntegerType extends IntegralType { type JvmType = Int val numeric = implicitly[Numeric[Int]] val integral = implicitly[Integral[Int]] + val ordering = implicitly[Ordering[JvmType]] } case object ShortType extends IntegralType { type JvmType = Short val numeric = implicitly[Numeric[Short]] val integral = implicitly[Integral[Short]] + val ordering = implicitly[Ordering[JvmType]] } case object ByteType extends IntegralType { type JvmType = Byte val numeric = implicitly[Numeric[Byte]] val integral = implicitly[Integral[Byte]] + val ordering = implicitly[Ordering[JvmType]] } /** Matcher for any expressions that evaluate to [[FractionalType]]s */ @@ -84,18 +94,21 @@ case object DecimalType extends FractionalType { type JvmType = BigDecimal val numeric = implicitly[Numeric[BigDecimal]] val fractional = implicitly[Fractional[BigDecimal]] + val ordering = implicitly[Ordering[JvmType]] } case object DoubleType extends FractionalType { type JvmType = Double val numeric = implicitly[Numeric[Double]] val fractional = implicitly[Fractional[Double]] + val ordering = implicitly[Ordering[JvmType]] } case object FloatType extends FractionalType { type JvmType = Float val numeric = implicitly[Numeric[Float]] val fractional = implicitly[Fractional[Float]] + val ordering = implicitly[Ordering[JvmType]] } case class ArrayType(elementType: DataType) extends DataType From fcbc03b69376a161d9dfb463cba57811c54e07b2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 17 Jan 2014 15:41:48 -0800 Subject: [PATCH 501/778] Fix if (). --- src/main/scala/catalyst/expressions/Row.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/expressions/Row.scala b/src/main/scala/catalyst/expressions/Row.scala index 0aaafa2a35366..670221848d827 100644 --- a/src/main/scala/catalyst/expressions/Row.scala +++ b/src/main/scala/catalyst/expressions/Row.scala @@ -86,7 +86,7 @@ object OrderedRow { val orderingObjects = ordering.map { o => o.dataType match { case nativeType: NativeType => - if(o.direction == Ascending) + if (o.direction == Ascending) nativeType.ordering.asInstanceOf[Ordering[Any]] else nativeType.ordering.asInstanceOf[Ordering[Any]].reverse @@ -125,4 +125,4 @@ class OrderedRow(ordering: Seq[Ordering[Any]], directions: Seq[SortDirection], i } return 0 } -} \ No newline at end of file +} From 2b9d80f2a1007510b5f94724db7331cfb2a6c180 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 17 Jan 2014 18:43:09 -0500 Subject: [PATCH 502/778] initial commit of adding exchange operators to physical plans. --- .../scala/catalyst/execution/Exchange.scala | 129 ++++++++++++++++++ .../catalyst/execution/SharkInstance.scala | 11 +- .../scala/catalyst/execution/SharkPlan.scala | 21 +++ .../scala/catalyst/execution/aggregates.scala | 13 +- .../catalyst/execution/basicOperators.scala | 34 +++-- .../catalyst/execution/dataProperties.scala | 106 ++++++++++++++ src/main/scala/catalyst/execution/joins.scala | 17 ++- .../execution/planningStrategies.scala | 8 +- src/main/scala/catalyst/plans/QueryPlan.scala | 30 +++- .../spark/rdd/SharkOrderedRDDFunctions.scala | 41 ++++++ .../spark/rdd/SharkPairRDDFunctions.scala | 81 +++++++++++ 11 files changed, 468 insertions(+), 23 deletions(-) create mode 100644 src/main/scala/catalyst/execution/Exchange.scala create mode 100644 src/main/scala/catalyst/execution/dataProperties.scala create mode 100644 src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala create mode 100644 src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala new file mode 100644 index 0000000000000..370f3a4ca2497 --- /dev/null +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -0,0 +1,129 @@ +package catalyst +package execution + +import catalyst.rules.Rule +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.types._ + +import org.apache.spark.{RangePartitioner, HashPartitioner} +import org.apache.spark.rdd.ShuffledRDD + +case class Exchange( + dataProperty: Partitioned, + child: SharkPlan, + numPartitions: Int = 8) extends UnaryNode { + + def output = child.output + + def execute() = attachTree(this , "execute") { + dataProperty match { + case NotSpecified() => child.execute() + case g @ HashPartitioned(groupingExpressions) => { + val rdd = child.execute().map { row => + (buildRow(groupingExpressions.toSeq.map(Evaluate(_, Vector(row)))), row) + } + val part = new HashPartitioner(numPartitions) + val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) + + shuffled.map(_._2) + } + case s @ RangePartitioned(sortingExpressions) => { + val directions = sortingExpressions.map(_.direction).toIndexedSeq + val dataTypes = sortingExpressions.map(_.dataType).toIndexedSeq + + class SortKey(val keyValues: IndexedSeq[Any]) + extends Ordered[SortKey] + with Serializable { + def compare(other: SortKey): Int = { + var i = 0 + while (i < keyValues.size) { + val left = keyValues(i) + val right = other.keyValues(i) + val curDirection = directions(i) + val curDataType = dataTypes(i) + + logger.debug(s"Comparing $left, $right as $curDataType order $curDirection") + // TODO: Use numeric here too? + val comparison = + if (left == null && right == null) { + 0 + } else if (left == null) { + if (curDirection == Ascending) -1 else 1 + } else if (right == null) { + if (curDirection == Ascending) 1 else -1 + } else if (curDataType == IntegerType) { + if (curDirection == Ascending) { + left.asInstanceOf[Int] compare right.asInstanceOf[Int] + } else { + right.asInstanceOf[Int] compare left.asInstanceOf[Int] + } + } else if (curDataType == DoubleType) { + if (curDirection == Ascending) { + left.asInstanceOf[Double] compare right.asInstanceOf[Double] + } else { + right.asInstanceOf[Double] compare left.asInstanceOf[Double] + } + } else if (curDataType == StringType) { + if (curDirection == Ascending) { + left.asInstanceOf[String] compare right.asInstanceOf[String] + } else { + right.asInstanceOf[String] compare left.asInstanceOf[String] + } + } else { + sys.error(s"Comparison not yet implemented for: $curDataType") + } + + if (comparison != 0) return comparison + i += 1 + } + return 0 + } + } + + val rdd = child.execute().map { row => + val input = Vector(row) + val sortKey = new SortKey( + sortingExpressions.map(s => Evaluate(s.child, input)).toIndexedSeq) + + (sortKey, row) + } + val part = new RangePartitioner(numPartitions, rdd, ascending = true) + val shuffled = new ShuffledRDD[SortKey, Row, (SortKey, Row)](rdd, part) + + shuffled.map(_._2) + } + } + } +} + +object AddExchange extends Rule[SharkPlan] { + def apply(plan: SharkPlan): SharkPlan = { + // TODO: determine the number of partitions. + // TODO: We need to consider the number of partitions to determine if we + // will add an Exchange operator. If a dataset only has a single partition, + // even if needExchange returns true, we do not need to shuffle the data again. + val numPartitions = 8 + plan.transformUp { + case operator: SharkPlan => { + val newChildren: Seq[SharkPlan] = operator.children.view.zipWithIndex.map { + case (child,index) => { + if (child.outputPartitioningScheme.needExchange( + operator.requiredPartitioningSchemes(index))) { + val exchange = new Exchange( + operator.requiredPartitioningSchemes(index), + child, + numPartitions) + + exchange + } else { + child + } + } + } + + operator.withNewChildren(newChildren) + } + } + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index d5929d54d2418..122f2031c8def 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -82,7 +82,16 @@ abstract class SharkInstance extends Logging { } object PrepareForExecution extends RuleExecutor[SharkPlan] { - val batches = Batch("Prepare Expressions", Once, expressions.BindReferences) :: Nil + // TODO: [[catalyst.execution.AddExchange]] should be in a phase before + // PrepareForExecution. We may also need a phase of physical optimizations. + // In this phase, we can alter the data property in an Exchange operator to + // reduce the number of shuffling phases. Or, this physical optimization phase can be + // a part of QueryPlanner when a planner can accept a physical operator (a SharkPlan). + val batches = + Batch("Add exchange", Once, + AddExchange) :: + Batch("Prepare Expressions", Once, + expressions.BindReferences) :: Nil } class SharkSqlQuery(sql: String) extends SharkQuery { diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 6116d8397b090..83d50e4990e5c 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -4,10 +4,14 @@ package execution import org.apache.spark.rdd.RDD import catalyst.plans.QueryPlan +import scala.reflect.ClassTag abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => + def requiredPartitioningSchemes: Seq[Partitioned] + def outputPartitioningScheme: Partitioned + /** * Runs this query returning the result as an RDD. */ @@ -23,12 +27,29 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { self: Product => + + def requiredPartitioningScheme: Partitioned = NotSpecified() + def requiredPartitioningSchemes: Seq[Partitioned] = Seq(requiredPartitioningScheme) + // TODO: We should get the output data properties of a leaf node from metadata. + def outputPartitioningScheme: Partitioned = NotSpecified() } trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { self: Product => + + def requiredPartitioningScheme: Partitioned = NotSpecified() + def requiredPartitioningSchemes: Seq[Partitioned] = Seq(requiredPartitioningScheme) + + def outputPartitioningScheme: Partitioned = child.outputPartitioningScheme } trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { self: Product => + + def leftRequiredPartitioningScheme: Partitioned = NotSpecified() + def rightRequiredPartitioningScheme: Partitioned = NotSpecified() + def requiredPartitioningSchemes: Seq[Partitioned] = + Seq(leftRequiredPartitioningScheme, rightRequiredPartitioningScheme) + + def outputPartitioningScheme: Partitioned = NotSpecified() } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 81ccaafd10d24..959770045a10e 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,16 +3,23 @@ package execution import catalyst.errors._ import catalyst.expressions._ +import org.apache.spark.rdd.SharkPairRDDFunctions /* Implicits */ import org.apache.spark.SparkContext._ +import SharkPairRDDFunctions._ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], child: SharkPlan) + (override val outputPartitioningScheme: Partitioned = + HashPartitioned(groupingExpressions)) extends UnaryNode { + override val requiredPartitioningScheme: Partitioned = HashPartitioned(groupingExpressions) + override def otherCopyArgs = outputPartitioningScheme :: Nil + case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -86,9 +93,12 @@ case class Aggregate( def output = aggregateExpressions.map(_.toAttribute) def execute() = attachTree(this, "execute") { + // TODO: If the child of it is an [[catalyst.execution.Exchange]], + // do not evaluate the groupingExpressions again since we have evaluated it + // in the [[catalyst.execution.Exchange]]. val grouped = child.execute().map { row => (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row) - }.groupByKey() + }.groupByKeyLocally() grouped.map { case (group, rows) => // Replace all aggregate expressions with spark functions that will compute the result. @@ -142,6 +152,7 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan (@transient sc: SharkContext) extends UnaryNode { def output = aggregateExprs.map(_.toAttribute) override def otherCopyArgs = Seq(sc) + override val requiredPartitioningScheme: Partitioned = HashPartitioned(Nil) case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 75f5808618df7..8f8b4b39e3e16 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -1,12 +1,12 @@ package catalyst package execution -import errors._ -import expressions._ -import types._ - -import org.apache.spark.SparkContext._ +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.types._ +/* Implicits */ +import org.apache.spark.rdd.SharkOrderedRDDFunctions._ case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -45,12 +45,21 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { +case class Sort( + sortExprs: Seq[SortOrder], + child: SharkPlan) + (override val outputPartitioningScheme: Partitioned = + RangePartitioned(sortExprs)) extends UnaryNode { val numPartitions = 1 // TODO: Set with input cardinality + override val requiredPartitioningScheme: Partitioned = RangePartitioned(sortExprs) + override def otherCopyArgs = outputPartitioningScheme :: Nil + private final val directions = sortExprs.map(_.direction).toIndexedSeq private final val dataTypes = sortExprs.map(_.dataType).toIndexedSeq + // TODO: This SortKey and the one in [[catalyst.execute.Exchange]] are the same. + // We should just define it only once. private class SortKey(val keyValues: IndexedSeq[Any]) extends Ordered[SortKey] with Serializable { def compare(other: SortKey): Int = { var i = 0 @@ -106,12 +115,16 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { // TODO: Don't include redundant expressions in both sortKey and row. def execute() = attachTree(this, "sort") { + // TODO: If the child of it is an [[catalyst.execution.Exchange]], + // do not evaluate the sortKey again since we have evaluated it + // in the [[catalyst.execution.Exchange]]. child.execute().map { row => val input = Vector(row) - val sortKey = new SortKey(sortExprs.map(s => Evaluate(s.child, input)).toIndexedSeq) + val sortKey = new SortKey( + sortExprs.map(s => Evaluate(s.child, input)).toIndexedSeq) (sortKey, row) - }.sortByKey(ascending = true, numPartitions).map(_._2) + }.sortByKeyLocally(ascending = true).map(_._2) } def output = child.output @@ -120,6 +133,9 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { // TODO: Rename: SchemaRDD case class LocalRelation(output: Seq[Attribute], data: Seq[IndexedSeq[Any]]) (@transient sc: SharkContext) extends LeafNode { - def execute() = sc.makeRDD(data.map(buildRow), 1) + + // Since LocalRelation is used for unit tests, set the defaultParallelism to 2 + // to make sure we can cover bugs appearing in a distributed environment. + def execute() = sc.makeRDD(data.map(buildRow), 2) } diff --git a/src/main/scala/catalyst/execution/dataProperties.scala b/src/main/scala/catalyst/execution/dataProperties.scala new file mode 100644 index 0000000000000..2b67dd4242251 --- /dev/null +++ b/src/main/scala/catalyst/execution/dataProperties.scala @@ -0,0 +1,106 @@ +package catalyst +package execution + +import catalyst.expressions.{SortOrder, Expression} + +trait Partitioned { + def expressions: Seq[Expression] + + /** + * If we need an [[catalyst.execution.Exchange]] to re-partition data for + * the given [[catalyst.execution.Partitioned]] other. + * @param other The given [[catalyst.execution.Partitioned]]. + *@return + */ + // TODO: We should also consider functional dependencies between expressions of + // two data properties. For example, if we have a GroupProperty(a) and a + // GroupProperty(a+1), we will not need an exchange to re-partition the data. + def needExchange(other: Partitioned): Boolean +} + +/** + * An implementation of [[catalyst.execution.Partitioned]] represents that + * the data property of a dataset is not specified. + * If it is used as a required data property for a physical operator + * (a [[catalyst.execution.SharkPlan]]) (e.g. [[catalyst.execution.Project]]), + * it means that this operator does not require its input datasets to be + * organized in a certain way. + */ +case class NotSpecified() extends Partitioned { + def expressions = Nil + + def needExchange(other: Partitioned): Boolean = { + other match { + case NotSpecified() => false + case HashPartitioned(groupingExpressions) => true + case RangePartitioned(_) => true + } + } +} + +/** + * An implementation of [[catalyst.execution.Partitioned]] represents that + * a dataset is grouped by groupingExpressions. + * @param groupingExpressions The expressions used to specify the way how rows should be grouped. + * If it is a Nil, the entire dataset is considered as a single group. + * In this case, a single reducer will be used. + */ +case class HashPartitioned(groupingExpressions: Seq[Expression]) extends Partitioned { + override val expressions = groupingExpressions + + def needExchange(other: Partitioned): Boolean = { + other match { + case NotSpecified() => false + // We do not need an Exchange operator if another GroupProperty only + // needs to group rows within a partition. + case g @ HashPartitioned(otherExpressions) => { + if (expressions.toSet.subsetOf(otherExpressions.toSet)) false else true + } + // Because we use [[org.apache.spark.HashPartitioner]] for GroupProperty, + // we need to use an Exchange operator to sort data with a + // [[org.apache.spark.RangePartitioner]]. But, if the groupingExpressions is + // a Nil, we will have a single partition. So, we do not need an Exchange operator + // to sort this single partition. + case RangePartitioned(otherExpressions) => if (expressions == Nil) false else true + } + } +} + +/** + * An implementation of [[catalyst.execution.Partitioned]] represents that + * a dataset is sorted by sortingExpressions. A SortProperty also implies that + * the dataset is grouped by sortingExpressions. + * @param sortingExpressions The expressions used to specify the way how rows should be sorted. + * sortingExpressions should not be empty. + */ +case class RangePartitioned(sortingExpressions: Seq[SortOrder]) extends Partitioned { + + { + if (sortingExpressions == Nil) { + throw new IllegalArgumentException("Sorting expressions of a SortProperty " + + "are not specified.") + } + } + + def expressions = sortingExpressions + + def needExchange(other: Partitioned): Boolean = { + other match { + case NotSpecified() => false + // A SortProperty implies a GroupProperty. We do not need an Exchange operator + // if the GroupProperty only needs to group rows within a partition. + case g @ HashPartitioned(otherExpressions) => { + if (expressions.map(expression => expression.child). + toSet.subsetOf(otherExpressions.toSet)) false else true + } + // We do not need an Exchange operator if another SortProperty only needs to + // sort rows within a partition (cases satisfying otherExpressions.startsWith(expressions)) + // or we do not need to sort again (cases satisfying + // expressions.startsWith(otherExpressions)). + case s @ RangePartitioned(otherExpressions) => { + if (otherExpressions.startsWith(expressions) || + expressions.startsWith(otherExpressions)) false else true + } + } + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index fc5b745781642..1f76ff0f3245b 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -9,33 +9,38 @@ import errors._ import expressions._ import plans._ -/* Implicits */ -import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.SharkPairRDDFunctions._ case class SparkEquiInnerJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SharkPlan, right: SharkPlan) + (override val outputPartitioningScheme: Partitioned = + HashPartitioned(leftKeys)) extends BinaryNode { def output = left.output ++ right.output + override val leftRequiredPartitioningScheme: Partitioned = HashPartitioned(leftKeys) + override val rightRequiredPartitioningScheme: Partitioned = HashPartitioned(rightKeys) + override def otherCopyArgs = outputPartitioningScheme :: Nil + def execute() = attachTree(this, "execute") { - val leftWithKeys = left.execute .map { row => + val leftWithKeys = left.execute.map { row => val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) logger.debug(s"Generated left join keys ($leftKeys) => ($joinKeys) given row $row") (joinKeys, row) } - val rightWithKeys = right.execute().map { row => + val rightWithKeys = right.execute.map { row => val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) logger.debug(s"Generated right join keys ($rightKeys) => ($joinKeys) given row $row") (joinKeys, row) } // Do the join. - val joined = filterNulls(leftWithKeys).join(filterNulls(rightWithKeys)) + val joined = filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys)) // Drop join keys and merge input tuples. joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple ++ rightTuple) } } @@ -63,7 +68,7 @@ case class BroadcastNestedLoopJoin( (@transient sc: SharkContext) extends BinaryNode { - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sc :: Nil def output = left.output ++ right.output diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 2d00f3ccb7972..6b859c5e0f48d 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -88,7 +88,7 @@ trait PlanningStrategies { val rightKeys = joinKeys.map(_._2) val joinOp = execution.SparkEquiInnerJoin( - leftKeys, rightKeys, planLater(left), planLater(right)) + leftKeys, rightKeys, planLater(left), planLater(right))() // Make sure other conditions are met if present. if (otherPredicates.nonEmpty) { @@ -135,16 +135,16 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - execution.Sort(sortExprs, planLater(child)) :: Nil + execution.Sort(sortExprs, planLater(child))() :: Nil // TODO: It is correct, but overkill to do a global sorting here. case logical.SortPartitions(sortExprs, child) => - execution.Sort(sortExprs, planLater(child)) :: Nil + execution.Sort(sortExprs, planLater(child))() :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(group, agg, planLater(child)) :: Nil + execution.Aggregate(group, agg, planLater(child))() :: Nil case logical.LocalRelation(output, data) => execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 3546416402f11..cfc84aa78c6a2 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -1,8 +1,10 @@ package catalyst package plans -import expressions.{Attribute, Expression} -import trees._ +import catalyst.expressions.{SortOrder, Attribute, Expression} +import catalyst.trees._ +import catalyst.execution.HashPartitioned +import catalyst.execution.RangePartitioned abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => @@ -48,6 +50,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionDown(e) case other => other } + case g @ HashPartitioned(groupingExpressions) => HashPartitioned( + groupingExpressions.map { + case e: Expression => transformExpressionDown(e) + case other => other}) + case g @ RangePartitioned(sortingExpressions) => RangePartitioned( + sortingExpressions.map { + case e: Expression => transformExpressionDown(e) + case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -79,6 +89,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionUp(e) case other => other } + case g @ HashPartitioned(groupingExpressions) => HashPartitioned( + groupingExpressions.map { + case e: Expression => transformExpressionUp(e) + case other => other}) + case g @ RangePartitioned(sortingExpressions) => RangePartitioned( + sortingExpressions.map { + case e: Expression => transformExpressionUp(e) + case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -102,6 +120,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => e :: Nil case other => Nil } + case g @ HashPartitioned(groupingExpressions) => groupingExpressions.flatMap { + case e: Expression => e :: Nil + case other => Nil + } + case g @ RangePartitioned(sortingExpressions) => sortingExpressions.flatMap { + case e: Expression => e :: Nil + case other => Nil + } case other => Nil }.toSeq } diff --git a/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala new file mode 100644 index 0000000000000..bd33b6bcd00eb --- /dev/null +++ b/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala @@ -0,0 +1,41 @@ +package org.apache.spark.rdd + +import scala.reflect.ClassTag +import org.apache.spark.Logging + +import scala.language.implicitConversions + +/** + * Extra functions for Shark available on RDDs of (key, value) pairs where the key is + * sortable through an implicit conversion. + * Import `org.apache.spark.SharkOrderedRDDFunctions._` at the top of your program to + * use these functions. They will work with any key type that has a `scala.math.Ordered` + * implementation. + */ +class SharkOrderedRDDFunctions[K <% Ordered[K]: ClassTag, + V: ClassTag, + P <: Product2[K, V] : ClassTag]( + self: RDD[P]) + extends Logging with Serializable { + + /** + * Sort data within a partition. This function will not introduce a + * shuffling operation. + */ + def sortByKeyLocally(ascending: Boolean = true): RDD[P] = { + self.mapPartitions(iter => { + val buf = iter.toArray + if (ascending) { + buf.sortWith((x, y) => x._1 < y._1).iterator + } else { + buf.sortWith((x, y) => x._1 > y._1).iterator + } + }, preservesPartitioning = true) + } +} + +object SharkOrderedRDDFunctions { + implicit def rddToSharkOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( + rdd: RDD[(K, V)]) = + new SharkOrderedRDDFunctions[K, V, (K, V)](rdd) +} \ No newline at end of file diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala new file mode 100644 index 0000000000000..ad9e93165e790 --- /dev/null +++ b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -0,0 +1,81 @@ +package org.apache.spark.rdd + +import scala.reflect._ +import org.apache.spark._ +import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Aggregator +import org.apache.spark.SparkContext._ + +import scala.language.implicitConversions +import org.apache.spark.util.AppendOnlyMap + +/** + * Extra functions for Shark available on RDDs of (key, value) pairs through an implicit conversion. + * Import `org.apache.spark.SharkPairRDDFunctions._` at the top of your program to use these functions. + */ +class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) + extends Logging + with Serializable { + + /** + * Cogroup corresponding partitions of `this` and `other`. These two RDDs should have + * the same number of partitions. Partitions of these two RDDs are cogrouped + * according to the indexes of partitions. If we have two RDDs and + * each of them has n partitions, we will cogroup the partition i from `this` + * with the partition i from `other`. + * This function will not introduce a shuffling operation. + */ + def cogroupLocally[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + val cg = self.zipPartitions(other)((iter1:Iterator[(K, V)], iter2:Iterator[(K, W)]) => { + val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] + + val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { + if (hadVal) oldVal else Array.fill(2)(new ArrayBuffer[Any]) + } + + val getSeq = (k: K) => { + map.changeValue(k, update) + } + + iter1.foreach { kv => getSeq(kv._1)(0) += kv._2 } + iter2.foreach { kv => getSeq(kv._1)(1) += kv._2 } + + map.iterator + }).mapValues { case Seq(vs, ws) => (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])} + + cg + } + + /** + * Group the values for each key within a partition of the RDD into a single sequence. + * This function will not introduce a shuffling operation. + */ + def groupByKeyLocally(): RDD[(K, Seq[V])] = { + def createCombiner(v: V) = ArrayBuffer(v) + def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v + val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner _, mergeValue _, null) + val bufs = self.mapPartitionsWithContext((context, iter) => { + new InterruptibleIterator(context, aggregator.combineValuesByKey(iter)) + }, preservesPartitioning = true) + bufs.asInstanceOf[RDD[(K, Seq[V])]] + } + + /** + * Join corresponding partitions of `this` and `other`. + * If we have two RDDs and each of them has n partitions, + * we will join the partition i from `this` with the partition i from `other`. + * This function will not introduce a shuffling operation. + */ + def joinLocally[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + cogroupLocally(other).flatMapValues { + case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + } + } +} + +object SharkPairRDDFunctions { + implicit def rddToSharkPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = + new SharkPairRDDFunctions(rdd) +} + + From 73c70dea02e0aa820f9e70b377ffa7f811144fcc Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 17 Jan 2014 18:44:18 -0500 Subject: [PATCH 503/778] add a first set of unit tests for data properties. --- .../execution/DataPropertyTests.scala | 154 ++++++++++++++++++ 1 file changed, 154 insertions(+) create mode 100644 src/test/scala/catalyst/execution/DataPropertyTests.scala diff --git a/src/test/scala/catalyst/execution/DataPropertyTests.scala b/src/test/scala/catalyst/execution/DataPropertyTests.scala new file mode 100644 index 0000000000000..67cd333393136 --- /dev/null +++ b/src/test/scala/catalyst/execution/DataPropertyTests.scala @@ -0,0 +1,154 @@ +package catalyst.execution + +import catalyst.dsl._ + +import org.scalatest.FunSuite + +class DataPropertyTests extends FunSuite { + + protected def checkNeedExchange( + outputDataProperty: Partitioned, + inputDataProperty: Partitioned, + expected: Boolean) { + assert( + outputDataProperty.needExchange(inputDataProperty) === expected, + s""" + |== Output data property == + |$outputDataProperty + |== Input data property == + |$inputDataProperty + |== Expected result of needExchange == + |$expected + """.stripMargin) + } + + test("needExchange test: GroupProperty is the output DataProperty") { + // Cases which do not need an exchange between two data properties. + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + NotSpecified(), + false) + + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + HashPartitioned(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + HashPartitioned(Seq('b, 'c)), + HashPartitioned(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + HashPartitioned(Nil), + HashPartitioned(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + HashPartitioned(Nil), + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + false) + + // Cases which need an exchange between two data properties. + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + HashPartitioned(Seq('b, 'c)), + true) + + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + HashPartitioned(Seq('d, 'e)), + true) + + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + HashPartitioned(Nil), + true) + + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + true) + + checkNeedExchange( + HashPartitioned(Seq('b, 'c)), + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + true) + + // TODO: We should check functional dependencies + /* + checkNeedExchange( + GroupProperty(Seq('b)), + GroupProperty(Seq('b + 1)), + false) + */ + } + + test("needExchange test: SortProperty is the output DataProperty") { + // Cases which do not need an exchange between two data properties. + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + NotSpecified(), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioned(Seq('a.asc, 'b.asc)), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Seq('c, 'b, 'a)), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Seq('b, 'c, 'a, 'd)), + false) + + // Cases which need an exchange between two data properties. + // TODO: We can have an optimization to first sort the dataset + // by a.asc and then sort b, and c in a partition. This optimization + // should tradeoff the benefit of a less number of Exchange operators + // and the parallelism. + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioned(Seq('a.asc, 'b.desc, 'c.asc)), + true) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioned(Seq('b.asc, 'a.asc)), + true) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Seq('a, 'b)), + true) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Seq('c, 'd)), + true) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Nil), + true) + } +} \ No newline at end of file From 0ff8be7b53e54d6bc8f2eef91c0e5e9f1673931d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 17 Jan 2014 15:56:26 -0800 Subject: [PATCH 504/778] Cleanup spurious changes and fix doc links. --- build.sbt | 2 +- .../scala/catalyst/optimizer/Optimizer.scala | 28 +++++++++++-------- .../plans/logical/basicOperators.scala | 2 +- .../optimizer/ConstantFoldingSuite.scala | 24 ++++++++-------- 4 files changed, 30 insertions(+), 26 deletions(-) diff --git a/build.sbt b/build.sbt index 10c31f547c03d..2fe38ceb72ac8 100644 --- a/build.sbt +++ b/build.sbt @@ -72,4 +72,4 @@ mergeStrategy in assembly := { case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines case "reference.conf" => MergeStrategy.concat case _ => MergeStrategy.first -} +} \ No newline at end of file diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala index 1f5e4cf6bf86a..4525d8095f039 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/src/main/scala/catalyst/optimizer/Optimizer.scala @@ -23,8 +23,9 @@ object Optimize extends RuleExecutor[LogicalPlan] { } /** - * Removes [[Subquery]] operators from the plan. Subqueries are only required to provide scoping - * information for attributes and can be removed once analysis is complete. + * Removes [[catalyst.plans.logical.Subquery Subquery]] operators from the plan. Subqueries are + * only required to provide scoping information for attributes and can be removed once analysis is + * complete. */ object EliminateSubqueries extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -33,8 +34,8 @@ object EliminateSubqueries extends Rule[LogicalPlan] { } /** - * Replaces [[Expression]]s that can be statically evaluated with equivalent [[expressions.Literal]] - * values. + * Replaces [[catalyst.expressions.Expression Expressions]] that can be statically evaluated with + * equivalent [[catalyst.expressions.Literal Literal]] values. */ object ConstantFolding extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -77,8 +78,8 @@ object BooleanSimplification extends Rule[LogicalPlan] { } /** - * Combines two adjacent [[Filter]] operators into one, merging the conditions into one conjunctive - * predicate. + * Combines two adjacent [[catalyst.plans.logical.Filter Filter]] operators into one, merging the + * conditions into one conjunctive predicate. */ object CombineFilters extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -87,8 +88,9 @@ object CombineFilters extends Rule[LogicalPlan] { } /** - * Pushes [[Filter]] operators through [[Project]] operators, in-lining any [[Alias]]es that were - * defined in the projection. + * Pushes [[catalyst.plans.logical.Filter Filter]] operators through + * [[catalyst.plans.logical.Project Project]] operators, in-lining any + * [[catalyst.expressions.Alias Aliases]] that were defined in the projection. * * This heuristic is valid assuming the expression evaluation cost is minimal. */ @@ -110,9 +112,10 @@ object PushPredicateThroughProject extends Rule[LogicalPlan] { } /** - * Pushes down [[Filter]] operators where the `condition` can be evaluated using only the attributes - * of the left or right side of an inner join. Other [[Filter]] conditions are moved into the - * `condition` of the [[Join]]. + * Pushes down [[catalyst.plans.logical.Filter Filter]] operators where the `condition` can be + * evaluated using only the attributes of the left or right side of an inner join. Other + * [[catalyst.plans.logical.Filter Filter]] conditions are moved into the `condition` of the + * [[catalyst.plans.logical.Join Join]]. */ object PushPredicateThroughInnerJoin extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -136,7 +139,8 @@ object PushPredicateThroughInnerJoin extends Rule[LogicalPlan] with PredicateHel } /** - * Removes [[Cast]]s that are unnecessary because the input is already the correct type. + * Removes [[catalyst.expressions.Cast Casts]] that are unnecessary because the input is already + * the correct type. */ object SimplifyCasts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index f631899a86f89..1f0f5d15c21e6 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -9,7 +9,7 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend def references = projectList.flatMap(_.references).toSet } -case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode with PredicateHelper { +case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { def output = child.output def references = condition.references } diff --git a/src/test/scala/catalyst/optimizer/ConstantFoldingSuite.scala b/src/test/scala/catalyst/optimizer/ConstantFoldingSuite.scala index ae5dd48668566..46088ff4b9c55 100644 --- a/src/test/scala/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/src/test/scala/catalyst/optimizer/ConstantFoldingSuite.scala @@ -36,9 +36,9 @@ class ConstantFoldingSuite extends OptimizerTest { comparePlans(optimized, correctAnswer) } - /* - * Unit tests for constant folding in expressions. - * */ + /** + * Unit tests for constant folding in expressions. + */ test("Constant folding test: expressions only have literals") { val originalQuery = testRelation @@ -48,8 +48,8 @@ class ConstantFoldingSuite extends OptimizerTest { Literal(2) * (Literal(3) + Literal(4)) as Symbol("2*(3+4)")) .where( Literal(1) === Literal(1) && - Literal(2) > Literal(3) || - Literal(3) > Literal(2) ) + Literal(2) > Literal(3) || + Literal(3) > Literal(2) ) .groupBy( Literal(2) * Literal(3) - Literal(6) / (Literal(4) - Literal(2)) )(Literal(9) / Literal(3) as Symbol("9/3")) @@ -99,13 +99,13 @@ class ConstantFoldingSuite extends OptimizerTest { testRelation .where( (('a > 1 && Literal(1) === Literal(1)) || - ('a < 10 && Literal(1) === Literal(2)) || - (Literal(1) === Literal(1) && 'b > 1) || - (Literal(1) === Literal(2) && 'b < 10)) && - (('a > 1 || Literal(1) === Literal(1)) && - ('a < 10 || Literal(1) === Literal(2)) && - (Literal(1) === Literal(1) || 'b > 1) && - (Literal(1) === Literal(2) || 'b < 10))) + ('a < 10 && Literal(1) === Literal(2)) || + (Literal(1) === Literal(1) && 'b > 1) || + (Literal(1) === Literal(2) && 'b < 10)) && + (('a > 1 || Literal(1) === Literal(1)) && + ('a < 10 || Literal(1) === Literal(2)) && + (Literal(1) === Literal(1) || 'b > 1) && + (Literal(1) === Literal(2) || 'b < 10))) val optimized = Optimize(originalQuery.analyze) From 583a337838824ad4621471975136e35afe9969ad Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 18 Jan 2014 18:20:30 -0800 Subject: [PATCH 505/778] break apart distribution and partitioning. --- .../scala/catalyst/execution/Exchange.scala | 52 +++++-- .../scala/catalyst/execution/SharkPlan.scala | 6 +- .../scala/catalyst/execution/aggregates.scala | 3 +- .../catalyst/execution/dataProperties.scala | 106 ------------- .../catalyst/execution/distribution.scala | 88 ----------- src/main/scala/catalyst/execution/joins.scala | 3 +- .../execution/planningStrategies.scala | 3 +- .../plans/physical/partitioning.scala | 143 ++++++++++++++++++ .../spark/rdd/CoGroupedLocallyRDD.scala | 2 +- .../execution/DataPropertyTests.scala | 4 + .../execution/HiveComparisonTest.scala | 4 +- 11 files changed, 197 insertions(+), 217 deletions(-) create mode 100644 src/main/scala/catalyst/plans/physical/partitioning.scala diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 14c4019cdc30e..764c106912786 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -4,12 +4,13 @@ package execution import catalyst.rules.Rule import catalyst.errors._ import catalyst.expressions._ +import catalyst.plans.physical._ import catalyst.types._ import org.apache.spark.{RangePartitioner, HashPartitioner} import org.apache.spark.rdd.ShuffledRDD -case class Exchange(newPartitioning: Distribution, numPartitions: Int, child: SharkPlan) +case class Exchange(newPartitioning: Partitioning, child: SharkPlan) extends UnaryNode { override def outputPartitioning = newPartitioning @@ -17,17 +18,17 @@ case class Exchange(newPartitioning: Distribution, numPartitions: Int, child: Sh def execute() = attachTree(this , "execute") { newPartitioning match { - case ClusteredDistribution(expressions) => + case HashPartitioning(expressions, width) => // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().map { row => (buildRow(expressions.toSeq.map(Evaluate(_, Vector(row)))), row) } - val part = new HashPartitioner(numPartitions) + val part = new HashPartitioner(width) val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) shuffled.map(_._2) - case OrderedDistribution(sortingExpressions) => + case RangePartitioning(sortingExpressions, width) => val directions = sortingExpressions.map(_.direction).toIndexedSeq val dataTypes = sortingExpressions.map(_.dataType).toIndexedSeq @@ -94,30 +95,49 @@ case class Exchange(newPartitioning: Distribution, numPartitions: Int, child: Sh (sortKey, row) } - val part = new RangePartitioner(numPartitions, rdd, ascending = true) + val part = new RangePartitioner(width, rdd, ascending = true) val shuffled = new ShuffledRDD[SortKey, Row, (SortKey, Row)](rdd, part) shuffled.map(_._2) - case UnknownDistribution => - logger.warn("Worthless repartitioning requested.") - child.execute() + case _ => sys.error("Not implemented") } } } object AddExchange extends Rule[SharkPlan] { // TODO: determine the number of partitions. - // TODO: We need to consider the number of partitions to determine if we - // will add an Exchange operator. If a dataset only has a single partition, - // even if needExchange returns true, we do not need to shuffle the data again. val numPartitions = 8 def apply(plan: SharkPlan): SharkPlan = plan.transformUp { case operator: SharkPlan => - operator.withNewChildren(operator.requiredChildPartitioning.zip(operator.children).map { - case (required, child) if !child.outputPartitioning.satisfies(required) => - Exchange(required, numPartitions, child) - case (_, child) => child - }) + def meetsRequirements = + !operator.requiredChildDistribution.zip(operator.children).map { + case (required, child) => !child.outputPartitioning.satisfies(required) + }.exists(_ == false) + + // TODO ASUUMES TRANSITIVITY? + def compatible = + !operator.children + .map(_.outputPartitioning) + .sliding(2) + .map { + case Seq(a) => true + case Seq(a,b) => a compatibleWith b + }.exists(_ == false) + + + if (false && meetsRequirements && compatible) { + operator + } else { + val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { + case (ClusteredDistribution(clustering), child) => + Exchange(HashPartitioning(clustering, 8), child) + case (OrderedDistribution(ordering), child) => + Exchange(RangePartitioning(ordering, 8), child) + case (UnknownDistribution, child) => child + case (dist, _) => sys.error(s"Don't know how to ensure $dist") + } + operator.withNewChildren(repartitionedChildren) + } } } \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index fc46b1494d8a5..ba34a3c178e77 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -4,14 +4,16 @@ package execution import org.apache.spark.rdd.RDD import plans.QueryPlan +import plans.physical._ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => + // TODO: Move to `DistributedPlan` /** Specifies how data is partitioned across different nodes in the cluster. */ - def outputPartitioning: Distribution = UnknownDistribution + def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG! /** Specifies any partition requirements on the input data for this operator. */ - def requiredChildPartitioning: Seq[Distribution] = Seq.fill(children.size)(UnknownDistribution) + def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnknownDistribution) /** * Runs this query returning the result as an RDD. diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 89fa0912bb147..99a67d5c0ff64 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,6 +3,7 @@ package execution import catalyst.errors._ import catalyst.expressions._ +import catalyst.plans.physical._ import org.apache.spark.rdd.SharkPairRDDFunctions /* Implicits */ @@ -16,7 +17,7 @@ case class Aggregate( extends UnaryNode { val requiredPartitioning = ClusteredDistribution(groupingExpressions) - override def requiredChildPartitioning = Seq(requiredPartitioning, requiredPartitioning) + override def requiredChildDistribution = Seq(requiredPartitioning, requiredPartitioning) def output = aggregateExpressions.map(_.toAttribute) diff --git a/src/main/scala/catalyst/execution/dataProperties.scala b/src/main/scala/catalyst/execution/dataProperties.scala index 2b67dd4242251..e69de29bb2d1d 100644 --- a/src/main/scala/catalyst/execution/dataProperties.scala +++ b/src/main/scala/catalyst/execution/dataProperties.scala @@ -1,106 +0,0 @@ -package catalyst -package execution - -import catalyst.expressions.{SortOrder, Expression} - -trait Partitioned { - def expressions: Seq[Expression] - - /** - * If we need an [[catalyst.execution.Exchange]] to re-partition data for - * the given [[catalyst.execution.Partitioned]] other. - * @param other The given [[catalyst.execution.Partitioned]]. - *@return - */ - // TODO: We should also consider functional dependencies between expressions of - // two data properties. For example, if we have a GroupProperty(a) and a - // GroupProperty(a+1), we will not need an exchange to re-partition the data. - def needExchange(other: Partitioned): Boolean -} - -/** - * An implementation of [[catalyst.execution.Partitioned]] represents that - * the data property of a dataset is not specified. - * If it is used as a required data property for a physical operator - * (a [[catalyst.execution.SharkPlan]]) (e.g. [[catalyst.execution.Project]]), - * it means that this operator does not require its input datasets to be - * organized in a certain way. - */ -case class NotSpecified() extends Partitioned { - def expressions = Nil - - def needExchange(other: Partitioned): Boolean = { - other match { - case NotSpecified() => false - case HashPartitioned(groupingExpressions) => true - case RangePartitioned(_) => true - } - } -} - -/** - * An implementation of [[catalyst.execution.Partitioned]] represents that - * a dataset is grouped by groupingExpressions. - * @param groupingExpressions The expressions used to specify the way how rows should be grouped. - * If it is a Nil, the entire dataset is considered as a single group. - * In this case, a single reducer will be used. - */ -case class HashPartitioned(groupingExpressions: Seq[Expression]) extends Partitioned { - override val expressions = groupingExpressions - - def needExchange(other: Partitioned): Boolean = { - other match { - case NotSpecified() => false - // We do not need an Exchange operator if another GroupProperty only - // needs to group rows within a partition. - case g @ HashPartitioned(otherExpressions) => { - if (expressions.toSet.subsetOf(otherExpressions.toSet)) false else true - } - // Because we use [[org.apache.spark.HashPartitioner]] for GroupProperty, - // we need to use an Exchange operator to sort data with a - // [[org.apache.spark.RangePartitioner]]. But, if the groupingExpressions is - // a Nil, we will have a single partition. So, we do not need an Exchange operator - // to sort this single partition. - case RangePartitioned(otherExpressions) => if (expressions == Nil) false else true - } - } -} - -/** - * An implementation of [[catalyst.execution.Partitioned]] represents that - * a dataset is sorted by sortingExpressions. A SortProperty also implies that - * the dataset is grouped by sortingExpressions. - * @param sortingExpressions The expressions used to specify the way how rows should be sorted. - * sortingExpressions should not be empty. - */ -case class RangePartitioned(sortingExpressions: Seq[SortOrder]) extends Partitioned { - - { - if (sortingExpressions == Nil) { - throw new IllegalArgumentException("Sorting expressions of a SortProperty " + - "are not specified.") - } - } - - def expressions = sortingExpressions - - def needExchange(other: Partitioned): Boolean = { - other match { - case NotSpecified() => false - // A SortProperty implies a GroupProperty. We do not need an Exchange operator - // if the GroupProperty only needs to group rows within a partition. - case g @ HashPartitioned(otherExpressions) => { - if (expressions.map(expression => expression.child). - toSet.subsetOf(otherExpressions.toSet)) false else true - } - // We do not need an Exchange operator if another SortProperty only needs to - // sort rows within a partition (cases satisfying otherExpressions.startsWith(expressions)) - // or we do not need to sort again (cases satisfying - // expressions.startsWith(otherExpressions)). - case s @ RangePartitioned(otherExpressions) => { - if (otherExpressions.startsWith(expressions) || - expressions.startsWith(otherExpressions)) false else true - } - } - } -} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/distribution.scala b/src/main/scala/catalyst/execution/distribution.scala index 4ffd5a3f61cf7..e69de29bb2d1d 100644 --- a/src/main/scala/catalyst/execution/distribution.scala +++ b/src/main/scala/catalyst/execution/distribution.scala @@ -1,88 +0,0 @@ -package catalyst -package execution - -import expressions.{Expression, SortOrder} -import types._ - -/** - * Specifies how tuples that share common expressions will be distributed. Distribution can - * be used to refer to two distinct physical properties: - * - Inter-node partitioning of data: In this case the distribution describes how tuples are - * partitioned across physical machines in a cluster. Knowing this property allows some - * operators (e.g., [[Aggregate]]) to perform partition local operations instead of global ones. - * - Intra-partition ordering of data: In this case the distribution describes guarantees made - * about how tuples are distributed within a single partition. - * - * TOOD(marmbrus): Given the way that Spark does partitioning, I think the order of the grouping - * actually does matter, and thus our subset checking is probably not sufficient - * to ensure correct colocation for joins.? - * TODO(marmbrus): Similarly I'm not sure that satisfies is a sufficient check to see if an sort - * aggregation can be done. Maybe we need two checks? Separate ordering from - * partitioning? - */ -abstract sealed class Distribution extends Expression { - self: Product => - - /** - * Returns true iff the guarantees made by this [[Distribution]] are sufficient to satisfy all - * guarantees mandated by the `required` distribution. - */ - def satisfies(required: Distribution): Boolean -} - -/** - * Represents a distribution where no promises are made about co-location of data. - */ -case object UnknownDistribution extends Distribution with trees.LeafNode[Expression] { - def references = Set.empty - def nullable = false - def dataType = NullType - - def satisfies(required: Distribution): Boolean = required == UnknownDistribution -} - -/** - * Represents data where tuples that share the same values for the `clustering` [[Expression]]s will - * be co-located. Based on the context, this can mean such tuples are either co-located in the same - * partition or they will be contiguous within a single partition. - */ -case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution { - def children = clustering.toSeq - def references = clustering.flatMap(_.references).toSet - def nullable = false - def dataType = NullType - - def satisfies(required: Distribution): Boolean = required match { - case UnknownDistribution => true - // No clustering expressions means only one partition. - case _ if clustering.isEmpty => true - case ClusteredDistribution(requiredClustering) => - clustering.toSet.subsetOf(requiredClustering.toSet) - case _ => false - } -} - -/** - * Represents data where tuples have been ordered according to the `ordering` [[Expression]]s. This - * is a strictly stronger guarantee than [[ClusteredDistribution]] as an ordering will ensure that - * tuples that share the same value for the ordering expressions are contiguous and will never be - * split across partitions. - */ -case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { - def children = ordering - def references = ordering.flatMap(_.references).toSet - def nullable = false - def dataType = NullType - - def clustering = ordering.map(_.child).toSet - - def satisfies(required: Distribution): Boolean = required match { - case UnknownDistribution => true - case OrderedDistribution(requiredOrdering) => - val minSize = Seq(requiredOrdering.size, ordering.size).min - requiredOrdering.take(minSize) == ordering.take(minSize) - case ClusteredDistribution(requiredClustering) => - clustering.subsetOf(requiredClustering.toSet) - case _ => false - } -} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 6f811f20a737d..380f64dee8753 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -8,6 +8,7 @@ import org.apache.spark.rdd.RDD import errors._ import expressions._ import plans._ +import plans.physical._ import org.apache.spark.rdd.SharkPairRDDFunctions._ @@ -18,7 +19,7 @@ case class SparkEquiInnerJoin( right: SharkPlan) extends BinaryNode { - override def requiredChildPartitioning = + override def requiredChildDistribution= ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil def output = left.output ++ right.output diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index bcce25697ac84..5cdb7c3016275 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -5,6 +5,7 @@ import expressions._ import planning._ import plans._ import plans.logical.LogicalPlan +import plans.physical._ trait PlanningStrategies { self: QueryPlanner[SharkPlan] => @@ -137,7 +138,7 @@ trait PlanningStrategies { case logical.Sort(sortExprs, child) => // First repartition then sort locally. execution.SortPartitions(sortExprs, - execution.Exchange(OrderedDistribution(sortExprs), 8, planLater(child))) :: Nil + execution.Exchange(RangePartitioning(sortExprs, 8), planLater(child))) :: Nil case logical.SortPartitions(sortExprs, child) => execution.SortPartitions(sortExprs, planLater(child)) :: Nil case logical.Project(projectList, child) => diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala new file mode 100644 index 0000000000000..2c5e15d6a7fe9 --- /dev/null +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -0,0 +1,143 @@ +package catalyst +package plans +package physical + +import expressions._ +import types._ + +/** + * Specifies how tuples that share common expressions will be distributed. Distribution can + * be used to refer to two distinct physical properties: + * - Inter-node partitioning of data: In this case the distribution describes how tuples are + * partitioned across physical machines in a cluster. Knowing this property allows some + * operators (e.g., [[Aggregate]]) to perform partition local operations instead of global ones. + * - Intra-partition ordering of data: In this case the distribution describes guarantees made + * about how tuples are distributed within a single partition. + * + * TOOD(marmbrus): Given the way that Spark does partitioning, I think the order of the grouping + * actually does matter, and thus our subset checking is probably not sufficient + * to ensure correct colocation for joins.? + * TODO(marmbrus): Similarly I'm not sure that satisfies is a sufficient check to see if an sort + * aggregation can be done. Maybe we need two checks? Separate ordering from + * partitioning? + */ +sealed trait Distribution { +} + +/** + * Represents a distribution where no promises are made about co-location of data. + */ +case object UnknownDistribution extends Distribution + +/** + * Represents a distribution where a single operation can observe all tuples in the dataset. + */ +case object AllTuples extends Distribution + +/** + * Represents data where tuples that share the same values for the `clustering` [[Expression]]s will + * be co-located. Based on the context, this can mean such tuples are either co-located in the same + * partition or they will be contiguous within a single partition. + */ +case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution + +/** + * Represents data where tuples have been ordered according to the `ordering` [[Expression]]s. This + * is a strictly stronger guarantee than [[ClusteredDistribution]] as an ordering will ensure that + * tuples that share the same value for the ordering expressions are contiguous and will never be + * split across partitions. + */ +case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { + def clustering = ordering.map(_.child).toSet +} + +sealed abstract trait Partitioning { + self: Product => + + val width: Int + + /** + * Returns true iff the guarantees made by this [[Distribution]] are sufficient to satisfy all + * guarantees mandated by the `required` distribution. + */ + def satisfies(required: Distribution): Boolean + + def compatibleWith(other: Partitioning): Boolean +} + +case class UnknownPartitioning(width: Int) extends Partitioning { + def satisfies(required: Distribution): Boolean = required match { + case UnknownDistribution => true + case _ => false + } + def compatibleWith(other: Partitioning): Boolean = other match { + case UnknownPartitioning(_) => true + case _ => false + } +} + +case object Unpartitioned extends Partitioning { + val width = 1 + + override def satisfies(required: Distribution): Boolean = true + override def compatibleWith(other: Partitioning) = other match { + case Unpartitioned => true + case _ => false + } +} + +case object Broadcast extends Partitioning { + val width = 1 + + override def satisfies(required:Distribution): Boolean = true + override def compatibleWith(other: Partitioning) = other match { + case Unpartitioned => true + case _ => false + } +} + +case class HashPartitioning(expressions: Seq[Expression], width: Int) extends Expression with Partitioning { + def children = expressions.toSeq + def references = expressions.flatMap(_.references).toSet + def nullable = false + def dataType = IntegerType + + lazy val clusteringSet = expressions.toSet + + def satisfies(required: Distribution): Boolean = required match { + case UnknownDistribution => true + case ClusteredDistribution(requiredClustering) => + clusteringSet.subsetOf(requiredClustering.toSet) + case _ => false + } + override def compatibleWith(other: Partitioning) = other match { + case Broadcast => true + case h: HashPartitioning if h == this => true + case _ => false + } +} + +case class RangePartitioning(ordering: Seq[SortOrder], width: Int) extends Expression with Partitioning { + def children = ordering.toSeq + def references = ordering.flatMap(_.references).toSet + def nullable = false + def dataType = IntegerType + + lazy val clusteringSet = ordering.map(_.child).toSet + + def satisfies(required: Distribution): Boolean = required match { + case UnknownDistribution => true + case OrderedDistribution(requiredOrdering) => + val minSize = Seq(requiredOrdering.size, ordering.size).min + requiredOrdering.take(minSize) == ordering.take(minSize) + case ClusteredDistribution(requiredClustering) => + clusteringSet.subsetOf(requiredClustering.toSet) + case _ => false + } + + def compatibleWith(other: Partitioning) = other match { + case Broadcast => true + case r: RangePartitioning if r == this => true + case _ => false + } +} \ No newline at end of file diff --git a/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala b/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala index 7a7a6b61a05b0..a7b639aa61186 100644 --- a/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala +++ b/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala @@ -1,7 +1,7 @@ package org.apache.spark.rdd import org.apache.spark._ -import org.apache.spark.util.AppendOnlyMap +import org.apache.spark.util.collection.AppendOnlyMap import scala.collection.mutable.ArrayBuffer /** diff --git a/src/test/scala/catalyst/execution/DataPropertyTests.scala b/src/test/scala/catalyst/execution/DataPropertyTests.scala index 933897cff3922..faf9c5504065e 100644 --- a/src/test/scala/catalyst/execution/DataPropertyTests.scala +++ b/src/test/scala/catalyst/execution/DataPropertyTests.scala @@ -5,6 +5,8 @@ import catalyst.dsl._ import org.scalatest.FunSuite class DistributionTests extends FunSuite { + /* + protected def checkSatisfied( inputDistribution: Distribution, requiredDistribution: Distribution, @@ -150,4 +152,6 @@ class DistributionTests extends FunSuite { ClusteredDistribution(Nil), false) } + + */ } \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index afd73c0eb0fd0..c98b2ebb50f6e 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -5,8 +5,10 @@ import java.io._ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} import frontend.hive.{ExplainCommand, Command} +import plans.physical._ import util._ + /** * Allows the creations of tests that execute the same query against both hive * and catalyst, comparing the results. @@ -101,7 +103,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G case _: Command => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _ => // TODO: Really we only care about the final total ordering here... - val isOrdered = sharkQuery.executedPlan.collect { case s @ Exchange(OrderedDistribution(_), _, _) => s}.nonEmpty + val isOrdered = sharkQuery.executedPlan.collect { case s @ Exchange(r: RangePartitioning, _) => s}.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. if (!isOrdered) answer.sorted else answer } From 658866e86d42d7fa91d8b84821503d44a78bf8c1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 20 Jan 2014 14:34:56 -0800 Subject: [PATCH 506/778] Pull back in changes made by @yhuai eliminating CoGroupedLocallyRDD.scala --- .../spark/rdd/CoGroupedLocallyRDD.scala | 85 ------------------- .../spark/rdd/SharkPairRDDFunctions.scala | 28 ++++-- 2 files changed, 21 insertions(+), 92 deletions(-) delete mode 100644 src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala diff --git a/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala b/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala deleted file mode 100644 index a7b639aa61186..0000000000000 --- a/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala +++ /dev/null @@ -1,85 +0,0 @@ -package org.apache.spark.rdd - -import org.apache.spark._ -import org.apache.spark.util.collection.AppendOnlyMap -import scala.collection.mutable.ArrayBuffer - -/** - * A RDD that cogroups its already co-partitioned parents. This RDD works the same as - * a [[org.apache.spark.rdd.CoGroupedRDD]] except that its parents should have the - * same number of partitions. Like a [[org.apache.spark.rdd.CoGroupedRDD]], - * for each key k in parent RDDs, the resulting RDD contains a tuple with the list of - * values for that key. - * - * @param rdds parent RDDs. - */ -class CoGroupedLocallyRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]]) - extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { - - { - // Check if all parents have the same number of partitions. - // It is possible that a parent RDD does not preserve the partitioner, - // so we do not check if all of parent RDDs have the same partitioner. - if (!rdds.forall(rdd => rdd.partitions.size == rdds(0).partitions.size)) { - throw new IllegalArgumentException( - "All parent RDDs should have the same number of partitions.") - } - } - - // All dependencies of a CoGroupedLocallyRDD should be narrow dependencies. - override def getDependencies: Seq[Dependency[_]] = { - rdds.map { rdd: RDD[_ <: Product2[K, _]] => - logDebug("Adding one-to-one dependency with " + rdd) - new OneToOneDependency(rdd) - } - } - - override def getPartitions: Array[Partition] = { - val numPartitions = firstParent[(K, _)].partitions.size - val array = new Array[Partition](numPartitions) - for (i <- 0 until array.size) { - // Each CoGroupPartition will have a dependency per contributing RDD - array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) => - new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) - }.toArray) - } - array - } - - // Take the first not None partitioner. It is possible that all parent partitioners - // are None. - override val partitioner = rdds.find(rdd => rdd.partitioner != None) match { - case Some(rdd) => rdd.partitioner - case None => None - } - - override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { - val split = s.asInstanceOf[CoGroupPartition] - val numRdds = split.deps.size - // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) - val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] - - val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { - if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) - } - - val getSeq = (k: K) => { - map.changeValue(k, update) - } - - for ((dep, depNum) <- split.deps.zipWithIndex) dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { - // Read them from the parent - rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { kv => - getSeq(kv._1)(depNum) += kv._2 - } - } - case _ => { - // We should not reach here. It is a sanity check. - throw new RuntimeException("A dependency of this CoGroupedLocallyRDD is not " + - "a narrow dependency.") - } - } - new InterruptibleIterator(context, map.iterator) - } -} \ No newline at end of file diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala index d26a43ac5720b..67ecac8e90670 100644 --- a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala +++ b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -7,6 +7,7 @@ import org.apache.spark.Aggregator import org.apache.spark.SparkContext._ import scala.language.implicitConversions +import org.apache.spark.util.collection.AppendOnlyMap /** * Extra functions for Shark available on RDDs of (key, value) pairs through an implicit conversion. @@ -25,11 +26,24 @@ class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * This function will not introduce a shuffling operation. */ def cogroupLocally[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { - val cg = new CoGroupedLocallyRDD[K](Seq(self, other)) - val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classTag[K], ClassTags.seqSeqClassTag) - prfs.mapValues { case Seq(vs, ws) => - (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) - } + val cg = self.zipPartitions(other)((iter1:Iterator[(K, V)], iter2:Iterator[(K, W)]) => { + val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] + + val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { + if (hadVal) oldVal else Array.fill(2)(new ArrayBuffer[Any]) + } + + val getSeq = (k: K) => { + map.changeValue(k, update) + } + + iter1.foreach { kv => getSeq(kv._1)(0) += kv._2 } + iter2.foreach { kv => getSeq(kv._1)(1) += kv._2 } + + map.iterator + }).mapValues { case Seq(vs, ws) => (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])} + + cg } /** @@ -53,8 +67,8 @@ class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * This function will not introduce a shuffling operation. */ def joinLocally[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { - cogroupLocally(other).flatMapValues { case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + cogroupLocally(other).flatMapValues { + case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) } } } From f2fdd7733bf4fcfb8cd37f31aeed2bf2cf521317 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 20 Jan 2014 15:12:10 -0800 Subject: [PATCH 507/778] fix required distribtion for aggregate. --- src/main/scala/catalyst/execution/aggregates.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 99a67d5c0ff64..a256c5f979dd8 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -16,8 +16,7 @@ case class Aggregate( child: SharkPlan) extends UnaryNode { - val requiredPartitioning = ClusteredDistribution(groupingExpressions) - override def requiredChildDistribution = Seq(requiredPartitioning, requiredPartitioning) + override def requiredChildDistribution = Seq(ClusteredDistribution(groupingExpressions)) def output = aggregateExpressions.map(_.toAttribute) From 05d3a0dae1fed23db5a2b0e8cf3f1fba5f11790b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 20 Jan 2014 17:01:15 -0800 Subject: [PATCH 508/778] Refactor to avoid serializing ordering details with every row. --- .../catalyst/execution/basicOperators.scala | 10 ++- src/main/scala/catalyst/expressions/Row.scala | 61 ++++++------------- 2 files changed, 27 insertions(+), 44 deletions(-) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 2bd91b9cf17f4..053401223ca62 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -7,7 +7,6 @@ import types._ import org.apache.spark.SparkContext._ - case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -53,9 +52,14 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { // TODO: Don't include redundant expressions in both sortKey and row. def execute() = attachTree(this, "sort") { + import scala.math.Ordering.Implicits._ + implicit val ordering = new RowOrdering(sortExprs) + + // TODO: Allows spark to take the ordering as an argument, avoid needless pair creation. child.execute() - .mapPartitions(OrderedRow(sortExprs, _)) - .sortByKey(ascending = true, numPartitions).map(_._2) + .mapPartitions(iter => iter.map(row => (row, null))) + .sortByKey(ascending = true, 8) + .map(_._1) } def output = child.output diff --git a/src/main/scala/catalyst/expressions/Row.scala b/src/main/scala/catalyst/expressions/Row.scala index 670221848d827..2ea4d36224421 100644 --- a/src/main/scala/catalyst/expressions/Row.scala +++ b/src/main/scala/catalyst/expressions/Row.scala @@ -80,48 +80,27 @@ class GenericRow(input: Seq[Any]) extends Row { } } -object OrderedRow { - def apply(ordering: Seq[SortOrder], input: Iterator[Row]): Iterator[(OrderedRow, Row)] = { - val expressions = ordering.map(_.child) - val orderingObjects = ordering.map { o => - o.dataType match { - case nativeType: NativeType => - if (o.direction == Ascending) - nativeType.ordering.asInstanceOf[Ordering[Any]] - else - nativeType.ordering.asInstanceOf[Ordering[Any]].reverse - case _ => sys.error(s"No ordering available for ${o.dataType}") - } - } - val directions = ordering.map(_.direction) - - input.map { row => - (new OrderedRow(orderingObjects, directions, expressions.map(Evaluate(_, Vector(row)))), row) - } - } -} - -class OrderedRow(ordering: Seq[Ordering[Any]], directions: Seq[SortDirection], input: Seq[Any]) - extends GenericRow(input) with Ordered[OrderedRow] { - - def compare(other: OrderedRow): Int = { - var i = 0 - while (i < values.size) { - val left = values(i) - val right = other.values(i) - - val comparison = - if (left == null && right == null) { - 0 - } else if (left == null) { - if (directions(i) == Ascending) -1 else 1 - } else if (right == null) { - if (directions(i) == Ascending) 1 else -1 - } else { - ordering(i).compare(left, right) +class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { + def compare(a: Row, b: Row): Int = { + ordering.foreach { order => + val left = Evaluate(order.child, Vector(a)) + val right = Evaluate(order.child, Vector(b)) + + if (left == null && right == null) { + // Both null, continue looking. + } else if (left == null) { + return if (order.direction == Ascending) -1 else 1 + } else if (right == null) { + return if (order.direction == Ascending) 1 else -1 + } else { + val comparison = order.dataType match { + case n: NativeType if order.direction == Ascending => + n.ordering.asInstanceOf[Ordering[Any]].compare(left, right) + case n: NativeType if order.direction == Descending => + n.ordering.asInstanceOf[Ordering[Any]].reverse.compare(left, right) } - if (comparison != 0) return comparison - i += 1 + if (comparison != 0) return comparison + } } return 0 } From c800798f258ad48ec2239a08c01947b6ca577f99 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 21 Jan 2014 19:37:11 -0800 Subject: [PATCH 509/778] Add build status icon. --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index b7987f0140f23..10e1353a82169 100644 --- a/README.md +++ b/README.md @@ -2,6 +2,8 @@ catalyst ======== Catalyst is a functional framework for optimizing relational query plans. +[![Build Status](https://databricks.ci.cloudbees.com/buildStatus/icon?job=Catalyst-Master)](https://databricks.ci.cloudbees.com/job/Catalyst-Master/) + More documentation can be found in the project's [scaladoc](http://databricks.github.io/catalyst/latest/api/#catalyst.package) Getting Started From 6cd442b48b89e76ccb64c07156caf03152ef0d1b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 21 Jan 2014 20:25:08 -0800 Subject: [PATCH 510/778] Use numPartitions variable, fix grammar. --- src/main/scala/catalyst/execution/basicOperators.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 053401223ca62..b2de473e549f4 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -55,10 +55,10 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { import scala.math.Ordering.Implicits._ implicit val ordering = new RowOrdering(sortExprs) - // TODO: Allows spark to take the ordering as an argument, avoid needless pair creation. + // TODO: Allow spark to take the ordering as an argument, also avoid needless pair creation. child.execute() .mapPartitions(iter => iter.map(row => (row, null))) - .sortByKey(ascending = true, 8) + .sortByKey(ascending = true, numPartitions) .map(_._1) } From f143f6128d08dd65c1f8dd61cac00f911df92d55 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 21 Jan 2014 21:45:53 -0800 Subject: [PATCH 511/778] Implement sampling. Fixes a flaky test where the JVM notices that RAND as a Comparison method "violates its general contract!" --- src/main/scala/catalyst/dsl.scala | 3 +++ src/main/scala/catalyst/execution/basicOperators.scala | 7 +++++++ src/main/scala/catalyst/execution/planningStrategies.scala | 2 ++ src/main/scala/catalyst/frontend/Hive.scala | 2 +- src/main/scala/catalyst/plans/logical/basicOperators.scala | 2 +- src/test/scala/catalyst/execution/DslQueryTests.scala | 2 +- src/test/scala/catalyst/execution/HiveQueryTests.scala | 4 ---- 7 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/main/scala/catalyst/dsl.scala b/src/main/scala/catalyst/dsl.scala index fb73630876ace..51b717bc0cbe5 100644 --- a/src/main/scala/catalyst/dsl.scala +++ b/src/main/scala/catalyst/dsl.scala @@ -114,6 +114,9 @@ package object dsl { def filter(dynamicUdf: (DynamicRow) => Boolean) = Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(plan.output))), plan) + def sample(fraction: Double, withReplacement: Boolean = true) = + Sample(fraction, withReplacement, plan) + def analyze = analysis.SimpleAnalyzer(plan) } } \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index b2de473e549f4..4ffb393ffcf58 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -22,6 +22,13 @@ case class Filter(condition: Expression, child: SharkPlan) extends UnaryNode { } } +case class Sample(fraction: Double, withReplacement: Boolean, child: SharkPlan) extends UnaryNode { + def output = child.output + + // TODO: How to pick seed? + def execute() = child.execute().sample(withReplacement, fraction, (math.random * 1000).toInt) +} + case class Union(left: SharkPlan, right: SharkPlan)(@transient sc: SharkContext) extends BinaryNode { // TODO: attributes output by union should be distinct for nullability purposes diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 2d00f3ccb7972..88360c6f09f74 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -145,6 +145,8 @@ trait PlanningStrategies { execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => execution.Aggregate(group, agg, planLater(child)) :: Nil + case logical.Sample(fraction, withReplacement, child) => + execution.Sample(fraction, withReplacement, planLater(child)) :: Nil case logical.LocalRelation(output, data) => execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 2e24b50a56a40..b66c1f718e38e 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -507,7 +507,7 @@ object HiveQl { case Token("TOK_TABLEBUCKETSAMPLE", Token(numerator, Nil) :: Token(denominator, Nil) :: Nil) => - Sample(numerator.toDouble / denominator.toDouble, relation) + Sample(numerator.toDouble / denominator.toDouble, false, relation) }.getOrElse(relation) case Token("TOK_UNIQUEJOIN", joinArgs) => diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index 1f0f5d15c21e6..f22d3bdbd867a 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -68,7 +68,7 @@ case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { def references = Set.empty } -case class Sample(percentage: Double, child: LogicalPlan) extends UnaryNode { +case class Sample(fraction: Double, withReplacement: Boolean, child: LogicalPlan) extends UnaryNode { def output = child.output def references = Set.empty } diff --git a/src/test/scala/catalyst/execution/DslQueryTests.scala b/src/test/scala/catalyst/execution/DslQueryTests.scala index 3f4eae929ca85..834f4849f4947 100644 --- a/src/test/scala/catalyst/execution/DslQueryTests.scala +++ b/src/test/scala/catalyst/execution/DslQueryTests.scala @@ -84,7 +84,7 @@ class DslQueryTests extends FunSuite with BeforeAndAfterAll { } test("random sample") { - testData.where(Rand > 0.5).orderBy(Rand.asc).toRdd.collect() + testData.sample(0.5).toRdd.collect() } test("sorting") { diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala index b6e3b54eefe51..920b875c149d4 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -19,10 +19,6 @@ class HiveQueryTests extends HiveComparisonTest { createQueryTest("string literal", "SELECT 'test' FROM src") - test("Run random sample") { // Since this is non-deterministic we just check to make sure it runs for now. - "SELECT key, value FROM src WHERE RAND() > 0.5 ORDER BY RAND()".q.stringResult() - } - createQueryTest("Escape sequences", """SELECT key, '\\\t\\' FROM src WHERE key = 86""") From 81a109d92dad3db41caf0874edc90c924f86da81 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 21 Jan 2014 21:53:03 -0800 Subject: [PATCH 512/778] fix link. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 10e1353a82169..a20cfd8e01274 100644 --- a/README.md +++ b/README.md @@ -75,7 +75,7 @@ Project {key#0,value#1} HiveTableScan {key#0,value#1}, (MetastoreRelation default, src, None) ``` -From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](http://marmbrus.github.io/catalyst/latest/api/index.html#catalyst.trees.TreeNode) objects. +From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](http://databricks.github.io/catalyst/latest/api/#catalyst.trees.TreeNode) objects. ```scala scala> query.optimizedPlan res1: catalyst.plans.logical.LogicalPlan = From 95ab38208dad27f1f9a32f0b98f94316152da9bf Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 22 Jan 2014 19:24:33 -0800 Subject: [PATCH 513/778] Use resolved instead of custom function. This is better because some nodes override the notion of resolved. --- src/main/scala/catalyst/analysis/Analyzer.scala | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index ae403e6bcf5db..8bd905f56f29c 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -76,7 +76,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool */ object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case q: LogicalPlan if childIsFullyResolved(q) => + case q: LogicalPlan if q.childrenResolved => logger.trace(s"Attempting to resolve ${q.simpleString}") q transformExpressions { case u @ UnresolvedAttribute(name) => @@ -125,7 +125,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool object StarExpansion extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Wait until children are resolved - case p: LogicalPlan if !childIsFullyResolved(p) => p + case p: LogicalPlan if !p.childrenResolved => p // If the projection list contains Stars, expand it. case p @ Project(projectList, child) if containsStar(projectList) => Project( @@ -150,10 +150,4 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool protected def containsStar(exprs: Seq[NamedExpression]): Boolean = exprs.collect { case _: Star => true }.nonEmpty } - - /** - * Returns true if all the inputs to the given LogicalPlan node are resolved and non-empty. - */ - protected def childIsFullyResolved(plan: LogicalPlan): Boolean = - (!plan.inputSet.isEmpty) && plan.inputSet.map(_.resolved).reduceLeft(_ && _) } From b892e32ded0ce72f24190954559f0261d5ffb0da Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 22 Jan 2014 19:24:47 -0800 Subject: [PATCH 514/778] Union is not resolved until the types match up. --- src/main/scala/catalyst/plans/logical/basicOperators.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index 1f0f5d15c21e6..d153b72e9448f 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -17,6 +17,11 @@ case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { // TODO: These aren't really the same attributes as nullability etc might change. def output = left.output + + override lazy val resolved = + childrenResolved && + left.output.zip(right.output).filter { case (l,r) => l.dataType != r.dataType }.isEmpty + def references = Set.empty } From 53b95f84d07178e6cac2f0c125006ebf5f4e10a2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 22 Jan 2014 19:25:13 -0800 Subject: [PATCH 515/778] coercion should not occur until children are resolved. --- src/main/scala/catalyst/analysis/typeCoercion.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index 74f10723a7cd0..f3b1fbb937d21 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -157,6 +157,9 @@ object BooleanComparisons extends Rule[LogicalPlan] { */ object BooleanCasts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + case Cast(e, BooleanType) => Not(Equals(e, Literal(0))) case Cast(e, dataType) if e.dataType == BooleanType => Cast(If(e, Literal(1), Literal(0)), dataType) @@ -170,6 +173,9 @@ object BooleanCasts extends Rule[LogicalPlan] { */ object StringToIntegralCasts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + case Cast(e @ StringType(), t: IntegralType) => Cast(Cast(e, DecimalType), t) } From 3f547b876e716821b5c1427b480d2037c03f8c83 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 22 Jan 2014 19:27:01 -0800 Subject: [PATCH 516/778] Add support for widening types in unions. --- .../scala/catalyst/analysis/Analyzer.scala | 8 +- .../catalyst/analysis/typeCoercion.scala | 76 +++++++++++++++---- 2 files changed, 65 insertions(+), 19 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 8bd905f56f29c..1c5fd0f2af3c8 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -32,13 +32,11 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ResolveReferences, ResolveRelations, StarExpansion, - ResolveFunctions), - Batch("Aggregation", Once, - GlobalAggregates), - Batch("Type Coersion", fixedPoint, + ResolveFunctions, + GlobalAggregates, StringToIntegralCasts, BooleanCasts, - PromoteNumericTypes, + WidenTypes, PromoteStrings, ConvertNaNs, BooleanComparisons, diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala index f3b1fbb937d21..1dadc908ba7c1 100644 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ b/src/main/scala/catalyst/analysis/typeCoercion.scala @@ -76,39 +76,87 @@ object ConvertNaNs extends Rule[LogicalPlan] { * - TINYINT, SMALLINT, and INT can all be converted to FLOAT. * - BOOLEAN types cannot be converted to any other type. * - * String conversions are handled by the PromoteStrings rule. + * Additionally, all types when UNION-ed with strings will be promoted to strings. + * Other string conversions are handled by PromoteStrings */ -object PromoteNumericTypes extends Rule[LogicalPlan] { +object WidenTypes extends Rule[LogicalPlan] { val integralPrecedence = Seq(NullType, ByteType, ShortType, IntegerType, LongType) val toDouble = integralPrecedence ++ Seq(NullType, FloatType, DoubleType) val toFloat = Seq(NullType, ByteType, ShortType, IntegerType) :+ FloatType val allPromotions = integralPrecedence :: toDouble :: toFloat :: Nil def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case u @ Union(left, right) if u.childrenResolved && !u.resolved => + val castedInput = left.output.zip(right.output).map { + // When a string is found on one side, make the other side a string too. + case (l,r) if l.dataType == StringType && r.dataType != StringType => + (l, Alias(Cast(r, StringType), r.name)()) + case (l,r) if l.dataType != StringType && r.dataType == StringType => + (Alias(Cast(l, StringType), l.name)(), r) + + case (l,r) if l.dataType != r.dataType => + logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") + findWidestType(l.dataType, r.dataType).map { widestType => + val newLeft = + if (l.dataType == widestType) + l + else + Alias(Cast(l, widestType), l.name)() + val newRight = + if (r.dataType == widestType) + r + else + Alias(Cast(r, widestType), r.name)() + + (newLeft, newRight) + }.getOrElse((l,r)) // If there is no applicable conversion, leave expression unchanged. + case other => other + } + + val (castedLeft, castedRight) = castedInput.unzip + + val newLeft = + if(castedLeft.map(_.dataType) != left.output.map(_.dataType)) { + logger.debug(s"Widening numeric types in union ${castedLeft} ${left.output}") + Project(castedLeft, left) + } else { + left + } + + val newRight = + if(castedRight.map(_.dataType) != right.output.map(_.dataType)) { + logger.debug(s"Widening numeric types in union ${castedRight} ${right.output}") + Project(castedRight, right) + } else { + right + } + + Union(newLeft, newRight) + + // Also widen types for BinaryExpressions. case q: LogicalPlan => q transformExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e case b: BinaryExpression if b.left.dataType != b.right.dataType => - // Try and find a promotion rule that contains both types in question. - val applicableConversion = - allPromotions.find(p => p.contains(b.left.dataType) && p.contains(b.right.dataType)) - - applicableConversion match { - case Some(promotionRule) => - val widestType = - promotionRule.filter(t => t == b.left.dataType || t == b.right.dataType).last + findWidestType(b.left.dataType, b.right.dataType).map { widestType => val newLeft = if (b.left.dataType == widestType) b.left else Cast(b.left, widestType) val newRight = if (b.right.dataType == widestType) b.right else Cast(b.right, widestType) b.makeCopy(Array(newLeft, newRight)) - - // If there is no applicable conversion, leave expression unchanged. - case None => b - } + }.getOrElse(b) // If there is no applicable conversion, leave expression unchanged. } } + + def findWidestType(t1: DataType, t2: DataType): Option[DataType] = { + // Try and find a promotion rule that contains both types in question. + val applicableConversion = + allPromotions.find(p => p.contains(t1) && p.contains(t2)) + + // If found return the widest common type, otherwise None + applicableConversion.map(_.filter(t => t == t1 || t == t2).last) + } } /** From fc22e018025859f58a33f1ddb3f40aa941ef8169 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 22 Jan 2014 19:31:44 -0800 Subject: [PATCH 517/778] whitelist newly passing union test. --- src/test/scala/catalyst/execution/HiveCompatibility.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index cbb2706c8dee3..4742a32b7806f 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -527,6 +527,7 @@ class HiveCompatibility extends HiveQueryFileTest { "union19", "union2", "union20", + "union23", "union27", "union28", "union29", From 7859a86d7383da978455ccbe5156dcbd780e77f8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 22 Jan 2014 19:50:03 -0800 Subject: [PATCH 518/778] Remove SparkAggregate. Its kinda broken and breaks RDD lineage. --- .../catalyst/execution/SharkInstance.scala | 1 - .../scala/catalyst/execution/aggregates.scala | 60 +------------------ .../execution/planningStrategies.scala | 23 ------- 3 files changed, 1 insertion(+), 83 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index d5929d54d2418..b10a9233bf8e0 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -73,7 +73,6 @@ abstract class SharkInstance extends Logging { val sc = self.sc val strategies = SparkEquiInnerJoin :: - SparkAggregates :: HiveTableScans :: DataSinks :: BasicOperators :: diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 81ccaafd10d24..1d3e6fc0753a9 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -131,62 +131,4 @@ case class Aggregate( buildRow(aggImplementations.map(Evaluate(_, Vector(group)))) } } -} - -/** - * Uses spark Accumulators to perform global aggregation. - * - * Currently supports only COUNT(). - */ -case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan) - (@transient sc: SharkContext) extends UnaryNode { - def output = aggregateExprs.map(_.toAttribute) - override def otherCopyArgs = Seq(sc) - - case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - val count = sc.accumulable(0) - val sum = sc.accumulable(0) - def result: Any = sum.value.toDouble / count.value.toDouble - - def apply(input: Seq[Row]): Unit = { - count += 1 - // TODO: Support all types here... - sum += Evaluate(expr, input).asInstanceOf[Int] - } - } - - case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - val count = sc.accumulable(0) - - def apply(input: Seq[Row]): Unit = - if (Evaluate(expr, input) != null) - count += 1 - - def result: Any = count.value.toLong - } - - override def executeCollect() = attachTree(this, "SparkAggregate") { - // Replace all aggregate expressions with spark functions that will compute the result. - val aggImplementations = aggregateExprs.map { _ transform { - case base @ Average(expr) => new AverageFunction(expr, base) - case base @ Count(expr) => new CountFunction(expr, base) - }} - - // Pull out all the functions so we can feed each row into them. - val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f }) - assert(aggFunctions.nonEmpty) - - logger.debug(s"Running aggregates: $aggFunctions") - child.execute().foreach { row => - val input = Vector(row) - aggFunctions.foreach(_.apply(input)) - } - Array(buildRow(aggImplementations.map(Evaluate(_, Nil)))) - } - - def execute() = sc.makeRDD(executeCollect(), 1) -} +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 2d00f3ccb7972..6d2bc1224f1af 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -41,29 +41,6 @@ trait PlanningStrategies { } } - /** - * Aggregate functions that use sparks accumulator functionality. - */ - object SparkAggregates extends Strategy { - val allowedAggregates = Set[Class[_]]( - classOf[Count], - classOf[Average]) - - /** - * Returns true if `exprs` only contains aggregates that can be computed using Accumulators. - */ - def onlyAllowedAggregates(exprs: Seq[Expression]): Boolean = { - val aggs = exprs.flatMap(_.collect { case a: AggregateExpression => a}).map(_.getClass) - aggs.map(allowedAggregates contains _).reduceLeft(_ && _) - } - - def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { - case logical.Aggregate(Nil, agg, child) if onlyAllowedAggregates(agg) => - execution.SparkAggregate(agg, planLater(child))(sc) :: Nil - case _ => Nil - } - } - object SparkEquiInnerJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case FilteredOperation(predicates, logical.Join(left, right, Inner, condition)) => From 69ed98fd60faab3c17360f48df0dbe7034614e99 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 22 Jan 2014 20:14:26 -0800 Subject: [PATCH 519/778] Output a single row for empty Aggregations with no grouping expressions. --- .../scala/catalyst/execution/aggregates.scala | 70 +++++++++++-------- .../execution/planningStrategies.scala | 2 +- 2 files changed, 42 insertions(+), 30 deletions(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 1d3e6fc0753a9..743a3b89adf97 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -10,7 +10,7 @@ import org.apache.spark.SparkContext._ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], - child: SharkPlan) + child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { case class AverageFunction(expr: Expression, base: AggregateExpression) @@ -83,42 +83,45 @@ case class Aggregate( } } + override def otherCopyArgs = sc :: Nil def output = aggregateExpressions.map(_.toAttribute) + def createAggregateImplementations() = aggregateExpressions.map { agg => + val impl = agg transform { + case base @ Average(expr) => new AverageFunction(expr, base) + case base @ Sum(expr) => new SumFunction(expr, base) + case base @ Count(expr) => new CountFunction(expr, base) + case base @ CountDistinct(expr) => new CountDistinctFunction(expr, base) + case base @ First(expr) => new FirstFunction(expr, base) + } + + val remainingAttributes = impl.collect { case a: Attribute => a } + // If any references exist that are not inside agg functions then the must be grouping exprs + // in this case we must rebind them to the grouping tuple. + if (remainingAttributes.nonEmpty) { + val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } + + // An exact match with a grouping expression + val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { + case -1 => None + case ordinal => Some(BoundReference(0, ordinal, Alias(impl, "AGGEXPR")().toAttribute)) + } + + exactGroupingExpr.getOrElse( + sys.error(s"$agg is not in grouping expressions: $groupingExpressions")) + } else { + impl + } + } + def execute() = attachTree(this, "execute") { val grouped = child.execute().map { row => (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row) }.groupByKey() - grouped.map { case (group, rows) => + val result = grouped.map { case (group, rows) => // Replace all aggregate expressions with spark functions that will compute the result. - val aggImplementations = aggregateExpressions.map { agg => - val impl = agg transform { - case base @ Average(expr) => new AverageFunction(expr, base) - case base @ Sum(expr) => new SumFunction(expr, base) - case base @ Count(expr) => new CountFunction(expr, base) - case base @ CountDistinct(expr) => new CountDistinctFunction(expr, base) - case base @ First(expr) => new FirstFunction(expr, base) - } - - val remainingAttributes = impl.collect { case a: Attribute => a } - // If any references exist that are not inside agg functions then the must be grouping exprs - // in this case we must rebind them to the grouping tuple. - if (remainingAttributes.nonEmpty) { - val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } - - // An exact match with a grouping expression - val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { - case -1 => None - case ordinal => Some(BoundReference(0, ordinal, Alias(impl, "AGGEXPR")().toAttribute)) - } - - exactGroupingExpr.getOrElse( - sys.error(s"$agg is not in grouping expressions: $groupingExpressions")) - } else { - impl - } - } + val aggImplementations = createAggregateImplementations() // Pull out all the functions so we can feed each row into them. val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f }) @@ -130,5 +133,14 @@ case class Aggregate( } buildRow(aggImplementations.map(Evaluate(_, Vector(group)))) } + + // TODO: THIS DOES NOT PRESERVE LINEAGE + if(groupingExpressions.isEmpty && result.count == 0) { + // When there there is no output to the Aggregate operator, we still output an empty row. + val aggImplementations = createAggregateImplementations() + sc.makeRDD(buildRow(aggImplementations.map(Evaluate(_, Nil))) :: Nil) + } else { + result + } } } \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 6d2bc1224f1af..e7557fd3086d0 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -121,7 +121,7 @@ trait PlanningStrategies { case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(group, agg, planLater(child)) :: Nil + execution.Aggregate(group, agg, planLater(child))(sc) :: Nil case logical.LocalRelation(output, data) => execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => From 685bfa145b125f5237b87463978cd4d5677d77b9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 22 Jan 2014 22:42:45 -0800 Subject: [PATCH 520/778] fix spelling --- src/main/scala/catalyst/execution/SharkInstance.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index d5929d54d2418..4e643f33fe141 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -69,7 +69,7 @@ abstract class SharkInstance extends Logging { results } - object TrivalPlanner extends QueryPlanner[SharkPlan] with PlanningStrategies { + object TrivialPlanner extends QueryPlanner[SharkPlan] with PlanningStrategies { val sc = self.sc val strategies = SparkEquiInnerJoin :: @@ -101,7 +101,7 @@ abstract class SharkInstance extends Logging { lazy val analyzed = analyze(parsed) lazy val optimizedPlan = Optimize(catalog.CreateTables(analyzed)) // TODO: Don't just pick the first one... - lazy val sharkPlan = TrivalPlanner(optimizedPlan).next() + lazy val sharkPlan = TrivialPlanner(optimizedPlan).next() lazy val executedPlan: SharkPlan = PrepareForExecution(sharkPlan) lazy val toRdd = executedPlan.execute() From 2a76fc66857423b6dd84eee3ac986c28878102e8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 10:40:04 -0800 Subject: [PATCH 521/778] add notes from @rxin. --- src/main/scala/catalyst/execution/aggregates.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 743a3b89adf97..4a5c408f3d5ca 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -13,6 +13,7 @@ case class Aggregate( child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { + // TODO: Move these default functions back to expressions. Build framework for instantiating them. case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -91,6 +92,7 @@ case class Aggregate( case base @ Average(expr) => new AverageFunction(expr, base) case base @ Sum(expr) => new SumFunction(expr, base) case base @ Count(expr) => new CountFunction(expr, base) + // TODO: Create custom query plan node that calculates distinct values efficiently. case base @ CountDistinct(expr) => new CountDistinctFunction(expr, base) case base @ First(expr) => new FirstFunction(expr, base) } @@ -134,7 +136,7 @@ case class Aggregate( buildRow(aggImplementations.map(Evaluate(_, Vector(group)))) } - // TODO: THIS DOES NOT PRESERVE LINEAGE + // TODO: THIS DOES NOT PRESERVE LINEAGE AND BREAKS PIPELINING. if(groupingExpressions.isEmpty && result.count == 0) { // When there there is no output to the Aggregate operator, we still output an empty row. val aggImplementations = createAggregateImplementations() From 6537c666426d5915cd2977a90109469e4dd2479f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 10:58:29 -0800 Subject: [PATCH 522/778] Address @rxin's comments. --- .../scala/catalyst/analysis/Analyzer.scala | 21 +- .../catalyst/analysis/HiveTypeCoercion.scala | 253 ++++++++++++++++++ .../catalyst/analysis/typeCoercion.scala | 248 ----------------- .../plans/logical/basicOperators.scala | 2 +- 4 files changed, 261 insertions(+), 263 deletions(-) create mode 100644 src/main/scala/catalyst/analysis/HiveTypeCoercion.scala delete mode 100644 src/main/scala/catalyst/analysis/typeCoercion.scala diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 1c5fd0f2af3c8..58f69650057f2 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -18,7 +18,7 @@ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true * a [[FunctionRegistry]]. */ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean) - extends RuleExecutor[LogicalPlan] { + extends RuleExecutor[LogicalPlan] with HiveTypeCoercion { // TODO: pass this in as a parameter. val fixedPoint = FixedPoint(100) @@ -29,19 +29,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool Batch("CaseInsensitiveAttributeReferences", Once, (if (caseSensitive) Nil else LowercaseAttributeReferences :: Nil) : _*), Batch("Resolution", fixedPoint, - ResolveReferences, - ResolveRelations, - StarExpansion, - ResolveFunctions, - GlobalAggregates, - StringToIntegralCasts, - BooleanCasts, - WidenTypes, - PromoteStrings, - ConvertNaNs, - BooleanComparisons, - FunctionArgumentConversion, - PropagateTypes) + ResolveReferences :: + ResolveRelations :: + StarExpansion :: + ResolveFunctions :: + GlobalAggregates :: + typeCoercionRules :_*) ) /** diff --git a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala new file mode 100644 index 0000000000000..188632c3d4801 --- /dev/null +++ b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala @@ -0,0 +1,253 @@ +package catalyst +package analysis + +import expressions._ +import plans.logical._ +import rules._ +import types._ +import catalyst.execution.{HiveUdf, HiveGenericUdf} + +/** + * A collection of [[catalyst.rules.Rule Rules]] that can be used to coerce differing types that + * participate in operations into compatible ones. Most of these rules are based on Hive semantics, + * but they do not introduce any dependencies on the hive codebase. For this reason they remain in + * Catalyst until we have a more standard set of coercions. + */ +trait HiveTypeCoercion { + + val typeCoercionRules = + List(PropagateTypes, ConvertNaNs, WidenTypes, PromoteStrings, BooleanComparisons, BooleanCasts, + StringToIntegralCasts, FunctionArgumentConversion) + + /** + * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] dataTypes + * that are made by other rules to instances higher in the query tree. + */ + object PropagateTypes extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // No propagation required for leaf nodes. + case q: LogicalPlan if q.children.isEmpty => q + + case q: LogicalPlan => q transformExpressions { + case a: AttributeReference => + q.inputSet.find(_.exprId == a.exprId) match { + // This can happen when a Attribute reference is born in a non-leaf node, for example + // due to a call to an external script like in the Transform operator. + // TODO: Perhaps those should actually be aliases? + case None => a + // Leave the same if the dataTypes match. + case Some(newType) if a.dataType == newType.dataType => a + case Some(newType) => + logger.debug(s"Promoting $a to ${newType} in ${q.simpleString}}") + newType + } + } + } + } + + /** + * Converts string "NaN"s that are in binary operators with a NaN-able types (Float / Double) to + * the appropriate numeric equivalent. + */ + object ConvertNaNs extends Rule[LogicalPlan] { + val stringNaN = Literal("NaN", StringType) + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + /* Double Conversions */ + case b: BinaryExpression if b.left == stringNaN && b.right.dataType == DoubleType => + b.makeCopy(Array(b.right, Literal(Double.NaN))) + case b: BinaryExpression if b.left.dataType == DoubleType && b.right == stringNaN => + b.makeCopy(Array(Literal(Double.NaN), b.left)) + case b: BinaryExpression if b.left == stringNaN && b.right == stringNaN => + b.makeCopy(Array(Literal(Double.NaN), b.left)) + + /* Float Conversions */ + case b: BinaryExpression if b.left == stringNaN && b.right.dataType == FloatType => + b.makeCopy(Array(b.right, Literal(Float.NaN))) + case b: BinaryExpression if b.left.dataType == FloatType && b.right == stringNaN => + b.makeCopy(Array(Literal(Float.NaN), b.left)) + case b: BinaryExpression if b.left == stringNaN && b.right == stringNaN => + b.makeCopy(Array(Literal(Float.NaN), b.left)) + } + } + } + + /** + * Widens numeric types and converts strings to numbers when appropriate. + * + * Loosely based on rules from "Hadoop: The Definitive Guide" 2nd edition, by Tom White + * + * The implicit conversion rules can be summarized as follows: + * - Any integral numeric type can be implicitly converted to a wider type. + * - All the integral numeric types, FLOAT, and (perhaps surprisingly) STRING can be implicitly + * converted to DOUBLE. + * - TINYINT, SMALLINT, and INT can all be converted to FLOAT. + * - BOOLEAN types cannot be converted to any other type. + * + * Additionally, all types when UNION-ed with strings will be promoted to strings. + * Other string conversions are handled by PromoteStrings + */ + object WidenTypes extends Rule[LogicalPlan] { + val integralPrecedence = Seq(NullType, ByteType, ShortType, IntegerType, LongType) + val toDouble = integralPrecedence ++ Seq(NullType, FloatType, DoubleType) + val toFloat = Seq(NullType, ByteType, ShortType, IntegerType) :+ FloatType + val allPromotions: Seq[Seq[DataType]] = integralPrecedence :: toDouble :: toFloat :: Nil + + def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { + // Try and find a promotion rule that contains both types in question. + val applicableConversion = + allPromotions.find(p => p.contains(t1) && p.contains(t2)) + + // If found return the widest common type, otherwise None + applicableConversion.map(_.filter(t => t == t1 || t == t2).last) + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case u @ Union(left, right) if u.childrenResolved && !u.resolved => + val castedInput = left.output.zip(right.output).map { + // When a string is found on one side, make the other side a string too. + case (l,r) if l.dataType == StringType && r.dataType != StringType => + (l, Alias(Cast(r, StringType), r.name)()) + case (l,r) if l.dataType != StringType && r.dataType == StringType => + (Alias(Cast(l, StringType), l.name)(), r) + + case (l,r) if l.dataType != r.dataType => + logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") + findTightestCommonType(l.dataType, r.dataType).map { widestType => + val newLeft = + if (l.dataType == widestType) l else Alias(Cast(l, widestType), l.name)() + val newRight = + if (r.dataType == widestType) r else Alias(Cast(r, widestType), r.name)() + + (newLeft, newRight) + }.getOrElse((l,r)) // If there is no applicable conversion, leave expression unchanged. + case other => other + } + + val (castedLeft, castedRight) = castedInput.unzip + + val newLeft = + if(castedLeft.map(_.dataType) != left.output.map(_.dataType)) { + logger.debug(s"Widening numeric types in union $castedLeft ${left.output}") + Project(castedLeft, left) + } else { + left + } + + val newRight = + if(castedRight.map(_.dataType) != right.output.map(_.dataType)) { + logger.debug(s"Widening numeric types in union $castedRight ${right.output}") + Project(castedRight, right) + } else { + right + } + + Union(newLeft, newRight) + + // Also widen types for BinaryExpressions. + case q: LogicalPlan => q transformExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + case b: BinaryExpression if b.left.dataType != b.right.dataType => + findTightestCommonType(b.left.dataType, b.right.dataType).map { widestType => + val newLeft = + if (b.left.dataType == widestType) b.left else Cast(b.left, widestType) + val newRight = + if (b.right.dataType == widestType) b.right else Cast(b.right, widestType) + b.makeCopy(Array(newLeft, newRight)) + }.getOrElse(b) // If there is no applicable conversion, leave expression unchanged. + } + } + } + + /** + * Promotes strings that appear in arithmetic expressions. + */ + object PromoteStrings extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + case a: BinaryArithmetic if a.left.dataType == StringType => + a.makeCopy(Array(Cast(a.left, DoubleType), a.right)) + case a: BinaryArithmetic if a.right.dataType == StringType => + a.makeCopy(Array(a.left, Cast(a.right, DoubleType))) + + case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType != StringType => + p.makeCopy(Array(Cast(p.left, DoubleType), p.right)) + case p: BinaryPredicate if p.left.dataType != StringType && p.right.dataType == StringType => + p.makeCopy(Array(p.left, Cast(p.right, DoubleType))) + + case Sum(e) if e.dataType == StringType => + Sum(Cast(e, DoubleType)) + case Average(e) if e.dataType == StringType => + Sum(Cast(e, DoubleType)) + } + } + + /** + * Changes Boolean values to Bytes so that expressions like true < false can be Evaluated. + */ + object BooleanComparisons extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + // No need to change Equals operators as that actually makes sense for boolean types. + case e: Equals => e + // Otherwise turn them to Byte types so that there exists and ordering. + case p: BinaryComparison if p.left.dataType == BooleanType && p.right.dataType == BooleanType => + p.makeCopy(Array(Cast(p.left, ByteType), Cast(p.right, ByteType))) + } + } + + /** + * Casts to/from [[catalyst.types.BooleanType BooleanType]] are transformed into comparisons since + * the JVM does not consider Booleans to be numeric types. + */ + object BooleanCasts extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + case Cast(e, BooleanType) => Not(Equals(e, Literal(0))) + case Cast(e, dataType) if e.dataType == BooleanType => + Cast(If(e, Literal(1), Literal(0)), dataType) + } + } + + /** + * When encountering a cast from a string representing a valid fractional number to an integral + * type the jvm will throw a `java.lang.NumberFormatException`. Hive, in contrast, returns the + * truncated version of this number. + */ + object StringToIntegralCasts extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + case Cast(e @ StringType(), t: IntegralType) => + Cast(Cast(e, DecimalType), t) + } + } + + /** + * This ensure that the types for various functions are as expected. + */ + object FunctionArgumentConversion extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + // Promote SUM 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)) + case Sum(e @ FractionalType()) if e.dataType != DoubleType => Sum(Cast(e, DoubleType)) + + } + } +} diff --git a/src/main/scala/catalyst/analysis/typeCoercion.scala b/src/main/scala/catalyst/analysis/typeCoercion.scala deleted file mode 100644 index 1dadc908ba7c1..0000000000000 --- a/src/main/scala/catalyst/analysis/typeCoercion.scala +++ /dev/null @@ -1,248 +0,0 @@ -package catalyst -package analysis - -import expressions._ -import plans.logical._ -import rules._ -import types._ -import catalyst.execution.{HiveUdf, HiveGenericUdf} - -/** - * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] dataTypes - * that are made by other rules to instances higher in the query tree. - */ -object PropagateTypes extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - // No propagation required for leaf nodes. - case q: LogicalPlan if q.children.isEmpty => q - - case q: LogicalPlan => q transformExpressions { - case a: AttributeReference => - q.inputSet.find(_.exprId == a.exprId) match { - // This can happen when a Attribute reference is born in a non-leaf node, for example - // due to a call to an external script like in the Transform operator. - // TODO: Perhaps those should actually be aliases? - case None => a - // Leave the same if the dataTypes match. - case Some(newType) if a.dataType == newType.dataType => a - case Some(newType) => - logger.debug(s"Promoting $a to ${newType} in ${q.simpleString}}") - newType - } - } - } -} - -/** - * Converts string "NaN"s that are in binary operators with a NaN-able types (Float / Double) to the - * appropriate numeric equivalent. - */ -object ConvertNaNs extends Rule[LogicalPlan] { - val stringNaN = Literal("NaN", StringType) - - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case q: LogicalPlan => q transformExpressions { - // Skip nodes who's children have not been resolved yet. - case e if !e.childrenResolved => e - - /* Double Conversions */ - case b: BinaryExpression if b.left == stringNaN && b.right.dataType == DoubleType => - b.makeCopy(Array(b.right, Literal(Double.NaN))) - case b: BinaryExpression if b.left.dataType == DoubleType && b.right == stringNaN => - b.makeCopy(Array(Literal(Double.NaN), b.left)) - case b: BinaryExpression if b.left == stringNaN && b.right == stringNaN => - b.makeCopy(Array(Literal(Double.NaN), b.left)) - - /* Float Conversions */ - case b: BinaryExpression if b.left == stringNaN && b.right.dataType == FloatType => - b.makeCopy(Array(b.right, Literal(Float.NaN))) - case b: BinaryExpression if b.left.dataType == FloatType && b.right == stringNaN => - b.makeCopy(Array(Literal(Float.NaN), b.left)) - case b: BinaryExpression if b.left == stringNaN && b.right == stringNaN => - b.makeCopy(Array(Literal(Float.NaN), b.left)) - } - } -} - -/** - * Widens numeric types and converts strings to numbers when appropriate. - * - * Loosely based on rules from "Hadoop: The Definitive Guide" 2nd edition, by Tom White - * - * The implicit conversion rules can be summarized as follows: - * - Any integral numeric type can be implicitly converted to a wider type. - * - All the integral numeric types, FLOAT, and (perhaps surprisingly) STRING can be implicitly - * converted to DOUBLE. - * - TINYINT, SMALLINT, and INT can all be converted to FLOAT. - * - BOOLEAN types cannot be converted to any other type. - * - * Additionally, all types when UNION-ed with strings will be promoted to strings. - * Other string conversions are handled by PromoteStrings - */ -object WidenTypes extends Rule[LogicalPlan] { - val integralPrecedence = Seq(NullType, ByteType, ShortType, IntegerType, LongType) - val toDouble = integralPrecedence ++ Seq(NullType, FloatType, DoubleType) - val toFloat = Seq(NullType, ByteType, ShortType, IntegerType) :+ FloatType - val allPromotions = integralPrecedence :: toDouble :: toFloat :: Nil - - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case u @ Union(left, right) if u.childrenResolved && !u.resolved => - val castedInput = left.output.zip(right.output).map { - // When a string is found on one side, make the other side a string too. - case (l,r) if l.dataType == StringType && r.dataType != StringType => - (l, Alias(Cast(r, StringType), r.name)()) - case (l,r) if l.dataType != StringType && r.dataType == StringType => - (Alias(Cast(l, StringType), l.name)(), r) - - case (l,r) if l.dataType != r.dataType => - logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") - findWidestType(l.dataType, r.dataType).map { widestType => - val newLeft = - if (l.dataType == widestType) - l - else - Alias(Cast(l, widestType), l.name)() - val newRight = - if (r.dataType == widestType) - r - else - Alias(Cast(r, widestType), r.name)() - - (newLeft, newRight) - }.getOrElse((l,r)) // If there is no applicable conversion, leave expression unchanged. - case other => other - } - - val (castedLeft, castedRight) = castedInput.unzip - - val newLeft = - if(castedLeft.map(_.dataType) != left.output.map(_.dataType)) { - logger.debug(s"Widening numeric types in union ${castedLeft} ${left.output}") - Project(castedLeft, left) - } else { - left - } - - val newRight = - if(castedRight.map(_.dataType) != right.output.map(_.dataType)) { - logger.debug(s"Widening numeric types in union ${castedRight} ${right.output}") - Project(castedRight, right) - } else { - right - } - - Union(newLeft, newRight) - - // Also widen types for BinaryExpressions. - case q: LogicalPlan => q transformExpressions { - // Skip nodes who's children have not been resolved yet. - case e if !e.childrenResolved => e - - case b: BinaryExpression if b.left.dataType != b.right.dataType => - findWidestType(b.left.dataType, b.right.dataType).map { widestType => - val newLeft = - if (b.left.dataType == widestType) b.left else Cast(b.left, widestType) - val newRight = - if (b.right.dataType == widestType) b.right else Cast(b.right, widestType) - b.makeCopy(Array(newLeft, newRight)) - }.getOrElse(b) // If there is no applicable conversion, leave expression unchanged. - } - } - - def findWidestType(t1: DataType, t2: DataType): Option[DataType] = { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = - allPromotions.find(p => p.contains(t1) && p.contains(t2)) - - // If found return the widest common type, otherwise None - applicableConversion.map(_.filter(t => t == t1 || t == t2).last) - } -} - -/** - * Promotes strings that appear in arithmetic expressions. - */ -object PromoteStrings extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - // Skip nodes who's children have not been resolved yet. - case e if !e.childrenResolved => e - - case a: BinaryArithmetic if a.left.dataType == StringType => - a.makeCopy(Array(Cast(a.left, DoubleType), a.right)) - case a: BinaryArithmetic if a.right.dataType == StringType => - a.makeCopy(Array(a.left, Cast(a.right, DoubleType))) - - case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType != StringType => - p.makeCopy(Array(Cast(p.left, DoubleType), p.right)) - case p: BinaryPredicate if p.left.dataType != StringType && p.right.dataType == StringType => - p.makeCopy(Array(p.left, Cast(p.right, DoubleType))) - - case Sum(e) if e.dataType == StringType => - Sum(Cast(e, DoubleType)) - case Average(e) if e.dataType == StringType => - Sum(Cast(e, DoubleType)) - } -} - -/** - * Changes Boolean values to Bytes so that expressions like true < false can be Evaluated. - */ -object BooleanComparisons extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - // Skip nodes who's children have not been resolved yet. - case e if !e.childrenResolved => e - // No need to change Equals operators as that actually makes sense for boolean types. - case e: Equals => e - // Otherwise turn them to Byte types so that there exists and ordering. - case p: BinaryComparison if p.left.dataType == BooleanType && p.right.dataType == BooleanType => - p.makeCopy(Array(Cast(p.left, ByteType), Cast(p.right, ByteType))) - } -} - -/** - * Casts to/from [[catalyst.types.BooleanType BooleanType]] are transformed into comparisons since - * the JVM does not consider Booleans to be numeric types. - */ -object BooleanCasts extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - // Skip nodes who's children have not been resolved yet. - case e if !e.childrenResolved => e - - case Cast(e, BooleanType) => Not(Equals(e, Literal(0))) - case Cast(e, dataType) if e.dataType == BooleanType => - Cast(If(e, Literal(1), Literal(0)), dataType) - } -} - -/** - * When encountering a cast from a string representing a valid fractional number to an integral type - * the jvm will throw a `java.lang.NumberFormatException`. Hive, in contrast, returns the - * truncated version of this number. - */ -object StringToIntegralCasts extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - // Skip nodes who's children have not been resolved yet. - case e if !e.childrenResolved => e - - case Cast(e @ StringType(), t: IntegralType) => - Cast(Cast(e, DecimalType), t) - } -} - -/** - * This ensure that the types for various functions are as expected. Most of these rules are - * actually Hive specific. - * TODO: Move this to the hive specific package once we make that separation. - */ -object FunctionArgumentConversion extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - // Skip nodes who's children have not been resolved yet. - case e if !e.childrenResolved => e - - // Promote SUM 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)) - case Sum(e @ FractionalType()) if e.dataType != DoubleType => Sum(Cast(e, DoubleType)) - - } -} diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index d153b72e9448f..93c9b4a36494a 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -20,7 +20,7 @@ case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override lazy val resolved = childrenResolved && - left.output.zip(right.output).filter { case (l,r) => l.dataType != r.dataType }.isEmpty + !left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType } def references = Set.empty } From 563053f424aa997866d605ba6132fda657307b5f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 11:07:04 -0800 Subject: [PATCH 523/778] Address @rxin's comments. --- src/main/scala/catalyst/dsl.scala | 7 +++++-- src/main/scala/catalyst/execution/basicOperators.scala | 6 ++++-- src/main/scala/catalyst/execution/planningStrategies.scala | 4 ++-- src/main/scala/catalyst/frontend/Hive.scala | 2 +- src/main/scala/catalyst/plans/logical/basicOperators.scala | 4 +++- 5 files changed, 15 insertions(+), 8 deletions(-) diff --git a/src/main/scala/catalyst/dsl.scala b/src/main/scala/catalyst/dsl.scala index 51b717bc0cbe5..6efb5cf873c69 100644 --- a/src/main/scala/catalyst/dsl.scala +++ b/src/main/scala/catalyst/dsl.scala @@ -114,8 +114,11 @@ package object dsl { def filter(dynamicUdf: (DynamicRow) => Boolean) = Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(plan.output))), plan) - def sample(fraction: Double, withReplacement: Boolean = true) = - Sample(fraction, withReplacement, plan) + def sample( + fraction: Double, + withReplacement: Boolean = true, + seed: Int = (math.random * 1000).toInt) = + Sample(fraction, withReplacement, seed, plan) def analyze = analysis.SimpleAnalyzer(plan) } diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 4ffb393ffcf58..8b577beeda9fc 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -22,11 +22,13 @@ case class Filter(condition: Expression, child: SharkPlan) extends UnaryNode { } } -case class Sample(fraction: Double, withReplacement: Boolean, child: SharkPlan) extends UnaryNode { +case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SharkPlan) + extends UnaryNode { + def output = child.output // TODO: How to pick seed? - def execute() = child.execute().sample(withReplacement, fraction, (math.random * 1000).toInt) + def execute() = child.execute().sample(withReplacement, fraction, seed) } case class Union(left: SharkPlan, right: SharkPlan)(@transient sc: SharkContext) diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 88360c6f09f74..8b87e4d71047c 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -145,8 +145,8 @@ trait PlanningStrategies { execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => execution.Aggregate(group, agg, planLater(child)) :: Nil - case logical.Sample(fraction, withReplacement, child) => - execution.Sample(fraction, withReplacement, planLater(child)) :: Nil + case logical.Sample(fraction, withReplacement, seed, child) => + execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index b66c1f718e38e..a05a68407d239 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -507,7 +507,7 @@ object HiveQl { case Token("TOK_TABLEBUCKETSAMPLE", Token(numerator, Nil) :: Token(denominator, Nil) :: Nil) => - Sample(numerator.toDouble / denominator.toDouble, false, relation) + Sample(numerator.toDouble / denominator.toDouble, false, (math.random * 1000).toInt, relation) }.getOrElse(relation) case Token("TOK_UNIQUEJOIN", joinArgs) => diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index f22d3bdbd867a..3682399344240 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -68,7 +68,9 @@ case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { def references = Set.empty } -case class Sample(fraction: Double, withReplacement: Boolean, child: LogicalPlan) extends UnaryNode { +case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: LogicalPlan) + extends UnaryNode { + def output = child.output def references = Set.empty } From 9804eb5384a3ca8ece5496d71ed2cae2072183e2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 11:48:25 -0800 Subject: [PATCH 524/778] upgrade spark --- build.sbt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 2fe38ceb72ac8..28e9967946e3b 100644 --- a/build.sbt +++ b/build.sbt @@ -12,7 +12,10 @@ scalacOptions ++= Seq("-deprecation", "-feature", "-unchecked") resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" -libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating-SNAPSHOT" +// TODO: Remove when Spark 0.9.0 is released for real. +resolvers += "SparkStaging" at "https://repository.apache.org/content/repositories/orgapachespark-1005/" + +libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" libraryDependencies += "catalyst" % "hive-golden" % "4" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden4.jar" From ece15e1412bb5f3fc7ebf1bbb24fdededc28c515 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:02:56 -0800 Subject: [PATCH 525/778] update unit tests --- .../plans/physical/partitioning.scala | 8 +-- ...rtyTests.scala => DistributionTests.scala} | 65 +++++++++---------- 2 files changed, 33 insertions(+), 40 deletions(-) rename src/test/scala/catalyst/execution/{DataPropertyTests.scala => DistributionTests.scala} (68%) diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index 2c5e15d6a7fe9..eae6aedec2686 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -13,13 +13,6 @@ import types._ * operators (e.g., [[Aggregate]]) to perform partition local operations instead of global ones. * - Intra-partition ordering of data: In this case the distribution describes guarantees made * about how tuples are distributed within a single partition. - * - * TOOD(marmbrus): Given the way that Spark does partitioning, I think the order of the grouping - * actually does matter, and thus our subset checking is probably not sufficient - * to ensure correct colocation for joins.? - * TODO(marmbrus): Similarly I'm not sure that satisfies is a sufficient check to see if an sort - * aggregation can be done. Maybe we need two checks? Separate ordering from - * partitioning? */ sealed trait Distribution { } @@ -54,6 +47,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { sealed abstract trait Partitioning { self: Product => + /** Returns the number of partitions that the data is split across */ val width: Int /** diff --git a/src/test/scala/catalyst/execution/DataPropertyTests.scala b/src/test/scala/catalyst/execution/DistributionTests.scala similarity index 68% rename from src/test/scala/catalyst/execution/DataPropertyTests.scala rename to src/test/scala/catalyst/execution/DistributionTests.scala index faf9c5504065e..61e074127d05b 100644 --- a/src/test/scala/catalyst/execution/DataPropertyTests.scala +++ b/src/test/scala/catalyst/execution/DistributionTests.scala @@ -1,78 +1,79 @@ -package catalyst.execution - -import catalyst.dsl._ +package catalyst +package plans +package physical import org.scalatest.FunSuite +import catalyst.dsl._ + class DistributionTests extends FunSuite { - /* protected def checkSatisfied( - inputDistribution: Distribution, + inputPartitioning: Partitioning, requiredDistribution: Distribution, satisfied: Boolean) { - if(inputDistribution.satisfies(requiredDistribution) != satisfied) + if(inputPartitioning.satisfies(requiredDistribution) != satisfied) fail( s""" |== Input Distribution == - |$inputDistribution + |$inputPartitioning |== Required Distribution == |$requiredDistribution |== Does input distribution satisfy requirements? == - |Expected $satisfied got ${inputDistribution.satisfies(requiredDistribution)} + |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} """.stripMargin) } test("needExchange test: ClusteredDistribution is the output DataProperty") { // Cases which do not need an exchange between two data properties. checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq('a, 'b, 'c), 10), UnknownDistribution, true) checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq('a, 'b, 'c), 10), ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( - ClusteredDistribution(Seq('b, 'c)), + HashPartitioning(Seq('b, 'c), 10), ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( - ClusteredDistribution(Nil), + Unpartitioned, ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( - ClusteredDistribution(Nil), + Unpartitioned, OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) // Cases which need an exchange between two data properties. checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq('a, 'b, 'c), 10), ClusteredDistribution(Seq('b, 'c)), false) checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq('a, 'b, 'c), 10), ClusteredDistribution(Seq('d, 'e)), false) checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), - ClusteredDistribution(Nil), + HashPartitioning(Seq('a, 'b, 'c), 10), + AllTuples, false) checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq('a, 'b, 'c), 10), OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) checkSatisfied( - ClusteredDistribution(Seq('b, 'c)), + HashPartitioning(Seq('b, 'c), 10), OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) @@ -88,37 +89,37 @@ class DistributionTests extends FunSuite { test("needExchange test: OrderedDistribution is the output DataProperty") { // Cases which do not need an exchange between two data properties. checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), UnknownDistribution, true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), OrderedDistribution(Seq('a.asc, 'b.asc)), true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Seq('c, 'b, 'a)), true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Seq('b, 'c, 'a, 'd)), true) @@ -128,30 +129,28 @@ class DistributionTests extends FunSuite { // should tradeoff the benefit of a less number of Exchange operators // and the parallelism. checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), false) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), OrderedDistribution(Seq('b.asc, 'a.asc)), false) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Seq('a, 'b)), false) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Seq('c, 'd)), false) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Nil), false) } - - */ } \ No newline at end of file From 043e2961689d1e3936aef56a25fce3510c4a714d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:22:09 -0800 Subject: [PATCH 526/778] Make physical union nodes variadic. --- .../scala/catalyst/execution/basicOperators.scala | 8 +++----- .../catalyst/execution/planningStrategies.scala | 4 ++-- src/main/scala/catalyst/planning/patterns.scala | 15 +++++++++++++++ 3 files changed, 20 insertions(+), 7 deletions(-) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 8b577beeda9fc..d1187c1771e8d 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -31,12 +31,10 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: def execute() = child.execute().sample(withReplacement, fraction, seed) } -case class Union(left: SharkPlan, right: SharkPlan)(@transient sc: SharkContext) - extends BinaryNode { +case class Union(children: Seq[SharkPlan])(@transient sc: SharkContext) extends SharkPlan { // TODO: attributes output by union should be distinct for nullability purposes - def output = left.output - // TODO: is it more efficient to union a bunch of rdds at once? should union be variadic? - def execute() = sc.union(left.execute(), right.execute()) + def output = children.head.output + def execute() = sc.union(children.map(_.execute())) override def otherCopyArgs = sc :: Nil } diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index f37a663be26ec..c631f3ebba512 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -128,8 +128,8 @@ trait PlanningStrategies { execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => execution.StopAfter(Evaluate(limit, Nil).asInstanceOf[Int], planLater(child))(sc) :: Nil - case logical.Union(left, right) => - execution.Union(planLater(left), planLater(right))(sc) :: Nil + case Unions(unionChildren) => + execution.Union(unionChildren.map(planLater))(sc) :: Nil case logical.Transform(input, script, output, child) => execution.Transform(input, script, output, planLater(child))(sc) :: Nil case _ => Nil diff --git a/src/main/scala/catalyst/planning/patterns.scala b/src/main/scala/catalyst/planning/patterns.scala index 670923b436d1c..147a8fc6dffc9 100644 --- a/src/main/scala/catalyst/planning/patterns.scala +++ b/src/main/scala/catalyst/planning/patterns.scala @@ -25,4 +25,19 @@ object FilteredOperation extends PredicateHelper { collectFilters(filters ++ splitConjunctivePredicates(condition), child) case other => (filters, other) } +} + +/** + * A pattern that collects all adjacent unions and returns their children as a Seq. + */ +object Unions { + def unapply(plan: LogicalPlan): Option[Seq[LogicalPlan]] = plan match { + case u: Union => Some(collectUnionChildren(u)) + case _ => None + } + + private def collectUnionChildren(plan: LogicalPlan): Seq[LogicalPlan] = plan match { + case Union(l, r) => collectUnionChildren(l) ++ collectUnionChildren(r) + case other => other :: Nil + } } \ No newline at end of file From 4f345f232d5bb5dfee076bdb6da40827adb5ee55 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:54:14 -0800 Subject: [PATCH 527/778] Remove SortKey, use RowOrdering. --- .../scala/catalyst/execution/Exchange.scala | 76 ++----------------- 1 file changed, 7 insertions(+), 69 deletions(-) diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 764c106912786..1cda0a5d67a48 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -29,77 +29,15 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) shuffled.map(_._2) case RangePartitioning(sortingExpressions, width) => - val directions = sortingExpressions.map(_.direction).toIndexedSeq - val dataTypes = sortingExpressions.map(_.dataType).toIndexedSeq + // TODO: ShuffledRDD should take an Ordering. + import scala.math.Ordering.Implicits._ + implicit val ordering = new RowOrdering(sortingExpressions) - // TODO: MOVE THIS! - class SortKey(val keyValues: IndexedSeq[Any]) - extends Ordered[SortKey] - with Serializable { - def compare(other: SortKey): Int = { - var i = 0 - while (i < keyValues.size) { - val left = keyValues(i) - val right = other.keyValues(i) - val curDirection = directions(i) - val curDataType = dataTypes(i) - - logger.debug(s"Comparing $left, $right as $curDataType order $curDirection") - // TODO: Use numeric here too? - val comparison = - if (left == null && right == null) { - 0 - } else if (left == null) { - if (curDirection == Ascending) -1 else 1 - } else if (right == null) { - if (curDirection == Ascending) 1 else -1 - } else if (curDataType == IntegerType) { - if (curDirection == Ascending) { - left.asInstanceOf[Int] compare right.asInstanceOf[Int] - } else { - right.asInstanceOf[Int] compare left.asInstanceOf[Int] - } - } else if (curDataType == DoubleType) { - if (curDirection == Ascending) { - left.asInstanceOf[Double] compare right.asInstanceOf[Double] - } else { - right.asInstanceOf[Double] compare left.asInstanceOf[Double] - } - } else if (curDataType == LongType) { - if (curDirection == Ascending) { - left.asInstanceOf[Long] compare right.asInstanceOf[Long] - } else { - right.asInstanceOf[Long] compare left.asInstanceOf[Long] - } - } else if (curDataType == StringType) { - if (curDirection == Ascending) { - left.asInstanceOf[String] compare right.asInstanceOf[String] - } else { - right.asInstanceOf[String] compare left.asInstanceOf[String] - } - } else { - sys.error(s"Comparison not yet implemented for: $curDataType") - } - - if (comparison != 0) return comparison - i += 1 - } - return 0 - } - } - - val rdd = child.execute().map { row => - val input = Vector(row) - val sortKey = new SortKey( - sortingExpressions.map(s => Evaluate(s.child, input)).toIndexedSeq) - - (sortKey, row) - } + val rdd = child.execute().map(r => (r,null)) val part = new RangePartitioner(width, rdd, ascending = true) - val shuffled = new ShuffledRDD[SortKey, Row, (SortKey, Row)](rdd, part) - - shuffled.map(_._2) - case _ => sys.error("Not implemented") + val shuffled = new ShuffledRDD[Row, Null, (Row, Null)](rdd, part) + shuffled.map(_._1) + case _ => sys.error(s"Exchange not implemented for $newPartitioning") } } } From 330a88b7eeefa3a8c9be2c0170a6f33cb85879d6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:54:29 -0800 Subject: [PATCH 528/778] Fix bugs in AddExchange. --- .../scala/catalyst/execution/Exchange.scala | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 1cda0a5d67a48..2966e26164ebb 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -42,6 +42,11 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) } } +/** + * Ensures that the [[catalyst.plans.physical.Partitioning Partitioning]] of input data meets the + * [[catalyst.plans.physical.Distribution Distribution]] requirements for each operator by inserting + * [[Exchange]] Operators where required. + */ object AddExchange extends Rule[SharkPlan] { // TODO: determine the number of partitions. val numPartitions = 8 @@ -50,10 +55,14 @@ object AddExchange extends Rule[SharkPlan] { case operator: SharkPlan => def meetsRequirements = !operator.requiredChildDistribution.zip(operator.children).map { - case (required, child) => !child.outputPartitioning.satisfies(required) + case (required, child) => + val valid = child.outputPartitioning.satisfies(required) + logger.debug( + s"${if (valid) "Valid" else "Invalid"} distribution, required: $required current: ${child.outputPartitioning}") + valid }.exists(_ == false) - // TODO ASUUMES TRANSITIVITY? + // TODO ASSUMES TRANSITIVITY? def compatible = !operator.children .map(_.outputPartitioning) @@ -64,14 +73,14 @@ object AddExchange extends Rule[SharkPlan] { }.exists(_ == false) - if (false && meetsRequirements && compatible) { + if (meetsRequirements && compatible) { operator } else { val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { case (ClusteredDistribution(clustering), child) => - Exchange(HashPartitioning(clustering, 8), child) + Exchange(HashPartitioning(clustering, numPartitions), child) case (OrderedDistribution(ordering), child) => - Exchange(RangePartitioning(ordering, 8), child) + Exchange(RangePartitioning(ordering, numPartitions), child) case (UnknownDistribution, child) => child case (dist, _) => sys.error(s"Don't know how to ensure $dist") } From c0b0e60008b9bb1abbbb0a792b39ce3beb4ec444 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:54:41 -0800 Subject: [PATCH 529/778] cleanup broken doc links. --- .../plans/physical/partitioning.scala | 22 ++++++++++--------- src/main/scala/catalyst/types/dataTypes.scala | 2 +- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index eae6aedec2686..2252fe6e1c886 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -6,11 +6,12 @@ import expressions._ import types._ /** - * Specifies how tuples that share common expressions will be distributed. Distribution can - * be used to refer to two distinct physical properties: + * Specifies how tuples that share common expressions will be distributed when a query is executed + * in parallel on many machines. Distribution can be used to refer to two distinct physical + * properties: * - Inter-node partitioning of data: In this case the distribution describes how tuples are * partitioned across physical machines in a cluster. Knowing this property allows some - * operators (e.g., [[Aggregate]]) to perform partition local operations instead of global ones. + * operators (e.g., Aggregate) to perform partition local operations instead of global ones. * - Intra-partition ordering of data: In this case the distribution describes guarantees made * about how tuples are distributed within a single partition. */ @@ -28,17 +29,18 @@ case object UnknownDistribution extends Distribution case object AllTuples extends Distribution /** - * Represents data where tuples that share the same values for the `clustering` [[Expression]]s will - * be co-located. Based on the context, this can mean such tuples are either co-located in the same - * partition or they will be contiguous within a single partition. + * Represents data where tuples that share the same values for the `clustering` + * [[catalyst.expressions.Expression Expressions]] will be co-located. Based on the context, this + * can mean such tuples are either co-located in the same partition or they will be contiguous + * within a single partition. */ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution /** - * Represents data where tuples have been ordered according to the `ordering` [[Expression]]s. This - * is a strictly stronger guarantee than [[ClusteredDistribution]] as an ordering will ensure that - * tuples that share the same value for the ordering expressions are contiguous and will never be - * split across partitions. + * Represents data where tuples have been ordered according to the `ordering` + * [[catalyst.expressions.Expression Expressions]]. This is a strictly stronger guarantee than + * [[ClusteredDistribution]] as an ordering will ensure that tuples that share the same value for + * the ordering expressions are contiguous and will never be split across partitions. */ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { def clustering = ordering.map(_.child).toSet diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 86d289dc77525..2f00f499ab828 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -4,7 +4,7 @@ package types import expressions.Expression abstract class DataType { - /** Matches any expression that evaluates to this [[DataType]] */ + /** Matches any expression that evaluates to this DataType */ def unapply(a: Expression): Boolean = a match { case e: Expression if e.dataType == this => true case _ => false From 6377d0bddebe69e72070f0490d39e2b319342d66 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:58:33 -0800 Subject: [PATCH 530/778] Drop empty files, fix if (). --- src/main/scala/catalyst/execution/dataProperties.scala | 0 src/main/scala/catalyst/execution/distribution.scala | 0 src/main/scala/catalyst/trees/TreeNode.scala | 8 ++++---- 3 files changed, 4 insertions(+), 4 deletions(-) delete mode 100644 src/main/scala/catalyst/execution/dataProperties.scala delete mode 100644 src/main/scala/catalyst/execution/distribution.scala diff --git a/src/main/scala/catalyst/execution/dataProperties.scala b/src/main/scala/catalyst/execution/dataProperties.scala deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/src/main/scala/catalyst/execution/distribution.scala b/src/main/scala/catalyst/execution/distribution.scala deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index dd2085950fc3d..4e9ccd6000c69 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -91,7 +91,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { val newArgs = productIterator.map { case arg: TreeNode[_] if (children contains arg) => val newChild = f(arg.asInstanceOf[BaseType]) - if(newChild fastEquals arg) + if (newChild fastEquals arg) arg else { changed = true @@ -100,7 +100,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case nonChild: AnyRef => nonChild case null => null }.toArray - if(changed) makeCopy(newArgs) else this + if (changed) makeCopy(newArgs) else this } /** @@ -116,7 +116,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case arg: TreeNode[_] if (children contains arg) => val newChild = remainingNewChildren.remove(0) val oldChild = remainingOldChildren.remove(0) - if(newChild fastEquals oldChild) + if (newChild fastEquals oldChild) oldChild else { changed = true @@ -126,7 +126,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case null => null }.toArray - if(changed) makeCopy(newArgs) else this + if (changed) makeCopy(newArgs) else this } /** From e170ccfe7193c78ad9c3caeb42c4f6f27cba7f81 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 15:17:26 -0800 Subject: [PATCH 531/778] Improve documentation and remove some spurious changes that were introduced by the merge. --- .../plans/physical/partitioning.scala | 34 +++++++++++++++---- src/main/scala/catalyst/trees/TreeNode.scala | 16 ++++----- src/main/scala/catalyst/types/dataTypes.scala | 7 +++- .../execution/HiveComparisonTest.scala | 1 - 4 files changed, 41 insertions(+), 17 deletions(-) diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index 2252fe6e1c886..8e973bcde9937 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -15,8 +15,7 @@ import types._ * - Intra-partition ordering of data: In this case the distribution describes guarantees made * about how tuples are distributed within a single partition. */ -sealed trait Distribution { -} +sealed trait Distribution /** * Represents a distribution where no promises are made about co-location of data. @@ -46,9 +45,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { def clustering = ordering.map(_.child).toSet } -sealed abstract trait Partitioning { - self: Product => - +sealed trait Partitioning { /** Returns the number of partitions that the data is split across */ val width: Int @@ -58,6 +55,11 @@ sealed abstract trait Partitioning { */ def satisfies(required: Distribution): Boolean + /** + * Returns true iff all distribution guarantees made by this partitioning can also be made + * for the `other` specified partitioning. For example, [[HashPartitioning]]s are only compatible + * if the `width` of the two partitionings is the same. + */ def compatibleWith(other: Partitioning): Boolean } @@ -92,7 +94,14 @@ case object Broadcast extends Partitioning { } } -case class HashPartitioning(expressions: Seq[Expression], width: Int) extends Expression with Partitioning { +/** + * Represents a partitioning where rows are split up across partitions based on based on the hash + * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be + * in the same partition. + */ +case class HashPartitioning(expressions: Seq[Expression], width: Int) + extends Expression with Partitioning { + def children = expressions.toSeq def references = expressions.flatMap(_.references).toSet def nullable = false @@ -113,7 +122,18 @@ case class HashPartitioning(expressions: Seq[Expression], width: Int) extends Ex } } -case class RangePartitioning(ordering: Seq[SortOrder], width: Int) extends Expression with Partitioning { +/** + * Represents a partitioning where rows are split across partitions based on some total ordering of + * the expressions specified in `ordering`. When data is partitioned in this manner the following + * two conditions are guaranteed to hold: + * - All row where the expressions in `ordering` evaluate to the same values will be in the same + * partition. + * - Each partition will have a `min` and `max` row, relative to the given ordering. All rows + * that are in between `min` and `max` in this `ordering` will reside in this partition. + */ +case class RangePartitioning(ordering: Seq[SortOrder], width: Int) + extends Expression with Partitioning { + def children = ordering.toSeq def references = ordering.flatMap(_.references).toSet def nullable = false diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index 4e9ccd6000c69..b9599dc9301d6 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -89,11 +89,11 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def mapChildren(f: BaseType => BaseType): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = f(arg.asInstanceOf[BaseType]) - if (newChild fastEquals arg) + if (newChild fastEquals arg) { arg - else { + } else { changed = true newChild } @@ -113,12 +113,12 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { val remainingNewChildren = newChildren.toBuffer val remainingOldChildren = children.toBuffer val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = remainingNewChildren.remove(0) val oldChild = remainingOldChildren.remove(0) - if (newChild fastEquals oldChild) + if (newChild fastEquals oldChild) { oldChild - else { + } else { changed = true newChild } @@ -207,7 +207,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def transformChildrenUp(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformUp(rule) if (!(newChild fastEquals arg)) { changed = true @@ -217,7 +217,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } case m: Map[_,_] => m case args: Traversable[_] => args.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformUp(rule) if (!(newChild fastEquals arg)) { changed = true diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 2f00f499ab828..6e864d047ff4a 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -31,7 +31,11 @@ case object BooleanType extends NativeType { } abstract class NumericType extends NativeType { - // Unfortunately we can't get this implicitly as that breaks Spark Serialization. + // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for + // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a + // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets + // desugared by the compiler into an argument to the objects constructor. This means there is no + // longer an no argument constructor and thus the JVM cannot serialize the object anymore. val numeric: Numeric[JvmType] } @@ -42,6 +46,7 @@ object IntegralType { case _ => false } } + abstract class IntegralType extends NumericType { val integral: Integral[JvmType] } diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index c98b2ebb50f6e..830ce7a88395e 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -8,7 +8,6 @@ import frontend.hive.{ExplainCommand, Command} import plans.physical._ import util._ - /** * Allows the creations of tests that execute the same query against both hive * and catalyst, comparing the results. From 167162feea43ba2e07d2a49ab66b640d62c47205 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 15:28:35 -0800 Subject: [PATCH 532/778] more merge errors, cleanup. --- src/main/scala/catalyst/execution/SharkPlan.scala | 6 +++--- src/main/scala/catalyst/expressions/Evaluate.scala | 13 ++++++++----- .../scala/catalyst/expressions/predicates.scala | 11 +++++++---- src/main/scala/catalyst/trees/TreeNode.scala | 4 ++-- .../apache/spark/rdd/SharkPairRDDFunctions.scala | 7 ++++--- 5 files changed, 24 insertions(+), 17 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index ba34a3c178e77..0f5617746f41b 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -3,15 +3,15 @@ package execution import org.apache.spark.rdd.RDD -import plans.QueryPlan -import plans.physical._ +import catalyst.plans.QueryPlan +import catalyst.plans.physical._ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => // TODO: Move to `DistributedPlan` /** Specifies how data is partitioned across different nodes in the cluster. */ - def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG! + def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG WIDTH! /** Specifies any partition requirements on the input data for this operator. */ def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnknownDistribution) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index b563041590c02..c5a75d1412ea2 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -34,7 +34,7 @@ object Evaluate extends Logging { @inline def n2(e1: Expression, e2: Expression, f: ((Numeric[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) @@ -52,7 +52,7 @@ object Evaluate extends Logging { @inline def f2(e1: Expression, e2: Expression, f: ((Fractional[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) @@ -69,7 +69,8 @@ object Evaluate extends Logging { @inline def i2(e1: Expression, e2: Expression, f: ((Integral[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + if (e1.dataType != e2.dataType) + throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) if (evalE1 == null || evalE2 == null) @@ -112,7 +113,7 @@ object Evaluate extends Logging { case UnaryMinus(child) => n1(child, _.negate(_)) /* Control Flow */ - case If(e, t, f) => if(eval(e).asInstanceOf[Boolean]) eval(t) else eval(f) + case If(e, t, f) => if (eval(e).asInstanceOf[Boolean]) eval(t) else eval(f) /* Comparisons */ case Equals(l, r) => @@ -226,7 +227,9 @@ object Evaluate extends Logging { case implementedFunction: ImplementedUdf => implementedFunction.evaluate(implementedFunction.children.map(eval)) - case a: Attribute => throw new OptimizationException(a, "Unable to evaluate unbound reference without access to the input schema.") + case a: Attribute => + throw new OptimizationException(a, + "Unable to evaluate unbound reference without access to the input schema.") case other => throw new OptimizationException(other, "evaluation not implemented") } diff --git a/src/main/scala/catalyst/expressions/predicates.scala b/src/main/scala/catalyst/expressions/predicates.scala index f21c1cafc0e08..b8d578dc1c546 100644 --- a/src/main/scala/catalyst/expressions/predicates.scala +++ b/src/main/scala/catalyst/expressions/predicates.scala @@ -83,15 +83,18 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E def nullable = false } -case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) extends Expression { +case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) + extends Expression { + def children = predicate :: trueValue :: falseValue :: Nil - def nullable = children.exists(_.nullable) + def nullable = trueValue.nullable || falseValue.nullable def references = children.flatMap(_.references).toSet override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType def dataType = { - if (!resolved) + if (!resolved) { throw new UnresolvedException( - this, s"datatype. Can not resolve due to differing types ${trueValue.dataType}, ${falseValue.dataType}") + this, s"Can not resolve due to differing types ${trueValue.dataType}, ${falseValue.dataType}") + } trueValue.dataType } } diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index b9599dc9301d6..e45dfe55b0ebb 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -163,7 +163,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def transformChildrenDown(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformDown(rule) if (!(newChild fastEquals arg)) { changed = true @@ -173,7 +173,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } case m: Map[_,_] => m case args: Traversable[_] => args.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformDown(rule) if (!(newChild fastEquals arg)) { changed = true diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala index d0b0261895208..5f430a223f0c3 100644 --- a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala +++ b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -1,12 +1,13 @@ package org.apache.spark.rdd +import scala.language.implicitConversions + import scala.reflect._ -import org.apache.spark._ import scala.collection.mutable.ArrayBuffer + +import org.apache.spark._ import org.apache.spark.Aggregator import org.apache.spark.SparkContext._ - -import scala.language.implicitConversions import org.apache.spark.util.collection.AppendOnlyMap /** From 42859620fdc5129e66a6eaaf613ab67bf5fe6375 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 15:33:41 -0800 Subject: [PATCH 533/778] Remove temporary test cases --- .../catalyst/execution/DslQueryTests1.scala | 74 ------------------- 1 file changed, 74 deletions(-) delete mode 100644 src/test/scala/catalyst/execution/DslQueryTests1.scala diff --git a/src/test/scala/catalyst/execution/DslQueryTests1.scala b/src/test/scala/catalyst/execution/DslQueryTests1.scala deleted file mode 100644 index dbce6a9743ba4..0000000000000 --- a/src/test/scala/catalyst/execution/DslQueryTests1.scala +++ /dev/null @@ -1,74 +0,0 @@ -package catalyst -package execution - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import analysis._ -import expressions._ -import plans._ -import plans.logical.LogicalPlan -import types._ - -/* Implicits */ -import dsl._ - -class DslQueryTests1 extends FunSuite with BeforeAndAfterAll { - override def beforeAll() { - // By clearing the port we force Spark to pick a new one. This allows us to rerun tests - // without restarting the JVM. - System.clearProperty("spark.driver.port") - System.clearProperty("spark.hostPort") - } - - import TestShark._ - import TestData._ - - test("inner join and then group by using the same column") { - val x = testData2.subquery('x) - val y = testData3.subquery('y) - checkAnswer( - x.join(y).where("x.a".attr === "y.a".attr) - .groupBy("x.a".attr)("x.a".attr, Count("x.b".attr)), - (1,2) :: - (2,2) :: Nil - ) - } - - /** - * Runs the plan and makes sure the answer matches the expected result. - * @param plan the query to be executed - * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ]. - */ - protected def checkAnswer(plan: LogicalPlan, expectedAnswer: Any): Unit = { - val convertedAnswer = expectedAnswer match { - case s: Seq[_] if s.isEmpty => s - case s: Seq[_] if s.head.isInstanceOf[Product] && - !s.head.isInstanceOf[Seq[_]] => s.map(_.asInstanceOf[Product].productIterator.toIndexedSeq) - case s: Seq[_] => s - case singleItem => Seq(Seq(singleItem)) - } - - val isSorted = plan.collect { case s: logical.Sort => s}.nonEmpty - def prepareAnswer(answer: Seq[Any]) = if (!isSorted) answer.sortBy(_.toString) else answer - val sharkAnswer = try plan.toRdd.collect().toSeq catch { - case e: Exception => - fail( - s""" - |Exception thrown while executing query: - |$plan - |== Physical Plan == - |${plan.executedPlan} - |== Exception == - |$e - """.stripMargin) - } - println( - s""" - |Logical plan: - |$plan - |== Physical Plan == - |${plan.executedPlan} - """.stripMargin) - assert(prepareAnswer(convertedAnswer) === prepareAnswer(sharkAnswer)) - } -} \ No newline at end of file From 6d6cb581eb07a940986cea2395740de3e90502d1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 15:54:02 -0800 Subject: [PATCH 534/778] add source links that point to github to the scala doc. --- build.sbt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/build.sbt b/build.sbt index 2fe38ceb72ac8..bc74bd56ff186 100644 --- a/build.sbt +++ b/build.sbt @@ -72,4 +72,8 @@ mergeStrategy in assembly := { case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines case "reference.conf" => MergeStrategy.concat case _ => MergeStrategy.first +} + +scalacOptions in (Compile, doc) <++= (baseDirectory) map { + bd => Seq("-sourcepath", bd.getAbsolutePath, "-doc-source-url","https://github.com/databricks/catalyst/blob/master/€{FILE_PATH}.scala") } \ No newline at end of file From b20a4d469f8c1a7bdc1baabcdfef892a6631c5c7 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Sat, 25 Jan 2014 01:37:31 +0800 Subject: [PATCH 535/778] Fix issue #20 https://github.com/databricks/catalyst/issues/20 --- src/main/scala/catalyst/expressions/Evaluate.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 9de519bf9431f..3a92313724414 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -27,7 +27,7 @@ object Evaluate extends Logging { e.dataType match { case n: NumericType => val castedFunction = f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType] - castedFunction(n.numeric, eval(e).asInstanceOf[n.JvmType]) + castedFunction(n.numeric, evalE.asInstanceOf[n.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } } From 9e3d989b67d0926b8d79b0ab128b3cff609157c3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 24 Jan 2014 23:32:55 -0800 Subject: [PATCH 536/778] Made HiveTypeCoercion.WidenTypes more clear. --- .../catalyst/analysis/HiveTypeCoercion.scala | 42 +++++++++------- .../analysis/HiveTypeCoercionSuite.scala | 49 +++++++++++++++++++ 2 files changed, 73 insertions(+), 18 deletions(-) create mode 100644 src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala diff --git a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala index 188632c3d4801..74aef2928c2dc 100644 --- a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala +++ b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala @@ -5,7 +5,6 @@ import expressions._ import plans.logical._ import rules._ import types._ -import catalyst.execution.{HiveUdf, HiveGenericUdf} /** * A collection of [[catalyst.rules.Rule Rules]] that can be used to coerce differing types that @@ -38,7 +37,7 @@ trait HiveTypeCoercion { // Leave the same if the dataTypes match. case Some(newType) if a.dataType == newType.dataType => a case Some(newType) => - logger.debug(s"Promoting $a to ${newType} in ${q.simpleString}}") + logger.debug(s"Promoting $a to $newType in ${q.simpleString}}") newType } } @@ -89,33 +88,40 @@ trait HiveTypeCoercion { * - BOOLEAN types cannot be converted to any other type. * * Additionally, all types when UNION-ed with strings will be promoted to strings. - * Other string conversions are handled by PromoteStrings + * Other string conversions are handled by PromoteStrings. */ object WidenTypes extends Rule[LogicalPlan] { + // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. + // The conversion for integral and floating point types have a linear widening hierarchy: + // NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType val integralPrecedence = Seq(NullType, ByteType, ShortType, IntegerType, LongType) - val toDouble = integralPrecedence ++ Seq(NullType, FloatType, DoubleType) - val toFloat = Seq(NullType, ByteType, ShortType, IntegerType) :+ FloatType - val allPromotions: Seq[Seq[DataType]] = integralPrecedence :: toDouble :: toFloat :: Nil + val floatingPrecedence = Seq(FloatType, DoubleType) + val precedence = integralPrecedence ++ floatingPrecedence def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = - allPromotions.find(p => p.contains(t1) && p.contains(t2)) - - // If found return the widest common type, otherwise None - applicableConversion.map(_.filter(t => t == t1 || t == t2).last) + // Make sure both types are in the precedence list. + val t1pos = precedence.indexOf(t1) + val t2pos = precedence.indexOf(t2) + + if (t1pos != -1 && t2pos != -1) { + // Found both types in the list, return the max of t1pos and t2pos to find the tightest + // common type. + Some(precedence(math.max(t1pos, t2pos))) + } else { + None + } } def apply(plan: LogicalPlan): LogicalPlan = plan transform { case u @ Union(left, right) if u.childrenResolved && !u.resolved => val castedInput = left.output.zip(right.output).map { // When a string is found on one side, make the other side a string too. - case (l,r) if l.dataType == StringType && r.dataType != StringType => + case (l, r) if l.dataType == StringType && r.dataType != StringType => (l, Alias(Cast(r, StringType), r.name)()) - case (l,r) if l.dataType != StringType && r.dataType == StringType => + case (l, r) if l.dataType != StringType && r.dataType == StringType => (Alias(Cast(l, StringType), l.name)(), r) - case (l,r) if l.dataType != r.dataType => + case (l, r) if l.dataType != r.dataType => logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") findTightestCommonType(l.dataType, r.dataType).map { widestType => val newLeft = @@ -124,14 +130,14 @@ trait HiveTypeCoercion { if (r.dataType == widestType) r else Alias(Cast(r, widestType), r.name)() (newLeft, newRight) - }.getOrElse((l,r)) // If there is no applicable conversion, leave expression unchanged. + }.getOrElse((l, r)) // If there is no applicable conversion, leave expression unchanged. case other => other } val (castedLeft, castedRight) = castedInput.unzip val newLeft = - if(castedLeft.map(_.dataType) != left.output.map(_.dataType)) { + if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) { logger.debug(s"Widening numeric types in union $castedLeft ${left.output}") Project(castedLeft, left) } else { @@ -139,7 +145,7 @@ trait HiveTypeCoercion { } val newRight = - if(castedRight.map(_.dataType) != right.output.map(_.dataType)) { + if (castedRight.map(_.dataType) != right.output.map(_.dataType)) { logger.debug(s"Widening numeric types in union $castedRight ${right.output}") Project(castedRight, right) } else { diff --git a/src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala b/src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala new file mode 100644 index 0000000000000..30455a161f257 --- /dev/null +++ b/src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -0,0 +1,49 @@ +package catalyst.analysis + +import org.scalatest.FunSuite + +import catalyst.types._ + + +class HiveTypeCoercionSuite extends FunSuite { + + val rules = new HiveTypeCoercion { } + import rules._ + + test("tightest common bound for numeric types") { + def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) { + var found = WidenTypes.findTightestCommonType(t1, t2) + assert(found == tightestCommon, + s"Expected $tightestCommon as tightest common type for $t1 and $t2, found $found") + // Test both directions to make sure the widening is symmetric. + found = WidenTypes.findTightestCommonType(t2, t1) + assert(found == tightestCommon, + s"Expected $tightestCommon as tightest common type for $t2 and $t1, found $found") + } + + // Integral + widenTest(NullType, ByteType, Some(ByteType)) + widenTest(NullType, IntegerType, Some(IntegerType)) + widenTest(NullType, LongType, Some(LongType)) + widenTest(ShortType, IntegerType, Some(IntegerType)) + widenTest(ShortType, LongType, Some(LongType)) + widenTest(IntegerType, LongType, Some(LongType)) + widenTest(LongType, LongType, Some(LongType)) + + // Floating point + widenTest(NullType, FloatType, Some(FloatType)) + widenTest(NullType, DoubleType, Some(DoubleType)) + widenTest(FloatType, DoubleType, Some(DoubleType)) + widenTest(FloatType, FloatType, Some(FloatType)) + widenTest(DoubleType, DoubleType, Some(DoubleType)) + + // Integral mixed with floating point. + widenTest(NullType, FloatType, Some(FloatType)) + widenTest(NullType, DoubleType, Some(DoubleType)) + widenTest(IntegerType, FloatType, Some(FloatType)) + widenTest(IntegerType, DoubleType, Some(DoubleType)) + widenTest(IntegerType, DoubleType, Some(DoubleType)) + widenTest(LongType, FloatType, Some(FloatType)) + widenTest(LongType, DoubleType, Some(DoubleType)) + } +} From 5b11db0a2eddf734c50084874a104cf17a8955c6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 24 Jan 2014 23:42:11 -0800 Subject: [PATCH 537/778] Added Void to Boolean type widening. --- .../catalyst/analysis/HiveTypeCoercion.scala | 25 ++++++++----------- .../analysis/HiveTypeCoercionSuite.scala | 8 +++++- 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala index 74aef2928c2dc..1d7ba1e0a65f3 100644 --- a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala +++ b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala @@ -93,23 +93,18 @@ trait HiveTypeCoercion { object WidenTypes extends Rule[LogicalPlan] { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. // The conversion for integral and floating point types have a linear widening hierarchy: - // NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType - val integralPrecedence = Seq(NullType, ByteType, ShortType, IntegerType, LongType) - val floatingPrecedence = Seq(FloatType, DoubleType) - val precedence = integralPrecedence ++ floatingPrecedence + val numericPrecedence = + Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) + // Boolean is only wider than Void + val booleanPrecedence = Seq(NullType, BooleanType) + val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: booleanPrecedence :: Nil def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { - // Make sure both types are in the precedence list. - val t1pos = precedence.indexOf(t1) - val t2pos = precedence.indexOf(t2) - - if (t1pos != -1 && t2pos != -1) { - // Found both types in the list, return the max of t1pos and t2pos to find the tightest - // common type. - Some(precedence(math.max(t1pos, t2pos))) - } else { - None - } + // Try and find a promotion rule that contains both types in question. + val applicableConversion = allPromotions.find(p => p.contains(t1) && p.contains(t2)) + + // If found return the widest common type, otherwise None + applicableConversion.map(_.filter(t => t == t1 || t == t2).last) } def apply(plan: LogicalPlan): LogicalPlan = plan transform { diff --git a/src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala b/src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala index 30455a161f257..4fb834e373397 100644 --- a/src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -10,7 +10,7 @@ class HiveTypeCoercionSuite extends FunSuite { val rules = new HiveTypeCoercion { } import rules._ - test("tightest common bound for numeric types") { + test("tightest common bound for numeric and boolean types") { def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) { var found = WidenTypes.findTightestCommonType(t1, t2) assert(found == tightestCommon, @@ -21,6 +21,12 @@ class HiveTypeCoercionSuite extends FunSuite { s"Expected $tightestCommon as tightest common type for $t2 and $t1, found $found") } + // Boolean + widenTest(NullType, BooleanType, Some(BooleanType)) + widenTest(BooleanType, BooleanType, Some(BooleanType)) + widenTest(IntegerType, BooleanType, None) + widenTest(LongType, BooleanType, None) + // Integral widenTest(NullType, ByteType, Some(ByteType)) widenTest(NullType, IntegerType, Some(IntegerType)) From 548e479883c6da18e939ac87862ee04d14242797 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 25 Jan 2014 22:05:23 +0800 Subject: [PATCH 538/778] merge master into exchangeOperator and fix code style --- build.sbt | 4 ++++ src/main/scala/catalyst/execution/Exchange.scala | 6 +++--- .../scala/catalyst/execution/basicOperators.scala | 8 +++----- src/main/scala/catalyst/execution/joins.scala | 13 +++++++------ .../catalyst/execution/planningStrategies.scala | 4 ++-- .../scala/catalyst/expressions/Evaluate.scala | 3 ++- src/main/scala/catalyst/planning/patterns.scala | 15 +++++++++++++++ .../catalyst/plans/physical/partitioning.scala | 14 ++++++++++---- .../apache/spark/rdd/SharkPairRDDFunctions.scala | 6 ++++-- .../catalyst/execution/HiveComparisonTest.scala | 14 +++++++++++--- 10 files changed, 61 insertions(+), 26 deletions(-) diff --git a/build.sbt b/build.sbt index 28e9967946e3b..a1a52867305e7 100644 --- a/build.sbt +++ b/build.sbt @@ -75,4 +75,8 @@ mergeStrategy in assembly := { case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines case "reference.conf" => MergeStrategy.concat case _ => MergeStrategy.first +} + +scalacOptions in (Compile, doc) <++= (baseDirectory) map { + bd => Seq("-sourcepath", bd.getAbsolutePath, "-doc-source-url","https://github.com/databricks/catalyst/blob/master/€{FILE_PATH}.scala") } \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 2966e26164ebb..2238d49c33a39 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -11,7 +11,7 @@ import org.apache.spark.{RangePartitioner, HashPartitioner} import org.apache.spark.rdd.ShuffledRDD case class Exchange(newPartitioning: Partitioning, child: SharkPlan) - extends UnaryNode { + extends UnaryNode { override def outputPartitioning = newPartitioning def output = child.output @@ -30,7 +30,6 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) case RangePartitioning(sortingExpressions, width) => // TODO: ShuffledRDD should take an Ordering. - import scala.math.Ordering.Implicits._ implicit val ordering = new RowOrdering(sortingExpressions) val rdd = child.execute().map(r => (r,null)) @@ -58,7 +57,8 @@ object AddExchange extends Rule[SharkPlan] { case (required, child) => val valid = child.outputPartitioning.satisfies(required) logger.debug( - s"${if (valid) "Valid" else "Invalid"} distribution, required: $required current: ${child.outputPartitioning}") + s"${if (valid) "Valid" else "Invalid"} distribution," + + s"required: $required current: ${child.outputPartitioning}") valid }.exists(_ == false) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 1fc81dd665a39..014aae7110676 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -28,12 +28,10 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: def execute() = child.execute().sample(withReplacement, fraction, seed) } -case class Union(left: SharkPlan, right: SharkPlan)(@transient sc: SharkContext) - extends BinaryNode { +case class Union(children: Seq[SharkPlan])(@transient sc: SharkContext) extends SharkPlan { // TODO: attributes output by union should be distinct for nullability purposes - def output = left.output - // TODO: is it more efficient to union a bunch of rdds at once? should union be variadic? - def execute() = sc.union(left.execute(), right.execute()) + def output = children.head.output + def execute() = sc.union(children.map(_.execute())) override def otherCopyArgs = sc :: Nil } diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 380f64dee8753..ef2d6205f2b40 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -18,22 +18,23 @@ case class SparkEquiInnerJoin( left: SharkPlan, right: SharkPlan) extends BinaryNode { - - override def requiredChildDistribution= + override def requiredChildDistribution = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil def output = left.output ++ right.output def execute() = attachTree(this, "execute") { - val leftWithKeys = left.execute.map { row => + val leftWithKeys = left.execute().map { row => val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) - logger.debug(s"Generated left join keys [${leftKeys.mkString(",")}] => [${joinKeys.mkString(",")}] given row $row") + logger.debug(s"Generated left join keys [${leftKeys.mkString(",")}] =>" + + s"[${joinKeys.mkString(",")}] given row $row") (joinKeys, row) } - val rightWithKeys = right.execute.map { row => + val rightWithKeys = right.execute().map { row => val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) - logger.debug(s"Generated right join keys [${rightKeys.mkString(",")}] => [${joinKeys.mkString(",")}] given row $row") + logger.debug(s"Generated right join keys [${rightKeys.mkString(",")}] =>" + + s"[${joinKeys.mkString(",")}] given row $row") (joinKeys, row) } diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index d20cbb5d70174..959115b1278e2 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -130,8 +130,8 @@ trait PlanningStrategies { execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => execution.StopAfter(Evaluate(limit, Nil).asInstanceOf[Int], planLater(child))(sc) :: Nil - case logical.Union(left, right) => - execution.Union(planLater(left), planLater(right))(sc) :: Nil + case Unions(unionChildren) => + execution.Union(unionChildren.map(planLater))(sc) :: Nil case logical.Transform(input, script, output, child) => execution.Transform(input, script, output, planLater(child))(sc) :: Nil case _ => Nil diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index c5a75d1412ea2..efeceaa42d305 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -26,7 +26,8 @@ object Evaluate extends Logging { else e.dataType match { case n: NumericType => - f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType](n.numeric, eval(e).asInstanceOf[n.JvmType]) + val castedFunction = f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType] + castedFunction(n.numeric, evalE.asInstanceOf[n.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } } diff --git a/src/main/scala/catalyst/planning/patterns.scala b/src/main/scala/catalyst/planning/patterns.scala index 670923b436d1c..147a8fc6dffc9 100644 --- a/src/main/scala/catalyst/planning/patterns.scala +++ b/src/main/scala/catalyst/planning/patterns.scala @@ -25,4 +25,19 @@ object FilteredOperation extends PredicateHelper { collectFilters(filters ++ splitConjunctivePredicates(condition), child) case other => (filters, other) } +} + +/** + * A pattern that collects all adjacent unions and returns their children as a Seq. + */ +object Unions { + def unapply(plan: LogicalPlan): Option[Seq[LogicalPlan]] = plan match { + case u: Union => Some(collectUnionChildren(u)) + case _ => None + } + + private def collectUnionChildren(plan: LogicalPlan): Seq[LogicalPlan] = plan match { + case Union(l, r) => collectUnionChildren(l) ++ collectUnionChildren(r) + case other => other :: Nil + } } \ No newline at end of file diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index 8e973bcde9937..1fe9f6a4306a6 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -68,6 +68,7 @@ case class UnknownPartitioning(width: Int) extends Partitioning { case UnknownDistribution => true case _ => false } + def compatibleWith(other: Partitioning): Boolean = other match { case UnknownPartitioning(_) => true case _ => false @@ -78,6 +79,7 @@ case object Unpartitioned extends Partitioning { val width = 1 override def satisfies(required: Distribution): Boolean = true + override def compatibleWith(other: Partitioning) = other match { case Unpartitioned => true case _ => false @@ -87,7 +89,8 @@ case object Unpartitioned extends Partitioning { case object Broadcast extends Partitioning { val width = 1 - override def satisfies(required:Distribution): Boolean = true + override def satisfies(required: Distribution): Boolean = true + override def compatibleWith(other: Partitioning) = other match { case Unpartitioned => true case _ => false @@ -95,12 +98,13 @@ case object Broadcast extends Partitioning { } /** - * Represents a partitioning where rows are split up across partitions based on based on the hash + * Represents a partitioning where rows are split up across partitions based on the hash * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be * in the same partition. */ case class HashPartitioning(expressions: Seq[Expression], width: Int) - extends Expression with Partitioning { + extends Expression + with Partitioning { def children = expressions.toSeq def references = expressions.flatMap(_.references).toSet @@ -115,6 +119,7 @@ case class HashPartitioning(expressions: Seq[Expression], width: Int) clusteringSet.subsetOf(requiredClustering.toSet) case _ => false } + override def compatibleWith(other: Partitioning) = other match { case Broadcast => true case h: HashPartitioning if h == this => true @@ -132,7 +137,8 @@ case class HashPartitioning(expressions: Seq[Expression], width: Int) * that are in between `min` and `max` in this `ordering` will reside in this partition. */ case class RangePartitioning(ordering: Seq[SortOrder], width: Int) - extends Expression with Partitioning { + extends Expression + with Partitioning { def children = ordering.toSeq def references = ordering.flatMap(_.references).toSet diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala index 5f430a223f0c3..1599bb51cda4e 100644 --- a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala +++ b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -11,8 +11,10 @@ import org.apache.spark.SparkContext._ import org.apache.spark.util.collection.AppendOnlyMap /** - * Extra functions for Shark available on RDDs of (key, value) pairs through an implicit conversion. - * Import `org.apache.spark.SharkPairRDDFunctions._` at the top of your program to use these functions. + * Extra functions for Shark available on RDDs of (key, value) pairs through + * an implicit conversion. + * Import `org.apache.spark.SharkPairRDDFunctions._` at the top of your program to + * use these functions. */ class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Logging diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index 830ce7a88395e..1b4d01c3ce3a9 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -87,7 +87,11 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G hiveFailedDirectory.mkdir() // Not atomic! /** All directories that contain per-query output files */ - val outputDirectories = Seq(passedDirectory, failedDirectory, wrongDirectory, hiveFailedDirectory) + val outputDirectories = Seq( + passedDirectory, + failedDirectory, + wrongDirectory, + hiveFailedDirectory) protected val cacheDigest = java.security.MessageDigest.getInstance("MD5") protected def getMd5(str: String): String = { @@ -96,13 +100,17 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G new java.math.BigInteger(1, digest.digest).toString(16) } - protected def prepareAnswer(sharkQuery: TestShark.type#SharkSqlQuery, answer: Seq[String]): Seq[String] = { + protected def prepareAnswer( + sharkQuery: TestShark.type#SharkSqlQuery, + answer: Seq[String]): Seq[String] = { val orderedAnswer = sharkQuery.parsed match { // Clean out non-deterministic time schema info. case _: Command => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _ => // TODO: Really we only care about the final total ordering here... - val isOrdered = sharkQuery.executedPlan.collect { case s @ Exchange(r: RangePartitioning, _) => s}.nonEmpty + val isOrdered = sharkQuery.executedPlan.collect { + case s @ Exchange(r: RangePartitioning, _) => s + }.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. if (!isOrdered) answer.sorted else answer } From fd084a4eadac3dee487ffa060c8c36c1073ec8b6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 25 Jan 2014 16:24:03 -0800 Subject: [PATCH 539/778] implement casts binary <=> string. --- src/main/scala/catalyst/expressions/Evaluate.scala | 5 +++++ src/test/scala/catalyst/execution/HiveCompatibility.scala | 1 + 2 files changed, 6 insertions(+) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 3a92313724414..143c6f1b670fc 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -156,6 +156,11 @@ object Evaluate extends Logging { currentExpression /* Casts */ + + // Binary Conversions + case Cast(e @ BinaryType(), StringType) => new String(eval(e).asInstanceOf[Array[Byte]]) + case Cast(e @ StringType(), BinaryType) => eval(e).asInstanceOf[String].getBytes + // toString case Cast(e, StringType) => eval(e) match { diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index 4742a32b7806f..26abe2cc135e9 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -158,6 +158,7 @@ class HiveCompatibility extends HiveQueryFileTest { "auto_sortmerge_join_7", "auto_sortmerge_join_8", "auto_sortmerge_join_9", + "binary_constant", "binarysortable_1", "bucket1", "bucket_map_join_1", From be1fff7d2d069b92529ee92dd0a9b5f07a942c36 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 25 Jan 2014 16:38:30 -0800 Subject: [PATCH 540/778] Replace foreach with while in RowOrdering. Fixes #23 --- src/main/scala/catalyst/expressions/Row.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/expressions/Row.scala b/src/main/scala/catalyst/expressions/Row.scala index 2ea4d36224421..7f6b858c77673 100644 --- a/src/main/scala/catalyst/expressions/Row.scala +++ b/src/main/scala/catalyst/expressions/Row.scala @@ -82,7 +82,9 @@ class GenericRow(input: Seq[Any]) extends Row { class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { def compare(a: Row, b: Row): Int = { - ordering.foreach { order => + var i = 0 + while(i < ordering.size) { + val order = ordering(i) val left = Evaluate(order.child, Vector(a)) val right = Evaluate(order.child, Vector(b)) @@ -101,6 +103,7 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { } if (comparison != 0) return comparison } + i += 1 } return 0 } From 907db686b196febafff0064cc8b01f03ccccdb8d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 26 Jan 2014 04:29:52 -0800 Subject: [PATCH 541/778] Space after while. --- src/main/scala/catalyst/expressions/Row.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/expressions/Row.scala b/src/main/scala/catalyst/expressions/Row.scala index 7f6b858c77673..432d79fb5862c 100644 --- a/src/main/scala/catalyst/expressions/Row.scala +++ b/src/main/scala/catalyst/expressions/Row.scala @@ -83,7 +83,7 @@ class GenericRow(input: Seq[Any]) extends Row { class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { def compare(a: Row, b: Row): Int = { var i = 0 - while(i < ordering.size) { + while (i < ordering.size) { val order = ordering(i) val left = Evaluate(order.child, Vector(a)) val right = Evaluate(order.child, Vector(b)) From 2957f31b9aa558423df25bb730ea9fe2d24699ee Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 26 Jan 2014 20:55:51 +0800 Subject: [PATCH 542/778] addressed comments on PR --- .../scala/catalyst/analysis/unresolved.scala | 2 +- src/main/scala/catalyst/errors/package.scala | 11 ++--- .../scala/catalyst/execution/Exchange.scala | 17 ++++--- .../scala/catalyst/execution/SharkPlan.scala | 2 +- .../catalyst/execution/basicOperators.scala | 1 + .../scala/catalyst/expressions/Evaluate.scala | 12 ++--- .../catalyst/plans/logical/LogicalPlan.scala | 2 +- .../plans/physical/partitioning.scala | 45 ++++++++++--------- src/main/scala/catalyst/trees/TreeNode.scala | 2 +- .../execution/DistributionTests.scala | 8 ++-- 10 files changed, 55 insertions(+), 47 deletions(-) diff --git a/src/main/scala/catalyst/analysis/unresolved.scala b/src/main/scala/catalyst/analysis/unresolved.scala index 54ed4d9802392..d198c6dd8ce1c 100644 --- a/src/main/scala/catalyst/analysis/unresolved.scala +++ b/src/main/scala/catalyst/analysis/unresolved.scala @@ -10,7 +10,7 @@ import trees.TreeNode * resolved. */ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: String) extends - errors.OptimizationException(tree, s"Invalid call to $function on unresolved object") + errors.TreeNodeException(tree, s"Invalid call to $function on unresolved object") /** * Holds the name of a relation that has yet to be looked up in a [[Catalog]]. diff --git a/src/main/scala/catalyst/errors/package.scala b/src/main/scala/catalyst/errors/package.scala index 41bd9c1d8055e..0f24fa5ad9f43 100644 --- a/src/main/scala/catalyst/errors/package.scala +++ b/src/main/scala/catalyst/errors/package.scala @@ -7,7 +7,7 @@ import trees._ */ package object errors { - class OptimizationException[TreeType <: TreeNode[_]] + class TreeNodeException[TreeType <: TreeNode[_]] (tree: TreeType, msg: String, cause: Throwable = null) extends Exception(msg, cause) { override def getMessage: String = { @@ -17,17 +17,18 @@ package object errors { } /** - * Wraps any exceptions that are thrown while executing `f` in an [[OptimizationException]], - * attaching the provided `tree`. + * Wraps any exceptions that are thrown while executing `f` in a + * [[catalyst.errors.TreeNodeException TreeNodeException]], attaching the provided `tree`. */ def attachTree[TreeType <: TreeNode[_], A](tree: TreeType, msg: String = "")(f: => A): A = { try f catch { - case e: Exception => throw new OptimizationException(tree, msg, e) + case e: Exception => throw new TreeNodeException(tree, msg, e) } } /** - * Executes `f` which is expected to throw an OptimizationException. The first tree encountered in + * Executes `f` which is expected to throw a + * [[catalyst.errors.TreeNodeException TreeNodeException]]. The first tree encountered in * the stack of exceptions of type `TreeType` is returned. */ def getTree[TreeType <: TreeNode[_]](f: => Unit): TreeType = ??? // TODO: Implement diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 2238d49c33a39..698167e16cb7c 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -18,25 +18,26 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) def execute() = attachTree(this , "execute") { newPartitioning match { - case HashPartitioning(expressions, width) => + case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().map { row => (buildRow(expressions.toSeq.map(Evaluate(_, Vector(row)))), row) } - val part = new HashPartitioner(width) + val part = new HashPartitioner(numPartitions) val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) shuffled.map(_._2) - case RangePartitioning(sortingExpressions, width) => + case RangePartitioning(sortingExpressions, numPartitions) => // TODO: ShuffledRDD should take an Ordering. implicit val ordering = new RowOrdering(sortingExpressions) val rdd = child.execute().map(r => (r,null)) - val part = new RangePartitioner(width, rdd, ascending = true) + val part = new RangePartitioner(numPartitions, rdd, ascending = true) val shuffled = new ShuffledRDD[Row, Null, (Row, Null)](rdd, part) shuffled.map(_._1) case _ => sys.error(s"Exchange not implemented for $newPartitioning") + // TODO: Handle SinglePartition and BroadcastPartitioning. } } } @@ -47,7 +48,7 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) * [[Exchange]] Operators where required. */ object AddExchange extends Rule[SharkPlan] { - // TODO: determine the number of partitions. + // TODO: Determine the number of partitions. val numPartitions = 8 def apply(plan: SharkPlan): SharkPlan = plan.transformUp { @@ -62,7 +63,7 @@ object AddExchange extends Rule[SharkPlan] { valid }.exists(_ == false) - // TODO ASSUMES TRANSITIVITY? + // TODO: ASSUMES TRANSITIVITY? def compatible = !operator.children .map(_.outputPartitioning) @@ -76,12 +77,14 @@ object AddExchange extends Rule[SharkPlan] { if (meetsRequirements && compatible) { operator } else { + // TODO: It is possible that only a child does not meet requirement. val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { case (ClusteredDistribution(clustering), child) => Exchange(HashPartitioning(clustering, numPartitions), child) case (OrderedDistribution(ordering), child) => Exchange(RangePartitioning(ordering, numPartitions), child) - case (UnknownDistribution, child) => child + case (UnspecifiedDistribution, child) => child + // TODO: Handle AllTuples. One possible way is to use Exchange(HashPartitioning(Nil, 1), child). case (dist, _) => sys.error(s"Don't know how to ensure $dist") } operator.withNewChildren(repartitionedChildren) diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 0f5617746f41b..6fb7d870454a0 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -13,7 +13,7 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { /** Specifies how data is partitioned across different nodes in the cluster. */ def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG WIDTH! /** Specifies any partition requirements on the input data for this operator. */ - def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnknownDistribution) + def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnspecifiedDistribution) /** * Runs this query returning the result as an RDD. diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 014aae7110676..1a25407ff6508 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -53,6 +53,7 @@ case class SortPartitions(sortOrder: Seq[SortOrder], child: SharkPlan) extends U lazy val ordering = new RowOrdering(sortOrder) def execute() = attachTree(this, "sort") { + // TODO: Optimize sorting operation? child.execute() .mapPartitions( iterator => iterator.toArray.sorted(ordering).iterator, diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index efeceaa42d305..febef1658c209 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -35,7 +35,7 @@ object Evaluate extends Logging { @inline def n2(e1: Expression, e2: Expression, f: ((Numeric[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new TreeNodeException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) @@ -53,7 +53,7 @@ object Evaluate extends Logging { @inline def f2(e1: Expression, e2: Expression, f: ((Fractional[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new TreeNodeException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) @@ -71,7 +71,7 @@ object Evaluate extends Logging { @inline def i2(e1: Expression, e2: Expression, f: ((Integral[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new TreeNodeException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) if (evalE1 == null || evalE2 == null) @@ -218,7 +218,7 @@ object Evaluate extends Logging { /* References to input tuples */ case br @ BoundReference(inputTuple, ordinal, _) => try input(inputTuple)(ordinal) catch { case iob: IndexOutOfBoundsException => - throw new OptimizationException(br, s"Reference not in tuple: $input") + throw new TreeNodeException(br, s"Reference not in tuple: $input") } /* Functions */ @@ -229,9 +229,9 @@ object Evaluate extends Logging { implementedFunction.evaluate(implementedFunction.children.map(eval)) case a: Attribute => - throw new OptimizationException(a, + throw new TreeNodeException(a, "Unable to evaluate unbound reference without access to the input schema.") - case other => throw new OptimizationException(other, "evaluation not implemented") + case other => throw new TreeNodeException(other, "evaluation not implemented") } logger.debug(s"Evaluated $e => $result of type ${if (result == null) "null" else result.getClass.getName}, expected: ${e.dataType}") diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala index 792eb88e77a28..87ef2319cc2f1 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala @@ -45,7 +45,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { case a :: Nil => Some(a) // One match, use it. case Nil => None // No matches. case ambiguousReferences => - throw new OptimizationException( + throw new TreeNodeException( this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") } } diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index 1fe9f6a4306a6..2ede02f64048a 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -20,10 +20,11 @@ sealed trait Distribution /** * Represents a distribution where no promises are made about co-location of data. */ -case object UnknownDistribution extends Distribution +case object UnspecifiedDistribution extends Distribution /** - * Represents a distribution where a single operation can observe all tuples in the dataset. + * Represents a distribution that only has a single partition and all tuples of the dataset + * are co-located. */ case object AllTuples extends Distribution @@ -47,25 +48,27 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { sealed trait Partitioning { /** Returns the number of partitions that the data is split across */ - val width: Int + val numPartitions: Int /** - * Returns true iff the guarantees made by this [[Distribution]] are sufficient to satisfy all - * guarantees mandated by the `required` distribution. + * Returns true iff the guarantees made by this + * [[catalyst.plans.physical.Partitioning Partitioning]] are sufficient to satisfy all + * guarantees mandated by the `required` [[catalyst.plans.physical.Distribution Distribution]]. */ def satisfies(required: Distribution): Boolean /** * Returns true iff all distribution guarantees made by this partitioning can also be made - * for the `other` specified partitioning. For example, [[HashPartitioning]]s are only compatible - * if the `width` of the two partitionings is the same. + * for the `other` specified partitioning. + * For example, two [[catalyst.plans.physical.HashPartitioning HashPartitioning]]s are + * only compatible if the `numPartitions` of them is the same. */ def compatibleWith(other: Partitioning): Boolean } -case class UnknownPartitioning(width: Int) extends Partitioning { +case class UnknownPartitioning(numPartitions: Int) extends Partitioning { def satisfies(required: Distribution): Boolean = required match { - case UnknownDistribution => true + case UnspecifiedDistribution => true case _ => false } @@ -75,24 +78,24 @@ case class UnknownPartitioning(width: Int) extends Partitioning { } } -case object Unpartitioned extends Partitioning { - val width = 1 +case object SinglePartition extends Partitioning { + val numPartitions = 1 override def satisfies(required: Distribution): Boolean = true override def compatibleWith(other: Partitioning) = other match { - case Unpartitioned => true + case SinglePartition => true case _ => false } } -case object Broadcast extends Partitioning { - val width = 1 +case object BroadcastPartitioning extends Partitioning { + val numPartitions = 1 override def satisfies(required: Distribution): Boolean = true override def compatibleWith(other: Partitioning) = other match { - case Unpartitioned => true + case SinglePartition => true case _ => false } } @@ -102,7 +105,7 @@ case object Broadcast extends Partitioning { * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be * in the same partition. */ -case class HashPartitioning(expressions: Seq[Expression], width: Int) +case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) extends Expression with Partitioning { @@ -114,14 +117,14 @@ case class HashPartitioning(expressions: Seq[Expression], width: Int) lazy val clusteringSet = expressions.toSet def satisfies(required: Distribution): Boolean = required match { - case UnknownDistribution => true + case UnspecifiedDistribution => true case ClusteredDistribution(requiredClustering) => clusteringSet.subsetOf(requiredClustering.toSet) case _ => false } override def compatibleWith(other: Partitioning) = other match { - case Broadcast => true + case BroadcastPartitioning => true case h: HashPartitioning if h == this => true case _ => false } @@ -136,7 +139,7 @@ case class HashPartitioning(expressions: Seq[Expression], width: Int) * - Each partition will have a `min` and `max` row, relative to the given ordering. All rows * that are in between `min` and `max` in this `ordering` will reside in this partition. */ -case class RangePartitioning(ordering: Seq[SortOrder], width: Int) +case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) extends Expression with Partitioning { @@ -148,7 +151,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], width: Int) lazy val clusteringSet = ordering.map(_.child).toSet def satisfies(required: Distribution): Boolean = required match { - case UnknownDistribution => true + case UnspecifiedDistribution => true case OrderedDistribution(requiredOrdering) => val minSize = Seq(requiredOrdering.size, ordering.size).min requiredOrdering.take(minSize) == ordering.take(minSize) @@ -158,7 +161,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], width: Int) } def compatibleWith(other: Partitioning) = other match { - case Broadcast => true + case BroadcastPartitioning => true case r: RangePartitioning if r == this => true case _ => false } diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index e45dfe55b0ebb..ccfbc7237a564 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -256,7 +256,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } } catch { case e: java.lang.IllegalArgumentException => - throw new OptimizationException( + throw new TreeNodeException( this, s"Failed to copy node. Is otherCopyArgs specified correctly for $nodeName?") } } diff --git a/src/test/scala/catalyst/execution/DistributionTests.scala b/src/test/scala/catalyst/execution/DistributionTests.scala index 61e074127d05b..db37a8e5aa30b 100644 --- a/src/test/scala/catalyst/execution/DistributionTests.scala +++ b/src/test/scala/catalyst/execution/DistributionTests.scala @@ -28,7 +28,7 @@ class DistributionTests extends FunSuite { // Cases which do not need an exchange between two data properties. checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - UnknownDistribution, + UnspecifiedDistribution, true) checkSatisfied( @@ -42,12 +42,12 @@ class DistributionTests extends FunSuite { true) checkSatisfied( - Unpartitioned, + SinglePartition, ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( - Unpartitioned, + SinglePartition, OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) @@ -90,7 +90,7 @@ class DistributionTests extends FunSuite { // Cases which do not need an exchange between two data properties. checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - UnknownDistribution, + UnspecifiedDistribution, true) checkSatisfied( From 7c92a41d73cd89717ea0998044f0cdc8b266bab6 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 27 Jan 2014 01:30:18 +0800 Subject: [PATCH 543/778] Added Hive SerDe support --- .../scala/catalyst/execution/TestShark.scala | 75 ++++++++++++--- .../catalyst/execution/hiveOperators.scala | 92 ++++++++++++------- .../catalyst/execution/HiveSerDeSuite.scala | 14 +++ 3 files changed, 135 insertions(+), 46 deletions(-) create mode 100644 src/test/scala/catalyst/execution/HiveSerDeSuite.scala diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index f297c14ca805e..bbb6e2eacb8ab 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -11,6 +11,10 @@ import scala.language.implicitConversions import org.apache.hadoop.hive.metastore.api.{SerDeInfo, StorageDescriptor} import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.hadoop.hive.ql.exec.FunctionRegistry +import org.apache.hadoop.hive.ql.io.avro.{AvroContainerOutputFormat, AvroContainerInputFormat} +import org.apache.hadoop.hive.serde2.avro.AvroSerDe +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.hive.serde2.RegexSerDe import analysis._ import plans.logical.LogicalPlan @@ -19,8 +23,9 @@ import util._ /** * A locally running test instance of spark. The lifecycle for a given query is managed by the - * inner class [[SharkQuery]]. A [[SharkQuery]] can either be instantiated directly or using the - * implicit conversion '.q'. + * inner class [[catalyst.execution.TestShark.SharkQuery SharkQuery]]. A + * [[catalyst.execution.TestShark.SharkQuery SharkQuery]] can either be instantiated directly or + * using the implicit conversion '.q'. * * {{{ * scala> val query = "SELECT key FROM src".q @@ -37,9 +42,10 @@ import util._ * res0: Array[IndexedSeq[Any]] = Array(Vector(238), Vector(86), Vector(311), ... * }}} * - * Data from [[testTables]] will be automatically loaded whenever a query is run over those tables. - * Calling [[reset]] will delete all tables and other state in the database, leaving the database - * in a "clean" state. + * Data from [[catalyst.execution.TestShark.testTables testTables]] will be automatically loaded + * whenever a query is run over those tables. + * Calling [[catalyst.execution.TestShark.reset reset]] will delete all tables and other state in + * the database, leaving the database in a "clean" state. * * TestShark is implemented as a singleton object because instantiating multiple copies of the hive * metastore seems to lead to weird non-deterministic failures. Therefore, the execution of @@ -164,11 +170,11 @@ object TestShark extends SharkInstance { "CREATE TABLE IF NOT EXISTS dest3 (key INT, value STRING)".cmd), TestTable("srcpart", () => { runSqlHive("CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") - Seq("2008-04-08", "2008-04-09").foreach { ds => - Seq("11", "12").foreach { hr => - val partSpec = Map("ds" -> ds, "hr" -> hr) - runSqlHive(s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr')") - } + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + runSqlHive( + s"""LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' + |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') + """.stripMargin) } }), TestTable("src_thrift", () => { @@ -194,7 +200,51 @@ object TestShark extends SharkInstance { catalog.client.createTable(srcThrift) runSqlHive(s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/complex.seq' INTO TABLE src_thrift") - }) + }), + TestTable("serdeins", + s"""CREATE TABLE serdeins (key INT, value STRING) + |ROW FORMAT SERDE '${classOf[LazySimpleSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ('field.delim'='\\t') + """.stripMargin.cmd, + "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd), + TestTable("sales", + s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) + |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") + """.stripMargin.cmd, + s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/sales.txt' INTO TABLE sales".cmd), + TestTable("episodes", + s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) + |ROW FORMAT SERDE '${classOf[AvroSerDe].getCanonicalName}' + |STORED AS + |INPUTFORMAT '${classOf[AvroContainerInputFormat].getCanonicalName}' + |OUTPUTFORMAT '${classOf[AvroContainerOutputFormat].getCanonicalName}' + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/episodes.avro' INTO TABLE episodes".cmd) ) hiveQTestUtilTables.foreach(registerTestTable) @@ -203,11 +253,12 @@ object TestShark extends SharkInstance { def loadTestTable(name: String) { if (!(loadedTables contains name)) { + // Marks the table as loaded first to prevent infite mutually recursive table loading. + loadedTables += name logger.info(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) - loadedTables += name } } diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index efdaa94950ac7..c1e4748cc2a1d 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -1,16 +1,19 @@ package catalyst package execution +import java.nio.file.Files + import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils -import org.apache.hadoop.hive.ql.plan.FileSinkDesc +import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} import org.apache.hadoop.hive.serde2.AbstractSerDe -import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, StructObjectInspector} -import org.apache.hadoop.hive.serde2.`lazy`.LazyStruct +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption +import org.apache.hadoop.io.NullWritable import org.apache.hadoop.mapred.JobConf +import org.apache.spark.SparkContext._ import expressions.Attribute -import util._ /* Implicits */ import scala.collection.JavaConversions._ @@ -31,18 +34,17 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation * Functions that extract the requested attributes from the hive output. */ @transient - protected lazy val attributeFunctions: Seq[(LazyStruct, Array[String]) => AnyRef] = { + protected lazy val attributeFunctions: Seq[(AnyRef, Array[String]) => AnyRef] = { attributes.map { a => - if (relation.partitionKeys.contains(a)) { - val ordinal = relation.partitionKeys.indexOf(a) - (struct: LazyStruct, partitionKeys: Array[String]) => partitionKeys(ordinal) + val ordinal = relation.partitionKeys.indexOf(a) + if (ordinal >= 0) { + (_: AnyRef, partitionKeys: Array[String]) => partitionKeys(ordinal) } else { val ref = objectInspector.getAllStructFieldRefs .find(_.getFieldName == a.name) .getOrElse(sys.error(s"Can't find attribute $a")) - - (struct: LazyStruct, _: Array[String]) => { - val data = objectInspector.getStructFieldData(struct, ref) + (row: AnyRef, _: Array[String]) => { + val data = objectInspector.getStructFieldData(row, ref) val inspector = ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector] inspector.getPrimitiveJavaObject(data) } @@ -60,14 +62,13 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation def execute() = { inputRdd.map { row => val values = row match { - case Array(struct: LazyStruct, partitionKeys: Array[String]) => - attributeFunctions.map(_(struct, partitionKeys)) - case struct: LazyStruct => - attributeFunctions.map(_(struct, Array.empty)) + case Array(deserializedRow: AnyRef, partitionKeys: Array[String]) => + attributeFunctions.map(_(deserializedRow, partitionKeys)) + case deserializedRow: AnyRef => + attributeFunctions.map(_(deserializedRow, Array.empty)) } buildRow(values.map { - case "NULL" => null - case "null" => null + case n: String if n.toLowerCase == "null" => null case varchar: org.apache.hadoop.hive.common.`type`.HiveVarchar => varchar.getValue case other => other }) @@ -88,12 +89,7 @@ case class InsertIntoHiveTable( */ val desc = new FileSinkDesc("./", table.tableDesc, false) - val outputClass = { - val serializer = - table.tableDesc.getDeserializerClass.newInstance().asInstanceOf[AbstractSerDe] - serializer.initialize(null, table.tableDesc.getProperties) - serializer.getSerializedClass - } + val outputClass = newSerializer(table.tableDesc).getSerializedClass lazy val conf = new JobConf() @@ -105,30 +101,58 @@ case class InsertIntoHiveTable( new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory, "test.out"), null) + private def newSerializer(tableDesc: TableDesc) = { + val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[AbstractSerDe] + serializer.initialize(null, tableDesc.getProperties) + serializer + } + override def otherCopyArgs = sc :: Nil def output = child.output + /** + * Inserts all the rows in the table into Hive. Row objects are properly serialized with the + * [[org.apache.hadoop.hive.serde2.SerDe SerDe]] and the [[org.apache.hadoop.mapred.OutputFormat + * OutputFormat]] provided by the table definition. + */ def execute() = { require(partition.isEmpty, "Inserting into partitioned table not supported.") val childRdd = child.execute() assert(childRdd != null) - // TODO: write directly to hive - val tempDir = java.io.File.createTempFile("data", "tsv") - tempDir.delete() - tempDir.mkdir() - childRdd.map(_.map(a => stringOrNull(a.asInstanceOf[AnyRef])).mkString("\001")) - .saveAsTextFile(tempDir.getCanonicalPath) - - val partitionSpec = - if (partition.nonEmpty) { - s"PARTITION (${partition.map { case (k,v) => s"$k=${v.get}" }.mkString(",")})" + // TODO write directly to Hive + val tempDir = Files.createTempDirectory("data").toFile + + // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer + // instances within the closure, since AbstractSerDe is not serializable while TableDesc is. + val tableDesc = table.tableDesc + childRdd.mapPartitions { iter => + val serializer = newSerializer(tableDesc) + val standardOI = ObjectInspectorUtils + .getStandardObjectInspector(serializer.getObjectInspector, ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + iter.map { row => + (null, serializer.serialize(Array(row: _*), standardOI)) + } + }.saveAsHadoopFile( + tempDir.getCanonicalPath, + classOf[NullWritable], + outputClass, + tableDesc.getOutputFileFormatClass) + + val partitionSpec = if (partition.nonEmpty) { + partition.map { + case (k, Some(v)) => s"$k=$v" + // Dynamic partition inserts + case (k, None) => s"$k" + }.mkString(" PARTITION (", ", ", ")") } else { "" } - sc.runHive(s"LOAD DATA LOCAL INPATH '${tempDir.getCanonicalPath}/*' INTO TABLE ${table.tableName} $partitionSpec") + val inpath = tempDir.getCanonicalPath + "/*" + sc.runHive(s"LOAD DATA LOCAL INPATH '$inpath' INTO TABLE ${table.tableName}$partitionSpec") // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which diff --git a/src/test/scala/catalyst/execution/HiveSerDeSuite.scala b/src/test/scala/catalyst/execution/HiveSerDeSuite.scala new file mode 100644 index 0000000000000..2fea958cc7cac --- /dev/null +++ b/src/test/scala/catalyst/execution/HiveSerDeSuite.scala @@ -0,0 +1,14 @@ +package catalyst.execution + +/** + * A set of tests that validates support for Hive SerDe. + */ +class HiveSerDeSuite extends HiveComparisonTest { + createQueryTest( + "Read and write with LazySimpleSerDe (tab separated)", + "SELECT * from serdeins") + + createQueryTest("Read with RegexSerDe", "SELECT * FROM sales") + + createQueryTest("Read with AvroSerDe", "SELECT * FROM episodes") +} From 6f44caca4377623570395d595c0ee3cad28dbfd6 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 27 Jan 2014 01:31:46 +0800 Subject: [PATCH 544/778] Made TableReader & HadoopTableReader private to catalyst These two may be replaced/removed in the future, so remove them from public API. --- .../catalyst/execution/TableReader.scala | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/src/main/scala/catalyst/execution/TableReader.scala b/src/main/scala/catalyst/execution/TableReader.scala index 0f48788812301..50da479d1a6dc 100644 --- a/src/main/scala/catalyst/execution/TableReader.scala +++ b/src/main/scala/catalyst/execution/TableReader.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.{HadoopRDD, UnionRDD, EmptyRDD, RDD} * type of table storage: HeapTableReader for Shark tables in Spark's block manager, * TachyonTableReader for tables in Tachyon, and HadoopTableReader for Hive tables in a filesystem. */ -sealed trait TableReader { +private[catalyst] sealed trait TableReader { def makeRDDForTable(hiveTable: HiveTable): RDD[_] @@ -34,7 +34,7 @@ sealed trait TableReader { * Helper class for scanning tables stored in Hadoop - e.g., to read Hive tables that reside in the * data warehouse directory. */ -class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) +private[catalyst] class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) extends TableReader { // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless @@ -93,11 +93,9 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf deserializer.initialize(hconf, tableDesc.getProperties) // Deserialize each Writable to get the row value. - iter.map { value => - value match { - case v: Writable => deserializer.deserialize(v) - case _ => throw new RuntimeException("Failed to match " + value.toString) - } + iter.map { + case v: Writable => deserializer.deserialize(v) + case value => throw new RuntimeException("Failed to match " + value.toString) } } deserializedHadoopRDD @@ -130,8 +128,8 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf val ifc = partDesc.getInputFileFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] // Get partition field info - val partSpec = partDesc.getPartSpec() - val partProps = partDesc.getProperties() + val partSpec = partDesc.getPartSpec + val partProps = partDesc.getProperties val partColsDelimited: String = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) // Partitioning columns are delimited by "/" @@ -156,7 +154,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf iter.map { value => val deserializer = localDeserializer.newInstance() deserializer.initialize(hconf, partProps) - val deserializedRow = deserializer.deserialize(value) // LazyStruct + val deserializedRow = deserializer.deserialize(value) rowWithPartArr.update(0, deserializedRow) rowWithPartArr.update(1, partValues) rowWithPartArr.asInstanceOf[Object] @@ -177,11 +175,10 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf */ private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { filterOpt match { - case Some(filter) => { + case Some(filter) => val fs = path.getFileSystem(_localHConf) val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString) filteredFiles.mkString(",") - } case None => path.toString } } @@ -212,7 +209,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf } -object HadoopTableReader { +private[catalyst] object HadoopTableReader { /** * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to From 9dd3b26f6b2b1e2935cfa0ae288dfe73f87da890 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 26 Jan 2014 13:26:10 -0800 Subject: [PATCH 545/778] Fix scaladoc. --- .../scala/catalyst/expressions/package.scala | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/main/scala/catalyst/expressions/package.scala b/src/main/scala/catalyst/expressions/package.scala index f1677561b3f59..877b428258b71 100644 --- a/src/main/scala/catalyst/expressions/package.scala +++ b/src/main/scala/catalyst/expressions/package.scala @@ -8,24 +8,24 @@ package catalyst * * ==Standard Expressions== * A library of standard expressions (e.g., [[Add]], [[Equals]]), aggregates (e.g., SUM, COUNT), and other computations - * (e.g. UDFs). Each type expression is capable of determining its output schema as a function of its children’s output + * (e.g. UDFs). Each expression type is capable of determining its output schema as a function of its children’s output * schema. * * ==Named Expressions== - * Some expression are named and thus can be referenced by other operators in the dataflow graph. The two types of named - * expressions are [[AttributeReference]]s and [[Alias]]es. [[AttributeReference]]s refer to parts of the input tuple f - * or a given operator and form the leaves of all expression trees. Aliases assign a name to intermediate computations. - * For example, in the SQL statement “SELECT a+b AS c FROM ...â€, the expressions ‘a’ and ‘b’ would be represented by - * AttributeReferences and ‘c’ would be represented by an Alias. + * Some expression are named and thus can be referenced by later operators in the dataflow graph. The two types of named + * expressions are [[AttributeReference]]s and [[Alias]]es. [[AttributeReference]]s refer to attributes of the input tuple + * for a given operator and form the leaves of some expression trees. Aliases assign a name to intermediate computations. + * For example, in the SQL statement `SELECT a+b AS c FROM ...`, the expressions `a` and `b` would be represented by + * `AttributeReferences` and `c` would be represented by an `Alias`. * * During [[analysis]], all named expressions are assigned a globally unique expression id, which can be used for equality * comparisons. While the original names are kept around for debugging purposes, they should never be used to check - * attribute equality, as plan transformations can result in the introduction of naming ambiguity. For example, - * consider a plan that contains subqueries, both of which are reading from the same table. If an optimization removes - * the subqueries, scoping information would be destroyed, eliminating the ability to reason about which subquery - * produced a given attribute. + * if two attributes refer to the same value, as plan transformations can result in the introduction of naming ambiguity. + * For example, consider a plan that contains subqueries, both of which are reading from the same table. If an + * optimization removesmthe subqueries, scoping information would be destroyed, eliminating the ability to reason about + * which subquery produced a given attribute. * * ==Evaluation== * The result of expressions can be evaluated using the [[Evaluate]] object. */ -package object expressions \ No newline at end of file +package object expressions From 761bbb8ff0886f24789d2cfc8f00cc77b28d33e6 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 27 Jan 2014 10:08:47 +0800 Subject: [PATCH 546/778] Generalized BindReferences to run against any query plan --- .../catalyst/execution/SharkInstance.scala | 3 +- .../catalyst/expressions/BoundAttribute.scala | 50 +++++++++++-------- 2 files changed, 32 insertions(+), 21 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 256f07efe02ad..595506e5be939 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -81,7 +81,8 @@ abstract class SharkInstance extends Logging { } object PrepareForExecution extends RuleExecutor[SharkPlan] { - val batches = Batch("Prepare Expressions", Once, expressions.BindReferences) :: Nil + val batches = + Batch("Prepare Expressions", Once, new expressions.BindReferences[SharkPlan]) :: Nil } class SharkSqlQuery(sql: String) extends SharkQuery { diff --git a/src/main/scala/catalyst/expressions/BoundAttribute.scala b/src/main/scala/catalyst/expressions/BoundAttribute.scala index edb17a9b5a6e4..4e0158613b063 100644 --- a/src/main/scala/catalyst/expressions/BoundAttribute.scala +++ b/src/main/scala/catalyst/expressions/BoundAttribute.scala @@ -4,7 +4,7 @@ package expressions import rules._ import errors._ -import execution.SharkPlan +import catalyst.plans.QueryPlan /** * A bound reference points to a specific slot in the input tuple, allowing the actual value to be retrieved more @@ -27,27 +27,37 @@ case class BoundReference(inputTuple: Int, ordinal: Int, baseReference: Attribut override def toString = s"$baseReference:$inputTuple.$ordinal" } -// TODO: Should run against any query plan, not just SharkPlans -object BindReferences extends Rule[SharkPlan] { - def apply(plan: SharkPlan): SharkPlan = { +class BindReferences[TreeNode <: QueryPlan[TreeNode]] extends Rule[TreeNode] { + import BindReferences._ + + def apply(plan: TreeNode): TreeNode = { plan.transform { - case leafNode: SharkPlan if leafNode.children.isEmpty => leafNode - case nonLeaf: SharkPlan => attachTree(nonLeaf, "Binding references in operator") { - logger.debug(s"Binding references in node ${nonLeaf.simpleString}") - nonLeaf.transformExpressions { - case a: AttributeReference => attachTree(a, "Binding attribute") { - val inputTuple = nonLeaf.children.indexWhere(_.output contains a) - val ordinal = if (inputTuple == -1) -1 else nonLeaf.children(inputTuple).output.indexWhere(_ == a) - if (ordinal == -1) { - logger.debug(s"No binding found for $a given input ${nonLeaf.children.map(_.output.mkString("{", ",", "}")).mkString(",")}") - a - } else { - logger.debug(s"Binding $a to $inputTuple.$ordinal given input ${nonLeaf.children.map(_.output.mkString("{", ",", "}")).mkString(",")}") - BoundReference(inputTuple, ordinal, a) - } - } + case leafNode if leafNode.children.isEmpty => leafNode + case nonLeaf => nonLeaf.transformExpressions { case e => + bindReference(e, nonLeaf.children.map(_.output)) + } + } + } +} + +object BindReferences extends Logging { + def bindReference(expression: Expression, input: Seq[Seq[Attribute]]): Expression = { + expression.transform { case a: AttributeReference => + attachTree(a, "Binding attribute") { + val inputAsString = input.map(_.mkString("{", ",", "}")).mkString(",") + + for { + (tuple, inputTuple) <- input.zipWithIndex + (attr, ordinal) <- tuple.zipWithIndex + if attr == a + } { + logger.debug(s"Binding $attr to $inputTuple.$ordinal given input $inputAsString") + return BoundReference(inputTuple, ordinal, a) } + + logger.debug(s"No binding found for $a given input $inputAsString") + a } } } -} \ No newline at end of file +} From 9e0d8409e2ed435fa280ba51518f491d717ed8b3 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 27 Jan 2014 10:09:31 +0800 Subject: [PATCH 547/778] Added partition pruning optimization --- .../catalyst/execution/SharkInstance.scala | 1 + .../scala/catalyst/execution/TestShark.scala | 9 ++++ .../catalyst/execution/hiveOperators.scala | 53 ++++++++++++++++--- .../execution/planningStrategies.scala | 30 ++++++++++- .../execution/PartitionPruningSuite.scala | 51 ++++++++++++++++++ 5 files changed, 134 insertions(+), 10 deletions(-) create mode 100644 src/test/scala/catalyst/execution/PartitionPruningSuite.scala diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 595506e5be939..77599ba8e059f 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -73,6 +73,7 @@ abstract class SharkInstance extends Logging { val sc = self.sc val strategies = SparkEquiInnerJoin :: + PartitionPrunings :: HiveTableScans :: DataSinks :: BasicOperators :: diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index bbb6e2eacb8ab..642a6b436a7d3 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -177,6 +177,15 @@ object TestShark extends SharkInstance { """.stripMargin) } }), + TestTable("srcpart1", () => { + runSqlHive("CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { + runSqlHive( + s"""LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' + |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') + """.stripMargin) + } + }), TestTable("src_thrift", () => { import org.apache.thrift.protocol.TBinaryProtocol import org.apache.hadoop.hive.serde2.thrift.test.Complex diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index c1e4748cc2a1d..de00dee8a6f47 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -5,6 +5,7 @@ import java.nio.file.Files import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils +import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ @@ -13,12 +14,26 @@ import org.apache.hadoop.io.NullWritable import org.apache.hadoop.mapred.JobConf import org.apache.spark.SparkContext._ -import expressions.Attribute +import catalyst.expressions._ +import catalyst.types.{BooleanType, DataType} /* Implicits */ import scala.collection.JavaConversions._ -case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation) extends LeafNode { +case class HiveTableScan( + attributes: Seq[Attribute], + relation: MetastoreRelation, + partitionPruningPred: Option[Expression]) + extends LeafNode { + + private val boundPruningPred = partitionPruningPred.map { pred => + require( + pred.dataType == BooleanType, + s"Data type of predicate $pred must be BooleanType rather than ${pred.dataType}.") + + BindReferences.bindReference(pred, Seq(relation.partitionKeys)) + } + @transient val hadoopReader = new HadoopTableReader(relation.tableDesc, SharkContext.hiveconf) @@ -34,16 +49,20 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation * Functions that extract the requested attributes from the hive output. */ @transient - protected lazy val attributeFunctions: Seq[(AnyRef, Array[String]) => AnyRef] = { + protected lazy val attributeFunctions: Seq[(Any, Array[String]) => Any] = { attributes.map { a => val ordinal = relation.partitionKeys.indexOf(a) if (ordinal >= 0) { - (_: AnyRef, partitionKeys: Array[String]) => partitionKeys(ordinal) + (_: Any, partitionKeys: Array[String]) => { + val value = partitionKeys(ordinal) + val dataType = relation.partitionKeys(ordinal).dataType + castFromString(value, dataType) + } } else { val ref = objectInspector.getAllStructFieldRefs .find(_.getFieldName == a.name) .getOrElse(sys.error(s"Can't find attribute $a")) - (row: AnyRef, _: Array[String]) => { + (row: Any, _: Array[String]) => { val data = objectInspector.getStructFieldData(row, ref) val inspector = ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector] inspector.getPrimitiveJavaObject(data) @@ -52,11 +71,29 @@ case class HiveTableScan(attributes: Seq[Attribute], relation: MetastoreRelation } } + private def castFromString(value: String, dataType: DataType) = { + Evaluate(Cast(Literal(value), dataType), Nil) + } + @transient def inputRdd = if (!relation.hiveQlTable.isPartitioned) { hadoopReader.makeRDDForTable(relation.hiveQlTable) } else { - hadoopReader.makeRDDForPartitionedTable(relation.hiveQlPartitions) + hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions)) + } + + private[catalyst] def prunePartitions(partitions: Seq[HivePartition]) = { + boundPruningPred match { + case None =>partitions + case Some(shouldKeep) => partitions.filter { part => + val dataTypes = relation.partitionKeys.map(_.dataType) + val castedValues = for ((value, dataType) <- part.getValues.zip(dataTypes)) yield { + castFromString(value, dataType) + } + val row = new GenericRow(castedValues) + Evaluate(shouldKeep, Seq(row)).asInstanceOf[Boolean] + } + } } def execute() = { @@ -113,8 +150,8 @@ case class InsertIntoHiveTable( /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the - * [[org.apache.hadoop.hive.serde2.SerDe SerDe]] and the [[org.apache.hadoop.mapred.OutputFormat - * OutputFormat]] provided by the table definition. + * [[org.apache.hadoop.hive.serde2.SerDe SerDe]] and the + * [[org.apache.hadoop.mapred.OutputFormat OutputFormat]] provided by the table definition. */ def execute() = { require(partition.isEmpty, "Inserting into partitioned table not supported.") diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index c631f3ebba512..17524f88399ba 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -23,9 +23,9 @@ trait PlanningStrategies { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { // Push attributes into table scan when possible. case p @ logical.Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => - execution.HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m) :: Nil + execution.HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m, None) :: Nil case m: MetastoreRelation => - execution.HiveTableScan(m.output, m) :: Nil + execution.HiveTableScan(m.output, m, None) :: Nil case _ => Nil } @@ -41,6 +41,32 @@ trait PlanningStrategies { } } + object PartitionPrunings extends Strategy { + def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { + case p @ FilteredOperation(predicates, relation: MetastoreRelation) => + if (predicates.isEmpty) { + execution.HiveTableScan(relation.output, relation, None) :: Nil + } else { + val partitionKeys = relation.partitionKeys.map(_.name).toSet + val pruningPreds = predicates.filter {e => + val referenceNames = e.references.map(_.name) + referenceNames.subsetOf(partitionKeys) + } + + val tableScan = if (pruningPreds.isEmpty) { + execution.HiveTableScan(relation.output, relation, None) + } else { + execution.HiveTableScan(relation.output, relation, Some(pruningPreds.reduceLeft(And))) + } + + execution.Filter(predicates.reduceLeft(And), tableScan) :: Nil + } + + case _ => + Nil + } + } + object SparkEquiInnerJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case FilteredOperation(predicates, logical.Join(left, right, Inner, condition)) => diff --git a/src/test/scala/catalyst/execution/PartitionPruningSuite.scala b/src/test/scala/catalyst/execution/PartitionPruningSuite.scala new file mode 100644 index 0000000000000..f13317ddc9028 --- /dev/null +++ b/src/test/scala/catalyst/execution/PartitionPruningSuite.scala @@ -0,0 +1,51 @@ +package catalyst.execution + +import scala.collection.JavaConversions._ + +import TestShark._ + +class PartitionPruningSuite extends HiveComparisonTest { + createPruningTest("Pruning with predicate on STRING partition key", + "SELECT * FROM srcpart1 WHERE ds = '2008-04-08'", + Seq( + Seq("2008-04-08", "11"), + Seq("2008-04-08", "12"))) + + createPruningTest("Pruning with predicate on INT partition key", + "SELECT * FROM srcpart1 WHERE hr < 12", + Seq( + Seq("2008-04-08", "11"), + Seq("2008-04-09", "11"))) + + createPruningTest("Select only 1 partition", + "SELECT * FROM srcpart1 WHERE ds = '2008-04-08' AND hr < 12", + Seq( + Seq("2008-04-08", "11"))) + + createPruningTest("All partitions pruned", + "SELECT * FROM srcpart1 WHERE ds = '2014-01-27' AND hr = 11", + Seq.empty) + + createPruningTest("Pruning with both column key and partition key", + "SELECT * FROM srcpart1 WHERE value IS NOT NULL AND hr < 12", + Seq( + Seq("2008-04-08", "11"), + Seq("2008-04-09", "11"))) + + def createPruningTest(testCaseName: String, sql: String, expectedValues: Seq[Seq[String]]) = { + test(testCaseName) { + val plan = sql.q.executedPlan + val prunedPartitions = plan.collect { + case p @ HiveTableScan(_, relation, _) => + p.prunePartitions(relation.hiveQlPartitions) + }.head + val values = prunedPartitions.map(_.getValues) + + assert(prunedPartitions.size === expectedValues.size) + + for ((actual, expected) <- values.zip(expectedValues)) { + assert(actual sameElements expected) + } + } + } +} From 111ffdcd5b7a0f3dfd0c649c91ebaaca69ca7793 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 27 Jan 2014 12:43:53 +0800 Subject: [PATCH 548/778] More comments and minor reformatting --- .../catalyst/execution/hiveOperators.scala | 24 +++++++++++++++++-- .../execution/planningStrategies.scala | 14 ++++++++++- 2 files changed, 35 insertions(+), 3 deletions(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index de00dee8a6f47..89fefa0605c37 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -20,12 +20,22 @@ import catalyst.types.{BooleanType, DataType} /* Implicits */ import scala.collection.JavaConversions._ +/** + * The Hive table scan operator. Column and partition pruning are both handled. + * + * @constructor + * @param attributes Attributes to be fetched from the Hive table. + * @param relation The Hive table be be scanned. + * @param partitionPruningPred An optional partition pruning predicate for partitioned table. + */ case class HiveTableScan( attributes: Seq[Attribute], relation: MetastoreRelation, partitionPruningPred: Option[Expression]) extends LeafNode { + // Bind all partition key attribute references to the partition pruning predicate for later + // evaluation. private val boundPruningPred = partitionPruningPred.map { pred => require( pred.dataType == BooleanType, @@ -46,7 +56,8 @@ case class HiveTableScan( relation.tableDesc.getDeserializer.getObjectInspector.asInstanceOf[StructObjectInspector] /** - * Functions that extract the requested attributes from the hive output. + * Functions that extract the requested attributes from the hive output. Partitioned values are + * casted from string to its declared data type. */ @transient protected lazy val attributeFunctions: Seq[(Any, Array[String]) => Any] = { @@ -82,14 +93,23 @@ case class HiveTableScan( hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions)) } + /** + * Prunes partitions not involve the query plan. + * + * @param partitions All partitions of the relation. + * @return Partitions that are involved in the query plan. + */ private[catalyst] def prunePartitions(partitions: Seq[HivePartition]) = { boundPruningPred match { - case None =>partitions + case None => partitions case Some(shouldKeep) => partitions.filter { part => val dataTypes = relation.partitionKeys.map(_.dataType) val castedValues = for ((value, dataType) <- part.getValues.zip(dataTypes)) yield { castFromString(value, dataType) } + + // Only partitioned values are needed here, since the predicate has already been bound to + // partition key attribute references. val row = new GenericRow(castedValues) Evaluate(shouldKeep, Seq(row)).asInstanceOf[Boolean] } diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 17524f88399ba..5c82c47142e1f 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -41,6 +41,15 @@ trait PlanningStrategies { } } + /** + * A strategy used to detect filtering predicates on top of a partitioned relation to help + * partition pruning. + * + * This strategy itself doesn't perform partition pruning, it just collects and combines all the + * partition pruning predicates and pass them down to the underlying + * [[catalyst.execution.HiveTableScan HiveTableScan]] operator, which does the actual pruning + * work. + */ object PartitionPrunings extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case p @ FilteredOperation(predicates, relation: MetastoreRelation) => @@ -48,6 +57,8 @@ trait PlanningStrategies { execution.HiveTableScan(relation.output, relation, None) :: Nil } else { val partitionKeys = relation.partitionKeys.map(_.name).toSet + + // Filter out all predicates that only deal with partition keys val pruningPreds = predicates.filter {e => val referenceNames = e.references.map(_.name) referenceNames.subsetOf(partitionKeys) @@ -56,7 +67,8 @@ trait PlanningStrategies { val tableScan = if (pruningPreds.isEmpty) { execution.HiveTableScan(relation.output, relation, None) } else { - execution.HiveTableScan(relation.output, relation, Some(pruningPreds.reduceLeft(And))) + val combinedPruningPred = pruningPreds.reduceLeft(And) + execution.HiveTableScan(relation.output, relation, Some(combinedPruningPred)) } execution.Filter(predicates.reduceLeft(And), tableScan) :: Nil From 4e5e4d43dafb9eb59f80f493a48e1f96b43ee3f8 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 27 Jan 2014 13:06:47 +0800 Subject: [PATCH 549/778] Added PreInsertionCasts to do necessary casting before insertion --- .../scala/catalyst/analysis/Analyzer.scala | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 58f69650057f2..374b5b882fe6f 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -1,6 +1,7 @@ package catalyst package analysis +import execution.MetastoreRelation import expressions._ import plans.logical._ import rules._ @@ -34,6 +35,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool StarExpansion :: ResolveFunctions :: GlobalAggregates :: + PreInsertionCasts :: typeCoercionRules :_*) ) @@ -141,4 +143,29 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool protected def containsStar(exprs: Seq[NamedExpression]): Boolean = exprs.collect { case _: Star => true }.nonEmpty } + + /** + * Casts input data to correct data types according to table definition before inserting into + * that table. + */ + object PreInsertionCasts extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan.transform { + // Wait until children are resolved + case p: LogicalPlan if !p.childrenResolved => p + + case p @ InsertIntoTable(table: MetastoreRelation, partition, child) => + val childOutputDataTypes = child.output.map(_.dataType) + val tableOutputDataTypes = table.output.map(_.dataType) + + // Only do the casting when child output data types differ from table output data types. + if (childOutputDataTypes sameElements tableOutputDataTypes) { + p + } else { + val castedChildOutput = child.output.zip(tableOutputDataTypes).map { + case (a, dataType) => Alias(Cast(a, dataType), a.name)() + } + p.copy(table, partition, Project(castedChildOutput, child)) + } + } + } } From cb49af01d58c5f79c7e1977781231059fe134954 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 27 Jan 2014 13:09:45 +0800 Subject: [PATCH 550/778] Fixed Scaladoc links --- .../scala/catalyst/analysis/Analyzer.scala | 29 ++++++++++++------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 374b5b882fe6f..9f410f31e9c7e 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -7,16 +7,20 @@ import plans.logical._ import rules._ /** - * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing + * A trivial [[catalyst.analysis.Analyzer Analyzer]] with an + * [[catalyst.analysis.EmptyCatalog EmptyCatalog]] and + * [[catalyst.analysis.EmptyFunctionRegistry EmptyFunctionRegistry]]. Used for testing * when all relations are already filled in and the analyser needs only to resolve attribute * references. */ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true) /** - * Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and - * [[UnresolvedRelation]]s into fully typed objects using information in a schema [[Catalog]] and - * a [[FunctionRegistry]]. + * Provides a logical query plan analyzer, which translates + * [[catalyst.analysis.UnresolvedAttribute UnresolvedAttribute]]s and + * [[catalyst.analysis.UnresolvedRelation UnresolvedRelation]]s into fully typed objects using + * information in a schema [[catalyst.analysis.Catalog Catalog]] and a + * [[catalyst.analysis.FunctionRegistry]]. */ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean) extends RuleExecutor[LogicalPlan] with HiveTypeCoercion { @@ -40,7 +44,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) /** - * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. + * Replaces [[catalyst.analysis.UnresolvedRelation UnresolvedRelation]]s with concrete relations + * from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -64,8 +69,9 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Replaces [[UnresolvedAttribute]]s with concrete - * [[expressions.AttributeReference AttributeReferences]] from a logical plan node's children. + * Replaces [[catalyst.analysis.UnresolvedAttribute UnresolvedAttribute]]s with concrete + * [[catalyst.expressions.AttributeReference AttributeReferences]] from a logical plan node's + * children. */ object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { @@ -82,7 +88,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Replaces [[UnresolvedFunction]]s with concrete [[expressions.Expression Expressions]]. + * Replaces [[catalyst.analysis.UnresolvedFunction UnresolvedFunction]]s with concrete + * [[catalyst.expressions.Expression Expressions]]. */ object ResolveFunctions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -108,12 +115,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case agg: AggregateExpression => return true case _ => }) - return false + false } } /** - * Expands any references to [[Star]] (*) in project operators. + * Expands any references to [[catalyst.analysis.Star Star]] (*) in project operators. */ object StarExpansion extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -138,7 +145,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Returns true if `exprs` contains a [[Star]]. + * Returns true if `exprs` contains a [[catalyst.analysis.Star Star]]. */ protected def containsStar(exprs: Seq[NamedExpression]): Boolean = exprs.collect { case _: Star => true }.nonEmpty From 57799add8e8e8cdf42aea407e0ef26d4f9bf9c83 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 27 Jan 2014 13:42:06 +0800 Subject: [PATCH 551/778] Added special treat for HiveVarchar in InsertIntoHiveTable Should be fixed later by adding a new data type for HiveQL VARCHAR. --- .../scala/catalyst/execution/hiveOperators.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 89fefa0605c37..4bf486f6478b1 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -4,12 +4,14 @@ package execution import java.nio.file.Files import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.`type`.HiveVarchar import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption +import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector import org.apache.hadoop.io.NullWritable import org.apache.hadoop.mapred.JobConf import org.apache.spark.SparkContext._ @@ -189,8 +191,16 @@ case class InsertIntoHiveTable( val standardOI = ObjectInspectorUtils .getStandardObjectInspector(serializer.getObjectInspector, ObjectInspectorCopyOption.JAVA) .asInstanceOf[StructObjectInspector] + iter.map { row => - (null, serializer.serialize(Array(row: _*), standardOI)) + // TODO Should add a new VarcharType data type to handle HiveQL VARCHAR + val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector) + val mappedRow = row.zip(fieldOIs).map { + case (s: String, oi: JavaHiveVarcharObjectInspector) => new HiveVarchar(s, s.size) + case (obj, _) => obj + } + + (null, serializer.serialize(Array(mappedRow: _*), standardOI)) } }.saveAsHadoopFile( tempDir.getCanonicalPath, From 99c67073d95a6115af378c429f81a00b27893b3b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 11:48:25 -0800 Subject: [PATCH 552/778] upgrade spark --- build.sbt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index bc74bd56ff186..a1a52867305e7 100644 --- a/build.sbt +++ b/build.sbt @@ -12,7 +12,10 @@ scalacOptions ++= Seq("-deprecation", "-feature", "-unchecked") resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" -libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating-SNAPSHOT" +// TODO: Remove when Spark 0.9.0 is released for real. +resolvers += "SparkStaging" at "https://repository.apache.org/content/repositories/orgapachespark-1005/" + +libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" libraryDependencies += "catalyst" % "hive-golden" % "4" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden4.jar" From e9347fc31d8d1b66d50d17ac1ef1adb675745c4f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 26 Jan 2014 22:51:47 -0800 Subject: [PATCH 553/778] Remove broken scaladoc links. --- src/main/scala/catalyst/execution/hiveOperators.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 4bf486f6478b1..761e525da4e8d 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -172,8 +172,8 @@ case class InsertIntoHiveTable( /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the - * [[org.apache.hadoop.hive.serde2.SerDe SerDe]] and the - * [[org.apache.hadoop.mapred.OutputFormat OutputFormat]] provided by the table definition. + * `org.apache.hadoop.hive.serde2.SerDe` and the + * `org.apache.hadoop.mapred.OutputFormat` provided by the table definition. */ def execute() = { require(partition.isEmpty, "Inserting into partitioned table not supported.") @@ -227,4 +227,4 @@ case class InsertIntoHiveTable( // TODO: implement hive compatibility as rules. sc.makeRDD(Nil, 1) } -} \ No newline at end of file +} From 9fb357ab90bca4d8b801983b952ff99085eb11f9 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 08:55:35 +0800 Subject: [PATCH 554/778] use getSpecifiedDistribution to create Distribution. ClusteredDistribution and OrderedDistribution do not take Nil as inptu expressions. --- .../scala/catalyst/execution/aggregates.scala | 5 +- src/main/scala/catalyst/execution/joins.scala | 14 ++++-- .../plans/physical/partitioning.scala | 46 ++++++++++++++--- .../execution/DistributionTests.scala | 49 ++++++++++--------- 4 files changed, 74 insertions(+), 40 deletions(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 40925bd6bab1b..d7fcd3d25d46a 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,11 +3,10 @@ package execution import catalyst.errors._ import catalyst.expressions._ -import catalyst.plans.physical._ +import catalyst.plans.physical.Distribution._ import org.apache.spark.rdd.SharkPairRDDFunctions /* Implicits */ -import org.apache.spark.SparkContext._ import SharkPairRDDFunctions._ case class Aggregate( @@ -16,7 +15,7 @@ case class Aggregate( child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { - override def requiredChildDistribution = Seq(ClusteredDistribution(groupingExpressions)) + override def requiredChildDistribution = getSpecifiedDistribution(groupingExpressions) :: Nil override def otherCopyArgs = sc :: Nil def output = aggregateExpressions.map(_.toAttribute) diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index ef2d6205f2b40..267b020dfc986 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -2,24 +2,27 @@ package catalyst package execution import scala.collection.mutable - -import org.apache.spark.rdd.RDD +import scala.Some import errors._ import expressions._ import plans._ -import plans.physical._ - +import catalyst.plans.physical.Distribution._ import org.apache.spark.rdd.SharkPairRDDFunctions._ +import org.apache.spark.rdd.RDD +import catalyst.plans.physical.Partitioning + case class SparkEquiInnerJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SharkPlan, right: SharkPlan) extends BinaryNode { + override def outputPartitioning: Partitioning = left.outputPartitioning + override def requiredChildDistribution = - ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + getSpecifiedDistribution(leftKeys) :: getSpecifiedDistribution(rightKeys) :: Nil def output = left.output ++ right.output @@ -66,6 +69,7 @@ case class BroadcastNestedLoopJoin( streamed: SharkPlan, broadcast: SharkPlan, joinType: JoinType, condition: Option[Expression]) (@transient sc: SharkContext) extends BinaryNode { + // TODO: Override requiredChildDistribution, outputPartitioning, and use Exchange to broadcast override def otherCopyArgs = sc :: Nil diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index 2ede02f64048a..dbcbb4fd439bf 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -28,13 +28,20 @@ case object UnspecifiedDistribution extends Distribution */ case object AllTuples extends Distribution +// TODO: Add a BroadcastDistribution? /** * Represents data where tuples that share the same values for the `clustering` * [[catalyst.expressions.Expression Expressions]] will be co-located. Based on the context, this * can mean such tuples are either co-located in the same partition or they will be contiguous * within a single partition. */ -case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution +case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution { + if (clustering == Nil) { + throw new IllegalArgumentException("The clustering expressions of a ClusteredDistribution " + + "should not be Nil. An AllTuples should be used to represent a distribution that only has " + + "a single partition.") + } +} /** * Represents data where tuples have been ordered according to the `ordering` @@ -43,17 +50,40 @@ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distributi * the ordering expressions are contiguous and will never be split across partitions. */ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { + if (ordering == Nil) { + throw new IllegalArgumentException("The ordering expressions of a OrderedDistribution " + + "should not be Nil. An AllTuples should be used to represent a distribution that only has " + + "a single partition.") + } + def clustering = ordering.map(_.child).toSet } +object Distribution { + def getSpecifiedDistribution(expressions: Seq[Expression]): Distribution = { + if (expressions == Nil) { + AllTuples + } else { + if (expressions.forall(exp => exp.isInstanceOf[SortOrder])) { + OrderedDistribution(expressions.asInstanceOf[Seq[SortOrder]]) + } else { + ClusteredDistribution(expressions) + } + } + } +} + sealed trait Partitioning { /** Returns the number of partitions that the data is split across */ val numPartitions: Int /** * Returns true iff the guarantees made by this - * [[catalyst.plans.physical.Partitioning Partitioning]] are sufficient to satisfy all - * guarantees mandated by the `required` [[catalyst.plans.physical.Distribution Distribution]]. + * [[catalyst.plans.physical.Partitioning Partitioning]] are sufficient to satisfy + * the partitioning scheme mandated by the `required` + * [[catalyst.plans.physical.Distribution Distribution]], i.e. the current dataset does not + * need to be re-partitioned for the `required` Distribution (it is possible that tuples within + * a partition need to be reorganized). */ def satisfies(required: Distribution): Boolean @@ -81,9 +111,9 @@ case class UnknownPartitioning(numPartitions: Int) extends Partitioning { case object SinglePartition extends Partitioning { val numPartitions = 1 - override def satisfies(required: Distribution): Boolean = true + def satisfies(required: Distribution): Boolean = true - override def compatibleWith(other: Partitioning) = other match { + def compatibleWith(other: Partitioning) = other match { case SinglePartition => true case _ => false } @@ -92,9 +122,9 @@ case object SinglePartition extends Partitioning { case object BroadcastPartitioning extends Partitioning { val numPartitions = 1 - override def satisfies(required: Distribution): Boolean = true + def satisfies(required: Distribution): Boolean = true - override def compatibleWith(other: Partitioning) = other match { + def compatibleWith(other: Partitioning) = other match { case SinglePartition => true case _ => false } @@ -123,7 +153,7 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case _ => false } - override def compatibleWith(other: Partitioning) = other match { + def compatibleWith(other: Partitioning) = other match { case BroadcastPartitioning => true case h: HashPartitioning if h == this => true case _ => false diff --git a/src/test/scala/catalyst/execution/DistributionTests.scala b/src/test/scala/catalyst/execution/DistributionTests.scala index db37a8e5aa30b..0485ff6ab7bb1 100644 --- a/src/test/scala/catalyst/execution/DistributionTests.scala +++ b/src/test/scala/catalyst/execution/DistributionTests.scala @@ -5,6 +5,7 @@ package physical import org.scalatest.FunSuite import catalyst.dsl._ +import catalyst.plans.physical.Distribution._ class DistributionTests extends FunSuite { @@ -15,16 +16,16 @@ class DistributionTests extends FunSuite { if(inputPartitioning.satisfies(requiredDistribution) != satisfied) fail( s""" - |== Input Distribution == + |== Input Partitioning == |$inputPartitioning |== Required Distribution == |$requiredDistribution - |== Does input distribution satisfy requirements? == + |== Does input partitioning satisfy required distribution? == |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} """.stripMargin) } - test("needExchange test: ClusteredDistribution is the output DataProperty") { + test("HashPartitioning is the output partitioning") { // Cases which do not need an exchange between two data properties. checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), @@ -33,48 +34,48 @@ class DistributionTests extends FunSuite { checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - ClusteredDistribution(Seq('a, 'b, 'c)), + getSpecifiedDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( HashPartitioning(Seq('b, 'c), 10), - ClusteredDistribution(Seq('a, 'b, 'c)), + getSpecifiedDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( SinglePartition, - ClusteredDistribution(Seq('a, 'b, 'c)), + getSpecifiedDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( SinglePartition, - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) // Cases which need an exchange between two data properties. checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - ClusteredDistribution(Seq('b, 'c)), + getSpecifiedDistribution(Seq('b, 'c)), false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - ClusteredDistribution(Seq('d, 'e)), + getSpecifiedDistribution(Seq('d, 'e)), false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - AllTuples, + getSpecifiedDistribution(Nil), false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) checkSatisfied( HashPartitioning(Seq('b, 'c), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) // TODO: We should check functional dependencies @@ -86,7 +87,7 @@ class DistributionTests extends FunSuite { */ } - test("needExchange test: OrderedDistribution is the output DataProperty") { + test("RangePartitioning is the output partitioning") { // Cases which do not need an exchange between two data properties. checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), @@ -95,32 +96,32 @@ class DistributionTests extends FunSuite { checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('a, 'b, 'c)), + getSpecifiedDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('c, 'b, 'a)), + getSpecifiedDistribution(Seq('c, 'b, 'a)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('b, 'c, 'a, 'd)), + getSpecifiedDistribution(Seq('b, 'c, 'a, 'd)), true) // Cases which need an exchange between two data properties. @@ -130,27 +131,27 @@ class DistributionTests extends FunSuite { // and the parallelism. checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('b.asc, 'a.asc)), + getSpecifiedDistribution(Seq('b.asc, 'a.asc)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('a, 'b)), + getSpecifiedDistribution(Seq('a, 'b)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('c, 'd)), + getSpecifiedDistribution(Seq('c, 'd)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Nil), + getSpecifiedDistribution(Nil), false) } } \ No newline at end of file From 040fbdf02262e2ec9e02f7a789196260ad8ffc06 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 08:57:35 +0800 Subject: [PATCH 555/778] AddExchange is the only place to add Exchange operators. --- .../scala/catalyst/execution/basicOperators.scala | 11 ++++++++++- .../scala/catalyst/execution/planningStrategies.scala | 10 ++++++---- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 1a25407ff6508..487de52dc1cc4 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -3,6 +3,9 @@ package execution import catalyst.errors._ import catalyst.expressions._ +import catalyst.plans.physical.Distribution._ +import catalyst.expressions.SortOrder +import catalyst.plans.physical.{UnspecifiedDistribution, Distribution} case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -48,7 +51,13 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class SortPartitions(sortOrder: Seq[SortOrder], child: SharkPlan) extends UnaryNode { +case class SortPartitions( + sortOrder: Seq[SortOrder], + child: SharkPlan) + (override val requiredChildDistribution: Seq[Distribution] = UnspecifiedDistribution :: Nil) + extends UnaryNode { + override def otherCopyArgs = requiredChildDistribution :: Nil + @transient lazy val ordering = new RowOrdering(sortOrder) diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 959115b1278e2..558ccad8bcdc7 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -113,11 +113,13 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - // First repartition then sort locally. - execution.SortPartitions(sortExprs, - execution.Exchange(RangePartitioning(sortExprs, 8), planLater(child))) :: Nil + // Set the requiredDistribution of this SortPartitions to OrderedDistribution. + execution.SortPartitions( + sortExprs, + planLater(child))(OrderedDistribution(sortExprs) :: Nil) :: Nil case logical.SortPartitions(sortExprs, child) => - execution.SortPartitions(sortExprs, planLater(child)) :: Nil + // Set the requiredDistribution of this SortPartitions to UnspecifiedDistribution. + execution.SortPartitions(sortExprs, planLater(child))() :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => From 4802f69f4236a32d30ddf021daa31d243a472872 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 08:58:59 +0800 Subject: [PATCH 556/778] The outputPartitioning of a UnaryNode inherits its child's outputPartitioning by default. Also, update the logic in AddExchange to avoid unnecessary shuffling operations. --- .../scala/catalyst/execution/Exchange.scala | 42 ++++++++++++++----- .../scala/catalyst/execution/SharkPlan.scala | 1 + 2 files changed, 33 insertions(+), 10 deletions(-) diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 698167e16cb7c..076d4a8e69a9f 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -5,7 +5,6 @@ import catalyst.rules.Rule import catalyst.errors._ import catalyst.expressions._ import catalyst.plans.physical._ -import catalyst.types._ import org.apache.spark.{RangePartitioner, HashPartitioner} import org.apache.spark.rdd.ShuffledRDD @@ -18,7 +17,7 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) def execute() = attachTree(this , "execute") { newPartitioning match { - case HashPartitioning(expressions, numPartitions) => + case HashPartitioning(expressions, numPartitions) => { // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().map { row => (buildRow(expressions.toSeq.map(Evaluate(_, Vector(row)))), row) @@ -27,17 +26,24 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) shuffled.map(_._2) - - case RangePartitioning(sortingExpressions, numPartitions) => + } + case RangePartitioning(sortingExpressions, numPartitions) => { // TODO: ShuffledRDD should take an Ordering. implicit val ordering = new RowOrdering(sortingExpressions) - val rdd = child.execute().map(r => (r,null)) + val rdd = child.execute().map(row => (row, null)) val part = new RangePartitioner(numPartitions, rdd, ascending = true) val shuffled = new ShuffledRDD[Row, Null, (Row, Null)](rdd, part) + shuffled.map(_._1) + } + case SinglePartition => { + val rdd = child.execute().coalesce(1, true) + + rdd + } case _ => sys.error(s"Exchange not implemented for $newPartitioning") - // TODO: Handle SinglePartition and BroadcastPartitioning. + // TODO: Handle BroadcastPartitioning. } } } @@ -53,6 +59,8 @@ object AddExchange extends Rule[SharkPlan] { def apply(plan: SharkPlan): SharkPlan = plan.transformUp { case operator: SharkPlan => + // Check if every child's outputPartitioning satisfies the corresponding + // required data distribution. def meetsRequirements = !operator.requiredChildDistribution.zip(operator.children).map { case (required, child) => @@ -63,6 +71,13 @@ object AddExchange extends Rule[SharkPlan] { valid }.exists(_ == false) + // Check if outputPartitionings of children are compatible with each other. + // It is possible that every child satisfies its required data distribution + // but two children have incompatible outputPartitionings. For example, + // A dataset is range partitioned by "a.asc" (RangePartitioning) and another + // dataset is hash partitioned by "a" (HashPartitioning). Tuples in these two + // datasets are both clustered by "a", but these two outputPartitionings are not + // compatible. // TODO: ASSUMES TRANSITIVITY? def compatible = !operator.children @@ -73,18 +88,25 @@ object AddExchange extends Rule[SharkPlan] { case Seq(a,b) => a compatibleWith b }.exists(_ == false) + // Check if the partitioning we want to ensure is the same as the child's output + // partitioning. If so, we do not need to add the Exchange operator. + def addExchangeIfNecessary(partitioning: Partitioning, child: SharkPlan) = + if (child.outputPartitioning != partitioning) Exchange(partitioning, child) else child if (meetsRequirements && compatible) { operator } else { - // TODO: It is possible that only a child does not meet requirement. + // At least one child does not satisfies its required data distribution or + // at least one child's outputPartitioning is not compatible with another child's + // outputPartitioning. In this case, we need to add Exchange operators. val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { + case (AllTuples, child) => + addExchangeIfNecessary(SinglePartition, child) case (ClusteredDistribution(clustering), child) => - Exchange(HashPartitioning(clustering, numPartitions), child) + addExchangeIfNecessary(HashPartitioning(clustering, numPartitions), child) case (OrderedDistribution(ordering), child) => - Exchange(RangePartitioning(ordering, numPartitions), child) + addExchangeIfNecessary(RangePartitioning(ordering, numPartitions), child) case (UnspecifiedDistribution, child) => child - // TODO: Handle AllTuples. One possible way is to use Exchange(HashPartitioning(Nil, 1), child). case (dist, _) => sys.error(s"Don't know how to ensure $dist") } operator.withNewChildren(repartitionedChildren) diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 6fb7d870454a0..c22ec2831fe1b 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -34,6 +34,7 @@ trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { self: Product => + override def outputPartitioning: Partitioning = child.outputPartitioning } trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { From e811d1a287aa692fe4d38246230ad0620630a4cf Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 09:03:01 +0800 Subject: [PATCH 557/778] markhamstra's comments --- src/main/scala/catalyst/trees/TreeNode.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index ccfbc7237a564..814686d8936eb 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -274,7 +274,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case tn: TreeNode[_] if children contains tn => Nil case tn: TreeNode[_] if tn.toString contains "\n" => s"(${tn.simpleString})" :: Nil case seq: Seq[_] => seq.mkString("[", ",", "]") :: Nil - case seq: Set[_] => seq.mkString("{", ",", "}") :: Nil + case set: Set[_] => set.mkString("{", ",", "}") :: Nil case other => other :: Nil }.mkString(", ") From 9ebff4776cab41553ce2376f2fb652938cbaac23 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 09:04:52 +0800 Subject: [PATCH 558/778] remove unnecessary .toSeq --- src/main/scala/catalyst/plans/physical/partitioning.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index dbcbb4fd439bf..ee382ede8a79e 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -139,7 +139,7 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) extends Expression with Partitioning { - def children = expressions.toSeq + def children = expressions def references = expressions.flatMap(_.references).toSet def nullable = false def dataType = IntegerType @@ -173,7 +173,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) extends Expression with Partitioning { - def children = ordering.toSeq + def children = ordering def references = ordering.flatMap(_.references).toSet def nullable = false def dataType = IntegerType From 70994a38897f3af03db6a841b999e7218924f2d8 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Tue, 28 Jan 2014 10:17:20 +0800 Subject: [PATCH 559/778] Revert unnecessary Scaladoc changes --- .../scala/catalyst/analysis/Analyzer.scala | 27 +++++++------------ 1 file changed, 10 insertions(+), 17 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 9f410f31e9c7e..25f29ccaa5ed5 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -7,20 +7,16 @@ import plans.logical._ import rules._ /** - * A trivial [[catalyst.analysis.Analyzer Analyzer]] with an - * [[catalyst.analysis.EmptyCatalog EmptyCatalog]] and - * [[catalyst.analysis.EmptyFunctionRegistry EmptyFunctionRegistry]]. Used for testing + * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing * when all relations are already filled in and the analyser needs only to resolve attribute * references. */ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true) /** - * Provides a logical query plan analyzer, which translates - * [[catalyst.analysis.UnresolvedAttribute UnresolvedAttribute]]s and - * [[catalyst.analysis.UnresolvedRelation UnresolvedRelation]]s into fully typed objects using - * information in a schema [[catalyst.analysis.Catalog Catalog]] and a - * [[catalyst.analysis.FunctionRegistry]]. + * Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and + * [[UnresolvedRelation]]s into fully typed objects using information in a schema [[Catalog]] and + * a [[FunctionRegistry]]. */ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean) extends RuleExecutor[LogicalPlan] with HiveTypeCoercion { @@ -44,8 +40,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) /** - * Replaces [[catalyst.analysis.UnresolvedRelation UnresolvedRelation]]s with concrete relations - * from the catalog. + * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -69,9 +64,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Replaces [[catalyst.analysis.UnresolvedAttribute UnresolvedAttribute]]s with concrete - * [[catalyst.expressions.AttributeReference AttributeReferences]] from a logical plan node's - * children. + * Replaces [[UnresolvedAttribute]]s with concrete + * [[expressions.AttributeReference AttributeReferences]] from a logical plan node's children. */ object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { @@ -88,8 +82,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Replaces [[catalyst.analysis.UnresolvedFunction UnresolvedFunction]]s with concrete - * [[catalyst.expressions.Expression Expressions]]. + * Replaces [[UnresolvedFunction]]s with concrete [[expressions.Expression Expressions]]. */ object ResolveFunctions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -120,7 +113,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Expands any references to [[catalyst.analysis.Star Star]] (*) in project operators. + * Expands any references to [[Star]] (*) in project operators. */ object StarExpansion extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -145,7 +138,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Returns true if `exprs` contains a [[catalyst.analysis.Star Star]]. + * Returns true if `exprs` contains a [[Star]]. */ protected def containsStar(exprs: Seq[NamedExpression]): Boolean = exprs.collect { case _: Star => true }.nonEmpty From 83d452081100447b45d93e6b2228dfda130a6473 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 11:22:03 +0800 Subject: [PATCH 560/778] marmbrus's comments --- .../scala/catalyst/execution/aggregates.scala | 10 +++- .../catalyst/execution/basicOperators.scala | 11 ++-- src/main/scala/catalyst/execution/joins.scala | 7 ++- .../execution/planningStrategies.scala | 6 +-- .../plans/physical/partitioning.scala | 51 +++++++------------ .../execution/DistributionTests.scala | 41 ++++++++------- 6 files changed, 56 insertions(+), 70 deletions(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index d7fcd3d25d46a..f428b892de341 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,7 +3,7 @@ package execution import catalyst.errors._ import catalyst.expressions._ -import catalyst.plans.physical.Distribution._ +import catalyst.plans.physical.{ClusteredDistribution, AllTuples} import org.apache.spark.rdd.SharkPairRDDFunctions /* Implicits */ @@ -15,7 +15,13 @@ case class Aggregate( child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { - override def requiredChildDistribution = getSpecifiedDistribution(groupingExpressions) :: Nil + override def requiredChildDistribution = + if (groupingExpressions == Nil) { + AllTuples :: Nil + } else { + ClusteredDistribution(groupingExpressions) :: Nil + } + override def otherCopyArgs = sc :: Nil def output = aggregateExpressions.map(_.toAttribute) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 487de52dc1cc4..b44c1964fb261 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -3,9 +3,7 @@ package execution import catalyst.errors._ import catalyst.expressions._ -import catalyst.plans.physical.Distribution._ -import catalyst.expressions.SortOrder -import catalyst.plans.physical.{UnspecifiedDistribution, Distribution} +import catalyst.plans.physical.{UnspecifiedDistribution, OrderedDistribution} case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -51,12 +49,13 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class SortPartitions( +case class Sort( sortOrder: Seq[SortOrder], + global: Boolean, child: SharkPlan) - (override val requiredChildDistribution: Seq[Distribution] = UnspecifiedDistribution :: Nil) extends UnaryNode { - override def otherCopyArgs = requiredChildDistribution :: Nil + override def requiredChildDistribution = + if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil @transient lazy val ordering = new RowOrdering(sortOrder) diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 267b020dfc986..fc9e38df7f995 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -7,11 +7,10 @@ import scala.Some import errors._ import expressions._ import plans._ -import catalyst.plans.physical.Distribution._ import org.apache.spark.rdd.SharkPairRDDFunctions._ import org.apache.spark.rdd.RDD -import catalyst.plans.physical.Partitioning +import catalyst.plans.physical.{ClusteredDistribution, Partitioning} case class SparkEquiInnerJoin( leftKeys: Seq[Expression], @@ -22,7 +21,7 @@ case class SparkEquiInnerJoin( override def outputPartitioning: Partitioning = left.outputPartitioning override def requiredChildDistribution = - getSpecifiedDistribution(leftKeys) :: getSpecifiedDistribution(rightKeys) :: Nil + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil def output = left.output ++ right.output @@ -69,7 +68,7 @@ case class BroadcastNestedLoopJoin( streamed: SharkPlan, broadcast: SharkPlan, joinType: JoinType, condition: Option[Expression]) (@transient sc: SharkContext) extends BinaryNode { - // TODO: Override requiredChildDistribution, outputPartitioning, and use Exchange to broadcast + // TODO: Override requiredChildDistribution and outputPartitioning. override def otherCopyArgs = sc :: Nil diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 558ccad8bcdc7..0001c6c281ed9 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -114,12 +114,10 @@ trait PlanningStrategies { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => // Set the requiredDistribution of this SortPartitions to OrderedDistribution. - execution.SortPartitions( - sortExprs, - planLater(child))(OrderedDistribution(sortExprs) :: Nil) :: Nil + execution.Sort(sortExprs, true, planLater(child)):: Nil case logical.SortPartitions(sortExprs, child) => // Set the requiredDistribution of this SortPartitions to UnspecifiedDistribution. - execution.SortPartitions(sortExprs, planLater(child))() :: Nil + execution.Sort(sortExprs, false, planLater(child)) :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index ee382ede8a79e..dd91b5beb4f95 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -28,7 +28,6 @@ case object UnspecifiedDistribution extends Distribution */ case object AllTuples extends Distribution -// TODO: Add a BroadcastDistribution? /** * Represents data where tuples that share the same values for the `clustering` * [[catalyst.expressions.Expression Expressions]] will be co-located. Based on the context, this @@ -36,11 +35,11 @@ case object AllTuples extends Distribution * within a single partition. */ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution { - if (clustering == Nil) { - throw new IllegalArgumentException("The clustering expressions of a ClusteredDistribution " + - "should not be Nil. An AllTuples should be used to represent a distribution that only has " + + require( + clustering != Nil, + "The clustering expressions of a ClusteredDistribution should not be Nil. " + + "An AllTuples should be used to represent a distribution that only has " + "a single partition.") - } } /** @@ -50,29 +49,15 @@ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distributi * the ordering expressions are contiguous and will never be split across partitions. */ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { - if (ordering == Nil) { - throw new IllegalArgumentException("The ordering expressions of a OrderedDistribution " + - "should not be Nil. An AllTuples should be used to represent a distribution that only has " + + require( + ordering != Nil, + "The ordering expressions of a OrderedDistribution should not be Nil. " + + "An AllTuples should be used to represent a distribution that only has " + "a single partition.") - } def clustering = ordering.map(_.child).toSet } -object Distribution { - def getSpecifiedDistribution(expressions: Seq[Expression]): Distribution = { - if (expressions == Nil) { - AllTuples - } else { - if (expressions.forall(exp => exp.isInstanceOf[SortOrder])) { - OrderedDistribution(expressions.asInstanceOf[Seq[SortOrder]]) - } else { - ClusteredDistribution(expressions) - } - } - } -} - sealed trait Partitioning { /** Returns the number of partitions that the data is split across */ val numPartitions: Int @@ -97,12 +82,12 @@ sealed trait Partitioning { } case class UnknownPartitioning(numPartitions: Int) extends Partitioning { - def satisfies(required: Distribution): Boolean = required match { + override def satisfies(required: Distribution): Boolean = required match { case UnspecifiedDistribution => true case _ => false } - def compatibleWith(other: Partitioning): Boolean = other match { + override def compatibleWith(other: Partitioning): Boolean = other match { case UnknownPartitioning(_) => true case _ => false } @@ -111,9 +96,9 @@ case class UnknownPartitioning(numPartitions: Int) extends Partitioning { case object SinglePartition extends Partitioning { val numPartitions = 1 - def satisfies(required: Distribution): Boolean = true + override def satisfies(required: Distribution): Boolean = true - def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning) = other match { case SinglePartition => true case _ => false } @@ -122,9 +107,9 @@ case object SinglePartition extends Partitioning { case object BroadcastPartitioning extends Partitioning { val numPartitions = 1 - def satisfies(required: Distribution): Boolean = true + override def satisfies(required: Distribution): Boolean = true - def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning) = other match { case SinglePartition => true case _ => false } @@ -146,14 +131,14 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) lazy val clusteringSet = expressions.toSet - def satisfies(required: Distribution): Boolean = required match { + override def satisfies(required: Distribution): Boolean = required match { case UnspecifiedDistribution => true case ClusteredDistribution(requiredClustering) => clusteringSet.subsetOf(requiredClustering.toSet) case _ => false } - def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning) = other match { case BroadcastPartitioning => true case h: HashPartitioning if h == this => true case _ => false @@ -180,7 +165,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) lazy val clusteringSet = ordering.map(_.child).toSet - def satisfies(required: Distribution): Boolean = required match { + override def satisfies(required: Distribution): Boolean = required match { case UnspecifiedDistribution => true case OrderedDistribution(requiredOrdering) => val minSize = Seq(requiredOrdering.size, ordering.size).min @@ -190,7 +175,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) case _ => false } - def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning) = other match { case BroadcastPartitioning => true case r: RangePartitioning if r == this => true case _ => false diff --git a/src/test/scala/catalyst/execution/DistributionTests.scala b/src/test/scala/catalyst/execution/DistributionTests.scala index 0485ff6ab7bb1..2a3d85b5db40b 100644 --- a/src/test/scala/catalyst/execution/DistributionTests.scala +++ b/src/test/scala/catalyst/execution/DistributionTests.scala @@ -5,7 +5,6 @@ package physical import org.scalatest.FunSuite import catalyst.dsl._ -import catalyst.plans.physical.Distribution._ class DistributionTests extends FunSuite { @@ -34,48 +33,48 @@ class DistributionTests extends FunSuite { checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - getSpecifiedDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( HashPartitioning(Seq('b, 'c), 10), - getSpecifiedDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( SinglePartition, - getSpecifiedDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( SinglePartition, - getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) // Cases which need an exchange between two data properties. checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - getSpecifiedDistribution(Seq('b, 'c)), + ClusteredDistribution(Seq('b, 'c)), false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - getSpecifiedDistribution(Seq('d, 'e)), + ClusteredDistribution(Seq('d, 'e)), false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - getSpecifiedDistribution(Nil), + AllTuples, false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) checkSatisfied( HashPartitioning(Seq('b, 'c), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) // TODO: We should check functional dependencies @@ -96,32 +95,32 @@ class DistributionTests extends FunSuite { checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('c, 'b, 'a)), + ClusteredDistribution(Seq('c, 'b, 'a)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('b, 'c, 'a, 'd)), + ClusteredDistribution(Seq('b, 'c, 'a, 'd)), true) // Cases which need an exchange between two data properties. @@ -131,27 +130,27 @@ class DistributionTests extends FunSuite { // and the parallelism. checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('b.asc, 'a.asc)), + OrderedDistribution(Seq('b.asc, 'a.asc)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a, 'b)), + ClusteredDistribution(Seq('a, 'b)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('c, 'd)), + ClusteredDistribution(Seq('c, 'd)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Nil), + AllTuples, false) } } \ No newline at end of file From c98ede56ec504632e0071d6912cdbd484a1dab61 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Tue, 28 Jan 2014 19:25:51 +0800 Subject: [PATCH 561/778] Response to comments from @marmbrus --- .../scala/catalyst/analysis/Analyzer.scala | 13 ++++--- .../catalyst/execution/TableReader.scala | 3 +- .../scala/catalyst/execution/TestShark.scala | 12 +++--- .../catalyst/execution/hiveOperators.scala | 4 +- .../execution/planningStrategies.scala | 38 +++++++------------ .../catalyst/expressions/BoundAttribute.scala | 2 +- .../plans/logical/basicOperators.scala | 4 ++ 7 files changed, 35 insertions(+), 41 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 25f29ccaa5ed5..0ba054d7b3041 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -1,7 +1,6 @@ package catalyst package analysis -import execution.MetastoreRelation import expressions._ import plans.logical._ import rules._ @@ -153,18 +152,20 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool // Wait until children are resolved case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table: MetastoreRelation, partition, child) => + case p @ InsertIntoTable(table, _, child) => val childOutputDataTypes = child.output.map(_.dataType) val tableOutputDataTypes = table.output.map(_.dataType) - // Only do the casting when child output data types differ from table output data types. if (childOutputDataTypes sameElements tableOutputDataTypes) { p } else { - val castedChildOutput = child.output.zip(tableOutputDataTypes).map { - case (a, dataType) => Alias(Cast(a, dataType), a.name)() + // Only do the casting when child output data types differ from table output data types. + val castedChildOutput = child.output.zip(table.output).map { + case (l, r) if l.dataType != r.dataType => Alias(Cast(l, r.dataType), l.name)() + case (l, _) => l } - p.copy(table, partition, Project(castedChildOutput, child)) + + p.copy(child = Project(castedChildOutput, child)) } } } diff --git a/src/main/scala/catalyst/execution/TableReader.scala b/src/main/scala/catalyst/execution/TableReader.scala index 50da479d1a6dc..040d57f95fe9e 100644 --- a/src/main/scala/catalyst/execution/TableReader.scala +++ b/src/main/scala/catalyst/execution/TableReader.scala @@ -95,7 +95,8 @@ private[catalyst] class HadoopTableReader(@transient _tableDesc: TableDesc, @tra // Deserialize each Writable to get the row value. iter.map { case v: Writable => deserializer.deserialize(v) - case value => throw new RuntimeException("Failed to match " + value.toString) + case value => + sys.error(s"Unable to deserialize non-Writable: $value of ${value.getClass.getName}") } } deserializedHadoopRDD diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 642a6b436a7d3..7744c8f0fbc29 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -23,9 +23,8 @@ import util._ /** * A locally running test instance of spark. The lifecycle for a given query is managed by the - * inner class [[catalyst.execution.TestShark.SharkQuery SharkQuery]]. A - * [[catalyst.execution.TestShark.SharkQuery SharkQuery]] can either be instantiated directly or - * using the implicit conversion '.q'. + * inner class [[SharkQuery]]. A [[SharkQuery]] can either be instantiated directly or using the + * implicit conversion '.q'. * * {{{ * scala> val query = "SELECT key FROM src".q @@ -42,10 +41,9 @@ import util._ * res0: Array[IndexedSeq[Any]] = Array(Vector(238), Vector(86), Vector(311), ... * }}} * - * Data from [[catalyst.execution.TestShark.testTables testTables]] will be automatically loaded - * whenever a query is run over those tables. - * Calling [[catalyst.execution.TestShark.reset reset]] will delete all tables and other state in - * the database, leaving the database in a "clean" state. + * Data from [[testTables]] will be automatically loaded whenever a query is run over those tables. + * Calling [[reset]] will delete all tables and other state in the database, leaving the database + * in a "clean" state. * * TestShark is implemented as a singleton object because instantiating multiple copies of the hive * metastore seems to lead to weird non-deterministic failures. Therefore, the execution of diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 761e525da4e8d..754b015c9bc0d 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -36,7 +36,7 @@ case class HiveTableScan( partitionPruningPred: Option[Expression]) extends LeafNode { - // Bind all partition key attribute references to the partition pruning predicate for later + // Bind all partition key attribute references in the partition pruning predicate for later // evaluation. private val boundPruningPred = partitionPruningPred.map { pred => require( @@ -193,7 +193,7 @@ case class InsertIntoHiveTable( .asInstanceOf[StructObjectInspector] iter.map { row => - // TODO Should add a new VarcharType data type to handle HiveQL VARCHAR + // Casts Strings to HiveVarchars when necessary. val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector) val mappedRow = row.zip(fieldOIs).map { case (s: String, oi: JavaHiveVarcharObjectInspector) => new HiveVarchar(s, s.size) diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 5c82c47142e1f..6d0981f423a49 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -34,10 +34,7 @@ trait PlanningStrategies { * complex expressions. */ def isSimpleProject(projectList: Seq[NamedExpression]) = { - projectList.map { - case a: Attribute => true - case _ => false - }.reduceLeft(_ && _) + projectList.forall(_.isInstanceOf[Attribute]) } } @@ -46,33 +43,26 @@ trait PlanningStrategies { * partition pruning. * * This strategy itself doesn't perform partition pruning, it just collects and combines all the - * partition pruning predicates and pass them down to the underlying - * [[catalyst.execution.HiveTableScan HiveTableScan]] operator, which does the actual pruning - * work. + * partition pruning predicates and pass them down to the underlying [[HiveTableScan]] operator, + * which does the actual pruning work. */ object PartitionPrunings extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case p @ FilteredOperation(predicates, relation: MetastoreRelation) => - if (predicates.isEmpty) { - execution.HiveTableScan(relation.output, relation, None) :: Nil - } else { - val partitionKeys = relation.partitionKeys.map(_.name).toSet + val partitionKeyIds = relation.partitionKeys.map(_.id).toSet - // Filter out all predicates that only deal with partition keys - val pruningPreds = predicates.filter {e => - val referenceNames = e.references.map(_.name) - referenceNames.subsetOf(partitionKeys) - } + // Filter out all predicates that only deal with partition keys + val (pruningPredicates, otherPredicates) = predicates.partition { + _.references.map(_.id).subsetOf(partitionKeyIds) + } - val tableScan = if (pruningPreds.isEmpty) { - execution.HiveTableScan(relation.output, relation, None) - } else { - val combinedPruningPred = pruningPreds.reduceLeft(And) - execution.HiveTableScan(relation.output, relation, Some(combinedPruningPred)) - } + val scan = execution.HiveTableScan( + relation.output, relation, pruningPredicates.reduceLeftOption(And)) - execution.Filter(predicates.reduceLeft(And), tableScan) :: Nil - } + otherPredicates + .reduceLeftOption(And) + .map(execution.Filter(_, scan)) + .getOrElse(scan) :: Nil case _ => Nil diff --git a/src/main/scala/catalyst/expressions/BoundAttribute.scala b/src/main/scala/catalyst/expressions/BoundAttribute.scala index 4e0158613b063..398d34fcc4449 100644 --- a/src/main/scala/catalyst/expressions/BoundAttribute.scala +++ b/src/main/scala/catalyst/expressions/BoundAttribute.scala @@ -44,7 +44,7 @@ object BindReferences extends Logging { def bindReference(expression: Expression, input: Seq[Seq[Attribute]]): Expression = { expression.transform { case a: AttributeReference => attachTree(a, "Binding attribute") { - val inputAsString = input.map(_.mkString("{", ",", "}")).mkString(",") + def inputAsString = input.map(_.mkString("{", ",", "}")).mkString(",") for { (tuple, inputTuple) <- input.zipWithIndex diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index 81ae69c2bd14a..9756672c85654 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -41,6 +41,10 @@ case class InsertIntoTable(table: BaseRelation, partition: Map[String, Option[St def children = table :: child :: Nil def references = Set.empty def output = child.output + + override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { + case (childAttr, tableAttr) => childAttr.dataType == tableAttr.dataType + } } case class InsertIntoCreatedTable(tableName: String, child: LogicalPlan) extends UnaryNode { From ab77882362eb8d0ee784ca2b23b0ad4485156199 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 28 Jan 2014 13:59:59 -0800 Subject: [PATCH 562/778] upgrade spark to RC5. --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index a1a52867305e7..39e0d9805b088 100644 --- a/build.sbt +++ b/build.sbt @@ -13,7 +13,7 @@ scalacOptions ++= Seq("-deprecation", "-feature", "-unchecked") resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" // TODO: Remove when Spark 0.9.0 is released for real. -resolvers += "SparkStaging" at "https://repository.apache.org/content/repositories/orgapachespark-1005/" +resolvers += "SparkStaging" at "https://repository.apache.org/content/repositories/orgapachespark-1006/" libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" From c263c8460ffb6b1c630ad0185bf0628228ecef55 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 28 Jan 2014 15:09:41 -0800 Subject: [PATCH 563/778] Only push predicates into partitioned table scans. --- src/main/scala/catalyst/execution/hiveOperators.scala | 3 +++ src/main/scala/catalyst/execution/planningStrategies.scala | 4 +++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 754b015c9bc0d..68fe99b5dc9cd 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -36,6 +36,9 @@ case class HiveTableScan( partitionPruningPred: Option[Expression]) extends LeafNode { + require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, + "Partition pruning predicates only supported for partitioned tables.") + // Bind all partition key attribute references in the partition pruning predicate for later // evaluation. private val boundPruningPred = partitionPruningPred.map { pred => diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 6d0981f423a49..2a26269daea1c 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -48,7 +48,9 @@ trait PlanningStrategies { */ object PartitionPrunings extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { - case p @ FilteredOperation(predicates, relation: MetastoreRelation) => + case p @ FilteredOperation(predicates, relation: MetastoreRelation) + if relation.hiveQlTable.isPartitioned => + val partitionKeyIds = relation.partitionKeys.map(_.id).toSet // Filter out all predicates that only deal with partition keys From 677eb073f635815a2aa22a49ed466b84c785d6ed Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 28 Jan 2014 16:14:18 -0800 Subject: [PATCH 564/778] Update test whitelist. --- build.sbt | 2 +- .../catalyst/execution/hiveOperators.scala | 1 - .../execution/HiveCompatibility.scala | 161 +++++++++++++----- 3 files changed, 122 insertions(+), 42 deletions(-) diff --git a/build.sbt b/build.sbt index 39e0d9805b088..9b5682847d946 100644 --- a/build.sbt +++ b/build.sbt @@ -17,7 +17,7 @@ resolvers += "SparkStaging" at "https://repository.apache.org/content/repositori libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" -libraryDependencies += "catalyst" % "hive-golden" % "4" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden4.jar" +libraryDependencies += "catalyst" % "hive-golden" % "5" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden5.jar" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 68fe99b5dc9cd..ef1f88668b675 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -179,7 +179,6 @@ case class InsertIntoHiveTable( * `org.apache.hadoop.mapred.OutputFormat` provided by the table definition. */ def execute() = { - require(partition.isEmpty, "Inserting into partitioned table not supported.") val childRdd = child.execute() assert(childRdd != null) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index 26abe2cc135e9..bbd492851aa02 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -29,6 +29,10 @@ class HiveCompatibility extends HiveQueryFileTest { "column_access_stats", "concatenate_inherit_table_location", + // Setting a default property does not seem to get reset and thus changes the answer for many + // subsequent tests. + "create_default_prop", + // User specific test answers, breaks the caching mechanism. "authorization_3", "authorization_5", @@ -129,20 +133,56 @@ class HiveCompatibility extends HiveQueryFileTest { "add_partition_no_whitelist", "add_partition_with_whitelist", "alias_casted_column", + "alter2", "alter4", + "alter5", + "alter_concatenate_indexed_table", "alter_index", + "alter_merge", + "alter_merge_2", + "alter_merge_stats", + "alter_partition_clusterby_sortby", "alter_partition_format_loc", + "alter_partition_protect_mode", "alter_partition_with_whitelist", + "alter_skewed_table", + "alter_table_not_sorted", "alter_table_serde", + "alter_table_serde2", + "alter_varchar2", "ambiguous_col", - "authorization_3", - "authorization_5", + "auto_join0", + "auto_join1", + "auto_join10", + "auto_join11", + "auto_join12", + "auto_join13", + "auto_join14", + "auto_join14_hadoop20", + "auto_join15", + "auto_join17", + "auto_join18", + "auto_join19", + "auto_join2", + "auto_join20", "auto_join21", + "auto_join22", "auto_join23", "auto_join24", + "auto_join25", "auto_join26", "auto_join28", + "auto_join3", + "auto_join30", + "auto_join31", "auto_join32", + "auto_join4", + "auto_join5", + "auto_join6", + "auto_join7", + "auto_join8", + "auto_join9", + "auto_join_filters", "auto_join_nulls", "auto_join_reordering_values", "auto_sortmerge_join_1", @@ -160,36 +200,25 @@ class HiveCompatibility extends HiveQueryFileTest { "auto_sortmerge_join_9", "binary_constant", "binarysortable_1", - "bucket1", - "bucket_map_join_1", - "bucket_map_join_2", - "bucketcontext_5", - "bucketmapjoin6", - "bucketmapjoin_negative3", "combine1", "convert_enum_to_string", + "correlationoptimizer11", "correlationoptimizer15", + "correlationoptimizer2", + "correlationoptimizer3", "correlationoptimizer4", "correlationoptimizer6", "correlationoptimizer7", + "correlationoptimizer8", "count", - "create_like2", - "create_like_tbl_props", - "create_view_translate", - "create_skewed_table1", "ct_case_insensitive", - "database_location", "database_properties", "default_partition_name", "delimiter", "desc_non_existent_tbl", - "describe_comment_indent", "describe_database_json", - "describe_table_json", - "describe_formatted_view_partitioned", - "describe_formatted_view_partitioned_json", "describe_pretty", - "describe_syntax", + "describe_table_json", "diff_part_input_formats", "disable_file_format_check", "drop_function", @@ -200,12 +229,14 @@ class HiveCompatibility extends HiveQueryFileTest { "drop_partitions_ignore_protection", "drop_table", "drop_table2", - "drop_view", "escape_clusterby1", "escape_distributeby1", "escape_orderby1", "escape_sortby1", + "filter_join_breaktask", + "filter_join_breaktask2", "groupby1", + "groupby11", "groupby1_map", "groupby1_map_nomap", "groupby1_map_skew", @@ -227,33 +258,28 @@ class HiveCompatibility extends HiveQueryFileTest { "groupby_multi_single_reducer2", "groupby_mutli_insert_common_distinct", "groupby_sort_6", + "groupby_sort_8", "groupby_sort_test_1", "implicit_cast1", - "index_auto_self_join", - "index_auto_update", - "index_stale", - "index_auth", - "index_auto_file_format", - "index_auto_mult_tables", - "index_auto_mult_tables_compact", - "index_auto_multiple", - "index_bitmap_compression", - "index_compression", - "index_auto_empty", - "index_auto_partitioned", - "index_bitmap_auto_partitioned", + "index_auto_unused", "innerjoin", "inoutdriver", "input", "input0", "input11", "input11_limit", + "input12", + "input12_hadoop20", + "input19", "input1_limit", "input22", "input23", "input24", "input25", + "input26", + "input28", "input2_limit", + "input40", "input41", "input4_cb_delim", "input4_limit", @@ -262,10 +288,18 @@ class HiveCompatibility extends HiveQueryFileTest { "input8", "input9", "input_limit", + "input_part0", "input_part1", + "input_part10", + "input_part10_win", "input_part2", + "input_part3", "input_part4", + "input_part5", "input_part6", + "input_part7", + "input_part8", + "input_part9", "inputddl4", "inputddl7", "inputddl8", @@ -313,39 +347,46 @@ class HiveCompatibility extends HiveQueryFileTest { "join7", "join8", "join9", + "join_1to1", "join_casesensitive", "join_empty", + "join_filters", "join_hive_626", "join_nulls", "join_reorder2", "join_reorder3", "join_reorder4", "join_star", - "join_view", - "keyword_1", "lineage1", "literal_double", "literal_ints", "literal_string", + "load_dyn_part7", "load_file_with_space_in_the_name", + "load_overwrite", "louter_join_ppr", "mapjoin_mapjoin", "mapjoin_subquery", "mapjoin_subquery2", "mapjoin_test_outer", "mapreduce3", + "merge1", + "merge2", "mergejoins", "mergejoins_mixed", - "misc_json", + "multiMapJoin1", "multi_join_union", "multigroupby_singlemr", "noalias_subq1", "nomore_ambiguous_table_col", + "nonblock_op_deduplicate", "notable_alias1", "notable_alias2", "nullgroup", "nullgroup2", "nullgroup3", + "nullgroup4", + "nullgroup4_multi_distinct", "nullgroup5", "nullinput", "nullinput2", @@ -356,8 +397,7 @@ class HiveCompatibility extends HiveQueryFileTest { "outer_join_ppr", "part_inherit_tbl_props", "part_inherit_tbl_props_empty", - "part_inherit_tbl_props_with_star", - "partitions_json", + "partition_schema1", "plan_json", "ppd1", "ppd_constant_where", @@ -375,14 +415,23 @@ class HiveCompatibility extends HiveQueryFileTest { "ppd_repeated_alias", "ppd_udf_col", "ppd_union", + "ppr_pushdown", + "ppr_pushdown2", + "ppr_pushdown3", "progress_1", "protectmode", + "push_or", + "ql_rewrite_gbtoidx", "query_with_semi", + "quote1", "quote2", - "rename_column", "reduce_deduplicate_exclude_join", + "rename_column", "router_join_ppr", "select_as_omitted", + "select_unquote_and", + "select_unquote_not", + "select_unquote_or", "serde_reported_schema", "set_variable_sub", "show_describe_func_quotes", @@ -391,22 +440,46 @@ class HiveCompatibility extends HiveQueryFileTest { "skewjoinopt13", "skewjoinopt18", "skewjoinopt9", + "smb_mapjoin_1", "smb_mapjoin_10", "smb_mapjoin_13", "smb_mapjoin_14", "smb_mapjoin_15", "smb_mapjoin_16", "smb_mapjoin_17", + "smb_mapjoin_2", + "smb_mapjoin_21", + "smb_mapjoin_25", + "smb_mapjoin_3", + "smb_mapjoin_4", + "smb_mapjoin_5", + "smb_mapjoin_8", "sort", "sort_merge_join_desc_1", "sort_merge_join_desc_2", "sort_merge_join_desc_3", "sort_merge_join_desc_4", + "sort_merge_join_desc_5", + "sort_merge_join_desc_6", + "sort_merge_join_desc_7", + "stats0", + "stats1", + "stats14", + "stats15", + "stats16", + "stats18", + "stats20", + "stats_empty_partition", "subq2", "tablename_with_select", + "touch", + "type_widening", "udf2", + "udf6", "udf9", "udf_10_trims", + "udf_E", + "udf_PI", "udf_abs", "udf_acos", "udf_add", @@ -433,6 +506,7 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_datediff", "udf_day", "udf_dayofmonth", + "udf_degrees", "udf_div", "udf_double", "udf_exp", @@ -454,6 +528,7 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_length", "udf_lessthan", "udf_lessthanorequal", + "udf_like", "udf_ln", "udf_log", "udf_log10", @@ -467,6 +542,7 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_negative", "udf_not", "udf_notequal", + "udf_notop", "udf_nvl", "udf_or", "udf_parse_url", @@ -475,10 +551,12 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_power", "udf_radians", "udf_rand", + "udf_regexp", "udf_regexp_extract", "udf_regexp_replace", "udf_repeat", "udf_rlike", + "udf_round_3", "udf_rpad", "udf_rtrim", "udf_second", @@ -498,10 +576,9 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_tan", "udf_tinyint", "udf_to_date", - "udf_to_unix_timestamp", + "udf_translate", "udf_trim", "udf_ucase", - "udf_unix_timestamp", "udf_upper", "udf_var_pop", "udf_var_samp", @@ -528,7 +605,9 @@ class HiveCompatibility extends HiveQueryFileTest { "union19", "union2", "union20", + "union22", "union23", + "union24", "union27", "union28", "union29", @@ -541,6 +620,8 @@ class HiveCompatibility extends HiveQueryFileTest { "union7", "union8", "union9", + "union_ppr", + "union_remove_6", "union_script", "varchar_2", "varchar_join1", From d4f539a9a7c0210b609e68a0fa49b1d2922b1205 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 28 Jan 2014 20:15:38 -0800 Subject: [PATCH 565/778] blacklist mr and user specific tests. --- .../scala/catalyst/execution/HiveCompatibility.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index bbd492851aa02..f70904551a01c 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -39,6 +39,13 @@ class HiveCompatibility extends HiveQueryFileTest { "keyword_1", "misc_json", "create_like_tbl_props", + "load_overwrite", + "alter_table_serde2", + "alter_table_not_sorted", + "alter_skewed_table", + "alter_partition_clusterby_sortby", + "alter_merge", + "alter_concatenate_indexed_table", // Weird DDL differences result in failures on jenkins. "create_like2", @@ -63,7 +70,11 @@ class HiveCompatibility extends HiveQueryFileTest { "index_auth", "index_auto_empty", "index_auto_partitioned", + "index_auto_unused", "index_bitmap_auto_partitioned", + "ql_rewrite_gbtoidx", + "stats1.*", + "alter_merge_stats", // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 From ebb56faaec54c970fa49e8c575facfd6658e37ea Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 28 Jan 2014 22:27:35 -0800 Subject: [PATCH 566/778] add travis config --- .travis.yml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 0000000000000..e0abe3e54726d --- /dev/null +++ b/.travis.yml @@ -0,0 +1,5 @@ + language: scala + scala: + - "2.10.3" + jdk: + - oraclejdk7 \ No newline at end of file From 8ee41be08034e1a66ec13a0ed66a1b59a3ad0aaa Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Thu, 30 Jan 2014 22:38:45 +0800 Subject: [PATCH 567/778] Minor refactoring --- .../execution/planningStrategies.scala | 27 +++++++------------ .../catalyst/execution/HiveQueryTests.scala | 2 -- 2 files changed, 10 insertions(+), 19 deletions(-) diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 2a26269daea1c..b21d391bcb312 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -59,7 +59,7 @@ trait PlanningStrategies { } val scan = execution.HiveTableScan( - relation.output, relation, pruningPredicates.reduceLeftOption(And)) + relation.output, relation, pruningPredicates.reduceOption(And)) otherPredicates .reduceLeftOption(And) @@ -79,30 +79,26 @@ trait PlanningStrategies { // as join keys. Note we can only mix in the conditions with other predicates because the // match above ensures that this is and Inner join. val (joinPredicates, otherPredicates) = (predicates ++ condition).partition { - case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || - (canEvaluate(l, right) && canEvaluate(r, left)) => true + case Equals(l, r) => (canEvaluate(l, left) && canEvaluate(r, right)) || + (canEvaluate(l, right) && canEvaluate(r, left)) case _ => false } val joinKeys = joinPredicates.map { - case Equals(l,r) if (canEvaluate(l, left) && canEvaluate(r, right)) => (l, r) - case Equals(l,r) if (canEvaluate(l, right) && canEvaluate(r, left)) => (r, l) + case Equals(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r) + case Equals(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l) } // Do not consider this strategy if there are no join keys. if (joinKeys.nonEmpty) { - val leftKeys = joinKeys.map(_._1) - val rightKeys = joinKeys.map(_._2) - + val (leftKeys, rightKeys) = joinKeys.unzip val joinOp = execution.SparkEquiInnerJoin( leftKeys, rightKeys, planLater(left), planLater(right)) - // Make sure other conditions are met if present. - if (otherPredicates.nonEmpty) { - execution.Filter(combineConjunctivePredicates(otherPredicates), joinOp) :: Nil - } else { - joinOp :: Nil - } + otherPredicates + .reduceOption(And) + .map(execution.Filter(_, joinOp)) + .getOrElse(joinOp) :: Nil } else { logger.debug(s"Avoiding spark join with no join keys.") Nil @@ -110,9 +106,6 @@ trait PlanningStrategies { case _ => Nil } - private def combineConjunctivePredicates(predicates: Seq[Expression]) = - predicates.reduceLeft(And(_, _)) - /** Returns true if `expr` can be evaluated using only the output of `plan`. */ protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = expr.references subsetOf plan.outputSet diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala index 920b875c149d4..f20e93591f613 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -5,8 +5,6 @@ package execution * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ class HiveQueryTests extends HiveComparisonTest { - import TestShark._ - createQueryTest("Simple Average", "SELECT AVG(key) FROM src") From 2486fb71dc89f915c4f54a95e42211c79fc99e4c Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Thu, 30 Jan 2014 22:39:00 +0800 Subject: [PATCH 568/778] Fixed spelling --- src/test/scala/catalyst/execution/HiveQueryTests.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala index f20e93591f613..cf94e5b53c916 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -23,10 +23,10 @@ class HiveQueryTests extends HiveComparisonTest { createQueryTest("IgnoreExplain", """EXPLAIN SELECT key FROM src""") - createQueryTest("trival join where clause", + createQueryTest("trivial join where clause", "SELECT * FROM src a JOIN src b WHERE a.key = b.key") - createQueryTest("trival join ON clause", + createQueryTest("trivial join ON clause", "SELECT * FROM src a JOIN src b ON a.key = b.key") createQueryTest("small.cartesian", @@ -35,11 +35,11 @@ class HiveQueryTests extends HiveComparisonTest { createQueryTest("length.udf", "SELECT length(\"test\") FROM src LIMIT 1") - ignore("partitioned table scan") { createQueryTest("partitioned table scan", "SELECT ds, hr, key, value FROM srcpart") } + createQueryTest("hash", "SELECT hash('test') FROM src LIMIT 1") From 61e729cc21afcafe64af1befee2efb54271bf6d8 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Thu, 30 Jan 2014 22:39:37 +0800 Subject: [PATCH 569/778] Added ColumnPrunings strategy and test cases --- .../catalyst/execution/SharkInstance.scala | 1 + .../execution/planningStrategies.scala | 54 +++++++++++-- .../execution/PartitionPruningSuite.scala | 51 ------------- .../catalyst/execution/PruningSuite.scala | 75 +++++++++++++++++++ 4 files changed, 122 insertions(+), 59 deletions(-) delete mode 100644 src/test/scala/catalyst/execution/PartitionPruningSuite.scala create mode 100644 src/test/scala/catalyst/execution/PruningSuite.scala diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 77599ba8e059f..9841aa9b3b0ad 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -73,6 +73,7 @@ abstract class SharkInstance extends Logging { val sc = self.sc val strategies = SparkEquiInnerJoin :: + ColumnPrunings :: PartitionPrunings :: HiveTableScans :: DataSinks :: diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index b21d391bcb312..908ff3c1f719e 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -28,14 +28,6 @@ trait PlanningStrategies { execution.HiveTableScan(m.output, m, None) :: Nil case _ => Nil } - - /** - * Returns true if `projectList` only performs column pruning and does not evaluate other - * complex expressions. - */ - def isSimpleProject(projectList: Seq[NamedExpression]) = { - projectList.forall(_.isInstanceOf[Attribute]) - } } /** @@ -71,6 +63,45 @@ trait PlanningStrategies { } } + /** + * A strategy that detects projection over filtered operation and applies column pruning if + * possible. + */ + object ColumnPrunings extends Strategy { + def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { + case logical.Project(projectList, child @ FilteredOperation(predicates, m: MetastoreRelation)) + if isSimpleProject(projectList) => + + val projectAttributes = projectList.asInstanceOf[Seq[Attribute]] + val predicatesReferences = predicates.flatMap(_.references).toSet + val prunedAttributes = projectAttributes ++ (predicatesReferences -- projectAttributes) + + if (m.hiveQlTable.isPartitioned) { + // Applies partition pruning first for partitioned table + PartitionPrunings(child).view.map { sharkPlan => + execution.Project( + projectList, + sharkPlan.transform { + case scan@execution.HiveTableScan(attributes, _, _) => + scan.copy(attributes = prunedAttributes) + }) + } + } else { + val scan = execution.HiveTableScan(prunedAttributes, m, None) + val conjunctionOpt = predicates.reduceOption(And) + + execution.Project( + projectList, + conjunctionOpt + .map(execution.Filter(_, scan)) + .getOrElse(scan)) :: Nil + } + + case _ => + Nil + } + } + object SparkEquiInnerJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case FilteredOperation(predicates, logical.Join(left, right, Inner, condition)) => @@ -159,4 +190,11 @@ trait PlanningStrategies { } } + /** + * Returns true if `projectList` only performs column pruning and does not evaluate other + * complex expressions. + */ + private def isSimpleProject(projectList: Seq[NamedExpression]) = { + projectList.forall(_.isInstanceOf[Attribute]) + } } \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/PartitionPruningSuite.scala b/src/test/scala/catalyst/execution/PartitionPruningSuite.scala deleted file mode 100644 index f13317ddc9028..0000000000000 --- a/src/test/scala/catalyst/execution/PartitionPruningSuite.scala +++ /dev/null @@ -1,51 +0,0 @@ -package catalyst.execution - -import scala.collection.JavaConversions._ - -import TestShark._ - -class PartitionPruningSuite extends HiveComparisonTest { - createPruningTest("Pruning with predicate on STRING partition key", - "SELECT * FROM srcpart1 WHERE ds = '2008-04-08'", - Seq( - Seq("2008-04-08", "11"), - Seq("2008-04-08", "12"))) - - createPruningTest("Pruning with predicate on INT partition key", - "SELECT * FROM srcpart1 WHERE hr < 12", - Seq( - Seq("2008-04-08", "11"), - Seq("2008-04-09", "11"))) - - createPruningTest("Select only 1 partition", - "SELECT * FROM srcpart1 WHERE ds = '2008-04-08' AND hr < 12", - Seq( - Seq("2008-04-08", "11"))) - - createPruningTest("All partitions pruned", - "SELECT * FROM srcpart1 WHERE ds = '2014-01-27' AND hr = 11", - Seq.empty) - - createPruningTest("Pruning with both column key and partition key", - "SELECT * FROM srcpart1 WHERE value IS NOT NULL AND hr < 12", - Seq( - Seq("2008-04-08", "11"), - Seq("2008-04-09", "11"))) - - def createPruningTest(testCaseName: String, sql: String, expectedValues: Seq[Seq[String]]) = { - test(testCaseName) { - val plan = sql.q.executedPlan - val prunedPartitions = plan.collect { - case p @ HiveTableScan(_, relation, _) => - p.prunePartitions(relation.hiveQlPartitions) - }.head - val values = prunedPartitions.map(_.getValues) - - assert(prunedPartitions.size === expectedValues.size) - - for ((actual, expected) <- values.zip(expectedValues)) { - assert(actual sameElements expected) - } - } - } -} diff --git a/src/test/scala/catalyst/execution/PruningSuite.scala b/src/test/scala/catalyst/execution/PruningSuite.scala new file mode 100644 index 0000000000000..25ca7230bfbb3 --- /dev/null +++ b/src/test/scala/catalyst/execution/PruningSuite.scala @@ -0,0 +1,75 @@ +package catalyst.execution + +import scala.collection.JavaConversions._ + +import TestShark._ + +/** + * A set of test cases that validate partition and column pruning. + */ +class PruningSuite extends HiveComparisonTest { + createPruningTest("Pruning non-partitioned table", + "SELECT value from src WHERE key IS NOT NULL", + Seq("value"), + Seq("value", "key"), + Seq.empty) + + createPruningTest("Pruning with predicate on STRING partition key", + "SELECT value, hr FROM srcpart1 WHERE ds = '2008-04-08'", + Seq("value", "hr"), + Seq("value", "hr", "ds"), + Seq( + Seq("2008-04-08", "11"), + Seq("2008-04-08", "12"))) + + createPruningTest("Pruning with predicate on INT partition key", + "SELECT value, hr FROM srcpart1 WHERE hr < 12", + Seq("value", "hr"), + Seq("value", "hr"), + Seq( + Seq("2008-04-08", "11"), + Seq("2008-04-09", "11"))) + + createPruningTest("Select only 1 partition", + "SELECT value, hr FROM srcpart1 WHERE ds = '2008-04-08' AND hr < 12", + Seq("value", "hr"), + Seq("value", "hr", "ds"), + Seq( + Seq("2008-04-08", "11"))) + + createPruningTest("All partitions pruned", + "SELECT value, hr FROM srcpart1 WHERE ds = '2014-01-27' AND hr = 11", + Seq("value", "hr"), + Seq("value", "hr", "ds"), + Seq.empty) + + createPruningTest("Pruning with both column key and partition key", + "SELECT value, hr FROM srcpart1 WHERE value IS NOT NULL AND hr < 12", + Seq("value", "hr"), + Seq("value", "hr"), + Seq( + Seq("2008-04-08", "11"), + Seq("2008-04-09", "11"))) + + def createPruningTest( + testCaseName: String, + sql: String, + expectedOutputColumns: Seq[String], + expectedScannedColumns: Seq[String], + expectedPartValues: Seq[Seq[String]]) = { + test(testCaseName) { + val plan = sql.q.executedPlan + val actualOutputColumns = plan.output.map(_.name) + val (actualScannedColumns, actualPartValues) = plan.collect { + case p @ HiveTableScan(columns, relation, _) => + val columnNames = columns.map(_.name) + val partValues = p.prunePartitions(relation.hiveQlPartitions).map(_.getValues) + (columnNames, partValues) + }.head + + assert(actualOutputColumns sameElements expectedOutputColumns) + assert(actualScannedColumns sameElements expectedScannedColumns) + assert(actualPartValues.corresponds(expectedPartValues)(_ sameElements _)) + } + } +} From 605255eb979416edc19c005f0bc7b8d5f13dd44b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 Jan 2014 14:55:06 -0800 Subject: [PATCH 570/778] Added scalastyle checker. --- build.sbt | 2 + project/plugins.sbt | 8 ++- scalastyle-config.xml | 125 ++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 134 insertions(+), 1 deletion(-) create mode 100644 scalastyle-config.xml diff --git a/build.sbt b/build.sbt index 9b5682847d946..a52936e8f1889 100644 --- a/build.sbt +++ b/build.sbt @@ -31,6 +31,8 @@ libraryDependencies ++= Seq( "org.apache.hive" % "hive-serde" % "0.12.0", "com.typesafe" %% "scalalogging-slf4j" % "1.0.1") +org.scalastyle.sbt.ScalastylePlugin.Settings + // Multiple queries rely on the TestShark singleton. See comments there for more details. parallelExecution in Test := false diff --git a/project/plugins.sbt b/project/plugins.sbt index 4a0b5f82a729d..d2c2af17b0fb5 100755 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -31,4 +31,10 @@ addSbtPlugin("com.typesafe.sbt" % "sbt-ghpages" % "0.5.2") addSbtPlugin("com.typesafe.sbt" % "sbt-site" % "0.7.1") -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2") \ No newline at end of file +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2") + + +addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.3.2") + +// For scalastyle +resolvers += "sonatype-releases" at "https://oss.sonatype.org/content/repositories/releases/" diff --git a/scalastyle-config.xml b/scalastyle-config.xml new file mode 100644 index 0000000000000..a3c1c1f5bcc49 --- /dev/null +++ b/scalastyle-config.xml @@ -0,0 +1,125 @@ + + + + + + + + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From 08e4d0589056f3ae6e117689596420bbf7fbbbc2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 Jan 2014 15:59:55 -0800 Subject: [PATCH 571/778] First round of style cleanup. --- .../catalyst/analysis/HiveTypeCoercion.scala | 3 +- .../scala/catalyst/analysis/package.scala | 2 +- .../catalyst/{dsl.scala => dsl/package.scala} | 44 ++++++++----- src/main/scala/catalyst/errors/package.scala | 4 +- .../catalyst/examples/SchemaRddExample.scala | 2 +- .../catalyst/examples/ViewsExample.scala | 13 ++-- .../catalyst/execution/FunctionRegistry.scala | 44 ++++++------- .../catalyst/execution/MetastoreCatalog.scala | 3 +- ...ategies.scala => PlanningStrategies.scala} | 2 +- .../catalyst/execution/SharkInstance.scala | 2 +- .../catalyst/execution/TableReader.scala | 3 +- .../scala/catalyst/execution/TestShark.scala | 33 +++++----- .../scala/catalyst/execution/Transform.scala | 5 +- .../scala/catalyst/execution/aggregates.scala | 8 ++- src/main/scala/catalyst/execution/debug.scala | 2 +- src/main/scala/catalyst/execution/joins.scala | 7 ++- .../scala/catalyst/execution/package.scala | 2 +- .../catalyst/expressions/BoundAttribute.scala | 6 +- .../scala/catalyst/expressions/Evaluate.scala | 62 +++++++++++-------- .../catalyst/expressions/Expression.scala | 5 +- .../{functions.scala => ImplementedUdf.scala} | 2 +- .../scala/catalyst/expressions/Rand.scala | 2 +- .../scala/catalyst/expressions/ScalaUdf.scala | 8 +-- .../catalyst/expressions/SortOrder.scala | 2 +- .../catalyst/expressions/WrapDynamic.scala | 2 +- .../catalyst/expressions/aggregates.scala | 2 +- .../catalyst/expressions/arithmetic.scala | 10 +-- .../catalyst/expressions/complexTypes.scala | 2 +- .../scala/catalyst/expressions/literals.scala | 2 +- .../expressions/namedExpressions.scala | 26 ++++---- .../scala/catalyst/expressions/package.scala | 38 ++++++------ .../scala/catalyst/optimizer/Optimizer.scala | 2 +- src/main/scala/catalyst/package.scala | 4 +- .../catalyst/planning/QueryPlanner.scala | 7 ++- .../scala/catalyst/planning/package.scala | 2 +- .../scala/catalyst/planning/patterns.scala | 2 +- src/main/scala/catalyst/plans/QueryPlan.scala | 2 +- .../catalyst/plans/logical/LogicalPlan.scala | 2 +- .../catalyst/plans/logical/Transform.scala | 2 +- .../plans/logical/basicOperators.scala | 5 +- src/main/scala/catalyst/plans/shared.scala | 2 +- src/main/scala/catalyst/rules/package.scala | 2 +- src/main/scala/catalyst/trees/package.scala | 16 ++--- src/main/scala/catalyst/types/dataTypes.scala | 2 +- src/main/scala/catalyst/util/package.scala | 16 ++--- 45 files changed, 232 insertions(+), 182 deletions(-) rename src/main/scala/catalyst/{dsl.scala => dsl/package.scala} (77%) rename src/main/scala/catalyst/execution/{planningStrategies.scala => PlanningStrategies.scala} (99%) rename src/main/scala/catalyst/expressions/{functions.scala => ImplementedUdf.scala} (98%) diff --git a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala index 1d7ba1e0a65f3..9544eec10dee4 100644 --- a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala +++ b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala @@ -201,7 +201,8 @@ trait HiveTypeCoercion { // No need to change Equals operators as that actually makes sense for boolean types. case e: Equals => e // Otherwise turn them to Byte types so that there exists and ordering. - case p: BinaryComparison if p.left.dataType == BooleanType && p.right.dataType == BooleanType => + case p: BinaryComparison + if p.left.dataType == BooleanType && p.right.dataType == BooleanType => p.makeCopy(Array(Cast(p.left, ByteType), Cast(p.right, ByteType))) } } diff --git a/src/main/scala/catalyst/analysis/package.scala b/src/main/scala/catalyst/analysis/package.scala index 33794e5d898a7..050763c619702 100644 --- a/src/main/scala/catalyst/analysis/package.scala +++ b/src/main/scala/catalyst/analysis/package.scala @@ -5,4 +5,4 @@ package catalyst * Analysis consists of translating [[UnresolvedAttribute]]s and [[UnresolvedRelation]]s * into fully typed objects using information in a schema [[Catalog]]. */ -package object analysis \ No newline at end of file +package object analysis diff --git a/src/main/scala/catalyst/dsl.scala b/src/main/scala/catalyst/dsl/package.scala similarity index 77% rename from src/main/scala/catalyst/dsl.scala rename to src/main/scala/catalyst/dsl/package.scala index 6efb5cf873c69..5c073008fb73a 100644 --- a/src/main/scala/catalyst/dsl.scala +++ b/src/main/scala/catalyst/dsl/package.scala @@ -9,7 +9,7 @@ import plans.logical._ import types._ /** - * A collection of implicit conversions that create a DSL for easily constructing catalyst data structures. + * A collection of implicit conversions that create a DSL for constructing catalyst data structures. * * {{{ * scala> import catalyst.dsl._ @@ -35,22 +35,22 @@ import types._ * }}} */ package object dsl { - abstract protected trait ImplicitOperators { + protected trait ImplicitOperators { def expr: Expression - def +(other: Expression) = Add(expr, other) - def -(other: Expression) = Subtract(expr, other) - def *(other: Expression) = Multiply(expr, other) - def /(other: Expression) = Divide(expr, other) + def + (other: Expression) = Add(expr, other) + def - (other: Expression) = Subtract(expr, other) + def * (other: Expression) = Multiply(expr, other) + def / (other: Expression) = Divide(expr, other) - def &&(other: Expression) = And(expr, other) - def ||(other: Expression) = Or(expr, other) + def && (other: Expression) = And(expr, other) + def || (other: Expression) = Or(expr, other) - def <(other: Expression) = LessThan(expr, other) - def <=(other: Expression) = LessThanOrEqual(expr, other) - def >(other: Expression) = GreaterThan(expr, other) - def >=(other: Expression) = GreaterThanOrEqual(expr, other) - def ===(other: Expression) = Equals(expr, other) + def < (other: Expression) = LessThan(expr, other) + def <= (other: Expression) = LessThanOrEqual(expr, other) + def > (other: Expression) = GreaterThan(expr, other) + def >= (other: Expression) = GreaterThanOrEqual(expr, other) + def === (other: Expression) = Equals(expr, other) def asc = SortOrder(expr, Ascending) def desc = SortOrder(expr, Descending) @@ -79,9 +79,10 @@ package object dsl { def attr = analysis.UnresolvedAttribute(s) /** Creates a new typed attributes of type int */ - def int = AttributeReference(s, IntegerType, false)() + def int = AttributeReference(s, IntegerType, nullable = false)() + /** Creates a new typed attributes of type string */ - def string = AttributeReference(s, StringType, false)() + def string = AttributeReference(s, StringType, nullable = false)() } implicit class DslAttribute(a: AttributeReference) { @@ -94,10 +95,17 @@ package object dsl { implicit class DslLogicalPlan(plan: LogicalPlan) { def select(exprs: NamedExpression*) = Project(exprs, plan) + def where(condition: Expression) = Filter(condition, plan) - def join(otherPlan: LogicalPlan, joinType: JoinType = Inner, condition: Option[Expression] = None) = + + def join( + otherPlan: LogicalPlan, + joinType: JoinType = Inner, + condition: Option[Expression] = None) = Join(plan, otherPlan, joinType, condition) + def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, plan) + def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*) = { val aliasedExprs = aggregateExprs.map { case ne: NamedExpression => ne @@ -105,7 +113,9 @@ package object dsl { } Aggregate(groupingExprs, aliasedExprs, plan) } + def subquery(alias: Symbol) = Subquery(alias.name, plan) + def unionAll(otherPlan: LogicalPlan) = Union(plan, otherPlan) def filter[T1](arg1: Symbol)(udf: (T1) => Boolean) = @@ -122,4 +132,4 @@ package object dsl { def analyze = analysis.SimpleAnalyzer(plan) } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/errors/package.scala b/src/main/scala/catalyst/errors/package.scala index 41bd9c1d8055e..df829ac26fa93 100644 --- a/src/main/scala/catalyst/errors/package.scala +++ b/src/main/scala/catalyst/errors/package.scala @@ -3,7 +3,7 @@ package catalyst import trees._ /** - * Functions for attaching and retrieving trees that are associated with errors in a catalyst optimizer. + * Functions for attaching and retrieving trees that are associated with errors. */ package object errors { @@ -31,4 +31,4 @@ package object errors { * the stack of exceptions of type `TreeType` is returned. */ def getTree[TreeType <: TreeNode[_]](f: => Unit): TreeType = ??? // TODO: Implement -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/examples/SchemaRddExample.scala b/src/main/scala/catalyst/examples/SchemaRddExample.scala index 32b61d20130c9..fdacdb734e0e9 100644 --- a/src/main/scala/catalyst/examples/SchemaRddExample.scala +++ b/src/main/scala/catalyst/examples/SchemaRddExample.scala @@ -39,4 +39,4 @@ object SchemaRddExample { val filtered2 = testLogs.filter( _.date match { case dateRegEx(_,day,_) => day.toInt == 1 } ) filtered2.toRdd.collect.foreach(println) } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/examples/ViewsExample.scala b/src/main/scala/catalyst/examples/ViewsExample.scala index 5bc1bc1262454..15156893833cd 100644 --- a/src/main/scala/catalyst/examples/ViewsExample.scala +++ b/src/main/scala/catalyst/examples/ViewsExample.scala @@ -12,9 +12,12 @@ import execution.TestShark._ // For .toRdd execution using locally running test object ViewsExample { def main(args: Array[String]): Unit = { // Create a list of named views that can be substituted into logical plans. - // In this example the views read from local, in-memory relations with schema (a INT, b STRING) and (c INT, d STRING) - // respectively. loadData returns a copy of that relation with the specified tuples appended to the Rdd. - // The .select uses the DSL to add a projection on top of the relation that returns only the column "a". + // In this example the views read from local, in-memory relations with schema + // (a INT, b STRING) and (c INT, d STRING) respectively. + // + // loadData returns a copy of that relation with the specified tuples appended to the Rdd. + // The .select uses the DSL to add a projection on top of the relation that returns only + // the column "a". val views = Map( "view1" -> LocalRelation('a.int, 'b.string).loadData(("a", 1) :: ("b", 2) :: Nil).select('a), "view2" -> LocalRelation('c.int, 'd.string).loadData(("c", 1) :: ("d", 2) :: Nil) @@ -35,6 +38,6 @@ object ViewsExample { println(s"With relations:\n$withRelations ") println(s"Analyzed:\n${withRelations.analyze}") // Print with all references resolved. - println(s"Answer: ${withRelations.toRdd.collect.toSeq}") + println(s"Answer: ${withRelations.toRdd.collect().toSeq}") } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index fe76314a7dcf2..5ebc30fc982df 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -6,8 +6,10 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.generic.GenericUDF import org.apache.hadoop.hive.ql.exec.UDF -import org.apache.hadoop.hive.serde2.objectinspector.primitive.{AbstractPrimitiveJavaObjectInspector, PrimitiveObjectInspectorFactory} -import org.apache.hadoop.io._ +import org.apache.hadoop.hive.serde2.{io => hiveIo} +import org.apache.hadoop.hive.serde2.objectinspector.primitive.AbstractPrimitiveJavaObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.{io => hadoopIo} import expressions._ import types._ @@ -38,15 +40,15 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry { } def javaClassToDataType(clz: Class[_]): DataType = clz match { - case c: Class[_] if c == classOf[DoubleWritable] => DoubleType - case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.DoubleWritable] => DoubleType - case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.HiveDecimalWritable] => DecimalType - case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.ByteWritable] => ByteType - case c: Class[_] if c == classOf[org.apache.hadoop.hive.serde2.io.ShortWritable] => ShortType - case c: Class[_] if c == classOf[Text] => StringType - case c: Class[_] if c == classOf[org.apache.hadoop.io.IntWritable] => IntegerType - case c: Class[_] if c == classOf[org.apache.hadoop.io.LongWritable] => LongType - case c: Class[_] if c == classOf[org.apache.hadoop.io.FloatWritable] => FloatType + case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType + case c: Class[_] if c == classOf[hiveIo.DoubleWritable] => DoubleType + case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType + case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType + case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType + case c: Class[_] if c == classOf[hadoopIo.Text] => StringType + case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType + case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType + case c: Class[_] if c == classOf[hadoopIo.FloatWritable] => FloatType case c: Class[_] if c == classOf[java.lang.String] => StringType case c: Class[_] if c == java.lang.Short.TYPE => ShortType case c: Class[_] if c == java.lang.Integer.TYPE => ShortType @@ -82,14 +84,14 @@ abstract class HiveUdf extends Expression with ImplementedUdf with Logging { def unwrap(a: Any): Any = a match { case null => null - case i: IntWritable => i.get - case t: Text => t.toString - case l: LongWritable => l.get - case d: DoubleWritable => d.get() - case d: org.apache.hadoop.hive.serde2.io.DoubleWritable => d.get - case s: org.apache.hadoop.hive.serde2.io.ShortWritable => s.get - case b: BooleanWritable => b.get() - case b: org.apache.hadoop.hive.serde2.io.ByteWritable => b.get + case i: hadoopIo.IntWritable => i.get + case t: hadoopIo.Text => t.toString + case l: hadoopIo.LongWritable => l.get + case d: hadoopIo.DoubleWritable => d.get() + case d: hiveIo.DoubleWritable => d.get + case s: hiveIo.ShortWritable => s.get + case b: hadoopIo.BooleanWritable => b.get() + case b: hiveIo.ByteWritable => b.get case list: java.util.List[_] => list.map(unwrap) case p: java.lang.Short => p case p: java.lang.Long => p @@ -174,7 +176,7 @@ case class HiveGenericUdf( } def wrap(a: Any): Any = a match { - case s: String => new Text(s) + case s: String => new hadoopIo.Text(s) case i: Int => i: java.lang.Integer case b: Boolean => b: java.lang.Boolean case d: Double => d: java.lang.Double @@ -191,4 +193,4 @@ case class HiveGenericUdf( }.toArray unwrap(instance.evaluate(args)) } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index c46b3c3d4655d..48fda0e4f4372 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -109,7 +109,8 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt } val tableDesc = new TableDesc( - Class.forName(table.getSd.getSerdeInfo.getSerializationLib).asInstanceOf[Class[AbstractDeserializer]], + Class.forName(table.getSd.getSerdeInfo.getSerializationLib) + .asInstanceOf[Class[AbstractDeserializer]], Class.forName(table.getSd.getInputFormat).asInstanceOf[Class[InputFormat[_,_]]], Class.forName(table.getSd.getOutputFormat), hiveQlTable.getMetadata diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala similarity index 99% rename from src/main/scala/catalyst/execution/planningStrategies.scala rename to src/main/scala/catalyst/execution/PlanningStrategies.scala index 2a26269daea1c..d39d383e24629 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -166,4 +166,4 @@ trait PlanningStrategies { } } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 77599ba8e059f..6de1d1ded8168 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -153,4 +153,4 @@ abstract class SharkInstance extends Logging { } implicit def logicalToSharkQuery(plan: LogicalPlan) = new LogicalSharkQuery { val parsed = plan } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/TableReader.scala b/src/main/scala/catalyst/execution/TableReader.scala index 040d57f95fe9e..1f7221e84bfe1 100644 --- a/src/main/scala/catalyst/execution/TableReader.scala +++ b/src/main/scala/catalyst/execution/TableReader.scala @@ -34,7 +34,8 @@ private[catalyst] sealed trait TableReader { * Helper class for scanning tables stored in Hadoop - e.g., to read Hive tables that reside in the * data warehouse directory. */ -private[catalyst] class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) +private[catalyst] +class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) extends TableReader { // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 7744c8f0fbc29..9af118b6bd10c 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -68,7 +68,7 @@ object TestShark extends SharkInstance { /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") /** The location of the hive source code. */ - lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME") + lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME").getCanonicalPath // Override so we can intercept relative paths and rewrite them to point at hive. override def runSqlHive(sql: String): Seq[String] = super.runSqlHive(rewritePaths(sql)) @@ -89,10 +89,7 @@ object TestShark extends SharkInstance { * hive test cases assume the system is set up. */ private def rewritePaths(cmd: String): String = - if (cmd.toUpperCase contains "LOAD DATA") - cmd.replaceAll("\\.\\.", hiveDevHome.getCanonicalPath) - else - cmd + if (cmd.toUpperCase contains "LOAD DATA") cmd.replaceAll("\\.\\.", hiveDevHome) else cmd val describedTable = "DESCRIBE (\\w+)".r @@ -130,7 +127,7 @@ object TestShark extends SharkInstance { } - /* We must repeat the implicits so that we bind to the overriden versions */ + /* We must repeat the implicits so that we bind to the overridden versions */ implicit class stringToTestQuery(str: String) { def q = new SharkSqlQuery(str) @@ -152,14 +149,14 @@ object TestShark extends SharkInstance { def registerTestTable(testTable: TestTable) = testTables += (testTable.name -> testTable) // The test tables that are defined in the Hive QTestUtil. - // https://github.com/apache/hive/blob/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java + // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java val hiveQTestUtilTables = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' INTO TABLE src".cmd), + s"LOAD DATA LOCAL INPATH '${hiveDevHome}/data/files/kv1.txt' INTO TABLE src".cmd), TestTable("src1", "CREATE TABLE src1 (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv3.txt' INTO TABLE src1".cmd), + s"LOAD DATA LOCAL INPATH '${hiveDevHome}/data/files/kv3.txt' INTO TABLE src1".cmd), TestTable("dest1", "CREATE TABLE IF NOT EXISTS dest1 (key INT, value STRING)".cmd), TestTable("dest2", @@ -167,10 +164,11 @@ object TestShark extends SharkInstance { TestTable("dest3", "CREATE TABLE IF NOT EXISTS dest3 (key INT, value STRING)".cmd), TestTable("srcpart", () => { - runSqlHive("CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") + runSqlHive( + "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { runSqlHive( - s"""LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' + s"""LOAD DATA LOCAL INPATH '${hiveDevHome}/data/files/kv1.txt' |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') """.stripMargin) } @@ -179,7 +177,7 @@ object TestShark extends SharkInstance { runSqlHive("CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { runSqlHive( - s"""LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' + s"""LOAD DATA LOCAL INPATH '${hiveDevHome}/data/files/kv1.txt' |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') """.stripMargin) } @@ -206,7 +204,10 @@ object TestShark extends SharkInstance { catalog.client.createTable(srcThrift) - runSqlHive(s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/complex.seq' INTO TABLE src_thrift") + runSqlHive( + s"""LOAD DATA LOCAL INPATH '${hiveDevHome}/data/files/complex.seq' + |INTO TABLE src_thrift + """.stripMargin) }), TestTable("serdeins", s"""CREATE TABLE serdeins (key INT, value STRING) @@ -219,7 +220,7 @@ object TestShark extends SharkInstance { |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/sales.txt' INTO TABLE sales".cmd), + s"LOAD DATA LOCAL INPATH '${hiveDevHome}/data/files/sales.txt' INTO TABLE sales".cmd), TestTable("episodes", s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) |ROW FORMAT SERDE '${classOf[AvroSerDe].getCanonicalName}' @@ -251,7 +252,7 @@ object TestShark extends SharkInstance { | }' |) """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/episodes.avro' INTO TABLE episodes".cmd) + s"LOAD DATA LOCAL INPATH '${hiveDevHome}/data/files/episodes.avro' INTO TABLE episodes".cmd) ) hiveQTestUtilTables.foreach(registerTestTable) @@ -337,4 +338,4 @@ object TestShark extends SharkInstance { Thread.sleep(100000) } } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/Transform.scala b/src/main/scala/catalyst/execution/Transform.scala index c5f8870979a3f..3387716f27627 100644 --- a/src/main/scala/catalyst/execution/Transform.scala +++ b/src/main/scala/catalyst/execution/Transform.scala @@ -31,7 +31,8 @@ case class Transform( val outputStream = proc.getOutputStream val reader = new BufferedReader(new InputStreamReader(inputStream)) - // TODO: This should be exposed as an iterator instead of reading in all the data at once for a partition. + // TODO: This should be exposed as an iterator instead of reading in all the data + // at once for a partition. val outputLines = collection.mutable.ArrayBuffer[Row]() val readerThread = new Thread("Transform OutoutReader") { override def run() { @@ -51,4 +52,4 @@ case class Transform( outputLines.toIterator } } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 4a5c408f3d5ca..c15414cf002f9 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -66,8 +66,9 @@ case class Aggregate( def apply(input: Seq[Row]): Unit = { val evaluatedExpr = expr.map(Evaluate(_, input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) + if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { seen += evaluatedExpr + } } def result: Any = seen.size @@ -79,8 +80,9 @@ case class Aggregate( var result: Any = null def apply(input: Seq[Row]): Unit = { - if (result == null) + if (result == null) { result = Evaluate(expr, input) + } } } @@ -145,4 +147,4 @@ case class Aggregate( result } } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/debug.scala b/src/main/scala/catalyst/execution/debug.scala index a1e262f3e3035..b3ac54fda5b42 100644 --- a/src/main/scala/catalyst/execution/debug.scala +++ b/src/main/scala/catalyst/execution/debug.scala @@ -26,4 +26,4 @@ case class DebugNode(child: SharkPlan) extends UnaryNode { childRdd.foreach(println(_)) childRdd } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index fc5b745781642..adfb06d90bbbc 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -105,10 +105,11 @@ case class BroadcastNestedLoopJoin( val includedBroadcastTuples = streamedPlusMatches.map(_._2) val allIncludedBroadcastTuples = - if (includedBroadcastTuples.count == 0) + if (includedBroadcastTuples.count == 0) { new scala.collection.mutable.BitSet(broadcastedRelation.value.size) - else + } else { streamedPlusMatches.map(_._2).reduce(_ ++ _) + } val rightOuterMatches: Seq[Row] = if (joinType == RightOuter || joinType == FullOuter) { @@ -123,4 +124,4 @@ case class BroadcastNestedLoopJoin( sc.union(streamedPlusMatches.flatMap(_._1), sc.makeRDD(rightOuterMatches)) } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/package.scala b/src/main/scala/catalyst/execution/package.scala index d27a4721f6cc9..7eeea97aee53a 100644 --- a/src/main/scala/catalyst/execution/package.scala +++ b/src/main/scala/catalyst/execution/package.scala @@ -31,4 +31,4 @@ package object execution { case NullType => voidTypeInfo } } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/BoundAttribute.scala b/src/main/scala/catalyst/expressions/BoundAttribute.scala index 398d34fcc4449..eabb086750750 100644 --- a/src/main/scala/catalyst/expressions/BoundAttribute.scala +++ b/src/main/scala/catalyst/expressions/BoundAttribute.scala @@ -7,9 +7,9 @@ import errors._ import catalyst.plans.QueryPlan /** - * A bound reference points to a specific slot in the input tuple, allowing the actual value to be retrieved more - * efficiently. However, since operations like column pruning can change the layout of intermediate tuples, - * BindReferences should be run after all such transformations. + * A bound reference points to a specific slot in the input tuple, allowing the actual value + * to be retrieved more efficiently. However, since operations like column pruning can change + * the layout of intermediate tuples, BindReferences should be run after all such transformations. */ case class BoundReference(inputTuple: Int, ordinal: Int, baseReference: Attribute) extends Attribute with trees.LeafNode[Expression] { diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 143c6f1b670fc..1d90ba2a77680 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -12,77 +12,84 @@ object Evaluate extends Logging { def eval(e: Expression) = Evaluate(e, input) /** - * A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type and do any casting - * necessary of child evaluation. + * A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type + * and do any casting necessary of child evaluation. * - * Instead of matching here we could consider pushing the appropriate Fractional/Integral type into the type objects - * themselves. + * Instead of matching here we could consider pushing the appropriate Fractional/Integral type + * into the type objects themselves. */ @inline def n1(e: Expression, f: ((Numeric[Any], Any) => Any)): Any = { val evalE = eval(e) - if (evalE == null) + if (evalE == null) { null - else + } else { e.dataType match { case n: NumericType => val castedFunction = f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType] castedFunction(n.numeric, evalE.asInstanceOf[n.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } + } } @inline def n2(e1: Expression, e2: Expression, f: ((Numeric[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) + if (e1.dataType != e2.dataType) { throw new OptimizationException(e, s"Types do not match ${e1.dataType} != ${e2.dataType}") + } val evalE1 = eval(e1) val evalE2 = eval(e2) - if (evalE1 == null || evalE2 == null) + if (evalE1 == null || evalE2 == null) { null - else + } else { e1.dataType match { case n: NumericType => f.asInstanceOf[(Numeric[n.JvmType], n.JvmType, n.JvmType) => Int]( n.numeric, evalE1.asInstanceOf[n.JvmType], evalE2.asInstanceOf[n.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } + } } @inline def f2(e1: Expression, e2: Expression, f: ((Fractional[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) + if (e1.dataType != e2.dataType) { throw new OptimizationException(e, s"Types do not match ${e1.dataType} != ${e2.dataType}") + } val evalE1 = eval(e1) val evalE2 = eval(e2) - if (evalE1 == null || evalE2 == null) + if (evalE1 == null || evalE2 == null) { null - else + } else { e1.dataType match { case f: FractionalType => f.asInstanceOf[(Fractional[f.JvmType], f.JvmType, f.JvmType) => f.JvmType]( f.fractional, evalE1.asInstanceOf[f.JvmType], evalE2.asInstanceOf[f.JvmType]) case other => sys.error(s"Type $other does not support fractional operations") } + } } @inline def i2(e1: Expression, e2: Expression, f: ((Integral[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) + if (e1.dataType != e2.dataType) { throw new OptimizationException(e, s"Types do not match ${e1.dataType} != ${e2.dataType}") + } val evalE1 = eval(e1) val evalE2 = eval(e2) - if (evalE1 == null || evalE2 == null) + if (evalE1 == null || evalE2 == null) { null - else + } else { e1.dataType match { case i: IntegralType => f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( i.integral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } + } } @inline def castOrNull[A](e: Expression, f: String => A) = @@ -120,10 +127,7 @@ object Evaluate extends Logging { case Equals(l, r) => val left = eval(l) val right = eval(r) - if (left == null || right == null) - null - else - left == right + if (left == null || right == null) null else left == right case In(value, list) => val evaluatedValue = eval(value) @@ -138,6 +142,7 @@ object Evaluate extends Logging { eval(l).asInstanceOf[String] < eval(r).asInstanceOf[String] case LessThanOrEqual(l, r) if l.dataType == StringType && r.dataType == StringType => eval(l).asInstanceOf[String] <= eval(r).asInstanceOf[String] + // Numerics case GreaterThan(l, r) => n2(l, r, _.gt(_, _)) case GreaterThanOrEqual(l, r) => n2(l, r, _.gteq(_, _)) @@ -204,21 +209,23 @@ object Evaluate extends Logging { case And(l,r) => val left = eval(l) val right = eval(r) - if (left == false || right == false) + if (left == false || right == false) { false - else if (left == null || right == null ) + } else if (left == null || right == null ) { null - else + } else { true + } case Or(l,r) => val left = eval(l) val right = eval(r) - if (left == true || right == true) + if (left == true || right == true) { true - else if (left == null || right == null) + } else if (left == null || right == null) { null - else + } else { false + } /* References to input tuples */ case br @ BoundReference(inputTuple, ordinal, _) => try input(inputTuple)(ordinal) catch { @@ -239,7 +246,8 @@ object Evaluate extends Logging { case other => throw new OptimizationException(other, "evaluation not implemented") } - logger.debug(s"Evaluated $e => $result of type ${if (result == null) "null" else result.getClass.getName}, expected: ${e.dataType}") + val resultType = if (result == null) "null" else result.getClass.getName + logger.debug(s"Evaluated $e => $result of type $resultType, expected: ${e.dataType}") result } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/Expression.scala b/src/main/scala/catalyst/expressions/Expression.scala index 97f9f0320dbac..a75c1f3bf4ee8 100644 --- a/src/main/scala/catalyst/expressions/Expression.scala +++ b/src/main/scala/catalyst/expressions/Expression.scala @@ -11,12 +11,13 @@ abstract class Expression extends TreeNode[Expression] { /** * Returns true when an expression is a candidate for static evaluation before the query is * executed. + * * The following conditions are used to determine suitability for constant folding: * - A [[expressions.Coalesce Coalesce]] is foldable if all of its children are foldable * - A [[expressions.BinaryExpression BinaryExpression]] is foldable if its both left and right * child are foldable - * - A [[expressions.Not Not]], [[expressions.IsNull IsNull]], or [[expressions.IsNotNull IsNotNull]] - * is foldable if its child is foldable. + * - A [[expressions.Not Not]], [[expressions.IsNull IsNull]], or + * [[expressions.IsNotNull IsNotNull]] is foldable if its child is foldable. * - A [[expressions.Literal]] is foldable. * - A [[expressions.Cast Cast]] or [[expressions.UnaryMinus UnaryMinus]] is foldable if its * child is foldable. diff --git a/src/main/scala/catalyst/expressions/functions.scala b/src/main/scala/catalyst/expressions/ImplementedUdf.scala similarity index 98% rename from src/main/scala/catalyst/expressions/functions.scala rename to src/main/scala/catalyst/expressions/ImplementedUdf.scala index c16e096c93345..4a287e0e0e74d 100644 --- a/src/main/scala/catalyst/expressions/functions.scala +++ b/src/main/scala/catalyst/expressions/ImplementedUdf.scala @@ -6,4 +6,4 @@ package expressions */ trait ImplementedUdf { def evaluate(evaluatedChildren: Seq[Any]): Any -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/Rand.scala b/src/main/scala/catalyst/expressions/Rand.scala index b6a0a9cd080a5..bec89e9a1da5b 100644 --- a/src/main/scala/catalyst/expressions/Rand.scala +++ b/src/main/scala/catalyst/expressions/Rand.scala @@ -8,4 +8,4 @@ case object Rand extends LeafExpression { def nullable = false def references = Set.empty override def toString = "RAND()" -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/ScalaUdf.scala b/src/main/scala/catalyst/expressions/ScalaUdf.scala index 6681e17c01a75..05bde2133063a 100644 --- a/src/main/scala/catalyst/expressions/ScalaUdf.scala +++ b/src/main/scala/catalyst/expressions/ScalaUdf.scala @@ -4,15 +4,15 @@ package expressions import types._ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) - extends Expression with ImplementedUdf { + extends Expression with ImplementedUdf { def references = children.flatMap(_.references).toSet def nullable = true def evaluate(evaluatedChildren: Seq[Any]): Any = { children.size match { - case 1 => function.asInstanceOf[Function1[Any, Any]](evaluatedChildren(0)) - case 2 => function.asInstanceOf[Function2[Any, Any, Any]](evaluatedChildren(0), evaluatedChildren(1)) + case 1 => function.asInstanceOf[(Any) => Any](evaluatedChildren(0)) + case 2 => function.asInstanceOf[(Any, Any) => Any](evaluatedChildren(0), evaluatedChildren(1)) } } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/SortOrder.scala b/src/main/scala/catalyst/expressions/SortOrder.scala index 967590c030359..04a70bc2104ca 100644 --- a/src/main/scala/catalyst/expressions/SortOrder.scala +++ b/src/main/scala/catalyst/expressions/SortOrder.scala @@ -13,4 +13,4 @@ case class SortOrder(child: Expression, direction: SortDirection) extends UnaryE def dataType = child.dataType def nullable = child.nullable override def toString = s"$child ${if (direction == Ascending) "ASC" else "DESC"}" -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/WrapDynamic.scala b/src/main/scala/catalyst/expressions/WrapDynamic.scala index 128d3de9cb577..64150901030f7 100644 --- a/src/main/scala/catalyst/expressions/WrapDynamic.scala +++ b/src/main/scala/catalyst/expressions/WrapDynamic.scala @@ -23,4 +23,4 @@ class DynamicRow(val schema: Seq[Attribute], values: Seq[Any]) val ordinal = schema.indexWhere(_.name == attributeName) values(ordinal).toString } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala index c5aeeccc54c88..0778f40d23b52 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/src/main/scala/catalyst/expressions/aggregates.scala @@ -60,4 +60,4 @@ case class First(child: Expression) extends AggregateExpression with trees.Unary def nullable = child.nullable def dataType = child.dataType override def toString = s"FIRST($child)" -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/arithmetic.scala b/src/main/scala/catalyst/expressions/arithmetic.scala index 3085f80faaba6..a82dfa89fbb1d 100644 --- a/src/main/scala/catalyst/expressions/arithmetic.scala +++ b/src/main/scala/catalyst/expressions/arithmetic.scala @@ -13,15 +13,17 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { abstract class BinaryArithmetic extends BinaryExpression { self: Product => + def nullable = left.nullable || right.nullable override lazy val resolved = left.resolved && right.resolved && left.dataType == right.dataType def dataType = { - if (!resolved) - throw new UnresolvedException( - this, s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") + if (!resolved) { + throw new UnresolvedException(this, + s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") + } left.dataType } } @@ -44,4 +46,4 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "%" -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/complexTypes.scala b/src/main/scala/catalyst/expressions/complexTypes.scala index 241f056b91c05..05ae9327e9d1e 100644 --- a/src/main/scala/catalyst/expressions/complexTypes.scala +++ b/src/main/scala/catalyst/expressions/complexTypes.scala @@ -32,4 +32,4 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio } override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType] override def toString = s"$child.$fieldName" -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/literals.scala b/src/main/scala/catalyst/expressions/literals.scala index 16d7f52b124c5..2277bef2dbf72 100644 --- a/src/main/scala/catalyst/expressions/literals.scala +++ b/src/main/scala/catalyst/expressions/literals.scala @@ -33,4 +33,4 @@ case class Literal(value: Any, dataType: DataType) extends LeafExpression { def references = Set.empty override def toString = if (value != null) value.toString else "null" -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/namedExpressions.scala b/src/main/scala/catalyst/expressions/namedExpressions.scala index 64f21cb66b210..2ba63d4042d5b 100644 --- a/src/main/scala/catalyst/expressions/namedExpressions.scala +++ b/src/main/scala/catalyst/expressions/namedExpressions.scala @@ -26,13 +26,14 @@ abstract class NamedExpression extends Expression { def toAttribute: Attribute protected def typeSuffix = - if(resolved) + if (resolved) { dataType match { case LongType => "L" case _ => "" } - else + } else { "" + } } abstract class Attribute extends NamedExpression { @@ -97,26 +98,29 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea case _ => false } - def newInstance = - AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) + def newInstance = AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) /** * Returns a copy of this [[AttributeReference]] with changed nullability. */ - def withNullability(newNullability: Boolean) = - if (nullable == newNullability) + def withNullability(newNullability: Boolean) = { + if (nullable == newNullability) { this - else + } else { AttributeReference(name, dataType, newNullability)(exprId, qualifiers) + } + } /** * Returns a copy of this [[AttributeReference]] with new qualifiers. */ - def withQualifiers(newQualifiers: Seq[String]) = - if (newQualifiers == qualifiers) + def withQualifiers(newQualifiers: Seq[String]) = { + if (newQualifiers == qualifiers) { this - else + } else { AttributeReference(name, dataType, nullable)(exprId, newQualifiers) + } + } override def toString: String = s"$name#${exprId.id}$typeSuffix" -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/package.scala b/src/main/scala/catalyst/expressions/package.scala index 877b428258b71..ea9dd6169179c 100644 --- a/src/main/scala/catalyst/expressions/package.scala +++ b/src/main/scala/catalyst/expressions/package.scala @@ -1,29 +1,31 @@ package catalyst /** - * A set of classes that can be used to represent trees of relational expressions. A key goal of the expression library - * is to hide the details of naming and scoping from developers who want to manipulate trees of relational operators. As - * such, the library defines a special type of expression, a [[NamedExpression]] in addition to the standard collection - * of expressions. + * A set of classes that can be used to represent trees of relational expressions. A key goal of + * the expression library is to hide the details of naming and scoping from developers who want to + * manipulate trees of relational operators. As such, the library defines a special type of + * expression, a [[NamedExpression]] in addition to the standard collection of expressions. * * ==Standard Expressions== - * A library of standard expressions (e.g., [[Add]], [[Equals]]), aggregates (e.g., SUM, COUNT), and other computations - * (e.g. UDFs). Each expression type is capable of determining its output schema as a function of its children’s output - * schema. + * A library of standard expressions (e.g., [[Add]], [[Equals]]), aggregates (e.g., SUM, COUNT), + * and other computations (e.g. UDFs). Each expression type is capable of determining its output + * schema as a function of its children's output schema. * * ==Named Expressions== - * Some expression are named and thus can be referenced by later operators in the dataflow graph. The two types of named - * expressions are [[AttributeReference]]s and [[Alias]]es. [[AttributeReference]]s refer to attributes of the input tuple - * for a given operator and form the leaves of some expression trees. Aliases assign a name to intermediate computations. - * For example, in the SQL statement `SELECT a+b AS c FROM ...`, the expressions `a` and `b` would be represented by - * `AttributeReferences` and `c` would be represented by an `Alias`. + * Some expression are named and thus can be referenced by later operators in the dataflow graph. + * The two types of named expressions are [[AttributeReference]]s and [[Alias]]es. + * [[AttributeReference]]s refer to attributes of the input tuple for a given operator and form + * the leaves of some expression trees. Aliases assign a name to intermediate computations. + * For example, in the SQL statement `SELECT a+b AS c FROM ...`, the expressions `a` and `b` would + * be represented by `AttributeReferences` and `c` would be represented by an `Alias`. * - * During [[analysis]], all named expressions are assigned a globally unique expression id, which can be used for equality - * comparisons. While the original names are kept around for debugging purposes, they should never be used to check - * if two attributes refer to the same value, as plan transformations can result in the introduction of naming ambiguity. - * For example, consider a plan that contains subqueries, both of which are reading from the same table. If an - * optimization removesmthe subqueries, scoping information would be destroyed, eliminating the ability to reason about - * which subquery produced a given attribute. + * During [[analysis]], all named expressions are assigned a globally unique expression id, which + * can be used for equality comparisons. While the original names are kept around for debugging + * purposes, they should never be used to check if two attributes refer to the same value, as + * plan transformations can result in the introduction of naming ambiguity. For example, consider + * a plan that contains subqueries, both of which are reading from the same table. If an + * optimization removes the subqueries, scoping information would be destroyed, eliminating the + * ability to reason about which subquery produced a given attribute. * * ==Evaluation== * The result of expressions can be evaluated using the [[Evaluate]] object. diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/src/main/scala/catalyst/optimizer/Optimizer.scala index 4525d8095f039..e0d25dfd5f4e6 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/src/main/scala/catalyst/optimizer/Optimizer.scala @@ -146,4 +146,4 @@ object SimplifyCasts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case Cast(e, dataType) if e.dataType == dataType => e } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/package.scala b/src/main/scala/catalyst/package.scala index 3355b6402a834..52e9b39db9196 100644 --- a/src/main/scala/catalyst/package.scala +++ b/src/main/scala/catalyst/package.scala @@ -3,7 +3,9 @@ * Catalyst is a framework for performing optimization on trees of dataflow operators. */ package object catalyst { + protected[catalyst] def Logger(name: String) = com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger(name)) + protected[catalyst] type Logging = com.typesafe.scalalogging.slf4j.Logging -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/planning/QueryPlanner.scala b/src/main/scala/catalyst/planning/QueryPlanner.scala index cc4c89aab7f0b..b8bbc32b007d9 100644 --- a/src/main/scala/catalyst/planning/QueryPlanner.scala +++ b/src/main/scala/catalyst/planning/QueryPlanner.scala @@ -6,13 +6,14 @@ import plans.logical.LogicalPlan import trees._ /** - * Extended by classes that transform [[plans.logical.LogicalPlan LogicalPlan]]s into physical plans. + * Abstract class for transforming [[plans.logical.LogicalPlan LogicalPlan]]s into physical plans. * Child classes are responsible for specifying a list of [[Strategy]] objects that each of which * can return a list of possible physical plan options. If a given strategy is unable to plan all * of the remaining operators in the tree, it can call [[planLater]], which returns a placeholder * object that will be filled in using other available strategies. * - * TODO: RIGHT NOW ONLY ONE PLAN IS RETURNED EVER... PLAN SPACE EXPLORATION WILL BE IMPLEMENTED LATER. + * TODO: RIGHT NOW ONLY ONE PLAN IS RETURNED EVER... + * PLAN SPACE EXPLORATION WILL BE IMPLEMENTED LATER. * * @tparam PhysicalPlan The type of physical plan produced by this [[QueryPlanner]] */ @@ -42,4 +43,4 @@ abstract class QueryPlanner[PhysicalPlan <: TreeNode[PhysicalPlan]] { assert(iter.hasNext, s"No plan for $plan") iter } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/planning/package.scala b/src/main/scala/catalyst/planning/package.scala index b26fe64399f53..c3a21afac2148 100644 --- a/src/main/scala/catalyst/planning/package.scala +++ b/src/main/scala/catalyst/planning/package.scala @@ -3,4 +3,4 @@ package catalyst /** * Contains classes for enumerating possible physical plans for a given logical query plan. */ -package object planning \ No newline at end of file +package object planning diff --git a/src/main/scala/catalyst/planning/patterns.scala b/src/main/scala/catalyst/planning/patterns.scala index 147a8fc6dffc9..ffed9f44198de 100644 --- a/src/main/scala/catalyst/planning/patterns.scala +++ b/src/main/scala/catalyst/planning/patterns.scala @@ -40,4 +40,4 @@ object Unions { case Union(l, r) => collectUnionChildren(l) ++ collectUnionChildren(r) case other => other :: Nil } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 3546416402f11..e06ecab6c2a21 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -105,4 +105,4 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case other => Nil }.toSeq } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala index 792eb88e77a28..7b493374fc779 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala @@ -73,4 +73,4 @@ abstract class UnaryNode extends LogicalPlan with trees.UnaryNode[LogicalPlan] { */ abstract class BinaryNode extends LogicalPlan with trees.BinaryNode[LogicalPlan] { self: Product => -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/plans/logical/Transform.scala b/src/main/scala/catalyst/plans/logical/Transform.scala index 2192a58fe2991..50eac7b3be664 100644 --- a/src/main/scala/catalyst/plans/logical/Transform.scala +++ b/src/main/scala/catalyst/plans/logical/Transform.scala @@ -17,4 +17,4 @@ case class Transform( output: Seq[Attribute], child: LogicalPlan) extends UnaryNode { def references = input.flatMap(_.references).toSet -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index 9756672c85654..34fa3b68b1e6a 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -35,7 +35,10 @@ case class Join( def output = left.output ++ right.output } -case class InsertIntoTable(table: BaseRelation, partition: Map[String, Option[String]], child: LogicalPlan) +case class InsertIntoTable( + table: BaseRelation, + partition: Map[String, Option[String]], + child: LogicalPlan) extends LogicalPlan { // The table being inserted into is a child for the purposes of transformations. def children = table :: child :: Nil diff --git a/src/main/scala/catalyst/plans/shared.scala b/src/main/scala/catalyst/plans/shared.scala index 75730aa139390..da47d3d9c8f36 100644 --- a/src/main/scala/catalyst/plans/shared.scala +++ b/src/main/scala/catalyst/plans/shared.scala @@ -5,4 +5,4 @@ sealed abstract class JoinType case object Inner extends JoinType case object LeftOuter extends JoinType case object RightOuter extends JoinType -case object FullOuter extends JoinType \ No newline at end of file +case object FullOuter extends JoinType diff --git a/src/main/scala/catalyst/rules/package.scala b/src/main/scala/catalyst/rules/package.scala index abf1be98f1abb..6f4ea86c71d41 100644 --- a/src/main/scala/catalyst/rules/package.scala +++ b/src/main/scala/catalyst/rules/package.scala @@ -3,4 +3,4 @@ package catalyst /** * A framework for applying batches rewrite rules to trees, possibly to fixed point. */ -package object rules \ No newline at end of file +package object rules diff --git a/src/main/scala/catalyst/trees/package.scala b/src/main/scala/catalyst/trees/package.scala index 4b4dd859e682a..4ce0c28dd5d0d 100644 --- a/src/main/scala/catalyst/trees/package.scala +++ b/src/main/scala/catalyst/trees/package.scala @@ -1,18 +1,20 @@ package catalyst /** - * A library for easily manipulating trees of operators. Operators that extend TreeNode are granted the following - * interface: + * A library for easily manipulating trees of operators. Operators that extend TreeNode are + * granted the following interface: *
      *
    • Scala collection like methods (foreach, map, flatMap, collect, etc)
    • - *
    • transform - accepts a partial function that is used to generate a new tree. When the partial function can be - * applied to a given tree segment, that segment is replaced with the result. After attempting to apply the partial - * function to a given node, the transform function recursively attempts to apply the function to that node’s - * children.
    • + *
    • + * transform - accepts a partial function that is used to generate a new tree. When the + * partial function can be applied to a given tree segment, that segment is replaced with the + * result. After attempting to apply the partial function to a given node, the transform + * function recursively attempts to apply the function to that node’s children. + *
    • *
    • debugging support - pretty printing, easy splicing of trees, etc.
    • *
    */ package object trees { // Since we want tree nodes to be lightweight, we create one logger for all treenode instances. protected val logger = Logger("catalyst.trees") -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 6e864d047ff4a..360ccf7c7de16 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -116,4 +116,4 @@ case class ArrayType(elementType: DataType) extends DataType case class StructField(name: String, dataType: DataType, nullable: Boolean) case class StructType(fields: Seq[StructField]) extends DataType -case class MapType(keyType: DataType, valueType: DataType) extends DataType \ No newline at end of file +case class MapType(keyType: DataType, valueType: DataType) extends DataType diff --git a/src/main/scala/catalyst/util/package.scala b/src/main/scala/catalyst/util/package.scala index a7cf951a9c590..d1ea9ed3dee3b 100644 --- a/src/main/scala/catalyst/util/package.scala +++ b/src/main/scala/catalyst/util/package.scala @@ -31,7 +31,7 @@ package object util { finally { inStream.close() } - new String(outStream.toByteArray(), encoding) + new String(outStream.toByteArray, encoding) } def resourceToString( @@ -53,7 +53,7 @@ package object util { finally { inStream.close() } - new String(outStream.toByteArray(), encoding) + new String(outStream.toByteArray, encoding) } def stringToFile(file: File, str: String): File = { @@ -63,15 +63,17 @@ package object util { file } - def sideBySide(left: String, right: String): Seq[String] = sideBySide(left.split("\n"), right.split("\n")) + def sideBySide(left: String, right: String): Seq[String] = { + sideBySide(left.split("\n"), right.split("\n")) + } def sideBySide(left: Seq[String], right: Seq[String]): Seq[String] = { val maxLeftSize = left.map(_.size).max - val leftPadded = left ++ Seq.fill(if (left.size < right.size) right.size - left.size else 0)("") - val rightPadded = right ++ Seq.fill(if (right.size < left.size) left.size - right.size else 0)("") + val leftPadded = left ++ Seq.fill(math.max(right.size - left.size, 0))("") + val rightPadded = right ++ Seq.fill(math.max(left.size - right.size, 0))("") leftPadded.zip(rightPadded).map { - case (l,r) => (if (l == r) " " else "!") + l + (" " * ((maxLeftSize - l.size) + 3)) + r + case (l, r) => (if (l == r) " " else "!") + l + (" " * ((maxLeftSize - l.size) + 3)) + r } } @@ -90,4 +92,4 @@ package object util { org.apache.log4j.Logger.getLogger(a.getClass.getName).setLevel(org.apache.log4j.Level.DEBUG) } } -} \ No newline at end of file +} From 7213a2c466d7e30cabb2a2fd07bc81a8d7e36cfe Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 Jan 2014 16:14:32 -0800 Subject: [PATCH 572/778] style fix for Hive.scala. --- src/main/scala/catalyst/frontend/Hive.scala | 226 ++++++++++++-------- 1 file changed, 137 insertions(+), 89 deletions(-) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index a05a68407d239..b0af22caa8eb2 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -121,12 +121,13 @@ object HiveQl { ) /** - * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations similar to - * [[catalyst.trees.TreeNode]]. + * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations + * similar to [[catalyst.trees.TreeNode]]. * - * Note that this should be considered very experimental and is not indented as a replacement for TreeNode. Primarily - * it should be noted ASTNodes are not immutable and do not appear to have clean copy semantics. Therefore, users of - * this class should take care when copying/modifying trees that might be used elsewhere. + * Note that this should be considered very experimental and is not indented as a replacement + * for TreeNode. Primarily it should be noted ASTNodes are not immutable and do not appear to + * have clean copy semantics. Therefore, users of this class should take care when + * copying/modifying trees that might be used elsewhere. */ implicit class TransformableNode(n: ASTNode) { /** @@ -178,10 +179,10 @@ object HiveQl { * for equality. */ def checkEquals(other: ASTNode) { - def check(field: String, f: ASTNode => Any) = - if (f(n) != f(other)) - sys.error(s"$field does not match for trees. '${f(n)}' != '${f(other)}' left: ${dumpTree(n)}, right: ${dumpTree(other)}") - + def check(field: String, f: ASTNode => Any) = if (f(n) != f(other)) { + sys.error(s"$field does not match for trees. " + + s"'${f(n)}' != '${f(other)}' left: ${dumpTree(n)}, right: ${dumpTree(other)}") + } check("name", _.getName) check("type", _.getType) check("text", _.getText) @@ -201,35 +202,33 @@ object HiveQl { /** * Returns the AST for the given SQL string. */ - def getAst(sql: String): ASTNode = { - ParseUtils.findRootNonNullToken( - (new ParseDriver()).parse(sql)) - } + def getAst(sql: String): ASTNode = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql)) def parseSql(sql: String): LogicalPlan = { try { - if (sql.toLowerCase.startsWith("set")) + if (sql.toLowerCase.startsWith("set")) { ConfigurationAssignment(sql) - else if (sql.toLowerCase.startsWith("add jar")) + } else if (sql.toLowerCase.startsWith("add jar")) { AddJar(sql.drop(8)) - else if (sql.toLowerCase.startsWith("add file")) + } else if (sql.toLowerCase.startsWith("add file")) { AddFile(sql.drop(9)) - else if (sql.startsWith("dfs")) + } else if (sql.startsWith("dfs")) { DfsCommand(sql) - else if (sql.startsWith("source")) + } else if (sql.startsWith("source")) { SourceCommand(sql.split(" ").toSeq match { case Seq("source", filePath) => filePath }) - else if (sql.startsWith("!")) + } else if (sql.startsWith("!")) { ShellCommand(sql.drop(1)) - else { + } else { val tree = getAst(sql) - if (nativeCommands contains tree.getText) + if (nativeCommands contains tree.getText) { NativeCommand(sql) - else + } else { nodeToPlan(tree) match { case NativePlaceholder => NativeCommand(sql) case other => other } + } } } catch { case e: Exception => throw new ParseException(sql, e) @@ -240,7 +239,7 @@ object HiveQl { val tree = try { ParseUtils.findRootNonNullToken( - (new ParseDriver()).parse(ddl, null /* no context required for parsing alone */)) + (new ParseDriver).parse(ddl, null /* no context required for parsing alone */)) } catch { case pe: org.apache.hadoop.hive.ql.parse.ParseException => throw new RuntimeException(s"Failed to parse ddl: '$ddl'", pe) @@ -260,7 +259,8 @@ object HiveQl { /** @return matches of the form (tokenName, children). */ def unapply(t: Any): Option[(String, Seq[ASTNode])] = t match { case t: ASTNode => - Some((t.getText, Option(t.getChildren).map(_.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]])) + Some((t.getText, + Option(t.getChildren).map(_.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]])) case _ => None } } @@ -279,8 +279,8 @@ object HiveQl { } def getClause(clauseName: String, nodeList: Seq[Node]) = - getClauseOption(clauseName, nodeList) - .getOrElse(sys.error(s"Expected clause $clauseName missing from ${nodeList.map(dumpTree(_)).mkString("\n")}")) + getClauseOption(clauseName, nodeList).getOrElse(sys.error( + s"Expected clause $clauseName missing from ${nodeList.map(dumpTree(_)).mkString("\n")}")) def getClauseOption(clauseName: String, nodeList: Seq[Node]): Option[Node] = { nodeList.filter { case ast: ASTNode => ast.getText == clauseName } match { @@ -291,9 +291,7 @@ object HiveQl { } protected def nodeToAttribute(node: Node): Attribute = node match { - case Token("TOK_TABCOL", - Token(colName, Nil) :: - dataType :: Nil) => + case Token("TOK_TABCOL", Token(colName, Nil) :: dataType :: Nil) => AttributeReference(colName, nodeToDataType(dataType), true)() case a: ASTNode => @@ -348,14 +346,18 @@ object HiveQl { NoRelation case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. - val Some(query) :: _ :: _ :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) + val Some(query) :: _ :: _ :: Nil = + getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) // TODO: support EXTENDED? ExplainCommand(nodeToPlan(query)) - case Token("TOK_CREATETABLE", children) if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty => + case Token("TOK_CREATETABLE", children) + if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty => val (Some(Token("TOK_TABNAME", Token(tableName, Nil) :: Nil)) :: _ /* likeTable */ :: - Some(query) :: Nil) = getClauses(Seq("TOK_TABNAME", "TOK_LIKETABLE", "TOK_QUERY"), children) + Some(query) :: Nil) = { + getClauses(Seq("TOK_TABNAME", "TOK_LIKETABLE", "TOK_QUERY"), children) + } InsertIntoCreatedTable(tableName, nodeToPlan(query)) // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. @@ -378,11 +380,26 @@ object HiveQl { sortByClause :: clusterByClause :: distributeByClause :: - limitClause :: Nil) = getClauses(Seq("TOK_INSERT_INTO", "TOK_DESTINATION", "TOK_SELECT", "TOK_SELECTDI", "TOK_WHERE", "TOK_GROUPBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY", "TOK_DISTRIBUTEBY", "TOK_LIMIT"), singleInsert) + limitClause :: Nil) = { + getClauses( + Seq( + "TOK_INSERT_INTO", + "TOK_DESTINATION", + "TOK_SELECT", + "TOK_SELECTDI", + "TOK_WHERE", + "TOK_GROUPBY", + "TOK_ORDERBY", + "TOK_SORTBY", + "TOK_CLUSTERBY", + "TOK_DISTRIBUTEBY", + "TOK_LIMIT"), + singleInsert) + } val relations = nodeToRelation(fromClause) val withWhere = whereClause.map { whereNode => - val Seq(whereExpr) = whereNode.getChildren().toSeq + val Seq(whereExpr) = whereNode.getChildren.toSeq Filter(nodeToExpr(whereExpr), relations) }.getOrElse(relations) @@ -396,8 +413,9 @@ object HiveQl { Token("TOK_TRANSFORM", Token("TOK_EXPLIST", inputExprs) :: Token("TOK_SERDE", Nil) :: - Token("TOK_RECORDWRITER", writerClause) :: // TODO: Need to support other types of (in/out)put - Token(script, Nil):: + Token("TOK_RECORDWRITER", writerClause) :: + // TODO: Need to support other types of (in/out)put + Token(script, Nil) :: Token("TOK_SERDE", serdeClause) :: Token("TOK_RECORDREADER", readerClause) :: outputClause :: Nil) :: Nil) => @@ -415,23 +433,22 @@ object HiveQl { case _ => None } - // The projection of the query can either be a normal projection, an aggregation (if there is a group by) or - // a script transformation. + // The projection of the query can either be a normal projection, an aggregation + // (if there is a group by) or a script transformation. val withProject = transformation.getOrElse { // Not a transformation so must be either project or aggregation. val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr)) groupByClause match { - case Some(groupBy) => Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withWhere) - case None => Project(selectExpressions, withWhere) + case Some(groupBy) => + Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withWhere) + case None => + Project(selectExpressions, withWhere) } } val withDistinct = - if(selectDistinctClause.isDefined) - Distinct(withProject) - else - withProject + if (selectDistinctClause.isDefined) Distinct(withProject) else withProject val withSort = (orderByClause, sortByClause, distributeByClause, clusterByClause) match { @@ -485,16 +502,22 @@ object HiveQl { case Token("TOK_TABREF", clauses) => // If the last clause is not a token then it's the alias of the table. val (nonAliasClauses, aliasClause) = - if(clauses.last.getText.startsWith("TOK")) + if (clauses.last.getText.startsWith("TOK")) { (clauses, None) - else + } else { (clauses.dropRight(1), Some(clauses.last)) + } val (Some(tableNameParts) :: splitSampleClause :: - bucketSampleClause :: Nil) = getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"), nonAliasClauses) + bucketSampleClause :: Nil) = { + getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"), + nonAliasClauses) + } - val tableName = tableNameParts.getChildren.map { case Token(part, Nil) => cleanIdentifier(part) }.mkString(".") + val tableName = tableNameParts.getChildren.map { case Token(part, Nil) => + cleanIdentifier(part) + }.mkString(".") val alias = aliasClause.map { case Token(a, Nil) => cleanIdentifier(a) } val relation = UnresolvedRelation(tableName, alias) @@ -507,7 +530,8 @@ object HiveQl { case Token("TOK_TABLEBUCKETSAMPLE", Token(numerator, Nil) :: Token(denominator, Nil) :: Nil) => - Sample(numerator.toDouble / denominator.toDouble, false, (math.random * 1000).toInt, relation) + val fraction = numerator.toDouble / denominator.toDouble + Sample(fraction, withReplacement = false, (math.random * 1000).toInt, relation) }.getOrElse(relation) case Token("TOK_UNIQUEJOIN", joinArgs) => @@ -634,8 +658,11 @@ object HiveQl { case plainIdent => plainIdent } - val numericAstTypes = - Seq(HiveParser.Number, HiveParser.TinyintLiteral, HiveParser.SmallintLiteral, HiveParser.BigintLiteral) + val numericAstTypes = Seq( + HiveParser.Number, + HiveParser.TinyintLiteral, + HiveParser.SmallintLiteral, + HiveParser.BigintLiteral) /* Case insensitive matches */ val COUNT = "(?i)COUNT".r @@ -655,16 +682,18 @@ object HiveQl { UnresolvedAttribute(cleanIdentifier(name)) case Token(".", qualifier :: Token(attr, Nil) :: Nil) => nodeToExpr(qualifier) match { - case UnresolvedAttribute(qualifierName) => UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)) - // The precidence for . seems to be wrong, so [] binds tighter an we need to go inside to find - // the underlying attribute references. + case UnresolvedAttribute(qualifierName) => + UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)) + // The precidence for . seems to be wrong, so [] binds tighter an we need to go inside to + // find the underlying attribute references. case GetItem(UnresolvedAttribute(qualifierName), ordinal) => GetItem(UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)), ordinal) } /* Stars (*) */ case Token("TOK_ALLCOLREF", Nil) => Star(None) - case Token("TOK_ALLCOLREF", Token("TOK_TABNAME", Token(name, Nil) :: Nil) :: Nil) => Star(Some(name)) + case Token("TOK_ALLCOLREF", Token("TOK_TABNAME", Token(name, Nil) :: Nil) :: Nil) => + Star(Some(name)) /* Aggregate Functions */ case Token("TOK_FUNCTION", Token(AVG(), Nil) :: arg :: Nil) => Average(nodeToExpr(arg)) @@ -674,15 +703,24 @@ object HiveQl { case Token("TOK_FUNCTION", Token(SUM(), Nil) :: arg :: Nil) => Sum(nodeToExpr(arg)) /* Casts */ - case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), IntegerType) - case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), LongType) - case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), FloatType) - case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), DoubleType) - case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ShortType) - case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ByteType) - case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BinaryType) - case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BooleanType) + case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), IntegerType) + case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), LongType) + case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), FloatType) + case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DoubleType) + case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), ShortType) + case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), ByteType) + case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), BinaryType) + case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), BooleanType) /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) @@ -701,12 +739,18 @@ object HiveQl { case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right)) - case Token("LIKE", left :: right:: Nil) => UnresolvedFunction("LIKE", Seq(nodeToExpr(left), nodeToExpr(right))) - case Token("RLIKE", left :: right:: Nil) => UnresolvedFunction("RLIKE", Seq(nodeToExpr(left), nodeToExpr(right))) - case Token("REGEXP", left :: right:: Nil) => UnresolvedFunction("REGEXP", Seq(nodeToExpr(left), nodeToExpr(right))) - case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => IsNotNull(nodeToExpr(child)) - case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => IsNull(nodeToExpr(child)) - case Token("TOK_FUNCTION", Token("IN", Nil) :: value :: list) => In(nodeToExpr(value), list.map(nodeToExpr)) + case Token("LIKE", left :: right:: Nil) => + UnresolvedFunction("LIKE", Seq(nodeToExpr(left), nodeToExpr(right))) + case Token("RLIKE", left :: right:: Nil) => + UnresolvedFunction("RLIKE", Seq(nodeToExpr(left), nodeToExpr(right))) + case Token("REGEXP", left :: right:: Nil) => + UnresolvedFunction("REGEXP", Seq(nodeToExpr(left), nodeToExpr(right))) + case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => + IsNotNull(nodeToExpr(child)) + case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => + IsNull(nodeToExpr(child)) + case Token("TOK_FUNCTION", Token("IN", Nil) :: value :: list) => + In(nodeToExpr(value), list.map(nodeToExpr)) /* Boolean Logic */ case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) @@ -731,49 +775,53 @@ object HiveQl { case Token(TRUE(), Nil) => Literal(true, BooleanType) case Token(FALSE(), Nil) => Literal(false, BooleanType) case Token("TOK_STRINGLITERALSEQUENCE", strings) => - Literal(strings.map(s => BaseSemanticAnalyzer.unescapeSQLString(s.asInstanceOf[ASTNode].getText)).mkString) + Literal(strings.map(s => BaseSemanticAnalyzer.unescapeSQLString(s.getText)).mkString) - // This code is adapted from https://github.com/apache/hive/blob/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java#L223 - case ast: ASTNode if numericAstTypes contains ast.getType() => + // This code is adapted from + // /ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java#L223 + case ast: ASTNode if numericAstTypes contains ast.getType => var v: Literal = null try { - if (ast.getText().endsWith("L")) { + if (ast.getText.endsWith("L")) { // Literal bigint. - v = Literal(ast.getText().substring(0, ast.getText().length() - 1).toLong, LongType) - } else if (ast.getText().endsWith("S")) { + v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toLong, LongType) + } else if (ast.getText.endsWith("S")) { // Literal smallint. - v = Literal(ast.getText().substring(0, ast.getText().length() - 1).toShort, ShortType) - } else if (ast.getText().endsWith("Y")) { + v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toShort, ShortType) + } 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")) { + v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType) + } else if (ast.getText.endsWith("BD")) { // Literal decimal - val strVal = ast.getText().substring(0, ast.getText().length() - 2); + val strVal = ast.getText.substring(0, ast.getText.length() - 2) BigDecimal(strVal) } else { - v = Literal(ast.getText().toDouble, DoubleType) - v = Literal(ast.getText().toLong, LongType) - v = Literal(ast.getText().toInt, IntegerType) + v = Literal(ast.getText.toDouble, DoubleType) + v = Literal(ast.getText.toLong, LongType) + v = Literal(ast.getText.toInt, IntegerType) } } catch { case nfe: NumberFormatException => // Do nothing } - if (v == null) + if (v == null) { sys.error(s"Failed to parse number ${ast.getText}") - else + } else { v + } case ast: ASTNode if ast.getType == HiveParser.StringLiteral => Literal(BaseSemanticAnalyzer.unescapeSQLString(ast.getText)) case a: ASTNode => throw new NotImplementedError( - s"No parse rules for ASTNode type: ${a.getType}, text: ${a.getText} :\n ${dumpTree(a).toString}") + s"""No parse rules for ASTNode type: ${a.getType}, text: ${a.getText} : + |${dumpTree(a).toString}" + + """.stripMargin) } def dumpTree(node: Node, builder: StringBuilder = new StringBuilder, indent: Int = 0) - : StringBuilder = { + : StringBuilder = { node match { case a: ASTNode => builder.append((" " * indent) + a.getText + "\n") case other => sys.error(s"Non ASTNode encountered: $other") @@ -782,4 +830,4 @@ object HiveQl { Option(node.getChildren).map(_.toList).getOrElse(Nil).foreach(dumpTree(_, builder, indent + 1)) builder } -} \ No newline at end of file +} From 5c1e60043c4b60529936f93a1536d021f28a2460 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 Jan 2014 16:18:55 -0800 Subject: [PATCH 573/778] Added hash code implementation for AttributeReference --- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 3 ++- .../scala/catalyst/expressions/namedExpressions.scala | 8 ++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 48fda0e4f4372..f5f7325885e71 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -2,8 +2,9 @@ package catalyst package execution import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table, StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.metastore.HiveMetaStoreClient +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table} +import org.apache.hadoop.hive.metastore.api.{StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.AbstractDeserializer diff --git a/src/main/scala/catalyst/expressions/namedExpressions.scala b/src/main/scala/catalyst/expressions/namedExpressions.scala index 2ba63d4042d5b..40766d999b509 100644 --- a/src/main/scala/catalyst/expressions/namedExpressions.scala +++ b/src/main/scala/catalyst/expressions/namedExpressions.scala @@ -98,6 +98,14 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea case _ => false } + override def hashCode: Int = { + // See http://stackoverflow.com/questions/113511/hash-code-implementation + var h = 17 + h = h * 37 + exprId.hashCode() + h = h * 37 + dataType.hashCode() + h + } + def newInstance = AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) /** From 7e24436da3de67e3b33c310d0c761b2c8e3d11bd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 Jan 2014 16:34:59 -0800 Subject: [PATCH 574/778] Removed dependency on JDK 7 (nio.file). --- .../catalyst/execution/hiveOperators.scala | 26 +++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index ef1f88668b675..0ba73f2404fec 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -1,7 +1,8 @@ package catalyst package execution -import java.nio.file.Files +import java.io.{File, IOException} +import java.util.UUID import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.common.`type`.HiveVarchar @@ -182,8 +183,29 @@ case class InsertIntoHiveTable( val childRdd = child.execute() assert(childRdd != null) + /** Create a temporary directory inside the given parent directory */ + def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { + var attempts = 0 + val maxAttempts = 10 + var dir: File = null + while (dir == null) { + attempts += 1 + if (attempts > maxAttempts) { + throw new IOException("Failed to create a temp directory (under " + root + ") after " + + maxAttempts + " attempts!") + } + try { + dir = new File(root, "spark-" + UUID.randomUUID.toString) + if (dir.exists() || !dir.mkdirs()) { + dir = null + } + } catch { case e: IOException => ; } + } + dir + } + // TODO write directly to Hive - val tempDir = Files.createTempDirectory("data").toFile + val tempDir = createTempDir() // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer // instances within the closure, since AbstractSerDe is not serializable while TableDesc is. From f47c2f6f3572cb15da916c0efab7839e485ec905 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 31 Jan 2014 14:32:00 +0800 Subject: [PATCH 575/778] set outputPartitioning in BroadcastNestedLoopJoin --- src/main/scala/catalyst/execution/joins.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index fc9e38df7f995..39a172b666f69 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -68,7 +68,9 @@ case class BroadcastNestedLoopJoin( streamed: SharkPlan, broadcast: SharkPlan, joinType: JoinType, condition: Option[Expression]) (@transient sc: SharkContext) extends BinaryNode { - // TODO: Override requiredChildDistribution and outputPartitioning. + // TODO: Override requiredChildDistribution. + + override def outputPartitioning: Partitioning = streamed.outputPartitioning override def otherCopyArgs = sc :: Nil From d91e276fb303a878bb54ba156a3087c204f0e167 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 13:40:59 -0800 Subject: [PATCH 576/778] Remove dependence on HIVE_HOME for running tests. This was done by moving all the hive query test (from branch-0.12) and data files into src/test/hive. These are used by default when HIVE_HOME is not set. --- Makefile | 2 + .../scala/catalyst/execution/TestShark.scala | 36 +- src/test/hive/data/conf/hive-log4j.properties | 78 + src/test/hive/data/conf/hive-site.xml | 197 + src/test/hive/data/files/SortCol1Col2.txt | 4 + src/test/hive/data/files/SortCol2Col1.txt | 5 + src/test/hive/data/files/SortDescCol1Col2.txt | 4 + src/test/hive/data/files/SortDescCol2Col1.txt | 4 + src/test/hive/data/files/T1.txt | 6 + src/test/hive/data/files/T2.txt | 6 + src/test/hive/data/files/T3.txt | 4 + src/test/hive/data/files/TestSerDe.jar | Bin 0 -> 3838 bytes src/test/hive/data/files/UserVisits.dat | 55 + src/test/hive/data/files/apache.access.2.log | 1 + src/test/hive/data/files/apache.access.log | 1 + src/test/hive/data/files/archive_corrupt.rc | Bin 0 -> 1374 bytes src/test/hive/data/files/array_table.txt | 2 + src/test/hive/data/files/binary.txt | 10 + src/test/hive/data/files/bool.txt | 33 + src/test/hive/data/files/complex.seq | Bin 0 -> 1606 bytes src/test/hive/data/files/covar_tab.txt | 6 + .../hive/data/files/create_nested_type.txt | 4 + src/test/hive/data/files/csv.txt | 18 + src/test/hive/data/files/datatypes.txt | 3 + src/test/hive/data/files/dim-data.txt | 4 + src/test/hive/data/files/doctors.avro | Bin 0 -> 521 bytes src/test/hive/data/files/docurl.txt | 8 + src/test/hive/data/files/double.txt | 16 + src/test/hive/data/files/employee.dat | 13 + src/test/hive/data/files/employee2.dat | 7 + src/test/hive/data/files/employee_part.txt | 9 + src/test/hive/data/files/empty1.txt | 0 src/test/hive/data/files/empty2.txt | 0 src/test/hive/data/files/episodes.avro | Bin 0 -> 597 bytes src/test/hive/data/files/escapetest.txt | Bin 0 -> 254 bytes src/test/hive/data/files/ext_test/test.dat | 6 + src/test/hive/data/files/fact-data.txt | 10 + src/test/hive/data/files/flights_join.txt | 20 + src/test/hive/data/files/flights_tiny.txt | 137 + src/test/hive/data/files/flights_tiny.txt.1 | 137 + .../hive/data/files/groupby_groupingid.txt | 6 + src/test/hive/data/files/grouping_sets.txt | 6 + src/test/hive/data/files/grouping_sets1.txt | 6 + src/test/hive/data/files/grouping_sets2.txt | 6 + src/test/hive/data/files/hive_626_bar.txt | 1 + src/test/hive/data/files/hive_626_count.txt | 1 + src/test/hive/data/files/hive_626_foo.txt | 1 + src/test/hive/data/files/in1.txt | 3 + src/test/hive/data/files/in2.txt | 3 + src/test/hive/data/files/in3.txt | 4 + src/test/hive/data/files/in4.txt | 7 + src/test/hive/data/files/in5.txt | 19 + src/test/hive/data/files/in6.txt | 19 + src/test/hive/data/files/in7.txt | 3 + src/test/hive/data/files/in8.txt | 6 + src/test/hive/data/files/in9.txt | 6 + src/test/hive/data/files/infer_const_type.txt | 3 + src/test/hive/data/files/int.txt | 12 + src/test/hive/data/files/json.txt | 1 + src/test/hive/data/files/kv1.seq | Bin 0 -> 10508 bytes .../hive/data/files/kv1.string-sorted.txt | 500 + src/test/hive/data/files/kv1.txt | 500 + src/test/hive/data/files/kv1.val.sorted.txt | 500 + src/test/hive/data/files/kv1_broken.seq | Bin 0 -> 216 bytes src/test/hive/data/files/kv1_cb.txt | 500 + src/test/hive/data/files/kv1_cc.txt | 500 + src/test/hive/data/files/kv1kv2.cogroup.txt | 1000 ++ src/test/hive/data/files/kv2.txt | 500 + src/test/hive/data/files/kv3.txt | 25 + src/test/hive/data/files/kv4.txt | 1 + src/test/hive/data/files/kv5.txt | 24 + src/test/hive/data/files/kv6.txt | 100 + src/test/hive/data/files/kv7.txt | 38 + src/test/hive/data/files/kv8.txt | 75 + .../hive/data/files/leftsemijoin_mr_t1.txt | 2 + .../hive/data/files/leftsemijoin_mr_t2.txt | 20 + src/test/hive/data/files/lineitem.txt | 100 + src/test/hive/data/files/lt100.sorted.txt | 84 + src/test/hive/data/files/lt100.txt | 84 + src/test/hive/data/files/lt100.txt.deflate | 3 + src/test/hive/data/files/map_table.txt | 2 + src/test/hive/data/files/nested_complex.txt | 2 + src/test/hive/data/files/null.txt | 10 + src/test/hive/data/files/nullfile.txt | 0 src/test/hive/data/files/nulls.txt | 1002 ++ src/test/hive/data/files/orc_create.txt | 3 + .../hive/data/files/orc_create_people.txt | 100 + src/test/hive/data/files/over10k | 9999 +++++++++++++++++ src/test/hive/data/files/over1k | 1049 ++ src/test/hive/data/files/part.rc | Bin 0 -> 3214 bytes src/test/hive/data/files/part.seq | Bin 0 -> 3598 bytes src/test/hive/data/files/part_tiny.txt | 26 + src/test/hive/data/files/person age.txt | 5 + .../hive/data/files/primitive_type_arrays.txt | 1 + src/test/hive/data/files/pw17.txt | 4 + src/test/hive/data/files/sales.txt | 2 + .../files/sample-queryplan-in-history.txt | 1 + src/test/hive/data/files/sample-queryplan.txt | 1 + .../data/files/smallsrcsortbucket1outof4.txt | 5 + .../data/files/smallsrcsortbucket2outof4.txt | 5 + .../data/files/smallsrcsortbucket3outof4.txt | 5 + .../data/files/smallsrcsortbucket4outof4.txt | 5 + src/test/hive/data/files/smb_bucket_input.rc | Bin 0 -> 253 bytes src/test/hive/data/files/smb_bucket_input.txt | 7 + src/test/hive/data/files/smbbucket_1.rc | Bin 0 -> 208 bytes src/test/hive/data/files/smbbucket_1.txt | 5 + src/test/hive/data/files/smbbucket_2.rc | Bin 0 -> 206 bytes src/test/hive/data/files/smbbucket_2.txt | 4 + src/test/hive/data/files/smbbucket_3.rc | Bin 0 -> 222 bytes src/test/hive/data/files/smbbucket_3.txt | 6 + src/test/hive/data/files/source.txt | 4 + src/test/hive/data/files/srcbucket0.txt | 493 + src/test/hive/data/files/srcbucket1.txt | 507 + src/test/hive/data/files/srcbucket20.txt | 118 + src/test/hive/data/files/srcbucket21.txt | 120 + src/test/hive/data/files/srcbucket22.txt | 124 + src/test/hive/data/files/srcbucket23.txt | 138 + .../hive/data/files/srcsortbucket1outof4.txt | 118 + .../hive/data/files/srcsortbucket2outof4.txt | 120 + .../hive/data/files/srcsortbucket3outof4.txt | 124 + .../hive/data/files/srcsortbucket4outof4.txt | 138 + src/test/hive/data/files/string.txt | Bin 0 -> 93 bytes src/test/hive/data/files/symlink1.txt | 2 + src/test/hive/data/files/symlink2.txt | 1 + src/test/hive/data/files/tbl.txt | 10 + src/test/hive/data/files/test.dat | 6 + src/test/hive/data/files/test2.dat | 6 + src/test/hive/data/files/text-en.txt | 95 + src/test/hive/data/files/things.txt | 1 + src/test/hive/data/files/things2.txt | 1 + src/test/hive/data/files/tiny_a.txt | 1 + src/test/hive/data/files/tiny_b.txt | 2 + .../data/files/types/primitives/090101.txt | 25 + .../data/files/types/primitives/090201.txt | 25 + .../data/files/types/primitives/090301.txt | 25 + .../data/files/types/primitives/090401.txt | 25 + src/test/hive/data/files/union_input.txt | 8 + src/test/hive/data/files/v1.txt | 4 + src/test/hive/data/files/v2.txt | 9 + src/test/hive/data/files/vc1.txt | 3 + src/test/hive/data/files/x.txt | 2 + src/test/hive/data/files/y.txt | 1 + src/test/hive/data/files/z.txt | 1 + src/test/hive/data/metadb/.gitignore | 1 + src/test/hive/data/scripts/cat.py | 29 + src/test/hive/data/scripts/cat_error.py | 24 + .../hive/data/scripts/doubleescapedtab.py | 24 + src/test/hive/data/scripts/dumpdata_script.py | 27 + src/test/hive/data/scripts/error_script | 26 + .../data/scripts/escapedcarriagereturn.py | 23 + src/test/hive/data/scripts/escapednewline.py | 23 + src/test/hive/data/scripts/escapedtab.py | 23 + src/test/hive/data/scripts/input20_script | 20 + src/test/hive/data/scripts/newline.py | 24 + src/test/hive/data/scripts/q_test_cleanup.sql | 10 + src/test/hive/data/scripts/q_test_init.sql | 132 + src/test/hive/data/scripts/test_init_file.sql | 1 + .../add_partition_with_whitelist.q | 8 + .../test/queries/clientnegative/addpart1.q | 11 + .../alter_concatenate_indexed_table.q | 16 + .../queries/clientnegative/alter_non_native.q | 6 + .../alter_partition_coltype_2columns.q | 11 + .../alter_partition_coltype_invalidcolname.q | 12 + .../alter_partition_coltype_invalidtype.q | 11 + .../alter_partition_invalidspec.q | 8 + .../clientnegative/alter_partition_nodrop.q | 9 + .../alter_partition_nodrop_table.q | 9 + .../clientnegative/alter_partition_offline.q | 11 + .../alter_partition_with_whitelist.q | 9 + .../alter_rename_partition_failure.q | 6 + .../alter_rename_partition_failure2.q | 6 + .../alter_rename_partition_failure3.q | 6 + .../alter_table_add_partition.q | 5 + .../clientnegative/alter_table_wrong_regex.q | 7 + .../alter_view_as_select_not_exist.q | 4 + .../alter_view_as_select_with_partition.q | 12 + .../clientnegative/alter_view_failure.q | 3 + .../clientnegative/alter_view_failure2.q | 8 + .../clientnegative/alter_view_failure3.q | 2 + .../clientnegative/alter_view_failure4.q | 8 + .../clientnegative/alter_view_failure5.q | 8 + .../clientnegative/alter_view_failure6.q | 11 + .../clientnegative/alter_view_failure7.q | 8 + .../clientnegative/alter_view_failure8.q | 3 + .../clientnegative/alter_view_failure9.q | 7 + .../src/test/queries/clientnegative/altern1.q | 4 + .../queries/clientnegative/ambiguous_col.q | 1 + .../queries/clientnegative/ambiguous_col0.q | 2 + .../queries/clientnegative/ambiguous_col1.q | 2 + .../queries/clientnegative/ambiguous_col2.q | 2 + .../src/test/queries/clientnegative/analyze.q | 1 + .../test/queries/clientnegative/analyze1.q | 1 + .../queries/clientnegative/analyze_view.q | 6 + .../test/queries/clientnegative/archive1.q | 11 + .../test/queries/clientnegative/archive2.q | 10 + .../test/queries/clientnegative/archive3.q | 5 + .../test/queries/clientnegative/archive4.q | 5 + .../test/queries/clientnegative/archive5.q | 5 + .../queries/clientnegative/archive_corrupt.q | 18 + .../queries/clientnegative/archive_insert1.q | 13 + .../queries/clientnegative/archive_insert2.q | 13 + .../queries/clientnegative/archive_insert3.q | 13 + .../queries/clientnegative/archive_insert4.q | 15 + .../queries/clientnegative/archive_multi1.q | 13 + .../queries/clientnegative/archive_multi2.q | 12 + .../queries/clientnegative/archive_multi3.q | 13 + .../queries/clientnegative/archive_multi4.q | 13 + .../queries/clientnegative/archive_multi5.q | 13 + .../queries/clientnegative/archive_multi6.q | 13 + .../queries/clientnegative/archive_multi7.q | 12 + .../clientnegative/archive_partspec1.q | 10 + .../clientnegative/archive_partspec2.q | 10 + .../clientnegative/archive_partspec3.q | 10 + .../clientnegative/archive_partspec4.q | 10 + .../clientnegative/archive_partspec5.q | 10 + .../clientnegative/authorization_fail_1.q | 7 + .../clientnegative/authorization_fail_2.q | 7 + .../clientnegative/authorization_fail_3.q | 12 + .../clientnegative/authorization_fail_4.q | 15 + .../clientnegative/authorization_fail_5.q | 20 + .../clientnegative/authorization_fail_6.q | 6 + .../clientnegative/authorization_fail_7.q | 17 + .../clientnegative/authorization_part.q | 37 + .../test/queries/clientnegative/autolocal1.q | 15 + .../queries/clientnegative/bad_exec_hooks.q | 6 + .../queries/clientnegative/bad_indextype.q | 1 + .../clientnegative/bad_sample_clause.q | 6 + .../clientnegative/bucket_mapjoin_mismatch1.q | 42 + .../bucket_mapjoin_wrong_table_metadata_1.q | 20 + .../bucket_mapjoin_wrong_table_metadata_2.q | 24 + .../clientnegative/cachingprintstream.q | 8 + .../cluster_tasklog_retrieval.q | 6 + .../clientnegative/clusterbydistributeby.q | 8 + .../queries/clientnegative/clusterbyorderby.q | 5 + .../queries/clientnegative/clusterbysortby.q | 8 + .../test/queries/clientnegative/clustern1.q | 2 + .../test/queries/clientnegative/clustern2.q | 3 + .../test/queries/clientnegative/clustern3.q | 2 + .../test/queries/clientnegative/clustern4.q | 2 + .../column_change_skewedcol_type1.q | 5 + .../queries/clientnegative/column_rename1.q | 6 + .../queries/clientnegative/column_rename2.q | 6 + .../queries/clientnegative/column_rename3.q | 1 + .../queries/clientnegative/column_rename4.q | 6 + .../queries/clientnegative/column_rename5.q | 6 + .../clientnegative/columnstats_partlvl_dp.q | 16 + .../columnstats_partlvl_incorrect_num_keys.q | 16 + .../columnstats_partlvl_invalid_values.q | 16 + ...columnstats_partlvl_multiple_part_clause.q | 16 + .../clientnegative/columnstats_tbllvl.q | 22 + .../columnstats_tbllvl_complex_type.q | 17 + .../columnstats_tbllvl_incorrect_column.q | 22 + .../clientnegative/compare_double_bigint.q | 5 + .../clientnegative/compare_string_bigint.q | 5 + .../create_insert_outputformat.q | 11 + .../clientnegative/create_or_replace_view1.q | 6 + .../clientnegative/create_or_replace_view2.q | 6 + .../clientnegative/create_or_replace_view3.q | 3 + .../clientnegative/create_or_replace_view4.q | 5 + .../clientnegative/create_or_replace_view5.q | 5 + .../clientnegative/create_or_replace_view6.q | 5 + .../clientnegative/create_or_replace_view7.q | 7 + .../clientnegative/create_or_replace_view8.q | 5 + ..._skewed_table_col_name_value_no_mismatch.q | 3 + .../create_skewed_table_dup_col_name.q | 3 + ...te_skewed_table_failure_invalid_col_name.q | 4 + .../clientnegative/create_table_failure1.q | 1 + .../clientnegative/create_table_failure2.q | 1 + .../clientnegative/create_table_failure3.q | 1 + .../clientnegative/create_table_failure4.q | 1 + .../clientnegative/create_table_wrong_regex.q | 4 + .../clientnegative/create_udaf_failure.q | 6 + .../create_unknown_genericudf.q | 1 + .../clientnegative/create_unknown_udf_udaf.q | 1 + .../clientnegative/create_view_failure1.q | 6 + .../clientnegative/create_view_failure2.q | 6 + .../clientnegative/create_view_failure3.q | 5 + .../clientnegative/create_view_failure4.q | 5 + .../clientnegative/create_view_failure5.q | 9 + .../clientnegative/create_view_failure6.q | 6 + .../clientnegative/create_view_failure7.q | 6 + .../clientnegative/create_view_failure8.q | 6 + .../clientnegative/create_view_failure9.q | 6 + .../ql/src/test/queries/clientnegative/ctas.q | 5 + .../database_create_already_exists.q | 5 + .../database_create_invalid_name.q | 4 + .../database_drop_does_not_exist.q | 4 + .../clientnegative/database_drop_not_empty.q | 8 + .../database_drop_not_empty_restrict.q | 8 + .../database_switch_does_not_exist.q | 4 + .../queries/clientnegative/date_literal1.q | 2 + .../queries/clientnegative/date_literal2.q | 2 + .../queries/clientnegative/date_literal3.q | 2 + .../src/test/queries/clientnegative/ddltime.q | 6 + .../clientnegative/decimal_precision.q | 10 + .../clientnegative/decimal_precision_1.q | 10 + .../clientnegative/default_partition_name.q | 3 + .../test/queries/clientnegative/deletejar.q | 4 + .../queries/clientnegative/desc_failure1.q | 1 + .../queries/clientnegative/desc_failure2.q | 2 + .../queries/clientnegative/desc_failure3.q | 5 + .../queries/clientnegative/describe_xpath1.q | 1 + .../queries/clientnegative/describe_xpath2.q | 1 + .../queries/clientnegative/describe_xpath3.q | 1 + .../queries/clientnegative/describe_xpath4.q | 1 + .../disallow_incompatible_type_change_on1.q | 17 + .../disallow_incompatible_type_change_on2.q | 6 + .../clientnegative/drop_function_failure.q | 3 + .../clientnegative/drop_index_failure.q | 3 + .../queries/clientnegative/drop_native_udf.q | 1 + .../clientnegative/drop_partition_failure.q | 11 + .../drop_partition_filter_failure.q | 8 + .../drop_partition_filter_failure2.q | 11 + .../clientnegative/drop_table_failure1.q | 3 + .../clientnegative/drop_table_failure2.q | 3 + .../clientnegative/drop_table_failure3.q | 12 + .../clientnegative/drop_view_failure1.q | 6 + .../clientnegative/drop_view_failure2.q | 3 + .../duplicate_alias_in_transform.q | 1 + .../duplicate_alias_in_transform_schema.q | 1 + .../clientnegative/duplicate_insert1.q | 7 + .../clientnegative/duplicate_insert2.q | 6 + .../clientnegative/duplicate_insert3.q | 4 + .../test/queries/clientnegative/dyn_part1.q | 11 + .../test/queries/clientnegative/dyn_part2.q | 11 + .../test/queries/clientnegative/dyn_part3.q | 9 + .../test/queries/clientnegative/dyn_part4.q | 7 + .../clientnegative/dyn_part_empty.q.disabled | 24 + .../queries/clientnegative/dyn_part_max.q | 16 + .../clientnegative/dyn_part_max_per_node.q | 15 + .../dynamic_partitions_with_whitelist.q | 17 + ...hange_partition_neg_incomplete_partition.q | 12 + .../exchange_partition_neg_partition_exists.q | 12 + ...exchange_partition_neg_partition_exists2.q | 13 + ...exchange_partition_neg_partition_exists3.q | 13 + ...exchange_partition_neg_partition_missing.q | 6 + .../exchange_partition_neg_table_missing.q | 2 + .../exchange_partition_neg_table_missing2.q | 8 + .../exchange_partition_neg_test.q | 11 + .../exim_00_unsupported_schema.q | 12 + .../exim_01_nonpart_over_loaded.q | 24 + .../exim_02_all_part_over_overlap.q | 38 + .../exim_03_nonpart_noncompat_colschema.q | 23 + .../exim_04_nonpart_noncompat_colnumber.q | 23 + .../exim_05_nonpart_noncompat_coltype.q | 23 + .../exim_06_nonpart_noncompat_storage.q | 23 + .../exim_07_nonpart_noncompat_ifof.q | 26 + .../exim_08_nonpart_noncompat_serde.q | 24 + .../exim_09_nonpart_noncompat_serdeparam.q | 28 + .../exim_10_nonpart_noncompat_bucketing.q | 24 + .../exim_11_nonpart_noncompat_sorting.q | 25 + .../clientnegative/exim_12_nonnative_export.q | 9 + .../clientnegative/exim_13_nonnative_import.q | 24 + .../clientnegative/exim_14_nonpart_part.q | 25 + .../clientnegative/exim_15_part_nonpart.q | 25 + .../exim_16_part_noncompat_schema.q | 26 + .../exim_17_part_spec_underspec.q | 30 + .../exim_18_part_spec_missing.q | 30 + .../exim_19_external_over_existing.q | 23 + .../exim_20_managed_location_over_existing.q | 30 + .../exim_21_part_managed_external.q | 35 + .../clientnegative/exim_22_export_authfail.q | 14 + .../exim_23_import_exist_authfail.q | 22 + .../exim_24_import_part_authfail.q | 31 + .../exim_25_import_nonexist_authfail.q | 23 + .../test/queries/clientnegative/external1.q | 3 + .../test/queries/clientnegative/external2.q | 4 + .../clientnegative/fetchtask_ioexception.q | 7 + .../clientnegative/fileformat_bad_class.q | 3 + .../clientnegative/fileformat_void_input.q | 8 + .../clientnegative/fileformat_void_output.q | 6 + .../queries/clientnegative/fs_default_name1.q | 2 + .../queries/clientnegative/fs_default_name2.q | 2 + .../clientnegative/genericFileFormat.q | 1 + .../groupby2_map_skew_multi_distinct.q | 14 + .../clientnegative/groupby2_multi_distinct.q | 13 + .../groupby3_map_skew_multi_distinct.q | 36 + .../clientnegative/groupby3_multi_distinct.q | 36 + .../queries/clientnegative/groupby_cube1.q | 4 + .../queries/clientnegative/groupby_cube2.q | 4 + .../clientnegative/groupby_grouping_id1.q | 4 + .../clientnegative/groupby_grouping_sets1.q | 5 + .../clientnegative/groupby_grouping_sets2.q | 4 + .../clientnegative/groupby_grouping_sets3.q | 4 + .../clientnegative/groupby_grouping_sets4.q | 4 + .../clientnegative/groupby_grouping_sets5.q | 5 + .../clientnegative/groupby_grouping_sets6.q | 8 + .../clientnegative/groupby_grouping_sets7.q | 10 + .../clientnegative/groupby_invalid_position.q | 4 + .../test/queries/clientnegative/groupby_key.q | 1 + .../queries/clientnegative/groupby_rollup1.q | 4 + .../queries/clientnegative/groupby_rollup2.q | 4 + .../src/test/queries/clientnegative/having1.q | 2 + .../clientnegative/illegal_partition_type.q | 7 + .../clientnegative/illegal_partition_type2.q | 3 + .../clientnegative/index_bitmap_no_map_aggr.q | 7 + .../index_compact_entry_limit.q | 11 + .../clientnegative/index_compact_size_limit.q | 12 + .../src/test/queries/clientnegative/input1.q | 1 + .../src/test/queries/clientnegative/input2.q | 1 + .../src/test/queries/clientnegative/input4.q | 5 + .../src/test/queries/clientnegative/input41.q | 5 + .../queries/clientnegative/input_part0_neg.q | 3 + .../queries/clientnegative/insert_into1.q | 11 + .../queries/clientnegative/insert_into2.q | 10 + .../queries/clientnegative/insert_into3.q | 16 + .../queries/clientnegative/insert_into4.q | 16 + .../clientnegative/insert_view_failure.q | 5 + .../queries/clientnegative/insertexternal1.q | 8 + .../insertover_dynapart_ifnotexists.q | 9 + .../clientnegative/invalid_arithmetic_type.q | 3 + .../clientnegative/invalid_avg_syntax.q | 1 + .../invalid_cast_from_binary_1.q | 2 + .../invalid_cast_from_binary_2.q | 2 + .../invalid_cast_from_binary_3.q | 2 + .../invalid_cast_from_binary_4.q | 2 + .../invalid_cast_from_binary_5.q | 2 + .../invalid_cast_from_binary_6.q | 2 + .../clientnegative/invalid_cast_to_binary_1.q | 1 + .../clientnegative/invalid_cast_to_binary_2.q | 1 + .../clientnegative/invalid_cast_to_binary_3.q | 1 + .../clientnegative/invalid_cast_to_binary_4.q | 1 + .../clientnegative/invalid_cast_to_binary_5.q | 1 + .../clientnegative/invalid_cast_to_binary_6.q | 1 + .../queries/clientnegative/invalid_columns.q | 4 + .../queries/clientnegative/invalid_config1.q | 3 + .../queries/clientnegative/invalid_config2.q | 4 + .../clientnegative/invalid_create_tbl1.q | 9 + .../clientnegative/invalid_create_tbl2.q | 1 + .../queries/clientnegative/invalid_mapjoin1.q | 1 + .../clientnegative/invalid_max_syntax.q | 1 + .../clientnegative/invalid_min_syntax.q | 1 + .../clientnegative/invalid_select_column.q | 4 + .../invalid_select_column_with_subquery.q | 4 + .../invalid_select_column_with_tablename.q | 4 + .../invalid_select_expression.q | 1 + .../clientnegative/invalid_std_syntax.q | 1 + .../invalid_stddev_samp_syntax.q | 1 + .../clientnegative/invalid_sum_syntax.q | 1 + .../queries/clientnegative/invalid_t_alter1.q | 2 + .../queries/clientnegative/invalid_t_alter2.q | 2 + .../clientnegative/invalid_t_create2.q | 1 + .../clientnegative/invalid_t_transform.q | 1 + .../queries/clientnegative/invalid_tbl_name.q | 1 + .../clientnegative/invalid_var_samp_syntax.q | 1 + .../clientnegative/invalid_varchar_length_1.q | 2 + .../clientnegative/invalid_varchar_length_2.q | 1 + .../clientnegative/invalid_varchar_length_3.q | 3 + .../clientnegative/invalid_variance_syntax.q | 1 + .../queries/clientnegative/invalidate_view1.q | 11 + .../src/test/queries/clientnegative/join2.q | 5 + .../src/test/queries/clientnegative/join28.q | 15 + .../src/test/queries/clientnegative/join29.q | 10 + .../src/test/queries/clientnegative/join32.q | 14 + .../src/test/queries/clientnegative/join35.q | 18 + .../clientnegative/join_nonexistent_part.q | 4 + .../src/test/queries/clientnegative/joinneg.q | 6 + .../clientnegative/lateral_view_alias.q | 3 + .../clientnegative/lateral_view_join.q | 1 + .../queries/clientnegative/line_terminator.q | 3 + .../clientnegative/load_exist_part_authfail.q | 4 + .../queries/clientnegative/load_non_native.q | 5 + .../clientnegative/load_nonpart_authfail.q | 3 + .../clientnegative/load_part_authfail.q | 3 + .../queries/clientnegative/load_part_nospec.q | 2 + .../clientnegative/load_stored_as_dirs.q | 7 + .../clientnegative/load_view_failure.q | 3 + .../clientnegative/load_wrong_fileformat.q | 6 + .../load_wrong_fileformat_rc_seq.q | 6 + .../load_wrong_fileformat_txt_seq.q | 6 + .../clientnegative/load_wrong_noof_part.q | 3 + .../clientnegative/local_mapred_error_cache.q | 4 + .../test/queries/clientnegative/lockneg1.q | 10 + .../test/queries/clientnegative/lockneg2.q | 6 + .../test/queries/clientnegative/lockneg3.q | 9 + .../test/queries/clientnegative/lockneg4.q | 12 + .../test/queries/clientnegative/lockneg5.q | 2 + .../clientnegative/macro_unused_parameter.q | 1 + .../clientnegative/mapreduce_stack_trace.q | 13 + .../mapreduce_stack_trace_hadoop20.q | 13 + .../mapreduce_stack_trace_turnoff.q | 13 + .../mapreduce_stack_trace_turnoff_hadoop20.q | 13 + .../queries/clientnegative/merge_negative_1.q | 3 + .../queries/clientnegative/merge_negative_2.q | 3 + .../queries/clientnegative/merge_negative_3.q | 6 + .../clientnegative/minimr_broken_pipe.q | 4 + .../clientnegative/nested_complex_neg.q | 15 + .../queries/clientnegative/no_matching_udf.q | 1 + .../queries/clientnegative/nonkey_groupby.q | 1 + .../queries/clientnegative/nopart_insert.q | 7 + .../test/queries/clientnegative/nopart_load.q | 5 + .../queries/clientnegative/notable_alias3.q | 4 + .../queries/clientnegative/notable_alias4.q | 4 + .../clientnegative/orderby_invalid_position.q | 4 + .../orderby_position_unsupported.q | 4 + .../queries/clientnegative/orderbysortby.q | 8 + .../clientnegative/part_col_complex_type.q | 1 + .../queries/clientnegative/protectmode_part.q | 15 + .../clientnegative/protectmode_part1.q | 21 + .../clientnegative/protectmode_part2.q | 9 + .../clientnegative/protectmode_part_no_drop.q | 10 + .../queries/clientnegative/protectmode_tbl1.q | 8 + .../queries/clientnegative/protectmode_tbl2.q | 12 + .../queries/clientnegative/protectmode_tbl3.q | 10 + .../queries/clientnegative/protectmode_tbl4.q | 15 + .../queries/clientnegative/protectmode_tbl5.q | 15 + .../queries/clientnegative/protectmode_tbl6.q | 8 + .../queries/clientnegative/protectmode_tbl7.q | 13 + .../queries/clientnegative/protectmode_tbl8.q | 13 + .../clientnegative/protectmode_tbl_no_drop.q | 9 + ...ptf_negative_AggrFuncsWithNoGBYNoPartDef.q | 20 + .../ptf_negative_AmbiguousWindowDefn.q | 28 + .../ptf_negative_DistributeByOrderBy.q | 19 + .../ptf_negative_DuplicateWindowAlias.q | 22 + ..._negative_HavingLeadWithNoGBYNoWindowing.q | 20 + .../ptf_negative_HavingLeadWithPTF.q | 22 + .../ptf_negative_InvalidValueBoundary.q | 21 + .../ptf_negative_JoinWithAmbigousAlias.q | 20 + .../ptf_negative_PartitionBySortBy.q | 19 + .../ptf_negative_WhereWithRankCond.q | 21 + .../clientnegative/ptf_window_boundaries.q | 17 + .../clientnegative/ptf_window_boundaries2.q | 17 + .../queries/clientnegative/recursive_view.q | 15 + .../test/queries/clientnegative/regex_col_1.q | 2 + .../test/queries/clientnegative/regex_col_2.q | 2 + .../clientnegative/regex_col_groupby.q | 2 + .../queries/clientnegative/sa_fail_hook3.q | 4 + .../src/test/queries/clientnegative/sample.q | 1 + .../clientnegative/script_broken_pipe1.q | 3 + .../clientnegative/script_broken_pipe2.q | 3 + .../clientnegative/script_broken_pipe3.q | 3 + .../queries/clientnegative/script_error.q | 7 + .../clientnegative/select_charliteral.q | 3 + .../clientnegative/select_udtf_alias.q | 3 + .../test/queries/clientnegative/semijoin1.q | 2 + .../test/queries/clientnegative/semijoin2.q | 2 + .../test/queries/clientnegative/semijoin3.q | 2 + .../test/queries/clientnegative/semijoin4.q | 3 + .../test/queries/clientnegative/serde_regex.q | 17 + .../queries/clientnegative/serde_regex2.q | 23 + .../queries/clientnegative/serde_regex3.q | 14 + .../clientnegative/set_hiveconf_validation0.q | 5 + .../clientnegative/set_hiveconf_validation1.q | 5 + .../clientnegative/set_table_property.q | 4 + .../queries/clientnegative/show_columns1.q | 2 + .../queries/clientnegative/show_columns2.q | 2 + .../queries/clientnegative/show_columns3.q | 7 + .../show_create_table_does_not_exist.q | 2 + .../clientnegative/show_create_table_index.q | 6 + .../queries/clientnegative/show_partitions1.q | 1 + .../clientnegative/show_tableproperties1.q | 1 + .../queries/clientnegative/show_tables_bad1.q | 1 + .../queries/clientnegative/show_tables_bad2.q | 1 + .../clientnegative/show_tables_bad_db1.q | 1 + .../clientnegative/show_tables_bad_db2.q | 1 + .../queries/clientnegative/show_tablestatus.q | 1 + .../show_tablestatus_not_existing_part.q | 1 + .../clientnegative/smb_bucketmapjoin.q | 23 + .../queries/clientnegative/smb_mapjoin_14.q | 38 + .../sortmerge_mapjoin_mismatch_1.q | 28 + .../split_sample_out_of_range.q | 3 + .../split_sample_wrong_format.q | 3 + .../split_sample_wrong_format2.q | 3 + .../clientnegative/stats_aggregator_error_1.q | 18 + .../clientnegative/stats_aggregator_error_2.q | 16 + .../clientnegative/stats_noscan_non_native.q | 6 + .../stats_partialscan_autogether.q | 31 + .../stats_partialscan_non_external.q | 5 + .../stats_partialscan_non_native.q | 6 + .../clientnegative/stats_partscan_norcfile.q | 12 + .../clientnegative/stats_publisher_error_1.q | 18 + .../clientnegative/stats_publisher_error_2.q | 16 + .../test/queries/clientnegative/strict_join.q | 3 + .../queries/clientnegative/strict_orderby.q | 7 + .../queries/clientnegative/strict_pruning.q | 6 + .../test/queries/clientnegative/subq_insert.q | 2 + .../src/test/queries/clientnegative/touch1.q | 1 + .../src/test/queries/clientnegative/touch2.q | 1 + .../clientnegative/truncate_bucketed_column.q | 7 + .../truncate_column_indexed_table.q | 9 + .../truncate_column_list_bucketing.q | 14 + .../clientnegative/truncate_column_seqfile.q | 7 + .../truncate_nonexistant_column.q | 7 + .../truncate_partition_column.q | 7 + .../truncate_partition_column2.q | 7 + .../clientnegative/truncate_table_failure1.q | 2 + .../clientnegative/truncate_table_failure2.q | 2 + .../clientnegative/truncate_table_failure3.q | 4 + .../clientnegative/truncate_table_failure4.q | 5 + .../clientnegative/udaf_invalid_place.q | 1 + .../udf_array_contains_wrong1.q | 2 + .../udf_array_contains_wrong2.q | 2 + .../queries/clientnegative/udf_assert_true.q | 7 + .../queries/clientnegative/udf_assert_true2.q | 2 + .../clientnegative/udf_case_type_wrong.q | 6 + .../clientnegative/udf_case_type_wrong2.q | 6 + .../clientnegative/udf_case_type_wrong3.q | 6 + .../queries/clientnegative/udf_coalesce.q | 1 + .../clientnegative/udf_concat_ws_wrong1.q | 2 + .../clientnegative/udf_concat_ws_wrong2.q | 2 + .../clientnegative/udf_concat_ws_wrong3.q | 2 + .../clientnegative/udf_elt_wrong_args_len.q | 1 + .../clientnegative/udf_elt_wrong_type.q | 3 + .../clientnegative/udf_field_wrong_args_len.q | 1 + .../clientnegative/udf_field_wrong_type.q | 3 + .../clientnegative/udf_format_number_wrong1.q | 2 + .../clientnegative/udf_format_number_wrong2.q | 2 + .../clientnegative/udf_format_number_wrong3.q | 2 + .../clientnegative/udf_format_number_wrong4.q | 2 + .../clientnegative/udf_format_number_wrong5.q | 2 + .../clientnegative/udf_format_number_wrong6.q | 2 + .../clientnegative/udf_format_number_wrong7.q | 2 + .../udf_function_does_not_implement_udf.q | 1 + .../queries/clientnegative/udf_if_not_bool.q | 1 + .../clientnegative/udf_if_wrong_args_len.q | 1 + .../src/test/queries/clientnegative/udf_in.q | 1 + .../clientnegative/udf_instr_wrong_args_len.q | 1 + .../clientnegative/udf_instr_wrong_type.q | 3 + .../udf_locate_wrong_args_len.q | 1 + .../clientnegative/udf_locate_wrong_type.q | 3 + .../clientnegative/udf_map_keys_arg_num.q | 1 + .../clientnegative/udf_map_keys_arg_type.q | 1 + .../clientnegative/udf_map_values_arg_num.q | 1 + .../clientnegative/udf_map_values_arg_type.q | 1 + .../src/test/queries/clientnegative/udf_max.q | 2 + .../src/test/queries/clientnegative/udf_min.q | 2 + .../clientnegative/udf_printf_wrong1.q | 2 + .../clientnegative/udf_printf_wrong2.q | 2 + .../clientnegative/udf_printf_wrong3.q | 2 + .../clientnegative/udf_printf_wrong4.q | 2 + .../queries/clientnegative/udf_reflect_neg.q | 9 + .../clientnegative/udf_size_wrong_args_len.q | 5 + .../clientnegative/udf_size_wrong_type.q | 1 + .../clientnegative/udf_sort_array_wrong1.q | 2 + .../clientnegative/udf_sort_array_wrong2.q | 2 + .../clientnegative/udf_sort_array_wrong3.q | 2 + .../queries/clientnegative/udf_test_error.q | 3 + .../clientnegative/udf_test_error_reduce.q | 11 + .../clientnegative/udf_when_type_wrong.q | 6 + .../clientnegative/udf_when_type_wrong2.q | 6 + .../clientnegative/udf_when_type_wrong3.q | 6 + .../src/test/queries/clientnegative/udfnull.q | 6 + .../udtf_explode_not_supported1.q | 1 + .../udtf_explode_not_supported2.q | 1 + .../udtf_explode_not_supported3.q | 1 + .../udtf_explode_not_supported4.q | 1 + .../clientnegative/udtf_invalid_place.q | 1 + .../clientnegative/udtf_not_supported1.q | 1 + .../clientnegative/udtf_not_supported3.q | 1 + .../src/test/queries/clientnegative/union.q | 4 + .../src/test/queries/clientnegative/union2.q | 13 + .../src/test/queries/clientnegative/union22.q | 26 + .../src/test/queries/clientnegative/union3.q | 5 + .../test/queries/clientnegative/uniquejoin.q | 3 + .../test/queries/clientnegative/uniquejoin2.q | 3 + .../test/queries/clientnegative/uniquejoin3.q | 3 + .../clientnegative/unset_table_property.q | 6 + .../clientnegative/unset_view_property.q | 6 + .../windowing_leadlag_in_udaf.q | 15 + .../clientnegative/windowing_ll_no_over.q | 17 + .../clientnegative/wrong_column_type.q | 4 + .../queries/clientpositive/add_part_exist.q | 37 + .../clientpositive/add_part_multiple.q | 24 + .../add_partition_no_whitelist.q | 7 + .../add_partition_with_whitelist.q | 9 + .../clientpositive/alias_casted_column.q | 5 + .../queries/clientpositive/allcolref_in_udf.q | 21 + .../src/test/queries/clientpositive/alter1.q | 71 + .../src/test/queries/clientpositive/alter2.q | 55 + .../src/test/queries/clientpositive/alter3.q | 56 + .../src/test/queries/clientpositive/alter4.q | 26 + .../src/test/queries/clientpositive/alter5.q | 45 + .../alter_concatenate_indexed_table.q | 48 + .../test/queries/clientpositive/alter_index.q | 11 + .../test/queries/clientpositive/alter_merge.q | 41 + .../queries/clientpositive/alter_merge_2.q | 19 + .../clientpositive/alter_merge_stats.q | 41 + .../alter_numbuckets_partitioned_table.q | 59 + .../alter_numbuckets_partitioned_table2.q | 84 + .../alter_partition_clusterby_sortby.q | 23 + .../clientpositive/alter_partition_coltype.q | 57 + .../alter_partition_format_loc.q | 32 + .../alter_partition_protect_mode.q | 26 + .../alter_partition_with_whitelist.q | 9 + .../clientpositive/alter_rename_partition.q | 41 + .../alter_rename_partition_authorization.q | 22 + .../clientpositive/alter_skewed_table.q | 32 + .../clientpositive/alter_table_not_sorted.q | 7 + .../clientpositive/alter_table_serde.q | 33 + .../clientpositive/alter_table_serde2.q | 20 + .../queries/clientpositive/alter_varchar1.q | 32 + .../queries/clientpositive/alter_varchar2.q | 22 + .../clientpositive/alter_view_as_select.q | 13 + .../clientpositive/alter_view_rename.q | 10 + .../queries/clientpositive/ambiguous_col.q | 6 + .../src/test/queries/clientpositive/archive.q | 69 + .../queries/clientpositive/archive_corrupt.q | 35 + .../clientpositive/archive_excludeHadoop20.q | 69 + .../queries/clientpositive/archive_multi.q | 42 + .../queries/clientpositive/authorization_1.q | 89 + .../queries/clientpositive/authorization_2.q | 111 + .../queries/clientpositive/authorization_3.q | 19 + .../queries/clientpositive/authorization_4.q | 13 + .../queries/clientpositive/authorization_5.q | 20 + .../queries/clientpositive/authorization_6.q | 42 + .../queries/clientpositive/authorization_7.q | 15 + .../queries/clientpositive/authorization_8.q | 8 + .../test/queries/clientpositive/auto_join0.q | 23 + .../test/queries/clientpositive/auto_join1.q | 12 + .../test/queries/clientpositive/auto_join10.q | 17 + .../test/queries/clientpositive/auto_join11.q | 16 + .../test/queries/clientpositive/auto_join12.q | 25 + .../test/queries/clientpositive/auto_join13.q | 23 + .../test/queries/clientpositive/auto_join14.q | 19 + .../clientpositive/auto_join14_hadoop20.q | 18 + .../test/queries/clientpositive/auto_join15.q | 19 + .../test/queries/clientpositive/auto_join16.q | 18 + .../test/queries/clientpositive/auto_join17.q | 14 + .../test/queries/clientpositive/auto_join18.q | 27 + .../auto_join18_multi_distinct.q | 29 + .../test/queries/clientpositive/auto_join19.q | 16 + .../test/queries/clientpositive/auto_join2.q | 13 + .../test/queries/clientpositive/auto_join20.q | 31 + .../test/queries/clientpositive/auto_join21.q | 5 + .../test/queries/clientpositive/auto_join22.q | 5 + .../test/queries/clientpositive/auto_join23.q | 6 + .../test/queries/clientpositive/auto_join24.q | 13 + .../test/queries/clientpositive/auto_join25.q | 30 + .../test/queries/clientpositive/auto_join26.q | 10 + .../test/queries/clientpositive/auto_join27.q | 25 + .../test/queries/clientpositive/auto_join28.q | 14 + .../test/queries/clientpositive/auto_join29.q | 45 + .../test/queries/clientpositive/auto_join3.q | 12 + .../test/queries/clientpositive/auto_join30.q | 151 + .../test/queries/clientpositive/auto_join31.q | 22 + .../test/queries/clientpositive/auto_join32.q | 78 + .../test/queries/clientpositive/auto_join4.q | 34 + .../test/queries/clientpositive/auto_join5.q | 34 + .../test/queries/clientpositive/auto_join6.q | 35 + .../test/queries/clientpositive/auto_join7.q | 46 + .../test/queries/clientpositive/auto_join8.q | 34 + .../test/queries/clientpositive/auto_join9.q | 14 + .../clientpositive/auto_join_filters.q | 82 + .../queries/clientpositive/auto_join_nulls.q | 29 + .../auto_join_reordering_values.q | 31 + .../clientpositive/auto_smb_mapjoin_14.q | 296 + .../clientpositive/auto_sortmerge_join_1.q | 36 + .../clientpositive/auto_sortmerge_join_10.q | 57 + .../clientpositive/auto_sortmerge_join_11.q | 36 + .../clientpositive/auto_sortmerge_join_12.q | 31 + .../clientpositive/auto_sortmerge_join_13.q | 81 + .../clientpositive/auto_sortmerge_join_14.q | 29 + .../clientpositive/auto_sortmerge_join_15.q | 23 + .../clientpositive/auto_sortmerge_join_2.q | 32 + .../clientpositive/auto_sortmerge_join_3.q | 31 + .../clientpositive/auto_sortmerge_join_4.q | 33 + .../clientpositive/auto_sortmerge_join_5.q | 28 + .../clientpositive/auto_sortmerge_join_6.q | 73 + .../clientpositive/auto_sortmerge_join_7.q | 36 + .../clientpositive/auto_sortmerge_join_8.q | 40 + .../clientpositive/auto_sortmerge_join_9.q | 536 + .../queries/clientpositive/autogen_colalias.q | 25 + .../clientpositive/avro_change_schema.q | 28 + .../clientpositive/avro_compression_enabled.q | 43 + .../clientpositive/avro_evolved_schemas.q | 42 + .../test/queries/clientpositive/avro_joins.q | 79 + .../clientpositive/avro_nullable_fields.q | 59 + .../queries/clientpositive/avro_partitioned.q | 73 + .../queries/clientpositive/avro_sanity_test.q | 36 + .../avro_schema_error_message.q | 11 + .../clientpositive/avro_schema_literal.q | 32 + .../test/queries/clientpositive/ba_table1.q | 13 + .../test/queries/clientpositive/ba_table2.q | 16 + .../test/queries/clientpositive/ba_table3.q | 13 + .../queries/clientpositive/ba_table_udfs.q | 36 + .../queries/clientpositive/ba_table_union.q | 14 + .../queries/clientpositive/binary_constant.q | 1 + .../clientpositive/binary_output_format.q | 39 + .../clientpositive/binary_table_bincolserde.q | 16 + .../clientpositive/binary_table_colserde.q | 15 + .../queries/clientpositive/binarysortable_1.q | 21 + .../src/test/queries/clientpositive/bucket1.q | 13 + .../src/test/queries/clientpositive/bucket2.q | 16 + .../src/test/queries/clientpositive/bucket3.q | 19 + .../src/test/queries/clientpositive/bucket4.q | 18 + .../src/test/queries/clientpositive/bucket5.q | 33 + .../queries/clientpositive/bucket_groupby.q | 75 + .../clientpositive/bucket_map_join_1.q | 26 + .../clientpositive/bucket_map_join_2.q | 26 + .../clientpositive/bucket_num_reducers.q | 16 + .../clientpositive/bucket_num_reducers2.q | 13 + .../queries/clientpositive/bucketcontext_1.q | 23 + .../queries/clientpositive/bucketcontext_2.q | 21 + .../queries/clientpositive/bucketcontext_3.q | 21 + .../queries/clientpositive/bucketcontext_4.q | 23 + .../queries/clientpositive/bucketcontext_5.q | 18 + .../queries/clientpositive/bucketcontext_6.q | 21 + .../queries/clientpositive/bucketcontext_7.q | 26 + .../queries/clientpositive/bucketcontext_8.q | 26 + .../bucketizedhiveinputformat.q | 33 + .../bucketizedhiveinputformat_auto.q | 23 + .../queries/clientpositive/bucketmapjoin1.q | 107 + .../queries/clientpositive/bucketmapjoin10.q | 36 + .../queries/clientpositive/bucketmapjoin11.q | 47 + .../queries/clientpositive/bucketmapjoin12.q | 44 + .../queries/clientpositive/bucketmapjoin13.q | 74 + .../queries/clientpositive/bucketmapjoin2.q | 112 + .../queries/clientpositive/bucketmapjoin3.q | 78 + .../queries/clientpositive/bucketmapjoin4.q | 80 + .../queries/clientpositive/bucketmapjoin5.q | 85 + .../queries/clientpositive/bucketmapjoin6.q | 25 + .../queries/clientpositive/bucketmapjoin7.q | 24 + .../queries/clientpositive/bucketmapjoin8.q | 39 + .../queries/clientpositive/bucketmapjoin9.q | 45 + .../clientpositive/bucketmapjoin_negative.q | 27 + .../clientpositive/bucketmapjoin_negative2.q | 18 + .../clientpositive/bucketmapjoin_negative3.q | 39 + .../bucketsortoptimize_insert_1.q | 78 + .../bucketsortoptimize_insert_2.q | 143 + .../bucketsortoptimize_insert_3.q | 52 + .../bucketsortoptimize_insert_4.q | 65 + .../bucketsortoptimize_insert_5.q | 63 + .../bucketsortoptimize_insert_6.q | 156 + .../bucketsortoptimize_insert_7.q | 88 + .../bucketsortoptimize_insert_8.q | 58 + .../queries/clientpositive/case_sensitivity.q | 10 + .../src/test/queries/clientpositive/cast1.q | 9 + .../test/queries/clientpositive/cast_to_int.q | 30 + .../src/test/queries/clientpositive/cluster.q | 65 + .../clientpositive/column_access_stats.q | 162 + .../columnarserde_create_shortcut.q | 24 + .../clientpositive/columnstats_partlvl.q | 20 + .../clientpositive/columnstats_tbllvl.q | 38 + .../test/queries/clientpositive/combine1.q | 17 + .../test/queries/clientpositive/combine2.q | 51 + .../clientpositive/combine2_hadoop20.q | 50 + .../queries/clientpositive/combine2_win.q | 41 + .../test/queries/clientpositive/combine3.q | 41 + .../clientpositive/compute_stats_binary.q | 9 + .../clientpositive/compute_stats_boolean.q | 9 + .../clientpositive/compute_stats_double.q | 9 + .../compute_stats_empty_table.q | 12 + .../clientpositive/compute_stats_long.q | 9 + .../clientpositive/compute_stats_string.q | 9 + .../concatenate_inherit_table_location.q | 21 + .../queries/clientpositive/constant_prop.q | 20 + .../clientpositive/convert_enum_to_string.q | 10 + .../clientpositive/correlationoptimizer1.q | 264 + .../clientpositive/correlationoptimizer10.q | 130 + .../clientpositive/correlationoptimizer11.q | 73 + .../clientpositive/correlationoptimizer12.q | 9 + .../clientpositive/correlationoptimizer13.q | 18 + .../clientpositive/correlationoptimizer14.q | 149 + .../clientpositive/correlationoptimizer15.q | 32 + .../clientpositive/correlationoptimizer2.q | 188 + .../clientpositive/correlationoptimizer3.q | 96 + .../clientpositive/correlationoptimizer4.q | 158 + .../clientpositive/correlationoptimizer5.q | 79 + .../clientpositive/correlationoptimizer6.q | 322 + .../clientpositive/correlationoptimizer7.q | 77 + .../clientpositive/correlationoptimizer8.q | 110 + .../clientpositive/correlationoptimizer9.q | 71 + .../src/test/queries/clientpositive/count.q | 17 + .../test/queries/clientpositive/cp_mj_rc.q | 7 + .../test/queries/clientpositive/create_1.q | 29 + .../create_alter_list_bucketing_table1.q | 40 + .../queries/clientpositive/create_big_view.q | 246 + .../clientpositive/create_default_prop.q | 10 + .../queries/clientpositive/create_escape.q | 10 + .../clientpositive/create_genericudaf.q | 17 + .../clientpositive/create_genericudf.q | 21 + .../create_insert_outputformat.q | 30 + .../test/queries/clientpositive/create_like.q | 35 + .../queries/clientpositive/create_like2.q | 9 + .../clientpositive/create_like_tbl_props.q | 35 + .../queries/clientpositive/create_like_view.q | 39 + .../clientpositive/create_merge_compressed.q | 24 + .../clientpositive/create_nested_type.q | 16 + .../clientpositive/create_or_replace_view.q | 32 + .../clientpositive/create_skewed_table1.q | 10 + .../clientpositive/create_struct_table.q | 12 + .../test/queries/clientpositive/create_udaf.q | 19 + .../clientpositive/create_union_table.q | 10 + .../test/queries/clientpositive/create_view.q | 247 + .../clientpositive/create_view_partitioned.q | 83 + .../clientpositive/create_view_translate.q | 13 + .../test/queries/clientpositive/cross_join.q | 6 + .../clientpositive/ct_case_insensitive.q | 3 + .../ql/src/test/queries/clientpositive/ctas.q | 69 + .../queries/clientpositive/ctas_colname.q | 48 + .../test/queries/clientpositive/ctas_date.q | 26 + .../queries/clientpositive/ctas_hadoop20.q | 68 + .../ctas_uses_database_location.q | 14 + .../queries/clientpositive/ctas_varchar.q | 22 + .../custom_input_output_format.q | 6 + .../test/queries/clientpositive/database.q | 216 + .../queries/clientpositive/database_drop.q | 81 + .../clientpositive/database_location.q | 20 + .../clientpositive/database_properties.q | 26 + .../src/test/queries/clientpositive/date_1.q | 87 + .../src/test/queries/clientpositive/date_2.q | 18 + .../src/test/queries/clientpositive/date_3.q | 14 + .../src/test/queries/clientpositive/date_4.q | 11 + .../queries/clientpositive/date_comparison.q | 38 + .../test/queries/clientpositive/date_join1.q | 19 + .../test/queries/clientpositive/date_serde.q | 112 + .../test/queries/clientpositive/date_udf.q | 69 + .../src/test/queries/clientpositive/ddltime.q | 45 + .../test/queries/clientpositive/decimal_1.q | 18 + .../test/queries/clientpositive/decimal_2.q | 40 + .../test/queries/clientpositive/decimal_3.q | 28 + .../test/queries/clientpositive/decimal_4.q | 21 + .../queries/clientpositive/decimal_join.q | 6 + .../clientpositive/decimal_precision.q | 27 + .../queries/clientpositive/decimal_serde.q | 37 + .../test/queries/clientpositive/decimal_udf.q | 128 + .../clientpositive/default_partition_name.q | 7 + .../test/queries/clientpositive/delimiter.q | 12 + .../clientpositive/desc_non_existent_tbl.q | 0 .../clientpositive/describe_comment_indent.q | 14 + .../describe_comment_nonascii.q | 9 + .../clientpositive/describe_database_json.q | 23 + .../describe_formatted_view_partitioned.q | 15 + ...describe_formatted_view_partitioned_json.q | 17 + .../queries/clientpositive/describe_pretty.q | 80 + .../queries/clientpositive/describe_syntax.q | 46 + .../queries/clientpositive/describe_table.q | 19 + .../clientpositive/describe_table_json.q | 19 + .../queries/clientpositive/describe_xpath.q | 17 + .../clientpositive/diff_part_input_formats.q | 9 + .../disable_file_format_check.q | 9 + .../disable_merge_for_bucketing.q | 21 + .../disallow_incompatible_type_change_off.q | 7 + .../test/queries/clientpositive/driverhook.q | 6 + .../drop_database_removes_partition_dirs.q | 27 + .../queries/clientpositive/drop_function.q | 2 + .../test/queries/clientpositive/drop_index.q | 2 + .../drop_index_removes_partition_dirs.q | 22 + .../clientpositive/drop_multi_partitions.q | 17 + .../clientpositive/drop_partitions_filter.q | 37 + .../clientpositive/drop_partitions_filter2.q | 20 + .../clientpositive/drop_partitions_filter3.q | 20 + .../drop_partitions_ignore_protection.q | 10 + .../test/queries/clientpositive/drop_table.q | 2 + .../test/queries/clientpositive/drop_table2.q | 15 + .../drop_table_removes_partition_dirs.q | 21 + .../test/queries/clientpositive/drop_udf.q | 6 + .../test/queries/clientpositive/drop_view.q | 2 + .../dynamic_partition_skip_default.q | 19 + .../queries/clientpositive/enforce_order.q | 13 + .../src/test/queries/clientpositive/escape1.q | 28 + .../src/test/queries/clientpositive/escape2.q | 122 + .../clientpositive/escape_clusterby1.q | 6 + .../clientpositive/escape_distributeby1.q | 6 + .../queries/clientpositive/escape_orderby1.q | 6 + .../queries/clientpositive/escape_sortby1.q | 6 + .../clientpositive/exchange_partition.q | 12 + .../clientpositive/exchange_partition2.q | 12 + .../clientpositive/exchange_partition3.q | 14 + .../clientpositive/exim_00_nonpart_empty.q | 23 + .../queries/clientpositive/exim_01_nonpart.q | 24 + .../clientpositive/exim_02_00_part_empty.q | 25 + .../queries/clientpositive/exim_02_part.q | 27 + .../exim_03_nonpart_over_compat.q | 26 + .../queries/clientpositive/exim_04_all_part.q | 33 + .../clientpositive/exim_04_evolved_parts.q | 39 + .../clientpositive/exim_05_some_part.q | 33 + .../queries/clientpositive/exim_06_one_part.q | 33 + .../exim_07_all_part_over_nonoverlap.q | 39 + .../clientpositive/exim_08_nonpart_rename.q | 28 + .../exim_09_part_spec_nonoverlap.q | 40 + .../clientpositive/exim_10_external_managed.q | 27 + .../clientpositive/exim_11_managed_external.q | 24 + .../exim_12_external_location.q | 29 + .../clientpositive/exim_13_managed_location.q | 29 + .../exim_14_managed_location_over_existing.q | 33 + .../clientpositive/exim_15_external_part.q | 50 + .../clientpositive/exim_16_part_external.q | 49 + .../clientpositive/exim_17_part_managed.q | 48 + .../clientpositive/exim_18_part_external.q | 36 + .../exim_19_00_part_external_location.q | 38 + .../exim_19_part_external_location.q | 41 + .../exim_20_part_managed_location.q | 41 + .../exim_21_export_authsuccess.q | 15 + .../exim_22_import_exist_authsuccess.q | 26 + .../exim_23_import_part_authsuccess.q | 34 + .../exim_24_import_nonexist_authsuccess.q | 24 + .../clientpositive/explain_dependency.q | 34 + .../clientpositive/explain_dependency2.q | 31 + .../queries/clientpositive/explain_logical.q | 36 + .../queries/clientpositive/explode_null.q | 10 + .../clientpositive/fetch_aggregation.q | 6 + .../queries/clientpositive/fileformat_mix.q | 16 + .../clientpositive/fileformat_sequencefile.q | 17 + .../queries/clientpositive/fileformat_text.q | 17 + .../clientpositive/filter_join_breaktask.q | 16 + .../clientpositive/filter_join_breaktask2.q | 40 + .../queries/clientpositive/global_limit.q | 68 + .../test/queries/clientpositive/groupby1.q | 15 + .../test/queries/clientpositive/groupby10.q | 48 + .../test/queries/clientpositive/groupby11.q | 27 + .../queries/clientpositive/groupby1_limit.q | 10 + .../queries/clientpositive/groupby1_map.q | 12 + .../clientpositive/groupby1_map_nomap.q | 12 + .../clientpositive/groupby1_map_skew.q | 12 + .../queries/clientpositive/groupby1_noskew.q | 12 + .../test/queries/clientpositive/groupby2.q | 14 + .../queries/clientpositive/groupby2_limit.q | 7 + .../queries/clientpositive/groupby2_map.q | 14 + .../groupby2_map_multi_distinct.q | 14 + .../clientpositive/groupby2_map_skew.q | 14 + .../queries/clientpositive/groupby2_noskew.q | 14 + .../groupby2_noskew_multi_distinct.q | 14 + .../test/queries/clientpositive/groupby3.q | 33 + .../queries/clientpositive/groupby3_map.q | 34 + .../groupby3_map_multi_distinct.q | 36 + .../clientpositive/groupby3_map_skew.q | 34 + .../queries/clientpositive/groupby3_noskew.q | 36 + .../groupby3_noskew_multi_distinct.q | 38 + .../test/queries/clientpositive/groupby4.q | 14 + .../queries/clientpositive/groupby4_map.q | 12 + .../clientpositive/groupby4_map_skew.q | 12 + .../queries/clientpositive/groupby4_noskew.q | 16 + .../test/queries/clientpositive/groupby5.q | 18 + .../queries/clientpositive/groupby5_map.q | 12 + .../clientpositive/groupby5_map_skew.q | 12 + .../queries/clientpositive/groupby5_noskew.q | 20 + .../test/queries/clientpositive/groupby6.q | 15 + .../queries/clientpositive/groupby6_map.q | 16 + .../clientpositive/groupby6_map_skew.q | 16 + .../queries/clientpositive/groupby6_noskew.q | 17 + .../test/queries/clientpositive/groupby7.q | 15 + .../queries/clientpositive/groupby7_map.q | 22 + .../groupby7_map_multi_single_reducer.q | 21 + .../clientpositive/groupby7_map_skew.q | 21 + .../queries/clientpositive/groupby7_noskew.q | 22 + .../groupby7_noskew_multi_single_reducer.q | 21 + .../test/queries/clientpositive/groupby8.q | 31 + .../queries/clientpositive/groupby8_map.q | 19 + .../clientpositive/groupby8_map_skew.q | 19 + .../queries/clientpositive/groupby8_noskew.q | 19 + .../test/queries/clientpositive/groupby9.q | 67 + .../queries/clientpositive/groupby_bigdata.q | 7 + .../clientpositive/groupby_complex_types.q | 19 + ...oupby_complex_types_multi_single_reducer.q | 17 + .../queries/clientpositive/groupby_cube1.q | 49 + .../clientpositive/groupby_distinct_samekey.q | 15 + .../clientpositive/groupby_grouping_id1.q | 10 + .../clientpositive/groupby_grouping_id2.q | 41 + .../clientpositive/groupby_grouping_sets1.q | 18 + .../clientpositive/groupby_grouping_sets2.q | 23 + .../clientpositive/groupby_grouping_sets3.q | 27 + .../clientpositive/groupby_grouping_sets4.q | 39 + .../clientpositive/groupby_grouping_sets5.q | 25 + .../queries/clientpositive/groupby_map_ppr.q | 20 + .../groupby_map_ppr_multi_distinct.q | 20 + .../groupby_multi_insert_common_distinct.q | 32 + .../groupby_multi_single_reducer.q | 49 + .../groupby_multi_single_reducer2.q | 19 + .../groupby_multi_single_reducer3.q | 101 + .../groupby_mutli_insert_common_distinct.q | 0 .../clientpositive/groupby_neg_float.q | 10 + .../queries/clientpositive/groupby_position.q | 72 + .../test/queries/clientpositive/groupby_ppd.q | 4 + .../test/queries/clientpositive/groupby_ppr.q | 19 + .../groupby_ppr_multi_distinct.q | 19 + .../queries/clientpositive/groupby_rollup1.q | 49 + .../queries/clientpositive/groupby_sort_1.q | 282 + .../queries/clientpositive/groupby_sort_10.q | 26 + .../queries/clientpositive/groupby_sort_11.q | 40 + .../queries/clientpositive/groupby_sort_2.q | 25 + .../queries/clientpositive/groupby_sort_3.q | 36 + .../queries/clientpositive/groupby_sort_4.q | 38 + .../queries/clientpositive/groupby_sort_5.q | 75 + .../queries/clientpositive/groupby_sort_6.q | 41 + .../queries/clientpositive/groupby_sort_7.q | 28 + .../queries/clientpositive/groupby_sort_8.q | 27 + .../queries/clientpositive/groupby_sort_9.q | 21 + .../clientpositive/groupby_sort_skew_1.q | 283 + .../clientpositive/groupby_sort_test_1.q | 21 + .../src/test/queries/clientpositive/having.q | 14 + .../queries/clientpositive/hook_context_cs.q | 14 + .../test/queries/clientpositive/hook_order.q | 11 + .../queries/clientpositive/implicit_cast1.q | 13 + .../test/queries/clientpositive/index_auth.q | 15 + .../test/queries/clientpositive/index_auto.q | 28 + .../queries/clientpositive/index_auto_empty.q | 21 + .../clientpositive/index_auto_file_format.q | 19 + .../clientpositive/index_auto_mult_tables.q | 22 + .../index_auto_mult_tables_compact.q | 23 + .../clientpositive/index_auto_multiple.q | 16 + .../clientpositive/index_auto_partitioned.q | 12 + .../clientpositive/index_auto_self_join.q | 16 + .../clientpositive/index_auto_unused.q | 60 + .../clientpositive/index_auto_update.q | 26 + .../queries/clientpositive/index_bitmap.q | 47 + .../queries/clientpositive/index_bitmap1.q | 18 + .../queries/clientpositive/index_bitmap2.q | 35 + .../queries/clientpositive/index_bitmap3.q | 47 + .../clientpositive/index_bitmap_auto.q | 52 + .../index_bitmap_auto_partitioned.q | 11 + .../clientpositive/index_bitmap_compression.q | 13 + .../queries/clientpositive/index_bitmap_rc.q | 53 + .../queries/clientpositive/index_compact.q | 41 + .../queries/clientpositive/index_compact_1.q | 16 + .../queries/clientpositive/index_compact_2.q | 45 + .../queries/clientpositive/index_compact_3.q | 19 + .../index_compact_binary_search.q | 132 + .../clientpositive/index_compression.q | 13 + .../queries/clientpositive/index_creation.q | 53 + .../test/queries/clientpositive/index_serde.q | 49 + .../test/queries/clientpositive/index_stale.q | 20 + .../clientpositive/index_stale_partitioned.q | 26 + .../clientpositive/infer_bucket_sort.q | 158 + .../infer_bucket_sort_bucketed_table.q | 26 + .../infer_bucket_sort_convert_join.q | 27 + .../infer_bucket_sort_dyn_part.q | 87 + .../infer_bucket_sort_grouping_operators.q | 57 + .../infer_bucket_sort_list_bucket.q | 33 + .../infer_bucket_sort_map_operators.q | 74 + .../clientpositive/infer_bucket_sort_merge.q | 25 + .../infer_bucket_sort_multi_insert.q | 46 + .../infer_bucket_sort_num_buckets.q | 37 + .../infer_bucket_sort_reducers_power_two.q | 46 + .../queries/clientpositive/infer_const_type.q | 66 + .../test/queries/clientpositive/init_file.q | 5 + .../test/queries/clientpositive/innerjoin.q | 23 + .../test/queries/clientpositive/inoutdriver.q | 2 + .../src/test/queries/clientpositive/input.q | 4 + .../src/test/queries/clientpositive/input0.q | 5 + .../src/test/queries/clientpositive/input1.q | 9 + .../src/test/queries/clientpositive/input10.q | 9 + .../src/test/queries/clientpositive/input11.q | 10 + .../queries/clientpositive/input11_limit.q | 10 + .../src/test/queries/clientpositive/input12.q | 24 + .../queries/clientpositive/input12_hadoop20.q | 23 + .../src/test/queries/clientpositive/input13.q | 21 + .../src/test/queries/clientpositive/input14.q | 20 + .../queries/clientpositive/input14_limit.q | 20 + .../src/test/queries/clientpositive/input15.q | 7 + .../src/test/queries/clientpositive/input16.q | 6 + .../test/queries/clientpositive/input16_cc.q | 11 + .../src/test/queries/clientpositive/input17.q | 20 + .../src/test/queries/clientpositive/input18.q | 20 + .../src/test/queries/clientpositive/input19.q | 5 + .../queries/clientpositive/input1_limit.q | 18 + .../src/test/queries/clientpositive/input2.q | 12 + .../src/test/queries/clientpositive/input20.q | 30 + .../src/test/queries/clientpositive/input21.q | 10 + .../src/test/queries/clientpositive/input22.q | 15 + .../src/test/queries/clientpositive/input23.q | 5 + .../src/test/queries/clientpositive/input24.q | 9 + .../src/test/queries/clientpositive/input25.q | 19 + .../src/test/queries/clientpositive/input26.q | 12 + .../src/test/queries/clientpositive/input28.q | 10 + .../queries/clientpositive/input2_limit.q | 4 + .../src/test/queries/clientpositive/input3.q | 26 + .../src/test/queries/clientpositive/input30.q | 23 + .../src/test/queries/clientpositive/input31.q | 24 + .../src/test/queries/clientpositive/input32.q | 25 + .../src/test/queries/clientpositive/input33.q | 30 + .../src/test/queries/clientpositive/input34.q | 20 + .../src/test/queries/clientpositive/input35.q | 20 + .../src/test/queries/clientpositive/input36.q | 20 + .../src/test/queries/clientpositive/input37.q | 16 + .../src/test/queries/clientpositive/input38.q | 22 + .../src/test/queries/clientpositive/input39.q | 32 + .../queries/clientpositive/input39_hadoop20.q | 30 + .../queries/clientpositive/input3_limit.q | 17 + .../src/test/queries/clientpositive/input4.q | 8 + .../src/test/queries/clientpositive/input40.q | 16 + .../src/test/queries/clientpositive/input41.q | 14 + .../src/test/queries/clientpositive/input42.q | 16 + .../src/test/queries/clientpositive/input43.q | 7 + .../src/test/queries/clientpositive/input44.q | 6 + .../src/test/queries/clientpositive/input45.q | 9 + .../src/test/queries/clientpositive/input46.q | 6 + .../src/test/queries/clientpositive/input49.q | 4 + .../queries/clientpositive/input4_cb_delim.q | 4 + .../queries/clientpositive/input4_limit.q | 5 + .../src/test/queries/clientpositive/input5.q | 20 + .../src/test/queries/clientpositive/input6.q | 10 + .../src/test/queries/clientpositive/input7.q | 10 + .../src/test/queries/clientpositive/input8.q | 10 + .../src/test/queries/clientpositive/input9.q | 11 + .../clientpositive/input_columnarserde.q | 19 + .../test/queries/clientpositive/input_dfs.q | 2 + .../clientpositive/input_dynamicserde.q | 18 + .../queries/clientpositive/input_lazyserde.q | 30 + .../test/queries/clientpositive/input_limit.q | 4 + .../test/queries/clientpositive/input_part0.q | 4 + .../test/queries/clientpositive/input_part1.q | 11 + .../queries/clientpositive/input_part10.q | 23 + .../queries/clientpositive/input_part10_win.q | 23 + .../test/queries/clientpositive/input_part2.q | 16 + .../test/queries/clientpositive/input_part3.q | 4 + .../test/queries/clientpositive/input_part4.q | 4 + .../test/queries/clientpositive/input_part5.q | 12 + .../test/queries/clientpositive/input_part6.q | 4 + .../test/queries/clientpositive/input_part7.q | 14 + .../test/queries/clientpositive/input_part8.q | 4 + .../test/queries/clientpositive/input_part9.q | 5 + .../clientpositive/input_testsequencefile.q | 14 + .../queries/clientpositive/input_testxpath.q | 10 + .../queries/clientpositive/input_testxpath2.q | 10 + .../queries/clientpositive/input_testxpath3.q | 6 + .../queries/clientpositive/input_testxpath4.q | 29 + .../test/queries/clientpositive/inputddl1.q | 6 + .../test/queries/clientpositive/inputddl2.q | 6 + .../test/queries/clientpositive/inputddl3.q | 5 + .../test/queries/clientpositive/inputddl4.q | 12 + .../test/queries/clientpositive/inputddl5.q | 8 + .../test/queries/clientpositive/inputddl6.q | 15 + .../test/queries/clientpositive/inputddl7.q | 33 + .../test/queries/clientpositive/inputddl8.q | 10 + .../src/test/queries/clientpositive/insert1.q | 36 + .../insert1_overwrite_partitions.q | 35 + .../insert2_overwrite_partitions.q | 35 + .../clientpositive/insert_compressed.q | 15 + .../queries/clientpositive/insert_into1.q | 26 + .../queries/clientpositive/insert_into2.q | 31 + .../queries/clientpositive/insert_into3.q | 30 + .../queries/clientpositive/insert_into4.q | 30 + .../queries/clientpositive/insert_into5.q | 34 + .../queries/clientpositive/insert_into6.q | 28 + .../insert_overwrite_local_directory_1.q | 109 + .../queries/clientpositive/insertexternal1.q | 14 + .../src/test/queries/clientpositive/join0.q | 23 + .../src/test/queries/clientpositive/join1.q | 11 + .../src/test/queries/clientpositive/join10.q | 13 + .../src/test/queries/clientpositive/join11.q | 14 + .../src/test/queries/clientpositive/join12.q | 20 + .../src/test/queries/clientpositive/join13.q | 20 + .../src/test/queries/clientpositive/join14.q | 17 + .../queries/clientpositive/join14_hadoop20.q | 15 + .../src/test/queries/clientpositive/join15.q | 4 + .../src/test/queries/clientpositive/join16.q | 1 + .../src/test/queries/clientpositive/join17.q | 10 + .../src/test/queries/clientpositive/join18.q | 24 + .../clientpositive/join18_multi_distinct.q | 26 + .../src/test/queries/clientpositive/join19.q | 58 + .../src/test/queries/clientpositive/join2.q | 10 + .../src/test/queries/clientpositive/join20.q | 14 + .../src/test/queries/clientpositive/join21.q | 4 + .../src/test/queries/clientpositive/join22.q | 2 + .../src/test/queries/clientpositive/join23.q | 4 + .../src/test/queries/clientpositive/join24.q | 8 + .../src/test/queries/clientpositive/join25.q | 19 + .../src/test/queries/clientpositive/join26.q | 17 + .../src/test/queries/clientpositive/join27.q | 17 + .../src/test/queries/clientpositive/join28.q | 27 + .../src/test/queries/clientpositive/join29.q | 20 + .../src/test/queries/clientpositive/join3.q | 10 + .../src/test/queries/clientpositive/join30.q | 10 + .../src/test/queries/clientpositive/join31.q | 22 + .../src/test/queries/clientpositive/join32.q | 23 + .../queries/clientpositive/join32_lessSize.q | 88 + .../src/test/queries/clientpositive/join33.q | 23 + .../src/test/queries/clientpositive/join34.q | 31 + .../src/test/queries/clientpositive/join35.q | 31 + .../src/test/queries/clientpositive/join36.q | 29 + .../src/test/queries/clientpositive/join37.q | 19 + .../src/test/queries/clientpositive/join38.q | 20 + .../src/test/queries/clientpositive/join39.q | 16 + .../src/test/queries/clientpositive/join4.q | 32 + .../src/test/queries/clientpositive/join40.q | 43 + .../src/test/queries/clientpositive/join41.q | 16 + .../src/test/queries/clientpositive/join5.q | 32 + .../src/test/queries/clientpositive/join6.q | 33 + .../src/test/queries/clientpositive/join7.q | 42 + .../src/test/queries/clientpositive/join8.q | 32 + .../src/test/queries/clientpositive/join9.q | 10 + .../test/queries/clientpositive/join_1to1.q | 58 + .../test/queries/clientpositive/join_array.q | 10 + .../clientpositive/join_casesensitive.q | 8 + .../test/queries/clientpositive/join_empty.q | 10 + .../queries/clientpositive/join_filters.q | 155 + .../clientpositive/join_filters_overlap.q | 27 + .../queries/clientpositive/join_hive_626.q | 30 + .../queries/clientpositive/join_literals.q | 9 + .../queries/clientpositive/join_map_ppr.q | 41 + .../test/queries/clientpositive/join_nulls.q | 68 + .../queries/clientpositive/join_nullsafe.q | 61 + .../src/test/queries/clientpositive/join_rc.q | 16 + .../queries/clientpositive/join_reorder.q | 71 + .../queries/clientpositive/join_reorder2.q | 44 + .../queries/clientpositive/join_reorder3.q | 44 + .../queries/clientpositive/join_reorder4.q | 18 + .../test/queries/clientpositive/join_star.q | 54 + .../test/queries/clientpositive/join_thrift.q | 12 + .../src/test/queries/clientpositive/join_vc.q | 5 + .../test/queries/clientpositive/join_view.q | 16 + .../test/queries/clientpositive/keyword_1.q | 19 + .../queries/clientpositive/lateral_view.q | 55 + .../queries/clientpositive/lateral_view_cp.q | 8 + .../clientpositive/lateral_view_noalias.q | 12 + .../clientpositive/lateral_view_outer.q | 15 + .../queries/clientpositive/lateral_view_ppd.q | 13 + .../src/test/queries/clientpositive/leadlag.q | 82 + .../queries/clientpositive/leadlag_queries.q | 53 + .../queries/clientpositive/leftsemijoin.q | 23 + .../queries/clientpositive/leftsemijoin_mr.q | 20 + .../queries/clientpositive/limit_pushdown.q | 66 + .../clientpositive/limit_pushdown_negative.q | 22 + .../test/queries/clientpositive/lineage1.q | 30 + .../clientpositive/list_bucket_dml_1.q | 40 + .../clientpositive/list_bucket_dml_10.q | 35 + .../clientpositive/list_bucket_dml_11.q | 36 + .../clientpositive/list_bucket_dml_12.q | 42 + .../clientpositive/list_bucket_dml_13.q | 36 + .../clientpositive/list_bucket_dml_2.q | 71 + .../clientpositive/list_bucket_dml_3.q | 33 + .../clientpositive/list_bucket_dml_4.q | 71 + .../clientpositive/list_bucket_dml_5.q | 38 + .../clientpositive/list_bucket_dml_6.q | 97 + .../clientpositive/list_bucket_dml_7.q | 70 + .../clientpositive/list_bucket_dml_8.q | 90 + .../clientpositive/list_bucket_dml_9.q | 71 + .../list_bucket_query_multiskew_1.q | 56 + .../list_bucket_query_multiskew_2.q | 50 + .../list_bucket_query_multiskew_3.q | 57 + .../list_bucket_query_oneskew_1.q | 61 + .../list_bucket_query_oneskew_2.q | 73 + .../list_bucket_query_oneskew_3.q | 61 + .../queries/clientpositive/literal_decimal.q | 3 + .../queries/clientpositive/literal_double.q | 3 + .../queries/clientpositive/literal_ints.q | 3 + .../queries/clientpositive/literal_string.q | 13 + .../queries/clientpositive/load_binary_data.q | 13 + .../queries/clientpositive/load_dyn_part1.q | 30 + .../queries/clientpositive/load_dyn_part10.q | 24 + .../queries/clientpositive/load_dyn_part11.q | 17 + .../queries/clientpositive/load_dyn_part12.q | 19 + .../queries/clientpositive/load_dyn_part13.q | 37 + .../queries/clientpositive/load_dyn_part14.q | 37 + .../clientpositive/load_dyn_part14_win.q | 38 + .../queries/clientpositive/load_dyn_part15.q | 13 + .../queries/clientpositive/load_dyn_part2.q | 23 + .../queries/clientpositive/load_dyn_part3.q | 19 + .../queries/clientpositive/load_dyn_part4.q | 24 + .../queries/clientpositive/load_dyn_part5.q | 22 + .../queries/clientpositive/load_dyn_part6.q | 16 + .../queries/clientpositive/load_dyn_part7.q | 14 + .../queries/clientpositive/load_dyn_part8.q | 24 + .../queries/clientpositive/load_dyn_part9.q | 23 + .../load_exist_part_authsuccess.q | 5 + .../load_file_with_space_in_the_name.q | 5 + .../src/test/queries/clientpositive/load_fs.q | 21 + .../test/queries/clientpositive/load_fs2.q | 20 + .../load_hdfs_file_with_space_in_the_name.q | 9 + .../clientpositive/load_nonpart_authsuccess.q | 4 + .../queries/clientpositive/load_overwrite.q | 15 + .../clientpositive/load_part_authsuccess.q | 4 + .../test/queries/clientpositive/loadpart1.q | 19 + .../queries/clientpositive/loadpart_err.q | 20 + .../src/test/queries/clientpositive/lock1.q | 29 + .../src/test/queries/clientpositive/lock2.q | 36 + .../src/test/queries/clientpositive/lock3.q | 32 + .../src/test/queries/clientpositive/lock4.q | 33 + .../queries/clientpositive/louter_join_ppr.q | 71 + .../src/test/queries/clientpositive/macro.q | 26 + .../test/queries/clientpositive/mapjoin1.q | 38 + .../queries/clientpositive/mapjoin_distinct.q | 55 + .../mapjoin_filter_on_outerjoin.q | 31 + .../queries/clientpositive/mapjoin_hook.q | 31 + .../queries/clientpositive/mapjoin_mapjoin.q | 11 + .../queries/clientpositive/mapjoin_subquery.q | 34 + .../clientpositive/mapjoin_subquery2.q | 39 + .../clientpositive/mapjoin_test_outer.q | 54 + .../test/queries/clientpositive/mapreduce1.q | 20 + .../test/queries/clientpositive/mapreduce2.q | 17 + .../test/queries/clientpositive/mapreduce3.q | 17 + .../test/queries/clientpositive/mapreduce4.q | 20 + .../test/queries/clientpositive/mapreduce5.q | 17 + .../test/queries/clientpositive/mapreduce6.q | 17 + .../test/queries/clientpositive/mapreduce7.q | 17 + .../test/queries/clientpositive/mapreduce8.q | 19 + .../src/test/queries/clientpositive/merge1.q | 29 + .../src/test/queries/clientpositive/merge2.q | 35 + .../src/test/queries/clientpositive/merge3.q | 57 + .../src/test/queries/clientpositive/merge4.q | 40 + .../clientpositive/merge_dynamic_partition.q | 46 + .../clientpositive/merge_dynamic_partition2.q | 27 + .../clientpositive/merge_dynamic_partition3.q | 38 + .../clientpositive/merge_dynamic_partition4.q | 42 + .../clientpositive/merge_dynamic_partition5.q | 38 + .../test/queries/clientpositive/mergejoins.q | 10 + .../queries/clientpositive/mergejoins_mixed.q | 43 + .../clientpositive/metadata_export_drop.q | 8 + .../queries/clientpositive/metadataonly1.q | 45 + .../ql/src/test/queries/clientpositive/mi.q | 21 + .../test/queries/clientpositive/misc_json.q | 13 + .../queries/clientpositive/multiMapJoin1.q | 401 + .../queries/clientpositive/multiMapJoin2.q | 189 + .../queries/clientpositive/multi_insert.q | 266 + .../queries/clientpositive/multi_insert_gby.q | 33 + .../clientpositive/multi_insert_gby2.q | 19 + .../clientpositive/multi_insert_gby3.q | 52 + .../multi_insert_lateral_view.q | 102 + ...lti_insert_move_tasks_share_dependencies.q | 414 + .../queries/clientpositive/multi_join_union.q | 13 + .../queries/clientpositive/multi_sahooks.q | 30 + .../clientpositive/multigroupby_singlemr.q | 33 + .../queries/clientpositive/nested_complex.q | 22 + .../queries/clientpositive/nestedvirtual.q | 27 + .../src/test/queries/clientpositive/newline.q | 57 + .../test/queries/clientpositive/no_hooks.q | 6 + .../queries/clientpositive/noalias_subq1.q | 5 + .../nomore_ambiguous_table_col.q | 9 + .../clientpositive/nonblock_op_deduplicate.q | 45 + .../test/queries/clientpositive/nonmr_fetch.q | 83 + .../nonreserved_keywords_input37.q | 12 + .../nonreserved_keywords_insert_into1.q | 26 + .../queries/clientpositive/notable_alias1.q | 10 + .../queries/clientpositive/notable_alias2.q | 10 + .../test/queries/clientpositive/null_cast.q | 11 + .../test/queries/clientpositive/null_column.q | 29 + .../test/queries/clientpositive/nullgroup.q | 31 + .../test/queries/clientpositive/nullgroup2.q | 31 + .../test/queries/clientpositive/nullgroup3.q | 28 + .../test/queries/clientpositive/nullgroup4.q | 31 + .../nullgroup4_multi_distinct.q | 15 + .../test/queries/clientpositive/nullgroup5.q | 26 + .../test/queries/clientpositive/nullinput.q | 4 + .../test/queries/clientpositive/nullinput2.q | 8 + .../test/queries/clientpositive/nullscript.q | 10 + .../queries/clientpositive/num_op_type_conv.q | 9 + .../queries/clientpositive/ops_comparison.q | 23 + .../queries/clientpositive/optional_outer.q | 8 + .../queries/clientpositive/optrstat_groupby.q | 6 + .../test/queries/clientpositive/orc_create.q | 103 + .../queries/clientpositive/orc_createas1.q | 48 + .../clientpositive/orc_dictionary_threshold.q | 60 + .../clientpositive/orc_diff_part_cols.q | 19 + .../queries/clientpositive/orc_empty_files.q | 18 + .../clientpositive/orc_empty_strings.q | 16 + .../clientpositive/orc_ends_with_nulls.q | 17 + .../clientpositive/orc_predicate_pushdown.q | 248 + .../src/test/queries/clientpositive/order.q | 9 + .../src/test/queries/clientpositive/order2.q | 10 + .../queries/clientpositive/outer_join_ppr.q | 35 + .../queries/clientpositive/overridden_confs.q | 4 + .../test/queries/clientpositive/parallel.q | 28 + .../queries/clientpositive/parallel_orderby.q | 23 + .../clientpositive/parenthesis_star_by.q | 10 + .../test/queries/clientpositive/partInit.q | 7 + .../clientpositive/part_inherit_tbl_props.q | 8 + .../part_inherit_tbl_props_empty.q | 4 + .../part_inherit_tbl_props_with_star.q | 8 + .../test/queries/clientpositive/partcols1.q | 18 + .../queries/clientpositive/partition_date.q | 45 + .../queries/clientpositive/partition_date2.q | 55 + .../clientpositive/partition_decode_name.q | 21 + .../clientpositive/partition_schema1.q | 12 + .../clientpositive/partition_serde_format.q | 4 + .../clientpositive/partition_special_char.q | 19 + .../clientpositive/partition_type_check.q | 24 + .../clientpositive/partition_varchar1.q | 43 + .../partition_vs_table_metadata.q | 13 + .../partition_wise_fileformat.q | 32 + .../partition_wise_fileformat10.q | 13 + .../partition_wise_fileformat11.q | 19 + .../partition_wise_fileformat12.q | 26 + .../partition_wise_fileformat13.q | 17 + .../partition_wise_fileformat14.q | 57 + .../partition_wise_fileformat15.q | 26 + .../partition_wise_fileformat16.q | 26 + .../partition_wise_fileformat17.q | 34 + .../partition_wise_fileformat2.q | 18 + .../partition_wise_fileformat3.q | 18 + .../partition_wise_fileformat4.q | 8 + .../partition_wise_fileformat5.q | 14 + .../partition_wise_fileformat6.q | 19 + .../partition_wise_fileformat7.q | 12 + .../partition_wise_fileformat8.q | 13 + .../partition_wise_fileformat9.q | 12 + .../queries/clientpositive/partitions_json.q | 21 + .../ql/src/test/queries/clientpositive/pcr.q | 141 + .../test/queries/clientpositive/plan_json.q | 3 + .../ql/src/test/queries/clientpositive/ppd1.q | 14 + .../ql/src/test/queries/clientpositive/ppd2.q | 74 + .../queries/clientpositive/ppd_clusterby.q | 20 + .../clientpositive/ppd_constant_expr.q | 24 + .../clientpositive/ppd_constant_where.q | 5 + .../src/test/queries/clientpositive/ppd_gby.q | 26 + .../test/queries/clientpositive/ppd_gby2.q | 30 + .../queries/clientpositive/ppd_gby_join.q | 24 + .../test/queries/clientpositive/ppd_join.q | 38 + .../test/queries/clientpositive/ppd_join2.q | 50 + .../test/queries/clientpositive/ppd_join3.q | 50 + .../queries/clientpositive/ppd_join_filter.q | 116 + .../queries/clientpositive/ppd_multi_insert.q | 45 + .../queries/clientpositive/ppd_outer_join1.q | 38 + .../queries/clientpositive/ppd_outer_join2.q | 38 + .../queries/clientpositive/ppd_outer_join3.q | 38 + .../queries/clientpositive/ppd_outer_join4.q | 50 + .../queries/clientpositive/ppd_outer_join5.q | 16 + .../test/queries/clientpositive/ppd_random.q | 22 + .../clientpositive/ppd_repeated_alias.q | 33 + .../queries/clientpositive/ppd_transform.q | 38 + .../queries/clientpositive/ppd_udf_case.q | 55 + .../test/queries/clientpositive/ppd_udf_col.q | 48 + .../test/queries/clientpositive/ppd_union.q | 38 + .../queries/clientpositive/ppd_union_view.q | 51 + .../src/test/queries/clientpositive/ppd_vc.q | 14 + .../clientpositive/ppr_allchildsarenull.q | 41 + .../queries/clientpositive/ppr_pushdown.q | 41 + .../queries/clientpositive/ppr_pushdown2.q | 29 + .../queries/clientpositive/ppr_pushdown3.q | 10 + .../queries/clientpositive/print_header.q | 14 + .../test/queries/clientpositive/progress_1.q | 9 + .../test/queries/clientpositive/protectmode.q | 63 + .../queries/clientpositive/protectmode2.q | 23 + .../ql/src/test/queries/clientpositive/ptf.q | 306 + .../test/queries/clientpositive/ptf_decimal.q | 49 + .../clientpositive/ptf_general_queries.q | 34 + .../queries/clientpositive/ptf_matchpath.q | 36 + .../test/queries/clientpositive/ptf_rcfile.q | 24 + .../clientpositive/ptf_register_tblfn.q | 29 + .../test/queries/clientpositive/ptf_seqfile.q | 24 + .../src/test/queries/clientpositive/push_or.q | 10 + .../clientpositive/ql_rewrite_gbtoidx.q | 170 + .../queries/clientpositive/query_properties.q | 15 + .../clientpositive/query_result_fileformat.q | 23 + .../queries/clientpositive/query_with_semi.q | 6 + .../src/test/queries/clientpositive/quote1.q | 13 + .../src/test/queries/clientpositive/quote2.q | 28 + .../clientpositive/rand_partitionpruner1.q | 3 + .../clientpositive/rand_partitionpruner2.q | 14 + .../clientpositive/rand_partitionpruner3.q | 9 + .../queries/clientpositive/rcfile_bigdata.q | 19 + .../queries/clientpositive/rcfile_columnar.q | 15 + .../queries/clientpositive/rcfile_createas1.q | 35 + .../clientpositive/rcfile_default_format.q | 31 + .../clientpositive/rcfile_lazydecompress.q | 27 + .../queries/clientpositive/rcfile_merge1.q | 43 + .../queries/clientpositive/rcfile_merge2.q | 32 + .../queries/clientpositive/rcfile_merge3.q | 32 + .../queries/clientpositive/rcfile_merge4.q | 32 + .../clientpositive/rcfile_null_value.q | 40 + .../rcfile_toleratecorruptions.q | 8 + .../queries/clientpositive/rcfile_union.q | 14 + .../queries/clientpositive/recursive_dir.q | 20 + .../clientpositive/reduce_deduplicate.q | 45 + .../reduce_deduplicate_exclude_gby.q | 8 + .../reduce_deduplicate_exclude_join.q | 3 + .../reduce_deduplicate_extended.q | 55 + .../queries/clientpositive/reducesink_dedup.q | 20 + .../test/queries/clientpositive/regex_col.q | 35 + .../queries/clientpositive/regexp_extract.q | 33 + .../queries/clientpositive/remote_script.q | 13 + .../queries/clientpositive/rename_column.q | 57 + .../rename_external_partition_location.q | 24 + .../rename_partition_location.q | 20 + .../clientpositive/rename_table_location.q | 17 + .../src/test/queries/clientpositive/repair.q | 15 + .../test/queries/clientpositive/reset_conf.q | 11 + .../queries/clientpositive/router_join_ppr.q | 71 + .../src/test/queries/clientpositive/sample1.q | 15 + .../test/queries/clientpositive/sample10.q | 25 + .../src/test/queries/clientpositive/sample2.q | 12 + .../src/test/queries/clientpositive/sample3.q | 8 + .../src/test/queries/clientpositive/sample4.q | 12 + .../src/test/queries/clientpositive/sample5.q | 11 + .../src/test/queries/clientpositive/sample6.q | 50 + .../src/test/queries/clientpositive/sample7.q | 13 + .../src/test/queries/clientpositive/sample8.q | 15 + .../src/test/queries/clientpositive/sample9.q | 6 + .../clientpositive/sample_islocalmode_hook.q | 39 + .../sample_islocalmode_hook_hadoop20.q | 41 + .../queries/clientpositive/schemeAuthority.q | 15 + .../queries/clientpositive/schemeAuthority2.q | 12 + .../queries/clientpositive/script_env_var1.q | 5 + .../queries/clientpositive/script_env_var2.q | 5 + .../test/queries/clientpositive/script_pipe.q | 8 + .../test/queries/clientpositive/scriptfile1.q | 14 + .../clientpositive/select_as_omitted.q | 12 + .../clientpositive/select_transform_hint.q | 24 + .../clientpositive/select_unquote_and.q | 15 + .../clientpositive/select_unquote_not.q | 15 + .../clientpositive/select_unquote_or.q | 15 + .../test/queries/clientpositive/semicolon.q | 4 + .../test/queries/clientpositive/semijoin.q | 81 + .../test/queries/clientpositive/serde_regex.q | 66 + .../clientpositive/serde_reported_schema.q | 9 + .../clientpositive/serde_user_properties.q | 9 + .../clientpositive/set_processor_namespaces.q | 31 + .../queries/clientpositive/set_variable_sub.q | 15 + .../queries/clientpositive/show_columns.q | 26 + .../clientpositive/show_create_table_alter.q | 23 + .../show_create_table_db_table.q | 10 + .../show_create_table_delimited.q | 9 + .../show_create_table_partitioned.q | 8 + .../clientpositive/show_create_table_serde.q | 19 + .../clientpositive/show_create_table_view.q | 6 + .../show_describe_func_quotes.q | 7 + .../queries/clientpositive/show_functions.q | 11 + .../clientpositive/show_indexes_edge_cases.q | 27 + .../clientpositive/show_indexes_syntax.q | 23 + .../queries/clientpositive/show_partitions.q | 4 + .../test/queries/clientpositive/show_tables.q | 35 + .../queries/clientpositive/show_tablestatus.q | 20 + .../clientpositive/show_tblproperties.q | 11 + .../test/queries/clientpositive/showparts.q | 4 + .../test/queries/clientpositive/skewjoin.q | 137 + .../clientpositive/skewjoin_union_remove_1.q | 68 + .../clientpositive/skewjoin_union_remove_2.q | 40 + .../queries/clientpositive/skewjoinopt1.q | 42 + .../queries/clientpositive/skewjoinopt10.q | 20 + .../queries/clientpositive/skewjoinopt11.q | 33 + .../queries/clientpositive/skewjoinopt12.q | 23 + .../queries/clientpositive/skewjoinopt13.q | 36 + .../queries/clientpositive/skewjoinopt14.q | 38 + .../queries/clientpositive/skewjoinopt15.q | 51 + .../queries/clientpositive/skewjoinopt16.q | 23 + .../queries/clientpositive/skewjoinopt17.q | 49 + .../queries/clientpositive/skewjoinopt18.q | 30 + .../queries/clientpositive/skewjoinopt19.q | 24 + .../queries/clientpositive/skewjoinopt2.q | 45 + .../queries/clientpositive/skewjoinopt20.q | 24 + .../queries/clientpositive/skewjoinopt3.q | 32 + .../queries/clientpositive/skewjoinopt4.q | 29 + .../queries/clientpositive/skewjoinopt5.q | 23 + .../queries/clientpositive/skewjoinopt6.q | 24 + .../queries/clientpositive/skewjoinopt7.q | 28 + .../queries/clientpositive/skewjoinopt8.q | 27 + .../queries/clientpositive/skewjoinopt9.q | 49 + .../queries/clientpositive/smb_mapjoin9.q | 57 + .../queries/clientpositive/smb_mapjoin_1.q | 53 + .../queries/clientpositive/smb_mapjoin_10.q | 26 + .../queries/clientpositive/smb_mapjoin_11.q | 34 + .../queries/clientpositive/smb_mapjoin_12.q | 49 + .../queries/clientpositive/smb_mapjoin_13.q | 36 + .../queries/clientpositive/smb_mapjoin_14.q | 243 + .../queries/clientpositive/smb_mapjoin_15.q | 55 + .../queries/clientpositive/smb_mapjoin_16.q | 21 + .../queries/clientpositive/smb_mapjoin_17.q | 125 + .../queries/clientpositive/smb_mapjoin_18.q | 65 + .../queries/clientpositive/smb_mapjoin_19.q | 41 + .../queries/clientpositive/smb_mapjoin_2.q | 53 + .../queries/clientpositive/smb_mapjoin_20.q | 53 + .../queries/clientpositive/smb_mapjoin_21.q | 77 + .../queries/clientpositive/smb_mapjoin_22.q | 55 + .../queries/clientpositive/smb_mapjoin_25.q | 41 + .../queries/clientpositive/smb_mapjoin_3.q | 53 + .../queries/clientpositive/smb_mapjoin_4.q | 72 + .../queries/clientpositive/smb_mapjoin_5.q | 72 + .../queries/clientpositive/smb_mapjoin_6.q | 78 + .../queries/clientpositive/smb_mapjoin_7.q | 55 + .../queries/clientpositive/smb_mapjoin_8.q | 90 + .../ql/src/test/queries/clientpositive/sort.q | 4 + .../clientpositive/sort_merge_join_desc_1.q | 23 + .../clientpositive/sort_merge_join_desc_2.q | 28 + .../clientpositive/sort_merge_join_desc_3.q | 28 + .../clientpositive/sort_merge_join_desc_4.q | 27 + .../clientpositive/sort_merge_join_desc_5.q | 26 + .../clientpositive/sort_merge_join_desc_6.q | 27 + .../clientpositive/sort_merge_join_desc_7.q | 33 + .../src/test/queries/clientpositive/source.q | 1 + .../queries/clientpositive/split_sample.q | 115 + .../src/test/queries/clientpositive/stats0.q | 70 + .../src/test/queries/clientpositive/stats1.q | 30 + .../src/test/queries/clientpositive/stats10.q | 28 + .../src/test/queries/clientpositive/stats11.q | 93 + .../src/test/queries/clientpositive/stats12.q | 19 + .../src/test/queries/clientpositive/stats13.q | 22 + .../src/test/queries/clientpositive/stats14.q | 26 + .../src/test/queries/clientpositive/stats15.q | 27 + .../src/test/queries/clientpositive/stats16.q | 12 + .../src/test/queries/clientpositive/stats18.q | 21 + .../src/test/queries/clientpositive/stats19.q | 105 + .../src/test/queries/clientpositive/stats2.q | 21 + .../src/test/queries/clientpositive/stats20.q | 16 + .../src/test/queries/clientpositive/stats3.q | 32 + .../src/test/queries/clientpositive/stats4.q | 40 + .../src/test/queries/clientpositive/stats5.q | 10 + .../src/test/queries/clientpositive/stats6.q | 17 + .../src/test/queries/clientpositive/stats7.q | 16 + .../src/test/queries/clientpositive/stats8.q | 33 + .../src/test/queries/clientpositive/stats9.q | 9 + .../clientpositive/stats_aggregator_error_1.q | 29 + .../clientpositive/stats_empty_dyn_part.q | 13 + .../clientpositive/stats_empty_partition.q | 11 + .../queries/clientpositive/stats_noscan_1.q | 37 + .../queries/clientpositive/stats_noscan_2.q | 32 + .../queries/clientpositive/stats_partscan_1.q | 37 + .../clientpositive/stats_partscan_1_23.q | 37 + .../clientpositive/stats_publisher_error_1.q | 29 + .../test/queries/clientpositive/str_to_map.q | 26 + .../ql/src/test/queries/clientpositive/subq.q | 13 + .../src/test/queries/clientpositive/subq2.q | 8 + .../symlink_text_input_format.q | 24 + .../clientpositive/table_access_keys_stats.q | 218 + .../clientpositive/tablename_with_select.q | 9 + .../clientpositive/test_boolean_whereclause.q | 9 + .../test/queries/clientpositive/timestamp_1.q | 72 + .../test/queries/clientpositive/timestamp_2.q | 72 + .../test/queries/clientpositive/timestamp_3.q | 17 + .../clientpositive/timestamp_comparison.q | 28 + .../queries/clientpositive/timestamp_lazy.q | 6 + .../queries/clientpositive/timestamp_null.q | 7 + .../queries/clientpositive/timestamp_udf.q | 58 + .../src/test/queries/clientpositive/touch.q | 17 + .../test/queries/clientpositive/transform1.q | 23 + .../test/queries/clientpositive/transform2.q | 2 + .../queries/clientpositive/transform_ppr1.q | 19 + .../queries/clientpositive/transform_ppr2.q | 21 + .../queries/clientpositive/truncate_column.q | 79 + .../clientpositive/truncate_column_buckets.q | 23 + .../truncate_column_list_bucket.q | 33 + .../clientpositive/truncate_column_merge.q | 20 + .../queries/clientpositive/truncate_table.q | 35 + .../test/queries/clientpositive/type_cast_1.q | 4 + .../clientpositive/type_conversions_1.q | 24 + .../queries/clientpositive/type_widening.q | 6 + .../queries/clientpositive/udaf_collect_set.q | 30 + .../clientpositive/udaf_context_ngrams.q | 12 + .../test/queries/clientpositive/udaf_corr.q | 16 + .../queries/clientpositive/udaf_covar_pop.q | 16 + .../queries/clientpositive/udaf_covar_samp.q | 16 + .../clientpositive/udaf_histogram_numeric.q | 5 + .../test/queries/clientpositive/udaf_ngrams.q | 12 + .../clientpositive/udaf_number_format.q | 14 + .../udaf_percentile_approx_20.q | 85 + .../udaf_percentile_approx_23.q | 86 + .../ql/src/test/queries/clientpositive/udf1.q | 24 + .../ql/src/test/queries/clientpositive/udf2.q | 8 + .../ql/src/test/queries/clientpositive/udf3.q | 10 + .../ql/src/test/queries/clientpositive/udf4.q | 51 + .../ql/src/test/queries/clientpositive/udf5.q | 13 + .../ql/src/test/queries/clientpositive/udf6.q | 23 + .../ql/src/test/queries/clientpositive/udf7.q | 22 + .../ql/src/test/queries/clientpositive/udf8.q | 10 + .../ql/src/test/queries/clientpositive/udf9.q | 20 + .../queries/clientpositive/udf_10_trims.q | 12 + .../src/test/queries/clientpositive/udf_E.q | 14 + .../src/test/queries/clientpositive/udf_PI.q | 14 + .../src/test/queries/clientpositive/udf_abs.q | 30 + .../test/queries/clientpositive/udf_acos.q | 14 + .../src/test/queries/clientpositive/udf_add.q | 2 + .../test/queries/clientpositive/udf_array.q | 8 + .../clientpositive/udf_array_contains.q | 9 + .../test/queries/clientpositive/udf_ascii.q | 14 + .../test/queries/clientpositive/udf_asin.q | 14 + .../test/queries/clientpositive/udf_atan.q | 16 + .../src/test/queries/clientpositive/udf_avg.q | 2 + .../test/queries/clientpositive/udf_between.q | 14 + .../test/queries/clientpositive/udf_bigint.q | 2 + .../src/test/queries/clientpositive/udf_bin.q | 11 + .../queries/clientpositive/udf_bitmap_and.q | 12 + .../queries/clientpositive/udf_bitmap_empty.q | 3 + .../queries/clientpositive/udf_bitmap_or.q | 12 + .../queries/clientpositive/udf_bitwise_and.q | 2 + .../queries/clientpositive/udf_bitwise_not.q | 2 + .../queries/clientpositive/udf_bitwise_or.q | 2 + .../queries/clientpositive/udf_bitwise_xor.q | 2 + .../test/queries/clientpositive/udf_boolean.q | 2 + .../test/queries/clientpositive/udf_case.q | 63 + .../clientpositive/udf_case_column_pruning.q | 18 + .../queries/clientpositive/udf_case_thrift.q | 34 + .../test/queries/clientpositive/udf_ceil.q | 2 + .../test/queries/clientpositive/udf_ceiling.q | 2 + .../queries/clientpositive/udf_coalesce.q | 54 + .../clientpositive/udf_compare_java_string.q | 10 + .../test/queries/clientpositive/udf_concat.q | 15 + .../clientpositive/udf_concat_insert1.q | 7 + .../clientpositive/udf_concat_insert2.q | 8 + .../queries/clientpositive/udf_concat_ws.q | 43 + .../clientpositive/udf_context_aware.q | 5 + .../test/queries/clientpositive/udf_conv.q | 75 + .../src/test/queries/clientpositive/udf_cos.q | 8 + .../test/queries/clientpositive/udf_count.q | 20 + .../queries/clientpositive/udf_date_add.q | 2 + .../queries/clientpositive/udf_date_sub.q | 2 + .../queries/clientpositive/udf_datediff.q | 2 + .../src/test/queries/clientpositive/udf_day.q | 2 + .../queries/clientpositive/udf_dayofmonth.q | 2 + .../test/queries/clientpositive/udf_degrees.q | 14 + .../src/test/queries/clientpositive/udf_div.q | 4 + .../test/queries/clientpositive/udf_divide.q | 4 + .../test/queries/clientpositive/udf_double.q | 2 + .../src/test/queries/clientpositive/udf_elt.q | 29 + .../test/queries/clientpositive/udf_equal.q | 12 + .../src/test/queries/clientpositive/udf_exp.q | 2 + .../test/queries/clientpositive/udf_explode.q | 22 + .../test/queries/clientpositive/udf_field.q | 55 + .../queries/clientpositive/udf_find_in_set.q | 23 + .../test/queries/clientpositive/udf_float.q | 2 + .../test/queries/clientpositive/udf_floor.q | 2 + .../clientpositive/udf_format_number.q | 57 + .../clientpositive/udf_from_unixtime.q | 2 + .../clientpositive/udf_get_json_object.q | 40 + .../queries/clientpositive/udf_greaterthan.q | 4 + .../clientpositive/udf_greaterthanorequal.q | 4 + .../test/queries/clientpositive/udf_hash.q | 17 + .../src/test/queries/clientpositive/udf_hex.q | 20 + .../test/queries/clientpositive/udf_hour.q | 13 + .../src/test/queries/clientpositive/udf_if.q | 34 + .../src/test/queries/clientpositive/udf_in.q | 13 + .../test/queries/clientpositive/udf_in_file.q | 12 + .../test/queries/clientpositive/udf_index.q | 2 + .../test/queries/clientpositive/udf_inline.q | 26 + .../test/queries/clientpositive/udf_instr.q | 33 + .../src/test/queries/clientpositive/udf_int.q | 2 + .../queries/clientpositive/udf_isnotnull.q | 2 + .../test/queries/clientpositive/udf_isnull.q | 2 + .../clientpositive/udf_isnull_isnotnull.q | 37 + .../queries/clientpositive/udf_java_method.q | 25 + .../test/queries/clientpositive/udf_lcase.q | 3 + .../test/queries/clientpositive/udf_length.q | 14 + .../queries/clientpositive/udf_lessthan.q | 4 + .../clientpositive/udf_lessthanorequal.q | 4 + .../test/queries/clientpositive/udf_like.q | 21 + .../src/test/queries/clientpositive/udf_ln.q | 2 + .../test/queries/clientpositive/udf_locate.q | 41 + .../src/test/queries/clientpositive/udf_log.q | 2 + .../test/queries/clientpositive/udf_log10.q | 2 + .../test/queries/clientpositive/udf_log2.q | 2 + .../clientpositive/udf_logic_java_boolean.q | 28 + .../test/queries/clientpositive/udf_lower.q | 7 + .../test/queries/clientpositive/udf_lpad.q | 14 + .../test/queries/clientpositive/udf_ltrim.q | 2 + .../src/test/queries/clientpositive/udf_map.q | 8 + .../queries/clientpositive/udf_map_keys.q | 11 + .../queries/clientpositive/udf_map_values.q | 11 + .../src/test/queries/clientpositive/udf_max.q | 34 + .../src/test/queries/clientpositive/udf_min.q | 34 + .../test/queries/clientpositive/udf_minute.q | 9 + .../test/queries/clientpositive/udf_modulo.q | 2 + .../test/queries/clientpositive/udf_month.q | 2 + .../queries/clientpositive/udf_named_struct.q | 9 + .../queries/clientpositive/udf_negative.q | 13 + .../src/test/queries/clientpositive/udf_not.q | 6 + .../queries/clientpositive/udf_notequal.q | 23 + .../test/queries/clientpositive/udf_notop.q | 10 + .../src/test/queries/clientpositive/udf_nvl.q | 13 + .../src/test/queries/clientpositive/udf_or.q | 2 + .../queries/clientpositive/udf_parse_url.q | 29 + .../queries/clientpositive/udf_percentile.q | 75 + .../test/queries/clientpositive/udf_pmod.q | 20 + .../queries/clientpositive/udf_positive.q | 6 + .../src/test/queries/clientpositive/udf_pow.q | 2 + .../test/queries/clientpositive/udf_power.q | 2 + .../test/queries/clientpositive/udf_printf.q | 35 + .../test/queries/clientpositive/udf_radians.q | 16 + .../test/queries/clientpositive/udf_rand.q | 2 + .../test/queries/clientpositive/udf_reflect.q | 24 + .../queries/clientpositive/udf_reflect2.q | 70 + .../test/queries/clientpositive/udf_regexp.q | 6 + .../clientpositive/udf_regexp_extract.q | 2 + .../clientpositive/udf_regexp_replace.q | 2 + .../test/queries/clientpositive/udf_repeat.q | 16 + .../test/queries/clientpositive/udf_reverse.q | 15 + .../test/queries/clientpositive/udf_rlike.q | 2 + .../test/queries/clientpositive/udf_round.q | 44 + .../test/queries/clientpositive/udf_round_2.q | 13 + .../test/queries/clientpositive/udf_round_3.q | 14 + .../test/queries/clientpositive/udf_rpad.q | 14 + .../test/queries/clientpositive/udf_rtrim.q | 2 + .../test/queries/clientpositive/udf_second.q | 9 + .../queries/clientpositive/udf_sentences.q | 22 + .../test/queries/clientpositive/udf_sign.q | 20 + .../src/test/queries/clientpositive/udf_sin.q | 8 + .../test/queries/clientpositive/udf_size.q | 20 + .../queries/clientpositive/udf_smallint.q | 2 + .../queries/clientpositive/udf_sort_array.q | 38 + .../test/queries/clientpositive/udf_space.q | 27 + .../test/queries/clientpositive/udf_split.q | 16 + .../test/queries/clientpositive/udf_sqrt.q | 2 + .../src/test/queries/clientpositive/udf_std.q | 2 + .../test/queries/clientpositive/udf_stddev.q | 2 + .../queries/clientpositive/udf_stddev_pop.q | 2 + .../queries/clientpositive/udf_stddev_samp.q | 4 + .../test/queries/clientpositive/udf_string.q | 2 + .../test/queries/clientpositive/udf_struct.q | 9 + .../test/queries/clientpositive/udf_substr.q | 75 + .../queries/clientpositive/udf_substring.q | 3 + .../queries/clientpositive/udf_subtract.q | 2 + .../src/test/queries/clientpositive/udf_sum.q | 6 + .../src/test/queries/clientpositive/udf_tan.q | 16 + .../queries/clientpositive/udf_testlength.q | 12 + .../queries/clientpositive/udf_testlength2.q | 12 + .../test/queries/clientpositive/udf_tinyint.q | 2 + .../queries/clientpositive/udf_to_boolean.q | 44 + .../test/queries/clientpositive/udf_to_byte.q | 15 + .../test/queries/clientpositive/udf_to_date.q | 2 + .../queries/clientpositive/udf_to_double.q | 15 + .../queries/clientpositive/udf_to_float.q | 15 + .../test/queries/clientpositive/udf_to_long.q | 15 + .../queries/clientpositive/udf_to_short.q | 15 + .../queries/clientpositive/udf_to_string.q | 16 + .../clientpositive/udf_to_unix_timestamp.q | 29 + .../queries/clientpositive/udf_translate.q | 37 + .../test/queries/clientpositive/udf_trim.q | 2 + .../test/queries/clientpositive/udf_ucase.q | 2 + .../test/queries/clientpositive/udf_unhex.q | 19 + .../test/queries/clientpositive/udf_union.q | 11 + .../clientpositive/udf_unix_timestamp.q | 27 + .../test/queries/clientpositive/udf_upper.q | 2 + .../test/queries/clientpositive/udf_var_pop.q | 2 + .../queries/clientpositive/udf_var_samp.q | 4 + .../queries/clientpositive/udf_variance.q | 10 + .../queries/clientpositive/udf_weekofyear.q | 6 + .../test/queries/clientpositive/udf_when.q | 57 + .../test/queries/clientpositive/udf_xpath.q | 8 + .../clientpositive/udf_xpath_boolean.q | 9 + .../queries/clientpositive/udf_xpath_double.q | 14 + .../queries/clientpositive/udf_xpath_float.q | 11 + .../queries/clientpositive/udf_xpath_int.q | 11 + .../queries/clientpositive/udf_xpath_long.q | 11 + .../queries/clientpositive/udf_xpath_short.q | 11 + .../queries/clientpositive/udf_xpath_string.q | 11 + .../queries/clientpositive/udtf_explode.q | 25 + .../queries/clientpositive/udtf_json_tuple.q | 47 + .../clientpositive/udtf_parse_url_tuple.q | 42 + .../test/queries/clientpositive/udtf_stack.q | 7 + .../queries/clientpositive/unicode_notation.q | 13 + .../src/test/queries/clientpositive/union.q | 18 + .../src/test/queries/clientpositive/union10.q | 25 + .../src/test/queries/clientpositive/union11.q | 20 + .../src/test/queries/clientpositive/union12.q | 23 + .../src/test/queries/clientpositive/union13.q | 8 + .../src/test/queries/clientpositive/union14.q | 16 + .../src/test/queries/clientpositive/union15.q | 18 + .../src/test/queries/clientpositive/union16.q | 63 + .../src/test/queries/clientpositive/union17.q | 20 + .../src/test/queries/clientpositive/union18.q | 20 + .../src/test/queries/clientpositive/union19.q | 26 + .../src/test/queries/clientpositive/union2.q | 8 + .../src/test/queries/clientpositive/union20.q | 22 + .../src/test/queries/clientpositive/union21.q | 30 + .../src/test/queries/clientpositive/union22.q | 43 + .../src/test/queries/clientpositive/union23.q | 17 + .../src/test/queries/clientpositive/union24.q | 70 + .../src/test/queries/clientpositive/union25.q | 23 + .../src/test/queries/clientpositive/union26.q | 89 + .../src/test/queries/clientpositive/union27.q | 4 + .../src/test/queries/clientpositive/union28.q | 30 + .../src/test/queries/clientpositive/union29.q | 30 + .../src/test/queries/clientpositive/union3.q | 42 + .../src/test/queries/clientpositive/union30.q | 44 + .../src/test/queries/clientpositive/union31.q | 97 + .../src/test/queries/clientpositive/union32.q | 75 + .../src/test/queries/clientpositive/union33.q | 47 + .../src/test/queries/clientpositive/union34.q | 44 + .../src/test/queries/clientpositive/union4.q | 21 + .../src/test/queries/clientpositive/union5.q | 12 + .../src/test/queries/clientpositive/union6.q | 21 + .../src/test/queries/clientpositive/union7.q | 14 + .../src/test/queries/clientpositive/union8.q | 10 + .../src/test/queries/clientpositive/union9.q | 10 + .../test/queries/clientpositive/union_date.q | 32 + .../clientpositive/union_lateralview.q | 43 + .../test/queries/clientpositive/union_null.q | 2 + .../test/queries/clientpositive/union_ppr.q | 16 + .../queries/clientpositive/union_remove_1.q | 44 + .../queries/clientpositive/union_remove_10.q | 57 + .../queries/clientpositive/union_remove_11.q | 57 + .../queries/clientpositive/union_remove_12.q | 51 + .../queries/clientpositive/union_remove_13.q | 51 + .../queries/clientpositive/union_remove_14.q | 52 + .../queries/clientpositive/union_remove_15.q | 53 + .../queries/clientpositive/union_remove_16.q | 52 + .../queries/clientpositive/union_remove_17.q | 49 + .../queries/clientpositive/union_remove_18.q | 53 + .../queries/clientpositive/union_remove_19.q | 90 + .../queries/clientpositive/union_remove_2.q | 50 + .../queries/clientpositive/union_remove_20.q | 45 + .../queries/clientpositive/union_remove_21.q | 45 + .../queries/clientpositive/union_remove_22.q | 63 + .../queries/clientpositive/union_remove_23.q | 47 + .../queries/clientpositive/union_remove_24.q | 43 + .../queries/clientpositive/union_remove_3.q | 50 + .../queries/clientpositive/union_remove_4.q | 45 + .../queries/clientpositive/union_remove_5.q | 51 + .../queries/clientpositive/union_remove_6.q | 41 + .../queries/clientpositive/union_remove_7.q | 46 + .../queries/clientpositive/union_remove_8.q | 51 + .../queries/clientpositive/union_remove_9.q | 55 + .../queries/clientpositive/union_script.q | 7 + .../test/queries/clientpositive/union_view.q | 84 + .../test/queries/clientpositive/uniquejoin.q | 25 + .../unset_table_view_property.q | 64 + .../queries/clientpositive/updateAccessTime.q | 30 + .../test/queries/clientpositive/varchar_1.q | 32 + .../test/queries/clientpositive/varchar_2.q | 36 + .../queries/clientpositive/varchar_cast.q | 84 + .../clientpositive/varchar_comparison.q | 40 + .../queries/clientpositive/varchar_join1.q | 35 + .../clientpositive/varchar_nested_types.q | 53 + .../queries/clientpositive/varchar_serde.q | 102 + .../queries/clientpositive/varchar_udf1.q | 156 + .../queries/clientpositive/varchar_union1.q | 47 + .../ql/src/test/queries/clientpositive/view.q | 51 + .../test/queries/clientpositive/view_cast.q | 16 + .../test/queries/clientpositive/view_inputs.q | 7 + .../queries/clientpositive/virtual_column.q | 19 + .../test/queries/clientpositive/windowing.q | 434 + .../windowing_adjust_rowcontainer_sz.q | 27 + .../clientpositive/windowing_columnPruning.q | 32 + .../clientpositive/windowing_expressions.q | 70 + .../windowing_multipartitioning.q | 42 + .../queries/clientpositive/windowing_navfn.q | 31 + .../queries/clientpositive/windowing_ntile.q | 28 + .../queries/clientpositive/windowing_rank.q | 66 + .../queries/clientpositive/windowing_udaf.q | 28 + .../clientpositive/windowing_windowspec.q | 36 + .../queries/negative/ambiguous_join_col.q | 2 + .../test/queries/negative/duplicate_alias.q | 2 + .../ql/src/test/queries/negative/garbage.q | 1 + .../negative/insert_wrong_number_columns.q | 2 + .../queries/negative/invalid_create_table.q | 4 + .../src/test/queries/negative/invalid_dot.q | 2 + .../negative/invalid_function_param2.q | 2 + .../src/test/queries/negative/invalid_index.q | 2 + .../queries/negative/invalid_list_index.q | 2 + .../queries/negative/invalid_list_index2.q | 2 + .../test/queries/negative/invalid_map_index.q | 2 + .../queries/negative/invalid_map_index2.q | 2 + .../test/queries/negative/invalid_select.q | 4 + .../queries/negative/macro_reserved_word.q | 1 + .../test/queries/negative/missing_overwrite.q | 2 + .../test/queries/negative/nonkey_groupby.q | 2 + .../src/test/queries/negative/quoted_string.q | 2 + .../test/queries/negative/unknown_column1.q | 2 + .../test/queries/negative/unknown_column2.q | 2 + .../test/queries/negative/unknown_column3.q | 2 + .../test/queries/negative/unknown_column4.q | 2 + .../test/queries/negative/unknown_column5.q | 2 + .../test/queries/negative/unknown_column6.q | 2 + .../test/queries/negative/unknown_function1.q | 2 + .../test/queries/negative/unknown_function2.q | 2 + .../test/queries/negative/unknown_function3.q | 2 + .../test/queries/negative/unknown_function4.q | 2 + .../test/queries/negative/unknown_table1.q | 2 + .../test/queries/negative/unknown_table2.q | 2 + .../test/queries/negative/wrong_distinct1.q | 2 + .../test/queries/negative/wrong_distinct2.q | 2 + .../test/queries/positive/case_sensitivity.q | 2 + .../hive/ql/src/test/queries/positive/cast1.q | 2 + .../ql/src/test/queries/positive/groupby1.q | 2 + .../ql/src/test/queries/positive/groupby2.q | 2 + .../ql/src/test/queries/positive/groupby3.q | 2 + .../ql/src/test/queries/positive/groupby4.q | 2 + .../ql/src/test/queries/positive/groupby5.q | 4 + .../ql/src/test/queries/positive/groupby6.q | 2 + .../ql/src/test/queries/positive/input1.q | 2 + .../ql/src/test/queries/positive/input2.q | 4 + .../ql/src/test/queries/positive/input20.q | 9 + .../ql/src/test/queries/positive/input3.q | 5 + .../ql/src/test/queries/positive/input4.q | 7 + .../ql/src/test/queries/positive/input5.q | 7 + .../ql/src/test/queries/positive/input6.q | 2 + .../ql/src/test/queries/positive/input7.q | 2 + .../ql/src/test/queries/positive/input8.q | 2 + .../ql/src/test/queries/positive/input9.q | 2 + .../src/test/queries/positive/input_part1.q | 2 + .../queries/positive/input_testsequencefile.q | 2 + .../test/queries/positive/input_testxpath.q | 2 + .../test/queries/positive/input_testxpath2.q | 2 + .../hive/ql/src/test/queries/positive/join1.q | 3 + .../hive/ql/src/test/queries/positive/join2.q | 3 + .../hive/ql/src/test/queries/positive/join3.q | 4 + .../hive/ql/src/test/queries/positive/join4.q | 14 + .../hive/ql/src/test/queries/positive/join5.q | 15 + .../hive/ql/src/test/queries/positive/join6.q | 16 + .../hive/ql/src/test/queries/positive/join7.q | 21 + .../hive/ql/src/test/queries/positive/join8.q | 14 + .../ql/src/test/queries/positive/sample1.q | 5 + .../ql/src/test/queries/positive/sample2.q | 4 + .../ql/src/test/queries/positive/sample3.q | 4 + .../ql/src/test/queries/positive/sample4.q | 4 + .../ql/src/test/queries/positive/sample5.q | 3 + .../ql/src/test/queries/positive/sample6.q | 3 + .../ql/src/test/queries/positive/sample7.q | 4 + .../hive/ql/src/test/queries/positive/subq.q | 4 + .../hive/ql/src/test/queries/positive/udf1.q | 5 + .../hive/ql/src/test/queries/positive/udf4.q | 1 + .../hive/ql/src/test/queries/positive/udf6.q | 1 + .../ql/src/test/queries/positive/udf_case.q | 10 + .../ql/src/test/queries/positive/udf_when.q | 10 + .../hive/ql/src/test/queries/positive/union.q | 6 + .../execution/HiveCompatibility.scala | 2 +- 2083 files changed, 67866 insertions(+), 14 deletions(-) create mode 100644 src/test/hive/data/conf/hive-log4j.properties create mode 100644 src/test/hive/data/conf/hive-site.xml create mode 100644 src/test/hive/data/files/SortCol1Col2.txt create mode 100644 src/test/hive/data/files/SortCol2Col1.txt create mode 100644 src/test/hive/data/files/SortDescCol1Col2.txt create mode 100644 src/test/hive/data/files/SortDescCol2Col1.txt create mode 100644 src/test/hive/data/files/T1.txt create mode 100644 src/test/hive/data/files/T2.txt create mode 100644 src/test/hive/data/files/T3.txt create mode 100644 src/test/hive/data/files/TestSerDe.jar create mode 100644 src/test/hive/data/files/UserVisits.dat create mode 100644 src/test/hive/data/files/apache.access.2.log create mode 100644 src/test/hive/data/files/apache.access.log create mode 100644 src/test/hive/data/files/archive_corrupt.rc create mode 100644 src/test/hive/data/files/array_table.txt create mode 100644 src/test/hive/data/files/binary.txt create mode 100644 src/test/hive/data/files/bool.txt create mode 100644 src/test/hive/data/files/complex.seq create mode 100644 src/test/hive/data/files/covar_tab.txt create mode 100644 src/test/hive/data/files/create_nested_type.txt create mode 100644 src/test/hive/data/files/csv.txt create mode 100644 src/test/hive/data/files/datatypes.txt create mode 100644 src/test/hive/data/files/dim-data.txt create mode 100644 src/test/hive/data/files/doctors.avro create mode 100644 src/test/hive/data/files/docurl.txt create mode 100644 src/test/hive/data/files/double.txt create mode 100644 src/test/hive/data/files/employee.dat create mode 100644 src/test/hive/data/files/employee2.dat create mode 100644 src/test/hive/data/files/employee_part.txt create mode 100644 src/test/hive/data/files/empty1.txt create mode 100644 src/test/hive/data/files/empty2.txt create mode 100644 src/test/hive/data/files/episodes.avro create mode 100644 src/test/hive/data/files/escapetest.txt create mode 100644 src/test/hive/data/files/ext_test/test.dat create mode 100644 src/test/hive/data/files/fact-data.txt create mode 100644 src/test/hive/data/files/flights_join.txt create mode 100644 src/test/hive/data/files/flights_tiny.txt create mode 100644 src/test/hive/data/files/flights_tiny.txt.1 create mode 100644 src/test/hive/data/files/groupby_groupingid.txt create mode 100644 src/test/hive/data/files/grouping_sets.txt create mode 100644 src/test/hive/data/files/grouping_sets1.txt create mode 100644 src/test/hive/data/files/grouping_sets2.txt create mode 100644 src/test/hive/data/files/hive_626_bar.txt create mode 100644 src/test/hive/data/files/hive_626_count.txt create mode 100644 src/test/hive/data/files/hive_626_foo.txt create mode 100644 src/test/hive/data/files/in1.txt create mode 100644 src/test/hive/data/files/in2.txt create mode 100644 src/test/hive/data/files/in3.txt create mode 100644 src/test/hive/data/files/in4.txt create mode 100644 src/test/hive/data/files/in5.txt create mode 100644 src/test/hive/data/files/in6.txt create mode 100644 src/test/hive/data/files/in7.txt create mode 100644 src/test/hive/data/files/in8.txt create mode 100644 src/test/hive/data/files/in9.txt create mode 100644 src/test/hive/data/files/infer_const_type.txt create mode 100644 src/test/hive/data/files/int.txt create mode 100644 src/test/hive/data/files/json.txt create mode 100644 src/test/hive/data/files/kv1.seq create mode 100644 src/test/hive/data/files/kv1.string-sorted.txt create mode 100644 src/test/hive/data/files/kv1.txt create mode 100644 src/test/hive/data/files/kv1.val.sorted.txt create mode 100644 src/test/hive/data/files/kv1_broken.seq create mode 100644 src/test/hive/data/files/kv1_cb.txt create mode 100644 src/test/hive/data/files/kv1_cc.txt create mode 100644 src/test/hive/data/files/kv1kv2.cogroup.txt create mode 100644 src/test/hive/data/files/kv2.txt create mode 100644 src/test/hive/data/files/kv3.txt create mode 100644 src/test/hive/data/files/kv4.txt create mode 100644 src/test/hive/data/files/kv5.txt create mode 100644 src/test/hive/data/files/kv6.txt create mode 100644 src/test/hive/data/files/kv7.txt create mode 100644 src/test/hive/data/files/kv8.txt create mode 100644 src/test/hive/data/files/leftsemijoin_mr_t1.txt create mode 100644 src/test/hive/data/files/leftsemijoin_mr_t2.txt create mode 100644 src/test/hive/data/files/lineitem.txt create mode 100644 src/test/hive/data/files/lt100.sorted.txt create mode 100644 src/test/hive/data/files/lt100.txt create mode 100644 src/test/hive/data/files/lt100.txt.deflate create mode 100644 src/test/hive/data/files/map_table.txt create mode 100644 src/test/hive/data/files/nested_complex.txt create mode 100644 src/test/hive/data/files/null.txt create mode 100644 src/test/hive/data/files/nullfile.txt create mode 100644 src/test/hive/data/files/nulls.txt create mode 100644 src/test/hive/data/files/orc_create.txt create mode 100644 src/test/hive/data/files/orc_create_people.txt create mode 100644 src/test/hive/data/files/over10k create mode 100644 src/test/hive/data/files/over1k create mode 100644 src/test/hive/data/files/part.rc create mode 100644 src/test/hive/data/files/part.seq create mode 100644 src/test/hive/data/files/part_tiny.txt create mode 100644 src/test/hive/data/files/person age.txt create mode 100644 src/test/hive/data/files/primitive_type_arrays.txt create mode 100644 src/test/hive/data/files/pw17.txt create mode 100644 src/test/hive/data/files/sales.txt create mode 100644 src/test/hive/data/files/sample-queryplan-in-history.txt create mode 100644 src/test/hive/data/files/sample-queryplan.txt create mode 100644 src/test/hive/data/files/smallsrcsortbucket1outof4.txt create mode 100644 src/test/hive/data/files/smallsrcsortbucket2outof4.txt create mode 100644 src/test/hive/data/files/smallsrcsortbucket3outof4.txt create mode 100644 src/test/hive/data/files/smallsrcsortbucket4outof4.txt create mode 100644 src/test/hive/data/files/smb_bucket_input.rc create mode 100644 src/test/hive/data/files/smb_bucket_input.txt create mode 100644 src/test/hive/data/files/smbbucket_1.rc create mode 100644 src/test/hive/data/files/smbbucket_1.txt create mode 100644 src/test/hive/data/files/smbbucket_2.rc create mode 100644 src/test/hive/data/files/smbbucket_2.txt create mode 100644 src/test/hive/data/files/smbbucket_3.rc create mode 100644 src/test/hive/data/files/smbbucket_3.txt create mode 100644 src/test/hive/data/files/source.txt create mode 100644 src/test/hive/data/files/srcbucket0.txt create mode 100644 src/test/hive/data/files/srcbucket1.txt create mode 100644 src/test/hive/data/files/srcbucket20.txt create mode 100644 src/test/hive/data/files/srcbucket21.txt create mode 100644 src/test/hive/data/files/srcbucket22.txt create mode 100644 src/test/hive/data/files/srcbucket23.txt create mode 100644 src/test/hive/data/files/srcsortbucket1outof4.txt create mode 100644 src/test/hive/data/files/srcsortbucket2outof4.txt create mode 100644 src/test/hive/data/files/srcsortbucket3outof4.txt create mode 100644 src/test/hive/data/files/srcsortbucket4outof4.txt create mode 100644 src/test/hive/data/files/string.txt create mode 100644 src/test/hive/data/files/symlink1.txt create mode 100644 src/test/hive/data/files/symlink2.txt create mode 100644 src/test/hive/data/files/tbl.txt create mode 100644 src/test/hive/data/files/test.dat create mode 100644 src/test/hive/data/files/test2.dat create mode 100644 src/test/hive/data/files/text-en.txt create mode 100644 src/test/hive/data/files/things.txt create mode 100644 src/test/hive/data/files/things2.txt create mode 100644 src/test/hive/data/files/tiny_a.txt create mode 100644 src/test/hive/data/files/tiny_b.txt create mode 100644 src/test/hive/data/files/types/primitives/090101.txt create mode 100644 src/test/hive/data/files/types/primitives/090201.txt create mode 100644 src/test/hive/data/files/types/primitives/090301.txt create mode 100644 src/test/hive/data/files/types/primitives/090401.txt create mode 100644 src/test/hive/data/files/union_input.txt create mode 100644 src/test/hive/data/files/v1.txt create mode 100644 src/test/hive/data/files/v2.txt create mode 100644 src/test/hive/data/files/vc1.txt create mode 100644 src/test/hive/data/files/x.txt create mode 100644 src/test/hive/data/files/y.txt create mode 100644 src/test/hive/data/files/z.txt create mode 100644 src/test/hive/data/metadb/.gitignore create mode 100644 src/test/hive/data/scripts/cat.py create mode 100644 src/test/hive/data/scripts/cat_error.py create mode 100644 src/test/hive/data/scripts/doubleescapedtab.py create mode 100644 src/test/hive/data/scripts/dumpdata_script.py create mode 100755 src/test/hive/data/scripts/error_script create mode 100644 src/test/hive/data/scripts/escapedcarriagereturn.py create mode 100644 src/test/hive/data/scripts/escapednewline.py create mode 100644 src/test/hive/data/scripts/escapedtab.py create mode 100755 src/test/hive/data/scripts/input20_script create mode 100644 src/test/hive/data/scripts/newline.py create mode 100644 src/test/hive/data/scripts/q_test_cleanup.sql create mode 100644 src/test/hive/data/scripts/q_test_init.sql create mode 100644 src/test/hive/data/scripts/test_init_file.sql create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/addpart1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_non_native.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_partition_offline.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_table_add_partition.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure6.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure7.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure8.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure9.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/altern1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col0.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/analyze.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/analyze1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/analyze_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_corrupt.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_insert1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_insert2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_insert3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_insert4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_multi1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_multi2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_multi3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_multi4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_multi5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_multi6.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_multi7.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_partspec1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_partspec2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_partspec3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_partspec4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/archive_partspec5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_7.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/authorization_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/autolocal1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/bad_exec_hooks.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/bad_indextype.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/bad_sample_clause.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/cachingprintstream.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/clusterbydistributeby.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/clusterbyorderby.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/clusterbysortby.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/clustern1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/clustern2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/clustern3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/clustern4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/column_rename1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/column_rename2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/column_rename3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/column_rename4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/column_rename5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/compare_double_bigint.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/compare_string_bigint.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_insert_outputformat.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view6.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view7.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view8.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_table_failure1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_table_failure2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_table_failure3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_table_failure4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_table_wrong_regex.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_udaf_failure.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_unknown_genericudf.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_view_failure1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_view_failure2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_view_failure3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_view_failure4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_view_failure5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_view_failure6.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_view_failure7.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_view_failure8.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/create_view_failure9.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ctas.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/database_create_already_exists.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/database_create_invalid_name.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/date_literal1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/date_literal2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/date_literal3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ddltime.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/decimal_precision.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/decimal_precision_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/default_partition_name.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/deletejar.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/desc_failure1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/desc_failure2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/desc_failure3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/describe_xpath1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/describe_xpath2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/describe_xpath3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/describe_xpath4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_function_failure.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_index_failure.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_native_udf.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_partition_failure.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/dyn_part1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/dyn_part2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/dyn_part3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/dyn_part4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_22_export_authfail.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/external1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/external2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/fetchtask_ioexception.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/fileformat_bad_class.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_input.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_output.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/fs_default_name1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/fs_default_name2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/genericFileFormat.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_cube1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_cube2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_id1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_invalid_position.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_key.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/having1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/index_compact_entry_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/index_compact_size_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/input1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/input2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/input4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/input41.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/input_part0_neg.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/insert_into1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/insert_into2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/insert_into3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/insert_into4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/insert_view_failure.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/insertexternal1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_avg_syntax.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_columns.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_config1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_config2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_mapjoin1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_max_syntax.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_min_syntax.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_select_expression.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_std_syntax.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_sum_syntax.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_t_create2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_t_transform.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_tbl_name.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalid_variance_syntax.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/invalidate_view1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/join2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/join28.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/join29.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/join32.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/join35.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/join_nonexistent_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/joinneg.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/lateral_view_alias.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/lateral_view_join.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/line_terminator.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_exist_part_authfail.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_non_native.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_nonpart_authfail.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_part_authfail.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_part_nospec.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_stored_as_dirs.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_view_failure.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/load_wrong_noof_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/local_mapred_error_cache.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/lockneg1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/lockneg2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/lockneg3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/lockneg4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/lockneg5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/macro_unused_parameter.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/merge_negative_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/merge_negative_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/merge_negative_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/minimr_broken_pipe.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/nested_complex_neg.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/no_matching_udf.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/nonkey_groupby.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/nopart_insert.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/nopart_load.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/notable_alias3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/notable_alias4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/orderby_invalid_position.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/orderby_position_unsupported.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/orderbysortby.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/part_col_complex_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_part1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_part2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl6.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl7.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl8.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/recursive_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/regex_col_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/regex_col_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/regex_col_groupby.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/sa_fail_hook3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/sample.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/script_error.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/select_charliteral.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/select_udtf_alias.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/semijoin1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/semijoin2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/semijoin3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/semijoin4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/serde_regex.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/serde_regex2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/serde_regex3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/set_table_property.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_columns1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_columns2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_columns3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_create_table_index.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_partitions1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_tableproperties1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/smb_mapjoin_14.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/split_sample_out_of_range.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/stats_noscan_non_native.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/strict_join.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/strict_orderby.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/strict_pruning.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/subq_insert.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/touch1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/touch2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_bucketed_column.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_column_seqfile.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udaf_invalid_place.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_coalesce.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_if_not_bool.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_in.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_max.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_min.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_reflect_neg.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_test_error.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_test_error_reduce.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udfnull.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udtf_invalid_place.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported1.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/union.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/union2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/union22.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/union3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/uniquejoin.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/uniquejoin2.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/uniquejoin3.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/unset_table_property.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/unset_view_property.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/windowing_ll_no_over.q create mode 100644 src/test/hive/ql/src/test/queries/clientnegative/wrong_column_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/add_part_exist.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/add_part_multiple.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alias_casted_column.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/allcolref_in_udf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_index.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_merge.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_merge_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_merge_stats.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_partition_coltype.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_partition_format_loc.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_skewed_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_table_not_sorted.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_varchar1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_varchar2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_view_as_select.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/alter_view_rename.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ambiguous_col.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/archive.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/archive_corrupt.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/archive_multi.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/authorization_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/authorization_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/authorization_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/authorization_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/authorization_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/authorization_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/authorization_7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/authorization_8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join0.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join16.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join17.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join18.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join19.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join21.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join22.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join23.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join24.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join25.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join26.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join27.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join28.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join29.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join30.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join31.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join32.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join_filters.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join_nulls.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_join_reordering_values.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/autogen_colalias.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/avro_change_schema.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/avro_compression_enabled.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/avro_evolved_schemas.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/avro_joins.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/avro_nullable_fields.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/avro_partitioned.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/avro_sanity_test.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/avro_schema_error_message.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/avro_schema_literal.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ba_table1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ba_table2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ba_table3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ba_table_udfs.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ba_table_union.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/binary_constant.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/binary_output_format.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/binary_table_bincolserde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/binary_table_colserde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/binarysortable_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucket1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucket2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucket3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucket4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucket5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucket_groupby.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/case_sensitivity.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/cast1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/cast_to_int.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/cluster.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/column_access_stats.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/columnstats_partlvl.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/columnstats_tbllvl.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/combine1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/combine2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/combine2_hadoop20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/combine2_win.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/combine3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/compute_stats_binary.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/compute_stats_boolean.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/compute_stats_double.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/compute_stats_empty_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/compute_stats_long.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/compute_stats_string.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/constant_prop.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/convert_enum_to_string.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/count.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/cp_mj_rc.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_big_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_default_prop.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_escape.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_genericudaf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_genericudf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_insert_outputformat.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_like.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_like2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_like_tbl_props.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_like_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_merge_compressed.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_nested_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_or_replace_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_skewed_table1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_struct_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_udaf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_union_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_view_partitioned.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/create_view_translate.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/cross_join.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ct_case_insensitive.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ctas.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ctas_colname.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ctas_date.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ctas_hadoop20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ctas_uses_database_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ctas_varchar.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/custom_input_output_format.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/database.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/database_drop.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/database_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/database_properties.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/date_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/date_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/date_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/date_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/date_comparison.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/date_join1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/date_serde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/date_udf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ddltime.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/decimal_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/decimal_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/decimal_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/decimal_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/decimal_join.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/decimal_precision.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/decimal_serde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/decimal_udf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/default_partition_name.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/delimiter.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/describe_comment_indent.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/describe_comment_nonascii.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/describe_database_json.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/describe_pretty.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/describe_syntax.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/describe_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/describe_table_json.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/describe_xpath.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/diff_part_input_formats.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/disable_file_format_check.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/driverhook.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_function.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_index.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_multi_partitions.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_table2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_udf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/drop_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/enforce_order.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/escape1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/escape2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/escape_clusterby1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/escape_distributeby1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/escape_orderby1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/escape_sortby1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exchange_partition.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exchange_partition2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exchange_partition3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_01_nonpart.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_02_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_04_all_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_05_some_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_06_one_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_10_external_managed.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_11_managed_external.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_12_external_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_13_managed_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_15_external_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_16_part_external.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_17_part_managed.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_18_part_external.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_19_part_external_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/explain_dependency.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/explain_dependency2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/explain_logical.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/explode_null.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/fetch_aggregation.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/fileformat_mix.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/fileformat_sequencefile.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/fileformat_text.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/global_limit.q create mode 100755 src/test/hive/ql/src/test/queries/clientpositive/groupby1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby1_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby1_map.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_nomap.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_skew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby1_noskew.q create mode 100755 src/test/hive/ql/src/test/queries/clientpositive/groupby2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby2_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby2_map.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_skew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q create mode 100755 src/test/hive/ql/src/test/queries/clientpositive/groupby3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby3_map.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_skew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q create mode 100755 src/test/hive/ql/src/test/queries/clientpositive/groupby4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby4_map.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby4_map_skew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby4_noskew.q create mode 100755 src/test/hive/ql/src/test/queries/clientpositive/groupby5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby5_map.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby5_map_skew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby5_noskew.q create mode 100755 src/test/hive/ql/src/test/queries/clientpositive/groupby6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby6_map.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby6_map_skew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby6_noskew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby7_map.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_skew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby8_map.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby8_map_skew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby8_noskew.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_bigdata.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_cube1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_neg_float.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_position.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_ppd.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_rollup1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_test_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/having.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/hook_context_cs.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/hook_order.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/implicit_cast1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auth.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auto.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auto_empty.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auto_file_format.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auto_multiple.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auto_partitioned.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auto_self_join.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auto_unused.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_auto_update.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_bitmap.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_bitmap1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_bitmap2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_bitmap3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_compression.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_rc.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_compact.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_compact_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_compact_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_compact_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_compact_binary_search.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_compression.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_creation.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_serde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_stale.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/index_stale_partitioned.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/infer_const_type.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/init_file.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/innerjoin.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/inoutdriver.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input0.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input11_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input12_hadoop20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input14_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input16.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input16_cc.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input17.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input18.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input19.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input1_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input21.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input22.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input23.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input24.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input25.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input26.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input28.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input2_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input30.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input31.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input32.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input33.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input34.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input35.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input36.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input37.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input38.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input39.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input39_hadoop20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input3_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input40.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input41.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input42.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input43.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input44.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input45.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input46.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input49.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input4_cb_delim.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input4_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_columnarserde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_dfs.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_dynamicserde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_lazyserde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_limit.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part0.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part10_win.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_part9.q create mode 100755 src/test/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q create mode 100755 src/test/hive/ql/src/test/queries/clientpositive/input_testxpath.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_testxpath2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_testxpath3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/input_testxpath4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/inputddl1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/inputddl2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/inputddl3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/inputddl4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/inputddl5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/inputddl6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/inputddl7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/inputddl8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert_compressed.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert_into1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert_into2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert_into3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert_into4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert_into5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert_into6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/insertexternal1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join0.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join14_hadoop20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join16.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join17.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join18.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join18_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join19.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join21.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join22.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join23.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join24.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join25.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join26.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join27.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join28.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join29.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join30.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join31.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join32.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join32_lessSize.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join33.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join34.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join35.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join36.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join37.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join38.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join39.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join40.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join41.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_1to1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_array.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_casesensitive.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_empty.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_filters.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_filters_overlap.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_hive_626.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_literals.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_map_ppr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_nulls.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_nullsafe.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_rc.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_reorder.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_reorder2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_reorder3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_reorder4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_star.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_thrift.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_vc.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/join_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/keyword_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/lateral_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/lateral_view_cp.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/lateral_view_noalias.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/lateral_view_outer.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/lateral_view_ppd.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/leadlag.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/leadlag_queries.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin_mr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown_negative.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/lineage1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/literal_decimal.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/literal_double.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/literal_ints.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/literal_string.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_binary_data.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14_win.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_fs.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_fs2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_overwrite.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/load_part_authsuccess.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/loadpart1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/loadpart_err.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/lock1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/lock2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/lock3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/lock4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/louter_join_ppr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/macro.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapjoin1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapjoin_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapjoin_hook.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapjoin_test_outer.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapreduce1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapreduce2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapreduce3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapreduce4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapreduce5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapreduce6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapreduce7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mapreduce8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/merge1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/merge2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/merge3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/merge4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mergejoins.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mergejoins_mixed.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/metadata_export_drop.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/metadataonly1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/mi.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/misc_json.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multi_insert.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multi_join_union.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multi_sahooks.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/multigroupby_singlemr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nested_complex.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nestedvirtual.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/newline.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/no_hooks.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/noalias_subq1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nonmr_fetch.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/notable_alias1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/notable_alias2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/null_cast.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/null_column.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nullgroup.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nullgroup2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nullgroup3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nullgroup4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nullgroup5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nullinput.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nullinput2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/nullscript.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/num_op_type_conv.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ops_comparison.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/optional_outer.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/optrstat_groupby.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/orc_create.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/orc_createas1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/orc_diff_part_cols.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/orc_empty_files.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/orc_empty_strings.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/order.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/order2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/outer_join_ppr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/overridden_confs.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/parallel.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/parallel_orderby.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/parenthesis_star_by.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partInit.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partcols1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_date.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_date2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_decode_name.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_schema1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_serde_format.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_special_char.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_type_check.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_varchar1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/partitions_json.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/pcr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/plan_json.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_clusterby.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_expr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_where.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_gby.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_gby2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_gby_join.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_join.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_join2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_join3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_join_filter.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_multi_insert.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_random.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_repeated_alias.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_transform.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_case.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_col.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_union.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_union_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppd_vc.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/print_header.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/progress_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/protectmode.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/protectmode2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ptf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ptf_decimal.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ptf_general_queries.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ptf_matchpath.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ptf_rcfile.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ptf_register_tblfn.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ptf_seqfile.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/push_or.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/query_properties.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/query_result_fileformat.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/query_with_semi.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/quote1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/quote2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_bigdata.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_columnar.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_createas1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_default_format.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_null_value.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rcfile_union.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/recursive_dir.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/reducesink_dedup.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/regex_col.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/regexp_extract.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/remote_script.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rename_column.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rename_external_partition_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rename_partition_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/rename_table_location.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/repair.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/reset_conf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/router_join_ppr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/script_env_var1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/script_env_var2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/script_pipe.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/scriptfile1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/select_as_omitted.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/select_transform_hint.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/select_unquote_and.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/select_unquote_not.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/select_unquote_or.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/semicolon.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/semijoin.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/serde_regex.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/serde_reported_schema.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/serde_user_properties.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/set_processor_namespaces.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/set_variable_sub.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_columns.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_create_table_alter.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_create_table_db_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_create_table_delimited.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_create_table_partitioned.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_create_table_serde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_create_table_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_describe_func_quotes.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_functions.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_indexes_syntax.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_partitions.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_tables.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_tablestatus.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/show_tblproperties.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/showparts.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoin.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt16.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt17.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt18.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt19.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_16.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_17.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_18.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_19.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_21.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_22.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_25.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sort.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/source.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/split_sample.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats0.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats16.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats18.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats19.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats_empty_partition.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1_23.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/stats_publisher_error_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/str_to_map.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/subq.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/subq2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/symlink_text_input_format.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/table_access_keys_stats.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/tablename_with_select.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/test_boolean_whereclause.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/timestamp_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/timestamp_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/timestamp_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/timestamp_comparison.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/timestamp_lazy.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/timestamp_null.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/timestamp_udf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/touch.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/transform1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/transform2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/transform_ppr1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/transform_ppr2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/truncate_column.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/truncate_column_buckets.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/truncate_column_merge.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/truncate_table.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/type_cast_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/type_conversions_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/type_widening.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udaf_collect_set.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udaf_context_ngrams.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udaf_corr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_pop.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_samp.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udaf_ngrams.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udaf_number_format.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_10_trims.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_E.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_PI.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_abs.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_acos.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_add.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_array.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_array_contains.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_ascii.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_asin.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_atan.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_avg.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_between.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_bigint.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_bin.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_and.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_empty.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_or.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_and.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_not.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_or.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_xor.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_boolean.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_case.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_case_column_pruning.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_case_thrift.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_ceil.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_ceiling.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_coalesce.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_compare_java_string.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_concat.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_concat_ws.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_context_aware.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_conv.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_cos.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_count.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_date_add.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_date_sub.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_datediff.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_day.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_dayofmonth.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_degrees.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_div.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_divide.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_double.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_elt.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_equal.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_exp.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_explode.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_field.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_find_in_set.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_float.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_floor.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_format_number.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_from_unixtime.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_get_json_object.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthan.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_hash.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_hex.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_hour.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_if.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_in.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_in_file.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_index.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_inline.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_instr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_int.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_isnotnull.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_isnull.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_java_method.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_lcase.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_length.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_lessthan.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_lessthanorequal.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_like.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_ln.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_locate.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_log.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_log10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_log2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_lower.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_lpad.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_ltrim.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_map.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_map_keys.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_map_values.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_max.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_min.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_minute.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_modulo.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_month.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_named_struct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_negative.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_not.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_notequal.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_notop.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_nvl.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_or.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_parse_url.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_percentile.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_pmod.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_positive.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_pow.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_power.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_printf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_radians.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_rand.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_reflect.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_reflect2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_regexp.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_extract.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_replace.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_repeat.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_reverse.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_rlike.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_round.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_round_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_round_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_rpad.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_rtrim.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_second.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_sentences.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_sign.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_sin.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_size.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_smallint.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_sort_array.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_space.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_split.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_sqrt.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_std.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_stddev.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_pop.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_samp.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_string.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_struct.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_substr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_substring.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_subtract.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_sum.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_tan.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_testlength.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_testlength2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_tinyint.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_to_boolean.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_to_byte.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_to_date.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_to_double.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_to_float.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_to_long.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_to_short.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_to_string.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_translate.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_trim.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_ucase.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_unhex.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_union.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_unix_timestamp.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_upper.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_var_pop.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_var_samp.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_variance.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_weekofyear.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_when.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_xpath.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_boolean.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_double.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_float.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_int.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_long.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_short.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_string.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udtf_explode.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udtf_json_tuple.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/udtf_stack.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/unicode_notation.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union16.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union17.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union18.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union19.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union21.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union22.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union23.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union24.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union25.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union26.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union27.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union28.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union29.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union30.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union31.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union32.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union33.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union34.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_date.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_lateralview.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_null.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_ppr.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_10.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_11.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_12.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_13.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_14.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_15.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_16.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_17.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_18.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_19.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_20.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_21.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_22.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_23.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_24.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_3.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_4.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_5.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_6.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_7.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_8.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_remove_9.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_script.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/union_view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/uniquejoin.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/unset_table_view_property.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/updateAccessTime.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/varchar_1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/varchar_2.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/varchar_cast.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/varchar_comparison.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/varchar_join1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/varchar_nested_types.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/varchar_serde.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/varchar_udf1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/varchar_union1.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/view.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/view_cast.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/view_inputs.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/virtual_column.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/windowing.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/windowing_columnPruning.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/windowing_expressions.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/windowing_multipartitioning.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/windowing_navfn.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/windowing_ntile.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/windowing_rank.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/windowing_udaf.q create mode 100644 src/test/hive/ql/src/test/queries/clientpositive/windowing_windowspec.q create mode 100644 src/test/hive/ql/src/test/queries/negative/ambiguous_join_col.q create mode 100644 src/test/hive/ql/src/test/queries/negative/duplicate_alias.q create mode 100644 src/test/hive/ql/src/test/queries/negative/garbage.q create mode 100644 src/test/hive/ql/src/test/queries/negative/insert_wrong_number_columns.q create mode 100644 src/test/hive/ql/src/test/queries/negative/invalid_create_table.q create mode 100644 src/test/hive/ql/src/test/queries/negative/invalid_dot.q create mode 100644 src/test/hive/ql/src/test/queries/negative/invalid_function_param2.q create mode 100644 src/test/hive/ql/src/test/queries/negative/invalid_index.q create mode 100644 src/test/hive/ql/src/test/queries/negative/invalid_list_index.q create mode 100644 src/test/hive/ql/src/test/queries/negative/invalid_list_index2.q create mode 100644 src/test/hive/ql/src/test/queries/negative/invalid_map_index.q create mode 100644 src/test/hive/ql/src/test/queries/negative/invalid_map_index2.q create mode 100644 src/test/hive/ql/src/test/queries/negative/invalid_select.q create mode 100644 src/test/hive/ql/src/test/queries/negative/macro_reserved_word.q create mode 100644 src/test/hive/ql/src/test/queries/negative/missing_overwrite.q create mode 100644 src/test/hive/ql/src/test/queries/negative/nonkey_groupby.q create mode 100644 src/test/hive/ql/src/test/queries/negative/quoted_string.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_column1.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_column2.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_column3.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_column4.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_column5.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_column6.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_function1.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_function2.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_function3.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_function4.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_table1.q create mode 100644 src/test/hive/ql/src/test/queries/negative/unknown_table2.q create mode 100755 src/test/hive/ql/src/test/queries/negative/wrong_distinct1.q create mode 100755 src/test/hive/ql/src/test/queries/negative/wrong_distinct2.q create mode 100644 src/test/hive/ql/src/test/queries/positive/case_sensitivity.q create mode 100644 src/test/hive/ql/src/test/queries/positive/cast1.q create mode 100755 src/test/hive/ql/src/test/queries/positive/groupby1.q create mode 100755 src/test/hive/ql/src/test/queries/positive/groupby2.q create mode 100755 src/test/hive/ql/src/test/queries/positive/groupby3.q create mode 100755 src/test/hive/ql/src/test/queries/positive/groupby4.q create mode 100755 src/test/hive/ql/src/test/queries/positive/groupby5.q create mode 100755 src/test/hive/ql/src/test/queries/positive/groupby6.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input1.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input2.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input20.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input3.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input4.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input5.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input6.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input7.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input8.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input9.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input_part1.q create mode 100755 src/test/hive/ql/src/test/queries/positive/input_testsequencefile.q create mode 100755 src/test/hive/ql/src/test/queries/positive/input_testxpath.q create mode 100644 src/test/hive/ql/src/test/queries/positive/input_testxpath2.q create mode 100644 src/test/hive/ql/src/test/queries/positive/join1.q create mode 100644 src/test/hive/ql/src/test/queries/positive/join2.q create mode 100644 src/test/hive/ql/src/test/queries/positive/join3.q create mode 100644 src/test/hive/ql/src/test/queries/positive/join4.q create mode 100644 src/test/hive/ql/src/test/queries/positive/join5.q create mode 100644 src/test/hive/ql/src/test/queries/positive/join6.q create mode 100644 src/test/hive/ql/src/test/queries/positive/join7.q create mode 100644 src/test/hive/ql/src/test/queries/positive/join8.q create mode 100644 src/test/hive/ql/src/test/queries/positive/sample1.q create mode 100644 src/test/hive/ql/src/test/queries/positive/sample2.q create mode 100644 src/test/hive/ql/src/test/queries/positive/sample3.q create mode 100644 src/test/hive/ql/src/test/queries/positive/sample4.q create mode 100644 src/test/hive/ql/src/test/queries/positive/sample5.q create mode 100644 src/test/hive/ql/src/test/queries/positive/sample6.q create mode 100644 src/test/hive/ql/src/test/queries/positive/sample7.q create mode 100644 src/test/hive/ql/src/test/queries/positive/subq.q create mode 100644 src/test/hive/ql/src/test/queries/positive/udf1.q create mode 100644 src/test/hive/ql/src/test/queries/positive/udf4.q create mode 100644 src/test/hive/ql/src/test/queries/positive/udf6.q create mode 100644 src/test/hive/ql/src/test/queries/positive/udf_case.q create mode 100644 src/test/hive/ql/src/test/queries/positive/udf_when.q create mode 100644 src/test/hive/ql/src/test/queries/positive/union.q diff --git a/Makefile b/Makefile index b0cf880fef001..2d7b073202613 100644 --- a/Makefile +++ b/Makefile @@ -3,6 +3,8 @@ all: s1 s2 s3 s4 s5 s6 s7 s8 compile: sbt test:compile +# There is likely some bug here... still a good way to get a feeling if things are working in +# parallel. s1: compile sbt ${ARGS} -Dshark.hive.shard=0:8 "test-only catalyst.execution.HiveCompatibility" s2: compile diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 7744c8f0fbc29..c8abcaf22b9db 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -77,11 +77,8 @@ object TestShark extends SharkInstance { * Returns the value of specified environmental variable as a [[java.io.File]] after checking * to ensure it exists */ - private def envVarToFile(envVar: String): File = { - assert(System.getenv(envVar) != null, s"$envVar not set") - val ret = new File(System.getenv(envVar)) - assert(ret.exists(), s"Specified $envVar '${ret.getCanonicalPath}' does not exist.") - ret + private def envVarToFile(envVar: String): Option[File] = { + Option(System.getenv(envVar)).map(new File(_)) } /** @@ -90,10 +87,23 @@ object TestShark extends SharkInstance { */ private def rewritePaths(cmd: String): String = if (cmd.toUpperCase contains "LOAD DATA") - cmd.replaceAll("\\.\\.", hiveDevHome.getCanonicalPath) + cmd.replaceAll("\\.\\.", TestShark.inRepoTests.getCanonicalPath) else cmd + val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") + hiveFilesTemp.delete() + hiveFilesTemp.mkdir() + + val inRepoTests = new File("src/test/hive/") + def getHiveFile(path: String): File = { + val stripped = path.replaceAll("""\.\.\/""", "") + hiveDevHome + .map(new File(_, stripped)) + .filter(_.exists) + .getOrElse(new File(inRepoTests, stripped)) + } + val describedTable = "DESCRIBE (\\w+)".r /** @@ -156,10 +166,10 @@ object TestShark extends SharkInstance { val hiveQTestUtilTables = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' INTO TABLE src".cmd), + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), TestTable("src1", "CREATE TABLE src1 (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv3.txt' INTO TABLE src1".cmd), + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), TestTable("dest1", "CREATE TABLE IF NOT EXISTS dest1 (key INT, value STRING)".cmd), TestTable("dest2", @@ -170,7 +180,7 @@ object TestShark extends SharkInstance { runSqlHive("CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { runSqlHive( - s"""LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' + s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') """.stripMargin) } @@ -179,7 +189,7 @@ object TestShark extends SharkInstance { runSqlHive("CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { runSqlHive( - s"""LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/kv1.txt' + s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') """.stripMargin) } @@ -206,7 +216,7 @@ object TestShark extends SharkInstance { catalog.client.createTable(srcThrift) - runSqlHive(s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/complex.seq' INTO TABLE src_thrift") + runSqlHive(s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' INTO TABLE src_thrift") }), TestTable("serdeins", s"""CREATE TABLE serdeins (key INT, value STRING) @@ -219,7 +229,7 @@ object TestShark extends SharkInstance { |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/sales.txt' INTO TABLE sales".cmd), + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales".cmd), TestTable("episodes", s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) |ROW FORMAT SERDE '${classOf[AvroSerDe].getCanonicalName}' @@ -251,7 +261,7 @@ object TestShark extends SharkInstance { | }' |) """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${hiveDevHome.getCanonicalPath}/data/files/episodes.avro' INTO TABLE episodes".cmd) + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/episodes.avro")}' INTO TABLE episodes".cmd) ) hiveQTestUtilTables.foreach(registerTestTable) diff --git a/src/test/hive/data/conf/hive-log4j.properties b/src/test/hive/data/conf/hive-log4j.properties new file mode 100644 index 0000000000000..885c86f2b94f4 --- /dev/null +++ b/src/test/hive/data/conf/hive-log4j.properties @@ -0,0 +1,78 @@ +# 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. + +# Define some default values that can be overridden by system properties +hive.root.logger=DEBUG,DRFA +hive.log.dir=${build.dir.hive}/ql/tmp/ +hive.log.file=hive.log + +# Define the root logger to the system property "hadoop.root.logger". +log4j.rootLogger=${hive.root.logger}, EventCounter + +# Logging Threshold +log4j.threshhold=WARN + +# +# Daily Rolling File Appender +# + +log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender +log4j.appender.DRFA.File=${hive.log.dir}/${hive.log.file} + +# Rollver at midnight +log4j.appender.DRFA.DatePattern=.yyyy-MM-dd + +# 30-day backup +#log4j.appender.DRFA.MaxBackupIndex=30 +log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout + +# Pattern format: Date LogLevel LoggerName LogMessage +#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n +# Debugging Pattern format +log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n + + +# +# console +# Add "console" to rootlogger above if you want to use this +# + +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n + +#custom logging levels +#log4j.logger.xxx=DEBUG + +# +# Event Counter Appender +# Sends counts of logging messages at different severity levels to Hadoop Metrics. +# +log4j.appender.EventCounter=org.apache.hadoop.hive.shims.HiveEventCounter + + +log4j.category.DataNucleus=ERROR,DRFA +log4j.category.Datastore=ERROR,DRFA +log4j.category.Datastore.Schema=ERROR,DRFA +log4j.category.JPOX.Datastore=ERROR,DRFA +log4j.category.JPOX.Plugin=ERROR,DRFA +log4j.category.JPOX.MetaData=ERROR,DRFA +log4j.category.JPOX.Query=ERROR,DRFA +log4j.category.JPOX.General=ERROR,DRFA +log4j.category.JPOX.Enhancer=ERROR,DRFA +log4j.logger.org.apache.hadoop.conf.Configuration=ERROR,DRFA + diff --git a/src/test/hive/data/conf/hive-site.xml b/src/test/hive/data/conf/hive-site.xml new file mode 100644 index 0000000000000..4e6ff16135833 --- /dev/null +++ b/src/test/hive/data/conf/hive-site.xml @@ -0,0 +1,197 @@ + + + + + + + + + + + + + + + hadoop.tmp.dir + ${build.dir.hive}/test/hadoop-${user.name} + A base for other temporary directories. + + + + + + hive.exec.scratchdir + ${build.dir}/scratchdir + Scratch space for Hive jobs + + + + hive.exec.local.scratchdir + ${build.dir}/localscratchdir/ + Local scratch space for Hive jobs + + + + javax.jdo.option.ConnectionURL + + jdbc:derby:;databaseName=../build/test/junit_metastore_db;create=true + + + + javax.jdo.option.ConnectionDriverName + org.apache.derby.jdbc.EmbeddedDriver + + + + javax.jdo.option.ConnectionUserName + APP + + + + javax.jdo.option.ConnectionPassword + mine + + + + + hive.metastore.warehouse.dir + ${test.warehouse.dir} + + + + + hive.metastore.metadb.dir + file://${build.dir}/test/data/metadb/ + + Required by metastore server or if the uris argument below is not supplied + + + + + test.log.dir + ${build.dir}/test/logs + + + + + test.src.dir + file://${build.dir}/src/test + + + + + test.data.files + ${user.dir}/../data/files + + + + + test.query.file1 + file://${user.dir}/../ql/src/test/org/apache/hadoop/hive/ql/input2.q + + + + + + hive.jar.path + ${build.dir.hive}/ql/hive-exec-${version}.jar + + + + + hive.metastore.rawstore.impl + org.apache.hadoop.hive.metastore.ObjectStore + Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database + + + + hive.querylog.location + ${build.dir}/tmp + Location of the structured hive logs + + + + hive.exec.pre.hooks + org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables + Pre Execute Hook for Tests + + + + hive.exec.post.hooks + org.apache.hadoop.hive.ql.hooks.PostExecutePrinter + Post Execute Hook for Tests + + + + hive.task.progress + false + Track progress of a task + + + + hive.support.concurrency + true + 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. + + + + fs.pfile.impl + org.apache.hadoop.fs.ProxyLocalFileSystem + A proxy for local file system used for cross file system testing + + + + hive.exec.mode.local.auto + false + + Let hive determine whether to run in local mode automatically + Disabling this for tests so that minimr is not affected + + + + + hive.auto.convert.join + false + Whether Hive enable the optimization about converting common join into mapjoin based on the input file size + + + + hive.ignore.mapjoin.hint + false + Whether Hive ignores the mapjoin hint + + + + hive.input.format + org.apache.hadoop.hive.ql.io.CombineHiveInputFormat + The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. + + + + hive.default.rcfile.serde + org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + The default SerDe hive will use for the rcfile format + + + diff --git a/src/test/hive/data/files/SortCol1Col2.txt b/src/test/hive/data/files/SortCol1Col2.txt new file mode 100644 index 0000000000000..21f11729cc002 --- /dev/null +++ b/src/test/hive/data/files/SortCol1Col2.txt @@ -0,0 +1,4 @@ +110 +111 +210 +211 diff --git a/src/test/hive/data/files/SortCol2Col1.txt b/src/test/hive/data/files/SortCol2Col1.txt new file mode 100644 index 0000000000000..83ad1ea0f89f3 --- /dev/null +++ b/src/test/hive/data/files/SortCol2Col1.txt @@ -0,0 +1,5 @@ +110 +210 +111 +211 + diff --git a/src/test/hive/data/files/SortDescCol1Col2.txt b/src/test/hive/data/files/SortDescCol1Col2.txt new file mode 100644 index 0000000000000..d18cfac40495e --- /dev/null +++ b/src/test/hive/data/files/SortDescCol1Col2.txt @@ -0,0 +1,4 @@ +211 +210 +111 +110 diff --git a/src/test/hive/data/files/SortDescCol2Col1.txt b/src/test/hive/data/files/SortDescCol2Col1.txt new file mode 100644 index 0000000000000..ea929de39006c --- /dev/null +++ b/src/test/hive/data/files/SortDescCol2Col1.txt @@ -0,0 +1,4 @@ +211 +111 +210 +110 diff --git a/src/test/hive/data/files/T1.txt b/src/test/hive/data/files/T1.txt new file mode 100644 index 0000000000000..79158bee62d21 --- /dev/null +++ b/src/test/hive/data/files/T1.txt @@ -0,0 +1,6 @@ +111 +212 +313 +717 +818 +828 diff --git a/src/test/hive/data/files/T2.txt b/src/test/hive/data/files/T2.txt new file mode 100644 index 0000000000000..ebd65dbdb9031 --- /dev/null +++ b/src/test/hive/data/files/T2.txt @@ -0,0 +1,6 @@ +222 +313 +414 +515 +818 +818 diff --git a/src/test/hive/data/files/T3.txt b/src/test/hive/data/files/T3.txt new file mode 100644 index 0000000000000..957d4c8cfc0bb --- /dev/null +++ b/src/test/hive/data/files/T3.txt @@ -0,0 +1,4 @@ +212 +414 +616 +717 diff --git a/src/test/hive/data/files/TestSerDe.jar b/src/test/hive/data/files/TestSerDe.jar new file mode 100644 index 0000000000000000000000000000000000000000..f29def6f8c97ebc4a9cc56b558509330764cfcfc GIT binary patch literal 3838 zcmb7{XH*l)x`vS=9obX~1QY}eC6qt_DbfO)5}GllDKVi)5ru&C zCM^`DD+Gi9QbLg~9i$7wk+bgI+jG{sKkoa@teIK!y))nZn)O(i(9!eIF#QVGP&GB0 z-^5HqPXmV`bYzXp4CIbGX=vzZEKC@gPBUo#C(YvjH1qt*_}dKEF*7oNT|%7P2Xhko@KsQ}o;~c?$3QEy7 zlF!x6YBXn2*Eo^FEGv!(Ne?tSD*3l>(d`qRZhq@UmpNt+l$n4D6ciF?OwCz3*gf@4`cI%A5|9$h{eEfY#z8y5RAB zy!+Lgt((-bnwyp3?z0fj5M<@OfDHibZtlxkla>XI%PsbU2lbJ10Tv01E7%!NmEB
    &}U17Ck+VNhGrs{Tn z@9gI-o;y0_sV$#hU*?P&Ulwd|B5pp82+MmI^xZf(jhz29_UIzYXg1ldTC1|HnHZ{? zk@TweL#isk#Ptoha??D$^5AXEvtRD4+Ah9X5b$_Ai%fEeLY?~nUI)!4T&2`(RCix> zZaT3z>yp;|>GC@LRP++7?kVe5k@~r=X+7=Rb^V8HrqZ>I`Ru;c_s8D=hsbOpjBB_< zonyNT=QcC&gLKLuV$ylv?%LvILXyfdJr&>0jZQ#l&}3p&1*%o*IK*c`%{B0=L)jM>ELC$*jT@P>u2)s!;na8=_SnBnYLy}Jwcn0Yr6Dsu)eR+u>U)SiBkz3XI$39E)O)z zd7D0hZhTq@8PVe$`$15Rbzx`_Rfy+5$(V@#6SNGQOog#yNjYNA&IrWHkR+b$+OR+l zTLBp^AE3=+rkt=*pJkr8EALXL9-C26;7luAZaM>@WTJXE)v??c#+ zH&DQ`e4jBVTU}`hG>W)J_I{f2aS3@_%c;Q4vokOtIdKb@Xz`%Ji@}blMqMnfKDN~& z)XO(Mx1wFj$?2bG-612?$HYB?q*YGIW3o$+(O#7rEjY|*c4ZiJi0cyL!X9z>!KTe; zJIo>ubm7|~&?1tBkU6o7uofKI;t>*@Tsv+0UMGtJgh|6d5Hq)!cF{n^Y4u@0UcSqU z%duTXA%`XSI0}ypPqs zVal`L8jG(1i;m2>vmWKT?v2r&ZK0TqQWskRd_a?IRULX4udE%%rjRCLF62GVI762+ zMdL)9EWKKb;-(g4T$wzYC@zj~+e#vl(5!(X)uDAfyH$_Mv69FCfheoZPmtVlm4+Z^ zsi4J6V8CxisG0IZ(;Y!Q`!G}$zN+w%cRN_>RGbXs$u4}Dx3hG|wzioEx-kc@n#AU7 z{%bHa!N=%=9tV##v{Ln43(T$zeBAfgp`}jfq@(UB7JTNTB`|jwaQbjJ$zHt>(l;-|4Cef z75>}k{N{x4@jk>y;Oq9d3uBU2gv2a<@E+lUZsyizt9`=B4Z@nSODiUxzMiMUlnDx) zQ#fFe#7U4kWf-{Gl<^+3S#NZS?r76sbtWp<%R$@$F{f6L=XD@Pj|KlJw(X9QqOE)ASrmyi z39NiGwGa(C5`mA;kJrr=&{5U<^8a)gr&f;co?$|IbgkW>@$?eSXL!4Hb}hEL5c+|| ziupn{Pd=Mx1Nz~%)wv4#0GY-!8XB}wiW%yUeCHpctAvntF|$0HI>1en7Ebtk{mZY> zv;OQf){m{3iu9luHeOTC%w1jPmL8#ch*9xP@a1TX&L&!nCl)3uma1O?n6X@`m$>X2 zJL;~`owt*tz7dqSOxyjGaT=~7%HKIupF+mGT3aG$iYT(0~_X=HKoSt$T5>N{@viA*Nl|hP8Z6vryVF0{j5*EL+W`w+#!lJk~=$y8PK7muAT z?o;|CLW@^wbUF)tGgCxlLot^fE9ypCb&Z0oD~hTyYaIr7384?S3!QG;>K6F|YIz1n zlN4C@Dtw`yd5zM0RgA7F-OTiGVjF)3c*22?+SCfRjlMkN02SyCt~g#_1EMz%lbHI9 z)f-M3=x$Rnrk*gqRK2;Vzvl4AzHbWP;5N)fLsZ_MF)%%+o*r8ua2$5mx)e7zhgg41 zi)^SkvPD|d`eyGCCC-SSs(a~H@9Yp*!*+Xmcsr-iRo=DCuYw5I@lGeL2fLCjGRN@A zpSO5W{0%&Pf!IFuST2^`U?uJxszz#f+qq!w_IhTwVqQy~41b@DYmJO<$W@K}`mP+L zJ_kT|eBD!=CCr6?$SUu+pX$A@2T-ze5OUWB01Ep?AG}ez6bA%Jx(EmJjofJt1 z5BD=d?*9nxXAIl_G0@K_vHzo}pK*Bq$3Z_M<^B(}VhL|&0};N$h~|=Z)Y0w8M^Pz0 zVfwO&9JVY4Us5ktvJ05*TJi?`7VSMI`UQ9~%;8c<$~Xy31#tz;(S7s%Sn3O^Y>r}K zBTIb8AmD?gx=AF`N=4`H_PT^xe$CB5Mx;A>_Ev2zA`TKlNG+bZ?zqB^=2lMOP8{Hk z9;s+tu7@R$$nHHRq)v@@b-N;c@mICN=@6$Q0Z=xjep#)S8!|F&_J)dBtFfR9--a!_ z0ez8DGNT;H5)SmaBmbEfJU1j=OtwK^hj#^f0(S8^qf}~37@hykZm<(qx$9op4rerL z5C;nl9zIY!Y#;2azZiJxnK#aiy}@5mA+ddi^tTHb0t{-`_QZtCfFV*^!z#pC&u{ZM z^9jTb!yzupMXX9w=N~cB{tU%C&lZ-b?`oNNlzre%othX9>G)46%pG@33#;-zD znDwu;yL**cH+%@8w1*93wH4oBEku@HW!PM<$TlNJjsGR7Wk%NHkNg?_Ez4x-&uzw) z#Im|cV)R}%mZRbMMRFr^@Te=o5zXaKXW%rQK2!E~ZOq5C0fsp_k3xcQ^1~vg_TtZ> z)i4k>z)qdw$9s2zh*|Zs3PA$O#F!71MwmVm*#l>47FI4ia^1!@Zb2jqy?#q!cXM?y zxQzz~TNZLEtEe{PLAH_nKv}5Tx~`FdmRU^4g1B;Q%Y^5Af9r*{FyJXcmG`uf4ro2# z=?kyx?dqH<%L%8^zf9h;ycsH5_vgYFgf7PDM4m1cd7T(P7v)&ew)i=!qpiB2u5GAT zYhF+ZGgxcw?XVfrDPyzhm4)RrYDz<3{nFF}sTGG6OGD$?^*E2=i-W3V=T!fYS7^4@ z9~t_9?K{juLN3EsHSe62bd_Ph1Z;T&#B6^37O2cVPYGc}2>cloqv4_>@ue?nUNSJ$ zKdQ7gXZaen^2ByFie~Sw*W%XGiP7bZnhUAbPZ{VTVnU(9E^fITjjo=VUAzt`7E3&h zO@D7|IN0i530pwuo@rt8#U?~)g$bbB@s81NS`;)N4(QGmV0X$n?+}bp0^4Cl2w$I& zD5NDz_idx~KX@VG?x~tFerIh*ZOJd5xnp7c;7G&rhRY3;phQG`gilFGINN?T6ac2J>ZT}j z_m5U>?r^gg*JbWDZNv|Xfn=-*0n77=2*s(z=R4YUS|f==f$}NFH;ZQ4nfZwjXQg~I zQT@@p)nb>LPQ^^=^PUr=W>OZ8qB$aqaEP=S-+l~J!rtiRq%w`ko76RrZL6;+SC}1m z`je${xY-2|8n{OsCeg)fOSg<|Q)WHy9R_ueK%TdCUtU_i zCcwz4begCPT(Qk4#oE9R6T{8B;2$d3*gp=` za6{3XcQz`J?ljsW&lpQX_7l?4_D84KcsIU9@zV{Y+O%K5Lxl|%iciIC;Uq~@O38(G z(H7D3!tt7!9V0NN!>XvfQzU?K(mT_Ab2DW#Mw8I&P9{e4w~~pY>baf zaybjuTQW)G&8R_-%ayhqO4JU(?Kk$+|8*)PVQhgq2;Ax`U=vJxB^1$q69GmChbIw# zVdL!Is>k?WelFX37SEVPSc;Z6ikZ#n)TDjZIzHE8Me}gaYr=;O@7HO}YFhQRci=n^ z^vUX|q}R0ryQObQ)+eGPI-rKPA8)xKHTQ*{y*KC*kzMA;NbD*mMWEQxyIf6KX5Qra zITQxB=2&seTwICJn7thx^W!?6Mkuyq>_Kf2>^CT<(3hPn>TSXR!HnNp#EBl>+|d{Lc3Fvp%h+Mo;7ht9*9 z%m*~%ROFCaJSB&&LuKH^GLwh%$zUb&+O_aQDxraB81j!6Qa}0TJZ6!+?9v?xH1_=1 zk4yTm1aDHIabR*S6civKEf7n8QqC)t=jB*n`l_65-FGz6q14bh-m1@ihhY>$P(5v2 zkhAgyl7~q6Oi8?EgxETZaPul9PqJk432^&>GgANj>)9%InXysoYuBexBx9sUk4#(9 zRtJRXmzQxLo;F|I6h%~fm)-9@xP(0`cPS|d9YbL0as?#xs<^Qx?z zuFy~5%SKZ9BccH5Co8wdU$1?n;H}*ccqAA?ML_(Gzen z69P#d!)`JBn5B?!1)Wq!TKgjg-fSzlqcvc}xjtEhF9^?L+v9AhFtZ;h;Mrhl%ECOc|-mk~eTeu>?FskY|9g*L!-p#=Xt_9Y;DZfqb+= z2vb|P<9k7_q`^-b${dC?mUZ!Z*0jU~S(2dUBF+sbo--6Q2p{wgkCSeh=gXW4sFq zxX!KBBYJ{z^VUunE(9Mcl=R&5eWVYLzEjB{@xm>aJq^D5VoO!&Lq+U)&98JpR(?)c zWvg5K=1;hR_7Ye)LZ9XXE>pD0fwsUy%3%}-Pc#WY_?hwtqhTeS1bmpdd%7Li$wPre zj1E-E6nCa6=9EpvIv)3sQ1b&dmjNuKt7*%l@oEJLrF0Fna`pyu0=J4h>NuKwxRiOp zw%Krt^GOb(x5_O0`3#Lc7vd(06E@NNx^V^m3|N&0pD#$XMpVhAG=Qycy-gfF zmI!)g0@h1yJF}R8&mx+VP4otwkR3Q0`E}8?rM7?ijM$U^u9YGTgzt%(LEe&Wh1dNg z9JHd>Rl#Qf5UZP5>&6w@v3Dwyccv)B9(R<@xhbUN1&L0XoVIu}?9uu54O{fidv?NC z8X>BrXNmZzQh3m+z?=OHsEHlD~NL;F^u(2^D}0a{563W(%Hj-TfAk@ z_Pg1~M%7o<_g--Ayy$EQ>}Z}{3M8vylude_4A#K#52oT5{_i6pFdaW*^Up(`^&;%R z1wGhYUFnS>w;u`}-LY0ax1|^Y(Nn9mc8e(?(We#zz(gSdTocznebxIorlJ4*R7U?4 z*8F8w288z@D{%sc(&3^Si_tBM|A#$pB=&Y7xW zY@MBa0q4Uz(rV}(nHp;=!Hju`VECi;cuXwc;H@FOt zAnatAN)n!y)t6}_)+n7c&}0a&ETdZVJJf(+t1#&Fr&DR{?&v6GX8n*k5%pLkVZON` zgG^kk_%T(~qJAAmNJ#=A@x`N_d0(!R>Ls^86OfX?xw%Q8^k@MSfo>kkDI62`r7xP^ z#Zfi|ab-QA25nnKL5Ye!HwlF5NeWp%6-z-!JR3Z}!m%i1Ng*+ZUmV9}xUxbs9T!;ISAs=|>%lA4Z4eEbhSN{gLv? zMFo-{CtrPS{Wv$7gG(N&&BMKU31L!15|q(j=ng2rzjuAu zP#V!AY3NVPM+o%5h%g*JybS|kjy*xn%!nBEWYFZgHNn{EFS0xv=4e)jAC&CV!@!ly z->#GL%@F0+O8!wYP16WM)u_Ns{TLO4Glb_uGyZODRm&cA-a~Cmb1t?Wg`smy1Vn4v zR`~)^nJ3t7@DF}l>L8{(sO1JYH&}e3f%MFeJmcr=i-9z+QUGTrcTo7`zU{f2yO8;k z)$>4);TjOe(GTH5OCo;TSU=QJ9~i4kmtA)}E#iretZB~7w97pzq_5Tq<&W)v`|H`-cQJ49YZj6R11NIhfBK#q>5m11?& z^aq|d#`8RUIVPX@USoCbGw@<&J^gN9Q8#r@~WfQcq$c-6}pXfo>$K$i30$ zjbMKELqyrEEWXaTs}@aK7SC(8=e|9Ym$UQ8ixmJsQjHbSd|A z7Ja%3JzoUgvRuWIh{qbI27~3lm?Z}^7mkr2FobnYm5wU3!Nb?GZy1}1V5V%H=r!z} ze_ROPzF~flQc)9|IDWE*KP&C=z96iQ{BhCv2Ot_*ll56tRYSZ`0E#ip{!+Ql)^CAt zVsi@n2~(YlW$}yVuS+v_EA!TKH%4-wB0N8O;#9JERSR7z6zu{E)@@ko@0yaK^rFh8 z+e>{qq^4Szb<+#cD$cZ!m<{-K6~if25fk#heo0Xzxe$IXxZft1l*oMjm!7^yzk4J> zMBZ$?hM&S{>e&#@Q$U4d=`HoT5{2WKfEq3;8I$yJ54RL zox2AuF7a?HVF_?6x&)XtqbE?(qXkyzF|Qku1j)J%Os*g3X23Sl^0AaKPAqEpi-TMS zeMGL(^nCl|m#LI6%%3n0qyax^D|)DncZj5Csco!OO4Gkt3cW^p(5109ucs2*r!lMO z#G`sBr#pkcea5x_*b2?w$v(bq4Q>4=!={?raZi^rhi3sm#Q^--e)R1Ko-vGRZDgG1{G74si}>r$6X13o`&MA%8meWg6~#&SpSW9+~WW2 z_zL@zatAP%uU)T5na*G!t3H3@H-GXQr<;w&l!HpEWQku-To#VhD4AYOobKHkT^Y5% zw|GvTxezU99)Ti9|GSH!OKvUE7acbejWWDP>LcrHx&)-{A0i1^3e3sxq)QmU@ziD_5Fv_D7%zVwoKfxtU*?JDrh;z9_Q6w@mcI)TaIt zzM+|!)d~0x*L3$i46(p{_CbN_0o|ox$iJ2KFwN9m_4m~tNeG!C?m!x2_fXn*-l&yK z>Y_FJc4k$GShZx6rXtb!B9gx+FC%k}4T(KGgT_YLVN+LSIj=gIL}p#$Aq~d0ChD?b ztP|2>nhSlEJc&=)?-(ghu5JPO;Y^#dm2q;unufpcPSk6ZWuB+gG)KAL3`xSHrlCG?~`FJ94=@FvJ94D9P!&OdL zuxJ812+HWR3gYWSO^@j=Z*T*v)_46EEt`Y6M1OaJrk>&iS7@f6Hckn;Ah{h3x0D>k z6kGJ$VA}K6o(5pr9dMk9kEVZ+c9U|I7%?eHhn&heyUK7cj`b=JfOIdb!uE;m0|Ekf zNWBeYbBHgDoe^c<4dlCfB9eY1_At5+STX=d$z2fz11?Dv10KH*{&js=H(xl)0y)Og zbJP15ERAlF{SG=H$M~H6wMsK&X@TXtdxS|n>orc_Elb6+=}UQQ6bl*_a*X2F5~dI@rBDd;SM{xnfdlTb@q zWjZlTp_|QlZxlhEEIm5@N#UNPR76WHGx1|?W`)INnv?gt7Y9}SG^^rRnRN6b8PfHU zOIplawkbBTywgjHe^*owlAVpGa#J197oqZBrF@aza00i9gh7C$s^cfA_w+55>`$Sc zRD42PY~u<6BC#v7j0sJ%H|@B0p~EYzyk$k!eWk{Y0f%~pnj^ul2+K02$jQHO<9cE4 zmW%t0!t?&J2yG9|a(iUu+4&j^5MF%fAT3}j@M`(GSdCpENa27CauwSM0;aHm7q897 z`=PrB&Ij(Unel6GZFEcPH$GbORokQQYjVh27A|-+iU)jTL&^OG#51+rkw~mk?$l0X z(?;HXqIQ7`8v*k#`DC9EJ$#+WT1@AR!rnCD%@!kDfyq^ zZy;Zdh(6F8jo;xLXZL4QUKS&OLVpftTV9$8ffIj8+XTC?p9zDF_$9wz`Yc^}IcGKa zBc;u9PFShD+86snrOm1r;48`_Ok?)im>jnblZW16!r?NuMg{O}(t477Z(f#@#PympFltgls<2|+iVRnQ3OOs^ z9HA$3?aeVEs4Wv`g*rXve31z?J>PKfcaOuCKf-z>TyK`k`V(3pGN2&@Z3X zYWknX=m!f}$e-GTu-|{xf$|hV#L(mZOeOaK=X)~eJ5%i`xjhO)sPcQ)(*sOr%KFgF zSO4}1=AhA6QSVG*Z&j0(CBBk%>AP^8N1VQ^=&Q{JdX+8%sO@gm4CHM3ZjqB_R zQY`QALC2qBKV|VVeF>-c&xZ{4D3lp>YhGYI{Co`w@q@;$;>B*W^;~ zIE3v($62TAJDvUxWPu&7`qgEg(-q{XIPb=+Zpzg@BE_~?h~h^#vOsWoq7#_6uGEN| zG>V6Bi;JbmRi9TxX%MZqZ@~hUKeW$>>c?SAXyuG!;X1ZqJ05bqs5CHrPIYvL9~eJw zmNafAn|G9S24~C8uEcg>F_oN+-Qf^2?(V1XFUUf^D3Plam=M7I{8j;=e#3qjWruR$ zW(v`+LrC8(o@}whESSX!j2xt?nHB;hWH5Jl90?1Vk~q8j&F8Yb8ByPyHX!ZCAvlIq z>p!K59xMikg{$y0cnZj_N^#R9lD1PIwLLXngw*><|Jf~X3vphis_K~FfhGH8=v|+% z9r~AocYabD$omVq%#xzCI_xQJTxmWDB^S&P-DUk>=UP57KPlHN(L-$WDC+Azy6NF* zl5H^c(Ccql)^ag^v(pH^0l2fz=a59v1aRm%(kGa9gsT_Vv7y+VDBJ*2CJ#(!LVo1c z!5C*qd~mI%@t`h(W9oN+Tb^g8-kB|STnqNjm6?_rF!DTXP{g1EK|=;2eF?G!>S<53 z`UQl#6zmJH7nzi8Rs5the2);?Tu7i+aySou`RJ4c033YbZ;elw-)>T!B&JuC=(; z_Xv{aKsTj*L#U5;%Z`)wki`+kf8R8>M5S`4TeiWSBwc!8xJj`OotZ_hvB|wgm^rVy z@HVCP*rhi9=SBA@0%u?deIn z8c3pbUvlg3f!PjZo^vTbPL5TTAL~6xTqBt-}vs^JTCu?CFK6sQ~ zml+Ig#7^Mx6-G}{VYQ@v8{EsPg5xmM1>dyfaR55tQY4l@#SX#u5k_8iVc(Mr-cbTv z5IDxnb#lePZeQzxT5jmwn{dUB@G`##Gaf`-z?*()DZ*)zd?J8%nGpM8I2d_Tp0tf_ zH}u_l$VdCjE!0PNn2)V4K(m*LH3Hjrf|5x-8#_jYqGrZ}KQRD;$7b+j&Gdd!N$2^W z1?qmM`qNZm-Mkgc>JERcuOz8dAyw8_59C0!LTt5rA;_mEA3{j&j40|;It~NJdYjh_ z$xlw@?c5I#CvYJ7bSG^U!Vn)SW#L1+lM&DP32jMg+WQMjkfQ#*;g&`*EK+Ul3(B+O?2vOrM9{ zHNhoBSy>_m8!hA4hhRCIP3g~O0Pef2fATPjKe^X!Arof&4%h()o`tu?{vdj}AoBj6 zj7=zVxUdP{TLP)g4`+7a6J9ZRREeE;2I@Uz;$HLbxXVgX&lDvdp1)_0vpmtKQ=rTz z(6gcb0wamqMODMcTd5Ka@YFyKmK*3C_&c?|&5%k&DX`XJfo~0)OI|B`I(v zmn$am;64lfut`;?vl?fMniFvLJS{F0P)UcEc6C~De-_pXrA^*Q*hR7@!MM@Ae z0dP|dUxVQD2ZSf=W+(%;oDHewPf}Qg+O}5Vv;|dqn+izX{*@*QxwT|VvnHDhYb*c8 zV)pp>c;2F!$6+jD?>Sts(HWk%VD?Xy_}z7ebba?*6W~eCjeqaOYi={}E4XpUymP>g5}#(@Q9y0hr;g}2PG zg}yp#lo?$eN!h`c|JE-2x9&Kl*hqNg?A_etYHa%At-xdQ^htlc|exFi;aj zRihtn><7tXgDnqTmP;R>gSk?*JY;Ck&C3d0lwJ>MDVs^f!iohTdM*_&z9jK_I|y?k zwcQHvZdsHu%?Lvcz*Z+lUHAyJSPxBa-VCW}_D7cI9833eVLzq{sIahG&g>gi&-PlB z%IT2jaYa`KXSUT2v@(TkH-p2P0sn2j+kg0X4L1K%p=9=wv5gOl?XsL_LXx{36}89l zbDx)H#I*r1Le!)>W^2Y*myxntZIn~GYm5;1a+_nJA6dxQR&Vjqy)Uj;q;XL|pjpmH z|8Shf52Fnblq+dzWp42`ILrPi8nqFz4yOvB3?Qhi2I)apablwy6`g?9a_!1vrkdSF8SXjt$N{-$N$y48A3 zQ8%_%TLW&mwKZ5>wxg8q7|2oT=l$a*Jt3h28}Weipho1E#LQU4I4(2ai=cad9}7j! zNfmNj4%oTbixnABtqXX!kSpW9#srmBeF7dkjw`3>Pl~zcF)Dzr*^!m?g|{hg+>XYq zZJOoUJb^psUxR59P7T}7YTel6oBC?cC|kq$!Z9dbvK>x{BHcyiQy}h~`NUX1=TAwf zm4c6-4yDj+$2cvD_fF14U`2!s17NYm*su^fMJEOq5j=4DWJxRM!e!FABX_*&wVl$d zGPwt=OR6EuOZ=6zzxGs-*}I_{v7_op+fzbkMI7>!qTZ`Lz>(CU{nM9xeyN&uW@}V) zeGn<^k_&a`?s4$Prvdjzm902#pgP4>R$P=HYKpu37q)f9e2DZ%!4&9b!GnvPd*a2G z*L#$(->o{CJ`3!SrP$E|C}spZVTJGu*(YvD@dT>pclUc~wnknAP$F;f>AlvuuX$gn z)kXUi+*4Ossq!3q&JhHuH1M1O9uW8TI?X|29ck z<+TE;udAYqU@}RtaP6OKH2!|?`BOLXGeD)`O#5&C zCr)Q}clJY<+f?E&t#!36a~V?Q`}o z9j{|d3f8sC@qC40pHAPf=3%-vnk08gRVwM>wr%A&GII5Z$q-?p zzqDlZ>YS3*Uo{vhXA15m>=)J_!t|2z!hDTEdXW#9KkuneRnf~mnJDsjhVeTPF9Q;y zIEJ4+iq=K#ecXbhjQ0&=M>mQ5YAr*URdPb3}C zUWG2mgzvP9F2p`Wrg>6Fdw7zzNIztZz7uPO?j~v!9#}~06lf$vU@tjZ>czu7mJ(J9 z+5+AoJ74giUgb(&=#%{v*w8oG)G#ueG`>=8|FE$;sy1B1Kk$nQ{EX-Gbj7K?BcN)2 zq}N?oSGe5#zF4%@A>-Y8PmL=C_Md*o+nV0!em>$vfVkSU}yI++i^3UJT5P-+)T=B`p3)7Z(QiH+{=63;ZVJnYlSV}y-;dLU|GzcTF6*=~rvyF858 z>8+04<=&`=?P74bJZ$7~*U0o)7SnriK-}qWeL$t)uY}HRcknlBtthHCt6c)U&Gvvx zVXH2RkHe*$DzG4m*!BR(q_9;R<;UStPt|I3AeGgwgq~n`0IR@X85NM_EsL&XcQBge zEr~vBdqADFRv!hP^-@U{SQeFUcko@|T@<}{Y3L!#TLhii_CPDkTL!&%Wyp!+PEX-i zP1LT_?n(1!Lx3hF|=1pp`^pwMu{zi^~X2+NbfuuEOQE~T@Wp#DtVWW-J5 z#Z^O&!+$W#2-sgyLHH2Smkq%m^GM}QwJlP(zenb7{sr{J_3LnY>ssV72dq~n8Kwb zfE)FN2$YSAVnO^DMEHKG2>91Bi1(aBsB5hakBZp9jB*S1#XW9=g8ooDmk_Sp(?XP? z-L8?2Ig;O!00ak-YA@LU%qTC207voy)G=yhk%H(xpo=FAY8#3lEZ8i$vP{7Sk_i^| zoX8#}07!O=2|$9lhgrL75BY<(Cb@?S*o3_#X^81_1Sp|+C-*f&AfdcKf{f6-^ZH^S z0_FBB06>ykD*%x6mI#Cl`ySW#55-xsfEZK^OCV;90h))PLdmfBVlurhQB2ZN+6lpk zO(K9e3cMrF=r7QKz*NGG^)yfI*}A6x#nwAVSK2ghqZ8YpSW5GrwwWAFh?u*i%*`o%&MFG^})1M$Dt4Zqt?1Mj`JP8EELHo*ov8sJa z_Jj~rnSVC)Yn)qw8#ebZo*%on#CDp5UDX+(JPjan)wM(O2m3P-J0%i0kgByubcXo1 zLr0S{cYXVHE(q?+954z|6Pu_14R2Km`YpMq9Ndr$$Qj@P{T9_@1>Iu!?YM^N3x?x? zIzRy2PhtlH09uXenb!ja&CTR5=fx7R54Xc=sQXQ;W*ei&;|U1b66&en-w4{0`d;#_ zs`v~+ANOthO&=5sttGmr65LOC$M2tCg#<_*kcZ}(&{GZl7TR+K|6ydp_VgL-0IU`V z0qI8$(Z(1AmwgTaXC(b9Y`nrTt%(nHg0&1MdE*C&;fpMqfX5zqVDC{U%eFi{A~>@f znT7AZ!HY%bZzWLYaZ5srMgKCfc*6>Hx~=TLb(y| z3hH4Lli%Y*V9Cg{38#5;7n47XL-QO$CcNYtU6(#3p`kJ-^Y5WeX?fju5dAfZFm!7@348J$Hs`kMxI@?EZzeBX+7M zv4wF&*|rKF4EBY6#4&|wWT2q^;af?PiV}CY&>qOENC9*9j0viTFxwLpsx8x5?paQv zdCMu5Ex9=FHGyme3`5vGWWxECP|YQd_Tvc<9+|go9?sD*h&;(}9F9rIBJPi!BLlvq zDitqe6|U*W8K4LKwt`I3HH&QWYijq-~|0;!w}%&k5wK+rJ3cAfHtvf`7bx2bj19sNVQOpOWK|ZlvI^l`{Lfp?!yl zI0la2{7F70^Oy&)e6vVCq}m=Z09$}~e@qG-z+`-Ox2<$7 z=^W*oXSmRYd1$c%yt??$t$2yIFz`!Kr)H8U6=Lve>^h=2ygG5~(DOJvVkqQN)sG4i zC|HLP5nr}|@6a2mJpnJd^}eTA(e%d-?PIe(0QEGg)!F=~iS~KMX+IsJwRYNf=v<7UJrYwPxl#bz&4N~u` z)?`cAeUTEJsE^v(Btyy*PS60TKbk!Q1gEywk_HM4TgvrB&#!MHfasx+Ai*>)RJe+3 z&cR>sN3EcknC_Y_xa(%N_@fEUIf;v>Fb#L@GljaqTn$8E9HwkAnzYbnRq8t@tBn7SoKdPFur(Z&UTqG^<|jJh!N@ z8{Vxu+2NhqND;MAN9L^&cjDUrSFE>YxtSR`=2`3m8Zmus-#@daTHqw_ZycutsC^5N z<3m&boo6A;Z1@;^y-ZT$HL*X@Wsg{?8p?uoxEIZv5Fbdv!*C6#h7(r-!+Jtwphjf# z`hlaKVTNpDLY}HWUdZBDk%Mj_IsL@(k-mH!$0gInj)Cpy zGE5S$VZy{<9&@whXgaHif58O;FIi1DW3mw6$UVE*E~VNx+Hu0U zY9BQv^-lJ?H^o{Z)OT@j5lu%9cD>j02afN_PK6d!huDCWLIECG(<_%7bAud*CC=>o z@aU#Xc$)*B4DD6|O)Y{lm@wMw;Z`KSpU&hRj63}V zZ@6=WdZ~WiO`a-=^=q@TPQ~+zjp5y^-37cBVDICsN?N+NDyNlExYKw0vn)kE^93kL zwtBO41MGl{>AOt&aJb*KZ+it ziJMU3YOEL5&-3uCAEeCAf+u(=?W?hT%+QHRKkh1?c#_w?iZs+Mrgq_*TZ&7vSFmUW zUf(LRy7;B|Xs0{uF6dqfMHNOwsg4!@>}}Ef#90C?PHYGu4N7x$Y^_`Kyd18^zjBXO z(EjY@5^j0yb<;$7;cd1nu+Z$|=%m1JeDk>v{DGI$)%9w_S3zGqZy<~aXW#rtSKATy zgmA__5>t&DE-u_r62m%Jf{Sr#v!MTp`xU}$BFAaTFz+C_lS#-9GfU-WLh*i`1p8

    tA)4|HHpOitxjpLL%xkKB>`uzL#~c|2(YHOiCW+OAOFa2fAv&=w4X zeHegi+?^1NJFAzIX+4-}o~|fIto-1_**mLz$2{e7)9YH{>Yp6@g6=xn{P{Pvda@}k z!ebjD+Gac5&L)=aN2w{U7Aqt#fAP8&(%L?tQ+Db#dCAd?BJr}YU321!tvmF5cb}k> zA{kb89<{bhk}S+szWe@Dx6g3kF%^`CL6MNa07q#Y2m9`5i`X@v_c!yn!|ndj>+jT0 z-uD`uJK97kgR;K~Vh^u#6H%vC9?J;g4^LtVlNfhJz8=%F^6seH+U)Z_Hv*Du-@60> zNm~|~Jn)BSe|ysyazAF9rylV5CNsO6N^E~Gfd{gB5@)e&mO?kKIk9aSNJZ%=wl{c#Zes%~`DefZ<(u`vpO|h(kj`sW17Hx6)#duW@N5k~cmn9g)#u2DX0Qy@Q0Ac20#;5+#{b@Aq|M#&Xoi$8w%5*H=_(nygy??< zB_SeZe>sIfzh~RgkQOe#afB8>Wvh5A6K+TpOo3%(E(_eE-kj#DynijByNA*6OL^A7 zC~!tKX`s56(Y&dz3orCy{0waZ2%q`Xm@3`qmE+V(^F=cI>&C-W{V-POxvI|$^R7@j zP7Frf?l0x>saXq-Dq!q2yIaKisbYB_*0(tj0ilbQSB)?o8EZDI!xCFKUer^3YGi%4h zm#*>;xO6UnEs$^VpSO--nUQ%tcY+Kunb$88ZC8b@d>Q>#BLv9g5c6eh$|b9!d>Mh) zgDonYpDX5q04or-qYUaeT?)!Zd)O7!n1pGD`dKi`a=R|rO}9a)RUncPLx}vQO;3a44EamN+pekV*6n#9cTjR8)k8UY)G1sQk=z3JJ0S86` zu=n}?TS!&P@f920!t(yTKGUrG@~W0#bz*+Ce|&%RC--}sUyXplhBj4KXLV;O zF({5Dr55+;XDPKY_8*J0XDKiS1_rIHCPC%nv|6+-2PvfG!^s+zcaGtegh8nHfvk$W zG3twj}G`a0MD0|rUMKZ+<@koJ#)U(_@?3_W5=`4C5@txm zxwF=5y)KUN{Pv})tMV}t<1*Z~-;GW~zM(2MP{Lr*J_?kh3|hyLgf58>q&3c}wKOW& zo0_QkkO#)Dy@6s3DlN=1EQ1I3ew(UHr&RRG{bP6gjJKYp0wAhjtE@syA)CHqzKi&sS4{gHXvsn zASngb@SRcH=uRSTB!eE2#G`>x%Mt>xlT8PKm?l3-AOl@d1B_*sAtg#yY~===DE9}j zV2gSpD*>f!5Sok8lu>iGG`0qu)3FSxE*VcY$EAzk{a`|Kg*8~Yv>PoqC<|cB%1bLk z@BbKjngSyhB{aj=bF>8?GSdi;Y~4DYSw@2k3DDFpwe~lKfN$+ouxd0@ z6$Ny|>SRGb`3w=q@ix%Gx+qfZT&}H$B6cpS=2Ueblc+uiAZKd*=|p%OM}e;FM;Mmx zcL}bVZ6S>q52 zFvorozMQEfL!Oi=Wp0|pd&DMra14p#_j~|M8Xt0w6@jlVF|Sa?9atV_<}T?W$9*K!34Mo!mTNyk){>3(8>^m->pn3r5 z>aO$`3AkUJoI`Zz1Hmb}qJ2rk+q36PLlSWgVz)3nVq)+=mM-5yGf6UBEU%1#PN*z}}!LFpMVC zV8d5GzLr&7r>{pXQ6I6h{+a5Xix)bqlZt(Op;XE|kXmPieqJu#3nwkBTu%q+hN$w+ z%Wq>y(~t!vth&sqyE*?7S}Ahl!CUKds{gSYQflSs2!W%1f5Cb&1IAiq#BwDxfh39} zFEEin#v)?q+hS(rleN^;{G;1r2(A2)D8qElM>~xGe(2Fpe~*g@e(2%86Ex^-L#*xt zyjY5|m6N%=>Y!_Ks@8UOORxk$ARlv!|Dgv5nS^-d!Z#dwcW2stj4^rxnbwmc>LnI0s_M?HW3UR zGGPYRoKPEYkV^~zS96xUeO1kt*v1 zbW$_D8tDf|i#J2CZ6~1Sm$*bBcbF$z3Rds_4}v+TOr+I%yYK8rh%mew34ict<(pJS z;#?IrN#=#ovGFVxX#!Vdpc++?Fve{0gmyn+{?!$kr}yYQCGX>K6}P zdf3d;e?8|OHr@+ua%@DEcMa5;&g~k=7cPz%Uk?XyW*ep~pnEE^$mp-Ked#t{fr9)( zBA3|gz@ZetSN3R@Y?y%0VlR`U$Q<)50ozW>bn~)^O2Ni9o7wM8TX)30;Zj(Bv-NMS z6RFgA90j9t2@ht&)Lf!F+qU9INYv~~jQ!t9a$jX1YP~%+6gA9xs&Ni&pEE*XJt>~) zUmSM_aqOYL0e^%&`DZCUyd!6JtgVCC{QDtWz$5+Vx2-_FDFRxgQ;^`=Y59WH4`VcW z`e>o_4w=jO`{+*{8%w^z9%b*?B-#Li=TL>>M{fjqa zN85MZo6UwCS{6I8I2S#bA01kTC*AuA_>dz|1h7HH0S;i&zyN~486;9>bkm){n8-T; z%v0n<$_be;(hJ7w$B|8^w06=d$X2?)0!^Dva*^BA`G-7IaNVv2@KZsu+(eXpl+*=6 zUS`*XvEy=43CE1v2p+}Os~Ud`jf2LWOfav1Vt$bSQG^!TAfT_#{!wSfyW5!AS<^v? z2TRAh1L(cLVXq1nZNasnKk|zjg0(9yr1!WA#g$iM zEf#HzgIZhln7}_nL}vDCU3%`KzFZqwBs}F%IUM;1XwQh>M|fHc&3U6xVRsp&Ql3y7 z0l&I95s@52Mn_&cE15ARygcBdd-pCnXVkYrrc;J@A;QQInTB??a)7+Fyof8HUW%=q ztY(io$X4sHzjd%J>+5&OF#~Xys=!w*U|e|j5-N$f?BLYX`Quo;2s5&d9~@Mz81ErQ zox4ju*h9C%zh}p^Lg|M)|Ym_K-Ch?nXJ zdPFuIs0kAvV-O#BE(fHleXBPWxrOn{9~G{@D?I7ldbI8v4|C+_rhfEI%f!l7{A~zP z@4cyfpUGQe_m1uF0!JKv;mfI!11zfCC0`;!5fJ4Cfj+=tkNLq^oO_j6GW9s0Yt@8z zg3GN|tCT4{Kd?JeBsa^-Xo#r}te#z(plcw2h8Q9VEP@AkaO@p>(oiIAY;#|T5p zUh_K%)0iYj4tvabCT!7Q{I355Yw+S&6vSMtUrt(+{Cw{zmi}?RWNr+S0Md z>48%BYMnW^6KsC9ojI=)3J|sWI@2QvPV@>5dX5HN|JmpLz||7`{?DHf{&=h1#&ECj zmlj4n?<-ctLPb{g30Pd=`ltMlgrOqh*)f?_m=3gz_?#yGVoOUz0eTr0x|3;T1$l22 zjLt*PgSWj?aPgawK?&ma048btgq#6rau!q> z`csGqlT{gYvWQygy#d2`?97}2c5)ViN;6u(1}RG)smc!LJDjcg(C%oQ7FcdAy*GK%`%9I;vXB8639l zr=1UpZ-wB5O-;ySD<}Djw`GHd?<;PK8TPx*l~}yKMkkCGkT*A)x@XE`kD+)VCZm%QWy$t{-yCI>aR zns85e4C2IH33N z&ULer3pd>L6lFFvQH@{dchf9YQEo-@>pgk34Iu`p?syZoE9jxQ`Nf(jmA}|x!sQCA zVUi=O(6n6agG7}IXiRnm`QPlF$f^%L#K4TE0KycKtH;-Ivzd&8?)0_q1R7lS=U&{H z>$t+l06G(DBZ&T|Q{~YtW6;N5_2`APTQ(C4Be)oTaH|{UxyT;MF2vFt1g%@w`1R+S z5XV5KCy0qDCUdbTOU^{2fgxbjs;?i7-Vl=+!kddbPIu(yaJ5O?&37HD?pT4T?(y6s zu9~yPTHl{tjXJJ1$p}_4iDd4&%MR(3LV8`Rvgmdx}qaRB*(Q$py_9W5XzkD{|$&|=v0 z$mfq4Os4YB3lQv_a2gi-f=+^zuI308#ZztBC&G0Q>@=30^k@)lh7`?T*Jbt-WDy^& zfDjF}JK&%84Ku0vb%;mqi(Jtomh2F|-2iSn0IZ_aq~N!1x1(jBXalEVkP|0dpW4NU z#&!a@y$VPJcwLPT&X!e{O`Si{WPooQ9SDr;FnT4s80_QRZzgcr2lSHNl$7_bnCm6< z&X~t4r%ssP)gL@C|EeQWUmPPd6jC5=0bC?Y`+Jcem{6`vSazc2nndmmwRNgBT^MZy zpV6dfK+Unjv`RHi%RHdYSC>KJ%mNkWwot;TngTAx#Q=Tnwl0PrG0pueik>#cgo5T( z53{S9eD?Pr@jaebNzOfZ(q_C_MjY+N0m9fXxh*ELgMyj0j$9t z_f2jY<&Um<k8iqrL*A~O_&%vNCO;w8!Na7Uj**W) zEs<`SR+8Sj&oMqIe9+Lwn`^6sKinKxeG+J`_*PXXM0C(y!%E#7V+&i5bAW0X8hokh zD!o~XAFw^nS0Jv<0pb^{(`jr!qq@ck-qj=kE;5)Rzr7ging{?lDzhOCVJTlZkd-}O@}=;YI}Bp^G!MU zK&gAepF)YV&Mj#{?f6QF@aN+>&xM}W?^sA|DbiXfm&1A{=qT?Aq-PD~Vpgz%hr_G2 zWdRCXLzX*~N{l7K-d3(9fJgMpFeP=r9s20{L!Xsa_=;jz>aBj?0(M8Z37LC`0GWZV zAOvNIHz=ezF_w@#N(xCk(!&OiK0LjX6&(xRYkoQe7kK&f3$xd3?t9bXOM3cZwQ3bn zi&hR#Bt9k4$Q%mf*4i{k%%^=j)Cjjc8dWTjSh>nJvjW|tgLr#pz=m4-nWUnpsC#eC z0S%R{&k7DX$dOb-UOKV;Bf3tUL^7Q~zu{1H4M?UJ2~oOwb(%3K?Hqgbd~v@x3!Y^- z53|ukh%O6~uc;t=6vPazVqiEo&J-qxQvo+b6{_4WrV5ltgat5O8?k)=)>Cz%pUsql zpGvbg%9O*Gz5`wyKsGOq_Ls`nt_E1YYl?tJmbqHHC(o%?VL=r|{2j%i+!I6s%p&&KxPcepS&K(MLafIu|2r#dQz#p{^zQUBu3^Iy`E{XkhT1%o*lY zUw^6#T?^>zjKPGyErRA1_qjCY<|u+jGgOyOn#Yr#?_LKAz`)Hs1*`YElbuJr*=(>F z{q2Hj1Musk;5hFoyhC4_#}S7$Iv9YmN+`6$z-C&_c_Caf2b*95Ix|&Ndz6>Bx40b* zX29{JmUE8=J9Ea1TU*s&s2QQ#GnV!{AE7~pu%$2ud7R&_q_De6;~*L9+Kp|(2(3fv zLSVYl!WVp%EMr^^JD=pYjh9jNmqJG2o=SG5@XK)%684r%mX9H-!_(tzh7Ih>X=K9A zeJi1N0k?J`M;|jY4#!96x4=mrXK~7Q>%Xo;#Wlxh7`d*IA%t5Vk8q0027er(0fuKI zv>)P!kC)Dn6!#MH5OgKiXT-av4|JijBzN06BsakVHvhm0RL6O-7f-C9TAkfc3o-bv z!XGT?$Bb_G>r&9xwJDYcHwZ?44|y*+Tca85fh)wgZj`bgrEW- zi8eL{V7~A7w-m|>F!`4C`_2?@qIj(1#>Y66l>XGcHxf*;tc5Rw2-%QT!imyi6OyRa zhD5fhPi9nN6_%|%!If$xV?TULzw`!T;+yK6Dd87NdCea?Gv-!mifheup&4v#w|#pZ z?&QOx8KxYu?u*UP*i&A1dygoFtvi-T$<|U-0M>p_{TWN;58hcf(Uvek;pC?ezThC* zX%e^Z?rakO%t9h*!J@|R@=mOy(#yfI_xi)pH}KbTX~K359K5h&f<_!l%L`j7K8Y9$ zIexcQr{?04H=7m*IBe`TLoBNwMVE&t8O~Fr)6CN-xdpi->8q-mzJIWWxc37k#~yl9a!5Vl5c;Z;7hI zPXhDJS&Tg>O*ubxc8u z5UE8#kc*Bnjqc7M*h*71W5-HBjWb@ftXe6FLD3S^y#TI4lT?F5TVi<0802syI;bDVya%gcoyw&FgAXk)Q z`Z)tPlCPVsUCHJNnb%hgOb7ZjzMBP;QrK1w@{hbaNN-Jc!tUBso2Bq_3^yiUWmM}mNncA;Dc89+us0KFKn6@DLPZs=va zUOUEa7}<_soqoB#`~cFu@-2Cj9I0;oVD+5Q3(-eBcRV-cxVHI#@*$zapJ3o)y5^Iv zHG_zP>FDsrMNr3Z+-mHvSJFn`^0?lUr5D{*ya$l#KqULr;7!5{2>{U98aGN>f8x6l zWD(tB8an_pZQP)#N_BRwJV*8z9Ud@qsrD2VB=MuqL!Tb~xo7d5>?tS6AebPODlbTP zd+4`n^r4QJB9ydvbL>**Yp9sUG%|Q2(W+0FTz*(>k>sljc_4_B$~1m^cy*v~gW=We ztFXoNN$xQ!G){H+R~oR3@SNzY)PjAI>>$uqC7QNkG4CAd-QeEBd+GzvS%cBtiAYi`cyi zx8*(j3(+$al^*XXLXmqi5*AVwkxB?E>kT#gh7LPWBP|A`_!GbzF=$mbIGm9(?TyIS zM#s|98`;0K)#&VtDlF6HOcnFN4SP8=_#4kR3v7V3C8jRvVg*=MCq!cao-Bcl;bC z(F<>u)DkKhTRNaZ-4TTOyk=MJ+dc;;E|ISKur5TKzcdDMFi<~OExsZFR(HfVJWqyF#CKxvn*d_{&WDz6FB;ts$U+(6Y2(YB=NAB z>?c!l5Ue~8k=yceNw%9c#O7SKcr$zBih)hniAb4^n?edV4*R+&b<<+tl#o&vZI#4X zL!}gN>_zZ)RnNqOd0{!)1_AI~a5!J_Y5#YPMOfcxKZTv*vTjL#6UIs#{CN$&EKXT? z4?QmT=MC}af9OzHn*X(Ma@wLbY*+ub`zs(u%X0ysj+G3>5kZDGO4u0ZE~#G>9ew zaiQt#<^Ul@)!W=kEV_#D9@7nyfxo!0X)hE|C7ciP$E7JRdLO-0+1Izr?{hx<2h#zOxF$oq6h{eRtO?aqW15t6%}L zWlQMe=1;iGUL4o%16GVWs=A3dNE@zL-$CW&&D_&8@aLX5RVo_p;*yIakbXt8qUAi9 z6ob4rLDL!DS}JVJ$2VyJ|8##>{A&l1vgPXrma*s46;tPrpSRQ zc&)_e=d*CAu3@&_1bf*7#CV?66f*0^iZ^9(Rw-vtmq}EO${AZg+8IYA-2)c(!zK;8 zk6C2oPXO#Y*Qo{z~{|R)`I10jfD;TnX_< z62??4#Mm+E02z{s&6yFpvHSP;``P2JnMT{0I?eTB!8@rRKk{Tc;{WI$9A>#r3BEq$ z{G2)me7wISeUq^ZO+YXsi7sSLrEr3wLzP98=of>v5ixXF z@%^u5MCx>h?t}iOaxr~>{zaQbVu?>0^@WR`D?=8z463%f%p$F>NlN+4OOY;1N1_9- ztwcS6L;39o)wn@g2RE9ZH4--Hb;A>X%<}nJD(`PR-(R!Zy>+d+z+ylI`UTD`Tox<_0%j?p%TYXqu3)aq)gnL$@Qf+!-HgC zyHu7#_ycvu<8YQ-oka@bMSZPibJErrES7t9(~6(4Qt{s8vPkl9+_e}md)H!oD;u>= zkjOEks?!U7X}ofiNEqNz{bKzyM!bzo;p3r7w15rQ@uo3uj#ccWP0`SChK$()gQven zDdiX6Sma@bgaj9cM+#kx&2t4UO^C@QGpyINMoag$P9y*Cl7m)L*RCp@+@qUcj`6c* zUZ$E{&)Hg2u0A5;c<5T^m0WSbah}&rM-`S0ump-|ryN$K9!;-bL<0n#gsBZ!bBma6cnl?7mBD&r$V@y zl6jgr+*O-!J$pTF=C);z5qA=STr}xDRiGJh7CveIa4B`)TS)}rJ*Ld3hE8gU8D z9a>^7Y>kf53Goxm_*V|eTcRb`8;=ubZA_IomY;VlS_FWg;n=z*Ms0fbnN9|W@Cg$% znARi_w-{xBIg;_hy+N({BS+Uo*XY7b^tX42!OuvE$r!k8L zXV0;c^DDe~^a!3YaZu4$>38O)(CpFi%@+d)FI4%-2OL4G`i>+0AZVwUM)oTXmOJv8 zM4UaD>22kdr3ekpT*h^TNnK|F$^qXNE#T zL+Iu!_sFPV>LwEgw)n@>W+2V>1!S33PiV&4SLP6ZU`RJWBkDhK-${`Sxl_;h(~g-~ zd2CH&L!e*I-`8e5qr{HQnla8OF?N7dOK)V3OTuK_ z%?_dCw%^6Yqz9~6Yg_MvKd&=btrN8Jqv8Nw%ztx%@PKQaFR_Ipv=3NJT#YN=eKg(!y%|1?Q=>sPjR>Q zAwQnQcU`8|oT_ZzE?6>LE%>(VVcfAw#X-&8pfuAsU1DRzT>g_{{FyXj@ai93n6;K1 zU;EnIUcc&?{*S%wOBenn>Pu=0AphsIgPg?gg#b`&cK{WE$4A=%)$sQFhAXdxR|qNr zi@GqL`-5J0CFv^Q2F({)LXem!#26`JFBs0plHKg})B72Wxt~}>NJNx74oifMv$dAb zMC;S#vV4L7PLn3ETB4OU=Qs~aZogt-tb?FlBm{}1vcrisL?bOt#A}s0PsyZfC2Oq) z?oI!&qhZBd0lN_m??>SYJ1$JWhO0mu43WZ0mf6tudgn?av(J5s{mI|X;4PX!!2&Q* z`ZLt6YfKdH$1RaZ$bZ&bE}cWw^|dta|H-RI0{3;`Gmb?f1LUO2xuWqP4}ZGUajwYv z1EH^^pnD?}4v@78O-KehY6E0F^;L%Y;N>m#*>J%IP2yf5{P*CTtjbe$`vSH1&)H17QJ|u^%?F@|*(33-ly|w|dRc;4hk5W1pWueMr_2)f zK_z)CorCvDV_hdqqW5u!c&f|-_aSBRxvr_iO1TYh--_ZHYy>p>9J-NHkLg&d@k=k7 z+K%~dX@S{IH&)n+e?0XPaCt)49yZEm|IVdMLT4%V0#Z99PpDC>OZ$cC#TCLb!demC+qaq6|%FJPCZTZ*D!a z_SuWPNT)VCRs601pVrP>4Z*OUs@wpS*iZ+9-o)8^>9uF?n6N>fUscV=T9x=FBVfqik6jKFhYL+#y6R zYOI?Mi=a;a{tX1ay&X|L6U=yo$+N(bCaM3AG+3Y>Kj+cd0T6D{_ciwtAxidc!0VRMXo zo0O?rWWk1<;9UCxtW9YMH`N{)bl;%xI`6_C^Lw)_#h@`sAq{BBIR2bque{P^^b@Wa z8bDok{uXpfG<@T8#2l14)rZuj(Hyp7;)E(3{{W^Mvgv2-3czVk$Xv)&%mya}eyiGy z-n8Va&Js+fNCsXb)6J53l%FCLR&4@5q6-5%Jj24y6lQ?ToYUnZ)=rTWghc&rPOFKc zwa#ilIYu6VEgAkc5t+8=)TD{B|4jiN!{}^*nIGqfNVG(MSUTo|>rO@QJa{!#r1K7< zzwm|+zA~LFb!>d(aaW|3Z5{a#69^mR8Dr8>0MT8TFmFD-v*7WMrJ;96KJ$Hfm1gMw zuV+aD{!|1c+R1CeP*c<5a+pF7Q*I5R0Yw!hW<8ZS`zVGn7BH zF(^SocH;1LYk0pN63;wZ+Ju*$jUN14KO>hq;}%yrY22pHZXxhdImphG$U*AVOJ1CY?Ym!?Z45X`XN@u)`k8^TWRT0?nfOu{)1VvFsc?&* z;Ep4DOR+pR_(P2s&e;H~i9n7rDNA0K6~vG+$~3k&4^3w|e|IV(khz7-1#r#^E-_o1 z8bQ_Jd3Xal3L5G}Imo(6xTbEfd=l>&fTe#bFnw{&uaHSB1k1v<$B_Y>xkxvNTnflL z!kr!RoeQz^BceWzg3(Pbw;t##)zql+(^ zS%k@9O5;tSivu6}^9L~It>kbc-(ldKmbqJY3m4shUT4-VIn%OTXPt#$ znIs2DNv8KVne$8u5aN3k;Qs55c^*(DC)nK3G^&-ZDTLj*lNJ$+!X>gDFhC?*@tf@$ zV;A1uz4{j!OG_jHInZipwmx!13X~-p?ScfY2S?R67JBLz-Z8QSZcciCY1yivZiZXx zQ*bO2Km#n-DLLQ6sSx(3R4r}85e#mrF_9Pq6n0S@A2#w zCT<}v806#7Yi(b;q@=DM3V_<-dBG3AA99}?fN@ZK+R>>#K7Q@WS=|QseSrF-c@p%N zLqDMKN)2Oi`A+DJY4n9oNvF7>bm1FVCo$(2fgVE&^I@L=HR{~J6asV0EFUMQISzrime)i2&*U>p)&Qc=j_zcI3etFP-q zej;3bY3SmA!)7MIkpB^-RmLOX)DK4H{P*o>)PLpm@zFQn|Gr&=2TGn~Q~;S&lQ8)o z$=8Jc5v0W@BEh5$B_N^I4<|zX3r>;d?DZ9C$xlR2qfJC2tS3wH`0s>qp#K>dJuM>< z39J4jJ^jBE{!@bgPyhPnO#FXENpZ!r{|;@J^aY4Pn~nN^4-J=Aos0yXwvmKHRG*Mr z^xsC4{r`;fpGKs#qogmyjEI8q|2F;uukp16jI^#~Bsf52^1AB0GJ5FejHnncwR+^( zv%D^~$RHRxlC($^L^7$Nr`VdEZF-{HTtnp5ZjdL~NyrzKHBVJeQG+$#KJ&xg)kk;d zLtaXXjNix02Urj3R)i;`se!oeKvOsu>4Uhr_&HP!Mo;-SchD7toyK5A@z#Ns2^nZp z?Z9Xh#@_(_p_XX5lRoe?8AdhzqSgtg3Vt2s0yuZ6!}UKMRARhMVRbt)t30{%(pNgySR*e zk|GLt6k*2VG8_!C)Orff(R0Ec^~3bX!OzLuOWKvgOb(@6$EX%EzFwg7kkbwS) z)@TB_+Rg7k)D`{xjwoR{j#y3A?vNo>2#HDBl+yuDQ8rf@XW2V<*J9YjF-^^spBI%h%Ur_pY+?eZLb8Jm=TVw$l}oV$`3h0B&Hu8k^78SG2eYj8 z1y?0BBj!6oO#EUsJL~MZ39}$GhX&7&FZ5YJ!H)!k3u$}&#&3s4L@YnF8)OwXqEL|O z*D5Z(J2d)B3UOJjDL>9yz%VM6E!(BAi<$J1-;RUw06oIhsI3$Y2~ggquE!@alX)bl zQF#SdbSxm{e@r|8$QRhZly;y+u?urB_-|0VW(2EeOm`nzpIw8^OcGEb8E0HW@oH-T zc@vY4*r-kc4rx!}cSa;2q(NBFt&+s9?UrjJ8)cc^a%jjhVKP$4z1Wtg52+Cjrl>gb zyU6nnQAnMb#ib%d?;b;a1P)R6)Df3N*!d+Qq)OJCp~L@f9Ic%P<@2E5zM1_Gu#0~$ zieKOL>BTYsHNOAHXp$z&P?D@l(E%FvdTW~4UB87*<-f}fBCMbJ+Cv+^)U|!xC=u5Y zs2Y=&8rzzM5z}!|Q^7eO!J4+~nNA)|gp7sCwDQn;ND7z7e#9U>dgby>kN?bdmHzO$ zU*7)aKpGE>2|h-ZhmjYR2xKUr>`mB_Es=YY38d;nRzX$4aHj4#AmSkE;5eV)JB+pj^5I{`?z1|$v`p@P9VHX?f${xh z5E6E@o{>!G{9+=#gk9=D&_|5({eVC)-(Nd=6?9LCd9+W2c{EP}{c-qX+G;yyLH)=0 zO%Ty#r3@fx3kwf&zim!Z%K<`lc1?`Nb#kkrYKu_;Ry-QNm^@A;jaR*UOVB*BxR=Y% zBsZyvST}76|5_}Yc~5fSan36or3sW1w&Zs0LTC?=KRbX66qb<#)P?D;P4N9{bh=e>@G4?gzqfvBP{y*K?a5qPebDtg zdD&QCOyboVI&c6Q5hB{P=yw>eb9Hf`=tq+t=Xp~V-N8d%rhinb5p%5eb&I9m6pF>D z`2V8ot)tp})~?}FihF?I?hw2#Kgy^=MP`Qy$tnS0N*_guT~J6miZH?Z+(r(Ug0x0|zUN&gjxVctS9H9E=0@+e7d zyxlz-tU|WJAXLnQcaic*_%MNhtRl~mG&~vC>HJK@alOzroYL65ExjU8PwW>zOs0tg zPp;y|Ux?i;-^P$@z#{Kl(Y(+iHr~ZIBWtYZ8zg5+!NkPxG%eQ4Xf0}e#QyMf-@z(~ zPq0#LVw>j?rX`e(?Cz_yry4ORx)CAAiV7d3Y846)Hl&!UNR2M3Lm{E9#9+~y$ zcBoP~mio;O=~rsFBsF(%&J(W?Qt_Z6PW*OUN&sM86wO{iQ6P@LO0)937Kw@+wN1(* zJ2OieCtcfvBN72s$P$G~Sn#3khK#o&l8P>;JpcME;}-7+=acR*x-j>ugfft+aZbvdl#ldB{*E%9$0> zJoOR~t!_(@JZ}ypAQS<5)&p+Vj<5B~0_pMt4x`mF~>A303IS>%g4SvTv z>=PIluR&%@L8$FO^;yk$IhIVL)T|~-o_6|}GTQVDXQgcFr-b>1X!%$g>&lOdV{#|g z-LeM11o7IzD!Y9+aTc_zqeQuCO*}YiiC^xwj-{`v=wrH#P( zqQ1eg-=l6tflgsI@g7cA^?GGt9Y@WKr^L;E$SvOH0Xg$+=L+_25o=1o_|Mqn!01M3d{Xu4@G~#PrQX!5qVbzQ=y+Lym_;&r@a*=P%>B^q4!1SL*7^vyr(=f2p zte!Vt zG0FGqetMD*{POYyXN=Co04lA^>34-|L`H>kMc&5+l_mBU;E1WBR!bXymOyyWpmx>@ zr8I!^EWuY{F#A04L9!HoDWMOHji$p;=4>>8I{Xo|RUc;AHo}%L_J=qdcg{(EiTkGi zbDKvVMcx=G8>$7%MmGzoE7PPU27v_!kE=t!ow$^gAbH($%eJ90QJJIec^d^|dDw5b zb$Z{@(QV1deC;5p7%LmM(5>{cig8%KYdQL0Bt5cJNcp|(DT{HWyUhaFC4{M;D{RkZ zK-$iSX1c&AC_D8CRY;)!cWseNBEqA^OrptI-rmp|0aK@A3&!GYYsB)OwbrS;zi=12qpRJZUJbzZ7RKEEwB_XAK%~j~B(j)BhtCLK~vujqKU_=nj z5rJ8Zc)-PMj4t&F#Il;1DdEe3Iu|6688~+U4Nz1@$-6tfkM_qY6tdYGf{Y zI3Ys`jNIG2;#%Z09&>g73cBx#2PRcL_+>}*a#KiLqn`@MH zzcC2ZrnZm^sgL>8t{vOOKAU7D^+$mJM;61_!!|P4BW7{z&)@v0T>4^F1tV2Idquvj zeRU|;!(HG{m$BMo-=Q_KJSO{RG8;Ou*lWJwC8_^O8vj>f{cSS8z4+d)#`w--gn#dd z{((*Zn!m}PTTznFM$nSK4=nvZx*9hJ7yh#gnNk05!y3;|qsze7RR+ti=T z#6SlRX)3s`slOTN*iAI2vI-Pt%&CNP{O7Ne*jfq{6_HYE>JQ0GUa=oriC)SBsbqjQ-bZ>32e$=;PE zVC+t@X$~x-?h3&u=enOU&K+TOChlB)qeRk_pi0_R4ZpaH%~Maf z9zarULnFGJs4_Wy_q?j{f*I>(gf8@qv9+^n*SAa6Mbi64z0qAs$JY1VYi7$RBis_m zbwo&3N#^8L;OGdR7I@IeIo?g41O|IbjtTHof`2KS80A}=kd%fvWVMVr${?;gN5%B; zMV|(#G~aE%Wa0pjAQNR%npLIt`wsYPKrDo(Uj>DKSbt7hZzhg*sJWw@)-W_kUkf4O z*XWeUdg2?c?gE0{5^1DzzJiqjIF~i}G_%8pHq9OOnQ)LfF%`9|3(4-uEbs-lK{OyM zm3UWjW^2Ae-dwhAo^(TYp+S>mZhz8Xm8q@)QDVdMhLOUiapSAOc9;R?0G4O-g9%t+ zq)N?3K+--c8z+~>KAaA1;!O>oD(pGAnm93=uIP^>lR5ZpJ#V9t306? zd8542i|?W_X|&0tlTr3X16(-3ANa}n{j5elFGxr(ywLI}hFU=S;1sz=v>&a5h|Yj) zWg9l+N|&E@hLxNL^EEREu#Y5W7BoTa`}kIbb!@ZgE0H;GL1N?zf74F^tK7TO)|mg|jg5-xMrs zYS@-IulDax0l*;K!O_xXRip`;SwFm0o4MWVj>HtME6{u#mV zqA)j+-hH*6^-TWP1BSa#k3!qfSxOwneePB6x!*yL>*c{O;aB7>P){-&i-~CeBLGMZ zlD$(7X6Z{XmTjXmk?V)Wk{`4OV|j|(MYS>QU}NP+-N_FgQ1)P6lt$FbI={!`y>>d> zbxIsS44=V_D@-5wjEIYmQV1|0o_6N_VZs~2h>koRrX2eRQ|RP?4ZmNeOCxO8D|>(j zUCg{fqBn+SkayB0BRz4@mqU|nU^^>DLqD}X%$xvzhhy>nTV%Ysb{pjvFzoqg+ep08 z7sX+or#ohc(YE*rj+VP2SSt=vQ{8tT5V2tTq)$o$kELzxG}e9=g;|spHO76*t_*-l z?KDm1o4L1T2WXr{=}Ykxsp)2!L8u!~)z#%}FK~eiJlw_36BaY>OxuK@EaS|!Z{;Kt zadGt@Ik6{W9OKTA9=pkFz$g`wO)U2k&a^I{`5U{&cNBXa+s>p^iJg;WhPxN=KZM96jQ2(H|mVbz*eGMEZ`>Wbj~XxlampCYpEcL=OsyZ%OV7(xiWm zNb%Za$_V{lCA?J_`Z>E^_D*J^oau8J+m7vA8s`~dSl|w?&=?}36E@9hS~Yr3T}zhA4Spq7hoO4)EP9H$2I5~tiTan*`TZ%rqScvC2J;KYoJZ1;v?!WSvb`n$ zrOoJWXhX&`X3j6U1=6%!j%&W0-FzFpAS841UK&E1a$1oG7&L7vQU(&PTF{7FYt5*l zL%VlXRuxG%#??5LtaGuLi%XkAPD03j9XuC=Pvde{|V&k%dK8g4PZM4`YcZ(f?qy%>fIwK>&7U}>$18ex8KfCK(q%|-25*Rt}DVB z%=+>Vmc0#TRe`j_1?H}7@Iu0^ihQl&v-Z{&ZKZ|Nid1utvU{dFiP1us=mK@+YEPAW zXjOGd7PzcxPrZ{Kfd%af_1+PIne5q83B?{zqmPbCpCGX~$mHu_Qe{DlSo+PXgJ ztEA~mJG^)Eqw74|(W2)(`(-TPj0^!C=KIPW1W8Tairb_Li3a;|IVjJ4@<0;%VAPHmFYYy^bok(0q+=WCaMct)D_)4tP?F3-u z11|`$^d#)C>kS6cu>>Rp$#+Ds@^LTtu;L@{q9SUe?y@3kqwm5Z9HZ{iA{?Xd;v&wY z?(!neqwhqJmyL~(&H4i1l{mfP`mRvXIlY4V&hek}`g+;Gci50uW2hGV>(Cb22h2(1 z(2*%NR8r0XCtlHai9yC;STRI>)l>x#5jvQU879=MF(E=0yeQ}yCLfhXsG z{eEoyu*g9C!Exuwxx7sd>Ban9Oh62!gkW1Z^;=quhU{AKFC?SpnbxI?5J5cPNASCb zw-n$V*}c-ieZHulETH+QFX`{1EHd6)^N?9NdB?Z>;mO$Ug8P;%Mnf?O{L+z&w+P_B zp`x?j@xX&Eos!H4(&)x9qr(=YcUag(AQUbKW_^zlyz6s7#n1JOc6M z^37(l`0+<8SzcHUoS<$RL%AWI_+2Ag*zoya&bz3-R@4A9C^iJF5Gn)3f+(q( zaCG%=X}}wC+|lBe^^qYNH-zBwCW&xE_)wAYaX9ELlo_{W^{Gs7^H@#LAnq9y+dh-& zbui^J%Mofc>w!n8IRN-N{eH|25CNx*5FYmBF=_$lh7g{{ z6g@8=rnm$sH!#QAWZN^vV8()@FM>Y$a341y)&SCe3c#oMT^hkjl<<1_ zeyTRjemtj|5Y|Gy{$i^{sF>!o6t7Bu{0ee@T~^$#Ad>L!&>zl$Ax6+cIYGUCY`=n9 zs6JBfJ_HOwA;_WH_SB}{zU4)Pu}AdKOVvMfKpt;V?xwSMbH7d(<~VAjIa(yb&F(H%3fpS;0fHeq{(b{ z=hQryGR_wqp2|eX0*n%sb>3HOb0<@SpRSAdF2$$SiT(>yCaFKpb*W|Ah0@iV+E>@q zsaR>QkJ!`hiU{U7WZj;fT_kZ@>NZI#5_mq&wExbWdBIIM*_&FyqEZ-@cJtn#cY1kW z=o9Cy(A*rg{!!>M)vih04~(Id5MZOJfOX2lo{qfx*KPk4%VepPqmwRPwFZbNZd4@q zOCVigF1fjj=xLny&04g>d2hD-Nu&SEcIIdtS_UTomq@gozo_HtTaEnGc%HkBBuRQi zC5jLCOzUw>p33nmXSv?yr~!a6E)V;OkmY;&Ort)l{uZi)MCZHc4)A9w_48wy)|(cK z!&>OgMvcnpDo4PKVcG{kQad#vL3WJdfa8~L>o1b=8wHCSduS3}6yqq0t=n4@g6mhY zzID3{{PCpaW*f1X{4Y8Pu?~qo0sDWx1v@Dz#nuI%;=ymbLbUztcr~bWedEmiOK2D>W2X~%%u)I9qnF7wPypVs3_oS!B`(Lbr20rJo8)`1P zWcl@s^%D__XQkI2G|y&-vqxK)`xb{y@5((Ho2W7@G<=0VDOsV5UV`icT`Svjs$Uy` z`}~+;I350P#ya~omGuj5?byQQXrV8Jnd8e0NlL(cLWL141$&9-Zm);OjDdc_4-l1ViGF`Y|77P3KM2#pk&Z#gGca6ibEU_*OPXO}#Pq*r)~6JN>hGKSVmH zb@!~3F*@#C5ma{vkQENufW?YU>8W1Ja7v_$97h#2%gMKy3q3x{SHn{ zP^GGodMm<-o+29dB`j*+SlS`t*l2XlngJxVNXrc;*3Khslm2Dn^Ozrwx_D8lP6(D@ zBHu5Bzg9X3Hd(Ptjy6lkWM;-opodFJ6P<*Po(!-U4NmtoP+#=ZY31KhE~MLog;JNF z=Nt4()Z_cr)jGNcY9ytzwbkDOdn*&;)Xve-oLqE z7DolkexqSaRD`+Ef#6(HgUb*8%y9n?nnV6fQ`X z6Tg-1`Rjs~tY-OhDvGkg(GD!pPFYqNk;#NAwnG%f60V>Y*I^C|ZDG-WOPx;`F;)czWq}@W$TbBA zqJNMe>@?qrQQKh)w~%cX=vyEP8Y0ZOQ53yNcrucsWYhHw1|;OAy*18AxNfcS=RzPD z9tT|o#f2CGPTDzK?v8E+7feYpASx`_=j&%-{9*7Ne1T)^WiyoQ%MlYazajOUR{xtH z>tYXv?JuUL^J03YU%<1+!r|LCZZOhD)cV_#ITKv?)lvjGAo>dCLNF^e@E!a^oLf_$ zL`{RD*9Q|k{4?LPX>`Q4aguLr!mg}XZ+1Teq8xqg#v$(yQLYMvC&T-cKMWLtGX?{~ zC91S32;_}v%ZP|?6x)aju}N_YV(2`rl3nsnXKB5|APK+w<*S1dZWBk``@4~!m_-Ff zYXplMaigpPFoKy_@9D2w2}u^sLR)cxKl=^3$VoZY5r~)>G}#%-(WY16PEMuBZSmSp z!zq}4m~oMY)U#9{5oD~OGwGUuE%0a1(L$Vlmp1k1AWoAF|CVw(>2K3vu6rfdHb@IN ztApN`=&NOoF_LzW{b?b5O99Ap{w5AdWa;EM4q1ac3>^2UI1D*SHV$2o)}zHdQ>4D6 z!`#S}Zej^KY|$=arnTccY9XZZ_zK6=CwmAxFd~B=+2J zwJRPza3?HLU%;H6DVtixwhXv51aJIAzB-kT+poJm3*=IqAP=*zTe(2RnJY&N?xaX= z_Q#ZL1HrD1dG|9N=q9ZigX`5uTKfPkVY!+pdW8wS5~n4gh1Jyb#WDABv9wjE2&R9?ouDuSAg&dWGtebM9z5|b*aLw`lii9|k8|pKi#%ro&0G?$Kvjp5vs?cFBKz@laSj}` zFs?(G)}0JNzz=x^N+3_sI}Kr_|KTsA;v8JG&4eGwQ=}o#5Ua||n5DA82GANqe$gx1 z3>x@&oJiQ5Sh*_$mMY;nN}7(NdybSoA5XIsBx){%Ie`wIOW zxZL|=-+#3V0;7DBl#``9#k4ZX;P27dQpcmvG2vo=P=)arpsZ$RrD>oj^i-|2Y{?SX z<^iqs6*Eg`;pe5>aOtSwesWhL{Pq34Qov7_%MIo|9y%=sqyp?3lZDvwEEk*y%)*L4 z>+V}}nKAL0Wt=3Xg7OEQSvQK?V)VYXTZtX5Oh5c^a=@C65;hQtPUDy$(4Op0%%Wmy zVp7I}dt$9Na(IP=Ht^gJJMC-(!r2%Yf3>mJ@TR*VyzuR z(DsgQS2QP_=5SbIIihqq=J&iLOQE`2kf$-tCS;<*UO!H02l;gH3Tmz&>Ns~NrVjy= zgKK(KTpv8YVyV_^;>Wr7^M1wS_(>(-|5ud7C$#+#Pqt1)JC-Pf9~E&5ABSZq!3o;| z$6-LB-@y(@I*1Yq?2`6EWd*K60o#a%w0?oD5nF~OR!!tW^Eax3+%oP z(vjr&k9G5!S-(LIMU7|_Qfg(&zNrlBZouM+JU5mKvG!nN-t z0~;=v#T8*5s9aTnYu`iGOz1ZecbyUTb`;n!Jo{a`;PkziGVZ}No#^S}_Nd>H1wv14 zw7lOuq~W6J_c+Fk^rcKffIo@7qH%Kwe~KH=Q?N;5X`}hFyVM~ihz|HC#5@*afIp!< zUl8ycGs*XRWCW4a;b45{@JnmB;+~vA=gCDM!S(%MYo&?crN2JI^eOAx{o&Rqx~$0X zQl%p}xnw;~?Yd-dM0#wd1T{AOl2pb!zvOgq%tv*9P&%_jh5*3&`%BU#Qstv=PN7VD z9Q5lS7lhw{anYrQ$f0L8e^z0#w~DTuRg*KR{*tNTS5bXqV!3 z<4nI@XQ6m*57=z)dyDX&vzngJqU>I+MU06GUtRGAM>A zw~mi6bwN->|3#06C}<{A2<8>E_p>&8XjkDVF2W1XaL9;gIrAis7k?|smtN~g0TXIq&h|0Nhp-2e zlp!jP;x!Z~kCy536SN?zNk6GCv+VVg^nYy2?}uxoTX@BZN}Osad?$sGpL3q z_FE@a%;RE@Ox3d!gTrdxsx=f-2{Jjm9CN5Ry%^*bkVmW>Fp1L(3mf5$9XsZ$rHrrX zn>40ut2#I+7}YjJw*vdPBDb=fSXYivN68aO)w5ueM$h*%>h|a-WZ1i6v#AZxC)v8$ zbrgy$e+6bKYIDv;1*E7K6_Fqt(4_jWWoM-iCY&VOGc{D=y(Y*zJY5;m=Vv~D%&bd} zZQnmsY%d0X2S--Y2V_q~wn}(rw_mK;QOu6G%Nz_P8;X@^6^l`kC^(GjaD+N4T9e~| zBS^#p(9jpbh}=VJzeMMGJemBy2;RRn%6azlyfsP8-?~d}5G(nU{ncLUs3^BCBQ092 z?Md{!1r3X5Jh}`b`;Nlozl(M)|d2H8Jw#1Cw17@#p8#6v1UvwSlUy#@7^8p{0}+Gzm9$X7g3-7 zFJ_%C8v_d*t2&`b#Ev6`PLUi2RTTBc1%gx&@jWUkl?UWRkr8o224^#!Ra=uY6L&?< zj8YXSaU(Gfr6_9X5Me|jrs`|`r@KEb-h*$?f!}G<8tC~lT-nZz`}$1L8Dg~Yo7vI^ zFfW0_ml?w)p_PIb?1>|EF`PK#`6t3dsarjWx=-LqE8M0k)YQwZBXVaR2kuf@p9`)B zxPe+1p?PK&%I%i~eS_z1hYA|UaKlrzvx_Y5Dh7SmJq;rcDq)Apf`PUEpNaeB{rHuT z-2k04jF{NV`}&9^>&#k4Ws7i%ZakkP6AL45iPLPzuNpGnOM$8NI>xa1KD@At4&*(V zbqBDtKMUa}Bo|AQuLj-~c8ORj)Ufe|*B$m@j*ebeW+MxX9EJPMk|S0p?9zD`dt9h% z=eF!tAtopOvzA$3ksg6{p&vtoO)^ANic(EQvw3Zl8g#6mdgq~>%(QxYC z$r8}o`;p?LmS8Kli5$B`{Y}t0qgV=^lel3#FXm5nN<$K+<&3`lhxH6TZ99m3lTi)4 zsWAShoGAT&3t#fT!>6G17Cz!~icSy%4B;VyGOv^LToVhAxUxn8xdbvUBVuH6<4H-_ z)SvMS$~!G53V`&}y8xUWZ%BeH<_^bAqvht6wyUqFJM0z$P=Av5ZqInJF9+g{!55?B zWoxoNK!n2_gLi?R3@nSfi-@HBR6M9$<7Z78iQOlLCB zxIQ{KL_i%$wfHKaPu>4Y*Urzk8}OCpd{~4~l?3W;uda7jhBcqhk<4~hQ8Ci_Py9OH z;#Zt2qCf19sOerf9Aa;{e(x( zfGtH$=Kh2crMrkkZ$#zEJM=)W{s?8+)(7?e?#Q{t4h`gek}9|7!r> zl>F4`|B`c&XJa6wzh+|q!BYa>e(HZOIDc(St=W-czDEKDe+|a;;Q2r)j(A(yM_mwz zk8Yma6%s1c{|6qi#c&Ard=dMc$PprnUQE0itXu0dU!C{OzFBuow|cSV#bVRfx7!~{ z=B_FK7g^>x7w4PQ%d_Vy_x{hs)vLED>pS_nK$6;)T`rHV#ki%F3$TyCrtt`gN7qao z&&q{`z^3Vl-kDdnL4g6WJs#DWX_SG!6`t1ekeJ6soq?bQUi{jSL+4$rfuAMbFFDf) z>M+@UhDehnURl5Z4ZA2nOLm7*=1CnOoIJRY)Fug7lHK8#(Pw=lmFmR+JhD5qGV7p7 zJ!(v!pgLFu+rGhef7IwnzbRmUNgfh@EtpGMJp?B@6JSsaFC#(l~zxD9P;* z$UNx-dJ_gkWQdan8Ci&SJ~d2^FG(D#^(k*QX61iaN|*@kJT@!ON0Rfqy6!VGzhi&;! zpfm3*j035Aac>V&)HF8jKY`dYedHrBzuU%PqU%Os=yt^s)az;(n^xuF z8}UIFbt!#Es5T_ZV-vN5#rVV%1zCNv2G5s(he70*rh?E>#d{BNLJp39y=H|trIi@l6%iCxDOxWZM zJ3z96X@iW{Cx|_2gdHQxXb>N7Y##_dnjoidYmzu4TFup%RMOS>lcFhGzK8^RP6&9H zK;D8aRD=Msg1XO^F=9*saeAjv6!5QK|*%M}0_>&{8=JoILO`u(%@e3>+P(P2EGe-|)G_s!zF!e4F zR0UrWfASaN7j zd=LNDP+`)go7Vs&OO+rYO$ZG>YpAenfH$1s_tOOi4C}>rn%4&*1h>fnDgM~*txDP@ zOW>R*OT_T;IRB!B&u(`_@fa=2Ol;jTGIs{P=m3Rx_Tve1rc84Nnq4mpb#lTnCU*Ru z+Cc%qxed!SvXmjAe%feX0!O;y z_3$6#I{}jIWWC8luy^$!Qk(`5DOP)fRbQ=nee|sh#wJ5F&(LR`uA!+`i5Af2_OlnB zQOc079nrb#FNC6A&?g-4hP#NzPy72uI-|YYCk=N|G|vJrQa~f{E``n$B>gX1!047M zJUDr%^-e(NDH-7WOU> zW6d%@cwU1Dec_QPi?Sy}nAl|@G}X$=ZnD(wLh8#1`CeH(r$1OLceWzrI7RuJ(KdSZ z1X+>JLpx!%&HERyE^cDSS&h{so~>z2-xq_wiB1VnQ(5Y&`6@Mv(JuzuK3Z)0OE}8< zjG7?31WSmHpr^X~F1&Lth1PS6}&$tQsf9N>@_PTrYvbi}Lba8=MzQ zG!~#*s%sgr(Az=o74fSOF~-ctI&`Y+IptuI6~yy5E$n45KpO?`q6`fr-?kST0LXh} zVfg3{p6jlUbSo3mkC`FYFo|Q8?G!UahQ1Vy1XJcj;0+oJN*CjgoP9n_z$+cw{&L5c200Wb zU{*Zz+92-RUcxlK5&KY%D1_U`8;vuby;AVdqITowkVV4DAw<#hI;$$a)sIYch(RcI zs;<51>o*}9u5MIcMq50dqpwuiIfK2o5xm#5P6GXt@xlM$4P>vqS3SpGbhNOkEZXXGyZTD4X2$v+s;+APN`-(13p$h=!zAqXbIATYp= zgiXtYn_XoU1VtP2CjII5UdkZu!SiH@I0_XnVlq&5do^9iwuH+Y(XGf!+C^3n1bf&3 z)wbgsOz^UHGJPIbLTQ030?p0V0*o-QX{PPxHsT_{Y1 zKdz{jC#us}vemd8VcxTd;wx`sFlZ?B5z;hFYGIQ<2sXO47E}|n@LW|2@TjT5a^$tq zU2n!ue!46_b+F{RUCuM?Ch+yW1;^>FE~PKC9I%5WNERS+i6FbN%XZw~SD3_d8lrK% zW3s%v-2Skc%i*_ivpqfS)4ct7OEUhxj?4l{ON&U|U|W)g)ove+?J7=-uNN_TLJ-F4 zB`15dVC^%Jx}SyRv6Q@Yn4d?tqiXc|z2;@rh)68DnP_rE52bQtm?>i9!M2uR5X5IB zDhXD^f%!P?K^{8EMddjhHGoAd`3S;<*k6-Nl6eruOzciPgS*ThGmDP}wK+@kqX2R< zx`1lx-o%O@_WR~+x+(t1SW6G09ny&0vyNx*MH+HwbDBhm;a2ihXb40aa)0t))My#L zavN~XbJevl08~Wl&0h*s!ABS$p62NmtVvU^zb!QpT&n8sEf5~5C+N@1uxO|y=RC+U z^hb0!G!bj8*mfUp{)Rf%N77g@5M}$ph$f}(Nh;4(me`AZtjGA6_Kn)p1YaiAfh23F zN>q;!M|b00u$iW7(gRuPUpGtQkkoP$X6fW=GEZ6uHh@5>!hlpcH+Vu!UM8s|qGV@8 ztF%!LlvKlRQ!Ttd%i_Ydaf(3LiTI7YSV<$5Hei^**{R$D*^GBSBfsP7{)i-U>W9LO zZ7+(AY;L{!jUMp*e!^#a8rA6ee^55jS$0QEi+wrJZ8A1M;T!h zG1;P>IXxGiWPe}j6nG{BATRTMjNbaN#7S94)TW42lG*S(CJSf0m-+xJhQya1Okeo) z)!Db>jI&cJzt9ZLLRrVRqkA`N=dD3*JcXQJ5tojMrXFT?HS7~pk;6GuIHw`=+Znaf zcvm#JWvjMrH%FsttjX3@G8gtSimhy!5{y1Ai$t+<_2SQ%MPNBwy`(3_;4icx#T8dL z{F}#7oZRtJ_Z%2@s>d7>3M5l0cG|}k&a1@!39OG^a#RZ+3;5VO4d*o4+zfb=9KLlu zw6$L`=mELeiT6h|s|ju|r9TVIjX`6+F#sHFsOQ~#tWx{;9m`kwnZGvI%4u(0?4_r| z8-3SU((8D{q@B%RLqQSOL|pn0hP@OEIZm%t-KsQwH2p7&A|0Z z0>`)G)8<5CD*@=&QhytJypjQtcN%?94q}iyc<6I&K;)GejgAl=zq7yqa zcz~_4Odl_Qq{;#gA_%FP>_-Xt9^q9?6AQC!F&g`8Gnj=$YlCHV(=6o^9pHk?mdC!P zFkPK%8AL=boR)`@(33<$H=~&J=kT3b0K6DFHJ3S}z49^Ou#8IXzT-GQ&muvnl!n0C zKGisIyEXUYzP0dGR?OiTd*(TOsg5}v!t8pAPy$p1HdvA>G1z(7Cw3sJz$bLp%7JZw z84c;77_4HNc+Zz&fI_xX(kLb~l7gqoeX^f@-TGYF=%?nI^o^?|uUdtoq2exF3+s8Z z^U0lss}}V`f3~;8NQE{7nR~mmBn^pWL{zaqO7f>2#5vsZ=N_eDJ0OBq2oaFyxb^C8 z(;Tq(;UPhid{VFh2Vf#ghM!Z>-mK$K&a8BA3`RF`sm)7F>d2a7;%~HyzpSP5=@-rL z-8;8RiTx`CcMADnk@<7bkS-CC?wTDUPJgWNAdO^+0q)AoZ;GAmdESI5FV8Ji{~6+V zCdD+7|6%yp=}An>%8$6~^OOfsUY|IBHt}f#kj0`9ci%6yf?(B+XlBfi4D?~jQb5hT0fHqk_iMEVe*u+zZ(806k z3zu-zNi>;AeN42T655wUXK^!C-OAP15jF!nK=SnoHx<_L zp}R0H9w3LOrUr2#0x^TE!(tpUjLf9CL}59t0n+pr{FF%5{Y!)~YKE`^hrh75;9(1` zb+WeLEHbBiQFoTvI>HF+Ad02gB4PCVuhd*f{3Q%c6gZo9@?P|UldF3uA1k+MoU4%< zVRU&Od7YZ=_j3^9f>zkzF%^T0KSPB{-YMS)83xc$H~ZDB|7knBpg`<1D^2%2YReb_OB2RaE z=3a>2thr0u@yEjN%<{C~3CyHJj_*PX7C94Ob<_=4ks7NvycKPU*3}GFkjxjnVb_eM znAQ*Y5Tk{B9YZD3GN}Z8&qH)JyMl6^Oq-WDi%MW$0t?c>^Ak$fmulTlLSRuM!A)d= zRvo7U3+fvx;K5`isY z&|>$3hG7*r{AXX2FS+jE85m5nN)%mj539;N+tJuD>5CG zjh{5_ugU5bO8iz%!*x8LT~(cNusRHZTbKrwmtY80rN=k*4Uj;y0&&;uN~(Z6FIqbe zjVTttE&1#R{i#)I-B&LgWd#?^<2>5*q<&CdV<0}WhUA$uq+Zfkf`Ntkl84o~I`;tf zUI}J_Ed!LVP9u-}TXIQb4h&P%KRi|=cMgj_bO;G|hTQD=Y_{9&yl!(xZuvKPm0YP9 z952J3tRooTpgVQm4%q2ts;#2g>YBU;RIJ0=&X=)nUY~^oE7E_RCR&6)_&J=vhJ4UE zfI_|Hlk}(;;?Y(C3pOkts`KczI9qf*bJVZiUggohay?_JS#Ul(*H?=Y^d;9$`Iv<; zD*mqK1Cc)5^LBXxiq?Qt8$4}aGaFeAgt3iOF`TFX-hsNY11W3EqV1hS;f&rklFfOz zz_v~siXo(PTaB?d#CMx$Ir9AmJAT2?^8v#rs?!11t#>l0cb-#D&Q#jpWs|_$4iF1Z zfu}Cq0FkhRKOa2ZiR{*3Z7cbMs?tw`7Su_U7S4Q)aW&{v|IP~4&Dw7p9EHf*oEIxd zYx>J@pKN$b%Yxl4lRqVQH;>tfx#V^12}T`YUY~^QR$U>l8c%$3?BYt%b6ADYLt-6R zC*-k1HvVj2YSu?Z3=9c~?4W<*rqUkw@Cv{P@t|p8em~GI%jlKXaz?{{Ui-798+i!I zpVGI+hO!wd0VM^6V;nxc0*|+`FA}abqC~ zNy#;_7oj2VAAgNYx%L|S(tWbV##!sF{TDeoU*0Z zmIX1gXM!^Z4ve34FMe4fcfxqF)zbTL^7vL)s%Ng20Rvz1@7-gZOZgh%10wxc%7HAV z*8~9toM3+72d6g~1tN8*rxR-&JQ1nX-CDic2pLR;qFph=qEO1`A3KU5>UmZu5C5@W zR+=1MBYLA#c)|BG87_XN*pIYyQ@d1QhFMLBX2cj&&hMRLcKm6R!}U;;$&@OLXle6{ zls@q~vxN=Diw=j-1>%-}27@3Xys>R|+_`l#yx?tlQTt(EH}&*5n}b}d(DImNSENU! zyVN)XtR;nq_60>oc3yG6yYI9TB*Q`PSg2K$s8uFp<|Wurm1s#c00+W!yyfci_IIN$3~JC-acr9O%ZKFG;WRNV?1=6Xtb0blwY^AsD!8>{(g!i8WgA^3u<}lpjYBYk#45H zHZImEK|50G?{x6rA;52o$9%?>+*Xh2L$(VhT>k*WNYw^cw2XSNt=We8BbezzYL8=@ zmd6);!}}mnH@IzLFt1n=Wka$;p{#-^wgM0PI3Bw!KZ=mE$WXGv-RrYduJq$jg(A3x z^yMUw2P-cajsyxkIl-s8*AVadaC^L0%6~s&he-k@m58?a%`;j271q#%REo%3Urtc5 zk8aAlD|z^f9l}$ZH!9C%TcVdZK#=8Kp(O0Y-`H=`qjNB<6za1UFHx{-yA_v=fZm4V zD$TaP>=iwD557_JhjH=Wr^z-iz1+aKF$)?A9Dir*2E3S-FSwny&$fmIi~*_%@ty%Ruu#B)}o zIoMw8{ch=xKO{bF-jvuZq4|^=5XQ@9G}fqEkgJiD&?(RVVRvDdKO?T1e|TgLOSl}6 zKj&5PAZJGri60(nhBaU8qhs5)ZQEwY$qsjHTOHd@ z$F|Lm&5rZtf6jUC=YF5_W$nF2jj=ve)vTKHn!n4=Q9?#~t3JnvYN!8?7-9a|S9z!d zc4*Ipk$^CC6b)%=fSnV6Js*MR*pLt8XucU>;IGiDW;(dMKkrDx|Gj0OffC3i8SsaQ zGi*8mY+9-3Rc4G+Ulf;M0M7>-_FAJ+B=Q>GH`M+YenArIs||lyf=}lGRoYs@Awzj5 zq~Q=knCIXADD~EciE>EBveS%n_2o4P#F>v3Zc597_fazVnLPo+iaGu z%1UaL>6PHxSf4>+zjva~%Q_?IcEIctJ=)x$e(;qL75K3WrAJv0;x21Dh4l##TG=4! zvSjiJk`q%P#D;3U6I-B!?wY-7ED{IRSm3Bq8YHu()W@(B{xQS+w;=(T;-ABLVP$0>WTX zK%b5zd&Y^`vu)M6_DQ)rs;6i-=->3XngfANk;z5A1xXT5ka&Pxi4ciE$mJJLE{xR` zNn8n;3#z|ua1o)goStJZ-*(Y>pQar@hh@Oj81Ej_9*d@P2Yuhl6t_(v+BckDPv}yz zdk`VObQK!mVrr{{9Xp{sKOhYN^=yptFfsI{ytw$WplTem?!vHAiKql--oPuW)0G@^ zu;ksko1fLDY(no{#tHQtEQDUeEwdgSWhy7H2J>!NPGyLyB7`#kLqayx&zKwzy}>Xo zP2`mWb12NS5pb8UNgIs4C%QK5T<{^WhoV;^aWA7OwqThhLKXE_#TV>lG=pFHON-G{^7+7)SY zR}7`9LK=!Y-RdKOcU}60>4UlQ?Spjfbh-B|n3 z$P0?_dl%Ok4#^o!*h6ygwqU$`R+D7yqkyg9JB#o;iQb^oQpQiSHVNeydsAQhy6&9o z{-hlXX?#}cNs#R$SC%wMeAfEzSj}yPCsZo!nQI4%p#7VsY{vBJnDKTPq@!Z+FpiRL z^%(X20rghvko7`UsyN+^8kd75_`Wvj)!F@)dbD)(m z;sj=uV}xbUd})`qYft(fbo-6;@zw%yVm>L&wugcjWprDCS8*#mMjftoP{Vj7iRDo4 z7PNfHB1-`}y#=Dk`IJt6PLcv;=&m?L)KWqPFPrwft^zK=cI)WTxw2nMQ3>>jQod&; z^I_jIW>x2mb&4CArIm|qlzXaQ&&h0M!H}^G4ncTJ-r5*hkLX8EAZ=Z0%nZLXB${gE zch~syP?Q@uS#*I=l|dp-JzklXV;9GG50GjkhCYNFlnpX{AjS>MdA?hTNAja-f9`^6d@o8q za+^v-5&JVEP=&|BvV1#zkZtj1qK1C_(tjE`OUsPQ3_;#9`|Ac!3QzBNsJIG)d70-U z?2H0fdbR%blB6;Tvxo%fCfMsJr!89Yu|a;VD)6B>Z~}qqbT5sBKY1MFQ)Dcm6%ck~E8?VuyE+N9X^OQfk)pI^yr$2&zoX11TD zBo<&XA}{OEvJmA9P=uxWD~YJ~!c;zN^Kva{nhsHK#vLEcLZ;GAoGE775N=ylW*DJV z>$xxn`4K&H3;3s|{d~h3YDVxSNKpMRQNk`8*#D(9z^7g#gFv-h!GlQrn^cloQ}n+& z5&t!elDM@1(Gr0OLj51!!+(Ea4sh77vpn_2T=Y#(M^8N=lk!pzsLITjs z_d(OSiNtW~@S#-&sT%@AM@@J^{(*B~bs~S7P``d|kGs1H2?%BAwl)@-k@R^CePUVQ z(fK}CP0oAX#$qwO;e9B7`8PK=_S-~0twcRNz_XjqcM+%_`oS))#As&%UPx=velw|`EUXFnEyP-<4)r5tF*beX3FMNMNd zTRYLv7X7OJ>G%0?H~NTB^K!5!18h0fg3Zy|HlZFUsVyznAtDgr1U})9Hv`@}TsvN4 zMzO^;3|=?byC63z+SU8rH?*80jPi_E-<(|BGdnVe+_Ha(7|BVyj}$vBK13XZqqtZA z!u|l~T`S)5$xh)yyTun6z^AT%NO1v~ zjF;Pv7IZew-!m^S*Qm<(LjlaWy5567qxqcNfgkgehQ}3z-dZmT8Wa1MXM9&e2e^FX z4VMD%GC55OdP%+%j6lzfUZ}|mQArcAimwv;g$l2rCt~v4-2QVwJiRD0<3{h$O3K7U z=DIneDY5G}xu~wshy8u0Y$;NxyQDV33g~-O9@%rw2ZszV?c!F@=!L<&8q|Mdj7Tof8!@p|L?4o#{?n&&#e6)rpZ5ul+rX1jFR_MOuV^ONFiSq*`BW_rWmc$a+Y8M%54N%NLQ8!3BnixJ1cU-Kdbiq-`1eE(~)C zu?0dXlKgR~3KdRxR{Q|;cZOMzDJ$A2U~-`t>S(<<2K+64WkU7rWuQ-x@v643#1>yh zuVbvm*ZI4kCVt52-FZ|HBLwa&LWN(A@wx;t0GjA+YJYTVY-_1m6xxWafA+ib*{kZ()V3KpZQqNkwdce? z{Gw!1(Z3+2EfjQ+=p^^q<@SWlPD$Y2-7kC>@9!@V zBEC_1-}%x#;%5*l{_;Lw5i{Q`YBJlS;Bo;glaWqZ0<#7K_+c$`@~w9L@e#M&L1Upi z!q z1_~T<)sS$7i(2@fJYt=AaYv8cxH{k-T83Pc)3#=agw4P&1XTM&@}yNHCZm|f5j+&l z2ZC|wq(&@B0&!6}T&+3REvmLsNQB?5=zd{0Y-vMETU-Y@aexi$iw#6fe#36jUNI?0 zn)|4=??=NY@y8#+-Ix%OfMM1y3x_iVLO1>eg= z+ba8VfTgmUMA}H5DRWQq94JFL3aqWH_b;`S?MMsV3)8=c9D0TJ*L^Y=r1}duv~DL- zxEu%L>KZfg^nk&r6Y?Eb#CqcD#%-TT8GRwHcbS8GhIkgbO`odjwMxX_3VAsM$06%a z4u;(tL2_6Sg1SKGv?A}qVv#Hu1z+0*Pw(*WqGFNh z20Qf~ImR-6ZwRh2ig4tL_+;)wKPK7uvi9U~OugoVL*}m*7T7JcB#hk!*;PsVq3bWy zrxEuhmky(z*mpqWeM9_bl>s=O3-G?+ELi_PU|!r8D5|B26omFaWHX`ZFZiw@4G2mq zLOvoCAX;@@9#siVFLbN;qA|%iKC(1fhldUhhFDwhQkgalim8_Z=w=z(n9VI4?gR7T z{{>PvmMI9I)esnKoxTT68M-%mv|G}$$z^Gnz7p8>`<6Bo5`A z$*w2aT6+P+{oCd_G6}_DLcqy%902>b725L$;99i@iNn1O`_OzrWCxPKgL5hD{9joS8q+;El;oczBs5}V$!ca7=eqJIKysxcIkS zNtL{A2C6od7t&M6#JMPI`|BPVAz!8-KxDlhnF4GF)!O(eu&p|$7_Q3F+AIIX@3T|# zDx}gmqspFW=B??2O0A`5X8A=#NyIZc=>lAa{q56EKbN@Q!<$05D#NP_x7=^V+J`r~VL=?y9&fjf}S+&n#2bJ&rJ`dWK4D9H~eXo8PJ)?l-F;fT8~t z+$Sc-`O67P2*=1cyay3_GtBrKlj}`>r4vTjytp6F3+!dA;|hJD!0uLjw{X;~iqI#x z>6lN_Gc}a%57G2g=9=W6Du*Q#4rU0iMHlt6B5@1P=zr?nct!Xo64*Dtv0cz97xxZZ z=E>AILV%jAij1P1mWO<3uT0>C0%E6131zT)zaqYvvPpI!zSENTHW8jKk!H~)w43S- zjhbVqF1T+lu8%xymk=Hedu1q1zWws88*H<|F(q zALA5&e^DnF6nPvVz7~kZVXa8{#`hl?aMQed0Tua9>ACoP4Un5*R;p!MwPvIm_9Cf+k zdfm8dcc1#WyCMXDVh#RqP};K$8R)17C-6L-WBD<_{KzGKW%!IshP@ zBT?gr?U47oVo?uaUjm&H+)Eszje>W%`{98K{l|S+{ha;7a4p!ySjVddz{LqD& zo|~^*L&KVl0Qjjr!SgPqt4kPmReW=_==iLB^dE?Totb^Sn)coP<6jljtXM zz#~;Z>$z1x!>|>}S7WG0P1+K8I|jUpK{XotPa8S=L#e_hXA5zi-(b6o#)Hx)f8pJh z=w-+@Wnn4n%~PP}?gYW!)c|6LMa&qezLc`0>{7Px>iB?a_)cI1Rz017b~T(^PwM8Op@H1n!@qPVbF1zs2H%*H0!%nEz`BRExE~lv@gFm1bjD&8V_YIv_ ze+hcs3cNaw2*gPV>bT0dG-6y^)hVQvBv?<# zCD;wctGsmwD!UrVB{+@-F5x;8GZ@IY3lDQhZpJLHwbl2HToMrKmYCL$2JVi@z$hn@ z2DGM|frsG{JX^Tl4too0?)kVj2&1iw4BkI{BQVC(=nKCXBLEyI4HkjDS8#TPHt+Zu zZbuTIlJW&1nU`g_+#fJ1Y+14{ck=x((5;k-x913tB~Eq0wO{L>TB`E~w)j=Z+y~SL zx_E>|E)?14T(eaO%(ygX@C}qbe9v;Z@I~TN`m)Lz{IJZVK=A5FE8*;bB%}5l@?=MW z$IF|p@C2|*D%uW)(!cP0Mg^`8STq?zJQW5lvGZS+*dwV83xBt+$;}|Q9+J5cjxJxg zy=ib6*z&^UIkY$m&-TLfo|&dcUbh@MOkJGbB6?eEQ=k6$*8zFOz^YsF3mjDPA8^qB z;s*Ug#{cI!f|&ZF01+Z};wxyk;4p%C{cj%W*RBNf&oBm{Nz=+ubrJo;o_W+eo%aWJ z!p@g02#LJGRP&dlriiIjrWjNlMBsITrxZ}uyw8*ztg$SMN^fXosGtsebXtIgFgAF>72MUV8Z>1@5^Pf`)lL++r?qA-FC$P%_Su6 z1{-^M>?r~eRmgmq;v_rdgiT?3KhkP2WQ8q|`kgK66J$xmkgRg#zU;H0!4pZkl&4wl{pxz+Vm+Hd)3kyD$^)nXl5V} zkq7&c*(YKbhlRW#B|k>U%+m0w5u59YQKSL8ZN(S>On#sT4o8hbWiWyBV6S2GTMwz8 zGn?S>WB}PA1VqwyNz9GarfZ{jVSpEK`VfR7{ze@A6q9K&>MZ7lHsrT6y^d<+`Oz=3CG}FdGM}Qh2B_4|6LuJsU1j|7pKI) z&})DlxP?<;AmM%koPowQs*ipzNvnLX!VMqzk;fC|BaKJY?h&sK=rWB5%C_n?+IY@T zD*79!FHBA^Kltwf4RqefE1D;C{*oPVAEDk{&%GMXvK@z3tzLnZb=;?dK#m)D@K57M zhkTvhb=iyURm)ST#JZMbxkZ*ZkJEH*4y;G}(%o}KON#?} z9i3#7_RufSZc}#4>@5U`o|?op@0@D-usdJO3C}(=R|4J2W+Iy;a3B6L{fv2)USk>E zTm&oe0>8E-trH^^TZmDL2x~$(HOfgXWo@Q0-T{kBS~jKmIQmq zR!btRvLeVjWo+Fnp>ZL9-_YYXRD^)WgurBQGSCw*8BihBKSj~}HQTt_^0!6e!?N>g znbg#hv6W-e<4aY=5{6nmh4Juw5EQB&@7ijuRYI`#s<=L+O{6rD&?NeNLs_h@n-ums zObp_z<>eJ&(QV{N2<+B2o#%Q0+oElD7(qLSV*mZi<|&mo%+aTr6B zAL6l~6q7O94q=URO?pO)%R)8SliHGoct%!X3pjm5(Z8@+_04E4i@R_3NaDnT#v94m z&LgLFBpcgl&{RX|gVHNQrQD10<>${!P9q(Fciv<%uQG zY@yZ(o>1>{CrxhvGm{lC_6jk$R(>%nH}9MATWcDq$*+8Gcf&G%Z__dUj!&dklF4Gq zYV$!R2aDST^lU8h^R=KOG57^u{C<>b8K6~2i6y#h)$O)5+*gMlCwTMPkyOe){Gv=> z%PFL`tq~DO9Z~cBFS!6$0oq~jX=|E%otz*voQy}A-n{k$zO)?x@s0{9<;6$URj7(? zrtHL*3rk@K&sbFi_ev=!$HPidYt9HvM4ZdQH5iv0qMX+hjrVZY{TS{gMxk(8??wAp zPKdzreF{J3!=qi7+%5y3wM(4Pw)1xWRlT>RDj2GA+2o~&U-sr~5VHkZ>cftRdw{^^ zC7Emq#`Upz3ywL!&DnBq5OPvisZ!p!MW*Go+USVX3MUcE*n&;0jy&89Q|p7z{v`01 z&LKjBY_zc*%6!Go5_jxk7pV;43$gYSr-@>NfPR8r$V8rQ2=fm(_R7DBI&PinsvFVi zKa&xW*1YTZ!As&)G=4&)S3~hT!aSqL_{r8cBoxM)<0;Vs&c)=Yt+PuyjOdxT6m;Y$ z@MWckP-tU$ni6?-kd!*p&FzKg@ne$Rydc_{a9fNJ?1zmv0v0)iF_caR!zRhKeO$5K zLLFgX4lI478VDQ+>D(C}3OIQO2GCfX+7r^VNE21sN0IwYW1NiBZH?JxwW{P;z2eqF zT?={O3o%ClqQ#G`deKX)Kn)kuPUJ&Lbk`&@62^nUqIbw%S7KK}oti?bKC7jh&Os zT0mkgHo90Zqx7BEN`PRQK)R4o=>c4sFA^7iQRn2IGg)=`eMBIvaiJeU`nX-1IqD)p z50e{URr8X6osx}>mD?+7Z@iABk%@28O3Yp?y>A_9IJL_e{oGiTN1R+b#6oE_v`;46 zJDvvAI8O0EAM}7X^T@9MfM3{=v+dInea1d#p(Om#hZmO0JINh}|5*8knr3#SwA6~x zF{LV9)ECL*a*oBX!>KIlqXCb4l4RE&Mj8ZIC_%P*qzgxjLM?NILQRQet;9UJ4udnj z6EJ{~6IjHZ-`EHZ|EV+){hUs?E==;Q3rVC?3U{kHoPaXendTz|mWrfp3R=9OV0~yW z$@IdT=~!;eYd0I@w0&e5Iak&-b*(oWy1^Y`t%nwQO6^@IG!y*`N67+%(@{>S8Zn*1zZ7HbzQ z3WxD`Jptqm!&3UW8Y)9&!pcVERDyn9*~l`;NE^Dcs33F!{hB~e)xVK(A<3k^f&k|n zuYYA}0`n$_Bfl~<^qBuuxA~VZ(*HTjL8jL5gTQpNJ^5nw9vBF)2q zJ@nP)=^zD?rIb9skb>3O&k@FZ&ZpRUMCQ6G&xI{d=jf`!|^xg5a=h*?}NU z|4CV0!U17hlY*dddU7`wB5su;0Ld}t4XukGLPQwm>N1Cu z;DFOHvAp5$uokDFF!#8fohmEqirw1Q?tE`mf>e`?EU7^+pm=ids(|QZk~;DJC+Iwnjw35 zY?8sx#=~%{z!s~Y6N#9mK(vBsK-2eEyYrvQ-?&KoaJJEcp{96{R3(|I&=Cg+6Ddev zrJ_QU+00FhjgqdLH%J4~;u9!_NH>#c<@R@EEWLmmGN@kUWf`JGs|jOq$pt=AW@(BI z(yAm@MGXrH2~5$@FLHTF2+ZYi#_K;u-Q*gaoG49r5-&gF%<8*xA3Ktv5qPQO#d{Z> zW7ZgKFtnbLxhSY0sfcP0x#s~|iN;`y@4W?=u;C@!&3pYLMzgfN7eFEf*9tD@nyeq^%On>PS8Y5D-KUTPx^bSV&eKMN%~6l>J7-Be6+Cv& zqZH)0ghuJ3KXSN_33Y9nO14n#52^}FVg?+)x&G8XZbzer0zJVwk2C?tuhXalMp|{_ z1RBmwzGI_+J_(m#0B zVL*?B=-fhXl54@xuE=&4&873?y?H34 z?&QgC%7d&vZUmK9ph$zjd9H9$KT{)1Re1k0C86zMi6fgzMFwE1;0DXO-*iRtQg{Uy zqqufd1jNjUaDLf~(ed`pb%i4YD^vy0f4{yFgVHF1<&-bBNk2|PM0gep5+z??#7=GK z;uTyFyB1%LyDIHec~KM}w|zJ)lhC&;87=pWjpr4k0z@@Ltby@z_gv|ioODfd^V9am zxl4=7&?R7Jn*_}5WakqZ&nx0TaIS%d<7nWhSRzQh3U4N?g%Ow= z{DjVa!uGy~t$#x2dJ`|b>i3Uq2dcY`fYNv%G=+qI@^})8=ALz};d40QBnQ!cX^BFR z0dW(vr@eOnPB9v7YjF*HRVlOK{@)1jFDHlpJgC2Z9h?*OaZ&!s*2AQ3e_55ZAc=wC zf&aS_|9|rMkg0Kni100z(jYGXE^L08qW&wv|E~cYK=xMvQB_0x)DygPY=es8hAA#6 ziuul+!2m``*)2&x4ilKNWklZ}lI~sdDp)(Wb(NZ)T+}sAYqO|*6VX)eP`xxD#WaJ(fr)S>SlpzvJyy^mD|AAleEm$& z{0mZ)##+da%&yXLn(9%$D5jUm`;7fc(0+JEmhoXVLVd=pQksF^2Pu0+;D#gXki4-F zm|0mGnH!lJF$_pa3if@!BNAr4ExYn%=9Rg`xt+_hw`GN9d&0-cAF)CubBXrr_o#1 zi;IUP@gyVhbs|GEZ5LH-`+q@yR)(8T8Yhg=?na^8Sk5t;7?G9T-K{Zk{SkVl0r?h1Y2|=D}zimWwQ)VM;EuIn**MYnF4;v@P5`TkgP}-8F9U zC+jWFdQ}p1R@TOx$eAenO_~ZIzF9tu`y`ZRAj#^@(7mOd1u1D+-ptBk#lIUF-H+Ib z!ehs|;q)xTZs);-a+_{+?P1(2FG!4U>SehTXKN6~k-XBs)5%@G8f&K>tj_|AuGkv_ zU#zc4C^k>a;G@lRY$>*R)E_e_L{ zNL~O<9!o>nQq9%j?Tz*7t3?Z6_nj_D6Uzw>Vg`Lcg^|f{2&+$-?)B*o)2th*(JQPM z)lPsMrKfCo`f3{{qzN%q33}uBC(Q*XOH9ecx=3my^0I*1-iH{;qIdX~NWnKCNCan4DM9`4Y{lYN6cXb$sa! zB5|b#1hRw)^#U!qKZ;S8Us7K6`Z zRHBPSz zdA!L{Tax#@bAGzi~Fqr(9GKz>~d@&e^&N4 zPC0ccS3x?i)_8raj#|*4ral}?MzJetZIKH3X>xE@0T_zCn+&Nmvi%-oq zJn-_#lMLJtv&NF_qEF=5#;oy1h<)|-L?$Ovp56j5qm0xZD9hhG!3s6`Vyu5Vh6?M( zGc_eded`KlVlK)DeNP$yv85=n$xl2vqKZp8;ipEPGWos=yD>QBi88#mu%OOeItv}V z${=uY8HVfmLj@D09ApL}{9aQpn8x&(`qU}fJahw<{oh64`x*|pov+|R^`GGKzY+9b z_X|OQ{P>r*7ur`-YN7C306qrLcNL08Ncgy6P%~|%Z)v$KflG4(Ude=f*_XdC!x$i#;3A2obn7PfsfW?IqoPG(|pE29`xgDN1E$hWDmhE53cB-w+Ejryc24nbii8&QH2GtL=GgspWy*6J_9EDJ&gg6X6< zq?Q$T!v&BZeG-qm+KU5|bPVZ3+1R}lf<=_M4($k?qQXp0GarCp)!U>nQk(om( z6cqq!(Rr&jmfFxyetoN}p*)pkv6Ok2^^|R=S+j>s8YbG_RJ^mZGX-+UA>cqCCAxAs zCupo|x$~d6CoX&x;J~$|rJ)Bcat+oRUBBe#Y+Y~sheya;o?#b-Ftbb!;41ma^m?4}i+RM-hEwS-~IMlFMH(oQ%n zkZ==BOU7;!b(63%-k)X=30&CB$`Sgbv7zI^2~|@GYm2A0_tH5mSHFzq)wLy-CNiqW zWS=e=vVz&B7*c}OyO5`I7iANtg`;KL8_YS@0iM3ZMz1J(dvElT$9^YM#zRro*Z|PO zzBw?t60bCb2gd9(qRhK-!=;36Q|+J@OVxxhxz2Z=PDMV)@QN^&)K;C!c+6kU^*#F%YCm{RbDX!J};Ksuybs94G%fNurH|vcUQ8&?Tv51FYwX{U? z9Vh6g6~||IrR!8)e@-&X#F*CY@0~6D`qs56h4|brzchZ#9OowAWo%^!l3f7lMT<;s zNu#`9bxv~IoL@(y(5c2$3EWzdoHXX>G!HqWrNR!`-sTJaM}BQBKPLVY+F3D@UraNq z&@`<+>H#$gnrm{M6lL-(w(UV@)MPBpJ(~&^<~pL6jo;+wzD0b-VcZ6KLT*1wz>>UC z)xp}FV^hwQu?dO0o+X*Si9Vp31coUgc^9{W7N)f2`Rl|}w z#m93izoz!a?l!@`Oaz28sE2H;(e+Cos^OWyOv=q)TIoa)hP~;uoHi7mHVkc3#XN%& z{Ouo$R2izUD6wM#{J}f{p?f1~KLy0P%~E&8m!i|ZfKFbXNMhp_yjj0X(G~!V2je4J zH~M!OT;qDBV$sds{{}8BKs9CN%K2M+j3^vUzY+E@G|HOb6BzONiBT9F(ew6V@R}`s zU1-f$!W-`SZG5>UKL54zRyDPa3V#Kbk^j22r2eC%O!*B0nOge`kqnTn>DB;-jLcCQOlrvrod(7jwrA>&g2CS56f0<3v9z}4bwAfEY@P1O7Sx4_`VSkrroRpzWxt;ArQ5aNFdpSF1wNouunz=MJa~ip4?B?r zerXbB^2&vO#5%su6A_UVq;;@H5C@kc4e?KMqb=Ibbr6b6ol zAvyO1z+OWTnIr>B`_6)WR7Z@6B&eC-3DItyh;pk3T#?i1iRZ;k+ruV$bb@(_|ELJZ zXc!ldWCyjV?dfh9gx}RE3_9GX5ivkLcSi{9Tv2}n1InHlJ_`2$%s9gas?@zLYW2H1 z!Z{kFF$QJ7c50}T_Z_I!t|9*P2&k5$6+}3qcNZ(O`4s|`;k(P`Sbfxsw0Vn^@%7cM zxVuZ{WZzVaTy6@a<$0^;Y<={K+Bl1p`ML||e0U4zKzXa?z@G}FBd>#{BX535%iNG9 zZ#;EM6Rb80iKgt1Q?u>SP}2=JP*=tEGoaKfj|@IlOIMh>Li3j>cbq%3eU#0yRWkg9 zI+V{+wEAkFRkGsg&YzRB#hkm&l745t-QT7LzfO?mCQz~ZCZK4A^jqC3^as>lDD_1j zgBnaYfrv+d;;HtUeA=$|TAd2+ERKk_N3D~{r~D?N!l^Rvrx)z6AS`&}01dDs|$D}7QhO!8Ns!8L=<363yPD$5VCETSqp8KJl z5z9gWlttY4p+=opF=1$-8R9#6x=Q%d`p*c^aUgMksH6kjF%y85b3zj*v+!oe-<pPAi7O zfTkZnO30IaEo|${sA~-!IxLf(KiahA*%lTNqM!SfY-*Nq5t^xYzJEY>))Z{rJjk9* z_m8&Md%Df<@)>HG_as6U^JDgv&KWfiqiTF>J@YqklIXJ_qAn>$1~<8>*iNc>M;sxR z@Fg+`h$yJ+bcL*{25!XPl3-ejRaujnL-1k%=)=$qlx|=QBh6rAIZx{)Pz`#9*qQrL;V3cp>@un1iO)djTDLJ5W>}Tp zJ$}Uv3~Xrclu@ruY(iC4>&Uj(JBRGwUiN%v#wx5iIeVg!Aa?i+9R%rK%azhOfGc7^ zyIy{yY|DqSpseYlc^jqeB(2~XaeQ30)E_!$J%id8v*I+G8eIYcqmnaQAMX?prDk$0 zgdqls744PG;5!}S-GCj#KnCH;PBUl~uXIB)&NXh1$49#>%c~PI-4%>z@q&jAcSoxV z!leQxcDF78mq@Si@(1z+FMelV7jFrG4o!Y>Q1(VL*6DI#mo^|wAvb4v=;yQ7K;vjj zn^oeZeq!(G0x-Sd<-A5QhoyZBEny1tc;j}8l(;P!V;N~>h0pek{xPZ0LPbVKM`8L! z4xc%kw7^df2gSR)#aji}y`cd%Z*WjwU%E-qUvaQejc}vSB&qnlOfS$Yszw}jUI`WFZtOl2ReJ>Me7DMz z(dGx%p3DbOv{NPa_K>RTN$n(WDs-9A?vjK4;zgXUJ4lfVPbMmMG%fvzF#(RiK{b_& ztuXUDPa+qHBaTbMZM35hJq-;2e0}l?DzMME3djfMWRpT=ZssgLUp3N~a?zVsrs8&w za%gtVJi%I5On=tp!D%jRg$w<%kBYb2Zl}Oo&Nj^D@QMn$GTQloBcY#Tckx!uXP805 z0V1dI>k_nhRfR>L+r&7d$cd|gj&ZTfXpE}fprtybJ!}8d7=S8D={*wc48%X zxHn9o7kQhIbeIe={)WPY`Rg~wL<4(B6&|!{M~zP?21_}$cY3Ja3qcGMWGuTli7sg< zmKDIzh#I3^C>Q~an7=ZzT~P@qBT9wk4c;L{pHZfovf7g zs#d1M-^nFzj4%!^tmV;wj8Q(iP@0%=_mcf_Z+z*jruWw&`3ao1L0RswFf4y;E?J*fwQkqTFI* zh}eBuAkX2+8h91+U@Qul`;Bk+w?A1146PPuE|@<&Tn2z5(E5Zj>1XZvaaN1bKGBJ# z8Hpqg<^cqePBTtrDeF1+xkghjoa-yUD>%xZ(^a^$;wHjQMdFcj)WsU2i$x!hH#Myz z!5wxbLIj(}icw+#SU&?IHF%9N(CBf(8Hddse^921R>Y7354htz6BWf96@|BZI6|0B z20B*VY-Snh2W(Zpy=hSV*19c~_w!`SIe5yv+3?qR(jxxdjiV5HQ<<1c7ws83_!|On zbVQ5f7iqxwjz6s*s+@Gwu<8q!+NzM(e1)YQ)0a&^FNfF#u%A|9ZO4u2+kYe&L}{6K zL!)n*_d^>;xFHV~VENhKdr!SG6O5xf1jA~Ayp{%F!*POFFZj;yYE0LzC}Z|i4t`*P zk;JnpF_R{`jkThV&#RR(@Q|GYK|qHs(un4(C#BnnsiR;>wBMdTZE;6lw`nD_H?-fn zVIdHZ!C&$Yz&c9;i>zLU9Bd|Rp4)Hr7$?fT0K@bB=?Yijxj07ZBgKjzN$ z3fGsWy8^^jw6|&^Xp(KokMLx1A~`hSk+IZ_@v98XZGz6j=axA)4hQk$2?Gmk1?O?1 zxwG#_+VY38>BLgd#<>!Q)`4!l1v}MDTtaZ`V>7u1pmO65_4L+a>JvDk@dbGL>KXf! zHlO#)ig#NPmh-)x_F4BUQ8=kLv$b8WR7AbwIH&SNHEbiGUok4aZ)iV)>yvVC@UxYpmYVigo!q81w=G%F=$YkO zxv$ClVadVEugkthkg`8MkL8F*FKe*_^HyLQqOm9suxdr=*#dWpc`h0OZ&V^`4)k;A zQOF+t=)RaVFZQ%U-Yd_D&T&^}nV0)Jehlni_(He4ps?K_+jqn|@uQuF6!E(!skcVh zg~p08xWj&0Zdi)Duy$uLLbKTr+uYHD9NpRPwC#A)+666|vu;ghjl8pNe&UTzVHaXW zE4{U=Pcj6}&-8u!X&gXBXxLSk@0ULyH;528r+)YVNp_)*yb``&VPO(dFvv;-=T&^2 zzc2wY0S;KhGbymSX9(E>AfM{6ei}wkuoM(BDKHE<`XSSGFbcIW3iU9#7N7hfi%-O_ zPgX{lxM0T zN6k08L{a~RGYb+IOIuL>XN*v)d-Hvnx#u(6;T}inQFjy6@1sPFx;W3d@i+a9p?4qU zeyR<#$C#>uPa{WThqE)z7#hv0Z0zRaiFz>ylrvv;{o|i&#56B^r|&z|s4r@3wzh9M_tiDzwM z+dPze=bMfITiXxjOK%uA7w_MpI-YcNMkbkKFDsME^cI<8DK3Dsq=@5(Uydcyl_}*i zi;ATXw>8V!83DtUz7Um;H_3o%WJv7iRD8NS6^!!Jj4$KpB3A+GPGG9Nb_3uI(gi^_ zi$!N*dR0%jfhkjD2Q(ux(uzRcrF~)S)1!(d_BtI!(3`f<%`Er4A^aw~VR?Lncf-dF zvmducmzi0g%hxQYB%gacp>^zT_FvR?v;)AOe$aMM_(qIW zC6+ZSe0CMsW)+^7=lY_%kg^mu&!IWPZnYNbjZi0|ZH(%vGL41ty*r8ly_D9my>YQO z&b4`(sBQ_LuDpxRQ(>P;?M1`IJM8mycnU&cG65d>ZbG;9fVuBnN6(%`HhpEVmB|P! zf3;q9pM_LMe*d93En78Bi}pG_j6{WWFpzM=qLmB@UKbbpdPXP`7h^jmIM;k3ixl|qKW_CLRIQN zQiT=-3lQ`FG>raBy72Xmm|9^4f(#Hu`)C$S1SZL}8Q2N^f|pbTn^ngOX6~1T%qrsy zav<7VGR#TDA`l%&jhXnM;SL$BREtVn5Jlgh0ItTHxBTRu=)ii zTQ=jUF_`sDuZK^zyfqdYJ(jUz4>|$PHY1-Sn+gvr6ljktFUIhabD&xeM|Uy-3+%oR zX~=&6ZvV16Ap=~!e@HvC%7Hjw{!6)f8hu%)V^0{zu`7hRE!F{PCNH#bbGpebb=~CCvo74@IT4ige_Crr{DTXd4Y_@J zD=uBGIvX8Sh4#w0r`|QP6aV2uly-FY<}qS$dccY}{hT?-POCs-w`RXgiMZwrG+K#F zf#)HPR(aew%ddh%1>mkV+37Q1u-Bugb?zT;XZ}9YsT-VidoW|joK^umY`;^nk&jlW z$RdxLjlJLDSI&7GX8qP@gye|b_LzZz2cNbH_f0M}7iuDqAS<_e7+MCcjfglTE<__}`E4kbT$YGITxkjKYQ<_WcN59xjXHqPf`p(t1ZDR{W#$?1_fH3+B`E|S zZ`dn1{yKD*pa{v~3DylDI*bc@oe^g}UIBTNSKO9yZdoYkf*)GKF=z{d2auydhwoNz>;SS{z*&F`pmwRf#y1GV2Zd-XK>>$*nRPPa0q~uZ0|qv7p_Hp zwFZMbM90e6@PO@X?@(`ScxbJn&uw7>yZHTnIIp36AwFz)!=vncF^D#RXXxIF-SX#_ zFoWG9D-3h9RJ{MHa_C+zF|qWcZIQJ5r4)5>W!LRIDwK~@Qb-? z5`l_cy55pqb?zrTfy!OJ-pXAE=jUvaTa3%)YenWS4tte^Qw9g7lT?_pu1Q=E&+eYQ z0?x(Vwb?e&r(q4vHToNXCDEerV~emm2?3P%*@XuYOc9?}WoIUSFTtvyv#Qe57 ztcb{uajPi|Jx|#&SV>;Kt%#MCN*~ZoegXroDxa=_PcfyZ=Z6eNoQT$Nte7>59$qBUd5ABV!3TPX&9VZZUhn4g63*^lt+=y9M>YXKh>@G81{K!hD<`%s zO_!ubOla_CS9xJ~Irqg1^_imqwa`(wWEqfyR{cwy3EE%1s7Eh-40APd;qZ{-{f3Ci zgdHO2p>5AZtW^Nd8qt+ZlXq&V%oG?#qGMuw%XA|46Wa|=tIB2-mI@EI`W;@*GgmI2Dxm0VQ`nFtWyRcvN0nZ_N6Q>`cR0n^>-gCdjn7IJqMIXrj z+HiOzgqgmO#6Rk7JzvaMzz|B@EJ_&ws`A>vjAL%xoprAnBR;gLyPG}G!ecx9hk7tw z)DU*Obz^HBNv_Ra$&Xdm~0byI_nY;z*zT2j=K0Q1gk}kNesx@u(_t3 zQS@LsP1MpQNlMqIP^3YY;KEb<6lyHWpV-9E>t(beMrYCpopQb@?@r%5jjfH87*qzQ zDgX0oFEKKl85yGtKXf*zmsmis63kzgz6hJqx4de&d`=siO6OnJ{~4_KR;gbTEz!e2 z4wC~g-Hq~aH&Eu-A2UdRi3)r2wJg6hIF5B|T9RC|O3a#+?W0^3v}zYyUsoDiuY4k- z6y{$yg??w8R_8nf=f+OJVI8}KkBGQzr)*r| zG7(nSL$JTU`0v}LjfHX7UwNyX8w=+(>6Au1Lt}EmI{Ncy>?xJ`F`JZ7Z^Uy*6H_En zlI(P)!&;KUBOohku%l#;>Q)=E_?uD1<>quI5TaFi8nfu?4PH7yP*N7;kdTIRvJ3E7F;%WbUFJZ?xBkVLY%$BwP*6MFCy7jp4{)lfR97xXCjhz6P^!;4O$n6 zay$ds?Y=oL2ywEk8WgE7%>{(-J+lC@6E~Pk5$SOOG^a)xWU)WwOMaT}lM`Ek=`xpu zO?SgoNfKV|1^VaJlC~+hxuybEV#L+r)r$gxM<=x!-FP3N0Xwe&ivVU z8deJL5pra~X}1HGM+#mt);OjD6kL3P*+-8VJK6EZB&-@sTv4#4)X%9QeFp$2npY<+ zh_*+BoJ!LM#`FO6hy5OuhMEtI_Mn*zJ1MPbzzBwmCM9vcgI?G+sGA7t=H_wZoIaJ#7oOMi;CK z)uknh3Tg_fONJ(l#o8f9Bp-l094@yEarc7Btfp07rUAF7DIs6a&tug!1~!>}q_W0i zi+!e}rgBUr9~S-5WBK#+47((9e4femU-7d8#VKFR_R) z4ph9gf8-QuNt;KscIez2Wc-HXK#!9I=p1VQoDJsqG&bB)IcwjsM$Q8^x>^wLZc_!O z7|Vv<1?WyBd%%#J?Zm^u9(lCxJeqN~)X%4C#%acA+;4&4(xq_9&KRc3l|1s)(K3`i zkaZPuh=7AxOulxV#>Q0ZED|$f^wGZ+j!);g3}n{g=WHrj6B)7l?(ZFZTW7i zuAJMx7Ir;rc0KVqz-o4$6?;!a8K&wdW?Pwsw!DGFoTO8+{KZ5~V>RhV-Oc9bG~l~K zgM=aoqrWhL{%+Zn|35|ae=S=(ttSZY|8#4+y&!byh~D68!EB{9aN3#L?-|z`7 zz0(^67BKwyoSB)u#nuywqD@p|#wabb1_l@1)S_YutAY`2W2qX^A}y~{P(c+G_0}68 z;5SptP*ISeqO6-2^j0@>+Z_4b)hcs%WVG?mF}k=@DpFfMrY z7yd_a3LUH9##DLciz{os(GlyP--}a|Q}0l$`2e${Lvw+@Q^y%zp_v+4Ua6Tg>~~(~ z{w1099CsE|?^>*Y>8X6qJItwfaaO?ilmYwQSH=sc`Dbk=zyf1~Zpu8-4g;fkj2X6C z>z*Z=-Q1~3E5{h4JZo6Hnd=W714lHy`J6q*Q|2&3^RwomLFK!Rx^7XFth!Ss4THw; zHz}k;1zn?Z&syD9W!PL@w>Zj1{f1Lrw>;`x{l-K6T@;D>dson!u|6TFjl-@Ma=qnFbS3RfG0Q6Od`_yl`XttDwv#UI1(H*My@YR1;M?IG6;AQq?zv|IzwR3uN%GkHx5Y+3>c7}YU^ z>7D{rt@NPb9S03>jK#

    1_qNT4zs#Vq^%Hs(_k-#St+{LZ*}K_(9*3G66x+FDSwI zwn0-bjKO|)gIBF?g1KuNGB7p(6E7fOr!1=Jl>YqEHYD~*0>)!#PWkBUSYosQoi5yA z@n#rE0w#HcAsLaMV}Jl;q$hho+ZL#7XQWUZxY@d#A${NibPZ`kATSiHj`G?A^oz(5 zH;^9elEe`)kSQ=1jDYBxFR&fjS7=8R)Bxo<3djiNFA5C&K}rOb0d0!ffOH}PlriKA z6orbRWJ(N}1a-S`f%Vjk0AhpHkvOskW@a{LlI|qaDa}k%UN-^2)8-n?E-*lA z!3fB&u>$$Q9#K4VfahQW6xYz8)6f~vn$TQOT+lHsK<8z#9T!jo9iXg>=b9DRG?^nq zATP8nsUuav>w?r9LNdfsR$}cKMTxj z4v60r-XR8^LGnxnb_2i!zK0Te(mObyZ%EgbU_p^Y4n+VjpFa=FV=|Bb{o*_BV3elam7Xi9JqoH6)8Ok6` zm3dh#{BE|ou+4Jjkmt@GnJ#O*dyc@<1m;Z#7O#GQR(S^Ut*3E9ru+Aa%q6<-snA8j^v6(0mex@goNN4mR(spK&9C zl-K-%p+Fu$>=&?py#&i1tep z)$5&J-gS^X;_}@YRKt=aA5a+yI&dzG&;Jsh)$m<*x85b>?|0j+iWvJOQ9qtJF(&|tK!aasr1Hl?USs4TID?@4|-dKW?sNr|rY|ODhSN~QD^K0>}4C$WIAyevl z(g3vyDLF}{E4n>H4u|<6ysBT0@f1~1W(?}bPK+REWh>p1`z4kobYV7Z5?(yaJ`o%M{EmGvBh_Q9&MJTn(EgvnPE$yoW1_A`Z| z_vp%I57)n&qL;k%Gwyq-Coun=ZUEa2X{!HTzU7`pQa@u0f8TtRKWX5{bpGNH-~~SG zIoR}-GSJU*59YcLwZlLaYrB>@2T(!{>LIMG#v2ccvCMC8`AVuRUx=D6FG}7k^wO}f8_=-4IlBXm z_Iln(Ela3CrBAr*gXtX{nwPq@o=|RYYGm{P?MEu;Gb)9w?g@*hqeDcm6WIdIOU~*6DbXjq z<}}tIyr{Z|IIOayxN15I0Pr3>yy=DxVn-AqAEHc~&o*0%4M$$4UM4}bJ7`Ito@pzh z?#de{A0IBr|*-)$_B3as9DXP;Nu8A9B_X%u5@ks7=?<^nc1~iqHEE8U*q}1CA z3y1MlLtE+MlxS<^URm8j_Hfxv!c}Q>`CXb3?DglFXIPYuV$x8PV~OVFL1=08Ch;gX zaO~I(XvvrlIZP*rZxwh{w)f*&3zMj_Qm`y69Na8<`L@!t7RO2fQ<(#0){>Z3JHPt3 zx{|B{6WX`*aB1rQ0^-$pi)JKAq4$gr8(dXl;TN=$$4pqnzO@arY{8%%bAd`l@5kq5 z>;~?;tJH6VI;YT`=NrTxre>@k)xM?!b2ozvap`vn`oCVz`Q8%UQH}7D5uJ$TI~iCE4F<8iVyJGqn&7J%%t;wq(g0OTvxdnyaD*PzoDy??g; zfwyl#D@OVr`tqM}r6tzJC@2=p;SZDvjq`ax@kiCG)R!G`u#6aTZ8gDd{d~)ZL$;JA zkG{9CWpQnbCes}#D6kxwtJFO)TKr6>^Y`U}k@jX2g=pGA`iT75T9g=vJc@=M+Y-+e zHlVM}Egk{t3n+mBk^VOaeosYn6`^oWpo6v|VA9E}6ng{H`6L(Ru?I54GIN%cF$EK) zq9ocDR%)#{Tz>Z>f@gmr`}}-iCj5<^S==WKkF3FZjJotB2b)+z3qOZ|pMd~_@P{mB zhY?nUK_1E%5ojh{VCKBc{GU_!%XM`4Eh74YGsDOf6#$6@GP??su8}ZJQ1pBYBQauL z5r64TTG_~yoM$3bkgK_w!m*GOVP)r_hj=6M#@TX|Pns;|_Fr%+E@7Uw=xA$eEt`OLZpIqH*dOgQh(f1ovgR zduQpRNC1}I7W)^kzDmZ5f%4DwdpF2GdR?w?o4!Bo!|tEdBJjLbY@lwKzN6Wx)3Sbz zN({vtH1O>h9O|#{NaIV`iJ!hm{cR6HIa~Crl&>`HTdh2LEX7Ssti9ljDG_+Y%iIlV z>f8DofGs4Q>07c5Wy2@W6S^<~F~XVTps-uTj07kZI;la9@-9QkU!c**Kx z^BRF{3g#oFA5ZCaHn0K$rkUiG*rUls za2C>6F)W($JpXFBETgCuz3vdwbiKRtTpP&JXHVv^W8?HFW;0H)W0|MtPAT$|kP{ar zG57J(Za_$zhrAMh3rui*2;Fy0%^`AFF96su--oRbT&YektWIhW-tl&lxb8TKGEI`g zdzk4--|mjT?k-#bPCtep*1w)~3FIjFv(wrt2Ip1qLlE{}FE@(FzrHr);pn40Fe9H9 z8^Iz#)oFgC*^O7c!DH`7@%Y)7)*89LADa>#RG)3tiQwsW7c~rT<4O6Z{wd(|D+l=g z**ekRrH12xBcLGVW0K0pXL`RKx1EWzdb`A7XkOCFN)L-a?Dw_jtGm$O~)$}zmW zSpxeS)qz5N28W4)xYt9_Z51h!WM<%ig_06SZILn0Bc7m%z53eCT^YTm5>FotHO^nw zB(dQFo6wcxN^5)FGibVkH#oeEBYuEyyeo09O?diV+lTHY;pfV>lp^2r%6KskAe5PH z!J|AE`V20L7q#~k-OWOO%KfPIVl~*Cn)ho zM=1(Nz+v}3>J}T0Zpo6QmB2N;A`cE<<|!pi;Fai|OwgPt$YjozqRYgfz5@Uc#-jQv z63)VDtG}*d>Dz`%DzW_!%0myoDcVt6^yhl8K2?4FZFp5yWhrIr^I6<=2ucgjSOMY# zRx|h{=;w%xqG4va!X;_2{&2uFT&Y-2!~YW~$z_`RJ} z-rtxWh*mle)Yu}CgdFDJi&8*h3VOTP*A7&;ruAm8%4weCY|$966Pl!{!iP=Cfcm6U zy`$m>?U=*T+t1L#Ljq^#NweWt5j_P1rWZbjmrW%;il3@>!K43J^FSaz4p;?(>7NP% zEW|n{>K-_YI3MV)QR_9+qUI^*kR|(C=KGS*{x-H(dRYax|GE<3Y2*b|VOM~TjH*NT z=7bfCc;k+RqJ$QNznkDKj|~u2c_m+t*zhRkT4T>GN4Pr%_DuzMlG?NU618Xi=h41? zBm~@trN8Fzrs;w2Xvd;E8AM*(>Efx$IOCJe9tS(;d$>G;*zmH+-9}|Qz}xhY&AkbX z3`c|T^5C2`R3zFN{lfzAiSZK|dY1a7qzLDAmek55tLP5Ze;#esE$3G%%B1R(8e`QI zQ3$qDx>P(w;?dR-U`7zak2Hct>P#4%e80l$ihfHk+qEPTvK0x%rsr4&2`(vnUgk0V z)M3I?erh-G-;gD{Wa%7BKvPE`EgdVIh|V%LXnk+WrD`7no?QfRxi6MkpOL@6VMS3* zv>%rj)}x8EQ@@ljUbl7~iUjRRpTklmj@}a4t}V1T6t`%FO?4F~_VqJOk#`WIx%nJR zQ;L@<%6hC*6fwBP@us*R5#f0q3nAE_*P_hiqQY1tjTw$eATuu#S;84$yu(eRA}_7G z2Xo`3*4YzijBWu)C8vrk3>&wcI7f*lT9?v>&!HH>=)jWHDicc9;lhWkOW$Mk1WXdU z48qigRA_VrAMEu!=vyT&?nw@g_luBc_4e{+DYNqRivK#inXeGTpbp0_E2dR5Mq7DN zR7i3CLp01HGjAB#aHE$dV4IiCr}WOW7G)E~I$0p#n3o42KHE867QNekLw4RlQBNTE&PLuB{3m{pKZ# zPgmnIxwLQha*<+llyzImys7| ztf$~#ezajI4}FIRYdEX!Ffx%UuxOzjIcp)jOtPR>F? z%{WQ=gR0;@F?7~06um>JDUtFQ?LCyb;jP(QaFBMyr@81;Cp>U;r|OcgW<%<#3~b_Oknr$}B{T1ejI4S!hiNHJ{d z>luJ_gpXg}<>(bhNw%q{KQnPkdeB)nOH7Zad&sb%rIES>F(Qf6Cm203X99OoesNE{ z+ZmF|js$F9*45fzZO=Mop*VBcZ=UMT@B?d`f0#P2zd+N58=}kXGR!K_>`&}T@_qt@ zt56j}VPDMPm~3xSKOD_6Ti+U{NU}m~Jt1I)iB|eU*@(m*)?UiQl>U~-NSkQKelDKM zOS-l{;yX)?iV26T7gJ;u#U>-PO#9USY?cB)i+%YQL8|}Eb2RNX%r_8NY=S_VO0ZMX zKqP&Lmq~3tq&g*}t7L)wbg(vr8&yD8evXyVc|4_cV1Z;jQ(EwWO*|UyVLKJ=Bnyyz za!GArd(F6ji&5;Be%i->8W%h61cmQc%-cXxwQX$@!dM(KJU6dT? zzbZlf!dE?eC1wnz-JgEiVfi3=c)^u@ zw&DswgqwP|Gv+$QY@X)&Q6havi*IQ4yInJ4V8vYCx3gN^(+}GnPpyiK5^jJ~2N%h0 z6aLGuP3dcvHi5twELiO$DeEYeFGhxZw}THBMSS&Ohl8+iB%TDWWv&duT>vr-NXqBR zKK#G0T8*+U7h8^R)nm#^je5zCJ$M&;>ldNnF!faxBGLR1A!6x`rSw?F*UBC@8$%!V z!pr1!TwDpNAp04Y2}HdZHk!xTv4|(0+=DZVHyB|;mB!2=M%cly)@`r5ZzuQU!mJo8 zwLAeL1#8iGymb6J8n>ZV3_yigubEct%uw|wz80Nu)Uq&7`rd2MAK74uTR=_NJIi5e=|n)f zeaI9sfUN!3b>w_A2WR2eE!ozQ+Z^tA&Pgn&LJ{W)x@z-{h+%1WVP=M^(~azhu&v6k z&YuEU@U4zi1%u$U_ho&CKYrwVdI$J^4hbeI?huYyD1H|D`v2KZpTYRd^F6w0Ke*HX ztn&?fDZhO|28asWy#Ug0Uutg_pR;eRSo1v26kl6gYpxRRF+j@&8-K)n!P+PW1$}|k zG`9D@JnAv^l)=|+gHZlmLs&J$Q}LiB?kNsOlcS~A9sKST{oAp0|Z z`Owy4^!(iUGQiE4AMepNSZPV4HmivnbMasL>yR>;s&8MC7O#jpGQ z&#{ZXJ~wtRL5zqqPKXJiIipy?F8rHJNY10`cjKhZTfMKe&vF&&+}Wbs!e)y&_+ivB zvCb-D%QZ2r-b|Rn(SejIVVg)rA=a2#$%*LWc2Erm%{_QsMC(Ion`hW)COhn5#Ut9| zF6DG(S(97HWV`4J{w?%lIYaH3*Rig+3eBw+IY$ zE5kM%jZ1<(w2{Q`5B5MPuxVYtS$NP8XSVyKw5cA zKP5nOqS>uk%WgA~_+lF?>d1>K{W%j5ugB8{r9=cn})l&&7n<%@X(His=~ zxUu?!Uu?ld>iu~^CU+Y{k|Zm?Bv!XH3MF6$zm`8J>DTPjkR8V>nvFj%t}H{py$444 zZYsyKA>}(g9AWzaS|=QNnYpu82}!hn!WwOxY4p)-(t$YjlF@GAD+VZd z#i$!dVJ<}sFs(7~AUp;91#3-s94<~7S}=?BhUP!Lsw;8XZgpNPZeKd*;&C0k^AZZ&7UTCl|vXIk3e$d>LSv#*3X$GP}kVA+k*K22peOEFCBe z;oZoWVbwd#9XR^pk$r)0x3s#G5I{FuqW56=VBA%@jzskVd0XU-0i0cqHVk`<@PFU-tJf6@$>O@ z(LAQU^6OwsYy_Wde?tArNU;H|^yV9` zaHtfBtVSWYllAOH$M^~4`#Uh*_PgDN#0aTkg90*sGmG;zJpvZ6Gda zDx8f)@BhI88pg~UW2TTXAX5NHi4E$6LzVFLUeg?ZCg<7zjmZ*Mn5nlF=#*P)!VIL6 zP}yUNywL*J;xK!Hzwl(CSX``OHd?M9DOK5O6;h(x=!ukXtMs;EyNP1heB60wSLXgPMf$t@<9>Kg zZev$vV^?c7J#|ot16QJvyJHaD5e!G2JR34vll^(FW7;Bmm&Bb7XPep5bF?4hD9p~5 zns4H4$L%Q4V-4mxg&nwJ86wC`h&naX)s(!7gJEdVxOU--Awr)C2%wtD>%}=_kl<;5ZcmF{POV=~sAfdDy)@K&zCi-hG zVP;K)mfcC@Bgd;F0*!uorFvj4jWqv<$Ld-Kxd&U$+k`}s5jEHn$|`hHDMJhr|J^(T zb+QZMjQd!9nE=fLaG*q+>bg-~yNQIR3czTkxsleQm6MmG44yNV6<(&0+b}clPE&Il zD$vhR)HfXffd6FfBIMTem<}BGZv&#ih23{u6^SH$5U=1)(~T zcJjDv@(b~TMXhhLkhUAtyOGw2sT*dbq1p&^kS@?p;-MP?P(0z*0P*ocdLi)~+&UoO zPJ7;OZxOtTKQ-Rg0MnLgXZTbUr58zqVUF3%;ZFZIvge=XHz4eu^AIjI+q6P=<~y6P z?ru6OifN+}x(S~u<8S-ANtZ?GAh{2e=+@xa2B{K|KwP95K~9QTgf4RyRq8~V>>drp z8zR<_uy2|OaKHlfKtavzY9J?`RL;=5T3G>`3%3@XFsKiT>w^uy|GjlDe9UoCtm%|w z(thh<=@`Mh{LZA?1-&u0#hYyi-KM5}m&+m@DIQ*{p!;IB>~GWo_Ec?+DOVf$Xe5fO ztFx~%3#%wB!E~dxzNQ9&2rO}T$I?1qt&Lk%2^Z!7U>Jzv4Lg`qqWye1*; zNIeqM84(r}zM!Q8@cL#9-+a*&F$Ch*z|Bqu?Q1HVl8;@(wAyp8snTW=6smJ$3loIz z6cfBKQz=*ufafgef8`QEYTve~CF~KIJ@M-AunrppD>UB`IQM-vA_x<@35`(RpeI&m zl&1XwAg~`=62Z;JCMM4KK18GT7jn1y@0e(hke0TD3={n`9AR}pfv)TfB*M=qs&{Vp zfxA>F{i*_n@XL&S}c-W>2JyxnnnwRF6>m4H$k;kCVYWBP_-ryjLLMh9&b4P0icV^Pq^N~5tg#GOuEhMcFrwoU6f()jXT!14 zjfwH_0Wbl$vYTC|WK3a!4{YJ!N@yyHB#C?|Ms{@{qIRrGLSgAP=`|8#g?peys1njy_dj(LzgH3L&&IU{@AqPP6(M-9JWYvB`M z{ZfVbrV!Zc5e(g&JK8QDjyZ&wo|`}1=FN9NqA0n;Fcrk$%W=QwVhB|q!2BBpQx9DA zg^z0RQKt6?#$BjVv?8M*jE6b41IIIAIIrmXB5ZTPinTS(eDtq$AhhA0J=^u+J65m= z5NZE@Y%2-I4vy6(ytdXAN9XS(_S!mtMzS$(t?P)N zR}SYcj7$+NR_vV#A4K+UuJ(L(Cfr6&&__~*wSQogbd^`DJ!)1EsZB_+w+&75>d!Wx z0`6a<&lq8_9Ks&73-}pYJJ+?-FyXrgw-?CX8L@V52gJ+6@7`Ev4HyL$3#tZy_2YMA z--(u|TkFgle}Y5Oi4WH?(Myee_jVxyL2Q_3>e=l4C=spbzrHwAyXTJ8D}yAxAnmVm zeBim@#4R7KzN9(9Gttjp9p7Ry4wFNN{jTw0y<^u7xs>z5(GQ58GkwR)AZYIm`u>jd z0;0d{?sTo%B#n<7kkRAY`C-0NxM8Y4fa zDv9U3O~HijFs!CsS(CBQHItn2ypK(m{smuyRZdYTOSjJn)b%H=8F5*_;p415ZtpLB zckjq?08s!CvU(dU7g44yA-o?CjosSy$s4AQY(pp(zdQo>>8xr*^I8CA%g3p=Cz2j% zaUAApkMwVm4w?3}iT&=j`q9!H{5sd2^pQ84W6z=|^n_s|pI9+RKG#@Xn5{2P8Ke0j z3P#!dsERt0i@Jjcj>bM`Vv3A!6EK6 zl+P81pxv4@a1p+P=tBVNNiXfqw_IPm9D4xD-74xnZy(nC)KE@NQIB#ueo`0etZujM zWQ9?MqngS}whMGV z5;m@)Y;xwf-Nv*B=P$aNk5zJD4$EzVPO(VWb1jV?Qc;YPWa zbF9~_9Mx`0sj=7Z5*G)f#+WA0E?WzByb`&|4u8J&Ck0wx`;E5lrXkN(p;BVjd*mvC z0kgO~2EzvWNiKkn+Wengg<_I;oeZ=qACusyYyZ88qy}?ixF^CjjaYH;N;1q=FvYI6 zn%rx#SXv!@CWjy+m6M+bhjzR=Q$^3*Ck4ti%I}i4WhOAnV?lKTG1NieuMv1yj9vX|iHOZ&h!yDkyJB;&K}DQ7bgw{(IZS(`j~#|N!u8=F z9R_eA8be5)l5nB<*PV)7nObP9)mY;YQ7A`f#oy5lv<-H>Tp{!r`)f|dB-NuO=|N74&{k0B(n=q=4d|>nzj0(sO%x#E%mAWUruC%j7N9W#vp>a zvy^FawQ4lwwzW< zp7*wue51x8Zii4(7U!50@nDzsxDF01{41xSNmT)o3!xedrXGmm&Aj&wra6|DJ_YPW zAVQDr$bzRlR8H22oXv;ELOJb+(kJnQ|8h&6%~siMVP* zl_Xmj(%@}b56xtg(fa0RWu%hlL7H1#4l;5W400X99fj$dFu`KdoIE~loTzjqqudw|3AutmhW{kereG(B~ z3_UGg-l#ymtUUQOXdUJ>Zu}Mptd9HV*{&;S3l#h{&5hcVH=emEv=O{Q;uw&TwOAQ#yS|c=TDWF4{M(-Nt~foY zRYH~AdyEtZH$oRbLKlV>?25UU+(7MEOn7_U$yIWq?6JX6G;(kjb3ZAvX6LbzDwg@D`IEdg<~- z&-)?N?17gkVU#j07ktM=i0(h`b3x!k{|*G(8tQl0@7pgP;2+s>&rv9-R{+ZQUIbF{ z*Vr#h1KXL*F8q51+tK2ek zK5RWDcx$y~F7V4;+EpY+6Ra&%+-u?bNiUR6ir|7p%(%ExNwC7rVlbIhorw1B{oDIT zi4H`ubiAv6DwYAr7+vx97!N&42ftXc0VI14hln`O0-{U|k%Z=OUuFPx5~U|0WPTLZ zpdcy7>Q-KHT;KVws|7o0n`}69GnfOKaM4~C_%PZ9qZF2(Bebl$N~Lhe1R2hQitvN+ z<2h-hj|Hx)g5)NIk*li3mlMPkd{_XzkT)|i5=|w;m$JX?7>)BWTc1mYS)?Q!QgEDO1O1`elE~36 zf)3el^&D|ocLSsCNwyTb=G8ze7Kt~W&B<=?x2+Sf2rT@{lOCA!4l+`f43)JVXXD?n zOUMv;z#(&^f1-3h>XR!DwP{{i&BJC;F2A_BBR)efXPFCz*MXPTW$QNPPJebeuzG9e{n1hW3tVz}|L4*bwfY zQQfMG$H&!@y7vR{rji`$Du@1b*f9^ME9Tl=cQ~odBxcHFJj6JWY=cl4okT%y70U3+ zsa=K-ra~6Tt--sbh8s5CGU3(pf#!UMi$kQZU|s#dC`Qv(enuNM)^KONcn#frGV)X* z(9hvO^%FE_Jta&SZ`WBb1F%bfY9z7Pp@WpMtL7x=zK?kRl z&kpj#hf~b})eg0&)boP5C+?UK-!Q2e_X%ngt=wBa-w)_Nf6Q@;*}Za!zPf*W@cH$k zh5^(gsQvI=4{bf}@ktrI8ccqsdn7V@caeUgXnnx6KH#>DJ0xyiF|GQ>(*a<`Clt9f zkhe3!*y#b%*>ZSulP{6f^YyXg6bNafcc(tGW$dZ;# z&!DR+F-^0*2ZdA#8?s*qEL8=uY;b<;%ZIuqV*Ck(kil@0U%uoun(S$?)*&foLboy4 zNxlPo3H~P=-m9NGNumAmC}JcdLs$l`o11jY`YK)IzGT~0dlE&Fgg`_yzINP* zR%V~W=#QDxwW~arb)V_@e1pE9nj7j+V!xP ztu_VzU#k3dO=)J{RUsML#|&!~*4(D^1BZHBaueECS6NB3-XsNn`Y%3_RN0uZtg&vn6W^So79j{z(6Y9O!es0y$pb|j_us- zC=JxJ{zUNlhAF4YpYDF?;44hTC)PnL?lAR~#2vcG<7_uI+C~Rulnm~l4g#;Ge+A1^ zsd8)NRG8kZZT?C!-=6%6sUm%%w6|zq`(m_{^FR&`dN&Ijq`wY!GOt^<#PO_S=DxJl zOR(#WM_d1IfXww26u8IFQ5G~G=ZBJj?+%`SzqRZSn8G2o`;hHpHV%RUvb_7V<}A{~ z4LE~OY21d8+u?oueA8(6OQXo&%z~acViJtF-w~^9+jUPD+B5SwN%BHPZuAM(_pVYm zTw3b9qX`__5;8iLq{Z}Yg_fO`xYG2PHl!b!Sl;mS0P4Oix}uEYbuUw@6)LkS-&fv{%-oiEiv*Pmo4>qurwRwMPSHei7wknSiy z;0A55ScIS#ga<$?(Mf0ri-cw&8w_?AMu47Y91Q=-@*ad4KvRt2$AU(z9-`K-oH@l4 zoG?o|7C;m%yjUQbKpF%GNlZ#3<$xsI2)ox1q!k0pQoF;aK-CdI4UTzJ2ntU~n=#l8 zgThrSG>Q$5cSW5s8Up$=8HT|~P8ensJ;PI{AN_YRQ8b_w;UfRc}Vvm}M(mxuHAu^n~%|^oW>}*fKi?Mr+ zSHm*%W+4HaSc;wZbfU5)AG)ndPm&x?ic|ErA*`#PY_dPpl*x7iY2J@Raw7QrijM!x zRb=LICSl9EP-zcDsr6T(G)PHtHC1*Lo70GwBNNS}D3=I|w`uob(r-?(t!p8V^p16Y z)du3Oml*NdFISza?=pjh#kB0gu|7-4=d!XX)|djytvIWcljyo-Gnm-qaI6G!wX*h? zrR^Ao!OIWaWOOJ>3e&qKf9mv!LmIpzj5nK)$Xq*#4h>YmW>xDK2b7(YR-%cL+D&+? zj@c&KCy{UH##)~?(nhLDm?|sSzFt#~&6;1I@MzskV$+KoY>`p^)f1Dvkkw^0GZS$* z|Cb3kKMrr@4#Ov#NiUk>UE_q2eu)fmZoT*=Ba>N>oMd_tC5kd&TNXd@ZhvIYQKL0` z5$Zcb6~;f_ISQS#en!1jBc-jUb)M?ieG4Sd zbK}e-=c#ZQXR$n88z4mb=yvES$9`>Fo`VDEF)q)*Z7#vl*=JXx6VUg~+2NYqGFVWF z3q;dMGM4?xRe}H8^-D^tD8T&vZna*BiqU);j@V z6c0cgoFxq7WaXucofS;E=? zC0-sAc4le<*bPZh&52z%M1tU9lU7BhTckkGjz^6P&j3e0if|jtY|(1S@lp`%0caVHFc$I{rpF zAR^D2>q^(EEW|B&pbT6%`9Qkqy>z#|F}=xK3x zdk^g5z4p0svQ$56xk*BVV;;QfS^Q_z9+E|b)v%nNnQm&B!8w(LvTlu8UAx$}Tm^1} zsS~Si88{Ha4As`A)cUpuq5aQzbGE9~|6^Uzd)kLFQ!Zm5;KRiKY0G7X`zgm|rt_)S z>%Au&0VE^TBlj6l7|}!WNwg=8tQhI3J;X|qhwdad8U?NDE_WBWFYpgJ|s)> zNwH^!Tp!`7K4e3p??Ftc!BXQ?2#ODyyK)x!ulX&SS3X~Ga$ zU0H1np;*PRP~h?%P2f0uh>HFfS?}0hSsN_scG9t(%-FV#PRHulwr$LC#^~6#ZL?z= z9oy+xJMTJQ_F4PO7(ZYe?3;+95O;qnmVxgb>Ne z9)*t_-=sSffV2a&gAJW!+Ce4!fN7C=VzEoSMz=}0R^Fi8fw_!#z|c>=X7kbOH*VGJ zM}Gnzz@~W14DIP;tA_}vj(jhe~d%_d{N={waY_)3=%F3u&buAEaETcTj|8k{gfL1*)e>IjV2fC z_PP*udNu3QwGeg@9_k zoapcKuVDe=BDkfFR&bo|EZpm*FuWN$HOcHFCYQWs-l(cY*G)Lvn<10JwC!NI?z-(NWB&vwMcC| z{LD2sxgS>j6c@eGZKEa3iYkdT9j$qE=DtQ!HLqhFXPxi@-aiwPA8FNIjg-0CoR+ht z^WF-N-s@;&WDKLq@0Yjnb{x(|oiD{_YE+?(#tVtt&{ix*SME@>6V5Hv8vc2qKgbTD zG;jpo^AsB$%JrG06-j8)FMt4ehJ1>`We+Dn%`kx7O!s^K7Iva^1XX;S3?aH z4zMkB>70?VfwyBd{6lv3PUYrzvdBQU3M0SkXyQJ=Fx;z^6CDzpr#SqXQCy73Xs`x! zV1>aEHFjnlklS@Zo}Ljd1k>**7H(eSQjNt=x#h+wD&$yXXPK4%sn-&=Rcbx%eX0Z~Vbxr6JhKMn8G?J37swO6G_|hn_2QhAIa%5m zRY#)2e#%-{xLSG-P;Rb8#n4^}Tbpq1Fg=m-{#gDlFfw(8R8TIsoWjtx?bH!Gg&#=o z9_dfRlPUABcIy2f8uM#rEU%aP(UVHCGiknZ%nE6^1~M{QWAp(_gIguK{X-`3`L6vq zs($D=w{R`|EPA$7dgM1-b;hbJ^bbR5kZ!z=t3vzHnFgW##Qg9zxT(}$w78oQmpfW7V z`*d5WjTVfyAL8B!pxu!A`}leTL{zSrk^Y(m8@#vDMQgx>-q}DBlg+O-BcddCB)N|- z-Hee6s&z%^pH-wqFqpgE-zP6d$US979bPVW&?NLF^5(@jQv8#w88M%lkVV6OWL$Ng zC>B{$FX~7zlqhog2T3#W!nH|i0ePErXh^4iJl3I%~COBITCR|6US<2k}rkMP9Tw4|3|i>JRYV>GQqLCxXBErgcqR zO&sI85@NR&&h1u!?Ck*mFlY68ZA%{R{SDgDzx!U@`W@ zVq@aCk{{0eW1w_H4$3zUC>VwyYZ(5Gw7530?bZ!hTT}WK8efd0N;P7H8A@S>Hg>mS ziMGQ6bL(g_$U(8Ur0Lo!Sw3QZjF(P2+Ep!%|73K@%i2k)i$f~oN1t9Rfl`%?%+4)8 z8R)$PX(|3YLvlEsM4HBTuAO-YC7ezVEH(#(lwEGd0cVYnBN`MYpn-tiAV=Z~#amCf z>5GXKq}67p1WhGM+|miNncfeU4-svXTOqL>7p0D>fdzKvB1YgPLtZ;bpr=o9C(btXvX5m|ZHIU&a{dkrcz`BW`Flvqx=`aHu-&Uj zgGLE130zExKSG<4KajTV7h_zck_BdIyp@thk^MwF^Dwex%f-}e?dQnBnCor@o=4j~i!$({z z_6{)bCB-T$Kq>YfB2Mj~fz%3za}^zN1vi4?ZD}03KNw@Q6I^Rz+?I`I{uvhVBpAYI z5x;0L(CuN%gIQ&<#uS-U;$$hXV2x*GS()JX9Dk$=-=g0T<-wH%JM}TjVo3*=G1Czv zBY$B3mzUYiyF^KUNwJu){#SVnCYoZn_dm@X{FKL1WOU$=inao}2*yWpvYU1?_yMVE z04;HmPI4qR#?ONoa5NIi2x=L=1(GwnhE2}CpXl%Kg0JWB$mtrA>^F*Nz{Hygx3(m! z8EpM-=ZA^MNgp3iD7|YgOLB=j+L1M?Z5=!>~`A&6?IgZgGeA7NIN2} z;VyULAtdAS(Z=#wV7I8bsB0o!r);fwi=#2 zhX;>*_0bSI$Fz46RV{fP?GUtHupqkULV6$P{a_Nqx2u1pJVjkabx8l zfzBwhADKRcp#1~!09kfEn=3&x3pOnnuAG6zQB<62@ou8N2mr9V!=G=-Tt>e?-Ayjx zqTL)o1Igd~eC;W+6-t}2c@gMtJYzS(TulVBK@*y;)8k9BzlN>MNx2~|bQ%!!sQT7yn!H}~$Z zUFQfW#Vca9KrKz#rZ8Ul&t3AS^3-%WveT75ZGs|DL>W^!ZvbP;+Nly4T2izu=RZULA@G`UvTr&S*` zO(YHTwhTz0Tx$3+AmG=iqTVk@a0pqXErCp1+D`s)$*m)|uHIw_r0f=ZvfY+Xj`e+_ zs}oGa*02pGzfqe1QGLyDy-j|NA>i$HS9~){E$1PudE`YT+yK_&GGdxNJ{ohg1E5?d zuOK;lcEZe9)*lI(W#N@;8K&fiJ%#4$6>I;OuK(qy5^fQAOD1{|Jq`~BU0Tc#)L5*W{YwexMIbXZTI zQgbD&!uZi(HG!~E?9Q3j}VLG=2urEGG8;K^U&$RrucWO+?z5=<{Kphn8;Z~5l`hU;jV3;Z$ngp@Xc zMA;Ho^7U(`=9_6oiA5~z`x}sey3EzMu1$Uulb*04F)32>GXt#Qgh#-ND`~3n>P)d9 z(P3o0Qv!}CY~8=!c{V`)9qhmJl!w=WN%YmrO#eDl|Erhz1?Uh3S{q>eXP~YhCC^|X z5BT|o|Az^-k%ahZi^2jyKWjn;3rW`ODhXn4Uq$5W`2EYaqDio}c}arKqN)bBqIBjI zgp*?%vb40crd6@1R@H=D(R9^v!^hXKYSKUhc9wIs<$c+8n!nBmrLe#03^1{GR_u6iftJ}j^L8F_w-&CGK|=clIOecE z8rx*xNANnt*HTPQGrMApOCG)P=I&&5LMtH;SS*z;h)?5Oq6?~ zkbg#a1K1#&;dC^<5j6LXKnN=O4_qq-lnU4MjzBUp3g?|OB-)15SD|2^w8E>yZxA~) z^fG{dA@?|Z6i1eCAOi?^FH>BR!{`}jV!d6@F>z2lP?k?Ta8k}ZAabxb=)ixe;WOUCXqmynoq!O}u`=7rcH^ z)-G>9w|O>7_e$ox^|m2(^7hnQl}2iACqtuJpqLbYvi$OS7C4LT? z7_lk?IbUv^eJ48nU6s|+GEgaiC4UEoDndez9k6L^QdNwvDW%Chr)lhhcabrPTT#+d zEP&5q-QG`7M?6BnAs4PE3XRQ_zP*dpFj6nYTxYq(ok3bmfb@{cfhf=w;1D{R6gg6T z4n!K}t%{o@$dePP5ExtM%c3MRaY=Vl)^gDWpH`u+O6?&@fIL!*;&2y7Rv|FRD{vIE znJ*q(ElNvH7=Z=shxYe&^?n+l{Au+8f6!YrrU*PfHZdtNJGjo;EsJQJ`bmri`{-Sd4xy42QCEpv8G)y2tKGr3AN-I?ScfJBsN$LFZQ}) z>SFjWJ_{E{6UVnAIdN<6zp&kZ88FkDVqhm-_w)>Y+??Pyk%%IV6D$7_n1$1vO~By4 zR~bdtPQSHUEnqI2g-Z+2SQcliUs}Oo7mNRxO=s(i4Q=7Tk8I2+(a1M*8mcmZy!>D0q#FF0nSJc~f({vN&nQ-E*ARj!9N(P`@CV z94TK;Dc%IBsF9t!-rU;uAly|5oP^{rxRd;h% z&nu*>kbSv6Yfpy3n?cu7Td~4?znOphUC6k$u%8VD9!ddOn`EkLZOzV&4@4&oR4w=^ zFM0RKbO!`OOr|lMVWzpU8rYh^<`iF|teM`aPsI?$)*kjlofM#QWh+_b&LEn@sbIz)Ej=SEJ z^f{UWtOoZ+V>A@ zx27-Ny3!{{-f(T~Y(1E4afaGHU&~nwI63A>Ft+3pH9$LPaW0i~>i%&6hra+{T4mET z!`{OiXvPDc%+5Wo7JTk>eab*x%@l}g^z-(}*U>6#Y83FB+)o;+O`04So>5iQfA>~l z%;K&5*<8+_Smq zZ`2gh4Lw_@0%A&T2fk2k&`$O4sxN!oc~zFc{G2!Ui-hQCrgUn2@g`KG6z|Gq!nIcODmi+58b{&L$*(S9;0v^}mqRa;!WZ z2Js?ZEY))+r4crPt?A{R%+63U7HMpSbXI?5%mi!UyxFUE>#o1|$xGyPM^7oJ7uPjw0Y0bCxj%q^XYPDO-3xu%V0b&I;tyGe*Q81n$Lln* zWWO)6NLA(vot;c8*Gg4^isZ$9Wr+`T#qjCAsM<9%p*gwCu>!psJxui=nWZ?KkM<_` zrlzdMOwGfsZ)raE7u`xOZ1Y64+a6hT&MvD;GMP~B;=R#9c!qsuA)$SV?(Nk73h974 z3L)mdc0^Y$zkqGLo+p6CV+%_R26h|eBd7$p(ZZl zCLoGuq8zqj7xnHiw;6rd-r)NmIz``j?Ax!%d~qy<*-HgTv;~bWw+NJ7xzWp=@M$p6 zy<0wO+lnMdTnnuDz$;Fuv#3-MiUSWkefqmPPdRPWkM?+7!LpUGcP(8}FW6HX(Ve~3 zNLv;u?FC|*VsM((dKK(s5`6Rs`pnbc>`GkI?VNk zTz4}XHW#+pIS`t~Yyu%S+bfQuG#j`Rk*l|VU`3=(}z zR{_e3tkeR#?SfX+@kP0bC$4#lmfxERtuzB%7Ukpzw4bl{h)V9PDVsxy-dtuCx{Zqh zCN4}3L%wfN7bF!ugvWe*_uu|%Vwx*m9l?6jY-T+Tt=}{Lmw>|k`JL;l^0wV%t3Dyfm-_Y51;Yo1N zf^im1a1-Zrn|Cc*+_d{8W|6n)^en*1W(SY8g%WQzs=)B0!{aHYyB_~-{Pn!_QUHFu zd^HWgko$4q$tt#E0(l_aF*hQGhvVXQr~@V8+Hk~R$bMK*hNlK6#)*&=0BMUI#D;96 z;MQW*8$>7DkfkO5WdqD4}Gl%YHxT}+b>en9hd_@eB|@b+{Rg+ z;%VS&5#f$OeY#w_)EM$u4Mrqqt^v90YLtrOTfjO4?e%|gTv}ehNmCeg*OoOt)%?%f zH`LQCf5XAgE#`@jJL|ktxen+&q~B!&{@>n9?c3R`2yjiAttw&X*sBhy8E>KLCBjUc zLx0sjiv|lRhwV^WMy#Q)5c>wfY>~#uk&CT+`jdj;L@Vo)B*mbN6AGUYXB!b15faOa z(hARs?9Q{OT-vb!VSZ+v2Q2YZwQV5(AtHu+bpA_{l8vUoEM&y2&`NmQ?=6}4zU`MY z4u_#KdVtjCBjND!!{a(M7>i;AEkPh?_Ge}LYEYSxS73y@u0~j1j3PUVI8p+kS*Ccv z@4=~$y+mq`*fYVhu~ik&$t7Rg&Huot-P+H-+kX{G?2st``=I_`Qy>3xNaM=C29=_f zzj|AWC?7~&=Wd$0x_N=r$uh1TRrJ)bvT?@;*9cHqz{q=EnJ6yhE(g%|@R20_|mmf}y;(>i6pO8#Va?tPZ z3=YA;K!A1B6@gI4{P|mjcqbiY01jy4NM-yh3=htzA7!96LR%7+rP>J1m{t~twwR@S zmr-1WUsz3ykPGm`%Pb^Mf4cHg3vs+sLwN4wQ^GPD=5snW)`^w|fRd*9eqw?$*-JZ03YkIp<)bf7lt%*UFFo}hbQ2vWQJ3mGk z18F}-yoEFTDo|@0%d*{7BBwJ&rEOBLd8QdDs?jTpLkFt2&~iYv*({F9&5;3DD}T>B zFT43%GW0OrmfLqAiFV!*y#d`9EB#44Z|=~Es3Z$u7e~$C(8Sne_50~OZo?n;eg$-| z7m8#tdSM^ui)~_`m(oau_LVA9Np2O-;{{uQY3@OEt>zr_h?&U1=j0w~SNp<+C;S!& zwAzoEc@N(Ju#%5hzzzNiDO)r>bwOm4G{T~hP8-HA%8>XGB|^9AB@BdSZV<}#5hi!B z@ByX2BZVxmydcLFMqVC@J+izYp}-P>kfzZdpmd1l=^5u&1SlMEYp|a9S8-1Ao=Q=_ zz^v#8E#luWu-D~N&5udMgFHl-*)teYPdF!N_8F?qAi^nwi&@Y{T5oR*%;yckh zYd-$xT71yWWyA45YP#9~LrqWlBBTogpPlgz0DgkH)-u-PZUj^^2pCSe8czGdd86s% zgGln=b>AIfF!M&VwCFOZ;~VGzvP>3j_7;{}Ub4VrSy22`uWVU?8R+B}Ocv|>XMlX{ z35zYzf={4WF2}xU$#xj&{ouXZx98EL`*E%$=ksyI9vr7{9F9=!IxJ8ElfM|a?Fgwr z?V;4C4418Z-5H3D(OJ4pmld{c9KS(6$`BtV#YzYg=0_m1gCQe!?}L*HMg*yfbF zj3=iU?V_dk(HesF5g(G0E&l`DDUAR}z${^@=uhUCxQyT~-4(=FyZKhE>O+W5+a0#9 z>LV4;WSkRGQl~$})~Y(x@TdHVwpgW;+_i^tlb2~Bop~OV+7h`~xRrksvvNYzDY{d!9RXm3zZOfT#cL25=y%l5ZsxPPKhPFWA z^gGe;Lx2yj`dDh{XmJ_P1y8vGN)x&oky$Mms{;PRgCxZ`z4FIL&@|5A6RY_}*Tyd< z0n3T#=64%vLaR-{>*rstr7bh(0m)fWI`sE#uWI;cf4@~M*rS+nRXxGJbv2XvC1oY^ zzY#8%PEnO@v*VEhYi@H{-TX2{`Q#9sLop0|r5i8=l@l>DlWJQ)+DkN;DD>~4_5$fu zN;Z}77j6VDrY45io%$+9>J%=iBMk-RX@c)P|+ zL+JZ`IeGxGG$tq5FijtWecPbcJop#xa0&HzJ3LZVn(tN#Jc~^x93xESkrGjSF2rZH z_IB#x5d45uuhwz|n8QJ{DXLW!*d_iup&Rq$5W8pzCGTVu_ACvI>+Jyy`3 zCMgQXqipx1aCG4CB;GV5%?uxHedk9W_la5vKc>dt{)**aF?dsno$iTTfXc%cEu%Kmx{>B+NUw8y;BsQHp79k|<{o`MRyJOG z_YxI23#W?yKuZM%uO(w^hVWLGHSrZlspl;QO|sI=-{GygagQfcO#ol5fXqE(jl@3mMqUyu+9r+II<6FMZq+Z|1Ic2x0#5IiFV8QXok;BAYTav}Q4m8Lw zYzzKadI&wTwLZ$)_X+l`M%5Oyks%n86c7SNl zw=Mvc<~A=Zv7*^Yll$Wdc3LNq-r0QV0y%k8W#v-@gY=3cxGqWwS4LGXJz)V-kK~GA zze_doh|RV$j)z-jLhac&|B?h=&AnjP{%!DMP(NX#w@s`Q$puV4&P3Mi*(rQr8Y#93 z{v=li!pb71I5uqq!9z1*;YcHo14tO+W-tW`p_@<&iXf%_O(kyatTFBJKA3C3d#9i6 z$CP{hJe%Qd`r~O~&h##A^&mAVC`TgUCw^eQ@XqgjGP;~m{R{_x{)+HaK_BSsfg)sp z_M|=(z1NF?M3 z#)?Fn_ySmn{>{{4@y32gt4!&=3^`N@f?&Qsah#oTRo47P`*Xz{_P1gBqgVJ1wPi#^7t#Xv|OY3V!-1Gmqa_TjkAkn@qSi(^MpH#1J1xx{0&v7=q z#Nm6r68SAbKvQ>}1aFgdD2?-PyN#vNEj7XoI&jh%1T`TI9&CZ%6=aI z&k!0IQd$_=+=;)>-HnzFjWW=0vwz*M+Kw{4r~Z4)zizd@%YUP)z=iHU?^PgpO8frE zbL|cNanTE8e^VI1!;P_kO9v{YkIk!e7sIie&L`?Brx)X=)6S{X6-%?LmY1>0C0Sd= z7gJi7|4>$jW5Ha9><8Bc&p~-=>`lpZ*!GpjFlrE9fGv^DTA@Wjd&)#-)O01;{)t-c zmjAODvtK6wJ}WhFn$HnD9y|rcXGLm~02~|si)ySa-})QHqo;QyGZhHR6_)+#wx_`Q z42MfM!n?rfkY9@gSZ8vhItCQ>ih=WGrp7R}=MJE>ZiE>5NN(?etGEA(U;s~n*P*!9 z#2%}aBPN|;`0AM}z@=8`JLKUDfTy%GOPNsZ!4ZTyB=jOfJiu-+IK=jL@%jZ_op?a_ z@)8b-5XA0ufP1^VsR6OFdxLoWLPK5Nzx`9Mf<|5T|oap=hyTVJrNaOuNv? z4DsW~Dugq|hc}Czv@7;?rq2Gu?h;mIi@-~ukrpC2>w+b4nSOrH;2qm0-X_J7CKxtI z!NIe%2L~)z4}*x79Em~YNV8@!$5dN+FXqgV-074O<%+VlSV|bHbA0-1G-(ECXo4>F z-V1g;GHEQFY;KWoeTZ25w`7-m>5{*ZqWfG|1C3e@ka;=|Ta2AfpR>*0D0a-8^2(E~ z#3=PyUN3s%Hcq>lK|#lJW_2u7j&zFu(+Lo(cLVy2&ce=XG*||@H(UKGwYqK;7how! zh+Yl@h$sZV@N4IfO2*;r8P_=3AdRh0BZ&P7r|`jLq|(eQOp};TKeysCI{SWMm=OQ0 zrR}nTOmqb?7xZ{G*w6C$M9A#y?AM(bLtP978xiQN%MLhWRtq6ldJZs%L#1Fu48?Ro z0s)6Ywiw*~e;9gm&(sM_?EP|Z{Ye%sOpT^*eEutS>vzoyQo85qQC#P_6wu@*8a%q_piXR7s6C*2^@Mu8GR zNg$Jn=x?c%WiqOpW#JeTn2hPmoUx#z;4SdJ+JLM`igcs~0FBzbz^zvYc11mc?QH2Y64yU{%l?uZoPGFB&7em^HLPn9g+lCqvL8~})_slo|rUl_XY#Nxy z%>WNMdCc>pOkaLO&^Mgp2671i!xZgG*?}{fgzb)F#+*u;cBS%xHvU0+Fj@rfZAp){ zGHX-3zi-1NASl%E8<2fl0Nkg^WWE){S+#>uhu+m*<;RGS=)nX;DQ=o3DrQdW(WLQ4 zj7zUyfRcK&X!}-sNe@XZXY(30Jb*qzFwc9IMm69c(tOo^rOb4pGTDfZNK>o1A|0Bd?(iW6K9adHN) zyeKA?0U?%C5%UN<+*p!BFc8MG$a1 zgXUoVS@mtr>f|Wf5YsK86;wIB_kdw7aDf6nX4U1Zv2PCumb)lB0}k8@02nKap4u>c z!Iq1Dw!|^c-Vpdl`gv!xtyh9wLtlMM$J9E7n#r$4Pr`4c-H0Ajvlrk%HjpFa{71-h zE_cNey%uoquZw*B?Tq)>PiO{jLRRt*Dse|m`ku9;dY?7Hy-p>Hqxhg*q2T09@{pi{w)b^R+ap4EsCn# z^cAy9>G;EL@a^o*fuZqD&&HiEH2mL?I)9+lA;qPUdw9*n6&=tuq)1528PMxF~`(b^QcUil4tq918^ zdD{p6umIhhtWbV{vk78-QOTtY^Z3w{Kib}WbfSEspaZtZ3JM8{$Td67!;L1Yd?KPf zY0(*PY+=Fhg9kXV$Ir&$p>}j_?^rA=1aIC$<7_T%hG$A1f~~dsmp3d8I+bwZbMal{ z)c(LAEsaMuBKmoywnZ=brBx49r;l^$zb?}am( zZmB!eqDxZ8_~MKXClTx5n#g}%-|lKeC*FDt{)M>)*#?Nxa8-WbSCU@YEV6*uU}aK; zzT{J{MbB~INj5m>)-uVuWn1BnUHeTvm1d%*@T!eo>;z3uI;Wm%&xfc`6uB=X^yClO z`9y5}cR|qPd57V{A>B1b^{^q-f~a}}zLx7D7{6Rgj-eob(3$dEwu&IP#lVDMW>Om3 z^zT>KjZ9$HD?y8iLyz#~^6;)+{OCJr+b3vKlhxU*7*G>;PqC3IR#$@DEqkdx`cCg= zdL$^2@Cu|J+N)%Ze4SHKwk6!m24$S|x9>bG@Tz2=mu+o2u$jwOaCwUm;ns+e@GR5* zBLqye{!BzvP9j*84pQf-m2Iol!ZeJgE#9oDF9*`g$jfQ6iaFWm_1qIjoVE?UWz}MC z#OELtdp4FMxN$_5ZJUH>-iB&wJDKWU*rHB%Ezx9>#HfZhbGC7&#>l;w-eq*T5fGvD zfn|;)x?rD6bg}2U*9onW2gzMkn3}XtYx`g}JQ2IJWcj{#d0Jcw+a=o!P$6l2}WYX?tS zu{6^X;jL>Pn+0`9GZgg9-z3|MKwf|E-1okEHTzwF4$4w*nbGC2b!J5x9SljKcCGJV;5$sb`TI_=NyVb4eXg! zJF~b`vkujuJ0N>z%?>yswv|T|Xv9jRc*hZ!an7L*bb!k^>re(7XXn=&IKWl2X#FvO z&hDc&aK!GTH-OFl)&*j+dWr`zSwBG|>Mve%AnGq)$KrP9?PTEkmF?(ryoG{3=dPg< zKPz@((-Xg3n`uR({Bkji`$}?47WGmI<`(su2}1Mxz-bQ8AWC4~C?3aZ{%8jKvy9y3 z;yzotk9@+k^{uX)U)3&lIlpp}dD*RvYIebxTsgmb(oxy1kE&ko7`$qmW)gPUt&yr; z{+NuaXBurw>8+EhUfvj;s%H{yjoQIvd8b%XOZhFQ>Q?obe%URO>Q>p9ZMlu|fo^%H zcG6=h@K#UNPW_-p)ia0o3*Pjq`s`0jq;g=d`W#2=Udoz3=BDaaTCbdhT;Wtg3oIGa zE#Igb6972mb5nf}G(sQn1kBLe89Y%8FhFQbFbjU4LObvJNBZ@9{U#|jQ@-? zqyv+8?7i6qK+_*B!&R3WB*Qtf3{W7Xu1Oi-QtLuCl@dzoj!7I)#G7sCEB^ABAkf$;)v0mIz6Y8$8iuJp}%HYVHy zU?t&H1iz+}U&QEwG`#|LkP&iF4Pw57@4A6>5rD43LEMo$KR}a6+`2Z-!eQP3G;9{2 z&XfbyfB-^|>B{6a7}y~Tf-z3twh_)K^Aeb0GgDM3Pp%?6o6cx$QeO0q-q-F7GV{5? zvB?v`KmkH5%ek>f@(vGZ8As6MDR{>jRD<&|am@s>2Xq@hWq*SVGLbZ&0U()jXXb?& z!VRcEvxgrM)GGR1__GbJB0REn{<#LC4U8iINqX}T3h?*U)tPn1ui-&hpbP*h!Ue)7 zd9MgU5CFutO@BiN>cH`vye1x)N4R3`9J$s9SHYS4Z&An)Ml|pH6qY**G4D41!@S(C zET{u;Y3vb}7qo*48UbV*tc+aK4_pEC3_X%|ViD|FJBO}G21*e0nSDmDfyf{r%UkFU zG&lrQ5=jC{BuR!L`+y2a2|65GZPX{*h z^h557gX5>dVn-0U`alLD-~pygTBCNnK-xHVrWWSMUmwInW2EIqfZ%T*vdbIBXW^c? z&I{c1;*IrNS`2U)8i#Q1hRGu-1~|ZmGiBK7zavHP8RPBxM#TmEKpf}+beldQZ&mHN z>%0&WEjkL(B>7jk(EMQ!$C)b%{F2Tgx}UHTJ!r=V zxgR@mZxxQC;w%A+zUt81UzN)Th92pR$Rkzl_@i3lyCDe6EE|>Mich(W62=?@_AWe7 zUG4OPTDn+<(TtIfS&TBR+Iy*-U|;(6V<*zBvT>;=I$il;YZp%lE*{9~u%~q|H20+YV1P{>WPKR-FfX7;hAhW7^!oPISgIXpB}> z?JlFY!6(@aTlxG*v33YzR4`I4ak$X3nK>x)=wTG_$Y8AS051o0@<%tdilv-Axe5fu z?ywiz0=Jb2wkpQG7@rw(Bfh2J;>erBumM>i+oanMk$0e=-x`Jc#?>GZ{9@&gJ#E;1eMv;s;MxWkrM6(C;PujYIkzB{OkUr8W zJtKclZpGe081?%{T%TDv&u)K5L}EC7pOK-)hTXw2*dTX;H0RgRv4hWb?{yyv&j{ncy21wEh&HnzYm z;94$M;}%E4meVly3{AFe9h4Px=rdf1Sz>w-9&_+-eF^y6G&w=f&YG@b7g0^2IUAc9Pn&yOr zauZONn?@ph9NlfvQy{N4@>TDmtl|!8l5(|G1Ag`9ron}_T9`rLc258v&G`@Zbtpt) zDjaSqn#GqxlB&*`U$?m0%szEEF4P{O3z-NVo!EIdLO1)xA4tGb7+n_?7h*gPl;xN2 z{i3r(D%`PqQZ9JDol(+P@lI+JWY1>lpd!7zorUppOWPs5o>ET0|Ej4NSb$PRJXcc1 zdVmtx5sstH)TPCG{ZaWDM>q%ptA+_vow03sty1$SPpqE)YiU6iGxWZ)6bYhBhmUC+ z*?ZbxQQ6C|oKrvLu9d5P^6se%xUu(GdFNCh))nTN(fM&53>oEkINHUmjn1Aoj8~5I zf?LXkez*c39&A=b&o0ysm3}l5R@j?WALx`YC)V2XvD)8_t6f@o*!v#I!qnnBfq}am z`D$6<>#BR*8x`w)R+B4I68$pRyx`l%v(qk$i}LbdIjq( z+4=Nscf6`(Qh86IGpT|4KM?A<@u$6DYumI|NY_DyRE9VUX%%`J0RT!76KWG$y*wf`SW!Nf;-T@8Qn*LVO7^Xy|;jcFD z%~;pawq_$3k@XD0ie%~)xScZ4a?CU2l++HkDFPlqheLtjIvB}m%eN2x8jPoxp$Q|fxhMD*~^wm zK?SLlQHK6X)-Yw_LvC@cZar~T{#Jq#t_>R#2T!cz)T)H|n9{mZl5~3f16APL;D)`xhZW8jJ%Rd!^1cD^~t{W&#CPD?jGAvFPh~DC%yWg5A zkCRQGD7Vr9fpDwhIeT~l)+3yxRz2-44;}KO3jY%IID;0nq}+m(QuF(hx~e}2Jw2_c zw)ND2PPPJAUiPT8s#P>P6BJKN6N!zK)3=)4h4#;bT@kC+O6EexjTgOy-o0i@sUB7b zd5KzeS0_lkVrSMtxrG0C_eSreS-tNSI6IIJupdM{f#=d3i)T-Xj6YW+^1G`fx%VEj zneI*GFCn4Bw0$F}1Pt*Fq#lYKIPexS>)-k~=U-`F+dr9XAGsSInVhoyYQ^7!y;Cd& z)?^RlvnkXc;$+>7vrsiBU z`!N>pxQBcP14C*VZw5+QdKW0GY|Ah|Lv1|DTegSpt0|y$~(&!wIeY0z(e5cO&i%psFy9XXe0635NY5p7-f$5xFL#^$EfStmq*Vfx) z^E~GXPfH7d|5c#cTR|dRb=7mCkb`L0kavl|jGDih5{5v50pQ0{s!DaV>kJ@W4R>e*Lid$MB z120D;%(1_Z+?CGus?D@A=ApW&WHQGd`R>3*K2yx#R6g9ICZ9{+J4t2I3wv2vNz(0P zw)28%s)XT*I!j+I1!6KM_7v83U&8#J_cqqo&_GJOv^3yh4CY3I`*oopV&VInVi9l5 z>Zq!OdxR)xUkgGA(%-;1=-D^SE`Ej27x*S7q2-#jyw{Q=atFBft}+kH4yyQmk0H`I zi%7*@Kem^qrK(HYTPn}bKGz(ZHD-ZgOxLh32c(-lVC z{cjw(-so89leYBxzfae)8Te$=TZ#jFDrc=}OvIU1X~s0a;}USGp?wYyD#%bSy_$;SxbP;OLWn|w9ikS8u{ zjloeMvq4HFU-`z?{&e1T^0QaG@l(&_oIxe(bk_XazOnKgd(Ys_yLwz1%jGoYc10{L z^oBn&bS}S)bo+fpxCACE+vx1yUP6q-nK(j>pF5a=WK}-aY7Mf(9%TGhK;6CS&04v= zdlA%Zn&g7H?cjb9t)E>Q{r&LmPnOM6ETYjIHaw2tv85M;oTi>{c=dtQi-IvJ&neL0 zhSo7ynW)9Xwk>%nx#sezOLCZE7+TCfjkV{VR~KwBv7nbU^AEOuh0!b&hUce>8tLM=?#KvS1vj)JW@^v9br; zM=lxDGitdp!b4d+Rv~NBS+^}Kn|C;LKd(_xz-z*r^9&)|hk+L6Y*)zTv)XIWD;PEF z&E)j^U4Jb2r*=>+q-2Qx7Ow52qLVK*4A0Zu@@_5tt+BZ1Of@e^Sp1>)XX>YqpP}VSN~bv zdv&d~?{#08CYScwguOuB-V*$Y6xgIft*ktCr-j83Sk15po804SDu}eve3o4n^X}Rr z+J%G?vGJPr!ATi*bQ=_+8zR)QWs4_%Z5PsB&eO(NEAC8S>1Ddg#h(|f9#eASCSy-_ znWbF8Tcn4y1zIzHp9z^i>w%I=vlwT^U$2dzL9%~-5$jKOtR@+GNJ6|q8A0R zoNO5MFYXAc{es&Pnky6tD)H|(_qHN&`yGD%0k9Ty+b~++iDVILwv~9iT*g%XwJuBI z<@&knUMf;GrNENQ=Yo1!%SbEEn zLimrrg@RdKzC8@FCGU>CPyJ4bW#Ra{YjV@vMzI4{)WN|t`%TZ7tqHs{-i6%)L*9wM zfIPVQ8rLpOm0o`N4v6&?Hc%^Juu(fVXCOU6k5PpgqQ4e~byF{81XF{@c9#vkbKG;* zUcqna>~#2#+(ryGu6&G*1J-;f+>29_qlLS_TvK!<^4;s`+WL1^7Q=0A#pesn9>3<% z7XJI|C7sZ0=i7%DUzjpkIz+DmfmS_SY-6A(R&S&vkhyE3JMOr9{4N z1H8&VFjXOk!N#|U{KAnlVW!bd$!Zt?WwDLDIoJe+yNMdp3HB+R1t`JPM?sLhxW><{ zL}=QoLF1LbDemO8!@wvPgDv!L7mpO?+KTJH`jlNeYnhSyYK*NBvlka;-V2XXKOmza z9(@Qj?}vpgXVzkHqi7BnP+WMW&NUQ^7~AOisEH-NlzNHI0ECBm5!^4FVL$$er>%FR zwc>6_CO0;4$WOxo%Iqlhyw^b~HCra7*m)*#XmB(vyYyE&Uv|cP`uYg1exCNHxk{9V zOZq(RS;4PLHpj=lskh`s=c|AGV^8z;j%cTkDH>0g}w;q5$ zQN~nDo{lj%sE!j)mb*$w?dGr4f45pyUv6@Izpls(4OjyElh1q?AY)Md1NvjElLS~sLK1UFjjB4QyY+P4G+B?5c<2u&4ImZ-@ zthx_=4Cf0GH!akBm*@8ZojE)2b4jhgR?Lp4WU1Xg>0qhoW%!ePbAE)K!uQ}+4ztyV z`{WSv=1Wx*@y7RM?kpPgocc05^n11{jBJGq9kxGA2yw5pQ-)lkXNhb-_kIKg*Sm`K8^ZyPt)!d=*q&iH|`95wVWbwtOIdu(9h!V1m2KG1X z{)}{YBC{Wbs}IIzWGpD#K9J>%d}DyF*XFARa97!fbKCjAcF68$kytiN(Z?>yUxpaP zE>Lz3^=j+Psx^RyDV?Ngn};yL457KU8t}C0QW-l_J{y;7KEnhVGBnqa0-+K0gaia( zeQje)`)f&WA(T0Hb*j-koY1`$syr0VHU#0ZqaO^a*yVoidX^1`nHP>o*)oPeU>I3tp!lvio+3 zC@+{BC#2RWJ96P-)rsch*lcf>}#r%|AJ360Re#C!x}7sHyCjDXn}l*G)@ zK`%Ghf8wf~#Q-$lBx=vplIF{+2zG~{F+E4evo~nG9eH=MZ(z9qH?+t% zrc7Z_e)~w2y7^Cv2(Y{*9Y+y|*iBTes#v=s;@KgRgM$E9>xE!(-(WKAjDfuWlhnQp zKZimbV_9u>2H688vuj2@Il0 z3TccW>c?$8a4SXm$Eq3uiqkp}!VDoBC4iVUBL)Mr27^6FuAzP-0h6^}98y_LaliS= z*&lMHQv9PJJ~T>=rT6+SgNqyEOw4reUXP!!Iy`8|0pyFZ0Ly?Ojvn+sm}C#bEkQt0 zR-`)~4qZmc3^Wc?+!#d9xNQ1lV48IozhohO<#cmIECa8*KQq9n<-L4B867m@pFy@WyCd)%K?w&LQ0vo zc-BM)Kxj9v7Ni!1^HlhQLl=y;y(fMy&2>4}boCFM0P&VD#h~z=|(U?P%Xq{C}6B#tLYaKr6`0sF6_@)KU!c zMfvz=c#s78W|I7|c$G?3jmRc+SwaWeqLE`NL3jwo+Ne)lKU-?E@gPBBZg z&o7$HpOLnniwwA0qQ3Cg7!#h*6#9ivQT4!$+WNs8%L#?}4$xd7n%`i7sU>|r%yop~ z&ymKRZ>glAd)nRYpfBne3qWqF0+^$4p_1n1#klPQ9^WWtZy-Oob(r%Ff5tu9;hCcO zT%p!Qsj9mC`EZ`N6OT5j)xWSr+LDa88=H_ZK29aKMAIl4GwW9-LFkuWBk70@#Jp+) zs*yT-yb%W)1H)02ybY*Mx&6Hs?ppF%oas1}TGNhruL|y3kjuN?jWeB%??XZ460rk{ zQ635Ah-FkR3WwA7ldpR+=n6QoB%0@9?Hfk}Z8*7C)26LX?T|2d>GCSJcSVl(FL~(``R6kEm+C)jCv})Xp%;$f4QI?FNz@D;cq<-(({WqlQlt(Y$ng1%WDzU z=LU98vIr(BkR`5nH;n61b7{%qwhWwyZLUhrQvFVxs{x7oJ0sO)uy2yVAg~x|L?gU88!BxP>LeT0i5-5Pb_;MG!Ng8hC>P#_ z63XX}9~3ikz(ovYR)*X^CS@T;s+u=S&w(KqKc!C(vNcc5jvTXO&5Rl2S0zq9l0j7k zLgiEDy61vntKyyR4_6&JYX9J5&fX&$1=2Ak!m5*bt9|zgFJq`g%M{*41Rz@ zMJsiX%H zHQu>I^XiVQ?k!8ymJqKivk(9I06b@##JK{>vRA2naCfvchW$5ll0Vu`wo8&!b~V!$ zA=z0Gpc{6rYswRvVY}6Vew>YY9mBN-^YrI$JQAm2@kTm<+waa*e|zEIoD*^ganc~S z@Mgqol=#zHjWL+RI2}XP&i21xhJHluA@%_;_}5)(Kq?99J5;sd15xG-CzTuQG6bQZ*q3rZLTd7NMIj%|UTxzhm&{ z3JB(5+_GBhJTWJvE3@1!jGbe)%oA!rC6mh-*caonI+2)u+mA!H0@Bz}=L;6Da}9BW z!4Kk+c7iT!36wvjJM4*2ia*ulRFS-vf0d}}^EserCBVg8G1{kp<)KtGD#;|(GPH2v zu(=#_jb@%>p5$I|d;$6p19E_Y8FM+@4`v;qd49?EkyOl6SW2GXLuqmb0J@8yBZy)r z74-Wi@&x`{8)XH-%B;LVtkROa0>g-z5yix!Ql0!>$GeJVo!sa#R%BSv_CGu9oy(Z> z)NxV`5y8^q??z=^vhklygG4Hc7!&PxPF$lnxG(s^M~%csj3}Z@$L{>#xes~y;kV7X z#+F$8(2c`?54(U#NJpag8K%2Hro#mg!GB4{u6RL_JyW|+2%$ZZamh&VqOuAPX_l}z zIAtD1*%4|Q2|IMB`=1Z>gF>O7Z!b{(P<9JoFJ>RW2`oX&9X-Y=JChon zTgPRYBrJ-{MVeA^bw%?^p0+x#u889~>mzujj#9QV;Ljkj*^~39d?^9FI98$TEt1cS z%jTstrl4)K;AG7qz?-U8%D%14c3F2O@51qEdJki!4Hq1r$N&8vc2(U)&3^yu1!5jO zM{G<^dKktY*dFbxL_IDDGQ}BANzoHdP#{_*l4J52T5L*d09GaFWZL()AoGsg*rsl7AX9|B3S1S5;S zk-j~{MBEJ=YrGz4BD#x+7?ne-O>+AgR*vLpMkZIlnaRVyN!fyilQr?sau_^qI^3s( zIb;RLtOfLEOV5G?q=~7bk&3(!2ZXgFK29GX@AM1(GD4V(y)=2MCFg0LfuSizZ%HCn zsqNxEwCTPwGh>-mJA1!6RJtit=0hAjXt1G(f(5=3n|du-Q61JqloLvt)T{py33zLa49GDzmi;jdG zx>EF%c|8x6{;98klJ`yMv;GPtch>h-V=wS_ZsBS!C-!?FYNg)O-)tDZ9wXW zSmjE7p()Vng?@nqMe>Amcg-LPNvrTono)tQbC62Z5g5Qz$>Kqr^QP8SkdesaKNCRD z1m|JlxOV5w<827u zEFKA91y*QS(^7+D)zhOnZ$zRyUEE<`U#v7@$R(>Zsm^awmYeN7K zyast~=7){0y;W-Vxx8xu{yc3|6k?xh4b~Q02Li08b=>3XBD78jH^|k(cJHedC5DZeb+oG~If>En)N4>IQt{rofe6$+&|{uv8P7ACKp0Qj{` zW2z?O+3DHFkWX-Xjipj0363 z43eS^{3mJC@-OL}s-pGEnG5t2GkEEYFQtbWYz$B@EuMuzGF_(q-*za?v~DsQnLx?Y zd>M4d+ihaO$;pPHX0Y^-+DGk|^Bpvow%|C+e)fjZ$8=qM-ub5?uJ}53>xw>*Gk`2B zuCCZllnP=C`=gn5Nq8zJjXzIG>4>f?Ic}DJV@VR;6)9Qwv$17&#i`Z|-}ck3BaB(w zUmc&<+0qaAL&jG{85{*eX2puJ4$xn=&QMqSqbu*?(ay~bb@5jVDEnz~{9=?yis=$| z69Pn~bdDxnhKQB)=EXYTP^2Q_PIW?b{WPq-R^>3&w3nZP0M-xP9Y_TyOuvThaOX7h zvuO?s&<5>z>auzDM*~|2sMW}%7kwEPKI836I*x%@bNva`2wXqIaUvBc7WDZi|3hi8 z@T&j$vjr*R?4DE|=*ZoZ$AI9aBm*Le^fQ;{&?){|OIk`h~ z5-$OR&P9ls+di{{K+0{YSv| zVHK|cQc;58z$j>Iul-fK6~51QZmRt2utd&Dq6uiI-c;q~tE^?&Vx1QGj|ctVCs2C! zxCv4_WoYjd_3d0QMUmMp*{8*DRoDs4HY$E9_1keaSl8~mD-9x5gYmBbO^rHxS`s|Q zDmX3|<^ouJp*#VnrANfs?(KAGqH;Gu@8u#bkRZ~M)K>EQvT9pE24(YO1B$FN2kCb= zP>mbmbUX9nB9v{lruLMhX62td?9iEeug15}`K9fWcW;hUNV3H|tMk_5 zZzT5fNc9N$pWBQ?FwdWl*YG`|uiM!(lsyCs**KaDS$N(#-h0J)zN=OgUZ>gEUBDuz zT0~2(_sHrCmJ20fod4y^b|fZJq>~eZm}ak5cDrCzL*3Qhd|1F7Y)2ssS|JlcC7ZMD zQ*gr+roXK41pElyb831={f~*jHRX%G6x63r`X2+@|8Ey7O=#->SU~?*CkUVAP=}7% ziG>PnfB^d+x;JohqZLmlrwJ{K6EUzqCUC!hDuBbd6bl^ zyb3g}al9c(i&RVZW@4c9kxqSwqsyveZJ5q>KzWC@gbmE$>QxqwMq<-NNN|IBTwnQ@+Cjx2=?R4#2|R#jl`D^ zwnLj?gAMj^LyZ9a4DNY@*U)=U?Ar#U^az3bvQ_Y7EsV0t#d5`aFzDUF>K@VXHvA*u zo{vb6{_I#D=#1`t@MWp@#h~Sgg#h))R=_+azux{wK>=XJ0ITnFLE|ABGn+SkG2Q13 zodGmpfDQL&pw|(DWcF*J`H-A(HfR^w+_T{24zWVnkc`~LoHO)GXt#*7ew zP&Nc(1_(>&ZF(`RMrgz!DztBAF>oDWM>~wl+6~yiHl%M{acCn9qKCu`VnZ(=^cElD z^j03agbp0NGX4vd0|XZz7Wv7K4M4KvKr`XKcX&AOsjHx7y|KWJ#=|edb%!9Z-WdFI za~#ZbqT=9RModz@nok4j4zR(cUy&U$kecNxD6cdXhS+rO&BZIQuGAIAv=}oEIvGhQ zyiSIBvCylz75QaBM|zApmPLwkhcp@E`wAJo&V-3L{BfvNWp`8=<5S^|n?!8;BdQtD zeufo-Z`%9*89zNb6<6-;G6?T76(1noiVt^DipUVB(7oC=^(WPos(9;m*gY)=hW^@R z&YiMl&)(`~Bs7xx(LBcb;qVB?y=nJG(qA%G6uavr$Q6{JZZ={=aeNJ-`dIdEAHxR% z!^Sm?rP2b_TE3zZU+S~X&h4~2_qMdo2^80=uLtrY%lg#A#|Qf9WI7~|)eJQg?bgim z=(P9Y`n|0ZmrH`(k5TS!W>Rv@4Sd|co>&rP(*6jPx>=A?uIxvp6rRUK2kM)TaQ4Y1 zM?Koh?X}E;Xp!x`n2X0Y1Y$Pd4`vTi#=ZkB4|C9Bw;zAFhgn&%tIUl6*Rh-B8!;1f zRWY>YePz3P`JAQuZQYxdl^kbH&l0|vNL~V6Rz$;sM0!%%m}gSU3gy3X5vy=}MI4dT z#NfK=JF>JG9BwUr4Y1o2E<0Q!0>`Qw2wfi=4Fmb{5B_vn*U$T1kEnG__R zj=V#uJh}o$C4k$rwGzQ#pVrJTMQyltOFmwpLVmL9%S1ewGn_w|?GF|{r>al$BY4ku zBLVKr#OtpHNiP46lk%;c)I^U<#+_vzMwZ+?DF$X)ZNd>EnQ-|!+31Luh?9QS*WAnn z1l4y446DhaB0|3EGnYDXmNR*t)G_YK7hSoFzoK9C40;sZ<>ed^2W(l@_(k_qGWYj@ z);j8siqmZnD&xZ=X2P1&8>Qk^RsLuq?yeIkjUU&zDKX!y%r0!S_6(Dl^s6hYLB1^G ztq1Dok*fUpYSPVC;(aN~#U{#P;XGCvkSUM;!u$Kqhu;Dr@5UfEZX(kE8LtHw$%D6@ zH0K=FyEUiemm&4oQ|*=OL{5FPwG$&2hyvWz|CJDrFc9}}ki9HeS1}&)7py9h^F9)z z&bkoYpYyp!-zc`d87H8nEZ)lQX7YE-3qKV@U276m^bLwY07Auh+SM0=J)$EcbQZ2F z;*4E^D+^=f>{*d0SQYS_qI!^ADN(^Wzh(4mu| zWy9D9JCwHGa)+)*`M{+cmqXGVt~#--hHgVPZQaQw^5|5R)(26mL8p|W+4eEzm4Y-* zaHT4BZQ5&4d;e5qhOe3SLC`X80q7!0UyntTaIsuUk;%@yxuaoXXz9_`D3k2_lR^p- z<=*1tVjSO6&d#@jVHoLPlqekt@@uChJ+fGu<2x8FwjTEDw~&IjYO=0AUR^>^VnCy- zYx87@!mg}!WHovbL2-Dk;SyI^l6NhQ?0@^Aih=h{yd&pv{LdryWPcjCyfC}_nq%rlXfb;x*|7!r5z&gB7v@>e~nuqIm9A-2H>R8Wyc@`gC4 zDrcFi^GdU8_%^GYZmdCd-cX%7`xE_roe^E)bKPg}WSjH4k!MD_57u=&twmg`8{F`t zl+Xr38Zj|h5I(;!-}8>=u@k;B*9k<4jSYPBsPm}jI48fOG?P4w0h8_Wf>C@kf+;JXtZ7)^Bbltm>f!B z$BB(jyBg{tj08tr?D~;M+ZUNUdMgWZN^M19My2E^8@3inU8D21UO41~aTJTmGN5pq z(JMR^thB_8WeKzSnK~0}KJmHtyGo06G{0e8H66PhFHWy?fo3^9#0ZsNn41IGro$?~ z?|(`{?0|jRUjb~WuP+4nmPy>tGW)P}HfL^gAp5pt9_BaaoBKXn(AG~X2yUu&Co4=G zQN%ymCM5tHPRSDFOPqSXs}o*ttMgm-h;0m`x&H~Pe=Pz3(a_}!N&Mw7t8%2D7ADXY z##hO~w$Y1j1G=K_JSZ-{vy>ua>a?4Md{!M&*&iY+KBUq1P)azU``k4^*d?du4r>3r zP(QNm{F71GKBs&9TZ-WT=8l4F1(7bB;t{om32Dl!(0(Q%VbO=Er-AgNck9Ld&7PQ6 zlc#bSHj{L4f#pKafTY)d(bhvjl7v53kFIKo*cgm61L|s5onthekNfR$M$$I)WbJzD zEH#!=HQVQ7%!_3uKvPVZP{B}LS>vn+qgqP9UtnuREjozKvfW@KxS+m0x}e~|-EkF3 zr{4)5gn?onz|^RErg3}U{bb%+ZeCd@h>%?Rt-0w383dQNn<&HFO2ewTbJFNar`MHK zrz%@#69n3y`s3WJJ!4;bBp{oJztHLdw=e~0`&4xwBwS!;C!lj6up-KM_UsvP2fjG{ z`ssZ+yASnOUb1eY0B(1Q}-!+&wuF^&6uSlp{Anz}3c>j#(! z{XyShh4EGbEe2=~=RFV= z1ULvLs>Nz-SlKkr{~TYcvqU0tjuTqr^JYKu?6voJdf3z%UY`NGhF!fl0EojH zc0t)W<0w|~CQi&qytsro`+|Qdw0Rk+PQiVVwiLj?_pop#Y7Mr)$ zd*~S$D?QqT!cC(`Z!??)uz~ERuBpjZ!P_-|d5As7TUAX|0hG!AN?b^l(vk3FaMN9% z+u7rXsMR>gh5O68wkAF@_hDe1_vL~U47%{N^*GULnrslC&+y2B(i_o?Fr1q&Rdea| zFrFi3ENtPY-B31k)FZE&*+_uS6R5o!cH7qtMt^-kD00FJ#zv;Qns@sPEi_SIxBuy8 zX38STMrgvt*rssSUv$pXynItVvTfuig^-b0o{#qUZI3}}zOF`P!wE_=6-k}??UFP@ao!%E9?bSzAU;WvdFMRK+y+LQb6d(&4tyK0|A*@ zIFH~kr0x48!TZl;eOtTD1!kFz43NMZN*rg<%tOMRV^p@W9;j39;`rh#EhDTY%BaVLP} zS_>-7Slj03V3b}a{=P27QrEkSscYzzqC{%A30wbmSzmkEZhG0*({R}g(eYkTw*9Ig z|JKhY_HD0L?^dVpTY%25AICX=lds@HYKHv?swVYWkF*J@rt~R}ya?<-ll)ra2AuPm zckBdbljrFk4G9m{!p7rN*20?OFPy9?9wiBQGk&oNS<`+w30X7GP6VA(V3YA}>maFd zHtV&f)UZ>q2-|!!^ja!sr9KTWUw@8 zB)-BLVAXv!PmADIP^YFa!ICWxxwC^}A_m-I4@$YPBmS_0K!vN}2PuAfz(3}qZd!o>Pa^VAajZ zIhwd8ZP_!2_(4R>rvTiKX)9BTkikzbO|bu@KFJV$XIDTC8rmO#Wsj67L3KPZAYzW} zBzQ~wWQd<&t!^D|1JQ%#50fw;)E+D%=#~ET89zZG+#q|tm>ku1L=M(Hh>#eL413TR zt_(jxHC#a07tNMq(t|% z3N>j=YS~=i!%GVX92J@sS|kx8K@{FfkUz3-3}_s|1vJ_<0r==QVspcc2zEl~*!fc> zxM0x3k%7IKY}yc7(|>#$)%$b$75gl+=eojDL)-o}LJ&tl4}>3ZJBkMI-Je4GhnsA- zSiWxp2FVZg+}Q&_(MmYI@On&sY-{v>WDQ!7;5r=$)FZ<)`5M!63gWyb#KO4L9A8b# zoo?aUb}@Vg7!;x4bHE57JINqB3f%=u4vFjp5Q8fAZ#}xg)XWRQ)GRB)K;{rR(%two zCNR1Ql4zt8PEc-+*a*zR;RTZDM7`KS0H6qTwuTJW9K!QA0-gTtkB|VEo!&5^nA$)H z!`D5U!^{qwk%GsjUvh`|&UGP1nU6xkikO@qX6%!rDskp=CYUV>Y_Ntv3V1b*2;p9N z%==~?OxZ)8xZ*x%SSE3+g!fB) zBU>A#JPl?iW*4=dA9);exxU$7luct&R%|j&)}T++ux{zZgZqMX?qF+wS)>8Zk>>@P zyHKH53d&@=SYSuA@swUty_8<7mmB2NG_oy!&7f6Dj-Z@+ps&~CRZ-@;bg#4O9sf;L zJcal&JI>H2dIo#Z_y$eR%Ytv1b1P;o-hEM2k|mew+;J79d0bHUWw#9vh6sD|7AdkV zWPRvZ<&P}k_i!{83>j;p4IJP2Mp*ntPavK@&2%ql{}Hr zaG}`MGF371hzx=voK(cuFX5>2hVhpZqT9wkL(P&XpA|>AuQSnjB;E+&H+^WaSRXZz~09D zER-#5J4Vq4`L=aCQ}G8@eH-l6$l(_J?jNj1ltZcS%5OzZX{w_fIs-!SDp;$7mkLEQ zw3K|OK_M&QlDruyjY82qo~T6y<5G0m-*;Q93Krar=&sj=+5)%-*f8P@ny1%qemxuC z2*QYDja91#L?B_`Rr(=s=jJ9P0&!VLt+{LI@AB^}Uk~H~T7UNcsE?9#(3CqE`pxcR z$IKl{FnlGfSw6ef|2XO6fp0TUd9lC3bu#GpSHO!`zW0;YDE|j5aI}b^^Q8)XPqeop zR_5<@+IQ;;c#12liXH3Yw#$*P`_c46;L?AjyKJ^<#M$(B)ahuk>>Wjs3RCcj6yB-h znqqm(8+r^&GPrdqN5d2APrTBDocC90Cl=Tb^`KptPH@W_%(oO|6m+Bu zFHoC0_ieA^mhPj8x5=he4o^iF|QKV z3Z-v@dfs^}n8uB-f7^g3#8oSpzNo&O5X&$e5DmII>f0rd>yEV} z+4Y~**m-t6gHxF(66zfuBD}=JXhv`L`e~02U2}6_=T!4WB6(z9jV!X0`f{EQ9i6|w zQFM=jcU*5k-Mh%bq!C_@+Jx)=X{_OYma>x0mDPG`sU}gooBR`}WcyUI+>b2Ea~ozt z#q-3hg%P}BvNRl-^2!ge`dr^bPJ zp}jDybed#4pJe8G7V~7*@ww((uQV=C-kYF=Y?W$8&D3Pvs_SP0!AOqS%p5nt%5!0H z$G8KKCc;FT=gaohFTh5vfskn$4^#OT5lUBYB6W#mAI)0}&K+`@(o>t}NbhVO*4pBi z1QcoSoFxTj>{*jMB{?P1&um|~LK$>jq{vT+GE(iT-bA0P2fj)zPcEYx7K;=|w-FvK zws_gHbNUWeo}w;3p_RB6oZf3YT&?GJHkW0A0vCZ|FJ*~cn)z!V=TdpV*~PXT0JyO4KZh8AYD<7w(YzQ>O{szz%!bxVDj23s}%oom3wcwJYi46nUO~$MW)U$#fe>uM;TxjLl@^b4e3f2!zd1dwi ziDAY&3kGl-4@6pfMWZ{GZkLFDrQB_icyV6>X%o-R?VlLD+W-QWse1SeBl&N#=>G&2 zJzPG8$$JUNKHy_o{B$>y#-v4Ksi7q1Ww#g@B}~LkCH8p6sBA#cm?km*mwo`+U7x}j zs8c7(*ZK-gXtC9#WEOFF{5c^;mo(NnRS>Df8vnKjzD%{w7-Kq0LL``Q_|Z&Vhaw%V z%ANQpZfSS^of4x>g#UY9=heTsc(&6i$V7X9@=NK}p8$oghXXA82S&1wXyS-mn_SG> zfjB(X-_>}DYvIO8A4>|tvYHJ*itgil|A4?hFrw25^=Q6CXK_Es>Z@@pG0xqN1ecx- zlC@k%KYw_C$vM;zB&=#GvQvc53}t`T5BuXl2tsk%3UfgnN+EST;?CRMj-6S}k>J`) zD%xcC-papNv4dE1okRsKd1YbU6Lifyvr<^hr6}DM4Euz$RS$9VBgH|1#6N|CZjW)_ z!<6^5#e1O_sQhStRbofV?9qjM^hgavgS5feuYwl-iN9l>I8uEbSeOenI_#1Ty-b`g~* zAIpMOX>f9krb6fdx{612Qe(WEldIDwJnLC*j8R^+B|Ou(3(xbV6*;MzO{2ceUJidN zUP(JPH!q*}^qb-?#?MeOqWQ=n6}sY3mtfIA`ra%RF$_==s8GYZH>%y-RQ4O#+(zV zTFUQ5G4(HS@{!HSonY)sqEax;^)o`>p&86=`0HDJ z;b;hL`+5DJ<;nt#&)5z$!_)7pO4F)>nCo5l9xfvt<$>m(oA!K1 z#6fGW@&o4Tk`y1}$)?QVO+osQBL;8*!kjy2J~As9ghA+V<_bp6-Io}?3B~opRtK9#iErWb-~K~@T_>OVN4PA?(Oc###B3iyvXPS!2=DjsUK+vlG`BfffQKd znZOl<#o`{H4cH!&I`C0sv2aN-V8Cbt@O}GUE#(n>pW=L2C znrYkffzI-dQzk~QD#0uzsfTI-F4$zWt2sp7H6gMF$2~%PxLim z-cE(0thFL$pm}4Yr|6Qed`)hDcgb8@6h-Z>9UgD39ZGVaqH10S;(z zwTU1u;kI3)Gh{6N^3#tfpkK7?kH}aEDgVKjXkwLY>%gI*L)Y&-oGG++0%wEE%u=Vtl$2X-y(eGi^s-p2>djx7YynM)ghS zXc}vd69~80xw+DH=13?$2%XrE5V~xXCbpTVOC=KJ$MG|X@R_LW_K|>ewLY&f1ERcr>yrS;-@hpX{ify6L-0{2B}YZA6nzV-^F9~Ne$<+Y+F(TYvS zYNqbCtnSde;0LcFT3K~9vKvlsw$7q-V;cwZo)Di4LSi^r@A%glpPZ9Qa+;YHcDV=& zh?s_7C)JDS!<%y>^{g}k!oz0KaDH_N{|H8B5^jX|@pM+;LnH;+uC{9X`dkl4WLR-v zBp06}Wuw((LtSJezN8z=S)eHo8fPa)vKM_-;YkV=Bu2iZzv%W5es+hJ`KsV9sv0%Z z1nWAgxshbk1XDFArH>YP@k*>a*uzgED%UM_;7pwO*`1X%c49xMZU_J-#IuCawfK}g z%muOEk*3l+YzBjHvdr<;EcsZ+syqn#BWLbR>PFN&0Kv(%5IbX*@_j-I_)7rJI?|DQ z0XwQ12JS)Zu%@s=^@LSqT9btR;GHBgFQT1Qt2;plf{_N^&8SucyhmRXdf)I2K8EOc z3Iw$yHiT#tbAwgTz042xS1m-DBLXij4?=lI*sH%2nk5`>(JL85+z2chsvXs{zhc>8 z?hjx8irdK3;G6e9%BMY;e~8Z?F7%gd|3?ePO! zJI!UGpZ-^Y^Pl!>=SO4K?0+RUA2z(yS@0^*1Rxc)88j#fjJoET76j3usUvf6e7rjr z8q-ayRf`ajkF_vX(FwH&1WHu#@k+sW`J$nrK^xTH$QQzq3-uFenk?kK#ZU@DewWxR0796dJQ5QgljV}+rkoV`#2nRFpU z#W0`2-{PDg&)oxyQ^$Dk#+b17xoh#@-7~vbI@6e4@m_|fxW;H!-2XY^H~pHq&kNGB&B*Z(P9639|jg+>6W4=9a@<9}Iiiyb$t?GO|| zpsWw8O8cNo(<8Kp;lOt~{UK;9S5ROJy*V^n8Iu{3$R~a)8)d7Iyr7==GH}xu7gAKcWWEsg1lNu-Zk{rzWszhVZ8_W4XwtoJz|Y@Oj>)ndcE zz@D3@DXv7&pT3YUl~OMAfO|ZRWZ3Df3$WBUyRTO|2ke@mV7U+Hxn@vpH&pnJYWgTC zkN4Wf^n<-U{#jaAi3YG7*ttsn>a3x;PagNBCTcp6x@M*Mu{8VGo|gV>^tURk>N{$G zT!4Z&=wreJnu|j)1znIGICIL~TqV$vif0G<9JHw+f1}&Al`|hYJVb=VW(;v9zIL`3 zu_#!WM(JJHg8?xl+Gr4n(3IyBj6*CYOVs>_Ci)-W>4Ka{53K(828 zl0WK8Ze!xYW9$QnPNv`zTc3ras0J;ni z(!nE$M2Mo=?z5m?!`i#+^G6Z1C3_t@B(jp)afY%v1Fpqa)H_lzGEmxC(wH{ace{|^ zr4CQx^ex54nsl3pP5oB0GLED9t;A*$e%cftbBgp!EAzN|vdZ9?`efl@fban)Z(Q=J zW{`x>d&I%x-u@*xFXCsT&4!UBt#0(2Mzk)$k9->0{?8Rs(tUT|%9nR)Lh- z0gB&}02j3-mFRhF-t)To68Lm2&pQbAJkt)_af_B+_o#oRe(n?y^j~W}rC44D_K&d6 z{QrdZi^hLGTZxaz0{@M#?rzP_wyBbYA@JCTp^+r8hGE!~B%tk%d+f+|m=M?j-jyF= zdc7FVF$x49%St;fKd>*^zkmGx@p@|2e_Qk${ZrM(L(@^oRmq{nI(+!q$dDna|FeI3 z)4Y1a+LjatrZ7-ruxIU@-kzO*b&Eu*qMN(t0yP8FmbfJn9#}%WUd?6*wSP^NNisu^{pbeHiZ^6$$&($vs4=Y2Xl zQ31}-tg@1#^bK@ctX5^uekD^`kLU~d^Y48-v9Ai%|Hwy%khq!!k6}m*ZTy@HC)ZE} z6hOLk@myY0N|Jfn*9))W4gaEcka4VGWcXYJ)js%g&H5sx`CpMIirh^Pf+?~J1X^+C zxyBBSD%sQ)+G=n8_qI?7m9x->m%4|#chiaF zpofJ;O$_A@C2te6dd=##NCqQ8^VLxZ9{ZG1KUG(7nKLbvg<{gbSi9Qe0~k#H z^|x2kFUDA!eG9!Wl>g~P-A7L=kbf~HaNTDWng(KPHN{#a18$g@)5^l%YrscYZsyW< za8Iz_4{ZJ*HhW54Qf1T=S`k&Z?=59N2xHl|w^MNn^A=@lUHvjo(B%bzx&%DYA-y#7 znC=VzUu>OaP#jVF<%7GsyL-?O+}+*X-Giiq4esvl?(Q1gg6m)*xI-YYyzhT&KkRN- zO?TH+f9a>Do_pus^E=0#S=mxf_&>e2Fj1T z2Yc+j&13znO+a4g#9^6CaTPbLmYOjZ6W?vhpl*4hkSL`2T{ z*+&pTc;$530DS1M(UP5jA+Gzuw$e;K)gURwqVv1xY3JGC7rYLqTdnnL;TGNea#V1^GiW|2jx|f2X-!62Iez!_#K8Obv1HDo6?ijNL z_uv_oPM|=T`>IoMuCo$o4iv%0Fk-l&?6xJ*|Gjn?8#AAK=#p~>D@ilKWMLi6BNg+E z>ru2SHmGAG%Go%No4C`T^(~5^SM*zKxtf@muC&~@Km@qtRZuPz-b>YtA88l-8%_*QZ2o}!Z^E6|vGn%q zf2H3{x?560AA02C|8ES>hbFlR4LPm9@k5xbqz#D(s7v2mon0jDG`;q>mX|UPKu~qe z$s!jJz@xTOq+=#Q4&t^3F@H&PAn!56RJ(AaNsITbP`Yr#lj-MiMckx`-)0jme*Av& zFYBA+;mOa{=Kz4+d2;IP= zYWM(6gW`y2Ox==ubc_OpSG+iHse?OOPhSxM?Y&Ee*MLx<{3;G7p$J|RV+fl=45J8p zPivT3lDIV7O)|SE+#Sa?mvTmX7+2DuI$W3mJRA8GMail;Y>eZYLCLB;tR-ns5YB|- znnGEtHOw8^u7v@p4|_(o8)5L$gQX-li-GVgocY>teeCOUN>cUV{YW1%3}r^JrsSFS zuzjSDA%-&}m_)L>D*O@qx|dQ&Wmq~=ghE(#xGYkmR0a#jwvw_&qgXT2Bbu^CwHOQA zHkc9yr$RSDEOMz(hK_MTwo%0*fd#3I*(qgY$MPCYcat^>m;f6$wj1JZSTq>o0^Z|b zCEc_55hY!Z@j`W6!^Tybs4k0nmZ&aE*Er^jl#zhORhB5h#-}#sH>#)?%j+M^4-;m2 z%%sU9;*C!VxOtOcB;8kKT;mBaldeBYl(WTk4RdeO$YbM^D6a4Xm`?YVIm+1Lx{5iF znL`)cNNpjTNE60v`7@DOK9boIm`qepb+eF7Df!;GWSr_|$)KEiWBHRt9?HTtpBW{w zWT25tA{8{D7RZc}RI=B|C6+2Ru|Tg&$Rh8g%P*6Pnp(oqs6j6uHol;vOGqa#*~n#- zD$sb4Qevjd_!A_*+5~MxeT5mAhR`i%A{@8^-z~bU2ziP56yGlmEkt_-AOtFZBqZ&+ zLmHz$kp;5-N1SaBxr6?c+V2hhU8G+SIub#FoIRnR8ZrqX87p1Xgf?&pVT#s;I?xhY z3zILi-wx6q{^=u_1_U&|H0~KYc=I?|`!(*NJ~-2H)t^IOkh{=9&cM4s2E#LZ2IV=r zi&2s7hR{B=VnLqu!9M{F(6R1=2xr>R0#PDDfeA{h{qi|;3fL52VsIEAg+d9g*H_5O zT$x|+nnfCs4S5(>*U;V2j0hSS<(^_p1!a4nG{60t!^SQuu6nrw@p0|Gk){NyW zhR-&_F~;_@10AH$7=peGn|`?mg{Q%IFWnUtq)=qP4V}R-e~t@B*Gj~xA6_6F{25Js zTz@b$DdiP=pftRI+%5-X9F|XDKNGYO-4$wJ9=c1!kKi9-{j%g?KRyI?fp5@m2nQJW zQvHg%r@Pfv>#O^Hgo7k>eD-;ACv;`y# zH0bNz=RaE$UKG2FIW9Dsb$hII2J*Yk(6!biO6{U>Tkuc*bel3hycq2nB3}db5ngk) z&O$_-Rdf>4hfN8Awdd(`t>01 zm7dUMX95?W_#neD%4Y&Op9Y{y5Bf9CgyElbAot;!DSyygH$0-dkgG`Ssz64;6X!u4 zqknj0nxMX>kV))@6Srrt9QJGae@8V?B-wyJqn;c2E|#D26hz#fCii}{sPP8z4iz{F z??B`7HSjBda7I2SyUPmsh~A#pp9GytdBq&~??2T8O$Zk9qqrb|b)!y^DTu@?wM9IxqxyQ8Zs%u3-PBw%9sX6OuEu>UKLmclJhC8GF>*9OsPZ{+Uhn> zgCp#{IM z8h|5xt)~DGvqm=(q@mhyf9^b{i#o*Awe$d>7;UX%2{hf z_Ei-!t_o~XnJYTpR|M1k1G+EXxF#~{3FxV84_SX3S5_YxdS%uW82osJGMO^v+j7QS ziWuR^I+YQ?wsQ7Xu+k^c*O=l8_FW0bS@pzGFB$C{QQ(V4Me?Oq3Bs$MX6EpFkzBLX z0`^h`SB%`^b6ceceV;)JYSAC$W@}rXKtxDQo7n|SE`4aWiUl>ZmTd3xg?ajTKx3?} zV!`gMhmmBgOXI}ndj(_7VO>QcS{YjnmCCHLS_$m;M#g7VUt7Tz?wg#-rQt`6ehR1$ z@wNh_q>E??d`jW#P9^sK3Wkqlir#ZY<uykTJU%J!iG8MUrg2{cOmS% zudzM*QSBqe7j6_~u2AvQ;K0A>U2AwZ+<&Q-zA|qUNzl&gAhtnV&x8vTqPCt;Kd(*i zftK@kNbbkX0#(eLvqhD>ve@w14~fCG&M~8d(-(j9*$5r4M>02S63F*tfeR6M4A_sw zpYX0m$EH{eJNl{D|wvBzEk2_aNg5R@x0v5LDJoJ2wUp9Z}}y3ijX zg>ZH1o_MefERTxN>seilHX;_W3->bF$2eNBIb{bnP`NhTi-~O_WK`Vc=xLP%B*HRY z1X3IH6;vaNV*1)z0eA6rYjH2F+s0bvsqrReR*>rv?JAZ`_s%VBAbhTvh7+#Llmg*= z#fEV_J9xYB_5f`=7w%wD$^E#74a=l*!C+_%u0|P!rWP3oS$pKiZwltjIGTxm__29h zxwm>Nbu00(c$(`Tv*e9be&myEa-KX~p$Mt19<{clc2j{Le-;-|TRqK}xLfJBwAVW8 zz)<9RN4(>x0hS;ygLZMpbw7r+0m&YfrS>Tqy~S=9zTb}tLjnh%(leX#Oy{#vj5g6D z{X+3q41Pk)QN7(CY`L1fnTXFWDi6nnw#xJkT(h2|6!m`VfC(_mSi77)Maomib}9uE26_zF)W|xrsgHTf*bL4BE(&|=*^>$zhY@LDPDH(`wX9Y(THm}QQLNa! z&hL6ZxOSzl^JjDo?b_)*hh8>y?KQ1f9p!1od~z)$e*)E#^@;?j1aFr9;eJ11h^u*N zau*EP-l8tvn)$QwK+D?nGDzbHl!Z>^;(FAxfaT;R0G(}Y1ZQf&(sH)>%y5~~ud^QX zKij}H#mvGjibQ?+<8RG#=47xZ9X|s+_fcmK&-26px<=fw1hPfZUKsHd0vGGpZ7oUpc;6`zlV9m z@)gh4Yrh4UNi|Mw3m7s9zFd4ExK{KIcIqGfMV(_pOA;R%vofiWymu!nvFJxW;V}8I zO&hZ?tjv))WoE@!iZ#Dm`0IhLVy}WE)jd|D82GYOSZMBW=dd_x(j8$3+k~wBky=}p z^T_GcgghUZ^4TSIxlIx(hJxC1mtK-5N#$n2Mn4>)3~5sI(CJ^Jr+Yg$X5Wehdt>*e zVp@_;R9k054L7C-S*(XHCscum11^+mLnE%PRtS*`bV*~W*Yjp(Bn;(f*bnu=sMW5BskM;r7!;=y;o~63C*v&tuItiXsB{Zk=`Rc2L~g`qgM)8= z$2Hev)1F}G_PeattlC$rx1HEQ+n@(Z__T-n@Y9n?SDyV41?x7Gq&lHi>c**^-#bf* zUa>g~OgCJO6>yZ3>;|qovEu|Kr(d+`Pk~!QDHH|9|U#LnHpyAF_Vu)g@VOEGKbNU&S;)AQ+lYg?Gg z-jgQ-7EONUd|a$YM#tVo=z-)JT^o46wvE@3Z{t$=UA6P&v3fzU*!X*!CUcv_2LNIN zP1CPaY@8!^v>6OR^p;{metr%Ig%-;kt~C?YnpLykBve?M&4x;wkTEskp{IptCaPGu z{?Vf!(RAd}r;98icr>0dwhsg3ou0->it(btWOl9}IhY9NjIcNS#!JG&=p1q=8fq&i zq`oC=t8#mix8GzcVUjyexmLpN{eha=pXT`8YgTd+JdO)BHoBy_msFh?_Zi8g`IZE! z4Me}vOL}T?KI2#7&4Cd;qnIS3KC%v^$ILe*of_vnl0s!gk=yjy;A5RoGoj@J66kHU zevRXTVr-9!1zzu~lpC|@894A!pYNTq`-=X=q&&{9kWts1^3cbk8~Z5=x&mVB6bZ$w zkE0^h5u^%=dBlo3vOT1Tg;D>qY-9AOFaY{ifw{9fROny`%({WT+7^tWx(9AY0gNM; zRy{NVZq(>yv+g3WLX293Ev;05(J)@6^8$Q}Z|Jt5KnMQ{`)K<=WTlG{Nx92pO3=CZiStnQCh-~MTL^oqhCD}P{-)UA!vzcxn4kJD>fB)B zxSwrGB$-kcF|*ieshNn%@FX7yRsDd%3Xs7xPAZ77s=`50`Xk3d?OixzCV#8In@K>! zs)8sdCr5BxNFMo698?-B={UmhW4_UD4a&*c4$ZU|pEBUsoP~u{Ro_lvFtxpIo)JG- zmA;v#`gdzFA&oi3PZMD7vEaC0QR(nWzE3r7%*K$q$jN5@D8sM~U%or)k9@kUxo@7~ zLyf-e#OSmvmxKpRv6BvW1x=<%*{k$QaIa!ST?bx^z>jJ^8nzW)iJH`^4`(BMR%zuq zeCs}cccd@=MOL&_otCAC=bb-f36NN7xRs)vB}zb>ztLVVe2oEm?BzyEgJTu!9I>Wc z)xDEg)KIr1Hof33EcKC6o$8YA>M6X)ls_DG$%(uZ>tGH`cU$>|$Yr?920_An1*9*3 zoTfgka_iiQemZv$>85YSt5+nvrx5-tktTPuvpd@kmnDkm7TP4V>vung-cS{Li@CtJ zt2KCcRSXKE)l3KE#2{83-^Rnj%Y$y@1LywMKOpSBHl3*N*_WtfV~M2wvMDv`2(*#3 z>jfi99G89Z`hspaw<$JS10B6no3G`%eBs#(+O_f)@GXk2DL^T>8dX`ZOWEY&`NNRw zb6=E+D<$9Zw_ifAfid-{7`}#kAHPI)s=U)E>QjVMGi)#rQBX!*S@3gOLkdUIp>t|j z=2ES^ib288jt?hzZ&b~_mErSIW{)Um0zJF_ow@VLC@oN~p>J?UR1q-0Ao6kB$d}+s zZr3059U`(zle33#xk zsTlhN?IW-hVvb~+ip2BlMc?BPIb1l8xxrEKC?(G;FQ`!0Nr~NTb-To5v!U4WP(aLeF4S)qXf6ScG^$!m?97xUV&RGxiv%gfoa^q0+al>dCG z9a(8+t%SbiX41WKkEXbDqFY(&XA!qeWwELf2`>9!ph;_dpu#k9VRxt5LU5vB2O?!D zEDN{}$DR{25iVl+y@OqjW{<5WNbHE5??}`#EX4a_Ti`k!^phcuRvwpDCMNmhVTcO# zT$@urm0rpOIX|vJ)vcU(R_#Td<(uBW;D`fRdzXUSdF89BIb)WkElsCx4m|=r(eHMj zTJaI2h8o)sA~Db-l$+3syk(n&kL<5T{5t`0g6wUb^-Xr1n2}Pd^@FT%cU#&^?8Vsh zf1b`6e1hRezkL*Lz}|}3DB^mAE-LiIaVdHt9F(lEES+BmTqg4A5yHSkFwnfdszghW z65Ms=cb{Sm=Km7XKf;H`Fok`m*qnD2z(&3&{`@asAX%H_LB?BX^mrgEn?#Phs=Fbz! zy%hV38Q0Bl(460swD&gmbV7`sRmPQ+Py5M%^*$PmI+iCP_)6aY7NleTC%=Xpjc`cAWtzm8RB7coBJLdROD4Ezk}6w0b5f|2D$mL4&;*E} z0D|1Lf3AfXUc#|Or|CF4ChVKFU%zvHzY2VB%DWRZ3l7ObLM!%2ItJ!=pkg;L5zKdO zmv^qBiRjLi(wz8U^r%SlDc|yUq*vE40QdH3;WSdOTT0uN~r3-EDs^zkXhEBYO19R0bF+9C`p>u0l zw^`!#p59*^oFO)(-D{uZ_viNfq;$ z)$)WSxAXu*Bm73idv4h~U^tvlyDin{YY+c&j|a1#uz3z3IeJM4_u3?Nl{@%%Z20;S zzXwK0VV6v@Fc9)ZY(EsSi254*##79Z*V2$*vtetljT(jII6uaM!c3vow6L%F3uCFZ z)E4SX-`eG=E` z)n(3B+hGSB|M=#iwKyC3(se~Z7I9Px^OOh-Dmb7#N5iErm+#bUEf+>dkkL~cNi=)o z7Ba|}MA*o0QAzebCt~(lh*{6swIn*6{}cma(pLyec4RGlmt?9B|GEr6q(*@>*D-ml zak-C~+UD6Tev5I+GbQkBEI6u2$*$P;{r$aTg6#_$(j|h`Jz1^h5N83HRf; z(47Tg?%w4cK)wr`A4FHcz*DcK zzcI%s$JKA)s{*DUkbznu+ic1~X1QGlfQ=F5-endR^0+e*NQ*bAwX zKOUh!Q*&q1n+NFO9)E}Od;>cA?K}KnT)h@FB>s0zWnKfVm7iicXIJMs*01V?5V2P@ z;~!+wVjkLj1+iy5%)d{=q{?)sami&=mcYEy6TPbc&XCt&6?E>M%-21Lvkr8>((@2r zESWf($QP`U6|9jq%n@G9kv6Q6S*($s2ymCqAS6$AB+9MNeZREO1j^5fEL@L$B6v>@xFlFz_?4bv>ecCBo!E?{`} z8&yk3JC_Bq@oPF0TbroeZxg5z)Yj#F!pk*?NALc}Lq;|1a^HREm0~5Y?$aw)p3Yid zwrsUC#*YW>hPiCxHQX-m=R%Ch@P5i!q&skn3VIitoemh?^|nePCvgpf zly9Uf;Z9F}Ic2p5YY@t3!PhX6_I7aM0XXz9YkmWo@mnMLP~W!@8<$WMe*s%4b|Q=7!!_(NTLQXeVOTB0LcAt_1a zZg8)4Nt(OI^8apOzF+(Zdl0R)m&L29?!qFq3uAUTF&XsKfSYtfIX6-3IAe&{g>%0I zIm+apRMkfrMn|LIsY^`1B#e)LfuY4Ksz21V`awIkPtRqfHX!>rFg z?;sjpp|!)kA-sb&OvxO=q#N!T`lk_cu3$Zex?IToq5SZJk28>8bXmJcCw4t~$hJ*1 zuWI4YoDT3tKY2J?FYty*G~Q+a@g%L4096Ur1Xg9;|Gm@5i+mSo;=_bo+6(M z`BVNca1ZY9)jaRJP#)eHkeD^&6Jv@tb5xn)5hK_Lk-9Mu2-*qZU9s19bA z1GZ-YTxpEo%Q&jkh71fy>&nt03pKRW$rD7hV!p@t9S|u}b)l>`p~NstQ6N>hnvL+> zGRzE>%+zI#)WPuD)#^kKHxEmhZx}Lh0B3ZtBuHyIucX9haQNfjhkyQqHea1p37K)D zeZj$Xft7Yc;G7*P)oeU={hZ#?6qEUv9U5&^!_{_^$+iZuWm)s`hxTJo6Ta}d3;TDY zO*rw$>ldLnKU=)6$bFuZbp$pwZAPdJe2;p)rhBm(Zh5k^@+B?Vu^J^uPqkCjC!jbO zMcEF=_Y~?{t6HEl%5e0m-Ypq4;EC+88QFnI1fGa+hUj^AgVi)LZ;D%x!Ow_qJ>92l ze*dfYx$V)r!QRuU#eKo z%%*&U7U!By2bADfyHAI9@AnU&t=A0;)S_0HMIfX{zRqrF={`OflrZKjaRV%>ww&0%C3xa&V0h94^CDYz7GIgVU-FH4vvrOuFD(9D+?os< zDJsFjy~9D(_2zF2&UN5K_H-3A1?SX!Yx-k0v<`=pRa-WyzA%8>`u_LrA*4A;WTb%X z6GEF3@J5cjizZ3|;CH`bqz{vYcD!=vOp%3^ydsnzDi6B7VFQ^*XFfL0QWUZjK2~== zF(E_yTL{Vp(X*GKW-Fo^t0T2^(iv?yAazf)>^@EPz1yxm|5t%u1GM^&8&&1~|H}JU zOyB_$$!}OkSKh9_sh-bA=cMuNIWj#O$Gh-g=SKYy!a5+m5m_gqsK@0~&!T}(sBaX= z61Xt0SoooW*}z+I#efF_7=7{cfwtN2XSrC{VUk%E+^kXVaW48pq0<`x;`tR&nc%pI z5-m5%dG_~6I{=Ubef}6p%=8gC)!pKHH*$9FEp%eK3444(^Qb@3iWM$c6&}K8K9Uc+ z>C+8BHB|TMT7|(a!t4k)A~mkD8_J3;YgurT0^<>L1D?NVeAPG>Y%my5^=`4uKGBymz?N)?$Koy&5(insX`zY3q0SA~1?5#Dg|hSRy5Y{DTG3~x{__B;Xms8oib9e$;F*Qp zcoX!w*(&?ZSOrwNbkZlXh^AN8N3!Ohw&iEmR7 zy{JdLK|#D>xx?C_DpG%zge}65%B08Wp~-)bPKWrO75HTtzfm~$2J{7(;6aKBIg$Wg zw(>J#sZNFF$7v$2Qhiu$Gpd%zh}Bq;Q*T^$>1iuYp!)1l9j1kfDP%aInB|CQaV~mavVn~N6KS;A$Nr~SuC$ofkf4r$G23?7 z2#ZcLv}*F}Ljbx?XXO@lRSx#?H*JYr9| zNZarQCQvwuteh^GSJoM+ckDAae`yJwQs~=3Ya)`@4LvV<#d_t9%*#!epP;ikIT80Q z=@_JOXm!N|o-_kBe5j#g>Q5c8j!+MC-N#hta4#mXjF;ffYQNn={`r`G%_zD}f)>w2 ztuJDAe9aSl5sbC2ISxlkPd~;ds!x0OGu3l~5P zSMq8`B1~6%UEMyKx5EAXJ4vqT@A!qEwDXg~_a5~C%w2=st8MZg0eyU~cue09H)4!9 zQ3f{w=ahY%MJ?92D>aDsNP|ZH<*oujUrXqGxLRRk*eBjX?K3sMTFf!@IjCzg}MQl+(8Fk`{JBfmer6L`7 z87vlOkLXr;1RfWkd7j(Z3!)q&-+aQmsoIw|2nZ3oByC3= zYVqbZI9wZXI;tnp?`$m`_h}tW-Qd$<%_IiY<4-kQzLbgvn)64y5`!9@!DXX(ipIv) zuOt197K|5=+?ztx!Fya zsAD!JPgeleo6gP$GUh6p=GZM8E^*&~+{?kt!geuI?26w!G_`x+Sa%TZuASQb@wy_m>y6qr z*ZAL{zQw+nhVMh*AFd+p6Qc!rql(nsxd=?~vSC~sdI?j9juMXebh@w7;`O_x@xBtWCzuK~h$f&`>8N%jrYwwaCCc zGm~RCx5&G}M`dlb`|)7@BQY8$`|X&~fRI9PuF3YQ8hg1$EgtCUK*u@z<=gD-_r5Q5 z!29i3-Y2KsH1>90cGSo zJnJ4PgacupaG9OTRCQ1sWg|VMK7UVN!Tt%=ose{LkxpmfN|1+JU__hHV{;{bpc@v; z*yaht*#3&TBpe4tcrOqHK>1hfSqqq%h$9Dxb8LCE=dz3#u+p084o&}7TCR8F;CCdA zM9_C4+#hmBCb`}V4d^K^Q{0*;`8avf5%AB-qb2o?uw7J1*ql^BMh4)AC; z7KXLZe|fpt_T*bX_^i{XE)Z<&*%HEthB^7x>a5R^i0^PbH#;J1)zgpmGZ%4h}Feiw9M#5UNcVQ-e-GqUins?TSS-*41uri@p~|)tGGjlxBEkLevEqe%%%t5 z*!wcs#J#V&vky}Z25ynh%Th9aZ3$MQil1iKsUp(0xqSxC2lC(4qqICI+Rdlkn#i}J zdvDX(H+`*1zQB6W!rGc;J6CiF))k=$7fNHz0~ZXf9+l{&G759SYq{W~`; z7pEB9on#9*(ta^K3^{$nT9Wy?>sA<4JB*;$#@PNBGDn))e3iUg0@>sf#qe*(cCfv0 zwJn|h%a1?ybWnG-iQD9Yax3(dc$3^c#c?H5g06%nNoD$450zwuX$%-S)4Ea_g_DtjJ-B>|KgBE=9YY&AmO z3h3Jq+R@)#!6SO%li0lqEE)bW0~TBwQnM6)F&E@v{`JPG{JKN(ApW`eRf$A`L3kHc zuzWxH8Tyy)2Sy%WRCz6Jp{HdX-FV^?qoY=9dG`-gyWZLsjTU=759VxM`Ge)$&r}{& zaeUkjRL^zFaT}FU^ZZ2W8Rcf9En!f{FT3#Q6MGrtM0Uu{x140t8#S ze%DrOHTJDCiPy{UzfhJ+f1{!2j_dipLYj2OAe$%Bdn!? zUF{EUV+;QOG^qdGpFVv`6Q6~k{13f>kfs<3S^M7(l~f=2-=>9-)*lUtnbzO@!4*2H z{KwoMJG;5HYQs)n*tf~&YOXRJahov+J~tAF3C!IE;`Z!fSUU>>L|-|4)dZfPURA*x zj-ob#2iKo7c$0W{g!7C)b=vUT@acHh@0l#Z1n(FAIAmnhsVGEED_FhQ{7l^TLY06> z8cDw^Zm)rThnp~it(IwAuJV9C0njn3JtUi+80^S{^o%8MJ`?yYY{Iiq50m^Ee)_!6 z$#prpr=lmXcb&rCz!jE3K2-Tm{%?$vZQ0Z;Lg&4S)Q9RsAC$@ifrllkP`f~PVNdnV z7%n(MlRuK6^Bn`6L1gT*EoioyK_pv^c>AlC<{xzmsKBh$5u}zcIT7DlqU?g?K`+^< zS+OzWn&12wyFkRe*atcj~a6W=V}{T=z=Resr4y8aKzWCi)FKZG$6=j;>r}*HrQjr-C_(& zBXgRDpi2-?*ix@ykH4%2!Jzi=s_;u}tGyUrh4`jykfU|XtJi;6F zgRfW3_N>k>x580Z>SqJgbTbNDG{+l5tmMVFuWi?Lq2+L;B^ru(X`tBVaPu# zY?A1|Y`}^uryLxe`sst(rl{EO@^09^q)KL%nk&2g`8kpe93>vU(NWKkR~;T6PP|fi zx_kOod^W63)dVH~SrXhz=NQ7yJ=e7>ZY=-SyBBjam)vu7OUL1Hl4_2nzLYg6B;MJQ z!~JlwvB4dlO6yS4Pp%qSG`vo*#Ultp8XF^N`qz1(i`?_N>MOrUwPdR-H z=iKNVb(webn6}dlZN3Qbz#_AVH&<6YM4UkZ%q}vI{)*(m^2FLp`YUx}s%gGEwqPVK_%s$DFo5PrMnQ;2|N)3SM z&2!xW{$##Ek+WnG#%{eb&){quDQP9ngvH&H>d*6ga>$nyy%Be{BUeW{acP)7N&se7 zJ7Nr16n+Gkb3A2{rhk}IrT#;md7Xs~?{D5fXQ~HJGxr*cUYGDmIO}%FaQv~HaDC(| zgaf}MK%{!)@MHKNmu*?5@vdkoo?6NzAN$3TNk6u?%nXS zUM$o{*M$4OJ7-!$JftL`@9m!~`4>1^qrN;lLP22jl(fX4K!?Y`S&|Hl{{bya4>4%A z<*-eSl(khKYByx0G#hK;9OjD){s}vUpjH34J*hkE<~#kh=k`&+c`ea_{_*)=!sXZ7 zsapZg+bPaj0Z!T6=S%Soh+TB)2JA;U*F5Y;`3BucR0h&`Fq#@5KtE5xp={PW3>gtA zQ6`Zlp=m}I5loGm#Ag5=z?7C_PT`XS+eDbtq{d8EX&A_&*`~bKifEJl;Pb<1F5;iUhoxzRXs=x(l;t)VT+@cjXoRS)bt7hF z{f)r9n0+ZvIm3N&uW7@1G;h*ifaZ;ngq%c-UxlA|A%TtrW}!eP?8DW{qrJFDh5s+T z4=Bi*v0Z7P30pvaz#Usad*BvZzUISYx2tvo5Vz}gtzf=l`-=<+(?8`0>0>J}u2$@7!L%_v#RnNG z@kuL~mz$*zxG)bXfCxdl$YUH+R^{g5pEq|ycEZIC(XopNM%kmyb^7bW3lb>$=Vn~G zkWxSI0H61~MP)QP^A(AQuKe&M(wApyCP(~1z94q->^*PR@JLl;bNpDdtIrB%?b9%> zPT^5s4eEpBnIvGue-uRQX~BZ^8Ad-p5m2(~(8PKT*e&~a>YN5~4kLn6SozcytkS_j zo|~OY27iK@hWC)A*)=%1*qN>P680=W>iC%ifF6Dj@+3YcvNe15PZ!*wHRNOV>`52W zp=V?~c7t&jjG+>eT> zU_ejXzDwD#4ucn^2l6vG_=08VQlT^G5`@HxWr@7fx$CAnBf{h!imwT8FA;94Nl}fn zh_t8C5Kl4c!Z>7$%!@l{Yxa#In}wArM1GhM#Duih2BwD1F|CMH6Bxb19$G{upvzxC%V_(GTsd8jm29SMViNNk<%OW{1!)Dp{3Vq>rjhvj>~filcaQ zA#E{|jt!OLHE52|fE{t-@Gi@JAUX5sNi~6M>ITV? zqz;crA8kotZK4rBM%dvOMxKUr)uBWY`}b?n`YUKLM-X-3vQ;;)6^ zEU++RFU?yOctgQY`c-_yyW@^@=mnXK-GBVA5O`q>p7Hx&iSmdN&O3Fk`ZeeBD?!$i z`?WGo-|jM-E|>{aH^R&Kru!;C;#^-BxpN!+MroveEqz@O31HP#0Z-no>%#bN46PMU zNgshUfcrh(HNy~IC6#e^1;0VF52|K&A%EnsG~|uIK)tkxu(CcS?jzS9Qox)UQ*PLk zP4!d`>P@7|h#&?o>tEoL0U%H-JyeowGPPLTCVl@yxRy8Op4zN)>>!#%_*dSvKc#D! zx$D->0Rc_^y2ZyVr@^<*14i9{%`TR$$-YLzqqyy4&qv8{IN6;zR<+0;Iyst9(60gjLi;H3E7CELPm-YT+GHt! z_7XFteKkS!OVL_7zYu*|OM3Xz>&Vbsd*4JWEZclVx#8e~NJ2eoKmfA@)T@iq^7cU5 z?3?39W;d^;S#5`;+5A$(AxKzNUS}Y%LXSB%zU4ke&K4!1jkOPL7A>LA_Dc$sV@lUh^_`Ss|W_zgsYpX|%x$w^vOxyrB zQyDLeo3%9k@a0L3a}&#aB5w2o$J{Q#i}Ui%DkmX{VI`FVtTE!>BZ!G~bMYBcvqt#L znZq}$i#KvzJOUR{9w*R*D<1y22uD+^K+kz)NYR(CK2w4rzvb#Er&pPgfe#cE_(8>K z%Ah#{>%G4bR4*HRMjZB&IMRQ<6jNjBXmMnbi;Mp1rMJ2LUu1n_a3$@!_DpQsw(U%8 z+qUhj*tTukn%GVzwrwZ#W$*o-Q}ylBKUQ`1^RH|5%?l&pDAjdv?V62}-z2KYlrmXz zHe6>^i~H$T@gmFdtum$pyiAsr1ctqr-_2jYSQM+--Qa<}*21M;j87zLPZ3M2Lh!UA zFehARIDAxo*~Nkr8w7Ix$(hD&h1=#evl9U9hxs5*K*4!oXok?nu2D~KG>m-4HncZS zz3VmCOWktf7T$L-$qtH+6HPthG^=r?GMxYJZYY@#*S$>U3{d4=oL&`dPPbbX-708c|2zqyaTE4e z#2!mZ_LQ)l9^Alo{o;1RsQx?p9P%AGt~Wj_MZZ#e-aPX#D|T*e5x0|Yl$FGgA5DD2 zmW}y){%Fr3WZ_0R;E{?#|LV!YyL<$!sNOcEFW(Gm5{Jtp>lvNqh)@pbWIdx?8lVoC6HlWLDaPxyxm;{+;l;{NLx%^P)h~X= zKm1u-qA#ne%#dxDkZ}*@N3p(Z8s5LWe>{ZD@TtbLi)QgKy!0ek!8xEV`9wf~T##!t zvpJnSOJL)svS5rpu9Ws*-AAJ(~mVf~HVClr(yGo)M7ptT9>Ly0(yXem?PQ zXDPXIQc=ie)8;vWA36(9CGG@N@+E|UIX{X21gC?!x8}JGvv8kCtK_xzeQNcOTo-(@ zpM*Z)$U*QRIpmq&UQ}Mff|KwF zA98LW=w@@OK*i0q?J9h^!jW#+JPkmz53Ni$xnY0G&@c* z-S^PoVWUVDW26$*u%|NThyWA>hRn-UTun4?*A(_*L(}$VqrfV=-e0whDt^p;(7)d&=d)Xq*(kCif!7~n-ttJc` zC^gJvI$s~93%2@Id9FPKvt`R7K_RM6MSL`i=21_RB{-_RRZ9W-cASecLjd`-2MFzEHG zF*jswu#*lYf65d9&MW7{GB1ivSGyV}8H1MXNkP`+HmN<}4^JR@JqEIxx!y^i9T{HY zm7Z+5UROv5~R?u27A??s>yJ%5z_{> za9R_p9VL2@*k61UXS9pUsJEj$jXRbln5iqo5?ZzO{AdkOM9rv_i`w{xJ!G4X9bL49 zU%8+bk9eo)0zd{3UHtL!!bvzd|7|}_P*ntE7Ot`9w9UsJu|VReSc2e{*kcI?yF7OK zE1Cay4HFZ_u287xpCgUcaoSMn&_^W;) z7P(zHL%&=sBi5>@3Co!tZsVn4oaviUwXhtMY@ZqUEUS(B?`S_~We zJXSLh4D(wv&N&L`MTsqXVNbcl;}goVX^5cn>3Dj6#JjVW`)gcLihSQgcQ*lLyTV(5 z!cMelDUv8t6{gz@$PAARO_b?T8B>NNbCzrT0QReUNNKWsGu8W^<1g!H%na(3ctCOy zwX3RL7MaG_hyNx8udU9NElm1z?PSZ9$L@}^*0Z;rzNnJwujeA{4pNH+id!2)p{MDD zGtNj-D@sL8FlVEBbkTExSku+HsOBVqTag5`(OZ^z9jW-yiQMApDTyy#yNZ3X#ls{x zm7_^P+AGERRSLZ0{)ae;?OCHxHtvpLiF9k8-e}QMecP7mg7I`f2y1Q^i|yvCOp3^d zyb(u&2D3^A?qS#?QwkRqSWykqQ}TA#JypHF)QbI9SDHgt@*oFbs^e!l7^Mr~oRX>H zlz?)|Da!09f)vZ>R)52LBuuQN){y@`)O7Y%BI-6QyZms3ShT>z+sp}>*53niy3jE5 zcK1U&e4VmOfypu0Km4Y_-k3BP<|%xUIoA)d9Ly@xV}1ts6r9)b9YycCRM63@v-+J| zU$i%C$e8V+8C$UyfRawZw9pSgn9l3O?vxWXw*Q7#sb!ti&kNVn{l{ zdZnIxk2LgtUCxv5jUMh48H2ii&EFU=H)MdB_NV^w3-o~hwE*c(@7pC#sXq5MJDhab zt>Lk{r6+mWE#ooUz=vF8`@8bL%?`GG;}W>ZtwcmqyfhpIFS5(4Qi1`{QB!(iq`c#3 z`gGIFRn|NoT8%8fL$TO(%gu!>y%(`{Z>cHfH6u}J0_QJ=AaFGg1<_ra_X|$Rlvzfe zwf0Zb+N-1+L2K)=nL=;O3T#5qkl~1D1Ici}r~EeLfR+t%%14|I>}k{aFo(UsvNWkl z9Pili)+7ID0Ii|%LZ|^;1UQvHT^Eg(uE9#Uud8*+;1K9%Vvyz4cxT!}ulyh`tIPEh z({E>wE#B}j+Vof-{1Pv+SL9&-d{bRblYKGslqMcy_<=4+s10J<7KL`;m?kcz2&vXD z*cLJD0UPban|{mT6=lc~UFTOcee}9XVc800Gs(t?livbOcOMP7MEM*c*_|uGs6)+> z?Pu1elif&)NWc4HjKZ5PU(TLi7)iG0A?cJJmq`zt2ENS`TP*&mW&=}lE|@P)3_peCryNoea937yfyA(v*;CX-iP0!7rkWGjcP+O!!Fdq z9=xfK3}PU==+^dZ7w>G}_v}!YBe2KpH}K*F{N7B-Z|$FF{+>>B=K)Ji5uIJHG2x)V zQ0^9@5c3*G({9JG6f0z+M0LrNl9Yl-z8hq4&c6mYK@ArGwe%KZ@q}!4e!C_f2Rb2} zB&*TFw#EybGMP50LtRf2grlk^hgTo6FXckqvnV=(J|5Y+r>>U16O1YET{SukArz<< z5gpJG9eJCj@tqsJ1GJl#M>VatV!Z(9evTNDKIC?t9W40SCXEK;@Ig&ABm>mU?u;y*K<&dHvf1!;AQ zl++LbZ1P(Nk7->~t%Hv@h9p*+5G6g*s+y2n+N8>yU?JOX-Hybgl8??*=g1)eTOb42 zip1;rv>Q+}Sc?S@_MJO8M!xLM{o68P%H6=-z}{P5S3glaHa|CkDYZ4XgoZEbQe&yF z__&)uyWp?|mO`+g1H^7Z?mJc@oAPZ-8D|s$XzjklFsv|&ePHptEmiBq(WB}~*$mS= zs-C-SZaHlIX$zqw-u5BuYZ(hRKGvyD&-Ks4&x7@P8HMiT z$`}LV+naR}HmA?xTyZIM-y7R{&K}~J-xe`F+y+hg{B~|{?_|413D`*~3)Y+?R|ic1 z67RuIdFn>e);<@od={CuVS*@0WHyYT&xIhXa!++Cf`^6S&Uey$BVZI8)X%%t@IZ?~ z%TVLlJL!zhZCQ-T1MtL~Q0!{nKdCw`LQa{T?TfB>_#=6;FQIdfjHfm1SV6lM{hk#u zlaAO^uF>(7e^%%xSf31kE=_`Ftw-JiyajBcJ6&>W!}mvNj_Zper$8@K53y9WAn6mN zG7}F`M9PEzRtZ_L20jvQMi(ngE@W792A0l7}my8Ch)Hg?8n*8FQ_R z4^chEY|NX_!1hXk)lQWs>$|jolYogSlDd%W1_DUAw67Do6BJ+{-C%J;`9Q7J_}`vE zL!NH*RY~8>fjW8}%9OoocY529w89lcSDH|eVchJP5sKH$y^G@uxiM@&seFJrS%?G$ zxp&&#-X#Nut2|O$c0%un0U6KF<{OfxekFw9QUh2XV6%b78#o^D-2QujAx0O$_>M?D zo-Pvmz=S$*?T)wm(k)Zh(F=+$JqCy!iu;aCG1iu%<*tnhY*%3x>^0R)pVxTiP0t;l zcb+#K{072MThmAnY*&XKZBd=CQ@wrQsl1eWw@+s1eyTcDC+PAD7NN`lV*RqvZ{d&}6G70A62a zUeL)Qn+Ko*gH9VDO{Z*}&X&p>0OVs8_B|Yc{PsdwH z_Fp27FjY4M7(sTJk#5}J9s#-&qkO${zf|< zI8xl5I6WgmJ7zfacQ^98g99g5YzjCLVqp^QMQIUY;m8gR=7W@mu&NPG_8A=Lm4n6V zw6+7Mdwf{{LcD?WJ&7z4;tb-R2DYukgQc!p?{eskuE{{tFp1%=%=|%$e!v8X&CTN6 zLZE=(n7-rRwb{$P*yf#nE)Iy}KJzSSR1WwG3iGLc&$LKr14cTsE3VtLZ+Xs1HRKK# z9*94eK|i~OmuFm@Rb=|rlsinxmCabRjaa(QGP|JxqQ)Kqs$RpE+rO^fAbr@4&sX(t zx>Kr;i<)v^+aJ6ow;xwMm$AL=OwT#_s?B`pcykw?*Q|N}V0qUXpK}scAFpY9Zk1jS zTk~SW_WWsl%*tABWK7NBFaE;_^}aBU0}qw+!sM)qZypYHD>p7XW1I2^CR8eMRT7M1;r!SiawRl_xeqxz$l z6UGN!m08TQHzS}^pt1G+jMo6eTE{))VxV%O?xpTdaqs#W3}aan(zqbH6E(|VcbI>p z2x0M^mIU1_c!0f@0}#`Jg~gWR+rC$ijq0e9G*cH6NrXfB~B= z@vy@?4}0cDaR44uGBY<*Co=G3Kv(rOI{ZPd%&R+|*?D?-c4=L;aKtf20Ib3Cq&)wc zrF?NVGkP%{iZKyU!bBLB^^%qa{WeO$2~-q9Y)(QIf7oPV5DIrtKvThZ9cGo}KFEgQ z>)$ff6HCz$;ol;xv!MU|r6k_?kfhFbz=Nh<4*`Lt^0vXF0ftm%l-5lVd_lml3xi_y zk&Y`%Bs7gODG(A`yo10NSZt#u{Ngemq*C;-{o`;XOeM`cPo(KD`A3^mwKLNVGfYEH z%R|4ywBPA7uExQTvDapCJUxoswwJCNJFh#}_`lv%PT)IaZeDD1j(1iWixo+47P(vzVu0c*4-2zX9 zlL)8n(*+DL4BDoDff>WN^v4>6STQlRjoC-;3Snv#8Pgbr>;hxLGf?j<(Tu)CsqmQ{ z(#Iv;lJX_w8)}Ni2n-}ep-xWBGExo)sxw89p06V&YHTSe z)mAF3R-5MD=S4N$Rvgs^G~odBTQEXHCpdXgeW|A&0BPd zti~tl01H#;yo4Iu$TF->8Fc7S0dB~1ragc2@VLaYM}3hu(M9K(U?f9r)Ayqnr9|7y zGGiJDtwW!Zi_e@?T_gr<6`zS*Xx3GjSV#RkMT~SS9 zMgIt-aHOn;!p?g|UJxzjS4{NAWrEMclb|sp2pHGS165&ht;;o4A#-_FNsiTTywS74 zqnvS0UL$u_ayt!pPf*nrKao5yDbsxrVKs{@>U^b(!IQ1=ojih4DVQ!$nK&Cfk1OWZ zYF48^32!{>Tp(FyGcoO~%T4`;y z2S}N&4_)%^k<>fjd{Q>h<)Jxm8C5!3M8Zy*71%YmvY4Qnt6w@pyiiP?O+jyWjAxMm z)Wj*n1|sR8Tut&xB2C8^F%EhXk-VrWlN0wik~*JF84Ho{%=-S>%x7gJE$E`z_kPqKwvmV#eP{!*`9Bo(Tdcx39D>JxK@~o4Goaf{*M%5{u~l97?npWk$@^oPCpYW4z=!5tKUdN6Oh7C9jMG5+5Np zt@;P{ztVj6w?}5cHwuvw>c4|m>c7+c|3ohofQAx^8p;HB#-woAjV_B)-%e5l*Ulp*VDqReJGE0`Sip!f$xK>zhIZZGbiUfe8P2(6_-3 z2q(=bBLBzAz8I)0v>;Wo`*f^{Ioz*)J^?sT`52o390&rxW8k=YsW>zNOS6 zs+hF}QgOPNT)HU^aGOja4uZoFp2ip$fVg-hMJWhtoYdVsB~i^d1p&I9vM5FYx?NPv zR0ru{R1)JZ!qH6g)Z3ra<1Y{qQL(oPQMCxEx2&=gFJNMr5d)77A!qhJ@0=#e;iRa< zi04=wBPyk=shj)i9q|ND5f-j%bBXnB1GGbov`9^HeN9M=q{U@IJSDL)L&^(#fO)- z4f&CypgNwc*npLG#w>UtJ|^(_z(5|mObZvNMk*fY7S~Nz?Ha@{eu3k)l7!PB+m(6+ zTgW)WZ{#}M5FzZuREbLu+`iH zkzo~pjDK--ywEYkYLJ11T8geBLDoJbFYCLs>$)J#m}KrvkWGMB?hM-!4@Q|#ozPS+ z40&bzE^<7m!I{a-hD@{x03jKJ%`$*>=xnzk6KhYQhi#jm%t**nAq(EHWCD>`Hq#t% zzMjOW!cv5x9lPhw6u?rvko8sgRk%*H&?eya2Jy$)^7Wfm)(@Ki>ioXq{E0E{%)PJM z<*E0>a(n}sAvC!i>hxXV<*2CdzL)9&F=O+1*uKORCfR8{S3p(~aG=`+;yV`Bc2~xi zu9}}{+(rrJv_{D%3DHFJ1!iilyHRdruBmRuyf}3pcU0Qw_a05ykw*{ZViVJ&l>ldn zOe#>8y1rp?ACoY%Ophq#IQf}U99PLSGRg8o#yOcK$ltUsFHPm7T0Dy3DiI9h*<(m+ zL0x#}2ZoHSkFZ`LU`YWb5kwt7eQ;d-mRDH}FRu!KWOZ%OuC6A6b!9L_U(|0Oa$=cr z?YDX0%nfTfRW?kws1Kj>TNmp{cZAVyUw17nJf7JReQEz@q412%52Nxe$+8!*u1GJ0+ z)}GZ5tXw%iH}nZBZ>1IOAU;C)mKR(8S86V^P;+OX86M#wkY#>Bhsat54B%a zbb&E(mU+5-K)5R|Y*VIr&ZfY=HMLBmn7L0D*cvm-Oos9;v~mPvhEXb^#El73gr}cM zm}XQV%{)W+8YT`^xnZA*qeJxY<#`cT7VBqWb+Swn%ap`_r-)?UazmCj0E1oMt8z*e(WKP%DV6{D;_F2jaDcE5qQzXs%4 z-F7DSWG01bd>kElZC`&KO+UZ)@_)kQLF<9#gF7gF=ND_p7!WlH&Vc)gu7h- zI}%^qO@2@;i8swIGGs1T%-{Q8A72m#*lzI}tUciI z)8Z#GkZ$U4MzVbyZ6tdUQM6QmiS%xPp8_VBB2$f#_;8c@Ap?yO7%`KycANp^C^6-? z1p<@pPE?c@M(Qml3uf+qQ`9=g#7<-@38w3iR!EIiV@AfpWEB^aNA|fbz0kJ^-3}Hf z>#3ub6Ky8S<2i~@2yvBL6>>>?9I3eJs)ovh)fWC_O2d3{t# z-eo26p%!BF)gIMsq(O((Bz+*GQV!G9HEKP^ARXN!2SwW*Z~Y>D2DKsNw5-G6JUGj= z(nl)NH|N+iB-4}ea{kM`Mf_(+C3qUGnNn(lr--{md9o3nu0hyPA}aS8IVUFTP@gfO4oR5yS}7k5S*TvJtwd>+#dkZks4P4i+gS#Q4tFkEPp?n9?UX}?n-|$iJ&vM|q+zW*LQXvAl}c4Xa|HA=&=(4Zd>7WMjgz<0 z*Ca{lU^G_aO#*XDJ2S#bl5uRV)Y~;`$jl-sV{tkz@0e6C-6L{@$ULfeNG3|V+u zDY(6tRer9-{&Y4(;N_4>Afel%8Fq-V z4mC%UGJrdFKt{j8w)pC&_!`DA!Z1_NHtl2_f7*kxx}7H5&wf7u{40v~&JsB8eOKVC zPze7WMpM5n>HdkMcpdNoqvG1 z|2g>{x8bMyfdfOO=5@k@rBbc};RBQeFbGbTVb7q;+Wet<{DGqoV6c!o6j~DIrfZ0I zYhQp5Ciu*W5Lg*sM34YUw;QfPxKcY4PBXm68^=7)4>@0NpO4&r=k7aVi?MBf+(1`Q zX((|>ad3U1x?nyqZz4A*XuH2j!FY+ftk1u5HYgZ6rZyU39m-E$Bx7v@dw@(oEDUoS z)pxfFxdeg49sy;|AG^0Pm{B%=%!p;hUH4HyD&~| zDKUl0BC@@zGU=$RM%ALLtjbzhl_@&Dt9}?4^(j=Jv^nqS`px5$HgQsFJdt;6eE0*) z!b*BvTwa=((N6Y7mcwi}0%+r?*q1sFv6WN$ogPuIuxNU!tV)0ObcJDbxRcJx&H{&v zxjBnQy{<8(sn95L7nI@Rdcxy_nmQGcypVGeHC&n^nNcqg@>ElRHh~k=WlCSMo?%se zkur)jR)J`LgTCnaI~mpb;t@cnh88M*&$e&~Q!3k)ev)NjA!($<0f5PJ5tuj2d>qGH zkEg$AV}zMUxM88=5xh7|KsA(G)+o*H{uQex8pR;d$)ag4>sRL;rpj3#>zB0=s)x53 z`T}{8AB!YvZtUkf9!2E$vq>qN4kjOY-?&}cr)l1}j~XDlvp=SgyN{E2QU@KyN>|`x?Lh%Mt%IqzSv>>Ahd-?M1yR`DG20*N8tGt#sq<# zP7Y!5W;GePScIlI5}+XJ3FE&VcyiZTUHffb7ob&qMSgYT`54r=`AU+wILvZcIx5wh ziuH_cXgb&GxCh*oW4VW+`#paBx$QN=5n%V5mWtHkHQcsm9PlJj+HpYAKBS>tH8NCG zSTGUp^(C2J>h^=D=$ccAeiu>m7Q@Cpf*~iq%j02>^N~+T&)_E{P5cSE_%5jly%8KW zip~(0?k+5$7{5u}#~}W{zNH=o6T9{AQC#8w8O41&AO7z}`F)qlwE;v0PAqJ zjlN>iKHAWK#{hcUTt|Hljqjg9aC8;piVZL3b5M&={JW`~|waC_PoQFbHqy zPiRyqV`9Y0y0oIKq$DH5%K8bpbiC2&hK`04lzz>*J@T4!{e1tK<8-z4^}OcwV|gzN zv>L5RW7bw66jTR|0xgRTV6+k?Hif}xv=}uq$zd_PZ^~mZoM_5pGQ6-l`g_fKWe3r? zkRTJJ9N9RNfE&c44+?tuTdikQmAYYwMr*Q4&|t_@fDNKyGeVniV!)L8q71D_G_IO6 zu3jW=4TolsOW-Q?mf;DaL2cLo#2Ck4Pn6!9N%;b~NMiU$hY1ckpdoJ-E?+-#!Un($2WC zy*YKnIe9E6Ihc8!1(_2^>nIC6Rw`wq;d$CsZ+$Li zI7RjP%hYyd#W}C~yliHDO|)4KmU8O$w=L=X(KVc-zfTRFPTE#si*r*GbaCDE>i_y=+yRK`xRN z4knZsNTQf&#(GeZi)J`U8l?p`?jB8K<5O70yLizrpI-j8swf6BXtPhQD>POI+i;aP z%yhze#-sAwcXx(Zlq%R={kZdXYT)LX9%2#{8G5M`K-+v{C}KSph_k563O*t2!MGP6 zDpJaf5p{=Le?_O}X&L?vc+C4dgu<)$rhs`Uj1ebOp`?fcX^T`i7|~grpOJ{n)LWmZ z$tBm8t%AZ9Cv`5WOp;v8G({aL8e4^a1avp215tgG1Op>f2XD+Vn{`4^s*xYb)P~~2D-9Fe~tr3rN#k8 zt$jqnGyMq(k5Ffv!w<|@ANX@Q4<)l6rn3$z+TMWuZYVHm#UcukW@m*Nl$tly?e!uZ zx$MMQr5ktzB*mrUBH=gs*#}S{{k%uwZ*(hUfLF-x)U!95=-j?LsWAxlTLrMLL2G*A zjZniE`4BRWa6j<(dQk8>V0Z&~&CzD0bhd)k>Cu6)(Ny~+4%3KirJ6<-19TZ;cLswq z>9jRr`Bwd&fB-4OjGi+Hwd~!Hw~7c577d{N1e7jSnF@kLe6Uvz0ev%Y<%BEE!{@O^ zz@!*Et^XIfjw~~`NbINdT9{VsFEss5vRzHT=?4zoU9p8qZB#=8vk`?Pyb&vwOX7w` zc8abEFgyD|XzfQFD<*J;80Ys1kRzzeSJtFD0IvbRH8t{k-H(#Ux-;1319i3R>MFgU zj0AL$--bE*4U=Le4}-G07_gfbVd^Z88$DAK|`^vQdJcPN8>1uJs>G`reZV z_hMI3YT4~asz0E{rcv#-gbOT39gyo%YX@M~GM+>YnFC@JEYXl#)h5olrbXo6Bm0n^ z(Q1>FMfCGGLm|D3WhQOQ6Db#;@_c$1tMnSS2RNN;h6lLTl=>@v;^ZexJl`EMz{DLN zeF(hzG`z=$ZT=5@KCBq7o^9R2(bq%ycupDxow0DI}aD1 z5!J68{ZBuIkFdlS$kJP8`g%QkxZXV;-+_sjY|_8Lc6)MN!MzX<-xvyfL7udWIS#_< zoi=8N{8td)M0+W7N4WyUg2JUNfU-oJf0t>e$JDpISjN5jiKY|qt7ob1$8`+#(l+WI zSjtP}uW^5!gnISyA<1yPCq=xoBJYd1i>4FloZm~D?BC>vfNtsmGT%15C|T9b77NR7 zJ~sqmsG6YrzqLeW=LHE>ErudXZ5pDfr)*bAL;xw|LxKE#_@q6;@^v{I!JF)MA?(1{ zpZkxneTd7nRqE=^b*6#&`o!Rs0Zb97G=+n8bN1J-QQXKiKxP=&kVJggAChHvX@!YT+ ztaIjEIU@r&CG-$)t@nWZBT3RX2@a&Aj99Y8`cN*$oRe`B<&m`@o*o2+LEwHZenjx_ z2onk}_?)!igUPy2HF-7gR7`xvCIF-}$|(Jk_V`=!oKb|r>>md;30h{42=6-nN44+adEeqkI4VvbuK_ zzo!iUQ~v(@ErX{5y1onH-UA>QK)Qzj1_T2IlcC=)h(N!vSP>u)++aVA+25eLO!Q*| zW0|*8(gbBo71h<%HqFaumgn=No9FYgeT&5{mR43;R@G9KF4ZbpRyt>JyvH&r5=@4_ zdTS5%bG>GGpL0GpxSg)Iv}}PHVr7+Yl~6TM0ZO+?5v3|TwR@nb^yM$DfCx3^u8KWY z)Y?jSxglNDcV(>N1hL|9Dj7wK6v;*v&8qo&Dq5ve_heal$^v=${4prpidh3F8HE%3 z;1Vhn#S@6&5=!SH!9yq+wKBzgPAC;h7Pb63!Fv=g$^2I+T6yz)K{4tk@qDAMv_Lse zMtpQgZJHa%Zvciie#ssW0AH|8d&kq?4)Q~N2h1NE)b?%{;sSv~ZHLK!2J&Mtl-MD; zFBQ}k(pzZf9jHr_j=N-H3zK6cbhT|hR5D*>&xlWX*V`Z3J1sI6+Iu*Zfez&*sjn22 z4)vv~4;|E(;6@1lXm=e*hYu&18l;ElM#aAtQhVQsx|iUF7awg0AO*w?T7%ZM92zXB z6;=z)a6eaYMYM4-BCSn+gUe5Kqr-2yi{w88?X9#UhLtmj3)G9yrnUnQ1VC|D-k}Eq zpt@@hG`^&U^g{EH-!OfZ@AkHd4PdwF4P?GFhP2+)g|s^4hPZ)j4Q>E<;R;1@T)b zlN1_mR473Dmc$1Dc(KZXbbsdh>8Y|}CPPJ`aR}?f*v^?(Re{-O_FZB{56rM)#p+oq zlNCaq%$b9Aux5mBLw@KMLR~D2kJPeg286O`Mu4+u#)z?S!NFTL38q!vYDKH=v13*D zX9KxeG-(Rg{Ko5lh%*VDw=q4rh&pu+QM4Fo(E8?8rJA|`oI{vowc!!X!Sc%tRf8CM z442ocMMDu|uwAX(eHK(3IlGN1+P8Q^$Bf=aThig!mqpT8@?(=Ev~KN5Y`8?Zc5nJu z1fb8O4j44MDGSEaQ|vVdMCZ+InFp#&IN~jv9ci#-$W2JCO*2`6TDVk~wf8bw=7wZ~ z8Y096lz5~8W+i(O;=&&c0({W7Q{@;}xw}!+%(S#6>N=`n*-Pr$em#E!-;Ef>19iJa zD+KW0hoAEsO_`zRH?r&}?Y)Rn!tZ&|bdO`sCgG&|al`8s zx#1KS#fZG0zJl4CIKyp3l{TES6_vs~_=${6D^}|P3EDEdmgF|g7zqU4(58CD z!)DVq;njeBHqCp*-^7cL3$08*iMC|HNomeQU5hA#{ z;lO-`zOV4`|+pG%1s;Q;j=-Qd&NhZG?7RlAb(?<`Q?7slUOp^LhzM zvYC|WR=sTO!fU#TB+FG}X<)Z*bP^NYa=b`rJxWLj#vN;e#!m`Zu|GR1+Gw&ep_4^>h_GWM^GONpNBLl+ak z3L%=^it4xxwGVw=*Pmr|Q&|oRW;DarER&`TX>Q|clg#GyC$a^LIP8GBy@4nD{yFd^ z@K-}Uv%#V4s@=TMeS4+xWpa~PMEIe%w*-?C4r^>IwVLQEqL;JH*_!&nB4TWtdfFt} zpta~m>F2sAquX}rF`2!KOVM@4Oyp`nOY?wByirpoPQ|SB)04&h(d6EXb;`xIb%*W7 zBt17V8u1I}`xTgP#+`MXPFJ)9@MR){$E-I*Bh9k49Ji=TM)WZala)p+%6kGVC1XN^ z3<)<)WziVp#St538{{zNMpoSd*fHm^YE*Q1dVGv*&pCu7jhf-4bxtC}O|P_UcwrnAQ|Kc0Y`2MaLK_X-u7G@6S3wdKP<>7A&3|w)uFx`-U~^-jb}9Xj1j+mNl7^WDMrp z%V8Nz+!>SWnk75*{V^A>n)j|5BfsL?wcnSv*~}4c$sxZ2m>lPiLU27{?FHZ<_%uL_QN2dhV z1<2{o9wW%#&W7em6X-Oyy81MGZxZBhYXcltb!@$YRd9`#c@;bst zjC$6P#fO9SmC$i4-elMv<&;;S@AdGQr-N-vc3vuo)RQ0fvn&q{vFkprdKtBfMi!+d zsAQ)CvT{S6ljazJVQl`HKq*KmVCZTd&pB!j5+1n*VkSikaek>lWJ@;#;ttfOJAlgM z01E2#D^K&Ndqt|IYCt%JpS-NcYWuSR4MS^2R8a-_}JBrK*Ply|)^#O;#0 zwgLw2I%Ei}RoLawR^8vB-%}B7!nJ^<>_RlN#`W;2b`uo9Kc=5u^N>e{^$-U_z`)1Q z1AeHWQGPk>$>&{ofm!M0#lVVn1KPc*?&&4pjr~>8x007JR8oQc%Nci5!p#6{NuZ?p}u(`-foSv*Z$l1gdExxPjffPN_ z(3P*Y+F-=Z2-3?yIt|&u!#V!>jHf0cUpt4%4y_WfEE&eD)u7%Uqb3IvHEtMuMXh;2 ze-G}{suSxK*u(Vq`@cgIvmS0O4}G&rUEq0YGxX;my9SQ4yUE;l2Z6o+OTXOA*9Lvs z!9pOl1XAYNIe&;WW||nyZPz%}-5jsJ2wF0B~j)9<s$`6VcwYrhHuJeClyB0wA0- zFIw^r>2y2-Z?y-WDRm@?G0-2}ar z7V>rq9C=?aZ&=M|R{+^x7`c!>;YAGdILiil*=Hx8*~!$`?1=q1Yt{Gf?Jcw4Pi@6I zz284UKcQ$aOHH!k9aM+wqT)>{RwhK)(Yv)Tln_Vg0H`u610tzV>GUSf_Vt;OvUHxx zEKcIg9nKXAd;N>I?DN{j&RKr>`zUYDSp$r?$L(!?fm{t69bvsBzXXfONeMF~^-^ZP ze=-}J*>JexNUvq1v55sa$7Gzxz+Sj8rzILio1+d#%1Te%bl^?4aT+KU?K&O zkfxRZr5x-9CmJ=p58g+66W5g2rgB;s-%>)Em50-no)di6|l7X17*%WwP z+-EA^7!%x_$}wGa{VNzu{c{7hEs;*$S#lbe*hY&P>&IjHKXp${N}DSBY#=gBme1b!5!J(2rxPmDp?!Kez`2a4?6ujK)Y78N9t=@RRGa^~@kRLy z>R)Tr^HkNWIcF(ADylNGMxq}#CU}Q<+40uLdb#o5V2#Kb`vtiOl9Xo^cK%>&9b5bu z?ivI_%&Zjw7k0{Mmlk(*Q5S8T$1x4&b|Xk~OZzd34~q@@Oz|0?mc*{fI6WfXIIF() zYEU>4RsxE+9P=^WXKV?@@OFo`gna@qZgm~ic9UA7`9nLU7W~)x#pF%lk?mgM2vS)> z2Gjd7_O1}21qKNWmG`jz{JeH9!|hJ^{?AdBdGo+|cKdLQg8pyEs5l51;xv*D6#8b# zJBaK5@YAo@p#J@lGA-~Cf(#hMWE~97V}w=JrLpf-EreRMW>W|+W_Pww)#cWA(ptk= z6Df3{i7KqraN?DkxXK#52)M6Cy~vTtViK92o#lU6_uTYJL^d=C+*@wgmf zh(dI;<&2Vhxn+$~M-=USSqY94C%|yl?;9jj1V4)R$)NR;31MAjfPs?|exy5cXeHu) zlsmR)B@)qK02ZrZu!NN*sjo_KrQze$c37wZ17Amal)qteepr71^N4m=ZAiZX z!wv&FgHJn+Wtt1JM(h~_CNp-8{SR`Fx8D-HV4(0P2?Y2yzj=<1Pd?=0%fAANI_D~Tx=3QR6>YCB= zA!#op6d5nV)=+M*dj)Oujh@eNjs;3RDuop7z4aF+(mlRBG~51b%-fIxWZPo5ay*b4 zJv!;=6Cz2MNJJ@4xr+he2b6wspW$#WYeJ(~C|#T+Ic41JRo;GWZ$A<`d02EqbzR=hO@5RS}nG zc*kC>F$i6JQY+QATau>?_0A8BDl4Hqb+pg4H3VnMirBFgXk38g)%q%e{HwuVN@@0_ zvo=(_0#ZnA%Yc^DdTK0cpGT`tvwnvT;2zL!aYx7;mJx7U`uLpMR?_TsKEQhfo12sV z&3Vrp$~6=drtvIEw*IwO`P7zur1}=i>8*tr!g4C+2%#&Vo{NC^>CUw3JkZ3-pSyAIA@G@p+r-^h zt)pgkd2nGY4Y_V=(Udc|&nB!=SE}XVw7KfZ4jCC=&{2P1tFKUHb%;}cep^;*3H&w7 z!~J=zuca)bIx7GNUgNZ(pNn#hRzi+VLH|Wg{OBrNk19=e+P z97G5_2Sv0Q|8YqFh2KecUiBng9a4V+G)&!epT|lg1{_vX`+m<>A9M17x3PbBmrj!S@iThZwXXn0r@n=B z!2`~9PzC+QSJLUV-m~*z#f}>7U%@nKz;*R*?d-5Us6fyS1`50My(|7l?(zt zS6irZ!Oig)@Pim%?gsJn1GKQ`rE1Bx=-}gZ)&9tssVpiJz+=X3-}BDZJ{?OF_k*go zM+<(;Cg0($MJgm&t}3k4)FeDlDrg6fYlEmn3&Pr)5;i^TzT%JD4Qe)+e&ItE!(1=C zGp1%Z?IFk9hUKIEEyx36%I*^C3HJg1CJ^FA^Pp_#z1S^{4wJ0m(&dkJ;WP24i30xy zi({A9X^VwFLF*kuRI%FSij|Z{b7FqWM<7Ko-&7``nb^_9Pk{04%IP%|5iIATdt+EY z`u66k>$Yg-_MXjPgTE$W+YisW)#*gg9oQzSpCduDutmBSn_npOb)>a7pT!UO6xWIx zw5}e?6#IvnNTfw5N{^>-E0mz${;q^tBwFsFkpnEpfF6n8|Pe-`Bo@fG3uC;RIZ zGUC`*;YMOv---2SpR= z8sa5w~D)q2GcJ7Z{(HjTSIom0ELkx;{ZMA{ zI^#b4wZjUwS?{lTLa3K^;5_aiv!^d9;PG;M5yu8^ptMT` zq){5{me5%|!-2%Fww7}fk~bFJ+06vys0J+Tl=PWdvGBl0Qv)${A`Ght_n5H^1d9*Y z)DoRKCpaeu7j>%qG||)&o!ny@Hy(QzT@k)v5X6P|0UG2s!59AQi@idN9wH-J_3C?P zMRqJ!bnvk2NqghxMzcFN`DRCTQMk>b0wpZi?`aQ<()6m5Ce-M zWgRso53~}%kW_0iCeOhsH-m|Av}aH5v4F=ot_0KxQ~RS1a=RN}sk(bMm~NiW!Z=)C zX`H^ws`2F|3;Hvy&tb2ihJ9p{AWH38Bs#)p;>s6nA<8~`B!(O&sNuqpCSZrt5{pow zScsDqDBIU^LS^*lM%`#3Evr^_q85xOKRh?=9tm6Zhm=j7!G+Wbk~X(gbJMjDeR6d{ zCk!YWD}Q3iAXOK+XQD|ZA!vfL7>3mlRcA@l-?*|V$SHMgt9|Z1h3OC=rY=D1R3Skv zl7LmY-JhJ|d8)6OceGLxYBH?7jBA-%Wo4Tt>o}}grM@+#Ip_IpG)cz^%{(h^1i5;Y zJ&CB>FIKh;TiQ99jjAYrKiW$MI$Ni$r41Ojr&y)SrNE`2@+?0)s8ZsH-L;ACvuAy< z8*X6oEGqqyFEu+TYVxcuEnnT%H|gu4B%^!q@V!QMR1_Y9y&)_3`x~Th%sJKv#PyhX z#e_$#iDuC4(fd1?D?ckDwvxoIPphUEtaZ7M)zB%S3Gsl@4kr>knV&1rL=pmvT?53& znGQRs3D0*JKuC9*ytxSK@qT*i*f`T@AmpHmowerQEqE*3QChgGFxkYK*#(?zzu~(+ zbh_3tF*)2i&GkC#mRQ}*^2ynD6@1BH`g@R2oNi7-!2W*bjf;QRDCfcc z2*={?Yu)8lMnoMCI^5;_@m+aK28QiVqckSEM|cdho@CcbMDrINnT7Cq^9G>o6|s3n zFsC=qhZl)XZdv)Pd3zy6NW#~2S+wf9DG>FRTq-IoK1QmaMM^B&V z%)fe1Q?E0w7YAO3#5j&CfQHWdqXGxUDPqmk=^qp)BVf=;>W{laI`6yzwM1DIod>Uh z&{dzMTVZd993JlOE_06QVn!g-6LqP7FV@G-z5o&O;-eq;&9%+ZR6EUWbwuY*Vm90r zF$&>!r{;KHjz!u}eNn0(--5sYeT&rc2lm=_#$C2ve0zjOVtEa{8Mq^YvifqXDQGAO zsbnpOixzT72;;d`=n|w`6PzXebu5+J#`2gWU`-r}#+08^1*a+=_^bjvdb%ha?2>Bv zu^OKvpHTr#keJU6_nwLU3VRprk)W0ux*;(eI^7vLvAJls<^&YG9xQkK)FCP~OjyY7 z!6e4*Mm)c9DS5_Nz?XgpGCY9qVX7_&KF?gSJbv`_34IeGzY*$r{KS7r8Z$0M@fYy- z3J!VayW62@BdX!$H32!$GeKBHho8gj5T5Xul}{3ObBMK7M7))rWn`%-_LR+Pz|#TI zmU*%XAf}uhN_QT{G%!s!K|)65T+b7g&J(WJ=ZzSFz(XgWjjD!;{tCAh_V^{Arw_W| z-oz*|b*BU1d?6w2#oQCmeZ@iZdNyh(D{1e=x58|{b>u+BsaUD2@b2r(7#Hv?2Wx`o zZ-V`|kD%X=7#ZwMfUUDedfg(o7Kf(Nhr7L1q21+O3+BBdk=-G^uA%w;72zH7pL8tf zY8@OVAiYH+rDrD&n z>hxqfST6UE7n4tTRTjj4E5NDL7^HXb35xCMo8Y+{KzI*gH#~Yroos`H3(_Bwn8!tA3uq!_hAHo5?g)$F#D1{UI7p=?clLNs% z8Ft{85{S#|0^$-#*c|}QGznC+Gb>Rj3Uv$5a6c^0q%+#KC^~DDtG+&_Gp?361uNwa zrF>Ki2uHkqEY5W;p}P>wkc*SByTdmTN%IxkISa>&5QFS|3lS2qmT<+<`(w!KPp-?i z1owWL(Z#g$pOn5pi)vm+y-XFVo2}SiHny$VO&W5w3Q+{xTvC$dmt>*JPJ8yCprHVO z198;p;1R6g6m9$nij>D^Y6XLmkxXlz{z(b8VvW=kjUqR#l*Yfdbvm@RF2fD#yybuu zJ)8bqwG{Jt*`wn~n66QJjdP5K|1uUWg4P8ZyfP_CHD96X+P+3e-~TJrfElp)yDo zG2S}vD3`C0*fXJ)`!vejVkGURixM0DQ0q8W@o`byI8-{^uqyW89k7P?PqA7Q;|xB4 zV!E2bLuMBUOtYU;k^=se1b5N14w&d~{L6;X;+93U^5BqDg`PRVKmrro@}! zNJ|eu*g3JuIVSCGDKLHvnpq}PT`e9=%U@}`a5=?Dj2t|We#@`U*XFrk;)0loa5}OW z!($gPiPAJos?}$u>9>wx@0A6FJ-Kbs5ZOZ3&k16x-zJsk7g``=FJNAUF-wCuG=5qN zlJWlr>&-aA3WP4cYWqZfWj&b+NF{A8@Ei08jyt$qNpe%~@~`XP_6_l${xn{KE}410 z|CT$b0FyL_wDcEs>C0t33YrPAaj1Xv>?y|3shWKKWW3pT(;)vKfxqa`_gJhQvmU)K z(Cw%egvESx^MkDsxeB`8NH_Pws!RQoC#%47ybIQ6S18|4+`eFkd(M5q9-6-}&i9NH zH1!v>$8-|E;vfI@T`*){D}nl;gBScaRQ16I{|}Fa1rHJrxU7Yt1$alIzy(N7gXNTI zD%FEN?HW=#44IfNQ{`Y*H&KmpNoC-Q}#tR;w|ZXniBp2lvn!A zFC#5Y7Mz>y_b}o4+dqr{{pDeC^Hc100MfH9YPComj2s&q+xbY1IS*S4qSiorG&F7$ zn^4MCM)agP(2sOi4)@#eqwWAVTW4ih1PyGSUQjd!yv=3mCrju>rTFACkr1Zp)HI9PM!`9?TQv`aCg@Daz@TfxCFx3o*)N$yKgt`N7dPKNz|!SGHB zfu2wRG$Z*yXDnh9-F`$jEKpunZ)s#4O9ffXXDsY0VA&&pTiq{M1Z&e+w*Mhr`3kbC z{1L8M{Utg*@*C88b$aqAxOw}tMT%2LTjI>w1NZTohSCgY@90%|B`FGyJtfuFj5%2Q zbQw|G(5BPEV$0h{hS}&R%5j%O5O%X&am+v!UT>@Q7)yBEQB&_Jxa1H7^=*~2mcXjK zRFPmESnj7@#+c-i1UjNX7rW8w~XCiamLocVX-RI#x1clS&?m_Ev8JzXpTXw zwe2M;1`=-L);CuUJxw3+Vjm>juTnnhD~}xBc;#P|1~T`^kF6{O3n>i1leW3`QSGkm zc{M;d4D}KYZ|gf0HjvU+A-u^Bx2ntYlq6LHf9;8L$?RY)$`@+SW!FlJHk_Fkw6Nmz z*!imINRm;TW{=~4sCA~r3x}Qqa_U{mznjWekH_HVs8Vge-KRJdY#H|C0Y?!>=!d)r z)v$;4$?a*4C#mtY&79#2&5_e@K=v8d7T2t+nLqZX*c>UO`!zd`EZJMLcFbwhF8@XW zvq$+bOtaO(_!43oJ936LH_B?UD)hZ5>Ay+T-|LTKl}Euyk3V^hupkAfh;I2g?4H!4 zN5|%q*G6n4bbf}r69WxPuHACfQ}^*CpfV7np|>{iww{1|rSd}{Ya`CLMSQf9XWbAe z(Ua3HYYyGuG?G~1N~jbD$ITeB>c!=N-OY+CmN~Av@`4J=%{o!mSJ;bO1THFZgE}C& z%R#kNaz8VRo?q6NI((*{27(ge?lJ=f#l7zAikNq|9EJS(crmykloY3=V=TKqq6a>m zR}@2`grq5MhTELv##%@1X;a!?W7?>=XY7nr(xnUoEM_AY|^j(W03F{IIa#=BFXIGD3 zf>V3-(q(V4*B~?;MJJuMWxVh?Wy9{-dXw>-U7PydWV6PUKiA&Draafk?8@@InV2jp zMmVc({d{o4kx!}V?ZvDkasqq@HXy+@uI1}nt}^yjs#5DI<;HxUn^b;qqW_lVzLq&= zmcCvR)pU)PI@0so2tRw9XI%l(jdD%wUDA26Fw2ec>_}1e64J*eKgA*D(z7D(3a#(; z3b{Kl+>Uw&b7LZb=-Mgpf)!bS;m2w5$tAm z4P(I(_hUJ+nMzzO5O=)0%$&fdhRKt<3@%rKec}f+TB1_(X0{)WP-`P3&O(i8*$G@L zMa5Q(2EqAN_={~RR*Pbgt=&**C(^WKTB1V3*pcYE|y0g9QcChSL!~0Iz8w( zuR57U6|PaGUdE z;P5X+(W(mtquGW8^6Wq0%)hFh|NVp-n4&(axT=gQm?n^o*DL@}`qf$?XeS=^;Z>Rj zL7ASEQu%XC=v+)KbCrsT)V%@}aQgQlNP$~Mqp;~DKz+V8rwmM@!5|RF?|Pn*ku_Ys zx!DQn1zyBm8~r-|bu^D+`)8;ndaj%o>2^`57WxP}9h+);VuGY12s7$Pc`YRY*iDGb z_Qwp-R1$QN^ig~E54n7JBmeSU|1l+*qhMFMVAc~a@#H%7Ls^U^L_}wiX z0(^R%?ERG`^(hsDDD+0skM?5>S|ePQMGcP z&Psg7-j^Zw64$b<+{wFQ%SL?Qo}Le5orm`z9Kqyz9o^zMN|XVf(G|q%^lFdwq5FAw zh6>Rrd*LF%BH9V!u)^dHBDu!k{_5Eu(uaqbUYH#DGt?FvSR&Zt&<1~9_$<@-hxR7N z>t@CDRR4=i!+P^aQrWXU4v`E%v}RNrb_nJm?Stgz%EKXgRumozL~k*$i-i}MWen(e z!9-+#kM&rsa>{%ej~Lb?8<{kGB|`i~L-H6gwTTUoP`rrWD7Jb|S=YWj*4J*;zFTAsQMfRY0h!8G90a#zr@B5}{KGj5#v2DKw= zQHqWFThuW2J^)+7#udxGEII+^PUYp_1_oJsl+o@(RpSfw3FW^=-T$X1`3PL}G~o!y zFU>$I$hrS0&;HLOyg85tk^$y_Jf3QpAbEhcY9G~002dDWj_i<&3Y1)fp(uJpuqUb` z7CwSB^fYWE!*F5+l1wIoO$qTU!Ut%m%5Pef!QjyJ!99J~sm4 z500F7_2l2|`h%C7i;I&_v1CUQzj4yq0rzC*{hZ7nx4>8k914|S%Tagomu7*MGZA1< zNQUGi8kV!TxO|@qG2Zk^FY@Vwh=jDbwzzm7E3rqUF*P!nhhnD&EJU2?G&Kly5p}5I zj9Vg}jmAo`6C8QM!?=?j>5LfJQo3V`_#vX(r90n>jI>7k4!)}z6Y3AAvv4*Hi^YJ? z>%u*bptnN0i(Z?>xzpC1EHcH|&j)76q0sPrb?;1%J%Sd@U^V&kvHZLxrf%eWL)O(gQ;|3Eq@;#XijP~BA?0sdkXrKz07L-1*Mv}kSc)@tmRECy-hd`oo^9xB|K z-@J~pte!vPIjwI}Z#a$K{#x*|YswVGBsbFX43wLGd3R>kpV|}J$|(j8rx?X}o+Wrf zNHM>9)hh>d*>(@YQa64JU!2087AQZIj_wp!bMuFbnRcJV;OO|OChE$!IQ1&UbA0+2 zsd93<9r2xUbgK^ONwmc_IJk`o*6zezX8f40)fE!fvG z;Rgerx5#(!TZUDnpizK4bS<)Li%U38=?0-gUs3K!#=zv!ydxi;KdxMT%ShQqZo_VgE|(xG2uKRQh+WkK!X>r+CFrzQ`fk2+$H+C+2A(QJ?4ho+%|Gr7!nsq&?=9B z40PiM$#g0D8;q?l`sD_C@2$}_^>BNO=M{nyc3*Q(r-PrrHQ~R&1pz->M?my>< zmO(KYO{}Jjajv+XHeu`a^!OR8j3y2@Bnn9{xNcX=M+c+`LrG>@&spw!+C@DHB~ERc zWYt?9RpM{K@AyF5YwDK*2-1-Yy=ho@CWywcfiO)y)Tj~nQMm?Y6{2I?T$=KNo?6VlCb=Jb6z z503C84=L8HX#?ghC0I8RhOY+oBY4595x~AJ=iFMGOCey^<3NpDXQV&<>Kd`$Igb5< zIp1gkT^PBgK?E!IiIYkpI`z8TWz(;;Y2VO!3EZXp(>zs-q9%zW7pe)y?sfBzdIP+Q&EN;9v-SxMj{x*1VrQzfSi~a+{x!Fs;E8sQw0y>Ua~6q zd^+`_#2AAVhPn})Jz>?Oax~FE{ZjQ=*S5cI4@dl3{_igrAB`ZQsX|YDXkMri+8QQe*_f9rmc*%}|Uvwp|r01yEy&GlrHE?Gx|A*fv1h zC;_Nka!SIH#K#%kAc5fQJG+O9J1f7a`S79X4_S`8k%?c{@dA?+pzat2b2)aVj zrLlyBwP(OH;Vx%q$b`%Cro!fQI`Q~%ML9wHY1}X0DUj!)4$|D=*2@VwxqTMQ+W!B=2 zt2=7}+Z(fIVyM@x8YkJ65K-EPRaKp& ztUO;UiQ9rpx`T!zY*ZpHk;gp}7m_&?bd-aAb@xZ>@d6s~I5x7CSh@zsn#wW2R{m^;`vmG3xwuHxrGc{G5983ai|Dd{3ZzvFv2N-gK zN!5HNQ}2EyGTqet3UN;TUY{D2B=~g*_$lrlyH2w&Z1M1B^?v)FIG@+1e);fI&R^x% zd;f-KYhqv(cequ=b0efa?anbi_=$74u3CH>FX^%;ETF+Su<_*b$`LkTP4rV4z;}NK zj%mCXr9M2q;EUDGJH!LMbS}t%M<7PRtww|~f8Dfsp~~~L&|t{KK)l)99*dLGxDupLSj%jIZBe;a)ilt|HHmha~gYm7xj;Uc!f{dr5*)ShiJ7X znRcYfs5x4xx1FZ=qKsvKK@JJufEN4^QiQr&(ig+lPcbV_Nh?=@e3xJ*lN7#$Jf^~* z`223To6{N&(T-^@rkiQ_QNMf{QWeTxFAO-!?tu&W80Wc#^U<@K%UxBG>joH=Luv%ruv;|3L|&zS=h<0P{6W8u~9Y~O(yF)^F`0cX@ar@kxvJa)b-#L9+g*uTXx zu*D%?UT7Gwg~Qwz^&0eix>oiU&3|XPKcHzhb{(T>BPDz38vSKm2FyY;c8Nqe&7fym z<1%8JU!VTK+PKs}oKkzVLK2x_l{^8IAjtz0g8n9VK~P?$bJdD&k|@#wk|kr=Xr>T^ zE>TeFxYv5mB2l((ZCT&iviU$QeOoL-v?0}C{_mT}BJJ3>+rHPWHbLa+Jdo!WHNMp_ zUeAJC3-v}lI7QRW7ytGu&J(MXY9CjW`+r?g|07;Y!x4r&`k!*{qj#ZBbCQFEOGEBO zp=s9t3aRox)n9Y12&BS)>_I=;nE(9gwHJj1=pM@|2PuLYw%M0HyQHnXRHT5n*fWwG zNkRdGE}dkvM#zQF)i(I&eCsVU>>r8SGKPq9w#9~k*G*^KWV7d|<`xF~5A7~y9=UJ%WHiO~MHX3M83|}E(n{y-~A^+3rO)^3M=N)bIp-`pe$U(vaK@02VZQun6craE)LtsUu zk}O!8)moLMwlDONV?C$JwyA#H(B`>X9D^kBpXIIt?8=$iHqvfH6WJ%84!puZ|M$;Z ztXzl;_+DkE5`&qsQhW>UOS@C?+zH(Z5)QO%=44U95w@}=+aNf?S?u%eSTa}a^WC7Z z@bpIIfp(z86gzgVQS1-psyX2+re;Vc!L|hP&Ratky9B;yfh^CK8MW>!C92N7`Nb2 zM~AK>uXB(rH+oLCc;=rB2GH1^gBe2KPl=sqI%wFlq+;nAf!Aa6-#w2mf11VB`DZu!(_05R571>~%d7VWV_BVfoVbQR zK`hpQAv9@E1&l%*^mQy06L^Q}O{1%GyZFR|y+haFsKP(XrVp?oGQ1?Hbga=fgqZ5T z!|Xbhitp>@0}-Qs|65hSKYg*M`{>ewK2Gibn~$7S-a?!PrM+y5KRv zeUGNc!`yRU>p>!g=W{5i0PYfLx~VJAMfm6pBEzG*q}&aBS+E3+3ana51nn+QQk*8P zqS%Gpr9b83D*}yf*Oolmr&w~jq7ASCjkVXo4=t9?lLS-l*v`IRp@v7E@!|ARbgq~E zCU+IXzqnaj6J=Z7(_a*qv%Uu68*D}z)G_WF6?Xjbr*`wJb{HQP1->Y~1%=O)Ys|U} zP_LwrXbsAi{n5)7y|XxRs@7Ret66dq*obbkCnBh8lIuQxk5RULayGh7i`}C8Z65Ss z`Pg)rI$|n_mY~d|q*6z9c0|*8IX8sy*gm-vI$E2&68V27G9aXo34=%*=Yvs}U5_a*DSW6S$ z%Z+_D;`+q#$!29gqigoTB&3SsowO+CqU%UJe{^VwceAjb;e`U%*a@hAxg}GF(GAVX zvj8UBUJwg3pMr;yNx~J@2)=pA>oJmzT1pd{7m?0nl}st+iRF;EM)q3=bM9GpXjHtV zs$`Gh?iG+S0fT%5H1Y$hrn!UKg#V4G%KH0B@{fpe{jZ2hyR?U7X~tB9METFvb?F98 z-K?k#N${V(|ASRao%Zn^E-hCDk^s1>iXnn3h#|^lrx_n;)$a70V!IB!6-ENDLV*{I znga1X)eJ1h*!FW>LDuhq`~~7$qil$D%6>g@(Y;-;E@|?!OL5!5*lou7`rG~M4bG>e zC0H_AV;xc19x=E$ib6^=iV9gT2IER-Hc7KbCyB3dfi2Ng`l;Je)rfGAY{9_8wCP|u zwuAywjsm@}-vdcM*P=oIFj*?Lu&JQs;nG!{zRbd(?_>r;uPN8itobTqtqryWQ|fD6 ze!_s)WGNYPqa2UVAHI* z6EOX1{ZXv%U|VHAJAi4g?I8l3kJLSf*1Egy%&fPoI@fth6bUdixTb+>?K;RdqOXwhz^<7~{pMx* z;Zxh1LP2d3(~BrJeNa+%zhu4c+;J@E$Ux6hlGVzuq#>>90~$(vOFRH3rT#KKCDv;WPh3$N)MPgJ<+54#lW`07qLl%Hc}Lwi z*WCb_p5nS7cIfjeExAOc1mf!#n_WO5PG-_JbH7T*{pWbl0Hev_W2Jp}Q{$`6Ouf5` z?8fpT8sc`+cl?y$+iRfG%O=hs%O%RVWsAxn9bOT{eE8CQKq zv|2qTGjtcJOft$KNug=aiM!w|f$ev}?|jNb-FdQIy1tiBfEb+T!Qtx!e*|tg%VE7@ z2omFMtO^4?sfiW;$@$-79pz>7I=i48I zCoP6;cbwy_5etpv9QLtC+e)~`$5M{~ZluwbBPtCwa|+0>|e zW}@da;hQ9rE-%`%IEmeP0rccOSaY59xX2N>zi7MvT#7nRAPZh7CxTZ*8j<8PTkwu@S{PD4~SY|A~PhJL7+9Z|2 zLM~q!-Ian>3;l*dg(Bi9ktTIJsJkYGbi#Hn1i)m{lD^Fbnq>7$*~SCqATAE4gJ^m!0Zhh^ z0=-ylu{|ZYog}7wO~cAplpy9&p3KJYK?s1VL{^L(!~@okHMo?$i-0PNN4{QefX6V+ z);K1~m{Ul!&SQSGsJPi!?-TD(H#M#aB0r$B|56Z?<-&rtY4+#=M5ZdB|KjRojQl4* zXImGVIWKKH02B@&vRInDR0C0Cy`U@6c;PEfhr`!ito4^8Oh zD_Yc43D`v-9}ItrjT#0~CCZ4NUtM4GD@rIfG0Yu8P3~icx?JXNvv15equbf!T~5T2 zR)WHv$x1#W+dB&b;+H4aH?%xv9giNcGGS5>H?7>)ZNpy6Ps7hK*zdNIXRbauEsK$q z_vc5b#0x5A#5;yn6IM5LY+N|Wkz4~y9*g!h*g+MiZxF>~>{E2~Tb+>zf{idIzeT`I7UI1*VCzRPS4pbc2hmYCsf3ScAIG&-gGeKP}EACF$v-NvPY|m2j9YCPNE1 zD+R}b2)mTHV6GO`S+(yi>UyAwCgV?PobR-=7g-0BMy$+YcYUyTEHQ@CpMs#L>d~?9 zSDly6ZE!~^wP%7uVxqT>>G;rv2_1D|Qg(YpV)8%h|QtXRk4B_(? zQtuCOq_?v!%ZIlEpj?u~Dt7r8i>1bZ6mhS1Xz`_{qu-?&=^JH!-hTG2rR9hZ@K{2} zYv2-GuA`~{Y5rTyj#Z2Z0D%GXMU=q^{SMzf_Btrcr2?(Zgu-`NFR7zI>?VcjP4Dmw zXMt7-1hD!#=rz6b$2=-cU4t1T-2uZQGRYgaI1xl&W=K%sV7(x-$jdd4i>CPvIQmsA zN?NbME5v*aPSOu7YKHx>vV=ar5mzLiZKr%?Wt_>D!`cEKjr{4D!m-$rPf(ftH{K9? z2(CUxl(m6}iM%2{rJmxry3QfqckA>a*Dr%g(>r=Y)R^JD--nWs#B7yAclacPuBMK~ z&lTxC_qMngn;6X@Ec@!#E2fFpfDlmwp2KL`e&Q(hkJw)%PnN042;5IuLn=rfP01Zdw>in>7LyLMnA z7e3`i1Z9!+iIU6|w{Q)xt3WzBe<-`)f~3C{{NsT8>kmlIn2nKnpKA_#VAOTJ!6L0S z;n}puxm#cJln%k{YJLs}FE1`Fq-@e){qEjVjYHkeM2oW!G3HgQMH@h#qd?3NL;cfu zslf`=$e1?H_k4?nsjAj6$ub^%<6e$iFXMWL+>#SCKI@xF_A$n=pX^z<1&m>(ee#%r z@587s`1IRRb6H~FM80WR1DP`&N}dVBj2R2mw0KyQalgGY$ee>J{H^PAT7|7!sAGJqQ~jX8&E+p;au1}!Tb`6o?VD$6 z>;6lLVT3LEMKXL;9$OTg5`WVC%EUuu@Y0sV5Qh^qzj%-}H^{sH1gbJ~o$f84hj>z_ z{U(BjjFL@lpqhU>msrjXkcXeD^5ZCJeaiNw!_bdwb`!%f8(NRr#k&*=+c+_}+=6?I94qIp zHh8ic#hxt~n@c4Xhox?5!frwR&Mx>yk3kQ$uL8$gnpNR_WDiyE69+6EKIvr)&(UjVX=fyCj%6?*+Hyy*DJ;$u_xBN zZ~a^9jv}WySugdb82V)yu0k0uwGgB#_V*Tznpa;*Fu5+w~KUKrTP~ zHJwOVa6@&)7KqE7BE|oG(pfqCs|>TE%oc^J%{VfqUcF*2r9spU0Yxn49fec2LOYJ- z-R@zagyiw$1FJ@=1SiQ^gYOsiQ}sQx4TWVl6sn{)%UP~SCt>IbNFF#8Rc%4>?HiV` z;Q;suNKA;he8{+6laYP{uSoe9j@&+1OiboPXOx)=LLe%R@;PWC=nu&5jcUMUXLrp63Ww7wE=4S`*`}}8x6=mA4+n2+t-oSIXl}*V7|z{p zpE6WIk9qDOC#SiPmGfdlU;J{Wwg3y_O43d;r${_>GsQM3IUSCv8z5`0HBqGDuoX#^ zwtkW8w%Jpi4YS)Gz$7(py37+dE6^P(MdRgBpaH7aQ2mhG+reYS;_u@)#h!J>pI)Q2 z*z#5h#Ob#idf_fAq&`rs+^1uBC}2^ReYWr9Zp_WXP{o`sdoO;hIr>K8!vw8$(a#FU zP<1f~RoB-1V{8K7vg~kN5vzkXAZ>EAt-PLb!T-|y8{ge=H~LYde`*)o$T*ZuR_T`X z=Ww9xS&F*bV_PJGQWB?$`_E7}VtGW6Kh|KVHmppt2|6KzYbcJ;cE~q|o~wKaTM0^t z{77s87%f=gD)iB#Pi1*d&D=Wid|T++&D7&W)To`}9Skoo!vB>1UV1v#T?ijwAJP98 zj`_c!z_h>ikd$fmgCBcgSpUcTI!Dc3Y4!u2IX9v|`B?9QCh`?GscKdO9Ko8W5<$go zO7T@*`emlk^Dc3QX?>mSy;te&3x-TGW#*yhhf8Y%emX;<{LEUK>%@(~!`iHG!0X#R zWEbR`nzDLL@mDQnRU;aj64qE-IzSyA`cOHbSDizl9CPTWG?1*^__OP@n_4g7m=%qY zW-ncpZA+9nh_&N3(^!&uKDMID+$Z%S^Di4_XWMX%$T^9dLb>_!MGI{!bunS+ngDWk ztW=Zfd0(6029`bAqei&2ZCbqO268(Sclmmf&(`Jm<0&^{s}QQ|f51teY)JU*WNRgV*8mvdhK`iaFNni@We)C>M~D zOm$+v&Z3c0LxKa^6M%U=w|hfv-Mm=BuMufx()NuT^z--praghPp6W)xSozB)iV;n8 zJWp={sru+iOOGQ)>o?+?@B6w2mOL$Z^+ege6Xi}r`@qo$F2~^(hyltv!wL%_lp-tL zri$nBPbatSxc;5e~Nv|;gFTa zaN)Cw{_GgluHrWF2(gcNylQ!bLY(jvyJw7&B%CH}P`U>Pw>AhxSJ)DyBS1lEQxT*} zI)OTLn_|FbDGlr-DcPAaC9BvP2&~~Dp&;*JxrK{?9z)5Kb8DsguXQlHClHY&E zuT@6S@~wr_?Z=csWuVTw(qBF$^7IW4NKG+HK)EdF`z2P%2I=VKNOY54eKCm?C`VAZ zq{KRwSHmtI93=XFdn!Wut7&5ZKVi_gAHD!iqH2fdk=jzBomz=E4TpIyzGu77CwJk5 zG7UkOuh#MgsGJ76u#=Oll;gktl{+uA!nxUr$CQs`XpAC^7$8n&<%eul$DpweY z+44g0A=`Cw*V94q?d{@0__N)hC*2I4kg~I6e@kRN-Na9x>JI`*6dlvgN9q1SIw2Kj zoqjB=?`18(vYmQ71_>!tEH6Q@Y+o?!mJurF60SIE8|5;-I4rZw3|=V6Uc{!IOeTuL zKG6+YCF(4(oj?@JFr`p1)aEp4uMt)KG``RvaRN2fD30QbsCJM6X+>WE65mE*;kU$H zJVm3gA+*CX?bX|uKAdJw^1U=Z9NLEqMsjSRP|z&!dam0TiRqcc3$PHMQckcjcS*W~ zS7of+i^Eg4Ek`hOi4_W}zRCri0A?;3V`)kexd*W3!ajB#AM*;PPFK2Bc_0l^MKcUFRV+v2YNuSc(P5tPI7yYlL} zGg*HNcfQIZDjA6l({SlYU>bzE-XvNt2t~}qCAd7Q2`+QecW4Sdu&g(u;$~zxJV>2Z zwsEKZxy}0$tl;hM3m4#>5wFSDTTfu!%#I5*9PQG>j}z?H%-YSMUBBGX94UTi>)AIE z9k57V$V7-7++n2E?`0hXk86hmFf>)Q2fPM3k~%q=&Qn9!h+m_U3`netmeHN}wWtV$ zG$Bwaz2FtYM#~l$nEAZ~ggfGx-S32E**YLkLp+K_MZ`9qJXU4R<$F)283h@$?XQ98 z(4(yiaujt1=hz(Vk z3{Q2*CbPpSH{boC8`;O&r|Cu?>h_QHlC)_q#<$d3!GFw7815g$x^tnY%rK`2Z1J6S zm`$15u}%XLEry%4`(h~XxGp~A97h}&AG)Ew{rF41Nq`9hO32U-e&p|oeGVY=YGcKw zj%cg$X#QYD{74Vwtx9U+fcZFEPm-e6uRl7=Nawd2WtRVotapHpbnCi>yJOq7ZKq?~ zwr!_k+qP}1<8*A>9jk**(tn+P&;9;;zc)2%jMUywRq(96_rjcO{!XA&J+^LJl;P`M zBeIsCH5GG!!R6kv!ec~7h``J2Q0_db5a$`(7WYT@2TkPF{1eTXL6-VJHa9`tFo_9h zCn6v=$O9|&EIH?_$(pY`eu2g-lFvCj1>pAWY=ZadEhUvt-g_r|3ZVFjdXX;&*e?*! zP+HccQ|2G$_IxeP`?;L*!p##p4Fe)bgp|AIog5_kfTpkN2>QEot!PgmNQ7Rk$7EYk zX74MRA)A+w+vgCY8@(I(*xzlsMdqz}4DrJAuX zZH7qGQ>utz6a4aVfoP?6X_JRPVHP@mNWI~Vo!)SB?H*0U=VUmzr~?tHO@d!B?|su2 z`(;fJ2x!7d{Hv}F-8_a7>nXLN8~`FMSo%fh1RwTwJ;AXmYubU|brAp9S!W4rekarv z2pxryW{xw?#Ca#lG4Yx?r~PLSXq$LZVa z8crB6T`K?IC|P1yAOz-C68GaeeH$DQOL(9~>JY`PsD>;k_giq`_=gqy=^NIqd8R1 zBtC`oft&p(P?uZ4K-@HPCF;2R<(X0vR8AtJs&VVFz2m-sl&VObXkI-eIVtlA5EEQN z6f2puaMSn5nr?nf43{|rD&bCC`vCBoj~gY+c0~FC?)w}(+BEv9nG}L_h8gfZRFT=z z;ye$1l78QMbG@leDmk(5(P<`_^imtwu6^sGA5}Cp*bXl#a@kNEN~K`TX%p*We)PQJ zM7E_#_1J?|YOmQxcNjrrB2!GK&FM&AMJUF~2N>PG*D+7!tnElIHb`uxGUI%|y<}OL zVe*#+BukpE3r@l@%GX*~%gc;n3)-Xf8#cqW$%oA}e$B3H^`aP<+NRJgwap|c=F-A5 ze8Yu5tAN5U`U?KWn-+KWEb)lr{0LGXC78HNxc5Vwa0c(%U;t1TRp;bZd9e^}pbTb_owqivLWvJ9&Xx{^#3dWNx4o zsTU|sJB2T(|Gy^4GY$MXz)yor&-ViQ6ZN5>NbQ$?pfLXeAn?I~6|w&{Ef5IG{V&u0 zZ^7(;rrUwZ2Rs#AtSgH0ijG*M03!+!WWnu!g>fc?TaO3g(h zL%s0GF8SS>pJ(GgRhFF>mVaHzgdNZ_-X4gAZ78An_}Pzb&-InK6Tm)~(W6&9 zyPZLWIj_PFOF1v2&hzd(0H}-e!lX=LtrCR}e~k@ChoBJ|31CGG7-bx*?CSw^sk%u7 ze!`0gMvmwZ7%-gy;(j!w@T16)f%|W4okx7c9da>Z{Tdu^eF@ma9K5e{s0qyP&}V8f z?M`LK5IHqKvpUQ`41M5~hz?<-;?GpTj(Q0(NZeJ%llNtzqZCRT1Dv+Nu(ojAqYWHg zBVEr`N8*VEn8zt{RCl7eGVkpM4}1cbP}YAhAz(Ws%JlpkASlsx7Lfvws$v6-QL+3H za9x+;COazNT7C}O6XEZeLBRwS>B=%u+>hkAS+p{3lwKhDhrJ1v`!k7~$i5gC4Ot!_ z-b!HavJMb2P?Sw9KKso)?0CO@zPtjZMA+;l<_Lv&g65&>s5+|lvVx$&lVQrxWNAC* z;m{@2*bXAV^ah=vDOCY%`x!9c;cB9Dq=g#7`5_@u=EG`u15)rqlG0cUj!J{%aME10 z`z*>h;*LTh;@T!mY%zx^VHXR%)Z02(#+GyNVH~8~-BYU_=bk4mDoEI2GkR46dYig% zzGJPC2-p)o?7GI5oS$$>TLq1j*2LX(HAD{{MUl<|jPFwdukwIIz`PSOic$CX!`Fmbq@a%m(_?Fn<_kF(m64yv=q| zSR(G{ztObQDbbKGoU@Uijubh%z$r~)rdaCpZKzvwC(>^=fwYS9G+w6!p|IzaoAgq+ z^J&bwIJ@wOR51YZX<_+qU9W!{Z=;BLu?ZV5V|>SZ8nCNEBR@rW4#mAx-SGj3!|%SH z8eFnR=-U187@X7lL#lF9P0Kx2z7x*B%i)!%gBW^VW0x@^yzai0q*XvY-DQw!^w@8L zl9U=!gTITZkB}4e>hmbMXV?Ct0PHgij>JapMXR3768Z=*_~lV%@=sn3M2iUo=x`44 z4XE(r7vF-Y6&h`QqAt5uwK)a#mY~sF^Pi|KFkSt`&Vm>l!jSl`>hmAqkh?<})$>M0 zkkyv;MYN+(C1fk##N|AwTBJe)^{tDb{aVuQryb)u{IzxWuzBwAoK(*q$x%GTvf4|$ z<+9%(5O3qemWG%-HsqB!G8N?Ph|{6~VZRzVN~TtoanG0sJ%uy#QGEJ1ey#sK7={H~ zUW$P6amatg$LW8aiG(>s_;lG+P<+6q3#u8~AG@Y$`}RrtmgOBHs1!E(k^(eInLuTR zT6#{MBKExl+5icU+pLD$Q;jf1D2D{#k9!p3^x@IhRKm>G zrsqmU_N>B4;Y8q=!#d)a;aC8o*z!zRJTrSK2(TYKES}Qg)5uzIXI#;TJb6Q`JTr}U zc;Rk*v#|C1EphEm%45HQ@;RU}XD>*8JOYEv(8}Ki7Y5!2BzdqA$rc+IGY8yRCcd(Y z1#_;=9UHcX+#8C_QXkeJ%i~mvcFeX-sTR5QbpnP_#Cw(_u^n$KLb3pctAlfKm$wZK z<0@s#U)_KD7pQRZ(!+K!nDg|k%0{Wr*ay~T5Sl3w=7<3t&r%}RYvEz#P_jNpob;`A zqC5WN^B$x74dl{9buQZT2Ju_bB6Ez!tVKX&TZ|2vOxO;}nhokIN(3U1k{Xt@$0!n0&2xtm6H1iy1UYHYubR(8bdq_2C(9F)d5TNH z{joIPhb?{;Lf-+frxZ4gUao_uHQWhuhK96-TakHBOu>4T7MQc3=uS7%sWp$Hs7g&r zt_PCbPZ{LWs%rWF{3RqsK@B!hv$iB{b)tUNq9enh4Rk2$iw?glNF-qr_-ibth8rx* zNILl7sEzwdej8+(7G_Jf)Wz$Hysz+pnSONW`Vo#JD;x_r2#9qRwsSr>Xg=Tz?ZAnu zrbcPcJkUOik}ve#BKR>SW1?5#4ij7G%;$~YI^4*{6FTRn8je$6Y#vsI;yY+hYFp&u z3Q8aZ)@oUr%ou9IQ0f|@E;Dg4tB9}hz8i)^ z`h#6ngPId?+X3lOAVK=`SyQ?6q~V^H>YmK>2FyJSr@W!CbeLTC1}W_X`eO%g1;yao zQNxCUlMSu3v*%QAye|1}FG^P!&ixiwV1x{kE0lB;3Kg4^Nv;n958PwEsMfHAN$QO| zH%>Y(cK#YRz_l$|t^J1M2eG0E}zWDM8cCh|%Q` z^5vM=qvFz6o@PX2kZ?y>zZZm$7i7>E^x&7+p3iS7s{V+xC$P`Fl{ND(Zd4nSEN?}L zDC!HG#U;*&Q64DT=yukncn(U{y7Ad9nuYR4Bk+9tBZi?kc7K9?$*{n4h?ACbFTr;S zw{i;uUQbf&QJ$_3q%D-X>x;res%fEiU7xK*?YnfycgtMOe8Xm8gO+%wonmUnq@qw; zK9ZL{Ana@-1pL7KHsKCcyjj>j-zOZ}lD$*%ValCVpkztMpHsGeIBmbMxc0kepFI2F zT%3VU4jCwVjebp1D04!!>+WhS(7NF-KbOu{)oe_@njx7~(@?CwmJZy!Vah2+9C2Ej zKhU>S{IKqL}B z)!4x$>E?aw`ScZpNr+Mut4Qu_Bs*r8TP3VEHEnXfF9=_{jrpL-C5<;zV04EFpJvY_u=Wr;l{IP8o*+h^$5^gq&|M3A`W|ttaQeFAOVQOss z58+_Jc(wS6obZ>R{sMW+LWC5YLUk13MHjZde?pu2t3LDwU?%eh`21gg-GAbhIYi?2 zjvP?!f8F}O;SSLvB2@b493o!3QQrT+i~p*?6_f+RlJo-5@P98Cd<8}=z{`QB-~u8> z`tCd;E+AJ$PXSd3jeilbY`YE(>HUjX->`eNo)Q!yQY^Ko|-#-u7!&)!$y8H-j^J+}(2zm%B0(OuGie z*m-{>uqzFM#^qR%%LU4XQAS;P7juSP2>}wc#s2*=p7Ay5N@Pq%x!+sl6BvBI$ErAP z;I!cLiTy;yXXPWpEA0Fl6~fWvI>Cj(RAW&h^NXSjp#erP@fD%XFKnvw3A^VOeH}}{ zI3h-Mj7yls2~V(ZkARGLfW(RM`!I`_1!?2+e;OJ4B@-x|0UiI-e;t3A|G$%QI}o-% z^RJ}@mV>GRLY3rXK$Ot<6{e+f-)ScMIFbDm7RyqVq8m~(4Z&J{2WkF`(v@q)EHMDCHL}2VNVU}oL`@xV^ z5(jgl$cNQT6OEuw?oZ(|bz4V#bXhyiEX#o@h;3)Ur*n`)M_fzHeadGxrJtvP_5!~l z6#lwK5V6j=%XY2HDtesTpx%cz7kPg&u~KSUh1XS?lI(qHl40_Yny!q3mR1em;kNvUm86m<`ciQ`(L9zwT3tUQ^Jyj~9WZ zt{7?J)iM16|7Z0}Sgbo)1MiJG(1!)Ce%SxJ%kSu1^d*+npwY8|?y1c8o$nB~o+VdVaHez&wyLCa!DKlz{{ zMApm4^vkh@qjRdhx^`$jmiB5d|HNExVarxb6*~51M3ju=HDUR2o5_Ge!0^-Wj9J5! z9;$|8O!R2S&FZ(-Wq{?-sQFQsPToSx^Ft&3hoV;&@gk9LHJ&7|DVFbvUzRKit*h}9 zC#4oXi>!B`3ABNtMP)JSqz9?Qxd?3`hw194h^_mTFTU@>p1+@LqO^)N*2bC4NM}LD zh#v;1s>@QXQhpJ@{sabp#K(0fz^`BU_qOaEPA=x21)C{Wkt)wOmI1tDdxEQ%05}ZE zGg5CVx8TSvW0Ld_nV)z)p3n~{qvy%mnB>@+6s_nPn&gja(_7r0nQ3#&)ABe;>2_bObr_J7R;cw|7nE@<(!qW+q? zCjn-aeuR=nk=?OQ^s5`+fG1bL!*=&gn9$eK1~f$>;366p7M+QJ}r7MO_(ZNb%wf=+6X&3VFi=gRJCL^Rn?0oV8jCE9>$*5 z5e*Hk14cmU*6NGTJb8h_kj>fBTmUNw1RGnF8L2!>1eB-5kbMry9BF~Uz$Q-683xjF z1Ed4Y8M>qT0Qc8Z@SakoIQg&TZd(IlUz2(4jpQXznDMD~=NkiLcrcg?hXy<1$a|^} zMPkmzp~hhvn5#%BTX;Pc0AnyaKzXnizmQ2D0nD#`1K{`u#vy^NERwb57E&)#Q!r)@ z75r_c4B3e>R^_BltBsegghSU=ooCva;~|-pn}Mgc2nrAAZwC~ZogxN)g}-bFc1p+! zi5t04H;}wN=Uorp-4i#ZcAlHQ$>DnFWf>euwJU&K!P(Kf+FXf@07AhTrzSAF9VC6$ zUG=WlvKQ4qB8NNeE_ta#g#CMqYkAw3W{UXJTo#a79h4_YR_?bYSx#>yInrs&Epww} zJvYz{+F;aY2xOG#i*;-&TY$nx+Jd!pI?*gFBeXd3>C8_% zS?y*l#+|<%<-5yvHzc?;mB}=UvFjC0z-jYoxF)>=sSU^H!^iVnLwT83*NdN->3Fia zREji+tqwEUPOSN-?onZ#HL`cUVH=6ZZ8DkTX`M3X{M0|G0Z6T05agMYC%wk+Y_;ZL zQwu@lgJ26=_#`+JV)rC|NjS2;Qr$--a+@@sOA(A(b#wAI-w!J1(N~!v^@^AbT*czN z-?Z(RMGjAt05huQGjly3%VulOLH@Q6r7Y*7EHanPgYlN8)t}a4v`aBX>e;*4&9Dqk z6lHW{b5GWsfQbErIQ!F;&3T@EP3poWEao>W`BykdR&0{}TafawO8dGkbnlN}3(eq5 zM*W^Le6e|8N&X9GxnD(s)N@;m$#92k)-XTO_?Z=|Tjq&SvKuh$611^1t)EBxWbe2V`yb@ZUsNJE9 zrw*zuX>D5RYnSQoU4%X^2nwsh`%j4ZD-y8^(s3<|%W?*gLmc6Z9*aOlmI9DnflBbY zBUavevjq_Z-omp7xH&@}T=BopC=c*d&<{9?MTpn&&4v8n@sr6eq{PXU*$T%?E>WWX zln`S#0rtz^!2V#i~8C&z<>p(->(Zz2{`GjJ%N z-UA;}Aj<;T4$~dHc%HYZ*b$eqO_}rwawEse1F$#DS0%7l)eUYnl;Txi?qU07$KaCM zIjOhvNG$t&ZF`$lyYqwpUHld3I`uN#G8G~-EukRjae^EXEQ8&(T}9$%jVTxvfofmo z#n%e>tTpFS5rsVvVk&QCRtbnxrn;6H`NcEBgvvuMoPMoh9NLuxiu1bQlVP z@Qo&ZKH(Pp$x|3+Wen9>F`DxIyWU;6uUbh0wV)v@bD4;5#eKPpTQj;=%rLRyGvn_S~29AbwOsr|g{8cyzHreX5O;_1G zE0a30brwsj;|Otl6ug=w32XI}>S^l-sdABJQMSVjze^wRF+1^b|8z+Ch2InDo5UCX zua6K+6^5eU?Z7?IVrViH9W}m}1W|)O2i+E{2?O;V04NL)O2G*s%_uXFP9Ue&1qFJ8 zp*Ubj>Q3lN(9Gzd9VCS^Bk;p&TPc0SqN-biX6Pn!sxgO2QD8jSJw}sotZZUM6Xbqk ztM^@*iv(rqo}>6Ymw9e>MknLjO{WZ9B=zLF_7^}35~2dxTZFUtXwd^)qomo7AFd`N zwrMig0EPYTQh($fUsWzOIfZ53GBc7~EMwjp{U72Yz3iAlAN5V(G&vI|elJB0LEsEo?_H#{a!$U4@ z_B>jX`V`?diL-)qvios!w`26dtV`sf6Xns904w|1^K(A0jgkPrylZ7=YD$K3NpSO8 zAP*y1IPC37I4~O`I6mf`#t%M^;RI`=S$&=j*y@sQVPNR@cBq{+yChGF*mmZLg6z2Ze)&$C+4Df}fm&$hV?<4vwDd76@EyvBXDFS%$R7N?Ch7E0#HF=;DPwZ3=g!UPT{^!5^zTXjUw)AP zm^}oEBF_XcPyhw?{{COi!`~tz(ClH-4Tga-@bs&Qkib?_TK`!9p>gRwBlC@%{Jn4CxuK(#od18+U|ydQ~^Wch8rpTZqZG?z*1OQq>LtRDT|s)JVHK$P8l z3R!ooHp+de>Rc~s6IFRzaEn?NuF?Swi)94}{kqt%bM;-^hu=@u%s57^y7gcuKd(la zzQ$8ro_9MRHw-R(4%9)16Cpp3a9*o`a>&;BNjBkrm}Jy2bfoGH_ol-6XnUtgW5LsI zxBuH|vgwS{lGST6>a^2N?O}z+j7&eQ=bF`${W^W5I)Lh2(E1V*(vNcsc0tbyI3u zu-}<)w!zi^N!~hW0|nkXDQpl4R(`l;6o;P1Epw-0gF*WWsu%2lER)=~>@5{B)r zdBPggq`b~OxHp5>_~9_m>G5*?JUeAXDT~ANovc$%X&jE?TyknIR&qWvqa3zT9MTvc zwj*3jdSp@;{vQiP z0jT_In=)evLo|C~QHl+)Oe$!|QS4Bb*3xnbYs2lE=c)P&GfP?#d@mx8h%X1A6byU8 z^%S39|HOLRtKft}o)7VF<{a~$PQBh;TyTEDxVFkzb(9&hgZE&gv+k%ik{WVG!>Kzn zUIB9fSpPlV?ViI86_L z)ZvU1*&BkR!VTSt`lsJGHUG%Pe~_}$SQ82}dR)VUXtF*5*}ofjIuo(ryQ>k(XN?(F z2&eI(&hzNSPhRcRLoVCe(lr4`yt)5C^+_PB}?*QAEDPs^w^O|>K^ z71ozdQG5Ptor+baf31`GuXWN`O-7Y=+NCb8pb(=lis(OKx4=BZ9;kgwdgQczGR!|H z1=bcR6QsETl-a-lmaCHus1V#u;rb^oX2Tr^{bXYkedquX-qe|@v-_*#u-~y42a(F2 zR+UsT&o77Vy_t9H1RG=KdMAj&k}k1;#EKejes4|IqGrBuA>|yt@}g+*x`Z0iLcZzzPgW10JEFd(o6 zFdxXxl)^HkYFe_u0zX#f6E$#3Tgq!c%j5Ta{%!HIK#c-A*D`Hkr#uOWYzCDzhKH^( zA;8rfB?U<+u8)U~3caI9sT8;Q<^VfPH1r$w3|2IGm^b0&ZMUCxYGoDQy+n~tz7%wCM}j<$<#32piH$=SF0oALSJ8+ zS$N}FD-s3@m}R{9e78-2-1Ta0 znM~r8>Daaw4olC6L+(lV9X~S|ya~sY_tWD+jk7lUu3B6*T-5sc8;79Na6^>YCLh#& zR~;)IB0^Z*Dg5LLBm75z(bJEcmT7&Y`(J|7SSvbwe3|~_i>VnQMzKBK7BJ+jZz40< zt6K<_ACm1+H#+>p1S04A+6i+2M9S0H^|Rk_(ES(p0TA$%2BQ6aqOhPFNtDL4MjSAakwGk}g6`k(w0if4*mKRTT*v*Kc(fi5z=HiRGJ zYvQYRHN~3=2<^5(CFgoR%yQBMCb$I^B3{%LRB*?{0yH2S)H2k`fl;8lK?;g!z5zc( z#0U;R`A;eAHZJY$`TT!>d_M0y_?`aA1-h*>&!xX-Y@RP6nesQ-e3gc-aIWVZ%K>4G zRxhPt23A0E<$Csa@gY>4yk*NEaa8uKQYFpeR~&kcyBujZ8aZ{VC~5!HvMM&YQWMoG z4XZBcu_`v5(nD1q{Zeik_9`}w(g10pRACzSO19b9^$yX!UXUBqHFC$?fpReWdqWfm zzT170z-L%r)jboCe599{0Sj;eO292R;?FtC1e7&~1`)&Zfl!bsPVAdshA?mZx06Se zlHN>mJ7tH`fi__Fk9(Rqj0xy30R!j+Y((^qnFDGdgqR&#dl-QVK1{HyK5K6|0~~>{ zUToBlSHnd8N4uGc)wY`Vb zjKE1+X4G`G11{atLn%0DU%fq{KnGYpl3P^#*~4s}&!C&Lm+F{LctIpj)iKyLO2_;G zPB4VC7v8?IL$Q4y=wKdwj986ZNc=xYp6Ch}FXgu%xC-Ylh<)XU{V!?q1LM0%Xc%Pvnh&-9&Ju zv!m{#H`aU~RL3Y=;NRB-(CyfSOxy7e-l=XtJ(qU5K$C9SnI2d*3ua4)YhdY^bc(vy zvNss3&fq1Io-`=g3-mOt>#6cAu9=%psW$ayr()pOEai_sWlYF+2&T(5m)P4jtEppchFcX6Grm9Ronqfz9t_RBUY0p9Bx9~3700(HZ)T$M6h;1W zkCCOeLiWA*d#I7d4c$ymp~_=jG}XeicoEr&XIOZNm$};+Nx6VEW(q7_~RKv(sdlBm~ zU1pxCnBkTyImg6?OLzknZT*{6Vpq0K*;`j_hiX}9Wx!LKTB9x9d;?3aeCg~7ex{|5 zJGb%nc1UFm`SX|_JroY2HeIL}ZM%gRflYjN+WuAS6D(-|u#d_43SD8+Rb2n%!tovS zoi2|ZSu^gGfJtqpWCop!JFZ?QLEgGvM4sb4MEtAvWK9p&enxO;X~JF6=xe8@FjZ_& zErW@v9>7G0F7Z%@(#Aa-JfBa8oJ*|!k$`kF>o~&oxGBl;eLYS3=q)L<#st`*d}q@- ztrp9s#i7S==^C2+6}yw6^W$D@GHHD@yOJ^vHke6J9M1>qNJ)JLmAamkn8SQ@vn!a4ZMw z$Xf00sE(yHVg>z8#-S)_-SYbG-RNv99l30ysFsNj2Vb9)3zKwg343nbRf7HCZ~jkA zIKZYBN==-ZH*f4+GK{twc_4cs%Xi!r0hEv^fjCXGg^mLL%!$z?Cv!K)k%!Dj9)HkV zg(}XiYbco=#pc6&Q;8~vcF)O+?nR^%{B5@|$G3gn`~BJSbC{?~N=>N@iG2&QC!5~pX( zj|#J3)W(j+d0E#n6Dup0xEA7R29=#lmS59#6e>H(Ihn%!l}8rTP|~AYR^x_%!J{Te z3uhGNXyNIynx?~G%4{5JyU={y)*s-FY(51)hLSX#sBngTKcm>FSebBRzo@E?8vu-6 zYK1?MO!y5(7VlH~gEe>mjH;|W!@bV~nGOo?IUwK)2V+FN#uIpKSJ_n2J|vGxvEZ@LU29b zqOoY?^#;jEYELT*jT|c+28R7IPynend?&nN4exymKbCEY8lZWKxFB>qmy_=nNz`EE zKOu7%R>IcbAc!+D@dh_~Ef0W7m#$Y#C?Otn9tJ+BOW2?peTZ&K4$DXw55_XRd35mLIhCRKE94V`}SAvA(@Y6C9eG~JJ# z%sbPT+wXoP23GY?(Yq!?*dD{(~rKI zC)soYN-{b%t34eq!$Ro7zyk;yR0(J+R4%YD{>`(GDcH9Kl@pnv?r{2sZ3e>;?31PfR+YnVhABd_`&WANR0aA);abrh`Z1{4&F zJy`jtO8pe^UoK}qb-EvCjc4y>EAw#PRW{-K)tz!sePZeKdn-+(9FuN-RI@0y zU6i(cSkHWdf%8xUk_kX-xGc%O8UdC^JA7f7;JvRraaf}#kKZQ?SKgpReM=s{Bbehr{^wx&84BD&=^8+U@bG4vt{>ayT>a=A zwpxwyP8qM)CUyS30&TJ}C10))-%jERw})d+nC_vr6n zz4Gq`%K2kOmZEn&4_w3>2d25vAJSJS#FkEXt4_b*@J%G&qlQk6ady7!D2$vc&rBbZ zy&gP1t{lz#f>r@u>_Gaj!hY7qR`i&7v}tLC6Z{6d#LcLPmU;zk`xLDn2B8hm-MYR(BFS z+XLARFfeBI56$(=@fk(!$U)4w$T)Tv$mp92pG6|%GdKl^oM(GtK;Gi9^3Ae-0)^5* z2(=lJ_syz?X;nvX4T%4sDqXTndRCWU%DJ>q36N9qlOR1(Pbb+q?3OyV3FxfUcRqV( z*rN!RCk>JO7D~L_AM86^@XHtt-}(`uEV<`~TwXkK?DCpBwY%@}rkitDO=4ZCK?j9# zktE_~>of?k>Oqd$<&VjOu{|mRl84xTf9Nrc*56jJ#ONPfKEMyv418o@If1`3WWL5F zts`H55~15K>>ZcXQzD)?vSmq)DaVal3z*H8;Mkh4ck=b6`gTP8Y>1|KXnb3pXQfc# z8((eK*auxX$4p|w6CRcp4iySJnyaS5^B!Vad*KHhvyR25jd)=pq`bT_D%Ez0uu zrU-{1VI_U}?;24wE>Z2HS$-Kf^k~;_#5Gjyced)OZgkgL=`1fjE%>Qj9nV|97$!2(^Ra5Q9CYX0;&wMVA-md#Asx<7Tb-Ru%qMCzqYbd zlAby9_#Knb)*MDtW~{G;)$h{ePo%)F+Qo+}h~xp`*1jBv{s-Q6{#|a1nVUcFFRTIh zFJ&mjsZ3(JNOwUIm|{I-2WH6i5x_Qt!!{%gk)C?PxFkDhcWDvrVm+h>KpK>oFVz7l zGGX+K?l2}vT~r3jof*2XXAR78j;f>bgcLd@3gvRnlB4q2gf?E4Fe!oz()@^2JQy}4 z*Lgw%${5+wVL=E4UU~zJk-WU5W^jUNmjw5DY6HSmjwl(6TS)S?b&a?M`^q7}30sQq zyHrXt7Nx@ET;hqTtT@E%j|NBSuvkhnQ)W67JUTRq24DT!sS3Oxv6oviDj#_zQr0=p!_jG`4|~Bp-t3EA zROZ~mg_PvCG*X@qJD2-AUgDVfe=vmb`B)sX+^f3AJ~T`u;kjSKxj4MyYRbc$FqOWWOTc?m`Y+eqru704Gy+dkOz&|98samNBkoNE3Mq;ORsO>{q zRvMCIRl`)&8P0Bz@>Nsv7gx&zif0qSHi^$;aOsJ!OxPtV0yAn?DDg(y=G698<0iv% zD&%Qfrm31yaX4jq0Ft8>ehj6M3b*}f9P}A7kzy(Tz}?#fr$fO zGL^;YOvd&`>#mdw$Z^Q>&6QMN)MrO_kcqn_Sx*u#Hg>tW21L8#sFGk(yV>+m!tpy= zX%Mh8Gz}w7LuOS@KGOEK*-c@_@(FFT4g-T zG@pCt_k-e6P#uIRs%?>JTm19@w|F@4-=Ow$t%GX;b1;&l;|j*lE^GW4#h->5+d^2# z8p!NX4_KLJ0r22z+ZPI8^hElBm?PzgT?qGrwp8{-BWoydNhl?78j0*t{DM83Hd49d zR0WoH#HQJoAj^?6f|g!X%y!w|bB8b#8lnaST|DPd=_nl;!l&~JMEBac`3|7fQ9Y@{ z!}-6%fFyJC9YIHdg}JPqLBB!4oy@6%ada5E9pZ5y0kkhDik&d%{ltJp2t>HcGI)Q) ztOiW{wQ!d;h3<#1x+6mG0Z5+(tM9PkAvky#D~Yz~wOf(zxKE08uef#&uizKudmgKBC3h)EkLCa{MrFbb;vR zI?4{lqlLY#&^B)j*A=yx=ZM|r93L$P_h^+DM>zu-a-vj$*w!F=V*Dp7_w~Y%a2i-G z9{4Z9=|3mI->>ux5D0>F!z&=U?{@tknC#y(A!%A3gc)h9d_4RBl8nIbYX=uW&(*|m zM#48XB!0*0Z6|uIHn~l`AzIH{a!*)9fdE9at=fP{qQE(kKNJ1AnFbbbuTMbA73N&c zcP02Z?4GtR(mbaW7~V+v?E&nA!-8tr5M7r$S*m8}pIot~#`6j4>O5u?Yn0W>q*R&! zvW`S%dNCL?s4*H@lfi@vqB zk+7Id84O>iVJ+?HuzMa@h)zvW*0pF+oIQ7M?7=y=QtT_8;Pl;#wn|w(b-a$hgu%^x zn!g(HGA3XxL4KOwC1R;|B9-6=FGlE^$KxS(9zN_dS-xou^F`kNauN*%<{99tWmqj= z9t`ro2dd=SDjTX|rvwN!(wIR|DuUl?-)#i&*Sl4C@rOA3vnnLqkgxrL_~#xlM8bRh zgL`uPZ!j8=e8%w4f1M0?2QB{LFY3aF?evM?Blq2Uy(=?jSt=%qo(1HN6E zv2YJJPROoIh5Rr$6FhSvd^-SLQ!lAmUM_d9{fz!hT=6qE9te^7?zQG(?*rIf_sMUb z-Tm|Vi#I^%mI#B~lzd7mBfWvhND4?(WF+HDye$o>C-dD7!gvJ06o%{#g$wJV8&iP^ z3P2TNuL>uKR*#@IA_+%?A&DD6v4jm-3hPFEGJ$lEh?2w*F~Bkt4Hv}lVj&yrj`Ek7 z%I>$tu#K9U-e$&FZ6O_?5(}g<2Sa1$lTw>t5Kel=#GCpC>#(TuB_s}2z`?JY9Wl>1 zQfzQO$?B=47Fh$(QqH$n>$oT0_G?Ve#RJ25cM&x3dMirs>?`A}=0k*0%hVK?nyUDZ zqx!naGTenU*cK0dxc=(PQ-}8p-JFTBUe~qUvRL3A5^Be{iIt7Pw=QC?$DbgdgeQLMw>KQWnRI3> zIXo`TewP3Q5w}A~v=-T)&0vW~ytpVs|6JAywomR|Of2lZiz}qHdR_s6C!7$GU_I0wYyTotFa2Ol6QgL$4n4K8A16f<#&r-RT zKT#`djFx_}%{pgy`;DvIz1=P|7q&|=9?4#Y0Y3mBc8U>$L_+ZU5MfTzXs0xLEZ={R zNE=zeVqz^s(z~+02M}22SiFFKFO~I$#q}fcioQn~w{~CKUD0HzRyTzQb8v>%?IOfGgC=K?PY+M_k>b~*K@9ysomVR+J zA~Z)^0^^j~c2L}p7mYfc5A|Ze)?4xL3w5EOT|&*IPMSss zS*C~FtxcRpiL`a!`6r2d;unseT$QH85;<(TAQE`s-n4Qn2kRFsAcLEhg^KR)F{rq1 zDRl|HM`bfZoPX;tVq!npfwh;X|7vQV4lF-2q#JG^q5)EYh-O@7B)_thlv^bjH0e&K zZ$h<>uRqycyFL@#Z@yrfgv0^k^Gn0UU{5(*t7M! zz_u?yQWEGUe2>b@91qv5U)A&+8E1)|S9$VQ^0l)0TH52bg&vHTdXk{}i!N%eC?^Dl zt;=fNJ0Q-9BDSd*XPxK<6Z?E1EV{r4nWAb4pf*$^Sqr<7`4zlS3gFWq+OW#g&t5v8 z=~QX@uu?h(N&4gDZOUA2&fjm2t<;f+wV|wl%wsiEPP2V%v?TVI%RL+%?D12n5ycG; zsYa8pmwej#VW<7a4YI9)N6~WW0)*F4{l^>ZJHV6t5gLBtX@cBCMW@Nh%9m)D)SQ|X zPd${D&BtpWwJJ}@h@bq^MzshxQC9G!(S9D9H!pawiA%*o=U`?{6&TYq+2Et65zH7L zWqB%aC{#?8FxJ^q8->QR2V;%OKP`qGN8JU8PjiSkp|3fA|NqE($LLJjrd>D_+vdc! zZ6_1ko=l7hCb(kTwr$(Ct%-5PPIm75S>OKNcRzo+SNE^(Rn=AJ*>#{HhY2e1)M|>h zW0+RR9_>~X#lVDl508zs%7l6kQ(t1ECp^Ti7T*CXGQ>(VCIYtnCcA&^-)Wj2Ov7Q` zSI_wWcnkkOZ6W!~MMVJ+_b;nG=+nkaDq=kdImb6kCv5>jZWojc88y!*M!5ui0j~<* z!>Y;AdOSF!3qwj%N*4&ir@E*`Sx5!52ded9@wm+S4>tYx!#<@C)X%e(h1$aX9~i7H zB!dYolxF&4e%~Pb#lU`}HP`+$hLK0Zb8Mt93=|KN@F;fL1Hj|++KX&onukKa7sKDf z@lJ~wx{9U6a<6Z38$YdN$vqJr-GFQ9c~FSN@>?kCOAuOUt{n~kAT$kD0%Q;K-MJlS z5k9SYrHwxj-CUrmE7~1}HVU!2Mj>1P%LLEj0Lx2Ew_# z6jQMqwckG88RAbVth&-5?qThZMUIjO3T~Yi8GRd7uhw%!#Y(%G?l*?qMDy^O#nKXv zs7ur~J`A-XrSzHm(w}DdUSh|v@9`x(Q34d5Y65$aPXBz^E^MfEf4@>;_J1rNAfZ8G z6fs5r!}CRE1;=PUI{`!Z&x|!bx|k^WpL@wnXJEttc`NiUHLT3dbGfC(0p@Gc!~Q4V zbh@HNE7cjg>$+K?0smm3D>;jEruogg32;YLapXb4o#_$rUjKi*Db%B7G0_!_j!RE9 z)T+w~?{AkMi9jy*BtmG34@Kz$G~sJe_x|{le@g6;93?AI6-7E3d)_; z^BM)%xASr#2m4uw6ni#1%M{~%k_g;dx*F<^1!7!VjV|t@L`TN+aY`pjXN{1!j3o_S zcWq#)sXr8cIhgPWoMsPj=(`s!ffcGfhvc?eqK-20D{2dr-{d+o4lt3Lx(^?$vtPG7 zNEgB>uAb4JxTQ<4^TA!Hu5<5 zy{wL7K4cUzAx ze+cx=`V7%|3gCT`)51D?IQ0G($ML6)@^knr6=45ID*PuC{#VeklfQ3#ZIlX?O7O_3l#@kXqn0eO3`83=#uHhY4&E)IUW4V$(k}u3-zDBg)n}@=+486 z;R1p^DTPSmwxNcon629J-J0`Fr)76{eS+9yaq{Ut*bj=X_p>8~6IQjSI5A)Cm4ydF z>Sp}v=|7>+PO;o0F~|9p`4@l%-^}CB&y324Dm~K7S{`SAVW80KIfwHT&(SF0Q`wPy z?Wn6MZkT6;bQCQ0MLktcXDhy)B)Tq(jRnSZ@9Hti#EMtE@Wm^!O05*D<#e;gW2wp+U}!W?y@d9_zXXZ;9$wX z90C5#hDAw@<<+ljVEwOb_;+6ypHPhZf9n8t^86JTDnK5Lo*|pOu+S&1$xr-y3wlhB zQfmQ3DF&1+o`)4jovVfwuQC7@Y9JUq#Dwv?!wwWMAK3(r{hu_bw0PT^xb0Ycd%J&! z@>y>=+>eVM7vK>{^3uJp-81U#&kLp{OwtD0mq8E}uR}Af4?#>YFSJ=PVym~m=VHTT zK@;2pz6?GH>^{O15*6CL_Kg`Tpbey}B!c zdlcE{Zm63*`-S9snnc6YVUl>lCUReXUtH<~wGXftj6lra9in(wFt0); zo&yM;ee#||{GPdEw~Hl5|0xA1@A70*U%|)yui*dhQUGB453K8}7|7y(HG+#O-hI>G zzi&&G98q);RwNu*VaOQ%Xvqs*$LL(iaa*RI5dQfC4p8|{ffa~<19p=?kS1Ig6BEz2 zsivoPrN&>f%_#`@w$djPQcRo`tceID>!}9`|CU73fFhnUhu;4M@N#SSvoORt|TIpq2srsRB_qZ zTMR4ie<63E`xeJ_Aawn4G#E=Rk7$FEl^o`(C}AF z2V;qR-$_i}8>j8FUtGwR}Ur;#d72^5g&DKK|eMCEb0sZ#`MOFSfFY z9LH*rc79`}@=Km`ydSAZwlM_zN^$``mDlmQ>0=wgYNnRCYv3DQpaig&XeS!=Drinf zHrq3$)!-C0EjrFM1T4;mh!{vWr$yN$t|0>6GfYX)o&p=LGKhBXP%AA4iD z5y3Mq6vltihkz#qugxU05XAGU><6%3b*(zHk!C#-Ch~-i zT&PB)0TpacM0jQ~JVnK4JvVCEk9Zg|X&TrkrUqljSeBI39DT8z0A2>eSm>Cbm8yRs zT62~$xIF9Rd=&r=rqVc8@#Au>85weq2cH=Wx2N)2>~M`YDS67W*-y% zI0%6bD17?*$D~R_ovY2BcH>~dkgFj&_ehM2-DkUF4wZ$KCIM$m9^{L5)gKyE0t z1y4OSZS+4uy2`}HDR=*hQsRF_DY^L`O!v#`tM&&NGqTt)vZD|fKuGkPJm&0NFEyLk zck#TO^_BGg*y*N+sn{E|kX5tx4-& zW1*w_OB^%llt`Ns;@QDRV@MDUv87hl-gP32LG>C_Ku7=>?}>a_!y6MyCC1@)Bkg|j zN2Cs(fU8yn zlIuYQ8hV9F!ZrH1%8ApK$(h^*lyWkwGzVm5_zD_FAocHf4^uaTGMhy_6VocxbGvv? z-5k$#H z6zT?JhmwE2KgUp5u*|Ve9|QN=Nq4W}GlB&V^mCpRG_9(Y9X6qQ^mXzwM`1Z$rc@=G zv1XY|(y7=lxZ0amsepE3_nOAZO-<>ouueL%?fvlQ%9GffiYBLcrdxlizoCkys!$u; zscke;&K2s8Ee_gAanc!(1Az6zYExW`u|x#c+jqdC<>XCb3p*tmjysEEGnQ8-B_);VHg~~ zP^!yLq7*Y$_MQJ@-0Dv-*bR3pL!DwX(iD?JQ@5|tVRv?j(K8))G2hS=46$|`*?daw zU%Ht?7WkCOd>!A*h7eQ9(RA|d0jSe>6_4Xm0E9<#uv4}nV5Gh(_JS4uj5KTsk!Fd3 zrX@7i{|ySg{usaXO1UrVEkYl%&Me4_%b6Mj_#idHtZIeSza$X4F;oZ^tHgtoxhY|K z6r<0UBTldj>5f_T3I0#L;MJ*r_y_VE^xt{){0e3ah>ZJJ?|b;u)#Z!>F2cr#Bm^lW zUqIE%f{X%oKE}Dl6fU${Zdw0~TyK4&DcqBi2xvq+D`xOCg=EIJ37Nu-Hg~Y$@$u+N zczN;sezcAR@@TK}#oAp~t0-$x5`sK}Q_Lwer(vHLXF_LJWhys9#A4I!^9w33)#?Wo zZ3HygUTX1=1L?NewQJG4UPE#{$tY`q@G4@=p+~>Y+ToF`iovGx3Tj&scEg{T{p)bJ zGI+{4(=L*$Coa^}jfT(#wx!z$7}nK?^Y?EGmcBNxI+l=|5<6ss+k3I};@D6(mLz<^ z+Tz$$(*nkL&S+VBs&eUD-#m=m78zw7nG(!U~birH;LcMf29 zn+7CBw6L`y!MCkrBDG%GZ*SSPB#iUPP5`z{5~I?V2Uor)5vZX9iD0!plY_^4s zu9OUkm?j1C`Vg7I$5U|3vs~ZUpoViTxoa2$zT9%0_K%5kH7N3)9qxE@X@`&Z zpeJh_E-4+xsUlknQHnh!Tr%FoUC11OnppoKE#@pUjDUV+O5T61R+GP!Cn^7#K9MC) zet;nXINTZ2p`5j&5;^aZUfznw(YxvL+?`qEk$v=?0*I;>o)T*7$&i_)Yz-Y)@ zOj;@vuYh4r{DoxAuYKY5OK1I;f{w>B&1dT4mi{Ox_s8caqu`;JU6Tppxw+RX3gHxe5tj@eolV@f-PKEcv$c$v51B^Hn38*yAsHaYD{2Cg z>3lV?JNkF{zUkTq7V_YZ~!SHT6NdEyCUd+b}qK zt1D`7K6-by!Dq${J20Ah>?(GG40Li)yln$*fN2PwbCnW!7?6+hPOeQLGLkq=8!*Bk9@R-acMl~z2`cHuv+DhgzU(96R$X;|vM96uZ zi8{Y9cX)No9+m-6%|KRcRSY2j{aavfI=DTBKQ}}TBm@{7SZJ>&BrR+XqAa$qI*66K z9+)t=XFoF97ge$LN4kgHXuP z==Hev@q089>V2)1?}cKj+9?KZ!e)tO`(+!!2#oI#!&5&&zhHR6H#4sk?qC6NAv{vH z(cq`@BzqZmB_Oi*mW#Cls}1Y@3ZN5DO-Hvn&$>WQF24d~IL)=R{cOtgmy0T__=p4cgNlj_aBJ~k_I3+$bK zzQ>Y#l_l&9vxNXm?r_5Ei^}&dJ7oHG4@T2Gaa5@T2{x)j_&StO)Zao#V1bU1SIPk* zx?&hx6(PSiazk+Em=YL!qT$$VFW ztT1K++EE_rq&kjmL- zo-|Oyx_Py1#e|((Xo)C&Nx{7y5$m|;;ac~U5K zUd`>1V{w)SH&CrgQ%K(n*c&~lMIp)lufcixFL z=l>gL4JTbjt_AcrZT(4AU>STxM^JJp&PG4lSU*)H8N`|WH54ONe*7*C zb)mmnFjeJxl%tNQkplmy$For>HPP8$(1}cusGRcJX1x)d!k;oN-+!cn1>0mr;4tQ8 zT9wK>`l~X{ApE!4?Dz1;i&0zlY2aV2@}R{W1%#&G;RVTB5Uu@K0J|-r{6-YYbwy?PhlBSbt81ZI_2E@sg_nY%G#zRQ| zZ6O7mejuV=Jc2*DUU`$p{)gr%-~5yVl9M*O+Yc>Grr>pn0V6Uad{q-Q-n1sVamR&4 ze~e$@OGS|RK?XLsbhAFMg0iKk_#E}bx@0GKTYxt}+b+}8PLDW5o(>5LinHLI0nWoP zwN#$kH@gVM*fI&b2$_CE|LZgx&Z|NfS^b#)?%+$DH$u?`izk$YoEJ7b63lfdO2f55 zzV9aFWRWGaF`qGMU6%2oBo;bMcac&ZRm3Xy3ZMOgyuW+4VC4I^q*Mv8F(Y|i{3vmh z`T@Bm5@&5?D-73C>9Jx_AA;vjp#<psj0nwh!f?vZuS9uL7B@z$JSPwnzE(dj8io7j16^nMCMC+LT5qV^M*Kn9UYU^ z5REG`0Udu2dz7(mO4-z|Y`65!s&s;x{s2^LRAfzTHV@Yj?k^E0p#yaqa!#Y5v=>KQ z?8>v%quUkj^tf{9wE5@$pc`G!&_p+U{MnjG4+=~k%Rh@kzVaSL&vpFy&9BBr5piw1 zcWHpV`=nN@>r3x*;Zpag%C>ac6vj&i0-pzkCp z$+dO5oR?WPJv+v#y(DK2+!L^mgHB&S^lwXZMeFs??5^59=fLxk`*NU?`4 zw@d2>R5hVElbR>))o*V*wz5Vm8WgNKk1;j3!2x)=vC3m0GKH=o3y)6~X%hpADjT=c zPQl3!)3Ezosd^NLdd6C73ljWs#+Dl9ye2EjpEiyjHyYYDJl)e$tYm<=Jkodp+rnjp zSVg{!8s+!gs>oa)8I0~;L!bU56lFoFrd>=sHi?SWG6AsolaGCze7nq8Dyb;u(Nxu# z_;RvZI)z{Ir&HgdNa&y{YqyuAf;mgXC*S%{v3oJtQ^;wwZ^-|!Ug%BqfA|iy+zIX5 zG`U_o7Hyo|53BhDZrA_?KkOFd^`&?%dsEh8rTpSB^{}9w#B>^u)mitbfQ`pdJHCd; zO!Tqrx4c}`(yJah#EuUhFqmIxCl*DC3t4H7*JVZH%reAZq$qaDv5a9aMx)9m^@~^u z1@*a>&&LyKneb^M)^`%T_bIgqZ^zlSC~DkLs@cSI#}j4Ye9VCSs?nK%EsLuChL~EJ zX_Ptdel7PRwelTjY}iu`|H$q9%&m~qa-dZu3S<1y^P(YCaVj66M1C0z4E_5z4IS+w zY$AK|dMypJ?-y;g28Kb5%Y8nyJNyNZwEZv^WKFvbZ0tPmYTZ#_IKJ^%IU{@TPQL|S z`1s>X=TJj_qQe0ogzxRWm~d|)pFirrf7>O9N@v&Km!nd(p~yhJ!i0xy8>!+z(@LuQ zlhEO+Lxd>`PS}})b^9tCu{i35sZ6^5ww)>T@|U5T-YA5kQfJWj03igGEe>Tcn*CVym3TI zhbR2#Ut>(xIsb+-rx6|F-3xt!6&2%+()SWOr?fShr;Jj;h>cjo-m2v*i_{iVXYV|igN z88QXnuVs7|qB~&n3QfC_k@SjqKF#E#E_!|@CFwj}7$Pj%(LD_6to>1ZsVIg9KRCDM z7Iz~gB2&bU=NkNxIb!LCdxBPgo1QEtE!K%r;kNfkXU9t zDLE%z;EVa)&)LGLjDsDYb-feI?TYlJ2XRN9Q zMn_o4ZPym6(wQv!NhQ{?E{bR%6$yDXfd6OJTmspKfCHBEqf_VmXd=-6F#YY9%;C)m z1rI(91TVhudptkU&lHu!e6}^5uvafYkW%^4oc9x93Q~J6BnVl}?KH8a>0bk}IVX}& ziMT9|k9Q(|h|fa>z~OG>OFgVz(Gj{(2Vpgt8(1elcXF_s!1+q&JL2_RJBsmaPE;!pC z%^N0sN26iec*EwB6At|S+W~9zjjbEqU|*R+2P8|!4>EQvGVaangGaF&C=xBdD|Tsh zA;kA@D^BWi48>?T`SCxB_T3I3g@><)(c#joKJW(irrEr~D0tQy7{6KGws!ikRi@~LTReT2x6{!WE7AGA(StyNhF%-4AoLG1h?pw-C>ixe zB{@F1BJa6*d&Khs*E_Xk(}K@l#P!jl5m)-b?CVDY*5Vc#HSukvn@ zO}EtNpH(frBo?p9d_s*&ZfvXRi=g!jbhBpHe){2_o1a7tZe#ADn{o4C*^Ym?rFDxL zE)iA-=w)CvOv8BT3F46-gQ2VX;+c3epIXYfsO=p@4ClIm_~@~_a_`apru8gIstz33 zkc-J(<5`k2{EGlM!6{z>O=%17N?DQ#wCNtMKNqMy(NBUeEFkw@^vm9cgBtwu=XE)nl~)%)@`W+w-ayp=(j9{xNxtA@xxgb(`P|Q zuKkrRt69-5rBDJpb#>0eLQV1}qk0exB|WwmK}e`w)B6CDDFewpz9o32_kgJ$X3Pq# zF5!o<=A*|x4fn`&Ndr;kwo$XcmA1|p2Mbi>VVyoCjegw8REu6UgY!`9d@kkkKm#Pt zJ2q!yB+mjo0cU>^=C`@g+vsBpSvrAq*g%DTUEC;tU3>*2!(dA;9P>T&7O>(zFfK5{ zzEmLFlplc6gxVA8C=Y9#$>*;dc%~b@`i`DvDCL*xkggd#Y0PPhC9pw^xbwpS+BJ0lt^=zY z`OzfY4f;`OWHb01>#%h(+mKdC!qnM1Gy+U@(J_Fg;J9tm@}=H#bNWHHeFYKKMTpp? zPjx}cL~w$5Yp|fdJ(9p4ZZizE@-umK&#B={2o>5JiJEh>(C&n8S+o|9kYPFV4PQ5@ z^}HoLSq-~>sP2mN*aCnbWm?s_F3)Lf{LMp%!9rvn%rTxY#$u!J<+oIszLv=Dx_EMK z=pLZIGxACb_Ja2L@6v$YbB%ZPDPc_OwbD(zz|Xs!TzRC84Fsejx+ci5pKi#e5q;nh zz19(;7=(UmZs54bm)o)&ygnMj$kj!6ZU`GP$z%0y1RXxh{ozmEPn6y%bg$0=X1m|o zemj>1T8bLd z#)`u^d-=-jmUifC2<-wV-O$&7Cdh-l--!TzMk;$~4M}zlO#+%S>m3%dT8!eL1u%0! zc=VL`;tUR+*c2T`I~pn>TH2BdBOBwYjT-^rpYvAk#Rpxu_wmY0U)E0hA3Y53_WA%C zg+;x3T0SXCO~BgMFKfUwanqf zWw`CVU1P^wu0%Q)k&Y5lJ_cz)zOhvcCxsq5z0|;h2)&~A!to${iz5E1o_WgYZAtP7 zIZ8U6vfnt*&I(!Gv$oRIr$NkYZ%E|1%~XH85D-0~4xs~JW5L+_K*z-y*g^q(Eh17g z#HP?;p2w)_E$Zk!P=TP7gFoFA(R<_jAQeKH4=M~4xM*FJo(Hp7$xkZUcF+G7{r$6> zZXIr+N%Q0icxB^~KkL|&!=^OCiC*W>=bW0yC=#Z#C`2<{MyT_FSB_=sOIFCblFe$} z>=W-yjizf|v=lfV>pZ5JFG&DgI}<;*Owb6kFuO}f=nmnh#4*l#hSg_gfe$fUcW!_Y zDP#;TG%&3fB-<{24-ZtN!tAB1#a0g`uTi24wCRKJ7+?X0{hgs!?^!fv=lgzI!muiY zaVqf?zXJY=k?!DUzfU$RrFALXrUZW^C^a?Bo!`V4?m!zqbl-w@_*)A=gUWC;M!(O} z0(Hsf{-N`(oV47A z&=6Q;`<8p`6n)55kx*2_r;TQu%T-ZQFzTg+P8u4IPaTL394J`>m{JpFM_J-${{*w8 zE8p_t5X2ePM)y%Uz|CfML2riAIZK9Ky5zlXKpb0(J$_vhQMU1(Otmuz2T%Khb{s#D zoWU zX+-D8#Smy^&$=NO`vL>sdj!9(adC(k1@{dDGh+%Mqgb9$7Nrk?{jGW};P0CT?q)tRQ zfn4{zZrp3@=>eA(kIQItexT-)w8L&yHCMF}x>a5Nx%^Wxrv7_66P)tk^Zi`_aI4>` zabI@2Sguh$8ZRgaWCDr6LWe<$tC8ueMOBw|%L^zczmo;5Fc=B-{_B$ zB?D1mKIwK%P;HZe>@eGLz$?@)`rT;M4>FkVFjO#pWGQ65bTF9GR59R6sASSq3E<^q z2~r}3Aqt^S4I$GZPZc54p-&np*;v;QCb-8UH{Ah7u{Y&`YOy!%0Xwm`=8#X6YaNtsv}+F( z0P=NrpglI91~lvLsz^z&MTI@`+aRnOFbQL*?@_%Z$Oit3#q-Pb!7bglt&vZ0Jl<*- zA&3*kz5!#XHy>QnJdo$99s^N$6gGZGG^iR{u>sI$`jiARUTIJDCgn4z&V?_j*$cso zO=#if3-Z|9r<%or;Z4%BtIh?QRny0&?}-~#*B1+9q_zAc>o>s3vT%#tB@ar*(Km8S z++`11#`zekiQiQTs>bOudGZC3hv$GyHL?rx!hSHb*EGURt2|&*&L@;)IX4Sjt;yW= z3Iasq2$(!k_G7^tu)GEDS_J8VL&J}JXVS!m#Xg58hkw;FLgX_GYy~!89Paw6{|p$e zn1KgaxJ84=;tVn28-h_HnkZT`5z<;SecP(-6?+N-3AeUqnlI|1gwJMuTZWw9B?YBx z=!JL7-h~5gU|F$YBKnvF<|Cx-5P@q_?*pXW$ol1iTH$Z8F`4L0HG6B~cHM%|u(h$Z z%_4SqaCm|6I0{3JELnlFW_)<;IMcn)ELq|DIMe;&ELqc5x;+hWsV*qZ-jfa6Ou7;G z@J^;33GKs-TnEd9k6Gn9k^CzZ~TNL`;Zwl`x5xmOaQ$h zEBn-l<-J)J{o#f)_ZtMak_gT{a2&h6%Sm`ZkH1+52IZ(5UQ~Q7nw4n>fyIa$RP;wI z)aAj_=EN%{pWKi~OM9phYd>a?2hOEQOSVRP_HyJt8&11hqpkT_Z@-x*h#t$s9y5!; zNIa48H4>}!-$5dyHI59reFLBd3qWA3#{a?#Ki$j|6l?s6jK|EAAbI48(>mcAflqyi zGxgCz)ode>P^$(d`1?M(7S&!7fmfYkgnS)SLCnu6*jl-Yb7RSxV+H?YLPa4N&vllp7&ol zi`?_o5+o|fWA!C)>X{|(d0r#o`9=mgi6e>9j&!HDym)!qnvszGY>H{@GsodIF>!Rp z@wlg=Iu+t*%Kp3R#kk7+Hfi(CMg6NaMgEWyj|4#Qe%$COc*wjpp2IAzo`Q!Q&co#l zefm9^SCN7ur$1b$R!YBL%>f6KVsMR5+9swYjM&)q6Ij=kH_u*5>eGqMF%+3BwBk=@ zjb2;e^CUJ1l{oNhZlR^-AVoEpFfYqLr^JV@=-A8V&K;g7tB@LAKyfFl$L&7+klRV- z1Vu-PNG(w;glv{c#FjE9M=_b@@KT5if=IPjRt08TnO3sSssmrX!vLPtvOqB{%QWDk zbC>Ao3m`hQd*)+-u$e(-JXJzmb-&J;-EED6GBRu?G(_^J^C3viA)OZamUi=K_Nkuy zrhb@P*a;e=Fax(P`zr)`#w$WFozH2cb@BewqVn}Q!l3nZT zZ9Jt}+pB$7=B~yC`$m7N+g754jY>gIUU6jdMj zVRrAOh;aS$ZIcZ6@tC&_@f3sfH>6&88D3gs3x~J*{A@CcwGMFA^r>>;lH7CKc?mg^ zczdJPVCQ$Lqsw+)TXUW<`g*qQq~$*WL+jR>w8^kcwCBiSGumB{q=Z(iBBo-cx_x!> z%#5fhS0C}vJMvVRGGWp=@;UwIAn#B2QWXTe0ymUy<2ISN_++n|zZVveC-8Sv(1Riq zKEH^a3STVb0-?N3caa2}B;7&b!1-}bBK}rEn*hDCWHexHQ{i;S4piSDMdmjkm{eju z&nC^kte~mH{}K+70@b(*4q_bz(U}`mIM^tf6x#)ea|6kb$R4Xf@L_2Gn=atH{QZ;H zHv?x}k=_Ndem>UY#h)d)?$>cAldNZ@38v*FcPXp` zbnyHuCu(pXNn%6~v$~09N#-cDNG$0!&*l#+?qyDRkXvKCaap!%R@as=xuL4~(s!to6DgUUyA=xYn4!nG2`2=k5A!Mj z=U-E^(*;G0-r%t%=Qbv#5WDK^JK|&=UU)!_0)xjF~`;Hcepje0Vlf@!}b2=e`&$A$bPa71y zn55-8je#tYWp=OHNm~tau+LRM*%LPc;M*;`Bh!O!e5$W5;zpXC4W93?X%EhP`UOfDEacC~*wu6AT&)PG{yJCJCzqEJ_+1YvQ5c@!jU)T|(GE)A z{0dMJDj9n`L)F{%u#Y8{!_(gMW1`+hVy6E!K_t)X`I7VW@){-D{Fv35jeH9Eh%drL zHO*)Nz4+YSjY5I=kr-${lGU*cK=}oAYbch&bR3M|m%v|YXffdF%}Cqg2EDB{PQ`!gTZn~G894=cjEZ^{v3K_E7#eb{=d$K~$}?<3`R{$$u4U;Kq>9p{~x zu@ro`*XMPqG|I<-gX`AVssMb7eOT!wDN?rto6wT8jPPQjOm_#I>5jGqfL<0^g|zhR~ScOzE-HyeuhJ$AIq1lb%(e-cwV{9nx6b4^ddFz={BaU=#GA;BLG28( zq7~Zs1l>j_D3%2U_yVdXGI4_y!cpp=@l>HksAOsr3Ifk#w`dDgh;sTkwMPjSIQ691 zF|D>lF)2Y&iOb6*x(LVOh8IKoOtkD3qmvS*V925QWpp)S` z4!J;7IiEizkbg4fKS`gf6+wDn1Kr>Vzl9i`UdWTaWxK=Ct%-GqSV6-%{{n-4(_jou zeMN~ZO*&=JY~p5LgatJsP059k2~f^ROz}pag*$dlFeY3^XkN4q285+?(XF1>DXJV4 z8Lzf!C2N8VQah0TE*cDwmA+%a)q=&iYv*jq1)m;h3V2#2%fhNEKk*Hw)`i{Q*@Awh z`15+)87F23f6w7F=$U*{g%1}^o;Y5^K5dn%bE2=cc-%j7kRnHaO|8~ZuJY}wuDQ!iF!Ey+Y7?! zMp-ZYbBHS$Ojiw-d0YALqI0iyav_#^yVdaTUc1dIKV@<1kpABeE#lNv1icqlkq^i4 zFW#1a@NJKM<-h72<0fhz$wSX3G@c=5^G5&>TFzNo#+T&22Vj1=Xm1t1W$Z2c30CZS zc`6Ynv1hjh!g(^ztea0ei(r&2cH6$|*j#SUi%#9(WTk6NsruL_e5qgZJ9$jc{+!F} zY_Fy+p+%iW89yq#|5~rct(?R*__<>D9eUGVJqUthFsUXdNCyVav`**;HI0SlOzWBZ z(#bCl7DSB}SitiUMRFp&3M_^dK0^`3tT$1x0bXVKo_*}>8glS9@G8ydvdzJT3) z(kH--cs5mQa#W?jI~iQJ!oWqLlMtgUBS&*TJeOzNQjeu;>s4mV>gu~qZs(iNqG50A z8yHIdBclHXfk`A5&CB416Y-nZ{gD)&vXfr1C7gJXziX^VTpKS@(PWrKmm=epf=FoovGWBjG4u4Pab9f8}0?Y6}r9!FX*sfxJJ#2nQEl z)=DVNd!h9#Zy~kLER!1Us)~PKPD%Bc&vxl$Ot&&IVaaL4^<^V!N{@GS(01OmRm7Z8 zrC|1flPnG6My$E~C*H&wh)H6eq@g-KgTtza(T}x8Js^|@vox)_r6K!46|ZDb&&t3r zU4UQ>E`l`sQavuFxL_;ZzI~c;sY7C$9=QXTD=%SD0-5NVT2%tgU;Y_-pLMlbu!qG= z4Y(^hPEI)6D+bJiDQ$-F=$+spm`2k zIVpkf)T_v2d=}1q%Koo?;LUM6>Nh(A^#HV>Gn0O&A?MJex440uE>wk!fYKLD@dr&x z{2X3;>O$fv9zIv5eB z0_q)9J_D{{f@0nCZbZtni<<$ryyw}1g4HX39F%?y-U6Y~Uy1F%4s<>qfZIjvK))QI z-DNm0f?kt$?H^766Da0-f@})_+YvSbOUA=inw`pg-b2jl^uz6}=t|h;xF%O;qyBS2VtNvhNwZcN(L0l3?nF;s zR7oMFArzk*l13SWtLs|n2dg~ft6w65a9RVQd+d+nMlvBR$YuzVWP<#o2U!YRCb9hW zOby+q66;jDQuZoPJl!d*eszI2(;I^CFW?`XI5Wdu-ZUx*2plsA(*Hv&`A3^T44`K> z&x9QqyiMCV!~t?BcjvRa(00XE&rWZ-V>gU%{FE|ZAv9hU0T~lFwmgq0wk0; zrUg8dD9|laEI;SR9?3)c{xBuwm)-W+7YTvB!Lj6MhRJ@uhaQX_tQmm~p#snTX2iHx z3^JU;oY~G8@koA-hT?`wrzHZe4WLzLPk=?g(W3^(p-!h-I}ua5=!%nMwGL%W*MbX_ z{IEyoj|Qdgkmpxb2-e+wmW_7FskCo19T}k28{O1`oo(W zKPjHztX2(K2dOL1EZZ=nZ+cmlU&r!0OM3kceozF@g}1J@$!=plDA_fAf3C3M^I&~`_PL5;(s%EU&M9Dm^WbvSg4(~Vko zp>VWJp=6>=DP~eIyli|1fN#73R<;Z^Q#Z1pc=lsI>|854lcrX9gG6G4M&WtxAPFqA zh1)<_?`D{bn$^nGODz_r*?$;z7W?hOHm+)VMzU}*6dtqIjVN}{z9ANC8D%TFg`a91 zJPP!@`6`@m+Rs-mkt1P6Uv*##2N7cyPaHY+k2usVBF+V3S`Pa`4ujGhQo@FS(sh;}Epj zfaH8L-QOfjU$d&$PX?gYJw~f0)k(*%|c{b7TQbWOUM+Ni`(B2{kmk zMFAB78Q`*g=>kv8J-g_^^Z$pevkZzO+SYb(cXx;2E&&qUgS)%C4+NOt4#6FQdvKWG z?(Ps|aF@X{1pD~Txxeo1s_q}%yQ`~vb?sei@AW<}kLYmcpb~PGiw}T8WJ(CExJs!LfhY(FMrvkz{0K7iLPqLfGcAeeqmFPA1lOIS z5PlcKkqU8{yw5xWpR*Nb;@XJLpR1DG*KHS?Dj{PJ2^-=C=AX(eNr!?ogd?nBe#!f4 zh%1#m-BkG~#ai<_eiAraEo%|gayKEOmTf=6GopVSr%+&gOouZPSYITqG?)rt1KJZP zVtztr7aaVHNY;V7LF9b??E#*0Lj$YZ!LHpLi^2D(B21O@W;Ze!<7=83|2(xI!SbKc z*EnW%1A`*VH@Ybd7!9j0>K9s9Z#hv%=c5K3f2Ii*c=F6)oWjJ(r_Lfi1mg-A3$R;O zr5f^`r?zG?RIk_j!ahvxdoCf~-vU3t+&Fbx*~X!yTzQx(Dc|LlJqL7B)~a!_8Gl!8 za(4cl!das!&1l+44zp!Fpimt=TZc9233B!VG@CS+iGp+7Ni zzapmkUaVTz>WB+Ej_B3_N0#&pBpM}a_rG#7M0J{wn@$iTj7;KP1m0IDOab+AW_1L= zrZui*b=u=h8H;4UeKu@|#@F>PG@Qwd{B#)1wb?G&R!~FF%qG5^Ae#pE!ocvq1(q4Jt#-r~^VxTd;NMay4Ho-<@ z=u?XBF>5+uEdQYiQ;%ktJP%Ors9c|R*ZD1u&rQWV+wWjW9HxlO)i5cHaRR3|Ie9BbVUp&Bspt!=H_}CN2 zao<9dmG0Y##)Slq>57z3l)VrFt)*dMB}g{4+cy>)D12{ujkf`g2wDm0S_lR8 z$JmFwO`mAX7AIh^9YO+(-9=jtb=Bt#vU-6=4_)(;^>Uum2U(V1XYNz6siYJ#^{fNG z_9~OkN({o@Ij?o=ns4GNr|d+4u0I5)kj*KJwn4%izuO@DM1U}9C#QFi$$ss3YD_$` zPAy8^4CG%m*rI)_G?b$ZzXYpQ*gkny?1)@P_M;j7O?plpIWGie4kY#$^j@sH4GkGv zCFs%%9`8ijC3djx+pXbgY+~VPZ@Q}!4E%+j`+1JBafE5BbMAp*t#_>}(s;BQO!~QA zVnZ0H8`W7X(QyQV@y=mY8e*fGl!OKtxu?tA;+3UrdG*Yj>h(~s8a@T|0wVBDex?19 zC0du6$16Siqe;b587Y`P7IvF<2Hg7(DQ3SnCIbGQ@aBm8zn7lc-sxiJu>Wy3ZF}!O zrQrW-a{3>^%}+2EeumpF7Gay8Fudb`8)f{*bNQVdLYA?$`_2We)k6UoEqT8t+Q_={ z`J3=bwj#Oq6Sj)--mDIjB|F!%i|8bUv5=vS6pr0aHi&qg4lxsNil_fiwSMm&`=;XsE3d+Dfzy+3hgtJS51_eKx85J1%2lf33L6 z%5I{+m)2sUf0Q=S+1SUL)9Dee`)l#KE*r!TvD2uHE{oPpAyp2@UUVpO}zaJqc3@(2QWngM#w<3{Mp!KrREs91F ziO_nOl_%hA%w7u!|4MULszon4MKM4;I*}OyG2q+NVqf0bY|%^XsTz`f_G1^=nMnod zM`Nv6G?6+Fo3gkov2%_vvs&L z6#cp9>u@LB{S%F=Di%!TUD>~V!hkErQbWn@SXFdku>oPw~xelxq1#UKtCfM z%496CM`>*EJE?ic#JClVW88|X-@2m$i)J)cEeub@Uw10%KrTp&LbpWq``@fOZ?@GLm*Sgr`fJUU29vr4uug zppamBzC3=qF|V2*KPh9q^ovby0ir~z<-!(5%@(J$XpKLVrGK4VY*?09plHrzUJ>Vm z*hr&GQ<98PcbhLH(;`cxUAoH0{Mk!Ui=Pa9*pLs z*&<~weUIWz6k!12`J{3D8}u@J`aeDu7pQ65%3C5SAMZvqf6FgVmUz0mh+v=ad!9%N zIIOoar9{h53z4j)jgQ>Iof_{7!D&pFf2Lp;nzOUZ&-OZi9-`h7u(4`SmQHD+l>rys zZ2LzKYK*!Yj2o8wiL5}yZqWL)e`y4htRJ|J&jI*Py-KdN& z)AS3q(anN`{cKb9GO5MqatLx%Kj?^MOy!`nzB7cO?*&bqb9Ba7)-*Uht4UOr+AJOZ zL6Z&1T;K6({Uh(;*tI?BstTm{p)BML;8v<+Fk#vBFLz;+iSY06Tm49nEI6W}_ZkN7$`!6T|0zV#CO@O9=@#x2(7OjVp6dSBS z{6)1hKFya$x(70*r*U(K)&U2TPny3j$VP?w)YgOO)ZG>->HH_n5TAhh^Xb_P4fDuT z?9+q_%3m)8n?6n0%e0ShARGk>ucav4ex=P)+4wdkE$px=p`osMd1+;?m!$SVhOZVF zzS>85+>o#+h5c@zWRDs za?5U)%eRm>eWgOA(SQrAV8CPCw4aCywT>GK%=)XDp*$gr``gR;jmp$y>Sk3Oz@ihy4{zw^rv5e)1yRIWS9@G^ zE@}KAUQ5fG+M4NO+B9Rl^fN{74fky&1CMsgs;^C|+gx_WT@rd6XTETe2wG^S&lODQlxiQjgskF%BGqRNz(Mj(_wJPFWT z;#q6nlqURCJZwdi;VNn+aG01Gzvd;pohb+!a60lv@lc@iY1k4Fen2nh$ zGxB*=IA#ec0~f9fDNGU?@QQRjOtq5E9pF=gr?~bop~>a zA2*Q`2F4DSi<~h4y@`!qG_3mHH2%s=E1N6*S`IdKn(+O$0OmBN7KEyP5H;47_zBbN zi%nm%zJV>57vTyfVZ*Z*8GthYK;`Z_1RL-j&{Gq&q7%_}fN||YwR6XbyBV1o%Oau;NV>66!?Zq*<&to!?N-$`>a_4%36%ZYbM3afkm;NQfTr zA;(}d$hZZs!!%dYvG<}tN~M+(Gc~ey2?om?Zpre9N}X=Xbm`Pw1eFpQ0J~)Ceh)-0 z(moamd|fbaC2G~Y6bl*9hVKFNXHjG-J)-CNrB91S`Um95U5lKPY!CSdVCjONj)!m0 z={dp9P#BTXSovGP;qhTSlo{)c)$3J|P5jSe;_b(8(9UU(?uhB9BN3eIQL|BD3uBt< z7&WglfOz&Z@37T=fu>IvxPAN@GF4)O+ag+5)apwI%CS|EY$9?EA<-x2sPTo7 zmBPc5Ac1zWjmhyp2n7~mE@q02+CsCZe%LuGg<~^panuL&mBElia%4o&uRyRqIk}|n zHipZmz}Mhw_ycN7j5#nKXI&xbBpwx^61!xPW!GfCHV@f~dSXn0mnqx)ORm)43K?C^ zq!-{`XW8f@JHaPI^Yq7xB!o;ln8}RT1~o#9EVh19{kREh6n`sbC0`uQ#)JKrGuo@C z+&wk7C?A98&8m;>diaHX^}rqz=1t4hKNb*I?+yWfQqih^>{##L0fTZR++hR(__KX5 zeD>q=e7M6z_!%3KsMpSG@=ruo$f?5CJg5A+F!zH6pA7UpIpQY3}} z%WP?t8?OWHfOYo+!gA2JJXP2~X@2RfmK7+5ks_i8YJzFxr$0Q)w1E^?R1!T(d9${P zfL~_pQkQfyPh+>_eqX-J++fq*a6Ntw;qXf>q}(H#vI(+X?OgtpH$l~?n$>O@IcL1w zQX`^68EKd=-Xc0i`^-+VcD6mUG+q~4tS8&!h-&o9aH4fDxt}_)%WLxex>1E zR7caZ{I>2UXtU^K#U$|jvZK%1+8o(6o4PTM%J@h5F?dB^Nwq1VyAM7fB^a2H5Qz44 z;FtWwEz+gz0{8AfyNDpA^0Nya@+y(Fm~;?%cP2wwO@;HoAlW$>+aCeQ*xUZ~lc1n- ztK}=ec4;MGA{ug2g#yazkeJA-8{OCHn*8w^X4n&=f5QvEe*^?c2L&tNC^?mOYnm{m z9r^jLX{iWn{bI|H;C9MXv|FQ>Y?qP0_lzBC(nLMvAHMZI_Mp6<9Gz$B8?R!Kt}O`Y7cA$7@|Mclhb6Pn;Y5dEjh0$A@g-W{6S+%yxgR)zL zGiUil&(3RQtF<^4B3`R0J7X2PuPei!@)((p)koTUcy0c#+yu|D+Igza{~>(iw%x%- zzSF%qDF1i<_uqW(`yL!_8R;A(&|u@!;Qk;>g%5Lsuc}uNFgI@^^ASv{`1lbV zP8)@!uJ&mNO&vuVzi=JUs#9Ims&l4Y8hLNG^q4$>LUvFg+x_U755-Sa0A90gRzky&>t5C5_5lo}SX-k`TEj`JM@ zw8iCEZ_w_2=GkK13*>Q0bI*zUW4%GMr$X|6u_x{w`5-=wWKGXVl@7rn`8x^)R>Xa? zevXZM9|6!v-Wc~Xc!C)AR(OKw_O`8`BjdKmpc=Jrsm58Avi(JS&dgIg<;Us`Bp`Uazqmu~~)m!`7nz=@8 zxDww%221FK`sM=JBwjYNITE9&o*)q!L?Bp`4*PvuAs1QpU}yhO<)c{(GM! zA|>JhIt#HD&oxg)V}B`7&ocm>_W@>)35m*6==93c@cl(q(n7}`cct|?9Us(^#aj(2g* zF4H{1L@*a@NCBy!ZG?Q5Sq+oH$-5SeIgXoQrA0E;8YF?n(M}I$5X>O}E0IM6Sc_O+ zRd6;zOB6Zyb3nKPwL$&WelUW}-F`Zpn7TA)fC5ezq%Gz+G!lOOAAlm7@=%hH&nCSeOO>U4W&K|KfY-lt(BX{QHH{jQuWdwj44$&?rzWU_Pus*(b zSVk&OW;_zUdN@!=n*U2#lnk3Nl`1q`9f$wxseeF9Uq=iG=Iv(`86;4q=3rSF-DOyn zpaD}%Ymgn9M7z8<60`cffOS_Ve{&Qpu)OU273@QU?~Zik&Q$G#7b*)A141*^9@H_{ z7!Rvs<`p<|{ zgki$pvUas>iI)nh&2ga?77O71LVk3h>ozk>ZRZNM`*zAONvDS=*ifB^JclL{Qs|)C zOZ32Hpza!xHJ&%eo&bsxKDlsj=D@vy!m`Mw7h5DTsa>FHd8&GFxmNX(52}wUJpovM zZ*L|2l%V7ik+&x;E9;t%Fwkz+>8evy@zFw%${uYZj}779GEi;Vx<%USZxeP9O^T7( z`Z4TfqZaS!YwKSc(T6Y5-FiEqMK5EGe)6WI40{iWxx=(?cr zKGWJ^;_bJ7bJB}kKH=JchBn${0a0FDTI<<*c3~qT$m8uu3a#CBTuC1@4kw8^kV_hG zO#yExf4#XO%Vf)mEk23*_Ollrq2F!bJ%-f+jcd2h@G3uM#LWnUf&KOx_a_{jzhlu* zmfMkIS)tP8**vjmlbTko{SX1#nc4fF9OI5>wExVN(G^DO?pNm6k&n(IGQ~FWXigAg zS~zM+7XM{uI*xHhx7ziAmx&hrfs*FlSg$`UXiGc}9gkHxXr3kuv<(%Oxs}Jk7kO5Ur>t(u8~19S-%L>0y#7ciD){m z{T-&2+VF=j@kjFUZsyq4nfIKRaJ}M(o#?YeOUKyHK0)*d1L#V z&XREuZW_9OmD|I)U%ie+;dtV0JF7`PeTx6{M^KReyH)A}2;Htc=zMo$!2_RLDMw{P z8h6rJ(eT7R|MSjPY0lf-8!%UgDz#A(Vu6r#-wAJtaw(Hf_TkN zXA;-t^|$*6l0D;Ufez~Uyc7nm@;!Lb+RGoG_CDo(9-n_uqsY+l0*ot&<5?&x{&3`( zpleFJOc%ML$0;(seAc1M86VIx&3w6x`U-GHH#jb9r%OB&{&k3s5DG-BuK#XhrxQ0> zUSv&QAgtPiD0wR8!8bN05#FGaflk5)Y72Fu)vMI;Zc=1A#JKvLjjil(pwOb*x$gAS zKZZrwG|X{2$-U9tp_v(u9)1t>^B)UW&^rkj0k3@OBc-O|(t6nLNvSnj%*faBQ?Dvc zFneV+^WO+X{b#K~D`B9euQQh<%S6FRbtS_D6NV~!#K*s7CkZFks*%m>!+{RJp+7aJ z25Wgky|q9d$TWSGJS7YCYrE8k(ezTEqSrJhU=L(7$;rKfiv&!XB<|}zDQNwEi^k< zS*z2Rn(x7T0uxrX-psA-`%gF<*^!B=;96cXgWtV`Chagwp$X!kZs`?p?haSjwXO=t zA3q32-o~z4B7!&!NB1g!yuq`a!#5e~D%96vMH`_}F=t@KihG*FfdJh1lU%lU;}AX8BZ> z$(mL4t2O3jCH!#d;>qq|z&Uj`Ul~KEr9J6SGP%DJ9j3rMix=sTwNpC_f?p7ku3x#X zg*G3E=04wOKX#_s&s!JqcJBP_*)ptLGhehTDn7A2&gK&N{5maQviZxi-^weABgann zOlC962g#;tCMr8%m6UI467`Y)Hva;$;@1*7wH)w^u>cy1#%{cik^xwojDS|bN%#~e zwD;1g{+tJ1)q3Tg`xD)urE`j^0Qcw;0dRR#lttaB^=z^O4)^&-!=-Rs$Eb6LjB03nK`G3n2KNS!eB8Qh3 z6cqpwkxq1Ta{HK9Y>7CWy=3&j<}m+{GK=ewP&M!Te#J)ka?BAZ6Qj~s@^wgyOe55^^^6 z`CXrzm!HSKbsE384TFqS93H;9d?R0_EJc-NP1}^Fhv`!%gP@X%fn39M|Egj(M*7dW zjTM%!AKHo(bf44|kFSUol^PqRPVAG(V-Dogxk`Wf-)ZzK{4V0p{jqHwp2Ab@reQ+> zOt954uE@)Dxnn2RIUx~nf_`dt6l(1!elQ-E;I66Ti6rZ<8b6Td$|fd0nUGb>NbSr# zl?=LX3*G-bzEm~s#e77@1%GNT@WCn=Yzop;y1x3ibj4_{*5iA+eF3}ZTKw8&jC}ts zvc6(l=HK6-7e~#W&oGZy!pN`kUE0(ifG6L&J;j{dcE9LNfmx0$-5$SyWS@+tq#TKB zEb9C1H;`CKjK;tQh!cNtF$VGA(Zp$1DdNB%(C)$?exQ8sp7L|k7fnSa1Og?AeN-p)wijZ?T4 zdu3%|sZ2_y6onziPMneS@P4a=#L}1Ap((gNtbBa!%LX3aIv>h9!vXYGaRg4BvKY1L=RJ>9^ZnZv$^Hi`2UYnJXai-EoeIq z^kRPymUFnvB_%?*Y)1UAA>s1PW8n|%)##wJK_(U5!rwKsi|D2c?UK=i9D?X6o>m=R zcB~QSHaTfJt5c_a5!Zbs&kLZ7>2UQ+Dzs1}*!Oo?mK7>c*-D&6AE^1H(-BTD)a%k}=uv~f zQtJIlXHU|5RWVA2uYO!{UQ{s;$DIRl=rC|5zOa|Y zUp;cv%p#BBMZzh3qVwww?;??8C`IXIdj~A|&T;U}>pn(|48j780OkePQYj$xhuE}P zwqN0)`ysAT?PeQQn>^$)98gxB*9C~0=J_%##)G@u-dFu-QWH!m&}?RTMEXE|R9Fx^ zYz`1C)p5cFk2=L+D6*`#@mJ%kkX6nyx+}6|L5j<&`%JY3;T!xRHN|b#vXLg9=sIkm zQNy3w1gpV6WCN_TkcTqYDucdXd)pNNO_D^5Jz3O5KH!I--)GPc_dl61dD>_Vy*T3p zL6~r0{Tq)N_55eixJV+fB!Gk?M0eD{BpgU_E!+w3`pyUG4r$vdpuhWdZRpi|Rgcvf z8Ga2z*heLA+6I3#9KJ$lv12GQ!T{Dt5~EELAD6i548zA6kaySessF{O zzjOK7(tab!Ld&&GPV;~EA%$k(^~zfV;p;LY`!n{6B#Kwj;fHxi4tsNI`#u!9Sp>9M zjB0Kymtl!|0;GW#A$ZCWSO8*Ve=IZUD>sM0L3Y zt03ua2k-OWfEjDNlX;*R-oRMV+fVJ(%ne`xj7%Qjs)yaH=yYDv_G=Z8>0gPWH=BYt zxq`Q-f;TMsK(ty!3K%tc!XZ}@1}q6edPjEXp)2dbM!J(WtGyBA!qizZEBFw~9Jr^_ zPBK)w^v>KH*wYa&!rEmuWnIF7Hp0s!E%g=9pyy?ho+gsYbnbaY%X|zQ^NytYtY1I2 zs7>H@NqNA(W#J$=aM`8K6R8;8y{ zX;LS8l1+pZH{Mh*VO82?D7Ljz#;$-~!k-hD%b!jjOxDpIAC)#R|6IeY#$)vrL%B8= zK40^FQEY`09}FdGW?jSELzgQelt0lM&&Tb2iyyk2wBQBij_T>wuDA&^f=iaPy?mkO zVGcQsyM)J;wRSUyTP%l~=crzjDq&eG zQh>g}cjlOd+;RcN<^=lS*tv33#*jY#5p~Mhf^QS-{}Auenr~aBNs<7w-Sp<4QeE*- zqG+Wp(8mr`shZ5$LCXJzX3mShR{2Dy0V|YhxJ}b6&eXL8?<>$pk<&N*vWFZgAob1V zw2pV!*o@B!cfq)|shPsQbCx&}R%D43EY${BI@wD`rR69mR@ zYmwuq2ZU~RB{!L=8{^Jo6-<*@>oJ(Ubv&|em{Oj=ywg5ZWr*Tz9bw2|owB+qUp6>N zkJO~d^DU~|(Y${`4(~jA$GBwgLSbOx?_r2Xs3uY8?&3jE_=L?)Y1^ zL|O#sA`FR`u-RQ0qJ+BW=+7eZiB3uVcG}|s*D~3IoQzjzKu({cFC#+0dLd7RoL^8< zAk>q+PQ~zk}3w3|VA@1~a&kObZuJSAu3yvk+K9_Qm!%y7Nm*^>t?bcEIsaz23llA#!SCg0!l}tGa zCN>ffKw_)^MM2s-P+FaWeOz8iS#!XY4jW{^FPAfZ$;X(DLTJKj%C1D1qeNXlqt^GN zO+n&!p&HyD)ZEfp!k%6Kf4Te-$w0xhJjsU-u;TxBNr(U4%CBArUI>^GngLIYi4ukF zfsH)MZ4@Dr?H2BM-K)UU`;PZvgjqBeoi+CUyrM0EAPQL}2<4)YMOY!|^zJggdNS}{ z@z(ViU$7LJ!)$dIAfO|;%LGpWv_RsF|VucVU{@2vuf@LT#DNtu`#{L;0zNZ3`$oJYX zEH(!%Cw(*811>=`NZ%CqfD6!V{+-#dDimeBcE)-Bq{3*E?Bm+#FWJZC(XZLWKcj=Q ziEE-Q`MI_52Kcy*@cgog3!^2Kr9toP)vD+-e)kkcm*qkBY>+lyoBghfa%WMrE{iHX&Ak7fxf zj#1Kf&tl=w9U`a#X%OgE?*6TUhO%6$4{>O3RL2l%pX&?-R;?DtbX7%F6loJYY0Z_z zJZaDUib2wX{A8ij7z)xRR~s6rf)=w-st*-tK`MYOm{q$!SSU4z_NoAq1gcfL%-W*0 zF}7Ne8kT&`p#p74CyR;NP;3=Ifq+wF$H=DxkHf4@c*n;l2hXR`5Lr~-Oe|OoS4bX= z9n6F*Dg)MsouL5_Ab+Lj3h#9y;;ZL0o*ku>?L7j4(Wi~E>5Za{4@2tVJ2TL-SCu&I zY=A(?-V1Kaaj--010v7_us0F& zm3}TFkgb^-xiNN-!zc$4Qk{o1t-%mES!B?H2aM~p!aBddeE)vN0s7IU0>*=Vlj$WO zdX5P^4>9#j`+gV_YnKqastx~(@fr(WbOwk(bgM=596_}gqO^8pIgQqzcA@ zzhtP7?0tgomh81{2Ya**A?RX*k~gKzn1V%6He|pquBH09XWg*$yz~gCWs| z%wZyU>U|l<7L<^j znUxa@7%@X4AllO`P0#v9X|i$f}~zSm;MyDDEyiF_g9?SOb!`fEeu=WxN(UGJa< zjxA~e09Kk>=S&^Cd_`h#uWlFcq?ffc~Q z)-~0TbRP-h6}1JeGtOU-$g0p_StBjT*EbeZO=pm#&t%+gQe^ob(PZ>#v1hP&Jby(E zVzg4bLg(l1sX4wwVA&aqF#OP5rpdm36%n6L;|o=%@>q@QAs*4W9p z3eRBTyUj+oAI}Np@96Dwlo*WqsF_>`$Hr=0V0Svo=_IWR5*_!%*xC7ke82hF^%%Lu z4i2(N4b4daU~u48_)1PR=L7?zC5S3-qn=JpRTH<3wh)6s~iAu1%#| z3m9X_Uu7TPx8xKaZ!k75;vQ1p_<@$FlMYzXCY80))?B4ZK2x%R*=()`&YKC|wLTvoH01iv8^-8;17bSnly~`20GOJ#U2bI0Dx9kzoRW7l z*4jD+qg55GI>sNh`(+vc%Jt@5buoGjLmF;*09Nc+bqcplkYX8?yS!lLr2KeExrDfm z$p~v){lIR+0mf1{<~LlR_YYYUIopQR-M!+536xEWG~$`LS$;Xe=JKHUvh1zR%Jy-W zsIlft-jn{ux_|oQLDMZiDqK*Gf?j6usyfEh+i>%;2N$HYyIz1Sn>!&nmrT71O|kBq z1kJ_o!FUNa&lOsfC|7J3*_gJsn=QG-RCGYLCBHu|PE6r=63YkPM%gr793z|-!ksa{ z#1OE`_j*SXU?Z&uiH_mR$ulH7jik~|5wdU7iCdi~#6paka?j)jjn*i?Vz~QS_kATZKf$Q962t=GTVL zgfi{n1Y?m5b!xze5+Q}jSP^cq^LWR@X*q4zl7{L;k8>Q^EUh^fmZ|+Wsu}z{L4hz> z`8M@S3(eW}_!hbB@=zyZ28A)LvIW6Nh_Q-~kS7E8(-8%S20>J%*MFnOL6U7O?Wp!I%~kS$r^tH-!gSJsy9>;kZeK; z4V}9f$@Oj}vTULiwQM3e!;T$_5b`Px2~;Pm^z@7wQvt`r@zN$j?ewVv-Dq3B;=YHyp9U42&D8B;Cw_RPNC ztR}+AY(b#sawo}FTCB~KzJ^J=o%PA?Q0#aar@kl}QUebn$-%MQA^(#0V`wZh#mGLd zzU+G2nWgGgrX&^@&Rt$Lh(~QjPYERg=p%*ADRw@6UV|ii)R`H4WY;(|f8R}wkq;58 zxHmiB+Qjy-*&xu`*lWKU&HqHOv4aZ4M$tLa83ERP_6?-&jA$Np_Fa+AAh;lFpkDnx zVwCYm&3F?Z{hy=9N*&7LKTmrW(vXFtUy|^19H#rG@ksmSdyKHaK%o_*0GFR%Hn1J{ zcwo&<16#xd4~5V~;wpwb(=-qzN7uEP^G?q{*l$b@^~LicG}bs@FHuWn1usd5j~Np< zRfqtey+%D{E{pnAeY#C5YWxnV!0xY=z$^W7Ql|cV z%Ks&XM0YRyy*cx9A2uA)zuZ&riqAJ$ZXv5VEDbkN*UB`IYoinQ)l!jY$#x5i;EQ9z z5?$X+iKYyv_#&IA!9E{~%g>4%-$#Hr=(G3l-XccCxYwlnhZPWA{+*ab(uH_BV72up z-P|q7%px*(S3=*7X;NUV*m*&4hX!eSUHUZQvrtks<22?m!?B#4tfNAU>xaq<`-Z0v zbrPcIt4|sm_6odXs)(wWBiWs!COIcN*Nsj;0~PPiXiiYwJn$_E+g-#cYGi?>W!A@w zkhm9HzDI`>wimLJpt7-}XXw+Pt{${u^YiUj}@9-V%rL1)7pr-=0$YT`=naPXH% zr>`{+3x(9VE+-ig&X10iMil%aWCQC$x=3??L=!vu%NRMOrwWbd?ZuQrY3oU)JDO# zMz*i)i063B!(i;8N(lc|CKLx!t&O837s_baQX^HwoQ=Qw{vCLdSj1{TDV7 zF-;OTOHP`lb=GRVAiH8USQ`eYT&_%Kt#qTw?1{H`x_(&6WHrTjwol+}pzBa0X!mO> z2yI9>rzAep^-%40Dq;S8c6y18cRk@$EU(=Qzu9UcC8d3^8<=g_Ttg!dqvS;B9nZ<9 znjO_zRye)=QPr?T5($V{T$AoK>#bbqc1dd5B(`EZj0XsZo_gklwEX_^jn|#jaWoE~ z8c*){OmH}`+zpBe-( zS}%D2gQ2&vY=BqFw#$WoX`k+L(znXiCPn`zu+f>!`qs=Hm8 zyboUn2N%sTlz9PBc(YI_SbQY|>2|u3T}VmBU4B@j@Lr+lN2`Xs#TYt_3%|*J;wfFe zNph&A7`Jax*R5R3LV8$PTh4H4DLT!U6l`3468<&7P#wQO=HvZ5p75DvcZ+LtJj4vR8q+(;M$@v@(~T}AW)X@+ z_E(_U>bz>N`bE3geJ9K%0c|JE|3qn?%uNC!%*e$g^ZEE0PWP)E=G)#f2l6zcCY{HDGyf<)gr11KMGB#d%9*NJakY;WWRb*)jdMSkS&3vxFOu&0HkfO;o zZ+%hQ)O0d-Tqux|K$yEvhCh6U4?BF5kRZ1fk0phUn!q=j-CuIK9lVadaenS$*0^;* zaNv31x+{DZaxTb

    -3tNh*L?JSiLpNC9T{*E4k&H+uOQ5myM(mt-5%814T!`rZ&9 zfJtEr_aqA`Qvvm)Qwr2g;(9&ft$mKu%fo|0im*J zF9wUT#jK{8c_PXldCxIi-psIz6~8tW>=OR}*gD7HO1f}uPwY%=+qN;W&B?@>*v6h{ z$F^e_hp8UETHcTD8`5UzY%2*-&E4mW9T|5G`9N9Qg2{ zxkT#oo_GdM zmGY`Ejy@gWY$aKBN3laC25)9QlYS-w=j->|B?eM~6O2L*mr?yh#>Eh2ZnV_ny<}F) ztx6!F`Fc1lu>;ux%JQ2HecO;dJZ~~@^@!mVZBw07aL)PTpjqu#H7&B#%yudku?7+O z$PoAY_@&VRrHAt~HM2y`K{e2&@`P7;;Vgudd1nlu2oat^&_@M!LeV!mHQiOxvQRRw z$W-dP3XA1rJ9M)V;EJF*l1~6@`%e{qhJhs@OWc#*vxnd3-G+0?x!kgl=QsBc9alsB z0^>-r3`U>E*LXN-ka9crm-zvye^~ei*&his!l&E%<4=AcrGB{Y+%F;_0J6N`XW!ek zv_3{a9s}Tqi0|FX;etiX%z@*mRbVI}4iG#;WPI;ffZiMJ=m`mjokw_wq`!vnGH%2d z&-yqR0O@;&^fV~>wzB~L?oM(?Cxy<_5>v3{MdTnuh{U!E394vG9e`+GIG3|F!!6XC zitA4@=s#4xp%WyxI96fpNRgbaBfh$AZPWmClYrB=hXcR`qcz`F4{)-xZ~DlDq!Dk~ z`F5zC2F~Ll{z&lnK{`BR@$_5GD!k|XFo_U`1|K~uj%gL{I`PNGMjYt55R&@bd->$ zRdy#4Lg=)5!*;kl-&N3OuV| z@~(5FR?-Ju!S;ph(h$o9V0nhh2StcZi3DaD?6WY)>h5fqCQ&nmy0M4zvX!-E5S<~VzAZZH$w)GFuC|%kP#g1o(1cZ}p76@` zlN%n$qdhSoI6das58Wm&Xg3M?8H#H`>M{@$Q;&aHKao{kmGbg*%$JaoK=ZNz4HaGT zwkqEy|LrUO{`x9S=H*Au4o@oBJ-SVBh)*YWqDIe`ZLa39{h%_Y6JDqb2H38pnL)?H zNewxZ;n?hVw0<$jExKWvJf6}BcbQVyw#sTXNX?q6;9!=;@0HEtVnhU(x1iJ+CfBe( zaCVm6#2PvE=x}dSN;vgH%j_A{H9Xad_D(ShYiOL$`V_?YXn_n)rBGA0Cb7)OC6@3a zR5_I^HsF3-n3+-9#S2dju5uZvtc$1Hd+hY)N~YgS#GDaHW zUOXf+>CJpX4%fS@_EG`y4xgbrrnSxCX{+|6eb1oaVIo@(11Onc*sEX}it7>hbE_xs zeVA!Jc(Vd?Wxgro0=3fub_jaD?TF zGZ(>6!uXd?!D*UBDcC+7zjvkU1Q&AF$EAII3YO1v?doV1uAVxOH&$&t_h0-^AM*A3 z@B`c01~BeKKnjsc{J(S(%Vx-BY1^!bBzx`qx>Q1Pj-+qf)>j2k4NS^6mI9iqLR#F7 zOSI-8?OY=ISu6DC z)rBr%j~!xu^w3DQnMRM*EB7alCu<8#A^-iM$M#Kr2iKH7^KzU~G$DS6!AE;DT=;Jf zg76hr0q8stSB#7IlGokBeOPW^5F$NC+D}{rRCi+8Ia0j@r$topT*;IQK+jT~~PIx}-M+1rHr40v|Ci#NTXj zWoK5u2xJDZfpZS0Z|(^W+Bdrv zG*A?WFuGTyBOLa=l$2dyHlN5#h5&~H`>hg!Un|IEv}@#%plC1jX80O>^L&V4)K=cK zLcD5P%~nR=RwkFQup`yBQe*0r7E-seF9iSBS-xx~$p>Du6?29|OQ5>v+Bk}R{h)p9 zC>?Mb*~Kk*>x@oHZ(ZgNe)3IX8a#Ic$T#>!8PsP?2x*|C>qgzujmEE2QD3kWH@>K!R}aJMz3m}qkC zvK8B?&kN!AcX;&Yzbn#?ky(AWtk~#7oWj%yU1tGO6aXm zG+!UzWx^}XcaoQV<#w;#v!RD~>D9OiTWWAMDH)Hs?c7akMSFlQkTpAL|E&R7JOSWh zH=EO`y5)(lOun2Rb3QCSDsCC{^etevk+4U{X=&_!w*TXg_S)_{K?ai+K{88av~GHk zaeOge=9;r$nw<3~_{Qz`bzMu7S!DkcvviYJL^&l2Lv)QmW}DY)OmDmVyz(bVib_g57c^ zd^kU=8m~DId%UL9fSH+Kyk&GM4(ZnhUc23cdi=)ttYUL{AVIV72&!$>aa+3H1$+m} ztH&s}M#<&uNVLbwj*wJ?bvMZfsNNQdZ!;FE-WI^c?3~1h%E|mqy{#yvW)M(Js%H1V z;W-}Qbj|t__fE^F0bx<(!o$S}%Wve=jcjEqznZNLJl%b5?zp92_@)SLMM)I4b_aa| zq8Vg63RJVhNNyZ+!pI7$*wfSveb)DYsqL_#YLJmZKINP)GELA#G1GDl%4B3>l#e|DS*~ei&8oWR^Vzk z4TdALf-LPMC(L202&>SlsB-Y93d0%ImT2a0DS}M-QMFYA#hAU=o>g|Bz_1sWyzWj~lSrHBCfe4XQF?ui+ zk$MP4-c&x_;HyV6!~)s3V=g4SyJo}b=DgZ z&oKD6H|w)7uQBfo@YoBr8|(`j>Hg|Buqo>k+~axug^vs-4?tMm1Crqk_BPE2~~FRNT#a|6vdmsy}cc}@WXqvVV`UF!`8*BT~y zOkRkAJ}Tvl5MIW3 zeg#dY1rDc4I$nXba-)$wuc2^2^rNwOo#14M=5VGlQ-Bx(FguHX{6$JeTWhqEz@mCk z>5j6B*)Pw6plvs`{aUW~vLFaLPGrVqMSf1GiE|)>QO&B>3v2BM-f^?WBzoC!oa$Hg zOKDgS5~XO@qx|`a@m%Q|A}fR|*+WZOt;h9UoH_$riv=M*5TEC`_-I<1fjr?GOoNN@ zm1(DuD1caBe)8#vQ1VbxInT-LXV`4c1QO-<+xhA@$Un@U0T-RKdtQ2Nq0vQ%bJBBU z8m0zaIIp|QB!l+;$gsB~1?i#?8qXSRd7>yHAIQ~vGZIh$)ae5N@W<|*`Uj-p$d3@> zck1Tuw1Ut+NVu0Or|>>hl{?-Y{12RN&^GYTGob4l>QiR%?w<-4HTQm}6l&!3s&Y5C zRedOLWmrJXDA)L6Q|skos;-Wfm-KCi(Gg58Tl6-{7Z_=JS)U|dfc4TYT)Ipg|B z2w=d)%KtXV+=9K2Y&#v7)YZOlL8-6<;hXCmIhI_Lvtyv*zSl$<9YU@N{lSjh`xi6l z;F}=H3RRPW*?z|CSvvIsy<$)Y1ylef$F6{%119VSO=JG|gMP}wuR`RngMx*meiX3% zE2d;r=t#3oYE8emV!<%)|<$hcAl=>qUU zRXoe1@%(?6Wwap7r8NERsTyZxFP&zQ)mQclLiyQN`!%Vy75nmhGRd=;uX;0_T=0cbcTU+9J7 z7OD>goJn|Lrhl&R*J7MT_ocb7e+48cF=jb{{r)Phunj8SJ<9A*Pw;Jqp=q#T&)^QHx5I2@ zMO#=6-GlbXptCRlBYX6ugFV96?Ap>n`d~v3vE&9?aKweXUoYeMny)GO8$g%IG%p-{ z!PpQ|2G7b+o%F-9BO;ERc>cO0_dt1VV0d;IYnYM6`+0KJD!#momCHuJ`<350w|6!j zZ~G5lIC+FSCg0?T*r1j%z*$`MO!_Y;x4L5(ijn#-?*ZDc_iWjzT0eHOo?HCs; z5ytpKBBO`UjIB-C+cPML=yyxdXdM5Z!0eFr32ki{g)@{QCa*;3Dl8+|qa0IERn0+y zx&zw?+p|AxkL6i@17$l(luhcw6o1diBl@RFQG>A2g$Zpws}YO-8bH1wjyvk@W_G*& z>-X%yG=8v;z1-KXp&?YcAyhUX)@aQHm2Y@|9J=;5`urj!i_wNtr5ZM(&H2c`=?1lO zrNklgd)@Iuey#BOD?`Nn=wNEw79E| zz|hd5mVUN6!5=JBqKOQ za-g53$T`Jw;In9y_H&PGwrga-&d`@!=Wqnmk1JZzHLa>z@Gz0|BCQIJOFXtTh6k&UZH}87)iNEHF-&*|yKsma@TdGJ&mOK* zS3X{0)Ozv-lbew)JZ=Fn9U336S+`_?4<{W!x*vBOr$2o4* z>%|JPWJ=`i|nm{|q9?#)a$WM#Rec?wE63p5gX zJT^ASI+my!nw(pLL(^zKmVw|tEPSjRB&KccUm$kOL&hm2l*)_O%9Z=ylqBm6`xS8P z!!1~jwDEufAw1HG%#R<^mF)iGcb+1X{!)%D!z`+|uJCaZ>%^hQ z(h8GFL3pr|&Ogk?@lhEg*Sjf-A|(-lT2$#!(Xw}dTGU^q!TMhFk)qQyD6`8yL@?TO z{1Pk?84TD3z6q_}q6jzzdE>g255#o1U^r(=!$3WLx7(7iLYml>+|n zP*ur*n%Q#0uS(-6PHBzA9=okV`5Vx247^2foiIkA&Fl#JW9o4ckCORe2em61)}?6@ zEfQlDhOdpdDPgNR;{lH`F2DZDc&8&B#=Y|a4$(!FUocsB_w4gCK@2^m$vOSmHH1#oRi@ z=1(By%8;lejmR=?SD<^Y3B3P_w6pTaL?$2e`@Kc0(|kPqgZi&Eb+eKBD9q(b4yfScfdaX*AVX?ohSg!zZ*U;Cc6a(uVD z&V4%ofA1z7zi96_hTu|l)9q3tuqt>-4VobQk@Zp;e8Uh)yABNjC{S!+I~xumpwmkE zk?l#MlSc4S?=^>WQcOoa>kjdw=Ovr5@UYvZvZFk3lorMAssT~)u~+-A+oSP+8ha^9 zqbYG%np8!vg^#B15AXnSfZ9M>{B^uN4to`*yZVHJgk1!nCGZmYl0#yZ9h(!~Vs#Kr z!+6&kHm4wL7asW&NvTA8ibalg>IhV@|Ni~NolYf=`(+aNPw zAmmHfgD7GVad)C|P5ns!j0g;>OHjK;QAg~R(jV{?>iSiNorvq8lx#*u0+aa2bvsGr_$7r7+od~ zt8UQi^DQE8<*Sdu=i-OgDMxYpn zwCN?b*y90E7PUEh7EnSwJ~-hC&xi8}X_9Wklr+VUMe+T-jjv*pt0{1vJ+xBOE)uS!;W13&-~4XWXzH*D{7< zouf4%8Zmv#&ip>r5YXn(;yR7A>XoKbOysF`SoN*Hhnio<*$;YhLwHLAcQ#Q@`23nG zYl)LS)6GDiW{oy~Fl%<-QZUuhJX}uxF@Z}C@tSs_RO$FN(Xv#g-_$?;sQ{~zBWWz# z;}@^jT#!~bYk7XXRwgug)TXp%%Fot`Jv#83HgQ;Vu`5t4sJNAoK#aYE zjIW9mR(RTb=574teFev}#aMh2zCW7gM?SY8V=WHR^N@!kpC?$AL(?NOQ8IF4b9n%; ziTjXLyike!h0->C_v)Zre}IQ=fFzlKDj-W4^7G3$gouOP$I5i>>2b91r7Wa4bu z*doqt)OQYKy6G~Q4c&|7g)N<&hl5t6?5_A}U&}VG%|P{5Dv@X{N1b>p4|+VKIbJSr zBVfk*yx8!MG1;5zDcQlvF{;GYMgyR4fBEX3JB!L}@AW%PkcPU>Q%5eAU!8`-I-{U# z1&_>!?D#4Ku3{ivX>$(ChPqrdUyr&Co-2}W1l>*imr@lB*X_&`CXQjLTK^Z5{*tTE zxVIR{V^e~RvmfjsaJA8m<8IL5V{kG01v9fhjJGARB7QevFQKzz;0H^Ar1TgaJB5m=Z&zw|@h7tQC=70%X|YEYJb&Q{yI+a>(R4-Y?vX+W#Ut&xCM{BA&z467Yuazs-t< z_>Q$pd26Pim^f2l7++BfN(P>zCqRG0fsa_J%a2~Pp#|u4?whPU$!yGV(gyj$tDtXN zD}wRQL}t|8X3+n@TolIIu|C*+Uu;BCAMpq$Xa#)}mA~I_|HhJUS@YQ+{Su0o3Ii@Q zemfuKeUk~U(YSxct%?VLg(Dh`y@hs>4q=^caps%kjHOAu(AnMG@*kaB-qQ* z0{~L=wOeD3{V848s{QCb7MPELT zG?6JNhBV_SXwo#YaA+)myQYB(rW)EuYqrPlPW604%>#;{R4LsKdc;D=SBgl4$id&O z!psV(o0eTs&v{<`AyIQfI+*`{u@B>M&pqdVrUIVHoTokL5f2NtvvXc7kFqxT{afxX zV1C`D$2+SHhes=LuuR~^kVnHp#6V_36X7~>pd!dYjO@V|DECJI_$pOd^Z8MM0x;iy zVmiqO0zE>G81U*vk=X3$+Hgv$dUw=@FDFu1=}_rB;v)y3X-Z-}>1N zucGD4w`0;fQ&ud$BOZ-?PdUrIa}NAgGTtGc8O~jddV~Nlv)SE_#B8;9*(^+qhQx}C z#!uaiUAQaGImEyKEC9iOye6!#HWn6Wya`Yf)^#7#)>ELyBLAK-OkwA>2-AWb2@2JI zgDmb^a~>2$z)|=kbnUIKeep{bHhH5dWOL8*otiGnyu$L4#XW|xf*1$U_8opp)Y7TP zZZmHF9Tek8tgci8cWYwnQEIHO2d9AIv$=u@A zy&%|O2wwnjNH6%!Lv>TShQVF}2Th}j2VE^K$CZ8jD zgLbXH5-!#Sr}-lc;+s77@Enfj3B_EYgrbFtnq&~*X(*f!^$|twF-N^NpHudftS_Cb zkY~Z}?L`#LQZzy|I`!8!MmTa({j*O73A*xQRJr8O8nKio4M36hyCd{s?ypQVt+xDd z&(6@Z2Y;83*+szv{!d2TR^NZM@aMXRZ0XaS@C5yT)xu{veAdb5p^;`X4NaX!76^@* z7MlQ#lNJ*NjR-LKZ=UvX#5JWjm^`0c2$774EkRFZ-;o+6L9cF2zdEn}%{eoh3Mwgs zjT7OJ!4nHzRZUyV+FD+{SEl21a*eGJ^nEg0^H|G`uq*;Ud#L1BJ!nBQ@BPhZnh zu77Vv_K%O(8`>}E8=AAF`QOF!g35!^gW7{af~tck5dp8*L>f-=0~Ywl7KZQ z=(FlDHa_V@JHviv^ts6701X5)WGj-_LR`k`t_;8SD?3u%iyFBl_9|KaQk;A>M!i-) zkErE7ifCoMem?<(Elf4$ER}H&DQlrKEaWO56zh&VL>7!4`fPQOS=uiW2wyE-FBvHU z$11*P02G5=ch%rP05;2(MuK=-WYLMHGf`<|lIf#}MfJ!2rxJ{sK?T zAcD>>0wTxe9lCWT(!7D`q>^2_H=rw^YHrpVyqKMD4c6C2BFxNy_v~zSpjkf7-sIn) zEnryGPu9TBG`~C%94)%?8Tf(WlP%2Ok2ApmXtz`k0jn6tmR0&P?4>FSOXuNf2NKndR6eW0b zob;QB9l|^hfy!5%SmX5CdJVlKUh2AP(_6`N4qy)0?eU!QnzHW`TS>9jEN!;ZBbS)~ ze9rQ_B7TlqhmhZHBPGbRWr84pO>oHM{Shn0{@%PtuE7>SaHjV-H>KIjHZw^`qdqkt zEsLFm(EmxwzdE+psbp6vATO@hjgZQV_oVd7fnrAb^=gUNw$!>~saw>>+_YQ2Dpudp zP{rJCYKgh&RaQNYrpi4G{$-Yt$}1C4gO{edRhs2zMYA+=$x<^zrtsFR{crLHQJ-;O z`*Ss&Zo4$f6!-mZ8hfv}!r=bp(RPl`Gdok3ilV^9O^q~Su{wgcw%Rd#n1&7|0T>~* ze*l(m@V0A^XX@4$P@0kN!MaRkbCq8qmZwIhH%*wXW@@wl-fT!-RdpJK7JdV0Pi5bb z6S^I#);`NWRTn_qHoew#*gXv6%oOiBcxRuVOxtlF5MPLEY<} zzUe;%Lk?T0pV)N7u2K1^PeM3h62R<9)T(1{o%sQg{x58kQJ6};zp&_54R~7HD!2fShB!>b1BgJ@n|S0qxQ2A#&KJ#HR4<^K4w`sQqW`|X<|AxR zU?Mq_O=9pZnGGx&#(tI3xGpSpn#mW7C$%lY_^|8SnKshDEZt9PbA)un65QB-$H{Qi zI}?{&Qi$%O$(VKxn&X1;W;>r=BAdhD|w7%5}e-Hs7de$q*3SobjGs1IF_H^ z`>#7dB%rJ7fAXN6ov#_!i)~+p5x?vY=`d_n?%6YJ)dKdkvHz&vC}aOozj448)a;f4 z|EOqJ0{d!ptAO9M{KUX;S{t={sM!As_Kq1oG{At0XGiQl)f-WU4;?V8R;~8+m2?_X9Sk_-rUjpPkk3>k zN+q8THsH9zw1{C^2Mkl$u8NIYv1h7frw*pBY&XE}QwQ_>7p%;waL9B z)x0TX*ir{~RX%HC|I;2SiELLyPh;FelrpRc|G}`wAZ1t+Zj0qyLs6?S^bpxDj-G|> zTtwlZJ~S2CE`@G@<@_o7(-^vpT&)cM9=Tcr2v?SJqc@GC=)`W&LH|l`nnyvRJwzGl zrHTHAeUnZ>qBB&7eG@_vP!zt+P)G}-3`VD~#|SGEBP2X}#vlWZ zISu=r5I{?)y1!Ngjmcxp(#M>DP&m8psO3_5P-pdn5K+$IP^mjypGy}Lcn+rRYY-27 z;U@5sA|O0iTuBH?Mi3a8>r~Dw7>XeM#XS4)SBa1ZeXJ#&$xrk4_+53lcVr~udZHZE zYmPizT6QjuYD>PH-D6}c_StaDNfQ8HDEY-NE!|1g>H0MYk!sI=8%w>Z@#^6PqIy^b zd}Z_W8C9_A1e%)|5{t9g7e@R_d>jrY({hV)Gu_GAp&bj|vHEcc@nw#13uFcSOnes8 z)giHE_+4G#KE5SeGCPySGYqH$IgQhBz=;~fieGEmo@x>WHORLJap@HWwG7yE1EQen z#sw2Q382b{YB;GgFo87iB`S6X=8%){wAnr%SUD7%FJoYeEvq1O_d@gMPzN$?i3H|d z%U(>y&Ik93`GPil&!2dU(XX0bm-k;#Lb$VG{yfLxRdO@uC1LLtY6*5eUy2G&Fk?ndw94 zBqPlg5-US)xfZZ0Y)K|Kp4nutwuK=p=`)il?JAgB8wSBMwuP}4Mw7$}*Y-+_Q9?@D zA$=BJ`j{xI6%k4Jse5_cH=OT1G|B^;nS9uuHm3NVOd~Q}ySwX%06$I47(?4Z2PLlt z8O*8YdK&FL$xL1F&i>ZYb2`nkQw_|MZ6Wr*d<0RIvt#stREA$&Tyw*X*}%T#Z}H@l zQ!S?XG|SHR!XhOR3myBP-<2(U%n1jyhJ3u^$aRCXQ57Mm`&Ur!u1ohCHFt*u_}0E3 z;1{~ftt8ZMDl}RMu*a3;ZjeM?8G^9%R=-~u`g0wZ=@qG$Yv~2#ec|6-973|I!oA^F z>ZWSszTxOCh|u}`Slz?6rr|V3R$ftMuTUZt5u$q|nO<4;|4@H_sF=8$92Pc^$aOG zHw4ffWxbJt7V9^J=1=>OlpXtye*a9UGldrzZ?U8R^`Xjla`OUG)N&z&`mn4;VY8{p zZRN=WSQyvv;@xw5cTftpjs3o`c0?F?;1s5Adhd+~&|V-tXC&tSUhXXfui#Oyx(08x7>*st7iL!|-i9(wCvT{(Vmf?vCO!qw z0~BEf``_79s)TKY!M$QPL1ZY^Utb5ecDO=55^{0k02aBH4jE|wt| zPrxW#*2)uwN~%N>@|G}vX0SzWeOfpaMmTnJC+(*&0O6I|P1Ae*b=IbbNgZNtR259Z zvBTil;CKn#8y1^2=pOwo;h}&2oybn_00l;WnZTz56=d(A*icu|K3=ZHg?_i%J892+ z$>{%~>g+ZVSw+cB;PkGBQcn2?ksB_k257#ID%v`QiR>K=Mv4waOG~F44 zm%Ifiyg5u=JRVt^xn%f6&tb$E7fgb^wej84w#txqC-Zzli@M)_1J)n*qXh)fe%|=JBXr>v zkcWKa)x3S5URb#SoN3XMZQNqA-WCC97=MsHzGihRX>;`7qKy5kO&6wUXlp&70$E`s zvGM;iXMsIrWv-FUy+sjJ)ZwC6Q-sa5&{MLNB!S0U}B+%0UK7I;~ptbi&YKy7ab-eeMyl;+Z;bkjYsmBJmua>^O zPeJ(LD#gV{`?8}4vu%ko4KW1;mT)h*G90lOX{zg}X?*Xp6IZf_SwS04^LhUe<6G(K zZ7As(%Q|Pz_32Hy6gMl~aIf-IKGdIM^M7cWS|KU@pQHrE3}|?caLJQWUXbalX6PdVJ^4T)z(u}NFqB- zx5i#_wMk9UN!3ZwSwWZdv5<0qXp3 z8gQ>aR$$kIV~nOp(1D~P%F!0z$0Scol2YzrcCNX2d?X9Ho%e^)qn7YQ9Spaer!Z+_ z@@7}D%re*5pKm;wG(;O>)8h--%U9OBVsZj-L5lF)KI^hi5nH3e34}knl5O|M zR`#Y!l8E3N^JoFU{`f4&PK=oDSWA`F1$BpM23|skEP1IYdKlKVjd>D^k$PT2^gzf# z`MSKmN2f>pKQ#05INYI5t%nMzm1)w7bsE7eBl)x*{}N)q9i%KTylZlML|K*Jzv=*M z@GD2Xg&Fl&9CYQlPsNx;-kR~Xr@|rKL@M+!pJ1l5EyMwDh$3D`iAYm{Q018iu9zGNJsac zxr%n)G9rcDR5R&o$)g1k9TXzxWY~rdi10?nK09DzNkGrJLzA>mY{JeQajzmcwl7$F zcgg#s6uG^eL6}uo!PrrBQEqc`R(xNj%4>X70to?6~!Le%HG& zDoK|OL5#$5+lfJ!E|=dOL`!V+p^SuCuf_&Y%a8xuG^)@8>vIg%o99U}Krz}2$`K9`RIi1Q&Vj^4jG6^Cb^8wPF3s+RSlo|cVQc*w90y<5_ksAAx3x>Y*?BXsuQv%qc@{IK&p6YS*LT1X6Lu17{Nr`*ebKl z+DNa@g`Kxz{Oi6^N{S=x3nGy_9WVmOMFqnL#>aUiNT^!Yt5~kGuJSS`2W_=rwFgJ& zB@#qiMyij7AB3cu)D+WGV@&`$R>+QH^7!0cIKk@NSifz zo@sKO-eDQV1l@My_;JVrT^k^(K-ydN0NE+#Mt?MvPP(Z0XOsA);S_qQBOW z4SOO-5(yH&#((gGcmZ6Kro;aj(0nGa>~CWuL_aEoj+%un%ZIGmd74F>kreC}W4Wiu zMq(YH@xreS(%F&bgE!dKT7|6}Q1&7w3F0*t?UrNgf%-f2_S|W`ku2{t>yvL|mp3sw znAh$BN}-BVAZ>x`b;Is}<7tPx0NQr#Z2FvahI0w1s`%w~*aKd%Y&M$o{0?2luNGn3 z%y{jbOV65~_QW7IuYa^1x#?2opq4EO(#|Uw;1y^|%M?q=zNo4DglG~|TYQtXP*Xn= z(cRin+>H8*{@8DJ6)=w_Y|!@U!sz7WHrS3@gWS*)+!eA~HX6Shl?7%0&DLACn`G-x zw`aA%Sae)$*ayrOZFnnu#k<`$9rv^s-mk&A(bmX5&(7rB#7_`i_<(1!j=$$`L1`4} zY8@!mS>{=NI9rF6$}{+#e`e9Ld2PqreYEU7#0@9renxD})4AhLJC-;$C_wxQzYb>> zN;2vyusZe7EPVZovkD2-1GMtJUb5^gmgR&C4+V%7wF^k;MLun?Hjg?l={3~!=N?|2 zG4*?JIzOF^Pbl`}-I#@9iDJxY_ZY;7Z9(e9^cLLlTA%9}j(rsSgi4G$%*0%H)Jbvp z{v=g;)U=$rSlV`EEhW1EBSU)vtp4@L9{=P_c2Xh}f{nSdvu)qmUGrr}DL`sH_vV7* zU6``Wt_FlXx{t!0@@=%^i_Fs!j(piV+%Ld&8>r(2iSNs{TQ3_2FNl{}e0^UMLM6;0 z$Rm?1BjFpMdomUE;Z(DC{k9|rtn^jGMI&DtlKq4edFsQp7e%97-9yD=TDVwU1Os8M zhDlaIex01(>&}3mD5qAI6ADsgkUtttNA^0&GoX^N_cWgPiVO4wbM=e$cc-=r8{T(E z(rQPi#6JJ7A9+{J_>nfn8zj$#l0JHQnPRV=VfXDKwREw!GpMKHfBB;5r?iHe0q(NA zr%lM~F>I=NmP_+)ZQv(pd>2ZCX$yksaK5hHcb4Z%GqmK+9VpzaZ_5frBiqW!NI8!^ zbbz}G5y9b27}8t224KMvntBsmXVyFxwSn`vc6s_`WJgeT4W4)~ab(VbVOj)KBHx~L zD5SJ!nAf*FYnz}BXMH>l2e);~b=?-KTfWQUtclsnFUS;IRlcJhnxAj6^%sTbuSscH zEMF^4-L}fE>`Id{4Q9y>3ZN;cJ4lpZ13*7)xX>%9Vr`}F@lN1c;IG$& zBEIOB;=ev!nCUqys_cq?Zemr8F8^rUjeJ}GvQ68nwD$gGJO91r&HvEh0+){!-xk}2D3nhxnT0;F<5A^0R9d1Z4!9OCzcrwRYE{S5!X-DffhENW79DT9~q_Hhwng;84s9DVg=9L)}*s;$C(}cJxR<1L%yIeC2xZ$UI2W{NByP- z^q< z8sfm}Z(dAIb<%A;D939vLx^uQLV&pdl<79d0r@xO>PFFsrAM8H;SUy;dO$SN<&S*| zsbPI7kia3IVRf=e4+pM_`t*R!A>@083wLheM)X9opmWE3Ou*q6SWU4$o00g;$jjvZ zfM`;E>vOd6bFA?{k20PrqQh?K%rFg1<>PiTuBB)G^_}%UzY4J`B{UsE+5!w_m_}2T zSX&c#d`b?zr;i4Chb(51N+IpaF@7EGM(A(Y!qpq=_D0VrHXj;o{fBSleD3Um9 zGkq6Er#l`&$=^GTkF^?L6Cxbm<7b1J3FQ#_TIo#5G=yYKZv;JYWBL<(P8PBP%J729 z4&xo<_TqHI{8Q8(iu?f4YYo7!gth|~h{%#Weo7+t{J8TLDHzB9C?H?J9uL7=1shyJ zJc;~t$Pdo#feScpZ75Z-2dWv0U z+_G=b&LB_2ayF5KgD}JG$EU>fSFWR%p!u#gUW8{qUpJ=Z&Ue*NPYpooHpIj=iEcz6 zq^LtM=e1G$7t&u*&Ttz+g1bf=Fn`ovM6SqPdvPDSJUGf?#tT^d`6h%ZY4@afaS!kc zwa>vv#6A5MK0loO$f~!Y@{%5+Wp(iu76=I4aXLI7c^>M!hbj2O8p6qByPtVyfqkJ}<^PQQ5J?8CcC7tyjE0VPDE-iT2R_M^U*~cBFxDWS zLwJ7us1`JQ9UWqB7w&{Hn_||rvmaPB@^VEAh%X0#drVdZ;6?&4-GS0<1 z#_MfSHFZl_zCd{&|GtOyhg-jysgHSW-S;hi4MShDx0QA0C>P?rB6E%Yt&7(&8zXNL z=>GO7_D{A2>5AIjG5Y%-wfg+2t1iQ&Dv@p^+QokdM;WdshN%pAdJDE9}itPdb6OXu0x|kLZExl$U5V zJ4RSqjy!I+SV4$JNZLkoP|yDC#L+FS^r4#K;5(f-H@x7kqR-b(4Oy(RsR}-gXX{ZL z0#->Gxceq^7lFwz^01GrXYX?~ssW5P8YQ_2<)#fqE*x_}S;mu8XFwt6=5=$JUeanf ztfzF`4Md0^HOQ-nS|G%vi3wtl2XD9esb4i|X}J1Pr0h@P)U5GOi`a4(OFoU$j-OPI z+VztM2{h~jCeFUuXk)K+i^m2G2g6gvkCu0*hd94wla!ihgjTa$Q;jl5PwV-9oK9ub z92;b}DA&*e8hdZ-mVNTNzETJl-X7jt5;0D_t2I1! zK&IrIbmQkVH4$9M@0g^&lp_K4+%CyEM~L}IWIL780_+_XL9vbwY_aJbD-)|6Fm`di z?p!?GXV;eNCNz?O-F=XyT(k8LNhv4{>P(9e(jXv!RGcS3aczWFH8M{e=7X5vnshcg z4y|a&)hZO-Vrjf_VE;mj33UUrI}|sBmcRgvgiK&Oiak_v>gxT!VSa*?ON)>FEYzzi4&~s6Vc?a{GbNs8~W(dnvz8SLIF?Rd= z=tb)7=<7xx$f>}E|F1~OHZsp(+wxfd#}CFALRsAR(#qM$L`lRNvX0`;U!=^hWI?a7 zl7AAzyIRHcA<|qla4$;bt?U}m#TFD!sa#?CN)++vAA%baQTI!j(itAY*5q*7v+vITHX zg85yO`PT|2ooUll&Y}JvUGEf}$s4tMPcq5GwkNi2+xEn^oo6PtZ9TDV+qRzA<^;3z zf4|x%`>n4Jy1J@Q?&_+(`d+KA^}9L@bptmjQ`7a# zE6`5Vv`(!lg58QQ#HEEkB$BUm+gY9p2+A6&9QZnxVjX`) zilO&))w;HKs@(CtH>`XX-SJE@>Xjx96DhIp;j6E>)U?wK=gb`w?!M^PZE-XN-@gM< znDoAqRa6LDiShaiB!9A39O_hV_G1DCV*@dVr2 z&vj1nv0Wy&HJ)M4vbU&p77}3MQOxZu=dH-!aq~6|MNsZ*ipal%oq_ic$if0(SR#8` zle)(;#3~piCI`dLUJ02WU>W^&2{)PuWqRpYgWXLx@|UW zKaQ2)j3r#|%RJQy^J6qsD8`pMDv7ZW<5M>McU!-q5a~-~z(;sXvM>3qIuY0o`(Von z^o#w^Zf<4LTZeL6vdy{3OLODQw-4JE)D454Tszu2`IzB+xUo%+Zs#|^F!6cs?D-b7 zHSf8;H{x@h0S;jW5>xhd{gkWlQ*$ed!ePhBhZ|?n9G+ZiOP}HEulgrHCW&q$iczFq zH80-D1dm>PFDw{0W41TLHF1`;j!%LZH0D@Do$%p4NNC5Xa)UK@{h& z8a_yS^5wa?2g9^v!c~0$WYQeP{!~e5`+Z^2mVTc$vd={p$^Sz7Duo zOZ0(t!D4#mJ0cv}44#~?g_~ZYj;@DUg|xUX8s@5SieYypLE@ME6Pj`}3roneo%Mmm zoDPNk*`Dm&*kJ@_8>qS~poM^>at-|i#eDbg%SHeNZI@HlbxQQ@o2}IUvK8F2{Ku3& zLLvhIHO^iB^X%(ewx7B$q$pXYV!}Yy&Cc72Ba4FNH>yZmT?V2mIc07Z!)4lTZZ(!U zK#KKI*5vPb2=MI6^{?X0F;4DAvBvxhHgJ81!k^iCh!>%WIT~nt&bjA}<&VvLTQJ!1 zL&_u2eCIlp?2l#jFZ%E_2%zta-<27aN&_S@krUGz>Gb2FZYSNO1htTNl7jSMuBEwX zL0Lifr~*lj!u^t{0@b~^7s<*oTP$Sw$VIwwSY|J`l~J${<8gHc62PCZIf*DjT5$q6 z>Z_!q#5nB6nt^@+al!#b|xr33dYXLIDD;$p)*52KdI{&+V5-~fVw zfGB7dno%cV1=zcY0PI%q@093S!+GFnk{D!=(!}Mx{j#vXMkZhddgB5jJ=Ocj^WR|p zV))XGT3{>PwDtN?L!>$-e0%(UqgeSzlA~G$F!KU_SOT^!Qs+GbKtyQ}{=uZvTFuC!n z#^%6R%<$9TGr1Gp3}2JTz-^49?2upm`RvUzrRt;-o}MCFje zi3Y=l%r}e2k%+}1~5WH-ct-i!y zE?B$sg`=HZ=3OcDfP>Y!ADvATGSPV9QB{#YBMQn2QV#48DH|31M8xHC!tt61W?x@U zb-U-VW8R4#r(DiAvzRYD-x26^CTs#AtM+hVp<6Qly zPj9{C$$N`Fvu)Es<`=_h?0%R zj0R5V-shJ2I6@}0R#)yw&n%R-?!KS7c=B9yKqhOsu|lI(Foee7i*Ppv`;|Ln>NNA~ z;iL>;82zII2h3?xDXSw^kM580HH!_%#2wxwm2V1@RyAnhsehCgXdKNEB2E_kK_4%) z7AHKvU&X0b`FKWAvKkYfFiO=KTKtXM8fjr~l-;W-9%%OU9|Hzw)abla$IU?(G1ESu z`65$Xo~@N7)~=i}v>0E<*|IB=z*-wI(}9t;n}8v=1@Mzzq)3wPoiy*A5Xc_r;0V>^ z`8)R(j{Jl3Jx0({{r0!I(m1U)J6u6PQ>}Y*;0wGHO>DG8!SRWkM-HsZpINMM7nsbG z?4mhVE3B@B_Ojndj=>kdC?_;;X{+NO@l5|wc%}8_zC^si&+jLn)|URVI3(oJ4KC^}!|E(+w1dYnqFrf^RH7-THXYRN7$WRL1i zN`N5si&CE?>}SDl4eV!im{$as4vH*#oeJiH87O%uzCMjO(pY;kX^2jHG9_3MQ*z9> zn+|idpD_|un=VPbjBeainITfV9(uGu1T)f1`Lv%dYrp?1vV_5~H4p%@4{>1H9JrCk z@@2wqX33hU9SJV1oeJg-VCFUq}WH;SG#iLqIwqx_VS!ma0@H-*k|5$T`lJ0tUc__yuD}7KOpfC zO&tG$n~;d-8#oNW;0%uc*o_Uw^}ZPf(QXUP=VS-KwgpwVdhzC?hij;j4&&=EV&&Q(geQqd8=X>9WtQE(cmudl$kWY>(#(uvox8s=d(5 zA(+5Z@6?M}QLBR&R2_19$)^dZbmjIRze%9cVA{F$OqRn$h6JAm&t=4agjt0P5XJ@P zGySM4uXZQ}R6H(mR5{pMmRH;1v|+&?$yD0eR@+!vYmK0~*_`3poa5S@o$E-*W}ULS zZeguNYmucxj}&w>Y%w(|Y~8pG?sdoI%=V)UNZQgdCzC(qIQobny|x?Yn~HIU@db~a zlJAgMomf4F-cu?$rJ*u7r`A-ZZUS#=of2mha~|9PgC_bH(i8cKe&MZ{?ET@6v4p2h zmI^z|j>fF=wx0>!?I}G)Mv=>fkR&>dtg{JWp@rEKki{0G2We7y6qw;Bmlq`K&K7p; zLD3K>lolwuuEkTtI&|AOv+Ye!Q{F!}7w{}&haQFe6dmO0GG=GYipQDL=u&@(w9gSD zc~BYyZiU?VP_AazM~Kl8AvH1^%p+2THC95UEc>uU%-?BiJ2&=8`u_Mu_HtJARs8-;Co#i3x>O& zJ-?5Wt7w>(({&DGRUDh}5Bu$y5yYG56dlP%aG#&YMi|muNwwtEK^lC|lBh$Uf7nh2 zD2%gjvt;A<1M>MBfz=2CQbAqRpPL*5ixuyo&Z19K_s-3k6G!>Q1_S4EZqX$dneEz> zOjS-VoXa;SMRQsBmpfeM&e2~bFlq&!?E5aaW>6t$P)6n!*SWp7D8y0CLC(GD^md`K z8CtoReUPV0mYp1JYfMZemeDvHygCJ-#j;&p1IwBz4;L(zy*#zVsFXLfujAlT*|CzP zOK!sy*k?ZLwP_AvihE*RB$SWS<*jU-AnF~O5=G`IAx=22-QcgF5_x0(mzWHL03;Qp zaJ3ljf@$PNW^5+g$XenNzZt&}it$4;5YuNV!+j3*+ymRbIlk%kBA1oVJ|q>0r9um( z))UvJvm$0x)`(}k%ib&EmjOT5SwlkN-z&6yB#uFi$7^EP;@RXQ+aCodshr#MX1gq9 zzr`lkS+PFY0EbadWCBA;|0dk zp)_&97aFittD@Y|AVA)d+$WAyp$X_0ofqF1Y0FgxD7sV&y?XJ}m% zc5L4{X5qdlj|n(IV;2!+$xR#j1A*rhe-&Tw6MD*3Z{$^gzboKsAZvLZU11|KV>z#j zsG{=;EUvBN2030g08ux2VP6i-W1pjDYad@IHHcDq88=Cl%Z8HSasDMZdv$ocGp4VL zi@J2R9F`BVaWxvYW?cJRUi@h6256on^7jm^gsVIILZXBgt0&jC4Te=w_2^aG!pG@k@$vjxO=n zY6kXi)}cHi%Ur=E&0wplgpXZHu?40`YFx4E>}~4uiJC-1FZzOiwZE!70jESE{2XZ-4QPXZoHcxX!x^+DGuKge8@|1kXYyH$&IZm%2(x*o$9|^68xm zbkW=1=0I-kv2+=B^5&@f-nb>0^5Y~2C{LzA7U*NkMO>`uIq9CPiBa1)g<5w+OzV!U zK*`pd7{GkvTSj0o>4i#HFm;x3y7~0VgGrlPN*kXWlTR0q#TJ)wcJovbKp?Y<7%9`N z1Sgjj$uIs9y|ho?u|xl3b zxNxEI6=g?D{wb9!!?;kzIR5$-Y060gfxgptX@Hd<#X$4JSkZ%ejf>Q&k|x@DZOT#d zVq?sZD_#DPrZ~SnHKq~H2QMNm%Ahsf$dAfiO#E2FtT0)nXj(_qCZNuy==qxy%Qjhe z%Usa26Afa4_UbGN#U2|of{i%ya=3REgU&2X^#H{!-83pxUbyBg#kNGp3o@}4EZbNl zCcuhgIz^@osQ#0N9$38?Wq%5vMS2=0tBkFJggP%SC9^R{svqeBBsfWSH_3EmoK7U{ zd!65J64bTd2UCv`(tCIb?#~a;pw$?4Dc%?^D?q-;a9Nk1+8O67jDa%rcvq#`lh`> zAnzB>eIuvnmT%Ln2f|~6afGn-<)6wz%^(BU&xO|fMj~1rJXI|XUrLIkjR;k)YA2+s z(xcR1or;SXspP`6$ZhR|!U9U?S&6=)X;kABf3?i9$AZ3c7FW`7FF0kd$OFsw4hL_?vg~#Y_*{jP z`(RRplt89p&p6&@T8tH)<&;_d!=#~%D-Y?SyxQ4+BWjJXIuJwEaRgpM{Yx`OOj70;kZKCD2!sBkG0L99@*%3)+I zs5Wd&>RCb*{mn|aLXzcOW)vId?FdAC%OM&*Vi28zlT%2fMe8h?mS+r{x(EiNlqbqD z`6~V~U{@Q40Tz|7n<;u9mBCP@A3g$=VXrYL)lsB>f{PlYj-Pmg8`>FjOx_8m-*0B` z1F4GI4T9Ubv)sv_{7G)tWxZzqoMh&~%JkRayn0<@ZsEbW#b~~hbc5AdAzK4uKm`%U z%r6d%Nw2UHUVAALty#U^Z3KrIUz5{q!(%Pg4w_xSOJQZR?@;gRx%dlI zt{|hxLDn!^gx$PJcwP}QUcL{{0Ynb4kpVeU@l~n_Q_gKqQ5PPpmCLI06!2oiUR#)!se z2tNKHoG}ZVkyKZuR~EP5-|+CBc=b#MWODm+-13K5wff)`d`*m7NJV283mmCaxV>SQ z1S3jGB~ncpm7CR=rcGDrXO$-f5@a0$4v^Xi^h6mp=GqDZ^Xz<9i zcNCSw>ra*U(uJ+-)ju#wA;gPLK6v8Ym%X99^6?pMe zKvh>Oa+kej<#k~vbf$@xk34TF@Yx}~j9JArs4?%~Y;@hWGF94TaWvC!ofvJ&P0cUj zz;DTCFi-HCdYRwuMipy*kKkkghz!#axeDdN3N=rqpIUZj{m_xV`f4!FJzT8tRYxsL_NRcuI~vAZP(}kZu|}ha$_D zuz~Y z z@WQA?3FoY0Utt7=wRJCrF>i~BOb*q8BjfD$#oGLA-06%rF zRhHTPGWUd}MPCddj&vQppfa~f$Or}=`S_uz>Yi_R1zGBO+Gyc1-m6hLoCD1$eaEl9 z@IX{YToK<34}dB%gYe&F!>+9^l}t%%!pC} zSrWPrY$Lo-52Da;ir51n)KkOroDNSPycEybnGsDopP}nXj zcph=d>ou!WL)W&SL^IDfNo6f@M@4F#(ll7!5G+ePKW z>*WFXitR>%8=!hB@76&6LiN<%)dk;yxltCx5{kYkjVE9y(If5uzo)>jK)>d14ju5Z z(7t+Kp@+HXo@%?ye@4DvgvCNSatmOKk>?;9<48cnK-S?aLi@_>-h(T`b)t%~cmH(c z3&VKvM>dLrk%Pv^@)X~t6s{bp?FA%)E5hI}Ne$FMd`j;Eu%W@pb}^LLGUV^sGDIz` zTnRC(8l`r9?`#^i%&~T3ap~jNg24}8G^11Y+@cx!0aj{%3~bb-fw)z1!R#*%(aU=+ z;5`c`sP8I;ZZA;L3_#;(>l@Z+h5k%fs88)e-5W;SF4bKPSW8za&z5ldmPq>0p{S0*`3~x$IsTVuiRV2m z+%6~wOIIR)#v3Bst=Jy&T0p~YRjDo6WKF5o^^TP zqM`mmZ9Xc5bEr@mlXIW>mkyIz8MBp^H;etP$XHS^rs!gg`LE40m~)t}h85_{k3i6L zt69+7o_<*=$`#9n!i3z-r(;zzTZh|7@fi+>wgibi`GDJPV4bDjHNaZ>Y%Hm3n|qf5 z$^&hjhNgzmb-F%cINfxP$b;#y%yg7ZE3zN`o9EI6FRF0Y}Dn*&wzuXiIm)w-C;G4mckZab0^9_ zXAebv#+n%DyonD~CxA+Q)*EhyRE*tRL`m1}H;U)!K_|XCdBsV`138>@eURO>ZONR& zx?XX#zV8)*!!qXUypRE#?fG}mIvFdkiu8c{G*&s)Cgz~S5(cE*h3lxT7pJayERKuo zB{dx&Vp(Yw22xN+%jM{7tu%ynyQ#yGLaRhgjmntv_%rh)3-H`|7~9Q1HhyfNyqeT6 zPJ^dYVkaf0iZ8?4aopa0qF~`3ZNCPHRrHq3WUi%+(#SRWOJKIu7{f%xX?iV|rBVnJNsuywtpWZD+aNR@I!WiD!+M zd3OF{*5?qx4UjjVT+KNso8|-`n8K$UOf~G zdU)ec-0ficdKONw`yHlp4x3rH`q}!PqmV%8>`_-N#EpFKX;aWKAf_~lbw)xs$<7Y3 zORdpqW09_T>THJJR1vr&Gt44O4ewthsNt4UTBfwDN+@FaHGfcn?P1>>; z^Gi2t0uEneF1>R>3alf_-flh}_g=X;=P~ygI1|;`7Q=rZOV^sX;?Ke|vfK5nBRjZo zuslYphDfY6CHJ#=c&k?%&+D(0jqzce_1J zF3F^&R`lyk%tHZrxC`Of*1`519PR7#LGGbG^Y*MRE#U9 z`Sn&1g^CKc2F0fy_L>z3gl$jC;hF+}S#!Sbg}`(&D(^~JL%5Fh@n1LnCugd<63*Lt zzF!q<7PhQ~%Blh*?-blhN7bhF+%B_x?s04Mi6K-X>725+>*+>4>B#!h3^!vo0P5X1 z_e^S!pq*)K726V?3%QO5N+0=-EJj&_`J^+=1gDaT+q%SCTg69mC^YAWBI7rtb_9dd zRNbPsObjkoJX(~#!*6V!2hTrqybhp9Eh;CY#wI;~AMq55-)G;t^Q_=_EQnZa#^t|R z{b4OGjT%vTeGQc)a$!>uVv8(;26W!sc)Bmuqu5sA4%w-rX{`xF@RP2ph?B1JO;(7J zM>S|;Y6TlF8VTSkiFoY()8qg2KAJW~MZYM(fgh^yxpptTm11Nd3qsG6hp z1^Q8_a){{=+M?cI_C1mQk)9<0hCnAee(l%l6!uw1Ok*pWSsRNO^6tzX!Bkjs;_qAU5NS<>VLksfnniBxcTZ8Y{2r-}#X4e?Q z!V~;~Z%B9}P9>p3vBm=^06N##K<>}D2JM6Mky(^m#k^wv10r#{g4TM@31=ysk&dny zO!F?%mTW=q6cb0EK>)r>dCnjPF#z|Hw6x|f;=>NDHO~)Rc0<-8+BkJJOG1|#NTA>3 zb^jyXXWom(D`(iPaP?E~RF5-^W`dpYj|10CqtoVjA9FL0iYtB(02br+$%Pf=LUzCz zTSj6BER=_`p_4T_5OKc~f508v!UDUcXl~{pOHOJU7DImb{YiS7s#<|`>>$7j9L^b> znqq1#W(X@-%>p*JG zltuELPyavxBj?8^pp55F)8*Rl(05|;BJ(?QbnhZh@bh)ip~4gESO_xRwLo*KxZ}QG zq2u?4i$(`tjyvd~kj1>VJqM0dyGJvQR_Q)z+nuxa?hWsRWw}n9^dS9Xe7kA36`YuJ z?>jw<+RY2Tm8r3|YKvAim!79TVnGz; zf*sz`k$W0p;fPcG2KcE&qZ8cb_|%m(oewm3t`PAfH#1nt)hub-sgaKxk-JT~t2YT*(_wN}WgJG=Bn!U*Fl^e6`>zkx!H~dG)_@}CV zP2cm|s2Z@DzwW!-JB0Z6kGmD!5IN5T{cznrx9mG_w{AP4TvvGgFgvP~L?e;isW&VJ z)Jrd_fFo<3VXVaQeLJ8(r*M&|^VeR`^!nqqIOW?+`7Hl}I2g{!V#Pc9#rsU)(l<5A_KNC#sSKBxeTo zf`;sgzghpF{wtv}0hu{Jz5px61pljqnqh#XP0iVZfKN623-KeB@DvFxmEZ{i2JoLV z%HMvwS-wR&L*x%hLDOI-^aeQ45E+`jE*YY>Y__1ZWh{ZAVJ*5NbKd`1@rCFmkF{An z-Rb=R3J||f<$YyNd}jawQVn>HzEH%7<;wI5vM-Gov zBf5=jG1lbz;tFw4=KU*8@sjJBt?`!HW3!>28qr{@myf< zzI4I(&QbQ1VO7y7UwOFF>#c9AdQop~IgJrA8nC8>lYlX$3TL;w{?cWB3)|Z&JOJcFoBEOFGtN zccoy%3NQE^$H3Ws++=Nqz@mDlNl?8pAs;wB0p4OY1S&q8ts>RFLn9z9`N!l2P$}5U zz1C70Rd%lV!Un%2-|d8bufB3K#J)~$G1c~!Ov0($JYKiC*ldv=5`AJQ3{>IfAIhRB8yr`xq|FFocQdk5~xFAVqyde1?xMkKOOYdVE z6D-aAh#~tNB>U{A<}0N8{Go4_gfk$I*^li8T-c5{D(jeI#D4{oe0HS!$3L&gT%Ut< zZQ_h1>3smB!=r`Hj5z?27KFSoTll z=eypC+obnwKjmLC(kIw| zvVe1Q8oKfQ+qV#y|5p~ifRZLbKM1~>{Di6Ie<6?ne-I{UPz@rgs_Jy4`X>T@HV2i1 zHOkAzfn&sX%hxB?x;A7s@Cu(Oo(i6{cq_n-IO}{|#GajIZvhSApA38WSf8giEv`2% z{9d1l0ASd`slAji=%`v!yEX_0R$B8nT_HqljtJxkW(X-i^79Mw%Z)?>w}Km5%R~p_ z0*nX%FljvPam)FI29ho7Zlo24N_`_d@TTZd=O;tI15h9fVTaK-m6+zD7t+)mCwj47 z)P#TaWd->BNMSBCT<(Rn$MX!!`lBv~r={94@-o@mj@qMsLvV$+szxVRVA%^}z%8XQk<>J?*8aH?qI1QmO$~MCxHX)9JuVQi6{oLv;HtY^9oHcJ*jzj+-2D<08KT z*caMy{1B-)e^`p7e+XXL)Ly&kqU98pk7*D2KK{m&E~}{KSFK_u;vO7A>+k|JNtc{B>9u24aHPrpDbmm5FRo4*O9ThTJB}mGj|d~L>w^!>>)0p^v(`|4!W}h zTkL9lYq%KF@=fg&y$DjyX^s|5{+#8RXg)ZkqSX|0tus06sYVlc{TY#qOXzbQ1NVQ_~LQ?A2mod zJVPQ&t^I_60H`f32_BnKTu)8H#r1s~I6@`~k_#E7K{QhQZHf7l?ne{TSm%EPP1h75 zy>b$3ed{K5+G>o7G8^&pAmkx_`zrT2`(`(*s-{R?eVvNC?E5R91~N3fSHFXie_K9R zZO1u2GtM&ze9uGq1K%7w4^$n~112!^@^@t*HL10L0K=w$3Jkr%T^&dpbUx|6M2t%H zT7^DuNIZ1Cl3i~|PO8Q?POdf1!-!^8kx3_kU~1#~|Bz8dOQ`MyiozZkX} zwc>rh12O`tV4JB|tG_gb0QO3asmL%izzC8Q)}7|X?SF(NiZ)HDR<=(c_DZ8xw~rN_ zPZiMDfZl1)2ZjD#7l1^QruJp*^MfT+xlx8BN+j1#LD#MyOVCfjuP+)``9l|-0Gz95 zlSqLz8>5yrF+0W9rbz%g)wBIdp%_h&O(`CoSJURvNZG7slTV2-vudYH9-aTXEt+CJ zu2M!fl>lVXz8h6(qw_MeYNk^koxkPa0w8>vhvL3O5-0@3UO2LJT7~LdE%!q~oS10Y zhI-@Mg#LIz1JhVHB6wdyJa;N1;9#=*-Z^yR5dvU(PXrN!c))l{kiJhK;(_dVZ9@b% zK(4MV_zt0$C&p=3q2~7N`((V0y(JJ@y}QPSxR@Kn--uX!MSj1+!v;nRn^BoM0+^GS zCTjzK|e~ALYfkXU& z3k)>EA3zT&gSQW1`N23~8v^?y0X66!(|hVS@{n8iRWM2D*>R>VL|)Ji1f4%6;RMqZ zBC+16BlrMG2pUexDsj~P1@fO~0zfUyF^?hKw6WIiZ`IiWp+*YNSdb|A7Z0O~DMAx#_$Q`rQ5l4c7NBB1!dy5n0TVlS- zJ+v485SuR#qslFHr}hvS<|Esu(vZcC1V--Ay_zo~qsFb*JD@m1u{?db2^(9d%ssLfa`eWN;s~vkRp8OnLPm zxq;m-+q=HirTGM#(YTGByrbDJ*t6mL%k!&r4{!4?T294Y?kz6H&Y(l-3s!O0(2Zuw zGsmyIy`39ajDOSMpM7sHlz`aoK#$TFhL-JptLuGk8vkCn4>(56TT%1RFwT4C2HyZ5 zwRg_g&(MxsmiJT;>#uj0PMIM_?3i7Ia)`x*IN^R>b?R7M%pbI(0$1w5aL-&Cc@q7i zF*2cyK##I{zE0ibBUU;?Evch;1w$<{I23#(AlmIO2%$9dg@0-i1%SS0!&SV9#+Us* zsa1lg)63Ze;~?e$U_Vhbl$Vh zF-i(rgrgAENAM6dCeJd3}>W3g?yy**z0ijXuZCOrrw)z^z89 z+M(G&rC2d@`y94#50A6RLk!J0O{pOWFJOZL2_sFmddI-dVGXndjRutHx^d7g1@!?^ zi@Msi{Tz?B3obW>O;NsA=rSsih<%h~Ac%uRk!dU(;K`&`MOFNfkWe=+_tF=qujhFlxw)WN6l$phabw zHrvg*nkjAKAs(kiLr;gzO2ukwHccnp(rzggkgeuDpBklO>B^exNmGl5dOIzawaNw< z*~{l)d0u6EPO*$LRxH6xMvQ*eW`|~Vo^t_)-2Ln0l6s_TD|T-~Z1NS0qHcTSUhA10 zux@WZCk3m@U}SNwLTxbOvQ%dH6Noffl;|=!DL!+`VB_KvBebIa7XgD7#Q;@3MO5tv zz|6#lt$t!hjysp#Bl$>SnNLH7hfDXEZTxN?<&bz>aNrkw6)9SL{CEgI9zP>OX6)0^ zuhFz&YLeBIu(Q-QlL$d_15){QcoOahIyQ>I!_O&$R-<|&C>ckH6)FTnBa)5OXXTPF zZ9%^Bia?)0Z8Liw1$7upfg^JzkKl#(RiJ231#YBpOU(u)+MWeK3Fm-7ZhR?bxwKM@ zS06nhiB@e%;UP2FL zK?a3HKO|w?$=$QKnRt{q&1M!5*ZrCLR0^(yHc z9pmK~p_e=ilios)A?JA&YZ7XT&r_XIcX56qP1*Ck*)lHAOXMKyf2aMk`M!6@)WUIa z#R7Ra*;CBp9bHR;`{?14FI1gWflP8eR^3819MT=t&>GYjP>JQVoUzDQcBGa)1U$Bz zh=DFyf6vYs-A@73UeQeplY?U*CoxII98T2PWalIfThs~#FIraVS8jeT!KP~BAW<3=(~ARxhYH5)K?A2ME3=+`}$_+IhP z-l#MX2oTL98q8y{Iw2u_i3abl3#17Dxs*6r-0FX*I5q@ulF`4^pT_ z&s}%dL+#t?qCZ#L*ty;U z9CI!Z3f2#>?60+Qpm$#X)Sn*vAZn*7xX9K<*LCywE9|W@hs?JJ=@*bL;kuqjlfnqg zPa|A0`K9$iQP9@X8S}B9Q}0!}_3r088H`U>1583K=ageqpW~1t{62ihHN4r+v%~L7 z#!N9B*o-q{FP8!@kRn9y8R)6{&P1KRYmt^6&&7iPl13GZ%m-0s+%iO+QE@@{SxaOg z9X)$jIHIP{_+GPVoFPYZpiQn#30SMn&9;l5e1iC)iCkA--4$omFIQR$IS$=4%)@DD zv^-mP4;I72*kUUX$zOq2MN8CA(!yDGN4a6fe3W2_x1JoNJKTToCA-)&dBP}PwP$o! z<#z*6O|bGdmH5)k5&|E{PBl9R<`M!bqW5I7TNoKs}E!l1I4x z+wBIkGIYl1`+6QqYCnY@Smqp(?u{=Oq(dm6>R`8IpLjKdvESVn@;w0-%!*b3ny?#!s95k_QQ&J*UxwM#P-9+s7JQH&#!jNmdNssTH4d}TQSNC+g4~<=4GF*cdu~aPzYU1 z-{Z+b9#ZN)j&6i)f3) z294pv^o!NF@Z~!EFhbJGK2hy#$Mw5mg~41Of21yFKJ2?dH~s2lPM2#PXQv$zBm|F2 zb7jl1A3fdeMMh5(tD*_q@=r_iXs_OG2j1W>U?QW>`TrlXzA-qmwrh7Lw#~`Jwr$(C z)tQ(*QOEW~6Wg}!WMbPkCOLWD=R0-kJMWLKuIl}(ckR9IwbrGnkx&hR-nwU)kKyfK zO~7g8J-{;ZWytR_gT2)BAlVU&Rs(~DvESyW`>NhTg_&h0b&YJhZEe(7PR|e1z z1TC#==GRd-n9UMSK3g1r(}O5|^YkGVywO_qAY0{-#|ScmvdG?yu5wcOR1vaPLrkey zaV}(fTSO(tqGpJZ@W^Kg&!PiPxdKn4$$(-Zx|Yd3&h8*bocIelSsM1{i*f2OC6Rlo^ax)U6|LHYe^H*a3^ed-k+|6L!E7m2(br^Tq z*KML>hGxxNrUaru@Y(dMpLZO5shmg`6X^&cblAdO`#(H}f3Wk}wR}a=edr@KI{d&f z#$a9y2%cr^ez;*_t*~eQ1s#8-r3(D}rTjXacXX!~=h^scQzY-`$&8}RJ!vOUW2dVXY2vEVzP@*DcH-3PV2$3)#hQ5vuINUX{XP7OE;~^w~gk!^v?9 zH|jZt0hW+CbF9rP>RfG|329W~+y-ZEQd)L05b#z85_f6Y) z+%D_lgiMNtf*&q!e*nK>dtzg^UZwc9KF`eKFxX|E+0z$`o)7?-kC@*pv;IeREGJOL z;0>4I=Ocg_D~|mjZQ(ck8W<1`);q6UkMn6lAYw?`e6zAWc;S25n23g_NJfhI)Qq^s z>@|;Qj|~6jNj^Tp>>AtgN{%o!ROc#m7fIcMr51a3U5&$h(i%%SO+-@B?P#sF7Gt_^ zsQv&Lza*09Y>R5Sn|$Tnv9zUPkLx`Xow;@LXFf+EBz=7g6KH~?T)D2h}EQ{0yGfRf!Nr0Wy8+ajPFx<4A218`jQ zw0KZY_R#_a3Rmtds*wztW040*JR z=hW%r0(IuK=Dd&RM29RF7fDBjK2eo79rZ)6Dy51A?)XakrCTy8zL;{|Gyqff@%&Eg zMYKiIe)8?ze}OaXiixv6v@c&=Y5&(w4E&#Xg1Ae_|B;tYpCBVOwW2~cLjEf}{a;t$ zn<}s&+mQaf5(5TBnetl@5hTc0YFH{C*5CHm#+mc5$FHvkFMynRFauS;n zR3l9>DW`OHq6-wsUTJvIGCUWrcBNl!s+=!+|Gs)DH;97P8kFMuiLI?}3cUU}7~gaR z0>QTsI^lL?P_!|rF-636#3cqw;Z(&H#0`QAa;JkRGUC5~4`vuhLpec3%=rqW%;zE$ z$;&9gfhD1`6uy&XrWyc*Vo?_0z~tZ>Q-ktDrI~34QZaTXrkQfKX(nZu%eJuu{!|A= z;0X-4A~rdx_OTqW`7pR5Ggb6_YS1wI*~y#~`bgKNugJ03_>vouCT#&u!hLuaYezUT~dYF~tdMyBlE$_q^Id0Re9d8>cFKLd#5!TAO*R(vz;u)CN<2-yM zcZK#!>ybVoIRKA}f0!lWP#ObucxBqW<)YYzd}=i9O>b86)4LS?p%7wlK*D8DQp{Wf zdTca^=3cg}@xEVcB;1}rR|nFI%y=1hoeF0}Jty7v)Qd!SJk?XON&GWZS#Gg*@m{yL zCfrTM18MSXq<3t2$w4mJj`T7=_+iMVStvuZ+YR=3`y?P?)kd%%Xwy81#7n-m5SF|m9^4Qyd&9U5S+ zk);tp0Hsx8;DWg7TvbT#-80OLXpg%+8blW!V2mFbJfBy}vI-1RRSjqYG@7+YaFE>K zx~mH(oP{+jyO5G#agOa$ytY7h9>JXot>=*^9?F3=AL?E5oBxWJOJ zic8LCr`eaPFj8~XL@1bi(FTsxLEad|l9s!dt6k!p65gH?u9mu&H*T0-UJk9;_En8J zRK8>BZJI8o1OS(JqYfdf#V2)4X{*{OIyXJiY?iac=g4H?MK8^Mj;7KA@S!I@j({fIuYX1?BQKpZeMZ{j7>S2s zeW17-QHLbsZ}2oleZnzIq8xHrU+{YNc1$;K2f5Y`2YtwyL$OsyzA{IYaH|HH@a(d8Xbsq3crO~*Jgfla)pSWd7P@oy|QsGhi6-7 zEIU+l&;BC4?w8k}9JA)v;W36R5pz=b4RPfPwZ~{&$IyKd6jLrN3xL(n9WAAx&vkID( zAn4qDOdO6dQ=&OZH%>qyF&F~pVt*Y!6OoGYlkzss#8?rZ!1571qpxf^av6j;o@@$y zJZ7J;6&$Hli_W#<;}U6ZeRoXk&69hsM0QRj`QuK-I64@WMI-r~n8xrm=DA{>ZON@m zeiJC9$^ORtmyhoD;i(-Fp`-#&hpc}1CF%Fj2yGgEULfO)hA;Pmc8KMl^2_KK+h5m$ z%OTNEe+XwZt$=YJQBZzpJRZ=GAKN+LWXN`P$f|xjzSal-Cbyof4qU5$lEw-Ci!}a! zg*%_$DXj(I1pk)ftJ6=U(~1sq?%z7SMFa6ab=u$z83h;>E3eSUgc6*+I-Lm07JtuXGtD6G6#x4@iu!$$Xt7 z?h}u~Jc4{U|3%NFqc8ydko- z#_P{g7VP5jra^L>j;wl#$>41HLM=_o3Q&vDwlagWXWsU?e}n!sz*GsSY3L9S(;gT>#GZfpSVTp4g!d|;1W!F-ua7zy3gcVCx2Xka!S*sN z89EN0Auh@_OY8Zt!xI}3%dq9bXk?k-Og^V_e;n?@c4I@K=EC^)I@JG(t)-6qi|O_X zu8Y{qD{sS?A!Qv%OM}PN@+1R6PTR!W?CLyc>X6})eZ}Sd@9tBT1WLmUs zdAef~i|0Rr4#&7!i}ugg5AVPK>Xi#5b;{*AG6672eNz?xGpCQ?xhB0DR10CHy#gi< zqdQwrVP9i}Ni#o{tJ+xdE-(qCpDtB`O@WD)W z$RTA-@vgTR+atku# z7icVUZ4goUEV!su{Olc$-AvEe{SOK0y@Jg#K;WJfjjq!e=+m{OsoA)q98bMXqmf_= z;MuUF?Khy$e$cF0(gG=f1&D~&jovr>hK5ZmKG|LDC84KVo80Hzuupz*FhQQ~1_XU5 z+JT;YnA*(3>TEXH31h_4bhe{`BC$nm`G#}sqXnr0J3)%(bn`x7k`p#byy74TbDA}z z=snG{`Z=BAmP#${F`vO&`!Unl2=>Lhits*^U018(ao=zuw^5EVBF1pmL>aCVs<32k zRXZ57&Yg6(WM1{taW4E z;^R8Cw5yMS{-VnKC4(VSjIiu_8BJk!_xCg{rf|a3hV=s3Mu1ViBR|eDGmL~6q*bK< z4C@WS@}adkshFj|lYUa;bVUfq?~@zO34E!Op{WpT=a61}E7`KRJKuWWU;~gm!GdV| z(G0e;jh21xeu_($ar_x7P|9pm5<{TF;!=138fP%1|AsVj+q8ZMZlZ~mxG)a(;KjGz zuLl1+9v4tZrCpDA+CA{*r}X*PXqe^)ZV6X=@QQk=c-6Mt^?{$268Eu46?w~Sr4DFSnFtEW!gdgr3=AG5AG>$v5WSQyL5Ai=b>zVaP@_1k!^)OC)pba z31uwJp(i+7>CcgOvCr9U^CE;LA!zf*X_J1OAiIp{xzBw04+D}B`(x*|}`oBhTts{kFH#0(U-~N1ooMj^y z68wn~Cn9StFK7pkqXbA;3kp7WSX=-06;lw!bx_jqYUFwVo4DIQe+8FkHxRk=<)iqJ zL3#zWrIs=NHu?7{(|PK7-P`#wrpwP4ZVROo$%yO=|Ib$zJKSglO$p^e`Oj$3X7XJi z=|9<^w7clI*`rV8m}m-3CZroEO!-QW#Xp07qJtS=162Hxogn=P)KG-vg!F`}ehPf; zgVUMGGva^2EDoss8Um4_Om)Nj^)m#FW+|u-j>A-_yX0O8AU43uMY$#aRr)J7L_UO` zM%+L>Tr~C9I+`CIy5dCWG?m731FqXZZ}1egR2+5V?e_41RI2Ur@Kqd5Cum$u;vIU- zY*AGw=D$1G<+ zX6pvH8g|rEY1huw3+wML-&RTr>!DELC77$y71j~=_-a99A+ymyTU%WBLV-PcT*C;*4xw!)2YMpz(Q#SFfge+gr-C`@ zt={p`x45)p#}JhdTvP-&9OSMEidctpYgu!%P}czyll{l+m@2j$9Z(3P<(kFAf&Mcs z_A|BGGHSFK&C*eb7<|rw_4SdtvKZ8&PF7i)xj0CDZpt272q5_}L`!;LLWpxCyP_sg zQlaTNqaqE7a1?d5@odt6cW749^U@d;os?io7GBxHx4OK!nZD}_P9`Fo)rOD`EACL& z{(Bj3zUSNAbp3&Wsg`2Jih{DDJBZps0z;N=U0Mfs!bnN;0!-TAG+I7tZS*RjOqZf> za2^Pe_;%ARH{){e*%<)rsCXVIgM6%9inX^TCvl=?)V&HDjFMil*!)rSUceC%niE@n==|AE}+W5)pt zw(%p&uJe)M;91M8)0>deD0o<$3fr}@(5S1+8vnZJ)fmhrhg<@bVSML#nF%~-I0_@B z4Ge2aywX1*KRUNCYZ9kjLf7vHVX!|4^duMGLG(zn^#PKYr1-YJY&&=*JmwIh)`x}E z;}u0@ePGqvM=T|VyA@Fc(8Dk33-1~82W{#ep|s#OYq`Egt?7(x#7MM_aVZV@-N-}m z9cuNa#8P+@nT`R!zm>$Q4bozsQRT6c1^bw2=?E>lFLl8f+MxQY0rVkxA?-C|lV(tF zl*=L)X<+DKIig$zWS+c7m#}Xb={fgS%%lu*Z$z3!h>--Qu%zgWWfP6F_GssAm8BGZ z$Nab~@^Ub()R}t35T-~kpm{UEED2+26zJ9~kF-Z-61D((+uVG{^yRY8-_{2JL~L-c z8eYA!-hXOIp#`g5(S*%hv#!8+ulPz1E0J{l_9kT5xUC`ncxNT!Hyaz3peEKyh&AQ) zeLc&ybDv95MBsrs+NBhKpee~JM6~UD#VPyD054xcH@ssG5M;OeLzwFqmPW%ar;$i- zWFS-@`7p>~{};4$i&IhSt^qEqCQ5p%+gtBWRP_e_Pty;P*K~~ZIcX*Tw=3J!%jbsd zvx<@d|0>Q2;S6;(^|a3Seo+?N7Z+NGQt`wF!Q`rrd;ymT>~h`38JN{wi5~vQ52FB+ zK>PUeLAC2l-=}O5@HTyUpY_V;xY+jo@^_c=XNU*tkTWDJTrHdh<~XgE(v@0|37C_M z%&`V)4Z!CLZc!S`@m8ocSsTvS3Dj5QTlU}q&slpnQwK|Y&|lVKkf6kFuBKS=Uu zj)}e~W+u;AVU!htRZR7m9F(la(`9;w+e&?xVSP;}Dj9X0q+jWAn$;JjuVY3F8K3jLxt@eOHE)qU_muhE|4=->u{#A`-MX1Ym>0a`lw!qy^)vEVAJX^1B9Ba*AFBd=o$Pg~UXlcyRI zo13WYVpa{ZR>>XHg-KLV$fO-T`|gv01BET-Hxowo155iotG0cI@|=v}pn>1>4DH_gEp~Uo$Zpax=Opzf&Y6O!;mR!$hqN75Gg%{}RoR z=6~fE`9M#ryv4ajyiwY}bpV*QMEOddqE>>|S@#6OvgXX*Ohyn2fi$kts&IAwSuCsBFyn-K0%?6+BdhzaI6qE3U?r_YuW zlF~e~It7_=y`z;}5dZ8E{O7@bW`LTj_?%>MKf}cT_cWXGp9i~J1d;-%wut$Tgs^7Z zEZif+gBMqdNQz7unk!@?)(=G(3l@$(_iGH`ZwqSK?QqAi+h~w}@hd=Hp|GDTyo=*V zm5>lAJEpa+Yt`^O(fON03zV62G{o6s)-E_Vsc^ikn-X|+o$NgGd^B|2fB-OVAewv+ zor1g`_|}QvAHzOw^^^t#W!pXlgA8pv)cdN~-gNqG@hsT&=B`>nn^qp8eXfY`cz$Ku z?syJ1%>~=^h+T|578x`>lCdaEkb@#rgVO%O{)(`CXaN)~h}j4vq39$*Oy2+`K`MMy z0F_U9_6UlL_DLvw209n9NZRn|H0hNUO!_pTNJ~ZqMzKg$w+Ap3H3N%_^hycFxL;%` z{2iq+Y=Z-*pjaroyZWbkIK79KQ9g#diIh=2hWmDT^i&^aGZ{d8OWMB-YzmIAXjk?r zJlb*R^RvxNxH)L)>I1EKoC+%1^)}gh^)MKUNS(b#vm+Gj7AF+!N9XY)TN}%LMCZ%f z#oYFzVG4n4K5p^G$%X&m+20c}F>vW8zmZ=kf(6yqk5_v0L= z%|y+7|D4h~v9a>l$c(lL>iM>asPV>YPPh%`N~}mlrYoP$s4jCiz_}3 z-4y6_0yREes1}DchH4s-IvW&t`R5OpOrwumO_t?qHcV_a4RmeGX7QGVTzNtIQ_N@v zAGO1R`0K;V`{{A$G^I*D*-5qSeLgFh51}Ip$MviuacUKs31(01u)G5V?RehxAVi## zlVkkI>=;*M1RA?=ZR$EsR%Sb%Ko!g2iPN!GpmlKZkd*2V7g6PdrS^L@tw$JNXk1F% z^DFc6>MHuyoUekWEv&(C7i*23ngOP#_@t!eL2(M{)GYoGnKTeBfDk&z4BZ2=ik3CJ z`wBb41_I}4W)+6jk})ScOLy7Rob5mh=LC`a(Fq5vMHW71vqyjMiM)e%EGJG2FFLFm zpmR+tGb=lV16%kD!nVxFer0$^@BEj#Urf-m7QJ{4Xucp(^iL7$!>@!ru-&W_4XU`$ z6l)d+ic z;;C=x7|Y%o--P=J4{yiz8{R~moho0OeglR|L&nf3eSPJnBw0q5;>+$L65a#zOh&un z|9;4l-X^?$xP^$U4+?_AO?+r;uh^)sQnCHTF8YymF}YZK1ErXtT)b{1c}riE0#vb} zEnuZ{O{?4MKgUpq)0@Mu=O;}&E&pk&pToE3WtUM<$Z&gBDUBzGJF?rzD%_UU@uW7> zFI*OXYfb`ZCPbq7Z8uDjyEPW-w^{OcKmW3Xcf8_?vD^ZOuj)=L&{cWAd_4Igz!z%* z&p2vBs!U(mOKE)4w0Z*hBq9N4<-lFKM;J<>Xcbw|RBos+V7kH^ocH8#M*U|KK(qaj z@JvgHtMg~AhcA}A_S)jEE!+662MGwzB86XV25fp5)|s(Gye2{P*5E8J1^fCgKIO4E zFD|R9tSnpfm0-n(CnkoL&xc9rrdM`J}&c?`G)cUwJVUZ1&9>fD8$7SqcAYiq|_=s<5 zdFAIT-re;62zg}E%)G%H;$5V?!WCjQh44%IF@E8ZL1=d%X^Fa%+GjTLElQh}zTI6jP;&0?*k-e^5^twN>ws zM(}rr1Z0F1HtzIdRjBN4r_`|Ok%e#X0Dh~irVMoMmsi@@90USE5f+9+>vcBC1q6NS zd?CEa3`cOKVXy|%Fu<@>y{KG&o)G3}^lAVUHp);QR|u@k+*@1d7~qPGZ>2>t_M%VS^~+SL21jNu=bMDj_J8)9 z^uN7g@jr)eNTmNdO-bPYXCWMZ`MGO%c69w=YR+J4XYA}8r3UkS}R#(MylovqRGyhMmpf!!?KdmJ&ne81Jij0VI9^|wDv7wn1@}9xa3p9b`qn=t4h!P-N+FoqmLSGlE_J* z>Nb6oMuyI^TiK^Omccv{A8=?qtW@(BK7iy!0lvovG&=P- zC`y_v+f{9}g?_}WSM(Oo5!#M$v9#Bu`$TMeffoCOUTr#%iyD2u4(y)U#W#hMILTv; ziyhZW)q-@@nBtnkC3>#)jGGqO=7DaNM2=?V%OuN=9QcV z69^c3#fy}pk|7}{8LzbH2-v4OsxRXfj>9)WAnM$1-wmzZ=erwgklzq&(??0_e$caU zh4G39lEOv0AKxvg01But;`fz;<(17CBm<5m=MQJNHKGrc4}cOeRKpovB$$zzh|#}} zV_7}i#xloufG;)8h2={R(7zNLLJ^RldpF z4s(Ywm`w*#;s9g7aP%-M!^Rz@&bM#cC|HSJ98vgV!eBbkDBut-NozV|o?fH~Q%;_) z)>i~fTu=tyBy)p!mcB$T|0%d)|M_m3Q zB#Ha>+dnc+-Qrraq~lo@5<>EKDW$Ms-iO+h=pTv4?-9 zUZypxivNpYC~@W=x=vDN>s!n&a(n2vfS9Y9ZhbF#q_w(=wMM(c@uY&drljUmTXI#z zTn}QHo+J%I%&Lg(C$7AzyksD1?x$w>WF5MW6C)DW;q^;8kCN&VCj zAV~F;hYw$!LNbc-*bST6b0s89`RRwvk06b#cXZVqg^J9_SK3<*J6<;cqsM#vaOEsPsOgmB6hgxjR2B>cspW`OpeT( z14#yrShNhrjXSx=HAj4sjvh=H93D-G9PUj>pgNktLdXRC-I)mzxPl+0e2^q@iQ=F5 z1jw(s<~n~O%lgD=XBizZZX1U&?&hIc0z`J`Ht6Ge`OyxqEW()E#lJ@(OY;Rc$@)M# zCV2dkoVq&E#LXJeB?oqbkY#)_i#EG?$vVT_WqjZtrS-bO(KZJq$vOku$g?at4Z1tf z7M4W*j z3yNIQU#Mr3NI);Pv0S}WGPj9>fzY@H=ts+}%u;azf^X!$T~WRnO$!V0+HOy)p0+ke z$~KNRZv$!zi(8Bu8s%N&+)PGoDyrd`pW-2izw=6iO+T=B>(&glgve=Z{Fu>$nNPAu z^~6~FKmu~=*mjddzYBXuJAIUw+LuQ|uUM^vp2hh1^nt7PE7e#rR@YZnt6ViD`F0By zlX%?4?}psN(XNf>%7}9e3a&EHlv&Tqt4G;cQLvEQ5pbPu2NFkAAKrs18VHtDV@3}Mf<)SgR`XgFh7Hj#l z`w>i!V|gPUyPlVF&uImacnaFs#%R=#%|ls zLH#lNb22%f)crmEm-p^IZ74xelFx=zMG8!dyOM;y@v`~2YY8*MSG;o$sQG4xSx7$N z?LR8E*3PaYBt5_R1Ye_0f zIC6R3+H{ulg^9+QX1uvRj3@^;g(tk<6$^FlfZ0k#gu2pd#p#9ym($;5dl9Y6On;Os z?OH7kN-Dyv5_O zY(rLe3&jhB>twQ`A`ebNkQ~<(wtl6#y89!}b!aUQq@=)4yo+!&q>%IEc9&WkY(q5a zRl>_J!$RUQQYyZAaZPo(6&jihP-h$Dwb$DgbZs1sV_jKx+Of^4K})BbSoJ9<-A@8J zW!ZCO>V5~0llShK9x>gwX(c38Bco)>(#G++sgH2kQ#HW*1q0T3D|^BIe4&!W{fH3C zV|iEY^Xm*9I?{&h`2eB_h}7x#Qy;a^0g$SYGeXK92{baq_5HN)sRSbi6wzWnHRIU1 z0Zb6k3#JSlxmfAd81nTmWCF4CE8xK31%D%7yt|ndOs>Fzp=}!i(Q4Xhsk+ud5XM5J zvH9;TD149j-qyKYiBkLPihK)vni|5}84=o|>-Tbhe!vO=qM4BJaeStN|Y*Hz8{sx4}NDBPSdK44Y|cXsXl*PBS_4Go4t|368cd zp9andHb#@R+{DVLm{`;lTYUM*Iit_jlbDA3y4&C;C!~Eza@7rrlI3B5r&b_DgVBPM zWRDLS8L<+H+8=|X&?)eXxVtGXN9!nOEB@?~LPziJZGMgR78lZ=Wa&c*|k@ z;z!RBaJQph8?o&_5d{V)RbhYyPQJLy`rHER+Msk_g={%f+9(@6%HLcHJ;D|W9XFv< zX%g^BdwI>jJXjKnYN((PaX z*SLi80=4{gtI?WF;2rYxTL@ogVnjo?nAp}Lz7hR% ze`{G~ zL)iHd`b20L76Dd;hfw`?@sjZCEy2aupoclfE+)akF68ls_>-oEPdSyZYXXlUg)d=Z zr@|5t#q#-4xy@)EAIvSGcn7)(`~nrze=d!hGm4x|Prgh2&KAbZ>$Cku>CpQ2QmsTl zc9Q3|D)4OhPv-+NC`Ic3?0P=_+j{vEtYrG9>p@9byF!Km4*U-=x!!0tcP_)M6oh5# zFSVLK(U^n5^ixC^;yk226zj^g&O2eFbse!So$D`a#p=C$z9 z3^egwk6`!p+JvO+uN4sPJ;~ulhPQy9fm#9Pkwk z4B_~;_gE3ARoV;;IN?Aoy{^hSi|}sDj0A>{9zFNsTVAK1jtA?xmOp-25#x1Yj;I%^KH_u_#3F4iJWhL+mWFzr7*31{Zl%{0y9zQ&R|oi;=^;VbXR#df zI%BM4x>n+#8V&roUrp*MPh5S#G|HDP+!wqzkvi@ZpH>x+?_+p?>cn3^R;b;C!|^jQ z9{R3WBk?Dv6*2Ipg(Q9QzUVgg?-*cluD%LYjSgP!abWhLBiKkCdnPv%qPEa~D~-by z40G1MY1hxKWJ(DL(J|kAfkwyYy2CBOF(Ajy!93IT`nf(Jt6nsAmt~s=y!Y5~?kC!5 zsdB1s^@ecOc1XbI(vY~cNd3zK|B6KON(6oe4M%zPgME!MQ7rdHva?$n6oVSkpc)%! zewQ+HiyAE`V#dAD=-)TO0yXMEF*wSlGB^{$bxr-^F0kIes5`WXxr2|p-dz{-^^6?F zNN@+{6*l3_a?Pk~3+WY8n&>sEn?Yp)vszWG6vDffsI7{qtxly4VPi4>WwYc(5D@TJ zdc||Lar@=pC5(zlE9oZ-!s64{{r`hBe49TR57QNpu>V$baFr+e0ttBGUBb$#&XQ~sf4OzuhVgwn3QN9xvz&=i; zHIx`&Plj>!t;7gB0^wNfrrQm!RAcszwGuB0UcHI1lE_IFT_QM}>j2f=zOFYD6EuM0 zxUR}SRLf*XOL|{!VGW7wjTL}-?z6mCO3S9~Hg8&0qUtOREHz7p2kn<8YwR(kwRmZr zHW=t50^S^zY>A!6gPF<$WVIx}Kl>ZAH~}iv+2;S0gV7J<^dy}1ibdYkCSOC855 zcEm}BPGjy9%qBBtrv1%6z89{id_|iX;U0P7W!KB>VBRT=B>Xv7KhVEMLi(Mag0N7XvntZ9AlR$K_}5ua<644 z_i40uHDBV%gG$w?W{Jf%P(|43eayjok}^?5=NZ|?jhMzIMT#{*A-Hg$XEB7N1%L(D zFyK-#;DmKo%w@Pn?FH?6OFidJN(Vym15AnV^C+$W+Csj3f`EqqwS-9dPrTUwePghI z9r&;w1m^c0Bg}Q_$8q*v`32`HUS#R@tKd56KN_UrsJUg(;Bxd5)}>or>hRs%=Ae|+ zgis(bG3dX+!C)#;B{|@YMl}u}2SbnMfWgs({wV!@IZ-ooW*3h*kL+9!th2O1F#aGMVnIRVfa1W%?%wf!BojI2C)nsS>1wn>(Y zYP$t3i*oxJ&46S(3C&Nrw@UiS6}Bqr>JjZ-xHpB|M;xtG{H-9g3+;OGy8z{mqu5(T zXjS-=BxLRV8oMu z=m*BN;rDmleq`}Css4{puu!n@fNyZ(xk_|!TaJf`c3|GD23|wFr?maVCzu}{U zC2{yhx16AenV*oqLO=}*88J;DhV+&>q(XbI18@+SpGxo#`eN3qf40GUn0)2y-!cZ{ zT1SN6hIY_`5VXAFFaRdB6>jY79IT!l|}Y*zvqfpg7g+WTs5=CZ@Up=hN%7etFfsukeKO+VA7p8FDGJ5I|{Cc)L z+tmiKmoylep9r;8fPmeBS|w1?SU{w(cIr>c(OxpUxZn|lQRM_kWFks<)!{-B$kJLS zOnh@=oVKlamkL+qdgg8a#xwm6MB3135S*l1x8bWCY*&x7TEkcKP88!Zn81L)Rk;f?(OFwX zmlq3&pjL8~v!gX+vtX=mIR4^hIj z!>=Ga>KGXj;Uygk%G$)>LhKu!s%N7BURc@cZjf3!wDczK$BGu;pFD^i`x+%jDv_g* zn9z;N2gwE{K0N&LL%#?#ODj0EnG-%SW)=~;3rh6IATM-gas`Ov|Y+e(B>k$$Fcp#jx)WLEfXSfA@%vnx~(_AN48!?K7z z`Hu=m`N!WwuGiWJ!YyIz2XEo|)=<~2(j zkXQYj=9g|c#`sOid}z1XLh76Ga_Dj9ZLq{wY}hBk2vqpWoPV)Cz;Ji|b}oJWRzrL} zt_(zuM`1nH)k?NjEgA^#-H#1+$B<*6%;!TI%jDHqpY`P4?ItP{k~9fQ5lp@kF>RQ> zV_iVtIFa^S?=dWtr@;I@>e0~l{*kv{va6=}nq6j@GD7k?{-J;J;g$YK^aqo0FX)fa z8p3UU*pim|REv^NWco1TE|!iI>^n0C!86dvl}-b$ODD=_YriZM55Dk6<#aOFty5<@ zbI$`RY&k^sp$Bmd8BQ?P_P9@ORCB$i^)A&Z2QeQkxB5Zxn&gxaDb~$UOqjO>jK9Z;h+a4Ps_|udi%|To}>xW7I7d&|N3n)x<%&_)$o7lT?iE6w2pnUJ2KQAg_MNC9sAqjHqI)R$;C7ff+m7|HW=^pWn~jTVv?aWQax}42 zbxXe8!fXdoRL7H?mcmK=FfT-Zg9%_Hw;w5z_#dN8&YT*m+Yy{?970}9XDlMb!S?zu zjDpWJaX)QPZ36Q}4S#(x?qm(I0}GlZ6^&DRvkEG4do<{kAOjZ1K!xcazck2CL?ObE zpDuE^vwfGdp>V4R>vUjWC~*maH4pDYM-3RFE}F&Aa@ z^=b(pkk+c3ewUlJEa#Jt8}@ZhLri=WwY#Op4b<>7OQqqJqY^fYbOpWqNmfc_$u=$? zERWiStWdO9SvzSZX?vHC)KN0r~i335Nr6@m~oHg~kuMSa!1vK{KzzN1zIeowK0g z!_OM>0^=X#3$0wakNUVn7cx996L*#we>VjB8cRrhbg4QA@R{NRHD$G5mA;?lq{{U_ zH5opLIu=(6mOr540hL{mRS;(+xYCNc?(3LIB-rW2ikPrz_m~XGi)M{#lo|9`j*0JLJ}SYYe`EHiaisiC>-s!F=b#66HW~?-%l-5fZRHho3@PPZ zbRK*NN>68#@P!&~I4&l;O(7xBWQv=w)?~n!+*RumhZR4w`#)s8Wl$Yqll~ptA-KD{ z9o*gB-6gm~uoD~(?(XhRkl^kf+}+(FyzH}E_11s))7REX3#(Rs}^IPiLnHLq8OhYg)JGMiIyc+Qk@vDK?#H?#SDk;k&L zzbHc50vNG8f{wEF@wCW3aRmu%o?V)nh}$GCoZb8OB$;DHpzbL0LRFhR?NGmB=rtj4eZGY z+{|ve);~TEr;GGu?TX?pk|V}E9Dm6YSmRVhDXh+8Y;11wofrt9CF-E@j0IYSx4WYt zzXg0+#&^Ga7QQ}x?ZkkeUB(l4#7d`L4u7<-j4Wb|s)2sL-eIaCnP+^!N&GgtNV!+S zeDWR~PD%Y!w#5^W>%c%e;!be%`2QBS<$`6Xr~O`05xlWD z$7GMa?Mz0q-`7otx1edfZn7}Cgg)j)kd$y%t3MnHvfI+lTU7z{BR29Je#8vNw*+jE znL4!;SvkS%kA0r!hz=n?>eQ^9T%aLLzD}s2J--LNn6ZzLFT6?0u1uCR*t+S6>ZS}U z^rQwmy7+j@0$XDw^;&VO^RqpGD=3lM-xhk!0Iy!M)V1Y_;`~Z7eKH=+IeD9r%Rfww zB>7b2w;YUaK9tuq)aUC_G~JPUDP^9$wZ_4;+fWYOPueF&l~*jvCeGXVB7_yt6v;Wl zjOT-z7h*h{sYqM!IoIJSTWy`urew}oqs_tpz+zsb0VjTGZ9{rRzyEW0Y4*91R;w*O zq@zk4X@MY7o$-y#?6BkXf)MvYAKPXGwNtANb;VM!JV+3*zmbaMM!)TFPs=+{KF;ByVOhDK7&R;K-FF3`{z)n5LkH&Jyc85`|(4q&yDVtHaVA z{f5~G&OfdmJkQf1rGwOV!qu+(D7t_J)0%*@9!#AMd8fcP&MSh})s7AB0pc|f7X;Z7 z(8TF)5((yZw; z*bzP9j7wMY^eX8k%+fpyEErpy1{CKj#z&=GqsbNcM%z%iN?mm770F+0Y$6lt`eQnE zh6N`Fx}vMb9|CWnPeODtYIKJtt=0=$${W!2#n$Usd#XzRwX#N^cD)_(&UxOSY$p0$N4`I+H_WQruvWI3>DK4;8(NhO^^*GPL=wRsUP$%PVb!S z6Bo9E!uYc4t36;^v$fw;+nLx^sE_6LCnQQk4I26KajwAOJXJoNxqh^wi&4EK1E#HZ zdIYak1W!o=CS{I`mE_P`X8etEfWgd02&OlRG*pg=>EwJp&G;GCQHrOHw$JMwY;b;| ziD5`t%#dKf&l2(kAqC&&E=e~W1`XAI(>1E6|3RsU5Sh_3 z`@ULER4}=w5R<|Ln+}5SYc`O-wdecSNO4f<6&euh2hf$=g5$^;>zwIa2C@g4pb09C zS~L4r*NCG;n!-npp&^6V1Zb?OlE>+3>kdz~3qma9l_61jC!0Hbo0A6zZoU9n+3M7P zlT+4Fq&zS$f?jK63tF2CE43Zv)@xj~QmA%bWj%{+ak{i1lP?PnerG&xnSkQ62IB&1 z@j|c2^bwi~S|o?b=-jF1fJd?oM{I@dm?g^YSD~u;h~_3_UcoXfVrht#Auimy*TNkv zP>QH@+v+BC>tymlCMLZ6G!wtVLS(%Z)1Vw-!+O^G?7XznA{~0=u*2B+H`XgW*aets zH&EoFmDhrdDe6t6;fs8VbC&Mobp=o#Q2H+nWj&VEj}3q0?ai zRT22`#nf8E!ck_*z$0Er6RyBfi5NEi+oH)qh*-F3jW9Jd__kV#lMpT1_hXz51B1PfgaP%NX;o>Ou3+KZ1$)LlTE4bwi zi&HD6a|iCtx&g_8FuWNNx8(uZjmG3wq_Oi@HM>-hZoO5POj;!DcJ-W`v z>0H5w4R+-LOWNBU6$M0WqYy$EW2v_1s3;$yZa-;51t=*lR^DAjV+{;FGLX>)JE-{$ zK{1=>ua>7GVw3cv*}5l24lB9$By$gxRvg`P*W8uYGr(lB87B{+$Mp^8fx(I8ivqof z4eQb7Nm3{C5j|J_s14S~i+t^J86Smc2b!HhH0#O+`avLVce4DEf(rI27OBq~Tu6s| zEkQK+!`6QvAo$W`p?Uo`#eJa7fs7s9df6I|4>aQC<<}Ra~3(ylG8|NP?M7Qxp zS|f3NBpfF5@Ho|`8HQ(N^v=ye-T%R23qno2>3>y#rN3sJ{@|$8^>N}U*Qg6TZ9tCM!N*x^C@V^ejcDA9Y_B}(CoTdTmIXxy>0Zh&RIoxp zK;9Az$}1!yEDi?Cxfpq|T+}7ry<{#Nkmj#|vgXV=%Th=r{byUSc(*m;AgQ&`+#T^iLqfR68NM6rV zzOt;wb!|eAmZt@>MCd%e4>BkNFV0<2I7;zsk zeh6k22GjC`mSt)tWdW%xWovEsFTOp!+^MS;T-uIkd@Cz^nY$qt^j4@|l7#__{hUi6 zc(r9~7IugKO`M!P}?Ru`?Gz*Hu}qgiXq$8vl{911`|iU8PX_bdEUNGOcYN zz1QhVm8*@2FTXp($yK!eL2HE&olnNA(5JGDU@FETR*}t%@Q!;Dz+k;H3(&}H*&x*z zzf){Pv{7MmC#WFIH)?{k(es}r)|Vj+DiYtqP@iiTE2Nnepy2C|7o*uZCe`h)7i$V0 zw|VUgRej%_G}GMnh!lOk4Qsdx zN}lzr^MIT#F`_FJP!$c!P2P@8o{=dCKFgi$N7nZUf{lC(kE4&pMN~D913yj z|18lPzGDUC(a`0HxO^KYb%9hSDWD01O3}PtiX-e(9(Sj|tdRm$n0s=q5@~~a$hScTDylQqsKr)zWbR`-^8h_MT)xtOYAY^OYkQq zdHJtaVgygTe4{NpF+@nPKDWm)0|T_1y&?#^F$R(RHi5a5c;#(Kx<&0W)mdHVJnLgG z&l_Xca#bS+4tk=~rSKc_jr@&>mKMKJ2o;I#OX#9CGOA|-PYAhhuygW`e1{2BeiJGw zt?7N0^UPKX4SOFP+x?*9xDa?8+60FVL@!#p2J}0&%~yWTw&Pr`TE_Eu_wJPl<{I)o zbH%u!VgUcmKSF6eTpWizDq4D>NSqrVO9O%n(o(2bb+$mGsN7}?FtkvGOA3()ylzmc#RW2D+m5ncxxvD!UrsIWB8tqj!eDE0 zrL9M4+g0kvHm01Z$mQUF*79Om=x>~feXr$Kg(TOS>h_whWabIH{3>g(=`6k+f9svW zBqj~)1;vFO9nD{r?a30`-nxj+oU>iI{W=?5Wh;!Cp;bk{{o96rfeD~fWp5uvnr*X{ zUp!;!?y%ilFqOJf2%ff3>1WY}JCwK>e!cox?`^Zh=OUQU=l=^REDX=;4xM<`_ zf`C2D_OU$1g-AjB`ubj=0-x25=Bq4WE%4^%tYe`5H1)_9ZK06_FDK782%*;=N>DfGDaB9czU{G zzwGR6HLUA}_4G2pk(A-P96YQN1124oQ>D*ffKmdlE7Ljip_)^(j{es|&szMOa5LbC zG&f>7*>4kHmMopF)OW*1-S6-uu@#}Qfc4qyHW0JAZ}~1grcq^``Z65>ramyFvY@KK z1w-IjI4lt1s}dJVxli$jc_$%woR_z_2;s=R3@70_A6X$IBWoruo`q%ygbYE>8?z$- zXUP?_Ys%%ek0ifW5ljo=`n=HNr=QC5BQM+*8jHomSS5yNJ2ZeY3ePKC4?5+*8j2YI z8PN<{R+Ws!!5EVr`tN>Km=X{t>3+m3W5>&~uFhtWQ4R8tap#6LG_fPe>czrU5IMw% zXsz)6ryUa(g1EamG%>(AB%ir)D|VmB{E<6@vD$@?$o9nNS-Ed?!2PH6ty=XyJyiK~behcBzHRT+Om2!R6R4>@GxZ?A!&%V-8z?gu&g^`6Li2bIBC-+07YcGudy! z1IV7`;HO{>T;z}nPO`4)^spKs>N3J*A~y6jI|A%qGJV7nS>XJr)GO;;okP*jPsxWU z*)xPK`{gX5Y>SEw)3K;|mXdm{lbVx97ig z1&ilpg+45myn~HnjN={Pia32E3Gt!CvRn&ry7k1Fbc85PX;7MdBe-&iq|E#WCy+pp zGPDx*0skWSAu#2b--ye&SRe6we?1+e&>G-Vf_u}vWhh%5SGbV>tepHUv>}F+a=@Lk z&z^F~uNYZtrKkNCCpOvfCg_&1I>SEY`0_s@n=?%!gq*MV9{9hg5#GuF5!s~Nqp$+) zMY zG;w}M=RA6x_MF=P{P*$>?gZ)t$V2c!d$7;MvBv*0QB9u85BJ4^=Sixc{03d^@s2Zy zEY|kd*QF%pf~Udp_pt29Kb`fi3jM*p+jr zshYGu7p_NJyJtI*!bFRZ3R5?#N&m4^0{sC!v{)1#aPi-%u`SuK=?Ff1Dva5VT&SMW zgFUs=I@s?)HwT!_BIf-O%^SbBEyoHJHZ1QY{_Te0Vm>1W@@@bdO%oYvhG;TS(H&I> zl)0cGLtPrzA}g;)@_^OJsPd)u{{}%ucFXFF;RvW|?}EOL7yS+vf>SoHUzpW=ia=2v zgmO$@_FZOL+EfzZ5K=uZvR7-#!U}3sT@Z8YvhLQcj{d znj{swvnBYzTbJXFw7$F6APp!`6j`U2z3j}d`uBMIS!a9A(+9A&<0vPXqV?_b;BIGa zQRBg|)BCHoT6rq#^|}P}^sUD$QRh(2Tu|e1-dxZCc!-=n9ru#YmP>o5!zLtu*k~ar zb?9m#sC0N|(ItF{VBz07fj#<1{ZMMew|*jZt&(V&l+M)@HHiZcz=T#2G=&7~l!54k=O7A)gn`L~RtYp< z(m*any%^e)+%A~RdC~xrOp|PZ0@_8waoj*q!ZKroklZe*%z6BPp$tDmgcQY183j?| zz+QrvC|Y*HGIazsIQl^PIP@aC-h#dz-m~1 zvNXE@vlUuMAX)1q`0)H0ToqIRwoZ=>E%h5wKq*=j=?FEB9Lp?ta#j2`Mqnju4OJ_2 z0FHXSC)}pkb_iCPWvOnDM6zvoRcN*eFhJKM8*1IR2ujhM8vT6@{H%Oonuugs?wY1o z2-OQ&gh_;`mjt{63jCY_(xIUTI!A;h*qEl51p&~297qjkNaYp1tq}MDnM-mlALx(8 zosNKz)t5tZ+|*NkK8?j44apmAU=qN-oDMmS_8!#(gC14alYagaA#3pQJHO&J5Ck=4 ziGt2Add=I54+*4xj@p(&$m*ZO;!ZV?zUJ19wTjtXLOhOf7=n8wA+_mqYq>gSC*KB+uC3 zTCj7l4R9d1A=ISrQPgS10hPx3ah1|VZ)9a5IHuMdi$3l9d{t4}A!>lP?Kst8_WQOe`dtQa- zx@Dk$4=|8*!tG^-&rVq^(I4Vp|1;H!vt~$3Ze6h*QIWo1WLgY zX*{F6Ye~R@qkk%(V)>)=c7l|jaY531d>HXe6cXmwl7aKEQySt_pAx7akTiWmPqd;3 zvGk@SLLu8C68ksyAe`U?Q182@1R^Q4*vXzER-jv``A?dmjgEfbX-W0ss02Td(7?U+(s~~*zok~J}rxYzq zVJ1Jk{ErCg{4=x!cep8Zw=`YpXcC|Ja=qh0YZD>zcNAYl(?H2Z@QJ zN=#>NAw|^4Ry81M@w=AJ? zDt+0lY@h~YSuzc+Xitr2$-;vG3}7(2inLx}~{^i-o#SmIjBGA(g| zgX{r%%pK}p2;XF~z=c@o(=}-S1TxNqA+FhP0JN;WY1M&p08Q!5IHoi2LAHn!)mBA# zBaJ&}FAwC=Edqu!QSnsi0@S;%guZkw_6-N9HTbhCT<=Gbi4`gd@4nf9RH88zLR#4>-}aOJ2+A!F2i{JCp3{s9 z!j+!C7ExQZ;lQ0N)ZDucj2kFNje&032g+_q)X$x4&@(8vCcIWA#qoI_^qgW;7Jemt zef1(K@!pfGJ`h_}mkaBExP-ng?u0Lb&)T4Y8Pd4gQ14gyz6~ z`10ONIpwYD3s=!mG*XPDY?>IcTfxmJc`m_C07ta6V> zMDHM}ywfNQm8&oapzH1+f|EPM2@Mf%^p*!~X6tZyw$N2XbQXCU zxU>l>YkKYZK+rP0T-`=)^)rmNR(%o=$YWprE=->P+~!TaxR#_+F zk75OQ(@}E7UphTh!k)6|3WBfQB0RWOUW#RwLjO4Vvu)+10>#M|*F!V#s<)^Z3>4@h z4RR7XCAee(g?H#nwMBA3-B~cYp$I_1*{O=0ibsR4?GFcv!zbk_oRs{Slp%LOQf^dl z09~k5RhblKKVEuvlXvZ*@myE+B^mJ|H4S#4HM5S(yRz0t(#O#eL@X zYTU|WX$lXO*b(m>bmW59x^3!tOUJgEXUh3Q-8q%+_hs&c+o1!X6rAAn(M_J{4-WiH z{>*??Ofa@HVuP=ty(HW5*riDu5Jx*4hFD4c!j)vlUtV|l+t-aW4bOOMeF#K4Aob%$ z#!JyASA#Blq@gNh?q#&!J~~GljDCD`d;T-cZeZHNTMNqhuBf}%MQ~x5Ve7E6d`2+c zn10iRY?%#>@;N8)@q+EpVA)ooN3}@;Iio zvfH1w{Fihe#(`+@iHJwNu0naq9^L2ENOj)COVuygcyLHS!;{b<7o+C0UtkidQp4tpwUX&wRG*jI3xqZvTGj*!pF;0sJ+(E8`5JpAKL)WIa3P1jVPTJF$*E zd3;WO7OC&qT|*{5U(=34T?wiJ9xVlgVkiR@HqZ#R`5(8ZTw<>(er#AhFnC>3tzNn` zpS-{m4+<;*vtgeK44sS=hR^-@LMT^HnSS^H)V#gd`5Lt*2Z*9(JS&S~L~`GaX`Nxb zjbjA(pFkcN7I%pQbgsxP_5`gG0y`o`xEz$?T^@t<*gA`D{bR@D`c^##n49yD`xCXj zSNX2lWuYhYYL57#CpA}$v!9mifeFS#prT?a5t0$x0`YGXDr~E zF9$yDwz=>@tx;m92wwLmy^IyU(U$C)ruLXA(AA?FptK%9et!yVp zm$?D(v&aSVP@G$Ja}!r$P=+e6!@jA#;= zj5g~PwqX#K!@XT24$7N0JcFkKRd7hBh*PkmOXKMW{Oat=pWCaM3qv+w*F-XEt8|5F ziu~Cw+)oHIlP*SlQ{ylb4Pv}CwQMau_7)?Au`#bBFG%uQRQ=gP-ZnRb7buw9iHcuY z>^pG=HuedOBep653Equ;0L(tk^-fq;b0w zyb$Rt9KKXIgBLMmskNK;`r3I%tew$0mQ}jFXsPZ=r`(4c7yO=-Ze7q?j_e0QuA_J! zyeAmR@!lJEEwQG}{q70r-4}F-^m9LatjRZ^WdO;Shhm6L?I{D2aZ-D#6 z%0FkF%Ndu^;7IP)zncXS$(I)6Fj1yqEpsT zSPPKxz}y}c+ci!f7B$fA1qOj^h*jJxS;x3%r$Jxd3edM`{*3QD)^6g?m10$A z~90Bi2fH_`(fXwBJvz;BFaVr9V_$Nt4y<* zO41wi3t4YVDO;ME)wZg!q8!<;b8*`mm`19Fe;DJTPT1l*;rTP@-w-B~x)^+ow<3x= zGN#0;g0p>}BrEwwGc6E@4L9q%H35e%J*ew4`ELQ=uvX>2ib<<`f+jMls77c3t^)?Q zH;CS{^LTVB#hR+BbY9fV{M$LC=+|->wEIFtGuw;BmhZZmn@&-|+2uEHr<#BQhf^oj z1yv!o??+zz`JBoa8q+fTxr1!<$v-QPPC}Cu9cS6-RdBWWFpYpmHjM$j-{h(J%oZqT zu9CxC*l4L=rx=EWZ*;)KoH+|gPT8>@)uxNyW)ZDBbC-UZfVk3p#R4kvHS>?r4G@am z*7S%#Kb%80UXHa-aW8XQ~9#NT{Vkei^!h3H9Du{HWQzvaj z?_|qS$4s(Amo5O)q^x6~Tb38g&O9U7gR)FVZ!~xWUXw-;JDF~w z0*h?2chHQgYc|;KtIKQ+Om4zR+G+CDh&nsHlAn}D3NrxTy=_79bANdYh4o6xA8^)m z@DAABySVLEjisJTPTN+EWp#O56;oZ(x3KRuUijM;Qzd|bxEn;bEO*jb#~Fuc07X&B zbhu_TU2S#pY_A8yTrQQ4ZRLagmh>eHT=xk)+6Sr4`<0sRkmngh-5qhN`lY&ioBZ)V z-o&CQskn+^64VFPZebH9$;1y-W~R;a^TtiBt0$gkD(FZ)=bc8-g%8#!ALHvIQnH*}-wSn;K!S(DX-?l`}`u3`9F8qr{bY z0VM==0pIsU&c|g!7E69)ni8>pn>029#j6GqzTgN>JhlsJ?bE{xqPZn!@}9=55{zGM zNRVdPB*{V?-*;777r-t)0772sLl$X~-2pW*4j+g&970BtvGz;^C!>1Js2wI3iRL#H zJw)uyXwYtqJ}9y}f)WcUS4|!&0BghJ=2_T{=oQ!D2px zzy~4(9pQ8IYS_}*QZTKE)ONuFPhO{JwNUF*c7c6A> zpqjF_$jfxO#A_xTkb8BrI5envj6}B8kk>@TVqjEnLUnA=JRxr>Qf$iKj^12k+o$f#EFn`*PilI++VXkU|`~zh)skvH_yl zgyQEeza>72M{>_Zvq4L2%*vp^^PKK7 zU{711Z&$Vz$dlzxmC0(_PzlimJvG3U-n-JN=ndsZ5M^OKo#MMJwWR#{Hv-pLlD2FM zlKDEU+k$8jv7TZ~leydq@O4-UfZ#RH5t~7^H3B$W%8MgYCN{)Z5pG;E?Qi?@`0d9^ z>IQnIw?>)+-*{`fh>Y-hU&vBRbLfy{H6uKxD_H{8TFnP{44O7dC72bbWplAIDBxBm zGJN{NYd5usV~=@c+AJ3!|L zh$@Lz6oRLAyni`KT(etXplnW75CW5N9LQ~GI6ak!JUs;mn`dc7tNG%Ea(^7a$~(XE z;jXeLeijIX!OHlQ;7^nF-ux)+9w-nPh7Fw#=sCU`An}iNO#KB%1N8hMf-gtAjKqOy ze?7p|il2DK`jTqkBo*)jdtIguik-RA+AM7yG&+eX&F+oeWc;ct8@mGKiP_XB=Fuy_ z7e~1#M27VX>PFJ;kZk1Fc-;%VF^xVH`3@cpYnyTCC`4F4aUOvKI2G2do?EZ#^l;58 zSkcky&fsl<=5s}#3Gg&{5h|o&H2z}50xWGAxyc(2hKe7GmLC z^O?~{nB83E^awc=&mMYo0i4RYn+NpOlgp!x$Ky^1cc|KFEKa6KKy~ZvQmPs zn{OZ?`P9@0F{#{yUsJh^l#I!_3Ed%c307S<%!w{S-eX07Zf|DxWaHbvF*etcteck+ zsjFid(mw~cRFR@QoncJc`L0_BWf|IrjWaQf?plsX)KIJjI(26uhM>bQ;#!_6Y3>Px z9Q#2k+X>HU2+Y$Z5F->@$bSYKgtZ>`4sSNzD{+;`NmvknRb=TNwyUfre+gUVDv zSa53?$ro?J>bw=#{G~f~X)GmMR}vynpi3M;l`!|p0%&e94U=d-4+?;)&|r`2wI+G% zk=<}K1HSnDnD`=jPG^|>iGOcGmvk4%I$3opaa=-_SF8MsK#tf?i2!YCJ);5XZp4TLm|`uxn-Q(`G3D8|Pr{ zp~c$MW!9J-S9)lZW2}X4ZX>g{s%dQ-h~B7{*;`^`dLe~Cexp^%^EPwY0GK~DuACt^ z0bbg1z__nO2*%`$l;eh$w~b<$*NqC-@U(C6x{M}8HjPnLvyJtHo^*}2{E{bL15%%3 zEHH3rHM&=Ycm0h&!qMJQ?l(&=H7yXyyUNBqdsR1MBa|h8v4Lk<9VyBB!)+A~Pw#>_ z1Ndts(O1j2BV)&^N4Sk6ZeQ_}5Rp7w5vZP?V-r%UXIuw@IwiS>!;I<8Hr9aIlXnbA zVtC$TRdeMK)ONg^+~-JqIlzVO*ckw2t#f*TsQyr8Dn7=M>M1h0tPxz*_c2hRNhlVz zXAoL0_u7m~6k(eDo_Th?3R}}Nkz%)o&ac)g7k?4()=bz~9UJl#Z8qWOc6E0*2k!8O z5J600mMaQFD34OjA!b^GVFDY(3gxD62+zgA${O_5jgf7vdmLITL^aly!;xFqppeh8 zok$%lLaep8`oCi)O02qobS~^<5o9Zh%wS!Iwr60cCZo-|^A!zPB*IL~5 zz@dn8oQFJIvt&UY=WVmLej^&rJAzw|`s;$u54Y7bKLJ18@b8*`JOq20+bh00NNYLd znyMpu(oAoMumye7!m)2_h3^{W5$rwD50SWWdPnUM$#>{(8$l4OYae|_21@%_Zq#M(0|9XxAt9R=(#ZJ4m&-fyLaMUAf>O{IAO#bYc^L$f=F$E zCWQ%JJCr-v0(4T9o7UejfS1DfWvpy%4l$5!m)9D15JA68GCKer$+xWR%-k4hV3&3x z%en3*fN#>C9u4@2h-Ta91rDXPH2Tg5Rn6R!;pg_P1Sfeei$~Op6}fTh+}8h?8??)D z!^~tdm?W4}unTPpA*8dnYAo{Llv)kqJ(mJXE5wd!!wi|1KQqv~q`_Nfg=r?81jMyv zgVkv7>ETv76`cq1+tV=&m**YeU7GT)<3e-b^S681wu@(FS52IT3O7Bk@By?4j8mur)a&@b_0+|H5U6d@}QJ zFe`ld#+iO3n&NaLgv`sxG{>~E##AE`rShpUu&rIF9xNtXcch^fAeY6hG>n?Oc&3DnnAxrQ`5N9^d}d0 z^^v;Q6=V3@sXeHp3XGS}tn|hlQy|n7V8s(DWGwT(H64%+c}yG0eRX{wh=_G`$cvl zjcJ%e=(-?lzT6raji5bm$(=RyM{l2PID?4?OKzIa)%_VdGKr@z4-_9^YrnG2;<>=( zF8MPruq!KfMC*oEoVhz>>8|sg;vWsl_3xi?z!7|L-YuG#J?hB$7usED|BBZ5TqcJ@ ztMIkw^j%tiWEN-G&&VCmFYnitnJY*c($-3p^eXa@AL9~=CNQq9B$Gdm2x1nL`{t0R zd8=^x#|iD2kRA!DO3Gwm^nL|R8kH{W01zizfv99;9158@N-ZK*^!=hRD+43a->j*k zu@mv;`yw^R9r-{=?N5X!IJ3QXm@!d`WyRS8p4m=+GV+Uccf{7E`cM>GE( z?}U;^TyWN=1=5LA$EM2LQ}nLH6Op!#eF1=MtoZ?Epa%OxC--H#bC@F}_KiLQ%D^Jt z_yYslpWGdXy>x|B%0{sT`LG4>CdO2CVEJ8Yd_@zA%fbS(2TnAN!@Zi`&Oc4g@LTFK zEOHfu@nQP05cJ*fGxtC4+=1L} zsawBetsL>(TYylIP(SnT^exsu{;PfF>(b}QcaDE(pBdc@uXkGF$Up_e{o2UhcO=%R zZbr#F&jIr;qlw+CNVb0g+M_}kK#g~DXR=T2tN|ei^pD9O(1&C_^90@KghEZJe5VG>u91s)0}E5jD`~c1AP|q#YmzZM>nOqhVql;2 zm#+k-W7RlC;oZ!N35znP!JqVjbDD$%hz87(5e`KlrjfI;w%G0*;a|W+10PfKU;v0T zO<9OYj5yQ$h)9?g*~YXqecQ~AQX(cVime^oiV3?E3H!zrumwWP5?E0fE6Oy`J*N4) zHM{*Yr~v#Rv?Ak;R1QVg!oTcN;d}Ss7YrVpL6l_$FMwuAW7GT=mQzL5n7@PfDO;Nq z)$VnJLt_eVtL+!`fRu|+AX|kB#;jNAj+?5xBFf6SiN~Ps8gilyn=Y{eZRgK0p7kzH zW-;(Te{a_gP$ecpq;1q|@RjxO?r>W@6Cr4TKaBya$FBbS*uai5qiipA%&)sa(^g%EfDUmU zeu|Xdyu-pKUs6hX?u+qXT?nC9;$|JW` z1037HXdJm>fW9*xB4(*r1UE#?cdJ4Qp_}ATiXfyD+Izh3zojs=Or)sux|b>q1E3k^ z!>Bc5IK2@kflQMO>gcP3QKE~%p|ia)Lo5su^6?Fn98h;K*k!aUa|2|&f%AOkOq>2S z!4Qo`_&LJ``$RCBcK57y_s(_U)FgUteNOc#7mqS&q&b281M|m-PrWz}d@ZBosV+e3-25N}f^14U9XLctK5}*7kAiay*j@UVX1E*%Y8Bdwrpp$F;~R{b>af>ULZMcX;&|fShRe>t`e1mjfu^iC0da-To!g2EqHi{w?4Xi? z{Oizbt7%SYdL({a`Y}K4C*TtjS_76hTTH`}74>6ty!+A&h{>yBp#Xg8LmS0e8` zS?q$W`en0W;V*wruq;#bN+%+s%a^lkmf4M3pDA!I+l&R;mk=aI z{If+aVw?T!4&YZk`3z*CS{I(QpOHEP`j~cxmP4j?D)rBfNvaDCYRlQ%!gCxHVTBvi zS%leZ94B|FUBT+zsc z!Tw7wm=Ss&TTA`z+pg{ZmJ22>@%|5Tdix%QlPp_x&0${nyTDnK&Td+h!g%qz90Enn z0#xt-kqyBUN0uT( z;ZZ#+vtU|A+}pNT??qWnc1;>k9udr+)Z2yTeQgL?u2PU2`cvMS2Dlle$Y+zso=tU8 z$3b@&v*Wy{UZSFdhmy4Xa?vm2Uum3@dnUe?x-d7?wJ^#(Uxm5BDsNwuZ<=;;EZ#Z! zW?=DX&PBC9`?+A|6Sf>T|-g66li$KCYaZ#n`{PawzNR{J3RytrUm6$3G_{*{PGPQ}0l9Ioi{b zXgF5)5xeh&!qey6ojHe*!STj&%+Qa*h$jII6;h_)eX1# zJsE`>_RQQd&eQ4OKSPE=6S~36U-Hn%FS+jjD-TUI35NdvUpUhLNJIy9;JnlqG5=ke zTW(J5gXzHDF0(<#la8|d!aAeTl+GJUFxDaberA41`I9C63pp+mZd+N!G1ju&SUXQm9Zwn93wc7pS8pQmRlz&?a!pqp4Bmx3Z#FJ4}S`h@jRy_{oR06sEN?gnbEqS9kNhIFDLE<#HF7s z*hlQX{~ufL7#>;Mb?tViV|HxYwr$(C?NrCMZQD*K72CFLC!O?{`+4rYkG><&(K|ISbfR?l>y8tleVN?r;thvH%1-hFz6;`8*`4>uP;zz_Ke*$ z3r+L|!*XNvPy+TvH~Jh9`nzGhG4Kptp!yS;{9;l~E`~Rm^!oTPYYS5I<%g_2q=ss5 zr7>$8lh`%Jv^~UzxMh64A&h#UBaC|>P-0&1hcfvEuABZrb02;A)@kY!24VUKoA}-a@pr9$I zB}*tRED*yJD)!Y2M`-ZXijih%OJ$cC`J{WUibjAt{2)p7b)*i=GFg`a<8TL;YD~67 z@L77?>>fSV{@FUrskfv zuRQ?4OzIZ#1-3-o$xP41p7;PUBszIER?VZ-L*i`~85yw`jX{&q(YrC8^FXzwub99X z2V)stx1h|xmb>=y7w}|7St2yz{^~TF9iB;VNJQ%l-oV8Uu~O?-FDt({#R__W-jOPS z!@Gqia}F@o?nvk2fJ?IhHDAS+VLA!f_v-Wpqa*;MY)*fC}ssQy{$ zHPp|%wEl;`&A1vl_k7)FhT8as`U}KOH=WU2-1Fyohl1CBp=vy%d#&RoD$uQ(P!tf? zscS2U@PqfT1hku#*2)BXlTfcx+dC79%U%Vv+j70ji>%m^95dFd*&`z^JZsp}agrwo z&yn?y#01_32>iFrMb>Z%UgB)()WvEoMQR4iWHfc`gzD&uo|tP!|AKhNYOLkJvBo7U zdurB{7K;tS5_2gIEK~}z3A&O%5i>yOItQVrL7nY`UyB~l#5IUm4ZW1TD#wOdwZKti zamb|c7m8{oa1;ZvvjFpJ8rGtr*|yp$1H~lppxk`sJQEx_v&o~WraQ4(zmP2+vny`u zsKW!!TM(SBlB7D0!8%XiG*6T5NLW3(3B71Qx{_5=SAE~?Q%>|ZM3_n5TLa+UrfGyG z2O*p%o3y%m<}paG7E7!x9oBV53fJ?z#J}Ns=K22mo(Y+0u$c|yFbkD4k(*`uHWo@g zkG+_`<&R411V?3SRR~)&H~_#>ar6Li4<*ygCc{MI;H1G09!ZHia&Ov!cO^rvGFzwW z)NFEKPa>6+uPQ`|qP#z827`cvI4#B`o!AYI;EgtiujK{%q&ebq(gg!`M&F^1}SOGNpBkbN7#3Jd` zI}Tsn=G(h-LQt+(ZM_1x+$z}-!yTBzy|}el&g~2aeLNND&aA8pvIk@(@Ylxb(4@~; zp0Pr01P6yk(DvEDS&@uQi0hx2T@@_L4*WQdCRZ$@%*r-rrVi+g$f_y|(G1BN1}~+C zT3w4e!$gp^g?p-YYuNV>?dm)hRr|N`4Fp=&1ZYW6e77lm zHdHI?;6kC1GSp<}Q_WS}RLBNG<>&FDxbOsA`cf1Ro{gcLDFYyrJRmESG~$4Jcv(7{ zs+JQjXB%Obbj^1}{zB*sOvO@AHw$ zpaF7&tC7th+>XD*&q1FGh)|Lw#w+r0pwU7C60e4{vMH3vE(f#pL_j+X(DQeKd^sva z)X0hynqiX1AOS#&3Fx=$|eWHNF-qB!i+s zx*Vh@X6DjlF8$Me6!9zS;o^39?cQ}EJ(jf}5!}8^G{$OjLkD@G=sf9QgN>yS;s+(e~aK^^TjLlOXKSDTy z87z12LhLYSA{{jQ(}janbS%gjFSg17Eq;0msV^jvH21Kv&=|(%;tZhZ*iaSxWC0zi z(Qx=UP_;~F=2#Njihch~;Wp^VoE72Ap4-%g4Q3Qkh9rohx$R8J7)aE&Efca84Hjgr z5agN`0f0*jk5-{1oERjd88*ubQm)6+o(Y+!z=&=`McfOIptL}mpP}ipK)lZnr?NqX z$xkdvDJeW57#h*IH^(>;Y!x-TMq^N%_K z2O#MPY!f>boL4dzj8{14JQU1|HEd(jy<;YP(2V-Qqk81X-GW8TX*XGoqD4wJgqG#( z^C>R|%De1SSYa_!2*$#!K}c=ob1+fuuW~7PQdV!|(uI37R-TxQG!gFapV8Lbxf z>>QMj9Krgnn({uzNECFN1qUW-0c>QMll;%n_`LEJB9w{|OO^0>!WVf|%=yA3f481X zF*AeL9ax^t5Cvg`l#77=>1GiqSJQ_&CnUZkB|sV>6=SHENO751v&OGqkiGqY*AWYy zgmzJBs_J(`AyQq0B7tVZ-bkbt$t(f)WhRt}Llx%crhDqO`rXpM;Qws$`mO*Z3Rv_Y1}q_*gnS zQNlJf$qDbN3KBb}7oqi`A)O+jPl1{U!qg$6)Se3?E2hIJT!0_#xpaIO25t0YsQ5O< zMbLcG;ioG{qO(DP4sk3)qFV95*mRY$FQM)|E266MaD@h`%ggced}3KAAlu3?LNX{E zr)4N}GQAOvxiFH6ES7e607fpyRV?*$fewvIx#Yp+Z|*`dSE)i{#p>M#C2xwRksWD5 z?>wcya$zCmIzagNM192P;y3X6`a`R>!!kWLoQ>$LC?;WCZ-Se3DLS*WhXma_@r9~6 zcT7L6&uN<5mO2{tY}O*KPbvvvZVWLH!8cjtc(Cr`+$gutD#}i1j3Dn#3O4NEms%tJ z&%npg?$eEp0vA}THHlq|tApySQr@^I@?qP27rSth5p60=I4#vTYuKY-g0C!S7&d)8 z-*&$*@Yn$phT$}x+Q6yt+zD^m5F&}IymjywLuYQ<;9v%12xq1vWLz1!n&>*~sOzu>oBY|8Z!w_v* z4!z@>?n;LM;fzwp?v<<%&3aLAjezBQlRQ8m_cNj8I7V&UDgVRi>MP3qZmUrTxBI?! zSr5YDoBcQkeSgAjUDWi*OGQ-e=u1Zwx=9y>vE&fK*F?(@KFJ>qAV8kvj|y;_qAH3 z@u#QuP`b!L>M8|$g+@c+38bg;KtrH>0p}~<;iRGRz{^!|K)i?U374mQLF*~nAzgv) zi4F)zHKcfnA=pOE-3~3>fdKYWQ&3U#sn!wksmH6|aPigbzybUZmtUF!ws!=zNYfvMbytqL;qu~RySEi{hKNG+xk4a!&_>0D&f|6G8p?BEabjm? z^Rjy{?cI0C5&+Dm;evS8TpWHGntODSCXo5TjoI#kU#(=Bk-iAi(U-jLijkmA-v+M~ z9Tgq79cr$h2={U?1Z)d6p!Vk9hogS5C+4yBRF>>6lPH~>cT89ZALJmufJ`a$I7Q&uCMmILwnmiTFv@^hJ zk#nub$gHvDoggo;3aF(Pp{<>hrR6>3(s6}t)_2A&)Muq>=C3@oy|Qh53pkr_Tx^O` z$w4oGwN+GZ!~uVm;HpJ;J*EqJB2JD5OpG`naWZnnb4=A~>oC+?aCb3G+7k0mSC@Td zygdi$9RM6#i6S<>ajr5-$3 z@Gx?v;N(W|J6N%#^YG!4uZ*6)L)s~fn~2ehPb>LQy}{}Z;Ns9;Fd$E4FN4xR3?4VqY{S!q2ZRszp zLUENfiqtb%t#f3e)~#f4g^a2A>LYsh!X?1cD|jws&=^_FZmg4g6z=ysGik*fjKfE_ zQJbrB_Z>CT`b%9GL~%!Ghmw7w5~tcmOEXUKhIxm}q+29&`_o9zckePFWw zYY1OKgIknUPv;VEN70c+L9~1OU#t42!0NDK4J=^5HB#I0j*2#A+Z&>EVIw_agv3^-x3E3l=vwWr^mF-0=+ zh&rKj0+;EFH3W5tB5P3oGB(HtZ42icwWO*UE}>tvsVLh#Ye-@tTP{$+Gyvl2;kER` z2LkUtE8E(GMnolgNZC38W&;4!cfghrx$q}-n<3`A&?nLs{~TG$Zv2zNI#vuL))G@b z+cwrv%gjo%tw={eH(_~@wz=2=5`5RMnmGQls+}p)BbZlVrqiJmN8>P=ZIpx>LWyT= znnr01=s7Nwg!M6f(hL25NBpUvHR>8Q;mhm23f(r~J@7Ny-e^p_@`%_p9v~+_@Ucxm zvpez^A$PA-=?_KEyVS|=u-%azXg&t!`Va&r8Tjg5;V82mzO^VOM^;1g9U`tU$0Wnk z?Y}r(2bK%Yo{=M1TV|qv;yFD+c?0_*Nwp^fD8l zrXlJ&V$uyl0cN5=)2_t68h}JI+`Mp7iXjVg(Cgy@@a1PX5az=_U6i zhy3CdeEYwrv!?!fLZARdE8Bk2)_(e>r08)XK%>^}QETmb(b5bcSTT?Zv1I&|rBU2( zwY8^|u5mx^W_eTogGh#CYyD>j;YJ+O-1@c{Y(F(U+x$4)(~SB3@;$Zo8(Up`XrlPy zFWWq$JTn7Z=tPYfvsn>vQDn`D43wNjvn>&LFxw*<9idU6D~%?AVm}}lo5S7^?3kv? zsxihC24@`h-p%w`!3Fd=XnXIGM9I%OlF(Q^hnZ#tu)%AKGiKJH>N%*;ILx_k+A{y) zi>X-8Ohz(2RdhzJ+vQdH(s;i4x8D!kF#6!w?8Kb_q3>)hQ-3rFth*j`BmpX+c14_s$X!k{$ghM6? ztQ-*|j{FZQ?o&eb)|C7v3U?cC;?1e-6JikMGALpZ%nJdA=g<0lNImD4xXie_$+5|n z23kLUzs*a@&pCl0Q=Avyym z1)%uZEa@qw?*`C(tc@+s`w-$Eg+&n9k_hp|RVex2#q)m>egC;<2vcc>z@PyC?UiH( zwYkb>mQc1T(UF1;TQc5+M8xHRC{luymw&r$30F6(oa|V!hT#kTDlDsrZsfdD3{)jM zLDBPgxE)Por8l0xe(vw1e`BxX{IT7q3bBmif&+fLIwPtgO&p&kcdhgoF-{XRRCrjqqGAet^E`=Tc z#BV1L^krnx&PIoA{Qz+O4P42QMz*sng7#)&Ce`{)+|1&RX|*m%BC#~(%c;P$imF)v z9k17l6B3-Wu;<8!73LC)fP%_a(#hENK_VZzm*4bC6-yKl6oW?rU_@G*p}D0xrXprR zGvl5W>mpmoH>iEUn(x3o`ilpg!Gr22w)2qKV@kK6;&y8^4&60N7$W4~=}#p}8qB;A zIIJ%UaB-@3ILo-yhW$h-&EksxGG{;M@b7Uc5fu*;c@iV(I6O+v)KD<3Wno={^v*Q> zPVa*DrRESH@w{8Iqi6}kcHz&%!!f$j)%5kZ$(E}lrdgXzU$|EIjg@^aix$Pm9deIW zy~C?G>@b|h2^@va0o#n86!!S7%}N{Y8!*ggoa`%_>03BatTc5d|IgKLhue#t{?gZn z|BI~g4*-lIHLD*C;oq*wq87|o*QEJ#Iqo1U9S1}NG72)J)R1iSr{38wik~#`Bm}#L z9u(y1i6#b2%u~c9J!@Ll+WKXST2%;kw#C(JFlzTdE4#G{3%aS!x?5VS)vLQ#mX=!7 z)+^OJmN?Cs(oI01maqAGe5!n()~qkPm)5NRTIVw3ya5j4hrdk}*H(DQ4$7VIgL#M#;^IuLmL>Tuw; z!n~pr{yhU?zL29&4fwvMh077f9vh?eP!(y9+zaWAi9_{JANc!t++_Q_)bZmxe<@4+ zLxA!39=ZdQL85>AOQjg9O5QBbAHBlhBCDn_9jt7*cC2jwR#qsu{3r#r+W}PrK1PfUz1_1V~THy#RTgo`=Os#gqOlxQ3dNLIpyQN@f7n|r1Ob*$zBW9w#V!$Isr&X=x`gt% zi2^=)KAlP<=tbVi9KOvQFQk7Y3+cxn^!vnx_?Awv{*;O>?;&96itn>@=)raf3IQ52 zzeaZzM`W26Ctu>?u3f|)MSx+!AA_Kar%|FrP@Xu~E@b;+(OYIg)=@!tGsqov_p)v* zb6BZc7Us$%nD8jWQEZL+d$3^p;RbD*9DN=PNHOT_gGAMX)lBl5*1yB$T+Qwx`r}3F zw@M&{y@W@8hAWafS3n|5AE{g$^Z@8FZdHQ|M@4jr%ucE!SAh!>5(w@87o@G>BG;*vXbw5zC?LGLGATJ-Z)r0Z+3 zM7xyevWl?|+v8-DP2WgNHl!Z7^9uCL%GTBpAII*^Wuu@1F`BDNmW&4zih3yo2q7_> z`YKf|CJsaJ+76!81rrIR2>^~(7eGs9glvd!k`D7iK@DFzlyt>>iBoehw9))lf@#SD zmW7jbaES2bVhmIBrQgMFH?G3Bl%jHL$i|D92TYT^=S?~7MdR)}@fHS*80UenIDaSM zJp{JV^{s->YC)nDxLKY(Fgp;Vx(irsJ+$lt;W_Qs6%Qd#SwPDq`S3_~mhFC~ZBm%NvF@ z{1g++i9dz^so2wv>4avUk|_ORVN^#HXF~e&Fe*Ld+>0Y zahY}di7ef!Pt>_Nd~Ab);TSrWDYla z0Ld44K0&7~{MAj>zM`zG)NN_8Tg-TzxJAUd8%l2_!GI@A0W`!?6MYZd(b8;m^ZgKA zRCT3c?XmK#BSeWR5>lhfK^juIe^<6`XzmNEPGwj3bDvP-8iJqFXSMaOoZ1+64lI!O z*)5^(C|?C;cyVAXlM1c-XjduR3!imo;k$CLtdGrYBs9=Q6Y}CAuz^Ldv~V73 zM?DR&O%)kV08(GM^rvc#|736{>9OqwRGVtQAFYp7a7DD6* zKNmD4(MGwF6@}?b&SNnERmzY{oLH8wj>^G~Bt2L(HDer2TZPAfl`x6({wo^|ls3*L zy5ufb+#~&^#2_Ge7;Bet;B#M4&c{pZ*(85ytqV|X~9GWMQqv| z!Xf{A>22-{l*2l}oJ(TtYRHRaA5Ow{mq{ZY*gh#n)U2cB*5rVkWehhOi)&G;3uEC>*zlv79;rUp)8mAeGeLxtPdayR z5DSI2GN6bk{!DXBjlP?CPJ0`X)#_1A+o=!_M$6m|_L;3&L`Y0OEE$bgZ+ulcZ7o4U zG1a)elDc|INV=XB5MB-$vNuOE^%@l7T2O{MXHukFW`M#>-c4!6J*3R6X zfpeN789`k=k8-3eWy~s`rv7iBjbptJCNsz&8~}GPFmYQxEtOMJ1?akw3=?ph;Tavj zM3GC+3~x2_86uiC8m4IHt%g^pl+WXOG+tQFAHk2eM?{J@WcqZ}G{`Ycv~Dm8ar%GEs|`N|B$`_APS zVpx;&Y4~hzs=@nEyfl(uq2SMaX%YJqHmtP1Nb2eO?JJq_sA?g1 zyFG9j>2XW{h`(; zzCzv-@_R$R^Di?Pj7}1nkPsQdygBXlNgrro%d-t2_lYRwS=w|o@-@uIHD{5H2bhr} zGRCJ$OA{pk&w(aM=nPCSS1bPf0QCBX4`oFD+ZMJo^wun0FPbRJvRq=W2i%1Hq+i>Cu6LeRvhZvN$Jx7`A--;L;S5RBD5 z4ld|?`(}j|&@kz3i*i+GBChOouE%cqej{NdD z*s@~eEtkUZi|@`}@yAoEfNM%)XY4U4T1~dlFT_qe%=7$bk3rH`eea0Crq9FXkU%GF zv<kP{5E2A)~Oa< z-=7T&FwI1PS@WqOG_HxHu9GFcZ5>NV>Ag&**&OQ`V+m`Pl3*HEousU@2^|H$7o?=E zD5YPSi5X0aL|>^)1+3Jnr#WN9I=_Sd<>$XqApxr6s|M4_Ld=n{d} zGmJ%G7U>4&@Ch5=&`j>tbMmMSb+z-T^<9GSh{N3SR3Fu|@~91WMf0?TuTM^_Pj!9c zXJ@VrUeET3mVZ-aS>ij@#`^~EUK##`W`G$_v6=ngRgk)A;{3-In01Y zT^WQrr^lx9leD1Au}gmSHnR8!_ydDY5y0S>XG(I0RpC@COLn<1!Tl6u4KCYGB*4xH z-);!bh1#yLc2}v-{&dXA_XiUJ2JZ=|*p#-xPY2 zvg#*d@JG(}^m+HN7k}r{BhDTf0bt0oZA7j_-n9@6wh&Fa5V-gNYIQ1^CC?usNN)sz zD;A_D8l@+ewa#`X%FP?S$>RnSduh0Z%i~7g>V2x|lL+rf)iq-2fFaLRn>w6x^z@T= z+I5%QBXT(AW~Fn&J9hYU|El8w5s}^aHM+%&=f)sBMkIyH|`)2Y0p@zhcM!S2TulA zY|YV;%uCTR2J!7A-V#!$DDM&sqM$?=+2rksSG_P&vpKjnqwu!agINyJU0~%M?-=io z79Q&b4CyqqMKZT@ zqPbg=Pn{3rnmSIO?U%%RQf(XTP?$NF2s+CQy9~o-?WTom4by<)G-{ry+{0ZQ{QmJz z^=B*P{aY2{w{Pdv|FgULRYU%NU#YTdV2XeyfsW466zvvTTd!GzIRUH)%3~Av1iQ}k z)vJl4Jc>|AClCK$hd)tJP-g>7=N0mC)`;}N3LwiUh&-tED4=`Tz1A$>U=8XuD2zj+GEFy#_SG=Alj`M%{4nK5UQ(jSO?|Y7+)e%{5B&z1 zFbbnJn(QVIfE(cn89Gc1G7 z#0?_?8qjFvs&BR}YgZVL2AE9M6G_jsF@5u`Uk{ZZNKBQ}r%dG+<&EK1@dAMADcaM# zP`Xtn^NU%{8Q!PT+u54FF-7G!Pte~vr_viRrt%B7FY|!XSH890DBm;P$lqgclOCkr z@J4N|e4)ity5;tf8~ifJM)2POa~)bE`1dQRK7swqZO`mJf`i+RwGjXV4b)y+*j%;X zyiE~IN|G^4XGQSLR}KpR-a##Im=dv%#DS%{D;2wm$sTe)lJ`Ne?3!b!28XMX`2ggkQETwR%KQt1VY3E1UF|TVIS$sz%Hg;E2iiqe))8M7i#ZZ*NX4L~VuD7k%0y z+ZY~TD`dBNvP0%C01);kUh8ZK(qDf2?m72{Vx|VLZmb832rVvLG!CYT~OgELeAX zSV2C*3YAA7720WbH&_qXx{Tw@X)3JOxoNJJOHG*=np(4(TpEtpG2KmonaE`;ZG zW8|)TlOu6ti#|;N*+)?~D6NvM@^eDsVHeDC-+qmp;%Ln7EL!^cO&HOSLxH#a&H!0x_~6I)&2hT?jGkwR+wk_i?E~msz4B zhQiV4L-`sN9hx}}24raG@LEsnmIR@hIvEU)!??2uE$`1PY==A(+!tN1gLcPXW_3_2 zB28@K{1~U97ctFriG%XF|<3!W%JP%9_G=os2rtxnDpeK)-d5;4I_ zQhdf(WgjlNnfb?#%#mSA3hfP<~Z*}}K5*w1rRw-$vdVI{($$s9N1K^&v-ZwB57<=D+k zXe8JQkh7_vhRPz>vk8oyl$UoU4DVf%=B4bUPG{x8tw^G>ZYy&=W(-x_i{UN!sBIvP zfO}_L$^Mw9Q!rTN21W~LJC?kBSbeshDR9TQbj zvnMv?pAUnhqshM7#IvB2Ua-Cd*O$S?QX0~K70A4nPU|_Wokb`c)Ee29%P959-QXaz zJUp&D>fNanT~7B;9kv)jq_?NY|Mt1l0?PC!H$~1+GJU2%$&{W>VC?_UlX|7$p0$ip zB{}vqD>crw!q-CMc%OtSIzpjo@XRiw@bbmOtgvl1l<_I`x!~wx zcpf@7#xk+xKc|1Zok)vFqHu^!EhCMNp{i7t31|h2D%2;K{4L9F3Rh{FvQ0^V29}no627QQ0Iz39#P0*N z2HB&OefPe)iEStsMm$5WQq3FeJGnksi3AY1X~3p?xa2ugU0KAIQ!@lx}yJi zCsf^li%or}#Ay}ALB3(X8j6VPf@tipq6$I08>6I=HC(mJI^F*SR~w^_1?+4vxdd+z zbLhwQ#tX>0LX#(N`mbA&vBvE9f>368FjrWU9gA_-EoyR08?a+_el}((pFdIc15P&5TSrWCTY>l%4;>-19rK3hnon43R+P7N<%HcS4NlT? zPd~X~(pSuwJJMV+$V)MKgQG0R@0ub9M=TljJTb_>u#jDGlqqMy;g&}J9IKYX1?!gm zz?~({-w=)2u6CZ;+IppI$Qr5nF?EqZb7ZhjPLA)#N?2LqyRLjw03aBoa6g($Ap#<- z!0_Wr)H>pzE=wj*jYo!Bo;b$59*&;af7x&!jzp9dd{9P*0?nY)0w_up6d4>MEsBwv z=%yZK+m_JOCbbb+NeT0{T&Gug1PE=&UN)x-*cD-{M;~_9k7@5}7zf7ohhFH%3yG^s ztRVp6k5yW5vEu^)ZH(w67aXJbHwldE9YS}0G*o+uQum&RK3ne;nf~V^TFd>B*kvyp zko{ zhUz_-+=Bl0Z3O54qJ~o&qkoX3&QyazrINfLptX)3f{p$g@3PzT{ofh0uoncx*5zX` zng5Dsa>xK*sTf|U?d>_jg0yKoqAF(oUz-cW6eETBYIrYLhSC{W$smC ztez=Cl`VYN_S57mSGRlrqc3M?R_F7VAtCz#pw)2%PVx0ojL(S+y@r2G_o?D{KYP1* zo0&Mr)vxCR6~)ot1`L4D@>Yhf>%*7d-lxY6)H2fRPHVWEi>5)UA%hOv`Kk~1_yCi& zk`;py8#bQZvyd-H=(VXhx=Ekz9WUcROBR_qMrAr%t@}FM72h&8nk893R-Hq-oYW<^ z?()uhi9KBnHuyCpcyig0;wIu_jVU8V2yEg{smREVtrFc3H*dg(a}}OFIP*YUNiDe^ z#!#TG6yfUJSkk2YiTNau0U1qj((=*`l#?Lx9IzbnFbx`pTe;43`2`+(l?u_~gIGOR zmw>6ocRH33GYo(lF}u6CPa?F=CxWp z!ig5t{%Au@(0uh(|)&bLj@h1zM zu}wfA9SBVUFISFCyg;7wyG!Uh;3Yj`E>a}E?4ki~;^Y0Zp@iUpcJuwR5}1Pj&qpXV z{vJ#ckTUL$Y>F|IYdI&E0c#n)5dob~W$SF12Wc@jI2Qn#i9(q-2qQx_?_$A(Y)+Q! z#a(yf<+A%Ya*JTlKjfUG#VPK%4-DpZzxu;Ja4cP7E2YedV;b+i^=dbL?fd(Aoc;I5 z$G-eGw?Z6X`Oh4LT^UGoVGqWjNJ5>2+k~i8fN2-ezA7fV35qEOQ;A8EiHv&zcA#O< z47!+kKS}T^z8ENXzecDEhHgP2%sUXLi#mWo#V{vIKu{=+7g401KH%{L-Ao{@iJ>!i zwQ@%j7V17K04D~Txqu*67<$N2qt6mYk-=XP>^M&G1_YieZJ;jT-b2ej5xX@shyl@L76@Cu6vdg)_nWw7voHufwWFEI#V#b+l#C0oL zG2|vyt+aY;%Fk$YuCU_{Im5~sF0aYR&d$8?g-3Wm$f{hwH=4Y#0-4P|v!CsW_|n!^ zV?QB%X+vsb1ZleJf4jbKeWoy~To_dYCY}p2D!ogiHRoAvskL#C{xCoDo^KDK!>PMZ zP&J-SW9wR#Ri6H>T+?EliBK!=6YF(flSvP=cm&sm5H7_&LG z3&3aAb<_hQyG_nU@0Xlh&$w)sdC@pjYoOFFRv6BnE^J^Hra8;JU3SbQ=P4-Uj;`zV zsL@HsJ$f;uYQLC=I%H9p~a$nUVx&I03 zOn*sm+i=-Wxg3miPEligl*xhylyI+kB>rUjP^Qgye@!wOH!Unm4$W4oHqzLtA14x^ zo7i7}6Oan4{w&=2E9HDdgB1QpGOR|1g1C~UC4i@A*nwRGT)6FA#UZ!K3%aGbB7?D$ zMGxZep@(&>s5~!3zxLQ~n-|)uZO1F7#G86Nq}U_Ylh!Kw(NBMI9LXUQAf31V$TGc< zI{lORXf-;!7f+8>w`-B4DIEKkUKrms;u_61IAR{e08)L6=L3lyAroipt?!cL z8pUv!cl{42Jz?*c@%=XbzclY_FEZrDxcmmvd)xycqhK~L1dhlEenBo*=p!{A+K37>v)BY1MmjiaD<@6PwxHhNTpC9?>q_Ovxe33>waA#IhBL)5ba{Q z&3LK+ZL4x8mIs%LDO0Mbb${O|LYG-*oZ44@qqpc`ZCkanalANyq$P}ocBHwh>r zJ{mBQy*gH_Y<3h$6%Vhk%;9x8eoX26W4oW~8aSZO_Fp_ZEt1%816W|qoFlxts(icc zB^LWiw`QaV1Da0LHmb6ro&gG=$&E> zt}8TA>MOw7J(3DV2WS!X4u~u|KVkCk{{H>VTY1V$Gq)Efev4o(lP0WH!_|JDVT*7I zbx(&>ZntlTryhK&bHFo};~M$+$DWTwR&3Zkqa$Cr`SFPshyUMSjmB_U^5OTGj(p@( z=mW0_9QugI)_XepBaAVY{o(($WP18)GaY>mFU))i3k5joAD}4TAiv&m3TVRr{3X)2 zV!eXZ|GSn?zkx{uB$fY(!sUWWXNZbWRM}A2011j}qizd;!P*Nb$b-m~#MVwsUttzT z{Y38eJl-=!scztTFQn0zC$j1VlXmvd2H1~#Z{DBYcOra)-xCC1W^OagvoHV(jP}z0 zNCdaTZZkF*>(xb;yjUP9mCOxCX|Zk)PV_?te@YuMXblIrUCb4!C;F1fM4$J5!Z+oM zC;q^dFs=vdWgGr5Rb-)az)tQ}ZBU5Lz*d>$GY76}iGtXgW z2ri6vMLZoh@fgi)Gx3O;d&6m`-mYCw{)2RLj6wLRk`vZskxq)AVBe6*tHS7>%N-9! zdunn$rZfnkdp~BVy5iRxI8%y|A?uK;9)uNhFT?0^IP_cI#rNrJW_bebUFo}S`{7AX zI+{0f_4`PziN0CKPd+ac^k$>9^4t}3m-F9eIw_ao8XcrZMfAMxT{zyny3NbAH>SJX zQcFMNSwDtSU>(_xbIsm0rL{i-yT?L< zb}b^syRp=Y&exrzPuGvlKLi|0O|ehTK8Ng!aSPz?51qT2RU)DPbTWC!G~Nqm6XU6n z-C+Fy{^VUCd!UCCLuit932Ne%hy%s=(0l?1Wa=l5IE0i*LiEZ7JR9)V?0U@Sr(V1b z;Dp6Zk`YyDYTQW<=X*f`;K?&zss|DWkjvFJ{D7M}Kwc6R?A!R(6%51q2Nz33?ehLR zq7LMju!-^o@1l~%aCVL5?1(x-A@iWejtIyl>84_|gl#61(|NFm`Bv*+R3nAI`r0ZC zktc1ADKf<_7C*Daf|^OkxH0l0HT)3QF(mvJk(zIFVG;R7X6bfRnAdvDnpi)6yh3X0 z<X3$AX$Io^Nkh6a zXt?dP=Q}&}A{-Q;JPX_pJ7#-9WKhpcyuuB6|hy*qZt zwr$(CZL^bftcq>hM#n~XY}2#c%^E_wV@!t1**k7t@?6FZ*|5bCX`J3G@i09M3 z-Wv#xYfC*pRJaUgA%NZu;=7RN*K{?)jy}87P^wk>MV}!Yu?WqdKMSM^D>(Z8M5T)@kcLVMlW{MXP0$n558g7rK3YOp9(lV zPnBFqwdVZ$BF$JbIcW6_+ai`D()j%wc4R-8>uwu-`>4)vJn;N+#%xC6nF!xX!AO<@ zIIHYYh~9uV$uu_!WE!d2Wf_ntey5+GVF|XHgg}*UBq(f-G{4o$-&Ah^i%XONMCh8K z4#BkHF{LG>!3Zp1nmzmhQB^|X?u@D5CeQdJySidk-kI+K*8~docswjGvEC|1 z7bFBvkQD?|GD~N3iE5^KPCU`Sy~RYpKKn>68GWDvZ^m9q*RZrFq*2Qr(XGL4d~!Qu z-~MyvFW}x{ogs}n&ATWC3I`=tBI+s$m0&?{AqPxPM{5N9mkbo+(b#4U$ui+)v83=v*jk5P-@N2h9>Myd_YH5*w}BT{_cOkFrGTqG{$tDq0^ z#dv6@jZ-T&w3fl&WGlDS#tgN)cVmO#Kay2NrIiZGEoN5xVx< zM!f4{)XT(i2z)uGr#6rv1HfHneHFr8#Q0GGc@bZ5i|BdH4fA1IIO$d7^|&6wfmh}` z=XMe$u3KCq>G@SV(z6n;&(vjYiWOyQiL~^p!X&*Uot$BY0+Enf52+>mUP_Y)y@)ln zd*xBqWI@tyxErIx$t@Ay5RgP;(^|NJv2Ds~pW=%3ThssB6e{GYWr zt?msGHBAo+5*j%Fj;fCKK`-C2UWXP=AAfriD74;0-j#qcqXH|orm_y&BHwG@fB-k@ zu(F9p10Crf`AxZ=4=+I(FHBj;T3HRxG8uUo(eFCwS=e#`G4>;9zdcKy+TLTJH{&4t zh3nXTn)`2d=U;=D<9Ws}6N6cxOK?o!>7aBV(jfexZm@ud+^;0iSZELRUoaq)o_vG* z)&$enToe1+1fPjWxx==62FFgx!?=9Uw@fg9nH!Q5Xe27JwR=Z&BmxNzOzWXcv9ZH& zByjk!raSiQqZ1CJ*}NPP&#<(Qg+U064ulb5mB*#Jp-}7Qf+FPXQ8btvG85Pd9_!55 zy4nnb1CanyW8}^7sFktsF{!`=13Qyq`GKPFh?STy@!!Io;oT=_^@#W+1d>VNl-Kd$ zZ!1K5Ly;hJ^g1I(q~c=C7_X;Q}e=Nd)Qq7hP+6o05f zE6y^_Ms(B?_8QQ)DzQ+)0_Pf@?VaQB!e_z= zWd8~+4C(X$BgJn~^AtDjq=TsSW@?Laea&d|unX(a*AbH5(&S}Lc17~*+-RwgtUC4i zE+bYK8mm^yZe-~6)oQ$Z(y}oS@Z^=hVcClH8te%Rn|o}tP+BS_`)cLf+}HQWu?JYc zfQJDfyb@cWZMR2zqQ4yy>1CPOKI!<%0^@ZU7NDyhNjGl10~rt8KO_^G&m=(++^V!s zm9^i&Nn27=nwRj~+<2kC^1BU%Zi`@NNR9MSg<;f4(_Rb|elN4rx?RBfP>@_#E0G8Xw%MIIfL4ZkC@P#JoXNbNACJ{l?_R^%5inR>*5T*G3kOJF57D z)~mW>lF1I-*rfOYXcO|4%_;qlJFmxoITG(!5?nM$UFP=CbY5UI7s|AUHrr>@osK&Xk6i2iAo+SDoo@n*n%*~N2iz`Pk zVeac2<3EN6B{?eF8)%}-Eu*CeG+-F@Ox z$JUAc9Jn*$_e9RdD&B(F&f%9YP<5*TmHFh(;c>KAlIHyoXzKSte(#Mz&;*HJHd{Bj zL^3;xcryW13dL8hdAlzE(UjsbQg$vxG(Fi= zl9~>!;0n;@ig5LaBJqaB_rNvI4h?X(!rUq`%IP)klEk@Eg+AflY*{^(Kgq+6Fd**yS$uq` zd0+a9GiYWF^TR}-N1Xjf0mgO#^0q4b>lZk|M8%hx-;ej$PXi6_UqZ?rY2033n6GFW zt?kAr5LXWr*V*=ArKLs7BoVVL&g4o71EQ!Dj8oxdqU6?c>2I9H9Kaa_I8RpX4v^m> z+;%N*e`B6~M9RGChw{9}YP4BKGS4R?1^BTC^@X5DWZ6!FzqZU~2l*;S7-eM~fp=`Bbd^gCQZ(?7l)0 z0-sBrAIGLylFNLmh%y`m4X7t|3`ssmlge7Y)U$~WZlug6S?l^B8!l&-Tl(w`=E-9U zP{5iPIdKj4svJALS-hI)FwA(Io$Z}_mz~z#{_*U549h?ldc z!jdHbsZP#|jLeed*I-KF%RTY9^sc$E-#vJq!wY47TmA9@=IO|;U5C$V%N9KrJEG$~ z&C^UDVKL7n-pe6>loF7@tFh^Pj21nemi5C_ZG+R~QxAhU3)Dx4y&rtFVgAYVGuJSA zf5s~an?cL9eq)Uu&T^A#8H>jm@FBTj?>tl+@RiM#l(`!;XOte!FL7i5lV~0jEt*Ag z#o3jS$BGRMF4M%%YIo1N*A#CzXE9f2!&7F~>)jDsoLa7eXTfhFLbU%F&gb$%!-zFy z{WdV~G9b5m)cHVUpGWiQqV2DkSF3q$nOu729ODQ7rqF8*z~050@+B1kN=)}hb}Md^ z7Mz&R8XEQZ35Nt4I6utCK~;o#5jL^{M?W?Ny=@=uIlTI3@l1ctTL-6VW^8TL|9!h5 z&#c+h*dK}3>zwE4_3ooymv_+GJYL|ksmMGdtF;E707obG+d+9ItfB(2 zXp1$j%^_^|F94iroBXQ{k}=m8CfPo2&>jI0#~*}sQbM_3ilA1`0FCr>wen~5v9I%h z$;e%p?}uzQV!=RCg52=Ev_uBH#|Q^3-T;b4-9%CtWdcIBP$_sk`6myBNISwTPO4>e zD)kB4b%dh)@EA4K6S(Rn$^9Nxpy*|&Q;CZs`!T5Ae3LrOG7tc|e!%jCbAgn4)qCk? zeEeyEm!pbT8J#!wm=T>f{umN{&f+lgSQ2eG`q&)($=#3``eyD!W~woC zW#*#-C74?wN?Qmo*`?&P5(SBOYl1II3fVp5n7r%Bn3?arYq|;<&4Op?5xYy)|C#K> ziUq$9?*zFk&ia)J&kV~9%LL0p%40Ur6ds_&>KS0#9h4B5jYJN$#2)JMKr*Zzrtk?M z9&(f!C zgjiEVNKOk6UJ^PmSP$qM2p;*9&Djw^JLD>?EoWYd(I1LUXcNl!XGuj zY)p0%nt3E&brd$ybYi}(?>=hds}6YYvc@g(H4a6h*845manta0(JPtzxx7Y)A1{mP zB0H(rV5v+!EX9B1G4O)2i*y4C9hcue7gk1R#{`&wd(xr``6d=(eexvQuJ+cRTD zw&cP>S)78-RjNELe^r1S$-vb%!<(sy$@(Ppp`7*GoAAg^Gu&vLMwQ8rPXJeGJEv84 z8wtJdrbTL{QZfV6Yaye;2~&|9U)|&vqRdc4ME2vYBNF-!+D7m>hvvy{HgOrVYGA`+ z=voJ^0~|U`UFv=|MrB&Dsg4XnG>Qk3O0@E{Np?x_WE8NKQrN>zj97%Eto@qAUVFoDnFXZGZu@xGJUpn0c+$t>(D9YE*8=#2)QgqqY+ zX>hcEN!|Jr0xwOvnJ0@u5t35tT)w$UVk+yRP-qUxGhLYSATtbtEtUkk94-fXy$Z!J8S*f92GNxLbbhKt{T$uLTkyz&A;S@}k^g)yA24%3EBZ(@5p|Ng`K2|I3 zYSpx58yz`MT2Cm;u^591Q&5VJ1@|a}P6h(RIh|^?eF<~8d#(tF@c~4~iJ&V#0_Inb zC&EOxtVh8l4aBo+(F5XLbfJbF3OK1)@$zSqrBBte-j%=}4)Ev*|G{m?U{WSf*o=TOq@ZDc<>b!Msb3a#0^j+AR38QM4?*ibKi~GLof~GoC}O8i zED5M4Q&+f!FM(n(|9Ds=bcnfS++6kJVTtX^URqYlCfte}`uS__oFww5X&Tz;^YO>M zcjQWctH;R?BOr|6hwiU;h~K&q9KRnrUtLUJ)Ppf*%kS(3rup35(PsC*qde+&C3#4? z@gIC6h}(EdcP8Cc?@MAU`r>Ux1cUmpOZ{6E?S)yaQ<)C||BcBJ$=TTQV36oEUG{TEA|@w8OJdhY-2DP|GG}>763~qlTzQ)4ib~X|6-GSO#ha_@uI@m zarJt@Eidb_$)|c%TMvtXxL|X1o9S8=PqJV{$V8S|WIC0w9#eCSNHmCjkf7GsnIqv) z5xqa1is57kHlevTA$@%mGai3q86uA95?&g z6EkwnV%?+rO#ig>Z_%v+*GSF!DLFU7|G#VO=MuZC@<~(p1%iZ}mi2*%1lY%K%YX`_ z4gDNloV2H9zaVr$)h`#;T^2`@&@{meZDp?7=V-WG;v&7*fRLUMN1PEAw_Fxd#^z!@ z=G}gsns|D8dxmt36~#rz_4cgpRp{#g3(bbZjEzEpWpGnAqSCaT1R?y+c%Zz~+?=TF zI2vEE!~>sl=1f2PU6Up88OZFl8Q||#vKvq~gp_c?uWCGi?RQg^C^Qf4Q9|56997p@ z9I+mja2?E%!*fu)Z-;RW3ky}DM0$3sWLimXdW=8lga3YoD3%tfk*sl656kyH(W%&% zw68gBh3$3sjkkSvVTF^YAnxe2h#DW0jYa}YuSgtBMzO?YI#Hp&$gWL#75O59QTIda zA?NG8X=J;aj~FuV2&1-Oyr*s*Y{RAL{G>5XMXHQ{9vJUv*D-`tyP3pWpuJ1MxFRLSg(Xh%i$1 zjSfaHr4*hc)5f4U7Yx*%K<5ySHY-a8vcS0X-VT^g@`>wQj#@`0ZHgC=%lJif0Za{S zL4*m-hN@sB{_3FxzNCI`OkOHCGsnzIK4z^`TGB1(LQH`{`*yps39T$ z%WlX~xBhG^BKanL!R-@~LQ_(X@Q;VvqlX?lkKRfm zk4+hUnu%MebGwCZrZS%SwL~EP@d8@$fQDQ zKxsze>|}`9%5#VY+gErWk$GCGE_cf^eGy*#;Kz8nl9?-)At6l5gV8Eq186>a-C zVyr_gewtZHWq9@?@gZ;lbeK%wbD6{}3f8(>=RUfaEY292#C|aNC$$Ew__UM;)+iI> zcb;p|jAZHt(ltgmIDeJAAXKGmgt-mo8NRdwaA|K+PK7CUB-h0@b-%<#wzF7F;?>> zXVa3+PaD+eQkty-7g&X^QF89LENKB#KdbNf=URV{JkxPzHh{N^+epq+_gZF!&CPT> zMR$#yZlF9d2?~TVU5dKNDV^Vs7)azwa`3uOw3&iox%N|<<;~koO@jB;!v@xaQLV!A z{LF7YYZ=@#M0ykhxZzFBcq<@+m~f%_h3Nl4s+<%U;1u87Alt+`cd^9|3y>mhOD;1a z<#T%b57~{V3Wu9$ibO}ttqil@J~%Z8J`kc=#ZdO_rDA5O&&BJ02DUwM^Zf!SRbNOjb@~a9&jOFE+6t8w5l5-k zdTI<}y&P@sVhpjVB~Hi>BVFh%gD4XJS|-_~OpVxt68Dj7`@M+vr|pE+c|bVq59acItLoe5}vayie|)|3azk~v%432`lo zIw<}`mD?Y9a&=ox(HpY|3^i=CH4IJwOIR%rSB$jwjRGV5L>QEM3S0!+AXzw<*c??i zwomUpFSTEMdc|z3p@J+lZ-J+NcqQrMSTxkO%bTT6mF5?>Pq2M+WyMelC>?tOQIReS zjt{dtQoQQ=%EsCDmewL2(NEz;E!D2Zn9~{=-E_W47IJQ2_(1NeX7(i>;=PR+fl#cSVDzHj$@9a&6P}CV0%v?!9+K~lVO3BC0WYOqY7`aXAzc8 zEJVTs%j$lg+Lhg;xOOf_YPVKnc_G6mmrG)wnXKSEM$C|6l;}oPPq8GQtq>`rxZ6o_#sP9UDhLLEj#04vkoRr! zza3@gVgK5Fr`nE|Tw|qmTA&d#beb)~Pc3h2LY=zQ1Hmjh8^abAsMSSpQ4V>Ug|2^k z(ApOc-<(t<3p1<@FKew)Kg(w>j6nz;KE9VE?tHO^fcAWdi-%HLMx-!3Iiz z=ugm#>U%{hqs*Tw zz0hE2as)Mt=rC$ug&t;fdY{0LyiwYWXpa%eOJL{@cw4tEFNVeW&Z4Movg}xk9-JAl zn@>>9tLVNC>Ye@t4tqY$>uzy$V-MgM_g`~Kh-CiGvH6?!e;E`=*rQDl=UEcV;PCft zFmEDTz8c5=Wx-7uvLBC6F{LwA;uxyvG^U`+oA&*PI&qy;Hl@}G@-Sh}tHb^J&z7)n z8xsai^sgwAXj)kA$dm^1!`4_!o8qrL7N~E@IFD?lyR6AO!1sw>OtmD&&+dMC@A69c zH>819=zZP}@NH8>zx?xS(yx1zUBP)ySU;5lffAW_3H6tpCSd0i%hhn5W;D1Rh-wMW z_>SzuMO)oV#7;H2Wf3%0kkVR{0^Ft>V2H7>m*cCHq94`JNIeKPXHYJhkhYHW{&CUk zAxb!AwPNMxr%mF-qAG^d`u+F5;1$|zD>-bajUkvUIZ;f};Ntq{f9lmh?64M$Pk~+$ z;eU!aCTd7o6p^yD0FicT10nHKgfnzyByp6q7-~ogpcG<^iFw!j>ia`=_rsc{<#gW9 zpMGF%(bnjuG!B(mnohEV_UP^;&;^p22G-BT+{ht;VrU@Nfpft#+`e6bSQ< zrD#nPTI4y3>k~XQs%Qgup%v4b%dj2YRIunkA&MVj%5|3E@a*Ye+yq01k@`j9`>*&7 zVPzU-GCUQu%GjO)B0W838ggAu^rjS7%Z^jk=ue+dEzP$(lNdVW+_SOrURn*|<$z%8 zbqrmV%)lOtnj_Y3Hd+o1DrMo9U54^~B}5;Ad3xLi9EgVD46H1JS{7=D&gHlF6$5@? z&ut_H)(N+F5N>30kDADR@{boCu(!z0KOsCnM-5_e?1En!^*nm7=G zW}LoLIfMjz6SQJ0@AlWaL{i%l<(Z@JVwOm

    Y2>I$CFd>saeoX|}l`9P92*XnISx8G+$frg>gatucaR zJ9p(PCp&~$f#y>MS;PS`Q` zLPEP|>_xuqBqOq&g)CV~ZgNxnD;Sg8Hr^C|K1@|W{C`i-e+>%%umS(6dF()tFo77& zbxk#Mw2zAnV+XuL5~`vMP9ZGtt^?%AD`z!7^M?&;-f~zqxvUs}e!F_R=muWaw{xE?as^&anf(vj z4kZR+h&fFhvcK=!VhXr?<}P9CwE-u6k_+3jWr zEgDQUtVZ!QS%~&wfc#la5BLLm2F_+-N@0+aZ`2yhqy{62oQQ;;FPxeWw+=8D^aOyR z#G3vL=A5P2C3}$az`a1(5^W7u?Q?^ymF0?JfVncw3r3tF-eq->dBo!-*BfGhF*FkD zOXDIjMuQ^PL(ix!Q0#LBC)+?eOZU$@Azm3D%Ytqk8_R+^Fm-m7`U+KbF49oChyu$} z&Fc-BY~UN&E*e+4OO$0vIRP;1BdBwp%BU@oDsPR9ube1}eJ#CUajZ@FmIv|jD)(KD z8_(qZ3`6`X?*7KlSUbraKgC=%F-4G3F(5=xv)QaA+$@*@vor-XF* z5oRh zglW}5q3577IpoMl#RC&D!DkkWCt!pM;nvda94%xpdeu#1L2?(yqrXwp5|Sso`IgoX zsK#G!F7XId*yhXhXH948n9|8z$XMYVJni!J@ve_1`a8I$2ur-mJD`IYGK zbJ&$i<$3i*Vz#4Ym2H<=bt}7$glik2o4T2CMwt|cniS&6qn$3!S@cz%I;$;rs4F*1 zM*Xq6k-#|hDF86;hM8ISb)|#Q;Qo$@Rx*Swpu+wzsyf?->- z^5MrZtQ}gefAs3t>L&72L1ZSd*f(8yk`A4`0{qk71W9?XzsO#hy2;SVMLO{%s&%dbG(&%bR^ zoIIHv%j{Bi7_EwaJS8>FJ$C3UF~zxjd_?skleaFHDWM*(AWngftrkSRrHm(Y>t=q< zfuo>cy9Rj9&tsI%?sbhOo6qFRu~=ivjj5F)H<7DC+UWh=L6uA4cm@5IKlB0~E9`} zOI5K>;X8Xmw%Tl|0X(~|DYNBO^oUZqSngqFN;*^6vY7S}=KfS>i1{Hs7LSJ^Q;b1V zJ^&q$?TUDeveS@*if;2*r_FFTj_>0+rZdmvn9iV8^D(J$<`!Ig{L;}=Jy_Dm*Q&-R z7`6{uMoUZL_W}F@*DGqeU|=?E9;s*i4e(KnW=Q=J^F?2rLw`S@EtU^QlzQ_Zy0tF1 zN0@(8g4B2XFt6oz{I|8QzQfE{g`0bYQPdQje$!l5KHCugTIuRv#0)<^R}HlPEp6%; z!2Z#vFo6;+I3Kmc6v5-H;qDAQ6c!?;5mF<6GFIs?$Y#(|&@i>1MA2O%b&8~{e~F@e zzUzhgdsQ?JgzKB?F6R!O6h=+Tx*+z84xO~;U~7NsHP&@(D_U$^r`!Kp98H-KJ?i=i zPW|1ydz*fmayQuK^t>6T1mM4@-Uws+(f0%2%pM7`w-TMgqaj!@#<_%tB3KmCoJymk z%=Jk2Rj{+n^cePGu^H2xN}?xOek3|IN86fpX$+-DJ7FiWT*lvUV1G-yF~BB_cdQB4 zlsZXr%n8<%c|?GzO}w^&sZDU~4)%l5C4-k*p(&s-&I-Ojag)MIb_@!}C5M+@p*9B6 zg0sjr=yow+T4mbg`z}xn(x6axygiu`q_LBnDN^Jt`rR}5#>^O#hST|`4h%D1<3~~D z-eX38vT~0cKu`!;-g*&C1YL1i#m#?R+b4$70}o5TXYrf`Ny?uFN!m50nC3NtogUjk zfXcWj1>3kLLe;sxt`?z$O6NNzB2@&q^I_jJMnNbj@t9JGRkg>}9wjLRjczl+N_(gA znDG7j4${FGLILMr@Q~AUqjBkYmbWNLn(42BqX6%Y$|EKe{?UsZf$=R{QqtG!K0&6{ z=>v0$k3)a%t|vT?P#9m0e#VT~$WegDh~c;K++=asC^Cg)2H)U;HpMjaQw0FV5JY&~ zPhQ7JXwhd(@sXf->7(9Ta82;(tM-{VxDuNw-C=T}_8^W`a)dX?-_dHK<@zdMcFPGd zTvWIte$5LZt7V6ht?bCMU&=&nt};}d9g!6gCJm7!L!sVvrS>S*YfXjL6>Q1sIj&uO zrt%mR*tq*wPC#{LoRx9<=oi2Xt5GD{+y2UpxTWqWpo3N&@S*MqlB9D)f1r)uRcpk* zpIV^a3;Ea>7;~))(SO|%h>TZZl-#R4p~)0bgw~ZS_)SQ<7uVAbHOq3@oQ9jofPPtd#=e1L7o4@Put@?ib8j~WA>4_G3LR2Yoouu9bumx0^VNiIEF>RdiawjkT<``yqj}pEL)Et(uFWuQ!i)!MoTShxE7$LMC52xsge` zzq85PvUMl~>1Bu(G&iL(FU^sgH$$VBfDu^?GB?wU;cBo~M#M_dN-HwpTgfYc@dtfz zO~NEwUt7THWgLMp{0?9U;9t(+z1}t1jkSv-7~UQ=ekZEso7SgDIhx5J#Z!gsrH7$h zaGMk-43)2d*p2Q50pwL?1LzyIm*x*`gU4P8fsUV1%)U0A#$pC?R?|53Q;CikOMl{+z zrTXnCWbQQ+T%0Vd7`;;JzD&hmdjdw$%`FZpW3|kZeuk(W{1(gYt+)P>%6?4f)MEbP z34PL|T|NX>ZkP;6Lq6G_3HpV?|~R*O*CL5D7unpsBS=nh&7J>nrapZuzUldqNlToh!K_Sq<3 z17ZfUxY~-4@g@_@=<-=GiGjtjMc3u(3TrL>WjRZ&LqJzqa=1vgtA!NHrey!ANSVD_ ze$Nh8Bv(+_dOlGu{vLw)BF84Z2{V4##?OX7nTB_JD48bSAjFd47Kbh>;1O#!9YRNSwi(^*KiiO2iM-7tHORy9C zZLQ*A`9nM#=AcI~c@JAeX=3xE7Jc?FEG(%x+Tw6(InHv8y@mYgo2U5q!DJ&#Hk3H1 ziugx$X>DCRDg0KhUt6pw&SDgaN(YVL^jxf%C+#v5d<>^L0E+_0YS8YF5eWeiCWB>c zdBBc1iK{22SE?kpSbUB0SlFC~Kdi*FbbyY{$u%y^Om$Q%{^-=&3WXKLu{^@Gcnc|J zP+GHsTz}SI!bZ!yDU+^B4gtD=D@t@xXL?b;m0M``s|1UBMjNVpT$_@V{ogPflM87D zTU4vpFG$U`!8Ri44~8PECy11Reg@3Kk-!8}sbj*jBZ%du^WKJg5$5pwbAEPrMmgbHEp&3@yEi^C!RA>-iE~HABsW~3(B~veb;-G?=#~&UJHslveQ%PZSYEa?aIISgCf1A;b+3U-vQwq64A^7 zMOJu|fMucb(NDzLlN~{1>wPp+I(CUT=2YCGb*%Adrv?rlkf$phv8WSUcC9bg)l^pj zskdM=K3J(~fBWDK5eL5ZkKMaK$Xm{``C&0^D}sjiN5p=?Rc6$>`EsyNj1{oS1QI8L zyai>5qnkX7fq@g2OcYEs&fg=+;b~kQng%*1e}S&DWlMI6SJQHLC^U-&!Ba`U0&71Xy!$sy$X(+1^T*buJq^PDdEddm=4+ zJ0*2*=jJB0Wfz}S*y%5r7>RxH0?|KJ(4SxC&hFM?d$H0c&rcv~rw>m8aJwFy)AWPZ z|5_KY)M0vFoTTVIm;z|Rh{Pj*)*n2rf;xQ+PO;kbR6-7|%TGefJw+imDnR%cDsV|s zPX%hbaerLvzuuani&^@d{exiTS9pwq%rufcinC65{3uW}@H=#hKx_FpMb#tY zx?Ne|TdwS7DD85`?X$|YvSrBM0C>tB^$pk(M-+j@p`R-Hb5e_ICBAph`%Kch#|g{U z3rkPB(~a{?Q4Oc+m3v7{c!ohYk37jcxI06f#tgZE;vP7e*UOIFJrJ5Z>ob8@LP*;o zNIg(auk?ChSN%ZEBgpSx+x-f!V0^)ZeeO4L*WiBOP&bM*F+k|?4Se=5#U1ijG=;tG z9W+14Z@b6W#C{+^@6RN2!`?K|84{$s5TUZV9WmoU7Gu%4c*l--r__=Dr@nx5F@cr{ z;uUIGX5!F23MXO*^nS-Itk}3HZov|#SO4%;`c&>9PoCx7^$&Dc)FY@U)R&1KWh=F{Cl$li6|aB{HFq1-MKpo zW(Nt?xQ!~{#)PF!9Nl98tsJSO&LCMd%CY~V5cCo`Wx;x=P+yQabTdRpA`z{$|bfiXQTFSb59N%h6VLis=sY4K5I9K!5xCUG@oWA$%zfH zewTrXg^LvDk(2L-@`Ps*MeHn?+#Lz}f+Fs|MMqv|9|8OajK;;y7l= z!URAls6lpZK^UPw56gi7h@=rZ%-*!8Ju@7}Xi`JqfGRB` zl)KdWUyk}l85DJN^h#MMZJFRZ@OPlB4W>Hwg-nn10(x1ONz4qDX9-bZu}IMruWcc=aKmCDBW z8?hb!LNC4t5+%zgg}-Csv3!zc8`JY-paF%Ib@?T$9E`0vRq}dbQjrTGZJ5g)@TPJ( zzbNR!by!QgB#as8>5MQGLw{o=u|XWp|JC2zo%FqTw*cvPp-GwKB30UCQY1iZyHGfi zVQ_nB*Xghg{c}Z;@#` z_%|%vo(4aYj5}@|336@=5#)i(-JlYkpE(JEd3|ZU>z_RSqkivkRPB?b;2vB7&YAgW zLpbs7d#RG)4f>cv0>63Bxhym$p)L^5J$Wj$GASR4vlPMY$3-PQ1?8?-f)0e&vx`vO zO|N5)#J>B$<>qcg(m|+$p6!GdamS$EVj3VNvsT=G4E6gqE!iwDhMF z!jkm=YT5tShyTxX5Yq=DbQ&J$r1^0w?=G%@IsQghAOg8`-i^Y}c zAM(S3Z7)8cpJS@8-U+!MQV%zc$zh*mYP~H~7+00c+LU0}o^B^N1{czS#F{jQ7&i(F zDV!^aIzYja=}pQ(YB|YosyoNCxWUV$(_0z}LQwMWJK<#{gNK7mmC%_96Ea07?jBfx z1f@9#*woZOrwsxxZ;4x_q9(BPImp`4Cj0o)Lr&iPJt>O;lH2|J=J#>u`J6cy3mTo2 z{%qf={tUGu0qYNZbh@U<4rk!g#+g%CV$esaIP zbP4HJRQ(i=JuUspkJ69h$ulNtTkCF$F*?`zVedblojngzR;qZbmdpm{b%hgD>IV)= zWHNAW8s<(ra`l&3W^{pNr1?ReH?A@27OpddSEn*Xde8NdCiD~-mAB|IU+XKx()sGc44?nQHiX&4mDk+i6BzOVH)9yf3*LUbAiRR7g zs*^`jUo7UtHL8sWY50XxMNf+pxIYf_&1@H~q1(oJzgX{QRv52g5Tc}0mp6!^X&HNb zr8Vl0xgW?T*V<%+acUS_#a^f_JED|Jaf^O?MbGV5|K(%2=A0Zp8Ca|liFIWEgpvID zNTtSkri{Sslf)x>`}&{m&^Qy;nBmuc5yd3gZLFOhaeOi->HAQn~7 zZBL)lZ*Ra`;dLP*kNeQBESaZdg8G=tJX5ulckm+lP(gb)IYq(RQD%j@mHSYkY{Oor ziaJYf)o!e$v?;?mgIYk9fzv#SS~K0bgjzGhxtUs5{w8bAK_-kEMAc4}NB)$hWIZ{b znpt(PluYiF-Moyt^Hw=<8O>lC7$)%<3aNNZ?(2u$c6Hh_2>Ge(p|lGJYpBLm?5MTN z9l}U^PurD$UDziJ3zS`}ht$&gi0)pYZ#nPU`OJl=gs^ND!83L`#ya?Y?b+{R+c@rX z!B-e&g%xz}jd{^e4^X6<`DMpfbuEf{w3FZG5BL2hzHc5@5x9LOhK#Qm3A+-A)cB2R{juxI8@4Io&kMduC^~x6v$aB>mQ~O%cw;YW@1ZtSd z9XCQ~7OVOY^@<;rLx#}=YG=x>$wO+Xeda@KUTgZ+8R&)*oUV5oVdJ8MG`-<&lKW_3 zKbXEA@4?7g?4`keh{o%M0DKT>eOw_Cut{_oqy(}wW?ke4uVsCN&?@;#R?_6}lZg9lE*=(!Pp)|+L>)fa1|gCg5 zR#vD#Hj;n#dQumtt9zkRs|Tm)wafaWt`$@Ds=NO@V#;!yG)TPa0K0!v*(w5JXlymw z#0tcXakpB{c;A%+y5qU%PJ00@l-bqtwRK`ty44t<#VqbX*E#0dy+$ilkJ3`xIjC|Y z{G;K3g1Y6vg1YqpBg1@D;B_`pck%Y3;|c$vVReYGpsNry3 zul@+5INwa38t{Jovj4-@J4Q#=e(T;((y?uHY&#WpY#W`VV%te|$F|k6ZQHhO+v+#} zXYconv(Gs-M%5^MS)nfmybeN!0 zPhd{z1Op=WTi@>(LY*w<_gQlN$>Gr>yApI+V<$_Tf&-T#yRO!zIyMVqX%D@mlLN~d zUthh_lBMa>zdwc_v^2a_vlzl!C7l(V=w~tO_HG&g7u9$U2}@L5skt_sRs{!Vg5(hX zb@|;o0%#tVqe%5JN~OjPJT)M1dZp0=3)VC}`>^w``kfh9``cYqP!Z@E+_vshgkw@N z{Sp|AyL526)muKu3gl;~G0XIT?a+-?qq~r!?!QA8?*+mPqrhtm%-(j*9?vU%>D9^` zHj@GmpaXR|eOIV>d@pp|*{)G%@$VDHd^1?^l(Q=F;zdDsZsqJEZXny2lVsL2{)a5# zn*=(w47tn^&boK9{NyJlx=e?vi1+klxlW2TwaLQN4O1U&HfyT-q|dHe;IpXsRi`xn z0ddWFNo}6@m?eWZrB0H7^ypZfXo?h3r1LaX9%IH;rL%-cqo8dY(3P}Nf*IHeKvC7+ zyPVX_A0kB72ey%Af-Zj%Dl`LdNr}5MM&%&|VrqY7Lc%q8i)PaCK1)KO(?NRB%@WQ) zXEUNzLoTik9?Er=Y1rvE+1i9>3&F094`UTudHdq~8Mc))6>UV>+%^R_XH&09wP=%d z)FA0#JFPL4y?DQT1G_wq7QbKyOd^4}Q@0XaoZo(PAFllMx)Cf`K zY=>jK{{9NU^MmA)Z6@lg`L}}%GbjWeE}yudY`)xffh2IUdfi>Z&4l5qF}RVKY1@$e zN$KDmUovo(4SO{PQ)RgjnSx6|)O+kKlfeG_3eX;h{eZETfR7H6 zR-P>XhEws zDj7+|3L_B4E3{pVsbyu8WkOuPz_PmM!!EdYR??M9}rC3I39VqpDvBU|qd+<>fn*@XUVr~OQ= z)0Cd<5<2PwRq8+lBPwnuQam~fhdw8T~7!HNk7veh3p$Yv!+0s?)zgy0cfd+0_FDK|M;iVy*l*xjvG78fF&U1 zjB^EebmmHZgHgshm2m%OVraPn5aqCX4-{GZUTG;?im5a)7Y+Y5qmm$#-HUgT4#7y+ zZ=-({o^m$EkhXrx7FMqQC(m5Bj7eds{#91Z?qVphv(Q(!h8RHmR#UKu)5{L}8OriY zt|5v<_92lQ%3j|l>yqo&rFI>86>&qS_K|}u4U0_1dPzk;z4HT5M`vr~NrIGGwgroJ zROSVfHmzGel!vky9;utZ*2swWkd*AeNBiiHrNTY^qqVuSHkv3 zfJeO~d7{KK3H6;SF*}%dx%n+;BbfA0*h+}dRX9gBocRFeR4gQ z_<{Mcho*1&LX zx2Qg}dL^QzLtMCc7>S8zL*X%GK5$UM7uMz|sYcX_YQ&j%gHqWSY5inGEXA}Jdp@Wa zab&w5{`P>}vm8AIs*$U?u^1G9Gk6$Vm)o$*(3753=RW3Xhzz%&7gtsCw$XeI~LTgKJWJ| zioeM7Z|Hhjfr3SPq4Crca|Kz(4Nf@FD_TR^gQgu-JKUp=Q%q&%Dqz89Vo`~>nj0)B9Edo^1 zo@;ITWz`v?>kWR_=ZbUYB`whX{9%;u_NTAoffftoKD(xYu$!dh>Qt18mUo z5~>2(hE9p}+=`pdr3s${*c*+j?T_m0v1{%BFmP3OiBRUt47yJqDP5J2sKzCV`kSF} zaj3gHmA%!3bGZGUvQj1Pi32;g*llx}@YNkYT-DAi7AhNF{6-~6jrZsLLIOA1rfH8w zQwnsvMFQt~%i!DcgTuAP&HDL<)6t4x07(yYjbn-ph!`?V#F0?Ui*^fD%Hb$Vt`b`l zfF61Kji-uk$ia)I178I&7F78~F#Jun7n((eh%^`(n8VkL;hsPs5C!!D&N+teeV(=Z z%`FCb4I4Gl%s$f_mTL@konmT-jX;q4gYPdMjkMF8obA2T8^?$C4ox}jg+%r>oY)p9 zsCU)(*@up}chf1Nf+L8kDdp0IlNGl09%!;}sp5tIVN36TfgdhI{JU(Kx$*IL9v(0y zoDXT-sII@Kou(+aBy6t%2!WL>Fm9`_T)KYFHZlbJaSLmAEW-A5_T7z+fBnO8p|mq7Yaf~NM_;DfqAIi7)%Qpr|3pz3 z1GuaEMqnT4wKEF#tlbiA#{Svmf5G)-yt;!h+fF{A|C3ej_$&3z?@o)JpYM?->=s|b zSYVAH_3d9Lb9%`;rw#Hy|Hu&jujbnS&$#q8{MxQ7$|>{;q4TC71io+nek*C&!15&hUTXaqdym z4%2CBC@Lx{ql&TqW>&>T!*iTwFDET9{6>ec;#k1KTvWJ|l51A)J%!al81$Zjwaz6^ zeeMQoBO;DIq0z0>W$t%l7{Y4o8baZ*Z_6NYIgk2iv;(MV4ae9n`*BhAAw5k_M2W!p z%qWB=0dDAhjb+4YhkPvN*n|wezCM3n0 z{^GJh)3_VA7+?H3iA%^EGirs94dIS?T#Ej!c*-DlXHn`oDZTFq)=K4QKdC0aU*X*@ zi)W-1qTuTNjGW7uaY|lT?eTu3cDv(2e zQTqNPm~gXR;a9GB5)M*On7qpHn!K<9eSChRcSF(#vXJdIhG?O>iH?Uh(;0O_lH7|< z2UNnOIV8#agaFG3`yNtBE$ZyV1DYsE9>Zn)(1mJ7aKznZ9XAOGr#B=xBG(=8fL&sy z+~K7pmvy8Yp%^%TLPN7PQtf5vr;v5}nFfSkAGNoN)zON8J&xH+?c?K5R^pN`=wP6EcN(UuOf78bIEgLcyno<+dO z96Mx{IM_{Sng(xOZ;7XBZBBxc_G2?|KR*p$P!8M;xD%O9gKQOGVG4A~n1NU+zZxTK ztIJhJSgVvj5Cvxipn8#Q>LI$nfI>A>@3|;NuE?4VVr9~N~MJp1h zm4vv|<-)8o*b#G&!8<6Pna| z`KWOdL8I{YnLF&IxIU~j+Sl>r!>ER&xxw6-0tnG%IvQkkvtw$1kC88dAJk)R3>@a_ zZJJpP&|_ED*@7eIBekA^2J8$Fp+5#xTM(ZYW2?Z9p*C*&yLhsQf6a5n3w%XW%)#oA z(VzKe-WyT$BJ?T?fR<629y6H3nO~t=m$Tc3lnK|wgMSTn_)0 zSmn+_s|>Zz<|_=PWYu&BZ1UfKQJ`Xs-*V2seh+(Jvld^i&HsVmeF**PJMx)BAtr+$ zQJ|W}OrfOyyU$3H@n4_dgRH)o$gobQmo zS%1F29sLA5ANW9d{qd|ATt-&bHy1m{k^@v8!X*52{H!{(pq=6_2HFnkX8GNOplTHT zo3I1GDovrn^7{^CNuw_v_ql z8E__e0yJ)Vd);h_?Es;w?)0y@FKYY2fWSFQ@tdg7=4CW+X{14jIB;wT>kKvrRJ(LZ z;BWw0V}v=eVRyd?0w_JD*8`n+FF(`@d}-h%XxdhKTBo;}!Po7&GIVO%Zntt-0J}NZ z3GYV(zrX)Ce`p$y(T7(W#RFTAlQSh>_PPFC?5LoV zN287BpJ}dbig#Q7OtHFH6|xa8&pJuuFlT_k!^2f6zlKr^5sq8!7(^^HhWKT9u2pDaqwe<*_G*mv0b52cQ9Caa7w(^z|0V;b8 zEuP!_gZncs-)X@`}=+f8Y zJa7S6mw!Nv<8z~xJNg~;$wxdJjkyiFQ=zAisx5f`TJ@m^6ea&ytfQ@#_V4#p-ymPBL^)(f zR~t?a#I!rf7`lXG7VQHs6;GoxomuDc=U~$WeDyWp5bpA8*s8GLZA~HUCGd4;0LzIu z*!7fWt+GLlCb;!#tn-N@%##QfruHc$eE{9wk{>|~f$FO8ZSEI%TPX!rmTF5u%1ctM zQB|j_nx~$Dm2OFcn@aW-k&}i%XQ*`Ir29`iJNr!HMf`ks?1neQi5qY~sdCRHN8F?PC*jSCf4`JyFFesPQ~FiR{>6eu-f z>i;4s_i#1`n*+52a>VQY z;?nZ{herNC{|Wz9V1C*_Vf`BgQJHH=IFy!NGU;)1ibg zElgNUC)zmLLpAzh_Yxzi0Y`M_dl{VR>0bh{B(SK6kucK73)}Gk(4X}e2N4nUN}1!N zX);4e{~#;8$?d{HRYO>R-d&tndz~M$L*$oI%~gA$?6P}FQD!OAHw1mW`rk`|P{=Or zy?TV6tJ}c=xww_-wf%vevOoC5&)orgCrBlN$vI9ff@U78l|^_ zm-wwOrz0%Upy{&lj#$dD&@@r0`WgLFlIKKh(ebHf!8G3~^x2n{bDY+4><}+SX@+lI zXm_T!>P-C;RGTR~Ka5{e9+I|pH3o*6w$V^dbrnGaKggLp=w}7c{ixIvfMoy#0SyDL z)c4Md_lA;BE}qFd)ON@^MF@)OgFTU;40zh3Wp;*%()#?pU__Ee{7#3KK z+sQ$z^!~A>*$0-kxO)0HBV07eSrAZJ8HC1Y&#k!FT8pe{3(nAQD_=x}l&KGKyb$H<8I02ZPXWng?>lKk1nCILb6D^peGi)WOcB<%sAg|}!nX?-P8cN67+PliGO**c_j^DFF!6H%FGEiE89 z6jA+7J1hp9rYYHDfn@O1w_OZs1+Geo9UWLdI3wo8l!{b$!GbjEz?VA(6qh9#NKGZF zqo*ni)e;wctUR3g7XUp!p2orbMGCn4zm?N3EtEX@#2$(e6a+Sj_=fgq!U`Ke_(KyN z93rlo{2RjC*gi!nb1H1zKZXm7mknil)eG8d(pD8>75I0B( zf?6=Ymy?LSx7waI+^$-lGPt5u%xw9g3`iL#yZsO_Obm+95#_C2NcTna&<&|b<=J^^ zCPItRS*Uj9LF5_clcG$r_9U1sVJ(dLBnm82ECYegvTi-0T8zd9rSOd7ZaIKKI!Zb? zj55%p%qE^gxiqd=tllX)H7XsAAy^UJN0wo(DS6+^Y) z^WLNpbthAW$yZY7AeBs_CUBjyT=i~7*p*0E?#UY;{sO(qm{DK(`6-pvhmYS`i<<`I zAP>?_P(sLElnn>gsmM3p%Zlym=QML>TL~T81NX_)a$~b@+c0RdonFS|VSwBD?I{!h z<;uuFDYr2x{!G3Vr*ramO%^=W->oyB!|u8>msM#AxYmqT{vc64oKdE5i1I1CltZnb z`A4c>gV;N^pHn`@<46I1I|kmB-%je*7?s1xKpc(VI!3<)7}6O{8;LS@ZRlD?gmEwp z6LM6<<#0dVAKFxp4x-k6NUquGRdG%_a&oTCK`hWeVsW4g*VfR~sRyH06-m{ngig=h zWk{S1^^-*NV$E9!!?j!pE{vvDg5o3L%3?M6-4~RKBBC#7=}M!UG{V2FTpXy)6EtCq zg7VbwS~!N?O2Th}2uB_I7kj)*q+H&gJpkt7JS#O)0%XqBqX^>npPSY4bqz%1KT9jE zN>%vG;75P2M{O4C;Ts9P6zp7#Co+P1R)$|*jBoKtg*T;;S9V&t@8a;}1AZVOgo~ct zR#4I@+8seRaWx4z$=7-q$$9!76_T-;f`-Oc3r8X-N`W>yr$o^i6Jj!D^tF9aSUTF+ zxHpn-lN+J--Ij2J6^i!Bky0q18E1dkjdw}QckOn08&98Mw~@&YJrueR3L2n-ZCanyw3Tmk@wfVDBeQ z6rr~#rFRBmz71O-(}w<*x;G=x`a+?T^U=h|}%oPfa;}{)cHwpkJSX?V2o-zCdR4X!`vF{>yEXGO3u^k0mD-y3UJ_SrG3a_yr z1%f<6P%+<3zVk>z#rm5tgLp)PVpihhkP>tDnQ_IDlBs!qxW0==G=`Bd%zxanbjqy(8vXhB=`+Tr*cND?8pzn~zv8r(caWaev zu*RYDapCf#p=mW7Lj;=M);F2#$%eRCEO) z<#GJ^3Xid61u?p0FZj&36Ylih$lh+vz_K8YJpGM&v|2z+WKutZrLt*3c6?HCe zD?J<`L=_9#*+;&j`6ti+WNw!G5c4lu_{qv$`dQmka9V|v0a6DoY_mQrAzUhhHny=I z3J6L+gL<`T zuTcI)^IUitN?ZLB^IdG*1)Zr9dGdICWdh@5id(&8>VG%4ksd0bo=T#b6HoJG6JI$C z6@4Nksy_XDyiV6%G<-57azFEW+D-*=+Y37VRm!gRLFx-s2t-PMj8Es?SQbv(JVZ}b z5VU-dqji0lR7yIdb4vbNUsu2S+Uk8^`(w|8INZfPxcsT+k*_5ekZ+6Uky#Q?gLwj9 zQG;@Q6AH?FBTAG7C()DydrIaNYRTsnZbB9m>H-#~hJ6?6_>>-;ywT^U>VC{KF|$5| zm%}{BfWF>fV!J!lz^QpvQ!jkIg0d_738X0f$@MJxf$YyUZ+ETyK}jd_fu*4Nq2TW} zZ|4N}(1jqN)YYeR>XwT7(1j?V+BGy@vSk)nCb*4XATYtD_(2j}Drf|A8m%>Qs_^wi zBG8QwX#_DvT?N(snYd5-iz7A?>C9#Q`W1#i01Z<{i`3!qg@SeZYTW)Gdw~iMD0Yy> z3J84(^Ak#nNQ)>bq#6beD?OHBgw;?A1)>@eN@WG*8z-e)pC*wE>1_lno=1+wYM}*< zMQfFm+iXxw0n0QTx^f0*#T=y^aqGuw=Y7?&)!Nl042eJUq43eB*QMK(Z}eg3->s&~ zZ{|IJ+PMz~(DjWwa2|Kh^u7DSiM>EK=uDFyH*wg4i8pMxOk>#K`;_nDW~qi_kjETUW1`_%k3~TL3_+e? zlheSE(A|W5r-JYp4_4zo@3SO85xvVH;`igb6hYy4(fm#3@(e>%(T9gX{fssU$-^*Q z1o=|#X9tvxQJ{H7>MYo9!}-dB>nz>&s>bjpHp6)TC&-T&?v@M#LpP{$8+(}MnIa2= zyeb<;C1#4I+0P8K-CqI5Q9TB9gwakH@Tp$>PS}}kfHy_$j{H;g+RIzKhq!<6&FBS8 z9P?d?`V}PzLcyOmV~r%IPuZkrk)}3LTW?guhL3qprz9b?mIu&n> zoXijZZ01(HN7j-}?P9JXf`jp~mh8tsW`;S8+Fi4Bq}U+@J(YnJ2lvOiX5h~r9(H?a z8EsBS3(Wn{(fn0>iMc|SQY2%JxACCq@nbG^#L|F99s7l{eZ~RMmcH6>2W2M*yVvHS zW7U~ivM|G|Y`~sriS^~mIWCUPco;=}jjZy7dTWCjR-jWv1i-jxt&J^p;uzNoyKQH3 zwtvGG!4eINlC^bF4~J-aJTDhY)Ac}>$h2o&ws~r1BNjDV_NSGCo7vN+JF^)cA148U zoD*5f5`DND7T++499m-D+ElS&NpwcsiqHJG#}SO3iwk-sW0dTcly))2n<`WIof=}sZU350*{p<|lBFnqk~1JL6F#%rK@ zRfjVi#5htOTeQVE!5+GFE5$KXz*L}BP%V?fge6|9sxk>OO-`CBqaih6Y+W#%#F!jq z(CTfhH#yj?NvSU6`8k~)f6hqSP&8koTfQ$LKK51CD8X(n`!P6-br3q&Qjuu#a`N)7 z^0l^TVa$GGU`X%p3fOB5I7AERJ@-*E7D=7Zh6fg&c2816p74R9-8ul>he9tHvE zfUG9=QWaXJ<^T@23u?34|mDYOq#UpFBNila`*N z(lXa)tH(BjOR*~@M+|D%6kjeNlC7csQ!ca_cF+?@1GcQxrz?SpoT2OOY2pH}?3ZV~ zDy1DPSMo-V|1DFsWALV0fLkAf&PLEe9z|qA7EcAD`gW4)$EZE~uF#3#`V7b1< zoa3cmNd=Q~gp*2UD>CAI6* zVGuy~)N68-W=o#4@kRF`X^q8=A&FXK3y~NJ z{zW_2_hw`&HJQz~E&k4=QXl}IC@N5jZ+Yv0(a38bZL#B5YoW#P<(++7ScpWJd-0lS zQ+x;DH2I@J%sTwF!w~5RHL{J#@Pgy{4fK&kXAY+*>xIgP$zbGv#=0mrsyw{1`_xjE z+ZT8Y5eY*U75w5GdySb^Gsfu(hz!fa0z9X!j0}x4<~|y*X^&ixw<5O9MEN>zvz!n8 zgY-QR;`Vqq)$RHv&WbESd8{Unh9J~y_wrKeJWPL}(<5X_x9{{*Xae*6GJ1@?1k@73 z55}{N-^fAyG?oh=KM0o|@q-lQMUk|fUd?vT<5lJ8@BDB#3G%ndgc$VBP zxtAJ?q;NKKGh)D{7c&u{;xVq`q6T}XNbruE71-i~8lA5N`HnFG%Ikz#gW#(~uGVhB z-@c9^!Yg+Up<(DJc7u~5aaBt!$R98EoA##mq;U6K|H|SoFT64_SL-X|8rAcHi(EdJ zSZR$sut;xJfgcL5$LZm*7eypgIV5_I4avyb%)5*j-`Qq;BW_+i(u)45BeSU8+{zZSY*d}XoK`B($rugLnkOgw zs2}>QxxQgcWiAC;R}t0eX2YyP%@CSU?dI=MAv#4?Y{CgV11oQm7+QpSgKHn#u($;M zD)4rIekS{3MH zkr&L<{@bp+S+rFYf>Z#1ECBc{>%FRCA2O4H#zD* z!WwSam*n>M>j9*!r!Z%L;k#K=ai&g4{EaXSU1`xq2$F*ZNp6^cm4RZ^$dBt#&nGZ$ zma#x*zv!6w8tb7~7Tr$n9rY9Ap`OqZ;&=L8mcS9;QQ1@z@t@9rnxk~kOv%TkQ{uCw zQj)8`?*v7V2boCXNRtKiAV4vv2@|w*M?fCxLcvB*H%EYAAyt|S4S1Pp8EL8a{QTf< z;T8m|wkZd39WAA{4)rQm)>q~iwby#@%MgIj_Crn96+7_JZ!)zU7y9P^ z`P%Y0SXJ7nbUS8Rc6-17NxAe)$Fp;Ck}+3zV+!l4w>~Xr2v%rtW{)RnFkejqzC7W? zh5JzHJN*Q~1m%RZk)AhRAA$=Ma78>E#wnGt8M}<3iklTtv{Txxh^Fukd`GCQdGv0{ z;W|^mnLV=|VuKv`a%vXRtO|V5-Go?an#2tP+(-*-if{&T#lZ58T|XXu z0oJN})N1Ec37QpZt5o)&hm9a1*%kXm=(kaAftYGIlBm3!#5pKPFp^g!AfeNZE2Eix zbYt@HoplZ0e*RRXKY&TY=dnhqGc4f*py@d-yfl}S&|u0|v{?lr}~#(%W#bhIh>_w&N-p9x8r z7>EcM7&sVMcdc$*Ff+&u?CaQH@Zh_A0tXJD*dYp{WDoDbJ-Fy8kMW!4(LKCqCU@_V zJvyQ#5ATURXdyW1W`qWqpuYf=7tTG~^4K=@w_3g5zX<7W=>2_M%S}3S(47-@^l)H%Lq?@9u{W`KQHlm{K{z|AtGB$h5LjdDFObEUM z)WN-oXIxIj>(wUdXNq-RMf*e#tzJe)Jt+sxyXSfH@$Vv9VfCy^ zPQFTUxNN;)W}E~RGsUF1wYF*2E+zxhJd7v)x1lNJPzQ^z^lW-W@kRW;BQQ~*c*0=J z6?HFh%KhvaZNYKFJSzurk9ccO@`0%3%Fe&g^D2?b84$S*<`n+V1vyvTHT6B?~|G zQ$VUw&yIMQRG7F4hbSs^gXH}F9mxtGw)47dXe5$fzN{n4kZ~VA+aL#s-+J6$Q)-Tp zr#P@!RBJ*QNm9rv%1LvOT92(hyHJtFypfCY%-D*|QK)cBB2{WyDxLkEwWaWZ9_Y+% zfy0G>S8km>@KT)?RvMyKqY@^Erk8p-xepsF!ti#n8@+6ypqc=@jM!?(1Nppmu z==}DNuw~%*Kqx!F{F`YtSjYXPC7Ux4QaACh`l4gk;!Ectt*W$Zobm7c;chX zMJi}s#866nkyWjQw=Kx$gC|it_3u>ku!nXG&^lD|)nPS9l6R3bv7#!#%p!P{?LH={ z(nbnvO14dSZir7=mDEYW-IB6JUsi)J92t&qxgen~ksbr@Ax;mJoEL_X&;j@GvnFfJ zTwO0K!Qu(>2OEiHz=+s|rnULzSDFM9dV7wGLfxng935LKermtXt847+F#4ZMr`}|9buluDyatlr4^1K4*R(kk#+7-{LpA{_r48&@%nPG{ zV&7~_u2g74;eW(H&nS-4J{2e6)$8I96Tt+;K^_Z5?0>UU;4JtKs4YW)P6YUjdA){H z)8I-9<4oPC2K29d_Z)Lgft?kw?1M8H!E1(27=tx4Abd6Ujh*e?Wr3}>U$@=hqu@wZ z;4b-H3`og+KTsrmz!AX&wFG^$PXq>(1HllCJZNX3MiIC{;>Z-Csn8XAm+T9|HF=$>Fvf{qYol z6}~2Av1Ho>YTeUp`p4}1?^5=i8aJ| zDxUGdI3X+WseFn8hIT2Qm$%5zLXS}c9NLRBS|>gURE3>#9!jTmg(^jqgKk{ax%ir> ztdNO>BUev~#u>7j@Xe66lHgZ|EI!@r(x2plPO1lGO8X~h3(s&b>0}pW`z}by4w2kT zC7evgysZKYZUa$%ZSJ>7rR5&0F8rzUs7Efj@YlQlmW1)IL2*=H&Y&0g|7+FxvH(#h zmnA}B|92ed2Iol7#jP|A3RuNKVq=V=P(XcsQZmG7-)-&uSGp6WoSa*qsY>y)5F~_{ zceW6i5(I3q&`m7XDf1+EAGfbV#tg0?z;{~dFXU-UdcZ7Nzw2$=QPvyaD9dT8rR}Zn z0-pvyYwde{)SX^3+S16p^?Dr)XuDSk`W<-d?1p;0yLC9m~^ zE4LBBf>BkkDODjQszrKqz&8h7A+KOg^z0#tN%FVLr!z&`{O*`Byu(8LreYMYTdp0o zt{KV7x5Jp*!Uz!pvu^+C<5utx0ky4`yc(5a-PkZk7Gsr|rU+YZggnnT<5}-@nwupv zS%?EW^HJ|?EUh(5EOT(UN*S2a8P-P)?jQoj(C}_w2(DoFf2JIanCG?HNwE?fLLqZF za)^H3%~B!Fe$(c&m<87Wl!_4y_{;Dh=Vu5KB)`MA~|fsIEN`%%h3=DUOd!=5N86U5=#20U$~szJJ?A5^p782>B~*1QF1rd zOkMkNa7JV-=MQZf7%*+1K*&stckFg_RwdL-b*c?nyr9S!TB|X-Q}dFb{?=QPRlGma z`k5R(nyTSxrNGQuqBS=^jMYbD39b z<7{IpB_)GwPlYq(mx);JjWY&0=OW%{P*^A>asK>WRL=EiC2dmY(>yw8^XIb_cZuvJ z@^uR;W<`u$yUw!e17)voNVo8pJ5_QaG{__b8_rJ?8K>C)8T(^Kp+n7^mXgN&O*`aD zAGh_ z;VP&r$Nop63&eAuz7}26{{$nAm)M)j8mZNBU3nQiJ0Be@=TltL9JLC5zq_L^d(xN7 z@zdb8oE@t&GjkBQao)1h$d2`FQ+Huh-DcAS_36B3hxmR;IUt*3KVl&Z2#qDjjGd2s zAw_7az~0X(9kCm;R=|+3+^z)xGSdc~yBHuXWGuHKK&eB}ItmSe zvnect$>J;E!E5Tf)fd(Dq2_g85`Q{>UQHX^2~I1^h@4|ih+U}&j*iz^$!}UC=5Q8| z7#7o**%&;W#FanP10s(KztJ85!@{;4R=#Enm&xl$O$mc}8%g|2cpVp49T4+eg9i8_GrDr5TEnuM-V|E6! zECZ?-&1wxCp&}}DuyCBz-^^uyz4V9+9aFeyE+(VZ3rdbO+;PBg|Yf&=gm>SFax+Z4ReT=(! zXTnTvzXe?3msX;idp^6l&Y9NppR>B0SVpPwZ>ZYHy@-kaqQwVqAtCPw;K5(d{%e+> zyvfcG_*!rx|DOvE{}&b$)?IZ0^OHC=as++Yd>STzDi?tSK1>K8Y>t*Il|h~Y!3>d{ zoSHl`yx(!t!6j5Iqo;VLzND=PfgRCU+`lSC+h}ezqujQlZL_et&}iAvs9qs+0@@lU zCn5KJjKA9Jdiok^Z#nujlhS|u`!sU}p28$I><{s5UxB>z0=mW|Cc54cCMLSg*$NK! zaqkbVNB$;&@Oo`SOmw#c5IBF>fhF!d>U;8W9|=zOd9J|jdfG+Sf1e2#^tdr&(tp@x z(tlqF@A9}w!>;nWL1x-~-2J2fJ{IoQ{@jFpCErir{OW+ADi~&;^otHccW-EhTccYH zobdsOH;Glf35q@mfo<-O#yv*0Z~4XarbMqAhKl+HT83?&jZW31`cBi0VE}uKSuNF# z4XJw=?1u=>__&9I10MI4o=pRm{*gk#$BorUTsLzun3FN{Uw0=7I3T5 zEH^l}+mC#shWi$1EqED!gU~qgT(kTPYa4f?qRX^r)~qr(<&YO)2KoNO`W=mH*j;|y zmtxx^=czrV=I6ErPz6Mjs`EM22ZoNA02m7 zp7syJYF}Bv++dtWQEMxEoic$f3a2{6kh86%yEI>pS`Z?edJg4+WRp7T&n=zU+P%;) zJhGLxUUnl&3%DT$ojD5)ZVfyF2P>zemivxdbOIUz8J`JiwiSu50coBGwY^~r-xHi$ z7ElQelc&=l2lHy+o*(pplWzC`@a!N7Er}qPS=}JT&!|a$ArE#UW5S|i)8yLyqj>KR z#jpu*6|IbSO`a&D!SB+d=f+W6^A2K(ojoKJu&l%&J%rox=vZEbT{m(t8WvEfon~a& zxH9${FTeXYkguJ*a_FK!wZv$G@}gFoA7msd)fwP|!k=U(IeAQA}yL;t2H5VxBL?QfnQKF&f~ic}JX>JzAEIwv=eav;oFV zWrGuoI$BedpRly8onW!6YUniR_RhC9Hex!jN#pT;G+8gmYweP{Wg4y!PzCR`f)*1_ zPv*xt8`yfu2?lWs>y;g|3% zbD54D>=wUqQGE#V?WK_=!MfaO zZP+B(ts_m+VnWZTFf}t(|NkTGErZ)^nr%@tGgHjW%#Q7tnIWcFwq<5!Ic8>NW@hG? znK@==jw#pgyU(e8?)mnuTP5j_o*K>c%=C=9d#zO`%7u^m1jAu##rv}dodoq}db|ex zgJ1p)M{C%ApC!gV1<4R7OoKs8^}_ZOFA#)SMB8gu%TZw<-?nm-I6>H(zA5%XJUl{v z*_Aq<^mSK*I9d%t?Wo8>mLQ_p$Z>n*+#69aK%0SUicnn2i#JMboe=V;5Unk$JW=lv z>}=vtJg1@}h29I4sjX<~y~Bhxfp~`(EOj#UKBwhT60*@E&csSU+eLLfXR-~m!XBs* zJQLqB3B|}I&dQC1p0?cXM>}!?iR5VP1|QoVnhV?g&N=mPl#FwLB;kn^X^cn@!^sTk z8bknX@S7!r^s=;j*X zpxuYi`MrrHzm}AsJB~1p#Q`3}YNvl~U0EilSzPk7^c+nLqj z#*TYsRMDLw;fuOgn*j7-L9CBye`13`jXUtIe}!v~oxPB5h=k=O9Bf#Lz`f=*01*B7 zmEfL5^$H`^^DHKn{1<3_VzBkB945Bb?5Fo_i5=L1of09TZK4V)SQvcL;7YpLL~Qx; zTx(52zk;rGDi-^0_h?ydX1XNh-~)weM{>&nk#Za%_y#>>OM-*d#!mee?akf`=OIGN z3^zy*D^PG1Tgmg$^zHlFk9t&gxz?Wo?-j4B#<+%T-zY6`=xY;=kv}zsJ8X9o)>hj1 zTe<6gX4w(!X6lc8zaghS#9n<^TgAXy_9% zHtoX7^Fin9rR!E3N1HIn=N-ln?Mf*1X*(;B3O+p$2+u-QDXpD!E&ENc|ICv!SE?5o zoCnA|OEmzU<7WWg0*y3c9|01{^7C{?yhi*yGSCytnk&FDUZC!IdINMJAP|OFX$`eS zwEK9eu?73opjE`NKTZt2sPbDz}nJl$P+UK12sT5QEQnP#CeeKzV+O^~5$Im^y&Ik-=k-Gn5hmaFK* zbsk56Hx9fN;5G#1PdFXxcnPiHVWmgX(uoEyKI+AN43A&y_zX6|jl%{^%k z*c!RjzA^UC822O}2m%dLH)1m$|CyDI8T$2vpt-2a$-ue3k+@MRLBW!epFm=Y(*%qX zU2>G*7%f5SJ6uqsgJIbzlbtI(x)YQ`SIRdxQFZc;m@r|sI+0B7Pl=}){aSudRvXio zwHs!X?pfR>Aq3Ch6+d9RufL3l9ar4YLphLiDoir)E8+XZaS+HQeTFZG`k0WRSlgjx zPyg?;G2U7Qkn#GPsM7rQTazz%x>UDxATfhmVPM_>dyBW`loaGmR`$Nm6w<4G$YV+Z zcQK}o5ntRQ^&@6hm67r6Nw8QYJFDO)^>voKd%$aJtpdydRGz$yU(O@YPje5c|KMJ! zk!RCcG0?ZSNgwphM{ouA^!IR5Z3UE8XWB(U*jgjcJx-_jS(@bxCmWitLu?aiMp zS7^&$w1x6c3xyB^L(6k?D$I*p9^#_`8&uF@;ys}uz!$nXs)YxbR}_1fzG7!IpgHQ~ zaAclXqrml-hab^=ovw?lLLcChfI!b~(z8-CR2gi{*b`lcJ~3zh>p!j4Q_+#LA0$HA zE3HBVxfM`u-Xu&N=8SpKmE#@P$ulCF>&%H49Q!=x#XPg$=I}26A(!OMy$%{*dpP}n@ps8P5#F*^DY(VI{+^yy%cTnUwL zO$SqCU5-h37Mskh8~jh%RimU@Pji)x zOFgTd*68RQ-!38AUD+_mZ?J)`rBZmg>liPSm$^@{{}`mVj{TwgGkJga`ToBQa>EYw ze~!4T_R}hzh-h%1Y>&GCuswbhC)%!7>micEse@bmtoMKNPZgmw1S=qoD-D~!m|&IBY;enePm(7Tb08nrxG?@@CC39!VJ}ZV`vsNY*W_6EksBM6BZTRDPb9AWyItFd$ZF%40=URw#Tjw^gI+bBt4n#I^ z=0W^FC%%Bl-|h)7z42oRm(RS8%(_9 z+R7;$k#=nwmn%Vui{iX4;^F?-adK?;*6Y4j#)xk9boKPiAq zXp*msF4~Z~s=|-{7z1QH9bU`W{sCc+!P+_?!mUeqV`vav(~ zPrr%tb{o~-{@pIZw5*;EP*&y`O*HuY6&(#I4;HM(2KY6xbdn1@SaQqd5xE4n08WZc zJ}*%4rILxVn)InURC5fiSI}4g-R3Rk{w%Q<9*I85 zbi$@#guc9rDT}n}XoTS-%|DCvld=Wq0P%TBW_$wj{(o010VqyM4nh=fC}(!DX>zW9 zLM|jX;60D7JqfFo6hl*a)<-YI1(>D3^t}vqu}@Ak=~E)l=UEtME8ME|@=(xez{*#I_3wNE!pJYMo5#EQzCi zB9YKyWtg-cNhlrcZJ3*}%17L?m`cqM6qel!JdRlN5pz+A#jjt=yf@8g2_cCQ4(`q_ zMLg31e_6cPGbeWXp5sdR;XKj<-YklC z-|bFjgq-xb-%^s}m;~hc?+V!rq{mTn$RWNq2*MM;LB16XF7EYK6XA2HcUv>9dy z{=^LRZ>NPBkneY>_giHEJk_W*GOq$#zb+=uXR+4~+5rmDqwc?m|JbdI_9hVW({9fH zQTvnfuM+P6wi^NHpE%}MGxkw055aHjBN!syvJ1T5!9|JC;K@W-ISetRVaxc_Jksz} z({PWvenPeAtXo=1SXJRX6{VM#!m)=Hhsz;qR%u#2t+h8@Ho4Z>{>r}FGD&77Q~A^V z_SEe(-SxE5u<3-EEqJ{>0H$mwAU{|jo=IgC6e<QZMx=Ef*WuzxAHi#(x)*kvB?S>qk1?Pqy{R->G7d?UeF@l$L7n1xd z?QU+UF!~J{I*4VLMB=R>lq90rA>YQ|oB{jV0USiXJq@)%v+WC>gLZuc@s(~{4dN@) zHVc%uWUmvO)xjhciO}3gOue{5R$`2}C!`wlMTtir-(HYhegq(rr*g9=fkS64SLF zTV>Eu1mMeC_zrY1F8s`paxZJNs43pGGPx*IvWzkaR*C&h-{13@VZaMxLPExW4?qHr zBg0DKJAPdUMZ@|V!aWG=0tsROE4U2l0BIgSI(A#!;E(w8mw_w%4-vn4O8dSacpszJ zEbut}(^T3}cGC73EFktbf3OHDZ3rdl88BE%?O;+-OvOY)KEZ)&f`JUAD-Q2(ehF0C zeTbDHKM=>8lz-ASYH$qRSKu=VxMFNBK5Bv?Hpw-XyAJug5Y^9&UoHb6eDmOpKO5R( ztTGU&F`my(T>O;UkalwR^l#t&HQlxRx!uEdG$0sFl<}eybF-ebBWUAZE6 zhOYSsbdj!Dy?d|m!NZX_@KDTB=|o`#8Ghn|9_4DYcQ}KQug4u~v3xmDx*DU)R&tR_K4NVH=394aA&|yWI{Z}xEt5v2S8Dz zuxTq~7W>wy(^Wr^{j!%H|7wW;GgLh5mF7Vc(Tunb-1r2G*|$Za?zORB{<;HtLmiDi z)5^WzWz+AGxUc2ZDh#34sSL$jo8W_l0mwdM*TV86?tWf5arHKSCF9)UDb@`;^sAbF z3?><0dg#{?>Li-6_KawcuSgmJcX{WCO>KQvH?_ zrMhJ&k_9+t2(XwAIp|A9oE0F?`|k@joE?J(T-1-;0r!#{G|^+57ZI=do=e15@(`2y znF2hWZ7s}zKnj<_F~-NFZf(;1A%=Qd%R=2K7S6?vslf-h$FZU}K#jHOFQt6y9GH@$ z0u}{lm$b7R6CGnmpwq|d4B4?(mFK#XTSszYMZ2KYB`Pnw(40n}7W@Rpoh( zznmD;_=OUiq9KDo*R(B-fn}Z6w)_PG*2>EKR+|6&^rLX9Ex#nN`{$b#%hW*nvX!JP z@^EykNc5~l1$4-vU?Qkaa+UH-up~0fsj?!fv&P~cXYxw!&qcgoX33ZRoYjzs3Vn1{ zg)CjJZ#sspf7ZncfA{vWNjcj@U^97?rAddCNp7w!C8Hy|srz>8{bJz{wI~#qE)c_y zZJ(@Tjw((;&QUbzaK@|hXnYP$N0>X)ES1#JnjFCS!_IfHzhec8@~zNzWPCU&_n^Ez zXVcTuVU&CYOc0*p=!9&o4-@GGS3}WF%5|fduHNsi`$#C~w%d%*p*tOk&WVEtOkN zs^SCz)gZgnsB5h4eiX#UxXfED|&!uy%rsN`Y5msqQmiN@DeGE9Vwg zcGsM$-`%z4#B^WrFBiwtQ+Rlo1dRWjYPyLDWVKTfsR@Odmk`(;af-yH9N9YWM_l9M z%qn|Dwl~{13gq8vH0wo3`AC{7y0(ki`)2b`@)?bdIQnr1$ZAjV{HUHO|L{H;y|ggP zp|h~E)L#ZIq~Ac7DaZ}J(;s2b|5_XMaIBKe&PxyC@mSVT8fXtbOb#w$G9g}H;|`qf z()o50ZIxvbX7I~w=`0+57eSw=pRHj-qx5B~awv>w8KVyk`;jxRgh%XAWw9nB=naPp zBkxO<&Ua+xpTZH`p(WIViwzh64EdcEW8ll=Ls$i<7Z)zp46Qi`LsDcNL$}N#Vfb( zGwNli7u%bY;0)JegbGS2IyGKmcLh08x)I#OYPIghLb-AI_sEH*2LGwiciw~1)>*5l z#`3ROZiG3PC64ibd^G(x%*;pWs7D=#i*#~9Eutj+Hw%?<>daOu#-bGB$)g1I;fiv{ z3trI^-$)JvM&rrWx!8YSB+^xKG=+pjXCctZWtGK*d(b?e9!+HD=(uIE#OGEMZ4g{m zqbOe(+p1yQo_x3P3nz80)p0#sOGOV`f@v&ek>{tIo^iPyF4dX;f@*ccdUm;TudN9N zdY-SqC~4-i&Tbyu=cE|EMUV=v4Hoo`z(L)&ugWuXLIG4|e-Fu-F|l?Pp-X%utJgXv z4Z%2uuX(Jz%%_FV2&X@Mt(Z;O?u~1Rz?}oM8|ZFwVj=AN58U$kvq$dEMjpH!Pp3Pp z2N&^Ss04INiwUNN#v6W@2Y#2|XAw;XxtRtZ&Q2_*l<6;>Y9BUMS-{ zKeU#230&oNice)1#BEPRYeVRPX{dg3NXrmn%Z-()=Z^sK_3uvN@WIX&@Y)(<|>5+P_#(_3U8+e4MJYaRymd-Ww0Rjp;(u>!idZ?5E3#Ts^s}{72MvlbaqPP!)zWe1bV3Tm zrzl5hNi!#{Vve-m9Vf~zRKAs&+ ztX5d{U^KCm`So31OR#Rm!E#@BbAd)pK6;{GkoG%ZgO4J$V-Agc1m%ts>;n1$_W8@A z3|VR}!lg32zX3G4zO3*^{Na_pq3M*~Hei=+(<-CV9TbmfiUH)`cvkYz!}H)}4WZ?s zQ1AOKX$NrYxTx+xIJA_+erlN-(C_v=9p#>2uKJMJi!!Z1wA0RFtOG^~j47v;%J&z* z-j=151%~aT=VO4M59tEs4o14uJQ$@L3T8eg)OA8R1f?Is16nYqAqD)6&$%?b0}DKv za>pn}Kd^?rCBuj|1S@aDc66J*F%WGWQ6}86gG$n65^MvcGKu)9g|N}LuY2i)oo3@P zYd{(kINzIsvjmTkC%M3z_!Fh6Fyqe*BvUQ6Vo*calO8YTQ~~}`{Ph#SOZGX<#Hrac zE&=J)@bFrq5wiDSU*HpHU;}iXFxvio+X}n=vp*wu_4@&RKIHuXag7|UbU${$_DwP_ za<8HA#-R%+Y~y{Di?upan|KSwzYFzz(ABF=nVA}(E7f*<2}!r^zMkiZ#_fJFWvQhi ze4m0V1SZ@79))y2u$O)6LmHb^UA8LICDpbJi|Q$ltwfETfZsFSO^vSn7xvt0uks^F zp2ehxf+JP2B_Nr1TE{4&X^_$?%#O`Ul>4kJ+5!?}&vWC&_<_x*Ydh9)&luq%3Hwl@ z{jQqga}0Ly*wxY7k}v_JDA|&{oV;+d$7qp$h+owlAZpMTDPDnjop|JX_R|#?oO^4Y zXu?OK>5Ol!A;;8L0MuvA4K~^H>q(t$Nl5+%$5r)0Z(85}Bs8=Q^tpeh^bv;T8Mw1Y z>_i2%^uvn3;vfV0v^9f((?}=3Wy;5K6lO z?ytqJWPsO$3d-iccT%N)Yf<&BpnSrs-Pb77Uj!oWc>-(($DB~;uB!2Y?hgvAQ*2pS zzhMbYk$P8A$l!IG2WCGNWHecANL0N%4~0NJ%2tjM)K4nZPYZFaZdm8i6?R`EX*Lvo zdJB)#hTKjF2w~(?g08BcW5?{RfR`4g8GrH1A}SD;ItFUI&0;^-Z11y!gc*GX93o`?smqpKsY0W~~ym$!49 ztW@b+;{YGrSr--{=JKas!H~L|T0}uOf3d%*Hp1NeXzFv<#9z2Ot&Fz>eiZWCEdd#% zpSO7tzEB)2Q*vL-*nskWxM43CL}C!2>7QEWZ~D95*f{-GwNM5$f4u|h&oa;`UnJcT zx%<`=8@I@yd!&ZDAeM`wy>QB!V%K13j__lWOgFaZImS1Ri@dF<@ki4VjBShF=mkuv zcci#7Iq8;HI94a;9zz<>r4o6Be~w<$0Vn>(g~CeTPn*A9Xj|FD01#B;)?_+By*(sy zFo2{Ml4>jbzykx>w3JJjzhHx~05aeUO7Zzch#lolrKt995z?bg&AQ4X*xLfkm&NG# zyeX^-F$U@#$B=KAvqzj)i+2ihz(T!63xnj}O^Vo@C%*{;ajs@on0dQMY;z4lecSb1 zYf7BOFfg|M^fxIRaLCl=qy-X#v;YcylnB`dS}G*+r4HEq(e|YJ>(%+9yuWk}MsO#x z4k(WvWLmkYrr3&gBWYUj{77^SL~n~=Vu%9gS5S!?r16uOsZyYcP$je>sI(Cdj{S5p z5oiqg82c<{BIon4&3aFGG1UbjxC{YN@s90C6ofx^D7`)dW{kP-vbUGG^^b^O zEPEud-coks7tZ<0tQK~N{6-WVqxVK-I1wr?yAI1;9h+A9w9e#6jY##qHPV4dA}6Go z=PtaieOj+O`J48KK5ZSebMJocdEe(w{{EXC_*T=LWR3v_W=s1Y@z(#&Oxg~8W+uJ7 zv9z#1f*I3B+0oiLJLv*Cxlu7hbjbwQ&>#w}zm384#~TT(W}Ty~o(Hb325Ns+Lx$++ zT3TQ)ls}-8lnSm<;RG$guDtkaR_RpPS^2#AepLCoxh9<3a#QuZU+{2zR(xOb{pNd` z+Q9w$dQJsaabpMlM9Z7MUFsi0_Y7+FpQe4T^q)uQF5Q+x$kwoz=+l5^p?_}nzoLE4 z^ItZ3lOBozzKIVh0pH|@Gpv5J@GeI_&sqq;gP?`+%<--fN!EhB;&P6 zuVb}Duu%7iyyw3dH|R(iNp~fn*P!n`*mO50gD-vwI(xB^xWP%ox61Vje3rJ1M?kMo zkWw1<__?{)p|OD%<3<>|J3u;v2N*C)&LJweFR_wtUl=g<8L(^xDuSzkhjeO(x4^fq zkEeEYMmCD|UO=EA9HXP*&M=UY^cEC1s z(EG*bE&IzO?~RjK^CmL3=munfje8v!4iGHf9p^3EC66w94)?1;XpNMDo*8%Az46$n z*tOcdJo_xz+sRoy1Z?N?3 z)tp-==HA5fq!3?Pv^><#V)1xd$X8&05C!S$!$#FMqLK()*VsW!G1bb9?@U1(=@n=L?6U@_c|dyOj;6 zaH!c;THf}em628*T;rZ!_C}h8we1&d>jX0$yOP;;GF#~~WtSm{Pd8>W9-cJS%Up|oeB=hz*2i{$a}Y1LBQY6KOKitcZN#s;UajHA zZzJo_30PZmNPNX;`Emk=Sp4*W9|pJVFcfztrELBQ@K54}eP%azoYV(B1wN7W&bLp7ShMz5(n|KbcaP{i8+)S(FnROqiqtGoxW6qpq8Kh+Ocw=5**U<218O2Pu0^K>oZaJj2#UgF z-vVCkmKq0CFvShPo)n*(-J8f=kiXXz!qP|`1?4xTGU*f~SE=aE5F9{TMce&>fY+Mh zW=IO<{DMO0gKyU`r0!VA6Dgz1wCr3lo5hAL5El5W4WUn+H`yYT2p7@u(IDV8`UFXl zRp2meYJA9NA=DHHX(X8BW3>ha^h6+I{9FVbC%H1WX-{A2@V{yCrx{#H7=B& z4ODKhry$E6ZV_3{^VfuDm`9mu1HJ7j()teu4)6yVCQvnQ0;g821+(z0p+o`9Xxh|! zGZ~R-X7P9}4_$MTtq?XYS7D4lT?wD2v3v`mweAs2F7J}GgHUI^tz!?d5`ITJ9p1LQ z_JEV$5QQduaj25@md7vHX0L;%uQf$PJD@hdAM+#!YS=l&p|j6DYZsP1dj3*fxelXzSSa3jHp$`+k9U@o}sfRa>_>j|1F znA75qJZ!MdT`WDQqxVBOAK0)?%}`P7j4L?fZO#rU^*pu7KZIETP^8;)FcFwH1Ze7_ zeWSSS)ePjr7^rF`PO%43Gtr5A7SFApsrvndQZnkWJ7nb!)Tz$xrj8<Hg z8CY34rB)j<;ceV1i7a+Pm;1(5F$Z!VhqgvWBJ)OJ;4hzB`2`whwY71$=xij!-roVOn5d$|QZv6GO?jrHR6IFOYH({vMQgg;kwcCnLVB#9-aOtXJKx!xMw8P1DXB27hq($nJT;Qu{un39{QUsD)SCx%T{IFAcO_`iT`sadc92$7;kw>X zYYa@x(n^->Cd(N>&gQTK`K&xNZ^Vi>zb~BP`!_A^?$5&qxFKQ%85T{m zJnNvkxw6K7oLZeEv)Xs20R_Zmk`_5N(5v3(Ey#IPQap*Y-X`Ph2EQTm8g(_t{oN;1-i%;wDr z(_I_&(~PiZS-0dKv|-V)QVh+umL(py^zA$4iaef$78dY@a{?o&_%u~I$Vd?skdG@F zd*6g^BE%Ad=95Q)M!K(jCKyVt&o8D*TBc4JashVJIvK(}2$i31lYc z6Q#gZ_x%V+hSz_=jrx#Kl0F(}0rvchS2r@Sm38>cfnxj*3FLp-bkSm9pi(|(2q6U( z1q}w2^htH_udZkc^Lwm+Y3?zi${fb-%hpgKH(GM#qF-$9dZU*`3L}S$E6!XF(3`Yx zkYqcCed{|MrEI<;yVLYW+9zy-<7s%h;tFhQJ17R5WZJt@lNGWj9AnId^Ex+L%AHo( zvKa-s)AV?v!ql058kjo*1M8*I;nJnx3_>fHv3;r{^H{<;esfUDP$8vp=5`^vium$R_J)Hz=Yx z^_6+Mi6J0p@a0>P&JYm~xz_-Bt+INWEEmF8Bt;AJKYB{i`aR{nYrm=Bt=~Fqr&8lK z3#R#ik3y?;(c}JJBKAMv1YLeln=fH%OCn!q*66Izl;S?}(R&-C+zF;d-SB(2vi(Ia zDo`zQ3|pmq5qlo|9|Z(1duAw%)-gp z(SzC88sOv|JO=g&y^6fL!QMw&#LORM? zQSeKhJcbIDe3=0LQ(^vE+}6*%?%lyGvR2%SUGHr?deH%vN5R zxTT5{z;dIDrHfRVS*9h*Fnt#^t&4me zyalsL9pIN8R3Bn!kJI*(88PBGmpdNC;GDTXMn!sGiYq~+w3}NC(Ea^ad5;9> zaxnklTDH05e$aW3-xu)Np{lC5{UuR4H_mO>qU3%j+`0eyb~sM4ydF?q7A!baf z<%sp8Vg5uMbHg_}Ky>DBcX9T=P$!qD??{t)Ltnv^zkGVp++A`j?@uqf_McwV?H%?L zF>9kmD1qgR^NqUWd%9}7c4S>qF@~x&?Ls3=WOJiA794k@w2uW%hRlX*10fG#%(K$V zB8xBdJyxRA+bQ1%_}?gZvxdROC>B0Gx5uweM^jEYt8K5ZIX!O(hQ!ojL4brE;sKvv zreG!fW_)5+@1bkH0bHaX?1Colh2;J`Aa&M4jndXaBP_de-XIHPgsu3)@L;kDOFT)< zf^9?sNqZGB^{-{Em1NO$QJN-&X<=egic;uzb9yPJGQ+T~RFn0bEk$hfFsXDhv#(Mq+sd(71*+pXn>f*cJmsh|NTZa0E9U`ES&1i}>N8h6_y$f?-s zlQW2Ec{C|A{aJH?e%C4-Z)#!#^&Aq~jDw{p_WGA^tDcX%1&-2OvMxK-LE8t5sE-$q zz}K2Ja<}=rXT>%uK7s&EjFJ;@cZ)-yyr`-2{&wN&UtAsf(d~@1S}>&rAh^B)w}-JZ z5P1h~n%!x9qiwP7d@cNlJ52W$l;bpmQb#r4Of=3yAIcnrdLxp-&_@p)2W+9nqSR!P zs&q@&*s+lX$Ie8!gPFBhv*3hU(wCAJMt>p9E)8oJpL)>d(cPG~S=so!cBv^ADQdeC z{k9C(lkjvCn4;!$%&tnZ`>orRpqZ<&5p5QZEfr$*3@<~^RTz1xV96L`4D?iTBWVRxq)oP)C^% zk)*9wg^f^EONg>jZKgU+p;#E&DylBq_PHIhNECJr908xYfQ}C5VhjdXTYv|KdIrW_ zoP$%ljpnV3a}&s8(RqIfVtu0-BPprN_6ZKV=pW(8^a)>Jo28aX%iwu;BMWR;9sAyW-;-waieo=_ew7bf~k zc7g(}M{WKiSM-sge4(E+TO>CcBM+BV&?mCaFNG0X>|P6t927YY=+o%@qN$98s0ai> zt`DDVQVg~{u{9^D35nmDWop^`ExURV!jjWU^+`6eO4`?zTv8E0-blAj z0iOVgDVSR8QKsp>4;5ee8mck ze%^^Uwf&fM@a-mY>DQn}>r~opaZO~92+=O$A5A5>WFkp^mdtVFWX`LKcE+Huli$_V z+`%X>?FC;g>uahZ4Vi&zAl&+As0tXdKUzLZ#lQS#AtN(595o2=k5-mY%@;3SCL}yD zZWXi;Yq2uF@b+J*{?tR@Lv-*YzuQCTPITs)DBl=WzVBk}nSr8udfk8ED>^?8v^`}x zIbC*5zJJ_}Yk^snI#fDVI#fDs9J%-2r0&TOP>e9p(zYf`VsIdsBD2a9&ct%$?zR7f zDn`P9P=T1J8qGfisEO&C!uf*BRNH)Mqk|3* zuDY7m1ruGhfirRkLcwWsZHLcGa#yqngaA^Bj`G5D5l;#UIT*GKhIgytXZ1=Ad@L!=nux8_{uwE5p>BIF7-K*2-2qejY7zo{Ndv#-d81R5Y_xo2C9ia#m z6cbY2BRP)!kO|j3V912j-urt9vr1HgB5j%M69qi~SpfpRY~bk14lFj)hilX8xnxnq&6t#4*?sSm62O`54?eY!VqAV_$xHn^ zF(<~nexwVV?0s*ih$MoVdyvHfTZj##A?z6zKBeX^bjp8GlXc%5R6>28bf*7FP38;< zM-NVsk^>26LlAv<%;osGpkn?Le_*Hrhm#@*1;-2mw?bF1#bCmERd5_RNfVKW$^lYo zKZB7_+1lcormX@}udd#~StZ8Bw)0NJa+d-TrFw~@qaiNki^@|Gp&|rJhWBw2sJW}^ z#St2L=#$oO2}a*U;!J=2HUkomHVsjDc$S*V@A(`S|5 z2Z9SSGAl=IAK*l~nEDKWo4xtJb6&v|SPe24OfUQY7mR|#v;e2}a|qM_IgC#iK#a%w zUt3?`!EX603=9kfjIAq-qALt@N)7`YehLW-5)6WG%UtJv{KT6eWXcXK+!u9X)~fgn zqGDCO-VstiTjCkgT34S|2kS^Ad89}r(%!-Tk>07^iC(ZmK0Iwd@<=3WQU@5#&#HJ> zI3eo9%&54*Y&fdg8#WSLiYcU z)+AvNQ)cnuFjHn=;WP*kpLb>4KU-+}A6sGiTn;vHa83mG-wV+Hd2IMUAN$#hh?7D= zjD`iGqy*U7*;4+mVMN7E{u9axn!eVe)93}U4GpTR>8)({BM$}9!hgbs7J?i&8Zp@m zvyAA-?TVN=|0|M{XJiD0$?bO8_2+Pd!%6t-?=A3iWBtB*e+_6W1OyApSWQvZf-(6; zR#=4rh(oSQQn6e@i3%&Vl=NoMRr>@=8;g7m?p*Qs#2&HR7o0@+UCEpD?cUn8`|NaP zY^H`SYq?Ir#fJqlTtPu{X?5=qbQ31#6_07F`fbNh3@pYT5W2na_D=4hi=ORWZ3nIl zCf*)L>k~&2HPK^lCQQft0p&rZuOl$)6v}eo3Yxp(HqSfWmH*O@^l2H?vk!F^lZj#N zgPLw4xm@;uW552+7pf0Yu*!u+MTI>{sVdN-!n{e>5J7Nrv{~ZWy!hZli zz#qk>jbTr+i)F)ZvU3u1f5atvCFzb+VFp3jMJ6V6)JO{{h~OUJi+%iCxt?lc6{h{^ z>^1-U(hB>(g(^j@J~t>DJRBh?;rG89i9K#-I+_S#Gj!w#+O-Up1xnNuFyehwVv%a_ zFV4%GOt>$`bXg{U!$H69MI$LcRV$_NmEEYazUo%dPG#AG7F8N$+*^72d!ah5Th26}BK2tnHY8juXM9O{*_ z?;{7qZW}XqfrR6%_Wa~%an4x^#DZrsHBHz-z7O$wJ-BzpG?`tYZXNke=n|buZ5|c3 z*?E?WsLwQ)?t;g>9g~pdWzkdG4PQT$JUq<#%2bz4NcT#R#!Oc}S%}(^1}n0EW6iEZ z4ca#l6Zb@$;ElHaxC3=$I|IG%S#2f^ql$Ye;n$`v@7x}6sROp)yy6&|P5g7r2RDQ^ zb~Yb;czI*RKht}z-N8#{ z23e#F!a?4l58O%@a8yP}|WNkBP*2;rz;! za*s+_E87f1kLOpcFj!-N=yOVJh%;wl?vWqE5Er+G^eUHoo|0u$WI9JvbdRfM$AK@w zH+^)CKCe)ips;yL@D=J^xmzo!6~!_H$5URwl@av-vq}80V%~XxJY$Q9AC3a5Nyb{{NQvUQJ%gN2SiN zoBA{L2_Z}4d%asusR6Uyjf<3HuO4Y-0etVbfp??aNU~Sz7lXG_5oe<^l8#qj1>%XlT}{55w5-IDKi$78EW2!2D-#8%I(lpI5NCX&)4Xl{s2@4|0({JzI(lda zT(OM+=PPbr`Z3b52M{lV9uvNE5+EDod52{(8DMH;w~6q-TE`*H3t6tW2sM~mch5$( zx0Zvt-(27CS%oake!Cd7L`1H1182o}59q}buQ`1>tyr|LwoVnRF1-?k=X{g5G*}rw ze#@!sWzUyFiSb)<}Ss zLIRJ5BsDl3Umk$tQD1}E5GfrI8XHLKV+h4m{<2_K6TgtZ_3hYX1h0tXp)rmNAyjgx z6y0)9XkT7=5PcVRa}SmlUm8kks8u+GX#owJj5`x~ z?BpBFKj$RhpGX70&vnZ2pX>Am8IByJZlyLaf+e($6CZp{%K|OFrHY_>1C2nBdM0lc z%Zfwt%v2}`pFqvE$$FjhM#^uDpEi3_Aw1^5qhHtfWyrfV!^`n;!%K<#&%^7w8<<-i z`mcpS0qhoy2?qDv?L2=8=+8n0TKDqp??m)3do%V5Vn{v&I;^F6*vnkB^;d<))~Lr+leqt;cDOTb1E=(4NZfQWRTQAu?; z{eB=r{)C?DWBrBKACcIaa|L}+9npHzs+#{2wg=|=^=)FV)4c16rt(I?M|9hfeWAXr zjR$iUR>1fMs-7vX1wT!_*3XUs%o{xq3qepH>@*fBwRJ#gB*F*>cl+F~Q zcSLus#Fs_(&54j+=T@u#hp~5nj(q#pMLV{gbdrv3+qP|YY^P$|wyloUv27b2cdX8< zz4tlyo%`SSzVT{|8o!Z@wW`*bYt=WgzBwn679ZmT)KV>aXEs&pD$GyOnix+n8}P|h zgC|e`hr#q$9oA$DN!E0^S+#4*pO(-NlWxy0?S0&%oXwJY>P5wAm9e!iEtES%DpH4j zQoD7Xn+y>)Z!SChkVkbbptkhy~Z>oP=K@<`f*zaMM_R&Y3PS+l|Ya zDD(codEHpV@ydK?^oNncnu@}uYZ546w?5ex+xR>tnF_D^G!xYb{2aBD+~*`zME>E- zOWC$WaP(MFOj_E$#|DCitfw~I{R31{)4ciV3ah(x= zG&ee|454DWXt^{5N_6nLDUI+BbQl2QVoegl8K)CPhug>J6(8P;bnj7d#D8(xv&gVcC-;aNgIk*hhTCPZY?|o-H-tVRss_;SDh=!(f~c=?@hRzh(2F_2=i9y(aGI&KHWkl>%d+G#Q~M4Fujn zOPp0_WKB%)pWyln+s$|`!p@`>_1&{j<@`>o@1#<4}fT%H1 zN^Ux#St5N1`%~uAq5?yS-Rfc&irRu>zfv_1w~peUX1tfYEg*eB6YL1M(f^V3YKRK- zj#M_3(S(rsMCe5lAy83J6v1_wI5dCqd_%INBosEr2*=nVKUde>ys#ky8uxvGS;89( zIwESdSIZQ&SAkdmM>}I2Fn-yY`;05+4h!#I0>C@CA;A;pF;}hg=3(2EW7q-x2ZCyYw*8~b%#Q)oR@+7HmmZa~BDkv&rZFAbOV0ES(7-l((?qqb0VrQ;s* z$48FSwj$o)t92YfopIZs(XzAJ2pQo08V?;e^+7U|-8W#f*6w)|{`N`&*o4f1Qe~Ms z(;>LW%<_0Tw{#-2;K-~*O*O{|uvjN!`|=BIVXJUHTvv_CJHN^~R8wMSQO9BhejvvV z3d?n?Ln}?~S7WbH+R;_0Zo)isu-x5{6~7S8UJMShf(P1~xZN&u%nhq2gQccOvw&=` zNV2+3jVh<;I|gJEOBnXEKMS8n+@G{vmO2!_C`l_K$*02w&rj&?l1a%MfPx`fgs%ZG zL5-2>%MO!_8|Y7ibW*CTI>sBY01WQ0A^}K@uQH2naub8d`M$I{e1f!h5Dg^t%4p*! zF30|rHig%6X_FiV`HUh$7 zSPd+6+7{ftD;a*#eo?;*7o4x{q&8_ed7Ry@*FImifJI)tpXvaBH~c;}-ZSCM1Grd- zSl$s-i2#!flMs`X1O5?>uhw6YH~~G!&*X@)&W`tnnj+xW-@}bnMc}d6T!_e~AP^k0 zm-0;{MNlnJPx~Z6b2#e`$mwOA#WJnueaa8m`BTlMM_7l8$9_Gl^83cZ;72`6j*#e@ zeHpmD*R-2hWp8XzqB6TX*D;^Vf$xB;HBpm3Bp`OLv0N2zp|r^=PAxaRZU>Z_wDlJ} zoeXU2964>VE!(nWHGM}Vkc+FKe)Hcm?(lt`jg}9F(LlOIdlnD>tZA3p4QXb`s?tb4 zmG%%GFyfv`svBprqK9D90LxtTo$%$hSX1q1G*ztSUZ^cNiY?wjZ;4XZnb&d|6xUOG zEw^*qZlq4-8A8sL6`m<9j?9S^@d&S)nxnMgmLA(WU@oIoVhJbpHogwC8;s5>=L zL2gaB3J)<_LMXXMlB1{3lOEd#t%r;1(&e^L@gv-^F^LtokO$xNbhK;m7Vk&bT|$1b>5QD8f`e87w@bB>*@3gLiQOXVzQ3&1&q6}vq2A6TpP62K|>y~ zC61HnG+r%w<|cQhP3Q@F>y0mz{SRyz4lXPA9CW=~X0@I1p(>!-RCaDgkU5kW@)KtS zx2~WrMrND_Uc5fcJ+S#2gAO+J+wuj-d1)o7LuJU);d?;sx05pU=}8ca^^@{!KT$x^ zE$WAAm8?i2ShWFqRAzfzfbZ}7eWvZF0Fpn%KUTu)%!Z#Bk!|Mlit`4bdj7QJ_3aYD z47wHlH2G*8@05_KM!nmypVhugj??Mhh7f2;l@Hp53h)^KpkV23IknG>`z($h-BU!BMza&P(XA7c|j`X#`NP_yps8PTb2Nc3r zYj5DBa~La)Bd}+}{T<~}`e^Up0OAnCze3S}fm2#A4m2V0zhYDWzZN?<`YnPwTf6-4 zrApJ8GMXBKbK`=iV}s*(V6Xc7AbwMTrJgSC29acDb^w)%{|a`X{|)AzNHvEx9x{@- z{=2P(;dZzf700S2{ohN?98Y~+11^f@|F~4(VncDD@zbUVk)eS1v_N8#B68^Go_e{O z;B9S06mCFBOlI59He@rkA#jFs5w3WVajJ@%VUAm-#hBp`q+Jmlq!7`f(E5x?*`Ak^ znT)&Jmq$cHENfg$Hd<@Vx%x~YP#pZ==33@0LCginV@7L37|5TrsTX)*?UPQtD@XF* zc#{n_;F@IwIowvN5RlyL*N0Su-mc(GVA8o?ihTV}13P8k=P59S&OYUJxF+{nt{ zVZowH){er$58lq5>@4XgW>vwRumHql6)UQ z(^b@={eNH0{PM9%k^)NI`ro$Gwhb_P)sG8}3sC$Cyc$RLO)8d>EvivMA|?p_s*MEM zt=$F9hk-%7&2iRZ6VZu{hUAx$3yvbt7l&H56^X@gJ=NvewYFAw{iN{4VigVr;uLBX zF&=z1V)tA4+z_OZJ7YHOVA3dc!HL%PqV&uR%$1>s#A&Q~4LzRO{TakhLaG{E`Ysuk z6hQE{sFml^HdbQ~hS-p_kIFW4NHw4RtQ>oA-O%f z;?m~1^@v|*qU+dyIc?I<9a%hQerV8ABpgz_WJ~nez-g?HqKolObo*a7G z=e})v?|5FbJZ-+e&OBU$Ebe--PfK@^I!f=GMQD%$^iw$L9L4w9B38(@<2^L?`67_W z_!BuPZW{(YBj8Z;Bz2RYRrgsUX2^QtpOyD%BI?L`652?aXqku|we}+i%LmmWm?Es{ zCr3cAemlnR`9Us)KyV)R3%j({LqE0Eqvp`7=1+ud%wG0WZ&vlibfV*WYYNf3(TpDC zy=;yHM!;ZU+tfxC2kXJ#Fwl#{ekxxm)x*2fPzEUcbZVxi6w3-^Zl<9e;fJ89*C+^i zhkt5k^8Xezjcg3ff(=hD61*oAN*OeTY{Gdw{LO3212c}1cld^L01ci3JJ-}Ba?dK% zV$%aL6OX~+`mO5_s($ksGY&RcL;a-?gss>M@T9F>hgn??Z8c{p)(4*<-lauT@1!i^ zA_=$JVde4Qz-eB^M>KW5asjJcldGzw9j~@M9FH5*QW;9HSm@S-TQ6rFpF(3!l@vZt zQ-ugqU3rhM6j!kw|NY)`!#g&rsASv^&SgZeTW*NRRm_bS0&nNGaT;bh52O9`GY zU?J6wG9Y@Ke%V6Zjf1mcE<#S;#?3ei?a5(NckKh9qMz^|!CT2^dNK_~S&}HU8*6%eU8nNO7->ycz_ni$fxESK$YZ+=-;}j6VqAS4-Vw-CNEN$zr1H9KA zrV4|9qasl2X(^UeduidF!&!&s)bsGE^sxq(V#LntKU#&Fp9R2JDE}yRs?1BbN;AvA zl;dO3qgD+rZ59db|)ceQr6^S(JWrem(7wAn785irxdl?k;m1p~jGOhJ49f0atvjA-f zAEa=JjxUO;yZw&fL^sGE)WxHOT(R#=HeX{uN6UynBou`c$TD(i$~#060>WUL_Dzwd z9x!9M=VO|wo7#tf$W0K|*(4ZUa12#AzaNAi+2?7x52}C2Mf8W*;g)PDqbQn`SK<*` z-MiL!;N~7fjH+Ts2p~)s2xvCue)zb2fmsttzkWekTd5y&N9jdlv4<8LoV3OtNu2&+ zQcm^|*2gPQfx?gKZX=wKn4jdQ1#)ln%jIC+H-46%7N`dO?}+=Z z`%S1mn70!dJ5ulb`(}Y@gK5kL3h$WvDd2U;2Gc$yw?~7Ws9jOd7=TF*lxukRRrdco|xcs(((F@_-IUm*}6rf{)0 zacSFJu{Ble5A#E;5!Gki>I13}?WkA##Q;#Pju{2N%tFw)g3jE%;3xdb0Fyhv?fMrq zwk!rrP{!Umo^=o4LR)s;>=h=i6^L_huJm*9F_M*&e+6;I9FahtJGzl#V^3K=9;m`c z&M6*3`J_XqWLV6zjo-x5GcGO9mKt z-NAqoOS~|1lz!UG5jUnipd}Npe9OLlDF)g+*Vwp6jvOc?3CdeJIx)2Z?vK2=<`u0| z^^Uo5jid$FLTDX40}+#+q~5G)6SO1j8KwzQfEQ<*DU>MpA zNQBdh2m*>ZA0(K_wY$<4jQK*4-5dkgwITTQx~_#YH`!;SXVMUqhUQ_)vmbH?gESOfu9E|W+^5}ep zf$tt+BwN{tY&zP4m`<(I0gKRA41NJ<&6s$?VDiAO7NrSmC^dEDU)eOzK7Ttq70bsC@Sp$dh5jXt7X%tQ&6frmDy^3q8VSe%n4yOD!=*6gl?eh4`#+EufD9wd zzb0S;$0pN22jjM-B-2Fz$94kW|KY3t_gJVj1=_zq+<+E30=L}xaVQozc=Z2k0wLYs z6By{A<8TKh2D_7igO~n2VU-S=DvhoI9vZ~$9~K7*J+u^Vk>?!0J#cXNzbAy!Lo=F_ z{Kd4u1bWGCknG}+3Dtv*q92n!AG?c%lY`llm4n6Es^X9!^$zmJ`W-z0W%ch)FeE{W zj!wq%#F5rdG*Ca;(r6e`blEbLr;-~b%)jzCI{hN zW78}`+Sd)nD)2b8P1UWT z_zVR$%)PYgoSD1^7!gj$gPLvBI0w!GjhuQ%Tz#9wMaPUb9uc~VRWOYMhmPjXKnQ4 zJ_GB`c$UyGAd9zh`j^1jgQ>$fl>^8A^9&XfjGMtbLxXN+@UT3f2#o}Jj>wZ6ERTU+JT zsjuwOSWI2NQMgXEg-!|j$*ghUY8OcWwtGW5Huy!W=_R#62In1&8d4hraMqdUHMN(Z zFcJ%7O+eOFyRIa<-)Bxl>A>4yX`I%e7Ofy{q2HR6Wkecf%Aq*rs2{L019Nje9_AbRj)d zNJJ^Q&2l-`X%m)%W3p<9k@UFLh2y;gy;a+rr1d=}K{_chJ>(fx#*30VLmJ&4`C?jKVVBK_4>$?ZRZJ7>;Q*y7UAygGXr(j94v^T4KQK7 z(I4o$5iq>?f|jfJIuSK)j44Bk$Q#^niV7D|TW7>>X`RRn3rKM}rh=TGVR|)*Lx6|V zKSKHh`%8nkx{sCr0EXfcq5cJLw>xl{02LP}D?1Ajdpj3150`&PQRc38q|X1%jIvGW zlm`_8Cb{jkG}1d^)QLtd`QyU0;GzX72B4RRY}Ln_jxx)nY8COPXj>W^Rr^vN2w4hd zIx`#e0H#f+pJqlV{7KN^tFblN>dg(e`ozB)rIV+Kp(zd0ks4`@0*FnF&}88EjL$NS zT31hRz4hJNf2Hc=6x2fXomJ;VBPMgou%)}tT~sgj{UX`p8OuEU_s>3GQ^~2G03w!l zgXhG~-&VYA9QKq0j{sg9=c9^F!8MW2{(3g`cR{N)tfQuB#kbA52SF}vuSj^Ax4?5; zPcSA?2$WaHP~=OpWNDH(@Em2Abt25t+|^G|>mFja8_0uQ49<(2mawV+*meobXCz`kg$DgUlsP2yKgt{!7X8n?_{aY*QP*)mGeiILM=n#Y z1#dOgNsFG^w!ms04uVR1yvAxyQYImF*%G5=3^ZVB$R%hNYQ!=7b_mMBXaA>?M|dm) zMu8D6D453p4>xzD`_T}Nuba!aTmTxZ?L0h-`EmBgr}GD}hO&jz%kT!k0;F10QZz5z z4~0h*J{&sILv_%EI8>CEejf%{%=5yj0DeLbS?MGYjsT04wNo(`gMQR9u}_gsNH7W< zPhd?E*a5vs4Ygg(dAU!E#Da#5vq82{kv~FsNfNzMy-;z9nY|#T>5WTgV$BN&X%A$Y z;E?v43C*&@GIYxvt$$}RL&Qh^a$(3A?HVfpVx_v-~j z%c)SzjfG7Ma*_r=$*hYL(;$eJ$gm|6m##ASxMIVm0_k6uZ zH*2qRYoQ1K)VX}Kvos%^)y~t|IJV>qpmc7iro8VO++mP!i3r-;s)&jH6tr zacmNWbID^xCpF$^0Fq7Ri?hqkxh=JcZx2H%WAQy){UB{`vUJV85;BJ%QyGbcW0CR@ zRuRDjaA>UD*kRwLN=C#f@hU=N92#3f3i`RRi^7C&Es4#3>6)(~@QvroGB`z2rX?8t zf$t_$Y-RoMPP#2~Qp67>*f6;&nI2IgH#>ycQsI{Pn0k^^0En}aFlRbfQ!mRx+A}$R zbMyZ?pjRw&ss|?U2@Hz$oMuys6}^f?FUsR%r2tA{a&pXQ8Ru&_B zWC$s`EsRS^Qt1QxRv|z zMbZfi={1s8l94nq%tKrVR}a>$CNc(}+OBg)l;b`o`i-PnA^b9?I58%FI5DG@zJuru z8j!EbX9Eh1a`LJCpx_)*VY$N~v&+u>knnE!y>39<_C=`onRACEI0NFO-e9Z*hA@1F zMSZ3#Zy~JvF))9oU2ypfAs*m8a`%{(-Qsr!S6%4&Fi^IUZ-(rI;J%Z4+*Hkm8w963 zYiS1M1-rdd+Vmp@5^99_y%PoBlpzlpoRUYcC;<}47>|fWQ9wWEMJ0)QCzgdk{|u$l zI`1$Wg^+8hrobl*@uMbUa?zF3Gr_r2QP|CE1vY{g2X_immzpx2mnzvs$OaqGnbsc7;Me1u#@PN32D{9c;1Dk2R z<%Ripq35Qd7A{-BisXv)A0&FhhA`%vx{A4t@_9tP)%^9B$zM=%B@#J z%k`H+Gxbz0%xt*yuqf8vG|HjUvCFQ-^=+0#X?sZLE_ptWpDQnoX03n$Zr7oU?Sh# z^pfI{Pon~*s5mPY%ePR~s8J+S3mfx=jky}2SQ1!bC$W#}*6s4Ym-q<^RQ|%x5>sSf z{f(37?K|W9a@}hH_~J7d5K|iCikAxr(Sx&vv!}h_(iETi35$zJTa8v=SII@ZIfpqL-m$OWav7Z#pd;A7l)$uATf#i zP995M>@e|SSk9j7DB3pM0x1!_%2L*0W*n*`gLm=-DRn=``%t!7kC?aN*QFkSsxrV> zlb|@kupFrb)!_G-q^W@%B;*P0RFh)1q{#Bbs*p6q%>7#-J?h>NNWUo2Ga4SmEPNhzX5B{|2c0=|_=U^O6P>U5PFhd0cep6L0Usw} zgdnhWes(~N0joI5K@zQ_UDD}RA58evT99Pgd2Yv_MAFewNE|Xq zH+nq1zc`xKZnQ0k3CG9ao0MgB~;fO=E+igte__Mz<2PJDnH|CqIzm&QJ27rqY1S_$0yN`NP0R5 zo5YUUDF8r*K61zsy1KiIu3sEAYNuDw1v~>}=kQ%SMrS`W%y3K6_YnOiX@}A4-cmH@ znvHRfY*OU(8kU5s_qoLcy5ECth@v4Cxt`%t?GhM)9p=cb?B>ppI#7h0{Sr2XNm~P^ zbDJ4cvZ7(F&Ujeu_u+4oHpM*d$%u7sPPBdSEPdC`Gdr>81IiV*z+1l^y%q+@bxi^4 zBNq6(0^0&CwWA?9gnTq+@m-%x4MxV+$~B|(CEhN6J=!a+yZFdiz1$FueItXLGzvXL ze&66!(7+ZQB_<}`5?K{z*wWO+DA+-HqH%N`R~ulv)nmbQ7%JM9r+GTJH{wO#K>P0R zPfhRHmqaj2QN8pN!*OYVFP~;v!nzRf(T>-M(V16E2i^XN#n};yhG*%WQT%%e^{WF6 zL5Mw`t*K$QAPM%Z3kw6$B?R{+8TuON$vRe42?%TKOmlZeIHA%W%C9`+=qdLrc_~PT>J#hF6oCLDr731g|%Pzg))FxeJ6* zAe}iI=6`b;!4xq?(*E{SR9zix%$P}SO{AnqogMx`LfiNqnSMp|k=~Vxg&*@3Ye+Tl z!R&;nzI0wNfdQuR_tehYrp#rNQ@xd8fyJi<0{H^e-0r|;dq3NG@~)qhg;9(oK!Pd-h1JDCG9yWnRaHgHiAKJi0Yb+V=VPWpL5jB?bH|Qc4~+T0 z@mcSpsu7eG*x&I;kbY;#ZtwEtYNcB-mb}-2U-PTtZh*O|7 z-^MP&As0wjXU86Pg_>N{9Ib0lL%*Q?ruksGP?44BlDrzwbO&B7H~7D=+s=X~dI%Ra z7Qa==6yrtve!Ig|QICb>eg4~#2kP06zb@7_L6QHrZTV|gE?A&R0IGkp=ltvvr|kBB zWB$ZNF`!;T?o+M&0nq?!j0YDA+hbVcB891`=XTI|Zuki!kZ)Ex0JTvec^Z7Ooe4quP69>h1=FjmzTh{BFWE!|dWj7iv#q3oFC&f&=sHbHG?B0Gd1 zUG)r`SKbx{M{VO%T^OmL#=8p!g{4+)tSWd2zTe(cC};twwf~*_s7RBZ`9qbGZgXMc zN|J>4fIIYur~}f^Dzm9YShEe*i0`ZF1*gAakJqQe>pTZqTOl5gA|G(YCYI7%$X&a= z>b))aHNs*UtYg*5j}pwOtMbYz3?9@m^(QXI#dgtO+9|uZIxuFIFlG5Bf2Lccr!Ypg zI?xW1q;uc`(EHI@a>qjSWqI6&;JWcGL3cEv=OjKs0uH$bm^bf0BOE&m;hN=eQwfT# zQwIt+3Z>`JB6)I;aB0wad-d+BwY<(soINXD&R!2i>jb@qAF~_%)(mS>aWrUl^5!cG z+?sbM5m342sHJxu)UyOOZLWcml-pkl!>z-A_pWY-yPo!Bn|D5`3x9;oGm8Sg=g~eSX}=fxcGq1Wvo*_ z5qm{HJ)YeKWJf}^hjEvp)|%WOhILcxCM~wVaF3YrLB(R_{mL_P^|c3;a1*EoEnzE7 zO?dAul}9`d=hh<;Y4>Tt`O8|tnqm1<*vDXH0f}<2qx$=Fdp@I`ahxCiEH4#T*H)Lp zH;herf4XPlv(LB1PcXMVBfEPM-{E1;nCM0KctrK#*P;1U8+4mry^Rb0WW%`O57dp5YH@Z1#f3QXQnuc8mM>!1g&?_qA>7+naQCsZkCbWudS{TWo6y&>iT5`c(G3K2gz`v;*IvMGY@LMLn*$E+cD?{Kp| zSq}1hkrq`vPfm(Y_ha>7pp?N4V~{emG<>2dw+xR#6Nvyv^>a^?Dsc}YRRhk><|TV? z8Kn~jAOWG{ihiBvPXICxC;ZSg2M$$65=xgTGf#X!nxDtO1??en)osf3G`_rDpUE@c zC6;LPFrMA6u4XcckrKN6Av>bz(&4Ke4BqON(uGdKMea&{-LCQ|zYxxzZu8MwUOu0| zO|a7wx}Jcj59?)_{qf)I!RZFlsBWPC+5O-3&wl|O9)RV)ZSQ{okABRY!_jjLbR8r9 zRT%+9R;Y?CoH&w>4tslWxM!A3%aD!KYE|`jhOpP3Js5d+(QzygUF1V|ZZb9j=`v_JV`VkRC=}<+JS233ZPmpdp{Q zu3x>O*jQ^w8a;+{pfF6#9(~G#A{1JFD1ToUUMbtuF?C1~o~c-~3r_^ip0ywk!j`ot zZfKY2YrhpH45k!5hhxML8$RIy3BF2OpRw);tImSWR6X0kGOf7lF*sGOeX2S-o_llC&_JGlK5JCf{?M#9OD$3>i%*+_T+oN@+X3GL%wOy!o0)A& zj<%Ijm-wXQRB-r0^{rqU+|MphAh%zj<3B|TK6_Ew!?7xGifu3~ex;DMhx;y^Mp~yD zRof(ew7VqxgHs|(rMXVt+-@@$n}-ZJi%yM(>#cM^H>RmJrBlEvh0NWwR-1nH1l(nZ z^Sl9;l@`7p+f=A?E(F6GU^|4wXq+9|+RK35dp;jfBKIJ8sKg8id{jNwwL|B;o zJ^YbS%Nm$PnO#jd()Gcp;>sdZRPvpvm;I5^BCS0Y^cGA+=6XlK#yem8%ZPN!QLANd^OTf$ zoCHmoy$R6j+}DIS!RvkRKw8dpc%3TM%RgY7Dtql9+sU(od{SDQR=A8)_cg?*bTfwm zb3qWF6HH5sMTYOGI@M--F8zHUUFJG?OST=<2i43itsQR{Zj*@q9B(=(vOYS~AAkRr zi7iqrk(h)ZCzW*r03h7!=2M6%>_ZT*#adYI4C^S&JzDQLXCr$E|7xq-CO^ul08A8* zU~UUh*djNJv@&-~m6#$&C?dODkALeht(X2y(aZ}I?#?~SMsUcDcKRK>eI9{ckK!HK z@ef31qr#j~xUGv5V{vr6{m8jZ&?O}5#_9TuPjI?(=OjKeAUqxg5!MLhT4v;6p6QiD z`^*wgiqPapn$1-Xl+Yrb;gHw)VF7t_RkG@%%40m?_M;DbU%D;4-YVD z%Qj~{>~4m+^jIgyNy-QpPVWhL01{hQK-5RC5Q6QurBB#(?#wp>VU~dz2N$p$a4P;{ zMpL_Q>-M3o07p~&nf4A@Oa+NXtD)#;n}O2(g&YoyxVj zvi<(}q#yu>-RFyNZS|}V_*{apG5ll|1Rjhg6z-hKi z9jY0PJ-Xi!KxL{KjhAk2omqp^V!oPbW}RA-)n@!HOa{`avhG}JXP2roAQA${mhtk(m-kPi>}vd~QBjs$$#F@F$|K>J zoC>!Gk-veD?>hNw+AjhedMulOWu`LIw~DO&nuYH|W@m|qOWI?usgy$RZbj;u zg=q(@wXqfq3UM)CvBKq2t(Urdhm2~ErB=pw@LH)8dAQZ8&@Y=nL+!JFYeE2o-Q}CG z4IM)v+E3jvYmUSO#>>?zR!ai?Q(L7;o~+dq7_%N41+(lOnx1dxYkk5W)g+qJOfNwY z$Q`O1WZ)jZw;Y?FFr-+Q^#30HzFq4e46*O+m9f0D6~vS8^(Aa#DWqAg;dhkVx|XM) zS>^g!!*AhblTur3jNW*xs@OMxV7^#SEczu`=b(3@V0U@4UWBabrKd$o>4hb)K}-RYO{I-SRlvwjCc)rvk9c}d z-;YI)6ZhiIT>y5cq(W#e%l$o{2UHBrCLHAizQL)DwKAhh^ zM6t~tGy-z7*a{b2zVc+aco90X-ZFDOJx*KVD~fF#sBs>mOAJ)|G^OC0IpkS2s@u21`LK zv`4$!<-$vi|3Em)l;ssjB6QWtxLH4kY~TC@kr~u+cz0>HKDCK|=+`;@Rhvwoy5^aB z=`BXn=#|Yf;VuwV{^A5cJD;n?Du6_9?P<5@uE$cp|kvILdb|za9>vC?O|6j>MxgA zoysQ`3K1%d&M;-7$aNY@XGxfI!dY7UmHjQ$WP5&>g;Unp91=!A%|=xEZ)qIn80Kj1Lf)rB^t( z=EVMl6SgV93w1|SJH5X{>O~?&)lodO{1{;1o!A^Un7;c`Q)Ck5wcl&ro&Ch!l!<%u z%F8A8tPd!>41{(}(jPKgYX)lugxKe*UPG=cbNb>XSNuUBRW>&TyQ~=TIC$d&Ughi^ z+5RT{D^zhP`q~Er49s5xJ750?c?akd;{n+ILC>21&MZsgy&%^D)hfJ<3|Frt^1p3Z z|F%rg+#H}NWc#H`rm^q4q_HISW!kAVzu<)I2v&>AjaBKAM$WC8fTX~GnTn9B+B^%V zxz3N9&VEmu&e!_-pAYXAUq%|L_p2kLBb6{X(9zK~VUN)#X*2r^*_e+-rPn9Xh=W z)qlv`9bOSXDR0weR`|4Q7Atn@O2;f#S8CB{K;a)F1Z+hl9A6|TPi3}xc&sc3OsyqG zkYATTG<`pg;A&%Oc9-6DaK6#>b$lcwnIY$W=Q%g1UYP`}6_ROqHrQdoHEP>m&>?+( z9_>BaW@xnJes~El^Xt=L{rW2Vw$eNBvW$S*eSZ%JKC@2N=_cO?sL-tkUuSk3bIRfx zo62A7Vy~3xuX<3P(l1WE-T2{Um8kD!sf#nE-;;}Tg#xXRe7@A>I%KYXE;=>2gU3db zh@V}p5xESY0-0iqu584&HmMKNTr@PC2~sLfSC0h z^=)QxQA5+B@fl8?w(}#H^qLcOc}T~3dVOV!R&%c#hp=-_%q8c&-A<_BiB2rYL#w*m z!5KjXU$th)?d1~ngn5IV*@;7R?IN=F>FWZr|8y(9-ZJ-&Yiw?!m-a6l2?>+|J$RvS zGWOm8Nw|5sUnGg32c$u<792xc17tW)2tM~nZ3$m5-*fLl+5`mnaJ)j#xFlw*ykdI< z8CPD=uWL+Z&@TRzeBvO&@RC0iRC(#T&Z-f{@58oC`sSEY1mc|*fH>JIZn5?X%ZSJO zseawMX>6l!4;KWQl>(2N64NZK#~9p4IF;Uj5P9!VH;<5MkI><6;j_9Jc)B(jiDQVu zc8b&aA@0XpRH-xml~}elucWOx(~&hhqK4S3OBP(#gZ<;^oJP*Pt0(H3@H^%ke|Y$h za6OJB1;;*`clKI#x{eU6ttZ$TeJHwK&u^_sPu?LrR`I@$`z=a+_G3NvL)OD}f)7Z5 z8sc5N^-kj4r*2W}j+89(=0wRiXOqxR9?Cyz*#yrRk@grB0UC-Lv+Oshm3x55?y6Op zTK{v{`YLCP0+4}TL=&;}P1n$&`JcMnhTwYZ@hA?C6fyn-)*PVl@Z5dC0!+m$ee%Mg z2#sIKvC3Ig6sp>rjmr^A=MAY7D;QM(DY9bB80M9UQfrfgEc_6Y7m{JlXYnX@j3Nl; zlFTBL@G}4PAUETYjZ#2kIk2Kji^bLs!m(l`NyITKCC@+^i)K_wk{x$TZ!o3DcYTae z^4oz0{UgyMk;M^sx3y4f{fv))HTCVWfaj4ScvX<2a5^Xdo9O3J$)P>~Ewa{rw)-@; z_vMzx0ym3fsy$XFW`yIk`NP}X`3f-Qlk{enQKKV{+s-W4<_O6D=(5~HvV_kYNBRZ- zz$EDfXtYPT9Q(ue>$wm2zIe|-L}DJgnkd&N@CmSbn*$@u-RW>={}&J|zEo+d0)ZF` z{(k~7c>G^ZJ23Z$2taCL53FEE>R{vQ{9lw`dz<%|75Ch!1< z6d*nv2}d4W9YPejKD-Yg!j{p?Y-}eU_W*lEdn?C=1bA94+P>+Q|B^?~B$x1{3&G52 z)}~oYUN)sNvYyA&;NOzSA0yr(Hb1=&lI1u~Exw7YVgA+K{e1Z%o3YAIchS zW>i?~(oFm|u-A*^AWx3{V#0YUi~Jjy%Er1@O7E!S5BRBLVts^L$I?yoRppX$&obuE zDOh)k3RM)3T>pscZ$bSdvww7$uL7R}un#hC z8IinjeuykQG?AG#a?#yA+JrD2GXliSY_nqvM`x)qBwqdEX!MTjv_5138q zxXhJN+liP*c2;Cn#!is2j!vv;QOoU4uN@1oS96>0vp3gx8e%n(Jo)ANy1gvi(qbpS zNi4$Ksz|Ba#{yfU+->B zeS=HPN|-(z;@LIXoCgHpJ>chjI~AEK7{^UT?B@K<-!&>E`*x-OFXAGK0G=)(V_?B( zBNG=X7c-~-w@6}+x{k)8I{F`GQ)prI`fo?2mL#!7oYNTSKaMJ!b+zr(YicZ*p`gtZ znZ^{)>`w_#We8vQqmXO~wNFJf^4{=XWnL9KW{{D&EwI&BsvmMp^Xvit4`c5b9SPTN zi*}q;%#PEsZQHhO+o&WR+qTiMZQHiZPSR1|eDB`poO}0s?-=(-{j5=AJ!{pRYdx5A zEtQ%3&qMBxA5>g_N=Ro2EBpmE8<0fQU{+UOVC7%3k zT}&xxj-lAd_twe-PBIst;KrEmm~(w`i-v&R7ZGt}N<6yji?5p_g8XI*Bd|PgjAG4n zXSW8;Ww3h2uAVlfHoYN7W7OV>8MAnB^fGhO3}&IpwqCP3F%715a)GGE6SGH^06 z#PggItShK*tNBx^bc$A{dQuJxkdeAsZ+>qn?UAhuNc-aylPr`>mdL4GGd`*2FwBDq z1JIIbY3#a;@x#LBP|3Q!KR_&>^ZnxXMpUGfvm;>pK=GC0G|v@6v!Ap5qEIzo^#e9g(GFs z>=UU8C%<@o1rIj|d0~pfjY3O+pHNGSJ(MleJ#|zF@X$VdUE;#$eQcACjw0>m1OGYH z%l^6`)x_z1O3J~#Fx%<|k)gfNu!L85|K2y=(;YmPTlKf7oNvSP$}FANlXH1>-!~#} z@ctq}WtO{LUbuO6(x7~llao-aNIQyTyE1Nz6w|thSfT50me9#1FVWqfcvUHhjbMkzkk>1g&2f*hDg54VyCxEY?BPVJxjgezS;8r+prCEp6;^0?_F@; zfz8uoX{(+(a?L z-3*}MJ{hv2@TnX~5iA3DsR)({&j!4Td{+G%3H%=aC`Pcqb5ltGS zelPRC9svw~Xp9x1F!Be^zC+j162dWfX6M@Y#pl{gGt@~bLN)-=CKx9VO=a{nBUD?v z(NsBed&J^dppJR$UDFRb_qcO=@kH}Lc0$Yx3}6ba^(epmI@=*YgJJs0)-(hA60Zh zA35}kJ4;oYL335bpKq#)v6-rMFz)3AAUCR_#NCwNK=c_5&q~a&N_qqH;+q1+cr2{W z{CFvL5rXwUFT`iP$ z&@#tgm!2iFWQSWxRs{r+!DKMZ!@oVcv`ouXnpWbSrSTPwwErd!(jXP(K$!KCXx=uq z`q`d7_nPn;;G#mls)a=pFJ(##$?8JJWHa5qrg)fCU2LYr6|kNxOJdBmfsqBD;vOsH zHCmDi-3lDQ!`&%<`7$1iu(m-B618iKe?xB-LQu`GETvyuPY1()D29PprWy%i@JvWoUT z6+!+og#7L}S3KkI5>f{Ol{x=w03J;1k*aE^+2xOjNrNSI{5%c;5TQy_w2RX07p@ZT!o}x<6{>P{LFmJti&2ggrAv)&MX! zX6`6eFJS<*Vp!YJQ%U|=JCr5m-6Baa^moYME(2+<|Hko zRk`H~{rFF2tC)nt$qj1Ly(CZ*&8OB42}da|u#LSf$lz4GYPzzLY+tO2#R9dF!x6Wk zsAu8zWvmq;!e~%TD=QkvMHL`i7^o5uLxCp~K{I3lQo`ca1`25_EP_kPQvQahJ`^@G;N)&t?X-3o5w^-iY+!f_XLo`*BG(s(bjG` zL%VR{WCL3pyJc95(*2!u%a&=#S&0+SQaQgLGhCFfQjFoUQu!Eq#{D zqy{4U2MDZ{RmW4Tv1n1XAKDBxOBY;EZn$~z)B@$fA7U}Q9Q^tu&x*9tX-8W}-|VXm zN*x}=F5))w7-?RuY5X;+F|gpl1o6(P%P|6mV+#(MEPSu!I!Jb?JBmQmjFjwhOSmfA z>-bIL0Id4Fh~Gd;8k(9J1j8oU&F~%jBQ4!uzf=BxRT1_QW0o%{);z@B>z7fXTRNyW z$sM;9S>(#L<5o7W%WH0B%J?(9J#Yggg3Zz1*2j{ds5kG?v<6CEz%}Hp)s)o1X$t} zOc7v5KE|uu5f7eQ&WOk2I?2WsM1w_m zJh%fniKoCHisX)S6xcV3V&^(ZhqVFELeSi54j}6Y7SvGlw=XH2shZ ze4&mLk9x|)7kI87D<+Ibu(jo6^Jn0tui)oi)>o}jg-lA^5g$64f*oufG1&s}qHSv3 zTW%{G64>XC4TXD5fsX9j>I-KgP0NahwDed3rJF$E0800xLgG}dKFP+Y4+W(0qmKf1 z2-wk+C6CNP%jAQGQZu+2N}D>;@dk7cSL!O#jhqG#+?WNvhsLu`K4x1Lj?PbeKm zh^fDZGz^Alw854{x5^!Z-4FltBRX*Fe;1wl%J5U+|0`L-9}kC-`nCZJ`@d5FvWEZN zk(jLh#~5`4{d0NQ%9Z8?+5mRn;3w!Xlu$Q#V961~U^rp~yvpOa6!uzVv94#6)8!*y z!aUpA7D=P*O-f@iZGL$00b3ltWFo%h->j$5dG^XChUL*>{!{I%$>;62?!%ZW9780DgtLrVuq08cGUE4#HoW;OgkVC|;@pcPKq% z`(zr*eguT+g+1yOmfZ~GlY2lfqUOT z4M4yhdA_0F9Nc1tn)r9hK+Hykg0;rZ*!4+#)BbbC2F!xagD3^HI6|$$Jc$D+NZhyJ z5diul!+)?H)cwF__-cF+WZCP&op#`uAne-%4SjZg`4)=kJgcJ9j~faaI>O1*x0i^^ zH?`+X)g4vJ$uqDgP1PMxs^WwCK$SDp7;$oog+6m2iJod$`GV11w!bl}>JvHutFe1a ziM}%+4pj9aAW**pLsNYR*F@(Z5JgwNV?^&6Bv;Wn_}tz%vnMXxuSrwALonm;Q5)iC zpv~`g-U$n$&3+T3w|dYbsl-}{4?h%5sYk3K$Z4eAN-NpJW6_bY)TB*dYS1h=FG1!U ziejYg>ANI3pyL*4AZBZ5G)aj*Z;0ZsW#w$P1kU+ru<)B&K1@Zlu^e!%vlqzLw++a7&2V}M29`Pp?OdMyl51jk z>j)Zj>J8^X6e}Xei4&>F;_Yk*sv>%DW*w%#%~Ff)n3KbRCtn7+v%0=NO+kr$G&tdU z1D36Jiw{=n2EW~ecbAkkL&HgIe`K@%y#3iQ$D;YXl;tL+n{~hzFjoQLj=g4}cj+i$ zP*-w4kt0gpC2Y{z+}oBVjpKMt)p8E~%ybV_alWZQtF&14ifQBI7+|`HZ0u~?CT`E) zK%AT7u`5Xr3Q^oLI2h7N&{XE}4>D7v1ll=eM`uF9F8D?|@W04}aw|9fwlNW-o{ZZZ z)tDPcuL?b@&UH?z3r95<4;@$}8nFO4rX~ibZF}G7Byo`}_&^^Na*St|3Zo61d*z zlVZvVlUWY79=W$Fz*14yE$j}-6a`JQtB@k!(RaLjflF8pV`OT~$jU4Y zoOu9WPlB-&b$)xQe#mX0af8`x4k$6QF+uz|OZX?`vk-vj~O$kz%l9M_jtbuXVJKUOHmIM*RKW#_V3-#BznnONIF7X7X!RmN3O z+oZ}?QBfng`iGwjPSrVJ3V3I0=-fYla!{8vjDRPjW=9RRtvs7+;IkO-yl#Z$8}Kxd zGY30DT_9?wy{vdGVo|TIJtt7YzYLTpblYo~T9PB9_24>X<%72Jcr2ciQun$eedpO~ zT2L*hyy3d$2ZHIYoc09KBY9=taqMZMKe?sqk{^BmN+vFf5Tm);foyKk)p?|IT~Wj+ zQXD_(Um0mu&iyZZl~kM^#a{&3~Po$c)IK=BV2Sbq==shoPXS9XwJ zOGeQjzHqsbYzKMq9r}W0*5r{M>h@1hSU`KG2+GpZ zg&tZ{E1(eC!|Y`W0YMx`Tk0lYBzB8)3DOjH-@YepJf0nidY0g~${r&inqqi*Vl-F2 zq8oWkRE(e+cg38x>L62efF9NN;uGPL1)cz!LrmOF>pPg_N0rrRS5$l>D(wzH(90qc z8TgA-m--^C_|>;WH1#?lZ*-+S5MZxS@bfsk*S4##aZX#QfU8EYVNPuwTm8`Y{i_+L zbcQ`;WPSm^M)T5fT-0$`_D<4V;oaIrixiN%!~;j)j-;NT&a~6iLYuGa&^=J7Eo76B zY^$|}^btQ&X7z>MWCeL%Ra|rxq_TNLEx^VkDdrlY3Q_C~m0`df9sn zs-bZwrj2t&KnrA`__ixHsj7~8SdAABKmW+fpYSNx#AgpYFOlQNrHc&2M51o#E*CT} z;ha|b>cK8YPpq4i+Q2u9=)+^A@Oa_vIFx(ZgFKp171Yl_td%1K_Nw1g2^7UQDq*&C zir4J{TSVuvagOEQTm6=&*j+JD7V(TQ(wJzfv#|SC+8e4vwn?|_n5tSSwKY<(ian9B zDjmB0ddSc4hv0VHqKo{n=+Nc%Y5;d*sy)(4RYmF5hXnsu){{rz@1Fmpz~cO~WU{}e z)OEhR;{QiUZ0h*mjZx}nK-3W|Ked$3=1xor27@S06cDi#VxOOh^Fi8GHsq@U zvKvWFEGggBI*gi}WW0*py$-uAB{Omw%QsNio2q6@9_=hM4z;@v>BmAjZaVa+t67;E z`TxF8PERu5yiI#gT2upXUva*1-Nr-!2`XROBJ|5%nj&_}S_}47(HE%vD)(8@dz2Xq zjnsy+A_^i_RCQGEywQOQcVe{t914+392$|@4rLfE`-x2&p#z2#nDE5@%y3?qsvN^{ z1``P^RPv|MZ{0xi=>Dh>X-O<|P|3+A?X0Y4kHH*&ww*UD?RBSOmUwvG?g0m};0fmt03kL7KwSE$l-9Leep8 z*gYSLLQB!XwNWnDe=pKF8udGg4jxm#rhF}qtIyxD4_UxL#A~bNN7d|24bH(%EDk@07-rQnPKqIa} zkdww>+Wf5`RF}`bSaJUJ6Y`=n*UNLPiFR=j=EYix! zp>0kKzu=sw+aDf;xNjlem*i$nGtNzP>@W#B2$b4MLn6loKD3T*9D_?QI>lfzlO=XP zSvd*zk)T%)b9iz(zt@P2eJ)LmrwD0bJbQ~hR2Pr1Jr(HM;HcaE9_AJHAFs^$={31FU`VqEEFcj?B*JmuK= zHg;qt#Ep0>x39LiH!kAgSli!RZI>FryK34Mx#ylB6Pm|0xhSrA+S9@B?xraO14SFH8@bs!M=P=?!=KOYPe$vs|?dUaoKt&;;m~)wqgo zLYkL1&TShrhb&;){*6+zb`o5$k~d!VSm*`A+Pk<6Ps$kZENS4x#s-n>w7sNnQ4`+M}qJu`D&xij?d`~#ZdccF*J z`trbnd5+^V%>jf$J=$kBv8Ksx(J?&LFB~3lw#)`09`Xez``ouzJQ7xh0K%4DbY)qyqfm9b@#sJ z;UB<6um}rS0|eRc`K&>K=3YXnCp*^AV^te5y*$g_0O-I-zSXgqyUBf^7ssDr9zlEI zoHqW9&IoPsk~%+(s*fu5Ix7;k$ZSy;$XP9L-3Vf^#cgzfOIDlCkk@VnV+%;y@5p2i zz_Zt`{w-bKJ~jj`{cokMJsZSIr$r^z#>(70zp1*YRU^W`r|9?HRz)hxaE>gCii;(P zUYe?3!Xj-70!Zcj@Do4BX&YnI)x@<=W@%tYhY}DG6eWKCq8*pAsQg%1705{gQSDm? zaSCBWvCbSqn8B$FzLFy8_NPz2&yV=z01{{!JFgNc|0Nq9x3WXnFm=-JgIKF>9vl43 z`|@4#xjwgW!j4AkXbr>~G>N1e+iF%F_H9;K;`aSIj+O|S@3>n!&1TVoz&@Id3pgTE zZ_;yqM!$wdYYyuCZMwowO)LMTSMqX@BbEM8!(>Hs$Bjyq^F%!mo;$+-Q1lxXZbF#$ z7e`C{|Kn(v|8BoaeN#sj|C$YijSsHnyAA#wZO2Mxl>r_W)&v^S1aAOgn?Ezg1RI&T zRU2F8yHnWr#wb~f`-=;dXuDTI(N;&+?MeK>7?z1j9Wy|uUV_wyA;0KdoW zz!|+6pdMV>uZ~;>t-7ef15~=x7(zwYuXw3|;*}ts3#gQI*)Kp!4fc=@%R=%b1PJLM z8l&=Z9$>WmymB!0XXCeuWzg~pbYZY zpAnwlP;G#jXZ9BM7@g#|Xk`gb^%WNY;Z&9TkzJ%u=dh$N1kI&{Zd1{L!8O0J--^bX z*_@JFS#M8_pT%sl?c;B1SDM;80&>G6qzvjIv;(D2SZczgeau+)cG=U zrh%NxqvTB0s5@tll_oS1;Hu=i2#dC?xG52Uc?UmU{3^r;IQR5wr9P-mN_OtoRRJ#7 zs4erEZhNe6?G-|b!zx0^7kwDQ&8_jG#oI-_<^c7*MHhfrih%^ zqaK&$@Vzati*#ad!LWF;i_=h7u!n`8t4g)h%x;^0YyK=GQA`kVYyw$k%WWB+_ouv8 z{~(=O)w(=1kb&(jG0<+n71WHAPj&$YG-Dez+N(OtcnEZ61zmJu@+jiS@IIv`WU_=* zx8#CxgN%7%Qh(b)x9MFQC*(ClPKr)VBYD&5rLgrUCC81Iw5FQrFw=}y6S<2}toe;2 zphWfHnfDFBoiaT4lbSF3=#_|UdTsD@ua-%R!>`tvTH`sj7WtIfjCCG^H?vk_mp>cw#F zq%TSmcnvX;!e5sx9)9*|JGxBg8^Dd9L3Y%^phR&DtB!%=`qM051JlrSo!)^x1A56U zdW|rb5f$uFM9LqKmSY}j$-*FbZ~W?-B^Eq9n7jJG9Sz_kTth#rCpj}jO%yk-C*GR; zZHTVS=pM0cnaxshHaWzKo6v_~H{=j8i-7$MES9FP^X6-aVFu{!7TGM#q(!M`hv}{* z>kSxUX<6<~39cd6?CHV9D#f9&{q!(j}hSR#mv^HiFMQ-+WSwRWG=aZ1da_bzA*=~BgE z-m{6*JR>;PtJDLj=XKn8+Sg+aZ739wx zm-%xwSRwy!gof)NBa(S^iZ(VR5y7G`(5UeoX09#dw3k;oH&5UkL*WZqdbr$jju8hO z;Rk4_P?B*RMdWffLuX5ta?arggC@szu3;VIXGoHtKJwgeQ=RFa4?wdWZrSQBfe9?VzFVmfb;iz& zy(YLCV-MZ|9F?T~V5(pwDO_dB#gj5LD-BWf zoN}ekl`^y&XVr-3um~%SquV_P&hcBLkW_}ws6AFV_EcjwJK(YdY_FX2=ZVyTU-tt#U{%<9pWLcetL#TcqydJ@;1Gfg8uz5dK|a?9VWF zYY$BQrQ7d5$^$#Ms3AL69^lW`9!YzQS#!4>Z};Hcm3xv|%eQEDE8q+jFoLlKVH5?J z?$-Sy3W4+tKEUifoD2$5*8E-S42=kP3nMJPn(!l!9x_m@mozf9-GdB`vDiunDOAB` zoltqp0{orjL2Mt*ui|eh8TPF|S;ifyY~yZZhQ$D0hQ$aa2rCj}D5&9xlZ{`PcDV^Q zc*M`j4v1q&^=I7W&__t=7AS1DX9BkV>tUTcK6b-(Fd#{e=)Q?Y%VWrvKj~hgT_T6uQ#Q8@`-39`V zh&FK{JZ5t|!@wm%tgakQ^#)c8H()n5vv?(Jh<58JJ!OJrN`vNzsV;v+#%Y(D%v@0n zo|jG018DqfI>}NLZn{5V%%Uw^rmaMjXG4}lO_*HPg`|zKYg@haE>A$UR=fR@F9Efj zWOZw5h%r_itA;9#-V~zz-HI)VUVer}|B2|MMEHI<(q4O9jy%t^aIr19?|nX5?EwL% z4k>(bb<-fmJ^E}^{V*c)P|iN-j*Mp3wnUWM1XyZ(fy_*knbl~S?tMRmx^X#w<gUb)eTF5cV(Z5QY)h`hS~XUo{m%o95W5@Jf-~1OXGM%6 zX<*kyzpUE^UFJm!5Q0s&wJ+*PBaWJkJx7*KV+w5zf1o8?9s**A_k)lrv3I{cHT z>&S)#H?vD5uB~0bS2DZ@>&B)(jHxhx0yv*Irsr!A!MxPyRsV|)w)F>^OO`%WU3P~X zyX^3vztu9nw}JQ9u=t1Hr07{~7q&!Zbmaz-%1#+=jn%1^?Mb{mu>;nat^oGcyfDN@ zB4A(cfQ@u%ibsq_?cA$KD+A(>0gXO_(Znmt(S2>XE5VHd{df1+d49cnLCF7vK2<$BS0~5^|K+J8RkjdSV#8_P^i17ircaNHe^d^duNo8{?Y*v(jlFt;^D%GxOGY=#Y*ISE8z-G8R&l{r`3S^#K@9pC zV*wp=c95MWD07$BGgW{M0=8EWQ@`*rgACc^;j+diX{>qXnB&?SPhRF@`{d*GJ!pfI zdM2C_;auK5gIO)d@Fx;}VV+S59z#r9NVZH5btE3vM78LCdnywwAeHC#pYm=O51Poq_YRr-tr2 zohnL+DU`bVrCcA-i?X6H;4T2xjC!!fL%iUIkz}>Ja68^$YqRqc&}1g`!LL?JYDd*N zDd{~by{o$A#d2J8%Xr9CCI#%Jax?x z8le-q9?!ijK)qE*wzj^CqPeqel0_cb?9%var7gdGx(&=Duehg5)7}pLWPAfh8+%1m zOS|kYj%Q;*N;h<7W06`n^!i&s2?j5Ut+mYzfJ-Y=i<=8>o|-s7oj9GJ+4j<+naWpM zF@i{;5PVkTAotd-A4o8#OdB02wga>1lO`5GfSOWbN(TPd|B~^$*JMFllzN1_XQwEB z4GBwNijNg(PMp#OiOSxHEySc=IPieCjc11!gn&!O8x-nG-7pEipTn!rbIq$Z8=762tp84`%^=ht9sF&+LwS7*nQOXWIsP3V zpu1L_6|Xmbl@2WKJcRQpT&4aI`A4!-Q>piNJp{a&E~dOLeW8$)E^#5C9L(Buamk}b zTvv*;-}(#@eI?3BzmBD`$1GdpdTv74m$$n>@iLX_wkMeqM&BL#R)K=o~bc$)iAuU-hE3mG~@O4bDoxsiwVAs+3$& z&&@u^bepOyUlA8cWq{kH+-d+1pMUp`b$`Lv3{5^2U8}fkz}*DL@#mx$_clp+IJL>5 zs?s?ZRud46;CC6dWwrNPU%oOP;=$rgnWlP$aS zU*Pz;{xokcdowRNG!xXM)^bh1`+Npe!l+Fklwt`+;p~%0tFUg8OpFqv*27w)%2LP5 zzL+}5t2}BwqD7)Cb)t-vV^1;FLsU_0O_w>)j8pSXPZJ~KP)F00ZCve&(1*X{O=>FEK*5O0OotNOIma{&_r?6OI}yw5`xtyH|36UqyP{Mypi?Q z2wQu&oY?=9Z0IBicNqT~k)Zkah{Ts=;g_3?95g~ItRyr57$pn%5{?9WIN zgp+-JWa_CpFD2wKIj(coRF}aJb0iC(UP$B+sZ)b6yg!3U*1V`6Q&M~jS|g}uHh;F- z&f5?=VXANEJzjwPTOJ9?B&mdv{6_l+95IMZ%Z~e25kCK-8p`h<^&ZIcL6+RF#en>K zG4^~ggsFa{NR&WVl?^#mMKs;gFL5wK1p&Gkb~NfFA*{$Sty@S?t5zNXncX4WaS+Im zbS|0j*Hd{+zus=Eg0KvVZi{{T^sqM&hLe@t8GR7AUB(fH5$F-y z7;ap5&TGfb6U-yb1CL(DX~uW~UM^P#okipUPCQqRnn@r89<~ggqt1MMFP7=_IfFwx zJ{K39Nt{Z_Nbe6ioYv||_#*htL3FT|S<@e(Ndc0NSRQ4av2msrlD}oDB5e&h`*oYl zQkPBnFwMeBymA|-hGS-HocVr$s3Kr~tt{njmRm?3N-_uA}mnQG|m1@XvsW)Kwm$EYuU z(ZM@fe3(IFpwKK%2C(9+9$`0Mrw-c2b0eKDJ0F@)ob*SOy_UV$6=+GVU@MQd<%DE( zYr6+TJGumbdCP(M@x@INNoQKj5;?uHR+{FM#;~v%LT?28*u`UrW0!?vxu54zXd3_l^yEX5Zt{n{8R$JHOTBm;usQ|~pGe}KoPFj!da zAa4m4mR}KD-Se^}&*t|^Ktxg0)`4%4{H6oSUH?GFiAc>84c)JDVeb~u9`zA|>$l4d6iX3*!IiG27YlnKdtSxJ>p!mWJwNPt{$odq^ffm2&yyME%M<>; z^%e~jp-ED2B)??+Z%L6Lfxhvw^8ErxAv+*aAc_}IPd4+e+JqnQI)Wn|)!={DnY zElrb3jg2)<3~@c(ae2#3q*|Rmn%1di$}El`BAX6NcYNY5H3A@AfwRnpO3vLi+20k< zT!^>)7=z+FPI4j!Z#`aj!sMDB2GqOhD1~kq>6E4(W*PDe4=B_PN!)F#Imlf|AO3OsefZ`Q7R$MAuGK~*zE596FJi|VhEToAesF)nEWqf{0?-DQUnAlpoyGl zc}~#0QHM3p7GfBJ3ldcA{%H+*@<_0ZyeNcdEE{d)KkMI&YIB}a@|X({^PIu6_t zL3w#msFpi%Its^3m?h?as6Qud`%F!C=+uGdLHp##EtYy?!~3nS%m?&WOYd!Mj>8#- z=UVP?2WpQ8;_VhLAzuol_x{IqKSKEAmG8zG;1YOBiDVCb^K~u+u~-C|s0c9K=Ck@= zhW=7+l&@$G1v}+@rcP5+o=6M20)ETk(9Kp91{RPf9N7cMr_BscIT|LYqw}FcVzdNw zeTdF}}iMui7~fXv1F{<>6N3u5)Ah{o@xCUEbq}{&fr{{=bg`67av% z@UGGS#gqCKY+4sHa)OylZX9O!ubLs7J0?IOUKWA`$jkAM5| zfY>k|U^;^18pp&L_cNM8BPNL-vK1$YF@Y;OwnRLwBMtB*CZmZRmvfL~QI6EUY{RHx zD&Mq^ie8v&d=QfRmKQ25<9WH5m>QKGO(6W$bjM3ll2fZZ7A&kwMqjIki9GXSo{yi| z4akqx+X8#Bh%nTUM|}T>H0{u)Uw(ZZ50w9x#VJDb0dFhv^70(8X-a>HS_Q_@6ibM^ z^Hs$UoMjzOOT1%45cG-@vaz8P4Ry0LWUPI>e!hS+3wmFyTKwT_nT(X{?3&9(E+u9=)e& znO<9?4y1mQJ53*l(B-112D(CjsO=l-T2n^}a!*^WoE8UV{^j9;*goC7SJRo^Fl-Qg z%UNkjERa}`^rLt4$@WHgD4@m=Too24N|+IikIOG2%*x+bEZq5n%WF6qoWRZT^3Mie z-XX1Ae#OA%UorTWdN&38BejO(4uKBxX;6k)jsx*~ z@3^FvB_>^jG;UP9w3=?P@*CqEg^%^{5E3GuHPYFjO&yM#?M*Kt*4|=e9(Eu!G!B#e*>1ihNdz`Im zvyt7SHYB1-9)JWmBTE%k9~0K8%KJT)^n_0qq#rx!IDC5AI=zd zs+)6X=0l@9(hht6(%I>UvtRjQHd>_fWH%vPE`eEI5=BIuE&Ik$mh*4iim`JDa{YFz z+NYoY2@(!;EZv>t+gF6L{H>BIR|;{ZbV%uy4;_yc&GlI7Y+sV=O1saA|cC&YYk;@V7{4}y!P?w?gHcJ z4GB;MvxZ!TH5kpr29V7&(4ZM=YKW}$17sjOv@T+F@f}8z1qJL*rGa~Jg>i_WP8sn8 z9!1!2XI57kd$q5r-ok-NH(YoqzC$+>(Z-gU9r-GWV!y`m{`4cpVlmO4ArJJY-<}3^ z;%{A(A7M)A>d%U@;KrSgIE7U8Wc!&k=Ztdn@U)%htizefi;WjI6I|5$umH%oC{F#j z_y&%DX~89HT4ushURrPoKLF)hxOdhtzK`MBl7^NM7q=kgqK#{ab*6dxZY-x*TJsP; z1YQ=p{pPovB;AV`COR%T#G{QV{KGrtCwhJM-CO-YX};j`pIN+ZaI4>3U*Sgnw+{6G zZbVA`uekqreN$Oi9#s*E*Zr-|R+G=F$sZkQ&8UL{wo}T7Lc>Ujp>7F$$F7CFwSB8< z0%H(Z&;zPsLljPSe{9cmzUdZ85tMeyHz{s5&B>kErr+})>vR~{FR`U#l}#l_?V(Np zGYo{KCDXi)BvFhsWrSE!Tb7WB7^CGraMWtlP-g&MMv}S85ORg}6mv z$aG^Lft_~7F$jG%$J!1?{iWHo8m8s=0dLR#p`Cbq6TkzM*S}CzZ`HQikSo0Op5lgk z^%R+M`w?%}qWH3iL5W7dOp-!7;&!!5C+x!sS3s>N%n-HdjauEn4PTYEQf4?Fv@5!2sq)D<-sH3C zO5O?U=5>&%xC$@Z9TZb-1QreW*55waE&U*Cs9H)}2pOHl^b96BcX)POn_EoQ^1k*h zD@jg6cCDHr*57S|437)yr*&tRfZwsC{&qJ#6upuZp%Y=5{k3b*D4K|w^A0o6$zc+W z;4!%;^&CDlme{9KwS-DC9i5Qp4V>@m6=YXDTnmL~Q=~k67&beKyu%-3AH@_8vvFha z2n#~y6`AKK5i$Z^zc2?Br2R0JWRi3X+s*SyVi_5oQA!EEhv!5eNAz+BSW$dahDk5s zznY+nPX(r=B-BWX12FL{-pCj4?1W>PKueHwwk^VP0e9!xMSKPt1*?=QIUG*p(rE~p zi6jgroj<*9#XC$UzKp2-K>q`Y)Y*{pf?uc3;j5JN?>o`gGym()o(iQ3jetlD`K3MK z<_3x21}WzTNtZg?4a=g=p^@$9=S}uF|_*;M3cy9 zWqzc3DfjT_9e1LDHn?Y+@AO*B&BXTF+YvwT9mW8F=(EF`DT|kTI(j;KuCW}yP*-5A zI>L+h?0V?>D|mP04-koqp=c0-bQ_T{Y7U9(P7WG!xPO)bS4<%sAu7nymtID=9+hkR zN(35)bQr#f!F-okhUzXqz&cIUNH4q+($1i-SJua17vt+S8yyLma$6VBA>|X%7%;;? zh$N$eUHuE@kz}MHcB*Ew(UwcUNWB?<%jDU8gs9GRho$`i_HgDQy0)e)YrHBo_Yn1oE`(gw2%V&UYqj-uMR@L7_f~iMmbe zLzhaW1kk`vukFXqLsOJ66I$_O>?2xDZRQr6au(XSr3{*}P?-IgY#3?=7cS>yqdePc zR@@b;iN`5GA#g=9q50Fe`AzLkaoABct-I67o3|qg3HpROT1#=e4sEEq(;6d}CUc3+ z^sg6P0~r!SrRtkCpLCN{vb<(>yU%NpvQdS&883N@?m4Y!4xZ;yvCcrVI(3Thu=Iwg zrs_LH!N|twb;{Hgc!NZ#TzujgZDYT4tq{sn$SdpJ-lKvq%`1_2{OlL3Fc%Hdl zuPeREJ20AV*g0n7PL=47$28vX->_A>cA)SM$UbOr>8v|4_3VAhJx8y9h>eu)M<$N? zGg9^{XCO8SX6@*ZOZKy9ltZS@;g^yso#L&Z#ZSHh=dQ-N9#Yci6RCHoM_%!fVZs=r z`q*te*I4^zkFIMZP8Sc}vZJy_@gyT~Q?Ey3bYMekDDI6d)kk4b{WNHEnUDNENX{P> zIx@18_YRMa+Q{mD|6I=GuoOS>zcZLhVTmG#-&OTxY*6kgy<)M@;$J`@Sub&|#rg`7 zLYGO%`Bk}aV#&rT^Z>t6tb|+Aw~DnsBIXuv@rh9BXaC@?5A%6_O37Z{rQRIqScPT3 z95F(ZU84U6WHP)}+-%9ToVMxw`NyHU`?_c&>dWLQ81i4Z0V47>AajcXfc+16Af(Px zAVH?u=t4uo{ofHJ^`|a02>=;u7JJUJ>D_iqe=*`yqcgT{}ke`ClN z*&mpa5rYuI#)yFciqo>U$CkhmNm#^v9kUbi(b5S@C#xyI0{$iN;Iiqx3o}GuaPrOAM{b1w3o> zxx^^JhPqN9PJo?*r?|^k3rhm_9W`s8RUT+Y4N{&Z*HGlHLN1% zt5ua!yxbvy)%yr17wvVwT_u1kLtJAGwnC}D(rxrC7Ye?6fWxVUm`M9u?T_dba5}OY zjU~W~Ujmb0!Q6bgtPMTa-pMabKA+b;4J6&3o5%A(@viT|dm3ZfIvME|waFJ!sOA~O zj%29s68EvsXXLct2k6%7i%G@k>Ox?NwzwL{G^|sLG8YiOhy&_&HZIH#7{HxHpGMSC zLY|~E>bTnieo}8=YMA^ARk2(pvH4JaLaF>^c-0DhIAAd#e1tgm2E$@e#PTDh>SmD1 zG@tU8v5-gx`L!Zi{Csk8%7D3P&EXoX{zRbkHRk03NFF$2d>V;k1%^GX2<8`tqOcR1e#V&^w$YTvE5 z_p<8*UmT(&#s()WRlf>s+)d_~f&w35>f+#rnT7-6;YrX1Na8)>;}}yp-Uk=h6+}kE z#%J250_-g?d5D=?tt?Q-dC5S)b#RQJ)Qm%?)uOW@eIa|N`eEL}t(s-xDfbX1I6ONt z0_Dn)8Bv;{vr8Gig45TY{;sAYLeIg32wSmWwCqvYHcj+BLl>vh^*&;+7 z_63$3kw*dt2Sn7Qsrb|Leth%^jw)H;?CRtx!uD+%St=c>_2^Y6rX(d~#;m;3xG19r z0Due;(q~{Gshmww0_Vv{u;T0hWy`lDz*p|1;=4YzS$&wJG~dsex09PakHp|RnT*j} z%w`7UoI-#^RXRv;4^*mNu}?^`%e&^Ok)Qf%d~))MJ;VQz)blldooEogSfO@%&wQCt zm{21`0Y-5`WLW^|i^hf(46FIptV3KRtYFRlzMeAg7vKZe+9Jpt+#%Ip@^@mudC1;t z23n|({8Oky>p##{qM{ zXOj;Tye5IIUm{0aJjBQ4gH{E2$L?{K`4qgBt#C?;jLBZpm;6_p_EO*_YmxCsPv=nM33z2s)d` zljTVda{!Dxi$+*31KGfDv>*{m3&LE60(c0OYZR8s!Z>s?cn&fk5o+}E*{mETsadg+ z`@wE;Y=x3Qz+(QbMy!tlhWzmdzRwa1w6hsTAP4;n%XE1~e}vz*R7KZG8y~Etyj*sl zJ-))vT2_9O2@PS>_=?@gzI{GnC- zQ7SfM+7egS37ic6bH zH0c-k@?47ld-cm{1Q4f|&?!tCE)X5H$Iia?x0^_hb%~IT?lRZ{u|v^sq3-6dFp1@m30s2{H-tq5?ADxLL9+3F$NyZR%KX53>>VanKE#uwq*|r_m>U`s2)VIIHl^&)n7mEVQ4vA2#nt)af z+yo=ka!FJ^w6i~soXoT=>FTw#-$UnXi*cnveSOFc2^@M+mUL(>QC;Vg&ACBw`#8Q= zgh!+7_iUpcvy3eNJ@ZHuS11J)GyT6}hOTzw0(n5;nKv9bIaKN}+S)iG8Qx(r#5W_O zp&2Uj!mSsZHS_d1$9Nqb($86cv6;mh>8PrRZiV&i%)6Ppgay7=@A-EZ_iWsLy8Lu= z=SL%0UR!r&+NYYPB7lg7yEj1-kXh6C8K4D$xX z{WVu34e4v{IjWLMLxf+BP@n(ggn$78{0DgnNH+X~ybQneHybarLEgY38}ByQb?6}3 z%u{e&geatFDvM*svX`x(vN?KXpM2kxyBJjK2_;G@j}pzgPx1$RT}scgQRK}OfMjlf z_J2M;KJ5_)c~n<2lrvPeR&HlS&_+^<;1_;UMfVGRzKeB*}tMQf8HHGlr7E z!mdQ=_2pZfY&NYS?@;Nixl;jZ@&tkOjc@z33-m>BK1na(u1olo1=<*Hk%oL7c2H_9 z{P3%G{C)gc8gmY%q}ZlkiMl`0()+ORK7oR?XcXlSf_d2Mv;$=KsM+GUeTVNR6jN0Q zss!_lkGYRM>*#;5MeMNkz_oD@YuyaH_$eRo+K?%v_>I38&*)C~Rjv;n`<-O0d1xTw-w;|m`)_!8Zr z=M)iK*Y}wUu`L)C!y5ihRiDV;7Gdfxwn?Y@B7;F#Me6ZF4St9ov}#7!(8g+lY*OXJx~H`k(v1 zwc-D@A&&prkfw&(oEeUYSEz1#-=erX8m8=^BNundFea7-N#cZU2xV}KQ;Au9pmF>6 zP0;phY{b{t0o)gIm}h!&+$i2{EiHFBZc`9Hkg3VztMfNJr0Marq*LyTf|V8ogRMv@ za0u9N4H+Ypq5f#p4CU(F9N_}kw5xpepdffUChrgCm|S#u=2`jgfLMN*z~$Oz^uhOA zFp^~aP8|_>#C(YFI@)uGjQf&BVq24T7f-sG-)yx4G0xe=QoquvVY(3Tz}M(f&B+#3 zz}ZxFU+?hrjZ`^L?X(#6LbSLkH*TTp@ z`UIDk4J!%MTSIzz2qlJ3SWGRqEJ`El261piosTRVvepgdc$20wO~9}3aOSgqv5~px zY$@l&E%VtNj9|vMQBAncYe#%1w3@=_=vnv`jT0Ds@n9OSz#oy+JIb>SH;hGw7FngN z4IaBO9SMLx$=O<;GI+ulneY>b@}_FIPz76DQBC1GVl0*^c{v4$2AoCFU~^}UQt669oKKLzMM1L3Qgs^eLPyRJt6#1uHpL}!X)|cY8pNu(i^ZsjySSdb?7}=l-eq2yG zM%!6;9fHH~dDJTvd0ggroyt!TR;PEV4&dVKNCc4@4-ZwZkIR(}b@!XL_4(NIv@NHZ z-~DBN&_Miw-6j~{z>eD5RkwzDQST1wWovFK*}EVQ+*;Mq2BXW@hygD{nw2ZJ*e%<2 zn!vfnN=*o{#~Y&`(%50ar9_~EF`f;4S48;sZPZkLT7ZpdT)pu?+=X#%0J*|WWcGLfsBoT%^P0{KJ(SzJ6YCY*%imlU zjyJDr3AITvD4_el=K`R6{Qw}&|Hh!iDJAHY?cpywmG;?t3G-A4&2`ggtUf#;2yZ2R zn!$KvufY)$zPHRp{r$1s9SiBlIyvqn8?m&s+r8*|_LQIL`tN&xpSRP!y01NmiFt{5 zc}XqRoy~gL%u!WYWI_z8L^7t~e;_gqua}4cgHN&aYi>OXTVQTL-_gyBrPzko?)gA{ zv?uIa?zJ0v0FzU z<=wVWzcC=4`h}=u&UeZ5SZ6WY3R_C@Zf`OjH5@66wF`UDEUE~1i6a7e0j%qpVhfb0 z?l0u~zdtP%YmQo5!4HpQArDST!s=A_{j_Ruz>bO{JRW6`Sh5kVAW3B|2h#4{Dw`U4 zm?X7TB-6T{eYp=P+J9vHlJa$ulr0JL{I+JGlp)}+tUIF94n(0bVD8_#VHz9dZ{2_f z83khCfC4m>lt?5np6$rF1ciRIcW+ZO0pA=e<{S=&Qc_Tcm~q!pZq|GK4C5798(%C^ z2-JBKO%p=)_PN|_vKC8vqNK8_Th7DF_~j`8(F`?4)Y^aIA>-;|e)y4s3E>Tf-RsKo zZ|brH<_Ia2MCB3sJqW!qA5JZ037ptYNTMw;?Lk1%VZU6d%vRJsH%D?>$r%2612?sI z*j3z*W1X&UN7TUXUx}TQw|z?=iYUB4rD~k=6kHCG$@iU;w$ILC%6g_I zb@(hNEk&5W0p5zGhvhq5u_PH*=A$*`%6E5SuNm6vxOY4dn;viUd%D$Z7+NiEoh(Ru zB7?*vi^K|9xo8_7*7elH1R>|!8Z6Oa_4`_mRPTR(!LGr=dw68wQ!U|K|4GuOOykJR zCvv0I*F#EL)!D-IsRpK!FFN4DQ|U%S*&?IE!boICIVbcxW2Y$r&EsY?pm&BjaSU-7 zPS)wy@lTy?;kHNkCgka7fyb)vmxP_)1>o2M@t*G*)$Wbp!i`jKdM=3nz6QHmRL4Oz zXZ3Fchc+j4kbG7QBq$GPT(wUf1N=fAE?E(@ZG4fQMS&c{f`BNCziuAVF0NukHr6{o zk7v)ZRl{u;nRWk4G}XnR3Rz`sB6IHQ>1D!wUCd|c2Fo;E}pmn*Gqjk`uzn%r?4Jx%bhR<_uJ81O)>$}{< z78pDtpCETI)*io%rx1g!mP=bk6BiM%ZC7lO!4rJVcmx1T;!@UlUp>~Xyz_LK5u?Ty z+5OKi>FMYRUaG_$M4_|pzO|IM${Sq$9mZ9kbjEWH%2sPR1_pH&j zSuqen;ZQ|EdmQjfUk`=1#31tebogi;u+Z?9hK30_7TL>+fwP#4HWzJc$>@)cE|kOL z&^S~cnhsv36#Tv(Ec}KK@-u4Z_ndKe8Ja#{M(66@?B2b|79*IHHfu*|!BKs~)<(|2 z$cR_Dh+f3N<4+SV#%*vO=`Zu?Y0P==2ITfLfSQ|6F{Nhj=EY^zo4E}#q}zF1iw$@6 zQ#J3awOZhetZJW9U`Akl$Jd|wX$6AX6=>4zsUf|D_B`(yVyrSU!cJ2FWL&M-qm$HA zsZp~=V=TqxLcOV~T6w}zCBX8RQ;sIKv4)(*ACqw#X06XfBKF-Ff}^q=k#BvC7~drU zhoUvaiWs?YdTelQw|OGA5Uhf{k%nE-Yxe}0(&@_$$f_5`CoeHp(r<9@>aBc&v2;pH zXH0s&{-wU|+)jKHfXdkDUm5?|1`BbnLcoBqry;zJoHGRG&yhXhIRHdWYr0a0B5s(yse6xNePe((G zojfKmCvxY0*X4G$`+Ro;J%5JM%U5SAj@Ag4-uzbXi)Sa4GHlw+5A zQRn=O^Au5#bK+>aPMD{1+~)Vg0@sld*`={mV{6w?Y~>2=jH`Bi#%{Q*3odGo;eMEN zpR^i|)sKHW5XDz9qMwLksM_&o%Hnjue@e7t$gTA)dzd-%WYkfu++)@^VE#S_Qo^fhpnsc4UT{eoyur;#G49B-%)?t zQpN5Q7(U*4R62Xsz+%T`6W#t}{9Y%5R?&>z?*Yh5OUz7O6!ys@y3V~44TGra`gP=$r#$lx%*)_M_WjCkZ zXx+KdV43$KMcyPtSZS?pDZ=61Mq|3|WP=*11em|HBO@!zu<$BbL1%WF|2Xp^fY8uT2XST!4y@JAtnflUfy-i#93tVqA#@8?X-9-~g)egygt6ik@f zyo9AM-9@5TZ=3Be3Nog~K-R6jV19+%96J{Y zJmx*80;RdB0VCDPh#BAH^>i<``_I>Q^WavsBHj}w4_m$ph3sN7hb)7}vU#3@cQ{?N zKc;lcMf_yAoXQ2Ty)j##hrRv;%3o1reTmTiGaAa<|6e%9U(P<%iIG6ief9ry4{sS)npY@*Ek>i{Ej^H!RycQeik-ako+I03gG_WQgLg+FZ51qP8@ zRg>{$XT$|@9aWx+>Kth6{W$n^Cibecx1q-^*e4h^Smrfjopx~O4ALoZ0(L8ST#lDk z*j-~t9+!)t#d;Q$Rl#I^F_2XTQ`YS&zgYYG{SQg#o+R1z0~*I;8f1@U{fjwqnK9&T zhFJ-`A#lrnh=!9Ayh~vNzrx;c_bFK(geK5M-O$Pd#ZrVaeI>4~nD9az^wM$m8zbfh zFkc7*H}E!Ief`iuG5eyPPC3xBVp44HkS4&m7+dyn`cMt@Yaw0rmX!|(RKGvd9xY+t zII19F$m%eJe{QB}^>fLC@uhF1DjMvkwo>g{`%cJ(VZHK%c5N`PebUXe!*?4B@G@Kq zV{gH&@f*`*!p=@v9U+cMK8Epu$YSWeSa8>UeXH<@eo?t}u_NKB-pf7oYajC?-QtRx z`D#i#~cz0F)lH+4G|vn@{dUpNK$Z){M4Z&jtU|t-)M(^|ChO897AEVzyG9%0|N_6 z;&`Vvua5Iv`&oPWO@C>_n7l-W1{f2cH^~7&_ckM+?tVAdl{#ECCO3XtZAcM(j?XuU|v23$1oz8Sh!)O77Ar}#*ifD|LjOEL1B-?`a_8n;pAU|a>G2ZWR z&V`(j6n(nNLfT;GaDjsof=)Z3Ff!WcvuYVh?6`zaCgt`b9+>-Ok1$*46PHrB2k>DB zC3lxCb9J8(+6!Mo4;=VN60UlQET0KDY(Vc6h_WwIrL-s}5Aw7AV6ZOzuv7(-OqO|o zH-Wj=#QHcfVi>=<-r$|m!aL9!8>glpzo*72{eDfSSDjr7Z+tN1JWs(DTnb8td#ehL z|BP;>PPv9}%f*1KP z+dqv&mr-x+$HUz1V1t902WtdKanvExT#Jc|=J|!K_$bl~xDh#n7`ckW?{BDdROzy-%x{Av6f^<~*PB$|< zP6IbNCQ+Tz^|THgR-7x=2)0aU=L@%Li!G0`lkA!F59~%HBEn|On4i)huaH%VKJPj) za7P$NooPAs8=~~YP25rWf)94&H!JeGK12kU6_ra9=cE^D;UYFiYYF8`a>lTT;5CbI z*-Av|ijj+c8t{my?lCP7QNOwaul8|;YfBiX9jpopuF)^Um_Gd;Q~cryi4s+ssMwh^ z3Q{apSkgDkb7J`cuMGvnf}UB^)H2K>p$Xa30eC%XxsC)?c*_iT%rYc=f+H4>3^A1K$7UJmeFlQ^i7k zZsK3R`%BP-{z)Rxlq!^&JM={+KeF&1{$^Lgj{%?9?!0ftjD~WnXL6;P6zH+0gyEgD5nMW__K2s}N?unDMrdukOTY<&OOStf~$ zU% z@iZl{WlT2M!TUE#lrCTZEvMUT#2v*jC$>7zdFm!lVZ>iQzEP1z7sUx>oXF>L9bWDz zkIck9UU4@k(c$Gyz*HueYnu)6C9De&waXCPxS=F8>1&K1x5jQpA^llw6D6EH7w`dm zZVyCW*mT|a4sAb+zdT16&NFU+5K5*P6cp1+`L!(*>1gE2ggt&SOqpTR3_VMv)x(Fy z1toUMJ^sKP6*Bz*w(HwS{Vc%ZZ}Hi5(Qv@zK`guay;u$2+h&7Y@d{B2UbLB`Da{14 zQ_??qjc{eYQwu75_kV;BHS_=Tfjn&><3h~l5D*~e!_aQz{~+qnHeDP?1V7QdqVT>4 zKRR-ZI;lt3!dzKdSsfn>-*G~n-k_S5aqwP)jT;jKf2fJvsD2NC0Z_ z4{-U>DzNWh?U7uOzT}v#bCljS}6ait8i8wQ6 zL3PnIKta=Tp21v(?7nuUkK9?BUM+~W;FWfO%)1tq6psGpv4~ptFnghih^ug}FL{2; zUuLOiDhc^qSF~ijP{y6T5R(FaYhq#X^d}GJu7KTt6aqOL?wvl>Z#J-+N~^1##21af z?f(pBV}jW;XiV7a8fQhh#Wa^Sdp~fA2s-JI>YC%UN~?P)+cGIS5tQ^o2jyHzMNmzk z`f*Tl#g6TTSsAt=G3Dq-ex{-6;`bi5^*I7?4wtcf8mKbxmvs-)=nMUncR+NLDS9l> zE&dVx8hgVx?HuiggwNxi?^agdhfSsnly#gabMks~+aUgen8UdQg_|R`mqIdXCF)Le z;Kknwp*Izk%zjYyMf{^a$X_h%ugZG3Fc2XaUlFiD8(MSX7|-#Nb3?CSB`{?w;XV|U zgnUu2)5py-a4BWD+%-xmg6k_up^LxQV?+;6BD@)ee_`BWe<2v2h|ErZwyt_B30w$Hqw#fNL?nf-mmruM&7H;^wz8@{l;14m z)Jw|*%_nKs#v{YkQd&?TYh`A`?I&u{IjN5x!#Og$!E3>tnlJZQtePFF9K_gyB|dL~ zRkS+M!=UeEmIR}g<3f+5?5#%Mk&l~5fv`$tTXvg54XVy8DFK6kT!HM_G^;jyv$<_H zcKy8LWHl-zKVXl2?P019mS8Qbzasfby6O!Gzhm?LH%|wO?w|7X471{SzRmf;sicf~ z5q>;dP7J6>QP*AP;jGOnV}xfEfedfopCwErJ5(vvG(7*b^7&?Y_~~4BX$jZu#0IYG zM|d~wcap(nQ`FWbCCpqV5vTpMclpF@clcQjijFJAiF`zovvTACt-nz!=DM(O5G&7n zK)xM6e&?xi(`$2~$OY=$qj}|`Yv@qF;p+o4m%BetD7hN|RMghoW$P z@}#|m>j|8}!)%e|@3os0l%YQ?mcO1PG2hkteR<{%Hgp*4`&$WPlc*<%Le1^v-%9xZ zXL3Z4FVhH!py^r%d?Z|H?VfR@67}>DZ)Qj&;!ts_bhL))ggc=#Q0Vj0Dc1SNd-nOD zXms%Vmn8O~&kjYif1Y04zGj~JAKYHu-I2fAuPX$q_IHAR6Oq|?mqRUWmBM!rd6=}{8tVEz(h^&_q2^0qnBRC!b0HqdiDT2zeKQabYNPUzK zTO(rQVJ$sac6g=6%9lCI39pcUAsv{f11Y%Pt?t0Dy0@%`E-eXk)@w(N z9lNkg1DvJPuEXAGcMvrb44U!9YiL+F;mR9RG`1lV|tpuD&p`*g? z5|DC2hji;oaBoSnP%yXWhhcmZgx zbQtBF6)b9IkkprKXlJ)<=2T1i1?UNnj?v$LtoAy(jczW^-j($Hfig3Dd}Bf5RwnNt zcaB=NgAFKaCFw=<6)QThK!|V=w19zc4kLMz>kY(wD~s$^MrakX z+Ad$(phZ9Zd5&J*`b*&pcEw`b9m<@?M1|O&oVC736uSj158e!F`R0IS=8Du_lgK8vWPrH$hqbtglEl%#xx64Gy_GnrH^e3ScLkp;tQ0* zjQB{p*>kuzxCg=VrOKyBm1d=Ny8`D~B{5&f3cG>^kkkg_M3QRSKaAC(cx4s1e_h9# zPi_T9eN%ST;wR>OEoJZW8eIBF0owb{L# zw(53NPL)eWBF=w&m$c9rebB!V?)N(NA}HFIJlX86?SvopRP=*i-5 zezvn1%&uz$Y8jl)LY|$!we$Cqd9(=N-yt`uZ{yODeTc>B8gh(+?z?NPCu5Z|#1bLJ z7~EqXJlY_ zdGnTDv3R!fwiv}?nh5PQs-U6Y$xsVrIGj*5 zln5!%tZj_QvMcAAJAnXJm$sFdUh zIe)_|WBJnoYlQOjEp&k}R;?<9uy^KndD+f>tb%JXf9|uQKI%T~$j^1kam8Pd@dNa$ zRWWNsDY^4{k!e~K;OlP4|2`qRK4#7aLe(|)kGdd#=NH5d5vV~bP{tXK$S2l-6aMgK z1STq->gfbVi#VZJ4F&xsca8p#wc~q+w{w}CxwauM;xDg3J&rsVp{;;+&I`FHFKb23 zVwU|bA^L+nxAn=VtGnOmuhi5J7+g{Xm`HTc;);gqdke~XlXQBVwItpMz;cbi4|j&( z>}#`FNXTHq;wFIVip^1Dn7+Fx4O*;PMMB0pP|`Ga)KS765j(1!peJw$^4$68dvp)f ziPo)s5`Y28ojZ4e3!DpzTIW?5{e9fqr#$?K?CH039wq|uwBag}qwh-{e5VMp48IbO z0u}%cgjW8nf_Rj_Y@aimg&i~GTrJEHlmHA4!(OOu4samS%x3!7qSC_zTx;nsPPvJY zM$1q7=+PFY`ftG>v|--H7Bh;&XoMltc+v9pKc*g~tiD)u!x$gd@H^_^2LMUr*GTdX z!8?jwTkeJdQ|=!W)6mIFT1xDZ#{G#!BxA&(Ppz7ig`#8<^$Z_|*Y8dWw-)mNClcoa zlG0uQD_kHyP0$jdx5Svk23mGhZ$#gvl|8w~@-R+~MRK=)op)n*{RcF3m+VBp>pL=z zW$pZT6})29T}9tSgXm?wE1%#J67p@oV2BM?-(p`Z6LM0sU}D825>Wm?V63^tJF8Rlc>PGS)DuVD+b;wWLSGXnRO&X9HXh1#snx-XoT<^SG)=7a z@BdjA5^G?1KD z5>K}l`Ft=jv6TCCc{TU`6=eF`^!)Vf^!Mo*XNmr3xT;uTRT}F`6x1f9AwW{b4lWLd zK>QmNQhJV*4A47Lw~X)gVeRqIh%(a?5+B1qtj-x>LXu^kO}SwsYN>B`&HKyE(Z@)n z=~O7YFRqsNE`p$z1OdhP)MfleB*FiP*&ah~^&-rJE8WBt=0&ZuzyE!+Jd-M^wEu*2 z2S@wQ!^f!mZJr8C_uK;K@iSz&BKkD_82EUE@7MsTCy*%)+3SZIYr40v_0hXQhtJ3} z8~K-wdQF3#YK~E9IDP8%PyEhY~h_ zA-&W~LMG%LZ41wSA>HHh!~lwPUqHfPRyu>hV~(@%f$kOmfxhneJLA#aqe6_fjuRu} zo->h*3y!2p-j508vv%{_OCqMxV{0Z6^_(N^@cF8>qM~m`U#@j+(`pe3vS%m}V|i0c zt?`-}*Qab@#-K+byJgxjb7!_*bD?^7VB!!s&Xj>4v zBXzk;b3Z`|i_;JYsgbEq4t|(*sQk|7ov`Rp5`V2xx|X5Js5(-EJh5@1vJeSk{nV1e zogUbpc7&EEQ2NZ}yfJ>$D%|^S`@6=0F$aZUS^i^?Bv;-&nT+we-T^4LpT8Mg zJ$P_$i#oab9xB*d&n8dU&kv+`>~Nd! z-E1fRlktTki#sa+=Y$G8LhyaNi$P7!tx_shkqX2Yq%~Uk?lb`M+r$;21_57L`lc z38Fk}@vEIlxuscWF;(RMB1 zTe#M-r9|C8tesg)^Ih)lPUO=&p^s2<2khGH9l?*o+QAUNa>17wCt^xRO=WG-Xb?8~ z!A3GJ;yfga`e#|>F@B|IhH&TIT>?6@!9-WSAvF80NU6`7d$`nX>&1YHvp0ACj(11kXS z;4Ya6z&c?CgQL;In8{SJn2fSKmWi`~@M@eCqJ2OqkW2xbDOo?W1>Mo#X7Mh>uhaqp zLJtd|>Qn(Q=O3okh1Uc^3r8O0 z0_*1*LX)o{%8aj>v}aDb8G(I9GymlZexJjb5%ey%F_CVW`LE~5rRP&Zz}VKssjY?D zd-c)Jt4jIsLN5N0m;fzaf)Yb-bF{&5@ju9_%72b78X|nOXaxJDU z6gE3V{9*j*JItufl(8}mM~w6Z-kDzVq) zaTE{Az`t4kI#3j1kjEYt-Ns&nOc%lO1CzvR{=G*=5kfS*(dA((1?Y5xGlVumi8peX z4f#_!GmZrVdW2Y=>b9CX(Syrjz34|eWNw53Yzb;o#}P?YSdePkB#4ggaKg%(u2i9f z{mk69G+UvEZc!c<$U-e@{XJ7C3VIj&NuSQ}lI_FpJUB`)NVlpeIJ#X=r!mv=mBnx3 z<*$>QZ)H$FMXM+1=quB(BYjJlpgI058(!muv(Bk_tc5@ScR>h38@m6A5LDF1(C`0x z_Wc`gpi+hh{g-6=SJpT=H4a7Jn(agowns$yk@O+U2XaZa)=^svMFs_@Y&aS zU33v`Yz$uNx>NF6Sn>xFhrfl@=YUw@rS`TM{kcyE)d$in3<`tNGxAEy8RfP;$R>vk zi2&{)jXZfGxaf&^hS<+~>vsVQCu)uax1b0!e&*Q64n9+_UExU!MQ;Z4&qE+A7R_*! zGTK|lBFjwBAUWi|sp+datkhr*>(xn9WmF`bSzY+ao8v{s=|0Ecro9#Kto>TEv%fI< zw{%7=7N}r3{ zQ7gFF$RbnvscB3tx_AjvL;^T)5)UAl!|D>Q6yTU2i2{s2SS8HF2=a zRa;zBnU7R$CG!=_D%roW-AydHM>v_U6g?rV4%HS~wL~C|4ti6Abi}wp?IVTO&#+|a z?XAerm4?77TkeT`g887Ak&?0HxJBz5@`GFNltR$ByXbLBp2TbVcaUX=pB8#HzeKbm zhcXxycl~cLD%<$Zm<(0o$-hd3R#$(ONccZIYzK8HZ&3oHdcUc)PnUgaUyZTINiR zCnByKpE+m3mVeL?AyQ{1PuZvR^NeDbJbIbnKXKUcW~yG-%uTWZK{dRph61=VKEf?v zmlZExkCk4I2GLiNL>*+6Ef3Ges&1M0wO3F=zS5w3i#Ap}`CIs0fkNw~TK zeP}rTv$M$B!;uE)?EJxp#=%0D#?6_A|3BW8tG%KLMQBCdQrh=Xy>5#{V5lXKr-0KI z$Jnc!!C*-?yC_bX3_>1vwEL_SJoOw2%E!T1s@k1l5>q@uWMr_|42G;9&4~mEtO0iFf%nhpe`;x%eRVbwx&>2(B7O z@pdN8oOixC<{jZ<)=67m!Q$4lrN#kWf5jlKh@P|*7C$0f*N%&(Y&AQqG+R#P(R$5UgQl#uS6UK09<8~Xq z;lf@=y2h%i*$k`!F}rNa^)jt>mj*eWJ=3xvg7&ElWuMYt%e*WGpMH@VbZvUp-=+7N%D`&#M328NS}4>3^fMav>4=^O6S|M(JGLsqcoP2n0e-GjN@ zt&OViR8btC9=r3Oa)>&00CnKaE9_U^|9>3z4-A`-4c)EtuXp^{3$%c0f8Hq!m-AAk z6k(Z|5WIC;spKSJSve%AOBGm$@R9>p$qYqW*LfC%-aIq$@JcIO7e$GU@%!NTA?hZq|p;KSge3aogI0hDzFI5|{~; zf`Bx^{Ib~U0L~zW+N##5*l1g2c{jsB;nNC$v)mvRoz52=d)W^X8Osp}5|6Ia^JhZ?veEE@6 z`eTPsNH8t|I69qLQKE>hGrD>#vAujMb6uNYsAEys4SnD%o`^zQeM}}N&6wk6rg?v| z6MA}p<7wCSBpPI(8zP+- z{*ni79!nGud$=(eziU1E_*@+Wf`dFyJs+ZfvGHq+2u)9?ZX1v{;I@)Z!DM9BJg&ph zI8@LrpchFxEBXY2Rh~W}6g(=Hl@%=k$!-%|R0f9a;ullh%m06*y;WEoS`sxHG`LFx z!Civ{2=4Cg?!n#ZV2!(b(BQ5~aCZ$Z!QI{ePR^OR^Ut|+U+x3n*R(x!)vj8*YSk(w z*eDy_)zF!Q?&cUB!c3uNxchp2Mz`1nl`U%B=Luvgo~RO(J4zvwEw%+>z(?3JBBQl%hGX@x9b2&INnlxL|khX`BOCuoWB(df~X$A~)i~clVT7qWQ6_&@75)3Qple zs!h%ltsNo`qwT$S8YiZwmY9#rIl3lpB!K(gMTG=YH}9$gtXBKuO<+j1`Z7#w5;vdT2oN^Fa4+Op@ zpt08#Bw;FR2Q1n5aMS9H(_9+crcF_2v9>ceWTwvo$ReiPkyuP%2j>{Pmgm!VTZ~zv zD5b8j&`KV*uZ}RF^7YlvbjKD8`7mQe3F(fQ7T{!T*y?jwV{aG_8$Q^2y`A%N;DCP35|`1c=fB&gIL=!l8?pqvX$>>=as~PkPx{Wd z0mWtT_}3r2>s_!#UANyBuZfJ3;-gOb{0IS|LSTHes-U1?4kP7O$;^Qswk)LxhX4(h zN8S`|xvK(Uboja&HbpPqO!lCs^JgPsLaOX@Ybx?TA^$Gi+5_^ltHCD%^B=MLuLF_d zeh2sAFI|}=72Dt8*{_k2k2a(phT%N|wt`Jn$2pm?vPl3ehFnDt4hFwnOa;+9_tDna zgDPbckwj5}^t=4XxlkU>ZW?+W)QHdN~* zhp~R%6VXnO{9y9D*KY}l)D7Rm5)WPo4$H)aV2Y)Ua!EIO~sDozdybH>{$k9-k}t1bz|WB1Tkw|lXq+!#*$8*kW#;|39DiAw?`>6L3}&NUX< ztctRCi#N&ET=j!dMYQ;}sEc|cI%du5@IZljDBEChr(7WD%Bp~jH)3fvM8$ZaOuVbw z=ed>b#}EEJhKo}y)Ax0|U)%IAp<*(Gnwomsp$HBectEl-qefvtg|P*Bp;Y9`;uJ%P zSh7nAUIP;QsvQ(+X8T168-V-z7nm^29`|b#BwsPQ00k_wW*H79`qT_rrEAf*7RG2y z!u*ejme^Jcespw%(ChbIDbDO`GfZk;swl^;VVR)H$K}wT1aWQS>*Bd zdwF?*>cDAaS7o31U}Nb~Zln@evK|H!$A@1I%T~sx#bOIxw}Zt;(CTsIiJ&ld9a(RS z28u0D*MP7I4^!uqIzcWaP0&WBA$6_L$CXh{YSPxWPP>Tt+S#7Z&8t0Rg zLektijuYmV3EUZ6efY6nNG`N5L~WuyBsV%&%TPoDytAFfd72_^K?{Sv%nY>c7EUos z5?zViY|8tHb%kD!`UNh6lH-;W#hA`4SntVg*JVxyZc{!r7KvxYS^$>SPK@^Ofgw}y zLBx*l*XoHu7i&wNM#zvkOP$zU&ad;tB1|rfu8T@?h3`7`KJ2BkPk{CkBPP?PP!%JY zl5CKI_q=Goi|Qvl0T!H&*;udb2YS!2AKq>aC?w>>tUqDa?vfj(tx`aADl_l#K-?fA zVrljSvaE3@^$zY>Un3S+6W|WaFt!=)kO$9W`%s)rhB{yWQP1{!!)bvGc9}2#Y#F@3 z!84>(aiI}|)sDcrq+pS?Iz~cG97}peI)rdUy9h_9Fa@yg)tTy^2mz5thTc~*!_);? zcLH!8M)U#eUOoImJtMw1&a@5doDI2eroPSOazA;#e*`xwzgsGntMGpDkUS6V`Z`}Q zpDqF~!vsgigh2ECi)N|>dIXWW;&nAhhls;ruG<92;0>opO#>Ze5>WBQwGYIQD0|)Q z=F+$6QpJLdKJ${|Azb~g9+oA6fp~w|)+3i9Hbs_1D+Jk;&xUyk6-UkBqKuP6T)UO7c!DN-k(w!{u1YaUv`x@tB&s6^8u*ow~Xli+esT-|IeKjS&Kut4r zrSN$DWBD^8N%b-CmAe~k{r|h*_n&}`3;a*YJ1=C{dXZzN2`OrJy~f zsrgEd&Z-Fx*xjXb{5uxBin_!w-dQGes2TYNNR0NZd#NoF!V#k4LiAw*Y23nI?64&% z7E}a3h6}l|q&(DWvJQuz%q^Rf4P+SWIFWj9&vU;JG!)_hzk61Z$3snzDiZ9QXr}Q- z<<=7KN>MZx3TZVrYUpH#?xws!qFg;K8+U1m{w~&d*hYcE3ClVYBJfzxp>Z)Bt7#G> zBC|)ZYvOUP5TB!%}KdKL{rIuVPFXh2rf` zuvbX^BOd(QE5OO+&41!}b9*Nv@Q%U%Rfzdpnc!bI-VTva8I7>t3FB1$CpB+&*KPZY0b2y0v3W?0i|^B%v=$!Uj_ zBq;>nIay#7uz<2OsjAF{hT5Hx0Tu|^^r>phxZ-dc-HHy~26iB&z#C?=W+HLMO0fT+=9Z(cYmy2?3RlN!Zjg)tVYDsS)Sa7UyP z5C4xMk_p-L@4BLURm1{yblrKE*e`_@WZYNk=jtg$fc^G`yXnbxl1r`yg$f@7>~W-s zBYRT--%Lu7+j?CimaHFy?L+`}tbW=(6F$l7)}S~|u5ePuh@hhS>+=jujSAsZ^#;mPD(ufnz&?a7WX%jPyLPL@%FhusVvW1hpbGT9J`*m zy0Xq<7*Ld=rG6}Y8D@qv!=LnNG_%dbrtwNHqYMoR2P1hx8P+bA z7PB_K%CDQkL;#G}G05)acAhLr>6Ph=zIb(pyI_~+(FKpnJ?>?PSDNx;-XZX)5^3gY zS4(?%?rNR06A%5cBp}Zpt%z0;8OQeGl_x4$uzvCtjO6omU?K2fI}8=q@?j!K; zGw@fr{dsSfzVCQROc)f33;zqy zOVBIZz!{jkS?nzMnp2Het*x~|(QL$lxR4RSjO1|q6Klc)N!M(#$(jCwl`{4&O8h>}km8@qB zv4f1$Em(Ku8}-qb9<-e71@%Fv?wxp@vRv!8(YrX(bU$Vgma-Gcq%|}61T_J#Ix;41) zN*FU-lDM4Y=>b|F-&2X;R?TgD!}48-kFIjyyTV58&@?|s8y(r`S?4AmW^YbWX#i1d zl{9=OP7eCWy4ejr2;%|7@TGKKk$k_#A+Pi3audsk$?0-nfHr>Kj-72Be5o5~S5a)0 z&B}RjYCykRM$v9%d?K7z{JO%j#chKEgOMyAA5OkXpu$R~WS(S5rf@(?{8fc0UzJ^m zGXblA;5N-5mS3pWm?(djzLv!&j|SLnTPvm3xq4+@;#Ck<2L~pevC(f+v91aGiQo3S zF)uhqvUd(c%f;plyo&BXmpers3XD9VPwgC`yEvb~;cZNo(e#Y9I=%n&dEX@~JbwBw zIU*7Q6XoAJ-v0#j|BlAs2L|92eHjaf@h3$SS(+J{{?A~{o??`Nh6~%O9<8XGry{fJ2GT~KacY` z-{p6Qwl)ENh<8T)!autjgSFA|G&yQ%5|xB&BDpyLpQ6lxWIJ8w!EKIAD`8i8CLo)9 zF5_a(mvM|%hrIf5c_5zCZl@-z>Vx!fgvWt28&;))v@7-u5p>PDsRCv6!$fGy2QU?< zjAsFbZygczBX#uYEm=V;TdX*|0UuMf2I3^5Pfje|#27}fb1sxGp|u}syt1J>}@ z)zA>ORlQjrByYsVFxP^jDK6*k0l4=Ez1%A~rDtg86&VXAypD@c!0;-&I+UMl^VA$@Du`i?Qp+F)psZIuj7X_&e;pGiQ<_ zO5MOb5G8U9JMZM#2N0aN$zH!G5>zuzH%Z^7VKWhRf7Hi3$==-HPe>-*b{lQcv<2O} zjnVIlTt_yNQ7{v@cN`R`e=vy5I6xXSBVjJ+4%Rhq6%o*^*&)dgX zB^PbethpB4PhtO9j}wR=wPlvIkogWCI*kXG**aIW(K7(KZmYsbUG+(@Y0MQA86>1x z7o%C{^xCL(W9ABJ7Lbhqx?G>=75k!pW%T(_zke7}Z+y+neNC#P1%PB4F_OM$;huP< z4#e`ti0>%ZSDyy2-ZMwKkDdcgv*yWi)b+ie+v<&(d&=a0X&e7U*+*9MwPPRp;VKN| z{29!;R5HNap~H9|XUdn!%f1>%#LiS;$y%WBy&PJ(&DBP1Snu92OkFbDXaZZG#5Z4( zoXcDbyK=Pfc^}|$F%`x0utUIxa{sIjDndQ&+2MdTFI_rjjU5B!xr z67?L@;u~*74A2F7&Pd}WusRJV2q@ui(pG7gxk_q8uOB6&ar{90<+FW1^L^Iec(p~v zy(isCTZVilH{>q4k9Pjc6C0HL`MJR1A?Tm?{YjSp+Y|qln3Vn?_Zw;N8!t(n&VVmu z+4K5w)>0mHq)OzI1*#^9Qp8Pj74LLU(+>%SUSq31QGh!ElRd;{%dPq)s&d`KBV!|e zY->kgg~Q*S0A!m1ngN<2&4hEwzHdk(;!@&VVYCIosWQTS$S}}&Fpj1`Y>HSZQjiul zr=vzM#!hf^;RZyq+E&dM;eNh=pdDELOE~k(-~&>=B9L4bXW#HFDcduG5QD)yZs(`9 z0uFXJ$o?zFIa-Y%Z|6M@y(M#_4oiP) z?8qlR3KhVUPdfv-5YsI=t8Nm9*Q;v6ug z)-QhcR2m!iZ1FYV`d~Z7clqFvi2ZqejZTkO`b1#!p5BS&UE)0l#T}7k>{R^Sqfa2~ zMwv6==GTza-N~ipse8FDSM}G(Ev^l@XQXKZX7Y9N+Y2rST2}ub3KrGjxApVOHCCaf?^T~gPOY#!wz`fNYyr9huAy}oEsZ=HfV;Qvt zGbw}KNb&Jc<PKRlhYQzv#8 z*0^>vxPUJ&cQA$!F~}o$)yP2J4~p#7hK7?JQq`C^mfK4ets;s{uS)soT zLfD=)^qj2hdUP0=iCpclB#JjpMVHEQ{zeS*vRNj!S{1+?V_2m5=?(>Z=0+IrjHxbz z?yhG$S&e+)`EB=S#3d!xv&8BYK5?G5zRlh#*iM_88ael3#^UALxkcd82>hqQOVWU^ zz5E&ZK@+5v1+6EOggf{3jGBS&sb)$gTjn&3Cv)HJs;$+mm7;psm^{xi4fhHk2H^Ve zb!*M%ZQ1PkhA3%thY02BJ}9~}PU86|Ct*YmHQA^H_8lyixkr5yi~UqG;XIEJ^9|1y zW@f)!xoJ)BMz-*th!O&LFUTybr*ZUYE!yR0-OZ0G3$p-M_zP-f4iVB-8m8-?KLvP3 zqCg~o^Hk#Sj?#mEhviQZo|yZO6d`ptC4W(V<~`IdH9#bEQH}-yZh@fANBrFR?wuKea6q=^G)P zHI(&}1I74Yvz5r>FV-`GPAwf7*adI@_cavki@%E#&L&1SDI{L-$baSCe|zG;8lQh} z)RFmKuUAlEI2;n*Z2Vm;c_Tt9OdOd4q#eab;rm0+&eiDq($QOpoM)kb2fWM`9CX>~ zpp-Oki=C0Q7U$i$Q~NcDaw9o#kkjg`jeN_hZo0>+Zl)R!5TU{XD-3UXAs_)4ur>w? zV+A>)07Y2K9kwu+G(pDdoqWqmian8#$cRYrKjD4WS!JwLRwQ&`GAb?O#)G%o|@6fxUKBi)m-2?ukK z3-OQ&4N0yOO>mfX+beL3e|@Bhy`@_D5N=xFyogJofNpO^oq8881)nn@y-AW5O2%(m zt10aFc^aNNZp^x3@r{}mzlfjs!=@SYpyvp3$02l7TsHly=mS={E97>FI={HG4!?M~ zbTdw^3@*lU)F%rBJzVmy4{v`rY&^% zdfPQgTm42iv{=mCKH>``X?5PzYSk`|^8s_0Aiycn3 zk;ndF`pQb4RL!<&6+8pF# zhc$Gr(vzW+QVKW4BYg!da%qdmeRM_>V-c#GQvoV>q5sW#QOFQx3d7$Hb%Ld{4 zN_A_=t5yu%qBw$-l>>{@3?}#vLF|X&A4qOiVg&{2c$ZAEPZFltGxXV(&L36pv2{=n zW@xVS1M!6t-MBZ@TV$&noq{*bhd*T#Jx4BNBKXY_IOS8PCj-$+ahpu{ad|Qdr{Zz5 zM-Ub_8D27-n%0{+*jK`-p*|0#K3Mb%#8n|HsaqfgLciZ$lOg0u{Md=q2MIF(gV$l{ zczp(6)4&8G%3YxkHq>26eGpnrlp~w0(o$k4y*}*y$zX#zOPiWU)LJPUk)>H4u4j^i< z4Tc|nD*lZi{n8xy!XQ!XVgB7SsT z{+%vcVRtTTpGij_eSxmx#M@l9wIO5NRvxUexx8aX;fhX_I?WR^I}kTWhtrv*Z?zvV8J1S(^lZ;)U~L zvWodtE-bFTx`vG$P?k5K7hwRvTlHXm0?Y@^2cViv={+txUPL9I6SfdXXVKDj4$#yu z5f%i-V^`BAJ%E4CRv2n7Nh;WSVD`#?8Sg(Kh39`jir;bFh{D9)#>ULV#me4}LekHWNc&hd!r`>g|m&7iJ6ogSozA%=zo_VIsQrn4PY7pSQP6e-6TX0mTLARKJl(2_mzknr!#g?Yq!bSLYea3F-J z+Hy$IhCfzdbA>S^zDOvK|9f2!WOgHDOoRp!`=sVFn~Tica>vbR;si$Y>r{P;C$Y$} zKg*e%4BNXUcS0Egr{K1OI6hAN3^U8{K8-7Oy*d|ftqj>pxbD*kLZI;;KYHqOhXX@| z%XCeH>aRm78A}m=D@KHrcer`|O2FyLHY!~Ovdx{+U%E|%-n7-8u!E>^Pk^a9S&_)4 z83dgs4~ZNGvKjluB7u%r&Lai{SdKBpcbv9yaLQ8a@-p_I2e!*tMsLT;G}aF<3hkva zkuC#TxDjls#0zvW^E3}^HKMR)#!jYG{6lzTD)PVZjYkqGapms6)o|y2y#?whI#LYY z$bLLnk~Q{8CpA#`S}`XYMR7{WNLwPjGMY)08$_vG@V-Sp+#PKsvGybuC#<73o4bg? z(`5zCN{duvP-&Gp(QG&sqq0P2C&R|6!6G_j{hkXYq?4zAFQhUJ_A-%Wxb?%Sx#b-$ zX=o!b#)o2H&2-cO$KVwPCd_M<$Mk z5d8K=k1^A^-h;5=9Jwc7jNhtxZ{6zq#bi*3T=C7LF6VmVIRQzQBDTi|})~s5knEty@&z>m{DDH`(OZd*UV$%STGVloi~VMz*&9X#gjm*Pkwfj~LlM z_BZ|KF*ExAbIksAoc_|jme19*#BG6_p(E)+IL4t>6Rq(8lfY;Vx8l)=UA|`?bh$Oz zxx+t{&CH`wOiEz>R$t(fL8tl2f6iLxy_1m!v>zKu%V}@-gL1%)z$l{RjgO53-)d@t zs4=poJ*bS-yL9Q(C+T}!Ngsmta5v29wb2m+N%6(e=DI9}J_Jx9fyp_@c!82bfZl4f zCNpUli9X-R!PEe z;pPkKopo5mC7EPNhy%4&d%V`@)-r5Kn^;x&Uk6OYuR%nS-@qa+p7iFD+&Io8c`2ok6`CKa8_^4#|+Z6;1G6g&KGP#_Xy)N4{;X zLRbT{IyqD#9-}w!vfkqb$SkFo?SCKEE*Yx$3L0!C4nkPe9IRx6ZcrZd87Fzvraj1z zZV}BIrJqJp?sRJ|YA^Jr`RJx3mC%Xi^vlIuop;*Djv$XE;*tA(SQp@1bV;Qnuzx~! zvzsNUQ@p9?>;gjqlrrTfHk{ zOi|Q^)h9)n;U~gt?M!7t$>|}hTr*6;YH|`Vx4KfkM$D&G@n^k(k6c8s3#PA}aq5qa ztCT#rNEg-TfC18Pc+qIf%BT~rz|ibB{BdDIlz14-H)Z*lw+>|oyi#KjQ@#i4C*|H3 zJS33XlM^`W1rRaFYgP$)+II8 zfce_d5{l`kC0n$XXfONv=U_}oidB9EAB=(jBI7`UnUnqNe=#S~Qe=3*lO#!h%eu?k z%PbYv_x@h;6yEyAS|~?avO~~VIYKHzd`&)Oy0-ers`0zrZ_?R>;92}$e%R{(?&eV= zy!MuV3Y&#_TbjN3*l*YNfvOJpiV%(<{aem~egd0UT~WO{&yYPDP_&^KdmWSE*yBhJ zW)(qqH79idsDKHfBw*p-OFDD#%1&}##kl^+aLAJ~&5Ic7=5DmELU=9Fx28#^XG>B|Jb5l-Dj5?UXIwBn>ls4uD9q{IKNClORcdnzO{xci$akA8 zadZ8e4xG(}kB*EW<*AW+ zBYcY%qi}p6=nNk7G>$BRdqRQV_=zI4RYKb$(ilDZ7wJmAVY_9?UYdh@s#BDqQ9N5A zv7{f2w{!H&t?@q6;Kl~t%m&vKvA9!kpso>fk}9JE)6*YUtDPt3{0Mfnmw#McQ}~Jh zD$4!wG%TR4kpsow-O(gfTZL&wEWZ<2`lX>IEb6v58`%A@Gr4)JJ)h%xOi?gF;_WL* zh*;km-Hst&FkfNt;qt=#KAdN~5VQjnzEkdfCL#8+8*dr+9Mc(let87LlLU`~KPv=I z=$gO&Y72M^<~m|wf!U4#2_%gT#R=R{pp1X>I|)c!OhZX(wBDc487(gVKziWJ_3(UKPk6GDzl{c=M$>Z<2MjijMY}YD^z1nZjaO4|Fer+U{jP zjv_s38!H6ys!loWTczddeT;6A1AYqLrM9Xp;~i&qn7@p-Pan7K&Qr~?*`I=+GTE~t zM7W#VhOzyqIQw+y{koJ#mA|j z8SmRDvoGw2-LBdSwY3ekb%%qo39CS3z>~YIb5i;)!pdV==oNgR>E1;}IIvZ%ao2V! zu`AC^)zZ;QJoKg@Aggef1lz{KW2$rtPsqNyB?zCiLUf5Jmn~%__*;bO$(voWyUZa!h_TA-wUV;BL;_}CL z5&oV8D@^=D3<9lbNlPrkXhXCsZkhPI$!H0Q6>8@7=%)ryp&_Sqvvz=CfYvwB*CeX= zqWHbSFb(|IDBM1_!ngv)lD$zL9=2Uxuolz8<@Fqx4zg1-!qU$&e*Srrw4Xd$Ec#tP zM!!k459&9Tf-Q94-2p&Iy#IrE50e0}l2;;751uGn8rjCGn*X*ey_A%V^lk~Ljjm6q3HPL2 zX&bAcT(L^v|1rKZH~j0~JIT!kl?D2bOxx+!1o6)~kz+qCv1+GjkGmt7xUy9S*+kW! zYzHR{TzFe>k{eB{b9wRN8U2tpvlL81D;|vuWuzl%S`V>}h|9yB0=Z{Q1$5rBx=Y02hh_DY92y~Kvb|QuCWaQ7OVVFtN#+rHgPYM!&m@2Zf zjWS_uGlA$Jq0cWxBY5IG+oQ299j$NSE>&|5ntM*xO9+r{XG>(A~$)&b)63vwG(*{ zagNW#HpBeINKQ_ngL722F634gK>GQCf=n?=DcsJEKS8Hw*w%(HHk&PyO8&7^0P?g2}X2g;Iq zV9K3yUO_t9sHlyU@9~e?e`CGJwg=c@i~cLGgVXvSZDepS4mD*=89XJOl%S)6$%56d z@TFQ&unh7#AY8fTx=S0HzB}QIDr6a|;jvRWU4A=Yl2|(FS2@^$I$V9#I+RqH2Jh4-y)tLlr!~BXDBqt(?+;d= zdelb-&w#eieHXtD*%o|q`QUYa6nBExS0?NPDFbA$r)s-IPD$+mzMDQ56KQ7y>7!{{ z;t}FDHfcUyQ!SA(kB*BOWBOPkTD6ceF9Gk$8U4RRnQ#?2K4=lpnbBFjgE+((aN)8w zLZ^L5=&vHXN9)DG9YUUsfQY9GwFw{oJV-N|t@bp6ifjw&4{gZp`_+jzPwxMdbnaE_ zAUxO1_Xw@D5-&s>IGJn+U4%h9^;sv{Ao$o%`9b!0Kf6o4{Hd&McWO$wrS`IDbQr*! z(`>L4v2N=a{aCMRD010IAOBpIgorJEKen{P;g47Lxa*7m2))k^ftT8q!;#a8i7@?U z0L`2bk$JZnlB++2`Lr39lkn7Q6?7Qw&U@O4-2Y?OH|G{YY~-We-K|7+Ol#ZGT25!c zCgS*Rwv5B*IkqXGf$--flfyH9h`*OU#j-lsk%OHI_|GT^=0wT?SLDH~!d~QGXPF`d zA!yag1?X>3XK~J8jSDCMm_a=_86NK6yrprPEHaLnl-7Eh1VvcR{|w&O*|t^}O0~|X zu{oQ!-J6e%Rjmp5K^B9D$mC!^VDi!s0v$plmJKy}uQJU^kTGQ#baH}X^%pXbDh-ZW z4@;F1vK6wzIZ#uCC{1zsr(%x?GUe6>n_qDtLSDnror_bzth2=KQvd-ee`dJ}>{P<3{Gv z%0oD&weZ=KdKyl&FAx+rHdc@C0mg4DDFHjtHnh=!Za@`}YqTIz_UwV!jBIanR#(u^ zn&Fr(Zc1^Nud~Y?Y`wXJwW$^N%TH$@cTjoqAyGS;K4-mr0WHVl$C`GBa%le=%nlq(x4>xz=E6adHE?yMZIBxq{7_T zOpp~ibh=S(MOh?-osbBcg~gAi3#Jm1_PCX?Ke|S63-(>gJfarF_rh>0r>CHzm9vgiK`6e6ayA;Ob@ zIokM2Sb7q%5ShslYF_;Z#mvQ+QpB>B>Gg85>57u{AI{I2-;Jbp+MAJXi1P$D#8zR#AawTcI*l zLds7s1t3Fo`R_u zWGBxdPV|vFqx-^Rh-zJQ;Ooc~nxnHd+~`wu@Gzd-dl7_zSLojs@1#FW>;_jn@&0)Y z_*XaKkHt|^&`{uiw-j{Vi~MdWEG=0bPJC-PkjF2HC;EW_A5SG$8W8VM$Cq4gb34%T z;Z7AU&f^Tihlb_rb-<9hd+lr5{iy8D=H)SA2P_Wq2q2Z6*}`zOLl^-asR*rG5Wz9k zN;HmxC6W{qCNd+nt2oSR3y@d1tpX}keol99E^KbAMbQS%b=`2Lyh`9fmdLV@1&r+CF#TM(FZ9uuh06tVYDL-PR3fCGUP;zA5uRfutdO9gjpZ zd8X>&KBh;QD09h)i_B^#xrg5L=j|!t6vBzU zLLq<5`Yw3|x8UNr4HI@juN>pMacLwc>+|Q*hW~ciaLN;GuHU+k|2G@@-#`K}rnKOt; z+yO=sTJbedJVT}+;=y~j_RHg~t+`WT0f=8{Hxbo(LDXW!=rlTh9~*el;pKm&>z!g0 z&tI?NWhqnV`D|S!A>9|t86pz*6GasV0uMqx>9CeP$31dr%k45s^-k)lMo#MxRW+%4 z1*d2eSO~?B61|bk*b^Au#Zc3sbG0(dAGyhv%CcQ{Gv~nBENawdqKqppp3DkhG+Wc{ zNe?SZjdEjFR-i0hiy`~-d7uA^w7Bmi(cKAitPAIx)6385v0RFv{$QLrIN>Z$0~E*( zid%~_wu!E_*7Uvh!3;Tlo!UF8hfk0(V=L5i@`ilWqK}6Oo8=0#e22O>kXgBmi@9GT z4`6}(*^+WrxEj`haQ&Qy>aV7dX@JtBL%UB|nYu!#5b#;5jKw7;z3Pe7Uzvogy&qES z3jl7g_!jRq`272MOPATn*HV2&(f^djZ>8C8M!)t$Uhb>M$NEpiaPV(v^=&|oVKXIg2;5Z{%LO@r& z^V6jBgpB#c@#cbMd7|D>>=L5fia+=;{ks((YSRbeqU8YT()N;jBo6i*5iPko6iPV) zb#?HM05O>?LzzqF#XR;tj3>8EO?&X z4&2&sc`s^a0_Ksrm|YKnTV2CiyzYe0qZHyqW5Mtxo7!CZ)!lv@)T!l*VGktXCZ@qw(c z?_>>^!7DJ{`E-(WTNWM|oXMO7>t=bsjY33El>=op9B& z=zo#k2GAhU8i)^BQ-L^z*N-n#w^WGX;kBQbfJq+&02fP>C5C_{bfJP4be#$>X+7&Y zJXqh7p7DYOw14q~bmuEgtY8G`zi9bsNl)wKm%Nqc>#XZ=&;CIO5Ac zV#{B?kit2FNbuiIVE^xZE>VhdGEgjNzWcLs?s~_cu77m~R)^k96zVhTmj_fY1`^-e zAjRWz#)5NPkBHD8y%lPi&OlsDbJp%3w^VEfYo|GH{-R9hPAx6+ zr8|#ydkR0_w8B=*s6nxE47k5Q>-)=#we39A`A&q7M$Zafn$@-14mQR4iwkQ~m1y4j+D@Gr9_f{x?vTKgY>K?h`&D*Fq zXxAy4hwZt?q(~WC-9^kK?LT8V{Qe**zb`WYIPj0@1OEH3K>fQ!@&EQS{(WZt>Rk9L z>HVg@@@Y(us;LaOO_O+?kuUORcfsYM!i_+Z?>R1m<`>}E(c>)=DLyA#wc~D5wAMY`GG6MNrj`8z)T#@C=s(2lpJ~eipQG(DG-jN4wH!iy>cn z8c9xHBYi@ICK+HT{2X|}Q@SPRt7t?kKR)3Szj|`F{;i$dK5V#6LA#Q$Xv~h zS|j{Vh-KBWs4JiRxxKZ{-l2oOR? z*M@e4pw#X}en0IIYbh(7%%3tW))|2MtL_eIBNI)Ph6z3`ANd?dYv$s= zxUIQPB~qIH!^Q5qMkZ;#4vIkd$-%M!Hh78Hv6bwcRcxl)I>D;C0~tSVN#4dOg&GhI zPL{v?ICKBI80>XVluyE`7Ma3H#-bBLPi&M~3Y^!2EbCn#RxJwy{2Qi>BXNMZVqDx=yq7r(ul1`F|k z&BtNO4fR{hbm^bw`@IwQ-zV<3`M_D~Uu&oeQ>}ck#PIuaP$Uu*-${{BY@x7F$|qh07Q`wHu#XDP-M+FOkrV8AA1mgoIdK>CgNbjz+k(&w%X6Np6c9gzPL55mdbvm z^0gUv;pb)FKe+XLRr~p@KKkP_C&leesy}cac$XG!@A{o7&E-k2U%=%_rQe;q?R_sC12tFlmIMWLY^JCa!k`re zy_9-fITcr+DY8^RK?;SK6P1*JRPt6itL4XUkwAFhI4igXIruUDoMM{-n<9ws@!5e5*N>|6BeX%j?sPmF=QRZ#-bTGquwtr~g|r?B~a=EURg!bT!CLGAw99KX)?qxTG^()bRn4w|~M zI=Mm|L6w(Pqe^Df!+Ne750tHBn@StooStmsCtq3nM#IFNw|@8K2L`-;IMU}?jeEcZ3a)ZD7R>TISGlbuuZXv4HI1(Z3?N3U5u z`~epuO9$4nah z%{L^`-I)3>6GWRM8zMi~oG*Gg%fpw-JXSADWb`w%LF}RC&NcPi3|6tL%0IP?bg(6B7EarTB|! zr)+o4yFQ7y9sBt%J3e%oYT_cTGW$v^pL8FbL6~|nF(9hj_=3gRmNnK;;6W|MHB%uS z@lS6?Iz8zm5Ns1L~KM zR{E$br*tRe3W!n!+v{E;AQKABi6#0+24HuF+EtCQU8;!Km-@PFj zWgvLF&AEv^ZbtATTGv)`jj$m?mF`=CU6<`Fy37@k71*~ddqqeiz)`t;%erN@ z4Rt)b>P5nGS6^t|J82H3%%)jPq%vRdrW3;YW-oG4XO3%Hf-|U+lTVYa^A7W}!YnVY z#?0vF&eV1?7Je0OTNJ2R3=SaNGh6}oJwBkXk-VRMBtpL@y?7N6{9%b(Cvmn3T;~xz zBPj%(Pif4YbdzaN2fxGm^8Z;zy_WV4E%jh3u>1_nyIDlg-Bw*np zv+YOuI3A%d{i|9n2=FSOM2w7v&Tbej-*&h}egNz)HCRal=L?0R4E*p^xn{hOprw;FcNk0k&e0juey7_f_1SqdAS2~cXFFO;@8o6ro zyfsYesZKka2izjH(HB6(+NtjrW^}Efx?;|Hf0>n3 z%~C|IQ6yKi7XGqo`c+xUn&RMtO7mkbeAYYirAw2^n zyM7Yl#WQv;Sdmm*81$_}xI(Z3+21Pz{O#ap9@D_zevPQR?T;TR?dtLiZL<)^>T8{-IUAvkPl z8iA;n%~M0MXaokb`z~RCUsJqXm(3evd$}IpGyqYrnHNPe(@pD>s*o<6!>14W@#$oT zGA}>gCya(yft%C!1=67sf-(^5a0aQEzh1edSefjQ=hw4|4ChtQCOvc$;#~m^ zqP7OzoFR8&*b=d#>92dI&p7hcqEH-$bxGS!{Mqq;OUZ`u1xS=OC(; zCxylhls7X}DD@c%EwnPo#mYiCX`n~r943-!{epVr&3wSF@0}8Ulk2y4$D+Dub*-F; zx^SOupMQCK*Pk-z1u&qEKSaOr>z7RB@7`X-{8xXTzJvRp1mMp(hTeZoSEv=(d0+!$ZeI;KFdGJNR!RDX}& zX)=1+qM|T-9jIqB;62WTm|9@nX|-lJnv$@#hyUgctjX&qZ81IkivY=;?)n134pPCv zU3^PXS8H+xLnhzNyI7}1xM+VrRHa5&F@O6SPK+DZxEn}Z)bI+45QJ=?=TP6p*I~aV z^v;z4yR}p5GRU1+=U_?Du+F!tAAY*x`}L`;U4I3O&%L%R;bQxcNQio$KdX;vUjM!x za1DVYQ{JJVs3>X)AKImo%0s1)HJcihEm1HA{@u_(7>+G)_Xft{^Mp;%#Cl`U01pM* z&!M41K7{XyW}~FMKS4EG(3EGwF3T<@5Fh&q`ur^d0!{h&2ZSU+Es`&7lYhfU0kkR} zRL96lDl0Qddn$UpDTtsIXCszO3CoVa9u_rzLL{%-GVGYR|G6|?L7_i~zeJd|g<4E7 z8QWy{0aU->Ys=CFo2gC`>l9e*$nzzG7=~7pA&mht#fp#z^V9%#EJjby!5YzSmy%E| z4(f3MgBB-Y`ehckI+aa3_fl{i32mmCK|+^FC=JJ9VvtgJ67oU802cA*3O!00r1>-$ z--(&(LW;DfmZXW~$&Z~*k8Li!pebUls1A*uef}4AHZsR2ef`+%pTN_-3+cTMCPTM9 z)gghgq(MKZHd<$iash?0cH(A(iC$RNYj?M}LHF6sN1hP5!j{A(-;bK%g}JW88}#2$ zIV438-l9?(pp&aQs#GYOB*c^5NQ9y(%k)-o_3}6vZtx+aCWLdJ4YU#3vjki;@QU11 zt^J5s%i~12!FCM4aNGPxk(>BfjH|d0~Wi5%lb~3 zcID>;JK+P*IeSW0TN?f0EHg3T65fMByKMc#j&=+{uN(2zb>XfcA*-s@LFn8@5#5!$ zh%qA`Fz3BJX+G&$z{*MHx!x45Ovuy89cTA_W5=9835%|W)7JSM5M7WoRDaiVqi&Tf z7qCYAFPsgak^iPUSgii+_GaeXdGq>}Uh+P|W-lJ)c} z8E<%kKtmqD<9=v`fiDLGi*rlfKy^tx%(MOYJ{y<+wDt**^M#-tjsKb>N8dAZ42Av!P%= z&;TAMj;beSgfN^{xI~FX3!|+b zXk|Kzq)4m;i9ecfwUV{d-N*FT>ekcW>s3y0gzJ>P>S>g*da%1TnYTodSm}Nl;`tu9 zODE21q(0-9w~(bWwQibH+QQ+N>!jnFxH_*z`7^*XIn6U`0?-_!lDF7y4d;!|swbgk zqr1GN04><}y?K?an==&;*E1bR1m1lJS-a?(!JA{#ElPSlnn7XPazyWj}qsmDHCrST9QME%yvH^@MM=W`wUJ|80a4YJ3jzQ@h4 zVaC_TEbST~tYCh}>VG&(|2K-T{6Y~L{-Ow?^5_gcA5qGX6nSj6l^P-?7m7rm(4WzG z(b29b@8Wqv;_H-hH;mWE#ynYUP59e&*IrraGljMW#RXv_*5UwnSx1gShzd!fRG>8r zu%!}97q@tAI9IH3)N%@z%b9v;W~GB>!xpQXyAhWi-|E9mIYw>^P_EepP%Y_q4f^V1-e@Fy}cq0}GqZ5PD-c3TX~kmxB~AbJGMHTPtgNV*&mz2g-ZsNuQYm z&4oSh89+l44ZN=0&hVTfBM5eFIPX>iD3f$)=8Jw99O!5vOZXP$g{AW0*_ALTv%2-9 znKu`FdtS-itBqNh`5bzM^p2-SLu)2}cA(AqezI#_FK_&VV|aU${AE_cQ7PKHPfdLImIeAGXj z_=*hJi7!IE2}PUy>~-qiWlD)PgL_|P^WiZ3u1BfGBCNi3JN{MtV7=u=unxvL_$G+$uey4&OF}W^&*^oasa%( z{Wf|k1?4E1oiu#b#h%m*8UHsFQy%o>3A9o%; zP*vfHXy3V{xOSas_d#6&MglBqZw+=?<{Z;s&#c8G;u8lSZ@Ss+*_mlgp zY@&A9h**H-Bd93OB*pkz;ZvuvCVrf;h=A3TJFf2g9{v1JGl3NEx$J>XXR!R=MK~w| zE=|%in9+jAtdS)uzjfPPF14@gFFdfg@-#B|O}y2n{`veql?dIVd!x#q)EMN6pK;(d zf&^Cq&Se1+jEY9lkVdH%%s!=xqeR@qa}veK7!7N&?!GQ8ADx+_XlH$p~iY_Qf&?K`)?!n z`0qGKcS=gMa~OB6E%nx9?@wREOnF$mpEAqz$?!wJ9~cl(6N1X zzTNm;Y@AQoSPZjeGQhHE)w#K!j^mtXQ;hk%Vq1c8?J9tdI}bm8%5y4tlCQf*(2fMF zNFGHBt(c~_TY)ktkIox}4twC2ka|E6>Qw?wl+g-WMo3GhGJaH$GKsz`OY>M2m&?Mw z`%tcjgj|7y@Kh$ztRr_O%$#AQ#?`&tLcic9|AI z=oE3m-Nc$n54!FC@=8zu3GTM~{K&`4L7wvU}Bf6nG2lt9UIDS21BCGsoArtDdLaw_d*j)P5}- zC2x_r7 zq9N0v0|ltl#NX)1tQOI)%_yBIc}t?POroB+J7c&Tjpf$^cwj{a6sig-aGoBLVAErx zKtYhtewZ>gy1Tezqi3hTy`v2+COVy5)IysuarO~D$q=!&_gX#x?8M8 zVbU{*;z>;05>RqiBAFs?!{a8<1!-@;2eGJ5Y7@-3))*|o-_QNYn5;{)>a&J+#4^G! zuk*t>I^pHepj`icYW}sc9ji~5h$!CbY0z%PRae6eUM7S3hE!61uvlJa8Fri~dY-MNv zVbNGY7L7)K5EKQ;4niCZoyX{(4z!>GQfnPg*CAJKMLQI#3TU+a9~xjQiE;e(J^ee7 zDVU`kTm~}nk;!`>0~Tz1!a?!rmKH=IJ?}x>$88w#yV$qDpg{9((H3c9Y$?OZ23S({ z@Af?I+w0O;IKacGz1N~hY3*^PEWxVSO1T^j2rUwE`9`mrD|16su@ZKuv?+`NLP5|F z)}zHwV$vv&5OJHYP~?eeto#tzMPE9By??lYE!lJ1rL7P-bFccGq8%~3L-68k0BKxT zFW6in|BlVeX5If4mOtuoiRuroxV`AlYDx7Ct&+zJ7VJu|~o^cGuuzII7M{e=j!un8U>@OcKR9Lr!4W>r-QHd>j9;&yMzK;SbH6{k`t= z_*UKQ5f&*O8A0L~=L3+-?0j-i*8&5+>0**G~66liT75=1`*^ zgIlk){MZFJuJQ2|CuTj}uJV{{Nol*Yq-Bg4YBF4H^F4+fWj&*2@Ko#1Qq>GbC}S&w zopv(xu|qQqHH~bzT(-bW)WvRx#l%AjW9xwn#qOcVCby+nafVLevW=ApG9A1)`p43Fu9B$6 z2j1u9_zle+_C!IRq6K70ImG5+%8nTX;nwwpnTR%@1E%TuM=E2IqBNtIDg?`bZ_u%% z_YmHBM54=t7e{g9e>Jig_XsP54uYHl+ji&ZkJA3h=TrCg65I6*4Sw@FM-!*xmPM)`nYzI2$BTArj@>UWa(jF!0+ zWc;C`TtS60li0cLUMfenYL`A2bGYb+MRm03w%rdkyV`K(q zx%fDacIIBd`^shDW{rEI*~uk+%1$j?zlX&~vg!VyDG%y*oO(tu&}heIz>k0(fXkL! zl9(3HCkLhJ>%kT!gCB!^P|;I2MwQNF9U_lPSvGG~oGycQYD2-}ob*p}9&BmyoRASG zy%Q@x3V4R-iXAj~V@6I{Bg0oJ$bVAHRSREzrMiwdVSTQANOSjk5SBohYjDf>TYzY+ z7l*PZGX}wU1eDqn#{_|p36RqZ)$^e=tqo~av7W5XI6-v9`NYxb!**t-{u;@{L=7y% zgd>H}&(2uwF8=q0=@3|}8KV1Wh=C=ms+jj&F4Mqq#JA>MR_knV*SC6b57;%)3Z9f0 zXSlDU)!!ZiU>Z_4cheOtY17q>7r}NgTwr`k-*TU;OG%T7wdbtUg#wu#QUlf=>PMnN z4^F~+Y2x#B$SFT>JNMkH@6<qS{srj`qqr~I1h`zznt+H znJKM-hWXxhzTui43;+x?PXB?21%4YaAZltko0sx%5&i z_1-4HCN~^f!a00MUQzNK%B`{C)dN)S+_-(e)v5e&u6#L3>^-9a_jza@3VnJg?%4icnH!buj2D33%*O;V<~kzgHTa!S zzS1Fa%gEuTeL>x+Qj-^H`G=_|$xx}#WppxupojR(U>3f>5RAzVbE__=9(K|pZ!!Hf z^Fnl=xshRW5K&N)Uhji|yy7GI2d?Tih6%j(&<3)Ak_8tZ^l+dry@TBy%Rx~Q$`b?s z@%dN1yN92k+6O4L&Ks!5ItJ2CS0Mk!`6r{`iQfFxsP`w2O#WH`O8`t%lC=fx-Smzx zus3KSFc;4Cg3(DUP>2^m$c2SdB@~=?FjIwWl>GuvZ-V#EMf4dJCPqM2Kq%vfVu&UO zRG{6#&dFFh$EAz-<>mbR*{eeh4!dQ6QGwMCJcu&rIlu~w9e?y&`jixbNf8W34A?!^ zjK-KQYy|TWOHNB7e<&aVH2so>h-9}j?r@XEHaY!bQ=vZ8yj4?^)pq4;T*_s)k2C=W z%~L!`!ClvZhfNGxFg17;sJrs}oZr1}HA#h$p@=BpNkZu+@wQzGY+~ruZ&18sHK?Pt zBUrTB)ogB-x-f0H&%MdN3S6k-4KX_C2IVa$>VuSqmEAc9UlW0n`Zu|sTKAEOYLSnn zyPT$#9k_kw<2EbJevF{~{7K+Q5z^h@b!a|16WTBJ9bQPTl=Ap8dLfr2Uv|+niovrh zTGO@_`K$GhtlD?B3vrCxr@|?3Q?d1JNL{kb(velSQWYf~AKb`W?B|o6h0Lu)qP4$L zqMhu2);*aqBxRKYzTf5)f*RW7C^U#R#V+&4zW0`QR@gon{qA?kbJcJ`7wPvw*u=)J zKz?hw+(*pkRr)!RA_J3&bI?et--o%}wodJ=ty-t@Y|=-8FFz1bKPSPYkTZ@UMd|rG zev*x{M;@yhrRJEl#wZo9D%D|?#?q4#_C1ICArBmqrTg|BK)#b&!Wb`f{c8;dO9u`wC=lOpok)+dZiRqgkDPBnb<-CLA;FQc$Ezuy`!PT(F#g^y+hW|u1j83plJE$ z8SaPS6>0{Q{50gc+#1ZCk@w52M*ANpoy~a>281UVuVA@he{ue!Fy8VKP$%O8==0x! z@_)9v3W5G`OumkRMfwMxsqnA<*1saOU#+jCf9(MIYi?LYX3hJYKtDH3oda1-?Q=hs zNT#%6bXh6Jdv?0`FGiWx4J|#-6a=h_5^PL-B^Tn@A0VV_tjTM}Ii%-exnb z17E4w2j_Pw?r7|4Y&MPA$8TfwQ}>@RMQoP)LYg3wG8uj(F(nLiH1DyKN@p?eu~$lO zW|iBh3l*g`GfZvrb4OLTU+aluXQxkX3PK>a*+0SwBe!T8h@n9{S~d`0s8q0iaCg*h z@M8;Fgy00co%eYwp$nNM@7pZwi`h0G3-s9-pbxAG7D6lmwLSHY4z!sJV2Y)RA!oL0 z6}xJ8p}Pz_z6ZyXk1cZv8S*&@e{JtlK_UDd3}cquGRncP1rQ3emSmJt&Mj3LY@0r{ zjwG8LvoaZvjmU7N{XI6{k^Hq;QeH4)K>_?DKB6eV8k;APw`*V#%|_zKW8{A2FwhlE z_MLwcAa&C&vcpv+UEd-hP2U5S4;Aky*CfvrpNysZjwsYz27NN^!OuDXPhAarSNy7F z{TRB<3`%$@aocP}fm<%s7;cBkGL;B}tH8Oqq2TOB0#(Iq{M{2Yiuf6syS>qM7fRfx zW$?@hx%?|QF@ku&S^hEjVnI}uE-#dU*X6?x?4Yxd`F7$Tr7PlU>sarU zV{2h9yMd!VplQ`Rg0=AVQ==wQo>&Ztu1wY8P6X3{Y<;gsb&GvcYT2snfCSjk+(*op z=*Do1l=Nkx=k9MO??2x{ZqOgeui@a3|DRsMKTc!j%k+EDXrXC?M!LcTEt;^88(~iZ z$rhqAk)cd+W>Q2Yljm-(%`u=QVzpnTXV$;@lc$l#72A?OQ*K*n`xl9z;JcXYZD*ur z^6~?{f530h+mJBB&o`9SV4x_r!w5&xh-F~Y0va@F*)f#S z`aUVfwPNwsWyIrYb7*PECR2ejG5ujnLZkBhro7>GLgPGPkMxduK;e{by{xA$ z*B3>6S!4|9(Ml2>{da9g4pKj|t}`g$mlI{332~8^mo#O85W5OA^GooLQGmL*RX)}W z33n-e@(A8*S5l&%TLqiWgj|nODOS|-4PC<8&6Gv;iMqj0z0cRl1%P0d zr>$e#msiO`dQ$EH6eFAb_9`X+a|?vjZ}#iYH%I-zaas-O=OC$~0pzv4u$ca9_`o)@~)256;0-;uuF|4n8qYGh{wdA)Ai4gtHlC#6CB0sSH^WL~tDXiFB^HuO%RN1T`s z<03RfUet?x%Mn>O&(#mbAa?$t$)n(bY?0%Q zEkSx0`|9-YuRbH<3vb|-o14bvEB$op9FVv2{6Qh{!eB21Q0I6vuRq$t>0YsWb>1L5 z;3-6!^CJ@+@;*wQgf!y?C8i{Olw<7XRzg56=V&iB-1t^bz+RJEmj5^r`6f>gS!)0) z%7?yf6#9>nZrS|a9j5J_9|LUdHM(v&NjG6K66}!kQGB`-!O$JFW>3mL9&%)iBcl$~VlCFH(p29fQVUM-7W^(7jp6x>Oj$9~W}VM6s~x$Uwa<>tOgaNjhdStzD*L#dD9$ST z9j8z9xsogFm&5Q%E48?x5>A-nWCY$*MS1wN150Z|GNV`=LoRdG99!^PI|@;loXDpi zmbh}E$TSKk3(0JNmGNPEYP>+CE( zY2&$qd-ZEI!%i~D3mJwTMD&D-9ahX?Guw+bv-E0krbo<;(5iB)mdO`2W*YKJR)Z=iYIzOecHi+z#Xw>jp-08wZ0+el6Lf(x;a*rhT89hApTF>uMI||&E zh>gxcI|`QKEhx6Om{sMlQ>h9XuId>k)N~}-ZbvdfWH4dV`=6!^i(L9TRTff7-vf#E zVI!&2M)RoZrqFAHPHR;+8ke|y+952w9(^QTcU7IXTSEk42d0)SaG-F8A=(Wtj6IO_ z;>$OrHccBIqmWxddFz2_l`}uChisU%ls#n(g-8QYAM_ zRJZG?*A{p2N-ULB=?6zOqYwHiX$Smr;t^lkt`zW`E-?|gAj&aWpOud(YoH%NYldcs zs`1WOws-hZc(^rKxKjUX%SbC%Cyj*do`2mG)csIccenl71{0?145GZ@6T?QfXv%YW zb^wDK5O27z+5lYIR;I9zqkDP!sui+LTSEr5;ESjxrU*(-;2K?sX_UgODlHKEwNC#W z$}?O2-RtOtnQqRs)*J>Q=Uhv(_ice;uL@nHscs0LwmJ7V8wy@=US&`?)nfVw+mD2I zLNhcXn|;t&M0j8e3InZxuE${bW*jzY&QK`K8YSup3|jmq1c zZyeEvva}AUTdFhfvw9vi5M*V5Ih`$(r9>-TO8tQn*RLI(NyK#a1RdnKXdN>+rd>TX z1Cq=%75mb6ZXZ!KA9@X0{d66{cSo_Q~ zQh)bW8mP3-ijH4Hf5cPf^EEt*}{aTQT=2Ingijf0YrUoN zvKSg~IBf-T7t`m4*JRORvJnLKGO%Q=Lhz)8OfgB27A9QEn-&qMvku0oLD^jd#qC$Z zx+y$MkbK5P{H3SwB?|&LGoelN2*hu*JkJ)7R@lS4qU7GY|p$0B7<`92-v)6N84G*=~dmalPB~9HGQfAhlBZY6u zO}Ooc<%B3HV}{j>j;Ne6b52!*V`3-UO&i2IvRN9WEn(S27c?F-RTW~7=*pD0iD1O) z6*`Rb9m`a~sbx3qwLcm`hUtFfn_YaYbg@3>q27_Hdx+YNxLL~1SwDU3R6n4#WiUcM4jW@($AieG&tu5nAb>EQJc_m?Qol5)m?txMv3kw*m~7Th31G|Kf0 zj@q|6hD}ZjZq+xd_u6y}ecBrpQfHRNP2{BRs6Vq4Odd51pfBq&^tB+#p%O!tn!m}# zCa~Nzl6RNrlvv%DuCUC}34H--7jih)9AiO)`=x2uFyo3j=ZjpS=bukdrg3@iAxPKW zIy!Rsf3cd+@N+LCjXThB84ODC|3(ZmN$=@o^tEj#ZI&QnXqJe2?6J*6Lp7+l9jRh7 zSs?HpDbeF2rRelmLmQ?5^J)VPwyiheGhW6XT6I|6soh_VG+j@WQqP-#;!w53D!CKl z=_hRagbnhip@O{!j>hx3A_c)7NkR*9OolP+xF$t)&7wlwCgi4=$*r&|nJOj7y(ynQ zKA(l|cZTlU@*_WOB=dT|-J7RUI-5cUboZMR28G67;dLKNuK2qL*sr&)YWS+0WM$`}qfB@dG<9xZc5EJ$~k_r`EyQ zcaPAh3cqVHyPD+S$IEq>W4+qHW3`IQ3x;xcAo$MiBYi4}j{w&O)YylUW1-!}oZ|Nj zKE#sy3YmN;QFudSO%k`{pWqYG#wCZ^2EI3 z7#L++*hJErUZo1R0>_EYqv@UujO56-V7rP{AO8nJd!T|Gqd+QLW#fRcUl2aOv zwXGY0?enksOq4f^z!gwb@(K3usTRclv=c)Aqn&vB*We~KYh_e*G#?v@nwWA}K@n;* zsd*F>Mn5UG90RBXcDTA$=aX=&I0VS#Ri{P9E0`<(nV%DH_FrMArTXbgS#oC{tRDn7 z?9mU0BXL21*3>LIp1QZ2I)2~H&h~+xfGERRS1}ap!bDMllPB|{+N6Pd2dC3j9`Fth zc!}~I1%s3)IY(L}K~I9wQo-L1aco--@oe~uF%JNI(Vm5H6D||o4orvupJHwk<@eWR3TJP`q8H(cLs+W8m}fJj z=S$Pv&}@m$P3)1@l%3zwRGybi;)uM1)ii4`pZt+2wWm3)v73aK-B2gwt1T@hMbk7M zmWN%G>lA+wshk!ksajw-)cFZu9lQPG^xa!JSW}}34arcQI`tU!Ij`aFChMKZcR=wm z64TPVYo}XJ|8_!tW2vXmT+3}@vxP56#*_;bVnv`@HbIxK%tW;>X zh-R<);da%(lH=y&@&ibbe_QTIR44{R5iblcthYj|rOB7!s)W~Af9zOer~G;K(qT2U zd&7N>E+?30#8TS4&45K^Z#4j`5+7LS8o-=3?c`u~nVU%K8fM(&kP}OPc%PEWF>oIv zv^t5KC&+~)izD4D)LCbhg-aJwY~(n=`OOUDq_u6G>$HldQ8hW6hML>cHyOa*J8R50 z@_Tt^-HMEo28@U%oF>l^bME28j+t7c1stq*| z++ziorHG)Dy$Tf`o^Vf#HEwcFf-~I*CKJ34>U3ptHD?eWX5t7J)o#_Y|0J7bkmXyE zlg^&@=1Oi-2_dvsP_MsP4OM^u5lwG_gf=ev(XZ@ncPL=b9t7cg;_ zQ6BpYfYx~;7^scj$2#CB8Jvf0>-f6B?q`mov(f=$X{Mt5dGZ{bE_2yzf2m_uR)i-d zs;vu(ZS%eO^f2?I#elNVuo||iMM&aA>y8n1o)|LWN-kx)(zo>qKMpfs|5?aHBQ8(@ zoqNAO8>>(EZe?)llH=a;@G!R1n`2n5kvdz#;#xwmtUzjbPW34ZTvFsyp2d{8Aa^&r z5!>bil((^m*^TvvH<&e9z$_Zmft&%#ng+IDeOUmFMVGv4BC?X<`_VO&gjwd&!ODO+ z|C*HrxuI*lpHrE4M0_YlPoI$B_&N@d)4mXtBvnd?SoBEibkk56pr^>J@jE8;y$BItU@N*}#5rH)*}(nX82w6< z{6+nMyyU(FD4gLmTV9-2jlN4d(opsJ;}VH5!o@fvm9 zNz3?A>5ds}c$Sw53O-rK)LlJ6QL`qXLdev74W_-3@j|fMDyg*awUYLY#{9rJ?;b+^ z1C3 z)Hj@f4>jU+S6>DG+ctC=B^-dd%)ayxRy-Hpx&qrbn6Y% z4e-%l?r;M09#LK5!X0jB^Z;La=l4)s|A_cnSDAN1=78W-wm~O+ydT&7``oW?61uF> zf-VqSt@=Glwu%2ws^EVYI{xe>0VVZk1kj$LF-#hlcgRBsKXil*8*8f5e;|ae$s^L= z)K~Shon|(uuCrVw+u+}vEp9avitr^2O~#bDQ`|DA^6mI8ke2n%aX0-e<8pN5xg9i5 zWu+TSJZYO=JPK$SV;EwXVi>T^*AEdD3^$WQ<_lR#S=?m|#iXOuF(o8a@y02a)M&CD=-6-ukU%5+a(eDx(4ty-3LoXO*!*DluN_7)jz= z4A9e2+j79mr1RF+^T$C|;UF9))(i~4nRk2%aI2T^_3si=^@&Fc@scYK)3DkC z_Lq$;yA$6S!PxP3olQPtMgDlT-ss;)J>n(o@XQeBn95tgu8_SbWPxX|wjiLq`{s^I zDRq5AyO`n1_LP`Cdr@rl^ScJ@sp#qcePkH8Wh%0BI7u%t4)z!lV9K+-mk2|JOzzD` z0YM*p6_tKPMJ6umkE%% zgEtczfo(ru>>6~Ct$7Tnl92cNtgh(vzl|&SrFWNgc6yOl2|1gX82??%^JnYyONTo_ zL%mNv%t5XyBc)$(7>uZ(M47@%i+h6=ZhY0>dZ(qJdO@3(#yo3n*V=%5UtcYBI-s~C zk|8?c>;lDEduC3ELS6ejYsNlLiI# zFwZOnfX6gq0-2mq$RNA7&Gq35uA_sY29QRn16akbxCm?Yu#)(i>RnrH0Op~=fcz`d z3i=Xb%BUHRz_^v9+x)?@pOrmTw`I1?V*-@xcS$PGmf6h0Q?BCB2DotFQ)qL6A-NyP z_EmO@wQ@%&`@@fYQ5Hp+kl)WznZ#m4xQ1fo>nwN)-(y!F5!KB{62(PNj%3#n_nKEl zhmdH8EY<4102N?;qND~gK~NyD{=4w_&khKg2?-VW59&k$_J0BM^`$E}=C7W1-1r!B z(Z9NKG4Hhhk{+0t?tNY4u;p*d)&+a3$Lk*#78XW^K1{8qQf{FTGaDns9u4jQNYq+A zz98awd+LHn11MOwW-hR}rU3?>q`j;9Q+f~#k{;A5K(=nfD&L5*Ko`C#r0FIJYBoGm zW28-)LO3IyKjn~hvqkK=_Avigvz znkJPn8wuH3;oZcNyAA^G!JxS2auihyEc}`CGpK9DrHs&eogA(ut3N(ME)eDkX|8*V zpF$0d!X7Srx1^;D-{@VP3lU^UdG8emz}0(aQQ4!?>I{0?l^tWx$DA)_+G*7?hd>{t z`JvLvqT!-SR27y;r@7rEi*$@d7@S3#s0#Xho=oMkZJUL6L6ss6d;)V-KbSEooK87f zNm(;E%dKJGTESjPpqKi^;1+_Jo$_BbV$-uv5X6r+8-Bs>o%r0Ymm+gFfIY0^bjX9t zLF%B%!ukL&Y&NCy4L~=Y)_Eys>Z66*dPSw)#P93vt9fV_Nsy7MVVCYtByGxDEk7#&FY8O_2<_ZYcXf>!* zD605?=veyxnc7gZi$rZ6CQO6>jnhh75lfw?L#%dMspY!IPP})aCJ~W)aisCI#%5CH z8r^pVZ{0C`2%n~OZ}Cxh;WK1lywGU9tZmi7m+JXaFZ*3RWaC4O#w#{Vl8WLl-3ZbE zD|Fk*)e^#wltv0{J*#5#c4*dDEq8kuB^qXWXdsLsLCa`tRb8ip8 zUr+Oq}MZp-DI4aquW8TE?fxy0XRXiFAvyeDE)zu@#!bzvmNRHagA_J%y>le6$$>+gftu6 zXpaL!BA35J@CtYWs`r=24CtqZLE=wzSByNI6D{pkQ@CmSB zz2{#g=RfkS;N{`w@J@Q_lKP`ActU3ra;`sM;id0;1fa^hiqlk`u_`NO%0aqJ2tw~B ze*7NMvN}w%1H-k08pbE$X@14@1@~fHVf1!a^Bg{!SNPEm9x$&%JpU8w#Sg^L>A=!~ zoOQx)S&{#tUi+uRRyMZ(tKH-eO8TFY_5XGXcVI$VfgxM=xXCpqX+73TJ-UUT{QU5t zfN?PP@oH%BA52_a*3!K%{}6ZLtFcfX6z^+HR2Lgc_5*9c{?clQzi4mq@sy+$K3l%a z{q~_zGNyEv0gm1BrlZhLtC9iIQTWtz&U}FM1hf>9&$2J)itpCVO@PsHf1T0qUAkvE z-l|JVYY3=Kg$8~*zWcP26cL;p4$vQ6Ff8W=pyVWSexSpVf$fE}s*2~4uezYA#)Gp5 zo;u18a<;i@B83xEK7o&M@scIx@FblwzNELe2akT93QwdQwzP2=G1vK!s(wl;K`{~n zTH(3Gs-04^%)BvUtOgfW6;-V$kx`LL9L$#*qC}q_oQ(~G`bNb%I6-h`ZupeCQJZuSFlc`^M;wH$|vN1ez1;4*?%lVAC$@50}g7i+B^a2iH)9vK)&9#N@k8h=})0hN`CYn?kc&JD@6o+^Kd7 z4g@XfW6Wy(amEK3nfjuT#*jKvS8_H zmB~_35-UlOtHf8OCFBQVM^jW@TzL{WMPC{PsRKZMd+)_>YxcUwd=9WAKj|g-1yK!C zNY(!U(LbvLem_16pwVieMj7?tSygF}o+kKQFkCBaEdn-=aDjG#(xcS|v)N;LwX#g4 zy)-Mp&jWBT$tzM@rjK;R9Gv=&#c|imi?ASIAGEa1V{*M5oIYI@tf^dnR)l4V z*z7Al_bdVAa7L>1Jgd|gWO8YcEV?{5B}^Iy?Q$S8q=hwwkk+P^cZ{i}md zZhL3dD7}(Z<8BRDjiznC#*(rr*I~!jivZ13=2ch`Ab!| zq^!P|3nGFKpwHhZJJJ!tpZ9ab3*za{QJt?l1<*v#@)MRM?x6JzyBfITDycD09!$Bg zQpX0&yfXsdoGa$_Kwq$Pyu01~wtAXO^O1^+NQ;PA2$+&Zii;s>Zk(H$*^|#k^weuh zn8|b)*dwR8Q;+7@P^*+qdqE#9e&9P!3s}z7uz5YpN`8f64Znjq=1{bAXG|<7LZj|c zQYwesRdE@%lQr>#fpqwe0M5Uiz-vD}00(xw3@U&fJZA{$It(dIKC16mMr>MI80RzS z&JaM5{q62vG`GLdO2PoGs(gnAhXvGp5jge#jJL>RTk=@%7fxLA6C^+@xy#YeJp>Ee z02NYER&AtYc-j*EU?}(VH-b?p6wieYhL6X_Y@bP;kF5Ku6% zEZd`3K_Jjhevc&o^;&{}Lj?vZ$x43%^?LJ~kmJ<)d|ubhR}{wt&{PtJ!9eY89SUkL z7|PRI$%=Ma z!=CWGR}k|a5x_~A_ZbsI2g08|h;h>ZB_Fy^XT-3l7rgC@Mh_#+R}r9bBu5U7Z|uRzxMZLf>-dl}kO0-X9+msyXbp)_J3Kf>%g^CD+e zH6WafIu}>!caw+jRPRr`W`yb^k7b4}rx2%mCD9bPpP)nwx;s7}-u0a?l+@vh>3^Zn z!|T~2WqfSt&jn0raLfaB=^*c;yF#LBJtD7Ck>NQ%*%v`@0Y=5MWl(0X*xO<1Pbv_1 zF-x)vWPN?-hK=<_IDU{_oTlNVZXIg`*TU1f^7TH*LGAKCP=JTyQCTI_mNSnhesrgm zP66ARCER5k`#*erWmH^SkZuU>?(Q1gB@o=*-5r8EG}^ekyL%wGySux)1_A^LuXFF6 zc{6k8y~Cmxe>iJ(?UG$pUwySot6y8#!^GU?7JBs+*#XWY@Tb3iZ~aXu{YSQ_Vi^tW z-1uXrWv|xe7v#b1s^;t2`HH9B5%4T}bM6dV$4jE(bu8<2#}pH=`M(!Gd5#`CR)9{H z<=|2)}LSV54(@s+Pf0{epEJKnmF5<-KC_r)=_OpH7qG+a{45id7oe?b^Ntt^G{ zxkn{E0)@yB?IgX43AX$2Jc1*l?EB80|qFFo+|mcHE|`C*GbY) zGOt@P(PooGmo6Ewq>Qhf9n0s0-zuYtU30lgn{l3X53g0FWt$K;*2r-ifWT9|N6Zy) zlIyU45Tcv$0wC@nf}oy~ad5UB^4DtK5i)a;_@$m@6XjQb!3^Y=Df^oj^*7PPQQMys z``^CjKk*%ZIUs<9nLP;AA#P*>8ddhB_#0^bpA^E|fA)h9ch9G^DPpTq0URU>765L9f(m-LmrR{7sqVrMTn1vC!;wNHP^a2; z$~Qmw7{9oP7*gE3kQ+^ava^$0(HFM~X4U#K>E_!SZUp=;Z9eo?`&QiTlI&sh+4h2v zH7-<_JB_P`xZSP+%0b5lNdD;$Pw{`H}7?6S9C4ymhqm?U~v%xs$~qCI%nqY&oO%Dz2`c3WfOh z0j_lK?+Wv$BfB?JUcXN8{pmF8D!r> za41syz;y=ugytYJ!`{r_9viNm7!Z2yB^jLsiI@3Y)BmaWs$NCOc_I#FGM{nDaUAOCR8~Lvq31tv+NnGgf zAW(N}*^b1;pZe^zi94%7x9=0g&SxyWiMEz=!`gkuud(?a+KhG8mI$(=_pVq&6)&UTIKIsOX`{3I(b{ zW*YWFTcmA}61H2FOOm8K@+5`thy~A6UcDmu%9zU%1YjMOsyX6DKi*x_ zX}R9*+-2LPk|=e55A$Se0f7d&vAMl}I&)jlpw;XxbIb3D$7~xm4k|2l=RKj`?_u=k z+Wd5Puqnmi*_Gc#bAR6R{hZqDJz0MJ$0%-}g}=9g6yfW?Jo{ft@b^cm6frDuYL_4i z27p4@%=NGBu_D6a|7d{HxO|@wh5&Ych_aXuHPsL#JcNkv#XyUHk!=_}Q z>2$Gm%FwES(#A`{;&+zZ$f6X04k0mAXveZ5E;(bCn3Z!%BX*-r$}Fz;f-9|WNZ?<4 zoQej)&poAoh@sGTo)B%~rj9EDZ!z;%G|;Io-r+Az$AZ_GlYFvehpQw`GaM3@lXf;L zM4>N2X}c-R8!U~03B;)YQkNfAXkNcyqI;*Cm`smOZ8#`(4=`GHDb8QC1&B+J(jbd! z{dqZ+INbHvHuFbKomzhe?$Jvtlu%V=TAJONpihpbo`Cnut5vDLATHf8*{%;nPvk!! z{*iZTYSk$XNZtznc0&I>!2B^hd_c9Tjl#SkhTqm)wFw4FD@;^7W|I>xQu;_7@&tku z3OXs7Wt5)hpz-o#O-PTz4e^OY_Y)@itSmamk4eG3T;@Hi^u3UZ19XL*Hf zC4o~sO+bFuxktLAVB9)}I%>3@1xg`XPDc1tk~3?~p#?&zpTh9` zwvskz^G`WX794e9chjRplz71~fzVhXQ5G{pe|nMn+%L>26drb$ky_Bajg6+y)TCK! z=T0jw<6CaIhTeV+aZ1h@GrO+^?a@+6&NQZO=uF?8=kk*0&`nL0a&G)w!^Ws_D?5KHM{96a$HbjrBVaEJfWY83f3NkWRP-gB0N(E717 zc8uAj!ydQfw=X}!{xHLjer6MiOQVdEO09!D!sp0rvlCuxzEf|E=udvDbp-X9_F*KO zv>yfTkLTBu9>){0P65F+V9fXN-;dUXYTMXf^nGJ)OxVk6Q8kqf3cR3`D_Vl;+GKLZ zE`M(}Cs7uFm%fd$Bya?SBjn>&TapThs-p)hSSBjgOJG3Ez^93(N2U1;$~|!^9vgiY z9)TvAsJQ%E9OT#ICsjd`m`-x=_8N+TK_XDm<=?@MO}ks<9|0&@p%Kr?Gjqrb)&`=< z!)V%K4Dw0co^B2IqMhMvec|mnA#v*=OKt9e;_kmMdH`1h ztsv|C_8%!au@LCuldbaCwmgAmy|xv5co-}trI`XdObrN8@Y^qe9XH910VnG&dd!G|TaGPA9zpZd**J7i*J**o1+RLX`yhc#B^v1U%h#y!C=gX&XewQwZD)+CnWhV2Z>{Sh7m=gSI%LrzVhrpG!#8533o zBAx|9Q6{hDM=rKePA-6cLu{;L^bPJ?6-R7iZ@cSJ1kVLOk6p4%b;YZ)zm=+?3C{G2 zHSjKdNWh$D7_R*ed~6D5dhX1|Ph@vI-DL>px}?kf_g^{XhT?JbG00!m|3`d+{Qv{{ ziTk4%e+Y#BuHAqX13HzG2o@jEYyI&H0`R>IRC{WQC0P&=I1vItU;_1niO%2$SVm4R zYNFad>mvJt=05Q*v4SzBkpf-;KPY(!y1T$Sc6xWFcDC|NW+)MIp&PRIdzQ>d!^VA6 z){=|RIveJ$__*wcLcomB;&a~heaOjCR+(-@Kz7|V68R-=DileyXST#12U1?kIhFM(;1 zn53*NX<(if>pZo=cej7stn-11*yZ=gb7V|Pra<_v-6_LT3M-wT6pkn1g_Q8A#!pXV1)Q(6Sp-TS&K~y-gN*jaKnqp>9wb;yLRP8sY33Z%BGEw(^1>DA#S_xNZ z2{*w=P~|3kp-yqhLSS?(XjARqm7{SrdP5M*bK_PjphQ#>L&> zU``IfpiC*o!VFS?@{NU+ znT46v{{uQaIXD|QJvexrxsf4w#0PK)im4u?zbsW&LK&?Jq=t0=Qqj~uP^LeI3Y!Wg zjDnLIMFtB4MxMHY3rm;UO$O^t{gI;8pB)_>`mgsafLbZdp!b0N|K1Zx4oeAGxaf~+ z3ky~)DE``aQqTiagim?mq&JMS5{qL2|E{)4fg!s5R(29)$5lKz=$+=A!*k~}HGDjd z@2^*trpcA^83w(O1Bkg^=iANtCO$652CGPhFT2f>n-vW|A3( zb1k#I^|WMX5Tem%<%oQt45DQCqQ-@%Sv;|9xkaF);G6JW(On_nsOv8Qj|~?sdi%2ryNl{^R!n0;FA zSuU17s@R?nWMQS*R}E7fEEws5xJQFwu4iVwVXphA6-Dw6CWS_9Pf6$R@e+by_Pi{JA5?7QGCV;PQ!Y6Mj zhz$#w5EnTXyf-|k+TR6RFmF`|Pw%4)bq7A9Ym*b+s$t1)uWB_+q$c$KFyXxd9C+79=k3FNO8g z-sA_Rfrzd2d#ih{gGaZ(JHg;J{0ihBR8_D`kscC?DSF@dDBF+t`#p3Enjm$=f{v*F zII-f@+7HMe0e1bbLKFtbDf9`U0JlSgzmti*3d!-Kg!>TKzQ*)=%s?=mKpOAWC&k+; zYd6?ZzS4I}7KGnk#Ijy}gf1-bP8*%#Z@A9PevCcV_|UG&Rc}mYVZ1ny5rV@2?>M2! z%$yk*4eA8aG8o0|mZ{b}@-Mp+GTrC>Y6yF=LxYP<9Rtym6$`40t}KhFAyO%L^c>O#ITAI6_txC`*A&=CimHep zw2s${2boU(G^K%BFD%8a632m!wbnJUI#fl1dkN)(-rBmYK7(4wT`?YcKL_TsCjZ*? zH*%(XO{SB7WBz0~9yB$O7q4LxDUyqWRTqVU!+NH!s#fyt?c+XZ2hjlj3d&Is9Sh{g zQ6;|00_}TKOA#|YIaQ$KXFm}(I>s26Z@GV@41N?>CJd6W@c+Y1XJ}waKwebdAAtu_ zl!W{HOllYvtYPZ0KLR4&onR)fHwdW#`ADZVU zg%=*u*dZ?USNw9d`NdpOLQBbgfrU$k4{J*cjtcA%9 zeCEOC33EqD?3@0E#_;+X31QH8$jzkHOauGA_O@KEYrRStvxhVBWlZupacq zYW3!xCQ}qM>-2J=KQfg4Wapxi{bUC@<3mQTBQs^ciIRlLrOT{UjO8vEQ^=*nGbGK< zhOi$>i}TG**7iJot&`#*=hxh*$Lk#z!}T@q&j7!Ag?&+4|0G_E1sw%u0Dt)x5V#Mq zPT^0}>wh`1KYioBuesZzf3V+*O3{15#m20bh00xWRkr`f6f@({k znWlqBg2ON0w3+mqtYi~|vQQs1HW+Ha1KIuiY`DX1rH!6I-|d8i%hx6BO;wq%Gt*F; zjqb%f%NY(W-=+}8CtYiFf~jDlx$$q0qp6IDKauPaagvKjS`+EW{4oK$$Hm7ZkWf_F!^fx)&8H)zdd&JKjkQuI4k%-f2O^$U`Q9ELKzBspQ;LmimTAM zI8qU6+G_Z2>AgTv^gq?8tpIUs`>ca5dHE^diR0yUXn!^r+Xe0i?gt(uf~4J+I>Yv) zRgMLv>ilS|e*0SDiGjqMWXTf0v|<2jdJbEX*P8^C4YOfgXnaEW1S1YzKN+pTnbgD# zj49go;hkgFR%AWd0OA1|pO&aFEEJO_wO>(kM4{~2K!8HY;7OMc>eI~h9MD4SB^)*c^a4T&AOebsw< z%X_-D{pI*N*5HHrO$%Zn2sYUA$w<8~F!%+HNYdjZ6j{&s>jKz+&xx;7sJMD?txR*JSzE=x{9bOG z)5-(29cX{{01iw!G)`fINNgL;^c(g_M*PPh!*o~SV#aD%O?W4;Kb*D(H)iJtx<2~&Z&eKDWeA8Nba~?9s!vL$ZbIK z(Q?`4Z}zuYUE|D3q?NDOqC8llsOqe$5@wN>!Ig7Gt;IIL-(6!a7YK3+Ktmi}318p{ z?pk76QwhEcfvw0ZzOX^AU5C;WACR+MQveX^vVGBsHeIuKAmNL# zvy05dA5p8m(uhM z98t0z$YMay1P!=+%D^&zept(3Wg=;583vLt{&001=!IRg41Jc2lJ9cqyE?Q?LS{qA zJE2Y+mHDdZ+FKV?-reh)>z%fsBGoOr5CIbOgQ>V*Ef@6m?os zd|z3VT@;H8omi!^dSg<~fpa)Rzo99s9;b^;EH32u9?%5qVuiEW;C!(7T;s7-q|(H% zk`IaL6gRSTQXvaxFqVhrbhtwmYbD1H#&!sqZx6u9ln#WUx%VP9Sg_M3(QgMcz&`Xp zj!d}W)wue=qT-8LQRii?KfLLxC)#FH<-tyD1u(a8~GCU_}+SHSFddVXjTnt~GtOD$zmOIzFm+h`FD!77KyaXEGRnoTuF;S_X3hkoSQao)Ay z!I*|daau7lcV(GY4IU2%w>UbB(Ehg&^Yc%0?3a>LGwU=~;%#pjyN}y5Zy`Cj=2Czr zwM%RYcjl|P26i7EsYmhB66~P%Mi1_JVt$EX_YK6J_GTSf758iM2qF;DLCzrW&Ebr| z{gsmf=C$2uMDCMEL|kac8SrhU6!n@C$pi5BG52vD)?=2 z^I1tpa2>;~K_=)tC~y$F{N89WknG8Wt8?*WlVU)0ty}Ru3Nmsf@LByNmM)tCwOe1E zal-T}+j+{A8}R~^BTD*eF_FqvT*V1!!%FocI_P9EmQI81U; z1;)m3YUpffBq!=|1FU40-=C3;%9lQ)4EHJZZ98-FLC-~}5^paM8=z)3yz_!UL^EmDF3w*`ZjWXZ=NQt;SBd>y?@36oF zPE8=elCX=FPCe{dbqig*owwaQtsQQ>e=4M3g<@A;LH zr`%)aQTYYT*3OEWEDW?f6h<{;MvVc*uG77}r_%YHkN58m5&eOPu;Xaz6jjoiiFG+b zIJCH^6lS8N)V+B$BgSLMY;aDJ^qPGiMOqa30cI9H8;6(B1WpWe>8~4%xA-nrdzi^=I)rByr$A6!8rims z4k*}goumi0PAdMV2b*7N2 zPO5!)Qpls!B=z1Od^!hid^pC2X4FCMf-QMOI|LJeAE70SET&_MMmCW&78{nFg8~o> z7HWHs9|haRgWXUzV_9*(`iu2_-<2JE!wPgj+5gTg|Ku9~*9`wG&He||Vi))256~`T zn^Uf!{`>cwK^dg#f_xs>2om92MNF&PuJB6xRa@Z%%=@eN*~FbcfUNvD4C32jL(dmx(NJ@yDoKbsl9kHHMmH?c}T!cz}_=sOB5CvBXVl?>^H#R z4uRCCaTDN!l@~cqn?*+p2sh$~>O6?+rBXf^B1INI?O7CdV_rUL!V;fc=b2J20sImT z%@z*=m8yui;W^v0QVIh*#fQq3*>y^Ii$N4Kai&0U@qcilW!?YX#Oy-`xdgz4C!6`46b8qTuWEaK(T0%su00Q4K!^Io&KvFNrUdgjUr@{L5edlTe2wBx4v^+c!h zRW_SNNyBy*;5VEhOeoJY&x0d^^Tn~G6TuGmh+~*Zut}211awRtGOOEIT%8CI9}pqS zvfem$W899A>+rUMCB(HBa*9%-l2-G<3l)r(RnkV`Lf+p?A&p&5Z~6$!eZeKGjJhs;|+of9BFOq_K*Wx@C0Z=bo`1 znxN~^O6z^m7>UJ?;y`=ZbmJ+*lCsmgV6EdFNoJtGzB@b41~4rqf7X5x9N-LO;pZ!5 zyJ)B^7pyGDUtDl?vzE`~I&+K3Zyh5Qp|jDtQ%}}jCmv81DG=LGn8(@N?--YeF#+E5hqFtqrUh_=OsI7KoeeGzdV!=rX*WEE1t%vQCQs1 z74XOMtq*R?JMGp#i;!V{$M^_^oK97eX;B|S(t6dZIc_6;XQK4fsoqy>+4Bu|l-h+) z*wbrCyl?I9VIzxtV$JCLMxO&auBz7IS4kaXY$#K}p_V~QJ)AsD2W zHO8gB#lyaVO05gj>KK&k=-1;HC00xGw~6-mBW)S>llnLmq8IeUzoMYW0K)`LJ#Eq* z6`0>pbWy`=wn{a|2)gym;3lN3cGkc2Kb(GWn;EhX_Z7|ixGjtO=@hP`(~P})dnCnf zgUom5N{PaL{QKiyapYAD+y?q2RWoH*T88`Brs*AKxH8~1gr2l zkdFd~jQ#DDH6Z0D8m@Ok{3a`F0Cy7?#tM)8hw& zO!l!i7UvfW83aq)3Tb1ELJmx?q*)2`V%n!ds79u73%Qexk;8G-DTP#N^Mn~Wx%{u^ zK7bNzWT%vRlQdphC9@POwe)Y2vN_AiYKbN7f}_Wa=S;06VW*pmCcLNU1ux(9X?*Q$ zmu9z>f*am{hdL8wdXNW$^Kp_RljiyxG^PK{f(t|J;Am2hiPKXt;XYW`?(4<7?B&y@ zw>WY5nSC?lx_me!_$RtP5cp8tfodXt|I7Y?KdakQ<37XUrW&xpq5u-isY^>S;Am*e z;SlXIK3KtH*rEHArYER%>HrKm=InnQJ}3C<&@p<8HM~x6&D~o+2}_Na6<5C*Znm;3O8Wq} zLT%XXA0%Hjkd0>Bq9Kf$0eKNmbMYA#?Tiw`q|#p;Ob)GhjWQqNJKABpP7hf4H(S&? zoj#`pDb0X(illt8f{90r{+Mi5VQ#WKl^zg<3EB)jc>wxHAOt`ZasXn$tZ0Uc{F5Qh z=0^ok^LF*Jgi>mE&hDnr8ax$elSp6}*bR0wOb`|uvXP4mec`>-J?~cTzT3He9Z!Eu186g^%s3Q?O}q2Q}S1ncXbBwoqaXUZe#G;K%!Ms~G`Axi-w{@p+@M zEUAQ-S)$yMW!vv^k)75k&E)dBgG)Zld|Bf3IKEq}9pYU`{Dm{^mF?~oi>+$fG+-|$ zMX@krFQ+0-v(d#y0AK3HvXZe+nw&piJ76vV242`u+Q>nm65V@^!GkmQ1JO17S8?bb z%i=smU(_+6KLbzsCB#h2e(@?q2K_faTzXCM>qHKm;MP*tvE{iwa?k>*g3T9K15H(x zxp?J3kpmO;X4cpOgp@5_u%8HGMTVxqW3w6xB6jaFzt?7v))WiFG3+F*8Pb;)`lU6= zLlzUQC=-er)}BaW@X2YVOw%OSt3n=!In-d0ltrKb3#y+!1*uX8k=vdL-~S-cOA>gD z$}kx#X+|m^m}X_4aM01khoOOeX5>I2A|kzIN2C92m_;?PoL@fsREI}~rsLu(!1nBA ze|A3l(2K4HT+pIsRZG3oXU3S$K!NgnxD~TYV`D5TxZOaFt>E&F9yOvd@UX{OViMp$ zNQR68UFYeW=8Z}N6%xPkXT#9aa)=^w z2`cqR0+S`RNBkmUBFU}0{dDtUYUC(gW*cUG!x=x_m&W{>7jl@x)7K9fp1D$4NUYnxC_j~Kt@jT!ccRKW~B)wVOQ!Ik^1*wc?+ z03Xj3rsZFp?H9`=$~BlF*uPjc6xa6M+ihzKq!vtd-PY*bG35+R3Po&N3DC=2;O|=q z`b0|aNa>CTICzI0Dpzh=^`_t=9+bBZ1+G|H+Dq%S6tN9J#E5P7nN+*EsMc(Hr2*G7 zG3O@(k0*o3VS5s7Q@?}np3=SE;2HNd00}>!{Ky4K%b<=0HAPr6$ecv9hJs-Svx-b@ zGsK4(8_k&~de|p%mLaj_X5L1ckRDoZ`!uudu-VqcYM6Ffc;r&U+S5vi8u%oB=UUNG z;Y!t=MKQU@_$`mu=z7Hrj?$fPc0It&d zfU;AOchJQiA7j&K`3L%iD=)B>3`h*WJX-Le0+b=%AX&&C(D6}~D9ulGxCat$_41jf zkoqW=9vfo{b+<>$@0mK4>CwZGEoU){y3pgnRUu9rAg{cN=*o2965lJb@I zP*&J_|EpGI#%>aP1p-ltA^m&S2}HXq4aip41I>ry_{p!BifHB;&Wh(@wT1W?f+rax zY6{Y5YJY4}v^b?p1t8}S zWA6*KB!9&*;>Pw5WbXgaUkT-c4Ngr-S}7rr`q7&58LPVL zXGw=inV?&n`|%gK3B49hwsLrgBj;jaY5DYrio2BfHIj(JK4<4JqqHVA@?3>&)SFT# zt#Rgtj{Z^Etpd`YZZ#y)3wU<)m2+gd!9f^Hl9>O%mWz^K zK-u`GI5JGP0RitoZu9tD-GI9D$6tnDr{;w|(Pg-oufqN0kjth!)Ig9?o_?KLBmtgS zKFrNtf8iqzby=K}0Fa1b+W9~T8^-_6&ZD7uCg090j>@S$(I$~PX&44jKG2yHJa8YX z(73svcK(vFihT6$Vq(!}d3u};X zn}QDG2hGVoY6o^nC?+T!FOBnAB6^K`lt=9#>2ItRiv>yJ0;nrRtXbDERRgyOT?Q6I zo5qeLH54ynDE5C>C>*NeO4vlis%ZHSt2R<|1Gmfh6&x@DGs8o<6wmdS6bK3-_8w|a zYA|$s?Ny3;zEfc&ikHugq>;hN)0G_*Y1+GMUukcRQAmFQR3)66lh#EK;GWo zxrDj48;w&abZp>v!7*-;6)6mJ$LWpCyG=f}^xErKg%4^MmYTD5`9>=JDL^|ymWk7} zfEw}uEpQ%F#40j7LqsT28_aWo(AEiiQN2_Ro8UmT4&p_EZGdYlI^1@HSSfl))v41< z)=wfPEe+nsDz8idwskZ8@hw&zYYL)g73@pJj2jfU3YIQ%jOThhy*~qEyaEe&4o=qc+dPI>5%H~m9VeR z1owo94)(?7_6DH31X~PSOZ8otM&S`9xc~ISylKvLx|8xQvSNB@Gs4>k%z?vgMf#LU zBf>iph7krP4k`{cJd!1eA8J>yGOSd^f%>Ksq-;)^bCnxIe&xt^AG{>a>=5d!jtp=A zO0+u0cxQA#)r)Wcst1Acj1Q?r83<^plaL7Te^Lvq)M!Z*L_o5-%vTWe;ct2VTaTfA zAtEGMM`5d*e=uk~VUmG3Yzk5wI1gh5v>*dl0*JB*x}uu46O{@}u0~~Icr1uy zhydwDwc;y!wX+OJYk&~_ch_k`WZ2*`9Hyyk?`wCW$CoMH>FM_^01Qd63@gwqE8bCk zhzIxBOg9cS6<{=y78f5EpJX(amc&vl)W-=a4}no;sg#$mmbM(ST3i74C7ER zww;+Tc$HRGG7ZVdt&$*5E(Bjk?cpbUKRyw=4R@e#9|6iy9CIiK!cH3VaLDPh!^ann zEYVybs*4TIU>%U8*fi+Jvi=aav~dwm_o@xHnm6dH_y=mZp9|rn&F7ai4w_dgS1`oMkk*jc3 zj}7Ilr83d1t_;6_5Rl(*@+R>L;Yz`!BbnAo3qajwQ)Wj?#W6Wv*ER}jqf#Es*|2RI zf!-gEia!pbbox~pY|Un8Za$DUitWj$a0FKppJv{Zq_xotU!G2W_(@~TuMp<@xR+>y zSfY18om;W^g%UGc>$rJBzoi0O|3-CO#ExG?Ii<%h0nLn*MJijr+j3YPy1a}TZ6oKK z5r9}zr!KuCS9ey>_E%G9EkO(G@~2Ox#pcS(E%+6KJ*w^IcT$BTRVLI1190z5+=|qU zntm1N%at4@Rju&fJLX`8nWi&5D2OV9w)1~;3ZfL)W*f&Zu_S%nR?9L*lF;m<7Mz1u+R^zZ%Aq6`~R-u-{c)Hl;3gQ>9Tg|wa9gugkZ%QW>t ztxXn@bVMC!AQ;%cp(6_%uH^UUTRKA@#jbZefDIg=#)FAYM; z)US-|`XjMMZcglxK9xaFy5QI?L;@IjBR-$?OGOwb#E^;%-x4A2so?cBvq!zSeC(c) zdX?A18gExr+hd$JwHJAjV8zgaQR^cu`?lhILkJ!^Q57^Y7x;c+_ld|^q#o-4(B+zQ z8sXHObn!b(%(PvMbAXId?>Ahm%koG4)_`-fi=UdBtpT#>i$#WNdgf}b9@zld1+-9k z+L}szi*ykSWi*C}l4Ubeg)}uknoZMPRf!V`%iLbd9DDejbic)OtoyAmmrnv6*5@AR z9Bt!$yspcU>p@@0)rC_n;plDnFSP7&YKN`k%@7+_da|M$fTddfI5r+Y4*qexoc_7H}Oz^9Bp{^A>Oy*`Dcq1vS0uSxynWBW{Akp{0H z7iSUdO2!*QQ)rj}LtO#>GJ`olT*q8*7=26Ch=Yf245lr4VUdd8a&SdH&KSO1 zStZNx^x^XuxBGXa2oPn3qx{j+o$m+cJ`z#E0W*leWf7R%8N#9K;Fn$7%x`AjEu||Z z_EoTQm4$CCom3P=56d;Jh2PVg&v^i;$d^P$AlKn%^-WX*UcSm!^sAUkv6nT$Z7ahi z^saS3pL)t&vX>kuZ#C>~u8OuM{RT!WtJczjyy#oz#+2*#xn%dCnAhEceN-=)BC^M% zo%Ds}gw2ia7vaYSLh-;a>l^U;Eh<@mL(Yz`9pX1Z5+M4IV*h^`M+oTotJ;tH4>@qM zx{k`aI?8VdC?j%v+9W3#5*TaSo{^O_9eYaB{M|W!^s_BAtY zkKnNwN%6{xyq~`dxkPl-zD3)9rgOI@4GK z6|GERY!0-;w5taiZ?9H5f*5WgW}FN`X3BnzghsII?A)YH#|3i*nv5q%5;4zW8H2`) zM#dNE`l|wZ^vS1y(2i5=2iBe){PWeKpRl){DApf!F#tkNbLz z>!%{6KEr)E9J-|OEji~J<3nQetl2MUA7zTO;%4^RsBC5dF5FcaDVnud4Sn*?G%UKi zKNBcbl~1~TRK9s1P`w>*UjDeKI7|~mUEeF6tVk|_ei*mGk2R?ItV9{yB(1iNGEQZo z@~CYRCS%DdJ?ONES|ufcW2bXWNNB`SS%Q{P68W_84(>^c*4jZzn#Kj6bnP`7KjZjd z5FP2k%8^(M2*0D$If1fTx`x?db~$2;j!BT7eEG97(zmyt3y zg|dk?IF~;XFLc++D&6Ax+$dHWtIxcho1)FLtpwYdik*Q}Nt~6IqcikMHs3Y{o_bXBbwBNy{jIE)N>9?VwoFPcBQ7?>k zX*CG40Gk-m!rU9NH`un0ws{Pp<&Rt>;#^QjCo!T5Gh>2CL4D9RR6PydG+mZUQL4rhC$QX?$-=nz)4VGO5RXo29AN|k{Fk* z;D$Xc8Oj^%GqkMFEhc+_Xn|l4G3AyOd;*Q573r-PPifYuIK5^f z#HSyLx9I`=?AZn~dRzpsMP33`z7a1($TVr7gyR5%!pOzYzc@4uM!Y}cgl_{pj&J*_ zdh-s(Ii|lI6F6EhTxTwg`8GI}kt$Xr80j2XSXO%PTf4nxaCn-e8-0`K`aKg(-sVU% zOp`(XP|wKb!ofH>U&*3p(V0)_y7w|hSVHk5%jy@vEVc;Yw*hy*qW=<0RCZG@!s)yK zgBO;X#pV$Aq&^J7r$!eRd-fiqk(1)E{MZ8fAA7@(s9Qm8D#)g~eZQ#knqlL*gtn-r z9pY_^{Mw@ASy~Y(&I@m2W=}(jK6T8_b|*&sA$~o zRh1XPHH`5~PqW1AGooXo0yLwwVOOt^7OH{&hpxAZszckBMT5J$ySqEV-7UDgy9Jnm z;O_43P9V6uyE_DT*Ed)8KKI@8R(oGGTH|Lj#_Xe4^{VRBwV!__uA~+3Gz~y>TkyZ_ zrT>@+fulsG|6KA^pr0C$q$Hr&$Bf>dlI#;VD?%lq1a%~oOQ()SZ4AwfzN~6emYfOR zG&!B4&Y}hyM${c6!mnc_F+6*lb-Af^>Fe7C0L9KDGW^;t42~oWA3(%*gVqm?=2|Cd znUyJs)>}7Su4dkw>;$&@Z6Si()&6$Dm~vu8W}lbJl}&y0wKWwmc3>j7bcSuFAR}JX zgf5|ORvw*FL5u9uy=H8&2ff0<5lq%uMS*KSL)e+ublAY?+ngJatkCj*6D_Z0UJZ0HTd0DYtZN2DzsvI-&<>CIH77Ezu4!3aUC zuR{HB0O>hhvCF@y=0woftBY=~Hek1oY_5`;(YvnV1m#fL5@ADYjE#E8fpH1kl|JXG zHvrFA=1G*I?i1>v*3Xh>3_v7mUyW2!qNx1H-rDc4H=Y=j;k=S~# zfpx}%;g`)Xtk|9p+cx}Y2eJyd9uuw`;$$Z1j4Va49;JOmqQ z90^7Ol4)Mcn)-PsscKQSz|kzN*x9Lk_9Wvi_`z2T45jbdLUtuTw2P*-wGH;$2;mEx ztY=FTzw?eycLD%73PhWF!PU4DKSllfxEg(0boZVo*1BvPyIS!)jqoz;x+>5BW%#YPA^NsbOj4&Z!n#xddy>F4%f~e^hW@*vOpZah#N}Wpqi3;S10Qtrijy%i6Rczo6BBN7$Jr za;(ayx#QEdo8{DA+@`{!YfE*^()^jS%jx8UL1?wn0kEg>PJeQdi#r$FKx)X~gh|RqC~H$kmEk^lIqdZtG?r~P*xfy4 zZ<(ch9RhxsbXTvyIi_vF8u#z0#gjN6q3cVd@#v+{zz7WvauL752*!TD%5Y5zSpyGu zF{GU^^z?u{$t#HWm{L_8%L>F+Nlt4kpI5-&4by}ibYI+njZ53V31r}q>u)`WPBD-{ zMo2b-fdob(TLD*(q6^flHM${(x2T1O!z*bw&Ts5S6EA>TJC$t(yC=rkTa{3zKZ(l8 zkyECZF3eY9OTU2t-3yyWUsUn3{J5KMFPtYkE)P0#x;j6 zpx+r+(Qa#udi|Qbl2XZ~;*a+(6;v_tYI-8(Y`#(&!`%b*ZPcA8_u@!qIJj*rP6@9o z%mb*a6{wMXD)DvcQ_sV@k<<(nFPs=_vwaYy&Q=V16Oa71+ZFIH*(uCAc?dXzR$rd{ z=4z)W`?U3jm#=kIcD1$FuyQ@=UN-+;&3P()DAVY&d=j(l-mKU;B-vfgtWKBy?9{+E zh0{@r_Xk`TttgXn+Gv3n=%j__709CLCw^L7%5Mk3pkwyaIfR zqF~pu@c5Tb%*k;wI*$9@fe#$1Z+-wyn>a%#wH9)w&-B=z_$Nv=Cd(ivpq*N8@LkP# zrze`x-cJwl!YTv5Y)K(wm(+lnLTEYc!}-XcmCphET>=SRfLU(_Jv4N@!uPtS9)Z;B zZSZLg`SRc=ldaOflAch}0tG`k$^qS7UoN%_)v3U0v@FKU=Ne!=C`~a=wR;{0sie-r z12uL=#YE|jj%-BiZR2fA#t1xvi1o=Nl1DO;h4a%k3biOvxmM^fCmFzHx&6;?Qdk$K2-J~k)(}+zG*aWg=a-2S3Swi+^uTPypsoUEnD2rzt>T` z!I5-mYdkSS-1hU5br^&h-O*80cWwYzk_Qe)d%=jFlbqci1Gg9|izUMCI~*=Hb0srCShQeX)Sfi~I} zJa3g5ccfeMLz5r~Xg~rB}II%(pS-pCB-5ZzN1d7*E&2z=|Wj+26c$bdTY zKJw6B%Mn01QJo~Om9`vz01E^@skXs8 z;6{d_3JYHY<1-_%LQvABA_ih5QQGSKA&`^B)KN;I#}|f_B!&eebSoo5wfuAIL&ec4 zZABd^bhc_@a-)T7hC6&z6UB7pU+4jUV@>z%EYXyJJXFoU#x^PcCrapl8aJdATXP^; zr(>(?hW+`z){vb&*B}c;puvlBlzZxEmZ7on#lNSpg{9WS1+2;9d__Zdt>`=yTNEBi z5O0&25;f{30EWJvfsyeg5I1a>Vz6Il{s;N?fv`B({3Ra)T?%EgpwHIl&BjO9mPhAZ zfz9POnB_ofq&|))>%*|8%AhK1b}~vFAa4vi86^otl{in>AE5$$xlY?3_G<(O{i>Sg zbZNYFf()e~25WMIr8$aWU9vb_ga+ng1^R&?2CPOox*-Hyt7CVC5geg$UG#uAW(J2V zS-&w~#1Q+LSV^$n(exEJ;R5oY1cL9;LM*ADX9TpT6z1;Wh{YfXtRZ z9063rz(gu{b7?s%>}1cII31UWD}Tk476;lasxe!>h!0#NsWpZQH9^bQa~;}tj?Kh~ zY|SY6lpzRlIg4*U~#r*$gY#M{Hp#!b_k(5w1a#LnQpDslQo-Sv~W^w z9J!Q!nJa??0-Qv1gFq3Nj}om8TkHsWV5qFd({A@pa zlcBAx*&AZs(VCTjqjfhkk2Q0lVCwD&EhrMfEZ%;O4TmfXqZVnXA**ZzK+a&-D0hT* zH)v_S`$H&*#h#qC5)Dmjb&Ra6F;8-6*l>Qhy6C5;xy{EYAyZ?;kA`6x>NE}s1m23@ zNi;W7cwzH?WD-V=c*3NS_9QWcUTBt}bRVB#pmby1>bYg($;9$O>|0u3FyNQtJ*Nu^ zWrZoM(rHgFVipi|0mFhk0F=3gKuq9jH@MpTY{`e>_MzA4>+LJ3&+>^2~Z%ceKr;kmRUGDdpjT4RFmxN zw9L!zasRC}KkMG6)-@_KTn77hfbKKVl_-7@J@I#}iH#cL-r-NVEZSFq|g} zmJ@c$6|dib^jmPuHJ|W1l$Nhaz*z`|q(Zoy!0=ZR(d6Gq&Vc1{I`!ZrZ*k~^A02Su znR$fer04VzKO&k@wkOf9ASYRqJqUX8+OP9^a-G4$_;c%Ke_)a%lEUx0K^K9BPV%z} zi#Y1@hHKy0_!6Cr)!v>2hE#{SZicy|9ed6u%(B!Ljy+rvwIm3k_)w zBv2I@_%6~170`*P9OaS^nG%S`qfA#G6yXj?XSe^6*q8V$z0Dglu7#M?K88|IM18Rv zWvwC$u!H@X8*|cpb5>jZn0g67h|v13ol&g(Cyw+q<^aH$sq1M@Fo6)-R2z!PEk>uC zk3fjRJf>p!TLdd*r`pLFxkn^834Kdsb>}ksr%A09yt~>Pd zcN>MgK?4A+!Y910YAL>^%=s_#4{Fn6Z1PijOsp~03AvIwXrgoqo|c`C$UX_eD1l<5 zC1BqQ1&vF-fYs&#Wu~`)Q_bIDYSHJaCF^P#&&p*#@%vhO5zFz-Ax$GK9Bb*g;bFrI!<~ssb-bj|M%#QSAX@$UX8n@Gh(( zEsS%6iIIenuq-6~g~-Yz=0Q&)p{a7MbFAJhlnMSg5HOY0EisYEHn7>-amX=2%WAQG zI&r z=ZE0%@|Ja@cf4&G4dwu9qVm?sMvfCR3~bMV8GP$nCaLhN6Flo0Ks(*(S104aHP&de zIX`yGF3XCc^=cS42vsz5dnluj_5EMfVL$p?jMU4^rSbYL__momX>Mc|ln8dLIHgo; z)O00(jFecPmpXZmPSXWf)V<-|Ys%x6d!^DRZpHSd53k!!d!GX0xwVj&1&_zFu7y#c z23v=i@oTfuLUa&Thux0s$Tid6^uoL_IrSQD>#J1UTQD)_y>wy91x^Vhs!|NSH<<5cE+R$wq1@&aG4@iqEYM5N;Yx;Z*Dt_xIt-*TIhvk)Tf-U zB5P~xPT8F?CxM<6gW0?@WnXUSeP!`Y(t)$f>NudKxU0ZNo?Ft}t^h|aHVcL5rfURb z2zAGNuFREl46BUS-Yut-EuVL`&bp=D@rU%wL&&r^z{uMqm?hA6J#T0aWJ1Eql-+#* zKsHh z+@H4htV-Qh=vt+wO%&OF&g-Q|yFq~O2@v^t_e&5Ex0Um>@s&*4;V4Kq2+SpkO8T;V zNFIfP`&scW8P$a7>i5@C&=KPVhgG|>lZS0&c6#simTx~Vo^Yyvv46?qF%@vF-xLxe z&*tu%=6TV%w)2UZFL@1bA<97U$$SMFiEdfSx?>}tQdqEcWJ@Gc9{TXiZ*;3rexVbl zZBD0^<7$V$3x$V=teeU(lJhjWn{`_e7?M(&*2$V9KkjNw&iC#9@hJ#>XZ2ps|AzHX zS$U-!+9d#@VYPo(m4DKifjuB>ik2KQ3ZP2WKygk9{d2>2gcN#`b?X|8VTv0j>-^?#DTQ~d%GJvZvHsnAs)Wg5 z3oNw{0cXiHmDSvId^9z|NzXs9G63C~bYGmiCKt*=eU-a}G%QlaMj2bp%0#LU8UTw+ z$-)@G*lULHjhVhTFgPDuZy+()O4qOyQ?`j?&6#2DK`&)3Z}0md?W5=R@*(MCGMG&k zl#NE;f%+KYyP>MV*@Xsu&AQXNImeX_7tWmlOcQUOVLD1I6u2z9EFJu@*P0tYRr|m+ z8xc}GM%M@qJhl7}fb{_8L14 zP9!DY*^gkc7(pqSHgr6d+N#*@HveQDK_@vUkpOIq+n6}*x)v@1UDcZ|JU}zP5jb6T zdRwk2rc}D?LOq09uQ>K5FDzPMJ1bVYjkz*7Ea>{-KjG=co?J&gK{2B%0O;^VpSQ*a znXh$FSi{ob69C_|gRj40VK&@-B+QG!z=wTPB$MX%7&W_IrYXtYCWSS`F4Bswf7HonwK|b1gUHdTO33iWBKZv+5VA-Y=xzDJ!_hT`ss_Z9HzjXYaW+91j z_L=5N%j6e`2s6vFVd_yrcPK8q=_L-u(?V zHHgymR05)qO5i2@-@e}e!GQn#q)ibVheQ8+8xuW&_7q@_NgvSD+gv7}&<{S1PT>%Y+QhJ);T({h8E|VR9{w6J>{BpBB zTAr^nBL?dW6x!Sm{~7FqLV+HxDXN~kX}O99Oz|2+;E!2Xg6!mo7O~ilTZk&;B6p+D zW@V&M(8g4?yIhRrGeV;4&b0-dZPEjp+BQ3quO59bGNu`pITJxvHC|Ccf;1dOF z`Ew3lN1@lA9XGC*M2BYC(m!iiyqX=GI=@>-Dy>bM72=MEm2JORT6KVT+|`JSVt6V~ zU_7^r(zF8K1!i;C>o~+d#w8NB3P(fRVORlB#l^kIZK3oG`Oo*BSdsjc9|!=!{tWh*O`>Cp5~ux9nG3*OY`lF0fP7!$>M6X{edIx?h)M!H+F z5OQ$D?|@u!O#+(HKn0EcFn*IK^TrMXM!F#r@leWPC-O<%)TUqHri zwkpMZa1<}=H<5}*QFjwDkKO{5P-;|4Hmaa7Th4XO#(POFrbzX&$j3I{jEH_hbo1Qn zA4?`dvw_!J^tRVdbUtgSrP?l!`%FsZle?JZnH?iD0uTLut@kjSJcBh#fquNy?NX~KI@yFar93?dnA|kh%QjO#&To(285kH{+8`I z|1H^c;PFNO`}nT6{#&-UGhx90XW3p>f}R7aFnaLTj?J^Cb}g;7j(;ny5<}jv#o~TA zCGVdLzbtnT>fy~3%&=1Qy3v;GB8sLB0MBAqweEX;KN;)mqa&aq|tF78Z6y4)@Psw1=@$EbJU*+@^ zUBDj`RB>1AFcHyV3+(47UEJ;H%;U`@b&!jg|1PkEV)(lZ2|qv4YMyA2E^sXn0&V~c zehM~$e!+3r;PMUNkhB1Z&!qmUy#T*4pA`ZhSKb9wwIf=jI-CIgCg1Jso5i3-b-}B= z^5N+k<}u=@a-hzSgAEI*6o~{J1U0Zxq1NwTDwc2)&!>T{6_;xqZ`s4^U|fCO%*sB} zFYH5~wy`C;we|^O8?r|QKVFiMw@12+)sAzA(GvG zCjg0ljy~4$m~Jn@X*Z$w?UUq4;azw37$aDMT7YX-#v zsBgEi`qA$FVZBJYE{z;AeN!2%F@4h>++)p-y)KL_G3962<6_N@yVk+!O1S=xGd1d| zFi2#|&$JiA`a!-IBtzeuEi-RYi%PY3Cd0mqPjx26s~jze`KR6!FQU1@6Ds0X>&yhj z7{#2WmvnR&6cdYAIShoV+^HC?Y580XK;W&_o;Kh`uZG!2FQL`1MV_e{*^sFjpr^pT z%|QR61z#IlJDs&(r~v`>pdmb{gpnDlZUpkrU+=1y`s)V;c~*de5h=&^%EQh+r3S!v z<{5tNEZb(THoca{%rU>Fz}!-~0$U&7g^swad+H33*6rIZpcm}ft$?-fiGl@0?~K9P z4=lji_bs90TR2fwf4d^XSGqFME!t-E5%1Y(5$+Lq`VsKDobu{a6#(V*J0QeKs)y_; zKLG364T5P06@uw@^_%8#1-usmQT!K}aJs^wJ24HJ(u|EP!b(R9le9$HHmLx%*AWqs zd%2;^#-_^Sq}moU`7MYOza|DiF{Q++JAy%O&@A3f$YWuYrEy>|o)tHSiitHIP6P^j zIn}DWWthOKID4B4`&z~#yc>7%y*Sf6^c+7GWfdNKmmK>Rmb_(o9LFvy2qc1AmwDmx zWVKl6d1JUCnYIX9KFLGs8lgb88XCLsQ#9&TLnXZRwW*wZL(xKh(u^m7Kf#>%Ionu7 zF^xzZ-p1PJ=Wn_{sW-YX56SwQ`QFxI-{*VxTwI#y(xL|ouxyMH*ijn0M{bh!=XRyn z!{rv6(W7qyRK@p5A*w{S*J&{55e$$=US=2NYk0}fqJlUkyQ;Z<#ZF|!g^7(fiMU%5 z@j>iS+)$7%-k5);@jcl3jhI+X9@cGB-IBN?=6;OuZQ6coruZVW%mt#_H@MPrPoh51drWlq>4B;aA z{1qd->26;-%dk|sA$><%iWfn+y-tn%Ed$5oqaqa+WAy+FXYTopk(4#uX983~(e3T6( z6PBBwg6tgZG7SaR*6gySzG|v{>{#+?PSd*cz02A7V^H8g>ICiV@Nx#%7 z-k&Uz-}DmbJ8HVwAn( z{R{Ibn#4SnrY>CMS-)G{AGTNil5oeYyF8%0!aTFw8#<5-sI0yMgTw6VNd3qzgY<1CM zLn)veq~3cVV-}yjL%S;{uTMgYeXUwbrMrfj4qNXs=16N-ns-O|;bGzi<0dZBx1JP@ zllnNAzcl;yB}G;qnNBc{El5pTQ@K)MrlGS?M2d%V3NRyZz14gmULG`D)$UE2*MZ${ zN*XWSGTW-o<8zyFh%07RZV7>ECnB75!ZM{NTXrCa?z}ms*H{$96&xrYP(EB%8Fi(6 z)BqdCFmSA)ID3F(-6`~4visn%3aPxWhxi?Ba*u#S{qzKX(v1uyr-p1!r7df z<~HZ>7$9O;GK#~7r!bFkJY!jIahMBRNk8&)9GgCMWTjg1v|(`Vr(hM?v_jaKlnz9N zv3Fb9n+0m$i@$IOSKO}}F#=j6QHZA@G(!gKYd9Ye)K2*hGwOn`-8&C*z2C9lMEgBj z6d9PMZY(@Yg3hT5qgF941lJ^{lx-6n8AdY@xd1DVP9AN6dz#Z8mM31liCMX30j3C} z#fDT6teDf(pZ4rwj~Fd8++pe_5cc!&CsX0Nf=$y0+Ut~q*`&Rj0WlG^72?Zl*EDb$ ztS}Iqtpb}DH}q8K2BH7}EepW3&7YecdUTgJNPfosd; z-gb@M$S#&{40rzH*RHc&(>84YgS4FsVb_@Zi&q-Foh>D%!7ZEe=AMe@wqiszx?*(0 z1Aaj?8>TT#iZA#S_UElaFU71MdW!Zog>rct0V@wAW^T`wua-9=y0}f)+V`M zxup9{(n*RA+{&AzWBib4VAwIH|CNtRCnr0Z%F@mO{ zG~M@?qs`ya;o2jKfjNUQO0Unaqn?zsBlU@&xa=x1NQ>`0pE!9vX(RHF+AL?$2#lxR$xq zy@Qp+q!>e-gOhlY6Ubtb$*ki_8-*z+YSH&0Ws2TUv=ofQI-cCm?lzgA4>+t8Sl_Tw;X0R* zmacvLETYFhnJKq>AL%Yme(SPo#o>ETC{7cVqz-Hm{Cci*!JOgxsD)1e;T{U!=zG!) zKpzytkyl}wP5hF8Bly@O=PKoO@%=-w_iL2nF9j<1TTPn|U(j=Q{A{5=axzn$TUdE# z>g;l6_tGn5^1w9A+we=nV7~QAS@fPNPr#0^0MrA<4BnVC=D>Ppkb}W6~gTG%NblVZ%)^Oj+Dy4&6D& z4e||jI8K1QbxYMwbP&RNz0$0b@TLd>M8xQN?G>_DaFNRI-&07x_nZJrUK<-&)!w~I z{JYPWBBR?`%U>18x1P5esiq_ z+rs4CN|p)Kg&_IKapFLA@!dGW$S!~5&xTacx_Zq=VWj~-s_22Sa$fg)H`9FCh?FN0 zH8mqZAoz^r(O8%-p&$zk`<(b3Da#V!-Wa7pqM27*!s*uy&LB>8CUJ(~L?-Z1c#Rmn z1SH~^;U3Ty-$2WUW9GKve*XgK9gy?~nB?(wK( z0j)ptkDssv?+j{Wy@Mw$N^zz%;pS#i21zWsf)uL9YHVIS`4W)*>$3EhAAdOzH`MAS zBp}X7{&(2ee?{~FO%5tn6e)5GwJrws@y~Hap_$!S(j&7V`tIMkeqMi&XFTCH@1JXCanNJsV;b=c@(`lt@WMw0Q+T-WJvFF?e(w1Lf8GQ?yoEs_(}dZ zubJ{U8TW5>0wlx00jj>8x}d3}f7-<#79T&1hz1m)LpSaVTY06V0AsQ_e;gQHgRHr( zG1|I|W~g;PRy=*l$wepVwCcP9!I8${g`MPP&5!2g^}C9>YGaGO&fI{Ifu>+}FYfAi zJiE+x0sP7AX#4#!bx`xA(Xc4CvMY9fDARZfCfP?rsCU zglRW`CDKUD$FC|X?IsPDhp$snGcF>sr++8_M|#~EVEqcd$Wr-OfG+!XWkSwojGymq|{f zWL#pN5z^1zTmb7G>)jakj~+u-{P?&*1sqX$MPLXN*-?TM`UY#Zkte1eYj&SL4ioc% zI6tLHs$`$kuxC~Ym6tban^|TrBMwo%-6GvpX{H#cW z7cFYTTJ#>IcL#0lc75DF>KTt~$D%;uU6YvHT zOpe7K_0n!};g;d7()402$Ci^#VZ_1}gm6pT60Da&Q_zRH1h^XQpWiDRM5}4-kE4Rz zXBLY%IshTg)J8IS2W^+eC2!RXhMx{RtIl17uw@R^lUkj`C2lL7Ep^-ad<&vz<;&WJ zD_MB-?k=0^>3KzqQdgB+-_^W@);L6)UVW(e5|u7mbPdImlvGPg#ee6-b zo9frLm`TVB}2Ox(U?p96DpD*%|=b+HxoBgNxMM%x&$iNZ;Lisl^i z!3)^jDz1Q8kcx8I7c8A+q=h*}U{l|+DUQls^q#d!4Q?3Vb;+%fmZrV-Ym&y}6~Q6O z6-o7d=n;Dv8^lOgv76$Z058(`Ma4HKtylCm*)C~yOy2HsO)*ZBpK#yvNi zL^mU?zm6jTK{y{BFNt&?s-Pv8_TWqF#S2&|C{&~MD`JD?d@P-~)<5ZpClObyuqfZ9RL;XVZpu-b9%eZ+a^PqCAgOlJu!2y=xbr zCW{k>O=KP(C46{E2{K#B79K-y#?cggMOc1pG914QbRcN0ljgNt@pAFj*c!0ok=J2 zf$1s^IH)oF2jc@;(<~=hg!h=QD1E}~KBzUdF6M2Gi2Xez_|Su^-cMjo%QGehjv9B^ zBV_v@FwzLEZ#xbQMiU?ox#WNd-rzLy>~9tCppJF?*v49SU%wfPB>j!$Gl)XlBPr!5pZCPNWVvvNqrSyhtkB|IIy^K z7^UZInI3ZX^4@}bXj{zEO*6m+hN9y}RR?VbmCRbo&uhw(mThfBYXGYB0;u zmLV{WS^@Fz%LE6G{{M!o{#Qxzx9>p!=KcItk$jkTgOcl+mm988t=me#8Vc5 zQ-}GhBq({ynt36BYPp-H{YpD-eE;fedHW@cb*}SK(#=~#BXmeZ==f)sUe^0$Gw>aH z&H;SB!3m)*DDb0PCqe6p=?9YJ69n;3YD;8i|O27ohSwW&rvM1F0p* zLsOHd!*uY2dE}fzT~c=(LqtVL{ZqkVOzAyNoOEDrSw}S!m3xLj^JPXFZPY&@aO=-0wH%pH!B;3QlkU!x(Ts@ukAZZXHwB_p{-9rjzyLI#_1uYJwghS;Cv+Bcl; zOb*q4pCHny({CJ%?g?j`m#Mk%u*7jx@DHw~G@O*FBxS>5M>Js`Cjq)Og80zxCm{-R zX(vPWpv3QqyYJ5_0e1k5fL1Q7?|)ooQGHredvI+${Of9o2$W;Ya1aM~2{ySTUH{vb5C zspg{ltXAyO`jsXUrw9my4!P}*stQr8rwji=^R1c(QitQW&wkn{i8K~e7R@6Rh@L*5 z$}F90n5VC}Y(@EGJX{Te_~2|%zVKPT)eOrfeH-Y@)T>N91>34__T-#kL7c+e*Aq~Q zcHVj2xqNhIjy=NxV+!*&x-GnMKCOM1e(Gp_kygW}eJgpBGDv`IU1_Zd?!|E-V zfZ|vcmXE`$lR5>MlbUB(iXdL3g=kGb9yq@L9IIwY zQ}>tnRL*_uo6ORlyv)Y)6WCNhUdSbv5WqBI~3B8N+U zc763bq!p82I387tSAhSLLOj(iLDC<|dY`DsOYTEH8H>;}1n82M8;?1$uPD3@J)dc@ zA6kKjpR-FyctlTcB!D#DT`JESo-pFYN#s zaneTAcM^Mowt9ybWP{$Xwbanzs}MxpKo}qC018KJ&BfMfF`33=GVz#WvGrxW&lk)M z%mcm>#}d~Oew~3dB8~K>#o8Ejj#9uve&3M?p@>~KO$^cnUzXYhur3l*5Ppi!p}SW* zH{A_7#CymiuW3ZRaB>m7jHN)_+dOIRA?bYr(udj;Mg_X#sf>(#D4BPwJEB=iUF>}i zaO>J4qMku0+_ zH%f6++30KVY=BRh^1yT@=mJNR((*Tx11B!!x~qAIA^2O9W+UpDtwf`PHvycpTj^wq z;kqUTIO-iJs}mUUkc5&xFl_NwqL)!a%nD87eaMbv8n#gY#48&t)ix3PBjrH*AV%FW zyyO{I;iQWDS=B@AC*eV!^K4THk*ZD=6E%DE*B)+vuW(uTA@a0o`LTJ-WvC$#8E!EM zN!Po}Y4S5pb!Wq`(7E3ScP-763XdU+-!a77ggd@|Tw9*V$YrC7&8jT=)s@I$;BEKi&fQC8Z)Get)$^t)zIp38a-p! zk~qUEvuZOX&cCUtfjJV1p7GJ7j7n(SpP4&+YmWL`+4^6`jpTZ215hBWAaEcGXahnk zpz}C_zi1op$HNp*M3eLGO64uF`NEGSs1B3^fx3P(!p~Dcb#2ufMRN<;-J8AWShV{< zR=Kg3G7(V<<5>vPGw#aj_0=v1|PL{-^KK*HyI1 zgMFikpqzP8Y!1HRlOQT@S}atJ2N||e8HD*$(d$&q20J)h!F5_5%S&42QZ*Z;Qncfd z00T@)%1{Y34~p8O5Prjy?8AP?kPzkfr$5_W0_5~9;y)PwQ(8hc7kI2T|E6C4Li~SDG^M*1h6Dhbg^X4T z=1Sa1hXI#J)2(Ev7zHJ(5TI?bT}K-%Lwx;(AAQ$02!}-W&*z2Udr4c(<)h*kG|+WIWyw*?Cz5-_TSSAz4-m_JHKel zz2sf}4KB-A7qF56{<8)C7t8by$@G6eLR`RKTuh8?jZ!8WU^oD(>uTtJy*jNd2NTBP zFNdf{sH?>htE&N}N}H}AaBal*9A}$BLYym+V)FU8hUCC%{odsddl{0eSosfCZ?S2Btn&x<%m2!k_$il;o)wt_|!T| z_S7zfjZP%%wLPOfU(ZFbW1vM+6wZ`y?~SQDWtQo<2?H(_=vXy$4Eu(3$#GEj5@sgH z`@~-C1|J}U_&Yy}T1DPWHu;JqDx~uu$a(CXFh`qjx%Lb5~L>o%zkD zbGvtVT?)HG+a}uFUV3N=r3y0vciz*m<*rd%wjs6OA~#(F%y>dl+RYgS#ZzIqYx*pA zQvJ#5ApMYaD8aQN5TPIg$?<)z9TN-(d4sW2e39$4f_b9r~D0&An8y7bh8?Phm`? zm(2lvt0)vx`ly{Ysxs7Y*;im0Vy23Y(ZkLU3o{ARIi&rl_Bk9Vs5cg!%M^0iS`u?| zLvH2%1q7D{W3nv|<)ea8s0&gmhPxP~JG+2+sDllx-n_S$#Tes7XUn3!0oPuH%6=dZ&kVsrwCfKHu$L|SS1~CdZu%0sbf_IgWlYT^n#)WHbX&g}(#*`!X#`5vSr0~yr8fI%G4GGVqhW z`TkGt0fZx=8G?gV>eAIOV3KVF?&geR5fpB=#jOe~fCaud*O)tqTVlCARs@u41-{-Z zZo-O_sl;+fnv;jKWEuXLnYnlmzSuR>%mEaFbB~vCr#)JW(}lZa&n?ll9VB5woWLQ! zbdQzZKT!F){JdDmVH(+j3(pj!YV^E2{PA-py;f#zz?OHf^XoEHw6cH4-9Sl zYYS1cfTkwA>bZ0jZqc1$lUMhV|A(!2e5~x-wzVr2RBYR}ZQB*wwv!dxwrxA9*ekZ} ziYiVedGp)*oOACz`~5K2A28+`WA@(0(^{{1;UDhSYB^M+{+-%$+*Q)JVpG!&TTDwJ z{g&XLArNonw_vvluawS}o!4)U5SX@+(();W{s-xH@RGi3$$x&hXs>87JA7Rlwe(J6 zji4dTE2;fZt!FvgpIcws-BR7O?i?aeVE6a;(Lt~i6Sx3CZ+Qf!gi5?tMGH=B*Xj5W z4c$gZhsnyIuvD44?V4UntjNH7we8!X$@j2Pv_;}jp>R(A*z&B}9{PMwV2E^&t9HJC z(d>7U9~9?r*Gt{h!x;STo7^a+Mq4BR>ihO}$(jbG#*5abv(#?wHYukd(q;DAaoQnI zoz{U>`;c2eF`D=)^q|F0%U{k5x}U7wP-S^rg50Sy5LlB=u~rLjff!chaZH_aP+lBH zrw207CPqF@RD~py?ik;b?sph#0+-ugP`up6kUU~+gdy4>rJuOJXI_T-NzGPa!WBd* zo+6B8OxUb>`CsAEleSYCPYg8p zOKsejJ~AEdg+tm2C>^txFG^dwRUxftHV#eH%a-=tXBXj1ozL@Tli^34&IuH+@;S;r z;7guaV%-_1n2<)mhVS9MoUCp^VwI>w0_8*FF31?Z+b%7E;YM38kJQoKQ*R)PbfqkNkZKLOSy$Y za2}@yGoq*#wC+z-UO=a!lPn2JMV&xWisw#J7n+crz-l65J~Jg(J}g#nl$ylaDIvKd z;V*R+m#0EHU24>{6O|1@FJB5j?~>q=ePX`0MPWS;#}BkKZS(-I3!OzW8sU?5KzJ*v zf3dji+bd#tE@K#)MoMhSWrlg06M5%-L~K~nE%sxnd0VX3z2CmTo=dfnz}+X9r(>iO z(DwlQr;62o0BDYVRj||l;a^VsZy_zU8ww5(qpqusB7w;_D8U{K4kA{g57!9|!5HI@ z)RKzul@v1zVJl+&+a#lJ!i?j}4zcG#F7G4cBTR2qPf$Cr?4eVj@;6TwS#g$_v$NTs zs|*(N?Wdf}zwa{yfbZ?GX^zz+w~Vn8j=bu9XsBqA;9r4k(5IT%5?n1)= z$?rMOlmVFZ6o_G|f*hfwI>`=d!qNk5P-kFX4rBp(I*GdiF=P&~gIkgYssrtju;{9? zlI;OxXf)|5^^|)oPhg!Ked3iy)oWIwDz)0@FRK=&*C>V?vb-EQNS2y}$tv zudS0cOOKWhxy!X1XZ92B;Ck_mCUh4!ho@GXTvBq6x+!?pi9`px4;2zI_W64xyiA=^V<$7s-u<_}9JnqSFrx5M?z~?peXSgarb%fLK zVXn@Z^rCE$POVpBx*oeuU90!Ri7~;-QSouH_^~H%qK8YQed|cv9S0_xD<3KXHJv*% z3-TF>PLz9xZgjkbpwJVRJ8eQ$YSe&cQWOhiKNxp( z{{q_%&EOHbthI@#re@ERg{M#vN+wE|-aI{7M+#~$&QE9p`x_EjCG=DY08&J^uB(gV zPCUKW9sWw&PY*JOrEwaxKV%|lehvnh`<=F)oAnlUWp(lZmXC5%nG#t5xpucu$11W< z5G~e~a10Vhp!3x1PO?~NF+r7AXvXx~OJ#41;-9wqs;I~l1{~*LZ8l3B6hH3fK^Kb$ z7mFWfvz4j6WcRiWw~I5pgMxGsp6B>>*k4pH&r0&zTHytkPMzBRBGT_khl-HLGW9Qj zqp&!$YBJs`pu{mMF~JZp)+`7=(otz>H})5{jgjJn!=sBv04XSV{edbWBYRBVYK4WP z)Wb27pR6+4+?Br69{oZ9;cb9T4+`tyQwE>2(Bx7%d40T?{H7Jw{iD7b>AXxhc`(F_ zPg`qB-i*9SlS!zjZ4g+gL1!4wKkL{mB!saAE?-2z-~a>5_38=;6&E+wK4&8u1RY2x z$ROF~zJlm>2B|yvc^{Gy6%5~KIQ*C=(v2qknjhwgXQ&r@W4&p=+ah+(M_jxi+Se5h zpO@wlWE`@1%1ZjXgd7=Qx!Kkug`&3knLjzmy3;P#s-(ID(cQD6a0QIJ8 zJHIS`eKe!KTjUu)XD2@4E;As`iLdfE4g~w z;>Bsx-siJFBE1JHgTO;Cr zx7mLTI&+xPZJG7P`5^=LAhYuuN z+s?;RqbCXn3sslyf{?TgOQk+0SsIrAYV)V|PN#M%@J@4C5|_Zk!_f1m)Zo7F+Mj zADTHjY>hQQr{708;Mw!=G!2a>>!%-U~p&-gGuU7IjoFm}{m$5rYvW*q<^LWaiX@ z#KaOx0yB^^xCW|XDa^6eA!V>+V#3?#LlukUA~v+jo=r6NkN8W1yXdi+VM!~_KMQ(X z#7FC>73(m_Nm$v2Iggmjz5mMNG6p58tM3QyJrH{|dH=A59GW0d0oHtIAkrpw~|V4nvJ;S1Gxh8cAwr9bP0O z-6r13gN;2~F((|B3~*R166%M;dmlSxWTG_H7zGg{IbXn3=sar>9dh53#JFg-*-5xuya3xzfCOI${QD>)*LT z^_W*|uI{pR-Ee*@&39@mJVPDIwNgwfRLKZ3*h?|pL=h>Gf;+`ZJ`oc%JRd+LLkF7vT{(@{F)T%{c?-ZML+lZ;4B&x zEt-{I$_9Txnrgd$H5lHW>wrV(=s!~D zdWb`89v?gQL0RbZ?Ew+^aHAbq5lg};mlwwQu+EhHJ7JEbdF9V_cv+n2&<=?fe$g5( zmnaI`JEH9a{k00itjy^SY>CWN^@(2D`m8&4XB_1?nd#}WGi`;ZR}H9AY^7%~`X8L; zcbJ>*O(N9ge|b$d+v|h;zW%A)|Cz4$kGz}a{{tE>O;8mPEKNY=9|%j`Rz*bu)sH36z? z*!RVZk7D_<`^z2>V@PR=q^6FU+^^{)Ggsr=zdqg{+`sughaT{cip0`N#1{1_9eNBI zI!-w!b57vG;Kp(BeFd=)| z9)@MguWY8C(Z$U_7!O|{`<=su^tK=tb$fxxQAGlK#nptdkOUBCiKlNUJGyedq9V)m z49dl#pNo*qg(Cw=4NVn&#Ok|7;8 zbvSY2bI0ne$^lSJbPUzzmIb=rc(G-dU!30L(o#(1g&zg+m@svoj3@4uqEv3z#H<{>VY9zVZ-uAQX%s@|;dvv2d;xTn#_R7;Bm^m9 zsUL8eIRJu6nw((O8qfq;ZAb+STB=0n#aS$=nKz(xjcA3uxytf7x*zzu^o1uSMueAn z|AeSm!rf$fN4;dBCNsDC7-x+X?mOFPTtR^q0Q3G_oqS1ejdtDWgo~a|#RNzZo2&&+ zdwA&f{e+Ok<+8W!zV1(E4xaQ>S~6I(G3@BJ0{{$DzPCT9w}HJd=9t>HV;DIT!|B0JFD&rm_03mxscH*S=+c@ z6e1~_J9znrIo@;lL zj!JQ{jZ2$)qpw1Fn0SE^MfBK}n7A7%7N7zZt0Br}Td)a8*d5w;2cgw^h8djgrRkhh zV+E0vUc?a?t@}2YxxS^^<)zHbD8N+u6b*eq5A8m;ipi`#C$(c-by`U`UmVY+)ejXh zKN$iq{%#&xjYu#Pm{{7~ zHWsCbcm~5RNLSu2XE!07Dc2^|01p0wF~wPwcZngeC=K%abeF>XOL~V7Ke@EqAeG~B zydTW?cIRg)c;!Bh2*3$I`W7W%m_o-WB9KcaZGIil%Ij^kYw3g>AqxbJpyaz%+Kw0m6ZE zPg7%>Ck=Q*&r4DxVgeN7`#r%?+Vcs+5YW9n0t>%%bM9It`!Q^bbZj z>SAmm|K0Ox<)r%>?KvnBcM&G`RL~4&Uwly*+$y=!XX`5-Xa2A$>f`aZRTAx$^pe1XJl;cPnLfzU4XU7BGZ^?MuYl2` zTP2y96S3E5(=$L5YtL11!q@#Q+4FR!rI7x*e*(%hd+)#pvhbI+H>j&GJ_BWsKliv> zzg1Mr1aqy!agmo60Y0ybdzK&-kq`uawHG;1s>z~c$TJATP7?io5f9rH)UqbA9Bc*% zmW9uaOxbWriwEU~!#|(Hm3k&mP=q_Isx&iiICtstoCI(?R@sFY#i}E88fgyB?G!8P z56*w03&mjvIn z&7K_miabn4SuzS@sYWh<2J}>bo}6tO$tE$F1qVtsNblOnLE+*2jFPW)4H`L|@Z)CT z*EJw3mj@s}+U!fh8+swLc|I&2y2jsUvmbkGp#QP#A)Mb6)go2K(kiwSpFtnC3WH4P z`m%0|j%>e5&JxrhkwU*?i~M~n__PoIXP$cnHxT?}DAiltSJ{oTN#&W5Z@3s+NX**R~W4xflUK4Hmqlu2VxvyhYGx*X2 z22&VG-w;Ay<+Ie6K}oMrDOADE(o>Ef9}9R7A!X$w#u3^A5&;4E9+hm{Z=3C$f=Pt2 zTn2DQvMq|Tza=kOODx`*6xVpNi^ zv#(=mP3CP;#8yPrHb zwgdEa7uzA}1*9*sTG$9gM8is14Rmcspmpr7R>W>5NxM36`!V!(XLM-;CX8)ZgE=7J zt;ers>laiFfUgD+vaB!O#b$OQ+iYFp#GKVQ>5M&L-F!Quvw&~MzzcIT1_jFGC{ZHxI^K-))#qVOU?-J-Qzcd|wra;hq(Q?{tdZ!>&gp}_TRVc=IC`My5af7_2` z{#QGDjzgl)*^R@tam#Ig_c`9#ZN|NKhiixH*YyU&FaFmLiEp;;R7R>pVn~`uD8%#; z9%@6$;giGwJ(Szda1POKs{P3DHqmaneLSRVFTz3~D8f2lxp3Hzmet99OY}Dl-ogEE zQRY1kMsXPQE{&&m^UZb))CC?yhCEStv~FHdR&5wdke*6G^=Zcwv|5^%PlfOa((Mg0 z1BTs%d#t&}j@iTCqLOIk8!(d@1Dj~|3=5<&2bG`zCmN))G<&?EWIBC+kP|t{4RMvA zxVmYHW{eVt-#Cf(s0DP59dm}^x+(L?8FM9%D8B#ZAttG;Y=L$FXCD6>>~E#}!|k_{ zyG0G2BM4gmF+Rvc|M97(t1=wL(xOH?;IJEg(iKWW;W#2aQ+2Sxg;}aqlpF18+)|S* zZQAV&kn{t*87%79El;J%Dq5!#9h6w$uGN+G?rO$i`mmVx zK()D@M62>AF#eO8=d+*eGWoY3&fBl3RaN}&M-J7rGfs2!pIBNQNcQh_NRz-yehc;! z2mX>a^1`evnB$}uS!HYO))1bc)*#Szu`Ww70FI^ojf`_qEB1S{);^bO`_%{H5wMNpr+1iaHD^VW~IYPdzIwOX6o z8Xw*J4IwuY9~$yT5v28^_j@dk-)JdjW zsz9)Zp%rK;lpw2sEPD!NMPgP06@PzcFK#k6WD}Wdb}({^j7(V?n7dDRd{i7^L)U{U zr={-xR`Fb2n3&=c;AP&8V6hgQDXYT{I6q~JIdind?=vX(A1TQaA3ZPZAMyxL{~=E9 zq=RUc=1(`tnJD7#XKU)&l6JHt^pGv7WA=E=;>cDu44Tl69ucG5v6*DWTn7|rM#G2H z>=f1B3udph2NykM(SfUk_YL;_+fjTG6ep0-Ii2;ISwSJ^tGj~#>h&v(g!K&;(8HF9 zF3Gd0pM-jYEY)V$A?{~+kL5SONqsuK^-A?*&Na5_XZ+42aKbflH(Sf zzwi1P5ir{&zD(%@DAEd6sjl*tU7-yN*YXM?R23-RNdDbN(y4Ps!oOo2h;9))1yHiS zh!=`mE9C5O5Giz56ma=L6SgTJh+Ugb(hp>m?yxjiwbW9_IUin! zX^*30Ni_qABAsQLvcGZUXdrrFenzY}S;Dt?r4|Sq!JM(4Z2|85fTO$8j3#5qMFk6F zZ@QC14s@}5)I$zyqm;B{Lrlkt<~(CuV~2A#C9|WPQ3^LK{Aigs84k(Y8Nxy6&}x{n zOk<6=LN!58inZ>5=_N!eQ_yoGvG$(N~QoP-)Z$}zo` zqB@HF*KbFI8$8<}(J>eq#=hU#nN5zNc9~2@SqNUYHB)yL;M+BIfWFu-HuUFE`L=y` z2GDCg*lYT}F5A5awbL7zU}k7fHHuj*#kY$LLl`jsaVYbC$z%)KBjxe<6zNw?1jQe;EkqqFJB)HN`(I{m;P&V#{WnjFn}HnZFig{v=8yrV~vTS zSULZ(;BOJaCNe=jli*xmpNUX^27i(0e4UZ`RV8AXM%oPS5{wBvud>-tnNH~B=8)PH zGa`!JrP}+57LTJJH&ZCrw1k&@<|C;i#)j$|ycT&GzK@sJ*}lE@Z=?i(_r1?=_xsI# z*?o0jxu_n}y8zDs3@|;^=axQuFh80bo`4;gze2llUk0MTQoE)BKLZgZA&`ay5ic4z z!r|J8;SV|Dg7;bcz$ zB;6{OFpbyReN({-94DeM&SNm8K0#L2+GYnlwO{cbK0wy09iudq33tH(_o>MD+Oq6qxUuGx)EB7WMR$6lzqOf$$Wru$__k0lIdrL+N@Db!WIV(5AfJ6rEH^7^17y7z7D?Xz^cp769agB?I@$ZQ3+Gu* z4OmvxP&s@$G8{_tQyglkwK86$KQSP)M{62&a_7DxNimwfwAHx(u#W@EEBu@o5uKLZ zG@Qau%KIJZb&Shm-p#yNexY=Y5SJXy)xZsr=zy>CTC3sW+|6TaOw$lVVry67!=2IkbwTp@cGqcVm2l9 zXrtG|$lK!!mvX|%#Ot-;D-uQw9eCI={u=(eqfwwNofV|1xBJ@An7}X8^cYxoLmB>k zhlJ0@tD;I7#OcqXS`AZUmABK3iMv_ccL1HPiY;{vmkN{9q5;R)k1JPZyx%`N%N1iK zWYMKd6Zpk#ohFK(j;njLwc;gy6<1%4Xq-6Bzbd5v;d46u%DKUPMYEwwE=3kk5Cn?4 z+2vYkGNXuuP_2Pm?$yxuP-^^iwrSp5uayd^<`LoceBl?DWp!FhYV0K1&`f7fLk38) zQJreMk5%+@aNfM5G?~`I;1RPu+na|5+FIu;98kvQ>+fV1mal)TT}q)#mZURHn32yX zF?r7?sj9e6kEuO!&AR-PZyj=zTDOtbH>GRF{1 z>o0tD_;90sf$j+eeXL3#%9$g^y%Wfl%_4V&mp+4UIzZqZ-b%&A@vh_vIn(y~U&t4@ zB3&paA7%bobxR$QBVqD=uz8KN>CBO)4oIN_m7fsQouJFU!>#3Q@`yVGQIr^VuFa z@zCb-|3wov_Ol=Z%jOlC?z`XIvRg!L2Y-N@S5RTIk64@LOhnz8v^B7~O~3HX0XRjL zsr36NO>i`<@|X;gJ|b}a1`H6pexF`MNIp1t;&AlLWrK_H8Io$HMLH-NKZ=PZ^I|(g z?umK`wVGRBP3oz-u|Ncagw+Lk^u1NDFSd#Pyhp@3zp73r=vt-FH++PXL=RoFFv_BB z8P7fSoYuip6DlexZKjZnwic-@x-Jl{*T<`-vN-O!bENkhZdzXVV_q>M+~!j#(7 z!f8xb$mWXAgrT5AYRF&F;-hsOxdTKSFVpbJ;21Y(IfdnDg329KgUQEVa7C)I z3wQ@rd8Xj+|CQ|P@zWFI`o-wbIDl^tStBVq40S5DPLgp_x-!psyxK1@g*=t-!Key> ztZ_kr?yIjW@=dk153sbHpYz*@R_^zn_4KYCP0t?r4N z?CUX2+x~3dt-@J%;J|M@@(lkVXH)6R(rrjsWGtX~e@%3A$ujOdOxG*I({+-qn~Sxx zHs2s#S9V;6rQ@3BEfpKd&|Yr5Z{kgZJ(X#^4TP!=hT>!l0C1;Qow)<48TX1S8K_2J z#C4&HxEU)cqMNF;(qoHf2Ma->HK0gvG3O9R)>FWnXrVo0%jiIOW8~866F7shJJ&@Y zc;xrbmc8f}Pz5@2nMGQ3!f%G4IH(0a(|$y)bpJ$g%4 z-wBsk6I;=y^D(fwFoSV*AVhar!ZMorJg7;3-}a6{$aBV8j6;R zA=yZCWf$*q=2{$`DWF+p-4Yl)ciz;zk=lk#{>Aq@eVWU|Blvq0;))BSOf7>nZPSKa zu|PWofvrk87x1d^U68A}4PUQ|qKvsq0YQsAGjZDRRHmr0mG%QbA|L%()RKcOQujdp z~O~8BaaMZcq zh4YMK8_|_%kY{0D%4APj(VJ!L@W?WMwY=i@UL*fX2mf87^;jfK0pTtL6p8R>LIocB z2cMJkq6JKdH<>~;RjrVH!npp*(H{tLeb2pRlQAe53UT50kTO(viqvez_eD7lwsu{v zHnB0jVXrl%F_>o5jTQgOp9WnjS29TnF;{2=l1|`l9t@y$ccE6tg zCEEN?wjDs;(&oP?g^*VjXg7K-T3XHiLG}f@jG5$Q zTb!pHw(t}U zL*bwK#|A;j6H}!fKuqsN>r7*(4+jU}j=&u{4he!#z)q12Bg)B}V0k5>bWj|#u-Ilz zYjxJ&1*Z+Utm>fV9#toJrcbpbT4tRkT5kkv)h10tUu6tDSbX06daRKi#yE}yE&#Zy z$a7Qm1K2udicrrd|_)>Sw16UD6t%T&b#9KySOXw3}MAAv5XL zBaMA#k&WorIj2E7=g`{hqnmDh&AshF+AyIs@EpiZrszv`&+W!hlTXxxm)Lr0Uftw% z&>c-eu`6rmkgTYVDNNTec}4%i3xGG7-(=$4BWj4I7q_{(+C6*kRhC=Ko%a-wSwtr? zxGQrvrP3p}o{HnnVu@L~WR2CWV~PX1m$F*V-M36<(k9tM&1`CRffXk7%3%;|W2+EuGU0v+c#O8{m-*c);!+>65cJ3-~DNLUBrk73q>q6epJD`qhrQ zmoOCE_5N9GO`iMH)01B(_i?J(S5IRYb?XM6v3bE=QP0XtmRnMWL@JeP{xLC`z!pXv z?>#GEp2WWA`(skFWbqMO4Ip}(c-C7OU7au%A7VI$7~A0$z4^$aIkrb%*eBsxhO_jU zcARoNVBCIIztH=R<~|P7B;7Y;`ScP5iw-&aDj8Q>n87>TuQfbM2q}T!^d0M&tgX#K zv;$B1O#I+dj?yo;4~G-CmIk|4#)*TeqTMkPpJ=TmB=x8O;nXwE!vb$>Z~w3WHnviP zWWw`J+l5cow@*-oe(R~i-S{gT!o&KU?~$#DFMV4GXB}5NR^t5BcAAu!;Ld@(g8NnZ zdbVRNm_-ND%lGwP7Opp5q$uvMlf&-IbNs)y98m=1zapjq%7%94rvFU_DapuuO?>$% zjgd=gU4HcoY8KuIQmR82@)6AEFIT%Sw$w2(%@VZbd(Pse#D_-zXV#Jx z&l~Uwu8APe{`wnxgN@PlP+)`@iVS+<*ean0yMztwswQgu0(ssrD>>A-({RismtoIk zXA0o8=bCEfR-`0&C&h4g%&lwXK}(twj-efmjN?Y_T8}@RPB>CVNB4`-mxgHU0(NNO zVs`Dt5FTDIBTDqR$+%=@hAQIH4{R>)O%E?g4f0tbPYoXQG`;#o#7Ate5PuK!#eU@O z<&4i)ABO%jD^NJPlu z?`TPj>06xliK@ere^b?i{V?LR#IL;cGrIflfGtqev{*BUaWVtU?BBo;=?N*ul+_n8 zr1>Aj(Eo%f{nrbUvNN`Eb+&Xf{r7caNc_x~*$#2!mlV64TvZGHJ198#20pBpS~sLH zsh1+seQIqKtquD;DfrHKP%hY)5i<(kVX}FP5d>kN+!M^ON_j7Hi{F%E+;XHlTYK&()+oGyU zC6JQ_AFUwGs;c(fr?9kbTIiHDH>i|~sFUivLiJ=F4@#5w7h}QXkhDsg*T`^x<8AOQ zv&r=@0{E8{ur{CjIgzNQ@Fuy~+N+44>b+o9JfJ00HALCYU66#FZoRu(_O@TGdN(H> zCcWkh#z`p1!SpOXNAqW|pniRAXaDKPzrtSFrIUC;zg95q>-k@Q@&AtF{r3v~*EOvM zjE?pZ+NVBpyHvYiK?pPvtF4F(Xvm#ig%MRv3bdFE!N}+(S%;tvK5Iw+a>v$T;{$n=X76ykT&x8F}BCvR6TUyd~2 zlk97~+?OYkI1oU2e^=>+Me}YCI&R}GKFn3iZWFrCc%XpB4F-(-Bq#kr7IQDz0T$Ly z>Mb7DPSUL}mO$cdVsx76UyTt3lfSAXGbVp^MrfG*hz`=2{m2eh&PefAVq;^qm=LsU zIAJ!bDTmx>A|1e!@X2{H?pY!jBbc%32V$dYG`_{bL3aXd#*7kUs%*#l?QonLMnico z4GNjgjGaQ8`$)$~he)SPFp|<^ zVAqXAn4+BO>EK;96bB-u9iWo7GSZ`D&~Qg2GB!sJS)lQU*r0pE-_kc{4nfHE6>rdR zl^BB+s4)Q}zoF%K&A{V!51_>zssbf$J;CGlfzRA+pFH1sqQB~1s@!~s@lC++8+&}uHLqjw{0eZrA+PHl zjspyDL2r+=KyUX`p#DW^lN_Ue;F2klFWmCBAnpgKwpPo{#vmW3&{_Xd##5~NZdKk^ z+4Lq2tg4eYb3a1@l&7t_8A*BK=B}J8odZz5FTC zEcgr<(cGo5SStj71Ff9SL@!%ELx3>m2CZ6uSBfoXL9>EIiIZ3QjvNu<0#K}4M_qoS zysKT>Z*PV2iU4!g#K|)NfkTn6pewfS!49JY+zIci%47%xRb)W{@#*NSt+Vi$MF`s( zCMuxUM#w6jD{QoLkvV-EbNG0ChzE#6Rz?7SLGmEy&Ohzm9>dpn>Rz;nkz8laExUV2 zBkWb-B+QMWKWsRuq8#)x3{N~8;_Dl=PBrw{ZJU)ND>Bd-pqW~NYEiRBXmo52RqS)Y zZAOUgU-H0Y=_xETf}5esz&ZvrH8szQM?=ZYRX5OFy_OUyt9g3%;%i?;rVTg?xgG%g zY=_0zm6wX{_9~!EzzXo@eDwLEl;jihHxa*{4Cy+y%LPexu8f5%1j{U=emlCxkR@gc zyZI0#VA+GiQmKP+clZdN(dj@&e%4=*5n6s@Ip#{nil}P1q|_P{ff`L7E&AU50ibPG ztD+7!_lm8p{x$K_&^9wvhLJMWqYR+lHN+UH&Mh}DXYzbr?Rpx%WyfK;+f@YzDuI&b zSTD1(kfq&3m>=ql73z`*%E$%0$JVyiLSStl@8TCJ5a)^_d<_F?K^h&Q_LbXS!8WEX zc0D&fso-~H1xRP#6knx);N#P5N5wWjG%ql)^2$olcl$i|N^5{DX2Bwh-U9VX*85h{XhP+xWHW&c%C?0F!!w z<64_}9_rQ7)vFgc@K?O9vPNqy$K&#>qSmfjUknQOL0@j9RKk_5@J~^#lH>6m<8ovd z+X?h}lbNn6I~-3(o-#oA$2$>Q{FvAI{D;MbhgJ!rsUNb*W+fF36zIY|<62SJ%7~X& zb9j}Owx;e@jhe#Y(!(PAa;4RZT zR*iC=Zf}^qu{0NW=C%fqUR!SO`uJW|==e~dsW90ck^58yeFK2!UnZiFm6(KH+@a^f z+rQOJy<{UL_pT3&kHP~drxoeq5#oQCX!)g!f`tmP0)GmP0d_$l8pi}nJIeM+Urjnj z`nX_qI^V#~V~Y`ASZftPY7|P|?c0gKn+>U1V}9krmcy9!t8F3_xG*G`dbj*Ya$8Ke zp>SM4-9s>xY8(duN&l)2bcus%_09T@aYL*is^OZDit7YK z#*B)SVg!#NBY7WhOiA7Fv#gE2G zi@pz(qZh{%MxdKS#@p-x+-93aLq){N-=&JcV(YY&t0&S$s_y%3+MDMgrIv@{eYgs^ zvI4&aONs+9p<`@>xQiF=kv64eI65b@n0e*S23Hl+C zZd?b_>9I#<{U#Xkl=zK%;Z1267n{F2sMEG}i8I3eXoZOrM;<+cLgxJKj{}9YL%j5} zKjSh0V!_g+7bePt>`+qpw`s6>3z}m}aX;<~S#nyVQ`-jait5CAsDpfC>)R>mHANb} zAQvz$;$DUaj^+gT2erK+)EGGv)sSDR@#(iDeQz4nt?mvh7pa8W?h?}LHLoS9T0bzG;G4~AVq7aAC z*uJvJzuKt6_DkKc3Qh#m@I;0mQ7m|5#qy@b?5*5!stvRAC{7$@Gz9QHaITZl3k3mM zq|FnOSsMc>nUkL)EU1^P#Cr16w)F&pr3vDmBx8506=0!?e9@IjQa_k)Ed(U*y=eQw zc1vku+0Yy~_L9&%v+3ruOP%Qlb{xYBJV5B2G2$or@ZyX=lB2@=00UuVE(ezR08$z= zsCSmyEV_HOyR+z@?0bG$j4o!I_Q(MB>1b~{!~LoDpXPt0nJ6#>`rhC+!Nof`w{jmn zn9`!$l;kaeKVf!oUyIF{9`*W8GQq8sGe-EnA^#N@cEEAetT&c*_{(p6oxrwIj;1`Z~xJKYToe6rmlJB zbX6X8tb8-9cG-wR5OKpD(tOBBUNbAV+ND{R}bVutmGn&<6 zgFQ5Cr=T*zxQuExdv*nfKCcU)j;e`*F;z$k`AOT|IO#R|G1N1L13dN}`J?yHDE^0+ z^ST|zX`PT6dWr*AB14-MrfFub?SM0NByXtO=WnTvc&vjPJ@9x>hq{vu^&Q>o-MSPb zR=s+JTRQh)H4tt!?9bV7XAFI$I<62;&xva|tY<1tF{qgxi3`X4h4`Gx%l&~4D({eI zO<)+9LUT@3xD6@{4LMyp1#t&V7F8R|2SYq=@hF#>;WDL|QKA&<$bs{{z(2}%_tf(? z@)u-eg#91Q#eYjy7-=7Bh>(Dg_(}PJf5J}+BpLj}J#?uOz~e27v?E|t)zvV{BPN+) z%@QqfWN;3&rF2*mhJM@>M3>1q5|P_Y9|@fJu)GNT+7|ehZ6GmP8cT<_K@QDG$B2op zhRTYGrm9q%AIlc1-gO|`QKgghLmCQyysxc?(@8>l^&HfD9Gk|*eN-~qq{r-bKF7PJA@H#K-bD#D&He3ANgBgh^cAQb%jrp=vPpPC| z6x>uFx3SDI0f6UfERKg)}O5xvIn)wiAfXYcmk%S!~%l(|qu9Q|Ba9vSLfeCXV;#$Ts~ zTe3nL{*e}`h}NKY!-OA)JVwlf_E()A0UTVwpFsMmcm(|EYk(fh$x~j2b2=~Y_BLpG ze{rBAI0`f~^haelNeOX)8(c6u9Fqn1Xfvu;0=%3%bPy&akYK=p{U;wvsV(+ElP%Tv zN#_^<+}YK*K|=i7<-KUB!A^|?x4*JkK4iciq9XNSM%cnI_r6^7NB4w zb1v|mMAr0Iin9os3Ro_n0du}aW}sDvMuVI*tx(Kpi6=35y7Fk^cc(MuyOu^o{7d?p zQ??gkE_SxY3HhA>)5Wx<15*FQ^7Oi&t1bng9G8kU1P=X^y53B>+Y?_n4ED&& zvX_&>3?o+m`;l^kD*dFU@Sc0yitECIPLE{zhjRVP=5tjv__Cq~_z!Cf{#A4Mkc?^<^c%MmfKAnCKD4r}^z7G4JhbB4pCGcr^ z&d^v%Mr2c29g>$^)=oF3$XX6kUmWQ7bJa< z1iTG(Q@y_0K&-zqkOz7hwOV2}rU@y#PZ11yTCATDT?`Ub2|7D2(Oxjq&=MeCj1^P( zBZ#S(iA;_<6!*AzTrw$!C@upNJt5wVIP^RHu;LKyAXSUadP`;IxTWw6y+bH}i_!Z5 zBv}~;<}^haj9QR3uPtXP8SIS9<-M+Or4bv@JW+I^0#IbP-5{b^!-6l4q$b_cbD*BA>bqYl*v9XpR?ib2IJ zPv+=md&0qCW39pX%VvNlEXrv9{oLuj`mi;(tjSVG_G}{-U)8kbA1=L#BTS?DIvgF^ zLz$RSahO51KXea9o8Fe{KXEp|&S2Sy0Az2x668hZE!DMAut3v1Z8oLN5OEH5NZot* z1t2@EJ!L1tLHI6p>6mj&?yF@A!x2rgNcO8DAlw)v&DS7VJpvFcK>V7kF5Z|=leAQ~ z%!yCDmqfz9mIA$xZ_u3q(__8QJfu51g;YxuKZ)#)g2L}Ux1z#N?Bw5j40H{&-;G2- zkP(vdu^6~3J6Ihmj*EH8+^PW0wxkN1O|{;h(=sB8a3`T3P-goVi*D|n-h#9*-)AH{?S8EJtwEE8lQMD^&!oZWY7LM}_* z{JNN@^eO0h8_=j#{M#zc_gw2;spU;J^qZlwCd{K$@1Nh3Yi~TbA1_aB0wA#aJm9iI zN&&S7LLfgCmFvq5qyh?dI|4DFWx-9s0qXzkM2Bw;kgVBpyM)Npq_CCo3F>p9G6N>6 z`?~5#LcgRB+ocbar4OS9U?G!G$*T%c%n|bOFjzMS*$nNh)5V3-sSNS|Kfc~FDy}U` z7Y*+29-QD5R(NoC4ess`94Zjp-6gndaCdh|aQEQu@G7UfZ};sp?yE7XM*ZG<&AFz0 zVx8K9Vno9+ZmiO$dTE84WBGtgNw^>MsB36osN0v)T(xDhI=@L#y`{67G>ro$$X5~) z5l&wDfX)yTrU}YZ+rByaY<31>UHXftThNRhHd;BaaoRGtr0B4(k|a!%)q8L3p)#bm z8@6xu^DM>hETf(d1>9MXH1=kd!L!>PQONOzHV!kL;kVLBRbaWb^fE4g^z$QoA^BqN z*mlS)u z8Rrr46U=Mu5M`F!nR2^3PPWU)4D)X?U}x;>U>44Ay+2>$8r88Y`~n{h;Pic#-cpbg35=D^{to?hu!@H_heM1F&O*1S zE7cgMtC=f3C$u+Yol&u*0`XjQGnNk+1lE zqEuq{<08Ge-!ep<(yND=Eg8-~t$Z1;K{VSZFxjJSSlUVp=m6?rR2KbdOWzf%T;&u_ zcX@qA#+i`YQSa#T^)7Pzsr6Odj)NRwNPCqLp`-HG$E_Q<`j1(!63#X@{s&g0$b5BDfaoMGDagz!;@r&Q=6Lq zrlRu1 zQ4^3PlL1j$N^N~~?{=FMEbu(_l&^_3VlbWH5fPKF&Z%%$i}y4GQX&BtodMZe$!B0{ z7sJ10gZ}SinNp<*<_>^Ur8}6{^+uD%DvzID#5AAmTq-8zLS7)BuCL_pj3LjGD#Ap; zE3LTfFHMsrS0?Uz&$5YD!@{qY{5h!*DMxp#YtumaSTI3aJ^^7j*L&*Z`qcMsZzb#H z@tma#0>7^bGc_e>3Pca{0z>k=2Bj!Ba{uZgH%^|)JC-{5-z#}KZ zM%U*fWrR4hz0+lFq?zv4c59>}xcz8*IZ|aDZXO!2NR7YbY4Y`#1z_tY#|^0y=%glJ zq6lJcqT&8h0X1L}D^hFm@)W`!q9!*$!Ma=C6n)#tospxiwv^$owjmtLDANz$Ff3$b zw}6fWX03CiQO?2WqusQoz zgjt~&T%e-+H-~1kFr2Y;EAuX?(qh_7 z8{{Nsz(M(duFmdK?Dvje{l{M+8_5RbEFO%1?d}bxpcLwwm#3hJE>WE4E3CUOFJ>S$ zo=##E@rv+1--Vn&R%yy=JRywnV_46HpcID_mAEmwHq|a@y6TxA(vIauSW9zmV11nY zQy%%7G9Q$zWfv{mt_SIrWxm_wpu@39&j39FSbg*Jw96=|>NslpObz`>2H8)nOzy)g_uMwoI&T(~EDsUlCqOLfwAU9vwDo%e0 zt9o?|-GmqQa~$l&TfOXyj&4cd|4lH&EFU%PFgVju~yHopC5ANT8m;bmlJ zqtZS}(aEbkV;(9gK526`WwxKfZ^YaDB~|<4uz)nv2AY=fbHR#YL2w4w*s1Yu%(=8tZbt zX*t^?BSwo5I>VH(22bJQ2iDZsh}1ak4DdjkJ-Y=Y4&Wa3by^(EDC^mba0hLFZ>KL zjJf(M{hJ8dD2DfwNZq?(j_N=_#BZGP)LNvhZO2(kMGP&!@NBcGYIM!j_LGzFVV9{h zV@JH9jAyTy-d)EzRIK6?-2ym@)oY9P(_R4Rq*~x@ao4Ad_M4=~l$)Rt+5nbc^c>SQ zCEY7;T`uLFz`ElukHX1z3D(P-Ad>=UC9aL`s#No$ARKrorRJ zXlUQHa2k8Zl+?)MD8|EI-|||n&ldd2Nt5|!VX0i0s=X2JPTvhOuUU(LnVPeUwdlHd#iBG0!y~gxoK-1Sa zfsx2mMfjFn*tVr1=#Ol_SUVMmq=y6oULpH#9kSnq7#NnMO-E2U=v=1!N7GpMy11qb z2Iu?#Pc0Le|G~NauQC>_j;{Hi^=lJ38zjOHn9G17!uUvV{fcfA;ZaKX8AYbH(Jel+ z+tOYRQ~Tt$(BT#84lWUotfZX2vRv=3)aJGjRAPOfmewCD$5D=U;%k0>?qiYTn}z`D zdO1b=VDg~QpAIkHuZM9jwkx(PjxB~Qo+wrkp(LItt|q1i61N4CZwF#|Z@TwJDbuz( z#YeE>73-p-yYvLJ;ho?gQ2faw!D0q(xqx>(NZyxK5#7SfRLm5*a;?+fcf2=0cSpgD zJ71FHrK~mkAL~-JXgQm}95go{4l+@#Hcx2gY^D?CI6H+zOrg6|4v<)7ex>}*!(G?B zIj9Ld9c)Sk;Z5(1F$7sJaSLJ?)gIO}{#el*Jq2jJ=BGdtU&PRiTJg|^%nj&97c$7y zqIV7>N^686;j>-qm$9lV>}iR#48gn9Fcqn&jhwSzvy@)Js*Yuv*T(L1#1q&i;iQ%o zcz)KM0?Qh;d`9gZ;_c?ntn0rj%+%eYof&XIgnz&Sx$mLyWF{b$Z?$lKNt(lUY42%H zZI!LRoY8?);0VAgdsNfeb?OQO=jL-8SX+Z|4~qw%hTdu-eT_YpFf(&?_4R@^DD$=q z7V|~<*u?6i=-7Sl>X)gGecM;Ron^c-nA2H|3H~F8gpRL2`8hcb1>Y+N5wn=|`hubskAR?^~E0 z3IU`<=rysWvM>F%%4W`9QJaIQSERoP6MK~+ec&(Z6GcAfjFez1Udx>UeoYFfscH5( zhbfpFnVBTiRn$4=HMeSQrxhJ~r?-XyTFSqvnZNQ#3y<;ab!M+`CFe~PjpC3J;Yf9Z z-njA^4NH94;ejv78t9HEOz=-c$MBr%81ynM;{csvO?c^zMd%2zArs{dEmLO^1$jAm zM;C8Krk_WpYOl5Hl#O-HIq_c$>WUH%{tTsFo;U=w7oAjLd$QJAQV~Y5A?mEgx~Mm% ziU)R9Dril^;8k2)p;7zZb-d`59`U7v&Vt~`c6^_FSZ3!~t^DgxG1^xOV05XErfmsC z{JuB)69ZzCzY(0tj&tk~qh&ZoI^Mu6IEiFZG$h>Sbh8|0pauH{Sgwe0Jm85PlR4K(io7~FN~qaKdsWO$5r zkow5a9P-0Oan-<&i^&%}=QI*SC>+g!Ntizt?4Ks%MtnXYM$sQOAJ=!b3!s zB$f5igq8V0jRyH^_Jp~STp~@vP}5VApL5~{OKW~ba*(X!-E6?zXEf`_m-krw@>vz8 zPOD0SUO3WS4mQ!2K16ZSPL<%Re?=C!1wQ^NECyPWmcem!uAxX?!)kR?e-{$dX|5DP z_lfNMe8^T=FoX}!Kj6QgI~uF|8@1(;?C;>$OpJ!V7M$vbQ2sR^{9iS=y$B*4*bHMy z6UzeoT|Ry!7Fb6Mza3D7C_RV|4;|1ze^yr_Y<0qmfE8*^iNrL2*6@2saCz?FY8&bn zyYRMtNbokZ^KPSb_IW~11Z<3bG&0h`e*f&-)v~c>{^s|(LiXY5QmQ-PTjsa+Zvo%( zzKKJSA%MLA)MjR=w=HKF1BFU6h@RrRGzJKlV~936MIE!RAVP*63f%?gn1(xq0SFr19-IE2`Uv2cEzbv zEI3q;@blv$4P9ikN2rZKb;vx2V4Eg5wgbx{p^o$460J?(fhV#b~O zqx@3lS<<8c#EVogP_DIzsgEXF(%+v0#5(=yk{Jkn9w-QRal|G3inb* zMf&mUTJotf?c=r*^S< zxD*|Ip=&5K7*bcI51qEART41MqhU?Hd6Yf-MwH&HeotEkWhze2Cg2ztJB8(Csru}s zwUjbO06jO<)Kjr%n8FHNZSIYnwH0WR8(7^JsLpLU3=BJ!^MJ1NpZHPrZX*Kv;ww;#5MAZAJbc*IwaJWHByTR_VbV|6;jIE zQ?7CYb)vAc)wYrmBsVY!qo3pW8{^$tLBcD6L(fZgnwz&T2-hYYUi!T! zdXfu>t;5E%j5KwwRX=)me1m<3Hpz@Fb8e(=2?_{dr7dFBg3rTxFzP&bnFuA74lb;S zz=$?u8spj-!K8iq8K7u;al%G$aCPC9EC>G#KdghNv3V`KdjFZAAQcbV zL6;>6+Ym!rn~QFN214EyF4|k%le4a?sJ$GivXba#t+}N}7)hfM(o#iINdkx!MP}-x zKXpjm=wE;SN!vU-&r{-oBsnx~kBPXpW;2A6Kf(}bY=!|}ZSu~Gs+KY$N+G{fq~PTr zZ9&20i9ZFtN_vsoY&Y6nZS{E@#&XK($T&ewn>tsD`TRdLNzeU)xvj$!f85BIi+bM_ zudHTU0PCVrDIeCHPsx6#5rr<@V%}-rp!f(&F4wBU->WZeoH`j8+iEsfyU*pirgO7Z zZONg^xIjTT+fSLA2IP^|N$riM+{7o<$0yO#8_ddRl$H2MSH27#diW29rqoG-;9fyl zQ!_tD&qUkR&R{96JCoB{l&^J4R?139$jj%g)`FL{W$l=hb!TpNuU0KTWs=o`|YjmuF;rZXj;(d+L%g0&- z^t#{&snF4#kFvYMxL2Yq>7WHy{cO`y@*blC<946DQkMC0)KG<(H2Gs?H2f-k5GcjskFfy942DlIu#6=Cj5FzsreUj_n zZkJz|Cc+XtK)T$~Fje zP6-U$R%tq%A#5^%un{hIW`uczbs!YE+j@#RHm zD{fkcd;`xYggdFk#()tmY&m5G z^Hh=t!r+#kD5GXR;P4*rbT!2(C>yK-I?6gXhD>Q^D(i8dL4xJHQ6bqH@l)(4r!IYZ zgXct)PNALDTqi44?C}0}Vea18z_BoIT-Ij{)!nSz6%wjXh{GvOHgLnlMkjD8Nl=S9 zj{5@^nF0Z#!D!e$j!~xR9E&FLp1J}(TI7LLM@OTIk$x~Mf`fp1k5$)$|I{`XKKxpwU^q85c*gj@cAs0MV7AT=ZFItvZz_lgAo;&o zd*7AndaL>d4VwmfYcx9?lMW4N3@`d{jdq7^jYeB8YI*2af;Y^d@J(U<=lp*8*@$Th zD5&f8aqpje8$x6`AEIjubIXkM`htVggC~gNhYw{t4TEOncZm8?6@?*ho)lv;@PC{pMwQgKzKr+pM#x}c3nin zmJgvn00&l<<$ML7^XP&Ms;iRd^o?+WMZ(LLBY+<_(m%5M-v#wY&?6+PO$Mr6ec6M= zErx27D*15p`;7=t>27LZag!oSS@8n-H;k`Pe-!^!;UxbTrXEZ@`ad6@4=IFR2qYeuqOP<@MkQ z#*ou0X*NuCRqjm;DvJQn2+Xq{7{Pu;Lu%gI5(L?M!?R8(-6f2NCgzfIEPMOpgoGcj zUD&xRikU~{;&TYH@*^$3hBA=aoALqxGcgN$;7r-gluQ{S&;GtyMfF-wo249LXtEDkUzaeQ~GOszDJj+#@lVce9fVGqCwb;n@RO86)rjv=%29b~2n z{KJRQFw98hVly%R-LFeZaI+g1loZ$rnbZM;Pp5_dxhqIHQY*E;Cg@WlqZ-?=QYq-y znyg;Y=3a`?wmPrXaF29o6Wc^fM;RO~3z25~ui{hZF7nGaGta|XT8 z;Wrw&)^Rj|Lc@P~&I-vbfVX50zd1t-8dBs7qFh)TjZA zT|$;&p%*AZSyQ$*n7?sdo{Y*92!5mA{}tCMV45*oN{u!;VoJUaA`U42FN+B~XsRH_ zGU_*&#xyV4(->aMjI9ifp^KxZ2Awho933`QnGO4~2~xR$VD4`al4)maRZ)Vx2e_Ys zb#+(UJA}GEX!Qj|SrM^wjdGcL#|2=8yb&|r3>bY3;c!L`W<4~-gn?7VcVZ1uhPxhS zPb_QLhw0j_+J0IU0NEPAPua2+lr@+H-&+4Xb{bfFmGYK=ZLL**F7xnL+5WjNvk`;r zz_8UQN7tWP%}|N>KI{;2K^IJbC}Ef9UeJpwa>r@qy92~>g;f>5&sgN*$E3VP8}r+Y z{x*cb8lKY+(-@mKQfB_8n0`Mjn0J8ldsYAtQ{zxN$tuStOK?OKE?STf8_{1#cW^9= zHvBtBk+0^*UwLlS2mTN3mvmnT^__4#=jO&c9qAH(! z1C*i<+yzbNQc#43Y)OLfA=*h`B~|(a+RMkiM}2;5QOeO@EA?Rya+bB-H{_3=w^`UInK@@uRF0LGg89GPcwK%NvDLG}pVUvege;)3}G zrxnk_$r*ErsK+RBzSP~CS;-D@+#ncjRHy%*El{P$Y@9-}!HrPY++`{~(3M0C;!8L* zC|MaoUcj38B7j723C&f=C#6v>%Zu%?&fu7k5S~ss? zgF!`!m7C!Rc5^hBcd5*5B2jK7N#QAteC&@cvfj`hWr|+i@fLeDW9rnlC^EOqWF7Np zuU1@ZORK~mph`|s;&0ug>u62}2I>gj2zp1;wB$(>yw7;QE_O@O0?qtZfW8h1i)e+u z1=7WK-$6>g#nv*xdaQ|GgWq0(a9fTJb9BD zUUMw-c)lwzn{#2T^)IDZZVcL(!g0;q5@*HeZ-TBY4eRaZ$&mVARzgp=)6u$wb%xNa zK~^L^2Z!t3_uiq5gD-rCBZJE-aB_I=mqlnS7>fhy!KIpSvK!G1VS*ou@da6PV17$uqZ5U)oK$_Xi%dBfW9xmN$gTsXQjLJvxyzN7NMmd5dX^zyztK09(Se;TS*%z;4OBp8B< z4k7*$EgdZhR+BCQIup+kvlA-<&3UpVYSAq6LbnqGvgtTn#d9gE{I@{$!RRW4z<5M2 zoSRR;YGo~v{7{9c;Wff^r&a``OgjoI-|(wpL};1LYONYH4DYm>llKL~Lz241QjeKpobJn5)JHftd?p?7ZP)i` zze?pNCRrBfn!R0q;Ux9P8I9Gx5Fu}2xC;a7O&KK>pdUStq=uBVW z>=+F$QgB*GedT90o!b6brubXXN?HI+oM4#_V6#6BG5u0I(btr6) zzZPcw{JKUNJSjkJ3jBE=$65fk$PC)yor0oO$YLPCx?M+tHF2;WIN`UzY@GD&cP#6@ zfgEbOnr#*^xDhaV0&Vb1pCgS)2%*fFr#D1dhBlP8Nbi8!OW38UNquQJL zt*98{UDDL+D|QYwFV@xjHEKX^Sf>lUSW9<~)+CaGdHE0<(>JibyQF^K4-vIDa=l^f zep>HMOJ>@3l)K&Spk-R8+PO4vthD|5XVWs)E}iWv~aVf$SJ$VV;zFxO0-5>=G`0uq8+THx`^I zPAdL&dxx)1Z7a6!QBUO@Ta?y$?P(ABkuGgw#)x^BMiy>vlL9XLlau#kIiSuDT7%)i z@xkIKI4H7&awxwz;)m(k3&#tQ@tp~%9EH;&M&&GGMeMPi zW}R;WR2`LU1uK|cA%8jx^1mS%^qfL?_StijUoWjj2re})t!2 z_Nc9&Q2K;=AQRO|O>Cr{y#rFR^{#r9YLzcl2-{8@*$P}c4feO>nYD%O#9o_A4PiyD zRJBI#d6R|Z%x<-4b92lgznr#_4Gs%F9j&ZciGOnmGE05z;~alir%WrtdrDrd$63>2 zNB>0tfg!4L<29)hf2|VE)ps(Y)+jRM`Dck)-hC&E+8b0xcywhjK4*rd@7ATwY*7%eufrE_I9E%p;_1Ip znd~C(j5{d9#Vqykk2NsFPRo|XJ1S_M5=%#@3b2H8WY*}IeLj%&gkUJ9GY0`Ka@d77 z9?`~q-u?U_7KhUPsiDSTHIOSbVnqk2*yOR58d*cWAfe1NG4>i;HZFINgLb3A`f79T zA-!{O$SbBX{AVshNFd)Fegb+)SmC^uVX8FoPNH4M1+>@X=Yd9OB)dnY1SB6d z%S|BI=QWQSd7Q=^=+=FH+=vK>M%5=iwUDViTL(y_tlTW#iM( zc2;OTtK>}iR&C@dR=|pF2e~bWcjcQjV}0nhnhk2{ycoT-GefYp1XPA}oa2-EHKA39 zNL^iiWSt^H;lWWZl~N4rf~LgT-iVUhX$BCa>XJr8@Bw83T`r(&_VyyBj z>F^k%#{9p@;K$z6*Fa+pep&Q=E*_VH$vb(-?)JURxq_8Kv1ypvzL_#r`j4MP?ng6O zv1vFRT(Gp+;)yLQBCrFr-j7#h}nfU-4wQxP^Y#)+Zym&q$`M&nH#o5&AuaR zMnz-Ii;rPT)jz%?tv;-uW#-Q6^L}6G>^?cj+XO+%zPJfPv_&Lre((C*3JaXY+&6*W zaNGZ*rvk?lJjl|>#qw{02j}+x`27#;ca~C{0(hd4`8Co2A5%&^C|36~BBPR+XkxIW znH0tkp$x2+LtEMC36MB;Tqu~Rq#P=+!}6JS;`D5M()#}S;2No$I+z(%m-wk>6ZRtz zLQ*00gMO&*35Hj(1q-x8yp?*1*J!*n9%zw!ubk=zOMQ3>=4JwF(|V-{x1_IigO?wUwoXk>w8jV4o;r@o@JLV#H$xv|Yl9VP!LDuIzk82{;Not3oWJST-7 z{N58iq%lrToD!(|nW_=XF%w9&Wa>ZAk3I}h<3Ea`j^zmn`Q?fpfQ`pL+S%8uVbrbu z{DB`H?K4gjwm==1nXRapE1N+2W9z}W+t~Hxamf1zxoyEA`Dp5B9pWaQWF8Hm2G4Kc znd9nS^N_O44>V9Qmei&kq%^WksG*c7cB=%Ds5P*dJQ#}*s9+VkQvfS(cVkjQhHUX% z3wMw(Y$At(_bXPt&lI!EL;~ffb*k((Kq8)QQv_3-lYw%=5tyX!N1q*0L47i`)DTrX zd$CV2Tisq6 zIY88w9L!ooQV<867;xFOXjj`yHC1dY@dYZ0Sp,OH%OE+1+5GEEHKaMWnK95E4 ze%EmRQ$-$86yMOai?Fj4UIGJ1U}}o+GYl3~F8{)Ua9x=_rCGFhf@~zQav+wf7LR}T zs4->9lA#tYKUW8@DZy<{ySF?f z`r{}5k=PZ2*`U+KjITQP*OP4Hujx~6PKoDI%^K!X zL3Sj;CzFLuMCtc`9Lc58pzk&H5d6*I+s34$IPd@hj!#R>}!hs`aNRZ*)-+hBhn8aMY( zUQ61Y7=`I0u0>dw()1EGIg`v4u#UzUW(QvleO4j05{K8${RR7pF%2Le&sujoh|G%Z zkGBjv3S%0%XU^6Bz2E^4$nWLphM-$Ll|EU$qJ*xDu9VeSDHffYT80;uV9x0&9fP`5 zxc3vumrT%1ZD?dP|Ku^<&5 zQ%DBWlv9a?ca@bpFH{aU$(ZefK!kf}1h%Y>uy1dTZl)`uJQu9Jp;HUZ#bMd$@|zv3 z=fp}QKIYez22x~R`SPYuCj=@(W-LY6fY*5KHT*=l$@sQje|8uCf`7H zC`#BM_qIs}*wPMph!tE~<;$v0rSMCM#wu27hFI0m8Gq4g_!M}dCm~@dEA<>!_ zit@*Os_?YW$$MON!{;EsuPAm6z*f5|Eqs{7b$1LR4M1)sek$yT`r)34$*t(6sViuO zEzJlrN-iOO9g72~e5#X8+S2{3B-yE;3y~I^fy6W55p%eX@AZ9pG|r4uj8mkzQ#8&U z%Hb%-KO+@S-!LR}hEZVF@eaiZ(`sNbB zF%J*f)yfmwwxLzAl7SpXi6NncN#trfP{!xc1g-e}x)1-b5rwkhhS$(ENs?%_GqijSzAA1N5}#EduqRSGS>AhVxH!0|!_)fne{dY0>bLRJ|=;a@As$gSWHAQ2ZqP8(r3tbZ4<-tBP&64jhu z`fY$N@b4^ivxP61GL@P;$u_J|TTNE`S(+Jo*g zg;DvLlR&2bI!?+>Zk3wcU%yn>Q!Ikk*C%~&XaHty* z{Va;1h);M*)c;Xd-L=t1W6axMBW`q^73lrbzk@7>s!v5x{E8JV1mMdOoKBy$4n)^} z5U${~%iSK#w?oiEb({sLMCarX)x%0fDDZL!onxenl__rR0VSd$Rp-4DS z$Ms);L}fRqQbjvt)3QJfgQETP_nX1#H-mRmMl6{GwuJpCrqnBc{z4)CUItz@IOO|7 zJ#Jc=lwM(&@)G4ehzSk4j1YtGib@^yG@Sh6zOc4RTv8*T&Vu2Kn%Ey;SDbt2?4R~7 zlg=nij8o0>t!}|C)kY)>vRG<h@U}0U^zM{HvG(+hOceQ zA=L?Vbu#6fD%Ci-hypm?mY3OU64E2)vRt1It7j7q|L(`dLW zKO8K6op!D{G92q^jKIt!L@e>T*z{WLA?#6ZW0#|(w?wBFsgb%CgXNEoD#|tN(PEI; zAr-D&YoP|(qk$}3g`tG^d|44KG&YW3^eIYo9Y=1dg^zgC00B?2vTi2Zv8B@Ha+}du z`4n861eaHW7GqS1!rh~BjLj39F@6dgypWuE!L zaWaWEqn4|pkdlXZ(B=pr07Zu(t@FXI{53ks&tXqR&{O#F5zR`##(xm)?}M-A2>0O! zKlq)07rHSJ2>+x$jFfkMbg(IN^YV4Wg7Yqo2{Sqa;gTFlE$%_t)w&l z?L#-DA5Th9{`E^zqw7Qf+ozL{OOeNkgMYZ(auvFv&PDy%qn|rKj_NZLznqGrZBb53 zM07Xq#^YBNofUAuXAcg_uiR7t#Avc`MjRP!mYsjIFv+?LmjZ~ZH4W>WD5UdHxP&w4 zlRCbN@3XfMMJ$z4edjuVuopI$!|C!}#|jEz&uh<#A$`(=Pb~eQ%q;62L@tA5lZ66A%=BRaeo_HFDoINC|;US@Xf8wt6JL%XK^C z)`B2gaN9tfyAP{qV0xb?mtW$C;O(D<*`=TUeZvj=4pCrN{xN##lj4(Q?ZltzsZt_K%_G>- z8`RT0CZSa%Ncx7=tA~^@0P~mBkq4#+kvSmIZ^r2?@0s z$!Eg_C&Op7mNWX+0Q(>=1+2u9OytOfkE-nKCk$TSSp?8YRwd3U2Rz7``=UOkjKBX4 zUujPHx^QsLeEv7F`=8zEM`y4_r=c+Wn&rWZ$Aj^xH{L!X-w65xfD@#q|3P6CqII;H2*Dnh_FRZjO zHtp(9vk@_i0DRwjR%&F%Y3Q+0Ww@ugyfjeD7k$fzv-DF}hf_ee?*d#)bNJmI)6pht0 z+PkNpGJkr*dWRu|_oI5U+!neT@Ni!V4dCA#TWz_YY`M?oo>^(l)>LfyK|0CN{4in!GSkL(7gFI5CWLOwVD{mI6pi4qc#BwzO<*;@uJ$mI z@W>qx?-WJ1_0$H~AUjq0D7GTc=)Upv`Bb$s+<})!#b&l#ZiPol*A}A4SnpmB;ag5s z%bY<>$f`?RFid;+8f~ZEc}BAN`zAw#^Q+l+Q`?ul%mMJVTWq2gkA}XU|AR3`OBH2IDaLq=*;IhkECDesm|*B z;ZeB`e0(ZBL^hv(9>5g^PP8^$7EGdJWkq)=>eyXj_KlB}arvzH6)}diYIqf$Ktlmc z70%;XN9_pmD91^xw7j4Dhms`J8@NQJQgX4~c4_%Rj}F-{#MzndmQlrSUIu~WQ#5bPDsQPvXvZl3xTW{zE!FcX^W0eKSwZynkG1?K z9Yi=;s(7P#N#r;=u2^UUMxx?HxLy*eDpw;b+rN(9Z2!HUS+Sq{3sfp#bJ9%@6wyKK=!OiNHHAnK}x)u9gQY<0r@}Q)C@lbh4kyTn>Nu3oyhC2`TlbBTfWv zmC=X@#sQOBZEx(hANJ@sqar0++lZ9BilF-npRNw3teVMKI&3obP!W?M4DCb`0~;@; zPECy}qM*D=vlaqam1dvA?2?PeqS?fsg1xp9>mQWY+TbZz)T)SaJn8sungq60hC`GX>LpR1^+GlTOAzyn(nu>1bo=S+zwnCLwpy6YS5Z4NnH?T$t zyD738v5(Ndmj~wY+U6yf^-ufi7oLU}+fx0=76fAV0KT-{jJ*1U{8t8l)8cj783qCP zREGWQRHisR!2fqC_)j`S2UTjUX%K;V?+Q}R=|+e_0UV8am8ygyhaZwPaw`#Gbch0y zr}1A%dLdwG*pH};lJ=k;kokojtngqMm{aa%hb{*9&ZG1rp5-KzFmXTKQh4v5_}<;0 zcJc6Ryx;!f{s61A{}5@CY?5u_ZxYwHtF^7QUjw`%<_hUk-fSM)0!cU`cVN&R#&6%k z6FLcpFHiuZBQQ9GV}1}oAwuG7po{5-k0JhlbiGq}rES->S*h5zDzK+`#E}Z?OyA`yvLm59R01NL%C=gg;>hHvsFmHabZC2DHY=s zM>!TlA@13T1AWmoZnDwbRo92tlq02?=!bZe*HxLKo{CbHCg#@i*`w5 z%b&rPRK3H`sNo{ZM#m0-nXFLj%1VjjDB@WC=_>TlDS32@S-Q;Xa2w3$;6Bc7Qv1>a zaPW1P*umABO;v}F<7^xlF2QEUTwiLc9d~o~m!(ik=9t86E*+Jgt9i+})9csAuTbyo zoBE5o`=8(J>U>Up6|ySTHA32c*zTlr`OMdLNsfH+o(wnDHm#O`n#lG_y4#4O?i=@kiRur%{ojEgK~E9nXa5n&$dgE_3-84 zG@^s$l!9u~o3+}fknjM9U?MJm8pX;pbI|Rzxmg+)t+i#)u?E@({*EHm6wH{s2wSNY zOr}KtK@aA(E} z1>PfQ4St&sDH@b1zah;M7%AMRT~t&nnvD?f(=Hb22!AzQhx8$z@letTqstas{=A0y zWk2e>NXZ`kvAADnpnY|vvU>GQ!c@PhkhwZ3r3TtD8qWPQu&JxCX``?S)Vjzj5O_i9 zOE%D0`dwQ~ER6uwOT>UmvNBUWMLMQ2PeCiQE|UV=|f#WU2YeW9=Ou%nhw-eL%@VrQ~N3!B*f3%ZL+5~piNXTC&RlU zRbwJEWod5BWy*_vnx&k_%p!j(l=3JL=J6$zQa0zA?}b@9!X5?pi{#l|+p?m;JM0{E z?E-#(GhL2RTvp&5@*T#0gf%|bfNR)t%y_@FKv~XX59ZvLtMT)l%;m+}SlxVJnU$2& z`-P*xD_*gbcPvtK(P3sJ5Zif%47~m)9)^=uWSsx&(rNsEvXuW%^S90mMjQxRRHTj8 zb6reyYt(p(z6*Y!y8kQPLw7NX>IBknaS%v$BS;)3&`W5RslMx{!r*W;Io18M;qLwJ z1$-MO1m}?r&$9GX(Gbg!5@%G1BNRhUlaj2~DBs@>qG;CJ^q3=)33$*v5`gK&h$DmC zl$Ewk(BkFvgO~%3esxk!JP?>oF_(Gq>Wg<1M{!}qnzUX67%*Hk=Nv1Ph>F19Y4gL? zV?pDt5WEPUj_-3`7u+2Bkd(hlz<|?&gJM=Bfz$>%-fgchVnrHZDFmIjg$2H7xMjmI z?Z24rb}=O`ZTx{+2{tRWU=5Pd{#Jm`q?*uEBIy?`mq@8Xt}f*OVy6jh24a_ug73ws z0Y0K=75A%7^s)@lF1W*&v_?!z8neJ6d}eSRIwh*?Ko{%aBbZf_v;^%QdQkfp_CMEp zjHTC{&{w4^_%D4e+!wpXj z3^UmFMn}m>p>klJz@Wi~N%uR|1J2s*Z5F#$?JqLz^jZKeE#|m4#sMO`oENM1dhJy) zbTw=ji-5DmCNGDwv*2tOKwRxTNdKAdo9~hI*3;+I3-E|1=IVP6(C04A=Gz z(1mj`S1M8Pj)OmQoGTu}%3Y~cBQC+pU9OZP{*D80=_*^WhqF;K3mqkI&8iux<{`nf zxFZo>MfN?`Z?5j3d6ZQ;`Oo)E7MoiychvVShu#p2OfDxeD#T0$Z4}~SpM!Gp8<|p3e z5PM4w-9o-$Lhhm2eOi}&dnfiCi-Lb9;;_CRu%F*W=3w6iDsh}TZbr#@Ohio@JHt-3 z)uBUkG!u~%pNMw};xL|w7YO3q*dq%N?MiX*5MP>0c(;O)J(r;zdniCl-)y2J?v%(V zM#rS&qQsn~_U9aXzAt>{-!ei!k#2H`KV$}h5zhu$;fy2x zKm0{GMbScmj^UK^QX>{(-ui=>0P3MRVt%?^1+h-bT^HgHp%S%c+^pZ%G2BUeHr!Qx z6oB%<4DPA{7w)PN4%hO28E&<`Ex_t8hf`G=6TmZ17JJy?q=!BQ;)i~Tr?+0o#doV- zKo-tT3UT+3T^;V9;7hbLZreEk>!GdF`QRl0cMuTx=L96pL-B`2ppalBq>~nf%5m^D z)pw|aY$g(>L+C-`I!kGk_3W@w(|!~%>mO*WtdA&*?5vM`wwWy2#Nw`v2g{=a=0Dna zyq-;&38JHK9pMovzKai98dV_&2b;_$iUq^2&1G4bVK+vh*a~ojuA0lL)EgTTlPS zDBk63&^^hYi-^3taibc(b1nXT#y%U6pcHY4Fy8z-LQ4=xKz=^2ZvK~mjM1S6Ux{|w zLUb%BhJOV1=g@@qQ2$(flh#0$wZ(<{4h9_>2SfIHIir(IL1NdX9ztSBv>goo34gq>Q|3KGg!T&6g|@Uou@{xzT1CcBiT(P!v5KWAM4(~SSCU84- z$}^~Ky?!vs$BVyLzvG4>M)k3irKOI!CivjwD?g;eAx68ru1tIUmgd6m*)(x)vk~zn z5W8MAx>E4q0)WcyeBT^p3Ylf;6(gMS>IUc_N`E@2YBrKOQYPt}ZA?>aq`-p*Qd}Lz zc|A83Q(&+{R}}q#myM1|y%AE9FI6nswO;GLX~kw0A^7R1`#TMEXS5!RpR3Q~7D_Al z=iM)StQ<*x>TT~oKYFcxESJ-6WkEp>dlq1@RedghsQ5tgz6tv>vtuKsdYhR-CS}Xw z_%*x_2VT_XCyM)LzdDu_#VXo@0Rzekj3b$tjKl*1g+GvfCMIbrD0q3*4xFEwIvRd} zr)(SBOOfCmqqSjf)_O$u$h%!%L4~<|(j*x3c@$%wd_$g)x%uEfK51Nt( z6}S3NrXT$Tuc3v}YSBAPeKmxDjB*TY!P*_icL)N67{?jrU`xYrVsH>I0&OEX(6IKj zUdY6#jWbP(`*3Ib!$7G>s%zhAK!mm2j@Ox4uiz{OB}3CfvGL{VZ~x3go}mj&VZaOr zDIT;sZJJ~&q|btR4eebWL%npgFRwqZ{1mIUq_s{cjTn;9iG(J%X%3v``z=Apq4R27 z@W{--T2BQ z%%cQG^XL(U3fq9w7*q#XhVlo z#on0iE+lv051uYu~uZsTYjiB%(Aa3;>m(z6n0wQG!#sRpH%8AqX4EJ>qCGZk*#*| zGhI~dpOFQp72+WV!f{My`-AZSUWgKDOXkzDH}ly6i`7Q|Bsd2h5MH%wQ?MIA5 z$rqYmfhY6vb2OUuKz~-E#%e*QKb$t3i#SuIuW!{3Z}i!Oj+FExy+f5mTbELnS_J8y z%b2k*@d+d0d5+HKxf`L)M=SYl84MBUavZt3kkkxvC&q~`Cd8F0Tabn8X$8uFzvsH+B7SXr_XSf|Qa4}5W(JIi)Zn+WegZ?xn-qyZkmu|E)Rn=Y=$f#G6u-w} zYWy7&dKPY4fdM&U`de_G*@18P`(C$MIlXGb$-6;#wv%w&C1}gqo%w${B#PX-aMr5& zlXt|>n`MF?wykhG;&Pa^kF)wVsN3nRInGteszv}wxR**s3Oj1J9nj!!pqBau7jTb9 z{$Z!r5-{EFW%tQ@<87W9=Cv~5`cYcAx;AG3ePNkqAZ|66)W`5}0x8h3Asg|x`(o~1 zM02nZTL0KI`xQD~*DshYwZJfZWnXl>M3s~o?m-p_b0wbh2e~~u(D#OA?)7=zC>GH$hV43!$6V0c z`qrQqj~luYq(whIW-ZA9SPO&K9d6}h&ZQf|=w(JlR=v;JFE{hfT6xz$Mu|!cr^is2 z`TX}YMiFk%cpQrEr~n-5v|G1MbCsVZatDeSK=i-7jWWwdH`mZ{6&}ld76uk|Z3caZ zo3`vnsgoq7d3UWT1Or0cDMl(TsM%;4@OQSQVH;$&+0({%H{iDBJNS?CCXH(+6>j|S z0y*2fO)s_Bqq2BD-$OF0hP*1W(u}yC#;+L$Cw=Uld@zrSF$5WM1NVF^Ap%q;8Ee27P&nm~IxG+aXhH?GyT~{raupJjdAUA`MuD7EIF9M)B1<@Au#N#dCV$d|;N2 z>ksnmC`}Q&#uN}(;5V;!MGx>G4g4O|Fngp3)ON`J;t{qKcxzA(=k($j%)POj&-j9< zy#cnrO9%Z#P03eL*`1Ju=}b-f&fo;o35>M%MA!F(pS&kqmtxRaKi$r3sMHeWgkJ3O ztLI?Z8n=SYkeyv6PV1lE)nXkji!vn(Hh~}D*fqoTU8es9D7$mxPlDzd1IzQg-(6pt zQ`H#D#@E?#wjIC;8OcA|=Zb1t+-9(?lBf&>XD?~QcrCWolWTr$06#YAG8q4J2DIp^ zqI7F4dXR2U*#_EnAI`*PC%xQH0P&W{L->^FKi>@<94ea3dM~Pe7=S}M0v~rKwV%gN zrgmzlwC`YA8F1e!!9V=rFD(v&7Smi^w6&3S$Q8!Hry^Xo4rQ&g`!*+vH-v|s;MJm! zjv_|oXbA1E4+8dc=xrD9E44jr@2B#eJaGJA-0W)o-_bk$6@wb`dlx%mfg7+BmvH=6 z6&?&OH1U_JcSf7Ooz>zs3r4h99sslr;T-6LDIXmu9;fO}c>Lg)6>ebS>7Fj!_3@#o*d0p~)O z82oD|bQ;+qpiB9uoD{)CzIkeZPw0T}n&TWR0SXdkV+*{y3<9DPcH*VBJqlMF% z*+n^el+6LVn)%8BIb=mfrMWh`mA0I$%y-`^_tqss>oYI1SgbDJOXTIIjf*FfZ{3H`9di%?HYl)!{`Kxwyag)HMp>%3W%S za7iE3Z)(xLkJdG(fMWQXwdGbHk+V97KXNLMd?dTQ;QYxi2C!mVLv1zEWVZFxJLKIY zPN|&_i!^&xLABeQ;jotj&Hkv%8jcgU9z=wB73p+ay-k^_*ElB{Fwzs z3>TO;I_Ra%GA20}SM=rK1{b^D9!n6Amer2&M|r+I@VV+Dy}CQQ!0#nu*LBscY&&LG z^gHVl+K@nnCp^<6I(~fJ!|p1G%e`f954E7375QGW)I+{}5rqkj&MM7JH1W%Vc^7Dv zZi54hA=hZkimz?_9x78E16^LS0#9NY)nEAT>!0;3G<=2iuma&Fej^&RrVv7Iblv0!q3MA2@a}Ry6Jb(AytI@Zjd_b zY=JI@KICBlEO~L?H&E1kQ02;Kxf#_!T`L1)bQy@7Nvnobkfv+w1&KK%+6?WM+&d#{ z)J;UdTB_Q_Wsny`X>d&`-+RBq3eC*NKZ6?S_|8wPUuwdifA!A(dF9fQXNjt?-Z&%>tjsK(FM=6s9ztLq35j1>r{#})$gg7ZX) z#Ejj6$;`Zlp!l&x8lNY50F>H~+g)B=Wz;VmgdqaQ?H$QBo$2kdXfW+!Bkhq1s60 zdh_F+C=zM$6Nx9HGbPF#;jo0g9M@)pVPndaphuZ6FCU-16HC)06u!4vT(3|ECf0g!i&Q0Y}}4w8hWn95F95AQ^<_XK^#aU&q`WthNGMpm4Z z{yil$o70d9T+1O#(^l^aA34ReyFH)_yo!5zns_s2SNU^X$>DiJyLVWdJJe|)ld@fd?2$zGLfs`{9D$A<4H2F86 zEOwV3qdGaQOQbgWRgIQ4Ce_G&Kc*?)?{avT!449Cf3gA9<(F<%G?#2zo3;H}nzpal z%^`k~1q`)blCc|fL!`OG$|FQz~n~W#_qoINS>woyaTO&yS zS#{`?toX0N@PBMj4R$NOw1nWIf=aRZ7tQs{909YU#9>lm2`H%E9eNz?tp+QX>f-{l z27Y1iN~lb?+am(ZOsDHb(#izK<5^xu*<2Rp&c1IS*U*Ca71Ywy>Z-bGPI6*eVq!tr z0pa<) zrwNbc)usu{tlIvQW-2_R)b2gUsYkv6w+&7dx)J7dB}?cOr!waQrXBeg+w1pH7_t43 zGoNGgbF$thz!;iZWIyZL++UC1iR@x)Y2r*4n;)-GnG?df;p_cG~J ztaT!zQ_`k)-g^UrvnWJaAx&-<2w}712b5D06B+orrHxv=kle;LlhbQ{ZZeA+n^$`z z2@Lz%7&LZ6n1nx^-YSLeoSRAZ(!OX`q;9}X-k#OBcn^^+EAQDYqQ4Lg+#~B=k;wcc zb0Yc*Pq7OzuBSvJ1~2cYvqUK+D+NuW+uw6cWFzO@lQT!YkBnz#_}#)rP|picGDb4| zNOBUZ{<|cYMdCy&eX~?a0)7t~dHGpF5ciAU45skL1)<^nH3Lpsimq$;LaitN8o80p zv#ytovt-k)-PYiyL~BoyTI!_9J>N? za`7B*#$lQK;Vt9L23w{wLu!UCE}pwd4wu98w5J~ZZV1Nkz)&sZ6fxS+R$^nN!T3-! zWE$e?@Ycf+NQ?Jt;W$NLNUms(9?I$LWL}C$j_Br(5CVjE6j4{{Yo}|MkU*r6J9s2V z`C&!~PB^E~@zzjS=3=Bcrl{ITY`EX4dmTT!b%s(S&q9B<`lZ65GX*z0if$~rHF=1$ zTn#O$ta+|xn?uEMIYH#V=9ger z60OMzQRb!t_-&SiD+$!Bs_96pi>C zZJjlY`}>BaOAS20M*=Yis~5I+KRGjplT~u;$hX_>indqsFcWUo9IG_?vZI=GOy!?E zB}Es~Rd5#)1Q~gI$H{o)`Wr~Wl9O^KsnuP5;o=sZI*jN+TjlwYjI#J7=}!?OQv@W7 z&5wM%zW1L%_;wR7nd!L&FS`fQqZ*_sHi}09M(u+ECI_!r)3`*T)bIex2Kg^ zBfDsh5Pe=(;Nb{@&wO6I%(EZ}EAsj;+9*uKs)Sh-Z9<}JGQWJVzj_qTVSOTc`#$>1mjB5V1r5m~{&778iQa5Uu!sP zQ}(~b1lkBZVCCyD;Qhb-sGy4fGWU*K z!AkS;MU}FmpTmr|#G4p!33_E>3{kJdE2FR21PEor+h9)VfyQ#?jn;-99<@`Dn9NP% ziz976M1l|@PD70@Z;`uH%YaLD^Y{1+TS3OLE67h<8qD(%dEetCtM~u*?|mpHFjvAY z9R1WC@uXPTl+P4h_ZWwp^p~=nMy#)WrVf6|k1bpsm3*H58$!O5rzHch7gVKKSR+=F zD2z7KhRtZshf}YxI1{8%?ME|LEI9`VX9P+dXWIe$jx(kfU$H9@Q*AZ5gC5}JuSlq6 zx+HR(wG-qcEDaItm3agHaDkh}ia~RhC>JM!hU2!n?bHdU(Jr08^*UTc;W2~~#Py|eTy`&%ieo2vTV2Dy`GhonC^i95!NX%m8*fufopq--Y)7 zStul+f}_j-3FcN^lgHIS(}RrR%Qmv5gx>eZ4;d_cRCJVu>T-|nAREq1Mqf+2Fab&ooScre6 zyrDlSW91wwHYk}cW*;kKfEBVmwg=`<(5szbzn{8njuL@nx-5=}CJ_9?oS<07LB>Hl z-g&LJ%ddZ^+thtJ8h2VUUmx)Tq^zc!bt_CcbW_&4=SoxDR=b-HZjqYi-PWmr4wGZ@ zh}en3MXeKz%B8BrItZBD^=CRUWhc5f4b|SE7!&rQ2L(6`;@=HSuFmo9wrsT~$!o+} zc2{~c;<``rn0@=_i8Q)`1dD-PrQmMyo?S4hjk_b`!>wW(Lb79|U15ZLtX+t)AV zu6FIodG6ASMxJ2s`NFCzq2n%c`O4^zuBVAC&PZ&0c=j8f^yEH6mgfv zS<2hp(n4?mty<-fJZox}+V@muFlR5y;x^X^dfgrF-wCv+U2JH=Ku;^FwBc09yaRVB zLB;Xlrq&;eRrRtn(pPR<#LvQ}N2=~ectS^#df(pVDf0kUyBw4qlGLzXp(7Z~9h3^4 z{=Xk{72aT+Z5$`P?_u6E5_(9QLhkp8UfuJkO)YIPoy-K9fxG6~nnEtBZ&H6VU7c{! zZxGw=I9_AGkgW)UL`e$w-{xF|pV2x7&`#C4fBKCGOW~9)P3FXUFCGX+@7hW2T>KXu zsXFK1ne7(8y*IvczJWM4NOm}Ku<4%S#v zY}(0%2W5v_RctUC4HjX?Yc^UA4zcxUg*a!_1)ck$3ktYi#VHLDVPix^Sj$6^U&({f zw+f-kUCG0#wrEY*v4gv+4S@HDLkvSgaJ2JF}edtqOicwi6EVCfHC;{-1;txsQj26HfW#_iC-Ihc9y z4DeuY4qX!jKQZz4UjGlCgK^QHV?r@s#i(Phm*jzH*AgAIe>F8_Fx(4z}}p` zuFW#$R)tmB$<{9Q)3M3JxU2*R-_(Jf+gX4^To48wME3v#+QCfuE1+(>f?EtwpzB87 zVPi1Tnraqq!&IuxfNUu{qI-+>v0c-GOexkvf8^z&`bzv-fN9Rv>r;O&@jvQwKzHqP zM{^w@M8EIV0`G$32LtEXeu8;z%R~LhHN^B*=mT7{fiwIvV15(pd$=Y9X9zT)X$8qP zeQxvDX$8On-(>o#*4yC;)M`T(ng<-v)MJ@DE>g7;9#MJxiqikyqwM(YqjJ0G+8vzangV?98UUVfZ3nJ$Z2%7NQ0h~9F7}t$ z*@7VmnWpjZ2Po8vEHS=li-Ok-q`~ON%%OFH_~vZ`C)XWe)u}yDD$qJXY0x@hKgzc^ z9JTwrwz3gisH#hKr$;VBkD#{D7V{OrEOc|R%lA|RIh4%5eD95qRm*)qiK4q zt-P=Yl(Kf0ym*%asLB-2_k}KE!di;j#9NAJ%gO|U+HF|d?n!c@tj9-6gq<-AZKVG^ zyq~ENLq%3Q(pQ&G-?4NRwJ_Fk;2a9jS4$VGTb#G7%QMpo9>9?>)S}WR51oofGqE2z zr>h;6q8r?Az6?#5ey9-HD>e~DlvvF#C%_^HUNrNq#=6;7)fGSDa{gkhspD~n2Ow5R zvE}0WsIom?uV%<0THPFsRV{Fo)Rm+P=Oyx3Che1{S6Ev7;-OD`x|TeWh8K@8uw?Su zWJY0m1=M7DFdi=dWUL@-Lo0qDt{aT!iB&~~{@q#$G8^zNw*RujbigOK0Trbjz}sR2 z1S{voUXx*okRG(}05eIEju4;8T1e%{qc_P`hc<2-#JGJN-vuFgd@&{o@q|4vQ|F#7 zTO7_mAsWstKVv$kXt2Uty#I-HW(-A+x(? z$AQn8@xOI-O{vng#*#bG#@_DA=dG>-J})NAB}2FirL7K0pC2SFGSncY@VFS`DCqyp zeQW(qX0+C3byS0`-JX^t7wti+vL~s$%`R7|HU(li6le|+UT8^Q28V0s4_{Fm>NT^d zjB}*ARQNsJrJa^w-$kuay19dPs8hw)0$W-P!9vfh?036z<{e;s&CVpD?q1mqs9eU4 ze&VS7Sag~OyrBX-#iNwKy*sruOeQoC>4yETWntW*X7v!0KZT?y(%uv&LSIK|mdTPJ zNnmC>*{gBwraZ3R2MICx+bh$+eEFSwr7y{+C+TtVwn6?6K?^|1V92J~wKcwps{HIX z+4z!J*~jQ<0#7dk=J^prWI_=q@N>(`VXR&Gx+0iUW#O#9uC_Z{pM5a;YE~FZZQ~|4 zn6ZEZZ_X`B6tc-wz&&v02icSJ`8Ro7gT>1w^2vYH%qeBuB>%evB!2)PDDZa&2hj%CPL6dfz=prO*(3`=VkLO@h zWwN;j76V)fVIsN{M|e-}uI_Ea?)92ie|(~twHbNxCRJhr*!Ze)L+0$p!;E=devnX8O*2HH*=+KzmWUg*?Z==+J$OLo*FsiYpOhwUXSpkvtF5<%%Fz^Wg;R zz!7nVbR4XyP)ZF-)-+6s5FEHx;Q#_5+Vj;g%e2BRnwNrKT`vP1&;+kCL10Qp7ONn$%`#dyxTQ~o&`P-9j)AA?7rep6MV zaXD7LpmMx(-&25&@ z0=M@CemceeBQ~Kga4Gjt-6wNZicSbjM|tnsSvu8$Pu&%>WWKbSD_^WBHQTv$y<0Ja z=Gd*3ue-1BTh(u}z10nRhmsN+B**7Q1ox8hjUmtiuBLy~ZLy~_) z^EXc*PsTtTj+EjkGtEJ!D)Zcm3q3Vzd5wZcoeiqkb;^(nr>gdHUR>%xB?9YBw6nDjSM+tVAb@y)WN*OEN9s2N zo#jxMVM=bJoRtV~-->LR?2RpDAOetb-}%Ky58`CBRXQoZtVr(%Is# zxc73YD7fPKg>hYnS4YkTzGlI95Vf;Yyu$B{3^imUFECk;WEt5Tq2{R$2$OqHts<-@ z5A98KR$27aBzvZtNH^`Oo8|W&@`waEGIw5lXdy*=z@0B?#5r!GkD`=7 z>KmJa-JJy+bDP-re-@d84s@$dtQIP*`KF#|0#7_MFTD{`B;bQf!L+~kEFC=jW!p@b zG<-Xc_hIv2{@RRD(RQj)k(NG0jzajdAk5-G+~*uFkX*>qI!t-cUA$LqX*x-Tj5|0* zPJ(<5;9y(eLa+x_Tuh$}`A|nAd5g(aHlN&3|SH%dk3gRUU zdNNXtsrVJ?)A&r#`W{SvvsDA3$22SVZ2uJt!78NplMqbV50qUE$BYIp_%pIgyX#Uj zT)DMi7c0%{+}<7pi}bf4Qj;~6Osi%3H2$ysXRPe57|BJOomtBZ&4SR(i_oFaS2^c+ zgWcMRk_~ZyHo*g*lI2%HeOy5dA3-`0bzOPjLIxS`ZExP;2NUbv4M#A3gDsb%7k@${ zcFG>+*(P}rTdL8!RS3O8$>l}lEzWorSDJM#GG@Cr@O=qUAiniT%58(WMQYcrSEEW4 zI5rV9xw0M~tWJf18A@c6Qc59usvg!ZR`^P4ogt=YY_8nYKn)*-@|4SKgIL)_N?o75M*_nNGq{D5_F@N7;GQrEf_f%^yiq~j{^Ku3ZDi~d#GSU@j(GJv9*GmOI4M|zxIrl!@{z5OVqQ;nMjgr4hCadrD#C!aR8zkJ0VV2LZ4kZjrNpm;f`sv2( zaK%vb4WX{cL~kTGS`HK&iyZ5goC*f#RxDAGGT$FP*t5V&p>>ZN15sB4c&!N0eIS;{ z;uy3LRmg*O8Y*PN>JT=&Q+;pdF%f%KNbH3I2QIRb+}#@4>8JSak-URhea}DUbFdD} z76!<=ejQpkiSkjmV5|v{bec_I%C9%HpxS&jZzInNEI4v5s zG7fCWuW9ass+qxOkBp^>Wr+PqhiTJR8q`O%s1qqpn+VEfI%h6}S;SlCUJjI-gl#bi=-zhxNoppB#r=fXXvtsj{oYjUn`!gfUT7#e{l+YVem6$?O!`DJ9JYWl` z51I0J?cgdj_cuv5RqFz!(8n_?w}Zpo87F73v1B)w1QrK=`W)$dkojzlrkY+7xlk1x zJvbJ}PAAZ50GFW|QpJ@xV8w2AKm**ty2criuyldQnQNPyEDx@mP}65cRb)Qyuim4H zeOEm=9!wR%VV<}kR848UGPfmO@okX9t_8B$pp5t6=|<0Tk2Y;tnLo3H+SNvJ;B5W! z=+f4hlAoIA#GR4?u<5h4HfYjqhr5#3r3{lVL)$VvlPh#}RFqgj!mFt^ZcJ3P#KRzYfcG@x(dgKA zg%lYbdv@3mad7p}gRQACvrs8rSEH@BF>~X-b&av(XHSIcG(Jc=@=vSJgQ6^i$VXc< zRyhU?;Ddq%E*bliHIpc4|j$Ys+X&CuKCKvJiL&_BAfZ`o|-hxM={_&yZ z2Y-AZ438e}j)VS71*!UJrCNT27sO!5v{feRtNsVhN#1a3&Q zA=fj(AMI;g*ALQu1!TJ-M!|3TaK^uU9b(-;)UTj<17lB^GiM;f#_l&jR*rR)Wqt@{ z0y`}ceWRica*tVQq5&TVUih3xB=oo$$?Tjct!cK$;yVfLD_2Q#IgSF*FjnBepxBZz) zii6<=K*}V|;v28qk-1>gBiu>i7Q%TI0gqP`?d*PkfYJSRl!R{kGDurV>ow@0)@xvB zN6ct<2C544uMyGXJ(pYHPiFzLecZa!Jr($bUR#P!)9&Wg%WKxRYwlONgKDxn6f0F@ zZ*Ls-e+Jq^P^(+kdA}W4VYZ81v6p6Ws|iFAm^y-MSuUext%y;utuVaW9lV=&0=KAB z2G8SZ2ZXcUU^&!ApeMs=BQuwW40(gj2)U;|0-YyH;@?4zwj2{`{7QOa=&>;G297A@ z%=t%Ww-v$Z)8!pv^$!u+F9iCPg>az-DA93Pth*vMC%XukPz+|N$n`zKEJSM@b5qDy zW9U7%%3D$+dujz67^7&Ll|LbM=>U|0Tqm=*K@C?>pw8x}M z$zRu|p=s8+2!8@IHbzPyFG|fNeKvG%w>5O?&}k`w_>AoN1<9(MPyXeuR=Dls>`;~b zCEhSL!PRi5cWCPS@o@{^YsafPz9a)@f$O*Sc4Mjq2Y$W-k zf;g;>QCFHI8aR$a&RCN73mbmPNlpa&Cp-sZRi3{V{9Wq2rd4?C{CjjGsBoobIm&Q& z(Q32@jjV=;&Y`u*Nb~-8qb~(brnSG(d+8v3&&xc?N%*x-7)(EAS4Oy5lP|@j+9SE& zik(N9Rd-GI2u3A{bxdr;WuHXYdu*!_dayg#I@)512RIpw&x`mMdhQ+mq>}wFp)r`5 z*~3`GYQ4PT(wn^Dib7Vbm!lla*+aPHLt`U}AXz>ssc6MP_ns|>PJBZ>-73IKqh2i} zmM5)i(`mz^d#|qMKyUmDB>NFI>EU?G6XCZ=;wh(qJ@OH_5!w;F5#ABH=rug|@|}?bYsW_EIdZuW zRz0CrOT>JXogd{hD22eLTGlKfu>DdTBk=c7$pY%uXb>w1bhw>#f2{VRSv0uuS+HSj zlv%&|<~$wGiBaujMm!dqk>)_U+hSxLlt*j*E#wzFp8*u;4afQyK0sMxNAB8BeC=}*}k~m<@DJ4K$Yo%tX$x_YGB|?0jLd}+N>!v z6{Mfm0JGg(T1k_ZgJNCm1{&N0)f@&*&_wWPjWkMd^%i*iW3dMj``m7u^nS!=iI(+? zf06dSu+Pr;69!@NWx49BH1m++ogmse9>y+o@O4_6^~DwTifr|^VpX4Ob1aAGS9r-% zG5&@}bLCN|I3TrzaR5*yj2crh|2}{UO(062;RD8R-)$Rf;Y^HmoY9TEGGito(E)nnq)aV7@|K46 zk0YUqs0WB8OW9*iv&SW=*zXP@uYhzDu<+JG`~vHZt0&A^53qyl#Qf;;STvBvIZX`a z-9_%le>i5nt%IjKU&n#|fA`b>Cm8iiwKMVw+c^E z`lkx_p&<;?epA0LWa9e;iTerJE$aaGKBAK=-7+6HWdb5pqErD$TtG_R} zJ}_~9?#1F@(cLY;50YuP)E^yG9;A&eL%CvDYa9L>f!YzT(qyua-4>!{^Eeum0O-sR zI-m#^7MX-u1Wg#&Z&SpK2eV;1x`68wJon1`C`6fOwhah8wfdmEUsm)XAg#-0g|F}+ zPm-O&zP<3E{6bbJhn=zFKy9#SqA(e#i}w(}_9r?^1p)vrPFr2B*ykOb3x3x{Mm#Em zO6mc5y%GhE!dPV4=7ffp?+Ptzz_jk>w%^jv+|gJF7!`%sbP#eIRgw+K7s)Xu`<7yd zw6pfjh8WGlEqu4+4qe2b!MiL$tOE5zw0a~ueYbF(3Z@Bk_$aUVK`8>q)GJYyLx=x> zly+q%DY(9haH@Y5;s1M7kKzkk$QTBNlJa#kA*CR6!9W1Jv|$ZZ7t%haCbHUC68v}# ze;J`j_Wv>xhWjbPT8Tt}i%Wo_Uw4r_HX+B%nHH7+4Ss4|RIzbkvtdtYv$1GbDkrl- z-zw~{dR-T@SiMkHt&WL|T-C9*TE3>oR(0~Z<#pr(Dg};u3A}vy{PaE&IQsbH`sCWL zdm`Wj&s7I{O!iV3iGvujBae9FBjbLI2Y)v208M#`_C8JDdLmoQI!X5QuubZJ(FW4e z*4O?PP7ad8aMR*6jiZ1qp)DRWZl}GyNDazJ4NMbFU7sMG+TW9XWO`8{Z(-u<-DkK+ zjiTq_h2k_Ek-&2_ECK!b91kk*J?m$sh?hFm>HP+53ON5@OwcFKF?Qxj;7RpPAG1h( zO9BmgP6cJba=`cql!}$&NI*)^NqfUi5U{w#l;t1U|H{WTy)`2@aEkTGaWzDELukEG zBJa3mL^T*4(7k6|87hpCN|X{vz)Aoi50G_ZSs5^8NKjyG3mWq!&jIXKU;%b>uIb2u z)BC``vX>^#0tu+{yupObFL9^}yB+dxJRh@RCD)~>3VTKr9|2=5x;_}dfqjEa4+H_H z-JjkWW2o|P6a>z@a2nH&58&0=2d7%q-J=UGx_l!GDXpH_6-RBa%o+@i58pKmc1|le z_XlmToOdzT*BH_|qX{{0Y??pd@!WUmct?TziKw3uURlrh8SK$>vIGDa)b1lRb2uI` zkV_{ROpwqOFlt^br|_5Obd|(i!pwZQEvIm2-d!}<>(E~VYlH-NiebOnjGZ!iC&Gs= zI^H_o7mR>~C7vcr^khgq!dm$6LpWGz4|u8 zu~Pg87hD%TLcsRksxX}W!?B@M614>=wksXeztPGJKIRsYDuB(?7GP%$IV^~Q5$m5M z{V=2!S20XZ&%BFEq>fE3e{LC!A5X*&>YD*Vvt+$7kk-@CUwdSWn6a#-{T`wcb6ITXK zsd6cpIPV!oyF$A_NK8<%ala*m_;+w$H$7Q%P>7U~+lkaQ%5r8Q&IQEHsa#{FyM)RA z#n(ATSJriVzK~RG+qP}nw(S$+RCHoHso1tvv2C+r+tqoVdvA~Ke#hvs$J!tFhrQRH zb4~sJzO?a64!WI{AQu4(XQH@q@=ZSe7dZZowRZWRjG^=@PwJ{ zo=VEMsDOyfYMNA;5&gV4ZNGO-k>IGR98S?2#;)A9O@yxcLhY@`l3dS_6yf zQ*z8!xIL+|_R8Fh5PNeXu=ai^aj|BXf~RO8F#|$`iL6EYcIhp(cP(Nz`PcF{I_Ra> zJ^xL%TFm-zR&@wFmJdhE?1zzXmjExbB|+W;gnpjoT69@smVD6%VjP;n(iB~eVFdcl zrs;GyE3#?<Y~9ChrvG44E{<{Pn1e9G5w{j$jiY_&LptA+`ljB zS?+6hosX7A-Rg#(%G`X-uK+tj++8aGZ2x<03%-9mdV=b@Xx^-cgJlkzRgMxteZ8vk z#vjReK8(Tq@*nN(NvX}^TIwouh4TfsC+(-{C&{nvmb%xL+OKUOAa-GC7k$Z%ioxny zON0G!ogp8cRg;J6*jFpXz__ZTEV=>-f0E0G5C}B=5WCkzO;w$Y^S3hk-zB*T7qF>H zU>KPaw~GBCOg~S$r^gJ+3d5!nt)DCOixUd9vu4KXy*%X^rcVEWH`?a8aAhhGT}`Rd zeqmnMi|YErh&ehYaW6bJq+;w{Bezu-6)vlj?7KqR{BE@)|;EhYe$2Ceo z6hY=AqCuYzd^WCnCp`eXh+hq@G~FRRBs^6W(Zk2JC5_^B!DUf1n_1-Vx8elThf!}5 zE$hxQhRgTPNM$$Pa3)9oc5fTdkX2{kou*{d(Nz{YMuc8ZMp+ij@aTDaZ$r1i#FLnawD6PcPk^B>*TL&GX5-; z(sDKfyOp$sUA+Vt!xzP2G)ObXJ|{h))8Wq?8nXH;ik9-Py-vxHIq? zG+WG&QvmGT(Yd0&21oI%UWQ=6{baplqzdP;^_H1UTbC6WX@q!#Wy>+A6^UB6<;2Un z1Or^r!b#HhgUb}R_;EXvMW2)nw=v49!VgPe0wy{`+nLxd;i6mxJ-A)-k!)uYId$PC zo>a-K(KT5BGoI)+LIAOyL{WumLXr~4^N9T@NDR%@gg;b=!JC70Y$`Fys?M3&$b2^F z#t4@fr6927^*futDGQ6)pLE-=X@~OZ&4!qsVKt6p67y=>5mBvYHx1D;Pu!mO9E=da z$5i~^9C8$4L$!6-4;@bHB1{GEik8y?(7#TYbOk^cOt_fE=G&6-Mlx1ReFY)NogL|+ z1aqfeZ$1ILL%U;;nTFJW`QKDT9FOGRib*M~m-n>xoW&*-wdX6a&=pFRSc1v{*Z z7>gn@B&I%GCXpZ>|FTg_rtrjM(E>0amolivw+zwht>Va1N;D)h-vn}I3_NpiDjbw7 z%IXuGZv>g$HLF*2UC-M+FnoJ|&pq~jN%r`cy%TP3WRIta6F%($DP{vmcDCr`L$~x! zGgM;@%Mne)9z7*{^zo(=%HfofMXXz13|UW=-Eu)M%M*5-DYhS~k^PVsvzHD`)LTuy zM`ZKUYz$0CQ~uK|RglE3?8F_fCbp|9295ojPaIM}KID};`q)vQZ_VX}T47t{n~ZoZ zD~~Ui2x0f&V0_?b4+(E3Q26_(WErOs8*7ZvyOoJqK7@;jI+b%gC@T5 z(JVo9@%8N^m}a5Sv{u(^ar@urQ{o;vB4P{&hO@rQyMUB$Q zS8K56JEv z#dA#dNTVgpi*11JR5{*TGvy0?AOG=ek*WusrN1XE5<~ zj&%)Q(bnwogWW#LVgTtPH=a#VapAH%_4l(0+ZqW*5lbm)BH3Sesa!~TNS#sNls&0E z^Hmj4iUVHc`5+HpUKMLMEcAPu)zxH_TL%G)NCH7>Z=n@Fk%(DR#IzJ(VbaavHU3Ct z@}2>7{0d3MsfNI^IwO3F)NI2hkAOi-Y2CCiUSB}Aii0Cx1E^u2mZY65b7ySsKQky!Q9Qfjh1@nd??qJV*Ia#QzcP$e((qn?4|If8tIMz*IhO3bA93| zzL@h}GauDoLMa|!`F3f0cg6bqS)N)v(3?ZEci69SFA#xKd-c1;CmK}#ywQaxfEtZI z$6Y2)`%^wC(cbVM^%TB-kEcr}6E0j)^^zfnWmFjh&K~r|I_NY>iF=x+)W^I=wMtS_ zU5&rJI1gtFf7Kh!xz?nq=S??2+pExw%9htIH~e7huV$-Mn{fGUT|ZLyMsEww5sHi} z*Y2fqNk9YSSrP!#n1mywYQLWNHP#EIdN_UU7GLJ#^k}LE!->q``c>LXOiF~IN+Y%5 zX4i`)o?KtI+K?He0X*249KKjD_NIczXMjZW9IFSK?2n7{JweAZ*J^YBD#~; zu(%{XI;^rUfYQP?n~vB!93q_9XUxpGU4lWiB5dQCj7Ayu%V^k{I^5Bxg$}uz_YYz( z^;d%VjrB9Vt*+nBO!8SG>nifo`srgeT${^jq2yhhu}wtu~mPuLNs9^J{v(FPom!^2u%YX3%&1>20U)xa^Y~8KnXp;&#)0#O^mi?%Y!wUJDwcUH9w75`dsX9 z7SG6&JWUvX$Y3Tj(OP(f?^uDkhu<^~GEvouR)^-hk4SfBYdBTCJIU9-l7GbfZYKxG6CV#y8kG;OA;% zOLC}Kz$wb0P?iF{1}cZkld^iDiI?gg8XcuPArBviQ*w1n;{<(xg}KeU07}LlQsr;~ z05TEUG6p!RmHB(+3gEzQe2r=rmBM9#>chgoBhF-$I7^owi+=X>3H!7~EP>w=5W|O^ z$-<;p`=!hzgx0D=@d7q>@HN{}Fg0Hr>E7YySQkBFNSWryPs>h}4GMX~#7wTXrHN`| zp@^)>8Gvjp?w7LN#xwYxUD z(?5^$AH>Wbil5v{U^|n*J(Juzlic1&92|-t9LBI7h-0&mz*m;@EXdGUaVQU(lW5IN zF9TbujhK*Rp5!WIFz-s!;$$R4nK!FWXi(0N>FKhxy(xuH3orToYdE>abtJVjp; zmt!Hcl<_UUdOqt$RY2-FR5_nnqp$j!L*wDN8}n!@z&{98Yhhk`p8HKh=>eQi@`woF zT{tqhJnU5kS)xjUHvFTiha{elD3K0hBnx#c85m)z z0A)raT30+!heo`@pL^yJcZOB_dAIFWY*HrmR;SgWisz$9%p{EGV@UqI`+mnmcc*@5 zT7Ht~A-IA!t_$@6g8=pa?v4IUh>tOF1&=JOR~_ney|Y-!{3yUAdchz}qq{^{Hn8q1 z{8vSEUq%sTI3XLie_pt5TmTn{Ro9tNBLC9qm89P<$}iw9&{#SjeJg9DcX0@D`H5mU zTlnJ!Yp9U)PHsLEa*YC3ewJnBzTE-U4SP{lXq~x6Q8HdyXAxG(!IUvXS9M9Wq~pBI zaxdxDYxdlH0&Vq0qS7-u7e4Eg>!As6EdfwO)a+uB89#Nh@V1#i)Gi0o?0<#T9G%Q1 z8Xu8d&>{j8M-%>F4u>6bKDD!GLb$uu9qS>30o)N+COLYL5NXksCla2!&$&+MqTz1}HBPsP^eFe294oiB#xE*IN z2jj6$_apY;v)vMNaijuJCntDnNz~iM!#}XSI`#1f!8{ga?$Bk<@K(O^N6#YlU(30D z?6*t)r@Q|=avc%VAv9&K35UEfQjr5#kVcOeYlljj47wVfJ<-L)9v4}bH z7k_agpf7(_Nc0-L#&^?hp8v{eSnL)jurAsx`;&PFTi@^NMK#j{klS3pH7dSl!lvEd zYDdohG*u{w`PIU4$@6^XG*v%kO#MNd$956t=DTbjPgI9A?f~jsSOm(6C%loi-r&3J zX*}*8LJ$Xh-U&|&jJ<1m&3C)trF*#f(AJsVmLRFKrj}DG*aP|dt=}eoF4>@r687u0 z(u?7NZS8^vMa5%)`;j~kPyP60 zGCEpqiDe`c5%xd{Q4|yd|Iga5q)5Kg5Wt@?gA^zxKJ3!Md=}j2$}OTe8FxI7ZZIn0R|?GIKMHC`2Io5@0i76@nQ}Q^3MysUCXWhEX#Pwe94?+sH(53?}9TT%RIGz2T^}#Mhh2qCiy|;N-P~2 z`P0m72N$jHf*8_i%=Lg!G| zpo&OH{96E$kzmOTX&%36S6*gYDGwe1*}Ec!vF36)p}U7`t{SO-Hw8zxSjAuo-py6g*FJKgF) za9*Gf=Bq6YiIK07H}bpzkfdzJDrTQ0{TLueTB>Tz0k!k`s3;uxN_lxCUNLksMlq)c zax+M_oi6B#lo#nfp42fvfXkn%`6bWN7~-bcD?WxFRtlK6qAQ9~P)=CWK8rOkiRQ1m zUmGN%v%uO*mviW@WTIKfQ7JA5jKirW2e~42;AZx54XG!{f6^0x`5&h+s@vipr(@!s zWR~g;(|>fKPvq*Dx6<#>VxRe4&RZ-LypyZcmx}nZtJIdTv)K4D?kmoM7xEbF<4`-c#V(*5{pC2@|W@#eFS0Xjtf+ZJGN7chk>1u5YCD~~bE0IUCkjK=+Ozm>dq~mPmY1@@a zPa(}T!c1?CSJa4Cr0zV|yyS|_V;{2U9)R}=@yh{BjXOP)S zfU4&at?|-s8}(~3vixjRX;iWM!1JdDwi3*hl*6k+*avh;HukXWywqPiC;?DnhFgg5 zHJ)h-tojczaYB*ksTA>(YGY5@Day0z_N@RD;3U7))Q49&vm1?wPKkt{!h(F}IJq$* z>lDp|O04*SGrn#RrHn|Qx;+vbst-R!sZ-UeTnmP%X3(~@stv- zUWmSsIB1Ub-!DDUSgI1ffKUxYQ=LVhd-QM!nr}j?i?`PNC zq{$nqJfgYKCHG!5rTeM)pk#f{1L3ZjDk zlDLa~Me8^n6l>3Q)~RJCCnY6KjjeGqlE!i!p6*eW^&4-5CUoRm2HtTWadc{n@!o#!0{QB^5_stN&2HhGxnQor$lkU@d?cJuH1Iz1QolPg{O5=jPv7*;v8tmFM&QUVN_qXAA$!;gGdJMEz_?8~JAh-Pc}5 zPFM@Z%mDf;w4~@CMR;0>CYWG+P3mYXYemJAoax^VwYbzuVF=s#h|Wf;?^Yy*|x z^V6nN$>Oe&Slt-ZG~@H@xjqp(<}jU7JhEgd|(=PJ2)#t>r#dx*W0C8l7y{tqgd zknS%I#EU=u2*|O?XE2bU8-kd^Q4lN{IA+`y1uamVpz#LY$&oE0WZ()egd>Q0O(|Q1 z*wPddXvOZ!=;+vwcmuZn=2Rm`D@7W;!Ts}J&^q5?MSgxW>i_Kc`ZPT>DWBgB?tl0x zY0us$he$)+5oXnN3_Fk@Qub!aNZ2^n=P$xA@TB%|^1lUh<<=HXk~%yJ)zoQ*3gqTq zyQUiCVj5VbaN@tfiIdt1S2xI2vvx{evfBYK3>3LZDuZJ*%y;6lS2jEaSS~v*$6veJ zV+@|Q=BB=d?g&gUs`}^*09B~7)UPAMvrC>z!!cCfWCy0y-h>Aj)ZWU#<(5|i&~0sm zXyBU#`{;yd1*t=Sbc7EuMvw@BHrPcXu;Ml z_Zc%p@MxZyF!k7IzX4Z_1Ia#(^t48 zhu8XWTSG&vZ(p+Kfmg)*Wjjkxc_4vpd^ilu*5CU22aq4(3N}wTo0U6grmSxffo~kR z;EL1Jtxk$Pb(2vd68BDg#>daLrkt*-m=}vHC+}4z2lWN^6s|MVbbQ9k$zml|ha}Bm zZ+=c3jaA8w@CS3o!Ug61+EhEx5(-+z!?-B@6%}JGO(N$+K&{u%{1}+xVSCW+kMve7({@6F#!$bIG;(oV!Sk!sKUsG! zd>^ocSy)B#DylB48gANEs6^%a(7ge{etEL&qy}%Eyp!zd3yY!4VzEgxd-F_iWyib4 z$o_IK+zS0_N&~fif|Cq)fu6m1Kl)Nt-t?&P=Ep(-)PvS=@ag~CF`BY)u3@fsd9^LL z%dEjdxYA$QUa=6zZDYf|-xp&0Jk}Tser^Eiwfkrcr zefrI1C@hZn586&m+?o2D8CE-%+oJZ&=-*#?=ul~~nQx69s8Wz8Atee7OV50C_FbJr zNM?yqfW#9WeQl~gBi$xK@SX33{imfBWXrul4t$IaGzEk$Tl8F@Wjkk0n8)@urX~pn z4zORw}lHi@hy0+`$0fCQd?t zT2n&Q5qP+i==ik>>uq!>?26VV{24FY9ywNx!n92q7d<@ZBEM2YrIQ;PKzQT=A8D!| z7@rifX(`=*X9@~%5%vGFfYbrjn%vRevnr^8e*bIPEbP~C`+y>A^7x(TT&*H)mTX}p z>Ht?hAG+M#z$+X{A}U{Pmt_TzJ@$-cml#VS2s!McOS3_sET=vGX}WM0T8h0}u|$&^ zl0sE(1T9S!h0%kIJO%3(&Gqtx9BsK5h!vqAH1zAffkX|^8a)i*Yd!8lI89U9?OJXa zSnV6(_N}`-V^hw@0jvQ*O>6Z^oTLvvsZsJ)Hk4N3EVh~Oo0yRj6+5b@sU4|EZ9ypc zZYNY(QC2~gwDN*=#U&eHW-FzP&zyP2(ji{DqSi9)$0gJNvVyVehT35!J@0Qe;6;fR zZc8GA6b6ksQHoc_obsurYcYGK*ruLjzxQ4(Y|=gQ(x%p^Y00;|JO*hQdk}pbQI$cl zv9-&&RoC7~$&p?gL<`{dG2s(+;yrZ;es6cJq*RC3^ zAd7FG3jEV6reF~3U5|YF)ZAwyjhD9I(&81@opP((?kdXmKz^sW$S6mdt9{`1*7xLrfYmej74&UK) z^Ow3*ASBa7<8G@yiOahQRht&*Loys>dUX2~WQ*ThXQ;4aAGX%Wd z+b7p4)Ty zoe>eu+-7>^GON`t#ol%c2a6m8hMajz@Yyfoy%a|!SH3*FQkIuf=u9<^kQfApmUQ3IJ@|nRAR;5R2dBC z($eb)iY_t95*pU|id|j##{!AL{%4TPc$mp){tzah>mc@|QlH|F9? zs0%i(q#U`yHplktyd@DSA0|UclAkrVE$)s;(l09ml}U?z|FcKX;;s#)KB*@yQBLa$ zywTDI-hHkI6>U<2-@d8bm*vj|w;!}qaq=KNo+xRr^YZ9a;h195Z7pyV&%l+-ytm872` z-}6)vB^0qN!Z}d^EwX)9P(@Nwvw44|I~A{cuU85Fv=oC4S()tgW^DYbpX%~vr&!7E z?(&6*9o*BGm64PgmL8VUjE_yJ!4*LO?Lit*Ml)8yxByjBb(};UD-o1*(aZ}FknM+C z;KWj2nen%?BDuy#6`9CLf~ByqB1b7h|6mVdy(G1*I8MO|m}AP1qsBB86{V-EsWp#D z=sv2iHp^t0XsWH^7usno$;vI*&5_R6(77h)kon2ZjN}}_#PDdZ?2GNuQJC|GmO4l9 zfanv0wtzJ@t`m7*uAgPD+M^fuqiM>{ZnQV%d6lG~Ml0BO{Yy{b`gphDjZijf>brWw z(w1N|!sIWiN7s@kHQF*4Y8AjFyd=I#<1PQCM%QDb?bFE*xfd_&#ryGmW|W&Wfw)qG zz}aUwQ}Z+?UOAqr7dOhdaDER39=u;zd0y;2U;%^{=`iKM%@p7ewU4o1MSu2B)~HL8 z9-Pi%7i6>lg9l*QW)l&_Z^lgBS zl|}`o9t+OAqKmu_>1Z(G$*x&sb>|)3`!7u^J|*Q15@;%9)kid_OI7(b5XTl79!7W2 z_JDY5*YI%edoe%bC$Cl66Q-z-=auDY4xM31>?HMV#^rumhfVs$ih!m;`MSop?5gcEoKR+GHDcS$oZ2l$oA9kCZGz|6s z>}x52ne#uhp8q}SneeZCB52FM$>vlzB0w^CwpYJ_RN2z1yeBsH1t~$8p<;as9%a2{ zBY1A(Auez$M0JM>8oflZFa z0(;Z`k=y%{rMT>Z=SIYb#hLeU5p(&{Cv!0aex3Pu+oBG3|7!M6+$gI-WhRrUK35hB z9#cWvN2{r3jIW;@D+o8 z?E~lPG|}!4>E|vhzrr%8H<5I^e2nI5aebY8FIZ#bt?n5}{eG2A=P|t;Sdzir*bFE5 z$`(gC;Y#|77`?!FQis@@!+OTt&@7u^zITUu74GC~IU&tBjC ztO$UjrwGy~Tzv1XKvjiHS{YGhsOw)Ju{`xG*`I2enLIIXx0l(#uc3P!SPo+z8iPa3 z{BhT%kyAiZANoBQEQ8S}>A@xDuDI*U$V={>Ax8>S()U=$N=HJhyr5#NMcZLtXKbS{j{$?2jRRl%u@Hi+U&|BmlrqK+(6|_K``WOhV57F=IUzQHK z#p%x-{9Wo?I=GH(d(Cl1o9vyVbu%z5%lbYTOf0|#)g2bln=>iDm8&%tm7*1E>!Qy9 zxvM9COg={+7wL1$hfJg!)PsnAGZn4a#98={R2g8#QY?HpNuFd09?3W%VU{ENO z0{Rrkasp+p3~EN*k~0@u=^Pdf)6Wx=rM5Cx{sgA29_7%{NormuT01A#=C0c&=V)yZ zN+_#K32RhV4Qlbm;ZDYGXpNa`KVahl%UvQAIhNGVhIIL*2JCfE1x~wVE zv+C{*@s>6>QAZqCY4endqUgsd-w9f#JADG3+yyjz#NN?)@-q_Qagxg^D(ccqxNe#O zatzUY(g^+!cIB}WAfG?N(OE>mEWSJW`)LynL5Dy1)W?OAvjx=~s?w@%>wSY`5l8Qt zC&;w%{KfCkZCCK-x?a^XuYGDkq;FA^Ui!V>9s_?8?xK@R&)_6pBsxg40MtjNdDUQ<+KNQbH+<6z|nG@9~Y1E*J0g-?y_DSS(@G- zgVe`b_M>qM)U5NJZ8-U~#58+FDikE}4c*baR|(Ut3Y+4gut$a-I88Da;z-m)Jq9!e zHh&znKLNk>+;Hy&y-B13ouBbj9zg^-GC`X?W*Bg@G2Rqm>!Y^YKVWsCPp?&X5Sx2m?Zf16e$z_IupoR_nEgLziukl0W?6;+__EF1t-ghD*FlmhaTiq`cbZl6-X14Vi z#RbJV#VNul87)Ry^SIvwwDFqp2hQ*?!@yHYV>JmZ4uI`;c(l&!`hY7Q94aRCg7#x5 zfy>;^MeYg?E)QZK(Yo(-TpgO)gBhg{zoY2V2Vp3as_fR` zP%i@Mq6=M^U0IFD4)j)&sn9XcrenXJG~!lPpnzH-1BfIfq;9tDWZw~2dYm_n2B=io zX(lO9tHd0Pb;w<(P403#l z73SjP6pYEWyabS1vEyLwIh#odbowjBbjv>AFjV7lR2pz|wcmLFAk1GDN-Z%qOdHk1 zrHUwHx4G@C3u@r$YNhtnIEs0XiExxNbf^cyBj^+v0_`i~7U0@(OM&+S|Kw>e$1l(D zKTn6EpIhtyzpViOb8I-)LD4`R-1=%tC|hp1pst%Ej~0*$`hsLrDX3&sTVklwXic&6 zL!5%eDBkdVb9KXBz5VjFs~PnrHzxa4;{KRx1d*f_YnI>7^Kv|Mi|cXSdrI%%68P~3 zBS`U4(&s|0uW=0uZ&vfk*%bkz5tKe@4p30@t6issPpSFj?v8os25x>Ss}cBVy^bsh zmq*K^sAQx{@p@ZH*pM#kW~ z>Y1G&o)>mbZbD%M*(3M-Pn+7nKkUgH*zntlASF%g!KY(;zcvUgxR7fg<5)iSq_Cw2 z`jjPSNC61LDbY-~Z|v;_8h_^Mv#!92pwVYj!>0Q?5fqWScttK>&s(sA$L^gn(bWru z?)8EnQ_q{b1HoIgV}ZMLMN^0EH}oCb{Hh>8o6l->r6Hj38G>&X4}GReUqq-=gQN(R zc18tVW$9p)xP;!IN0t!yHx)ud*KdX-%qcDuX9K#5s?03mWU+||e5tVsE}OL|D%NZn zB{yza3TbM+uEvXXFZ}^bjYG+m+g?+g@py&fJ!Km z-egeMM7zamQ2dP31drKPc)6EtZ=EpZyF9bm@}2*aCAg0gjR`R@tu9h_Q?3)3AD9#$dj4MY3$|C{Xsf}WcftE1$Tkrv(;%uWOp3} z-`|yCnV(6_eI&n}vhFV`GV9?E%R*$z#}w{<8XcP3HlLFIF{y8AJQH-ZnJ^PLEHJZIwb%2+Z;hdVc*`bkPD0`eBu@`IkD(Hd{xUF!j}>uix6)L+)xcO z@@u3U{j*rD_titKy=$nu$5~5p8CLI*)n(tRyI%#|yT~9p>&}I1>t-PwSY1#X%;O%M zQEsb<(N;FDLMEh^`Z*mIgK8nzy{Fl`Jy+!xVwI7W2>`(YIrjoC5c-PfE=I{sdhtv! zpxK}7zjHq;K1QwkS$ZfI?}b_)4GOifS%@}=DN&pb4%jE?(J?7gox&zLjny7^^78{P zX5mOLma+O8p526xe~uC>(D7C+AhfWy-4J%F7|mv2e4Bg z0<}%*8JGC>`pMxQu%!)%qqL;Sfbgva46^N}J;MO8^@7pOTyi-WlD7ckFJ=UIBwZuN zOTtD$Wph3udQxTXZV`?eq^pJQAn&#)P4`D1-@60Jsg_XG1sh$0D4aZZor)`M$?3L+ z=bUAmRn(o?=OI4|o0teEa~FwWKMm4xh}4O}Sz?y>97~o2bFXb@WyxKw&M&RF!X|^q zQ{+J8MCQ0n$p)4>yN;GB8l*6BuPDsM6sH>tOu8hizR~bFQ8}0=jBw%n>=rrIuAo<` z?7ZeiRch76w}3(crP6Jy*;QRV2<&Hs0g@#TQ#HlG&fwIcjUy-jZCqh!`rO<^ji;rN zz2-RvW3)AJ1O#t%*K?PaG=c^$0A;F(4jQm~c-6OgySQbeqc*bmFHa1Da`}QPJwnrE zMxQ#k&cU>P6W$BX-I;LK#tCt|I=r)t*53E7s2^jioiX`Y8{E|{G6BFGC6HeX6hf!} zVQ5%(;W8G2&Ly0;U5zg~$M6aSpWF9=ai#K}hsJtMl0e{ZQ>gU@GjuuT)Q>VVC0byL z>lq(sp;GMLso9l7XbS8y?c(S($Jmn|T_6K7(RKR?57Z0E#qBgXLZyDyu=?*y@S5dA zZmrwHSXxp~{zbu>cSf@5BI;q|CnzeeZmmHYPJ=7pfR8SP3Bz_M=~)ul5i@xAZ%QEx zM-?dqC_ace5uFamlfZM97AN#E(o&MeS+q&;hO^hfX>r0&aNRM)2x&&lvwF6wU z^@#CN_P)oPx!d4w0f&G86@KpPuP?!SdYfF1=;RX}R1mFX3?-I> zGS&U9TwaJ0AUoiOHwD!MIh$mfuyycjQ>`xoW^joV#b5b$3adwP<9bl&%z zt!hEc1X9TLuj64$mkr{6v?e7?sT$q4%AJsP^JJ76;IW$qQ}1Ez8mU=;aK$_#o61sa z%zcXAxmKVaSrjtgvqC)B*xxen)7{!v!Tn$1Svqe>;9C1kwD#fyR-zR%iad&|!@Q#+ z_;Z+&U&OFr?^3ekd=YU1n2hFbZqx)V7!w6C3eaPZ30+q~s}X)EAHB^t#=^hon&apN za{tyskbe*-fFQA-zx{Jy5YsBr2Yu!lw*Ir>;h%G8%Jj{ruu7$trw`5m+Q(pgM{MThM zX>G3Hk)D^Jj@Q<7LX#ku3{}hJc@^3kHV<_7$L+-ZCm?yV!;?jO<8c?y{X1mDAo81H zPd)TUkxATjRMZ+`@j?fzF@N2A|? z#vNPV!TDgJ{-Grsh3+_mSVcfasXNdhB#;PfwUGxOunz}wbNGrJ#KF>$xq}CLY3hOT z58b%$N*`3tXF4u~aImL_&uk-iSbo*JM(fqPF;Bn31$NlcfIU&vAA^k&j;aG&PR-%K8|;`}eYBXJg7Bf% z2Qc{(4`g>nAH4yUC&a!$>a0NxRo*}*w5i+A%1Ltg-T5N%3gS9VyPm)@O_2x~0btkE)pxaufk)L^_{S3B-`~EG>Z>Ni8-E{E zL+W)cgZzZCeGLJZeiI3^9jrE>XPhcKSdA8R&G@}5==v&Tr&+FGIr8x`*%j#3zsy|U zJ|;Po@IHT@e<3R>V|gE^N-Q+*L9GSM$~v26D4s#9n`tstC78)aH%Uq~IRw2~!COgT6Z*+~eWNM| zoCb8GEj+w)o!xhPK0RQO9MhLgpZS{ZuwkdcYTsPtLW7Xp{S&_-Ch0f~eo!fhu z$JcO`7x@CZL2feTb`gB_;I5|i(`t|43};WqFO9sI(JV-X)k|o~7`?4G6{={|^s?4B z)}$q5)Y4@x!rM&^8W)LVOxfN%*zzK=QP3yJ+OlvEQR1XU(Xy=R`)ROS(c>rcc&x&& z45gCM!;hyDq&Kd?_IKE`1ibmTPPU_wi>_&Zv7036J|}}p^{df7Oq}5|?FT~aX`Cpv z;<%&odeXC$!6Z5Vj*JOtIpDXYN~LAFI1{!QD>hKwzDvRG4YvQq>I`!q$=fw0P`^^Y zVO1d+C_B#@KmM>#oat(qJjayuSs*qKi|mMGm~*rvLV3*4=(X}Rui|mA&+}}@NWMXk zCjB$>VGuDQY zgLPy-c5j@xvYvu@ofWV%0vX$`j30Rl6YJ$2QHCt<+-p3C|0X9{GPYH9rV)}#l{v@k zM}?d#{E>EI@~&a14U|kI^3)kFF06;nL21jN2sp8-?gj7OMvXLuAC;L>{CvTXQ>JR3;cnQ#*G3)M>hB9iHU5Rw}>(AqQ*AI}UdHbyRpecqm)ez(wFm%@dhM%2xIW{7hP{+ht^troBB+ z0#$MCb7V3{1Lr^@g_jWO4>cRtK@%mzbuHwW@z2jlg*k8(-Z;nMluIgf$E| zM7ffFistVLG|bAs>;$L(Y?C4VksH6jq}h0E6$cyH^F$MYugwpHRF*AqF52 zOb(L!H(8fu<2$p?Xs92EE3Mu71!cNxT(Pl*lg)P6pZypO&3!B_J&+r!T@uY3-hIoB zlXcZb>yT%VjrhE1AROuVILEp_U-4UCXG!0|H0nq!7wB_u>d;|7mr~bXI}FxTJDM12 z(efT#J0~2yCbi$Z_HIebbn_`N*EKNMBOEWJoR~FvKMn(#eiK~h?n@`kc7S0PgBR;B zAd55q&Kw62>rX=>WI|2<^zjH^-}12S!GUy5M<|ko(+#11Htw3kA(@y|jj_ThLv2gl zPFLn`7f8g(*E*3nyo7!GXm(+_JnpJQ+b%N}s5w`c=}I%$`vd8r(B(dSu z&devmaHm(GjB1?Pc%fLgOreC1QTeT^^c4W!f^k*W*vYx%mgA;c}N77(+ggfUGk3u6WBi}>lxa*rKOaR~4 zUu8<&c~}f469N8(2i5K%+&Yw%yLQ3LxYem*#hEps;(f!#PjoA0c<0r#1m9q^@1>cs{t4 zEHehoRI;|z03OfceVE*0=VlEGm)xATr-1pf?l-k=5s!Sk7bOdvA zr+I-tE_#8H!S<_?;bh#8IxC~#YZup~0Rqte=|J|+Kht4p-`g?8k>0(-q4=Lt>Hjxh zl|^)cM4AOgM*6RcL)%mHl!~$Rd|5pSb7KG&L5!(PVtQmXNIvc_Zt4dFB@bqC$6D#L+Au0> zEMlHL$L)707|z?-`4%>34&X%?7YmD4a+u?HWBW!PGj1dTltXF_JHi5!$cXB2sdqKbTZhi$_56t zWOuYW>P^3#LLR|CAjZo**h)oDqx;rAAam96|~`hXLFb8(CNe zL(S2a0vFa-OcOog6`+w96pBhdGs;CEn!=i5qYqLcA2>^2K~woe;9rdC@g}PAO8wBf zUYGr2KwQ@B;=^xb5MS7^0ER62Hq%^vL$NpuRP=6>y=3OjsuX!Ooi8Y^TKC)rLq!ff z^VfNiFx4qa)tCq0DoolQlaT(E%n|-f&F+v`-Ud=J`X3&95){He^&*_Cm-k5MK08YmM@acVZIO#CHbRd-52$>QBs*JHsDKPUN^f?MyHb{JGnm=o){X{Z)%Z<8}i{64|nP zd(j&r9CuM7cVdHV0k_Wttw1+zSKXnA16my>TFhMI4KOF@i?dKtI82PPQsDYj%-P~t ztqUH1R=6Np)CqnWwVl=RoSC06twf%;f|ItAt^-~UTkaYi0oSkR$pD8?TK@(4*<4>P zdmP&uivr=M;>J;#5VL5*lEYZv3#98Bj69kpTF2o@*Tq5>H~Ue~Dd51BMJw@L=|MdK zv5yh{$_kfV22*@%3vb$qms)nc-3Rv}-nf-aU5`rvH5Ug$G=Mjt1UUqkChz)ed^!{K z=A4${a-{S=?bFNnVE?4~B+rfSRdxJzc9eQO?Z}V%j50q-K-Ek6lAL0f*2!7ufk9bT zeHFS95Naae5~Oe(2b?%SB3-ow@S#&9Cv&T083jL7oHi)XUSXa0%iJVY8zS*3sYZ1!PV;{(vN)ViV`IUoqJJlv5_U1 zf2dRP=Wct$+Y@?3*osl;8S480j|9%{agXm>z}nS{X_=%89#o@U@4xqu#vgZG*ZD(n8Rs+i6PiVOlz=PE|EI{k_uONF`_H~f zXGmbk6l~3LG8~7Pia3T{j$Mvpr~|GIm50?ptb7GlOUID~yvK~Das$E;e^)2m`uTLo7su34^9p*O(Re60_VcN>D@v9o@}4iwuv{&^ zgyiUSbhQucd^P^b5weuAR1%Dd>>T~E&vNU8f2E9?Cfk4 z+#E^ZAGJHp!Nr!i*Y;Ty1ULR`8Ffh&eggR;uG|tD&Pvnd5+^GxrqNJd=jf4QOWO(y zH9p+%Z>IeP%sA3wd5KMMYzSTp*rKGul236*x8vq4v0oZ*mCQysclL_0i|;KmpP7w9 zUx89bo(Kz^PfeVRLJcY4E7hgUcE&HlQF#+Us}Y17UDks${2zZ1wSDF1&m*7^P?EeZ zdzW|+?viXBqn0N>A46Ztdqmo)`)SAH=^RJ!%#zY*_Lw>jqsA7O)78(GM+#EIRL+!U zU_Bk)JRkV5Tc$4GzE=8iIcr#&L7aO!@VadWmLLCk4 z{BgKv&QWX4fMqi1;nWviYv#3Wn8!;1j3xD}3b;k}Qyu(;Hp}<`Xg-Q((aK%WfI)a| z%`MfLm%3Iu#WN9nqN*K`SrXBlUF1HE@z-sM>wc)b+~rL8#{2T#k3&qFNpz^VF73CC z1QB8hzWjVXQE}d~gBo5!q!)-gYg>k$a6PgcpfKr0(3l`9gCiBc)8*BFu@ zQl6}!UULr2BRkL1hb-(v!Fc1v?>^nnP|z3nm;FiNMZ_nYc}GQH>a1)u zcD6$BNP#}TK|nH^t9xID=DkmmcNqCrPjv<7Z{NJWBfYi~Gqlw%PH9C`T>sqx%$GH~ zbpYKkh2KhDh#27pU>227DKYbl#V}hwzB1M8tjit(bW1yJx75~ekkNk06IfCJg%xAt zK6?GRyw8m8+xcV+$tcjM)UuGwcUTgn&V%J76jp6lg$*p?G+)Jlo83{m8yz3 zh`995x#C9^(TQBFdx(lz4&Cy}Pbwe+*;jIt7jY7dG1vUxpMi2YYFGKQ&_{q#f}FiV z|9O!Ufq8+G?Y!5cc{1P!P(sAJZB34HGf98~K!219=T~jEHbqsM%%1ek9hf|q{c18e z^(#-Wg`HgW*>2S-qnlRMUGf52oi6Hlvihth z%DpB<8_IEvRggfyuGz?@@{>hjc|!h)&REotb0c2CR(uKR>G}`IU*&wqC4kik~y5IjhRe zkR;SrZ2ptJzRaLumU|)(%S}4~KrMr;64`#4U<_8Eh*+umoUn9+;via661rnNNms*CEY*(d)kOJ8Dn@PhqrcOftm~d>mtIw4FyRp~q6L`N}X^J$}hD)2>F%&yi!kCmM!> zoexRF)medx@k})Vt#N|&%xskIg-#{FDD_QIMp+b7al=R;ZGEvacd?#fMRWXtCz^W% zWio8)RTjKcsmL3&M29EB(;rM82$+>P7c&~D2D~=_KYz&W7x^~CIs)K3a^IM(z?~9Z zCxzdE0;n1O1v9FjfP`d=f(9944HnN`CoP^z^c&Y-BVnAN` zcpkXzDnKvHXVx@yvdRzZyQ!hYWv5rgVpUev)MD|YbV~(nC5~pjH?Xb7G=p7y^kTb+ z|9~0mGiHT;diw)8*#^sPy2ch(-x~|(*`^duM$_ufu2_*4*Ayx^nn|iv)5`VTTl(&R zRO#jIof@Uz@Mj5j+#M+&ytXQZ$iUT?xrkY5+wytjd13jnFDT<)H5;Pcio%+tCiESH zhvmE8rN7~9$vWll-u?;IVKdM%3jTLEE6zp6&61`%sH(r$}P$*h10L<<9!K` zIB{RACGyCE3=e=MVNrRA)E-RW{aVSMRh$Pcc+c_E2;zoLn{jxDgIkXPp_QIb&tA<`pj`*=aXt1D}{*}DDGMvJztW>>o{$r)_fy5da3ct1p-6^!f(B6 z&*o*P&wRIdnK&UY0iXm8dVl?&0;($4LpA3H|RbfY3iy0Fm-9+m)bpS?ts)K=g0Sjol!i` z)k(kj-`(4MF?YvEmWH}4jaHMTBQ7oAv)C?W6YMn8?`clMDJNA=`u?1&0O5C_ zR4+21H;ZwaD%$U-c@z5V+ktRH_P+i>8y&|{EyW9``Ae!Qek|M*LW*OCW)Oshj=*rdIB%LI+^25T+KDmn8j(ZVU-1h$K`cB#&Z23ni+=(mF^3D-p$^8}gQ)OOf`% zPMG(R&REhn@@xc&UP?XR-8m)bD-J6M*(YPe-y>W4oe-v`@`E#sNAvz6**i}O^BG=^ zAgY&T`}~U_Iq5!yPNdD`%s%={Rj8P(wErnETJbj0{g3zwQ+v`bDdBv&62FVW3>>Qi zgsbgUXVXn)8(_r$RO=A&$nvQc>QYq>o0P{4iZXc!YrQWhk9_;5hvK}lAX)~2H@5#| zYalFShJTO^DC6L4`d_`(f9p}B=1GXRHm?pSu1L=e;9X?YvS|Y0#?n?e7XL%s6tY;? zJkrST#A@dF&!v2?LIf$7Ym_g-=1#-N*n(bg&g0AedTb6-NY(sQT$}E{x=vn#P7WXh z?s`5+4AupENkaIuneJ$IY-=nqJ{F0QwYIaI^0OIb_`7u}cvCX80%79C;@!p%Vuc_Z z?}142b?$YWqrG=%aK5K2#Sj!^B@y8-r3yU|ER>KOcgJ6LC6p(zc9De40Mwbo3=7Jn zEM5th3L1Y+6P&woOZ{jp-5}oB^C=^4>pEAwle4ZqFy%D+YoQ#sh%+;gkYRdazG4h= zCAOZsQLBJOz$(b;>s=Rfv2-b050te;e3iD(2vcy-x7EFnqpTcC}X8?XM zUr8@D`i6v9d)U;hJ&?P(z;1{Gw?f0B`zk>BCuhhiuw+j(*{O^ zBlJgzww&gx~x;>n>SIJ0e9SvfJ= z0dFode)C(r8u-QTLGuCScj;0d7oyA@^MdB&8Z2gz?PLU{DR3g7>Bq#E#}AJ%hi%QK zDeb2AIyUpEk6nv6`zD=&ni5;qLhcM7`YAPz_dOXCs5ET-oqE3>QDheuAhXI()91vw zEjF=EmO83sAZ(?KD!_9y7>rbj<+qEU4DiV9g6iv&#jnMT+JMbPZt;>5bJ}KK)|oPt zLYe1sd>3I9y?{r@8u~{s@qOH;J?*Cz#9lfD*zDHI26S0sLd&O3onhcHSdUDfh8wRIwuQYXaA^|p~q8^Z?>XjXe!~VtK zE`&p=YjTP?aP1AGrai21_H;KGZw<<^Ss_;2zB_O^mJJj=t+i^r$Yn?ZDCy{nBB%P% zr-ecz?uBmBMe#uKlS+xGOzR$?&vC=C`f6muyfd;F3(w2VOf}+eY5PQ761ARDag?xQ zewpO*gT^AdL5;0nBI^NH{)!}VF;~D~x1m8q5rTA8>RSxCVy7qNkwEfZe3{-KJ!XdT zN5F49KlY}s_jds*e1UEm>V`F$WijmJ{ZX_*UrDd11V>dgGu?frhI=KEEkZP3NG16| zSOam%8{IIk17WnZ#kLQ$nwA8y&MpK&JDc4yB*80Dnb;We$YbTEZEBqyL+U*|K z?L_j~Xpr>NMP34gaqA0mR^!r4Z5Y`<;hOXZIqe!=v$h67cw7e~*`Dh^P$zt>sA;FR zwjb7O%eRGB+)elNI&G0^DEmI6X=mqrN1#@D*^eqcXnK~4@|VNG6u2&7{z_cVW%Wk~*f>SOWsNHfb8h0i z?%82mmBC8NbDh{n2@AvXnb!9BP|4O@PY?;2UiA{-Ng_laoU5>7q@?TtZ*$`{IXNYt zb}4xSpn9gZJQNu~6vYY(>pMNPO6X1DiR0F4_9_fF(RV8o$LG|c7jWBti4R~rvYKp^ zGw|=GYqg%-v~x`W?mG(;FqIj%7kq( zpl%2UuBb1NslL+W_RD_5{3zAmnZy%S9rvs2-7vRyOBa3we34gHxK2#|-y!AIfip*l97>oOjSfgGZu9XApvDF|)sJsniF>kVvEC*`9yX>!?3I zyh||DGY(%WWaa8rVfaasRaVGOo>?3tYlBVD8Uq~H%=GseA2uw5w+NB`0UIiG8Y?0T zax}_+8;}%%E4Do~sGefQ-kD&1=}%etrQGI%#j&R<>dTk)fu3zafH6Q88{pcdI^P(c zjmM*u>e=t-D%rLfqV7nt80{Cy&;YU6lu~PZ!XoiwvZqRlU<)GpGzxxr=Y&77 z^8o)^Y%q}5cyO<ANyJSE-W#{(~8 z9#t(MK75vX*K^$Qm$n6(`Wz#09t&$POQTi)OO+NGR=b5gpCzJF&_~J(ozwDw>HcEb z(%cbtCIrjK{}am~*@4u5C7?#l5EVMirv-xuK%_uOXjQ<`XKoG1efNK*2Jne<)o4$< z5MHJL0Wvt>D=0xqlK)N(oPu#xU1hBJPYT`V9}61%oquLm1kkO^2G)J5NH#Wln+dLt9!FLT+w$ z+m(e#YmabOz44pqGarmT!TK_(aTLP^j^GmyFP;kFs2jG zF?|hQG02SBD^hj?Yc1WB;40lwFWh|n>_I`)dd7fr`-FeY54hcmTfSsAkke??n+v|$ zy$tt!q<(b;19CDGb7Ij1ej5-+p!X=BbQNh+lnp2zX@yK6Jf&c1IbJG>5=g<35ws`pg}orifrA(snisBiT}SF!$@l?@(-XkH{%ZEB(YH zo#P6w*=XpBh{k$zB!Ft#0iPA-dlu;R^a5{nH;_dSNOMkZOMx&a)C7Z!dSKAl!%IVE zOfE-2pOHaGYKD!aE0qR0XYQCpz_S0N=;*cAHU!9mJm`TCB)c_pirH`2Jik$$U!OC} z56QCFG&HK3K%|Oe@#@bBKrrqsFwy&M%_Ql21r0fj}hiYl!rpydIXWZ&J8I>&tF ztBE$JctzIND)k+PJh9WP@@OCtq5yL07j)-D7rG2SFSXeM%M#bp;=GHk63h z&}O&-LA8)>=~^__r*pE!{^(~VnQHWm`Xi}1QR||F-n(;_`hogYrk6===A<<~~C!3BXI0O>W#lG1nR#C$v zsU$RG!#$~LR8f!V=-o(U*!P@~E+5BW#w?Yj(m28lIIUbY#$>ZoUGe%jF*{kzNo(Vi z7U&L0x3f|Xi9UDIe8rl|tB{Poga_+X(%I|qb21gv5XU&5SsHQOImqOhY?z(xgYT{og?=;H^%GsJ1NC$N{i}B zo8@y*i_qF^&Md8Pd^#T;a*dMX>KK7?Co^vip9Tj)^&G6y=SD1xfjO{wg$ zWnsDd<{o(Ru38j`BK<5VJLX{Ems3MJQ!L#pE_tk2u3ezd3qG}*M-Q1iRh>eYkGvFA zJXpvDv*j@9+45Stzyq7BtH@|>d*81awqb1vCW)`o)}7q7s+HGd&mGBk=1cr@a)qZSAMW1gV}R1EC4$pwmhwn{)PQEjwOqHW5-RlL~BA=B8|fcWIa6 zuQf`y&DNp0?n>3H6f?WNA_}C%O7r6~X;LngTVOW72y#IS{`uSwU2lBq2d}!bOZb|1 z*IaROlcOy8x@*NA!lkg*GOcv23;c91K&PwP9_NlzRYwV|>AjZ=&N@LZ^(j)XOH-A-IX^e%QvO*Dqf zlFlf02&z(F1~+7+4W|R$pI9Tf7=pZ_ap#y#s^w*~2nEE&*ThO7nl#w5^ax5nS&~jW zI+Op2WaeY+R~GI&8sVN(wWQr9MZAigRhxY%oAW=oshw~OJl${|_P5vG)(V~Pac~Rh zoQo<5_L)4A5bDiiDc8GHXRFoBl$Xin>)m+}s%o?uRqbp>C`*Bfrb9LQS*fvDJv}zs z^3`5Wq?m7k8En;f%6YY+T%WA>PiB{uiSpIaTSL)EPT6Ykd$-6K&kZSmV{RMGj3n-o zV3)o8xZF?b9AP*{nZa}JD#c-CH0VuHxm15`8}au5^~usU;n|QA!fjptJlFFqJ4pGa&wo<5|Ic~LBBcK-NB>o4 z`|k?KC`67DmMB1?QvH7J5QPVH5^KXDoXmeaZvS0oLTnr(<*E7=_da7GpeUBf zM#mKgQUTeQ$C1KU<%qvwER@QNJxNjJ7(2(7GM27_G2yqyzA<%`k_4^&yeO30s19G3 zGjNkPSip?TG?I&-XNgfNIRQ2b7&SuPh9+Hdu}CDg_T^h~Oyy?5e8!`aecwiNLof@% zBv@jXTb*?*r2(Xml*)Unp5F<3OkGK_@MLYD{ z#I(MBcubEaf|Nh(kAQn>i#Zg{ND+Xkfq=f9n7{20Oky83>f4OGWm-k*)Z-zaL<4}S ze$sm-cqyh+CbKC{TIp?W8*c0^Oy4MI$Jh@?8o$^KB$oz0Y7KQ^p(si|0}0QE!CaY{ z(=onL2cpJaj8{$g3oRB`I7%mJDwa4c?#Jilw&n{GNf}9`Kq)kZ2aHl91Tn?-{N@S; zA-^Q@lBia0kwDT^Ysrq+np0WGWVt6D1 zG&i%7$nCHs2$}0{Q>jRI`MUbI{NWG3I0){4Mzqa;4843~g=XH7j-v|5$hZLC7#kN{ zR-C?l@bwJ*b#F8LUO-vRVwaT>jFNlBl5s;4xUkvgzA|W_#w%BmHCU#)6EjP;oQ3%o z>LP&L^^H%qEA9HTLG{sGGiI)rV@qG2=KffaP%Q)S16ZW(2yS^_mV1xRZW^AolLL?8 zJr!n?(Zec;u;;zChwBp%?puCv``~@a#6_}5;04(4iBOE6d&mTRxIuG%-iw7N`B%wN z^Wj<T$71fEr|6{*Nk`&-TgIPsO|XFA+@_FG8dvuXJ=(8J2A{7 zegcQaO5}fjR|*ZtIX??FKtv=N)NnfOu)#;1)G2Lh(tKuxw#T;}2ovuhK3TswO1h%& z$Yx__O7huPX3MAQpMdLJt5-kw@7?H8PX6tvsZP|>cej0@$5Uc~w%W<{ukyzZ*2_KL zg0}D;rpU$buVe|1WoAe`Q3mbi!$Qh3$W<9Z_MKpukTrdbtx{Y6)0Pp%$3eI$0x)V* z-^XQ9nhVRG>YoFUQZ5|);@+)Y@&9hJ{ZncyU`9d$YAb3G|2@rA6;(*!=aVMqBbWS@ zJ*kyi0T~7pUZo%;Jc^<9OEW<|s5~d;m{5nrOWyzkUvGRU%xVZfs`_0(UZ-PJuOK#Q6Kx56F=Fm zU=gpDK%E$76ibY{xwp5ZH(lD6Gkc0eN#(#*m-KzDBSAnh*>ZJ~IV%p97X%ku$;0Nr z`9)+3%!9B%b!#qLo!>1(871{`V=T7uhS-TR4VUhV`5C-Q)F+4}ZJN;VcLgGQzqUJm z@r7x(+PT)(aT_g{FA6^&zn|}U#;h#_3HG098qUulAz&7CU6vsA^z^aZQ$JBb3xQEGWmBrn&Cbgr@WtaJm(4>rhWn2}98JEES`_ujA4R&Pu$|@v zL*XewM2Swzbv{t`_JaDn_AdSIgR zIsnVLI^2x9FV;95rba;0l3RZuEO%|0cWB|s(Z--Woj8kvH&sls=7ociqs@9<-p@-% zM$hJMhw3Y-UB_>A1|S9Tp-MjXOEM{i5>c4{F;naQ1RlxFFLX0sJjZ%?O-%>#33p4+ z_N-nwqV#VGTVkasDkJ;=$of>Q+ap)&L=hDQFa)XfIJ*s;m4&KyRspV&7Ph>h3v{=5 zOZO2#_W#x)og;rVOQOlu(i{EJw|i~dkQOQL5ZItf)ljC_uh-F;eMzN**5S#5Zz{5xFLcV!Uhvu^C-0M?MDP{xtK9mNe{ZZN?Z{|(%4HA&jLE0Z+ zt9VFgz>5a-aGRZ^o{6eS8T&6v;v<6Q4tcU(2!DgCN7M;hv~`SUp20J ziC&Tj~cOVLWJov1kNz`MYb<@$mRA$r((vOfCYa}zvNQ<+W){MWn0r4Vf3^i z4h_calXn#RMdP<}K#pH=9PXv+ncU0+S6_s$44`^ty4n>C>B2c$AnDfr+x|$u{M-Jp zS3UH`NZ+AyY|vP)_}3m_p&{N`5d_av{*$FaHWk>j=An>avwpEdhE1K9*s&u`I*hm! zWIC?enw)ie7AcrrH*7Jb%75|&eUA5Qunw48x5gMjI0k(FZ3g>MAXvn4?~d# z+SA5QdPSRVE}yd+-W8gT4_8L9prMHfUpE?s1e-`e@fFMnbRCtSo0HxLe)%*gi!!ZD z`cQ2y|H)Fi@8H z-aa9~Py~)|F5tl%OpVOH%e`2Z`~~%}BtewFo(41@%^mUDU?EE|$(kDC&xD5Q*ll4T zpqQ%_vLkgAmp-mJH?O|ER6_a#iCMHm+of+4)SDZd~D!^pBa0~P4t+ktQCubQ^WMCF<6(3Vbg^I%z*EhZaO0Y zD?EiP?o2LNMftX$L|}tynAyGPMpp>jbi2BQh8%eP`7HVP)l&Q$x_ShM5|}#^H#ERg zX&8)4?5iaA&4xZ?T)J->NmIUN=*pgF^BLkiUO!(Aj?Imt?MKxNmnkrT* zG@1_rnA=fS-_YyAEACqD0s49^Pnx4NIJuCYnLLh;R(UA6ga!sJ*8=EH3)66m-5O~VDlC>e99?Y0Yh;RC zEEbQyK7>Cc@WGi`bT&D0l5-hKw`VV0<8Ce> zkOefN2;mUG!k<3_l-cZ1`)zvW5*Ku-`+i4|n^em29R%ew5Sq0&=i;mc5@eb_kUdi+ znE1!3Tb<8r7Qp57s6~Q$^_txFjn&S%=Qf1zEh4iFjgmI~xh~~Rl44KV*{7&yL6n8s zoyPSfh}b?|V6MIu^ZSw1C!?ig)_BVvRiGjO`Rd05bM5d#N(vD}A=Ya3yG(jL+MT)Q`xg7s%rqT(f1>}d_H{&twY_Q#$W8^-CIwTm)_^3#3)(=K+8$qQxU z!LK#2{&fud|(lKueKRS5X@5Ac8t zfowz%@0WY(duV?!>kmpz6(FTx2x`_(S|-`P^|uv-rVLCVlVj!WT)4 zUPb~}n!t6`(7RKp5}o{NOd~4!2nWF{+^G-HQzNti+cAYG$kf{smU-|r+u~~y!(VuB zp=+BFMth)&nvXJuNM$=?&fjBdyQ<2)L6s__+C4Dg3rP-p6X<0OLjFf!T#ah18d9Bs zJf~y@B9O7fRz1^qs5ZO;8@!-@I2!LAGeLupde?<^QZ6i zT+cgl`rq8fp8F#RY{I)Z#tpi_A2@WrqcRQO4{`GP_OTm4xg`YoRyHB-2E`c03(%za zwjlr))Io{R8~8c8&fs12LDNtqghX`Vgk7^xj!;GTLS+|*K`w+sKrz+`JSI#mxjehF z3&G$K46Q`$S4uf7Vwwqb3j|t(O89zmei@hiU4>9LjZVR+a}b9$;Y%52Y2{}+VK6od?(p9?Vbu)76A>i zy!V0=M72lp&>J4LCj=vw2}XP950~9V!hA#MP<%jp(H<%TGaw`lu2KstgJCwQ_6A&{ z!s%)bU^W>df=v__-W37KI~8MP9-6U9J3q$Kb|&sxG33^C5cJanknY-t+SKLL4WNtC z%@_!Argx1)e_;lw#CrL(V3U9y88U}Is(%TFE7z6R)*h@=&qppwF+&MZj%5McGIW#E z0r&y{v7`^R*h0IC>OZKjl?fNoRuwUVP2@4c8x%1@$uM7}W7&b5lz?=>r@cIN>W>7)qCb?qxZiGT08JQZGz^{j+%3bpROK;HV{q`c@ZFNJkc1hJ zY)UYaCvl$_+j_sd=2>=0Z@68xwVhhBht{s_JFv1DwWf^6-!>h^@KbNFl!_FjU z^VC91d)O=CDiHX6p(;IX!Mu{NZ6DEAEs?jL`3T=N7qyM674@^``f^LTFJhH#xq6x& z<&$hYGK+@APLP zdyX6I_{2CvVyxXOPosU_VQdbj#yv@^3g_%!-}gSJufzg#aEO8NAF9*QmY8*1Nt{&D z&lutZJ}tV9qo6jYZJ&RR2lCR)Sw-K;>6%XmTDBk|r%rdUn~B;p`QlyebF4z#2k$Uh z#-nd()^Z)2@WQ*_8G_V>0<`YC-I!eTLuy|z4&1cr)4V}WGS-6oMt$p^7W1LVGs-y{ z_Upx=Ih??HO@wVlDai3|eHooTZP5Idr`uLG{+GNN7AKv)_1>on&ssL_)r&2D-Y(pF zVeJ8fM`;oJp!Fk)u#`Ns9~^?&$CR##8+;wwB31jx0<-a2lc1b(L5BwdqKfXJLILZQ z#cUnD3v=e#PB%^O>i`_ZDEUU#(dG+7Ac=z3q6e^;Rm(W*#=Xla(7Lz{T~XyI;Gk9` z0}9UqnE2}L-1+0JNIT2yZbKl;)$??|+xDkPHp+3RQB*;b!0_skRu{mym;Bq4GJd`K zYNOFbtPShLa;S6pW?#8kXD%Mq^(BhRY^+euzLa%I+~=j<;IKQD6*YOYhbSeHG=VQk zH$A}lg6q2zG4H|bI}75E)}7g2VScTwM~}=_8Ypq2m;K$iCx3c!HUe+-w-k21D?Q#4 z1*H5+<>+?5Q#A;Dph_%M{v3X>UWzMNj<8``tMVBfHekzr$;nLBWAJrxzyF7a?4b93 za)ByzW&(buAb(CQ!wGu=&8L%1f1y`z7?djDfrz)qP6X964-`w(4|%$JRD6r75aat< zKgtwuQe`4MQ_^vYEp3hQ=BX+CTiLQv*R6G+hdSTcQhRf^sU=M7*yZijaJvwFKhw>? zWq}fHMj5PG$(su`22+;oom?p!0b=)hVi~9HhC(rm%23l1bf#(F-7x?`y1S*NyIXM39n#V{=#*{-EnU*x zAuuEa6mbunbFQ`Sd+lc*Jtv-d;)?^{e6H{BZ}-vnXkcjnHOA^2BhR^o#-meP54Sl1 zTw0b)`qJ~`QiX1TP02mnWsyP~hO62OruS21L%fAIaQ+Edk6y^l@e@zl!m+UzTs*w? zgL!VAXjyzHYP>M#_!8kt%$RUwaK*UU1oknZe3@e;)4OPESCl|r#%4NBl22N88{kaA z()I9C4o+vQSWqpKWoXdUiD_xPE!d2&U_=M*mTirt+9vxv?&mF^zwASN#U7DG!j);k zOxkd2#|$^8I`LW#O)*b=g%ueov+~?tIiF&80*#IOIzdl_QqS?kF>~{lHnWu#uK^X2 z!gqKa33j5GL35xRTIy}%kI-OOpe+@}5=|xNG9?m5DlW~hZy#d9`V9cWHr~^}>Y!#2jdCJpW!*4QYf)lAB|MX%9SlIHY7 z;hP2Lv(FT{8Kd8faJB1Z+)j+{MpVZWYpsI5J68BsK8aJ6!tz?}STd22%v2OupA&zV zObUvLkZJ?=pf~Q%M^i+dMC+RZw(V6TGYDd2Lj1YNI5X~UBm95j?qKO+L+41X%ntK4 zIRq|yN)o7HH_|+FYkD}+1HrC*mFh0sDC>Ifzs=VuY%*`;XcsFWiSzot_z5vZ{I!Ji z;%lgP1I+t`!`!?p&_8?~(6c{z_m6_I1x-U=I95U5~dN4?W+#$n(k- z0-PKwwjg~=SF>V^ZHA1&w&EHlbDU0r5fVD`K(a!o!s(T63#x6|*^Za-XiG0$Y$lw| z%Uxl``ZLchAK}&dx;ZSFMkI%D>p2*uR5or7c_VIbajj?-X+DNJL?1G*p_&`o+4z!-(N*+a9 zGtZ8VO;c4gbDKCip3kQz356Pyx1)-13M6(Iwy3XiNWFD z#h#0}&rCMH9(^c`J#t?=+*|)i^;j*wfffrvEC^?C^i`n4% zqTjX~-JsJY5|k#yu|2&k(0BQS`24#@T*Y)iILeC`ij4n7H0UQ8_<)jro>u=Cn{8-v zgdoW$#a4N@iU*r>!Nd{pQ!_(?GN{znycJ`W3|kG^>~B0E@cZG2(a3bkSb;p>8t;*j zG^VSSv`9GHvL}Riuk*KrY&cANfv;e>f)$9DERT6)tp+XQJ`?F}OBrHOG({asVgXHd z6l^i);QZb2_4Mflw&7Q3$f=bCNkL|<{-j1zifMaC&ny;Y7R;;~Cj0iZdrHRV_ z+9msu!#(+IxD=Z=zTd!+xmh*H?i|Q0Wyauii0b!Kpu)zeY-ooqZLC;TT+zDr;|^NQ z=YU{NT>;H?f2Mv-gWpUEM9yznK=hn!8}CxP>UiKXV)0ogDfi(^{8j|!rU~MbQE#LA z4J)WTpxc*h!m&Xs+r4Slh&Q#ADEo2l7kbX;Wfz(B@$~Z_KBGE#*P7sSKnQ_lW$L85 zF79G^xj99<>?WUxB7Pqc5z`!P5Fwnezq@Z`JFOE(-*7s*I%v;o>p6YjdEbf5FUFrV zX6a(Zt=hUS_haC=BP6oeBnQnWkL@0j;N zhj$oGLC-Er-v4$zZkiN8Lj9HUY9zUiIpC?q0m z$!!Hc0-3chd6U#dyN4d6EA_oyKEDSOq+vjRg3eg^!5K%BhBWbgL{oeaU+2qwAAzp; z!ehcwM?~;3qx?@>g9W)wiF9u&zt)`;?{3(yZhh=cuc_+eT=w+}g@qJH3vs!;<4P%M zKCy*pd7vF&q?FI=iPTC68i%(C6aa&|mg?W?*w+cq);_{L|2!{+fvTTBF+%X)E`xsT zk2>Rz%UA_aQU17$^b2;L4XdtWFI|g=1kRCp%BVNan+&VH4_G3|L^(wAR71>8OLHmR zROc+LD+;!#-AbsUCkmIHpeA?}nVIytPS9Q8)=gN;%v>mM?i=z3P*Rt`*3z`;Q)}Mc zbeG??we^QQ49KKmlU2YMYBRt9Ym=-kTqkyx z84!12U_86;<$y~4iz!BcxGM}e9F7p1j1C~}$_=Gi8Mcukm!fOf_6b}D_N$cbH*msW zB8ePvY&xW2WDdr*5GsWaG+fAYaB|2zaBo5sLYPA=;6{*Dv5XWTjv+@_uQIm%{=f_f zhQdtw56q0DAfFh}29dz@aMM&=NNgch$R-PD=vWR2oXUIyl5L9p%@9%KJFp@BBk>d6 z4$OU0STERx{uNSw1_Uc0W4jEhvDc{(t^kStUMj|&UWf>CVqY2EE0jA$-?(kJ`>Zg= zZ4P9$xj}{d5^isY7%{j?jXq|Hi<?xD0e0 zBmz1PtCOFFH%&7q4N!}%M2DJR@jz9iA zBNo%zX@E@54K;?BTNKK&xifAM4)aX2r4=l_bKaIxk$Jc~X)`KgkPFp-s!0 zz?psRXlYBEZQscCUQvy@E_)CEO{bILYi=$(y1lW*>cKwb5g6}CnKW2ZJ45?f>oCti zcI&*53b*LcV%S$b3A8U^(|;h8+DLE?yuU<>xbTLUk<-;dQ^yQ z8Th<&b#$9F^;-zke~=XQab7)O4$aI~bM&Me%*>M5g>AL*W*6ifi*L*#uF))TcLe(l52{q-ULK(XORcy1Md@CWZj{<|Za&~}f`dR}3NAHrQMt)ogu zpwkQV0k2b%76)VOlxrUTOHyVZ&I z9~XeUnNmLhz#r>|n}NSP2;+!NlW0%SGUin^6WcWN#;pYw>h)h;IM};A^+rO&YS{)m zgq*mbN{2B1i=L3<*-2B32sWGjP7kRo*(p&0=N#?J$MB-Jk#9*>a$yMw2YNT4E9g4J znl_wz%Vn-F!JijCe!`qJI3E8hr+<$c643*Fm@e^ayRUB@L%~S_zum}!SZLhBtWRJu zMdH!RFtI(1WJXs}n}vmM)jYfiziMpXDhu9G-|fZ_OaTJStlZ|KOY%1AiYLR7S&1oiquIFG4b>Rz7W=cU z9bs)4ZziP0rksXGPO5wvQYl78XFv4f6t*O-WGmXZ%{$3j8&EEbCIm~Q$nIRDCVC#` zE4xx2FbA=k{n(k2mU0V^DB&4tJT_4RKE3Ubi@hq6U`<{W1JI7IkY-TD%Jx7XKA|aZ`HzHt%#ZVcuAo$u3XBE z^}w5(qWJV7;;@KkJnyLKRk~5K7ISL1F3V4vUj6iQ{@!4?9NlvZgr7gERb+#vooI1p zb~>sCi~c(bN({-R@rRpq9*nKFFocz?7y-$I!d>7>E@*KisbeDp|HAn?q3R87a1 ziG(bc(H}SAuwrmd(juvV0&QgMKy;(H90u8;1Fg+wi0h+55@_ zuO-kb5PBY(k;@s6YZjAvp(}j>*oy?Dd$jyyh&T&w<+c{&(gq+4x>>Jf`hrs+*a@8x zjX#m5jX%5P2Vvs1ofS|3Uv%Xf8+z1d#Y!P|b%u2Lt6+qXwI=}MFAWkpazWo+7i81@ z5i&mf=Fnh9>qib{L}UdtmVIOVdZg-$-PIV7 zDz!1CL8^Z$n?MrW%C+!5;WYG2k#t6)9oFW!Ol#Vn7@A-sa(hnfGKVky z9%LnFV0>G4DzXOZGF~O>?4EWHc&N+zNr@@F-H0FVG|6;#5NtZu8JXVISk6&)<=9VPK@eLcXX8N7d;jj(mSGyo!M_e==ZyYHT>TG ztJ22LE`u%@|F|q{&6sm!W4ILRvX+LrXpUMMU~SD@QHE>eH_UsXqYIvk+$*fEL?>0J zb<|GE%|1W?L2Ij+FT2zRSYtY5iEAO3%nLaHiRFUOu=SuI#Rm8R`mZ^CkT5wCeg|_e z6WoDKu#G8>E955Jqh|n^AKy90FLepEPFOWMEjI{eae?>ycr3K6l}Dskh_En6w6A#r zd<{Zqs=!K?WBmekS;O_*j>yidQc{{g)GmLHn@Nzin^a_Pe-wzsMzgyxDmn>fp(F2$funMU({qi84zPQXs)^ zBMAYrs7ZN5F^papbH^bW=*(lwQu+@>71GDSdK z9IOnXpa1n5|1>y!{>hn5fz?2NYCoL+f-PXQU}zVW7{Sz$x)`8f^;_BBGTK~&L#WTq z2VdgRvTQfZj+U{LWl?4f>rm0&+b!+I-nS7TT5xnuVRlDg=mKu-{I;ugEJ%Wpiav#z zO(9~q+IcMVYPG28qN+Z1@pTMsgrR=w-0pm%4~6BWEU~0V_@}g#J5TtE{6?9J&rM7n z>WM5}Nhmx5MOW?YlQ?d~S^M*!xD>xg1$QwW01p-ezUJv`8uFkM&lxpbf2oXO&L__Y zN?HN3RL7If%;j}lJ{Jwm{C<5jiY0PB#Wkqb9Ss~W`hlT6bM9@zX~nX8`l<80|B|6d zZzNr=Ha^-4ZJz&bX3y+cJtMn|3VP@WN^(AJ{-8toI>?1`#~7N+S{E)*{7;6M6d*62 z9tOpr_k2wh_P?eEQIpHmIDraR?{mC>2coL(;@dAa9|FVdztqX+N@rJH&{fHHM-SxU zN=7Wvb;VuR*2mVPRMQK~Ii$JB6W3AEa?E8cOV#UkD9o{{YSSAtmdFa^KEgSn!Z^8; zid|YR5&THb2d@Z7eombEt}d8DrJ9kK|u8Qx9-aAi5IBC9{jrQL8b8ORfyma;KGQ$6d z%Ye^6)PHs0=Vq(-e2bX@%|^yusTnnI0OEpGd}+zoI3qWe9|uSV!fOTd;?s*;0?G* zR|(#Q0oE+=%Yn&foGq)lQiEmS2yw}%0P?QLVCrX_$wG1V@fl~h{~KquRBg%s!Wq&( zI4g!5K}vY;U{IU^pLO;C$4TW&4UR=Jk%VZ5WMD&ac4hsHGtj?rwhjx$84Ra#lQ``w zwf~=R)~|36{6}Z1kpG1YCZxlgbcA9nfv6K7$#nx0_2lH-Uv#Rl{qx*=~GmooNu+p^4$P@KU+yns-l{n1%S zq7rst_Qc3?>u}N`w4Ok3Ix}dZu<*Zafh1T=T>h5M&4#m|c#F0wAH$uAvy&rt`G;ZRme_!lr#t{j%U?ck&T-ubjw`u=J>?@d8F;hzWyZ;AzfvYdrLhnYl0*~Gdlnx>gvPZB5( z4Axn7hl-n^$D4B8qbs|kph(uEDncD?!QB0I8$;&Ja;Kh38)H8_+w@)*;BuR0+W7v< zrW9b3kPcdc(fQLS2NM))-FaeuFLhcKV!a9gC8*qC$H3XM<4cI!h5KhOck>zkOb_Qs zLpzg4_l(`^8n0Hw5rU=1aJh}hG|mN&BL?F8mV_c;J4Bxz0v))P1RV+r-s$s-K6p`koMq$9)Imi7mqNC6tq$`h~4rn?U|mY)0BH)f&aDOaI$y4xt$opwNDVZ`vRo2gdp z^>5LMnjRs));=GAqF$9VN!6wM=;FK`z7LIls$}mo(BkF@Pj;Aqg|^FYKgxQS1<4FS zTztI2(`V)QdawkTq@}B`GfA|7-z8R2oicwRp#o)RW(Y3fY2R@WZ&o4OikYdbayL&? z(R9;ve?(xJ*(IRY*+F{DRI)TR?9ON~CPc zA^*0<{d?`=v$W(&7LaY35@lp$?1ScoO6~3-ZkW=&N-@L#yON z$v1^f=K=~!*oV12_z3eKH8Yk! z$Lb6#swQb>4#V;!pcBU3EVINnniy&Tn4D16q(HWwmN%)Z+1kuND{J1Y*I24I4|_1| zSZImyLtHwGQ~H?qQoq2#`pT$PJ>tEuxztsW?nXjb$k6Nx)m?ynueS+j&KAPGD>?WM z&*bKLQtaZSpDd5)l3#-g+oL9SuX9>P`jy;i!grSxhfMbgV;hSPGd={ReStoZ(>c9r z4|l)bLJN~yD?-Qs^plO(T90AY&DW6Qy2NUhr`fra>;H(GhbD`Zt;b+D=xVeZ7a8|; z@S)UeVckh3>s!oO{nMzpspNx2Q<9bvp|D*7=|OQXlgL}kBKqrw#_60H z8g6JhOAP0(v|^nx=iuadIr+X-3hYXmA1JZxWQE%GtB(pIh5=o=i|Gy{M|L<>H08LwqcA0faf=+G%xBAR|kdf z>8BoHfQG0*ultuXGhcE;gAE}HeUv9YWH2T#ogi3Uy?w@s9j^ca-d=Slqu^vi))ksj z2eHsuHr)YM)O8FpJ8ua+j77JFW;XtQjiq|+_+sxq8zGB#(BJ-3_>vZTzmy73apf&* z35_Ovg-p5A<^bqUgskI}-67wL2X|teenaeA`_j!1M67BDc=mU)j#e%0Q}rdCE#KE3 zl&`(9r+f}zWgf^6mpEztR5@I}?)5oZoyo6vC@#VX`$bsWsyym5_nbo#bt4FA506I2 z-)yzGeXk8VBR=&`SnbmkbYI}=3NOTP*QPYmbAN7Uxi7jtcqn|Q>dJ_9CLIZch67x8 zVjy8diRQll;xX(RSew7+Y%$M0p|Q5IBrT|iy`*b58{0l`kNB9w z>%)Af>W8#BB2eai5iW`($c2#q`9a`Cokh6_;4~9r5YdrR%n^YpG4#v)tcA?;+gD6? zx8wcx^@fKwh}er>CUwDn*hd&!23pmDJ4Mzqf8wrg$=mapehouKavfIn11Bm>=)+5h z@5XTV5K@B-a5Om5p0w=6vr#nO=%@B=f33x`bHdHuxdp9`#qxO>3k1MUS#(g(wQh@f z6u5AJVn~G_@i%n}exlqg7X7+M*h~uz=LNl*VyJ-}CNDhz(z?V_L%Vb2q2-Kf?#=On z6MfRfyB8wkLOJ@+Qc76R~JPCHZmd)xSz(@I8%bk=IrmC&PM z;uN3z85C+_>{bOsR;|vhOo{!dql7s03?JkrPwA8I*~EgmPcyyj6!+l0Q!;sNo~}iX zCJx3EeIj4Tx~1%)L=JQJC+)U@lQ3w5T95*4lP24d9_KV?C<=kdpF(Woi!gIPokW`> zg=!4WVh?d~aNX}k+eOU!(&$Z{MdXbcMo?d=&hluT;meuwNT1d^on_y#$XL8tu53?{k8n6p`^jqow(n~3+1-*BeMxvzVu`bFyUatIeLOA%_TN@lFPlN z*m6d-SID#9X8AMlKof|63fWLC)X%H@d{%YoQ~g?{+|IbVg!cq(&T}vDMwQ5aUzCwm z8Hkp#RS(|x-szfr7;eld|Kyqwf0QTwgGC@SL+_ocWk|q8aMV58* zj^6f|0N!r`*s%oDkU);M@b8Flhxdx!_?6i&6W=3R$ZpNtXvrS(0Qk4CdIs=FZoVWq z%qR&e8Zh%aJ2xr)b$fF5Y4=Y*qxo;A{icIH5dWv2L4T^z{-d9zv%BD| z?FZ3K4KXAg&|RI@JLbUQh`@wK2g7OR%z3kmuAt7l2%L~?g;zq^Q2KS zBo+w+i;S-ABXr$3kHkSwL<>-cc!rE4oCb|)3D|!b?O#L#{!gR1QzDqBZsUa5U=hI)fWN?TI)^hvwT2Kf7#0%e zmGwab8vI3QD>KCAJ_Q!DFA?va^iek$aR(LU72Fp6eFg0AFiI9^LDL8ZN|*%y?!UzI zg%hjkD|W4eO~ihMVXC!_+@=-GXtxU&pok6o9u$QDY>VG+jIj;lWbJ>MuPu#hq~=;h zV_@3?-&2A_IoZ#7EKT~ruNZv^UETik z7y$Z(<{-;`7mWbe25%OA1Rp!7j;V;Wr!vaPHIb$T*H46dy)^yjiq>K?CQn~dWAK^U z0R{BsyS^sNRZ`=_6SN0Km|WlCIT-w9f_GBMqP704*BMxs2-x5ue?otG=`Og!9Eoq0jtA};|i)nJf&UQKv zw8QwHjLNVIjN_(0E4tp;w`Kr0PV)9LVwt^^3f60AQNsfs9w8f07r z2tb)OH1{v2J^7tGC;f(C8P{~!Rrich8%|qV_D7p3px$kS?YS;t1tw8Y>~i9%h@j1n zp5xS);60sBvE2KK+R~QdUS^5#_W zc76@H9rD&S3w$fuJo;k8vq?pX6o%I7r_J=?A;~TGd|zdYc&o||HdB?Acr$dG)%k!c zRrHefOl0Jgi56Max%`xKG=V!$zDDt5o9g)lr<{z^KX=~}qsuMGmy~gq+&aMe=Txez z>4uL5{4BTkW(f^u$~o=QMv3HmMVTh_{Z-vT!?9+LV#AG2>y6T*=3DQ@M+_WS`({j% zujd{DBrw50vawrc{X97)uB&qjmeb~d)!eL!W*5|z4VIeo<{A^IY6Qh$ldRc1yMVlB zrLFJ#m3o2NN3wWu@#m;B$KFO756J7VFK%V08sreTkDb6WesL@8M`ru1Lb4*^^KyC3 z26aK-DC1Y)o<^MG%0wKF^XMCW&I{t^qxE6tpL|Y6_Pka}mH>i_rHahMYGUB`ehrmj znz!p)9dE8ubZSp{D#DxVi~V@o7a+cTC6P4W-2GhC+R8YwC3-LqoWBViuyy#%AF@9Z zs5jT=$;-a;HCw_Mc>O?b%aGRF%3?J*!|A%7Q~T=z9r>{4IR^Tl$3=(&`XY zXo^oky*1$o(Eq`scper0whjRPsc3RRorUBSA((jWXS3;uBZuO5SyQia+@B+w-?$Ce zb40UBT8oB8H2ACfgyLGO3hsoC2v8+iGp^W((3%JCz-`|eaU&@aOHNQWpfOjuYg^%- zX~cM7B{1x$cE3%n4>5M#YbNUTi%s#n#aGpbyEE{Y6;yR5DZof9IcKxI&h%nun~H9- z=5SEn6#ag$F-y1v;4NhTExgTl?WeHNb!5&6A01b{qqn>x8*r3WXq>Xjd!ww&-A;Rt z#>h*&3+H6b(IDsjat&WrDe~>|_ia-f`oX8@WQbkKe}I-4>n!pNG;3UUA;JdU9i6xV z8#fHjP#~!gWsAG@w5ybWhsP1pHri!~Ah%e~XDHA%0M-3T0k#m=Gfww+qm@O_h6X1# z^*g6Hm4)4(*@x(NP`<|n$KED0G-s|HE`Yo15x3*x21MGs zZwJnZPB9WXa81V5ecTp6khkS+rj8rwkuRp7um~eL^2W)t^HDEfLSkvWXxaKHUr-9U zk@V=TJ|{Ht=Y*!4uzKpP-7K%;41a(%d(EYtM|4;4oX~*KhW6U|<*Ihv*lsccf{ogr zX(L*lk3&_#98R0(J|Jh@Q>w%zcxfWCXGJB={}`g@>yvcBIk%?xv`6aeLqRaTcm;@U z0voOy6YlO`{7|R(?#|K;*(_4jYtU-#WZCd}>7Cvv)J)+kb&I<)Vh6Vq;b&(Q^*7Ii zp*|xOG(_F$ckCuNcp@AK{bTy0L^Ft?6l$vZ-2-6n^%BvrbPyc|70{moUjuH#?C_L z3T7$(C9Uyxbl3^HJRUe}ekq`!*Ymp3m~q!NUFm=un$dVoCp;ors-3SuW7Y|Vm@@hi zNkDD6xcM>j)QqmSfZrP5CE}=?G^_PVrjJABI?St9uVtF|HARcA1S&_D6TayQ2#&*9 z+a7VURBAYrq2MjJ@7TDLKd&1n`R6}?C%rmn%44z~4^D&@A}vyYtEv91qPb;P&+|sv z-)D5hI7geL2nUZp?^~rwF2S6>iN-VI*Ir&0CojFf`QSF(Sc(aX^2hUScwu&9MVPn^ zd`6wyO5`AEm+V&kw434p(ZSyI5ti^*7p*mm%8UO1=K!f=(P`@OvJGAMksB?Qtd?VH zLjQoo(QzfgC`1KR5A|t|QE&Vq3gwyimq5#*ay_9<5n4+C&5oCmclfU`d^3d;Qs8_^ zHI!#^vc@H9g~C~dh;-dM()e@il8Tm$$Q^tJ-yvdS zF0P|i>ip0p0#SHp0`ncPjdYHQ#I}d}fEXntG`dq~_EiRseM`pj4P4nF99TrTNVVy` zk<_1z7IcMi^?p8rT>agbXaUHdZ|;>nwQpoYGaA`{^Xz%9n*sfKM*UBjC90TbuSa`0 zD1{;kV3jPn5RT*2+XNOVTaIoK$Fm_+2&k-=;(K|7BL%(%>r=n(y$}S2R zm^h7MkKb>8=z0Hi)qDHy_e6H4An(t`_M_;OKP;>5JzS{JGHIKBX4x^m4e(zqGf&Yg zSDAtH#^oD$vM!I*;|oQM$bJiBB<}hb$~YLH;VcTOrU5uL+O{~e)NLg2JaonA%K&af z;)7nF!3ERFHU^CGZ`78{uYYpXv5A$~xw2 z-R(<=2s}j)CliD!AaUCUuDNS8x06eG_bApD4;JMF>%Vaq!4a z39Q;oKl6Pm4cwyvwwJ327-)m9foZA{hA37Mh0qqfEg-`;YmXbCfIZo?3cJKAN!<|c zT1G=qsM$c&qX2msLdjBuzF8gReP2tH-d7W&k8;OsPHguE!-vV7ejOC01M!n_jF@IB z!c}GJ$28TECJa!)UfUUuS(ex*Z?byftSyc8zBmdiKnnZqJu-M2ABY{S#zZ5F+6{4% zxf$@mYruv2%-FS+W{QFRs80l*uL?O(G2Sr33Z?wc3j_$-B^V~MFqezu=sGZ|DYx_dYOP%;x z+N3Ac$`Vv##ZI~e)N%gxp+-ePq&S|~eK1a`WC*a@#s{hy`AkMqra+!t(Nt)q;Eu~! zQTBXvh7fU!CiJOY0CkVC8Sg6OIiESLt$q_t=}7v}Vo#5Vt~9Re(E$BkD6NwN`)b_B z+Pa|$EiOS0FIVlLktILN4?O0s{)%`t*Y^=j@#(vL2eUw)Ev(-94Lu=4)}(azM~&42 z_xDk8rc>+yR&Hxh{q=Uq$yd@-oA}M$DmIiB)hU}OXKQ^cr2wl%_@B)C*_n1dsjPk( zSM$%~#}`xRkpca@~kq!{AFADk6QepP?cSN)pRyVBel`tH2_JLr5GI}aS1E_ zQJJBVxvEo5jb_wDDZL@$%JiPRf{6TG$|s zI9uTNTny6hr|61*sUaq4g;@Q{lJWY zmUPweFkEmE-j9u-J@q3hsL6=;Iq}d%ytJk-aO%s6p2(*-pe8eenrvvGgBeq~bhsoi z>boSN_3QrKEi9lm-*6|;@{)ewr#8$aVvT|b2cp-Urkv?_6$6^2L@bhc4(JEHq-?Yg z8mjqtg@3h|bJu46Oi;B)fRapieljj7y)8}b{iu7(VB*N5%R7b?D9NDR!-X!3h&4aa zZdO{2hs?di$ z+K5}&l(&+hXfu1_Ey`Pi{gq;*uRyUI@+-WM?PoEvux1`y8$1IQ#v~tJt1VZFP@c?B z1K>60yo$lce3E)iyGhy3Gp&T|3f5S}700nE6lBhVy&KN5wDKh1#6{AtQqv?V<7~c- zUp|ezG$7fddHG|(ZGzJffg_zrK2XuboRn}p&Pq8+lRo}SF(oMn(Mf|9iA{G)iMsj} zL0*Jn(($M*bNloMW;<(MdoDbc?{K)({CL2Gg!mzDmFIE>PusoI^K7y_br-G$J=Li1 z!vj1d3FS%hJ}yklgm^ROc;WsXOxlO@ayq;Uop`{h*`$+D#6~o!)0v2E&68hTs7UuOi@Ww{x1+y_ zy;$Gx4BvvW=c=Ypp1sJMlV-z-8#Ko`AaUz|h~N|xJ|7hU5vRD*Axqf8Vs4W>Ih6fI zhY9MQVORv$xgI9ouNjmNPzE6Y^)P-ogBf9fz@jZTxHj_DZHec)Md9V=>UiT*0}W@y zt-7CNJ}w0?6u+sC&?9Ig2h4qi1^yk*hM&V(8Z?|80W3QIgfo#p;Y{nFa0Wwp@Ep!C zuh$RApbws#{Te{pcf+69h!IR-4bFOah}M-~e-ET|YsK9xy> zxR4mp^2pfs7HYTE0DZX-Nnbs9A1{m+*Y!zA!x+STFoevH_-xI4YV*ML<@4_EuWe&^ zE6-osMx}&<3X|W+eP@6SOU%BXZO1Kc>yg_80@NAq*dnARLqq`>HV~Y)=tkgM(z5Rp znxmy6K6x)xzj65DS`4fh$vze1Md|>O(Y+{~f5U?)K2;VZAF+odHa4t|IX&s)U$G=S zA_auSE}|s-WP|g1!i%f%T;7vv!{Djx8--Ze#@7Kw-sO zfa79$UN?dbKa@GWEBR~Pm@g^0U`xaKN?w^NDeO2Ux_8J_JWkl=P1c+*SRgZ#_bA)N z`CZS`eH*43o&^dA(+oZuD;J)TE>{svR5}u@EzYhsTyni7uu1kfq#7%$II7S%s+I|1 zhEj&iNds*MEfh~}f{W;zbVm+9`}`;Yd#ufd56V4aD? zfRjkvpYo;(Xhyhx7M6Dt&yZ--qfPN0N0WPgh|I=TmC#X4@6Bh|qaOM3g5Sl0yMV7s z#g(Zm4{MlnK70|Q-R47|uR7tuv%b^(jcPWN$~BLAPb*&>X6jBBIHba~0W93DE@E4{WLuNNh!u96vS`%Q1H?bfzQ$PABitq5Le-_D?iVEDg*_e^85nyc8i89hT9*UCmx)Dk@-l7AoEFMzG^KBOx&LMZT< z9M2E;Q&U1g7d`TBklecTH4HZEz3g`czB&Z@ALO!esFF7oHLnFlVSh2$r|4MeRB5v1O0znHVOz)J?m`c zADyKHfupIPp9A??Y8S^GV>AplBg`)6F&-Zxw9|U-Hu7&mRxt zzZWlm@w!-VPJA^y+GM=!Abk;HZu2XRQ8fn$>HlT2a~lJ_H5+1G^dA!bYu+g75W-NO zT|oSuUrMyAlK2(ZIvVQstB_xCN4S5GMg<);q9SA9z6ymt2BPu&7>EM&F_1>&JgEZQ zIP5`WJc$sqoNb2 zC&UIB8qMe*$=@^XNZf~nMQ*=?dz}K|gkGXW?!@9iafbXmbV~`^%-DA{+VuJ@;1UOx z=wHG3DEKB5sP>hW0I{GG!_#enkZDDz&61$WEVT%xmcC6VhQYuogi~Qwxr*I29xr*2 zpr2fmO%i0z9gwn(2=_V_N;4K2kpR6Ide>MyrJeW8=}GAauAO*g;0@^8Ak#_>*kZIY zjO)xYgaJA+&i7e(hR~xnN7^faRp?_N3fMtg^4MW6^4P$Te5^;K7_I=x7$)``a4U!? z2s)M|@1Y;UXp@WKRTzH*afX>RSpPU@t~@gGeQb6(t-_I3_HYKEa{56Ll;WRGC+M0i%fg>YUj$qp{Xk3V z@YmVd^z^rw>fDwZ$}RrC&(1z%)*PB?6p72O9vUDaot;(rMlUEXYVscU!dM$m(c|?$t#?+{}?05Mr zGcvRI&HcY*X7z8GAsn*_@XQA=L(k53%j9wa8EPVouZ@IEyNjT~OkuLSC2co;^|b0w zFtdiQyo!t;*J)Q46Bi-MOaX0=0>3zOh#Y~;tl4u-v=F^!Q{BR2t~E$J*^S2IRa*_X zri`CuhSmm^nUl}6%ufGXW>fzmv#7shcK1hSXaANNg!gZm8UK-4#y>IxKGQ6u_E~0u ze`MDCzhrjwUoso`51C2aP5dvJ5l;W7%ovV9_M3v57bei>awlW4hrSP&l?`#1-ox{m z9vM{<)^7p38~(mK``>40TmL>g!^w-~J+rrEqqx&)2K?R{Sl`V~@j(s90Q@~_T>j5d zqvv48-pl7jqce2T82M+>=-~~%ypUk6g=3=-P;LhD@Rxp-Y9cgQuzB4NeB8p~;~VO9 zvAhc(c*pk_%SykKloTiQe9gY`eML#hM@%kHF>;=x@%$9X1kue%(eOv-3n88g#*!z59|G6-Gv%2KQRBjY^e5fxKLlte4!*-Jg#MObjU}aJ0&sdY z=)}ryMvZPjM<%7n28=7zXkx8xJ)|rp{Zf#y6Z>L`1V1}VCOc}0%Xs4Q7 zZv6i^Is4Kd3q3j0`*U(O@wdzVJ~_iKY)V>5`QtKL-Wv3;WX~=uhTPH@)a=&LA1nId zY%_ce0`SeNua{6)OaQ;J87yZ3KRPe;KQ0ucDV_-!nB=>YsTiBQ8nf0SXQ%55N=WI= z(Jl(tzSOTz3~=CMZ`_`?9;diFvx|$nBGnq7sWyqppU#GL)y;`KJF6pt7@<^ z=m*LI$-&i93q{e=R9Nw5vc5b-l#WK3CD;Ss@N8Z=S~5=Y))XIdJl~v6TtaZJmsK9L|t^r?Z)l*Lc(r}ZbmBH}SZEC&1Zl0vS#vsdfCtkw3l%lJ@+!yb$<0kXfLb1U>DN=JXDD5TuxO7G)fDl zlB>1Y>&z;4ws`Jk(CD>|bM>13?_Op`T=v|{enETL|KaN_{HoX+tuGDI-QC^YAl)q~ z-QB&BmhSHEl9KK&0g>(!5D-yPd}rf1=RVIp_w&MkF#9*N_nNiVcZGuevO{Cc(GB=# zzl^CbF}<1yhQ?B;6_UU>FUEIm#OW3$saRJ7%yjBji6x zQ-7lu0TyPJlM5^-u4sqHhFYtyBPro_%5%H|h`boH|7l`s1hmIr2t8=Zr+T1b0&(?F*CpNo$hq z4Y+e=uUrGSsR?w>TrmRM9hEU~je*XY$e+^LW9#3YGiF1tz9O(=rUP`&P*t9BjeBZ# zalwEx6Zp3>duD+sIyqYXt;&)eR2EbLPNUc9l1B_2;FXPsXnjVZGoxb}IX*?@u+5B|+k@3SlnyJ-v;`?2P7Q@@%2|p;29K+2^?gU3Zbx?!kYvO#R`eM| z0}=Ap1RGw~CHe|E6Zp>oiQST&*U3**q|zuLWVr#1yb)&SO{| zI&MsEq~Y&=S6<9&J7=$F;w-~sJ|;40U1Uk67ybs#B$9i~=bET0 zcl}d3GwM;zs)?+gJpY-qr8KruJ#u`o!-#KwoE35G!sdQCsg(B$gsW#$lI_cEen(3} z))YTfx0QIz+yWPD43sDwVYyz9B9jkHJnb`1ybSCOp+mnVtUN*2nCj_x@? zM0C}U42FS3vhF)c{#0;7!+g^Zl+Jp-k5Qm1PotNdDi$a=0i`qX4h#ne#Q->$Hp{`* zO5qC`!JB8dOp=t5DEI$gIwSrMw@jM}$T5BZ1X<(Xd`4N>9B6w*B&vknlvcYh<3_K+ z91yDO0xz3FTH!AN8uIk2biu7Do6Ox84`sB?VQC~eCYBzr(xk6e|D+i2yFrK3fMNEW zVl?!EdQLH3{gYyBoMQx~`Tmn)QAF2p~xyzKt~PBGp+rx;%xp#cT6_-+4SM0gYo zB8v8eZMJ?+corHU#VB#92o_~Zmqd`U|B+(EFpxGzD}h%G=IU1qMiNgmjxx^Lc6|nc z)RBRmGWa|duv3QKm9Y(?2Anc){(eUIItuSnR48_NXLP`QbBBxuK#u9HVwgkveTd1- z4QNIMeV;A^fE1$_-6ihrdvJ;o8-OY)|D9r_yv>CC9ZX99C&ehK{PuVM1@?cZ7WbEe>!$!?%0~m2P`759c?vO~GQ7~H#JV4(rhKkgVFb{IT zaNP?6s_eU0oI#b69fTw&EoAXt`h>SRs%Fq@#wp3P{`*8%B-*$w0-jv8e zB5QtW&m1MzLOsHq!NFQLNhmgXQKPbw2_1^k7u+_>5Brv^_tHoBAG=KBk6k9iM7v)T zA(aSbft)?IhuV=^B01%L$d z9gt+BIea?Gf}y1=pX zs(~&nkave~kzesmiA?w%qSOM)#^QTH7$%l*Q0IHZKzCF<1bXA~NXH2%jY=0A(xUFU zw3YaVOI&Ddki)k;!H%1O#DJ<7TjF*Tag)CU=xquv?k$-U$Kc={q)}t`+Vg?@;%n_J z{M;w)X0~Uu%;tZaWu8M$V6*Hwzu5oH`3+!})#rfCGN*r=Wkr9RWjslqAh1~``9V$T z3apR~5!PeM?oRY@0dmpR*Pn ze*lK7|ejWSnX2O7f{(OL2krCB`MAX*M}OR+A{i@%6n47+{Jh zi%a)KB+E)Nrz`qZxmE}jLl;%~1VADrqfEMjX2Mu8*I^HPF5C=$k^%jyAZqeeT+~Tx zz9MM~;ebM8kSZX`FiyskR5MXYwWbwI*)i@U|67#V&1}DV=~16{(lf@oWFfN=PXa3pI_XCvFx>(G|6z6q$bF0+?)+B+Ef`3W#ni2_>%E^ z?Th%Rrz_)bil?q_@!S?PRhBpMl+m>w$eh)%X1}`S@M9LFm6>yoO86b0K_Df0_)MAc zN&$zV1krX)y-q?=YPGNFtGB}cfIwR9E3GDajGYZ&C;xb5^=hQkdKUKs9}9-=r9i)z zrZZR>zdd;~5u6|kB++6jAjiu{>d|4;=KzxI=9;>K`WA4?+Ld&pS=RTv!btye%JMe< za>|nQ5R~~yFNbBy34O}B&E(@~$l<5|s+f_Izi7Y(SIn?y9?^k{88Tp{ECLO)r|fTX zefh^g!wggx!Q%}tqPblsV@N`T`=??yuV1x7F=Z!`3$B<||F4SKe>!D;j^;qcjP}_n z!*r(nyJEKYSH&z9TrqP7oHByHDrSEgW`5Pr4YQ#{!sT1gpNbhPSk9EYAl&8uQ!xwH z@OudwfB-6H{hif+SIn}j2BXn|z7}?k(;ALZa_nYDziJ$@tD*CM)8yQRae)!;if6S~o4c^!MH zNAvoWzbd}eakoNW;W@!L)^fx;hcMZ3=)D9jJ~j4d0@v(VVhFbqj$N38O~7X!IuRk zS14VOM-+DxZaW?3MEZ4*=nD~!UClp&Y_PNHA3^pBQtJbD%HXTn{>_P@b_D8zydko5 z`QOakpaP|^3MK}5W9Ofm@0wfRO<#_vI~+)5b+V9rzzhcqGUjJNmi~_*`}MaVo9Qq5 zmmm`X3$i!=CCHeb1=+)^pKcfsU_o|dCa<;ij~?Ux3jzWbWWePB3o^pL1=+V<%^9#D z70m@7T1mub8#&xPI2zjU{KKNyW*ei1S@(N4Bk z6@!4E%uzSlPQF(bLl0v@q)t{WDUKBZ(&8#q%5XG^tAH;w3=1u0*r7w~7f&Fee& z5iTODp=e=a>jlhsyWoZID&8}t*7Cpir-K~sRUSwC z*mG7b5UrU!*J^s@Du_QbeCvmeEfw8;vpV5C#*CFar5v}aPUN8={`S_{@iN%O#rg%* zbs+ag4OutpJo;7o>#vfmJ~MOl($%+P+3-%rTZ9({+c+2^1rd1?5NbT=nF@}^zwP23 zg6xM7UqknH`+2|H-kS@!ebEhS99V07H@B`|;na9!;!5ndwbv(LN@!i#YgBgn*?FRO zQ?B|lJW0Pk5Q<|Nx+^pV^$hS@Ib_FY`agTfMkMHX(sc$*u#H>S(Q+dQL@kK8XVOvY z&6bIX>q%)6=u7f(Eww~#E?SM*Bio)(mUuL8D*fyZWvUf_Sxdpx{SrZUmnObF^Cc`r z+PBo7G~vR?O+Sfa0tWN)Dc&ms6=Ke^ zt2h}`g9VYIY*U9HZEbg(-=-i!HrBkEx*TP!1=*s|rgy>l$#sF5{%iuj|TKGCDKqMl91>t{`+(2D9 z+t{&qA&3a!fDOndkwD=s(jR>^Ks0wGK}>ZNOqxf#L**|suyLu4016hBZ^sQJ+o``K z1i+DBG0r2LFdttJh;Bb%0O?BETUvj;0nm0E21%$swGi~1#7nk*%1bST_yJ}Nl42lK z$%z4kEfX%G{iC*d5XyI=)nBZG=%kPV7zzTtKVOt{1eTk1GfG|BK)h`ciDOa`L@dDg z7wGpw=%Eb@eIw6~3OH+!?M49bUna`3X&#bSwh>-gt`Sd!byzPWK_Vo(bos@bq+NSp z7XSi4neb4weN6f4T#J>xqoZ#BwqC9gJ5cr#lU`suSv}O649QFGBMfLOg#vafp$JZd zahRuFL##NBZKS2$VKpsiRZPD(Na}6tTgv6Ww{C=mjqI%0?!1L0B-lU(y3-MCD7R{5j5R!BHg6 zLTfF;>W^hZRu0^jalsG%@1i4Mm{6JxM;zK%=TycU@JQD$lf*m%TYSU1n`JYU( z3Jf+?99a)w*AwwLlk$CLGY)15Pb_jp1XV$KtM;w=vuND_s6QH#8kpmt-zUUlf0J;~ zrQqGzIcouM`r#%HQcsIi+SXIn_m=55E1BO$`5E?LA8R*Ifcp5V5<9 zK!5Fs@N+JMA|KyLF|LBgqxm5|)Qg|hm|YUk)K zckQZ80MoCb(`fiL%6evEvE~`mZ{JacOBwam1Hkm3D(*~LwMKupHtG|(m=R!*i-{A_ zm^BZ-Mi0FCpt0!(#y|$23KGePvS)w=AkB{g+Jj6cOGE7Fu&*K@-PrHHH>RfF;@3LL z1>Iejb}lJ<0`GMWH|Eav15@2E`Z+mW$38CDq&hojxufh6Z^Pb(q?KNI<#)ItO^!L@ zvh^m=>l>=T+}@vsVl)`{H7xctE;hJ^kG**;%Me*}?Vp>&{v8(5iGp&r1kFX#Fq+f8 zYL*Zic1B7{aQ%t`)Yb-hXskJf$=CY;#zHo6!2bGBB~qu~86W$Frh8Reo3|z==!1)3 zfa#d0aNq3ax|85&lH7XMwsX56h7wZx1h~zXiWFhsFshc>>_9!2#=5g?1M&U1P>K@3 z^;6<8O1DbWqmkfk_M660MrqWX5IYe>7gKbKn)6Z z*vwbNrtja<#l@Xuel%Fue_5Yw^(~e`_-$c{%sEfQs@^);4QDEMl9W@HD{C@wMmpw?mCEMxg7g-8FSW|zFbBKlMeMsM zw1(p;R*(kITvKh=B%y;)D~2`$`cH^x)Zn$Sg%DPZdxYPv z3v*S6#74iW3O)HMF={m+|D-5#HwukaRK-B$`tYln5_1`>oJx~1rR*<0)Mh8jJG$Ll z6>&^wS#b$#r;S(cR~s^F`@MZr(X&+b#s)G?>wbM8*=lt&SZ=TJw=w1QY9-ryPTSBT zZP|v^7=duzh>GE)abq@|;h0McHj(9QjxzLemO~L#{FRT0Ob;*Z%7rbdCrUa~GKW1u z9Z`w4{jXJY)M56x67|J!Kgh*l9+--X3fh|AM{Od$7SIAfZXG9INa9Qt6{KlC=_@Ub z$kU%IyR|OkYI(8o)dVO1DX@L4)Uo_3_Y=^6QBni#J=_}mz&(|&+P1lX?an=0l8QqT z7k&*yB^fSGki0x;DNO7M=vdQ04u(r*7Z7?~v0HR{Y&En=A7|={$Z!y(P1(bIf6zJq z?oss~hL=D)Da3K^zpW2@3W0gyDgOR7gs9B;#kaOksv)8;Jicbrka%{TDg;L$>vC%q(=GuBagQM>=xe;U02yK*k)ViBo;zZMDPZ$! za@#HEy?(t_`wJ(tsXCL2OFsWl&dwwNF9HjsBpLZSQLomAiEh-qL3y39oCP@fPd}Bi zX^r`D;gC1Aq>Vn%^Si$@_S(}Cn0mV{nhSIpuKj} zmyJg{bHH)sG|X4iM|-Ebw|9s;OwvT*lyt=1=K+{;Am?OJ^|xIyl%mk2hea)+dm>YA z>8|Swm$o#ot3R~Qm{eN1EXEG^G^3!B=;=vz)=MoaibWPXRm_b5$xcKtnjd!*x#0`m zVa4}lv!dAQ$YgB2Yay0CP(;4#d8@P4#IH-ER(@_5I8WTJM3?q-eSK)7QPnn=MMr=G zDI5VR<{5x1JPT~}nS_S+{t4I4XAj^-T63qvZj5eq^EyZl0Rki?eRrwk8gIYM zFWH5O#Y@BI% z4hKRreGvz0$tF+r`|`i(LiPG#fT1EM=@^ZQfZ_}Nm*RVoMvXxBj9&n0SUb7?6Oa6l z!L(;6KYZ#llz(`&1i@HyE);05bVK{-wL&qoIBYnoZN4M;1?>;vvmdPd)Y%)gJz}V%X&qNu z8(li}8eG@46((2w$2~_jQc>|rGFme%;D>PDLt^3noUMpxRb%;{qd;_{jZP)&DNU(} z%b%%bwjujk+J&0Ew;3cKhEF3$k zvQt>^p&j5r`dutwN~BY!Zgh3_c>QaNIF&qyS<4}Liy2-@%^iVTFu0k z!YeBjR&DsHxsiNL%@$JzbW_zg@20=qKuXk48f|85Y2-o8rwY#~og*%X#a z*?f_IcDh4&;%@;t*sqIzg9Z74q`%W-e+l2hTzFoEO=v0^zVYqVxZeoPQaqYpmt{4J zVd;#|gO`kN-`%Ibz9~$WzOVc&uI+yX0r?Pl)uUH31QfcrTK*)zJ)L z8*)D6mP8BiAH_jeCOJR$p&VVsj$b^51%wiQVwEG7NiRRAW|aPQzcYhCKFEk+G~%PO zEb|&7?RrzQ_==mpg@EIP9O|AVMfwYkTz$J*n0syl#)|nPWeIa=$+!dbV{(8#UKA*J zsuevB6x1A1#vibcY;HrC1VV(c8D;Gg!CjYlfP$R7m3UI7(d32Wzrv0oF8tCN$rQc4 zNF@GYoa*G2-6;2QY4zMv__bGp7a@;*@@lkWGotA8UYZ96S#CaH#S#BIBnf!SqJLbt zVeCLN#x!tejVNL0!!lJ<&Z{MhaXzX)D`_Yte5GIY2%cdgdf?Jp0W-}1&_oKDOT82> z9(uk!kEB_7J9!MH<^9!KoAo^eIRX0WVmij{BH3#L`2~4n=eLQy(Rja|!1_yHsrMkA zoPF%mZqpDQ`|Bu;ZV89CI1PFQ*fS1c(s+(As0Ir!qqm8Ip#Y*tjtYR}rvvuVfLtW{ zb1sq{sISdZwTC_{tT_O~p9GymCdDQWBqc4NZGw@2<{CAhL;__4Fp*XUwA-GOkuQUf zFofd4v_~p$^!{iN!e6*YKxEkl>ogZpO2e~Ic;mcPHpVcJbJil{5-&pTVq1ZozJ!FS z3%j6qChbys+uN6q?fn91sBP_dy##RlI@{#8Ed46jx44h0ZzBM>$Rpvo>Ne=o4~FEe z3jHOWztY=Lc#;S|Y;VYO&22dXJ`^u{iN1R5k1s&^WbOLsv6mK7-rJs1T>Su$pROoO zT7Nq_%DVI;PrEYu%XiG-Dh#Fw4A=@ggwR{Gw{Ix~wx^?xLjlJOf4wI1z`2(~J0UIA zIh#WH?JS)sw!boZ_Vz;5ad;QS<0ZD#b)OD4Kz%HU1S9!1V7FnVpm%tq;QK#gqlc4X zf*uvo(c87q*$E;C+pySQY-tan+?G*z4p?C~MmOg`^kU3a(fqLCHYrn*6f3Ux12g$= zE=z$GZ&|yMSH|X#>X{9244pC#3cNH8oy47P2+XxG(oL<71j{WQ>tVhNbg>(6IZN3Y z;uncab(dUWdvI%wh}m2t0GLX%#oveFH+t8y*3@j~S9a~Lu2+>wkjur~ zxrCv>Atf;1x>!7Y_2u4o3KGU_j{ge6K6xTm5<%ly<3m|2rT&u2|b z=|-_j{NY~9{3aL+m`YnX0i6`MW`3+sklR!UsS7)0E?%p}xqiPz{rTNHh|lJZE`saw z-*?m7vEEqMbv`epu?QV**7_Iaaf^PLNa!D9hUOu_IMkd5Po)XArDSuu?B&>w*qR?J zCLt0-NXvYvYp|Nh{6Tf71tvXmvJkc83ccySGQn`Ae(jPO-3dagN-X+J1VpH#lppuKEkKIVG!N=K36u`H9O>M#R3}$d%$$tI?A*3oNn-L8!9b9N zi!rVZSHzOw6IJ{!!do*Xdg0g`VFSgTy&#*tA`P2q5gp0gX?o6FqBQ#KjIz?w1uolN zWG?o1#mRb(P20ya^G8#d|EIyNlnZ+GBE3?M3a3dDqDl>CZo^Zu>Y%J?KxObF)az(k0Ed%Tf!YA9S;_Yx6i*5-NKv7F32 zno@b3WO-2m1Fm_qai*WX%yM-F8O(vmslp#q7dTws=vkJS94#8tWt||>NP?b8A}Yoi z07-=8rS$I3(<_PCi5^uUypwvWS9|XoV!|nHspbh$VID#teRE4xRj9%yx*|2Xc~hlS zusl7CD8dX0vf{_UQ)$;3?l#IL8F#k)-vN?{DbOeAQTOV0V3;HZ*kmXS{AH$_jW z`>9i>%mL|YIfqod!G>LU=^!{h0xYnqZPhMat?SQoxS2DhMEn7yu-ZU$jX=L@Fbt zI4b-}v=VKDFWy07F{iTgefC z6{pY#0=zhTh`L}DZ}`nO`*m=r0ImPQZ^`{&r{pgyF7}!B7`@J7U+I-7ih9a-qL)Un zS7G`1_YGWzB0AM7q>PnYJ3QKRLPWgM2jO7@_F4~ zDV({r^P+7qdM8(F{gcn~lnZaaO!Y=I3D}A|TLse|L;uhohrqN)x~|kh6hPG(=okSo zksn(K-lV`%nmBkV4JGb@4ZM_gkTdGpUyThKq$>&tNAkLE3S{nyjJT=n zpJ5{X_J}T zv(^7lR3%mA%19RxnHSNW`ql+mUQGC%IEW?XNuNb5JIYAcAW7`#tP;cK0u7)&9;`D& zHs7+6^kB|dRspofKD$H7Edvy6UiVwVu4>sE>@Q!bAHG3v=0;Gz2WXE}UoX?Hst^Mi z>cKP-7m9$(ydqhzMs=?gDc^I58YU}O)9%agRl%nGWcWVh?@4^5l2!LqrK@Zz) z2iDB(h{ZoVt-o(UL|3~3w8ut=AW@;`k`Lm~2x2fzMC4z>Ed4nfspV+%k0jFob42jK zHk>29XfT~9GoXXG9+#!Z1-v_l;QJ)^(9?b(7H1@``cF-MLIMh5NgG& z+n8v@ft^gHJnGK8cZb<`GdUMi<5T5tE`CBFk<(L9P|y^qyy}J59SVYGC4j0B?Qkk_a8uu8p-PvoGx;#WHiX*TJm0k2`etw`P!rL z>7<;|*OuW;fi8E#9$9~b1|MAXWN6`;Js*&i{3`di`>nx*5CMTTg5E5ynK z_pSW3j>M?Tij>!!b-nNIgFXlXzX$zN?~A4b-~F;D%$8TMl!CuW`YhqrTg`EAvAI)g zufl37%>wq!+ZQHKfhMMyMzgQ1p@iSXqpi?DzPmy5R29U$@IzKx24qK&&a&Rs@(?$Q zi*P0s1-{a0QEv+Yia;}R{U>$_K($^Zlv_rZrYzc>&a1pW&ZIUgD$G3+-IQ%jX?{hKJv-2 zmwtWU+(asSL9fyzcPxG}FQMW)|KJGjudxIDwF0>Ai|p2G7I0vah&tCtz*K?q4$OTF zc)GFqnZL5KlDQFZ)vdDWgZlz{SoeocE}zQ)BC2=TXU4SuqKo7lg4z5Mtq57$k2Kcd zP}8cYlGEF7zr%+kuu!!}Zj<%H0}Zxkx=02<7r_WF$25??L<+V-KZZ7gc7c>ipu(fg zAZyPdlS-grLtqLnhUO8EGfpy&0-P}LZW`c(@$|dF=P6tY1>>T1C2UK9pjm0W>H8<) z>uA8K$t-p>Tr?tbfLLMyFe8sJyGmei#EW1E`m7;X4XAerZtFrHb{sIC;Cs^n_G3sW z+)f0>lYCRe_IC`D@C&rZSAZu5he7fd?1`HNB8?HWsooasRYBm%0ux?Tf+HY>tIzNj_ToY+(P4bhx& z{IqvP+|=l+N`*X8zpNO)7YOLRPnXu6{ehB&;Gsx;bx+JhS#B?EwX$6A@{1A#F0j>z zq^S>D8vIjWQ$AM}l(c(Q`iJ=F0wz9YQWk;6#^zt~dib(FEmbFDSdf3_r1 zo=*RLQiLL%_Zhw|!a?bNrg^j#C_wf%&k)CYa zxdK^-IVD^yp99<0@+AQDF$FwrodiH1QOap2uCb&yMQsH3U>|hL+in@6gKLxB@QVl$ z?Cdk^r#i(%!~8iZ4;29DV+L-#fC~$*Afk{aJ(w=i@x}Zwa83TZvcrv*;ps%FSR16) zo#esg3KhQ@fmQG+b+Srpm*JuIuYmv>KeCjh@pK|zQcpiY1jzh|0GWRv^)E92Vfo)=ei@0JN?kCS|7gk8P17-JVahu7Wv-m7$>8iV z-B-r5OgB&FSRV(w?wC^in}iebyn53wYzQ47b*9d}VFNJpQADSziv{<{)zHhT;KQF~ zYtM29D3-A&qEj$kQrik9y4X zB7aR>OJ`&~Iv}^ip7prXL?j!Tnp@ecb&j&LSHqBVeQ%F3)8$V;cb z5&15pfik{`*%@&emvVN*8ZDLQ%9XOR*1dUOSx1LOQg%De%BjskK)kS*NL6N0Y?BGI z)usomsLdK(r`=lIne+h&R}qs)Sq6Yt$4!HfcRuYAaNwM?^_?*t?YOv9B}tuJev zwXp7nInFBX1j4n9zIMNi&*0~!fe>lSP7hN4{E!or3esw2Q2bp(P#%esTrW}`XDUr$ z`_Y^OEmE3IUr0BG_}qLY1%1@Ba)E^174s-_3Hy-5ZeCGS_NGb`1<{n5@)P98p3Rb% zLpEgP6D-azU%*Y&PwuXVCKA&rA^rNZas6X>gaYrPDGxu@F#UZ)lsw~SU~D*#0S7*& zvQH7D%ZDEoe(%1j>b8XBS12ncHAtTG%zzx;D!+j>ZA>Wk=GbeVdregVH~7qete{PU zr|iW)D2)P$lGn;&Zf%#sQhz3{7xe$0xUNTsnQ=lImO&zeE}U_3k}zQZChJ0 zMSlWsTSI}jt+Rj^W-TG$ZEFJXwsng$?~(oN-`m#ae{EYE@BZ1gW`5qbPWrQLT{ntU zF}eis`5g$BoL!DVpjNtI3^ZvovdaK(Lz~=`i3&DJTmqmC$vHb)Xj0XBy&viDa zOAEk1i`ecqa9`s;r@V9lI3sT8f|y32VPKo==u-Wb?P!o^=Il#p!gS6$KP2B^=eAM| zs+BALT^q2WW~Kn%Q2U0P`ZM7ZOtMftHh4pgk@pr8*ihrYGINFk(v$q>dqQ5=xHZ1s z+?X}U&yBppiZ%I%d1)bUYg;A5JnAAx0$@X}dV&XMcTD`1r6+q_XD&!-6kf)`D*l~g z9YtMblT`iy^T^hYRZiz*JLiFkH)$t7^A5^cpf}~-CI0&Q{r2;Iw=mQn{^Rf^Ihg;r zCV~v4Cqn>e3n1C-MOeB}2WAl$l^WnhUrg^0F$g{LAGg8$$1ogVLT&ZA&IWk}|FPV} z3FKqC9zme0U>2lM3QFfGm=3FG#^Gk;^JCClT^+{p4ybIsVo)-~Mcsfn;^+h<+W=Uy zsmp^U8)8?r*bO$%Tx-#9nKF|b<~afV$vq_3mtCb?K%q^~%KGb`rtgk0OzO;l7%**p z-U;FLHvkfQW!C0)=ibLg@*e_Z->=JR05nn@z2Zv<0Qym)8390lenV}@*5fmv-#S`B zDFKjd3;&jE0z``1?z7eUKN#8^0gLXj8izti0FE5CamUiSz2O zhK^8DVYo)5+b}TIbhQi%XjmrMGUVh5M|r|~liJA=hfptzW4st#3e|XTT+Vad&U4ys z(sf?IEweS+nobXOz(A!ThZn`>puk3KIYC`8tu{TCoDhHZF z4f`x-!H0H|&~M%97+E|mi9fy{SvZ%6t4vFb61F-ii+4r)O^i*`AOBsn`y%d%yav1d zZ|Y)v0xGk-H+e=6R>-iOxaCpNb&F14Wrb#(OqTEZ-DoKascw9Cdk8P7w9W26*V~TD z!|bWZh2Mi@O!A4$%ozsDJYyI*uuzQTLp3X^Ej0Ra`ZES{%$2Yi8;%e?t_E)SUZ>}2wK+Zan`N&X}6XPqqrydnF629EO(iqO?Ke=Ui|wSKZ~V4 z2t@dF5*IXVP_5RuC#DQm_ot@y*S32Q@ZZf>=a#ldkiyAK`>067@{oy|7$+?{3%}rO z*s>qU+`jN+N^*mIt!q9WEIL6$Td#*5LXCgwyxf&_9-yJsee-3j_X*Y+;>KO^OU9Cm zjEvf7Oq?#Uo8g>`MATa>yf}FmgQ#SBYrb>|$TF-DdQdQtd@dtGneIIr|UI}&8P8tF4MN1pmT%nXHym$sXQ1p^h` zfXrkTY7(#OSv-fOg5PnvQ0p;3kY%~-gp5u4jiJr(ymrlvG5%Ghy7jtI6?9yiQ6&Ej zzCeu)e-5anEZXE<=Q_(>or2Kl&DlYajuhGzwhmVAXpt%$Pqwv<2k&+osg53Gc8)G? z9(+!cF#hdqiOdj@UHlzYm29vu3IBN&ykfFnk5ybk@JA#1`8~*vpeLcQQ zt5H4z`}c3}YX&b$_O@_zhV&Q)>m=Kv@Ic+&3izAJE15l`>v00vNOnEsxiM=+3)^EW zZ`3{JT0F;uvltbM0gWG;w-h#K1s^BOt*ov{K`sLMej${122+6{M%;e!*LGBbn=~an z4%w}1-wYn$XoS(2$UmN&Dk{&N8Y4pPAOiiA2rVZC_a4*!2X2f?=|++R=sShZ>SK{2$avZWGj zyVF@Hy2T^|gUdMuC*g{)V7TE29{hNca*?=h4NPae4O_EQXP<9I zX@eMZLXX3aBk<4!260cL$$K%Bi@kg132SR+(oP&3cg=;#n~QZHW$~om`y^UdYXJ*J zDm1f;A)p09&oHCcdD*miDMK!c?(eeat$JB3Y)@j9R^;ZSnhpD`5sf~%*8l9OISxG! zI5!;lJi=je*(N8k-F~SZ^*#J|5~nAP8ZQfoQ4-$jw-Qw+^OWFLmLkzZ?*Z41?u3ol zGQ_yecs#Vj3Wq?@Hn}IGkc8@m3|CeU4rllaap>BDZX7Xv zq}5XxU@C!if?rDNEuU-29Ro@$a<6Cxszo?{=Zs7Zp=}UW= zwfmMC{X@rW_E(uM!3x=Ljymnk;XBExze^~_x$0Y&Y1Qkz6>hTJS&xPYefbEq4ZV51 z&b-tv?l@_0+X%u}I9*WhHcug}K|8-i;2N(4N9~h^&!JyY5&Mo~F*X~})6FG_LxvmB z=xIqgsvW0G)|-YFO*U&Gx@S?TFou`hA$))uvtUeA3LA$M*J0xfQdUEh+RRLy9Sex* z*K2h6(cXI6IOk7vZ7CGuPZZM9kUP{L*#wQGPkHrUdl_`Y(Je0v7^Bsh{we7_y1-{CWfi z_zjF04CMR@)E&Ci{D3N&5@vdk9U@A)jWHrJC{@EyWx*IL(5OT0Gd5mGrT$i29^qyavJYYdlZJXYMFa&SO1TebDzRl+khWV~4rtDzIEiBylcg=p!NnUUg75M~V z&PKnR78x3u64@LX6X}B926b6X8|3NUc~}G;-T>Xt9e6E|6%D;J2r4CeWsvqU;*>+0 zjcyONZd{d}j53ZKpO>A6@?9W4CZ6T7`dL=7PRh5VFM=BWu2F@&1e5jOb%bnOPo-Q7 z+w1IRxCYU}stI^~)ZZ&5n}$13A00DzoxMB=vp#!gmCy02s4*2}qWEf7BE=SkHOrU7 z%Jqo6zE0RvcvzU3KOCf+>#4S8g*54W&T&h!TrD6uTe%w=G2VArk=teRB}}ifLZ^#p z(Ae(NOSy@hH|QaJy}(?^ECbSfkzuv9?~HS$01k0och1oPY|9(g)ZEXSJtNgH0fXf4 zU|}x?#6hS=94$(uA3BQ(QNaGemMAimYKEgep4fWUB?gV)i*u1TVas|ovN`> zJ-MO1Y_nI8<1CpGCeONz;ZHpGtzV;-kn<%6BL-~WKX^_**rU?1a-N_6Xk}&~RV~jk)_{$AD3!svNj}CeAO65&%AbZ z!S;&rSM1L|8~E_ED$xED zGzf6`wX?7)pGV%f<8Gi)qbJYRvg7vsfWYY{`jt;7^BJsUxL=-gYFXp-yVchmc=MVW zUL|kbyvS2co&@6RKygW*AVZHK5#vJL+SWQ zqIb&-J96Xdx5Q>+TFd(&9NOkP*VchI)gXyvApspB>y`K@u~H}o3JOGHL)ASrOFAPO zHibvsv=B|Cm{}Zuv0$zh@l74Ka$d%Ok8fcgabQKv`<@qk1iR$|9>jqoSP4{HP!56%?A71Z$N@mB9rowgv3`(p-P8+JPn|bg_aT=aCB0?8UEZOA7?hL; z5ecrHY~tS!_LbydjJ0+rIyIF|G$m;;HUf3;i- zw9s*<3o4R}SZtQW`X0k6W8iV9TVdVGdELz}Ew|RP%=z|e7Pn~B<5RoFape2)AH z7F+5^NzOZ?aqG_mT#kDQD?(SVx|7>4-WOcBtnj`IOQS%8g50g9hG!CkeW4rHY>u#- zWHCq8Mn6&CG|3=3bMm53hCt=@CV4TcK0J1_&N3S|Izi1;hD7Cws6OKVzWE8ZQHhOr{gEKZQHhO+v;>| zc9QOsx7RxR?7i3b{iy4zAN8ZE#=J4^Ip&z3ApS1^|JIfL1&}Wf4h>kH^lvYn?KRk9 zXJJLG*)f6+gz~&SxFyUqXp%?~5ua^W_WPNdVYlpg`Qq@9Q#gLVuM_T9Whkp+Rz?|K zcAIZ+xz3(fm#@DM&)L6duf^9B4#w3}8cUC4!msGbzHPa#@Y?DZ$QsplD=$=XV~72g^QLw}xg*D9%6QeQS~ewbQ8j=nWK) z8Ae$lp)&zTCFGXu`;)vZC{s6YE9tItAjNt{m0n6BNxId0;d6DfBh2+4yM7X(_ihTFCzAJAD8Z2J{1*yrp}2<7^5aH`hm}@sTL0Vi zSmkJxu~{aNcgK8|_bu(UMs{6!fZ!f1lcX|rB(=^-Go8X|AUAX)Vs^I?>`LL{y>^qA z-9f7=Sk_7zMc0@`sen?kQkg+i)-q~IgeYQz-C_?vK`LCnwThnp0CQ!%#MRpOis-%&eGCK%1lTuhidnajP(umjeP-~q7j$| zOaTA_8K_dujCEg3v9QR3 zCu`_HLSkrVZ%4wIDz^-c4Rlrf*D)8EP_ddv*frFMN}z#_K_nw1-vOvgz9auqx!Pn> zQhB&CFc4k)W>?->gR~pN-MTS~K!NGzMB}icg7ZSz_ zVhA5_05K%&7*3klOdo&@e#$5+UqjDY@6OuJLnlXD{8HMm5)vrf(B(E!r$un_K}nyI z$(!l5wSib%t)st!6VKm$Dy44A2)KMP4={Z2d#2Os>YT@9Hp#3(3pFgI4M5#_l8FDB zHkkMI&CnpyhaIKJz4d&1E+9s;)RXii=i`SV#n4new9Zv~>+MByT9eN#s!#qhjOtQ@ zTkxQ;{G}xNTpMttx8wRzHc?tXqZun(AtUB1w4J^Pb+XZSB&i>6LCxD&<$jxI=A26u znP?CRdCFl4dElt+-e>^E!-onbdPxqw@*d)KPXW&lm<>24m@VXK20>O?XpRrUSNlUI zY}6pn;$LAdKd=X%fhVN}h>Guj3u`NmUia)jCqQ0Mr2h?G>VKV*{eu_qzv1<7a+$D6 z4#h>Ai$`kR8WQLQV^ z^VRIw`zO0O-OrE1Q;0qyGnc6Iv^mCdW2sRW3~U$dZ)uQ9Cv0(II|5KnWUyFF^PvQ! zqNLqGvJnLmizxezSdY5C-)IOD5C0n`XfB zPT}xUQW>U0Y%l`Z$7AYDo@L1R_$5ng5H zI;n&BXjby^Q;YZhsjG7gb&M6C0h#W^iI)lXcPe@7pdbkQM6%^Sm$LsqAesLKWU!L< zKS2I2DW2S@SSh|*O|d0qbFY#ZA|;_i913hb#p!6?uHG)c$>oV_3xc}65D#yxl#qD* zF*Wr)-Pvr$!wcvOQs;*akqBFYW`bsdF4J0w4ucNJh@j|DY5KO~56u-Aw-hH98-na& zjRh2~4PgXWVu?!)f-3=dJ?%QD=-UjPe=e<2rvy{N0f@oh$XeSCJ6dGZesGPXpb@0j zhTMdV3>LuGZoreKV{WSI7!4rQBmXJHE^M^vG-Ow; zgtjASV21Up71DLL+M{Hx4%AYJW1%tV&oGdgg7oS-LR#{b2DE&G*-;dVe2DD#R+rL4 z@u=*p^Ps@s#luO6ydElfm)WFiMd(5_`x_2qcRESpe$2TE2BUnn84Ea|JTJB>k_fvg zInI|$b^+(c>Ni3?c^P0e-!smL#$-lP-lz8bv7_f)dAI*2;sk8N&bOT>!t!BRmy;)> z61w{KojB?J0&L?Nx$~o?jPP^;OAQ9Q#0MPg>p!ak2-vrzh)*bc{u4@o=fAg=|BOkn z&oP-w$Aa`ZcP2>7<1(TJ$KGc!0b;?mEIf+(z7VQJ36Zi@Q>pR{5%qxvEft$P7?d4` zld++^SEv<;7Aq~Y^20i28!9H(-PHShY#ZzA?r!h?2C0k52;_viN5H^)LBPPLfzJz5 z#Pk4kQsmhb9~1=T3o=Bxrwz3ws}q>rE(oJ966_cOrv{luutuz@r=5NK!N!*@9guP& zC4dHFt7fjZnkGBV zy}BA#0?T8}s)l+jLy33~@<|HbheG>#Wn^%|v)4dHcTUHosfvHFOZ38O#lEKwDvT3_ z5=FNIqiUJ#>EQ^{W6;h4k?U+yMQGh!dA3cYfwc1nVrRC+%1*#PG|w8**UrghDT=<- zk`S>o7n2p5YF0@xn?x^oZmtu=AG5fTm^wKAkgOddywme#YkUi_W6>$L+LU)0Znz>x zmO0h6zeNqNac1m~db{4jL2$0%_1*(g2ib}Eu}UPWagh?_j1K0FbKadZ^Li@g&G#n5Op+kHNd}@7+1`*D3^ri^-8g0pba(iOLZ-}nUfxo zus?INV_rSazLERUqT$JKrnxa38m#nZMS?@>kn9g9xWATGHW%=Vq|1?LzOKP3KyIB zWwDjuUVYn)FsS3TGdE(bLVv6gU3h-O7xsRRKV0Y_P>m_wPoyaS6RE#R z$$x0Ue=Umtf+abA`ZGy~H2Oyxt~lBLT`YQtz%MX*WGHYb@aqZ4gI|U_jp;mDF=;X0 zgJ_gJ;jI8sIO3i#b}20*8CH1b_{_|e{V{*%{q5@$f&nHjiW(^zNtL8VDxr~_Ab0~P zNQ6*LmH-ANP?EeZ3?f9RT<03Mqu3f4VywA!BQv@()3!b51qC>>Z{7AUZKbKnKL2<_ zLY4Y~3t8`Sl`wkwwS}vyR}zRCHc4O%J(^@xAsv382mVLuDZm zCzX8lQ8`)Ak_&8gk983-=FV=`%RmA`ZTs4xXY~pT(9U3)*jZwi9-FUXDs>8s*%ZOj zB32=(&#-fkk)~ufPaamt(&Th2gFvn`sCoG#FB>T&21&^^9-m5nj%W#R40>U3Frgvl z1c_Lh*s}rbt6>Rx5{4S)tY{#;WsRfcy!bcv-jENd|Mpk@SiiQ4?3aeFjY6vIBI3## zz)l$xi~E-!$e^3XPYaMQs9%cy6QO?+sQ(w7e_>Os3FEG#ivIhW#$)`*)OOJ=qp0Yz zNR}*7rcq9ky+cSUWx<6}#X{dqFCjlMogp2@R#Zg9VYf9Rph2`rK>HiquojXQ#CNn| zcnB3V1q?+2(wzI9NfQaZjBwwtyw5$(e;RW;P23-<-{e8dka0>F z*a>OG;q3_&Nh0wi5v27JC_g0%L{K70>qStcN%s>(%1J&Q=GsXnNEGQ{T1lu#_Txp0 zNh+B31Q@`akqqx7$#Tu^ILY)4oP;5d4;lbyMnK32B&WWJN&LFQI5121Rxp&E;Ug)i zh5FR-Wf{JaqeBD}!aV^tPhffw{#fv?KTnxV&(P5%LPLFQ_&`x&c&=oWWY)MKnL4A# zXCGvySBAt`hF6M2daki?Fa)%`O@9xWdziI z)5sV+p@fWvLoH2J5m1F=tY~cDyGi_XBi^0n(LRCZ5i@0<}muUDiuI4aPxe z!_9#jnfycn!gpp5EsRHZ9SnGmi6d;y7pw#0vGyt}^d7QE)^`sAT&-ATP|g|y4lL(( zocFVc5cJ=2{w(VD40zW+No@}*s%x)Q&|HHBLi!cyG3z>i2?3*|R>h)CX{qf|(zMlf zywjQT+=ZpuHaAy1($8Y6Q7A~%U^knDWHlG-b8dX05!<>g-sn!~%WY!bVBT@vZFjCBn-W zu}`8$v5vb1PJ!3+#I&5aGNB3t-(l7!CY}|zQWqWEx-$psPx@79>I74*514KQ^q(5Y zE*EuERFa~3rBrT}?bKweRbGc}n8SB*t}nANG!@JS$ekyrUd(LkhtSGKH&;gMQOfRU zzBPV!o(z#sV#R3>Xp~HVFPDrd1q$S_Z*2B*TXBHGun4ZEdhuvfc zOJP^jZ$dK zn3lMSC<7ySz4rdJJn-9NdaV;&UdR_JW+o*Uy98S>Rlf4prCh~E$NN#mnW~gaC1N5a zy<{C5Xe`%4(83B~pJ@{tE{hV-WY?cXQL!n^yMKBMs&-$#H}ewg2!0(uca(~24Lu4- zir^dW)jwTXDNbpT<<4a91joCKcVDWtgRz=v8U;cNvjKk6ct@v{xQA;!sWF!$k8R;b zQ25ij$cq)AIWMg0eSj1_JO(2yfR!<0ZDQpkjYZD{GB*E#Ev`*Davsa-<#m@Jm3RLJ z1C&rL*pTobk_3|ADF+yOikiL^Qt=XNEmO0IQ?&SP6b}hkslbnS7TK2VECbIH!?TzPdUSjgzT#}0<>*E#H(5cCLrUF>g3OX&G0@4QDnjLafc z+frksSEBJ2t?K!{SBl;J{((#Bip!gLrz_VAcNXo~2AxvZ(|dj$%s526@pHg0 z;ue>Yi33e#hE&SASej&Ry(^M@woq+kZhn(%AT{A&z_m#5st7RGsrM~Z{W z@s`Vlts$(G1KoxP>5K;({5zJ4N7(5i+m+iOQ3c)2oXr<fAK3$S#YWHW z6Ex)OfQLf}mF4P{=T<(hH5q3ip2dwlIWcRJNV)oov%2e=vk{&asm~%p8S%I6H5{}% zn0d641UU0%Bt)XD$KNn9#x%lMbOw99zqEw_`R2rOZ@WjZ&TpF7Je%#m-boy&LBD79 zL|CG+Y-~Rp%_C<(zb9B^jOYO6=J@vsz&gTYwA`n4-M?bsXGZ+^$|$o$$7cmF)YOuv zY&Y-d)F%4zPL6Sjo2ig3C1HBz4}xKC2)PYVVhujk?qDDmvq)Fnpx7bM`@5*MxODNT zGG_!nMwC%11aVmgPD3ydZ84a#R83;YCoEXfQ>dx`cwUF1ZrO!pL%wkHuqn1c){o&* z4KL)oclPThxEx{cws4oUMK6??V>v_)=q81m_FmK1>^_zf&P6%nn$h$pHrwTyCgwUg z&^u5;aYtlJhSgNJrSugb2f~sILX2YqjeX?(vk1ND2G$cwWI+~jJP6Ed-jV47M+j(4 zHb$R~t*MMODg1zmWqQ9c@0@!RtNtLq7-5PMUHeS>)iJ7rn+m|VQnL3JF+HU35&Y+o zewnJ=q*>uKGpK#wwk37l>Ph*Xno}Mv#)TY0+5+p7jsM3;eQ6D_ITrh*zaBFt-OLcV z%GklGg~UVx$rC5bIBj%Z3F?@v!;cPGY734YNI{e$YOa~~gFr;Ac#$orVX$s9Mq}E`3;EeC#w>zM z=^#8wUUwQmXp=;5J@9844g2Th4>p`=xs>J{p$RD{`{rz*+EEJHq9Q7Xig&W6ORWLN z{EvXfps9$ILyB$LOV9lETl40dZD!w@ahkT_eM^dKCF{_lLtX2Ejzv9k-vO z2qNQ#^2oI*OE9F<2)Z)XGeiN%l24&uxyf}ldSAx4YRtD{CDlkc5|+hauAJ+u=o*V5kcfU=1cZP|20S{laN<*`y3${I z3r)4IQfXL`FvYS#HLeAw?YbdjqbK2+d1_)MNZNw)i0rMh9+k5)T{AcsvW`OB;KIZm z^w>$hTe1nwOCKM4e8gijcjth^InHSyPX4+i=dmV8cTvP+lUtUXDA2mcDC~@T+HCz6 zE~O?5{A2Y{ZFl51W#XA8-QP(LFtOfonJf2VOfi^kyJJ5kj%B-e5ml07Kp>mt4Pqs) z%QME3EwpGhxN~iYy}MUC;G!gbZYONqB4^00&4mkb*f|1sO(>QD;1ET^OJkEO%N*zd^WZkP1w7~^8Ib%PQoroE>XDL-U}%4d-*V;hnlH zz@c!aEGc7Dmxj!9czC^r{g&Ij?tS^2?CD9**)HN7I=FG^M13fUwQJ$aIbK;!e1fFP^8dd4ISj@AE*v8 z*)DFHhN*@^IqtR}0LJ&nr259=YBJgQFI%*?Y=0#ICW-$crgAd~Gv}dHfeQ{DHT~8l z{CK2?JH*0NWoN(4Ob0bKd(6gyi{O7LZZqWy0n6`l<(*Kd5;SDlc}7VmiWOTv`?Y5m z3f+qI1=x?whJY6L#R*2(IW7Gt|0Rtha6!BShl9N+fdB##!P$l@(1!LaE~nfOtCSaG zN-Lelk-VpW9eBgdl=P86555AecSWq`E7wY~+1Q$2-BeucPa~jY6(RKx8TB8Z zVqlisi11T&gh~>PoKT9RP|HVO2r{Cii~e>M09V$w*Q1SI#PXMoCTum8@1|aFdCh*z zeuC&V(|&(<5l)T(Ru+#ZccFd*B!=KuyFrDZr*fD4bRT(_>8D43m+H4idspkfMtj%m z|BCjm*q@8`uGz1T_O9B$i}tSD&yV)L2!XENp%7I0*)EPG9l(HX)G!G_-_lWsWbd{w zW8=I}b}2(=?8KQ!oz@jC$)1Rnm~Gckyen&F^V}EM>gfQ_a+ti0;W!w%4(60xCsdfSl(PQ`6$Xc1(bH~ACBo1JBW=w)U z8Llx^&!;Rv{6-XAzkd)&)f3T~exZ8Hx|M%Gk?2S^yN7{3YZx10f51%DGaYxm_xX5a zY~>39U+n=@o96I+oAfXo%`MZF%B^)ycYwAk-r=^f_nh?$bmAb@??@jiy;1RWp5g-z z;TYy4@NucWH`Q@5X~~hFBuB5r^%5J``sJ3al~&dyM=x*0_5Q%5AaCb`+UnXB+;cTc z6derlUb%q5+%P7pS*%6@xx{{~R!&xba~axCmSOnmK)NhFHhs_##;dSHlTV?r;}p$S zrYhR3VXx}-%<4(u;~!*XR4SyUQVBa;sCXcHP+t-Gs1Z8TDJ{}DZOL2iD*l)<*34xn z^>e7`P;%or@(IYDxRIA5Q=jpq5w=W&+3)$7mJ=1U_Kkpj=nYjXiVb>~1JQehGTReK zR8joc4mkTN!CYn4m~zJKL(h#p6{KKNoAm9o4yNFXmnD*@20|6P;{ka~l_@^c$E7p6FpevS{ zkVYe#%aCsaem81nE34yIc87nejXcBXFiz4#Q3U05$mUpL7x>Z>A3>87kL`&pC1&BkNwr^p?p87J~WGSnm7tZiAio#EELM=xKpoNAPG_Uo zY^BU1v6;>;&$nUhh-|av6d{>Z*Rbu#lwu&*M-?Y>lcLGVeKarc|2jiA^qj-jB0?_a zHUADI=_Ztmi4)xBUi7q*kiH4bmDda6BI(-d_G>Z+eM_EKd(A>g;ap3Th{-@D_>*Gz{eLQE4#=gk+I5 z-xG+RMG6!nOyqgWt@HV~I_RBKP)_AZ4g-NK<#JK2Ww|RiRl24AkE#+X6V(6w`;P2sWwGkJQFNBP0o@E$YcBewXneo{#Y9tb*(GZKSkS1=(#c0x)iF)aB z$mp4;sGod<@Ox-xCCN!)j3#!gdXxHis+u^Eo7yVXT7^Hi;2y_M+>82AyHK@_<@Eqb zu}OJvPrhZI$PI-)n0U$mRLG(uE-!^FzNe?{ZF2w8DO|3olQZC{!EJ)#TW0e^XgTwLlY{tTYNo&HZE*#XI?UD#@{E z`rHY-Q1iMy-2)0guA9wNriznZ`dL&7m#OEM_1WWD$HoX3P}Wtrdqr2i_jEQ3#1dfn zt0H;(v8pUqsfM0_6(Kzr4UCDawqc%>7WiiAbYo!E>j$pLSm~&X#MbPEEIgeP-Nj+rNyHM&e#K` z+{M&;Bs32!YwsCHiYLRR*DQHrb3|D08RD_qA5jHHSmw-&c8r%(0EEHm1!4{$qQp8w ziaWYTr|ZD}(8FK|x!u>$!5;Y>>{~M816v3AZx-$bCP>4&Oh&oaG*lZRt%i+H2HWG{ zx`E8N(^X0c$h0Ov*lD{hV^PQQV`2f1*oHymGSgj!n4?Z|s~EhXVcU$j?rJ`aF>gW)+t5|~Cuv&c@5 znft+TxssO6(Ry){6=ADZyfiYU5Vz@!sfNFF6c)OGFPI!qlB(WWKalVZU{@ctw_&^p zi;!P7>5D`iNNm9oP>^2o9Rp1Yg^!1NP^`{V?GHW0(H~E(IZ`#4&H!{`4OZA*=-;$m zvjM4^8QJ}cDg=W5&^};%jGg^4d1666?0mEXE0J~AXy?`0(cO5pd))yVI^P0*<9=<* zi=-*D{VePSUh$`Y%ffq(+B*rfKoPdj{@R4Ze!z!n<*XQegla&cF~73>X_FDJ1?n1f z%G=MYS#p&@7CXy~unO`LQvAp_t73j5;|3YKjLoy2-EZs)Deq=}qmjK&THD!w>T1po zxq1{{li|1|N0+4pr6~sYyVTSjkh-?#?4E7hHWnfb9J{{rKq#=nLS(j(>Q5rpg^yTu zrEiIkJ8)$?Vk=v5uCFAVt@L#M>bIJCt~$8W)g7V^H=YmCC71;TA0wlOWZR#HKAEwI^)md}QK>UlM^yhh$id4)GK ztB-7Ohj0xuRpx)+3Q2IX3S=Gf0ByyTyT3u>qX-46Wy>i^#@}#Vi>Wo zP*M<-nOk(>_8&G)y^zD@cmujW4hW)#AN0F33r4g5!9qO9%=Z^T6{WxKPJh0A>t?rq z^6m0MG(f2dFMt^pSr-`|TE+voP|jD2$P)m3Jc{Nt1#r3{OdJLVzQ}2{0iz-@#2j`8 z&;i<5qP@W=Ga6WfmLb25%3DUXx8U8|4t6SQVdUkQiz17~wfM}iXj5{QP-3CB9t*zZ zec^}Qek?h96liPY(d@&k8R5 zY`B+JGu&6^^nDs}*xR|A4rp{~To=14=F4a`lGAoxnKdIi-d*bKU8rk(l-4K_9@MY% zW4voEiAAcnemw5}2O&1`Qp`Msp5rUzvs&C5#t zBgd#aQ)Z}$V?h2L{qFPO_Ma}{dgjx(B73a=U+vi3;OL8zkCT3qh2 z6xU@G%6rZbRXm9;Fo*0p+Ssus)j~XHg`&LF#Y#V{w zx6YQY*`YYSIfHwYeMd(Vg73odqje0qNSN1+12*AC?^GvvS)5o|1ClQvF4vMpf6Q6Z z8T%dPpTHEDtJBo_WQ9`yUn2F?`)N3`e^QmH)h91^`Zin?+%+~2Ftb*d+FrL z_(>Ai5Q9DKPT8Q4IlUTG=RFIjHuGXpQMqWE0+ui^u-A}%Ffu%VU*P>Wy51}5P@-{J z1GV#QSMKb6&ri~?zTG~Edl)qq8Vhv=LcXtGxttgPNG`(eC{kop6qy=*qNu7dt%zg@fz8Uv!mJK^6BOV@WA z$1v*J4WAufksR1>3+GN_6Pk`?)~#D%Hq_pB3RgZ@1M2akjZ*D@QFeD_Gdz!D;F>sz zroyM4lg~n<$6nVf#x)Tg+cL3-8wI%b-$G_0Zgfe2}6& zwsy{=h=nfs-3CM2u)>v5Fcu?WYQNxPOb|H>ia0mGT3p9*2f9&#;j>Lu8FE@HP_^fh zD_Q78gwe=~{A4m~oWVh31W?f8Z;t-vx%)`{2_qg*7nUC7AiSp1Wzri^k+o*8AOq75 z1VYn0aDE^PoUSZWI*S)AD6ZNJZ*JRCgrfRah{zVGz8hB!hTpwhV^Gfpy&=Q;1PJCD z&STQdsjHmgT0Q7msFkE@PPZ(s&MIgC;U6EijC0_+NS?Fb;QxsOqsaST%bzC%egBcK z{(lqH83Ek4zlm!VLn9kg5(z_R3so0O8|Qx|v7-~T<#7~|Mt@SY!8*-rRdjpin@Y;C zpHj=qg2hAHDsAvBK{>5uk7YD=wqbg{TKL?6z2s-ir1PRIAb}yiAM-Q+NoNk~{8U$f z?BTCrTxj3u^mS}s!BMPTN()6YNa&Oo9g0ciXZ}(Io`7pojlyO#r^|F0;`}+Y5axe= z+}s66yX2c3ecIESDJnE^|KW|Jzvve;1Jp^-UufSrTd^AKIXBn6ySMQwu5_`Mouu!$ zNx=t|8l;HSy?BL%ND`+?(39`5L{q*Gz+t@&VLEaQ{cX>astgaJ$}X0E0P%Z|lm!tT zaywH7SRvBag5miibga;960yXQrLgQIl;Ag-!k<43cR~5k zd~V!(TFMJ+%WJp6!U)MhCJlpMj{7*$dAIOSqRhkBCeM+c*7d+&om>4dY8Zs$bgoaRZe>r=qnPs<_}ccbdXEe6spg#Gp+%lRY@sF@q&sVC&HQfj_|1dnnF&I^aU`2rgv8<0R62 z|9NOoZ(>y8YC=JWY_LaRH6zCG`|pZe$Iz8@O!ZRsZRXJaE794+}gg@-`gO5{4T^Hzh>en}1zOqT8I(>cRMqGuGi+1lKhqR2r(7^o@+Wo5Ch zdSBK+S|47iB>H5Xxu5JWF!=hCu`Ss+FUfH#6KBiqeBaYdyU4DZ<4jZ<7X5hqKmPWK z)$y}=?|)0^>)6@7*gt>p+voK^RmZ6}(|><3Y-(K(90CY&YHtr*6b^+hzgqm~V0-+$ z{@3$=rTz-8NGPdO6iBeCHoDNzLd1}Ee>K$zZjf?fkgo8uxd%r>St1`t?^#((M+sB! zJRe&N2}4C$Dn7GaNlOt+Nqb8|)?$#z;Irh++^UFGk*V^w(D;fLIK+WGA*W@>U$O;) zK9G{VNwL$1XB=E@KBd@zD1kBX2Ml7kIa&bCHw-Y`I1?iSh`*X@KW)5svql$Av6^B1fAbOc}|D}$OJGSk+gKwGQAniPC3T6&gk5@rn zJwNZ}o*`mQu-Xne?tb++Y}P+O3E?wiHU9#FIX1P?gJEYgm}pIk|DwTz4mnUINz$_- zvjG&>0ET3FuHcpzIV7twe(AYMOmy||_Fn680G@QQsOh+r>^J@0^>zm5-UnGw{1p;9 zQ;AL}uO-XEkI&tCV3(7c)TMH%e#ma#I+S@-{?QK;ZitJY1bA%-dBzWp5sOR2S4e3t zKukeIaZYI}CDp@tan{Ly%@=QgDZggrOrQ2&%*Eva#G7K2w-~ww8)k8wjQ9TS>0*7`#`oj* z6KWsHiU>`ZJK;fbm^x94$dCw4=bk_(oE(o^K14PO_zNjZeq^SdohOMRK zm^*MNT3?J_tPN%&tRdyJlM+ptWHzNEt$r9fGq#x+1V=!jJp@N!<~tkSmQB)S zrd4>l2VJ!FwPsQvbN@@dmJ+%yYfMu15hE~PF%myEtzU+=xGHlUQAhqwc2ikz7>C-* zvjgYkeLKkUvC5k^r?c?IETrK3BAEMjo?6Vl&wrq$`Xd)YKIOhJ=M{WmA3@(DAiDeNw*@P^dEK#RI<=4I+x zqpv-4J>}@3N|4}AcPC9{?Hfj>Og5cm6T_d27wnt$jMta=FxEX>!o%MM~ z8bpbw7iN5MxEV|JwbF}#_%5vTq}~UR$|mjLH>&jx@aKeK#5?{|1_#uE2`YAYE z(&on?&yEjxHA9?Qs54)Y5b*cAVYkrTdkzGNRIZD+4@JL@Sm}w4g(hBIDY(BmMDj#C zDi-M--FM|zhvGQ9g4D8d+r*Me=Z@jv=8N2_%AbHf3+)mT=4PgsyZ7|C)nRXIwC-8%GzS2smS6K9Z@aOTjNz3BLTuX z&S1G-03|z{_9bMC127Mar^XV-|<{KP3R{HuW zTkAkC?{vW$>}ZtT2g&czxJMGLIEt_~Z}rxkOSi$YiM5ekfOUp%H?*A!PLuwcKwC7x!diu}$rkF4mFGAr2DaFkb3mK-C)eS7w`?v&(A@Yv~ zH5)g)gAdw$Q$SD2!Fe%q>^oZ#%Z&ZF0~o4BcA88`wGw|QSznPUrfedkEPoTJD9@J# zYBO9w>M5J(b<|VRZ!p;E7<`gm;5UrzgqDF`&IU*1e!WASY-58^2^1sPbg~0fLeh=M zUkFOMj=2=!U=OF_O*<0{Y$*D%SK;^%?_xAA;hiWxdg)*XsU5=U9pfyYK9;vI&b#T)lo$oJQpZmu9}){RE#(y=5TYjYd#IBwN#UY~$Qd8r z-e?3LOkFNi6q1RL4-VBy6+{QK^^=bH0+=G zo)`Epe;xmYfRxZ{^C%GlINWU^oxFpRGf-AT%?SZAN?RJ+u-{22n6m%chRG0Se0Cj| zyqe19be?H<=I`C%|B_{-KavtVIV0AutY~;qB<~272pcwVL@)dt}>3! z)2P7`o@|occCyXX;&aCLL3%M81(tXc8@Q!6(|phtF)Jh`O~NGzO|+GgNMD+^3 ztJjmyyU$l2vre7T_rxZnkiH@lXl#uVy6pO=NBg)i<0)(alI+1>?_18u27Xosrhw$7l>Vz5xL#Qvt z23Ie~M%gbEZjeL138(FQhtM;5hwn3SOF1A6s$wl>@x#$>4gaSw1bGe!vxLO zM(VQrXOWD;7(}p*0LB1r>=Eo4(--hTa%{d~2ZF(104Af(l!GO*ijH28k$0kDYXr4p zZ$u?S`GM7Y+`c$f&Hz3Dkpa)_B{_osFctt*x~0ic0p8+0mAezPmcKw&OG9y-RP|^P!udd#PMX)NR4rSpj4tSB)FF_?hlUJsHwHdOa-1AxKHQZJ(paLR!mL>{S49U<0Ik^egseL z>zH<{P@g@Lzws5;;Jttu+R7k`uSJ6RupF%xTu5fiEU>=)-sPe+Mf!H9geK-C?N2OBRk;8arD zolCT{6%&XuH8c_x?!aKvp+8T>zDPh}u9f9zOT_;+KC?xF%BY2T|H?bErQXfnPn;si zwM&TL;@OKlF17B*NR-)rIy|D$cayJ5zzkfUip@y6HI?1HT_92*Bu)rWCi!T8kcghL zlt{OmJaccCy51qWCSc-`y<}tREhjV{mqKJZwVWPfchc=SizF2iSD_=LjgK?YF6V>F ziRU0UC4Yk2@{G|g`?y)G-(v`sF&hZUg5Nu6sgQ<7ZyTOVRl5JY~85P&c^Wb-3G7F zo%Dly#3-%M(|;t0+HiYv>-f=^76p4w?8q{?cLeiBbajEThwo$_0vvr{K~G;`7Y!GG zWf06J&@Oh7G5>t$VbW*TIF{Oq*T6z;o;Z8`P0ojodQ7Ca_RkQrCp(7) zPG(?zo%Zg@At$t{;O1l^y zbv?U{U?F+~A0Wc2|4qo>1xlDiZD?BHCdfgfur=Z(^HgQhVRPaMVYaRxHWc_ML4kD> zXVKF`T=E(n>X-q_i*|CDZ@^LEbuy3p;N|IRpA^#FUw8bxxO5`8x}`mSrlw1rke&HL zno$sw9U7fGP>Up%_W5=~0>$TZ%t2z^B za)B-+387q5bVgx9F>!Pn)(IG~8KkV%E0-qEfkN6Fd5Ab7%Pr%J7G)<-BOv$bAh`lYE4k0iV!oDtdx~90bP3T0`?qzY991adLEw#jM`QeZesL4?1p)J zz*zvx@&85+&nZ@ke7`7csqwQR|CxkX9d%yp5#HBF1yO`JCS?V|-W(WNx;($J!pL(` z((HosdXhFH9-1B-gZ+9ECeqAMUOVOBYQVgH1?5uBzu`|Y=LYywHdjqti1wPy(|3~d zkb}UZm5FB!_VM$3(nMT#zpJ1bEu-&%0}0N`);L!R!C2B@keet1X~@iy4s5(1HD-oz zVb%901tYNs{G*C@$>*cPcwPJNX-ER!g|J@wL;o0`Z71 z*Rq5cN$G)L3ZOFD&ki930!xghK+^&lSbhy`;U=Q{{*u zmJD^eQDVi4>qbEb`)@&%#F*gRcMSi~;l#_9dXL;ZsKlJe$_ZmHzB6|3Mq(rt0@O6} zjM!}IZpiWcwDz#=UK`_Y?fMs&YDBStfPg;x6I+;Q?J`<7xr0E^4ph3D{a~kj9R%0Apu))N&+@@bFrLyF5xmq`181I`I zXW0)u)~juj9s03*7MR}=DZ^7Z<=DRm%wSm)BnXQ`07Fj1yG? zdPmI?PEt2Nmo1a~-oZ1ZwbrtJ3Xw#T@F`VZ9~@&_*3kPwJwFq@WlbwBnyWPEd|W04vP9>v z^7IgJ2m^$~^dr2MM^+WM(k347-?wZMFw+zpE`?psnz$rwVeDk36;!4xY6mZb>k!nwicEuJ5cCB=C6Yg! zxqAXN0JfxJJR^$Hx=GT9FVx-B!KM**U>IyHqJYQ%rxwi-e${jg`@En%Szk8n$}L$C zUO%^-4{)6C$|-n4<9XI!j|ut~`eol3XC;mAUo3y?;S~SuVUZ&KE^mOXTxs(YP_rFN z^|Pt)QSA4nAaad3M0#y0(VQ6hhm=+=!Bzlp@&B;(jp22+-L`Go*tTsqwrx9&Z8gbi z(qzRp8{2kc+qP{tIjeoo-rwHmJ^8hM=6bGi&pF1#m=-;mVgnEgf`0WDM=NNZEN^;a zw;t^&s_YC0a>-}AS@Kz+a$@_pLj8SFo^z(*!m}m#YrS}IcO32`vtADJH_KsFj}q+L zr}5oh^b`F>n;+Kqxhyd-taVe%>Wcd{4O~^XDyoGZYQ^j#x)i66#l=2hM-`;Syc4bq zFidn zRuVxggI;`!-@G?Oe3^R35sVrlKb;*=%eM_1BQP3=q*=|2oY>bTnTa5eDxYNi3wo&X zC4V`%PcH=OKJ^9;qGD!i>dN}+`$ITVYxkN0G)WnlybE@}LAt_7A6>|el3o}{tr##m4(z(PnH$oOJf(|7t71E83o1nX z*~0w0EBk*pFk@w8z?9Gi-Uk*q<0ei`F{qmqF{O~}B(>vHV9Ui2Ew~~VaInlx@GyN! z^8%wMArD2w%qvj*$m_T&UwGcXaX$EZceH=v?2Ys<^S1@(LKm{Yfl`A1ScFqJg{2() zLV?_a!xErdyo!U9y(^37b=5CPe`&sR!2AD8Gmd2`7EG4e_Cu_eR<)obzUqc%gNMW`}0{pb~Df- zxkFO(7IK}HnAqp4iLzIz(Hw~eb~~yYqry^krB+6>IvY20$I1O|LMrw75)dNQeLijE z*ttUP`$2f#fKI}wNRnhcfz2G-6NXCVGo@lbi&AySA|6%M_cA$T4@0tRk~#GyaT9yi z1T*hFNcoXV?j4=?&Zn@vjzapXBo9-6hduP{D=_&*L&DBy%s-mV5JfCq1*F-${+DL^ zBiTT{BtkK8Xh|G$F?=XkU@&*+oe?zI0f` z&FtR175`5FLl40IjZgbW8zxjNSQD`3PEkYY!AQ_?2{avwUGs%6()JN^0iJydiHRp+ zya@^FnTjp|WiD%xFBOSAdO`d~OQhc_aT&__ldZ9IZXe6yinzPa>s9F_qe#ja= z+vgUE()S*lEVUm|5!ztju#wR$ID;1YzSF$lVPD_0QHAiCk>}#E<+H|}ok5@75wUyo z#o~U!g-^6sXd{Y(7Hq1rq21@mB8Q&=05QJ2`yVZvGDmx33R=BRr&m+#iw@K#-I32MvcpcM2L-(ecik;|OBjT!QaU|d>452)ppnG5qT zs&}}@DC_7mz0k*_X?Pd_1hWWDzZj)}qvT7{%_)+_9huQWYgwRc6@?8W{y?m!#~Ntggts zE?zC_kFH!HV*!h{{$%s!@$y;XfZC_%3}Hn4@5b>plvv^Z3rtJk3Zp*-!~9v3BZiGP zoiWp*+qq0Ig$GE1SK!h|xE17=W(Tz3+zZ9F_F8!{GQT=~R$=Pn&T>9VQJ=wx9CGG6 zg~|s}!(-+DJml(EWaCH2rL^9Z55^SMWsjV)kEeRcjockL8X%kbvU1aON%*(yN|AC+ z;RAYL2mf*cN@}Y8FDo%y50D+|Kdw1{tO35-m>T`JFF=Iyh77I{(i;V{J4N#>mX40? zZ@+XD)FWiVW08C!<)1n9qA)(Ix*A$*r+p*k7VkwliZL{c1O5QjBVCkeV`86_rIe?R zCo}i^w}&zO@2+$$9!J{=VO*Fu3>eo~BGn)KH3Sj#k+lG5yzxv&ThIl3vPS;V4?M$* znT!fhk%UX4{Q)+vcc@1B%e*L{l-`X8LAWfzBD-F}FOR(bin<6Jh#`b!IaVDiJ6RoK zVU2wnYNvuPL&2aSkvlDwE@!HG9m&*y?-LUcQo!Og;kL}OuFw^b{~kjNA65CHDV&YI z#2L>+a|TefA}8TB>#h1K@6!65&h+k3&sTakAc^gp`(U1ZTEG%8EZ9u`mAvkm+mJC5 zCukB2{L06O#Mv-{(2CmlmyjlwgRwQrTuhgGYf>+XtXtWUyG2$P&&1U*YnBta9_xF! ztS3q>mGeEd#E9-l4ddK#>&8~uZ$bVau<}Tm)vj`jpEefdvsvdaG)UIhhKzK<1hQRvQ zGoVu}>TA?abnb{_uj8m=_hglsP|-o6w`q}1_3#Sox6Lu1?0_8!X(<8toXM0U)$jzD zb1_~Ms+AvYJ3T%Rzfbil;81uyyWr37~(=~&oe)VG(+F-RRYlqfNVy-a7SXXrX;#n~e|KRF`J;1aK_B$*!o~mIxA>25TiO=L zx6L*Bk8Lno;KZ4Nu@0c5q2oJvTm*7s#VXXtY{Sk7*c6H+Zd2}ROPczq{q8@$!8|`; z0och=JO`^IDKGbL*D#xSJxHy@&&2m4cu1*A=_o)YZ+AWw1sdjI`M4{Seor4E3o%P? z8P3o*#SEsVt^D<(kgbst<%z-(U2HMHq1j%^@RKwyMO{K16#m2uG9wh9t!sJ0(5*P$ z@1M2MenvdygixdcEJHG1MJMy+lyxps!@@Cl2DB$kg@riZOw%f@l1{=h9Ih)*UIfVM z;=L)d`ScD28$Jq)$g5yr*Fo`EeCj3uo3OR3jc&MX8uNlKsP?e4;1*_*nG+%9phKh|hvmtTyfTo6%3`kG#s#mzv_)0@B-AfWm+c>AYiDpJBtfW5iwyCvp+&pUk_d zHzM~W$FM*r3E(yb_FX^+t2?$ErgQ7|5NVN*&o|GF=4&iG<;ZDAh(=uU)UXCNBn~vj z8+t5+w+}%iXxdJnY#@0fZl=&!@=1cY@9jBiZfrw8m(Z7QC=g9p-E~L zXqTkAef$Pt4f8D)1HG&(1MxIw&5TSyWZyA#)f$FYWiGw7)G{E#POF|NRR|T=8)A0!X zuJ0O1XhIHCsCh=w^T~$)#gasm@RPM(A2!+qCNRIln}L1xlO+Nf4a!1m4K?2B(dI&V z9<`|7RD6-8!nsqNruvpxSYmVrePf_t7g1YQqN)9gdKL~Hv8>f>j?0gt&gXQwDt@AA z;Qyuh^ve1GR033hCPjjf>_$KiBI=fl_WetgT;-+}l0Z-y{WnqipHTU)6MBKML-O5< zXu$UoRc+cIoU}C6A)Vb216U!r=z8B71oe_m)Y8zbq?xbEMKQ7Dygz{45VdhQLT%2m zpEKSO0HCG`{|6gGsc!!OF=#DpVk(wyER{|oTGbMmgql#cSegRnBYG8@{b<9*G0YNt zJ5AuY?Fb5BmHn*2mv*_Ex!IQ`7=FM5Nq)dWJBL{NA#p3z2jZ-9NPXds>G$m6>Rh0V zCZ|D@NpMoLX@lIucVcyf*uiWcdaPKrTKe&p@&X5(b-JEoNxg`Gq#%`##B<|O3?&0N ze4kwBebR-BSM-9yLXT|G^Iw{x;GhMpba?V@JDS}AJ&JhGRP|jHd78{k);w$(xSsmJ z(_e*RA#}zj;s(XW$%3#ft^Rc=||C)=&s#=X}f$(Jdzi9VAt&;zMs?!q* z8sMYQir_DZlrFaW`DaE1MdGqYNO^mIK(9X|=V0G=N>R>NPFR7?C>93oSjE$GvRYQ2 zmp5L64{pgYk6?bCiiTuN6lL>iRnqZFEY#ssA#!25>sF=4xu$t%nVUC@Z*EXRsPcfMOzkyGIkeyhr#bHk?a~{3{-B-B zrsr4I<^bq62&2w5zsJ9pbMA|p(?uYJA^iKT{2AOyLh?dF0+c7%{~_i5KY0v)C>s*7 zL=#gnwICG53B(zC`-KRykb|W@*0@k=o@)Og<*KO`f(DNtUlseEB@#^NHBoC>ZBTDXz4 z7y>H|pr5%iCs>CDj;=&=z@IHO6b1(DhjXF)WU`JHt%|8@587cnR#qd}C(J0oxDK!La`JIa}HihE+upDo4gO2)E zH_HhdB)@$5&A1t40zd2WT&yho_*NkXJ>!b9L+$^&vd_&J<772B0Fs8`Q$|Isr!wonGrxaOJ?)N^vj|wUvBxq2Ix;B2qB2y3YHxNfK7ga?n(e`kOkpy@|_0PDSHM zjR3O*jW0CTbcei^V6d40o8-cRyHvQ=yUc~Fp+(98n`L^hEuNRzOFV@1uXn}BoODBa z$T6Lx8`mUEX`8+d36_h576Wce2a^{A^o$PHsv;>P$(sfojq!Bax-jT)&Qqzo$Qc_H zP5DhB3e#D=U&i>Fj-^iRg4fI>jV;?ZNP+PX)Y#f>iR{bX)aDm2@5-gfGbqdq1JM)9yuQL zXgz-5k2jKU^p|AaAlu;hwUlx3bAs^$ViOA|2No6_JOdm9ym)|J4YXFh++&7AY@3vg z{eb(?Jdfl20m!?n`y$|APg^8zN5s;@6#sPK{*6QC zkp8;lk{7l+pdpD@1^>Jj`iH=M9m1?+_gGhh=aAUhoOkz~AECwx{rF0v9!g% z)Io)a!YfCvko9cTzF?;E;)wFQ%BiD`~On9YfI5+7s!%4`fN1$3!<)-ASvD4=>X7PW8C&-8&ESa>^fS52; zP{f*mgIv+n<2+TV!VZL+pZWZn;o?O&)CO&ey@m;JgsgQDwUCdt_cb(qo$!&a5mrY? z6Zk_Q@=i0p{F!D%{a?ZUZ(AJc1eW)2TO79!{cWI{UPphx4dtM7yqqjDT{O|;NmAIt z7)3DXlePscnoUrhAfkl~;-SyaTmJ6^b*s1S zbJ7j@G410E3P0c0GM+ZlI4;^fS!vhaPcz&$cRR2oN;A)J#zAwtv zk5ysf(Ta+WWx_d}#TtMG#O?+)s~NvG#ct71(p;GV8#%W+kwGnPs2*nE4xADYOC+7R zE83(fyvtCMIS-o!E{U}*0I;XWx< z8Vu=c>Xl;s6})8(#AJbEmV`(hP!4BJFY-hE3a7!OtM)@1&&~C)0(gaOWPKIx`TCd$ zeBVsNAsG09liAtG&O79;u=NpMVMiZ+V$&LKW9E|XeBl!Bd`6V39{E)f6}?FtHQ^CR zf!Cyjs`gk%k^Wdkfw7a6TGYdyS~SqgKJ~rQ+L*edG{p4LkK)S~PpZSt0^F}DQCi>p z5d0=FRD$_aqcK~P1i)eE1p8D-rIj&d#au6gw>)a=ke4IHTUH3)BUVqqhtDOtLlzW! zu}RRgX7)D|VcHE4;EDA%XM6=wN0~(ggjcul_;J~{72W%9b;u!6iq3I4o(2>s_=&)| zG?*i$HB@QQ+&rq-;BVszSH7OucNJ=P5Ia3ix(MdjCkf$oY=GPBqSFYxq+Tt(*f{wr zcm}oi@7YmOqym&>=+4HnmUOsGtudVoJY+%Fk};6V`7$~85s}9S5y~cCho`RySSW( zWThIiit}fyR#z64d1+dumk4%^(9dXk4oLSYYxJb(+EsbZu5O;}$r#K5(I!EBL))(# z?Mz+Ws+!qIh^b52sbTpK>9VV{$11Jvi3@ZF;|Xo82>|aE6${l6!u>i1E(&)>YwF~P zMw1&Q=jd)T=QEeOc-f#;G?pXDS=5=CY^9n`XUWJx$J(|rwb@$QfZK~|p|skH3qE@A z+L7&XF4tS~nHbGo7_PE$Ai+wvv)tEGJ$f|SM!5>?PQ8YqlA#Psm6lnO*4?_jGkUSs z7!`)V82~1~)kfv9sjH#UbbchCO|U_}6^7Ja6a0-dK7DnAS**>WB6@UAxw7dN1&@Ew z*qEJuW=hz>xEvee_j}M}WwaO@!F4kJ_WKBpnz*=et$V`s`MRx!b~QHm9IPI_uh&Y5 zRTmloh7G~6s;6Sna3%$h_>%0nEteV)j&*7V6)>k#{Lx1gzkGNt#)GUqD6)kw#E4pP z0XJmihNYU=60XYFPt6%m^0=&eoMX2Nu9Y+QtnDk4qv`mlY`RtaEnaAV_>U_#cP@nc z3?HKl-=y!g0x`_O*XdBW$}2zTMFtR$rADgqYs;35N^+r(hVsG@XRr3icatrk=_>w}FrPDI3tsM2H#aAfy5KzW$qpa{`e;6WJeBmvBePqy1M z?Eq=9OX&Ldo{&R_O1K%Mgsp&ViQK#-yF3D%{B%CAH{2`p47Ga?d zlG88NCbIe#qKHGPB;5Ta=|I}adE(zj2A$rAw+K={383QcfU>4JZ16$ETqj%_<$zYh z^{`SEFWK{`;pFf_6^2Sir1MLJ7hQgUp#>G2XyW+sgXc=?>sod+EfJ5mp1y%QG4Uk& z^~m>MYjf-C^LsaL-@ung*_`DH8)d3jT6BwLm2oO;+Ccsv@M;dkBsZg;LOq3dP^R{0 zWoq@MJrYFe$S-mwtAwP$=BJ}*J*dln><$5u~jT}v9AqyseR7vvx_ zaIzPVPP#l57Zk?VACA;631LQ_Y~Nat&Bgts4>WW8G2Dg@9&afV=5;O3GuoE>+f> zikUxplLX`(ILZD5gh6-0`xLJ*azXj&(B+#&$ww`mh(y;j9hPYUog zqUz%|mrLpw&Ow%Q3#9tTP%8$#63}R}PFQPd#b~*MLbilZ-!I2ju{Fb|nIk)VUJn$^ zM5VkiHID66?2^Fzaei!zhL~POo?B57?WIowp&NC0U+rU&aJhk)4~&6o@m~nkzi;A4ES1Bi1mt&~G*->AIEqRumqR%zQ`}31 zEcd@W>qpTk8ogo4Y1(CGKdu*FFMh2Xtyy)YDnwKi%l~?Zwps#MRtmvY!*DOT_D1a} zqmyn`+l6Q|*s8ZnpnMBK)Sfj9lQzv)>hs3Ypp9=RUcF3rdUZ>1Z*|D<_i6~$j^QxO z5G!}uh}=V6RJ7>%ShJ3P=n2YdFU|OOd91TP2F$t znth{H%d54-=e|)uA#V=yJiepC2 zo}5HJF?75B=kGasV_1vrZI<~zm()3 zb+WLNS~kPXJNB)%nO_pYKl+od%nDOtGTcI)R|V**Z(GsTnWD6-Xci|9o}tW~v=x(O zyq`_zsM$3D<}&O zf+K;=+)1vO_ygyiP7h?`)a%p9fX2QD&5+t|+x^kc>T?SQ+#zNRdhr~A-8@i)-FJg$ zj!Sw=Tlypv$aG{c(C@=vb}rukdM0n0i7IZufQ(+JN&i*8ta^gw2go?%ih(r8S3YqZ z>MPoY`Y-$Fl!MnIWC5S&DUjMxaEBUzvZjZqXY_kgk;cl-utkS_kEeB z0cuoIB~mXEMLuK3Y(lD8y5! zvJR(T2aI$OD%@De5H1XCM-zqfhp8X52{^@7K_?mmOr8UA2uX#cr$#7^sn~Ey{6n5@ z!gWVHVxDeXY+S6uHurqoBa$`T&C$~}Ci~iKE&B*tgm$B4J)pK+Ut@zk;#5!P!X^Bf zePupM9ZUG=i3g7M=-TfsKa>6L-q+3f;k&WA|3NY*b!v_HG!NItzK_nO?NFn2hhCB! zOS`jEjux0cvs>kwZ9}-t`15d)eSO*hu9{CPy{%2X*%xSi)8LBdFo*RLT)5mm^}=tr z+aIxjBV!HKQvmBToO@eOexOTL?Znhp5?G7&x5jY3>w|=`ZuMr)0iqK1i3;reg4HY4 z`qBjiTCagh!_&Uay*CUv5gsmTPteU1a%yYkdF+0LLxP^$_eS}CwR{_u6SyTzV9a|# z)_DD`NQp(SjLKANz4%lEecG08RxWEg;yek^W>SBj4lu%Y`=!TtGdQ<``TMty?3s^A zKUy%$^Z_<9LZ74y@>T3H{hurATR*89G%UY-A}eHc{ytp_KSGd)^;EtErWXR;;u(}R zC=;sw^}3$~R~!#q_oGP2djHUHP%Gz|;l0tjDH;yL3qq|)z6{w7YCL3}VUrNf*}F%S z)7d8eL;}Dk3~8<3kTr!en3etNQAR!Fwdb=E+ry zHpk2#_k=uy%@sXKN@hVJAmZu4t0Ux?r9dQJYfANSWApm=5g&p@+opl`Acz0GQ4;kL z)Z5$yAfY9d{|2pG{kc~tgvO_`%uptiJ?F2egC?`aFbMOQU?_>kZmDbor}ot|wPB{l zq;E#8jlRJ~9sj^e`Qm`x4 z-xVAeHJFIskE%e8X;l$fSeRi!EdW{N$WXcme|5GhSPLG&MQ5oguHPeK!p7T853f+w z$y9Ivo40V&_sMrqE{#X7R+SHl9S~=fxhWs9vU)4pLovGA;901$KQ`8J%_pKcaiNA4 z>t$pD#BOJTlhIQq^oxsIrSewYkAbGS?jQDigI%Zxcb7z(dZ4-9KV%)Lex-*X=gcK( zSZgs%rqKpOdd%3W1#0Z8rA0T(3{NQYn-<<)>s_fO3az=e>dUn?im{ANzp*fw43PpMv#(e6e%;Z}Q4 zS>nqr`KajNO;fZ|;rz3VI%xGgRQZ!NTaQudIw#<~vijp!K0?9;KZpjyY*S4s-T*r<@UP!qxN&$~sV-=6ot9!$6+2PDB)$*(aB zI4#Qnw^?>ELQW_fR= zo-+8U&Dud&zdzYQ&WcGB<`m(KP%4HSOae=??g;vjZvVPND5373D}V|s-v8SE09CgB z`;P@Q&2mWqi<7ho4u_b;>x~2hh)~h9M*-nc#Lu>I1yP&20m=$o7vpq!zA~>Mq%JjD zF@)Mm_9p~){t8{IiB|Aq(HkD&?BmqqB;h9LI*p%C5k0%lVsouIXhr0FEMo)UoAE)j zhZN)_o(ywNqp4O^VCVye+)IJfR|W&5r4de!8)*w=Dp2E2o^n9dV5b9+W)laM0l}_u z&WZE4_@{9Qj6;lMCz7Np4`?OY2L1w4TL1+f@WUu%Q~n1CkiC9#$<3xW@`e3rxS#fD zfmN9sw9KO?`7_wkJ_bnS6-g5NdXpmuMk(|aQJpfFo4Y(5FodpOMK@@NU~{7A2d+7W z8tWvDM%D_MRI2d+S8XdLu$BEMUx>pn(K545hUHyfO!q8-f6*wyTbpNcUO%v6Tm9a9 z;{h-wGKU&E7lmqk`41Xr-tWWVmb)#VgWxO|tW(>@aD%$*ynJf6yt>Pp#;jHvbqhrn z;Nj0y>@0YAKI2x<>!D8BxcYdSWCFenD7K+7?hwG~kuz98^q4FZx7u5uM;Af> zOa}^;Th4qKl=278DvAgU6M$c2cep`E3-KgU&mDU$8YKtmpJkLx&`A3^sD4`ZL}#;e zf7n2+4;B?{Cs0hN#CrEDuH8ix)feI=D*WW6y3hCI0(^1&0)%YBaH!p7lyw}YBCr=( zBNJj#dvThFl?wNRq#l|XiHl8E;_TnuPs2yapO0d8tO)R+zW?Ax0BB z+3S`A?3g7VQAstT+nuUk!Qudf#xEc={`ynutAC|5kOQI7_FrgB+9Cjzm;b_#{u3M= z;J<^RUSgELVipEEtX<zLb$CLr*c$6IknV&{`U zC!uNE%=stkzRN0S^NDolgVi)1z}p-Dr>ckn5b~JNS5>O=4RyMHe>Rq+KpF^#0b0|L zBqxldhf)=&awpF_z=q+d#{j0QBH7_RciB#fa<@LJgF+%pPjV)0jQ&J|f=5sqg8PYC zz8guIVUkhGW?~<{-ShS$@hBa4+g!=@vz@_HDsjmHLKs|_*OfQW_{YmCOn^e0XFD6q zlfQlo)JXizGgO-gwEX^p#8$3d_)(CUr_gK2*|Lz|sLW3m4+(#x7JyCbw&_b9#VbuE zR{))!vQJ_x^|u6b$h`wC&{Y@fmqZUEnW0nMpB!o1hs75+=Vk`uNHO%G$mcGm@e=WZ zoZIoS%%c$&c+D;WI*2%Mw3!ARo^d+xM#~u$mYy9hBwQbyQFZKGAiSw4Q`d!+RBg#Xt zARfVeImaoS9%q9J{v6Av)``h-d>|%o4TZSzh*1O-53x70y|dv(66ZY#Uj=93^MsEh zqD#Dc9u;H};sj7UQE!Mn$;k#x*J~$4Rux#Kczf@nTgBgU&;XPX5d^bf(9jYs5m5shzmW}}-bObm3L7LMn4VsU_vS20EoYmZzan^>2_sIaptPx_Nk@{_e`F?jYe;yjkJT4t@f2E=L5M{k$z%V%U^J;&0iktB%~4Ck?5B zW4B!b)JdMJ3I~wGgYUAhN~rgIqOGp8n@B_VGutQ+0NbM4+a;vgJYS=US7ltY_`~5KLNoP ztdrY@P9VGMmho%P{{+DZs~J!b6vMoyvxEH;1gp)xiy`Orbe<`4JkcJn2YKzIjP3x! zZRxFmeQP&S5iArx6=#9dCUaT#)g=6%!_cYURw-32oq3Sve9ERC#Vs8M4$~5UNICOI zqWvC2u90k|6BB1P^A5TW^u=u=p;R?=X?Rjv$D-NWq%kW4AOiSDZ9iChayMvk-< z4*0B2451NhYY*y?L@|d#f8G0h42xoGa$WiB{u^<~)EyDIr`HdqQ0iO16kDbgE*G1}wF(LV0p!>0|X;;?vrVI_7Lfh??FS@5btC$^!;hwp!9 zqT<@nasGtCe}gb-5*3a!sS*kf9-srl9o8Q@sFx&Ds+jE`YP-ge1l~r0h4RB@I&TrF zr2mp_O@1WSv~fw___x4u7@}1h8~^2}SEYXYVg6`kI7FU}(-i0N^e2{^gNqB54{07H zhH8GrhI(BreOlwjf%LA)6nEW4s$BJ!XUnw(@+OC#wXz13LuOfKRt@Fl{K#7|E777lvtM$&nk?SuW!MlSvpDwai>VsO|LQk#)d%c?HdT zpc1+pQVF|2I+9Y~O&(A}Qgm4&L+hM`$0uVqtpC(kI&GII#COT#M^7gqt!xjrZWFqE zL$ujm@0U(Y@`6gbdNxFpG$`1#9WZLKlU<=Cs^JlFkKB_($R@kKdmqD;VK-6tTQHgY zmy%rr{)Y9GuQ@cL=%&MhCm@l>9b&nI_ABZX?@`@etS3Ky&oLmuL+ckozqG+L0z~D8~BR1>w~STpAUx~UB?HF5U8 zxTAp6!rA$B`AJD58lkvRq~scYM}V1G_Dyn^eKcUVFB&hYqwr@?)6ar9k8mAg4vD86 zRm$)Le$}wgHYuxb3tMz#%n;ald4dxP(y{$cgbNKD=P?#W?NPmfq;MuFiiwD(apL>G zQhkmmPv=0e+uMLHmN6mBF2Nc33p+hd@Q8uQ|EopX zeE!-ot=P>7qNU5lC1X9rPS@3C)5nwTDjRDvFp4E8ejHMk4!{vCQouEjYjHXqbR_AH za(IFu_~2OVaR_;-6kHprmXIfwkk5C2I{G*|_4NB-f%&@}uVPh2R+*rNg9HeGW1o$a zrHCRe(cx-oNF+;wjg`r5q}ZZMJ$kw`4uoNFs89Y~Oi7HY2%bMr zzPM|9)%#M(0>VJNuVToy;#-a-YQe=~+e-P>yn2@R5YiatJ-im&5v#&pe9YEk4mmGL z;afZB(Qtz-oZ?u#D;L4IP)ylb};TQ8fS6oLLr=t-6l&f2B_(K94Phn&^Cq+eLjo=O@3TrXM%c0M_ z&+!&Ij_cW^hu2gD%+@p2Yt#7@9tt>m(t&4al_2lF{hc!)b^mpx2}0t(chdi6;iP>6 zSh9b}Ac5~cTI9cyuwJ6DkYJH&mvZCEFpMuK7=p_!qm_PS2NqiL(90yQnYEH{M=1fm zBU-%@{Syp%&iD-lK*7-cew537yuJD6WulVx!-+@MW&xN=7d^N&%sl#PB(c9JoRTak z4d;;rP&dn#LXlQXcLe~+b?2k|0qH0|+Gw+Gb#~P+F7XsfTxR|d2FL1UWC+x1pG1^dtJCLY zq-u^F<;q1g!~|}-*qR~n4@!t`Fb-SiBL0;k4~aUcaH!-X$P>#F0Q9Z1t57*fO>LuO zLS>KCSZbcVv!r}VO9LC3 zrTdZI8^KK`G30v$3?-ZOdv@jY`H%`+y^YoN6p#v)?4A=(>JCBX;(wL#y6ak{OIL}t z%z`Le)N%(e%LE|p>!3k!lG}-%hOv`Z9;P^+hK;*os}NiG(12MCV1z>_+0p3QGL`^=b%l@O|XwBi86 zwxn0T!O~>t!Q$_D37!-DcMbyb6b^_(-&=(VNYMr&UIEno1aU;ox34mln6X&+h|00~ zNtgx6OiqQ092=+67E`=pWt9PuHpJ1UU|OsHh!M7L)qlju!hea8)xToIM{UyUhcVQ% z!5=a5^1z1n=K$@$#mN7pg3KUk%yb5Y@=vAaoA00KsJfzzDul$>L!MSvy80PS6!{Ya zt}FUQPSGkGP9B+(j8AF+F`b>8$_NCw{YLi2<|l2iL_|$;d*8VHdmHsWq}imSl*}2Q z@mr9fwI^Ko2uGB^8R}mOBaRkMgQdb$v6w zPf-aFrJmr{s=Brrm`#?>l=hK}-oC!kMh}GRL#OG3PrK(t58jb~M0B`Zhm#=i0P=^o zJGS=SThN@a&Yj6zysej0sSbtop^nS%KtP*XWo%&Ng3g%1UZ>A{r|q=Po`o!yOV^=F^d^bg@0-Z#mu6AETuDf| zTOBa%Le}8~rTg`clRY{8-DWK#ZP+lg@wfels3EtSixjo$m|^ur)7|(<}TJDg@R5J6FOZ1+UqE3)emWzEWbmTZPK_b+w&ZRu~I8Pg((sh?!bufi#jHoEAg(GIn14$FuPIM`UhtRdwORhi& z-zhO|PdmwoYkDjie4g8@;96rWzWoU&=v+ZjaRCEZ7|~cP&G)WBF23;;gS3oFqY=>y z18;3n5>I+n=pUQ$^mm#n2I0kv7E}l|`S)ZQi1Ug8{+t-c;&Zp5E!{JnVgC5B&(;lJ zQAi0DE<{K!5e`$O7mI%_VMKkpUgiPiKtS<56jW`hdGz~JZQ^&I4G1@SoP4P}$aMC~ zqvDe;>0(lI_LjT2|+%;H%ZVjA=8~AQh{NLPatXOF0BXM0QFU5^^99_ zHR4{&w0!Yn8u2Zkw&Cem$a&A8sQ=4%-BShJVL*I$2UKiwS#}!XO^Y-F+$pz}0nqw~ zb(t_yg$7EW?H^uaCD`a+9~Wc`JqK6NWR9E~NogcB3*&I}S$V`2q%)gadmc%ew#*xKh^-35xI)M5xM1!X?+}I|57~!YjDT4iZ|%e+WmMg2-yrj1+ID z_u!aP2h|+(*Ze42yA`rzgT;52^gyzHqKbI5!)Ga}X{!5dyI)J>Q{85SAR4=OEVH8P3$)Nd4o zbyu`=Z2T{|;qHcWz^DSoZ|39a-_H5m&3!>z03R-qqD*&9Y=#ySvqwCLzETE!$RQrx zG=mZv4I&2Jee@j-gbt&UG*s$>m`BfZ&=3OPFoR>SYDn6O`5TO(kQ@ln0MnAHgdG~YWD9s+C&^?tShX5;Ov_)1Zt4Tn*l+F`3(cFw(@rK%Ro2V2fQrGz zfg!_TRz`RjUd*mw^7wZ0-IUanvq!&!!MZ3R=-tUb-=9(>F&VJY=^xjhtQOa_NDTt6 z(I$#t^J}5?_kLgBXkUp*3pW`*q(cBKp_&EDRf|Fe>7Lmk5l&n@Zi;uR-tGm%_r2$! z%!BRk#n7l#foW}WXVK_!I`j{c!qaZr+$UB2$O{p19qU#26_{u0%{w>^bNMAAPcZIq zW;ho>{uz-fqqJ@c7^MW4)4kVue>O2Nas$t)Rsnfct=Oo?8V1?s75 z>ge8LcGy|r$|)*`wyu`f$J*d&tB`22GZZxPP}XJ2K|%{+qlk{Kj~?=>@XS#y`$~JP zqrQGD(r@y+bavl%^CS>p*1{f0R|#V@JOWIk(h3MZp&it%DtwlSN7^Xy#yc}1M+ppV zO3H~Pk3pHL*+5$Nsa24IG>3}d$`YnIf2G~Mf$6K1A2QtXy2&2!SBq*9I_3{GqJP1z zxku;d=bd~Y5F`BZkG3y7T#6M2VJ_<5A`9sX=5L7;4)D*uNk(!A0Cx^~0DwE9rDH?n z$7#94SqK%1h^EIcDVfHJ;OJ-^oROET+@j65lvbv0T+agCe(}j7n}g3{1fHBN`HNC^ z{_Xb$%R`~c_ky3}W%I%fnWwJ&Z{GW7<1gcFODUfA8Xd2{Gz=i4I{_<~_~-&^S2>|c zYHfgmZ4DaV@@-f2UrJZN(2bJEjL?nJM-d+Fsf|cCxVjx2xP~2VxWFlY5u*SH+OXer z>l51_!hX;JwZf}7N4LLVNN9%FJbnmU(F(8T>^DyD!>2npaZK;~943W9shMu#7}^Kf zqZk828m^<+#weSO4=*CZEqirlTCLF2lWYXIiU!(mBV$C>JaUKGTX?{^TR#%mz;8$h z_3S*x0|~cKaeVzP;T1M(aWZ=g*?Idk6n0o~GP~k|=8nIB2bUkh7%e;q2xl(A#;rIf zT5cX|fJv8*@B%$PIQyCpF#IK3n2&A1u*X;+#}yZR(v=jvhQ~dOPTSZ?$B#;jM@qmr zwMED|ftrJe%fq6jk9w3=3m-mi!>zJqrT-ml-A>9(w*&4hYQ@*(nY?o{*P%&zJ~K4B zFvVKK0Xx+DfTk$3#$hvwvocQF8a3nAc!$7Tb2?_TOyph{cGFdVksIWt*Sy>bn8IGW ze%l{ww?kEyMz8~HoF)W8uunO5GoJxhpFTdes8NV@SR7SYU?_H+vS9FNekm{8Y_ive z(TUEtm5EU^QGs_9jf}ApqI9i5nS05jEJO>)$<=*Dz%g-MN5Z$|o+>hHvk+CJPcV=o3_|UNX#^s=ffYGo)FY}DW*kZQsEoicLgo9wyHv9Dy z;W$uv9Y6Z+xQc^`?pu?OQ2E||BLtiMeD8y9y-$@oqPE8ou*qwuc+)h-zJipt*6!YI zTQHVI1C~f@+l|heea^&sUgRUd`&Pz#w!JY`!o$W=3@=5ND#uU{~LippQ@OFBYA zrTQ7V2N9%+gRm=iu{oPK3htFg=r-5$$5ChfuaNM_;@KW{HJgk`!`y)+fn6eG87xji z0Wmoi;GX$tD!zSMwY#wg4Z2;!ks;|aExKJ(^kGabNG;4FwKK;20S(ygY%28P7UG9m z-J{cz9f#9Swgmu16)g;a4-S=PYt!TIcQXskuh8F?Gp_hY@b1$VCMJ;@`hLH;qxp!V zrynKj=UIr3rt?X#JByw+W!W=3tEW4Z?c}IdnWd=Wca4pdZy7|)=N!`@w zk4}2gpepLLE=rQ`q{JCx^`>s3Jhpf!Et}}Mk4}GaOu^ot3a%5`VdRfj!(!vxia!{y zE5V_=jeZJO<&)&3-Z*sB%07uvC_=iZBsic$R5jm-yBma?@l6JgD&2^A3>^2y4Z>$G z=`n>zQXIt+S%Dl9pgf!T+mbI%ex|zn`J)J?$d8m|TL#Q1#Jpm29hpSS%Po~_-wCBT zr+uwu(>d+w&a^Xa_kKF~1%goD2s+il#I+_UPY!=ziZ)XT58z9=Gc4ptGakE|E?oZ( zHge@czSEH3ZnXUGr7etOUh)}7=4feUj&cUf+FKdAI{a5wfG@gwF1#S){1(bN(7$q8 z4>JNLqI?bArb7@DrbFDRzHn9OuvFQl5w2Wv%9iCe=`knDF1u6xO4nThoTG!v=w}eM z;|jURPav)H%X$*|sS4G|;=|x8Ke0K-;3B2c^-{S-wt(Q-`BAxHE*)Z$5qoDw8f}vl zgaGB3eIZ;sfJ7{p<+crNjYednKLyC44a046IAS|k`FBhwl)A=`@YJsx0G?3!EU8ha`B(21cVrHOUiiF2pZ~+yTL9IeE!(0YK(GY~Zo%Dxy9IaG z;O_3eg1fuBySux)I|O$p=vzDcocr%R`__9OimFuwRaAA2+1<0}=rICdMV--`cDUPS zD6RA1O-Nj!&FOs3*upf~d~gA~t9h2*HK7Sla6G?g=Z{2I?b>VD7FJ=}ewKX}UP=%? zV>WS~uBny?qHRc0btl{KIR~m+Tx(XEo!afZK@fd;5Lv?xdS*1FfqqafmmRgmL+GM- zc_GtJEs=vdgK%B`3I8-scpO>H`Z*z;xO0#B*!gcg4R|y%(jOoS#xeR|R4M>C1j+|!(D$DugMYVz z03g4ksfB@|-QU%@Ugftv${^~Suv)Q7ujt`-{OX}E>ObpR^}K&V8~gY{n4i$a{G|S+ zqgvnBV`7YX2Fr&q7;%NVIiVXSlO|Z5zNDvLUBiy_uBXyW`7ZPZ_P#~AYN+mN?d0d@ zgfF0U-Rn`=dT{cf)%nm;`aEmR+lBztA>u)LEboH!PK4A#bWz%3_FjbC_|imvSVsT0V0?V0s55V;l)M3KE41x zRHZ`&x$vS8thG`-m~#cW&~K;`)8%@YZi;i+Dni`%rK%CnN(^(umqY=XMF!VXgk@ck z%%W*Cbbhl~xZSn^nT#(!+le4`y_b+}zJNdsuklN|UUtxm04qSo+yz(0jc>^H4cS3$0DTp{>jyYsRG-L*b3_t zJn5K$0ded?BqoF1(2N2Fd(&#oEu-K)VzL8`We8fy4xag;h<62;1YRs~f&An|JS~pJ z)-g8J$5r4gfQLi;Q)AOLtR!GGuMX*INNbd9+{whKv@y9B&T^DEjL&qsuYog_jhEkE z+-yDW>%IwkL~9=dmZQ%viX?NcOACK@YnfB2f&9V444Y}i@6Xzps|R`(x*J>z>FN+@ zCD#SRp$fRONJ&bD!vv(j{Zim}yaxDqCLB|HgI1ykwwl>!IgyRj%`}lUR%7s2n^S|B zqdw^0rV*pv;KPD$3$mR=pKGbtqqDSIO%@wdedWE8l&YGjqT{%eqxEO7wuH!w?3U|M zBXHamm}a`vCdnk%%lY}=&qfpW7~ zjk*jw!t*D+2WRpk=jEmo3vgI74MI`d>11)L2E#xC5d#LxXOTiX4Vj}W!4$9O9itGz zVB(Lj6!C=&Eqvi6?D3WbrlQ}V{rSn>odEirgC%WeyV(XYeTq<`jzIm z#j|vGw^C@i!*ySQMoKr-PRYaDsZ4t+Zm|V#y9MzbrVD(G^rF?c?9H5b@Id(trxbH3 zbG9B^wr@ni)r@hL62eb;*@ejZ*FOz!hvjB5^l8Ou*i+c3ZoH^z<*Tv-LUC$oPF3KBZf#+S0 zg%ezyy@3fTPvIdyj=k#0Rh>fhqa*fLzp~fO-w2%=SJQS+YjCYvFftDFiE-w$jiZS( z7V~k_RC`;cPzH%O@0xYYPdl~+Rr)mKjb_tSuE^(NpE2hJzmngb=UfrSBSHMSdvp7<JEl^_h0_oaK;Xrf zUq_wmXUp>bJS8VM`H>`0lbh*_8>%;T3fofm=#s`-PuK43(j0?nA{tp~e1=RHx1vsI zHLf`w@Mk@S2(p$iBL&>fCNzbYUp$>!+^cFKK)2m4Li~X}#SsjiQL(rGBbbn^4`v7d z(IR=F@Z@{D0=>Wm2OegjTfPdOQ4yi14M!c9+jsW(m7O?8B|40uY?=jbNHeKy*NQHs z);QK#^Sb#Bo>~P~HX$v(Y;yz+zf9&j?H0sNphvZJ?JghUV@%JFJ{e)-C&(Igemh{4 zzy)Uu)q<28*0&sX$!n5FDk_k)kmaUJ*}MqvoLZ)}hu;2;4NW{Br*Wilz_;9aImA_F ze_5MvzVr7voYj!-(-EGBLUZ>$H+8#hioln)Aku^xNEz;(z2zi}zo zT_hf#O!hS?1ZOciCVv~cK*w&q5efm5Z`G(y`zLb3mA_$OLwt9m7P<~b)^?Wv(nhe1`q#p{L^MB*uTM~K4CEKR zT=gLAKqXY9uM3H8B#Chi&1+9uCWOJHJU>4ha#_kE?e0syCtrJloLD}vb!P_$1bZSw zK^o|w>gddV6zI;03jNyg1Qt5O0;}IWY1f;U>$b}8(*gMFhgmu2Q%j4W;gJydM@T|ZSU^c&BkT1)> z04QO=3lR2F0q1{AhU0|;M9g$Z{;PGsrl z-aFr^^k3Q|^!0k7@rY3T{0a5^8N?NrbIMXme*TUaL)s^lVt^S?5=iuzkw2o2K54^$ z4hvWLb%A~~V_5^^H8sLTSaMo$b7%cmsAF(^SZF|djSc^?KMGnO1+^S#KucW*6Iy;# zV?jfGQ%hZo|4-Y-)LvQF!qkA@)xnUM{_keQQT}g2RHL?eqq?cLJfBcEssw$-;|};h zA0;tqeG-b9OPZC)*R}L@3uCD(m0~$DCb>;X5Eo$CFkWZNQ2LzvA;aGK{N?F&5BsNk zli^ki(3=E&^ph1v75o}??uXJaq;Dzsic-Jm4ReA@{IT|FnCSGkDbc9QvZTqE`N~`b zhZ>bR{gxNZ9)7m#MhyPysO9F`s#p;n2Gh-+$7vnYkC>dJDkQqkv{88+TNNFe0VnWx z6D>;r;yh!=zimsm<5ye})rS5ofO}SzUC20Y4}^fZ!9?zthsoqj*Xa>yUE#Ag;suRW zt%C(~R#e73U&H*94RjkH90FwxbPsWm%1|sfah7(Ys$6O`Xz$w&yrbNn$Ad=&NIr@s z@id&>h|8~5KKBu_ZyQ?A5V@Xy z;yVeP^X?=#(bPCv(!%O4`j%F;hwFm zA5~V%Qk0c*=$|3|gA75G7}EL_6f=*ub*-zNd#l=Z^O8>SaPgdAo=2UXNCxRdB_HoZ z&SZARgR$W??aodQa2qIA8jX5jmDzTR{|TA~9Bar>H$~3R8y091>>pz*L%F}Q!bqT$ z#yFe8457@6zQ9^xU}GZI+})w+W_N7xe5v$Dhi<>%KEe~-v>?@qtkSTK^hAWVCLfcq zmN?^$>A(unv4=%FrzjuuCyuak}CVN~1U($S0~Q9!bKD*a0*5*4GiRrl*}I}s@gaZREZmB@bb{IN4fwN}42jBp=T2(sYwsf*@aMBTk5 z4Kf6u!t9w-FZZ3%Xt`|CF`Qv1!#u&P4P0Agne~_{uKM#9{=g@pGEL@l`r)kn@K!!m z&Z9#EEz%0C0 zDWb1>=@I&b@znT&oj_x&G3o>F2x^(mH%sSPrvvL}s5JM7cZg0U>O5rUA6&h*)xVJ$ zwfQDL%e`!vc=^n;^^Gag*z4WP*ATL&X-RhV{{aSv+nm7w7-;2V@MmV3aYN#!EDxE7LPB{z>4(1(UYb`?pw2h6ogWgj> zo@PEVlw3ZK3a z?O#bF6qGa+#ysnLdn9YB7CqY}$ZP0sH@=iHt#qt54ie~Ixinjyx=wy1xVu+JP7D-> zh@tZ|Slx}*$#8McTV`_2I5x!TaQvM#n%}j^{NSEw^1C9{a6*$NR?f=S zpH|<%4mMsVr>wziF?TDBemSJ6KL?MfOx*~?bv7>YberJi#5C58!;oq(7nC+^`sH(|Iv#)0Bl`E{p~d=R;aWGU zG&N~2Z!phm=L?fe3+~4L!A3a<8*9U3$?V3``)~hCJpuexPijH+MEQG>a@3#!q>!T2 z9G!Je1SYadwT>R-tX~W&soH?w^1j)#Xd7AsnGd*op);{&JKqqiO8V&;5?accHx-xe zGe6!+>hV^vLp*I;I0`kltoO`kn0n+b=>=(WE`;Y8x@BeWQ(`Zff!3S@Tad{q0)ehq zSf5x{Jnh-I4f(TsS(j=*OzcwpDrJ5Vhgwr*MzD$GkEIi?brKXI6jy6#BQ%=LGd16! zDi^)#SeK0(S;$i_^TQF*QaXIeNCO;9s_o)NqKT(ubtJ3IAC1(Ae%+(QRdZRR#GVIq zVfFRWs_bV+a@JY;+s9|bHVSvZ)^||#(a(52q|05!y2fAly(GkAkDxf3|FEYWJ&01F z8&C>Ak-&yDXm>&sHd)l>G6}Pd)R_->luPX)j0{^Tw{891`1)5t0s3rm7!4vn{NNh2 zwK^js1r?M?xwj29P(g7F3s$6YHoF&ZAWTlvn&|qwpri|jx}pt&urcw^ywUj$fDQa_ zZN|SE3Pmvm(7MSRspO2nR{|JwB-D8F;VScrkLkDs}hQx;^2K@TDyJ##d=SVydmw(IZD z_+_WETfs{d1(R8c2lXL4BQOQ`6@I?|JXlFGL*-GPQb7ScH>1wbK29I5z-Zx4YoZ~* ztURy2U=c8Ai)suH{Kfwv9&w%FrodCq$2!jr`n36;?UHzC*B?^A2dHP~!C}w7p2$I) z&11PAev+@Tpqjq>BXo`iia2IUt$hNcSk=gAN7#))c-j~)UunHbFGMxa*s@}C{8Epk zLGZ2~cTECFON&a|xc4WOdS1Ul8uJBJ=4L}4 zJAEdlawUHIJ7syY#U>>NBQ(=kR5j0%ASYgG<&+Jeq$oA5ZKSI4j=o-Z@Uv{Fl|z(c zOx~xR9sO0=hN+hoL@vGGRG&tPU4sVee&4E`;T&t@=Rq1CAuo}BOlb>tvkW{3$4=+e z9ugTqOtaC;3tGqyFyTx=5d?DX+)vy5WrqpGZd2SUGC zmL)+Ouo`>i0HcY(9KRSk#8~5Mli+KZix$!3)}5p?dNqOT8|R$5?r-d4EnQ0LPLW>w zuunjuCnVqLEGXLOkeH^ai?bDe^H^49-e1oVhb&7Tw?|M+PS)9`LkGt_9m`eLyLj{O zTLCyW;8CMNGm)VsiWqHU+-^XhcvFV4*M53iAbD)7WJQl($RN423M|L|SmH->zZN0l zn0pd8;gyW_h;M2UbKXxRU3`2RkL;i_;hC@QFLWXtQr z(xkumhHAM=R*_h|;Z-Rt%=V)Dd|+fh28VjGElb9DuP0Y=V#k)`TdPxG_r6%y>@m$i}y*BV1uRgt8K2v5eGPb4yYQ4U6yl=Rl13|7IY4$*w^Ix~XOg1nb z)Hh8)8YkHfJK|*MqfZcR$YV?pZRlf5kaf^wLXdSR8_^CV;wIviCgLXYl_w(amn#88 zUZg91L|()z1w>xtD|^JwFIS`Z;#URuBVXWhK3eOFLdgYd^3Cmz^D%9u;A7dALm+Yk zUIDfNU;zXH+yIz>b^wqf0MM%rvg)vklJ(&WXryn`L94DDhLXNIZmY5fq^#=I~L5g;`9@L4fW3 z(EFtq7xW);8E|`AG42`H1~2&nU;>~FJ+LC_xJGP(wvhpu2EfO-u*n@y2A)CKh^;J; z+_yb*= z$^au5SGc;NhyQk|i|W9l&G!M;tDj-$wuxcd^eS15wJ_iN zL=0Zw@X%z}>VOu${%2@TYE8pE;~z^>6CBT!h2ur}rvnMqpPfr>YsYEM=@n9V1b14> z*ou>Sqv2PkNY@nu=TM}SrW83Uh~tJ95EU35kme5rf4H?`EyjrFjIXp0_GrO5!(1}g z*Zx?$T?r>S{Z)`{s-UYRk!qCOLK1sQ=03rTT>4|`7Xi?k7XN}8|AGd8vmYwsB)u%$ zqXN}`zpekH8XR>J$r$_}q$-d&5s{SE+FK7!Nhn&g8&Xn3y?~Ycor*2N-huvNgR!v!d|$m0e2;$L}}>V(E>gE+wGpqM?I=L`n3S znn%<6P!tW<>K6TKG95}R8F~vT4V2~NA*V^#jo!n;eYz$N0hoMbv{?VVbhwP_f|5%@(L|ri6sKGdX(4IS|(*7Hdnyo|89+ z6Yz5eY}6?bOZLGiPB2A7&v|Ip$wTxNAFp>C^>Z=tH9XakE*Qz$(suE@gT2y5#MH6! zUd_VjvwE-~* zv1s6riJUd!PJgqB{x7nIk?x1$ zv6@&s-^pMT!}c;>yB<||Rd@^IhvOP5$b!)z7s z%y}Swnf_hg)&MMK8GY)UZ3k&~NDFtGDNSOR6@C;mW_};FDwhUwZCDF9Yja+DdV8I6 zb^yJcIQ<-6jnkZNV0>ypX2{lxyo=UInrrTx!cj$&Pr}>*z>$_JwU^Nla-@=Wgrhoo z{SitymK&&t$jx&UE>d|-=@lloDrya!+0?^>@CXwQEk-Ru?TyTp=+ZrgePq+^Lc$}y z*0xE6`fX-?jb{@ID`Qd6Jk;^)V3iB3d9VtQD|pd) zutg2m|B%yfWRZXL1&m*wGg!yiP9>N^pqNEP$w0RCs&Ed#Wf|!Qa2l!F4>Jo1&!jdqN5Juvm^3fg3#^%UN(ms%(Wb)(^u61~k!Lud#rG34iSx|% z?im}!6h6l?t(2Yt(>AM*O56LYbq-jGdy~!hNuQ;_4WY?d32*yzXP_o;T*RAD$o$4{ zG5d5|5N{^+mN{U3CqtCBs0nL@530ZjMIzqiq84Ee4tqbZCHF4}=_K?%yVXqS!Gson z!+(xsY(yfnZH#RsSn)OlUIR+>wkgUA!CPq8#MG|J7DFK0a@O_xDb#E!8*|p1MbQ;T zH0Zqts^ZnQq2!KkOZx9)n%HN@ja?L|>m|6Pu z>$@vjhXm)L%}=a>?v<&np-nRT;=@RR>U!?R$*JXt&khxDG`49cRjHbgvP%-D#TsA^ zM^<)9-*v;52JKL%^4*I|Az+__Ugan7zO5%Voptyw0mohY`n9%=F39fj;C>UI8f}W4 z@}gghg|3ptxxN?!Ap+ZD8N*fvFyjV|WRYuCgbJqU{h39Ierh=M5zb9brYRz32$$O+ z!O!i>=aXhq((XE0->6460o*D;zFFXn?5F++tCJS!@m^=pFX-5TSlQuh#-qLCgVp3~ zc3(q}ING+W*9bZD@d}7|_$kIi&m``r-~W(5F~RPirhw2OCLZ$ttxduJ^-T8qx)!>$ z|DKKhHR$^H1oV$o{TqiGhqw^O$P%>$tU%*&($8MJpD>R^-GF<#sV|u=^1{_93@k{V z)9A=}o|p4vZ(u<&n86(>kF1CXz-DKAK8(g;_QuWM=j@x!4=-;wuVg=A#zuxll45MN z2f#6BXE>P>6c0{*Z5C)HVu2yftdh<^!hB++H`;-P5JZKrQ6gAC^{*ml^c@2fi|QpM zpHL1&DHlINGv-o~(pT)@S`{bp@qZ^y9u;l zrwMt-(!;pK@*EM-M?#V(S4QXh(#71E~9qeB80% zt{GZvz-Np*wB@(5Tu2_3#;gRu@f0qyAck;Q8l^Stup zP=_bY$>{FstxO2-b+K++StJJQyMyQRwK3QNn#|hXKeY$Ekwg^29r@0Zmm+UD>gA#5 zH3k-FiokwSO34#Va7!k7aS{+p{~paM(je{jQ3Fqbbw#mJG8^HVh{NYfCKC}f&>yBU zdJ%vJZ}WPw_>JpD3=9R6oQ^kiZMK4T{P?jgD}b~7P2Uf4X+7zagv)-#La=*dKIy<_ zknJwB;DjA3g-p$zFjb#{$x2!F|EPuTzKcA)a? z45A3WEyT1fxP{{l+F9^tK-&-piHplaQ%HSXb;Td-w&Ryl*CDCQho7k+Doml_JRY|4<_~d9=Iu1{9K-G+j?_Esf7FQ@HItomjK(014l>JKs&dYdur_ z)NDdi%`^X?stHeBR#7ohG^841D>uN&u#tGhiHSGzC^|66;6=Opnc>?o$W&Q||8%JH z%kvd&7y`iG7VbW9IvgC<;H!acP`@sS9>_JO7uIi!7E3K%^GFanh*GyL0;KjT!w<`> za?$YiAr2KmDN2$jp90M|`t*|e$wPCgBWjam$-`~YDA34I9Cn~!4IRsHG%<8-llpuA z`kld5Q3x4mG7w8Wu}y(GspH+B6QaF!K$+277jh7-a`ll3l~zx>U!qD$9UKNsOIIhb}| z&X~eFcADtLF$+B|Pw~m6&n&45PSbR?2IrN@a4G2$0L*2gf^uWv9U+b5v^+)bUVYzV znWgH!FCoA9P%J+WsjklDPz!`-+df^{4-`0&S9C&Ob2~2OZ+XFiROVH!B(`@Q_tTE2 zgzE1a>;o<4WbC?`4$H$pZ^C{oGr%zlVp3hYQf1mV%2+3^(vrJ`7>a) z3NI#$1LhRP0LcdkRc@-uZ#+Zic%Un7cEt7LY2ifmB79zOxNVpjeYC(5#eu9hxQ1N} zwsKT|x0fu2NDfAPgc6iO>mgf9I_bIN!!?M*7-HKA-2Q(aqkB z`7x62t}>3%=oE%1wbq}%ykuNycqnD>qg1UW`*IjCZN7=1JTFwtq_$j*%y89?6QhPk z4v?KjpGD&Q5&FWSv+oDX?836xq_(4Dq98@;tQ$QqZI!3tY!dzL(XLRgj!mfWn)ijZ z7Mr9%dUp0eyMw$_4y4EJwEZ4Y#KcXwgX9|w){hKQOE^uEmXT`fOYYzalBg%fnDQPj zku6%F?FdxcB;E^TfoGTdYqv_a`xe?~NzM^to~_B~D_yVeIeJzQ@0g}|`_Ck5O^7cg zi8o-0&Abg^?o=kBr2CTH zTCqNhm&G_Eo$y(+29^V=XSB(9c=kC+3U#uO_jW;^n^Y@5iGXY$=V3IQ{y9}&uNtiu zxEUVY|BfAt)x_szZ#lM_7PyHVDs!f!Y-9Y2DY$YB-xQApO`U}vijgnVWbUH85_~g5 zfMsVj;A(N$9`WdP#0x!g7&YA@-c&UPkEq?ZW7Ro#ObyFO_|FS7m!e|>Hde(5pg=zdBGWO1NTt~STKf_3w6!JliTf`+VwI2Phyd`crH zhdA0?5|6WXj|4l?+pvhsqZw+6RGPs zb((`aBvJeOFsoTPd%;)>b)G72<-0Y3Yu?vC^SOdJ{s~$T@$bu*55WI;YW=^IU?E#a zQ>TA5o`y2qGQHmcp3>qKV&JdPfG|;ENDh=G+HD9GO$EdlgqPBgY$VZ_ z4_+ue%}%{!&lc?m#?J>2H`rZ}2oP=`qq~3|pV7xZKp@(X^4Dzz9%f0|M9Sd8jLR80 zRE$dau&$7Xm-fdJ)(up~Yh|<>&WQ4JPB*ud?iW*@8=7H6BEj|B|LEugy(?X3tq$J_ zFq|X%9OY0lIOdw$8IvY5urL54$c`ipNwOPFGfVg-ayMCX)`2N(2jjt+ydYEb`o$8`TUK!6u;_(0%Z0pm zi*AZdnHTN^@WG$SZqSI;U-Tm68_t7iAI%{yq_)@xEPHhsKR}C^TZHpW+0uA0o{(ix z5SKk*$zK0?pd;+$X}$#oOY;8~EEzix!PL;;zk<~bvXS_bMl-dTKxne-oeKlC{oPk3 z7=2Em9?Cm+*!=k;Nj`D@qjn#7Fsa8+Jj1n)TN9S$^z`)Q)3+C|bg;E<Y^wfRMcZkc{RoI6#8gPB-8m+d=dFF$P$e~I!Uy@|&Y#zswqxXAQR zx-9O~{_-?m90pkWipFSp&czMc#~OyLCbEt(Z(hF@U`|pkLdGPLY4Y`sC9$>`>LSg3 zW$qQEh4vK+I%i@m_y`mshJQwgoEkuvcvt~INHn4bIAe_!(&Yw&{<0eM^B)9>pP0~S ziH9^Gg^Yg(QMOgOmkxznT42rZyM%fO21XRe_dsEXe7$V~0!RjwrSv(16Ufu?0Z=_Sy?+SOUU$%3vf-wi;|@9I@o?cuF`=7_ zh`@u!BT>sQVbiw}KI!6_B#p^NEKvk*eA}-cyAUjcISQBzo+3;N-JuV<4jxnDI7?@^ zOg-G;X*;F2c<=U=AK41870MW*GQ?ss-VuUO%bUdyF=n8^P*WPEM};ZZDh))Sq#Ldy zjUJiVYKdyz=D!p4IOqg!`3W0T9WD%Aq|%!bp=nt3oa%(uu(;YklAWSHswZ=*8qt_{ z6SxxDK2x#1I$L1Cj0-JL1d}W)|ERn9v;QXi_-h>n6l;fn#vAl1NDY*VXh#5(`{F-Y z<@AG>ijcz}!C$IQ5Y+ZTPH$?mYWz^CaU#K$Adz$V@R?-4!iQr2Al07q#*H`QL%`)% z#jlE*U&>&=F8P7hP}RJ6>?|p4Deo3lFaLti-W6p^KX=E3=l$Kof3*(AJ4yG09-o-1fVAE35wDgf` z*0paMCcJR0c$X$B*w$H8*Vk;ACGHO@x!dbMN2yF9HsDA1VVN(Qpz|}12;ce*k73ni z$vdUvLvlWTk(t&x&z|5C?phNHZ}z3Uk-XL%))s#gzebPYP4s}0+RwiH-lQGlRwvdt z_4MabtCt*JL<|Dd?f(L*|EdW#y87n2#)kiOLZxIt#!8+NaF&bwrKqTM8FXL>EFgD; zDzg->w#WFG{TYJD(r;bZE&SK)BG|X_lvw;j(nv;j1uu zbsqZD&X31nQFpIE4vaP4EbQA{8~#;5e`uKPNR+lHZV?4wBgB-A*ckH(OxS7&(~yIj zZ3x-0Mmk=kk!yggj(sedlKKe9x0LHd5k;l45$&lYk6oi24r41(+T8rUCgNtF==*S# zP&A*^YtqJ-NpqwT%tklwBs9WE{{BZM<9B6C=U=&U-zHBEMMpg7F}NB-T`^GL_hf4! zasiU9m^h1@q^-IO9TKk;mG0@6gs4aFbHNxp{SbA`R>^rrcSxUzckpYV!Fx8|zZ2%J z@4S}lV-V+@;Fnh7oUCzDRN%%URq9$G>~E<37w|78A6Yx5lFkA z!F)RvI7w+xKPWkN$J0)a$b#8!=P)#MM@74v#lRBDGv)Da%&lV)znrf97q@Qoe;@mFWlIHQ6%6jsh0**Xa>_YMSS!?-1QdyB%Kng_LPUz+ix$jS zQ%Qa-S?JXAJ-=0WzSFLJ9-B*k`}r3C7AQSgODu_R-gDc_zRW#2yne9JZhZdsdXWhP zV+-9x{pc^7omW+G`F%UPhYU88(xc$A8Q6mkt4-;Ve_7wd3cE?^QFz(j^BFc01()ij z=rXov61J24rQ|YI760h3lw-GE$M~Xp8#wZ#wLW)~6Oh@ke>`O9?2d-=7 ztOf2gYIV0%U1!IB^sn9XjN`miM_Xo$3|!yOuapefy`Gv3-!Vx4I-_>K0V*d+M6@i^ z4?5i_GaU%5ipP4@X`IbpKbj3|BH!86kGvj2RA}bPKzWqJBEE`-Pjuu>DSs_j_#llYSzR{?-@*c&{fm9{TyfN)-kh$Y$ewDk;lC ziHd%EA`&jx+f$vWrWupc$MkpFbC;3AeD`+6CT{9DPbt4IScdN>8KA4LF|iD(7W36s zdOsjuz0Ewb1E*ed1M5H4n{b=-Ewb1S{0{XMDSgu=@v&=0Tc0n^3#o}PYjdXd zM{us+P{kPn=-;xS5|wSR5~I5?N|9eOnnMUQ^!GS~>HOu^@;F5HST4@(^d8tNd^v}< zaTB5kDA5qF**`z_&6_;+|Gis%wocuZ3nE*5h51LN2f2cwCtflDpc30?K_qs6#Ziz% z3ktUsQh*sqG6LiHp7~tEjGI1 z+Mp8ZXSy6ctVG2z9Z?~$Ma&peg(j;OgVY98E(SpcB6|QCBauII$An|7x}oRFuL{nI zDOcjwE%pGVf?K-$btKkiX~6<=*8s~S;rtzUk7X3MZX0fXN4%`BSMoW)Tf)Y$L_%_e z*Lt>5iHx(knO*NXl50Z7hr*D;QF92q#NWF=n&Sh0qdWe#p^fPzQz#@}dL^WV(oB3) zvn(rd?WqD>;090rV0O!I>XQmMtZOI4z<}YBqobz3P)YEwC5P7+Wwvv@)W)5}E^s)M zOmCv3ZQn5@kPyEzeRI_Xl8#T+SknkRzwN3{4MwR}c@`2I!BgZ~cIUnkRtD6liLa%& zo=N;3=dWiBz?PPlQk7ma zVG3T#&$?jlHh()Bus4(NRfWyXYWeOf2bPd~KdX>d6UkuF44*3yEe=z0kPH$PF)UDb zi+6+m7T3|rS~^hLkvZe3x<^;!y>s%WdgAp*G+ahIt0R`TN@+-%!abEi5jIYqx;Fwm zN(+KKNS7Czgal}P>yK&P7EDE4lo+x@oB7V=H%DlCG|Q%GG2VNHl^BX!W7J17xw5d~nFs1E^hxabc;QU~_j^s$DShF?OxBAScj)`+4FnEV zAEVhYvr6pJ)Y?hs>apq`xgBzQuCI`TcT5-sGqO>kIt)J8reXM;pHMO3S#8yUz0|kR zAid!23Xqa*zAk}}(ioiqK>evhx255vkD%8eTa_J!EC(dWrclv{kdJ9WM-11G4v2*H zHTNBp_4a>X6&Jep53z<-kK24t7NsWWLq=lm z@4l4Slr`M?;w<0gQG9f*ny&}G$d`>}S+qjM=;r9L*P!7x<}ISoq$a)OuP{GEM)5Jb z)OXypU4za$>)I6KUInTpmNi5VS4oJKHu04mvk!AX+@aH%3Sb#*(BUz$)p+PGgf-6? zX4P$7Vb;xn@pL7Im^JY0#HA6u09(Z@0#5H|^`4W*Ps=G-UJZb!_;4WO*xe#OSyhCg zYv~?T2A%Kv6g~K;;}9w@q7(UvWlAZ^I6gc;%7!pj2yFJ&Iq+>`=4ZszgUQj?VMDjY z_!U-icd8bFJ18utKE()XFxkwDpK#jC7 zfg0@oF)v}a&qEOVFiL6+U@hPeFon@=-~EdYw|~K+Pvt(s??5@tPkhg1u7w@wP-T1N-{|BpD_d?WOmh4{qSP zz`r4zulZp+d?Dvq&xZQ4kqiRCjm`L8ES>5 z1+_Sb_gMsm*2CVvYq4-ndA%mzh&}N)S(ZX3IuQ{}_*J04$KQjKRofX$?5Rj)yDm+z3q&S={*L8MCPCEEyzIE>G_JPXY^q zQwdb>D!=ls(=UkqC}OsLlo4OqZ1z6eL@zPQM3k$5L^N70@0Aqvp< ze)`1ft3$4P$q>iz_~kwefcSaDmQjFLAWAoATRC7AgCEcfa!hmQF6zOh<$|V@bSn=f+tL9bKQ&+g3|`LCpiSjx`n9uOCSRXc zj2NCVx>Q{>V>DxyK?{)CR;{ZM7GrpF_n1zCu+R1ffspRoZG0IZUhgXa(`lmWMqtWQ z_18WjE8c(?9&R@`64ojwe{oXT@LL{3E(QJUW})$bAr1A(aaarkI1)>x{V;OqD(3)+ zLw(@>FJEAdcI;gf@O!L6an}mH@3d-mm~XpU*m2m6 zK*rO}kMsWibrd4Z{dn@Gylsgw8A%1lCCgOZ>$_2Uupuc`+dmt4*C8qsozkv1?QIv-}L6qXyP7FJw+pug&BI0ncpO& zZZrg4*x1mpEC|r($9}{=oQC8vk7TWW!}jN8Fee^%6A5!*C;$|HwOlyYo zcd!Fa;wRIW1VG4)alj@+6w0wjCDfxhP7sGfnpeiv*kh{GD7tE8D%GU4)yZF(EW#>P z=}Fa)C}y1ep!ksytI*blS{x>xRyd+Q{Y_DEs zucd5G(>32xR0eojwv^1~qxK@%!mkY-GODnY*!*4_d~dP2KlU4#*A^weAm$@$@@84U z%xe}-*#h#@$>i{~dhmW&P?j~sJei{w@P`_Gr>;ZE+<^q4Ug2i7d9s#+2P znNx%;b?c+&bkOGg=V|g946B=~R)o#O78?&?wXc{-N}=KW)^ERj#LLE&IP4xw>LGJ@ z28QH50?4k|y#~-9akRzv9>q(#16`upBD1ViyY9%=fP7zqUoOIHZ{5(R7a`L~vRYsj zr=+=F(AdfJu-orENGYXr?HEEwIctl6>H(-F2L8tlzZjY19PTh)`DxC%*P6rbC#Nah=$4LG3}-JJsq9S|)TXmDt&C;%qNSdV*lGr8uYr6i@&Qx%zWNrp4)b;+s0_vg+J{{h2EhNykxoGVH3JxMCiZ% zhU5~db@4z!f;UACl0M9M{WBuhvsFv*0+Au4|JlIXXaR760v5XV_KLd3fA2#6d!*qX zr>+1}qaA#QtD_)UqYc4_7CeFsf}}M83j|BkE%@2n54G8#OQSoJXBETY0mmEk`S>u1 zhi$hMZ{#Yk{AX2v;`%9j=gTCp*8TeA;Q{Z%{*`1eID8}%*TSWufL5eLZ!d$T<(O?cm1<(Zc+B2tQTaI(00M;Q_ zQCD@F$$xn{02~}q$T)1*yDe8E_`(|MYrSNb=rCg3wa96%)*^l(p*gc|_^lBD83|L4 zu1)yuc;n>mgPuhk$+#06rGBWiX~xH7(n-447+k|PM);Yk$P5Le>{r;h=&1C2J2?4Y zuqzQ@CVY!VC8J)huvTum^=DEE^}=sPwQvp0CR*^f7HdUvxfY@s&l;VAKdmXUOz)WN zH=G!v4|_jQpaT?t_S2P2rvsb=<@KYT2rrKp!?PLLii4DXz>fC7dzekJ)idYr8y<$L z#lC@y-O9H{C=d&j(jPyfrR*k0D>!&%kp)=P6=1rEoO((!Ev+(%SWrJBS3kfqIR|8n z8p%v&>!I76_*kHX8%k%FP8o(#PJN4Cb`4{iy82XN*pZ#e2l=^Ic3G-KhO2DOOytx= zpy>>mDgJmT+26h~GrB2+;VE{*G%FS!d|2`x4k@|PKVHqR#)J?X83AT(a)r1eZf_x8P22f(Cb&;O_1&C%C)2yGw9)*I>a35ZqmVC%JQH zYUbXVx9Y3QJ1gSMS|>t=_#F1+-V;Z3=+-*UbK-Y-#_j=76S*_QI)Qf{l_~ zZ6N>yPKqi3As-_YPX{+4745IvRl)2S>oY8sygw{}5}s`d6^tAD2*nLYf}pJMpr7#h zd@ru*1w4yR@@Yb*3oCh}0T%KVR5`#~q3i1DdOEh5 z5@A1e+L=6x2xG`m6LDvuuZFdPhfZV;?(-Ak4d!|znONuY@j5E&CD}RjwnDhyQghYI z*}1$gJI7k4gva?YcGw!72=%IVAN=4#HF~DSX!R|X<-X~2+^xw?d_A_wg;c+Eg(PnVL%b4@Q6VACi?H52TJD(sPUD%z@QkfV~T3-np z_Aree?tO0!D}yP>U2&^Pwv?1-bAywplWF)`-Tomndf>*D8hIWFl7%YVj-iSW%)#f` z;L90rna7WE5dcDDlMglV7}8Ae%;MFkTA8zn;%-JVg~^_4=ou7pn?qT?_sn#&MbRX+ z6C@@gnc-_Rf;(f$JU;gm9+y!u(VQ7yu2)hJtFzd1r+w)^`B|Vlj2Bd% z?f!0{sDF!bJx~n@_LC-+b^l5_nMi?=WTuK;?Wauc4FB1Y*J2kZRsVADOoLSEJ0MJ( zf(X-1xY49qVn3|`1s#{=lHI6`s`FTFULt<|QM}Ql#(edW*=XkYqMPP4EC)p}1Tt|MCddkiBS)OZ!aQezV0>hYgbHUtZ~VI!iL#6tWQiipf(V&?I|&38vjjzV z-_?v05`lI8RPeV2wn zOJ$tVl_h0h_hD1kfW6DS=uIO_VoTjuL!gvF2v=+D(Hsj#OWmB|Wd3s#Q@0m2(DQq((5*(_VRC|zVbl!a*EMYb_Ko!tjOHgrf}_MU99qrj`;)QwKJm$%*S zdFPv^$y1p%)&z9nhK%G&su5!mdsg7}I)B`&M){40q*USbA#2V2X$6b?sjR`Xm1Sqd zpj3IaUy)RKo!>aQ`BC;#A2pXL<7S;79l81Oh8-%+@rFF=oOAWK(2x4pv$w7H;;0R; zLsFMOa#UsKGyf1)=ju73Rq03AK-Ev(UYM6}6GJE^i7kz>HMTe$gzvgj*@-8djfAL4W zH%a640GOl)4!}F0p2?rgQS#R<=3bCuB*gkc6K`!?fJY~3!1@_f6c^*U(f8DD6pR76wTnK=dr09o)35-y- zP{b2P;~BR|)YI(^+yF-n*R7?EX51oy6Jdv!cXS()48b|huu2T6t23AcjL4p2Zif-p zORx!*m0y$r`$(0$f!S-B_+46o$*HBwvyHM@ytmG`$DdwacG%xk3FPY*o=L#f;qXqM zl>|y7WSQM*_9o$U%yhBFx9Ghu0*?7JsW>`r7#YzY?)8O@NF7Wul#fYrDqzX)#xfaj zwlJ1TeGaPZ;-sf$Uh*Rd$88G5g@SXP?E+h4s}4SdN{nHz_FVv!Z#jWmxDIx4R$?#8 z?P9DkpF!>-tPI365n@%NRfZ#CcXf$iO@wpy+=^DDDmvup1l}Aia`!GB0aef6t#20g zu7;}FHl4~c16bUWBwYN_(`~DuW}IoZ35XhP?1IwzPLaMJ216hlNqF-Fg$R4LRE4aU#!H}G}#-I9CpE`7wLsx z`P%4DI0h#R7Y3c1erx$EA=Eb|Uz`9b*kanM&sd)5uc>Hev?u$~#cTXgh=#krdq%|~ zr@l{Gqvoa(YURP*$K#w@Yvs~k!+3AEc4mlIaAfObg`LR7RI@m-66i-xdMhZd_??D0 zn|DcAyTIE#?QuDRl4Utcu&A@S6mv?ag$&SQS;1jrj~`;&-^jSMlUkm<@?*Y%;W$Gp z^fR1;@JH@ym%bLq&;B|N&i?uEGrm{p)%vX+`*!>AX1kKVNA*Ky7~;v28?F z4#$300)e9>fIZjiY^Y?EHd=H-d2dV+nNm#oA-3 zEZrG}UJ?Vo!K7kYb;gg*yq`f3dl$%8$_YNreUPI^@NKYdN zowxYU=>M0h_GeBkRRSO-eXj)N1W@A*8p1fG2o&#nU)xyyw=I;UO=uW&-N@}e?uDM2 zK0Fbqw~oi*0DWD$2jLPk4!({K2N``kJ28FEK!`CoHW4UeD3SMuV4P7g0Wm%qK9GI- zy5O-=fO62cfQF$^aVv}j{rhIn^Y7m(=~tIV(y#CFPvPp{cC@_|#3e~UN7c0(Wb|{V zn+D19DjZjQwbA)Cb>G_&$z$0hG<=(HP;b%beUe*|D{BT7e-9Ig7L3glo?2xkH07Q9Azk$JUb zt+Hp8IccP6#63~rb@kW)hx7KaN1k=P-5;0N_+7{iaYHGTsf#Q%2?Bs&mMsQ4+TPmg zKw7~Z1*@n#{6?+HKxg3WPKS<3uu3UUp#F{}uiFwKEJd;COZqZwy;84xA4OzAY&Wdm z^p!Pauek=b9V7N_z{y32q>ObpZQgMSQSS(civ9bijV_m_QS2Ffn_T?qSoxgk_GV79 zy~d|j>XJS$K??#VRrg#l4vg6jK!F*qMwpMJ#xer>O%qpIcQx>qokz013DuR8HmUhS zso6Og2k=n4*B8?ZrI1ax_nFuze}FlWKz#oz8mi)lF>M3<6b>eg7%rQX*KP2AqkQ0y z))w)?%48iTX(mP&s&B8v3ojS?&K7@3bI)FTz83=)XncA1+f?a#|oH z(0K9=E5E1eEEP?K^?er%Ysc{s6VTP66o~r4HZsHsUuUqX@tk)+7Q6?2m6xj~Uu#Qn zvI#PZX->L-@P37`4PPa##gM>gpgYP(5@d?_NQ;ZT$i@~;LCHCH6TGIlP^7;HWb?ua zbnwMs>tolgOup=#CUIFqNALw_UH0LyVN5j=+lbFq?}hN}wUezh2r} zE0R0yD!ipe!Y`xZFu=28T^;h3%!Jnd#4GtVBdWzXI+N@6>j(XcUNASm``RAOZ&-qx zzXdM)L-9^-5Dwe_8;d0TLwMM~MS|%sF`=w+8IK=eOwH*|`3wA^2mAqq6avD<@UVgQtr zOv;tdx1f3tN(YSsYnGgwezIb3&3*rpqr{^jrTvbg=OQhqbK}O-Xw;5*8&$NS{@DPk z;&=_tJ*%mEDhDJqN!lSx1qoccZEq(5w77oG5Wr4;x}rU?4RoADh$rK8R?WG#sg%LC z?{vC65vMxQkT}Qo%K_V8Rz&`4NGTd2MJDpj%DQ3OvMb{rh!*(Vx$lTMQVTWEB$?$((leg^34UXZs94+w_c{|QAB-XT2P-y*^MClbG7;#YQ3))kcx3f0*4jrwJDR=U z61Evy$;Nnau;!A}d}mHX$h92{Eef$)^AlEt%xC({8mn6(OwyT`;5zwWDsxK4xqJQ! zJE3jzNq5qvBpYJ5f&DYR#YJ`Ro~BOb-oScC7MR zyq8N6J%tvK+SmeS1UOnJ5ZB^UL|%ZBkV&hIC>N0PiJn{DuqG1XD$VipH;H<+=}%ffH8&BNIL&QYF1IbM7q zAol({7)jWN@Cbj41j~OSF{ivd7Bwc3hYJfrVs8RQ3I*&@0JQ~gLH_f++M*WCtN3%+ z0XfXUul!V3Xt=4Fp5!*^z-!FgxyJkEh=h<|q(3zj0X>b{ULFD|f+#pR@M9utIq`5b zeIgFH_bAXQY_vnWuIcbImT$j88j&J*hOD+DRL>c8$ETlcJcIGip_4&~(q`_W0EzJv zxWI%!s{VV^MP!*U7h3yRc;o)bCKivL>l;l!jdY^C0dTV-FvC0zWn$8`o zj`T3y z%o{Vx+sG;4m5)B2q@rUQ-@3Ak;t~46CD=!49KM6!+x}s_QeE;OSl;|=<^HEN@_(z$ z{*k4;6hLQkbw(BZKBVp42fmZ=-djTdkGFXrlndkOs! zjS)vcq~i|yj__W;jgCZ(yj{$IY3M8Un+mtdTnuYAyx5_IQSh z4Q`q*gFCZUWYXdeCmYeig^ZVm6-SJ52fDlRv+v+WuXj#8{E+!uU z3Tc7nxP4Vx;Qj^`wJ<2lR}NnWN#T-0i-;R{VwoqF^Ge2Ams>Oyn6xLM4}g^Vau&V* z85pyHOG!tUNz&=~_6KyWPPB|uoK_&JYaeT`VRjgWUDll+82Y*{MnUEf+9#42#~rv3 zn-#hB?<$o>;kU)gH&Ihh`j)IrGM{1|y+=nJk6!Kwe--7Z6N8H_AncL;=k`0zuhQWh z?r%$ne`HCofBI8u93)~;E0=h}D6R-p%M*HbB_sL!=c9|khqOs2y~RvYt8Ep#p;#E>0#wJ=xcNpsBRR! z^y#j|%JN6^b!BH&8;RqiviIBiGo{B!1LfCi6A=?cvrU{Pt22jh)pJhw{0hXf^{>_X zwi~t&=!q`BPs`i|t|ne%+i;wSB}J>B-=O780d4EuUF^TfduupriWI&wpQY9dPJTk= zPMuzHZCzDG_uouAsA~#|ws529sF+U|qh5zAc&?#}YMtqSF+izR&!B(o*q%18@!xw0 z_0~DBDtPvGNRaDz?DHS)?nN#2gB39Em`U>Q?`|7MR0vv|MhO2YcIgoE8HI#W7vL}2 z2J{Gk0P>mXz2MykNMfzWJcm0T5W98Tg8jf_mV)Zgqn<$Rl+WJ6$jttRp0WWXjjhxq zIuAfSFl*gb9WYH&H+qE`pB(%wn0E!)4$} z2DF?pw-|PRvNve6p%quGVIFpK`E6Iqe0p%(nDvH0k2S&rN&<=&+z6Zoikd|1eG(DY z0z(c^kI4^hKm=KIz7OprdFV$&2$-?dSe~iAyy$4M@sOPg)Z0d$1x5KG2BQQEiY+N6 zt(?fxv`QFXr$BBaaE+$igHb`1?|NV9;~gmdqC@SP^ky-y=Guv;jIKy_yQgoZpLiGl}7Hdg7N03 z&o2J>?%qjpRCyh(*KG~s*aMYih`Hp{tZiq>+%bH4KVoGH_nka7PCmLjH(-@+6M z)r({huIO^H?IB(KF!uAN#$Tg7#KG|(WVi>hZ%wi5p``<9mvK2SaVUJkScSNGjPx;l z8j-zzC8`he}pdx+;Wm+SJp;t6auV ziKS^n9Jvn~CIk@$)7d7YCL+%uYAihTb=dEOpI`2Ls~fm2T;7K)@OdVv^x&bRiomh6 zWBn7@T|mf3DAuPbhEhGLURi`6R>9dk-Q+2JfDlrqJqchgp_h1gwqNwT)xKNH{$gZl zkbgL%@<)`P#5VBC<5k~#E-_&B3=Y_H78Wfb8fipluCOy$UgB6tB|0oh4)iPd<|$o4 zOnxl^n$6gYVzt_FUoL zD*RNd-vZuz*VfabaP-V^a6%T;hy)JhA)%1{9An)HRHjL#zVG3t51q{xZ*db8&>Rj! zUqDat-*;!^N#?W36V2q*! z`hSPbgQs%zo9b=Ez9G6fPr(!-QQ4%46x=hyHBg3d6-l&0y9!iw$Ltk@?%CwS8s~lQ zIR2KkICIf>3R})$r7lrg`qj@l%$0pL=Xe))Sg!rP>oV(MNjuMKD@YbO2wEYcx4&-Z_Tb+#d#m|Ag6Q@4(Hg%-0~{{U z&~JjCe-}~y#pB<+`CrR1`+p)ar>rshi?j^zwVIZs(TDSq5>FJW9haB-(zaToU}Ik6lC!3Y5u|DZe<22%rQ25>*WKK%6BqN44Us=#$Sf zy^0{PbsvNTb6?q40>Kh@3@hy$(H_`?96~!TpG0$2#QW_UC8m#h45@1wx~gW9Wc1`e zI;OPNT!x1~45{<}lz8V)c=Y(rg+nmc;dK8C(d;?-cVs&34we>nl_C}IFjv!JxcN=6Wn@#w{B5|{1-A~W+dwWeFCy+1hCmC0rZ zNt9DQKh11~)QDIFNG}mHV=o;}8FW}s?N#s(57@r(c=+Td$r?8Hp;=IKFBP=69lCu9 z)+B}_`0~3dHuVwW{YB(|{2x#x5uC%p{2gog_zxuhlvL8n8d3>=Nh-cKQwp*^zr?MZ z0OG5U)_IZTbquW-FJ*r!Dy^~AF_*T8AVKBlN=nt~t?V0$MM+|dTtofAK5IBe(%InN z`e+{`Jyf?q{ff1eGg0eS&4{B!;HXz;H&WHyBcR~BQN0^;G7N>Ozo-8(lzOQ=$v29L z{Mc>|ziAu?eBaymSZtgMA%{zj?;WEMHT5rOi0f1|hjY&SNY*#$O~hQDsbC_l7W(go62ShLTyO|BXt0R^vZJeEx3 zs)~Llz^P=Grf*?Ed1<(A4H@84P37%M@cDP1H85K)4XFAI4{6e3y3DGkWNa25OwT*dXUS{h3CXaafG$G+3xgiLs zX;gjD^+-WxBnZm`)b@Y?iEGOt!loY%9JV+>Uc_D;!8V0sCN!Fg`&}a&N=>b>CyvI7 z1PbV{!LoQBGqV%`5t1hK_?g&LB8OdZJ4S+Q|AVtkS#Rs^fdmKo_70e9?>+l0X4PZe za?R*&0@reyehNSXj^uGfA4^;gi?n_G-s6?^FB*R`IS*v` z6%#npCkK4eBS!S#YP>yf0S|S*ghozQ(p_)lzfZY6S z{Y-IMT&VXP&DAGd*_>ojCC2^X3w%D~_8;B45Yes`za*9a9TcQVUfKZce`@)EjDAGo zsv@p3)<9N3G&su+Ne-H6PCWvhkkyE&nMKW4Pcs_-$um9qC4VxA#8oy@D|xOTMjtZH z2e54{2hV%Ta!)oEo& zqge(#!Xcso;o!L7gacxG1Or6jqR^eud6n;^HZO6MHcB^AH(EBVx)HjgH}JcqyXm@H zeV2Xpx`TC<7|5|O+bCRhY>O~(pe#~?CHTeU$Y9{(v(@EDiqY{_oscbDtFBNS%%AO?>CDd}s}9jc3-B@~Se_K zWzskBAg4T~@uEKsr$^Ab*if{M)Fk5CugjaYSZ|pPVRZ%xXN?YrmIT+GKeT459(^VNBR9=&$JReKjn z;!?}Z^v;Dm$US#OZv|g?g)6k_G5vmLpAi8Q7!}JA4_@~Z=5pg4+IsL%$3L)g0p4!4 zw!*b45Z)h6~(dHE>>L853*_zKsbHVgxN?cY7_lt9Bb)Tr=DM)L&F@N0+7GDb>Z`tQDYF;TKb97?9e9<}bR3vfJtQB*R(;=9?#yo4 zD{F9{D^mLDIcHq;KsJ|NqsxvR^jPe&A3#n|vm3 zpnc6%YSCd{r1I=@5MK3tTUkdg<62qPb&jya*lUhi`x>(pTjy(U*-Li>w^NgJz4OaD zsS8l@=!d`-MLmV~*4+i~175mexxtZ(ZDLKPT^B#R0r(Ge=prn^aGrNE%%2&_@qqP= zf!OP|QzOM#>yA@ns^fQwehwkUyV*3zR*ts3$Pt{ubv3aOtMBx~S;Ld?9LcwpxlFez z)Yl|9KX$u@pV#^^ha(SYL*skAN-_3(AzELUp9Ety8bjY|BC^lEaT%tge{I!VFr!~o zuI!_uuYuO&ifW?FJhS+;WAYTb{__KF$KC0@$n9MzOL<)<+#c|EHc$R&2=fRM8(`u8 zv%*P=dJFrdO$c z3%Tw1JwyN*ayw&zQQ1w*vn@X-bbn{MB(1VCtummn*4HSnm=k7fIk9Tq?PZ6X`y1Z% zrN`~`PVJWqtS;e2Jxv6I_u=Kf>Aqx;0{q(Dp}wtf07yv4Lf>3p1$-RfS&aWEO69j4 zWIOb?`J|r3#KD&+?#h%ODTs22!zHN?Z25bH(M$7-_hP216nqg6OuH6nijnn8YlSyXyn17z*{?W|=ib*de%(dPe9WtZ$?S+MI_3o-&mBfbJa@h8?}r z^o#t!S}w6hZEe;>;f7HN#Oms5!A>;@tUgw)G)0rXjq4^nZ63BIUW-%PvEVkcWzNE3 zY_dXRswcRRp|1p;V%o`crd=J2YfW2mj>P2_zDLMC5N0mGgfj~IfJ2-xZG|m$INt1< zmP6-Vw+m)b}Ryel>9gV7#m z`X*$+fn$y3PG1u}5Rz4ji`#`Abc(GD=|h5lbD!K`g=fJKU=xph@Fby`|58Am^E9-& zNvnafFAMWs1T2NC&wvlQb(LH+O9Cfl%Q1Rj#{e8zP)BMcn(vE}=%d|`Q+7YvC)eQ# z%nHxYBv)(U{Hw1{rAcH8iai^V9puJz1`>}3dw!Haq@j@jv?(O-LBO~Zr%KRM1&CW=OKM1tkkX+z6 z(rNh`D%TG^0^?OtSz09KWg9r|aqAKM!F$tS9)e`oD6$aa{E776$TG^RBCdB%Y3JG} zjo>axUUJ2t*AmC158X7E>^^4lX<)(Joa8PCA>{A-^@;jbV#Clr!CLX++<`}=eZ)nu z%<}@zXt#gx5|0ux%6}|@U_(CBd1pl|w1`n!K^HMa0DKr07QorVlq&{=9}ysMf5=Dl z2u2}Xz?N1v1h*y*JZ0QIrj(hdfR7vE7Bb}BFJ0^2f22RX?#JfgBGLXv6$-k8Lul|_ zE@EjD-NoC>rlM7x*up;g2?nLO6i@Ur6bhF!nXkbZ_gii&pmVYe=gg^5gu8v>;*m-RC6W{K-Ah~N>^dJ+|8j7sj-R>ZmG zEiaB-orzYK`;chs(1xPT2Ng<}FP0X>%>ji%r#u*6n{D9fLPSj;_p{-UlP=@ufg;11 zdBexU>1CETpMI&RT1Wb#NuaKF<-c~d|GJpr-`Rnjsh+i?jg_>XwVAPzgX167QGkM$ z4GL)EwRfQwO}(!nRGmhGhPFQKE|ihcG!Hg_T#3*Vw_%paX-U|bVH5Gb&yS4_hw$m` z3&Qm4M%ej@0HlV3&PhhI!wH9p^-1HpP6wFl7b}EYRx}ngRxBI|oDoi7kLow%?$}7I z8Sf!dq@DzbTrskk{%g{cjRwN`xW%qm%ah@uBW=1IlF2ryT*BC`G-pMBjH(e3k@Yf z?+cF{A3_@~_2%=7z~&Z#g8O3E!9)s9h!Ef;{ROFxLhb8jA!{8`#zSL|YUadpF8DD; z9w$uQE%jdE4a=?76)lH-LZj{_#@f{gSQ>{BqLh$$D3sXZis$AS$^xu=OT{Ryw z_AReiL{${-_}IoP`I*el+>1Xen!n>F;{?DncVN%I{z&b{BXQpXPKhR)47fBEtgO8! zl=1*hp_?EAG?HA7zBcErOrQ~}oVb?9;8keH&$IZ4cWOG0+#QN=fNk|N_2W8fTc4*DtN ztuUr<1m-36(i2}NDsqno`vXFKAL3kA3}?WEHeKNLtkH}0ZX%KzdNZMG!Jv#+{87m+KM%0V2b;>D zVw@XT8bXc(IM!a*xG*_^;MV*&OPaX3mCnC^O-R|2gb7GJgl39|EkJJ+8&R3g1L{_D zV~3tUn2~RB@xm9+oPXv;Y(fold;ipjElNT#(yU6R6{{>7cj$3iI4ZZU1*pRW46vvV zb~GiF$at}TGi)Fw3pB3eMz7iS@f?0!dTq$-j-zDvSFF_n!_KuT8rY9Q+glV(pvK5h z_up6eB)2B8Y8V6+sfXtOPESHU_0v(p7xH}7VF{<1%4g~fG$F5QVWYFa_Hn`%;TmaG zfy3Ww)>B#4cS;atsrn0#PXbxsU;yonj4h1}96$W&rST88qVW5+GoGjtv{6d1(DzJ2 zxn@O8rRQ$id@FQZOAu5OZ!t}u9F3Lp%@8Il^+0aRmO%{hS&Ak$E z+rG`J2Nu4QK=-?Sbh196tN2uG?=*G1YJ!M8(-{rVcjLJk+pYB(Vf7odD4_D9{_JJL z$X2OE1;?+mFzx&PSz*cbgd0d&v=-+nd7bOqec!0rzH24z8G8`%Bka3)VxPCgMY>ZE zk8;_8fM|9v@YHp9!?^XEsk3Q@bu!A4dOaY8ES`zp21N85%s)^6+wUF`68z_xx&n>HQ{Rh)N9lP@3}$(o^~UjED(8 zb08m1iR(-di&1Ay?Z9NuVNbS5(hbDoH4TRHmTIHuX^|J2_US&wFj&gl^xjbX%BEl< zL6S}&wATJcfBe-+H0%bz!ek(|GWbmDVEYGLT-P=Wyiib3BuVF#u;f^FgZW#rxqaIx zjWccAv13o%;O$%6ed$6;XV@?}KowhWGkN<`4f5b##LpFo3*$rY5s|5=$7zPgq~``1 zvSRonzQ84{Y%?Tmfnf+k8E5xm`$WH$0$W8S7(?ti|7`v1Q#LHYmTB}{f3r2ns%aCZfe0srjZ zlKaDP#)?;3q3o+Vzfb_dcY<)&uQ0?{N>_3vKc&NsBwJvIMb32g#jiFH&qPBeS#IP! zK)2I7CwrZGc>z7aJ`Q#uObU$lR0i-sv#QdwED{{p^03r&PaWKdFu(LYCy$l@amRAy zw8}>e5@0#V)DCaR0l<@un&=#q5bjn(2512|Vc|#WA>{!<)+Z01aX|tHsAo>MU@ROG zqP#SH0-+^*M16&HI|V~M7PQWY?pYRZ<1>kqv0iZsS?q?-gL&CZ3py{ow)Jwg0@RFA zPrx{aMQXM3oGiW~#VIC7k_cC5dOg1u1AP(8+jTvrl}-g|a6Y8ef)nJ1O{EjRkqXwO zQ5*v=Y!6lJh!@;&r{?+NVoZ9U zhsV^cK$}+*jqDWP0!MqqiX51+ke_nczMto?$}~>E?US!JK3AZnya&48-U}cfC|Z3J ztPxM}LNObVU>Sv{nG-q<8?fmebgan#foa5dfhyL-OWD96W21CYL`LLzucv(qMNrkJ zcvNBA z|G6XaYXp-v-2ixCs60upZh#dWcpJ|j|MBTRKg@3dS~}r9060La7)e=NMznyajG#C( zxNXSzL{Xna#sVqbWo1+}STyt!y#SRsM}J=dVuXB!S7j9~M|rnZ!hi#hrS}VUA8!{Q zPTEB?JEK6xARqffdOCWz0&d};;c9$8e^k25w#|>GbR>qUClQMU6OYLkQXrQnMUSQa z&g$7$%11Nrw|oWkNR^shlD6k)86QB*f*7gUH=i|{oh=;Pl}ck~^RjxdocPLN;*c{Q zz=xOi8noYqe+&4EUE4grtDgPBF*!@9gIO0etmQ042iCv#q(NTIojg|iA*@wCQKulU zq%|b)QkZw*0hLn>j+!bz|4VF-I2<{Z5_6NzjJ3Fj2J&X@}r?Bo1?{t}2 zQXpJ1w`|xu?e}0_%C}vEi_XBU3a;ZOYRw9~7lF~-hgmlttsw19<1LIVW^K{%|;T7kHI-Eg!cXIEK$tZur^KzelcX9 z;(32kW9Gp|4xu7g-VedoR!p(6Q8tL-N6%BHozNUu3%){rrBDE*p&iEacVal#G;H6F zihtSU&AWkmK1WPg`9q$)WCkZSdlsI?` zH49$OAWtI#K7yS5PK+=7gnmbaKJoLiemLRwNI+CT2)Xwc;P${rx(F21+4@UPlijha zABUiRTO16OJr%|ZeUbk1&}P!yJxSugcqqtJk<7#%Bq+sXKHdQFa3-NpGN=O8?*>EY zk~NS+WMr}?5F-xq%`uxUWq;;4KAG${omPjhq za}L0?RVxwJjFDN0pG+%#I(1cA{O}daDqHj6Oj}K@g>BrS%;1Hg#@?s#;zH%4NP}Rl z{kTUO0(SbFd!t2KbsnDYL=K+SDrBr%rM=QgXl%{PM?knC$l3e~6@QXuDx226?#{Gn zxXzMPI$Xd#P)Hs)d9nsq=BNkhsOUcUbU9-aA}>TyG{zvTXqb|v<;lMlgBOVWFyLoM zfDL9Su*ZZQ3ik|=HE|!6 z+RYjCN4_A4+E1lPU_^Wm2sN{qq**zYX!RxI;K|Gf5immh#qb>2*%{U z7C{@{02-j&s6;Oa#YXWsmWV=G!ebH50yQkhvLx}PB?RSHE*OX+vA30xLnNEQO2vdExW1t<-Ga%BWeybueqX1Yl|v5m&1oFg3W7&Y z_|_io+S8spB9Q`DHEqH$(zb*-=m-XB`Mh`NSb!nS)-ZengO6ez!@laWcXEYrj41%% z7Ocf_)8*4bbzFGzb4Sr_sxk%gdA#%z#Ny9U`~B`Xl#wzE!Uv6D;$MT|SK*)Z;R}G^ zElHBh7eETd+xNqe`6*Z*fA_)2bpl7rKuhlw==t|e*rd*1{)YBW){bUYMjvFHEG_@x zuz@e#3J3BxOdnN@3w~<>-z`ALU5$a;Euo*YsIx_cNe|nX{%S;~diT~lzG;nqj48^9 zcYOi33(gP8E|?U|FX{c4$&HC!lR(`H^EoTAH8#v)hLtjPLyfaMM zCPa%{7;u_Ge`GD_KMavE!H!sqK(99V*Vp?qNx&r0`hX5&{5i*@<5&g?We+Wg=t(1V zr>y!THcC}NbF5)4QRCuEvg4)-6=f(qsMbKC5J#`r=^5i;{+SN)$GJSl?}Bw^wKUx` zU;I|B9N=5(gW?Eoghfu9OwMVVy4cMrxO{?O>^Y&86{NTzsgEu52rz1O>9y5;JwOm; zW(3MS(ClX@3o*&b8d1hEL+x<4pabUl0=JkDM2I_fNwTQNUj+iy&s$OA^2$ebUFC72 z`UA#DL(hcy(tZkHH)jTu-R^klsp{N3DpIo<@2y2NPp*DMK*ff+H6ifJ6j~>>ns`09kl zJfIBPq%$ez61b>y+&EqUs;>g1UCp)!`5&7yR*| zI!_`W>Zn$u>9`Xbeu!3Xucy0RvsAr>E z0C|8pb45jZ-o+Zk&KgbpxhALmIK~oK>Tj8CJ3cd+9yG#@e;s8|oj?FoFf%X}v9!_q z=Z!IPzg$Pr(ye^F<=#S~i*cKpgW*|TQJ?3sG4hK)ak^?8luC}U`ONPDF9f)CS`Glw}c>2V(8f=w>J^CqUT71HNsQUr^bX6d`?E_pU>hE_a$frmx4NxWHV!Q zqZ^Tgp!E?jVD`FK>T)#G8_*GVUy>0z!6i07?Pid20DiEIET`_kurgYSfNqeaw87)UPFaCq9;Ca<1{2JrB zzmD;rSrg{>K>zDvo&*Igi(giI?8C-&%YC0MELzWLj(PLZSDF-Hu^`UDF%;lGA72`B z9ejI^HdK7D@H}fq{7gs<_woi9i+GS>R4?bhH9GeE@#hNj%KgjbDEk|t&sLf>cB>+T z{E(G?)W(`fcuBK`qb0L_-0IS_dh$TRq|~PHOMi2hW5}HOh7oEEGA38*T9uio+K(wR-#XePt6SuUkM1(SVXl2(%@HrO#FOz$ zrFc^cNjSU5!1oAHzw>mH8?x+4e6lt8VYGN!CskS- z$C=|$=+U#)+3t`xBwO$9?OH^*b`nBhef~M55p;tc{LYZCAH1pE4zbCl5oUE~u6y`^ ze9A}F?((!74r?7?V@ojZm?+p<}LZ@xd z8>>hYzD`|SD`*wFi8Gt{_t~56vsVXZ56`jIKHj!Dxt(Q1&VXsPPaclJqvRAwcuhD&wvon z0QD==L1~KinA-&VDiK8!2Pm*2qogl3(b(F}C)9~-x_rESqac_-46@@!f$lpZhgP>c zOIu}sTnjhxe&%?E)y4IJU+?F^yrrpTEb5+}D((?*dIw-da6$|-byw||MzjFpu-VS_ zYK?ige$fCpS1%bSBeHRcKA}b_(65VDJ%|#2j*`Ck+T93%=YQ<9N#!@ex{McnbaPfk zDj?g*5F$UI^1Gx3Ii-Y~vv7u>dGu$0xF9$!5@5zmFr1(3<0tYM=Hq5j3p8h^xlnBr zt1p~a=6<=0C7k3|l@i{1OUMXpBD}Me4M#(sfjD(FZ%d;%U>?_Z_R0+PiH@(*qMWSe zZ68!io#`Y?33q-JOt@72nlRKnVjmc>pVk%~sviDwyl~q*mftCiihIAjm}{k4PZPo7 z&B=NVC6zy&HpaTr?W8`7P^TCs!$-m$AgHqsFEJ6~B@iD#)>A82Lyo&h+Z(AhI3Q(~gu0kZ2TsGF zmeUXSd;U8OR82jsysP8baVlz|h4YL>^x4xt&XvtV+I<-YO@)%b)S&;I38wNe?~=-b zK^F`csKUA`&7r@X8oDvtXp_f9R~w3nz4sRnhJfHFqBf4phYgW>BdC-tr5)QnJoYh# z9|?2iU?$h`>tbnzRx#^W2rH~`um)ufjpgs9%7^n_b|tG8n6nORUU6gourpPgmu)UD z9w+TTUrd@WJrCN0qQE|C(bjIgqq)&I_oDFz>Y|q_$J4l~b<@CUE1#wME}%b2bx)!{ z33Puzf0F3dL4T6&UPpft>~4j7DcR70d#Tv?1h=NhPzggRK-rTUEG4{%oAb5!_Uau7az3E$EKg^;n4wUltCMxwsjY>aYr7-sm zP^Fl%@0i+HKPy!2E;w}QE;*EAO1UY2v0uM2wE3Cr8B^f1>(VVK$`eqk~J+8$bZ17tnM zKH?GL0e$GOQDx5e;}rv6B#PR;N8UFD0{aQLM(h&stu@Ow6+h;0Vz2b-$N#!Fsm4{R z7v43o-+?yMPAZJ<$=dtwU~L$>ygE$p=Vjy~+P##OzS@GLse<;)BLXn1b2 zQnLA>Nfs3wYgPz@U|NXnWZvI*(zHi{yeIHSYO-G|i+tarCf>XdV@GZX%1TuT&<$%| zh^nJBgo!&lvhEHn>_hDxO4x_GJE$-l^IR}lSXGHtF!kmS zXfTikXh;x7O*Cnc#!WODjUFzp7GP_-KxwZoE|z+c?Dvq@LL@J`pStONgFM&1GzQMY z?sXJ@LM!LcL3F>OjKC7N#ojjm@f@(vqiUOHM z7Pk7})8&iC3JX%sqfQ~Hq{W^W@(qnnj5)GX^p1BIui4P5By%#@HAuOHZRe4>HL)53 zFk0fOuWPyq)Fn6*{2OmAiv+7m%G02#vY>2|p=?s2vaHuJd;<2*Sk5b4frMPBC?82G zHeuy&5Z?{tYQu?1mjQ!+&SidY>!ZaEXi5GwWdUI}{>tX%(1-i(u>Gen@tT%mUPAJ5 z3e-9aLWi)fJWd-2o%D%gVo`NUidT4Jezz>7c-q*#bzFI*STVaX`M57?m!o2tY(1x% zd{z@`#tzFZi`7=`mB$26$p7Q)Er9Cml5XJycXxMphakb--Q6v?bAr3O26rcTaM$4O z1P|{1AM(!3H#6_Ox9)$To~lC?)I)Xey?XEN)vEz76bugXBj5cLM3V256Xj4~z@6=5 zA?nuFT$0vY2BdvV-P$jv6_(Zvez2*!a$+-%C#{VSNK2c#!4})xy(KuGexI2!7)Z_@ zv5UE=E3C_4kyV)n%z?~n>30!ci~r26)w&|D=btTy{nhvw&}ze%^+SLZdhbTBy*d+v z;gcr7wCWCWv1svoSf9V*8&ML0HLg*&oKpNksOXDd`p2`$4?!hG^cdKYqVsW$nYM|I zx4}`BQM+ibzYe7%(?H57(IW1zDFnRIcEUV;QW`yaixw~tB7*{qw!Ug`VPPNy1_dyY z(i@7dDQ8x}p3Hi2q6k>KXNQ2-TC9w`O10VmwDXD6^hL-aX%=Z?_je80A+81au-H9w zzpiyP#!DJ31n%W)$74(tU`z(QO!+~xW1QZIS*oyCMBc-4J>BFGTs(2n$_y=6Q*t<5 zT3ay5+;&^+S22xGnCK5%Oq)V>a)=5Au#EgVg&vj_D^;gtPTrd@SWTV28V)LTGY^jg zIM^2bwpdT4{&Z?)I@ynQWX~zXy=S;oF^^3= z#NHsHo}bB%TvoyVCI|B)&5a`INZ~;WU|(&ossGbaYjkLgJgb&T`6$t2c`xb_0*4gY zxHgeSb~=il>Z?-9#6n2uXjb4vTH_(inAf+byI&ZqkO!DurEC4OrsE?sPm2m>%Ec)f z3{v!xq=%dmq6YJ9H&UI%^3%Rxd*4jHui_Ty*NjrgaoUFDWfpEP*QPm{Hf;j?FVO5KN)prPfkosiWDxPepNJsjkr=80_0dS!O_xPuj<`k!QA#Ly5NE5bLHQBfPr#26K5bk zB|V4!&cM$V9ACvucfb=!o>5OI>Yy&>+gMD?s_))fcge~-7>6o;Z20}ceLnw-i>b=g zmpLjcJ_m0Dv|~i$6XxY{f5L<@af;!FYu=cSk3S2mJ3XHhW0!`82tyJo&;XB2{ah_l z`ce$NR~w`Ze3-IBIe@OKU*;3Ozhb{}SIw*-7zBQJ^5ufAX-n>NpfGepHcLnyZ=j1# z3S1LPrC=UcMIF=(?1xe&FKSI5s10XgG*k=I)TQ0l5Y)gm4DUl*xa3Mkb;U4jK^b+) zLLTG}q$eArW~N-J1$YE<-vWe5#{?172`fBNZkfjpc_QZq2Naf7U9jjA=qg1JhFu?x z&WM5;x-=KRa=34bQW`CWYS%!!8M2CfA&xfWBM`>SE}~@RCe&JoyRy%mC+>hFp`>m= zEw`}IJ$>B z44$69j*TdGqak_WWEL~r zyT~dc54JBoBv1yJ$GHJoULT&wj4M_&H%K3h)&@K^>iSy;9yx~5k7E2@)t zYpZHHROc3`y28Ef%Z`YpyQB)?Y;r{zo1 zJ`lzr-3s#p4=~ODyOh&O6Q z+`g|&lO!aJw8p7W^zqo5XoDWT-`AFUm8<;CEwlC&Bmj4dc%J62EQZ6bSmo>1RXC({ z8QJBucE>zVMd6=LMdnwx!dDo}+LAU-OlJo@cEf#i&CO0sB{GskeO9|l&We9)3F&|` zPS@(?jrpWWy`cB#mvJRyciYk*k$dP(w#$2^w`98Z@Uky;vC=wPT;^ zC^{0PbD5~%3ry!&Sfi@uNFWl3Fggq0(Mbczx3F&$j3mSYGwGKo z`SOu)#w2lwjoMP_nJ&UWv>Wj}zb_*dBLxOwtljN2)9Gi{?#asodKbsM3EhZBd@Y@^ zs31xJ4Hg?ZG+cRXk3^4x#Wc-{6;2)23{wO)JS-MqA;(yymxcw-vYo*{c;d|al+9+r zaOl_kXp`6s6;y8vb>>6%$toIl!RjM(7}1RoU2etd$J!X@pwpv88jwKP zOhZb=eslM8o)ZJSLSK?5!l%b5oaIl?m?$f9=L+_iwe_bAg=ms*h|#=#?U+Yw7ITgE z2Du`9ci-GwCv7P3_A~sq+fT}*EwGNL8V5{~3VAr2Dw&%7#X_)E7za8K3pmq+EULEB zwoDDFPdPn6usYlTOkT8G;w+;%nf}4A8Tq;tn3NAd+!PMeSISi|F}&YeKcKW9xtZl5IJP0#VkiMTOy5dK4`GL`a50D)0bEzc`@**-!1V3CBjKT}a+jL1@ zY4|L8|4bCpOaX<~Gvm?M*XKF=BmT{NtIMYIyLr*2R{dkI7Pu&&SyiQ04|o@c>sNb< z-YS_Ryu!XiUx;}n!iyNn{;rNJT<_93#u%ySZ{14uO^W^mTiB=D&WQDd0$DvX*V&Q) ztk+fF${|Zm2zldD0l5@T7Dt>Uc4_ncP-T4Bu>V``#r`iuQM#6R#)soxyn)h2N0R3& zrdsA0o7kfvseCVcxGbHxF;u{zcJ^SyRKALrypit>9cqRri?mH8o41o#!rUU zH&wBk6Z4o16tk~Bd)$^>{JG5T64Q!uqcQ-jM@BeNb*$Sd5XEvyM?k$$ifL;x0a z%H|wU;U}sb8;3{fFIw(5A^lkc?R|xQ@QXGG0;SQANw`E6Uc*b}U*vRl7VkdJe+e)~ z-xa|7fC5D$A13|GC^mnU^O48l4VLEH(`1Wp&yC;y>L1tR^=`A40NoDw8-)Ecfdg#p zA7%j7CjNWp{ezXN_(zzDFJZ6Is&wgv!Nw3HtR0761;)=b2b@28h0OZ6;9@ z*R+I$M1O6tD!^ai_B?x&O8aDze>@|x>B?n+!@bR2!^PQZ`UmTm%l%|imJxf~a&cdiK6ZGk(`eXW0U-yg-azd7ySpi|QJi_JfS z60lPIeyUKnifo*lUl0N({H}_3)VM8_O9V#yz4;4gn6Qi_D{+j`+i(;vI;QVeLh|)l zAy`+ZpM2&BcX0cEppx5_O@)1y&BkJJw!5Z)6joOpK@*YH6lP%*g*oIdx~q@i5N~C_NKSx#SF_G&YE>YMn~3Ln(uCIc0864nla8oF@d{` zO<2}^1pR1|NR7Breh4aJoD+>d7Lc12JwqpR4VpgtGTSchRu`ys_3Ijc`55HN6UXoJ zj~7NssyN>J4;TI$KR*h;4`Ji`udVZ6ghU8CR|r>E2n1IMIiNh=A9X#!;^*hg*}flN z^PWNe2<)X)$U&jXE7tJ5Y>ycEkgY|+c|$pbez4XwlCz}2B?UzVB}HNZUF#YGVf*%9 zqI$nf`o;W!n>ic!{8vPMA9e=y{?EOK(iFm4IP4VqBzP>=_t*YKha{rp>ikZ{Jx9+< znG`4ZA$M7TnwGsKFwT$rQ$u{5A7X`8XLFqfVL@6-N-Z2djsc9Kx*PC4VSj%YaTsv0 z+CSg*zV`knbeo$x|0{aL$O9+SA@y&t$toG!T3=Jp2q9EL&UmVVA;Sn=Afo>{Ag#zQ z+5|q9UX!_>gWl)%$tv6HnWf&XkDWigK0Jeb3D3gtCgdYr;qUo?@Ph-y2I=$S2_l8l zZI2OPc$Th1c8{0c9IRNtoy>h-2--=|LeEq_b21C6-+n~2=LS@|K$XXhXS5o#g_l~1 z0-HG1tWfSjL#1l;QilP0Yb3H*2bQAWy4_PFqE$tV&fPl=O{{)4QaTAIu9SGIL=J6~ z5 zElct71)dU~iL%?E3q2gf%v~qipH`hFoKBriGX>sWZ`?klT-f5NZ@W`Q%kLoJW#E0n z`{u;^>)wAyI=Y&xWA8q=ceCV6Dl{2WN*5Q*w2mB6kOw$3;78?Tw?7DYI-3W0S)WQ~ zf|(-m{?)XON^Bw|wGNIj-V$>pG37Woy0yD}I-=_80KFx@ngb>+jg*F3Z4jc0cknTd zb4A(WKzz7YbI~^cy1L>(YdB2wfF8@&{?xsIeFQF?!tk>bzg}^+3OM1j}+(p z6EJGVs17;(fox}`qzB~t&jbBk7ecPhk3bFZem7duHUNrt1Kw!o08-0T1x&EXlw;qT zYd*=Ut%qswSV_$_QVZOPk#K^9PN zli5-tGU4|yBy{H*cT=O8>eX~I2|}`+d3rdu^Bxv+HC*ADlV5!_6S}wj$}bzeBA#SN zi~`7U(tRv7o(hpjzGHUzy2SMQVPhPR)(p?EPC2z-)8ab;I=l2znfOvCjMHg$BiGpnL&`H)m8RPQEc^b*bAazO!D)PQlk2Ew9 z^0=MptM%Gx$~rp)_&@$OUh9buLWeEKm^X_B_}K;+4v67+K#Fo9U?_g6iG&euP!l-5 zgggz=ae0n#RR`OPCDtw1XSG@IozoJNp5v}~3&FxDLY5?QBTHf%giD(_5Mx67S%9)H zSxtY9ZXXh;QZ&3&Nj2qXF+yxy%N`napnFup_&ttfJqnb-F_`Y`uJ@xbF%r?O66&kuCysQ;qO_8(A&C>_dt)))(3j59 zW(M`RR@ubk+X5*g_0=|bwu2)}9?x(+Q+JEURF%?5@G%snJevew(rjz;xq>eB;bXHUr8uJRXfSSB+84|xlU@sSNGzms` zb2JwmsD}ys7_qSdzdyMQu0}tYV3mrU&J;oe9t}ff?>u_FX_i5W3P158jZXx7A#nD_ z3#0O@G_r|QR96u~L$vTu!v;?khT3uPluuy2UZ>oo670P`P|%=a@jBil;pJJ;C!|H-w+ z_IA#OmUjP2L;O?qtiDiJ3&Rq*%sW({)UGK|OJ7=Mc1RgK9tHK0Ll=^zCTs2Y1j=#s zG+3WSUJS)6)y`V7g>qktm)pde*F@Uz!^1I<^&($uxC8Ku3gKvu8syNyWyjS+=CCs< z)B)-BhQ`>F8Zs~Z8Xl0*yaVKRArKhOb@P-V95s(CY#LgbjMZ$;FAr|3uz}>p;Iy?H zA7)?(PgJ%WYB&#?;iJ;}%$B)er`vtH5GBGHCbD#f2#b4Z=ZZA3NtDrgTO_V3TftZN zL=N^tO9vFdXiL#vMH(KN-m9^UH6$f}vh1YX@tJ~!O6wdMs(BQEg)5Xx9q3q9!|t{j zgnIkc5Xt~uo&cL6%nGwJn({`P1 zAG%5BG&%Mj`XltS9G~FBN9^!#Oi$Q_h+r%g3iJw)I%U$$E-j6s(%eV1H-xDVFvpu9 ztQ&y#NA9BbC?~XbST4el^c^s_(!DM`o+|i|JKsMbPV21^cm!NO%>R3O4Di&(_#Y>? z|2yZt<>fZXsJKRXqt(k{Et*QbCjUO?MIUs|wkHgp7TnPN$yMWQWffZW%pUa zW7RE4UCc%(lJKNm*nY!|>60PK!}jN*qm~J#`Mft;7)X+7r9NwUpT;<0o2>mPg&Jfg(Lia6V9(M+T~fEzRzHUj zM7JS*lp!X0c5(a?0Bl(O1d2yF;G%?vpVy z*W`j%nwyUT1d%_1IAPl~DUyz*zqhmQG_mTn_xkwq0{L~Y`6sdiV0E{7aB9=SvBJql z#u&xvBi0@$k1m3O82e}N6f=b$q0sVXl)PwMV!K=nBlrknJ7elTd^*-yobmZu-Z+xD zJqbM5Xw~1}nCQL*IaQ-xOfRW$SF4JZ@)Xu-P$>0rNJ3RrBL8f6oYoe97#6#N@3k}L z1;BAO?giEviTAq_0pdlqHQQoH8b*|DzXO1^#zuSeUL9#%msqz?Bkl27fh5+L==zhR zD?w!>?D1PqA)l(X+mMAP8(SjgvQM2-mfOe5PfI#NK zL+Dvm#33*|O_V$l@NerFXRhM0Fv8P;swHU$h@zC`Z&rb?fL)M4XYEJ6TF%#Ajo>7Z z6UQAF^{`60y{yO=Jv)e1akwCKl4H|t`;VJ1NW~?>G%sYz>%KM&)zd$}VQVqRne5kxz^J&EzY%F=$EiK^h%d3^e^)mSt5G$T5T5xhg~d*pMx8b!+Q zjtKRO%d2Q(?DX~{o>h$TsC7fGgijyZm41jbu~y=oawl8EZ_ge0ocuUrk=fNU?l4>g z-#Jd>uo+S!B^f_Tw#<0N6ZUTGFa%t|^{AT97#wJ^*&1_Gp~JgH&6s1^fwM|S{i4g5 zhw?+f&)`30zHbFX{4W2Y(`!4ah1;h~sl>fQaT}udMz(i{v%fIOx(O3g3@aX-@{7w( z>oBzonzwAH{MkJ9En)j7VA0gsYPU`mJ1u=OLY+Y_UWg{$Hw-JwJoKV0k`+)8Gk+gj zrp6SUU1Vju;zBS_5WUVx0|y-syOElG#_YT@kzRf%MoiTfyswyzCD(8uRw0hQMx6xu z?ei~3`qL5FUK-^_1Oh~Ex!?*k`xWR4*?w2=e%KM>g1ja4{vAn0@7MUew&1;x7}~8u zhqYRth1!9PT(%utPuT#ZfE_@%yeV&^*8$~Z*2kPs+BIhT--&}r=E96P$#%GdD7^R=CCOeLo|){ua#pI^4Y3v{o` zau1&d+lERQEjON=fA8vp5gSD2aQ%WCmIDD5P4bs|;?~Jm79e>uQV|hHcW}&yPsOsc zRGarVKn)~~KWmC2nzkh9PcAddOB=czPzvS~ev3I+74&@hZ9l{>Sk{5h#fhJR!BMmm zHL1LGPawzVmq@N^ujUSj7R32hyU&F$s^;$Zi@|et8CRBv@5vd2mq*Av=@nMRd1~bF z`Kx;p7i=Ht^T0ub#EA+yiVJ8n&+->3HRcbscErjoKQJl$8K6T6Ip!`3Xo7JkyW*~0 zl$VYV^F_paSnk-BDNDV$9@tFgg~UmCN_n!VG}6aX?r_ag)uNMFiN5mRU^-tV8*aV0 zTQrMYT7|Zp|3QwK;FqqYM4Mq1l;{R{<4Otz&Zp~AG6 z_jzVgCmjQ7@;gKi{!BR%q&SguG+Dwq^&#SQ5M(-}%&3;G$0^}Wp>s4H6|-Y3Z?Wit;R41t znM{tmm{cIads)iP(;}@bSge?7#!@4B`b@*ulcv7Ft=Bnd49>`0 z2wOK|m;Rdi2z4cIc|5y|}y^{3am;*}^B>L5Zt390nRm8?d=l9;v( z$>JF_N7`|Eb+5|OqQ2@%>L73IY1cH{9$>;^uC=oIaxZf9d3qHUT{|CTLF%3};(4^g zdik*Y-brA~YbQ56Efe3O*{ea{ct15sXMWpw7n+bJ?iarM@}=MY)tJ-5u;T(}i}?AT z5trrswmNJUX6X8h#j~(#-Y4a&#qY9jVS#1T<{8o!GU*)OewsEmw~^eEO7?81^L>ET{C+2xZk&{mt5 z&2dh$Z=W-OV;4P?$+}%C4i_L0iv$5eO*=*GeC4=|&1R>B3aC0&u)j7R+)?uSdunTe-@KsdTTz_X}R_Z;Yl;;Cb#70J- z0M(;%TxS&$SX7jLd*wElS4*hSwVCa^KlB_Z@CUbBxzg3Evgso+#SWvbzureo-82SS zN~GpD*{MuImPo>7Z|)KENv2|+eFt&qdHo7W>`L0V&MhiUgLnh>tss8T!V7elUMptm z0(Wtf*{)sq1-YAs0l8Fz!V49EZhnySA^!=>yaa6*>myCy4UXC&X;TTxCsAc9rWie% z`fM&5B@QS}og%{twp}RuQR*Y5FoE;ZK;Gq;`Hk&IMWoQF;!8eJQa=*#Y3v{n>?GL; zvYqnSI?h!aU88gSU%aj1=*|%5)dC3oUJWO4i)P-@VMw23M#?R0(Pz;CS5CziGVuFT zq>n6UNv*-9OfK0d<)TdtN*T(+iC)@xYrQl)ZTeHQV#k%JtL_0uWxXT4c?rjGk1Fto zgm6fA6xo5R5xI1#3aD2NvkC-mU>Dv@s4{E-F07E{Z3n7Yj_^G<2(s?a3L*0srV<+j zk>G`rirO+;5l9QdR9lq*KGDeFx=%3eej&Sg!_xvD#H|r(Ta|LZrXIYIRlrjGeJY>- zBuO?GkS+ayqBm_oipzhN9{!cU27dU@9QJ>TY`7@=C9=^JB&Q(w84p7Y6{D*MAjZi+ zW{0OnMNtwqy6`9zZYrTIb59!viz4~_L14svg>AsXJHl!I>Xh$KJ9~6ZVSZVDmGMj< z%nV3(O%7R34^@v?Zm3HlblbQZ=cNcyrUe1_iwD~1lq^@;2-B^pvyehYrC0W;yPv@A zjWf)a zzLggEH9Bxyk0?&+;@2eh-?bLQtZE7b$&-3Ybtw7C(F9IwuSF~7877S5wfs`NREG)A$r104K`9UUOjMF`R8^*a)_(f<=<5Sn z6UB}yO#wrpCR0IyJg9<#DpAxGD@7cf2oCE3i#?{HF@P6f=w(Rc9o(YS;xKIQF}{10 z=epdECI?V34^0`t557^_8Q5Z|4)SxhhP(^cmoD!08E&g8u@0=IO@h0{qPT6(HW-TW zVgrf0mycOYZ$cBb8&n1d#SgQhTJ7_h}R=K7OaRWqSflVv-A-B{v;ChWk*;;c4_iO}X zbG+IubEs$6HPwh_7=0H_9zGLj{QEamSl`O)PaxhP&wPNxtVpqV%!KE8_hej7L-xomI%x<(b` z@0!?uLm%HZVOc%2gDXXH=f0ab;oJAUetmd4=KN~wobEm+vhvlsi?zmXGs`au11EKk$viK- zn8GTRLy<6`(r`V{4;xn6y=tp4NR>`Yqt_ptFgi%~M=bzp6T7$KIH>qJ6OvgyU{m@` zMPx~i+dH*T37$&FrQJAK_YM$_=^N!f7C;qd>^Uj|-pjkwt|7Dd)DrnbLLS$$azNJh zTaN8gW}rL7J{-~1c}%%fH5_h_na#B@eu+T%gy;}9_)sZN8eA%OTqxO)CQ-_JB7`rF z;E=2L7#EP`twc$ZBswZiS-SbcM2vG|z0pW?#WXx4;hm^e)ctcJ@My*tg2dQjgHlR2 zg*;5v`S4&XYx+ZUzPXu(&tfz-#+e=&yw9EZ;1-RdTu5%bV#+ujHZ1GxPmfGk=Y!u? z&Kc-qz((|+^MfLdQgQTjUtdYvmrbyjir!Ewk;edMzm!dT#E>^~Y+5@dZ@rAvhpKdP z*V_Y4)y3R@yY(`scxNC-o)^&7%QpvNqNdqe$&QXa9b>9Ipu{}WV(B^-F-$&Q#8_i! zT3d7%4y_nvCRx8bKV>Fa?_ChxSMhh{BJ24IynLM4TpyjLj&k6d0-$AL?b`K(iLL6g4eS@i$NL1$pHN^|l>SkjB(Zv&+J&4)&CM zM}~%M2xJoqrC^1Q-*_K?P+mX>j#|m9A$TLi=*zWOYhxmP;=;|Wu-#E{!}p`DLf^jQ zD7ouU&|cY?;A)${-&zHH7v3U#H(2It1H}Q*HjIA*vwt@jkTv;_>gOMO$(Wk`v#--M z_PwtY$U?HU?xgbXXWSkR1pQJoRo)k|T{`mdrehX8rNUMT1M!t63=$;@*k+jM024*g z3K-tYO7poLAM^HY_x->nG$Xp!og82b`H6teogc25U)1g^4b4X!C~S_EEcYHlj{U$uUn>cCA@?f6D?69UZYd6D<)430aa;KWelVhx)g`OsQ^N%2?m z9s}r0m9r{62Z?1ZzEb3@(3usuq@E+?2qn3U;1u*btQ*j8j-?DkxNfC%5m@@A5_?e? zUPCR8PJi-H2V~w#PaZQExarq$yoJmJOTy1~{OMyq5}ddr2AVkiA7+L@1L~kaeTY@S z5dI^5l@gl;2M+kRTGL;*j3~8-KZ2VSU}h|EP=;|iMB62)A3hdTMmZz*H6~a3PcQZ( z7zX+oQpizIxo+I~%rUq&d5N0R?tkB(-FWOdJaN^S&NFGKI@6>U(o^{4>w*F+c zuCESw!v74r2M!0FiuOgRi~3^>v^zkt3m1eR?L6*l4EPvK>dpm1jGM=g!5tq0!t?vU z!D8Nyy$kpldymZh3;dWfkLRu(aRPyh`?NtuUf-KsxV!rrWJsTqE?}wXT zo=rn!@B=i(P+^krFNC1!=sRDb1kjMl-6cf0AF-|@Osd7nl--!IkdO^pKRN@L-9^Z} zvf!e_7RSs~;mWXWy4N14MBVr~wwMA0Arv9Rv3a{T1)-UFX}8$Ku=}vdu*k@HV~VkP zdpEhEtA$d|aR#s=rQPwXCEQWV0-Fq&xQMrGEl?(U;=dpal6yx}l6iL%c!w2}dH0X$ z8;n|}qA(Z*`i*Z&LZ>WVvM2*Gy1^jje#KH|NNP85CC;84;V4ri)l^}TZXxJOlHEE| z>5`t{<0u=u1~+dsvKa^Vp;J3d&tHG}W;&ocIo9NsF-5OTAs07(I+A^ zmGs4abzUYw!j_Y$Uq-i>9k~}(QX6|(m2>g7pa`)Xqcp`-SsrnE&2SzdyI@9M1g@DJ zB?7&aAlf0k!Dj7CGbBdl*jkLF;wSd_! zl`={f?-F-s6z=E;6J#!Zk#(@yhMer;p4<=oy_%8S*4{{Y&E=sI-I?jx+3CsY*&l5k zMb`Rm`pxxcpD{z=b<=YJw`7#TD%M|BlL7>^!$+zyy&S|yxX_yG2xb&^+s6ivW|<`o zJw0jb%)dH3oUXrwwDFiCh|(hB;Pdp){Tv8Ja3ZNXESvix(tu(T7qDngQJL!b*;t$A zpebq5y$QvwX-H9hkNTq73eIV$FpOVR6L0!AG&uujT)qa4HBc)Q0P4Ihrjp?(e*htf z^CS5nGqhe)(Q~I7`9_}HCTSgooHox-ag?_OO~$fu%&|lZIfh!z{hO>G+!QPY@f{Uv z!NBoyY@x<4U^{0u2gPi&5p|;^V-5(#fxhPcWaH4!K$cRRs03+jE4IvZ<&e~Q35_t? zT1EB>S-B~}rr9qk0PZ>!92gUGjviuFbsJkHKAXdk*}zhH7UduZdJbQO+U2A`o>ghl zSdC&=szx58y3bkU<)K@@Kbwr|l`G3n=1o(V{B9h|F^fTo(<`o%$!%ibGZ~JZi4O6m z%U$mKL{ZJ*E2Ac_uu<4RF=wx8ti)TM5j)*jB+YH3Q?Cb1xUX9rdr>NRnVlWYy4Dl8 zBdk=vDI}x!vPDzEw|HdzNOrk&txi}-AF*%sD^Hfg>NDoz8Iq2o4h#YHLM3&-M5?Q@uxqKS}!3j-KqEJ`);{zVd8cc zX~`|IqZnt(@Uq_&y7Mn_TCToQY`WXzoHH;y**NslngL!XK7YXj+ppR0sXJ6D88boP z3cEg}**Q{gW8McBW?f{?s_is#{pi@jUT|8d2sJ0}wO32airfF&_g7s&--PJ{V(<~^ zdN#Q87YitNbL5dCadjH?>bPw~y^(<2?9+%v9*xcd9#|<;Cyq7N(f;KD4hdu5$+Igb z*k)45B{aYUJ*|!7Ov(=CbXfoO8(c4T-=%ZjHxT|TE2^pt|DTkH0xE&6o8&{Ya1l@< z`;$k-;~QvNAABM$w|^3+_K(9h&jiHk>wIp5j98Qz)Ank`R79?{PZb^|DU(1+>p$&A zS7Hyl6>}uj6N7aJZ9#C5W@8I!gd&Xkr7LFp)j%5{5-R3hZ^08$_Z2Be)BR(U2U85h zQPS5l-b;o&3LNAxM+lN}$e;OCU66ffPE6=t{1I!64}7qTDFKC()FkWQj-f);fB(!7 zez|ts2$M5L-?FzdnfGdo^NO;Y&YKyUNXviSKO}w)YRaNeFKtTO#K}^O8QngQH$D$~ z1BNL)NGOkkvS9;=4`&P`% zvfrMwo1VYvy}m%!1eT&X!UgFyk`BjlWmr()LG#9}3lQ8b0> zG#-UhsZVF`x$lIkqT3xCC_w6!9vP4?2?aMSN~6`KAOpL%yTyG;!-cw&WDnBs?&j}a zP8@?KTWFHltijA#dC&qo@JoTTP&M?{p7LB(mM5MvSla>%IyMjjNB?8-T7TD@R3Y#=a;ZiU&C0A-fd+qNNZ2dYw&{a2(8@A?}09E3Y#q;puNbm!S?IsXkM2^|l#k<^A zPEeHuIghRPwl^R?CW>QO&Ea_e8Dvs_rbzVOJi+_-;@SU&PyaOo^QUw#6WUbBcsPR?|c4Bet!Lp z4{){UIjBT1HP|`0dC;9vs$g*AdT+so)d0@y3$JX6s-j_24%8Ozqc>5wS7vG!O1R4U zZ8aw6fr7rEEOQ}wcMANlD(70H{HlV162kE)*lKC&6DU zs=8Jz`^qE}bGEunRy4w5EhfAQn`~)M?=;Qnti^5O4;ux)f+Dt-$=45>dz+e6uZ?vEewlf3#f6zS6^*G)>hPi%ZceChek2 zt;_T)0<-ryfan}teU_%8EV_ zxfoD{Ivgv(6QC)h$SymF!;PHF+wu@?k<3SY0dnm3As9Q{VWf*()bXI$G|iDP%9hud z^7(?$s-xO}^$*je;1gDjA<$7D|DU$NzlN#(RW?@65pL3^NIX@!@@L+{a#n+~yA2TahKyR=uiO*Fx)L>{gh5 zN(b+eSjJ~<2dkM$%OEmVkNE2J&xbtF-*vXa91ATIs{y4Rd6R}h`9ChzHaSfV5^ZcD zg&Iub?-JGa_=WGn-*i~6t7+*ud&OiSR`T|c6AmS89<_K1xNij{AvzR2?%jRn42K9P z_qQ=1%8x?)XctuIbl7`*T8PuL1o`rouJsZGT`VQU8`N-Rh25Xse~wRHHF_5g@6M@L zMJ^Nw0sy$zkaM@e#6Cvw5Kjfla4j4aBcGw=+bs!)j$(ZSx)QK1$p zf|jGm_OQi%-Ub;y{w~;oIyuJ7b;THdYi1}b0h?09e!MlA%!78~=|CTwu?8LwK}PrT z%q8Xd;_$oZC_-JJ_=GThK6hMyOv8H$bb|hEgJ(zdgz?Px`-SxSbc`V94Yw7MxrXq@ zINd1z(a$_JcM4k&Pv zQaPMCI*qpRd97RQP|!50WoSuKlC&4i4jA8)>Ko^+MV?gY^N23GAkP&;!|*?UFRjR$ znn>Gm;@$by+41(_90ZWTFt5Ywaxn((R;wrWQ4jm7nH72t2Fp%;b2B*v;)sC2V5!%b z&fVRg91&N;#DYz`9rZ%-K6jGvxv3Wwdyyev#s;62j@?N{^tB^ed8J7#+`$dRRz7#O zY*K(tkuFM=<#1hsszXIv+z%?q zUdeV&7z2*Q!ONJx<^r<{w}k~P3j zy7(G>7q)d~j2eU0rd@4Nc(y1~?N%;l-FJa>LT%z9a{ECQryt@}&N4&9kxG3DsMQD=O$>KWpFJp>2PcDDdAX!kD+f(84xi1Gh#LHr{myC_;q;{t<6!UBag zTZ~oyd{cQJl`bw$7nNz9m)`-;15(P!Xh zZYtGc0Q*%x=aQZ(S5UJpSqhG|_u=Yq3}qosYhjFMIJC}vofJk+q{+uW3h!BeA;yxT zFYS|DJ%d@>(*4H5YV3u*rxiEob9$tKy@YvccrL7yImJ|#B=3R*fUzwI<)@^BXv6PY zMiqBtVs&zLTs0iJWV!}S5283fuD75DCEh+0k<5oMM_`xrHrT#Llj#lTgShx#Fpucxm?$vct&mLja`SqSs#>8 z*DEzDpgp-`2&+nXi^JDKs0BGW30TWCRe) zd?mnC0jRkO?7;#J;-@mw*6({)>^IEU4#VwNP3kLqvdrp=U8X2>`G;cv^1EVx@`1mk z*@RCfH}{?I+YbN;m#BI`zL9!=cDN5dj58>^1%zmN7>5>xjRxGO7eCJZpQ};sye=O#OuhdUCo(^hqgX=268CfX{zbxju>jdDnl| z@jD0C(cXqp$kEi$`kh_`{Nb;~9HcbL00so7x!&)Ghs*;8KlBg#;ldDbVU4zuByb|I zF;F6=VQjVT3JAR;I=lR9kgN%VR!p~%vlds&w7K_JmG9OUnu^Uoz@{oO=Nakr2Zj&h z@x(54<&X<5P|^R21N?t{y#-L4-?l#7;u74UXmJhh?(Po7-K{u5gKH`7?rtsaPSN68 zoMHuv6#3Hjod3D^{O)|2$&fe0n@rYPd+(*sBib7G?)tFISY#L&#V4gY%L=9tFV<_E zx~JkcJj>3AZS3jZk*?7(9nrA+#YO@&PbK~msU_piiQYjl6~+Z2IULRaG2%z1e2dJ! z+mV5Z4V?r!$c7R~$JwoP;lXy)Z1owvwrW zRy!8H9`m{wc#i+tXG>OR#G1J}ajC?K0J{6do2dE#G0~(}LoUZdFou{g@If~;YHQ=U zwf@vc5xFaX4=*0Cu2ig*=FJE#;EK)3l5bSGs5gVA(&5Xv-AxWl(>}(B!w^3Q*UXkr z>!#PAO?$`J*jYEmkFc&ybOlnG@8NlKN0`3;p8Z}di2f7fa(wda@~Ut#0bH@^{h!WQ z^Ji9fPl)0|{;^E{P*(gO%5(d-+J3{wqx-LN#Kx+wK0L+NiRnkH*5aVy3Tg_{Mh!av z2o*mhP!|Y~kgBLeA^kj4{*gCT(i=jCxs>hQ!R=&8bDqoM^s1$<$V98Z0}P82?QPCl z6+&JsE*gMOa0oX9G&PnZ0lDE@r7>t{5X-mq0nAujS6aLerM2TcM^6B9pIpa)s!9aE z>a8T>&K>5dvR2;<(5#pM-;3+7xqrpN6mQu!S^}6U4mYBGdq0kcFap$VPHVOL1|t4EC}n=uYg8}~yJoN& zXt-^dAzf{^8ih|E$&gwj;IS7H$E%#|>zP74@~P~|D1?piWsbf7yHN}(mko3bZof)W ztzu#@g(XluZLAnf0#qPkma2)cu$e{`h30Hc91lU z*}uU~!&Lu9F>|wy?e@q_$+dd-6)R^UMBdGb64Rwf)X0<%VkP6D0094FK*E;O4_KUt zj6Wm${PJ_?X2#T?JkKD3=NkXPZ7yl|EBibL zb(N*MPPzn%2mDo<#|47?5XT>sPCm;w?$gFP-`#(?^x)j?L@)VPH)9I;ksfW!gZ)Z< z6PDuv#qaXWX+LNK1~(3Y^~Lcp{TfDKQyM{{2Z1QUUJH*l8Udd5Xpk3~LgJ01r*HR5$)rkmrcaKcMV)`PRFD_LWJnBO6$yAn zIlNzHpnjgb{~A&FAZKU0zLlQ`LsYE$N?Dd^o`5dMtSVqNlZ9gRbGvtz;uU| z3ixCAkU4FZ+ef4hvR-Szt~Co`zRY>nOq3sq#e_*p12Xnx0{8esq+K5hHA`b2n3F>a zwmJrZwrx}8K>;2!-JTUXbAV5He}(ew&Ar7foW(5zBR424=@;yv-O}bZ(C-k#nU}hJ z7SHH=zs3U|5NLKm$l*q>u*eC*lA%_YFjF< z6fS-1Z^5x)on#TXqp)1#6_MgvT#*}-eb8RCGce@oJ68nzlfSO~UhA0PQy{RCDI7%m zmz>EpjD2D2;`yb5n2BQITBllm&G}v~(;^>-q0gAM;p3cTW&?SQzLx@9p?u%xPgt7i z4bxmK*%mWTZzWXMW3Wn-y;EC#`>GNfQhu;lXv4vvh5C%pgdQ%SJQ?|1{P~rWODi4) zkh6Ok^8Bwe=NaMOA(OR7Thg+>Bn#$(?bppCwJb#6IA<v<(hQ%nItD zlANj44%6S(%q%cFKoW36m2ikgO)5!@j%HWLdOJ3@dTkN*Cgg7E<*Y9cM~4vo4`RVjK?Ww#e4WENiB!@aW@!T1 z&NHp2+Hq7zGhfjZW-b_|)!wi3ktGj7iC;_D>$c&xR>#vQmP~CpSFc@rnoVr);&{_aZJ?^I`c@t!zb23TWjgGvw`|4Z zA*jGMb(L$i#V|yCgiK1;bA&-HAn>~@o8}tb8;T^M&@M%EUWq&8hn9mlR`(cdWD}&H zmGKchnJK2?xrdbfhJ9qZJ=9r4r%I=sv-!5eYQi(ODE0>k(85XOxYy>N!=85ykP~+? z;Di}8ttuukW(D0TUFRIw{!XPCS+B1vK-MBEr0MFvEHys!-+A)?UrY6H{6B29SpDzj zEZ$nhv`x@gT&iJevG7>dD77kHn6IFgS^}~)k|I*-TlkWT2hr^JUUQm9h)VC1f z9@65CudFm48c#Lm|#&wdNre$DU)2 zp}+}=YYL7Drh{d7kpL2u9>PMW0`0a46`;r4Yl3S-gU-vq3eq$;I6aKUlne1Ha(J59m&8wIgdZBt|xAyNNcaL{1@?8iz{wfw( z>6-hX!S$5t04(~l0^7eP%+*`R8w1z%4L8|1+&yP0!p3#^gKrQA^zzk8AC|*lrX(MG zcH9S(&>EiNnnY&7VpO)KL9(~$v6E$74ryb)hz4^>cua`i{O8}v(>dzQ4NX;*1{ z^>c*!n4f-u_`F~r@=M3O3+UMa%6a7FDIilh35Lls`INr~yGxjJvgItLIpPOnVby}| zA_JP-nbZv7uT4_-a&uM9Y&~foiF)dOKvPx|I0S~?5PZw+i~1l)P!sN!67VNHqAQo? z;(?eY=HWk+PX9S4{x7@ym;0xo|9dj#xrAeV*6|ZIy69QTDlT2|l3KLpYn&u4g(zwv z=Jt>(NBt$odzNP?BR_ImSq|2Rt>{#|+<6I5l#gW#rs=EI-3i_+|AVaa358$ViP7(xxmKc8PZ=}4$=fU_1m6LkDL{-8>Z2Y0-!u060DSc*+4pU5@YDy98yukXY zUooC=rQe8Q5sQgl}oc6utzsX#-*qtu(}u`3fj1z7@hxUI`Yqv5#u~ge`Rg99M`9@D zX)I8*h-&7{JF+zq>1R3$y)|UMtB`_ZLVxtG_bUlwmOlOHU^$?0Ne zEHmv{caUYtb(!Wx7P}h)&|lyd_#$g_dR}Djc4yM7R4Lz^t&p!Y9at|Ebiq-uV0Rp&-85DpS+2^;f>VNc;>E6BfcsN>-nG{$uh%F9nMg5R>2d=T7YW?%#3h|6%ce z4Zc|GHxoAYi{bkZ#p5L?boD5d)A3D;qJ%fW%v2Z za`w3Mhv^x3QrqTZ;M9@}(h|Y&<*3QyTfZNJ-J+Lkih-B}6>k9tIlgD-Syv!_$S~me zkdg2sLLM^gQHU@XZgFmI!0KCv1l5O|#-As%=X22n6wHVCC2e6Y+I&NeUo`HC=1Y!tW7dRC`pyscfIr_y{y zDoA=*;TPPOp_gvWdEh6UPIIDrYbB$pnN!JOd98eN!(vF6WZWSMP@!LGdJh$cJu65R zB|Hf{N+TD>>!=w0178fJ`MPA_OlmZ%L@|x8Fh&{m3#D`);D#lE_kf4BOnorv<9;C+ zVK2Sm=6a{CO9PSVN`M7ut=m6;HRNmHrPgCVv*VM&!FzyjJE&Gdd(f6c9#Jo#rqi%l zoW*Y@^6}E!k=3hG+AOtP;5{~{ax8;YGSUz~cCxve5w*MF%$PGM+G?zHMa|oqSsy&& zRo6gFezr#GE?}Ce~I)C{@S6tj->LA0AoT&eV5={*AzbC333;=*5DnYu6aSZ> z_&v4q&_!{**uq2yn!HS{Ez0`z!H;qY!`K6oC(f&UaqLAPIN`AH`450E7h_S%ax$#T zEbye;I)`+qo3>ZuCp@YloGLTUNVb}HO4Y>AQ&@O{ceUWA8=u=$jvx8hKzmu zb^9RZ9im?MPO9V;)S2t5sjEzmB203?;)hA23ZI`5?{l$UhLg9DGuCn5^rmXOU`bry zcwE^X?{X`B@he-VOwg~hM0#=(xtbcV>Jnp8i_Y6oGv?a7%d`!q= z^!x5~f`V;J?r;1^(A|UL5;QY_2E0s-nFDW4wWlp+yIJo0JwXzs)`ir9q~J{?dL%Av z4b72h4$6eDWTFeSnjX7(Cm`DVGBacXxbm3}vjn1b%T)4(`Zuf1A!cB6D}s^I55Bz# zC(AW$NTdqe>&|oeWADGTw)MvX>xj)pp7KvoW9V9k{K$@_0b>>~-&;K2Sxwi_{BaG-T8_U-!}$^z4} zr$&UINA{7sChjcRijYYPM)!(cI)=pfoFA{fgXRU4G?Ei_xxOa_tIB`YGw|$LJwHF7Y;khE?EuAG(D&JfHY1!ssKPKqwq;|Ga)!#tmFfbIU>MS5r+cnoh|(;| z99xCh1y;{md}wwQF2oMAZnHJ{&S3(=NlIwg+_JS&O-{%INLe|axqV8J94(l?+y^-? z*EeNv>4OTJT+i`sNhcsoN^|OZ0oNfKD{m8VB~nNi=DzqN8G}62BE^6M{@)y^Z#5GX zBN`Z)%l5zCC7B#0(p|p_*(d3*sOQBiCL3BnUNwYfi=w_+7mE``<&Z*#-{U0vFBVOo?M~3&-rY(k%pk2oi42Xo-{;anLV(o>zHv+jZR7Dc zi^xoolP0i4QD7AN)`^h8GjkpvRUD^7u#j(um0ztcHPu;~toMP#Ajw8k1!FN|Hc&K7 zi356xeG!lz$ee(6x0M%25F`mt%6iZ4OKY+rO5he+q+f-;zixVC;IIBp zA6C)nRw0bjALpcJkw0`4Z&DAtC79ZY<@3ieqKbLP(tN~$GqVM)JhX4opHY=HIw#v3 zUlx~#KbLod&K@eN5tj96Q2n2ZsX!HpdPR@5@W&13N%Nv1K*6Ob;PV&BLN6p*R;mo;)C$cvPV4J!cgV8A_u@62x3}dFrEm3}V{9&$`-yBZ$UK=G( zkGr-EO@RqeRAQuxrBhUA*?AWqk+38l6Q+ zbMM-j&GM>iCVDz*ntWdR8}GIq$2$mNA@w%_)@~wZQ(lcE{d;mhP*-|A!>quJixpF3 z7IpDn4C1Mm5ssFn4K>oo$J}WUarJYHDKYso+t#_l-t5F(zE~^`YHIxZF{N^o zqw-Lxqc&&qe7L^1!GRY+BFJKR9<`IDEi=<7PbHtg7pT95fKKmT)CtMh-TX6a_nUC* z{}RIApB}5JuMKg&pQ$N%C*5iBkds$dkkluOfv=k=$~9$0&9FpfamyD?ToJ?%l9Vhz zSG@GnTho^YhIZ^W>;``A2Q+G5IV znhDNa>GX~#XdaWZ*%ewp1?S55DT}Pz9S>7#2x=@``ne6ID_z5VOY-+Ttb~BoV+98* z=KLFzUz<*Q7P%Q~a3(+JBG0X(>ZH)0=GD_Oe#xISTFW3V!%5lD;|H$p%6P zIS+4r!hP)FW}5t_`7Fb5)Kxzy_=E7|T5n^iMB>x?oxKbhIVth=7=&YqAGEB}AF2<# zzI48IS^0^5AofEiRLoGJT@gNgt5q?zc))|5<^y57TnS3u`{o8rF?;YYQr=EE)w0YsU!y=US`uFN5t|G~VM z2wZzwk~w@4_QsToJ5t9}j3(xrZh2R@``%iYH_JP`k#Ex0;bh#zqynJ`Y-?(S0+QES z00LR3MY(X6y>%i(UVzG-_BG?~JnrLscw_I}s zcJyMntAloxlNHTo9u*b*HH`#3wp)kI88@8GD%pIDRhiODTu?jh^q^h0Ql*H-;0XG0 z!YSds%Ge`PdGa%cxHgfOw|{xi46!*{g3jjPalwbf*~Z zoz2z`3A2EdA0D(@{V;xM;1fT7e&Cba%(Zn>&?VeEoC@{)QS0x%V?@31Aj)9`x;kl= zQ6caJ*0x_1MYZtI>-AgVZ?6eC;57zv@_}E_U$9ic?0`foo)_*;xdOOpb?TwlOKA=Z zo{3VCL--3EjF@sJeQit0T~wa}?^BM94 zZ)(KEt$X}4ipSGbx%(9a*&BoZF<%eaEB@c(KXa*G4v=9i7Y`GA8+Y%&dxS;2upN{r z1gEm>GUf#n29dmaaJh{(BBFSp2wiQd81b8>q83-{+G+Mctz{xw0ve71mu%q#Fn)XT z+pyouo32+eAZMI>jC_K8$+qZU@-)kJ319K4+#Ikj^O8QwS%o@wto?`!+ul_asBYcJ zy~#!7zBD?#>W}!i)I%3rfoiHIqcRX5#y5{5-4O@pN9Xg|iauy+D1qppKsCgLRrHjsxw>Oy2-9ApNP%XhWfK4Iw$j_(iKmCtC+Sz*svvF^jykhE^E~@J(>&We%PlD+ z40fa3v z)3z_%UvXBB`{ip+-B0z;r5!be0**Fo)}(9atv?yXp0&07$H>92{qSSS#q*8_RK ztW0Fe%U$!M`G&&od!HeTS)wnqx`FF%`ez_eSn;6bga@pj)Tp-D(l%b0Ir-3G^X z$9GaoOrg|^uuv8%2wILE zngGv!P)r28CeN8DCmaj`w)s*>=i$%3FlLrDo`bMZCDAFL(JJMHt9K28N0MPG;d+86sM~vOU)sElc;#v5G6A)}7_oCsa zUcTKhekCq>WlCMWr+*c~RDF*NCtc?dlTW?G6=3+OJajIK3onlFbH(Hh`#ctZuF9+& zPoiR}cb`h5-W}8;sK76%D7LuB3~mwUZRM^0v^KtNh#67+Fh|H27xIl(T_8@k5;`u3 zU;A#!D5`rsD8`2Nk^}9_u^jX82l3A}jHjV;bOZ>FXnuI}9 zma6}Xm#Q~hlSy}E_<0L1HW5}`wsI&*Gq7YU+8#bw`$c(g9lD}~&P{d3DB!hS6%r+RTD-RgEM&`7Lm#dzOz+AjZT-s zLoX>miTbFxClXtjgm>~Az{7>Ac3-;jL6S{i<=lz7@uTwilit{buu@DZ=W;J7aHUwL9iJ#)(yF>p2j>Km*H>J|B(sT1@BH+!$uOF?0t6Xqe7(&X zQ(cLF1hs#Y@4jw#cbFNEwMyOInu?v6Zf*pkt29-mur&ulTIF_!f->kujGN^I3vWf*Pl_xk;f&&N+dANz+)$45jksWjylN#k&xc@ds<+*l z7mn!JrC~NHoVNXx!>KWAJ*1&DA-C5jQzQCd92AO-wZ@v^`7yY#^aIKwa_OGi9MaFp znmo&|Pees&A(CcF51)11Q6}wpnm7j>3qaS`4Wb*RH2La*VSHyj_gk#=rch%>Khc>? zcN&0CQ1dM4nOZJhNzI-$6w~!lg=kSV-ITV$>P@nwgfsD$g!;fhEkb_hA*4Zg>ZosI z3v%>tT`ER{#A1(auu=IQXQw2qyAyA;cmg+lGH zuU;_%{yPcrDz)bh9tZUM5YTXPRyTtH+x~0j-6P>oQzZB$U&coCtjyRTt?m^r$LFxN zp4E~aD0ksxL;`jPS)15+Di_g$NC;aBft4I5_>ghR;z#z&BQI%g?fEOEnUTKf`ckva zKyXzgw!6BP4+-q6L8X;FFe4_p-c{Z^*q_ z3BPjoq{y?8Qu6vEq6v^65N~&p>?90Z3{=8VK5?tt%F*#V!`ym#NhJ_aNTGLd(TCYziiZj}bP2=jN`EIHm5@oVO%wt% z`BIGm(s)^-Tn9f}64LxIrz;cZoF6s+2o5sb-Bnh7ubI9ux0zR+s5_`g01w34Qn3uP z#`*E)B-wmoz(^7CYoc-wI2gNrr7x~1sP9YPH;FHYHT84;qCP|{qdFgX!&d%e{MHoB zdu1WVFv>sjKoD0C0zL4r$N%cGF z8inRaCgJbgKg@x$ckDoMDRo`T7%cghk=h2mp4y!HHY=T-CjCooE7~g}T81?3y0W~o zydq<|5K6En(3&0-^pv29I0r`Wj{rkSaVnU}LFi977~c*j)M~`qU2V*V9W~vkAGwsS z8%JnY-~lQ>8m-A*t+bdI@*%KRX~gi4W0wSM@u%$8s5?C}WNtLTc($!s$~Z`}L>jhQ zu$!ZtEe!{h+W1;os-q{xbCB|h38xx3-|JAL>>Zk)uPw2F*vm?5&UJsjMym)fOuDIy zB)8evh>`}c8MV)QE(*B)%CSBp{RX)5A8-w;6IT#UZo%_nyF*dbBi`cg0vsOld-^f! zwG0Ef4s!#7Vx$U{DoxynkJ3?pipj^m4H1w+OwXI++1pavNc=8(3JwS0ic!!m6%7v2 z;%y|h9g{n#&3}Ezirv4rg3P8Y0aQ?XLvCn{VS}A09;t^U%r2J?pHzL5H=mPu{M=M5 zRv_X6b_FzU{$5zG&mB2qAX@eLhgMs+iUENzFn^}`r|Kbm_C6I5^cVTxtT@A^fgQWW zP*Gn_&IT1bb_PW>Fac7Ss6O;|gMAI_O<^P2S6lE9_ci=17(I5oq5D+ec`p6vVDT6H zD}x^0H==6JS;6X1RO60lN6CB!Jd0~)9#G9;qlM2sHeMR#EqPxQ3Tk7zxEB`JK3z)W z`Z#RIO}}aVE)9h<4hqm4{cbz@Br4btu8YQq9%V;vW!+l6IfHy(BQNN~GMF45b^;o3 z%<*qk{kiXxspPzbgenS)w!Y68#XD{w`G9l- ztWu=u`ZczDZ;&d|!azN1amBekVuOxOyW`2LK7~zv6E>~L3`*zEWeWlzwY}veC}a^d zUTF%}3)y={(D-z$ke=+-=~0iJJLNctKaO+hw66VN(=tbD%3D<|i7tc#6;^jCi7eZ} zIOnt2~CN-)eX8QWTk30%s zFh=Qg4uZ?@CL1`9z)+xeLk$;0-|kxbI%&f^I%x@{>BcpE1sAKA3_iR!`khRxOX$(s)So-h&JnRQk>AnY3 zYoJ9u(Cez5>bT&zg197b12_uIty`96>@}8PmBvA4HWnECroTxPATNY2>)$!hJ#6f zMI$_wMaOI=7U#Zm5R_}7=OL298tF}(a(=HI%2Y&{5mAbR5F& z+%fR`yEpIG4dZb>ly|Cp4q(YUynCrFCdqBs*>imn-Z&JZq4J7-ytx5vX?OK3nL+cS7&x2O=|V z719&-M;q`1luKllmZ2_B`2AI)&M-v|%Aiph&%4Z2mDT$g_qa=Tb1aHcL{?0S%(x$wE>2O(zgkgZ)EdE%6qsBmV578rlh!A#0fy=)+ zU13w*!Uaz|@kzp| zJ$p9QSZ$vK`OI$^D~g<>pS@&}LKD1h%$xU9>FnKST`w?o@#}0{2KJwp-mAK)<_^EX z6~wML4s#kz3dh9t)2KiGMnEsbbvZDMs)Y}#sDAI&%i^8j%n8_##-reZzg_n)EGA59 zU$V_*ebe#`Zt9nvz#) z_$Tx$xFHMFy_w3_nC-BnB`${U-8zV5DPaVR=WFa3az%v-HB$msUuehbAWT{k09(*y z?z1*0g!S2y4|jO>tIM4i<9wh=9P_9Z_=gvg-8DY_eT;uTmGZ8p(%NHpZ+X9fs@2#? zBEp6QR-w0jT)8%~udD|3BrCx8w$z5Btzd|Kp$?6GuT?Q$`WJIhzDTJlo_u#fT|?A? z4)vF}4L;Zm2^)u@!YTsYVfrTAw4PGkedHb?_jX8Z0@Ozda2b@9I+?vE>}`uj0U>jr z!fyA19!pbjPrCmUhcvZ*Wc+5=OZ|txkSad|AWTKw0U(3IetX_35VR?E=HlYD&P_}d z>cGfYdr@xMN_ym(fjygYFBu2KCaFz1Q;JARp+Gc?EUp1B-MVyM_Nz%@nwQ~&RsKM} z77P24rR4fzGH??ToMadbCm=&kP*NNt9k|O;Wh&Xv*8r%OyU8#qH3i&3FNT%2hM93! z3@z=Tf)?9*r|VBpz@1T-JY4?k*8)b|UR!#eOkX;2o{R_lw?D8eX&$o?K>rw5)hw4ukpuN%luh3t{6TPMj9^OpkVvM``~(vhhgqVFFw-Z5@OMheW%7B5hQM-UCgHczLj8C4P} zAZ{NuPf3p*eC1}Gi%U6{PEeG&f;}3dJ@O@Shi%ZjDVMtFuKJTa?|1r88e5hK<>zgH zT{0D;{~y60uV3;hK?INfPr+Z-qr6Gw%|^#em3x3lUvYI_K_0sHvka5oam%3;LNFC{ zA>>48s&~=;x?Fbhu8a)v^Wkfsvcniy)v=x|_7}f{iS(CmSEs~17%dL=D`m@N2gW9Y z_MZgs?crg0P%D<4>%!Q?i5KIFH_40T1+w$(;3vt*WC zfO7m;CgGM)zf;r5PZK*r{(Y{?+!Nl&K!lC_ zPhoFmp%A97tpf-^%Mh$w?DfS!3uFv4aFK(|4$Oxo5oaHKhz^RlZsnw#a5|eRPW<*edF^9p2Qh3e#ng}WfaS8jPI`ACEk7esqASuivh<2-#EVzw80^0Fvnsk8o z8fGR!0^MW{b6Xk@&C|P$MNFhrh6YYz{z_BO$J;VeSpLPR3soU1^YG9}+9 zqHVZeoCJ<)!Yni#kwhrz0yf3ltFPZmz>nDkmQ+Ruf|OgTlT-(WOsdAnjhO27Y2$w| zCTAj#cR}O=2_>ua&|tOz8Mo~^Xts&`CZ_SDX%=@;c=OXE5f!!J@%7~B47s$h8e579 zDw7>c-QOj8@aCs|uAy(SBt&G&WLk37)5elza>A;h#x-)@!4&7gHTs!k6!+1JetuOj zP>#VMZ!|=2XS8<-@xt9FRsdDw(!kZAh=|VGN2*x{bgx$xm9n^T^8)`dwX&JedHXKF z;y#K^vNBJAOzAx?K4-z(eJzs$V;;^!rmOEMQTZ%ChfuYXq)p!qMy+0_l39J9sIwT= zpZg?0Y_SgLJsUy7*dvv~e+wlHRVVDHI5hgtA|*)TUF=WG z(p3qHKt&3R0W04|SVNOSbLh?IDmaH1@Ta_=zy@~gsNTu(4zBFEEm$^lO<09g`2^!@ z)7I~K$PC~{94-yMMZQtJOa}4_a^=2>XNzdG>{ZjK8JPy!jgLJPWw{ZvR@jvl#7t^d z$eisdSWXGBNm3=b@h*%2=V!z=DSkF(X%BNmgDy%WH$%o)dHkA`GNLBhMaBlBa*8y# z%kRDQRNmSz6cprKOKIN_GRWi5dlK=?Rhi0cite1qAxp{6YtBH~GV zg3!Ofjf(c9_2uVWss1uGsbf;>WRl-jI@Li5(&s7X?Y7vpi@TD7{iu(hy~Z?vP1#)R z08p#aYTOvurV2aeP!&|rIZyB@L{Q4EQpI59N3uv9VK!3PM{ucRo1o~niUpH%Ike_85gkW=t zek0-b3Bv56ZGA5HAs!0vlzEJ!9(uqpZ_$plo;q;u_9lyl zv+mbLV*p1TJiZCEj^!=c+&g^8beuCglr1dVxH^4_L80f-2IsYRc}9#`PEo+knw3Wc z?GLYAwd*+=iy66(j{ZT@s8gk0Gq*EclP&QIxbZ@+3JXC`v6gZZ-~3be;CkB4kS|J_E9)M8iwj~kjx3jN)y$z znh30v1$WKSS5L1MqP@s=@*`w1;w2`cycl;PBNWgnreHAobWuCs zOu4o7*+I7g#S(IO0kRJn%Byq~JJ8h8@qqj=fty-%V9;(%@3;A%j% z5O^rrer_|3qjE+uyezDF!~)UvfEH#L6kL1WeM(4A(;y8nwwOWKsuWO0cZlo)TLsSTeMHGfGful&L%HCSha}T1wZU}uo2vtl@si!-39m*WyA+*sSrqU70WSz*(o7TqdmBU!~_BpCO{FbIiR*-F*BM)GRDmDwhX^pwC9 z=&y+vqTfa$S6C0U2%XV%E9<0J&!u#*rrRwKh{M)FDz>PK8GG7N`KINJVGLW#FpBQU z@Col#a|Aq#6Wtayo)r2IYrfrh4DG?sD6z22iM-2KXQ}4Mz8K9V_kZ=}NHZz`F9yQ{ zDuiU7i{)5-K(Ore-F$MXZByck@;u!L$$TB2$J&RLW6A7Gcd}WTPKEhjzZ^OXg=M%v z!FGl+peGS$=+j6C7;YC^qONP-tY4ui5|-O|dO0+pf*ZlUQAKwLGIX3-ani27?u4bj z(eomg9g&pRYxz7#o6J6kViK>GBtu;-Lx8Or zkEEycE{o#g)s<{tTQ`u@IQyxIk<$L8*)*nqsF2)UU_*`}SRWQgPeFRS8vi+V2%Wn1 z!TQwmiO>U?^w*zCVO~ah%Nt0PH26+=gnRajQT;#{rJe zbposwCSjgV59kmXKx!SWTiGlrnjB&t{n%O5@Dt9rfhUW=mC{q75Ac+4!<@#eVxRfJ zztJgN9G;F&>Auq;;PG3&4zC!1R3+4^s7})!X=XwT!82c4r_Q0stHp0bSpEaX<0Yb( z>N*ef)sO+1ggyocLiy1;|4~A|Afg`Oiat9}hdSS1FMpGf-rjR5xnsvr%|c-yUj=o9 z(207K%xDH*xM@;D&Nll>#Ta46tOX-l{iCnL=L4cc!`1{Nt7P#Z`7L-YM?~NE#JiYN zz?5g}&xBN2(EHDW@4cA#yn~RX*yS=FmjWrumYhQeuxw;N>tGAw$%dO7f;E}?h*`m5 zmv$9bR0)qLGy>_P&6emH@Ve^Nt4%9H;}&QjQbWT)j&*N)<(FfPZ`ZHYN~xzU}YdD{0&_o3|ul@@RhI65CSuAf_D!;;h*JL z3V|jnUC!n_tW_&4Pn7Uxhs-!yts)h`kW5f}vfo1H6|&vVAMft>7y{YG2we$(5F!yq z5o&Q%+3L(0+qTu_V-HgmeUl;k@TzwYrY}vN3^xtLQ$wk-RS{>WGg}{MZ|n$T`7S(& zpYLp&Pjmvk%bZ!1Clw@m0n}bCtt}r7(`^nLwqCXTm|>-wnW5ihCd-_qMkNflJb#qF zdg^aEiqyL%d(2!&ysrh-x3|{4#6Fkm+6-OGG}(eaTZL3Q?%O3F;=@0GLv3F_yteI{ zO{NCIA(l!Igl^+HPBAVSBK~^u%VA##twyxYOB6QvnlwT%o?lR$xB1~`W9LeVXo5A; zX7ds*+EPhIl3wkxJJ2M4wznrhJmJ2&PrZB9g83w7fh)P=m~$)ZThE~pGERa-U7nG z_ikZiUapR7z_P-QF0-~{>UMiF0p_srbZ$7J;ulK@R^>AZteow-neEJ5vU2yj#pBpe zwMN@SC;IaOK8G?Lji;-y9SpIXc$pf3C=MYJkT=5Ym=rjTPokrlPvV%}G9KN(vX8T$sO^ic&NU(0*C=ym|Pd%bl+d)S>vps!35oq_&hGS_3xb6 zvv_|6EySe8{wq%cNt66>+JW6D=%D`xGxIOlE9+F|W>UXMrihH#ZC_}oqM-o<0UV$f zP-x0=*fPVUBt6?9*{Hb7n2cQDS7D52c@fG?sxRamg|w8o1w3_qeq9%hM?y%FA|P;6 zx3irdejvGfIRY;}I!syOPd`pIQIHgtj!9ycA8MgNK&`b1n!n5RL;MxD7jKl>?s`!knQ87$gP5w5+D~ZZ;X}ZGY>{+eYeErFX6K2WiElgpbn0%mj%I) zNK5RWdjZ7nAx!N`Mny>lUIE}he{-DKdWgmvdzZPRH>X_!P`YFnfg)tc%j=Z86tB6~ zUUT4Ma-e^Hv!M6`6PK%foJ6K-#kO3qs%1sK?(4?9EmO&7M8`^v)#;UA4J1q}dbJx1 z?|wS%319s@we-A7vOm7`z1oN1U1#1~?*Dq1M6=;@hw-Xx_$94wDHPDt4pT-t%&?fd>TF)?Sfc#b~ zY#h>KaZf3X5Yq)g&=(NWkYkFh>5zs96?Z!;2UKzHMFh?_eWYAPCKkDh*9pCT~jt5x1um zQ6%@97-kPh-|KpZ+`ozc+KtQXuk}v&{+aW`9iGW&%gUAOLs)-k2Quy1@$BXyA`?ESc4TyXV;I$}%jE#(rb@P{qTeZ#R1t&;aWw^VI_0A^95 z?z2!*nQ2tvq&(=Z!3&ChQiA6(5!TZlu}H3SWlV!9klj&eD%DD_Fnfn=pc*-27#^|- zL5Bu{<^?|(G-lnaMPOyt8RJ%`HWZ>WrZQNUl%J9xXTsUf2l?hxKq}B+%$Y!(s9y>E z9*jx=y)-=i#W@0k0G(N#Dd+3An>&K>9Hx$>{lXeRFse2Y@;~1d2m2_fs$(7l3HP7MuzF?>amf} zq8ge^msJ@R-QLf1_$5@$;@Jr{G~2S?#l97hzowzFfwX2vNALjVu_9DKYm}#|2&bFw+t2uIwtV{K_pBP&Pq(;gor&y4S zPg-LwExm1U8NE{&{DZU~QpqxGObMI4whI!BJ+9yu+5ba6y3@D8}msBA zsWhqLNLCXgG0LUdO-knryp9u$Wd_V19U&yW;cnK?>>UZtvulF>9Hye1A5fwa<|}4w z(CVCBb5_Oa!x!eh9zJNC>1IW_HXvvmUb?A1=-H{~VvCA@oY4$e3Fwj-{VUhxm(P;I zJ4?;2{<_HHa>|w6gg?fr(l=bl+V?F1cHNa(w-mdWjeL_QgVj;J3oG7!xz~_B+KQiT zbC1%|zcfo6@j^hgD&zx=Z0-qMeFb{V3fYAzf1Ud*3${why#3^Anm$zBBm>*FapHQF z3a9EI#W-cOLxkK8@q@`MVB2^XOxaD7#}LHS$rP`Ur(L|fX(XUHbZH$1^e(mm(cXEj z_1`tY%f~UhoEB-t^&G}=$VgK%1kg704NazJxvlmZFO8RXlEDJBnqcB3Z0DCeA`f>7 zM;F$P+wxtKZGG$EkFD8sXV`ynaIrO5*cUvj`H=JL(KOERPLxlhdjhd5>(W@mps$E8 zyxOSvzn|5Wj9UfmJdsU--JM(m4KZ6uE~dolFG^=T!Afgv9_2}r0v|zvN1^#*&XV0` zWQGLLp-|5_@Gl`|or-8<7n;`fKk6x#gEO!R`t5fTGV^(Os26|qiXc3R=GF_9dy}A) zMZ598E&h1>$QKgnJ@Oi>(Nkoq8%Eo}%;Q;gjj11cVoT$fwU>Y%KD3DfIGLI@OB)XG zguB>AHvFkie)0DD-%>tJaA42NhN<*by}S(G)d(UxLGfrA2(8bS2$24QM1A2E@14PS zB2x9w^ja&Lw!!62)JnaE8n{Wvdb%HDe1Lkj-2&mVUR81`R|uxvmAmUgU|{I$=#s5_ zsTnVntS^pr>*zt*v6ra>2sy-}8Wo&37HxliY~a+1=vWsfk^Ac>^f&m_;rwQg3SEyS zwnyD8v@>F5F;qhE+7=KS?{6;74>C#)WTEHx2 zuPmPSYNkdztjqZT;o!k^Ts*BqIIXK%Fsv|;rRs?wog(7esUVC6kQg!wX;c)mD%%ts zLc1vai_9KH(PZ*rGU@DN4n%&9;Ej8tT9)EtN;pPBkZvJGy?PaEi7=nW#l!bbFCiD$a$fHCCU{VrU{ zdovTQS7bDDfBk`XqkU#mMxx7wupVwbrh>5Em|Qb!&`jbxZ%v=i;yYh@F0N_G)Jf?( z-vYg?9rbVCq~u*cr^N3iayHYpaaE3iR>OYGK>Zh8y@q`%p!3o{PMg)Jma(^GhNj!d zfg;2@`WSyI#`C^qKK(+^ku-YlMq1KltdU`Qx0u-6YjzzC-RTkArLd@P`i7Bp`EZyq zZjP@E)xuGX8~xc$VXtqB<;sh>Q1?d(Dz^SBhLt!1XpmUK{J0<#I7T@RQ2Ta6>%%vI zdROh#zCq{;@I${U3O5WUa)C+`27|?QM;Lt~pKt0ub0ZLNt3E#!edjXN0@dJ*?Fki6 zkbMH3Dgk8oAmA5tVSabRL;IbB<6{Imq^xAkX=XqrL!Uy;)}z8W_ikly)%@5UXb!kl?)Sn?gO+$}S` zGn2*@-i~p&I30rpepG*^n;M?gQ02#0rk6jZX)y%fcPr-G$ z(&ljs8a7E+2Neq&u=^R4ug~-wX2Vh9871jCMflKPiAzX6eQ~PAc9O*WOJ6uwv+=-}DX|zWjn|}U4b8EcoWi!V z;LBS*Uy2#Hm5HUa^j!J6R)wmKS{v;WN9r?P*AD5Tu+iDp6Q+w z&ArUvmO9y76+5l_;5B2Kt2w&E-achmIi9vRhw7EZ9}c;kz4;coo`|F7DV!jrQ6nHl zmPYLeelQ|i!Q39c^;+c@a+nglMprV#pc5^urS9f`i>AN?Kj3?9Q;Mi1_enkXAi*SwhT3$2jrL0IR^QMENIW>McGk`qGA>q>~`(L ztx;vn0Mrb3_+F(AZA?UtcghxlgHE#N@=NQaf}tNK>U>KAWgqT>0UX8~az@3k z9vC)o`r^w6xaPlT97)7<$(9~me?e-pxgV9|Ms4@tBaJ5z9VkSJh7`6FV%vc&BTIFi zD3ki-Kg8RMWY_a@HFVNvgKH(W>*Joez3mIET#%d6RX*64EJ|l2Eo|+|nsjCAw`OW{ zX&v|}?91;Nu4hP^6*4bLOAzqE%$%o z8`=EG6fN}#Y*yTJ(0^rq9zF)t=`M`!$bSnkgJ1Tzarfjrut27Dh3}sucQe3pN$&fj zZ(%Q?o`qnlqC1ehwrQPw>!5&b(}AVO4ttFMPTv$5XS10JXI1F!s3_5_ZRpNZ=L3ys zv#1f*XRl6neeYV&3`<4jLM%tpg~7Cnp#B%?^1MBanx7rqmjiE-4TB754vyd3bQ`w! zky&p^_~`||lHlm60%a4=G||(RsmT^VkjQ!HoQ@FEyn7{*=J1b^m3qFZuCX%t#T5NT zV$Va+)wnF+h*3_!wFS|j56V`IPpicl4J(GzL7`fNCAN>NeK*q^1G0uTz_J~&b@1Nl zTkRsL0le2w0~NY9Y#3{qrS!Wu9--3#K|HSG)`U6l zF;3<9cXcMo8d}(g@yWPa?@!8)Z0J0xqpWDzd)X0? z$WforYkFVWQ)ziiYXqgTJjcm|mCIHX!4LK*KSBGB|CB>7%dlz8jAr9f zF5$VMm8YA9&~e}g#8*ektR-+ zu;L*Qgo{7w=eDdYbt@I)3pXg_yP0%z=>=84ru~d8IQoGH{9{`1CR;=Yd$;`R}g4KaTU+H&`}Oj|YeXFdLE6De6Ru*K`#) zu|q}-Jh$W2{8Aqa9xcv*C+b-(b$1%$sLq9_8E5)xMMsZBc8FHxW@%aK&~x(dzhF2S z`jU>4EkD{n>52Z+n1;+&?OR9w_RWy~{|n(JN^21NV(Z1%AVa48ih+R!ViWZp7n#w+ z4Ik;)+wt)7HmU3PJ@k1~!{8jE+J)~b=in&G8FQwpSXHQO(7-=oiyNegZ@`|Ee;ig5 zFEs~eJ+6&jnZB+akIvTZg7Ami_BYf!?*9x6;gxAumc+5;#ID_Bx5w7zaG4(KjD^+U z&Rk<}9G)M5l*IGo_Gp+Up6_Rf6WBUBF^MSH#>t*|T!<|u>=6A_gRpk2OE+25PRs9w zM+cm4EH1qBBzuumh1}n{6}E-Blf~nsme_*{CFV% z<)U$ow?M3Xe(7B7>05Rz@8$ptKs-}b*ruvc2ERm=ybwCn5yPo9K6gKcZ*g6_l@d&9 zmfoXSpv;$Z9$xkM=ETdDY4*x`mYE!JpUHbU^1Sukd# zn%Y6s(AA!&(uf22k6;@j3lGbkZvVsW=ZC{vtZ%yQv(9OzA%h2NzpY&CoVOO4!^9DI z(Aj@C6o>3*#=#%C%jhlBIE$Bo+w)+;qc;GkS-Oxb#v`TpSsY*-TkJ}s?qCS?Z1kFy z4?$jJ^oHQC8k<;y1q4yKF|9Zg$#|UuSgCj~y>2O3kg2Oz@ySkp*6!-$;(1Q%GWR^9 zRsVv@hJtc%yS~J2Y>>~OTo>pjS;$qzu+w^wf`0oQrb~EWqH;e&A zR&)=3!*QvH8Qyg%XdDZBri6G&e@O{NQ%*>>Fvf*siC)=%rgr@Sd~V5qLJJya;uHtE zn_jOo-p9=5y7sAED5S|j*z_s3RtZ?rp(i5y`v6IU2dJp-R;Fb%l=1H@^zXNs*he29 zpYSDc_pH?hTWEtm6C*g7Gi~x|-hzNt6fwL(Ea+k$c%}uKd}bg#0!q)>Q_CVP970=z z$WPP@%<)koA>n|S^+B_udAY6hZw7Pt=Hr^SlIc5qBXXm)fOzdq!3RuQT>izfVxJ-B zNIKuhpSbsrJH8;cc2Uker!-{>zW6B`HhkZ;uNP2(B;A#Rs$e0Igg9PydyzrLV6*W2T-O7A=|O4t<)Xdg)Kl@_IZ2%_$7&5_?!_>&lvGn>mYtm=~G|tO(bV_>@@NG&LFC!Cl0r-|w zENc{IwKCsb)-A>3Rqu>VM$M{+vx812x7x?*sDg`jV__(jnxg}#V7DDO4f{)1OZm0N zZgV~Z;&UY{dCCF--5pxBhmJQ3t)&G6-t!dlM+Y}J=dz`Jb3c&?E?xM3B&XsHr>DZ+6tUoSC5kClt2Bx_$slA?GW zd=^o9Q*%ijDFhWFF$g<0aLuF)(S<}G`0vp8lV?a?;MVetq6-+2iAZsvMUr^)YluO| z^`B3_1*}Rljz$+Cwij8F{u!EsmU{)(C~099lA&|HzW_@!L$@SD=^?r8$QpXkXMsAC z3xvW#Hb8jl;xOr7g(~`JI>j2XiNZwu1K65^sK=W;q>vv_&@I%Jx+*u{3dWP8B!_#8 z_k|lPOE8e;#WfyJ*yW4L!p%4rAK?&gCtWwK{_aW~c6k-l@pMI3O_D{TZ;IPWnqXDb zF$*PuA?V@i*h&WvlQ3-6@bf5;F$gzyU2);|7TI_sF$l5w^I1*ZUFGv6Z40YfAZhjv zjc0q6-pk_ zi;XZI*#3BeF@j>M)T%Kl3a44BuF$<*!fitgr((=Co*9PSVC1#x&>C!vcBf~V^5A+c zJCJiS|I#W=v;n{LrHMuQp_D6QC|4(aHH`+8AXlcBt&&RykSMOI$SG_h%daByXg>s_ zL@=PEFCI^dm$vzhxi1B$bfJa1r;F6>cP45VX#Gd`&NF5c`!Ay9pbXpoX@zI>F8nj) z{AccG28M&1#K1*~xAPS5f%P$O_daLlOl)uSS5FK@R2~=_v?F)VoVz`05iN-D+sjjN z>2xwU7aM{3Lwv9`w3|%$pe8p;*=uCAsj|gI`3EXvhwUXA*TUHVZIj(n;#uR3L38@>wB7{{5^o3FB zH}?+-oOjWUBOPIy2aDo*{qLh2LBkXy~{8%X*YE)>|$9 zDtFZXyWHu6BjKiHCm=Eb|EHC=!Kj}Wj|o1})=tul&2^7qT1ZJQ56j+sE!f4EXe zwXF^MLCX;wi8NvfB;TZRN|^YkcYKM%d0a2Io}bS)J-;#S@vXVl|1nzaM}zW3%+*i| z))dCMWrR(4RURTklGVDdk5fn5)Viw=!3fU?8KD^55Vq_C>b!{M=T@X0d^s`vyx3(f z6jh?m9{aVqk$(!nsdP1E~B>KkAs%j=nYP3=(t%)f?}@Lo?N)m$iqVlJ-u*`Il*>?X*3v(Lt4 ziXvNI@;I?|R<*4{=CJYjPeva*IA8Q@{atW8KUsq?V_Ts!32Qd267Eo7qt)&5WyV-@ zaM>SmqLz>Lk?NC}()~RmCM>y$V9dPad^1Q-ROhO=62{V7C7VIT+q}J^*N-&Y^GcUi zC0C6vxc*andS#=h)4zVf;Q!s=n^i`{O*8O7L`+kNg~bN!6K4ODQ9JUhAlXE`|L-t% zQ3PRN2u2+)n&@w_H?){Xd;RgmM*CLX5W$pOIsT__f?u(#_&rlxyHiYjY_GSk_XvCN zUGiRpyo4V7;DY$S66xPk>LVE5xxJ{t6om4{ z%B1R!bpNc>*-Kh&7>laf>m|rN)czLZy2+d{MXgCVr>;4;7FxRa%F5O~_ zjmZ2xC!T!i=|$(5(4KwWzLC#}LW3&b6LSD_R6?DNG@cBYz~$Eqari{L92Gk8xaZUX z{__aUc|_&Vz8<03*ZBXN&93ae%C{8^B08XUp|bdu03jgLs9&JZs1IFT+Ypx`Oaup0d=yl#d&{_0>Jh*@-cS}m0X^=lDvw%V^)$Z*K;02nY-NQy7Im!v3~4~LkO{6IoDwF6+g zJp25%sVeIP=^&$xAZG0r4n9ieccNu)#+ZJt} zCkuj8gPkUYd)zz4Y-2ac%#aam+SQtF{=@t)*F1#dsL5PoYZn`eg!#$XdToo!&aCbS zUan{PmbXnb$8J=TfpU$BPR|1E*x4Sc2o7k1X=JiBCeLV{cr(4!str_ht0P9}agvUZ0KfR_Yoo@kFFCLB zk|CJC;%m3q&WE$qhK3i3(Dm%-$XtMjMR+e9r3H3RKI$=}cj~qmw>5tl`d2I(Z}Z*T z@8^OpYPEE15=!wZht7#S6;Yr%D!0g~;DF_F8G>wy7-0jIgIaf-wpPRG53^e~X-wF= zs&TZk<2nf^>AD%jizPc1(y#nK`h~%*jwiJwjlMxzb-bp(q;;gp@3Lb`!7#ku_Z-TpoicM!L&iWcsLsXW0a4Xe0DT)7VJJFI$HUqZ{S;1muo(1kxASY7J9=rwpF zr!O!IDB|MfzOGz`3#PRmw{{Rg|EgWWEt8h#OUUZhBvQA9#Pn>Rht`*cQL8v$Zp3)N zS1z&{mJQ`yXxj!Z^DPc4E;jz*h}sSMl41WlW^ZHv5?5Jm)LMo4mR`5DpCbD{BiEn}?@MH|hr($$Lv3)rRa9%dnyrlORUUrg8UXJC`Rm*VB)DH6vE zgK?sE$m-<1W6yOuTw?ByEKlIh8Z_emn0AbzsBA|2>j{WC2s9ARQuUeyOFci4G){bd zts&kmbzf1ipc1SZ`gA?JypB|ah*|wPvVlyj?G^h|xz;bp;F9cC+THcY?k6*BY0|4v z0{z>67XvPVNPG-mZ`Q&8UJRh1B2fW3T2Mx+s~7@U@s|Gn*8}N}64;QT_D=BNs5q$b zYmxn&gMnmYiGLh#g;UW_;p85yvN-KKv8=N=_1)6AYwSktiLJBPPj~D)Jv%qu$_9Hh zUs!P=lYFutcX}STJy-D-7C-%eD}S>+xkUA=0^emMsR^~9(J8iSD3p(4)N}*F3a6uV z(Cw9*)#o|IyU;H*nkALn3VWg+Xg2D~CFhr;{Gt%4zpQL=#Us$QqY&wM3#rSkl$52E zwUou?W1|YAOrq?>oy8?F2vpgNPudH!(6_5kFvZ(wJLTsuqAr!($_l&m%KL2f(_>dV zl)_fKn2{Wvj8S$@228rg!zci+COBE=fiRt|D-nX70}-&DzXP=0@*x9s;1Uoc9PQWc z2)LIRPoa9nCT$wFkEBAPIH$S3mOy| zgY*^~7tp(*NPy>}KUMds;_ANq=tINJdIgD{_SZ6!GtE+j^OH9ny!z7M%kAOertUA} zrXHr?YCwBk(<0E_AO?BD_=)W)2Ps;gU%hfH9UkKH2wjTMubhkEuiOXw(ze^NPJk=_s?fde2 z-;m{G-bm(f9$fLZ9C+t!ggR3M)Gn$&g(+M-r73LQAmw=MLGil(=vA0s5U1ePr)#zq zpL8$$<0E(C1{P45WC(wS;qd+L9%gmO(e3YOo*Ql-8#uCFG=EUO5aul-DY}0NkcVTwb56fAvomPXH$&#RsZ7V7 zAi3j9ZS(S)shTb3LfP|7Eha6&{pP0_vrHF$YyY0P4xT#?hpG;L z!8$VYos~s=`wZ)s&A;LtRupx-*FAn7R(n-%& zDj~h@zEV9GRoC!Jfg7dP+-@)W8#C3?nDbuz0vgZ0Ph~Ut+|ywfgkF;u9va;jF+RZzv7fP4khY)QZ8h5&wy(tizS{^Iss8r zwxtK;u*WHF7>7;OJ{i^FY}2)(SnzsyhKX9(H%XZ}B*;zFmakhrz?rYB-EW;DX}?=c zIov{>JKeR9Et`dM($x;o&l7+<< z6p1iR<~K2QK)U!0_7@!m0L%^}p%uKI4x7O0ypHT#=irT1U-F$(U#C5=CsihN0ABAB zWinQvu(Qfm>xil~_}D3gYOMcZmP%%^)H;Q2nKEio9?aiV)wq=%}KR&v?bBNMfLygS-PJ)R*9$n1cH%qST2R&4zyprj|)d1+_ zt>Y0H1qrfQMFnocM|yT0YV1KLO_K-?uZ0c-&|%AQnwXsiTDRS~EFH9>WucamM+Y?# zZ!-%?g`dap@p4q?R$BhTiFKxN%C1qGdWbfdj&jCnMU=365S`t^sCJy-%bTRWdLGP} zk(a`xNr|Ob;YCc*F1~J13%{SLN&$*cHSayi!B&G$o~#Wt-hWb^UG58SJ#7P`G$ z^cAdFu}?vEvORF?CUgw0_f;bZ@p!cI;iceuMnFxEcT|9VMU${|5gFe!j{>AZCbXKF`LLeTQQ<0 zPZLO+5w%;Bu%p|2nA=B_n=D;KHX@`mnyxT*5NQ?&RpCX^Yzl-0M*+Nl7f_E;`=ngC z5c6_`>a3ChOWZb+bB{_cN(yw>dCA`d5W{dCkm5w2X|2F@(X48&{^CpAR)&Mu-h0F4 zlU# zHf66+*)s>>DVXGn@_K=qClyZp!>it=zMXl{@S%&D03SHix@4o3bfONzi)bhrEyo{L z&TFF`*V$*QlrUnMEN8BaS>Nf{vAN1Nw!37hltj6~J=8j4ivdvJV@z9S%N2;poyda- zP2*TR!AB*Gb8#g*W}V{m58WCysRa<9;b~=$BpQ_^uvl7)oInrDzpumxt7ug)r}6dO zmgjzl_euTE!3=^GMJbtmol~6sr38eW9RhQ|hk=)t>*%pYf52%Ub@Bw^9c9@)o{L8s z+*I8s9PU*fng#OHrMb9bjDS^U#yY&gXc(9~&X|3L;I4_Q;!$55;fv<*^s(&7Kupvy4_nbs>9T>#*fF|- zBan^-%3e_sXNr5YsfJe!l}UfGM@j@F=CKVZ7kNt_QFjWqx|5k>CKJ1Te_+w!imzfj zP&qxjodeuF&Xe4jFC)r_c>TGsJ2u-0|31lOXivy7OL5p#VyfCJsEwqI9APD$?$>Rq zq_l&0zCXd@2Qoj4=*WDDLucGlB&X!zc2QROI~jm--A)N z1_8exa$`Kf1LF-Q=9zx0C-~wa{3S}Pd{GbJt$?SCYFZy6;%g0RiHC>PXa!?g(_ePS z6z%kBxRULRBzAkKx{dy9g{eW0BM(Y_8YuYO8vNQbaI zP&B5pssF5uT}l@IN3(It`-&$F_wC#Mm%Y#b4MF@zQS(1XTGj#rLs|?s5)7bk`xUq% zz-8UUxt$qr(_*8m8(bRMt|mrA8;t(iTr>P*VRYRcUX_Y%^1NCNNC{R^R@wyzlD)VP z%GLf(_Dpp=X7jine>^>(!UgJT!8uX%sRaJ{Mh8+0sg={uZgEB&>aI4)9M*ws=}a=j zk4wT>Bwy}sw_if0)^(-`vkG+H1bI=AGE6c1B;$`QIo9TdLfdXXF)R8dYn>G_*cy}@ z?}uuxoWrEawrN*xKR3~6bnK$wO#Z2uBbBLHsUD7zij@e%icCGE)TrB5m3V#)tzdZ7 zL&f}b9-MA~FF>Iq^Ndy|-A+;ossBf08~W0R$(XWPX_trR^|RN^77ci59%Fsa42xJ0 zO3j!~X;T=_u7C4I?h0G+)xgQ!9voD&DB&;ge`^kMld-zDFgeC{3%ju)FM^W3%#nKc zFdnJfC8p2$n>HuVvU3VOXl2aX22+CuHVSzqoU)n^YSTLTc^p>3uG=0;S#fKCP+$C4(WN-qEo?Fg+pv?GP+zSz(5Evin zdygA#iO1QI{XQIazIkK%57g9?|8u(e3ufy1-)P;J${aUs?;Q~V;P@XZeyoPQE1Eib zfP-|WeWSnZF11!UGBde7SPgXxx)q!h2ZNL@^0j@E-D#>B7iCs!J}B)=kKsGUGE2=# z4EMi&oKM)0g?Z|)y_6!`5S`7)Z{fB4;k|`|APeQ&`WUyp3w1E@M)w^(Ndk zDkn8s7V&Byn2Otb_(=exE^+<$Q@r5zwgB}n3q>8(QAAaE!_ax5%wu`9jmv`6(fDiS zzAMBJP;FnmMULChKy0+@7(8k9?~DfPd51s~s630jX4${f_RJjR0~%H)u47OUsBqTAM@LBlSot6{+{@;(GNfg1}FFj4Ui(GD}GHOSD1d%E5xVMFlU zHBdL!TvA{Jw_mv~9Gxif^I88UoO(?lrZmw46j!-VDxBSy zd6>|Li-)kjW@XxX@|J@$VW$*qix>DQ0AwG$$(cb!hFu(IYsFj(h%* zE<1hH+h2P!*^Uo!FMG zQYLB*O@%MhbwX=2SlS#!`4$8azD0tud!8U@6iN21m3q#`Emz35xG)?qAlK z7&Vgta4XYWyW|0~Mcwp9d0)FlUdCXK?%%B+In`J>u(%{H)%LEHWsE>~oPU*uyIOLn z7~#u#;u^@j*jKX0$7FtYmQueuT@_zIOe&b#JP zf0(Zw8phFtA#3EMm%QJ(7Jgh9Bp>e6g%h2|CG1wgTn$-0|4TFkBkSCD?5pI3goY!vvrZtVhzf{|CL~ zG31J-)`dCUJ35}wivM3*u%7>MTUsK|E0#{OnWW~e$_6}|5dyG ze=C&#oCUvzm#c{zlSvvQ0U|drouRFTw~8_PDI!7!W9+fHT2*iEfd|Gi62cV&l2pG2 zDHnQZo&brhDZn8_Vp?7ZOSeJ;E6Z|)C8-NkCe zm(;#P2PKO@>RB)^qr>tw$Muqc@^{bYju+T>1`7u4hP_2hzm?!w>8D=cOLg>A5K1Kd z)DW6R{w6ly4kMIu%?2Zsb`1yfEBTrZ=2z-99t@MLm-xUH`Bnlm*}gjr7ezP8ekum% zmSY$Jl9qBHA{}S`VpOb^c1u6}FeD6?U48yCB%O{Od*3&- zWBalGdV~({X|Z?&@AfeubTr)Wj%B~KAfW@}AR9rPdq#{2trFpE{SXqKmBp+G{GC(g zIsqYT&SKCayIvVHEw(Y+_&}JCFeELK7!8T-1{y7X$61c?x>gH3n;;Wt0}57m+RS^-C1|f($@mM+bL=5m8w1ysw`NLT z{FY3``{I9+@6>>H=KaIJrc)`|#q?67jo$ncMFh>qoJ&CJkP9$33LJJ}#EH&YPL9mI zmjE?r@hHe0a0vI}TetI*O45Y#D5@CI}>^t(A5+v+ZSA}I_o~+V%Vnsk*LzLo z#stE{+Q8ya>ck^cC0*Jf7zSY`wY4-)xs0{;B|TtNb;^XLsfR?M|i1zCNlX<6g)ohP4D zsR`ddh*FJor@kppLX~u|sFJC=En!4ylr$=BhD;R!@?H3^2o-Z>XwH4qs6HFJl)g)Z zdLHi5)_Kl=AvL;_7J_GGK?18>kQ^ByWKPx#rL<1e(I)2RTTWD3iHd+5ShJpU708M+ z*f|hc77{);x>^ysO~$}$tT8ZE(etsyu+@~r`F)^i7tL7e`Nb89;m2jKhDP32xPl0UoDn^Y^HI`MC;b~NbO=pYn?EhBhDG8&|f`D(3{?=Z?R*;LScBe za~vExxY=bC1Zo!9K@>o%+w4ne2^7s8%>hR2KWVD13_NZcQFI4BPVnXLcx84GG*6_k zgajbj=^gge{Mdq_^2~+O>kBzi1@%bO9j*}4&788be{MZR;NiLQhGsBc`M5c z@)Xa6gA#;)b)Fm}zsF5k=JDOx6x-e{hT8~%*)f*vIwOIlR?27|+v8Veirv#s=>vD2 zo|xK=)}Wg|q-WfEzX2xZw!(5sPmbn-r zM9LHhItX})H&ZNR5gKHTY_!HtUInK64oJ^plFExTRLc`wWA42ndGe~WhWwkxJjWqi z$GXBdOJDd^n4|gAvV|gXF&h)+x^&X?h7b%$#r3)jthkVUH0d0RhtJ9J%nGqYy;wA1 z_J?DB#%^hn;dNY5?ssL_ne^&_`3Ac`Ix@~jT(gBp4?Qv5^ zNzrCM@l~W?`^l3Yha1h7c1ku>v-BZlw&hUWXD?~|zy3Z(UU z!unA?GNAO96o)>i-Ym%eaoKLu+e;h0nfwi_uSs%Er*cg&gfcv9XYmw2dQArrkn+9! z0Jh(&xl|A?Lp=r z%UBvSzr8_u?E|BIIF#z7_(J zO=(_iBP^)%DI5XzXM+ABJIW({(@Gr7V+?+6@bLgbV8ATe- z0QDp*?du)A;LqW-L~$~MA}g>ROYND_#wTw~o<7-|NmHME7Q*tNn49mu+KGYqOYXw8 zn?=S9W9Ey=^!#6(zTc#o8cgz+)TT)>7loO;MwnN)Nb-dzCf=tuDYrn`sOQ}Bk7_!xwZf!+W2J# zq<_$g;^B1-j+XvX>A%$MsxHiq(t4p?cInC4Leky7OBX6(!}Ggo_rT{l$B17Cj+ie* z;CH$Q(l7Eipfcs%AZ!*+$p-OkW;MfA|OMgV9X!e$Eu-SG>i+_7u{|{nb;*Gt5**-iYrJuy7Xf zRJ0VwQvnAg<__nT@c4XUI> zgn(}n_*epk*LDb=MiI)`4P}F4Q35b%l;wFD&)WH)(7Wg4yV7^0S-SwtQ++0`1kINNh|7$UWW(If!usNCr#JmzU!i! zJ#SZiN0!B|;q@9YhW3ABjtr3-*4`E_1t~zA5I2Su&tyq8^#dV$m-VtwIEd z*Hpc%l1jEzy>fdqpFi0}S;A)WHEqTJ(7Tj5otVEE?Ex@FD@Fhe+a8M^#!<;q`iP{0 z_#fLfW#xNZ4-Glt0E*bfrn}4_st>6DY=%^b00V`ujnFH_|Fs$VZ(!hmp`|aU2gEeu zd}K(VL>op+cQxe$X)}p1es}vll$!GX7t)QmagbkwXyZ9YfVgo(k?Pfa|aVPg8ou0lBaUA27rfP6H==lh6AmGdkc?pf6DFKg?R&d`h+U&HjuzSG{lRl=YQ0}yeaK5fqzKsAvL50;wUDfp-9j(sIChgVRe?}*4-CXyR zB8Kj@I>yD5e-NP3fMAIF#1SNf{)84Jq`t0MS8-04yUmDcH!4)VM&i2(1knTm>g%>l z&~L$go)7`bdub3Ka(jkq=cu30%(W(5y#8f($X91cHQ6pIi|>MP=DLb7;XU!EM0*f~UfqfZ9T=z_G(LL))!bVO=bi zi|$chU~v;WDdnr~)%T$QL3|K9q;n9K6!@xZC2B9K7NY?7Z^kxcBq0 zv6rRGxDeo%1v0UhB`LVulCm6~c@H*soRzC5AS&!Wd?~g|ig+Bb3zQ&j5@1W!KO;X^ ztrDb)NP;B>-LfP|-o%R78oxvs^VIaP64#{?DBe-9L*;7!4_|K?)mGbW4HqaB3GQCp z-L<&8ySuwXiUs%L6o=yO?i6=-cX$7|@8`Tf&Uwc8lChJ&8T-m!GS^&l<~@X$00)b- z@&<$5kbjsWRX7%80k{U6@ zD(5jo6zt?4)AW2MJ}<>OAKB9vC@$7#@^#Xm3c1{=^JtNvJY-y$x-fD>k6~pq-gB!KvcEexEA7Sm1`4bI~)>T<>Yi2+&Ch8^F;~Ll$N@m z%PYW7r1T(GQ=F2yj>o<`H1#%ATzRXTac;Ras2km zgLUWxlp7~Jg-n8GSjaZ|3mV6;cjV>s316?s$L5Hi;+){XOebU}G2=x8-n!w$1r{rK zY)bG_^K2+Yav!kZM0MNeb+LN62-Y<+h!O$8{mx7*nSfLj>6liF!-HiS+_nAj z36LR)-?R^-saQbp^G+M^Leai4ZZ|SQ+mneRL_{;s`+VGq{~~H=U-Ld1G6e%xX0Uu{ zFaMNJ>uLg27W#PPd5HHSD|*<^X5)hCXj^@{$os=MGdTuVa@_t(#ahwNd+&tH;K+O)XsCekTn&T>9DkQ!?g$z!yS}?JyhCk=2L4 z*{v727M5-s3rRn0>~2^ucC>r;g?;EQQHI-dqc;zsLBeqgP7FD@BrgKqj~{;X6yl8f z;IQ-7wX{1b^x5}VB%{qaENQTatRlHXpl$_|Y@(Ejp8P7{QWpoxUM!bIevhQko+63g zq(sVUKL~~mr8sj|q7%DQ8n1WEVt{6_ncZGbTBjdJP<7s%Zz`0kUqczG%xCzjiOT>1 zw!pEv33}#Po6{a+_Xs9fbYl2Ph%%wxGr?xV<{j0H7>D0Hlf8$EWy+*^bAPv3q~U{d ztvy8U36TSMWtva1jPxt7fd~)YY>hCA14-pYicE8Wo+p(xz`Qj|V}2v?4{=xuHUJ(p z3g#x4g&b35rY}tCG!7jCg@bZ{D(Re0WTyXpT$=GDoB#}6jMhv{L!9eJ&;%S&x8^WW zC9{$+Ng=wl*&gfh!Q;qzAN5e+*A1gU&*1S^STS*+QBwN|EQ*xbneQxo(Dvya_@5e_ zbeyQH1F_-MaVWB9p?FJlA6urLHl!-6q$U_=N1IncPe9GLWL)fgyFU}u`$0a1SZjI* z)Hy^!wC_YS)~2=3n`;{ucppz}DvzxVd{IijEdckI%DMJ}p<%3dYpiQaSW__7d)Qal zd$4P$&Q6?YGsoq!U-SEcov17^@`HzK=b#-}m5gz?986fSK<5O7goD8&g`q|q@>QP_ zW3$RRhht!U)x$c=q_6aeuYeaBE+l>%Sdq`l-iob8u0c9bfd`0k5+YN23$vL{B9i_$R|M|*f}Fr=Y0E#BRflS^=R3b;eK zh7819p)}V>*p*V=@?T_4fek4?J&gK$HT5~!Vn6E}e}0NyDD$Z-R#c}G)uYC{8gadhK31lUVH$Nn*iO8yt3s;XKfgqYVvW(3 ziZP1$^s}cu|8id$>2HrWB1iO6BY{o7Tp6SSl-hm|}nZY<6P1Dc+P9 zQ7wixngyBVzwBwi|GG^&C)3xI?oxlpSstvaw5ind zgE#}z*TlJK@ptWj{hhUbrhE&VoF?{xr6zAE-Ib@#$hDZ(joK35O~OCu0hC-yV~RZH z;`&4P=U7fXn9ThHt>GK_WfK1yS6B$MN7m#9;iYPG#-1e>nEqJ(^?~EcR|I*(Zx;s; z6=|&eMe0mVQrOdv3OEmud194RwhPw7%M^z9M=EkM@o1Wv-sIVc}#p9&bfDcz2iu9L~<+m?$QFFYF8ea{sv zoeHasX-VD!2HBk+&W_4fy+C&W_L?iiIxE3GJe1Ff&xr0`g2eI2j0B_)&9p@1))fq% zD!HRR4kk_6mQXX_ucg4sOzhLth*~q3%xXzY=ln#O>vE$>btYB)At0+RW|4VHPsh&= z;=pPcmx3z~oQr0E+Z~OwX`eeI?{-t~IW%6dwZv|o3VB0@mpjy_Q!ss3oS5*0&tEOm zjX7z}>~T-sp%6gj!uT*)x$Wt>jz3w-@~DB6$5Ca7S0Y}yy9w<3BguZK6_E(hA{<}Q zC)MI+5h=|zrla7&3mhEiIwOQ!IW^>BjFM8pNX^`5lTICHm>*?r@pvoS1?rV2Fh+z(WyNlurk zGhY-VDutwEDmPJXxC$q+Ch;vZ2m*Umd7_!8X($782;@uUw-9O^&;!CifL)S?MbIPisNq!v(eE4%7Y2 z>3nC5_6HU9gj9uLn!*TQ@)S#YJzR+oVWR0;R|ItZwCLdCn$NaQ2Hhv!u0`b2YlC8i zG(;tk0=U8t{w|q;3-N$aiKlTfw)s1`>1zEHwE?5|nu6g4l|NDCse`uE;TL++l}Bvw z#r=pLmtJqeBO0PP#2P(WRAY>wCH~;Gje<5=W;p%?!H~>&a5t+^ZL-g(Rt*8!%<0W^ zD!B8|d)fD4cO6X{f+4MHJ*O1uBa>{Ns1By0HsB)F_`|#aqWNt>2S#h%{Pjgi0G?NP z#F^!}$He6*YT2tM>n6;|7d`K^giqk3W3Iwr&Lc0Sdb_G8E#sD%%}mylQG0HT;Jz!> zjw#go*#yTuDzB}Q^~9=yhIhy6{rfn)1C{qY`;h6`9zvnr0e=gG-Kt$`jgE|_)k}|- zFYpfRndt$9^5n?l8LJq~pJ_b2Oxy^wr?T6t8w?Gb_~~ zt<&IBoGQG${cmft10)~AM1HcZDI4DE-x9|l_>*nyPW1lYKmL`-9fy14;zs=PrTF{* zYQsOhS^pW1{rgHb`-=Dj7`Gr+M#m3~XgfmQDsKCk;fgLdPTryMOYxPa#(QV|6S>Eo`&@9mhzjfNJS7G^u z_^3KmiU9T`0}4jD&%yUDBU|kQQ#$UI9aHBVX*X?)j3mVxl-v783L$TSR%^j`Zy|v0 zM9EDgIJFgkPr;Wg`ezniWoEXLP41AVYyw=mc``=J+kCu3)S#}H^m-9Y-Z4r~7n0*E zs=o6Xdju!^qb++WFfB}xnDm0ldm+kHgwHb;n!&YBuo=#cw)Ap+On?vLfz3YR;@e7jwS#aov28|^+#Bqj3fnFYUp#Ri*k2x z)4h0DV=-Kg-M&L7&!@xB%iY^Mm;s?HoHRxfU73bPeU)KgcSD@;k{AKmH`P&YjldlP zd=|i**vv`Tx9)n@*XQdxK7U>z<*0r5UN_@W;T+(_Q}afGza<9+vhbCFr6~gT_25*w zNs))tNNwH4$fMLFQQWjB?L)jkP{6ip9xH;0coYsK6IPHsK#pBa)F<+HCpPvzGtI@Y zMwkL$jJaL3VTM)Sb3EZwSeU2*cQHa_xJv{mowC;aIBtjO3jQJQufy<8i;(y;{hZyr z+^QCa7Uid+_eb4|PCZCerKm))UNnt_?!qqsUJ1LruOL~GHr z_x9w6I;NPt!GEwQb4a;y-p^Y-{CWMK!Ms&FJSC7tS82>HkP*=FK*f1lB`M3yea5(M zDF2fa%R!tqKASV!4fTSbhpuQj?D56%m1}mk8+t5<5+l+iurfU1eR6M0p_dhgU|U_4 z_f09Cm9Tq-P@+s(=g>w?rkwV6vx>UO!^VweUHYzMM5!~Z3D@5u4_~8_g}D88FX4n1 z)(!|u;A^xOjP2bg9sXSgK-d#Szv-ITS;ZIm-XrfqjddgoBuA za%ngxG19fJu#~cCWYaDA6mR4b>`f8JV6NY)jIkBwv!@8Zhz{)<;}yG!Rp?M=j`Ub< ziL;q^16Dewt}`e0cz1xFD<0225tRp0Ein)g6`p8~f|NjU+pdi_<|sh^lr<)7%x+h; zj%Zig3%A<%z2!^}x;^h0$}cJ>Sg|o9Aug5in21v+DoLDUo2kZNJ%lx}9=h{xm;2AW9C+Ha189yCB|9!DlLAME=atX|L#HU9ttp8=Z~W^0FTJ8`%%*xLYQzzkpl zE0aY600ScJg##r60hWkZ72e;s$uaQ>9vyq(D72a_9&#}jKFJ`b!7;}Cb&gYeqWJ1> z(2i47Mo8^CeIk*YV3@tc1Cta+c*4ZwlMF^E5X70A9ie}k=!k=hLa{}@3J4fCX=}x8 z3^B>hCmR)?hR4@}CVDJKRPM+EAih@anaD;LgeO@Yew_;fwjO+q44SCLYu*RQlKfsZ zhFpoSO;aMmYgDDHOc9w@IeT3FaT(0=nbD&g)w|^>bj);%ekwQUNtJa5EkqZ!SVh4) z$YFVk3Pg}5_eRb6HX)TqX;-R(sw9#U80CGARQw_*7sQ=HLBZ#&dH7o=0xhpIQ@C9FBppPPGd zq@$<#%XW-d!qa+W83(z@8jVYjQtN_hc1bNWV3TFRu8!}Fa|v6Yp@(_gzM&JO*hjd$xQF3>!RDA_ zVd3JFK*4(gt!^BYRtrhHTwC0`DI)XzZ+%Y~2kws?Q>)vnqB~Hpf<#N&ZmZk^w&X1# zg$!tuj>HA@k?}G;t`ZCgEx%TguzW&hVbmpQV7>_gPGiwjP7#JPadpvu|K^k(4Rpaz z8o?8W=MxYoZ3zTPOdx>A@#!pzIb>EI@>k$xvqcZLw#AeZM)M4G!d_slIR)|37Wh6O z{)4B#4YbX*9yK$^eOS8-UlMgpiD zi&l|RLcvM~w5MhZ-2jAIlT@-};3Is4mL)AM-75+1&Z!X5^eWH6>cUWJW!3}YmzTwC zjY5r_B84)gvX$=|Rfa?fq6lsn)_+k*t?#4*&a_%521es#);RGSC-{67vi?9**Ozv_ z76UJ&$A3}fBL<^fVs=9#v!Eq z(ayOhHiZZ0He=rXO(6JYw`(Fz6wnb1zamXLOiP;12~!U zHr`~TJ@l8Y1VPm-e9aejTn}xlgqg~kJvh@Lz!nIrW$MSC~3e!q*mkK4#kMm9+GqqUn zwC>96Q^?kG$VrNU+6yE28`qKex|((Evey(;zL%@F4|Y>5zwk&p3e!5|!hRo+b?hOH zSQQ##uL~ng*{r3o(atUHAdN4P{dU?1B*)RknRj!@BItU~HP($~*p}nscI4H>4S)mV z@w9G01q{zTqHCIls(4-yLQ+!=?Ks6HDW#c)HAK%i+w}wRnzSVAu~)-WsWgu3zkCcW z?bv=~?S7|U>Bu{RgCd!1J1t&z6Qo;!uCLHqNqQTPch~$aqmH9GV!DLuci zx0162-&g^%XaWX@;@GK%KQ68C@09?lWgl*VgkRbxYERhsFpI=;^@^}Vl4in@OKZKR zK$mqoU)CtJ5o^*ccEsX{C+N8AXA<5XjW5NUoRILo^7c8_XnMB#{S0zQ zu1a^qOUa)>Q%n;>VX>*CN+N@cMSc%+BiMR`Ny8@0l-Lg91pNhk^Bs5x%*o5MC&h9J z@!nu3BY;CLM&t?_1UPV0w%q*dOh|B6iC*`ayF34T)%(-PLH&Q$01^r{(+h3W{#gT{ zla@qvv=salBvw70X-E?LN@&iJ!0gIWJ320Lr%qJ_33(@bDIVq?|1-3@V|65>*?c^; z_wDLt2J;KgBD4ct*WH&-+79?Cw1YDg`lyR&{-KH zKzXeE008`L=F_8pC2@RMeew{D+)mQ%-Q&`*O}vL4qx9R9xL&DL0s(6LniN%$7H-9G z5wDh|B0BHS5X>0z1w*J|9(EQ+SQJXRZC$CVJ^@efTVBX2nqE|!%DE)jnJ?(N;v5E) z2ou;CQiz3NRNUu}OWMPG6OA?hk=LbgW@WBaO#ys+)LFQ`-Q%AaytT|cSh5D~4LLHF zy}i%-kJJd$q3=EJvZ6Pu8(KtOAH$M(l5a$Ql0bHTB|;$cNx@osSUqImh_#G446Y!w zpE(4aF|y_ZNu6{hN~6&FEkX_NRxg-&IUo1|&BU|Cz&65jgY@}gfEHSnS8pZqnQn1M zsCB6Z=`HbZcFTprqKU@kjjsC*lBPT4xyB%cXcd}S{}7erNi64cls_WA&2RX%S&Ckc zAUJ*PGel@D^Ayhmw)f2n1+s5!+H|%Y>x;Np)nEBy2<2Vd#f$2bcAD zV8iE0hQ2-{y&>8ga_^W7*C9@Z1Y{<{b4DA+nvI`7-SfCxet6$Jbbg^XmL6=2)_1Wv z^vE|9?Vb1^TMsoh-r1@`L!fT2F?>2qN)-y$K=4_oxJ98y^rtJzz9ZN<~3-9{G7yi z+aINKJ3frq3|lV$j}y-;Rip0t+-d!!d)qR&FfHF5Wqdlod|EfM3^)?_ox)S}5iY!D z(YHx!8F|makQ(z>^T=6^i|#c3(Nb$vY_iT2j6>sA(=b@?kl+69y%(WRseV2QQm@Uk zg=kjMrWoP#{@`OfXK?GKrByLyxT{5$3bek}EGo+Yn>j;!n3iE-xDv?+iSwySa&5J(?m zB-$xO2hQ_JGIN0k4CK*H-v%oju-z0VTvIO9+2wx3sfcRW3iK1gXMhe95sQ9M;m4d=B%oFQNz0Z0n%(HY474U z3*VJw@3Uh$piLT@=5xJFJ#y&xV>vVNDB8T^2`PryVXe}Hi^&4K_>jUbhpU@+Jkv*B z=>GFS_usgdCqUfw7ro;gA9dl}D=>ny-^{(2Ti81&Ys}nhHh`UDt(>**D38d_K{_P1 zk!p7ql}v3hwPO-o(HjQt(5P71$HfqA;i*Ae@A@(R{NRYgIY$pcYh$~ znf#nnm#DwogIkSZKhWQvzm*}%!U>(gH?U#9M}*Sm>;_uQX>+RDj1TG?);;134Zd*x zHlmA+8I-JZ*#Dx>8X-HpiDvri&+4p|a;Sd=DPT2BDzZH)It6OSJ*r8h<++0n(?WH( z@0MRq?jv}oy@1k_=i2A(-+qbwtA(@3=cC8_|L1R%j4Kb%m#jd82n|$PkwaBP)BS_b z>4JACOAm?MkL(RL@Lb3yMs^=2WE3X0J}x8g=s@pE|EaIj`BPoC4u>#OKJ8I)J2R@>symimJ(H;{QCzEa zKc_BN%(ShK@ujod7qmC!^}Q8=Eix%m@A@b?(&b!9HvBQYu_P&ci={>eO>IR8|H$PnMD3AJ^}XJwA-B?Rmju)4+s=YJ5-4BqnX^~+mKNDm{9W_;IHdTzQ!(Bblre+R$w4G1npLv_gx zJwZ#($xwq6>SH$eB>zkq5Rjy;iN8 zyaX7w(J5CuJE6hU59S4>Jl!w&<@ZvKxI7Am4R}`+#bA^O26C>nY7?&5;{xdW)t;H; zIRh7l#>rSb;zKQzdJ089MM8qyDsFKmprs8uor0e) z#NbS116EoB7M{Td>x@Ve(}U!l>nR@oImu1%?2m%aWNiPx(UVUoH+hN+UJ+=kxbhE# z%cs3EFi>LKzL66&9f5UXEgBmjubQK}!Qog=b5ux*tJo>k4fpt+|3z^-pL%wrcbwUL zVl-pv=KlQ^+yJf=3IjlkqrqHdV?Y*&CW8tvCXfGX5+}?a!6jqP6e9_WbJ`0sY*}sD zOkTZjI$oBI2{9xAN%vmn0G)|+){0((vBG=%q7m|s{ZJDp?;Tr3^KTnelr9)a)8C5? zQmmejW!&<9S|620P3{qxr`Sl6A>?L~1dyD2^o@K>4H&q+Qabc!M#507UwAJ7uydO0 zEp0VjwHj$$w1Xfp*xu&+i5|2WgblO1&O-&9+MaG5K2L=Q`M-}v^B!IsD1i2!B*Xz;m1v!X} zo;J&hrIfGQx5w} zOJ_;1;TIKe*Wu~%*X?YRSooxhg9huCJZJDEgvn-Nr6L7DL$_Z0i-fuh{hcYY5*uQwwVB2r_62v^JH|2ZS04 z+~PYT)4nT7ZI&~gzgv@F(SOX@VI@gh9O9@p)3frAAu~~kgH|K`K(^^TLsi}+-$&{& z#34HDsKYlnpO1#a0&!xJM3mM^T*Aoq_yjMKM7~vQe$|`+P9F1URey5IYTC1MZl^vL z&HH5R#Ks!1Ywz&6FOEoCZ`t`uoNL!q#!`w8nUtj)j+cqu+C>hSxy@Wu?e&Ekv4O#B zLk*w;V{FM9TEkB4g`LU`qrrAl3l`~J6wRQDK?zdr{6dMSIE}tc#KlHp&uzRR%B!y( zTf{?-B7WQEW`AZKmi(!Ccc_zY3O=r)3PKZGI4Tx{2mK`xx@d3LpX`XF zL|$!yP8C&RjvzycmZ62__GwqykBmgR?kS)?)Hoe(vHR~$?NZpGHDG$IMxK|L{wjAMw zca;m4>{%T-G!mx+4j&C#*yh8j=;dI0MY%9(O573#{4jY;1^w80002!M6F{BYUhs&U ztvK(X2P+Sx2Wt;#Li0xu2)F~}!Fi8}vGl;on!JSW$le0@gYy?q1;4PW3YHd2rK!@d zmKIaL3MkJti7MNz7EmFc&#S=mmr^m<&Hn=7vZ+SMTU$|$96Nl2&@3LL0x_`oL~O03 zSs5sgT)N7Z_sOPl<7UmOuyo`XW31*C!#`#evTxs{EeB0lSh3*R_G1Fs=0F#gCHVR? zDij?h#dbcT%6>A|)IV`q(R&D4=OPHHJkn89=vCa)z*tLR|6bl z6S<){UTw4YW*V*2X0%2QXlVy|rVx?`o}7W=Xle}z@L>ln&GkSHY7~ypzJOZ9G2t`@ zPM+bJbaYy&&cT3cQI#KYIL<8tv?D`hHTv=>&cS`WIzpLY+SNXx=xUOvsi|Owg*p%U zQGXF0Bu8z!?h8><9iH|AGn}`Zg)ppN!2LC%MOYh(>$_SBmRd2w)MHK4$)eH5_rR~J zXbWoz=rNMdp#y;(3FmU|MX<^%Dv|T$9y_f`U-vgw+4Pt7coVz~% za`1zH<;J&q5X01J>LIk-DJlKolCzuOdV1j2WO_48rQ}fdl+^Zn8O1%G{0Y-`u(_XnHhf4q=@|W%Vc)Xl3b(IT+*=(D zeiTkar8(^+AwYBhcF)&~avB%@ai};Hd2WU&WK`6KMv`c|8xs~}$&E*9ABkF5iq;;M z#smz9e~R^W?lLMU8p>?RQ~6ove9dOnPuwduYX~_qZ_Jcge|SyEU!ss+x~-E&sS=Ib zcZ)($_psO?O1z*bOp%~RP=m=;V^DbCmw<3q*)20h{r3CQI&Fr4jY;V*7RnJOZ7@9b3`v{#J!vjD;$)eW2_9=$s$YeA-L0gyG)${Br6B2EnA$IZhtBziQGNtl><1)}U zJQI)7RGEoIW~XMdz-YoS&Q7XzDekkDf9?RKeNSg-(}VFB-=e`ER~W$f*KmHe zsUi!P-#y`oGewuUJ;#wYFh@;{sm2w;knI)~?Z&X$c9>0xoOF&H8)~Edz08f8n|x6R?t5oy(-CJCmT5u~-*XErA4)^JS9qSCCXQi^r}`0Nx4^uiwQ;bnZIToN`3+LD=uZyiT(&domBv#RGEVL!#Z+A zwr-_%)qM2Bm)OHvVK|`aAEal^+Vrxa3RX3fCaRZII%ktA@-m4{$E}8f!%=o+m5yKG zekAo&$sIz^{;;EAm$}lVsgGfQRbg(^z4oA4Cl%S&30wAehn~epssCn=R&*da>y<%$ z9{F4Q&m>)~eR9J*nR5YfmP?#F{M%&$o-0JjA*9Drn3&apHR+6?591&qsT*4q{@KiB zw3|!jz_@M26hu^ot?Hku}o5NzyvIMRt;%{HiU~2Y@589J>R*; zT0fqnnRf(eE7x6dHoKu>NN3gMKAaf~tTWE22(brw!yVS7sLTOie}acoPOw)`=gt}z z9M4~HhEty4+7%x1Y9LO1zK0(~GoMIS2o5zgxnnmeJyl&@q|em##t^RamR?VXjL`<^ zJsNe{cS!pV=yYe;%(9;I9&R1UQ~FiX+6VMyJ<-SS=M#;UHoVq_)xQFr42jy~wl4`+ zQ-6Hepta2u5!zB%y>|<}<50d2KHvdau2@3;jyp{!L4ikihdBH zkz&ZL7)-CeR4>^6_!j5RqYeM7!QII8#gY|+11h+)itP1V@t+p&lI%5K2=>dDHRAt5 z!38frMRK4MKR@WI0dUE5iSW)imVlzRfX`82!sqpWKy$u(z`yd%@X1E$NEkp@#Syt3 z0i-fXA(=?e;DwsDI7p(eWnFP|Ql-HeJuj{V>0so{EPn||HcP+w3*aG{^TXOwRH2AC zq zeh9ZP)(%eU7R^6mKjhz2Got~eUCU;J*kpa|unX4*g)5@lTsz6ZZ-9p+eX(z3iL9O2 z%)mQ;gDM&&{Lw)$L@r@-RqE=bj7Q?-Ek6kt@td6PW6x9EiG^hkZQ}W@)WywxPPedH zwS{E&QX2YVyAx7a5#NxU;H*p#X8!q&2JJ{_&c7tmvxDC zLi{m#en<<7BX&u9|B`uC{T!~zgLGrnK0$0NaYjzeJFTigbM5)n9*Iypi;(Yll&_<$o-q0Or8a%`Hg#t0PitqTJom)=%NE5Kd!dIPc?;w|lv{B@ zWH4D08cDz^;;rhS3DRBCT{1LM3zT^>o1pJ`3_;L(JH$i5C?G--!%z>h$S>>AB%!!E zZR7n0SVV2Zq5%>9jF5*1MwBw<_D)Q+#wZs~OhhEXAw!13v+a;Q>uCSCbrTnQpVsF z8Ab4BG!ZwJN_Vj(@#?OsL+LH!*9~SkQp09RR_XI8s26c0>R;<9l%fokL1R*pJoUdh zOq@>8LT~*=sQX7OrF9zk)jOR8FVrSR4j(47zSZ@tJ)_X)bB)ldc*-y^Los-_hlbww zNf`rEBxhz*tzwBdE3ngi#eZ^|Vx;_xuG3<>wpK1woRw1Lmh%Vw{tKQqlh$okn$BfL zSH$(ZZAY&_l)-g~Zma~Sp<(D3?Z%jT6y)-4f^OQ-s=kc;1L_{jF$T-mpjxxJ;o5IF z@+d>gR({4}bghz*+njY)>32acI&5wPC5dc6Fj=mdIu=leaHZ#5il36IYrx+}E#&W* zY{GN(45=sWV$%5a@-)z)y4xXXQyzw}HayDpEd7$TR%p~SE1qimw;4P~dNeKNm=^18BC^AzD^UWVT7na@+{{I`+?e(zP(wvukoEHVN3k zUqwr*89HCKC+9{r%l;~hjO!9unfPXzCJkVhpsw8oG$8w$<6G3b50^2zGZlAQA-_#0 zm8feLQf521#@C{+lbB{Ej)jZq@x0lMc=JV>SKhX66QpfFW*spJ-B3#8(MBdCiL%Hx zaAt9^((L%Oang*DezqWwoB=kBfAlnqv~#akqdYZIkDsVz!$g{ z4;|5Et0TW!Da^sJZ(XA-vE*@eAWIdTMY6W>C&AOQ6%>eef0!i%*YnU*7OlR4|~9UlPG53O?Et zQ+B(L0Q^1dT$h*N5i(9w?+%F%n74g>5u422w}G_QdxmfY&2V89BusR3Hzlx!=P>#k zA8Zis;1P8A6ppnKc(@@du=Tk7Gq(ykVceL&Kk{6?ef5iLs3%nJ!yB+Suu}iTf@Va_cT-!^$ z)+!7AKAxW7x=>m1X}LTO4f?xF0@=QuAP9Yn(f!4Alp9WkNs@A}(Bm3Yg-bwP4u*z?-q>s4y^{rlrufqbh_C4isArSvN2X9HH! z#4%b7Vvn2)WlP~)6M%_!p5az+-^)!;iB!5(=c+25QnXrTyDAHtfC{-|GL6s`imG0X z!-RIs8D!0n;M>hOl~lbr{v0=HZ(A({Key54$DA~+RN(-DGDwlm>%a+X z09M{KWq3z7hfMuin%;uwn=1zH(jP;S;;A-k_S#1mDOpgQd>Uc{O99qEM9ZN&n*Yy1 z$yMOU_ND+9%5SJy{+mwCY?u(Y&(RpJW{Ip3b|Y0<-XWE(ZtCmnF^rV9aFe&NhqtwG zq+bXyG7PoKIMTNO7UWr@dff%3xjf1e&2^1YUiRAc*hnJ<;Y~;I`tNZ5k6tV8;%D+g z{jbT+XW{YrO7_cx=L3#>&IHi-X4I`B>&w;Db0fa{{`{m43sIp386}3lNaC4Vafc;@ z73mk?LSk+&$Dr6R2f&$b@*|j!^F8?ab^^a}_B8~3U^Lk58Z7~OBRY&@-p&t&V7hK!lzm0>Tvp}Gjww#ICN{=lW_X) zal;ut6iS!AsQCR0H9GBOSFwHG-lEUz|CDjkQve*Gu%WSq>HiQ_m35W>G2>V%Tw{kP zq$@|df*>=nqyl^W0zr%rz>nr2VtPpjuT42$mrz*rqgQ{6hFeJ03~wXlN<56i5}k@S zrO@4dqPhLx^!G|uXNT`M#;|XA%6Kz)?p%OqZ#-wb2d@1*=)4WMQ z60%4oV2DSmX*(D!^}1qslK{E9A4^C9J>PK+E)5*t7&Z)^swoT?){f}UCc7B=YBw7L zfI8)BQ4_#KIz#e=@4PTOQrKQliQyKsH_rHVhJ*nNG-KZEJfnO-rsRL*eEC6}Dsy0+ zq{)6)&|=$M5YBg`E>oc2u(iWakVywzB6E)>YB11RF&dpPjK!c%56Gd>uj!Y1a~0&A z7!^v5%`7xqqf5LSG_&qcUf9|W0cZzO#FB(R zD=VT!SJ6F7eps57E4h3a?wHeyc{O;{TC^gh!l<37w3VJ^24$>HUwrR1z|FNR)UA$i z)qYWyO*c5?UBj1|bYV`r|AMl9I7PoDOIR+rZ9yA0HhIJsWg9s!o_cDr&$4?8)&l%) z5EK`2Ldm77Bn(e_{~86v;TH%Y!c`a^1^gP zSc88igT(VYBJ%2Awn@*Qk6!0}ay<;F6v8kThdFwrzSdL|T7>z9m4$)Qo?=W zR+?SXVB5ll6#CB*>%cv2Xd5zD7ZjFT*6A|Jo_=YL}(_{w#D|{+qp} z@BuFYWd6^e`M0XJI<6|(I|6BjNlL@d$u#PAu7)ioBMV5PMJXy3Tkt6b#s)2lULyRI zQ9wd!0RQtT;ibMx61lD^9j^Xzz0ci8(F-Z`Y~SWF?UjFjyt|B_s-<5Cxy9plkbxmX)!o%YTc3`D9(bk*StOetc>PqQxgHP~ zqEj(jl;huvzegfN;t^7Y8mI_z6rvYlCGVyMB&4WFocTiXp)5p{!eXZ=G5%zN{KJ$J zn|MO4^eEG>yZUQ4iL{ZdIgdDmOtL24r@{!5g<6el$XKvH`B9-CVzlHD>V}$jv=~Ua zg^JE@hSkDKaTz=~Nk}MVe;9KYRIt+Dr(`IM-SQOm148yMsX+`qV2fzW774G08Kg zKfV3}nunr6N!&VveZ)`9U6KgbUf)Z0#x@?RK@IJ_?8J<35IxgAyoNo~O0h7}>ukkc zIe>hd4tCkdWK@aj@-Bevm`S9*picG!O4wpt6BSU6p*L_`h$UAbs|}=uDhRkt zD!>oF;AFsQZg!B-gm!vg%sI2CZ&uk&Em(P&zQGBMcLtw8f-8(X5gA{IlI=MiT)JYU z8sZezaMwN21|_&w$m(D&6!w|;3$jJad`&c7tD{{;w?gl2?!BgmN=kHd1 zV5nY%{*WS|B|$8p{Mt_az|y9^VOq=8!(W(yAki~;F_R=UgEQTjf&VZ#qHL_OPBozw z@l>W-LZC}H%_MLAQ@;i{%K~n78MY#-2S2kfx+xT?o|#7(9@o47G6cTYyCQay4awQh z&?={=AK*&1ba@G^L7f~WBqQo3{N`(Sa0tw|iV5?&lIN#fx5$;Y&bJP`Ce zQLp>)^IpS^BdF)R%}yIpq(}|!Jr9m<{8LRj8&7-hd=?On(4Q`|{~ul77#vyGwmTC} zY-^HCJ~j$ca=rYRnFlKTi6Oc944&q&?$rV@m?(jcl!gH@ca4?C5-ahp zR`Ys=y3F{0zklFe4aSJ=ey<&2tJ5Q8O4zvCnxgv_vdma*tKY}J&}|9i)X{pSw*7-Z0!@10lHWC_EoFZ0U-bSYvHT;l%2r_m1R7< zpb4F-Q0UrPrA9w8 zf9yO}dEt~mxOVru8_kA8mF&fQp9d*?M;s-Yv{O|MEZ*zZ!?J}*2O@6Pk-S`qZ2r5O z3zy<$vqE6rhjC6zZn=e{$eMUs)UWW=s}P9Y`|?P5L@HE;9>yE)4Ob|mpc6J?J3jq( zL^JYPX__HV)e5_5p5!y&?7y~Xx__ZR$Wa9n^H*?&B> zl_Z;V)XcwYi68|bNeQq$YZ~SMkkhhM>HESzixV-&&9B=X8Td& zLRu0yp0b^$Cf4=3x<0>MquL-u;3soad#T@4pYABYB{XXYfh+N17WpJ1JqIm-GSB?5 z$GH2Bzb*SMTI)N$VYEShMHz1omf%y@{p`XdMi{NWwuo` zIS;>M%n)O6vGv-8l0!hZ>eG^L^$?atFxxn6I-rMOnOkN*n)04>-~>?OJ2l;G7=rql z6_!^@w|XN3OoM5$|EWY$|4Lum#11mI&hA3ghv9Ij7}uP&PAsb-#C#=vto;~+i4NV> zemw@$#Z57IQbPS(RwS@w?2f)%8={$h5%&OyQ1~X*M`|OGmaHEzDSdNP&s69M`oth5 z9-&Jpsr!GUH zwX^5*KIb(!pW^cU-%S>C2TY^p>xtI=H=^`Kk;YE3clzqdt-d4xh&)Zfz+`z#B=Cch z?@};e^4|VGe_xky7eMz75qInk#5&Ig2>{SI4b6;2uM?O)$X)Exv7jFq(k>@#C$sC^ zn%?$~QmVe0UMu(12VjA_p{*;~>-B(wBcO3BUaR+@fWtteCzy-aAzw6;PRLliDu(WP zLfafm5K?Rb6p!U*LU1S{zRY#N0%u{%Quzf2P;JXn2U5d1_EpA(jKs}tx1C}LvvJl8 zF(=TTB0+_k;LOzY*q)jXrlMFvFk3&thyzx>#xl#-)|1vwU1pFWPo^b5>X&SL*WaP` z4`>E=^i|WTO(Rf)2<{#WG?2*h<(fd7<1;k9X800--3QFOUJJe2G08~FP77rII1iD` zIE>gc3@pvmI-{BV*?}Bz6*fy`=ej|EYkyO!uO2G4b9gM)CfW2jG+fUKuQE8d2!esU z>}*O8EydzzNj#1_dP-!X)-BxGMX5pgV>a4IZKetV8Qh(zHWC-I;TaU4< z(@gyXdP7w)mY-AD+M!CaleKS}tSx_}()^*)T$3a{ikjT1#R4`WMJU~rz4DIJzTUJ_ z=Ta~ADRKV%{VeClT)n8ov_9gL@d4{hk?8?oGl+}^Da)aMUlz$SvE7Dv81hwax-sE1 z$k|^S=$n<}&;UH2c@%TTMZ!GJBNLU2-kcK#%~aUOc|D#VQR?wk_@)nSn1(^jh#CeP z;=#|T#~!FD^b{u?EF2d5nt1-hR^*xlq-mN%`8oO#rQgl6*)g3C_g5AHU(qgU^h_E< zRo)q>fgy(a{k3l-WsmXm&7|Ewdx7lgiCpYg-B$f~gZQr?Rnryj*Y{s3UzQ^o(hx!r zq`7wjen>x7u+;sDA!ErPQO!VRtuHs7{f?{I5J**N%~vX zHdx~`*P_cwpVzyQepVZwS-7%>y<4 zHRBE$M!~;(W-|GaOmJZ5SRGSq6}k)HqQe6{v|a5{>fleC54Yg~rSRI?2B{7>ICu`k z?c&8kRW`fwc$EzM8-uic;arYQU*YZgi`W=CuyuQWUPph0ZX?|%H*gL-id=|;iSl=U z%LgAa1U_hANtNygOSUnhlg^*|B8Q68C$b5WG|g7h?cwc+1(ilq_1K;p7J*kLLlcIe&%qVM-Jl!c+7bcTU>j-5qJmHvo4DvElAsIMa)HdrA<&|3(~=#Z1_ zDx~LA>R_fkDgvsA!AX%HQ++YQiK7>w5KlI{|3dS>4wlXYEnH$tTW8v^m=km` z5iYijl3w%pFZ5^1O$p>W63V2FkeK#MW=e#76mu7>8Uelmb~c+_{!UL!k_Z)QT|4dN z8eQ}*k&Y9b{_o${Ie|%^z_1uBB=lkQupiCQI6hWcv!r{!i^c-uI12Hi{-TI>dyK=m zq>yuqm2Fu22$Z~_rSQ&#W#t(&Go_V%{#y(~mmDPPUxo1aU(4hdZZXFJN0Nd_fq;<$ zo`Q%12v!)81ZBYd$j^$-S}a^`jR`bI6poKde4X&Q_CalTHC6SEwdUAxaG*Sb1LKYsT@q7prH@_`AJ-i%(ukSL2yLET zMBa+I&&pE>RNz3x9#yAqxgH9LGvO5AxyP0Q*cqUqZCN2DcaE15rC*XTPSGn2$Ic!$ zDrHtW<17k5jttSR?W@ONaFP6Fus~%X3+_M^BvT*l_vBeJvb-G&vo-045bZkE%aha1 zC83B`=aa30w|W(!6)-Xtgi^$g_V<#>?ImZ~E?BZ~hX?&!c$FMG<6Zgi6)xbBUyL*q z5Z@05SRZ*RsW+#5@02twauJiS@JL}Qd{5?CjPbIY#^64Lpg}!9LuvEN3!^MjsCk20 zt2ktr_w{$W3Ef2zy|l41vZgsiZCLPKPBA#48r1L#c)Jd%&z|6a_}3%g=VYOl*4M-M zmnJOre+$w7g^}*8;V1!dW42!i2k>)bLiTh$A=%N&?_1a}B!4iN;${9gN|JgaFk)9D zGkRMkIb%6N4^(&1pVWx;N&JQ*0c?hW>U6i+%&fiq)He&_eAq-h=Ws z&aTfvO(ERZt2}70{%%z-0X^DoA->yXZ?Gi~BC+m2Ap?3=??JG_FXY?rpRjK6_z+`; zyrO^PIGTw=N`C&rQssJ!`@I^SSi0dff4DCr=9&HDY@VrA3*jv*ic_eFY$%Tk|0O-) zg5vVh^~Jnr;wccY!QTm=9V!k_Mnp{UKI2E_@XPTfmWkS73GG({5->h}#bXgop`WK! z_vT5IG2Gj5A+a1z+?W5LH*A^4Osx4dXI7gJDgqTj<$hhQs+cburX+YsQdRbabG<;P z-y+1J{FFsrd6p~fr6Eo3XB6e=&mShMXG71?`$);WQ}ILq+PqqJdkh9of3!B2A7?bs zG;dVnLdKhfX{m#M=EG;Ge-=tuhex22_4v-k=5KxmIGOcOH2F?ql_+85$YP3EQWHdU z$fPA}HN?a=c6C=voqwZfL*?=-DXhaH`SbbB3n9UWKia3@Gje>_Ux-D zw*P%Y|6hB8Dupkcd9xTE8KB{`ri}VwJeIsyYI{ySAT!5G?3Yl_asnaKYD&RS;9}m& zK`qi)g~!skKdOlRl(xgSAB8~*I)?=h6_nSJ5i z@alAmuDsvvA^?EcL&v<+9Ww_Zd>wz$LxDqyKp_w-6LTl(ix*4M14yW)iW>w5acn|= z_DuX_8*1U`nOHM;b|8gCdN%;G=ZyTRdt@4#5l2tlJ}^PgHbO7Rj#c!sxqqmfN{Sg9 z*Ne%vw=O->9HxQgW-l|Sqa7C~pgG9>6*NqsX5tMTW2|ka`4xi0h)z!^bikW+N1d5a z6ZB-v-78{S1pbT?2%z|$71ommIT{~NO`*xaVTN6*iiA7E?O?ct986}d9qD%sQu z3u`SdUVu(kNyoC}P`;&kp#QPWZKRh<=~=RuJEjxh)Sw|WrKMe}mK1Yk+)>`U5%y9s z7kTCxM!}8H15;X*o&pw%;+YNggDpvEzzlVCvNxy_Z9ZSbDekSHBhn9)sdUg*{eYUu z)&U%ZsDk5i4IrfS(2CnPE8Q?2@AUVysBj}c;(IT`<*MWC!~On6TFLit580+j+_s`y z^2ez(N!y!NMTg(W#R{vJ8LbH%UpbeSbCNCs{N#Vkl@MqOg@e{X#%NS47{fMGtZ0Su z9X$(KW6i(^DAPcHRYcEKL^D-H*CiSas^yLB2b5BfJOCJ+K-F`5B)pn5W#}0C#n?}9;`U7GR_4%Y|P#+lpNsJ!PH#HJ{c6~1p;nejl{Z(p`!^RiKny1zaG8{|1^zUO>CNbYcU2>s2PfhDCZzRWE z43p|yay4r;C$?8}6l&aDzvF_t<8bHqEa@F?c5J0{{bS;GW;5aLW)lri>o$;csZ#@V zqHP!|d^O^&;=piNB*AwK>x^9rBGGpA(3{49mNRHQ!nxMt%ghlM3y+rIq@80J<%`M| z8VB54Z*OzL^9Vgnaj(NU^E9BXV%2z=GDUUTL~Ih_h*d{_2N_oR(}+&Y7%Qw$kUFX? zOIF0o_SkskuzfiV5$z$-5mKf#J46Bo1c>xYRm=`cE@+s3$)F2VVzvX zF{?qV7a8DFMT!gu#8cJ#_sMaCyLPMn=>+5>2DRcP$mQ}R0NgK#E{ap?-!f){U}cbr zu9}3tfOe}&J8;6AT}qITh!C}E#=kH`Y|Zf~&XM71TZCIt5nY6};bIOBcsFRr+W0*) zGk&0u*3Y`!`h9iUOg&k@1t8&9uV37!fnx4NKWqt-aMf#fgd9eIM%-LrL!AYbl>ou} zs-Hi3Z`cx0@*a2djbGiGC0=pkj=Nle(L~$M--wV;3S57@gO)_wg0a0jpC5I9bm8$^ zelvb_Fy7LGN46w_E0#4goYQL(&_?oZhrZ|8SQhh6^j#GL0+*&4oV}x5V^OT?gZl?q zR)t$ZtP1Y2bl+^nYQV-Hv)AMb|G=0AASt0&xCK3(p+;>YHbcM0W^VL`TwE{iX^Q)i6d3OP_AMYaC#R(RTG#S+Swjrd9$$ zRU!3W0Z@{==#Zd?k}lxMblUQI@_p)TGsOqIJH0^rnQGIBiEr0;i-8cpWFkF_ZkKmA zfowr}(_II4$AEM}Ya=$3#{!TsF`9|5GrCRuq5QYNKM}>~ zOVyj85k`v#d9m&gV;KmZ=v!h&bV)-}xvA$4AXB-!_mR@p*@uOb6aax)jOckf0+>Ys z38739^d5;LM$*~x*F@O{x3~!ysO{lqXzkIzpf|e>!0mQe!0mqJZ{rE0wMQJGwL|Lk zyivb@8$$K|A=N|CkLn$u0G?wg-($vJx$Oi?@1fa)fvI>+0z~r;{tewZ2nEhpb4w;Sp%Cyzf?zSua2kS)4H=(Ay#jLd?;5-WNAITgu+=K zTw!spJ2r15)UV4tBCF}y-!#O$n{IKU86bQop7^8eOUcWH1*6r9WJPp2s|!^_p;n*l&r5zDMwTuq6ZkEZxn4mxG!L^(-2CCkIkP5WL#&EB5C~98Q(0 zUPaELd#r0kS{8tijNMaa={e_d+9}v~W24@|tj16CqSVwg<-HNRTJolf>=OMfGhjPI zF%3bzniOUnQiWa=amk;e0r~U%+6zzVCRcI$dK}R7piw{kxz>c4Skp@*JrZi_#fGHd zx-pcbgVd4KLNS)~#=TX3f)%or@V^iv2qrO>q|6BmDN}%Ds)z{i@zeaWFBBuUPo|3% zycW*l5Akq^=q#)icQ~@?lcwdfcnbEE=*jWfN$;&Yq=)B)i}c&UTiuajCnWXj%_PBPO}Z!%0eAQs7_p{Cag8du#U;lwEe z7ZKELVF2K9MS4!e=^<_=fE&@uk`KL_5mZ8BxaJWqW3Wa#_wJ%pLO_Xn-&r~exoCXY zlW6GfCyXPc+kU_Ka_C?Jx_?l~SMX=s4~7u87gZh8BE-vyon)FF8k&p^oQ3IRd#6+` z8_5VoSS;BT%is_$={fonlq9}(<%WRhgzED~CQ$$$OP?Kilzv=vl6*$;y%K}OR3nOL za|iiw+%D!psx6$EjXdw7<6CQs&U*00+2jH1T+UcHQAw);CGg@YI^6cIf{MTCh|S{S z!c0-n1%BFr`>sx*JM%A1UT5u_w&@-@=WOC95vWYgw&8Eu9xds8RU>0wEGM}~ow?8N zY`+0vo9-m{=1Jwwcn%43eLMN-RBN-h&V%N!IBN3bh;8hbo1r} z6aPjXfWX)W8_rjBWuy+&X z-%9&%6%A$zleX5TdN@kz9#CtGf3~yvWu{d)qc14Rv%B+i-FR{?#SN0Sb{sP^{C+4( zyQtZzR7HQhd}Ez1iQ~4(cVYrD7a#ysK$_!`VyWH#l(a+c?kPkiN6`+;2twC@2vxqz zZiY`jYS#8td{lh%$0CWOkT4^&0p2 zVx2SISdt=NLY!f^|Go44GdKR{Bl7hnB~}3r`XB6*8Nl=(Z>O&#^DhuIB?dx@yl__x zNrYK417t@^B;scPcVV|Y8J&Fl!azSB z4vQ&|l0h=)?re5`=rzm!(rO+Iul86h2cXleF@{&#C>g!lhB|UwR|HJJDJj2~Co<%4 z=2W-%wDz>BLZM)?rbDr^o80i|mNI2=P7{QkRI6fojI>;~c;)x=5a~#Da4V@s06){2 z7|9}zusD;~k-tz=Dgp#6EKix8>xEqH?M)|*VMHsH(O7m4*uLwQ36q^Lf(d$ez@j(o zOgoOd2(1>={k~7xGpCsf_AF<`7hWnP0bVwm60EjRWRYvy0?QWNUkTCJaS0tVxnK8A zgpe|b$j1wJvznhP!|3d>UfLSXDzbXEchEvct<_%fM8Iiw*oCFL{hU7@H6Xa~RG!}@ z*f;hU>J-Uz-?&ZPy=|nUw3sal0G1Sgp!8dy<(JH*cp07S9A%Uc5zOAu(`dE*-=rdq z=re1`XxDhFkT)7gJHbnb?fcI!I{Q~3QoJ`Y2&Z0XKgul2a1!z?b}#!{=st!4F(lNl z6{vwmcLI)OdX+!NBg9V@f7pN9r9mG1#&Ph|>D0(Ir~Y1k2cM~ce{oFY2k`WhKs&U0 z^5%c?EHE4gDZ*T5*@HFVjC<5_h_spicr}Hz3CakI$ha(z7ZS2q z?vF>r;Yxi&`yoqegWck*@uUlf!mf>8+5VZANhu6$hQDUzi2w3u`p=8-&s8F&vkXoh z5Z6fAO1W*c!0RVtWt3njEgcM|L@t50H?(q5kJY+i)7S>_u3|}#DU)bN`a!<8fer#f z&@X*Bxyj^o#p7gpRkfwp1zN1H;a|Dk8PFaO5Re}b|5fi&6vVMo0zJ~u62^1-!v#nP zGRP?^+CA;iaB3>+g8|`?sY*A9OPI)jR-ETvqtHW`HCVda3A~SF99B0^(Xn-;A!JR>Y{=P_P9f9XGdw+9YwPeqvK}XmVPvy$A4))gdf&wl zh>OhQ0|yUaT9I?VDLsAg6C?FxEKgrj-YcInrXyJ|DY^fs=-ErzVd-S7ZJWPY7zhB#|(H1w-$yb{Vv$$o9Kn=K=#5#v)KE?4z6buF6n)*xR4E58rmw(P1#7Uz+(Muh}-D20L6f`JPf z;GHKJm^CYicuW1jM1H1vq+KCTmAuTlIh^?Sj1Z8zq(l8xRM7t)MBWAk4l)Jz86FS7 z)%L6Jy5{)Re>5eCI%2j56eBV${oKMIR6oBddvmkFAvJsa5%HPg1B6yvoIV#aKn)Q3 z0YPUuQ9Wq9Gp$lm`lRdP^6<2!NB50;s6Au|zldAfEA@u{Y-a?K67!~q{=Bc}n11vJ z$oinVUkE0`^$$1D@w`#_Qo2qP_2MRg>pr&+U--9ter zn4M`&{c1mzRg3LGaCC#u$aW`#%>e!4Y|ZNMqhOM41bx2r19nW$7NZ8Giqer^91Yd} zkW1V>RFCW|x+)C@^Xbn5^%mGN(*?C+dxaB zI4Ifo-v?*S+@qM*IVli7O?+cJ)WrFoVg6Y=Ktx=@y1uZs^#4M`Q?@hVzNlu7c1}*J z|KlU3;%s64AF~}b3pLy&q>c&djk&o#Mtc4_;Wf;@YN$kg4`E_0@o)uoWB-C&as_O2 zd8s-JbdB0NkFi5y*GF*$YYJCE^a0UY2;u~G_DRB{eIechVgSiVNNj2LrKz9)9!F;B z($hj&YgOsy*&)F0)9cFTTgP)m5S_$xZBUi?b7{~K$zSE(DdKk8o#db`iRa&;97tX0 zH#kUk5$*EmaS0qJhM`+(j;)uwAobB`goc3MP zh@3qvaJy^3F>8e0^{O6G(RNcPRde6+m4iZ$S97_3Sk}jPV8D~duG}!Cx~O0eWb@DN zg;iGa+@c1X<9t_tDVe6GtTRTHv%KS$%F4$^ArS0v2d~xy5}DFM zvK$M59m-E<1<(iAhjX-_4{`&ML2*3f-D`Gm>!2eRS>b%KvZjoG9-M*ZIr-7&j8BBQ zhQs2=zgnIPvhj}ufBZ-u73p4m5@*Z{LAb8%y$h@7t(c;pZMJD+L(rL!Hf4)D`~cEE2hC{pk+$h^x1F0S38vvm(vRECaC%IL#MZgn#J;+2KlP|zh>|eFB^jhUfh!`->DYqVHDKi@>iQ}*a zIK)PcBasD|8%GZ+X#{kL9;UXmTFwm+<~kaERt28R`=@57PUvEa4=QP<54WHw8$l0` zHX;pciR5(-IYQm-D|)_*@CZOr@M7(|>IBAi_1>v7R&q7&wJ1yfshvmVbM%KsTdg~3 z+%{d9HF`CPMrO-CFb-nVz`A*88Z{3GIBtDe4B&HGYdzLv1(GQN=OD7BZ5gFou8k8C z-6%C!Pv%)NCLM@0R#G#iQa41kav61J3^{z9fwxkfI87XcAdhxTIE%%qOfAChPB}oY zX(773Z$9zQ#}Cp~s@UK!G-ju4)IMvoZqz<=vuxDw>4hg1)}FUJ;MxIxDaR#%bmq@~ z5@zb%1h3!gew3CNojYd4YnFIjlzqB5y`6~U6Djh*3}-#Al}1>(zi1y! zcvR+Tf$KGXjX37JVb`G?Gc5t#xQVjq&G0dA(P!`#)Ks@R(P|Qx!DdOtZNki<*eY`O*MVR4@)6Fkkp``h%*(6`-KFWTfI3s}qQD#ja2)AV0^!6& zS1H8aQ4~f+-{821G{VY%8l9TVha)&NM{=6aLTDr07-RXwC5iE{?z{fHLM|msMoh03 zD#7QPH-g`H$rbVTM#@rffRDffM~@%brBPwjSZ8)YicquGj|!dwE~*c+?mPVKLfN65>_>!Y|-eL@buN%y>3D^12p3n1-vAO_!>CIb4k-shGPMh2vP8 z(+osqFfLVdI7Q(wEg73sO4w)$KMBvb`E`sH=S$BPr-;Q<`ud%ou~&wV73=aZG1`!y zYW3`u+I{EoVJ!3U@x9HLR4BC1MZ4o)Dhc=Sr4)1w4fm#(6gy3}3?X+DR4zU+4Lw@8 z%|}e45-&-@%@1Dybav)y8l-2bP=$||s?is#)@i_f{ZpR23OQyeRPgJ3=WAzcSm&zz zdqy#ITx;;HypDJW+zbz7Q$$)^#lZL)7F@Ca3*6CPhB~?rcn>s>BSB}0;vznz@0Ss# zFdMr=7mDrr>)t!5Fv6CymN@=0;>x!V1?3f}V7A;{9%dDQm8xBn)CX7Tv$vzo=-wWy= zT64@wsc(|<`iIHK9Vv{`SI0N%&iO+YFRe2(!D({oD)VYR`--pA8}#~{3uv^z3aAyt zBHa2{;6pb{wZV{3EE5=Re^!RZJ{5m6diEr@QaO&hI&udHkrCyf3cTG zyGg!}2AHmNH|rO&4!28>>YrTW$R!vi3}Ks4MwSMPpecdNf|(p*V1jCr{+4IYlV&N1 zmbuE?7ruvg>$VQpa@ySXL70cr(4_>+6xiJ;TX4$k2r|WSwlJ#PrPOf~69{jkcRaRczzPqiI9jXF1mnhIlm{UR`DD;ttUc*nJmvM^9bDO#hvWIpjeG@K39xB~Vj zA_gBX=XkEtCiENksG)?V4*NwmFMtG?rMz+jQgM$`bj`jQjOsmp_`uN|n0ykk_Az=F zPOS{rZlRYO=iOi=P$uf=cxTyoW63e`W-Z@R6afD5U6X!lhd5*3p$JI^RXCFgS)iwl z;o9q#oG;p|SN&OWU+!m)K_t9cx#ED%xPo&=euy(J_-%w0nf{s>y;t)s8cNkt41-6I zBZ4YIimd#zFy<=!3bMyGhX!XJQ?kdcM`N>R(k%>C zquTWpSs=eq!3-I_Va`{t07e>^jo3baEh$| z30Cq)OtI$@4tQDPo4 zP%Gtq_bm|w{Q4XHEv2{`y9{fwY#OXf-T-cqfgO_*s}=LHct4a9*aCZpmCU3)Jl$Ol z+#v$Wz^-A@8XcP)n_(Jb8nTbu3kRFJD+HrImqDLxH&K>{0LrGJ0cv@09|eM3f2D%q zt`X2OujSA$hs3Q-v4We_?f{a<~$Kp$9073fVx z2(ms=-~_A|mMaskesg+H-?mB~j4o;S0;~*d1*LW1C*n3ZrF94&;x-ATHZ-4_0c0y? zrJ@1ZRVmJzq&_b#rGH^_A(*8DUf%$08|FI6bRd^xF2)&FE0!nbm+cOP{|0v}ju`X( zo)9*`7tT(7+(CiOF?+5BZgp>2Jj=jZJbV4c?|Z`X?YSbLoiB95Sf8ed*-AYgoEN z*WdaAVPhD*<9C$O$p@b-1$)@6dIP=#?W}sk`Idsh_tMFG-dOTx;`0;DDEh2e7$EIt zg*i4$!=SyBNUhzoNGTc0!z{g3QJii>Wybr+SQrSlCa>XTmOTJ4?)rqCawrk@(kOt4 zn_|-B?K2jK@0~IzEYGwu)B{6Uo4H|{*>>RB^Zk;nZLxXL255nSVrMrg>FN`8^}ALq zGlN^JH&^M)esbo$5}n9+?scTByIRZ+NOndjYr5m0Dcf8uGhuqO{i?iL7@c}3YuC-B z+Rvmi=Rpg4sh89&$*=Q?t&Q#if=4#T z&v-JhJBg&g8*rJ(>tc+qTquIg5-7e-$nS{N5CUZnVBX)m{C^y{l(izX#k6F!`xn{h z=4|@QV`03kf9w>==5V1BmGCI7;@Kmd3SgH>Vy&^*t3L%X_51APF7d4!P-6jZn|&^e zbc_t_y5;ff1l=j}=l{80?_q5{qNh)`NcY7-EI2@l?lijV~ z|Hp12q}|tUwxkH{F&eJ5?<*s>n{OX48f*`SdN{vs2oHC04}M?3Zq8Ltzx(10-aR_D zIoEc9H|!0G@>CU196{ejF!|J-g6(jvA_iP=)BdM>{C<>62~zRIFLdq}3KWFJVv%gq)X>e@@Mvzv?2-76%wa3QVvlLYL=>D? z%=W1(Cii^=A(#;Y0JheVVEqkQC)xdx1 zT}hiIDm!JQ5^1k~yn%|h)DlDASkZbKAN72^+ql-7OraQ=MI2}03iOvZ&soEm;hL?D z_h}+sgI8ovQt(=wSR`)XvdIo!k=}5A8FxC*pe?B?$50?)GF=6r-$>O~r4??jo_k8w zofMi_xfRKpPQsn}@*F=*KfW{WG=hWvUP&pJ%&F9PHPX2n+ zHB#uE5Jzh#7_WCcjW{xJ9n2?;P(2Tq+uXTxtrh;7Tgm{Onife|5{OfsLh60d*||is zXTD4%}bI z9%g_4Y7_W~&S@U9$?R|uq#L=q;25u42b+^&8y|EAxUTWk)lOVJQCIo7E?ne@z@=Th zLH}MW+?xG5BckI_`O6t+p%XOedJ;5M1Nd#Pb`-@_rki zm7iRS@1{NTO2>MIT+H$_tdGR(}bXNn8m9dyeO?scSXx2U~H-h{brhp-=>AqF$qCVC%g@l~*DKd2m}m z-w#W@pUC&uZxrz+_tHWA2M-dtr< z9}X(`3KMfta{HL58I|Pf#zGN0yjA+@qDBvZ&6(#@_v^zC65_30_!DR0)pXG=H@GfDKd=E# zA!4nvL4nG@D7jCbd{?8$(%$PcdW3fh5B6~52xSuS&$jO`XjE;EJf)|;S7l(Y^-+Z%`mgW*$(^rMhH#RRdHdf-`=9`GqQiu$N z1ESun386gOooziuajosJEcNRAsR3&n6+#y_^;p;&Xy>=V!+fPTJeaUxQ50{HyBheP zoEI1_>^B4`$3vl)_{@yy=7)u$TJ3q034KEy_gXi=R?(r}{fL&$Xj&dV|n*#STK_HZ#REp-2!?IY2&v;q=*G;PS# zOe-D}1?FxmQSdjozXE;(ib5mL3((`&=M+sFUdbQZ(I#PXDC(bweY6D%$DM*AqFm-L z9)^S{x`req0>TxZ;^xN)P43C-{jF%-o<%=^z1AMA(gV!vR1Hm zEXK;Tubz?VzHW3bn6keT83By^f7(<9d5X9DB-jN~#HnL5kI9?;#Egs#wpv@|2EDMc zT3o0kFt^2VM3|`=-Q`?c*IlWUDv_)>oY%cbz@3e6qZpvN=#{->CFZ%THtR1imr(4J z8Am)au#kL``OB(BGc)$QLDpirdKnh^;#?Yk=h(h4pq}}(0ox1Cqz&l0wX*-pg`0E_ z%s*slYdZWfX6B$Kj?2DR#P6N>OH?#DI6i`({Ic<+7h`s4yXFU|;DDY}El{++{4ZK@AP{{smTV*p3RSS&0E+no^7n>^Kv>LSQPVYS$|$xQ{Ws++h)StwNv~KNVv!r zJIVMc^1sYI$zRoM%4uB9^eQHKnr>YeI&(s3D8eQ4Fel$VYTgH6*-Ub+#=9jpIvZTh z6j)(iVacv)-tA zNEpwcQI92A4Yw+d1j;#)jF-j^_N%k3#2{JUv-o(wK|c?IG5Ter}Tn>9Y^`@K|mHbf6m3T+lxj_S7OfF z4aV(UIA2N*r>~eV{2t7`1}NI!AsxSGm)C~W&-+l`2wb_I9;SQn8?B%L#bvQlsT&5R z1_)^W=|2Msc75dS96E23CGX889U#yW1iAAHTlDa1B0GRe@^+GqeiE5Y{d8tQB&_es zbGqvZ_WorxybxsA`PZ!!%UQOq1e9OCOpz9?@r%0)qqq-y)m2>vAVdo8%o~S(^)KZX zb=jHf5UH4Zx*{k3hwS8yu9ECC$5|;EZa0i8ivzVtJGRU?Qm{-^%r@xK-zMjEcU%j9g=YaPzx^DcCAa~=)zh;L*xg}r60kwMv%RDtu zy`ZVqFuYc$yScfhD9#Kbp%oJ{LX{obezwR4Tv^DQ?{p;CBgOP@myiej>2?LO0$^=J zyz&6ZLEMv))yS`TXdu;SgWaHFCu>l%oD!Z(afcGk2+@0b5@pcPx}+a4wSR^NLR8+W_Xq^J9d=BxoP#M+eaCbM;&J^N8uLzvziJnm2O+X#D3 zgTnbh?&?@nQSv8f8|aq>4R)vtT1b<>keMP8ay__Wpk{`sn;zJz?B~Iv>e%NZ4DNHp z*7jI^4D4TNS_jn0GYfAx=K~7vwjjI)zt&19id`w;+Zds!LZ{f4q4VvH-Z|XA`*#AC zhZ)#JcSuW3WGO8l4ci7RoK)GzbS60-9le4#O#jHi*KfGO`$qD|xlX!GGBq0J5CoE4 z+=b##g3EG^WjT=**4LI_IPC_wUc=ra6Hqxblf^!`+%9T)J)4VcV!D6 zlFQF_YiJu&vUd6a zm#B%cc9I;Y`x`Pm8-MhCxsg6(=tHi3$^9Y8XJdLm+%un_MVk{%m!a&6H1E-yVY0hv zu3684WHlIq3iA81e(aTPKYe z5(~p6pl=Bz>h9_*0xZCbvS7B1XDj{L`|HajCFfote(}wDJlfW-`^gcbVP0xDm7Jy@ z%p1+mzLHB9tVN@UrI4&D@LWW~NEaF*Ylfy}&R!+!5dOyE5dn(r0^Pym+TwZD9E>QO zl3O#=JCtXOBQb>!206@-U0VTz2F+WQBS9YY&{a8pRZ106JxiGf4vmWRcJ(P(#4f7D zqbYN&P+HpkGB|b25J#E8hmzyTjww1!+KKhWJ5oev+J$4Suo+BQI&WLO^HCbs*o*n1a^^KG8l!C8NbPad*?18`Hl zk4p4i2GEU4`pvJt6wplyS;scqOi-?m^7gH`Ys5p@WXZk}=x?m^J$H;lsscS8x>?YC zS&xj9h;7Oy-{1K+Q*m>pCOq1Q)|FzNpB3xcI{nq0^pu>pGPDP@jzsi>`R6dnd`Ku>7LZMtZg#h*Fhi|TG{N61 z9d%}5kAO@viE#raE}3S}nNgWsPR{k5(KAk1H!y-`NPNiw4wn@7XI{!JG>kXl=jf+s z^YxQi5sRLcG^ z)bf&_l+~x)EkrKm^bAGajQKl=vT3@@Mw>or(!H#e(Q<&H?YsLMq*8l5`v>8jGGS@T zWIYa{UW7kL2uA{FlOZtnwDa5EEp#RCfDB=;XDPGYhqDN-{?55xURrt?E1@+gZ31xfT3 z4@@=!sbMF-XH!h;9B(SZ`}{6E5=vDb%&}kOJ1IFgf?8TsGj5AZ70=XtI84xAMHk&h zb0g}+7`sr6*_ltGCC`ul(FBati^Cg$Tai_!8Zhp5@KG zQ#rg{+0N}&G2p|(2@;)w~bs#(XoEC7W{k3HAfa!X4^NA$%UPJFhxuZJh zILZfw;gisqFh$d)rrBZyoE38-#Y%pY9GhBUPjn^j4HADC@r3}GGvbU?j=}`a-Al8- z*dm7s(^H2s%g{;~S=A;QoI&Bd`hQ6}@(irUMJPiZTpUJz0gOnlGTdW1P40sXE_zZC z3RQ>{w#?RSrgA6hm5fH@dc*3795BMa%+^>l1aaP3 z<|cmO%A1memy4Ux$c4t!l$Ym+s}Rv0H%MuqOHf_B^SM9nKp(u)x$}j}D>|o0^b|Q) zCG?e>g?K_2(D{0OVjn#IAGY2yD9*L%8cl-3;O_3hHCTeXySuwP6WrY$g1ZKX;O;Ji zI|SFDXR@E?{m%K`eQN5ishWRtcV8`6_gc&T^IMkkujHxTG4H>#2VlOEbu4)Hpz!bA z7P&`&9?(`{ObS8}rE@>xo z3 zkJOV17mWR8``UP3&ZXJPPxQc_hnc zcXc*PVdQDCNxxVUV_;^Pe!hQ#Ddrfbxz#!1AM8lTNH?s{pGM`L&(od9jLHw(Hx)gy zjdi8^6#v6Efc;~HgA4p}`~JS*{|mX9`VV3o2?7b@e|UYO6?K^)z>E^js@gwG%P(S6 z*k{Q&@46x4N#Z4Z_*5v!h|HRW#hM!GvoT-j^duzu8;B{-nm@~J~M06d0k!v&jzIM6m2@st!1)ImP2_1>hw z>Y!Z&-}8By(%V0Dq?*+PsAglEZ%817F{y3wG@4W+|e@pNrHiWuo6z|GAqgL86CqzK?r?cA9rqC8I%sZ^CukrIM=`aD}isbY8~=V72{M79>|xR-{=sx9_Ova94kKOW>Q zcJhdNOu(OK?Jv*RbNPLokRG-#st%#Hclh-6Hf@hLrucwp#`itcg@Q3MANe1Qj!ADsg_gz0@kE*%Z9=@G9FX`JH7A(me!&cHKRUKLh zX_nF>Tcnu1JFi3NFfr&s8_CezNQH41Sxk zTC)^MPu9;DUp7y{doo$hQ-93QJBhoYKk9Y#L{52Mxub5PUD={;q5`jUQ8&@Aj8VU$ zU$ungMm`3I=|?;!hv`Q?Mu%-iJZ6V&Mm`3HeT{ew4Fg6zriM|A@=@+=lD(1d{36T2 zx{3;`L?%lRknm_lcIDm@=GWWq^8q|ghXo-cpgairZ}%y6%!ZmO_-e%X!7&0@bjBQl zwpGzUEs6m!Ooq&q^+2DrZ7Mi{LW<#KrhIs&WQ9P(kR{2|0Ria9ZJNFc-0IZfRADr{ zWfcf$T)UKEQr6nqe2~}2EX`ZYzIX#?Le@01+|-G4cW4GMa2$3 z!V56&qDBpSByCrCL`KzY%d&P&UgB(g-o^K*h_c_&W$l_OAl@oq^$SBDc|;YU+NJhr zkILOa!Ugv70!?0UiBqnyi(ivvU!${qK3!lk%U{FAy`s)Bqkq9rz_&Q#DMcz1TnD(F zgfvNZztAS+Q$Yu|KOtM8wg-WXwY`iSlqbK@{lK$wP@i3#fQ;8Fk!%bTTopk=vVxnK zCGV-S`axcWKO;95mH0ctsosk(r)j}Y7{TqBR?nMM5=ba{w+4m}HwF?y@@Hhrp_#7$ zChtXfsx|5GuO7<8ysUV!DZJPhBKpI0P%X}!9e#aKH-M2(fi?><8g1BcnVsg0lhdvH zrjLuaG8bv#$*aa=cJXy9a1Wh1G{s=c`kEybboCm z%Pii@82II`D{`8ag2D7?pP>>yDMfQ-)cP~E1a$Rh;oO3eWPcw>o;f)4_fxLqv*2o_ z)-P&4n$GwTY}%zdW}~ILm^8YL((($y4;G;-Dmq`wMY)pg!mnpgXe=V+_)%SjmUJY} zlqAWU7GwZNfyX+^Uxe0E(df>9tQ57r-Z$t@nF0Cp*VGYEc*2?%qO3kqxY<@UeEi7K z(R%LEq2J)uN>c_|oVqHD+ejOmF4m)#r}07lWBqSbE6MuzTLV3Zc%A8 z?>M=Z3)5NfREhL?SJOA6=AwOo9?98^QZY;6ncA5~geM|pD6aBAc6-y=+3IhV6#qQs7+jJ#w9W9fGquI;R{ECUtXPxi8K3uQjrWu9yo_VBomb5;>2 zSZ6DZE({A8>Sq$4`?e?Vw8d@d%MksM*m!*4RX>TXo&BhiPJ z$mTlImtSD!XNf@)ANOFjwU(*Do&U}7iXoQaOh9`elHbIR&O^!g3zh7wc$o4x2!t6~ zUyU(0Z&A$vrE*ec`zLNXWv`dDq(6yPRI987#XQa{M48jRdHYitms$GKfe|gP)o2b- ztDUTx>X@FzkH%d?5g=pT9#2+>HHQZ%m*ha33wWAlPA)G}T)9+bEj;@fvp-q7nqWMnB@kVbj%(<G?p2Q@u{|ZwoOW>z zp-)(zwN2!4H(SikD=%xHJ=}?)v?aC$v7abc#8?Aos3|*91BGbl{`?-W7@}iru~yQ; zs=?)d-jrxDVPFoOjr$wUbFh{turq?1B4kQ%qa+NuzzZW1lO@nRIxk<=u}Y6E2HWHk z6z8s@u^sjh*TO`7J{I{LF;D}fa^wU5lF*Wbo=R{vgfW8pb#qYe3`cN+Vy|`wZ^&89 z(P&hx)CoYJJQ|m!oJ^=H8c(>;W*M!pKodikjGSi%xt27_Rtgtd&en1Q(832kM?YaFpQe)hCEMX$v5wjmC!Dv07acj<7%VB%R zHf`mP)%+pv$=Yul3Z@^Ieg+FyH(1M&ZMD5EpK;f=WoMA@!L?#OeDT1XqQRm)2Lwd$ zQqU0zKy$LVvp+uqDX@aq2CseqICHl%^ksCNCU9PG`QC|cRkr|E^7*FG#!n%}B`7$3 zbUdy^I=*NO{k(#Ga|9R!gAVaaH-t6F-PuL;CJFy1fFZFz9aIy9A&N{7QHC31of67juv#5jo2k(h?I*AJ6NnH3Z}=K)T2Kt1LfQsI z#o6_ws@wQCV1Mpuxe3`mn&JE|Q>zSCG^`~rn7$az_KR4K*ibAAJTiG%+lV`M-Y+Eu z`UnH|4FgM5WN>zocgD51V=BydmR+8R zeE+Ajq^^~{UnS-o3E4e#lZqzk9on~QgGV}(*j@g%>KmjtK4<*DcRpr8$oLJii}ry5 zIi#?CqV;f?mw~_0Cf}(vNfS_c&V05Z(qp;*xWR77X6%kK{q-j+5%fw*aEZS0FGl55 zt4>5JATuMguF`~6A2Xoqf!1l+FmTllCNSQ889j(#Ud48tfL=q+lI!U_5kwV~ZRGp4 zUf>I&J4)WV$^sO+cZBPm-Pbd}21^41%o3OCid2086MFHmH<leS}26R6hq!V?EYoiqG1Mj3y$bJ|WhOtCm(?xCWa8N5y#fRnE zUU75L)#@4~&IP%{B{@*srzeOnn)sXe{5e<)15mTSIkCY(d@2Af9P%wrmW@rK(f75q zI#_qEsZDr&t+IyZk^8tGce(S&zpUAG>D%ZVq^_O5tTO9!;?R88idgTl1B4E9hMZ9e zE097Nk^?`6*-8G-7t*mXJokgUcpY(;$hj)?_5qp-Zj@y07y(1-7u$_&G&jKs2tF8| zZ?!NZw8Q;rHu4Y~x}Gj2L?s9*z81>5k=M@hWsz^W>RayWL@h(-nI(pQ(6PE5ke|;S zURvJ&ftThvtWKDMo2JtLX`0|Zz=A(LYt>tjX<0lFhFFfd5_ z*M~_jqbjxr#3}eIkN)`NPMtQa>JWEIvZd<*v-JTDrawJYNQMSYn~7+u1Zd_r(b4(! z*9VQhq+k|=T`Vgss?Rb9@uKcUU~*enqrLvDL_;QBxTYX_+yxK{Wq z-x>y=hOS3qAj|kWMpIM@9m%%p5ID7p4muW)NZ<(D3Da3};<82`sW8vu#q%TzR<_2+ z^K7+|-&zeFlA6=BD}{GHg3zDByh+O{etEa;C*q@8#<4fP_PiKyC0~Q+0*oM zMAc(cdCM@ydCJ;u3`*fk;#$9F(=9bN-D`Dd>WG;Eg1Fl2uF>6HL!KU9hz4yl>3`JK zLm0Lll$y1Kn$O4QpRpAl@s*8|+IEU)2G?sESw74$^{ZSP{rDL zRSkt7JI*u=;Lj}w>!>?a-Pc!l(q$b=(Ikw9W@90lDEX`K6QdZdgo*@i$$S6TyxsJe zuZ`IUEN3#r`m2^er4nWH8iki`JSEG<)OG&f{NDGIug|}7{XSkI3E&QdvR03_P#KF3 z%HlSgV3>kZ6jK!8j4{mn!nQh?f%YOO-RZCa9&7<9R6&w(Y`CO4N@>o$GPDXT)G}&4 zFdp(fXio4cEHxytQ30wQ6gupc<5UpNWi=x|zGoFUn=^MgUr6W$peYT8Iw697RQ{HrW zy2Anq0qPR$E)h$VcNcXPfx(7PK!Dk7pGJ(~DPgr*@I#p~h|+v<0+~3oL$B*ivy53< zr$VRUgds#?!KIStS3Uxgv8r|ju(jHiUv4{;p=#;G9o^3&)7KSO>@-91adc)?B`or% zcpb#i_%_?jmbU2-BbU1Dki{*$w(ACF7S7I)98%v%&u$NiD@VI1@GjX77)PIC_th}W z%PxO{7et~nj?jTe=ejFOhh=}Fy+`~7#ii}ZFIl^7aq-uAGv8eXEpuVW_;(X-1zHRD zvaOca!ZG}0L8ohDmPDt-6U1~}TBo?PSU@DogX9M^v0tRl$Z zdlaZdN!HoVF`vvj7)to9wuv2f{O+OTo?V@A(}ho9LzK?lMSw3Et@WF?2K&l>m%PtL zNWDX^VS5kZFQV*X7Pe;2n_CDyX7~m9FGz(3Z!es5t(zz-D6=|0>}hYe9~kQ)MOjFrtbTZ)UfZqpl3nQ|@ekQa8rOZYPk1R%GUC_BraRcIG#XJo~Mk!caoce{Z3+JLq%(^F) zm+f%F+36zv`RwpH_QRs%oKTO@47h-*4^~A0^|~UQV%ZfM7?K5J49*#F=Zbx5Wl!Kv z9hC6l4V1(ODz_LJBVpU<3x`}sYy@x^#v@r0QNMdk8&;4 zQNQ7%Kv$opK&BD`3f>8%%J-i%!?h60PJe8pvntyG+k3Hf#Qm?d5;oOgoa{p-3Nv_V zN8_r?B-)+kB3Fz#$U)d^WG_`IpfZ~U+~0b4R_Yms8bSF$iTPr7)vD6^0cOVtYJf9v zMr!X_RWw9mT|JI4E5}grm zTjn2JE-^YL6pI`h?hsHlgB>qEG~#N#{Ri4Q57VW{2u>cM|1PN-)sUqAU$2rHs-~YV zH@JfE$CO5}WA3t)!8lXS7r{W=FiR~1;F=Ae+XTMRarmJSM@{_$i8rcxVs>Xe%PmlT_5`ebj=cL zNsQ34V~h&q@||~#HmvOGn2YrWBzrw8+=`UiEGuaVjHf4~+K5~o#v2~XR0n$!y8RHw zZFinw?q}BB#2?JnswiR>Ng|>x02znuw5_KC^rX{Yy)8dC8l>6KlXEo`NKi3}7KZ2G zxGGrZ7RDoxTkAdw6{|h$r8vM?ek0gC;`3hcBT@h^*k)4JjXUEE4qVmw39EydY}%4m zXAl>0UbaW)vwwkDRi3+gZPRJ;7uNiXT0f?l43?4O3WO`AhAsbOMdvj|R6XF&XInXD z<$JW{1e&A38q0FA3C6&DAp%@N#M@DF+Uwcb9(=X$$3qm3A2f(k<9IjA$mmrG?(z$X zTY1w3U`l)(=gipb;p7t7>OCJ7zg$p$%vBT!eu&Ws!N-tpgs8xG}_WB!41z`eKVD z`s(Z@9fmRiOl^P$YLJJ7ZqYFrtcbfY8ywOkH}@|5BYIb!c3d6_Rv4y4nEj_}M{GD# z7U#p9FV5%IelJfiP~C`)+>DOP1gT$|xT_tw3C_v5%c*cuWHA)j6h#I^aHX*p9cO!? zDm7ce8n8GOn}O5)v>})!D}>>`+oD2j~>zWD0>JtsMCjr1E z;*oFe8AGHY3-728IEz`Gns2Y%F@*}Zc|N=<`CWYmd||m#&=vGbn0)O5h$0j?K8{P4}%h{1}Ig zL2?|lCy@LJnX*1-?Fqi1+4=0m`!j*jRm#;KZ{-l;E$mIs6#K{t(Gz#mTZ zUffkS&n@c7IX|R9dma-njO7ozxZP~c9;v8~g!JN$T*sdNmB&p220gE^r-Yt1atZMU zSO>Snf14;mRH)fqusdx0f87Di!105t|FYas^h z$||OO4VeHXbj3i8f$~IC{~0jX_3y#JSL(BxZ!h0Rzwp0;a_5oxJKfP$dUFl zY_v}Ku5C#qso6vrLCs_izSryNtMpE~A4jbqrei9aef( z$Fq#GWfW{n;px2={5%;P|Fo$q0duP1Q@8@#UV?x{HQM_PTn=*6~dURNyvs zpF=cTN@xArR^=MdfSF{vA($&BXQ>ep9*F-2G}3597(3}UT`|Q_UL|gvON}+-(*8L@wZ9W&6+?aOH4O?YGrAhoU|%ph zSJ@p^&{<-;{p;`7wI{qLI1p?m6VzavquYO1N1?7eICWLx$&AXV9Yj8i7P0uda9H3C z?FYaERLJ-FYD&d2X%gdpbqk0Bk_Lf(L&}EB3Nu=k{HzgBETPsyI|q%d?NBVd@iiHF zD{1ubS&v)3#aIc*)a5NRTD3Z?RLq}Xcvqm5T#H^mp`+_}AfF&Qor_Z(U8!cnK1H|J z&ms61Y}(aXQDmiHXJH)ZnRL%obDSuts0+;UKIifjQToC?BT z0)I44!ppv~!d=iM7c~^570xUoS|PrI+5TpZe&C`y0JB_O+b3e#VylY zV8hiSm;(tP)LFO0%Pwho|FhK&%fj>+b|n_prnbS*f0Kl(4rq)H!)f%huq2YmjZNau zcQXrO*(p-IoW91~Nadq8IW?g1cR);$Rlu(QRf-^@5#f7 z?S7E~=UMB2i=+R~G#E4Wq8kMon5}Fr|5sgLj%BTCO1%tKK@yPrsyN82je;VB{%d@9^2-CLmFw`Z=HlyzB$kr=%KRVs z<=b5$l8DN9L>8r%X$zOk(G<|&bJeD0cOw^zIbge z3@FN4cW4=^8YsSy;JT!SU5cm-IkBs5zrOo%Dv65fQ%ab{9NdtW;-uV`b4LL$ges}d zZdI;8@1Eh8Gy%sa-){r74lndvM{jx-Q+U#v^xG_zt{{S()wOnoN3fJFH(*Y2oDL>lHx=KOE}OD8mWd5t=$bsIae_LD3MAe_4Ydhq_h1} zxmB(z2y5ziI!b?CHkn=uFx!K*qBQO7?wejKwzR#9Ys2e*yq*EdPtfXRXOyYXA5i30 zt8tnU(6PUe+sVbv>hC~u(@sqLmZOFE0|CkTHFxz?KUxy2?2(%S6z5XUQ)&-R5pf;G zbEc&^f->Uh#DMN(y6>$MB~iA;gUlPyjy_TC4_}j5S13iP3gF^yo|5c`f8v^wt)h40 z?lXE37H7()A+6e11n+!oOx~Y5(qi%zd^qCLk=yzZ?pvszB=#`f92xmMy?axnd+R4U zCBDP5XQ8TXjO968-9S$1zU=j8Qy!?hbNT1e6)A?ANqKXCBao$aM0KP zH(8B;0{|m+RR$RgsAT&WcN zf%a^E(?I2g8cpAf;tJ-mAnGtBq`$Sr&m+fKKG*r&dG+S>AvX5|PfN{KO28U?2K*{3 z59|48i~2Vr%8#i~<8W!}r1_;vN!^$q)m4Y5IaKOMw%S5~rj=HuaLkbG3QNW?--gTXmaaCMifU~x{1Z2;#-LARSaG@pl#S+zO zxw5)XL#SUuCL#Cl8}ApMfKa_Z>sc;!HlvA{_qb+BnsI-x0A^FxV+nWcaG*(xnqObP z=i6K@6L%~F0WZEw;ZxwvVQA1UpdsaSEp|7si%?+SH#S41SR?D%TWgGHHAm3%&rD*K zx!RI}f_nfr!7{S$jNDFW=m!2y=X_?imovJ(69z@}@xW!2bB)K*90$5Ca}pJ3QsLzg zIZrl|eCB#z0SiGWp7EZ_kbGaO*At^Q2#Oh$ryLH{;Jc>1CLhnD2qjBSQ5Fo=kf69h zNRr232FbLA?eQkyp%(#b)a+cWx4r635C@)-*3Fem@+4gOIwoJvI`1jflq_kBYN*_6#J9GvI;c_R6n zRHEF#qB?>^NL~?{LC+L@sh)B@$b|f)W;QYa=IVDWTC=QOivJ%UGw6-nzhW~V?`3Q< zk2058>Z}EY$Aln?5%E(HVelb+?QtfL$?wqV%-&?#&FlCmEj;?P1B5T$Z5Zsb9DBeL z1~;soKy%3KI0QFcMzR!n%cN2{-o4|+V$0UHsQ z)SnO8x0x2&bU8{)QEnDV!69~CSdR5(r)E>pBUscqj)tGy}Lwr4g z7+k|^Z1?#-`Ok~QKL~Xu2A5nZ;OBq!?gFjYC+8U~G z62i*mV|4yZRevx<1rLSXjIK7`=M({D_^SqXssxE0m`BCIX9u@!ZO|3SS^nJH)7J~S z*1zbp6ucScIIV_Cd`LBEFsh*#hr~2{xg`%^F9>&zC7PBHVVF%~e{^M9k2Pu3ZS44q zS~sxD3*{*@d6Twpl(^d)i$`&eM{bA2$_~EXW&$CQMxqI4Z&f^QV;eD7usPdqPgj*U z2b~At$qUZXc58RL=tGEb3ibB9c1EpP)Qd>=o0r3}2?c2n0U^XS36Q4m4iEDi+NRSn zo?Q1nq(Nc86C)ZQ1YQ^IuGF~lIPawkU0|#4I_wTopkRCX#$kq>>NlK&_r&k6ZABvH z=4q(rg%SY~WKJr$CF3MaIjrB>knjA`O4G!*A!@}SKPfR2=Vu|uwJ}FlBLA_I;E`1G zAy_72t(1$u677`HlPNbN*1Y+c{#=8q$-aEAfN5c|gZMWg4|ew?n}KaY2zGL$|7{p9 z@PO)E0|^^wQsZIy;Z@e z^w=z1tEXIHI2N-3ChKAFg_#4B9*w+EZm&1GJc-$ObM@^nKD2Q=)Sl)Y8%*_dA%v7i zk`W*i=^!K8Fwbk9ZOib%Rs)DjCGK$|s(|L}|!+YS~n@^hz8_5_wTYG`&s0Yel%&!kXiUGYKj(MP~@MjX!>N?7ylF zOh0}Z7H3r?Q>u7bR0Q`hO>`+QbQ;SdL2hv;({u&AfEPzp4I&(4PpYv8wgky?^H)iu z9qoWui=O~lKKZI!UO#NYq+zm3j6(CBOklSF!zf}B|TYGqljzH0$`Tb58BM9fkp8~6!EctIr#{RMx~?uTGH27 z(#>HmSV&C75bB>eNEerABu+$AegEnKaBZ*A%ACkcH z2v~!5YU#-vK-X5{k79Glm{kWEAIVePtljgjIoZ~kR$(@KN>1c(U?&}hF|fwQx2o0a zuF**@QL2Ocz%msdQ*2$k{cJ510x1eEoO)pRO*g}j%(`d-gs$C`R&kQu@Eb@ir_n|i zo4qTa{4DrpwIVNRw}$O5ZF;&2nxcBA-WrD~q_FX(Z~1f@30(k=&!@2B6D*BPqpapf zyr?{mNl~O3YgJ0B9R#TXL;F;Bty~wdT_@@hAaC5H5ik{H-Opv-Yb*?=1wGJO zy5{;zq9BvRCjFHu^@nptn)b#9BlTEx*VP3&Y|{`Vt(FViUeG=!qe`?4=!RnbZJ3y` zqU*F;T70@<_$RL&WkwN$!rQNsOZ;-r z)SH2f>)#1l_XCQ{6pBjql9qX<rf{_>~WzjZ<)zDDpTwa#M`GgAc(qmxJ?1`Tx@TVlnoo3)F7H-9IQA;FOi;zE0;*&$%KGQTvX zx|!f&+x-3eA(LtILq~rUnDH;iBkh)Xf4eNy{FTNa4>mwwM1ZXm=l1G@u|SC8HA^(9 z5Giw6V#+id*Ds)GTDJBYb|{%lkX^{d4`1B&qCdY#d&g_~LmH&GF2Y8GD(K<~XBBy! z3{9dC)T1_(C+6WxqQH#43};H5^_N!3<)4$Ph%2*kvOOgRNv%!r{DjlRt7QlFkMkdGK$<&E@$&+viw~xL!WkE?TtYSKynqM|y3lw;aq8KLG`tx7L zpE1*||4bmwGmCol4&zMumd zd|PV~X%#JLJSXO}6_UVrWXj@1CkFzMyTirQ4U|DBuJ9nU=`h@tmow?-m>6{O6y`K4}~(jSTSY?APR@OE^PQ6z1!wvla>Kcz&=m#7 zL?D3`?G;Ed?|n&srvIw#p%?F*YN~P|8PM@{wi`)Ce5K?oR}mg60fXjO@Kr=Jb%F! zurm|JD4DfJn}-&L;m2~Odt8g(F`P}Z@aqqNG5leScU+|+ma^{CZcpi@_T!;hWXgNu z?#&{uL}n<9GR4yEMGbawC@@F7C;{kgrn9ybL12<;k&!r&8970!T(zRi!KK#_YUNYR z6D+F3a-@o0m7bAxHFr0tZqnAiMoojYq$aQM^x+}&P z_@XfPo70 zwdd#E-OS+n);yg@-9u%;;ig~JA-{>=_grPkc7HPrF*$I*c+HOWo?P*sdVSvf`u@oQ zeRIJ;biRhU7>z@2#Z1u zs;cbxVv^pnffOY@leP&%;4oR8IdBYc3_Y}idq?3za3t^!OmX@E7pyYnjYHmaXQ;*= z(c92q`YOQ?E?hh7v?1LPO+@Lm#oF;p(mryZ@k_WqVZ>b461GB|SpfZ`L?7M`-r|*i z2;zZ^4A64eNGx!`z*sJ@uQnJ4ugSB3T7o%lAl2py^IWnt>Rv|LBI3S(=}wm4>}1i( zX2TuP+LarHQPLt@E>*gOLBA3{wV!;3)r;{b4IY&U~m9n{uh7`~#zNeP0g zCf?VM>aR71gYPRb2+AqSTa)H+cTTZ45w#iLKuZ^e4>zg0Vqn@IQffLIdW)@&q_{Ip z00z;gop1{|x>TRRE^!478F`x78LW#ZCXJY{yddph;+QHls3H@#~+_l^0no<&Gt|$GQr-fZ!5ov`p!om0z^O1WTBj`Do!>~9k`_GhgZ!cC=21T)=tz6 z<-n6KmBmG#1yRK8W{@J(n`D&!9(c$rWdgO}= zNltZP_HGgZ(lCYN!%8(iVUY49tm*)Lja0NfaS*-ziJ9DZ)$|DO6^bOc`z@?w3((=| zV=tmr9I4foi$UwiOl8LEkuQkSEM6^|{aSuQ+ygn)EkG0^3Ia;9AVyafbkWY8EAIZ$ zcEsbOdd^Pj_ajLpRm^Frars99gQwB7D&KjE=ljc18^zYIu`!?8ELzeC0IyL^VDX;P z@5Ovt@D}=SnAeOaDG-9}-KS7GNIBZ;n@UVbLHYq!uSCae(jnOeJTn1XF;d`?>Hhy`kbQ~B095|(?bG056$VUT zGNHKhB_j8%2?AF$@TG>aU2Qg^hbqAbC+$?$yv9A1cYRAml(pa z0cLtZVlc#PG8j?V9Rsdw3&b97vI18&v>lXRli-ArYBaq>mF9UefTM9|efE!VSElV| zW0)cSyyqzf61Z-%AZt7I;D<1ELL=7D%si8x=G3=cVBY}_RrFGH`Fuq+{hgGmU85Ud zsXi#RBu%O7SCeJsA?|#{+O3S^(!Df5m4Zt@#gqSYneAe83&ss%B5#&qTxN0UnCp}? zKN?JPy#;%V=uR5!V-RH_saCHc&e@zJ6aMWOGl(~}Z(dFqel(hg5h@{Qz2nPLgNxqb zfph*Eu!^vg)>izNuB}9D=K4VZwmHr1Ty?m4$MLOHft~Vf)`?%FTy$6LkiM&x`Lleo z;Y*m~TnuevdY-B5v$mK#1kP1zOd6Yuw*)>xk#o}{lvP=DzD`~A_7u+A5Q(n1d;U4O zr2HyT85y^98WnP_yF~3nnh~*u@#YTS6-#ZdZ5rwG{;&Hvlk+wD0j28b_)alK=&Ch#zFz%ZKXukP=}Yp* z1v4(TLz^Lv>CR=4bCV!?c*u%g?Q&3!H+<}OkxfW;OZ^(pe}i9FUaD~ERX3JJF}rD^hE5OSGaC)U+1i@ z=w<2^W^&ff&(MnXDr7v~YyRQeY2ulTIS~DiG$2eM2Zxu0>BO>w1W%CZ1c4fJCy=<+$)toJB{Q09$z zN?9h-*KZo6g$%sco!^Ogx+S)Gp|vDbGjZNhn0$H-y28i33CHZ4pn4(GM5lbTJx~7` z`&?90W~+lekmz5%;(vGaKBR^RAfu%|`5_Ym6=lKl-6+0&>iHNw$Ra9HFleY~_;+Df zKhn}L_(_O5Y{!b#>L|~I_DVcBuc~1r`5qx&#Py=IxAMg9FU<}nI!4BR4=t_ne&@HL z(NL`|HqaHM4W?1SL@gAWR?#fx*o&K~4H8A9AGV?~ib1Ke(MP+++zmlY+c2%JY7V*p zwz&CH0Y$-FF4g2naCPK0eQ=t6j(;@YC->f<*xD0l@gDqXUFL3rHdqIa4JVhxAFhxp3FKTcU5rF`}D7yr~fen_&c?O1_?RUKnDpURe>J)@AhiX-!^IRS6R^< zBSPbMQIV?d09!@QJFw1oRTexyNrpqFE;OqN@0^V}{-{0_^rtHWzGrb}!ZuYVA_y*svP96Js)_BD2wx0c5Z;n}{JFdvcxcLcKQgf3S`lEE{oU4jPa zKeuQM)gvIw@q?qJ6ztIFlDuNYwb1_g|D{RSkLuuiZGOTr%!k zsJrcF2ZE=OLtDQyDUkMSQw*Q-N^VEom@wiE($LoJ*bN+j7s!R4$Sy47v05*_xb&!H zH+2cKID_F$<>asOf>VQJjlAJVRjh4;%*L|H1C90JFhuK~`Xb>r#B+M+8J4S%S3#Ul z+%p>zMe4%QJ~DxTA8cRQqjco=5%xC8;wjtym^fuzE4iL>EpMD7UtU~n=#hFy9&7J0 z9))k?4ggGfN#yG!3I->v=d*tNR%xz|uL3g?_!PbkTq<|G`~xv)wvg7$09(}c-|c(6 z6cSe|2`w^AYUOuiuxhd5j1(At&Yzf8xk6#}qv_|*)kR+kGM9x^0S$OcI*g|9IJ#SK z)7v*}=9ZqF+@xqgkBS_fXHa9=pALzCbC z9?G7eiHR7n`6T~szGE6>E#QAv8s~)6M~%KJKw>J?&&=X+J?0_fpi|zLk<^Lo?*8WC z44g}!7_SrrCR3qAQJrMMfr*ftZONbmnI6n0I}btNC6v9TRPg$lAw&1whp~@ppDLkS zP`3^M#Vp!T!WyEju*zu&j;QKF;{{kdco~4?;XOT#5GU(bs6JqontkrC-MSqwl&{kF zq9pDdi+U50|qVL`jrRIv-E)1dXw^XIvCCz6NFo?=dn`w>HBiV=-r zsAJpA4}6M7^CBn7p-R&4VTM*?9asSA_cRB0iav!?Nd~uTL$cp``8~By#MXd?t`S?h zwt9=)Q}3KEN^2nc6Z;hFT}}{8?^o<}$60*(U}KFxHz>TVO6_7+UhO3?WyA2EAqS6E zzJ`?Q*M0qyIE}6MPBc41;WsHIU74pdXIwZbQ+H#Rdmt|iM_G&Ng7FL$ZuiyOmajiZ zquKQR5m)0_86@B;H$HO)s+FnOZ_%9u>HV0qEl;LhTFurYF0~;h^YA=mD<_6C(#X@u z2e{Bq7O;@=lPHB>3msCM{R-&h84SKSn?wz`XE77N6n@)bRHVQsCBzmV82k0l_NwUd zJ^f$xjHQ3u_)7s23vg5Uuks-y#b?bFiWeH#9}AodCgF^;T9u`?AAy{t?4$~{zRNa@ zZElGZHk??#N4NTyKi|LU`+OVvJ~!FOb_bR}EV&0wB*Xe*1Ir z@b%;PE(}~VvWQ;-OTk!0pSQtLhH5j0NL@0L$-5y<^{4F9o`bX$g zjKa&-09Btdl0a+t#h^27kWw6GN#*5O_#BUbYHjcsV8(+?mBU4Pi4}9HaV!#0GSaBc z(4y7Tq|0E74YbD6-{v)2xg~ts4M&u>V~grXe^8FS9cnXMU1gZtx%Fk-sMYZL6_g*vc>f2KqCw03lP($jrkE$658OlHoy`*e8z(r$KEx$zZWCxrDZh2~#?kS7Gd&-=3&g=9q?C3b^2R6G!jUfcku z$?7lA*t!NdldWx6##%Lxl#tgI?n`AdUPAHJF{1EZ$u((6$ws_XHLL(EndU-uBWD1q z@NMM}U^_CH`TH>K?l#;om5w7d0U5__Wm*?{z@g>*S}ZVJu}(6J`i(?nFf)wkxj@aoI z!9ldZBjX$-j6?DD0~AFNxlETTp7Qy@s}D8 z7-QL(bIzH#L!KZ68}s)w#q5HB*gGP;i&WiUf`!pL-d({@lJiJmU3Xavb>%>wh(?PzL5}?>j#Br!7*c-uyijasxc(`8hi%bxFZH)+nV+>*f-0$#hKOk zJB4sgwA$UN3I4(?Ale#Y$Uz)>zK2vvsxT>)(Sfi9-XxZ_bT8D8Rt$b(pI&%Qikl7C zbh&~~quP;zZqvi+2hKIYZUAtk?R}`=s6B$sXr6-&H-4}`W3=#Hrd$|VCK&x0ou>}& z4K~J;oJJ7Uv{@hL{jnXvSo4Qdt>9Yee0kjL&u7^039AP~K)aEu=;3tkJuwOqgGXN+ zA(*I1&Kk$B-vdRJ@@E<1Wke4KG z;zJZCCl=b8=_K^UVz9MfdfN?-`H;C)8Tw{UB=jo>lH#HCso>z=k6aZz2#3w$&F+Z# z$1HbrKui{r0TI)BJkzoeK5U`F>a>#J1xe}^TjCJ?eC!ZwA>nC?@#@%O{@jr#{R9~e zyc|3^lctl{{3+wr(;aa56kk~G@HW&f`!*7C5GWCKd;AQah?B)7`&MF1L|5dcnkawU zIX7B14wl0fpZY&fV{sTwSGctu(@gmsHASdh##i*z0^CPKyI)?wCM-Ssariug$zR$R zBZ_k*l+`(G;Nr|O?Z97H8fVhviU9c@rXG-CP53&R8E?@KEvR0{tA1S(Q@T22xBTso zvo~oQ*tGGs&LCG3(T1&Y4vFDxvt=c7`i6xMGxrB@pH$#rkN=we#PipA+5m5*m47-j z`2Rnj1xX%W@!z^Ya}l2>-W-O$RK?D?R?YI5W>*>ts&NkkUHU>SlE4Li1cA8ZPtAkk9VL4}Djh&kQJn7-(-kqK7uVE*Pn_&;dE( z(_YBo+nOJUa2#du>d`?uMBdlJh> z1#}tdmFAbHP>?y0*F^Q4i%Xv0-hVI2x{yA+x?j1DBp(j$zs{m&MtYsiW~R-aw<9%t zVulIyOl|_rL1YW^oCk!6DzsIq_@4-R=GyxISgTk+a_s5bd8hiED`S!igoD`dIgK)z z@NUb7(x0c&&~1@&sc^QbxSQTvNthC$0}K`B9ThvvIS1vk6LgIB1r}7X3zT30VA6~< z89H9*Oo5x_%)`kx2H_8?%GOElS?ZvNi zW#7Kse*nvung14$fY2!=;2-4FF5oY6%6VQZDLW^(X6O^b_jS*pR69QiYw&AR#>-{H zynlICXgA?-j_@Qtm<78ko3WAnjl!>W`HpQHD=hIrsw~9CafX)FG35MDKU!{*AUd;z zFc59z4G{vv9$~UUr}2(=#ea~~i@(Te(7%z>18KnjBB#Va)5;1&P7Q&` zDaU^xr)U2lr@me3fnWU;FHfBa5P`@k{XfWQk;AlSOV#PoJ8Lz;F(-JUaNp`JzOsFP zb#UT4bo)kUdb`O3MNFATLsQ`VE<`HRU*wcj8Sp>IDbru%bobxLDIbMvg`(Ol%D<6Q zmsKEgDmsbJ3u#5PqV@xZnTdRT{wIez!DiX;iN7h~n#ywi+nn-g{^t?;(`FLnW!>wc zrN(CrsR6S}^WV|<*5`r?zrvs) ziz2rNH5wRyjgZ6ENAsq-{>j{Ltb{=!lfJ-i`nyKN{Mf=AFXx?dxP$0r28|KX&22gx zxW3WN4Cq{ZB>b{;Bji`Mlja}p?*cxJu7Yk;h9xK`io=3~Mb1X*C=Z+@QOQ_43#R&) z!J)L)ki>Xk697i9MZ}S5n6bE*7)uHwhq>9We-fFaotZDSnYJfC_VU(Fi;$Eu{f53| zrCnym?K(p_e9Asy&DiXz*EL+qiZ+=GORl0Kh z#pqbX-WXDkY`J*%4(d-Xe;mL53E=S6el!$9Tqg)3n;0@Wb%tbwEB0i)K-7FBnVY-qk~VbSgWw|B7(*_ z_0km3JF;*xEmVyU(0{&YIEZgu-++Jn0ske)r2uE-(g1Z_Q8ZujURr3_pzjf?mYBNM zTWhS6(iH~Uxq@|H^DWC3UYJ*93=JDH8q)560x{Vg^K<0VAJG>a(>LN}Zt-{5$)FG8 zS!w2%<4#u-8_%yf0v}+p;#HU)@cLFyQvLP8!#G__wfVbY!S-z4CA(hm0v0Q$!f&& zAq1o@XxC}cUae&Vwq%#bsl%BlIcJo+$rdeZ@ZaJFU*ITpR^jZ2m3h(onWhY?e$deEoB zViM8Tg>$eRm=WXDNP-G|P8?GI4pCG+10XT3xz_npvCM*bfLd=y+h|Iy^Mxi}8yB{7 zb{Bg$D%imq&VF`a)d|*l;$RN(cKDJi>FGKK{z(@xM0E zadCUG-z(Rh=PXQTi!yv+FU?66YGs3j*?m0*SgQwzbi1|obAQ*DJnW7PC9yKutVh?8NHs%fF(^)Il;CfLy zL@rusK)V280<+m`m*c^0+@E2n4~7hk{&vUUXBhQk7LDVDupmJAl;qTbfC9ikgi}y< zM-%vXy~R$aG42>BW{~uoZ{wm=KH&#ql-LWy-35urb0#27>I&(70%5PLf(MKI0~0`& zHG|B6sC8h(P1kK!a#elhR}c3KtuL1>Smoe~(EEg!S9iC+ zFWXkQ#E6U-$G{cB#)QiXJq8$Ciu7TE59q4&MhhaHQ4O6Y%rxK%0VrOpjA)|`It=}N z&bq;#*W~a~gjTk1WjF+WOP4#NhSv+m4hYW4B1YPWqGR5y;oHm;<9m5n8x5w|=g1s2 zmPhwc+%#Ev>|6w)B7ZcxGDfuCr*S_GNz<5rP`(WtTA>i6UC2Z|JIfCqLpQ~_{XW3= zn8)`BlW_4S{-H|q39#xI$pQD!ndMT}L*xVcMDA``t=h^)@`{S6aazUVaE-f@N5a4V z+I;@n-13Ti%fQ~AV*nT|Qsyjaeacq&@of@a%a4E3r4sEKevD!Ew2>#ru%)}_oiPBw z(T&Lg_Kmq#kGT34M4y-E~(0KJ)xxtC@LOgkeM zh<|zJY~r8BUH~babp=4IIazoZwSE*8XCPo=o&z)S*uuI z8w?2tD_7(~3*iq*eEz3dPGx1&EY*>^21q~Zw}L)uczSj)!!uR#%*z(XDUKJ>a+TYf)2bD;5|s^zH^w+3hM`N)!roV*e7}w3l-(Cn4HBi zU`t^_xE=$WYmZd1`#4`XUr_ zSFx0P$yyXX>P{Ttth;g9u_nrXb#61qje2WvW;{};^6ZNen#~Q}ULyuBM*0?Q9=>1u z^RM5fk=2JX&>ivMs9u?yTd-i%eau&$j~ubX_YK#9-ed*)Au)P+Zb0q+g9|b5ri`#X zY4k$k#}0D;dpFmBLR;am*-A5189Dp~G{-re&Fq%BptQ8UW8>1D7ITmX?-0Ha#z_n+ zhj5!pl@Iv(>z7-!XO@i>&i;F49WQY-=={_IxXvPtq0NZ?io1N~#7V0Qk|6ootW(aC=p*oSRQ=cKNCCR`Spl!YXv^py5PsN8i#tDYPy%q^`bI9pYbdcQ z{Iw=PsQLX9*&90gnK{pG%*?_XE$1IQ?z57MEh}Ure2T1eB#bgr`QzDr)*sKNj}f2A z9x~H3{r3mHJZ*YSc^^4W^F4Ck4rw#|f-}JXpy^G%7KF};yViuxNx1fa-cE3k>;srW zE2HSeJIMCQlXVjBwg;+{z3KEZkiIGPS)=HsT+={zCtn8#3P=Oj0(VORLm@4Z8A9C^ z2kb@O)CR6Z-t-0tMcxz#azx&=2LR=@C|4DIpjefCR(DKzCY%Efsk>19ByjI=;@A*a zThrHHgT=8`g3*N2iTYzBcKDBx7XZcr{SIK`ai*L@28^l5l|6n?IJ~vQ8bLnDQlYtV zlQ>KRu`#jT?{I70tcHPxiMw@_c;2Mp3`Uu|1&WzH>iL|b4lvUqj_Ky+SX0cI#;FGG z>ATub-2EI_UD$&jOS%S6?EMXJ3FimI(_Jz0;y1#)*n=@QyOh;C#)>+F`v65WY+wHv z@f&6s#ka^oT$>vXZ_4oQh9~)8fT7G6FcI}4+5QO+N&`J*r<9*-#fQH`xr@o+*ivNg zip3fm=Lu9ExstQT~NvYvfAj zaMGH>wvmG?O)fwzvovR9Hvm`~x63vM$kzEa=B+jJ7g{eFIC+2PxrBq!*jPZv>Fyex zKQ%LIv9q`+vhlplghsx!O*dU)s?_pXGX|$iAwO08iko4lf2M=c=wjzeM#Hd~1-?0A zp~h1~KJa$Ua~uP4_p#!thchnr> zbd;KlR04qhN@M0S$kQm~VcprJ`m>SE_LysPl|1)Hg@9)JJZlyL3rC?jY;uO+hAt_# zODeYP&kezMaUUa>j5Pne`_h8KN<&zw)e|)*E42n*TbB~{LJYVPV-0D-2G|1jwntT6 ztw)1xT+VTqEL2@s3f0h+0xT_s2kaR=8`g&tmTrR9g-AnC1sOj zLjCscvT@ESXZm2T4jY-`kHd(seUYs_+b0Cv?m(OP7#HgkJ{MnmukYz5ynJElVc0s% zG+13n%dNtXW$LxOuEcq3Q_C!$^Je9ikuu9?npmBK$|GN*M@_(5tgp`Z!f9F!%P2(7 zs^V$I`Qi4(PW`sEfb$D(mbqeh0QjDN19}cCp z-thyBq-EFNfvR>J;c*~Mbw8g~KuTy*W}_bw6uYri)Yrq3PQ{+Fwdl{_lvRY~>NSrl4XG{e7O zWqMHLon_{0_qllRw~bSQgPJw#1G_~@#H)JR1f9to7hXl|KK)IOf|;umJyAw6tOngF&XAL6?jYeAF9>UjL~julFiChGKZkYv8Me ziB!-JbVNOn?k8SRjT36q{A{SAYG@_vP*3{AH5@?U63V>|rHgHBdtj-#P+8bF7b?Z+ z5G4+XG8Yj(Y$;o4r4(cr;>31(i7JXyb>XtJfQ8qvm!>vFbz~QmVc}A3|Ai)zUs$J2 zbP~Gq3(9$V#)791`Pm5fIEYFd$oE2&F3Pd(L8Tl*Wfy{{!V5D3eH7lOOo@gBQHYe1 z&)EPrgTOXN!sM2lmTiKngfZJjttZ43bN8~8*+1U;TwY6ELnGC4fxw+7n`YW z35ljiC)$wqM!VXqZb8&5oIbjioGxJ$qLM7++njR(pR8;C^RbM~ zr=9A~{oO`bQXC)y9Tu;RihW4P=>YhQ7vtl2N#EqfO!y4>0>=Nv_* znK}@U5`)PzEggB=89vFnpz4|49Nr!-yjmD@`i=&BpEN*CT)8OknNeHK@-K6ZlrIGLUL%2+(2gzPA)r!hcYw9w|9*@kLG-3eqN zbmOceR=hom)!mevzMsXH&cK$HHr;t4R(XKy&P6%DHqSJ*M_-Nu7jla&s}V}RJJ0?; zfxs@P*oEf4mESe|^MK!Cr2iuDk1V0{)0LMW>dTiv-Nge1#lpIZ5n4|hD5H~Mn{NjQ{Yq+=YYpOpJH((dfXAsMS_ z&lmP)=5CjdIY20U!a%jx-yahO1HDtJws5D(p93Zty;HRoMG)x*inx_xkc0}Nz@U{- z^h-<;RyfLUL$q+for@7?I3+2%@Afp4e#vi=?@h4QBzs}@MVYA?ZARL< zlV$2Yk7*k6JYx+BKfXJHXlDA6aOwqoh8vS6*@T@N5d3j-qC<&B`Q!TcmBI_@gj%(Ap{49O4?ueHOmAg ztO0La_BfZeN8$yuU9P7&X7-gOiQhjxol{8O>JQW=Fv;whx}u|xS(y1taeSIB#5~|{ z4gLGfnXVG3w4MC7(sok^q)}{9fl(t#%KvhC2dMiE=ZRi3jo#H5BfFHa>%%8X%*xn&}cNA{xS5+9u>j1q~gTK!N6gx#1`y{8ddu_-MxtUHs< zv5I)M_t-h;mYai$3eQq2(59rDDaU*Snc_@N*#gpRB)drKa7r0#!rtjMoqvUoT5qhO z7V~lzD#)j*w%{qqXg(zaD6GfsC&=VSb?vD9J865ZQjY|2vnwS+cDxc~ztZ^4=T09g z+hf1P54P(onT@(6wdvA0k^Ra0IL>!2^cGcALz`|e7G;56#Ah|r-)ewymzy$s37 znH-s69!92}dq=1I;3BTojd+rfNLo&QtBOrtU?lx)nu1gQTow)~pg% zFVtMwpYvF4#0stfyaDb+KiANM^D4TCh1IRcu^_Z+-~xE6jZLs&QaW5ghoShyj+v(T zyf*Z!mTW8~RcqU>Lg?-yY2ghc4>1jN`F0EE%HTJ5Rpz|6DXbHQxP)GvUs&$4Ce|oO zUCHHU`mfP-DUMA!Mc*cw+T|Sz;0b4Xqvz6$bp))w@#V59NJ$tV%gk_rB$GSgiJ|)X z`L9`AO}~lkp+&}W0}5I6b4y|zFN|TKcpzs-Cs;ER0Drl$Cz6TND&W!X`0t~w@Ch#u z3{;wyM`ObB*)(logsUHC0M`*rO%;VpI51<)_y$ekz@cdbf;ophUd*^I$Bj??M)3iz zcb`8L&>I0+^TDRmGsQ?G!g*)C_1bmyt6Be3=i>?50H0uxchEQ-Vh}YPJX|!q1Z^Av zih#4!XyR+onGp&bW&@U6T_Ci7+2Pm&0GY?ocfe%(3QU}rwFYWzGRX~WI^E6bF?pKx z(f-yxF`P4jUD1cGkkQsZ54htHb;6F?gYu;exMpM3lzZ2c4q9^nYN#dXxBa|e&pkaX zvG&ECO1W?s88hP&3oY-Kp<^5^ekH1k2A+5c*gel7Skot>)Xt%rFU8abGM)}3J<08Ef#DDF_HO$ zFXe%koGwGEW_nT#&#fuEjoqI@50(yY-EdXBp@DAkG*x&rpDLoA^5A^#l&jo}r<>`0 zk0#S85s>!>xnKA~zBF+?>I&;wI`iIWXsG-G{9`XJiY<|z5uj?3;Jr$ySjusp?U;15 z&GZnf@G4s7?`fG_nw$hp5z6fhPE7Vs}oFiZW!4ObogQ?@a( zQ077;Y`wgCpIfoam@CQD!)<=(1_Xi!!3D?MFR+d^o+6GloK-5bKC$S0CK*V@BxkWJ z%*CgVIgWA}boFp~A6--QV0M9q1hhR`l8(=DJ_zzVZFjzQTy=GHd3@Xg0Ixh>HlKO| zG9!;ny#bm)63knEsDfE9;l>ABn%S3n-<;W(eqR>bezcWpACuXaYTp*SJN~9CGCvX> z+s&lXbGKiS!F6Af;bD)8;f?cfkU!FWHc*nB(DV-zIg5-Ovq$j%o`7I2BP8}9tkRU>?5>J{|Xp8Mh|>gn5~_boNpC77|bwRrg8fF_5SNm zi2#37os=C50bVLY!0T?A^rS*2WT=2wVe*qUsjzCp0Ol4#x7v1j1 z!>E-LIZJW!f`A`w%@eihn9a7wWEBQWL}Zp@U9b^Hw}c% zloR{gXN=R9SSdYBa;#k-#a$vfZEGD1>}6JY7VXUZiT@}~pR(uHMw%o3MRInSkeEdn z+_DL8(Xa{x<%9Bl-@O`XQ)KeI@vb79Q)h0;9x4|JpG}54Ky!066Pdw*pez+}5x{g| z)zP4Ls%MI4yG`EN`~Yv$ZB-#5yG~mrJB>kRXFVqIy0phL{-widtJvib{oI$^^x;J{EOnU#Kc@h>EWui?Z;jX@hD> zO0#9$Jis(~g@tqCZ0vb)Ps^XH3b0OKX1hM5(^uf?n5o9%DKwseY881ZdQP}PycOA$ zSSK{JFt6pFY%Y(AplwDr`?KSP7#wbj0!(a(7|IgjT3XAf-Aas{zL2QKOMmfr5 z2HWDwfhC)@XkA@#%1LSSs2zL@irv*9I9s>ESoUMSW#SiT`lLDOZFDj@2~m23dWD zGaD}$g+{j-E);^5e0x&X?E-7c?P8+l{O=jT60c&KmmFnVdlc57GywgB?6kZwXdSvF zSw@`27nFco-QG(H{TlA=b?u=0vbZfj3w`31d6pHnnWW>q9^Ls#@QUC5?{!-etZUo& z=lFP`uJ8_{>`NOyrLMN}-sJ(4bn*u(>(>|+?VHGX+9i-3xjX#rJulA?66!Vr`tJi0 zDZETuwTI3wXe65ig8+#~Nu0;IZ5mEUXpi)!8Oqt?Q+Gn!mJ?ASzfx_+n7Cuy=KPRz zk?npD+}Yf$*=k#f*`RJS$`xjb3!MS-T2^Kf@J_E+MgURSVE~M$<~%cjsh3vDEbq}{ z_1=x@T~%*e9QyUT%`3lP8cnI`w{~ zwx?3jHZJ$JDfMPONa7mYslqPu!jra3aw)(qWWQ%`1DHhNohsni{ekD>4h?{>$~yrY zK7!&Co@xO2Gh;e@Aj5Mg!f~q4)CIb627?je{75oL`Kb@z3ibvvoVK$Q+X2=A?oGFY zQQO=0?P(`g$KUj84)h2J|L8MutILS8KOYDIoI4oaZYIjxNNfw37xDqQPvKq zy`bw>AYi@@Eaf(NUyW=TUPT)ZCFtWplj{@q&Fd3XQcIjhhj?4WoM(|qeG#@kzkKzAulS?4)t6bk^|<6H}r@o-f?R{|JCAR zTp&aE_w;{VvQF^VeUg9ZOqCr82a{p%~@ zmyi|hUOGfXzH|3n0@n_Tc+ZmI4y88(L_S zWuo?dU8kJZNlL}DIKU`o1={@BgrdAG<5))bbfizDg4F6#<)p`%NB33GnwQhmQ(1G>S! za`;Q$>?%!9?Q!Rr2eHw%4BJhG-!SzqsTx6fsuu|rXX=+b7%KZDz1eKayN80heRn}x z*sxYV+Ow-#(i-NgY##}}B~Gc}T0bq$nc=ifc%Q!R%6pz4gtF`C-ld&sJFevyznE0? zb^Cc;vy`Je-ik&ZKAlZg$Kci*8f;Yc|TZ7o}q9_dD0k?u#v`1Sx9m_?VBK(fe zT$;E8l~p(Agv%*+UR{x9IQ~(W%5>0aHLt0^skv0gW*oBstwf076rDUZQk{41)oeRj zM~A>ST9-A1TTOPxb|u6-J!%4rQ*|=A)LwajpfX5wmQQ^4LyO5J>ml8;5Vo4+$eFFx zW^2)T`&V}w57R0@+H8x3E&b!?$9T)PZTGM8`kB!&yoANzLLwX*Un`O7bp=$ZQzh-S{#Ko;% z5^N#bTPOvu2DOFQ_-eh6iuE5sF_s0@mRRKr_+(84-6%Eyp^avvNJBIZLVm4o|16A^ z=N7NZZH7ql6#cS^XZC=<<++tt^#$4YJEUzY%#b+P(|$jFP3?^d+AzF5&G{zuq}jun zG#8VD)o>ed!ypdz6b?=Y`LbW8<_lLZG<{1hy)B|f{zjBqDZp=>t1gH`5$|sK@zBenJ{`Vz?5ev>1Ngbo3?P}z`bQg z>w1E=8{g?z?jq3xDz`1@`|bS0T{n+cx<&f>KGXm}sd0AjNQyUG$ZSM=(A@kA4PG#g zYCa{diFd|Jy(y}q+Ivc}X`$RUPO(L2OF%Z1`wQ!g1w=cl&Cq-bWiNSs%Utfm5(rO8 z(hytKtxf$r4vBpFQQ)6gSF*<0>@MRu*F*ag#Ji7R!LTLY7kHI#$+HwemlS2o zDC$FiFIji^C3@Y&fsEB8vRg!z@#n;}N=9JsSVi|dbMW8pAk^KRyU6B<@;9NQONt;7 z4nYu7EXbT1QUA;^VL=2$xR14%cshu=aWh^RssKb6_K8&;FP22{}Pc0SEp&s+>uZLI248NLXhi(#x_0*zZ@GCvJ1sKGJFSS zJ`VVw)a-Z`xTIFPU4N@3DDl$leHr17;}%TRN?LRDL&I|-Mti-G*Lji3tVD4rljJNH=k^~Lk2lW`2s>Eo-0Ut3&Nf9z!qluM zGbVEe8qBrfm43`i@G1(d3Zi~_3igeVZ+;mFZm(wI&vO?!jOU7+wQc>f>rQxvG$(Dv zcmRaqaP4Tub!G*Tq%co7T2rm;l8M{dkvMil#rBA0TsJuNd+?r|#iJ6KGcx%innmmY z*WG(C;buBrM?M0rzN_FixaU$A*N-|G%U$Y|XpRS4N2>mpp^MVtB+g6>laUn!L=X># zAVS6glNcfn+?^pD1s+d0w)}-A&44T71we4#6VKX#uRbMS+>c1POgVm)NY<&4KeBvb z;(8>qG#W(I@_;5PMACEy3Eo-ACen;d2sdatV=j=z{2A&g z)IDufi^%}!_(}ESY2-ArKTI~>ekIumrvbIO>$HBRF)xxEwBU z2WnjuCIjq>T;D61i62vFk`tQ{%s07lVdg#lj_g>$Gi+;D46QN*VH02bzg*q#5yB-M`(Yo4YL}r?@fk^0jVk!IAqd}hLwD0f{;mVOtuE(W1(2(% zxdwfb$yDyfZIUgo>ZG&2M+*iYlvn{4wW4UMx{Axi%v#)uT?>iCE#`Xv7>vwTm~rGc z9FCz!8m7j=yeF3?m+;PMyn{AB!VWgkz-6QMUHciuF<++`VHZ5|MJyx~oU^MjDB8RQ zJKaEhDi;Tx;{P)l*}}%c@bCn_q<`W||Ko#+J_PE>)kkm;DcvhbsDLQ7S0|t}^Rbld zOy;p4@)pVwjVmNlJ|btL*}OL*J2L(ilLx`kXkS5Md2Vb=BU#1rU=3XB<;zQM(F_Qi zy;e987$r8I`88bP3ETwZ8}!?N{#Semr>RTs+OFbI99CNq`VXecPy*DNDbZ-Vmmu zNkh*eEOT~mWh8@e$_;IH;tie@vtCbrWQ~!|&@v;@z9>7xE)zDxz9V*xf!vU_M*Iy# zcKi)_yp56#E4F=q>$1zXZ9&Njy4^2FZ5-dim%` z8Uivb@>n!L1EvlmVqVOIJB5Y#=+ARxw#tR-c}>q$0`_vGQg-YEjETG9sFp`u?h=dz zDF=)3MLcOL;{`mlZQkw=vfzeb_5$>Y6q=uPC)$m6^MLNn@m;)kj`FwWnF+u|@72z# z)@M+p7VW8}E)5za%|lUg{pc!en~hcu6*@aOY>3Su1M*nc@y^s~uq#E%xQ~sSRnW(; z4yPdoDn)cN#vD+mC z!fL%tI~{+z{Ck?4&}8C-=HT(l;2DXqAoRzL4xlV{f&QczY(F?X6Ib=NnY1gT!OC$d ze#LEmM6*bmA}4(sHG?+EPLBqwU9C{3l#E0iRGzFiL#07jK2hZ*MlomNAjO}Gt}2;X zj8W?%RF&tY9REGhG0J$Bwu%Mk7s}ODl&Cy;023Qd%wy49qQakyJvjxpjFU$@x-Lb1 z4L~_{G|7RvtCTh#4Yu*gZJQFiDxQ{%l}E&4pvLi=Tln|FV`%15Q`PAXQ3JAs?>pJA&M{N14~CJ-x3bsLt}D ztj7P|%)%j$nAMTYvG_?aNS9}vGM9X<0H~wI?bVQpD3OUQaMj zg?i5>!==EJSvqYr=eKq#;`24fF_*w0zsnC7VIe?n;fQhfyJ}{s{&1^oR>AbW77mHq z%?xRU17Y3cZW<2sLZsX~y3r2+`2FVN^{i=VhlQ6tzKgQ1HPnw892SdT&z_*~Y?!evON@lI2MX+JiyKJYQ27?g#cBVtJ~q4Oa^L_5bcGmKGD;7p zSLU^7CA(^EW=O18bU-m&ARZqs*3t5kiH7}9z!Aoa8oH)OMpPJRaafFAF+3_{|Hftd zd;Q6ot!~z7*zXD7dyBkF3gi~2#%%-QlOXusX`W4Nxxg8I)+M=Guzl~WH0!iACT62Q zjt%c6K@-o6DQ(akqe627kTI#C%yd70m^=U>1Qu%@5HA~M3!(E?P~?NTd{Kno{J7H* z@t&~z>b_z*uHUZ71(eyJFmjJ!srVB?sOMiPGmE1RX=&7Z}2+3q~m8PPby+M}bxY zO?Z%qQc3EKan7;}F=*~mYYkq#X4Hrj&nx$n|Wy zF`JxI{RPwyp#9Sw@FmvLpm0|oV{cPvgq?YzRnV=PPGH{KlodxSb$PGn*7%m<7>a0) zpWJagGs%i`TXSOlSwJ|$1x6-%E9h_1pgz;{k~)!7`CJw!FV)v_iGm;XShX zvjVqe*Coh4dVC&$BHXp`Qjueg4^>?n#~UY=t{m*H+5#S+d=&S`{t=QNkyE&xKB+a- zGaIU=6?5k2=7QpJL^@s^*wtfz*N?b>u$@SjLRSh_9`w;zTQrz(HblEf-?d zJAn2e47J1z)rp1kwFY^FDJ{}C$~t)@a`6LyPj;Ml(7i;WVVqSw(s?MgPw#X92vM2D zd6&V%NzQDMBE*mFe+nW8GWZWUU_G=AeE$D>=>MA9rT;Otr!*xZV5BhK0%IR#(ve63 zMautLxS3Jtn9v>@9D^jK4O9F(LMyB_S4BeAQr$d9zysR=K8v-9VVTVVPl`~od+?e$DwdeAV&7zQ;)IkrZtnG8Lw z|CwfCWLmEfhUIVDjh^}_V!Ks`T7bzuG7P&^t7NQep9Yg;trm4lG(`4~UyLf3YLp1> z49O}t3eKf?jH{#)$zwH8OHx(BFUjp+(fJ;W+(x_?Y2|~VvOhFeRO2J`oS2pD;JtAa z+!&%kmS$?@EImo0g@dAp`c9aduto8)#bl_bAmsL$eK&u22EbNlqZT&zEdwHR-JvgD zYT5I>t#$z5+Prbg;oq zY0<7&+~#p9C4HJuVYhQxcLD8b%S^Z2%117~pNi7Tm?GEbW3Z$vUCa-}>h`(S4CFYQ8{~dcGfFk_Q*fU83+5pWQ{eyh;Qh)~=W#zj87bX-EJ2dQ9 zA!rS{uV`?7$>ZM_wBOZSF?{0ziJM=-CF# zMtbV#5diz3y7vFBB!Lu9T8KiF1XCs>fk4C+L>fp6J9^Nk(zykWh|tc67z}%h>}vQG zq3+o1h8)A*&@lv--?a35q&(!U)DreMjtPP*E<-m5RdTyj7=e zFWfPV*&aTO*)AW<;T~f|0j*vElD{^(`|3&rRD>Iqv)#`@TiUIjy-C`wu{~QDjJ==! zIP98%a^Mk~w7Zsc5k<7#q4mdxH)7x}p>>?axdc~g$IND2hdWIG>K8}aG~XQuovn!~*%j=^G;S?wuh0Kga5w4%m;SPjnC}ZwPK>?s#tHPkj6(d-_-^ zH>CWPdsGB`lLh7g_dD8{kEjk+-Vi|0 zr_@TrAX*Z)_R-=X$%bNTmC|jpmMv}#(JImnzIPqEs7&-f`PF{qxHwgypk`prTCMx zrFf-;!<0CH7!5Iv$Ck7u!MSBLqmfihqmg5$_;2d^=RL%dJCgQM+F~Q!Bv!6xisLiJ z502Wy8ZyC>2watO2#-|D|RWV(Qods9jDKiJG)#W5j>q@g}O(o!VgAYYPU{C4f4xW zh_~!9OO)~FR<4<+M_%A*av#Z8eP3$D2*25(ME)L7FGqq#oG=w@Hroca`_t}XCf{>F zN-*6va-*}1CAVglC*4Y|Qm;cvd;b6N^-kfHaL>AL$F}Ws?2c{Q=-5fe#_ZU(ZFKC8 zZFg+j#+m*7*E;*`^_{)*Jh{n5E@sV9V^q~!zjsLpFKUSrAdS#lcmUQkwYhG3^*${T#dpE99G)k?%fL4E3$)nSFYB0|mPwHY-i%R-KL_0)YH8D61 zl4~u=;SAmlB8o~&5@qxxl)49l`O%)vT0OQE=PPf121p3$!!Z02GfN7LPb_P?>uf_E zu`!H;biG_X02GQttiViF*h3x`t*DhgPMPA{x@GXlex}1}2>4~`y`IuJlTTXx$b;lh z*#o@-{%|hpQev2huMhM>FwEk4*WX+=R^unvj$7x_r)#sne9PhdMOBtG;W{dhm$rwK zRC32L7$u0cpf_1LO}9EtHJF^%=D>w8PUhJl7U6)R1u&z8!*{(`riz)J5qkwi(N@(j z5F3{?=j~l~M2HZFkr`~3G{<^P&!u>-B-&(&v2#4j2yLCWb#?Ijsk>2!as7^6pXBFU zL#@CkM_pnM)nMm&*?#0q&pJmET0eF8E-FS;?2!r6QHYhWl%#{(|KUV9^;| z4X8(5;*Bf^|6zgoV}Z;@M?))NEyU`vu%orUK`NW_u~*}e`nHjSHVvKnkqWG(7O5^o zwF0s>K`Ybr>7(wnBAq`NnRvo9J=c9S1esi$!gW68MRCi9-SV6V46E5Fp+@#QZbrux{HbPy4B8j4yQdR`2@*Jm>z#p-V z@NU_cvAjoeYM7&hLQfaWl^Rc&J?|0|eKAWH3<_n`dM*_fqRC>^RmUIkj?h$cascq+ z`qzBl_*4NQtV!}c<{DLXeg$&rsXWp!G5caZ6z|c~%N>>Dm9`-Kx-l-7`_6|ZSzBrZ zu$D5Ue817qx@}cM_&VXhA`j@YLc*b}*v;8;e_t znkFo#R|f(NHPd#xFQ*|jrCHINtN`dJNA;z9-oT*q`@*$_n0*)@v; z3weX}jl_tNvmG%J&2ff@x(JlBh_D~I!O%LQL0zdUy`*#^^z6!<&H02Jc_1NFv`;@0 zlsqx`{Z3}d7nkQ-ufV*ST@Wlo8S-K;q(LXZLzf7k*A-XhS zB3*pP8!cobaOmJj)`LK3-W10+=3OU2*gx^Rv~I}zY~v2Hby=)N4`-bHM>}$sZYZx6 zm2zAZn@*62rd&srUsxk&v>9McOjobplUI0G!t$J1yIrWl&%F15kzViOjqLDFQ)$dJlP~axYXc0c* z;`qw>BMtkp2-F0v>y!NHvr>FE$L-#{Yw&O`p^;&<1MBrdY_|;5DQS<{Bcp5XKjU=~ z&E%Lf;Ml?mB2q{j_c1TI9O0x3RZzv~l3XuhO!gmt~?$r83+Qg-DT8Wi~g2(4ec#`?1ZQOuqzZ zG0Y3AfRB)`_aYVVMF_%e;ky)lITi_ypW`)Ia%MPy9ZLy^g{e`?#Zo0?rsI(SB;;5U zNasQ8`Jk5)`;04Y%rKJod2=W_3gCIN@FRbh9M$EJoaIAVSMWxG4fdP~G}AY)WpoRu zEy#iWOr%rszwYQ$L{UP|RuXxpK{Esh|8h6P4_^7S@GMu!SVJH{(nW#!EOHK*Njp8& zY}4aA60^+3m^s-!Uh9kmLxHrTjH(KAj}^;XGt`+nSc(370RP|BXiC~F z0%}S%3IaxofC6w3{~sA;k=m&nni%@0oWAiN`a(-u1A4|k>4h-XM9M}qnlb?kiB3erlllp&$sZ3(8J zs6KfD6(gX~omCo)jE3c(uMM{(ux=^cg@om1ugTrT^X_Ldcyb^I3yxE5bJlnxmX&Z{ zJ%}gktpRj1wINlB`<>gOd53Sk8q%{^cM_$IeLou&6m7bk(XylYJ)_*Qu&_bGs!#X^EWsQ*CJ@bzQe?;4z=RTqS`d< zatcXON}pSU9!E6#X5~`!P9|raoOqk)Q1@7chFxMZ0%X<6kRL@E?4EZiF3c&R(n`-= zggBtjqnpX>S%|nV_;R6)Iikt=B$4r?#}+4!IUZxXoObqy9C$+sX{CV0aI6igtDd;S z7WGdO!;lC=3p=k-G3G11wp+jNrkwAnW&P3h2djiyx}A0->e|DjJr5?F9;)|L@)~SZ zalSGhX1_!tJ(xLiaSR8=^zgdS1gR`uUz+v6O{dmKx#ec`YyDu^Q{W`jOnX34n=wxRf#G zskgr|uHyt>U$o)%hN(pmSeeVt6>0>0Al#A1TUpG##;NY6EpGHPbq>&F`3mc!HLw7* zUs+?AZ(`ZubK>zW{9y8Zqo3Iq@}Tl#7L&&3iH<4~o#ZmTU*K*=b{lvW`j@7L`FgL+ z9+<^pxPaUVJ$T!FSNpcGW;X{LTA5c?xsF@BK_Y%(<<`?<=MW4Gn59j0rh=h;$JJZV z8Ba@soEJ2^i+poyC_BC}S-u`g%2#Iaf;0lzx0MBT##H&7oJK^fPl&&5rRZ9=2`O8d2sHoAM2J$>v;ON&nee0T+_hqLN!_?nt^I^jDdkR#nKX2%gpIzCihtzH zHBzOye$1>%6_DiLl$+~=_NKIxRhTR19qZ@RN(6Ko4|%=)%jI00Y1m$EvD{6LT4e{t zbHR(kS>inWx!nN;*= zV%jbz`XS~;x@a3*miv*pEyR?T5&2f%bFev|Jy~?p{P|V+5+9M2z7Gg7?93{Ga@5$H zw`0bp?4^=(S=tT?-)xBLA0;a*^c^?glXHUz;2B-V!{&dNk)WBOe;|vyyrRnUKQSQ! zK$fIndeE^KmZhFr5_YZcEmIc&u`C8zdE@nT9z$FALHuN~hWk^L>!eamDrHB29D+g8 zITp2k9Q~w1c(Gu#{$a>sumTKu-v!`G7iSX#}L`X0YQxk!v2{mB!i*$M96H7FY67GX^ixDMjV?ODr$SE?DX zc>X>;3UV!mR@%P3n}0ka^%jKp=YgsG%fEKsfqzSJ?nYz)$m=A>U@Fp*LWkPs(LK=< z(F^oN2FD65B8nmdt+}dW8wskH`TF zl5FcD+_)PMRF4v9Pboa|Rqx170=a}jmV`p7u`m#hLAOs0<`z4}9_)~Otv4_bLN~c4 zlo_VkL9Ca+t=+!?90De|Nz^eYk0VZ%eSuzWQLTXP7bd&dOtx*&ZBr4BX+Bh#uNLi+ zV-K4xmoX=XVOf(IZoz5!MW~j7OO_$Sb+d(X8HO4_FDat9dM+Dr1Y<}YuFc|vku&`} zDrVRnf$N(R+$P(8@~IqKV+ui^O7_iBUVog~dg4~5nZ*Nd0APj=Cb&DEiqkY4XN-kD zxFtV+uU>4=V@sA#rL&3 z)l>{%1LKqEo~@_nnQJ1BL+UYC8*ZfKH0Hc#+xWy=p`Ky7Yz(=XKyo5pr5>b2`r0tv zOr4B-4n6=Uv0&}Y*PCDMjk|w`zmt^L?s5fCd_GU7r=rbm$CR3Mrt;I!gILdvYtRMj zTl&=F78z#8gBSTknIqi{84o9UfzqKQknIEb$bAtR_xaLr16-#{B{xfV<+(cnMUN=E z=5wBRpiA)yndE+%y#qw(t{BHmJ#lmjdtyHVgboq&9RLI~9MG3}K2SY;k~!TR4=53} z+@41!3gZ$?*C1c9EtDke2=v;&U&{mv(I9pV(rsgdLP@q0a{A+wy!fGbLm|=GHxLLK zfM4OJ`meF^$#V0D$IU-^ITt_7{5~~(oAnjr++ApYLi?M_!~}j3lmOeBiT`eE0#g}o z3R4#Xa>{2PG7#~tv?>jBu;g1^92_jIs`}$eUvx(bd0$H!x|P6D`qCL@ttG`lvK;!N z2K&7mN}xLiwFGE#_X;GiE_0ciJ$m~9yg`?Q1_YP8>-^NftRdMQu>(x~4QEAR(I{D% z=+T*N4PhYQXjAKJqN#(Ge>-@OUACM%EHv_PCzh|8Ndo@hX}~wW9 zzO1XZ4Dcsb$F&}_jN@+xl+CLW=V=@%%wY_XZqVKtZ3Z&{sKupnv(I^lw$iKf+DLZ%F8XAXP0Dpd;;v zGF3`2x`~RCP)}$PRqcf~P)^!VS)^*8CN+jpU^#1v3JHMi>elGSu|JFaHThI9@3dM< zJxhXgc#ysSI_Y@$_Vf&_8cyB&n8uifn5JsFYldUZG|h&YDw!%-Dw#V>+y}0>{V8?B ziT(4SP2k0Iw;aK%LPr3x_G04D?<~0XLgHlNCV>O`tdJ&-oiqgy<;V_cy~D@@VrfPp z7?C6-{q!V1;2ab}3;n5iQH^v%VMr)3#lYfCn(2h0ARi^`v3YRy3Mc%ky=HLq+p7_%$=qXr?R&V)nU;wHtqh>ktnHIku+ zzUcl9f>$EtBPgR!jcFpQEOh*b7>*qcbT4!yCmQVLI=U3WgrDDU=>AaI+pcZwoZ^6ztcHDg z7B1WIrOegdiMIW5gRy?sLKaB?Vmz1!gqLMx<%|~=tHNY`10los4&%7;Ann00D7svv z$Mrkt1L~ls;tc8$DFuo*y{htkdJ`OJ0rz{+PG|z^DTbVi*xdX|bh(ZN3ym~etPZ)KH*o?hVQr*^)u}#)&U>A4YlG@jN2gV z+{frdv?5#p!OI`}sCl-In{JBKx-C?u4V8mP_O3*St~5W9q}bJL(XT<>-a^H~p=Ug( zKL~7)YevUb2b};4#RIB3x&PwWaP}zdMAfgFbVl&&MGAQa^4WhGc8W% zz%4O-AM%Vv4bIFJCOr7@P@W?nXBw z_8k9bPCML+OapLH{x8mYGDABZkr+xjbSPXRDcu%Xzo-yu{49Us$?X|SEXit`ncL!* zF6l7woq%f~=cq*{Dj5{X$?k&7X^QXm`gIEE_sp(Td)3?95tM?-3m1WzY=h2ZWc;O% zJsjI5Hsn;1>%a(<0&D7Oe!wYFZ`^Qn0F;n6Is*Hm3^|GXgeOoyJGPfVp`~ zd_cHtyjk7m0FK5K<2%eG3Z*w;w2Il4)o-8`L<+h_SGyIzYBYW^Im4W?D($5JHF1i?oI=X$ z2Od_FO0MO!n+6(Peb0_lAICOSX4U@)7n{V2HLHP7wXT0>&wse2%>RQ+%0hEq)7h8( z#lgPp|CdYJ2jr4+{KF-6&_6#bsp80h98_|UKJ>Z284I!5ho1Qq#LSB8#jGC@^{ayZ zK^nuf)fvCjy1)KGHPK#Wty9qg*WED<_GAHP)X?EArKcy#7KzSmru04VTBlpGaBxZ^fr*%DRL^H}cZ17iS#|Ahwj*cp8yG>!Pe(~kuTJ9>ofrfh0IT?%)R}6qNqDYVABy+|wE(3Fj?yke z2Y>u2m6=1^VgHYBqcn)3`^g0gOE3UE*OEXzB`2sUKSbm+q5nIB)ctT2>~YEGmaP)d z3yXb`w}Chmvr6>$Kz&mz;`uKIDboL#z5m5nfBny>z3c)35%{tglLiCIM#Cs8PRqkC zHLeU|MSDoTV+etP2m!~?!?vzjfpHVEsaXgo5>ND=O4r2M?^Lx3?#Sas;pG>{Kdd@C zHwC_&6&x8Z_l1Y3Az2f$dow-pv!vcIM~{5*00&8CA}}DRhd16Iz5kK{zq((qGlRy9 z**1ZPX89N| z+3#=oAZzK7?J8{fOB7qB8DOc5H_~xD~wsk(z2Y*3?!!(WN@ac-W97kQ8} zOM_AovOPbMXQ^_oDsi*RN;c$AT|wU#k7 z7V(^~r<9kH%G4_V1Tr@DyGhL?F(hqK%tK3D1$w^6dIda+#v5icNx<`>&bqzvQ=vYt zpH0$IW=k|M+_@QC{NwrIF0lcv4E*Mw0)PL{>IcRYc}hgu|K9hhQnUT*XXwiixJFJF z6l{c7*|5+^3DV}zIG|XuJ&_M$ioUcU;`^@Mkq(IqHAqavMzbIxJ@ zrgXAQ4(g|hvpRmT)_Qo(f0$|h_;G#6*aeC$@hx}(dy$j>pk=6b@|t?c9ADqwdnC^Z z8$TXUFs`({w8g~5&UN9W=|CPEwF@u4do*B=?}Hsi7FLlrEqpBwiqet%#V(WsEtDid z5Z@Rk+=tdLdwM#;8YY}TA(!a*2bH^C&+3z84|;XT&bs27s$w@(UMkW+2y8OY&Wwc; zqtZaWN3WwKHxvdUXDir$in&6NxFk1_S40)y0;=x`pLIO07wqS3_Pd6f5PAYg;?;TJ zo2!;((Ul{bVMt;xx==8aFUO`_;ml@f*pECINpm%TT;bXtf=6a3#!r{`6C@Il2=XQ zz0B}h|H#Q}L;8Co8Vf)q(NBi9feseaDXt3He66@lXgL8KbdOmW#q#>eNbFNvx-cQWKVJ$|EDlB4s{b zGOb5-C#|H1zu?&sq!v-T*+a*E>YW|8GE_H}q~q}`8RKH5YcOBNdhtoOuD=>S*?EHv zmyTEM9=u>vgHJb;OiAskUiP%pZ%b<`%a~o1<5%NHb-9cN*>{q@ z2YLR{E-#CV(aNZ59mNH=Gn`eoP|y90$)EUH@5r^kDK#G&q~6O(lZ%<1D4f9CqxjoVktmQ_3e; zrb>1!#i~={8-B+iTV{!U-1ZKji#cvvmKMnpc}QzGmaAG>%}^z~TRq4S|3QSnJk`6+ z&O%8f!qb2`@))eT%ow|Xrw(df-1&i`k0<8Go{785f=1h-#rY9W1dn@2N3mtI1cycD z;;;HX1!O&49rsYUfw&OK8Gt<-)`ZJG>FbuYS4-+H52wDY7}0LhW$Y55NATL~uFyTc zC_m;(*@39GQI&k5F#zjCp7AS>X85Hc71nA`s}ao&bV_#aEi=PqfxunlxzIb#FV~;S z@7SPy@FP?pV7}d?BSOw^L8pPSzW5H&7Vq1{!;u$Pj~QZn)a=5i``eo{zhj{De6E|% zt{50hBwU$O3D(7E+CykL1seH^&}8$@h1cC*|7mf2LmCeF0KSNZfvBDTbM|0P$vcE6 z{$K9DmZnbR|Ahp#RT_~86+-rzV22K%qp1!Yvk*l6>b{U#3iFjT3>#_9b5ML0e%+)v zBvilXYf8ZNnFNm4&rq1}}-0}UO7M(TiBP!K`F_jTpk_T zyW7l>;EmM zgU2cY+agm+h+O~Od$f&Rl?G!*4*tArIM+T`I)y=Bj&OzevoaP=EG$HYS$W6b9u*&N zHuj@QHAlJ!%zYc|NzteYLzK=>>Sp3CP2lz!2y@>8Z-=ykWJj>UTN)vOm=gunLX+CN zp)&XugRpnIB%GG3ZRafEEKMM^pEc(sF|K0kpAQ(!lYnsXL>uraQ>xp*MxH-e*E*sZ zy7$@FjsEKIrQ5hGdyptoJqfLh^%0AL-lkhuVR2Ts72GDAaKb_@w4WZ-nOX;efdUtE zU<)(eNd!T7dZn3GoDkOXl_~fvSzH(;5$ocGZcFtlL@eMp1UzZkQ!3Ztk5bT+GI{df zPiTOlmDNhqZ;dZqD4?%c9}Xs3sxLR}@#;0*>S%I&i{QiFNxM=fBVWTQDviYw@;lF` zoCFtr#a}Q63;59zoyW&LWQIT&ean&VtYLm};?aub8uGpR?3k`ZEuYJY8CK2MkuAAJTy@Of*+@_@zV^;$Jyls_z*^WFA(B80cNu$seL!H^Ib2sESI>NSAG9!E$wmw9`I zkRTrJla{hRaWx4rNRf@o-*HdLc{bc3Z(z-0CUz0scBa`f_{WPwy4`)%>r!Yat~&CY zyCBW04mA0S;??QKKfI7VaaJuBK2(|M-b6$o%8Q-m-Bebi^5(iE zNzsp;f_uj-HqECc(&YeY5r&f}d0jk5t-d$u?I18TDR^e7@9lV1Xb7lNQI~SD`JCG%y$ja>)&kB{P9*di=}=e^zUUkOyi$F z%FhXv$@xTMhR}JJvF|CG&kL>DaFVOra6D5gbjiG!Vrfo4%Vz*eQIJuRapI9A68aQ) zr0Elm?3^Y}4J-~39Tg2;LSU(6I#oz{1jaKjk!4P$UiYly2zf*Wnw%p!!!<6#LA^gO z-32wlUrBB&w;niGRkW})n@&oow9HeR*eCPOW~^L8(^O=ANEs8#*t8x&DZI%JI_YX3 z;QpS}%j;Q*+rRG7lK<)vr!;XP@&RHurTdtGmU+&AleXi7%#E;Iu_l;BPaYJ}t8yxq zYyR_mlfn^66{Q?$A(W0Q5N|4GTessNaC!O8+kLnA+xOdNh%Ia#26kO|U7IGwfYTnK z`T%xqg31CJChX{ZE}vY#+#CLu@IyLgH2)QQXPFv-$htsSTykk>oh@ldUVxmFh;fg9 z*-T_{Lhlmqc6^44|4VB+v&`?UUZx4NxO~=PBpFyCj-7bD1IzI{IX~j1R|zmsyK=`Y zCb&io-E?w~lMrx(HqFql{=NgLEE8p0Qy3m4s4ff$3R?D<=Hhexx_fmtP9}iI)B(!S zBo?9j@$ttp)f8`H`R^ zZQ9oN&w)D1TIP<#h&VUsZA7cp4f5gzGJq<|I$E}YzJ-;0I)^CG*9fshgGGYeG=JR)H-UT=B#`Kqb!e>LG4*=^p z(jQUDRf(sYZA_dZ+vb@;;17~G;*?KMNwa?(l7=hasPJeY{``Hd2*mCedeJXpN>olC1Yc zhJF9Zki!{LmUh61s0ibI8*VN3HfqczG zOO({YQU>9hw4}c!Dl|r!8nX5Zgw3_-tQ|*&jPCkbafV*7{`BV-<5fiyYZw`8)%$$f z2i5%=1xnbl3?5L4tPUS7_3(ma7^lik;l=d(4X^f=9lBM- z6dvneB3(Xb@6S{7W@dLy)ueyam~2?2^U4f%YP!mKg0p-p zeft`$`yw(*Sc%2ixe)tI@CP7yttuePnAKK+5Q-Fvzk9*1R`Gk?Y%8z|3zEQmEG3&e z#*i;r6kv%wL9MtjZu3Hv*RBO$bO(qD+gtg`ZdrPcZ$^j9!DmRXbi+8;3WK>Qi5s($M=e zwRv5b?y>WG+Q$&eFxKvJqto1`5W9ho<4U&Gvh1doPd_!LS*#w8!(-~&WULXX^cUE1 zW2hH})I@}t3imhd;T$l-X6*dgTiqD&Y8e4F8b%BmK#(b`8fuWEM{SdW&%MH~>X9aF zTRFl^VOS{$(>PW0g2hU5Wl^|kYX@VIEC!}nV@r#Ee&1tQe{L5eD#%-fFucac=K0rU z@8&;-cY<=U+r1;Qe}--F)9c34IiEww8|mk66d?N0f_hj>it@wm;DSJZV<>s%9O0G@ z`KhDy18PI*#6}xf4LpkZDce1?BtX_@Yk04RPSSH}^+Y}dC4xnkS+=XMQX=^g2pvM#K8Qle;useXV0 zuF&V^;@BtodiqfYX_nW7@e&BU-_A5LeWuCq^iJX_mY8?pyl1CckXgPDPUCrgE5Ik) z<{0d09VNbnRLMhvOWBw(PVN8=HRqNz@$KuswRitIj8mr{B&Of_&oDZ20@suqc*VRz z{LgoKN-Y&4Zp!8zJOSW8Zn-j+a2Q4ENR)#|c~Dnd{u)s*%-f=G(1Wi{bY@b^@#l-B zUD67agl`}pRJ#rKcPxyzK%?BW=jZnqNc-sDn4m;?F?vKwU!cB2**0K5c)gDZ^TK7r zy!YEB1j(L51k9ZZu(+p4(|QdcM~4gJRyg{wV9_M71|PTp_!HpQ!)9-abtz|gL&?dC zea!c?;=*N_z~?&H%J3`7x55`_yi7w+NOPnMM2j~T^@glx6IaTM_6iDX#0y<@Pa5h7 zVt0xp=@f3(934ZA^PKn?UFc30c1VIPW)0yH+FDRH3gB|;=tmJ@aPrV<-9Up8#eNkd zQq7XF_ZCFaPV>D7zFx;Tm!wCX|MQM^mJ@wN0Olyof8QMcOD7W;{s%$%mK}Y zGXgms^dv>7x5 ze@dVSVY{9AMhDKsuTA|fF!w+NsAhsShb^9#)7*1n`{}a$$cV-HN-f78`w_98bhuKl zmdFf3+?Jhf0O1e!mYCuY%xacIpDesMX1eB;-(VkpA@LQ4Pu$$dwZ|;zI_G@B=2aUr zR4j&tjWC-)ujJ^Vh!fa9`;$vTUb07})SlK33`nMijvIlld0Y*4jufanhzuY~wdomye@CS4q zm4)S5DnIDk6J(ynzlI;Y>`-BrNi`UiPRKvPvn7pAX9~7TnP6LE9BNS}!ZerlH@H)v zc^4*iNolagS^tt-4YT99F4^(cKY3DwZ<t@mkHJy@5WupTwZgEd zm!?`g0k{@fkes@vP-bIBc;__*y3nc2fAfT(QignoeEn^#$~|O%tlDL8jg|-hXC!$9 zjCmPSUH-O-szTJa>K3?0(*2-Nb47L(sRYA5XdeJ&tQvsko?|<}O4mG$*~9BuEzkew zKmKM%b;vs_z%RnzWAy(ib;14?RlwbL+!Ww)2YO1?I5Kofv=IU{KuJdi=#cwCS-ou1 zUNzY7X?q*0^@AtX5`oPSxLVTrRmPS;*faj#Lf1k0bu|0sbmMM;YOCnT#c z)2Vap?D28)-0sWe_OiR_Ze!Fpyi;~|n~o|Y+#eXG9rlX7PEeTHt)<3;11y+&9X*v1 zYIIDp-*iggBPJ$s0H2dSW$(rq=EdfyJt1Dt7Pr3W4)+tE_?cnTXN`o-*vS`ItYl=L zGWo`F*wnMkQXcv=mi$bbA)dt8Ya%-wMqd#;`f<#-mBW6p!;8)y3W>roMzl&6u?{{- z6-VO3ehS9)Zv_rp9^5%62kuB7z%NQYwoEZi>~g;JC^EVe0YnNUgKFXqz#MS|?mF~> zTf-a`$bmh7ZzJb>01r{XJUK6$L>g?=e7FIe0=AhuhcwtOls29t#YC!_JE4ZQ&qSic zNuHz>ZhK-O1Z+fSD=)|BZ#JDiSgU!<=_)a11*NLS=8vB~ZQ`b{SmhcujHOat36m^p z1__hgak2(@Ea0?CE-N|$N|u}5t#nKJqx(#Ne;S@O)S{K~z&JYmyVVa2qYwZHGUAh; zU3YGuW(T)u{T2w#Ue+JaKUEyZcK=>-pFC2u)KOGFqelh~&Lw5+zev5eA&e=u>Y=NC?N^ zU>@c3E6q>({U#BqNOvNy+>`dL9wz$SyqY&nvXD97Q!Qk?NvrR$f1Upo!e?y3tzw9s zt(#r?XE$A&(Pc_t5<=ixV-U%C2S_k(R^3j^$WOM|SexBJ{Ee)>f}R5n@Bk;#62M5;uBF2%$KRsK;7r6V z2N=POe@D>I9|<>QDg^=K|7~%^%cuZJ&qJRHf4bWwQ9*hn1a%D(N_Do&P0LgrVE2&2 z!LY1d33_{3>MtgX`|WnVUX`oyQz0drcg10iP44E_g&dcCC_Mi1GJ83^wA%!+DVV68 zbjUFdFitRz=-aoL@Gr3P%QMRh^?)ZinTAhON`OLYYv4U%#~oqO4uzROVIglVMB+s6TtXS{ww}}n=AkMNs)K}E@}REN=5=sWv&0(&+yIt znRW3<;-In6%WXJ;>|9L!0s|ujl7QJc^+N*aJYS!d_3h}F@-EqRz;*p!>uWKvJQEhy zG@zB;q3p%``{9tnm-eO1?#O^|P^XC5+@(D32L=VF##2~^OTL*fa zt=(o2wt6)hEA8%9)JzzYdbqQsvIj*{!!I0Zs5VCI4lw109aG%vR90EHLe#Iz#OTFC zeW|FJLr!th=9*}!vRYioJ)NDX@2sRHsjCZVVf}gIIMU`{B>;TEEDG^Ye0?`vjE6HB zn`Ws*U&RO$K9K++dF-?2uw))|0+BY=o9y*i#fw+Ngl05#%Cs~(Z9@!PGF9yTwJu4) z?9h!(Jtz>tXT6WDvnRI2-i|0?Z1--pB6$h*u&nj&kz7E(n1y^4s#(52 zHt)jQPjKZ^){ z!QG)%M-r^Irekd|pusJ%INE(i7%-yTn9HEUDPEqQlOEY6Q|Hg`ho^2|<}_8Aih%Zn zS`bdK2N=8>HMv_spk#jRcNDp~9DY35lEi-PGuOm0!Cq1g;u83e3G=hR07l4`7{=J@ zabn^F;Dc&M*lJ4MvC^svs^eSKBDzj}f=}RaljulKUe&8>%YVlF7_)o=NRlTz46zkB z=GiB}3?al1PH2Pw+z&FvHg)c|XJ3I)mjnNg(ZUYFPDibo}zo(VD<}-2j;$5KnHl*YJGnP(_usJ zSucD}4m*xGgIyl_%0n_O!HxI*FzR<_Fz~}GX_KVK%iBg!>e7-SrTDh1e7ZN>si4U1 z&+n9d7WIBIX5X$_F)D-Pt1=8;?P^l<9<>89HaIjTbeoqZnOtOs_KhdhZPSw}h2RFK z{gW?L8+00~%Pjn;08P%12B>7JKBCK@M)NR{rcMGVw6>YK@3tT41SN{p&c5PP{aA(a zv2T)c@VHqH;lz?JrdKtH^^&U0;Q_7s7N3w07x;Ptb*-$S4uD76`l#Et9ORrd*I_(ZJ(pdj#rW?SG`J}Qj znHy|$`}>K3$-GPUu*?JqC4ARHvxL{g7Mkn@AEX~QSn8#_fiDQk*V%PjgJ~`2HtnJW z{oMzMr9o%JNrnR$i#6k1#q#V2C&%T_FIn=Uvi?Jvih@(wqtn}D^v|yf| zgTh;GSOg-?*V8m8y1}Z2}ArI*+Y~Vo1-O&1!6`NnJ9iywEa+r!S7joXfJ;nCN{&r>XKVZX&%OAO_X4 zf9G8yyeo(@MaE4u1gx4ITM5Ls5=nWUV;tNHl&MrPq<;LP$@wwu=V1>_6}X^Z{v{zm zvIB|}fAS#00-`iw1kn#MK73sbT~ma>JCGGc=S|%Vf1`qg!H{An_Jy_yB^=f8bW@HY zu9F$8HdtnC9Q(vKS!NVBhS(;F>WUN;hcsEvvGL8mO8#Q|eMxDl{IYR&23p@}7-bLi0xq{^>zn11v z-<9WC1Z=6fRu1U7R`yk#m-bm}dZ!Ky;n!@>Aue4rU|yPDD`673b^8?{qVELI@gM(; ze8s2g4FrR`R)&&h8r)e&Oxyu3TaWDcVh(9#+ULUCnO!4d5;-CdH*?n{1|;u-(edvP zx7gVuuT60(hDkxG@A%MdOznKfe5`+}3_!fa1VCB2kKuMC4@f~}ou4CeubvV(S8~+3 zCxFQZs}H%lv}6o;X?lkb9OBzw`)8oIyQ@y+gX|)@GpmPExa*OWUp)&~8q~r*);x)l zSUJcJ&T;nb$k25T?&z?51PRnUkwpm9KCwo0)jZKgbk#mVMF2+K5dkUJ0^Vwa@ba}! z0MdwvI^?Htx|=H(EzK(gEzRphx*Lb|fhkj^pCP6sakVwOLR#oMC|c;-BwBRc1$9(| zyLFnwbEw`5m_<*WbfZ@UT6Db$b(*qxVHEDdm}A$ebTsbTVTYzl1BA4imAzOdN_}nj z?t++jO!N^^_@-!+qegJA>~(f%owN~VfH!eWj;&0(2gg$WH(AWrxYz*@*^%C%9m-79 z8qQO0h|x?Jy=Izb+Kt=;+qqsP`obR=nu)W)iC3}BZd>XLL}QAN1Ef7y$YrOJ1oyNg z1E+t4Nmvl93@>AG)yagqn&n{ZVxUC)qC?q*n=0y;Atl6jm`#ZbJ=^`2HbQF%0Mq+I zeyHZ_d2IdBxzST6HoFuR_4UQP!++{Lvcdx%=OmZdL_DL5i_(dr{K9{2i@+(A=#*x>%FU&)942Hd@w3S1qFJVs$~t=|<6Tm458tFs0pC_^SlaHS ze)ZGC<;?ydtj~bS4bx&UKX`U4H?)%1VQt?Tq#G<=Oh2CY@hH*FVmJlS$!eYWbw^?7 zLMXct5jUxg z#tn>C#z5zL*MVfWTbZ3-A}2l#?<4ufi;gF+uv-8^Ax?buGXE=za9?w#6gMm`3V?Vi zby;#5tvmJqczkn z9ATgkW~Q;2Cx{e(&cd`9H5YBhwrxj?rSt)^YGNtp~b+jS0?k*e8!O6d~!de^$hDPI=I>|Nd5{Vd~SROvZA2 z%$h3_5dwC;3-DFz%67&um*YNWL8Q)G99HxeMHo83re~{n&Zc=DO#8)+8&yVDwu>iP zJX`W)iG+g1NddmW(u-abw7~qkBsK}t5=l(WXBrJDs1#MZfu!p(?AMLC_QjRu6gQ!66RIrGv#dg7z0`k>QG{;L)&Fm)#=Vnc4 zLM=zmFaWcHcq(MJu=UfJbIR~v;dUGn=}CLy7>qKilLwU%y4f+6#1k?xc2y;P1Lkk= z3X?a98$HeC?yC@-+m<=t^HRNbF~{mCGJy(%Av*_1(5?b4 z9KPomwy3r_Y~C`3vwWCMHh5Q+QeIb?+~%%R(n2X4PUf&E{^qe8Ch6ko`0fuhlh}~W z1YN-cL6l0i^1w)QOS1rXkUCL?_M;(V@vT0Sk)}q5h>G^2Ue=)XN5k3@BiJmMv(%pj zy#UX`{iSi%!lU#k{oGypxm|S!6AB&!$#kQ4x%ij-S2ilB8rtn zpgM|FWw)73->=Z4GnqWp;}1KXqsk+3_t7ewb}3bict3l`nSMtwVNBQe$;BaM^#IO2 zj+$C{C!Qx3ZPgMi?vf6_ZgnA5B`UU*Vw z;)2j$dDDi1uI|^GCkPxgyi>^Z5-$vVP6Yrw35}{!AZC>MNnY0267X($_U->6>nyn9 z+SV?e;8M7|ySqbhcXxMprviZr7Tn$4-GfVj;O-DSI0T2ToO{2~{q-4x`UC8>*M4Qr zXKsyB6XhiCFda`v@1N73v+6Geb6&YU2ZU+j_nLVRZJekmJbbR1I3=?o8H>WdCG-K!}#-2WPGMn&?c$e<_)4q-34PJjbT}xOx#orq9C`mnnT2+)i zEs0tX&$d9}Hcid(CEx2fEbmO3eg?keN4MOQwq&er8nIVT*sVHnpA|N)M>}U4={1b+ zD&=ih^PM*mD;vo@=?lhU33xcLdqW7oCg0wb&TZjjxMuUHnR;~)lk%q=P}Nx@Y!rKt zd1d!&;h5)saxJF{KEsGRYS3dUofgbYwyK*XoQ0)eNzqM@Gk@vL^RA(Z9tu$Beq z1F;ERZ}*3L$O)J<#5zXQ+v|C~UC^;&ih=5e8UQ^nEAm zRPBMpH#~wV{6XjZM5Yq{#+NOunGBPSMrL~w_5?ib>c=xFT137_fL66iMbLg$rmH9J zcZC>9yNElK!8(oS%B5yz|MlA!Mm04*^5iCEyh!~HCS0p0%$hs399Pg)iH6xyHg$j3 zuB_C;&3igH2HDnHt)40H(PsSm*5a$T`xB7#!mF_rF!vhU$9ivFZI}uUsi5!JjcU~J zc*O*r6gRa|MgK|z<|W(ncVCVGU~oiZj3nKW|p?ZQNW1g$Da>QP;Y z%!4YD>PIg{WMbJt*2o~8N{-V@@2EB=@=vzg>dlAHyr7sJl%Qd*bAY58>g>(W{ln_5 zzYlgRK!mm#+>UPjdvX=;3J$?ORa=w7d`pR1!gi~J=5`>JO>Bz@=4Iw2D<3O{Pv!j9 z-h`?)hnPHs57qE1o{IX`_6g3hluIGq_B!h4E!R_L?X-usSA@Dj9jE)B^*;iwhR3nr zfv-rtP+6=-q4n6qJf;BzAf+p+A#F@!bxb-yBocy>YQb{Ra=~)Ra(*3<_6MdH6q^P4 zw2PrATp|MPECM;MYTmXuG`$p6JDsdn0t%)CTS=bw(~yg}DB)g$f{dGd!WzO;oR%x> z4SfzN%~+r;vCOr~Q0arr5X3b-*z}5(3yv;q_?2U>HAG*c84w(+33g1I5yfhzrBT2Y z6K?G>ivs;9zgJ(J=*k0Ic^adJ9G!gBp|ry~?2b}$xCqD5`RD2zt=fP4`_1gGbm=A= zWgo{ce6B$=kk1{oh0nIxr!}c*oo@XTu$W@qX_GqlP3|y2pv*nsWNQ_NK@FSFSoY9Z zE6S+lh}Xz#wO;<94gIE0~Y?h_B7nobM}+!LBDyimn^}WYlXF2VpI=B0fMa0%{FV3AEHUQ2jn}P zedqMom&>2_=|!D4n;t`qncci44-{SM>BP~7{b5XznAn$LBnfB;#6}8 zPMW3dWP^3>ovABsKOoqH3M32iT05Y(8d3=7ursZV@dbIk3Ygx7YRM~#aoWR`vLm$- z5-z;G+D|hG1l4a9?J4%IBPq;AzAW4ldb5~AO``hTtxTQ*bLzQHJ7Q!s9EFdd^b8FS zYRdHwyzjiqCMkg0T~r-G!@2^0TmsiP#s#0tbnXl`E@Nsl%dE6p1A27GafKa+p>UM4 z>)PZ>SJmk@u=Tc-HmF8hjyuM>OFRAP;w@hWk^0&mR_{adU@1$+Vw)bKJO+1l`h@ad zp-6$`(Wdpln%J0bsrQ#Dnaxp8ENq?BMb&}lFoUD;b?bb5kg@of>X;#BQ2Qm>JVYU5 zS8TpmO{ZyW-FZiMoI2+<20E94@TZD6_1H@sb;w%x_Of3PN$wxzlClw1{P6!6EzZ@A zurro;dM>&Y$IZoD3eDNIv98dh_HMG=(>x)w_C7%Z!`qr@bh=EHt#s5*kxy_1`w_X9 zM-ye4FL_BUdY2zfo<9u;55k;yRo{HjMnPk$gF98Ew8+aBzs36FS`AZwIE(T%9CLYF zc)9ZPTnyh`8ltQcH}Mn{Qp7N%{4TWIk)QbaXYfv*KB(V^B4{SCbh&`3xk(lqEq7Zr zaC;Y3)FT{6CgwFJM_fg9e7ka{-Z+kMW5i#iWug&70&CaIB;X0IzmST9yrGQ zOIMv5FNe&NN=kS`akrK>;&V#+vcGuO6s{PeJIds5>ie%U%$q~t6$EJeJym{);i)H@_?#5P;Nc*ab?AWU7en%0CbZlc{s3o!4T>wn zhxqBQZ%f-Upg-d78bA6dG2<^=)rj=4PLW+{WKCX4W2(0_BR0?=Ax_Od(QCYwH8teo zQgiP?j}-_al^PWC#gAI+0O=Da9$JPSU4`lrv)u zSHgP;{vE~H0Gv18I#sEF6$Y)U9Lp9;6#WD}rhYAuZPpx_eJ7+`44>X?bX>pVKRC#t z;Vduz14E5$nQ&|2G*MIcDs#9YyP0{PlVkd;ZzjJS$My%bUb92Ptq=mog2QT&;FO5G z)Ca`sKh8Ohr|n19aq}>gjz*L2~$f3rQ_xM#X=HZ zCb~cQ>)>@9IiA~R!=L6HlwY#&#_gm8afGK2({n zfC2cruIT~V) z)oKN>=9l!aSCZ2+0YJpQFRTsoui;65CoKB6RQH(5Ys|YGTM~D&mRDxqt9i7cKb9Hp zI+t}qJ|QhrY=aZ+90uC(phid#OOs%cZVt;Y;;ezi%`KCyQi2g`c?#dW>zM+XgojAe zz9L6vbTm&&?Nn_+_VQoBwyo%u(dTT@-vfc(jOtWop0CtR4_Euyb=K-=D_<0(AN9Tr zvJ+*zMcCSnv!<_IyX(JIG8zBDn%P%&!b=fh>zS6ywO*wuw&IC+g~6v)V+TMcZT0Fv z&y}h0$QmZsed6u#L$N-^E}m=m2Z*4H%c?M;LC9;+P2Fev_GdBGteYMkWX^201E@)j z{_~W+eZBY$eoAcr{%Fh30H}eQ^6qn5m~WWlDUO(j@YV1;SgX`tyVD{oh2Birp zwBDzBENK&YsT5n2x%9UPr4+-Yi6&i@5@ZE|xe*W0PmeXtciCf)lN|SA-iT^bRsY)! z|EWihyX>zuLGSm^Bp(R}+;9)EFflQ)j=(v>X$Xwan$Vum_m~0rqPW0XNmS8<9oB(U zzzFW9g-@m_c?gjYS0r;6y5xroNwPiE0W^?zLDM+x0PaAiY6nFh>(4kd-XE7R1Ns14 zEWRwOG%dC$d3t7w+Fkqx`kFmRtlVwJ0cm=tT}((-r=2VWBpO`i3filx!A2^uk4e+C zZqj7UFlXgKMG^@Q3mM2lWRQf4OEfNrOGq@nv3PD<-e8rY<~zbYF&i3c%D#l4rz?+D z<)+KR3$iR}h;g!yu~cqrWm>`}q~NmEZ(hZ!Hdvw=%+ZE2N}T55_uAN4ty4RUW`Fi> zaabd1wdd^Alv?z-O>l}LXyLQMaZ1ECy=b}?Z(Zp=)19>z{sVlszM9!ek7()D6@4#D zo#tDkU({u`1Qrb(rDk6#Ek@a4@H?e}+||S`6`C_9W#VY63z*5W*z8utbXnxW51Tr4 zbuyYJBnvf432dV%%#`7U_y*>!Pm-ZGdImib&ezCtU^<=nr6iU2eLd@)j0%SG;d{nP zMUrkB)5qJ(#ewJrzLicm9!jyH=Us(Z=UGb?n~?te_}0Cxj^sS4O{)n6hQCV8+mR!i z^3v74PxW-gs#F&3nkx<%6`2HDhGEELTJ1484$VwDMP-U9PXd77C{rCOnP+h~1C~b@ zjhIfx8$GSPuBtS7qMFP1LyYzVW(lTeX3+iC*1R%pDS)44BsnHzwUk1J)+;n*vybCM zxtkGeRVtl$vg(lzg=f0xcR@#{?$#S3$oNhN>h;lze(L8q*v&Qi~I7lmUwJ*YE9GCTF;@{>*to|V-+C-u6zori- zeLRtLybVMW6-}xM5{5Vqzc?IFijLt)>57jLXPX_Plj4`R{aq-MofpR{UTKO?h5seu zV)`bW8l4#x6zZB$24ZR6iamKygMBHZGPbKFNmlYAvCAb%?!geAPa{bdW)_}z5;;N2 zx&m{9_m(90dItIb2oIZ_pExhG;NCsSUE zR)MoP;+NEZg7q*Y@{2S)4HxPAnM=F4BTdLU@DcP1pa8(){SmT@^41#^2z9~O`$ejb z!3IR;gBabFCLiZLju70I#P9`p3SVLRzl6Nw9=6O!nIljGMF&}4!nAT$y*JOKzpwoO z3IvWSk5(g*&GJxwA@9$P_@({uWuc0cF;}OUn77^<`a3mH$^vxbo>tSO&|UCzWZycx zdUw2yvBNUVUFQ7Dao#C10yxM$&qZWgFcCaGCUV;#+hHCQ?+?tj7d?Nq9r9%8&yHZ| z56n81JcmSaO*)mH>prd<2Ou7t`{=Mfw-5g!4uX?33M0%DPFaG;3?wWR-o4a(Lh-pX z6nEYdi7TT1Lq=;SpGP_xhWv(99AOCOxLm*EbIzZXQn>#(eizCk#H5{QbzRaen`{~B zkl(ynhHMnwU0TdsV~>2{@A*?j!xVo6V+_U*eGf^lcLxRr-L*Y_g^pSMrBpSv%1zCAvl@_g15)O1xI%7}cXC#Ll&+l~+MgjU9MD4K~qwJma01Ve6Q z;dy|R>l2^)bYSR*IhqI)S*doku-(&AR3-{Siy2euY4fwf$Wj^5>CuVJsiWwva1n~3 zZ;EtLqB{fI}4@Ugm zFRO^VyA}CM1Kv{)1q29w4k0YC={N#12)6}AH>QK*mjZKEcAcvC%Tfc^z^x|ig-ff< zVyWMY3`OceGX4Jk`Hhf!rOXO4RyMV-{29VMh4>SeYHK@-e0m1XQ%mDyai;L;gANcK z(={J8C+}Ml4m^i2{5Y&*@uHWJdw^5>i@JY~SgdQA_qOElag&xnj3-g7X`4GE=%rZp zIHBY?L7nGGX_P<#h^M%@MXR-IGgyx)8OG_&s#xKx_e~{m60YnWK>z)fTNG?HBo5sBxazYMDcP!1yrt|E>AK z6PbU|-K*+I_&`O6xe`?{qf1gy!(WD>?MU;A+y}wW=B`I}sewvSt^tcgUKCC6;e&7z zuPtNLdXvz%E?$y__OC}n#sJ5Vv{_|6HpV>8z#CRtgt>K1R{ zhc9~q;It=le?I_!)FUkskPS>~f*lh?+7J5ynRU7%W8>QfXI>T-Kg#Y|X_3J;|9!Os zfjTKRU*mI0j;hD0Fd`>Pg2atwPTzIQt076G2P0RE&1aaS*QWIVKTdNDq-zJY#iJ(q zlz67QTtyW>(ytvw-Cd{xp)IOR)7cPQhJP?DZbg+oo~tT;soD_nDH&v;6?yCD8O3)I z`N!w6BA;E&4X&-8z`y@THv4zd6+er_1LXNHz7YJXgLQgADJePK#McD{WnFMIM*7=~ zWpI)J@_;J@$})J8))76NTJO};oPzIi-^7w69<6p`*?+1+;KNg0jXQXG9i$(okNuo7 zHvaxGZBQJE3P~OR+@`H^CkJGU)QJBB|A13)nl)Ea;HiQ4s}T_()w5`(7MOTZ5u@NX zNi8*kMZ{+?MHXfc1Ea#p8c7h*ZfgR7hHPcCQ4yW%4Fuq&^}&e`1I(z%L0!q-_{{fh zXD~T522_w+HSop~v&}f|@3bkK2X2`2;{Xbpp&x!iT3sjbYw<{us&owRS*G)DEf4li z{_C|Xl?zOf%!luuY)RGPz={23kxz`t%BD?>LIsBK0X=kvKfK;1g$uJV#sK<1y$UOS z+o96>vt1gG>JeT+S$va5LQBT5P3Jq7=bDN$0&qi%Z@~v+z*6G!6CFn-w;5s0%oWLM z6lfwU?y@b1z=DvPQs3AjOH{!vcKHu6n#e44c96xxrWG4(6A za>jnJIQMZI%aT*b{P%gz(HFb@(zpUgPa*PjzbRgGW^Ksg6y-itTG&PrLMW@xaP#{| ziNvizQk30nAr4I)(f%%Y78mor3uY;KY4rY6J>8}>}`QI;or7KuJ z(vt*Ey!3|!bV{ejaT6rH zDV@efxNFT`=s~g{d?Z*iQSoA|skWNs9i4tBQ(Mo*AGj}9`e0$BkW&a51s;yJ)zP4R zm~p8MqRrN+9)<@vsf8xYe-8L3300Xynii?O_FWdw^6mi-1;g7={w7JwColRsL>AW(f_Ik zxr@*zn{W`^M+eg8i-#rAS4rnrVH_y}@Q((*jZW3gTH_ITVD>QTY31h{tokEQ~|W7XA2<1J}|&h)=xosv((vM1q3H>`U#Cb?RTNBj6!`d7*hc;N4@L z+Y$FcxsE2-b8C*3TlQcO2Fxd~CdZI~tA&~YAv!RJZIA4uVn&Ce_WHmv@s3y} zP3pkW8(H4qXWE4S$+5;joY{=Rw{i)a6u>`lksmPZsw9hP_^0&7Mxa`rh-p&w#6gzh{zO2!3*CSPQ}HM3kPkTH`nJbaxW^EZ zDE|-ldE~ml+Lx(2hz}D(m@$TzZ(+cy%k`bl-s2aqM!?p zsL(aWwjq~la6qg8`}-QzRZJVjwU*2#gwES^wyH;#aj9SnSz>tzd|*BS^bNzYr1Pwb z_6Y}_ZCgI|eEQx9IfHkCB~N;0^SD+E=l8^TN#?PL?9CmtBmO{$qbA360f<8fnpy%$ zibgt5cmLFmLm45{!$dCD65Xo%`xNtzqN0J!pE$o#n&3+2++VirE8-6G(GPefpCjDG zifgCTXW4Xt-O*oIY=Jmfhh;rN=T53tsY*1Ot>46Ssofzc)^)P&Ygxt%4B2f1y>ER) zGNlOv)!LOvkb{HgY6^V}^5sjyYD%=A^7A2uB_7c81z7rV0N)(pZZ;r-2{;K#RHt#n zUq4@(KcTkCwiF*ss2&M*FcOG&{>cojF`q>wx)pmxGv=6hP6J*-z;=By&coZrukmxu z^w@v0>EeHYd&v@g-5HhjV|jzJ46KXHfz0xaJU31=!-^lNhXimyZo&T&uz21U8PwGy zT_VGhE7QR1s2g~I>qx!p8ex7A;$Je;1aM*8e5@g2KurWhzk^z={IGLRjjCyeZ6sKS z=znUl^%gO}fq=;(_tWTFZ{*r}k#DahFP%FA5+XEcUz@9hMZfdB@mxV%f0-Pm$w_<% zeS+X*9vS{F8G@xTT`}>SY&#dP!o=@IyNQW?3Er=k;S~5u|5Q6jdW0C}DH-xs6V>!1 z#%&Oi=E|4dTGG7V&#@2QDmZ+#QoxOz7+k%9_j!qE(rx9tJVi+Xo}yRhvULJKCj^r5 z^uDB#dvsHFJ!^*-`2PBQnxjY|$qR2Ja_9qlsLkhp#;!SYZ@%>4@4(BynXTYrg(fv_ z9?anGc}7MB{`Gog$!Fsd%$3k#{?H~%+odd~z6^mAqYoBtXDYy)A8B-l*KYUvg#z_L z|0$1@K418z5Y=Ci6bd2P)4$_x;{2ywPvHACR<9%DX`QFRQhm`L2w7TMQQA;?;1gOY zQr4#-DDjn05FQ5AL`G6(qYJUM25%)e=!!w7No5Gw6=vTy!MZ=6!?Et!W%w4^$?mlV zMtq)sE!Z#qh|=DfA}vb2SGH1tBuW0aG|*98{l8 zuhLe5faERIwen~`ZmPSxN$Iss^US{W$Qw#jdo@jl>mZSU0h93PwbtJy8y0Er<*%lj zXOR*fAOVxK90LSTd?$B#%7m}iXEwd2>|=!r=kVSO%8jA0E49fSyzeR|Yk~#d#4CZt zl9LVB#`Wg?`G9BSfnv74z4_~XsBc}jhfdQ#%39lB4RriimZS|<`GWF6{L<-&w|d3z zk{$MU!yge&NlY2H=D3A>1L{#)8uO4`+(?SEGa<8{z+{%A3pj;B6~-B)B)SAu>HP}Y zY1p*8VH(`Kk5%{5+qD&JE?55;h1RjQu>WR4G5&AE7r1f-6C4s}k%WL6XG(uP1?sC# zog1@i9}SaH>ct+b^1`tt>LF$@De1_LSvU-FbacEj&zMC?2Ba_HIaOH`;J&k7rAPZ% z>o>>4tdLkAZ2BKebbDJLJWsJ0e^fAG3%P=$CilVZs|vM+=RhZx^c@chg;%DU zkk<((N7j^r)3%gpqXtq$oZ;lUC-JIp}&d8_YXd9l+3=XD# zJDBKEFs;?|YP4MQYV=*x$=drMvGPbGDA2>H=-$O3o&N`*=6DaB;~^#**67 zZ$CgKIHvV8UB@KcL5fn^u9M&kOP$V2Q%bq@%{JgGK&$ly8qn?c)A&?^?p=>@-IUN9 zz03~CzlNu1$bKUv)%+25a#e<3F|x0!sZ<|mz`UP?!B{4jjfTUKJ9EM?xS(ZBrWH#w zufvpM_Xvzu%0<`2knjb9;~(f@Z3M2ZP1!K+Nab-${5~JH$)|~KV_uZqf|Ltj5lf-e zz9`yDg&Up@V$wm|{3d(@Et82ALsG_qKq6#p9-m`S6us>}s*iTMJ*t` z3m%u{AN-U-H&(R#T}=f6A?Py+)#iK%3WC%5gb=vAVm^G`+ehkU)>33l*Ws>LlSXR9 z##KPXS0He&bKtD+t=n&gNFNctxH(dK7&AFarI&r>PzCQSf`QTI@g&x+2@!#yjHM1M zR^$Xm)vti3_}>U1sZVn^4LFf6318{RGW!D2=tYv$9CMaAJGzk7^ejiU> zW+u;ZONb78lBE%Qs&stgd-7nlSbhLUd!1!o?;mvtCZ>0OB|?Z$bty8G6c?Fn?Fb&GbA7Yh|9osCG3)j5be!jdcYKsNeM~!V z&hV93pE)!Kf)$paI%};7SuoTSy3DQ}x~d0kz6czoXd8CusqEl%A}};0pK=#h0liII z5<8uAUCk916zSiVzbov4g~D28hR>rZI9L`bjjMVEwMU<09-Z&)AozX2yWP~i)M8?7 z{r-lT1;H7Ve>1TuXDXeR@T1WjiGTQ}XUx$lt;P{!-i>IWLc=Zr$HBBpR{0X4`JT&R z%LJFNTb3zBJ)<0D2MA|dReKO2Y7eN89p`paswNjC27EfI8J{LVI?)Fdrp~uAhf56B z>yFp|A{4*r(#ND+DzKE_iP#n_iW5`s|uc_`sDe%;iY7sVCsArmtaA|`V3JVd)c1ORr&GHD*R z#|Nyc&76h5nC5H4yBA~P-f|XTVy0a`1)?daC}1#z^N6oN@sCv#d2oGr4JTA*6FboQ zY~J9V7y`wl14je7G=fX55c-C(5-Vf-)kXx)8oA9JPOEnVie`@Ib8Gh3dzB)BBIMkf zf#*ifK>0tA^NMzMuhBK)9&ev+G1M;J(J94k=-Ud**zt0%{`_sH;a>GAeH+1-#>Kz8 z>3_yO|Gg-(we9r@me7KFn-@z>5gDKHGt!iAeOH6dzuV67Ni*nvMsRy%70B?N5^z75 z$TEC;%Wd8K$O=M&E(s~c3{dN9#R|{{_T6K46>J|O1m$n1A_Nt0FCi>i1Sk$KS-$8D zD_gv13{P3SNDVt%z9~R*Z)}Ps(>>rSw>>hx^Zumi;X}DhEqDXdPKq3^CJ4inGo4>X-t7KICz;d`n zHzIf>G@irEoX{2<;BJZQTKEd65w|m=!?xtreQk2C$C@w7q9*FklpJof*2AzPMW)bF z?q~_pTDb2)h$2ApG`epoEBqXw!gcUi(M#^U+RVqYPptp*QR)Zxmm(nZ0i*xst&OjL z^M2CV82z7=BZ3;n4Nj-=iBXO{Bs8w3l;55A))e`DbA1!VjS%}lkaxv6_c)v z^xP!!?aZQv)_r;tzA(VjZYv*MyAM5gjmf0M=Ch+3cz&B7{M z>4YYzh@Ah-MAu2Cz(Yf$;-q36$@;-ms^AQQ{gq1ck@0)l^{nm8h$3VDbsQMB`+45`Zqmch?W++8699zOrhv9uxxduCBbXKBK zg@mhwJ9LO$pW^ke1^Fcm3^e*Tk00MpnB-JSH?mT>A3w|6Wc~Se%Vd|VM%TLW0bQ>$ zt0#1Nuh1YGjYY%}Xl>LltmfdyM2I>ynhx#ma7)HPB`U8Qvr~;p5a)7C#Ss`K;>$!8 zI^=FeCBC!J)S2ZacFtsprrIl^<2{?2}amrHXQ0q8aT%U(5i(m-`3KlfHJy@+${+7A;PyVUzxkWEKn5l{ zo7m7Ai42{1a>e#(cC~&+9P=5lIx?;@#j@s}#!qbrdZqNbS`rb-Qgc`$l_p*V28&Gm z!FnBQtN7}v$ZC_Bf0&LP zDcCfGO~0yj^=4tP1@L4yy!05>Z>RQImB`Xsq~2|Udh=RT`*E9RRLc{oAiz@$Z1_ov zG0fEmbdh0Y^^#%rfO;FKDjb|Hz8I(APaPR+e`=k{sG8e9M&iU`)#`JS_wC5jl8GYE9Jfd z2w)|-CfP%Z&nI<{e?*#)2Tav{2;EbUnc@{bd)hdu&P)}Q#XN|N%kCogfYUlv>*K232rOx;IiI@*CowOd-+5i&X@ z2Awa=VQ~CdE`p_~^0}DSn6yV);kDvqJ*8M?d{mOPA^7F`QH2O6m!TnkgsRh@2N2G# zCiKdDj3R@uAjH)Mi+E|mm?aG)VSpga!{RnlNj(I8N$iwJ_~?qZP;CtZP^B37E2NKZW((|7cnUpa#czu`d2dXd)Y^KVOgmF;v z;V$Al2&I@vh+Sh?6UJkjyOsDByVHp(R`Pabrj8la`7l|l3c*zm^e`q2{6ekfN*dOy zxZHeGADF2j{lac@m=$`&T_k-9V<_Yr%nFUd0DATV;V$cT6bC>|)PeeKl+Lg zTrbmj(XGoO37j(i0Qr0Ks3R-FwLTj88GSqFDffzHSpS}c@j!z_HFUX%$f-;JS6JSwFK*L1OYMVpXP3vi;VG4|Ni&&c> z2gzfh**||7K%CWQCzV*=4z^84N67w2qG;(W4&B2rpfx-puDZDMN3c!keNbP4a*>p7^M_#f9SYfb7t70 zfr#-~>KB2$^ob1Mk^8*)(W2{pl4zc34S!s>j~g_N5ErIV4WU-@E6Qg;BVpHr&f1CQU9dxo7(iI(j+vtm;phSI2ig66Z?l8;e>>8|^bAvrpu6 zV{h_&2~_uSz|Xh+o&MDgR(}A?dzv?lsj&Lq=DxaBA9(R9nxY9F68kU4Cax=Wk) z;>h=9aV;uS6CHPcS$5Xbn}dqikO!ZHv!CqvS(_ zWI!QGk72ySt6%&(u4m5Q?RL~9;fa~W{+rGPW5%Hh3F4egrj!dYlDut8e9>aVCkkmd zFGU|10iZ5~ri{@oq4MiLgE~>(UUeSukizNTni5t<@CFY&p#Y@D{RB_~|C7uhE&}u8 znvPUcSc0J?pc*``o2QjAn@t%+fWaUsIn#k{>GbTB9|ZQZY-{?KMW9F-b6yS60HeXV zOD%b2Q1?5(*Ey`u+waeR9+H58Fll{?F_nakgiE{!yfyCY+cNGG?u<>Ok#kHoQa}K} z+6-=|3iJ`_3)|`$Fe12+M=C)W$rVzZ*O@ff9Y%`H!2}?SLO#{=ktQZ^Ghh704<}JQ2lO5zc;W#6GW zQ&L2k?Jt*9Xap!@GTK^cDCV9Dov)jNjzSuey`#vwj*=*W;@VPJ20KpH*$K{LS!Jk8 z@v1EokI5k(Bqv$moi*PR)cEMmK6ayQ|uapXC)x)iUhc9*kgdn;RrpyBR$f=_8mX>lu?Tlm-_7eK^Z+Jx1 z-Nt4|PSI2>hE$Aq4XSsSz@DFD2tt#_2urO(n(+tOVw3KYGx~qdK8Z}Sy397=Sp_-s zfYM~(GC6y);nY~hDw^bHnVo87wObR@O5WSUoIbt+@jdc0_&ZI^LJOiW14S`55KP^{7Zd$GE<~nMXHUR#KM()yE>9|UxAq7t_+YwDt< z5j$8iuqbM({Ee^OE=c>=>tkZ^LogYVtI$w5+e-OX2<^_+D~6IHogt_EFl^_i*o3^} zf&{g7K^>p+S>AlPrqtZ>A9FGX^5}Z_vk3B05;CqpczVv!ZbsP|g^e&=co${NJyKlXYToikTsj?T%~Mdihg`;akYr48W*-VG5jrQZjjjupx^XSv#Y3VN*!zx2vzy z*Prt++qwDIz0*zua&V0@;)1qi57P*0tAlUTnOZdtE;BN+;&*{&X2O?@t_=*Ea64`# z6WZ@NDS3Zlje|?=yk}J+7WCn9q^pe)bl3Z`%qmVW%^s#XEQZ^B@rfeK3NaDw5X5B* zBzovc+bZq&3py(1geFG%?bdCnul6fyR0a{8hB{JzeAH~~@{37tcBdo;e*HtvcXo8~^suC4`yax+rP`Vkcv&b!kl5zA!7WR+zv`wNnMhi7 z*)NWYPWqH&G*3$T)Z)dn-LTS3*lG8yN%~Qe^rN#37%Ip8O+wt*iqraVBB%Z3?dDG4 z^H(^7s*bkf-IS;`=JPfkcUNa@<^!Rmx}(0nkgp9uE-a0}d6&r%_7h=VeBpDmi2Q!V z`0?mI_>5XyJ5gt2!f;l$9lrvL*7or_?!KgH=l*IN<6+-cUt%*I1i!C^v?vtjZUv{^ zHB2jTUQ<>g98$9RblumyGqP`?JN@pBGu-NpCIY|QJV9;7 zYCU6E>CvP;n|x2--6TEz{D^5pB4YITPiyxR2tV?dKvoMr5dY70@(&9Ve4o&#I_v}B zfa?0r=wJe52{dY+!NOF)NqxThLi{34a8EQu`+7eXPXE8U`q9` ztrB|H;Z@BvZP61j@zFgCMfPrQUWzTcXcqV~9mBYGPhmfDF;ugREE%fN+#U`=rzFWjSNQRHA;RUW`oig-cK}68`d9jwaI(9>3wRhI+cn&Mc1DjPAdS}^ zhCUQa+nJi!l^Ox)F^UpbF4dR4>LrR$yykQyn4+aPgMH~++l|%N`w{iIt}UL72+u9G zhDq^ysxK-;DMNV|a?(uP`pLZA;!qu4T z4t*^f4nNDh>?XWBPkx)C z*zB+$-wnXXlsM&hcpB=@9A3? zm0rt8F@{u6dr>h&qzP+@BW1zKD%5;|m@}1Arjwe`I90p_64ksWF0Zx#*fPES(aou( zALh1HZg`c&ocZlT#lZRuH+%6E^rYcGNpWS0)*d6%3VXgxPVGvoV)ag}^2G~ui8U12 z{v!)Srl}mnbAQegJzGNEVPjvk@^|_5W{BKWiK1nHPI#bIui(-X@ue z_!*!-;g3;dCyd!(*XJbTb&0*sPLCBt7MxsBY$(=denE0EnFA^?$pgqmos}!NyJs`a zT>Xv}XB|M)BEdV1q_3{7h&yjg67vURNTD~WJ~#Y7_ZL#=XPXLOc)uB8ojM?8XO;K8 zIIA4^x)8+U$XuFWm1>jSe8%@93movC{3X<=#q4YzKK>>qr|EMl-&9LJpx-)O0}4~F zariWL;r~e^bIz4hR*;zkACVV1G1b=Ae#i#?8%kVz$e}r%X^n`#%+_8r z7SkPS?Fx)C(-x@ku|!lj`$e~P=AMJ@#OJ)1t$CZ^2s9F!U15Ul%oHUTGwv#|`p#8^PNFm=NF1h8T{_-9a{G+&m1}x53iT z%s6jWgip-O)nU)#*uaYUzP-(DB91iZx|xJkb3 zt?m2${cr75+Dk=N0PbVv{+FBepW5lOD!6uXaYy3(A7E>kilRKYn+aU2UFKZX6LBJ! zi)#s~_tU^s^+O0!xdJQgJE&He#!A2xa$F66-m;xBnnTkITDA&Hf!R?uGzE4=_aEo*UdKmU84BP@ z7<=UZA?ur?>+ZU4+t^N%G-+(xXl$E}ZTmDvW81cE+qP{x_dI>?cgOeMCu5wE|M%X% zg}LUMlesVd$`!{X6ax9yGs8Unq4dn5d;Tl`u^IxV&+e?0Kbxi^4gdO~Z@RHvSBxYP zvD)imLf6G8HMDvpHh*%&;B-0!Xi5!gM&1Q2KNH(=t^aKB=HWsqN0Onyb&{JXe+kO> zk_?@pv11gRjRKz=5Q_N3%i|U$F}rkVz@nqptQ~+;u;69jiaQ#WS9iE!RaiB%;R~ae zw1cYGi@+|(ZIOB3bSHW`EN3E@EIH4Sudd8s=tNg*1k2LqoRc)OQfuvVn8G3W{vh8I zJQGOm-_}PJ5I!jhtW}Qw)%yI8R_Z_h=@ZxFfjsBGf~mBXNDzsz7jOuG|FAh`Rn5a5 zvghNE2*vPpi&jJBV8FB0vUj{u3K@+!p<_dee&u6>W9+O%qS)YV31Brv_BNYLwExvhp57A3&ZiPIJy7sE4!(17V{J*0^I>JR*lBPD5MkMn1-Zh>pA_o|Y^I zx1U3CLo6T3%pEVmFsG_R+9Ou6H*3CS3!WmS$$`PoCsn>3{ll13L+yf*8Fyw1LJwy( zE0b82RK~wLZ{?$(ZfH+I@rie_8n_ZG8<6v3DEOxJDCZ}JzTDk!t@^<4ti>BS7s2)s z_RY5;Z&VVKO0)WCl9J8=gxZedylNI`t2X2vZ-V8tyj>4}S%eEZB%X}G(=Pbei6{Pl z>HPniV0p<8|Kn}wm2SmMN_sKyjzl7F7rZ5oOq?>?^mEQ*G7p|6He+g*=tQ*`5*%@7 zITFPJ1u>^vk|y%8L`NPkv(n~EL$#^ig1~T3T7VlYY_AwWsxV5gdWncVIeuaN~A?7Kghk|(RWK;A2Cb`m~O zftFO5uxa!A!AAgm^VyIfLu~zUMq3GlJolP4VjLo~r*u`gkaC4PG?=e{@sx_Wd zh5YXwU*I(6Zx90Nep3R>^{gGtq-^vIjqLxYb&FTjl%4yD$e1La#7G4Gvcda})g%fuL4 zr#IjoK^GJ|w<*6%PEJuywW_R3p{hW)9SmFjH`Xb(WIF2XN>tcYS=(iHwmX>6z3 zkMd19$u&zKoQYC}Vf;&A4lZ#u6jni)M(OCL{pjvTbiIj704f`%y~sJb;Ox-CTGxs* z&$MZ}khA47KawUFKiPF3liksa^%;b>K~ll-HMu!>8p?*FDPDBK&^B)4!;2ZnvKL`28`jNu77^y0)PQ-)0R6PRS9R*0C344JgDCWF zA~>Tdz(Y5wS>~Hy^AF-|B4W^9HLin+YlO>8Rq`wEO^-pjV|zaPUtm1hm9bUZHE<3X z3(lyMtB6{e*5}V5L<%Nwx$SjjYb8b*tAvithaG^)k};E5y`4d8Sv5^(w4k++#^K+@ zYPq@MFF$ZcHD>c2+PUnWe z%}2QHm8BnO+rLOVTYoQhqli! zxW&>!#Up6-@~IL-e)`G&$K#M$Qz)jeN$xXbgl!LDBI|?x6f|yZ|Le%X$QSfmID^c* z-;&gb&)t3)-GE>Hy?MC~F5N6;rLQfuYLUzhoOc zMhY(NZ`*+H5AxUsjNNy@-~Z}D{{gjt#zy#wBbi7@Ky+{%=<$XYAiE+XTByAG@`unm zv@SHznM4T;`J`Y@SlBi>OkQC&Z_H50i&7gMt*;)D#CV0)-`PR-A~7{>@9oXo^Q)=; z=wL}C3*IR&tEVNzkUqihpnk#xq`dr}QRVYCP$Ts+nLG@X@K%x}8Wdc~jQw|@?K@t` zD+|=cfJs8%x8k{(4CFo9v2=X3V=3H4%JtfA20!r%hkWxg@K01og#-x^;J7)x*#a9E&z0JmviXz#dbH&U;B z;4YjEQ<^S2mLOE2K~!`oL6{!B9z4s2Ms_Q{D~)uaX@q9=%j?Q@|;u^=?k+q%FU&PY4Jq=cVJudN(;JV0qmL zH7JDBp7Ovha62^gm3`7yU4Yz)ohqBjiQPpsb&=6%oZ6@WXO&)(jHIrzRC?R6fXpH9 zghW|-aYRF;al|N_E6~WeCZdXEutv7!qoReKD4Q#O{fyOuGlzJo$bu@GZjC@R53#UD z9ifn_KGv()+cU^!i0M;|Y!~Lh#i>x%6|yJkToJP+5a)0CGwG=o zp;M4@WDPQJf6sOk^x?v#Tew#5gCy1Km)$!>xF>r)LB3x8Ku{dB*@Q2=1Z(g{@Dzk% z^&-0j6wEYoXzB?RKVePYzdSxCGk^YmusBvegOzww)LK)L0!)x`KT&D;DlC3yd0A=2 zQAI@wY7X5{zO$PP*9I6gc2;An2{>qd7+^&=y^)m`eOXmHk5)eOLQA^m9eT`X=}4Nj ziZR_B;BVoq@)dCW{=Sj%hRr{`sUxdH852uxmRShd*^gQQREWAcHu4uNe5Bq^?fx~z zIMIwS{XL=b6M%59q3{7;KKevR!6!V4f@}LVn3o;r;iyXTbht`#bE^btoe$^wC|EL6 zRz&LsEiv@yy*E^pik2rzNvkFCRP?S(khsZOunt!sI*}kK0Vx45fVE^n!f*Jw_-=Ju z^-{8K;M3{Pd{7k|ovA!jQOXD(x7>Pg*{PXua`4AFvZaJy*GqIKH)%^Za}i6-O`tct z_(v%_5sRJRz;*Oy7ILTxfg3$flW)XbG`eG9kjhlD%m`I*VQUZ7k%Wt|3i+nV())YF zg?Q6}@!R$1KP_VqORx?XaHcyA^RL_nbTAbL$RqJ2>NnPnTJR&|$vFim;7WG8AeW1v zz~@v4_C+?X!Zy3Mwk}_w+^L!ol1SwIt;(9$QMv_o5}TQ7nM{l)4&Gkg&$)h+C^06D z>2`O~fV#l1G1+c*fr6Go;9|M11yR$QI;f|Xs^`%s2y-%E)HlZjWcR|+8chd<-~iki ztO+CCP{Ef-$CDVV!%|7DcLPJXl33&yHee2N?M?b+W`t{n-&X%g$J+Ew<645^;E3Qp~AImU)BG#vkDb^ zmfg4!$<4$whdN7Fiph$U!nMo3x9DPOC1xxIW3wQ4%nM=#wpJaz&2C{1Os)kLEnkJ4 zM57)DurdlZoD61DG4I2@&j+_&{<_Ix;v4t->FN~duvdf#q$_c(2FJUfO9CQ4HfniP z!QA;c`*-2jF=2EXB^_t^AnDb_3n&GK+ zfF?10Mv4`KSJz-BmXNf_dk4e_9G!R5A0KimkIUWp5)aim{`sYgOZa~A#ZN^RRuJv6 zkzOF9KZbs>hun<@4$%LM>zPBl5@FdAUM1D=`xwDG;NWg1HS;}WjX=Sl>nfMMHB!sY z8qe6y8}GdzR4sEA`-dn+bbh)N48|&6I^RD}GHQDf!WLNG=>U-og#WMJsxJK>p21c; zJPSY?Xas=9Q#mgaPGKd#X&G2YOj<8Q%p5KD{o8%?Qae^9x0~1OSew~Z=@R}o{d3U} zO%mk#X0pe$*LaT&K470>sY?-r1*`b2fM!f1skX|L2z*fFI}s~m)ep327y3vBNLX_E zJgRxcz<7z>6qn}0rjEOaHw)yX@)P`i`e#6El^J0%U1P~1-2TY(+~QH#($!F($&40*nl_*U z#coc&XtS^5`i-7h=aYGHpL4EUsFkPxoZ=$VzJJ%xS#x^l4@J-03ib@iL+~0?nFk@}0EIDhcPQF$!({@vfPS>PSI<>1Up{I( zP0}0KKX)?^;UQrKEY`jO??+DZ|G7#2Ez^MGa%Mod%CjP>3)&|&@v@YpGhmEdRlzc*XsP)bV~kt}!&>8P?ctM^r#XsOWy|~ikhQ;w z1B2}3oTB3I+eOiGjf?F~;~o&_B9pb9q2TY4m*FH1br#AoT}M&Lmd=TyL20(x+e0E7 zWIv9T>vqGHp`2C=oQHClAtDMvWfal1*S7=2Z11Io!Ln2Bx}`X5*vDknizTx`_!|F_az(LUhBAZ3gM3mgr7X(~Ta;mBLahv&Ox-OaupSC9 zVo)S24$3<&%Av?^2*a&c612g;c;K*}n_^Db4Vp7#G@w8Z59l#pAuLw_j9=sM5Of{b zRGMK_B}C|ppWV}KH6TzKBXY&+xyP|!kS=DcP3sxQDBDKI7v#;3*SaNBPu!+#KTvWH z$N=>Aa)#$gKxi_|o=_I~VZsDrKMSc5aaK*DNcFl1Vd z>CGyMv?`b}u3ys4gv2{47ol%)Bh4#m*CW8#;tcUEX@p~%b-g+PsfOf#?JO>#b0p^8aXqiXwC2#)`U0aSlo0PhxMr^>f1(vW>pE(G%Ujc zGy_n7?&Uv-SAm;=qGfV@^g{(z5Z1KLjz5lJ@$R9c+{41KjI1c>^pzv?g=AgGi#_EV zIwl@;W;iO6p3RSX=>gYfhoXR75ytpxG1V-aph7WJ@G|Thcsw`R7wm z7M7quS`-8c%Kp10hJ=|oa!JPZ99^b19pQPfrQ@7%g;c_lcqN67b&$Oy-{EWtSCHnt zA9l*QW#p$#C4*pbBuVpD-zkhW_PR6Z!=*xb?aisBRr0z;T6RE25=jAV!7NPDASI!l znSY!_XTRl!ao?70=*_aTf$p~$>8ujQa1==v8;-DWl&X7^8mL(!Yvc^asl6go7KaTO zK9u9feUrgqah|q>Uh>Rr6zp1f?QoBKX1wK~Q|C;d3v!g$EhI}1$)lOiS!KzVU zd39stn<4Q;$!`Mwd0$u?V1wisS~q3N{`D*8I=ABupB>Bvw134REs5vXVJe|x6@kNw zuAfhpT{cA>FfLg+hA1p-)x5f`At%_bS$wWRLZ^d%E+529*sQ&PM2A?ylObWXKes}q zayNKwgln^PlJRns)=-3LameTR&%e_RCutrwe=75RBCd%X6Pp*xanSpCu~j8Bol>5u z&&d+H!oEr}hZ6^z}R6$17S4sadgEWP{z}81cudaC>{k{1e1uy@-En= z37l6^peu>SI8dvAAs?Z&(Qf-+-%=~K74VlhrWmI72n{Q;#v~747}(n@pwB8s38xHG zef`>O%xXNW)X$oHRe*~A&eHm;KVx@o6r95Jau*NqH-5(6;@$A?q#M}IyQjw)#vawX zo$J`4d{8$$MoPw7&1d^6h~XGu2iX5O|)H9&ov zVLs1~z@mT;8pW`m5rUyPmbDc;lOZsnXQAn^}b=Z{@zr0``acjmdvQ|dZ`L+ z#hp=nLVCyAH^G4l6m<#n34NmrjS{N4ovnKNB`rVhG&9w>AcH-(c_sYwZx_F5U&^%) ztRcnW{>vKXZ&LV6z*eh(M*%2lD*`2Xc`QbUSbM-g&B6t5znX=^+huPOfZ^=qS;IT) z&q%EVg;7`XheYA1k3@CcvE4=EP3aZ%3a&zn7o>})COQ^$7Av|heuG&3zIWZeQfu;< z!P8m__;`J1`3-LAUyW~|JRptbhSg|THmsH?A0r=vA0`BboQAs1gAADXV@nYFWh0z0 zxR{){ji@IN3L}?UU%)>PDiLjrh@_{?m+q?B*P<_V8 zI6n)JiElX8r4dGS4F`bNz-`MCTl*g0<11;}mcdpoiFr|GVc1JilaBLl9_^g28pI8n z`?`cAf{dH08moCN`3TRX6VP|dqG^%$Yy1svzZ=tipV1 zg*mkp;U3ZUE3=up(|T6K)%8?yEM)e8i$sND$!3*S?ZVilX&Xt3UfyNX)blA1s`gYg z&ahHraoCLMr3hdG$meFLOlEsxWi#8Wg2Xw8D_;~GVXZvng3BhWsNf-Ovv3){b8hXi z(q_54TswA_mnny1>cfXrAN=T5#K3n_ zL+3OVnR1&?={;A;r4ftEf@$L2N5^oEOvbJMbLDXxJ?g=Ab2^CWCrA7n)(%$xwZkzY z7%uUz?J?M}j=rl*(GRc|W6eDFh%SPGBEKxbzAnQRM7iwBD<9RP95V77?3~BhO4|k5 zPbWheS3-be0HY1kwPsKFgo1|&?rm-&*SCk6!FT^MNB=i5Mu9k+LMA=}vbi%tYMOHb zG%e1?KhI;og3Dk{4vqlI?pJJauh7Rd`W~1vVsuEG0o3RGFc*Zx=a7$FAN@ma@>1EG zMTfi0Jw%_r)q-g@==rlaD8_*Z$I_OsFtAYfw@HGxdo->$q%+B&aQP;PNRpF3bm`Nh z5VG=q>F+Ie>@#Rt6+b-z{;Bam-n1|>fzYbke~B~vr^Nf;ED0>Y-w`5OnlzvOtht31 zI3l40f))6zdpVTC5yg`-aE91uBw3@$iY*U__a$__f_%#DlRl}&hqc+9NM)-Xdz?JP z_^o)hvQ#}?GhLDOWz{FZ9$s0=+%!jK>csIK+$&2*sDG^$<{R~#pszrohwPT~8*v`; zy#SW5aTjoa2*4>!`8z>8(i5S|wCld-;I|_#4AZqC?hl((6s`@oC;>9s?FoEaCc#^! zc7GSQN(F3P{XzsWAyh=KVVL|T;n*CO0kxvYnroJ4v#da>CE2gOg8-F6)efzgCi3RC zZ(jwXOvxOX?1D_JF)bIV7_IpL-4ZFD)&5iBy!_dy^Hf1^N~y_@7lKp*mY8pxS`@W@ zzNwp)6;6Sv^4$|6&K)?hN($5+owG-fkt*+b&c(%|7+sUOeEQM90Vz>w@f1D=lG^#t zp?BJv6p8=~>!p7qq1CT&`H25PLV+r&Kt?J9kV6@-0_}pWivD3Z>OSLoHfN8cNkQwJ zLt2BX>6DEWCl{;;4tFD9IamZuV{p8jqLv~rlG_(3pxIDUX(Yb1D7VlDu3nsNN0BG@ z;6-8a%ES)oaA~^dE-!Y?(P6Xkc+tV#PWE{_l>5cHf7aV`C=&7Y+K#W|e6R*@^LDoc zPZscWZQ@f2rG@lV@l#w^Jd4^^eGAzK5B5!HOWY?D=1pqr5Tq07DZi@{%$w#4+ouzX zo8(H>rxW^3dkYWb6YWX$3mZQ6PbAc@mV<_0xNNz9W7`q_mXQLV3Rr}g3OD@HYFh|` z?glQ+K47T9&t#lrh&GeMAkIWVZtByWR}=*}V1m(LsK|nk37IgIArK^uorPI>uKQtv zer7n%WfucI?0nOJNgjW>?Ff;TPWS`EM-S%W3g*+KW>kg)JEZsg==kStIXh?@x$2=kJ?^X>F4x%5%UvCG)hiuzD)z!1hJ^R8R9IaZAiTMFyx6%5wBhIk0{(oWl(VYU|ZhQPn8dQL?0%Yz9A0ed=mp2(kEw z`uPd-q$VuzJ68ohU0VS43^s-qOczDj3-S-@)WuUCwSpXIlwU0vb(lD;>X(KAyPzyr zhe!M_eJn0@rtHVxmr@RWhMc$z{*3tyg?yH>2hm7sVvb{^$&^j2Xu_Qp4GRxL-QA~0 ztN1i=U#;!z@Dse8G1*B=|^}okFuCBc&k<8i$Mo$&oAf((f@`zhc zt4r#4&|8TX4mXo~485J7RuN$(r(oBPxecnEw#532?2ppB_j1OSF%k!gUM5NP^`bpC zsn;po6ix86gs;w3_iAjw&v0=tGV_~Tpjo4~Eb#EsWG0HN_(8l3(aPNbSS%WnVgt4D z-;2xUsO{P4Xi_IUZye=EXM89*W`n{VJ%JrYg=0b>7AG;=i3!O$XE4J4jr#BibH($u zxjZ`O_k;kDaSfwg1YGbEjGbogRr7Lrx{>jNF7kb#P&}w6$xQf^KzBlO2{dHNklMkr z1d67NeC@Eey4byg+c+U$8A_!}pun_nyjEm@`H-AT-E3?nnaG&7Qa$ZGtw7QCo`Xhq zF4|&D(_SRgozzG<#OnZNGZb!b#(FkWdK{>4pbx|*R zA>=EQal~BFsYjAPxn&RwSG=7xTZP35%f|?@0zNipTL~BE#nA;|iScUp#!(CvQbOv=18C@lI%EWBybP_P@^@Co-bYrws~QRv^gdx z3RAxZU<-UZvKqry3w+_bIdw3qEB#gPFaA91!)FIZ*+7fNGF$_-3bI*zdBLwb%q|mE+OA=@iNy!e{}vXFsdUJ)Q6^P-dW10KfQv^@konG zHC+s2Z_tYXDA=cDRKmtpz8+h>ec{o}ELc2G3_hg|1{he!>tyD{h&#eCgOuJe-h!_tg|hrrK2$ESJf@>pu`8ObSzcI#$*XZMe0HdGq3NSAaGrSm9Z|ydp3ypI ztfR(X5F@siq|FrBOYv3G)eb9!aYt1oVLBu41deVSV1!|?VgBQU;#n&*+8lXDHKsZc zCGZJWhfx&K0w!>g-(C6E9h;{b*t1A3(+f|8rv|t5^$K+p-Ld&Q^wv|R; z_HQuKR&fy^@lpg!ZU@z2oBG$;8L^}7vY)@pZrV{Er>U*00{L4uYMY=J- z239k1@c#n2&D-Hg8@(dnale|5P>5z$#lBc=q<=f>Nr7}jaQv-{7^9dJ!mykH(bj5z zEW;y%LH#;a(DJQ(k_Q~)ov(7rLzT8^@FKNP?-})5k1b;Bx$Rrif0`nlfNy0+z+_(Z z7b*K6Mf$&06wp~f2B2gniweXpBWVqS_$B1)2vZT3z%hSn52QGXR~EK}n+jqrvG#oby6W=9!IeZ$r6yd;jxguaa3l0kuCNN5sJ^~p$c8DUW zn+RMsTA4m_5GF%Uts7(jbEgFLsmJJO(oiy^o2H^1ykR>vf=AOZ*sw+u4mEiO_p^)T z&nioK&Lwm&o$;z59Bd+NX@|v1y*~DdH1}?@MR#&4^gzX;c9-%=wk*qS@=SK&Ysv+C zN5jrit1*Da)-7pmuYzn%zjfd$=9*-d_1!4&D72OcbB3)>hOHRWU9x~?q}zvftx>D7 zh;trcgnmUmSb~J;b7}HCPj>!r-~*yE6E^tTef?!9stCGs zr$Pg_Kah?&9&~iYQ7U!A{eB8*T%Ccq@((HJpHo1O!r{?5qHDUR>Q(wZ8!kc6FU>U= zDD`G;hZe8Va+t-vgy-UX(;{?cpl`}}6Qx@4J!B$4>jN(O=3xLiZkXs!3VkO)jUXkWF!y*#6{usd z`Lo1lk7Dl}bJKpc^H(AF_U-`{*yyiECxV1e7 zcYCs;jnh1TssKl#XeFg4f z`8361mCPV0nCuz6zGS?zKTK|J0=}?!_gXn>HZAt}v9eB4XU+3NuKqaF6V;u;1E@5| z6f?qdz{;cwGdEZhC<6+1WQm=nEbRur%%QQquR5hP93enC1NKf;XMw!co z$uUwxgEA)FoE&*!QUatp@N{AK$y60Gdxq^P3Ki`Q#&L76gQw?NUC`k1MOhohKG1;E z_O(-QD=5*80*+xb4zf?u2r%A_y73sd=KfRrUv!}XVJA#$SAg#+boJBQ@jnf8QBjtD zSP0rAm^j#9&T$(_dKf3t7WfiOxn)*M-Hi=WDuXnlaw?B9<5cFO9c7p+n$a`*%{=UU z&LB!Y#4(<-Jho9@3Siaa4^Fp_6e}#a`H22{;~r|R=T9)Yb01NgJLOn)vODH`?_?0|8nH9Z**d zsQ>N4Yi&(e`PDMlx{E=M>1v|O12mPiW=K;vsh&RI@Vk!MknThebA}nY8gX*Hyoh{n zPB=^1imP8DD=R&~W|!uhrINh<5ZC+x)}@^)<)a^5A3`rkzPC&J(tTX$I*)Hda4Z5t zKlFh?<>pDBLV$J-qs+ZU=pbAKlbPF_Yc%bW{e_)$hW|&T7rJ+1p?F`txyhVd1P8Kw zX>uUmkla8kT=Re{Q+uThm>*Bfd@C;^d#{#a;Eu!1DC#r-`4I$V>~mH|2>phcw;rK` z8IcN3MUy0&?rsZCed8F+t7Z`#?jR)0AFni6!EO0eHcX=UZtg3h?0SNWhyW#A+-Gd| z)sR=r$2K+G{^!Pk|MdFPQqH3Boie7xOSV~}Plyp3#cgPgzzE0ydW|4=yBH`cI(0bs zs)1iWm0^d>JcKJ_*O)1TS#?m=gyQR>KS#MC_TBM_3%rWi{JpUiRPF<0^B>3?qaLTw zf~p&gfBHrL96YXnZg=tD>E&lK{P%=dczi&Nv<)f)QsA#3;#A8-Z)&S&>qKXMams3V zNLnQ^xoJhPpezZTcI?I`r?Zi2ifvUNAouejh#Fj1`XgZjagIWue2;|jswbD&O}(6{H^-n#`n@9Q2^)h z`f6779%OH*lW2>B%dKxhXczsP+a0wEc#2OVrCvi-i!Z7(7vV4!t}2-MN3`J+fg9Ia zA;~ZGiRSp!{0j%e(3+dE8ZC)%u&-@T0|A<1q6BR zS`i=^CRe)-#bXRdR}U9slHGyvaRE@>yX12HzAFkAnk9hr?ordN`7EL?(+c(`7YFLS-jchhTZfI3Y8Ox}ir~4Wo7;VeaB<1B9?fuD zWSy7}?r)i1#qXnA>IM+H96jKmYjlYeK7!68s+#u2OESB35aDAmF(yKFD)2${YS%d3 z;D5_StY$T^9LLl%IRxQ9gn`{W*xRJr960|w?KF%qyZL8Z67cVfoHRfX0%WQ7`{fkw zbca4lYmG>>Pz@K{8Bi#JA$ot|i(?N_Kzz6{HhExsb$xxl|M;pK*6}Cf&ksa+c3!CS zFSwjv1uT)BA$x~&)q>ek-M`6NRotD$Rk1zs{n$WFW6F=`A*-Vu_apXK9aZ90H*&O7 z$RkAc-w-l$Q=C9e-V7l&C<1g2cW0(B%I?>f1pD!AHO#4u@4sjZ45nn#^nBEaS*Bqc z?7}9n_!A+BrX`G*XnB^rAkmgit`jVtbo?lv@~r6oJzpGzOZ7QK=RQ^nU$Jg_7i!Gx zE_iCx+v9vb{r=ZmoG$1n_yI_+GX|c+e-C`WB>MUyVkK4}!xIB`OEgy#@RnWV5f#j5 zawBDF$h?B}B7@TJ}yNQEgD}VfZc-M416f*FQ7DcqqyF3;IbcL_0=% zXq^I_)zk4|coy~;#HXjlxRH`T3s&Y9M+P>FqV5AS_L(_Qd3@hw^Jeu33Vr!xX3vV3 z!bN=Np&tsF!}4}`9VXVRVTub=qVIOf&A}ia4OBU-k;pAfjMw#}M}9sKsh4F3#(n7c z`tu49ruKeUVIDP8nQqBu0K9#*zlYduc=2^~0=X401uJbSU;)9>zIXC|ePA9iFJbKN zxy7*{zzuj~`zMC@Nq3|x0+0N^CFB3r0RMPIB)*mWv-K?34dgW5xhuD!f$7QX^?NvnOP;X$Fw=M^{Dbjk1(R}L*E2-dSGZLpyzJ?_$zf3KH+09lIXG9Gu1b8ewWJQYgaqAULaIexz6@ zEEZHF-wzso4C_kXGu#xy?UEM=S5; zjRZWpfPcN%L>mpDZ!mmzQ%Wlno(h%vk;CFkQEO2=Yt+i za8&#YkLsXt2b%!mp7cGC&iC8Xm)%~C4{uM;e{?Z4;LQCf<1%Gma^xp(9qKL40%`Px zr$+uhzMxBe;g1K+a}%^c&U1otGrqv0*ZF*UL5Mjz<*fNhApkfQGBc8e4$$!=f?jP` z+boec_gAj&KqHMlAcqzvhOz+7-*@AL&TB^D?=yG71bTjX#rxzW$>c%`+*eWwydQ!C z$iq>h-P7PNC&XgSOG#(=U{NfDOkax;YS(`}^zEYdY4(%Bxm2{5Ia(3VS%PeBJ`R^W8zseAd*GT=U^nMR2Xy*v_c#e*m)lVe{rK zzp8Zpog@{O=SNTjkMtjB$Nvrl|1fR8Cq}p;d;^3lTd96mMe_dU6X2AK{0qf0JZT0@ z-p^OS(`*H-UK(}NsnW6=elwaFosy^6Q`K2-DB`c7rG)GBuuMaGEO0CmEG|Z8qa(%#!?RI( z!#KeEk%&;R9RaC-(!(z21^Lw$5jbNH%$xqky>OVl_b(VFXD^)~onXX>R%I09j@3{zS@-enE7%u@cw7xX{b%p81YlwCs<2`jc@DJDUP|E zO2e_7yDaG~f>s@u(6KgSrZBKJt?d`bmrAUmWF(`Rmdh|6f}e8)N%vc5P|WsUiz)!v z!q(BxS+N`eEvU_E#Dgu;oFH`NGun_%exUkM!CH2ByBNumNu4Vv3f8O9RJ6spPl?)z z4!jk^QD8;{N(&Y&Gt+pNF*?)dkSJ?rwL>I#h2t&qaLQ6#hIWj5kZE%5SnWNFC;K3c ztU}*VKx7{2-HrU#+uGVeOd=YRg$nm5WfL2*H1cwN@oK6pk{@4J4{K1Sm$`<>g>yoPoEO%7yJEUf0B!dj+uMEf7RN=P+ z61N#D$vUC%YlYb#9PilK@ub|`)0@ao;Q9-Eq{mj|;Et$&5T?AziUQprkFrqjM2R)0 zBy)9Rt|1n>cTb+b+}dr`kG;x(OvSVaP^ef&@Z!jAea zIG%%G_Vb-S(SVM!ypm|Q7M0VtIPI}Y{sv*RO#l(s6PCHdCZ2nk6W>ENI39TS2)k2E zypEqo=xwkU_}O!cL(j#iYo5!bd!*mu?tgF1lQKWV^fXT(DO|%3o7i@C(=-`5jymG! zV$k9p!4G2lZ6_ZY2FRkI_^OE9PJ^$7I%# zOC}SdBHQ_$pGkK$3hh85_XJT$B6UVMQ`H(aNUjomP=&A(r#yiImr)KKoq7u{df(g9 zEFRrtWM5AoA2Yu+Tx$s^_7Fj9qBM}4iMdj5i-X9bx+VyrH2SZ7t0CTth%zDoCvXWU z%flzg%%P?;fG=eP;OZ-i$cu@Kfg53I87dR75I9*f?v2JTG`nZBLY6Gs;I=mCJ^Z2{ zi=(>4%3HtaqKAJZ5%gpeOVCkIXf;VT5dj#MR_G>RNQ78oVHtWJwcH*OZM{FaIX6lJSuBt{&tvV70taIP!Q zwr&l)Kr`dGJ~+A;x9A*-x>H4@8O(1N@Q7SoNjU!)A7sjlD34yBM}({PdI5!B!j#dD zGcz!E^85a}ZVFbAGkuNiIAS)96*JtK9{m|~cDE$D(!4=3+w+MSPSau+qC*Hhx)VyF z*f;)Kjn=0RKypK};4g!$J`uhCwmwLSxlpW;NA4YhC#GX?Jwj)1=tcKM2yF=GGjI+= z0+Us2*ZVeZN`aU%NIrf_pO_O*ghjxOI0Ye>N5CT5lr`+ce}QOP3zLVa?t6%Fk0~C* zH5Nosk&sK!FTw3!)m`x%h!hpX+(TL+9!ZBVw^0Wpl3_vAs>VEfO(?ff%_*3buNdnD zJi;j%ni;raJ~AyLqDxuJ=5_Fbeo2_PZ-4U^MC)$>DqsPx^=~u<=xrhfKufb)DHu>% zHNPhY)&M1#C-sDJ0{rAEMXsBDK+TxcDxhXe>0QniKlf{H-(ubtzq%`4Z`-ol<>~7? zXcu&YmF3b>-E>3seij&TNSXqR5%K4n3g-9p3wC(7zISnb!?G7Rl|qm4s!goPC!R;5 ziidBtKt9)|%r;x2ag7DD}&2+m;%+^IovpD-(;f5#5L5SYn#6HYXu zW>ZRT@f=YF=v#_Fm2G3ny1i&mN)cyRIg^p0<|{~kb7sVGa>eZ)$H?sWD2M^Xouwx~ z;^sL!!a9QW++S~LmIo<4w@sn*Rt0TJLUqF1OQ+dEjM^d7pc3Iyq$z6oEi&u4`zZS> z+hk>`N}0t4`{XJP&gq2OPQx4O=1K?~a&FH-D{7v<>+$|6u+l1~$^Y5yEBSXN@K;EN zz7rk^!1^D2%1Uun0qAMO6T9kfNIa4dyi3nD;ToSRlpzRtZ!}O)`3k;XMN+e z;2$9$zbiGsck(5?az}0)JgYpBJfJ8c8X8($WFONmcopC87lUNK(C&JpsKzv<^wC7{ zM=%Z|4O&I$MR4H|;pF1Da13yeYD@*Tu>!~xgSVw%>~LfWd{8Nf?Kr_f2WL(B6Ilyq z#Ik3*)bFs#vvuW)Zy_qSeSJ=km~25Y;Me*%Kn=dP23ZFR@dxC}%Q(Kz$zH_5G`rdB89{fKLQ6od;xl+ZFITq5DC!X>F#DDSF^gBc5|4+TIO zy46_2swSakn?Od~K#a8H6ZHqQgSo`p=U)XAFzmU?(2pTuE20laB|CC+{GipG$wW<% z6no_r2weN9fh$%=e`!dwy(?Z={<$PuC}fq`Z<=?1Y0J z<0&UR@T)RCBB50%~h*yfzXK1nzK?R^A3QNt`+&S?hFh4hH1 z56H+@Ih@;!eh#%2l;rO}O$q@IkeNaB|xm%5v6EkxY1n>rP?D!khAaHtTA z*@lLIa+k-cbLyh{VtD6G_gz`twgBmQ=Z+R0$6)ue7VS+HM9#rOfy#}n9+$3A^>5&G z?)zBTT24kT~d2n{7TdpgWy>ktyh0|fV}EjG>Sg0 zqCW7RD$Bw%1?fm(5dS)Bu?FxY<3tWtAFmEfY)Xq52*4Z)O2S%W#{hok_R9xp7JXM- zEObbbJuvkBBBx|*QnVQ~;(AI(q@N;<*W*x?kkNGWuC9iW>L?}>Eh2@FaVYAFPPyt5CZ65 zXjp?-VA0HrX~j^kxF!Ijo?ADtSp`8_=8S9b#iat_IfG9gkv_O@QUescGZd3eI`^JT z6}Me1s3_!UO6ItUSn+akHApZ)9(exr-dHsP-YJ{84}X9vbc(ff$P?@L8v1|?`VZ17$K z|6CW9^7a75k!qshIdPK|Py$8HhVwaSOQaP5u`#<75nX#m6C|BQ*~Y`?E#FDE#gsK1 z1Gy~7WUY3+&3qT#e~k2)^yL}+pD&4!kP;se2#?qNm-E730OW6ysT_jG0!;oR`Zmxh zzH%2iU01Er?T%_mgn3uuiX0#?Etvl(-5N(@268OoNsb8ft=kWAOE9EXSm=eje2HXDQoKF;g{@+q(>cN#y*!U zQE*tg6FhZMvG`K3rBx&ZIF=MRrc4(TxaAx^WJJ}-9|kg)_4)BLF5s02ZEoF&Ydw(r zrCx+rPkNar2W~{LdK6Fvb|ZmWdG=+{%IP1>NDp`z(FnQB^V^;MK}5)hD23F?Pe|W9 ziQ5j7a0Zh|P|4!9!B0v}ksC7V+C`W||2?&V&D-0R9 zX=bt5@TU$v)K_5&y%DbljV~63D2JGIpQxCjyY-lp`OoV5xO0NsXH;Vf8hy$UNyz6A z_U>|n1SpGP>@tVR1oz<>mx=es$IFi{#td?D3UaEZLS6ap8sAqKTuRrxEjy6JXi0n@ zOLi%eZd;lVs0vc#ULh2~wQYf}LcbGrQBrSMD4}%;J$<=8bV?}9MUm@oZb2;vPzujL z=yti)-Pg6M=2_;GnZ`8Zc!Cm)4SMP$K?}`r9=#Fs?Y2{?tuKr*n)4Qr=Yh=ME2zR% zBS>hhHSzol>;|2jK}AcIf04ZI zb!SyP`$|?Xu{Mum4Ok=)vHGb9K2^H)7W~%2Llqlkg`l9GCY_lB%rK4pQsWS;xq#@P zSSY6-RMWPLu~qTR}9mpKBm!FN}gJy3W{B;CnqF z91)8C2nMrrr9$fiydKU{Djjy6`HM6Z#>iuMUy8l!-+C3|vCo)88`MuH@Urc@ZzbtL z*GVrQI^#DuW08tVh=jp{8K|}3?>Mf^%9P5%{vW>H0w(UR>lZEV?q1y8y)d}DySuwh zvB9CZyA*eKhvM!|krsy*FPwS$e(%Y-=ehT0^3P;4lT5N@?X}nX32phJtb)YCdi~;$ zZHuANG9Xp17zD44XI?elFP@@bvv}S-oid(@Ox}g0dicLr+3*5uP@V1*Y z3mqZKO$i0euhcCH*;34HdZhl13LLstjaMz5BZWc^>E9_2n5qwJkQ1OO>NJHuYr zeCQ?V_lA6oicOe!?2cs}e=Le+{r1=xdo|Hcdx%1?3zi}$5dCrkAP`NuF(-Jie3cum zv;2<}0!yTq5RnAl49tLwNMsKm!LWz{d?CTAxXnNXx*j`0NHixo1Jt&?3xo|IeoY4md2~h+_eZcyQ>a{LdD2gzm zJKh&2KeTku{_%52{(&TnC_rgo))g?os-ki&zlQm7fUIS*c3%d#MsSwfUa`A^m1fkQ zm@%`(1vUg46w_m|^psb&aGWgSUSy$~3J-6rvSy-WH%;S#h=-Ikb`%RMJ% z<{kB`Ll9)#H>(zR-iV4Rv!t?KRkj~mEtlV`fjYKsG#e59mrN9Xo35(~3bonl_s`6&gJgZUT1=^T0WZoB464W{WWMV~VZ9`z+G zr!=b=r!=aiBd5Gg6~6((w(?5ZM4IP157MuoUKd(>Y+q`sW|5ufPLqCIiEHtBRV8{yHV zwd*A>ROg%3qP#Tw+q#l`MrJREZ73a zt`3c3BaLb>gRz4horX-9t0$|7nzb&Y4c@q?1UtXTzS%qNvT4ziL~F6C}!)^ zBjSYngEQgQL=Rtx27gF-F|{CDA_U48Fv)riF1n=Jx9csl87;dc#UY`XI~dsT9oh0t zE4iXrYu)8oL7h)ga4CT`p*V_AL_>gCR-tI~FIIZ$Fun;ZGa3m2U+Dq_H|Y^~kDn;3 zr|-Onbt4Gv42EU&&h!!ws_ijuR7c4M`(nnV*Dz6=3`2^NU%Z=?X8UMo?#qMa0v-_G zhB`a?B1FT0`H@%qH!l*eMwh#@_gb&E7oQob7BH1cT22vnOjHZ&6$OW-L-v97YXjt7 zI$>nLUZvN>X31ijGJf76zyva8SOf|IM1k<{s)YsljC>4qejTDSevCmat_WjfrrudF zl&&j-6z4J)-WKlw*rH`sZ_g!768h!D{DenpiKyy)7ILusY&x! zqA^>svR$yb$93n)=G)B&$Zx=3g3?lLu}U%=Yvx(84iPf|PpE_+t4niT8WL2qrmx#c zKG7D^$`E`B@L{dzcOA3mlbiU{O{Z63TSq1<~ zKY}^lzfj1H0n4*pScv;IOD|K`{Pg5zBI{ zL%oFvLE0#r6}R~I<@?Ap>dO^*i~X2PmIkIZh*HEtEdn*WvX&J)8Rg9Y_@v+n3K@dZ zX#eg~#SnVOw1K|MOJY2~i8e)XxR+O+(b+I}C~=Y%3M5;rrZj!`_>eb^)EyL>E_Evy zLGMU_1=K@>!SHW?+iQ;;RcQd;978gymxu3lVX}OKO^J{tIe3*TI?X&%PL@>xrTZ*_Ig39)|3-_x;e&L9zY# z7_LAjM@_@)(Cxk}GRVu}ckX~!=(eAK8?=Y+zn)UTS+(T9Q=Y#}0F(d}WT2+LJB23t z8!W6+h62ySc~SNkMU@UmHdS7G%tJ(3?4u$~o+KY4laOR&up*`Xit$13V-}tMlg}e#$-;2nsZdAB`M=ERCGc88BVI zmV!S47Mpa&=K^7oQFR{=0qH12#^ImX;_IRDjE>C0WBBl@Ft>apLn65mNW19=VtA?8 zps63PJ0nI9y6{nbR7Py&arNIjmCc90? z*on!S-C3LFIL&pSxvaumL#)goRaEgKRd}*Gu|^O(=zJUifd+Ulvb7Sb$VSnm|MAlb z&$iB%kFP4bhupmsD}*-VoBgVATsNOhf%2-g7({x6-mj#6KE_oo`r@7DABmMqwfk<0 zFM5+CR)A3YTbCbmAAp(fx?c-(R>ebi^lDa`eYX#6YfRA!h42KPdWGb#&zMdP%V0NuP)pv9{N0x%9OlGUjpRjmFacd`2gxkm_d$WcbALkdkqlrZ}XzC8pgm2|I}(o-d~da2a36ro$vF!m06DTQinP%EHjN^o*eV&NQD(O$|FQ%cHg_Z7}EFH|3XRgTC{tZ_q+9a!S{qU zZ)POMf$FP*{IKY;L@$zdgb25hkwGLf=_sWaohYA37ptF3f|d~{nQAZ56M;Hj;0Q5o zrfZJK+h6~MmF7T%J#Z+H`#*_UF#nNO12?ZYfc9WbB>bROMYkkNTX{V?RsCow1$7%b zI=k{MxfmPg)=vnKow99?CS63yai0%Jt{%EbtY43R*;_Vi3Y!0p5qKj6Lfr6o5h15m z9>&VPT=HKX@n8AwJXp*+ulBw}^s@xuj>_Pl3})c8)49m*(FcEpVL=k2ysqwdhnz*~ z27bIQ>X(PCL+EC?2K8q{IwJ&7T{ra8LgvHn2L{D`8cBrbF>!__a#e{2U^qzCMSr+V zEmD?5kI)ReE{JBkmvQ|xA|O;^CJbAGuQC6*m?6uhFWO2OxymMShS}#+bc80wgFJhQ z8P)=Wo@@R@_zFD;G9f;XXpF9o<1Q2$P1J zubwM~y=-4rsAwOObkPOokkANJxQ~-Fe_a^{B%uoZH%8Bhf7e z0M%x*{Z_a-ySi|isRZC=RZ}@U?FG1#gss2l^i(Y$DzQ5&s`s>1w%&R%@D}T*0Zr4? zpnMlk?`VG68ZTPU&E$Rs1Qy+eh1?9!13%wRS<*U}xwWP;;>CcDN}fFyr-QB(?@*~o zllC5{g(wSO6uKSGZwl2H7hoJyL6BhS!?bA_&vu`o{#UCLI)SbbG z%(-uJ{F_`T-Ze`O>xul23Zx7tz-igy@nxPJrq^M@XZ5OILAoXHItRdy?%oG|5*PBw zMogBu8Y`{2DNMffLBq{Woj=mv^U*ufRh)Mw1YdNfMEn<2=J+z&V)t(yR_I8Rl@JM? zm@sTRYVoP5QKQ&@>Hb{pIP>m0QmNmmo2+*vaMdg3jv{4El&io7P(~!60q@6fhDT$Q z3|Wq2<7EN)Tv5YK<|njqvM@i}?Gq=lytn4vPa=MWMtH@hPg#rk?p~Amce9%6e(G2s&Jy&qgLFO%hRTTX}7^oTTA9 zxAv*e+B?<*Nlhpfo;MAW0YfT(oR0rCs-0h98y|YL4vv^C*{VWi*pkHn0Cl1v-#qKjM$jN zUw(qOmi9_`;jYrv3mg(g#7zozp36Je8=SEamPP_xn!g|xXFbh0df&LET%f1WPvi!K zUzPoK+PD%U^1KMhYUU}dEgVd5rpUk!A6-LQMoyWGjhnL%BACv!(AFyfXg64z5dw8n`?-ynOqyTRbVn*ZEN>OFuXgRMz#b2#a&RF1cM}0fU*&|uKvv|Br~s&?Cc4n zIzAZQGh)!k80GmxiMYePBdt~K%*m*)VaNP74gCo=M#J|o4CcWeUlKPYm>xL^kLS^m zI`<2oCZ;&DI2!sflNecGK86dg(#zb!LZ2UwKHLquqA&7=xfAD0i{_1*8jOArIk}U6 zmJ!(bzWntQE|6g@1fCwlCM9P)!6F1&lEI^*j-5oMq-{R$Rd43@_3^mEkUR*Brw6 zFl1K?c&KW0`B}Q0fbu&ZCxIt#dm7A*WNYVTH<0y7a#s#d_m|l!1!m^Has3H{ zlcAKn6@p;V@0e37!PXlLO*-5}yS!~v(Aj{4SHIn?eZ?(>IX$In3!U@K*61>{W+}@bQFSMs1(^m#J zmG1u^?mzz&%m3SQO7EaU#Z0d~K}H9$7@{7{9C*kwge#20JYE!buy9tJjgeL|ep%zu)5&7J7Ys2*;j>05swl z_#%U*&427TJQ!ZOgSweZyotR@ScOjFl|P5OPU1XX1%b|W(IaZT>99G+fOM$>fM|yI zJk(Q64Q@Lb_=Z8ea}3Zf*FS&|iXWo6hL9p03@|L0%FA1d$^ zY;yu7BWbQy}&mJ zz`xPOV44R}y0I53HcVj~girgKvzrm@kQ8ifKu1SkRv<|;0dW6Vr)Y#!BS zl4Al+r^C%9hp4qb){?G2tP*65vRPWmMZ8Apvpf*41mW`;QO=_Ca@OH5ST%_1{Su!? zhX{|cYFKLRn5^j;Zvu=xxIImEjNKb*djx6$d?;KT&VH;LAUr0*Z*>2Rj<#tvAVn0Y zGPzexsfqxm2%cA4VNVRZM{On^u5_xEO^f|e^S6rZ(;@EVZS5Rq({mA`4Ve0}c4Hr6 z>NNK7Q;8C>{X1|Vv=x4(vk{+0E}k||db$#xK@DeUp}_3x`l5l`MxmHqs`U{qt9%!1 zXzP`9F!Mce=H$*CU%^e9#xSjWqYluvtuMMZkl-Ldp|c@`SgRe=s-PR|Bm)~ zqjij_{u}0a{VPzD_IU1!ol$x#PNHLm&3&sDq_zA2u+KJmSi@!Zg&#W4muZU+EIjWM z2Mq{!evU-;Wd$`T#R*B(u!vUTa6c-FCu@O5-o$e~Zk%CLu|=G`FR;VKoXKWo5Y?G^?MaD~RkqkDW27${9J{K>S>cVH18!VUxH zHtKjg`sF_@v|P=ggnz2?>;Jx@f~7w=f!RE9ip78G`F&1D)QJYbV0L*i^bkahtL*lb z8dvSs>>_Po;(xUPWYL9m(w_Y8j!~M7cggZU+++Bi`s4fHx6Xe)2`$7a2=)4`=s^HF zl;t>)C&e?EK3F2Gu!2L5yZN3(PW*laR+Cp@uuZ_vQV*oYZ^bmKe6^|sF1O^Qg7hnv zFDXu7tWRmwq84viu#Q8lVH)77aGxt!6j|aeM^67>%rth4GY^%g#kd<{Ya|B+n1xqB zWBJNcUMAsDk6fE3uJbK!j8@d2NbMq-lQl?>ar&XI7j))`jEUtpa*IP&N_mpG6l-(k z)0?h++^rm-y(M|=CNs^lX{QX>)DOufVKp7h7{>;%GLEPLkeeFz{6?-B~;=yf2E`I@Rf& zh5@U#O2SK4+V=b4I6hu2!5^g-PR1M#Km9e0jpYD~ADIgi!nD|{Q~I(Q&u}-YgfSB6 z3`BC1HRm6byrd-%noiz0gH@a|r$#x_CGeg2F|M4WmFl{UJp{h7o8HfI=mT|=9EeJv zG>u3{8k2b`HR_z@%MGIu^%oct%ail+SZp{imV%LuB`TUIyv{P?@1zuWRx@kUGWAoj zNO!Z)APWhmJ?1XbVHLTa4kSf<$jbF5S*MMQRY*pJH3&#fY%pMEAXEJg1rf8YuXm~g z#Gjx5VF<0%_x=8@=HM7<>46aGT|44XYuR50&LqKW7L`dQLx?mfyw~ANlYzATh^grtfO>d_LQ*U-f*i~E*}u@P;38!q+y{@Ycd4Bs%q%)&7-5RRMz5tK3a z*~mYHwVm=jYOMeqK$Z?|AkuBj?NwE3%-z#nPRW1HMIC|56 zZv+uT6#4(5rm5SFvBWfMs!B+>#QgcRauAAE9z}kwp_fF6lF)S4POvq@v#A^Mh6@a% zL5xyN;r++j$7wT?47!;0;Zpb^r{iH}>(BGWA?*iyog){%G=3*{4ZccZ?s$PQ#IanV znPQvZLJRWHn2HVy@@!DYl1qw%!DvVv{}gboJ-mu%BExc=U)9WFicPgb#8Ri0>tVRp z1yBgpc5xxQTdn;bhCh*NwToBi(uR3Kg!7~8>m4reggcT~e4~epW=5^lckUtfOfu1x zm*S5&j4ac__xQ*c4kKm3NnsVeOljoxxYG`c4B_y00ig`j&WdISYD48wEt9cH_osjrM%>TXPK+vNuKP4+>ti0REg9 zr)EY!R z&J%$I0&(wi)$Cg@aEjQ_a&We#2T?$~*dj5c?9#y84~!`b(|E^d!(18&J zCI+ZfK>0Bi#^&FlMK&0#8)zmF#tL>a*wp0o7l4Yev2~6l>dj0mLHmB=QVDg+sGD^% z%kE9gLiiig%jqru<1KgElhvy$(GS(8;fDm$QN>fL`{iy-u1xOjwhDVng!EhThcr(5 zG9T^9qhev=b0k~3c#heyBG&n~+8REO@eogKg6p2n<`^&_qS9MI5MJqX;p0aG<1V7$ zOZk16My)GviH}>;GM=$@%XT$FkTBvT?TS|_UJ7yK{%;0i zcIiB)pUPb>$s0de<&t+4K2Q&NY-=}ScnT9-4r3COjYe=zNe$^cCbE*R^a#pl_9?>Sb6O=z-Q8|0t%11eCq1-*z1 zoBhb>t?5>D&2^s84&3(n0mrw>rm5X&KxhJx63ad66KCSLq*PLsBl7K)iu@9TY7Jlr zcdIg6hkx|&&HxIgqB*{NmC+JD)cYt(OQ)auQyUR# z>p1}CO|}|B`5jFBfP?2q4Px5`F9DW+r4|4CvN3Rr!Up{EH>iSk6|+l5ms%Ts%4)zT zX|a`POQIKkEswyhYmBOLP~;_FkNmCuLvkS0=UzH$vy6ALU(k4Vsvj)59t44JGB#Wm zRU5^OwTa8c#k6FyQE$3_Y7$p)_InT^uqa4;Bos)ZsKcZTZ)bD`+M9dX)sG_jYT)84 zrP3z@yuue_{2pSMx)^z6PDow3fb2`oYU^b%%vA`zLdSfUf%4Q{u?-6?Fx_Y&wZ{Yx z9ClN}dQ*$r)O0nlLLfpv*R(Z4{PuN9GyySlQ3q8XLp~3&wO#gS7)xAfdP5-ilj+o9|h>n_$Fo})_OjR?q#}dO9 zUgXkk+TJK>saf85ZdeJ31VrxvG*Zk8zXW}X{wOkCdnn*3h(*&x{#%<~gAwtW4y^Cg zWm|~*yO8z4ZT7t5VXF)5r^dSbq57aULZ4I|jgj=yr`*@Z$0kixyagEzFOW^)2_G;b z4Y4Q^oq@;}TKWO09ea-kXa%)3+Va73iakXQI+`>c9F zR&eb7Q(LhxL6=uxsOgdM<1YHpA3f`}1mc8P*MRBMRfccXE)%D0*jYanBA{$vpwHpY zo=vNA&*+6U^g70n5ABarEi(84J*x?l3^u#*3vP^?2_@S&s&c!`Kw|sx9J!9vDB@ei z{R7^Gq?DD8RJKRMySI(4ScFd8fgq;+q9>6nCU!&6z0;H6^vuiclvR|=+% z?@gk>raPmfp#r}t9e%>Do>}n;{t{*hYo!B)2w^&aKySmsL~H#8cCg2hi49*cnk20c z($yq+Z$=uTP7os$vQ@*`wa3k~FMTzF_Cr<1VaE52gN);heLDN`B4cU9GNsHAF(_nT zC<5dRe5rw9FB5EwG#vI8+azouUYn`gKu>%F2APb-)wCXtm?X{iPdGsfp&5Jq3 zS2}|VW)GQQc(fXllQs8b!l8g8`v-lq-AsDJhS=MEOm$p1uq9G5d>e z#+=v}F>A6+w9%MV7%%&#nbNy6rW=^S9 zWuyWoC=>vligj2ersUJLw1xA$cMW3PipDu6jg0}~_sSUWx!RPa^6KRRrK@weRT`_T zgHyBLq*91s0z;ZTSm}Ao@M9lLe9`FH}haMb9VM%1-Lo8yK7mxx!XEB zX?oZ?P*IsWIXhW_YdU7OPNr_Y>gJ{nrvH2o1pnZ_b)G;~Mdd-Tp%juhP9j<#PE?dF ze0jq_3=JVxs2e2zN1vS)yOk8?b*H%Js->9G(3|rF4yEu=u_4RdnJIrxXO4Vh=c^B1 zs$BIJChPst!6Csd^a>wS#fjDFpB29*N(#jJPL(!9BLlh(esH8r#1REeGA95h?W-(z z+v7#bUlrLMUTIoeFmJOWgNKBjTAuYDlVrXD$iqd%Da~s;ROyaaLy~i+tP!YL22B@%ng)Oam{0S9#>T zcJZcOCbbs~1Hw0K_)Q`*l`25HdG{J~!MSL`PEtAAjY2taX<{spovU93|1j1RsIdHf zAzizW%aOdJ(^)zvNgul(2HffH*iG4)JfH8`|sZVAyZB516YRvv5L!cM4 zLq%RM%}U}~PLC`xNKO9L0PpeTg@Pz(ju=9V*WB+ezMEd&~3fLIa zP<`v2{Sp4p(jiq&ZDT*TYtm_v}v3)6L5w9=IEp#KvfrngcCAXhX+ z>XQW2X}YIm*$Pa6G*9B`{GuW_^?Q6tGs;-n#iA(WvkLhi2*eq!4B4o(r+{UpKL%N{ zJ+<+0cBt4?7wmZ29{d{ylB^=5H^CPZ_PQTovA-P@iTD`H8fv%=vIVnjOKj`D+d3kv z`Tp6LvgMveuga>eiRPY`=TY%&f9{^DadKNXW?NM2m?0S4YWx+L#A?)T;H;Vja_KZv zCP^;&Gr~|?E6Twg%3|)Ri~~%gjVBLk%zFRy#>_sI;RV>o)y_wd+}zgISj3fUQpYZ3 z(W|V{jkjf-Y%O+lXO(WG)UH6(KkU3;5S?g8+o;{QQ7mLy)r(g==U!D2W^eMyFs$t| z3(PkC1*E2qqeYS<@yC?=+95V+Kq;J{>tm>32^B-zvr{Tcfv#)5^15ka%17wdu$A|SEZb4vD76Ooa#kn-#$v&>`QL zosrc_*8|PpP z*?D|&hqt1GR82xAP4uZ~l2=N`{@jxSCU*t(Pcqr%Tol7A6H3$W5dcRWQ83>~Wte&H zhEz)Ge9GZ%mAuW$Q55r}%nWe}kN}g#ZA3zuY+J4yp-czSoKwJSD%COMxQaJq#`pSZ zgNk%1JHoM;Vi_teLOGi}Tm%qBs=+t!L)a>Fw&|AC2nruUE1si7!&y&_dOzfu$SMPe z3r;4ypFQF|ylF?iYSIh|lI#?q=ciTn#}o3|!;RBF2L3|vZIV_?G9{U-+4ZwZM8bNX z5&r-Qs&|;)a?*gM8qOv#>F_<`s@b^33cd3`-ol!0Fq$4jhyH|gJV0|8Q0r5p9e*(# zJVLm@n3wymzipiH4qCWGuoD&7R?fJ{F?j>bAB$e12qeR&51xHZp9KCT)?wKOqR)ea zUDyBb=|zz4-Gf33oOTLjL+*W`=R2>LRczGSFb#|lhBz=O*DngkAAxAee!gs445`eG z>?0e?2TYSd+wf>|o0}=$s9aop-FD>Rjx9b&nThy$l)mY31V8`&2{$)!5Vz@@L~so} z*o1;@uQ2JL<7oShZn|sx;@oJ&@WXL9X}`2SdhyYb{7;G^plL}Yd(16G?Ct?ItqM(I zB$?w)I~PN}p=t7B7%xJK3)OvQlJts2utmVr@mVA?Emu%)Rv1LE>g7*&QnZvtsT&O^1Jls^caTiscm7WcFX6~0X^VT=2)8SGg~ zW?wm|)SIm*l9KwKl5snZdZ)`2(cvF&{!>cT~Pm7^nQZlb2S0=uoig7RSg_O#J@Cc zBZ((#`-{ZpRl3f!08h&QC2jWCl{x+xyhD?XPznFJnu&{HgLZGuvUIaLqP`f&*dRH# zGKZjteTrq<$j_u9>hL~x%OzT_tJIt;BAZKyjwhSsIXUVU>OMIx1w4d->MO%=r(_x68)EJZSp>%*o)-VVj(!&kRjUYuK( z<^)cv-HxumiK|LLwlp;OC5@#&Nc7c*Bd(d|YT6kV=uSe)Pr9YtG39^KDEQs@MI>eB=~_UEqB!=-WBfKK z^8%UeQ?N45_zjxdZ~Y~77w!GaFPinoL=TQx!Z!dQMJc^}h#YFy@S@hH>EoyVzqO>fpG zZf{6n?MyDzmD*$HVn!G~gQqvwuVxPeu%e5&M6Se}DJ025JRWF+KX%!veCE3^u`4q` za+JPftW*Gdh)*W=?y4l|Ss38OD};lrW%t|f^zJj~RF_&*P{2{tX@)-p1_K&}TCF0_ z$QYg}n;)Ei{#Ic2fDq#0IG$nVse1q$yc0eUQQZ9{Zo74pE0#_R`N*}*or>Zc)Dv}F zRq02J=5EXxY3Fjv%QKb{sssvG+XE5gduS_n$)U(pW~tCm&uX+CjI+u}jvHpM4zH=n zBlCltBbZuj$}-`dj?ej}dXBAcA6Kse0dE#Bg0Z>AbX%A+^`ETs80tB_=yw=-Wity| z?RfM>O}Kz&&u3V#M^qTge^Hh3Q4neI;Hh)`S7iFL)tnFqRO*@u(f1e>4s2dC!7O(z8=UUc+p5-)iIQfu!mzy(N z*XI3UlTJUQMWvC=+H_}d5I$+#;!D2_sB)g^cHD|RDvHU9ZS+fdts#50RxVH9QR}Sc zq=@_GAZM5TJ2?oLyE_b;S8IvH_uD#Y#d{QKuqbd?{~-bD>a8as814L(r?GI(Nre_# zW^RAKK!7|sWB4D($p;KAn_y2jFOMI@+V~uC zV8_YwNuA6CUfwt1T>5NszhIz8<9`e%+5R;VAJXep(HMcY{$QuLAm{Aw zRy0r^;ixewYO)(@6m*dgscO-&7{1NSfL1hn3h@5xON$6x$-YLiBkR)mm z^GYr!`BCD)J&Kj8HwE}8cVHjoOx2r;pZchDP&E92sZBLXRV(*OaZ@Fs6%4nDW8)(; z5Fk`tK$we&YZd`Xl0c^uTYw1T$e`iDDKt7aG&fnFzmEWdLRPM@m{nm%OW-)y`gSUU zgH*3cJ*ls;)L5Z+ZsDUgLg7(fKE#u%WpBP#x{&0Lf6qaz(Ex}*>^>=oBLWw z7QszCh7O1(A99J^w?%P2zyWC!7@7Oz?xTQ`k-u_v!^sa3NN{&200>3;ISECEKZ6p$ zB;wU;b_!%^t{lrp>@v`ukUvByp~z4ZA&aTR5LzTV71!%HmMHscY%^$52GJ(2P=X6% zw(#>cqEGj4X<(d24}pAH0z%45mH>=hfZ_mP6k~{t`ylk~@(o}`*JI1Go!m-sGoaWy zV?^|Tz!#9Kc+mkmUs6+_m}Ba-`S#NH{4EVFr@jQh`kOmxpifWIG1O^&)gXQItoGw{4Xu3X(r>*-iZjiY@-EyVvl z^s9k0-GKLwQbBK0^o#t{l7?J!Q*%C}{P@fkQJUTT1!JM{1Ucr;>E?UieaUoVtoimg zY+n{g$jj+M8!A#+(T{r@IahoeSf=WqoRx>+C_)J)O`y7v87=kf9l9VV1*MWK){s~I z(R{R5(STGOH!eN0%4P+5=ZW`?MZ;X3A7vx##J-FgX6p3WURaGmAD`s53Oi7}A0I{4 zN^E_<33>HNw**Z?SZQrJFm_R>rd1%smpm`Ny-849fd9|TN@)7mplve=(aR7P&LS;S zc9LLM+XaFb7h<_^R4fihoAlbgudpp8SNEdKF%atdoV$o1S-hFA|0m`PM-21Imq{5K zdPR@!3anE3Q5-^!>Q*jh5<(8ZP@Mgbdwecm;;4w}#7l_FQUN#KBtlu$=-bff^_ZCg zH^!ttYWM)XA9=iS=Ua6M0#%Z97+R_O^!iCc+m za+e}z#xG8B*;Ru|+)&fiOLPuxKBQx-9G!&WNzf=8k-SQtH)mZR< zv{5Q>U^Mte59w-(jMBr5wYIdJ6yDaB8eFl0u;q>GhCIpImS#Qg6ZOl}Z{k!})(^w( zVe!se-o78lB9l2V16#=x^b2OW4pdKeB}guW4p9BOK$(D zEYr`$R9U9eMd8DCd*U;wpL9PnNry^X>#WieK((63cG(2RchLJoj9fCLaOE<7iyD{>{&MnfSE2h&TZwNDHv3lyKT^!*C<}avFwL!=NV=q ztz3<4e6lUz3KR;18N|hLetnoMbHQ->9+c#-t|PabFp^(#1vnsHOf8FacaH+Hk1*Mc z&=Z?I)nMXK0gT*IQ&kvtKQa+(Yi&m5NRW(ZpXn3{jy3Ejm@%=m(jRa3Y6U9@ z{9uia`Lc9lMqXEt#$W17%$QwgHiFaj(>X&GVYe~W==E<078aCqumC3T_WXa~lG0cC zQLxj&dZx(fV9zkzbbBil5+GEm{F#}23q{cW7-DHb@~xh^8adUd9;uw7PtJDc3i;;x z^f^=E8`66e=G-vcVUy?tpFNH$75l_wPbRW59n*5pvT4p`VfU%+ z)es}&`1S{5NNzG(my|rWO)I5v0h85$yx7rBBy1wC2mTEgc1jG@90yNg84(3;G!jm% z-p05H($pyX+LT0h?H_LS{L;cFS)Y(oWBQr76d+Y9R|10D83rD7X{$z2GHPI9TPE2? zw)M$cqRAg(txs&gT_s3dj!JdQ?<#Y&^Syh*f{`Q9+jPspYblwJ26cBH+5HzIIRrP? z`Z!{#u0eXV1K2T#E*6}44Q*0v<}h!cdSJz#9_qyAp08Zt84gW5#pQ7)v=3WYUj<`|;B-n_2-bgpmmV_`^Yihts42E6w zN7Y#%jGE8RIS@u%$~q^A0YCIfW1&c4eU~~&;8>}*$7b`Xkn8c&{9RcEe0}wU-6J>_ zQt->TWxFsP9Ri>wz^V+I5J30EnmIkQe>cZ}2#hT{Fvq0~Q#0Y9WiE6!bp&YmNCm?# zIu<5+K4NQ6_fo~ahqf<8Nm9V4qLdzKd11?TF&2fbH@# zmjQLWB12XAW_#u$-{mg&^waj-ANM$Xul{a(3LJE_Qt*ua8+h@5^e8^+qM-pzG?te| z9(H(rSy@>(sUY$kARxo4(Q)?+;loDgps79)r~PCjNJ{*skds;8e$~oSBX9VrYa?23 z(~Kc`t~q+c&ipkaX=R6txJmI?MWk&d9=bu=x<$fg}aC0#$0`q}+ z0yFNxMyTLMo~u{+P%Q3U;Seag`pC-lCLaeT3iho$I&;10$Gs~Z_N_jWalJ{$y~`c~ zR~|*V-&o_`^$(qG4v-ZUR))D0r{;$Z6qoGGf@wCm8Z@ysxf%?yNEHpLqkpE(R}Z4ug- z`jK#;4Zc_L8Ymocbygp{ozRbjfO~4v@ZzB}lm5d6S5W-g8QM=c+xtg-2rKpVPZ!iK zykK_{EjOup`3TMPKKLy~l|lgHP0fD6ke->*WKeYfZt0NUPWe#BISTYn0f5LKB{*NU zKF6dB_2kAA`V2w9lP|w{<5P`;1=*4*)=N2R_BsGs6u7z%dG8uxjQ*On#|s~jj7oBD zF5qy4Ahr6AJG@M zfj)E}>H(&ZCIl_?Y6_d>eNvu+p`P>K6l;U>F(0SS;8|EgMDDGjMcUw5j_zVU4lYxi zwwa1RG;WPOU=NtyYeNUY4iJaXKpJBZlLI#}7=*{a;=fu32@ljjw_$Xr?*&3%(YB}W ziG)bQi-=wG4X}sMA{;=6LIa@lsqtwZiT@cS8lSX1M(ACtN8Ew7|15U!&=2ncF|bhH zvIIuXBR2V7dI$venx@vDr6Pe^GXBoxPwEP2jqg4VuAZ`C=CfZA>FI!Jpp~> z;w82jHi55J00X&Fc(IgSlSur&E%b=f-{^gyJpq7$6i#}gc={a~;Q?aM2eEQcdy<@X z@-X+SBfvo3&`7$``_7x_fYh%i9T>GY30?lb$U4X9OuFvPcZVI@wr$(CZFKCUyE?XQ z+Z}hDJGO0G-9d+w=XvKpGi%-tcddKXs!wO1+WVYS`}$pv-(C>K)|D;@D7pJSMhaRw zANu4Docvkj4Y?N-@`w!d0?4_gJ3+u=)3!nr%*1u-_*qX(yN|H@9r_a zuOk~BP{H5u*`7W?0K~6v!?|4jxx>GJ5-3ATuV5n9expa%kViU=muO%HxpwxvTe1l_ zLc-o9dchQXNhD(F?|zJhzv0z*$pl7`9hFZMzRw~%-ju@M5X~L{-7k9~UYSNA#IOh~ z0ThsrQt-hywK%XxT(}Y2T6hrwr?icN=(}w1JAX+~uO?((@tYA?uNucaVJ zyHH8L)(h8pixJE29^mvxZweMEjOKlf33`5s*X|)2sDRPNh;ifceeUQY1YC{Z^ADR} zL+c7CTKceJ9y5JG2CpK1SGJqfa(XY%;4{M9^NYcv3WZ+)iI*8Cr|xd=$k+9GEHRkV zE)#VBG9gTmvy@CaV(B?HiEQGGJ(qt`xl95RK70X8{Q7zb=KWxw`GNP})Am;qfPCA| z;Vwk;0Xo3ojC99vKRY>=YkF;s==WtW3m;~Ua82NhK>eY>j#HJlhLO=4Zc)a{ym8Nw zyIYmpDcA&19q+{cYXk56!77eR>c!x#(?2dZDy*i_U>6Zeqv^NDp=$!Y&#atnxZomu zp1>N&V7^^-O)^F=IMz>qy6Gq$_n;{qQK3;sr%_)b%ud6Q@x;H&=_3(`KZ>zr`5VVY zwe#1aCffe4A+YTt)ku~!(9T?xw~OVP-RW-JBDw`A)>2=kUN+tWTRBawNpIs8D<>{Y)Mxqgj$E+(4R%YE(rFJ$aw}`C{7K$9t9IV-nZj)4yAs-deeb)>gAt z`R8(`DDb4Lt1<7~|@yl_w8?meMJ zRc?XZ&eq|X(Cj`*$Kf>ws+-1D=M>ALwy*BSO3cU}0p{W%Lw*;_UFauB17MwRhxblZ z&?g)#U5U-;zh@i1QHf09X{8+Q;2*uT#HzXRXGX7^^KR(`?lF4``tG+_PqlJyoiMZA z&;Q;s(fW(gEnsA4c07L&`$Xz5xGqg|v zNhso?(&@o4!vvEj?3&z+`yPhD-qzU#(%|Cb^GiPCIxJGP%;HFhpmPSo3S-9N@OvuY zo5OlX`gIDYySXsRbHqze+jko4{OZ>6#B(-h+Y8c_!7U^5JG8s!(7U5&VbONsM=S4+ zG<|QoMAuHm8dr%dcY;Ri?7h!77JQKPPRo*^LpXv^{nO7$dh<_Lwjv>lC*zc#Npm62X z_b{mlH{#`JSq4Tzg~%$CE-`_!NXWf&Qk%z%LR_{hlH|wKTqxl3=CrYhtwlB|CM%}# z+mrlP`@P`|)p48fxfd1QiL70CJcdye9t%SSQ>z%-lzC>1zlpfK&?)pXe5;Q@p^81h z6lypl?=9OU0tAck4EqS^@Pv@Ta^8}F`sj#|z|{+L|CT*^GVvYE{BJ7+Di`VfHD`#} zX3YSM)CLwH{-YLnIxe|z`x z{Kamd(d&$N0PK@qJuMyJ+R868^h;*&CQJtM^W6^=^3bkk8IM*M_l1nq7Y@Ut`Hp|N zN%6j)I&mLAgzUSg%Y&~s@wtfotn{$T%)U5k8^hbPwbmft6=*DvEs**KT-4M&c`Mi~ zhk5;-r8~MPSQ<;!r-8ZCICZnSp&xX|Kf2VN%};Q#~8>eI6bD`QF5xE%4`I z$tGsH&T@#TstB8=Zy&Ru8}6R=;km9RPph*MlLr#Z)4m6 z8RVJ9>K#U*jAoAYwNPL4{>h@%&Q9s7;Z-vGQ*kSq9p2$t>6eiWe<7Cbyf9W5K%I@^~8%ze_4lk9?MAv!?WsASw0OA;B1IJo+s0_6} z-so{xAz7LONdNzwVN#=lP*=PrZ1l66d zuj|PO65M46)gepGx<20dN=4F?IBIUzC$WT+!{wP}Zv|HM!}p-NTcA_hSlN@;2pTi{ zbiDrF*jBrIZ7WLpj%`mGY5bHjNv>F^b!$JyPs^x%W%yT4tjTaun;Y{)Dss zR_9ZQ&dMFP>s;6gd}DMWsLD*ioM+zeXGM@C5}Cz5SAu{0QhoZBI|26urQ9d0^P383 zN->Ix4yl4XSurWmg1nGKmg1lIyKqqU6?o_`=edlpc#Dg900);3i%_kD1ew(EXN13- zsuz!iuNhSf2bF4E7REr>1b(-XS zbj}E+lct}I49g`dH?4lkYo~R7!MtFUBcl;B{=l1hZs+UH7r)tl^fx(0#U9a(=xhMoUl7$K8FUEw_u@wSw^9YI%|n>R#NLz zcr29z%(Zp-;yfma0ZhVEBbF__NJD^F#MqSY?Zil(b@@h7QvxF#HPxAGxrDmwoydgXTyEWY_ z;KsN8{2Q!56P0{TLlG6`S!6-=jDJRfVw6q6EAlrwNi6CU7EB}5&nM$^Xi+7mi-(@n zD;;Rw9r-;LE~uVdSB+@l8^@IV!iR|BO9@9itFYp8p0NJDyo{**8QT`2!za(hTdJ6u z&FAWVLR|858bK7R6c=^8Bo_>Yp9SJDf#Rq?SqKF5uM3Lf(+amm&MWf}j^2|bWW6*t z*etJ;@+lUfSMUkh?-mJ&5@v6kQHRa<;(gHP1|aEo5;V6Bzg(SZU9`E67MsSYE%zF( z)j1Km2G%VTelg;`cDbC4B9+L%H)YSNP*lJ%h;@(n@>)SfsB7i`Y7&QXzqwZesEtDH z443?9ywG2zOZ;L>h8ZVDfB7Mt*1hTqN{97%*oly&p&-|CP#FgD5CW^X_6vvcVyX~S znLUmU+gr&c-yxEOzLS$pd6EDACe~N@^ynanL+=M&&K-#Ht_L<{yzuS^88CD>XG=zw z%KZ#H{S4l@$^PbVqRh8>MP4Wmm|RLBLMfC!_?w_p+F%-QilaRZcOE9Y@I#K$!0cr1; z@?leG>5WIUHMppiD%SZmhmWt2u#aUi^x2uWNpL&?DTwU&*vF`~HaQ^_2u#AcUumby z32rvEl__s1f)M+Os0CBQcjSW)?q>uvQqi3CG4O=xnd#iNt}ZXdhTM{v^>=eQlZ5j5 zua{d7IjfScbI;9Tt{af0llV57s}juRJ;4YK5F}pCANx)3k8#sbjCj;&=J+W^caY1t zvMa&dNsSyc9xZESxC4=ZStHFXOE!Ex^@RgCbkg?*5r$GDH}b7)JH#xA=T5N_Wy#f7 zKez_KcZd(OcO5 z`i%YRHeudNYH9t;KKd;7qqRoT;yh~lrb(_@2W@i=Q}lu|#>F`Z5X4k4^i>}~v?{6akmH{Qu!5lW zi;8H5u10#*Xv!E*fIVXdm3?;=wS9JKK3?fM|n7HP26Fxp|iE)6}NBe9Dy`u8yUO=u`X5-bIIS#B6>N0cd-}l zHLj_;)uDHqA$dFZnT`EqQo)fQrCgZz5-ZyZn5op2tkvcASty7?Db>2TNZp85S!1=l z=wq=Py7atB1DfOps|3(~FV5JpUEUOEQ;omtCt+x=S3nj!W&Ka4&|~OaE%L|_hG_DX z-Q$|NQunK*(o+-v-tLB(>r_8~*FIsV%wWn4{($WaYER3_QQ zNMgf#4uI2?2b1#>(obGBBGY!LZR;bIG>%V5=@}MYhMx*Qh;X3^h0IIryVnaK=4|bv zW6=k&2U1VC2*h44m3wM*B(avEd8n1f(omCBD`ZK|f)sFPkKj@RTIOPwnRiw>8|Gwg zwF^Qk)7a>gvCC-*f}fIeAD5MWl_leH&0c1G;EV+hDxC;=-yy02wLzE@u*hQIhaUjH z4p)4@5wky=gLz`^(1I1;wI5;bKpHENeUebBdvMC*l6%k&E{QQ~iYOY_aQ4C+;)g%- zl@De8(jCRNkz}ulvg%KRiFn5wYm+t*XkR--8xCR`N8Nol$Nz4bDu@x9OC~0R-{Sfg zm@d2GHoGt6o`l6)|84)Qe<(S+%S~r%jr~Jvw~AxZD+_~txhZsa`~bZc$J7@$26V&A z^Srd~ji18}3zgm(U!_=50U(~m5LE(~Ha*d)N_!2%SC0USR@_664|U(W?6nT;*J~AS zB*MVoI~vP9hCiauDXgu!LqFPfm&e}#IIOxB69J~+-+#%qnQ%X8!z4xZ3@DfNaXez} z6Jdb$Tz`e2&EszGlVltbAbad1KuoYsRz=H4zGc@;>*WQT$XmXmJ(qQE%O?)ZSjd#t zEOB2>n$8`bFJ|zop4&GPj(;lRok^(Z`NHUm7^7L~{LOF#Y55H868dxFxfbOjPztw1 zHC~Mua^_6o;mf&*z-om5eHA&BXovp0?>ir@D#swy)f- z+15>PCcIw8nCCxmmn9p((SE!`>lTgb=o*fJVp6mM;iw1cm8{DXQrJ|+EC|bq(Jm82 zxIYFh3AQ2~Rxn-NiKHS;xlSAr@M|gDzD#0`{+rZBRW47~EaqP;d+*TRt`CN%9B8K9 zYyKNDp&W8FS#|R$OtE;YvcD8=V0WboPNdYRp-E>2wkU=8&dA+)ksqGRmwEewuh(JQ zDPUtb@7pKBOZy@Yf>5P=iRc9-LP%n_-l$LWXy=xQQrp%60I_0SCU;g) zatfV@YC=%}#7HuEIgF}ELXiqPhe7zV`q2uu)t$P^GO`td`gHxYbTBDByGk!JTy7{% z$s4~sOJq)HIC{|r`1Oopgev7=842fIxVg6Ul;46JSOum|n&CX~)6a=o1!V(LP`_C5>fl;iT{jq6(bj^sl zTS}o5c6Xu#{YU%C2m7RF5Dr54tBHkgwl7P_Bk3)lTURJ9{lF#xdK*Dm;-VM1>3qPAs)~>-o=y9*evlA;-BmixQew2BAyS!5B)pr_&s8Z}mj((I>4aXdd z0CGZnHqkl;PO(p5CHu0s6iNO#A$t*ws5hve;^-2}YQlgefUKePH&leVR#_eVlB$eo zLtizveQN%~WN-cj5Cc+DCRlZ1SYp=Sjz7EUFji)Z6Tk2x0?cquXWyVM1;!~e zgQ<25<+u!3Kk3J0R*lVPwNw^A+BOUe%+2qe{}MSIDK`iOnifWMC(IP$_nSC%=c>v< z(>pAs8FL}q7%j5z2z_Ikcg*(pX$#=j@g$Fv#4n3x6Ls#nM15lAEfV781-nFpaab^b#V%s0@d>6lrRvMxn}=;x^-?l zQsl?p^0EYA46^~}DezEytkIXn zg6T{_?cODZN9Nw8gokEf4Ck*8&JU>xwq1MI3MK6$ZCfC<}gVf19(-fID1_%drjE7Hfz{T5d4Vs z%xD6fz3#~A2~0UaPN73te8Z+x%MAD-CJY|82C*DN;`QBwJHFZ6f;6mDh+9(Mxk^^I zczYcVh%tuTT2xEgmpuv&x>p!-jayjkuQv!AJmMgofyjnpXU=zW@z5Q`MlFDsW>UB6 z<*iD}K!hg>;-C;qy0TpW5|5vBY1R(&WWI7&r?WpdaGL#`ziCrKy zB~L z*1%48wq<17v_5Jm*Y354)h;vplpy@Zp33-1;Q1;j(!Y+JV2wVCSF3y;^-Gu*vv(g9Q5E9VAcL?&R{f21#Q4jQe?$l{r!Bkgd3h;)*vhymF@d3kAo zol+|y-VLp_TlT0X z)N%gCB1TXe9qen(n2C@qgh7jF?U}=uwk{ZxrOGt^XyL{mB2~hHZvm{!M$7>CSd(^6 z>;cBK!Wv;W-Z+XT62&BoDCOBhd7Wx7-0|1qmr%Z)Q7&N({zrWOgS?}iL9pU4Si%~c zCFDB}7>GNCsf}7JXG0!W0NOm4@y(as?#{Z zT)L?zvty6|{i)@lksqLhQMoQ<1E#82xEX6h>gOBg9eZnqm+KD-H6i+mf3KY;+J&TZ?VM_! zKTE*5(|NzA2qj0Ut)W@0VbT`b8>(Vo0Ii1PiO)6|ZW$b&xfR3BX3-{_LY0nI$vngh z%JNF`H1Qoz_02TZ?&_j ztRhD{ot)RCFt7{oWo=u823vRzL~1xVxA(woN4+7lhxE5@?6$pp6FcF+*|#4wMz9ADQVj~5!z}slc_I+ zQ%#MF`;w-)ea;>trSB}+Xjuet7p*n~g8=N~DhhS{-v#6~4{Y`E0xI$fv>9br0_^CFZ*B?40Q30dW4_)25C2JzLUXG3A7pLl6f(a0=7#FEeGQ*6sw##o)OB)gMI4l?;MfmCO`EaS zs{lMZ!^16C-8P(s5>4K-(^!!32C-jp&Slrm%H%H-?lCraOamoD{x4GqeOR$#9g_xL z_<{zh+RJVgHH8ZDtGnEO6_Xoo_<}k|jMxE0grR2T*|S}ccKsu1NC$Puhd*JahbjAf zIAJ({eqmNoNZ`AkYQW_h?0JA48GfYggLuiUh85ie7 z#6`isZ35LoG466sG7CRZ@8uxpoj?4GV(K)6b8`9#j}_fj%#hyRrb5igfc|YhdJ3D; z48YXTl#t>wUtnD*_-a=|W`zM#hE3{ESwEaDLpyepj^WXAR^!*=jhtfsRMx#+SQ6d0t;(>7j$=D zKxCylhrVk6C>7^4dBb3jLdJdQbDxaA$eMl5 zj-1uIXhm65RQ>_F?q*~4YeYf~;G=|!@SP)}wV|_Sjos!sQf~jX>(btZYWNJdU0S(X z2mYc-Imzh8dnO9eFfNO&D!^3z*nrgl5DaFPMmWfK*qgvSybznf+r1De6lS0#Q(O_z zZJHhic2W9S8M{5OL{1yb-9gN!$A!21T#sr*$d=slFN!+5KRg{hod~EW}En)OJ^daFW14R0<1T;xuP1=spKaH1e2Y16OAIc2m|FYQm zSAL#)Uyp#1y8Ze=p$k_1-x#{u;tQ~1VF`6~oyHAOEH5?=N+^zuDq&R$|CSSvo?iEs zmC1xRSRj&89aZ7aOn>fv;LT)8kgAMdnk$>CEm}h zRozDQlEf#76qCJiAAYT#NMcZZ!Ny(x+?=Ym;1W^Av=cS)Xg)_e8hX?9OGq_@&a zq2sJlr15*zV`(h5=^ZXock> z2DzDp;&__mJCB)wlsid^cn|vUMoVVefe;h0bQR zis&xUEPKC$iuzq{{hpcrnL018P)sl%Q^(-HDHI=QIowp@Q3O=rW75ojNaGr+34X0f z=nF&Kfa|z>VsJfe!x_sf=l(M8kL^L5yZjN@6!; zR5V1!?T)}PY!7B2m<^sRy5yPh9(Cb6{z#3!R{r@UD|YmuYsgGUJ593$AM%c*@5oxx zch)uiiL;My@xu%RmEuxYzn^-th{}{H9$*4pqwe)Nf5_#nxzyc7zEE+{ma6R_yBK;3SPyj%agnkgn_du?ANi?B zW+bSJW8uz&@>UtMOiNDxml8u{Tv^p%H7z366Y(xqfi3qqEbsL@HszC?6vF$B6vcXp zr|F-H#-j7^W6PdFYFl-jLVNCa*y~#Ai^7gfD$W|?qE8(ZV2P9M)ff-Z;{@v+Hw1@P z>ezw-Of@f}z`?+{lDaQ;)=FR}xWm0M?r(uC)+Ba5*gwN2bHcX3v4m)(FL*1kQ5Nl+SP^D)-+Ixro zTh;Y%;`q;nB=sT-fdWw1QTvCM2PK+99~tDe>%UYXLtK^TwKH(iBr~6f*`<(xBE{8S zK2a0=hV~w;3r46hnyDf92>8{XuN1LMN*MW+Sas#RJ*B<9Tx>vpLerjSt~C(>VIVBA z>n}Np1y+;P=9lf1gy`Wq*ilpD?FfUx;LGPBJtuKj^|KJb15$2oCfZcPh6J6qsXZzL zyz&H4v%F9B-AF9>tZ}`q{`4jAS^Z_GRPg#eIuw&QtxbsE+e5V<`w0T!{{QroX4F`R%PzA1d<64*S7RMyXR46X77#pB|G& zMOnVR3)$ohUn3RumtU!t)=y%YKv zMP{g$tXpw92RXcQ(>cOoS^AUl}5H|=LfN4H%BD*93|%!?Y7aqgs4Z`HgdrQ z-!^=P1@iaS{}jo6coXCqGrOKKTHt9!l&U@Svq^bWO>fpJY$`rVVZ;LMCrZL30_Qy$uY7g zbvL|>Qw6)UcV5o59C~9UDT9;!RMj>_oNfY37idoiv|UHGO<<3;)IzHE@Gu9js!IZ* z_dC~*?TQwe|3I}b*4^<8ZWaRNdIVj4!VU<0HhgKmkpc$G!hQtpQv-%3W6JgjYev(D z!M6c(QGTLZzHSWFW@|&|!?X!2Ga5xd+ce-U z8JFt!4v2`(q1dE5u?=#bKByqjUmn;_8{j@N=gNN--qa8BEABrv7Xe`VkAZ6R|IibX z4D}G{G2$zJjOx^C?TZt!bRpmM-jbD*>^S?{pYFCvHTwn$$t7z3TX;V-F}_RUdVbp` z-Arr^zbD2oU!Gp?__}oj!Zoy>n8MgG z;%a-$0R0t_LzsSW4C|KE-ZHj9mfE*^*=pY>L1~rWy{7pv;#W-4ZuF$!ecy!YYi3mM zJZtuB+!>(+$UqMLX5Y&aUA;-!%e(FxgqP(|(LB5=3LchbpOqj_YyODR+8DB;dnTgw zZXZVIf$Jq37Lp||D+%2PeIN2=F(@-ysGp=`09e%9Wh%&SbXOby*6hrL3Vi2Gp-K2Y zUU#o$FBH885~uZ<(ZY)7W@A->Z#wbJb~->{uClC|ALfoi#VFQ=v*xB-# zZ7#30{|Ua)GAL?^4fsML!3H;W4nP=Di1ud!O<)+$Qi=05gg+Ds_Bne z351zeJ8(sC{&diyF$2aYqTre&_Q|Ar5?(>)df7O?SQB!K?@wP8d7l+bLG zh$|fs-gu)UkFME7@b- z|h_wP?QJurTRULtGEl8`Q74PY5PS;B_K zhIz!sa7mDukboe@qx`lAnFrZI81=7oe2ZJkL^SzL(_79&Ptv%X?+_v#C?a33B#52D zMp!7R$gIUPWI{%6zVHb2?QeX^7U+OOaieJsLP~>YZp+)or9_wCDG0#BR=;KmL0|93 zfSgNHcY-^@C_C25WAhF5^o-XqX3;Zkr_k4^i&BHV+yTTOADm-G!h(cPhwFk4=i}qo zP75+-CdT6or+p|IHN+tohOK@!7vp)OdM30cslnM&T4BQKPA)2tvDw?D*k-19+LyDN zSy{@@O`3Boj7~i8n$+;P$%>M4bZp$!nb>(pPYv2(n6b5t2#o}m@0d)wP-|rFt24B- zT~30PlmOhZUba@)h)&B&m#Oi!f#tkySP9@Is)W{J79js~uPcwVr!aGlc75_CXxfs! z6dt`AxxCeo%WO@iGg(rOrVWGvI_nCNh-sRQ=?gYg+)5tlxaSES1yL9AV*8_K1vAB| z)Da9GfkW~5Y4!rVRcmh($p*^#bTA#MGid5VK0wegLvy9>4w50!;!<`cjrcInb=b%r`-6=pwLu)9X5sv3zooXY5wdS6Dh4(Ef{A0H)d^S=Vrs%<9Dzs}1 zoqmiVhvS~=MLax|gBmz~#!q0R{CS~HKRd?9?fyC6wK5>aJRy-;22k+) z7HtJT1KW3d!T|wZ`cSpKR&x{7Np+2Od+yN=|Uc?a&0x&yvS!y2LZcfzduWlAg|MGj{uHM-oVVlq2rd^p;wLz){{ z!Ob@sU%U}qj3Gt-ni#`PQY)~hR!p}n1m3}k>%F-4(9xk{g4+3)`c5U)CJ^F?uen9R!dJsdxNySs`N z0#B33$C^XEl+{jjQtAsu6-^jmSxk0W4_T=8z~zUvg+&KZEiI~y^#{M++#{2*-b#-y_dmm5K8nZW3kk*p zB^}DD91?3X7z$zw9w|7I!yn#H926Rhb{vMB8U}I)@($F+xKou=JSHJ;SZ1e#E@wZc zu%UL_8mRUG_QWeEl&#8%1N#qf_6wOV8rKXBsW;1O`X1|h? zNRqTTx-2e|TW_R`%(1&VBslx(*wr@n$_tjincBnx)@X&TjXJIGHs8qxx0y~;R1T5U z;~!;b)JLbdl7D@RyE>_vUH+=}M}Xt5`NuqJv0?1Q#ErdseNxQCq&b@`O=A74js;}{ zb87YA;Stsx+{r1wn4``NXIEnq?n%#pY%-Fjd!fV_&5X@7%^FSlOar*)&NJi~z53Nj zil&V{aGvUIU=2dP#1#vv%x)MYURs;0vn)H53(4!;D7z7;QQ;kODmhoOORewS;^*eX?!Fd^qV$6IgrgCnI3g+jZZfa1ldvO>8bV|FVX2*-gcgGP=Dz%} z7K{}B(Cb#D?K}LoSj9gGs*w;_>ED;^vrV;jklw)uGq6Kiidk^N;b9nG5z1wt{ge%F6T=m+a3j7SZnJ ze}VDwc0n!rq)N0jh@gk>*jTSJruCUtya4q&1mc5jO(oJeO)SU8$7VH^PB8V5n=1&e zEoC64BPJoHA*KL|2;o3T32DbEa;5=CaK+IVR)hvs#gO0fs@~89KVn*rMC1JiHCRf!(7_zZUw9su70J0+B4m2V@cG!rd~V;L$Q-Vfv!pc^;#-g31>R38V_$G=qlzY^ zVNK}PN5&L>*9T4=+u^)vd`uF#?tekRsA$0-^5;*V4FCH~?SoN|3rKi5+1r@excz5e z_*HpKvG3z_4OIC9qb^`HKwL*e!QQRXS&W^<#?9YZs2T%$2uH0XXlk}!YPS#R`a@q5 zhmkh2e$;lfnRS&r#q>$an$MconrFG3I93(|K8C{ri=&mXx$e*uk5H^(32Df06ntcX zx;0`@#y%!>Obkeb_(W( zqVPS&idwO4{5)rcoK>W#31-q#NwvRkJ5Js7kINqcI(U=vv>_-7%_d%pY;Pj!r&?-$ zwVpDBq{9J$2$@+(H_y)uddGaSW5w@oWH&QrOYm~IxA3hHsFXq&^pb+rw4b~Wr z0j`C{sKTf6nE$wpr`>}VQGiJ`>j)Cy`GwqPAR*H%k27z8|EEUm>EO(Enb+L3M% z{J`R%Ylz{P(7t0Um02xS^6`_DPF`_KBDrjc{usAU$J)kqLmb^@aJ?!idmdFNNdLDm z%v5UnC2N%uE)8MlpB#wLKQX71BJd7&>>#*ZZC}Bu!s`W~1hw{kJM<%YcrL4hnUH<=gi zQqBdzVp9927a(@h#2Y!SZS_7q`wpdbt?@`6pq(QHv3&pxDUmv;+JgpU4mfRUrMp(% z{8zEi@0CQvZ2K0x2g_ zkQ1lFm^le%%8NatvT;}mJIa{3W-q80iL%>*L8sIvr-BWD`rVNR8#2i}F<{|N6nPsm zgyX&nSn)1NXi)BwE>!sP<+@NH+J*Sf5IPPLc_rPvFyaod#Zwp9deh6wkYU^FP=v=N zgz2;1ZvLq4R5`FKOhRxgZ&0+&RP;^aSb{e)5mXUDq}Fw3 zBX0~{gq;`kdJAQJQh0#gDD8wgP;71gfkQ}d9OP)-RntR~ID5mEoa2)b z7qu1Bw2jJsA4qslv`F3G)1hsim+P)m>7en0g<7FcnRC<%_yfFKGZYi&kfzS)+QVLWpTo^H7~;=iDZGVql#_NkLi#J@O}x~kzGi%7w533F^oLpm;eG9_g7bgn@Y(BS#M;L#exyIGUo zvWbE5A9j0OVGia+WR3gvFPyX5&qAK?kz{$3N%7xM>_f|E_*vW2abr;4i)7wx9de8E zVfx5fdIf5`rZ5)LZ=0k;1qEGQ)1IQ&2B*WegaZYCK!$)-yS&Z*QGhMh>FB4=9#^}| z@xeR@etLq`e%^CLf0){_&oa54Kw3_kWWE#0{sgZ?vzp|OIRJa;K&=^8AiPpt`yON< z6zaxSio7`J zDq;;FZ{Xl)7OrvLVT!TCkhAiuk9`e(0w5$Qa5=1{dodljAGdp+iEz z=PI*svk?sRqDd8JQ zP0#@h(CwfB68?KPq4E6dBq*iO-I`z|#GyhNx_Z}y*zQ*T@6YCux?37W9Y@sP#j!JU zt*%=sXEa8^Yw#!4SfJ$@juGWLiT(B?zEvT$J8I;LblO{sjCa&(C!K&Qr5BSWPcWg_DM z-~RcWtAZ@hQf$y}P*h!JjG|uCx`CUrAQcwQa&IG%h>J#a#far%(za%Qg}bjM0!Ct6 z5KH-CD(0qF*te>gsro}NV{B=nDgkpWyKW@|79Tr7PnCNE|OZiV%t@y^q5qdoKZ zN||v@3@a|8gYil)m;-NXtR5oab`201s%=_!%g)7WmgdQ~)AdOFB7Q9J?%v2`)-a;% z!FPqdxDL3gjd87CTVvx0$*S#z+FBcie?Af|&4A8Svz$IZO_?3#H%s%5zm#6(Q`dDk z3XLD%jG!hl_h7aDh8-!++*rcG1tL9ra3R1;KK)U552TpXJ=^3A{49ly=>kx0K4Zvp z)0r|$3#!hli;JrB4@yn9@ThHXIyUAEAURsBP-?^4LxFeyzNK;~{j=;-CFG>@V z*DOk-?#o#x$RECWnqbEdSi;vC~+YzZwtrRQdCO|EBykV?~NU z7m4oE;C9-g-18$?B}yB6s4iG$A6r!pC}s2QfNJDTTH(S(tm>92h}4QUAUj64a~{-EU`U46=6d;c zZth>GX?T#rFgNdJW!p1npp5y&+E|Q%_T`DUweX8O$`?Rk{pl}4 zh^XCVc^Em;{x`Y_d8@RqwdEgN(m?IMx*-+g%uh=CA#am)f3&IU+F%zXs?Ifu+oM=n z8^J;tS2eU4V`28o@rr81Ge{?DI|2y=@{j!PAXlickv@D@$}FY*VKr6_);J|HrvKcs zjSmUcU?$37KZE^~P78?H-m$Gd0z~|8fIfqvfBFyID(=IZF;y-Ei4WnQe_s!$+#5cg zFiB1PxJC5OE#KKwg8xO?SHM-ZtZ&l|djU#GgD4_mCn8`gB6bHN3Q9@Y?Xd$y8C&dD z)MFRcu>iXZy92xZo|!c}hV41`|Lxyh_j*~+`_5bQPS4t7rla%CK}QGw(fIWxuO=$W zxl>R$u8(+u?>^e%V?5Z-r16Wg%32m=M_No$I5T8ojp>PBalzJf+{D&M1E-^tQ<@^P zlFCR~R(9?rf0Dw2QS8}}cq-XSN9Tu!j!p%E!n_Qpup>8&BpF@HdR*=TcO4yrmL(;* z?8_wSH$#DZwvq+&Zg%H!!~OhpbjA$Q(Wx(>H<98Dg*|yYO`#ytG)1Ybw_}aS=4o6=a&Q`? zY`7a7%aY&IxRjZxo2?xoWdlfAS>X0+ACN9tXX8r~$LZX$Yu=Q)Mh8FcFjx}--S~uwWrc>99w}Eqj8ib&!Cm{D6QX}s0HWnMqB;geU8!W3vxlFYMN9ur(i|RMxGqocJ?S(au{I7$<4G@yV~mR z)Gk!J^RS1gT|^OCmZ@mON^5pwKuxxh2Q~;1N?UfHqRNsAvlSH>)c;O?wNYUB4&v7p zpz)7o==|9VFWClpvmsTxr|i#JyB0WU2<(@ISF4}N6meOKiZaC_`e(&U=;4|bx1_8F zf%zP~F?++_^k3%5-Y|jo9C*0lB7})TwW}@&RPG%|Tatrw6!r|Aye{ClFT~#fiiQHZ z`$vY}x2Bw$_t5#dMR#Gj7-SXt5f=6D(ICAqV?XR}Wj4ySqTIUbRB$akR2 z8n#P~Ois<}wXTesra_M>+xLU*Q#=7`_W0MqA2gyAvDj%F85bAOT>f98dq;ARZRL^; zi(LK5;x2f;!X|6;HWLzyKQcx2D$bhVivWs;=wcm@)g>W|6y*VTCoi&<5%isMGcsbY zNCU#T^h~G}-w~#23Qmm#@fI1&Twa~9B%6uCMV9F-2=)4!sb9IzfI7rP4F_vZ7WhJK{*;6-ea;3PX|)2F!q=RcCy@54GQ*2DTPdFXgJjS7T7W>(-qM zO6%x6LvCs+KvTR_PzBk)fH#2SUhaM}1$HxFB)bAMxw! zY2dSz_^-v)Cq=tra;tsGKwMT|1OI!W!jtj8HTV0k(%}CBPcuRO!QCmoqwL&;i(p&5 zp7bwO$-p~YnRA6b+AjxDkQVWn-YTdW*_fknlG%)23^wy3>AUj8$X3^yMIjTUZ>>eX z=`cz~cwB50zx9#v<(DXvOY0?Ib2D20bGW@Go8b--ZmS{ELp)-uRjvCsAm;F`}i| zm%P?1_Fyx_>l(zXnCW*8QIVOG0m~H5vJz95q4Yi@<#(f(D@;lLGTxOLmy_=~jrOGM zOXgJPaVoqSDX6ZiAj6h%D@NY!Qs)>Pw-^hYVk$G8t|D@0Dt;(_pREUIBrS4|?Vrza9UO;?;@s^VcZKFs{x)wLH23wnWP_fGX9t;^OQP&G}<^!sOtL`LMo7G;0pE&iqFmZ<``lHgpW zZV|b77d}_@&YH|>p=Uvs0>FIZ`EJnEeJJq}9&v)Jfl|gVui)iEm4Wcbu6}C($!%nM@*8=vE7T$c3 z2HtWT$8Ve5F)IMrb6WVE#VWj9jr_XYZW07%u0&Xg9W4K)DyRk7(q0eW5mov!s;5V8 zRz)41?#U$+!j)wzBFqAGLKO5W=9V>CcIU2RZIf1A zIzg)2sIOw5&ET90FAqWUtA6V~9;_GR5nH!omt<(Gse0C=YXYBYNk54Hz_ofp)NP+x~d`=_5nk&Lg#*xg?_g~8PU>CL~V_4@}FDoRQ7}h&_AF2 zA^-X2R=v_B?65M>Pt$i}e#`F+cPU&w*c4*+$eIIS)MIFE#Q7LA1AP**4J}&6RXqdJ zbC<%85uCX^!(=blUldDo$X*iSj*s7d9z7h83V??^VRtcJ-ht@2@$ zrYK8HZU$~8sJt1$%Hk>e3jv;o0H`3CHNahyAmK9NcIIBC(o6P2=?2zpr#yH!LroAfi7FgW_%kB~->z_a8-zD3BVPKVS1H!@(yb_7uIu#& z4SZV(D!oUsO@(6p+ESD+$v?=41wE_XT4Ml)D*zXJzlB|7=;29x>MH8@sZA@P^HFdW zqpm8(Jn#w}p!%~7X)s0Kh-4Hf3;RU@9phxcnDYAeEIBbRG-=z$_$|t42BV?N zr}5i@vF|}7p_zvju5vlKhr#CkVENDCM-|9;G7k zV$j`-Ud7ErVTE}7H44QN#dwl=L{U#J@!S!V_?i91ktDU$Hzig_71bGs1jU#GG}7n? zY+4E&R{ljfsPQIb`cXxIHO;x58~dAG0JuDAPcUg!fJqL<_2*ILB&Yu|lvrb`28YE> z^vjZrW85!hUyb7{pxCj{vx!h_+;k11JI54NnV6fle>&?0d#)n7149AWud$`5NA@Kby)jL=9Z1!JKVJnMK_8hIs=sQ=1HziBj z=sT&2hIH!^n`e#|26I%61%b!S(;$jJp)et)9rRt)l*w)zpOiy!^`WXL`i_+%r5)Kl zmAB>TABOI|jbc2Yb626brfW(NWuN3D=-{u{oF+o2a8x$2JF;=DNK=~Rol@Ayv9$Ok z47G5x2H@sNg&d7%s~z-s4*^bS8NE)f20r+d(&+K0z>93tc;}v=Z$%a+>07Gpp*T2X zNjKPII_x3l!tFaq<{5=MS(Ab)udb#+r@=LSxBPC{X=SxrME^W?kNoFrr}z6I$$Ro_J2nXfl+)$$Hdv?kOLJCjukRYW#q z%TcBD0}8?L`C0nTiR2V=(-z$wpZ5!F==_1m=InVD)R35-5S|Xn5cuGviU<9n!85dkc3@819JDga^21B0(hlA76I~NX4C%H2hs1j z(gh7M?KFF;!uyaV7nQDiL3x#Zu6k!nwp>zH^)?sr?!gQB-OP*1N!88t&mZ5&e|~Y1 z&ncW8IIAgLMD{?*qIGY+m7!s|`U+MuGpue*VahtFYzi82+@0}G#xJ|Xhv<(|zVzP? zm=&xf_B9r~r_c_viR)g5Tt`0Aztmt_uyWqcjC$1s#2@g8oyhT@<;1i3io%LWyY$V- z>&v`(B5mE9IU%%mqtDz*@Qcq^6_GnRwp-s$t&x#uMBP)+ULSQ-Ox*i^sfgs}?B3if zRR;ii6QM44iQecE>qlsHA(R>EbqqrJU*$PIxSvIX^Os{UU?Gd1Komrd@on20b z=G{>Cf8O4pRmWHUbM7y6BVRnzw(&MZaod6 zfp_^5f=NHSWmB<3sRtVV5CI<e?`xh~o1TqW457qWBm-1LMr5YHfid~JhhU^h9vW`qBVncBYOUHGu|z%8~6&a9~OXhH1Hx>Pk3ksemM5BG&*PB z?Mrq5gG!Oy=V%cgKT&uyb+(?b@bEI&(Qc>M8`$M0L%ZEGC{21j<@M{}@Tu+ELRuGu zg4nlwCN{=R`iO|)?I!X-wqS05KtWI^^7J}RP?OdiL#&nDeOHJpeRq=^8n zUR?$CBreZ+Xj;v;Yj+R$>!^jI{Nrn@@Cr5FepA8AqsR|wekJk817&>TbA`ON;;{Z{ z+9If00q}}~D9>6`XkEFkzJ89ndbb1piyx`+LZM)KJimFoxC4ltVJETWyxNyST?p|p zbR#)0l>Ncn7hp4Ifc);+3q=*iW}x}Kf9TM%3F1=BO#24PP`_6&ckh?V>D^KImtJ7J z#=E?id@i~6_J9F;;58eMSc?slRCrnS#LR8|=*}T0Kt<`J2dSV65?swtV!IBRtbw7k z{%MH1lZojjan5#1vP@K_j-ev^wZdLDT$$GpHDIvnm5X|3nZb%(&#Yj*Y}5&{AJlk= zjNkj3PjY>&WOQ*8tmBBfAug%yAE|=ckQ#3k4oph_oM9PdAfcgEN&LAnGJfhCK91Vo zZ+1j6V#^(|C6062PF6uB(&sB@J+bF{t7E|aMzw1w1kBp$L@LHEpiSY1Wr)>FUYGpa zhVJ!(!TO=9iZkf9=Tdw%a{Dc>(VIpOH2elFdx3o`fpxzH6xxmH_Gy6Y+p@M;KZno6 z%6W4!MO7sSiWTxSbC6;F_wHcW3|1DM)px5QSvJ-%fVjO=*s|0Us5o=>Be0va9P9Nr z$5u)+G$xjx`F#1_r`@X44%Tu=D>5IxixWHob2$utyu(EF)4lxf&EE4uUvzXeUjiP- zvl}M5@6S35ksg6aB&cS{37r1{fy&W$vcKzWw8ly-vZ?6k z{>JJO6lB0>uC!_1!-6}g)F+)w>X7cp2@ZZ%$O{{1qVw-;fi!ehON{2d-W==wMd2w& zQHw8#q5>cO_sL%rF3e`LZQpJ>4U_c&A934M&+;7moP$6xOMswl@8WsV!%F&Bk;f|;1Tw)m^Drlit0$k}5TmUD?du!-T&VJ_vLq~S2`WQiV4t5sf zWM@@1feDHDt|-gojr8h0g>Jiji&8`az1o@tPG1cj)y2dG?w#HMuA#@!#l-u#jwaQw zZ-!24D*vC0)9C#3VhAWYb$WeGsy)9DXH+HGz3YF#Ku*CN5&RR|*I)loI5P1AV(U$z zEO&vW*jQzTs0jiX_G#O=ej}l3Ht@~iBK6O!ks9kvdK((qkww^6&8U9-cvbmtn7^k~ z(m?i2)Kq0i-Cw*{Qt^B0pd#4D83|CF`>4{2Y&DeZNSu{X8M62nZwcC(Z}+VRJ8nmB zp}blq{41f2jL)?(Dy2RTk?v+XEDUPUZh_d8f9@>fE4vw4l10CHSKDhvwbryfe~d?* zBU;u~1OM$epHtdY?|h#+kiQgU61zTox~cFETAPo?IZoPX2RpS!Ur$s!te1++X6_ED zG;6NP4%}Y692(G@OT^C2p=bur)|D!-0}WfhH|q5ps8V1oCZ@wA*HE%2y>+Dk#oajP7%-@hGjLF=;&Ya_cH?nZukDBoqDz!u>-BS?BXQm2oKqeVT@-C}!WxWcbQ7)Z#6!8xh%lL~1lDyry zaV7m_^b9l|6nY4(=PjjB7cwRa+rujxDzyzZgqRnX(|2BEw4o%Q06DuP*(nc|JP?V^ zUtrT|C56^w5pXGgL5*1LX|6;l-ThU{fi4RHRTS@`ZjTPIXL$#IzVDR zI&_*WGnLAao~Dw#R#jM>vnUAA8GwrAG&)bAvJ;&aSo^PrndCWQj`;EzR0Mw zx>_ohJh7k&>oz!x)%Xq$nNWPUHOS~Hr!Tnryj2@mxfL|7qTxb>!DAW!W;>jI$4rvX zU*&DB`tcxe@2PhLycdq3Q2q>G-<)eXd!0@6An;y@a19mkt#K|zg+GvsLiU^ULRzFm z+0m?C9fTrL<(DFnqgLuYP`-GAK4??%G!Y7X`AUO!<~Ac2^+Bo7Ej#DNV04>+dLriQ zs5dGiduCVL~97)1IF!SCV-f;Hmnb2Yo*GPtdUfTDO|1J3cFmOX=3rf zqyGIYV@%SWrAp-~f`{llUo88HU~(uya&8kSny%`7SDSnW-FY z{^fr2AwcfXLjUEeLMH%iLM8doWra1#bCK-ygWZftxQU4!IX9KZ$GrG~Q|RVdy38iV z$9G_zU6E<3By&&C|9xQ|9sanX4!Wac^>3XA!}>G()cHES=NQ1M0cNMh;9tMIjL863 z$&0}jH%foG05G~U2;r~sJK|$ZHoHpluDDl$J9YCQARR(sB(CvW=xa<~xk~Pg+grB> z6(Rt+4$q^#)%Y#)GbV{PCa$EY7uVUxux$TBz$K#Qi7u~I$(RgvliV1~Hb3_qxd}#e zH=>gULx^0}nCQAoX!DXKc|^b9boRIF!1!rlyHw-Y)qip9^AWLG&4C*TaYeqvs&j0C z8^;Eo-u(SH_;v#x6Q3GC^P0wFpOc9@iN!8tChxq9e|MtTUch>3Aa!&))iNesoK3vh zj%9`o+rK)R9mcRMY1K8gjmZ;tZq=%Gi!!*Gq?$sl7=-b0cHdZm|E(Y=V5 zRR9%z^dZ=o4D#UI%JmIwIUX=tBgEQi5^7AgRxl||90Is9yXVgFp90)IEo=d>A3P)v zrpMZpPse8ha#9O6uc0vssR))WJtYfP4URQEc7XKqL^X(MgcU=7yqlqXw_@n|AhL|U z3u3*11NB zO0}XA#02l8xHNKatyC(AvS^l9)Z))9rYHT01&C8*KCF?fM|mvy z^21~abXP#yazbQX8uSMqA$?L(lOv;2$(st2Qv=E%sa~(f1+Yj3tYV1_PNe?_gXXEx zn2F9&*fA+G#w-e>lNa4#Fi$~>XJh|^3CV0Gl@6p*e0QEbxf>WX(_(OC%6~9uM4u$xw|9K3Wg}~D7pD}b@qQSXrCOJ z9-W*LN!lbxrCDa$?=~grG>lLYks(%gi)F^7O$4&ft)?b6B;1c@Jfr8wsxJkBE(pXH ztip0*;!#$r!1x|=PY9%KA^i$SUBSquRxp?YnNU`eCugG9dKuE;5 z6Ejkw9G9WK?SqdC0ep=|lwr>rCbT_C3OB*|8)cg_W8I-abS0dgmJmx=x|S3COPnG5Eb2fm_4?Mnj z)V~@VjmffhXcJGjg;pxd&Z<=0{5w#KP_Rf)YZD`=Kw8%47PwVVfpFlsoT{ zAAG&?CLKing&uh&wX*ovsO@S3XX3Dt4-yQDJWkV|C+(n?6$tw6QWIE`bLDvnjveEB z)`Jf3QA$;Tpvi6}fv@C1D*5s*aMC{m2h&qECqX360^i%KCU7QAqfM+xjxV=u)W@5S z>p(R_i^}uon|bj2hZrPQ+I)C zbsnR#C#x&)FtRRk7($oh=yobGb=Nw=u_1nvJR^GYihm^uJn8m#v6sC3D8qUZlL_3v z6Rajzq4N{(ffunR|8Q&@93Em?{$+P5yf1JK5IiEb%t?;T#6}o~-8Xga6MAg>A|5eC zUp~#S9^`aiE<=^%sEsp#iw5s%f*oS=8P=VgisZerF%O%3q%*i{ffa4mv5;Z0z?KOk zulMIhe())OX&n&I)h@A5>T!+{IFSZfAW#ijzfN&Ik^#$2MP3ypUvu7=6jzo!Nx)nX zgjM1t)cn5k;AWIShjC)=oPL=RIP0d8yKA{4=P$7=q!nr}9#MgrSCLn$VhE$IOLqBf z7D4WdFt|7m+xr^F=2hXHzZ|D*&*|WqkH;I2`q%NM8f!pYtMY1ObG3~NJvyd90b=f` zev1*vq3w9{(*AaUr2-~)s`uT7c2%X)g9B{%c+ZOiJl2Z1S!Arc&azzNo$13_UOP7oIz8FCi{ zZ_)OM)*JTNnB1!_)nSo(qfgt?T_N;VK+&Jj_-%j2_&O5ra@=OgKjvScOC_g)6^+s5 zg)yo83dO&v!4+K8p>kJu;O0X?F-^I@VpuN{@zg{jOKNg?=T2BezJd7-ErOgkoZxj$ z$(acqICwG`C`Iqah`DXI;Zx2Vmv+@l!rsvfW}@Ko2Z9lAxI8P<`s`e@=z+`DjTE@%6;qrHqO zTGBlF!U*KkVr%YR@ufQzUIFGM@a^#pqc43^7m}qM&d$^iG4cU+DrGBRgMTorC&|+_ zH6_tIxX@m6j#<*%?1msRItT>Ee`*kTMI|N;ZJ)@dc)Y8qhc}T)NpVBb@$cxJl0-5m z@##CmGZBieV7eUfSzlmkp-VFBNEOXB*s%c;o3BoESg|A*WVVon4&Kzi^ZJa;HF`*F zN@~i#3b5>ya{h45mXpEYM9v+G77$(uv@HvP#5x>fm;t@i1IP15r z%{tO;jr3NLdTI&rF9!<>ZO#}@@s9X>8K~x9C@u-MwxTG1Qm!taF|KnxDQNp2wVoz1 zs7qopuBQ1{rN_SGo2=Z+M75chnVp%7Bl!8hXo# z@iCx$)Ui^g+qMdxz|;BuHbS{QT^QaiHl=ZVQtHruQ$o#Z$cEb=y}_y+iWYtK+l{d@ z=B(5t-(0GmPJnx(Kqg8%)ssZlmE`-LrYM%V&jX6quNs1k-MtxOa!S|KocyTE7d^6z z|9*1|SWom9hYI*kJ`6vajf3bIl8;NhmBW`lanWJ11EamYQ*d>4baPBXBqv1vYs`4p zl3H9)v68Nzu>S& z0&i3Xgq#;?S&y$=-hGqaV-%pb;E-Sex;H=tEu|lisSQ=|@BDknqamPK4bjEKx4No| z#*EzP#TR)pswO3P0Y4gLi0$*p8WitI3frO~phFQ>p$EDqB>DqScLR&n{zEN_cO?hS zO|6Jipd?==blC63oTKo)Bh(c~dsFMG@UG-WHO}AoR__aRH#2S4#faJwsKVonfR(9* z`uKlL+{~J^T{VZ&q7t)%WTM{+Q!_S3VoLNM8`X`rBsrkzBpB8xgk)BhYy+bb6EGZ3 z4QLb{g{+er9g@b-#VI^skQ@pt>e~#*!@pbiSPlcZ2^%Jyi z)`W4iRdVc<9GS#fu_+p6w86i_O0UG+#w3rDU46G^jNDR_Jc)I08KL`;fk)b++~0V_ z5O9oOgzf?%W+A*A#j@$9$mF3^lq7O*6}QTyqWiU5g3ot6Vgq%t4ddmi#fwhyL?SP! zIkAJ2Q6I_NR!UA}Njb^$Ur19Elhve5p@PLyL~D@h!jNJesMUc9;h`l&i-xLPRf5^a zm(>o@^$TM#3lhBYrXyp9G$+R*Bc#>=vxKO$}U4ZQ-_e{x{+N z)!d)2gB17CrLOM1mYV!09ol(yV^v05sxWlFN+@Tspza6C<~^Pg zz1~R1#z~uvw)g7))w_Bjn!QcdB}X)$`!Fu0|Hy@$@4TF-!pUa< zW1uL(fE-w9;>H#@nPFc{t-o;{=&s?ZCD4;NM(GVj*Ib$y6^2xM;DJF zN2tJORXpQ^o^;8uj!@-P9_pm{FWxyE(DHovGgy`msKqK*1^sPg>_1ew671ATWP3oL zHp5}$rZBRYkj5u7W`FRwiWxcAgUd9pYNQX1L2Zxs8_L-H!Q<_sOU6z4PuEA^f>vux za7B|~DEr}zi-MMhqLu3GSX67K1QgpSY>;-Cd9vL+Yv<@f`>Rf z@+apjN~N?zq*YofP356!{rW|h%*t<1cXgw=O1qhPjl=H<5iLVnW8#54SjEp>W%zeD z@rNdJ;0Q50t;0F^f1#JJ15P_%?%1D;u8$+`Ak}W+xF+qQ2bIVp+p0{?{WGsGv;P;b zlCf2z-QSL3Fi9~Uaeid`9LCF0%R$-#-Dub8$ub1ycRXUZp?5Z8Lfp%mV(+MpsSO#1 z6>1ih7h5)T+5!S|lq^oPbR`-@FX~J3O+nQj6nvNnEFIg64NVlz=);1nkZe0sOH!wl zshzr@p{Je%I4RZDMq^5THHWkbet(l*kh@hnyhp%)ey1xTj-8G zAV`E-&8VoVUuKR5(SZh1X%lZ09Gx5+MPhnM@{ub-Ha?>lLbQg!!757nkG$n2sDr%Q zMyI9FO4Ts5$)Ed;jW;EHrjurL`<$qL$CV;C`I7Kk5l?13LFro|dw@_n!379PUlO-q zNxl!XU#)>|mw-zLjacz+t`qUbq^zr{Bl#Z89eK;9$MLzeBrQGdHfT`hhDatP(aqFN zeXc&NX-K8PuI4@ROf{I7r0KQ#3)pf4^f>yvGNj~ z>#hXJM0}Mk8u)2mrVi@3yp`Pg!DSTU3qo=F^=z(8RJk1Hth^iY&4jD2Z_!Zzdk~Mf z9R7W)3a=nmp^|*lj{f6}wV}X52Jah8szu(0N_KiCxZZ&(d9ESX zFYo87in}4IE!c@kL9vrV%VUK0#TR42Tc~%YbZPR4Wp3*&q&QS6t>?Z=MvZC28ODFU z{P{5$W`m)as(f&Lgeqw6HsWnhLA%CTbk6V}JmQ*CJnoeMKD)B1nFU|2W{cM9kxa+A zgJ06cjfLm~_@hkq@8ez-EeUPRYtzwHuGLKu%NsEG5i4**TsFarGEuF>(#2p_@kjYu zZ(0rU3_vT6RaPHX(%BfZ9TMJE)m@zjY1MDVf^4ytm$u<^&Q`Q!0(f=CFCqxf@(- z`01oUl6|nKI9U+=Oki!2-GX<}y`vvSwnHGKA`nC;)OjtEkX|h$%j`%b@zFX545f!KvKNakMjzHGmb-mRSr8O3H~^Lv3tLH=&ZP4?aI#C;d&)gCmWg~#La z2iYlBq)RHValK}A9ufhOHiAT~sh@r-;bx>u7@wPRZ|(ge5L(dDZ3T4_z`t^I$kuRd zXWkyhQ|-mEcUvw1bS9u;8&zG8LOqD070*oNpO#B23)me9;UjRZVIV`(Tk$SukAbTu zbcJdMoJ+2AZ#I&lr(1DLq}gn~N|)ub;W9CIceAF@vSjacbdFTk?)~*uzbFVyCws(T zJ83K9M~6!`EG>6Wf4Nl-FTHW4={tGWc4C#McY7m4hNEhWS@No#%wbG6Ce`T8o~%|M zFxUIh5;_mmstvPUDBhdQQPza~Q;Ou*5RR^?ipo^;q)-ncZ9^@1(VFMD-_4t}%mIEN z@S-xKy*2R7BP1KvaJB917fuc}M#Givqwzi`f>)P?x9Z2X0KfV$PIJLUr^_*Xb{k2) z9lL+SLr>`H;yNfL4u_}tsyN#ar(xVpy*`faauQM&qE?AU`|7X48<3(lycRaCGV0+4 z@b6<$GQgdy%J^Dsd2mPRm)%DP1~VXZSwZMVwJ6kuOlixx-rAizo=zOsMRgM!$LFivH!*wW`A4Mi{k; z^BsfQ%lMflX5PfA23J+zGiTyDSg#0=m{BC0lcM~AHzqwT%#29y4xH$Fp!Kx6C?o;Z zK{UOhi%hhxgEFl?@y;IrFVWj{%L!gu)SW>M+D0byC%Vpjy{Bf2tU!I>2_CU3bnD6R zRumtNjf_#)HW!;tsx0IOfcNKrxnlzd^l>VgXZGW06*Q4cQsM3Gr(X|1DRde|RKhb( zR^o?JiCxw+r)~i(9y*KAmGLrEDs5&!RyF5QUp)JMr(j^8!Es`|t;P{6YEU{#HtJLM!vJkSwriu2WZ40!O$5hygeC%RdJ@tv?F^j(|c(QVOX)r zsW?l8m&o-l+)^8|t?EnwemwY#sj6$Hj33iYveu2w#{MBzKshp{&m<7gl`gUBG|W;H z5bth0Yp#6w#Pr>^+W-jQ94AOH54iuC89DoIPw*`?}Ed zD<07_M{wFq$(eNTF3C3n)PFte^DbcbX_+!|nFjuKKQjwb(4G4$VdkybzaVxB2u1UZ zT%jT|kzyiKV*WfVX?*bW`Y;%6A?z!rrUy98#-%hSE|&I9!l6Xb=_j;C-a@q^ko@0bdS zs!tt3`&19`G!=w;zFEe*C*%9ZTY7SNn%8KxZ7O(wN9qt~-xubp@TG`_a=kLTXxIZo z@TTq&tvvZ}72Z-k6=HwZbQNvp9S~G41$jQ=Y#ueLG}O$Vbf_oEQ+e;_zqWP+ehTno z1^kBHYJ6GsLg0UdBV%pu?VTDK8;6Nc?<(qutX;j~nIIJLH(FeAY?!i7Rg4Qe@xkiq zteD{!L!fdnL=*cJ_BaQqijB&>`6~0c;KPTOpyQT_ey!LHZO0KnjyEDxd-GUY;FA2g zI{06}6C$X+`kagpo?+%ge)r~XD^^4W&>j@sEF$KP<`-0W2hux|hegjFeF~`jbgH+Z zz+dN*2EHJY4^y8$SZ87fEIpJU+GpEk6<$FW&o;BtI`~%Vgk{=w&^jaeh$Fd5*HpAt zWI-RU`@l{SY59=t7ih#p-uI@A-?SK2=-5It2Vx$@4e(}9_qAW4NDK)51cesfRuM^T z0~K>%`-|iD&xJBYh;tGD>JG<8Nv7ofTHdee*=ydtrD*e9;BEL&<9GAE3NN229o41E zL7LqxAio$J7v3`Oq*|a1{|6-Z@!1ZP}Yy%{d{>1U)y7WN( z6=)!uwDu<%-@C6QU*ey)``Gh)fL4O_#VmH{iwbH?R{C@6-)~t^j&8>E1&!FuefqA( zH|&T52#Lx6bFT6q*S8IQBy1Z&t`g%u`=`t;DVm35?8c?;(;#Fc+me~h+KNQ>!$yoV z(R}=|HKX>nG3ZpBfONmHYySPVTl!AKf+S=*B?LC~pgv4)-xq$EDXDu!3itZO@Mg3WO% z(xW8z+#m|A%^3bQ+;3Y7EE5AKi0yMT-1Z%Vtu<$L%`M2;VLYXbtABFi0#r(8R7$Za zEFG@G8<8LS<_c0FmJ^wKE=yhoqOu?odzCirG>AsUa+ev-m8|Ja8`^mL&KmeTN_^s# z^Fb?s4*-90)}~Ys4Sdi5?%%eZMg?pJz5?)K|H2X9)Z$7Tk(>d%YF#XRREo}_xdSiG zT$RVI;41uLCv$URF_0^rIOMs}>OU&sf-AvQM7;-crSoUjZ@m}zeOiTh4bs5p4dm(Z z!o3dza{j19V7dn0EsiU(q`UpIrNFlaf6)o`hil-+#Bq5()a;#m2B9_=G-9X)j#1$a z$=x_!HEVvXScUFY@LvSK86k`XT%8zm-Ue{+I*p4+xXcJ05+u&j(sDB8C7 zUurTERt-_83aG0SR&Atdzz$`Jxvo)$ipY%UIrF*@R&;tSEuO0EXUW>Ks5)KV9H_~qy6G_|$^YL+iy{$oX+kL1Od;4W-8uBil4{Ba)yU)b;L>Z#a zKR0OL!xQjrYPwsP)l_G*KJO=?kcKEk9E5+_EE37%gEhbQ++GMpoKc8~zr9BTe<;Y@ zka$kvg}4kF+m0HlJ_->Hdu6{!WKK2>;E^8MIN;oTlw^&PMC7xJLZz?tJv zNL>^nCh<+DRd|KECN*^PbEXRo+Af)TS%o(;2usAA1wGNu%vF59eIYHRww6yOTvHQ~ zzKPr?)3@&)AS~86VU{q)Zae3tcD7*tijeXnzG`&~G(f zze7T-x{R1v8m33A07I4AshL9;F%HxKVV!$h5drQZ$`^V8Z8S%FA{j zphk$4u)DYu)ppypCoMfIktKR(jaT;{^ z1R8N7!q`Ov|0bEw^IHdvm>CXio>nL5ldFsm$}umczFc;@UB&gQV5eJn#NO^w4~EZ9 zl`!3_)Or2GuSJOvhmJ@?1dlE7RKd;ZSW#XUtU|&sn>ebkkp8~6yz6Ap?*zT*!6Du% zdZL@n2k*N2?sKLBeozbE-A4u2r+ac#8?8DO<)yB4B;!eeN~AIA zp337Z_*`xmdcvaxf>>+`?3*d!1|+Jtxve^e`%HWPjh>0f1D&rB1ZP{QX*Q+u&Vp^_ z9`)7&J`^D@HvZqk)bMA?yyoWCsx_Q$HK99XD+t{6Tg&h?9v$U-H0YQK#78`$9J3>o zNHdaq(ARYl0h{;e3)fSTZ9)$B{nHrSr@& zVZ^TaKaZR@2n1)(lGZWpuENXxn%*{z7txvBVPNg076$)n z9V0_kD>JWRI=r2O7j&f=j5U7mV`Zq-Epts|&3O zSm)yrt@It&Dh!q6`{HM}A5ho@%%UNy$nbtDMY)g_&3P|!@U_kt)`FQ8;3C86=`!@} zLvvi4(UP0JLcNnADPa3Z%j^>d%Xph%e84k93NyYB)^vlIXtD5NGBo}v#xB2WbJi(vTG@gNd2(I?#G zO@omU1&-jg|H9{vE!qN4_j-u<6`2}%e|-xx@*|$xCu@IcCt3(461xQ*=BS9ukT7Kr z+4**QQ5f*az>5P;10v&dOe}DNlM?SW_)syu!$UiHY@ILTb4T($N#8$LFGV{8+WN*U zQi-AGUY||( zZq3rNXX{lO_@L2T9!tA&K3xzC&+SVNGmF=#@D^m@Xx^A5m)_R?IAqtg*7DaD4g81E zl4C8ppKG?uxSIY5<(+u2gF)kWa;LfKu8R1i$kc#{xX9FgiOKP-gWr-82hJVQm@aSS zLm@FZ8thgQ$492d1cb)+$0?lHcpT$W-6j*8(ve>45;`=MSZ`63Zb2E=9d@bjKVV1? z;)>SqxA*_buz3PGUrmzv+1@%4dGe0}PTT)~L*?BguXGp@(hU_#(3J&cM>x~NAYLwU4Eqf19g-_!V z%~iKZ;FS~^H84`%)WGgnI`iUP*f$6p4ni@uw%=0{a+=7QQW-|`>UPDv6$oZJ`&mjb z$fx@yXz1n%W|*UHjZ^7NcrPR^Pl4v)NTG}LNSt4F`+9hVJ3&!bj*3xZm_R~jNYx*t9H8qhEerjNFOnb8TDWuT@1UAN9+Sl(icdwC-8ws@{8t&=!*42Es8RRj6%U( zq#Bdc+fP_wD<+LftPe8Ur4DP>j2)x9nomn$Y78?)go@TjPZQ+S)dqE<>J)$9JAlbm?Dnl++p14{OmG^W1A4zg7MH1EA z!K=(+FRp`xZj};C*`W|gtlYV~%x~;yx)daIy;dZ7Vl9%q4d5i>cSSkTG37lxBFSA_ zkz}lrBxY=*Amye6ktC%wBeBqQF=?PYZF(xSN(L>8`d6egu@{S)Hi=7TkTGNv4GFsc zK_pq^Ad=ixlFX~!*W^1)W@=ut!dGs}$kfC0TP8I7BF_8(WiSeXxvgE{eiT>EI5 z>$Mg6JFltyUAad`a9;s^_uQYo^C2T~4Jo5DPaN?;epfMo)13kxvwB$W&ruUmc4t`K z9r~(&36;r@sgnF*@Ao02p3#(>oVkfx(MNdl2L)vHB(r%M%BC&^y#WsNlu@dO^R z)&E>e<{HWl`caJfdumYFiMOu@GeTt3%zArL>KWifNRXMlzBmJJ!fKkK8tw5^u*Nn(AD=KM^;HON?YQ@+@Ti zx_UmL9dTQ@Lk!s#_5VbTK>@vil9~Ua;h-^eJJdRuU!2hD)_|;;fjNh$nS4?3VNv## zFBrYQu`78^+OQA`4X8uw&*k5T{3e4T{aT@@%Hgn*!pK)^hV+$tm#hj6vUOZi2IRDr z^b*WCH{61RXIj`BV2~6|BB$|D&Ddi`lc9JS!uxnuS z(8lcM88&arhH=hE?^mOe zhN5?-xBD|gaxYve#l|#eJU7>*`T8mzG5Lmf_%m&iw}zt4Ql6g*K7$7Tn+{E5q-wV#{QWPN+cVG5Ust<8oW znq1lqO-YT8=O?A@a5aWnDDU{#1l;-=J%~k4IMpzq`ptDTEA|K1>T31yuRgv0m8-~< z-6YC+vimpvMs%L=96af1P?2Yg$p7L=Ce)OCno+49*%jQF0uITLO6==rNBtKEwW{3y zOvk{h7kp{D@Ya%NMD+iSo3K*C2ET{azl(YYerw=U55aD0`fKwf?&~ENcC%>FRqI&# zjco$V#9-Q$q{*OUGyl}{-Ht(^=>U_8InyH< zS4GXX@MUWwET55!**$3&G7Sor5Hf{f%aGkOC7a6BRf~S@@OcVF0<`2&q$}k~OO1^S zXdH`W$5oOYqk0}v_}K##3-BN_Yy9dD7O2ALay0TjyO(|F)Agt5VHg9;l38i_5P^!> zo5kYxH*MT$W~a-dVloOFEs&^8jYJp;QFUiYE{wy;qxqrc;4l=A$iaW?e{$%ygL7C} z&mlJw)Vf+SWRCk!4%6my4p&bde^w6E^y{3W3@s=8Cx^4!B$o*4ritVI4)1{s^z?L1 z4Rd3uXX1ZxP}e9nlSQp0Un0%8*6qD$*(`8P(&GAb^8YtidV3P9BbW8+EuaSgoxs&y zkoD%&|KLhOW-DyjgqHc}C83^Ruo#axGu0G_W#lP27L#pb`^T{GVtKmnuPUj7K|=TL zi#^oMGXyf6*H2AOj0*_GM)!f3v!Vaz&q=%0wNp>tiS1G`5riO)V&_{Jmx60I(w2kzIv!E8sI33Vg`QpM zKx3#^%2QP#fe zn%1lh)=v@O^##_JX~jqV>kL~Z`G1a?azZkS-!XR56{F*jrbm|Da7qJg@s`Y)!fB2!6lfmAAl zN+wlEU3ma1eZ(WqvUOVXKbS{^oILSH%i5aP5C^`E%|2dENFh z4S6@{CbpF`H!7J8iAU3f4|BxDk~aGkrC4{m-maqHub`nPufz__rcIi3;gRvtDezEq zXlz1X1iaeHsi=j?z$g^?GuyQ;VMg3?O7hweUngQItb~KxTycKU7Ca)&_2hKDm|sr@|TDPW=Kx@ zDSIr)n^hL}WN8Lp6#O>Gwd+W*qi3PSF7)REpu?n1rX=r_HVyB4VFX@H$Ag6kjbE1|Br}tr-|Cj*+9e0?F5nTz0JcZT z*-Xiojj2CBm>Eclvm`sSA4e^8hNSdKjSV0fFBHzQe!;V$-@;?~9N1QiQc}r#2gz8x z*6V*9=NkCk7)FEV#qUOeU-5{eA%}m+-P!y~^@wqAI(`9SEYuX;HqkJ8x|kXv)4a;&QrzdirVIncPl*J1~) zOQF)D4U-d7Qpn*+3i)F8QBKoBf1s%4AQM~N_;VKIz+Oy&1fhwcv&{%Ux1eR<$cgLh+rZ=d_n`k6&}a%(Y<#f0SI<>SDYE>w$>{(NXb zB7CgMs4v|$TGTG@I%-@m%>Ie;7CvTtUCHBve1~D)meyy^gD<_og3r z1-369aXKbe-;$+|fIf+dSU-0xT+N}jFdnKXHfqMkpn_o0ZWIXnjgr?EchGJ9{^L4;SO9TKp9@&#u@I_^>U|o8e4#=80yFEElHKBRu1Gd)<>95 ziwg#`3v~3rRxI4x#gh13(o_X>JI)B zOOLETp^n5Yhfml}+%Yyj4m#2GLeYay>QJaZX>f_}&g*X&UPzY~=o%?x(D?1HN6u{m z*Wk_k>rpZ}!K)`yD*z+7S+zX`G=M-TF? zm%fLOo^Jz|&+v$|6}3ZUoiZ*--s~$auCKO~{u?D9LXdcAh>6vWhMJKU7MYe16~hv2 zM_e{ZqfNE!@VZS}%X}0?C2K73c!9(G)YrFGNEKKIz{emar4e9(NJV0LsnA@rxE3J^ zDcCKIg$<_njwV=^BJiR^E4H@G-e&1W zek#vu@8AFRLlU6%01Xk?y5MW7)c3HH8Od12({0(-H?B8>2>y`1iGcsnR#r6Q6Zqd) z%4^lzvOiXaVqWF8XUS;Y-I3_7;8`cuV(tEM;Fg6)Y|I-(T9Ti0EM3T{pb}?l+Nyo`)SS(uq0#7;*+)K23ab|#ub*9tkGlBD}miM z&RYzbQz4Jo=takA`n_RfRE+vAgOsQX!yiDTLC`>Sdt+Q6K&x@<1(Je{f5#hxiStfQ zY6Q88@Q86g7Z(VyEVhr&*gM=a=&}{C?GQ>F9Mnvs8rNdVhTnMBIs=F;5L=Awpuv`; z%_EC4WK4>sABjL7WdvOf%?{F@pcPn)^*&^nC25^*>EmCrcX#4-owBz8r+d-GOxbTZ zi*0xEa~WT-buV1itsAhkE`$hvS~5~r(A`&IObiL%&o`#(>5Lb^7XdFe0WqV=`PGtq zgZ?=CFIVW7w}LPj5SN9PjI|`s@+J9e)#pbzwrGsVKZ(gMky-aiXu!_!mfpuVGM;`v zWg0k&HhVckZ8L9D?I2I9qvu`iQ4s`YFmiK2sdBiXfO>6sj$}^WZ$KtAz0F%*y~gPV zhoIR45Q>IAon=YPzF7H?FgMA9O_+TP?b72jhHp{mQ+Nmk+?p!{AMHN!>_$f*>I~;u zy-fU5l(rwiC{9*+E}&K% zLt>e)>`AE7VO70sjvJw74o#2EnHm&A9gv^KWruFWl-nxC4a!hDY?_ z?T?n^%_v1VY`*4l?uu#Ig)VQTfwgFZiJ!^dJvc5}be&uF@PvYbnSeG2w7y`2l&=~& zX5$!zf{Y@Zf8%GL{AfzP3ZiBLQ5=r!^TgyqMqcF!VeE`6+wZ^{N3~|wmi@FOA^E)1 zH9KhQusE>125X4%l&?!Xjw(B8{>%7xpIZIU`S=TBoC9mI)3DIMiuM)jkTD2u=Ib-7 zV!P5YmUd1uGPYuU@5)swRW8|LTNt??bcaOaA(7Y%--^$I-8*1mOYSsQ;5741Zka7H ziaW<4&M{~zhLpWIRe`l|@`n*By-R<71rpJv!SaF;Llqj@n$O}I%qwa&`7`jFfEROF z0`4ooxVkb6RHkkBG3kLp=t@n%Ms81m_Yqqw;udY?MJ)DlksrQD_22ma+PeC<9Jx!%uxAMkhuJi>w5{9bk^I=Ttp zJMeb2_!9R$@8h*ufIkFy9(BL&A?a`y(^FLJHGf9s+dDw>Jizl6n&@TXMp&nbamN&o z^_S+>;hw~1AA)YKUhHAtj-E8ECkiNa>e?(0% zuGmt)h_)BJ9w&Z&cg@T|%cjGZ(bu&lw*&3FJ~soi6krx^VF&fr#I=1!X9-s2;nCa2g6uy&3hzeKe8%Un4+*&F*A*KqaJVJUkqk z8LXM+c{#?j4 z`bn^e@)s`s0OZX8>dBDb8>7Vbc>fO>kkf&_RJ9hI@q&QcSWP75qj<31T}ik0$7iw< zfHw^A9B-@fa*w)+hey?xxV69`E(r67!8P+O@$IXc*mcO#UPJ~#$2(Riap<5iSn?M1a(chNw>T3gnHEQ^C^vMu6!V8wwH^bM6v1=L*9zyBNCz{o zSscXm1B!$oz2rT*Kc)ma&TYY0QIbDDkx{1jTYYSo(`ewmiHE0g&eKd8>uAf2iJm$(DE-;|_Qi>*1;%r7_L7fB#LNQ3rx3K&1_se^ye2LUS2Z^&!4)T5qv_ab zeaprHp~Gx3>N~?nVtgy~#d-@cQmDxt?)I-vqhb2I%uhpPVUOwij8)@iFR@C>WW?#$fW*!Sgm_Bz;nV zkGwx0w`9%Ix`>6F)N(E_uIYLLu=ExQj|1x7Y0yW#;X)m2y=5RBDO}S8x$`1UmiaZI z9T$HtnF83?pl5u0w@*~?!MbaMXN@mgvVck})H3&Et5k)z&}F^%dJ58-q(|SiTM~EV z=>#O8y$~GntRw9`kyoH<8r2I2`0fSTYM>2dXveG~mGBb>3)MlpHr8T4`bcaFw1FF& zm^CJ7e2Ln4>6YJatOi!vQas$bm}-JPN~!F{x$P>-PXF-z>q!{Y6N9=iQmWQ#!m`H* zlzS!nv*()!WBbwfPhnDRetR}*Vy95)RN(J1emWIok zPQ5F7oBPpp4`AOaZv8mj!~rFino_(|snEWX_+e=X=%K305XQmcAC!+r zWz9$YJQQ1%aG#0nw2&RNZDAIHoyB7i2nl*w^{Z-X`(|0 zvS%W_v~SCMJwQbs9xLOu)j3TZ+-y8Ej=(}k-Fu3@j#-M$1^u%5Ev?d;jT$j{$V=fv zY;zGBbR_75$bJ}b5@lwt3pE$aoggmf;OcIIU3KBC-E6QJi6!D9jIWV?+h0T!VwV$a zx!t=XXM$cQ!1JP9|5~+kUwy;UM%1lCdHUrnCGYAvJNsP%H6vii`5v}jC3#!LtN@Fh zsM>;vg=T4ff1>kdu)F~La$S1mmL@{8E$$*DP2v7}_Oy(lpor?!d@TpmnQJ-lwqtA7 zJf|tdgf!d*fAQ{P2~HY>0}|7t8w+xnaqD$Akg?X%*iQeal`c;9)SQIhaz%DzvQuU> z$Y`22UO`TTmQ8z6v(vCh7DHR_y+(-+eVB^-mUSpyqeI|kK^^{n1E9RN8`)kLRj;GovGOCu{d=K#Y2kotT65U!s#Ah(6*4@h>AQz5 zT5JF|84q`BL){^%9f-gPEF&vDr5ixVwew2O#~q}Qd`0|G&C!gFXh1sq%YYPN=msKw073bo<@EKG9Njgg8VB%j{Qljgbe)BJE^eMD{G-o$*U&|P*DwbU%2InMy32xfPqQ~h18y$*a(*jp z^3z6RH`k+=Dd*I)$^iVF@W-F)K*JE=<}0`6SzVM?T3m!>pWzrlXwT#>vOf>FkwDF3 zoVb3vO}lz9Zpk&9dNj(vP1%Ie5KA2#sntd{c$0mAE(WBk#W}UUqx)PCLNQ7w2G(t; zE|cgd=7=RlxHOxPrj1>^l0;Gga<~fJ8K#S^C1`mNSo!lw{at|c-*~v;nLAP!H@fMj z;gPTr)!kgE%t&b_m-5*VwK>-x!B>?CulTsga1E{s)gRgGN%Do-#*3VJJ==xUVhSE! zLHZ(07nNn`3=+12t?3hP03;dA@FI*|xUwDRwi#{`w%5AF0TgtR2jOCh7>lk1mG@k~ zLs&w1*X#3~yZ|fkbYUW0`3hdiJB))d=WkgIk#bO}d_dvxjrRaN1;~f80d}v#IBJ@g zK>bY<_TL%a(pz8h{+~acA)@cGII#h9CWu4jhEEvL;s5pEbCE zp~Fx$-r;K=tBbra-A&k2m4xWszc+3+RRvrjy+Moy_cxXF;Q6~qr!NP1wKH^TxHr|0 zg$as~*V54ucFn<=Dk;6h3xtXHl0%*sZHDDxF^6-Bp|tqZ?Q{B0Bt7+k$nYw z{yeqg2=_OMeOFVP<^mpWf+p_KMbLg_g3`agc!SK!RZQT?#895A#31uBLRsUU)RQt8 zI=RbR98d1o#REg@AP)x0>0`>f?%x79sxc2^cw-Nk76skZ&JCU`wd8U~O3@1e-@f(6N6m; zX3-HH5{(uVv`;1`>!&+d&k(k4&L)lffesviiBA7vofd3qi^dT3=|SJb(_Rh{-_jRYp>1BtB3T!~t~yhSA4c!Op7=IbTCY zqba)h5>Pp%V>*n(Fh)woSzrlPrXA&&HW3*j7-i=V zD%}4L%`r`0L>gv{gn+Y%I_8fwSGq|>hRC7MD_TQjKsRX~W+9NX#^qX;$VoW0hSB>+ zOF7JewX4@WBQJB~Ff4ZppLX?RRE3Gym2hAe9te)4_wCMmRQ{yBK{*Rs&KQkXh0tdP zz4gMZXav%bwfSu7ge_<8>dGabh4D4Br{#}-9@@fKK9(LcSl(1brxkV!;u@b>>YANi(>nu=OJM!7v zmVnKb4xQRUBiQzpkn(efv&fd)x@@#aDWAlk((>1QL0@ijc=LHd<^sW~ki;Q?5vPbEcyT<;2~(P%MpQ^Sy|VP> z;|ExqU>crMtzQ9wD*I8u8=MY8dvD7flDvFNFw_>!o&}hu5Q#@nTrq;DZy!m!yQZ9m z#t*g;D_e;nVX)OK8!6_3W5_>!=$B<>(40|BoN;Zbta8Cw>@iP+} zRUt4XSr}L=x)Fs|@NeG?p}u|lf4_LL3j{sTi$ERo)3I^Fjw@NR&3=Q*KC47J+kq&% z^+zFpiM?-kFNfG2DG=TqL>H}>cj2fvBfx2@+6e4slJ zAzpXfVc`fXpv@}n1_kab<%SvVEA56D?kn{sje-P@G?GC2NM?{Sl3yA_rbr5HB#9<* z*iYJvvf&3B0%d~~T4|CNLnLLA7F{H(bc(D=CPkWTkqHqN`Lu|saU+gtL-+`HbhNf# z*vK(MZ@&q1bhVb=%z+Np8$V6I1Bw~YFJoj(fJP)?gqok`cR8BwDMRgFBSEGIBQOsi zrAdg=XPjtRelxo9K_h_qtu$8tlMx&W!-zKnlj$uFmS%l>(g+7p%J-5jEC3eqoe{2Q zdQ(L|6N(1OkuarawVktIG=i4i_`wp9n(43L)aN3YYMe$q;)ep{0dR)la`<{6EOx`Q z9hl8602gHsp8>g^d-{L~Yd7iE-cNJX(@AX24RcrFuh%Qw9Y9zW_J%7w{J)o7uYY)YA~kH`WaF;`HbL@ ze#=FWbPJuAdQ0W`6<0S-M!X9&b8cgx;HQrEer{xdcj{r#*t5`9OY&9@9Wzswk4}!5 zGNHV%xoKyxxeZ~^*q7G+IjY)NF%-hELP8bo!_OEE%}*XJu|@I?G~#Z%;Ynv2B~eJXicqAnShw|2ja;XOKn}MS+c;8qhonqN|`96PnTcHX_DXsnsG3;Hp zSt)`6vV{dHwG62xv#K^DO95i18T43SOf{b+c&SMyVvoPiL&gMG6o2jU&DKp&rqOJ&rysUSRvG%5pFy^PA#<}iaSn>QouDMt-qb74X4 zV8PTDf-zM8=w_Ewvq3*O zmaMuh?TV5O7(E_r*%>I|sZSxj?GwHZ$g=Y??|VVw?&d({U-@sOAT^bVi71QUHYY|{ zKvsIizWKUS+RbN5QASz%d=zgOv6pa)yD=7>TPSfrZk0od3^>j6%aeSi3tWF!W93;+^oLTyy5X1INlPQRZFD8_vX*D?@HRk{v_uVB#q_+J&2J5h1w>WMr38f;aAh?6gxV$;%FMdzX28iR z7W^%6Ivvc%KC3oTKgywkEsb%?fa%PB?F=gYj^*u`6KRvA6s#qeQZ<{-Bpf8Zo7GX&&daI&F;*nP_kX^Y~b*V-rS;JFWVw|7F&@2Mk zl9=$VGZ{Hq9pn6lI%%z(O)Wk=@GB+bzmq3Bs5T*B+0(htZ+HFK2;k*mQ2-JW1_y7p z#>U}6SJmT;eEUed8$Ur4g*c98-rC2(iU^&aoJT=p{++m3_Mz!ym}^ zM03ryjyXwtrgExsgm*VHVhQ^Van=Nts?!~5k{JB*UOAS>hh*14-z(H56>(9)1XGMuuLuXK z2(olA`q%{Qe3M#DLI0e}%~&L{ z8HO(-^1bI)?5lqQUPe$#mUOpJoGZfvoU`8z%PFiUC(&p^e@jJCxUq%l9|=tealcC2 z$_o|=$b-}EuN4HOl^hgc$y}xMY~|VRDQ{5fYLQpv1=!D{7iXHcU7rArBBlZJN`18- zMPq!{*f2B+Fv?8G%4jC-YQxUBEz9qv-y51(ZFn$vI@-$f`?iiTK=~zPO1b^VOSm>u zF1D~$vMqJvKVM*UZ(M;B?4Lk|1v%~_;2+A#U9=4!?Iv4TJIwn_l!#dss zTXI$r1$74+FjlHm>>~5KrhO1@{#0(vP?O=6zbsrCp8onjxDB##y?wKPgtZQcuH(ya zm}UPUk@p4jq$s#RE9>c%&T-Rx!-%5crCuF#p8QCI!@m8=QiE(D&hh6{jmUwL%B7Ug z7arlY=PyEmI$EbdSjXxI#oqzG@&AB_k_h zD$(m)M5^7=4ppC}2rR-cm0{%4(AHsc+b?GL3O?>&6F?oL5U--ae)E3w2!id~pXu;b-c|$=#Sz^l3WvBpes)AhM4OE>B1}@;4*vUiMLm1IoW!9*S zyFv)jd?@%S3pED=Fdrf+t@;Jw{ORy0HzSzfw$WbI+4r5{eAU=Z%R_qTbml8XhLY)s z`W-N3Z?zEA_!ojG*Gl)j`N{Vy!qUIf7)uRFcUKG~B%~xHC5+ZXgAVan(HgyUhG>BV zzl;tN!j`y;_7O+fj}qotK*~sUd)i2GTNejBNWD2dpmcVmH5Hr@I=LmWLa3)xe` zvId!2F*>E~#9`ngfC~NDsVxa=SaSu+4I?Z*F=7CiY!V!^i7P(AP`Xdj?No z&01*8k5^DDFE%3REUPtD$`z}t2!SBoV2q7TViMnD`lofOzhTPz_rcqZI}lLOdcTZ- zZ4TycWyd?H!a6WPTgksB;aVks?&arHEZ<;@E7bn*o4A3NwQ*^gL=YwC{cI7^OqG1! ze&m~)F_Z9;!4hi^Q7_QnL+3_(ugiPwhJRlxT#Si1$Dt#<-mSlRI6){8V>Tmky1~r0 zMz_-~4lbxS1HPiT^}1wOSQUt8#d^o;8G=Qgz{he1Xr|*blZ(*4TvIb}kl_tWKHkH91QneX7S(v%@xWnNFMMJRSR$Syk*jL+Ol4@M-UV`I)hyivzA zU@QZl1D?#^G6u8Z{7}-0bDMkxcS>-|aL{$PjW3Ok)?%b(mqjCoO%KRCU}^8dJtVA# ztV$`#n8ux5uNTiYwB8$*5FS+b9zHuoQEK>R)4b#niq`kCoB)2cV#yD zlX{8Mv7_B0U~}Z>!R1qT5?*iKAVfbH=j*HVDn5GSYOqXF=)( znO#@w2pV+fQRv0;NB|Zkpg#{=&OIbMI8}+{>GQ~MIfu5BgSWA|H@59*=U#G7vSZFi zgB_=p%EbmMLf@PH5WmUYL@#?Fxr#t$>+Rp7sH>LqJX`WoipOlwxn?j)-BBNQ5m;0b z6N_iZYLJU($6TnAOd%Wep->x^5m?k4j#Bhu-p4WJYBMQ7!vk93&9Dbkx?xV8;gPu! z;-9hEmCXT)@z-Pwji~*#A5AE3L@b%l&6?GjSGHEV#D0T%O?nKQcCxjo-l`@ARd{5P z7*rR9MzXTG$ILk5)%nx@M%-xe1WQ8K7Mt@+Ofw!XOp;HVj<9p0%VFOEMahv<%A1|y3@<(4+I?*^i(&`IShql&6lF>FSP)q6C#PI3=q7e}h z!;Z;n@%7eSiB>e+IOknQhjwM&53g|K=ot=n(zlsx#j_~(g^KJt*Y=J%yn?~K0!;7G z73Sx-75nzUKcZys@pnphZ;3uYD5r{gWqmh*kJ$J#VyYFlBE93=uYbh@7}y2<7Qt`- z3JJ{!|2t@=3y?riBn#l<0{wK46Tjja&IDxV($Oes;3AW`BN-+%d8kOkhT-B& z$xTR@@0yB9WNR71A=8TWj+yESQDovY_q6_Zxo}?F$*8 zTDQY;y?O1XTueNg`W)_-zWg0~GhcpZXFC4&4LCbW&x5GXU{byP35+^XdsiOGS9?#7 z5~}8|JP=4NK&v!Uu9TuwFEvTcp-Zh$t5V67!K72KZ2nOiO2?>DsY97BBb=&H-Kq)Aht6f4pKzf9Y#RY(cXsBV=hGKI-2S?Ciwt8Nu4azgv7 zVxBiN3xn6N@B!YQ1~|&PMdnluq24|Kf_lb>!8-5(G4s{KD#9Yd`$_E8_t*EQL&ov$ zVOrFU`#5L!{{i=e-J-pL15rZ>()Y_$I~PL;0{-q(!C*~UeHUc9p#>=-@RQ&0z@+Ul zfx4is(7ojLoMF-qh(P58b|2*6`Pk1v{n$_{;G*D?1X^(@K=`f89|E5`d&Z!6SOQe; z(|-7^qYz?Z^k-da@aF_{j_3OR9`Gjw{y$=jL>OMbB6#?b`f`Gqm~zB&|yyKS!wbDZie(x>fJAMc zl=|7Is7Mtn_A^Q8sK$Xx?m@$GqxgFHqR8iGxx-YgfCTq?g5pk^oS2D#TwWR_3I+K> zr^QMM^&wnd1rA$-_BFobrbWa~sa!O*&-uNeK=)=v^yRi8b{us~=Z@FvkCff6bI$By z4O7Pwpb%O-hbKoyT5Gl!?ZA1>&VpFRnLVE4UMJ|vf!sK2UCpRdxFp+r$sqtED^EKj zZ^`B_XZGUKk+ziI`FKdjv3z2{x+`PySW~`9q4|!ot9;kPFy|ozgam^fd(286i%*RI zvbzIKOu*+=Nt22|19$rq)g&tc$#w}^`>Wi3;PcKIXDUNs+$dd4;XuIfnaXq%V;4E4x_J3B7Q_p=ei%%Db_z*8BR?LjbVg zDIQzq#ZW0}vPA)ZzT;})7T-%q`EZjOiml5V&>#oFCvK@LR5)+OyE9&DGV?aC+Ro+E zUHpq{q*&SEA&2aD&XjJ!Zzt6}zoSe;pgwM#gh}F&O_0C2g^_)fC+=3=`k&0Jo`r}< z+(;emIHlIek6qUJt|_TzX{ohIRkx3%n%J7y?7Fp$wUyT1me9R5EMlpUSR??zt(B`a zPHq7?sI)0+B3C_k)yvl3wu-L~b5V!1hEVj}jE#>kRGY|l359CW+;4TctJ8}Q=wp}< zI20E5nYXzYf2d!u9Zs?n&sr&yEV_gcIK;-;J`_eGu!=vHx+|#sWm8xZjy%D%RB-1HoxbhA4ML2?45D`F+_c`M|OV&HPCwqpR(7OoR*{ zyb(qPCFuYDg^uzGQJZdOGR@5ej$MR`LYE1vdLxwI>(>E8}M%68ZEUH^I|n@AGH3r#KMV?dQJa03ZENpo!|{`QRC?wO9XOP<_z zCz@_fmp4hqb5i+0Q%tN}BjEd6Yo^DsI6Ln>r=A$@jwLi26(%iUZ*Qk z&i)0QOY{J_G$CsNhkabYrr$?`jn6T0mE@-Pe|?vIq^G{?S;VCsV;T+K!^s$fGUY&f zYmyJ2W}}BWcn#tcllsu{LknJg3mlKdn?xKY0tZ5+T-EIrT!S)~XFKE6R2F3#Mt0qb z*bHTgOP{XL-iBmXC$a&K9$W9IK2L8ff@ zDSM73?dbSQhRoG!NrNS#$jsD=3g%f_i$D0MyNt{bDsX6L8`JJQY&J16`U*$z2`xe{ z9E$WG5M7J37kkP;)}$kw5lhP>^J)yUT0?VkiYApQ-el(& z^dRrawE{DKU(On!bki2*pNm)%xHMzZZwMCShVx$RqMs3?bF^~4=wm;gQ3A536c5U0 zxI)zDetcHl!$`oefeC5>etfqd>N8slo}bS276ig;-Z`Rtk-cBI=Z<`xpx<)KNEQ&! zo;7A8uxI-OWG)H z4PEi&rZC>#L(}Kw9}O+nxNf2C-}M9?fU97NTZX@;LsN=8%LWxqAq zvOa!l1HPh38O|HLFU8cxHG7=Ki6NJQ(!f#*a&QCOF?#1wnUHOW#=h(AJ0k|Onc}>` z4xlMcupEQ{A=-aPPAcn+un3P|Q2D7&cr!^(dvMBL%jXOyDj zO-U8iiAL@cgc9JIByYL-XG!nP9^}RFFp>$fa@z*_p z;7vH;Ydn%`Fp@m|b4B2D#&mgOa)OGfUSNN3$@o5P^~U>Vg@wW-2*xCktr2=7bG;%V zfI&<9n*^WNd!&BY@q6Y3gY?gky<>QNa~lDHbqC?oFXVY2aN(r_`13{BYh&hmx?!bd zPW-x}+4@c4iS;*N`RGnF-@{?Pr-p+8Go^G`Ex8g*b%~sEbB(KK@0>`jj9ngKd2L5? z)?&hSxpOXp(d6JaqR&%{d|E1c+bedI)vn$61XfP+3N!^Y+FW1CNxn|upsUx}C;%Xv zPyLBL!gs!bzNw6{a}DHvKHium?ihzSnWm{ha)Y}fIeu4K)6`P)ZQ%{_gDPF%%+hyv zGC~JQ&Q{%w@pYCdeLvJC+sPhUyXmB<>X*`BE^etf=c@AjS^lmT1H$1chlf2a;U}@6 zq%z$go&Ms?LRcu-_D*M1HAQLo>fqAG5%0W3(>;@%!3|NRpL!JDZ#e^{XZp|lrDNau zx>OPgAY%tgOEuxrR!VqyOjcfj2f_@=_Ucal$yO50gaV+(4eyc|ZA zCdq4Si9FBL?iX#-6Wb$}6q`-U84i7Jo#tsIJcDNV%{oKKF$QZjqELD9=StNKE`HCT^C^r`!u zPE^H)icG*12E3L{<%$b3W5dsa(llrt3vKy_G%DUzm+v+8%}d^COg8nyH))j87YuKe z6L;VAjd*_GGaBxl>&tBdJ8H|TFI{>M#sQghM2Fd1P=#2P8v3RJ>U7woojH_sKXCsX z6?Nr)B&|5{Ol`h&`~pJqi8AJpJZKO{J65%I*y{;ezHIS3>3X`n*q+@ixARPJt!_f^ zz8#O3I)_;mFfnaF=lqIos(?Xw?g%04{=KdjwrI<^A4g)o%f`C|^eaYVtP@g78QG*- ziWz)K^6#E#)w8VpU8OngI%mN*IPZ$`@q5oOQyY|TYx6qhmMc8o?RRIVHS?uoZmc%&op{Wm^84JLy>Ib1HG2B-A91FN0$GE;}|Jx*)X%MS8Z%j7h9JQuJU zwfC4aa{KRqH;XC%^qoDrdta-2s?8_Pt$Sqtd#2y>#D#aX!qWlazcO&a)Y??of$V=%*<@#AigepS6jWen!i>z2 z5^Ct2{ZbjQm7_)oAL@)sz5xKxr*VMP9AKimT6)A&HBnh?-qWR}2|^(54@XlpVTmZp z?(npj^|Z)ndrH^*=DWrqth%7lU?4$c1I2@3TT!Stgog)i-(xHPr_|o3^*UpS#?Ajk zi=@u84ifo3zo!;6B+h#7@)j7&zI~j_gDhgL17gb%ci5YQ^Su9JZ}7eye%L-$`@w@i zrBYzr(c7)H6=&vU9nY`Pyvu28#`)3CsDuk0?Zhy*ouN#9w_)G~_#nhprk|hIgD$Ql zXi9%N7=7H;T{OwlG72xBy3Gkn)I3z&&zCgQGhQ*(zp25Sqru7Y3Ua(l7{7>d9}E;0 zlMG$N`B}CH7+7mh1xZ{3Cb=MDY3j4_?lZHs#1Tq=nMFJD_ZTH~e+@=SJLfMVndC1j z@G5fn+*ZixNl%8;I8h2iPG=MU$M93fGa`y$-@b|e6D9n=f`s%$ZYbj91Z1f6oEAPQJKb%Wt8L}j-_(ElXx-Nz8LtmT#S@hI9qnT(54)vVK<2m{s}$(eKB-li z$wk_6U9`kAsW{WZS3TsMt(Qa$=<6bZ#7Awp6CBFp{`Kc{qUm% zc+ghXyGy-M`zy2#cl1iJ`3y=OGdF@f>c!@*@_F?ZE64O|e&bAc0hF#AMXO;!f=W9c8yg zjh0NhTkq89H_V#wK3Zk)lLr2yW;^+?%C|A+hwJYAuXv}WyU?KF-@Eqz|GrY#1xAr>G>QV9Jm-%E z)K|q<#|cP-YzzraI37~jK%?JC2pL_e)7a3Fku8;nfNzG1Zj?@|r<^hISURw>>~1J# zi;IinEt994Fba#mn0W)1jzd2ML zX{`~wB&HtbIg*|lOSmSc6*tuYIuTCtJb*}Q>z9M=ZBE+UsH97lIg+#z{d@>8a}MqD z=QrvKFaaGeJa(eCJLBoLJ4x0tPIN5<>8?N8rioZ5L>^>fW_hFV_#rOtSNxujX&gjF;b;~K6M??q^ea(~=na^;+ z86l^eE$}V!be3qz9oH-=?$v%1_;8tN!QHt~xT{?eV&3A~4NsqgF8)8xhgkd$(~Jxxev5FbznAu!Z>mbC$l}r z?W-BWj@1=ipwy}qL_9x4O{hA?{wBHdNryi1DmFK|ZBAsJX{)DBwrbA`GyP7#N~y9k zY*6vwH_!6$^WiC8|Ebprj5h7geDB8Xu|j=9)I>T+bKdE8$$1J1ZYzSJMHC){viYz& zf>$Ccbf-E{#&2^vl{_bKo20U=%g0*1X|x3tQ-eGtS^-M<D)Ms_do%82;R)%XE1g}MlaiMN84qg{<&Xr4jx^0I*EW_0$ykgD%YQHjHtibBAQ;C+)|z(enG@Fc(zFw%RxQ|hVC9l_(eGQh4?=MGXy?DnqFzio)M{Ucq~5< z5D#VJW;EMF>ee&Vrw?>bpOLk<%CSG^w_RKulINmhH24434qRmNEbp{wov-x1G!CZt z=W#sC=3Z)$2|rPe4#MmPrJ~Sl#>%FEm64 zmP$wLe^bohNDV8alJ@(UNx*-O$G_`h_KDcA7o0mv*P0tkf-y!Lr;WvUzcG{^t5&Do zV&4aUam{|(z!ceHK8SGilC5km0|K|CUH}4Uz}6MRlTtXt$2M-H$hT3yOI1WZiipEz zcF69+IlC7U3;t@KGenDjlGx%57RB z!6V`~exOu8ejkdpvkx!Nq0eJRxM0OpCjh`dMY?sL{8cCt*u~Y{e>W?kON==z!nOtC zj#u2Q1s*p&xIMs+CY@PV8J$`;=cb6SKXlhdl^kj;!SOJn8SbVQMhVmuy*M}d$wC&1 z6kH&P3RQXuUy4<4=611aNFPnYJ&@g`(x|4A=Gx2gR!>GU=QhS{k0ZeK0NBtJkS2R{pixLt)SQ{mmZ(@nMDBd`-4b1YA2Hluj~HcveM!3MjMbgQa4fJZevsC zPx5~L8k^rf-OjO5_%8a}z<*H$b&1QB0Eqqk#uebhV=P3hZTU_@g*VO^SAyQ{`@P+BeQUzLHLBLn+(%r#IJhwdRvZKRJ z>ZNP6NGx8;EY+K`FLKTw6<3OrH;+s=RFxZx=c$UZ#d#p9rWOAT^`mMFx!URnzeGv- zPC&gWOtVL%;`gr_McX-juUiRR3P~md+B*TVcT6cHdu_q+U8k%vb)fTg1}&PJBmOa& zX)>R*`G&y9fi21AreD-{{$hy&3B>O)_Ow46u71!r&}s3DQ3Jz@`9(msddG0I?UCg4 z@MRIZG{l^Ny9r#iRi$%M1$N{BteNwEIzszXzh;SL+3;`qqS*Ok4xhSc+MiLU2Iw>E z>RC-knThJfXWFIKbz{{Wmq`0PBOaf%uV0gOdzYjQ_n0*u?`S@&HabYWo)d9u%x7zg z|AB&HuME}P5Z}JNLjP|lm~4FdU$pRFzavQAor3`i!1<{i-~J`ceroiV9v2CO8o*Q< z{GEi4t>zG$6o}gIq{g)lrNP-{>^W4Z4!G^DkrW=#cN#Q3M1x-6J~s3J?!5iCG3RK}n|a-Xr1){YKlW|Uis#2SKex_6=lmM%*5b6l zNuZ*}3q$H^Rky-%le(bfae0Mb?c_;ix54pW_1+)H2H%?+ zhUzbd>YrJYhMF(9k8nC2j)Nd1yDnrSv{qqsEzKm!X$$(PDMPFj!xaVnNFh0 zq=s&#@<2{Kr!=Zb-8qhKR(%Ass$B(Bv+}@3{Zeg&N&Q^}vrB!UFbYNOHjz$Hb!1oV zwu3Gj)47H$pe*77!?}mdL1{QP>bZlAlh&k(%t2*%CTguHq9&?c9MDF0z$v~}3yk2y za4sjiQW@TkS}ThP$8699#M2xoi}RO7_(yr^0TgKtq{ZJ=0DUUMb(jr?02qv0eDQY; zfFkXIwfMUNpigs{Ey_y+Fh_gfE#6%afsJ_!BmS-o0ICejMm;x?eHKNeW8OxQ8K@5% zV%}zv8E6jUM|r6N9BB{i#rs$um+_cr|6GxY7lq3j?)d~p0f_T(YE1Ai0}Vm0=y?W?uI zf#pjX@q#ivtc>b5i~$c!7AKp=aG+AZ0zw7T6Vg3ONM!OoQkEg)@}>tM z@`F2&D+w1&A{%2-&r1OJyupE#H$vdMq$Ecyb8Uwr+v0l;;3$AKVIykLHsm9!S4zJr zWPrGl5U3pPJ+A*6j$d{U1@cpBZw}Ic{00m31m`EVhXDR1-Vgfm9Sj>}3>in^gaKNB zq}nvb>>5M{cz}TzVV_a?hOHLAKazmzP;Ul2tjyCljFGygk%5+t(QtY}fhhGu>z9VG z?Muif^?js!AAg*fzH_JhJA^-6ihy^)@(b@tgZEL~fP0kA4h-Jk0FVuZ-_wgf9ewjlCYQLH)tS5SU;IP zeDGbey)ESQMquCOKIel6uLmFK5A1U!q|K$1K*#dK zr9I@k6Syc`O!E+uUlO5%8Q13i$>k?_M58b2O#tL5X1`5<3fNE6J}RfcetJj)${S>m zC)|5*|0-;|$~Vh~FN}W0b!J#MRIj-HDM$m-8!!+G0O+Lg4f~D^d46dWD2lZn@)-H)v?q!`$^(~1BeX7cHpE-X6 zf&8MWfXJetVUmDgDrna}D&&)L>p}4z$pR~r2%{_<88gYda*ppg*QBVcVz^(SR1!DY z?EAkp(Yy@MkHwV5$BUmwvT$aI$?9d%O`nnFV~)Y(V-C4&P56Ns{mEGw{r*|pVS(XX zHYQ|3y6DNLxBr~zHi9bdB;MA9b+LU$Ls{0#y;~Z+snQ<#J9lV)VPwZ}0MWyW>X;v%A6Q>v z{~Sj{Lfx<7%3quP!C6|vLs?pK@v>fO=&~;!^6w!+<&jIzHu5P)x<^asbH^Hzd zGCbmdlr@@A#Kr_=VRWHF9RG_01d(NR(zh_&^-d?>Jyy3e{Kg)S@*0-ZiY;5_u*|k3CJC!>CzGKS!jTr)0C~jmq zCZ2n;j{J(Lpui^DY!<_4SmVD8lHpax{R*uXXTM4&zys((o3LB8(nwGuH`wx6ciUME zXq@togHQ>S!;=HR%84Ct_Vw_5$vqeFRn+#h{(LwRd)i6rH?-3+LzDhHT!Oq^3}i+G z2FKWE|Jn#4JmlI(NxiAT8d4zfU_|slf!`r*y-%*KHp!_clt?X=;}U$|x)6yF#_YOw z;w~B-7xc$WEeCx?*9d+jw8#`v4n|(YAL0AO_Yl}Zr)kvSjgIOV;l%3c%ZCoURpV+m zLSF)du*K2|e6pyO3L$*QVLV~2hH+jz0jw2(DUAhp2&6O;PWzA^eK$~9a|zzjs)Km! z=_U9RXCX9LFr5nxf#EObND-n%J2(}VhekV|t|>3!L}GzHIxMvF+KSb568%6wrq=Jf zu$AvLZu=EOAK!r#M@Yz)^9cD6<4c%`T;8T2C)jiw8X1o6Npt7{1}8H@+Mr)%KXF(`&$yud1cBr3 zkPyn8MH1oEg;M6^7P{J7O{&Rj z+%>qH3Gm-aIA){8!L_q4G{xQxc84)=MLxA2-aDF9I$B4;2hM?KJ2-F}Nh?|x_5-c( zb_kMbn!dVLRmIrHNToZ+!mqO*8~qW|!^Y(d;@OGz0l{h)_9 zTLaLSKI|l80rz`w14avl6qZ?U>ovi_N&co1gJi8bhZ=2JiSR~$fNl8^9(`ZX%&jYx zsprvMZWmeS7vea4TYFd&j#-H^Q;D+BOyozl!l7Nf+;8_sRX3e+DU_a(j}}Hf?GZ9{ zcn1n&Vi6eZu)>-1Fdx%83xRg5ovGCUq0x!7946X4h>lHYGhDcEEi`_ruj~9+*d)2i zlX!C;i1P3(Kz4N6_)EYp6q{0eIjS}4RbgG1?DP1uWm#JGQa`O@Y+*WH#y)r3D~uT8 zIbv2GMp6z(QM?UR)s6+>wJ648O3B8h*%Xb1Ul-`RLSVEe%rf)wRS{fZp6{eEJRBnV zTO$57TZbI9#=9j|qHs-WG}l`<*`;-@rmP-u*N%3oQ%pJs+fUf98oJHgzw<- zNt%ij*k5!%n&~`6aA_{SogQE5Z9j>e&S3C!sTE^=%b7=SbmMaMl#hC`@QQ6)Kx{*3 zBn3V>C27n1e4zBVztP%j0DO4c?BEPEMq9Sc2{;aY-Dl3-BeizD=kV3i*iKiJ`bBpC z^ipBnqXH9q@sCDVm%Qj1bGJnvbkxeD+{8Jy`=&SXVrJw^6ru4QVNF#iPJU-lZZ@JB z^)<|VmP`q=A*9zvVd;}Z4`yKppRL23Rsvs`c_29jX!h`cB#-)Gwf#@TC8~BS@>_^g zy(uxg;cy-tSUN+v0%hmUQqo~Z^YKDjok)xy-(kf5)Vu>;n9oak*HGB9w5d-j;`zo_ zIkJF=@)U!zmem>ShrnAb zb46muGp1rQl`NHr2X=g(6#LiQuT?Y;@;)vkskVdRl|;dt;hace&#EUm5&!r>x~`G< z=1tDPrF*|DF82j21F5Ipzg(&1SE^4~u+T@~3 zyMX$Vinq%yxu`V%FX2wjdZy=32%tIFQLSpt&lqe1wB@2;`&VrUDJ}?+^zb{jfCyvf z#i!42ex18XUiq$4j?6A8rZA}-xO~V1_$&U`!YJ~_&D`Nex&qsRP3Ff>@25kbM!qKM z9?DI9S?33PBp2VI_nEIlc59vl1o#Qc6D5f3N*LICbJ*9DhDCkdOZyhyh=3h5qvNZc z1#>;Z@2KqC3P#8up_H`{$wIS`O~jP%Kg2E4^Ol3tIS|M}8P0AaMPDe(^M^TKzW!{D;mo`0>r3MhYQwpg}*0J{LlCdk0AwFpl z3s4GfMP;O&+|}8`QS`ObJyP^q@N!~VivwPj5Rij9kr@!4{Vo3f0LH&K>jWh}jARO0 zq6l&pvfe{n;rMHc@7iU46gp8mn}94XAR0)u0cb-wQcUrmaQGh242lmOS^ky8Sx66V zX*23&z?_w!&-15P7WE}udLuvnl?@SM3qb9sydIXEji;BM^0m!}Yb&OdRAC_K+_@00 zP_Cjsvc!~uK~?Y)$d@)y5Xm%Dpf4S~@-_ihF9a;&5$$H?ye6-udW}KQg{LaJQ=#U7 zf++lP$qntDJdMx8csb-=m4)Si^eQrBI=bo_pxMq*HYa%dk!`C@v6YjfPjhy^;_baVCaLJ-zs|lPGsBi09ETp37 zc!z^wu5sDZ)M;8bdtE`ezO}SI61{&bu=cSi1WA8wa1iq)?rL)K`g3yoW0(WMMbl8} zIHo1n(C2UGx)sS^g&fu-kpIrR^^(POHka!sgU|d-{7TrsnvWZlh>Lgd)c2xc4R+l; zkPHf>wfF=&aD_D1ID>I#4IUZ+yO?B0iKHsLxO6!hX@-%p+F%RkHY`hnfl#@JyR~+t zk6ou{t#Pn4HJb5SM@OSW%Tn@|s?kkMd)ee`V{H*Yqs+y9IIXIsUSmH>!rkHPcGOwf z%ovFd$L{EeU`~Q4!w^}`03&ADvcYflccsTV5~mLz*mF7xHoSy~gkZRbq=Lw}9Q%C< zYk~SFs$DG(;UT$cHOD}^63VnHo^s3zjtPzYuwMYec4)Tly*PuNO{^FTB)*|z(LZq^ z^w!1?7M%VL%%x%L8T1@`L7?e{OEZjXTUDLp;W!%^tOD09DbwUR!rZX`REC(mlqYG7 zcP~pQFlSh^-$F{G#%N}bMa{y3PSDQm@O!aDEN(EO+GQLQVgk?ci)M>YFiqJEab$uI z#$N>eoE%_Hp3^ktsw1hIYcYfNigp?8Xn;BNw%E_|8gC?IF^br+|*X83KwL<7V*%CI5*{DGg-Y3(6c-QhxSj!6=vikT8{unsEDt$bu=Cg zZ>wWxhf&6Yk}jvKbBPZ&WXHB`+s?$sj-Bk--ocJ-+qP}o{LVSwcW-^C>Q2>6P4$}f&OfXBdAi^3 z=QWnBY*OhW1UyVl)18_K$Pvi{L>#?$CZOUStfgfs&06c>GOea7VUDpQ2N_GB;UcW^ z8z>FuxTd#Dt604$Syz#Sjl`+XjFC z#B@yN>lpoWb zs|yNylcw`K9zo82&NhNrEU=Wa!=%TS7;ra13?5`Mx4wvo)MB|Egx z!C5sT^$6~EQ$`n5FzbERd#9DDCR3;Mf!gzW7S1Ho8#8w%mnhdxW;o3-1QDp|m0*S> zvo?Ag_&Mzqd04c6p~j2DJ}cl=ICaDW-XLvt)h=t%^M{XCAiToBZc3_wEW}eB3ZT%? zSyEKEb@*7!PO)CB?o3$z*2=*oC45xAQ8_{KsbNhjlH5o$j}7Zae?>#-&srscs@P-T z5woRkI%|x_kWrvH+J;@gGAOX;@}&z%+_;2wcxkni}1j4C$wRgLx`PlqIJX zaVN}X#;;ZJm^5^1hwABinwmpFul6S_4HO;Yz0?ejPRu|szVlqiZ^+wbP%4t zNfY~=84#6KvD9lq(}zPD7Hz1dpe0jJtE=2#s$${5r9p>RGCNL(mLlG z{j^!!ykpb>tI4Pv80lp4(WvrFZDW-v^;|@#ni=W9p9;-02^L2vB0NoG@WZ$>a^@i? zF%i*O*gOQ;q7As;Z-VSH#k=G27PhFle2c*qH9^*kdFl)|s|AV-UQ{W*@yKHQ)9E?h zE)JnFTRyVMko?6NfVh3Tw8~7MSZofX+X}NVGm0zY;at2%3CM{fN(S~&G`1@%RXrLX zFX9}+>Vn2{`Sh*v;tdqrP~f4T)~uDvT7&=;6K-3bZH!{zlQo=ah3%k`s?og(7pS#t z4lrd!S!H@h^yPc}phJ=jTr(y2UwvUsY&Cl8f|$C~K?hDX0AJSt<-?m9dtS03H7H3Q zm>dMW^$sv4upHnW^atmMD&a?~Zt1IQ-1h4O_T4FSP0^vNzo#p<`l~SIYoo;S{UcG} z$r7H>uIrDFY~psc9|c|B!PnY8t0-U#>LB9@VIpl_rEl| zm2qwLJ;rMw+6jAjU@wqUqs*J~d9ahAQk)Qjy?+&i0P=&l-jL;~NeM^vTOjTfo+}TF zFg@THc}Cy0Gud~?;p`-q?u|haE^Bk))t#C~Z$m*40Q)k{>UIDg7B78>Pz$c-yi@Lo zF;P<@_1H$+&LapTD1-Y|7%_+Lp3N(M%f|7rJI`P`0?&6h3LB*Z+-iU0cJP@);L_ec z0G@1DyOU|HzsW?-hC|KPUqts;nYIs#G|*9*_Xf?Wo2SCUq0A=ZKp?HuIfOatZl zAfrkg!8DPBS=i!Zga_D_ZXi5Ha=x3fEcZV~x_Ov*w_;X8-)gXBgl5z}lT4V7x{tY8 ztbsh4NK9h?G>uZYop~op+^3z#_&aYiM!0?eut5G2s<0@H>J!d1h;^)yF` z?jF-+=d%#px#|~PR-g0xb7%+Xf`4colqX*!3+h&x$v0c z&}_nj$Qt4%Dne6Oy=m&g+_|@ot!-}sFtu?XFwIhET~={XsADoYwJViVFYouUI40YD zEl5ZV$OpxxGpH%M*C1v%@dJ^YKt`A4FnipD*+OffXyKJLaK=sW7^C>2&lUs9-y^SB zi#nE7OgnG}26PMIKIatFHzzUa^muq>om6%kV$|aCuU`-Sj%95VF88imB0#nPJ9ldr zwc-G=nZ!dLP=_r8d10V8JXn)1aj0Culbn#fl{?1Fjvsf+QIrc5?>$NWWcU8bN#!Yp z;SHiNG!ndLVy7IIL0IH($ncNm-w7yi(DIlAsyEpBxvEYnV%HXCaE#d03?d+xNvx_( zNWZwJY)mCygDh(c;`@BRj?sbJr31FK;Jvw7UQxK3y9{%I8Eg>M@wlk@x`6&^#8 z8%phZSn$EYk(6pAkJG+0;(mA$CPbmPx)Z_Kex>p#%;2K}3-0JKm3~!+Q}~FN1H4yl zKyW%5gENJnHJX0y@5hG9gz>t4afVs7>2LH%jytj`MAi;tyy9K@@{|V%_4UvALQJWT zGvssSh|-6Qq;e5*hJzgM&aocJ{=z+|XMG_j0yPTP zW^Y_#cAFK;uQI`d9ItVD(z6R4H=(;S7|Gs)gq|CLdT4`s6Q%_~0sskPl>cn+t@IF9 zy2)yix{azy1U^;_o&xT%&huaDap1=hyH++}&4p=r;pGgO8iFDi$z=~l*~+}gf zt<45@@a%1Hq~*KLjrQVA-P4SXN%0ZTAkar9!DUG4RAJVh1bwKBdQfOgkU!Ge#aY#9q zpVV`F4r_4Ff-BS|s_9XVehVwmL*N)Wg1WDqozIfW%3j_c5%`g+k#I0aidx^CuYz^v zde6I&vs(#n`Pk$h2T0!s^GMtDY`Iy9}nwoVLe8v2jW1oB-{ZLR!KjyE6Lca)sgCpIgc7b58?oqCV zxr}Ns&H^KzjCDmPq{}STH&D;QJEC>S9Cj=oSb6Sn@BsDE=cp8^A1`kKL6oS zPuB~$+1?Qg&(`qzyO0}fS2*4fgBg(@R$GSt2>V^dGTpnC8$?&iMjv2MZ+EEy;U(r= zyAvyDRhBnyYaHeUeKXGfpzo!M(ixYmGpJ_wX~)Oy_Uswt9sCpVs{2j+nZbeH@e}^@ z3Ev*Dmuncjf4`&@L)pzRKpgYfQ-gpGrceji0;R8bOPap~lUy3;*doDmo@v8Ah3|Ht z>FXyBU3@9EQIR`I6Mx2E1Hw!fbT9!2BA1Jmy0`Xax|1kn;HYCBC4T%n#^>fTvQax> z8P`xhw8Q%68pOmO4&Rnz-R%dyo0WInLWDfPoyThTybxTTh+E^6efd1?8=JC8yAW(v z!BZ~-GfQ{SqS+mv$H+NHG=rI72MxRHW*So)DC4Rzolsl3vqY(Ny;IS7`Xguc;5&Ek zXOSUSpy$v0Mih55>%1vHkl|VU-aP^8xko&sVws(AOuPGN-NUs65 zjvh8RrE>Q#L+#RoM+f@NXt8!-7_Wyrvc(y3!^MZFjfUHESM$Ctf`1vszd);exJ4dl z%p_l35xh0Ku=%GUKW4Nxe-e10_-@nP2e>%tG)rSiZfRydCf>;CDpf7oATYVMV)fAGBQrxwnwDLmbF;DkT2niz}0 z0)+%q$?G+UM+k#~i$x7#ltK*>zYp9C1}To%WW=i>yH~euQ_PL(n;uBHnUe*~$4A%{ z8!p-{2co%n{_QN9TnEv?6$0KT_x(TH z2s-D-KwU`LYl1$K0zBQ(4}3J(lMZbIEk`zG8Dm53jmciYItL~m41#(Ab5IdTnHpRd z7w3Ou|9sF&w%oxs5IWsAzPn%XQ%rU)mvO_4S>o6vDQMcwy4N<4k&3gL=Kp*Y+5!e- z#gLa>auiA>9y}Lb;x)L)Qb;uQAj%uoXBtV1bB9vNb2*+Flq7;`P|cJZmHp{cHrYg1 zT=xo2HY2Q|G(O%xoa@uWFHWXw5@BhXuJOB#$L6a#wd&E_t&OOpF@l-Y_m z4!0DOo-`FsMs$%CkMcwNOerzuQ)zN2$9|jTh*I-7l~=3rKye602q#AnSpgN(;+okQ=io zQwW`Dq@~ZrF`r)kWYQPEjcl5gc?3j*9A~zr~~y(JZZJIqBKr4k?b-j1kLAhn6N)(Ff6rb7Wuox37^`SK`6%DDOn;Oe)7ThyFlQ(jg}M?Ra=&T96+_t6+mU4c z1f*&CVaPOjC~KM(LH5xmW2%DYkwZh73mw~DH)gT|l>T5@p{%aJD%U%@lu&sB)nySs z^a4wV6pm{!9=9&YXli3dGf0hgU|GFd`9@W*CHVfTUk@sz9$c{&XFo+cnWs9_wUPR{ z^_Nnmx}21?II04>cKPB=Bo&&d4PgYAKd!H2_#u(gXHQW%OprkaG8A4jAmG`KbEY?~ z;0nP8pqHN_i25PZXUA_aavxnl-}g4|YG>}sO_v#?7B1MUtnvw}(*gD8c4+66lV_lI zgx5&!ijbop*SU-X{Z1{ z6~E_|{J08=7mPFbK!vo2TcmLz;Ks9lN$Zx9#4YA(Z~>aLX{3`o%Rjlf?AY1hZkelC zHmQB;mXdJyI8f10v(x_Ja>U0q!S+U`+mL&if%XS!gIYaMPjuQTzlipU1Y4o(>+Rb{%zocyf|ETKL6Zd$)9e ztqn(;gky_x0*&u6;s$>Sd(|!e`he2}NUpZw!EiYsdk6cFa`9L=xwfjPtD3!e@D17c z;djmh6`Tc@Q5mA734!Lst}^m>JO0kFl^5-B7+ix8BS@b<#Owt{gG4?^x(LHAF{Y)4 zwP$Y@f?F;1RLjJv1ESQ{SF8m7sp(d5hht zX1A+8lKE`r$pR4E?EQ-Ns2SsV9=fIf3~B$?3TlQTzd{=tdFyBO?BbNZL`E2HSteTR zui5Y0Mf7(_j%Fm$R*S2)fb_@5y@lyO= z;xk6Rzt$x5QhfpW(pe?sXOT%M&@3CTzihQH-jsMS)TDZ=yoT+=53$Ao5ZQ2ib(a|r zG({>_xcBPS-Gu+D5)B%2R#!-W6p-=dUbcc?!)WS@a(wR7yJqJ!YwoEjrEM?_I0?>8 zdlA;B{@iaiNc;jTd4~tzE*Km1pA*0{igk|Wie)+DwB|gQVTiHuG-#lj-b#tvl{6Pplb=?~UqS1D&t!}{zhyyV+K81?%X*vIpQhQgSp4&z-}L*u>GPOsn@x$1Uw zf1vQN!`iDIKw@WPc&-irmr~f_@GGBUsyMnQ zn=Np-`J42J>-31)R5}su(Zlk7=3nTB)b7O_ayBBul$G*28N~8ZUd#C>K z1Ksiew*wclA-Ki)KP~a`5TFE!4lS^$=7Zl&^E~Y=|L%xS^@;}h#}1Qt*^ii5*?|vO zvqDis)um{*9AvUBs9KFu!y&_BNJIxE5)Ceuk3l2mzrQd!{3Vl)V{QiAQ@*XgPw@KN z0M+LjRLb8k+9HPds$+82oXSF~@}Av1bCO+r81 zt0M)h&Va(uM#d?Ir{xx$5zQ!PNomy}f1jIg`O=nT5ngI5zbWM(VR6RstJVO3NAfEl z(bjd2czTsHf&b3PfW6mV^pKBBcUV@Z42Mh+onEVzcQkyS^BTINjx~EGE7o-j-?pGW zCu|0eJ4*E_I_lCbrTxVNd~i17zE8P_PoDk2^*Nx6B*=9#%2XrL>}X8*@-Mu6uyE{H zV#QAlRk?gc)fdVrlX4rd{U%XBxmNxRypze#(X=fl^Ud7f7S*!9(-9~ree-zxn0J^X zgRe?&5l!30!3Z3{271fWPbcI!y5>7$G%4H$aQfoyiSSS+B5+;U~4!N+n(=K7nWo^Q4 zIdwUZ-~(>_xd*|o7i$PBJs$NudXFOeM*dkIcWOhnu5hR5`e4QT>hBTnQI7M~%H2~} z*9W`-X!G7PXDp5*?chy1D7~SikZNx|sBmv&uqspv090E@w^tTQ4#Ol1oiY9=Y!H+L zQ5NEFC=Z4JI}^|>R0p;07XLfoP5ED!rT)%P=Y7BY-9fXY3O?^VRu(V3k zLc=t%0Q~f07LWySc{gnI!y0{}SZ9=@Eg_Pj9Vb1weR=oW=aHM!Y|72$t3H+?sU0_u z2Z2YJYRM^ac64t!hE|D9ma}esO%n4@X_QBf;!if_yxV_!5G9ZLkKDM|O_o>5`u6u4 znv@srTlMq~GiEfFMw|K?oftdHdFb7t*YCpD026B|MCWvF^Q9|jTl-oeZKgw|bNqA! zl`?!A~*d_f&{ud01cjb1-D4tr2ah0WJas=3|zN*(fj6#-aIwwv^w!Bu-F*_xMW z&pYNeur8<_ORifs_m;2YdLqwrj-O@7q=gs65=#;~meCQMQTFdt9!qu3z6e5<@||+&CVN$t zKBW%R>Lvo3@bU~*tV6c~gohC8I z0##Kkcste5_4j+N^d(UwGtTMH@lslMKLwRhkbt43cE-20aokgtU91G2F`mA70EI~I zd-aJm*dF=|UR`*|&OJv#aZihUUw_7_d=@nymu+esIQQ^dDceyu(-Fog4;N`z`l@E}{ z_0$=UTshWXmY*y}MefLmp`OuSh35$+a6}i_!^Jnqg4gzPS*GHy6w!A@;Tzb2gZ<@Y zLb=eNxuRhv5mB6^fl@PgdxVjxAs{riI=`=1pwckWOe7?|m=JeiNDY z=^wDKcgv+C#nc2xB>whcDA##HM4-gUpw3X+6qJFYh z!O_L1|uB{*p2W6({hyx$TZ~S|i2 z(pfOqDYa35sg>BF^dxgpTkORkOpDmy55xJe>AYDUnuC6%e>K_gDba6Y?u9*_tr*

    =vUolQO6b^5oYNw4Oh5I=uO?%~A*-0muM^~F)B}3~ZmyR!K0Kzt68!^V_(ap4Y z8jdsup=uvDwcS$dVsmaUQ&Ur#YfW*o9GSuK=#jBki{sf^|tAUJVN+qYnD);QD4~g z$)zM0U`E{+Y4_K(0B1!MxpRO+U}Wx~4Axy0#8}jrG@Z68CVZH%gIr``?*4gxAv%3Q zlaer(9{q!QtU8S#x#AF6W0ZQ*qDm&_o9Zb6Q*oz{F+vC*W<&0`KK|3Fkb~+-7I)qS z4Oh?!k-OwTO}e>Jb=8Q6o?>VC6uICLEyINHHcT8^OPzW5mEl{BgZC9 zifA6`t8;9Dogm)l9|UCFD5#pVVVC9kk(ZzEUSB^L2Lc9T2df7~jOBMOT<{A?m2uws z_gmG=eYIQ5iFU}JV?$Ptc{l7d;>~DM%s+VUmG<%t2rRK>l$aw%e1`tsg&r*#DOV zz>goP%1a=4sRqLclz{9MO=lEw#Gx;eAhg1|o?QpdUPBx4reJaSL)MZIF65Csn^^V1 z3PIXRch$BAUX#S8 zW!p_d8j8VW+9e%=CFi6!$}mPpiJHnJ2h{!nN2(ghoi$HesR@KO<>Cbm6Y+5_*yrM1dc3vILHBFPeX(DEG zf^EF`t6H#BL<7h~a^nWmzQN8b5cypM4^oHP!9A~%V8vxP=LWJfiiY+}Ga@7g9msbe z0s{6ZsPqJZCk&we8gQGMw8-kD z@T8;d{W&!|&BULo3SRA8dLES+Ig7sL9)997gRSe>25;GSSkXT=K^`lJJ z4ly)HQM}nYkf>`0!}F5Jb}Z&+@kK6eUw6uVewp2j_>t~K4Rbg?N8)3bxRFan-nw+k ztnC7{CD03}@@^1R@4?SC?)k#|GYa->q=m$66%K$zNr{1-Z8?*tm)SLp=A?pkFtlXo z=HHQ7fnhnnZUn%0uR=;;p1(LIvnlZ-LEHxf#h2J4iCgkwH;YGCOkH?hV8SFUc{(|h z3Sc&UL-G)dNu}Dzj8=<=5s}4uxeqy991^Wr#e^~fG+2R3CRdY)tKq0$;XnqifdlK1 zjL1sGXs+*EI`kz7lX|Pn5In)~lQcNeBYJ*XT;TUIRD(c}*9ibNb@WO>gX2i*fNlTn zx^^9_>gSiRzdB~WHS8Xo>@(ei4xu*qQP;nwpxtI+Kg7PrEvF&>Z?cyvnEpQu{d$ZP zz`l}J=>$o+jl^UG82|^Je*>5&X6X}TtLyceRQMwx+In;*O z&m{R8R4S~IuqR0ARPtLop&;Q{@VSr{5KJO?E=CF}VD#O7tT2No^g~h7OK3S{6Or}a zvL#qha?-jbRyiSS#sNeU4h3_LF(MJWn6c6j8w!qt>~9PaoV$ZipYMoM*5a?hQg9Fu zSh|^q<`_+@ zVr{UU$2pzdN^ianUf`N?NIRAFjXPekM||{MHfGs0<>7i|dh1WHEHL zX?R7k3;KR2kqu9R1Zh(m(8#lA9px<+xV~871}&SL*uF>#4{?KKDiPsrY@-nG&Hzm6 zk;LH?w7F2_G?DG0$7HfyN(*1B2p!q*Z>jM=9+KQZ&U;;k7M16>%rjmjfLVR@HhZd; zs*D(C?}*dEX`K%M^Eo{YecvWCzmw>pa_DV>@~vL`D?3k=mPsZy61K?A#X7sNIhYS0 zMyq&E(Wv-2oPkHWcJX;FW3qrvrM{R)ok2*VVJqR9>{{$e3;aoN6Q<{ooMFcz>(jvx z2wzvH;n`}c9qa@C<?Oo}Q~k;(l*j(X?aZ&}CV7kJpGT~! z;Y~B=5TPT3C|*D03XsgFBSB&v43iOQ=hrV-eIFem2_s&28c@MmefafR9Jij zevGzpbZstny`dOH&w_X0yu}M)m`#Ui!WzZ~wn^7a@|iHkiFzr?@c!mF#h$bU_5GzL zRuLDtdG{H|)kT3zc+5Gu=+CYqj8*Mr+I5Tb5BfDXz?15JyI}yt|8>J}Nc`uO_%uV7 zI5vd`aGGO8+ze3&yHzKoGi`COVJLmd!c;Qjo_Zc15rT*}c}}eu$ANJS7(6 zkn1|mEh5PsWfo3vnr)M#-U8C(V`e&F$4j(C)*igScTcvWx5__!ZXekyvKDYwqG=QI zZx3}uYht!QZ0eZq%A}qth)yCzllNLM;~OMagLXq#1HqO)BgzQ}IB}BK#E;bD@7}yM`%j99bYwFm3J1O`8^Ps7N2NI zCck29JykP;%8CS-NN0%r7?C&{*=Jtkh>j0Z;AO-L~N*X8=e0E;m$r2py_! zmO-xd{Wz!1{wWXd9Hj;29A2b{Kd{Pxptr-|$$)3s++oT5dgS8`efU{hbVDg^_PmI}`_?P9Im6E0sjw2wXj*TIpG!$+_ z0RPM0P_sMypDWJ@eAoT_k5L<{4{QE?+=6__rk$$m3sEfcZ8fB$hs)HJDwE{pZI&gGo~nrx4&%6($iN676o`0J`ePim0Jm zNJ&WIg2*&oU3dz(G;a#$eIhWzbWCl>g8^K#Ee@e!xQJpAXc*Q;@I zBEcdd<1aHIh61S3kXUHU0CJP{I6E20bW~)2KF(DzX_?-?3rq|e?$pgLjI-cDG*Djwv#+oCMh%SsI>On7`3## zOjDz$CsOGnvpZ}|(%c8_v#Pgr)Yz~ZHJ=A>&EzuE#J5Mf=7_RO}QOdCdr!m88=(ZJKbCcXf?%(17i0E|K{pRd&zL=w1eYqC^ z$rR(NG~m%$GZy91H}kG`CfKNHIxaI__#^Wv66n6co)-Lb^v|-~329%JFJB_XuKXGq z#TdJ+;p?1^y9XPf05DG5NoYhy*=ktnI69HUx$uyn+gw#(1OkJ&CZ`iftcvw0PGf&2 zHcrXK&0#Ao^h8FJk$aMqVC*xoa~Pj<;6nP7y$5=nIW3t@Ge%{qaDybqWQ(^fR3%dm z8`PIe*7jS(@SkRyg?ws6MBq~EBK?}PeY*iLNLQ02I_l@QeV4I+&l}3 z;@|)n&?yHJY`|^8-xoMxWcuknWQ6!nntpMUYha)y2@|^|8Ju)&G-Z2BVCECX6Ubi7 zKH9-L85fj931jH6G&gM@4O_SP##+rgBmdU;v%;-h z_|6V3{!0(z-@x$wnfl5CivEw+{^LWb_)`c(4IK}{|Ms?jxfcBs6o{yuNh;7Z0uG=m zzC*5$5j7-E+G19L(#U`YN9l!%-Wd@&-!u}F^7OJe)d=-`v_+5?9R(Q#p<9^9Zq*DV z0!X{HwUdAACjR~9;S=b0*ac)2VkS|OQKex=1f*+#B89I)NWx!cWv+XA@$+QasC%O*ELld;jH|@E(Q7?9--mJo)TZsD9=3yaa_DEA`e{&_7BHeeWflOq4LI8-mncBI~8`~HR-wA!*k7pmi`T)A=$h^^J>y^!;Nh7XYLIo_f<5eZ(w-B{P@@t1TuQO466Cn0iwH z%K%Ig=mY>=fQ%)sCHBDTsSAYBK(5W-Cgw-JuJe~i)#;@JON~eUh5Q>z0%jO;m^xe# z!$G@8bsf#Y9#_4Vc01M|LDhzYV!O^?7LClN)FX0Al~-7C0KZe=2KY(!d@mpp#HV~0 zr?TuB9YNJ6Y)Zw4VH*Mnt+M>Oj($7dpAM>ELrZ?MSlu1z)qdt!na>H<26^s@?}CrSRr=`R zNda7A>YB(FU%5qRbC*E-5t>}-hImw7dcFH><6O$eUSJ>E{jBNev{B?tIoGZrydn?d zD;$3HJgx*PpxVC+*%|8lid*b!-oRXUTgK~8d;_`K8rlDmOibjn=L|FK4{S9FX4d#KVyse$e&tsa#alS3wGPO~>fpWBgIrNVvszVeKKECjGR zUIwk`biu3ZHpA3Oq?@o4q=-Sfpz+FD9I=THD-_CwI_C3NDKD_n(W|x ztSu*GmZqKAq~z|KbD>uvxyZM;H|V`l=~zv>uiZ6$5Wb}sO&BN|?U1@$Y{;=ZNo0X? zABmdF6x@#!n}&CrW1NIZSWqM%KMi1}ld)bwx3gj*L$We_guQBZ5RQt0v9G@Pkh?+7 zLDUmYyCzpn3k}3NWsIU4CyMcmTo&t`E0RHi_N}p~bKp)4&)LmDPc)mn)>Gi+#yZK! zdtSyO2XUswx6AzaI4oNvCNCXhWuE>uz())A0iEp^)k6g&8YMcLPTTm_90~|T>OBkJ z75mk&GV&6`EbJlAX<8f-(XH5^ zOCzP$?%VzuG@da2jdVa3VupXwF?r3^gz=o0gkzT%B(S(912USC|rgmdp?KBAA1 zZu^$k-$>ltGB_QAj zoHm$9icfw+88HOy0yYP2hRUnJSn_*00N$jqg?!*wBoP_%$LIr@wy2a($TUmCpZyA^iHEQOOj zvLh~`IrR?DceQFepT&xGrWzF~ydB)erXQx3)}GEW;Z=uf`H{qk~yB=4u(}^Uc@#*2_IrPTw zP8)8`Ag5Ewq5Z>o zsii2}E;BeBii=V>HtP+>KS)QVpz&<<%_V~VtJ3@rmq?v|0KrN9O#+0G`W^h?nj4{k zn*aR_v464g|92&|0U2oN-^sy0H2wdxpg)HI)4YxXH2NPF|10f3%gx*9Ksn`sY&K`7YxuM8rV+H+We$ep%&<2ch)a>MwW@s~pRBcB@YqO-2Amp%C(%ERK< za-bDRHe%NS(QnKtrtK8QvV*PBF)ZPPRDkVBe|50Je8`;v%3e-_5L&`O7+`v652{F( z7Se4~AY)*97!R^Y%0X0}p+bKupgCYOuyLxE7;R{D#%cp36zG)x1wJ6n_9DVCEHH%Y z(7-20C`Vgo+jN~9;nXoWW$k$`k|SC!TY$Ov z+?dx6N-@?#hFUFNaq@Y-^{lUA+l9kkp5C@~rIiZo+9Cr$9Ysc!bFbDK$n(@6Q8g+~vGg@L z8a}eBkE=oooHUIr205yyK#9-eBtU|@ozholUT-JgLNB)z)D|mk&{hIsyD1| z*(}~*>SIeVqOQpZlgy$3B^0x#`aQDco=fAYU8b1e(Y?a_yhpntO@mmwEMTZGSd46_ zMQ@&q4f{7UnT}1I-2mH!>RK(QbIK!hC9RO+Q+9_`xTXzeqWtc z>T0ZF)GoQGx#ZZPA8NiKg}V*GEI6~v9W4#A24@#Y6k9i%$1HWk2{|RBKBm%OFhP@G=9dol}w|6O$4Kl zKix=?vMt|*OMb;<=m)v^iLimzd)Bfz%T|u7Fqa-7!ibbm$Svk)YniV%&-+?;aAKC( zT2Tm-h<7<%*5gZQd-DT0m>}Vqr2nQ4W4qDJ--YGIfaWLhI*D_9Afr6z5~`fyCIE5; ziS6P9`Swb}1ljPE?NS{FbI?4;tF^3JQIe>hW1f4m9Eh6Z1M~v|@?g)`74|dPGjJKG z_k}yu$;yo2!^(?*$<-*AoON|&gVZRPOlwSaYA>5X3YPfzExNm%&CRf(VMyth2e zdwcE&TpdQvApS!`7s(@@iB}TRopKk&>Dr%{)-n*kQv{I-p))~Mzehy?sn48uo!-~K z0u4WX$9@G8t_%9fmT! z(f~5DK~z(cHq}$4QyrKVP?cQ*L+!8g;!IuWMlFV-rW*cNg!>`{?zzH&SUKMrtgZJj z+^N)HZ5{C}{(bjvP`t~2n;ankeB8tMJA05C3iTES;~?`V)za>eK%F78laZ3GC)d*K z1caxB(;+pB0&Jp|ksQVQ&58LjZbHJni91nmYGGZE4;_& zFU8p=9X-POJmE?F<{cECMe)Enb#4oQ`25;KVGcM$nz>+|TXbT*V23q&-f+=P8OO&o z`f$T4m|c7bjX^S^WkpO#pKXzIhZz~^`g)S0+uoni0O%$@6LDMHta)8TBTl~FS4is_eA>?+})OE)oa#??}*Cw`Th>F+Ol31u%R zk>Zf2TUM26Nkvt&sBWz)78DQ67h*Xh%f!^V%+D$}S(c&D4n$+JKuYD@W!vF(l`P=~TSvK(t%?M`WcS9SNlT!u*a4;}RVgu=wf|FjKn zuuXwRMe4ikb8ms^7hzFE>~E32DeoeyFt(g3*Tq-FuXFVxPbdEME28`}6kYY2%bo8BRW8R95EU4_1{R`1% z(Fm0>XC}Dja-5N&fXEcR5{iMHP>Rk|3doN4P1g#w>ITmhGuuUQGvoh+P7d+@stho( z1H>mXwWjZzi5MOJAsus$;-O0E(y1Tr`cqxBbXQeQe|f>A$gf@A@U|orfoBVud|HcF zkWzFper(CCakfZ`jZT80zcs?`S_xjz%odd+m-}briz^Ix@bjv>1?z-?Ibke)RZ9L@ zqe2>7bAFFoKn6dSdnx+uNFz?!t)2mk@{Q&sBB-2=mxMP1P<$M&*$!2Ia-)YU_I zfR1jb_Jxzpfd{W{A>2|)U5RPJ>^RSN@dtvHrE@ZJ(SZoeavz3l>@4@_%m%P5I^_jJ z{=uIRwOWxDsS<&m5RrRD3CJXVVaj{p(OZ^3=7^9qv(CXhXaa_*N^+RjRHhO@@04Q6~FkT+U0v06afM_qr}U9Ayj{&*9dT3mv)PD%HnScW0@Tb4AN>A zpy}5xj$&{sMs!W(KU|wy%K0V#*=tdohEK3ZP~^H#iu@ZUhhT?mb~Lq7rNEhpx8@s%zc4wS&96ySqCCcXxM(;0_rCcXti$?(XjH4#9#2C&*!~z0X&t z>ihqiRr6+!tJ!+J^^V?Kf0|H%4bm22rJS?_g9YS zV>}saO#8geq$U~)ukQHU9R8h2uN@R&;z%wGha87zf&^0jklb7CEDCE(2MnlMaqKBS zE}{>Ff9_(K9c!<9pZBl>*#GCf0^*Bxm9I$Z4>Ay207H($Cf<3 z3LKg5g@|RdGQUO-w8ZA0*!6-GVtq>Q1A5q=g8K}yUy%ZPP`(-wr}e=ir5j`a<`{7F zjQLW`%$#Nnejssu2F=~OOAI2%F(74t4TF@qhwj-AU*N^70TG*Kj2~OS>jU{=9un>t z)|WeF6pDrIYYhqIYXeE;Ye%X7z_L2Cs|-m1&x_MJxT_B7!0egW#|Iz7{1n$`2;ksT z_*+|~4X-PP4!|3jT{8wgV*5_-+Okdhe{hN-cFyebu}zB1!8h!Uxne`u!>{A;Pwfif zyas+D1vBN1`ay*K>cAUixql^@!iZ41XBpidHB(DWK3&;jRla2E;wos58T0194EI<)MXHBb}V zw{JJoH4O$J_osx>+@*$8#Xe-AHd7ijVhLo0S=q-yUEWPcU1q(`)9;6zOl9_Opn%_( zgAwJ9?<0p9XjSXCaFuDrX`t8)D3SHVbjcbifUA}1hx77AQEb@*g<%>7=wX;7Q-d^j zm0)THc~A|46Dc}zN2FS_oi?1>gI0FgA=^+dd$mywLdnQGbNUo0*TUJzJAV{yjxOvR z(4cMy4UpUIS;PI3YK4=O^<;}ly2d2j8sFu_XWYA>KSG?0yH+6F8p8LuJt2gAtS|lR z7!cGkN4fV_`2@!A{pFk-@Qsu7QTw{k#qTWz!S070D)q#!L*Wa_M&X-+l{2JD;fpAM zDi{la>TcW=#I#QgvtxPS>yHQ{WX9w>Xz_xLCG!NsxVU zAOeW&F;$}7du)~JCvs#4T}S@iOxw1;fSzE#=j|}6L2$j|eV!2jvPmysJY7ehgHpOt1)J*9dr?nE>+Aov;T@OdYnqBt3M;dPo27~EmLs6-xOqn%V9-P0=u~2NDIjETsY_jRnBgjb%tA>W^dgm2EivF3| zZ(!(d+Gpf&H?G+GV{7jK6Vn>abX=;cfR>o zOKV*Pi}%NigkA-f((O)n0lO{GAc9dj#|p&Y3My)V9dvCeo10kns^5G;PnGryU8i5H zZBj>brQv-lMR<@IW2E>nR>48o#*>xJ^qCv>qIZVxo)!C`SlTj@ty+V{;&>gjZ+S38JF4{6k$8h(8Ren{`)!+JqMTY2YH z6y0^%Q#aX%vH7BfK9WQ8)5b3&B^c`{axN8q20dK-L?mw2oW1-ujJJ1&%OW{o#!`7B zS4ctjIT^UR@*dnbF6&CzrSYe@d=fY5gF0kDAx1Jom20lDvRe3T&Ek3LL?&+ep;1xE zI29Y^P2DEbDQP2N!AFi;{t%e)pPSI8ThB}UdV6tY>l>V*=b|Er=U~}stqenL?Z!_0 zK0OpGSdEDxgXXUiNBaq|1f=962#Xi*5#I}@{a(V|PP;53uSQj=!>AvifBp4!wZkO` z&O@2$5s6lUN$!e+hcp>kzlMAW&yk>BNhbyRdV)1Ex zK?~}xo|_`1ndjIQXDp+HEFq^@k~0qksG;F0IPLhJ6mvo^h4h@;Nnq&kH-l-@2|;B2 zc|r7ukhrwXLH*96avvh`3bH7tH-dFE*yUX$tR477kTORFg`}t#8RFP|XzAyEQn4xa zKU3Dk{uX=G)RKaASk{GN!At}Ii}yuYw0o;=54)@pp9hO(wKT6#4P&Y(h+%_XB7J6T zwT88uJVMgAe|1y9k75axl5I5FR4x}vuY)LyhY;BFvH-dt#NMLf2+?~C<6-&DIvUV2 zqfN$41r!caC|3wRnbuQ>OW$=Ru+h5UWT-FC`MY0@^i{!FRG>$rbylf>W@IPod&Inf zlaDVFDNzPZ&wW2FBXy|%QW>2p1>WT}t1aKp;-@ITnkbfb2TTM}JVUv{hOZio^$1rR zJD;4z7nW1wcmNpaqL& zS8jd_3Hh1IY`x_NBJ#O6BtXV8YiCYJOKbmmX1`~K*} z<51L-Oms0pVC)ate(QoZ{#;k}Ft64p`Rrf0A8WeZ(pkanw!W=_i zj0%XuDSvq4H*1%gn)qe~dy7VuId74uV${`*yWEfbCuiuwg8I^(CA2@sh`TYO{VOeA zUU7MxM%j?#SDLm)K$jbtX2*JKib8$)(h24smQYC7N(=Sd=43{hdV}Rmjzp;a!&649 z)%^~BIjp571nAyxjA)LWTKIxKh=DI8crSg-?`(Wu{A`nJfo{m5A zU-ca?BhN9>;K?qc;F+I>0#VN`p9!fean<|FC7=M0~oUqR~Ku;KpXwMl9I&B zx?nsHfhwc4@taPO7yXe+>o(El6u#fg9pt_=LP7y5Y?3-+fFeNdkg_&{FT;P==_ zwK|c{Bpx}r4T1-SR)RdV2WX|?ME&)`EGjPBM7;rF$F@0h4&*uEz*h!gW|t+u$~2Bg76xd^8uA7K^3Y-$xBkYlqg#Xa)*kf z?HuW&nPRT0{BxXWmx5 z3NH3iuy;hXehcPZMY_KR_1qKHLdxxhd_u(C!&(gsBnfi)86Mz;W&j`UNs%Ql+yWZ8 znAR%++X$e);Qqd-9h}Cr4_z3Aa~GwfZzO}r7?^JXn|9NIWU&jBj~Ov$@mnWug`;d} zLh6S$;ay^_MM&{rVr(d!T`Y^)_!&T8QnHkYp0xtA0byjeZH(;1b!VRKRSt+L64o4!)Ev^$J$JGSeK+r}XU)}= zp*DH+@G;>yLh2=~pBhp_{}w3h$n|W&6%`O_#Lo}PQyc3$E93qx5Se1Y;?Hl5@r;Sx zFTec*9nttfJbDww{GbsBRIe%c5r9BT7Koez0u7_^l#_H~+VGrj9^Bbvs_;n~$`q4o z5MKa;!Yf%=)wW^w6#QYi{4JJzf6AidV{MLI)qTRAg!pa3y1${xlu66LM%j^nBW9~TCmZ+mFG|B@$ zGD>R(P;l?1qSr?k!wYxNPVd_b;ZUUk%6VOoOEaXS5KlL-HN~$ zk7u@ByS@f}#lS?k7c1w3a=n73SxEJ^G8!I}5G(m2RIiq;j0_S5x8?!J>alD`0Ye_O zxvFA?Rt8CAa& zTGPz=JY8WDC_;{lb>){`gB7P%slA&l!7Mg8DERG72D)&AbL zB-cz^0-|0Q;ir*LD7R>FM4PB)BRSN)C3!{R79}m{E&vvTsU`f&gFI~q&GsEozWEia zrN;Ae+#Yanq5K`2etb@qtn`nk?Dek$tJ|eE2~8|1je-iaY0C*5HBsC}>mcf6k4}F( z%KC7O%;Z~Y6ShRM7RBm2>d88pWkG^jsg*0^`pB`e{9_v{PA{ymWp-XQTB0-lp$a5j z)iNKWBSy2>)P^Po>xL=zTw1!-;GPMY11~J|OG1in)l%EmJL$KQ_Q75#-!k;679IQuuQC2Ioc0Mu@h&6@l%EJ8WDu$y`trU+9%5iWCOpisu~sW)W-y*p zVl7+?^y98}|2KYJPvjW@W&K)j4~O57J3u#h<_2_6vQy~^q6G#Rj#NcvD$+HBnx2Q9 z=TOUSo6##63hT&XWGI(gTqx~3+bv&!>4;|>U{hnl<*r6&QrL~(7v~fIxn*M)$ac`` zLFC6&TtpyRMZ2qR)L&stZ2)%Q1a-LEr1Z%vlG%j$)(JBC3a|kk$+CG5SVqa&Di5w) zMd$07S|S)T;y3lha1vH9Q{gRoePuB6;1UF&rtm+OYLN;e(%WgSvm2cB7#NX*5TNSzG`?)L*u_dJFic493BY;cm>hASLGKQvcmk_33@`;ax>BrCEsFI|@8xLB_Wa7{ zz`l8uX^2r-i?a`(U|yDjlZD+$C4i&Bqs~iRn{yjTI(E}~jted!u>_P@1%y(p8gk+dxK-;*d@QRDN8#5-<0n_XHbUhO0@szkW_F&t)%pn|sOlrCup(zD z6Vn!wi98NWu$H7Gi)QPn#L#-Ql6pN=k?mR2WPwr|qer3J#nD8C*crpiVGPf+fO6~l zNF-Se?dcYQ{P@dYofKNsvFk2d&f7Ydr+x`v#I;AKeyTWLh8OEjcT_zPhxNDkL{z)f z-DQHF^1I=zP-H1-WRgOHCOC)Yb9CNzi?;919}HVSYK+BV_CiAQ_t%c4IO8lC@e=pU z`0AzHM=Je1(Wka$#N_dY z1PJC}I_?bYXYAG7TJ|5i0bvw8-U}2k-bEh(5K&VXX|Dx+fuS=FtA(`-z^e~hDqMbU z)!W!dwvQV=3VBuDgm}w(wGV9I-OXa{Amai!R%(HxN2Hp<*yMuw!tFv%#7t(uzMQi)3;vDI83>b{|p%)3mom(6< z^UDcoy5B|EzR>jjWaNe7no{4h;(Uk6VXdi12*?k=z^s+gkuxbs5t^a~^UI3lkdZLW z?p@9K38a$eyi$;E1PGrzyNIfxGEz0)f!_B8jQ*6Hp!Sj_ z&6I_9r!Bn4K`zhkD76XxHS;_J0R5+Z3_<5j0gCzMi#hlI(?0%B_xS(L`WQ#Yx#1Xqhim)Q^6Sq4ctGe^-*;)}|BJVuGd! z(*w=pQwnD}xt`1J5uhRQT~9+eSu-e)ZYnmIMFlD}ntcZw7w_m+6y@l`xy%iNx}h&_ z6GmW+xsgtm^kHhd>U<=JKQCQ2Whib{*Yt$8^ef>|a{i)a{s|h13+^zmp8R*5k_JVb z^%@%zNFAwt4&_wT(61szv*}i^#I@&uyyxh)8>oHh!JHDo)_9kE8|z6N`S!D z8DoWsFaz!XK5lVBA z*9JO4p&?Qd$eG7|EeU3X+rnuivCZR)1i8~WnK6O))0Qg;HD=~A?8Iyd_)A)P%~qhU zG{)9=1pH#cPI0bss5cTq}du4O)=enR zMhMBthdp%CW03Y^lo3Q>wFHbS^}R~%5>S=N!D5Y{JiOpH0IOTGsU=$0ykC4p8FLz@ zF42J{6T-I)E30)oCCr365g+mQ4{hBXi$<2=|`JKGDJ{`uBXuMpBrAy`elK6O`x~~MtwVdx*e^f)C)#L5R=1F$v01*h&KdQS z{0tFYGcz3DYY*ntw#}FLE#il{+&MDmcl8UygGFdA)!?nl}mcx zLi%^yAVWf{I%BzIu1S7VPHvDmI74yM zPA)hZSf&*R#a0{EM+_;718>S4&?f~s!pa@fcLT3r#GKKG4e16C_`>62S6U()A)6r^ zBb(QT3!?=Rv2w@uJ;O7a)lMN6FpnToF=I~|!Rz2ewTQCBm|T-VYOWg7BSj5?z(caO zt|Btxe}h6o@>DXp2IEWGHR4Nh-tmMqho@xamIgD$S!7$CG=dAPa!-poW`^F_GsEvE zHVdVppGBv)V2%+6AOdkAEjNsDf3b2;Atvip40^ez#jx-((B%pfLB8RjvvP+B!vw;> zQ?45`j>8u-mzvc2_o4D-1$e>#b}#MM;hU6fAy61l;ZtjnDQ{X=gIvH~WY(J8g@-iY zTfA1Wo$c!#+m(Z~Tr&nyFmcQ$z%(u(z$}ga*$+i@_o6{ zxW=D|>`bp+ARSnqC_V>kuQ7dBIKET6Oz_=_;-3@gA!c7;%z+yM?Wo=sa~4#gBykqO znO$@3C=8x@N0?0d%_l|esDVdw=*BtC+<~W-CLAMNL=Cl#yy9g%R4RPSH84DIs{1CsdC}znnT|GC3VS+e@AaUfg zXDYqIz>1l2v?7bA4knya3%$aKk69h~u2v^*~!q~R@g2@ z9Q!I&oXlmr*AE3!`7+YDf_^`$6GbR>i1JkhGqY`N&!BpP8GhZ1TfoxD&tqXJsb3?CCWxG6a>+V{%MvV7hIvj>>N4CztXLq* z*M8x~J8%O5x+0{uI4#+KMGxTsrn}gdqU8AGooT4ywD)If(0{NkTl;X}|L%hfm6;6n zLwXGpqetT$S4QwitC_n*9pCeuXFzYM@Hfi@-=aWb#a0;{Z=EpQhssunMBl6?Xo=lG zmJq{BMS*wbaTGOxZCfRILs{H0R_aQ!n!BMP0agfZkFu_zNm{4%3l(L+d779e|1YGZ zX+$W{^>V9HW>c#eu41L*pHb7aDuG;>F|%JkqJH2ym}=~x28+`bbUT{Km$l64_Gr-D zJJ``yAxS2w$omutQs@xfaG}o3r@t7W%~gIwn^IVLDP?wXJ6j$V4yAPw( z1x0cf?e7+nquCX?3lrLan9pX!vf3dTHCcETi>S0m_(`n2naRpihErm^qw)cQ*9vuK zE?RvVF4zsnyHX?jNwl(Np=&&s#9yNFxePJ3YL|>Su2QiwT^jwkQk1?i0a7j8n~;2j zJt?JZJLluZXan9X6nzAjgyA}=3(EOZN%Adx_@o>Rc-Qf~$Ycp1Y1na5aa6vjgl^7} zKfgLD<`IWhPa)=~=Wp1C1D^Q5KE9lk9=Ws!j#l0$jHg*a6?QV#f!s+RxdWS1_di+{ zn`B6{*3m>BQs$z(CfXGZE^0JIlye$NTgMl5abFVu8Z+d~WG?q`pC%kXOM6~+RIV<{ zKMqo|rn|eaNCc_@!RcCA-`OhfB~qyI_8Zm7Z8H8wQ`;s@6_pL>sAW~bb%#l0-e;Cg zxqyqw$2ROEAY{+9WLn4y?XzR3Gc>$yx4`j9l>Oqt^?(O9rL8?m$XdDCJ7x7zmoB_8kfs&J0znX%V* z*)a^qansm_EJKqfft=o+2yC%)r~D@tHJ=6{i@5WXU`yw9Do|b1ctU@GPtY$Zfd4UuQYKKE<9f2^21`O7d*Xg2Q9R| z0u;L_$y4)-Zi*R=j{ye!%&r9JX$m6wQ|Bz|&9_>>bQEcqSrVqiJ-=OR;c+72y^TDm zvjHF*D}L&Qnyckiv!4p-z>m!4l*LvL&P52)4wmpMH+z3>or#w;k`Vv4THozB4L>v0 zoi^;$UNoOBTgCMae<_g*EN4WgGA98oLjKj-J3{3mw%!;kb3grMA7_XXbB(R(NZ~&8 zwN3^Ym$UQ{{mc_Qy?;)KeRzfbmQC2Tw&?ru*Xw#_B>t>e`>}?kacQ*yj$b z%Y5^O@(K&G)zngbe7c2q7k9fC1C~522ec!g%a17AJ*lQ3(fg58$BCV(;K1jdk=L`x z_9$W>!%|WAh_J&MS6!oef1MV*Za&&5k4+9w+|h#AKW+v?zs*4?aIY=j2`>>@-$K0a?I48(LlSzY+mmCpq%tsBh7=^q zPPr5s7+p?*W%k{jJb1~nJf!_)WrPfc0$ukEiI(I2YpIq%lCcSe zJE5x5!5maL-iSD5qMpZKi>{+@7+|KsrurgeaDF<@4PBu$kdaBENpvY^AVu!3lNlme z$Wdwx@STj%1JFLPn7BrYfkcPqG2+<4JDBi6SvImr_=%HZ;H;HzQAXPiVSM`AIjP5dS! zT)M6hnM>mu%;_s@zPq}YDdqvrI{G3jA0&8iZ@8W?a)(7c(0B*V9J{*bDv1PM7csR9 z9OFZ6y*Rein<#+te&x zw!`ko9>ko)i?wj0p_$d>#Xr8{YE-a4fyv>=u3JxXS}<&VdoSGm8x3HHR$K7@HE8E8 zueq1dv?o>E^ydui*K~=U_3EHw8j)_5#pVPl$c0=9C{uH>6!*4NscqLB1Z-wcw2f^O z^-?pZwHw+P6P;}*h^3#Z-&d!Xq%<4(*U0!uBJa|ki`^qN?r^$iA@f@i-Y&4W;^K_s z*lB5Q?=hlmJUg?z-|v7G2Ek$)tuX+o37x}n(Z+`vVocV2TnB8P@G|v*jMc=$jZx6H zC&b7?D5+;Wv60#C3-4!zu(QI2-ks$ony6LRzHEMG4E^5UDq^++Vz9|kF4lEgVZ(vYO!!|oJC5=acZX@e{krB0 z>l49nzE5FF@iuh}7pP13ftz*>J4q>ZxmY(EGEc-JAifkRo5a8AQe|Q~{J|N8E8^ES zmRc*^@EI4zQnCb=hKtd4ZKVXlMnXqdQG3cGP89l2A9Z3+RMs?2&vQq@RLb5yN^<3oV^+av7^+gIZ`v)X40ETl zGLb!`ByZ%%Uvi1j;k8ct_(H8g;$QcnYLGlA_MUQ$+j!RrJd~j2^G5u`GYv(q19mEb zyg+Y>MZ=t#fsBXmV%n6%RD(7G%bPf9M$YSfG%g#^)K6_`8X{E;u*Vw}QRpi-6w1a} z6zWVe@To?)C&M9xyT@M~imO^jaGdS*Kif9OS39jQmu@o+_1lcftky9@h^(WPGGHoz zPwOom?00JI>&=oT-mB5OdDX`{qY1dmcp1N6wBf#U-)k1yCBc>mNny(LLAq7+m^j!u zQ)U6NY^6K|C0vIt3H3|(CDmluS(+FVQJH}0FHT|X z!l!dybnc*5n_wlWI2!Zzs!{OBn{E+dxH2(7XOHK_2@BOEj)LTz%Ksro6}?{ zIejHW_p)h|chEfhBd0k*JBk*&ZwTUr>WTU1ho+Mk)MTj}25)7Rqout(lw29Z?;TjK z#$JI-(c0OrEB7>qGW^SJ4=%Y!^oJ{dRU3iJJ@boengE@51ty>(wP$5Q5m{%i4zm^T z$%OFu4meVAqaIdcTG8MuasR|1EHYNav(xuVXu?_ca*yYxrU#bK>PH+PfadSN}oz3^fagr{2n40L0U&pGAd z+5z*tdL4qW{DUBmAiNdFgktmEvYr5-$@>Mw**I0d1=>dI{l!}0R643bKh$P8;t}UT zq&v0vF=EUxPGkEcR2>ncE&PE@N0)XD%$CpSTNT+S?n4ym^eC|E`YT^2LNYb1pKxjAILqR+W8Pu zdeAFWr+J_QlY1fg#5PnY;VY-Cmw|H5fCM_3Ou|WXekbdC9G2(CpRf+v4R zQic|;z#TVai{D>T&%aFmH0LQ4aCC=r#rMy!;MTQYrpRz>JSAXmZpN6V!=DGmnjOyF z0mW+Wb0I(Q-hXBg!P8qrbeQ!9JY#xl-B%nhX7nnK*LPZ*V{8pw=nKyjtL2M1m%yqEv*d6_T=RULXLY6R=;#G zf9bf<^8{rI6!4^AKM*jJX1GQt^hfjs$rTS(dnSbV4ZZHo|SH>G(p2{%*n;Ohj3}#dM zhJGPhu9Irgu&o|esHRB>oU$#Q%c`#w|58@DWp70i%|*o4*X1t7;JkVy*BNn~5cfeg zMRxMivdH*3ceD-`O{+oU)apqkwb?gf!7aN6DP$XReffzkv<$@h{*Od^qaj^AYTT_Xjd7VW^eg`G4Y-w zc94R}9pCF$OILhoU{{8*hBys%d%BAo*7+K|e<$XG_ucWU+dh6q|A!Q+Kq_Bdnv&3f zhub0#JA-D{WP5xtAAF$RrE5XM{AkExVX*JutY~72f7F2yc??yIap#WNpJDxwF)GwE ztg$hw^F$;$!l=IRf;f&nX``VrC4UU&FD1ySrKdr7QD_>dfK66bSjgF;vVA8S!>q+% zj`(WNh|u}!63muaj_ySvYT&?ND3YDbA_+cpoNmKD4Y>_QI4@Diu61Dn?*6M+g3EA- zL=;|om}^Jcbn|^k=6c;yo!9Uka-a9O}z(;^hq$SZ3#0iyM$qL;#yWkyjvkvwNAj8a9T51ge+oj zBW5eZ+b^PBl^Q=J!%^QopgVI^Goi8v-PJIJe-pU-gafuH?*nw5VbtMmQ}NMJq~KD}+TW#!1PMET}B%z+yE_z80aSx5#x!KT^n_ z@0!0Ib%u(`hs1g~Ha0WOb<(PE1KKT+#%3_g8zzsFqDqK>%@Wd2Q~Rf-kHr>qhl|hIo3}3MIBRZtoI*j23K@P5^ zx#Ro$W$c0*Y7u;0@5QZGS`r|>jdVEz2!w^Y&p53Fxxsulmgr;n`Hp{>@%xW=#eH-7 zkFaPWy>RS3q2aF8UE@kFWJ5X=2$ybxu*`sz z+Wpv|tMtZBtgb`TKhx3O6I{>Fa({|zDJ8P@$NZqIU40sS%V z5Fc*Uu1kI4BBX{QlsWR3M?zcDQ}fx2Qe@A>q#w9Ic_LFWdMq7<7_K1rh$@Nz_(1E> zl6zyR0oR#(6B&a|OJw~o(U4KOs9?~&0G56*LY5c_Mq(AhRSsIABI$ zHdufB`dXF-wJ=E27^^P!CjW;}zlOZ$LIVmCgA`N7OIkA23#Y3dHz}eB% zla;2-vD3TvAEG^e07uIB9^oJBK>mRV#9q6z54o>RFW3+1d=xXp8F*r0Q7rI*Vr2M= zM_MR(Fcu3_dGd~(gdj&939)p;5ccE&EKlLmQk~oU-0CLdmyx zs$N-6^S5EIb_?A0(I{?odGE{oELx^Xze(%EB4L1Zn)6RS-T4IcJ=d?w^XSSN(q&Dw z?*hU6Fw3N4-dhEb%PXj7RIWHwTDC|KZPoEf{dn3tQk4#*ulKHFma+t_GE*o!L?_>v z<9W@@-oDg*kGqzowwS%XA`d(m*;pcQM9n^$;EP1IG#3lwD(TURQ2`smY<-bS)dj*{ zjp;UJi-F_^0P*>9`if5_A?lz-y$;UKgo}F!Z5LIs-?I*?ILsoAOvk9sDwwUbOQmsu zXP>71YIrWXe29WfN~8NK$4QmY#oWHA70kw!O3j!+fz7~$iyL3#@fx!=ARDf(V^{9J z&{sqno{M)k$Sgp;G{iRJz&cp|uRbB7J2wh$ZVYDBpUaAYCaK-J>wV|4m`0 zNx{)YQ-p%Y=4+Y?3>4{A;T=nEH43s1iRSNuz{cDx(4pmhVtq=epK|FQf<-}UgOY50Gn=zx?C4(%B5zG?Wx1y886Kln z+Z3-uKyjreo;L8clqj>oz~1c5*5r+453sr12a|YJg#J+N?-{%+qcZY5Lji~vX?=TN zZw{mjb^*%YI=<2)f*M%lv31r557=`bGA9#!rrUOd?=iJ_d;{M&Og=Je16S?phigHS zL6hEJ0hjcj649kRLEil0d27{MmY3@cUU3dU9a=I~*_DYO==Y23CRKmf7DqD^meTF{ z(9+`F9=GzD(XAV(3QTvS(4RG-6PTXO2nLl-ZFJ5&bqgk*bPPFWb*5!5kw;9&HKt&6PS z)eZ2$lJo<+Eag{qQH5?ZeSVZ!d5R0G8}0aK;wS9i=5EBgNv$bQJIb(?6-gJBpNeC6 zHF{KDn9DazU%<`NLF>(Fl+iKXaXTA(LHci{6IJ7DBO@`v+vYIV!YN*LbGq3|6_qR6>h)fXA zn{Xj5XWDQ%+B+7s{Ux=f<+><@WZWT#1QT@uK!iT ze+6pmb`+9Z?}U@^~^V z9Z|6^$lEjqsj{vHW^Ld81Mvc`@L2oQkw^y2*_S0=diIh4QarsfNqUlKR}X$>n|$@J zp^zZqO8VCb$0XL+IU9Wa!3Q)xk+;nJoL@B6U&-7Uf1f}7i%;X=4~Fsibljk_LHw`k z|DV!rA+i9W{7=>XlS{#w3Rr^t7nvCU$w>SU;3UQh1Of0<)mKClMi*E}`KH_am5$gD z10=qF%l(;^Mc50OBMn{Zt2nzLeTS%a{g!=k2YKgTD&Ivc0}hn1gljP08=^9g6i9eZ zvjn!sgUL1zi>vq7w*&lMcuW;+`MqjTec{jWk{@|SoCpwpsNXD+B^d(;AvP0c>GFgE z5f6Fe#SDNUY)ExEV=ZwMQB(=Fgd8=NP!NpUG_w;11$4lsO2ACSRe5(Qj>SOZL+c(s zK5hzAtMcQ;EX59|2AORj5x4xvqrs~EnaYrsS2$3uSEh!Y5Q{$pGs-~E*AJ_&%727# zxwhGWsM)%)n@JfPAmx)MSrhTG)6;P%UF0;SA_WY z{z%|%f{Qw&#L2^v|{GDlGk4o4cp>Yl+7B)G)2=2g1KXkY)w$*3K2c>zFgOpzze@aw9 zffz$mQE`nbRVm5~jm~17ENzq9l3a+C3I`w@#9r)FBK{%f5og~mBs%gyxLHvxWD5xc;{pj%ZB%h?~}E=)>Aw~YYPx1u8|cA4X(}Q8ot1^ zSM%ls%4mte@_H*3!)Y)h1{#<{HZ&5VnL@y6jeyxHy&@3KkNusK#QuQ$$MaVPV7sZh zeHOFS|Kj)jORD=sOrfPRts^0|@YsMj{1@N(FQn>UsL%d45Nu#W1x*yNwLJID)XDXfe>b7_C7qmg@eU>=W3J5Pv zUZ=GzR;}dgJkiQ;Dp4!843{#G7I3sz#!aiGfR|3Mh{bQvj+gOb(77vCQ02KG=SBYm zfG#nXivXn-Ru|Rg7)pWNsy4XyO0CRk1k16IZ^Z*$FE*v4=HvA?$^rT|Bsht`@MwwRXGq{taWxw*(_bhzw(u=n^ zSKb{})g|Lb%gp9S68Awv#eO`e$aE~9u95kg5W@0pQZE<0LTEl_Qp)_1En5iAsQk>N z#06}FL(o`t;+7d_jczh1o0T9GIoHE%jLL@6B%n(A#{PJ!6u+k@cLu%qaGhOk{RH%Q z*JRZ1&1)bZf5Nr@AF|#lywWxe7VfCyq+=%?+qP}nwv83rwr$(CZQD)<{b#ymdcSbywwqd*BE25*&Lny}}Uw2))?@+8g=Bc$<1lEroDBfW3!02+5io zbE_CCj!vo-DBll{-|es7e?lkn`jZ3#iB9kp(+C%#x{qm30v$j;sN5?1ih_$+J@9Lf z1O}-)0I1(|j~i7l`<7S=vD=sb`&C4`h~ocK_6~r?^#=a00~p)%*~9!DYa01f+=au3-e&V zlhkUv1WZ*#+_!$?GQKjnvSWJ1D=@V!%Q z+g>&uyKknuUS4u+d4EV5NcPtHA^;TtBLgLHCA(MNsvXt!9W;UCKGHv=rtzBuciI(< zK;y=wV%!P<^vZm7OmG<<1|?g4Xggu{hkQfHM+veChbLP@(Ly6qM=C>!#Rzotr3NvN zg(@5Bq*;;&sle@af1V|SLbnOwME zW=j<+oVSatjRT#Bh0ii}cXei=+uH{7yw~WSkeH207LcsKz)vjbtL6rqb!J^A-VCSM zNnuA_wJoR-cInZL*+qcV!k z{0B7mUH8oEM`~dC*fqVhF{-~}Ju^{WyM4C+DHy?Z$irfS1j%H;u?YMy+@XYnG&xi1 zzh%JRs#a;;c>0f8!#@K}MZcivWpq*CY4YTbOD1)NZxI6$-^d=+)dTB)7z0r{ve#g% zK4JD3PUE6)pSz`7q~cq*pEW;|kS~CZ*wGPbYHanzUB+2K23^wWg+x_CzSf1>cyI*+ zLg@!_VKSVfISeX4ckr=w2(=2L12@V~V=GPxRE1AYIC3@wYLG@0vIq@Tn_prAEY2Z1 z&_k9u9D&|U&;sdR%4pDCRu{gBFp?E5rwAdm)00J^_fhbY*9`{(XCy()>Z_mbVKHrc05k@|~yZ=YNC3 zr5vj&r=mx>3fTvtPZ^jaKo%?q1=w?kP{AT-ou z&ZTlk*sW6Mr;dX7qEx6R!FZag7^hD@5jzkW4&1FM-{&80Jqg9f2Q$k{0{9gQO9Iwz z7yzy5ph@u6XqYV#V$l07N!@$`ihq4=`wyRsoq zG}wA2Ex-d7^?psKt=AyLxDYXLX zyl5s*YtZ0~i@e^}8-TB)c^U7bc(-6!2uW74L_NceHK}#i?_AWFMeVxIdY2Su&m=?{ zNy;mf9CVkvS`Yq=@@V;Ve);rB(o<2#!8kLH+4X3a)YoK_0l|5ZyxgGWY!IPk_}RK@ zWqV^XA8WiEGwwcbuV+rTNG<}aB9vBvU^BI)=)d7V{2O@|Kmj-y^T9d=k`X;3EvVKD zUqY@Zbrdw_@GsgU90M5I#MQS& zXot=J!igiL4iKr>p~=z*YkufkRZX?-K?*;3;y=kQu15X6t-)=Qah!yq94KZ4IhA zmu~r3q?eM@PJCf1mlYkFo`hz7{_5D7nMU$F5WM8!?GU<@(q`FF<@F4)+j4JM{`zML z;}B#lh39*E4*l=crGM6DTE4^aoB!{;R`3+sO?b!@9e*H1fTtI-D#{nh+LROv2$3YP zmESFd-xNiF!KT?W{inGu9~Y{1W8k)PJ+ zCac4@2*EKLdP5q)85k_|UX%WHY9qT%ztNgdrSwp+0LakQ0<->MW+rBatwa9e5NdR3 z-crM;!79PO23(*K$_>?KYW?!E^TPT@c1=U+yjK0m&Fb_hcw&RN8W}G~``JoiaA5F^ zTA85_5R6}ABL`NeZiR!vF`nt2=(z@KbN2KxMCT|EK)ZWyi7`m$2oE5=)Q4T~P%vco zbudcq0Fc3+Mvn&x4B5k34B36GMqV)?rnQp8>cM!ox1z6=k$8qRkvX-OC9j#0rU(i4 za4i#VaIO=t7~SLt?wj}ruCIxaDen95Zc4*=cgh$516$kO1%~ZGTntC0zNziDGm@NbG_n6iLte^_a~0w7+1OJriAu>V%(OIhsAkIhz*-ANP== z`eMf$iu(A3u8b^4ASA5$Z5L$?Yf2O@l03!NnEf5pqJ+rO>1eeJUeGKy>SP9E*%rLLam5v=TJwob?>=(21PVm$j=bMryLiq(H$pRg`$%X*r&b##^L zBbmTbFOM>l?4R!)20$kp;b)Tdfwlh%(|7NOn<*xYzfCYI=(74}x}0i&xQ9;$2j+sC zpjS{54+%kB$1ODh!<0=~l$?+d3n*iW0pQ&Eas(++|UAEncCLm(UXAHB5zbB_W@wic;c0FOGgh*Nv74wv;zC;5z zIH`xt*ztg){0r|gO11MXyxOeRFyku+Zn+^Oc#%s>0Xs!rI_~B2FdVxy*E?_ourvx^ zfq&Y@ETPKj~)| z#oDZ*jZ_eAu8Qym=He`7QJ<_r#hf`h#1X{TGO~GbD#O5SXs;AHzr8PY(5TnI?&K3!XPo&3HL=+d1fr*l6qXc_= zW`kiO$`V-in1WsBat(;LK`7lFpi)F*6#M&@_hmW*&Ocd*cre}I^Iy#x0VJ#CKY#8U zk~HY1FX~3&>r?YfVOK&J@x02RXCjaElMxctoMxV zpo9yIg$+R=gR7~0zFngTV;rh&8iOh^X)q)_=oKEITeJvMR*HXsDWko;o2pHOipAT1 z0Xno^{@q@{_+gQ&UJ>lZ4N!Qr4$CwKTeLp%vlE&-J=S4Jr+-N=c8{_cz+RvCZEjfJ zyBPz%|NFpM?~6f5b}FP={^hbgl2(TE8))d}iYC7y?VPj47fif-y;}jOPOLBDJCv{dO3uSGv4O93^QJ@)O<}_yr+hg@01yT|O)SFPc&9^r zD2)xvk(?MQC&W9Qwr3{jwwoyW2ol5o7{s2`JvsFVD~T7d*l@p+x;(#FpWM-2mk)jf-luD_6^OA2tTT0K*`5}Fo$hwW z&*;i30ejhO;i9{81<)d5W(tAKAiKoF$UX z8Eh%@yQhCG<_=E9;dw5y0ZaA8K$*iryQE!@oW4zFO==_y91s>B7^Eg3Nz(^a6~*Zq zt0`wNPi6uM{~I2@KiE4pmbHim+jss?0+UQ_?5RBzBaTcTv7A^4qNGM+ z>$_RoNU3WoDh3n%khGQ!VZwotg8<`NO~&Lr~)!He!Mb2_0?sC+0LYUB% zd`Pc)wH;fGK0pK>HPMLee$ta}XelPFIT9W9SrEBBcIp$+6o%EM?~o(-49=C8!PCwf zO=8QLIDZK-z!Uy>O~_IO=9$}Web7o3X1ej6eu<+PJi^KVM%a^nnIygSg2CO8?Am{g z1XnO`fFr#DZ(tsM`=&NJ96s3$B+XLbA}UV@{|Szm7;p@Ova5dkmx%op|9HWCJ&^L6O?85(P%rDS5us2#y9B_Ts07y(@HP< zs28+O_=@P)pHmk8A-Y~tevyA}O+-EsKfDQI|MX3O>Zp!~Ip|O*gw$?S{hV#SeC&N? z8g@Y10SpJON3snJ-O*>U@i@~@a>eG-r!*tGC9%9Pb2OgNW7ZfVZs%HFNw-sP3(#+i zkZbE3yMZ&ip;_9;k>w0raU5|ArGnhWmH2n9UXErvz<|Nw8pl}Jn@J5haroMsL{ABt zLWe|z-qhn0MBP(K3&r!FzX_##M+jhwK%Ut_0Z6D2Ce?_IdMS+%ry|0Ltd`7|pZYXb zU}ZOFE?-fj!yO&3f{cm`^%k5f&FLq@WT(RPY4VgKC86HM85^JtxPY^zB`WFCH*+Lf z0m+-Da-K2WPQrK&7i6^;9rQKT^=Pb+xpS}@C1{rOr}O)@o^>3sG};=3x#L?A5`NDg z0FiSgNGIXL*)9{tvlV?A?Xa{oaj*SJVv~|3sY5FglkC1+*eQ^K^=bf)3tXN~B-Qz` z0OEZtWu#yn9j+=gOC@qO!3$EIcxoVvKEpL~v0e*Zl7QEE?PLk*&9Fs#Fr0o%HHU6T zweOvGDqqK4H({y?^OK%W6?2yT?P1I^t;_&6$KQvzC}X0}=x3s|ZhlY=umg2~V*k1g^fd+Rq}>Gp%DHn38Ua*r=^Wf~ z#IU+`jvY|r+CI32jC64E?BCJH$hvurAE3t^aLL@VX$DI;rRd|Tbl~T%MKMTL8s;a3 zW!bc*O$Fexj;Ei~W~80!8AMh}IJmH3(8X07KFH0te z;owdV{v=sXgJ7H+8gzlDs0D-H`Z1FRKwP>?xdy`N^G0wq%QQs;DG34r(XW_HvOS zEH*LO;L|}21GAau5kOAy#NIfE4Ye?fU~ z%kgCby(zZ|wQ+<`TwzvLfs)?3$HWTlw$+$4Umho?uqLN4KbENf38!h1HzlJvyp&&v z>#_6saOsTNYweQJ&t)me`OcQ$lBP zYIU9ersAFo0+0<)lCN#5EP56z=U3_09TzzSNsLvkzZz?lq4zh?%~0~@xpNvo?M7sq z%qD1MPKXQUOrmO2l~&(iQZ7PPV$#2r=_z8L%wkXg0Y7>*A>WYQ+NV1hr=_xWm*I3& zDS6v>8P<$W5#G-*$&VJA1Jlk;oqxNT+0jO=0ZAXv{=Fj@cNjgD+9%EW*stFbg0F~2xXZz4bK#UL-2 zJX2hKW7Jf#|BbaIC@;Mr$hlxIo>G^oLl4~5#Fkh~`oo;y>=6P#jebY3s1OL{i=OIM zHn-Ci1`yHB{{+S;LdqV%+;A^pFG?8oAz6MG*rt>7?a%Y#pGPj zm@Ii!cY`wL+_2CHt2%;qlcdd~KRpuv(<9>in=Lx2Xf-Lvf>P_W(4XM({32f|gwCu) zLo5uvmI1I!a2%7r_whH6reWLr@QXlcY z00(!_8X}l5ae_Ib5UrzLB?WE&ii^@E{lRu^VxfFM1}rLnK{!C+faV34Ll<3)2u1e7 zp4MFn+95(C`!eBW1{a5_iFn0wWK=uED}WgZ`#?%OB04H`Ez3IlJ*CCu^5z&C z!L;63WszwxML(v|YgDycVLQe-?q945l;mA|MM!>czIcl1_qh>nB3eoIK@jl5TWR(U z5pqIbmHw4yPuowx#E@ykHZ+U>y+JNXHfRG$aE=iz4v{M(AdmUX;3N_2rV;6lXoQ<| zKsO){$&amX78hE>-*3PquI3LiMAjDoGZInv%M%@rfr+t^p^?53tswDAVyzk(9;E>0 z9RrR&GZvtaKc36*IMy2KgAtRbI($H9%r&+yDK_pF-XuGabO*B!1Bo%SpBaf}Co(K4 zXVe+_z@Q^6HEekr{|$`WdT!9B#+$u_3_JoYk+f>>#2!YQhjT0FvG&ktMqUHA)DPa^ zf_y%1f(gFgo$pg!=1?)k&gAmu(V`Mb4+v#CRkA%qu{Cs^rsAaRaL^~0=8=D zy}JTo`ozNV#iobKdgYNkF01Fc>zIqKsMFh8&E9xB#pzG7-|QGw(594*>$GFWVjrB2 zqkzBgMY;&pde$`V$)~;D0U4{wos@U4KH4Yz z6h)Z!xT}+-xg6z#PkC~aOo$ruO^a@1Srry5K`;wBfr!Nu72$>^fN^J~drMEo0ePkY zo??Yy=zWVtX(+O=3do_QEGO*6Y4M~@vA{3$d(vv-vY*o9W)61Gh;c$y61>KWQhS6ifHIMskPs3YNUYnI=E$k z3I5(qalpG*x0L%HIZ7>rl{HICC0dOufHN%=sf1PpELJUUH_o}J$fB;U3)j;&e!lHL z>WP56XDDZ*ntCiJfMGLgYjbo~&zYv8MNYQm`(W>OTS@9cd%u7<36t~T0ss`|i6cAu zH{y6wH|`UjsB<^$nIS(iOe{~w0Q2a}(Ry1D<$&C@A0f4~3(80uVsFn^qa}E?-5$u4 z6{NqNqYS3#l2!f&7BL&Px?*SC&#G&4MncuE-X0*Uqs#$ulJ`bA4Xg&CloFnF)WLaj zfy?jondF;35 zus2q#gMY$9yu$XODyV$_jZe@)ns^&Dx^U0oI!e4OR?@$zChEy1Yk+u@GUMbXrd76~ z`bhOtUasKs$v?a$jN39PJh@TVVI%T>NNUnW8^H;y5sIe*;M4g?;+e%2T4+sztaCYi z&hg0y(x*P-8=qU1ySX=fhvE0J`>Syw8;%qX(bZKaXAYRe(P(*Z*yIL!)`Bb}(_G`i zmlkaSOGsq-jjCYKB7k4zd4KhA`yfXwlJT=9Ad_nc&JIzalXqW0@JS0WA zVeY@Uj^dxU1+@uYw<)jnM!1>`?KPT`rY*d%5#XrBu(&RXb?LV-Q5JvZawDE~w?m|i zJ)qosBsB(Xe+QfXqXk3%f!N*pMh_~O|3Z)d@?d{KUDBBg%B$B(ysD+B?C(%G8;)y^)4DsRPEiX^Vt}Ti)wX(d-+1X(A zrOyUU2d4IG<{JUYTkZt`&D+HDqR+szoX5z!N3~_n>ta#?7=i0{MBNjrWUHp(r5)JV z2--wFpBF-Gb4+XHqLBb{P7l)~6Jd`^dRF))%up4Q)5W?g0%yApy4s|-(&$39$&~Z` zvyzXUlKC&cD?hIev#3c*Z?D{ck@EJwj$Rh+@;SQ+KPfbFi@*~Plhbz9&+oPe{=xg# z1#A^Ok8R~!`$qns>csyeh^J_-fS{z*Zo`uSl2ol7k(AN7Ga9ASX_z395e9!QrjrEm z@h|$DnJZ8wi=?B13ayfCIN^^4TlK$3-0Yh2o}Pk>no<=cBcQKGa~!bw#9{yvJ+4g> z8aJ$qj6JTmOlKUkJ!iUHZQtH=bOGSCNUoi$Cqq88(H{28&_fZBwDUSzPy6ZMDO|09 z%RXm#8#lWK6w6xeUN|6nAWpDOs$CSInq~uP9pqoAvsVG|+Q2T5rzo|md^zye`@&cR z!1=%)ppQ-b!!|&kpq)g9$dDKZZmPrPKsS)1yRp6lz`@3)Y{Z6$q((+V8|2std3)3t z?49@p-v~$yWDJDTNpb+S9cK?sYU~A|7;TxO1wz!ys|9s!^&A}>K>XOd-g6umCN4g! zKk+xXO-TXC$cF4CeM}W)fXEU8PmA-?H-1>CLC_FpD-Iio=K0%zdk@TfEX#u;J1VABbj)Cdp9>>hIvWQMQ+lW>IV^j^ zd_Ho71wLV4h1!6T#xSP17B!{Dw4R2^-0G+)u;?udU z*T1CPy>dA94J;SA!``#6*I%#7WW;|pzdn+brXhB6x;9(IK^eg1g)M)tLR+NW4s*`bUKN$vH81ng#_85vPwK{ z$}jMObVJ2atU2+EE|~@(J!_`{KcLSIbMh1$H8i2OIBCl;Ijp%{L1I8j%Y%2L0I#)A zVK75{joNS|Y}nkHI+?r@wRYdkutEON7T`KV7C%VObEUsIw2YTFvJjWcG@;{!dIkUc zy7XZ1cY3JzU{_@){MBmA52#R0D9zP-0YS$@1mKqob83@YX9Zrs;0CI5hH|BFA_SAf z>*VQ=baKpJE2dexNzd}_pJ-gPebwKN_Irm9wB-AQ!yLn+B0<|0Ma4qYlms7}vPb)S zSnN{Cj`Mk(7pq0ZLLYzDi+P(bq-v+>{DJFnjCP0Qj~Dv=!+j4x{0S} zc~kMvAEKFspDt&Bhs5c)oSKe$xpmu=W%18Z_hkN#M*cH~xAp=*{tmyC(3XpW*OY#O zKWdk~A**arr?zIC^f9IZ+3Z#*lq?}~v&vWCC$kt<#sIjDYgI-TDMN|L->2HK>mR&j z%sLCe*&XXI4d0!JiVLes6E@~X8G-Ir=C7+kD*MbHPB>kF*1iB&BuLyBu9?M_O@7|H z(7$1vk5|C5>+{6wrn>Pn;-Bb(G8J9Z-9)-};@w1l34T)hq%;v(p4Fv}F{NjiY%(@u z>LR_l)+8-h9$DSGhMV&Qk9fCugdX2cu3>0S*}gjeiAMP1w>|<4x302$#+9nm{6U|( zP8-E$5q*0%>v9YI@?#b)-*n zl(M@651wLO3`79%)Kfe{`I|*@X1%bHj4B7L1Pu{bSQHwHE=O)XXP_B-Pm%AIv6;-5 zKF9^-{LpoFY`D;dW_8WI7GS;mTP|&~%V?qNWb+u8=lPzVb!)(y<7&F?rC3@(0T^J{ zeZ%?UweyxW;yRsjzp4q315)ejZ~Q8Wk)4L!V8$Lf~5LDrPJVf_dCi(J~EBW`784XZUtlb4lz

    JDd0mvD`Iq)D6oH3#v}16l>rY> zl(_+fiX=WniYU0s=0Uj%6sUdj=83op=Z(0E=hI%BB^GWKlZp@Qlg@9clctARmCHv4 zD5iSplcxJzmC~bXDwss!vM322Zc8O_u~o++ZYmbZ#!sNl5$k{Os-9`3Q4g*tStKd0 z-@+%U)R#Gx>2<@1Drp3*D6L20l{yi71~db9Nh!368!GN-lbZKeD6|6xD9m4zB)VP` zBz*XP^(s<4?ZHz#Mc{1Qg7zI}ugIhxX0Oy~V=bl&Xteg2JKwl$<*21kT+0ml@A;U> ziZ>*L>6MldBSO`=(IFnoHmT=mGzCsiGS;VFL2?%p6VxFUa{AlLYa|R^E(f)Qy2k<3 z012o|9~+OwySs7;yod`KAHxn+xG{n8{<*@-SsH2=D@PweV9l^Xz+AuYINgj z>FI`HdO0TOSCk%#C4J^N#23A+5M3gFs?z1GDbFaZaKuS?X6o*|A2|SNvxyNr z*2A}(gM6+NWYFwrjJ~63-2zYtRX^KL-Wu zzB}2e^24_on*0rv_GH7V9(2I3f)A9?BavNF=5O60cTtEG{+So~)YO%dLrjK=eNoQV`jpPqekxnX zN=sv-vulmk4t|6f!|_VdlidJE7j~NB<@;bU7;vCOag~P>JosZRAk@!XUVW5^rjJ1B zj{OVg%A0zFZI%O2CkYt8{k40Ghz-=mx+W_Ja;E6-k*UjDFoBPR`5^IRnV>P6dbwZV z&QL^)zVG^RZj`SWbGK(+@tRPjYkIUKmEjrX+SV;1X+@|6F@k8wj6tagPdK)F$KRAD z$>`S>X2Ku}B%^1uoJ5eBFfrb4e03}f=HG8n70e52Z<$P4Yy@x!9&V2cqW(S=3&od> z#EZta44X{tB`g;mLToky=2$R)kP5Kb+pSdzikV4Y`=3d`Y9rh!nPu9zMmSzIJ$s#0 zucsIV<7{4vHx?+jIn15XjO{TRsOgl?%sI#$wl%b&oiT={j4SUyi5Q*L{eCSa8BX|C z5=U>fqaIM3PXH)klHUvmHtC)d`WKscthVC#TwceU&Fu^991`8vi*EL@9q+BAsrsGm zeCI$947*7a!4HyG)M?OVLG}{-mu*${;NjkWMrNLTrrFBC%|lj2fY1$FZXB%?m^jFc zT8nMI-}81ZkfNisu%j*nPfl>-hfw-WyjUH)xbwJhMFEy~b&H2uoF}TPxbI9k_9iE3 zer@N03uQ(C{Ah>ymDjQYKvI@i>z9Xv=W+hT(vxHSy24cBB84gTcX zST&6VtlY`b$W_&s5fO-J@9rmsrh)}Uh%ijHVO#>B$2n)q3CBvgz!`KA$8%97&MV%E z97)=VI-YRATC+Z+@Zu5{P=7KN@yjJ}0 z6`{vrvjb9d`iu2z(Ky`ZQu~i(Y(Zv@VGcB~JlYl1TCp;s4Nwo(=MNeh2A60&p;U4D zgn35fm0rGosBSYsz!YlAGk5i+--gp)7z+UZdMUbhXpOe00EKox{NN_m=L)(q3VDL3 z$7EyGr%cvxd6yS{hN{G1%TPTGV`U5p3q2b#blZRf)F@~ghC?+sqWfM4&F^yM(ro)P zTK)M-2=X1Y^^wd1gHIB?!)uAo12PMCU{`BBN+jD3j?2G2n}(h>2-cG>jlX&$fC4}p zvo#Q!taq>-hv)ec!|_2yCS_~X%c_rQV`94F4{b-OvYU1Jydf9Ra$5(t5-_Vb$*9wGtG zJ%#d#9g!Vmdx-FK7=CRiPup!9rejLHNikvTj(B|;8F)ao5e0P@u-6>7d5e5tpTm=$ zk0sJ^fl!F<6tx>mMF5ygxMktI{?u|ypuN&G8-$?QrP7_Iwq<^~b>8f$A>eee+mVU& z&r%BJ92*;+9<-%4^u{(ROI4IMoM_E+7L&KSEw+z2tBmiLRF*5nq)5rb#D&*9c~n2! zZ=g>2f$>7iscBNkt+d8mb;pLhp_9mpDc^3gwjH-4wZU3$S`KKQ=ZZkkc+A z*E;vsJH#mH&Y*WUxDhL_tFx29fJbcqWqddw)eZ{Vh>a%CxNZ){A_v{9z*(U^gh^`( zUA${Q;g6xRMFDu>3I+RMF`Fqea&vF z=qo=-U~ASMxfTVBNx^$Z$A0#1_GbU{(doEsCTFU_nRUEndf|BVE>H7!ibq3sYWw>p z4BwN;=I+T+zo2Kr0jot`SxJo{QW;38&+hm}-r;()$0Uh$Rb=9 zTYWSbYDiA3cJAyRyg&5!A8+b_=fOwVw-%*9@L!DLzwdMZxEXpozTFJ)EkF=0VU0kl z|Jf?}zrMi$$o=|~@gspHPjTqc;46d`lLMVoK@VgYkOIf0nBLVCleMIiwn)$Ad8hn9 z3zaZ66508bkM=Z}_~h5^{&bnS&gS-X0epUaf%F0|8*RqbksXvpRz@mAN+Ko`b0yaj z;3Eb2^8*P22ZKeDhZ^wt1^R$7h@cFDM(7LmwhGVyKqyS;@bxr;g5N2Q!A^m3^(2M_ zrOMs>*->UDo9$dXz`$ChoJUAyXSlwcp2SE>r&+s{0-UfhhefL8R@Wz>hhVu9Y}NOJOXaH^O2h(tU|~8$3XNtV)f3}SRKB*y>D=W< zo2?(}^=mwja{F<3U$zgGcN}nnt1oyRhZdy(OMv$hxfJXI0b*h76#UN_+d=_LKl6Ky zD&*BgpB-x*CS*2S?0accETAzB^tjBtIV45sMgoH1nAKSHNx5^_vLFSw*;{{n;dBv= zz{32cxuPi_!4cE>Q&|YkSphaF1VX0Ig2X+9YnFYU_@p?hOuo->xGUtXoZoQ{UjDBD zW5GuTlwF9+5qU6zf zDXSo>MuRB-VX<)kb*A>qS1w^DCKL0t8~yyfbm$#0sT}?_*~Z1W7ch%aO+j3EccR0q zl?{3z6h(9xtX4h`(6fK&h!A6sV^p{AWe)rQ*-lK+8v-J0*=Ymn{*PArt)hwp&`4qe zL4QK%L%tgs{=gsNE5ipB7Y9a!f?aG*NK3iZV_?i8{?!jPYs1Nq=U==O9yL)vO;PV`AzW)O{_IGN(62A_oct235^wEtAt()G`qk= zp9dO~l|8;}@&PExE(hAC8Dgu(?kDgHbR)zoSTi&_4ep_%J_9gtJ6{DKAfs2tPY-U` zpn$J~FQm5``_LeSj|lwDEvHums2+J&!!MRkDpxR<5&S^7@)xVwE5$REm&(A(T+trC zm%zZw9Xl*vSlD%9p?EV&JC+I7P>xEguTe|BxJ7m0-TL_-Y zJ$WzwftWjU*q&ZdX6%tG0H#lvCNozs7Bg4a2Ga+WtEHQS3m_`S>@}))0k{Xn?I)|H z8wTF|E!49aTL8QTTLi5I8|g{*2r2Ui*v{guD*Feu{u_9Ac4 z*V0_;-1&oUU@XEdOKL-uM+ibek+a`X@kNI8W_j-w`C9Vyl0;s~0`?N&o>}Kk@Y0@k zp57|js#!ewDTWRm02R*Vbh|xzdf1MSv|_-q2%TYukqOMaLE%dmJz=;;qOtiL(cP89 zrM;61)|&}ctq!x#-`RXkG}JzR%SpdU+5W`A?^{NiK zSAhHceNi6y>K=x)gGkyOX9zATB*}WDoYn6NhDzB%!Z0_lR;o|khje-6%HNhj2n_p_ z?iWjRVOlk1fa;1G?JB-KQ?iHT4)x~3R&@ptAu@RA}a+3slX)}xPLz%Yn)9Oz1P$q7pyLhxXZu|0}? z=+y}lLF5Ej0*%p8kn|)73b7LbADkB&mdy)3Sz7EqfNg-d`n|ayPgU@=u}^P(YwMEe z4M;qDRm`%{;rqXgLc`7=RYM7zvrO*f=I5 zx*)6rz@dgFmWy|(uTWIQ9#ZnUm%&{j?pObw+dX*r)*r}C>-nSicti#Isvls zm-@|Fp9+%r#G3wN;(r-)Wy{v`mgY)p7}Zk_09M~L!B$HWHOHfCk-vC=t^ceHckMFT z#x%5EY8FY^;O62+fOe9DYhk6dUuBkusRQDop!MBVXQzOP5}LNd4W$}Kc@7bj1kE3g z1JyHwo?|Ez-9hfc-2M1&Vi={>%!U|(XP+6SGsAok>Nq5vO?dY@{7+1*`5Jo99@N20 zK$d9VW@^<6Hoc-&RLGQu;nc{0w=*aC+>Y$fXNrEw+t~Rn7?KA}#4GUBu^=ZF#(n<* zl_0L>@v3zccx7A-jpJpV2+z>uU1(c2L{QE6%&(qdWlDxN5T#_?;mi4gQA?i4d%@Ik zbGQ=X5IHWoEEYOy3Tn>4G*)eN$y8xO3VnL+-dSW@GfslSlYnnnlIbRBLdYe2^J=cj`@w%=3Y z8>CHx5d}H^;u*n0Koe9vlFtoZUn=fm1YC%=D3bH-n&nw*HU~C#icz%~RE;7!V4V_~ zY4%YW>RLp8jz?-gUj!inGlHUB-hCpxdYe%g{?Rg9V5=APygj|oA{bnAZ-|9zn3?N} zs@ot-WvU?)yTn+9V>7zm@TE`t*oBIQy5^KW0K^kVLiMK2PY)%vxQWip$D|odF;5XE z=P9|JJNYh3ls-DGq7-j9=nUi-kTUn-iUUO0$pE6+?jf6E{|6Ux^$=G{vbX+uV-u>L zWuhw%mnvtnPC5?c`W!!}%UtwUV?lz;WggCmeTT6qIHfFG4K=qf zIqXw1bCfapvJyKKPBM3caWYfs=3LZ}@^rYs$?pOg6(uzh>DtKsYXVPxz?KBox5LwT z;AgbLX!audVv4#cYT^^lT2zpB*9mw%_Z(G}VCrOcPS`-|Kz6q5nb!wSPD|rV{W_5( zN8{?YN{sx#!Qo#78}WO}q{)cKYm75%4J9x1q5DxVR^FQwCmhX8@ z_dlbW-|^4y02FLs*p!=Lpv!-o;C4oUcmT@JuWc{gk40yciSxkvakVK#MVFGRfdWPI zsD)Ihc92(;LhFSj-e<9{<>be2ho zhu*w*{yrRZc4T^9x1D5PZNDhwe0=(W{Gd~apZZW^%+Wt)oa$ViG);T6Vx*;iH~>s~ zMW>~tzulSetV~W!c5Y5mPI_gh&9FWoO?qXe@fse_q`x_ue08S%9hlUmdqA7|Kw=2&^EdA zRlj770H4}7OaqWV)EIqtB2~u@B?0PNB~Vo-_A%A9C?hV6>a{GhOp}06j5!*_N~=B2_wFXIkeR~Nh4ku_314MYI%^#*EoRD5L+{Ql|idXYO{KC zAU@K1Z6VxLfw>_b3G|5VVEllAU4Vr_i`DdiJ^O%>(q@1YAX-yXdf}j@iqDZLr1g6U zcMBjQN$7%mL7}CH56dpOfB~8aw|A(i!_|$wPUor9GA+GzmtN45MmuV`sq5isJHQR4 zN&>rVz>W}!XS-0KowR;Fpno)TGe1LpPt6OgtsG!OcC zV?HGzygi^kqhANA6|Fs_KNjj;Y`+fbnymh3zaZ2#1sb3}zF!JG<>$gzAoe%dV5O$ z71TBPEgYoK24u@60Fp(=7(|7vH{nPzFTZ~SMM5zzxsMh&2H&l?PXS3vv7a4~A;Js+ z3qcczj-x$70Ou>Y&w%J-&ouDcQ#N$>B{L){rdL>h7Sua%AB2g>dFzV^wax`q zOXG|Oktb=~2_1B)g2>rO*k}y1C1Okw7!9xg38I3+f((X3qGnC%UuR68@fYp?hlZC9 zU2=&8sfE^#4?&)W;X+4FI_D_b-vqx7heXE8g?PN!?P{~z!Rf)Qz?W8~gQaj7t@z63} z0&@FQkWOe|G>EiDW3GZ~%Ma>XLC9S3d_5fSx{{9bH#@3Z_CQbc+X9{a6MO_v1}vPu z$i6ou4_bR@zX8+}I+!qU46d2(mFvBAQ26%^?JEc5-W@U&XjT>c_f<0VqyMAwmh`y; z;Um(h;5P5M2H_+0Mj2cuaW4F768e#5RCK!swuIEfdglj;9fVi>Wd%v2_wQ*z9f<;^ z$1_>)7_buuc<98APw2!Boah0BR+t7}C4kO9@mPsNd#VyV`a!3&bpYoUz#lMf)HqEy z$@0QaP(B=pA@*vn@$%0AP=8hq4p6?hRR+|piO$?BhW_(e5h#BE(i~Rzj{=mU zFP!;i0pfoDr8R{K%y|6<#JV4eN(c5dMNVvVhka3{BWEL)>T!(-KKc<1I96?msfsEm zOa7srGbae8Oi6R=4P?xZll89w;9xFb{zw5-g4uzLB`r)R4Di*e!!obt!0c%~c!B%~ zHIVgs1=RMdA+Vb4y;8<_<<%d!JTK7#rTOPK)Tf|;{)cpD)sGy&dBl#l*xethsN3&- z00MAD6mL#m0h};M7aiig9Y7!>x{9Vf#iBm^tyctH;q}vnd8`n{8~yZZ{~iIDE!cI1 zzY*}S0JN%}`9f}iNr3tyC5Z690rUp)pnVL;v_BR`+)nf8q#=OkJ+pCChlBwuB>cYAc3?v1E{DH61 zC3)bnC~$`PEuzC*U31|_LjU_zp8BxUD>d*{kzFuG1d{W6=mXK?eU75~e@EirI|JYW0Pnhl?9#hvUMJE7rYo<^plESu!p5h43rMjVvSEYnly1 z`QvldO6CC)rQQLj{bWPOSMD}Y&8C4?*^5K3uKnbEIA+vX^y4nKQh#afojUAFCOWs? zZ(F6{n%R?D(M}|Uil4T&u`{tTUVg2%d2OpW4>mPdw@o%EeRH^slGPQQu%_(Frw3YJ zdpo)4HGPl?O&E|jR{>7`j}MaZq!nu~evj)buw9da5DprgKdmf^_J|)q;BMwER+vI# zuL_uWtNA8luML}PuIs{YZJrhjm+mXJUk?;46L8l!oJMSH-RlipE#~80ja?PnH>zNw zahPa_N(~Q6NAt^S%h$sPtGKOCM%`yzU^VP7(bu#I)Dtc8NclTQ+>^TqH(N9+hc`v+ z@1M1BItaH{s+IwBnWcpw+dLyCdV{5qAf(C*I2(I}V%wTd5rorL`(O1BDwP0Y>Z@DnpCFQ&xt}rZ5Py!4B?zzU87#XSDi8{8y z=QS!lD+*13Sn2*0?m(9sD#!Y!qk-JaLj3pqvQ-0O#+oyc%BuwsZhj(^bHir6x@s)S zlFm|sR9Peu&rmT|@N4_RyK(m`?rCX@$i-lT8b66%EzAzrbv}Q0qfT#&A^X8N*o1D( z?mod#6KuPTQk)aeK^uS&b{a>KPlzbmvY5v)_n6hX zAN&BbJnW?AqTvK8x0Cq%X|Irg+|6%%z0cBxa0b3gQVFX^nMrEhu-qFhd>DVW&&d3PoNSvU)x9^KqrzCZ9+kYCKDhW%bm z;37-^%}O@{T@$uMGCndqRH}aw+7S=tFpYXdI7*~6wBOB!>hyT}o9E;CH>Cg^2uJEV0GtOhx?ZL(7B@)kJ_wfV7Z5qO_}6P2Z|oIKGHVhv++1IDJdSyYPG8{(>abg`L#j zA#5xBtju22m*TrNB`n+~rRlP$&SnUYJeD*-$(zE8d_7rFY&m&w!3BLwMHn|OOYXyL zC%H@CgwGZ%bc$71CTQeXWEA*wv5NWRY^{<ZZ`BOZakTC@AubN#`Y@o?ANJaUzq zdP(MnwdRS7#rXDd;%F)@THO`VR~APtf>oY$KxHE*am8vL!*u%4k*fchC}^xjY-f z2FH{o)*KT+u zBhIGk=Jdc4jG7W%AWiuB#Jqx=l5KBM+~B{v-(S(PLo7KKh~KD39LVOSSRZzff19m$ z7fx9E#$ZxO8XduTDj+vTJ4+f6gEoT=X=`e3p&1A&N@!ZV?Bdjdzv~hk&xd=*Y(P@^ z8~Jsd8&#}Y{~OQOGQH_;F)U)h?Q*jq5b!??!8RLBoUt-9OsaWkw8}XFl{HoCpHb>ZUt3hFa zr!Ovx)2CH|EB{4mRxh~@2`WEE?j)=`7jUmE0qT)&28Q-xU~`CVn^(i^kZj&KuEhHH zog*_pELZ)`{kE-#g!8}$XX~thtSQfR9wo^>K+v+fqaDtlr+1|GH7uLrpJN4^5TS~U zNrI;*F|y{HKdED?4%ZVrx*3IzKNDN=Dy6MSSneQN&=}#02UEcc5hy5Qlc71i6q_x1 zzcX1kSsP4^gIli(T>omK&NkS_$UDL29bPN8n*#5t5i!qC5?G2jt*YYO-0bLm0=YuG z!UuF(x`eITP>(I1J}odkKag0$!!jffCaRX4^=WF3tv6OPkMmcTc}9;E>g5$0I89y0 znlHiF(>$o9DRqsF1L3LU|79mAqBmdAJGJ9nMy@`s_9#fIr46ZzkEuU{m!n!G)j2vx zmr-BbeT=-2(-)s>-?f&OBE?@b)2F(iJc0!Vd#5+hq9+YAs+Ze}gQQIgv09JA^(chmK>9oca4kPinYUT5Tw%vYd~5|XfvVA{$TWzEzpYYp;7UN-##KE zlWOscwIyXZv;p?M4kf5rH`J(D{PK1qTqEfgBBZ=Vb7*cPF|r|3z00M1m&KJkRErLH zl8(M#4PMT^fjl8QSrlAob^ClI(V8N$V?|e3zt0>$#?G);zD%;W5cqMIRk{C}V;%DA z1SzSX+t(<~B}-%ZnVw($&ca->qg6GOSBs&x0A{@%=z85)JGu3oTP7Dy1P+K!h8(1$ zcyk38RnW(t7Q5osj(7{7PZ}?5F+R!|O@fc*#NJ;}U+mk1q@(cr^<^G6X_4$~*hXHk>03`BJVIv1FF(7h zFhvI?5RK5!(*&I0#$QJh$M%n%qaI;7bXOBA!{85!?;M=|USN*TfXBKnl6zK=gcb1_ zIl)Fz3|~;EBZ+O(ue;Tk=5Sa~<}J(!D80Gm zWWxJt?U{^|8gZtH2Pn%>4jSkwHX980vrCL%P~gTRvZ$X(NcRRNE#irKZh&JU4(evg zmXZx7Io@8H+P_R{tsZW{Or7cQC|#0j(xWaKz7Y;QJO`!`k+wwxo}s?R1`6wK5rHMugfMo@n8P4)1!^6zOELZ6Z&%g>g{-0& z$c~ILI)nis2IkJmXTCv#@HoFB^*`OJ4GI&_rY1q6|J?co?Tu{!yCiT{vR^dpX-`4n1G;|ubH7fj8#tO(edLvf^Gcq z0EkDGov~&|Y#S@JB~~J2>+cpbuXIeSTGxq_qt3gZ|LnsxmUH!p53BqJOC#8gu5`$Q z64v=K9+Qp5cIf{4$rPsG|2Kr>=5zK&8$9r9IpV-4bz?X4)^{jA^pHTOMOC|^gPk29F>Sm zzahO4h*jDvR|&pciC&WH{;{j@^XON|l}aNv7=;By`mc#}Ar^@=@+W1Bysn59(_P1X zfe%kHLhxFp2rd;7Cnk;nF+xOrKzw^Ve1VK*fRwalhV|1Gt4W!BBk6QvhxI#{HW;Ll zWoc`G5G*7BX{`gDD8H8c#paFN&DWBT9lZ$Sy6%)TRU93uyM?Zo(Y<)`F~L#~Mr5@V zWqv5_^ZxQB`~5KB=`|8mEg0BWC+PDBwfSW2Rq$$KcyA^5bMt!z>Inak^J%?RYCe&A zMZU@y>zmKrBTx$Ms}QUiy?1<*>6>3YCy(6)>_1rH2i$2jJW)!6N&CHh5cTc%P)9VF zoR19GjjenR^t;v(KSy{YxjO6jYdsNobh-FUxuHO%l%CKD9}~hnEOchqTw?hd%=S39 z?&R8X#oE+bBG+0nwN?i&IWq}q$NzKZ1`TT~K&u{riEKSVm!TY&acCD!xU*9JngFNU z9mDkyiw2K`WgBXlfQmi8j-&a?D^WRUvWy>>I zbIjdM6hCB8rv$ZDaq&7>hy6sthz20CoHZINR)Y zhASlNwAGYe5ROD;dTrQ3bTnplV!87T4M)G5+AY z137pNV{q5S8eEyje_X#LMAs+$GYxvyByFXAiN*GPl0LCc^+qJ;6}TF9#P3zu=+r&7_wWwRJ|2 z9d4`|hhRA2mTV!-97%;>5Fl-yZRGOXOZIlRCU=;A*v>U0(tsS(?XD2m6;8-L%xs7u z5yPHfra8rAwwPh(w0@D;&0@1ALpSkwBF)U*Hu4$`HpC`w!6uh52oVs6loFQ>-C4n{ zT#b)X)xkQQIL{{ z01F`3=afBb-6Fx9o@#hLcox6i=01FGCVCJ^rR#>I7Q=5F9*})a*#}Na&Gr$O(7=nF1ifOBuMt)6 zvz!pUJPM(t@=FIaOY-1&8wk*=Wo6KwLag(8b1K`CBb1}jS30f^o7(jcJ# zIJM%jv;TLfxKcl+9GBRw&?8}1o_dZmAZXuXoV;4IHAeIYhxtW{T7#)M3btu?G7BH- zHq;wyG2ea3!UGG`Tw5c0`42tHRbOz=ph2RDAY?FW{Qy}}c~=AE=^Q9*NZB%+Z5m!p z2YuR-!h~d~86%H`Thi>+CFlDtGXB_?S!SoP_Ssb#JU>bb_wO0Q;FwpWDz|;=)@*r% z6fQHjh{qb2iR2t6EG#xud#j(-jU;XCS15L+RaI3}B-V`{4IUQ%jGf?wc2LBJYz32v zf`~lcW+VEX!ftA)5kR0Z7x}TuD*aS(?@{9!3D9MucEhfju!nWrCVg9f5wjeJb`4JT zDTiY|{$uunZA0dQa+SVLJ0UZ4Rl1yS1S(>uL^xL!^k#G6&Zok9&i4B{BAfrDiSYG# z5PH9M@U^Y@trJ5_$iAz$YCBI-T;jj|U9@ep8I-oJKb87B3o9UxJycicnYQNspR1RC zxBd)zp^8MfHZ7<6juXZ+eS79yEsZ)zgTY4~K z4KB1+_a5i9+A@}a0Y#J;oV2mE1Hj>l#Q z{5dri{vIg&xeg#RQT2HE?veS6f^j+LZ4;F5&IKh8a~gc%+@^hsMWER?o2rl%XF?a^PWSht+6E8>7dp&gBV`)zEP3j z)iP!f2@qQFATx&z_+E_PTErNM)d?n6Mo&vdW>t3ekpM}a*y5*$`5QR$K!q;niPosv z&#oeId?QPNfoR|mHsjYXFYD8*PDQ)yKwaHMP#~^zb)Ixrj6Sk5CKAS>cORcll~-?c zeX)m)va?!zm4xixwUs=wwlEvg2o)`cV4bYTp+ zg)(9fYYRGjobPuGh`p5b%HEk)F3ZT+nSE*E&j{Wb8Ewe9lxU4Mn7FpmYO-5@WE^%h zXIgKH7wp&7A@o6Ge9*c;Le93gHlF)uOk8_^UD=4X-9pSxOq<2;xlvS^E+?h(#2VHs zmCzwYp7%x!Kf}jP9tqbVA*MuDK$AGHCVh&3DhUGhOq2a$hkK-^<8#ZC2PTy<_nzWc zBIZzRASR3$LNWNjU;ScuPZm8vxhzWm-gy-8fLrbwvC#2@yxoPrJ+Y^)ZAjIgZf?fa zmQIdS_R{5WvTpJUH;BQxkN#=sp;DTZySYEFQXj+nsA3%MxJZ>DYjmGHeQ~n%=Uv>+ z?+6gHJFU6to%{AU&q7UisN-)3TGN|Cy&t5C#-#nKPw1V=O7m3N)1<8Hy>KHx1LuDK zJK$aJ&#I!o{3Ve?p4x=^9`@Un^V4-9sdk$GOk=HsmqN|{tN(}{TUX%#VkUf#PQq~v zUm$MnM~t%<(zy034$l$)cEn9$7pXD(KXh5px{~swKMdmAfmCpln3-84u|t9dj|cU4 zmUl!9OMC?Wm}r;Vn$h`{btxWvOw%D_h)~fnA+K7Z0Yr=C-}7(kp(HW09=0n{>DmHl z)2EPK0n=S#Jh_O1UY@5z_r|nyu>3a=M_w%1>k};O)VsE{14;FqQ(QJ->5Gc#*r8z{ z+J9J*lyrPGNr5qZftb&HyJ)|*&4Yf8laNCF@MYzFMw@;Q8$=R9vb=*FFA`R_X61qy zkzPE-JT9p$L~cY(eEFX2l2!bFxX-z1(tWk^V)+&1uVYu=?a~C-LC9VW?$R zsYp+dq%pDV$Qk!o1lM2S;n3BDl_hwgn+VqcVhQX^JMiSviGhQ|#%2q3q8=W;u7-`{^MS2IkX^mAr(>c_?cYO*4njNKw>4)ND0Iaulv zonZY-lT^QVRhL(txp2!;rX%8($`z6s`9_$Jwbrr${1dBVOEm;sHl_nXlb zPmy_yq%=HogINm2)L_;rggkB=8+ALiR%^C9h)J?f$-P z4S(X1a6Pk@4%k&T!%6-!JXD6~TMNs_gzP51ysfr$7?@q-Gafkm91MuLfPU!i-q2&I zd%(lRw}g&as@5|vXE&=aO_nB{#wZ)LGq6w7pLI~8*zBN`4jQCu%Y|b9ws6ft&BFqr z^>fH@e{lRV?amCVZAjjOLIlsWuWlK0^bxQ5!0M0Pgb@4EIKRkcCt}86@eOeD-R5CZ zOs~DQiq{@P>F(2U9=%$ogJd%==4bbG*-saKEyg>3BW99cz0|*;nQhtDLqKgcyU$4X zTyhl@@}yz)|AG&~>p7+{K8hcp-Gw=NCSN`uGZ6w|r2erofd-){4h55y^+nsU8rpij zkz>K@?R)P`J~WS_n^|HZ5y7kE_C-w|R&d)!dGvoL(=>D%tU2&+PbuWy;J*T{SgQ2tC^8>xaHa9vs>I zIwwRT+%sJVqfr03F+ur2ZvoTr27bioqYO1QjF#ZVRn_lD(08)9)pz?F4P2la8_#q^i%tRD_!B6a`GHYZe{K^WQ|D!SXN2{4ND}IUC)v%0|iCzJY z^IiYOWF@b)DbvAA!PzbU%x%ecg_}vVqu;(6VOSRw=6)elL?AxTelokT@8`5-{G&KU zg~Wrb!a}d3IQR#_!zombGsAe5Um95mqjklp98~1)ZtZ5)VdJ$r*juvBu2=;a|GsaU z6r?bjpx={7ex+fv79wJ=zdSIj_b}SY3Nx3i4MjX-)h|3-_!0UIlloB7?9hweMxN2l z!Wu??0dsDHA_C?;PTJDL;e80DF4Kgn}!O5tRjC17)_t_48!|IY4=Buh#3M-M^@%uHHX@HjCbWmd*<<- z7N|_3g%+szo`%?&ny6c{=K@F&AI1dUR4RlOjprfB1<`GUwv%olf$E}h+EGcmF!v)e z%5;j7iXi_(txx~kk@fX!wYZ4cWZ!&YuoKBr9*?M@r2Ec5DLx+aa{tnAG^NB*Oj?PEx zB6rbhTOk21dPLmPJ+ke2x$1zxLV~bp0z7i{Wl***EiS_WWdp)2Zq|`yAWRUL(Z0c`|szlHkL6rGZ;3Uu2{1nF@E?S9R?Dff!d5*jm@zzV+q%9 z)_bhJXDWryx0tgbNOINUq?=H0!``%LR{dBF5RX2X>Jc0ZZ?yvwGAEX3r3%R=77H`> zdLS{=>fh>zL`3yyOLBf9!e(Edr(7`?SsJiaaPLN7DrZEvFwcdKrH6ab8s_4iG330= z0H|qzb1E5?HtIsd$Z}jNgT0N3HBTm#)2SN{7$nx9h9^~ol@=|)8c!jYK2d@2`nd8L z*UFEltt3Tf4#j*IC5+=aIm+Fb@=uWl45-q!dn6~35(UKx7dCd;osUF#@^NYBS-a-g zYgbX4VAz0+B29dl8IR86D0vF79m|=TXVopVDSs0;sq7rLo`rDLP4*N75aadcu2X1i zor$#b`Jyt_o+-gc%OXE46tK?R3xGdp!c-vzE~|KDTE}}!0&EEjQ;$rF1kY0?K+81f z?(lUn*(DEtvrC4;wu?I$reDRKhjubNr6g@(-R>A=|MHDve~2JZp(B_fA?&y-aMp?| zTF&S86H4zZ5WRB0^HTw~f5WtyxkLK7Lpr)xp-P^bXn@2kZ3KEEH?jM0JLN#A-VuKX zsD%Pd4#jCPNRQ-$U;&<`FY-8VpmB|1vU8Znnhyl_ku_^Lq&ec3beE%Y-he;tR+A&m z{>lBn`2HPqoMc^@!eMbKpYi`z%eKlDr`Bi{&L z0m`V?!04H%2QHpz&lGN5eMa~XlMJs>kG%uXy9iA^Pqd`ABj07$ikgd%BC|=A+ zu#^kFljn!(lolVLM*do+{>7{zhWx9g42t>o%ncV9FTxkjOpb@|w)|ypI5ZCUiv60a z4(_8mRFZhzdCd>J;et9^1l@`w-R4ygiAoY;%~J@`jiZjW*> zmWs8yG5(U_Fiz*hl$0G|HOz$mS&WhGBeICRhZS^#$;Sk|46LXJNe>G_G?W1y$uR0c z^23H9>r2nAl$lYg0eKh;RcLSP%VO_*=pU5Bd5HucX%fCXhjYKyKqZ48TF@1n;ucwSz86suw!uiOPI*yPQ*%^u~+{27AFtzj6lI} zHsng=Z^OmT-j4D9!?5ve8@H5$L$sXQ4A zXzv@Epi%J0Jfc(ULbT&)g*U4p=%5VlZuQ7o2070}X%k6;*w4N&Cb71pl0ay9WAVP_ zB_SEs@m$ZO<}+q~N2E;!I{k^ddJ)`hBli;HAK|T`eu0~@UF;KEG7Q{l{raZ`l1+vP zP_gx>wTmj50U|_3VE2T>KKTA{pd^BNCWA*S80vp*gGVD6JS~LSRB~cm`yenq_CB&G z*N%BY$t9!<`a2x!PrA@1!T9NTLcM&s@OfFMfvQW6tnedt!*uX`AgQ?n|y6_qnHos8R5 zKIYCuhq>K-M=mwASa^a*A%r>;{Pk63v_k~Ft@!LGs*k0|(>6zwr=|a=6%n)Fk7!am zxq;G5Cyd@qEsSK~2rgHm9v6tU2BnXifbJ~bKZ6HIj}|5A?+$>50JULNAI6C_dP19H z+nX2+v<47EKv&bPf zteyEWr?KSV5sSAWmlbFUtho?C!&hM)JBssUhCK(KX`ZO_bWFLWwY2Y;yxh;)WUS$_ zmAO3gV63{D(X&+1fTkZ1a6_Z7ymLCBN&V6vFkVi!Q=*>tilh5Pc+hv-h^F|95rqLs zAWuG3ODJJi~2B;t_M_;N$MK*qDeWrWiVhP>hfWk=OmAZWwy*FF0!tut}biK;5q zgZ58pTt|6GnX3~ur33AQdvWK5QTj$6C67wB8%JOLHgjnm2++ht7}0+~Gzj{2aWw0R z%T)onk#xUqQ%o;xv5gFwYg{2MZR+BFcj$>OZe0j1v)xj^hGX!sUJN7y`F4e z4MGdrmwCT4fDqCV0W8*Eqq|H#36jC3@!^a7&fY)ak4`e4?dH13G8nFi?mvr|Zy>?+ zA|b6$|Ggq47?5#=BGxaYxO-&I8bRRZ%sxC3NG5T4P|OoSwglTH`c9~20qGG_xEfCy%#!APIB z`_b|cexlzbY5tKi@J0uIFiyoxv-3nYggwQKv?}5)nwDBUsE?`=MSZ%DHR>y&rZF+%s%ftyZ1CI&QU0jn17Fn4dmepO z7JC$nat=%`W3u1&2-*u`&s&cyYcdV7NyLqnl`dw{f}Mudxxz}pATf_@4+;dTiAMeo zc=5OL+|^MI*QHE%u?%j`S-Ky@YRB+4w~xCjA1_7=xmqCc!{UsFq)qgkvB7f0%u1o6 zfWpe5uz`7ztj676>{(@5<}y556tXY3E5(TiI#@p$P~Iu=H<3s*s2HwfCD~4YDcG2I zg`U>?>8Q!+L{`-Zf_9etHiaxQO)w|!OK9&EpOE!c{V8honYvPvRH;u(DfxU?$l*=6 z7Yf+>7&;h`?JTitueM1@@&(!Qv zn|GN-57JQjNk8KWy#oK|tfZys4Ru*CVDDe~8{avG*{^hdqruUCSkwb<^G@(iJK=kO z$N!Z_Y)FSf%r4-0=K^}AYq)@5PHLRne+X<22n|m60a>L4+|!fHo2eBT$~ivBv6b0_ zlCrW9a|zH14xwJQe=PMF#4{{DAv=dtYMNi**dkIDIpPq(U7(mYpaRV{$XKL?g@a|HpOs}A1HPrk!J%Xk zXjcPW)6&XqTTp^U9Vc4e5@h{%2D;f+&rYM6xw+*^$$Qp^B|{$1B;xJJ@9(bb9N*t; z*SSx_(%7EQEm;6QxD68?{U1<(>d*#+J{Q4Deb^ph1r(q;j0fc6-hDE)fZddbbpS)0 zJ0A492rrFcJAh(@m%%V0^Dk56Azj?b`nUu7u=tKh52_)n?Rt|ILXYP516C8kA#9{P zkD&=pT=86Z`k@viQqLl|97im1*CME_9omrym!uvnxlJhw1B@ofbhFg}R8T_tV` zPZwWk63ra$lZPdNS8D<;CM-gG6dA?|yTB9r9Z0x?q$hZrA6t-95PpD(wB|#0;PUxJ z8bE4-ihy>TKOYhia&v$a#*f#_WWw_#y*@S{PK=S%t~VGAE(r1W%{lOZhrQK+M`gPr z`-^f&h=Ia(ACFHmYJlM%1o}rMFgTQV$PFKFniWC;E({rSt_mq?R~tM6M}^ST8w}3f zLc{m^20@{ax{}}AS*!?8_bfhd1KqqS|g7g|huQdJ0xwa4V zR$4{x$Bt2H+J*rG&ajD6s0l|6#^Y@gsG$*p<>5A!0uGgIrNHHMl7h&>=182_6(fny zr0)Z@E_G1ZF`~y`@T<4Py~ER2dtoYv|#BA4?&dEoaWu_wx9u{_&5KArV!6^_F#AXRFhR71}X3BY&$;?TG?tt zMb)hJ`TRZ2mYKPO2SGS*V$Jd7bY?h5qh55AX^SkS_&Z($U7bEpa}w@_x3t<&82JT9 zHVVEqB#uTd$c(``aEkpiq}x#b!~zPd8ZNA*6kN*9 z)Trx#hz^pGBt-~I-j{8Ord4a)hVhIhi3hiYkukK@IQGWaXK*x9*o2*P?L5I0?begr zSKeutk-$5yK<=hrBjedztPJfMGggLfVzcb<+LDoh^3&&+**hF}pBy=Y6S|K^h|9!lLN_^i4Fhm+7pTRR==7*?Mi@fM{o&Cc5)%(wv1|Alk0bC1p)YM>3 z*c@0v_)sy04SX5L)ZaT{t*prw1DtHs$keW~D#PQVe-OneFJ{^jq+8Pf>h#2}Zi}K` ziHb|_i|MyGOD4v3k1bE8RdU(blPHXD-76QU9T1_V5hT^v^50? zZ>Xfi>m8E#(l{TP8J7l{4L2@d^4`WG0j)9m-%wO;oyw!-`OhoH6$! zM)^kI#9MvK(~u~G0xb?kIw!9%Ti}FWIj=-W!-@>!Tmh-QQq`dVJkQx-=Ei{rqcEtikoHUTI?9+Ek zmkLKYq@H0`V3)h!lTagY<^_>A_!vwewd7VM7OpD8B#w>eyNOU>8e#d`cJ_v}OrL(tOqo}Y~wGI!^ zf-?fb{~m9S_Lw3vHKjPGmZq47J6!Jzg{aQ!%!sR&>#!Ajp)67s*EP4B- zI(L$fWUg>_+utn}_ycc`D7k?!Bh^U5U@2bU&r3Jdxkt$q9JBZ_a~0UB;E<)%L$z|V zYE@GOK354supQH}^3BH1)pO(gvco-&J=~t(izjZPm3^!lDtQ^y4eU@xuQG4gB$#R2 zgyw3IEl|z>#qcAl*bA#I8-$1L(wj|#OW7csFAKmGPLdKnkmF0$nn8x>Ymc_eJZOFFf-& zV&lP-Sozwtl)C_RII48owudN)c+&~%XuQWs(m`X?Qs5Qiu3tW{;#JS!t3k*nc|4RK$3w}(B4y1J@ir` ze*-iU>B%fQ7$b18Bz~9DlA)UxMszlZdyjW+*|#a&4oJMgzeADw9*ANhMp6sO9>%lJ zZZNm0B7+a7JS|Ur_p&oKCSG43ANd3Pr^k62j=hnQU70nS{bwWQP;&MXTiin->q*O%-R(_tck>1hyN2PaQcF{>7Gt{bh} zLi&n=;EcWMGll`GfiGhLO#@5EKL)$x;oAe>F_)^U+J`SYRlivtcq~q(=&#l_Qf^Ah z@83YP1@c1@f4dIlFd!+mf6da)O!D?fHy!o}W}MWvD_K)5Pdopd`E6Ls>Z4-B7J5?y&feJaqv#Bnfxir+qB4N6@zIS7L}DJLlQBlDxG8@+plCKja8&ergTOt z5}_ezyb?`rLeK`igT{gCFVXioG=~!Z2Mm;XEgHyT=*s2m+}(7eO|d1bh!HMnJZf}W z`*blll{}hE8c@J&duQmkmaYJTma>3|O}&7^(O2l@%7#hE%D;dd(FA9Be}Mw1RP+J0 zD!Q}g-Q`F{=;eA@gG zDh4P;&A@KS7|Z{Yb-%do=+L%06TdOTw^ht+NUz~E8X|TDgDL97sbI7f!LAK)NczR5rajDjgJII!^73ur!!0z0dduK6^+WGMYCxAdU~F zLb*ER2}itDd56`f^dQ}FNxR0$nSmDfCT4l<>z+rJ#R8~B`SG>X#4%Gdn$W{(s$L+E z(4lU@oz@ADXRv&cmh@f7W5nZXpFrGv!_GOZLe{BDazS>k6>`fpZI~e}vhwPWZX3wp zomrPJSKH&{db{O>~AmT^pqbS4f08JbWS2`Y_WUD)O~y){XK zG;yB(U^8z5h@LYI`recCub_6qs$Mg(W4H<=T6hy|?W++8|EP9GF=tsfe5y?@UMMA)^&`DwE61tzx# z%h^qb!q=Q~WfG(m&6YgK>H z-I~h@mX4^(EL4*kkj0vF$2vSah0-=c6fDm*f{b*q&KzI8i5nZv(c?=Wt2|a)wqAE&adXCBE@c~>Jp@7BoL!jaek$1Lj|Bi zKiC&-pwf@e?Xp-V&mb&DMQ}}1Y0}yHu&)?Ph83&6gd1K5f)hk?3EK4WOC`0qasCw@ zHD^02ZanqGno%I9&FNyuf;3BsWp>jR;Pb=y3MbDFwVOC-ZPgGip;_=cBp$sb+w)TwRP86KRF)QX5p?E1f z^c)LIa9ziNg*`3Lt&QFfZP7Z$Jjq9 zcMqCkLC60T=;(`|onzKhkKv_=lty`ZkYI#C|!459|ac8kE){I0K zQ8EHb#ASF@7)EdI?iu}c;*o8X_?E*Z?=Z%gRaJ*TQ)kz70@sI>T#F!UNzzabxS7&E z_9J8`sY$5%g0%28?v|GR%f-ihV{tZooJ8=#Q zP~rrkXsJnq;ICJTjV#FNF9e|vsHO7Y@vTt!jm0|L2gxQOiFjLHG7?2`WuYl=*q;5| zUcC*-g2`Z+&%9HtuB>dVpiVWcpAn{t8BWwQXTTZCD!d*o-^(bR=X9#k_=ps!KK&uO z0QK)`mwgy_?ZXus*#A*|u=vcjfcRY;yB9BigERa(lG94$LxbpKMVOe$PIHPKPn(v- zz+g9c3FUT+)_(M({2nd3LDaZCnf45(@|VR?9oR;xd}N3f;jh5GOcZ!BL8^`ks9|Z? zdkq%T2F+>Fkl);wCgSl%E!XWfH5-3`ZX2yzXmy8eSZ>JId*(kGIl!5fANE@#eBCp= zzC^8(iCf`gwhr}L374`rID-uDqn5PemqUdkRNjaOt3Gk#mWXu$CM-9^+?6CF@%_Zh z5lre>(?%-pm=(Q6)L>Q`A1@m3sFF0N$Q=noUn6%qLqgW%F8Pt@XK_9g9loIASVz=Zv*^X1hm5?yu8??4fJfQ4~ zm3Wz6Az`E2bZmb(&NLbiK6M4OaF{$DungWxCEaHqhct!GB&YY{qs(mL0dGO*S{ti; z4Yq^UEt=9hkkJc8w9{vxI$EMm!a`>~*_UbDvr%{dxl$+J?N6W_*0X2z$7W{NreQS& ze!Fp<2tfCD^yv7(P!n%@axkQ@Vi{^X)s(>w&NH2{^XP z^4 zvk1+6#GLNPNm_xyt}r#o%>r9e>Zs_I!f-ARn~*-ZCP7rX_MQklX9UPS^0;E{Q0*K- zI%KwM9n((8^MKoVB$Uhu({?zy|I^Oa9a=as%GdN`Y(hdkCswZS;m_(nxfRg9$nK0K zVvzhPmp&V+8O*y#wfLg#W^>cFJ%4T@QiLTY;y`Wth}TQF(vz(i+hNfDfc?pskMq&3 zGg_3b0>cAN_W%dnOnW#2JH5=7O^?K|l9DeTsskUgP5*SCAlO}KK1;vHV`we2^=16due z4_ddkIx-zi@l8n{*F&%j1$z1mb`w|dVR*`5*yQ)tlvjbVS22Pr!;L6(+%@OeG*-f} z@LGCj>3?s(fPo@YzI8rLC*S_ukvO%fE=4WmFE*Fkm3cPpvXct!PKj z05HaPNVoy9s#O67CGF@lrBXMU?Mcg?#0E6oHyDqwYFVwX;Q+so9)B6cb(^Uve9 zYf>@c-XchRYY1?K%iH;6*uE~LwIwF{#I%pM3o^&kc>gP}w-Fxtyk5?p z^QfC$n=}9O60gf^=fC{o%(J-hL}0#slfwAlN$CG(y?tpxFhGa^uwOtOITV54Av+cp zZdw5HBO!T(xbpV|ntcGptsqng1i=rMvL7ZLSsM!^z1srWp@8v+0lLmlrBOxD;dUxcG!x^1bFDF62Lwt$=_a2Lsan8zRa{ zV{~wKI=CsYen4+2y%Dfqnj*|VvyYn=mZ^YWM3BH&yAQ0Iyfruy=Fb6gpxrS^AYZn+ z8G0lc>tkbau_|*zq3w#z0JTX+#tg>%EsB@c3s3b*L&*RhYlMjHLY_m-4ZsJTZ3vHE zZXYW}N3p5A7}RFTtW|EiroGhy6AhyVyE$i5pxI*WlC0LIu?sBT*5q+C_u{!T#!O(=I>Gil1A4ig5~w!1b97FrOo$|4OVHE9t z0H{(l3T1Sy%z>z(0<>1LZE7lopSN$v=PJGQd+)aB`#>)mIM`3`K;lN*zjq2^5{prO zvA&CjTNo}F(E{}6eH!&E=1tT8htt>6p* zMEOEG{3+j}hTG_%4iOlV-XPin_sB2s1@DpI74Xs!M1j260)bmZoc^U}IdPZTMm(w$ZQIaYIMln~nr&?e_lH$ugf)Wd6Gg9xNlI21fi96gJ-vR$A znmSP`0R^sKRg>X=3q$_js_B1pAsC>yLqtSiw360;0`((A*LA}-PbW8kFeSs5u08-C zM2JX;nE-=12%+apA=FWLxqQ_epA*pI|5K)rIKkb6@m36D#v0agFK_K3GmT@%`y^w* z=}WY9huX_SJ<&>YP#r~q@imS?JSpjpZy?*BACwtXghbd%J2^!Uh7Ls(W@RAW`o+hP zN3UNUNB?cg9u)@_L|?dzAR2_BmTM%Pr;P!*fUaVX*YbOksA88O0gjiZX?V7z=4IjN-)Rjh>5Ov@y@McHh z9uh9RVnWW7nl74ByCb^}0e33b#qUmXntqAG`K>cX8Fs~%vRF4-J_B6mL7~8t>Rj*6 zm;U?=x*qdtyH^Y^P9i6t>>g!~38Go$lcK%HWX6rRPRdLcNybbr*PD=CDE*QgJ_=oc zuh^Ug9+gR4A5CIXA;6=b8XrhYTAXY>kO+R7h8O0|-|Lq75oLbNMRFtx5{#$(vN;hO9{oPr{= zA3#{hXfr>JAUMGE6z=YkLb<8Jf16aQW_*-+_USf&;?$sJRMwM3_8srPT1cXUSt#CL z;R^cy?bH3S!T5n*?n-7DUl6w|iNpmoLoN2At?|}mW)c&qzyWkC0-9i5b*+f+b{3AB zU>3--3AQ378Pgli?tm-=g_2}{7?>nD6BL}u@cVzC*)Q%M&b;S%=Lbg$Z$4xev^Jcu ziM*XB@7vd2JJ(NLEp$7upC11h{Z!xjLvrEU;fYwI)^C6USiHsqMqe$Z)C30wQOV&2 zM%uCG=BVq@R_X%`jG1wF*5ARy{usX6e&orD=Gpf5ZEiJKo&czkC&ZfW>N6#B2A8K0IHzxf3 z=u8519>@v9%(O?7b`IU|AMFR*^P@GcAdkc>zy(g?vFCaT9-w6NFV6CW;KqiGxX7(M z3)?Z5+Ogikr#a^YCCfAjpbhCLSrfopQxoQtNVqLn%T2nk1G3dwpYFKX8UZ>i zh$D@m@BTu2jXbdzKK5snd=$f@^@@=vaj6&>ij62C z^FAVRvl&^q$%s|m7Aka5qCi)UA=8*?#yRo{% zUa@P|83r|PkjqPj*IV6`NGRl(MAd8h$LEO+rLB{xKtz$xz1F{T$hZ?s=ye(`;!NOH zOF6fj-^%oaaWz!1>`ABdkPS5tRV%@>)W5u!gayWZb@-aCCMC+D`CKens4 zWn(7(AczQtdl9R*+(sdnw+sF1V&ed^R$>MEVxlDUBNG&Q=#oUj$~FigehoH6)06h` zBFSKW?;;|0t$hXmUah^ozMhU%i3%RchfXXy-)yhmPuM3MkmYHp#mtC|DRYNw3^PQs z%5L;SyfOwl7M7IOn<&rQozvkon!3g*oCqnJ_$^P|8^qh^(kE?iG0Rx)^cM^`Z@p~u zL>A_~cY3*HMUk;p%8`ebgO1b;08E;AI(U|6D(Mb{!QiVL-eaPtb*VRGC*`IPsm&? zw*r<(_8TLdu}$#y(fesUuhk7mosw&nET4@V9m?2aaJrv1rh_lJLe8W8wx6SGMQWB| z@Jt5P(ypG72kW?x^YQ$>_>Ma5%=_e0PZOCE_99x0hCCvm&Z6itj@N)O#Xjk|>em8r zIShmCYFX`dqL;M)ENRK+gbG4a6e~LS9nt&51IkyqelgJ7a0q5 z(w|Y(4-1L8r^B%J8B?p_HVZp*W>6;92SOElFY;1gaW6}~Pe=0>w{pbmC7mnS&FoQIs1eQA zO6No#2p#D?^@oDFq_!GjMvx=1Nn@Dq=KFcKIF-d#^kUPUmR_zGbaHnJMWwhw7o?X{ z>E-bn?f`OjE2f?xqmc>8kzsLyDrT{p>-6Rex(jdO(wfX9n2Qq7Kc?mc6D|Czq_%S; z)1gG5)oIx>9K#!5_0H#X5anUYfskF`LT^BQ_#e*x9w`zm7DcCSzcjNP<1$}?+QNk# za%BJYfbf3G>_&oMq;R7HqZKh>#at!r()qG+l{A&g#3@Zvk}%bi1?z$@43@@`>r(cd zHZ^;>CBjm1N<|DXQk}))Kuxegz?`*|k;G7)EX3v5{%*Ly%nbXUSZ`NuZm%R2qU07S z)*L{$0CyRvO{8i;>$V)&r8YO8iDkK5Q7T`>J)%PrWzyhPy<4pG2L{_4TaDr6;s)Dx zM5S{0es$37PLMmM#q1I1xH?~&GnKN5mu)JXei?elT(t=(-E`axsxfdN%}u8+R_5qx zpsXmOuqIE-(x)4U)ML!ln8_sHw0N*MjBHjPa9W0wNK4D0If(NtRT!pYnDrr zb%UfxG_v}Ok0jn135sBr)26;C5*TNHb}(`@IqCv9937;Bg?g;k>Fs#UbVY#I~K)%^q`(=Al}#l z{&9W(4D|by==~G~d_)Tzzr`sB7>t4G)H_b0SQA<7{*hkLJC`}YCYA`yL zWq5dyMxOX$0?c9+U$n1P1ds~ELh~?Cz5y*8^-dz5zI2XxztuCh8uS zs4InP88J@_RtopIX>0t8Gzcv6mmm*0;9Jjs>VO)_d3)loOYb<;|6R9&u8F|pfORYv zKN@8=dRLixjpbA=xQq+5O2RDet0q!NNn-nv@OHJ7g#|Do2*)WS+M%t~Na+M3!tfDM z4py;XlKb@N?|T(QO>UvoK88PB%>rN)5ct6RW7Vlluo&z+*VA< ztWs7+F_an;*URdE1(x&sxw03OWxx9_`}@G!AD1kX(a|uHzooZGwl(OupF=ij7ny9_ zSd(N@CF~1ZtLMB=J_tDtc&<%M2M6;3!*lMBPVGhTkL6+j&jRZLg5mvoF6}+ZN)1PZ zLI;!ZhlS)l8JuHR!>6#%MrpVq0XREDYaa?y@DkrZ`^$C6$K>{kfaUyp4uC~nr9?st zgVLtHq4@6fmk~f2kJQ@xh5X<`#anPdbcKEs-yH#Kf!d_C4~+@E^}&hV6~*Zp&f@e9 zyoP?`A1SE6LC2j1-m?1^9Xvf3#gN^&-~i)-$sD?3N^V$iW(-SW`1Y|l|8u1>vzNj7 z5ma;Pj=O`5zIdyqU3Oqse}i1TbnB+!gYoM82F|y5EBA2Wj@seUNm{T1x!tSMUhl+K zWKh%VSB?E!X{c3CP01m!x_FMJ-_3;u=jm)bI>Xt8*+&W3-~g&FcL=WLelD**))!Y} zY2=?Wfm`H$6U-ZR8fT0ZG_cWW=wDoKPqFdu_M^iNbADZRv1MvUCNoJ~Av4SqXB%5s zIQ&nNRuvU8V2mh2q+O2P&`23aFP_O7vYJV6=r7DX92~&_TjjqkWPSl(SCHLkUbz7{ zg?F}L@doB~<8)V>Pu^hoh!XE<(X3*ILA1jwoa2SIk$t6H@OrahZbO1y%)Kj?imK5_ zwR(v7lj=aYSpgqM#?R zj&(Z1Gf>MdGn|Td>Y73oofuiMLl~0Vs1mfAQ!x~iBePto#{~G0VJijGdILNT8Zs<) zkR!H!$m+;v z?di%e;mjg`I#Km~z7Pnq8bRS@HjGAT&}^Sd+~D?K;GjzyO9o5UyO*Wl>sL4BVICCL z(&%H0B1H_!vAL2@o=XlJdaoQeC;w3Y@;zOK6thK>JMO-$DuKml%4uJe{jzPNZTvp zeh|ySVNudI(Zr{Y0+BgSkGxW8Bu`7^e;+1lUVNU=AAOTL2qb* zri4BFjG(%8jigeOIwF)dETz80*hQ}3Utxv#POe0j0t2wTl@5m>)9;M@ zhrXEqn^|}P*vm}ROIFP5q#eT_P|e%x*c)%i&*vB+mlS20C z(_Ob|d|6J^AwZ{}b@C|{T)v!LZ8M6zpI|8dM{ ziUhf$-D#;_+sMBvEHePZNGYet?dQ3BV-+(8=XhGTkhMqBqIb^!31BPLOkO?pEA6xb zSpNRW?@HC$sRcMOMwki_RTuKeM9olAYDZJ3gR{vVtZORoif9{A& zW(+Qj{p0>Yvn;w!fS|nS^|Q9Mzy+PPsl(tKm8)4>ZOp!z{KE{ebsl+3uw8hh2aopO zmCc^5Q)sP(0o1jsxea>%84MRgVj!Bxj9Ie>R+$IX6y)kE@CL)qG!|nK!P*bptS``Y z(TIqs8(M0%k4F!Etzr0XGHv~WI@tPYn)GvG6|}DgSQwV{U4iOxgTjItL})KVD`u_) zSnX8IE~nP3PFVsQE2}PV0PuN2Au7ID1x}qCqBmAJzOP8E^X-dm-hR{Hmvp9YSX-=-h56Wg|~9l4maUR_#jgU^vMid>&StvY{@7yJk^HT)`a zQ6G&uHaG+e=|1Kaq#5a)KvZbP)@BJbvoAVgqbgMrg=UQB_BBCT;=Qo_D- zqJK!qBz-hjhVCCHTD9M{*ii{w2%F1H zR;yDA#Hs}8W)3j4K}1Xy4Y7YblPc0$oeoQ3CkGp(43y2(7NaY(GvxEiYj^f{hPhLm zwUfpVn+<@IYcmC=v^5xmWjHq2)NBoYKR0#5qP;LpEWTA7c5oN;gh)VHHXV>{UZ6=E zNu`Yi)C}RUY!&yr8@{vNZ1GZZ=^W(*;qLP>IzlUCB|xqjjkOmT4A>@lg^_0mL z>NL``mePVN+azYkNJuUNs`fm36zO|&7064VC9RUcfUeZXH!5k`yot`L{Wdli7+;4< z&Bmq)kzQwR8zguf(;sb=@DbNS-V)N)Mpbka#BGX3xR zTnma#ejPNz9h!skbi*8$(9M*59ZM9aCM)gARk;&P>c<@QKv&hrioc!IZ1jV22V5ME zwyvtWUuH^(X^1I5|65wRwK_Y;3;XRGI?n%>W&anV^uGu~c!pp9yqEkXD+Gl(vviLb zK6cK&K17K*vwV*kzQl@MXaJaoiCbY195Q0z-WigNIX!RR9OA^BS+NI=IlWLTIq-s+ zUAhMb53p<%AFzZ6ShWfd09d>w271h&>qBbHpUXpPES}p#?9882LhP)bV?wSipVi=Z z7H+8Ed%iMz%#T8e#lpqH{4r{oWAl0t>8qt+>+|f;Ej~f$MSY)1JEwKtVqaCk?5BV4sFm+2;UEo{XRX+xCSW7zRk-K#bPJ`rVNG zngD9U6ibIR*pyxT5CvNHa0@b~Ov64+K<04K!k{H!(XkBel>z%Kz{D^SHYJje{w;jZ z2kdfS7L+##`Q72GBX?p-HG~0Y)}E`&eHHLS)iHUsUq40H&^>w&7VPpUptd!7PvHyv zg-toj%h%z)?XdBCmXoN_+#MimzT9i5F&=2ZM-^kLp^X0M@Qc|cEd1m0tYmECL36R z`&!}P4by-w*rdKZhBllBoDfC`LzF)HhLAA{f3g3qeHZCFbwT zWiV!F#c&SkgDawCSKE~JOQTgEp>5aAoU#WD_`q#oe2m^e4A_R0pl6%7XYbL$)i8DT z-2hn!I^cjNB81AOS+#?LW(KSMr|@#)FoY`^Ryy+go43_}XgJ84(_#9DU=9dHI@ z;OrRQ#QluWhncXiLWcIaL+&xE3HZn2ru6BfOv&8_%J%snRyW{zCk@c2_)(?W3@OwW z!?d7lbFkj17iR?D_a%QMbP9U+-;`mU4+5ENp&}~KaZOjpZd$Pf_h~}_7`#T$A>T&Y z_ps;s?GXpK;I0|oqW84GxCh}+4*1!$@on*Y12^pn@?-4wF+&>AeTVEhgkT4%;93~C z2XE*HWJAQ9-@ft9-wM5bB@_oIoF9QwdIuhyAFyl}&vw>E?(!pcH!GaJp=L~#X290C zJq?_m5x%o$aJLVl0U(C2>1t%p3W>Y~%x2XD95N--t+7sdVftdN28edEK#GmGS_&s; z#e{XcW`ZAIpYoaTu1A5kl0R5EMtPVnLpv%Dhv8iJ$oe3stJ0pQviY6m(l** z$7sh6Qag8~k9^uCY3D|#TRI|e2L{t34skhiKmZFS@>*3vp?P|s0s8z2LhIQh`%Vqe z5OoncondRt9-2OEotR!uotXXw7Y@t|7Y}yC3#j7gKBNIros@&D zcI?QWg(LP(QIMDeNVk(8bdYv!Ub%DxfqO0-B0>Ft4tL?mxFTz?t_;Tr+$@-|a8IW# zVVc~HNUOZB(=P9=aQ48y@a9IWMV?taqOMMjmOFJspj|l9#@&FH&M@n{YOgeOIdKw$ zy|LAyHC7*tU22s9MQWUoOnM_**&{!>GV*2j3$Z6JvBT%g6(OpfIZ{NHA8GQjo;iLQ zxVB+yxwHk4;rG@il++;r+0N>-@dDupLVgri?JJ^RFPQ{MDZrNI$4kk;RbVXBGc#41 zum|o@Kb+lhR?eBg>Q+olxw8x)gp@dYFm0Di?Cessdt>U&9x-um%`m<}&#WBfjCCD3 zwW@&V>OZ@}I-FmSn=kH6It4&Adrs}%ICu+3SndV1JA)-IFKF3czr6)xyYK5OTHVTt z9>+5t+gsib0TAAeJl`PS`JJ768ZDpUactU=(wj0UXYYrOZw%R)BcgBd1l^6N&IiuS z(h;9~ymvTo`H0WrY`kVQQ*U8D`Ex%4*f48%_EUQ%HLRL@J#2Q-pi^zX6zXCpT&CtPFW}e);ZnKH9*E!2^PSfb; zlrHRUepp~8MMR6iVf>AtBRJ!_69E3@P!y<``HqBD(NGyl7pavssLOCzz3J*=L(6K;P>Al zo(oD__T>xi9TEA!cc-I%#2*?EQKhGs?Z;K4vg844bF4~20u(+m(R_^Aq*;y+xSPF* zF1V9(%w7Vzk9_|$wq}@qkOJvB9XX8xUT$nttDrG8E@Bshz~PzG;}dP@N<>i>Y?-Gt zeoQUw3nTB!X!TMj&gVI}9W-G5w7e2UZF;GBPcwx5W&8htIQjK5U#l*Vh?U z(?SLQ6~Q7JGz+Teul*lTS+0U%EC^jYaxZs`r*UBkYBG${( z-UI)%=1#1jk=dR7=&Y&Y%ACWJ-J1^{fMm4TEbPV_SsPpdMct%Wu^_)9p8QM9zCLGd zysjYXPd5QW?KBZUy{ExjVSkrZsH2$yz-j}8YpfKTrUwpZFqrv=M?(HagV#%-98O~L z!Sg+=Bp=B+PBWgc=wOzHTeG&howkT#WfrB0PRfT-Ed$Ou=zwo04Vz3@7aycj^o$xp z2e}J>+)7hP%u?4aE?!sQ?YJ0E`lu_c=w~o~r2Gl3OAH8gow1z;Z##2uc88%&4|f1k zoLZc(8Sqc@D@|EiXhz+BM|GRocDdkSBypkyl*7^FEc8~n_o)_cRa*HKU72d=dk|fz zr@)Mq&`dlIB7j{k0iF)mc;Zr2A-Ic2z-*F{mno%53Y>Bg z=W(lnzVUdE5_8vvk15a;l#1+9El$S2UHDdQFVM(THBZ`o?CPh>#p72WDbJ;iP{%vu z7glN82&(e`Bq}mCf3Ybu#R`u%#YD%eY^7RM#*-S!ejNgiyOYl-*Gc}$wBZ1!IPD!M zj8==mQ=TSl&}VEoZ3*i>QIh9Z$h+cfuY7-&x|fFu$aWtCDQbU$7gy6g^PRR|5Rj@s zaeC&Y7p_8aaXvHf*le?i(w+H-W!t(OFXt3L;966Se8vW8zPK&X#Vx--rW{Wa<1B@` zaSua5;rfN?{hk7R#ByF@az6sQ;ER)_E%+NpsyyAD#eIzSWwvZQJq|TqtvBQOqOFZ` z9$lu_Z==Kh)tps-#wgvv@)TuE;dkjOZsqjX)v zSz-d5B)VA$Xv)eOylwT`uOG*}P#ac?XMB4tHa>Vf)USM?^mf4aZmE`>YYrCLkxXbJo~lIVF|Z4Fs1 zH~RhyQmz+|OWv$xi3qS}Mae2Q1B+O9+p7!g*~OQ=x@82@ITN^gc(U?(OJ=*h$I}C9 z8sp|ff)kc7JDh<;$k>&5W3uyHdKdFVd|*!E#u?f zX=$9ENt!n6^8s#MxOlE<&lj|mi($`BOypT~t8i0yMJg~}`FD9!pXTmDHDh8lM_h7d zP+yjFI#ABMMxjuBnP;L`-`@X9d60Z^l*pv!|7U3>LAiI>&iQ>!S)5$RNpfa60F~aa zr?8gyCM<5@^*~Qu9UHYBaa}KP_8*~)r7BRBI-@;c9~%glVD>BE_qYCN z_~R#nb_v64xElZt2Fuq0Gy4Ib(UX_E^DlbVt|widM+B*-VNW=O`tQpybrl6be~~x` zEs>WCLsZnol~S(jNLd;ilZ>=VTTf^8y27Tp`@Sxu^q1ZuVE0<-&I)tDz;Wuu-}AGQ zUq{E!90S<&Q55}~?q7OOjA1ywC|cQHx_g%8c$+)RvWT8rds1!wn)O*YV58|!W8sj5 z4`*J;Kj#Qp{Mg#Tf3o*ff5woLR!b_DdF(xT(L4)VR61s5Rlz3Fpwu^5Pn$ z#?Fqqrd-n-akk)G(ywGhRn>Rq38Kq*wM9)quIZq;aK?f3)Pwg%!B^51k}ZI9!op+% z*By|wT94e+S?Q}L=7Wc2ogZ&8W!W&6cJ5cbKzF-}vB9iCjJT1m!%mJu2@kRO0qs5-g}-6;oUju6WYCF)Osn#vyrO zN+I%FlU~?Jhw-JQY>iSmBt=Wc?1kvp{3P(s<|KU{dGS_vV!q;D2XyqU!RK!?mAdR+ zy%cI?qAA~Mb0&)JAJ`v+&ERIgOseUuX$LOd)4FEQ?P#MTmyNh-H9e={L(QM6V%4dv zvUk+XI_&B0>x3lH>d>aSBk(?;(iRhJR*7Y^IBQTUXjSS>W%TA<0t)yyw-V3I(u#rN zgzDRgS@?^Nk>7E$J6cQQwYL^0GYo@^68-vcE=DQgePO6R#4`(k$YLvH6H?_(W~FY( z6k{)&q-zck+2#{3cUK7{8;mxew2j6G^@Wstcv+wY9);WQn-uD$V#r({^BO-Td&x8$ANGJITS6W~p5a(Ee z@#22j2YxK?EWYy^ zt3SQVRBfj1_bzAp#XmqC2%JS8DK9eD;hl;n5Etn(w+s7P7HB-|SS6TY(ltQ4FxHI4 zR{s2HCgCv|Mz#gp+f!= zq0xH#(rl$jzxOG!Aw$Wv(?Wq60lln|&5o|5!C7FFuh&k~ zQA1gkq0BQY%QyICY}^EPcU9mlTKlFN8%JknUACbF@FNzrYVu&Gq|;#9eZi!Cv%GbX3YknYc}jo>A^}DqT0F>w zehKB&EHZfVU$dOG8Wh?(KPfP4AgBmCE`0Wqp=WBIR(?#z;~D~uvs#627TH8AuMa@~ zrc$Lsm7SEzVh3ji2g)w~J(FEzGfqQZ5DB~de6OG~e|m!S(4Y`t^246YzS8TOQFaW& z*qV~-YO@-D7OZ%0(oF;+-&Pj)9-TNj39KL%jRiiv$2a;Z@N@xlQPa0QRC-tcVBlEI zCB0yo!@3zJX6eVCcAK2sU+`i&1AhvK9)Lf$%lSR^?&w34^VMQ~*uri9)w%7iireoh z@SEfLXjkn`1@}~-(05-4Se)H*zPc;4U-LpHJ?tLK_2yY@5J|8TN7DH!``jQH0tt0|Qs z37_Qw0Vi`oBEaUma0V8g@5#Ah1}6G=lPsj>Rp_uqnp>G6TGn%4lF73~74|$w&tz<> zpH#pSgt@OJBOnAcG9I%2JTJ=&8T=rU>8`Bp;axmzq1~%&;u-*yTEQjOb{m`!AD5k> zN2_>=JIgjo#W0ZHp9ve(tdGFkk8m{@6kFqRjiD_z+70qLIj?0liNA3csu3^wh}jn2 z`~0|3MtB6^(540h#tMH2?@o9THuhNYt3jxarUih~ku zWee}aS>|sCG0&X&ry`v>P^ZJxt=8Ty%re`=xZj@EsNXrW!p_OpU`8w?+~RaMI$h;Q zBWCHcyxfnI!z>8Yh+nKn5M4hpe$~_Dd=f_gjfM|j=)r_f^PqPS6M8&LI9c9vklzg`33@jxW8pYNa6WSL`>lN zQT-u7^9K@fLoq_jm546I!BXY1E&6}|OAi&WIFtc=ZlrPljsFfCHD_ zZ-)RK%^I!e$Y5^ppxYXPx`vLSaif42+p&|(OOCb$#V1ZV}q7HCS9AKnJPNk>p%Fs8sLUTBLpg23m0eCgElT0nFmC4e%blL zeF*YF7N!Z{yoZBOBVdu^qhU+pVH)aV)lPp|f!A&6voqes`ca#Nh!CNhj#akmMp zx4)Au99(T{Yh$%?>d-xjgMj{^#g5(D2*dp^x=SIJnLrdEMI{CBJ|28)!XnjtvY<@SwM!;+X z1sljnedlS*f8qM8Z1b`PFJd12mT3yX=z?<$3uPPNpFnqxVS?EZA?i1^#|Ylm4IqA6 zhZl0|g3j^U#P{f|B` zXD$N@l1Q^`X8Veee}Fy;d-;+;D|HS84Ek+cCrqS!ZU!CGw*|{~7`dF2v;MuIBKnfG zbv#3A%IT`vp@y11^G6TcCal# zrNvVhE0V5Ts7hNDjR*b4UbKDs&7y6AZU61>+aHie!}w-x(Jt+n!#eS9J?Md0Kv>dy zap#P?03ip}<#-+&r%DF>j#S)0wdn7nKquA<3)Y_rXg3K&REb)a&6Q=3ffS~a+LNvf zOccJelmR~}0?^ahX5=C0s_I$e%?`Jeu8Kt3z9L8zJ}N3}J20UFab4nevM95_I*9WA z=|_YfRx-_-l?b;3yRza7MuhteFaY*LI`5g|^e3Hsq1Ug7T@!W~-Qe&WOzfXELMWZk zYJD5EDBHnoyFAaB+W^~V`G>&R!MvL~@{pOq_?xy=FugFC=Z;i(y>OA6CUt~QG=gV= zOnAO<*xepOqBrVl|E_)0bz0ot;Am<}sja~R~ zmpwjKU=DGhzgLvu+e(DLd*IFP)V{vAFkIi3A$d1#gncYJEqva1pJ5O+8j(pbI*D22 z+&o=O9K2zaHqH1^nqg+ADmG0^81%TpKKc+{F*(&B^AQ|_4u=?aO(q#-axV?}&Zru62HZPinhj zmb4NzrK_Bm*eL}wM{4W9E9MRjBNdHol~faB#Z!$s9b?S2V2#G^o^9&m4G|b&RM*=D z9m9O3sPMkuMlX5HKR$2T;X!+}R)Aj;3dwH|7Iw|lGZd2y^%UIAK+LsZj>XgP*WH8- z{pj}x_r5~cJ)u=d-4PkoN9m3fz@I*e8P=AAYSb$G%mp|U>t)Sqe49(tE5e9bm_TGo zZBMMjzY13;WIjeq@EpG*0qfwe9c*GLCuoZJ1Vbn3!bBzXAj^*?3{Mii&j!eUDXT{( zm&+Ymd2(WyKT}IsfrpBSZg!*j`YtliVi>{slT;)=Irj}~h_Y2S$e<#u7l-ga`;(g}JqXE41n3Y#j$#&myKn74 zT{PV=#91d^31Ha=>yI34+DCTynfxcu91BG)Ol&aO5Y!q0#I8iQtHYM=2B+jg=hdvo zs^_Q&U=IjvO7bM()P_ZUXxW8O&rnY?|7KiJ6rz~q?p$D{4v2HUbXWtTC>uPo3mT2s zusgvCmxf44r{v2zbOud$+Kg_f?I;)o&@|s1Hs+hr4)zaR@~jCIKdr)v_MPYaqZb1Z@l~^`<$P-*U30*?J2U# zjnNystwhfZlTBz%{L+|uTC~BrJH=e?qle$tXWs=;rRzcr(*!r`ENa%Opt87x|h?J-6+&=YW7wSxk zQ}av6R>$U#hEe0X-sdK;Z@&b!Fyf9=gw)Gl2W~487wV0i$Cd89GGp`h>riG(3)J|} z!kZIERE{UkA%a^rdNyOi{iuFx8@n&za=%2kjhd1=@zgiz$??_H1OF0i$8PK^T_UZU zj~z=&B6x7%p=yLnR`k0~h0LFXbY#6NH)E@zUJuuY(X(I&;$x(M7bwK~g6olx`t<~e zdSnx^f%S7x^?8!Wn0{bZR+Ov-9l;`p;*b*yer&yRUL3W|y98WYNU{Z40>0k-j0ZJ2 z?NwjNu(5#TW@PHnATSOHu#KU2Yg`T28fp^I=?YpLIu*DqYwG@Nu_*nAT?Yb?i}y_odfe&7Ko^C=Jf4u`gw1)D zAI^zIrnheX!=wFIy}thDx@d83Iq{8xZ2;1ou~}t9l$FyBU83Zr4NBFV^D(vq>;?My zV8wGst#k0~&uEn|D2NMP%aZJ07wHgpInetVoPuuJQEe=U&Khu5Eb#IotBVnGcM*-y z3~!WIBW(M-fqZ^fLXVg~2C>>kyD^%@FgjLzkjaD5d*5 z7Xrf@|J$7H$FG;XddWYmQl%&~2h8XxL#T{IdeXr4ThRc#VG)MJpLQ9ZHMhv*<)b1S z8s=7;+O_Hz$c&duRe1C|IESK(ybm7Fba3bHS$a*?7E(D#Btj$(on{OgiB_d9q1yl< z-n%8MLSW;`(kh*Yq9W4B&^)VK^Isyo?dkfsz!B#NEhUCEic{}#58vVqk-MK?><*Z` zw6wu1eC^{{znVibfuk-F)HNBUx|79!T*seTISK2RQ#mR+Ycucg7G49~|?wJl7J_bL+h|?f>@9(#UV!|PKD1dKT zoc>5x+g?W?7zsIWPkC&fBagy_vWo9-Hkf@B>1-D9uOAepyqb?Dy)!YG!Q^+~Z#Q_P zDK%n@?Z*@%*YH|Ys)@yPk4vYsp6hxjI~^}#iR8ECZg}d-GHzMQy-?C4d-znGvs0dr(w zQ}PGm^b2nyR4|27z~iQoH`hsN&Q@J2=i5pTwU@Z-EokWkbBYWNd1=sGW^?4}vC*XH z!XPy9aJXmmgaJ`E^gvkp-Ap&Gz<1@lZS|zRp+e8(QjgR!H}t3$ue81EXy8nRB8gVQ zRz-w@WMXL7RM^8;)X!fL-xBg36TQ%6&!`du|BR8kXH5}WHwIxE{vG1^Evov9u;~B> zCxsj*9cht{58mjx7rwdfga9^2&u!n%7YP;URPuN*{cz;x84sdnr~7Dz^QviM9DRUq8Miq!K(W^jCECuuU7bX0 zPrH7+(e|Wp;XD+Yp5XrO<37ZGO=UE-Z6IgQ7T=8ka*E-UAy(A@5rU^!r|e+WL{yX0 zLThyR>B!R*?MeW6S=Z|2v+gU(PK;x|J|>-$&K@MuM<0wV##g1gSJOC^k@Y>zsoAKC-g>c>mRpaZQ+kb z-=sGUGyVZLO0&USzyemjc3?km*2JcW(3nHTXC$FWm)kdyp$kV{m(7|zGyov20rAL) z<%LI>>p`EDBwyQW$qM!j%HC&YJ898@-?ClDG$a4Evr(y(HsUH;yAw71JrSjDTCXND zxU|4jfu}Dfg&2XnINoiNvI&wDZ1`kqVydPox%bkJ2lhSXT!N8Z9A{n$u3NpYjmfm> zVPAJ+UlPP_HAprk4c&|=39G$C(uJB!Yo9u;;8<qJm)D|>Kf?O(|-JFxnx z-r>G4$)>_&rN9-Yb!l0XQ$(fUZ*I3uQ3sV`3?p}v;T9^U99`?8HdCFQoEi-yf6u=lleEw9BLAfJ|kOQ<)s^6!&Pv zQ^~h56I)_Skss1%Ml5?gv}a&tpYW}i&ds7c$dxtOB4j7z7EV7Jxt#MPy;3wiv!z{w z@us=6D~uFs#Do}qthbi3f4;Y-a3A5eGUL7-94OVz~WBPL7fJABGzM9IiR33dafPMz8{y zGN6e|+E~uVqh(Mt*EXdk@w&3H$5P<2|BU{FK~kL1v=h5q-gYGgGUN5USzIeG+QPd4pn_ zV9x}f(GS5H>{nJ6$UJ%jJeDboyIu*=KQEMTym4o_-wxueoX*yI^mA0@^^x(&zrV1~ zHX462dEG`1o&hsDH?C=ZLUf*e7y!xSE(!&sVvAS3#4b5*^U1)Flc=Ks;XlJ-6_Z21 z))ao5yTg1o3jWLf4Wxpp8Qx&Ib8kww7~0|7C0W z@0DpYI4anGtzZiv{?qUhqwQ_eK=1zb%~Y0LVGZfQ}AL(7dsI4W8id4ZW>i^I9jxj{?qLx3U3)uxezjznngC3y#QH{Dt1 zYIApj_8<;#hG~4%FNyVX&@V2sbWog*`h=eYuA1WsFJsm&pFHq;l2tO zSWU>gO3m>BDir(!txFOT59f*tZ0h?7=ZeFz`U`cF!7a5<7~Tc1-54SYca0A$>xBP>0$N3*aJgx$b`Net=`x1c1pA_bj38Pj%2MA+4?m(>!CwK| z+eq<&ZGA`+?#UNAmZpMiBehHg{qMkVSZE?t(6GKlooC$Hs>jg2!W&t9aQ`tvPt9H( zPgU&fADXAgmZv@*U&55W*U6N=o26(%s#!^zSg9RMQKil3MTlaEf2CLXNdP>u`yCzF1!cnmbN@rDxM2CG?Vvg*uNwH zpbV``{CSQ7rXpVc+nF-&WCVT~{W6JS`QD}ldxU?=pP< zU#`-~H}CA6)abY#`TJU;@G%|%!1Cf&r9K53dhPt5dl_vtFRYqz*G6zqy(O&k$c({M zuqXHE7XmG%wCH0&nya}#mi7tp{o3NFT8e7)`+wy))&(V%JEn*a*I%-5p1v0kzh^Ds z{9MBQOU1rczIoao=a$Wf($YYeMn2%iNsT2L(|c5NYpp#4mDey z1K*1O&R?R&sfsm*p1oyjd3-5l$ZlFcj@ZCCANP<`PQOgB7G&wov4HE8TWgaKwClAq z2b8d|HDzff0OggGzh_C%Z#}0-`7MS!c<-I-9!pC}`Ts@O;9BnkL!{!GI^|?1zNjeG zH!dyg2`1rA6x3maul-E9eEmLQua&*)%p4MxXa>LU<=$U2MKI-++k%Q5KPYykqQ3 zE-M~qhbwz5>j^V>;Ny2K=dzQ#Mpd!L*U)e@&}p*JhUh45*G9uM`wnAc(QegUf!37n zcb1GOYhCCcdu|dLw>rb+^^OcT4M&#ZHd|lkA6g_Wm+paFF8V=w^r`N~sU3#KEVg)Z zxeMH+Gn;*W-DkosWPxiQdD5=7T)Af}KZb{Ve~cukkfu$Dfcu@?lj`|Xrha#}$OMcO zs+_+RSkW=6{S{7O?6nEZHrXIKzxpYk>YsDX{gSplx_&P+Qh2)j*WNkvivMo!$HIs% zS21DKeFgXrjO*w^ME%h3 zW_Ru;n##6hbwoYf7d({OL~5uH=^$5SE79*G^`1uNmK8|`X47-uc-swe$;N4_eeMEd zWkTzmFX}Wn!#p}#wmFeecUnkVOy68`d7|s=qdN$GM1Xg4n`7JyPkFY0F`OwEj(lQXgw+a;d|=rb%6TSaK}L`sw4pjvantMsLumOym1dSeaL3Q4+6I zzsiTy<$s_izX>zg8-OSL^8}G~YooI1exj*2W(|A_)NFF-1!ZQeXD6mGo5OFVbEF6! zZi>s`9GUj{@=$_y>mtIkXSQ!qnr5#zsg!-)gs?-7Z?@(2LB+1opb7yI@Z+=Ry*VUo z#%sQy@a}Of=1tyV_ubaKuoriD#4 zcd%%xa@O*N;^?6F!+`YspT|5A>c zZCAES?GNjy2uBJcq`jr)u2Velwb}Qt^frz-jz8BFu{F}u|B-&A6KKVM<#h%*|gG>3#d zr;1sbM1J4B+0U$fptEI5?EG<+&mAqcNQ@F(ygVoG91XhI4~-=xkV_fnqhjJ0O1H2= z`GE->hki>dlbX7vag?Jo%Xa|zL77O1~9A{}vtV2&U{EJNqmBjED} zT-QFKE}U8*_qUOsL`tOsHhvu0bwhgmj`{u!b!AJdUBL=EpjPBlbm7x!J&m}=JxOSgPPved?o)oP)EXhjzY!F*+$5OsQ78s>4NW-zR=B+`9pxW~xOY@j8--m4?CHBjbUoDc)*Q^rlU0$IpRHoAdUc zLLDmnyNOO`h$3d${^2^5nwO-Tr*iYT_=VDu72i-N{*FSD-{*X)C`K_7Es8Ckqfssz z&l-WPD6eD@4RSSHaBax(`;!Eo-b)D+5{TB*JDejhXrOR>C)ib11SDe?vrfX=F#CMV zUndfDX0Dd~dGSDs#zBy_NN8z6S-IH1l{t*ro~?CWDsK1{ia*)9kv${4$#>%^hpDuo zIzVDC5_~zdC0L%X2QmvY7W;wXrG$Dk9)mSLTS9l?f(jpSRZm|dGf)cNO9dlO_4Ko1D?bEcBu+Ss$W>P$s%8J07f4{2CrZqA z;X`_2kCa8?IHpM(Q%H0F$*-Gvz_CF!R$Tcu1*bW%KqZrgn4@UVjoH|Ky#(OK+njpk z!z4caqMIzAR>@FmG0y~R57?*uF_M#=;y|+u>|@Zk#QlD&G&3Bie2+5eRgl{SskuG5 z@4&<;=-~e#BG+o2E3Jtkn*C0tw<56f?=KpO3>RFS+zm!6tP5Oc!S&l0>Y1e!B zZItPAXRNFBnKuwT@TI`~r+2ox=Sy8P26!C}Yio?fQckOW;+sT5hQbrE-l%5SC)T(U zJ+fx^%?Z*&;M8801Gjy=aWE-IPZwOSAgj;rpZ46>_{9=rQwzeEKmtrsiml)HJ_(%$ z{b-Ly8knA38-iE6Ow5xMr%fzKBRLp{WzxD>Yio#W&Ffv@okDY24>Q`Vb?`co+9sIf z-Glk9#^bxAe%mE;$GNHjg#KbGr^(r?;rJ(fwfMgkX0TfNeIKe2XGOS>T_uaeBMrh& zW-?Zaq(y#Um84urV~|NE&0d9z7_uGrugH$#6#sGO744H>7$p7r3RN)Ru{G~3H?>e> zhNgnz6F5b{wHzKJ;CR8BmZm%#0{#R7y*a1yz_?2Gfz&}j+5vu+y@{)>U!Fk-&_AuG zV{@Bd&22}-zW#a2B755M6K2QRvn16wD7t;y-Al|8k;AaM&Vj;IB z0^uP4?gIah`|vY&PO%WdEkgEYB+K*MosGVVeo(inpW6I0Zbpv zyfEq3l^I+E+?0!<(oIJB4#rIB!DP52hsoqIzThx8l_1R`BzdeWkMz#wQc{(kY~xy% zwMNUkfI{$855Mbnmml|?&5`KST0r6jT#$4)E-_a64F|F9q<}nlq=So`ZC`}=coG;E zCqHqL;T{|(KMrh;8yy4I#GzRm(d=6gS6kO4H|UN`T6al}Fk5#ij#R%XB1R0Yy0k`2 zh+nDpkGWn9N9wGe6XSNqq|5*^BE$_IrI z78D-^w~ta0Gz=+8j3{_CluEdbw3PC=dG=<{YkipXZ(X9>_jigAXGE|ilrppzua63m zDHIiHDr5({j}$3wKny9ZzX_AU)<{s~I0)57eY{-}7!K~QMt>QC%)YTaU?$1b5nKfz z>NQ?{PQYzgPe>i60I98z8ToAfEQA3V-!D$)#>0fJNHpcq*8u(Yj$q)Lv>w^S3`+Ix zYz4Z!J5p zLQUc9z?NpVNuwVM`zg`O1kD(^^-aMW^Fp@Rp$NQh;e9*0%05dVCiuj@t)^sKo_Ap0aRLM4BxcsXW^<|Nu*+X^J?2NG?G*~YUUu`$ z5O&W8e8f7gN4!vR%kB_U2p7U@|HM;Dzj%XjKe*wmsbxOZfkiCU!R~O3dXShb15G#f zY7w&RYB4h24VNs#ND&oj*bn9b8eF}sI9tYnej*KcLLa69z!Jcm=|utulM*~1W)J*A zr94_1!+|#|0|VboZMAx7GF>j~1@1&M|Gq`OAOvT;>4@>T0rGboIz)pX_%wEiB74uL z<~XF{W$*KuMq-EM^SOVm_uvF1Mk1d63t4=Uec~+IW3&|CZ}S44@hdFsCf&r@<+49K zndOrue?(Z99MHn-L~>UW!o0w>N}hi20;hFC-cn(-y3jwjZ|o>uPE07m2QxMrIcD0XgvbJkGEu0Rvdcyfhec7<<(ehKVt?Tn)O8FLU`}H@#yDEf zDShUwcMlFJ_$Y&R6XCHJgPa-@B6Qc>;YB`SZR7l#E~hHe_!9#FIXMfAIf5u!Tz_OS z5f3XKqR&-56QpFfLEDs3Ao;yw;hoNCG7K8_Elb+$zH6O>ouIU1Ji0W0vfaxs1Qhh= zPhTCZEw|j3G^OzMD`ujus;f%2fKMJoa4++5-qumpTPSk?2OHzTSE*h*F<) zjP-oI`zYr5I5nP-WoqTXz8cC(7?j<{4|q?T8sj7_Zhi8&@XyPebq|lCm+xcwKZn_q z!Mc{KRo^=7t|XCvtsT-MwXR*I*&?BLq1i!QtnxO=(e>}M|0y^5K`hA3eeWxNm0XsU zV9qY=!5rV&I%RCZqO-JryywtDcX5|{_jXv_~0=DI3V9D#w~p}t~d+G z>yE(2KNtV0lRP2pd-U-AE(H6VmAz_!10=v$~-U3{n6TQe$>;b z_WcqJxNR0HW+gj>P#_7$Jsd#^+aV5<>PYS$>y?uuH=m|aqgC2`Udn;zL@s#*HNG}x zKSuFK0VgGe#i2(Cc%FWaOG&Quqy5)2iTcle40GfQ1Lh|3mTmfjw`A!l22Lrn@-b9D z>BqB!Q zDTm$sWl>cSb3nI`<9#3E)Cg5C)mPSa&m&Esc$2Hk7q(gifEntO_mdk&@G7H5iBU#! zVMp36CY00P;)*P{hEB|%`Y@A=I@4DBlPpVpRvJEW4N#IhJ7JWUyjeDQ=B5h_Mwu!! zlcgEPeGjp>&G@Q~<5wm=@VoLbLefv$VFKXCUy4YW;OrkFjp2EkaNTc_)~8n|E21sy zFQ=@2ocOt$6D$OY()hqxq~$HiWyBdsJkqh3HMTE=@(P|zZRA?xxzf?9H_+5;v&tJt z+K_)?&mezoKYKc{QDT6zuR zx@oGp5crqPdTe=%4w~Ev?*5H`$N5jT+b3+UN)2bDTb1uDXNHMnE6O`^YQ9+{Ugmcs zsqy}~Y`Y^HAIXd}rHg0GhJXxqrB!)LYwN_Q^DlZQUh|6qV);kqek~cQnk@&*>mtpJ zVN`W;TEZW#&Ob)z!B=4*`qa&^NwL_8UQVedCV~YRK82_f_eKwyfpW5a+qevNkJjo{Px5_h6rWC+)v!S7kwvu}P z0~BVziT=28|4WsLbPO*(5{8*^>B-UxL3(@uq|mB=gNA3YU2Nmi<90w8Y)PI`zt-UGanJk zWnS2qNBCv))7F&#AtH`C<0?rtqRZ;!Gccr0?8q$6?gKa)C^1}-EH{i+3{V0mO~)zi zzZNT@IisKA)NTtjv*Z)UO_9}C`n_xJ4h2_iRQQ2--~VYja=j!s5_O6x{5?N%kXGdF zU4KrC!2Q~8&K<{@S|{i(YZiLsiVpt~1J}1Rl=Je_ArC8C?L$~5u}|yiODoJ}sHH~B zn-2F|b!0W3DQ&b7)(;8c_QA>0|(B>cOrLl1ZD&x zdnHm-QtpUlx4ug8R=T?3@UwiQnsf%weUJaPcNg{qI&qG8EnkFnJ`SSQFp4KK7pWV> zYbIYQo6h&N+t_7wZi-fu2NIBZiVsgG-E57S=I*%F_`L)vZ2<01YyF3SXjq-=-Z4cZ zmM!U76UO=Gs!<0VBm11k!{a65d75J+!5;{Ks&gQ@dI7Jl#8PDLkqlQHf;g};(QHRc zU|qUEtv>GL2S!>FF5v>tX~y*Z&c4*&y{3nJzZDso9Kr@ijM1pm$_gyQ+N<;{V)1p6 zt@nXYQEYDy#!z)m%rpB)%Bd%EYXYe!Ak%n*OaH>*=Czg(HR;b@zfJOQ?L%h;=wOe? z*|gbDE)|(y+cCT%K%mQdHUw~*q*nK2W4E4WsKcxF;LWYs1gz;cqZM&(X`C$xPfgOD z15l*BwtMU2aZB_}85?7X3hGy7BaJWoEr2vC@)aFVP#)7cT4A9j#W#@BS+JZOk1dNC&Gd30;VjU*4J`vO*qg^#nc0cEDt z+Z2^nsP=!S#a@AM2*6IjNWcVx-yIUXRw?gzNpvByQvq3j9$|-n+^Ja*vtwl4fUQ=V z^zExcLJ5Dw3sJ4dF_LlPTrGZ2CxLN*%v9jY^>G#vAqLhd@j9$@I>2qev5_I~yj74s zQ+~IK!A?}Y1CPo6>onMaG@M)85a9Lx`!^&jTV3|Ag30w7(4T|d-RgXD3x8O?dkv(y zL@vLpN1{X}nVJtg$tslNuH%RdjGx2G-NZHjraUlKYJC{s7QIpMp?1=OwF&uGAaV*I z1C%O9+LcIDxBl+wc?~5tXHr|Z;l&*AU1?G~I)|`>tgDsjr}Pgn%msd5e(}U-$pMow zPQ-)9phmo-MJ99KvgItZ!opG^L^GrT7?BXjjD{y71U8Huly8VdC4$*9?P3dHbMiU9 z3a`v4@*zl@`|fFB@f5u>_A)o!oc$&5nCm<`~E@Zm?j;5zO_$bMmkOx>tm}) zg}Z9(Pwf^C)N|m<_w&~Gmb4b^%AVwT7NA&8w#9apO*rx9*5r{52BLAUDlOKN(57)N ztGmK`2|Bb5EZ>J5*5;psWYwuhEOFCMu+xlV7Bl6*BU>GC1wzeguD|(FeRw|J-b6z3 zPXXiSHKz+o)BGZ)t`p;TP)}IhhpSGNX>INl4!vJjxlTghvC6E@4?o4m6G*&-qgU6k zy046MI0MEc(J_?CF&H%jI*@FtvZt*&QRw7(+t!Tx2zZ5_x>icN)g{i4G?k_|q*>@H z{FcfVwUWxf)3Xgev~wm*U+QU5FI8>#{LX_J2~?65Jh>OGazD3W)^MjvNa~W;oJ0-y z9|Nuq-5jeFj%QQ^0*F~K$au1QZ?vw6gy>zC8BvaFKSY*VF`_n}^ySO0Q~~=gS5Xe9 z2}+1{_1&vBJ+j2U2>tHs%-u6czVzR~P34;8+uTB-^6eTxtXAa0!RdjvdL5Q9!=cb!p9a#_=+VA>3(-whw#d&zbnR+J z?;c{y``L$nLX{fckpns*lsCAmlqP5zH~uGhuHk|1B*|}r1R@_| zWH+zM6ZCZXt&xvauUdCvT)7kIMm1r??np%+?=`}%%5o0btl59JF4zC?<)~5v+PUa6 zaBIU`ixqgJdL3C<1$b%uPV2a$TtM64eV2N)Q=%w0q=<(Eu_uw8Go@rOL?aHmfHN{;|XL$DED#pQp!+C)5a@#?~P&!f! z*ZvzQAJ2gB0n_!s8kAgc;*T2!NWxQ`nvDWCGSQ3~g#Iu<55)$d3PaE|SA9enGZM>h z!Q}Hl%nKUJvB{&#_q;3*Od*h0uwl$RSONDIEVlh#3IFHU3Ddb>OjA)9sZ18;--n!; zkK=2JgZ>EiU4WbJ`_g)O7$;L1u@bfTI-TXb66sakD|UbHs_sGW%|siB8c_mOOl~j$ z{cUc@AS4~AX%@yGJ16m_MePO*- zRxozZW2J#_l1|+gS~h;yZQ>WVK?fgp-%s`i2=u@2V~S~E1ZPE3L=G})Beom_YUD;# z?TSWNU1akzvkNQBuGb@WDo>xXo3`+lcubN*FD$9rn;1}SWz9EKv$m!*9(7Sy01a zl69gNOa*1<1w7F58ySp7w(gK1#IXLq#HA{yFT@heni*CXk3#}iKQc;_*B~-jUxZUt zL&uIORex&W=#oS-%zk{(&_8FZ7e&OM{_<(=fbHwgZ`<$i9=T!ElE4SydtG0isaleJ zo_MTt2cEyIy_=h@93?*=ior@(Or{xHJl z9YzTq<{WWwcT$h9wBpy(p)te)RN)!mMdS@TX}dVHQKe3uZ~bTy#QXe!QYBi9IjPNE zyAWk)BPnBIemu_k)C9KM>Njc4;e|=m)Bq!qo3}?(iwR^Jm9DAywIv+UD%O7 z&YvRhhoi2)-G3+=PiV^aKaFHh)?NUQzFzsou|*MHOAi@jvLt|q=SETu>ie-Nn?4)# z==Dvkj-Rm;1;JwSWz$Jgkc3!wrYchFBp}I(AihVokrJ|zq_dHxJQ|>x3O&ujx@#aJ z*rTZ!3@yef@B1_%Z<2=yo2Tk-P4|p!+kdE!XqbFUtJ^24kFwDaBlCIwB5gH&zAc(fai|JHf&-v^6DHr3-Ng7 zE(miqqYeyGdBp!VbnTX&6j2>6;#6k1^d9xlF?viTIH}&$Bg8!bt7SO`~dqA6XhDF-unhbaPZ`#iPHk&=eB5BJ{hm-j? zSLe;*mnKu?9E%vNqc*8JD1lB|&S)sLL9>UJlmsNzZ>MIf$g>k{J_~57S*RT-`;?;i z*7bfQeS{IKs%k2?P@QJSnmzcAB_sn3 zADu0k1$9UUp2PzM-Q#y~zB!mGwrKcUf8DzJ`mH;;&^%YfBnD{+1SA|9|G{0r)rm}c zvJ+VHihmKtDG4x1^h@G65E@wfXLdx_DxLO^w-_FrH+t0nNsaoS**uv!Nbm%ibIWj; znOGNC=$YN;SV-V_4N;{*b%5yX*XWV;PtJddNa2&4L)7RaV`9~@9T1o?UFuG`ds66V z15msO!77A=2uSW8Q2A*7QnS;g-K{#`8PUzp;6QOP*gUfza%*zKBN_3|B*K_P8Dv@d z!84AXQXpgYY z&*n@kakXn)cyeNIryekoxwFU6Qm7(4#R*6|Yn=2m4z3MA&MR`^Z%u8$3E6h|P}&4!?OL7d8E%8d%2qRUT1FCKSXE$ zxXu234g|poj3mUz#8<_ukQnl{l5wyIb0o8y;WHyR^_f?+n>5i!ghW!I0+r~QP|wgK z&bPR=x3W9I8$N2kzh8-R=-V=B{?6;~iW5#TzjP3AP^gU-7MLJHFD;8XQ2D)-F`jQ1 z3GeaAJ?F8pxN!uxhy70uCB?^p(rjGT#ODm&uE1POVz0mkKFXyzMNS+wBi^`otdhSV~@F&Sx>&m#Rk+-+p19 zr=$Aw$TE|>pJlKl3un9!@{nUA59|kp{*BKQu-&DG8|kzw(Y;oW-N4T-`(qPiFF=WO2&szrOu+Kq@eW4L4~9qxTjO*%omEkIqy2UC=?FvYXn zlw-#XOl_$ee~Nn??qp7v`Y+1sG3Qo-_#5JI{}0CHf2tcD%=5ocAC$lja9ohFbgpd; zBo+Bbk^cyV6z|D0AnWwUAh~X|kFatwp3=>|E-B#vM55@iUKrdp(J~7BSD)4&jxJ|F zuU%jmfq~Yy0?&!ZAj#!xCFNid;%LwwLd}TPj~rC@?$*__`e%t7zLogu;L5T7DOKg@_Z(D}WYJYIVe2Jf-{j}ouE z9eOMA&pE|_j{($w+_T0fdrg{R&|UK2jqbx?6@wd14es&)fr0r%seOZJkPx5-d@pdzl=n(a+O=sH4>?S>b(e8!dt!qQvULa z{jQ3Rlfk11fF^QnzneBI#Tv=H7JS&4wV&+}?R^EkqL{tIvOcihOx8$p*Bj{;SlQE0 zluopy4u-c;U`w7#HqynGR+R)XaMS2%vw-)iVysyz+T(MXbU*SbT&0TiiM%7SK^yM2 zEO9(h8FV?(nQgfk!Hf7<{FqU!odx2HcJ)5K9AywXdqfrYMJl3hd!PghrL>D6iid6_ri%r%f?{!ZfebT@+Z7 zoGvkQ2u)@58$at}HBsw4giV?cf}L*X<~qqf6Acp9Pa`-Y+4jjXHX*(*ASS-*ahF<* z`wcP{J}MjKAPj6kM;LWJk(=a5YH%r_{VT13rJze zuQsu{bzNGK`Dc-xqbN-as2S-evz>t}wzO$d0X0*4a*|I{hKZTnx~z$RG)ryE>x>u` z)jP4j63+n?#Fr$Rz4eenj|Jkgaok<>wb>*OrxOOtxinW5iE&PbJwv=CAw)gdfO9IE z1@`06`E-e@D9V^)={baXY+P>~87%lusIpp;_xTbn>O90U;>cdS0G-ei`QkQ2aqrWA z5~ydgA|iR;9_6n8PM9k0g#BlQ0Wu3O-_Esb1ANW!MsJ}PSL41JSoN~JIM65uBYDixNv){p6bZV z^`FqS?yc;#w@>z(@vz_C^&fKhAdLkD)H=OKrf?)cVBCNnvq%LLF+37Lt_v&~NCfKu zWp*{fM*-Vkp;&LWZ3E)EkMyXo+Pj5}&z|An$~Q?`G1He^Dntg0B&rAo3vt;v9hqz@ zI>tQ~*}Re{Pki?Rsu{hZ;h1(K96g3TMOmZLsA~KM6PzCmd!J;D%A?Nl-9xEdw1-q; z+6{2p81{5!H_M_n@ZBS+_!#z7Wx<=JQDEjhL0OTyC_=!^7b+9Yq0ccuQydD0J!x4` zP1G#lrh}?SeJC^LQ4we7ZF6Kn;;#>+oS$~vfjO`;0N zuhdAUiUH_+#j{FgMJs3VuySu>;hWeKZ+xs{xk?_UGjPfnCNS7!j;(E61=Gb67&O1J zN(nZs-3wXPt=)TB$diUc8&@e~^BY%bVRHH>hf5k)Sz|>TfyxAllY1xzB8;)+ zRyWNofl0%OjgJZh;*)!X2G2~fuj70C2G4A<;#N21EH5d;JB_Q%N*|PzGul~D3{tGd zDU}QwX(m&sl-3##(n?7Uz?#;Txh&q9rR0ri%u0TZ2PvfmjY~=y{*#&%24~Dlf2|lz zGk#Alz!=YoI(!Cp z1!a9+9}c`Ob_F$LHsbTD^jdwiXdcExqkN+|^XDZLZztwpXZTOpZqY%Z65?>ns?es- z`RcQO1kCaOETO=Hf4zqb1O%7lyP+S$H=^8AL3{=r#MV(TWN@fvC{0enIF{e95y0R3 zu>{DSa4WVDA;HPQD)1ri+y)khI7)ZoC>kUuy?XJKe-0jlj$mmYgnViFKexI{c)NM9;^0Tq-d z*cMqt2_gtLiw8`7;6n4=cz>D< zUdi`O=DCptY>#~w-lc=%$7@e__8o8*0%F68ucEXy@#ncAQM!I`qYv&x-c&K8-q;O$ z)pX`|^p!$eUPpF1e#1a~`O5Zl9u#JooTfgskL``ue{-q|%nAq(;)WYUS;Eyn4U_LU z3){N*Nh*;28CjXGo&!ns@DE2M&IumAgSth<~6P=B5OhA#yWrd*Y>R(eh&+>8e7D{9`cFYlx{L)t8A&h!+y=^~ni*2?7JLCc6fQUurArC&7N1h_><9}_1p6(9g;IMye@7X@s&fxX ztfb29tw1Go)ss~_VDY~}(w@)lEsN=S_SU)ISF4~Z>@ALYk>N}-l{FI$CP8uf08L;* zyt~%kJ<6=KVA!dOF=uFJ_)y=guFaYE`UM<=^Q@)=xZTj5Xz~_C&)nTDOpngKA5{Pk-zeCg6-6yQH zKnPgD(I3Co9%kCZrCCRWGlz?Rw$dS;oK=?%+bND|S!zvi)s^L5PS`K6+ye$?GO0q; z@PNAb^b5!KVIV%%J_MS_@Mk(5Bk+)K)t;t9XIx=bjO}A~6)?lJGA2`@ybj7+1x)98 zdEaY-U$>~e1Hr1^Q#lLB?2~`y*8xejsM%}u98JUfR4&-PBr(MwwpAQ@SqnLA#QmoP8MwdWWZDPb;pFex~7XkKcj-tY#t`q@WLy{x(9a zFq$I4X{635*FyIa#Z>$&i237xO5n;1`dS>3g&6J@A^T^{nCDho31; zDx^K-Fs%MyE_LmX0b4$|G&I{BgrR1o@i>i9FdH{kP<7jj@nP`fe)%6{omEgC(XzG^ z2om^!;7)LNcY?bFcXxM(3GVLh?(X{G?m>bR+}(57`}|kupPH$uu5~jt7rm-`y-&aI ztHlz8OWN&>{CcP7X09BA5Pfsk_KcxM$2vZW?bL?8+J%f^M^ZwF=aIZ`q%O+>{1yTdQm6s z{zaocPj+>X0Xua%@t> z5RCQlWk;;(NqGXfvP{F6YbHjed;wpOS?eWa8k!`m1(iS# z9x_I>=&G?^SieMZrmX4dduL=SEPL!2%ltM};&k z%S#uw&UV5ms@xF@D(0M|$>Iw1%%Cr!e5i>dS$|IECTZd@+9^^- znWJ{e$o$CAMFcI_UQ~yt)-*oFjFB=Ve4{Kz3V?}?8#a@WVC#1 zw3}VjkN-1&kF^HgxyFh-er_Tb=nU;v+MZQ&2U>5GVp1R(xKZ9=*J@krjKIFct zDK?EcdET@=6PoH3v>7Gz#ozecR!`XrLBl8&XdDRZfY3?5TwKiSg)M;z*xx5q3#Ch| z(k-C`IKJ!IM<4o?k}b~6h8Y-@)14HKLPuB*VhCSgr5y^d{T-lF*bG!}rGNf}!M6Q% z$Xw<@5#-p&zYgL2Q}K7I>32}R)&twz7i{O7JPfsb8_fA?L&BO%(Ki-I85|IHRY&}>Sr&q=UCEi@;8)8Rhr|3g(O3XbCHn2WYbhzRu(Dk zT(rc*MCPd?H;BPT7&;89chQ;$ zd}8nY>ac@EaluIIL&^oFw2as&z_Z9LW$RNJ0T9E{y<4Y+Ir_-nXmx{Ctrbs(D0Wvg zSt!{xO;7W?io-YNqRcl!Bf;eGxg?v~??{G?F;A;0z%b_YHg#9;uO@ zC4Aw@q$2@8Nrjn0I7LZHfU#pTXXhKTPwuNCVIy-T`sYl59sO@NIovGQXJ?Z;Q_Cfi zZ*=mLM%sRd6RaT&F!>EQIHRpezdmGY3)EAqUHxj9)~>?^++)jVn)q-v2t{wjBeF&X z*&j7J$MamjRoMMH%~h5T;T}?y6k*huo!z_!%}VjGiXQs1m~NyY&T@z z0`(KiQ66N0ouj-8z{#Fr%4MZf)$>zLZu;?N1Y&K;Sl7eiftE7;uNV|b@EJb*5Q(&x zm;q~I;Zk1MQn>z%-UYl|%mN3SzKnzrPJFP))Lc#=6Q|!#^d-_>{9@_c1HMsM*}koQ z#v#YxXx(~aIMSbe({kcg|K=vP-KNS^z42_TA`mSN5 z1dfI*w}4uq?-U0%ZVr`|a_-##@W!sg!>42&8WIvkoSR&Of?4_^3dP&zn`)sBVFfM)?vRS~oe-dSuZW_%D^~mO2Kqr^I@~8RcwzXu-9w2deqo)u7C=IH2V!d%tQY*JsiSRVjS%rK4^DaIf6?}VW?H? zH$9dEHX8Z10&HRGwwSn8KzdB+{#h~sIF3fs0KbSLuhv0pmM$=ZKRU!eFI%|y#7Scg zbjA$i&df=aJO^^fw?R`LeOugz(05L`!THnW+R_qsGmZz;#?6GE&6LkXJo-0(y&0Hr zTYTa4cMhK8uOe7nly;Gk;_>0BuF)^rO`j5&h4UakfRW5)IQn@5r`ej9dt!S6O=#M(4T4-X#N z>c%iAdvEZ18&H`eDv!dvTt%4)a$|$2;8L%nqC6snuQlb%i5amk$S(YSWaWbvKKB0f zf*4TtLcCO86U`~Jh6%%G>^fjL)~!x^BPxW?RO{*I)AiGZ(JnzqFXr0aF>a{OG8P-) zQa_}AN5o@x~Y;)dFh?%JevHC!obVLQ4P01zGS9t zWsw8vm`Zd{N z?!G4d1rLHm0yCH~+iP6SfKl4mD-EMP;9xs?Z|rhBV`4Y;Vl-WSDPq_M*+SKvY|NO7 zJg(cON;$3ASW$(esy5zOvFQ!S5tRHIt=Qc%X_g$)ksQ5+EZzq zZjRjaa9dpC_#^+QUCiGrEm`O;7HzbLsAU2;XVrU(NIWq69Xu=B`zDR$@2|J*0$}e> zc7aB%PVLv)KZ}UruCKkKq)To|!?b#C>Dk772}S*^OkJo3@%=VSXoK}c}cJ##v$Y-jT!HA_#tu8=u? z6$9gPU;w?4RWx=_%S;-!MbGAIcu$*%&8GJCjzG%%#^cGKAW-=ZU~~J@)?TXz1Tbc= zt#>Ol@Zj8vsu7q>No2ZG3K_r+zh@R^Eh=M0wkhK&9AVki-Eos^-Oi{~jj!T!JiNeUNzcEH zh>)2k3~SQHj$YRFoUI*851dR>nd}?0I&5KgE=Vg{l*jq>sYTlm!my1 z)+WZ4@vkrm=IWx+vHq&7N1^eWFZxX>ZfU~D z_^d<(+tXho^sTp6neWWrMV1D>rJgF|D&xv744M<@=OBdH_#y{TiIg}#62Z5B4lUuU zqD7*T7nHhSee>7BLtdddlYu}=Dy$ir_Fs2q*HFB%T`}>DK4q~C}2MZr$l_&h9Rg!=pMXi z$5FEdXkM2f0P6t0uy->)`UiIR`Q0k2aiWcCrfMpi%Hp7?QmS!`rxm)&7ArV^?y~Bj zyCluj>0O+o6*w==t!ZZTuxphrheRtG8lEv11zcQBr8ZnS8pppawfriE{tH^vzLyxV z+o|vAI~#MU7)voT^j1M#Rh71o%5+vCT1u$C zMR>vPf*?V>7f_hZas=xeZ#9k$4!DV^KBX<#$d2bvn6+G`Wk^RYZ<)-;tIm-b!5&|g z*hGWiccgb&(&S}pF}jtgN4GFd$$^>0-&fa1hWzZ+u`#}Czq8u9Atj|^>mbbg-Vwli zkgYoiUo|)laMsL~zF>El(*$$-IhHG;+n7L|$qilnvjQ``NDdS%&O`#fa?7>j& zM3t+da>^$!wxu6upUf4g;z{2h!C;?RT-SaDY@kD0HD9&X{V&{Y>Q<=gkhN{O6e6`rw{YwWQ zqAZB2OMd9^k8bxXLxReq8EIzVs%OK0_)qi@;1O9khAjpQsAEDb?7~Kvke4X zYClFi1oS+8TOJo@+$xH^8I`$IN(;~%Gxnt!-`CPW+#FPX2c1j-dZv7jTC^#^h-aSJ z8Hw?d+N~j^sNRzNtcA-g(SJ&_^pn!IwoL4GhgtiYNc-gs~X>I$p_Jq?-~V~l&a z2a$Z__Qf$h<^k!<5d?2I&D#oFZ3F()z#wfono{ld+MWIFuRD8p9z3;OoTah*lCCm_ z`)``ow@i?CqV$%4QOdsb+KOp(hi@XQ31pF-V zn9OB*$pa=3{Ryn$_Ug(QV`W2-t)XQM{xarF22!hj4yn}u6edlDAu}LltqE8gp3J*d z>@je_P^XNosdRzT%S84Z6~VIRcx%gfGegP!uCAP?`r^GLr<9SE6;F3!yp>|3lxtXq z6P?9#AI{Ba+Zg48IBVLXweBGptX@Jr?xm;l72sn*gNj^S_#g7kIM4Njx8QiijNz!y zh@$ssQ{qHG&fM84yUcb(OG5>gZ5WYyTY8^dXp&^M_o#F_O&>s$Y_t3#M@;!Zr>nUNiTs%@f|owDvcGfNCwi5x`Z1#3N&aE$V3y#w`E6@QUW<55 zo0yTYg@L&3r@{ID@)Qqkn4RzfRI&N?V(0=N5=DFRb{`FD&ZR6^N`*($>8w5mHa0lB_C!Lzr=#= z^=@kbm5Nb#o6=5h^lyL66?QRI6E9u8R5e*iNoApb!6di~|EqW(x9}E<%-FB4fQ%g- zO8twzFZcYmkx2SfFW;h#bHG0j7o$zTgRc%(06Ed=p(8spY*ZSi4>Q0k&+v~!Y&MlA zlu={Jw;9y^Iz85+)MS_TyhIPmy-=x4t+>g=a7Sq^^_q05S26p7>+mnDLVCDGHm-Do zoy1Is_9H{bc3b*A!VLE?a_90`L+?WHPq^d_U5pVdspDi54hI?7wBeN7*-)-QWrEIv zK)}o)V%Q6BgK%DS`dHQFhOIq@LEYzI!>c8NL#@ESDUTOtXQyJ(JeS=zml!TZSFvH@ z@LJZc+UBmtM+Nj!C$Y}q<&B8f^@kljRt5n|Jb@cm>Yb1R+3Qo)2KER834GNg1}`pq z4lf;!C_Hjj-N}94HQO5ZEM-_4Df=cnz>f9uOLEXb%GwGY_cXyMZ{3q>aP-YJ=}=c^ z@OZl&&kIrjf|r1s$q57?WAq_4T&2EE{)NnzLu3DCI+*RxSIir{)`W`M2)6(ZccjO_ z14GtHBgnRhq^pfn*o_djtL{_MjhL*dn-z^t0iGNr{gfNO!?TSoBwTiEQOMXjAXADT zAwVhVsB;`oGnIPE4Ky$#ylq~K5zWr9rk$LGlXczTs-TP$Opzal6@_-osyi7X6Uclv zfwqa}jU}@U)k>8-#G&kG8*s?lCCa`QDi4(=KTo*|3XhWzL~O?`KlZYl%{{XY_@Y|x zsPH7idQb4m6ktt;GJ8%V+_fnJM8x3;E=F+h1tCK{O_g2+ohCt86w)KHAfkt3!9 zHoW;;J;=~*s8f3}d;PqL!aYfQz*tTKi9ykRE4uMbQBlqAvfX%?yCC0eKYrM?5e+Wb zweWTfE*SR;KW>C1F<WPj`Uj&c=cokGR{^BgI8lf`7c#ziqdSn zocKE06%2xSY>-i`l-}invQ3cL{g2$`KO83p=2sZ*@%}%O2)<-5e4II_4%-EovG*qK zxN>#}Uq(#w-F?Dq$(T)piOhPASQ{)YBP3f{&>Ah1hJ}-K65#S#w!of|7KeOHU zZBMna3^A8Ai`7b#et5S6W*=abmd^3iho^%~mEXc%y<4O7VBIbw9GI?mHar{aTs;OB ztniR7LtkY2LG1~DueAt6HsRV{*+~V^(KEV}H`RYTLgw$^aDW!>#C947I$+wHz;$fZ zW2(s=;9VP&@llTte86ODF=ep>zPX^flgGddo=53MB}p_G{ed1}){#moX$`{fG`KL! zpnHM@&`3Z0m3NyD!@^WP0}!3Bl33sz|D#un)ZrYzpjVC3{pmi#ci!|rk?t~fKZqoW z+%Iq3qrMF7u0u2AMcSPP6aV&AUYW9`fLV6p1LaQZ*q?jK2HBkkI@P(C1w}QS7qtBJ zyU2yu4C;R6L;DLrrVa`8<j1l4*Nx1V|=}CCY#;O z2?zi_cRi+l_FZEcUHbYntvTZ79uwW78uO!1S$NH|? zFe<@_yvBqViubbK3FwSVbA~GZnA7*7GiqpHiJ$(^jl9)?2?}2u@l=d|d+B4G!)JO< zdS4IdrogcM%7C2qSQVi+q86fR4^kcDFZ??C#@o~rtF41?W*qWu^1 zFoC@x$QyS$()krhWiD+;p~C%+a)Vl7kO^S?S-BLvW({hA7|h+X5E`>@r{JO_o5Z1HG-465yA}M*1}y5T->&PEa2G@ z&r{`k#{Ji_$T56#Z4jG^7o`KzaF;vLl;*{}k9!OL;F6CkWVk!c`OTVsZY$(HCa!(Y z=2w`AD9WMLNSs!P6&OXfOLB|czH-0Gk!~n9#!X#FJ59DrLQDOSV!tTpcVkE(cl>Pe zzMkoz>N2zKPYbEo;!fx?!ipghHxM)I;MMBx_3F&fDA~#@>KT5O*&;DbPwODT8}*UL z6nP{5LA(m2k@Y#@OI~+VjrLT)HRpXT!cWZWIPg!!yd(;oF-%Vcj(24s#Mek;S=E%=og z+$-YlHRK1g)eGscB}nO`2fTZdljjgW4Moa?$4P!=rV^Dje1!`b{jzIo#W-sPCRw_wL31@J*EMO=*H=@ClY z3IE2wnI#&qv_1DS+xkB#H>qbf!Wx(1I2Mvg@NvJ#a;>?K;DrnEhYDp&3pIlgE3-?~ z+zjr#D>a4Mlmj{ug{e}&zH{3nPG7T&WXMinEL1CZ3o( zN7JwpxZ(1SfmlxL0NZog-?OY$!?Fn-ePd>1#?p7WQlIzOqI%2^%yVivHOoG;;r9L_UMnUd&ID zt*(x1j0w0y3b|(z%18Pgs_Ht!{un+|)d0TfaE6mfk^}!d0)CmYbktO$WCu0y;F47y zq)=p8N%nhtKI8*KaeStDD?X~@J+Pygm*TzNcS|8kUEF8rygGDh1seJ|l<$KK4O|EU z&9ltCE}S`K`&P1Z-q6+o!d zP9iz-x$s6A07lsT{IY~oaNO1+ALEFyT|U~S<_fpxr{Se6dt7!_rN#J8$qf5h_Faq> ze=TDgIR1zp;}-iGO(;9zzLtWm4@FQy=J)x`O;vYpIQxH_l7Qt$4<; zSj2sXHH7DMmzfZU8F71s?>!@?Gr#Wyo5amM--j_yfZRi`!eRRv3i}zZSM&}+y%xl0 z5TewRD4hUL^;Ko-(B=%AuPs8h@nPxQ*wCCELTO3RWO0A_6{bQkCXt>e6x%z_7!wCU zfa0Dzn0i=?a(In$7{^zRxA=sn2H)mBSHL{v=8()xg9cZ=$E8=b0%8N)w>kXx9Tq#Q zU%r6}_*YZ4bk8@|y1bFu(=_&oIrhj=@5|HOFogT~MDSZ=w{(8eivOLbFv`$~8Xg96 zr7vz!%axYgIb0N@J~nQU$CVcG5G?}FIFC32a)%ZJm-di)|A|jw%OOfw?uKt;J@%mK zi4A-c7!rl6l(HEYk$IEb0fq{S-e`?u*y9)kkgA_yp*D2v`GWN>l2&KuA>N zSTr{NGA&RN9cM_naV$!eRHbpdIh@i~#Qw9z*Bt5AA_FKS(g|txrP{C>GTx=ypckC( z;mY9ed*`UM=I-Gx2m(8K!Df$A!q73SuYq1TQmONVh2aro2qqKek5Kaf4)wCWegMcP zfT>I0iBBzzOIv2ZF*srIEYo1$sBw)K3pH7rbtq&Xi+e}-ro&T19@Cj6C!78-nq_px zL#jP-<*$Z=Tqm{lHP?}crrBZmG}O6nOpY7+A>W-qI9tra`k9Tb{FwXaV=d6`=j(@w zE#-24zaMgCAEF4krs{6s{kvoSH>B8{6l+P$48ii`Ecp{Mz7FubSX=y*Ci_Qa2D|D! z=k#~1-4@rgxVMB(=`dK53%Wk?xm(8oZ4TaZbUY1V9wduhOmxl|SX_l?c8GM%rACV$ zT*N25#`_gx-7t8~N-;zmX3&ZUHCj-oNtUH&`zxYwQ)5a?tN|u9aX2r1JJDiyt;$daZA_VvA$-qjy%HNw`o|E%MM-dL9*2 zQ=#uk9!@IC#KhUQ*@hsuqxa{^<@)hE*U$0I6HXg@vL^tU(T@Ep7xxy0==Mwwq16LS znjAAry`8o)z;?)MoUjHWY8&$0k)ol|{|VF3z4o zhSnw82#o>)+L2aU&W-amzu7gpt`tR4+R02na5w6CZFBJWchG{H_-(5GmcIGn9O) z%)j7pej4AP=I84F8+2g)P{F$Vi$eiZkzq zFwX&^X>fvH24QvW|d98MN`%HE-C`vMrvCqJ9NP>@}h&K?(2op z>*=QBrb@RAp#oy^;V{520V!`;I3cg);ZB2%&X${-mpg>qsNjh92o+2#dUjsMR)TQ# z%G}5zILk{cjPxp=-~v-O!XbZg;&Z1>9P|gC8yT9C#Qg*6czoysKz(p5n-Oi=S)N(B zgE|@Wh%t6w>(84QEXpA8)rttj2U`tNvL zpHO&wW~q33P$D+I;)(^SDULTS{fcWhOE~x}EpDNlvg1{AF29ZyWRd`W?Os(vQ716B zc{(els4RqqPsAxw5OKgb{As(O0&8Q+tFZd^<0$LXn}2yi97CxQ{)Te9XaOpuncY9s zskO-aq+=;ly*J4J{P=S$Gz7*ETC4T{BeqgRenS`nHcD$^-+ZUolWWmYXb^1@W=L%y zhCW=QZ6qLNBXkltQ&R|a4b8!@M4`+=a}xCwuZgb_Gwi|^#SA$w1>EZiCcRs(QvBu@ zt`{q253Y8qz}wr&#V4^kqI~L|uuyDdHsl84{Vx;n>fbJvd-Ff3^v3@(`CI{!{k0Tc z{X`TR$Z=$v7&N1TZA}tlz@+*kC%DrA;+G_e;<3=~CBzig-kU!Zj*fu9Btaqgv2Cc` z-}0y_g#AFK%t&sK9wCmEbcy9gR=JfI=;BSk?d46h4dYF<&DCu%vL9^4&{SNkp59e3 zVb{%itM5;gjjL;bHIc=40@pK#&^lz_6Y&NBimn5ykkgHB+`rL4c@5f~zw-ua`+&OInv z-fM!)zZ4fO$mFC5W-idpvj>ELMn}yPD0#G?!Gk!nlWeXW2zFuO`iBDBdakq)oU+3} z_SVd9fW{=*`1*rHrl2Z~wDe+|`0>!#f}u;faz!bKYSF^IoyjJcig&GPA=E_h#v4p? zJ(JkAe?!A`dzfs5s<@yvz%Sh1mBn;_cy@;MMLNN{q?hn#xTuf~W{XVBc7I&EjVl2M zZ^Sl=+M1h%hcWlZ95Ffd@f&A(ZkZ~81I*}bt=aq*e8R1(Z#~%&Tyo{Frh61T&S7&< zd#;pSayDzyuJ-rIEbFV4M;*zu$i zQ?Zph#Fit5$wIRtQReK2fh0pfpf=~gB9`v1>bsdX`ClBhK)>Iu1UO=dOy7o&`Kea= zM|z2D$?fq?Q(Lr z-$OHtNlN+>G)##9dV&BUZ`3OvJa7g{cXuuC^pSX@pVdt04G$X0RK~!FIr?T^%JI^_ z;W#zsM%-Jz+*_?f+<^g>Z`VPOpZQk>py&k)Z7Gpk(UMU|^rYUQ>j+YWQ7!q1C>A|q z{uC@FFe{z-|HX?f=wDcJAwPZUfcd`}G(}+zLKM(+ z^VS}`%E3=+;pRgZ!1P9QYL!eT8;7>u`oys^#2laT$N2c@mK2|VeN@dDTL>m(qpnUC zNnhwI?B%G~FUe2XG8bJTI8s6qLI^!+YUdl06e-@mz~PIlnVG4Z*~j(JxZ8uo5AkXD zii65FX@{MPUDBZg4M^NUq&iRB0br_K%{3tEKC+9gREIXn}~_FPIl)Bg(2R(Za|t;m->o1EJeCg2-rzNsfl-Q z8sH(@AiI)BxrlSG8ITKmko5ksV;c4~OpDYu=`nY}HVjG9TWg00Wi#n9ZeW_sm+Fcp zOi%h%Y^Md~AN|#Ll%9k~`LKWSk9J}I;vNGAzLUL^Ua3K;QDgmNEZ3LmYxO^at5&QP z?puV*uH2%6dzb4|Lg$t3n+MQ&h5K00d8PXr(0RrCoY1ukwl2{B!@L)EBV zLPOapUy?)Fsa_(%ZRBrN!EF?8eS~V4F>o7sTfog0tpg_*8Ki_&%q(M+I;Akk7+^pS z{t2SNs&*BO!0KZR&r8fpHXr~qfj$RY>3q{qP8e{+to){n`DKt42o`819B5>0X$~vE zw!})u*2J!M>y!GxcYg#2U?G|=ANo^9hz`2YhdNQw4>976+v93$T+}8e8>~ zQV;zRRXxi?+qYPeP0gM9j%0XrDjoW;^ZgBY!EvA)Yz#(P1N9+=X?qB87zhCr1tJPo z0pWuvSTg%Bd4d5YY&j#&tZh&*3-**DXT~;OuoJfLv^^7;43xsGJ!KCCP6DYhRhqTN zZJT0N$zX%O1h0T(K^$#c*4bNJhL!MBf9KlrGq)LoErYp0@Yo-YWmv1&Q)aE92KZn$ z5J#&%f|Uk|DvS1(|7L^(J*WJHVe25&L4*{hHsvuE2GFr($P~ZLjHyj5g=J1fZvvGP zCX9tR8L}v3rk;jG5i$B?AD%k?%{Y{40DEa4d@&&TWM9n5@SXk}axnbOHgR3zwmWnW zwJ*9BHi5|_3)l)|$JjY|Nd?9PZ8CaS82Y(u4GLYgg^TWZp+^k9P$PZ=lzx(E;87A7 zx<_tf1L!aj|4KQPc}sVE4ROId)X};04y0i7MCxSy>hL0Me41>9I*0}pylMMy;66|S z7NW@|Gu{+FzF&E`_%&m3s6ZKK7W#xf6wBZVHNl|H6f*Gh%G@C)oNdPg-D(h?ddg(q z!!RQJW>ANkerrzXDnHysTZdFv(V@;=V~~s%XjcAHwZlxm$U=*`PEA0Pt#qY+6&^ku zR-(+S`=@5->mU34L&e$Qx8ofnbV$RFaEdT_We{_f%AXQLSakGhRHRb(&kwG7kBWm$ z#Cbd1bsB?K?&c?hX4D&W@8vsh`Qn50+5O-h9rW+W|5P0^9y`Kgu5bTMTw-2@hQ~M@ zeHhD93bxoqQl#A{K1vPZYkQ_|V+WgIXB+uUI~_q?F`(=1%}MVVpzC48 zR6N32`36}a3^z@E#C25;F8!w%c$mFyoQB>J{tn+JBYC8CCY zq+8C3!u!b zh2h+Ys&O{O&|HNG%chBAEE?7QDCJ6L&(t7`@=RbLh*Y_1b>kZ2=uIWjSI zm%C9OVujaDp>XiPRQ- zifqaMCy%gjm>QFz@?5%{Xzp7E8W02JpS0L0(XGxYgaA2*W9eh#WA4KJ(A^_?QhI8R zpxXRI{F4u*sxqV5y89<-^eoRos;DkTbS3#v*Z@{EtJ?GGa%GZnjugO|uePdMP8y=M z^FazB2zF89g+D>$8N+n#MSdP45zPK;tyjuq&KPf!g4L?syES(lX}U&%3ix<-mW|HD z%2r7)e><`QH(qs_`y@SR%piAQ5Df||X)oqAyGN!w>s8FMwl~%Y8{fuV>oDG@?{CSR zN8eb~1>yHaQYpcTG>*dwK1IpTj98b$)D@LNmge&hBJ#1GF+KBvwA}67Inp3!A3LU3NClj?wQvM~glX5sNcs(o9AxQ`ly%k9W=Lgm}!U zEN4d(e*0TlPWPA!ZnJ)1;*c{8Gt-rVf8r|>s~E2}tIJG}B2jar+;n0C!JIb8!UuUB zMNS^Ylp5_B`3vJCfUiaZmvB$9D`K}OQnP~;BUq;V^d{Y8p7=CBS;8It`@4l8yQpl| zxa%00TC8Z+5>IZISJrEM39FvObC$U#cPV0=aq6k4OQtR)?(e_(LGtADr_Y9G{W6YK z+e7ZLbM>hV7$qW8kDPxrL7};mh#Q9u?SorD$CC-spL97SxyiBXORHN$iv75rIeG zZzHY2c}q<&s?FEofT(QftaA?8Mn-Mx{KW#Ksw|3prDI{$-?&hC=Xa?~S0Ys&=V7;f zx-mYU=V9vS0P=t69gSbicZsnzDgr9M8l}*U!>HR86o2F(n}d=Dcnd3LcW)9|;|34Y zv9A&j^X`9;;_dKghYg!)@pwSmL8bO6Vl2)K>U@(|1LuYP)Q^57ac6eZ^|#9DuwX=r8_ES5SFhi@ow; zr)`C>%9rZa?Z7K`O)Di>SW{P5PWG*^iF$|lyRc2SI{aiXX_}!+6$g1&R`Dc)2yp<6 zcVFo!pkXyow4gwBwiP^Md2mRjEo%7D?poy}wQo7H!~cV3GCxaTolE0ExQ8@^YvJ2% zTGQme|7ThFFnScV#ob(=HqQA+dgrRXu5$9Dzk}tpP3WW0J4SnxnP);?->s(x(bt#WxOmTH>X=y1DKW%|A=9OrA&(_Hd4PM178{WqU6nD9rIjP>R-y;{ zrY!YBUds{S6;h2>DRPt5TKCK%Le(fbN}jzrb<5C9_q#y|0d1a0iueAYC|V*gjGwLt z$fCf-h2PQ2HYUNp>Mt8T$8Mk4v!Xw(?yFA1tu%=RQcF?9JE|S?N%qbt7pcaGmAyQn zeUBtBVzNHvl7hSY1VuM*GU|kYBqutz|D%K8S^@2j6jK0w4sKM(yd%!ns$w72Ky3Lt zM~04ahQ*Czf)5#!zY%#~6ytc(Qyllkb%ua+ z<1O8rbwO;yw2$$Op2yxJop@Vy)nG+d3~S7CsVsz8$SY!k1J7L`|279IwyUk{VXiIJ zygfxD&7dQZmD;9KBX7q#{A#a7`M9gxLuOuha$>Z=k>Ij;YEO=ldcQpmzf_0@5aY|( zR$)4>B+HSGv!^{pf|>NAO|?2z<3MLAaVU-jTa`AOyJXl_%a)_C-DwC~#{N7_a+sCS z^qy_=guy225Pe{~rkL>M%bgY!?a64xcXS-NrK;wXi|PdPp1FftjU%QXcH9|m~eJP zVk>95PL?8hy~QqwQoo>dOE)sS@JlNZO3B-rw-oBs&mFtqA>FTx zjyIUhXZ3t6Xy(~!Z0 zCw`w(Qwcc?7MBD?#abd(n1nSzw zEPipQUnW+B<77y5aJG39pC3-CFN#Hc9~c55hEDInL}&3boudG1*@F}rpzUD1Fe_NioTe}uA)%g@QRS{2`InA*UfM^dRRhdLPp?e zcv#|>d}Ss(OPpe*x}aBS6vREXT<_^F<+o!e=q4_I|HcQxEWrh+qrx-P!nXwUZI@C) z=hHvRu0t)8_LM;DYAZd#4-1?u!9pI6{Fb>#N_DZh7=XlYF%sS>(=3~KUp&*JD3Vku zfVX3WC@gb%TWV0bN~*{D6m#7}^=~-0WATpi;M72bA}P)1?@JML2bO#1gIk@XTx*^O zj>1YdT~3We(Z!8~r;=>9QkBI@^{XSnr#WORPB_x5iljgj=zP$SA#y2%K(e6#TZT$a zA`V0R&Li%J)2|7p&{7tqum^9)iXR~s&IeZC+L(V%;KjQMkb`?^*Qh zF^VJyu;ABLROD!_NyUd+f-}s&PDEI-qiaPW9V|#q$aO@;J9Hz3g{TJhS5O+)%9Cy` ze2J7|SJ#?IXrC+5ogO($@b23DJ92i`yb)s??aKw+*t3IbapUD}(9$TG=lj1+<-cHb z#E>xW4Jyg0pyX{}rtced$!z>rgo*)2~<-@BI^AWtjWxjxiiA;0rID&4zWdqghHv zLrXZ3MB!3kuWV1(PxfZ`_iUVbb7oNQQ!EnZFI@Tps5U(EK6FNe>cByJ^e88~C@1_h z;7Y5mnL3ng};c;r20;(n7oP?ljQvc((5a0 zom_>4Iq25@r$y(PHd(C6;4O(%5QQHEfND^Fmjz9FO)Gh=9(zQ2Tcl3*QIyfF1rr52QJ z~rmQSycnO;?Hm&$ifH9Y!o zb*}16tiEEGk(ba7RaOV~mFBmBaw}G)sQNYz@=w1E$qx090-GaJ5D~-xp+(vH@$NsgXm{H)vqO3m3&FdtF+^7Ho|Dz6_yUPN5lX zo&J!%iSNsK-=UNC-NuvA3dG~s0;aaHlT|JWzLBF+td znqTLNNFRE9ZlfA(Azqmpx5XOy6ki=g2=nVU=%#abb7q7mhpUyyl&!YeyE0#B8vV=i z>06lU$_i%C9##L6DqixjaK;r$VFLfY{6qYfj?_|n!o!t6IY)`omcgWqMDk{= zn#+)dF+x7v4wV)*8`CTZsMQ?a8Gbfn@MPbMqQ*yP=^AhnQ!P(Zv_+p%HxmIpO>@oj zDrv}kxo32TOC6ORvIf8?yNOc#0X;HQ{FVBcAaScorrCKXr|Y#Wivll4Kstz`tG`ln zY(d``Hi6$eg*ElkF@-glYNpMK#aT<>;y8y;Oy?}Thv%{~`L5@K?jahWLWQT7duEG^ zmFvnvVn;c1_f*F!I;N-CEve96W(Pff+W*m7T)hwD@>}? zv{-E;bCP4?xU-DFapZs6|LMpHddu&PNf0c#Tvl~ng&nrXrKMq_Ct>?HVh z99{e|H3@>EDD=W;+8^7ixGK)ILy~mC2i~~nSQbn{sRKaXzeUZ0=J~$D-VQe3Rq;?y z8gMou=4G%80bmSf-xAY@rp_WTm~VmAn(zVF9z%Q3t$@|ipzOj8!Prwz-KNZg+tcXX z#@usuRbKi%C#CNBv~ze9O7w=_+r5Oj9go~&yo89qhJV2EPd)USdluhvML$J$LBBBD z(|aekg7!>ThT2n6tPEN&cN+rhnYS_eIp5Q8J+LfX0d|zjeTfhwfJc#n6=@?RKYra+ zk$tg{^?6&bkCyUmA+~IGCe0X7>;3$h=_gPXYMA7-(*_jLL8PcKl-aSnZ2QFwc;$2R0DJI<-WStyir8Fh8c@pM`>g~=4efAG;W039p%maT8MUNm=!UZbbmu!QR+2h9r z$*hNU+tu{In%IM@@*{BQ8Xblz_A$pHAuBjoJGF8 z?@|w6IT+0;Vl$k+uarfm-?z5!nk7P#0U+OQY}r3LJn2HaNL*$nr}TRld9jr( zxmxYqbB2rB3a0nWA0=0B=_R&w)$UB~Wynhz7*|+I58AdnkK0`+2gDr6&^&~gA#saA za!vm!2ke$!ElfW-%%ES4pir;_YAS8WIYW@%p^h(58c_atMu<7IA^TMOlDu-0gDj(s zGpH21TK%!+kq-{hZLl+iIS)}IEg%1SxcY-KCI@2w__4zG|F^bUzjN&U|7)P_;PbCw z398lM4cO?v-r4{0+x|z{)C%$m9QxnI_-7D;|JX>kcK!v{0{u5S5s?%8+nM_gfvVL2 z6eQ@si*ae+JHHp9QyK3OXjA#1Kp+7k0&cb#lsl?G8mbmDXT+4!k$#f0vOnN3k?~Wv z@@f6I%-mr1c3v<%sDw~{%6%ve^VnMAl!1DjrvJLhYIHu?KKp!l9%1;w@{%X4aVIM# z9-MC^-lqe0gGx&|N<&Ghsd~#FLK$MDXvIiNM73uL%fuT_6jT4*I!Ix}9w7llS)}#L zz%{1DF_s&`Gq83IJHy!*u`w7zQNht*n$NmqrnXtXcbuHVZgpHa?qtJ08ne{FQ_kl~ z*Aiqnl8=zX|eiXM1II(EZGOPjQg; zr-*8{x@)t}u4W4e=CQJNx0@SFL|MbZ3!~?*L7`;|67XU##joa}-=QVAalD(_m!I<{Bm;yY+O zO9^doWg~00<>qOj%n!G8H`KjQNE~Ca$v8zCmW>9ISFgE+a9-(lx0w`buuW)AK03PmG#{|pz)E}Q;Ag~ixpbfLbTiDZ#&pQm8CFiJWjlPfr4pYCW z3>86Y)z|Cf2PmO4C|Lnp!jFYaqWURSO4?Y}@snWOGh+s6ZB%q4A}91;G9h=Pbvf}@7|*A|L2s98|j!qQTXZYeI-F8}AaL_BES9HO=$ zb+sBtr~5#EL*_MvO4i*J7Qvt9+MdUyX~{abkh?L6zmU0Ecfad$I-!Ed^M?3e?l-zm za`>Lz`LEX=Hhv1+{Ahz^2%1CWhc~1O>J%yysv~S7loqN2uSX9;0t|W~n(_2YI<}4P zbu((;xnbI>VIIcqG9rtIO|0L+hfd(uIJ6;^%xuFIf;hrU!KZRc;)ZbPo(O_(3wD4A z!8*2)7x+oR6NG7s%MbF&I;NpJ5;PL?4+Vrlq#@m5t-RxEA^qGHL1Jbmj=)8g4b$xF zg2X!@ru8N*h+;xw2iz0mZX)gNm-v09q0)FtYv#6O-b2}RyFagos^=0|x zycqbZZO4&`u*=NrIwn(HTNHZqHM=&7?r-bWpPYvzIj7TgrpV#Mr?TI73`EaD$?$KZ zFd`9j<+ffXkfxfN21~sSmlSETCoa=1v7{9-OVs~i)=DgK1Ekx@UqOwnr+d_-s|+u& z4!f(|nf3h5!{?LC9`!}rIT2rGhJ_BbOD~LGoCwjFmy=jGXc2jzSN_9A(^+K5QU0F8 zopL1|oPms|`EHhzY#GSEe39RMr)eVd}vDw$uU{f5gnhwsrP zvmBMHeG(4Zg2Fwixyv-U+$OmLqH__fsiNvMSu*kQq^ZgV6ZI6?x%5d1*&&5AntmCC zh_H>A2kdDSZ8o2$)mFwZW*|K1QTu`vfZfU9x)!*B?z(ooEmQ3UsF`Z6SOM->!SO)a z@;h_J0`PlLLB@R*x8XR`?(Wm*K>)3R^G#5lsU1a`SLiA1LibGI_U>CO=w6mbw z$_prKrY{sLTi5MFn568}rIt~z;~n3VI_U#R{7n#5k}r8O}~rXaff)bLi@`_8HTSkw=g+Gn&@8>L2retx8bKAb_rE=19TniwflwRA5mH_j7etsw;Iq5`hSkL?}AU3ntT6UkzpSY zU;)J%Ix46dh+i?ZlMOt1^d5Z|6+)@Orb95;x5z^>>g|+ypmGHR(}q}@X3ObXsB%Wv z8qw_(391$M2Cw@f%1TL^B~{YN2%eKOqp@LUEG)EjGPHyYKGSClJ{|8D{N`R?A1AFl zKsG}wzK^f*NpT4dYWv#|U?@D{3_q66S4nw8~)tobB|=kRn9P5RIRoUAt*g zj!uol%y2F0Qw?}E_^$%;Cl_Y(Ye^+KcSv^7ar>oP^>X!6{VoNTIhB0@qyQ-Q(42BU zofLGNXz%E0yD4Vb4Hd~#y(S0Ui`>dvfPc#3qzQX<+8w$*GZ>u}N%+^%Fn%&&YhIkfG{~>5aw#pUq44xA{S>gr737ofy8aEt5!G zA#&H;ysrA1Qkl9+5(C0nm4Lkx#(@xFswrr5S&}iji9{{#&qFUHIFY=G(F`eHQ4}KP zfDXg$TOnhnG`ogMbYD;Qb^~;=&d?km4}T@6N3ph2Y!~+Cgb)v zc0h_O8uFhQWuMUD`oU(AqA&+hi@lr&X!dfpNYO%!=UMV7K7e5L5hYQc-nUbqsxb5| z5z|MakM8ICWBinGSFxri{FJ}K#IUux>PXhFpjKGJ_3RnZ$ zN)by*aAsx1`2iy`Iq}gZq8>kqvtZ@8upHV>Ad??RbA{enJlwGIx*-pie6ym|xw?$lI2%Ir-MJ4JfWC*Qd$ z1q_*GAx1_UZ;>QL8-E(3vb~-M3ELo<`OQhjB(_eviiC0H*r`rM+BCy%*y3%XQ4_6ia- zQl9`^S(z;e#|Mv6#y~t1XA5YOF9t`bIjDPA! zW+pN`IQWkr^Kk!DKmJW+`UlY|V1WGj7y20teZMZ#KO%sqjyxccwWeZ%@c(xRW@qHz zZ_oh6>N@TyYN%iGiDpje`jnvDYSPV;3m1N=0aU^=I)t+!G8QsIb7O~T@|n|v=~A}6 zos@6;i445wFtr^iErT`1J#Cs3!2-SRfBpF*-({h@wyq4YgHgzc!=JIB~!NEk)_D1q2(0E!#~$%!rm z$pGb{)k^~m4~-d37$pK60tzcJ>!$(MMScK14J`?khJM=t8zY5fe-Pj&!`#qDX$%fH z0h1kNj7wsQWWsVbN*-v7^2~}i+|PoQ5yXeW63a?)M58eEj-bL?$sc=xr7-c1QGxBz zG!HP8sB*SZ*O^_+GBaCZIn%1?1cBwbiG-<%S3#j~Wm{>GnrjkbGUM#`+5k6l~1Wg)0Ly3uZ^C8luIw7*3e zno+c0VWxQIoJjQrpTvjqJ+uoiGvcXn*mTD&*tmMQGW9=fPCBSCl<2M#%%F$w3c3aa zJF&X5Rip-CBuR6f?U998&??mz_$DMO$HSfWN6H$5Au-Pro=C<|yhSp9R@Hj~V9TOf z0c3iY&e)u&+78-5;c-g#@blTRTbQ%*4Wnoif?ru%87$6lrMz;(5;2#Hd=^F_r1u48Jr4pgX)If8WVs6=6+D-HSzZk2sLy_NYDP)qAxv zHPyH_kN&MPO)J9DTWntmfsM}mi5PNaEyroHp{?XXp;wWtop?PRYOe|i(Yw&u$T~<* z%wS=uIl?;UVCPe_lH0`sJ@#NClfLBW-jY^-^)WmJbY!M_} zH!rcXl&F6j%-@t8o1FRvpqs6}bGkze>=te>i!PJSRA&sPGAMcmP6y8Vj5#ihj*69AY_**h>9GN9wQ9KIXAp_tXJ_|H?}GEiy%R0fL@sDN0&nHsI3yn=;rKE; zQ|c?<;m_4~GwqBfR$pZ_o@2v&Z(e)E+7iXl@&Z?lSZiJ81?QvnPV>(W!Rb@iR##Z_ zS!Wx~!TQcjn@BYjWoC}ses>0)obl*YE~d;*JD{WvSuqUM#+3pN{KA<%L}lWsB9+|w$!1Kgqt5Ju)=QAnkBdF2hEOUK-zN@_{b9nn`D;D5&-Gsy1R z%QA;hsmDfEVWCu^rD~SCvt!+CF8uvZEL-DSH^Bwy50oDh|NAcfpU67DM+D+j6#|gs z|D2_*e1ssx|E1Xw^}i2K-aiPmt&PMWJpVmW|K;AE5D-(F{~#a&qE&4b78Fr^rCVKT zz=km)iD3~T)u`HZm)TgTYjH$3 z4b=A$q5TasV(Y9tx_tCpZXchHF`LVmuYJUTV|69_o2PXPkbJ(?^?-2$iFpCW+M2!U zEV8dLvcYKf8ydGn za1}>Ulp&j1B^UXnrxa-lFRb6FvB)>oyo?}i5^k~t#R&>y)xJ9p&GGcekV+FNlIqTL z<`kkru}8)YFxSqQUqyun!O)9)TsyWj&Al4Mc-q1Qw;c5Jg5@Q)5gv=En{!!iYQ(UR zlwcB0%hLmwdxZ8Y^A0fM@(ha0;ucoK=LvZEmX4YB(|e%_3YZQf_W#f>mxl(uog?OX+=`d|<9pjv1bfvAH_yM>)) zPqUBC<(%R=rurPqyz)V(?^!L8?d8?#9tYSS_kad? z!cK__AU5IdahX0tHUrFhaL9j5KTsZ0-XclfH0I1MNu}~>csqmlAor0pk$@aF} zg}AugJ>QA<@0|x9@9F1X)7`H($=*4N`GlLM>1YB606kbq3XhIN$aTVmWC? z;pu9SRc$(PZ~->01%kH`b!*T#R$yzCqs!i-tHhFe{LuCKalcG?!}tQem1UShSQ3bF za}d8_(cVEVQ<%c=u|~iE45UThl6a&*1#w&+Q_@+|118o{5ADaqylSJt-; zSG!JGGhv2PZ- zSg|6@ugTFYMO>1ZaEN=MLA$YCZLjUw3#_j#t{-pg2#3l|L9BPizWLZ*p}sHG%_m@< zD-~S%=OA*d9#_uyPh!(e4{D~3S4~y$l*?M{kRriRcdG3Hbh-P9;U1r)DGB9rAxU<< zz;_2J7J|lWWhC{#|1#hHPJ=G~OZ~lU}9(lQlYV@jj zL2j#xDTR&lV@N2WoHwt>!wI3JBW8GoWcW;V>|{ZkS(iLjCkMuRAm+nHciavsWY_Mt zyotIG^DD!Bhocn2IG*p$v_XR-#leq=8li^!5FSq?5Xg)1=Z zl9>GhVD0g9?n;1V>=6U&Uyh*T_QLb}@mU;IM2|M1Ou>ueONsR)6%H~4yv3Yv0l+JFwA2eKr6#Dz&ciM9(Z?nQSsvivbLOa}P@>!mhmr#sFQa9=@qo7@mXjm#o1b2BRJr&1u928l$<#1S{X176CJ#etFhU z>(cQV?l5h5YenfJfxSIs-;57wZD5ZiDSdcoz0lMpwWNrG^zbO8YqQWaC4tAKeISm9 zS36_^wp}Ha2d;gg(6nPXZo{>GD{g~Nd${BC(q+ALZK>B4f8zb}1;HbWprO_-50wBV zB2zO!!J{a*06Vw-MqotU+n?Jz9NZf)wEdpp;~p7(510R%;y&+QHFpEGjDR^1%Nd^Q z04?zF^ajSTuaPxm?}7FEVf1fr<_n7b9s}?vhtO9jB0{kht58@Y*R|}ns;~bJDlc*d zpGsMZg#{c18*)v4*^rq&Pq z&cIjge$edL`s;E_>Pw3cjqefSQ9`URTu4|HhKFV6l4Iu^yAjmd|8RiS0k6WTGwdw^ zB16H%BrhV=@B==CsRJhuR37r(C9!+e^->pD7WVoRr~^JVo?YE!eiz5{Y!UuVNINBo z(>HK{X#3Ev{c%zJQ z4a7~NYUH+6`vCoieBcg>)6)N*rq=x~{r?kWoSFmyiu=#sL6q9g00Il}dI&eA)-`{~;tU(o1`~8^FNAFcvsA`C>*6YSpY(xx9ScEUjaEm0;IGe+3#Pqj4qu z>{DO8zTUC8wG8GpojJxo@8|n-Y((HW=hrpgd%OGbd+1FEMlZ3IV!X;+>HZ8gzVcmN z1far0cIdY%Z}Im~)J!?RL;rhp48D@H09ln;rHJ?tM zT|6rryj?UK4n3u8(IQMig{o>1BTPZ#5+R%nJ*8%mBMgIDp=OaUOcabx)ufqE22H1C zkuE%erky{ViKbmXYpGA$yQELsm;9ZS*Cstc+od*O((#cSB;nO)+amc)dJYJJW)d2v`vEo?D@24pFSJV1)SlhAeg%3ge8Z{DP)k) zYw~z;n7TyOrm%a#EjPsBTG8w7l0ja0)}~Xc;wV^3{h_%_!96u1l&ya7i#t1J=d=m@ z_pBwJZ>b8NZ}Ebc;kF;aSrd}LVmbZ}a@)^RIbtW4YvM_`^>Bld1!(=O=B(~Mm*IL8 zygrjonZ{1QGkBx?CM2>2etwaaZyIB>mG1z3;|A=|l&FM-^#(mnh8A1K=4H;np%lnr zP%>0AdvH3dtzXjA>YXB|F_#(n*=_$J5{%B|BBpjDSpju=iVL@}*?5$jHqkP3lC1lx zXUbJdVoM^FS*8Gd3%UBT_EAR5)(~D$M1_Nr3a>1{ykx$@LGy!Ajz98_t_=Gkw?68k z@#-o?Q%@->dr4C($oDZ2!k9@oFhKuc$-vL&-FjB5Q6+M2hUPfJC~L^nH72q#O{-8C zSB_h~j0S6B*b^Vd)=@%Ey_Iecenhj}XN2k;4oTm~Wsfdf-YIqsy;Zk#5yO~2Vl*u| zhT9DgQAwqGVsh!Gf*dr{e*H4D7AM=Zfu}{U)RO1KgEMb7ay7Y4yiJ(^#1jGW%WY{i zhp|K~iz&p=d)=;SeaeQH#e&yRda3TIes1y*Z=%^1_qbE1bELC5Plg|Y9e#d8R0~o%@)j;VHqK)QlNl=}6*u(- zT4Q@}qODh?D0=F|jZ;!B8;+3z9aorv{9nU1vd{gpA{aDJhkx@O0P>F7V1=h`^hkVn@p7cD8b%flVu{jl!a z^QtSWxOPzXwFE7PYjIIpn{OAtl)VK26Yt})pjut&bqjXJSRUVwsP=MZ$3=NGp zNDoSr4VpWNj*}g*I5S!%Yz_F9Ph>dCW+f(>l5U#nTA9Y=1=-9FC{Zk&8L}p_)6Ub? zm^dhn{ae3YTSzGDy~CLrT^*MJQ7jVSO9Lv^1LKD%BJAxbDfI2*84pPat67%~ZRU4G zycn+cg|=oPrj@+hIt4>xg9GIJTW++L;AbB|%4T;q+uRH@nOro7kb8F98%rrs8*}+0 zI!`5oiuldkZaIq?DYFqd!5Hj`JX0=(l|4p{IKNt2SOT+atewTeNiO>V?myTQA3_{c z#|{!hJDfQ2Te&AavKQjfOfP;Fs8iP-7R}B4j2TBPr^*~%)l+x;@~%9IVEAG@HXq+5 zV@$iaTr&(k8z7(}DaG{*C$T54IV0*fW7!(R;OP^1%0!ef8uVc?y0)W^)Cq?VFJ&;i zf})tmnx^I7Np$Fz&4RT8G$=aL8mjt)4h_h}7M?;`>;G)sGjrCR;S}Ctx}tgdu^+kT zx@8v>JmR+FOz|ySo@)KprbA;O84kRp)4;6!GG=b18jBp5e*%Q#E~7&}#N3y}i8s*! z_>cnrUUae>UbpqcI1^6*w?;1Zu920~Rx)p4DOb_+b;;1gJXYHQ_FSs$eE*bIPp%vszijw$50#I@MMOq)UUIc0rE_H4%3ySed6JKzu{zKa z#Yx3XCyy7?6v|AY=?14C3MQua`*8Q!V|8v+;8n;e3}5Bt*sQOI4`~Lh{gBQZLr~_O zKjff#eo%UDL?!(Vux9!SGlq)ku3aOK;TU5v+D9=ip%Cn!F-e`YasYbijAx3;jerluhFUhJw%=&!=j)k z{4`J#R?fQb-80L?V1p(3<J$_yS z-pyZQKy0B8R`wus)h=Pa5s5Ib^#l)?nVdKoOc->W0FY|ljTq-wdkSKU5qOt9Z`}|W z2Riz}(6l!w-JqL%xZlD_D$F5({(hL9j%=*2M9p!v3;7+d871P(ye3Hr zO5!2Pfb4;EB#}Rz{WYe z>p>FeoD%4WOy+~4E&mB@oKFV(mt(mMOdG471t6ppqeUi*GfHzk5x2O};dg&%C?{jA0knI$Ru()Bm#%9Ta^@|jEU&-bW+k%+G@nOC;whGVf%zRj zFCY}>mX~7w_gv)P-AoUt&|7ATMH=t@%B}}S{Xry4iy{;yFu6P~RpBL7Y4Zat?od|d zF$wt6O!D8ZCy7BQv^$Gtj{cf<#T?7W7g>+%!s!f=n7F0%g{mEkc7>AyB#P(UcKg7V zJbWm1e@A0=4?Ot{PeWomgTH>T{6g+cIRn37;hG6+* z@tmDl>&0##v^x8N@rdv1b!unPuZk~khVjM}Tw;JOCq|~07gA!32R19#*cO;Gm<+%g zWOHG*D^R&5oEj?B-J4nP#y$44P?b@!KmxO1%9fPjBKuYPZbvZzVscc zqux};1Q^-6xXN>rg;S^VbFsNDS^Z4(^W5?z#RtG|F1r1WcpB}P!$^1}n;ju2$b>}m zi3x~u#UrFm#q+J`m!=!k>vr22wC^#QGt!cY8xIdD-6ki{lTIlbw6#g@Rnluz+A5O_ zTsb^KJeumH83_g@Er)qhW$_sWf!AM(jKR;$0ZD8pbg~df9W-X4BQcIq|{sWE0Oj zR!73$asZ-E(H)U%TnIn2WWqqlh69-`Hlh%)K@UTN{Z)^}n zO0Kd}hm!82M+2^*i%z}GrrDLo3(Xa9y^QD(*QDT@F<)*eOmJ|KuthNi$WTbHRmO)W zi%Hx&`3*aIu{nBKF_FH`CMF*`GqA#S&5{^Z&z?RH5L(Vzp(54W|GvZTjGDxD8PE;1 zdaNZfbQxaG>jr=UXajLY6p&8yvAZ8H;f*og>L4U$!wW5jxEcboAfy&xL^eGjKIMFV zB|4NqdXb}6unWrt?Pm@Q%FmqK#uhSm^LcRfjrF5JwGdsZ1E)Yb!$ zQFV)(vn52SeO^e`6w+y0sLkeMQ!4*Slopo)l}w0codzJmy@{$>@)lc%#r5pWPa)WX zI6bm$0jFPBq^ILsozXj(W%{~S{@i%9*7*#V6R5vV)2enpMNIXLe?)_w&5N3k-z)^O zJldJ|OI(zRDhTqjG|?x**O{^*#Mhhh4!PIG{43Os7q=v*{Bssx@5oxEZktXJ$nsj* z+Hnc>@&e#U7iQhuX7(*Lz7s`MSlowFaao|>ub)cs#wC1(VROs=!nYF~kRx@#vBfN9 z2Z&(!mZoP5VkeI7B4kuincPvZ@kcXll0EL2Yz_SM@Y9;?^^Xo6oeP@WV>-SA_526~ z{u%<#Q91-eFTa6^m&e<_33IROyf{|elZL_o{{RC@EE zNR=&wl2nT8;1==Nj+x{OE*9J<=H$h$eQ@qWKIJ@Ni+T=HIu&`oan2u?cZIp1G=sm) z=SyT1A0Hp5eV(`6cYJ<7cQQP${GQGMslyle75BJnh=6~$tAvnqyUT+>&YOmM*As>y zii_bWJQzwWAMqkNNQGof%#VHt2oF;b^CsVSM*11{qCBWU+(0}mmPOYG-PG7N5{Z0y z+aC(gKzeC9cQj=5=%Ey$j?Tm|uBj7|t^a~#>KG@+5XlKM6zSwfC4mzegfZdKwu$sh z6W72o6eGDyhs>>hb!K0kp?GfGh$E6mYx8JG?erlZxV64*Ba~)#%!mL6uog-)cO~+p zRfZEfd&q}2M$#ju`Gkph4-;O-14u!}BVJEGfNalnlVK0d0$KlPZN)&^1Fs^8!$Gn? z6Elu+ywToJvws6K;TC=#d3z`jd0Rxf-||t06GWtc1@l{qh@WsvSDW3>g4`n^-jCoR z*w3{|v&TE~9zUeerr?JOkUet}mV%_LD7QsHEGxg_Bt;mDPpTp#p`%)3!!BBbqlPTOu}n~rSq_|N<6y^lZng;qs)n{%lH{si(T;dpf!YzCl7?RHBH;Z zJ@!b&c|1E^cD;pgJVd6asq0j~4Gex>K^%##mod=0X-HMIR+QqOrEQAnx5?W)&*MHTV+ZFI)WQXNa%nwH%eR)0obKJJ(LO)10;0 zoN;Y_DcXQC+II&d9y(ixBLRtvZsa{VjGGmCZmI}7{2~O|5>I`uA|0{(H1QppD=+xnNsr9h`{1>@hC{A%>OjUSF0;FuWzjBh<*X@stwVJyQGP~k zI=)D?GuQ?cD}J+5f?%L0YcMul)HldWwxw6nxd$egS)OYjBjSRl+l$F_w62QjXmvKe zVzWBhFJy$iN99d2?C;_7z#lNV4nUD74tCgSXO3z#KRG_>Q7cxtB5kGHR3cJLGTL5J z!{!eVvbCp+O4^qVSF&R%oY3!=FJZPtH{p0T;ZOkX1Cv{f&K%xan_DjceF*cFzg@Jy zFWzM61z1^E1)qSp%G359X&2Z3UL@p|==AZM$tx}esQ*T8vgV^>S)@4@bFPtR`Zm23 zEHxj{7u78#=2)Ogh8drH%-|ySr1i;Ap?`DQgQNQ9g26TAK9#XtBO*^@I}qo)HlJo& zF@XWC5~KvBN@2%fZ_=&@Wi^?C5AK?-A;Z8q#OS4WmDAX<4JAR{!|b5?{mB`FpEYP3gz(3g%NrX2 z)rq{m_e`ZaEPT+$ugwwc#BC%(8Cmx`OcXtywx)34BjD-}&MvUC5&ADKiVkb?4uBH6tK4;gc)c)e+f23~<8BQi zhY3;ynR>PEzDV&rk@)Bv2;X0H+$#W^TDM^;WBlXNv7 zWI8hA#hYR@DZ3z93oN2ZL{jg)h^Q1(2W)we~8$rx?e0?Ksu@7@`gj4 zcH9g6!Np8QZ`|hXIK;LmwC3C(e@tK!*+dPoEsnK1gK|D_{$awqywrZwewQ@&|CQl= z4{H&m-U@)graHVJAhp^{gVg=28c7rkB1k2CMZj#;mIK)U1+}X77Ebji`QH7#$O-7y zg7VN=O8%;)JL4Ohi76bkT&xR=8*H%|(uxvETD63fa89{Z41e(X zwq&{4{O*;Tcb{baO6L8X=71(=tpAM{zFzY%YtfO1i9seoFcR~dHw z{1_lTdPB+M8=zNGiGn&oy?bC?+jM&QOv^@W!S#E~{6;`|>345Hy(M}R!M|V>FkkGw zS9<@BUIOm;K>miB!+q#;ei824;-Wch_VRd4_eyoyl_7W8lqq{JC<5p|8uK?7^Lc?; zoe+8LDDifflQ2j^?x83L211zQSX8-s?8=~l8gIk$sZ9;pV=%{KcQCEszZMjo!K(W? zV9^GNfPS+sa&g*}d95{|D&SaQ`%nGOx*k?K3z-U4BMk~Jv&xw!GZGZ_WGy)#C!(C#`&3jqG`!eB zpc&)ZK=FLWIvl!Y=CSy#YD8y+!e+ z1$p{@0H9x3hn7y*Oo6aP0z#6<7}(aGb*YYD2j#=Xc01gjX?dYTY$JjBge>`>jX>#^MB7`w(w z!GnV$T&KcBD8>wzt-JC3>`r!2S3PAj4~+)jL?9y>xDj`Gy#5hAr{ZHjE8`-K5F5kl z_Iw`w36kD*pyk`2pvIbRv-Jbolx?eToc$Whz}y5eU}|(-UGOdVzA?F|P#Fn2q-`l@&13g0cFPl%s8)A`> z?R9iBR(_%&JnFtsZ=Z`bc67|@rU67ov3}rBlg^frR$>$&aM_Xh1uu1poSz6Ar zY+ACPMIgY_qKL*aZ17~vWb$+5F2bt9@%*8^k@2aLv@z6450;giCknN?4JnMI`9kr~ zxGVwFi?+o|lF+nQtYgY@9Lt=xox8AiO#3nHgJvk!vBSs*oIXO#3zpnn;Woxj!&99` z$XR9V{G^G)Y4&U{5(0lJMGA^FymU~TzV*($q{2Au*@7EJta_`{WXDthEDetGsb?&0 z<6}1^P$V4=tI&}v$Q6@XqqDW(Vhpx1iI;%fKVgL(DE4L{_WcKp^tGdS$?QhspHpFj zQzG`ou_x)ZL@L||))`$s;20%B7RT#~N4TRg8RZ0hYQd?St0>$`%Ua5gp4t6d_%%I1{5&$_#)F zMku{$apIjIH%ODq*K}I0nId{y(b(mYQf4Cm_2Um!<~%0fy2pIeq%6r}>E5CEu6!4T zDAvmB%;tc>?gF&+$dz>6&A_#ftq#y%2!3x}qnhC#JV}av=~}N&IH^C&mPd0-mq+#e zmo;u~r|KMfJ~R$Brf!$>SY$x{2RgUkhqJJIwWjp+6v4Im>OaI74;Qx4!&${ECLeT$+O z)x-+E81ya&G+Nje=j3cJF4$Aq+N3f!_Y@!bdwn}7*I?df)V%eaqR}wCY1{$BznPaG zI+FB*cI5o5aXU3{Z6r~So=v5-`tguv1~~Xb5w@epe~s8j5ux9fA~k~Y0C^W}331R) z0sY%yk-3k*PQVt|t#|X9;o^h#A?0UplC16WY^G}6 zdw^R7u!H}JxTAJ5drt2fhaTh7ff!Dc?QdF_+82m5Dh|Vzh^LZa|gD33(a$=a!GpY08TR{y&20yxwxbWDm1tfL;~gB z>c$A;+!_Z%Qtem!djSDs?{hJI2=`c-7XP;oTrB5(g=QWp?!>u}8q-1|W1#Qyyf7|8 zANk*}BL><+1t56*unq zN-_IkO)UlyTNQ6;1@s2l(4FdY-iiimO+iGq1OB$bQRi4w%WT+=5v`v%$|e}fwadq* zlp`(qF1GSH%-clSB4J~TqYgFOx>)1lk%K9E!*CwAFwMi0@{|_gQB|{$^2YCqiPeA> zk}ZohNihqv`O1nz8d-C_q_wHnuVR5%xP^0iQ=-`+Ovr z%KxPT$VWjplr4D}jVdtP3R1m-ceTo{_xB$}(oO|Pz*{=*(c+fVOjLLaN|1Re_H9yE z(;;Q6CxlQJ+PI;$c&7e@;RKY5gxA}SGS! zqGaj@sJ?$Krw?d&7wmYW1!bJRZAx`Q_LS1HloeP6n^1`m;$>EB{E(Eo3!ZD+1#l%INKvR%L&=VpYEp zxwI1`ir!FXYe<(hhH`x(Jsb*QH&;qNLWLu()?6&xy?!w29^%oafaHoX{d4VxZ<_2W zK+Ff?=TJjy`nR(`P;U+1+Dr7nX{k@&DmFWCa|OTN^*&lCM{-wuH(#{c9Bce(X0=mg zH^_UA(yG1{{H}tYtBzol%N6i>4uz{QaufpG#x;W$5^M|y%a(hsHjVyH!i@V~D zX`%9Eg?gECMV`yVz4vPHvvu-5^fF!okgLpLBCB-vopK7yf_RA!a`e$9f+1L`JkSjQIt(x3w> z#4TiZ$bk)3>^>Bg7t+qLByY@5l!vjuVI1*1iJragB>0rrlBq!>OrE)u_wg?BG#K5F zg*q%yNqoM)d4dJbhR%m|AyEyhSZO;mT4 z#T~kp$}y?F_g_n(|73svcUb!SO=>wQD9S&XAl~<=rj4PqbF%8c`3<&wYTs7D zA;I#Raus0Lf@lr2wSK1ztIP`YDXgp5nnS6#XPX9Q%9d~(G{kWEMe{W_yZY=GA)Hdk zTt+GD``q$743mHI4?43;3lSjP$2rgNo@_pFpSa)TeB9m8_W*N5ZvZ_)Fi;t)eZzn2 zk&j6LI*APQd+kA4LEms-XrcZ)f_@IP01WY9iI81kcSule$-coMP;Zwn9DK)`^GbQ9I}5W*6u1NxY5Y zgesi#!YSG*I*r4yWlkn9MH0dwPw{ONi`c7Rr04Bby{wSgt7u?KUv(P@hjIhVBu)A> zx%)hgVguT;^@x<70oBj#axDtFGplX`M^`E4CwI@eoZWSzmU(df;LhYUr) zK5_Jg5mZRA;@09-By6{eVLWrtyfU+M+eZ4>C9bfu82g)xrKi8P^Ll{bbxIei8{3ad z48DL$Et-UJYCWaQS8viO0@BXG zE0-nQJl&KG##WGVM3TUQ1m*Suw30V~?>Obz(BpkXmlUPz3#S5cM@{>*3SqdHvkY62 zqB%>&=)%i`kLV_zuy%Bq7*WXPZ&KFGkU zU%&UcEj*k`2SAiU#Yqv6oAOJy)FA{2n?^6JW`(%heIEF+U{4?=aysrl0lX0~$UcV) z2r1qq%jYM)4|G089t(5klSGjeWLBw0x zm3#Or(-zSdv!5|bkb0Y-D&YyTDxVDSK`ydfA)bkQzz}RkAxNr1{*B;EgQPuR7yx-) z^D=!Q62_uu;1Ra~v2Bx?s<{NX{INm|g#^V6PRl~V&e|H(08X8ge`rvRlEQcf0e+gx z*ao8h3PWF9l{`wCJL-FCs_IF`_Mwlq6D-G^q7{)KclAmz zKx6!E$)}%xm(@@6R=pdbEj4K>fF8G~PHtK?hv($8a*8ym3{rDY>3t4BS{oAGRPYmi zCH&Yf8FvnHR+d{6$K2l-ckfrfGUGaPb0%urngY`g=+y0Ig7@4t(c1Vk@zSW(N8k$b zOi1#K6mV%QvPynxmb;Jo4omfpJ8%h8xSE<~s!ur4JF_F@L_5x)qaRSR)jAIj0&T{J6xY8V@DBJU96E3Zfyald;H z_@@drbkKT-@l7Vs_uqsu{#{o({+^Bfu0lbl{{5~v{X-_8{;hHLC6)jStH_d%j4Dm7 zO3=I8B-I$9!;+E-DsnkR$k{tbdVGPorO{; z97iYEjz+;zW$}^4iX0Kq!+gm z=z(DmIZrOpLm(Ug3&VsQ6k>w7AOU)p6DPd~@IthN1vlK6LUc!j**D&V-AB=Z+vo1Y z8r0o{-1i;~d_`&rc!equ{Gd7X>q`Tbps==SC)*P+R^EwXt0-$P8;*(^IM_7O;-bpq znvk>{PLis%lR>IPw$V~9PnknrkXbUvP+7>CUkb8hbY<`py;*_vO<41hXs|I8)fHz4 zki25ogp11>OS%WRwy;#pYLgwwoP|_r(HO6Du~&h)rmYnllWMq33Do2miDuY^Su&p$ zre&D8_0KaGRzG)0ykPHwH`FlFzx9PfXYlF3_ClS5T$lXto-tw53tTUZ7LP4ux zt{p((L{$_F)FEr+M&dm&gkDfK&B#~)gnAaubSE*g<|x$do%1to_Dx45iZv$FSjL1Z z?>sA7{pdwjmy4Sp3(bwwrp}w~uOP7H|&?PLl!UW|0xvHCfnP7&bBMq%OU;^ zhy)(z8f_&nE=|}LT%QlIrCDU2K~|AiIAt_*onV=t+!apYl*|;4)#R0=YtE1buxBo2 zq0+BALG|!cw=J3MM>(G>aJP52v?#x6Yr{2*BstTg9z}Ynu|Mk6cUb&N%9Ss45k9Gr zhRk|F@1|oO3q!*Ri*6lVEbF?gUv2=LpB`l4yM5AV(*128h zlBu+a43)Bn{1KjH8Ht-7-#FsJC(GkDGN(kM!a3wIS=(ktB9R~A_hi|8D(0(XE(zrg zb!M(3ZKX4T&jZDj2XdJ3YF(4wm*cq?BPYt2;r%ADs{4u_r;r&Kf){uJuuL&Gb>Blx zL7dDzLxw;88%2+)lR2|cYqyxI%mgm6MRUuPw=PE4CS!u}2c1Vh| zCbpyez&GcgU|*~pyl$LwU972*C<`7KbTEVNR+)sU&Y4Zc9+PDZ#jQ)S zLIRX|rzoAO&?jpZy>x(kY-TtLM?OO)XJvls;>5ghl)Q;MWsflz%4}`aFg#$j2Uyg=7TDVq zN&IhbPE0TK_=vD6V6PYXJclLpbDk-Nm%x~iWfzl*H#JnZAoVmcKaoRs@Zy_RFF6mQ z#4%Xt6$nHRa)q+*SE>H$N_!L}X*+#=SRexTM`O|u(ls3+p5T8S|wv zQ&L@p9#yk%`YvU}0&R$+K1v;kh?NAT$t zIpa&j*6>CoOm5B@g6uEEKL-F%06Nd;HwKRq{{Jtul=zYA=L(AS@8dx44+4ElrU}R# z@V^@n4(1?AfQ@gYD(2s8#%0Y&okJ#pMyL1~xbx3?LXmJX5*$o{1m5&{e5ur^Ma}U7)hi1_fB2wU+C|*Qjibr_s9L=#dB?g=;qaD9OIUvR)2B#ErAN-!!aOWS{6LVHazjI4)Vvs;8 zpG{XFl=$16R2z5#@Gm5(NX$s${cnHvc zM>j3eHXtPE z6I+RSlfd<}k*{F4VZXBMx_P;DIqrhlGUmqYvr`#>O3BQc%@0Pr5d_YcP3IZT7tR@8 z&i9#~o*G~dP~jm%G;`I?n*I6+1Qin1sMovrpDQ0}x5o3a7k&VlH^+fkZCXeoDk8G| z-(g=rv=9*tVB8>n8}@{Sh4CSU|7afXQ@cc~lON3cp>=2XtIk^lch&)3Tof0BvCVP;zGgc>1$yU>7sYRHMrFe%DqT1Eo~0eMJjSxUv0Iya4J0Og#WTvoi6~mL+HbCP{eb#53O5?c}_$JtBxG?(pYRm_~$D>X&lbL?I6h5Kmk%wML zz+r?oS$Fup5B$hA_HA1N@T_j9X);zagwps3iC(em00F9TR+vqJkPB5k7A@rfHj0r#zy~DRZlX zr}3DncQ#&Z>{%c+-nHlN z@LHSHBaydWIfRYJ`xPB5k&*UWqWaF3)g7-t@79lja;Rr!zo_f3gn>aI4KP<6vMcIMmzujvkqGARBA9P`T)z0K*1WzE~TO|u!rHI|*zH<}{9-3DUP zi<%}kaP92sQ|-7vmj<@64^PNQ^hsZedpu)Z)X1>Cfpgs{pON`ylrHH(dmNMak#Nxf zkd9y7Xpz5n$y9buH&377>MS-}dIZdEf4U3Ux;uHU#7D<_8l=3=-wv3_Pu_uK3PaH3 zf1z%T8uK5o(TWn$)lrZ8)IlSu#~;E(AK-f*0h!%$Hu&^Kn5{B4MIh4^Y~&KBA)f|l z{PHqSLVL6$`;rEatsAoH>(JfT&R2Q^F!f7kZvN!FHrII(<&suc&96(`Uc{uoxK}b! z8u3BMbNhon2Z(ElWNhqP6)m6>^9`K4kj(aj)Q_Doww~OZTWK6rccAgXEshaeKv|Ma3Q zKpr55|M8ID40GSZ1|A^j|0yv=tH~;Fm?8S1Z%9x8+kip(lPCZWYS4ktD`+h=62e8J zJjnhgizA9M%H(3JGnJs@ZR{axdsL&t%}57kHZ5MzOGtm7RR#$D6)$<08pd=0M<`h8 z=$PrAX{ae=1$@0+*#7{pB_k#=78th0%AI%>8urCPf_gy42S{qf)lv|W8A|paa4i;y zr|B8G7zRL_xKNb}=K_mXqUvB2Yo{TgmA(q3;q<=-*BO68W6zSx2n^x*kWA7V_EQxz z&`gF!So#zhf$Nv24UJG8GnDRI8>Wa3MU3(xt*OG6F*qdt0Dgm_1Lz0M9^ORSVQPy} z*qR{caZ;y_14#OV?BLCv!{h2K2NTUGnx2@FrpT4@0+OXd(I<8IlBa^Mc+nxNz9dMT z+%V#y|4i&FTjQ8*Lg$j=QuFv)9eeZ~OWhVJZ{XVMAF-KQ;s2SjwMvnmbWoj&RqN0; zWt=Ie<5!nfb$YUTOBoDT{;4ncE$fNVR8S@}-Y)B~1DLAL3>vneGL}QnsY-A2G^U-3 zueL&^a9$Ilvx_Ma39iX%(9c3df!-hSsbW|S8=3Oy_BJs1y@|!;20oFU=X6r+U==+j z%iJKRu(s`uB#&`lMoXr)O6$Zt>Z8Xn?-`KAaxYoJeRHOMD>CXC5R@`*{}b?yH)7J) z$g?C)1+b^DZ#LpKH>OK!ND7^JmWZB~of{&FmG`@*ayQecov)$72nla9IjOB7mAU;U zDcT+vYu202(54(ot-73}WU(u$JTx)nHLRA$QR4|0%yDO$kzuzQyxmicPUC;}Z9kn^ z?e2Fzh37Nm9@axgHvi?Y`vc>tMd8txrAth`7tjQuzl&%!dE_()M0XrS!OMY|+APve#^GQ@-25nW$>h;G6xyo2So&@I#*56dYQ zSyPqg+?X`SPsxMe9}Yk~2@?^n3iC4Qgi6RZ^UZjEqfgF~Rw_^mh z-yz7?)!jr!(|U5_CcE;z>~NPpwf)tMjs|#fC&)1;RJi=QDr&7E(&84g2JX_H-vCCS zC$`9^2w?GN%n*qx)J0)Tg(FQz+|6~ltEmQCm)Vxp(RWio6Iz>ga-OCOaC<#la_6> z?sM|Z}R%-Ut`}s)?Sbn#GdDjj6-HrMe`3{I3 z4-DC6gZRkn-+$5-1IvU0lW#Yb_`ls$>J9O?Z`y4FflTEL1R4ELZsH0`o@(&*t&Dy9 zi2&Kc8wBDAg!DfP=hCV1tsvO{_iNNGa3LV~|4DAXd3yeH)9<@b0$gKx$r5Vvu{mqU)G3{>g&T76-FmT;BHR|2keVU#xdG z=Kds+Oc~`)ymZ6M2*`q$i7P_eOL20F$f`^%3@f~N7!J;3elU$=VM)Pcl69m~E926H zETQ@2HRt#<)B(^;h=L7-W6`&G7;sl+VXC2Ugam3Da#3SZRAeE~Bs~gL7S90q=VY1Vl4!Kfl`zj{i** z`(NR9Tp9>PszB`b4y=5AbH$}afNcLK-1dqB(F1^=U)R&*LzaRy1xU12x4iy_L#iW_ zg+>sPiIQK~I^!}+w>5LO{!=4@LXhwZ@=XrTe3YP3PQry?K7I8*ozBe3@8|apzYB!V zo#uW!I-DPEibKt&X8S5!%W!Ojs2a}Hs3BImmIjD+|ZM8AR8gLV?HQub2< z?~rljYdHK~sxb&U7@7<8Bt48vgh4~kW1?1vPu{P4d!lmVY`V$ahF zy3oHyX(tVEr9rvDc}RQ0QDYfiuYX7j8{7Q6o+spKe`fRvo70Qoq+&yWXaP`LP2VEX z?6-cPP3y_0lDWoQ15JJa|MRrt+asGXerI-x|0}agtVa8Pk0n%U_g@5()Y+(S0{Z5v zj*Tm#h+pe!W$1NB%|=lOQWCXB^M$trBE~-0ZY39Krp;?G#ebz(fQZ8Rdw-hpC`+Sk zK!;xsq&sKcaGy-({N(TP{q7>h%JQ((nHkLW*88J@RDfskxX5UZ?q?vgC#BWJ4KxN* zx7sNt0fU>_PVO}$lj6K6G}Zmej|O0mvkC4r)`cX5oN0AE#p5xN8_c`AB}z#AZ3}^O z3XQ+1g?Hw65XW`PmW%r%G#o=s#15x%$ns}gdV6pFVE$^}HN~Mv`p25qXLQKXI>jr+ zPGsZOF_LZuR;nC>cGJtiQmf%$cHpcuNIkE4M@iRlC=q&|jhC?oE+AUm%I}w zAwVN295u$|;z1owxE!*WoHU?SXIP}xbxniq#rS+eR_ow(TDv&+ktr+ETud1oSONu! zBd{z@^0fotbtNAp0#mT?-9(D8%9EHNjODV6FA-8pmwP(u?)})D=#W>s@thbfb-grK z8&lFVcBFC+2rm$<7X?y?kON)N=+FKNE;&?>-^Z%Ik&DZnk1?dp?iVgTzzkNFfj6E+ zSp$~Y{0vm_4Q4(Ka(tqmEXbo$J?F{B;&}!8$F^p1#GWX>Z7cYH+m_+!e{5^E8U!kt z0THj|Pdv!|e>~YYef)p9+Cq>7g7Tj$eN7VjY@kE?BxZSLIwMcDwu*gsPsYsk%P&MpAIl$ z2}X^^F^~}lC}MpA`sKxlA`oL>I7oLJf0>{fM!}>h#Pw+couEz{iuK86VK@l(@z7DX zlG?y<6zsy!Lth(d38H5NkjhYc1YJO@IV$!U7ozeAX5`7snqY-2id+Y8&$24Fl&>q@ zfzf-W+d20+!&XQg@7X$UG0V_iIEug6gwhqC^}Cg4=q*SCX4=eH*qRcq9BjAfR%GWV z*7NCUA42x9o0DnHeZVXD_)HdD+lu!sx7qMp^X#I0pBl>5XbKvR2jQ>KG+GnU1_6D2P zHfNrB5F-i>pqJB96=9jwDY`XZu_+l@0{i|=Vf^s)MC7jam@m0K05~AUxNKM6hY7ho zUzji*FI=(Oya!XSH400znOK0EFojeV(F);Jy{2u^R+>)0e|mb(!YVvz&pg4|FPnR# zAIy89zDw;fX4Mu=&AVDU*gD0QmElk7D4IE@Z}`Zlc0$xnl9> zz1@s(CUtvGL273;5M$Qgl?grPB0Cof8+`@#KiTlM38bmsp1#>>V7dpS-(zET>{#Z6 z%W%CGICD}2lmbrR4MFM>9r7?#0w*F)Fv4SFv6vsfEbf_st+f@5D#N?RUc%VwO2U0* zaXn5#0qtU#1x4VvuHaWM=&q9(R%i-TWeMh%AXEckYN2ETNY$~;Pg_kQ5_yxrNJU(rLCSY!eluSPoN z_L7h>fFw5?19Q%YNl_)Vz;{^h*A1hp{Q-oKx)iT_6N_WuP6Y5)x=50xdfFZzbg zu?b^Ig3-~0A3#D(!oW!WeZhzV1&HwkjsjKROfP$VCS=*3;UczWOLXg2O>Rh5+806* z(X7jELQ(r6yLv-K%gbsNOIo(pRc_9CC!H@$P>GUhw@(eS-8TSt&Xe!@o7+vcIDfY_ z{V!<6Gr>>@{zror2tbYZpiis@0aObKiKGWfJTa6ea$AA|38NOWF$tp@@&O5>9x@7v zB0+d~0*4rKBFSFzU|m8QO*myj8dbPu0!5NYJV~*%Q3#2dRK5ms0|}#c2n3gKK%c=X zDRJnCN#nSJLqXKsCSIf&mgna9fIC+A$gOWkCZ=6Ve5^E9H-O3*n2)~T#|9?fKyCIO za>xW`cdrBKKntcDgOB*I$RH-)$gOBdg^5hRIaW8xfrd>oBbNI2<_x3S#51B{FzOH+ zW3OY<8N4zLtYB7~y>bmS!R{Eeii#256nyKl{~V?0KVmW?pfX0Mn-rzvn?f{ijWHx2 zfP-Z}GJ@?M133874d?Mth zfO<#!0C#8HEZA-FQ0trU5bLAnDBX4Ukm{?DdBL7|=WR#p^gluU0E(sXKxv_VC4Q#z zfN`hrz(JsRMd3pGz~WcC#rP2E<8D>&)AxMVCHume0d$XSy=e5lpDNwD7r#Tyd?k{9 z1#_c);P9*70{Ip167y8<8uOIxB3jZ5U_<)9qcK9qQuX)glDq4+$|8AsXd%75#uFVg zBr!~i#u7ih%0r|~kM@X#eDt=-E=(Hv@vOlYt_q`g!L&%@uTClweJBMfY%aI}%24OS8hBC*O z&{-QbsaH1ECxq)-)$|sY_3$FG*V)k94IuN6HCkDPJ{{h$8=B;-7f`%tv{1;3Nrf4P zno=}2Dag;Fy_BiX1Ys{4MQG;5K6pesokI>;0cRQ=>lzxwj~l*aKcdw#qv`u-IO?RcBj!+qov_&fZ-JAczy-BY(XFwm*USJg7M3)_$4ehJKr&h z3y)^kR;R-R$=unnwivVI_Q?tp^P{%(P_ShaN>-eIP?rEjJ-5gNOE86+9DMkVG%RP94s!!N#p)x^sU`GUS89BS0 zfRqga0vadZ)OTdFDhzs!@>pt@C8B;~wH_l-=Q^u8lA(RNy^c6HKfUJJnbo_xVf1#R>9Cu+pupEJGEII z9`Xl+^#%cW1c6x_4Q^p+Md1Jt;tNu3se>a;;-Z|H71B+DbUT~fr!WTkI1OnHS0xKP zlD5-bmHtt6NoKS-2^!8e$1GyESYSATPZG@CR2*H*W+|p>RDrIScW2jVQQAeO(^*?@ z81I&~PQVaztSmVmP8P@SAklC%%3`sTMpHbX2m9RlB!U!IYCMK5Ov?<2mh^uFlD_r8 zmf$C+foig9Fl&(+$F<5)lkEe7eA>*jZ~h)-MV}l?o|S56L_uVOdLs#5kiH7!!qyMF6ye5 zw4}~FJ4-Zg#;RzI?cNB;!TgPwE-GJX5nDdx6;}EYGcbC{IGFr^Blq1HZXu!B@p=6%L#ad6*v1QI}H4_0zXof!>oym_!0iK>Ou&nuJrv0&lJAB(=jb7+2OoJ}|-s@A6{>09r^tz@Ca1)Mo)Js_W5zxJwCdr4ubs_nbHs zZ4+)9#?1Q#|Nc_ZI=&cQJAif_@?i zEd=dQ(o%t44sf07JU`IGn2^OPgHwi*CpWGN3DA=1)sh)bPm6ZFHReS>ps9ZY&nO$= zgxu$`5lPa**w@jCdZ@#LYsJN`79*|}MzITsk+T61n%fQO<@;bV@ z+!*7u9UJpldOKj~#GQKMa~ft};McbesAIVNxp@9_(@~4uGJkDffVU&Z-5tnrU5H06d7|zx3!mQ*FU$tQcPXk@i*r;H=F)3+e$1W$EAMvQw z1>*b;nI$8@K)z$x+?hin*LeD}o24fM^{lA$k^0?sQTZ83`S9rdu6)IEh_Wqf>54EM zzXM~*Kh&B_bP7Z>5QwV(P7#Bj@3AId=lIKL52b_Ss z2B%5V{Es0mWHhsJykpq#t5%k{0}ed^U0fHZxBZ}-TE2i@+#sbS4HURj`u<_I!o=mC zs@oK#$5t0AnAv4qnjOteUt2*dF2hJY9tHi8&hv2wYUf6Kb{b7PQL+X!hJ9|@3rn3_2 zLitZ67mY`V@NbnDRqtAc@Pm~{BNoj~eQmR%7-rQMXk}M>pWdUFyK5_&NLO61v0ORb zC4Exgl$HcY&`lM)!rueZXY~O9DR_Xo(^?t3_#_=#T#RtnXI=L%5W&e{5-G-%nB=p5 z2!1dLfysgJd|>miH^>OE*+IuvAy9`R4XF`%Fs`dlSy?=L$;vA!<{`--dL?^Qz1M$r zk_tvJKe}MPi#GWGvyt>)hn3}jq$ae2V5QCiA^rqNx_Qdko0tkZxmcPR8oNl^nc4q) zf;n47*B(a{mG_`f)4TyH+;Tx--E6~Wb-QpId0afP@vwlll>@v(WJqqyM>(jEVbo!90Erw|N%EZOWva%~5FP8a)d1%cT zus4lj*xZ&x1a0I@9$thMG;af$$i6=dkS{cNJ%tm{cUXZ%!Sb#?_1B?Cj)5)E4e?ic z;~Q90YI{y{Z@qf>ZOWe%kI|@xnKFteMw{7!HucQa(TpP2zYVoDSkh%}UsM5+e=wwn zsiI4SCQl$5u;><~dv&=itX^5@&Tpnc)8+ScVa1kUrYs`D<&OJt2lZ8hB~_#hC{rXH zFA21xu&GuuG+-N;EywVsAT@~|Nsu~V+O_;^+S@E~1?qJYf*N9h{h=bl=Bg9xtB=K)U z_|ADq@n6PKS3nIM-W3&rf|m`S5!`(aAkPCPefdB1T9cl~0q=>h%q63Y!cUOpSLC$x zVNlu0^L)+q4ZL4oBwb|gCae}Au(4nIm0qv&5A{+T`%AFeGcl7kwa zw;)3L0h*1p-ziBDpD5U1OOB;TR-+(ku;X;TGPT@ z9SAlT?{$8xL`z8a2ax2APgM0Vv?JJdFf@jSW2hPaj%)!8M>$;ym%L=tW}WS4i5A2W zFASN0#u%ip74W1eHnAJ|F5bb9%6;mj4uz|LKt} z^gWMR=nDVy3+A69nJGV8h3hx@%Hx0IUinuFuUZ0P1;|#YlN%6#^)0pq&YSH*mLB>6 ziwkQLRz#(|k*AW3S)+~uuX~e!h}>t+sA{we@=7sCnlB^MRAkQkt~qnR@-jmHi~7;X z#Ta*pLr&n-1&Nrbae5jf=u}Pn>IU*EPn$81KP3l--{pDC;)r~HJn3NqLsL@7zLHjc z;wj;w4WN@jAdjz;PTjr2dbUDkSSUMG*{E0J`B={Ju&&u3J)umAwD%aE@dMT}%76Zp)@$s9oEQJ^lwURoFhr6ArVV!AC&gflT(CkcrD{5c3z1-=jPd zz&)|CNH%Y8V$J(sXp7hE%*39*@1OENxNozg8lw|9JByATU@nW0|Ea?MKH~} z@jY3vpG4)m*db^>{sqw8gM?uEg6g-d+sY1j#L(SCNKj3E%cViZ2G*B_8Op?1OEThr zM&!;jI^5~JXPe-dWuG&cxG%8NB^H-#mzr$wFZcaTONh^d%Cl#Q#pr&{(P+m;X^+{2 zyql0*SAjof?O_5^uw<1iH-M#@uQk%M@WYSqp{UE0nL2+J7Ib`P!$GG8ma=zv4@2Zk zYmJ6nQ6o!u?8py%QIR_?IaJ&jYGY^34xCIXAH%At$V3)q4x-bhUYAhK?KUeCWf?fE z%IIA?c$MfV--y8aWgP_`bbHCIIUD1E)u_i)l=k#RS=t@usY3fc(d~X0SUNcu|14EA zm|j)=G=p0WFdmg%Ne;Bia*c-0%e}jgOTO;;r+3%|2ZUO<_H<|{TG5jZTIOU%7WV24 zE3KB6NNE5GO(a%v&S*^R*)x(Wjqu&ol3N8}c6>c!+Z^)hnaWVJf-KQRlRsOP8Q!)C zmRYk?u)n?W`uUWSHPi8DmrB3ZjRjvC$taJchhxqDd9Bq&F*k@6l=K)LMLsFzGtl)! zjjAQP?*#NUzORMF-=K4nM`o9rJ_oP+VlF$2Lu`Ot?huKDhEWo}5!A#(wm8B?f~Vb{+&bC2Of3__9i zj9&m9^@VAh-Euzp0QE{U z?ykY6aa!V&MH#glc|(*E-r$Gg!SHxyRCT76tSyO=){ViM!5VrOhzr0SXkwvJFQ^wM z`m^W$P0FC~VI)cqB0|$=h$=Hzbe6ck@FF1G3Ey{M!MmfdB~xqPMN`op8b^3fJV{*% zE-ZeyEbDijUye_Vdjxo)S^0|c{)q#SHp#2sFM6J_Co`z5XDq#exNljQ$2?4>3S# z^UCcix#G!BC-$J5ea^C94z#8?;>ZYy7P^oz~CN_MOy@1*!6u(yaKl9`N(6@sjb-IuE?aR0guLauUE}=k_D$k{>-wZ-fFR-ll9ku&76edv7YbdiV0A4f zmII`g9EgJg6${7#M8*GB+3y80!@V$t0_9EjQE~2nfL;8+M!;1g*|Y zTdYV-Yp4Z8h{81o1KU}*8XLm7)1uay>>=4PYBZxbZtuc%Y&T&Y{?Cp>8ViJ8M;SA|WUchd4y&xqVQ|_1l!La;zZmfa9Z|29Q!w>;3gU>XR2@J&Jn)WyjkYY(m25OG)QyQWakA2wVIM zmZKHkYy>I!+p&L!d+&adcgZeu$ji8P&5lg!YT#X9W=cUOHtGo!v!WhzTOPsY;(jH2 zX^2;J5Y}s|4xcDi0uH3`a9ZNTzYbZv<7EgN9a}tDk`o$^CR2^@pUwQ^C!OsLq5Kit z%V7DBZXxJ@9R1*S;a`D+0{Zb04Y>{60i61WMbZXt4SxTZPy!|%(}H$7Az?v_P-qyS z!FI@zzo{y1@!ue6{$2A~1d6r|yhi-r*I*C6zk80TAiw}54CuTQ61&a02lDGbk=u6l z9a07IAHECN(0~?n1%t-XmNfuL^{+w@_|sYbdM8dB>?owzzbnD!?|=Uv$ROYZDRiNxbXWNIRbuDRN;6^<7g#;CkV3hmvePhu&ioxr{g>zI@c0i|+^WHHJ2h zChR_ptVV}mkeO|rM)sPe|HRzw5Vf~7CjTw?HXC2!J+V(8d~Kr@Hs5m28;GTj^}85g z;@>E(`o*lbo=cchTIbVasznubt6|(vAq(5EM2BmVHr3F5V!rJ-+$x)y9p%o?oXUS&#D5X9d;?WO^!C$O<7hTas@(r=! z1t2oU(b^cF20yH^x?6e6@u@N*VnsB5BpyO{v5=@nXBYomq(MZ%o1Sg1U0<`t_U|%j+IOkviG#v1A)dMav{ttH_tmsGz0*4j_C}8m~^;K0&b+kVy z%x8_x-%_F_#C2=hnL*53CEk$Y>_$p#G&ilAxwVK^ZEMcxU&EgBwMk><9wp3`n(>rh zA^y<)%r-$Xx7eG=&iLH(WNkh37t7--Y(hUVewig6$2>+|Lo=DcMvL8Sq1Hz}sBK?V)gIsi?!_@orbj8t zmAA*-GL(jRL6>P84MjxhqiorCS(C02akk(x-xsv+ax&HDszj}N&`Cb}xA%B3Yvy8ngo=sjrQlD%VbG=!d0 zN1T^fUB^b+aI``rKE$(vm`w_`4td}XW@rqMR*(MWTQBF`9=|!A<^%Pfw{&*f;1}h( zJ7m5sbivR_iRUYM4a_eJuBdaCXPk@iOWL6(vK}=fa6jScMXB$MJf1lJF*YOebEGWz z|Kyc`z!`@;@cgr;71sOaL+kI_&aPDC^HvFQB4fiNv5eWifQCLEUa<^9XrFIRc^+Qw zZTkoH}u7WbEeM zF5Gh6zRvi3J=}N^+Wzyjulpg>#YCnqMg;%GGG^S{1UQVyVUlzm7Q;&&qsT|OTOCqH zEkwDi4}VA{M7wJRUnl3S-S5D%`aR{1>=n?BFwqbjEo|iOyVIwFTG@~hje?4^YsS(F zXh@C5!>uv8^RCBpB4O(DWa$vn8_((X9+XsLHagoW%!yV(wSSaCCb`eY>8xr%>TF-F zB8v<*0wMtlTocvuf5NlCeB&D*a)H)IVjSIdfi44x1qNaw+#Kne?#V+#4`Ro-tZS~; z?QrFJqh|)KxG43l^A+z9Bhh=~#?yPF%0j_GKqx|8T!m~)mdDpE zz>RU_5Wn=`2yw-4ZFU{jt#jU9JVQ{6%y`{_hvHL`P7TGn3u`_7wn6xB_7nr{L*tF6 z_8GQ$nMH%^7tEp%xxs5eiIy8OklY!TPp>}?4y+-T>~FNhIhk}JOnjvV^}2~hZ)iNL zz7Z{GeoUi94C`}Gez};&aZkqgfrwVn0d{&+c#T^)rQF+P#K?#!L{THyk;u-5q?_Yg zYcNc6al(r)B*PFglVf^hilktZIdt&nTO<5x)0^;`6j!ruqpjiBTya)loI-+NJp6%F zl)$N9dF!6Euf$x1sMi1}ntc&+FcXbn7$wby1bZYzRQm9g^WHv=JoBjkkqd}}+lvErAbFyx1adW3-9;8lT zmp9(S6jc&bI>X?Yn*TX7QmIsw?4&h%Lb4bvx70AS;%{}dK5uj>VxQg|Tyj_NK2~$> z@B6-f_iKmLQE>s3*Nl{yhmDp{2vm=d`FK23P*ol&ScX;te54RvPM|q1;4`Hp*2=1G zAFSq+rLN80JHd@uS8`##lcLq2uTAD~OL_<=C`Ch9+W9kKOBPddP0wX4V@&6Vf6vLr z*gLaYLq(}p5AmBk45oP=BE1itPy?@Mjp62*Qfn{_{;}2^JrKIf9Nl3+8YrYT7`zan zROF&dM}qksCDdY`dhw~4arIZPht`P24X@r_aN$XzHHEwvOn>!*<-*TbQ+)Qic!w@|I?V!Ip(r=)>r0EZ`W#E?J9 z<#FHSUp7|k>&*$o8G~jCMSzdL>LPAGVi%zwx@75jlZqNV<9uzrb9!~O%bFpn;`bEm zS9zR^TRO@gRk<@meQp5pofk^>fYcs#HP1$H`2sG>^?S0jj5p;zP zlr;WP;J!-uBZPJN6RGnjJ1*^-uwtL?BB4}?!$hUEy<|A&6>p=KSt%g*i$MQDy_v4id6BPkz^Yo`{K~a_ z-2&ZN+?D^zuElaP`~<9%#W{sr?_)Kl9)#Far|OQhH^i+?Sn*o0pR9x{yO7G_9`m`? zwY~jRZtBY%vfy4rR7Q|jfOx6@DkclZl!I z8V47eY~aqv|4wB7XV-(}1E{VO0{ySF1{oB;2~KO`8dSiZV`1+Qztq?R`)AeFiDDVq zu?C|gFflL?F%UYeJ`3Tnc7i42QPTecN&hZ=NM|9T z+p@MGo&Tv5{wbf(&LGj+gmxfF|Eoy)4W?46A)>*z#qU9e{ku|tA6&7B{elz)f^}^$ z)Un=;^^Ej+(1&h{xE7JbHy0NrCSusQp=pV2&YVJHmjgrfabZb)kAd~TeB4j=uSXW8 zBw5Gt_5xny_>z$RJ6_!KnPv61GAOMw-;y_9W8(h&U4R%EwQs782Md@K& z_s7EA0pL2w_tFS4I;hz;Qg(nlJA+Or-9*oP8;N89d9m+OW?s|?h;mpsSNZ6WB<<)4 z>{0bGE=qv(B=o-G7(1l#^uB}`S-3Ip$t8UhRX!>9e(}VH7A`NO;TbgwybxVM@^wEe0G<6x2uao%2Ek=Dx_O}9cn?2q}2n%jCnW!)F!Q^SZEllCgS<>h+p75j>{Q`q5y z%dNvr(Iq0DM`|bCG+}|yA{b$E!Yj#aGlG&~gG<3LGiNy*amK(uEn2+3eTHUuJ&Uk| zY+pk`(lbNNQ&)4|?z41GRPI;ka<{8}WrQ^>jq&E`fS6|e%qkRvP6=9D6#xE~EA}7p zJLCog`4(|xo+2qa%h?IH@Z(eY0?(eeJf^ucDi>wEOC{AC;hQY?`>{d>MIC`X5K}b_ z!7hk+!WKV%b!;+qej@Kj{DyvKxhT1soN5Yn&*K~%Tiw}u5%yMQ5~M&^K9Wz0?O$aq z7kp&avr9D1e>>%O`}}YPWlB6j#7Wdd|qF(#pHgfM{NImE}L3oJR_*C;dg8*@uF9=ksh^_lDln zKdb*Ps{gjw;SzC12;|HI;6TDnVMcv5(C?5fbWG&i${Nd0CEDiR&qcs`yXnW(3D#Sy z?rfY>i<_#yauU2^t~#R2iksk7*8lw@DDk56Enh&E{mP(IRAC#i5k-Cy1WYJksf+2@ z*G<`E6Zp9DxUYdE_|99?{H-ozNfbjQgzF4$D%m7hhc2DMKj6a%yM#xKS|%XElX+8u zE9V>JcXBC2xAh+k39I~dJlDH>-h zh{MJ_5r1C(b%IQaD%HC|V14)y2d*Ce_Yng68<+uaeShC06cG6C!L<3FK)U{W3lsm^ zE_aa8@Y>kUAZPvsc>Y$A2H$oJ5Z4uC!#@B|K_?hU`a2ZzP|#4@Xs`bZ{RAcQLH`{E zaGMs6>jW9vc(C)rrp@8}rJ#}#%H?7sbZxN`X+ z*F5eg`Uutt*9g*xMchZ(#G({w(hEIPyygV?eAqVTI|@*5LafV13wxwGVG|y(;Tmdy zXWYAYR+H|6%O$=*Lr)ZKr2jhfA$!RmU3=e*sf7^~RU_q?aw#>RjC`2GcF_})hIde> ztF2tzydwUcJ|;1d<_$pmrh=Q>(ebUinWGN>i*7J#QV+n`?i>1w1IG^>?!5HZPo5K> zsio2}>{bvXbNeb`Tz-vnr=|mdyyf9jOH!S&v8%s%AxkPAoCZ=0)B~+YTc9r;VCT8P zP5bFGNgAh6vsT5@FNLwTM(gn_z^sxgN(K~)8T5kuQS}j8m7|f|d07<(r^`=I;q?no zY}K!_OG?pQH#=sG%5LkjlJdZ$TqO=qt_gXeIQD*DtQxiK0RyuyO%GEY{G&IZwH5lTamfnNNxcwJ{m zcl7`(S6^tt&k_Esj?R03AJ+s(T(Wxeb zHKQA(PRdodbXb^Aflw_*33oi8UFX2j#?D&+)c%Bp@C@-*ex%d{*9pjTo0)K%-u*_J zS9kg0K}{`DBT*+&y#Wl?^;HIOBf}pN{6eRqOrK*9*C(DBJ=T*=gFve8CBr7ca~;z7 zHs!lf|ILD6skaSCbQ?gf!?ekMhDGa^yiaRZMeRp17M^ASEsJ!m*+!42VKsriVphiT zYG~3%pVU=pl&dXacqXHa*IY^XjO^kpEngv{IJR*cvS2Cq9`~)HO`6hl^j$Qg;Ma3$ zpYWD9R@jm+hmar_bD0BQ7i9+)&agL2FhM&d$3sDa!CWFx4t+j?O?6AA7TGUpWz=%wacfb zP(1qeQxj17JUDp$k~}J}L4037qy@qe(mL2N)ioJJE{%(A!Ln3nRk|utZ@6BEO6GD% zBh+f`*ZmSrxT_}lDv_bJpI_(urNg(ww{+=cC7O2TL-wF9!oUc9kv^T zh}Lx?LTzFGUQZNo28$2D72yWqf)0S~SGco)_OKTc&>qG+TVaO#1C$k2zmg?0XoS3VeIM8R|5gw_z`M^aSh`JzJLe!U93-vC&Hnae18HSB-*cA#mU# zElYqYfWuZx8T{LWX|E9lQBoic%8#jcxIV6WCkk(=A!Z#YaDmY6o4$Dc4R8f5){>V=jX}l1nBJWjcTMfK7PD5G97!#-;>yvD)LFGxXtUM;&#Br7NodJ3bLHFo zvjpdxVrQV-2r}vCIr2pD20mi_!!NdT?wGGCiGuW7FATEEr!}>ctk)?kjM<*jtuP(j z4}E+6L6-->!HK06V&A1Zp$max;8()893r{WPnVL0r62oHdx4i~`K@LXLw4o+S{-%d#`X=5*7O2_rMTw2P^sVVeXlfQA}`iLrN?f0>FUX(XN$`7I7 zq;acT;8!;B6NLquhGtWE-U6iGF!ZsD6n5yM^3bJzk)322ZX!i&XlGvgtL`ERMc*;|>CRWb52cfv2{4fSP7gdwlk1JLe}9SjVg+fKQ-i zpBAfbEdp|TH&;f0z!@&*z~NC{o^1jj0kvtnd=-nq61r1Nt@)=rT`pnYs7+x06K)c_ zpIG_Ri&Fw1eHaw(fIabDo&wz;?~Qm0f(Jj1^d~?Uenf#U>B>r&n&=hXbth3z-htp8 zsEu4vDn4H{Ge;J}p;*16mr<2LRFgSMm<_YS6{pJ$-q4QKnIpdvO!u-gjJps?K*^#F z>ARaJAh5+a$B>7e@<}({a>b5ua*@sK5@wJ5ZVw=xm5nR|v?M(;kE%sHNa^c+l6FtB z6`pICilM>FNQdczQpHOmClrsmtQm;_z8XcBW%zx7%%4HM5tagR#T2zmm6S%Bh=lvg z3fxPbivg?J)ARHylWiSTL2^wkw6%l5F8nO`<8I|)&HXF{^KL}1n6YEpXQ)jWD0t#Q zV3BD99Wp`pT#M$j!Xl)LCowr$W}Zh+O(y{AOc<_ z-LX?rpQ+QsOYzs5LyXYlXlEqqb3kN}oVnkFY=w;hi$r8$q{tbs&Ge+#G%sga*{ARR z&q(>O@T5T^+4lu2;ffeCR51$e6lO~O?I8{rwTcl65%LkBLL{fJ-oV+j@U{=KP|a|B z$Kh0dt`HwT8obes%yB#dlzJmMQuf+o+~CIW#M(YcgUzGT*ds{rL37j=csU%kdoeL^ zKdd%-q49a=`gTy#WdaI#R!MTS54EX_2<-M{T+_<3|7>2skIct%eLLmdbHLk~Bs zv@mGiL1^8c{z)9BRuAMecWK_;xZ}cIBwpv`cr*9iG+C$j;=tpsX)d&0T13*ldxEC6 zGBC>EY%vNu$i$d1gcqq>@352NQn(3+3L7ej$bCzdD%GZTpzcXY0y)S^nB@zI>U&dwytXdt|{MeotB>XmVG^V<$Ht z>i*r9xR5o=thsz42WOrV-&%FEQ$S7U6n;z@k{NpAmfvWR+T$UoSylcL5aspDf=~5`mSz@-*Ab|-i!;Q;wvD!+Sk#S@A$Q*p z_q$wgz%0yrg@6D*jZInxa{%{o;6g39(+xU2jSb7(k_gFnd6(KmyTuop8yRY8iQ1Cg z3~kXJVSsvB$g(E?fs%kgXrk`>_YzSMrIrhMHOStJyytJh#f@n^ zJyp7YED3e1Y+?=@TR1KFR1U1t8UcPED1gcF47)QmG+TZ8{4(_R`Ua`W_XlyMDmvix zL2}da<}A2~J#W*Sf1=*Q^VY;JX&3_;1v@s69LIghy%dI+LRmkrsE1Nu)9eeVUM|yW z59gR9JR);!QFbsX&#Z?w(kzrs0qYlMNqn8fHrXk9VQ0lW{&mbUu*xnEL zVSE#d+;sS`GNya@e+#BnfJ0XU@_|W2Sny}^(`DMGmsB&Z#L3HBNayADZW|GXOT>v8j+nNkKvoCqXygilfq(m}fB;>LD$h3Cjmi(teTWD2)|u6Hrxk7e8KV4jw6wY8`_OFLxfaxbn#qqsyv2$B&Ha4>y8~bNF}787 z9Bm(@XA4>Yv&*MSQ-L<7U7u?u8p&@%69FqiFg4@dh2Hy^wB5e3e~C>)V2QvVViay$UQVWmR_w6*_tsBz`<|Z0&c3WfZPWfYjL?;9i^s6C z79!m)ei>%>cLzE1P_fi|y42u+JZS?5B*9++nN!pO4oEg(&U6~(>~_135j1t-)ISln zC!BY`sn{nmWFkIh;i-?TG=5I-Af1c7^SWy)T7;GQZVKGNqeKdQvOwbK8m3~GL{im% z#*DF%?q4OmV-3C!W(tp5A4If#mC(p)TAAwK zx`xa)J0>`Jbs^O!-48W~R{M|m`rFtC9OeJtm;`0OLJ0tkRToULzHZeRw`rIKkddOO zmAPXdT@B;mB!$LQu`73^pGCRfTczCv-vwK~D-jGGBjZ|(|- zkf3sPu|60W6=v+n(Y4@k&PI^_5PI%+Q0mLi#oFb*1Ji)NDiS|7IW_&MhHy>X4@X8M z9tH2_%Q@vhfrxZau))hlPxBO)i*VEVFnkGnnW7kVH1dcWBrm@u9hWoKG2QE+9lR6o zPR`=xZMnm5;l`9p-HnV=vVryYt=Y5H9@@-&>*`$Kl2cB$XC^K4spM_>P`-f~0sreN z_Ecr-&nrG~t?Ch0(nNPxW0nng2lwI_A#;gTJ(-iM2vqPIx z;7dQKF@^VeKUO!7h9_6OEX-}QjjEqqrxPe&sD}7=xXD~f_E+*b&b#habISAPj%_N9 z4;BGs#oASXPElws9WKyI?J~7qp&95=#R|fW!UW>v#yHPsa3ytvR^AHjVu|@m7_Drw z+{<&lLMG*Jo(mwV4;k$$ql)0}UTd#61dm{7I8^rlCu}I=psc1g!PD-%j^Vof zZv?+h8VsTn=C(7_d8BH_X_gl5!S8G zLAwg2`wg!7lMlS4@v>t4p`EhDT@Vj9Wm;@u%qs5@ z4@sbYV{_BtVw)C+^dGwhF3ne{8uy^xOT79$okE*J=o(SI<`I9S>`DP z19(O5W=bN0(e-gK6>BSiBHz3Nh>$atZ=vk29@`c5e?UZr#ISM-QV)EU1Ryu~ZAR1n~sI zmW7d+=_#m><hvEx7|b)I1(A_M z!GZ!X!9xzDm}uC*7JWoN%_aQzyaSrwG~NCQN%J3;g2WbRs5P1;(3+dcr0Hzc;gGnM zKihFB+b2B(9zMaCVL z{QCO!-S^UK`nTx2+q75DEw|9kRJS1n5z|KmXDvW-&?kmRU}_KvG*JDjI_Nutw@TkF z!j@KuB)YS7jP4+mv{K?Vsy^0ep&E{KBVF1Mn?9eFIz{wuBVf$Dm`P^LyqRg-$|Zma zB}F=vX?WDUnh7Ooh)-Xs#R5r(z1o^DXIBZV$xv(AJ`O?4!>1OMie$rPGzvf;C_!3b zGnxPp53mve@#0Xzpd^#Xhi0e5v{?W2 zkKqQ`ppMNTNxv?$!>ygNE3U1zXBJk5pQ|1UTUS+x5MN$M0Mrcy(NOv!adX+Zu%U8A zT1FV$u{-JvSqhazNqRR+;-6f7h{Qlc0h3aOKmX-k>kjS_Mi_wH5L&%4z20#ui} zk|h}>|Jc&C_HQRCNSQO1!t=dMZC1!FZsP$J>@2HkErc5SNly?$1-XEh`}RP-+}YQh z>(-0%gU<^|yHdrB*s70&Ee~p5T-={N+Y;4d%9^(ED`h))ofFw{X2&h*$Rs6tS69Sl zm!2IhVxOC!M^e8T7f5NWyA+3Z_+<7-jhwb?_3l>~Shs3uuJ-tH+DDB1pk(a8EWiTV zQLd=xh7scet-X9ZMcRMaRoagVWqLo-EW{F6S(EP2-E(Dh%IAI);?A>C`x&&H+yb!w z!E$6|y$L#4jb+!ngW~MWAli2J;98m6&MDQjv*z7w(MZl!7A#^Tt+r_Uk~!rNFUe=l zvq+LNC{v>t-D1~pbs`Qs?#`HYR@MNt9SV90|3dG^x~aM$@T4>&av)5T(b`E>x)pvg zzs}^CHf{O6Lh$k2b^n~O7n%0u|PkNvwG$F&+fvtNIxQQpD<;7ZPF)Gatu5_Q~%xSDyNWxysofRX`(_*>ImodlBn)0hWs#%uHo@JteNe&MZ z5eu-;Yc^J%iHmwFY0ZjJ+86x z;3)BE^yZhqG-wG(t0cn!7BNRQYlqk!U~eR4FRi5-C^u5bTkPUg7r#Yck+V)K2TCfv z?I}*33K?K^lb6&eJcXAUXBzS_FVbQZEM*hg+JyfZlPGKq<1et3jpk|A9IhgBm=*jb zYe$*+Y?HkrkC;HZpyCikty!R#uE*^D^;l*tx-jog*pI_s)NKBFKn>hC%fkj?9pS=9nV@^L`#GkA4J3fC!%}#C1cKE7Ro_g8%F`vK;;2Vx}K94c> zrzH)5Gm#i>;u*#Id+nk z3yOa@dGUK2eIkuJ761Bf! z9BZ*`AU(rtwl%?ZP!k*sS%AvC#@7x7i*6FYU=%S19*nvjvZXrxXcUEGBtv`ZiZC}p zHw&!Im;09V@t#2_uIgH7c~{XWxXz^puw$HF^#OU6%2~?5aUgV0eSn7d`O0 z@zm`8;Y&2AjGdm3@-vulY#cyr!}_IL!v{#<(HHQmxm2C7x+!U$piN6ChyTR!Zhwt_ zEL&X2d2Fu{9rGN@srpx=d`cL~J}ZKjjW_{poh(E*_T0L_=yZR8k|AhV9ir3FW&Pf> z>gc@B$hwh0iAM!#`6~@VIt0x+8BNxVlLZaISj~}+`Aym>LL)*mAJnj&ohqmF#W?KtUM3QLN^H4M+5vSa^xGxDXW z;MNf7wFvuHF%!}~X1}1iD^m+bdK9IV;d=<#1d)@&@w=TXqDDYGT)>h#Mwoy`7sY=B4k$m-p;-D6;0n9 zof)rZ+zgF*qFyo}__Mfad}1LL6g~3i3Icf~T-SLiv%`nnpE;8ruS8mxWxQ!y6tcwc zd;RIwkeK6ZL(L!zTh@n#L6%mqMrU~?SN9cA%l+A@LS**-hv_l?)}G1?J1s)^P^TS3UyS8$bc+5xE!B5mBe;@a*Xfn zZHKeV+4sV5jd#~= zW_FXFj^Y`f&`ykIGPm5AZN36HZgsaHK%$g3*h$v67RB9-Yvuv zF9dA97h-zL6H`O~P{z)$N4sx~+A>UiJW{nLdkTEqBT2FLtQZBYB}F>kW5dp;cdW`{ zZ7Wi;J^E(+Q7-)Awr7oig)SDDF5@w?ONY7O5|A#=5HA&5K4O-_LnsBLJc!b_MQ@@H zi<8jfQr~#>&Ckzp#NsNFli%l|dp(*;A&N5j$>+px6=z#|yU)$)#t_S4^Hn8-(tvJ^ ziK1X6QH{}=ln&e=U9YG&9IWl;XY7GWf=k+}161v5*rkL^8?sTP^Y_XA-W@fHosj=2 zrND<@)(|E`=x}%7>;TJY7p(?FbmpwM>H4;)-p_mY@p?!-vQJkih`` zEffi^qliKAq)>3c8I^u9v|$@~wLZ~&Z?@dH|M;9-zTRJS6*O0WDhhz!{TT$l^Xy?cS^PhBBTHn z-kfk2Cy6rb&4voTMj`z^5-iE=sBf#u;)C2_3`xaz>0azWjRw!z+)n^Xf1`5u6d4gM z=QBz#cGqU41=lT}>7|W&!rw?Hr&X0Df;A`R6qFS~rJd+D7Rt?v$~WqKmYm*(A2PY& zLgUnK4Jn1Gk^*s+Wzos5JDnXU+y8X9BL-Vsbz$%GnoJk70c+0L~tY+)<;>BEnnM)QYq?+_OGY zI?ap@s}U-E5GJy-o?f$Yn8}#P1HRud28ef+58)!Ub{zs zQQ;kis^u3a$N)uBr3a>ZaBt|E?bJqj#IPI1-2qiH+gNZ+IpUoW9?!_SGsh8E#Gy;j<`=$xO z(*!JSP3KmXDv!Gpxjp9?;7ttT39F&5Hrk?Gf|?mt9p8z_J|}=`Et!H}nHKDb0DYiA z($tcAHn;X?Fz6U8ktk~~#){TbaoA508Lz8-exGyAesRTDYCzh;I_S+=^~2?tJ`BpI z-}S-T!||vODHG1g5=ai3z~+1|tz<7&wFLSYmENY-eb<>z39~Nn)Z#^+xVm6pjKLB8 zXyIuvt45KL2}~y|rDj_BnWkdmvB+S`PUoq`6NbW4NVWQ;kc(jBU|=_X%F!dxF+;p|KM|d&xsj z+ZYaoOg50vD&mtVqSX;y^h0VcPVEGz7S_GU zs2NtO48(Ft_t?_6F?ofB;`1mPoeAK9u>RwZkg;z0i`u7&d|BRaiWeTaIjgG-T9!37 zxKW{{w+7?lKGwl_V8&~`L(Vg^1LwP zOO~h06Su&sr(r$o@;1imz&#ldg|h|Yque5aIcaIuICn?6JEn6{*Ctle-R!p3oMG&P zdfGNRQdjj?eVRwNsQgnUcJqQ!#pq!a?-tU1Y{u&!*LE%FH5X)_Pw3=oyS#zX?X>8@ zT1wq^p427{s_VqaC~Mr-#*bw!=>F-rFQhT_kEt>9mrW*vVyx*j1VV&y1mM2oaEBoG z!OYPX^~`9!nmCYW63g%No1c(36?R~}U`UOuq z$EYSiEA2&6sP?KV>_n@EnT2Osis9Z{Lx98-;h*E1Ob#9I_1X9acy{HQu(ja*D7!qG zc&CvYj>i2YJ7qIyg!>hf)G#w*Sv0?(rI7e*vR$4Z@Lu(a0My@@7keiCWvfc<@jCs? z)$%f-vNxal2L6UuJ&V*iD!^yX#ebeTU`rJ+5tji91H_0878(E_H=39>Snr`mJWPu1 zbb4G{>ZX`6Ae`n1TrnoBS)5{NyK{^*G{Q!9T*jJi-I|B36TVfDNVCUq4!nc{j|M2bM|twvGh(w z1>zSz;N}oHLDU#!#y5I%*Dv<)U6y(%^Ea2jT-VpYr2noqq%B>TOV(i2ls?#rog%BN zrBsA2u3MwiAHi>;a%LEhufpL)*S60QkhLZj2420}Wiz%Jde6=5D`m8IkIf{gGpMrj zzN;&NnFY2+Z79}CSy73S{uK>X#)9rQ<%#lP!23URf@208#5o0Sr0Jew85}X`wL>E< zdSSWRDpO@1hNuhPaNYJlMrY&tlkTG}Zs9i-eiVJ0v~9pTcc2&2(t#x|ZzUVgP_HEW zydFdF=E|3rv0N4&d#t2X#&)ALIXiD&b`p^*Vv^!4yhgk1jpof^XNfU+mBdcKgV7Nh z4?G>yT8t1RlfH@3XXbX;zp!W#wVTv8F0q+w@Afb*Xc5+lkN6a#UUam{e9fUwi(s3# z##`C+;oi-kaSwHH;Y$dyKjS-$j#Q&&wXtwHV^I7wfxHBR#V-~Fep@O^a^`|in|rmp zc|SVz38}(sExH0~%`Y5WZK7J%XGE*N^MJJu{sfw4%%!0R&3BHrDcXWAUJ68tQEpgc zPaN0uKX1|#Kf9zNr@yr@JNQ+GTJdV5w;q)ovhZdAQeU z%?p>!mT332K^o^^`wgU@GB<3YF>X>e^Ps&8nNU9w=$FC&p6yV6!>+M6MQ^bOpaVr( z{L%02lO51K?2~=bxkWKY0_l?sk4P+!`*nW!68f#id8`J~oKY2LxQ_4mp!;*ypcrhP zu$I5jllEH#F-s|fikE9kVIne_*^P_=yJzW<=EiN_#bqpy65)r7k77ex8P1%+}+*X-MzTGl_G6%2v!{aJbV8R_WM33xpJ0kWxi|H zteJa0_s$xxb{fqgMuU&BSrOx0@~~F+J@)KhA8a_IKQQ~Ud}rQYyp4DNs9jZ&0OxXt z%ff^1`HbkX=k6oC>xj4o*?ri-Z=a3wBfL}$!w#7(^yqPR|5?Z;$|UNzFb$~Ni#5GY zD@O6?Gk`ZzM<7uVyk7;nZO*k|M25~MtG*{j?pb={7Qm7YZ}o+!aCe}TTMA4#&B5mv zRJ|fL=Wdwzp*ec$JW*qsz84j|T)*J6n!gsn)8a`Qi8TyMiUwg4U%;>3L%2IGzqdS( zbaV#`t}`dkQ8Nb<(X#o?j8(qwW7!@PgBZOmqVGvGhUlUx1qSD>_)w|EIsd;(%2E|3A%DshXkv ze>O0demX6Bv^s7+sXm*w{&2(lDa*@T%<$dkb-H!*2fjZb?h14JGd^`3n{SLo#R~|p3Vkd{-+`uv zv%q?YCiD~TFiq+LjNMgLgf_3-P`QJn35hpD6bD#Zx7>Eb!o+V-TlwL#q>v^ zApc*5!vB#yrN3~%ih}>2(y3H!>VL^Z9gT?J+x86Av&l^WQOPn5|M&!T#&~}FbA`5?u7u!ZGOZ-|xJ|(<%pXX-W^}n_{IIDt;zBax( z*9|9oS>b8XQ_OW8I-V{oKjcuEmDYAm>*6>v>cI!(IC%8uch)FtTfqTrKtZ7FxR+aT zQp0oo7&~bj!e)p;$q<;VlYkgxOgLdqjFtHN`?P=Yq5qNW`dPRCDLDGMBFwn%2Xon4-NkgK24s@B`!TuNvCbLdByq3&_h!_ zBRD`6cCci-P3fP84qhTdu8JqpMFeLlp93XDZT&$?$oVY)@QcOz*l@t{AqtII;-AHs z;XEZr0sz}~wy%HKg)!e^#b?c=D~${V0utxz_m6UyE3~V1Vm@EWIou44 zKjK%j2jjF$;dFr628uv

    4c40L=dH%YcM%ec;cJQ{n!9P6hn`;|=~NA*92y!UDk| zU-fSc)sRZDZ5ec6gXEzQ2~y?t5~6rPTkBg5iM{r<=Q6K!L>Q#NFLAx!M!u5{#EVVq zX;FOVl}vlJI-Kdr#s5%`hy;G>i1i?q7v>jE3#fVOxG$kX{zQ!UkxLR^nVuCvv7E_); z3R0bgvTAZ?PEM?h3fTeNL{qHRyonKv>1`ze$^7^L4#B+6fUq570$YQalv!_j2Wlqe z3&^|IECdvG*4m9C(EPi5uamD!OsQ;DL4&f`RVt6ky$<*UsGTI|aV*;sh$>~uEl%f- zSF)baqqYc%ow>9jO&kQp+4jvWKH@}ogcKkV7fYdo`fuQKYYT81oO%B=80``5EojYG zL`cooG*HI1YTG`4_o_G$+I;xBNm?1BtTSf4J%YPEtR2;(lIHvNKdFFq;ec}KBk217 zC+Pnt=3vhMl@FDw$tz=tV!UM;W>^~dFa|}D<6`0}IB@OY$fTee;cyLlLskXUV-&5r zXoQ3)U#NkkxQe_GmA z6#N<8yv2oKl(EZ@anBRCS_i}Cu3Z>W@E9Dgv|O3g$(|{At(5Hmsi$WB^-p@$QZ1GK zQA{mAZ(@W$DOB+^$3jkX-~G5YS)5&0GDf z@4;~2N&4MluXE_^bzT$$zT=RZJlgWsuVg<)@)nrnbTDy-)aM)$9zM)&3Mh`I68@*l zcnq+eNce~z>>sH;J<}a2WS}iz=pz^8$oZe*wT9Tg9ZkO0I zYI#rXqd7H|!Qc;fDymtW#scpXvdjq{j$amGmj^fJ!~-}u@)^vggM%r-_z2Bte~G5M zX$L$O;s=qWR4PqAiKB#WZgLIV)%viIz@mquJy_bMqMd*f1#e1L!JOQ?>MuvU4gss$1ml@t@*jSn#E?2Ik2A=o&M&VgIE-;gnVkQZ~69~e|kzw{i^!XOs*+~DFCM| zNwh@dLurb4KI^s91x`1bREd*FlGE2WHs?L^Q3!itiQ}FAREkN0oGmj|?D+N`g5oEn z!E(t;bg!1rLibDiEzxYTtW~0|)zDZT z%X#F~@bz5Ls4gEdJ$in&-%e!Kunjyb)3s4iOOBE+rleQ4inwQh&!Nqx_iG!W|Ku|+ z)eoh|Fw%6w1tHVxXr``honI~*mB}OLOz%Q+8E?e_ngvMtCFZ*rA+DTXH1M!O=XlUS zx6uE&Zhelz)5VWULhIww|9_Q<|EIiw(uGQA1FGX;z*5{#uk>yxp3y%Q6bBxwt-C58vq>Z>?`*a# zUd!xKwrMFKRqgtfi;+sfzmd9|X;hB?P=XWbAw}r_hO!1kMP5#MT>mIIUDi`H?98bn zrR+PQ*bAD#&XIkeaA*T@LeUU%F~QQ_ynT6A^(xKj9%<-)A-*n>>=^TmS=#Ff*K`*y zIP$mH7msPHY(8sV48GtQuP2^Vf#)~or8n)#<)0IBJ-fNMmh;&KK1SEwdO46nCI??d z{u5{+%D)4xA3^5(Z^xw%c!niusk-CG@-{|b8-n%`@lh@<5_#X39pZ5$(&gXF5>)i* zeSEMaV6zu?ftzWLxmi#CUwz;=q+PT!$7P3QM}_r_7MHEj6x1DLLTm&SYa%SrswghJ zTmz^t#7QHTPkZz;GXaOeC|+Hf3nuPlhou6HxMfn`2&+osU$Xkkbv8<> zF{ms!fh8iOL9F&IYcKHpGuYNcysK*?a#?^39QH%+P%~F1Kv+u{Hxjf)h0fjHLC7$n z-ya!|5p&0((361v<_R_g^;o5r<( z!GBfN>@J^WaYFJm)WfEBYkt=XA;%09Kzb-}qh%hNs3;WiFYz)iOpb0!18dwHs11Y; z*-%vdbEK3!X&()K=$d7u4m%k~xWATfI-9QCJf2{*s?+Pmb*I zJd8i8e9yzG@O~44msMWLXNrd+|E?+o_Ed<{Vw~VCvu#Cgc|y@_ZvTxvUd~1B7g~fL zrP#Nbk+B`R^ziAQCvGq4!i{h}0o9f>-YNoCt)oT_u?7ex{r>b)b+*W@qcJ!yQu)aG z%hp&!IFRqpryU|BzA(FUv++%eL>H8c=$QG5qcd*uEri=5Xr`0TYjjPZa|9K6q7R49 zt=C+hj}$rf-?Gqi0a!C|Y0{Sc#|GVKf21roVY{QklAgVNuZH5j?|m*GN6s&B-zyXd9UFCx zv&OEYZZ9k>n_rF9-Ga4rs0E3`hc$;wMeIYU&5Wlr6!g8CKBW;nphB;GBA_+LE;Qve zqs16{S;vNj{+kjfPZw8JVL#K=GJDVTv15-i>e^~8lx1HUcq8z!s&B=P zV|qDsE}!Nn+6RvKV8PQx%|7%Xww5#4i2?+ScfKA^BaeZ3qy*#*|1pGH60QA`$PK2~ z*yy1$zuIg%UD&y=iSC%jTsl+=fI~dBzH||Is5WkJRo~Y239!u zF9}9yahtKbX9qFIG%;5%z?;POcj4qc%yE{M&4zOaoo~0*5mg<8f}_P^q;w;|Wbr4r z&v-!i^FOEIQayW`tCvF|ypwWJ{i2rJz?$IBA7QTIyRdt3xzF@`%|gn2x*DbwN0!n0M} zE%Bb9PvF9EJU1Sf+)rynB)Gc&KaFk&!k<9Ij{|e?-$m;O%2W^hv4O;mp2P^NgQlB8 zL75F31gq02*70cq2t>oxShp}vLh$R9{-SlNy0tl6ZN}hjQp?o3Q-Nq(wL-G4*Sh&U zpVzjzw7R4BN91N_i$V9x*!%nYpRu0HnUAW-|7K>I=O%EByDjrc1C!UBBL9BQY0$qoUh zLr8&d>cG(d*v%;c52#crBh6$}WQLd{5tA}#Bh_RTl140K6%s?lf$CIjiohi*HgVvg zj9D3FPErO{q#Kn>9;KDMSsdkGIkP;@;0$+jPf?T>j-~|@s%*x3|<|N z(L!a|1Q2|K&uFD$4@3Gg1bKnIiB{8$4ty*~JC*2cesD1QR0(2VH7I*wWUfcB8!ugJ3#D2U)I8lRMN0 z%Nr#JtwFIt#-?ULO!P;U?oThVTFhy0*3>jQQ0At*QK*Z#&~<^vo&%f1vCu0PfT zsV}4&`vuJx>xD%AQP@cff}n6M4lgnq5+t&>pqzPt4PSqKI+t;ODEI`&P4~b6-){vW z@T7YnG^pNnzJiDQHL8i!7o7E>0WXe9fS5-mxbOr=W2=5ulb|xHkAVBIk$JuooS80RnVL99cb-(O4fj->d99VWF>^0R7Y?YzjB)QlIwwJ#PXKTDRsE}MZ>c( zsMBVgw+~PMX(IR+&YmziJ}>`!fD2xzy{f?Zz{;mk@&ndBt=#Q>{WjhTiYTMDROA|2 zsbC?ATMQAhFH((ed`V)oNM>P&U8`a@MBN)&n6)z=S=%Yqvi~Wxa=r(y&?829#)NL& ztRLMh)(gR@s*4_NMa!@2@mp6=$<++$S4iRwg zKrU=?c)KtNV`j1+_kmh+-CLvN=hs;Msn!-LImblkTf(Q@$%#v5+scU$m_d#kAHsD2 zF|-tK_Y2ES`%C`Ig+LH|ZezskuQbInBg#j#60yReuxi?^&0Ont*=?_rQmjcm6_6h5 zNvGy^PwS%`E?bV3*2kQF3$Q{@b9+7dvm7o?tY;nMN}6f^53NZr8)trBh|e=&_}TMM zTb9@DQ;jBt-FSW2reBi4w2UXC-@%b(y}&w&%)<(f*l7`wW#=UrKCT}z)6G7>C3rY9 z@{b@ZC(C+x0*0}rt`hdv-P&)dc8OM{8J*Xn?OSWt;60Xt@)%X`@V;c!VN1hm&j;Xl z?)~yF*}YxSh>mQ4?#+EDz4b7Mo@wNj6ur}iNt0m zf=98E9ZQP81UYZgUY_>eE*|6i6SY$Q3mN>J%vm0{w((&0Dybp1*dyPn4c17T?nFta zN3hNzBLg0=5<6;&;~O3-ThuN5$YjpRcd~eZyJYX8>HMXT!(`tk*AU1b@u! za?1mWhaSL+9!!2&rzbpXjy4bn8%bKbWw9z6fP;GRmcJu(QJ#C(PM3|0Zi{ct9qmw3 zq9zIc>s?SH59(q5hYnCcrU`*cJ)|n}2g1;*>EofvyI%7Ar?TO{59BJF8tvw}7s9X9cZCnml^UzA;1DCc{_D*GPk_lP*1fJk!$70Mm=bgT?#zGR)zsKn{`@vKq zws*^1?M~ZGsDrCj)?s%9eiG-P0QT-C2@G6AC6hUl5W)Ok%3T;W$gmhlJ|-koVr;#Z zjpQ*D8vJ$>bj>@~EH#eCEhd?I@>UaBG9O^&P*UH1GKJY9o9j^jTssxi%qI&=<|slj zOLs8AXLSz=!ISFfrPBGdh-wmJmDt2c8#P%a?hyW=Q7g52wiD{308HI&QYU2w2Z|E> zB`mw!#otR}$F?w-kqP0Al`?YBrdb~S>}MN9sM5g|TfKc)8 zB-TCh9#7NQxrq|`U!y%L1FZ??Y}m^8zA7=1sKXF5A3`(j{6PuLwWz@OsiN1;FXL;f z_GFdnN%t6=^3_kB?oQIo&T@oPIuaTr7acSFc8da&A(GtHP zORm5-W8v`G*xKJvO*$(_$bIpascGs7)Y6)ZC7t1i(?o2?SvIEk!xk~S*e={N!A5;5 z`3BgEj~Gain;;hzr@+?}Ys4cl94P}{ltcX~L^v#(O7AB=fN?oTQxaI^Ff3)vs8qC3 z6$q0|jGT;L%#xeRB@>TWndv@~{Hvs)MmdW*fG2?mkVdht<1F8@gu7^F#lC-zmx#j= zLdBr%%IM2+Za6eNs5;g$U2$O9cE`%+b^(C{R*{K#5U8cI{3T;4No`RvI9C8$NytXm zB1`f1IO;h>;|Hd|wXFY}e^YsYN*vF6?9Jwu)%C3)yZ-n183&zNqDe(6Rvw?{URxQM zTleXtNYyw#Q7{U@bV3c&Z3U3h=vEl%@9*iT1g}WJ{bXk7JG~wrxIL#AQI+{x^-IOp z-_t`fFGI86;yh%>y8eJDQL+?FgJPS%S0lJors?X05`OD9G<(^|Y^-so4d2pM5x;Q* zV^@*vrWE*PyY3Pu80(9um!DKw29bNYn!d6o>%oOfj#U`TLQ#Rh(W#cWW?~6P(z=XP z7D4{wB#|5|1L!&!OSh*_Grf3ji6fPVX|O`7Q3X#8oioJD#W;e6LtX}aFU6O0&G%ej z_pOMq(4iy0s$!g2lIweet3xaG(7vi>)@Rh>7jEubeVSGg^S9Zs26qYerYfBKBM}sp zL80k3TgOfJ*5cC^3~tnT7i20^SKYhmR3oLe>ARm1%rkod359tk7BdK9GvoT`uxl~| z@r8M8E$6JcW+3~x2G9ujTyO}^{KyqUE2dOhk5}p~i|v|)fa3kp{I_vTLm=Zz4pnFF zZmOhZJ)6aznn?wZ`RO3tDRr;9G&2|saHkERM2YW@E>uCEk4Ow|6 z*_dDN%6Kx5Q4uR|t8i}{4d7mvb*hx^dE8Dn>a<~{Y<0gT*SZ)B4svP4j36v~^OoR- zK2De?#6WvRHGT$bVptUGq*Q$+#Fkp<*wDcl0XniZo6W}GB?yhzq;*XGDVvU$>;2t! zoj~cRpi8M)#H-H4;WcGdo7raDLD8)3$=8a;9-xdp6xJNczDyccGIwN?C)?u12eTQt zVjBnr;@bkcl;d1CUmeZ+4I!5MvWii4_YIY`W`7VU-hYF;ej{HsEZx%ComS_dx};Tu zkqcsNq2GpN>iOFn!zy{#Mghh>Q>6kpS0O`JG3d}Gf8I*crtRC_lzvtkgpDl(ifLqU z$i`?acaPCq_ckVr&Jt>J+%AYc(Zr^pvFV-PJlS}ZKlpFTN= z4vWe}%247nRxaVW2ThNcgw^36j{mMya)l8o8_l7`X1L{;Ode6!=pIp=eK~oTCK1aabG+T69hO z(KQ5){Pv60dB$H4oh3XJM7I3t8!Xz}3T3Rr%do7k(w}^_#idyh%#!lmeE5?&!Iz4+ znvv6EJ{XMpL6RWSf_8)fIWSb>wQxs^Kh!^827>@dC(Rd~H(po-lhCylbP>s;;A-DO zp88caJ3h4M{4NCQD?wCwCazw{qdk-N+43nkrs;O*;H56Rg%%4={ufJ9GFt5a6rv{}Ud<_Hb1+aPHe2`D)*OEH2bFB~{=g?FGdLNAG19Qr* zSL5Jv&OcEwl$OzXVoHZU@uT>Mss;i~Zix-iKho7V+N%`&IljK2e-`i>>ycNgJ9!+zCMj4c42m^+=tr>?^tT@_ji% zFZgDuaXVyg3*Zje_ej`7HPiLiOFIbGef!D?rc9&qE}dxlbbW+Akl1eK3zs_%o9n?( zJSM(|#2+V8D!G1l{=w6F6FcBBMd|@dw)LR1uuHzeBi+{iv=bRyL2ewLmun(-$PawT zgR0+h1!|#w4Ng}*!wLXdhW3sdrK};4y6O|-gtw9!0YIPXgv9dp8ib+Aeg>)|hWYNT z7mt~e@-1q^5ruv7OFYEWG0eLj1RAC?I)8SyBV;Iy*cB)_>t@AQ8g%DB(Zd2aATc$d z)i&VY?1{B}$9<%kL^>P0!WaQQux{>QC>1dM3cciL%Q>X!-lU@^`C=||?8uLZw(0Z2 z+&~%*W&4!#jUvH0JfG`<@vONg4z}- zSx0o-;I_hFii@O_J}TJ&+h<9d!i+O3r}xh?o{^0$nA*vv%DJ~dY2kD5@p0gTED6eh z9RE%APoQbv1EWIGm^#f~LvIAa?_lX&U%y0s?s>Qu&=lH+p-KK_f(LG{#1b6Zgz$pV z4m!*4!C7$54rFtXPY98z~M{LtaLje!r;8`T4<^XSIr1nBX7aV^I< z?Xl@k8QrY)T6CI3p;%R^jGY>l_}`>1p&ah`9XtoY0iXky`37|19c0o@t->#XkL#7; zx5ZR|`%;QMH`e^HXG5#tz? z)-JUBo~BNubVHO`A;CxffCSbIhBx9oyC4cj%3t{ygo#M;^hH%exC~;Q_XrAi;n`4m zqBc4tr^l=sKetDoBiM+BWFOu1Vxb={**cCSMNJ;eZ?gM1YWaWJgt!7W2V-kFV`xEEV# z0k8dvfcqsm5s5Dk0VUh+-F+q1V(}wJ-VI`2*U795*=G^q6ZEmV!Dk4cpEgLp2XC}^ zc^}Cs02>f=_%O3bOcZ^-_PJ!9+><%pHK|4IkaVFcVu497XdaSE#`|K7os^8y&`)ry zFQ7M&e~AUML(Ld*dM+XG+z#O`;^@Qt;OxP&h#AC+88%S(%H^qECiq#=qW)(J!uNkFw<(^TO?ZerA*=QKMY$|Fg`;PnbHQKaz z;D|6^CKI?P&sx-T0gjbvnB?CNSOfTInjdPmA@;u7hmd16Kr@tc2$wfz%~&?w;jd5U z-V7rZc=Q9Lrt8ZOv z6|SD_kS#qnDB=3yEbW5^y`%irGIXWeVOeN_yJ4_TC%K{jQ7eiX{2+x!{!cUx2wFz4 z*LURjMM)2y<-PKWTj{~@d$phcP`54VyC-{I4>P$NmC5x}%2Be2*6mR+QL8*5Ym{dj z0AvA{_68oAbslL(2LmTl=fs9&JhlTj=0_FinOsIN{6CVgo5}pJUiVXtj`93t+<2F@ z69D+`v-0*H{6Pk7OUuF9GUtU4Z6u*R;Tvo!JLS=OrHI}}l;+ZMDN$UalX z9^9?Vdp!HsFSbPfp*yBkP~NlvY!tBDhSw7bc#to_5aBRC&xBYuuA=-?Y4^}sE<*;Z z1uozH73Z`^ni%QC<@7>Y02DO-PI_Z8<*JW&V5 zZ3L)8l{O^Rh|-3{a|Xoyrpy@5+!&-}!6nFevZvVyy@N#v_(9DG%x)C(Zjj?tS9&4( zDJo@SAjg-;p@-J*l|pISD*3MHC;7|3e;6$4`5~T5>$^8QepfTB;Jx39FSS3 zEXtwipX_7%u_LpmkVM^BO7JRD$IkdVY z526n~&)*|_@nF}~BeKQ(GMRLhGPA`@nBA+g?jQQ+&F7Lbl7~5%kT>Cuz=i|{nQ?|; z9Cn`e1oVUi+&Siwbquu}{c&lq%y_T{5MOL^!k^KiXGd?Q5G7Ds%idDwObU<2iM%g- zcEF1h;TO}0SHweHZdiTB!^QOAJ>7nHtkYC=vj2l1?SMphl%w(xyz>qeboV*0`^H?W zu&gY@oN)DueWS8tt4ivND8UJ=dS=0w_|FP0n$q$yb(3EYPmeNV>ZZSf<@tSaimRDM z>D64Jb@Len&~pF5rIo<{HE1MrCgEV7PSAGvoHOjLO}&GmHT@8!;Qu`?f6$vR7L4*Q zo9z(~HtHt!2lh^&@Q2So#RX@J0_Ghr(qcM;CJz0D3wQHx)K|wTt*WS14fMy93INt9f ztq(iUNXz|lqW?IBxpVzL-;?9A#gtQ4Lp%tfkJ- zR^XY02j_sjec!s-rCGb8*S=b{a+`B+r5uEL{r=`p+V6GQd7tBV(RrBXcQGS)H4L)_ zZPL{6fg=aTrGtV(xPVIF6i`ygIS`%-MHVj&f`Ygv)1fuwMkS~`bVJo6H6%^B&9r9@ ze3ShcKKSboc~QQI4hhR*P!3WeTFExZ1Ze^#WD_V0Wq=MzCh4%cKzd41z?Cf^A8@4$ z$j7|$0T^RkNdk<4SC#-{z?CMz81u>l@C*A&2=Gq7|4bhE1-u_l{-!t*FY(eHK^pxK z8S##Br4D$f+qWhUWZuUm52V}ICV!J30Y^WyM!W;A2m$@rR|0@{hW+~@6$980TVqL9 z^gN~@I7kd?)gFK_CcJ;$?-VEo zhut8NO%r|`@P-Mz|2TvST8^6+k99 zhatq!S@;$o?M*h6XAWL`w1z?dmi2LklF?5@GY$1%)VR=}D3=@`NEP%jWqe`lhoC+C zFyT+I@D><&3Mg9eiG?ABIi@wJI78~(6~X0>AR-d8VBdAiU@8dCupcaM(B7A@U`LQ( zT=TF6qdA^sIGv7VcqJr`xNNZh^So)ec&6vaL}`yB?0wS|?Ijk<(K7@2T6 z7{kC|4u9ux_=gOr`$V=VH#{6xqA``ZhLpm{Jq!3$Q|#dM!8uS@jEkW9Xfs3%e{K{F zpJf!&q6c=%!XO@%-6*P--6(9zvL|P(Bgj^p{hXfpHN;k%;{^+soiWJ;MjJe8XAm82 z*@IR-_25>MeC1pA^V^ig3!GA-3q_a3b_h6p!kc0?>54jMlvR1uxJY_5`$_`e*>tmZ z`;psysskTA^~$sEEox6Ghs>OYg!!P}r76bCv^J)H@R5CIrdGdWkA(R(&Zm7}0l(el z&M52|@51^)NIa-8TKaU~rQ4+~Mo4fIW_mSRdh+253DtqPcT~()=M5T8fAXFw$qN`= z;hBPj?4X!ed(;pm{>DFl;K0%!s6GCGrIdPwYCw08-61h5a!|?s7U~W6HJ|ZI+jvkg zZwOFO4@a(3BMe%Xc1%haPjnQg$_B-h)FCZ2ahpHNlrt~n3*YJHaG}f@Ei-o&fNA^( zV3wQf#Guqx)LH^Lq{mL-hAf5ebpfpK6CarmK=k-Q_*!jy>~+*&(~LKkSgaaj5jHd| z?)qfcu)@VtAXUr(pm(j1*UGK)F@GIbY>W_Dm-}6X#Yb|3VBo>VR5U6Eu%n%`)j5lA zN@yZ7>m|65wqG=ptt^zOmPf7L;hy4m$IREd87_^2wW$|`a17a zrreV5CSjm2N{dBx1)Ft}k>Hw60yQc#USCjH13{!6K>Jxk+q^>Pmr<-Q1tM?RS7>>z zm2$~ZRogtA)gWP}qjMR(P|sl`;%tDGX1qw_0M&F-a(hguoX*+3-5_oDph0hhM@P}- zI?`K0H8Wn^m`%2=-ShMyHQvHdVMRWr#}NY{K0$Wr6V3ySp2)0NFE}wZQ|G4$+HE|v z+ddQ1lb8nDaxb5<0=p9yJ@rb!k1w}YXxOjE9@C(4C{&neCx(i{ug!SZo0z`KSgprgcVIe2eaSGu1{7p zFpPj#OS?}+d7PkkAuufD`$km*>-r3~R5|xdb^9ID^N96RSO%%Pi#LXgG@R-H{GM<1i z*~HOlvB!{?s%I#4D;f?=538HMC1Y3?3{S80G3oo>TwX}21ZI5M3>><-&Ja~AAfrO? zBN6g;tEU1J8P|ORKJLh^poGrPu{B$>sh%>r_V&fqHlWOisZN&+>YFV2hOGA~(&H z_=(nCWFk7biR`3#*UwSHa>6cVRYoSBf8bv8uUKGsz>5{GT(JhZm+2-m_{SBmiJyX_}uV>gdgk^dBUb4eMUFPX5xxF<`vf`MTMJ|sy zKRn!QwSryOnq((v(#rB}lm6XSF7zjL~%32si_j2diW)$!u+52!ah6i&o+E#ZlbuUtSk(3QTe{eF!COciFR`*@$v=x-_0ZBMjEuz(QR?sV=}N(7RX36c(~P5; zfPi~a zVXSt{mUz@3r+NYJToD{?ERUoZ_S2>*4ZLkCb-4{^9dOcWqp@ZZV#MYzvLspl*YVZC z?r(xLM&#rwcbFc~Rx|j`#ud~iQG8-po8;9>@G0hg_iTkxQGH2kGC>PZb#Q(Yq@HC4 zr+q!QM5zy6n}8)UziM)-gjZI&w#Q@jxMa<#osb9Gii6m|;;c-5;-GhIyDxS4VE%HN zfh>!Co-HOW{YJ%D?jtk<2`clQmYq>qahHK#@9wgr`uz1TUGdmgS6te$H+c-#(&-4= z2;%m(DiitsE7S+&#)oOs#$$ytYxLqXJsYJJPt6KizZPG3nuwCFHpQ-kI~4qU&cYz>7I+St*;9PaTg$Hm<(Ms@CrPAq7-e<-r5IRkfoo zqDC0Yt5*NYg6)3mWm67wij zmUZ{P{2>|!x`kZhbPeDS9?vj(F%p#P9z)#j(23ri+KriTAaTO>bwTcq(m^LK=ek??NB(29Mgq#6CmH8io&20H7ffjXP2gzvB6Wq21QC)g z6Gj7W$AZktjER#ZyX2HygG&>)CvT#vP7M^BmB)ogVi=4apA3`%t0ijRfIpcxX0>Y6 z(;(O-iI;WUU4#6?vevFDrQ8D5Z5(7wM)jqZ(FPPhj&?swi}~{(Hl&qA*#4`+qB1^z z31hrVMzxI+l3YL#raaMvq5o0x^_$J|;0j6nmB*T7UYa4pfTxX34Mw23EA4Qf9K}Fi8t%ExywD?k~H&MM((>;fAvLc+=e0Ydfky~vAG)wSnUPCfb0AtAEJe-DzA+$1xXhGpn_1Jy5%!n*xT%cL)> z)@KaoFVZeU%gz7>_d|=nF{&|lH$GE*>{o~g-GLRq%8*?X?D zRf_pk?ONpT4fR!u=~l7EX-6YJmm-DZ!MUd0$n@g1ZhXho^_@7ZXELsaSZb&6hGcHjIK93%X*G~+L*;U7 zVdI`8d(%9NOncJ7@tW5)xXYqZE$_qyR|Jf5!!zVv>P<7PJh0*<&B8dH(SaQJhndWl ziV@WzY}*5O>h~9Y-PmjdZF5z*ST|kN=%UL5MMPfd#^sJFp^GM1gC=BLI!n4WPi@*i za+2t`pLbofFTxt>4$g+tDonub*l+@U48Ep!wwyD1=X=H}dAh%^MBn3a)Kl0Kyj!5YjX7OTxr++=xl;^Ka3 zZ|dJ}7BJPS>ZEV)#6(+E>`F4j3a7m2OA3x16AGHD)}p8IJZdy$AP#Nv=)H5b;f&+` zc*f2Ha1rqN3y@|*8bLVALmRtcZbx7X5%jp=o8$3bsLeew9dz75CJ>uL*&;EBZW~5> zU>L+D2){P4)BtT1YQKQAR!rd5mQlgAJtPK#BgoK$Jcmp=j69?bDS`;~STqeyl>xnf zZ~~3Eh@t8;sRvHWY$0LCAnzqk^d0t?lMpBId`%i=Pn{)Rzo<_OzwfP zvKdI^$3$iN7S`X%7p{*L+>36U^|Wi>jNQk!wWse!=kH7cEs`-Nn(qT&^!OQT+R=t9 zykVS3wEK}gWoFH^@lnb>!+BKg=3K|B4}EH8%O%7IS1TibSnj4<$MWs-JsUWPV#ZkQ znTK~&E$CTS>fkM{@u-C;#GFYTl;a_!rOz@JTL!N}f;z)0EQWbmvZ5i^XON zlIi?H)%&C;gxX!G$I?bQaL9c>#OV5wUlnHo#v91eeptzXzX@|^fIP5+j4V^0zw^P@_)`i(uJD8ofD=3f|7R~@xD!v|9}o)4 zjP1V~X&+P*=5%zz4`;d9WO?VMuOE9Ww}A4rWNy0B(w@QZlqGqT4Gf<&H4_u)_4Hs< zP+fD#awp@*J&SJ3A;{}E3ACRgsbpSgM~DTGpi6E(>1~ z*HE{Is9@R~Cs_^a1g&-GoJlu6MBU#+{&n|yi=c&`*H)`p>bN@g56h+pTKBRiTG8gt zP%oC*O9NcwlRLCg&}oj6iGmx35%Q)e{8$w*ZpEY%-2V36TE2WUwc@Zr4tsPwj;0Z+ zry8>MoT4gaB~u0m@<9HD{KP3NjY9FYdyuIj#+Npw78f#{UsLS17YH9pc{Oi7{l~u; zKV3&GMb6n<#^-82j?_=wn{Y)oY$iS8M=lq`Q`BNz635@!%mtF*N&Bwc_&zb-0FmCX zB}Sxqla~_^fA0C41u@5$2k5q#RH*4ODB7#)*=MZEE$M-cmI&mlP4b3vOoT&xklgAg z#&$Q~hpmE5JzmNkD`Up02%U_5nDVcrk1gV~|M3{bana$HJ*#tcG3fHYqqw;{mmL+l zDLZT1dSqHj-8kG~J>IvNbV!*~1B{Embt4vi+`{o?woc1^!&TTDG}HZ(#|*c@=&XSz zeY|)0$OwX@raj_eSa83URC!HlD7O5~=Ecy^cy|(fIb_`rc>Z&YfW(bg1RpZ<<^PtM zgFHT7ZTty8P%5dK${@@ShxPMxMQ+2@#S!L~gn6|$PDMEwIEogPFRw6wO^ro0vDYQX zCdYQp?2e7E4D|f2*Sj$sMp(I~kOAJ|ANaC(rzzQSJ2_W>W#{6I1dn5$`Y5SH_hXc( z`|tnt-uJ#gpSObp8=-fv_~6$z;3`xD_%8#h_8sZtMWIDHz$d#*mx9&=r2euET|!-oRzlt z|0C-hqw|QmhTTSOY`d}U*tTukwmZFJ+qP{Rjcq%P(Wpt2ukZ7G@2_{QS!>pse`jXR zIcM*)_jRSa>fX|ani43uhPmLx&kZy)a19u1g?yR$iE$SfAj>sjoHYm)$`vXNeJ@2V zwdW0OJxM!@s#^C0z{aHC4}id*p`qT*3ShT_GZ@ARwT51Cmgu+L{B@qDasx9>*Vb?D zxF7XN7;E?I?;)rr7@uiITg~R12n$dMdW(5lWT5$x_-B(mOOA2rC_!$>dK|%q=mm+; zbqLJzc*N03y?)Bz8grPJwa1YP>J@1G^PH%@XkFV8Y&V8#KsT`7huHl7`K=+R;fpDv zn7JwYY#%4)@`luV5Vek$R$kr5{&B-68YcW|w~D9dtN7;|H5ZpAR!ehlnCx&f6C1a# zM!E$+h|_+MdI|mWqPv0G){>zGOQ65?9iT_PHB4lVD$;2YA&_evAeBI zrXgwhdYvZ(kU`a$E))2t4g85ao4P#b_O3GqQtFzwv<&7oQT-GR@kby@$l;#9~)Q^tYQn1lx@~19X%7IrG*+tV&F5 zP08;pd$TDYYf(Y>c^UmC$IK2LO%8Qmo5b*BbkOQ85KNt%G`$jRSEC(m`Xx^{!N3Hj#V^VFF#zt4;)Ak!}oWAMTD!Q(n}{p+(+>!=pIPDw9J zK);7^in~e9C3yOApJyW!-%ik>!J%sq!YlV-*A#20ls?E&i?PSl|2Zf47V>;l@q+2Q zh2#zX+>UlokNEG0G6Zm0B3E7{n(D7!{{ti)ZbV9HC+)0GUwHIRwCGz0k2Uf}<~#1;ozy z!FC-)e*}XToM%u`&lA=ROZ4I%1^aTs8nX!aL;Nc4KO>b*hm}RV^ZI@Ea+Ri^eo<%E z4+3KNZdn#xa4WRvjKc)xjg-Ue(#COwe?;$*@CK4y(0{uih^cpuYJin%>KzHDSH_o%&?y!UTqx2(@1=}n)2@=vrj%vVu8{7gOtor|OFYV8 zvvMg1DCyX=%cTn`E7@Re=8L4I5|816Zcr)S4Z<`1wW4}|=OY0}gDPgpUcJZ!77?Yv!UwGLCC^cH~Fx@`2j9aJ=83tS32|{ zvza&Ko8(oCMtq;I{kckFI+RJ_Wwu}lj{*b?O8RTXdOpvFBqq0ET&wP1)7GN}%J?aT z$I{)7$*}FG$W#sU($ymo*9^hD$y0FbU&?G24TaLyOGk6;>C1F=qGdW4q7lzB#{eq( zQD|f`o7F@9JU!vS+%44fnU_pDU-cp}?JdI9{4G7v>@C@5$-!uw{;1-OH=*Dlwj1z9 zfc(pg2>I+eSjYc!?%XXMyN`0X;Q*0SN0PA=Huk%4Dm2cL zIjKOU?2pii;Wo)sEWVO|-7(Ush&D~Av1bjio@etB#`-NcBx@1O`Yn6L`XH(XJ@5t; zN*kVLGd##fZ^?O*%Q)yjqJYtFDIPGjAm89{_c7!kl>P1O^FHJT!jBByXtY@WNx7hR zgU#SJ!~$2;Vb!o8Iec>9V7PUwn8T@xrY()!4AOLZ}!cg9^vMH&<=kMwA$-!KxY^3Ch4i{+Q53 zvoYH|#^ffIHrRrftCZb_C4+=7F zDxBu%1=pIlZ8ejgAoKEo_9npDryB$BDhfY6DtUaRWg=DQGyuW#Fovj^rMDr-8}<`S z_eTJ|ETO=8hy5!+w6t9lR+H=O`7ZGH_UGlbB9$VZ7_dF`9*jehF+R4Li*$ z&#(iBdd221>nv-@aVg75H!mamHeT*oQ#@sh-w>5%D;Gkt4=}^_>ZZW2Vin06KlzP; z_Z5xxI|%&^kG)RW)~=C}K?;y#IJfXiiq^N3ElQ_rvjw!t%P@+`n$12`)^5XOSYPk( zPCeD#I|H%>N6+VV+*7rq`#K$a^yf$I;d-i)n-d}_0!4B~%CqbcTB~$__($CMu=Fvu zDr@U>wPh^YR7~3Qq5IAs);!$Duxj1bL3ym6}9Cc$IJ?-IOqxh<(Z1;u{((D<6BU%VmLd@q!eN zzs4t6rahKU_}n4oWi1{L^3p|-Mwk03ZZZYAODmQwbA%hK?ipaxA$HXBaRC=JscZvI zm1$jGgp`fX@0sAUIouD*w518wJHebr@OYdn8_%FMI?+U*z8@+7^k;aCzus*`fL>5v zpRId`FS(hf3zR)sjL|DY>zfE^!Re2orgYAP?!WfkcX+JFB`xE!E<6msztV6TM@~vG?6Azss#Os zh&3k)$0JT{F{zVR8%Kj2Lx@QJJd;c2oRdXgLyMNH0r^O4Yc*R!VJU?-h5ZZEcG|tj zNQHof(()nii+ZP`xEvKXJH;PfbpE67@MooyZWN8a3GIMb(8BGtYQ9fqeCf&7qYY-~d`FPZl2kZt5GQ50q-Wx%*h2pg^?%~JVY6h@%1-lt( z2N@ba?O8RTUa~ZdTySC7TM|!r7PYK_S3L!KeM?YMy%^CJlmAq;cQl$7kfi**oyqw{ zmBMt!{JMBA!mlsmkA72<>}7RdqwqH;y#d&#i!cPZgLg<@TMYGQ$W@MbU>d+J)XmuV$0j#KP@+b+oN!4RVhT~|_AVQ>|2hZlLT*bf&swcI?aacT=3=HSZlf)S(seBeeJj*i)Pzn89K!^&^ zpD?PcC2XZQdu@8QU46=eoTx5tbrHpv$U>|x`n7*OyjfRmjib38a5l?ztYu?`L)dM3 z_gYCmaW4G`19hV?{{ijb>g16Z1@xiO(f6Pda4p`3K{pQ1^90CSI$Kf49+htN`R*3d zse2J)&1c)Z^^k9;eCw*N1wR?@`v(DD#1z|8L$7lPZ+hZydaV2Y5nj2J+tov_^tazR zAq6#2d(&?f(65QZpS<_C6uLAmcCo4VvC~i;!@MlRo+$RWq`N-S!G?KZJCj~yL--Wi zTSNb#E#EUoH%@n#w7%XGbh!ZLk_&2audsKYgo()e2H!reafPssh0S<=YJLUosD+8h z`ZQlY487wtzL@nu?=wK*j$D;<8B$Fj486cG=j?tk-p(LR!{5~TeP_dPcI{#7fKc-Q3(Mc*qV`%t?JR?y;$HQ5J*CQ%?{;0dh~F&$V+?63>E&emIn;TrBZai^|ohSZxpQ4D~kgO7Y&UcXyWGVs1&z?87We_b#iJW^4_vkzOd6 zen=122&@++-tcw8j9I4$DbpOb!<@#}i}{Z^w|8&SiGQpsp$if)hZNbXiz!YOSzt|l zSgS0v!u80m`=+~9gP5ZeTk(A&km0ju^{8|HrgJ|rHKS9$HG=J_Nc+)%p%%#Nbf6o$ z6@WZ;!U1c_h5V%++zt(PL6AM-N|DXlBNnP8uaiySqNmyJ{A@-ptNGq{Wz%^FDsA)4 z;)K{peo3J_DEkU%r}DC0s&HcHol%}BRPtOAiucNH%~HX8)fh2|UXjlDj@=V@Z-)-w zeHyqUdEpOI@J`DoI)Rs`%Wn0qcIQd$9qTba!{rUAjOa*th)R~+OBq!H8JeT^OorH8 z1A?A(y-2afxPR9dP@C>%f30g^ySUUbsGXSocza>+ZY4>?w4(`00!ipDwg|> z?Q=a1tdM%ogr6)uuEIp6fjedl{QW$Cu+VSpVcxBI9tjDOETyizJ-jnFeTedr%8)p? z!H+CM-J&eTWD|;~E3RD{f)p1sUvFLc4u!g=ZXG>mBWB#LGpM>7&uqa*Lt@_QP@9um zaYvIYW;Q^Dfsqt2R%kdx@WKvphp+docUa0!sFO_l>1zLT0TZL!r-c4^Mm*$i{AM$wNf#QAnnj(!=dDI#o3|5elD`V=_(fl>qZ*G~}u_!06BE-66ptQ4M8hHIr?PDo2-*VhPCZ){()fdG7Z+O=l5do$D z|D(ZcGKl{Ijavy~p->?I8zKs)fBw%yd8 z3pIj)eK+BJC<@FDi9tB|gbJmI8d^4nRMxxUsvzz&O{;hq44ni;Ts(_K)NB3KIhhO_ zD+`6G^%RHW5!~j>;>Z?PdQ3VH`f6H7O)QGxKw8C_Jff^V>bXGYjA;gRycrV2g0=DcO0+y<8@sTXd7j* zzb#56&Wh>jc|=WnemmH7LCtIRvDx1{D!E3+l%Lt*pQ$WL+dGUKY%!mnOm+Ti452Uu zdh0N9cni^?+6o_LyX9r8LF1ax_I9)GeQ>Mx(AZIvE0a+t(@(?GZaIsErkly6O*k9i zc5s2OP4T7))@&*MRWcDUO*X`GzV1#gtf`bo%Y+eBGsnp*3tdzsYd zRUCZDoCt^GAtDwJT4$uSUkZ(kYvBRq3G<4jyWiv%-uUxom!6`i&GJ}ZG6+!F{n7tj zOFixfFGh?$Arh@kb942m7vzt05CtNd5ntNa!mKBl8VA!BaPqt=5>1!GajF%v%Ecx% zsHWAwQP!g~M`fp#57{b231i>muSPD}9v%rCgcC0&80|>Mqe%G`hg@TTLV-9rcG)ng zuat4N2`I_4&(KFve||_ud{R_~%q~QEP{P1>A6TRz*yU^Kl{4LyH*!yvFrt&-?5D@) zj;4-gqwI;l^EPDJC2!0V$fLtN6ZfhSMWFmsv_s2%QKVTOPgH0WH2ufw%jUXm-gY26 z@#PQ6SDL2We50u8XRFYybv&T#fU75U{i4Jnu;WM0%c|AmsuvPI44lKWjpXLKx6<{i ztt8(%RueRw1fhJZ$85DegnJI?#rg~XI%ED{IOtZP-!`27Zq~h@A6gjM&q{ir*`sAO zn)?axAMR2*?7IS^pQzJrmK@w{flm(|*I$s^J8xmRPT%ZAf&Z|;@N`fjH{Aav(A{DG zPr~P4atHK3e=2q=Nfi_+fQ<$Nfep#!bQ)lJe%TOFFqgO0U`&l09%jY!56moq7k6x zwv_E1K3cH)HV{v+=+@BY4L4e__7EY$&Cgyj6yi^*3E~?hKrw@cxXSOG7B~2rENU@=^0c@fklChLkgKs>4m$m<`aO zGpHBE;wdB8l?%cF6?wVEAs>0WSurJaQu!eExw7yu%7TnA>jG_bO{;ww`8PFmP3wI; zdA-VT`;66^@Jx2+NJ@vLq3jGVVRS~DeRlb-ns83G+Y-v}#o?apw>gyGtHYnzZ^4xB zCE>lc`?~U|8T+$I2=dD8;tLq1C2edJs;V$nGA$(tYA+{1g_ZI!rTRZrB9?kqc|@a{ za|`YFq6p&3eFF8Y%7}Y4=N?*+(y)fQURgw>+AX@eUR8vwnsYfVNOf4h;yIo6N@EzL zzF8RItL~gjd!;!HS@BP^hg`mItj<>up`m`EFfsx)j`ajG_a zP~jzl`Kkug8)AM@yDg&qUK3HOb_=i0R~->p!BrGtSh0_*&Q}?6qITOw`@Jv%IonB~ za}XB0AzR@Yw{r{Dvyl*r=c^mqs^snoRL(c2xIRW$KFM8gs5fee#IJIsPF?VLYnC7@ z6Gf8UBM?{M-xX-o0r9nBP?&UIqu%VKfwM5YFSrkg{OUysCV&`4O_MXi1LHzjbQ+;| zru7*>I78S&t;6yK{upuCXVtF@K3*M=L`k~}U|q`uaYgEf{5a;%jCJdeT|EGCrRoR# zi1p~#NVpbxWQ5Xl?H9k$g!(SKYYRo7cY_Z4N)2jDTQ!`lA$P(AGos!U>N~+|>O0{+ zwe12f9?kGIMk}bR61zkYPiS60`pBRJNNyyt^~P#E`xRmJh8-xJsKB|XZJB)_h$mF9 zxIR%RBFb5rU1f+@*e;1(cL+yxuh70ws2-9VO9+7K1`%TVD*_2qNFNu(0W1az^*1HC zyre!T2z1z__FW#2M0f`y&9+?$4@9sn>?R<-tB!3~7;||cfxRhLooVgrV9bf)mHxx; zFC7>_dq(<9EMtW8nXgNC%@@=bZSUES^lxv)w*$?)CJAcGz9hY20DrXYB6(!OUzu@s zDxo`)I$?ufVfiF>KY^)W-vatJptebGKwv?XXLRrk>>J`|PgESKK6xV)a6gm_EPz1j zL=NVI7)8O6&6|bpTvgO}Vhs0X+>qZz$KD)8`d7I3Irkvo^@SQjvTavbCHKqP(N`%# zViQKx=T&Ird+qca)0}MBlYg&hFWrgA7w_{5EP-raCM>qnxrzriSd(=R?=e%ZExcP| zcLhp-{Kgl;5ak)>vt_yywr^mcE&wG!af1zEi2BS4F%2s>739163e_XlC(E+;`H>gm zL#U4sipA}nz);o*5-bljP40vRZiLN}+%9%(qdul?B z{L2If(8MOnl8V~xT^>dSxI(!i3}TNR&rA@aaYi}MzvYHe-F$IZ9#NbfCht+g9(54} z>*y0mR*&&jMAGmF~=mI*nf0AdCBRcV|As@<^8+8En@w zQ;=bPr1u-CzS>Q3N7@Ybxyk2S-Jj>~%44P-TwunP;cziMaa9dqAx!*DO`o0;BwJ=m zGu4^Id2WK+y*yiY#P$N$qkv8Kf~H@!&)%Um<==60`|mDls`xLmyjmO{>yp7Py`j|4 zl0jI$OLTX}Yku=<2HCv)JL+E&C%mmZFxFqZ4+BAnHN-rAC%e4*#V2G`2*P_-bej_+ zw7e1TXv@-YV+S0UhTZ)-K{Y3r9c^HbHwmBaaL?0yj(1c2|M{XHa@ei+GWwhSii^Xt zZwL^nzxK_E7KiEH{PeqINry&Q%@7elyVb8~Zo>T-YbKQzu>#Lo`lwg>4oW%^*;h`-_7{+&_2z501grG$PNx_IS@&$iN=rFd{ zjENfXTao}Gd)W?=YZ zn2><493l0tX-jJa@F6lM93dj%AWxYUK7jtmPg0tn<60#9vFs8Z%9_xH%J72gX09T>|m?Hf>NLh zqVrQe!K!aVx{I3nyHYFZ5Qi;`2gtYmnD++2?(fG+RXQG%vb*u2tt))<2q3PvU88R9 zvIe5?2_NY$`dPH7ecUnOL?l6Ky!+9N!ZL*4WsjkM&yPlaH$|hGUdv4J4~agJxjSE% z(G8pj8mlBEY|k+|e3K^&2AvrTv7tofqe)7Cv6=Mnd6&<($l}w%d*p;F2t!H&HRu3b zMm95$8E^4cd9P~>@u$D$ku+u6LWCXOpA+Y`SJeid6eN`rSD;ro3zu*tI69w1PoOg( z_WlU9ImkvtBtPVcs@H)~V&+4rUy=_ETyEnXgvsXjPS|R7sak_z4sx>M9OIzyW$6ax zrU9Nt2Y!ovp(7An{)}{E;as zh&h{WZ&wpdjp+-kIB?IughsyqlP1PS_=u;)*}2UeE*mi8EU?CJ!7?h!E+az|G@=no z+cRNA#Y5JdBI-^!oW&_pYhfoMl*hY)Twe!D`-w%>irm5lLsF&N*1g0*qfbDvtWTri zDmGS$Yr{tRCFPj>Xhr3=&T|e1sQMl!Aw*?eQ#b<&zr@5U@AV`|o1{r;cP*Jg}`La#gZN$RKkl3?*8F<%eU>rnfJF3CH z{L*$aQ^G&j5t zGXY6TJxQF=B3-hRrdq>tPt{??r#RWtyJFvG93w0 z)0_boeWO?I<|bZJ(bK1Brm$mJ&L7bWM{n~<`jz7FiJYS8{ooJ%cm_9{gZv-o zUv#nlG(2ec^A3~9zDIatuDxerY##oCqu*z!18J2Bi_;-K4$^^d;F1w5lwU1B_3dKP zI_LOB_`2-$%ZpvUnB@tL#bVBAkTWIM3yH?It0QN$)C-C{B>1<+;;Q;&T{tNO4H3ix zZCUkgquakDHSUfWe8k?RUFncDJJb;&$oVOEdiws+JpQ69eAnq<lY2}V+h#dC zGWf&A35m}tgKNclCerW36tc-;k%lPX%BEsOR@zlzb6XMNZL6t4Us8;Pcj+85PL0{b%wJnw4yx&oB~I4V%yDdmH&`Ft(b#ppU${ z)#whh{eC!fl=+5=$M*y%fd`@rTEUlm8^c=bCpK=v58;40#h($zaJGGB&O`R)C3eYg;6GjR_yPIh4(=-AWISW}7IZ>ej35!;Tq)u6=S6tF&3l4O*9y)I z*XH*MnP@upGF8x+F1EED*PGi@mKUX$S`mcI`V zDJ}(CwUJEq2WEC=Vz*@}&+J$?mZQ zx{sl=`IgU_R?y_OCQFjKQ(b8QKasL)nj6-K?QLklLlF7-SWB6W$toO4Bp8d5hfxVz z_o*87FbUbu?ejJnv1=+z+s z8A}?W8%QbRKQCYS?2iA2@v~->-IO&z1?rnJ4AO6I_tL`0pO)jWCXOXT<75Mz!9#*bB*#TJW8&mG>q1Vx;0~IKRd04aEIeA>)(; z{PS9HHH((<9SB}B85Y|2e^U;iY}xy=wjX#be-Qy|8G){3yybc_c<>Q+#cF&q31?-= zf5b#BCs=QNQrT!sc=~J*=(Rfbl-#zEVBzxvA!jp|+q=G*V{mZKd`;>#>1?HN9&`7Q zS^~P`>Jwp=1azJH>Edk^q$S&{s9Opcm{JjJJal!2+VV7VMR!o_W1?TQ?n(@gM+iuD zw1)espe;9ez+QqI6suHmgzETP%*=cG0lZ3}y2EBSk&C!zQ@(a&73wx94L_pF#H*sG z5Cay|Hc%d7x(a2{3xA<6W zI^hkuKnh`FG5m0f%wk|ZO`2JXwX~%s8h*eg>ZMp~^|TQAA8J#td2R9Nsa6iE#PyJq z2t3{7S?@31iJpNnR%gN~%(Kxn^+XZ9O7XKQAGMN{>8VfiMZs^%3q?^$HD3*cduXf?c%fXJzu1H&t=>e=;S(4h=LaKM4?$2(aXnyp! zrGkP9B1?vyLc{QIqB3AQxPBc39$&vJ0n7-r91si{xh_>!{PLR(!#e~ULgXfexxblj zb@PYA4M8MN4ubu{>KAuFprozcSuGLMcUcoU(fmFipp37fCta^L`t+xRm|Mwbd~t;p zISy!ZDC@G$U#@gfuW0G&&4OaQxMh-<^|vdR<*CDE8;Z=#{<0Qk^C9;1Ax~LV+5_WV zReWf-j{Se}vcxEWxp1C^zoD6@t{3SB&%cHKiU4kqIb1sCHb`jzOJe-zg2z$}F(&zf zr>O=9V{!>)R#q8jtCnK(AUVVGkSY{KcnZA|3v=?-9V6|%PmRe+uLvFG#qXS2F*Zt& zC7N4Zr$XWK-MlVxS(h+QGD)Z{7Goas;g(t*0^?Q~8x4^rz8?8O#gc9zi2@#41<}qX zlTnjV7o@BNt7_o@DQSKGePeet4X?+RvpaY_yv<3ZbydWdg5H1phlF07elpZ?h1cd5 zU>I-`S=^s!qY zS=&mYtk7((?~k96P&f4^EGT2dbQ1bO8wPIua2JC?e|-a{GW(Ohf4{nVu%q$S+bqSS zl0foj_|BZR*^Atdpm4e+L?_4Pr%l@1YUqm_b-AS}(dGB&F*sb7vlfi5w?qD2%KV`FnH?t>-$dvh)O}}+dsIis;SdXRq)F-L?nP@2Q<`pJg6=#ub-d{qN zJic)Lp{AxrhZ@KW{cQCISr#W}mDgrmbHj9XBxzAeHpQ2~MU)vPeui1qhDsrW$=)P8 zxNlfOx|+H*;p^a({LtUXBYFCCvr2Q$iS-ux$io+)(Axr&nh|wg-aZ1#hsZ=-WgcmY zNrecZ(XJi2X8bV(0;##T1cTnmuPTc+n%nD+G-^i!TJ*vF{j0i}-eaC(B)M#2143Hx ztiib`#Dw1$aps^$eCF3Q=69wLZu3(k1Uvn2c8vB`F=MfX@ZX{{&9=d=3s5yy1uOxb zgqJ#iX?RI(Jx)-IktY)=9%pcQh2U|Uv?}G|5@F~S9r_$Rv%3B_cbP%X7e_v(iYOJ# zLRpd1FubKGuyW;TT$dw^_yX3qf^ZQrwqg|Fx6Wi72{GiUw$V6 z2pYJQnPuNj#1R=v85PDlsyUHe^^yn>`QlnXBaTXhwqV?>q#mm4b=D8 z`=DX+4C>p;9d}eOT`uGc?;Y$L|I6etP&-k|G|Kyef5$yC)H^IVY6)Wcv$F8jaG=!} z92haf03B>^X8~@&M#2c|uFLJ)4k+lZt0feO9wf$Qu2mthT;RnaDqGoec4J!1v}cOZ z5r+A)OO(r57?>Tq+{A@6V-J~1HCTBO5T(D(UlrEI_ z?R4lArk`!U#>NG_5fR+LwC(Eu3_&(#@`EFKhO`TrG0J;RGPH}}#~VI=o}{MUgwK{H z0u8Vr5=@5w297#A=VJRXJ&IT21>k=f?%N{zc+gpqBjT&qeL=Ddya?dpJE#d8Wb^= z*A0UQk39yoTTm4}iAmRYA-NAZxx^f-srZm|kM3E+(l&f(5r(H+MM3ybWNSu0Ek*N! z5Cd6`!HH{5VfD&RSiBG*Bgp}CdSs%36Cw`;w+7f7E_`^!I_DcyeB|5)i03HG0b*@b zzp)5y5S=lZ77fwT${PW2PzLjc&0-8CwCqoP_?|VH?F7TIe)WaFKrx3~k(t@YK@!DdCg1r)5KOV~DCedqtRyS3v4Q`Y~ZtFO?DJy-L= z>S}>^Rgvo=c+_*SkjNdGZ@XXNWHdxlHGHa9~<|lgWo=|J+gM2eUn!ky3KLC zlY1@}az#IO>4EdhU~O9Au}rLKhQB>9TijKk1)lFpy(NZ5BQ)(|+})eamCC}aNiS8e zvy&8w(LT_WkCs*e?lG9F%Z|i~IIUwTIC5v@43sKsp{Ok@(OPe)5MpZxTG{TOKH7EH z`6ni&%oe7bu-e&59w(g&oHF3Wu&yj zpIzWk>0n~aiBY_)Ll(v6D$#abJ1<*4sxf|Gh7p-utsI9SDg)7n6#}>qqqy;%IJh?bJElk95`}_yLH@` zaMO*d6@S`6>Kz?i+N5%$;O$p9cB=GR&)Be4A+%P; z4coerA}|1?XNiuc!v-TSZ{j&S(H$P+1Dfh@x3d)5hEK+B&06Y3u7%WqE#B1=Lh{x( zNd&&tv)>3YgSmbNO?OZGH}Cd!>uuXN(RCDmSii@hqvlZg>>NdiU_-eO`k$2nF8bA$^q%cO$S z7Mq}c{M!Us1DGRS@xjb}Z!uWR9cR*`AcCr#BQFg6x4^RDeZlQxHB*sTOc)^i*H&j{HLljhoo)Qgw9!4= z)HoRMaPwlE?wnm)15JnUc9UnnL)0rOBr?w;ih=fXl$#5Mz% zG7eSOclZiKZ(Xe>3T2s*T(O}ZM7VZ2d*xhz!VHuaw+qz6&kZ0T#BJ?>M}U1xd3E{u z`tyzy{bHa>ZeRBD8I#7Lt*sNEX#32-R8G}gK9RJvf5Nd=a`-1BqG9feH;Tk=l0q)$ zJxZA!c8+<8o1yPsxWDo}e)qA%DarfDLs za;D)KPdY@tc{jSH5Cz>bC^g~1Q6dIhUL*w&L{F!>(ak~7mYmta0N;R#{s&caxS6Bu@S zE=7?Blx9$H+jd#zVDX07tFQbHvh<+)QlYSkdnfLy)7SIrieZL%1uC5>9F6(A~Sa2go6V# zO#qAUzK}u4JGE6~!;ZG~tsVUjlIt~#=kB()sy!1ctt(~j$}Krx8)6pG=w~mdUVhix zI3*-9K+Gn#Fna0FzudvsQSB{S#gXq$H;zHFW&_$XEiUOU6l^@>1i9P-p^Z$Qc4Ia} zd@iaQS_0+*;kXBWL5a*$IdGzDY&a8LOknv?7Y;}TGgju(Ea-C-Vv|u(c)Cz|JEW|M zH%z5#&e&P@ zN#l=uotEhr65Z>~Q4X#h6X6p}yTbdK3%3(Z`onP}#2P!SX}&%@)?8!xbjC(jmK}@u z$yzNr;(ndqEG>X^?kI<4w7G6WNgZpWMSWv~Mtx)Bz(qlo#N$B~rH8_@g@?m(?AnVq z@9J`u{)Ncy?_N6A2`q{{aE$NsKRv2l#Te%wtbE{}R{V6o>`iHZO6+K;Q|i7=N}Ux} zJE~iJk@{>sk`TSjG(fX(@{#0UwKC4UEap5P?VOdAstmwZEVcl9Qeb&%WE@d;c+kZ0 zyedhF9=cPYVRI+^eMngjk6C4aEK>P!3L)ZvzO@n|jf@m8JEinSx(!af_2!da@K;BP zdMe1o8Z+)vC>C>Jxg(@mZ_j+Er$l#s({2T5kl3(u2+n;SH z%y#eTU9W)~RO31+ISrTOeSgm%tz4Qk_dN83#5QNohG_K-s3Q6o_goR{yED!st+Z(j zg7!c8YWh%k@UJZ!!`pY@@Jzc9^S_eS$FcB8e5VH;a^dt*4KfKp$Ly*a(>{q%^e`T? z8dKy#e;2?PdGq2*oarCdo7tEe4gw#ze6pB>QWFMfhcM-63Dl3(tDq4Ip{WTk5M(40 zQG3-O#jz3$PE|4wlC9dPgs=orIa}Z;@5Gs)cP#HHxSkOG61kuBvUGRvr3#Uf4=M$| zYKuF2Vfw#&AYfUIIdjmDZ6d<47((M2B8s*lWBZ`7bcK7SAy|InH919Pm7M2+k;WpM zVEPJdjVn*jbT(9oYq(RJGF<6?($=5Sm4|F6_8bnLQR$F{Qzdx&w%|)Q92qr@u~S3t z*SW852E8KpB6t>AR3^fKvztNOD$a4qQSfDe=JXLFg;yJ5ma%cF`u$)2$;+7NLfPNqi{B zhInWi>@W*c%WYzoHoVx!h!L6u{4Ugx0~q3gslOCa`!hl`02b!1x7tr^C0xTG_qM6% zH*>Y0n!8lbT@aF6Lx!Q*Z-`9;ocYNga5;O9{f%$>O(RdcnC}99lY&1SUqTU{CG7y5 zk2LRRs;Zz5BxDCNV2H1ciIxP@>_}n4EvG=mbdFUeF^edxEK3n@btbyn~aCs0LV}oUw2C)q`o2yv$e7L6re2@yrH$=M~!7k zeT*0nGB>+W!+wLDpA!6PS~c}BK?fvGDgOETGNQ3xI3VGfA*>lMq6vP_*UFfUvyVFv zn08~9i73>9ZTBMj080n-WAx|imDQAK4dAzkBmB4*{9w|Q*Pd{GJkyx z|JU~!G5~La^QM3h$~YwTtP>UR5k{6SN)YUO-SU`|8_){(NByy=zO&`IG6N_GdX-9( zJ%xvSSB3?u=no*br}{v3?@{&nhgu8cB^#4cG;EL-WIu?s((nX%;kFVFJ2?a2?39_; z4s9A%J)SfBFIUMq|C|~f>PE1M@HItoJkM9k=qkaxvPALhqg9ZFl*MpmpM|-rN%Ezf zg(cdoYx{$>K?l_ilJQ*+U4VuDw!>o5@C9>J2BE5vnsT;zP*DNj+Vh54o^QnD)wjmV zggprgIp41M z&db=r^AcyO#bXs5M3Rvox&)A{LtI_Qcm0ebqDmNj?FC;!n8Rg6iNLO-3Co+VoN#m6 z2uo0^dE_sB?Riyf?Ee4G`$bry`kr(ljbN~jD1U}a?VH1}Z_o(PK;(}K4I}g>% zCD~qWY>D+dkZ6%WH1)T_>OL8}k0z2b zB4UOFo|+)Dbw_|3qzJjK1zuo-6oO^KUFVdI!~}`Q9B!4;-BDA z2UTXjX4^H}Y6~9zg1ow)mApKiKHYfRILzJ-xwsflEVB=-2$1Zid1VDCZI4;5ZX z8xI=10@%Hqz_>6q&|$l=4ptQlA43MT)TU8h-^%l$R~f1rm8Lay*Y3 zSnUi$upEr9!dguo^KXwRuu;j#i!X^W72sV}QL}5!q5aL!JP*1QNJpu_Fo{ZaX{z_D zRYjU)c5kD+Wfwnl;%6|%^X|8KVIS6thS$7T9j9kyY6XI>rR=;;KWwdQ<)mH>lQ^Qe zO{$uI6wt7Zu35$4mLHcTa9iLUmqm=}fJ*r8_Mghy@o;-=rJc@kb!3i+43B z`%uik9nzUA70Cs_nZs47lnjiGvwfvo*wZtI(JP+}!cWq6s%619&=}vdH3z9Cc~4fW z)iICzX#gekM)ygE+z8d&v^<0g+44MBe;w8!5o=nwK1onjlZ zG->Ew{?XC4%d&=MG!*u*X-e;GlNO$*WEgsN%MUtl*J`RD6+Rr@SO>O4ZrQiCzta{O z?=Mi3mH*|QCaeomQXE*?V7icC2eHZ6YmV-nYXF5Oe-?Aj4{oa0T&tDM>wIdW2RX;= z@!%CUOGw(1qM4+^`>Bg@Ba*Sw3hV-)$1Suf?7$02jdQY#H)%A-5>Z#oJ)zupBWwzF zJgA2jV+ZUedTz*)ZX#^T@miq{H9y~C;J!Rz3ns*wem#;N=Z-7Sf|sN@8Dt)ZcIt8= zMF23;{bISV$z2U}EZ_2?N)Cd_B!~AjD-+=Hgt#MIXe2uoW;zSSdJ)t0kF@boc=5$L zj||*+^RSm(h=qx1?lHm0csS(prm(P5&#IAFzcVJ+%KA=W8nftGpoc-RD$!PjDhHg) z#Go%_cBfeA^s2`7QpVl$bt0--lCyl>(KtJ}Nx6lDl7HX*k+FQ*za`OYtvWWE2c>>z zgHx^tpO}Pil;T3BM#q`2zxC-n>2kJc6Jctc!6~-^l4xYgTd$okN8?IJ(~WA{_lsZN zCStA;$$PR3a!0YRc8Uuze#z!c%m8r0bwsLB#5YMv6Pp)lV;-5jbwO^9T&e!pfOAZ3 zY4XP_eSW$|gJ(5+-1M}DXKi8p;DY-I_OYWO*1!D163|4db4;RYSNcAEpIrLPc7*Mh z;}W+*HLWDUI(ZeCv%Z>PN)z2O1~83rYtM;OMuG zZ!-Nc#(lhWS&&Jd=zx~(kU-xd4KpjBz~Gob@In^=X1~0dE4;CCi9w*5nRtF=rVx0G z{rHc{`dX+8`jO_-CvoflQ)&9Iclf`RrvJg`Hf#Prvd%HOlIYR$ouuP*bYt7LZQHuB zZQXQi+qP}p*tXF@$4+;;d-8wt)~tCmRiEoa)v8nHoZ5T;_D?T>Ch9*%#pdnnVuxyx zXcN%|$S};6Y8!~(aHx#XIasm~KRsIxAX`anzSSV;J!Zcm`o{>hxRBn5=FwF=7=2alW8k4a^X5N>Pv+( z*$H3AMz=Upnk9>vGH$K`WK8kIRc&SU(UIchn&`xo#dX7Bh1Jirw^*8_7%h#F?NGRs zH78nQm1WtSDGiw{QD~ox1op(fTUQSE{G~B!!s<-2ds3Kn5w{+)m?qz6%!WI(qCiTO!Sq*ks-SV$1 za{`pMc|r#5>XEtFB~>a|?YqnK6fNO2grO%m1rn5wn**wja!0fcY4ckipe)xkTecDp z(YICj$1J~gTG@&yxfy^}rNbH8_%leE9VS6O=2%WGf4fz-BWl&S?Vksd!<)M;w!aco zBSHeVY}sSH@ojVEI?PV&*C{ytR+YeQkH~~TgCUq@wt5jGC-;raKmR&d*WDTP zRgJk1ryMV~9nOu8(>gjS_xEpq-?QBxeH)1OHdVX7Fb?8PIhQp82Cf&H-7fx`z?=qA zog!4yMO%GSzP_=46U=#$8-o5@#KcNVizo!xq}IQbsL1HzX_y^IF8 z@N08jr?}JTV}rufr!XuCW^@aDamDVYUv2^ZfBiRlDpm>8lSwI92(?z#PyFWe+S; z!&@>!WuauUXJOsBI%l1nGN}mZjdwUgrm}o|3a7of9I+e(C!pVP_EopMzDd_WD0p`) zG-P#y6H8@|UitAs;Ebii)<>vFp{VMcm>5jFFp@sH8Fl_d-;+Til>8sXRo4e!29x}M z-wn}A*T0^UGO0f)Em;(?82dFuvR;>v<}%3l-0A6&A_WJQj@ zox}vCwZeohS$afaUJpx>s4zaVeo!Vo(nCbQ+GRuXYV;=3CvDs^3*`3)E-L()_J(h11# zm{j25ZHDU>-O3?5hn@t68m0b*1@0G{i3t!O{)xH-WD+9&`F;t4yorv`UTGj0&3*D= z=m{!=-q9VL(mBUgNnP}NAT z-Y$9aK+9`r4;r;NbCKpF0uCmRnf8^&a&4CkjGR!l-b> zMHBQPuQfGr_`_YdRUi#nvP^@I08{Nk#$Ah!QR-AfFIuZ~DJ{u5 zv=T=%Qf`s0_;gX_lE4vv)Zk%c_pGt-Y0lk@&^Pk26{JY|`scIv7#7Yfs$bd@=F68S zl>cwOx*LG{Z=%I#Jc2kQYWFJ!sO+;S{WHiUsBXmXfJ>_)q;aA?!Ujif935O4X68D&m%G@BDB$Nu69mY#H;{wzxNn9YAX`+DyMItVvOJd@M z@uESC8;GLtg=)tk=L8MRKjMSkPf81$nk*E(Y=*SM$<53;>fosS%ssdmMB zeThjL(f5T=AnpxCpYk@yhKZERPUj=NBl%>`P|!xL^gdBFXX;+9$Cb>e$vA-0aFM_L ztn|%8oWmB>Xu8+BoIh#hEp&>*yY6{&y2RIn9H|G7KB=8dIpwqaR7sQi#D;q#lc?T) z`oNiiYLmYxxXc(&PnyZv_ksRAY7m>;RW*RD$ZY2QP$(uB(&yUSbU>zICZCSe6&)|K|-f)@pT!wE_yb<#N-^PDoFT>y8DGhW<3~Lpbqk= z6k9OwPjSVz%9A_jWODlH>43tpa=CR%%_Hceu{+LJla7QzL8Fe2q*zGY1c=WGJ<*zq zwW2t&t%6Xg^&9RjbBcmu!jlxK@zvxClkAGqb_0K8Q@^#z>)eW%WkjeL>um9Yn~;<_ z$%$m_9MQHbYNA`msmQZ$Mo;_|{|aFKf{XB53`u+8kUbYvTm&sHVWF%d4R!mCJP_5y z7|n9O@`O!Pp3)ElTTa{-(Plg0Up&cg(Kk2i$NRd!iK2M5z9mQ~Q6#@*7gSrdgTi0M zK}AX*p8nPaC~@#>j09+Y-C-MJ{VstimxR$z*up!5#Qzx=7xswe2cOuj|Jf@4zY~6m z2VX5f^3E8V7=nM;WM7f;zEXX~fq*9+Mukj4IRrrBi6x2*Gn10O4b9EuWQ8leF6;t^ zsU!U{F;zrk3lVD=(y=s?6EqIv!_a1Nh0`@QR zU%|%vf?rjlwh=t(H%dr~gXkmaBIy?}pJ;hNMF*w9%LsFHZz5xqxEU6`^v8m@+UC8? z$JMcRZ1vM`GGpUxLP;**n0If74Gpnr zL39q;IO9|0n*sx@N#z>>A$lqLuQa&{__^zG_sG`e{>g2?YbM>>eIDU%1 zO0%$X^ry0X>HGcnY!7QXKJ1roY+}iC7@h%}C$-^CscAc`x)Ebh*s(zAYd#!hthoYr zo{Crtp<~2Lc{tZiH^$VYmTr4LQJ|iV>f8`f$!avkxBfVOpuTcvdH@O9qJbkQBQB1g zgWXi+Y}}Q?a2KxaJYF96mlYovWE&ECGV^1e?D%TQGhcrX#6N#?mY>%8Hi|@@rPWw2 zl#zo*gzNVz8BiPC@7z_yDAbVA@BWPbaOt7$)sbk$9`IU?$GP0PgT9B&m5gRooEFN+ zUOk@N{^K%T*HxfC;+V2qJaiGXU`OGNR|l7&vQeDl@6}w9vwf}CA)3C1TNk)>U*4(} z+%PM*1?)G(9%Xtz%d;DiRqr@|yj~s#-tbtYrKO=qk6~%1rCIze;k2u%1}AZpbvN_% zRZbezV&;rvx0xnYP@+H8uE;Ms!b7KY->A?5&q=W^W-d?L>tfRB%{@SzhNvHCM61c+ zIYXcSDQ(Mpdno2Dfn0%gaY<|SKbcE%waw~0H_d9j0fu$=5vy)0~ z0u9{s7;`%GrMo70G9J`0Up@)cRa|S5Ga|QCyTLBzezxGfi4~LaqinCtPz2% zp`?d!vs~!PI*_L|H)bwAqXE4-LZK=?a{&zKjdw)6lmwIvs1jI`5HUrO7w*r0Fd7oV zlLDogdO7q0Z!bj|2nYYVCPdybVxgE>3?3J72Dc%BX>O~h148`|;ZBgvG_A6aCD4e2#AkW?VZ3AYG#*bI)>T?%@|^28J+2XfL}GXtluFpA z^gNdRWa*(cH;baOf;sVchIClQ$4i1@`u-)@)IhZ&7FJtr2sZKGmoURXAGpu0L%>JN zozP%zJ!$GMH`u1wYD2C_Ho5c`C8l(!e6+;m{^5FsVhK=bP=yYWC@0kHAD^#)QFuSa zYMw(=00i^I((d?vF`d6yympTK;g%87WM+#S!186hdig{_i~eQGuQa z+M6~z=6^cDa|d>wSmaj(M9q(HNrr>US7d>}j63|ESm6!KkI8zuKu<-zsd{-Jo`k9lT{Th0vv{!f8 zvt(f))^7{u_Y-_?S8KEzs*U=5H3jd>obmW`2yZ&mJ{+YpI} z;Zj7#q&6}dy-^+M5Zx((mO*#KCTUO~Nsr-DMpmyjf-JdH6v+?5bonfi(;5+s?$k#6 z>|hF%6ex^@#k@}>Bhr%SAq%XGB&Iz=lN6|q9LKotB@3*LOi({yAjpNZb2dTv>Y{?s zTFeeD#uGZE0#kw$YrzsPDii_<6ZTnXuVB%bD#{qb2V=#?lk+`8ls$wKX^qVqxG+UB zLo!?#HzWmf#HvrB2ls5@te>ZT?XkHTmBn{EQoUwLB z53M3sjw`@`abrzbl0}(9G?Cg_NCN5aTwys_YL;E1p|l>F(|BTsIAEr6?M?5XU~|7y z^B&mGxCUih^z!5nxx!en=!}~PhJ1n2Ta1OmVE`_i1hs)WCm@8z-gTUbeF++oE7&u^ zCQKn}Fo-y{o`b#us}K+ZVM78im`Fd^bp}mHL*!tdk??SGZ-TznF2dvvnXranB5|?v zL_pXO4KE^5v1AUJK!u=&JmMM*9*DtUvwkWFlt}d+A~dw*cMIDBLYcPQEXQx z&LV)*w?ryBoSAWM&ZaLYHm0OPl~HoN#(;{45i)-RfaXrkSYCGJ;hyP75mmZL1(gzb z9K_LtLA)sO`JUt(hWBtZ<@!Cpg;Hr9&}Unllz;0j{tnGYNW3s2w^I_u{`m~l@)#); zy*oSb3RWZl%HFsf>^fi*MkwN&MDyCbL%+>mw7d&K-V%)e>?;^xGGu`|>!5N>Fo6N6tU zq?BGB698)iem@Mc_Ely?nBk?jrzzT?-gYZ%N%eRJD57ZLSw3)!5ZbFsxxdouUUWods@HA9z5D%&IH zjSX(1yy1CSAarGW{K!PpN$7#aTgzK0+CMFl5a6W4>0S~c)_r3F@H;R-+%V#MM`cFB zn>Hn0x)TE{_{hJw@qD}D#T%LysM5^s)@Hr~F359=#BqLdQXlb*Od%o-zQ;y$>56MvlkMLgTv{=+h)OQA&~ zHN`mVN$2;0*%=)m-xDVTWV{K|RFevMJZ{9y!SAcPs{WiklmgL-PD0F54pZ)#1Ym%~ zX(>0-T$xEm?`|zaWS9tYQ`@;@V*^PP2JSqkNyT#we2Kz!ipTw_yP!W-TL`{ z68$W4f5`3UFNX)v_3eVoSicAL)CtlO04f(bx_EBOl~{ehmT z@dN&AaFW3-=!XJ&uDUV_Cs`bbGW8FJr+X^ond!%YrUh*X|II`W*c05^d+Uox0-4?yC|N% z>gIA&O^4CsrLr=Qm<+Wd?NdZ=yDdE}KXH%z)xe{CkXvds1OG+ZN=n}GK}ms$@BXWM zW>jQi_Gnvwy4HjA!_fPafKO~I_l=CaIO#mrPXpMo!|C8aVM5{GxT}GGrh8}pIwma+ z^u{pXkmZj(=du;(Kyw{R9G{m0l+0uvE=NzNtLd~fH8eG}S5_7wYT7(b{g3?D+J@Ry zN;24OK%q&}1BU<|ZC5Ee+2)iVH^|CnCENsaIXo_8Njf320O}u#PWfyeODALHM#+-^ z>R06Q5tQLvkj%1R@80?lYTo!2D0Y+O z3?Ic1hUOkczDOA$)O9x&TBe(}1s65LO15QqvTG`XVe)F3wJk=PT$c=njq~`auBh8U z)5>%6Taj6IXAMrTAaTV!k2y-gbwL4FGE$9=-edabR8%nk&=}V;3o@SOTi*|oFxc;r zAm--LD`^XIuA~JlS1rqyqS@m!RB1j~@mM%+Up?7Z+CA&g`EOi!Q=Bdb4}NYJRIuV# zXHh`Lt8$gv5!3RLO~Cydy+XZf%VB%FB}hNhU}(>N3XT`$C6HgJ|{ubGUUPA5k2 zBg3tW$^e6{eICqHVwgam6Q+%%%$hSh=)R)fMQEmI3SS4%#G{eYPhz*M(6=NjT)r-zc@zJ28Mqx=Vkzv5!L--m;dTM_Q%q~f~o@M>x`=rhpl z=OZ3A#u={I5?}{8ggO9twu@S03%W#Pc=FBmt~x@*na9PQwBk`q^D4-B&n703N}s=&Qx@$5T?B7V^k( zS-D(!j?Pn*lXXdFMWy)6G|ALnt4Gc)W9d$}uO11XCU#KBQk)lEXGlrVelyVJ7^5~Z z%c96rW^Pa3`@=~Dvayas+dPV7``vHjPMCZ~p;}efFdik|O)egvpZ}RiQczGT&UrO` zr=yf1r}=OD0P#b`M{k3&%d7&S-NJ!}3$ON&*clAR+7u zG{x97CRO>DsOwa*7_SFK__1HACFyX|Qd>itk)xkY&0pTAK%H|9RKbyem_mM3!84-i z!>wNUI<7(Badn%8!5RX;nSfzmqHo7uE0WQ<`m;=itPMsV*GA}&x2}jp8~P;Ke^=Z= zOF%=fL|5H*Cr6y0Lde*)CW%z8u2)rRqI*SNAgRp)ZR#pKtd}vhAcL~^=23hIi=8K) zf2%bu_)O6@+w-9mw5m~G8u9A{7g_8pScp$DnkB0!5LaKT64Xr<){9TO7LLA*%ZV|4 z&`g{4;x|)4#yk&zgL$Zo#IrKRH=7r8k=GApZ#aT3I-oj!e|V3}l}ErI$|>|<(aKLn zznQi3Sra)^%w{vwV-M5gBtV4ETD%F>d7+A+omln>)Bhs`s>h+Bh#=F_EW(2-sh?o$ zfnSZZJTQHhbo^*!OSJZ2zeG`@piP)=QJbZ$5o*$re#CcsrG&2sqUdinkngnfDNM&6 zslC=g9XdDSH*9ME_P^%DJdQ-WK+85`)jn&ego4n2*X&%}te{m^+<#$Q7Y2 z9jy%;f<|u%8kT??!V=j)5(mQli-P;omffgc^H_j^bFK8TWsL$qYK@|I&F?*8UC^=> zFl^b{5rt6_AZLzuEl`5_@rpIZvwyVDxM1ysT{ZUc>4zHvshGC=cP0)HU@}-b z*F3&l;=h0s_zqkz>q~F?kvp?1>Pwn3#*{qEJ)`blkrT+V1@01G_VH%}>~vx6RFeVClGj9`v&-U;xVT^ra8Eb4hanqNDR=(1+0Z z!Z8^{c)$O3eF!zUA(=}s5wbvp&mUp)jvoi90POsRe*)eIBKSkIs|2&PihiS8_|h<> zd&h1az77=Zg#da3=E6Em=zG6u5Aiz#dJ(+_=MN#gQHa7`f!O|Eejk!gpbLWwGO-kN z;gNCR3gbOx?fd@R8P^9mGfo~XjbB9wFCBo`l7DykX1|X(EX!Np9e7UOxK7=yz;u`q zUpbdwK6bN;i(r2Y)$tN4A@T}GmqJAyRhx#|sA>cvr4dQ4j;P?OYWN_f`FxpkbT0QS z2``<2@JiPxK0C)c3d}rJm|g5)RSe?yLov9Txd{isZ9tIc>dx86-~&qWce+*63j;wp z@z>JG4<|1~EGPSpQ(h6AylkV1KkB2L4#{8GVir^|-t?LFPO3pq4Fa&QyvJidJyAT7WvvMV>Z%ZdLN0 z$=Zc)@EpomS*`l5Lw&_&UBW?&HA@frW3vRvoWweu{sUk2TMyD9UGn!yNMLQAV4CXkZ$ z@!IvBbU?NX{mI)iZV!YQtl>@fF%NH(Ap5+&C0m?;7 zE|<7=+|xG7fL(^rvK2+4kX^>!DmB(9P{<*}XrV%1!ls^V?2mF!tA2sqf#)DeK>4ct z@-Eh{@~Fr_`4%*loVUEGv_c6?o#V4Z3C{EJQ@lVS~$7FYo~9z1?_8eiweg z(y=e3vRqsiT)dGbM=2M7lv+olb|`lT&nwL;hQN>Q7uLm3zMS%Q$`v*&E8XKQSzvY< za(wL0O0j#{D(%tiJ6YzsB`cdzbOOXtI9Y#AM60?8jktl4bcD^SOQP}DRxs+Q?#ZnN zswG&kYOk2uH!!{3!gztAhwSD$?5-4KS6DwmjV!Oi?d!dsz4F4F&FcrZqelM*W}wb~ zwg7InZnuGdRP#3T&4jzzw^Obig{N*jJuT;g>vzA=w8vF$QnhJm@~(x zhh$P4ha8{yu^R719iLh)P!2>xKfvnHfTATI&~i=W$4$4ZR+xrp+u2>J9SD_LBpXLO zg_JsWgW2tsi?(0b=34tgTI`7r?YFUc#$Ql!?O+8)g&^dw*_&pBV5hIo3bzAEnOcoQ z(q0I?tN(}^nD2w4G+(eAxBfz`Z3u%$Uj~lQf1?Q~P7an$k<@BRndYg*O{Xq58Sw(N zX||iBx?ty3h`{V=$OdH}7*JENI2y1(5yE(&2S&T<(l4xjr?P5KIZfWhz2hZiHbHY~ zQKxtfhj_Cnjai60Oswk&>@sv-X{JUcS2xjk`%i-%#{#IPc92)Ez{y8Su4sD?+2>d9 zd1vG}6w;4g4naHyfWR=8o0(q$IOF23ZooZvO^Sc4;i=vv3;lHoqFzhylbAaes3{Ng*a%raq@U_A1DoU3Tik+ z)HM%#wE_XiyG3~^{$q9EAm( zEB{RR!DCmG+=y+u+J~*trc&xOu#B(W^5J%tc1%jaTwijY`%e_#1SH12g020(P#uM^ z;1dVi`s1XV^4AU%5958HeZ50_gAhW<_%upGdV#=wov@xqMsKNG!bgE@mv*eu+Y(Y` zQ{SY!pOP4UT>({JyJ}1_Np0hd?1dINnU=4kOWHq2BjqD5S_6-$yq(32>Pf%N&WsZ| zwRvn%x@uNxzh%)) zi6^WKE&yvkx7EwDBL?amk zzgnd?Dk1b5YigR8#Q3oyEA$FLV93Sjy(%ZXx|ZH!>k#IoN-!nE8bz)Cz5#`;vUsjU zs+!AHainngD`}NuN?MxVT$hcP(DFG`?s&WN7iKVp&dpkBa%(Qf^ybJYG_TE%kfi#8 zO9k$aYss}zFUb)85?IH`F8Od(&>H2hP++?T#E}s~(F@T;oa%P60N%*osq4n6g;@iQ zpEJ&C9-lmQo!JD&A9}OH)I7{D6mluCClCQiWpn2efwsm|jJ{!#yj)UMy7tD`@&;uY z&*YvuBMncKL{(?{Bj1$jsFhGlR@QzQO)?i#Lw|$@I?8~**Y#uZ zt@*xZIB*7nw#mn{+MQ*(ZIkBSW0W<}+eyzL#SCatq*nYs871j}d)KcRtE6 zt(-syk7^{*UyjfmqLqwFk|`Tdjz#8~)}EsukHC@{`;(2u%!kUmf+EFi8flU(qiNN` zGU=&I)s>_(w1I{pKP7 ziJ%pUg8xJop{b@&0yJvwRsW3Z`Q1ZA^MkPzfUJqF$LeZbgh`gMj^UhDj6$WdA9!MR zH1OysLw}&Su|P@4A5kBQ@-eoAc06$m z3lYv(v~Ek{x+g@Aeovx#cg`W(REle`D)SB!Vg8uo}dL!W^?Rj6$O(NYJC6jl~&>^I)z4Sb{^UHB=bcBnKY$f(Ipxj+PHG^ zS*Q#Sm~vS&U_1vhU70_RIv&c;%6&S_VO}*f0~$;)mkrqtm11yD+2ma$2ehaBa-lrk zckv~6IxVH}jIyk|YK|PGQ(3vR8TGDgr0D@Pjc3+cc@QzG(*`8C(NQ0w;zHHWB8Q5W zBN}5VgTt3ywFEv<4lgZP>QSbIZK*PsqdKT*$v=~h@k>W5s$7{IO$OEwWEEcXNLFf3 zX7Xv0`Bi6ijmgHH(wm~Ow(vI$8`?S?H2r(BMYqZ3%@2iq#{eqsKuZ!L!#nW@&S50_ zq=;|AIiM*R-f>T14b^APwxW^zEKd|3*Q+dKK1Ne#UJGvJyLqhjR(1Dn*me5|AnyxP zrxU;VgFCY1ne^qh=;7^y6l;5j_AqavC^{24e1>?o21@seh;M=H>oG$nSshuPpnG)9 zAx9OOmW&aPzHK;`4Wm6twqkOD0fiaHsb2SSl>pJ6CH+VMpy*F1`5Cz{JZ%XNwf-@ggNGeb}=nOJv_` zF;dWlL(FGHU-%Am|6%z25i`Qs_v_MMO^yBnzkHU=Iixj7$r6vK-Tf^*7}akRJf@84 z3PK}JeD0F)!kagD`o_BR%b+;n;_;kt2SVMiK)8E9?@r!R-jsc2L|A{m{;lX9Gn|h& zN4^7B-tAbJUuwN9d&26cA~h@0Z~K6P2XUjXo?Qsd<|s%>_(c+q;p!)xjWR}v^yb95 zj13`e555~D9m8D@Fr5;}L(i@_jv3rwL(qfIU829xrm5|ffK|%tLxYRb)=jyfGpVG@ zjmPk=rJFW!%M|u^@&3&AeT0xxhY?%m7oFP6+6uDyvIG-DM4aJDJHnAaB~ofbFz{iOMXx{H^DQzO-YtSONNNEKI>PklHABC@;9^%o2)TI2bz0V3N5&|*PJ)qShgx(h^f%OLY+}znzcKWXhd6lHqiChe6;dS zSd3;$cW4!2rZ1_wviYSLmCXhs_679r9p@>oUnT$ImELsuU9-BkE-iD-iA9RHu7Nq8 zwfE(pR82u7`2#c^_ypCzcV7!-tdvo4{~`WIf&T|lz=uuz>+W)$dW=s4P>x1 za6BP8 z_sPUXx)K>U=pJ(nvkue<^RJE4k$>3aP?J@&(fqN;JM}%IvhmLhD>dP2~*cxjkDVPUFkfN@^g+noakO6lt6(TdW=|>NLdecyF?cWWV|Z zA8BegVh=q}g=K)Rd8*ymTWUH>3M=`IHJ7qZ{Gr}U>e@6Ry^XZ@@{#v^v%+~h*ulYd zW(5Xab@y}^e!cN1-Ycd%Oy8Y)x7JP|4gI@Oe-KgYw~Xz4Pcubi_xjwmf7AziCleqz zAIr)KXyK30oWH8u*$WYp{cg6nFm_nl4u6@S5~aOfKpzb!qjm!9F6ZYdEfzCOv;GtI z8#LU29naPPe+lL6n?OM#&ND}rh{KfnwMpu@NXNN6zbwZ+@yG+7tYu11O>QpxD&z<6 zk!UqXPfe7k4yQfUJ}d5!0bWx)7ci)7KKRFG!iN_96fR^QA?$YibK7(4Q~Q(GD_g2N zo=YGLFW1alYg?YQ$3GCcRX!V36H(eYyg77X)5)1BE+n(_vs-5>Xx0ZAL%SY@EOJhG0~ zk{LFRPuna`yeKv{Q;N}@$@rUwbLdR23XA%X$#6Ry+n5T?(qc7rU(bvxDVUdN_EG|d zvZt>yqoreWRxE&Ze{$C$1?@(aLOY}8%wxX*}%H>p<-<1pw&k(a4JdGHFV)0>`+xjlu*f}Pj1mB`7V?TB28!RF@? zx^4$NDy2^d7`_gmTp4LbT^xSR!oj>7Fi(XC{(i=~zrb~YF6!gXgf@TX;F))=7hWen za28xiRP}?%GkmP9F1eldveBgF5jQNi$GASg82s@;pOGdZ68PqiN4@I8SS=3 z`IlaJGu`E*t61NKGq-p)^5s*IT~zLt%dUNg{ZZ0>8=C<+;`&49Nu5M-umX7LCr8ot z@Np;!H=d0iZIMspmL?M9QyY?xsV&JY-1XYxl=bs!xhiNHDn2614)K%~sqwCpN`;Ya4jAZdJ(tk+NV0M+^YJ>cYv1yx?P;oh4d0m*0ih9_?JbXv_?C4knxkQ4eExxMC{*1q@IDN?^Em{)j- zEcGe1po|6Di*{7@BXV2k`)}`{2#p|V?^t1*PcVAaQ$`blR+LzK+_m{^={&}4c@ip* zWXrh?-4d=?0VQabw|3mex-;myGm8sC;(p7Q zz(Se~9bBds1cMS8Z5_<|W{1u>&m$PgT+1GKZV9eH32YbH4}|Feuf|jATHOQn?_MjC z4?J}~EdTzC%O6=DjlBG6(5gWEAJYTWe;iu>CoPpCG>!xs)%Ee$AH6ToCA^gl>_6ct zB#`(@?Lxt?lxq@vL~-&Ju-%ml7C~(OV^5z%DV+=jBc~4+*1DtkJv6!f$yr}dj)sQ| zA_c>C4)T;C%(WF-Z%Vb0ae@4Ow)d4nkkCcp{B~LRWA*0h>gHT2eZ9QKw7GAQdv z09W7KKCKC)D5W{(iOX+(mC8zUOohumjg#CYKh_wFhTCJYLuRHimKp1U>u0`0cihFc zMApZa&wLEcW<+z0ZQd(0#ys^#i95h>Jj!N7cFcv#oOVyoW<+&-%I27HU&9utI7V;Y zt2S0+-m5oeXWpwd_S5{MJC@)4BRqC}+B+ncd#V$pK4!?aM{&$;;V(62W#O+hc4qMr z7t1{HrjKi!dQXW9qCTEx>!&-mWBY3~HZULlxs@j+Q2wKw&9ctcTu0$tsOb zEmtgsPAzL%Pob0^YK;C!p-Lb1o~%O>6_(5+iC!Uh#38#Pd*m)#n<&vofk&}O6P1>% zLm35HP1YfgYD>0G9^sR%%@~1^wNI63p_oe?$x7ytM=wYjIiqxGr|70YOP3gdfN13@ zg#9ZXWW@{Q%pX}e2<3F1fLDZ5(utTLVXrmU8FhvMXDR8Rz~(iILd}>tv*%ui#_jAz zJ|ms+6lLjVFHV`5HL|a;`^SH`%>53n!~+0PQnw!6BchE>i&xh%ErC8+I( ztB^F2-11OO`%&|cLRpBW8#mz57|bdyGr^4u@Vh&gV|di|K++DDN33#(MVrSw)KN9+ zkvp95l)Td1D5M(PHI`ZnYg3)^#(W?YSSs9-+I}<$FgJ>+#DjuKZ4g9;GB;d?6bJIT z<8?n7I$NAnnQ*29pFJMff8Iwb?AgJE&qA;Tk{ixv@f3f&8I4&cSfmx1!IB1u17?XD z0y{&38Nw})Dp*%eMH98*ixYsBMD{;!EN>itIAUY5cH+q4jIqU9Wez$MfX%~gkWP`7 zaeI+4k=Su8*=NiuEp!GO1yGI{K;F#19uNFywC^lGQi2-}9FV-&ugp3l4z&^fqV22P zIS`n>L4zAG{b38#?hNzqiEgy-8s4Cw_s`Z~y+wvl?%|>Tg@Y0P65eXw5qC-tCET4M z1jcsIy}|F59SFNL1pgbprGIn!C=32~xF*^wIVAq(Ao*o)&N+fCQ6G|69TW9=cr|OwaL< zjQ4l0#a3&GXN6;o#hIMWHl^IP7$365co7~PM3(K3$zOa$CN+5RRYZN*o7GY!qWRJ> ztrAd&Yf6?BxLYePBFZ#EAXkB{5~a1V9x?|((zNkM!g%LG`8W?)y*8n|;+~4IbR|d0 z=*VjQ5Y`g;7PEU1P%rfuVN6Lh@@cVIyVUUYP<(zR7h^f$UTIw66h)TIkh9+i4(Bzv zWHf%ORuVy1*v2lkd2gFs7lCvx_Bk}hmYLQ)~gVF9!5rc$}ed6-P^7LZ#7J5Q}Lm3^XZQawr|K)>1Hl|Rt4)e%f2}I zp$-etPW2@o%j-5&QD(IR8aKuK>)lSe2@THeDz|V0Jx2Vy?7#{jMRxL~Sw`6)F1dZW zVm%4&bonPY2)Z4nIykBNbSRN8flD?~taPoKvPgtMYK7HiC5Wz8GQ$3Z~Olcbqm z^d<^cTvL7$MuH$ms+OD<@?IttCVCkL4{}$xwT;fHJ=3}rT>Zr`SgDP!vn^wXw_59B zOKI;wrqwoTQ*?TbI!!D_BeqwjvCcN>5ai)7awA9v@(Wp8%B+!|>>X=t;;M3OO*v0s zWTnp;;9n8PCegBZGg++`$!*1VT4c=N_0C#LPQS?ViJoDZ7@?h5vGxxKppC`7 z_4isBX!f^OLZE5iUz0@&J0HT|qAotZi6+_1lKPu~+ncBqj2t`nD9x?3HPfbX^EzI6ZdwzeQEBU?-kF-8 zW4M-?)wQ$5mYoMcF;eU`o2Q`X)bAk75+aSIp6xVKi8{uGidA!|tYUmqc}6`pybQ@% z1?@D)GYgxACUTU0%z(qyt`(k#aA0Fo-0tJpL{bJB9R+{B!!@>$ivws=LVs``>y zwv2X7Y8{NMigkuEmiKoTtvg<8u~{jC?#E;fdZq*QHa@32lZa7^<(R~&G!+Q(MpL#` z8*pP?SCxv4IeKwi79ysNOi|Whhf9)*py@fIepf|sF%_NMWeBjXsiHYhB8T5w)n0m5N%CO)90xNs#cO7OY@@nNCy<-ziG(0Vt3djgb?QFZEOed9q2s zwqT(%A%vlGNB!vZkn^l9X#%i~t96u%Iv~2@DH|~5O=F~~NvOw?nC2^S_#(Wj<(#^4 zEA_94g&{l>t{B&12DknpUdC5StKvf;kK5u9b?2Q5VmXh=Q5H z%T82O`0)@P*mlTAH{o$wAQg5+M`u1+QaKKJ$NFajdX*Zlst5@7 zErDDc<(m2yz$n;lNypC2JM5QE;b*BSBVlpxF_Z$k*w(c4#4Fk{SS^wYatsPAl1mPF z3?9W>H9))IGFHr>+#u4}ZpN4@v0KrhXw`I>#&XWJao4=mhk=Fr6AEv+>$Prc4198H z!t{u%j|q40Fd&9|uIz?f|9LMbtX*wgR+aZ%r)27em3{*)-Z}I6DU=`Tusw0>2kwR5 zuc{L-7&G};WeN(_#7p>V5I=wsKbG<5lf3&Xwb(4FE!2^g<>cXPs{9yF^*i6KvG5 z*v&c%soES#fsTwH09&%;b4MJoS-^b0RWXof-q-4c8`VPj%VL@bWK`i|Gql|ur6k>t zrUoW-LEbRT*$&?Nvuw<6OVLlE3(jWlWK&D-u4}!DRn8Do#ROrr@eBvgzYxLz|5~d% z&1$o|afXf-2zp505&xkMDoG^y?YR{2cO^^cQ^u`=0NNasKCAth!iNHP%?;sj8=bbI$c4J$yX8KZ>fNTOA5)PG;g)O5_aC z^{#hOKkK?5WwlFa$Qb4=Go-_3GIGHQvlI?23vgdfpQ8)z-#MMx1?CBgfHCGrSdkLP zF4Z)12`FR9G7hcz^CB+7J+#!aDHvurW}@q%hTTa9MvWTUI6YjsWXmMzy>UZ{t>(mX zxPM{OaE8U!>A(JIc$ab`8A9EvduhZ3hkS-LnHP!Z1+fF^rjfvtJ3h?w7^~FegM^_U zb|t?G*SrczE{L$mWur9CfNL{mToA#8Yr-BRFx1CDcrgFnfA0(S-&+g@Hon~;FY)EJ z$NSlB_guFKED8JM%t`T?2#&^Wc}P}Q*HF+%)XR?G`M!w!Yr7tK zh2tuS@>O?HuLFzGEvFZYaUFi+U88NerJ1PpD(LvQ$027&g9QdC4#W-)!^zDL8@_Bs z9%LR=NS~~MMtuaW`;FT<=WHVLZ7vYM!xa$qUh+<#3f8)REN#uXpMGBz>C3_VKryo9 zlB!j;{8`nIb*`3cOf}cO#hI%qS>xB9U|Q**3vJjJ>Tcs94DuzvMZ>mDG`XkdwNZQhI>YnG0vi;K%fc@KU>%cjt?13JhQUhU& zRInml>!R&dZw)!kb4w#nXs7P^Zu6z~NX+4cN-S>}j4tS!H1F{Q{91S)J@YT+NQU_c zUn9;BQSZL!7{Yw7A52~t1_Ur zVwmg{oXElhBlHa8u~<6d$I?_s>!d|vq`@7Cq8|8P^7`TdkWDKMl1W(|@Lx{9guVfi zQI^eP`rOTQIc1jLLeo#Urc~wlO{5d`JGi5ucMWizv5<2_gHHYi`rzsd7y056ZS`Di z(62oaji29bWPP;$d}gboLA)5LV&HJ9_?sFE7_b8XyC%o?Hx@U4(WXd8^jS>r{6-d< zd<-V{R_A=PBdaIM8~98@^-!GH{Wyc^KEml1nmJyU{AEJH4P}^?#eT~P*7CX+sqX+s zV;p&mKBH2$QNRRpcCN5zT9zLxBp;7M^YgPm;4!aTJasLzpmluB z{o1DvX#Wk$x7H*)-V3(!Cy5%(Wt(bM5ST7ibP%yzpgc1qr*8I~6-^-H`7E(*&kxd- zE+cqIP^uX75FZV;%zam6;9i~q+V7jZ)ugo;dsRv2(TqEWyf$YcEy)Y2- zavG*`MtaiM+ZF4e2g&uWbN7?obJ!J(kuFyX0sSG?Z+)kH&q@;_Hiig$C_R(VhNSOc zr2`+PEztRxgux}(DAoRwMjZT5SOY&C==dQOK~2}B{Ls*6)*Cbsa%JWE;sLFh###cv z;r5hkT4OQhAfAj_ zx&s(JkY(orOHTV6xS`^HA`?a#xE#p+S>1)G9zLiKD%YnrLQo`@b>A%tEn#A(7YJau z;H208>#XgpbNDd-|39eox1;}nQQa(XQQc{cQe??W&bZpe+4QVA^!p(|o%?`D3(|-ZE~^~a?GUKm9fedn#z@m z$`va&P8w+4gH|uHEFltjzKf~%U8p$!!P(aTtv-P;HRZl`DM@5SNy&y3qb{h;^7EG? zSw-6qBCg6@0>k-<^t)5~e(olM6AQ&Qi96fwr>)T<@Va|?wq&qe6nWDPhGbaZJT^~&nNkxZJUhit3l70u zK8!qNWba;TvDZuG_>QvC;NGt39Q;R!h9;$lH>4$NqzUv(bx}3r-98IfwcE!|aMP-r z>KD{395c+GHmDDP+Nr6O)9ePTTV#hW&WO484Pz%Hmo>oG4COPoDSRu}l#aE*ET!J(`)9#&^fV%fnTMyszvrQT%KbJ)fh7$^^k@$f zy9oZ;v2~}+Lr<&|yjR$k=mv_f>G3026cVyaRC7P7U_CM$xVOU>V~D)$UA z+R~16+0Q2+_cwtFSIdC3_cVJ|A-&6zKqHjxlZH_gKGbhR7Yyi&JAK{z z?bm*6?%@2-9!8W)^gCBf9Qlf1y^dRC_^fMoFWLOZiW)Z2QlUpv-u}kZhlFvmfo1_C!g2{zU zj2KwEZ=4b<4F+b*7&@R3f%uaPq7+y>obCB03L1=m) zMo}Aph_T-EkUnRM=Feg&SF7d-E`#s`I9Z<1>&8>Nj8YY=I+N(~dk4EBFc9orrFCxP z0jI!uySGoH-c~yqU%R6G;vOAP*r%nBVbMf5=a%OwPTu5*g`8jLTO?(O!raS!_>nWi z%s^Yu)tsx~Y6HHWPpyh`^iL{!%1!viYk=jDZ$0W&K-0BB`?sqiDnB9HQSkfj9mU?y zZyhJh?|4LF0m!O=H<;stqU32oiI>GBtuSEUi25@+=N{Wy*zCUI4No_G-ss0O9{vc{ zwdS)f3(_x4LLYIt0nK67cL9MGAj;Nr_gZ9KRMa(M1HlgZ>Tcf9!?Q&L-VPe{wcQPK zFWL6U)6iGSDNC=t<34mM>HFONNT2rcfe`)qHQ8jwbHkC(`;Zgt=VT)RvQBSY7(K8J zzMh{n_(T0m*i||-#TnNY-RiR4sP#dE_OP_Q{hZL?Fpe)Ctwre8c<-{Oh z%lT@@`FAtFOfM!C*QTN+(Wi4kSKGRT>y=@^Gu{_dks+^WJroD}(BE|jfarUgViBZ) zk>#nwF4|M^!*beFiNm(q&!#Mrn9gP_ikNq1j>+^gX3CUNA&t~&!=%~}rVUz{vZl%; zU;S=w3Cu}TWtu1x{q&e%c%Y!IMSJ+(If6TjPtqPB)V_;3?BzH=Yfmrq4!iegP)~1n zke?ikRMQ zkb>;7{p$AZx{LP-x@-6Iy59srz?;lSbt>DokH&YT4~S7SJVKP%8eoh1c8-p!viQtxjTcf9V{-4K{YcT-6lJIbWq z05@B)524{j>Njvrr>@4bW8E#sg@OVJHUpN&g#Q4$1|RqnhzrJV`cYF+Syfkav&;^# zEw`_(w8HJcLO7OT=c=x=v##FhM{7B^#Pl90_lV{_TUT8+^n&wv>%>{Zxe zZbsU=ZW-P0jnADMLcj3XObkDkOutGA}?%fh_al_X;CNQL=($S7q?x=;7qe@ z@Q@TfU-;d68sOlrn9qQ@3hufku-srNvNI!xVZy8pWJC;hDPkbJlo{F&sPrJ+tZj)A zrN)P==CC-YF@VbHWE2yTj3Y4}>D%Df5T!5vVR2OEP}?@5)QXCx#~qqqK!tXw8Dz&^ zax&EG!tSj}zK$A;OO9z5r&OD8UNf%4Ad`VJ3+#+md-cB&j@d4e6!mJZq4L$hEcUl& zcKksB4DT~_Zt)Kb_1UX4_431V8Qza5&{`U56(?h1Xx=7I$_LsG=rMV}yOOEaHMmv* zpt!;WT`2kH&)y|p&u_1ouH#lx(;WBOu{EhSHssqpZyS=}W$)(BZ{w#_E!O4o4pBH2 zXD0o^{T(uc`4Syth^%;zMZaz)5$v>;E03Ijl5i&$lpH!DldjSdBwJ4NmR_$ip6`QT z2Eb2RkayC1hgoQa<_q#s=-;f9BU`q#^RP%ijykqF8~)kehNxkLJ9eah$JT^7?3qbL z?^&aAwFwUk%Esy@5qP=IX56?6)rIJFlx6U8<;ao`p{5p9ghJ{zF(_s-zSW>kH4|3g ze2N_T{WIwb&8~GM#ev;xCUetzvr_nymHzuHBwVwH-nmtj>qENp`8e*B;Wf(`VH(c& z45&_A8YN2!eI^+>Cn@{Y!~}@h(}XaWmFm2tCDQH;z?T`Y0DeUL5mFAf1*(QPsYa_9 zpOZD#xcsE~EI z*jK`aw+@2(Y`p?2TSj87vzO(mRBA@^%%1O^Olqdiq@-;ewIqV-Glp41aY(nAwrKB| z#jvz@GQ>(tBOWrxjdF7*ie$tXqD~nAhbzVuUt2iXRYxzbl%R!?ut@A&&&C({4Hg}h zVe$Bnh=wg|%@CYuQtj|rhUtufWn%S;`BpJcrXP2KEwHoIQsOs$i6ZxJ!lE`p&PtOU zeN=&;J2WjBn~5L^I}6RgM)$-Q|G@@feUpnv&Ynp)U&l+?NmA{HFf?+Y^u+N63z|>1 zGzhsRawqmdF4DQ&C1U(#0uuu`;$w>B=k}fWpSdr+vIm)0Gg(7OBPrT>y@=J4corgD zNcLwZd8s9@R}<2QHbwnmy?0xGF?hj9)h?V>-6O`XOG_gl5iu&*IN!TpH{sglD%`403V{A66 z+y1goO$gg>z~oik8IJV!OSR&jQE8`#tJ}}-ADa}GCWV`AS8mpbCwh@Rk+W{&9=x~N zFYX`8I0}9+{)$|`FS^10=7xJ_!MTUmoH;H;Vdawic(C@?-$%bg?vgo#CU>~gVYPB* zrIGbE=IxA)g?8ZdK49zu#7u}GWpz3ECYU!kPGfrps=A~(qG>L3&?7Q%z(O!Q|2}0y zBWq+Lw9*W50bYLh`{@8bV9UdQ3@;E#w`B1^?#I)dW*j4ITwyl_b=h1ie43|~ z?N$i9R>msJTHNn^<0|@V^f_ZDOmBkRz{vcnsdJgHgZ;x)?S-TsP=qM5J94#!hy@nU z8-A!)F;~E!hv9>JfT5&56Q^oZve@F6RDp>Q^|IwfZogPat7>_v%{{NQ+mD=Q_Ris5 zK!ekBp>cyQ$=7T^2iL!$pWukL29y&9&lrB5Bxz-u&Hb)$!a`=6!S0t9b$cvWm3=Dn zcr4(pGo8OSW~~+frNUAiBRxa~KAHxXVNI@>N|>{9u-%wb z;`A&^Hbk>BKdw*n6YWiIOfLA@B4NgIXZ~T3D;VI-;jzck%~o5o6w`c+DJSZzDSm|{VF%;`Z!ViBwpt% zb)_`@18+|54vyAp)Q~n+*{Zr2&VwG)i06Y3sS0ICr@EiAdPoC54v-xlr5r+_B`;nD zs+tekISoZwz(IV6@%TUww!H3Bsc3||z3*AXL{L76?Ezd>Wa;nf-T zw){avNqgz{au8xrZg_qCi6HsT?Tu>)xF`YTgI(F55Y^UF6m z$wDv?-|Bs!Qv40eOBOD})p*Sv5FIez%Ars!JWFB%rCTV6=)ucbROh5=)|9NxY^-<= z9FVL0Wjy@p1;s8-jpPQ%(#5w-1YOyM2t~iesUGGE3m_H3Di%U2CLHRLe_5md`>#!6 zgO`sz+ZUx`iTyuFmH&V$3EYs3X?H}BSior+a2AwLmGfE+$N@ny?>Ud1B5IiB5X|l} zqi`A+e3)EYb;uE{tqD%y%C?zCa(a_7Uxv1KcEBDul&#p9a*R-LKe| zy1-Ql4uq?wkcUWkszWwMfo?}D0Y}A|*Z#Dn4M4ux&*^PhZ9|^upSsRTH$luI3Ba9y zdu_#ihc%P-zHC4l$~GB01jbO|Pg*|n^lIDt#7h@RgqFf5#BJ zpR`7Odnu^UNg$^lRto0_;}MZ^=tOFVF?T;qhlEL}Dn3{Rkx@41xYTyh>wmGG{$3>d zqc46F@cn;$jQ?Oe|Lgcc35lInLyCe7%+WaeGC4;7*JGYx%4;X1HK8;_`g2i*wFS&N zk{W$ZhHD)husF_+K+fi9t`hkcNOy@B>30kN%TYE;09wc^_FvMZFI$0pnW(aYqWyN) zmHX4%Ww+43&kHDjKcBpQRakt5T`4eo@@sQ2B8uxA|27Cc^y^lCcZhDfU4idcK=^0j zev0oz@XuoXWnh9ot~>mhAbeAwG>YCwu}U?m|92;=dLPNL$*K}{GECDmz=3=Zze z=BFtnKz4GH5{s2me4b7*K|-a~iYpP#8R)qwpCsV=|E$#<%3JSCgC zvoXKgpV$7@Xed1!27rovC0i{Ht9ff^Lrd`4Ctis`nOW71)~U&4wRY(gK;x4LyLAfj zn#$ie`YfeP)R|`IG9gPTt|jGsnRaUWlNu5NGoO%9u*lW)w&-{wVGmXy7JBD)DUk%* zS;x3sW38+U9)jcL%D*a49Yd+#>^tx!<>?DUr>&<;$n;`_kJR zS}bRN`~5D&MTaM5v9RjGnRr|lIp1oHjHQbreg>DHBAH^anH8xv6d)IXgkz)1BN?|5 zUhOK?K5CV@4KEXd=*eJ59zoz?JF1gpjfQKJ6nsq91oPipEK{a~jmh{PIgr6fTbg!l z97$;kLs+ypN2gseP${n<>zEmaeR5<8F2^O=#a&t*X9ZHbS0{ApHmFnt@l8D66w=7l zf#W9dM@>c-cjFk({_9N3ytPP+>*9km<$SME61XN;D}RMIL0=DcVt2Zzb<9`zljrZt z6RI+v8I_S}P?`6gtC7}Dw;e#Yv6WeM1bNERYgut6v!1XFhzWNOFY4~kX5Il^lc{aH z&|-zkYVMi;@tTa>XqQ_TyK7i%9HK~3>3XC$Uq9|3)FMA)pM@`;XVWZSn-EgpdiJkR zGgjw*rq>#Q@5*wM+4SG}H>0)QTk#&d=eEu@dD1o()O1J$Jdj{S_et!0U;*7mrnM@PJy$nhv%2jm@&DM=;`}0nSfqe%K{;ew1`^}YjRfKIQ7Cf1H6$5{V*W}>O6FIF0ME9~m4ps) z_?5Xl55lXZ9Geb3bTDECi4-MX>z@vF&gDZgVdH8HE&?q*VAbzH@; zeSyXM1Pap<|-1DpDMjV`UvThq1d}k#*&+cqfqJi>7Fj zTl_GC26b43t;*T}WXs?NI05V(Ub8I;p3SP^IFTmjR|dUB#YJ|~kVq&sbN294!2gzG z%Sdxi%`b|Q9pQhJT1eo9QnC`EON9wLO5jr3Uu_qm~i+{(}58; zW)^Jb?|-Xj7n(kVQ_%&+*Vsw$)_;@BFPQyNXp-AVcIo6D*3Axa381uNTP%I+opQ-I zn{qMdU7T{EJma~NPK`E$lNJQNUJ0D!dS7;&WP4wxyx*SB@+0SmZddJF0BNT+Dr8Dh zXw|ElWlHj8Y}9nB7KUkSRL|g2{-{-Ip5dm{s-N+u0F-EIRYXd%)MP4?H7XQK{89iK zXN)PAs%L&F+eLFJaCXY&+66Um`Xx$HA{*$xYn7@*_|Sh>%rS=iM*m$iw+wfoY*{Dr zh7L!Et~{nxE?WQy@j_?C0RFC<`yNsV_dtuTdYbbUwlNy%2usK`0_CG(sJ9ClZ~*oT zjhp5gBOnd3m-dV|P?gkN6PHN6Aobmr77C{D+|0t}56pypQIt zx1e5dAKG(pKeDe@DASt~O*d@-ZUD!#O#o~F(Mvrh5$1CsGS-1O5pZQs6fExtHrjJ% zKOGp5>iP+62jw|6uI<%|$l6SzW~v-xPY1R3e-A#YyXxO_TL~3m-edYDAp~i!Spuja z1usn``Tu}Qj4#bRe#aeoti)kpV>}lU(K)K^(ge)H@Y7w}302-`e|5;ULWrJGhuFYo zVY|t%?E>1`@X?lPfje;TRsDI`V+^~f*Kp4qv?ZRnDf4{nI&zf~STOVj z#xab^29TeMMN!3VYKAzUqD3-+5|cNX5_H#!xHZL2F&dNo6wkk6tZ(=NfUA}Qel_z@ z?-E5QR=9RUvmBl1{|e{LklvMxCdCIRan|Ib*sW23jmoLEgF@CzvA?XANQzd+aaR{b zVgN(uxMym`%2a*@%6d@r?{N||?>~#!rp=`?1 z!I>Y|%#n@opukf#G*eYVMR7QZUy0W&e;Vb%CLK%B4f-LN@RB9ad`FlWUlDg;+k>2d zBM=n?M>P#Gi@ZXiYMutzP!j+Z7bk>MsSW@`p9i$qYbAfHc+uL~HFLF(jv$G69^Kl& zT{Lw!cI=-vpRdf{hKbITI=?q zGtwsUBdii6jKl2|yJ3>%995BMBQer{#UiKDZ@rC%IA7G)Z{3c}7`}k8o>PU2(pUb9 zl(k(b%5Qp{>OX0>rS2E{5qsKwB|noBVd8C^m78ufnPFlo5xDVpo3UuN`8l0a(qo5j zr<$Qr1!&K1dG;~TiO$BiNCRqo!YZfN!yK|-PJy&T*f4$cXrr74u$iBCw${H}$%q!t??*n>BHBIDc9Dts>&3}sWIbX ztN6<-=fZ=QPb`9NXcqFS#C%Fle`ad>Cj93$*U|-`yP(2#wY77L|F_!e=E54Vp9$Vi zHMu#R#l*X_ZBgoSg|CQSPvg{>d)25k`ctib`mekgqI65CN26vM%yfpq{fS(hiRb4= zAeX%u{-VuHFLNz-)p^lpjb!+)O;lNuktiYZ`!TPXPs-nDfeDVKvM^AnnR<0?#W3mTUxHeRc`iPsE+~DoDVKecH{7FUgC3 zQ&fpmA}u#IQfw$*TY}yeH&q~hVVe5MXUZ-bw?`tnq6+^7&i$DYX8 z;E(9`>{$e8W6=RU*?sutJ-qHP-3k}G^k?#!*$uhrIZ~hLj`ctu^O9#VW)9u*-T=K^ zFJm|JlpGa0txzS;(mKq+PPVRFHrRAszvXp|pM!rEll!ZG|Lt(L#xQ!pNt5>z%ad@^ z9XVsGX*#RH)EnGxt1R3{_lhEn5);4c9$cl*D}N7Y?ez7evX=f-ibIvhmZrl4Ne|<~ zpRp`X6q`$LDntPP=E|AVUF9zIN8#ulWH<9P8xt8Q-CaXG1Liu^04&@cIc_# zQ<dISeO_Bmq(xK>6-RtL z9Tfqe!(0b!vS5dSPM21|R+CZ?VmrD&RNA}JkU-jdqG=l)n;o!Jeo?RMqUgoVn~R}w zqEuQGj64sU z*AoY=u?`i9oiNX|QfL-oBXUE_QBHVpl5e0N`NThoGhUiWNVz!yvI{*JOzc7!t>I4 z8q0*4z6jF?9`VdD!>;I9koY6sFb5v~eFeOSApF>|%mXdfz?@vXxIrRnTo5B===_+D zy6!nFx~6AngKId?uquywwHcV~?+!csVI)9n^Z^OiqlRN!fY8JH`cNL7XDtxHJ(`3y z8aO@GZjAuGX-hb|#fT{s+WB+J{W;9(y{KIT=94{!1pihczg1wDK%BpryYVV{&!08+ zZL1a2d@DO^CW^5h5rFKem|y(IN{{e5b~n?AUA<%Yv*$;9zAqS7!|GGYG;V-xaX=QU z)9Zuh2VKdPar)B42dYbS;X_Cxj{=N(DsY1rW>zWJ^}wSIesuj>X&&~SR%vK)c6x}3 z6Kv9)8pl^U%S+O{-45b|^Pn4_%#01w9J?)gmpF+cqnsWFUEWanjobpAflP{x$3C;a zJ6o72=?EvFbo>(=fGYB%2L_F6Ex$+A>m>KJcHFE+}YpvS1VNEZEv7f9*QFk3KMfz2fz@p;**g zHS4OSnswEu=i-}nrQlh8KuEEg0b0k&rNZmlTg{l|W>118F;?wE^fIdb?H$sb=^_~Q zh9w#(^)JhIzY&xM0yBUR-cSwfhEKJG1KaV((S3R9Q*h|e&HeNz*aSE8^IW+`cLY~h zcNtrJLm?N;0upg16{?@*;NI=;c1I%a7{>*W7;B-{wxeFng{XY>@O(zQKx=X>0GPKU z+S|3O%kEn{l+duN2jc1?|7kNqPBWZQW!TS}Q}Z6m2V)@u38t+mYyp36#C=LH1mAA2 z#9V*zJ_(@@LhyS@{K8d-CRvb?CJn%qVc?<`cXO{2lvAtEy$^T5sF@A&2sXJ#xpWqY zc+(g4=y%0O?lc$>trgk@2Gy>zoH7oEZ5sHj=|@U#o$RwbvGQLU1W|4BT`Rjo3m(Vr zO1q;Gx$i_2T|Vv6JpFivNQ+bK{54dz`D?grYiGz}vu@w{a%b=EQfM!|44W*MIQPM} z-#HhG|H^6@J{Jl93EN}-2MCM5Z8s{{`n)#sby>yoeDq|PE z)G=uitPWLgA&R9FN&p3Sp2|x_2ecjtQ zdcD{yhwi>88=hX-t>^Dh*JPW~dAl0#(Pb&|OifwhK31LW>XjPpl8(9x6FmG88&sjk=xt4{>xqO1nn8PWj)-8q}m( zvbJa!sUkGEV#dELDZbm%Ha`b0no6ZLt;cj_E%-etJ`+9XpR5%+#fB;RU1NFjM{cj% zN_VaY5$=JuJDz5o^D;Wz&9Cfd{XQZ45n5mSF*em4WCy6=R?w3%s_CB92J9g7XrA>3 zus*PX{Z=$HQa7dONh;YXq-5fpv4q)?MOZ%^#Y!-f3XxSE@5va3Hyq17OuoTwtFXjb z(J)qONf-=s&}bBE4Od6TIVubj=xH>N_-1)%4(oRp?aNM;w^RB4+Vc%vQY(G-zjCNK z`xC0MM8Gxc@Z0^;y31A6#$f}swI~H1uZ=$ih;oKSmg+g&@>c-tQO*p&hM?GjInQ9H z%_U~=dJNWEXtv7F8K%`1G9MouwJ&IodkM!?1;s#ru;7&BV{vy{FWz_sb z%gt`9`iRXP=a1^PO7{TWhRCA(V{@b#d<{T73u;}x=FrN+U(=_VDI%DSlpHrPbYeEWaqY(zRlPJ)H^Q4MV_lErJA4*f&nOgV*{7ozRme0zv1&itD?8kgDvI|m9eG@fi1fidz$Y<6q zuKKdA1{BUxK3j*OO*(96o%9Z2Zkwt<7zKC}Lx|^LouIB%Y!Z=ZPr%v|MPie@7sGsFx+jZ;@nO9`J23lWo>7jb|THQnVHq?+p2gtt2q?bS-5DYZb z_m71dY5(It4;L>c zb(dLN?Nbd(2|7VsDnIfa<9j_V z4cos_%$iyIST5tJ)BtrOMI|0{!ApDOKz@R4OFq+>h^Miur(r zPX)qVk1RftyZ*xP25gB85&YGJJ0693YMw%Lf#^q;Z?b9=-@Bv66KZW4T z5OZgP7!6m#j^FQMOu3@^NE2imz@z=5EU$`kB}oFJ3TzcRq;T;|>wWr*--IGcMnwaMq4=XpGnlY_*WND~N~>9ae?E zFT$B4(3=eQgC8PP@1_Q*FM9_wAym`cc|!{A<_5?Q-?AMea<^OMM=~L{kFne%rc)?F zl%7kg+HM5k8=cCP+#y(~o^=4_dxsgqx-hDUeK%^9gpGIf(^^fYmBngtZt^4Iuxm>e zY@Bm{HF(5eaRr;%UucQXVreYXEfX}#uJ|;yk+=Tsf86wmz;2WjewJJ{y07fn2I`Gga%>0QF2~M$538zYhI~-{~NGxAqHL6hd3I zOI10bvF`&lpQ1zvzB)}7lkcb$sYZVZXzky({?ulzS`RWPmwDQOg?_S9*-@x^nafA8 zio&HdZb!zeo(G&Pm1rG01=S|pifxjV855sWD>V-?T;i7(ljMXRi=$k_U3`ixdVFY! zqD@5O+N@Iy{~~`%+?_$a|Co2k8f)jhz%f9!G&;CeH{o_xB6SD0MHN!ggzhZ_a9wd1 z4(0?Ns^DijcD6L1-Zu9EimQ2uhg+L7C4Iw>m(Kr@geDvi2rf@HDCQbHYqd-}oc!B{ z4E-_fRj5)W9$VZ08DcB%rP@2|j%~aEzZlBmmEz*Hc-mM=B>2!-p;!mAw`xi;K3l%b z*5$zj$|b!i=J5t9yjUXC&kZNuBiI?!M!q2G-+$jW1h?=LcS62d7PzmzEEJkw?|(ZK zl$8%eWxpO^oDrJ+TVU_%fJHP9mLU{xxtm0F=gn*|>>A>Tg+{Q1W4)_~z0naBRRwPf zY-fGXPMuA>hV{}G4UqOS7+9Fog2Lepq{kb!OC5&51Cax}BH{2PQ}n5B=-Pio^r6~4 zt5BQtd4_&bRRvEtf*1Wlr=K6{m}DW<7Xy?l{}2jwqJ ziTnw3S%e9ClxVIpju`T=?7C%4DBDz3;8Fj;{Clz)oPRQMrHZf(1~)=>e%768eY1(qQyx>os;+1E^s2z6}3%;R2}O z^=Rr141@Y50;(W#NgZ1HDFbXEYBAascD(}dAUNSSsjt-nexZ6~_SeB}l3%+7d=Fp% z3jz}ba}B5g+kgOIVq><8?`i~~23&v@fti5uLHNlV>KTd{s_e!=d}{65f!SkugaP~W zV7g^@4FUwfo+_S+{u4`*U+G_&UrAl5QcTOK-cTjeP+49{sy19etDx>!5m`!0QOPQ% z9$Gm8qmENSTT;m?qaLf0GGDo`oWfGcs-ZqoG2x+}kxwgG6{)FCSt*lIiC;vUq;{mG zo>5J!s#dc3efpY9Gup*AQH=09&>v70OCwyT3w- ze-ym9M+SzDK!%eZdXR(S|Af6wC2UG#q{Ddub=_Mk8$dQiic`dbx;7F$Xf z&u8qKq>-7aR($4|0?_+IdhTdw@B~2ovP^rF@@@UCc%a0d^R2 zyz13MQL)Im59nl10fn1guPH%%5Ly?352vQwBanC=_iE^j&u~6J{oSDc_^Ag54Giy^ zU$e?N{)CXU_MkH8cVGcgAncg5r|$70UJZD2bR+joI-r8g5Hy+Jl|X|0x}dikCqyDc znztJ-M4};C4j-5;RYO#O9`g4;V*#I(1{@s8N$ z7q!9hPT(gA+A%2$1seV)!ucG!H5h4P_ko;Uy;jG(w%8uD;eW;I{t4=HI=AP)aNR83 z+wm;my5>g&{u`M3cj((aPXBL^0JnFz$lm#hAk-aa`(Suy)ZXU#J1Feg7g1<`WfUQA zR~`2=OE-_zXXKhEw}J*#ud5oG#2A?cL9Kd2DPJ*^#ZeMDFv^VhWGTY|a?uIj)sW8! zXe0hMC>C~8Mu~~s7E;tJMUO2NYNBk2zT;NqWn3B8K59{8%tu(TPqOlYFFEnz zGkodA+1oSt2KQh_3Trl9#>@xFlQ28im)fWfzUQ&1POx7r{%SmS(6W|}&HNL;UGRkt zvSY85K-#=2M-p)CS=vOWKAE>HbT1T_4ye_Y6fNQ{=z>>BE&L$|-wQ8rjNn4wz>FCJ zBM~qyzbH9t;N}YRu3-y54+yq`h%(1lYZb)<#!wE9JCp&cZ7w#Cw(xCM`M~TJmJB94 zirrRF+j~l{jym;1)A=l-r4hu)O}6tVH&9IoOlLuW12S-OsT2qHv&n}3{szVc3jD(c zKp6Af3@aQ+4q!}M|v7)(hBo~OWKg)YFwSNrh zYc-)`E=k$y@erYjA2Y%X(+VU|lVL_c1jU->wDLO4r_!=D9_KolB#Gbtr|T#1HJrpkXM|y3@o1u>)s-Bqw-& zIz2jmqA!&TL)zd}Zel7fS`pSWqRH(^hq{f*MByqb;aCPUnP%b=Gur7T9$q_h^%)^>s@KEGlWWeA-2>Y!fmuhvPHeR_3YB)V%Ln;2l zfly<0eHjJ*ti`7z-bTI=H`y2L?iGh}G#c{k5MLE;9=>z(b}&0=&NagEijs1iDWZ~^ zm^Q4@+?l_XVNOIlo8u`NOQDaSv!WaL$>2c^KM9?pO^W?{zNk7IW}UBbeRAt6!jrmlbv6Oa6a4V>#(v83~Z(u%14M_ekUx z$C$%9BD^*xGi2H`0QNjToJ^x`NX}?PrTk=`a~ekzf7%;A+LxO+ z`+F#Jm?Q*%a2g4@Rt z4Kxe|`jjhWR+<(zdk2#-eit+xg=+S3S}v;ta@K51)?cMlM!mhK>gM`TtU2c-o^11A z9kUV>-XV;LQSM>^T``d;QJtKbQm*yl4@xx|scf@A1skVNljOp#e=)`thVdF@h3}z^ zQ%rRTHa6e@dRe7`YMTlZHK?fl8)2+8=tG&138LHFciJdp+g*6BI9VCGaqsox8tD(| zmhAky_GnwX4YUIY-EQ@?^$kpp?BT6C=KRJ0v=PIdA(+M9P49GrCdp6C(y+5MFRgjm ztFdk1v>FCNZKJ^ZC;UjPKN;!Pt&Gb4hQ#A98 znTgm73>bk$xa=(XN?6SGnfA|YSa@gpb7G+0vGlSL&G4T%)D0b6aM7E5;*I@bU5j*4 zCibsz6VZ=}Bd_8hisT`c|PF(t95x3DaR&;;nE2!3Lz}jurZ4M&?He4zK2HN(O{!t28B?wgMw}ekmg<+8zSoCF z>6>C*c*f*^GTF01dY|}=PnNIu12og@exk^KAFGzyd$Dp$wqP}u*S2^zS>4*kuf`$? z2R9sx_cnMYyUNms;N|war>s-tdK<8EDc2R^7Lx^Xbwh2pt}4s#*90m@m%-zJZbUvA zOLQz#3*t3J#l7&+inxj1h;2)T)YZ%c547oW9;EY_KU_&(UPy1xD3lOo2aqSr-O=dF z@+8WGT-j})vk_L6$@#$^uG!i9b?f1_!&K+KS2du2K}{?Bv|3@XT5+0#P@BkjH;&wh zd13O;yxFl=qQVt{@r@8JC8#=IPTO;(*-%+2D8VV5YlUJ)Rf&P&jo`n3?g4v~22A=g zM69{}FDQzs>r3{ip!0{GOZP~C4OV)$45TGb{fu#ca({B_U9nKR-wzU@7TZM}3L5yG z1;6E2Wbiyb_IKPCd-?#}XvcxYUrT6?0H6K+fM3xl&M!P~Bizd^@G@QC1z*+&DdH26 zf5ecwIX8#voxYbZSG(msw(5#l(s=-t@c+lxJ4RR1cHP3=>5lE}j&0kv?WAL;V<$Vd zZQDk7v|~Hz*tU(6=f21Ljqg0~80Tm0`n6Y8EzE1qxmxS61E+`g%t+6dzM+OLjgHNA zP{XfWYlL$uM0jqwLO~8Wsyx(*RsINkk0h1(panY-tUv!e1HE7I+)7cqHGT8^^!dUb z>Qa3mnGJCr32pMk>A1`Udt-OIG}sXVN{~|%4P=DRQoe;5y<)3LxCE?4`D`(>90lc2 zGYl%4jL(c|x*i;mCphI<)EUvPvPlz*7N)M^886_TEg--m;=8-x?*jx@SNfwQ`&RS z#$G3C7;;+jB#!-(?3H~xOYCI>8NYxG+=?@J_@#RP!ZMAq0DKl>{3O3)o;Av}OsdN|Mo!E|B+l@0@;YolgvKZ*bOf$afGR{$hm zW$9|QVKSZ_N#9;rF?EikulbbW+&jM4tmb`UziaM&%TRVZ~s zoa%g{eam*7Y7pcWm)G~|!G}#T9w>H7SJ2r=hh0cL{<%uF|DQz6pEcKj)dHMA&$95g z26MJmvy%Y66=J)tGfKDadz{mj4k$$bgpYtAO)OnmZi?bxwdFlhLD=K_Sr&^Mt9D=1 z9(OxTZx+WgnNqa(J4|bUjd0YFUh$a6ODCMszRmh`N3#_)citSF&)KY$@#{m@E#A3i(YuK4df~-miydnG zeLFKSmDX(EWF!rO&dgW{EiF)E?5X~z(3o}N%#>W4lD^NKDH>E7Fm5${TtdvDFhxo6 ze5i43sPh%)sCp!FDq3$hGlTtf6xYY1`*Kl7_u8Q*Yt~%uyw2bO^QFFoIbl@aEu-=c zA#36f1HyVh+2vOpMErPdpa^>Z;O=Qod;&1Kq?@}nn+k^f;{w-RX@rf7Qm_X>O$9Jv zSGA#bRD4|DU>@B8oI9_Z4(o`oD;k%WDRNP|ky0K+tuL&QnaQL%y7n`mhNL?R(w{A3 z-^KA@BzM}GvT8HI`<<@o74Kd)RPaz|1jpgtCiVcz9%)~t{@dQD@LV-E+8KZK60)D3 zy||%A#~&%iT5;DAmGCuioq*l8$G$jfE9#WIz)gz;-JHL*7G2Q!wTV zXYyOpNlCesgFYmt4s6hfl&+5oE+4Ug1HPs^&F$MnkM<=;JG|MB6$j>q2!CgY(G_ZD z%qRZi&~xMltVsLoG-4%sn`L8bZ8H68kM5M?NliUa$N4v=A#1|htmsehpFiloef9H3 zeqx?oRYxA99^yi`LVc$rn@{OB*MZTi45FNS*mBsGLQV1S_yGS0`5HYqs73kmWt;lH zk+1)sneP{x&x;ga@2`j{Q)J+XDcEY@uq_M<;BDalE>E}f{#361=T6J#odJY@e*`ss zBTu=s0*6h(Gyx|B3XugOgF!S}c1zb1f+7Pe@>Q_&!jY@;noFw?h9$8wjJL8f-R>a)R>%lcwo4hPEJ<>&qBG-&`pAI01u*XouNT4a!ka-XQMfy{7^cpIHWW92KqGO_Y`CGx*ZUy_Z(ojBT zIzKSf2=rF^JNKa%+NpuQ>8M|ls-f{1R|Yx4SZAu&hJk~}q&`6>lQ;zrB$GZ08-+nG z7Z>k03aJ5Qmy7j=Cb%QpOCodzFceWK_I2Kjv)v73?!Ak>M;2otqC}|3eivxEyZ?lo)CW>9Y zX|gu?Cm|Ona1YL&>iv{mN>+hBck~c!!6dszzFUFX??%=z`C0Q04-H}(m9Bo!|ZH}>3 zH<|@7JO(R~K|L+D;%C2VtPXfZwZF4E$R%2?&O%w)#7Vh*DH8k1P;Sxx70+^xKE2og z!fax!YBGEnT1st}3}3IuuHYPIUhr=y1qUy08nQBTq6e@51V1~J-cwJ}E#^G4AT!vJ z$vB2iD{paW5;P7i8J=%yi3Y10qY|}5<~RdX8S}Jgwp`SrBZSfP6c%EC5YeI594pd> z_mmVI>P)Ve=2*_xWIQD;P?4bLlMBnQ!Uhj-vaL7d5K9p$Htb6y60gchBU3F(A<0}6 zC0G2?b{s#kPsJfPP`W-X)lr~wVcxVp?a@(ic9-qp%HOPy{rvlM|M>6tJgFFEPBFhl5-tES(og+pfZiTHM@ z3VV-m(uLrbEyS)sGaGE-$>|5wBN7rIjt=B7aUYHVfBa)2@@@SCDYil{Y~=by>Y%GY6rrp6p8o(n+vQqZpQTO=yiCxla!?P0+EX z7bmrGbVwla4SgM)aipP?&)*&!G45!~X~l6(oYcn!b6-~QOUc(cZaRrWO|I?HdkX zImAb7Js*4zM0tWA2rsWxkbl9pU*j6S>5 zH#&B1x6A2OrEU;0^AX6R{ionP{pYQ^4r`QZ8DtOr)W@f?>+_HWD;K-v195O!bhnz! zD|UBq!qw!7|34Xfj5*|8^S`O#NP3@n7>N=IHRXsD2`}Z^5`6aGT%1k%{hwTn z3k1o5F(L(j5Y~^d9v!hktm~k%DCw{Z}?NMQ1i7Fk6 z?3Urf>3WiVwiDRi`IVzr(0A>N=T|MTD!B*K!eS}1kv@l1Zkz4wZhN!3-*axaUH0)O zBI}1VW-I+l8P0VoWiUAs$44^7!z2r2pQTm?MxPOhVbV0^+(NsDseml<;LFhiE>aN@3vvxQ zX%cM`T?#=DV$l6%$?hCrzRs|!HL@!0wuZeY?=7|kr%h4LHBs9LVvaO<_t5{;NN)l= zhEH9TKG=WP$WJ*Ict6;`F^+@$j4=T@5&#$#JN_AEv>bIQ{=9}&0cpLxQqn9V7T@Z5^YM63ZcDG#SDs1MU zB-cD_Riv_Ob7qKQsl?Fl|7}fdL)U?nT^l{gIW~>c(bZe zLq(Ov4pODQ6Imaxh-_8x#MZB-hCYos{#!*Uk(-1*!9=X}1HWB>yoAAyEf$_~iAUyp zw5C)OBBHIe9()uy33ZIe0H1}|F_SKo6R;V3orakzz&?5OO^)9yO$np5i+b4S7roup zhbqxKDoZV2{jf42SgN@Bg7q&q1}M#jhOMtIRxZM~ z_sM5fch;ajMOb7Ae8>(`=(lZiR9kt#uY{Ce)P$T>Q0$QN8Rfgf)`V=xcpo2&`tC7y zehABXbcDh!9aAe82z39-IgB79$tOR1DCYnD-v!hD=bQcI3J#U>M2-XtOj6ZV##Y7h zL3mn})ezPMYYbG%Me9j+%1iPGuTUrp4)*nJYFz$JLT)fxAC>`6JmI#*v2)1F?i0Cw zt3;PcsZdq$@D5G?hSPDS~=E2Q0q6 zfkAwa44e)HF`x;CZASXG`Zj>wit+dng3n^1UPk((WveoTlzi7wiya^Zzrj+yB+6E< z4TJ`#-^%yLPF4N6=?-4u-(xQ;D2G#NAO_NvVS3ySkVG^CcPn15HbALS z%_L5A35=W`d$^cs)XDySvR>ne`z+lN0zHOQv9?|Ft*9PVVisJ^r4+PWdh?N9ab^u# zeWqWWMx>OLR2mylK80Pat--od%Zce0=*1kE(q2p@VuMxZ5UR^yc!8@!~wk3$Z!h#;mjB|K#xb_Y;oF$v2h*<)wEg_Kgv!qfPBOZ z9&(~*=;7cvN*yZra`9#`^^v7B%N#$kRg>aAEZdsxRPT5wO*tR>gAzCH+O5h@T9=@e!AEj@Vj2qZCA$CNW^5Lg^8 z#7yT4!H9QQAsC79t7v+&iiNv9DHx)Pkl_cfa-h1o8A#mKPf|AN-d^eEOlW?$=7%ft z{Dj_hO`-gw>vX)rx3CxUC$J7U&%+rn&%FyXO1&2%svG{e#y~>S5V?g3k>eR(IzP7)@@?b>Ar*vf&~vdm9s{nYud+Yh3)+sB zHRgC@SbUu$;XU*-D{JYK;me_quZtA*ewTPtsmzVK&AiHsB-Ezotg?y$(iOaMWJT^v9Ef!d1 zg3d;_8!qb{v1RS7$hhI@)SV!PV3bw?o-?mN_0Fv*5Es2umNmRt2kqDyj{+;|Xx8v( zZV-$&?+g*i7B;Y%KM+lHR!%$D?8WH2&~^auVvcGc{<1-H==4Xr2F$0t%$p{PBO_!2 zgP{k5GUnFcz?7nS=uT!(JeIrA^UwpzeUGq*`h}xb0ZM-MBq6JsCnl<&a#=I_b=HV=Irt`+jqTK^*oR>{dhe2Oa296 zrvhmrhylGv7jQ9ZXWYuf126$JF!9COYY&6~na+nEB?bnWI;{f({lu&XD8H7V|G0D2d3#i0$xuY=%)Xe>Lm{b`R(~8#uY@G$FTDS2wTlO}}u7uIXSD4Y8mbg!x0NI|I z_k{djLt9U#hM(3K8v6_{=&ZX$_AU4Cbf;LPs=>8RiL$LI7Jp6iNn`AaNO-R zS)@dX2_G&s@tkhAxiSaUnH?SjFQ~8(a-5t-i;8(JIGSgtUANutu`c*}YCZ3^kp?3~f7c(X+e;hSZCFc=>b!**-;0XY4B+bZCd<-{A1nr)Ngxt+pk z{OP*C;5coSDjbev79Vn~NCBI6C2d-Ehpd0;h6q;qu)YFUTf5`K zTChz7Mz~ZZDyDp=nnkc-v_o=bcHF5c9<;}1$GbLiBDY>+IMk3L7pi43G47$Y(9C=5q@SeEPv_Nn*lq<$=;ETY$zp7inx|_;tFEXbz6j6VJd?$RqIb zo0frF3WE&KCcze-N64{0hc&rlgapSPy&*vVOPU%P5`yyI9EZ zDFmJRxDRoISiC+tN*4NQO>Tg-GT{bC8cbo~balvLXXsBdBGpUa448{*Ejy7Yn4y+O zWWWYtXQ+TFa^4u29>TBkTwR8h9vSnmGgONmzC+3e0*;>eh5owerxD+Cswo_77>W3c z;E1P=ZU{D|Q;tb@ZD1UJxO%lEMhl^Xdr-XUNm<~b&L?(~BQg#1kfPk84o9lg?%fu(ZIPqWs>xy9NJ&|5!L`72<$VO0h*TZ(JQ_bDZ31;^{R&7Z;`Cz| z1F2oVoep+=L`pm?N!g>KA)ZMN)g^Q~FCQtJAr{9L)@1Qr)d^p7$sB*l$fw;BVqJEL z-_`&N3xWD*Jg>AxzH@BXEE1CpPZZ9IL9WO(c^mc-wnh!0S9Q^zuM^>{H1m-zeW87t z_-8aj!hsEe`5euHVg5U}rcB-a4;uFB4GsWW$)gIQ1#ej7X(tCcD7WWPgxUyVv{4cG zLxEBL6p+1KX~rGduAg-(`Tm|jmJWp_`TF&pVkb>E+ywOf>3U)!?O@f^Z2#{CKL1y^ zpN!G&@Tfn?Sb6?36m1m+Yr@yC(997=YfGIUg>DwduSjtvuFfO(JL5LB77O>{X0^Hi zah+QY2%>CBd8L#fvI#7HLLtJ#6>M}fI{~oNXE6_17^;jB$b?zav<%IqikVgxs!f)A zGFx!@KvQ}qT22urZFJJB%Xvx7BjHAE;0a^R+Nqbs9S?WkgdMM~IhvTVcrKDgLX|*` z;cB)K8TaR%noc9&D|cfk^e~gj`HjDT)%Nm<(wwTZm*9kSFh49!J_(#G8GX+PRwd10 zH?n&U^KK=Qj$1iK+gU_j@vbD`skR+C%7W9G1yQ$AGq2JP(LzMY2*!T*HldLR(nYj) zSloEkHXEK@Zk|N#-882lbB-D?lzZkmD)ClvQNUG|44OIH(ho+=V9?E9KLD#tTioT8 zL=I%0QyrBG=)za4M5OiK3mGDHEIa|h@1r&Gvh=D!p{kYkl>Xl&c`w{GElh)A_x~8K z$7fcXW!f@MJ5>-LhtKpPDYC}{#j;MFLL@yc*hKCrqQY*G(`P#%OXVMM1wN$6f(Hv} zO9{_NI^^j=-&pct$K9lUd_OFVo5qp{2Ka;q)0bp_%F=~z3GlL}yj7B={<=gyLb%U8 z*wVDrUYK(WSuHZlW*hmu637>F$?qH>t;ydGbsdVN;*q3h*F<cEL%x}ATCL1^Z4Z9%Ty8)5`m0hBtuCE zA-_O5?WmM3Fm~L1x$}+$fjPbzji%0(=iG(zJ3qCGLN;NFbd{xGB^ywVLHKcwv!XP{vq zV)!A#C$bO83@@hvqW0doX|kQ@oK z`HV!As62cQGpgQ;)|KypMj2yx%YGkVQcBDy`NY|?*vPGU_8gmMXNMR@b!#nVpBV|t zHA0ENOK2$h5+*AZC<`CAj;)8UxK2=UO*!+F$8!yzZVgovq=H^O z)KdJqd}%^sP69j3p4n12HM}?>jgfj!442vF4~8@s`Wd?}_YTQeQ`^2w<`WX9RV{Oj zm!jd)^p4uht!s^xtKDiH{g2-IMD=ooKZV&$9fwEn8l5WCz=iTD>4zUn#~cd_pq2%T zl9Fc}&oM?_b(##y1#RokmdDCQRjU`f)ON7;gj1~NL?7eHvjBVPj~7$Duqb-?MT1hZ zYpVL6Sv2R}FRNeu6==jMOIgrewjg zW(voZweH@UK=vx6J1h&SXj_UAoQ#%im=n6Z3xXYz0^H6>^!p}Jc8*Om8na@B;Uw*a z3=)p!PDZ94DXgQ$G$K@qsWKsG{#C+J?1OktNh`V(F>h*ZaoRx~r^v)nmtzN+7>2rED~XZ=aH-@c+FFrpQknHM7hxY(w#mYVhhmHmgPcd(twu7h~L z-qznQ4e1P!Gp0R1nozuhg;AGVx9-yucENVC2gU1EChBL0T31y}+Xk^DgszR5l0B(x zGAHc>_1z<$lK1%j$UDoDjWwDrQ3hSlUtlYDk{(VL@>uogwA zzplPQB(N z6;-JHO0+mLri=2Xs=vLcg$zFRHIxnM87)NEB(sHi@o)Wu`BMPcpt z0ReHE(WnICWu(1`1)+j=15xc^XLmOoYQ8P@;_U~#*7^bzoxay@Z#eg3c^!j)%~QqI z34?!_rtqrFf=vxUH-xR{FM{0LpkEI6On91h_vq)QB__p)?lc=srjKXcan#*Tf9Bry ztJenYTVW`m)F6y)t#z^?IIoh602O_j*yH>21~Q}zt2kPas5*yPq^F~eoXBoT z6wUc{@q2USF$4IwN1mFZiY&x=&&gf#PzT#~7}_AGqKF&k{IH_)W2oN} z)$G<^=L{W=Co^ZICA3edBkG~#jR-KGopGWc+9QZz7tzeNxX{dI_w^ja(U!sTfZaUN zR5Ar(t#bW6DJ|%%!RE#rJa`n{Q)7@H=9yU_{uAnNom*YB0+_wUG2_B;k46 z3b|4%>>(YaVY<_EW1p+E(Z@UexXtEiU^yen?Ex*cxRa!FNT@v7$jxEH{3HeKy+%Bx z_81#I!W@G;UN~k9pezfAgC#122lqzDuw=STfX*&3P_jLkvVEd=M;lpNS5fAPcM+im+Z$$Q|=`7peOi2w17-TfUVOGDvG zrawI#U$wHrt$I5sJV&)%=_&$7x?c?QHv`o;y2U_I8`-!s5WWg)dMcQnwejz`i3J&J zse^ETpxCPV3<;~#j+MngVOzPNcsuqFl$lhn^X+05<{7V=9YQ58ysge?7N4oQKuX-v zN4ddpuaV8@=t^Vby>sQ&nX82FU41sVoU2oDDPo&`{tJ(b`Th}Cb+_17vrIOVI(DP; z1lW%~RI=0@z^<+x4cqC$6_*I7!TYspsIihX7G3qj>(_jXq^eabgXN4BQ`TDChGTA& zt0{+@=*yDc#7UtS9gy3r+wBhYg*t3dtjB|hoAN@vGVQW$|E=g!#iHNAuMB|VRk@EG zL*J$(p~j^7tGVz8>ChPC>XEGe{*J%)rNqUrVrJ(-VC2~{2p8O@K~IlD!Hvk&cG1|Yx4VH+!d6=!;>@Oo`%48~aO#=i zqPMFYd;%_3WC_U>p?K-l>kuUjQVRCJk1fq5KWOv&TPKV`cz!l(E> zg^c*2h8|(iZLe96`hi43y7<*eLIHvOJ)tpLKuQ>-#O*6{3q$yFGaH>X>|1+~+@G z2T9_dF~5{=TkOuH=j`aFfA@y$B0K|2q(L-iUr@i;YXuAsOhtTuzQSV9{{o>NWM#IE z2)w=6PH=@c6+i5fx+N+mzO~e#VCW2~zT++Zbbh5}OPcg_iFTPe5x(UWYfYki0g8i(K|;u?~d7ABKKEQ}K>@ zB_l=*uK`qFh9EL5U z=joG{QB$vJ^7r7=6;buUL{8EtD`czG+j`fD$F0+)PuameVzBAL2j&MoX;~kb3b>yc z!?$OcC_S*gye)VX%Qx1{o9xWD2>(dS8)*FJZYf zRP@aB!@a{YUKg?i%?ry}&1%}(w2NBSRZev_)+Rp-wf-ntJuH;#p6UI0nB~vC&tzv# z8xxYq?`q^9dsy>4U%F^JSaY9zKL{)NQrW}*DO&GIg>7lO^Z+ME_Yw!WqI=1M9?)Nb zpSudtUnM{>=sYExmJRsI{4j~)K}6psbitcNC8rlFpj zeS{FR-@tj{xB)JVksh0d>DVE>mzSk5EgrQWr)I5Tf<{=kKas8U0(s&3PK{`RO^atim|Ou{L463p-f)V+SONX9mxc&8;|85l z`xPQNm^XPIbzj&$?U94qH6pc1pazLH{KAI~EyyQokS-y}LQofi!qyuRe z5s#&PI~-*8Fo+T)S9oEwt;*YQ>lAT4xG3_Qd|?f48GQ+E6@AJ)&rCIZrJK*h_+oC)~SIVozSIjsmkDRtC zkErXUSL*G8+rQitwl69#@ZE9*+E&gmG!9)Vf{PGK4Jv#@nrOyD(+K_CZCm)b*wU1a zTo7vz21CLMAH7OB%79y3y#4hoY^WoT{tGd63(=joT$tA}owNKT=Loo=nR{h4z zd#@yjq;4c?SMZ$0Lf!9J(mb0K*XDK@wasPgrX*MxO9#ht^U|&kOHIE6J(&ae#_K6C z$-;E)eHaam&p^CKW-XS;kdP9R8%8^P6WX4h!!+ElS?#Rzw-in6^Y4aKCsx;()YJ{-(G+lt)=ZDq(A}jM zrTAy*b#kw5q3c)pU;hBLGVkZN7wL8rsj2UJWD84AOe-qw3nrh7*ylLe4;j~%Md*tc zd}f)b6r-OD3D^WDlGL zX@9<1CrJH;Wktzb!8B+-WYU8fBQ!iXT#D~iPXSt+_-g$vSwr`!}()wQQH`yO)!bYFj*F+j|x+RlLU0OhvkP)yW?qK?1)##r(Q zv@Dy?4_M5pWtpL~aE{lvrm5iI9VK<=8^5keGj@|;^=u?gL=B2B#$6Ijb2%{`)R4hl z=>4<_bCG{$7tpRUJKrqS&A~A>G04favN~E2lpI2Lt}rMy(phpEA5&b}nWkuhyH0$Y zm~N@qSHQpc2~@q>fO25_s5Hi2v5C?g(7;QvQ4zaI9NKChb2D1r znn<-!{5ow-Hz_t)uPP?a^9iZ%jjewLvnug?tCwaW$4q}~vazI>zLoXcn{7XYwr*Fb zNw3$3VbPb!e!R1b2F}ZT)S#Dmd7Rz8R)5;_iIe~F1@h|x(=Eya3S}7^<4Q+7FjI?` z=XDx#r^kqe-VK~npKU6K!&e3)^x<0;JqddADZkefGd!Ni7L`C2V3&&&*(`_~uR^YbgO7 z$d5G@7~-rEdDn_X+xQIbMNrU$2vhY9G=Nr(ehAGIT{O69$rnKF0XEK}J+qqFSI&G1 z`NQIpE1RUw)T>7#iMBIr&U!${8U)rjJz~_#ZuFBplT92I&{~}6_j^FVgECA23M1c&`+M1LdNyBHIg2$#ndU0UascqA1^O9Oj?8SbqMZvxs?PZ-gqooTb3n z8yX%K4Ll`^)H_NiXa%P=6O98Z{R3O6yeN;L!2?M@I_kmNu4Qi=Pq*Kin-(0x>jDLn zKxw>MVfAzPVMmP0^;hmA$@zm9fxQf-WyzOL`o9Gdt*Jbfa&IfT?GD=M<%=$!KGfH>2y8*j~ z+Ge#hm_-RkhLl@J@vF|Ht=Oe_?~xB?t=LR>oZq6HSYW+1gJJ42ANA?R_0@;)fdubj z`H}K0ekq|7^KSgCep8whDwZiKobl$H&+^+dBy`*XQL202(+-u)F%R>1t#>@M#R4kM zq)sNJVB?F0|HfkQK8?Vtn$TqREvH57gJ}#73h97{KpBLY3kEFu-Wd`neUPmZuo-nN z`Um8r7n}E<<>NktBdn4)&xm=$0>zm@bi;+6{sEqi1Q#pNAO10Ddv$Cz;KetKqVA`` zZ@iqnT^Y|16?)jUe?&?7Tj}i@N}(c4N>+J~EfyuZ&~LQ?66|6A3k3Iud5a-G?0rjf zYF{3N|K+cu$jDvKf3`sM|I5Vre~vR#us9$-Sy9>GfE4Ova9H3cvjXiyH+QMFvZxrp zSwVYw01VogP}YL6NXl=43LmB^NsFH?Bg4*qN!FEp`V}zon12!7$z>#LW#)AYauql8 znzFwIL3&>z+VQwbg2Cs-PLv{YKJ1=8J!Wn2K1{sbU(|F$F!(nyOoUunV!4C=qR>#Y z)Bc!Mv{e~i!cb|*521o1Ak;rQRV3jo*c%#b)k@Kyh&HPhC!o7BSoLX(w>=-!^BpKGu1^&AlY@nUb8UQ1X`e$&ZNVnlc?X1zvR05yKs)gBfmJy%eo5Z?@r`qCtjNOu$KC}GuO>!Ii6Jw zu5wSaY)shMha4+WO8fznu9SZb=VRBKOSDs=%u|`IiScXG-2HF4Z_&Nxt>9_VytWqg zri6VSh-TZn&uCi4uXjdI>m~8u$&=EbGe8^Zlm0;JQfuF%n-4_gir)L_bA^b`+FPm- z>o>zkR{)$YO7kWC?9#8y{y=L#x;9GkQNVArpz(uGWbJ&NfeAx1SUX)c*!ZNxPf!;| zB9>+bzn7)$KvY~@460Qz7JD?;q@w7s`PhgWMRUk?1 zXQZ@CJZWuKIU;3=IDMnNTox>QN@zXIt^*9I z4_gq+CS32KV2W(y3T0`p3KkIuOz=2;ajm^9uvLOW2ZhNqfV&Sc%ZU;_P|k5Swj19OAy(gbraM@NN@bj|*EtzEy9* zcz4KY@w+$9F5^aQ2$QfmAwn9RaNL5YVd;?p^3hMZJv=+YFv|rKZWFBv?o>{tMS$l! zyX+jFPHjTE$(CNYgVcvhG7b7_1{=aA{JAl25=UxwP3h;#R78qNA$Xf`z2u=H%ap?) z7&#TYM@U;<#p*=mF}(ip?}A;tqeK2m5E{OpnT5vXN}Ak?JpdQ=j$A`@1Qk z;MW&u@uf*Dv#w2y7j%dhc>LN8Xt8T6w>lX2@Y1B%$vMY|1xHs~27-hKwyjOX?M?Yi zQ(&BPbft~k(Ld{&EV#5Eh(E~^W}i0k{{_o9rGx7MleCp*1<(dRv16?&qaEm%b7R z4^e^xN->i38==oiNHG>|gU?W5Fyw8c4-~o;8zRfA&=`pHGm)uQZkC0sMwTxQw1(pr zsh6t-5_q)O!Q+;kct>%MqNnNx^1L{~ZF7_`*+d zfQpjkz_X-{odE#i)laM?KCggDt=v2b$o;v4v}yT7qWBov!_jL19>jor^#Zv#|)Fmvbn}0@aaoZgisT{G$Yoj9ktDmiNz7~kb>cKMRTcO=VBpbfz=t9P&c-F9&oM@BSc)gk z+|mPAvd+@z3n=U*0+1%zwuy*SJXTJfNNB+^|I2cm+_5|X|AZt`{}&|j&k6yB#!o~7 zJNeT|h0e&@z|paT8`ceNujznB($s~yJ=sjaSTN8SRYNZyLR1(EN*0XlYg8%2Qr3SQ zRaVpBMe3UG1huJEwO9OW)U?gL!IFYf%jcSZHqCd=@mV*iYByCac>cX2{u6E%k}QW;8!i1x`L7Z44Iv`h5`5aFR*oe=SnZR3Nl zl5hHa?V{TJ``t^R8xov{=BH9OF&G|tt$z0)I1bV!HW(k_W&#)==_cdXF3QbWFdyk| z;4j!m+^;dPZbF|&e6Q8MlQ55&z8k2QFJOH1o3LQ-0^Rs9kHlYrG@J6@U(43fsvTKNm$8OY56v18$wUE%~_V!eifM&W!0HV@!@K%1^WJPio_ zyKeB>jc?Aili#SfkOI0qf1LGk_LD{RAA|lY>~PZ1*UA9ptx+B zLbq(fK{SxS?TA6QFf9%=uDJyR19`^?qDM(p>J)pvPbAi2*2LOzvq`po!(`H+*_!8& z>anuR4*YAF6^Jj^68JYyyeDQ;0-8S@mGl)WS=xyU7;st z#|@f43>{^C3^hkcvS%%_9s5Ay(ZbZ3I$*`FKCoryJ2Zd94oQ1}rZ++fQ0mb=d#$&@ zGi@|$wF9<9(v6*d9}nVz>5)iH@{u%?Zp*t>rRSyX&ICf=;1bC;{?efN^NAzW2I=*T zl6nC?4UYJJ zAV<(VuC7!0rYhsTU$O@|Uk`P_`}H6p$?xO?=;DGJb$;w#+e|C!Aadg~CF& zBd9wJSaBkrg$!6J28E@D$SXQENDBMXC}IOzWRIq?iiD?GNY5)G2R>)tqeN8Xs&Gov zgQ7#YLDliU0BI57NDgLtL;O4^z_o6t6a=;fC10B~f_T_BAxe8-*lfetUqFnrazRgv zcLd6v#TB1kHpWem;YvYVDo#FUW&lHVeYt(bDm5gLLnE~Uew?Z^WzS>fv>L5kAIoV| zFsc>@iUy$-47^++NTZ#a;GvfH+tzw#_whPkdTJUsa%4Yxc2$fht4WBM1gm@O?Kt#Y zf0JDa&fLO4ukIF*B99&qtwZ!EtjvHGh*N|AP|n;}Xk19E9G zQ9wd5;K~kB5}M_Rm|E$ZNV}cgw%rhkC1Z<=b7<(qxG-ADU0>4|%?jq#q|nhH$c+@1 zG&2>Ahsq+{u6E1l`z|e(^~sx6St*$M2iOcxHD?9}GPQ+327iV7-=TIsbO!Pyf-j}< z+7{vE;6Lsl+c0_mJZB~~NaQ=TTim^W)pBx48W($a~m!o=Eo=Ri)^h(}jQ@zlxz z8NG^+3Zx+;2#9UbWD$nbGw=P)>2`XGEDf=1JpV0^2|WYocU?i3YIvz~qgr$l8kudLMcvJSqMdoa!?U5u7WN2Dw&6m|54KC=MBXw}DqW^IN@bWGPD0zJV zv7@Q-wDg7cUes^6u(rT(xzUv{@Zbl$3rk{Z9QVu>>Y(QNX7adYeHlj6gxS6dExB)* zYiI{EfX+K_7S*6g@9L21sI@g)e%Bd*3muL89$0_~oQ~?N=A2&3>s#54hRTa{Y?T(e zTN5eeSG(g)Ll(7t8-;!?bj6o?pHR)R&+2{omJVu%VwmXr2o+RJ3pT=%v)6_ z!c$ra*U+?vEo;{1&t_)Cf}vXKcY6>QASkM8Mw`l?W4eo1ZD8T(uWb$)+TxBp^gKB$0gO_mM{Kr;@~~R0Onu#kZz>qNQ6T8uzBjc34?5CA#o%DbRvYJ zWXFhdn#!}AW#O)aBK_@ISri{Vq6*C^1TQ(fD5?d;X+4C(oyx zB}ALxf=XyJS)^ELf#cgxbR@t6QV+F@G?%Ld#yv+S`C81j{i3#kt1uq)t-BAVIcbQd zR=yuEg|*1B2=7n2YDh_RQ6_FuEOYTtj9kPzVUqdjmTlZTX5HT`la{4vtY3vqR7qRO zup;-_&L2dY*Ly5L*wK$qkMerk13Y93LCvYX>1oYXHR6noX7CO43&?W71CjI&hczEc zKlI4h_#|HTjDyU-iBTL(m2{IlPOApaLFhKC=LKkJ2*+m_MF}Wl7=elg&K>1g)woep zK5jk75S%CoM-v=upXi+oRnGV(E%(ISa-oITYAET>RZykZ=Yb^2>&@PM)$xKl52 zc_}e#xt(!51~`3;FmPvJm2|$xMwb0HZQjkq7|%f6$q5@xDi?1{NE2;mE^-kctH~@t zWRWt2IE(|RWP*Pa^v9D(iaifmRU998!wQ=GX0xet0L4&7wb$v(^`No?Y)+=*P{ShD zUpT-mO%jUvvttD%N40sF+g$wd_?w+nG~1)NGS_4{HFyYFt*Rdo>L!A>^JZvBpM_`+ z;UhlS%fRgcE1=@YPFa9nQFLQbIg7tCrVUCJJY^tgOKYj?cPJRHft=q(Kh%27&?`qUI^Gb65GvE5MT&(svdOh{-yFU#-KVMAtX}=8W7Sj~+WrH3CMDft zp^sAvVH_pPG#2(dv6Mq0UlEj+u?ByufzZUj`#lwJ8#VZOlMy!{ov|shuRJrT*d3XA z9`aB%x4bN!7P?fTj(-K;lw6|WH%2=@ID|l7OY;VL0-I3oT}?q6~;4I!9^iW zEOZiCxro~&b~y`x5;SIFYHtsuP_XUzXF`ze;AQ-ji&W z3K4N0BkfrJdqp*@6B;;r(foG`bwI>`x>@+F|NY_NCE+(kAV2&RRAt1fqBFV-lr|WJ zGfTUHoH2diHzl^y@>{0U6P+^vejTw?di+5oO<5>JO@0x;84Q=&0&^DOi&}G#+E(n| z-m^suptSj7q<}{gYdO39IjQ;})m51bCy9g*N~{eXBdKn#hWV(+A>#WX@S$x~Pm+OD9R8bn*QTo`i3O zpik%gpf~`(zQnEfcW-`aGAAs{^oq)gGWB$Yv~zlwx@PYBhz*$+?l!851HDW#Yw+4R zy~{ec+|6@vedw9llRTO!|8h&x#%^|)^UHAoscxx;;lZ9f&$7^by?+!+!U}JZXUpHT z<$wAm-^@LgqczxrOk1}tX5|;PZz$(dID5f|sDJ}iDGf=B0-$*LRYjgP-8?W`5yn%3 z)I(ryDabPCY)R%|U9kJR9GUP=qI6vdmp6#NwjFcL&$zbrr@o-4CvfP4or0aCZ#fKq zAin(xarG_$y}e33Ehuhbd1OQ^*>4CTJdYp1y;gT5FA89yzrRQKdfSa*PZ4qz;t_7FRR1zDLwvw4-Ee6` z0}+omN+;iujHyUBL(9ygjrL(HZglP3RM5wI9Q%7I4x!-`dhdUpQT@JrRuq6gZ#QEj zjQ{p+q+c3e>Y;7d9BQEryZn=!>vAzA(w7xTRTFi+MM_mTvy(ryLs)oIFbjx^TQX%^ z4wW~vn2J>j3;jg)kEC9h$QS*9Cs+J^%1n6toBv^e_U_y=SL&bFO|E(%`{Yp_rC0YU z^`;{nw7%#`0QZV`fkm+#uP>^gKf0h<#g8S3gOc{?Ak!>Gm%za2nRS}7g)N28R`^NPzNhfO6$$&Eiq1x#U2>&a-j%+{)&&Tjmx%!>rq4 zIngb?P2ic1kZaxe=CzR3Nxpx182J8q^BnI_q2IO5LSBY1+AEf)vb{!2PXtaLj5r0mc})V@t}Nl%s#%a z@&b8p{uDu7%?w2r$WAVt4YZSK_ho{ncq46fg4}4^Ad|T;1BL7Ejq$-vyt(HecFn~#OzUB>R?QR$fUbL&V z^oZKOv~@8$7BxOxH9p2wfYF*Y&5Dk?%ThNoILvxdg_~0K-D2y|G_y_bP%&bZTY0c^U#pxumQ@D$Ij$h%V24b=uy0<}z$xl1@++$og4@M^a zGh8UGj^P2z6>#ra4wQUPKq1&lKV?@(3!>{P8}`#VFMKCEoN>jy#`)Fy_{)uRGS9B) z-_b!^!_ikn1L?IOI9cuEz&}P^k+A&Jha_A9CWrF~rdSX?isNhqHG{N<-x>&nA5OBA zpHoLDzD1wlsXkOX6OVW>(*><5Cwc8evfcVLzEZdzYc2nrxq?kl`Seg-rb1A)K)_c( zqZ=6Yb%fy;^k7xf5884cKmSW!UKZ9Ne~$tJBEbEB=C1$6%YKcLh%y=)Az*+F#|2?z z0X5XE^ zEpkCv7S8Tc1PGMdou%p9+%+DylGl&@do2*N_Hq;LuLX+=g8o2exC??7yZ#b$kqK)M zxGFEU;@Qb>7Xp@tGwxtDUUwiDI-!T0{Q*yU-`+cyEpf;Oj3<|cIqVzQ0=JVfWOFS|#O58npN_MR(SE)f-GuG$fZbm~>UF)tMA5%%Ut33|nn&PS+!*g; zi#F)%v^`?vGPcv=M8=+xVl8GR#$G`Alj#KxSPsgu#9YlU8~W9Abde zXa<$|a~FnA5l&9Bk&Oey9=Y2b#;mY zef^mZm852OjRb0$@UcEu>pYa93J!?#Nf?iQiN}6$zsn5qoTz-XeOq4`ec&E2*CUg< zgn4SCD%uncxYXi$=r}CxW{|mEV&>B|g3*;`*LZjgZi3U=$Ak(=8O0F|5gM4b3~bejxxqJ*inf>(MB&!6>*wk;XXNPY-UP+#>*ml zn$D@K)_J4d5&#U%CXT!(`{e|Qb=nbbPBlq%Pt0mM>zs?Nf^QoLe1$cJt>@3Y+3AT4 zjr2iL5~q*!rrpbmlt7UqP2V8ycS>Q&iS?Tc{>G%Q;krRN!1-%mf=;r_*=!MjLkjP* z9w@}(LQbSsn5AUN8KmHs9HlgYacid|fzj!QwWO$7Upe-s-i5Fv?-CC!qToKZI)MpW zc}goLJw|7sbJ7W~vOp_^58GxP6zx6b^nCJJm(AreqP@6sB~!67g!2gUaUydY*M+S; zmDJR{ll4uQux%tib_=Vi%K8cP-FH;#cBmwC{i?)@WmjUCebw?(o5A=+f<@0kRgs*t z57uZhhRC_})q`uICEV(ZZ zrD)7EFx6ai1XG=R;8SPOp3X=y6Pf{Vm4S+y>`CHp5_c%HsZb(1}fUZ(R*2n#+S?(-A=Uwr>wMEd)> z#riB$2ey%w2qM#5zHJ_BH7M9x1XTN(@_)t2Q67p1K!sVA^mayMvjB zS#K?Agms)N3kkfmcyO!-#OU~VgCf*#VgJT2EA6b;6;4p4?#86gU7EQ^X3UAz#O4&t z&BWt>CyP{mQ3)k5gkTXJ61g!G#vW*3j#n`OMInj&fh;Nuow7t$(dF0sRRvJc8CoFs zcVApk^?%XD@L=HRAkbg8?SEVz{--R6GXBsbLjh$P4qp%)a$qZifVEOkF*R)uCQZQZ z_vkKk^+Hsc|CGs2MOj-a#fX8M7sIYmC5o;69@34{i&AW-)VH8u2LJZ8?yUezORM~~ z*T>@#B@nSpE#`7#(Z15J^%)Js0<0m8ckPY>9%%bEBt$})H!7%z7qlRMVlfxE&{g;> z5X`5_pSq_4EwTw$gdsqy>P;!39xX@%St|X*I|v1X=~jJ+JHI+pqBoj{T*Ev`^A>Xe zoUAcOVuI1SS^8d2e*aK_!D&r?qwyiRk3EmaCkM-5V%gF@!)d-Nhva4iQV_M&Y>Je^ z_OB}XN=q)6-Yi?pMzcIeNH&DSQp+zL;N}J@ZdXe?i*J)PEMEzsBg;a<@@#VQnt&%) z;LjvGZW;YXDaC+*`yLU0pK-v5gJ!e6)`V_nxrs`^Qkmn}tz(B5qB6TIgd;EE@UG}( zerrE|wfV75x3vMh*O%9reeHp(*o1`^;^(880eG3+de2T(KEdVapHq1sFT>tJz;WtS z=j_H`AQhSUg+92`YvZimOIJ}jd1gNR382g&mQAR^>dMU zN(rn${dd^FoC{CH4JbMYh~od*ll^avw6S+WbbR}7^!|Sfhq?Q!G$A|AT#cI;*FRR?|LjqdNj67Wo8Rg#5VcKeKUv@%B$VzCO?~y(}a1nq!#s2w52q{p6x-6@Suke?r6S(KWNr13jOi7lf@ zHDkod6pDKMz>oT0E;_zN7?IgTvdMzYu%r+BzcC%|3<7A3pw6$EQ$H*MPC7kMwAJ`i zJxx|hrvG{Ma62#2qkmZbsKw${6kqbNvg41$_N|jvl4LI!3&^gajR^N#_W~CK41G;G z>o6r6Mq?ORG$UwmV#6#;)crIA6!aEJPyO7T zk@C|5fW>Klwp>R*-Wpi)kb+dVZf$b08pRJZ#9<^b^f!C9`>|#4PQ*6Oygb)D>+MlW z)FDqMMwBV|T&#o;OPeei(?06tp;zz5h3Q7o5%BjUz0jt}NcEr8LY#YOAlpl5rqUR% zT#sq&ygfcvNW!hJ*|WO71M;mGeh&gL`+a?REo z-d8GIZ!(|I!Y>5D*-|uZNUw@G5?S%jT!)kt zYzA3rL2he-es1QKU8nr=-u5mHh@o2dYPM}%E93j`Rd`Y^QGL$WD!l*yxNKzH^!|qg zXF%o!(h@rrLBBBY&maR#%!Bg1?qtx6lXW}#E)G;UCy1?|^2YF!j^QzhFVdWpxTtVM zzav=lb+}N`@Nz!&Kklcpaua6Syg=SorOE6}min{99^v6vIZ>rIQY~DmN$blJ#ULi& zjir}8uSgwO=~r-{|E#+CM0`+q=2K`k%(+E8>MFBk>IaDr1%&86g9H00I_Jm$(Bwp2@E*jwndL zcnFW7sY#%A69+^R^+?HyiLv>q#>it8Ex$Y~BRlqdI}KyFL=3Kf#K1zMk-Adjc3r!5 zuPi<4I<45H0ZV*GiqgTIpGNO50slHWtE&HVy?OMf^a`}2fk^6wP9mkfdS@vmzKUll zCBK@mdX4-41SXL(UeU9Z(q2(m3n=eTSqn(+vseo#?$=lg$nME31htM1S%qYe?#%^_ zjxbmO(IchtcbY6AhI=dXkDAGXl-GM!K;(#N{GB4Jm;PSM{G)2pDCPB>6%acDQeJad z0l6cmiFbmmUb_2d^N*g%PsV#B^ADsffciec{6h)I0_g9R%s(n8g;HLRSwAx-dlO$X zSU)9?w#P+BNewUq%#+CAI_dw87Z=dGTCj*?O8nHOh=H?kX{5hO91{5XQcce}x@V~4 zPaK0}ahpW{K^9|V0XK@t7{xC0E`ypO{q-K!r;M7Qa0w&-ppR-vZdXOslG(GB(I<{ACRW$e=j!ID==qiU0(<%$(hH^}E0 zqo61n4N#F}Oe4rWWlb~4J>~XHWcW#=1d_egQ4?f}#ZdhyZrjQ869-e0y=76A*uG0=?D-(yw_ zSdAGvgLi0yt6?08+w*oD!Sf}Vr(vmnF2FcawlBd_EmX&BMg5McQC6t5eR2Y4M8TJL zhVFw7p40pSHef#F8`5`V!Ha7xoY1;LeoN{0O-(n55;WFRMf-ZDV`w(m)i}cv*@gjX zI&6Jdnm4L8i4K&jGB?b9Ldf1i!Tg|Ug+hJCbfK$j(7Adh7&GM9(#(;@kr;|r8_-Xv zUD7wOeR$y4DE`5}o1lNzIzw7%a;FH$-eh8Y{BG6TrAIxX_xy!1ZD|LcPx*oqghb3e z3i~BU)ZN8z4KYUQlhfqRxFlUwxWNVb&cUzI-mSy#+Z*~{FGPa{zyYcksy-*p7c{@z z9R@mm**K%&>6ZS()Q;ePeXv0JqOPYvA8#;b@FO@E%6pnISYJ2zGs>n?L(eW}x_G|;=Fgsw?N3%VB zhd23w7x(^rH4A(O<=r+6^Qj19xycPWpZp~gBw~-!lb}zK?mbMvEG%b@0lXGmi1>yD z+{|Ie7)8|F*9e4+?*Y~d_qVAX}yHW_kQ!SQ-w5;g7`0eB9w zH?N4GNDzXenG&+UMF>dSY9SD1Am@zNR{_mG0!gF96 zJ}Yv=-!}p7gxV!{L)a$;UV!!<`a1%d+ur%x+cG59-7aT)X18&O+^P}u6{$BwPzXX@ z#UigKQeQ8059P}+B$sn1f}?o`vcBvM9H;bL^ zy$(Z2?j30i)u$M20Q<@HDS;yV`vpKJx$T7csEULSEGMmb7nE2Hl|X?tl5ytkBf>}; z12n}kKQ&O+-c=+HZX+;~X0*_nX^OvZmq24sVElVELTCAo4PUw<0v__p6YYExgW-bg zDFS~O7mYn!uQllWkpqw-9*xlMA=~l8R~Enex@N6A9p0(swdZ;no$*JiyJw8 zN0rDG@v%aE4{D>`qnE8E;m+Afc;}bM6{~ck@gVF~MLAwLhj_=9&>sX;2aT)~osQC? z+ryYBtW^gc-gPB56&*HZo>s=MH3kjdmB77d_8gP10frtu6sLO!@C52T0^C}!${hjO z?wQZR$lm*w=ohM9X%wRS$LN>T?fS?W=Z?sk*SzQ#EI=5=Ncl|e!%D(`=$U%YE-;+5 zheW66&=2F5K*$JXJ7wqdh(zN3H<8+4LTN(*bl}m~u|VFGcjoJplz0~{>rHxiL#B`Y z90kD!8sH&qaB$Shd9fp7WB0K~ad3*hj~K+@EXPV}5uYG(m+#1VN~+E7S9`HTbj9-m zAa3noBon*T3?A*EBuaVJl%OSxx{;9^(0lcS#Kf8Pphov zCuzTV&0ZTsDQI%p;lt(le`~9@469y0AI1#>GEv-T2H~GU3)*~22Tp|v^V6o{c@tLR z`q^J?5t7VDf_)@^D!EoJdEdV@SFKN^qNEZz7c3vZwA)n_N-~p~i8nJl4N{M|*RS-# z9>THvY>8adY{;D97VVq&_a`0V5{4mQ#peRic4gI*`%F0@r1F-tGn026{*M-LRZ1?v zN@(&=tdjMB#}qT{m(8M|W|&hW^qpBt=y%=|mcK*?IhYum_zw`63~9oc@&y6#2BbWlDm;))-(yj>fW-jP1bD$VXc+lcRXP{M}m7nKw1!Uz;s@z-7j+O?wGN~JovOq(}}Ln*V$OTYQ1&^YSsQBUoM#U_y_AXs&Nhr{F-E8 zJ0-is@RF?`)36+kIJ)d#FX_mI=x%))Vn60^8( zftptKU*@X7oF>2hg}W&->6xLCO+%i!$P;vBo)YU=lpa^Q;49NHYLK8N)Ij6{mLyX} z^Mj@^Dn~R!1ymufaba@Sp14H{kS1lXE-8U2nkZ?+Tj0UbEOnvPs}iEqC#expDC#t* z+I5Be6-JgIcPKZa5}05xm9$k6O-o|>)aXdo*1%tuK!iM|bXWp~PM9XoQBql_c-{sf zmytZamyxCJ)EI|4MG`i4t~_2=X)c6;CXT!8^O#9N!KG1q$j+5~-$`Qy;LA0Y9KsNA z+^eyy8V!v`rny&}ijys_;v3R>%W~zu+RAl*O-D2}8m_VO zKfcE6`N@&TuB5VTorgr)M|YnF2DqbpKfZIj^SxhN zkOK>!RHq$$MIT78(!m;&(e|AAZ}o7Q+_s+%IneHHQZk(%VBCz>ryeDrU}XGta=WGp zth^2-!`IdKzP1lwkh0=REuxB@Z01o1Ei(~iDGA;Dbt!1Ebe25&#!n{TwPw}O))qXE zC+lf;%G~lY!%xvvmGDGQ>&|KBJ}udTlNmk*C}Ltrl3qLCtn&BthOR~2zKaBS>9gm7 zuucFK5tU|X0GPnt>0`N)*F)5}9_CG``F9Xv*JkvOAigswZV=j~bU|ecKzH>PB@0H(DAC*|AnsG$`^uY>v=Ht>r_gV5yGaS1Hi^4(8PNWE&Lax;@3y~A)^;|4Yz)zEtFLUDt2^- z#Xg>uLMJ<9E+lRpBD5Q{OvR5*u{15HR~P8O;c;Ajhj8CITSI{;f@F7+5iiL@R08AO zD6GJx5e<~;<};5Mb04=wNNz+POC?XH^kO~%tnI6wz7Fr4$aRbcS=&A?zsMzG9^AxTS^8&}(a&U6UCX%1tR(+ZfL~T_9~;d9Q7YTnUIm ziAt{Uln;!YCd@q?`MB1qAE1$G}*0E%rnV=FOHT1)s;Dg6~789o-~Kvfv>DQVE1{i0c|L={X_T8c=Z{ zf8DSQKptQ5Y@=F}`Y|9mbL|=30}B#n?j ziUY&o`BTaa&GS0Ps*&9q55Fyv$wWH@Hz~fVIBmhB&VVA}dsD6|lC1#UlnJ>!F^MG4 zg{|sFmB~@pdc-Ws5H!%Z5=A?Sxs~oaU-#n|+XbVR*Vz0J%&i|T%SFdPc`i@BFf)3J zfdAy`?b4D5+K4Hex?#^x9idg>VEl1r&ly37jKY@Fm{5_`td8%!CN6Ae<=?0z1Rlxo zTsl_lT7!yz_$uC^(5J8cnAGA^<0_^1jY-pq8vDJD^TI2Zg8+ZWl6Qu!k|Ep+>Ui`} zl)HLfi+x0H34t)Q78jfeFlnqR38d7qvK2lCTB;F$H_Pyy`Cg5uSpIkuJZ_^$agy_&V=Ie^^2mt&PkQxupxd}Mkgz)^^go|l~+4=X>B4NwmS27 zS#9o%znhwMYw1gEcDD{=Jna82bWnfm=aI)43(3|Y0RMZk#HH9STW+2k2FHwYG5S}- z4@utF^e#asQ_c=c;Bc{(r*MJwER#VOH?rrn$pBz`orTxE5T(53s(r9M3NYwKDnERT zDkUpUxl8w!cjKn~?wi8WxOnKms;hqn>;3Q}DqP^hmMKK+!5Zn`MC{4YAUZ3zzyG7j zK25Z=}pY!pD+IF~XT}{kCxXFdbZdt8kW@Hh+xBOf%e}_L>w>+mYTJy&g z@nRyCbrm`f?kz(0w{o+OE>|2lGFK7^jVc(Kh?K_u2L7Q5EE{s5S0 z9z$7B{Hfo)fbX}4kFD;%0x0Q;uZL}>6RGb(}bAm_1~Q9O3_m7 zaXy3Z-vbFl~R-Zw9#FpnNM4E-q?G_Roa?rc(QFq^(La%UwAMYTRew34gAAWdg z;=Q8mEIfVB$iV&F>$tb>KR}IxVyN_kLvY37)_(qQq1?S{v6vt?)!>c zwJsx6mAYje$ZTMwt*I7gDDXxW7{;x$A}T!WG&f5XMDew?EIvjgoU#n95~{_Xvkbiu zs>vHO*IS{FyiB5PodlzxIa4S;IQnZ}4!>k;l<-AFb$7weFOi>~RVyl;nD&Z#I7U?h z01*n)x2->tRqV_f6zc*+AuyZ6Wwh}Z#RP#2zAio)FDh_}HL{OHM6;5nmt zHhRP~2P>Q;)P|!%ejLN6-1g0Y2uJw)kz`czFi#Fd(@ha4&hdKVcCK`9yOaX3B4khO z#@=TugwSM>M)zg5Q~mz!?-?fohn$SV;ewL%&+OZZ;^J1%tIhCm<5^eanlK@-@9?=F z(BS3V|Hj}hv&Mkse6UanzxCK-=^ms4&%b`O>u%hi2nDpvf9;qexA!+FjK77?Ku9>y z*IM%}YPIy4>6w60c4zwA(luiOI<5U%;9jOjhZ+v@_F=uIciFV1Jl=GhVW)O90i&Aj z71nroZ5UTuq4YSP-x{H7BF`JMVo2~9Lq41#;QOZGBdQB0_rk|8nebt_gef+mNr<*kARXia8Y%O_3JXJoI z_7o8?ueOfZffG|sL4m1%%kqjen!lUdK*VrwxGwXKiZa@4?7Ujot`D19>8iM7KDv}= zRN0t#SEB3r{rWM!<^^YOZKamzu+3>X^r5rF-Yp}eu!82AM0@%(bT{B_lG(mCJ8|L@ z)U7#oQ-5O2#9Aw9uG?f4mDTdvk@ayu`~H4J3v04pEjR5rgMQg|<8zG$#giLPp8YZj)( z`_c9*PQgCxS7jts)}Su~dXaPsZv;0~wb)bepe~65L9nO=w?noh;DDzR^ocm2bY4i_ zI6yfe$U-lHhZp6ql5sjp0r!hpr{&VZEV}h~9z)(eH+xb9?Ls@fMFWDN8?8i@>ijTk zp3Wenozt=ZFxBlCY?I1}(&TizFTp|8_&|d#fqB)qqjp*bLHJ5%>QJD#c+M}J^Vtvd zRM^+JKafPtY+O+20NrFbH(T4lnue_B$l|dJ2t1mixVTjb%9{da|39E5aR*ie&d$eTIb{eAz_R4j0sEnC_ivI8KLQTY zqJ~yZN(>V(42Grg@XD-GzTqYL#XPnc@jl;a{s2%s!t(JfrF%@Q@1C3mAyo>0o*&rP zKTl|_`V8Bb)&H8Ry4TsnL<~><({~8334z`7B+Yty^D~K*@zS5P7gMzop_^o$NT<3P zJa|*zEv%!P1@;Ysy`cZ7!!5o$+;~~FW|tLbbrlJHEW?6b-nTW%s0@jxSs=P434&h` zn>9CTt2&hDTVH9h2@4oE5ikog%)X0z6Fkf%FSa?H7m{$5KwMN&F27kjEQZi>I5Djo z3r$U~uc5l@Wb&y^iFJDxqLos)+6L{~icn>XDc@XAn7M1}qnnV;Ph+Dv# z72cUI-yZ8?ply)lKW!9)pKjlqfHbu@_$`nYfDaDFeQqIzztncJVc@i^w;ZM2ru)e1Uf1ZJ)2!w)k`l8j>K6316;mn$|3&QX<*x-~x>o_TC`|@j$g& zdoy&#mkLRA41%a|%Cj7KP_w_&LSJ8qX$!IeG z6|qw`J<l*-zMxOgN-{!qSeburKM29~erdgULT|x5L8;l5b_U1923Rv8MZixAEKZ zC)AEZE7%_sqQ31lsvD#y1pm>V2&L7T8yu?}9;RTXSMrx^r3j}N`j_;K0WW?Kp_`=} zvR>rCJ-c13KZw0|c{g}KJj(~}{yUi^uxr%7{y`vzg-IC|lBI#pnu9%szH6?t2c6C^ zu0y0U<~pdltmLgz*Pb|qp3&s*f^g2{Xbv`bXB4k?A)z6J!EX(E0G(qD)q_~h>-DO zt7r~WNGCLnkn>wpKjq;x@P+Wm6a#1#kd{`}u&XR~?n_ZC$|8q-oHfD-iFUcK2RN$f z<*(=hnBY0%0#Vj~X&QS+%GjG#S+vBKi|G!=Wom><=4Iw6rY)2h@=noUNa)LFH}dLN zlw;sI%F~u%yuHy@Y<{zmmhW>*0YsWA==A`uIR1b`8Rg0sCiK=vu;I;~P_w+$>w&kl!E zEfa;jC4&Yg7;gW>JTEzntY<)@4?oJ~7QAtcY7FhI9-6XKwRtoSRox1`py&|kl6yUI z^%_pRZq9+j!K*LNiXHC+NKfj~h#Aq)=(R5inqKmzhj8-gr}G-;1 zmS zTxY4uFf-Lj&j09Mnm$rRZUy;g%Ilz>d!UzJyl5^-r;?^-Yn6tc!(Vt-iR!DVY~sPk zmru4m#W1Jr9eq$Ie7RBG+BptIfLr z;epV7qV&E(acDiN-OK5oW?;B?WC4xmZPOr`f86TC45krq>lPHn^6oMqU=V&a7Jp^K za5|WU*=T0PyRe+Tx#!s$40fu-$K_&~33JZrJ!~_OXyjKe;@ONVboSf<;!I^hb^+4M*od_jyapHZgE#vfmmxzuMK( z>o-mF^Ggu`R<&f$fDsPtF`@_RL&2r^U=H7_vrQ&Vvmd1tBYE=iGi5}3Y z-{Sr~Mpnwix)|b?CYzgHaW-m9zgBXh{X)o`2zeV2Z!8Ma09|g(MO6?kDG><@QYz?X zrusF;^0we>sZZA{SpGD}_0;~Y-t`E(ajeUHcnEytDxs4h-%sF?WaMU=rj%esEb!Hq zsA3Vec`HM`a{o&QlvrWe+*HXvE}N8%5)RdG4|15GQY-lZ&AMylQl3b$K+coSzxrIZ zZg~gLRcR*VNYmhI_`@o?6KRR@_CmV8c+4gkAtCl4O5R~&Ppk}x2a@5(eVc;8E~zqYBbB5A+~wd8fe6d%x#Kd6VJ0>tDGouabd9U3Ehnd0|}BGq1`_Sip^ zTsS|q^+$zJGVW*kM1k`4$YxlU>0)U)l8R8r(U~IdDh#@`+k(DvpwIi9Gq6@<2?*WA z&qJ6Q9{q_?InL8TY@~5$ILgJZRi>%BK79nEt*Kj#RAGZ?4nvV-&2=^a6%uO53idM; z&K7Qp{hAr)-&pOJw=UWz0F)6!mzG@zUjh`XtDWHI-pqrP6iTZFPJ42RgbR zk}~RV1PzPk=|yZCCSKy)>9(lc3{^vkCgf%AB)K+?s@7~v==){M0ay}gtN_^GLRH9q zV;1P)uZS-l_@I(7u=L1~-*n;qvMlh^Xal>XF3g*hh(Wcc;QaKkyB;j4f6#|+;;HA* zz2jJ-8GZ<>vn^g~1+iF$lkrxHpDS2Y&Smt^OKZ`)7&H#LAEsFgh3{#iiQ_z(v8P6| zFzd#K*_h%_qD6?dU?VxQpa5DoCS3#ACt@y{bDX;j>uMVFI3J$eaC)z+HYjNGt_#s# ze_%W1V%!#ni+_08ywxT%pKnaZD-LJ`vTaJ@jVD}%lD^HQ{$=KwkH6xkY6*sz08GOb{`e z3%asl>L?%CWxfTMM^WlWc@e~$IEu2GGgADmvNGgL;E@$I)9L(4V9CQay1jwb+_a!v zpy%AF&o zfO0TPqdCv+>Hy_kBVL9MC-;IJby4|s|BK{=TA6_<*!nDPJI+;biZO*C*TU~U%*#qT z3tICMRY8d|e@L+*EbarNe$ZQtm%){j(B3ePJ->G0cf7O*uuY1E#00 z)zF5vjYZ|cd}SB>p+v-y{B?)qU&h?t5MwfaB4F-ZTiJOa9WsO-6~1khc{ur7giejJ zMWaX@1sX1cj&oQUQnU7jbA1_+ee;11{R2}Vaa!1i4$?sqnP5~XO+|?%O^>d*k!Fnt zZ;7;h7$4VK0*ldU$V<64U+sKa`-0*3#BjF?+0_h&x}U2FI$a1)xK@9yf8QoWTVR@N zTWX56S|Flp$A|W%ySk&WZs#BTZvAb9hL4R8#tNBQULh!rh7BShocHgA)7xE5^s#A< zLltAX_YL?Dgph88Rpmeh0)k-wf6$!mGYBhUwf~03&YQP}S-L}=>AZA_U2L$BQ8PeP zHA2zR3p1?FAqatB`k4#-nJk={i}#n;OTO=&+WYNeE;0L@Qh6nPyKXh+{e3+)4tlnI7oCqeA+|k;LCjXF~n;BSYJ={pf}2Y(He1z z#gt-?$kVm)LvS`2LC-Mq3gibt@K33;|DH*u<50WVC(VGLVz<8D>zV<1r|dRiQzqG7 z@JQ!Wta0P>?e&9CP6YAyIS_6~m5R*)iWCl4GI_E5Q@$z$1B;fpwBfI~$SwVk>ssubpQ*Mq!?>5&ouHZ3?|xap4h z@`OW*GQu9#Ezu|KSJ!!7?wBE5$qZgR`2Ijx%=20B2QmUIAzM zW0zmnba0g)yOr|Ji09YvEvjs}m2-k4Pl;8wVMTjKGYSPS(=OfE z5n(5gTXL1yC#^Xo=D(4X-3N0 zoUOdADez?B#NBb%vE&elW^Jr8*JW+g#3ne?tqaBEpee`9HHF_L%XoCUc6M~uwJl2z znDq*#sU@__ddk(y%Ox$ewsv}cM8nUPR<4XBjsIGN0((FG?~k^&{jT5A`JZ+J{Ju?M zVi5XIgy;#p0Ig8lw>z9jj?aZlA;j$z{CB zchf?ZP+yZ1DO9MVNM+FFOjF4v<%-2o&B$GH$%kb{lLonEVlxK8W#k`BNl^{uCD$WF zH!s9caIx8mq8qoqf?};!#|zo8;f*ZYlY+=)Gt0npfB@`}R3z9slb3R&{l>6e&~Q0y z#~Tw+HYH)SfiOXN(cE1)cq7x74x}<`^B4YsA@~dy{jnPfqW2nO)@+<`+zDrt0mSfg>^iF# zUBU8Qz%`r>+jGJj<(c)uVs*s%`z}fN2zGw@EvsSDE$>yv?f+x!9fLD#zP9g~*b{q_ ziEVq5Ol;e>jWZMbjBVStZQHhuC)XcU?|nbdhxgO2>fXJpc2#%RTHVJwepGLY1C`Um zSKoS>7&p2IusXZxjM|1vHuRN5uFAq))OL7q(XFMZ)zD?ROLxz{nHEsCV$Ip(6niEIriQ zA9Mum{cXrEe7K7d1IUjNec}$XFnN2yr(qV~AVUT$;HW+fUVOhIH%XY8s{s1!G(i{^ zHF}qx+8D7#PB>beL8Z?XOjFy&2)Orm;3AHM$PR%zx_f&|iufrsAiV&Gf)qN_XR(uE z1#ucV^=SPZU^6WbUpE9hR)&oEGpX6rA&&z&Tm<>P8?8Qd-qhS1aB~t@L2?qpJwRHh zpw6utFMlseSv(uG!;7C_0YQ~4(U+%b418**Cv4Iy(>cp~?6rR6pPG5j6r)5D1 z-W}jBUK*B^ASj4l!b&000zKGJ)K8tQWL|4*I{Zfo5bjS2B+M(765~PBjIk8!i;Z47 zQ6!FM4gQ?#elDF?Bt{n#I!m&ARyr_=$#hY4Bf_##_A9NKRMk-tvwis8UEtcHErD4l zMKLc5o|yV$h1Pw+X3}aDy?X4po5VaeCAc``$a$2}B&Nh5Jp@OR1PAiVDm^*@ti)b9 zG4?1A;ANb(IGDehW3CViIg32cpXlR$ z#b_D?m^5aGJIu~Nw)nlGQ6&Pt_$GDs2A4Q7VDPLe^;2d8{z%N>(~9jmc_EjQ!*%v^ zpc@{a32=f;omc+v_s`8y9!P-*2yg)wA(C@BT<5*Za49Q8IPSeO-tafj?=r>>$jMDcF@WL2)tGz1|!%+TDImD2_68L1OYua#;K!oi2_rXZ{cNJau9E zxu=6w5aFXcQw>7&_E7?hJF&R0ykK9ETw`=n9+m`7s(ju;DV7C_2J#-MQh;$;SRV^E zNZ;9XnGRuGHZGg^Upt;TrQ<*101HJAFX}Ts;vysaN%+6j2SKPD=$cs55gX>h_jAtz zT0qN5bVWTr$^dolD-2XL@lD!n<}rwv4WK}RZNS&0U<%C&Q-n8rPFS#SlJLROHr+R0w z2Q)z125l%8O;&XwkZFWd4%xalrOu@t5qz zUnhjJOFAW~1#{68Q0}z&5iU^V(C;EzWTXL9ls(x!aOm~hb>^dv04X5|*j@q?_C0s7 zRZfP`;CLgfIYUJ12c$R?dMZDZ6+2@!w*V%;{XREDmUs4BXjB!5%!>nM&28CfqNqPq z;#_jr%At%!`l=&U;oFWTI0^omxH6DEAOwv&7hTwYwOoNJ7T^yC=Iq4L@h4HZ2>U^T zL_c&_Xi+JN9u3qX0Y{4F0Sf+t;E;Cu)n%xLveh_7Bj1EsMlrFo_o(c-l*x(4RDe4$ zZd9l?5MgM=Z8@Nsqk=>Et_T83#QlQdfPuVs?X#J?3y7e+S+cb)hQXvT~uf` z&JL>MdDbksg1n?p)V@2bi;j5aqF2?I>qGKX3aV=YjqO0y z`_7k0A9P!f`kHt6)5!4ppLY;x5$U!Bu1RUpFL#lH0A2Z!AeskBzN0^9?E(xCoY@!j z3Jp*y?9%)!iYqsI`Acz#<_2lqCypyNQpBld^W&Y5?S<0QAsAZJZQ$H0{6SCI0u!k#`uX&mMcet$1;N!8G{PR7x)PCfli6YsVdol#6b zaCa@o2)M>fc1<&$MKL@OQ@9sQIKh@r6Pes?RZ>w=Es>yXAv*p&$KSGfQTd8B;}z_P zKY+?D(fWjIbPQE4xvo!Z_JFjgm%+M!p0pS!Gj){oTgb_LXnFRDw%;gOa;ZFvk;0;l zaTV={?w8bYCjnE;+iE0N(Tj3#IQ5X`nlhuk5uo3&LvcMZuKS^tWQRkls9u?kV7X=O zsD24cdmp+`QK;TpQG908*0y@k6xE8d+q~6EDF~(z zlYuXJuVM<0Q4R>2KDsX<6#nONe+~O2r%u@DSZr=qI06Er?BQ=2)c&(r_6X_vfFX>q zB4EN)qRMd?D4o~892vY2p5fcSe44#+|92knA1C-k40ecL{}{nTCw9;wVgmXV$E{Hq z5bm8cR#r0A>kY zdu@A9*~lK*`CPv^jWND3^HWT#0v5ZZG>nF>ZN?xO4PBWKnmY-Z9Wc*%Qp0nX!3&Nt z;Ee=uJ2G28EwJ)7TnWyL^+-u;b=YXG_SRg3Cum+u39nRsEqL+AqJHq5CCZQP7GN|t ztcpYRYjf_?&9AK4GvFvYX<0Ifr$ zfg*PJlW#Gp!Zjg``&d-r+Tx&tbodNFWrBHfhU#?7cfq?#j=?D(ze6c|gZ`8LlaC}5 zPJbnYxnIWJ|1%-{FB0INCwt;gb_nc$Plv=;a0)?SGjQx|NcR>3yy3f0-%zztKLpY?Nr`Q0Sp{K$?qaiolE;!aLqjv0dOZcY5 zQ#HUpQ}n6WKU3_f*R0ftZ#Z-eyl&Xj8`K1hz0Eg*FZ^vG=n?@! z%QXc*DR{F;RNb%H?`NomTy_Fca{Mvh&wK%RUDj!qgT`qVgUXW7H(31g=SzXJzJ@E+ zKT~{zbfN9}yNRAke95koe98JRD7rn=qbd!QyP2L+d`;{+eQkDcVLJRcz$dpIU}*J) zyIroq!8dzzVYK~ow_b;+T%ikm@u6-6ub&w;!7HIaD|524{BwMTuA9LGyDP8Mz-<7~ z)Hb_eFy0ZbsBgkuPvukZ>9!w`fA@cC>MrFQOV;bQlql3u&pInDF2F=4I6KuMDq(qU zZy*KIY;R28u7smA)d|qDW^hi6Dyv=E>M8Jx#o?k!O(|$MoV}gvrtW_j-S0m(_b#`@?~gm zKs3h8n=BD>>fw{6rKH2`?Ak+F`8~nbPs<(~!euyJfGbT~eJ~KUR_#Z3$9+EzV&lhKuDTsDchS;Ocrd}$naewRyKQy$Ibr=W2-vglpG+zNms+mAmL~?d3rOB*H?b(U7&q%U180Cdd z432m#C;b2f#!XBbt{+9PcR2d`shun+X4nA5&Nn@cUC7o?os`M2)jeTHVcO*A*TK-3 z=6S19Ew%O#y`veL+>IIVYC;PORHL<#r;EACN<8C!9;+PH5HTSq+0wlqwUqN*e6KZX5 z^D(vKIYNiCDJX}vIgx<75e4BK#gTi{4AUq1dI4c8NaiG2-tj@}Pp z3w&A{af3`oOLc`TxW#!lqPZ-NanYufQQ&w%^S--xt1<@te)BV^5FMb_yOQ%^$;E7f zIi7p|WF-@1q-P}adr2Hxgut=(p_8?}JQXj6PJHnhdFzL7VhiB!5AT3K({9(Ld?Ijc zm8B})#QGMS5kImOA7xcewDT1az+G@@M6mAFgj$43$8Qh9kEnuLBk12Fz5gzCj{sc- zQ)TB{j9JouQf(;8J6KKUuZ9MM5>A%M#zU!%pwcX}oaU`zvNYmw85tE>+xi3+q%)&8 zOeqS*BAl}ZFZTd6%vP=zOei;*$EPU&h9HF;_1>|7;Gg^OL<_CqL5ssSG1$XvkYXTd zb-~xI^RH@%H=Q#=Gk69ZvD!m~_X59z*~S@H#md^Tjr;Hkbm%vvD(fv^ShgxESl+9O zVZ)g?`ffZ(Qt?rB{#h&q4KYQrcOc9YGd_{CS+l*&{k0{BO zYH&?_!dW)|JPW9q_m`0~Z^WL+qg zfV6`2>`wa+4nrjU*yn4eS0ZsbO$6p9Jt^L?wscjgqaE#)xo+{tpX6q)A+70PO+#VonzQ{)Ql1K*gVp0 z8${d7&A~cADq*UUwWwjZFOm961NfJe!>i8~(_|KH&&U)(L{xO>tNghJ2o3KGgm6nV zR>&uot)iCr9cxy_UW<)aXy8};O6!495A9XW{wKwz*-NxL$% z9TTC^WN82{!KxfxAI+u*I?76^YMeBQ@euh(!&an8N_+mVed-rD#d90HSKtF!Yf?|7 zwwM~Q2)F;%WZX8U>}dN7l+yVV_2u)|Ku@~edDVqllN<>ixyo@6+_hCDLGdV=M#%yB zWiuR4A&s9?u94^^g$i#3K%E8aEG$K^?`&C1Rf!Jf^ zia0H5s$gz4qtqy9ikRfpZ~e^FF0g<0(qO1D4;?-moUAkZMw4?lo9gXqYPJoXqVRf_ zqj!X_SiRn`NNCig$dx}9R%h-9iOPB7noCZBy&xW3W&n^|p+bNcAj}-CjDY_jJg$u2 zzJg}m%fMR`xMIe^jrvm>k}JLCj^$Y|$qbgvMSsP%5dLZu8-9;?oQD2yc(TNb(O47$ z;uO8k_gtOMKqvJZ{3dyw}m)_JR{Wt!w>x|L+y1bDuIY#}@`pCDLsUHt?@ zz(!y-mhAz@09Pziv!~FkD>&J~Jt$X85<)WiL8bao6)1scy;B1SO1WRKo9?BtU@c$n zaSANavB3-byQSWzwZT{%+PVrS)cN77H~dNT*ycseW!(A^v)%D&GyeIj;c%y^!6xoN zDn1!E-91a+2#+z&IYu_7tbMSgT&aex_RkN)79$JHgH+0d$rm||0?3^55Y>#cfvtuZ z@0DnoxN!_2Ok`rfuC2u?XU_*wKF($yk~} zrAu#WX^Ff0d9$3dA=t}zkk=sIODMuCcPv74Vm|YqfSf<92z}=(AP4#8_k|DhskFHIWs?K=pOuivl0WMaYJzeRrI$%VxD_y=ugXBbNN3>g}y zae#*8c)b+a%<*Qal&j|}o(oly@|P)=KUoTSYTDRX*x)zJH!N6`*EKdcmpe?18Iq!i zHhDi@UmKm*E^1p`)-Pzk>b$0H2XyiQ9>9FC98*Ktl-^Q9@r$2a(D;*N=Qqf?j=CWn~WETlA1klQnSI7zg2o$pO z1T;`rN@NQKv|zNY-Ut%VwQlvJ&?;^dV3b@-VVGa}L40wx;^6)=HzDbH24B)=((X@2gRy;$!35)_bZ8LKp z3w(P_+#z;^UDgRTYf=)ej7s!xlMg*!qS@<~f4VA{AM47HpWBfp*re(QSb9ng5n)I4 zUIyTz->?w?Oayva>}UT(P+p5f6zuW^yoEqVObtcs!gtP)`{B%2fw9<(czxtJI?Hp{-9q?j0^9`lh#xTv4r3IFS^-_gIea)-m@E2F0;U% z56*SwB(>=NvP1eKW9AJRcPy^{F*&s{2%dp9g(QrK#@G9#jh5l7pa#chja<9lO(b~I z-nAbu82#h2sT<}Mj)d~~hAfwI^Vnr%60Qfo0&10$czommYodR8HQ+wQTX~pf@Io=A z0x4P(blb7s@AbEdVM$b3(53v@e`}ScyN}`fg@@LQbJuIOO^Ua5T2Qx0v*vX0UN)pF zB)dFX8TZk@m#WZmt*f`!5}nB$rGH2Xvm5uJd~oJU$pnt)yAxxw?i7xyE5*N~6nvJ1 z#adYf5_iu5{`S8X^e=hppUu{dd?4)KSDVZFn$@Bopy7{YkmL-zmPT9Hn}@Q1UMra(Tg|}Cg#Ud615i&Dj zxt5ce@kjbd_it8N{9kw+G2Lg9x~CUUIyKf)+pnD_S(pp5jRHV2YW7*6sqUt@>zTzI zymdq_zsM+qB-37x6@ptdf1F<~-D`JY)maq(&eFr}Y;VF71E-gNJ@9~O`~7nw2Yu(W zGwgN*;7yDFEX}dCo^#oLDk7ME2jK23tmHcgrom2gi1hqeT{|NhW0ndrQ5?6a zX4yC?7!aaLNubr~$K}iZy}=%K7(hti@cp$3K#a`{Rl$U#TxQ;3tLB9w5mBszYYrWh zQyeuPTl-vQU*F>c1i&?74fFaxYxi9L;2gcx%)N zm{`0z#q2yl$V-%IvJ|`zyz|6yo%fBnLHqp!!p-C^X-yhRsC?Vh*N?e@1=|d^o@`h6 z#Xl$Dh{KX zYaWu=6^Mj9Rme|avafg}!W34>H(BKsz{;w_ey`DDh(a^qs+R|+n!a_cvZSL9b@4aQ z^RCKC+%z>RNld<99vHW!v{k$?q*Rnazht z14j2uOM`!RkjzH&)@S0?X8OY+h@z)f-Qwks*%z5S#x+s*pBUK&{o%-074it90Vm5N z&UCd*GaB<~OxX7bBK8q@KbB3b!5$Z^LjxlmgX5*#bE0b#>CIxz20*-F5zQ^i#!938 zjCjWPYAgNK%VNhX4YLs%sJnT34TO2zEvP)q0c|&Ve2Vqg5Grm0=SafISzQA=mNcyW zC@$PJo!&xA`an-CCf|$%TA9y1zz;-YV;SUF+n-MRHL^K+#ujx%1Baube%SUMdPoI~ z`g80V^I+$SMPljCDXNsxu|RTC9Z)xe*-YJp%;jZs)vDFR)MxtM=?Nbl14dkJBzPg$ zY+iFBhv{h9=+31pq#6jx4dKpszIWxuZ?&KtYC z8iHh9e)qzAlHpJQw;mmi48IS!fmb*=0#3B^-_Y%v)?(&^d(vgEL!6>6v4up%>F@@1 z$a-dd%W@#GBoE=U&OkfbbmW}ap*U)_{Sw-~U<%Cjoaa8RgJ_~Ue%DfJ_PnO>b9V>1 zx>(FKEbc2TsosEWX9jWs0RF=3dI)LAVV5Y4duUNhU|zN&LlX)TkWBOhRDN)2m{BQiFtsGZ~o9^bv! z>Wr%tbo?Ri z?Vji8+@seW&L^@=nhu8d3J96Xx8Ic_)<8BMl7wN77;{t>V#BFX4(YFv(_qL&zp-?-Urv8!%u%fm^<*XPfH zmXz@ysS8Og6@i`*!op_2KGfq@Hb7p`rCiWQO&0)vVBGTs_{uLlWjErSb6ur9BBfP> z%Dg=6muP* zZcmLUp`%MAtH+UOtuDhzYz}juQs$*3DJa(6r7NKi}cY#~E)o-SMvGZd4Dq zwp)Z-b@1s&W5ksK6y zLU-g2MnB`n(f$KPr^hsCgx(GPxk}!!bCZ>r$lzY}ch?pzdm15D{oMaR9D1CRj5+nZ zd+QsLI`_~w)Ue$J^L5o(c&)X`-B%&r%1WSOJo}d^ya%=oFXFCY6BrSq2sm<*53#Wx zRfPyyLoRHi9?s#0CS{El?NgN2f$lV`odH>>DKOEYvK;a!7x^Ff$mO(Ow?+;kOM$QM z)dIfAWgB4@FkPe<{fSUqw)v@<75gr1B6PBG?&K9YUP<%R+7G{0!baDH*4ZBs%C77Z zSRmu{06VtneRZ@;5w#$px#VZ=(ORUZ8q-t_T#}&X^WWp_5z+1P7Fyn>Re4R@2taG=y|jCU^QNRRV~!DC^&ANExysL9 zcElccc!IU7=|4=M%|d6y^$xBTc$(=DDXC+QG%t51=q#4N$p^F2bK{EL=+P{2VU+ar zb2!1{{j<53d?84KpY8VbOIKEbNv~MScv}YdzYf<7(sY2k_MRAQwPe6856jQdF|Pm* zKxMT_&=qRqB)u1cOsx!EZDD0qWo30^(3OVu=I%7#*0*BUWac&jqHAmq&)@-DE`&+D z240zZZr0R6X@d=Jywu5afi}^wE9COsYFO8@05=F=y&aCecAOiu_eM`!`I>ep+w+vp z@a<}lH>iR2_r~}QXr80AD$rPx)l8ufK%-~fw=1F~6Q>cdU=PIMfOY&_D86m-_JHU! z6>#qXxi^l2t;9B*&YfCi38M+|NnOlO(3lQT(IG;}t(ro%GZL>3`$uifNwSi^3Vj;= zqn82!z{xo$@T&rdS>fuZC(CO$LSAYT-VLNl29NK4pjQZq87|ugQ<=e=432Mj02bVj zWmo*LD-r7cAWgoHzWZGw%f3vcJ{{LszQB;qT8YCdzXpkV>27-y(10bALjXj2s8E!h z3942=Z{&+|WKf@X$fX+GW)MossM`;bKl#YGLC<<5%zH(+5_!BDp_&MxhG5XG1^0LqqTIc)Nl zD`H3aVTTE1tbp4q;YrmU#M~)f1B*o(Dvh(tYW?N9pO>Ac?XC}J*p z@GOsS`>Ho5rmFeic@J%k0lpOh;)~mGE#I9Si570X!HGP-}Ju)@owqoE@$=XpDlpKWK$NS7*_t0VUjo$rL<g@}r{+j}?v z0^NRGUvQ7HMT*d;0qB}{i&B3`{Wy@PyL{1Ro>5>2W(-ii@J@OmGI076w?|65qiL;K z=Ylr*X6)Z1>^S4Krn)`eEP1V66dc;|wRf@N_Q7&LFy10uFO{x!@Y8Key?kGFLJV>{ zGdS=Ps6;+;!WM>Ho)NS>Jyo1&n_g3%O0_(k5C6-mK4w4C4*&}c0q3G$fQVq!aZ7gk zNlodgy9COS^oU@vF~e*qt((y`UUkFJy!C_!J3(X*<|W;XxowoEN=AQJdcx0`yN4`z zM8r!Tq*fg`@=8%~C5O7(911Y%2dY)ayG$wHadyNTp?XxYh*bK>$ni6rrjI-tY1p** zW%a&(t4S_B0O;4tE8iCWeWVpRkKr{BQAEaAGBWh045E{QmsR#UW|B4ikv1LSPy-pb z3<^5~iglFHr-F;>x>Pqj1tZ-j2Ch(*<+GUxBP~#-i!G6^+g|qb@!u?uj403nf<|K;s}&49DSTdblD%|ItmEo>vXL>XlsW=G*morG z@1rEXp%~b8#PCr2Q_6DJzZHMWabGK6@pgUba)H(}%oLtahp(+09-WUKi{^~KX#?Y0nJ)Scj4{!;ElIBu%t2PF9;~46wlFxq zrzk(oM*kH*I6!1eKJXNEE<01N&^Mrozgz!63(l|G(DslwD@vKwuXk#YzRQ-O*9h*? zs7bgR`3NR~VAwXy7qRTyfgV@y{`axL{O{CVuOO=ac!6-eByfw6w#2mhm{41$n%!3y z-x79sC_5rwF*}B?f zpUL&ZaRPjkLTrllSt}u`RnQ!&HPGsb6~}r-6dCYcN_U_{gS5ie}>ZcTg3nWX~O7iP3XaMfa(`<<6tTa3&kH>*1T7yd^Sn8G5 zNY^LReaD6cYDG%LD6Bt&{!sJf1yhd-uiG8r zK0Hx_uL`Z+YUy)$bEB_HK1}nTYZsap8X!yv`#bkA=cLwHl^!61ulldm4EK+NMCS`d zngUK&N?f?8gkO!N3JM2}(mBwd4`SRVGjYF545ZMds2ik+X)%f3Ob2C+L5dj)_`5}@ z0L6Hb4wsAvP8KA7q@(bobK8eMgA%0|NAi)+W0H~QiJE#gMA9TQYn|RaG?`QHRNE8p zLwGPFJ@#~HT}s&6G7k>s0}pG7R!liU@Bmh0YAY_88k<_Yl z!*b^#+)4HjL*6I#)SCSM)g>*98Cvs6o8zGciC^NBm{U-G;btV&AJSRHHP*H52r#^)=D~Z zZ5es&eP~Ny&dr{Gw5>E!k*dXj)8AqmXu-`WF{fi?NG2Fb{9B4e`sK#UvmWxqOnB;~ z$d2jx;8WQ1g8tonH<`fMnAQ649_Q%o9;+?hFtcg1N8i*r%W>?!*He|QkpgIv3HX)P z4XH&grxcFM40SXwNA8fdG-XaAH;l_pW!ORNCxqH!#PS`4{D@C6jOy}MkIAHpi0+BQ zT@4t;0{69NeX{eB*Et$9jZikR)W~A!3x;T>p+kcpi?#X9D$N(M!0B1`k2d54lcEnkXwy1cGDSEDvaYMnNc=oGNfA(PncYwx^;?yc0*cf;Z4)`{p>P;n=S1me8a-PqMKUHmS-!xHNa0$?6kVgq*_h>XB^*1qfMZG~J8erMhAU zc5ColIY~fVuw+5WGyrm~4`1N!f^*I&Mq%i-yj{4qHW8*02G`@rMaC=uO_g_f5V^ey z;dBP2D09vR6{Mczu3|}fy`lI`j>qw*7OvaA8IusS*)c_}48E5LxpK6@oQ z#7U-><}NCi;DZc1R*1O25q-zG5Ge8pI9TO`1E(&ZzB2fgZ ztSuWq*>X-#0tT`~I!AIV;RAxIGxqr+16pPK+Xt)Yx%~@#qp)WXOi2iE=+Uf_!k8F9 z`TA{@RhPZx&0vr^v9Sd#%3`Xl!0wQP#rP?>eJdExdfH@7B-bOZyhYI-2@v-_5F=Mf zl!bflJ*p^4`NBEUdtE)G`Vp1EjC={w+8}WR*Z5-Z0Wc#(?2T~g44+UF7{m@Xc?IrJ zWbx|}9u@_2z%9$a1Zl1~B^Ct-BCWbj7I(u{koW1;U5MD4dc=Dme|2@i4&!ELQnCP0 z7Gn4C{Dd6NP*i>&H%N{zLw%R1b;UKnI_1EthE>F|V)fnD7q{R$*(K{7UsvOm#1|0caO{+Jjd2Q*9FA&fq7{>d%$R{~2oCiQ{A>-foC zQ*HQP{eiN0x0S=?#D1`Sy|k*wh%14DCYZ4FBmzSHh}8T)#rD9?}xfY!_voc64WLfutAg@-H#)ou0NF~mWnx=>GGN}uOFlouu;ZH} z4O3MGgWxF6dnq!%E1DK6{F)DSQ~h;7Bye8sTY8&mn^-7Xu2MQ`Q4o)*P)6*@3jkoH z?Px)WOc~j#PskftPOXz;mhSl?I`RjzB*{!`j00X=YZhj)RjxTJcE-28paR#BHBev{ z6(hEK7%43W@om?~^gF{8j#Ue%s}~GXObdwWXHHz!PES`qA^*7_T!Z=cZT|nQ zS&hA)#!mcV-619ZWJE-7L?D3}{tpu7?*RmUqAepLVIwXXgyer3fz4lJiGS%^w-^y| z6Pw5(_WlKsiPby8|JP?4rzjwt|MT(`*7tumCVn-7Hv*_2I6?l6oB5wTiA2nZ=!sCY z5F`KYwe|WMw^a`cmFUfkh?a;<2Ql>j88*d?NDZi0RFgrGL;bMU;M5Xk?JlRyqF&ck z^~ZpN))$p6f+qjbA!TBH(&bo?++2Q>={fah4e#yy2l;MuVhuY2q44b98q=ZoCdVT+ z&t?bUi+%~Ynn7oU#dqCh6gA91cgVZn&&uWl5<`3;ag~1Vk{=~? zu>gLOz?~NDR7AS18DLT-usV-Nnl%+H(l-cgxkiu#axlK~_tI+=+{fTZn`~CSeR;uc zTDRU=;iQVM?))Tk^t4$gmA{9R9}4)@m|eZGym@f~CW9`)1sB_|aGkl}(oB-z?G^n8 z0b}>vqd$!I-o-UFNO`NyAK%dp7zrW8f+!fV zP1mF3b+*JXdY}D)JBc(37ZC4G4l;|(#T_NbD#efIYlamy^kCMj>dY6FR`wr~X)vo7N!|uxvgL+rIgJ(hhRcoAc7aoYZKba5axpmh<(d$eS0_2zpQz zpvB{?)wO4f#?7vj-0*bD*8NCGWlSd7KH3l!j9=Oj)9~2-{2Z6W6WKq-$dtW-{)T{)mb| zzhq#@`~`%si7d`ofBqK(gicpZ3iu^Q3559HnP?&+I|N;#7z-j6pk8^@1;q&UQ!3VZ z#u!_CPe~C<$t)ETROq(?m;@}gseHB%IzD}}~tf;dHqnAU*8F1J$eN*auE*BR_xFqU3pTdS_xD|yZx1~PdW`Od@v@pYLm;|ww@ zJY1yC!U1!Ao|=#oAz*o`eVc1MHp4(&D5a5kOlgw37;Dm&dD6KlYR~r^RjEAZm?_YF zV!X8F9J9p26|b?y+}PTCX7Lyg6O&sf7UR#@m^y?$2gg)TN8o(*$z2@*`CX%}PHJyT z7yRqX7r4hBV3PC>N~1u#8ekrB3C^N1JAvs?V6mcHKuu#4_^dQN-Dd`D%B8ee&(W&Xxo^T76X&TLG+J`onG0IHQRSOY*J(OU*n*|FcYKVQ znBD01AQWK&}6K(??lt&XykIe}AtiSj! zPEspMB5cx0`t5?VPPseRyru!ZuH}kwVd8}c6W9S|2J$wG;1ys2J};IN5BBr&V!maI zq20f3>a^-%mKm%|E$XedbGBJ(3Ma!44#KJaV6%>*tvX5Fgt(SdngV;$|RuJz1fTHNx`!(2L}YSk0a!&6b{R?uNDq>4CMfss~kcx>kCosohMLzh<>i>=97dEhgCWp-%WxH6&@)d8|#-g<*4(TNr zY2`2b;41HjG|}!Q-(Wo}uR@6F;15^wkDobz`;eU>5OuozI`u7fFdTIRWp>b80*s&m ztt(Z??yxHqvv53PRYBFUeKG9f>7Zjl`?U<8{N8Br5pL#ctUfto~?zbJML|In9e+16+aW)nNXmFE!e z82&0~^2q79L6ZI`dgwLt?BmEwoDLK{G?;nLQ%>JKorZpc#O5J8nhjL={3@5gg@o#{%4)d#s-5`w3QYcI{Y2x}Ho5iR_~HI5*pT@R z0hNfxiii!US6Wp;5yAX~hhDCtrj)a+E^ly!MN}=GZxCn?Sji#KmPI|YS_-$btVx}? zut#{8S!wLJok>(Wqe7}(2$ z#|g5!GpNIFe+nK}<+Eqs-s0vpzDf2|^E}ytnP7i{Dg5afK>_Jt`%|-D4RTSngg$wY z0 zlnNL($k2MG0S<@Om@AUldiIm~QEp|pW89I$`|1~s`5=8*Wfh&nY2~Tr;@Y!fOTMyG z5h$P{!Tr6-qH=NDg7PlhTIlcb)3`O(jO7O4Gv!=~r@Z~1nhsu@!?YnU+APR%s2S`0 z^r-E31MXa0m#vU6WjK1g-l7|p91PR&1&9aLipqRbbIzDm^USZhcEr8`1r=s0j$^kH zwCX8yicV5pH#0_U1-||oq?w}yGy=<<{Uyx(Sp8`EcqxJVSjXO@t|Qa(a;t}5QzGhs zSG+ovl^ViwnV?#W4oh&A{xmWvFH0E0xff$V)e&x zsI+&RUK0A|Ruok7j$ziVz zO@HPJz_;kTfx3uFX{YSrQRymY*I=)(JkSqiHqU!AvPAxL1_ge6f4BwP$W|k`_?x&i z*nzU`eRSO2vc4_}2_-nLK+ZC&E!iI#1OVDs$QlV?H82(d!2|{OS4LJ>0I@B=Tgb+l z=ax|HueWIz=KJ(OaWL|E*B|vIM>4%+WX7F6VLA#JN8-JftkAKw-V9qEd2 z#K4yt_nKM}K0a?Ih@_p_g8ty*(djjB_b00kxl(;(XF&Io5}BhCh+?(~%t2dJWgiS*kFy35^N zjl&#^KW+7w9pR=|<$fI1m8XW+obrrp{{a={PTaB3l1cG=&QqYSt`)Gqav7#0&u6of zYfuzQ!*-yGli;)qwfL$a4y_Pt6J=zWD*@2{_w6QV3jb14#DB9C(Sp zK%c@37*ZPGyRJ&cQR#FYq}HVQXSCeHsBV{B4>`Nzx>`i+Vx`>?vPy|}Pge4o8{aeZ zFYnK>^%^Ig%tZK z4a9q2_GgsC_~lIhnLcrH{@374pEtfCDDGB_iOfVlMHzn~J0hE7MY<9nZry6xYL!_3 zp$q1wTVxo7lEHi>TXgul3&RWHK|1glz$+8UAuu#-5y0yC2a)d?fkFmJh%hCm!( z;>Q?^C}+R2LsZ^A1Ip{ij1|Uf_n%{kVTC??n|wet-2o@LX0PoHQ0Kwp++FwBw`U2U zZm#IAhMm&$BLZVjHHh^czIb#Y8UCUP_E@@IKlx2>MF*V*o^IMGKTQ96iVWl46-z}j zllGaOZYUvUH>#|_q>tTvk%heQQpN{fI?7Lm#jGz}pq8W5wqmwTBXk?LjtcS&>!51& zSsk!I;!MT|B(f^L9O5V~AY`@s1iSH<8~V(MDCFk1?p0wR_)9OJ%_1QyWGF+0=`Ua} z`!q)^?-9ixe#p{ge+HA*{5$ZKz#D$qj0y&Bg$0%fh8_BQ_)1{A6upEez4K#O{0mun zO1V7V{@UAfknHqcO{{wc2x&H-r zG`C&R)UiJq0J=FvowPugCs*L%uQ54=`M)eh($y}A|pLiwnJ-#(kJ~@q&{M5h1Q6qE5;w^ zfC7c%jY)$tPeOC(y+d^fVTXB7YKBtJ@?(8*%GfvzeYKU4BabsBqr^rf-z1}|P8wJ< zia|r8yJ#?)z%6&L$?1^(eyjut77X2@7hOGU#Sdn$#k)_)Ns?wR`GpW#)KwTNH`*jM z1RwSexs}5b$wWw(CsT4%Wr12(nxRp#Xc4N&FDOJS{&#AY^Lk(D6zQzhBoi=V=U;Ev zLW5L-!BDlrduS~y9|})VcLmR3uHR%&RM=YR+l|_4Gd@%=|MIkPh~W(wdHfbC3Bo5~ z0M_w#c}RBUkjNm*!k%Q9t=TuwkahyL>6iK&k#FSpL(Yon1$ghci2Sb#^$S()KcS{= zY3dDY2pt{*(1`w@gH2(iZp?t3q551wNnN9}uC4xKE&{&Zvya>XaUBv@I2Y%!MLD~} z#?nF;!;as{(};w+YS!|k!{`RJCzaHEwxelA)*cFCpY79nk%;f)a#?w7MU#QKhur4M zQquVTh4)ZX#O6TI&*Oz9oQoJY@eh3zw*A{3T$YecPaj^g)Db#BcNOi0F1Z&$(AMEFcB`Q&v#ZThlbRl5*{>nLtxhjHX zPZJm57tkd}4$8F#h{;D$?&cIu*ov;5j8lClfBo@yoJAGOxocFCX@0P))9OqvQCN23 zN^^QrH&Tk0_BYS~th*>CRqTY4lWo*k@h5QKkkTtoyRO(bOS58q`b{>9<&P0!PyZ}K z9EgcTPDk>R%`osIt&)Fwl z)Y?Q%tXETkeOfv#-VAXPN7-xKRtBKO#*q2<4gY4zEs{4KfXKdy16x62UP!(?*fkh3 z56h4%6|-0xRe*4w6%`G;?aK4&_(VKc6JRF6e9E7*O`{Gp)2Z{93>aTPL|9!Ds^KWfx310LonCWrF|JyPaZq~ZPufttf7WAZ@_135Ips^y2Ng2}_IB5VfR7PvPJ zhz&{?#P)C|k%{;#g!4O5?*k_F6({;0PJN^Pe84f`p3nRb(v;l16BE}4B8Y3R_!;sC zbaT%rXa2SOwqM-3`D13WZ&!#BX+~)6%5qIMQA0c2<39nj1JW$n#8;ZH1Nnb@zyH(# zNcX>m#7_6+M#ck#soT5as-p*_eS%$;djopu3`g-&=s&DC(t=;s{Fx zNF+oooG{STN!_=hua`t1kj2QV%-DDladDK~#{qr4G>4-;W~A<69IRi}bdVZKjNxvr zy~t~sN}efhY`lUY5DbkCLgFYO3Z2@Nw3v4KYWD5u^L)!ZpVagn)l{?7v~AU}Fy!zV z+G}+LQmX;2^o)q3hug4oqj4Z#B1~eNIx*ihn6db9&dsiTuYc$pGa`oaxLCeQ$3Lrg zIy0oP&Dh8|UY-bi4d_Ai?W~Q|kgbo(QLbgJbAxAG#pSTo9(3zkX_D);>pf*-8m7Ce z$HpiWnl|Ht-AsMb;3V5F9o?DDcv{yS< zz z&iNxaTOnIb_51lMeu?>6MeDB^u=d174d4DB{A&!;=Ny6iam=z3(UWgT8{awgtI_~z zo*d8;!9;@IFRzvd$(plkNN-tOi9R_2JyR(Fq6b?v(XYW8hZJLI0MeF6)H4I45sq;^ zwyeAUf%AGd%?2r+$Zsd?(o9C^Y%Pojv=-8dqqTyA7^trmt70nGCR-4aS0_Jb_#_5Y znS>U=m=%Sh0^bph3X!KY&V2^iUSq^|Xh38Cz^9w=% zoQ#nZNQh+z))9rCK&+{x`?E`G*zWNp;@UuO*68$}3MXb8&F0-IDK|WwI#eebOE?g@ z=5y!&hPa%&$&7}u`oNxW(nOc|iA@n`bdCi*QCn7=Si?>%^m8{9^uDIlDdvP=-F5@f z`b}9ENAVb^q0yi`Ce=^iS|zo{g1)rzh8bmmniH!iYX+z9a8?3dlgC!OnVs}|j#c;=N}lps3fs}kt#z`{x5{EoaW_X+o_@9Nt@FQh-uhoMS#`D3 zZ?aPVe%p7SyK0^ne6Q(^YyG_6i^l}H8Zg8l$LJ~g(t0pZzsm}4#{kUGb`B(M+RMa0P* zZAVam0YDI7XcT3R5rBL24g#o*fw$p}28okriLG%DEoc}W>lH*Y<|Hw!Y0P4VjSqwY z%Hs9`*E#&K_LiIo_SWtMO0fG%4&GmzfJ6^%L?rbG2y!;BEOXW!LDRTpzaQoGR_@Sx zN)8x+`9o`Wt}(W+ftC<*))~XoxTlG$PU)gQj(Pz?SqH&DfOyWj6OktS2Z|}~-;p~k zz0?BE@#UjTKLv6`DJ?9!>+hBd8P_z+3T;RyO=-D1RD%R|0u+O2b^@+e*5|1WG#l>? zw}l_)6%#k6<-2Rk{9?MrsJPY@nCa;>PO8j0{=EzxX01!tvWen{q0DG*BI+-17%#;d zGFL|%1V|fI0DpCf1A%zWAI($IkycvHbe7`T?GSW3@$orm_d8W~vXqpiQDn|*`lEjV zKsz`M>F4F>mHUgExQMf64Tma!A?~vqI$H7K;o@B0{&Mnlok=TBEzve~qiA0qjo@W$vz-Jk1*m`(a=oUc z0lSm&FOg~zX8DH;LX4nzv&d1!f*lu{e2g<+7}- zqumrpvZu^h9tE$G4q|)>i+o6^v8BqP#hM-=Qdn594k;{<;bl%m=Gm>~@MhkiCCg%! zXgSX%1IE?13YlFl^q2IHtobZ;+Fa%2lyk%yG2N`APT^JEnj+!6*~G&;l7lsDV2az4 z*{-?m*Dq|V)>LoDs=tEg-*+r~z)^o)t`0sa2@1B!>5F4N(=A=F<+3Cx z^2EZ%N^3RC7^@AZTqvQ)n!Cg%JMA37Ea}PeX4`R95}OwN$jZ>!YSKAEC9FIhp@>)v z1(2c)K0!tp!&curjm&9lCzASmHxC)3bmp=IP=(Svp{jCfs_4ch6FJLVt)4FFr4{8w z8~(Y7Cf&Ae3QpDQl4}Z=&=B1foJX}a2*57kGZai_b_FGP%H9&14TGco6j`z(`?Of1 zE{6OrgoxS)B89b2fh4sHVKoN2hP!>X0ALM2_2yRrS}=ycoQ1FhpcOV4G7|G6 zFkICaP9oSrKdR`abM2fz=P>M7Mx0@)h#z?rPyh76V*>t&-38r>jTpbBEqI26#9XkN zF}57h+*KQhi@3T`!A%Y9@j}xDb181JCEPA+V{FK6O~#sQ&=X8J*1C+dDai9^BPFj%KY z*#qO${5~8M+^%f86ZBkK%(Zkd0RCK$RjHm^ zvYKx7jH-XYS}H1UNRw+VGcI~mxv*Cdin$l8nRUP>5}h~6bA{_fw1ZC?d5vhls~!KT z*U4zuRkyC++1GE6smqCB4v?Tj(yHr}_98+X^tDKqC2~LHn7! zVh5~Pw<~9KW#Y&b5L;!@@eSgY!)k9X&NIQY8KUGmf3NxQv5EJ`Ud-upB$&yJYv0SxFKGDb`L0BjizU;W}g^V z|6rXz_9eNRRwsO;06xRde^ep%k)1X~zY^KI@c%nDPydHpf}95lk)F?s3=6P!bWnG* zwsZX-!I4}Qc}FxBbo_RCea=GpX|6C*C1DtGY?wt^C5i~>^a|ocjl*+p%8rLEx?2aa zzZf~RLPfnK_r=KbBtuis^8@JQzNcrD(hB@M*@zc2(U29zQ4mze9(*g^# zj38HOxotS;O&fu5+_MdUT`Lb)idyt3-D-8?9yDw1iN4iCXX`7WaLi%5;?C)Y&GVbd zM0Aa=La((wGM}IM$1iyOUs@a&Mk!uZ`b-mvh~DV#CA5Cn?IxYS!^bjBqK)gWp|Xsh z+gG~LEPB1Lrk=ao*tb7()IcvF`;|(4FbnEVxU!7ca3kQ>U zrOg{uqJPXRrjCY{CgiHtbutz+OduFkzEjgY3!Tws#OR>e-x_2Z;~aYy*M2J1-e&DQ z#d1JA2mA~b*2ebD=@b3ZU(N>fT(`L;ALs6?PTjX_M!qh)zqn;-Yg=WN!p5QsAj$90 zlOUkg(zSFeOhD6_wZAxD68JOclyWE#-hFf3sQEtRO)r1J1$`vl9R zVs50r|CA#t-$Oc=VDTHu)rlY`Vq7|JQZ8A}VCFw~jJ>n*wDl_+iTl4KAiiR)L@02g zFV@ok*c>69j|UkE5SB12Gc1H2HosBQ$fEOp)=xYtt~+?HgHm!>wIW`so7u9=O(D;o zi|$7U5xqYIbyql&t!8BnhnkV;d&6a^aPu;Mh84($kG;X#U~0q_g068yVH!Nx&=AJ~ z2XRbqM>eihqlV`%c-S`XzwS_}^mYRwTBTyomO>ldtEFfNkTI;HgRCWVgAlDFd=hl2Cwl$;_*F-llS;)o@W3wcfc)E> zbymz+j|kB$fSCs)NuMhBRQ4yNdU(rtj={UoTPz}a?wT8^;6fPb48#+`)=zv|=pP}^ z;c3a9m71oeR;s#e$`t13XofmAwez}2&_4`K{8@p5j(_CVNN9c^hd1~CE9%u;wOUsD zGKAfKrG@_k7^l~CgP{M1P4;yl5ddLvEDoS7$l{*^LqniJJKxA*lSqMrMZzUf1yU)y z-Q0M^qY`PcLyZ*))Gpr;!!k;XremHs98YbZ=F^vn0$I9Pwpe&s2vL!3>_;lls*1~* zMEgY%#u;_Q&?y|700DV0TE zIyT9yQ$L0A|LH=~&ibw_a{G(aOF!a$K6DX~cF?1QXU%^S|?xnkk$M%Oij>&5S}* ze5|F(lFs=mRE9@w*dUt+w@%en<9G)G>2F_wN96iXEiLV{t#q&ZtCq6uF__tfD``l> zp&3}l!OWI_gj8?k_JliY5;cL|ax?w9WRTAiKg+8y;T>X5d z;5Z_HY68)gzuUsK1%16n5i7mcZsVUPE{oo@mwxfTs?R5UwCgyGVH~1I+RD%d9kWwa zmId5mY9`KiP&bjkE0Be5uSEK`IZv+vVMOj{bQYsOT0X%_-Ono(30L^mmh( zR$6-Rc!jVBtQz~W|AH7)95K!yzbZ>Zw3!vmYE2Yjaz77Pf;K@!itjl+uh(oU%14Pw zb-F+%Gxu6HXHQs$BgjMTn%G$#geET!HNZ_>=#7uNHM5FFAS&34o~?2S&t7L^SDkrj zjIbt`W|KdPDwBvxV$qPWfDm-p*@5(=K>)gppG3Sv8yqY$J{^N;`y`va$J+ViPLpjTYq9fsDE7PMw!^iG2m- zk;*eJM0p>O|Js5k&hzSWUk<_kzpC2l&bMCx{3`}PO;0d{ga^2xsiXfblV`PNErnCz zt3fGjk>36mY@{Vi^Ch?`1Bb2Wl+{zFx+I+;N9&H|JdA;X;k@qph9!`oqkLBY6$f|8 zlUEGjKX=~q{9XRsXVz5`+jcVV|MJ80mg|=BcFzAjUo#FgBmV5#s;zi5&i2YnYQzDh zJ+ec8gn;x0-5HP`AwYVA=^P*7EB;mOF-Cb8X@!xqjQT zldT)%2fTU|2atBDPGldb4nr0&T&^x~-+zSBQS3xblkdb&Q}BP&A2JZ5;Ga-LDi9q) z+n_w4X_5BA)s*#uB!QWC2|Q~M4;(JFsZ*dRkkkWx(6^Z>Z6{JHO)$QsS8P8oM&6~d@R979kI*511~%zMju$vRf{<8+c?KU z8hDF+#E`sGD^RU04O**xi7Rxnaige;xyq-AR{?7sgU}|ko5JjOJC0^S5rpIkG?_xw zLyxQzY@v2`v&I?GpQ^6R7dX)~y1a=4cTBA@_AC$6pRS36__w%7SD~m*nCT4+njEWI zGy^IGFswcl!-I64L4i40z%tCwHeeMci1%~V817MSt(>E>&ajzs2F-+bah~CcnC;{u zJ|OZZNSne5Q6t@;k$X2Tq6*c1jrCY?NG7RfidBkr$$=7NXx%|J_Fx0^PqQrHD{__E zwM-LcG%mgoco5V@FdhVa$B|i0MNzDVJ+bOE-fs>n2_!kuJKfT6Got@)Qng`x-On!jWX%O{qY7F5i1l>kXwV@#Bx zJPBFQIs>`yX+u)Dzl@zZ+n%0oU45l2ZM?;qrhqrPBBCUBh`-+$@5>vYZ;fGg$7%fn z5&fi=e@L>g`^yO~*Q7m2dPRR^irLD0gQiju`&!lR`+=VMqGmLQJwhcffy=zk9ldD3 zH$?FDh2pnQaxObFoh8LZ3j&nerVazQF#-&4rZcXaea)2HsF>T!KhZ%_UyO}$2O5O~ zN%N)tPBuy9b0|~h4;XB+>LvYrImsEI^XN+6(|qn#^tg6uy`$vwT0o69sz+#%Gi z@`5>2!KbMy_kA<`d9faTHYVbY>}3$D>`kKEDAHCUr+H_DP%1L+Rs_gb!pR>q^NTX^ z1LC~qEImN*zoM?b!s6X=V7zjr039b!`L4!Z)IPp1^n8#R3@CDk=F^PzE3pDfNB&5Z z_aDLkMQ*w)HySkzMzahJd)3tn0)(!fm$Q~C|Da#SX?ZwoeV~i4uo5O;<=&ECb$PTu zp)v$f&!SQ{7j>K*oR_+A^$r$GQzSLg(IDSrQ4*SOn7F+NduJ|t=;G%da$_Vf3?*+o z2D|T zkdy!*s5$TAUDFGX@0x*cZg2Pd`~1A&H%7AZ!?GwAjCQph4VF3+mC+x}YELrDRI|?3 z3StHOTHqko!AlkAX{ToB4Cj$W7=Lhq%*3+=5|^&Hbw*;t>hQK1LK4FSB^*1$-U!Pb z1qFPGGxZS&iM7zpIsQsoi_vo(T>Q5iKy82pcOOd%7We+K2?PL zqMox~{Z&1)k?KvM-dpMt2nLNG(nL}jP4j0wvTP>)nWz|U_yG{^_ U zmR;uCH9kQ>pYn3Tr_`e6hHF*Fv73M!_kM`00f!NLP))^?q2O+ugz0frp}OLqor!g4 zf=l5MM^j6bUfBmOQ0Gu8+Y2gu(2VxQ#X#hE5W;O+=<+2Fv9<{1gre({OE4y_3mg-# zxPYD!?H&;kp#J^4PUyAI}2D(sq<=^MH&CeGqyJL>Z5c3Qy=Gfv)?(lMbyjY)a2)UEV(g zxw5XAxJAReEAeW7V3hqVE;$5Seg~1&?B!BWElL2BiwN=$Lk^lNwWW-iOgYx1Yp$-@$ zJ4J5q)gr+E*EvgISjUaTOsoMHszZ*)ku8>>%YiA@ zTnE5uEzFfM$MXOUi?QRb63(#GX)+81iC^nV7$nV`8A2Akhu)|&5E7&w)(U-rykHZK z>Jq%>WhhJ>R0A5?=U@yHBpbr+ekfWUR0B@yNW9O9aX^L1Qv0VkiOvIwac*Gj?~Kq? z&`Sursd2L&QsF^F808AoIAT%~MxhXTVJtwKU?JhCAk27!<`AA$>K&$stQX!kMa}>t z3{}`QxuZu`^3`PMXIkwZglb(+ET^0TbrQpNjPig? zv@YCA`<2v=n(@OP<&5UVvIu;+?&SeQvOPk$U)r6g9WKN}AFZ=dyJ$a8|DXbj%?Vcj zvP{zErXJCBUf1ksm|D{c0$31s!`fg@FMibtbC( zsKNE$ABJI}qgfwIV-Jc>#r(~z>Z`^2K4M;C?pmMltidzK3h3E>_r;if+N?W|^iso~ zLETcx{2i=~fj8T8w5{ zD$`FRmSUxKd%zQgJKi?Vi{yfbJ-a&>IXc|xQoplWyi(==$0F*gl8QcSqI8D;52X0d zsOnyc8M|uiW->l4bh6cK+^k&{&QpsTfmu9A04 zVBjLcRMTK?B>S#!55ai_?~F434*6#!HrE^O(Q#QQoMg7`nX4=@aWyT{oWnF!*Th(} z(mW^d`PpFC(f$r>j+na6U)n-)VvyU3@Xz~VE9{KVGxwFUO%=|7UAYuXrF^Mof>~qL zWrfZ^qPD#D)aPAr`9{@k2bNxi>wmJm^sKt5oTPbOE%{Mspj~IUY z(B-}eNXmWm`k{v*H81jrcFfQ*d$Nj5`0!j<`bJU(4MuHg4APj)hVBL~!EE~tLS0bu zjkx`|a$7ToFP2)1Z!E5nKrf{iB%m#v}(LvL7RqZ)wwj`6kgmXTB$b@t){w zOe_KamY**%BVL^KX|K!}5S;d@4*9Wb)|-TTP8`Sddqf3X-pa$>Ev&bI0m@Ga=y_{6ohSl%Lk&zwKWA zULnxl^uB!#9%E3JUo{{Tnz`gm7R&i$e#A%ot_~?VFu9U!_;Pz3uRlYSJ zU?&5Zh!y{i1q~m-Cx040Xp%z9nL))3IV1K!>m%aFo8q63`9UcJB4zm93`A-j5769E zK`EFwvc7Hvoy0sc-?QGcR2r#FmPAdC%9XKX&b6O;C?sb?z$TxY~goc@46Z>BvZ`jPBoj~NkTDN>Ve51 z>Op+|99X@Bf}-5FhoW?@0aJxnFfIh+y9b5Z9mzKo?FQ?gYZkB@>U79lvdIPdwID?&K~*TWT@%7|P$^smV|PnCRR&0= z&SBHSu#*=Bm2A_~B8VMT9i7U#)NuY;Xk3d5P&ujvV_Qf8@BHBDuz=ohb1>r7?kmyn zL?!pEo#EzU<-;^?wUOp7g6|EQ#5l1qQFfoVsA!NZs6Xk`biHlO`1nPb1{P6*oN zh)EZz#E(xlw%?I!IO2jnUsKrr-FEc^^;oM$9o5E+4g)3MQEd~l`GZtFmU@ty=2d1B zuwR!~-8YdU#+nD)NP&O+=%{BYX;Zcqh89z7OBkB(z~t>fjT#3`enhczjg2gouvbTx znfY5`h5@~FxJ9u#>n;y1?tW0Ql(4l^G?LAY0G(efjES_{H8E0!G{z_oXOlB##@rI= zg%<9?9^9)eKPn3AiUOOp#<_0LTGv$v=xj@zTR_*+mmM#b?I|O}w=O+`FlC{rl1H_J z6$xOLwq8z>NqU+{GphD&URynHQxw;2<;$-@fGAexxS*8w)Kiz9!n}kYtF}HQUKYv5 zT`?1-MKu=8??fN0RD6{yHEvaFcW8;6B&6kV0UhLUDg{14iAqx_Xdvhu-QsTnVxob& z2w_vG|gb?(FkD>mNz^QZ@JARSP7}{;#qV}6`?vuXa`Dl?`v2*>6WZRtCHfS zsy#vSa;7K1)RQ%LMM8NdE-4!zvoLchzKx9vBi=Tx>D+`O?P?*h^}YkQUK2IasJn=4 znF!VYeC%XduAbS4wCwCn2gbj?j9EoO+23s}CDg&5%y#N(o-T{749GRUB1S3~Hc~G$ zrQ=EMujd)jYe{}wVMks#VJWTSWWluHnws%jZ;4}2E!eIptXOwd-Y*p3ROI1GzbQ@m z#m*bHDa|XP*rGZUHs=)xxAZC-p=Wh=kIyz+p47@XxUjZDWkYS)7-3Oy%osZ?sa{E{ zGgBhtsA1Ka{Z=i906i!WD7xAeTHWp88&O1{(X5`?K{SB(RROIvBP zt$;xevK^yMue3}9!PkFzKGU4NtQBLOd4Gq!82xQdxCXu4_w;yA�T^{unb4vFO^B zS%&T-E2qqPj|>A)`J9QaWce#a!TU-!ih;H+$KKf{bcU^I6nUTO&M@W<_r&OjVhPgE zvBLm6YUmt}J&l9DbZ`U-dal92porbE^O+8}>^d&YpB(us^n#JTDBiAJYDBwQAb>vd z4si%e^=3R-+Y-C%s|_kO*#-b;P|AC3O}^^CHp{yxCASA4NT^KFnqWFAwvborSY=n8 zYS|{tPW@CvWk&c=g83Fk{2H&oj`op`XD^3qN@XB)r1cjKJC^nHB1(eq^H~N@VCIW`i%q2H)q!j#)9Lq3e9nFs$?oAlvZm{R=e=7h3(W@lvOXpuw)QQ*T-LnCI`EBg+X%4szEkcGmLb7}8Op9sqyGKs_ z-&6oNce;uL-07>1w_)BO>Ag^OFWdZ#*}ztJYiHJmT$rcUzv&@}?kGtQfcLWzkf0n-5ChHkf29Cav?o<6KHNFQU|AWlC09|f$` zmnA!D)n zy&Je_d*U>II8L+@XZ2R(^{ZigKFkQ|2C8lXO*Uk>w)zV{)0ANnqe z74S{+j?4@76O`)S)C=Z&km@T<&e;6E{s;c|v_o7kP?`l_)dZXnx%keQ1=kRnk)LAhu-XJa zH5TtNCW%j!!m`-;vvay^Q1yU}%2nXT0AQum;qUVe62i~XV}{`@>i6zdgRVX%!_}G~ z?d$zM1kq^hxDu~ms?MO|x9$vRGBKJMEY$(JssXApkud6zgu6h}9;fh%f~_h*$?pD` z-=CfHQq}2_;`!-61CFb(-Zj%Qsd|HadspfFY63Y2D(UHSo_wn-Fw~yf3L<@qOpMvn zybWEnPf12WUtbTugytzdouP+14lNrl zkrG7M!A)TPPOF;a%r4Zh`^eQ3g2k`N4Oa`?%Zp-n4y;aJ1Sq2v!axpy&bKu;(RV?Y z?mZu+33%_pi2&;3(NSEV5o0ZdwjHW{Y(0tKSJ3*^Mx#hZt#X((b8uS@ zVNWJG_yry^B;lZV_+2mLn$a6V=!T?hoc65Ln3dF;wUnAUYV9%_=hryl3<%H}TceBuwZgA)H5mM&kgqzELV=11(ttJHvtq=pD=BrS?E2+=?yev-Jw3MDIui zMhxweG9CL0EW$~#n$;7|V$ti)Xk!zua4|JOxWD&+j)U`NMFb_pgNsDu<)usSm>>I( zAyB-JBpwvt#kHin&eD3^L@u5J9@%+Qw+NT8EgLIa#q5N?nh!XD19qzlsGxMFboMA^O)-OC|h5=ci~JYBs_1BX27fcU$_=0G1w| z>xOhSL}0=dV5kQO`Hs1MIDN>=&rmn6ID#M&fj8>qoUBFpE5YA0O2N5#gT?_xF>-s< z^$@Y?7h}^4$bDLeZV$uFBp%((bT;)(xdt=bE{D%KWN~;N&H^msOiM7q3hd?5bg|dz z$x9A!+A^oTUlHB=S@7Tjh}X>;Tavfo$#Dl)S@anu=6?tPM}O-Way*G?c&7$$SDsuZ zvyGX)lnqbhun9VrPT#XYEc=?e>JOGQ;sJldLJ>LQEqh~2s=S=swb^=}^V+t1kzK{J)qth23A`QvYzlI{3asCBC}S|3$m`3lwpzaY4a2ruEXM z?ugP9Qx+Sr5l4{d;g_FYsAJr=;N!psL^AcgRYk5;MBZ=aA0R3BZ)%7MnaU+Q&gJmz zef6a`FEi~4`hhy(dmduP8&d+SS!TwVu|#=+KUsF9h@%|91f<0ltP7?GK*0V$0^#vx ziY9XaoQ3lP!eAlNBa53c5>L3;4Nj=Tm~*Bx1I}>C+_nd#bJ`hW5n*%<3^PX!VYb{R zB&Pe65)Aaj<_DZ{uwqvSEkfY}Io|a7GULesyPc{2H7-6b)xOG5b@}DKdy1ao-FeAt zvw#^Yn5%lYQ36A|M0-+-`^#6E-1`U0bzv*_umLhEvi>o)S5CGFh) zpIJ}o0vPygli)h6b@iCE)!G<4soL?2xqvk5ZT3uY23N&%Sw>7cVTIXFAYN)|(nC7H z+DsHN-W3980)Jc!v%#fRM1*l3oA+m>`iZT>gUtHRWB-m$-DcRP9%LQo_P-3!Ee^E_ z`LlE4a4n9X8D2jbUW^9w9v-^}@_rX)*^iOPvT7>Dy+3I)c$5n^8D~?Chitay&|0^j z!@)ZU(o)xtaB^#|M%8%vgb>kS$gVH|)TdGuQ&Q(vWE-b+AbpBOo%L(<5ukn#9DR^k z?Z+drw!HFoN|4MnQ{){wBq{1UT@!kT_mUAUKJ621ttFP=6mIs!jo=Z&<&fN1FpJK< zOTxWf^ai3mi*feG>eVK;OO^QeRByCCMQ4`qGe=m!FS^1htg(AGVgl1$YBZuB@l zB~fX$Rg#GX61rNi=y9vRsV2Cp?16@(dEb{LH)B~^k~V^5G8x&-Z1q97oX-PHmSL-9E8GKow%H&1%N+z?_IWw4D$IvkX5jaakl_!Q(3Uta^ zvedfOMm)m*#F=kSn8pi(cj90&jmUljOrs{Xh^1i|J(mfFMh3{wnN84QvZA9KKF~fqZJ2<1ki%> zOIc0*%s{r=uD!nYqw+Dw$-uIv2E*)ZFF$Qwr$%+$F@4QZFF$*yzlpobH;go)u^g} zwX16Hz2=(NR9dlJREB6K!A`cloVU`^SxMH`v9?{g6xfhxow;-?{$fR9flOCD?{*u%`0K4t#xF--iL(NxqLlIm{jjza6c9YewfdPr3ujrw)OA*|w^wQDKiT{cyIBZY+vbpB=Zo31-|w<<-M`Aiho?aE{ucsC;IbdyTiGAd%+j7C zN9`pZ&biKDHxBN2ha{MZ4qx3ve-~&CTT9bb?jXAh#W<-(=UlI0l-mymYoNG8nCu~I zq`4agr&ggGeci5D*p17Vf5Q-LPuI2E&*!afB)MA#*Ic0yNz@^Ei-kd`_Cgm75?qR* zul`a32Y0&yYu{N1J(Kh&a)ypi88r9lXkRK=1LLy?{CLk%#%ymJ{WFy~7#1i7%NUFr z91fdEpH8MPQ(@XgA@Hueiv*5Ss%RyJ@VS}b`I1c0Y-Qhb#(=E!lR=AGUS@izSGz&- zl0p5Dho

    FWqMB^(C4DYZP7P%yl(*ujPPLtzA=&VI(xA>QArcd|n+Yk!AyIZ=Fdq zB2(WlZKv|5<(*sxt;&gvTWetOw$6M3$o6%pH5JqMR~5+@?lXUX8hy}fXOjjln06)QSv{h$`G5C@qfYnJf zI$jglz>m`k=9874l??+$%HMp>)B0CWDq=z+eEp4VOflCdfu}l%l%Fbm<$2yCZqJ!I zX4uOLtX@!~Ru6Bjr4iWj3kP=PX=I=oZ!LonK{UXKw}<@j#gwKiAYM(~)&^PDgq0)8 zG8uu&zPF&alh@Ql+EgUxl)|dMzJg?v&B`JgNO>^aRoP8DgC!=y*3!sMcp1JPK3HMv zCK}B}i>Hz)JI9gRY?*8=Knfn%SoBB3ll1N(iNJ=eP<71O`5OdcQIIr|Xu`&8A7v9I z{yPn}I^0uYz#RAhYS(x~#x5_Fc?FQbUcorqe(xG~0`Sw+n`i%q>DJBApxc+@n!HC6 zBP)a+MS)wBw|@POaXSl}+rTToYB|b%>vgK6Wcn36=K3*)e`;XDLlvjal5-Ijnvh;D zMW(X)PQWlV#}6#MJCI3#&Uus1=Q@&3K2|Yrn5yE7ez^PaELOQnP6RsNk8vJ#j=!?q zuiH#oh+h;f&wpvSnJAbXTvaaWcxIR?Boid#X;|#Ct~NXhlkc+P`(bA1gTexF~GE$TCZ7ZeML- z#V%4ld*pe01ZHuL0vf!j4y|IJMmlZuCT%%ucJ48tA=?4OnBQ4vT%Dlts=g}AOL$>S z3JJwE4nQ`eb6vIs_#jf3%@fkx2jR?BxIjI)4lLc->itiTtb>wm<@l6jHZE> zHMYQf(SR{oK;OyJcZQetP}aph?PlfAL$P=tIUw<*59Qje2V2gEpCLSXolR9$j_|;i zk^8OGN(hwvQ2TCJDdleY=~G$vBxiUb??&zk1~AD*LTX?E6H}Y73Z$xxZXb+F;|t8K$i4>Tv2M zrFWuhhDUh8mA}Elk1RIyi8ih1`kM25G}+>zZ>b+Mr95jCij}BQXSFBo=@iH-h%_?P z$ANx05#W~u=N_;_Rptj}zr-c<@v{O1>1qHP_&4m*&mOtC)by=_Y=BtPK3#%+hwnE*SnLev@{V#KST7h(WW_T52b&{jpu?2i3u#(cmfYv zt$R+*w4onnbJI&d=Z`a;tcp20LlIA#X`{pdCOjptsidAhe6z(%tSPryL$oQh!s2gb z47TdbwtL?yd^XRbm3+az8-13xZh!FHO^fVqi8tOh_EQ9!8$B}8jqp8g?g?j|!$NRS zwhgXpfH2~o*yIM9SRC#&1gQRCSY4lAp6_#ZD?`bKs(O>558kLEy` zzaazKX1q8?EyB!+tX7R!?$ZVieeK7L1?OXBZ;9i^;;B()wF3txe8}I4X&%DuOXk8r zdThzHMywSE0r%baE1r-ELtYi8xORSI>6&Fjg7rs40La%AC^47HSMJOfI1v5B0Fsy` zvk)f>r3LYt#D=ddf+h7t7^EH&(n^Wm9eVx@BHAG@E%09jl;fQKq>=NOfGm-CFc91M zBNUl7lPydjVjwUKU-%rw4KJoZG*hVR61#tCN^6S6>Nk`lb7n*TLq+l$CyXwCkdDtZ zmSaqBoy~V`fx2<+Bc>t24G4^{v3$;~*b)en`%DqiZN?hQ5YpAMhX-_Q+v4}X{T}&Q z^0pZx!qwYD-Dui<%7`o4-Mi(9uCh_GK9MYVY$}npru5a*8p?T zHl>uW*M!vQge5n5hI@+5Vuf|z>W%E*U1nmhC9y>j6Hft^IZ^WS9cVO}7tuS!2_M^g zgv>}ZID&nMA<{Oa!!V3&cflnoUKkigaG(UD=EP1LmZ@AF^K58lRb~5TVSkto_V}|N ze7_!DO*7_EA35E&4Qs2Qu{Y@9% z6vdv^@!pg{XRmVQI$=@D#GO|o%R{X{U#4>`Ow^iQduY);M0Y>7hm(2&=9 z65}2bkkR{wy!?8G0F2DEY_J~G)S^{*n%jiZgdsFh=L1ju#9*>bqB3CjYv>z+gfFJ3pVrU9t5=Xl#vI~+fGLGP z@wX<)H>6GmmgTc*S~Wd=z`@!_^US1}&5AHaqv>+to z6h2WzqG4~(AetL7rx)yu1DU_XvG?s>2?`s&0=5Q1vN`>acWibE<_@KNp)-!}dGL4d z=SB1ba!l=W5|=9v*5$2DZfnFB^FTU`4{MoosJT;vCuFy5Q=1yruHZwL+R}8X*~)DO zsv3^w*4b;b*1`=SV!<0-rsQZ$Hh!WbEl^7IVwo4jDQeN9A2l| z4#=Bq%rT+sRTG^nJj&Zv0=^ZV>k^t*CPKm92qE4Crgseb_Zp5V-l14mohEPWEZ^&~ z(esly&N{8F1E_xHiv*&$B~HiOE;%z`5Yq&&hq`Y7)}dY(fYt$?y;xmdPNyQB!6$lzdEqF^{!2kqPB zRtjp>e>Ba1oWt#Wojp_A>i&a{n(}9*v9_4Ln9xvG2v>-!Xx@34u_WYx;CGz?O%gXv zbo>weFm^++(_?=}gX{__!aem3!JMXf8t`N;0P2V^c<_)Oz%oXKm;T!tPM&vH^ndbU;!Ue#$b1bqOqswDINcPn6s^qYhYFI z3!gXF^((@HmDUqCXLhAdqlidGw`IMwOzf=tEVWZ>k{Hawd^iq;YEj}Gj)AxT8#1gj z`s2+!pRuN22n~7X-GCze0a3m;@iXitPB;~{`|LDJ6zbl!R3Wig>|lqTN{~NA#Kd5& z@(_6|#@iD+`OkU#l3CYSnbK{uB&xWSAx9pvQD3nC3|1ue!&B(DKN0X>X3f-+-0#8C zQCUz#9{KWbIi-LQy@f3B6Vy%$+lwZRAPz{`gFl0xK9QX#G_I4g%0PdC{|5B+oHq9S zPDh!=`Q=A6?x=t#gd26R`(3d6e9GEPl=J<3djk8zO)tt!u~!miiP9=M8P>)?iY0=r z=g79Ji9f|PR7@3v3`BS2^M~eX&)NabPaTq{>NEQ2$~&JPv-~lQi=;OKHz(H})`HRa zNxw@gP7&ouH(;7I!Cpk6I}#U!Cn=S7(4!SI<#^86+OUx^Qs?K!ovf!+S+U0>t~7>= zK}D9Anw3&bXQegu%QQksPvb6SDyjYKEyPa_SmMoXR_ z{U`NGDosU64ZQ^&{k6&G9X(Hr(HRcmD#`UFQQUJqK2lW7ab$=k7aAz(ehx}T1FWo| z_3}@xX$H&5o?0uhyNAn4$T4U5`)rsnCheMRb5_V1Vw7}XN@G@*4m~V1R@%`SXaYn_ zRfkwzEWkGD|tpg>er zIvh0^-g&j4ZCGA~EP`;Bz`F+Yp!;ptw#evq+HV==ETv=g@+NeH0 zRKdewO2PixJt&2P`0}^cIRiFDx9Bsa?mNPNK8vJ_491)9PXcx6zd-)2y9JPb|ECOj zO$JJoQV5BfnoE)f9=S8)npVZ%|8af(q8M68RyeQ}zrEY!}I_{T@M=?H*WETAjn%U>bcPD03EbKKL-i9w=IyA*sI#Wywz) zQVI^8CN@a9N#IUZ_=LyGAU&k4E=C*W^hz)XRp=J-u-t8Duo7DCUQzhE@juxeL`Dnf z#V150Bn0ttqK3>Fa!GiZTEV-osjW!`IXUaBLH9ZPL1I&%D1{kD^bH;VP>xqyl^64_zL-lP4 z%iXKLcxiJ#IC|We8Had(@?Eun`-oK?sy~t>O?HU2D&w&mNeT(^7nMdZD;-X9Uc;4{ zpFgP`3{fjVS&7guuszRc^ipxmX;j!?!zOcE4zKvOO_jzz!6Uh~)@dZG(+`7Jxo8)c zsY)b{1D)6wJ@1kLtZ%9zPieRl!=BQz#RM45x?;z~G%n%1@X}a^TpZ=>IYs%*H5+JF z8{?Flx&pXXmS&=|R zYV(iunivza$4fL^$D5s(il3XDhA*N-(~PcPAVp1P$I~@nmcs%Yq{QJy?1Q7w6t>1D zfG`RibFl+a9x;`Xgh*V-`hy`iu%O47^@k*9R?NqgW^mJ1 zfnt`&MFL?`SA!FfmO6N%I#iN+>BK7V3C75oLi1f?SNc)_j1Fn$%@i zTWZ(rd}}AYm0{%-I+t!W3tx$?W{$2P+g*r@HiLP8>#jamU%;_sB&LN1O)-}b&}N&( z&2qHa;rs{PO;d>1U(rJx1e@~KkIi2|skI4M$*rC@luEY0+mX=9k(Y!34U=IBbhF&>UArOm1I z$S18aRvJo(FvXywRadKG7aGY6@(Y24m;6Q%6y>A%!hGQJDc?x~$0A7(kmf0pdS3{z|@Ff-UTVlOd=hVtZ)6aWZYUFZ#^M`SkUZl!;3 z@6^P&p87=ogVhG>duge+7M0~?F*g&H?Hg*K0cf6%R-C7)XB@o=-pHYB(^$Y|$tUD| zvbNl_o2WiZEe0e5r#f9=0Hr6exos);SK+m8`MZM2KAz4x<<{?j>*1nXj-vCutwWl>ypm1y zfGx_4xd~5ShZo!>I=mxfPg7Z$WFxA##M;_zk))}t^Qsx;RoF3g8~ZADj`q9Mvz3-~ zCl%_IHJQCa&82TGtI6YE^?W)4?yK4t2FZ`gmv9HabX^YgSkbsGS;kCk0;e7>x)d@H zANALzB%-Q-@qbZKoS#K#@`W2~Q=T8P>rJNr9?qvZ+lE%IG|N!so6k7VN8OEC;p4f! zzpzB2=9)OFy@c7L0Tz1-b67z<@OT96soxQ%@^|fTg5fM)!?a9Kk)!c;!J3$Y#bMjt zaQw$YMA&Tc^D4V*{0N6#Tj|h$sRm-@`I93u+`Ef9L(~?-OE);jo$mVD^p{ z?h&ksdXlkZ9W#+%8g{Q7+wulNvse{O)G%)abHaFU?$*^2VxalI zIgl=vb-US7w+<^2w1cYytB*!&HZqLyoXYH4|GlqQk_q&x;Wpt`DMo!X>YDOT$4vs= z5ZV5yE#E)()wC=Nwi$ZzEn~;tMjTpGFJ?lz>g)7f)w9P7PDV88kGs0!RJpWnTSYSqXx_7}t2uvKea-+~i>4l-!14vFMne|cFQLXLb{SF-d?vt1Dp|G^ zt;H(XweeVHkR~A|)VK(5bt1vmMAjgm>LgqW+*N*~X z-VAc;>=wai zfQ{4xMLz5%BT+78gB|OK*H)l5gnzc2?X%0Ojaik_{faB^e4%<(mrvS*MHk0fPf?WS z%Y}YZPI9QbRy=C%vY0mgM!l0c-@!9Gq;Zd-VQlaia?kW56-M{Ng=iph5Dv6cjHA*-JcjxQlL3#naWJ&D_=fCe@Au)z5dQEl*LTgj%3*$UpHkaB)IQ z|AqA^rK_R?aHq93RP6nuRcIdnTZw6THa{XlL-<}Yz^f;AEPY!X0w+~u6NBAp1w-U7$R^@}iP1N9fF{LEBV`t^BLq~% z!64nslzzC7=1`ZaBX0$>e|mb<(nT|z^*R+|lLB9-PdbK_*tbI(pg~5rojr9JeWVIl z)=eZ$jGAO9rHvbjWj{xHUK>U9izY+w`tI<7)2c;!x|H9P(2$R+9PE`W@;B5jyF|S zU`&QNU`ZEFiOj5u>ozG~jGHiJYZrb>q%);#mo|%2Q}N9Y`pUq$ z9gfFT_#AcM>Q31k9b{@nua$%L2YcYeq5)@{%m+fC&SGjqNCEt^h^Y@ww#I6L-_*`_`LLs*+bF? z7~IEyMu(82@wK?q_C_0czVwT=P2Q)uFY+uG&%5d~xDRk<&fI}4Av8Ovn z@LOtMt2+#j1o+xMRH+pLMmJ$^isji$kaf($n<^QJDIJ~+aI=6+rYB) z_0vP=1D8(eqoYsa42av6?;99;#;JvK)|zJY*bBd<)H_A8_{H%e`hmz)zI_-|_64}f zf8)Q(dh=0ldjk_tI$Q7tT_O_w9!0W{yw9}cs?Q*i%9*thJ^+bZ!sAQ zA`?RX!+|YpD`E=(5X&v!X`@PZMl?0pNYAHG7Ux_3CX%wB56O!5S5J%EE3wSxIDI^S zwEjo&!TMHy$=WGADvo{McRks4Kj}47NznVX-D>y4qMs<6>rk{mya=rW&A@ffTK;_& z7!D`jbut*6^->>|YNDHT+YqH;(nY(^j#E3~Au_CB(nSN%9j3rZHPwxGlpEGzshRN5 z8J;oSh-0SN$6C^i8d%Z{XT&<%kAs8D>!?G9($x%#?QkG^0}pTC!Q|sygE&PlQi)4s z+=y+1leq~a4rHVk>m`NiDL~&P1=t(Q#wpOeVD{wic!GF> z7^vS6m|%Eg<*IzMkkP#j;)o1I`xK0*UvLOD;v}?@Rr=ISz+bBZVEC$c2(nOs2VKOu z8aH$#4=XQC0W^GtJN(|_efBq10o!|30nbCjP{yykQM51JiVsjleiZ>1N^rnWG=Dcdf_!y)t46TS`P1r()VDq<=)auoaaoO3K0w zNel~XvF1RVE`-m5L~Ts$TDLr7St{mrrNkE zpHRt!hqANE?vwA^WTPRD!09OsrMS61HZ(&F`nlOWlq1tHoh)$<>gS8J@w)4q;#Hu~ zGF;69?SDUng(0sFUI(w`21}QZ63LU=ec4^ZWS)N7896tX6n{yZTq`L1Ov3*=ujNq6xWJEw+!N($;=~NRaf)znJ%-hP(^U z%*GYRtQC{F;P-nIhiZV1@)KR;R6i`QS2TMK(7|EhsN^z}G%9Q8g#8IOYNJ0%Z`SJI zo4|!ZIfh9kH1=HID)$AmXLPrAp<@BoVYfASltl~lU_T&WeBk#oO||F6TlcPci?~ed zESbYqS(rL)lD+_4_{(W!WvM6Egk!mO#D?%so@@DTw=PbB@(e5YJxk4b*xuc#XLKo7 z(O){I5Y59PZy{cS%K_&+M?!7@-DE_Ji}vm>$|%4qb;ypavv`i7v+~}fZtn8Ixh<@u zZl48cRXf1%fPru|&WFpN)RYP!?BnB5)$QvLR67Hzo-@5=eoQhiwb=Wioc=I zZkT9r=fyuz2S5?A@xJD)iY*e_h_kj|inbbZYCD43dcgn7-~Az907iClknZkKv$F|Z ztmY$UM)?#8Jvvd$M&Sp#_VD?J0!KEVf>W}uH3u9A7-=SW+EcJirtR`o6d zX)c<~qC|Q}xW_cvjp$m0b{Fbkkd&O#+?pJ0*Y$@`P70?D_QC@YeAg8F9P<_b3i^Ed z%{q)pM{phV^UagzrgoUVzxMZi-J~S~q+xmARWt?HKUyvkdqFa0oS8!YNyJ^*AJQ3m!})ko&cnKcJI`c zgyUnK4T?_)#kxZ!X$;p+J5XRI6c2VW!^4-qJjV}nzY0!>Vcr_RoZlBMhrG6FZx(4P z2Kusg@;`IPV(0~Ty7U`H%>B*d*Z&BsL#8@Q>)*?vpQ!)49D@D-WIxRP`oGu=zdw+0 zskpt6_`rAtnSr0!!?{h3OI19ToER>$4xfF$Z%EOE-og4GSRQj~w4$3%%T8xw4=s24 zkaj{*6XiwE{^09x9Z%hR2`9wPe5>M6nr`(c{iVbX)KyZB{EI^T^hPm%?CiJEY??b` zZ3Y8^#<)((bw}Io-Ho%R?9KBZcGgWGdA#WX}KI_>Y?R zzt*7dySP9VJ%t5Bbl)wX?sIj5e!-DYG^G=4#T#Bkxlw!;28wax0M3%iT1DlxVwcG8 z`Zm~Bfqt6hUUm4d{@j9`noMk=og?nj@fbv!5zJNeD(&GGn3&-o`{lM9mMWO|2Hg3jMU(p*5e3Y#RS{JJ zJ>*{_b=+(-QB%&7K#vl?4%k-KRii_U>|mCxE`wh+QslCi4!^-iF=i~}SaR!5^j`uZ z#`VDS!SO2@p%K`2x_&!od&V6VPc?&DhSNh>5%EU+d`|bX{z@FzOlY)H>>#1}6@7Y* zX6-{75vxtx-z9nYb7&!!5g`ill14i^!!?ZhD%6&)Go!Q5*PGaoh$%XEd;o<;H<}E7 zE7OqAf$b1=Hy+zz1zkU*v%!oTW0zO}tt+tgxhIIqZ6oCmC{1#Ff)tql)o5gbrc@ky zdS0nxgOGs02Tb{McE41=;M|{|mLA}!5p9bM_Ne^wPx`lbN>9DS4MKwt!&Z1KfdcEp zI3b*)iOwmzc2#MS&2w0NI+Ez)KfC@t0?fV?GNpcV-}tSmT1bo#9HZWWL+yLDa-4`+?+XxqhgH0oGRuvOD zm2GN}UQ{wd4YZ5iisJ&Yj%TuJfZkv4!ngM?e|24_UX`NtO22{ci4IT-DAt?f{#Vc4 zl?LdOcFga*7l#X)K~ z6v!e!#>KEthC?**B0Ag@#mXX(e48FMW9m(^FN0$+_M$z^jq_{tr5<%+mSHCk7J(Q1 z0tFevL2?KbzN}0*G$J?*nk%nByGwl^1f(MZ*QFeK#~0)uIKH|PolEmy3d2@EeGnv= z6SSMKA8+4TrV&FPihuMmWJDo_Pz1uVL4m))AK*MRhm@H+(X(k6Hfwib$ieY33}GF> zi@;?qJ#c3%Jy4_&n0C`bG=gp`>)5(NP!Q~)JyiZkf!M=%5RI^V`@tZXJBki)VlUh< zh_ic#O0d6^o0%KJIk0Dqu;Ma?j3R> z?Cei@REarYGauI}R5=$r-MaOEZQfhhVU-;lgd;1Av2R@DeVZ9MZpFB4NATkO-KK?R56@g4pxVagxc~ zpF-eUOG84tAGMD-nMP|h+@&TuIC``ASMBKXne?U-v|%!j5pHsmkT&DazZRhF#3%@` zT`wdaNfxNb4=dFB#}$oIACsorYnqg<)E*A(*dn3#pBCq>A*VFD-M{? zYOURrr+mkiIRK-N`D*U#E$vF{4f)LlX3A_vOt6+g>MYjgf3fE1H4##c3V-oP(qv|6hI5Iko6oV30taNc`41w6$0_em_|{EslFdsI~j$jcBlz} zPPa)8E!ZCZBo_R)H<~A$0ltHrKZzVy;_x9yzk5&!OCh~>hb#>)6SF32nFnAvp zxeiU+ud;Ya#Ji~DGw-`QJDpj~$Tx#*9mlK{Y~ro#CdO(Wa7^e9+<^iGnFB4XW`pg~ z5Vy_!vFeON7uH`%hi8fR*w1yWN3V-XX=7IvK^Q{@%>*5l?#v9E44P)1^-~{#o84l` z+PZHO9(p$$qVbLPvhFcXp~Ft?>e|LWtd3c{s)?yk7rM5Uur7p`l~G9Wj#M z7pv_1E00&8D|Xk{Cs)&~u9wqN^>Q6FPeE69-GZjQ0@@q5f*>eWu)B&RYSh zw=0L#Dl@siZo#)xoRhfJ*-Y!+_#j|rP(N~LM$pdy9-M1*QDoYEsB~e5lj)Imq`}JGiGvu zdk3+Nc5vfb$^!vQJMf@m{FOE}hCVAffoXAUA&4wH={&IyiN`1{goSv#;)DgdYG z)ACTF+}NUU8{|PMnp}|-b4TzkqSJe)ete}_X~C39kH9&-rCT<2)e$B-9d8DqdHweC zfWr+!FSs-k+^ZTb{(9hNpG)Q3{X87;PH%b-}Mz-pYP zV>3*VH^VAn;9kZ_sf?N9P=#4cR7Cd#91gI8>(%7uPgT+{^uNL_vfqb%vO-e zKkQTq@-AHU1L4{7Eq|XbgBfuR3 zUsSQrOA!OVUmfBa?&Sv)?Tex&kv|<*a9qGbzdPnSx;KnS-%1#DTUbzljPPN@j-I$? z9|npWh7Jxa{~-c=A>Uy%aD5SRiAbe63>a-G+APUFpz3rkK3F@vWQg}H!OQa@#fp1R z6`cmubKYlBgBxm7vMsM_bs!6zS|BEuu^27oUR#I*8h23bN1uI)jX9*@@hwg+A)g)> z$6l6lTHF>%q3PPzVtPhh$@*kkRqd`P>cZV+^)#~q10`PMfr(c00ns2@ zT1u7Lt+#pURJpSKOU6r{CwQTwCM!2UQ>TYVq}#+Urr6%yKGSM-xkNnNP2xBz;#kp7 z`Vx~scDo3UQ3ex93T)~7`>G(^LYPGA#1O^1?b+RXxoGI$`7_6@bt(CRX>JCYtAqeU z6Q&9!PTTU$3tLv8hlg*v`9ey=)Ycu#gxa|lav6%cm~Fc|h}=?Kd!Bb(>u_yCMI$A4 zq?J;ltc2V$wubrRuS2SaBXLVj1*?4Jj-1}`>FGGFTuo4dz~B6XIO1&L>^&^2TjSBq zR@PB`S6|eTO&xZZYZOy-I0x*%=b`V4l~9!HRwXLcMM$K9($ORB;vwjZ>-)=9{DV4) zDAL*Q;Rs5N(fMiGI?)ukq#1TMzuAT8b+~Bz51=`*?&~t5(u;(!CLTnW3)t2gQtaYt zB{f6r&V;hv{t_}jm}CueEJQ(3qcBsBVrPFX&NFvyV^pSGz~8muEU|0*=IG<#8>s;W zMQ>-(*hiCrmWjH!NcfbHriOusS4Nk<*+ymA1s3oFQ0|liX`y_mcatU)5_p&@Xl8+! zd%POKmVw5)k(#`?WJ4X;m-&l1Xz{KU#%8zh`}h0+vBQ;kq03bJj&LeetDrNUOCzKc z65BDwrxOY8z~{LlGeQ!v!o-%8J0ym=$+A9x3->Z`r|G^cFp^gpf^w#+LRqcZ3bUHi zs3Ii&a@r`|#F|uxqqO{()}pYUKqXFtZ zqo*(E)N)c!D}8uD&qkq`A(|URr&tf(t@ToVo5=28cF*?a!3}h%j67i6kxyJ5MN2P@ zVcucALkL!l0YvBMNz}3Ejj_EkHE|&gx2_mSY%BRlP&f0GVooLT!-CsIUj3L z#9fH^u$0Z%cUYIkZ>d%%N3q3p6dfKvX}_!yUW;Iia!%BvcFMXy23`JmR5hSrP8-7B zp7{ocu$8woM#zv#Ggh5wblW4 z85_KVl=m}i{&@$r8BG~IELsawthLrGk>-aky1J}sivTUUA@CiB_Kqd0W?rBULjuT_dwubau%w4V=g9?;M8ntm(ag_dsx zsBz+ER^)r?DTDhj1?c~sdYVO$F@V*Ya31JO7@wgRfWPpyOL=oY$O`C5NJvp#0%yrZ zr8daZf(DVBvM=NllBXtlc=4bg{gIk_wwv>a1pqp#H}z6O5iJqRR@-apI?YYZ)zxdE z)e&m%*B5rE!p7$-cRsJz|7mTxYX9;iAGpuudjj{{Cs#lu7aVS8sY!I~21enTT*TF@ z42!T3#^07j`QmIR+@?k8u@I)#>kKEd^pNda;>?WxCEIt!u^)Yj#_GM_d)Dtd9U|&_ zDZ~Qa?J+QYUJMy@d8|jecfEvSU3=atFnt~l{nPi}84A_Ez>EDsDsVt{F@W-t;5PxG zyFU;g2GgeaN1O;Y4@UEN0LUf^p}RNWk_OsCx+6_w1`WoV2X}%zOS1z@r0N1!2mTi{ z0knsHM;GK6rfo5_aF%Yzn8<7oA0)-E$gdfc@qX80w~zke#Q}c$X;-rf1sTi1->(u} z_j15ySLTNvC_D%Z$T>t7(+(-okzpW`^IXl2k+bi) z4!5^R-*eF2{ojh6e|v*)e~Wg+NW4(CRc}OWRBp)XZvozFeF_eR0l9m25WfbdVD(XN z8ved;+bQ3`G5;&}(+Jw4Gk&T_`;ea0AMk;Ff>r6oINHXb`i%Had>vqb73lwf{fz!T zV$$aB5JLQOOzlrv16E0*B1rB#RFWP#XgI+*>JR{$yxgZ*pNTSl4Fb zP!RUV4Hs7(Xj&>m2@he{(O_qfhYx)e4Omai{(ty-$1us7He0yMwr$%+mu=g&on_m$ zZQEVyLYJ${wvAKoGc#xAocH@qUio8RJAY+F>=pNpd!YuN#Zu(3LXyU<5!Bsm3Tmpt ztzK-5r+#mHA z#w5=_0BiE9P2SM#!!$rNF+oaXLm`K^3?E}dD@VFis*XX>n~bX$$yQl1Mi0R8rQw;ZV@O`#;a1Pjm>RTW*2Mf8H~G@0{M zzUYH+RwE=WG9>(G3I$&>T?4DKw$fUu3LENFIhCGe8JkHt>KLgd2(}{P z*vkzVW^S%n>%8^2tx&DU9CCD)`> zMLXFXys>OSUQI4t;SL@ zO4pP#^JsQe-z9P;v!W=B;;4#ZiEvIoDdXnW;C$QPxd|X@7;VjT&^p%QL*JVg6K`g* zJV6S5w9qp!h1YoH`bCMS(n4CtXfqdXnWu$Mhrb*H`9dPS!xjTyce2tk#KbZ%hh_%s zj5W8OGzB3q5_iIW&?~`;BzKe#%9RMkisq4lE`1$LPnooR_2)vyCT!m7Hl7jBel)$% z)Jw}yLlwZny5Vq4K+h>?9Ef;-&wi#Bc(rTY4sQw7l{(5@$lY_Rs3aZRBG;Ka2_`+y zK4eCFAqA2{e$&$V*Hmo&2!|O+wrPie9IKfm+KgFZkVrU669&U_3{RjhnOf@T)uNlS z(0!QoXJ%DYiLBMIV5E-XQ8|YdW0CQF3E{dtwHLs)XQQ!s6J(R?BLCvGn9oKZYcuwE z-XN*EPK)Anjp!&|QHw$obJZOyeh#2~y?nNkLwzP(Drp*n`=_bV-jO|;4SYdd<(83y zuR%snhE=9YJF3B%H6!*{1i7f}B5BQjuqp4_qd_{%m=)>4 z1r%U?e(+I2c;~9QAz9o-4l1C5^~CW-p&$}D^CeGv)x6!)q0jg*fi~edA|ZsC%Ta;t zIZUKFv)yjhf(<31i20vk3|4e3JT<2*?U7=rXLuQj9`7EzE}?@$x(Rn zI~#u0n@r_bj?u|ZS1r|-oE&wx3Pf9gl?~L?GKXC=nOlO~4EsRyALf!R&JYe})|4Tq zOIlwqlKteZ#N_PRTe*LUTX1tDGQF9m&mjkPl!`#-RPaJN*Q?KC+iyU#r7<+ z!>qG!%r^F-g?iKUZH4H?h%iVF5`_SjRPJIEIX8ygr_tVIZ7iijPDp(-{>-?sBk$-s z6AK6Lm{kVkD#&RC*sA42R2mV=bEB%5Y~5`!OTsJuS~mNa2jyi5w%sQ#ud|_e8q@2U z*O2l((Pn;+%{#6WkGrsbm(91vC9>FYi}!`H6GIs?WPU!T^X3F7q+xr)`z3(NhkYs! zJDiQ$i%%#W2IoW}s`Nk%T*K7)h~fsqkP-O25KVp@=u(L`8{t}kxPoJ0?|~nIe)iB5 zwnBv7%p7?B%8dh$A`lhVo{As4>OF<==MaPcrb)L;M|iF)>uMt}v$|KkB;u z%jKDi8hf_>wM4()q@hH4BJMbQX432~uHZ&~Ysc?|oxAhon%ao{9B~Fn5ABf|a_;cv zU$_P?-2qo}BYI*WHpO7cpdZkb>5Os&s~u6bQCc>jsWpDpiSyf#$l^#ECD=a0!saNp zOxDR0HJ)V~CLv%aPk5$OUw5QnWuNht+f4$h~36k`p=z7(q{pix%e5wz=GCgVBX|UE0fN+FyQ+h+X2QBVD#P;Mm+hx8)t-$sG-g9^sGWzX4EWSkRg}dVB5#g}?&<+V3dPk_Mgi9D7O) zkP+RLDW}i96P-r0?e9Q~ zIU%6$cg>SOLix4xCA}_aGe_~#mxA(Tpvil=m3_Pn&J6M?XspXb%;&t9q(y;g1QT|h z$s`*!h6*<0_e4Qf>%VIG|O#9}ZSGE7$26>l!K2T*AV8nx~D#hwV_ zp8a{{AwANSA@j|;<`=#JYI#(=GPd#yhN#b5lUKCIRm%G(=$#Ac0l4j3B4LU8ZgsWSBSyCx(2xcv#+iS(?gTK91fHegv(!8V-cSA*+#9OKPND z>kUFc;UKF(Wg)k2ln1K|mcjC@nR3z>XbZ;q^JB>wRsj&V#lrl{fZv%-c#}@-yvazP zK|!f^zT@ciz)+4c7)uSH>rlG~ticvART!h<{SCpKJqir;R=%kS22dc-sYojIF+^ZI zkj$*l?=%o=1?W;1zdR(^;P|i*SXO*LZA108=(fD}{l%y2zj`Bxo&~qbu3X&g<8V+n zHK^ZFh5#N=Z(k+~ltmk`wS#S6hJ9I@1A4K9l61+QY^KA)T3S(;ckzB6s1uo9#H_~G znDUvOspE(XPMR{N6^a$L6&6DBr-wA5krjiZOi7WBr-O95i{Fp7AS0v1q;nUJv^5Ou zB8^vKooSy)dqagNvW^BV8qI|&9UgsaxejllI3Oo$buB&Xqr1J+g7w7aLx{N#4@jRd<~wxb;~ zg&`9K7HwgrFP-v!MaP>p6Kl$61g4e!g!UiiHUz1)Hpm0{}N zCV*y6wi)$_EVf9_YV~5j;wI8#X63XL#^#LCgk>4e3Zl*RoDFHLBt6)<`-X+`rUFq| zthJVKF5#*q8&>~*M#(Zct&Do`8AVqaoY1sjW-uWBOea5V3l>b8;4IdQ?B*G8z<^_R zm^a5P#<_s_UNH&6*dni%30!A#AD*%s>8-Gb|)&4mce2N>f!5peo-SE?GhRGEMGmVIHg;efU790Zks zAIcPN%ky=IsCYe9a*JM`V;wkLTOi<$$HX8UgLk^aJ<1r56A1>dSyb@aq3; zp&{yhci0q;!I=Ot-@&nig5SZhi2VVqDvI#&#CDkTq433!qJ-a!?Mddwm{eVBKCy{K zC5P|a*!qh2dB&0D<-D>NXYMaNAermrZKeUjf%xmsR#+xuW$Y}N_4+LceaBiYE%k79 zJytie%&?0o6=vB!$ltc?eYF^)_Bq2hGq|vZ&nTxB2Ti69LLR?F`bwy?yqOX7aJ;Da z7qS_x9Nk9EyEo5=sfl`C3OV1MYC`t7xj&ahpWCy1dnAWbkl%tK5Z4OmiGH@Z9&@by zy!r=9+OanMM!!R2iT}ODjU@a{8(GY0`kzJaYt4JI#SIJ(Wbo91FMH}iXczlH0k zLsd$bGfds1ALEOC!$si4ccOKob)1vHG|`2XW-9gHs|-Y~ zrL+Kui~XdauNvRUL|s;`bQT`r9aY^JEso@w_X>nUlP^gblMQuYZ*$C7cb%Tom0php zUT;ao++E$E7beb1vm*57Cu^HwT!0pUIG}lZBhF{T&@S~3C^_l1OAa>kIA}Zy)uM{@ z8EOc@v9uQAkpD&Joxge51ketcl0{14WvK&%8BslD6Lr{hxit&D(Rh>^98-JPiF7pK z47L;^59O1h$-P*2T5}VpCyii|d0gFIIXDg2k%B|KZ5_`bq<&f>tGqQ_k5p~*l3sg(vssofJm zem2}>;!z#3h@<%s*oe=^!x~JmnmakWFy*1hizp_-W5>r98WA-WRy)`S8(_FE)2aJW zu1dT<_n9qTtJ;jrxz^FZD5KBmZj@WQJ<{}>aPY>o;^2#d0e61IhYu1gCSq#5DvueY zE;9c(ByiJ;qQ?O|7siG{6Le)i$S;*2KId+hikYH^lBTbQE)`&FKC#ZTlN{+>kk+$+TJ z)wqAtzlqPKzD=UZ9n;{7X^dYXxi>>>M6q&Qf}(79to_e9BbDM~VzHPWEK483KUR?z z=j3%E-wp7uZ;{~t#rpkcx#oW_S3pjpwDgeRw~fHMI;)ztehWV}B>pSn@~&)@5KM`L zs=`ffsdQo@IniI#4+VU$hX0_qm`f)Vrk$FaeRlPE_D_u+t@HS!pt9Onj$5xR)fXA6 z59Pvx>x#o#;!)9-G(@7AGF$G~Pa!g-){;La5GhvEsMEE6%iCqo{DBsWyK@9$@X(FLH{VS`p7H!X?vtLV>MR(aEJT7+m@5dNH#R5e`rEE zy*+cCAwKWuiQ4(TTn`IK=~x}29$J0BpNet-S*(2g+Fzhp^NLiPiZ6xd?HLvUHRxJs5 zEF^`s23EKuIsT*$hmL$Yw=1V&(l&N0AedAh2qNf5Ad?%`X2|L9_8f|X>b}bR&G3Wd zoZ^^lMY2B2fB}q163MV83a&aV6?yGrfQ9H(A1vZ`%i?FCuUbl^1LLD#DppN3fwVUE zfm~T*LhuDZ+jdd+Ba`Nd)OpC_Rb#PMVVOH!FF!cJjrj7ef|}P*p(7%m<=Y~4x{#cuZqv6N9$^B;qCZo`W%a$oVri{^J?0A>uK|;_so}Jr{VXd z_m3Zk+~koS`QeS>5D^~r;m#toXn-3;q~(ZbI3#(b8x^E35!i_G@D(~E4H4Dh1yLOa zBaz|qa3&;gQC{WYCXx5j@S{kN-0-vrkE(D16vwRar|^FQg*Rz-<&aQ_;Uk~5kpL9C zcf_CaF6|=X3C9jF9)11@knP(25MVwM{b67}BK`W1&(c6oknQUI;$ZK96#qD==XQU2 z$Y*PyJk)C{pgi5%_!fk!f_bPt|=n%|Qgg&}JbdOF*?i~);-;D1OKLUWT zA~82$M_?0itQ-!T%ku$4xKpYLol#kCeLX&sr^NEoZz3Ap(Wk#j@O_- z(01h^J$4%4dF=&2{LVw)=l7v|JIk;PEZw8m^h05w?RhxZo&jQjXO1Dj#w_TUfk)mB zNoX9DLm-Y?{umrZz0dS@dzQ*mI*yhmQq8ph_^yR%r&{(n;|Ls$ryCidb`Y5(8`8eL z0PgcV)XifO4(}QxG!3%ef9FT&s-4EPcXhz;zb0`4jPD6Mb+C2J?=d^Bp&VGBlix$! ziD||-cfJsW#>W%cTzG_Gra?zs?N?%ATx`H)I-O4h0WIgQZ}sHsg%g zF@XxgOj8(xL(YxKBISknPMD&*m-OIom3X%04cxH^H74u+!9&`O7?*HO%u4nSPoH>g z#49s+bt{%;kQJQ9eD(Bl=cGfjgC_G1Qk{G)r9;1i(4Yettg06ptZEP&B)B<3KTp2K zzSNy`z+;zwHonx2TSVUUm&U>->rNQ68x$t%&e%bxO}bV)?18f;>&8P!bNEq3`VRGb zb6{A&bgp$b0(o;J&)qmDSa^kEhyFb6nm?Oihf(^~*(T*$)9j6}Ht|}(BRlwOgbcZN zY{D~d2ZR?OJc!WogjDrxtV6Ry;gKAiw|j%^7y5wg7wh*6XoZw39(LPd)-D!Ur{AG? z-ysn+RtOyEND@AwAl;lSv7ZSjjp#?9x}7>IS7FG3oorfBX6-o92yA(RRj{U$1l%vC z>EuNX2{>>WQTTB&UQ89(pi=4T3OhP{7}ZHfJ||c}p!DUUOA>d;NkDtS@gFKmp6*|R zXpjpN@b>(C^+XCqR&25jo9(N1r&5HSn1)tX=$}!Zez4|kp2~`b@a18l| zvIxtdULXVbCFJFzMOstkCrs0u1|q6PX0k8@JU+HhkJ{uNeCyOp?EQJt=8Y2!4b zqMLSrM6dAIW2Opk5HCV2lZ%E4Y{Jcwj_%XY&&>7;F)N7_&Hb1F4a&mf5=|1RV)Z!; z?NeRj9W79EO%!V%YPJ^@4wkLwMi2M47DSc>skbz?gCn=pnr%L{l!F#yFNY3nMf4*+ zrJSYpKB?tF?hd6Ut8}U8#{)W=$;L74eFf*x8~&I zmivZNbfXW-%d%c(7;D{fbSerwj%xsJ(Sll3Z~ARiG}|b-<`I{nPV%0v?Y%Eu5r;Kk zKycnZZc^Cumj5%ums9s7Hr~og3$;lcCWaLtj&Q=!Y(F*E-Z~eX+Fg_(5kX}(ecF1c z2OlJr8jXs_HSx7)s! zu|8fa?j&%i5g4B54W}Ii9XiSSe8>_2IUH>8*L2nBP$nP|;+kU9~H)V^djaYCd221GO<#qnx_XpUQx6L%J(k?a7!bn~O-ajRBHlbD}`w zIi1sI#9^{wEamAEk!I4>pVTUmR@xtBt!#hT+X!VUOSH7?q1_SBI5<{kRA20FF4c*@#=K6NBW)C7kT^U(gler#$T2n1RhpiN zWLC`sSL1DZ^_^|K$13P(&azSg9BhB`I(m_r`umjY|wK2#P?Fr+DE*WlT_MO?tyT4S#< zv%9ugJMxt~$mQQw1Z5L0bDJ2BP#6A^R%=_Tt+2JK)m_>Uq;KoQ+d=IEpr?7e)Hy7!yP!m;- z9?gP#CH_>O39lkepJEzTL~355s0 zlW)pdK{MG)Bc5vGDV|aSXlIm24yBmEs%K^~No1jRf9%T)wAU{9KrG(TS&7rb<>3)bGFQtg<=7_%n1hS5LiXi=fNctK zJ61zWED9E18S#|WCUj3{z#hu)HkVFI&3MJixZwDalk~=Y!1MRK&3rA{f1pcN;wh

    i!K58(*?;OXOxd7f4mi6!-fknQ;?Hr}sQRK;-dsb*uo?OO3{tri)5S zMOsnR2VoIx7nf7X4M~VwW?6#|V2x&o_6--jwn z(_OZv7UH!-hLqx$@7w|G3Mv^%opdD2e2XTU>=BkaDOR_#krwlT`6aNDQ?zJ!c~&w! zT?q@vL9zxlo{AEDU8}Q)D|48-xs%7kIIbRd?h%;~U^?wx6_$@=OdFI?C0aJqG<||W zc}b}j^XSU`812#3dW%VZek~#Cb(B-mE`hfc?$Psua0-tA0*7m)@tWo%FzWU+oI9Fc zxf1^l%{==~P1!uz(v8g9%dGe0-dCGnYe#+EG~J8P!VJ1!$9#%trcu1B;-vXc;wfI>W`Us!oN`9ztKd(x@0GlS{)^iSyQVs^N*Yy$LRq(s!b!#A zAu553HVz4l*e{7xzsO(WC~#G;XJ>8d^_^U+fc#+z4Br#2>3VB=x*0SGJ|x$LW0e;t zO6J}3L$z0o>3iAj;?zt3_1&c_5mV))3PkBXiR!xUlEP}&$zK+A6Cz#NlCX=n)Y3@r zG+#c+FqKCW_q@BBjal6Z!O3u)9gF#Fwxm~n8Shk;*gD2{`dA``@~0+WD(=NbN5gR3 z04*zdF6-Yfi@r|s%)DY{TJfS}HrtuWQOp7Oy1JPU;j5Ul#tw)sgH}Hf4Z*)YBE8^0 zX<`<8!yKzV7*~7qY}^e6TI@ba;urOT$y-o>UJ%TK`R-jIQ2m3}rVoM~O;3f5{h*U4Hr*No5aLG`Q33I5L1 zmmzjmydiFV6hK?|kZ$@Vlw!;eT_)I2b@fZTlzMwKt#C_}y0@VAt zmzX;N+dZ~jf&oDYmP_@M4SO#j!rI{w$zVVt21_Cm|3uXQrT+F)OJdv_dMsuPB8?QR zF@)K3R`baDjyl*w9yx0^y+K%#f~4&w^}`#!kD0^V|&Y6PYF9)RVMN z{8}2qB5SvR@ShydX)U$DgP3)U3O8VDg&pt>;Sh564EZezff*dUMKKEmV1%Xl&|5}) z_d&FKdK0Om0m6VCc(l%6kyb&l0UF@?mX6_GL+ddCZC%|*(t&Kxv>u_W0k*09%soqf zvq1{$Ez>`jfLNF<6Y|b$bwGeJuePYJmp@@Xfm^VaTv!(CGoPkZpBZ%l(7f|ao&e)} zwZ)>e>fKRlTw8jk&R!`Wwx=)|YZXxRzHyQ zSX`0ooS)o|*ThPlyYP8+)35ZQY)_e6}@UsM5reK@Po1v)2oS1#@N0pLlrdqBF-6TEPn zA430PO4^yjDVXh8Y$EX0&+&gok2jMIrKlcAhAvB7Gizwj031+#*yenTsC$tj`I;m? zKxRd!HSuq=`6p#_}ACdXX`aXARk0)im&q1=Wdt_giW@B{C*4ck%i18nQV?(4yLlDst1{s>!kl+gkS z+;Ut6oHXB%(XA5+wtmS0_j;=zA!fT{d?}2lC_H9JW{jp9w5I6ellb69{t_d*CY|O7 zYud(&-_eAgscInaSdjh|?PvGWlYbu4dx5wClf-)<>`(`Q0DwaDi<0E7f;|m&gAQ{~ zhV7UrjR^Dnyiw7P)Y?2R!0h5FDFm@W%Q(rF#kICP{CQa3$oY7N%Qfn4Pr?MMv^kf| z+Mz2i@aUMT#0};VAf_KIEV=D@%{<*BoLuLRpjTqt2Tn0?mRz-&uk!%!*b9Z^5yiZ^ zzTbW|@5nIF3fOm)<+dYgc{s@S(Yn}JIIXK!`W@pZHCi#XNB>)rw@qm?XPb)=s7=+8 zxde27%7jmQhhTPsK#BhY^D#_bVnSZB^LKtK@V6duawzr3vT}dqAGvk-f*uWKSc-UZ zzYSj4u{Wv|y-pAzVFTD9cKC@1(y~us#Pt^AdBVN|wKu zvyo8k8_tNH=#uAJhgydeM3~iluxua_nqPTi>dVs#2*R&Uh^+AvcC0%_}pxqka5+? z_q$3u!ULo`Mn!HTh}wr9VUd}*0N0GwMK56&3DHVv30uH&KQQEiBQ${Z?giQ8zIjnF zV>_@GgBi1-P1~s1aM1XWI0~WD9*Et+rs3aB>vAi{YXGHWX08?HnN#Kr#U@CuQ`Igf z3lL7-kxL@lz!F)gIAU9Rns!UaBEznp(2Wn#fgyL6x8H5&etV=Hi7`Vl%2Q%pPQ$v^ zqGT;^3nZ^&* z5r0enFev}ev$*f0xc}}5>H$Mw0xUgHmeKfvLd+3|kdc}|A%&R;88m`fsSIQo!Ht4! zaBa|yEm%@V=eHY1qs7?e7RZTMWf$0?WIF}di)As~3*e-Qbu5Tn0@!7r$o|gz>i2%0 zZnYl8kZLC$^v7}8ANIaoc0Fx)Z%uW-Z^5Mfs2&wS>y{l@N8`=kE`rWg1YARb<*6_h zZZ|>mmu-uJsiS@qzBo{IR2^goXrMo7_js6lY4*ICdl~j1nEfdC#F_nQ_b8bCsP|N{ z1d?u?vHId~q_OILJ-0+}jXz(o(vJB7Ndi#`9B$|PdxIVOspJa__7OOb{MA|UqZJ&E z({>Q_C)p17e*^7aj0+L62M|3%uFx2)#ECM3-oG|r^7(B8T8$*G=&kjSw)%p_& zq!xZ#fg9xcm-`13{7Zl2CqR9JF2MJfE?-(uMo`C}YCGPa|C<~)M9m*<47|y(ElkZH zGzQ9BuzTl`7pd$~7s;@_N8KB~N1Z#?rJ9Z2$l{?gqRCsbi<_$gxB=$RCyohI$;M!$ z-Ui0U5TwBv*-N|0kE{#tk_M)S@qiLn^?|!mxpBRDmi18>ne>Q@wBN&{_9Ih!hyPvi z%$KWn1L31JLf|Mgk{4VC7$Bkkq)z>i?cchgNBin`go-DuNoI5{LATu2K(Dq+aB!}t z_K_g?Bb=HW9>T2y7?T-3C7b#kff9;iMZ_vr6q_L9WTpA3(v>R%zbeG*&`@n~PnYIn1=E@;C%ZYkkBL2#a$7f{I675HSScx{k8LKCiT-mnlP_Cyal|xibLv+>qQ$G!Yf;+Wt#pDFPfu^hJ1^2|9pv~W5v@D&z~>8D>jac3tytM0L{iFrjnd$2bCBrXI0Uz zq-|$1Dft2TyPK;%7;#BHFjuO?g5snb`Jpxxvp%m=Y|Go3S>l z6U)lf6AU=S3{w`HUz{plnj8@_?r(^LwU!*hV$FWQ<|!@yo4(UISgm2mV>b283hQ-T z*Vdk6=F~xy&Vf`Mtgg>sTvPhs3tgZZOmk3<;Lj8QU=bn1H?P*3k)Es2*1}=R^<~H~ zXBQu?-ETZ}XV$kYZLF^1pO;e~8jLs7){M;-nkhXa>rNE4_%6@*>LS16xlKf1uFLEc z4(|zbOYA-b0FfY>3Yfuqx`Cp(0l7Dyjkcf?Aba%{xJsuhx8Hcx$m4EF$xk#F@4Ga61p&0-4`4EF@Ic^8H#A=a`4J8KjZ2>W-b)!(NmE*6{{KM`75R z?rd{7g*WqDPoo@YQokzKInwC1USK0vFxBw(ZK_^FncC?#+#1=!vjRLgB-DjhYHeci zYr>|?*~7ok6V=rNiq=gxnD?`GGPp78WEzo?MQ-!78+d#e2@_*~oN^Xf`d604E_sgW zAZ=zuag}y-{)POPH2G4YMe&;1OIboWqfUI7_*kHpnod&tAm=l5wb zzMDmja-M9Ln9soUQ8`UB_rx|bgsfk3w;Qb}u?4t5NqvUt5|a|c8=R_jcVeVCC9I>b zbQ)Id2dFE@`O0-=T0X?L=fCj&BBPt@1Hh6*l@g}N9qKMgC45(6zt5ssf{tvBNrePt z;|u{xxHLUp6Ubb}F@NmZ&_PGlnCO6I4*3^W~S&Z}Te}Bqej#$jx{p z8o0gTA-YG7<_1>S*4FbY{X{XYMeSX$(5ZRDiBTGxqUx=2Mt2!K=19Cyi|+6Pf-jVr zzdd@F@?*b5c^U}*rsqAlkeDG|0k+SzW@`5IXT46@6q@jfYy?>@cO%7WEe|WFIsM7+ z4Pd+)(7wkd>xPUsJ3{7^Tv*xiVoNDK_MAK5B4D*hV{6e14X(wqn<<9v_x59dwLfd* zYpV|L0w$#SGI(-BIK7dj`b8zHoj_daHKbSNg2f&hzNoadT}Iy8-YwlAKu~jZJ_}fR zly*^1luT5+?kAK4N&JX!mN|cnZ3muJZ9%~+Sao+K&xjy!h@#%pwbeljOPAssCHBx_ zHbFq=CWRgPGyQV5*|YjJHlW<;WJ^@b-&?{8LU~Iv(miQ|kG8&NCD_-3Ea#&5?g#uX z(?---!Jg*)E3ajbKW0q^pn3T>v&^iOz-P!%^N2u)4b(>%87cI+#|f~KCOg|=aF-T{ z(*@Lm;#jA|Lyiq5P4Sh`SJ1(4gO z3OP}edk2LkObEM^Zu?47KtR3tC=N$aXY?~wtn&5w*FRp9e_!&UmA~Bxn2`Rh9Qkh{ zGA+z^IT8}DD?O-yIrb%fB$G2Ob1_02L2rXma|~D5EfJ1PQ=bNNRf_*GzGfs>aHWFr z6WL`B;#gSpH#w#hV$j!J&pw+;2d+5nSyDYqmRM&QNB`e)Uzf|(rcB0D7x zFXN{iZ~+v(f=yMLqtHdxj;2L6aB#Q;<6vgP*lUlWRvo(zG`p6+@WO@Df9Fl4OVmb= zJJQ^gn?dcFhehfpV#Lf%60Wmd7wXquU3(^R=!1;puFqN*mSV&+w>vR;PkhzWy|$Ph zwg6{L7GJ_R$$A=eI6TdwVm$D~+ZVBo{%!mM^FmZo_hk9VY~qVh3CttSkFoWsctZ2H z!okhatikV60+@%AERYG8^T+taD`T|q@q9GC?rt`ws!? zVly-_l-YaXoP|Z-N_jTdm$xG@f3ogdraj}OF=qmX)AIZ!e~7*a$7pt@7|8%BYeV^d z>kzVTOVO}f4e(5z`YyZ{jM`Sf1ZcqwblV>7ucGf{_d4&4jEhqrkXc4Z)?e#QOsan1?tyfJbw3qJb4m^vU#4~NWnssb1>S|Jomm-u) zQr`MIX4r00Wrcu^6Xu&0aUaJ0QhSnFqQ1=A8;qfGlr7~AOx%mQmf?y3mtz(o)?3h< zX}0wV!#>Lu2~cOR`?Xb&GMn!QhZAw>%w^H&m;@%u6>kG(SMf$;)JX}wuhs!CZ!yr_ z-X0w9Msjvpy=ci@u1*l-6}PAc;Sgd5rMS>$L%IDdtU#k&(XiK4_;A-`*gi#}On)?Q z(qFWnDVL8KJoKp_Ub!k6%`y4$RI;}`K* zY3uM12mn=E^*>_Yg8zi!dN5hVYRX{@lY-1=C4IrjMwsx%T6{FF$8{U#bMU5LxhBA` zb@hA@d=J^VewX$q_QxkV=Yw|YYOe>I8CmAJZ;9UKpRY$x0zWqTVekY_+N<_@WAj{T zk3Fi5RmV`^5ZHL<>&pxB3(5^8hZ;l6u$NeqSpmE}0au1toB>f%I_%eY^jMgK?EyQb z=?c4%0ZHks4$ymCE9o;k@P1P}_;!bkQ5m?0OesUqwTB97RY&ckjEQ;lwl}typuE-G zziG4wQxH0*>rBqWRGIb3oKe;BHn;6$IX@pEC{p~kiygPC`)F%!XJJHS`i);sIKC;X zbpT0j&nO^yWFdVX;AK{{`RVEC_j^MJr?TH&0R*p30fq*>EyoOExdxc@F9nok50T;e zcT$3>G+zuzd#cH!wHh~z8FP4gNWKD6r$^2=9S}Bc+umB&ViPRu28ouO2!*P@>Ubx2q*e2rGqLoM; zju_(|O&+$$C1~PHQ_y$3II^~la=p00;mQOGjyqO!CW|aZokLftnR4ikKPKcXwki^` zcPt5{-BRg7_RNzgiC*F!Zvk)f#Fwy!o^x_5kzt0PIHZQfawP@JJRdu?pN93Fw15MN zTFTgir&tOct2R*6aVNoq6122Q84RHC4m2q$rB}0Hdl8H z7PxT3g3G>=>D`|gw>#wJS!wx7jQ~bC3E&OSHG~1%pV;N<tC@qh1#-{g@0{UQ9mva+$XGiCTGrz|PtYG!8Y zMEb8wVS5u(XL@5BLucn0C24sSMNGanx*X{me>g(M-bhvHpVZGndK1FM3XTCxOx3TL zQVlMdC#bG5fT0>56kW)OZw-vBPKV3cOwL!o$0aHemHNEoCCAa*CD&H=)y&7^(H*}( zDZaE$d^@G1Br)|IgQKRA!fvOC(r)N*`LJw+WCT_OSA-{sGI%N62FEpFFv~FvkrF~F z^pOfD!6;&Fup`8>3MQm6786ij2RS0+vRE4E`{7zLMp}=?gOO}L3xGdgG%lh=RspY|Sa>ve{w@K6}4twjpswDICAX`A`dM_ZcWhU1ss)~5_R6}+0?=3k?xb8ZUm1ATGTqE$}RBL8PG6p&v zc9_n|nkz#$Zwqwc@qUc5JdV$4>ajI{;It1h5-oKTfj|lsdvO zNi~hv6^2OeRRPcLmZ zl_c&0B1p`zJn78mR{XR1aq{LY|3O@4eAxYMU2<7gymZey&%vJTxc0*>L9#rdA0a`R z&d0&0u~SgL3V^Ws>MqF_oh|I|yu`ccZeS>;4e2NOjU7r*i#1#X{)DnbHmgS9B#wz<(y8pRgQ?%7_0IH~CUl55h zkjnHsO7tP1(5#?q;mtzW!$KusCI0>;vr;^gp^S_f!#Wl(khSy#E-$+j*CSzTrow5} z2So^ekOGmNPbNYZoO*hnM^pTLJ->5(XU<zAr8PWTzWcgMaiRCK<`2$LnlJJ zfO$hJK|i8l!d(*$-s*vT`#}ym#h4C4V1#Pgy>5@jBj86jb^m5Q3BpkKwV;#^KL8=< zh-r*&&@t~AU_J4Z9>hb#N~NtsXRIq6Zz8^gZVnSmLue+UGqO^nZd4q{0;o}wD-dA&TAalS zid#rdxP(lok)UWcHjV((9cyuq#@a5Hxavxua;J&^Ic>=M;?H?w=Ma1INr&Xt;2r$BdGbgYmM2 znxdM4Wmf(AYhWJf2H;5g1Ge#`&S4J$rBSM`K~=ivWv3jK)jUtc7j^f@w$^EN^?KC# z#?CyVBh+W^pz{->zf9A!s9T2WZYXf1i=4`oE^Cp_TPdH?QHsgVDni(sl@?pqv8ntn zo(amCES4<_{b|a`Pk&oUieV}TW^Fu)0PT@%Jzu7X=LuS(56L*un5weVYeMeig0pgO zX+AzP?&;XR3Sd0>uI!FZOC`gHIEe(|cY|GK#t~+I>V65hW*qk=FYBE%PUBcJpPP_F z9`Q$TE_wtiEBFw_F?82vKABHs=mm;P-%n@PwO14-;K|m$tmYsubD9THfwj5N@A&mt z6xY9-1eh;z4vIL{`EC+tjDAS zm<_jQe(V>zzdLzzw|iKZ4KX0ae?swCkMs3}vZRlFRfQ3fE~QS?p!lSu;MNPoE%rgP zy!t5OkYDu^K&^ah*ZEWMz{i7X;0$Maf(_{Se@X$+A^J{KN^=&0Nn<8c) z+DE?FDgn&)q5x8gLEe|UQ$q3fTXc~9$153YI1Eq)&jLdL+FFOX{SK#fy2Ayd;DKi2m{y zfxy=UkLe&kW1S?xs*yS*x=4e2lU?&$%4IgwHUlJ=PY_|?KN9(cdQs)&ch>kryx-X9 zbMikxKM$5IoV5J)pU*ap>b<&-QK$=&{E@kj6I*Nrp5X$VefXI6&@OKg03V7AJj3eO zQEBO@`Xto71KrQ)UAsKQOX1~1^IQKys=$z6OS<38V1a*&5BUN3A6Uil-%#ab?rLjl z=OS!x>tJK*V*kC_8acZ-85+C%{Lf!ROl>S}EnQ5V{u9eob(L|xeY51;IADGuQ!p%G&%l(se7e3m8mHLVD#(=nXfU1yLF$2 zsDC2W;pH@V&4kANI60nn?iJW{e)N4>P7~mVh#wZ`7IW2EW-2k(8hhimJ8rKq78|3( zYpSsrum)x#;5~l$`7=-O4}_`+8U>gR6FuX85P2y2Jxn(}BO&8)5W?8U^=`1B&^8>1 zWjB?>KtJXT_V_j(03_|EHGsxrxgT@}%R}BkiIBztrwn!~!WG{fmNwD@;tKN_Q8ExY zMi#NNu;Kh;qpZV(NnJ~BhO<>wORj5l9h41QdkQgDlm#MZw9@@5(@iUz+$e*4C*GiW zz}f4hH^Y6t0ZfS7t9}a9oKB03h0BUTU41GsiK?y|UAD{y@MLe@s5Y{L{a3b#s|H&*~ue_>ys?3|3UBx)| zmQ07$(KJmeBtF*q4!NsGV~0=x%z>KvbRy%6H<$MxqGPZoiyDoomuV^fHY>9g1tQQ? zS@UkixzR`*z+z()-CNDXfmgYuvoTyEC!$u36kl#tiv}7;@@niy(P~j;+oZEauiK}c zj__hbq$K{xnAuPEk-zMfqH}kVa42$pPEXR`St$oOw$eivSEYG1Y`^_Jw#15=_wiMYvVnw%~QeQG&eFgMDl0*af z>H7;jGrfGfG>)u+Wp7O^AE|+M-++!?jXvvKZ0TH(S=a8|rvid2{w(jx8~XI^U80di zoCF5-LC z=n)e9L1AkKgoWhee}d--<3OS)ZN_T}n@d$`6Wn2<{Ok$64CCC<>J|HnXVk^v9G!%$ zkb(=#%jYZZ^b5d-pXQrA`6Up?v6DqmY5svV0RSh)e+P#43exhx6L*he40JPFpA`ba zEp&5Hgzzdy2|^u4blsy3VU#hHQ3iGX6UB=sp745&_2wf9k{z;jQBxFm$84ID;6bb- zd`!e@I&B(v-pFUoQ_|%bl<#a&a(AtqDUDE+Mc=^Im2^v~EP0~1^eD?W#M!aeHmDyz z0k}7MPDoY~Gf`xSkB0R9qysN@6AT{5!slo!L~pOF%dfDWSOW#P}E1q^_E8e|{JACjZZZp5ecs2Eg!Ni1I(M z=Klv{|4*lCCj3nWv06W~EFT+T7kn0FlvM&S+^=zwe3V5faaI zRuP6VLLMRa;x27PyEV;Q;tjw(D~NlM6H!)jr4yiK>qQ#6w{(!ie$P(R?V(OJ|q=KwQ^ES!)zU)X=T0nXm}s#7y!JDJ|_Hxl*J4n!-PVM2ITo?T#tM3yee= z2^kU32=Aa^7{s0R+d>6VsMw$wA3&d!H`$Dq=Yak1w#23H?~dQyPfmVcFRsA(k!Res z$L*DSTFA~4j-0;-wjGGT0*V888^I$rm=y*q1K9Lxj3o_bs{eqYPe~Q6`z%#IX4;i? z;=IPLcd*g(qmO5$P&8H=Ep_NHlBv?uPC`GtKF7PQJLX)&=WIUm1od8ovekVxn}_8B zf&jk}dk!yFfde;8v16^b2=298zsgrn6EfEx!2Q-Y9X*ngIw#plu7w#KgvOjt@W_=2 zTn}CU7h!K16<4%n3y0ty+}+(JxVyW%yL&h|g*yaycXxMp2wpe@2o?wg zLO$;8zPI1^`n`U8j2h#NsvlLS*4b;%WphGJ{jE%;ksF4GOGGT0145hULUm8WMZTjG za4LmRD~X0gg0$d?3tgF|d>wOqPwsRII%K+umNcgr)|7La+oV)#58%2q@kJZ&rryt2?8yI)h#<7{DW)#h+Y3MWTgL4 zO|gr7Vo7_^*Fl*w%Ua6k!1_ZW8v zh?QmA<>kC|ET``XaNK+`V>iMpl?0 zvys}hc2Ji_J0Ec z+BWZ}zPpR5x#hn$?*9e_JXNOUAVojbOq#FPR7cg1NV~xuHK@9vB}9MH3KO|OSH0oZ zjkFjN8oC^*zdHaVM?*=#(oup%FzD=&KhEo_^0MAu{r-Lc7n_=|kLAUEA%Nxnj)JR2 zbOh$5EjTLrxIT)qX1LNYmiTM49SNGnW-oii5f(|ap64WCJHdzj=ZoH?h0hvShbJ>e zB@1X#b=Z~>ov7rN5}i+TTuKxp|sgT_{;ay+qOcof#8JyXt9WiK`m2pFB>Fg(-8WLmbMh}KL6DixDY=rZEJ zvo!3P2+jhF>d{-mEDizxWR2^3%l6@V_Y(L22T$>D>wF;kznkYjd_$Vn%Re+Q-<*uP zHynIR9LUt>#0j}NL{}_d2aQI8gsN|$ z;~JO8iOMeXd-#m42#R- zjE0wDhz0h&jE446BB#B;PX~O2?z-m#dsghbF2-vH%;2g@?3?kbW<$d=C$xQiD+{Cx zh^{^OI&8OSkN7eNIUKlDpDrUew$AV(pnm?B*Y{4vs5#Eb;`XGZpr!QL)8yZJp^DI& zY`nLiRD+^=<@pk*Ba0*|ORCQ`CQFTKe%5_kMw9J9sq|w7JoWlCj5LHOds$q_(-T%u zD;GL$>}hSICq$a2b#(b0&7G!_U4`%l?$o8%r7x5aJVwc>A0s}yDFxFVRtrSX1KbRs zy#muJcp&D~!#QJ;eEukNWks^R4#51Zq^RN4ku*+*Ka6ASA3O31IKB?ar6ruEVcf)Dm z{B5)$S!9EnK&J#yhd*q#NKXPF@-X?~vGl)p|iEE3%w4J0Hf{gS*|aSG$~v=hA-o*qc0NZ_fd7e6qUKQr7Q;G# zu&}4}#3lqJ3J{W?1bs1B%a>$8Hu9msQE?5F*s)Ie6UO-~rmC(Jvm+K$DX|K>mQbvh z;Pz{IkT3G?C#HR$&}+_~I^aGlfwRoeHvz?^@TvPimMLBqQpLCU`D|l{R2re4ZO0NS{0dXJQSvcoW0qzIErgsSf2>SPNc;sjCV3mR( z6&a!-a{7JUxk#QB$nvL7V{FLv_%O=H4w3t{4ZB;I&I;MNsQh|gb$}oLHBHsc1Ll1H zaQA9V*1?;HHcQQ#Um$XgPZ3XSCH$k+NOaPkWEirXua6oPKlwIQSrETjaIig)=t#>N zrpMFv$dI}^o>92KW(Cs=t)Y5ZRS;9q6f{+k?JX42_0bBNZj^pjIRK)$UrgdYVeN|5 z3WFFLuuW6f4?C-^0*dBq?0RHppRw_8x!1s)+=oPVB^*Cny|Bm6Ny~49ghpYX2uTjm zg@)Ek?gfPgub+?r^cS4Ik-aDEzu+o3Fh3ilFsSrNwPtwUjKX#6rkXcm@15}Q53~1Q z*nfb(c36guFYg&b+W*GJ_D_fYulSqc-#%U3)XBou!qmf3+}GT}@;_0rt+SKN`}+?6 zinqg64dp++OMa%1*k_m8TEQ>ELgTNzeJIW-l}QYXE!bV#iHVJEZi-hGuBIt0*nLr5 zG~_}e`F@<^KcAE7f4VzgzwZ43g&yw+U`%Cn(4*D=*3|GmxV|NYKCN40N)=HfQl!_7 zL93Z9c)jSJTHdX+C86eh(P(6DD61Gx@ZGfHETSi<==X{MyV_^P&zuiZSE8g}s9QNt zKho5dL*n^FY+D;A5j?_1$09voT|^i4CZ#v7&^Xe5;(Z+l@U<~yx=ckrc2fUx2L!)+ z({?@ZMPqdn#gpeVTjp52o_nIiq%d2aaq8hH3n3rSKzp5%IMPZQD!0CK#UD@16V)mu z1W>qeW)JPfA*2Fjtu7&7N__4FD0k=3Iaq>RbDN2*26Om`Dei}YMS|G`(WYYwbaHi3 zeL`ieUXY_+LU4zF>)dGL#-yM>1J3_2QUM8PlFa*=i}>GahW~Zye)#ZT0R7)K`Uf8W zznCcQU6TOg6zy$I&$jd-OMI+s2!^n3;;wMamJ~z)F&JkK2h$xw4VW_3G7aigU)Fsn zcT7f4F6+!TVpEipv(0U5LC$a(5B`bd=vT@_)qBk_sc!V5!q=SaFpk~9vNtFEcbnty zkN)~Uze*85483#Uz4Y)Nj_R_^`Q9?Xv&cO4g=te6q~1Wn3#Z;>g^|cUHHPKOJXMC} z%la4&%~QRK4Y5!eBm*wGLj>r7O#3`SGo!G0kLII+kyrtc%Ct|^O0>MA zZso29qVe95tQ96-eJOeZqK>6*>7ZE-4-w`L1EF!=8!PL!~6{7>oxE94)+!Nd28jdJEsLf658zKWs*L3zo)uO<8>lS7r~; z8|=UR8p3m%^Dcm+6Qn{jfK}s+#Y&^I*QP935aG$SL$JHns~odcP2J4zz>)TY%F8n; zw^iM*V|7hom3!tOB2UhK-)5}^$=$rIT>23CZs{EC1VC^No@rE@C!00A&*K+F6PJ=7 zT+O|j^SszQGU1tOx>T_;NvaV13X>J)A>24C$d}} z&AD?IS|_70vH0qT%AvH=!Pj)EF>0Mq<%_vs>Q6bJJXRJw*|XS}lx&a(50G$erx#j@ zva$+}fO32{KD@s4pBDGsy2XKdac!xV^=>c{NPP4^zP7UctY5mg+{-s> zF}%H^3bVayMHBgR9?~)G5mWUnqt+JC)G^?r0gYLU7EZK2C$7G|sR~y$^lmODasS&A zJEw+BlYV)qu)8qz6{>ZmxbJB5_neUmTXyu0jZCLKh&ME9T2k6C*$0yRE&5z(8>!r9 zPwC)a%^~mPsj!_^AL%(;wK=w0LkWremwwDu@uA5jKII+L8(pHuzJ@tf*2V=2{Em0(>h)C6og!RePh{v#oTGa$}dtopaLeMyZ|D zS~pw8kA}19dmt;Bl+@LvIy|>d#9f(&M=4%Xx|j-_(93rk+Ho;wp-MLn$!l%(FOD0q zWA8VSrCn3jQnWc0ABFAjkosdZk$Ss+0t9mJ13<%NHACT-?U+@%pM`P=6x6Ryp9@$d zD&6&vc1)c$Yb{`ThJ+td$V|q%m1~238C)SP(8fA{hPlVs64)7Grt4uUL(A3=s8raf zWW4R#_b3ZEvw8Gj+Oemxee|GEl4ib&9ape^EW%I95$?ubhvYsMUDY+@qxg)1KQYtK9K6m8781F)67g{n5RT)Wx{wCMN97Ug!t&6+H$ z%N#vRd2RC3uG*r99SM}OZ95z(u8G$dOBEm<;F!CRmK?+CiYx;L3(IFRNc{Y4^XpW? zj0;xTH*tJ;IK!yOZVBy_7v%Ikp!AnOI#-v}f|gW!Lun(3pY){Rb0qk~c3lcG#IIM=nor8u zVai|^bH)Q-EG&qG;m-)NxFfj`%BEk4u_$a@F2X^cN+3&}{44^JR8U;`kW({hF%K682!C4~<%WPI!+90~_@Arywq6?uI? zFKy^6Ou`w~QHs$#m~4FIS*pc@8w{l`%2Ev__#}bo>O^h_?KCp06PmUX2{&howZpQ$ z1now|Y$j+VjhWGq`Qz4u2Vh)z#{OOulNTed(cY}1;vce^Iy_aZy$hdo5m%@;eGMhS) zN^=yrLF5iCFw!5Wt$@@)L>_75(5d5vbn)?rs>X-}b`^U5E?1c4=P(%Xv4|4102Oxv zUm6I>2+zT-Xdc!RX;F2isKFTo8rh%^$~q$+e1XmyEmhc&V^kJ7D(ozcji?}3cBT_w z2dD9TG|NumPxIXI0VMq^0tbjR+PlwSEm;1vRd~>LEe0ETmAAezq%r*R z_V#CFMB~k9n`-Jf?d(NF!xPpiG-)!j@nGKg{z_8sd_pg+7VqZ3&~im5C^PUEt5bBY z=q#zg$Yh}tE>GUz>uu9<93$mx(>=D|8}JWk@wt9}?fyL%O7P$G82pdHJ4xz9p%Xptp>pgJ?0S(jy1LA ziAB0a8TJ-~jRyS@4&9NhTdB#}YAw`x{RnkRhBHu_2%9l6%OcmRwy^pDz0}oZVDL{y znU=mkzjWV!V&wble`PWG|7VJb-#`72Op*9MdN7ns9sh+`o&U2kl%{R3Verl|jKVQY z6hT0o5|k-0gE%uVpb)EhVNw(Z1d$CTLMiZ(Z5n_-VnlZGdQ%6ltZ@q2+10EGk=G`y z2~}uqfgU~X<&Kr@_Yb*^dDcY!!v`=zq>WAdCzb7=lA~FINgfmaDF{~ z$3tR1dvW92KHG6L+dfNi*WI@aJntAM8$FP8KN3BV(o|{49^EM8#vC4SmU)T_%a`k* z+*gDbPPuW1cb0t$4{N9Dq1ewr52V;Pgzsk>;v+QeNZaQ|jb0dC82)6wfE$755uxm( zf)rkuKsM&m4a*+y+wd`zIJF^EEg+R_(j{b{a(Eo~$Ik*JSll9{qk2L}VxPr%m+jCr zkbeV0q+o>wKglpYO3OPF9g?(Q>NykA4pJ`?{V)jj!nFYx3)f5r=a8KeS8Y@$P8^BE zv?J6sb^nmPFGQJ5VC2R(LK?4+(VP%<`U!h((g#0b!Ux8jy)SNl$_LvxCMWIIdhVIU;Z9rW(Ag((pM(7^o@Me0 zRtb^!m8Y#O-YwYO!%5^GZCGmG=fofoogKG@A1mlooaQ9PTl9w zxomGcWNSrB> zB8+M+oNCV{4Qh#mK36{w+Bh?-k149>K|Qr*OF zNT7dIwS4}hldBr5aZV$n)!I%a+lgr4RNWz@IsJyrMGq1DUTn+}2+DUZ$RJ#!y5xBZt+~eD_7iTTe9u_CiO-M-t9 z&9KC8X3Nx&!ui$ybB(6S`p6&iOQI7h#P!|Zj~*wVwH4-u>Po7znxs7K(Pam`qk{Ii#VLJGu4_2DJta&l8 ziJr~xQ#U1xALoRK=qgdhv8>u_sw;Qw1h_FdzVJPKlTG&fnVPJy!K7DG?(dO}s3ymu z9Q8GKiILj`qqksf*?k!TRoKDiXumxE`ODmE7wJ={#1rN$cX5!Eh-vLtiRF7v^rtLa zz)++kzGgl9F=6wO>~1D&X_Kx6in_6UjR9#pRaZw0rhs$%s0&K}tX4LG#@==Cv&<3h z!5=&gDVoOn6L<$d~aVA^~(dToLrx15!1tw7S8mXA%cTZ2`XZwP8te#MZ`HTdq8(=KK5VcbW;gBobazw z=LMn70^W4}<*!gaF%sH!xBF=39hJZl?dYbN9^+ zSYL}WV|=gG7X!<}CQ_`4L0JP`KX2MQb(yrMZMAD4lrRx@kT~<0i>u%mq=b% zLxH$dA2i!rY$cF4t6WJ*d*AGh?m+}Q|b@KPlzfshoqUc1NM6tC`~cugC**RZ_;XNbPv|Lo?BedeeA_KtPBQT`{) z@L%PN_sj|KkN%SXJ-cP7B#$s8f;nBCZ?^*%VkF8l2hqhC9WEy7{bTH0yeyWZwDw@b zz~=)hRT~zM6`Gign9=n(M$o{>_{8`(X%QW!lx>@7a9&nYiL<=($h<{?f6pECAf8f2+|M6b_SNl1+ zJ3Ck^zXMX!e;DyUJ8>P?emN6Z;=*cp@ zh)p3L9Bc$a4>5=6{jZVH0jL9<5mb=i6sCWa@u!dQi6U@pP+`~Zr}D(*iY4{=^Zt+i z*LRcMfBoN}$t2_iRvUYIUO7i&Fj-W46mAhlt1;_Udz5ZrMzb-URWHkYvJZ%&z3Fy~ zeew?=qfzOEik`k6U_>j^2^Bx(93V!s(DfBPWgP$n(av;z#ZP$$u+b!RfkjW52RPBj zbb-ZBxd*7x!gOy%PuT|~(EwdKKdWQ78IdHcB&aBbTmh}{tO%q|N!vx|N1KdEKTWyAj5xSMmU(yN;-xwI-)%idt` zum)1dyV{hr8CvDsv&s2ZTWmj{iOyPJqA6DE%`sPGX@nKgk=Mmlje!_#tC+OYGUBU1 zU(4iI>Cbf`Gth+`W|Nl&kU2Sw7fM4JE;!}UOlKQ^CgWqC7c7L3V_>q=MueJC(z=Nj z!b1PRyhU&?9T!_@5v&I#YVGaQoAqkdGXX2rb)T=fr)CjYC1Q#|PF)L(>sjW<-^q7z z=VX8d*M6H>&W&U;njG}C0X;{09XAJO#ly02x(68wb@oX4V%;saTG<*zuBUY|H}V)y zRBL4YB^Zy>Ba2OqS_L{L&l%RN>F<8_p8i^P_1=;{zo%u{GtMb?B9{wjjxM}BwgL0E zJ)f!Nffm6N32q;#=1A^J=P5to}<;z z57Vyqx@xSoZ*xO0`h2?5KZ; z@Rf&y6Hk6~$(E_d%zR_hFSgw?tpy-jb=z#~SZ6MCq`AP%1;g}=UUOTm;}9(Wa0&Nf z{&6j1e*h1d+e?5A3*0acQ&*T!f!04-ixMn)1Z0_B^MLu_pRVNj;|@4 zU@k<&%$^UIz?Kh{z>$xn%$omEnLQsNe?9;|U@+^&a3X*@0PSA)W^_h>U;gWSg#65w zNyal_N6%OSh+n&A0C9$9K=A}Kfav~H&qRQLfwd4y+t-BfdJ7?#zC7PgeM!C_@~nmM z>sXy5yxB>h_htep198Q9@=4ic!n@gB55 z(6h)B{Yl`f@mRoz8V=_mXA>h-oUgwkSuA%^`*M7Zed0S&`@(%mp72j70|?w%0$AKp zUrF70URm8g8kxlpbIh=Pvxpz!cE>TWj33bEh}`%8%D=CB@~J1c6LJn-ap*Mgc#W87ZONaq5mA*GCz=A{}ST+Bm9qOn)Cs&Na?+K0sp^uGQRt) ze=WTCi~o#D-<{IChf=dNwfI-*WH!JzKtmE8@ciZPY(x#PtcwJZ!IztSr6dK3LQ06k zeVTxVAAv|!NQRJdW8>Jrky6#@k@Ox!OyIhR#+jgGWZpga0v`0M{nc1~(fFfo{QHUa z>hKKI8cN=Mdt3 zR=F#lqT~WE9=jFy)VcL5&-A(VYrZ0<<>vu$bNU{;RrlPvu%5(AzB;G5e8%PX4}5zC z_i=o};-}W%UmbGui|_3{`V>#szrXV3{*^lIUw!tz@L@zP}3P?$zJ#@B!kdz@lUlbJ>Mi z1hdt5vp(iJFOWy%gbQ4TIFcJ^5GCpSlQ^MSg*cKsT)Q}uH;6HE!V5$giM#)ag2*gY zJk|@uhQhW+7VidfMPb_|i}wN|^%7NfVFZO*l;$3t zkxECZA-N#DAdXbRg>MiK>J;69tRn@?90E5;0{@61AbBG&NC1_O_L}xikhzEPDskhE zUSoa*WzBEO{DM5lr1W*X?X&8{GZYVhA&|^O?lTUr>A*IK9>M_s{_G)FlU8^Pi9Kdv zU^jX~IrA5$GkJ&+Ra14Og6EZV#9FCpaDpKr+{Fa-MHF*ojRz&2 zDI@P8_O?*46Ah#FG(54#5FOC!_C_Q9*ckEW(nkEWWug1UlCN3tP{ zRKx{eG?7$cLAD_eOc-(x&y8UKh`5jqT0&`~8OXaZ3_3!2f7+H`P`sfDfrLUu6A-(h z99)K)|I{OLLp~@E#ezXd?h_8y3>k)UMtcnh8;1OX6C!>33U-Dv{`9&Nwe~q5tPoNS z?}hR6Q-I`+F;r981E-UqzNicKpjSvFJTa9|EEqoI6i$fjDHZHa<6wdeK-t69Q9Lod ze|0;?Ydn}ZL=ftlW;kFi4GHTq)#9cZ7Oqb=V)gPDR07Sg?WINtESwPKz%JJ%LI_Vv zVI1FPC7uOA3fTxZ?)Ll+)Muy^xOg(&bQAhPSEz8Paf}=h6$#dif@l-8!AThBQyhX< z#=y=&?;9qVz3FjWfgu_Izb8B2;^qsdxv;Ur4XiKspehs%dMe(l@`}KKGS8EEK+Yh^ z$uNB4AO+=KG3OOq-#9qoULENTs(%Trd?$|-n3%x*q;A0TwA`}-p1+AfdShhsiN_UC zb~i5AL=1o!L=WwssAqkW52zkAF0*C1=L{t&@-!xUHaf<^edPj#2Ep|yB9XQ~^5J(W z%bd~r+ijh%7vgu1%h78{ZCIcDS6+k$VJW90d%l55&|Z{5gXID{;QZUqFn@FAIiL1? zlLz~cA_n_UltKY<@xEhoBcr`3;hV^(kTH6;+x0NqGcpEwlpu zTdKzr^5NsyJu*;8DqMJ1jZ}Dh4>!jX7CO()I&_do`Q7slEkdR&OSBcUsuaD%HG}Nr zMl|BIO6kb8r2-ltv#9;^j9S-|m z<6{jIS;c8KJTvFWWI!KTkA(gd51tv`YI-s9J28wGeI5|0BdS2~BolHRVMF<-XyaLw zX*U~?MK*yxDi@;3>CMH$CJ~fXo;1T$ArZC#g%x2#rN@x(0OB@Vh##k2b4wr#SwTvQ z5DUm78?r$w!eCNxAk`{Q!L`1*jo{7jXo$)>b*(331`i{-lCkZG-5G|_fD-LNG0&?J z9bwjZW&ne%3tj)prwr9AHzRWp!;_e}LDVB-9eD>#J;@MnNGH@Z-UrANi^R-if}i#G zu;OnaBBB%5h}^Y#tX0A9gHR4rA7RMt@%9%bi$A&3q{=Z`v+{b2joS8k_R01=?8|yy zT`}2z+&@Pe+nk*3YhWS#*&43(J2XX3|07Y{RUcr&5s!~ggk_VjvAed2SBDJyr%%gY zIi}umwPb0vub`gp5>=_4iPpc0K>40095yjYxxf)CIg*BS{<}e^S`$9@)g=!xv2<xg+?eQl>?9eYWAI|r^E z>KKqd5AvKr3`=~GD(N(9;R71D7EbIlnYj!ZT%$%xeP}x|($kiPO8XlGq5U(yLMy4d z%THH+=5BAfwzwJC8cLBns$)=r0|@3gY0{SQ)Po~)q9n<$tf-2%)>smnk}opkjl?Sy zMi}``rDL}wE{S=saHzpz@MFc4;hoZ~v%to;ugeGek+8W7WAVI=b5&3H`mf6`3Vum@ z>x`0knb|tWmKqtq=O*H2voJ`h46*fjqsEHfzxok{eIJf`m9D>+^^XlF=R9Q z!uqLA!Q|vG_fL?#=9}?MLO7tm_re&$eL6byNL@DK0ETH5}hmIw| zYfX;$M^y3`^MYDDOBBURxHj*n(ET9cuvi-9e1Qare0Gx4Ih!HZU>CNzTvJUQ_&P22 zYClyXYFeshYYI#3(4!?^DoyuktF+Kr%`HVW7K_o*cl4L+Gg6M{MNRKy z4TUXTO=U$jDE>a@2xNlSV3 zr0>-@g&mcan$6l8^SW&HJ#BSajVB-NK5o8y_&hTac?F>zRoS@#`EH=P5>btyoj_%*imN! zv-n=pnbueF@Db_0Zwn9`-cldOFXeE5#9z8B!y0Y3K>YG}V(~NaN7yAsddZ%Z{b4LK z3`?UlvlNz zT{APZ_+iwBK8>pNX$VlWm$RF^%WX$BavZmbDCCW1;Qt5lvnlJd@nn(a>ZbyGLZ0*^)v8|Z+ zWR_ypBIw+xxAiZdso!E(aCdxM6b827kZp7uCI*C$3P>D zFF-Mn{P_>jnL2>~hc>T2!P~94zqGU_ZW|sxW^M~-TZe(#byhdhOxC+x*@q=M-khE=p@C|kHA-iiCR^Z<$E!|^yUBp+*!eO06{%Ux z&e0mQ%IQ0jgOwM*p^EXd9%;N%4tDMe%5RV*wUclp81PmkVoM{H01$dhT~3 z|E`@e`Qetm^^m5r8i|-}5KE93LW&RjHz!eF?KY1HmD32uTA6VE;lrxCRx`z#yW$#K z5LMi_^(esYd5G5(P8D5Ol-_HmDc=bljZt=%?;M?oU=1TjRgOfL{tZ;22yCHj*M8x=nz#riY@Q*tY1H)Y~7a~uH9%xt{DoJW&fK>V9VJf7PO(TWpN z_@COwYo#)~iSGJGSgl%=Y{KH*-ED;yu6g_?eib z;03?|kT2je*|Z_G=n^N?*+-?Q<>TdVh(;NP3#(dB8saO_a>iJg>J2Z2yUf2qcFvVx#piMxVP9A~cqn za+Sm3?^7M5h)sS6UF_dKbbe?$V%VsJ#3BJHnGz(-GX~S#qv@&p57&RAhNRAf>w$;} zYm$t7$%OI*wN>K0<@i4HpcOYzToxNR7(Y#HiN5qN{-Sh+UB>Go+Sk`m(Iv3V)`!g3 z13|}!aL0?1ae(oxJ!W&&=jR_7$?M0&5j?BEjdwq!W!X&`;bFcIsIr7&^m4JaD@Xt+ zlC5Ot_W^~gUtNm4&x_N>e}XP6xuz{Umv-$aNMz`%K*MSaOff7|ii4^`IXJNvyJtc# zGA$P_Q;#w;eqCWRgxy`SaEKbnCyu3BtJE+=hl^ml&<9Hk^0E1hG>J2v(MT3^T8&yo z#21iw(j5;G1X1>XrkZ_~b;O?PB`&6>vY;-_As4(u-ALu9^e8D|#mvWy_?Tc`c0khzbSM4P7J4vhRJoT zJ?qkfk%=keQEE;pN$~L6q*Oe=WU)eT3&qfGR~M_QD+eJJ7h-6A2xaXEURECYpkp-o zYvPVxx9eOPr@>>XeoF_SUNmMiIWICEuqFI+6-;P6*>3VV=WRfA3H4_AxZ@osH~k^+ z7mHthde$jhzTQrL4}6lJmiKTb0@IAC&RK42n*z3ur28?Zn^KI)r;pQnOgrOK-UcA1 z=}le}0x->B+u|Xhq0&PIQ3qCE3U{+tk$;_Bn_GVTel5*+c)A1_fI(X(pA2F!g}wEg zPest}UN;p+ep6mycA;VTX$G>Dt!+Y{xv|bv#^TXe;vVc;JXWl0mh!Z z*NHh*O_%7RVafL#-aEA3hDUh{&0$;3a^sm-hq$Y9FHHSStH(97cDEsS(9SjxPr)$je%<@`y)Hsp>%QH( zk4E3ZRv&Z(?N}TJ?LNQ;G~jSA|3qx;McBH&4zI5KQaT=f$#Q?w>k>X57h`Zd8PmY1 z{ASeA0w&eyw7QO@ueV33noe6m=8-m`sj8Kss?xuT$R-C|roC_#?BbKU_SdSIaWzVQ zL?GscS@5Kxfk?Y81ZJzr9puu+M~M|S5pm344t^5&gL6Y@$a+ZhRGLtGCGA9^ga(0r z_iD5!ACm0h0vUJv7ag|HD;3dJ3_Jm(KJxKEo0~489hU+b?`kJ$C?R^eO=RoNl*h(S z!bnb^m>fVkBxcR^0?p^!m4wdH>Wj!1?dvbU4)F}&V8LiQQ)A7?4$wJvxk^w7pHRWK zaU$yY@ReX`3d8|cxFH{2BI;pXR>-;FP8Vjz5JeT#Q&9?s2u&A=7V>#g0Vo2O=jOy; zzBjaL#F5!xx(D5XAjT zrVcjeCKNI~+^=_dm*~J6#Bm@Fn55rlXd~>Ctn4>MzCm4s5-$8Hd|#OUe7s<+Z&I1O zbRC12M;+BWFp?RN#pS`dG(ou9Bj_TU6UUB>c`~qndPlc*!zNR{pGD)$Vt^?u1g()C zoeBVM6v31@qq0LV6c^ZeR>(xbj9)Eg$sJr?zcGIH>f-XE+?7Tj-Z!;r&b$pBKL9^$ z^NXA?_Q>9{pBB$R%RGmnDwE!Kb|2^zD;M<3H$}uZMSRbL3x#8dB@x8X7bdd(>3Umy zN_OoTU_jnDABEEmGin4%O)%fSGRw!9?*fQ(@Ak~a+Y5ga>f}X13HnMG zeNTR4kwlKLs<2v1*nwrYBUi@PsxrL`(V3~|x2A;4hpfJwaTRtnORsL_ zC93r3g_|6xR^~;};qtoCF4`p0)TYZyxVY-Cjl$e{4$)CZW;E1&1TjzSGT_TN2muJk zh0Y(iW8fVh-)8y2b3L%D^+oGu8LDTDHw!TySFlD>L;Mc*xb%zOK3MCap@RYAd(;bX z@(T}9t3Ol5mXLo;*n`T{>0WWzQCx-Fw&d$S^cF$h2`xJMvjBSgQCWoP?FRMZ2D|Jo zn~Z{kYsZ&^w^2qkyi1+jr2}zkn-0jV)RqS0{nV@1+*lVZ0}GX^s3K@4w9|ebits^1 z?!zFgL5%25Sv|L!I%Z}hDN~6#Hf^G26fXY|Z{|5@dgNjWg#ug!aaQg3 zPm|vS2&QX;$%k!%nNAXi(X@itDiEF8F>Fo36jO=xnT&W$12}wD}vM-?CQ!UDU0Of#dK)NQ9${)*6JIVTC-8ndka{rfzgKX_;JYHjj| z;PsclZv4gV-msSNF5cS)%MGp4z`bF$!Qa0+yM8g(S8c6E(z?QJSK(6?muQ#%1b=%& zJV%R6r{vG+RyUT*vb(g1-CyGdUN2w7T4S7tr}`_KRzsy&bL_Hje4qhM?gFCM82T#@ zLB;%b@!z_`xt9k!H;TPtJ6w5ACl%wm=Z&r%XQs|cWM3wIx~B9|hgj+BH&D(soLmXe zxa$y;w~3TfCiXBxKb^{Y_M zgv zNU1uHT$q@5ZaAs??F!M|>D^)Y;4irO&$zeh?y%OyBd(;pgY5vrzcI6%d?zT}R(x&4 zn`emB%dT8~rjLyqsTo>6G07@?-soH7W$tGoa>`}J3#`h*%N=|}V>*%Xi)71ZR&<7m zRCH(^%6cL#!G=F9oK>^VB1qWArYd@elRtkH_WX0Bo7DVwgP&xGHUk02&gSovi=R}w z^T27-tLm#lsTnPRI(7HCR(i?O!{YB)Z2FIsDWPD>N4k=xd9s*4M0Xn*tZ`qUM=$li zj=#a<+@bF4Jrcd9|ADQu?!AGszCk*@fzjSKtQ#CRh@5A`cuul5IXO3b4uH!ZiEN?^ z;LRRv{w5$4vU-y=GcVoD@ucI$O)}4XAl-Z=kg+q2sDqjhOyupSe7k?udBo43+C3a^ zocATedG)Eg&D;i$2EYj?C7e(Ne9RwB*l+gzoIlUti&cNTdSKrt^EzXX70RM&@M=Mo z!lK^vststgUoRBu8OfEifUhk#}f% z{OMZ5=#U^4Xn&GAW_eXOrU*>GqZ8JC=GiNG9pI_v>laZx!Kq{pY|NTu(QJOU{4Nui z#uwqN?*80rNBp;zFVk7O{ElFSJg}HAp=io{1N>XMB~Ow^iR{9Vu3RHbH=M)ck8WNPo`HWZ|Pn*fPQyIfA!V= zwLO}EJkFobJ!!nYwG!2&_T91HeI)O_g6O?0uRv}0mRnvxTAze7FFpy#Lk0^kk=cM% z3@HRqHG?~fGX3h|TMWd=BFEUg>E z(rQ<2No>}9H{8wKa*YtZpq!$e_BF*Ba9arcE+-$Y1nnzy|^KnGkeUgiOhtkqka?x6J`4L8GW zfs=?w5cAr>n;5G?m}!R&s&ou;B8w6bc{qZ{3OkM9%01`9OuZg(@rQRvp{+bK;!N&` zRmAu)Wo$zsWX{|%W!L5kb^5$$C+?*{=Q!vTo0asydDKh zlI(Jk{08+8&8)712!4=8`PrtRfsC&m(5SE68dKLdiPu5U2frFLw&4{IjBa9fc<&MO zGQpG0bU!9T*CxXfwDsGsg#?TPel<7*T;bAgC?qR%lC+MW1$QLO4j2|L>UZO(i)(IU z??U%T3*L+$)a@n|p%SW%v|Zr>F%%2coaQJWP;gi@n3%s9d(|}p8-?9owB=+kUN+_O zlFz>e;oMb|kF;pJH)zAV~VC*lcq8gs9S@s_0pobnuSQJ_4u zT+EVCd?)yG8-4V%u8uE2n16Bp1T(KpVkt!BXe_9HzExHfQpRgiav))cTHo9e*Gee# zrFSbf(hEJZ!@D~Ko0>14*K}IQADL_fgNXfHf&pm=MssrH&UVOk%sf36kdqEYX}8auYA zY9E^D$$=9v^E7)H%{6g2L3-F&eMzU1_G)f_f(iIN?I1Kpz2yPytxV@|eScX967U5n zoAjLVN9z786xIEJCkNe;ijP~v7*Z+6G^6?5L}<-EwX{J)7p9zkx={8rG+!!=PWclU zsT7S_BMXdBisuQeV!xmp**uG9&&DA-<8t;;LE7$xm;;Y6#Qjg zI9LqUqabdcbtar-PbtYZQ73oLlzpO=$Kc6M@F##46x?m<(=0vJ6#Lp7Nv5ca`guy~ zYef!3>6Gq!BJbPxlp3Ov{V#&c29a%_oV!jbif0q641!Vs$VSL~E8PQ9baIxVv88p# zj7YcRr$2iTe0{`fse8q{*AmokxGApKDU_&Ra4)QET}d)h{awF*!DZf!{nECTzz)w~ zuyDepc?l&b-=(VVs$~sjB;JR~+jfvbx@Ug!Ab68bLxOW}VS3b3wBd!D1%RnF9N!>f zbIvSP-#HEe6AT7OOm#Rq+HJB44WnSPDf`6o9u)P|VODit3@bY4Ld9NW#58Y{s5|DT zG*pwgJ2a@Y1XqgrG43V1oz_X!PDQpH;&nKrvS0p7(w3ghD;2tyROsRnZpZp6slWl& z9~9IVf6Q}O%t{V_@Xnw&uHI+w9o3ZQHhORBYSn7!^C| z*d5zR2OZn)Oy2X}Ip@yIJ!d{teW?0WtJdECwby>u@5yt+Hop;r7HpCs@pdP~7lFKx zNPjKxwgnwZ2xDTwS7ig8Q^kZ+hYZx23{g5k!YV@u^<9J?sT?Sfo(xO1M#w3R3t&5; zgsLnu%H@W*oFief0noQpc|p-%H*RQj6kX~+RC9$i(6C|Roq)|O`bw%%2$yn0cdbTc z1Yr;{FgGx19ZT^TMeGSJ>SXdqok6r4LHHt4ozGQ4ot^CQ1gTbH1eMBzmU;6Q11WkL zo1q?AP{n~mqdkEPtxtpv&UxRShjThXGurm|WlNfq*j+-q0f*Wbm&s?Hpq6W18BX~r z@87jUz4Cm+=2E1*GAnbEwF1HmIB_KVTRCVr1<g}`{Ppz?c%I;R`Xjjdd%R(84680kaCn7DqQ1?2V^KSv( zI>Cpp{%5}N4Pp&{D5&*9qyEUl7i9`ejA*i4h+3a9)kipxNbJ1zA3U>eHQ z9cuCLh%=ioj}(}GGFgy0Hp$$|&LG|4B~j$Tg6pU64$!5q$cvWV1PklHW?q1GV$o5o zz#M?PN67%%hlZY*Dc0&ElvQdJpw#xo^5e-wjA(`B>?$!f6@Z|M8r(xHJQl8`P^sf# z6A)rMdT@~C^q{i2ak77?+WZu@d2xR}brg79dpwJHUP?ce>L;}0>hWszk7swK;Qp@1 zDAk2q3aEViUR5@@R9Uw)#-Vk1$JmWeLg6IaPwvM1~dF{ra zKA_#>wi~?RO6@@CSG|Jx{?G&GqU{hBLxae1im`;|Du*|PIY7dq8(2RiAImdo^ey-2E3nllm&c;79~ z;V~e;Ahg zLiATF0v^)|MjKqTi+!kSYw!ghf891rp`oQa&zu4oOd3TNU#@0#lO(vRxv_KoEk=f< z*=03jM22Z^`{Y5bzao3WPzrw%7l8G>7{caG(MKai7J%&DsjyzPjw71adKsB#*T+;nfLhRfvGg>joSLh`Y#HQpZmu5nCaIMhe0lR+()e~tlX#=S z&-wNAc2CP7xN7=<1*N{55s-JUZ;CH8Cg3RzvV=L@=#P|h2hfY`dYdfxIpAaX7$?f4 z=kt~G-9fPmnkki`mvDTyS1rNW>4FxoO!jiV&aBxkBY}fp2iR3xnlWdUk`}f2 zy$5<*ziSERI2(Sz`+^WV{Nc;5hG$Ou?Y{WbmCHE`l2@ZR4=-5I1ul$t1S#VH3uYew z&WSH$yD>n9rN;^$|GNV=0>0KL1#V6p-l${D7m9a=NZVt)bY5@6X+QMW9W(}*AFa3# z0;FCl+NaHzIO4%m*?=PEPgDDw0h4rK737~z$f*tZL;Mg{4Jkz5(@oGrqYYtcFoL5% zg~wj9*8Q!J72%QU-mUh$i0QA}zIpf?c|x*w?xUO>%66eauzrc}AQ|UHOl3ktgZC9qvA-l-Ia8 zRZyauH? zviSU(^jkEsA|S= zSpT&xNKS952r@9_*cN=k(8h?IhcIfj*^&4(D0zjS(rCJ$_5mTKq1KSyLU9F+iK3#;%oa)w#hIac<5WEM5`q@r#=uwzW0E+DmUUO6+TZ_M`{vEs$3JjLe4vN&DpX&-OsM>)d4hjoD@Fdh z+EdEj(alxG)!EF*j`Y925VQZJTK+9n|66^Zn*?xBm{&v_&i_$OgX@ON2&QyJIJP84 zPECYdNRArwGb98X)pxls*8Ny}uK8%>8uSdD{97dXutvsHLCkbT^jG1S>q!^y>EriY zfycL}=VzJ#Zfo+I>^7s_(nu}TR_D?0l+47;S;2wzylC{ly=eEnXf-ni+eSk|*E3oe z!y5phQNC(ULqV}Xn0@KyRi}rXj>9L16WsH-t++JwxzrP&r~!?vdKUily;~RZQ&;bN z_S)*~PQzP*o|7MkcX5Ih7>~Bm%L2pn`zEKgSb=-TZIVdA3aJI(MT^_hoRv=}lRv$V z(J}7_^!C}eZ@sAYTNAB8oEu9vHQVT?4BY@XZ-yYnTJH|vdwH26_d3(5q^{F%?|U_Y zi<6nijb4q6XCG`B*bntXs-s=N{V>(*3!e=>2Fp>{)TZL)HB~=Ykzu!s_i6p%4n0R8 zsvwM2%viEZqH+UtWHl-KBqwHwcVrp*i%OH(E0mWGJ~55)=kPq7U$Mas^F47oY#TtN zBNoq^(vZMJcNSTz?rViuL&CussA)VCQK?lljKXO1I|F0O5jJ;F^9Lk_&Jf zBp5wP+u(`_dE4;s9#mW8T1!kx#&{(ZmjK-Z&=^;P8()dO76GFtT)1Nrs)5nF`F9pS z-UHh_M!DebvYaUvaf>8qxp$CHMKJ)XwipTP^^^VK_X#ch_k*rH|Ffj{e@HUfPfYoL z`}prf^HY&?m={Fy%jR4iWV6a@N>Il6p0eA9bPG`9Yyv5ODwZUw&Wef;65TYF(rb6$ z=bri17fbS}pCFmQ=_s}?-h89kLWjxy2?51WS=%V2Q?KZMKpWcT%nfi5%IY(vmvw}m!B*qhAu z!T{31lFUo!8ST_@UZ-$J9`Jf>S_^0He*m4XA_L5qXxIXiR#zT%4OW|b1z>oaD{|ss zG`L8*4qohCP9&sne}qu zcj_~+>nDlhvh{K}Yd-Q%--u_aufUXRegTw5K!(~%aaZo|!$<7gu-XGZ${t(U1K)nV zb6x+s0NH2SN|dJgP|(DK@B}02Y$|)RZ;}W$<;QmR0Asersi(k4U>j|nQffT(YFcCE z1N{~Xa+xX*z2$0a^;lmLE~vhjI^yvHHi4Q~rjvXz!Jw0Gq*)kWm3l@p@C}>1Y9GLV z7oS2@*XxxKlC65q)G#<@gfC|MJp^NMqaV%Q*j@7#zXB!4&M(L}UUfGFVbx(&%k=&$ zdArWpEM2Vj&^4S7#Hio*rQa!DeCAryR87_A2`yjcbKmAvhQM}Il<#BiIt2+Bn8G#C z)(d)i3$R@sF=1>xV_zL3-MI4lLi7JIBmTID3PArv&$#@@6eRusrr_VX7OVVI9z+Pq zFSeI)psld5Z5O6Zme+I6lPN0$OKvd35wG547>LP8ut%jpl4!UB$k=hs4;k|6soJ&$JhZN2@ zRf8fgwmF2yQ2^Yvd6$tl)*eN-+uvf=H8wO9ty`rdP%oHBJ+i}}x;=)l%{wZ2z&J`! z2DX`+c_XdUHw0Q4^o(Qf@#6a^eG(1!!{cb9xPpl0>3R7)%WrhJs0UPObPha_G1aza z!;B;gYUpaJZ%mMGvTSc(wc#518uW5o{?Kb$gWyk()&brH;(U|o={66Aj-FFYCBZ(e zDbUNG02|$iS1**to$h z8NC(L4FCpVl#3VVGGCXZx&%MSNzoICvMC4$&i;XGgpLqg>HplCjQ(R@{J+x-dRGFa zMysaUCfeercDosr_L7j!A(+Uy?e}8F9j=mljPG)&J485gnGg{fTBrENeWtk{`NOCG zJWlz1F@BN`Eg>%HSKld@1h(l6@lju>Ips&as_|CtGNHRrZz)}OM=_vxR&KWiEu(eT zY{P>)P;&wbwu^(X!OPG(OSbL6xxl@^8(|L6JAvCxL62xTN{%fA0=<%aYjycM-$aB5 zsNt-kj8sKP>`k$ZvCN-J!WJWJEQbTaFm!=Lc)k2#I&3!{r~^dsm^&Eo)t&26d^q&O z@Hk5ihs|x7eG7($wP9{m4G7CDF*Z*`!&v03yUqZ?0q;AGZ4}*+Z`et2$=n2jX**B@ zmso##uOz|*jy>__x$}C@x$_1c+;~F)N1iwy$DR;9OV_jj{t@6|;_Nku{roj?t$x1) zmVfP@ne4GAxP+VU7mbk~&qt9ry%7n#b5!1{Jr%sAYlqTn?2CnK!i|Exo~P8f>1zvo zgKq$mFM%1Q=lKcZk5f`KR^vya%zIoHO*_S1#U{An$!TU8D)OJAp))Ak8P<9HX)qs> zImztI7+I{yVj2lPo3$&v0g`wEDyeC#X3O=u$&?rv+gasOKKa1m*a6fz_KhmLTZvi| z3p^Qph5R`-N_w8rgjl1}d3EOnQ)W{@Vm-hsa=m((ah{7*5&!rIRrB*LpS5fvL#)8q zN1QxzvDEpRf4ZcN)ZU0GpI3eR+hm9<$MY7#6DR2m+A;9RRgE-(Ja?t0n2e)dOvSW< z{B=cBh+kKzSvluEv19|2=E0d=^0MJ_mWJTRX%L!+>B#V#E~}byS)+aVw;n-=a{#8f zv`~xM44%I8(~yuZm$}leG3Z>jBMKUA=Qi;v869=g(2oVB?$_ZVpZA5|-L7uMD;O0` zWEHpX#5IwsXBaI|=q>J$m_wK~@i{$%hafPb)F)~Q(5&g@^kd0Kg$5m@@UycZ79G3R zO`cAvBNyQR!OYZ5-`3Ej757nDol%qBxlelpL5f->$Kdb?>_{B7L zf5on!YzVgw{k$XiScO1B8JFEiU2n$pQyA!z_p5qM0J7>wX)U$cntH;?|bLY4eq;HBt>(8->@~lsbX_7 zQ&N!lzg)}kU}g2vS=t&pc>pq^p``2@{%X>0Wh{B54>nUC;jnn`JdL@h=xF9W>ckR7ju*D#BSxE zniH9wIn#3$l&Hmz& zmg_zePqfV8)h#t1@5A}>j_`v zClBOUvqr242n(OrMZqw*nL_RH#L!YeeS_hNV+iUIvc=$Z2Gv%mAB@AXgljVf=@Uy) z@kYLTg8ESQf4e7}+=QibC#$Tm2t;C&TaNtW#X3?UuJIdiUBu#_z4$%t&xq!RCoVMM zHP3tBw7}GDj+_wd+`6MFvt2=RpjRE;^6$ZIndvgP^zkp1FFO)CGnnQ3-0%9 zLI~$q6zvWz(1XmBuu`(H03xyyK$E7ET7^GtUt)dHB+j725rbuk{|>Cd(-&nXl++i{ zC#49N^F@+ z-X9(a&+EZ5=Z9jdMRH~g9w_`nQghV|$@>I^m})qY>O>syqb{ITw-d-rAlGen(< z`pVOAxs2=oQ>Ow-p?<%{0ZZTw7EhZZuva zbcLcdVESv#6%I0kOA{J%pwuJXi{2@`@dco6=~ushaCyvLZF}Vvyi>?p?>8qX=s59b z4d=d?C1`#z;IDT~z$|%~!G-B?|FhFBBB#{~Z0QZ>YCQm{SzkH}j(DHkmExH(QV_*S zw*Jg%${%qgVuOj8LhfhJI(J&8<*p+AhI%mxXIOc0Z>h_qysVNvgo|z$W+Bw!&*ME8 ze%5ArjptFq6e)J543})Ici8#e#T2i&zLr;Vgu@OW_srZZK-mLG5sgvqZRu3LZaR0* zx$QSJX*Q+=pTL3^$`O(F8Hl0~EGsY0=8*x>?2DGjhWEHklQW{H)F!7~IZY$j2F?+J zt!-f9;iJ;e!}>=f|Le+A#!F-k_bH>bx)Sfs*9`z!%}!y#lygtRUPkuDt^4ZE8TV`6 z;e=bx2@(D|!2O!%T0)m`L5zeiuOSOd3j1(Qnqz3W{Z$pFN^OvF%res0=QDZ zd!z{2ZK9G}Q%q)smdoBovozIlM>8FlWS#+?V7wbl)XDfPUZS}|KpFQ^+$jgBSaP^n z1_D7XE_z!V&XyuGjaYBhUmworTOCRk82|#laNj~3LNrKolCSc>R0eg38;F-gH&Cuy zB0NMpX?9y8?g2=iIcd;wED_8ROc5+tW(i3|+Zk}k3`~;DSSE~9+h*7(xT%gweIYSO zD9olgiL>DBKsg;HxBy^<^>E>RmbU&XuYxkz>efq) zZa4xfg|gMk(sVysl&IvqU(gYkIq@Qf?ay;DBF#IdqDAwKFC@>&w@DPAz|G{0TNV)t4xhTGSnooy3hhMdw`?Ys+6v*5D{AIO0z`L7l=y zAIRNB$^q2FaEFW7{s~fFvZ!)iT+kaQFlpq3o4`f)D^Q<0et@U5NMhZfWA|FET=mLr zpK-Si1>qA0s^7zGNgd?4U=tlrMS>G9-PSFn1~u{LgT$0*-JIvN3FA- znfa|(p0xOcdxyEK)>)0vWqXg!L(c$PADfT9X94c#oPRi2mOaGnotUkSQCv@ImP)0C zsX1}pc??U_tg%u7Re6LE!m}s3ZLQ!-585)Ng+46nC6zM0O9qLdGJg;L{>_li#?iCm zAsnZ_XgisW>q?tVuso8L#x~DWWB$XxgQzUcMRt{NU(eC2UCefWHq49|gp( z9R`rAD`b8A{hM#Fx5Ub$EFIx5q2D@n(>{5MSEHyQ4#q81`v*tmCu9$`gY?CW8WN0& za*&Nye~^pM5c1$5aXl}1?oinsi@&22Oi7 z;KQ*Z*&8N6536mtsP>3XJl=%ll;PM4DDDGvN^LdwtvKnPToCPq)I*yzE#S4v#MCxI z@t%r0jKKkN2Kuz_ap=MTnQyoLA>)p>UXwHIx3&w@^@W?+afs-a_xw}}16oFQ-0+qjnDlyqe( zR(Gipx)fa6#p(-g854lhsh8Yn9whTDE|=+4uaU>MV+T#db8}*wV&WW&cr0QfT zx%H4r0#dDNNbFkp{Akgb|9zl3JoKM z1lHpSa^K-z1Wfq~ss!9C*afmDV;ozX$~ROAJP+6nOo_%()|`k>`8^l}*abQk3Mb{2 zSl|GN1Q-DdC%L2U_Ktbg5K&;!q-CFfASg&M2nom@2%?QANEsM8xIC;ZG;hrg)o(W5 z*q|_$!W%eScmn|X>hkaVs@~smPvNPt}2GM&M2m*b6wpyrWW1cW7Z4_|FqNO`I*{>2%WvW^*(_AD)%&GuX%$M}Z2pX+eq?|P5bnB~q zKN$6}+?6((YBR9v$96;NKrrI&9k*RqCxw zaOV3e_Qz8y;$9YlOA~L4bKNS+e1r#oE}ZCYs#Q&?ovbVR%bqlA^V34;sPs9rl^e|a z0}WRpHDdsM##5@_@e4aOy-zS^RlmMB^C)K6N8VyzYaN#kX5Aal2^UJZGnb|0AQg5ulC#PJ# zGcy^@(eql(m8NpG_0!5n5*C;IA&zWs43BMRI<*2EG>0)1p=bM<T~$RA!%rNY%p|LkzB%b1?ekV>(Eu3h z;qn4_Cl|dD3x3xqd+$)dw3xi!RScX9;o-MT9eYsh!+2NMU^RY9lI%N{RhhL>w=FfD z(jJ0QQMNFh^-7~0&iWnFs7jl;8(cH?Tl3*AAkptlswjnqC+Ulk_87m!M%d8!IG1vb z3#qcZDANy38iPXR4ElpAXc_j1DgqtPyV@Oq7{<=~zc9rKxd_ArE#=BeAkN`t3{2`xLVWkn40U6-B_8%Y)yDAu@kuSgE}6h+@O8fiQ$a= zpCsSYe)Bud8VbJUkt6QG!?=$_Z3T^b;01uBR>sf<^n=5Mr$x!8FP>>imvorE<0uOt zK{FuAd~BnsAx1b-Kj9|2@8|a$$G+0IRez|(yO@4!s#ktkhDklENN;d6XWwjJ?8rqL zrmBQTP1mkmxb{Z`Pjc(^T=ynZ_RYkUYb^<>dTw|Z2;Qoi7;__sGidb>4pEHj zOr$sl+IN#jVtb7`jt-)Z2!HU~qXjd7Cot#{xpwy;w4?`|rYRl%S2#PKq1%P=1?Wn! z&_+0s_4U#n%Ia7dD`(9IhF$biPQOpnLD|ntE>Ye+)WUpN6e}R`Lkm3cq1Dl>mok`e z|LLprbICTg!F80{b?~H#inhM55NT_~x^TJiJTI|^XpVdIwoRP`4?_pWj@SlZXW@{c zLx3)%g4iwi_fD|iWugW8<~^EQ3;1XqImODV#7f?XW@fT1d$fen4M(7r^Jth@bmN~j z7Fqqdo&k7AAa1fBwv>s({q-Ll%wx)%8NQS!+IS@uk(@k|cv4S1FVyO_I% zLeqzQ5d{wL$Xna@255Q?XD(g9W#;&5V}e56{Ts;oPsBQYe{U+!O@r`@`#LkR#h_xp z$X^i*DmOIhMZ@SG?{eN8kLHRW=;HpVNtnL&_ud;(%(6ZS>b0o9++jNB+_HWM9K8_h z^@u_De8X+K@Z{KjLAhQH(;^MhdJB6(Z~3LxwM;X?u5@p6>%!VhbLmYBkmI~FbvYH% z8)hI5eDsukBQRDv-6yzwlZ`sYjR<2n#g6Oj2B&liPI(HzJrwD9j$k?T;{eUejtw02 z1yJBVzh+RqC+-Pc|9)q&$P3?jUO!tTCGYkRukM5K7w-I>ZWii}jvKrNpy=K?^lsG{ zBoLV9*c_X@O@M}cJ~4j*P-(m=+}s;<^8J~__W%8wP4wIH>@4y(!d>3Lf&N9A4!xtt zqk4Vtx{u2dZ(&z@J|Y7htshDyk9!RfSx^Rwn6Q;_Ey@6Sw}VDj?w5J~+l@_(<@vkL zt|-?466vH-08`!{g!l@&hR%ycTSq%~WWcdQ4i8_p+vZ`y*~6(bAQ1gr-_Mhps)@?) zNZD2TFbCNF_79jF660MO)#pPu;y=RNaQ^w=m2>+T3od z@IK-30JQ{$St}bIT4iW9QCrdqUpfFIjwxFcDjt^hynJMbbKhKQq|e@PEUX-B1dP*p zf6w+niZPMh(UO{ex0CU`9}XB{Bq)Z5sj4faI>dHYF$&nui8{qH zzB-xhw7Fz*J)TW%cUz~1=PX%LX5zPH))7|A^%?ggT{Ih$uLvp}W-6)uRw+_l}AenpZer zAbo-dnsTYYDc9(-Vh6c|XaUOYYO+#)3|GqKUHMMBSCP|D4PvL1I9Dq(TF`!l&RxQ_ zM~j|sncQYr=mJxPfefG)dCfZhtvzLdFrUYf6Ap(za{ArJ@?6+zfHq1WPceJ4fH`iT zB%|iglF79zd)P#}#G4hp!kVwd)vSzHuL4VfQIcdId{H!B0uDB|cm*)en}}a95A}>) z9hl*P=nHw`!#!h|fW8L}@-lm>41t@%l!TU;r^(wL%>pO7AvP22U|0qPSCl;}Y}iJW zK-3cvV$7qs@#r5SfI;Ch3O|ea6~`4CXRyjc`9XHVS7Z-U?V#-5>s=jK@- z$MLOs+wkU|4JiqKEC3MLl(QnVt}I#!$(k4U7$xbAiI0NO%H?zW7)~+Jg{_rLf`ZHz zi>fW}FZ^ww)hz0x2GS4h%tHhR5Na4?YMNyAxZMr=T5;Vv$y|!$`T_DahB$^5;>bkK z5U>YL@Va&79P9(PF+5}Kt+4_v|PH+A*+0j8} z`_&R%;}jLW$ELsiRQDTj{-kgJA2)QWGhbhOK5yuNpOxspuR{NI6C>|tYx{q#1-rbd*=yWX8b#1Hw5Jt~cWkC#Z_?6Z^Bnat@c+M{gf!RowRQ^SE(^c)LAM!{9-62W2 zBHQIMhs!tp_Wjlk>>H!w=b~_#y$qN+D*z9l$&fXGdPEg~{v}*6;7S9M40Po&SX4Jk z!RY4Bcy<+YBv7lI2r^`7)R4OJO&xQqk2dXwCYhXa)U)iGaQC5;OA`R3zzI`~M)9U8 zRLVB(zN9bzc|LJT8C_vynvFpNfc2NV|8a5(hiLt#xiPu>(@b;yt)traJJ)N-dmn7R zg&VcYpf~s<2ls(h`j^vpVvm6};LIlKUv)bVfsE(z=jI^vd4KiqBS!ug|N3uKmx|S2 zUZejE8mYX4g5ziV;GY(wb_oW5S)L3{-%ra|r#)PXL7@<{_Pu|JVhB{F zJ#wv3xincqszP^4Q1tyD^@RQ$H|pW zt@w)*Jr%63gyxjf?(xx_7n8AmT>q06=GwfQHEp(|2dzSSWf1v_TW_hTxOO< z6Kq!!ZTTn)_Q8vX=iHBhC+|F!4glY83z}UU8x488mV|cKv}OTYHbjp@L{%sWhuc-d_fc}F#3HS$k(*Bb>>HZghlKy|DPs+&>i9;lAPizF` zs2=(vX(bsHLn;r9_M+z)Xs6_yq-7?LjrOLcWvV1;4~~8dVC)s3qG4+fNfctGpQS|`b{7<#u<}WAXoOn6;z@LkvqsU^S$@LFD-9 zt6HaEsA#CUUEb@UU9Zo+L_+7NKtnIq6F1@{Y$*JVVH-*ohL`#(Bks_tCk|6-?izsN z58N~3E!jgib?XeHaO;dK?49S{>TQb?=v}(Tpv8B9?F@!=?Tlu^H;5cEFg$qz_AZYe zbJFnt2@7}287Pmtr992(lfp+wbJgu7;Wih0CMHn+0xsPO0%)Ts*-mm)_-MNhn9XLn z8Sx1hAD&byt5wQzk5d1`IfQ$%+W;`x_g0D1V#WJ08Y`1-SA1S*Vm@D*Bgzr3$6NWG zCFka@g(yeb>t!87zIx%2w~$6RRomTx+JKUWx1W{#Wzb_}cPY0tkuAE$bzJ)RvkmUu z4-~H>UY2LjJLqETyfK~jHTg*2y|`J)(6YVL&=Ba1dMjV>D&Zxgy@FwECI$TT2M)@X zW5F;alAr6wQE9tx-Q&Req?t7vyHyunJRGci(0Aa*2i2A3h07{G9@RWad)wW`A69j) zX19>TaIsjyCLLzuZar#ol?|6})6+O`anL}Ko>x_Tug-r6-j76X^c=Yx9&@=g{3F$`|&O| zikP_oJyyh3ib>vIYTbxg$-H9CH(D;vmNEL(Ya`*+)zeOH^b@vjd6|IgeoQ0F%irZ% zNTo^r1ivCXh7N{c$jE*OtxKiDg155B#2r~G0^L3JNY2wDPt-QI`6k`;(qURLqK8Ff zJLhedC>;!*;mIM>;w1hQXI4Mhp5IhdK=$M~-&d-Z*lXHfSBH}|+(-L_N0lC*w5maO z^m0NkMK{27bXD+LBBBEx7AaBQv270{_VwtSMt8-f*y|#-u({@{Nq5{*(q`-uHA^5+ z($W$c6^*-p8~%GgWmp^1YBNwYmp2v)i_CpD938CY*7qU7$>{-pt-zEXP%m2qOHWd*Rs4 zAP(#85Y96_!TBNuBN|50)=X}Ya);yz!^!|JMlWT65#z?ERFE_;3~0jc(2vtlLG}j2 zULkh+SUJuZ#;|&P39Kc)1y6LuC6L!dBQ_U4{C9-O^D$Nc$gJo$X9T^1URwK`>x*U? zp+c$!)@TFjjQ;Z**DhfTZF1z4gqo~=1e}nAS$vqWfl_Q&3-`+I_*o2?7oNplKbAIb z^k$Dp3 z^{EPV{0ho9KHlXg2K4EQHF8O(#(lvxSJ=c1jckjpo*}O92$l9ZBX%xFhL!b!FK)Op*O)e6@upum^>KOF4M_Tgzi`CbgX9~du7aO2q8%9!b3r18Q})DS+sn%3 zm!rtJ4`-k`^jZ(S3;q#j1=|5h7qW$whsA}eHXJv0> z`(K?;bsY^<4YWTH!sG}8bP-T$KnyxevLzXYU3g@&ph)l#<604C6mbdw%khM0Z_9&! zPb&~n4+!-8U4ppa4z8{Rd@)@7LgQxxwUJ^AN zsp@af5lAfcYh?P0r;tsB)ps}zRo}>Q>hAuKs9L6?U0jSO3=$N8^$(w2l(io|!}gY~ zO&Rt>yXM{7@@*|M&Y&nQ=^Aj86T?Z39HL{JV5mAAjiBh<1iV}pI=5?8M3!V@G-38p zE5&s#9TnK1TRSRwUGzp4k7>w`u6CQ*)5e_^G;c&+c%Gcg!C`QdP7xkg$es79t#H;S zM4bB`E$h>OQD%I=IMSZ546KhKOFMHp*W{8i{j4UJuQZ3-V@$|QLMjGI*03bK(&!|N zPZE%}z$`i>iz8hxvkt~xT6e)qLBy*d|1qi7>_{*!Zl8>GDk{b-`U_N9Q zKgOt0PUbrXcHqhmXt`xF0_SsP;YsY}+q6#(NX^8pnIkCxO?Z=qT08nk*wuz@hi)u# zC61Pvf(8-h8X?YN2c`&6N;R=wt6gg>%@vybGdzkqP~R*(>g6QbE@>4J5OOY%=7U>!RDDc`TO~4EJ{w-=!qjt>dpv*1OeI@Fd_pdk{aEXgCLOeR7(DT*b&Y>lt z$LGsV((mln8!C7gdtTucD|9%7qvL2Olf`SvUV7eqB+5$aSs`Zf==dW8Exfy9Dz*l&j}FF4Z*`2iDYP9 z#AvSxFWW>tw;i`-9kbQ$-(~CHvhP>m*qfPzAE~zscxry&tN5QO9JlDq#3WtQQYG?wq-F)8hRAlOKMUaR ziiQyYkg&F($X~*)Qr)=EQ_gBG{|KV+FNzGYYA`~yEooI4G0X259|)MCMxfJR;)#47 z>Nw)h*P?ij%MC0Sp1m@LdYv6`NK*K~Y!3PAabbK2I%=9;w4>&VdHq|nDU6&&gW&t6 zA~K}E_;P`oD@Oe(zqUlWBW-Aat-t^%?UhRl z;9Em~98d$E(i9eDWf8wVbGsoTXX0Ft*&DEN5GFm51lB0@%fTd#;Qq-dtMf$TMR_%o zeaXW2{S~9uI*e#);LUnYq1^22HYxDVMqtGz7fNSGSO8)8kod#$16NMT3Z2iD>|6#4 zRrw;+QV$stAWEwJBc#K!JYuBs#}(oyf91vSMlC*xdYNQRhOUvfC2Sa*dg228tmm*^ zq`c^|E%ud+T0jp*|Bdjk(+TbT*repAo2T4=TiVw`M!Varw_i z03T?|oN@>N1I^7!^BXw^hdrx(OictXkY0A-D>-vT^Ro2mk z@G!^ca^m^r&lA|T$-3yg==fkqBs?tH3j61M5oN!#qR66j#ECBPK(c2yBj9)Kt!to;$oypn#c8#~y(fb+=E zF0enHQ&5picylx>pc6BwLcJGJP%j!BiQ+r4U42mo1R5R&skw>{c{`vtkcXkWKGprk8*eR-|Cj(_j@k<#`#k!#{m0|)za~Kbk9aGa zIocX|DLR{(TX`zjn~7K&IsZSxe1MtKs{Q=uzro>lHrOVE(?8eX3aX50=*M>y@oT(%x zV&3YO@H1>7*EU{}c_gc}5V+2zg*r1^UUwcjwUUmu>| zKJI}82W=hwSSo&@CFKAr!ur_^$MC5xI^*Nt3TpF@m*vmgnf-Uhu7CTl{Yyml-xKJ+ zdI_4)p1Nx2e;k-Iy)rwC=cUq%JID6dhdCc*- zXg|z5pE^zdaQOiJa-wR$A3g+f3f=<63flzL1k(iT3Izt{4vRN?B?Vu`>Z>#0!Ro6q z@W|?GFo4MVrZ^DK`ldNB&HAP`@Qbzc`wn52v(;Ep1zDDMWkqw=T44o-T(gZeP=fBZ z9G1_zT|)1-5~j}rOuY6nKuL|Dd90|CuX4CZ-AA=rqWZd=JHBJEXvu5AJHBI~h@K|k zmf@p9*tQo4rqAG*-OC4ZiH?Q&RNgxR#;+eC=h@tA17?rv+1M+^@F}(iA;fqJ!K864 z!lb#D4qS%WI2S|LC+}UQ-l-ne)355&0lPr+1oryrFYK5Hu7hs@E{4cBcR#xqq*oAu z959_y+qoc@D4wmoT_Bg3Pv3jP!Q?&b{Xht>j7j3Wy^e9+-q2J)@-aPCw}BuAn4YrRxZHn2Ymc8e2+*Di z<30?9N3h>iGUK8Buen9MH)#zQ zrV5%KK`QxEYl7}V;PX>T&Da&V9$kN#TeGG)E*SfbZZq`b>4`JZX{c?lNQZU5OUGJ_ z!0`#pCoa@rNi92PDNe$5k;vS2_iNzvu{AD#5%9dQMCLm{N8JYu#8X^@NAy_jDY(Y_ zS_@pQLQ!>9Qwx(FVPW;bDDLFMN_l6(#GIQux=`5(M@~=#Xa8+yKfdEw4o|h zuzyNFOP~u4T0H#J8&zo?k8L!cBU-_Ug?nQ=r)j(oh_0&!bmxc;o60GScDc?*6%h$r z{FcolHXj72oZ9#?Ug^9gX0Ma$bF53}-1}BZ>u-3;Ph!duaozYdf1}z>j46;F2sB7B zJm}_{=2`Ey;y>5PjPBt{C(3q~>>lO$&HY;}V^u&nJRIMDRqJh~Z+k0ZhnDWQ@?Z>( z8&Ap9(B`Nt} zov>3iHjKE*taO%Ad)h>-NL88lf%AK;94T%mjJ8PxMD~L@xQB^%VcXH>+&V7S$~+F% z(N82jyYqJ#@2Hj8^`)(6lqK`1v-6F$+5Cc+m)5(aIpu{Z{Dq&DPR)b;5R=xCVtek4 z)|W5+C#V zl5}USo(9&6Oli)Bz zWjx1eG+9-`tbLTzMeh4~vkgZv#pyDWCi5zwn33Z(fz22$+16r#S-5_u+$$KlCZ_RP zE$pbv+g)TxITIBw{8b#qoztCh4LSO}Y-xKX0~vl^ZNoua89}R!18a?t5H>7Eoy7Rl z090rmAr>p;=7P@_S5Mq^g`i+nI6?qTx(#pb7jIXrco=a#r-T9}hE$CB9xF#m9W4pK z`mxc*Jv%W%N?&OuW<1eO{-Ir2N|RJXNA=0M5IHw-CWf+BM1yDE(YF_kXJhz2(lF%2 zw?gXBk<}cVPKfK586I=ScBn`xdTX{7gB%X~SxN^=rk47Iv>5TwC=5KlxcCE$;OVGQ zeJpkwFSWk%ar(FBAMx}Kk+E?QLuD?2pBV^>;YJAYD=;Nmo!ETcv=;%`X*%ffV0)XL z5&+vCqCdA8vvp3Cz1nv&eVKhwCcItY;n~KIS=f+t%terts5P#0C38-KC8DxP1uHSF zW&--p}myiqqQ@RZ2 zch{d9#Z}zaYQxbB_OnFImB__Ap(*dBYZr^QT%~oaG|;rE11)vq7|e+-s!bt_Yvuwy zR|E!`$jdYF-VyG{Q+AyP#|?I70_hcVQo2*ooWABk3@Zwo-Bm5MBx@#?g2()Im%-LU zc4GG1<_`~mv9oB9Q<@m}>qa8L^%+w+m?{ zK*PLWKe~rM6b7eV;&RshV!x-b^JJucBL283|E2jz>A=aZ?{#L{AIpe@!zH%Mg*bhu`-nf0lg@)5CvC+o_tm+NV~hB)a24 z^$;eH)Zz4u7IYQi&lGGG<&G9Wi+?jlPLq8;P_K);&1g6ip6!^;VV&)m%;C-VOcgwc z^CgR*6Uy^BtISm6ep2y`pLT_YZKc;S?K7Q?DoSfDYTr;YNFCgP)#9pLT`fykZYO=NY?7%A*0)YALyod@a%2oH*b^%PYYS^e#O5>YScWOZ~+W_n2P zWOGmYE=VamayB5oxp}SCcg%plIsL-|?RWA>bQ~P^2_w3pt8O?+NS1CePGh9)^viVP|l<6bL}6t44u zC`!Y+q(e}$4+{D&DcWeIeCSF&epgJo7xuzFtaLaVBkp#Pz-A19rxnEO&NOP5W;K)x zbFt-5Ov@hqc1-FXZY$=9J@mE&?9G0Ro6?47^)b?R??ed*7|$)b*K2rA z-h;L{ie-2$7Pd8L%)ke_VDHz6WPnfI@j#cX{3MahHUE3u52{&WU;fx>`W+x4+@VH> z$Q2?BXCwgw|Bv2gfI3^VCr8*mFw!C;s4scpluKb^$SMYqzp2p~6Zu-tnWT%$mpGZ} z;J}lKam!r!j9D_=U~M$h;PeFs`$DC83+y$tvqteXhIdC-GSHI_Wo9o-u~~J$_W>^Z z=D0D>HY>q1?8RS4+_lDz!lPm|k)Sc+fbAwpd6CYWi{j5nk zn&+}`07r~o60lTz=~{;l46e>CCXPQub%z!6qZtR-&iv$TL;<1f(7GUcTY^b0Nm9$J zkgKt@@-G!=R`+o>p=B<5HkL`%>DUaaoI74B!mbYBarK-1^W<$~Ju57-Bk>{QdflqP z4F2$9#w|BF6QnBpBL16Yv7-x_YlyMr8bR0lX>kW7hGFjmtd@gYp-ZiVZcOPeA$mu_ zoTCC5v_uzyW~v!Keymj7*W2eNy{wEbPlfibG##JtP+FAQQ(egu*BOcGKpV1zcWHAv z&G0~;shP<8OL)#$Zk5(9nx?}ycB*-ONdNpw^*QmXWSkPJxMj|AM>k!Q)ARkGRZv>0 zBwq>4uV3Zp|Hql?|Hz;I!N|V||0hdUv2}8>x3_b2HZlG`=!K$=(tp&;RuYUdL`jM$ zpm%{47MfvYcJ$qOQA1ErOVclCr@#}|iIc1XzL9|5lj&cKye}XhuwQ6Twpyi}%6S-402P0{g9AM*1!3A2E#v%nl}3- z%+g%yy`6!fX)d=AEjZK%Ie{Z-PPGB=a5?&OeM*x|#+&z{c)rFMPHZmpSIe=1iM^=; z?x#1Tl0l)B8G6`#E3w58Z9RCw+2Z%Oc#W-^xHN1l!sH2nlSNY=ONWqfPKuWKe|T!q z`f^B#H_#%7Y{}wf2ts?7G>q)^RDoUC#g@5t5IWY!KC2Q3?23?fEUlK2^l=mI7=aE1 z3)6@~vO$X?#SF#t(w{cE8@?XxGY7)Lgaa?%DHv!WA(ld~d=Ve?`3^;+nY2DjbOYMC zxO)waf<)nf`5^gX<@r|SW;dDUq+<g>n9}S+&xdT z(QHDgP#^z`z*AsZ#AT9uJ>PF@*UT+i3+W_tWQ`jDH4)S4ZXf9;!9Uc9viJ@e&z+LV zmBSj@bUSi0Xw*(3k@9DF_^kc7OOOrYZlg?a285w^{UGC3Uotl^n7F3Xc%_ z8z7_^SP_9bKz+Qhplno$oB)HF9L(R`IDf&lcFV@Kh4_u`;}mck8J;4H#Amo4%eZ|O zXjtwq^KUZSYbMLf?0WKskIxr4Y`+|Y2X50@e>4V@%k(O(;UFCML3S0xT$$2;o&5<_n)rY(ce6hmo}39T3JJ&i0?8RDsrC__oUr^CT411+Q80AP+i zgOnA!vyL_vno)W1AOL z$<%xr$uLyfMe}HZH(57L;_4QZ0F7&9aACG8LpSDL5{0q4^g*bzcnYJ}H`tb1oU1n#vxkD`lMZ zE3o<4{cpXq=9(1(9S-WA<(!Z6T9yST!?Of>KpM97XZZ-C=B)g_qJ>nBfGGYTpU?-Z zcphpW6cF#nkB`H2b}eLT#?uc>aSfa9r}ajnAov#l01puJ`;KnzhH;bn~4C?BK9se(?7J%S??7aWu^WTkAY8JM}c5Y7p`|j(a zAS2iR<0CSlR!kiLQP3YhsEDjJgGEP5N_;9(a013!Thu6_b?cVS&y4p0^hNHHl9qleHYG&ouY8eHUPKWn}*Ah6bmD6pWHHi#d@#NTKURI?1JUiR64 z*72=@KXlw{agW_o@;7D9h5T=H9wi#>bNN!WpiLKMRnT)pN}rM)nqwhU_(OXA#vjXs zzV2AGJMk@Zp~~;_+^IuFMzI)CN(s*H8{~@L55SnRoqf!zj?(7Vs3+xGchek%xkZe! z0FQU)3XN2iRP8WB9)>6_Ur2A)u+?@L%VF4Q%i#=I;O5H7B`7m8@0oiI2R(Uvxs}nj zE0+5i7{q_@_T>NdIY~vm+eAdk>r1qFj8IGi7xm2=ipoTV#Cn= z>|5hM*Z(s3KV$#@djE0${0sjjMf=|e|Njq2wo#b)Z;m{}%&m#%lh_W?pszMq@LwFc zEDSf%Jwx^=ie@GMs-|`yq+Y->au5&zuMg%81>vtwaLrDBWMVPxGsWXn^>96s+42i` z)3NsO2PS6(9x+l!VPP=Q+v@lC7DWPwz$$Cgig9ZXi@-OOd8Q3ChI)WNw_Y0{s&T)V z)&^Jw<*(n1#&88$R(I3oL=qsi<64`Q&tIu5wA@fs*S~{m5~*J}kaHYW&jH%!?-g1W zk}QOniLO=OK`G%_o#&_0gd(OX4fgd613GZs7E-*+CXWx1iw9nA^9UDW=Kr+Yph1|3 zqCSmn9XpU-D7G~42#%p>DF$UpZbb+kA6|4&@{BIK3zxwZS@W`6ju9)>eRX-!#<|>x zeNgr3VL`ZkZ|JQiND#;1mjWEsaBVExy2?xiwM@-tMu@jQ zSvOw;*V}%aivsO6$cPX|?vvW7B7KJK-3q_--2Yg_Q`yr(8$WYs{r?Ll|K~FPZ)w{9 zHjV!4-Ix7?ld*?Tzx1d*j-J-z0m-WZb&zllz41mTe@MweZwbjryKA#C1o}qj*LCAa zh^i%kNs49_W99|@70oA92pU^y{MBU{5b`z~ak|WuDy&)w<8$ z|K)nvZBKSu?U?^tgU5Hr=!LSJ+4~KQ4aN$|3~h~Oj#?|s2`YZozUjet2mkpYMT`MRA(?Mv{-I}0IIAlj)rPl$k zRH%pR0JBx72g_BcN6$cNlh`-(W>YTf3-2lcqeFWOnKX10+Xwbm11OgxMm0C{>qT-A zTSa$C{c9k+b$YS7w1(7u*%_4+YIZaM#?`@IPVx!0B1IW%CE@ zVZz!OY419~-t3~n-avJRjxcw|?pVF>tA4fg0*r^n+=Ynx@tHFMKe3K>C88bg5=5b9 z2*9sb$l-V9%YN-F1Ipw_+kj;+=gJ^Fc(JT#lUhSS<*MU0Ub#(C^Md0jWwJm!DV z;U{8x`1YsW4cFZfCg zg?!tKlT#hwYbyDekc?xow$P8oj-^SV{i4Epb4J2~b2fLjJgmmTzfhLY(<|079(rO#iSp76d#4O`FH2mv*17 zOdnrC10ydvEjt}EGcUP>{FimCM4qLUCHa;3A2G;Mlk+P?jLcOv1zH`crIS`_O#@ex zYGh40mrr#YjkV;)sC+Ko1sd|&qmWwQzEF{>1* z)1JLPKuk>fhON0p>@lQ|-?~KgVWYXxbx|Yd+{V_*+P3V%_QqJxZPDiONt3sUJI-!S zi&OFI)Ox$sA|y1TgVP&!%7rHVqD*HlUR@rY%mJr^tEKjWMVHjZ@;c?ZBK&-2S6HOo_FQ5S`gO0v;IfBp!71 z;gaqWiG@iE#Bn1Wyr4!viB+{XPQaZAM8))~XYI3Ro?f>i7s6<>aCxUnobIefydYt@)hwU2ogGy1Vdxt_wEooeS@HXdB$0yZ=!bW$8|(yWP% z#EkX?=o4EnH$-=)SyNA@hmcQ7T3VjyDiM)!wM+>O82E_CrU5O4i8i0tv9noY!Z$f0 zyu3&vl{ZB35C~IqR*g*JW^GHmoI6kfFwghLE~R9}aUHWRq@_K55^Z5%G9;czJ$OXL zFOf_b4Vfvr^#93Q&f^5SsJM&oY;Vu}jRQ6U?H_)9bIwA?ihoE$rEICM*>rSjGmdP@ z`m`FnSZ~kh5Df_}WKc5H)F1RYW0+>yAETaO+qW z&ToqeY8;!t)s1qb>AJjxP3i`%BApQVnbiD$kp zu&~uy%owK=ay+m4EKOjKW=k#wK%3poyE>RIACD1CabB8@Ux#IKzK_~25zv;ZYBBw& z#>U2L#>hob*NR~vRWwN`r13BpD$&h&gorcib7(3dMn3pT;qXEC%24!4uxcO&13rA$ z4OK;4#S^J*z<#=UvPk!<1oK<47!ofIO|z*YkP>AA6w|7=W5yn(;uF09sKZR>c}%>O zha8iq#*Nmd(qCHCn<>oYbSe~9A&JG4vmD^i%|nOQkTSLgJ7`SO**E@8-WR~febGJp zsNGy2Y2_TVXD3m=Z9_p*#S&exaIX{hMVpb z!!JCu349ZB6Ok3YPmUR zMVs(Of;Q#wWkPrC3rg2|aN3`lA>AQWFZI`_wz?n%3#V+e9XO!CdEy{8G z<~zc_omkk8_=zk~PbBuOnxHk)ZIGdo}(-3?H*cOB@}3uN^Je6dfz zKAS{cLLuPK1NOC>CbC)0%nRxo6>SslJ{C6r-Yl&=cDV|G3aNb%-^H@U1R51&5UHB< z?xKVBJ?Cg9o4?~XDCLLkJNx>`+~pKi#b6*@O@>mYUe*j(>z0QtODQ{_KaFjumdtL_ z`o|W2*ZH(mo)_Elzsq6D8qF*(WXG*@U$^Z9pQDU%4~(eMoOZxfwj)bls6mcWkNn z-R*aENe^Xl5s8cl4)vQv8BNNa^j8qM;1-){mUAu-os0qJn!ip)1h$Bw;dH;kiAZ6y!blktV6i-!Q%cN9%t>@2qV@ugoA|~!dF9y72%l9oRfj30#InT( z2D=h~iB<93f8t1DzZ`v9a~!oENqXX+JuPN?^s;(IU5o9|GQ`um;L*E(thvdQu<7Q; z39qR=v~Qc$s|cVR&?ZS}{4=CsT5LVyYsFRCq~q)2`1*=gGrG@FVS>5bO6&g@)?|TB zhwRv?1+uLXQ)Dp$$#TRsW*TfU7}^5bRRJDAOUOrCyx@SD?N3#PuI3O`Q$p~U1b1cO zmn{JZn^GDlxnQYcl#vJMu%d|}VGFUz0&ADqUwd=xiR7u2iLs=OF~{|hxQ?`=h&E}G zBRILSHI9?Q^a8c|(~qixc)Glo~RJCIsrTPl{GuG8J*OZx&u7(uA zae=$t@VuK511_(Ldi!a3x@J^8c?n#)(Q6pu92lNQezML#on{@`5nA7Ho-7YC*-901 zGsLUC-t8IxtqHTfP^HK!Ng73CntloQva|O%Ib{3!KltYc<$5W=&VUic+EP{6Xgl|N?bCJ&m4v z_R34B-F9w?)Nyelfx{68njnB7RmKiuw(w6_YN|XSL*F2Z;ndVisrp(zhi96CVjIgS z7qcnm1N!QN-K4Ms=L)5}QWp6ts{Cm6Y|}VZWzKoDjbZ~)(fDE6^M9LOJKs94gesTp zo9P`lTR&b&I~*rDUfVN19mg93qY()80#l}lSR#$gdHu!s=bM$!(;|SGDbK46g}--A zR+?Uc4rY4P>&VFc)3k(0;gZv`bC^;;hpAysL+xm0>{tku6XT|8V~iOe5G;-)r*hQ` z6SKfU-6Mp~FsOQ5p$&_LYrXQ2OTbHnYGjzl1Se<}Y-ATTlwi^AeDemzet~fLVz0A7 z2(g%)egg?3qr5>m+~5GcEW7fQ!D=L&o2Ch>uDMA562kX#IH(f9J|)_*6Mn{Np09V@jCvP;7Uv3 z8?XQeemS0=u~ndeZ1WuSHKe~s^nKW! zK0*IF5=J35y$)$`P=*rzF;#_6%GWc1)9~HUjY{=)+ciGI z9S9F6?VwgPa$4KWL;0;6+*1b?OZWkx1^!~l1{i#;%RK`8JT$RG0#eLsZJh)@dL^{x z`B-kZ%B>BTTfdp(8WJNzKNiMN>Bqh(sGsyi(x!ORmf0hediVIU?{j%ucVfyGlXl`C z<*4d&DHunarKjaEgXVqT1=7eFH**~Hk5nuovScD~s9%Mo*YYHPLrs&Ui}2^KpZ)@c z`Zaiuj(rR$S>K)Ies2%|jaCrk|1xVDg3)l~_p(|NXfGe8h$;B*nY{b0m)I}8;`@py zE)e&P*bzGITB15NxOM$+dsT z0XY8N_A1}TQe4XC3V4NaruA6jRfJK})1fTcPWJBjGW3mzYPCA{jV>3j(S?L_w#^x@64_htN8C%&9A|G-QkhCF`jgMphAl64bd9Xi-ToA z7yB^5V}>FmmunHX@9k%4Dhah5^fMb;v}361))U@c);{?VO9az&ADGaWLY0!=Kn%Me z&Yi1a%=>H{dveyQyVpBLGepe`wk*kxWIJRiL6h3Ck)5nn!S;R6&^`WnSgfq^_ zUA+y@4E>t5={bGfLg}4D>D{lK$I=p?ttmER9aHH&t=wmIF^9e>7GwSJt?`Sf^?v-H z;8|IbTgeNK4t1`@Sq(EEiTQ)0gD>y5iNwMA5tUE(6N^;piyDnZTlrlH36&N=jfGC5 z!ShOzz<2pvhZ;Q(XZS`pk9$^lr+6~u!qA*eU<=B{5U<)!Td>mvFtOT21$UyiN?snImLg%3T!)3m<0 zZ&n&PXvz-B(NkQ1qMqVM7LV^Cx+pfaRTEC(-h3{8z#4u;*}Pa!|Q3os>aYzKtg|BG&OjwC=6Z2cRl7o=nE!0~K3c}>3kkEwptjK0{CjPyQ z6j&iJZz%1EXrMMYB2*%fxqngn|2!r;P(@EbNVdf+&w~{ZO(1V=8Cts(WuDr@U)#k4 zR|bg_fD(yP(?r31OtatscIHtN#rK6#aoZ|$%p6A6E~dVYo4$rKijy7;UpLWP-*+Iv zbFHE;;8sBXis{s}#k6m`O*!e^Tswzu!neD%uB?c&?9_Oa9j|4eTk2MORGLEX#*+3T8oSGoCpgS(9n@Wj94J@VOpFL2w` zQq_43Z;GQSj@`4U2@MQ}Pw&^+GTBvXE@4kYfKJs~MtAO1 zb!^6%lOdPf0}D3>ti#EsfV`%X?ieH&Tt+HTV+?&F!Kq=mq&yGW*CFKKxQ@MRSMX*% zR5hzrGzacEKswgWxzjB&ySZ+lW>dfqXOLL&F=c00Omny!XQP*M8dZeuJql%y7BbN? zP>&S2GR0jm<4I`FBF*fvRxb>32 zjgsrRb8Dw}lwGdR&d2i~e#?@9KdD%q#pES2A-ZSWuLe)3=D}+g-=~~E0R@7?9rDbf!Kf6@uf=l1oS>#7OB-K{9*A&rXfDb1G{4d;2BP3r=z;PL{CRB9+?BTNAmP$@=DEueA5H~F#_=Pr^PUDgHf}w zOsC)&&fx;j^rs`HLB2r(_$ISnn2(avS@3r`F8SVNA~gx{E{L>R4(Gtf15nEB_`Kmz zCj%SzccWhvdq2OR%{$Dj3_3dL(2vU}vV;J<+$@EojarlO9ysYlP95or9f2C?qrAw@ zgLyx<#@GM3yYooYtQ6)WOyCzn;1_Y=brxcKv;g059z4_O28Q!W)NE;5lyjOuPpW{O z@htYwQ)LL`>d(GoKK}mQWg&XH0P}|U`_uQg4@hD(-AtlQl693vzOhku@i=n9C!pWt zUJ6RSs_O)qM{2q|%AJDy`b^{b+MX+lSqWBHECv z+4c$D6rUM@>qPjw5YS#^4%14E8tLN{FhPVCVInf#gYvIWU~DzXQQa3)Ux5M;IKZk= z4|eCwB-tIPk~6r|VveOKnU0f@1(4M6;|o82RHw}&*MRO}DcI1r2sF-3<|VNxyyYVP zXpQMXDo7&eb37*pQN8LRNxD}O%`L6g{8yNSj*-JhpX65clKaQgr|Oq;ul@=u#yY_( zl_S07S6iQHPUMEQX$Ntpt4UGPyI;Yki_*AjrpCaC%{nr3F}Kw&VMi>t9Dp2aIc>5~ zA?8}oJv(y_fQj>i>_CIyN-ayPJ~`pVJEIy201J6QEroZsV!fmvgJuBNjTd1(tuGuVy=PJoid z)D`Lyyu2Yv4&fl$>&G)kkJjU>C`TPj(87M>kQ2@O}xb>hUxspIFOBB+f~q zeczjD^igk?kXui4KmG9vnV94J;jYgj6srK4-{a!p9G)Y71=<4SM}R4$E*-MGp&E$b zWr)78nzkkpBtyjlctA?O8@Q2!#9Hh=k$tc{b}eP0x3`ixGJe$VzgKUAq}cRNc2fM7 z>xC>}tgNGjya()J-?k*DU)uJld`Mk&MgfB4?kyN`pG(LQt^vEh+CDJFbr{R#V3rqG z?(2kwp+ZD*tbrhALjjLoz*2&JtwD3wRT@ypNU#qMt=r%&hVf}AH>ie&d2$hVKy_N; zLBza3PAg=t66%S)*;23OC#2?gd!Uo1O;RNy?SqzV{fXmlZg-hO9TlnbXlLaLMUh>M ze$)JD9D4YZ3{sc*!YoB$QuM`bHZ~XMEWX|4Lp#lexI342+JKV-Y_*m~!I-^Q{npqM zHAckY11(hE7^C#sJ1Wf;7OO((Ritcz@Keg*{dWI+eX>vTAas5j?^qh63j8`|cN8QU zDt!?3DY1^aq`y<%YRouriUz|mX&C56TEg?lXsjZA4B^u+It~&r4_brj+!)dFCJb-O zaV1IjG_V3CT>ycuYR2FagJF?lJM*xP+Vna3U)W`s83Wb!Zlz*B3f-5Lzw~LFR8T&g zYK^r8;TiWl$%ewYa8Zv7R0+$~s}Kv|Uy;MGF(8wEzQExK`C^A$!R#(xII2L$A^cb2 zK|^82IHjJTs1QP*=;Fqz;x82An1!$qyh8Mbxmd@Pp#Z0{E}abVw(3miLV1(Q^So6tA|ZU<{4xXtBX2kH({QUmm`J(sq$xw{KOy$4szwaYnk7CkDTL{@ z4XWOPQ`Q(A&AFX3Hh16a-g=l$T|W;8Y&`$v zxrr{k=DN{=w7ni3exA3;xw@e#yu0akBXh}g|1v$Frlz7D@(iFqK0#7+jPkeS7TEDzuZinOZjizX;E*MW8f(pfX5(-Ffd58@<fgXXozgrrSNtph8VFRELX4G_%vd8{5%YkHH-# zL>6ju;qgkUgER2``2pDR2E7?;w9%pTrY@yoSgn?%cjIkAiR@ z0H8Y`$_JhDxD~}%_jd38o>Frg@cu~~Nn&LkdU0A{ZCXu}p#bKEi$t$<-CR7n(Fdgk zF%}POmetw*W02H}9=TRuBjZoZxu@Lfyr<0kuR*D$nOpSE(hacJva@ZnTD07>4BEvT zFW93n{qaxH+iHm^?ao#M2JV0?=cygScHdlMm7FN>Um=3DCU7v3Hk5BEXNWOH!lQ=? z&&D_kG?i5J005C6i4MzlULa^4I)BAxj<~o_;iET!KpRdc4Kd;bJd_6*=3bl@C}xJ4 zPM)z)zb>z|87R&NO1DnM^UWmo^gvG?3l*i_RYDh~Y^TYJiJ)R@T>P;nW&p75mYlgz`rx2y72^aNKeAQe}=S=PsTe9=1Htbd(Hm& zL4ESE_s*%}T!s07Mvo`fKD8e(mdA`4H$Ec$1aHuzYQ)oGz_YOrad%>_F0##}S+3j$ zx8Ks@e_G_MAo~RQFSdLd+GRo_g{-cr$d6DA|?)}Bj{=H^NHghQJ zj_N_wi^joEh#pPai;jRz%{^wKV})(qzXJk3VC@H21`y-XlBz<>5R7{qRmFzFk5+oon%!xQ&08QXy@Y-Havw__jYG<%Fl|9Vdycjpd3Q%{#k)H3 z<+o}yMd^+dy|JfHaG#5Q2XjL2IBU>T8+R%2a8d_>y~2!22rL`eXf~UOV5qquZ2y=t@MFPo{XE$ZjL;MnmxXkQFRIEw?XgngvK-%hG@sKR`dm z#nj}WF?fyAQZcSuZWOA?7P@wTe*2>Zd?xD3(}#PIo7o;5PT$;NZA9c; z#AEECJe8%$OG~lYWC`&p6iz1$9GCa8I4~NF?S~7p9;qLQvs@G8aDYeF0poe|e_yLR z3*lwxjNwhU&8A_D9(=%*HUlgLhV59!E^Lo4?edFj49>?=_og>qDU1V1;uR`0f*Ix4 zn6LpVaE*_#-7skUv^@s&luY-1S3WN4_%zi2qA@~xJyefU#pQZP;38IA_ASYca&11F zPQJqK%hzPELA7DDQUOX(-Y~Fs--x7qM^AI;f9)8ra(yUkJNVvj;yyQepi1@QvU6JZ zJ^OCj?}f}SQ+vDNdh!B2PmH<>;iz8KbXwc54N#J;2(X7hZWSa&b=Yr!Cbc$kCR0nT zvS_(nF_Rt8$G$TSP@YaE5M3Hngs8rau+T&~MW3ddZ_2T(=m7csYhdfEF3L}`;M|eV z9ey>?ptzMQ6f?Z~!bpC3dlaoyT3Z$;>X5Xo9O`F!ZUu!ZI**-djIn7M2o=Yk>#1~!K3a7rQ&q~RQuUGwA-}$u}WQ&CE9baZ%n8ELOSevzcgdl zB7(yMY-`R;y!6zEv6Om|L6YH6KUQo9UIloCM4sJdBGlyXT9{}mrydxhI!g&rSvV&^ zm(wtyi4aJ!vk7iYI|af2orY zBRyeR{KAqte=cLz@S5bW2%FZ@l87qpyR;Wqd zy9ca5(`^Ez+6&(7Zu+JE?VORMp7ZHY-1KmvePw-3m~+c*7mrDNyNBf)snTo*mu=$s zLbQlEkUW{Pm&cLXoJg>{a^~FN7k7o6f_?bc&qkal`OqdzkpO_G{t!$Rk%(us*ay73Y#0-+@;7KDo2dd|0zT?& z98f;%J#W7ai*^Wizc(;&M%xg+Q}sU}@M3+e?8>FKq+0QsN@-O<<={MJ^TFd@ZxvE7 z-R?QNLIM#Mtqb2#8uWQK;!eD1o8k+ei;V+|5@n=cb|`tQUQQ8 znDCF>fUd?i?eeaw~cEGjW5?Of;SwCCu?HRdwFMP6wfb!tGV= zm{njG7%a4V=!(B^IiQkYC4r3M-YfeD>l)<)qpeR07E?c>Kvx^)!!i@EAhe_ZM$|>C zi4lYGyySQlT3y|-!eiDb@pVbSj3NTm*0tFzygd9kz;Be zVrkl2>6Hc44~U^@4pBunYWzqXE5m)t#LY~eL+xxC`ZlZ-F3NNA;5x`TP zP1c_`u!h6ad6ugTz6xc`p0D}}dzjZqSkaS1tp=q?B1S&Z%Mf=VuvX41h?_w!GJ{FB8jemS!~aS z>jp=Zf+wck$_pJw&g>QVJXj)5AA|+Pm6gkp6%R< z5~F_oJMQom)_gB<^-UlZMnpb~?gK}&hY=I@X(c;RQVC?J7%eeXgk8RoREzjBiGcAp ziFItOI{Hizna#PYJAee)>Uj&j#zq(TVqE`hT%^{ooBcmVrk>*4QFwN%5Rq@FM2Hc5 z=)!_^Uu-B)9TxW*UdtNgcfj_3%4n+}>)v=@@wd)%Q+ar|5cggXlZ6KQ!~6VTVV(?O z`oAUf5k6yt7N0-`A948~yLc|V&YGlz7L5Q%E!oxVBpWA?O+OaWtysV7UwXMQ?l&F? z`}vAhJpSR>Vm0SaTR&c_aFtO%V8(m{{p8L?drN@#Qei;YvYIuia5*vY9zE*lSvxnM z#+_YyIqvT}`3!|4q(b`N-QC&=b}nqdFs1Lsk3w;K9TjmA{K*@L(2vmrtvpM|QbrtH zRIGquCpe^zaR7KepFVu34Xx^pB(iv^Mh+aF`vy;sXxL?Zz$-=7*95j>a1$nNzfpuG zvSIaB5nr+lrL*wV&Emrs*2=MVe!#@co5K1v z9q0J15t@Ai83Nb+nq!*Lukne~s@ZN@rZ^E`#V8%R{RU*U_S&u^HAoOmuNf!p3&5vK ziS!%iw_>_UXUJlAUJZbLw_@d=;d=_OUCWeaL1!Zd7BBA@qkJq(Zm?zWH&2V_gBf+Q zkyrvZZUDCuTmH6jhv)8y{#SKi)!BA0QR>Yq$cOdOKKlr%%q<)^JHgpL&fNFyWT-u* zY$O9dtpVVV*W8t&NXelDOSrrT>UbuLwh&5eIrCb&Us+4lt%H=wfYJSeE!lYb8fjzf(Qp&wdKXvZjiPEhw(XbM zmJ5J-5_3G#QRF6=aodv7YJ_m-jrkYQF6yv4XsuxP%sP*|c?;*C$K{>d{8xd~;CX}D z`HULnM&-*j1o~JO7x1*79oB*Lcj09rB`f8Ypz)NUa3nqEtA zI#$4w`rxmwy8C|a*US{{_D~>bIrX(wEoSGCl-P(m+FUen?BuXWY|DC4aUUiurwagI zlMSZ9`fS*KA7Kml<+gT#fCRqXdV$ciE}sUA%~-E-8lhV4R{aLO;l>fKGd?E8e&KM=+DVQC9;@r?WxWu~Jz|h|Q?=tfOl|{LzM8n#W-WBEnH*1+c zSlQi8K@m`PQ6b}@Q9djh4=qHFigbYWp7v)KJN~5A;I!i9E<)Jq8zdPPH%n27FPvy& zKMLwNKOYD|FQLLsk5-Vz$ggcu)V|0qWG?(W{?7Af3EW?>x8@E$sm7pXw%HhD)0KRG zWEr}e)yluXlr+RZa6O$K5e*4|Qs9t-<{^+cP0QTfNw66F;`4)~wX5rwC29drF>?G> z^hq9af5x*c#_76u3*IN3BOUOt)9~$#K^|y4=i_+*LhkVB$NzXjEMn(n>Khi~c|veD z!-PoR@nM|>{$TmNA(1PNfb|fm>UEW`C4J0el1gdc$R5;6X`>YvZA>5c^)WPk<0l`X( z!Q_J0`XHhm5A>9JN0fxhu%g)IWgLJm-Itzem!6>^Yr!P4)$U=AD$=OQiFJ7-8(T6I z+3CBA;I`^eVvE}9b6r2>Dpt_IsTfpXhfBjqp)~(gUSg061pk8ZmT3Y1fH6A+yA#PV z#c&I9lQK;+vvw2@%>jMUIJ zr3#D{DcOx>Nq$&YDK{!h6ac!sua`w;;sd<)s{rsjFzXgfR7`?ASPQ7OW$D6&WrZ{p z5guhGktP%wXgb6n13tiN?AU@RENGFEz>Hd9#vFZNEAocV?z?`hRNyITs)^=p996vt zdO+YY#dxaP?N)7~N1u&-#*Y}5gJ|aYLkKNJK8Kc{?84oJucAhGpPcEEoh;RaQ_Lqh-*xRN@&RzTkay8>h+Ogp0R4Xe zUqGP07t1c_rA+(ywZwE!7OIZ^1&K&cIj395QcUY`3tG-Oi+(Q}v3T#)gB(Q516fD6 z<>~gMQwM~57q@v$>(@itUDfv5iK)fa%iE_#cq+AfQE^H4 zIQ@t>5H#ow`HHe-~F`;$)jVOP^Pqn2Ov4n#$xfr|#W6c^95^Gyd|ppvo^H zGZTDfdlulbZX&8*%Ykz$obv5f=cq13am9Q<&rIdDJoS4atHW~x!?XH2A?q7Y5{QhI zMthedRd1E9krU9^0OG~V1N{YgcIqm?^4~O0U#tTNlBe5aS6d^RfAl<0fp_^qVt`_m z?v$5-TP=i?s))lv`0TvsejwjV`BpxyEcr6H05sJdr5g+DQHr{X4P|#>=gm9!P|)Q49TXiu!BqbkT_{X3=HcFr4?Ej?+8_<0N0aumF5e_$ z&+T5wppp8ZNb9)|e~jijg8Y|Fl#Tw5jyy&0J#94A$6_nRmRHu|uEaL@Z`Zw%C%^B= zI=uE8Cuq>n9OZ8>-N>Yy(9PyNpF`IzC*6mI3^njyw-*YmIVx-ASI{aj`vF24rNkC5 z2D~U9tFjv+qG~Z}LH0utQFRzKCHs+xsCtaLKHDD=wGN|Ff3qKqh`I-(lCvMisCX82 zAL922`snVcT|F%L8185=|5VTRe4rT|t5hphEw?(EU`tmm0(c#A zn=|8>YPm`XfBVy%Y}uO`j4(NofdEuj#xbS>@YxokFKZLFW^gE6S;t)_N1MwxWF)lh zJoOg%E(<|%!Sw^TC$PHwBOkgG*AOoMXaPDZLxsh}>uXUIeBiy)sFAEf@gBMog2ft4 z=`~Kj>maaTf^r9wD!;%1iQ_Ws!!?JvJc(9TP*0(we^DrI5p<2t7X2XBl7e0XtFTSd$&=8XI_V3UZ0FV5s^W0>*BsL@1pImb1gu_?7Pw*QS10Z<1DJSj~)R7I7!$Rdk_Etc;qs;iw zo862oe=YS_#(k`+{%J#$lgCw+oq%dTz)h?l1s$vtM;h)xDKYUZ5%uz(!0K&iqO91y zq}yD$>tRxwKT*}7(O{rpTRdif(L@?(F=-6#D_n=XZ=`xafAs(Lq^c^uaLKNK$>|$> zHV}u24?!Z@Ltc3Ysy46O1Ve96gjEwY){7>_e}O3bsie-;Sa97BlA_1cr0sjW!|5X^Kx zZn_+WH5$x2$~B+TSx2BWk}nCNarKcHuR+<((N4Se@xZV?Fb_M4+kfqYI(~Mjda2YZ0;pTB+^eh za032KDq;q6uo>py6qtjpFbDV4$Ju^5UD5leq$nSLfqhQxZagOt{bL|F(>BoEyaBkt zkMrlGaO1}o9TOK(`{?J5x6o2pKB|1(#yI*tTv8wT7&q7y2sUD{`D1Lk)0wr~f3)r~ zDy_PmqkeMvR~nt(b4Js;byP<>sEneA4#8#P?H9rsua4uBz`&o~wshC^=!jf@fe(~Qbe zmSGTP&}JM{CbWglgQATXBahk6e;<#EwvC>FhL18Bz3{xD1xB54s%=o?sWz{oiP|EM z(vM$XNyzK*lsE~>k>gk&=)~TyHG(gvTl(TcM^vC>)Q-=)1;kD9jES`qPZev>-P{pBLa!fj{YmMv!&O*_wNpYgCg{ZdD#oS&x`x zTWXUs>{P3_T|+HILnMf_2hG@5n+Y7fgM^Aqf%^{xEyOs^ge5M`#7 zT)={aZtnS$cH^N(%}2DSf0NfoWCZyZhhbk*JaZD5lenG1lgpL0M;l~eBW+s}Es`>g#M5)mnBS(e@sxE$LT-}EKt-F zbPR^J@hhpW-9XPA?#5f7FYKE;P<}z%WD1 z+zbK$A4H=Qs5D7_44abo z+tqgh2I?T#@d&m7z?S;Be6Iz;i*@)Yc%T-HP=0$1=lXuMe?3^PhW9U(VnCj2#H$4P z#nc*hf@cdani^_sgq(!u1$g$u^DB6!;f)`I#-N1<{|q_TZZy%u zSwFPV1sqBDNvNo|&6!6{rkQ|JYcsJ1n`uJ}-66?bf7nw%QTZ@u?mRv%=G*ChuuJ|% zIRa3@a$@|D6J7*g@%<(h^#;hFZmR!=o>L{Ab0A!u6ZytZr>Tw#5WnRe2gm)HEShGj zJ3`0qUga-GsnPEG^Ow1Z~VN;yzzcMh!-)=Rv$VMqC9zF~-$SzlXr1oH$YuslDu(a!mP9hFpH zuEUY?3>ytu2TgVKj?MVdAFf0D*?Up1x{s=YfHcZ z|9b~+z4bdN%U%Q8FyNEsDjs{k}`yCngE?M?_;D8Us7lYm^ZnX)kWi zfB)(C+%A%ude)35V!S}-^)EEiH&BrjV_%;f*xq0_Z6TQ8^fqeh8rR!ZF^1;(9P^|@ z9%cKxk?pjQ5ri$rX=}tpn4QM*n*Ulv)(mO_1p7lqxL;oqHK8QQdFAzT_lwb{ImX@fOt=O z=Tdm-3W=9 z7*X4C4$;}n@(UQ{ZK_c=vT%Mdf84;rScg)_`~_6p8)B*nHUApu-Bw7?+LD!KYWhbr zDDyD0e__FEF?8A({rR%cd7dzSS*O^W@4CbzJ_aQXvwQFCCZN~JQ;iWRy=Y3aRq%eL zk#-64n-@BP?of^~-Iw-8$Vc5)4~=gj0Hi)1$(Edx3pxe+(7YNofGS zYrN2jhQbs&J#E6f3Ls7rWFa!H(dvXHlmtR`9g?EcTZ07Bg!LEP^TqWg+Qs#NQtbhy z+5<|p2b5|Ql7t4579U6M!LrOUP2(f@~EH%DZRTn)Q66bAI3E(RZmj zCa*_p)%v7+K>ZbAoximaN1xbBk!bqk7z~VJrg<-XiHmu^+(=gkmhYf3Wg6A6+E|97 zlXL?CtthQZ-QEZYBJqp1_Zxk($0E7MeLul}^I$>#=5-An-}~t)f18q)jN6dMTzWS-sr@ zR`!Cb{W-yEe+GKb&?u`vr@cqblzEyoS;sv)1^Ir~qZtR35y=en8bi>cXj$6%8*X7U zVP*JRI3&7*W3D$a2`;Spn2UqqhL*g63T7#r^99{Idy-lABIjidc@qSq=s?4O!JS^6 zD)oawFH`G(Zsu!HTy)?X{;&*7-AfD0hF>mYdyMH!fAm6Stc|}Z5hM1j% zBUclGe+DrQ@6(v`r$Clh{YmQGVt6T(<&TghaLlWom6GkUBURsG_!UnJ?8c(8yeL~1 zUh;&(HVYPefW81^w=Uyi0LO&mN680~nDHen zl3sSh;8~{kEi?L-B?`V}W?|G@zGW%QK1V-+f0?>`g~lz?*C5V(Zmdm4ac1AjteWJwKKQbCSe$a+jTNz)vpcic}|LE0aFrW{*M0`FTbf;mW*LCZjpvbHY-{*OyH4JBl1rjY~RX| ze^+i!fa-y5G`>3l+%EKj{0%U&*CYuYjibX9RQ2tBQ7%U(xT)5;Y4sJ%|V zBV`D{Rb$i(RAPcc{vF6otu6Bx(tg38Gi=-41n1}=EvUVdpc-q8Mkt`A1-_Q2>T5fm z0r4ZQ@^nCU&O5mQ=njMM~hf447(b^`Q3$8!+FSFXiuW*;|ZDWPb>im zu?-}Iv+0E1p1qpN|BEYKp)*kO0h(6CqmrvWh zhg4<$LAqtCq@tao{5uqA@|#N68MK*d&&tu8>VAcHfvtEs#eA3we7Zfeo+>bpF#)*O zzLjRUTTd@A4HM+|?D7Gn@kgws6K|1@q1hkU$`w~>AP>0wBY#~4-I;X&f0VSsR@~@< zsC)5Zo$oc4J66+m4@xVoslE@?(q-6GO7Eh^Xo43^{q$f54Ut?LWvJsOe1FhlLuL zrn8l6IzpD^jRjPd;hN51H&tb%Zy|G5_PHXe1$k+jfw-v_wC@L|Ctg_7v6;5)hB1~o zq)E|w+clBu$+f|21T=s0Un6Bq&2ldwXUJUbp{@FuI^v$wl^GZaB~#CKwB zH#7o~oe$n|;Rb+h)LhTtmM)th46^E25e?=c}N^!*7gxV&Y9srD!LTv}; z*lMHGMIvj@WonLtX9kQUSG*acQXuFwV_XWxSurjRz_MX{JH{&*&j3^RVr(bIc1wSD z)+*S7v$h)@lNW*PtVv8I9ox=Df*bKPICXn_c95X%3-%H1Akb*E5uHRk2;%giP6~EV zez=3Ye;r1oDJ7gH+(EIdFB&46_?i9Vi0?*>JE3OmpcLo}GsXhp`m$#1Mwc&HdWUvT zwb}z;+CQDDx{~z|^hHf%H?jU{is&B~Zx89)jr8rrI0fUd6PyahwPRcx#yE*w%{j>dCc<<{%<4LGJ*#@(Ee_lpo9xka%U>?O4utJ7VyA(`o4wj+z zQ1k^0FiEE1`(@0|mL{v=kpTfC&(qKx-9SUn@!VpVS;ba8ktq@I$W4@MW|_7eac8it^`3|oV_c(}1fN;Z?M2S;w%-Wc5hIOffmq5a$+UgkgyMtwd*Bl> ze|~*3AH7`>tU>u;2dS$j6?3LjAjIL5fymra+k7GEI^SxzuxA9LUbS(xEf;#fW z;mlBi+WK^Cc_~zX?Mbv^>N3m(v91y0f0^Oev)f?8IdPnw^2F13=qVpVu*)zCvT4PI zVe6aTvEDFC&+Dt49R?WhD)mj~muPQ3{n>V?C;T1_r%}0QHHER%IFUq$!J-CtbOjdJ>cwEW)M;%p)i_ zGEcs89rF|@qnW2rNoAf=Wf1dZe=5D1r%H)qo>dB=9(pHR`JUQ$ApR z+L5iyzY+brm>-P-l&#Ec%nv&mZ$Zoie-`rzixD6uoy8p9*%LVUIPlncPNZp&;HUcT0vOklAo$ke zW!J6(+$q{E_;QW1As6IOp4{n>&)};;PDppyxrKcmp8OvlgEotMRm)iV-bK7RV=m#F zRkx{@?+3A~GdH#vP8YMQGqO~bq1Jcdl&(*etNO|dO6!mDP=Q`Pf22HaqSw?-^;hEF zL1V`q8ni}#tr%mqVk0!N!`rQqK9i-=$cpsi*RFENc?ER;cxxO%sQ1W+Ou4PFa!#W= zfvQCuj8M9Y+r?+^dPdmV$?T@@c4f7%d~_-ZFtY$N1x$)7uNC}ulaST4GCq_G^qKZf z<IDZug}de4U*@~f7EgD5dAE;+BzAwzX<*>TV6&>(#GMD3lOv||6%PEg&y zXjE0XB~XynET%!9V1;t4yg!7qvaJ^hT}t{Zy(rx&$hn2Qe{s9~Ug!$P5`1-8y$Z0? z)oKIQ+$_j>Mj_*%vXkyG0Gr6bCNi*z-)#;W2Zc=l1bG*7ewE-Wg-Lc*ifycl$Hl%v z(1A-8+xk`QcDW-@Zg;!ri9A&f6s6Yj+0c@wKvRR+xRFJAtPVMWU`E!4wg~qqVZa3jF-wct?gxvZT*_m zg1``b=_7rqu1_n#bR_sD!HlzpU5Uz<*=tg$Rjorlh`Xo|wVCqSE_##!b0)#j@D4rB zkl8z8*f~bsStNXUY9U^+Di!4C=;n{D4$$5~F9C;Fe@)(9RXtw|5ix&_I!yJa5vXKS!37SgSqD*14(jVq1j`Q!YGGfYf)?;%F!acYphN8yO#5Iw zvDPm-e?KG85+Lc4-*CP{n08o;xDLBdGwHs*ae&B#ozgr&Mx0jHC{7VhHQ}49r4v*9 zed89_eyo}n*PiJ%tpuGwGew5{Sv#fO6gID7bZ%GA(a&8gKx^z&UAiOAjGfNU?xBiIBmuGbNatTOU-yAdUTuh}``TT}Et|VkT z%G8W~ARmH4qLp7kclc*%0s~ab`hI95Zu!-1+$0?2uEXRw?C@FcxeZhDs&ec1-8In; ze=4|qh*R9FXfg-D%FWxPfmPNwpn=-ueWUhzJ2dvJeQV<#ey7Ia%c}~+IOKic+P!DA zrn=j;_?E^arn*vm$B_4o##H~17F&282cARzB)9%cDOQkU9dfCD{BM!HqlB_uK8R8| zjLAp+m{jgG)d^6M>48aFs{68CDCemze*q-FY28&23qtxjv+#OIBy({+CMd3T^ll)? zX>A`ar*Z26-^lP4Zf(bKIsNG$)Zop@keuFQrup`07_nS(u91Qb;G*4Z8olACTCVBG z4i0AEv^0n0%73*&W@~h(}M}Pf2&T# zMeZv3aHujak-r3myF+1R9zaDaf+*iJqP#&UQNlCV`J_0N%t?V)HtJPofY1`lNpbWU zHFym-M2Z$O?y`9z(c{w-oki!;mo1Anh7Z7mN4oWmx$*&*+ydmH2ZT~Wh$4ZcAc`EX zYx^RN+iUOUGud>U$*kK|we~I^e=`kZdTqzomhbA=p+SZN0s0BvUj?>W)z{%$YnER^ zegh341|mTL04SAPW2y4skQHj}i*Lfr+5n`7lwn;^uXtYUfXMT%KJ?BzE77OO`#sh6#T29e+N#3?hs~F z@8>Y>wkXT+%D|}4!+r7F$iB#X`XYS+pczFeuegbx&M7Z-qV}-B$RU>V%npPX@Z>%8 zTp#Qq(BH%{tdjt9BXkb-gZ!=?FOZ;a@LPNfG0gAurEez3e)(v0m-x5hMd3qA{Z=&c z_*5L1mvvUW8ehF!4Kaa{f2?i}MX-;xa(W0Lx742;!p;0=yzMA zbqK0I)Sj%N6X(TL2&deU+f#*@_2Pw9h+_hqf(ucF$j2dE5LO|E=2OjJ<%di&$OjeT zM;EsQh*BYbm`*dk*3L2(TttN^4XF_MJyeKNdU;vqi)I+E`7WwLf7oR9hPGU)LZCv? z7*Zk9g`mnodyVNDRGWD0G}9)kP@7nV+C)QGo7fnoP1No}T1MTEvEl(zAx44xg5A8;W3|9;By^w2CSIIw8ZLdi++Du#C zRPYQ8FNfIH@1WY*`roA1g0rVC6)*n~JDpSm7KpB4(X}j^e;pD4ge}m6Y-v5#%GQa|}?}$u57>&1}1H2n4&{6ib0JiC6D9O>ooWMuk z-(LW|@O}&3&#OI2ll)1e=i#2u74#~g0pHU^xfx$F1%xQX`U;e9j6tQ_yCyZCh<4no z$|SS`>{E7Ae~jN516&MkbGer5k||$1P+^AZGWID?;sNRi6v(h51(JHg5ov1j#8i67 z@_ie-`jR#J4is4UGKR3d$;zqpkjC~wq`gTxp2ED$PhpV8TDk#;ryKGlI!aLPWQMH7 ztb%-xe~ljF0_=9vmII4ry+BtZBk_l1sPF)UBG)0dTbnbEIQ*01J06YUdkE~Zf#Dq? zWJ;2Trd(G0HZ()ru7}wThVA7*;@(O0qM(IdDscg$WE@cLV&-+3^aiw5d6Jsz@w7$3 zw5XMVZO*K@;hRiBKBhyxj58(Y>S8*V6D?wSg3KX)@A$8#Pi)|$=G)C!VqMhpjv)HSTe#~g3FF%rx1|FrV ze^oQvDF9Wh9N!31j#KQ@2?!pp#dNw4r~4hFnT?N9sh||jHOhH< zsGo_d`5`6GnZWQz1v3vyPRwZJH&Ceo5fy6nTbX!E%QylwbFqR2@vI=eox&-f>hNLD zWPCYZBGM>?Iurk;JGl9z9d|y$_za3?fB1-Vh2po^$lSX*q^sL6Se^k*HI5JB*!#}$ z!guT6PdLarv*83fI~CZJ+SIvFT^2_MwGf1|+T<|&R~AlWIe zL&=3{_vvBe81ePRw)?W^8=+pa)u^LU;;xeV1QntCvb+biG<681j*`p}!_#PNEDr#% zruEA-^tWp?rn(~LOEo<>Ps4tP;7TZ-E*kAS@K_}<0pCWoN=%$sfKdo#f8YxH4*az) z%423}0A`o^ux$E?1(MNr-<@a%s?C#sZKIY@xWYd5y+hv1&X*9UT+msj+PfO{fRyRA zH@zL{0jvE~qYmb{nOE5OkVye?Q9T3M{mn{`S4@n;_Zlfponz9R6!0hrHX{t`&{7crBo2 zWg`AMOD(>}?(P*=DobHWg1Oku3(?>`qz3vxg!VP$44diEJ+*r(fn85O`o#x0Gec;w`KpU8bf3+miwjiPG~BGW(g>_-kh#~*0gugXYq zIqC5nYG+G3hOWGz52)!s^onUq3fc(z43&^6a47iuX!N|Y@Ksf9P|o}WT2>p28M=e+ zfJa;!2-{!Le+)mpa=!hQfZBA@vH|qxYp5W{3DV)W>9#|WM&wQ%g+87ADoDp9X7ks~ z%j544d^m}*6W;DReNK!~EeVWWcAS^+t?v9Y&xrkE?@6=E*EbcX9Ah3{Qkb2Z;u_VO zXPWeu+a(VkiP9@C>#gd3i|w@^lg>Q<6KL&V-=o7=f2q?YLs=PZjJ&(d&E$VS|HU8U z)65;n-CuBbB6mx*C$is5O)ucTo7&^kx%R^Gk)Pmiv*Py0Y^)3P z+;llQPfn#LcU75~@2MIF*>bnl?eSE4tb`=RvvRXt z(`U_Mu{3IybK2yHAaP93vpc7SV`g5Lo-r;d-d;6)Uafl|?01vD9tNu?E{OFI(g(@~pO&RRDY*cd=N$dQ5zNxx1*^Z7r^> ze-Mj`MOq@F_A9Eb)lyYer3X^nMa!*=0fFU!9Ml2cc1a;@l za=M@a0CBOWayc|nR&6b+u68e9RKD6;B6-RxmRKvGsugaQtE{3-EGsH6yMtCAPk*?# ztfB-#azjb$B54W!Wbf)q$rB0d7^_pXe?r#fmDM8Py37r=BRU>;wNx%zD;Hafi^|Kf zF`yK}G`hHQc~x1tyTqDyb#b}pO6y{&qL|W@!dVmss-jx-NW~%`4}Ub#QC^-`RK{>e zyvHKm_>;o!ib`ooskOSQsMt-(RbC0rMiDHMUf1-XR|t1E>%5t>8NJ0)0J^fGf7m_d z=hGvyopfvmYeN{rjg)Zs$7W)>R8h1DK#c~Y)Lm4?YQlWiUjomSWtBbpE?n-qii+c|@tGJlsjxn=~Wl8+XzA7gkodVRT*%!~(}EB8s8X!Y`X2d~0}Gakr5-noO5liPk!3Ld z#p%Nf;ZGHZo1mX0#tJ=!fyaHjk%MOP!4JrZZqM>E;Avn+xN9!sPZq{M6i;({f!IW5+mLE-KgE z(do|6V9W7<5BPePo3gmQf1+eoRar$&B{U5AmbpC@?sBMyPb`z&!8vChh=az^`U;Gn@nh7Pk1A8}dgf5^+PNW1bEqplh~ z=IZoo#%7GW_PXmc$4|(bn0>>IHv6O;NA69N1?QBhuDof}^JmPQHG9t7c?CD$a_jsB zw=FCzT2x%(Uc98VZ0WM{(1gD~_Y=hdn($Zq zGrwUD_*cLG41fIgg>PgiGVN!h%<+y6|6F{-c=wFaL@i%xYS($|>hHOCz3)Exe*Xgx zHazt3BO4xl?3a)K>eo;F=C{9la^vrxdit3^Jp0E@fBN%Zf1Z2(uYcS8Z!i4)#h13c zymj05S9ZMm+Ut!?JDYdy-qW&o-y8c696a=9>)|7Bz5UL+@3kF$|JVl~essM3#K)g} z`q@7|S3153oc!{uuRFi__Pg(Y=sMMX`pl1Ke>$iB%M-%I1^&;P5dORI|L<=9KYc>J zc>4bn^cRE)f6JFoz!f8=vfUP8lRVUbxFz9ERdf^h&_+x2fYl|q*NuN zpJ|mVqQcW*1&F~}?kX{uV3(Fi)u5bBk;-X(Q$XiO*Iwl*LnSS*$WvU(%irW)*ynZDIXbH`a;uVO>yuHP(goV0kP|-v(Lo(!o%uoi3Vht1@ z01+SsfB|j*AFu#q04G2VFhoQnK7&aTd_n1c_F8f^xsi}Z$upBl`(*g8B)=3$y8vH5 zdE80bf1U8Xhx}?PX`c$;YVt%LY0rbNNPat=v`>exM4rqi?fLM1h-{oe+GoIb1^N9< z(moTu50j^6k@i{eeS|#Sa~2&$zeIBl8IHkKV~LFr9a)`$FEu9Qj&URv-8-)(Bhg)R z9l4DRC1maO7;*!;<0x+sa&54Ff24M3P?E3~6#8TV;cf#}Xd)17At@u-&|o21 z3f0X6A}oXI3dkZ-4&8h+DJIK-EVqynQUT!HO5CIpz?@GOlPVz30NrBGBS6l(ju0Fmj1NGNTtkQiZvI$8&`@AT1|j&qe-!+g zJN#9&Sz`!!)Y>ZzdvXCH0Ly6(V4Rxvp(ANYmK4b!pVPp zXLtKT*AsYWh!jtlU$wG&p^$5L%y+MXHD^Up`TS~6@qD(wHGgS&S%rK4s_VwhpXsJs z04!!qRmq}@0}=7j#Oq1?xEK;Ye>SEHN01GedKNspegjO^Pfz-ZvBXdk6CDrv%)sPT zy?Jjr%Z2va-we~W-=H#`nu zgBIY3kBZYI#gL?Ni6m)eTw`oiOd*!B0<+JAX9wbutS8B03^9z0i;CA6HN-dxVFS2I zi~wIOg-_GlNP0sXy&-?^vAr4-s}c&62gH*BVgeaZ0xrZ47;9Na5j^=LS)T8}{|oR}jSS)nxyI>9CxkutBcs3L`1Xv$ ze!2n%>lM*?BI7iDjifK&)OU6)=?ir4JJwvKiPe$VNxFP3uyow;Br<&005V)mA;U}h zli}m~k>Q%&WMF4Xd%spff_No^1Q~)-5J<3IQDm(1FYbe~AbEz8TU!fv4ct zNNXcWhIqrcaK5N?0iOf;OS=qT-YEBnjdTJ>pq?Nl;%RF?pchhbICplF9Jd zW}rteGTbwyb5Q$$f7bqu7TQ)7h>l?(Hr{@m_Z5tXs5r`-(Q%p^_2fptA!e+$N|OTo z4tS*iUMUclGM4dvG95?oz!lB{=qH8yd@_i35uO_E z_DtUmarC>6qZfZ?xObCtb+wFwJKs}9=D~tPbYVeh0rd7secpZ=;qNcbTK&G-e|)k4q-zhZEc5 zTa8t*g)#XWf5c-Ehj*+tJ(;_~c7}n@c`ZDJT=!)B5gU?Q*lzkXLMq|0{+72}^7p8? z0jaSdJ6;0W@gnK1*-R3kKUxcccR-0qrZPE}%H&|14*>3;!(*Po+Z-5A28wZHAdKCC zFm?w*+XKh;Yos!KCL!Ozvk}`ydHS}I;vE@3AjLr9f7ThQ^o4AkW7}ve%W=+UypagJ z0kRC}JYZ%tZ*x7wy7n^B42qD641zJ(Kcly&S10Ym#+dvt zK1h$ve~Ss(3eN{zCZhiYJUcL~*AhL9<*2xRM$+%?MACa#5;2Osi5|vzWE~pp&m;Qu z0i%<0hlzd3pprho%e_gToL(=)cN*JcTJ?=u4nqNlp%c{cAK@u{l7-QqS_ysiWDtfv z(Qy=pUI2p$VCV}l^p2`Slb|ID-|EvTj|1-%e}JZQ|C3y{h^DAGM&D!-|3XY5r6qfWWYNO`Ddei^fH!$~q?HibTG5dz=G?U$84m*_0t1NL7@(RYx5=(@lijr~&>vgkw z&Gs?XH+4Ene*Q;BipIi4;9af9kCj+(lLP^76`J;-uvykH=jh&ZZ}i&Jset zCO4H=Llbk0$|X1H!|GnX99nn5LcvZ_Xbm?(GlT>%b*|e}T~=9~%SkN9$VlevW|h0$Rpb{~1H*|Q0yORc^A4^<7`MV{2okldct`ECy-bVV^k&Fvv4u!h-X#mhLZH$`w0?@3^% z0~F`g)uMa(>@w)PYDx|8kvy%-U9{pn)X5Vh7wbPSzC<@Wc%~h)zR5kme@!Cl=2Vo@ zP5}6e-Bnl`8l}gt`Pf4jO}A896&w^Hgr^-rl@SwP8iJJ_joIBIa z*lQ@`b*LL=A#JD!XIBQhf0eADoJvhk7+aAYjMK17k@M{MGc-{#(9brlD<=;srg4zN z9vO!8j}Y^@%n*aI10nBWoU5{OnN)QXH_d>~!d=d1wKG9}a3SQZsFFlGnM>GM#4fj! z>qt&{Wwkpf#z@%SYRV=y#~S#P0^zIRvEf?=%rM8oQv**GJT`b{e?$CT@Vo%eXYj=R z32V{7%1H;aJloGeVK=2{MbSbUT*ywjt2qaiFI*BWLNA@ON0i_$TT&|i`^rFHiKUk+ zg3^}`%uD2{?jiU8|MbZp5N)5QjbY2Lu7*;ekB3x1NOu1?%JC;6`jI4-e&Gfrp3jaC;pKTX}dB z4-efYu@NjV7zjGZ6-w~{zhtKkG8xN~N_}4|#EBL`UybGh{kB+-~ zP8%Gj_-0PL4TpV!FQ>qFJ{q|j$K80ImJpHFkC#Q)#OpQle>m)?-u%&WI$->@JZ$IT z93IZ+WoGjDS=^n+>%W<|UC!f|b6Vs20sF|qVU6MWYdFlc++D}hy*#~+*MnQRxPA3I z9{xRt=NaDKA9?)C9KJ(5{XL$Z63b{2q#6Ge@nE-v!Zp!mVE@-2{5`jXi0}{y>m$OC z^$6o{)*2(ie-HEsKiDJO&?Ed%kMP4i!jJR_BYz>EY$V~ajGw4O?+b)`e*UBL|9i{- zXV?GlE&o6DV=m-!?LWv(^+(T7IX&9@uj+@*TP|(K{@mTz)KOFjte*V)zfBP@_T-+FkrC;1weh?bf7kD+Zw4rMGv2@LeJfe`{?hly zo}d4%f3}a>PZM&@U99BTr;d;Q)hI#^F0dY1{I;srP2p}|?wYxq$Xz3M_1q=gJ^L_oySe*4cRRUzlDi6b z+qv7y-Cf+>@o;qgFY@qn+`6P%CF@wG>=?VfXLO%;pl{D$SrpEaBOE1!JFzgB)Thr5%zk!vS% zkua{ov*1AqgzIN~jm1mwybaHH@FYV;_*Ri=@GRFvm&0RnTuY)U5WYR-MR?wV=UaI2 ze}%qR!;=S3xrUWF$JcK`S4;l)bIA|PqS z{!SWb4}osmtX;Qb?P5Y#7+K0@>oyzKe`>Q>ZK`bvZP`vyYkDa`4@MU4K=%-PkO&Hs z|1VoR8O3>%L%|3=#9Izqh=0HZJya0c(h31Z_OMmIzx~Z&a_>cG;LGR7+c$6K{hv&} zJ5Z;Fc@5r%?&H`#_+TfOLk|IFcX4chpFwdPE5W_a*}%IC^~KaFykLH`l))*iHh#oRH5yoiHdBjRKDGJn%zZi7BY90o5yt5h%WlP9@t z2jGrzj&<OQQu7JHNK?&G=)!1tb% zy#-L5%hooE>);Y3xVyUrcXxLU?hs&t26uONcPF^JdvFNu!SC#S?)}eq&i?AJx^LA? zO-;3|?q0p*c~|#3d`|y0;up=~Xe<(H73M3GLiySr2qb!PLVI-!sG&NRC3Vl!3w!#* zCXz{q^erjW`gk7w7PB#O1?(kO)r3(ZF2n`**Uy-+4MueU{8qlU@b^Z%BPjw zl2Ffm?MBtKC#J%rv<~ZQu}c3RZizA6@KZuRx*GV;=tvC)HGCT;=tUM;T`n<)NRuEL z6`b(#jDVXlCmtvBuq+|t{qH2Z5%VIE>ZvJtO2O75_#;fc4Ni9qb4XVXR@il^z1^Js zc5~1UMzeg{#xXO$NiGTmxo`Ll1Ch5v2yCL7)*Y&5xF_i`m<;o8_tYE zrI>zyf_lt?_shCEy3a(For%GEfaf^k2I9YhKLaDiV7ZQ7Su!zqGrSgB#@=%bbI}Hj zeT26^0&-wK3k7tG@_P9mo~V;A8rPs)kICW)sUIR);X2{_oKjCWeQwR%ZG|J$yE@|5 zx-o^9Fg#>2CB)c0GJWraIXP#jP=5gL`aoD7heGKbJC&Qc6lISw)h}l`Ny(RQw{U!c zzXNJUbqUmA=tUzFdPlu59^U~!Qe4+!@lIvt@uNDbh5x{ngIfEyyR$Qxjf@}uB}B3%LT zo^;8->2?44nd9-xx0In}D~}l2=lF8=*Kq8jr_@W&Xb(8KrV8Cq_=TMY z{<)Z!nV5vRUa^^kg}DKCAqBx|oIo4p@<9lIgKW}@ZA*@WZPJ!)TW%e9y|%%9YHGb$ zzc!-TT$XQ5rm4P4WZb%{B?!a0PhkK!`*|~rudOD$`aovcY2AMKyj`H>vbxPVNhzpD*)G1*JP+`IKkkfXK+1h| z1dZg2HYGfJQj#YwDUFRT?1=#80Y30o0sd=-LfX9;C_?8ZOqy}wBt15|M4!7_TB+UR zuQ6ZpvU#+iv8`n$bAom7r8MOl7M#!B;3G}Fxe%-0TK)A2{OuB_IwSB{2nOF^t~;V) zUvmVkoE40uE)(1_dXUm?i(=;{fz{ZHBGL2T2q0Pt_;_U{8VI{=YJDNBs{yc0&FXcr z6z^;rE*hSeP{+Qh?2E43@QCwGoTYY~iQ|=v7sIlAyp_eLPrXL050KC1!*24#dps?# zsku~+VS?(0Z_=nCN%BcKF3Eyv@kO6;ng<~!op8ai@-klG4k-KuqpU=* zBS zWy=So&Kaw6lUIu!c+pvw0u6Sa5w)wTg6d=(jJpl!l{%-i3A#@m<0E%_kXHY2XcxS~ z1|8jld&-BZ+GJZddr4w7G8ex&vPQpt`k6y1J8|CU|B#K}h|oVfW?L5NXyLsn8dV|3 zp6CH3WAWS71exBx_lzw2Fh7&7nC(v^r%&Bfg_gqgZ)JQr`sLuC0HE(f`-RFC6>ME< zv&v;P_(;%U*<_0Y*Kaq)b^pz6%#J2r;~qglsjQ2^xPnGe6D7ti9=wmg(|Qx(pIADB zQZXz4cTROsTJL{9pZ_-|{x<^y|0f5BrO5v-7Vc=e9mfpNq^8M@O;S(~mNWTjVtj`dUYBXLJQ0)`-{v$iEhAo3uUfLQ$gmro#FPm9Kk4#mnI$+nA0^kRZNJ zYfWlj=%eMv1h!Te|B$U^UbPA|)qHeHewqaTe%P1aovTzGud9DioF44VONO7-pG|asCj3{0mwwiFnyx;Vu61l6FC zK`(ymuRVU-ceC)KK#a4k_}LNU4F-EEDh-_r;u!)nzc)BC^Cly~8%@cbryb5|2=|dA z!5VWrEc{?Z1+i`zh;{8E1nW_c10sg*b0~7Gzl&luI)?ATyv_Bs z9{@G8C=Webn*&c_Ci^_um-3Y)xcR!K%EshJU?HH{)CF1Kv87Y{u_@B1vBOU=1#JO_ z_Vui(Y(Aa~*lgo&jLhOhUD3K2zqGTn)%HLDvNG5zTHH@xs5&%?iBO;QO(M0B^e+b2r-r8*W}0 zBVXLAhOZf(5&shNXeGg3v!D%f|9{yaKrk@k|Fc6pKr-IJ!{z@I_5a=_piQ!^xcddw z@6wQERG>609a7-Z-jMOT`V2zx-OmV?$$Em^ymQVDU7q)sj4CA^eln(jRnR2@L~Mq? z2)P=#BX(`UX{KFXJQ%SyyqrRgQ0Zsi<=XtU*p7gRGe=&*_^7OXpJ1lL^acW`h+Wh+ z23Ws29jh~4bu}Ods&$j%z=s%Fw1zfJ?2daq#RTzkGr*t}gyF*!)Shl>G;LbgRbO!= z2gB6uha(1#S+jZGZA!14dc2lb4wstLNuic%rO8J_VUT>D#+ud`J;a2mJ>=zM%|#5Z zHOL3jW@y2es#YpyBEkgjfc62xeY(F?A>~Y8;@*I^!gizbi0V^gm9gOHBjnz%}+n4GlwYb|X;S#>`>l3z6X5m00 z>zBaZrqecmd`@y?-pzTgAaF(9{4iXJE~#V^NO$ueKbIpbf6~O5?Scav z`2c!7Lub1J4J6VuiG|2+J)_x}Y&z|gThJDRJ__%Rj4dSGX4jr@8fh=^zZTsykio4z zqx8ZkCR(+s+QJ)yOSM#0hU|wg@5c+(%zG+=ocJO(gY;Y3%>x)^Oog1wOf&FGr zcbi=io~41?^qzdrLgTA9;tHu}Tw~ATvs*jMmc029CHD4g+ojv=C~FFoeFrD}YopkN z2OTbeHVPule`mc&A!-I-r}(17VgR!=bT#qSF+m~l{$@6L6pIVa*7#OZ5M*{m)>dVx zQo@i7#UeCY3VkF1IZ7u+xans^pNg37yT$6F#A6-C8NkBjYfh99~ zP|!Ouzo%lVvayP=Cktwjp&Zn;U#w%GaOU7Ws!uDn2$*Q!2Ar^j9FBTw21%nacaZxA z0!dISGrVy;!l9#PFWEk(Pk=PE?4K{arW4d!Lw3-Lj*7kE%sD1{y_PttwIO^6GA8Z> zhzRUHrACm}&@ikB^k!HqsX&ciMl!wF7t&WlFqRaqgD@`mwh?8#Y%;$C`(T*XOwk!X z2NCGbFKTCGOjM_`&#tn^9YM(PF|FG8OSBdu&-5}tx5HTWX9)%b-`&Kn}FEY*iz+N9(8(y z^ktP};ir0w&F%~37m4isDisrrt!09)L~4#1$y%nvv-W&{nkS>wTNqV+qIlW`d~xy! zlOHwhQrs3z)_(`dpCCq0R@Egf6>@F`Ezq&Kg6yZe=8Z^DKS2WbMOUQLovgcd4 zT4=jW%q+Fu>p;t=QUcelM@lb+IP>f8hBXXmh*MM)8iITvJlI7WhAIBACaE)$%xy&B zR%^EL&?ifCXUqcxFB1_re8|Dj+YQG>MVJkpCP=4kAkHio%ezHs(kS9(t>`oW8Ojyp znTGSOSt=Qo%Bs(8$=tnDaRMxh3ca7XWFC0WMc@1Q>27^J;(#Z}Z8y9PhthVK3%@U6 z;0cDm&S9=yZU~PsBS@0;X$0-uX$Y8nbtStLOp^PAor6DxC*{Gn{27J2M8 zw++vb(%LIuRSL97-J#|AszJT{vy5HQnzw3?me@<1lg;j{G{!xGjzVue$9k5v2W)=E zabbl*eoxEsHkmzPy9Oiy;)rPg=`H(j7t8O>e-w{bME50cDEKkCy&bbn-EUeB_<(7lc`IT}y# zZ4uP$ga;8hl5OettD~TIaDQNSjSgLHFt!g+{}x8-GNAW8P0H?iWn1MQ6DOq_Y1yG| zN2RDC+S^woIXuiMJNgLeMuTECoEFO25am925I1=eLnfdP<6e1C=hE)4ADYEziC3~ZJ z0-X(+4~sKovs1q4bMJ5z-!%JF2ZB7L`!25vA<+!Ul|);H2}Z-Dh)FMZ&n=lg{QM&O z$X5YwMhNh77-95byMj?-9D>DOD1@(ntMVoZoL??ZP+(x;@c$4|Kpj}@lrNaDNI*4Z z7YvYu!k*hQYtSl$Ldu)#qdmcsk|p_~t85h}u?{1xLoaxIen`n^e=)R)WJ*Bvgu&0V z7g2+0Y1PEm&D#HQ-fP+r`sV*EfUr>r)3Xi))_#VL%RZC37Wf zC5syF1*^}PV@%$VlBJgAic#zI6;K;QeNpTH4*}lInA|CM03vVnu3eVK_8d!qG=bct zil?L`o~2vlPA97c9t=Gsm!opp_?RBsmEe}@VX?n%Dw`v|CV+mY+@I9~$tTnU$tOq_ zhL##{MNGKg)6e^W1UL}Em{PSHKOSf#+^=l{nbFZ{^{txjZ06dmjud|vs5@I4<}k8n z<>yUSkHVfgffA&2*XGyW4oV$K--)1mUK)#ewARCYpS5`r;xmplsM|f=QbCYpYO$g) zcG8F1%_=aWy!yin8O7#JXjw*_C*`&d%#c0lyMeq$n`=5L4s8uZ;T1tg>%hTz^&aO~ z_i`~5)dv}BR+3($z9jDsP#&ou;+mX5{Y0fvUDRasD;$z3T{Py`&4=Wx)AB?-gNQ)M zElL6pddek!8cd6d#e@4KzG9fJLpNXDM1m%!$;A*R&yqd+ugu=1>(*DX*s5raUC}!A zFZ)WQ0`BzKFTX*@=wyT9Sb68*Ne%j)(xLt5<7rn7zSoJ?L}nsy1AhvYuDPQktXbZj z7E*1a$TJF98!S0me7AEiI^jskYUZgRS=MEAS>D`zSdP;zoQz|jl{VZdjTUfS{)9A3 z?vwxHYhilFlteS_Snmb)+J3omAI3}_OZ&;^rltO z5a>m1_LTgw#yeed$=t{!@`}b#jAcL_ne%ytkJfJM6QivAF#(KLPqn-)Xj5^{PaHEx zH%z{hV=edyo!3s5HkV`@71)1pJakg|mO1D5eEXOcn4maI^OJWKY0i!4sElGssYH&@ zh;qt|Q-S|$7udIu<(vz$F6(oW7eq?-v)wxqk~NftL5I(n7U4ruT-JW`&$z=L$S2Js zeqZ*`CO)7+b_71${gFkO2P7Dn>^}_&3|v@Bpt9n;66X5_LYFQB07sNl3H67qXjy1@ z`yi|^MnZ3-Dmvk1dj!pi&Ri4uE4{MzF8W?W&#>iQn>qLF?2pUL7hkGvo)R7*Ki@YP zBLekkw`kbU_@7aD2%Y#3Jd^vVoe^#j4pnAYDvY+k`Xl|J>S5S7n3)5t1ECZl50SiSiqgqXVW%cDvPKR5DdY{>JTEwc*JX~h0YU3)+$$+2XH%1r(_#_vPC;?r!eKfvtN$2w#q zc9pSlBKFCu6ZkPhQSLMcPoZw@Cl;U}xsp;j+qAeoqhDwj_@#mwWu`A0xG^Kz-Tw!O)bNw43I(czB&ZSA@ zvAC5txWl-+!yu=8A%MkAF(iPcNvpub4tZ z2us84UpY`O_Le!&>FWDYSh=u|o~uyn?F$OUfN^3+OWH6~R7~21;FN%|fRP|JgJ=&( zaRdio;)SH#ya$0O$^9oae~ksm@~_^_BPS68zTk*oS{5XuHJu=lyC$^SGXbEW1$vj6 z2Sy5a6=y1%j={Z4rGh!#xL}btZoP%F!Bi%ECw!ow&_=b;IcEir@Trm`WSzN>becN; z7@{pnR_pe2o+2!!*RODfGhhU$t}e=04%q~ZtD9d24USxu7XmW)!)u?B(3AIso(@%F zetl=2K!YNJIOZ8FjO9oLt^1qKD#X$xY7XJUtUP><^tP5V3sL-<~ zsC?7^T89C|xjsaoVjk1je-|76XR#I%SQcQSz9NP&PVn@X@xoIHROq_U0F9028LXZn zEJ*Qq$_gA*q-&cF!1=EWg$7%SH@f%m-@!Q2tsK{qxZ8GLrKFgejhVL6R(Xyl*j`wG z@9(_OwLw_X;>Zbm;KS&{qS2C{V|cP1L9%h^I&%+tm{l^aAW;N@i=A1{WTD`|;k?+h$>*0#iT0{0DgBKyyd<0MXbf!*!-8JzPfhuu~tINVlQ{UJRa3&#o zRt4j~Ny`erUqt~}7u;;4v%e{M_ie;0bnX5im!2WF4i9lN0@ledKYTs3xc}}PalK4~ zo5N_5bM}?0JTCsuyj3L8yom}&f&q9E(IQ_%8}O)%dOoQlc?nSlf!j=L(4Q0yKtaq3 z#*OrZ!WH@Dz`5*Exr_^Ar^^8;X*)tKNYg2?VuDivZwNIPW=nu_Qb^>k3Xv`pqd1cN z#shIide?8(`Y9sDSa3@aXTnpYsi!!R&Nj!Gq`3eq9DM(vU4FUnem6TCZz*4E9MEWy5@4L4# zn?N}EYPAm^EWbF1>^bV;zmQ+(XXnN_D`N`OC|TgZtc%Dd z%>^)M;9%+q{g%Z4WM{*km#S58(l^M=?=-?J8j?1YbnVgVa?nf(5k{H55*#Kp=DD5Q zDE`TQO5m~6W4hz=@cqxz3F3FRjifqiQ}H1r_*3+1=~l*_ERkYuf{b#st+b;4Q($1wyK4Z1&+-?u0Te~4G+Li7| zn{#v_rrL=a(fZae*#<3tE`Qr?H{xwjB(wpO5*X-Mfnv>a>vtC{K^~bfROcl9YKx6t zlMc&mFJd^Xo))OG|Ok0 z1LgKBz&YVtFLUYlW5P@KdrB;+#DzP~y0W^mS<^jzmO{nC;qLP^yY82IlfZAmjwQqP?~KLDv?X&cfga;cU+Qtr|KB705rh7CsDZ)CET;r z1L{0P0+)bTlWBfk!Q8c`H+%F^>3kp{1wO^vFv*|2DN`iTB;F)^mmj+}0AMp@Fm(g+N*O2Jsgtq-=VpCJ17( zl7DLG8A@1EV8Zra#@uj^2mXc%g6-aqF)@E6++4X=22_mMv{f6xy(4Ry$WFB^~OU8U?}h#cgxY4OZQRs{Janl_zG1Uw8(A|e#t&zL+zr*ZSO+w zw6Wv)^E!49ZFo=63TH8{H#C}1Z*De%g+Q+zFT@RaXT0*09brxmuhkzHavX=NHUK`v z)yqE1xR76Sy#cu@{Hxbwcv~0Mnd5;^Dcf#nDH z;%5dA$)cIGR$7X2tBU4RPlZ20c^>KlX4y|jz07JZ{B>yO4iUean$A_sa+m5EmdkDb zWK$~xS>ES--itLDl*0jW>HOFkq-M+$fiu@#gWQ3_H809W5uc(LW^KnS{6gtd)*_AV zmTk@{)!XLFFNlJOaKp2Fxd!5JgAb?DiM*mT?nkie_(iL~@&ZFb;surwDRAy#TLg<|HtTYd@<@3J8k$UyJju8p@FG*Z* zyE|(Qs6$82)5NJAvT28)uNvSQ^X%I1)bApM%O0(IG0XmPrgz1b^KAp`{OI|$IX_nd z)RZ$+dm^l+vB$PfhhcUNn9$hagXu9Yjq*4AM9%Q5T_!zOmGYkcHDswM$2T~GeHAK; z($@>+4a$cu$DVARAuFD!b+=@p>*a8LWEvRkv=PiUF+Rlni1HWnS@Z`NX@Ho|8szf- zr?m&tq<=pka1fRND6QD{1;>{x$zKJ_vDX)}$;hNYGwM^yK;c*~1EFlzaqJvAJ>_%7 zLLhm%=U-~Gho)5O%-CYf$06;6iztW50c^?GdBTa_k=}Wq2jR2q=W*#fV=37F@6kAO zyzEW8IP9^a8c1X?#mv@5(6}L;+Kmo+!!Gn(SNuC6tXB*`?FtUjl3n=mWUd`b(Fyu& z7W#De#Wpwk>Fb(G=OUT2Ne8#K>O?Vuw-~Wq^;@RfoRvT&b~V~I@@757XVwjiPzN{c zH3df+oDkcShysf$Qaq|~d9HIU0_bk)*i@oq55@B?h`?E7-H6|SXQvt4Rue;7^&Zul zZytb?tvwZ>80sizH_AZlZyDnCiwq?{u0rPv6v31o4vKgz(Pn)fY@`#5rhYMZ9<%(c~HKJmd7mVZ*$pSntR$=(jlMs1~FoGlt$F?_Ur@;PLim z4~dm6$xy@@<}j2y(&qU7V4$Pa{Q6Xc9ci$N(^bB~G>ztmp(7>uprpz^HM8;Kgz0ZM z8Gj-;790$$00ohyD(fmB@u6Rm0=4Fdfd($#`#lfZq!$r;2o^ef ztq%~Yx~(v83ZHOjwie{CI7@UT4NCx?gzM`%6yv-h?NkX(G_|%$_pQb^?d^7X*#XR1 zOF2%pE_b&({0A*znU>16mUuV5wBBh*crU7is!?mY2z?OukcyI{Ku{cJJ98pX0-_(! zp18mmVUI-4YQ#jMpC5!fIH~pvY|LGOg@l;Pu~N4Kd}fKYO^a)XAKOkGTP*C=9)b>- z*ao7~vfvC^ops@xv?P1Q!ny*jK|=AFR(!#@R=#UWw$eQ$+#OB5>8d zXoD;%U|*~!Wko<T~lk_b7yTan-zn- zAcB#U-Ds|Hm>XZBg7dTsvb8qHj%xI#Ud1=Tr#+x0BXo2Fja>?7x4Ti8R775n3r(T& z5n+G4?SkBe(^t*Zhxfb_9^py%jN)N@`B~;u#KUsWLCE3l#C!D}_!c0o-f68Cng495 zXxC!(d*6dcnKW(1Y2o^E9b|~>I*irW!xL`E`X(nsde6P3?2`5S*3YVuK)WY38A->K zDV-u&=q@c!$JtMt7G|)up?t0r^ZV*2wpjJS6I6xAMG*&c@Xy;uj_wywzJ`tzA=Agc zNNyJli804hhZqMw@gl~dq>sk{{e!pxsT;*B8klIQm>tGE5o}LIZs@npT~aqxD!$}2 z0SVq0$P>2O8=A(;v@@1_$M9?v0*Fhvqx>aHAmm@XCPffO^aaFc&HvO=fBTWTj38W4 zeO(1Z7*!wv(Uda+ZWpUSFF)ivjGEd2ACyB}E*dIPrA|C_%G%k`q{N_yP#@FP;E&_L z+~nkGzW}0NmRb*Ij2bKYg(`~)Cp<^4ijRf2x6>(?U=nq$h*UhqJQq%w+!rnrj+i_a zJcN$?`_xYSyQzDrip0?c@Qr(^L2%wc@-XP`_b3I9k)cHQWSSLsxT&`fI7**DbU;L0b?QDU;>IcE`^H`=7 zntGtz!4E;(`t=NxS|lFf8~643bZzxzr%;P>WFEsB41=5uyhLz(%jJt9RrDu7o9f)f zU{7{7?#mS_02OQRRUWQ<5Cmi|rznG#OjWUly;gF)#F?+Lt<0|HP(rFbPV`SH&%18s z%u|}Bfo$e}zll#Zc3pr8w2;Fx@H;H-ZkZmS~Nvs=d znANT$>mU8ZNfROJ8f9jMpG!i1e+7KgP!nxxtUs=uH}ICBE1!%d)@{7;&iV;oC0N9$ z>Wme?{QJpf5?c*u8K$FdPV!hulXaTZ6f<=hwNjKt^n^q105fyDALT#`YlanAi!wzO9x}$?f5-mr%tpDVG@% zg}FlaFwsT{tIbpNqP%JDftf=66rp;B0bTARv|%d%qRAM=P0X>CpSZJ=l#0I)jT)Xm+MP7Aiiavi=Btdn}4&#lH#kl>;UwS+BCZL_|t5Q zGknqrpG?;o^lzh3=`GPyC?t~|5?`S-KmNA;vQkpTs6jgl9`+v+Jd_=l zmhk_0!!W|`FoNzde(o^fj38h@gAfK8hBKv-4b~Ls|B1c~MY*hRggBs2Q+i^2T9w7Z zELI$AN}7RDLwaf&TqYp(3lj^t00e{_AOO$_I0HliGQVVF##%sx**Jj3f1v`C<|Ll{ z$^A*n`THudTm2V2>am~YO6SxmRCV>b@lJG)8#AjTgqO;fba?XeE^sM` zD%g12FX{eH(6Req4Z^?8{MnTf{nfwnL5@q%%Xs~iaDfDo2n#2hsoT8|B3bZ1Nv?tb zkqk<}jGe;735y3*o>TxCH<`gAb(q3XSjrd)ga|C0nIr~sq2yxn%v&k-Mx%2kK)Ji% zPS2Typ+J;hqPRWHI2e28Jr0(X4GF8$HIFD@1ZP-E8g)e`x`WZ-A>sN+xRur;qVvlB z)dkURA?d4lQ%?jfKFb#INCnv~t)gdNVQr78CK|w|C~eq%b}g5tkK8$yJJD z4ka$8@bd%rdgajqNsl;m39N_N0fWCZI!-JFsJ_%?gM*rzeNhBT06e~ zqkwT1H$@pvXO}=3K~R47L`3p7F4NkBez;bnWcVJpbe!CK$o1FU8*+(-DTC(y=D+6N zN9132<-fkkz`#;IToGYYzHxyttp8Yv57hs$67r$xMx9LNy%g)ocf(>igOXRdR8g=* zKYy^=>qds{)Ni-2C9IB~Ek6!ZeJ4bgEiGSBuB$Ln9PAEsw*v!vVY7Fd9_d{rDyFYNHx3$o4>DTE-NL^5WOgIE$m@RtZ|S(j4Jx2zc=*3A1qN73KQ}Bk zP-FA2mwO+Z$q$^eLNp9YI#z2^^O)3zD!HL^iB?~qNp|{`uv$io^*7E!CSj=kh-v>Y z{0?OMvmb~u^~Xq<#l`~DBGYLImgsI(XyDVZOTBEKbz)>zTaU|w2k>#L;VAQbvIh8Z zMfN@Gk5*r-lx!+`vY5nfWe`JLk|Vh@5S>Z3lhjFiS3PVTy+*o|@j7GxE35+DL8g=9 zx_N*P{Zgir>biRX7JZpq@XnYc`p--#Tt5at@wpZzhK`-3IG6ARlijFC1mgu~2#dXL zd19t-CL9uB$wW;UKLo3BFE_x=Q>@qfvq~*^4DInxQv}Z4+5m+hELUT(Udpo?;E!4VNXb7jc0y#KKr%w<%{gv z7(<>F%Et8YXI-wNF=(gl4r~tl&vpmc^Tiy!X=u|S0(`$$LoJ8-`KH_r9S=W>^SjPe z)*2!xlP%2#+H)ylGTL3ZhEgpR>#F!tnO)QQ$(6p%xM&1dk-JH$s&O@q0`7?QQ*DIveb^M!<{d^?$d;owSH zk{f0aI$$wQqZwDQJPAHx2dXyQ4{SOsH+u7$9>Z_4U>{GiHukZ|`7bsnw^P~ZW$Pl$ zQsgf^S?jjgZffJ)Hj}tD@bTpFhCPur+pS{G)$=1(>KouL7N(yuDqf-;yPjxC$R3yH z8Sa||Rk=!VlnJUG!6wQPD~&FbF<UyzYqbWJZ#D1Rqbr1k}``5{>#-DnzS^f=h|(yx?Rx&j7}H z`Z9Q5$PZE;2>8A=AZgj0!6%jjX+C~y^-?v4AE^;#{#jpPi3E& z&&$Q5w~EqNgha282}Vw}1RU{i_@F+)UgRF~>KxZuVG?a$x*+L#znah1>kE;#ECgU7I6^9g@e+7$6=U5v_o z4u=vb=1L-Bn*N$#f?;80wU2MY#>&nZ57c(uLx}pi4b7@3+Z(RJe7gz;TVGB@8$77Bra&)#l3#c)}h%RxXDxT3fVtG)1`4R ze_B#`(zY2v3{AEk$yPbBT+sQg;-cuq_c&XP65K7}aZKs$&};ZbmJl)zB7N8EZd*hV ze9|lE;@+6g29B&$G}F&)YVLB>%Qb?@>H+$~;rIhgi2$U#wA9?d9zci)I^utrI)7O@ zP6LS8DQAMPSir87Cru1Z5HJVnUmJ<GO;w=hn_WedyUW*82ua&%Yv38rTY-W5XG>7$=@~Is)i= z87u-%d2y=fa_)dl+UOkytejtvSUL5de+)Wn4IX)_4#KC>D)cEkAvA~2A8R^=AY~qO zyKtB80{M|kTYo}++NtlHx(NxFCA?TdL9Y`dw-TOgBEWl; z-V=!C%;jULnqJ_mid9%afM?5}Z5>xK??5wUN+~I6cMH)_0B?(~77x>9--dRObW>%4)mCT6Kuz>%(=#cM&QTd>7^W4JA`Z)|;B z2SPI^5YDc!en4H8b43ppHss`lY4h=r4ck>S8>ei@<7=@mZ62)ba1REo$^g@`2Dnz^ z{Uj*UG~<_`+SX!`Q%Tb}3$yqJ_tRYysKy87t*PYO&h6&Y8DDbT`3B9rMsPdYy5;m| zTIZH5`=oJ~QfCM5%-hVpIaO#cmL!|lfI0~o#g-O698=U+jl!pFzbI*1yXMo=wRbij z9GmCWRb6z7i}gD**t8kX51njl1k#g&*yNr10jE=qhK!=O2<>bhOS5EhYMdE5^v$9T zk)aAe0v70GcnzwM6D!`s4SLr(IM#&JyIuxPuEUBw-$l@!M&Pv%!)HRbXh6t_J9Rk{Ej3{E?T7c7#PBTgEbQ8` zswzMFEPufFyvf@+UqEhsO^IGB31s zK+7pwN^a=6Rq<#6C&6x>@7>oNAol4Dto6*-0iFCe#x^Chf%gRRsBavBf(FHnw40OkWY39^v1atuwIKQ&toWLvI5*qfcN*fs3h5sSos2 z7I-8!0@d!{@oM;*6(Ig;L%xmG=Qgy%pBZVJ(V!C_VZK0io^VY5&BsbIqzsY>FJEYrkbs^ z4eJO=iPwW?$Qm^D7bnxAWdMp;97&z2cV?Q&o*Xok-NRrr5DJ$;8Gg?;X*GdDqd{msu-^?leU+~31m{L^BjeqqN*Q1^OB|1BD%}M$DHaJ#IF5e#HQxe!e-u ziDc4mpKA*7iubc9dY7-W%{B#Ra?+}ii;k%-apj?ZB48*88 za5Zaidri_V67p-MEPzGybjdTOo?Ky!CA8_HcHn$WnOOP=pR1)<4t;gfJ{QP-$ENf(26my6_K#55wl9 z$qKjec{ns>LhP5mPYXKI(soU*k?-kcS4uas-5b02UG@`|5wwLmNvdg$<|AOFgn9fdJW92H9vF?*+F`gl%$ z)U_GOsW$N}1-iDEO+rYXDv8@&RI)+r`3C+Rbt%XZ&)9+vVaVjWHRHc6Nd99tiOkRt zemL7*C&G64TBKq>>Ih{~bl6Uzp%WtlQ^2Md-(0 zo4q@KzhWRtrzqV8d3$pI?R|iYq^yg;65zA`H*P@qub>Hc82f*PWYCMk!lgKfz^dSl z6I~w#g4(JEef|dq@t0x-HAN2!2Y?fW4Z>UThk|EirL4l9i}yb(m<0IC%F&x$oIAPfUl0u~Sb?;w`H zDuO~zBw&O;0?<=PzySn$aev2}O!&P0TF9Rm$ebu2AdjsD)8eLLVbb3gqM_qx3yKql zt0WL7z$b%bgT#O&Lk$2UgQSZ!Gcf}H8$*eHYE^Orbsh1~?qeljiGglvvM%!)nD1t* z-O)-Sa6zPcxzIYF@llnsk!M6qh}pwbGmR-XnYpH-$Rv{Ww8zK+e4Mo=Sr@tvso)l# zh39}6b<01>fFqqg?&g)uirW*unoFt2{$b!Y7n`~K+?G+>_aTLQ-9fr=F zQ67*c*yFC=%V$qfnsLIFbbAdUa@*)jjYgQFR&p^_MQ}eDD!#l?w_n>ROFJ0s$p0Pl z*oU>q3Qilzci*kXa%5PJF!XbsQlc<04+E)_2WW zY~_yMun`_N3`2~G%V*pzx`j^3aj*(Qy`t4$6Kk0sm&?&Im`k&-64Jwmi@uCl^+4ey z=KH#u-bh245s=#_b}WRVG!8XiLy*iyK6cV)bga^%Z0l+n>nw5=nP#SIm!eJs`Ss>T zoy##fy{M_&N-@_+a{I@Zw>w5h#Jje<+iFcVuRu-BAnU5?3?#l%ZH0MT#~K>`2EVaP z%yItOH=Y(j8oU5+$MRD)4unPg9hGX&AltGq(Vw`x)jwH?d$&MXW@q*nUZ>dAsoTTmcUFGU zF2ENq+2#gETL02FniF>cAC+BAEIFdMe_o`T$Cf8ZQt^v@wNWgQ{n~E`15K=#1U z=vW&i2@QDv>GMT_y@`jz5~lKrcW9p9e1^KpiAh+gqIE1FE9N%1=c=@p`Ihh*=7Pu< zHj6iXl8CN2T$;?Be3Y&}*ksKJ7)YVpYF79GtE$UwSGY~qi%}`-RcIyQ3nDHDlqztPO1~UwHkdZU@4GU;Ef=tX&5|=+ zl|Xs7xN~;4x*^&u(@}LqvJ*p%Uic{cEN7%q^M+_4_>3~d(0HGtxhE=&@Y0ILJY}{F zB5cJ_Dv$47OH4pYOnUJqI0a{{XXyAA=iO_Sb;|E=yx8ujip3O!7l;2l?h_6S4$S5+ z0SIn`hz(SAG&i+2wzc*)HnBAqb+B_Zwg*M1{Y$`7D9EGqDxqdB+LSE~XJtSl<9Rs6 zal@nUzOM=1q zC)-FqNn;teH?t&;yfIm3T-^`BtQkYNFT4XdYDSK-Crgbk?AUCb9qgQkP(u)rafOi= z^UfP;?n&W!ko_U_8>1cAEikSWCbk(-a(m~Bty6?Yfb+SGdiPK-lEb1a9&&LOO9 zj*~p1FGuaEV;8JhU$Psfy{2xnZ#$`|dOlumnZP8J<5(z7RfZBHdEn{L1>BcD;5NV z3(yG~-wlnQPV&7l)oiuwqRaxU!;X{!tRG+2=WdEEfa*7c_po{%LyG<5(RNx`0kH7) z$LVDU_vzN?i>2C6JU`acRU3vg(klROCvAHTNAy25TX8P};Bg6AnH3 z-1Z6kubgi3b@hQHE~J~_4u-%*w??;wmyS$Vq=vJa2MX+8djzQ!C73=K(AhFC6CZ9?7Zw&p2Iq~6fV zR=0#07HWed88#MCBRe%HywGYpn<#lEQL&2UHg$_wx5@OUwMDtURx5nKrhz2p?6^4H z4Y7_YVq8wdy*o93|KMIp$wW&;wdzJ{Wt&DDkfovOww*aJujD*Z`F-AU4{AZ@=Y6{V z=vO!S?R_uPuT+Lyo8iVBH*S;mj(mm$; z0O=36p?!7xu+3-Cl@tJYBJKL8kUDt;18qy6;drWY1%ncPqqOx93%qSl@jI#}grsjf zK$>h0jw9sNb^7;MdmE%Nr;`I@f&TB3jr!v($0H%=BT)*k<H)0aDh#P<>xKfd^ogev<-2&>=*nUV-VzbC1{2OajMDB$+$| zU#Ms|tLzqXVtQL~=G?bW$HHgb<^$qSnJBJnXpIQ`;@U|=mC zt=NL2@K`CDI_&_$Y(S-ix;As73Q0BXwlKo^P6*xyl8PgW=Hef=k&a`HJw=j=F_NkC zoldZRK^*Md_8+M-4UbOlpC~veF~1>>s*vPS>V}6UoAhbQ!3`GHq+VfRh_IZ{&7BqJ zqryR3bIZPr@(-PYp9v#-8*$$;|1sUjG`qFE-&5q(1acrg4Jy0A_y`4-DV2nO(zWM| zLV??BFf*mvD{-1HLB>!vI~y}2B1sw*5OFvJdlPa`b(MVOzS;r)Wx)TYuUZrY;Xr8KaW`v@YMVVyq|1ao&f~IFhsPShqe0aBPXE~D~>39-+DykgxWt;gd{qa7?x`ccdk5>`jyVs#0v3=XN zq9chr^LNkGP}jloXv#rZH!J7_*UigR-bfYHf?$v_@BfipXKQUwX*+5zb(vKq8v1oNx};CkcKYAp+mtraW*@EC>*{$8a|%o*DrWR$o7}K> zHyu`|8`w9$`K3r&52fa(gIsK#c6 zDyxjljt?DRN;i?+D~ygBRA2*+Z^@3h=Yz&IGnR$byliGHl2~S(`D27_zTX}474G_t zdo)m*S?W!}qVwwO4a<7hhX%HWv-z7T6zbpjd%eW_q%QA?UZ4kTUvkdvR@r@TSLGb{ zwmYwxK-MAmwP=~EYwpvln3rNmL+jb-)_qy|C*>3Vhhk?J*Qh7~$z1n9-%$Krd;)lG z>f_(>^snew(8GAUyV?f^D!$$-@r!irXPQ8a=N!k9bFaV{l6R{3*I0V<@LNWdEmPGN z%n(dWDhzTqF=Z;_T558VV02+bT4nCsdUG@3s8KtaU=o@O89Do8(~K=`wOywpMP!jb z&E!MxAmsI-Vmx5zF#)R63uLa-5%cqJ$JhX4BzWg)H$vuA4Z__D{uh#PJ?0V@Kq4@3 zeL+o*4S<$u;!o@95%xuqxd+h3C$6)hL^q1B$6P=b9!5r)9;TwIe1lG2nR16tuBf~l zCaoB&!HM1?L?tz+J*s(#y$NSTIFk(r5>*VSiX;5C{zl+RxKTb$)w?<(e|&oiND%SW zv?$Q7kT8CNMw?Vgoy1ILS!fdiQcz=Tv!3&+r2M z*MXT|0*52P4ovI+7tM#89AgFz1JtnlJMimQ~o1dCpx zq;nGf+0Y@~c5YyT!nPLvQSq_T73ztIwGlE^`EftO@I{;_GrprIfg#G2#cOiqNZ^6n z?TWi4ee3=C3G>5{11}UmDB83cua6S8Q_VrOM-vtv(gv0WmIGZ2)}HzisN+|WUDXv4axhU6fjwX$ZmgoionDf$~Aw(hQc#z~cM707^rKK<5V4=M@N!}@FwBcs=ANpdi z4!=gFj?*ZwR<@IHxLRmak9+AN>8x%3Qb6Dd2q0{a4dzD@JN}>0tbEdiGfpvg6BlPw=5NT&(HBsE2ZLg$V z?<9Av*i)|18u0MrT?dg_SW}tEbJVH)H@4reCy51gtX325!mjUQFUJY=#dOyNpd*d( za17SmF@re{MVMk=hx&M}?5jg&4prNzI%1jk352|coue?EiNX^#NE1L`YR3gOhcU&U zB%xWP=6zocophub;AyMWIbn^9P&y3W@kK+ahg**gV|kX1yFvrISCa{chwf+4gA3UK zuqEDAcmp!Swz(m0hK&Z-Vx-l%YQDZB?Vw&pzRxbzzC=L1)u<9TbHl)Po-yU6zEHJ= z(=e~ZdX8~E!lvQs3)%ifx!t0CaFzzdM(!7Q8M7JorHZzE;5Mea#-)~t=Brhesz9|_ zYm%|<*5g4!d@DI^XYCQ=s>B~uyKhAZh#Ize;@pU@O<5t`DT*%qMRp5T30IXKlPnu} z`K=AO0)|2@rpuWv)e6wx{1Jx)c=n-CB*Kt5c%oFj%fG*b-Id`BN)?#_=8#j0+I;pQ zp9Btwwf)M>fO+_2vV3T)ahk#*o!tk$FUy~9_I?RQ08VC4code#RmH|kqhr?wJMkl4 zbiDmPK?ufqt+_$_xP|Bm^?uKY{W5gWOA<7Mg_Jvs-5LChrXoClAY{!G(uyiI@mX2( zQ^^VuO$Qm-JlJSZBq3Qr(h!AO)YRZcOUmD$%pyY1nW8))y=#9EgGJ${y?JTJot7n3(hGu;= z!5oQa7skR~S}qcLead3hK*LA=5~PzaK1!Mm!tw_zCA9+O1)agDoBdXHkDOFF@)^^h zEK*lOX`tOBwu#Ouv%y-wFBBQtD=!bnwOE!@z@bm9PY z&z9O#R(bjZdYa6X0OxWp8i2D3^H5mc-@gE&YZKw+?nswP?tIxi?V~UuCX5Iq1cAMe z*NmNMEs{uygOYAioBp&_ihzBJG|jt#yf84GLFN=4h1A;#ouVm*2E8eU0dGa5FgvIz zhW^~>PPzLw;Zx7@;9A}+@%Eo0P29kFaQ{_v8YxK zEOStsSDRPoQ<_nlITywg|7Qh3fhi-5*k1&mW^!mcfEiQ=e`FG}V+wYszHPLsXK58hI^`DhQ6ldwgH3G4i>NQbNbq*{!LV` zEhC*iJH0svXjL3AvO2sB2j07I2Pj~{R_~x35Kar5pbXhFQ$r0|GE;NnH3(9!Mflpl z*5*A`UQe-!Fx7;UR422;7nBa@5 z{oky{m^=FPw)?VeeS^X#mewT(+O3&MYquBeZtEc?oTD1iXkpg*CmsBE$oirJ@gfk=9Ng()MAB}6;i>t774SF0L=?noI%&HuAQG+~71O zg6r&UTn|(xzL!pn1Ow_OzvdYkY38ZCx#8MM7T5XJH4Ivh6yqkc*O}Sz(JX*jGo8{3 zT4yZNB4?jNX93Ow4i}4hcCTS0-)wjiC0Wbr)yljUT9>PJqS5;gfP49NmFb9G74R-J zMcIlHhd75}_c7@0G>W5W;a8RrJZJfsDV(@}U84tkVmQxt+RleOhXoFurDONC4u@(J zT=`MQ<1Qa3bTEZ<-*=rwo{5_|geC3l-~hEIjfp*4B~JJ?_OF`9QBiXMoNV1{BxACt z!@P9fkq3jdqyl|lf@_K0o=cI=<7e2;{31H-AX}p6K&2jpUMb!<;!S=znalv9*H4^q ztSy8im%$I1xydxJGP(Ctk@!WwJg+If^`OfQh&I=1#uOz44$qlUsqh+@E;&cjV zoL!#DQ4hW@r>iyAL>tD+u^c}I==+wADY`vj|NiuOZTa0^LKr)F>rVV36}Y{GbEkfy z{OK!IYw)>}@ynm?Zg-DE_TA`DIlHv|V@p=ip2sj33P@;$cQD2}nqOEw)dKGbP!>%| zavxyTm&DYrfV3e5C}mZKESthGkJp~(+`YAe-QpNI>3FmG6&iem~85Tu5e6xLTxl}}P(Py0p|SDv!y zY|c;Xgg;;_`Hx#oi4W5*NiiSDP|IJi8>q!H8;HncJR>F9QpP)>ijo>Y(HNHthZHp*Xh;i0WtS$7M6F)Ql6+;g6 zqAvjJ#A^8?r^ES ztRY!GY8a0S{S$x5{h(pekLU9lz(_g7|J<6$7Jwm6=FI{7*`OFjSrKLAfvFUaiSw=A z0_243bb_hk$g?mZ%N$y_J-Y;(hYie!jTZ-omk++h#{yr_;dS7lCtoJvS<|l;aD>9( zfQ@qbS-A@MiC}n7wBChGk0yC! zoV=GY8xIZ*Ak6n$-UNr1ZF0(_4L*-_hFv9*vTeD{a*Yej0wZN%dxhXZ?`n3oK9_3i?hw$7IWS>xiZO@* za`-wW3wIE$-EkYm?s;u=#)axz(w4@_M&FV z+BAv$T#_NqJ>qH|pD&RdI6_Jc8f<+iXpabpp#WrVtu7Sw0V% z5a^?(t2i%&{$}BJa-Nh6=U&E%uBnI7z_4rALmQI(!;*~R(FDXow_vZ^Mvp@Hf(sHP zinjLOC0BNz=_fSjtJ6oDAK>6;y`6Boxh_%x^>=?SoG6aY!!&~B# z;2z|vbdETR5&wqBeXed|hU_-UKmhp((_vl+atMCCJKHZEi^&1b9OP2-{!z4q>spHrb_`&!fTNF`Q%#DE z_zj}Sf>4$h2V6Kg*s?w;=%sJOt!Uc%y~8ozHIgmb?ZDd5gy3zF&Q5CNL+ftu1ATxs zRz1Jj(GSLpF{p4RPcB?hf<6GRkX6vuBYBP?LD=cw#;8XVdy!j!mOf-ttkWZ+?Eo z!H?Ky!iTQRS2mS60x++359_SI<0GH%W45+?B_4fAAf#Ro97Z&$pa+d=4Hv0V0$wjr01$LBD9Nk)`RrF>Hboj zOY^_~hYv@Bgimk*0&EB11oHpd!M~En56P~D(6qo9d4)bEA94?n(YsGu*_CQ0ogbstT%t|4p#`XzJW0=&C^_qTr*q? zSe6;o=c#e85<4t-6zTJ|k+c4(vpkG?cS4sVNj1ISQ_9cFB9~ju*?v}uMFStLR;nQo z`mTWKsf?-id-|XNSD72k))f`l*P~ud>Ww2g*q%7+9A8FtE^Gd)#m(CO&S#qeQ~AW7 ztn8OJwLq2nQ}y> zc$jbBo)<|bCV>MaYiYux=|fiYyzakb;1LP33Mvv7Z`F}CP%%?eE>N=K7R%zLqhhyc`vu4l-aQZ&CB%00{^Y!WyGe1k$b4 zzOH2*yw6=Db>oo+Nd|TH#`pgC#TWVTEE%lPMy7Z5+xEqzkvyWv(YStLr_^PzC^enY zLYcgaUNZUlSL6l=$?Qg)qduoFoim0%7rgKM~_YR7|rF7*I@-rl*;kkE*AW^-LWeEEL;Y3o#XGp1Y;$jm_ zq*-E@e$Huue~?;)K{yjj;KTF$hfKh~{MVC=QwmKEjOnoNRe}$SerglS338ACIuXuyagk5XlmH9(LX=A-UlFSv;4TYqdo`8(Q=JjXrqGxm`kR^Ugp zh^;M+SeTEM1$*6}``7C)L<0rACB3P=LdL~RBzi-8xL`%A5sI7 z4SyOB9_nrvoWlG;UkD-U7b`@;MA2V%@5i)d<95AnY3OWxKeK*Us9~EL^Rp~i)AMUN z#kh>UmX51tDrSL(Tkp+LerY+vhML0fZ?Bpys=|Wwo~QkpqBVsV=)@)RWvU{9_9kRI zlrk`A@T~|WW`TGAIAtRNg-_Ywqg;aDe=~6<=P$&6PlA{{E&u}yw)2Uv;!xOd}O%zn?$f6{$ANa&X1$;N{U8wvcIb?GJCY;Q`CeCy1kX2Pz50;=g z>XEL@U19(!1_J9$NEL13=VzL^1-jx74bRf|Q9PY>d5Mc;WJP}4;@ij%Cw5gB^x_(4 zzhsS$#3d5nX|^A#)M+Qx4bXr{^z4JyGqF`YpK!iH{rQ6uXkRTDJq$ySoR;_l+{QmWkDEGkWw2vL`w;pc(*ElOiIb7rpka~z)$;fs z_nf?|1gEJ?5M8cl!7(j`0uL~Xbi(RQZD}bW50&OFA%e7xfR_Vf$Hs+*d%H(JKt#Zw z3&n-{A^f9WF?OG*;=%IR{!f`W?7{x)A12G`Afo|0v_iemh5+vsb~O?Pu+8fNTqLN( zyfC_hDXYXigXqeHpLvydrN?x5d*RCT>vWVwuSl&tvsz^0F{EabX6L6rI7#JbimVEj z8gA)Ia}1Meb8TsMMsJNzz8rRNUh#LNUIX9$yy1Vadu$3Tqj}5UWu)0ubrcfA8^UVLYI}9UX~$+_@QTdm zUysd?O}V2`mZhOk=J=hm4(DcvIoej?IW;zdrXN_r>ze zn^b`u+lzO!+sk)~q*%<@+QT&2+M}*)9sxiKcD-(H5GGr@-<6d+S|Mm_I}o(BdxXQV zTZY51(+(PYtOYe`ZLabCM(oVpd60Yhq+r$TZ-sEhaxmqt&8>z5tEdp9SWcgQ#EN0_Zy2PDJUU(Ay%ueuWcCW>Y?MZ4*Cg0DujPOA?K51*q9S^MSg4$2h!MLb)3DroM@#%vzh1WD=1 z@=UW`h?Q|f2y4G(9E()WB{#!Fu-R>1rLRV_l}|5fpQ{M+1q$|rTmcI~eC{WUw2F0& zOm>o#dLNXyBUpa%c6yf0cB)OhmUCcA^d9UYXisH|{nniu$u^C$8GDUYmBe#iPHxa{ z%FV$$_Ta1+u3>Pc{J!=J&?4H5qH51*@HczCn=I$8rMoeP6xv>Opw>u;+VY}9?035j z$SiopYfeYkq{HFYMv2<(6mR$SE%#JDLMFxV_HTJCCc9ij$E>Sj z(D2(#1~(@;uDi8Z8Yvq!k6x}bJSYQA(Aw7Se@BU3yB}39S2E<0N)YEdlM;YeBKx*5MDKau|IZ}X4$Fi_3J3^A)YC=Glb;y%q7w#L5F zZ}u>#>q1HkuQ#hvFMO-bSfkqO45*I)50C@@aPVO_XZmFePqsZ%wB0MNr zD;|U_MyLg%;M&w`QSQClGb|90lDfN?@AH>uzps*iwytEL65Oai#>t#w^yDDC)H0qC zFK>@h+~`l!K38$b0G+y9V_c|Xg*Y@F_AjGxR__X6R^DE$2lMcS-p|`hGO8a7sN7ME zx98Ry*7)s!IIG_{AWR`g%I`2vzcz0O~b?S+CjWsFk# z76RYQ`t`>JA)wosg(zihS5DN-HfFy*PK-2sfwSh8g#!ayvNfzBdD{>Hp@(giiQM(Dlsy$SZbDTscXxE59_nz&i)bsijQ!=Hs8HS6pHkL=w`+OSZ!y_ee zfnC1z2~>TOS8*GH`8kHhW@P<3QT#V!_54^7j(1bU=bGbuo>g(uDo64G#+-6H)(Alz+&Jyy6`V=mO9n@?ow>*X8&8+@Ho{C7H1V~Y%Z1@S0)yiY~2qo zoCI(}5-cp1$Vxf8Y4K_4X__kI!$f{L==QLN(h?Rmta|>jenn)2J?5rnMkYp|MKFw( zS?ZWrezCGwvoL+I`UvI9G%?T(m0)bP4ZcbM9`B!Q;poa&Z3dhvfc`T*sfmUACtJWy zj%o%Er~e0K72roDcNWG=g<}U#C5y~ytsWRAKI5-*!DB|j#gp1&W-@cx*5Fl9@!jrX zS7oggSMM*riqtmR?maSd6BmN2B{1aZ&SBDx<;v&M=K*|aJnQWr@P!x<#r^b*oe#Uv zjJ`L!5txEr*Ws95o{p}c`HGFSIUs-@#iPOgqu%?~`^L%wpu7z7VV0}{kom_kk)#1S zSuJaA0(0%r8>}&>B%Y*1FaQgIu1rg#j<097zZPUpFUvZ{lXMRTVb!A(VMf=nVRVOE zCjn@5u0df4k75G`?b*Am-lZmlo66Uat7_-LV>hi~Rlgd;K82J%;^?a3i2;kkL_^BB zB7V|DRVU~1xxzD+K4K7-az)|O`1VTxD6zOhH|W0AwioQ8bChsJE~#ah0*`8B2O#^^ z^!C{qcJu+s*QH^f4OAQv7|WWYG}YFHg2U1s(D(niI2SSQgYyBRsiS^<(o&-ophBL% z;~Tg9TK!L*mV%Xwg9*E}2w+TG{<}jsF8hTmPNqa(O^k)r!TtovyvzBGgf1r!LdU?e zuvo0rw13)}nAIlbM3Y3rPYJfd25pHtd={8=YGVsBR5xszZ!D&G2NC0H;XTvk!bF_b zmMFW0vU*vxn|KAbrYcQTRCkzUx}e+aaDNENQ!@2Ni=@v6o*fcCA2`8ycW4_F8Yj(*<-ct}Ilk=X0BqTt^g*1A+> zlm9WwlSm1@;gJm=ShQ@!<~TN#r1L}Vv3X~^80mlMk}kg6dIw_MTFbAwB-8ww=Wfjr z%2D8wc7L9f_0(Xh9jT9OxRy#su{$6-?jUa~YVt4nxm~-O!AOYC7#>Zb|fP^G2GSw`rHF;6tO|@{&Ddfu_&t6tk9-eai~6ZEpFDw0MFkv#x_x4EIxou?!TQyWE=Tal z1dx+U&xcGKzW||&C~|WuZE9N8RE)IV1U0)_T``+N+i^+w&BPASv5IL%m(|c#t*Juf zZDg8OR>GWyThl~(4fggx_K4#o2cF&8Eo0iI8y0}8w&&<2wa>H|rwhWRVmMbDj+tk_ zRFirg@PVFkDsLxg-je4}(!ruz8wAe@6d`QET{I`U-tfSR+3yq*w?(8AzCv#C@pv(4 zT(YIxrnBw)!*;t-qOy+SE(bH~LJ+ExFvo5k$NT7Mta7^yBni7=@jWwoMlukr1qVpO z=Uky3ANU8r4aDb9?S9m%Xeq=!l-2wbj#dz=wVIlnradECn z7GS6ra-;OQJN4!(Owsa*8Z$vFB2t0^y^K*k`t<@`*&$RWTB$`;;AX!n1dS(ubR>ih zNx^o#R^uFbJ>bI1TvaEjTZ%f6^iww)`J-$*<bgJ1`Mia@_@q6_zdHAqDIpyZ zrhqW1?qVWs)!IT|Wsd=TjM;T$cRZN@rLMrtz*K_>b7!;^iaFf^I`pw}HFI&)RwOT# zWBAj|7MF=5A_@;wEb{BEyQJ0egfstc@qI$^$|E_Q_doIQ3dX9VMxXH6hWMM-vZ=Z? zqdQxEqgHsJkUloTIXgT2=7VL}+hItcIR=Isd$Y9i{@P`X7}%of5v1z-seD$_Cb*$1 zzrpPBNuR21t@-OG*x$VhJq;y%3XHD^kW1ONoe=jL*>|YoPLN2DGHpzXIlE(5Wg{^)&fjX878t3bZ-XX`&L7`z zTf5;q(AcO^vDiRS!X+jQOYg4(D`K61PWa5RBDb3`a^6Zvd?I*_`DiU%^E^$s9gTX$ zAsjXm@X7gV{HOC&2KHptmLTJYd8L|aFyV`En+ z?Qw1`@sS>UJ@7qbO+Y-p{!)lA_{w-Fy9Ei6I+z@9VRdd&g*IpS_(rfGNX#D$&F_x73g4|dBscbMUX=DijU6W(^<3GMPWTd0Xo%LeQz( zc!qq>R$)fZ#6M7j%X2ZEeQ;{>4(|N_cY_J)zhD-QDd6!JSRwF}EjyqQ|5qz4Iachu zW|4HeN-sDLMe$alQFJ~xpa;&Ysdyi+nX!hdQm~#`aI(FsW1yEIy?p>CxHo-Kkt4uc z_Hw(-aK1|2d3k=_qx^xdhG)y=zHcAB=;Uro@ESZDnR4kavppzSkGdZYh1M}McY8C|d(Q3WQhfg_> zf^u@mv^K84XFJOd=F;s9bju2u<|(@fEzQ+l;)t6B-&*bm8_D~gb?1_fYBv(HaE!}6 zPemHD^f|w#RMC=dk#{6TD2U0Tc{-Xf1gzCdqYsNFLf8YtndU_y6o!r5jq?~Vb6^By zDxpLPPV>v;vVuuY$3h-lpZv~4#D<0%-H1e`d*zt;P6zMobGl?b->@f@@YK^-R7A?c zW2Jy<@a2m)IKxHju{%%i-QI>+Hawx%yd0=Hf7o11#UfBeiAD??o*l{jA3^0U{q^Th zp|4)wzY+mmNO194RHM8_suT?=zC&Zvg+p9sOC^3^otV{bT{g$T+$<8nAzTQFD`;1% z!1ob;J8^Tz*6ZmI*>N82<`3f@6$;(O(HQKPC$XSvTUgcHFWX{r=7xO#;RNLdW&;5C zlb4}LTGznJ@bLdphJWe8U;luv2uJW&9ngUP>ZT>e$%Cm9kwIJcHh?ZA6mcYj#eA|s z9n=+K4A_O+F&``5ZX^AY#7>4HS2iI^KW|$}{c7RrDj=}6`H_q3 zVAvVY37tEND~ekN9%X1T?#H2uXOkm*hZ z5TsDyJ>>A_b0MyyYEk`-+C??YTvAn)U&RiuX&)pIIs9!-)h4D?s3Q)T8Vldz$`-+I zO(hTCjypisk;z?>PXIrQ33|Em#HwV`kgE&B{t_q!`V=M}o!>>2Yi?4zPUla?lHFSP zqn8au;h0HkJa})iD~M z{nPG+TJ?362Mgc>82I$x*7!dH`0J$qd6J8lpaEcPA2gU@t{WO2s4EL0jQ%Du@x4i& zwav=)(-(lsDf>wdZI1?$_SG3r|8y(dp8ZVMsDGfpuaF=O52=lm>~wd=8Rr(a^W@Re z8uSnRaw{q;tr?}o>e2vX9ZA?i;Ur_C5;0g-iksn7tzLiBaEh>D7PV6Lva99F4HxgF z0|g@Qsuj3TZF@I|Kw+wNhGZ}J5$g*O@tl+1PN-29@kUt0e9qPZ{7T&q+52K<6VqYs z-aSm?sy=hw0qoIhkv&NOO$|UXe6u-TI@{IP!^I9?^Vm((mkQsP1ifSDI=I*vG`m%R z#bGkiSbx=WUe->28B&fCu+sLpM(4m`c1Y!fNRY2f9$qbkZh>{ ztQr{LPyBz?Ci#~OxW@cPRi^d8)4-rOS+LEb2Tc*F00$w^0tkB^t{$8D3P-moBcxHAk6I$IXH z0dIYRXyyiN!fluNk9&%TRR@Q5ZV%m)>!a6BCaNRI2t4cPV#Cf-7MNed(@335vt zHx+)rWlHc>5YHmo$tjjb0ftFYdfDGs`|KNNVpDiUE?ufJoQSWzPY9P`w>BC0Fy-bv z1}KdrlUI3wtTw*H%_$N_X76nt!tQ^xL*j0gI9Z}Oq{hM!(i==M##RNvrEp|iIe$R| zg;QbK=U8$DOZTadV_h0$#9>>UWQv~P|5YW%cWMbcuwVwjs)YD&(usWnM_90QV3Mcx zkzs)U1rG-|0^GoB>O);~^!gGqHXo@IYCpqU5zEg3l&oZFgG2^2(DjN&^6{luPRvwb z?VAIrLg@U7D(Ofd?uVHl1Asq(2}53U=wvO@qTECAb64Z-4<6t)z}Cf258(Ulk>Q8- z#|ghG{KrMVE4;@|KSI368u$TAqibMIeUhkO8y>mV*z9m^=0M+U|E{@}g_W_DsnypL zqZ6~0?%G&$v@umMuL@t^?7-~AY+r3qL2^O#fp$^_-7DQ1jna?r0IGfqz?!4bfbqG8 z<7aRIAI7Qf9o9h;WlT607&hf7HQ*Rg!sQnd#`r5XEPlCaSJpds7hQx#p$?eHy6%Nx z3^-yc-Gy+|6H73UJID@`4Jmv4GF8$HuQ~w0qN%JsuyL0in5qWTtT+NVjZ2z?Qo`0? zN>ltRv)8eiC^CFm%_Pv#TNi)R7CTnpFBi#MX2xJ(-@`|p&%OR0ygMXbVv76$b_t=*M~!E(%ZdE-yHz+t5ANg@;9%1Tc@pXyPpy%cQgkX ztVaexEKgO^#qq@P$6X|_`dmxR0_acF#5 zPjH?t=Uo>p+Npi#Zc-f&55}uW4xc_(j~;W>705_1*7rad*KhMWvLMZ)`{tp>I~UaE zWp6=1x8ebr$x%#-!~m2<*<>xsd0JSmK>Et7xdT5jGM{ngq9`zTu+A5CaW7dH=TgXP z9|+OPel&1kWzeJaHmTzo^F>inDty4z5{0yQsJ8t@6GmgM&xH*0_8qF{%=3{v$(Dr# z^jhzqOTa3$RdQ<)cJiE04Vm~!LTn2gDf)PkX{t0^r&cvl*$a4*?lug*DYIj0TN(DX z2-DiP%b$fFImK09xCSEaO1CxrCPh{xoThclXTF3LzrDsmZ3o zuQ9tS)f-96B4IICQ3Vh&!zWyntP>zXCZzfqf!2c7;j?I3hbU>&<&5_GWupMaOBOaSy zFdKp`omwgv9`8S(ZDEhipE;`cM^bDM zE~>kTiQ>rl9;z3HZ>T;OdG%N1m}N+nX@z0i#Qw}cikgMEuAyHtVfs-d&mRjeK?UEx z@KJcY*ko!uBd4_A^&mzqBfEowg8a*yG6V5w{cd+d8m$IQ+`iv z#UWR%$!v3~yz6c|DlsQT{(bK3kWEU}%W9Er*NF0u(DcJ1(PDYO_=c~4>&b4&xx2J~(!H;Ttez;q$jw_K@8;?Dagd304NDPm+F~9(E zy0V)bBS(S3n&xUU!j@vQ5|a{(65}t`^R@Fe^I#?}wpu!K?ZF$8FME(*8#9u3vH;$R_5KTEz21tZ)t)tB6S|xS1%xt{ic?-V>MgF#&CbI1eOpTRV zHy+-y7~~rDNG(AO$brlgin0SYg-6NkhXI69Dvjhhx_|_CUDmAn`86I`<~d%L`-=8H zIxFHD6~^-X-&*+56t2ZUZHmofk*i@iz?FLixFJl+lYWMep?_cNmr?I_>t=#c^mV^o z9uvNCc+N%H_Z!q#lWAHOT{BfVa%_DiwaC;T%BL1q5^Ks4V-n1TVvA-Xv+9R85fXVA z!e=St2c#D5jx{csvnI4|i8E_h4@<4KTqQ8+wJhW75xRqireaw@{rhs?xS!1)w)UCl zJ=)}%48_E74Dx?2@Zzmp?c68ko+jsY`%InK4>dYj(l1yHz0A3? zKvTe=b-d4^Rcj_+oWFneLWf)Yg4x=t0VHeOR@z=pjImc~8w5gw>|N!~?foih2LpO> zPTg_iLIu%KhzqY-rp=v_Tf~g_aKvdnkgA;fjE#gMu)G?$X%UF$FOpOd)Yoo3lPBx| z?X#LQu!zfFf6QS@FKwtgv26N%pV9KAu;d$r|0!w)Rj`+7>-AHUvnk%I`|e4U}@$Sm#t+M zaOfLJ9j`KhM40FU1gx&A-OSE}`nn2qwoq0s+^Y;Y>yzCT@{UmUg^e`XoQvwXisL~L z_kI$Q-)3q+zHt10mE`O?@^%i7-umjcO%plE8n78IcwIR&e=gO0x$_+b3V)s9RQQSkw_wrmm?quQ)p{uNTln(MDKAkQ z3z|)EQpc|EOY!e$E4vkPRfE|<~&qD^BgIQ0-AKv97+eapvgRfG*MFlXy&zr z=ulLkWA*$C^-`VpL#!LNs%6SAm;+e$AUw^h5Civ^h^ib?^LFN$Qoo!0Yg@u-h7M;B z!kf4#Z&>@Spg$ky_m70#Sh-~mkk?0z)ntrc7`pKSwemk2A{scH@)!AaK?B z=d!JZ_38@f`{$DozpvvqEp(^rHZFAM_ibM2&MZHX`C1@bqEOE%>V6G6c*>3-i!aMB z`zbuRAQl`ff(jA~BF*W?Jd=vFIjZ^kYGVrm%+X{qWjV*{=+d=4wHmAWhGxfV8_=JA zCFFiM(^}Q$QNP>C4)ev+aD-`7!|UOt5$G$Ui6R+x5*omC*B!w64Gui7G*NqK07_&P zN7SS+RiN!Q8&Za_)_^v+a6$A^&p4Wa3#n}%xk(BOU#{Fml~8~AAq+?cM;6G!urk{> zoK5v>(oLmnG+S`EQFVkPrLpf*a)kCmPvCkT44M0ysv#vDI~TU%DJI-$qaN+wQmNkI z)qpZ#YMo=O_&#m0@wH0!f}omI!ZM|`teE!{I43tD`j_vKcK_td)4lSBVD=TIN$B5+ zqn)_rI&~zHL_G>W>H94J(If1kYHzq(N zk3>EXX6!~7BL@E58uLo@tJ5X<&sBosQ~l*5#TwbvP5v}-5^WqwowfH_lvd4jn5r)vd;7|-z|2VCILJ^*uZqc9Sv%3< zpaN<*Vq^A_zD&!Etha7EalD9F+KU37qS?x`J~q?t@3;Ol^7tbK-_JsfNc#m53882U z_aa8L@*M1|6{>P-S zWr9Vdey&nEE7M(+CgmrN#SK-80}+^6at|`6UuA%pHbKuuLq-WeH;$$9Tsp~h=wfX) zuf0{6&0%`zqAJH>Gs7Q>_WhYlet4v~Gm9@A2l^xdHm`6eud#daw=!_Ani8>SJd^9k zAWv0SMT-Hjm~q?gy3xv8tMPst;ph&Lo!t^4vUC3Jkms9zS=Wp#x5#V!8f`7TZ%!<3 zqxd&6SoDU_shZF%bn(^-j9L8zZ!-F$Hi8YKu_qkK3#Wu@^#}>U{37u$h{MiX6zad2 ztxz7QO#6TKUH(71&MGXfcG=b;1b24`?(Pr>?(XjH4#7HD6C8p&jk~);;~w1If;$8V zIh}v4z0cWuJ?8@MxS7mv)>l>U7=x-MRPPXgxZ%pwN}gw|lppoo#;AM1aBy6<`g_XM z9=o?BvF7ZJIftOWS@3dN@y6{e+=N{f5-Z_EIoA0P^?)gFFC-ximwv-v077AwT@129=B;cUctj7t5ysV0 z4j|6Jj5=ENm1eq=|3E3w5GX||1%*kGiAa~H?$hM#M17bg<2WbNG%h*bZPa%EA;mr@ z=YT%toG~}uqHT)RpKHtXd4P?S2PCsbA;lwbvG@)C=bG%skmY|mmORA3HeA$ zvKpyj>|joH~NS36=NYOq%6T>}9|jN(1b7wsmWk3Nd${>l?schSV#?ES%2H*A)T zvzdKq$9FdqjP+&h_0cpV6=CUL$q^k?hnmEhs;3(H3|E*(TWv;=d6DfQ21@cGwZojV_3O&{|(d4QeoGv53JQZr55^wJ@WJwW`;&X*Bc z;%Ct+4)WT$g_^Uy_lQjQrfgTDQJ=dw>E}rxGNX)en8KuP4F)fTMP{Fr&^|h0JvhVm zrc&_|HpuX9jfJ^h^s-R41*7b;g>UuSy1JjIAa}^-&+!XbfcQKY?T4S{1qu_ir~lN><-0H;0+V28Q~dd{Kv!0;5o8I$1=f) z&7=Xq#ndr5a7$<8^Hz__-kIRThIPg8mOzbOMKuCd+|tzjv>|#W2oQ}o`?S4sOOi=e zv=20&Ylt^fTzZPy?yX6Je{S@e3+E8MQ_KJxC*8FPW%P*bX+LKao_b*4;*qQ7&KSyO z!MXXT%Ha4*WlU4b@w!^0%ef>Tj~V2!0%eb2?n*=(se%soH00RCjh8x`xMeIIQPsFS0$f?6BG1)1$bli3%2wNqBdj`YRNQdMS0O$P|8eol1bE;M}GI^l2 zoYHzIjf7cB=v(X&%seA#2?KkEmg){y4@QG2^R`wks?N*{S>3W-F|r>!J-OVWK9qDw z8o`6m^T(%l+|iFX@+T(LNmP zI}?j_xB7yB0v^?INMkhcywnj8P%0PJA&a&n^1hmqww2U6=}%bxsfwQO^@yVJ7hXsG zABy2W2MfC2vPb`n%XWJgNc;pP)U;`C&;4sF%}S6oBx42XVBY;F7uwec5yAY|0tN2B zT*@F*vVZ&4%djbJoT#bba|1&VHu>S=Z zPuTw_SUV7q^zm4lzWI3vJjXfh7+s_mOsXvoU7%8|ZS`b@$uN2f2r4$k- zLERkFmBIQ$NYkS2iV3HSTEQ2V6gfAeOQYfcPI-_(LNjZy zHPW3;BTL#~5GKbf!7-Jsc$b3$x`0)A8 zfBRzSK1rElziMt#P3_K6tgDxnr4yPFOF~-fX+Ab19la-^5fy6PnsL<1MH)XOZRj}b z0!Y-G=K!9ki7wM1aAHLy4XgOfD7YmNA*oZsif0}LeOj8q99WQDht}@omsz8ptqlI)U>EYnm0YDm26&t^Sc)&&kN;9YNbpo#vOfOHh*{qUYOV!a793 zQcuoDds1ATZh)eECB(jV%jqVpjevzZ3a}FBb*cTz$$l&pW{s_G!3_f$er1P!{c>U} z-Pd-j;KahkZ_Ize#AmySMXEF7!T%Nv0U!q{r}+A-EZ#N;6`QK?mC9=`@@a4{dIOMd z6Xpyag{yOaKCCsc@uKl==0o_$dDRc8B}b%2GxNK?m)+90hNZ50NN!Er!W;S%00@N@ z=l(Lc_ht0MTp|bah6`S28^s1AKE1Hc(l<_wi>5}4o3B=xt|4z81DMMneX=u)qr%eu z@FFPoNpG3`QkvuGGDIBnCD~A^^(anvnr>CAb(m?40clX!wH9`57c;4C@jH*D^t`Xp z^mdlA?QxmS)xCHvT$f*^C~;heXW?R6rB&8lxyN@^!%1iBT%R2cRN3s7gLyj?DM1Pn{igbviD zPo}J-4=!F6P1T(u8`V!uSjDNV)is!E90r)tmyQYI<~@&p#-AAhe9X@ycm8sxzWZQ1 zdOcAGV`hNV7+2_;6|9^=`|}Eynwe8-i9=aX3NM$}-HWj^{npkt`Z(KBaY($`r%-6N z>{{1%Dr8vaBc`L{Vurq%JK)?gCSeG*D=hdvF~IBeAP20>j0ko=t`ny(us_VUKh&1v z3Q$U!3#S~E$xX?u4erT5zHn@>7+VhdKEqmwxJk`Ct-}z}5sg=Gz-_qFPM0*4kqdKI zpxek0m69a4t{836MyS^wIxKcNTbhvanX+M=>H$`&!GSq|G<9|X51`i~9Ak&j9jxt6 zyi`)BWi~y2CKu5axxa^8>#a9 z`GiY@ewQq=GNS_@x18o9=n>LNWK#d75NGby5UV3BJCwv;uGN%Q=6MTuQpH;1;R4ly zw}$m+`I(YA{0LbAhXTdB0b;?ebNV%KH@->UK|k?-8>{C^k$DZ5Z0ssOfGgWW zoyxi5IzsvlWJYw$!hEHFn1of$sC)h+@^dG7EP&~akrqCT_0LgCL(fa;2s0_nrE93q zk@4R~BTu-e3IJ&|tpBUg_`iKdU()}x|AtLfoJYV01SB}h6N#b^|4AIkYLie52rS2F z6N^4xSc+7U0DTmWS34e`Y#STfbmDuIcrkn?!Dh-Adr`X*mp%IajkS{WY~o_V`LL(u z_Q~TBu9x9ZNWm}dn(cI_AiNC|@hSj)O!!%ag)&YK20E?OKYNc3CEFdhe#AFlmT`ESXHffrzVt zt&%nLA%!cc?t2D66GeY{zBVQj7J3}nvf!lCTk<0lFXH=Y#aFsUBN`*7?{rRV3WN*l zz!IQ4y)Zl@`c9bVY&%tTLNbY`xUL1S6W%JRe6a6A48SvEvjgMimpJd0_~$SEY@TT_ zZPq{Z<4g}(n^|G-tw=uK8F{WTvj#HjL6M(=n>;a%JDRuwpM}4?4SfM|mQI!ub!;Z& z+A-wMAqaneT+I&s^k|7-c@wBL!N%)Db0___6?N$G%2>{Vk{xF1`DD$e0i^ZBkODa^ z4F!?LKnsyVm#-ja7UT(lH{lOEiB4qa70i?yJu^uaVmG0VH&bWFl~mcgF86VK(NA?i zXF__+GKkxNfPLSzwv?zZVyT>Y3c`}uw@8i3*lg>xg4De7h4=653EC4t>#x8sK+#sE zE7auIAV#k34_}(l4kp&mO}*md=#0Aw?%pN3dH*=uiTy<|XGIp)Kzi6M*3n(2QFMrT zQy<5IX0rEB?j2u83fW(FRO0_}jQB5h^qLtG1tG7662vbtzbq&OdjW z`56G!ST|WWR5$u#-v^B0;^5kO;4_TD!PUQb-PVeofcb1aFC>&&CupKmB9&*^Ulg)sGN+a_N#)a(oWGJ zel?``pcvHEj>NPF8uoK#lU2vLcf;wz><)3)RAH!DzW{CTPug6E|4c^8+uq(_dU-1B zrA-~M0^!=^#!C+eV0L!$Mv=kT_(E~V+UYgyjcjn}#(NP*uvS9a6;IHILSlG~-NWkB zo=iS0+6H}`)7m~4zv_1}Pjbkv)4E5lf;ZJ|UjG5ak74{LGa>)Ys{ddQQ~$YTI~$|p z0>V@k<=?Af3wgidXLI$`uQtN#l}dy-C4LOW&2M6hOqu?PLAf`pBqQBf@(X=``W`%>^|ef-O%F>B4cRla%EIn|}Qn!Vn4h;?qpv?TcncHtjKI|-D1 zu*6&$UdLSuk3Xflbk_asQKb!?t4b5L-+=3Xv=ljZ2txx*JWqO8GZe*(Ak zN;wsoYbs}TEk8;#$urDgjdJjhbba#-Tp|s5;za-XQT<0626^J>069fd7i~9F7Z*#n zf3IiszL;nnVC0TAncj-Ni&+n^6_I&Qp6P&zDhUtuJ}583Bp;r&!`aj{t`-ix^HLvM zxpS}})$vC;)hxq;R^@%^;%3uBV|Y~;n&0wNXTXb$mvx-U+UFa;%YaMX+ezLnf6lGP zdRD;GM8i9k5I`JmK?S~Y9cUQXj?%@mZU(O6`LqSZTfPz_GO_?iIr>DPur}>j@^r2P zwU@8th>R@36^=eJC^bzxo;;nKK;Gr6AyXA(JQQe{R5k+vWVxL-Q#M?dW^v&x>+tt{ zX5nJTmQS4|kx$hh9+o4#yGf+_!@TfO@Rm)HL-D;h0B2UM(K~KHM*M24)|4Fq;3ZEWzmcceh^?O9gYo2hVYN97z%F=TT72Qrs})RqDCLx1 zy5(_z3ljJ@waa_|@(*Js+6d7hY~s-nz5H}aug|gRav!?sHx2_xZy$W*S{#hGEd*l> zG-mWkxQwM>@Iko6_7A%>8X86Hj-=((psx(`#t#HvL57`}+cH4AuqYQOuvl1Bo$8J) zvV5(=jv8@QEy7z-Z8{E9cnePT02X$0?+XA?8;scAz z;2>R83r$OBfJR2Q{*{*D6A~@`Ss}RzUXKuwJa$eoIufpu~K-8Yqe_^04Hcrn}8A=T6JpNm`pr2T9S zl3PJ+tL@(N+yWg$CnrMiVX>s2qi_L4+9A9QuBWBfe1@C9s`W0WZKB-6$^6(p+>Bed ze9Y1j?(ocPR)Y&vW(J<<~vSQgUn8%!IfGc{s0mUVMqXYoe?(i~F~(g!nehF{-|M+tm)AZuPYo1QNg!qQ=l-nY z2od9pTlHB(EJQof+_h5rvf0yvLl*Ft(Mc%|TSZ4@XR$dH&ELy5lrq(_?nEm81UF?gDI_4{EcXLs70xf(jt=nbT zbJJtv8toH|kxdU>vi_oMm7B5?5ZhGZYr{}FcaTm6{2qm7p-uH=ijF8EX5cynM7%Iw z@zD}G_4Hh;Ma*ysfd0shSIEE3wwXs|*=|R9CYEyt1vQ=u$JYu@3@- zo$W?P*X_1b9?hToc{|EbuVa>$vuDtEdF{pL!E~paJM-DqK;L@%ySiqel;fMfHrStI zOgPTq0_;&Ob_rG2$MnnP_FU+K z#=C3M-ybD2zPy`nW92ZRXIT*wrnb#(QHYyLN9>-n*~otf(mNErjw_ZzMO8l{v)}rZHBgD4R9H z^Z+962#0B=eLqDy@2#6dj|vPvEY(zgog4jL8%HLc7B?9vlm{nkU$){>*|B3aE}%Eu zy)mz;98F7A2`47FGHTjI>+nfBT81IY25LhC&vhR15q&0%<80f22x1ouo3NFsp;JD3 z@*KaQDyqG0zB~+Qw`@2d!yJpEyds7G*jUa9(x=i0rl;XG=`Ry8W6f0r?1&Mw z{apCCG9V6A{5%`B@u;*vjrdarQ;+Q^<b5EvnMLXB0Bfwy}unDpjP5 zZ;ni$@-Dp#wOzSo>K{3`UrkA+Usjpza75gVdcIA+(@t_0-fYM zR35`T3^i2Sn%!f_xl72Vrb4D*V5CjuH1mrX(H=qskg2nM;H`-L-_sE9hr7bO0+Lpt z{}^wa^~nlh&v?FmCCv$qxZ^D&n;H#A(fs}zb(KSJjPM2ciV(uIo<@tbtMn8?-$ZQ)Og+HMOEzs^}xNa?t{EH7+EU&G)&8JTk`T)s;0b!fu zk(S2Q)c%g|tNh} zKc@M$OBe=oYn9><+RhuP>H;MBi_r$wJs3N!ZfMiWZ>5i92cU6X^H7MW?66+xEN!r! zO{%#(kMzRK@>bQUPYB#3TB$A&pPEOrZOWN_-WSUZZJJJAvO~xAx)J`I<%Kx^hul&w4d`e@!4 zoLWc6eXd4>Q=yM8j6FJu`Kyv^{}*N(^cxL-v*@l9um-`1PceU!26kC5;smcu5tI8P zVXZ>`Chg&&SHv^?t0BOxPZzUgjX`}+vB|CX41HWTN+j%PCvMkcGUfSFLEPwLg7)4g zTN5SQpsL`l$d0ds5uIe&-*-K~V?2VrKPY?1%=_(}zQ0F~cz)-votHzeDt?tZFQLv| zf~GZg_E}b3pWMC3tl1!6_(M&JN^a;VCO6HzTGBVuiet;z!EecczN7S)j5@3fNF@B9 zaPpqS`d_2U;5%`)`!A*@qE_H^B5#KGxhP;sw* zp7z>=m%&~|%67+GI>^d$YIlUK2ir^S};$jA4Yy6w{YFWoKZZ1whbG{*f7=V?6T@)O^A z-y><(^)?(@;N3PY8jG*XXh?2&!3X_LfC~CUKE9?AvNW|kw>-+YSqe2IZFyWLrQuU= zeei)O#P`Eqhfs&qO!u!sDArPF95y9}39Z4w>`UlhNbf*3Wl|VBFSckjT#V@+1Ye<} z^}>dHFYGgO7J?U7whWO1okTo+RtBzw{r%=J+VS|a22dZGNwSRg{* z65>Q7P1tXO&KJds0ai`nJk94blo$1Hhqz1Zm0ldm$K14rU}Wz0_JX!%ap-~<Y1cV?Kc;Y(ENdXycDIv$g3^_|@CW=>0TcNR61}q>bMYG!x`TFcHROU!D-@Ib%q< z*K#(`0$??`m`GJ$@%5QDI21FZz_@+DN;FDeqs(p2%;EfCshNp{UkFY)ydxQC<4C;~ zL23U~GM6W~4&bBm=KqtmM*lEq=@e+qtyOOa=0T6DEO0&N5!jKb07jmAqYl z=`OCB(Pn?!G-2a3h?4TF*Vv9)zOL5tOWw0Flv2)6Oin2amH+%kUUj0P+z4GY&UJyLbF$_ zgp-LDn zhwY^{x(ESyRn(?uMdfJnX;PuWQ3j5*&cSPI#049hu%&v~v5Siu#{mv?SGz7%LzBC! z-hhk7Hy3@+A}w8gX(Y+<9;AMP^Oj`xxEROno_O#l|HKe;@i;$TV&RcI6eGFHQK4!m9#%WA$| ztcA+TE@Si9R&rOme7Zs7>S+{p(*GbBB4TE=hUm+CF4PZ!Wj5iKUReXAeJl|qtOtJR zR>15aif3rK-~;uNA~w0Pr^innhcq4_`}df>NO4HVpAZyHGrvy7%eja;tf?UYHV_#r z9LJ7ITym58>!2Xh+*kh1)CbdOjg^q)bdt0T@5T?t5GrF^R}FIs1A~Db!iZHEV>~axi+XXdnoEMdPp`0RAA`KA@t(DPmLG zRo--Js#maazM}BofM+<~a;5}GJLV_0oIfZ%UgenZ;?j>GDqVeXnC@-TKr`B#$k{$X zG>n#<)K=|m>LfzKj%U%Jd-`@pjB_^Zm(I43t_BI~0}@h|Acmn%{bC}AN!{+4b2fq< z?JB(zuJ<3p6BK^?M|R}yuDzrQ;VIz$lc!jwh42*rlq2;}c+Au;06cuEzaKg(ph(lv z8IpY)=mbf&IQfog0`6=ekV%APoe^778w!y)tzmW>HQ{;P@tz`3&2_G2l^D$@q|}@Y3pUW25712DPo0(f zi|WRPojgZlu$t;4LNwLDXLYw`9cS)Va24=~54&Dysa>KrF4Y)w<_TJE2SnUw-b-b2i(zOmh_4Z&v0hwYw3=MVgE(Icg7tdxlK&g)&iT3UwFd z699XF%AI4WSMa$V7h$5}27p>i+VWN`x01mbGS+?QNWuK>J#^1EmV6$`mwFwGW6+@D zMQGs+=dix}HPnTS&o^W^o*us8w!;;cLE1~6P}ZoT`3>cpkeQD4I-liclBiX7+6^y} zFuImWhAwVa8@=_T7=F_gq^Ok3P{lPzU12mmirPbE9jskeQh{^(O~7mDRhS^-`p~XM zq6N-pD*d-$N(;VUICz6>&0$Al$1;OeTI(Eer`%h2~WIk zKe*4+LaHDqdIHOToGl}#+pO?XRGxLu;{b=tl0eH6dJOX>*^^G{nca)ooGeU-*o>pm zonR$0!ART`0We}#xbF0Qu2@=AJnDW3g&UWcV_v3iOQom~R~A5;B)W*eK|u8*OY}^J zJF&ZF<}v^P(=Jl@CTV;^O#?s?_o~{$wa%h8F#T!eZQU?_rTt9y z4DuX6wvt+9JxBh%-ZOW-PK|^31MP`?dd2<1-RN?X|M~Hr=$-i!d$8IEMvfE~6PqtxQ$f7kSzAB6?<>pn^;c$xP-}eVka99;L}}`21NA+p#}P^t z-tQAwZ|!@O#I!ms^GgR@wMOO~0998-St_wAHbFJuN{fbd9xlXyZnEY8?>n;G%EJbt zF_%QdYy&&gp2YH^y=3hcsz?2GCFUT>NX9hv`OG z@|a%`+LxQQn7iUoF!tQnzhVDd<-YqEKQ3FqG~3qX*NZ;Hu?fV4(r0c~KwTde7j~jE zjR&i#Ted>8=8grxspHemO?j96B#=1uayIO(fk;h9r;3<@OGbODQd}u$%1ZG@Lsr@H6)FJx zRdqMrF)zMl-?W?4dg^g9o!_6e1}*O0Vx8G;V#G8Jt|C*q&J<-xFbpt1ED6Z9sXwx7 zh=KK_BY`kSXcbv%^1ZKClQ`llxnULHoIR47Y8D4XwlPJ=WKX<1%+5`e=&a}6{u zbkRC?8%iX5AowETO#esW19=fBptqyv{pNXOZ*yCXzh75bk+IZw8ccKjXO8Ewc-u0p zJ{7{<8frWag~lRQ)V;Y__`s`J>4}8|Uv?d722_JO#wI>mxGgIk1*;v`KP~P6e7{X^ z>VWd|td9H&)*{;9WQ#3I_%Q1+u@VC__@K|=A$)O5n^+LxfMSL%9qWf*wBjsKE0%I= zAzrqt37p4;3M(9vQtaA2VXkqnmi@m)Y>sc|QR?t)ew%B{RIU6Ht{=W9t*jt_CNV{5 zfPN-yTAUn@vBrKj85T-Yz8@lCLY_T*L$$3zIHH#~y+HX(9%*2Epv6EItiS(r!OHFm zaq9kO!TP`VtI6t)%5y4^n+|=417j_Sj&3Xp=TR=bl*$B|?}yF9jVWE9vv*UUnXUC2 z#$82EuyO6vcbRVqA0t?exvn7?OAB=x`{x;&qo4c%7w$QJPA@OhsWtE1(vD=@E%@q* zIf)5}nWSU)1cr%+W7vFM0X)&@QWWuQ#6i&@_-7ZINVX|SjJ_{0SgqKht}t`0ZW=+19YK-dENc!N?&nb3AhQRpmR?k>!#TAg8qxXCgZOW{}!c)PY|d5+hfG z%wSnZyXzM6;sZhRWp{o;0@R+Rp$;jEC#B4AtmJLVRcGhbYck1(O>8Ln;z}YhS#F=( z!mT0brBOP+ISV@sQ1jW?)o|GI+naeTaZ`O{gJkS%2|h7+9WrZYa2VEdN2a9J!l*Fm z&U?V3HL7Z-Ku5ANZ1&6_$@!J>_1pxLSO~tFw066pkWY4nkDPM<4BZuob~oY@W=2OR zj}l35;E=dAvuo53#}x1_Ep%2me`WWXRbq+b^N+1wkweaBfVXUAkk130SO@+@p!6|T z5=>jrGRFS>85C@HRUOP1jZJ$mqx{UKOr;-jFh5CX3LjMvxPLMqBBa7Yj89tSN8)?K znn9z7PG7Guo)^{O@YheTdX%pTEisGV{`s8DHB1S~KwiM>|KKYADetL+3kbyjeiJ=C z{$|<#?-w#gW5bC^6a8(<_grz2h!$0mTv1%yE?9ky{dZ7GVJKB~V*3+qe6wn+h`>r# zq5firRafl$mQRvjg4SIy(Cvug>AL9Ej6q^Dy)c-UU(=e<24h-ObOrX(ZzrdJS$UEE zx!*v0H?=DTAhY7`EHM=Y=L4xxs8N=YA7JA7wf&6<83p=sAa;o#KVNOe1L!$Am(D(`<-5j6ReQuv;IzzTft`wx`Lj#q)lmB|5y}qx zvAxqc=U_d~z$s^&`fKB$Be2vKN$HUt<5&@x_@|g*xpRf-s@Qm)Q!*17IJXf}PyJxW z+|%5(-<(xtg~zbMse~*pz13^NB{0D3oAL)hATCxesXrek{unZQg%)vVW2<{V(A*@A zwQhsKav0IxA&*9MqN29b{iWjgOOSMD6!i_D*mS@_+88Tm(DC#Zb)%iD-BXQ2h*@a1 zS%Y`-=#6Y+Ez==Ry$gDbb#1+8EVE_RJ)R>ZkWJweo|5SJBCTjhKnrIH7jDP8Q^YEO z2YRhQ)9NECOu%u;;9@wHV=Au*8LaId6~X9n`U_v62mpVxm=hO2nf8OBsFB#`;sso4 z{%MXC@lMP>seAG}!EZ-_p>NDU)b2hLC#{VqgrVosXZVN!sb+zteK6l;^*jmEPm$6A_FrXiNzTurE5e z9&$H@b>f{9SN9mjZ&X(t#Z*2D!IWU8=-Ky1sl8a3rKQGU+tu6A0|(zw!}docmQzVs zD2dD;+5Pc`FUWs=9JuGOY9}OamEuG*y|Xe+L%P|k>iOr&emBh3`3-WOAOGWukP5*e z`AEfKjs83Q=U;YUQm}!!6P@z|`TrvY$b|lXqyRa6pbQ~)lFglP$P1#tM|3$JpO|E3 zmQ@&&g`qe3$8_4l1Xc_eRtpy0#KHs);{W9UxuL+pQ*7{A#6!L&8<3y>d1Fa6&4A38 zdri<$QnQvJ9*&Si3U4e+^gql(xU)D=q6ao0bgn{~xTI!y3KfxZW=$kIAE!hV!{v*V zN!ZhXt;c}f0E#hk=@Y0;xRaiJcdG{Gc%^1g_tCgt&&I^t;gc0j!&YkIAO8Si5z03$ zhy@2kI9I`!zgrq<&fvxX)bOv`CBoTK5`#%m435+S0!}GZW^nsH01*vH$yc7?%V+vs z=)rD49OP8?N(-IA+6+?^aY3*i&=Z*i83N=C&Wz3sx^Ro5foXuu$V;3Kmdisflwd_< zUTqMvHk(7K7=JrKbydHFgT=Bzm~1Nu5yPNh2lrFij#g*&4z&*9X0JWrW`6_W!!{|9 zma7vGV$IbV)xp&XCk}q`5dp)sYJu+vX_&!7tqYk%1_=AZ@Tw_ga)7xBqD8 zZ=bXsdSD4RaW5(gT)H=Woc&YeWw-h5@O3;8t5qNT$n+BQ`)%a6kyu;T@jU^BK(q9~AJ&Il-t7SNq`!ATm#^`uK9)t_$@->O>DZ|~~r z?(hyzxJ$u{^!v2b*pleO62U3EF?%|iA*nG}cPe<6#BNjQ8*MmHic7Gs@<^PTBVBZq zX%Nt|V|8aVe%zc0`vX^otD)!fDSKJ>+=NTgEm&6wHW4>g^u|E^k`;G_&0cY_Q4Ijg zYGQ=XoC`g6hD+8$UmD>VN^9X%X^uSftL_KRi3u8>4B9JgRexgd@N?z4pN$?1?pyxy zaeGviOOXhZBn4qb435N7t3My@2J~w-pM0eIU)K@r<`FXnjR2;BH2k}Aml!qQYMS{E zl1)s}Y*JeWgyn)jH}y2Px@LWhasZcAnBY(m%P^UqQXT7*nmG|#*$=|(V#i~a$si^7 zlASd}x>`gyQsvzgH6JyyY8F{`e}@{9+?amd4l94FDdJpe>gd@mt@kw}`|AQM3$rxG zz=j4A9^5L|bizk$C)Pnjn`^!nzdxBcpVM}h&VBH-^S{6d6~Ubn+FH%G=mUId=8D#= zR`V3UEx&IHUg+l%!X1M6j?CZ-zEEpbvpLS`m*sVC$Pyq<9rU%ZuWi}4W-dnr-%jC& z$@`-r9%7T#t8N;`aI4~!8gAJ0dRIryb~g`?w#%A#jb70FEYk=yMJ)e5)q9?qwNPC# z_uwt{wwB=_;Mn?OXcH0{I}6Zo26bv0XV}ToX4pv5FA=DynBJlJ6dKtg&pL66-!$5m zi~v?=yI8XEvF2ivPq=1D6J}#8^l?9X9&hBF6?6y-HT=LY3_l0}PDNT-8E)=#ak}dz zR&7>iHnP|7Cm+PI}JIYP8_Z9B+kl`rMGeMuTzx?KV_C~GwWCkE=feEATX}%21 zKVJ^olUo|GVE(Q-u9;`UH;!n9TlGCycaTx00=|aKkqwOSqf}`}il_{yKQQf!J_!5m zTrMe?xhr^3lK!-8mgr6)ck9Owsy_^LM+D}02R!DRqKA-?!Kl-i&_?4c(y->vJpn;` z`KAk+(mn8{vW+}JZxJwSlSM91tA{o2aUna+e{U2^u8o24iPU1PKmADa4;9c z_(`n`YO|ZEZJg8>a3gZ-x@w(%wT#X;G4ENa zU{0tp@A*7}+EoB@edKkH1(^4^tp%&aPOhj&vgH06p`WBFdSZ@>1Pne}dZEEe5fvW_ z8B_cT#iItZ(|_BF$K)W_I>$T1E{&o(k-bBh7l>#l+dM|q_m@RQvhb49TlG(8Y*MO{ z2tZ=}F#oGG4-I|iqgBcgo8c}%VD@ue?_JW9sRF53ml!3WG2g-Xizy>_;-csstX^ga z1NfWmQL5NllV#+b(gCcLNprhP;_bQ?n}*%4AG zuH4s5(q(fkRS!hSpQ0WJ{;lQl?v7T&kUIWXZSlWq`9FzYkXnwA>aY%n0Z^0&|8?Ij zb1b;82-FZlFblmilM!X@@~Km!yHmN^cFa_;Kk>p!)R@UV?ECA?T@ppHj96cL$?-BV zbtZU8R^$BcJ+Z*D=ZuSsh|YC}p&0m-#}Dhi(7$S8`4Wwm7}qhFH|_{DJBEf;MU788 zbU6sroXVnT=LshJ0X1atrO7H~@Oj|AWE@I6fe5*aO7 zsgtBZiTQUwgp#V9^36QqDzr{@8ro5YPjJh>%Pz-vh2AUweYoFL3!zvc50>OV2Li4G zNVut68*sP)HACgU=I7$_B^}i&di0vzA8?N^poxtAgPu&9fQ8BN<(FVDHQ*VXZZp*z zEg4ndyv*)jbK^?~-k3U(<#0jki!8sXvvo-0KW+Rjr;ph@27I`DxV*D`VB^LIikLMQp)lz!OHB6T0&>`}Gp5MFZgW2ko+!|B$l!wR)N!OY zi3mIge+h#R6-yY9!^`*(3VCS~X5fG~L1d*eMXdO8aQt-5yWJKh`<3I2k}$`&_D zOB_s!jEOml!JU40Cjm!IinB?kY;+fv7O)L?XjsP0$th{cJO78QuZ)TVTe?kxLj%DZ zcXti$?(P!Y-7Ov5ozS>TaCbs*3GVI$4etJWX72so_ulkc-M@RS+NaK`UA1?4KPGf2 z_4gkx(gz99J^>&uoX)fz!s1ZXU@AI17KU}J;TraD#6!|lzb10>8J{^6hYlZJ!nwLr z!VOV0CLO7rhicqyw|-t8dkQrvs43^J2lBpXfBL4Z;h8i~24KiTk)#%m(R{hE#gz1N z`eu(Et~-KI-eB{q;t6Yj`U3|drku`nDZxww|F2Sdp2mAwC{&@h!PTM+4yr2OhMA=@ zzQEsKM3yS+&RBkSx1`S=($2Y?b0>SJzl=QotuMpSA}B9|1aPhC>?ls9dfhE_}qA2(i>5{;NV~mL~HbEhC&JHjQAVTZ0yVEc*_ts z_(SwMl_4t9HPLL0Ivwe_ct+7|oN2v4zXrr5lEQju7%T8pm@_Wid1Q3Zz4icU2l)<* z2a>nk0L0m704;im8&Pb4^-qE`feJ2l@RAIx2O?y5Zb_}!04`wU9x-R=9*_AKH+_ri zBE4Bo{0rN7fzbd=^zgk52@WgW4!RpIxF1jJyec{;A>J7)CHgpgq~*QdfP%Q$Y9Lkj zvDyu*(PDIZXt3Yg5fQ@GbGff4y{Rlq8L+qeHE&IK(LsrrsD^P3SiD7t7aR8NJ30lm ziYUl^2L2XU##L1cwQNX6r`7@lmgKfJDby#-QXL?)L;=551Jt833nR` zFy&Wxl)!Slugrw4V^oMhYmVL_YC+E^NVZoxR7CjpN_3Z{`QTejiha+nwn46`J)}@* zp___p8FLz(pqrPMB`~G&Itpbz;e`aX#|G-*)jL&+s_OJ-qkbva&Q5Jy!WDG90tTHt zvkX~O;9AnR-&9G?K`vv_Vl@{KJdA)xt;Z8SnQD(3_JMg3t`x|xPmMk zh#b0jD6>&&IN_M&&emATXI6dA_oS{)=CuphT{w3H*T0RpRE1Hy_}*M}6b2s3AE;>5 zTXeJ%tjPg~1hnO%;K>%TG`24U`zOiqUfdSZwa2{_lx9Re?Y$|8{&<^e=C=LeaB$0x zj@TyuZJl@j`Y5!C6Agd5LZmi4m<=>uvD$bt;qiF3j457S2W8QNXuH;S4sI?t6s^rK z!`PCeLC7e0f*nH{QN5K(6>?rk3m4}Y(GtS6vA0tVn;z6lrfTFtx5;hM7$GR~x7wL~ zzvD(cg{%=sjjmybC?=}X7hU#4w~va(L|2xB&O*WGHcaZ^CmLm^27ZZ591&;(6Tgu* zm`&g{XZaAQ>E;8)JZTNuRq$-0&EpVYzHY*ms|2pFw>Ezu|J^>sf;`yDpU~3XXQf_n zj$=7i5G=6-G3I;CfVAe-tQd`?MeXO?4vS~)G`Go1p>9JY5|j^|?V0=W7w(dtdc4$8 zXBxXS`)r7OFWvT*lc*=PaX_jo>Bo&>17vwA${A#eBV5G%8-96t#jr(rteJxA`Z#5_ z8u+x@%rQzQzA5&RgSa8a8Q1*z*HfG~bzBe3zf%Ndkq+CFK9u}qyI_?hE~97B@^hFP z3Ug}??o_61+k4$HJaqcj z;Fg_=luWHM%m@m$+;{pz*>O~((Xi&%=JVmVT+;W=&3fQ3C;g;8(O7};dUFC_MB~^j zP=>LR=y=jW-xKM#Nc$N@>wOvd$FKxz#?8VjDjK zA0%d$16hS+atHJ-Q)NFow=1HP5|k$6!`3hQ0;rPNV#w zdPhCaJ)+kbYR2U@np=Yo6 zn;WxExM(zmi=l4$!$It2bfmSa;Z2-H&?yN(K4r6IgbV9k1|ZC}K{|xP8xQdP)q$ti z&t9tUB`s?Pesu4^D|7>hinN}+O33O34}OG1{oR|FS4!DV2Icg z;Tw<)cw@{EYTH6?`p(ll*hx+jxWi@wl74Vn2}8hTQ$=J;y(O*uj8cz?C^)(kAo^Yd zhF;L1KRJ+d)e>Imw^j)-lzr)lX=4W_Nslk2?HD+(m`0jB+-WXt`O3XwcZxX1Xvv)M zdgArPv8dWqYln*2s>vGtlU@1bt^PWPgwkCPySxE6Mb2B)>IK7sRaB4b;MS8P=!0lt z!WZ;1M%^j-Uba<@Q0by(gr5!W-7M;y47FR089ht4GoPhz>3I!zvdS(*Mzi`YxCd`XUk!^+PU?K?001XV17*n!1_!6oqqZEL7S4Xl)L30UZE+b94*+r_$ zB7$^t$*(_+S>+9Za!Ed2oStj-<7RGi_j5r7I&L9L`K+zCXNJ%kDo8Z~~wE*PNf&IVqum5rH{ae@$fg2gfT}|(==j3ZKW=aYp4CZ^R^EwyW zXiH;?)?%1xmQqi|XkKNkL{f@sT?0VKwZKRoRFk6EsJ z&(Ak6a9;sj9A+G=3pM!=N*)T?7LUlrtB|zZSRj-}(qz&}fZ~%$_?9lfIl8~wi9Oa3 za0em{oD#g=EQ@tsoi_d~Duq|ii_KEm+gBLdf8G#`OE@BZ=oP&JkSVY!&$VEvHj~Ya zk_fFzTUr39vA05f8B!S5==U?^{g{%IWsk#J$cH!2z|?BZqdL9jk^G?ko!@i#OH0zr ziezT-jAAYxQ<|OJCd{wq@_h4W+2nfCu62;R-14v6Pe7k}y|4oeOCx0k9=!-rWk!VR zA-O~fv8;I1CQ374d%Wui5BifY=8vFLW};ANEO5P0ct|LMnBM{dzuccia4*xs5KY6{~8+FZfm1X&f#^8P#3Y>gf41(bN zOlr8WTCsF+zi3vbaSXdJ#H^Cun(gb=uLK+(-h>HT3lHtng%~XlP}PTd=y>&F$zik_ zB=8tNG=^}IqIbG1S4iJaMtDvQtoDPTt0O*9JogPPqGIPsF|));4ncDtjp zF;TaNgM=WYk_0r?s4>-*fCc*A$;%5~dpFjQ7XueUXIbt*$24 zp!b35-dm}kJN18jCft)L9`EQKfVt~!49WS}4YGRJ6IkGwz{*1hl-&2M(t)$++ZX5 z3uJ%vmHlnC^5ZDwI*Dpt%l5iO0U_)tswer)(lCux*l;*8*H{^w1u?Y0g+1-Aa~0+&kt3|xwt#Ow;s9se!{1( zIEOXE{=mn-bHxq=mEm9e$R++NQjq1K{8j#ar?))_nENFQ?7Hii8c@{{o(57HUU#NSkIcl zKDn=wRNx1xJEqhvY47ZS)sCQ|A$_PSU*w#J&!U=z)HMg|H)j~ji94O7?Jsz6FMTwmcTiO9&Ls1aJTk(G1~?hK7nF4Tk*QDKvDSnW*(on7TeN%PtKe zPJL9E2QqSJPn~Be>x{|RR+gH!{V9@l+YK+z4@TDOZ+l~-8EJR(U+znN&)_!I_-rpe zDU2=5>ZIFD&LS#Qf@Wvw6mZjZD@!@kdG*v7({;;BH5D4IZL*5zZEb>zmpL12Q4If- z_uA=ehCQ$dIBiuDsc-QRwcxsi>UdR!(HNtS_pk%A`yq`SmY{y8ug1Zoznpl)qDjZ& zKLgFV`{GTc+0AgPK6CdtC2z@w?z47$-3kFPn|QJI$^olPKwh-HBS6FPbHq?B){gY8 zMnJWxmq0H*aDC{UD)bY3N641Fq594@!&#EPj~Ak115vyL7G5zpL@D0qe$0EDX@Do( z>=OxrM8nbkIm6Lj5yFMdLEfrNC1--#o&!8J8e$Mn9fuDia&3$4O_NJr(W1p6L zW4L+Lg*oi>bod$DbMRJ4n@tan+gvp9kqs`r-GRo5@*!Iv~L=8q; zILvVykPe?$tM{T^Tv`OwNqZP`vuN5_$hJV5I*0Suz8ea~u+h||PG1fT1uE&;WSoQx z)^vsK+pCH`qNp2&BuO}Q~BbON7mA5 z$>sxc-fZ2=*)an>*n{TYJi1KR$?w67Ek@mIY+Z+w!l<;w^9`7C65Bor+Poro6oO0j z!ttmztw=@heJN<7x#`bFss6b0nH7|H?NhX;zTv?4$x`Ik%`w8vKbReJ$#o>kw6WtA zfVP3MRHIHDW?C*6f8@+AzWDMX_Inrtut(%FCM~J{@UN!H4w!>4tCSQXBI=rLQ^u@Q zyfbbq<{drdo}Y1@*G0o^uS z%dZB|D~mCHZ~tHz@wC&4t5Ere$ouv1kjaxLdQ5sQ`Yhl78-v#E>gM9Qi-oS8DHTd> z8n0HKPl&)b{ERxXQLD44w8QIFXX<-n@{bX;Rb&O7f6WJOAkHsW#g0TWm zd)7MI2=?FFwK`K~HX>jcpQuNgsXe|eVe%{wodK&tb0~s?VkGli;gs-F6s_7)Kk9P)+*#Ich)-Nat4&-a(DY(TSFUdUv^3$f=K;w1Z zQ`dzqhXTUgxl1bh`GqXaG+LM`*@-8=225}$3lO&Nh!J#RA0K=A)Fx4NrDmYK8y$YB zuWNmg2Wnt76UGo{F6t}wv~4F#MVR%}E@jn8%1p;|?P!8VBlRNWmqF+5&}0UkPEvXN zlZ2Og?}h9K==h4F)x52^7c6?EEv2maPHu2gtOO7h%nj418^Ax&Xc?Fuwj~^a=0?9T zvA@v?g>V*yMiXlGjSjHk=t|iW`_Pf&h01Kjnqh<{nS`DJ_LhCg+~I~x{Di?uo(Nf? ziAY{6)|q^WBgNJlh@fLxkNvE|ZND*J45;hku-wfBlL57~rEi`lSY3&rUw&X7oiwcB=FPIgqU~`*iEwoV!JSz}T2P2Q4U0 zXmRTdMJdHy@>v8FH;l}f#P5E?`P>_TQeVKXPII<=u(3#ZdJR%jNaQ!_=S-v6a4VfK zPxSDao1S^q8|ss4k8SL=V6d$<&8E*!OISrh(d;T&3wEg9>P(+x|nNRz4Gxu_#!0?+uA(w`dm}HK@BXP@}J^ABT#>b?=DrJ`;^ceT z)~m-7@k)v9@gC&BC45TuHs^})M-rQ&&t6VBX5Fwd_Kl!X1Be;!jzM6f;BYXB1GDN1 zM`$Iw*6%qc_g^K&a{{59(|ZCD?S!W;bh@7eZ1q;s!O_NZgn@fSUcxS?`opa!EfP)+ zC1!m>SA45hcYrd_=0a5*xfX-S?nFPpucnXsaY~Ul?eeO+dijeBGP`)ua<%CAX;iK& z6sH6oQt%+rF!2-gE88e>0~%F6$|JI=uV2O2BJp8ta>P#tf2hKbgIip$e$-B>(C^FD zFN#f|eQ$xi3)gE`*Zo-q?foO!SZQk^(X>3|G2f=6T-*5y2q*=9+G8nt@GrrMPikD%aKwS`0|q>WVZ zLmLa^?TK{pgZuE@*Q(jO6>C%o!G8mRBgV8k%Hh2`HLI=B3Ia!*=}<@y!OzA{tsAFo z9+(6@O>#L;EUAOyN5i)8Rh0&q-6x z7xCRY5~_bSA-fOZm>?!^Mt_kS{S(5@@R*43ktC7avy61+4-0UHSYiy9lb<#$n(N29 zSlX>E6&~mO&Y|w}!kcGcF%G!+|74qupRF(XD&3x~oe8}Y_1pCtxyTlC`s1R`Bj^N8 zz!@tL8rd?(brw;O#FD5=$YKZEXu+H!IlnUt#Nx|Ze*9$43^Mo7{L_?7?rEFpyQQur zDXDO6uuKW!{r+P19d^w2L2f>#A?s1YO<`5%#_10Ha#wrI-sn-b8Ozidwd2Aj0oZlTh|w5RZT04~ z<0#4eS!ccoTu=&zy_J-ZDmBX7mdp-EzJfbup8j1Y%IBj(DLpf7pZka8%a|iI(OGtc zUP-rK%J~>=jr{RZPt(7G5NOW=5#S0`vO|nfUv8vc{SQ47D1#eHv80k$%_xQ|<4-(B z+#65A&t;z96wr+rcYgxv?@ya=qc1<$6R2G$ALP)1fy!JgsPUZxbqk-guX)3;s`b8m z`A1MzAA6m;1dA7F&U0~LFSlsxA!>?3|L7XIj1RWN z_DsSg8tZ}bqG7GYC;`Im-T4S*zvBXy0E?v&qJAHJWT3}CR_~t=DIZ;Or5ALnO4QJ7 z3SO#0Xly51as^LsulpW5Cb00d4(KgO;?MRKbJ~FB6_ydb@^)qOLP?TVoi$a{_ zIyaa_bd?^!8yoNUcTsJd)O()y{@->k64=4?BO)k7ZN*_m_@jUvQ|D)^ zjOi3nIRs976B7D9Q#vAeXi^w2>8QS)d7VUwSY&*@?{9rjzo%u@x1sK7`YKqljwJ4l zJv?qlN7JS*CX6;f?^;kDOxHwbdI*s`8CUBkU6cnhS+xk%Ra~9lUopw1y3=y)g|EM)|fq4Ue&eH7? zq6z&{=QubrTeUc!=!vaY|09{WFI>9fy`=>}#oJ%-r`xe-ELK&Lx6ul=&c9ox8d*mq z$11-j6>~3l`juryf(xw8IxN-h92#g>b<9u?Et)ZSy!-)D;y5ZXn!RUU`CuvldhjZ_ z>!Y5=GgPj?rO=uY$S$JW&Z78I5SoS^x`iS^1wgVK@2mN8DtY)ro4j`!v`1zC zO@@x-h5;Ys9k0dSmRtV;Zh`h!jcl_w@0cZFo*%rU(9cmcx7v3V)7B*4HtGMw?JB$0 z&G8>#xIxLmG$^uhs_erTEp(GIec^q-LBV}k_q7`9Tjhikiqiu{i_EH4tasCaMLuVl zFYWyY`tP++EgPTDi1*V#FdUT%9YBZunWul|HL0b~3g6vBpl|mt=>M-K!8<(gMFbos zNX^{&AKcqV%b^4ce-tlQvF=7?lAo31-a-drVBlnaPIUutp0Y{Q$_fu0_i;p4PSPV^I_u2R ztLgsWvAEP}-NQ8Aa9d)8b`c?$ilS*9&mTl)FJ@1uIhW=4!3glCUckoF0u1BjHn@Mr z$}vrT5P~6pe)Heh|JyuH=>(1(;s^ZKB*TisQk_H5SF&R`8qEWoid|cdP8wcM}_sd_B68 z5D*xUACMSO7oY>>3QGWmiad%+7GsQoSCuizYC;yGf>o6U6rb;b9$*0RcL^ZPw3r*i zeqz{VbufP4%t>(-VvpSKXP!gICdH!Tsd=m2wHgm6x8F5(BnV_3M*PrR%WE1b#~P@8>Trz(0^0)1yb=dpjOxos z%nHHDqu4^G)@ocC3LD4OPEr9V=#`pdvkjvCiz9A_>Re*tXQ1U zq&D860HeeTMzQk+DWaxs~0G7nZ@Bp*>*e%*bQEr9N3u}hNcPd zMvpNX>$@&sxjU;LibdpE`#gI5=c(uGgQ3xbxyMK%&)LjbRk$F$UoTZIJXHQVQD);e z=0o3qT$-~fqLShIAfNJAH;LXfG)j9!()Mzjk!@#irk`>k)9J{7M3Lzf{1W5&faGS0 zin!^vH`dg27Dl9S?Y>wl!|y5`ql#i~mM$$*qe?$TwPiiRi#hI;R=$NM*zMF-zKLD} zb@bjM3bT0o=eV?~uu=~K5nwEqS?tSR5szEMa;cS^eFv+JPkqJVDQSe(?i{)*}V)SJ_f;2b2qQwZ_Lp@6vqVJ^DO{IRATieY{ zbA9OKvE1`}csOML%2jo2bzpU5b=Z<`DiKz+Qx@xiVl7YS%oN5jzMCMzgG%`A14ky~ z=MN>w@>?C~Aa>Y2WzHJFj4*ON?2$H$6EYaLZm%m!Ea{A`(B>u#AlEn^`_j|IuP4|@ z#Lc$~*$pHG86S=V01Q^@85i#OmywKgUm!-4aeAJnTQ^?kZxMGV=Q?2ef#CvlV)lxK zftlDrt$s@_10`v+HH)DNs;Fy$pM$uvgrpA^tDatbw@-s8bRihrNJ_hYEvLL3yh8o@pI`gU&syEj+>=v93HOLzq<_WNVS2$Zq! z_RW^DrDSaFIAtxpRp?RpCu4If!hJRcy~j%#hOumTa?-USJND7I<6#+687bi2>**jN zb@T_&O3b_ok330ssVT1{4vq`c7#Ffq`d+|PbzZ(9KeD=nfX_7Tt`nNZ<=9M>bi$G8 zlbA*^ZknP#esV>Ep^M@PUbc8CDm?vV2Ixz&L$s;QFk=5&16$E8onOP-^3 z#4Oed8e5_t+U0X;r;8-7-nB{|zk3*f!@L90Lzzhj`BS!v^C3t5(-mVroRC6-pa%Va zw;}%NLi{bdf+Qw1Fz+QK&q7ku3{%!=8y568O%-UZKVfZ$L8yqVDj4WVx&<&8#3du2 zVJ$74h2S5{`@T{M0&`we2HH;|BN&~jT0Ku!T_>_yd4xdzPbj%jD1!l7{mL6}1YoopP2@Ro2<*vr>v`Jsa#?ZuU|{ zbCoCA)2jWKa{K^I$)Y$Q9e#he58jD%=u$RYT~3G?uH)}$z|{=V{C(J)j2n$-Q-rnb zsiR})vT)~FspsW72)<_%kk5d!Tl|YpXXV9G55R5$G6Ll)+u*9cv91UY$XeYvKYxGh z10b4&j_c4)U{JBOr(KwMlkAB57=-?Q16*NV?0uWNmB-KX>6Q3DopDu{SfDndZUfo>UD!$9Lq1M5XlaAW=btO{GL zmC0~lT?7_JIqt!bIcumL3a#1YkSST*hYEgd7PvwB@p}*XL`6#3 z`y{jV(`eK!-G&-;yU+)*IK(5b&X06^oo>9&;s)KBy{S=LNSMwA&!6a)DAvxQ!TihA zEO-S{M?;S6fe}>TKOCnm3@y@L(8q6Sc8!Y$NgH)1?^8%tE_b?22l)?KB-;&~t2?_J zkD)`1+p&?jv7U`V+=ch{1K&wSPMQ<#)ec9>wU^%?*OHjsqI54Cji4uo#*?I=ow4k9 zAs2l0nQ~@;3vQd%)v8PEo(8r4D~m*X)82dB2h?qiZh zp%^Hkuf!pb+iN7rcC(uFCyR~)SI^leVT0|y?+6>X*IoQJ?}siTcGw4TV!xWk8S~h= z;d1pzI?^~u@L~H*zIw1GC-9{q-(_j|2yYo6QYVBM-Y~DWh_A^@l zr~%)ec^Ob?uSsy z#}tTE01-e+_s$rKGmg*e&>6{=Xe00F!WMf&V6bh_pu)>9P9x6o{MU{LcJ)rKuXct0)>u$Qz+- z8u-EAWU_p?2})qE+v&`|vm!i3tr8zY-lj3~BlM!)?C|b|Grqyo7DSlDef|<8DZFb_ z2|ipz4&{ZGj|i@B*<%Y__g%j+3UNV_{Qj@!=KCr^)%%Fu7Axb+@QR;^r~{U#_wb^^ z6gM!6**03deP1-4c1~@vJIJN@!amg3l&j|`dy{(>5&yE<^qdiwf)Zp>J-f6KM)Yu?spRd2}aKE)ahEXFhlV49h#Zurf!s`U#l61o>v# z)%^(}kFXz?Z#*L0Y`DXvtd@PM`6>E`@Fp#%>-w7x`k5Zr{XdVizG!mARR|VD|Ahre zVE}eD0$_mSvfz=xW9A>>z+dhl)AA#-y~2>2SL@NCGEP%uRou#-V-}8%F%aHejZPI! zUvZm8tt2UtDzNIAq8;5J+*U|7lcHZ?1()0POkiIy_37&A0VX$q6L|zrlOvf^oui@{ zO^PmvCQ+v_VJ!bEK*W$3(XsH1m}_yW8-X%h)Akc|`=PbrTi2ZEnJs00jN~33-_b4gb1+Lizol2;FHCl8QRN(|L*8#q z9e0I9uSV~X17ldaFA7KCMnT-{QdMG%%_qYo{z@m5#kazV_y=q^87C zTH304@yqJdsmx2Sl4ko&PwyE6KN&^Aghb>nm-qC58gFFc?&tZ?cW|dAZX}f-|2BDJ z4m4_1g9L*4UwIW0i2wNzy#o_pL&h*B%s(Q5{8dL3A-swJc`zN~c1B-|3${6_LV+{@ zzW~j2IvA#E75R4xX30!(5yfjdl2BJ18%oH1S-i}i8DvQUEBoEc&JFU{n6>}|s3%wy z00ADi$>Lr>^>lDfvz?8RK0+v6+%B_;qvEKJqjWaCE@052S4N*S%vX(<(G2Q`uc@0St_`=Ra1YHg@1h=d&k62>ZYFl0 zC$;1DD?hEE^jfC`g0!}xRx_!s#aV4p#S3%XWtiE?du0^;Q zM1L1WzTI24_bEg7rW?Wt@D}A~L)pE@vt1mT?6L?w{%)qNo9U@MkLJ^G-5GjB{TFYU zJd$-P2Z@3IzhVIHnnNH>friHge+IzAf+$%3^9=d#bXJj_9=1(6yx03atmVLp5NHJw zDl$^mQY+Dm?&d=eUAU6^V?(_2oO|Xj=?;EK0V9XEvosB7Z~K{e{FCMQ!?^v#^Zm;S z-&e$WuuelK>Lb11p6S$Q@A7Vq${$Nq*F1 zxsU8LUArebf(%jgHnyYIriItr6>vi2$ZuHw(OUh;(4y=T6*n1GG_R$dXw8N@iZMeHWZ@R z#XSX_pWLSZAW!F4-I8nmLC&9tFUmJKp+^dHUV(g6Q%x8O>UkLJJg3yUeDPknf+D=t z7_RY1h%$FRIkG6MELP7uul!Zh;-6%i`sC@LbR=@%O8s2EUHT@UVoWyee)oOPp}h)S ze^cd!5Ojb&MD|2^i=8CLpzy zv^)tF-B#Jx&;g{=;*kavMN6)kwSh+}!GPvOD{*GOcf`N$8SMdi6b^M8pIrC+EGHDl z&x9a5FBXNzdh5clP|Jx$SnL=MjwWZ=flEEWp-H!KcoGA=jklaCBOg! z@(O==r@_FoTi)0Vu$>!-I6$fjIXn-VkBCmJT;}dmZjJQnV==P7Cd-4_P*6ywvE^!F zTMIdwl1Ky3+t30IlbgsZqh+(%F`K2Ldgy%_a4qnD4ZSq1Bv#StrpLFk+QYt07BeNb z(~+yED_$2Jn_q|eFcaro;Z{X(&qJYufc0JHqirZSwc+bBRgX|a#|;JYaf3ZKg*$81 zeo{>mWl0N7?c}D`u|*NEzFEPi3@D$5_bJr?ze!zave0$?3DS*l0~6+SvoRhh>j4>W z1;KeN^9^bkP2Us?SDIy{H}r_cTW2ZC=pUjqnVrdGnYhyx6-K^TErUQOHzA6k-y#`o z>V;mDMKc*Aq9U!oOh0^5rrM=CDL!EEU9{G;+%UVqB@`%1;J+z)d|cN`CVCN$@5Oyq zI={?Op~*Dfmg9=hQ;(YMu4@bbCH?suw9D+vc@wCym zAo=F@|IIg0e~sLo8~g_mU=;X|AjrJ&UsT*ok-@S9V}3X>4Y7f7JEPrXADEKLM$`zg zPgveB)1cVQiC6Cys^F(AdUz@7faP2CTNn4tZ9qfJn!@;6n(tjUJGJMRx0efyE+&4A z45}u1ZmJ_HeGFs_1&kGm-P-S4h6q>MG(npVxUi>S72&xabRs?N?z#{V9+7TMt;H6P zCgY#Y_agnL5e&E=+DU4%eE1Mba}9wA-95=t(|i$!lu;OX@E#Omdl*2)7pU&y8Nm|p zEQWfvH)Met-Q97L9&~l4%FK&Aitg!S62lt99AJGVI!X=uDLwD^s3yC_Ic^feUfx1v z!9<&)OzH*_44B_vL3t5`??3t@N)cD2ag360wC#PAl3OF2lPFX7fL%Xy(GE?NSkr_% zxJXZSmXD_E1lPkQU z6Tu3CFWmFD=-*GbcILM2)C+vw)uQ2r#Lw>_W0{S}BK?MJuIv+|SEHRBKNgJMR~UEqHALWI z)Kl%rk7hVA01p}`5mrPA1Q6VER|eiCat*NBnWBqv6|?kwHcJ@?VuRR25P$A@_uojo z)~KU7)O23+^|K_<&gxfrt)se=l;?*KL|yvy%`|S&m73QKf84Noar@)7A4sYbuS*a- zmv|o4QAm?yL(TFS1X^iF+b`*xg?*Z~kk0hVJBu{HWi5#k{E7XAahi5Wu6FvMKt&Pt zHWHBad?dT3N|XFu^*hKWpkDHdV@WBPxi!&qjMOq+W8RH5kc?uD~5o98GZLCMT@f~B5W zn!llxxQHgS7WJ=3Mec_p)Nn}ns{h|Z{?{3IE`YSnzEZ%mf>snUR4@%n7_$tEZ}Z{l zL>YT4bHz-zk%Gl#yvja8v0hw^UMCZ93#gT_Q#@)U3Ihc1-# z)AEi8xBuYY&PG2C(=LmJ$1n-xjlB6dvz(#8KJW6>nW!rX2UvQUqH+bgL5sCAHgd=M zX2VYa;_`S&dnWIReM4qQQ2d=K#*!cEmuimYb zAhr?HWdyX)!Kvx6ZQx@_+o08;%Qw z0ecCH26w;d1Hs^JFBjdC}RAdE&`4DwbnN?Rufmt^(M;jdr)LC z2E_nSZ@l;ETi#U_Zfo0u%?lyF9UOEz0moGf4SrP_qI#8~4TfC1`DG!hqfy0`213d{ z3hjL*>JH2LF1Dng>pxu@hP9XpN7;=<$F1E2VkS*jGdJ3E&~qvt0=1Z2WujVM8mhb9 zrI+m2#06a}vk`}rNUNbWM5QU;E78%GmFGKHuz(JzoAD(Nftiq3qTWF#*MgE4sf7d!sm%XppRH}nJ(6iG%an5l&W03g%u4& zj$I*cq7fHp8sULEu_cgnnXpI$b(rpPaXjES4s&j)O$lteJ9ybHeL|*lA70PuCw#_t ztqNIj=lD~~g6~6@MjgUs_zMMJ;0R$Ru4qIOF_WLzBl}{x|Fz#3&TafJrBl+Z4XKTB zw6DZw3cCGH|EyCZ;brc&faDl~{}Si_Yau&C$jSiTWP^tW*Jyu)1r;U#-F6wAc;}J# zRa4?)H4RL&p|%{U&0}ETpc`oI zEqmqJx9R5_M7;FQ{JU4psmO2;n5g4&vk}e_em5#X-o%r%W*3Y%cpVIzSY|kBN9<}* zzVKVjHuxw{Ul_z=p+239tCNQ%+bGhxw59o>z1E8g#;Ay-YQe&m}o=l+<_A_meU zV*#2C# zpNKr`$CUCgJ-6=|z1+Gz2&3$TSc3#fI7n5 zcS`wUMU6^gdSQb<5uu+*L6b1V+i?!D9ic6!#!ILOAp?a-aM!?ijPak1!?75Mw-UkR^LQ~E0l1Y+Q+Nx+gN`#gH%p50p7SS4~tHpjQ7C zS&YlmYha@Fx3jkm;irjU!qeZ7j^5l)SP%PC=aE~1xYH`;N)OD-V(JQAV9TYOkc)s6 z^rsoP@i!UAU8)#Dd-hQP8E%`s@ySysb&DUPt}o)(rLvROY2+OCzVhDQYq}3&yJA-;TDb~q@--_T0ns*6uuyU9Z4NBoeYgu&_LTS<` zUfP!w3ta<+q*8%9?IfmmW7)roAjf9`Ky;y_Iocgo9=%^zwnQkNtkvl_QDj5sScTCE z45nD=`|5xRA-QB(>VHV~D6tqH)44@?VF~XN@>2n8dlAM*#$f-(@DKB$Ri3H&IE%FU zB0qapsS6J{^g0Io#L`RzM*Tyb^P;aQtDMU(=r={b_3;jKu;NvOL(>~ad`mFSkfW(X zY#JU2pS=rRlbw}|sne&q{cczqCZ4w%WOyGKGO{tZUD%CCkuzzJxRmo3yy9=*TInJ1 zGWu@@`@frB6I}kThW-ODJZH#eyBO6I2XshHq((E+qq)4Sdr_{vfQ?OKeItc|hrwxW z63C%zGo91RmTDiWQ}>7KNba6)UO4*o{JkH|4xu*Z2T9NRJ4hGsG?P8)>EsFGv3Of! z`*~1!N_hCIX|7$aNv;i9ga%<%x-_X}PnRrgR7)aEZ(RugJcHd00Q$fA3I5$Vo( zrH6|P7rlR)KJ>(7Clj(cVE4%Vq{W|UwZ<@=k(FSS^wrX~aS_;}f4QtuxH@??_eFpS zIsx%Vc6ceXM01ru5vsujE>Q*shLKhI#u>p3prY$^B zxqP~X;#DGnO#qzuUkd$>dRC&NCOdDrUROZTNX0ZTl4YfuFLuKvdW|w!(SInzr)@tT zQ516%^NlmUng|kfK<)Y3zNIAAw|GCoIY#o6DFT*5#&ocT<%@M6$jS=mI$1@rtIxdx z%_m4xu5T8#eb)H_Mq$qfK=`q+J5bFbTY$Ky0+3KVK|rSW5?ur>IrK$*FUVm~A(Hl2 zmv5xEU0?g3c&9*@aoV_e4?6PHacXU_YdWGS*c0?OVM1rdjQSTjZT_zZ7KED)N<$!q zFmiuG0uMr&*vpZ#e*7|Bx#nO{xi8na-(IC!-twr!(}EE!sxeD^*4Pto) zLEfdbtC6uN;-=*9haG%mZE#;qhU`4J4jJF%S!xi)I_Vl0V=~;)X1xp&n(q;g<3Nn# zu`~(ch-=#8Lj3et^m<&_Lvr z#`#6f7ssMyCD0?ABiQXDe9I&w8ekU6an1|g{rJVrWk3A?@b#8Kbude}Fzy6*cX!tW zcXxMp2@+)E?(XjH1b26WyGxMZ5cJ#0Ip@7^o#)<@mKOL^OAi#9caqDf4+i%$q6@L0?4mJ5BZ?6$|}*-&QQGU8bl9; zw3o~i(^3*uQdGn$*R0E2+27lnY+n z<6J!8%C)R$J-+*i@->H>s(+d+#zYX+ssAM&sD(>hl@b~p*r!+;w+;kK79MOXsFW?) zklKZnRg!*Kgz!y&rXjdjfVFe~nTx%2i#a*2KP~OHva6U;sNZz<17RSBO|NGo!FLq& zIDHIJ-HCF8W6cUzXZjqjcrsQiDjdL!KxP0KbZMKo<+Jbz8D5d54@mmDEzU0`=!^pu za4BZrH6g$eCE$cD9+1!Lw40OZh#<4^$b@8~`LM7$RR0Zadmj|5>6`bYKI4V&(xgRY82HJi0sx6F=mMRQv!adF}&oIl;ONy-Dkhr4PG}+tJtFas>P6B0rnK0}-VEY7AC_&4J=k~L< z2bvXrO^UWRz&~#Qe+=MT+0YLTKD9Rw zI8`7u1qKyh7A^3HSQ>OZ^jene0WE}wg_r3HR}O}mW7x=OG%#VWiPl%C1ac1U{CkA?+>38W?;OhbTG z`UTqm_u+qk;yIKKtlUfjfB!360=Dnbq)q{l(Ew^uEPs|DdOVV*Mkh2wy!O4QMWI*| z=Hcv-%3jmWNRB@%82U=l7?lpljNQ80$~}hv5>pQN@x>W+W(Y9C_#}nz?prz|z9II5t)LJ4uzV@~7#syv1aX@Ra*p zCT=ov;=BB$IHLuos1Er!-@k`d0XZQ0k04~tKfk$C8jL(psv1lPedNQW{j8dup~HZX zzSB>JNx<|j&Xmbq4CRiyCa@~1vEf49w=zJQx@;s52_qXLpYMs;!oJ7l-Q_5&^>M@k z1W$90rPM@!q#%ZZU;-rwIXZv@Q>>RuXmB4}f-(jj)rsCPKxkquhYml)meHVtx$Pjt zN`^*uxgi-acdfe>*S~U4zltuOZbYB@*<@vMaqTq$FQGEUX~hyvpKz=z?M1FR2sOqG z_eGfC%04->=HhO=JF3NwW@6I_M*Y0RP!b>_9Vm$qo>45idBz^(@^gYEioPFNpk81E zU2WnK{@PyS0XyY$Vx^Zd;c803=d}(_v1O#;xYu-m0HpGsbsE*cTF)~^4V3C)Js-kZ zVOOG{H#@V;L@nF`21$iLa6oTj>SGiWX264uP+B7O9($qIWqhi z8faucoubwYX47<&K0vr_g}y6OZ;8;|p>)3uxDRU#F8nRmvse0`TLVL9$$vuU|9aQ~ z9f=>m+8*ubclB0GJWCL*hAFu-8hgOfT?HDUK)Pm#cHh8t=>aoyIA5TeY#gaPb)}* zU0p{7yU0|-(8(Po4y{F#zo4Y8PcEb(zI=)v@D+nHo@$*I5snpmm`vD{_%iqme$$<) z1DtS}mw||q4&?!0H=n94KJL;ZN5Kwkzh1F}&AxY#F#58R`3);Y{(;%;=_=?>;}B6A zX<8qWu8v2?d|OCx|3U-16)5dJ!c4?NNzaPORx>u5%UQl+o@ovr%>t+Jxu_#bt%XNb z@UO40FoE}S0eAP)e|Go(B%Q#`rBA)|1Lm4O^7WrUfpnvZwtZySW??ew%L1(EAaQol zo+VLEo{oBVazPmir6&*~iX%qc==1}*t;x*Pzclt?OQ43=$d6#n5vl^ueQ&LE689U3bf|sLq2aHB~-&-t7Eo)rnnM~^&G^K=T ze=H3`vmP`>05%{B0R7R4Sk6|LA3j&#kh8d%`ljk}z|A)Uz?0_)4t1PrSe}O(rm%15 z^Pm4Nv6*4Kv<-NaL4eErXCR_at?>W5#6N)uIg5>Du1r%?$C6~Y-;jyZe*EM_o81!r8hkU8q}dYno%9h z-d2H}Su=4}DPB@-<(0FCHeK5PT^w&MWi|e$i@zib-mPU4gQ6WZkr$IDQ>~;3&Q8vi zQlw3NZidVM%Ajs<#T?6O#ZdxTw*|LvA{xg>eLx$0Y88CCt7Sb(-v^%!Y7()j`_H?f zr#)Ay1-QJIe}*eZ1K_OzBrss6#udR}qx0Q6EDJ(GLH!ljwReLeWQ3wjRpdgz#1@Cb zJbq156^F8VTfx!o`lYwhE`dTSDSLjfNfh(2dX5}U(X6PoQ z=a%N?WTa{*rB}r+0{>ag8~00-i?6+qF`;6BSiC>Vi3hSFgm4~`A30wf75ot!bDg!F zh2a)V$`M3@t)Y+_@fc)LIyU#;Y?oz<45J6Q1o(fN3Ge}wn+M)|@Tt;)z)K@rww1_9Y9z`Iyd#R74 zegjF}^wCP9bh~~T`uZ!gBN8?7h1cc7@2&ZpG*|ld2HVsf-bK}1+``6+%E?vGrhAa5CAHK8T!`|Dc%+tHm ze)9)bqqsl!aNxiUn5Q?cRmiUL$6Ab`Uha*`adL?AVqod7U%&GU&S-w0)+a3TclC^$ z(!AiSs>NzZGxHwxf_nV;D*?ZcPoRAQbc+8JfZ)*oPmKD@ko@Br05fMR6BRR8RWmzB zTO-$hx84ND{yo)VAA{tT_4VkTex%>Sin}-i#GL)7Sc8H%;-_%0tT73DMZznMlZBxK z0cjsZllz_>;F>1%%;?y#muhUS+m3n7S}A0^OmwV!%)EK=4n6Lr z&=Q*3G^eyW=9x)uB-|-vX%}g%RP{zp^gVHnRtvEVO3oW$GXbZQ6)|>|&I3K_vM^tK zq2Br6V8M3RNgi;Czx@qTx(}K~#X#?W`sX43Qwsf;_m>6&ha3jSP00(hAP3KYC|^VC zJwdhm1#=Ek@W~m9>~)2qrmo>Z3(dmPjeE-cDux=dW8jHwA zz_nH$%D{;Zbq8oKMc+>9jAh39M`na0 zAmXP!nAgt}2mLVU^Ie;IO}y-R{nfTjd8Awgo;pv_B=k-D+5aMjQXEV1B46c9(5*>i zWw8VD_S2uJW|oowzX#k5oBs+2V8FuBG_XdLDqjgh4Oo{33b6=S*|erET6b+@Xw0eW zzL1x}kp7I+pbdyU#id9vKXY4N*Px*;2oQwxP2yfx_f{8wc#|im<2-SUsjjApNsV0-fCR%Fo z38$o_6)^n-*^04T`e1)5%l$q-`3rfVKexAH0N6RKL9-^Kv0f?+ufY3Ia;$%ofGX3B zK4oOY5N-w}l)PckdbHJtRkw`<r>N<{kf)oV5u{()%+tr2C%l5;`o(eT@@$s2*z9DymU9z^(k+6; z@vAiE3ttl%quNjBKLXNprU0DHIqiT_*yiAg&Lb(?wAuI0*xBE!(%5gNL5fSOU(JWs z6$bN!sr?mYX^0`pKz=ilO`-`mkD_e;y|?^vl9M}uj`}A@{jc=mZ=Z)@EeuMkb`1aB{o=Kw^;iA5@ zD+MqvJ4FP)g*mlc87#^P3%PE_Moh1`sjTUChVmSzvoRfedv|w&mYFO9`Q8OU#y15# zjV9)NT7#h>FEn1G_6-lcama3+vfX{#S;wbXZv)3ev3wUgVG@DMEWk*N>#yMR4m|@` zQODyBX#hL+_`W_={d}4Wu9vTZWdD5fPmBTWLnfKcbzu%y9l>4F?XYCQ-k zD$)!_jd=X?C|u?h1}5}Wr&)}KLRhFuf}w=Sl2Rs5=Q60nuE1x(ZGrZDYidGNK7=7y zM(h?Ud7~(-kg4#ekA)e(FlkvFyru&7J1Ku7SS6l$o_0H>6s0BuBi-ekcu#1&e(xgm z0e&ktRmQk^f)(STe*&4 zetr7NXh9~CGK5DZ`qGI$VR*K{VicD?4t%f{xI1AQ!xZ+^2d&O97|kSTzo|&1eFP&J z)28f2#!3^lbQB3%q5LFX=x6VLTp+%i_2wzd<7mqH6Fu)GZlfxJZiM|$H~NUWi_<`rg!(W*9>%ihn0(f3K$^qaRasz|Xq@{{EMf|J}O3phRvX z3=UxC@8h_quu>Drb!Nt!qDzAz;d1aSnwWcOjfRGTka&^2MJk#@U;|F| zs=vHye(~y`yc|doLdmCsp>DV^Qnk@+0K8CrS`SqczqxnQm5;}L*fn5DMe`h^Jg$?Y zV=|@KW4lEH%EBD;LMw1(b8h@5*r4`FZP5btjzRS#x-i2_La-Y&(4wKnGS0>n5gvyX z*fgV=Q3)5n56_R6;~yc#(1vckOG$hM$Cb5XIe7m%pqMlE>eySyhW0LvVeXrjVt6Rz zZaq6%-aM4AfPhm)0vST_m9C6dW!2Z;Nyk@s578FlRwy|Q%>UqfwvrM9aNcQkLk;CP z897l;eoF+MN%o<_AN%dJ7oiFRZ72LsSN&_hs}N*_RFP&FbWo90Z(0P()Fn1(EI`;7 zN9%kYV`nK}rDA<2XJdKN|CP8)>Vwjk{jbcONI6L}MOzhN{xUHgQMDi>pIm^0shOHh zYg>Q=$%vmUkX1eyMvXoKVe*RuGZ+bY8pJb{q%(+TgQAjp^#zcn**Bl{B>|D18jifp z9~i%|KnVYQbKFeiuR}mD!TqP_{T~bq`1NrCUloM@=#FPqFIuW01LrA2+#6nkQR;UV#cqM-hrDYOUaetZuHZzd1wJi5lcF%%X%rv)I! z|0Od2$0Ts6TCFg|0N}$hfYTVQUSV47(;mTK(N$v=#2*4ODwmaV8+_xWiIL#;GF~sL z9uWbN5eO(%5VS^b&a^z(mb8^xmDW}o8|>YK^pE_`k|d%;c9UWvm3lc-qnnpK%0zk; z7-N;AgqDgMs%C7dL^E=DGgns_jSz_^V|_dq>2x06`20B{p>v1}~hR zunW6BYQr3NEh67iiL?A|Z?0LKTb=12LkML_DG>pRV?msG%Yf$wa z@fQPG)9Jr9fK%vv-`*dmB`ijH8%iCP_!uE5LR0_GebUgWC63p_P@L zcQUK7JgN2Hqg(@q9oi*)#4z!x8_FZiaz?oF18Af@*79sDh+6d1zzZOOKzV8Fof4Np`xI6pwcxR|0IC32#fDpZE z&~+=!ejSX6H9!ibc9@eQJ61+N(g&I4+Bf^9-+6E%tv&tY%6WleV1WezRm-0O|EIyu zC_v&S6*0>K3LyW6(D_7(obHu^cC$u_m?XWQpNFTVjlqM*ZXSV;l)&qpHjEwSvUX^d zaWEn%ZVVEcCLKdbW{X zI6un;!C=JiA48=`aVr??CG&lCx?n=o!3|SSm=0SmFBl>N$MnL_459SLxqK;&Q>r6x zpW5l~gdlaeU!)BZD1do&x3W8e@x*?O6N9+Vz2Uh^9Co+o0Lh@1%5FKSD8kWjpajx2 z8~KZ=43oi~dPB00tbo)SYl-m{g__U?!cY z!vwB`Jn|vYx_L=kyijdlTqK5$q#p*oNJ>M>7(UW0&)tBfwI2Gc%7}!DO4J*Q+K9DK zEOkNI2qfQ60ghzN4YqUiLFh@TOk{_@M^#0^;v~Y6yHbZ0sEFaA(vn(gSk(&nvn%JE zN)ZPN%^e(0^2*Lt{mt8oQUY;c6ESbvX`e!#If zfk{60;R!KS)AVjN&v8DUotR`^6xQRRwJpb}vDI(847JsmDfl6v4J;pD z;Izim>FqAfMr`ZG&-0CT85bZ?%5;`1An4xQD(NN*Hp9gT(tGQ%3cp z!xo_X*rM=;N+<+bR#nl+`ShhY!MCR_Hd5Z%AKaMH9*@XSRFmJisQg?nNdue?1!ifv zWgY5|tD04-HXM4D67yPb(G1?at`5)=!UxX)b95_xJb`k~)kmy=URVoK3$~_w)%wR$aB)aAOZCwKsv&JiM*gxq+=*fj$v%Ak3PVOhw>*5gPmWdU^;oeoC~?M_ z=SpI6#XmDEA zS+&k6JzDvR(@GdNO@2gBsMJ!Dw97e@_FA^gi0~53khRQ;98-GAZ>)2s{aB+gA~WQO z+OVv1n_=!7n*gIhWVC6^yy7HJ`&tP z=xT@(!N_8Ckx{r%XnyETnU~*xQ94d^CmX)yhZNIJR;hiML_kLk1w;(xx=}8BJ|v5JMmVPCmnxGsj4|JlI5Qbw z$%(SzVOLd;g&CHyyPJxA^7Eg|?8k|s6z!hjqktO7GG44=542aH6eXCdgfn33>?*X= zaC9LEM3pv=CakBR?K{Gh^6L|4HxXfvNDC`H#8xn~tO*n4wFx8mwm|%_r>Gt~2dJR? zAXcy^^w&IA8}yPM@JbAs^?IgR6^z%C68TIqVHlfnNX{s>1s0_5vEiU1*QWyoaZQJZ zU6dn>>mw*2Rv*N~%Y7g-)ZjBGx|TX0-NfvDO=*yMve;NZ8DrWhS0qYGiYg#-;S*%c zCCbl3O)5I0HA}(9E{70f#Zk&I3Yc-jCMEr5mVJAx$SVI;z0?7Epr6kTgNHHwZIYEf zdowH#SEi0tV%$;gbg9}DP3{=Ju|Tv{o{Sc#qn48AS@J6<&q@?l`^Q|Ns(y}~*j-fE zEM|s6Q*eurKuU7+XW0?^(vv9yzV;$+;C!fdwWP6Hc7fzE1vh(syXn&bIY41H<0qzF z7UuK9Rbkw8-DY2z1f$PmH(8W&ZcgXS7xKNr-|~8fU%)qx9rNOcy` z^&9n}TMWw#}PG#|GzM9r%Oc+;@S{p&D64GYWX$`-#aOHS#bq-+*Z&8~# zaBRyQP~XK*LS4mv7(0c-T0LREtfQF>mY(!C@52saJ6^Zc%MIsNKi?DfuRZ>)0m#nS z+AGlRzR>Scu~&4{J3#dH4Lo;8!8@Yo3g^8~YonK^WY2{Y=~zpk+mPWY!EO)|vsVmr zkol~4+177Q)dX1~!-reahyKN}r|nkn!-ePucH{FN!+H0wFcDNc%FsjC@uDR;uM-%Z zP7_Vsb~sK@+RZ)TWtjt9khr)! z1mxsK_73*uU+u}iI+_Aue^ciFw1H$R$pJ;W(X&?8R@&E|->QitRUtKoGD*u6`Jeo$ z2|J6|%i`HPY`Fs7NopU8JdhPg->I+dTEp^)(*`mpO}7sEygod>LDdCV#Tdn)#3;qE z#5l#k#*Besx5MvHb5r*@F$e1dLS#MPlZVW$1WcOq!9)V8-G2! zAzRWQ2)NokY2#oxupV%(rJlpQ<3n@a zn2AvosRZSB8=bsdBZ>g%`uS4gQMETqHsxt^Rum`HLROQW{g>Dq;i5_bKfmHp>DLOt z4yrP_7*zA;P(o(JwnlY z3?~3rbqVUlTb$R}K{qO(b7SNu-hLh8m0a}kosC#j1GNGnGzY$<4GA;a3Uv{C2fpKm@A$$&Uop4;l;XYF8Vi`UE}?5d{a%ejHm= z^^5GVUDWvZz@(@SFiHs7z4j;?ao1BWXOcG1HnI7uerq=}fwKX(1cIaM2lQfWwB=1AZKe zU>(Flm{-~9Ym!vREy*XCz|d>eR9}1Q&EC}Ys z3QAEP;|{aQaGjD)+1YfGW3x#@ZSRE$>uX1Vtw(I=AGctKSgSTv6xvO$wun74>8da| z>1ZIn*;O0g)IJ90e{7o>VGGDz0I)IT8|9Q#f3Pm=syN9&F75`S1Y175>7Sf>uhy$g z@dz5hE#0P?SZn_FlvM5BN4_hOTzFDd|Hi>z@N$}c9d=nG{qv%tNSta_7)cULse!OS zE6yz}_5ub$V1(O#A(gxBE`D+tP2%dPL8NkjYX+NOL#*`49YrvJX>U`E4tHA?%n;cY9Pj?i$*R?!O;yI})!(O^2CO4|8}J0!{^RNPKOrqO!x;wq ze~tqdpjur|1y2pzSH4@mK}%XC@&eNm1rr9A0AcnM`cg_E{(4gO!|B`Z$jM3Z$KbWVM9YF{raA8xzr#c4%!`lnblcmE#x@xC zFf_hr4A_WS45>u;z#g@Wtz+lVHRlLMOeOv=fMX|K0Q873%&rwEmB>oqK58d+LN5y1 zRaFpiPMjoODsgx-Gb*Yng0!WOYOkLk8rU!dKiN)_XiZE=A2VtQl0xwANS)sZWK;-B zZz5~XKqPC7s0x@e*fhL{*l?zY-Y{tbn0`upUsDuCC#A>;4$eecVtoKS zfK}Z|eHa(hw8}(T9CM%5*+XQQBP->aaC*#xh=~PaAL3JYy;1}0RVoPG*RKiFvwI%+4bj-xZ;LuAA>}hkE|K4y>5TixmQk;)ma zj?R0E5T@LU=}lD4(=-(543hDS1B{cUNvf4HR#;$gr>{i^?-8A`>3_DphIES_EM?TN z1pO)x5$bjiaeZzh@=?_q*DO6pS~@LXAyCuEwyr6&I~kg<`T7`0H8cBT@F8#WkvTIA z$f2K9Rb#t7a$Z%P2&R}S=&txEr2F}-m{@ZZ5n}&XrbL!aFzUD?_<^)01fbZbjSl4F znGI=6MC`&BcHIrkfgp4WU+Xz}Ajc66QkVZ>ms3(JpAc2_*^R*Kb0HBzDh&98adO%6 zrE&+pbzE6I1Ae&~K+YDN?b8c2;yKP7>wc6#K{`dLrzZ?E5={W8gm7Zkdf> zY-6^&r92e1l9YCBIrMg6;8*O_D=~7f$IWBlR~2lFFnX`Ul$vBqf7!*LR=e}ayE1Li;R7}Hav|?Y^s;`_GW3GvLw3%Wq_(Gkpl2V(R|Ek-EPsY_ZSS1B zy}L&8DrNi4ljoe!>kP!Gv|07(m;(xcGS4={azfRfYHCyU2K6uu*%a z-5&z{t?g9*=nuBX08&m5ftt1dRonsc9shl2PedZ)0Fr@{8-F0-qvr370Z;1a#+(h3 z{ZMk9hIfYS89Q{SOgT_lI|*i~=L5gy;VB6bEIyBOi+qs!q#-RsvOj5l z3Z_WhAO{F%0TE4jMEBR(efP;580k#t;{tiW${T=O*mJxMQXMvVwMCRWLoV-Z(uqLN zl~x{E`onSn!y@ zPy3mLGpzgI`;lxhNQ?0S%&gBpE0LaS!uCJ%JT5O=lIP>EoToRvW`0k*_cVGX_g#pxdn4a0I}0 zYfc#FsA^v=TatkFvB_RPbZ#`7PN`aua(N#Q(!%??GtK7~QYSD6sBfmHiASSP2-lT? z~{3 zDUOd@ zok2S&42%235GAZ*P0(JW>c`Ygm0B*QA^u@9uU@sLQ}=g%7SHpIr)zGZ>HC$5nte!gYuM zU_(IJ_SaV`*P79^O7^M=%wIHLD|Nf z_~HpW!0QnsG_BA*3yWYEELP0+ z@FYR!VLh|B#I5E=4sV^UIMb0ywY!p$p$>F+oS-(lh9k{`&!s7A4z@!&6+zo?3dMFL zM6?p-F`&5$2hurhf6+5q9TCw{)d-sv7j%s$dLuW&f^-aeat{zCPiEVga+e+Q_}0={1=t( z>%PB%yE4!}%DT)o9p>O5SWv6Tx062a8ltntAL?6u8$6wkh+;e0J5*1pPc@5bj9S+m z7DP}6dnqF_NgAZV?ZxqJW}K5>VGmnR>Zf84nxT&FXv(D63=B;&(t6yp!0mHjkJv5` zJ4(`5648VULCThy{SpX3K)cxgBg(a9ls403J zD)_v&YGA-b<{F;=#q-G$%BUSCo6(7i@rveS<{DW7TgkYGWYq_0_aMe=Hlf}tj5#$1 zqv)&rBvI5OI_XXO#XQ>J!h*&8PLlM(03X8;GsR&~cJ69gG7duxPq=@+5 zgiE{og@YQIc($Ss%9?{dsN7l;!WS=TU=d=bSqPGUJESYrP7zuISy#ppyw3)9%S@ym zS*RcC-Z2^xbHF#C5+(rR2Lv?iCzK#qQV690Y@sY-sW?(eG+hbDq#Xcu-#x?(i3edH zJdD1iW7f_P1QUe^PG2+x6O{*KA04c|*p+p_GID3k4kCmDxd&UH4$PMH6=y&u%$EGs zrvP`<&JL)GT{#X#a0wWFW$-5FH*UKLb(|5}t+@qRYPjGaVf z)o{1~=>SNG2y~7DT{wW0BK+t0f=|BGL&8s;==WDfG}D2xQuWxrAU_D6%7&%GCh_9M(e6QLOoa}N=(n$Vw9`(Z!F6<}SGJe8vBf0u(J^40(%=1=A5;G$Xz(p52n%tNeb+d%9Vn3uhM$R&n)v6`F z6blf_Q-zCb?B@eIO)kRxh6rpVYSm)Prt5IC-7a(5)o)8V4>)5>QS(pOIhRdb=4(UL zjxRlE(Ibt2I5)zEA{DZ8^0`~iv2f@Yx$71xsXKL%&H1^h!bjPJ#@7d!6*UF%Vk|{G zlp4dHx4OKc!QO|ZNG3$g&5}%k*|{$$G&SSfX-8Uz$87_6%EViXMRt1~Nil`&-B~VU zCciRchF8u-5v}w1QlmAiHHHc!V!I!Omy48-&LyjV5T+b}uGV;T8|cohf;6tI1Z`7)Nz6 z?&*LDUq=K~sd10praz$E_-CJCk2RY3C~1d#X!pbKpMBKOVxSCr>S-2j z8!!s7!)vaw{NJaeUiEu7 zzq9U4w=3w&`cxMk{55tCjhyuhmgJ}Tj!QZKD^A?}Olr4+K1kX)HI_U=QnJjY208(+D+@RA5U2OC#X^|Rwok$>W-lCgAJLmyhwN*8?};1P5ah}D)0Vb+JYIKEyq zLQKV)I%9!vZ-mlV2Mrb@naezjm3u@oiQ*7$D)JR%Y?Jo~$M z?8D(~HIxb3{Wwm1trkn33iJ(nkv5w1h3_UNKe!Fmgd+!IEJ5$I#DpUYw}y*w=R*3m zMI%cR+)~%^&Rrlk+Ag+ydn9hZF|%5m+ey0`99iL~iX(>B%$%F^ym90Q(1N7T7!sNDf7w7R1xGPW6&n#pWO#9ler_MM9(VO4&7la zF|o_*{WPTJv5X-#9BV^#1LPVkLOqvueiu+#)orCtp0el?ik)H0GbSwOS}*ZtY(46u zpHGX(7WlpK&?q`6#yI62AFfdYNBe) z6ro&Ij=o326d*YHZ_USa>VF^EsqW=$FL#~nHIHdW5gbXqI zkpC^W&;?4<1ZzM9zcup5Ss))aT8pFg%Z^)M`*`3WgJzMJvQXTG+v-@E$e&Z&;#^sHDO>GfXYwA{%Awlz3As`{4Cd6p%G zpKX}kc_@_E4zu;xX8UOLkPHmR*0~$HK-yx-T6@v5OuTq5d7nV`gQM~9ojH5Oxvf9T zbj1~Zs)MpUEQ1EDcy~k=LhZ6;!4LD;$7)HX;8_3mTOCx-B0#{f4;fiKZ1=X<5i&$PHSxNgP(EVU|bQ&Uk_!W&~8k57;{N3P~z$> zy+xDLctk+BK38y3S)X=k1LGK7Z0ItdW7#j^=fdW_8Z!fgRBOE`^v|o*v1(Xh3F?95 za*WCqAxGUV*{<=jyD;!u&MxiTmDdi>g}YeOe1@!axo?ki1j}KC$jUnblfln^?Eih2 zJ9yPzq>>fD(bg}BaEx|EA+i!|BzkG@?3xpzruWqILe8EI_JZ@h6Cce%ujc7n-Q2bn z&Yo8^_r?Jr4c`kt=|;!MYI#7>4Q;_q0K$x|ZckmYU0&$+;9&UqXQ9~$lB~RaMQz6} zw|-LLbKphDew!d+9~Cq|Yo&5xux2@)OEk^^6(S*9+Qmx6PV=R2S?;;5eT8KOKjK9j ztJjqkJU_2Zv6PqZdr#9lYCX}Vsk|rJBn{G7+OcQrZQgf%1EIGYn|4z9jnJKz`I)$Lqsex2>D@y3!@P;5 z;?iAAQsUfMGbYaNq-E`tH+0+vvKfYrB`d+AQs6Egd4u3opVR=;aT#7c866PXG2 zS%Z>~_0#LH$tdPo^HFnaSPS{09M_yDn^H;xOLwI>J!ENKZ~KC7T=AG$g79;r9N7nP zz4ty1BPypF85a2U!BC6(IRzg;(iugGEx%u>BA)LuKoQ2>!O*lt@Ai}=qF6C*>@4`n+=4*lKvmGsXz8iRXl;g1k}s_-$_%*hSYVX!(cM<0ooQj zjkF=s#MhwS=zFVhY{DtIzr2qdw_cuoKOh{V{CiV+-J(E9K}lq>gK1DjX`WYzT_2DL zVdgY(^{l*x|8S;`?+Z!qY9fYiRaXas>1BmZj@y;FOeD(%O0& z>5LP@yni`Uhmk&S6r~VOK3-0_3L!zJ+FUN>jLj2kFF?-J-G4Y!eE;E0Rk8hxGo`Mn z^X)HZD)kR%itZ0*ioWX)XR04A|7Cf03qu)${*?ZT~}>`jl!I1B3{x zPk|)CW_3N6byZ}622%qKr7?*fM@MRQ5+~cRXB1*`hFQuW+J~h10y&j!(xn6F+R=eoN0I$D)%z#XP zVLrf4!LDJWXFI@A$F^tPFn&tf- zZPJRGiFrHxCcLhjXEmiXmR_oZ;$3|mz*5R(Z1^;R`=D4 zr72wZvwjXbiz%M^alDq+6V)S8IaQeg?TzosB@j*;s#zt}@eNxs`}s93@nRZDgFk_M z>rz5kh8$iGL39Kn*#gycMooVzKyu<{qeZyM;`)%BD^AOScK`Bw6A{VF6BIjv%) z&ZFhpUbJ?&*Mj%AwoxunEEDXn*IC} z!s%vNOxVMbL2vua648{eE{r*14d3v2)7`_}DYiWtR3qLiNe|BdU@WmPKRL4sRFdUw z`CDiMTVketzbQ8}rcV2c8l(EaTwscz%wR-W?-W2Jeo-OitbybcoZ&g%5>fHWaizAG z_}Uo;2KjpiSB`r5{Wn@c^PlmppU=n<9cBDtIvWEy>!Q2DJnlsCq+W(^$ey<&7@+-G zggETiOX3^({BJ(F^lBwGkgTOK4EBUi*K7uVfY19^acP2|!wWLHze8lT7+G6#ZV27U z1v^G*g-syVKU*=!evep|dWm3;ba*y98fNS2Xm03NP!1#?Xkeh~7x50Pp4%sU5zo{wo6d(sjjS&*Rjv*lEasW#CcuWH|H-#1vN1{D{d{Yx!2Dk;g#L8R{+nk1 zpFNC!Uk>R1|5j*}&ptWKpL_9`Nw&=aqM{9sP6K+B(&)%kOw_T{1&h)>f!S*p-=&M% zW!t5vhpr|26(m>{|E4}G46>LcQnIDl)*X$HxB487dvA3CpCNMmOdvr4+OW1X8VdH} zJ$V7`ut|o%5Mru<60~F^1$!cpppHrpV;*q?6FRbv5|bY?yt=(|ADYsq8elbSB-V#l zWtkFOrI;B3^1?F1n%3RgqXXGthDOT+_kzxHMsCwELH)yXQp1oqbBxje+YrTo;m6xL$>kKxf-y&T3vf9@C5Td zJK!Jn?S+JHs7&btEND-m5Ky*tUS|tD!sWRxXtg4!Vg%3@;hqYL|Ls( z!kEBU_pM#o(0D;)?`XKrAI)nq%pOXOquNbW=NuHe@_*X}KtaBs?wU>cl&bWoGwZjg zOOCUmp|d-bxBe?sFAXa%y1Cm$@8a|nUa`N-S1d{Tb446?^$^G;Qtt-P$k{@1A|j(P z`!edNHWIWRl1AL=<<@*$BHl?K!lYf_Gm1^l zq5BHvNdPV}?uZweGQV(1$so4rMyO;UR+6Co#@SQf!k)!zDvdJUGA3M*#)`NV>R%9c zSBx?Qj$_Hzf`EVibzdALb7}K@3R$NAMYQ?PB1Cek<0n@BQ_lMLcEkl{s(!*56_GyV z+Aoq!kXu73MFU~Sh|xNvJ4qxO1HcT+8UJc%x`x~OIKf@9JB{ru=kkBjn`uwlm`nUsIcc$W9%jT z)Pj(>(VnNNf2ePZ7YhM#4IsZZ&_L(O`bYSHJ;HHP=1Bi&_9Fg@K!?UE+MX>BL6F z(43kgbHy+-ic!UskD$5+W8z?7ze(|qH{y^^UEtW1sZe#Bb9cpHH-$^RUGdO*({UW| zWGH**!8-jrk=a&PM@^&b z%AdZQUu2IawUT6%g7J{nKgTOvIm9wH2@d74Iq68!_yhN;h2@8-KXbkd3&(9RGIJ+} z@ikrSFNGwt^EdCPmYv%RsZi2!GT617zf#Q3Y6`X>Bc4_u8mKmNm>9XQuJWY=eS~2O z8v&n9sOpu*nJts8)vM-0?rH<|;Rr)l7&DfIFsyuS*uN-S1RVbUx!=!W%e*cIJ~vA3 z)!aBUx0+E?j6DY^wD8T-fKwjJ8IutCVUfIXBOeq}R*~D9@F!(i-G0>QN$E$ZwQ%`$ z2bgM!=h8IrxksPcTSzF)CF`0fa^cI0r2KLV0@ccCDQ;K^T?I*kjyU7V6Y$rRzKAZe zS1@UhhWPi=VC@28SD;`xBHb*7%3r_gX0{GcTCeFkpKV~Vc@K-oO~&9Y+&h-5y6WqWDy1i~ zW$7{U4q3DG zQ}&mY22(`iTdFpjOR;#gp<>!k+5Rq=o+pb2sRUOjTkhbwyzJf03H7IZ6{`Vy=}$qx zac7UJ|*& z|78Dv-;@6q=06ttL^Vn4&(*PyEWVqFp?G$S-3DQ6DO?Vhb0o8`;F8*u7Q!;JbMuOY z1$ex*!OmgH!`SK&xzMWiLc85`D8Xfzr1QemY|^+mVKEx8AwFC+u%k+W%cN1=e@&sp zk)&Ia{{rtmF1)t>ZrbVpxtqoDBZvsa!f^NL9FkU&&6NK*I6B>UR}SE=xWhF%qG<6BaXHPDA8NF?&PSq(93d%H}dj2jhnSfgxP zqEi$00Jy&BvWKS>(uqz4WRmPEX7P#jFan>DWq%D+Oe$P)x@xv!VJS9qS)kqb9lRIl z;p~lI?W@%4j2;U^{r&tRASsaS!=p`tBUddn(MvBXOWnPv70%3>yJOKgDwci4Af!%m zrgI9wD{`(+XOBqz)g;NBk={4mBBzk|^HiDpLXwP6pSRGAN~~7#f+4YC1*xwI{yPvx zZPq0e>vObgD4F~2iR2N)nqMmKu+>sUak zI6P#&MV~RZ&feSWWao{L#>qhjM6&^{v%}#6^xa7(!8@y1%y$~!!LU8^{c{rZg!;H_ zZYQj4b_vE7>MxJ<7}gOOMx>o{RIPjA%zI3nf5Q8?$AZ?R1K&-!Z$-hx z=Ek4~i6NwOOP<)l-qdAr$C&hdE|>#-W2@K8j+$u9ET0W&h}@-g^O6>@3lD zIpuT1V#%ykJEfQNq)6*dOd4o4U;UrzWTc9IbUn~?4Y_V;Gh;+E6UfaT@5&J4 zYERPJ&+M9La+}0*Z>S|eei(QQYa*%Qj$6oXmshc5xqBfMgg;|Y92?RGooGN{v37Cq zzRnYkoE~*9#(j<^l%{-*_%%S-Uml$+8v#zw@x}p4ZMs9@B*oB7nHpaieOFmZbEgB} zHOjf8YnEX^-_FH6a}=gcrQ1y8;K=!%A!L*xbhLUps5&&s?l}1l2LZUFjo0Jmjw43MIuJ;4J-8&kb|USIA48l@5G#T1 zU{hpu`VH`#+z*P@yB^LNJPX}Gj@eddE)<=DXA^5~eKN;tHl^3U|Mr|kFifuS&x79v z^#9v)l0W5n&VQA89aZ!XySPKL_$5}|S+>l{7;4H2OqtuJA!cF4kV!3ahmZr zI6UWeH{Q9CJ5--9$N*%68cJ9t;Yp5XepeQ=b(?h^nweS_qiw`Z_k zwTHZ(zh~E?I+XwZip4*6?SucfJKE(PHT#nz$Z2crUK~YXuQn9lqOab)&BvB$L3o^15e?i`iq0vTS&d}*{=%kIic$GT2=StIiTurd#o*OtorU+6iR6! z#E#`XW=Fg&EqdzA@@rRdxoC^By6kgCm6m;c)v8BD@VPR6i=cYub7_^g`w#Vv=O|#6 z=M9^#O}Cp%BTV+WY$ysP)_PmjjMW~0x=jXC zb`Wr96rVO@$7Fz(MqTcdiO{0rlIy@9y9Ha&xR;$bDwG&xGjSOq)`)ZFDD4c4`NHI9 zn)DBk;^2phER4nkM1xAPvCOE063etJ*oWu#3R&DLdoeS(~n}Gn_%(g z`KI$Sf$tsEwr_^9EyxoUzLoumqQoa?G%i={altH68oFtmNK(vJsupTu3u!>APj%fw zB#e&h6Z41=1WwLlFG{6l8cnH!py95NTh^MrR}_mwWsj3iA@Ni|Lc%Hz(jPW`(UPR6 zvPBIYAgM0q?P^fZt8BP`3Fye?s~0>yW^ihPyxj&D^%{;77K;8(m6F|DFH_%ENX7ga z$d|v^g5Xh1Zd0nu-EZlb`vXWTPE4&mmtmGE-ejBKPm^}rE@VK>_N%r88TzHNO@s5* zkCxvYQmJcMD$8HU!vXa*q9n1R9i=v;qhNiO;z+P5L<>D;!XnP2V{5%L&)iK69D7qS zjktJ1%|pTWhAweDcvShuIpY|;DCJzxVm_)=JW;ufr&x&ubFHcrgbpmHRi;-198qCv z>tWd<&+i9RWT0C0-J=;_;gUhcyY!&^`MXfT$N3;KCyzzNe9Ite60B0{-}*|E2L!SdGs=TSpP5xu7aeu%d@qpsg-7ut@Nat^wLw9|B(lN@k_#xV#lYjOIP-nczG?NX_^;<`aa_DyL?ebSi>j{og(vAB4E zncVAD#?8T`OdX4QYdH|x7}U;SO&b5}UJ{2jUbHK#;Cl3}B9n2%mjwk8x0E@0FEQZq zovx(~?db2>Px(7#x2^ZXU#JIbMzu8-*I(7@M+gx_g-hvR7t(>}!DU?<-6|;-KCPY1c#UG; zV{ZI<0VD*7gd}4Xi`X|$PdxDsT7g?3m%MT8y&_L$z?NKvp{bLE5oIPS3|Z`4^V!o7 z!ZpIRBnhm7N+#f46CXYx4W`(ve{JprE@FSI9bP|&=J?|K69fuVnlm|voPKe3;D;P@ zlp($7>1pNS3O>M-cY{QCUp-z{`nFGzdgvXgAxf=|{jP>Wtb8#5t@alb0|ge`H>mNS zVX3u&$Ouk$5ml?cTH>AfkFGyr`0`E5GR5$VOiW#?vIv1M5@Y8WiU{dqet2WDhnGS) z!Xug3;YWdSsE%f9lw6_nLFbt&DY{6Uu<$Peq^YE%{rD5ou_L&nvaXw>A&rB>2P?WR zA`>)@;iaL2c}<#pwf!Y)A#-`MZszAHJOpylvyZKNV#F5Y>#+z=2rT9WeWgCN!w4+4 zvpv`kgt$PwLKrI)Q&M(Qs9`!)7#2BC+<3A6b!>JLQ`SUhPj2rXdifj@{mZi(FWKy5 zSEokwwn?Ylvxe#>IWuPbV)q&~CJNMT;RZr6PG{bDbZIGX15@eoym!Y^dM z4bU(!<)wlSAT!?pQH(5RAH9ajo+Rvg}|bcX{`p&%e)Fh zQMD1r*H#rJ0Qm$7EcZ?zZ>1C{DxJ1**O5?lX<8As(AbmIqUaGS^v3~-HQRZ_>->&F z9Ro*hZ@>tcS@Ra7mzS7#f@P_7WoqW=BC1_=b9>(!Y6|5Uz_Y!x6KW|-<$PUq`UL;R z1q+Cu`Qh!M-rn3vF?~{qN9;kbU#`s5^`s~%H9&hcuVCd*H%uO6rr%)$Yz5aBCR2(>&;f&a;2d1JK>2-=!nlQI4%X|FJCS`A4*uNo-Hp~&fV zp+LRn>9v%ui*u(lxlP(NVz=YMR+Owfv;zvahu;+j-BtJuW6zajbhpU<%T`p^=Ww^}1EbC(PKtKQLf$gxsZE@!C6 z?}U%fYt#CR0@F0{rFKrN<5eetq1j_qa-1(n^hJtKB+H+L-2<5>&_^Im;w~;R`cP`u z(vf!VREKWAk}R=!I>9b58v66=y(teo=>T_DmPIEVaSdE6WHOy&X~7XET>C_S4g0*b zTXX&hasA7;zeng^+hndWwyyAqn~0US7_`=%h>1%^re~TeQARs-(^~QdJ=fPopw)bI z3IupWJKK`qw-9Wrif0232cho+_wn(p85!Fz)H8RO$aZj-;}<^UU+{rXIR-6-C#-7w z>0QCk;g0Van*quO`h5eV;mrqEC1(IAbXc&m9m6&nAI77Xu!^Ip?Oxb2^T^&WZChQs z(_QNGa{IWW6wSMpCh<2~ZK)>1Ru$U38_O?~i)o>$g|jhB2Ot>%ja8b;dl0ca#2SMPBBqs-#py8h$$S6(_0JtjH!KG4xQF{e?y141I83xp0u$bk?*K+W@vm>o z2}~%4D0`DKfhj!=EK|bchB7&db`5VpsX=BXNxXO|7sEVy>^dz`S&K~*?#l-2S#9)^ zjlst{mCGTtgYTMyQV)Y!-RZ9zZC4}b8XhuDS6xj7Qm9^ar@ru&r5d=IUFanh=)sOG zCo$c}MJLnhr`B^vORaU?656=0DG!e$mz-q&+5xAVs$qX2EUVJqruiJXxW?arHH}?s zn!|XuZKKa+=OqF5|Q~WvRf&OnD8(ALif9v`q1_|+#Wv%-UKaFoMZO=Ac?a@SD z4CVBf^pqr`npn+(FvfjLlro^Qo$(UmPkF&NqFlo1=&(zAoLGp7?)7M{r}eQ`!fd;% zFORhowLP^XwF8Rm2HkZr{uZ&~@)$*tVAi-~70fDDE^d^(TTUX!Z%QK6fDA!A4?cbF z3yl|_QbZtDgw`Jr(H2SMKNKi;XPFm&p-P;Mlkpa9k z(O%Bv8hqnM1ntypc30qz>HMQKxg4a()72iIwrKS;=2PiD5V6#PKZzz=PM}g!Ax_8R z_h*74EbM_@tmTAMQ3CreAo%DS@ecPAf%JoZg5&u2wQO^I6WI75R_h=QhS=|IHqWJD z)0%h4eZgMk~3mHab!VCYqqGLFwuDF<`wH>zICr$ zZ7SJ9TN(mwA|jFOJTR%C{``8Z{l&9LJUd8l^h8hO?_cdCywMCZAl$zoOB1ZI9A(rF z-TfK*Cs(Q#dE3_S;0w)HHSh|^7Ik5CMQ9-s5l9xIqUn`~dgx(fTeVv|`HwXg_> zgT+H9q4ih7mV7ANslK8o<;n<0M9#k-?#dnszY_veqnK}NhCOEqJWz=48!mEb4Q%rG zQQJHWls$5I+o@lD{v_4BAK3&4Ey_EYJg-D=6#(n5RH5%kcB&er%pYtMydJ8p#nm#T zc&>M%xDAF!1L2S_<$v_oGPZiCrYs-idk(h$cCj(Y7&X4F;jmXLxXfV`j>NA@L6vf1 z0@k(=mc6e9&f4GFIYlQZ_Mu=bJ7vnl+rkJOkFTwmwft&zC>fyfFnh!6`8E%uuh=Sb z(J39g2c^vbuVuK_znjuH5h~Te-rSY@0dv0#JEy;@jS0P~4n6hJ(>YO$V*3H{PiYZq zyp75}OWXI~rTxs*AWHtHwEx-miA)&#RF%*Me@jaj(~15NIUqw5qlLtdO@`6XC{%{wvMoEu*>GQgKR7y4Pb_*8zF5cyg;pu z$IAoQpwhYJ_TNKn!B?=w;~cWJYZ47^EtiD=`n zy}KDxTa(ugf$cght*X;%?9HLsrJ^R>%F22siyg^RZ?n>=miDUWfd@SySX0k<0q=oY#)xFbh zNfe1Ph4R0W;FXz2XGbp{*6|Zht5>t~!$pO|7VeZrfa^x)_u?+z-c~~EX@(Lf=*2NR zcrH_gL-F~juglI^>zFS_CkrU@0W7_YKY1V{lUqI{%m**C4qHK`bjxa+lLT2f{v>~h>e3Pr^ zM3Kmk2fDsmEJ)?QQ;CNV%OC|Of|R<{^)NHkN7;2y7GiuK#5o#MRF-j^l=U&OUs(lYT4HS$J zaArXpX;Xc{mFnIT+rrP-rTM-gBFT7m@oH8ZPY&5NR@tfhE^Y35t&t{HZojLs;;T2er;ERb z6XWR|c3B0*A5YuDU#3GiFndCYb#j2)+>p)GEy_KDkZx2h+C6-*db!&%0bb}GN&r9aSKp+SDDs zPZ6}l+!3|&0BC0J@ZX69*crOR^{ruTPF~A_+?YDjclZETChmlNZ`ig*&tLk2u`Yi; zGxmvN@lRe$f{XxeOzRV$gSa_BcB2sFxzRQAGnTDc%McW#{V-GzqvWqFR^h@D%*r&c`RrfD< zs+>V#VRze-psRmIWQY));sXOMF}ZODv0>5&^%j0HHhZdG@n7VqT`jzmJb?GTIhz^VINnW44DS)< z99k@)>es9TGfIly6K+v^RuRrC_zU{_bDVG;{?8~?BuxJ;l+YZ*1 za~S}1-DMsa6bEnc_)s@qkwT;?No(POp+s6>)#ZFRsF|KeP%UjX#4sWfUMecs zu-4rPopee7j4EMnlOR%^xHvhA1s^@)%4-MwA+-$K8!b~FGJse6{DhN2h)p3+&~hRv zHeX57{;l#<&D9*myIxeXW^<3OE9~SPJH6Sb%qj0_&~bvaxqqy0K`aB_mhUV42h_+b z=H+S_&l|YEQ4!xOEH^hHV{bUQMvvrVfgChZDaTw6oC#Sz5UMx6Pzp^>h*>H+g4Qfw z+bfh=!@Rblk7kJsXg@gY6iXrFz~#!}XyogJ)lJ)ie!W!arbySt#cU=)vll%uVKnKJ zaCaKD8lAkM8ybG8@W~wCJIKQE_TnNe+8Yvy+mTJBy3Mm5pOp+tIoszS@mdUW@07(d zOhcRlnnX-!@*J}Aok1{#UJwtC>R_svVAiZj)VH2X<%O#qEG}uFX0|#O`nNfqW$^Lr zn^(51IyL-|gISv@bxNAnpo2~a!1e$yuz8H7$RDshN5?#@iU^Jp>l+UJeKUhLl z_b$T;f>18{)8(SfRS}Yrg{s-!5o0^|HdBJ*msH=%0jR!KJWNn(d|#5>sww=qhCP})87$=7?$y&)w(L!_ep5S4*7WqrCDNM`ScDs>wqjKTf1;# zgDDmF@Yb6P?E`Y7=Kv95xy&X5C3_I?&~ZS=Um*wb*Ctb-;hyCamzU@H&Uc|$Zu5a}#Pc(?#-KoIMxK89CVp{v)Py_2|fxfN$?r>>PtV{3YP z#Aa>`0Rq%JeVpn(F{V4EeB6^x?w3gYaZE3(@c89P49E7#(ZH8OMu(R{n|U-~f&O`s zKzHW`!S8T@+Z^o4+%ke&_u1*f2J$5mw1pnSZ}^z(jC^R82@_Kg#Ef|3wYjw+U4ENb zUWIq)g8BR$Dtkt_!YPcrW<0L=Fva|u(V`)AeY|imgp$33&NUf*&2;&jIB|ib!};Lz zDvL<|(c5R5-`gIl$-`MN`NBQ$+j|i)zW=mDCkCn0z2{Ux&E zZ%sEVMLT2h?q*HIQx%lwOd}tXf>4}-mM6T+$nqc2@4k#kaNC+a%S#Y^F!^Bdm?gfMHk=~^2$kySYD#>mvqBlW7Wy&MD;&XbNV&{=T=)ipuOzs= zdE>!Wf#|6)uI*woKkX6{WFL@w-<^x81IbpT_GuGRIFgQ3h)62NV@8VpGS+iS-zLmU zNmWBntWbgoHxz&Kp)jHXEsPFxxyYpNI4;D$S$rrlyQ=%KSh`e|;^(qo5G={2+ToHh z8L`yycnHT6lQa?>Z7+$t&-9}T`KBy12U6KmY6eD%Bxt-gz_~}li4Tkqmr$%H#zZmN zdH(9$C_S;_25D1iitK8ONs@NaHL$X_FE+59&aJLbTufN8t_sZqC4QD2*Gc@m)iJUo zu(g>@)5;jRuz{a|$FE8-$Y1kbarFwwOCI7=5@LY{zS<0_TCxLp!VEqY-v_jN~Xc!q)ldv z%FE!uLmIBN{f*oWldC4ADw)&Lk_V2P?$1HeWN({Xp$^jqP*X$CJ+C9DBL){hQav6l zw?}1aTI_g!Uf+v!X3&e57q{Bd*v<6yjMN5de`JeZvlXU`cw!AIv>j28u{Au>4_pC{ z4UQs5o!BmT3hvcbdI9rB=~gI0%RuQ)T9KI+8nJNL8><3^4VFPZw@w$5&7vfkl(mvL zKgH7NKI^Lsu*4KENech5&&V_wQsiinH@N?f9AnW?S$Dtzhtf;p;}rAH0?CN(*D0yH zb}bZb5|C^?>Ec(0LOHt&2v$kSJy3s!Y+}1|TswSL{QL%W=!Ha!b=h(_*{S6U99hA!^+s_3a!=xTb( z#(3;9hP3yfBJ(jxjg;|qL1YI8-uM`6CbXt3|Jhue zAx>K9LqycSq)BP-jYalM51?@931cqU`TYTo6Oc1f# zPF8SjJfwChj+76Pz&~v5CxU9My2FI9I6sPsfI5hr%wuz|S~ z)F{Y&Q$0%9Wp-SVoU#Z>2jpaCS3vp-G$%GTB+C%?;W9cx^)~#SM47ZCGc)HlLNF%1C;#bhkZ%);?t5a9R}561c>z_Wti! z&SG0x0^^fAXhZ*Bv!TyW1BT?XSR`a%XY!NVoEqBD@6qlaWHNYcfjR9JxN`c!wIxr*Tem8wVw>y(&fDBC8U4nA|Nw&$^XJ;i$=T+6hpS$ zus7fvW1*dDhw)gAywR9c8mbE@%qsLRj4cc`AseGkTA-D$tuW9Ws`ED?k!F#G8zWCL z(_Gb9)#TRT)@-V=Ht~om3@>a)8(7u}bI>R^#N}70CG4dAL))%`*G)lI}YgTA$A zgdV(I!Wq#?9kuO(F3wc2i#5duoNPi}&fg^sp!SuJlo?=HuG&R&D!w5f3#KJ3<^1)z z{y*SF{n5)oO}{OQj(GzLlMX3w8TYUHnH^T{u0G(>QMgOKspbH8Rnih!62`M!xl7+f zWA7z2m_uQn-c*<~-Cm&DgLW0yl)0L7$iY@VwYAc&jL2Osme!t8V(<_P^x#i55}cTS z-|r`#ub~5$qt6E4DCTt-vAFjK8%zKQ>W*+ZS=66Vhqz=d&MB%~`dTK1G1M>!@6jX5Ly$`;~Gy zp_8u`#i)xT5k-q3#!M#zWS20ePETq{94~ZDTqaH294A0tGcAKqI-UsNBMiqf_IDFm zWGe@iuX1%C_0!^J#?lCgrqsx?&ndliYG#B=R!{JgWV>c0FF=|sB)|{Pc+k{OPB68y zukFv^xVbD!a;dkt$jlwQ*V7j-V@7iteY4|kZ5G}vKD*l~K+sMGF7=Uu#M+35v4UyoNaDjG2QmH$MRmFQHpr1CnG3@~^D9r2MP-BYAF<1jZTt0QtY z{)|2o4?`?2RKyit-!V23z!iyj(4<=BwCzaZ^NVD@uF{#=vaDs}=Bsk?SSHu0jbcwk z1QYd-^xAOXscx)SDB825FhR6fsfQ(+O7vR)xR{eb)D8JY5GFjy{#pl&*C7 zh|UBRhjG>bXIe+r$(aMdQ#YxZd*)fvHzC;vwpZaM`y8U_0tb1{$6ssZ6w~c_YUP!1 zQ4LEBh_rTq1;`S}5#QfhdwA&#VYBwuZTLzvCn?@Q47`t2ow8lQiHNg%c0D16HD{?U zFkk-yj{BgLe!E!Xm$*q&vLS!4El4Fu{n#jJEl^1Q(;O>Da+EdQBN^jWhu?=gVP>Kj z94y%-vyg=?K@j~m)7Cizb>Vfev1?nuUO<-VQGN`+-x!CN99TH%Gg<{;NEK#0c{!h% zN7@i5-<`DFiV6A12}SA}rP9#H%p+_V{hhd$rO+EOk6|JyKZO5BXtm|!Iz*8K_SSH{ zmF*r|u2mV?J%T5Mb7`PnKOER;tdzZ)^;!^08@!mH5T;ev(vyfwlhSV;(9tJg0l@=I~2f{ftQ12Icjvh>BkUC_8l#qPixb@&^ zA&koKy<90QoUz(zog)mN-_)Er%OX}69r)pdMI9o%< zz02Hq>4I)e>g`JUX^Dqz?4aG>1eRF{$s1~I?mEd?7srRsW{L3xx}#ZPURm^}Y9~}@ zKKGoPyg|LtSN{B@1=j5$NSpFJwL&M@lVJ}G?R==%;wtfkyUk}w@w7az6tQh86AwG@ z*lYNf?(f6R|D^ijlJT%GLw)&jhyGt}=3iGT5+)4v|4#7|{t1H4XEL(1I@>5y zEw8miw$g2>ZMjxLUJX;(7~&lMK~E-&`KqFa0SWmJ%Adj}PzZewFg~8@{Wl}EFFSf_ z$?c111q&@REfXz^@uv|_4ci&S2Fid!mxS+=7|8j_O5QiaS;UAu#5>!=D4w#lT)_(b zob|#MS!u4{2c4nGwJNjWyaley>j&91d+!bI28Efw>e|~I-K8<=Fw^Zo|MkmB>J4(- z-*gnJBgWISTH#o42Q&nVlx$v-ey#mQOKH{{fwwSeOYTB6QcDo$OEiZC{SnC=GY%?9 zvLnd?3Z9oDQh4AMm_Pm-XOH?JGDQBjboVd0|DWO|_o2g3eN9NdK!^FRzz=kL znlsz@7sZcd^u30nhLFTh;s-RUnn4(ZVi>3^J}|BnXtZv@Kc ze?a-PhiWRLsiS?QX?i9>Av;*;*khxEu*DTnSxT5-QCz-6mSESXn6?NR$v<(z1*}Ad znldq^rI^bcOq8m;W~^;c_xdHMo&-C!PrY1fUbXtz z{`LAp^QG}wr=Nyc5B(-BWJ$DxY&RvOS+oO4va5@H70OA!>w`=f!b!9%j7%Y#fpQZY z5-pm6aT6LM-)V0YiK)jsIpWDsWKfBD;2s!H!DaDa$i4WY5V*k4S=;jslihK4``5SV z7jy0TV5;TwK9!35+=#u=>HZN6IW6x36Lg9P-{+a*;6Mt_m3?uhv_F(RY64O>C4?+M zr1%0<`f@MnM;5D5XRN#<}%rrQ(n-Bw6Q1*E@aU$E6@5F6yeI8S-2ol zKxKos&so>n=vP$RCA-Kll@?fZYUzsqFpTpy&^mMcb6%85)ojtgYOk%-!}?-O<{Lx; zI=99^lNTLm670Uy9p%zd=cgkTP@fUGLUf^py0*2^z^sDldv|l+a3QL7{OAV5s`i*< z=9lC+DGMJopWf;Eu(G+#9n)`)VPXu0i*R<5G~}a!19>{@JDl9uie{L~+`}1Z7BWS3 z1$J8gNf?ZF##ffB?7|D7Tg&M`O|xT1W+r?H&#ET=M94flyH0m04u8#j0LJy_>q!6ti z9JWX&OA}Chk^$?*jj$cM0pgB3JD{qLODN-D^46e2s_<`j%;LXoqtgAHAzl4tnesM`I>3FbbBa^;RB;&3 zs_)L1=EqC{Sv_*NTH;Zh=9SRgi-VR>$jB1MX->ICgbU(fff~NgSBh`ZbG~VU=YPvg1CED{3eVr6qF5*s zGdZ?nbeC!vqopR%tbs+{sc^T~BMt3@cT?TMrvaFuDjY|_Q_+rlaE^PTymsvl0RacL z_aBJ1GS~qWle*a5#B=F_^S9QoHdAuCgX1#;&v6}Z0Q@Gm2iBO|S9PdTl(Deq&F|=W zP<>iohe2@o-O*;tLtK$n$~{ z?cE6jn*Br~nq7rqKntnJ4QRriM#!`wgCuZ5gD;m+)DxrL6Zr4^l7_)U5WUkqaTeg+^p@23o4mM!g8wfSXOwQ z8=jwNH(c4!j z-vZ@l)WSEj(PhGxD1Cd>-@EL-efrOtx8lmq{!nRrZKl8^+acv;E8!zW7Kshzj;;5zx^!fwP6UOH1o?(e zHrRKU(;wjfZc#jUo5Oz|VFdn*(*MsUKTYyy2l_o*b@#lYCF>PLrL?T3Us=qdUjTy@6dLJhw$ zKFa=(Z}d8+JwiI;_Aq_8e3V0qh}74#h|~_L-@+Y2b{P98h5d+ze=-JQMKMM&1!7kj zhcHI4h+vareY$M0Q2>%4v_1z=IS`8+yXw;y(07MbW#Z1?=K@$_S?|6E17R|EgzS(3 z7);!;`)IKAMy{zr>MR{eJ1&4~Gk5U5^ihSd2+gci)hF4%`5%&Tv~*%!;;p9wY4};<}RX4^4t|LgbN^if-;QNP|029A&4qM zfju}ute)CpE;}1k3nIP}2^8lwwfdu^(cCI;;IMJ{LG!TZ=_C>jB(NaB>zsuqZ8mX3 zB-`6LMn0{|xfJ;keLV3Cb;O=lN|)w>zK%p_c7CH{D1=~|;x+;Sn|DmhqJaU2l{|7g z#vuZ$TvTi}qYtliU2P^FiWa#_VB|#ptHuoRc$r3@g+1$!eYoSB`6^R_w`xEccjL(us1fR zN3Uu`T-uIQbd3mA98Jf?2hVe1UZS?SmcCh?bTO>I;@NssIF7h!4qJaU+fp)Nv0dc& z%E*m25o$cvr+2vH;xs0qkjzV2QMpR>@mf>cK=o@aql$Dt@Z3e?kDO<}B>TLH<_59$ z2IYpw%u;gNuhFK1*0L=O{<=$PfA{v z`_Z_(;NfRbYs#XFZ^^q=uo)ijmT+ILCzotpbx_nD zO?HltyQC<+I^)@`<0f9WG8S>A+d5la$DU^R> z$@AdD90GM8#^uE);l#?0oMz)a5;+xE4E1Kr4kqmP_<0!=&5{o~o#ryHif~ht%RKZe z+B0R6pA?L?5V)35IE2Q{FQbk_`@ee92S@Y+C{i1YH03W@$g$Z6DTqC1cc{7~|3A9U zF+B1H%JQ9b(6Q~LVs&iWwrzKk3M=l8ZQHhO+qT(Bx}(ke&d%&UGrOPasn1pQuY2w} z_jhJ8y_OFfFWo@{KOW3^0NITeo%;kEXp<3IlvHC|NqF~8@7=}rGMgJu-KJi|s8J#D zJm_J~Iz&}hnlp-2*GLkP{tM3~C`3np&@nZ6Y6mmKbEW56dYFxgG1)p^M<$PtCko?3 zuf7JhBRxI=F<`9_A*U7TFp#i+#LO;0CO3a?ph-CEHS`1>!ixPJ5IwL?o44)U_wV9& z2oRM4_iQ-Ul#WK`i}@3d;y#=w#fL9Pl>ljsrhzoPM)g?SaAr7t6;`lCPN{NoOPl4E*_%ETaU#W7Mw7vrFMGi&EKdPCt#MT%-o*~dW%H$m_hp?B+A|kn zroA)Z2&BOfC@i#$=~vQP%icNX*OoLAYiy+C_}{hkhBFr>YEV8n^)s{W+FAYnxuz3P zx%virpYof&d9hp{pBjO>Q~BFWa@R-*V-PP-dpRM1OhuP{dn@jc-hspuY{t@wP(VUs zbefyCiiinW9S=}1$?S}vt;o(4EjuHq>EOOhNv7$ifIa|YiCR7oz`bMabu-_O5Nl0F zwP!So)MD-gtLj1bf!W8tu6MeK&~+#8TTTJBJBS}Zer{;u7i1u5jM;go`wlcYXYHo* zY91~S9V7%f+PEd&5+X)r*~(?jY1DN_mUqwRO1V{6eDVqCD?_5KMO>NLee{0+6@RPr zw9z4M@AfHou(7C_l)w>rbHhcaG`Y~Au^CNC%^+V$1L=->r)eF@%R!uW2?pn%v=D-% zX2WDef4{w{XR4G?9hzclN#6+uZQ{_6%qm^@*Z z)Dj5HHoPJX)%Fw6TF^$HMcOHKu(Z{_O6&6s-Tv9E^_rM;)<;8rxi-jtD~pFH$*l(@ zwy>ftt`S<>WdnxiEHNsTA0XH#u(x1eH?@@0HC92_$}%@?Z*5zBC6ovi)e8k7$v{05 zBDc`qpoFf|;IvB~u}f_<0?T)*-9|UwfQoA&grV&->g_{i-v)o9Gb5ljLlo25Bj`SS zO<3Tl(B=!4iz8dRfw6MhoZCa()i-i_i$f7}`(9DQT-ifjs!P69$S!48ei_)IhSI#J z;wDB_KveY&AyTCzF@b&8g!B(cH?+eR>P|hh13OwOYLnE-=EkiTjsVp)8{4p*+&WNl zlR^nk>}!+E0!c0Vlhgu=x1@8g-#}%fLMBF%HkYCHz`Rwm;Gi5Qz5Oz`)%zh!j1It= z`kjj2wm!j-JY`I+!y48V)#D2&3H?c09Dbjn7wz*@4azy7Nk7s=%B!0zekKP2LeOu*$vyi!)*7H5!F>%ft_?+r5`GcZGXhVX@GEU2tzu zl<-rPUk$cH+}+5Up89fq^+4L%P3S;#7OGl@eE76JK#Jr>=-i zIeg?HV~oF{eyL^7jG?UM{dC%2e+J@!J{bQQr%ADR7GOU?bpY!Bu@U?3Cn@mLUzZVrv@O-)x#40;t_ z7=e7nL^(ud#VjO#aU`KYc?HA|fD6d|Y9n>@unY9=^UUEV{c6@f&HifKU(Nn%-ha)m zTfZBPUyx4JLGnOQchyvrn>4!cF?z3?)U6r(6RK_!KO>MwTi*QdxT_P|*& zu#C!D2J(}c`tJD0p=qjWl4*e=gK9@rdwLkltEUL~icV-RY8!it_WL#%?l-kz?VJ^R zgyL$vJwg~L_{vOCeqlecRSn&Nkw!mZ#F_kJcpK7tz~WGpOkC6J?JoJYHp!J5VgN7sk>Kbp-Zojo7IiV84c8;qSbT zx5x*p&b^LSTrjhm6{L98lHKZh?Q>C{-wwm zC9~3P>Q_~r6Emfr$ZKW_LOqM~7A72;rx^=I*+yzlD)vLt0E@C!Q6h|nZGg{6l&7sVQ)_;V5lO^i&4T4X`fW@`jk;rsteHZ<+tdd{V9uA*rfeI5ARY*Id-?fR`opPGu*OG1N*o~n z)DtZtc9|EsF*dlm*=;V)p5>G~HRB+9KClO3jn8&$h1yY%{7wl_ zy|+;Z6^k1-O+R??{KyuInTEIa6^Kszp>+d4Qw&DmAVQQYl#XJZdDA%Zjc+bO?k!Eo z{RNW6MK*c1x8$SOCmwJ7n<$?d^n~EV;+>XC8L`9TkNHT>)$2g}oze5$oH}dXmb`BP_iX$ROg?Jb2lz42LxF_CvQ14)_SexnE}!x6C@`z4^#a%~;>j>= zP~U)F5avE?p0A&*1_W&t+=+sUr9QlzOmd6PNw^cO)hR2XOXV-a$c9&`R6V*{j@{7O zV98=ELzl{t=p4xaMk9xe{kd;<(4Dl=XE0nb!?hudouc+<1(reVIdZe|*Mv@oG z)LUpXCKoiOGjJnriAPwXHHu+Cy1^nL28zyo`4$M{h#0-9b!({dEH(0YagRws8)Wo(5LE-5AdTi|=Sx+!G5Tdb% zI0PKvy-&3qV^DWp?Nl+U<2u-0bq>eS(-6(2bL4L7{K&qQi6W0K8qv|^RY!Ej z+`4-#U&w}20!@m(Kr}CSe$6iN(LVB>Q_7qp1Ua@|P5ja(%c2|_)2i-1mDfV=0TeoZ zKP6KOXOcwwIhaG5diE8+*AIX!wHgk^c=PN0aPohgg{@MuI(~dcz^ng129jg^XAyJ{ z;By5Nzbe13_{lT{iw_j}hOGS9L>EF=mKIW|rZym^BPdDzjl6Q8kO}$+fH8Kz!8*Gv zAzYI9_6x7-Vyy|LlA@kY+P2HXq_g;b`Sx-0moh4t3782ux^LLv;ovghkp6jy2qN6d z*z)<%tDnt@5Op{aB-y?2MZzp{Koz=0?IbHuI>vbY=k>yv3oYebVpko_x}M$hk;ZBB zgoqA%!F6x3A9^W%F+q@FVUf0w9uzz-lK=9`OH*BR!L+7B?L+WPxBcpvS}zny zBO%C=sDn%jyNp`nswq?Cl%r3x&rtWkEFr5VC5I$XhMsblJ=6T=O+Mzi;px`JZk`xa zaIU}Je2@g|brMe^F(PhV(N$HYMe7iq>U`Mz>=vZkY?wK0wRR@a{Du{c=zH9TSJi2z z?SJeN{0GD#rzz>kBFqJ5mQr<$tJYWkL4cW?MhsX=N5McHxVR&|*|OwPYXjiW+|yC|2U+H}lgV`WD7?~43^zJIagfn#fdqyKs~rbcrD1E0Rr z&Hu-D`u8)ZPe4XX9;1Lk1YvSBefRs%s_3FGBK~}Agmhdm)|t7{YD}u&q0l?I$QHB{ z-?aEY+@DUHue!Rn1;8>)sA_0yu>QL}wIBcgw5PT|?J3}Y*i(#`va9ojp9IvFkWn+st-oR5=Oj8(Sqh2x$5oPdD~HEd#NyIF!Q=!F8XtPA`RU(1 zZn(Ux}lU5>($toE}*d_Q3uj4JRsm_VFzrHVD6bGZC5D3z_!?>G%B>b=~&LK$@ z#vg@umVAjJ|CL1h1YvQwj7%Qnem7$@egvZM=OY}sU)Y!3@%rABZ-0t zkuUiXf7Sp91F&Ma=bo4^@fKH=mCBxtC!QslZX3fhz%O0~+q`&l1&eK4LXubmwkUKZ^v_+M=X|<#Z0K8 z#uM88m+FThDV1gT`Q=Re@0V9#!pSq)FtkaD$oQXyhISeY;^@BSDB$fvF=2~)aF%}L z;S4frVVaG^?K4$*kcC|j*Hm8_Um9~Vh(_r34hLLY5IUyO8wnXQJ5j|gmQK&A|?$5D2G3lt{+Nh1ml!)AVj%-L;5=IU8ky;-NuS#qfNUp4D>8haXC3ri@{FkZHTp4ryGA(lURZn5GL4YzLV(~=l5?`| z|NN|aaf%bggwDsCk=TlTbQLB8EQ%|U)}*!5kXyIa&3ED3S1lCOSZz48>+T}mBsZUP z0KEQ!^B4BN1xPqHbTiRw~~0K+h7nj;++yyXo-VR z6`uoW3^N06>Ergtg@v0msq2V+Ge1Oh>e`D8aMLRho67Oy@+A(C=3b z+Td6lz;ZIRCK0}~VxC59IBf+{@PE+v$vHitI9rHZ`Mp!Nxm3?)_f?-1N@>`m6(X{LVrD3SUPZo&p$|)KPn}HQOhd!VMn+8^Y!O;-hMA;6r0m4fwsGs%1M_@ z9$M~6De-_{c~|JxuYJ+Zpz_)1jXsOp4V%%Hpaq-Y4IXrh&xS)C&b)3TG0H!_A;OM- z*`Oe{a_=$PEVT*`pXGr6h6n!rBeF|Gh6P2cO}fmhBY*JeG~tT{dCv~P z1SP>bn&N=LkP`*@0{n|ss55_X$Lp_Q9WMSFi#taOaXkG|^z^uZBYz2F;QXbC1b(97 zZ~*>8_9Jl4C(duDM1-MT;e7k*-R-IKcRh&c4Q2fK{@4lp?vFmZucsf_0C$W zd7nyD_x33~x!D>Ui%YRvdIzt{av?jVgU~zUa!*M)k#YHKf|tdGOjJZ{cBu*8 ze+#^rh_|`l{_^_}kW5NiObFy`7Jlvbh+Ca`W)V8wr4;mP%k4&PfM*HGDzdejBQpni zcIkTi^{$WLZ{_vRC{U|k@qQGKzi8YQv7vcI4}~h9Rx)fj1m&Ay?EB`>Y0atCw7?Z^heI>B; zxbyvzNlL@Kr&xIZ`ozHO{tKRdE+qmaR?a2nxl+KqO(Af9Wpc1Iv4^;IX1|2#K?pki zL*`*Fd7qcjJEmQTKATlXx|D2 zqVOMb3LApwgsD4a7*B+(as_qvVlYoO=RWP+7kht9&)F`m${;en(XiaN2cM6Mt=Id< z$@HhKi4otMTd(V{GYo%iZqjjItDrz-z1bL`T16DsFwKHrY+*Ol#+9T?)W)5ptYWr` z1L9(~%A)b4?6iH=zs$uNsPbG1F?Uz>Rd)~dRrgc%sj#2&5MRi3PsI2rIxI(e?mIOj zEIeu-HJJz`#hsSiwku z0QiZHvVBf4$uhvGpkrxZRuC;fmwHzWRiAE`995rY*BWM75||ofL*9{xsjo}7JEy%s zN2%?LOLWmhSPv63sI(01hhzDY(9f>X*_SV z)G{Kg%Z+)uO2S-jZI;L;@;Vh9@P{jKTEwNDFr`X&M~ zzct})W_Gp-CSpd9WkJgGv&g%bx3&yk={dq3Q>sroIW{@hz9S{kSXz zS!}x3IT-9wIJ->y-+6}Xv^COFNbEqL&aE`IXT?fF zeZ{5~3od)XSD!-5RJJ@tuAQAAzhW!JAvrQO-5pz7nuNLm^(0N+IM1_{3wt^AMct~5$-O;LjVIw3 z>r1+*-|^xo*R5+b@Z|f$UsK8L20CL2P<5*&N+-&UtK6DqU>oW?wL!Nkd6^Tb$s3(# z$RQY{wVC6o)16vm(%~JtV6!QOgpa{(5k#cvS|-)gX7F)9roMqlm5pyy(#AL9j;`Mn zMtgL351iW^4YzQtj&b1k6u6F)a&tSvd}lXU6ZP&ZTTejDOpa!M7+<#+`?dB)=~rcW ze z=Gzq}z9y99S@R{sj6N%!8)jlqzE_>BX-*jHv-3pDZ@+;BEH^aMQMgbP*4m`j{$X-) zrJnlWAYDtJ^6pG#m77SoZn`YV9q(-SY`01+CHQfB+1w9;(wl`gzFlz2KL5U*koBO| zG%Y_*aT2~>Bv#qeM>5YWK36M6R%?((mDBPS;;3Cg2OqEWvs04+7bg!s`l{8@QZWBz z{+AQoVLciD<{eHHn^b9wQ3&5>dH_eAhI)sfem*otXq{DyNdf66%NA!qRbEo(ptEU&5^TGB9w6KeSu`~vy)dYTy-hr;`#Q?&44s_{&{GkdGl0qK+^Iu z-al)~5Xfw>RSDg%7b^(;hbg1xlQw~KCz};7cLXm%HhCa8ROiXUMW*+H7^h&_aGJFUfr%j+E2!r?F#9%( zQmA6yU^hT0^RozwS%%$_&a~zFL2(6jo;JOLAd$(!zE)(_>_K~Ib_2kRDF)@L({!kP zE`<$pIxCUK9{rKfh8yoW;b@S!u*MS*TlMHD(Z#!h28-SMg?`+VlCzOAK*a^=E-guR z*vg2&3mmdS_nXZs@{;9rCbRI*h#PYGrWi^Ve*y9H9Er(sme9HeNfHcAr9I|MwuT23 z1w*ab@x-38I4nm0BbF8967wm~tvqdJuOk-_8Z8|X_&c*^n!HnPql!OW;6+pG%$n*@ z3JRrBj= zEUI8ZJwbdptD7slpq$#*He_DOb^j4^cc`cHRmZC@6HHnw?s!bhbu8&Be=NC<;1k+* z{yG=y^J7kzD^FrRBqQaiBsFbpqpb$hwL~1!dghxUBW5T;3nX}Vk;AwYLrLY$wiMNk zPM@_z-A|y-*g<7qCxZh5&FNRF_oF0`)r0c^Ij~qpU#v1g{ZFwIqQvE3JGQlegr4p| zP?m5Va{G;^_%*+H^(!2z$mWC+(GUTHYo<-W@-{EOH>yH;&kJ@u&y7)*z`!jeo7P*o z6&sCdcDDzvZxjzwwb=yXck>;Tj$*;fFe}T9QK1dhoJX*djQ}dnUafUsto};S5bN;f zSSm5lo@My=o@)daH8bjdA8wJuvE zLI#=o4mw9`=Ei~B9>K)H)M3Q44TE!%3hkmgkTa3Z*4oYST|4QM#Wv@Mcl#wh`JA&b zi&Uy3m`r0wB%@lJ)1G|V?yv>a2>iMqL@a?&bK4l6zf`}#5Vh8WJ2R=&ygv!=>X&~t z#FqgF#dYPq!A+(wIt%2Y9ZvuAmq0hM716y`raj3Eq=w>*hh+pBo^A=XGrkM2yL}4| z(~vhIOODulhAakhSln^}dGGL7nN_kC5ip*TgF8=}pbF=?#hWg{2LOM7a8~*A*O+ENyYl zC{%uJ%_GWTZn<4a>)e4q^q$6&NFW^V zCVL~)CbN7~W^Ed)A$G##`7YZA+a_yDTjU`96R37bmzo)&u@CZ1-jLWzYHeHH=4IaS zMC}iN^;8?rES~wk-Zx;MpVTX}<=Jw?S{2xJUT8W|9`6*_x{|994|3a>RA^-9v*)JX*I(e-38%DAZ z2Mio2PD`d=@Y9qvNv|qxTwH2TmY}mk35V{|^sZ^5X$S*$4F8A~j6#iv?oRwIrg-!9 zId?!VlGxG?%I$b%dxE*Y1Juz0wjvx#G8|(;^&K52>EF|gC2aH%SDepdyI;gOKZ8uc zStKcI{5*u4bgi~+%5s-j*T1(Moi(ZP9D&p&E-%s4KRg%Z=3N$b0++jTBwL+Du&ZrC z>`$-2NYs%cEc>sVh07{+UME-_T3rs0QOd~5XyodT`JY2drn!$LmM@EhW5i5dVx)c< zd_sC|ldhKITc**ZyT`Vi!g<;-8y1w=mfav)un<*)RuwBn6gb-GhuLB9M9>tv@j%ai z6m~!OVFpnGi4*#x!fM#TF)V1&8wtYJV8kxN-Vz3`{-6d&z|Rp+z%r6GFkWEHu~58x zl@Q0A2aAF8J3*r$45pX}T0o^*gYDyDr!#;H@KA)sXL%}?i$&j12+j8`fwQ3S!T$80 zdS@a*I6g~@tSo(055V}5%|7Et7st!>#l^+M1wSPczwh?j7x1!fM9@Q-OQ&_AM&%t% zw+5*4v5ZaMHA|z>b5Xn6`o7Gq`s}96RljkoocmlU(&u+cuNnNyxzbOWv)%r}-zmYx z&x*m6FOrzQg~*P`a5|@kYLFsZe(f5Ee1%VD;Z59y4Jl^Q9d*PA?ET(n$UK09$PB=) zGUptKF_mP}gtrP|3h9T(3Bm7239-SBV#WuVv?T4G!cQ9lDf;WNvnFqJL(Z|YrvEc| z>KTH?(h;;<2+wHahc#^@$S007gg4{&nKs4!4y_f5ccT}Hcgq%Ou;Ui;Wc>=(ZS+k4 zZ@%;v>s5Qd8Xjra1s7?b1vh#}KT_gWD{}beOTRnodnl3RE1tV0Z&b7;2y&zE#=Nr3 zQM%mPTb357t3r)F%I0t|PE*IT=s zVq@&ar1EH=q7ritO`E>&xY62t%vgOfpqj-ydiO<}zL&C5JIukR5u9kbEZpw&2*s$W z8k)mqF`US{5iEPonyf1q^hu7EDyM%gQrEequWZ`&t!&zd)4uG{K=*#v-1{YvCSOD! z%=NC?hdcfEvj+K>QHmZc+k5Q+Fv)6zn{lzy1{!0^PLJnDqtn2NEbdb8R4186`s~J# z@vS36d<|k;OYFdENh~Y(Q`8}2KA(glBYLUK>hVxm_w3g>J!s5B^OIs+CVj?1(+v}JF;SpW2fc*3aEV7Glh#`| z?%wI?W|8~JTKOesz;OTR6$qDqODY~A@lE6{Y3blYaSsZjp3=`|J$o&cP*a|4`y@gy z=Vz6MB#wPhW6=(s!;2G?^oS(}tBWe`52>ww{W1O^F3&H@J|Qc#`^?CaveZ~93koil zJAU}gjPiiSBU9w{I_#{);w&z#^&D?x%R#|shpg0Hf@&EChfry#rD?Yat=(~_V}#Zh>$_u(rW2ej z+%NIs?Ct6Nbo=H_15y;1nzO^FIPn)ODkf578oSgCnI0BCdFqY^^Xax|rIn4x)wB}LP?ZU6r)!}>UiFnotXNt}ECRu^b z<^&LbON*BrS2NaCfUIr0G1}YL(%KCsU*bCVr>%QN9Ne4vV?HVLGnnP|eGKpZ(_7hv zg22!uMOQOF5=nG2%yBQp+#zf{8N&R=l}Z5&0{jOHDLK7`8YskQJgGbhxNce_r?oL0 z7~1j~WVvX9_2jLrQEb*O>W?N9UP2C*_a2aBNiz>V9(9sLgkuFIAt_!56N`NWr3-B2 zRDAdF_r6ZOT5LNJx_AmabEkbIcL^)ehICstk4@P6%O#@FbaQt)!ooL})j2u={6g?4 zB&V0}6#Mr`UURJTx=*V_tbx0dNUI#z4Rmuh55I%Sd`xvSl&2oe^>{NQ5K?dP%Vj|p z5vFlo^AA#J1mq$a9j@T9O*=VK6KzUlJZaH@=@P4%0Pl?hMj=-ISb`BoQ+-k|@iGSi z$*96v7)tXIwm~5Yb$sS%gnNL?9nwKCkQf=~G`Ce!SY~+xzKxqV0Y>!6Nu?%-+pL=2 zwFDWGf4pNzPc$>J3I8h}j!OhUP76e4h6tEUusW=J7yOQY4Kolhc<1QAY8oQapoCy? z09ncW!7iJ{92aVH4^ue(%AldCdUqe%#W3cpP0gA-#T^`6Sak*aEQ${*1P0~+II7V_XX5LzDq8AfAN&)L;u?<+?kS2yAX5*q(VJo zp~Aw`Ta^SLvx+2y%C$b6!et4tq9Wqw7}84FHGT&ZC{oqmewS$hTlU0b89kyQ+rxU~;$7oI7{+yi0u2Gy8ef^S?X zCUdDetV_XDOR~R?HzxnCLtr!QvzBt8kCHhGdztyU4&mZ1iOCz3&ApIVl1tRHytGwo zj}qLcEam#G+U+@5kGy-8V%E-sF5BU(tnlx z7bXq21ppd@1oQ0s&LOP3MK@`-sNJL&Fu6@AG18)3(N?JHw;Ac}0w)p?>-j@*e;`qD z*q4O|8VTH*?`CrP?A<*4*qfkV0{W#Kw}4%3QAO=@h{pWd9`#ke~k8kbQy&X|rj#E(6St===+4hQ*H9%)Ss`qM%sS}JSso!># zH942|6WJO5upo1mo8tcv3t_1&38J53ftljJCpG^|EF^2Wz>p@t@qDH*{)I3OWB8hk zYwPIOI63w}1u}t0&`BhjYBiA{-+;+`$E*m-3LAcc7$L1RPQGLiTE!Ed`Vd)p7D(Zx zQP`oSL7>GYJV?iHJ^eF{Hd}4$R0hRno38#@o6i1N^|_z^BrtAuq-wpC78Cixf7u|_ zWRNB|N(@p#t&D3C1)*-ntx$ws#S z?Hx` zc8(|mb}QfpHb#9=v^&JTt`PAMMge3HC;?Rd;yGvo-=a_fAebn$4B{659W97y3Ls*y z3yeJgNZ5N1V^0bs=?#amCjr9r%E0K0-#7bfoU6 z2Q(vZh~Ip^Xaqp0B-Wto8^A1nm%GW6F7fB=wV~NH2KvJ{_g|Z}p!t&Yckhq{Afb2% z?c~={p$LMAO-Xt|rb_aPVB5*T2!tfhh`mUrN-)_;gUZiUbY(l`5Z&>E%+K+345VAm z&tY^7AundNs8cB3LYS^MHFRgY6?E#o%cixszBJ)%HwAP{eNU#d-vp8exmo2{l`09| zNy8&=JhW^B7)SGPeCfg`Z|t;MMsnCW#cnjTT6RJ9TEJbrI?JC#V|@}Gh{}gQ4NzbG zccSY~f*i*3;CyMrWp>Z&uu0~{O9!#F+OV!-2YsLG>4tW7>Nvl2#|<{#plRU;8I137 zKS$Cb_9mM-VfzB$g6UHHvPVh3e4rcFL-!9~fbr$ofR|m&)+9-#r=pz7{55gqkZ)?~?_pH$=!KFo=A_wA( z^h9RMD!#Qx8`_|^yOplRpwcSxlqoaXw61~Cmt-jh=AS8D8+^+QVP$Bv;@m*bZ(GRQ`raYeu)elBjm$XX=rd*a(Q)RO_PpOUN+Vb&ashWBVv2}us`?dc=khKYc)2JLYPWC=E#+pjUeA3TF&o) zX17>~EHp-0RB?RPs@B&`mFg9L2h4-;JP2u|R2`p|uQ@sIEhkB%(%5EC3VHFd*z*5g( z54&pl|0Q*&T<%jfEPe~trUUeEGIf(qv50R?asq2TIwXI4Z_FK_BjVYo9%O;u!}JY; zan0RH7)B@UQe4hfy8Ogn1>G@nb6YNNclKB@I?1fVXFp`;UgQSdpS#yhjzTF#1n?Ai z!mKr)Q5Tyz30xRxHW+G%`BHVHQLMlBx?=W@yQC2ssLt=nH@9OIwa>(|)lsR1^OHQG z&+GM8#(48+cOw&Nt>jF0QAQ+xeqUR%4aq7ppEqUkp$a!i##o;@cs+;pAhkVZ%6^e-~e zgsPCL2qA22J|H5pEK;*$CNcei-2Xo zB+`kcm(3$R43hKNXCyX!J0>TqEu;KCOty~8{Ah!3^w&spc+G6pqFv(m^D6=8ue5t+ zQ1RU2;l2CrH{lkV$WSVz)7YIHT+&SvzA8~t)$;vH%J0ETX)^1e2(^q(DU8%Z6=`D{ z0?s6AwO*z563OiN8XwNtWI{VRP9|Esixkmolk*x6!oF4WF4kO1X8~$G;<1)KXVJ`| zk{V+sX>qkeqM1gL=5>MDSo556QyQo`A8kh`l1*{3p}o11-SCrGS<6+{-$1vS!?dc@ zuV)MU@eJf4Q%ZWEU6c7LO#oXc%3&DwZ4j=z7+=-a57CR+{7!4x!Am_jStPl*PA9gI zF>5qw)ywb_B{dm}3;SrsgJTg8Qh4A?=v#;o z74))#80W`EJJsz}n4V=;@0(J7EDo?&!)(oXSW&TtmtbSJ*d8#yspLXEf2hmX0Qxwv z(ac22j{X>rOSN4hy!+d! z2TSqmtirYrbp2SOFN@SD_4eAkaFu!Efv(_1tg$tFWOokVDC;4$ zshM)9iJPs&96bbEPpCpljp5_)Sy?0hJAGA_Kk8*c&3dE;Yoi8#EC<3?F;G(kG!53H z%rvI2DGp)>2m2UmUetu8qi3nP<)?$>jkIJlm5AX2#hESTGsGova_Zw{Fnb^r4KWYH z>kcXL?1F`~rRUp+%)`6Bbiyw}MW-yVSz5v@a%r3QjF#%b9&=uiXMA?^ji=NEs+{|K z$)0J)({N4ow0w!ujc-hTh(j>54EUK}Csui8h;r5~+S%lWxVegkDepcUDmYe4!riVt}Fo<8ON43Re@3=*^ zL-5D6W6Jl^Os#aT{^4`z;nM6=O5ah*0Ncy}D+g9Z@7o8sv+%bhuo%HlIaU!Iv@~ip z+Dxd~yBcBxx$H5#R}R<%AVFWo2#us%xvd^}#Bw;x7*4&h#r)o84a$Msb5FI5Dh@I}xmA7Hp}GL(5`; z+V|nVtF{pE56Lf^SADp8jOgLF;=T_dOia?(&;Y%w_7+)YhGL)k1vm#qGEktM*3-^I zX}QkWZuCRA`;@9preQRkR0VjMIA%F@EDW)x-#IxNe6z;w6n*f2k6jeS3)qv6tbd74 z>-Z)Qj>EIG2mGRr6phrl1sQh;XyC~PeJOnaaArma9;$1e@j_$bPqf7=M#s%H6rhtw zQs!=~{zb0n#3-8*KIAO3;zT&V=7FSjpazjS})w|NBNNsrfpI#y?y`5(S4H*q-S(;D=sf(Bad8r3!<75Cs{J+xRQx$xw z2D1N48qkv0YM{OVoYY`6jT#_KIYk{?AZUnH!POjjddI(Q$=vYcrSvK|Za`tSWbHuV zCG>2vQw94Wyp*w11*RZi_Auem6;!JsH}sNX*Y=X>qvbO<(o~H6R(aBVZ>HqM8)Q)@$=kU*?88U>U8$ec zffNYWn+f4S2IT4G`L-o>!w}$(*70+PA>iq=BsNw#1<8Rv*tsQiCrY^_$OOWHA$ah{ z8ArZHz_bMf{Z0jk#w&S4)w>U&fb1=g$!oi^H#$Q z-?6CEgx@B6Zl}ArVbr1{-(m)8V=jIt3D*Wj?C8|Bpc5ty!af(!srNmZ?fPwx8nf|I ze<}tSt%Y79(^||oGGJY4{U%Yq5x72^>}rBk z1yjjT!=lIu!U%{KaBCu{S2tD@TE}5fZ+nC?76y&Rd zf?6aS(VSvdwid=gEceJRHiq%O+Q|YTktgI^lS-lboHhu)-|c{6{&%Z ze=Pt0HAKB!bnO13qQ_{aiovcc&2>(p)h?bGv^VUkEs!a#Z9@?cJw)2im0Auu$L8U9 zhfPXJoJ8=D)b-VHdhO)8IQhhtZ^;l~x*#=B+N4mv<|Jbb(5cctc9ddp8{={05x!*A zmSB1?b~mZS^^~aFf~^lC!`0Rpm!H*j+}~`qa_V1YXd58&>A86d8*PZc)s~=s`_nka z*ZHKs)nPC`Z(H9K`cky+(mn$!6m86#jsGR~{H;Q693gVBzl`Jhfki3v?{~e2#xS|6 zIL-)sCX)xu$*ds$}hKlYiWI#qw!_Dxp@ zGMwsy$))hf;zwgTIDX32pL4xETx6sR{@sc%W|^{)$vTG2`gye~wUfgK`r|oK)VIl2 zkBo&=c~$h4cbQv1hc~;qf%DIN$U4R9(`3MtMpsm-$}d{yc^>mk7xw8qG=W(HNGfzU zUc!`g*u7pu`q}O zA*(vS=tsS@JIL$7wQpW!fHK{>Xl;P<$}U|b?ikFvTlq1@Jen=i!U(ASn;7bGqrToq zNM)n=$3)G>TGw~+P&mqDw>5DzHy6k9W7G~0kQ>Vl6P*btV~Eq2e>gw3S>{zZUR+lc z-Cc#QH%^(J$P<9N2)1&71g8u8RphV`<#dRYHe)a~K`jHSva@b%5Il;3AUVewuZf?( zrmE`Ekzn4(pgR*vu9*>fG9Gg1jtY8|T(r2KlbD;hgWA7YXY1?&icG5{YcG9tJ z+qP}nR_DaFZQJbFwr$&;PBQ0t-kEP|-mhkAPSvS8f8TZ8``W*=_FgNw-a$&B{cdco zf`axc+W#8_OI%n!!b6#36U%#jXbySBZEdfl!~x>c+CQcV z!a9|-A*~tE;UCKd=i!Hur8s2djbE1!@I~#Tt8EXdYPQ27WG>1QaF$R;SSqMEtg%`D z@It+ZCT#t&p%xyKTr#+T6vOupje19{VSCs>7bAKXuc9n*p^yn(a*W;H&CsaBB-t5% zM8Md~8(Co6tYBs^Vq)z|&j-l1jl|g%l3jT?ioi%7I&clxC_ZVJT`ybESWQ;)reHWy zJ4VHy!{g$*j|>JGDp=z`o@!gwGi~?JPaZ2RNi;1Kx-sCj7R3^BwQ4W`W42V6XtP!l@sKV@q>`6 zP51Wl%L~%j3s#YVP@D0MgS9y=U*K)#aXY43F0JmBeg}DAF2~CcR*sdSA}BFA^|Iw` z&Kd%P42da3qt?~1wg6QEp{>ccGwUxz% zR!h8(vTd6>P|d7MQ7Jb1X0-Z6l~BM*8~0gSB-3`J%B zO8~^cd(~ninyspNBR}Bt049jxpw*P%7T?bjbU9~G4HBk6v~kN%Mn9w-Oq{?vKJmb{ zmou<+Q-Ot>+<;qr3;G1p3Sa!GR~WYWLN7!MDi&kW%B3kRhx39wFFsKhEY?wnzO)mB zZ@6uCYOE$y83NEeoF_Hy>>%(>Trc4+)n)7lbI0tsqWVB$R_6uhHG@x<;|p|*7GaQ} zuK$^E^_1p~Rh=tItkZT2^WR^*Fa;iyaW z7_A9<2kw#TfcM6ieoBpDPH%?B?w`1A-yS$BP;&NhRtEu`bqObK4m#|c6N3iSC;iP1 z{xHB6zLSSo+xF5a5@jJT`hYHE25VERijB;sdisy)1bN;WU&t2R&Mp~yuPhxH0z=}) z2xmP*W3P17ma@q{iLOoc+N7l2%pie%aiI6^M!K$E)R#r-N++2CE_nxL_izG`Bsy>o z;c(VK^a1;z8)J^xd&DxREsl%dZ33>2%dj=A5X83zx#|K7A=bRO2Nj8LP3ge+VlJk# zNTgvk`3zg?$>sQkgHdih-P*C*?pBTo><>p%A$Q4^fsWlnx9Mm76J&Ym#MA|0DBW4k zMqtxS@ofo(6%P2qxQAthm%N7b;#;8lS$Y;aZ~zns%$ZM6_GVq4RPH=4&hw>oVe&-w z<4cn^Bu}_o%D@*?I#6D2%^M!Li61^6aPEO7Tp&|5BLY->AapAO1|2BJopgRs;TIq)}tCW7?ERP<80h~IR(1X>!rQEj?c+jb@l>FJH3s6#jK`#LY6JGg4>%$w*Q=jtZF9B6eh$n(kv2rRhYeUMQ|TDE{RTLWn;p zdTbZVIr{FX2u-$9cWAB2~*5MNdj>vkoE(*p&RYFIhThcq&=2)@jHC}MOT zj>Zi*5V7;MGoxgUA3mnX#v{m2O1r&k2WMDyku#@ub=;bOV8`;hdu1KZ}HUs zEp4R!54WV=iUGZ8wSIUgub}a*q&%9ufeiQgeM9&MT7rP{jv;kCz2UYJG6W&8ImP7K zdBG6G*^=rVi3y4wnI-wewh|G6P_j}T2_M z-4J@z!cFEO^UOe?dw}%8*eZbBvb?q%Cw;7wiT3cjhi%z>8`CSh!Rg zoPA{w6%dA6q{>1vA-8p)V0K-ZHW9w!_ds4V0W>zZiUIN{J~6uwVEP2F1vC|dHUXRs zWk`)CjJXCNf7Q@o_9O%Jk^ah|yS%I0EP)8n5eM=6koDPIv{OySc)J}Zb1OV5d{cL>T!MP zBK2NdY5we$Hmu6aL0AS!@rCY2(TIt*)4=!An>ZqV5=K(|l|*NJEv4B95*EpBXaffr zAWk%aM*^t`is|;JCJs6ps7;K)YaKBFF?P@x^q8YyPlN2}_KkNSqM!!Nz>S0`9d)jZ z)i;T~<{FKm*3oqbr1dQYOv{$ zrWZje;n|%IBOGP6O0q@%nVF_G^PYr4@h&TB;n_u?WRvzl7pP2hSCS&@ssrVbO%}lw zj%tF`w-w6|6{<5W)_9({aQL49lNuY7!>kWe8bTycKC`rFmMYn%sd6mCfBOu}O34<6G?+d?hiF6q z<&Of!8E%A|?8pPO>BlG1xyGS7a-6=YE&N!@J_iAA`I4CJsmK?rPI7%fRdPyq`F%f~ z3l(E^(i6#nA|GgF%i4^zD7rZima0A4j>0 zU(y^tQQ>0`2V$1RQ+tSj3L4ym+xm#7blidA;5eVb{DNiEBn(bfW{ z>kQZ?{?&FEd#_yW-J~2uYcK!)V5gr+XN1T~9apm#b*DOPd56PH$?yD#1MO%vf z^^4oMWQ<&vKT}>?3QOa!VS~3BR|!v@V`;DG*%)7SavlVXjyKeA0H{2gFf*3c`jy-J z&>5YZY4oJTQk4B@c5xcD7pjuY59yTz72!b#cd;au3jOe4VXKPXw~=a%ZR`*zF}LzL z5fRB$k7sA{qoNmB(tkikcbsAzVWxVBW9BOT%<}iE>*{PRZ5LE5W>UNM>+A+Q)>h`mmKBpPz72ql1I)!Rj)iS9~7D@W!?xMS} za)qc6b3u^@>KM(Y=4Z>0BMG6x<2QS0I@tu4v~@~&yDvbBv3Nk<8>Jk9BV60Gj0E@i z%%xPeiKSq)70M(9cIJP1L5DnIY)OhE4OIp^h%(|f)C{oLJ;?gBwkAVWN`#-OtL+-X z$C*xTrTjFo9u4J_5M(VfA+iKf9cCA=EO^~mfe6HyHDHS2n=BaKvxnLZ6M*>>!A%(5 zV)hTm6jjjnDSZ16$54I1ba~F7L{}C>I_NeaM|z=J^EA*BUi}`%JlBWVU*%?h+1gX> z`UJG7Wr;k2Yif5k*k$+F;!YC;{mU^Fcu>#fg!8x&C>a06F%>}NBG!88iwd%#e&B?#m2ozvigr(DFLOBm@-Ay6gf^F zJwVwK%H#3G#}!EK`HLae)L5sZA%IGLH9pPlPi?j&8Z!z&{i5fGBT}-&E7>pW4Ol~- z2>!$HS`4sN7(V{}1JyZmX^=?hZJ)Ws;NToxFb*k7BGB^)e0b3FNEC&kdrOShyOhEE z6Xo0na_t(Z(%>hYLw*)VCjluwAXSzZ=GxQxa9?3NSdM*tc!0SX_~ zVvSoOEnzpaQt^TE=`1u57%~_fTM$v*6~4<6-mGc0k@hnia2u$%b)O4V8Y)%JHHXZR z%0G2Qi$w=N=^Ej8>UJtUv_aqQmz%qrulmqobK#EKbuA=E&aNX&nQ&KHYSFZ>mZYC6 zTM%eNlO@ESOnN*T^(D6A9ROZUBW2_76^JHog-m{M9uqZ5%rlXXY{151 z{?-yMT;qi2#c}1%GB5j%(+tNqtk8L5UIx5O1GPe9QWSF=EpdKltpiAJSw?L49^Lvm z4*J>TuGBb;6)xZ^4tZeMp?62&I02IMZw6vN*EuU-a`ra6vJOJ_9EZ1baa{f5V9%Lk z$eCN%%o8iuV#;^-UO_xYcj?(wQq^H%JSfDoP6za2tU_bu)7SSMZyl-U0vxj{+^EHM z$LJk?jOfCi+`se-*8zf@*^;z(VBoVtHE~>NeFB{gm`{4GayoMk{_F{oL1bsaQ27MF zTrJ_ad&=2n*H6&O$ppIMw4(D_J7D()0lul3FC?rx-tv#uz3*}ZcOsk;rF+HG#^4;~ zXvS%8iHu*8(m9g#T9$40RP@p#1?QxGAoRXCcmBo!@(g?XFu;(uwCP9pP(tnQs5NvP zyORmIPcqCUM9c;>zEQ#KkYb=sfbNiDr^dNk%=#49?1W;WP1n|xBJ(XP9oRdEo%c0~ zrb*cmN=YiOqZ?2k!S5cVOU-P&p%d61jK2s9&eXte?X%irRz{=83{{Kap{mN-7n}~q z@JS0F#EZf52ZUmtv*NH$Z==*4m2K=&E5k%rdgA=?d`@jIf4bAT_UlL}gg=4L*4fs$ zPuz1meA`v#LWnJQsxzWy=ad(|>Qw+d{31w~6N?n0Vqylk2rv@nqODQOgMu zrJ!6PJ)+QbM6Rki4()RUJlkbf7%k(M7F0M(uWN4)?&5(AMathzsIT@rMfJFCNr+ zIgFY}0M1=*6ph$Oo;s$M0#EX7aMYp)lAwLrweEXY;H2N1yjcXym2=`79dt1-EytNz zG&5LBq#s&0x>VJ}kvJxFAZyL(f+Xz5(H z0p<3~{c}E2%De|l-GxqN17@|88I7h>Yp~92OzM_tjhM$aCN=X6 zp@p8#Yj0~)Cf7KV=H6qtQe(_@X6KkBr(f4t?`Fiy;7}XM(~}nIeLMCo_pFkzH@H z%)Dl^?ReSt&Uy%`J0jNPV9D)y(eHZre9r}%e*uB%m>+>3;v1f0pe{`cZ~*M5)IO1< zP12y7tPKiqv% zf&q%YPvkN`5h$7vc9h@2yVXFEr7d#TF2Eb|+SLz@2#lSh7lNMLBlx@2ZrOJ!!`kmp z&zuH)QJe-KuqXs?IBnY?kp$-g{%PN*gBkiD0fGqxvJQU0DKWwT$AAH_6=WUpTc!YP zC_TwrtAG{6-zmFf0V_zqb9Ujt>`6RO`li5c2t3I9Iw7}Zfgt7muf`Pcg5u-1D-4ED z_6pf24#g)1RG*LnxFN4eUOD=H6%z3I?M!o@&=O+wFxBH?biHEjy~xS%>9JD|LFCBgRM zhRi%P(7S+YjZJXR=)O_KviSJNEk2kbY2BF7H(%mN+X4ynK%goVL^h28s~+51>}X0SQPRB8)J|+GXf#23rC2@tZn!rIGjU zL5sozW63Ru=nT54(Zgelb-K}K;tI+xfdAbT&ea3)iNy{m|2#r-l-kMe=pFgZqpI!? z$Zmu@SM)vpv3W4y!VE_!Y{!`@&qG$tXo)T%4(G^7F_k&xi)fzPY|@N=p)T~yfv-3$ zj__W`Mko(moXzXN#{^Joh-t}E2`jSRJh%{r&**MGWJ@0TWyr*f=$qtfeY4f(k!TW& zVXGQmG9i(uE@)sz!dBh)+F5k^D?!7SLN-^@EfL|BPsT5OGpZ@!1*pjgPLivTqEUC2 zT`sAYU@1~MQ9UUV9C6r=4A$W*%Spj@6tA_bI#Ic6=-6eTR5mmm7q zuPVqLd%8wtKj;r2H<~#c;~}&6{B-=Q%Xx`$Wdsy}&{7dG_X7Ud6UqE%8~Z%kpzXY`^R#c6ZeQ%VFkPW z1O3E)gS#@7lF&pZUn@m9k}ems!NNzzdqPXOLaoZfKdR8G;PziszDS(`_3`aGJsJ6T zjGqe|BTMHQU225y$`3Y7f8iInw3~G(-Ij+NoPC#v0qAU!>lo>iuw&kB?Yon}Y^L20 zXB385GfyhxPP*}j-@UY*XOkqemh#3Tgs8pFevi?jn%Vl$ZM9r?sbLxOXf$ zxX8;n@w>TU98=w?i@I@XbV$oX!siw-Rh7zpN%5d6%W;4cWk#C0*d-im<-=lyQXRcr zxf_(Fzg#GGf0s;cCPyi_y=*4MKFnXI=X5Xw@I&ETQ-3~c_CB_Amq*1^dxw7MNSY1Ep6}5hCpR0@J9MoO6tlqSmc9u&<>C z1n)oRA>VBCmx#pUvGn|k=R}gHj`xrfuw-E5>SoS=g6O6^ch4N|H)K7#hkaYm)D{w6 zduA&pUBr4egjJG&qc)yv@^d;!tyERiT3-nkTFuWl0X3#5>|dJ48N}){i$YTu!umC} zAf>LYp_(bYM-USv6>lVS^atS&Y;h|#KpdzHwWXnFdO-q%{;)ik2#mwITWnujZChOiMA{k8 zk1+u_!_f~}mi{3e`c0h>N?gG7Gg?GI)fZH1ak{^ELLIS^6R-eay0RZl|LyJ2tb|}u zzJ^w`1yX7*wIzR0a-EwXOH)2-Xv@+}8F_#ogCj&;(p9Dwm+N^JFHymKAljR72BEI4 zpvb)*s^vghI)&3tV0+wmTvt1_l5b!|IkOtF&ZD;!8g8n{<@iV9;vTNq1I^x32Xz}pOfv_22T)o0WoL*BGxJ)-pf&G>UMCI89~8CRM27@_uU!2Gip-W?~Z zNZcz5l@fn&>IRv$iWQuv4x>r|3|<4$tt=0q{=N4;PSFkPs7=ug^T@F9l5O0&xd8@! zmjA^c@yMeU2ka9>gDn4yB1MQspvkg@I}UT2A=L=Xgw_e*{2R|>ez%$dwfb7}^r-Cn zEoDeYV4KN`Lw(R6RZ@TpUiiagIG*9YjlyfU|I399AfD z%yUy5*C<|cooU7Qy-a2lS>Xc`o0GMjUErmE;)K2UO&oX3hAZ8Z*i=4*JAMT|cg1rb z9S7`33h~pmah(yt?2ZSwL<;fj<}HiFkT@VOe-El|6Z!#5&L13G*&QsUpje|7wb1kl zsg9k65A0+M`axdE@|&p|-j5f>`N~J_@4k@>71B93O{W=Ca*uky!W)>R~+DuKhAuZ8Oq{7(N@(VSr&6OXdAU%LB zO(UxfS=-KhTkB4av^I+2QqtCf%ZsZyfTm>qEvy)rd3$!W<1G{T}q0K zAlAg$INr#Zf*ZBhm_6?vw}c;%i{^lKUJ&)4eb;|zRu-yD-p|yS!L~BGF7fGi2#szr zoEf@8IMawcb++0c>cI#Kda>o2En8HlUz=M6ow2t%FX+ZSBm?-{ZvvM$ZW1XExU zdVfX*e}H!TjA`&4)!?7n+w|shdq>tklh8otnULzSdZ_NHMyk68vb9{qB=i8r8(Rfr z0o1oz=@a@(YuZg~pObEK59(L+aId)2N(;FIk0%U&IY;%&_ht-Vo#T3x2Jn}sFXXMs z-!kiEEUWN?ZZ3>!aUJOfxN1 z+jJzuwKBESGH;HDXs2(cVA|EY`L>#QXRA4RBcp@G3}{JJl3^}XI94w8X=|I!Fo4V5 zV3uweTw^x%%HO~v2p&u_hva=t+8!2Gr`b!_J$77>JD6Lf`6XWuLv4z|Wu-usIdWH{ zwEf_)MNznLlR`;*%YKG(XpbGJNH66joGxC7(76BCXm$LM&@l!WYed5Ezo88~qRV1l-ylk5o3PU(ogS^`qJ^Rj zd6O5lxQW~l%}jO;LCm;3#1EoSbtlc%8b@$SExHuK_-wC{o#qRg!*9GhR_Z2>s%&)F zwi2|7y;SPVa1K=XQ48rIWQV`l4vF>cAWy_B^9XnU-xR|H(wZ5<(4E83^dugCw*5M4O~Ll{P;; zZA(l~li`?l`Gr5g8U>xBY zp~sQuxW@sHArcQipd4Y2Il+O!iNv9YFxZ{{7Q%o9F3b}F2@osTZNnhV#1zJ-8<~x* zWW?#`B%P*qBn$!hi!ne@gMl#++y~mbAKwWLu><5b>#*b;3_}B2?qug5J4wD}IySPHL1&9_4*^yP1W; z^-bT?)4^`)JnhQdIim^lnpIX+7Cg|8yK%|t%xEKy1c+)@a`&rW^;GP=FHdEY=SI=r z*ld5FaV}Co;q=&Wv;Be>)}PjnZ%XmpNFGwFCc=)42;Q65Ge@-@$vX4T-==`2PcqX# zJdp|MV`#J%L=<-Evu34vn^hZmiuBsjBv#0q3J*_~uWf1mfYh1D- zORU~B0bpEH#P9X9Yu6uf$n$O#UogCcl3w`sx>ZaFoK?8ql^t5H>)vQ?06I44W}V!f z(;FM8v|_$QYQQmFoNVxOFPoWzHrLTQJiC4d8+NddX6-awj=L0%q|6dLZ^w60E+Ws> z=2Px4VvxAQ{u%`svpa7HvdrGVnJptbc~uB~0j7O-VC42I`z^c`C7J)q+#48Z6CW}SwZxmBz9}qA2!&GL{BS?SOmUzcj1D4y z7=(~xI_>4$gzQTVXA$B`k+9np0RtFo{Je!MXE0zfiA8?L43Wi|-J6iGZs4q(DT%uK z00aWIrcLk-yiA2}X;jz5xsoN&Q=tz8t8W;E2HyMz0DG@w@WpJm!eED7X!#8gE6QBM zA_&8uiybKsE}4zSy-|7}XK{>3)B6_pFh;pZnt#FyzsM5s9WUi&H@`PSw|$}gqrLYz zEhhcrY|#t(U+*K(K=b=LP}q20`ZrVdn;cCk8gbI%#5nZ`ucrhpGwn66_24)&2hJPH?skIoo z7=#>z8e*hBt^^Zf9SE>|{D3s3(ITz{mJ`JPT}~Vm|F4=UXK6$7pQ0J8vLT0}h^iZk zUIIH%-9$m4lqW?gF~lA4lTI`fHKIIiu{EDJxbI9ReH!tR{I<{I?FSwSnF{IM{kb^i zizT_jI@o*N%%$sc)vwGgen8LHH-_jSgay9MXG6_?U8on5Nr^UkL$!Wls94fTX^pr# z3d3?V04_${o@h^$8SI!vpDAX7!>|6PLU}T&A=s9DxKvab0-DLTfQvB7wm@p=eGJuG zFkSLR^BSxLRb%Fq_0?ld?-o3{jTAE*WF9F8%F1&q8tZc|Mb?Ed z%isXxN^GuJ(&hw@^dxe}O;PgVycZK*i4!ED zf76+UW_J}-GkW(ynOFF}tt0Em?oaX2$xx4ciyrth)JW7OC7rg#-|K9LuX^M4%5UHc z07j?cSG|;w6t^@%dZe%QQJF&!e?$cGVF|d#e0Xw6aUh>`QtGY%c7E z%5E;Hi+Xdv^7i7m&-yRxEVD^+o<>b2lbsy{r51Yu%ALMa?AZMcP$@dX@KcuJFafsjRb;cT(2LKQa zZsw->@vr^7f`pU>8Es1RyKzcVTf(-EEALp=*_`!Hr_i#5o z$(;H2*X!$>Lj>>l4G1^b%ZynAEdvTo)c85Yio6IXa910KrgdENVcny39tEje37{SA zP&~I%|MiGcw5Z2$UMqkbVer(E^cQ9}Gpe_zLL208K>o3W{^sDeM3DO#jj(evbr-28 zMG0*`)=F#Y%d~1_AIT93wn1HpAdVAb9Odx+kn<-(NFmx`;;kr!Cp1nQ4|M;}kU#+X z*O2BiE@$MQI};)yu0k>Q?_D2{YN%%nih}7$uH#smXbkAH$f)usno1++RloNbek7sK z;-g^HVpfT@q@9hZ)PC+}94aqel4)FY=QDgQZIf;g)EYiu|MPq;wTRZ}fwkxCf7hS? z^lZS#3vm2T@%ir}D)*l{`v0Sd3IU6#96|99wV}~ov@CtCq^$Fl^u+N|g1BUjl(O8! zwA7lL+$8YDsZoM(L*TZm71Ah!7z7e5XI@_vtdqzqZtrNXPL|f_@Mv#8@F!T%q(Ce~ z0|V17LtTId?4>|LxIf&#< zTLNe(%fqpgZz$U*G}NiRLeV>GcH2p6VUd|>NygO3v3^Xd^qFAU-d?StpGn$D4Q)B6 zI$2EdUnisal(yuYCV&(yEDX3R8zmR=8fl_kf!0G&;;v6zeTqRP+-cEy^4-Ou>cKvT zgb{8~M45o7>}|zDxbcjsqYx>!Ba{7$h@-{aRwBX?Rowts10bQi%;5 zi8=nW%=iKzdM9W9K<^%9wp^XR!p!b%-Negxy!qPbxeBLlDZ=#NGq3&It`q4| z(OzByE8yNLMx+ZPTnj&Zeb-R#-UD>7aLjp34-~6}aVTDsFL^a2w@u_8^zGp^&rlyJ z);JI6d-%<$ZzS^6T~0)S*xvV)+f(|4YeMY53TD?4HNz&|zWzENKgc2;Q4(`!ZbbGv zp+zmPbAKd%F!FRk{Bv;CF8qAkfQL5s|8}+d2KmjZ8W>;juc=jBJ`_yqu01f0`yVR- z6?L>vdt|1uLnLGZ0dz7K%Planxwb?r;b#cJnD z%{8vAR@GW%DGE*dHLg1^E$bbN*KW?1 zIKSh5+bpOC?Qz}JhyS}lIF8#7*h55vS3E3;_1j=-?|a|s0UP{5#`5u=bWz`UWJk%AzBdn=%c23ZFDf=L7bxi|t235PJCpn(*S zit?Y8pc)}gJ%PGHX!zF(h4AmHW&Jf&6u=7{bVtbnt-D~?m7{#u9JrG70qq+IT)dCi zU4g#tcSqZd#zx%?l&5G*&AA?M46vtqWxiIq{fVROz+Xeyma_W}jn_K`jW<{gjn|J0 zjYsAIR*J?0kh{7 z^p&m89Tmh!5D9cgK?IYpHjJ}P4hiOvMUfs38IGeOjLb(K3C1y#DB}(|e%hCcIu}q& zEi^yL=um*W6XUm7YLhy~Rkuz-}WVziS<&nz%^x*(K`r`TX;BUFOb$ z$4cZ`!Z8|k*1qNm;wwXu-)&h~Gv&3}DehjUbi3&OI)aFK?gaCih53#3@g!(ws z!Kbog0L+=^&d5}mPp#99yGZkQFBUPXkf>i?NKNr!xfYci>v+5c;&A+)qQ*U#cq>Gb zH`<)tNbuzQMNP(|VT!0)_l*hpT3UXnvFBPGspOQ)!$C;Ur>LnzZx&;-6R@k6dv^W} zErL>^T$~PXbyc+5%*K?NxMJIT0FfE>H6k z58Iz0+CP6X4;qnFH5A)oF1LGLEIa+q8tYtx1bQ;C5d*WPw9#^Sy~*jfZEeL%R>xEG zZ9MHz{xpXr=Vlg~rMXjH{aI9zZ2n%EGL}V_BTgw=`}4#ndyxR{W>S7H@rKA($qaie z6>yXKvvl7l;M6cB4NK5&xum}_;!I9Ac=7h5xO28iXGnKelPw#DOIru4#(c3Tu_}$M z%ghT}iQGJAY`()F=jCguVoTQ}|LiAhDrWkK_Z(%iy-~-PQ#GNt(Jd$GIjQH0Z$!hWt{cDePB>?<>OwNQXmkpJb@r%~ez>BNFu3!@+vsnB& zW}C=^lg_BKgtfU1W^MVO=+YFljFbvX{qsMOnsoL9rK{U6zFIWKN~JR{+0;s1y%u-F zVowB&Tz`bMXxM1@YXZDzMt(q~f@0!Z-0!aDhmHtq@qEqon_7ibNIoAcr~hi82N=4T zUmq|1vuo4k)SCS+H)*}WtJyurBHSTnZaBmR7!AX?djdVWxD0LZWzK_{#9|`)(;~83 zD()Ufcu$r4uAo901J0Kfen2&d2^l+ruwgQCJ6a+g3Ze40M@(4a{9&xrqYLMjBk?B<9b5X^WG$@*++@^3VmshahL|m`$)*J@kwPPEX%;Gt1(|N5eu_tBqL`u|!7Z?g;@+CPK+w>RdatxDcX@9(;JnxINKHP!cRlC8XJWcww} zglJAp-80IuYHJ+u0E_$}xfo6ath@R{QhYm=LR3p^6k$i;t<=0i9grPrX6JSG7IVYxiQbV03i{~oH7p1hb#NcY@YFt zl|kF>>*TYej=95~7}#`e>5T4pH7(`QY|Sn$yMMJZ$mEvW{k1=Rbp{l)InKA*`bt_> z2^~7t-QjWvOOPFor$=OY0DeaY{!O2hiq=PM^1Dw_Ygaf=c)z)&Z4bcEoygdCEmGDO z%61FxFzgPi0J?qYw*)h)oY#p!6WTQ?yUdF_yy-^PtagBGrLIY;R(quls5Hkl&}?c- zm*!n09fw-yN*0E^?i9mVl#*|Vb7f*keP{#ogl<3TRhSDK>rlczj(U_?NJ-T!=@8Iby^H&GP`!j88r%|2(xY+ zX@2NsZd#2rk1zQme~_m2>e%(~%)j`dYj-MJ0d|NSu$VEre|K~R*PBsWN0Z>w=J2rF zN*Xfh1-=xslbn$+%#+@Xv(zrp6;wR$DW^4CA3@QKPD2WQRYvd{hTDo3Ux1zKv>M5| zZ9)CrUT@#|_%}Lf{U*4i59-@D5t9Gz^?}X*|9mJ@b)BJTfKA{J5A7wi&sy5|wfjh8 zZkUZV#W3OxiiY3F=G;(>%_wEUWeY-{DLFR&YAixR?7A-V^Xc=V+7i;1P)U0N`Lm+Z zzyYE;-MnDByb~T?C~d9;v<0@$d;PP`|MY)3-aC!>9{bbY-#nITu>MLaM`u{gFM`U_ zE~-?Pr~$03l1tRAt)feGGPEkonOU3Vlx;J#YAIVw)>)VG%jIOVRU^rGD$!wf-3vf# zoeQ9UV3K8(iG}SUlUv2P81dyCkTjf;+>6T541&wi^ivFgggjslW6+s+&8Avb|Va71%3_B$4VE_h>U}uay_y%Aw^hR%q{>RAF*aLik z8ned418txZQ*Yo_D`aIwa3G&+ac>Rw=aL}Mdg;l-6Bi4yOqjT4))CD&@4(7OdL;9X z6c=HiD@5DnFA`w(4$-&Zz|bdck1S;7%md8*+#{)nFJy+nM}LI>))@C|u+;T00J~@9 zjw6OyXX2I_SDxuj(nmF3DMT4|Q9CmCVmv(d zv;zL+RBD9Wl@&pMNtL%duTu22p$7e7h4hu^J0-{W9kW-$fmoNNsF{lj@STMvr_EYw z1k9oiPYndk3NOd#nm>2Sy0(Mg0UcPq?k>&X`U^{HKI-Zlca*w7TT_>2l%C}!g6{lX z7jPkQolgx8bk($*h@gVAqKTCQ*7pu}Ea;~i508-EbR-4}8jthHiJ!1(f<1H0%Ioac z8?!6icnW1U+WbXzb+#Y3q{B!9C~O5NIUfh9X`hP%-5xB-kfP)gn*#eqfE`HeoR2#d z3UpFMYdI0NP)n9@Uh~X3#l)woIIqwGkcTo%3|0gb_rIgg*!* zQN(6MV^kr+sTroT&*C347fBWccSQhtZm@{s78V;elH3&Wzs-?UvQ3QQF1 zzg<9vMOD&V={P(5bvEjuc${t3{#a=AjeEuj;r2U z2S~WO%gq&3Fy)?@ks>kTy*7A-!=s?4&!|?LPkr+BK=A!O87!8fwfB|eAW?%f=ZblW zNlS)iaoCh5)dPoXySM4wbrb~C?@AU?xrM9yyuVZ$R{FX+IT58OBDoAhlaNw{Zc}^} zS^Mj}nON;S&(myG8UW0+B~8X`bJv#NEJmhBJ=(;xzx_IL>-l4Z_^LUG{B6(mN16o zRREhYMik4h1bxVxJm2To8#Oo}R`EDTWNMXDVfmb*r0LO6i|lezdz@ShY&M(kTe3f; zXzaLIbi0`W=rEZOxxuWNAA1TcN$V8~U2C7a7*XId!EXckW$B*qe7zwQERq%KQ};2w zqByoDUo{V*?wzOg*(A`-LFSqRs9A;eF8%iNeWJ{ZSpEXT`oUCQZE*O?Y39u}VE~KK z%=!a%Z z3N8MGO#blQi3>*_Vj?cEFe6<$R-}3utbx}OXX8wKtD_g|(u$jRA2hHJtamV2g}sk6 zpfL;r3~XW-M)m*_qUv7!vT<2jX?NAXw|ucCia9Yg7R5*|Kq`#3kLiR;Xi~IuA1EqQ zaM~hVrR5pSpT_;VQna9vAP5HUe@AN|`Vt+7Rb&H$AJ?|Z+BB&ykJ6A6r<*eKXSUKy zd8i@7id-Ll9zjOIBbHNS;LRr<;kk403P9fhps*uLGVg`Ec)Fzv664BcV@F;x9gt}ep1Qlyo_v| zNR}C)45nE)Ingblh-XT0MltJ!okzDA_rWhVW$MqLVAi&ljGr%^)!Gl|v{pH*OcHVi z%$?AAu~5Frl*}qU63KKo<%<1M*U;-wqNqwMPLlYXHkk$3Ytm?)#uSjcBdm3%x<#wL z==j#uCS^!I)CI_K*e-A<Wz;0;y2&*J24k&H^le7?A(^f~z4O1E!qtn0%IJR+k_F2D~q8iWjM@a9a%kcakN$ z?mm{vkNO+S6(XL_^}3ziAbY6i5y z8aqjwV|PXT*Ow`?D9f*gSi!AshIlB4vE(F|-6XB3P3%esm+sX18u7-0@O%{RZ4|VA z1CAEyT3fZ6HyFV!JNFnjZJgEn1^ES{JAa)uP*!zn))`^)(Q&p4JK1pqw#hlzNigqL zbl92{CakD&62`S0JKu{uXs3qqOxpXF+T=GctVxPhMWF{E!gazoiZ!xH9dVgfh#gP% zu7nW1NUd7a^tnmTiQ?Dk6@NxT*d&!{-^%%WwTR#j;wnMr5IUZmoLfXzt4*h(*KgaU zQjMs-rnNOy7QaFa^!CsM>#zOv?0VCl-&UB_?>*u zpqk=}n25_sBrl*AU0k+fQVz%UC)y9-#-Kn-Lfwy?XbyUg`GTF%paYg1@V?-pW%Z0JYn@wj z`z63-y+c@b!#8mO#?94RlIt_dF49i_2U+jHUuWBOd$&oGG;XXWX>2ErZQHhOG^^2y zZQHiFV%xUaq;cMLUH9|s{r>j-70&a(e~x3$G2ndKFuSyC4pZ~kM7oN6Zh;wO_3<~* zZG|Sou(x$-tHPHLzeqCi(MM%fjnltRsHU=Q&7Kd_db!twwC;gsFm&F|iH(;u8XA`H zG@269S2jg<);En*6&x!>6CZ=?uYBGVjkB)_md(awWU@me?KKV&kpd?jkmYdoT04S1 ztXO}?g=Xkm5T)1Mb6RUWC|`r_2mtTM|A?xYo$vezU(YVx{GluG8;=$8XzX%RX*))tDB|%?ywVs=g&O& zf%>|%zWhdAmTqw#RhDi+C7=kPW@(jKBsVwjCCg@Q6;Q;JW;Fvw>{V9RwxW)-)e@>~ zFj5~ZsSJIe-+OYd`9=g}!lJ%5!z#Px0S3@>pcsaXuQxfTPmGj$X~Faj;SiU zsi`_-Y`MnQU5F8u8tO!FaV79AEPvvtD%bK+Qg?l#RB zrUwoj7eIl)zpxW1C9s?s+#Oic5TGiT2kfj zDlETv^s92)71X`xV|Qu9>z-Tq)K$6d1Op1*@be%gP)@mu0ve-HHFwXMySdviqaU`S z#acqE5BB8LY@aaL0Vpki5MoCz)_+R?#_mfN3b}Qjpx=Fchll|%ksu(uQU73vw>WZ| zpNoE%Cx9apVQ6qzNBU5uCj@)5Gt`BHfMl*NBh)8q+J@QNJ+;;_-ZibD9I`gYvmXH| zJvgLR$fRJRfC-mLT2Fo`-erSMcPp6{DSZT88Ue__G?sG_36-))J~KF;l8OMsK~3Q?5M2kqt$|Dt3Fl&Y|PV-GQnogE{)}&BH8u zYyVUn#ew^@lJ)KGEoqh{>RTUOk-0(|pz+tC<^k(rRI17T;wsx*gAtv{puvDs7SrTv>VLUI8zGpZba~a`-aAf z&_aD0w{f}r?PGe|F(|HiZm-d83{KLhaVsm&p_Yx}BHdFP%+$pBifI40gY_yC(9!zl z?YIi9d0qRIO#qFuo?(lRdAwN&&gOeatQ6wRn#HgThNo+s)Q_vkq@Pj0%ve#FW(%7! zOO@^liXOG4>Kx4V=`4=;CupPrj_=)5qf0)g@8cCJm~t?v}6`_@s86Y*@M6>Z~2 z$&*5o+v>OkbBZyhgsSI;=|zx;V;pMbbK_!TI~1etJN1NIaIhMRgb)k^-hcERdeGVi z8dC+O3Gg_&=&ySf7d3;~+7>N0Ce`Id?2Q*cvveYZ*t~qrqpuL_% zU{C<2)mUu2WQt(TGlG1Ulz$3$VhMC7qz|3)~z=QgoRhWkiT^={~tA6AhynsPQM$MAHE}4t{ zS`RS@6YPw2-bK5f$w> z-+Us5TR=pr*=SLZ;JtJ}cyI-Kq9V=lG1-ch!@#(-^Ni}yF4G!dCCkHU+?tdUzKGrD zv4kBZ6+55!nmoc}RK_d+V)MdmI3=WDDpb5V9X8(WRAc*i8heY~l3c<@aWo{2`#3kP z)@Ag<`U=7I$^YW%VKVFY*7+Z0qKZbir8k$(_6x{{wahdL1uD&|sVQz1ML|>ABhrZ+ zoV-GX>?b~%=#daWAj24KEt#lRPG23W{iJ5G6k#bkZdey!hDK_{oh4{655_30@Bsa} zqi=K;UpUtim_$6lohkV7JZjXKJa9|XfN49;O;xth&{RQxg5_}b!SB@;`?*dzg#COqc6S$|>@*PuM4H;r-@Ix0GP8n2>B)M~Sy1%&^A=y{Q z#o7?HX?Rep>t(X=vTHnLOA7b}?!c*By9l8jsO&5I7)|M_S&}x1CFVOR0vlwC$yknt z^#X%yH%oC8RW1GJupy&KR2iqyd5$DvbtuVO3$0NLuZk7}BHA zu}&EM%^3au2JOc z!@i|`??L*~evMzx!qxVM;DVCn`$*^p=^iUdJCtudxHlP0c5Oo25U)^6+R0AgGH7iG zDsAIAq|M}|-m|Ej3w3S76Sl-}@JyPssGsYbvKE`5Pd(1aPA=Eo;%KtUUc*{qT(XA& zY=wLK74d`CT0HIsCt0z%P3bwDQMvi|f)A7#?mI@NRw%* zeHV7MUend^#RqEU=QdjvEX^n+#&YKO#?G8rIA{npA|feSmDClaEo)ff(X2>VRYAAO zcrE3PI$E1{{l!vD2zR2H5=RDBBC7rcSmjLwUKhTn2RjSB9auoe|cn+UpL!fB!0DlWb(Vphoqi8~b2)n$$P zd2jp3k7a6{T6Ur#W}#`QEyH7wy-$u|16~;Z;dMxw{3!=8ucJ`@zY_SrybfUbPvg%2 z6~E6$ZpR#&GBPPcN!B=;&X;RhBr*;RC=_5y=4Nkcc~>AR!N6`oMW^e+poP1tq@^{h zv%*LRxl2B8V=5@WC4?e$pE$k&t%8$gNCRQ{dr4dZU(!$;(rg>*s%|s1m6`oh z(N~`vp<#nWW>m1JBbZsBUC%9Fd2FaU+;(nD3%5VBBIqhAA9yUPO1fSueB^?Jr3E45 z_)cEK>(UK@Wo-!lnFT8!Nq~B0i>P|K$FiB5JG|ZOg4(N=`r5UsuI*4XSo+?YS)hN_ zQ15)~sCwAeZsS4*Osf)h6>mR4gMvH$gY7$!K;wL>>h8{gbBi?tN}0dp#ApbY5q^S( z52*}#|H1Yb5$mBCfbP(9ZjO-Y+p1m==e_j{Jqr@f1PXxU=X!_zF*8fd_DLHy1# zW2$)^Ck7+2*j?8@4nnb(hLa+E;ufr_>0oaE9PgSW)!7N=_GJX56qwLjjeofvok{wk zcu(^5uhiR1{2&oas!?dy^??~q!>`&;b7`cjH8?EsJD;Y)*0MZU4f~nv^{H+^GshNb zqUnH_KB}m`Ffh26sL2Zb%k4Jgnu|>{4x|=NI!TFnrb6;K4zWQn=-=-&JH;!n^1URTYl69=8SR^VnHEMQdH508;)hmKMvs6c{jX6gNInWTP zRlWDzY#iQnmF~!E|7Ff0M72p6jnnJULme8F&k#2OQ-MCpu8mhwJ*~GML_lI@dz?Aq zp^~L!cZ$X?*yO7<5WS&EdgoE^@V72+0|&~fqmfl~lksRY;P%X}6p-m->4`c;8sfWy zf<{6axytuc;xy{HX2R?wsK>ujV+3V6I}&!nb7C?kEqt=$gPF*;%O0LTJK(S5f$1hI z`@>t1iYes~oqFb@;_@Ccp5NkhO=h(CRa<2xRKv=k%1Rol5S(Ic>pZ535EpN$jstgq zPmwGmTY3&j@e||C91v%)lq05v-ntd*Ud24B{2`Swjah{SU)*GbHl?3ln;^van zOXGC#J!5wIw#u=6O0Io=gO-`Blu94Ud(=~B#DA)`^FFmyW70%EyHu`9#xuz8Hd8i@ z$c*)`NQ6pR_NJk*S=lgrasq_sRj-S~IeUI+NG$!Y28q-(3IN;WmQoxzA<#*X8^Bgg zH}LdPr7HclQr+M247qo)wtEv|d98W8iv2ISn?W6kfytc$OzviXg23cH;myyT$0!}A z$2U@qs<&(L_R^%k+zcvLki-nK>T|9zguC>6v|u&>i{kPnKmB{heK5+7UAde@4AU08 z^8GgT0QS^^^e_?2Q33g4fyaL5Q+}QYG!j zV!G=qc{vj7uNYSY&V=99(tH_liic$O2G=$*e;DUPt}~ICH4NBkUo1}9vc?t-$uw)( zi&M_e&N4M*?ON8w?@x}8!@LiWps|J74vNf70JJg=4XI!H-GCJ&8RW@NHSBVoS|PKo zTiYwayRin@IJ-o-9K6r3{;{;gLOKKa?lGDE}jl8(Xcfn{<`} zBDCV8^s*-4+{ZhJ*}2te3FBfp6D+8_~f&1Qr|ASpAmQv5^jKgG4vo70Jm zS>yQl2{!D=Ww>gM{Yc)x19t_fKWba4NUSb%MA?i_5}FGec^kUOmecI*uMi$5-1y1* zrD*&*&2KlY^$R*T=Pmq|MxB0%Z5{x^P-%=g`7NO*zS)Lqrd~ld$cT~pn3|POFet%c zU?zD4Ome9{TNQ3oJrjskQ~mFb{k=}^$8o@Ub+m&Qzl&L^{yfeXNwN0Gd|h2LLWs=liRrynRLteZ?Felq8pY*sDvKWS++qPMElC#xf@ zlr6_+CeA09AO9uF!5il(DF}c+%z?uw5u>AK3jffA&S{#p!&hsMeEta0&b_91{)w3S zF@K?^91hrnbXy{}!f1@*X8hn#zBU&sHkHm%yBMVVSsz6(x(Juue?}YM42`_nQMm6* z)|ghE>xhs}cI)IbOwQ1>QE?k-m@abB8w_=77uwbk0-yecc{6lsbss>+S`Bx3#};82 zNy>vvh~jhHCY$57xAD_8SGF>vkaU8VbJt_SH&iPK59*Y2wt`?N^QUVNK16f1Va*3R zeaY|rOEDj*Lw@l?YtA=VMeC%NddGlmw_t$*v_CaID?!E@G$VGsvQTx!OtNt62s}SL z*XT#uOfR`YxnZtQ%@zSvtu;7=iDpg8yJZ>YsNUtAzoM8}CE?%D<(Ah?9i|C;4y-jA zJP^9ppbk_Y3s_q(X7{}uWm0sq_xrn@RushP#?&}SN5D#Xb+Xa+f0Xij4~g}&a2}m)%d8cwGtPAq!MWnn4ePfnI72!8zhP^ z4%z7bov7}fEl}vas`hW zY}|s07L4xXKzB)kt3H?@i3|!w_w7!Ie+mRpXV5@?ttEoC=>uQuy=rlxV&+EFtuJM=|&>AoYTnO_*6(h!HY=~Wo#}4eI4KIb~`-$1JVQg2E>FT zf^8(SU^x$Pa1dsPBkUxycqeCsffvYCr(#B%Xrk$aP&HW8Ui%2WK8~AJRn|)%Tue<; zKe9J`+~D~=X<8;aAc03UX7RI>*ckrd5O=yO{kZYW?SJa0i-9n-L4e1yBufUS)E_T zPZ_YQ^_Nt6mk(}L%)v9c0 zsFE7)t_G!6Nh61@2G%p+xR+f}H~XyeuPNnLg(4HYXjzAr93bF*A}O|S*fIq&uShX5 zlH7v;PNseQ@BhvrD)eDLw}R21mFoXR|Nqna@+ny>5SHP8_4+LHF)hzVNt>gD%sE9= z@gzl?CpV;B8m61r?N6#&T8p!zn8#sJG1KXoQda>Gh=0CFX-Qe;kb=c}N?Xf+BYm5v z)AKV|bg7zH<1pfXzVpbZ{p|*TGZNUZF1Vuky+4FMP1Djh^zxI{YN1+_)~cbNk_%* z?PE^pMPyd#ftTnvsp%6$g~hSCHo^g?6Cx~ZllaAuIh00#AedyCZouZJ@g5kV>hUeW z{zg(g;ax7&YO)(IRQn${e2;#gR&c)JJ3WKL4^(=H%}LQ^ zDKa;k%qsI6r7;wCB7)~6Kf~XJ=MZd>1okG4sNe*r-I{P_Se zJ@^Z)?tCz%m;KFf%xy`QAME2>3YIHGb?lEauxkBcja>E4{O!*)qMU@3p)njjL z@%+_#>F_A){(9gUei(EENXtNnJ0wJvmKf~s5n7|xz>gtq&>eram92nm8A_^%5L`7a z^*d7Z?ThuULA;iVbl@v1jTPX*vW(%Ugj60UtRc=&v3W^^Jjfb^^Dt@LtlS6h>xvI-M?3a zwojgl`+lXJLfPZKOcHQevG3ThIp5i2(A@hwsu?}_n)|}XSHZ5AUJZhtCUUp+%gk6x z3cjG7%Ex-X6eo<*jO~fU-WtEM30}bIrqt)g+AM>o26e3@wOU9Vs zmZ8JNUlfO&aqRP$E$y$X+t&L}c|ejqE@F0TX)XyF&Od-K9!+v3>BpUP=TQSL4}Q}) zsNYXmTUai&Y?u$}p4(w&VJ&o2`~L!)hH70?HM#S#V{W95pRjC=BuX$CbhwzKX~lfY z>Z(bTiP{F)gU84)J(Tv8ejI(qMUHyYA|+5$9*D-IS9UHjd(@6#{xH!WayY6wbRA#> zH_cetal9mIEIJqzb+Kc@TT^ESHC4VlP~uA2Vi{~$`PTdyfJpl-3htRPHp@0uYhu8x%^RAqIV*RJNTx9z%XZ zd7VW(1_LujINHwD^C4m8tgv_9$6v6NhVrP`&}HFjUW{)@UW}hfc=$wdw!G$gbCMi> z%6o8YuR)--J|i{KO%wcS+WrF>ik+q@5LsWj5G@6uY0D34l&D{6N~v73xqp}72|C0) zt*j@(2H4nLrdSNMSnbocshaxasFRERVH^a)9~p5$dQ>5n zym?CK{@~Ub4ft7b>r5^XG>->@f%rh@GQ31&v7T-!&1uFn)3H&0tg)kQG?h|@Y(byL zULqxhCxeN=SqO|{d03p3L|~3WN?juQyxJA4lH7$bu z0Oi+gEDiGtk$59beSN*CvEtmyq=wOK{vZ=v8imVo2nB(hypfW}t`%`98BZ*xd^e`b z+|K-_MuC2~wTF}Vo46SHh@DV4$;xa2?hLepVxtB1iH)5ld=op2meaj$rf>D);Xsf+ zpqirYPUf9^0={uBRc45((PatiK&Ote7hz&E5OdnUh0QABYOh7cio+$b_8&0hF2kJ<5KPX~7@?-ITw`z12&_rkzf#BVm|c_MnsdX&dA zI@qA{OL~oJ7)Pc3Mm^R{U+T$}Zpn)rz$d7Kh*1McVAZTd`4KC*5TgrrJZEZ_gjH$n z4~UwapnD*37b6>n)PpIak<+*u2ls2);sWOkFr7WI)OVp&Cg{%16Pyj0>6hBJn9~ee zD^y~W8JJu*%G?ZX!fY&D$S$|v+TEDNu^6BnDVb9HWKAuWtVZi!n$m^&)8;-IfR-Bl zoRx@Fy3-(r7GI4}SQKH7VF4^u_-v{G%Vyw*`;3SDsVQz{b-YQ*-W%mFRtw%2p3p7K z-#7B>`RohVgpa>h8d0_VDi8EDd^3Jrt0$=Gx+AKDqqNM4Eao}RCbe>`H=#P(Jtl-| zLmoUr{%FPBm~-HR+5GCBx~*E24#23gJU+^4-vJ@~fj~N?-R?tz{wJ?W_Sz~#9p}Ck zx7d(HTnC2G&}Z2T3zC;`(^}|E#=%T!nTuAYf~tPIv)H*aDR@9f9nCDt3X_PhAvYfZX7~``TYBjH z#5wJ`IaWn^nu0)F+q~oBZJb!o+XQxI?VL8ENp6j;3l;QZtR5T{zvsx^=+qG5n|`YA zjjGOVcLfW!Z_lmoL${7Z_64VmL^`cJ*!;w^swG=xh8J&**tpDG@AzqMtkAIDLwC*T zR?Y5Ezy<{;j0aQYAKa<|@Z_{%i?%(eX}snm|H}KlA$g)&_j^NDklkL+nI=q=^EB_Z zFErP<;NM;nov#m0%ZV#oTGq3-Pj#V{&5$|=EwBV8$!=)o0LRY`+%WV3{0?Z`(QEn; zMR%d8gZ1cCgH{lXb&pld;LRFdz1AX*~*E_ZV9w)dU}KpU)xfL}b%M3Y;31Y?<2 z+^zVp=&vZu;_^`M$KuiLBYqgsod)N_1QaX#u}&)1qx>c|Tu{}v5adyV^V|ED zY#mjwS=YW}1{5!V>zL58@0cb|D#t3;5}J0@W0IjZb&&0@U27rLP1?7v+W1PqskEK? zG#|Cycp1L(5py!Wk$un%*A~akf~)#-n{Cu6LN*=9oaM=;U)}N+9O{R6qnhT!f%_CC z6m(E^yMC+}JfoXfiNMY1DA6wI^^Mx+0DadnJ1CF!vOH%55cyKEKljL5kq&%bC+j2_ zN$R7Jsr$@QpZ2Ga3-M)i8is~jW?wfp>|5JoEO2G`(;JzoOT+=zTz`-@UKd6yip1=n;M);eIufS#7p6-1y(;snY1h}VD_=JN; z<77LZ)jpqX31XTP({0{ZqT@ct!E+h}+N{Qm05=+l(tn!N8M51MTP%vvtvEPayE~g* zYzxKw(b|Z#4sJXrwduD#<$M36@*?NQB5bzcp0B;73BYv1bvH)4=#8)8>#l(FfFNLe z(sqDRz^QU&%leH${}$nuC9%T#s1W#vyd~p1n|mWM`rlh>>!74}xr%1*a&p$LBDw_o`@lnWbIW-u z9bt>NF2Cs)L>KeT-t|8J>8AiKLa{ut*b+xlE1{6=1tv*}-{0yno40bmveL&{eliHIr2ZK({eK3D-@{X}(tEcfo(1JFoF4cy_ zy#ai%&ovTXkAC`)HL^yRvf_0Ln*mtXu@FdHO9$M(vx+vP_cK=Lz$O_!!6B4FMlweE zfSqs}=GdxTA#i{m<%N84E8Pmnz}7o2n<1%A@C9VX_p%+X78hjLk=`QMxG&{x ze+Y5V<|9uZ4hhRbi}=S8hXB#dsTJvdO{4t??N|-xusv3>A`&6+Z^6n>6u@6_M;Ly_ z%RCmSFpn?QeZ56>HI6otSGeqMZh7voAdZS6Q;dKmC2xtd4RJ1yl#p9)|CpvF<(!w7 zbP4?=^;YlgORIg$;5AH)Aw6~xW<{WPz40<*>ukKSoA9776 z%X0Tk*ttf(`=Mpy2ku2a#)I;fa#LgUGl5rvI_; zP@Iv15JczQD=Tj*N;OR)OGLJsIjlz&*^Wmy{ljNx95FsQ+D0?4PKGUrJmXA=F;T+c zH;b^I`pkMA?9!p~Z3qbZS*{;tpfQiKXJ6k+Nh$~eqA3GL6^bf1><`FC5r@Kr4}ZKbNyF+DIqWGpnQj(Wf3W~qPF8;u)f?Z?xJ?y z65t3HfymmCHVE#nEagjN>{4YNO@H~TEW!T@ZfsT4-smx(=S`o1HFDuVpBjyXJuDOR z`3WbgU`d(o*-L-NiScTB`HMrIE9+Ys&)Jt}0aDEJHQS^pJ&H`8P$7-RLzrOajL(R%z? z>*L!2-fu)!{0c4&=WXtmv+b;a7d90}Lr?=eP!h{jrWJe01fOi&BV~XN8Q#J}V~`CQ z;9he~)bkzGl5HYZfDoC3gMVX8KwY&JIMpsr{BsSVFy|?3NAVk?QI$ta`pNRJ2>-er z5{qMA!mxAh`8_kJT`CYi9&VH5QOzN!Nzo2hb0ZXNcEUp*$qkZzvUH-vE+Q@+&|5qU zbJ@CO9FI_7tk9LrS(B_|#cZFQ^$_&}@WRaMY+k|t8dk^+_nNBtV=w5Ps>Fa=YUw_; zIgn@n1RERXcf*ct7QTWXD^;~0EOam$lJk~8817;)7V9HTSDtKTw%pM$^tV;+&za22 z0HKEH7!>tHLo6=}-wBe;sPUbt3RNxF+@EK(CRtto`KG?Q}*KzvU7 z5x)|gxypeoDW_HFpFUSI+9xto&Sz&B)r)FgCr z;6Z-A+aEt8|Dkh%FJR8g0cdsh{XYKV+xny{{oCh%j_~p28>u<)A|U&(jro5SvQH47 zWRiQQVM&v7a$(`Vnpir!SlSu8B+oD);{Y<_wdKJ}W9VapE~!$~F06`}QcVLh$&hIf zzF&ll$8knLV&Aq2VJzmv<)Vr7QT7qyvYd2tH(-VWjX&XL9FutT)+jzJ^=X3h!`X}L z?rtf$`V-h&(_mI`r8gnK5(Wc?1t}#!6J`OT&2c3th=$(gPy2D-Z>9ixxZ^uUe*h7( zyCVq1QRS7=!-h<@{=gV)2>&#@UlinI=^1_HT+~ukf7H%%C`i2fr0h`HMB#7Z>onHa zZ+W!9T9J9isdw5lMNHipzjw%)5g06`UF@1A8jGqEukcqWryywr4IT?KisIqULR;fl zH%ZIn!0tMYn0dMs-JMrzPk>euoIzhnQMs!sPOn`*VsAb2tUPdD#7e5M6|#4gt!BEdNv`CnuW;X>Ea^oriS zq7Y4vA#yNo2_8+2tw~ z7t@W$MQCz>fXxifSr^q?5j&%hVaiIcW1*BGF6O2yG9UElX7nJJnHI$?El*_k=h`=` zMvLws(+;<$#wI7E!>@gq z#YBbyA#jS(9^-82HE=Dg-ToY_(s<`fI|3f*C8v(#+D!s+>K`3#H!i~-ZK>tR9;^;w zH_7tHG{aupp^$y#3L*(Q7vqrAOOu-P#V=-xfu+3SlIqaWB?X;3MECR7@W9kImjS_L zOIW}Mqm!{lHf7W1wa^v{8%`@Wq$8P|iNu|q`mdZC+ZA}y1rfvOf-vq>FB*?@L(}=t z0EN=zEEmbQ^G=dBMG>zqK~%&Y>8kx9L8~$2)Qfoag7Pu<0)~ha5zxEn=GkIR-K=K6+cPr0W|m4b1PYNeVV5$EdT%&#JC-?PoJ zv$BP)?fx?Tg5E=Dl}CZIL!0;}eoHh1p$0U-2+RJ_rSTrL&Z$w4{3c%d3bmp{s4hqT zv12_;=NH-i&+(-&r7!sfylQ6u>-b6r#;WAYljyxQ=rG z6E(Z8ea^lkkOTKa+Z)?DB8M@(w_L>t3>}Q&ExWyvRU{;68@%T-a(NXY{^N##t4_7a9*jERLKqs30q?)l>Zqag|O#`|tKtA%#*1#vV_tu8f83l-V7(y?Zu zoC#Q8Rx(b44Xw!sO+DD~_MI}1>vh@-U760EOX2R?>syCHC$Sf$4brShj8QspOvn#n zpWH!H$3brsxWz%^xc_@|W&SEPX!M|cCL{+$T+v2j8Z|Hex~8c8p0;|#m< zjUHOiWJN5++a&mt7zE4{1hbq-RsQ`jt!AZ4IR?}d+HD$4H!vZxvV}UPOhqv<8t!Bz zkxA@q*z0)4F8h+txv2Rl8}RXn)`OkIamd-V`?Yws~@4zK#81(55U?w;>+<~69^`sG=c9predgHI1_vHG9R209Xj zeytnLkR=Xt9?g*VHL{8Cr82OtiwT18eX4RpGMM*piyS3E2zFK}1Ez@1!`3a~ z<+ztSc|cUYIS0aKR@xHrQW=b79HfHaFmPg7Y-3<&0wA-JCl1Ix$tY5t4y+MbEYZg7 zWI1zllU$u;iaalFf# zFCIa_0o-m@OY^|3t@Y&GE0-f>%}>jLhN(>_dZri@sJ|NyPZcPk{rC$BVI13%$meOnOyBSow!=7-CYw#e`SG z5m26udC9tOqLLbjEG4ywl@bepH29%`JUD$?YYj1&4}5raj%hak%t^Ml1- zNG{GLBXY9h%De?Y#tm{JHV?}UVf7w19il5luT^Oh{kLUj)0eAR^zP&K4ZHw^2Bhj706yWN5vSfHK=gk3&-kzf-Vn|z2TGs*bkUL<<$~zAr)=&QT! zc=aLQ^?JBu`&q;uKRAi0B$z$shd;FsB4KAwuc^hRu?;>~FF={gFwQ1HttQz-;)M8| z(lZNPO^blQH*z0Z!xIrk&rD2hCL2e0K)?)>vWa{}n^O#;ob0oxlM!c0a~1_I~}U`fG?CdYqrzL~57S2kWQjq!NudP0A@ zoP#lB+{4ppknq5xiSBfns}R{d9qjCNwf^vOb2n%HiNs@^3FMf*hcu)cI`x|A#XON!@U?)feC5fiPeDQS_-ZGdI=&Ph|vjaEwpquOxb)`8~Kmj?_R21Y+ z0&rSC@YzltZlc@xXraz#{8NaR&jy8UZMFPKGfb#w`MvcD~)Rpah%{rt??C-N-8phdZ9n?f-4J6Zq|VO6tG(S zW;!yh>cGQ^m6&8wQ?ko=Zu_-2#xl}o5|A*uLlYk{bxte`&BqP6Q#2ejnZ|sjtTEkG zJocK{_F?-X988(Kp)X?~KqpC&0OMnD z=n-&dQLO0N=gM+FbslwLylatuu1noe;>ryJyJE4ZG5`G)8~Wo#=&tIkefa3zX;NAf z4RwOF`~*)($266swN)u9bRQCcUnR|z@_X_ztK_=)9mQ=pRTxQFBo$nKedOh!!OVr} z^C_v43V&0P$zMpLj@;)3Jfjg(KxQ?|Kp$6IlS9T1&0`N&8^00YXY9d%&AP<+BqEaS z4mJo=uKybGSz8A8hVHwmNBBO>`LAHd0Gh~Lo2dSpNRycmJRdZeI{cLfTMNzw%;;Qg5<5CRV1)O0xUEQ+q4%c--wqqf*FTle)lwoB29CxyS!b4N?3iXrimo z)M&0TG@R>8iX=|5NlXY~AImYa0+qtYNdJiX#78kN=TnA4 zf{h&=m<#uw^bt(?QcKLkh%>OmDs6&eL&(%`$b9QGb5r>4FG@ z){K|O%yvcaaNw@iv(r*w%wmpP`(W|AMA*&LOeBz_aKDe@1zOEZ5~de+c}A)(dsPuP&G2Gts$8Kb3@_7ZYy#%)jmHb}(7d0kv`W4y1r~&gos>uIO6ajx* zfnW^9zG>RZN=+)= zX?5MSmWo0)`m5q@rfd&PkQ?;nMymH@CQJR%($$pyXP0T0n#=8~09lx>0B@wGGD_n` zpeGiBGrd&Hp%Jt*8+z@wAqouvZi)ADk1g_=2H^JFHWw)3Up3@DXNCKaf=5NC8#Ws` zO4Nja2|eJD23VQ1b{2%ep00A8vc1-eVW7KKSBFPcTIeZ* z>sCxySUeJIV4w?p%9=Og4~HSziY-SDIWbFWj*)DUR+%saxZ&D5w`dP#j}!(PE=)TPcFSthA_52p=g$ zTsec!(bNY-Lb#95pw+;5v(;gZKzV56_}k%gQ?9ET;VbhN6;E8aTgg;}9))2FoZlVF zHQm`xD5~osnPtruY|?hM0hy~U*QnOk8fi&xU$t`9E$|;Ew|)GFuJH>WaHtFOqaFA) zaFeK3^A#QiYyI$)AB1-?sTi68c@I$M*ANT#l@FX5i=&*>mGJX&A9LGsRWA4c&h!!; zxfzav|N8ZR|Mi;!SWZB~8Q4)6Y=aq$*+h3h?jgG0WS~tiUARvB+$$9+3D2a0)|K+T z2}WoAa?vz=r&5HUb-5pA`?HJ>EMYH)dHXk;aMtr+PHuAm%U$+Lyz|4+3-TvP9_1LN zngRpO-Y|bvC~LSZYKdT!o=nCz2X=oX1hHWz724RVFQhB)<5&_rj1novbFvzaBP=GB)1mK#ok)9RwU^Sw6%Qc zLygL{#& zXGL?Faa-^;xJCd#`yOS;wYVNbm|3ZzcP^ym|D@qOYkNcWYtC-7d3b4FCLqqF+$vJg zYOo2ZQ*f)~QzNd{k}QaM{Rlz#>|L7C7z+yQBVN;QR@gw3;Xf`sXD5tOD%s#?n=pD= z8<_aIIM&>Xm`GG-tNvZr1W}V(>*%mk27$Lj=V$bSN?8klR89T3-uI+{$ptPoDtAeH zVcy9}QeS6{#BPdFcPGTi50BjSLGYSbwXWa%!+|k^HT0J(ijwu@ql6TB_3M!fX`PO{ zZ7{XG&K5Ih^*&~lZf&((G zhU91ZEy~j-QqNWSZDUE|BX1p7%;z~*pXU)HR_vBNR#N71pIEn2P%9X%7~7zPs3kt7 zG|!wxU-)S9TnD2Q5!Qd;ojnDtb_zVu;3=BPIfjeo)~@38!e?bU>z>2unkSf@8u{$; z;mE1XWIh6~CDA4sh4o;}{u3LQMVd^R4_?Mv|9v?arv0Z@LP=IAhyC)u?@>yv22Oi+7yh@uzVA4 zcv4(G%bwC+wmKiJvm9;yeSdv>!0#bhU#iVFRu|V4C;N=ptRhh!BS{R$GD2I}7xvlD z6b(;qLG!3}2yEHi7n}_mk(O+q%OdnY!6q()m)1sppz^schIla_g> z-(SHWPV0ra#^R_h?tj3Klp3-VZ*hxeiAgLjg-5;_Pa9n`Q0%TG#V*r-#aU4#me3f{ zd0aQ0*c;GcHNv1aWB+*2ux9aCN8q@Qc2MECcJx!628ME>wgj-%{d{T~N4gKWqbgAB z^Ln(5rwoY&9G4Pr(;MrkHhlNQtRrH;x;1(*@1Hy@@l?8?k)$cWEjaB)nUlk6fVbJ6 zK)`GJP9{{bh9%)E)a$$P$|ha^lWkYu@awCH3&cj3N}hS+7~we^hxok|C)j}xBi9-e zp)xQ}4LHytQem|?D(!?-+$vz?n?^T?xQT?qo4i54$wRM#3uW9o{!2m;`7FDymiM}W3RwMm1tpw& zAUPxFUlmd{C&zkEh##;C8$XO8%Rq)5Zx}Vi3fw*CL3n{@&lSaV8jVyy!a4mQ*%b3( z{DvY-{&1o|oeA}4aaK7d*-tCfv_fI9phW6}TC?7)^T%43Pq34_3ZbBPTG($LErBO; z+C0WYe1%j5&5g27V(Z8l@2njFO48uPPv3v*pCI1w3Lfya!}{OX&i}{OI|fJEMqQ(m ziESGlO`J@OiEZ1qolZLT#I|i?;!JFt6Wf}T=XuXtU!CtcKl)c!S9M+2z3;vET5GTU zzy7?aDL?X25P;F~TXLTj$IvgsQ@aj2C23T!B>H3pubm<5Fb;(pTEw!S+Z!%CJwI0j zRwnU0sqDhAWhrL7iAB(f`#j7n6BW=lOs3MfZrAK5TUfXJd_TT!W3rEWXbr(bPZ3sf zT@kX;)?QBs2XkQq(207S%MLwD3#^1D`qAl_=e4yroPi$!&=JEWg7yv!LR}LEP0cY8 zPuYfbu8~KlGD9V6W=);ww7Amm>Ajn22vQxNAl10LDcvV*PEzmglUq7_5hGOXfkG)Y zgT)dKjw{;6ULmG(Rk{+cGfLAs_Cen}*6PLr+g+GMsJ2=!Y5@&kV+MK)FtI7fq z3L{^5%eOfw;ZtB$JoH2n88W4;ck1^8K1U$|X9;NAIp(hjV~0l;CIui!m9lz?2FTGv z$7;h^W^Yv)q8;2WjvhGE!0|)|__X0*2-QmB0X?=7u2Y7`1}}ftarsu)cKs0N8biV* zvmt6=S3i)-@1DQgMaS4!rPw3lULXW7fuHuAdHsQkQ~hy!tXQ$~?O!+MZC_wQ*XL~` z{x`0ksz2%Sq_{CDaAu5nC+9g_-kFV5KAmJrULC6h^NR{;x<3tHc^0cU2c~kX# zc*k?=CnBR_<@G2MN(Y7!_YY?Y{%*akFKl&2#zRAu(Omeq>{vE*s~Y7!Vx|yWQJkfD zCkD7ky=q-EF+Q}Ve?ok2;6IfdNPPxm) zt17XFu-wK=c!oB966y=&z&A&aL71Uco%#mOBWBmQJCahDevRj|Agb_$=5>4oQH5N0 z_uT*kGQ#>Xj1rJQad1Z3hT^LE_2F&=4X7We z*2(}(%Il0Ns?flIKWB$kQ%1y;o&Y~2d?2eTt&un4QeYF23@wXze$Ib3Hexr(i(*!h z<+ZD)y^+8=rfPPJv5Z2hVUL=^)nD-qy$7-MDt|TVTQ(vy^~KYEjng7xr#|B?nEwlz z#iog*y!B5LrNVh*YVmm+G5;HTu>XhtbS(JXD&bE71D{-gLIF%tT~qtSAvVAph3CS; zmdZU4wSMAVGu*=|*+Z(JRPiI4md~)AOQGvNQN4iEZOs>v?K&M7_=OPf6nIeme zY|+n;sEC8d`2e&C%M%X_(Z$;xJzB0A7eut4uSM+GGmsdZ2`XgSaAe!_XX$bjnJy|I z4o%`J9;YGvsJQEV7R?&x>Vyd$i*i;h@M%r~o4{DS!JFWw4Q<+MaO#TD+|5ERqKW9Q`mqDotd3_d(-h zH4G+ezW8Hc>Fex0>0=SO?l*fRI4YD3N9jQq3&UYy_DWI4RZeB692j*#S37TY==5T5 zvh=_`;S`W|6f%y(l(f|HuJJ%)Q!^yFcQ(;PBcq9cYZsfUh^JZum%9(`%#v;Zhc>y( zGO)+T<8HU=>}PLobdnDTBWp6*Vnu5p7&;B=LA^^c4%wZNd;*!}7?~)x%?@HDi!AR? z&|;e^Y$>cJR<(K_qn?+&xgu~wJd5{yOEa{6SF&Xh1)&4)UItvk zHKkg$3AO(s1k#jqwe={(K}xRIJu0{y*B4=A!S5SX19RgE)`waM{9oK_b;EmWnt+Pm z6(`BbmF_DI6v89=SV_==@c5qEDy7m<2eC}oE-&&z87a@Yr4zh{D+pD3W#u7u;W$}` zCze2~UjY?N$0G9JmC@B;at#7rn}+Y+Hrb)c8G|9710%L)u@B--@ANAocRd8$xQDe3 zu=*(8A+h1fTsGz&AeSkG<`;_ert_#w};8Vgh~;L?~yU?8XVxd(P%S|%Ocw; ziCq1b^@=wZA9BeT^;WjilG$dc$7zGf>ZH@R<(aKl|1M9M(D;VP_W8yO|1ZY(4|e|< zVc02?ji1ncumYca|7S(T0k9O#^?CmiM5cmMi((^+0MUnLU@>;lPN^YvHY!PbZ{75m81V<(hdh1|fD7I6@*-+D|QtIJLb{2&@y7*^S>+ z6}a@Li<2Q`8@=K@jZ;~PwTsC(wf@rd9(c`^17tSX>QjrwQrH!yD32nRvGa``=j1}! zb%xp8<+f%%dWP1NP&MxeFW)uJR?332{#DQPfCDx$K3`QCpP&CrXVtR+%bbEz@VQQl zz6}=ce@lAhjem-T4=5?@RZNjyAd1PtES8#rU*ixwBpP}t1&ZQD@me0jTJ1(G5!+QWG=ll4!=Eyq@iJ~2NSPCq$#bAUJ05W+ggRZmY) zfF!swp~Y~phR%rpju8}nYqH=suX>5=-E6DXF1HgU-QD1Ba`;!fbevv9 z^nxP}=9mu3V&hvvS4j{JEH&IKR@wJryo zacUO^6sTh&?s#}MVFx{Ao>dnULHBkR{or?1m;o0Qs|&~W=jIr4LSZvW@c{BeleZ|( zBO+OU1l#q;1@^$Y;*s-8Ww29_`UvT1LpAZN>fe#0(Kw|@Ew9Gh6?*X}N3jq~!g-qu zbggA4=d}k_l?1(w2Tuj;fOBBX8mm#XIIYh7MbuTWI%al5mjXPl)ZY-hwo$)>a!B3i z#8gyQs-J$Pfk{6o^-ahu`?!xcU)H)@mk;B#;&!;3Lol;0Ul5=NV{8{4O#&it3Em)LN8pQUiGpOu6L~gn835hsFcZndxUXsEvM> zdS+At>Xi$npWTnBV+po-`;nA7TuzKMNbHL}{3o$GL)!vvcBVD*PVGV%nMUn_Kq}6< zXp?ldW%+olkQg)>1zbl`-{)N-mf)J-eh}VWv1fmL43P<+g&c|hxXJTf2ktv4S^tk8 zTH|3dgV|>ib%6MS{C_{5+n2E4QWSb%@qyY}v>@Uq$Lm~DxKs_k6c5rJm3x2Y>()wq z<$7D#M#_(GT;Jmch__=|CKrdmv6uVDyGJrSZ?E5PpCPxQhS{~)k!%Q7Ih}g9TbUG` z4g1Opz|=Fi180u;nb%!Gu)w|S-Rp`+rfiA0mtGx*!7f9rNH&>W>tEVpBKe7QMnF9r zQO(g|DX#(-O3!&BitBZA&aF}+SRUWJK2Di`Fv=TaqOR@4)Duz;6;#HQpCK?LSx>E z2ZCGQt~56Lv^DU|F1q4vk0p{fVyCt^9xW~eoXrl$)40SJZ)^CjDZa4nm`iVW0DxD4 zXlK;D6M)tthz<$_37ojA!k`k0PQgA7$tN?|EwUYLfKNt{eaaOf+9vgmEZQdB4iCU5 zCFnBw3Jcag>53h8i+sl$(5c(^DEZ_ED?qXX4S4$*M4Ws@j^?M*7c2R6t)v0flG2Cg zp-(g4%jfoUTZ|{MrzOp27)g335GXUr)Rr*X_x^L|^>%^Do!b(# zK$YqfHjcaBaBqr zFHW%SAItS4#GcClHPiG47n&}_JeBv4Kj6v`IT@Cs@D$O$ld_Rs(y1lNWxNCkLsy%QGhP@Jp=%~-hO&^BmUI*jm($f)+Ut;^=QQ3jI!8V%4-LN-ekm2{CP2}P}R?aEZdyhABcA-r|2 zA*WK|^awnmIs$r;s%%S$>ORQW^n0P|n4VCom!T!IyP40;2) zWk~d3#4X@WrPVOtq#7b2K%o$4l)ed-=(VEEI`n;}F2Df#HfE603Oa(K=Vh!#guoFx z7hNL5z)o={67gaKe?padWw(u=-C75Rto}o`1_fP=I(Ic|VHsB;clM^!awZgWjJ2$r z6Y-%_CcJ$qg?#$0@IXelB@(IPg$`r#h)r4zRn$3%#%z#DU3B$(iGko)gKRG4Miqj} zhw;dv4-l?+Eo=HYI(X#dcukX9jZIPBu+LfD|7#bf-S}5Nl*`SrLjdn^zC9ZCSi?H2 zT-hdX{H|{cPtHImZJQ1=Bx$^T6@$=rC^kyDn82%PkO(|a zWyoR4>pc-`2V5G1{QwsUG~5!JGI-RLTX3RBXAZ*%*!S!`D-t;8AyMxk3g`s~wnB;< zv)^}`zFj}u7gyqS*A!rCmpXIDqcO5S=U)6x1WSz7)kPxhHB&8zXuSS{D3qshciN!c zDna_xWwFe2n8oTaeNm0^lH33j4~=L-lR%Wrd>S3deKer5(w9me0@+NX+WBAhOG%$= z=pkMMBv=lWkB7u>vGF+}R&aDGD@3SO-++GPT0x5WimJEd%N8iy!L<^=_P*{-J$2p) zgh8cZ`M%=`-Bk+@uLf?vRdbKRscU?Z=MH z2W8R37LkSJVv#ZPccN$uzggAY5_asu{o>jeW7KpYs(_Ad6~EA{R(Ms2iqlAOJ@D4# z*J$&gr$fAmC(3k9he*m#hTib0EypQB~WN|#Njk~ zD8Hf==6woaOwH)o5+b+Xf@|u|78O-VRo(n>%-Q1+hB^>$Gkc$+sv3=h$i!v^Z*4+q zDOiOiNQ-Fr&fX+w=#a`*B|vnVCP0Kcdp=_}Z$*t3ejm1`Y&TgxX4_({85v=2vO3fn zG=RH`{nZnVJzF9{-okBsdMn;-NJdx3ynj3b;6c2vLq!f&^)~B-cxPGE5*kweA&pYQ z*ecOQnmK7>cYs5@%d&=cS4!i&*j1@XWPM?N9m~T0hQs_u8=NqM=W7NYEAZKtup6tX zDtPfliO*1X&bueccg~@0cZAwhC58su+$DfcP8ln-a1l!Z9o6b)(;HQ7znhwL{_xj% zgw1a^B16w=9>l)#Hn~aX9|%-Sm@W=j&#b3Bu2#g(SZqPxow50>>YM|0;7t8U%G!$m z!tdn=SLcLM^lBf1ag0(+HvuPvX;sfPs$Gek%r=bbhCIW95#x$I!zdC9PCzCW*j0>Y z^Fh-JI1Wrhip4KV>K67QSCfd}iH`GEbV>&dn3z|MAJU_p;bhf^&*)@Gy)85!70hR* zkQs<`)I}6AvOQ3e4>;QumclB?b#Rmcoz#=~vv?(FLI-@+6)QE^p1@toJEY@n=q3nS zF2AgigM2z>yeRh~#5@{vR-Fu=i1s$)*63AXGFK96`NDKRYvV|uvv&e_B(}xMe7$BK zI4Co0Ttzo7Oh!Lfm%iV2Ldv;a{NP!~7=y#p=c2#EZ{$-BIyhwAGVBBS}r zD@J1YneUe|^B*fv8C1}BY;S!*)g+%ts2%Es+>_y|im}^>FB0{wXz|d_Flbtjakc~d zgBtGKl&tA>I-9Wig$4&Dpx_NX#Aa~G)EKo!o`+*@Q<&^1$;kULTxOYEE?5t>8Tn&r zp!(~#cd$eSTHw4)@8&SX(Pks&w{gg34NZB1y zu5BijY6VHH=xoBsGTvZg zGCEPb4KocT**dLYI|QABU*-E(=cZF{7B14x_nbC{C$bj7-0JgYNA*s^+r4@FC;3d& z87uRa?F+8mqz3G~lJ^wpnE!kx$w#w^oS3FNPGIXW-_Y5g?bKkI(iz!hN~WC?8T6P0 zSSkZRb`T-kW;jO=(dVxgXwYjx4mN!)^)jiMrhT5J_)hXdg>xrmjebO>X$)D-G)b`F zb2Sn-a?4u=<;%L3S$?K;XV^Wg_`W-7KIy{@yqxfIX0pA-*CDqGvOC*2IxJv%aFpeG1R^*%@*f$nl;a2nOZ&$Kj$+uJZ6RL1Z zgxhRg6iJ2dLct||t3xyvoQTrl*JkV$joBxn!<@r5W@rK_bu5euXn((0U5Sn=$+n172CrnDsrT-j60}v7Z{V&Y4YTv6x*h2j>)j z=khey#u6+Oh3Y6mNY=zkSmY?Sc0}$n-&!hZT;<)fl)XQOH4>c`_tKX;r8lOLMm(;B z3N1=@aWl>+yLFS|K?Fg4i=@hp{;IFpC(}=cMci@479b6CHV<$?%)UmLy9hQK^L3ba5=g@W31k#k zk-Bq*xwWA4$&IEgM&)V~&0jMX@k9*r!m_ji(0Pjk|5||<0SD~9!Q7S{scHNBah9E~ zXMHs^0rnYZxB4_r-THe>E8m4oIlPFM{EzqpKULWAv$mw71Z8YjE}p)fTqcR4C)R% zoy>skhwd3^?RL#%8`HF#0@|;y%Tn_1boj1GqJ-Pjs|XCAk0I48X*J85L|g+W`A<>v zECJ2bz2yBL^-sPifmDR@aiD$XVzl$4l{;|@rIePH95_)q z5xes{A);7E?xHG}i07IlZl_n6+OJ4bFVq#TdCRkILE0bdKW&b?Buq`D40`-}N4&q{ zi8kxSg)FA4<}b_ImP-b85q$8yW+#Gsi`~)pLo`O#=Jb-GcWiblizc=4btdV)sBHq` zaWOVDDRVVlP>+Mu8~T`G?^md-2eR%Ox2u2sOYVi4ZWBZxfB8bn`XAJ;lp&5Vv_Y@z*hCS*ff0@if3%#Sh%YXXhV&(4 zB%}sm^ZO$MD9;%nQ+l|#{=5cc-u{D>O&&asrY`a*k}WA+QgcXV$LHQld=JOjE`@rr7#g8Zii_N*-r3{Oi|z% z+PDGK$XKzvo<74cC*%w(rg)HaZOkm7K!6Nvreh{0DZUfAoQ&0Wft~oCEKVgw6rhHBZF#pft zr_aJ70oOi>Hx$0(!6w%QwbS=%)m6A^-$X3IAdmt^*;974V*W~Hp{C-mQPA#zpwC0W zK;dbIkgNeEtMjBx{O$Ao15$2;>F1!58R0lrs(bCW0dpW*NIlLEHyGGfh8rA5=4enP z00A9C>fjL2m6x4+)5l|_&!WEs6JK;g(<@d6T)8gG%_fF3gyT&lLIz39H5=$je-jH# zheTTR?FB4#K!D%p2wMs88bcJwGOQS4G^TOav4NU^Y=Ga*eG<==z4U|8bJ~1lg6Vqt zkvx0v5||Yb#*Y11%9(I}#Ic$m-w#A)@Qb;d*Z0+->SrIxAP%}y!6z^l^2nB-Ihr+q zg)E7G;^?H@D$enfjjAIG55aC9RtZ0uH~hFW*Hw1navE&0gGR-KJGVYK%S3syKmsut zvvHAriC%`;CU*BInuG}HQ@->76{4L^GU2O${lGz^b&WOb5p3gIocFxSbQ;*bhNKB9 z&LndLx4%8|snh`@_WxYb&FlPzD;UO?FA@K}Bcyp)<Q@ra#incO>PU~@z1 z;VZBf=_<9CYilhF{L`aZ%8~FK2wZAKrjNb)nSBepb8mWo!41*P4@10C@*$Sx`r*JE zI`Jm+17q_==6)~%3o#4gb}&0O=>?mJKf{6_Pq4}}3_AQN?Up72rp#gaEr2fq z9kU)xJ|KaHaMBz#@z{hcJIZ`U1b=JN4TpKKY$2WzboY_3Bst=!tig}FxKx}*lD;IJ>=cF z-*HekPLnh#h!O{Vl+LzkYxnY}rf|8NO=WSpKfJxUet`GDSwWd0jN^Hjqub(cmscRt)R~; z6-?c`$sW(9dy6izh5O0LI&42MDNP_fWGS_ z?EbsDtsB3K#DpEx68}RxsJHlZ?0ngJj$oSlq6?J^Hz%Gf{`) zE*vN)PKfSA&v7GC(k_$Qtv%vaIWM>ULicxpN8IfWqHp>=t8U$U6{Z21mVDk@kHE0W z+^*@gYFf~Ce-l#^*M1C$-t+KFK!=D9aZZ(#08&P_I$dij+2I* zQ)_h2QS(8mjFscGWq`&IcT8pv(C2qE)G%Mda*f;L{p-2&BE*AQp|PhHBu+4PWQR6Y>NNN41j;({vDS2H}Amwzid(0rsZt6Y8ACxdnCTWolfPCS zn3baTnNg_8*Fe(1kspH>9UIymMxQi?0Jtfp2qQSjK-5t&sFQkTTXyL9TY#^K2fRHISO z;U%rz>DkPNny!s=2PV^-?!3;A3Bi`bxrmYQScU#OY>rpH7lw_w>PreTeF+V(9}VMQ z?kn;?CctceAVW0CVBx`rf2fop>4o&xLw!a7bP zu*FH&@D^=>Xoo!M2{ny9GrUO5YpBF8W<-WDrR3|%^he=Qb=_OUM0u5RsWU|}PeE|b zor+(IXUSZer%_ouX4@@fR$rddQ!&GF9oNm^5Sk-0XcPxyC?k~f0rH^WvL;j70#0b+fdmB3+ zah#xfRvjzE4yUJxL^8Nd8mJF6q}sP0V(K)c!?q8IcI^0H z!E6=!DrYK&{A$#ZON+7&zGEh3PcA=LRm~~`59k>S4m`h_!Sj=V5%t3s7+ zSU>Tx5!T2zXt9yicNS0W&30i4>apfN`>b2fR<660CU-O$RCGsjv$nfNBSvqxE2gkU zZSxIoOc&w0Tx4V6J8>&MgoGY%pz3Ob2%IsE*}sri-wdKswJ;b%i(1x9QyOLYrkR6=0*|fQX2P<7PYpSK|so+a;m4%>uOTMVKqS8fK z745>M^=-0x%954NcCzQALte7GTCye36!T*Fp*;R39#3ABYeR3IKe=`!dBk3sw_7q4 zj<(%GAzz>$lLj$2adY<=x{E;iu3LsN`-_UqpCu)$=gexSGTr=3$;sP7v~Chp{N)&Q z@ym8p27xo^-!37wPf=gq9T6s?a}#XHF7SL%?f4P-!e28zNiiZq1gOpagfCYBkD_ZK z8+Gt~$0j>HNnw^;@zgT+`cPC*y+y{c!j|`ruQ0m}C1$ijUZbi=wAwu|O1hwVk{>?U8SQjkGwUQD!!sKQAkrKY>DyI>NgxXH%>)52A zfzd}WgAqgNj~%vce;rQU>Y48VB=i^9WL7n)hUpHN5{ZcvNq9oE6QocJo6cZ@IC4Vu z!q@%Us1cMcNYZT+ksNweo4QG(wW`GYjW!EF<|4o!og4X6lDKCwAvFWFPnhf@wC+U{ z&5>`=6IMBd$r~aG&FJ)qEG;t_i}^T&K+n9(8Nvl=CA17X55e#R#=@B&NPIoFg9pZ! zq?M^|cu6vHLQOCf*q^#EXTZ7IVrW}{tJ$S>x$^8 z!vPrwrlUBuLfD@=5~Kb5nos)=c+^@o>6lvm%Qj);>{0poB&iIsAm#***$798 zZ9Y{XzglBKr;K!USZJ)WBIRQ~6@zxa2T6A~-~*c-q^3;mspuIb7s()Ts4x*$eN+w3 zvlMv_!D2&Ic7VU>RC}raPc?H5NkW_ZsczoJ`cGr^X*F~+PRYU>BQe$yZho6Fg(3*%>6GQnefyNl$#=aU za^2qJ3Fhl?k!b+w{#L)Q=+Q4R`u;1nOQj-`iiH*|v{YuQ$-_)3arNt6tUT#yXKdZ~ z3M6UTlznOG!y?D2b{=`i&mEc1E5cigW>Vr`+rirRk{ZkMgpgddi^K;RO)cnA3<&-% zGP8PJHMcztF`!^yHi%H}wKu*G2DVXZysPV%iI`H$1IaYGVO4*)S_T{Nn&G(|ez(E< z8)LxFP%2?X#AM&#S%!IkFGs-xj<67I5=rA|JU^lJI&lc|G9H1BBZ8zED26af#=oD3 zeuFct(4xY64vQ{sJ={Sou}Ru>7qd?eb&+yG4_ln)1WZ+P~vr1%@gd|23LP2&IGDn9`8Q)_474+sCx6ffdGmj|7)xd96W zOj7zEd2!&f;#B}Of6D>}HhZN`?c6|65CnR+fdw>9rrg*x?TaPD+ZTaR4;nly)4Vhj zxxoBLP@8Dx@o-@uV~sZo<9CYhS!B*MGuPZ25S!@4ak#g0>LqJ5szq7E4c zj#CvTO(n{_B+JR}DTPDEzRHUJYH5TUa(Jc6K7~WiFf`Li;r^Yr_VUY#>sVVzQ=j&j z?ZG>_SPP5}&yv69cymzraNHV_rf1Dt(Mh@^cK=cX<*W z05WpF=Z`VAzvmA#dc7Md15|qM!Kee4AagOYF}@HJkaLP&^?ixalXQsK20$<({{Rpm z{`i7~PC(iU+NOa}N99f0wuDeer-mqiPz2!wNTE>SkmFEI!^EKL2>&FJKs$nf5{%e} z2oN_A&mt+;*+JIo#&y~1eO z{sK~94kZ`>h$08%UT_Ftay~*gl_-}_+BS!hN)K^iY0#xKs{f^wBBXwNJ{~^+ILP~y zj&Zv~r8JL1=bs~4htf?pNQS3@pr4eF^+7v`_Ed-=z2gkAm1uT8K^H^M?{pA5Botc$~qFVeXG>i z$5ChzXfIud+s!b@{*;HYw!^3dJoA5)o)PclPdBD&n?`Ptu7m!d7}R^J!~nknqhSct zmtckwNH&7;qZ#yjO2X*BqNhRX!KFd!^Dg3q5E%Ro3j<-iI$|&{fy3=90idLVhDc=p zjtrx*KF{l+!&TNuh2dN2>v7kL*VMA_M%*kbOWc2fch;OQC1gn~x00R*G*%aGhNqIu zWpismK4QLa#N(;V{xKw#Y*St%~3h+ERibPp~oRKgjSdD9eU+0R1PA}dxJ zz>A!jjf2N5;=I+J4lQ#6&iA5fMSql)%?MPdO3k}jbQcF4ez*jtsYW8EXi29-bt0N{ zRojl1%pn+714B#@JQE}nR>H-9&%S+t!ObYy_23>Gimh8d${bA=5`KaVGOT^ku%5qS zPXF|^L<1(tPBq}COU6?D82KubA~T%&P~AFp#$22gSNpDSd-VLEpZ@XH3@|O+u|1<4JB7Rh}vrK9+kGJ8?Hfk6Z3}CiXq! zvI{Nbp$Jr=@f}mk?YN72K#|&c)*PBTS>Fx@IpVVp2FL$&YddKj}KT0E4S4wS`bx5};BU&|b3cFY&JDPQ4#`@U?AL|kO zO+ShuC+iO3J7u3`hS;uAi?taIV@e_Jg2!!uwAE_XnPX!B74}0@SJ)SQ#z(A#R8`SS zGq!_GkY<7Tw^T%A_aFQ4hn=)GIl(CU{jEO_tghVjw?{*ufwI~57%w_^`s2**fse3% z-OcIURSr*bjzhY=?YM6?=UU5|zwxDUa3NbcAU}9*V|rL~vQsJA4=;7E95c8H%a2Pg zWa`7TYEO-~ao?|s)^+XeHp+OTW>+|8Z#A@`16Mu6&iV#@D%34Me4UQS2HNgvhl+lYq4rKvq#@PQ*}uFr4%q$>He;R+S6(U$T;)QItuSNa z#XCMfi-Q>;Rq4@VL8zi2ct3IgHD-C>8@MiYP#4P zI^s_bX2rsqLZ#w6E~Z>Dhy0#)KrPFC0jL>~fOff0%&=Up!Ed*&lQFEfEP>YL<5!j? z?X@r8hd2m%W&3%bx2P06Lw*!i|HgMF_;7y!goS*t3&C@@6bFraq;LAhP;Ek{D>_jqz@h@?r{s z_2PQA7D2LkzF)}wu;oXR-~({~%UQlTTV4C{0=+KK^YiT$g4ZYpn)c zWCD{eafk5Ek7zG*Z-h_Sn4B-cd!)g+I9r`ZCmf{HH83AbTE$0-E#E2g&AOJ5H%R~i zWLAgfSFCLJ`asa+1-gV%;^u5Ezr2l2fX4Qfte`dY*9M--b@n5r2q8MJOKb$4+$NLAZH-o-vN2VWjLGnyjzDK zJ-R=DPtVv!ZF3v``$`@?0R69kr$FPpWXV!*trKjMa}Z%Cy)HnR(5&p8ped_JjPpe7 ziSy4o`{eS$kx=()G=VF#PpUG0d&Uj}DAr-!Ozx658-KqQe#eJ>YSwxOxv5c32pv1L z$maJdrxmgE#)u-mh848rls0kR)o@s&hx%k z3?c2I1$M#OsYdK{%Nu%7e_ESL|FnX!!&GqM)9#?#1=sdjvwHEkalE)B}qnX`dOc9vS|8P;&(5Z6=i@60Bj3Kegq zt~_Y{&g)J%tP5ptM7B7oh_7iyB~EDKKENN-PVa3dUv#lu8oDiG3Up@W z=?y6cUNP+_-7Gds9v!TlO_YW1^r2t5|JHFFy&7)YZEU6=2DcF9{}uFs{QCep>y*Ub z2Zc5^_#*h^)zGqffEH%@7hD@!g-*DXI4G}1Y!B>We0FX~J^rTM8K{tuL2?@B=Pzkm z_{VB9Y+iNcW$8%iTbLIT);=xL5<%hL8>ba(9iDO_`dI%E-POED-;3wzq}}(vsk?#c*Wyw#`xQI2KJ`VpJr8u`l#oAO z;evb0j6+pT^BDo%KF}=SjDBB;j`?RKr6G~HKK!N2@qIOPDx@%h^OuVZG$|z=!Md~{ z0PpLjX03a#_O05TU+F>V`xVyZ{932-ZHoh#6}odJ>xww+H4bm$hQ)DHPqZ7cO2NtV z^LOzcU@d*X@ZW#swehfe;j?gGz7YL?d|Dx_s(ge}KDEm9DfeZc4r5&!u)wh zQ2WeKCjbNRq2$1yL15tvt(G={@UW@H3MI?7Cvj|`x~A=!?bYM%lGkod%Zx(H&piqB za#8g09;A_T#n#DonlI2V#Iu_`@k}8lfd{FF7v3w|frFvlX3EM&G-#{QX!!hMN8^Akr%m1<= z_j~zN1onYT(gC_ZORiEByPqiJk!5}8ia%rx*9*rB7m2AeCoeWH$MC0N*bX>?JG0ME z!=N2<1Y*dq0aa@%U65%k?@!YClaOkCc6mW!o1aS_py1loQ`u{0h;$lp8yc0_s0x4DTFuhiME z;JVAUReeNzt$}9?r>*+ETUUILe|uUmI|DAMJ*;ZY*t9B$O3~@dK3i4KBN{C`jc^(qvt3V`F+1No!S<3H-N0JXt9emar z#NO%DyBu8y6stPTF}T>PbSE`Nq)I<)pKT4MF8I)J!C zRTo8TH0wLjWMox#2?m`xkQt(1rDsWhnq=}a;kM}9MP{{*tjQ$9qF*9`G3TJ>)p0+P zU23&A2m%SyxA`624i3Y>e+lL~Nbt(6%pFV`Mxo_vCo|2(8|8bdL zPOsHQMNgJo-DqwfqkcV4bS^q0(k?P>C;+OS82z>X1~5mzrt*Y~~9PP6UE$Jb3N zQ(by`f0i-t@FZAkxSWhQ81I(k!&{6}Wn_@QMGLugUz{czw^$`sO&AgUUM=2e67v|u z9k4KH*gh3=`xj9t()&QV(sfO@wX-Lm&r65HK4E(~2gVwWqCDvo`mWW@fdy+m3YAi7 z*$AlpC#yePlDv>`I!%k&=txVgbxbLt_7Em*!wUGUPF$CgQo7^7O^LP`f`(oqj^>Rh z%qPp}bG+F-A|b-6L*M(u+*+Hn)uU6`7QRLNtDlzDvV@ET^?be-9ZlZ6m2Op)A24NA z)Jt@gro4s1QRL1;14IyBPT6%WMIA$tzXJrLH$tO{XPUQ2EQ0k)fQg=)OEr6lr-waA zD=GXojS2^$#TF63TTO{$y z$Xq6SR)jXeCI~I2w)BaRtq6x20DQ)HU?)6JDX0S1^IA*L9TmO{?{$UrYAa^3azNkh zB|au`$%jE)-ux=oNoxk+4}4~cj-;fqv=Z&DB+FZ-9Mc9*uVQ^qb~yMws@{}_RKi?z z6WpeJqTNx^vdnoog%8SmZyLYg5nw4%{-R+yf!xY?_JtFxH%L1Je5d*W>%6#5BA0x+ z=*E%*okOo(C%_;>|Gp&zE4GCLI-RG#3ang zDL4>@49!I4iVDsDWopl`!WSu4^E}dH-s%s_4K~ZosYlRi7WJQ40+R#M6QHW*gTzT1 zW#SclADOiY__dj#iO0hHjYS?+aMmA)`I0|ns<Hs_G~VkrcVEW zWo{80@WrHJUDv{xCMNN2wvp|nBGVr-PsVvDcBlw@Elky>+MV2PAgey@^!EnKL3eZy z1F_Q|E=IKexwb@g&-lI7&Gg4MC{XP(v&QS)QUQl0OZ^H(tLem14AAsUY3Bp*}Bp{Afg8$p4)^cw7`h-6!U9B zK6UG$j$-mcWNGYP3GRtod42Fa_sVAmhf|e)LJI#S$QmjjY<_o9{BG=?&RYQP{BtUI z-W%~o)TR^4a+N1E{!U^<)Mhu%T!HDND+p2jPqck~an&$)>qwE8R5%pv>ixcs6@Lw1 znyZQjL+&o%=KAZRfIm=ZxDD|Ak!ZFf;<_<=UU+h3ezVVOsV+K0(?tSDLZC^yHK&hZ zBDas{Jg?66PJi)&le2YZ#$yoG55||Dc;WzN%Ajj1eZC*O=>a$SGjYaRpsx*u-3tWm)GB zDZw*odHt-cppPX)b93`sVN3bg`jVtzn)UdstrLvt{Nj&O@)5dKit=^LZl03O=P94N za%#UG#fW4nR0O5NAZkFL1F4c5wr=jsGkC@(>BZ*%&DKu-@$)ksPt_Q;o zACZ$qS4rg&3$+4o(cM+0Gt>crXYjhIWm*$H2b>_HJ;@4vp0QE{w+&^BRb3$303Dhc3E){KckizOO4Yqdn0*0XH_l@Ko4ak<*HxY*w8=2&6th-|e+>@D}VG<%Z= zLZKBFb?SI0^OEV(chlGXqN~mRFb8P={%M#F^tdj-7VvbOiPZ0G-5;#cdt*l=QjrxV zqSeU$9SQ=D`p6hw8_4oIp_*8ZhhkS~lxtdPSZSVUs5Q(tRGL~EXb#`tjP22#YNl(j z*V>p^nt8?-C7H60Dbv}~X=^msH#XQ9uT0*!fH_U~)k}&(Z>rRR=ag{2EGpoX^q9Oq^6(T%Ifl%k?ks zAW;F$G&b0D#QLv)npt6*Oi;|%Rz;|Nt9(4&LGoop1&|5iTg4C-c;>MEA}>ClGj^;R zn_sK?DYuLU%bGtE)UP0#fojf(ZC=dn5|S+Ug^O7?60JpHS4{T0U-37$?nYJa^cJmH zb>c9qXzutHHyHDhz^22u%g;?<4(4j`tYv&aEwtngVb{Q;OP0S+B9U!dbp9(*VMAEY z!yf7jOauuJ{M}S$s$OC-YXpXZCA>1Ql~?9Wl-ynXw-GiGm23sDH|6~3OoAJg6RmEb z!G$!el5ZWZxY)9({|{Yn6%|*wb!`&d-8Hxchu{$0-QC?KI22mAySoQ>m%`oM3Jnek zZfVZf-GBexXY@twi@MxvjQzfA&AFZlUXfu+f1k}zY+$}}8+LT0#(HFStopg`tQy8; z*~01u=a`oh>oN`ac@VNeVB4pV%;DyAFwJn2QoyC-Y~yoY`-$nQ9d+y!-pnK{=cQLm zzFH(|TxT}>!_$gKHWhU)uQtdE^08k)OE2|?Az9}qTBMX$w`Xnf@{6`4z{m~YJB?dM z7EVx{n)kEc)k|K^*%?QbSBl>TQ>%T|AawonL2d z8p;m8m}cMiOf@%avz*KJKeeuCqmg&ZcF@MIJN`(x%4@iB6Ips0U>FxPt&qZgCgWba$c^Fc ztCmPbpZlJA?~i<$@N?Ya6m`Gn*&yxwj7>tGbq6+Ss}Uftu46 z>lxKox9V2cBE*u^RU>QGBCJAJURnE5$$Q65mos*N*W3kEy-v@1iBSK^HI$?M7I^ta zGb#fpzrmyMSD)QyQR-wKkpDm~S&#o+J9ZREAM`~PCO_f0gA(KOkUYZH64fG|Mxkoh zOA7p84aO~it$|+pVnnEssMEgA7k_opx&w(X1~i3jH{F2% z=JYwY-vmZ6jG8nJ%4J=pbSdM%)GbOeeelW3=7z^gg}>(KW?ajdmJI>8Rc+-|%a9;f z699xAd2$CYMrcet@iR-osBMB!M`B1(P=5EJo;6*HzJ?&q-kzXvxLV^g6amf%97IXM zjPsY3&n1GK^(B%CbP0$aj#f3NT|rfSIWbais1R-M$9Y_!mZ6gYs_=O(H1cUODh$Os z3@y#F_56TJ+BB9@rYC`9GiKS(3IDaN5q-^sov5989(7?l9TQoVAM@pq9m4HD?bH*E zQ^30XONEy-{1ur~*|>u|e7?Lz7%}5@+y}PRPgwBKTFNa9UR_e6+#l|W##IS z<-1vO`T9PdumozTtEXsaGvI1`tJKys!Alq}Ko;X;XZ=ZsKUZ1ujaH1bEC?ZHoYk=) z!Un1-!$i9cz!>YcdYh9)(LPY?tqA$LP-N<#k2u+daHlo`-e&h#VdN zPQP)fNEOr~wt*lzs1pyqQX%6Tz~_#+i?oH*L&DcLOe2G;iPAnI=fGOYG}kj$w5Tz zFYas2-Of7hUW~Pq^!Ge`ps%X}pV%nYs$ks_ty0(ctOv<>z%Xn6S&9jHj}1_kZ_M?r zzikEoLfyWPD{J6eEI~1Mhzzd+XFABWzfV~aPDSVYHmw>#%ZN+;iGkR)0xG%aowcZMwc@3ZD`@X#EDssD*TDEcq2 z-2WDTkUlkl@HKt*9RV3ItcT#Irm-SStMIaL_oWoQpxFXt4Vbo&hFek8DBA*)A`wkw z^Jf8ep@fEBbwmV?TYSi$=&`6M0=eP`BW3&b0sC1DpX0&RZyshIzM04ADc$jW%y# zVIJw$s*JOP(HPpb26h>qQi56NpBjVvv~F-WN-j*8`%N5&th%%h6mJyyjv>red`c{U;jLivwh$Ts=Uh& zKC4CWE~>Ig6VwZ!p-`A2WLSjcxh3SxaMqa-M1rh`Pip2B3x)!+;P+DfAQGzz(d}CQ z!s}cNywBR!?jsIX9%B{6!OZfr%R$kku(G zvXFH6wJV^lm86RG=Xpg59|U%_3>a6ARd9C$rBXl9NCG(uevB$ zPNNxcAc{l8nr%W1@&Y0tQzMsG;SX@A$LzD>XdSTQWV>LC=oSu}Xbe9MA8N-=%Mjl@ntGdGrjT}HvRFjt51EGNAu$>Dh2p`Akkj80{Qlp#T zq{0hHv)YRh0&BWl0;68nI`hL1F+2`b1ORzp zabU8#UyPs^F$sfEP_%|0CKpcr@k-u6vAQ3sRK=6P;~$Bf>iyS@@jdlhu`<6gRR5vv z7yNjQTaJtJ{S&XoD5E`OoW3DHoW4N@oW2n(^*30&$|t57^;?KC4&~VE`X9r62PO;w z@st{GUrwr@(DW*RCh>?XvwE}bP%HvE%^f2EXU3i=cQ?rNLQ@|qiO43zy$A|sYy3~)(l9%MevG_ zV_VPx10|Kle!pl(l380@M>SZhXndUz<1MpFn@=>9`Ndg_r~0kPo6zTksT9-Sc-SPe z$?`i)uu!Z`u}=}f)uO&`)d4%SmEp(uLQ3PY#-C6kQ5iTl3%au9(`g=lBFuY9XK$sy zX5G^0FO#I-mIV{^8XJP(D^mbxXfxfloL3;PN`2LDxRHV^c%Hjw_~!GVD*>Ys|7+b^ z&z}|Mb4*FXsoRO=@aBt;9SY>ze=)r)_+l$Nz!AO)Y80xAqlicT^cX}0Iq8Q#8$R|& z5r$Q*P}9A(9huyqM6#xr!3m>yHy(>msvX)!P6ko;VVcV=-M)C1-lBk);Cvzk?lA*d z_Z~5a4XlIL_TI%t{xJ6Va=X>Y`WVmjt0y4UeMyN`l)z%NZzWaF&XH^Gq4cd9&;>Xr z{`!Zzb8FL)UvJ^}9q9`;&4h3AsbgaLgRV+eZ@99Z4%y2_n+~r^f(@AySPc@u-rEX=BYx-QPuM`c!RrC%%7yLWnN+>A4M!y*Z%lA!v1)t>QSL>fth!<9e^zmoE<* zP39~?Nv!E*K|fw3deq}wNo_tia#t3$#mzrON_)xEiZUyoKmyPdoM+vf#T*{Mv@z}f zBg;?*ZwfW`+X^9JNS<~)ZLmlDDAI-jv&b8136?IqiwVyUJdO~S;hC$a7Ye=?V}>y; zHu|@RuOhLEd=-Q~saR-569XvgGQ!b~SIwMkUDaQ=K#WI|&(7k$-m$aw9?S zBENdby+vm`jJyFwBE)5dm>K}4=SB&289@qk#GyFx`WeJmPUw>A(ZH%7(%HL9%GDCM zZs{2MJ$+Vqk@U+*mMoQ72cpuMMmk}8+Sra!UR-?72|(K$nlsrYbeK_*A~_q<{8@Z+ z=}4{d z>}pOx%*$jv(%X{E!2s7VM@{?G_5EooQMMBN(lQoPZa!YPl{2Gz4VNoN_FDkcK0_^2 zE?>u#R<^#-u-Y->V|v=}7D^1J4Sdw+il5m^b}`Gzy?J?83p0C-Kd44}Um;aCw6baE zpN7a#B)0&nTaNTs_VhMwDekQ)t$MQjdNgf%G@h*~EqW6hObrkul-90K6efiC0Xgjx zG`^MThDol<$`M>jzg7i$#ep<{Z;6Gs`qC)S6GN8KK=%_y+Th$Y|z5wPjFdOM{ zcxC5)xwhJ6A&tXFeR?1I<~z7x)-UYCc?b)coe?0vD#e200N`-WtWJ{uscWyOZ1-dA z7t-k9-9fU4`G*plmDVb5oFh;GZQuDM<>tx;HsVN|B^HmQ68>frt`lV1+kkS|7G;vUtbmb_ zj~af_kd%e~#lf9=_Ff4d(T&W=M@JNb5JEr^Y)k_ox-B9s9t@~~fPD9xe_sY++!BsA zq0PRvM3Ba`MC?Er-v&*Tv~ZO~6rpUCrCorM8fz=-7OS3J$#`{G_w@45Uz_;zmM~ms zycG+@!u6dK?u&%;gq80B7+HHk5y#mMzr3l7ZtW=g;4+j_BijyRd;UhEI;xIt`>Hl|M#j?R?_W>ZrHdU?)O}5NUU(DXqb&&~qOG~Sf`hn5evxd3>ye0h zppk{3OFsOmgePwsji-dy+1}Ztz7HccJk%{h19iR@1hE#K^eq%%Nz~0pugG6x#8KE? zQ>BL==T*0}Y&~3^+~4DeDbHOFe5#(8nSZT0=FM6sC(F^Z5>#s9XjT34%EY~Y{F?74 zBy4Y{TSkVyw~|Wn6miSeEPmP7tx$3)B6{Va7T9_>yAS&POZ>I@?@wlDgx*SGn>O#lOnZ8fM_Qg&FO3|E!J)+Em4?vut?)cv|Er za$YVr?AcXmfC9H{lb1`A7_T-WHmuHA56P}U~9N59H_=uFh*9QSkh(AtaEjLWdK5ry~J#$k| z{T%o1#hH%mi<47#h#Wni+nua{gnutY=wd08Fnl~Jwr;`@#jW#Drj#aChF;hzs|VW2 ze30Mz|K(9F=k_2;UShWsv1%odG>H`$#so1pVtEYU%Xk#i@oOgah^LV-jyqyn7$jYk zHvW=J>XA<&p&d75o{UR2uS2&Om21?RXgr`^uNGeiQLR^7{O9?4+YG4P($r>lQ{wKR zBoF(5pu0lgt=UT))B8l~=O`Q|Q^mLK^CbE;miTFL9}-6Td2%0GU_KM;&zr<}v)Xgw zrY4OlnJN=U2a10v>t@x=eSNN1IL@PRX*=*r=8y8AVl4GI(|2UdVQBaVe3yT4c7xT~ zN@~le_PXS_G{L7*=>rVCF+;7O+}H7RN82)T$}T<5uhU8iOWVg$D&*}uhPa`ZT;9&| zOjEItOX0|tchRbApOmL;B5v+|3(fidR0_kO+VqL{XBxau$hJ5-X~NjplSgV6J-tux z$U|v+do?Imu-*4|hd0ot@94!vDzV^JO6rkJ$z>Tg43u)uO$ETzC^Mas_PdL;$t?K8 z(#l=cCYXD|lUL&?CRcEe7uWyTuN}Zx=;V>y-w9Hq+AdwFY9}d|-J(vqnr8Pb9P)>7 zlds(*2FkYy%}%Xb$gNwZp5;bkb^SU;=q`&-?FM{7OBX1qDcam3edvZez@-TuDDE{t;0p+mEL+OTiqtO@W-PcN?%54d{Br{+IZFl0l$2KBj7 zA}d0qZxi$_>|3$j1WA{1^GD8gGhw?U8gX7JkAttVPud{nkqwTAmF`Fd?$}gg+km&b znvz{uaqAx5OmmQ?^%u_~9FyS0-nT>_eBD2Pox`>xK$BxLkU_FjPE@ zo~@`!yo!0?eB%tu-H;+TU1l^+Dw~hljaF+LYr~ieayXKAobkliryP;as%a%4c9;Lk78#j#~R?1 zn8_|k+ZD#?<~;9>+o?~yw%!5v#6#9xaTOogDPg%SvcBlH^~$ID@x9~EVnU>n%25;6 z<1(ge9DnlZ&LeustS6pkId;k1in{)q5d6CisZpT19|rpkbyXj;N?!_4+DX68=yHa6 zjCn2nk}@dRJV;A=Pvg%A7vXgI!6QJ|Cm;B@xxMb-Tnp1!)0=gwC-RSFR{HK%Pfi%^ zJ+62c<1SmiZ}UM^5@mQd(zn3AkLW(KBUvr0m86L%#$NT*hmw(qg|%_p71N(TjfQ`Z zs9-PKnKJ<4?~R*wpizJ16G9F!KNy%V0@aJg_7-@28eWeo9E|dWs~_hC`U!YK-3q=Q z%zYx=imqGi7fO#%A?GO5Kt7pXw%q=%zpLob>HzXCZZ1`o<-3- zHjHOEILc;RZq0^wk!EIGU_olG!(j3WbRBCWh8tElG_9QA$@@o!4u*VZ3}rUarG32S zf0lEnMWv8DZ24=eXNIv37ytwXyElD%&-^X*c`fKo7M8K+D&-VeKehK%&?ujl1W#KA z|3p)lhFqnFmdvdrcUf;Z|kGO*6Mf0Z5)&KHs>i1hX{Ome6V?+imYml7Z(e8TSldxo)l zC4Us*w91&y&#;ga7o(C*e^^d1KQ@mj<^X7J~`N^x(w9uXal^O#2 zQ0KS+oVc6sTk_PN5NhU=ZndvfSg-MP^WNPEFC+i+Ov{_|z`3(JMi8;AkPz7O6p^8LBvGzR&;9uah9`jYWOa-oVwvxm zV(*scV_x&bmX<3Z_=`#FXqGH}@H%>^sM4L=%kT2T$Rw{_$yw_%Y%2?abM`d74bDvw^}XqdDsLB7E3t0n`$qhL4?ps<-1c!i+P+$P=?^V#`qcy;xRk_X z&3E+V9;B_S90T{qv2B=VcPTp8a(6yR{P6hV-!~0`vZnojn#DV^{|wsN)z%r&upl5_ z`2TCz`ah)ePjQqCaIU4Uf&FirKLiRSaTA87j3Gq?>jEJ`jg51}7FJM5p)+oA91NA- z!Go4>A!#WzF|lJyrdu8FU$05EHlg35iKcZ=+p_WM>Jq#@RIlm(eCam(k-NKNwd*z` z_;_oCMhaOmU`5(}J^)AR_j^E!)bC<|meel`4lmf-xfo~yU<7#peSqZzg!mrp7pCyu zg#{BxIN3d%ts8RwsfJ$lk%ei6i6-12L!dwMD?t{4F2M$Vo{3jlVJ`TGkZXNdVVnsO z2o)%Ztboi!$mgstMyqIzFNp_YLvhJjNa%MXoatY1MinERIrs`CXxNW|!g9b0vO%dc zRt8Bp8khp`E!wA?!@vwLocdzgpr7s>%IC0cj?U)}zX=^Fiohgy?D)v2WYDPONzb~j9g$>U250T_qFEEDs zNjunC2?t8EQ8ikMf++@pDob{(2nULn%!Y~V6N>?eY0r#tatw}ui0$DIW$BnC5GuvD zuXfzuU&Jy^&_?eUB?Sh8HjBk1p|ILsIqpMa;9@c`;(;MGF6YScj?5FN@!cZ^L-sXaCHCWt`Q&+=_qYYkL|8goo~%|(qQsiWi_AA8?i zc3uE7XCb_YVNPg8M5<4?21yqLs65kzK&&W@X-S)RZ{k z8iMm0X1)%_BiWx~pgQ}ep0utciD9d?ciE_VTR)F!@!8fTwXJKa){D!JZd&Pr3zff_=9F#?u8eie z1jt%9f;VedSC{U?DcuF6ViFH!V7yxeO#oc3l=DLomHE|W+G^9Md0EZnmYwC_FDrAe z){`FWGq2XO9_(|kD&>67Xaw`VP_Ek(f)H`hv}LM6JlXd9nI6>lUaA9CQ!`gyoV)-X zKQw+cT?81$oRyTi4J|#bRU z>#OGLkTM?XDF$@oz08x$DdsHEjbTX_M2+@^RK5}^OiAXuDHhRGI-Si{uA>c-$7iUX zOz+3-V?L0nr3c!KiFNzdupOaej3qIro}yIg(i5weO-;dTzw$rNn@pq>fU|N18a^Eq1=3Eps;;*Xq#*UDMrwqV5GDiwN>X zLu{*7MT%V!B};Ga*kXa?Uw;5qV!eAMo3e6VD#nmRI0A3PM|UY&>MLE0$`dsU^9WA{ z@F{WREPI666n_#XY9WX`os=ja%k2!~^bsF8aI5+XwBK>gmv{HH$ch27?X+8OYA-CR@CF4UioOJQ z^!*S?tzII@{1JwvLBNt5?7mM;JLXf)`5ZKn2X#Of0Nut}05xmnIQ*(j<7PA#C&v@R z98N?*lTUmO`Pjgtto|S+n1UcsO0k#M56fO#DO`OgL!1h3wZS*``Mu$qQQx=flb)8D;#(U{^Pk!g=GD zKd2?j-%3POLB&APLI!ycD^R~V?S&$1vn)=_aI2}m@orixgKo8|oViWvN|k?swS>61 z{po9A%l*8rUQHgyx_S1+`ZUe^-NAt{$K%^$-$|bDRc;sW+e1${ggTvs_ft9=;2!KG z1bEvH2fW_2qW!z2#|P?jvUbN8Mw;+~5P_-4c-aT+Yzj==?~tB0MXcfS*`UKa5S?*6Nn2U~JGqy95YJUTFG; z>T2!=`@!0au+7E`IO?CanFqQg%o`0rVKK%NOaG5BXJs{PKYcZIKMgg3r>HV78Kv>UxfK1a8hRl= zH8mjkJgdm0t_*pUo6`c?aUn%?tKw@PmANAm!bTd2%HHj)y*lfNaFb1PuLeugowVJ0 zRGNb1T;^_H$9iN|qNZ&gKytdF!12_A$`oU{nbIOQF=5^&G%J>AD1(BXn6irBBjt@c z3810AYb$Q%6K(M;0ve?#mwtY#wG}m7#zMg|rZ^`tbJ?Et^w$?@#ihB;2(53oNvZ<8 zvg@{ZtMvHi`QDK!9hDy=Qv^4dLF%q=v9a9{2WzsTi~3SD)LOwM0N3$Ow1c-w(OR?a zLuWxZ7$ws=Y7z|3v599>KdV|hS+n~q z{rQEdgv7~$WSGU_d702@>G{A+lH*YpvPhV;lrVexVh@~b_sZGi-_4C8n%YNQL?oCe zS7Mg)hM8W6BjOsIqN{GZ)3(nfWdp%FkTgJBB2|soG-l~u0DjAE#>_n*l-bXGeR6!4 zB`a?xIb@7+|7w{~ATkNJ47zNzL^psL` zf7|#t9|P&{JbkQgdFe*(5jI)$DvKqYo7HtpbrdNcr z+Ypb}RvZ(;=GQ{=T0<=rEL0#iw(}%|uL@_CQ?Z`M@UbAqomL_N))wmDb7k!C~oWeyJ5Txn;_SHAOeHhUXqTbx4x8r(w!#UfpN(^ zlzyPYguhA;*Vmzw56_j#j zR*>>`32?y3Qu(VAL2G3md8E8uS9kC)@`QX?9)P6d)Un^dN@Z2n*(?+M8vo8QQUS1me0wko)S z4moT}MCZpX4p;P&P9^+h5hOC=s`#h%zbq#`Z~+pc_v(6gzGDg%7!2RB(zi`RR!u}V zCRrT@z9{ukt>$0KR2pk;G+7}m52*^a&TrRKiwId;x3aZ0njV)nm+-PpA}5jH7L%48 zwR)5IbJVpiHcd+M3G!EVb~C7J>6zTwAZbn4R4sb1@_PJ1HBenvRjbs}gnMmUP18!9 zMFMCn!j6|e>>!-*eK*Rary4EeXR*lt^%_!ll9&dIGZqHN;JOzg`M^pFJ=U96uS9im z8tM;?*749V^r|9ov0;YbD`4{Z?|7)-(FP2@jG!Z%J5H^va(PndL~nlyGxipfbG&K!S3vb8Rz=?knwq&>SAk69sQ5ZK>B8yr+L(0V zqT&%s^{S`y`ScY{VM;Lv&`#pYHquJ0Q^-6#Hrgjr`O2n{(2swJqG6|^GCVp4`sxKCW#q9 zvMN(5B(*QZiGJ#FPloywKVD>N1;>^jb2NOXHG1m`?53fPIbOCsb-H+hI#1 zqoI~yuuMmJLtkh+lUfnB#;lAdtFH0%F|5K=x~I@!&q#e{@g#b7oQpF`pxT)yBbG;= z2ng~m{h2|rJcyysNJ^euYB4s9Mg{g(UEsVxXJ<1iIzDg>@`)3l29*O~Y;(m);*=1V zU{nWp`mdHubGZ295HsH;=#h@twW=D~BVXvQ7JpFu{|A6DVm-$CUNZKJ9(oGvJry z4ZMAP9MOpeW1l!aEuFLK_K*&r#nNdw4f@?%E>*V~m-;X=t3LLtJArWTQSCJzi`2Fa z+{yjrh356Tx9M-@LwBmbav33#{(gsdl2P{&+k+T^p~1=BnQ2g0V#sRErB%ctSazZi zhz-!W@qQefRFl0*j^xBgJJP5NE;Ec}JLAeIxGL(n*?C7G(|tMx{0~C#FXKu$6#@jr zDcOJBvHtI_89m+a7di?cHOW9}=?6A&H^{>CSs51{`fWu+7mL2D9xs%UD~Xd_v=PD3 z%S6^w8`hFv&T}O~cp3>1_(@IRBVNGvE$sQtaV=oQZdLYv|2nD!!PH#~)c3FG4 zHmPyuZn@Vg)UW_9;h5kcqy5dMrJkpI7ioHbZ_u?so&%HuWK&Q zvCzlzTB7&migi=6uR^KO%&>T$v7t1f-|+rF)>Yz zSGf0DN6E_Qf{#)X91QcTA%C5(wm5`mNZPN-X6qo%*1oH-H&~i(4*??;u=H?|wQ={Q z%M!(8<$(PaE4Am-z4P(7gXCj$Uta8y~fdr5Fi!Cr1I zk$GN%k_qR|<1>SWpf|ZQ_#e|O%Z~O=QA3Ah%>}m5!%F|@V?Nf`B~?GG zAogFBjQ`C72EhW69{Udw>wnb*2cV*$H24{^tS5vB12d$wq-Tam^WBS;1|2oaIq~>~Mntpzz`%{6SELYF+> zvfgag^zVvm-ct^C_n#yi<0hjfJpGa%7KzzTZ0Y@=+Qy&B3j1CI~hx35+>pwI;>CT ziNy4iOuHtACLWcMzu+OKK^F_Wx3yrA?lSen!Q2p)WAWMLX0no#lS9G|hGHsBS-|z# zee~abG?n@ z%E(SgUym>$iK_!o;0iFAX?R4PQ}$dTQ@@H9V2C5oz|$bO2QwnnQbb@3As|C0e-&}& z`VYi}2U;fB40;uvn3hN66J_!rhRG+ww3#Fo@eJNED(tpxp>LKM5#k0=NM>aY@})Eo2xGw6RQMTh|u5?p@CfW3It z90Wk>C#6I^itwoc+iHNjMEgsAx+%4vRFuXIBfTxc*(U?#7tl$j0Z6kl*j8E*4v=d= z{a_yUdCI{i+GB*?O{akGOE9JMQ;2E;%c*iMigUclN8x5!hv}8BAseMrh|6y7%|Zhb zDUctf=TQL(MOeLGCGfUrB)G=`5B*xDYmmFiri^dOQ6f*Z*eB7{suv+$rE6%y>88*h z)WctRsE0+8c%-Whe#{U90n#oQemYTdd+DnBh<(ZXbZ>@HlY4Ng`lxMkE-(_(L9eCw z;YMbhFwnsm@7l^-W@REqm<=VD!GY8$u_2IYT%4!|mBebIY{MDwu`xz~%F3D6EwETz zDG?Trei*2=YA9t+dT?F=DtyH<+&K|n_Q2VCF2Z`WWsLmN*8G7P~1+N2WcF07Ktq*!xme{v*n|vba^D1 zr%A|4w3$z6wlYaAX?iZMO2Et;Grqi&5$-p~q4XR=PB|RnoTq*O%dJWHuCYcSvD08} zB2lKv%(w$F%eBpR%_u8ZxPn;ZZ`-_Kw7(!TM*-LH?J3eM{=D7LfMMs^4adotzswc} z`Q)({G-`y~`x3{^e*`A_I}q3<;rPjfOECp)SmeRUd;;<4Z9Q5y>>b08?t|?iI-h*_ za(!m6fjcJIfeTqcM@s=}C)2@8i*D#+!ETc?IX7XCErDBf4sC7r!=^wo@5#(9U2YvS zsX=ph+u7;QlQqajFEHD+%Te@vZ?G zw`+ir7!352Pe*yeP%wiJcQJk2v_-68h4FWREf_W}))NE3-WCk}EmsVZPA#m{JEYck z0qZ|oD)`a!0KU~NlpVac-R%~ruvq7NUvK(-#kb%)^PLERih;k2=RxV^Cc=$$Do$Fn zJ7K5HO9rF3rMC0a0;>&E`BofyG*OoZ<>kX%G=GJXZ~?(H9EGy6aj^#X(;As7teCskzSSkVi?TA8+b#3w87`^fX zoM?cDiUJJ-jhZPVw|t{0)?#I7P9F)~sGpi`@Dp+ZkRt6P{)261R1vJyC-qfL{gB@%C} zRUTyzXu)z=8q>@cB9>zIkLYaU+em+@-xaq3V542ti+)H%m#GkKgWE@d6Ev*a}N5 z)dRil&=fFm`&yz{e4h@d59j#eW3I^@X)_WE2x-+S5xTQ_u#(xj=MK=2z)ll5M)6=JWcNhrlEV2Uyx$?#VrQhvEC@Q5NnZ&8$t&nta(J*Mt9fI0^ zd|~|469iw8UzdwrKd$G1QD1h$vb9#}-HBPBgR?hXr3OjZqdMQ4)?Mn$P5RK|rN2!` zq+Y3b&>{axkx>FleiMKqEE5l_QJ4z=SYWNKn5qcyeSbb_ayl|StZ_l z8B0UOb#ajfJeqKRolbu;<0~PPI7g-riYpRi&)BHbmLyqqgY4X`*zcKPOcLs$qY9uW zzL+WggNF))Qdao)1Vs)Vuo?JQ;^~_c9}J#xV}7W`HK&KLo`loV!T@1eP0m$4pHj~DZM(X*wnFQkrIRx0`acS5_Dk* zja6LkSJ;8oayF(v@eZG!hf#;4yV}DJ6k<;=1 z6Mgc%X^Aw>=ZgT(XwwEg{$cxMlWioUeytPYcb6^0fDlhtjB23N;+D@1nCyp*n%njn zz4qwkrX|m~PIu~}0KC%lc62BG+HhjmdPVN2_bQYp`;t(c?w4x?wetn_U4P|aw$Z&@ zo8u~XujnkEo%M_TZQ^@?X?uF%)j`GUo}*GV+CYswfgs%)pBhZOvySqti{09d)1kdK zY`#UjcEaK&f|B$1oJp5etK4iGv5ex{&n`Us0QM=x0OZ0Fg2qY{<7`4Vn^n)?pNaj< zlX`v=m0jH!tG1rH9U80y(bkW*PEv7@0^D2t=1nA3J1ORPJXQ_Z!`hj8>sJ{CC+8T!@m!T%Xmm2vq|Ih{JWOm4D1(m$ zRt=ciK2)F3tj=L;XS6_4Ya2azltxiUR<#45926<=JMvFqDTk2w*`5qt-SR!+yy=Ia zOpORsD5?hFp$D}n^$_FYi_BnMp1hH2qdLbT{Vz*(|Jj(p;r3wJb{VKJrhM} zq;M<46AevZ7S|tcb&>qK6Nr%{a4%+9S$vcCTrKzq)xy(FXs1i?Pq#atxR-#htg0`t z{P0VKqYpa$OpYE*C>^=N;6LR5+~QIDa#-DwARrbQ|7(2le^Va`)6oPGkpU{RpS>-< ze{uGKV|LlYq@sl68Tn}rw2pTppSH;8PZA5`tx}ux`aXu!uT zQwKdxJAQt>-H<%}G^2#0e50^Z^PyiRMIp<=%(0O(zO&amU?*77r83s|uBs?b-=|_a ze)X^W4i8mcEM9{lyNvUaKn~EU-C`{{uT9EBj>A{I*Gc4G*&z5nXsGo2JtJ~}=e8@D zeDjlz1`kJ4SVtiQ?D^LcFqa~okZ+V{*ft}JthtHS!kEQQQ7>@InjJ+hOl^n6L_$qg zj>!+eei@_;m22-RJAmyaU1&PV3^DPnbs2XHA-23_mh4>+PFU9tRxqHQT*9G~idHg? zNhBRsK&{r!`O<0%`BhNlj|cLHI9H||8%}=TqW^KWBYzs?)ER!|83(Rp-<}h;79DNE zJLG>}wq1%MqtIu|g#TYHQ~D?Ekv?776cH}{N(d1V(4&jsg*}=YphRdhQ~?o$p$$)9 z5Bp7WFhp_4gcuz%lsYuAp2;4nCX;mDtIRh83OX`($OV&j#{b2GmB+(xGcp$2iP(yJ z)bac3@r+@lXVcC_VoziC1;I(r)ynS-?&7qZUw8fhPpyCVtG*g_h8it*%SM0)Y6$B= zC9OS8fH3Lf!E`{(D;rp9F~0o$301gvG2Zj()*!&IjqLjF!1wLRP`GzC{`&C%?(IoL zIAAQE?BPJ_?MX#AU@D$0A5O!W2;`5nVHUHHdxBXT*uB1aBFzeIuwfM=X_&>0D&hrcuMiHY*$*c!k24?3qi z|C5qKX8wc@3MUX6yiozhv-#zKnSosdS0-M8U=g4aYmS9i1XutBLXKzmivqI&M+x$b zy^_H6pPrHb_&~tCKouYxaF)G$?8X;4-|R^MG>P0ddjp4TWd4K+N+$8-#Lai$Biq)ba~#+kh?H87EL5{+ zw^p%tep83o8&r+shqcPqOt%GD$NCln?!&q2zeMbE4jDA~T%DT5xsp_Ghx^Ea@MO3c zQ9yhg6lUlq3@EKfv%|hR>9Patr9-b3)RTb_;hZ}LL%bRW?OX~6&pTp8zG->E<1u!B z(W3881k-fOgH7~HZs7ao>UF^AY4kD1)M71zbSv~rU_EHB)It2Wnn9DCy19ODM5bdM z?P}KXZmcOeFl3mG=1l)C{nVfmPTtU0kPna%J8vH-$y#!Q{DQMH$N*?7QvLLk72VXd z%YzNP5(m3J=|a~g^S%zopQ?$~)#&#yG~?<)xCU9U4ld86rNGcfUI-bZAAUA=045Y7 zS(p*s8+NVA6OUoh4MMj*I9a>yhF-tw#^&k!ms;f~WY3GH8*apWkQ@f&K!poCe#2(~ zu`-7INYiOSK=9OJwQqAj@H5;go47AuHcdC%*b#{1QQWxvGR-tY~h2m+~EZHM)0Jo8AZT zsmv#$%GSZEZpa9xTHcc$SZ+hr%*L=f^oSF!ClzY&%%qnQRuERkFC=r@3`31~AxJ#)Ogz8H>lMNNA<#GFM-d;-yQh zta#26!~9Y=0OT9xrmo5!smdu(GoyQ!8$j z&>fH4MoKGO<4p9VM8tlp%(wiROfqE`YC4|Dgjr`?0=Q(~Pf7G;Qi~=`gp4&}#M`u? ztXs=l5PUugQ~13k&wt}l`YZH8b>?2Ch2AN?j!Ti0XDHI}&_(+gZjf9tDeNhZ%6maS zheFC{5jnL4EXc5$*_+jq=lFT{v$466q(|4TZJtoE{#;(yR}X0;LpX>bO%?oor{9r5 zblzP>3~1y?psgi&&=W|eK56x-VbbawH@okkk*R~Cr9rF{q%2Raw$kziew(MLR3MkR zy3XlHOOM@&Vc+p<7fr~_2o8P*Q{Ws@wRUO8>Xfc~h|oI`(Gm!5Ms-lDPi#Enl;?WY zW=T-CIKV`*e^s?Sv$qya)2)0p3@eq_>ZI26ne)dByhDUrhI87|jEQ z_u--mf{PP?&K|$BvQ>9fv}ARg7FH)!1fZyRmKCHk+KM@3(dKx6k?I{tM=s zYi8EEXUQwO2xLq8)TAA4?#mH9U6@oW3~-pgxA15j@r{z+`*3m;4W%MqM%kRWR-5B6 z1Nil}3j)#f=enfjtWCcuknL#~AH%2BJbN-<+<@P#NwX@`DR7SLrQk6t6D%#uL`p_W z?BEY@eReY^^RiBo1SBuQ5ne1;M^whQ7~!PZK_BRBWQ5-me8JuAp2&it4e%Z zbz(ecmi05>U@k0yT-FU5<0n5P(mJ@h0IYe{l+cDD9XHpNg~|KCqO?f9|K?JN6wj*q zMm3i(@@y6K^2nYV=IQ}8C&h3CnL4&a`lw-lus-DG!sTN7uGTM&Qc04Zrbk()ooOw; zn=EBF6&VHCJXlIjn}KJQ`^t+>^TJ|i=b>HZo~;W=57Fk)UdtDm8{@RLn|aif0o3d9 zCe6-CjYTDxiX|q?1}Y1dNvL3G3Fa{NG%Z5cl2*=Bx1~)M-~02_P9GoJTBbXQmy&Gt zpW|b#oUfnPOO6s2lxPtGVc#_aL)vS9Tlj7G6p8-s^t*-lMzAwXcEZ*XL8 z$CQs(78#Xdq#+xYGQE%YAvHPL;!!6&%UhYUd1Clv)2L3G+zJ$zl)s9*D>0kw!Z`vd zU!}R&^-_w(HqU1%>umV6A$hue*ISsA^~6xcMdw#1^(tRyM{8MQftd;!OJ_$V>1=*; zqhgRpFT$*@GuAbTXevso9gvV0l$FImi7?^eGHy;%k=5dD(M5yCUF`fhL$XKs^^}&1>oELeGH!Q9hnP-n^^-_+lOIn0rDXH3Memd>#F-wchH&K6ifm zsX;Z6=GkZ+v(~(0A8zA9k6rXKDYZNOl<5SQK2F&T#>oynn1WsR+0h9&kDvrU^}DX; zTUk9a3MTU*p=_T`dI%l--Ybc+2~pj)UpX&Wsc}T?gkl za#WUST;8O0`wpP$IHYllm*>`73dMPRNN##7#SruI(Q2%2J0~vHx++zDOCS2i_~Xp} zX@|7Blg7&#;e9sgJnzT-TqFB@O{;BvwLRM6Vq&zgtJpn6*ma--yrof5=%aHN<)7K~ zI`oBuf*NtnFC&rnkRwDe713iyRcDliR>g+KrEsO%VB5;DiN>2GM@a$n_%PF}YxQo;v z%>kdQkZDGra5;BvC#3Pyb?W!0^x@@C)6gJvceEy{d7)O!Rq{)F?%qC>C+cUoo}hvR z3x00YTgD`Al z%nkh?um-d>TpW@YgumTK7R|j_p9qc19TJ8SO+v|hC0TZ1o>_ZZ`%oU3pR2xgD&-5n z02mZwCFg%$3o^K&KObY}2DI7!q0*(V|CDhC*j0}w%^Ek0Ui3yYJg~eigCQA`T3d>B z3-6xo?i8p;Wcyv0Ft`QFJn%!g&53c|5O6kEcFnNoH*%O>eN2g%T!1MWjPY6;46g{i zWtxmMZO#rVug+9|y_xhIz4R)ptPPZR%&Gx*7Y56s*FJ6y`UI@oSqToHJQiP5nL0y3g8io#3ZcY{Q5zB zItzXNgX%mOE6p>N=NYy2d#PJ!zALjlfH3^1qAD+|H#P|W8xAoehVAWB$4|mwz@%6^ zYB$=2iDO6Mm$w}A9sxu_GALr)vM9f-Va%slIB&|fy5Ag94RXe~B(KV?8Jmv#(pTQRqt)7~65TkadkbyC2a zTEqS60TrbPdf#P40s}iu3}BX?e_}~$QPR(p<9xZP#?^`;9<2`XVaS=H3?R%<1_8DWV zv?z-MU?MRU2NqzhHOmblmS{`&a00JhvW%!2poJ?M;Z4lsm?Vv7=2$#WGqElxq@Rk` z%u^rVo8fKrk0gDOeqbCDd9Lu1Og4R9VTi0{Q1N0HjS9&`?l>x7p^q4ij;uYs%alh2vZkejuXQ}fRN}MXZtD5tf4||nThzO^}+ZU~=;rqnx z2dR=Fwkt)11~a^Nrfd`BOj#N8Dl2nes*eT)Ih{bB@NK@rhIJ<^ft+oKoJFL1i5M^g z`fdTYLg9=`->KaaKdi=gA(Knn#1=p+PX@XRTmyplex(l3BvXF(76(7XF<|;cU8rsu zzfR-8vY+##;Qck1llQ1m&ykmiL)qf$3z<%^ndb(}`LE)ZQUC_S1TgK01^?d=_#adI z6yE}PxRmc?&~X0=378aJc~}I1nvM#d7}`g%hdE;(5(6<6gS9Y+Q`&|ZJP`^s6=rLY z-ZW?o7CTW2tTprfKf9}aS(5`5CY*| zatQV@&S6kH2~owcR9Fd(d}sP@EO`UeMlqojB7@Aa6yl-BxZa$jY_sVvyGzFk4ljSt z&cii(t2*SnU~ukEx+r15;SLxKVKE~M{9#4%NovRe_t%%EBOLNe=dGqFv8}hkbyu#L zXg!8H-Rkf!CQZP?hV_!)HkF`O)ZURd2N)vJ4~)f(#jDp7dY2kvZR|_CDeFV>aK&&n zpR_BMuRS(w={J5Zf@m}c{OJ#Tp&1OpVpl6wdKt;nJ{T;qAGp0fCoUX6mwIRZ1F79kAHJydAnlSeNQ& z0tU_9MX6K(T~2(QXr6WYg(mX;QG+RPw7qNt7Vt3+&hrhwKnD8}_X_cW`-*kW(Y`@G zfazk^YKgvAwnht}eL&BqTd^EDW2yo~&<&bc2oBsvv4m{BEzut(>a(;y=$w`m1Fycx zRJ|E@wTN!$m%2f79HU>o@{Ud(R<-`H*7{2D@DETYqh#9@wy>p+ZDN@gWe?8IP6S3# z6?BzSydS41sA|F9vh!&ndRCgz-l~Q~J7BjIFoeVzvL6fp%zF#VHv&D=c}WhE4piL{ zf?2i&Nb727D%_B9@#!J)@URF~$;1L~q4}b-5ffKTI0?mvePk4cjGcVbfarqg4Kd8y ziRm@ANpOBMSu%jIE|V^QbYucV2FbW9>mjih$%Z89p~R@zBk%fG*^c@<7?ga*EA)~u zmKnw?{+A;FW9KLNuKp_DGCa>R9po~evn@g2KC4nypD5f;7o=-m==*|GZmeSh?vAF@ zDYV^dnf%5s^fHU8J0ZU<#FADv$2+nWnq!qRq?4p~v8E@;!56TVBSm!xt(Y#tX=-If zjX&5C?=nIDM=hK)lXQhE7?5RK!JwOPtv^l4oj}?EMSU)?$vlF6d}a7wD1{fG?v1m9 zcz!Ryr^%86GNZgxk-uggcxA)i6dDHcnV~R?Kf3VOYY(GS?qh}#8Wt<~1257gOrS{Q z+|pLT9K0r4Op~w`a)b^b@D892<&&8`GG*`zAzIDSJjj{$&U5zCliiQQtj5l);AD5Q zWoo|)`F!1h{1Kdh|H|x#+%}qT#oH;+<1#$$f+ZXP!#{|7Wqbe6p{XgPr8ztx!6*Y; zJ^xojSZ(~jB2`3yu$QaZzZ%vm)izXcfgjlDM<_DPeq7JcKsFXxA!JN9s_F#K!iov7 z!MJ_PTE!Rb5i(tK3m!X(e0&&VI+5A4L>0>Pe_$B1~7zkCU~+!AWvPJeP^^#iJN9D zKnP?p9v%+cvr79ae|MnO#W$70%7Wr4O>uC9mf05vD`trt!(dHdW_vx1e{H3nHuplsoOliCJZ#dEG=+S;ubfKum5tf zaGtX;#h#jQQi&hFkOC=j9_suW|87nD;$)y^&bq|bfW7qcotGR+L*D%5ioO6YhX0V} z3Pw?9_((e}pvByINRPhlmB)qu(teX;8D>w@4)DWO_Ttkyxy5|77@+s6ty^NnztKGec}@tbVfEbZ2!=7bw|Myod$tBt^cX98!LtgyjO{`(r3R zGx~j!Jy^*+S-3fnOX@DxxK8=Z(r>ywcxTZ|q=(98wP2pD^P98(tQ3Jtnw=zG-qSRL5ZZH?t38{YJ?hYg-dEc$QUlTM!QeQ_lY78d zWnaRr(kN6@Yoe~&yQiGf-=x1ws6Dq$Tz636cU2;GSfh}(29MnLEqNkr6c1j)y1NMv zLs`=XB-OFEMB2SmOR{spb5wpC^|42F$7U2=#?I%{t0gdcl7#e_b=YwmbuZ<@ zG4oTbM59*HIi5frf%+gPxwX$`RUIqfvI+WGBoF6go!Cz{)4br@%BG7QE%?15IQ-+; zPqM%`?;F;iT}G|})+NTxKorvWb+o&$yz^zcQi?Qi?ZG=>_e%2GtpXBmuzESl(ZZ5t zFJP|cq#N3QcEcB}iat*y^+?SaE|i~6B9&#|K$UC|Fs?6EWb>&}Cgj*x<;F=&I=$oETg- zdNA&*&m2v?x+CZt0GyjvX>ua0P0KAcjr2L=cdpuEjd3hd@=Yyw%ru;%Uz)A+{sa!Q zvz6WDTSi6g6<9LI38K?U|0KD;7Sz(L?_`^uvSz>4LQ8`%5Ki4~sf0k``HU(F7*4Mq zAFYh(Bz>i+^VPgRzftnIqoCHLYSrL48g0DqMi>paVFkMy98^eXEaT$sEiGQ`j( zy_#Xb)x=$SFNmLhW=hvU*`W8DKR;?Lb(SsOYO<8|G?4JmK;Gj)l7hEyI%@<&BT7xA8KES9ioVLRUch~s&)|u! zgaxC&V~PooKy4b?Wwjh)-_|6DoP2h?Aikm5&?>OwHTW7GV#esic(F;{f>5XYD>Dtb zM+v1F4&H)Q;?8`;zbNUCWfQDNi$&UN;*q~`5DY^O-R+4$h7KODIV~ZJCH*-oy=+fw zF^hx>z2_j!lW)BJB=ZZ*RP8M2N!Z$F5}o^k^RHI`SA5OqW=QrXuE-LI&lf2VuQ|PZ zc5w*THcM{bCMCW7%>ZOksyq8W)|WzZG^usBc_b{==wc;S*$R1;6;~Uw{Pjb%9OITf zPCfw#KWS!W|7N3Q)997;viTJ3-Gx|6;jks$>U_9Fvnrj6GoB=u0cYP%b5DV%`_mfz zZI38Gh+g-BI$XzK7WrlBS&?+sG~Z?DGaE4?B)ciqcX?J_J|#I;_&?377Q=+NOjfuL zR(%FvOJy>ecHTalViEZY)c0iM7-F`D{JZ_$;`;6=gHoIDjER&y`iA8GtNTXi+KP9ax_3uD|>s7of;SENYB!*fYYeRvDd ze!&r<@HuGo0elue5PT7k2n`jD70Q2P5M!bm!J^PZ!S6`3OgKghdbA2ad?~nyBg}*E zSZhccYMw5>xS1y>DI`Oa5^A0gxl53S@(1%qbxCG{mZjHV;1_lU^Lq&mn}n?qdhPla zDk-FLU?Y0qKyuF;v=QoFMu<%vO*aAHg>lLa&mtKS1#zvN2!xjg!@t^=Gv;;0v#Z-I zNs1Jmhn(D(+z)t%K^Zldr*cgRbzDa6l~O4R=29bJmcQkv=hVTWiye)PTmCH?`;HVo zV=173#Xoc9sLr&6+KI?6OkuV}M`Jrx;>#_rddPVf$a_p7uU&Lmq7A2F06nUT=Q`Jl z-{4SC(D*hTzF}1eI4^FOp9~n8TjlUWrZHn~xJXiX#><{|c}A9cMAb4t++3;nR|YdU z{tCeW(^{~g0Yrx+LmSA71%9BueoXpKr(p$D!CYyXs+RV%L)wkjIlgY#Jn03o~qgkO)Hg@GOB;Vx(y2U{Iy2t+R$Goxnw7ovhR4G zvRy8BdU6c>;I_GS<+fkAs`{0pO!#8LxhVEL*<+Rx!ggVVUctH30HUCsD8k1nNaE7M z;IT6IJ1s;Cu(u6Flw|$asiKn#BUs)}_`sBZ@0w`X$Yk z`^Cfc7+LVf&J{Aq-(J)o|BRHc@5ba+?8W@48kUBc#Amq8F@dDS6WFG5V5Xu!|4A>q z**VDirZT~^isGRgZ@)VGLVQ)b3~H7056AS>L=B2rKlcIbcu^ao^WKo_LnSLajHg^6 z#rNivSm*>6eoZ)8sDERTWObjL@tQ%BmTcfDhk;KQ)2py)n;~M(Uvo27vcFB2VP1F6 z_?pAKS&)j=y)rLbHz;ebh6@LOv&vf_&>z{Pq6RSvsu5j;o)pq*ty!hLgqOl=ZKsI8 z$JtQ4(qau*8uZN4xm%yr7*%VN8sDjkRRHy38utIecg)sTCL+fsvLh%bAoB6;Grx?T zg3Y6+LRXuUed$lLNymxh15Xa3w9_>+!Q&dgcz?H_a1Bf!iA=uH8)Am^OvB;%f!5kx zQ5Htg0r;viajl6YYKO}b!Wq)uqz$p44G~5iN+NA75XB!|Fu4q3Qg-?6%A4N{3pwK6eJOS66l>@kY8hvO zZ{?4z>O%ja7y)r%BD{_0NMxwDkiT8#x|DK-yk`TxqOD=wv2y}4uI`({%ER#oqtHSW zfH#t{=8spA6CdzrENw1`}!*-LqWs1`#8-_p?qVOGs4f*himA;~E_C*5-7R~%1mNw`XXM?Zcd<`IKM zDN;a+_Qfgg=3nm4&H57o|AwHI&<=eANc+MomKhFmY;NO2^XzlP-XXm3h#wkI{zRLE zf^Lm2;nMG{{{hZyMNf%#dZ3@V7WA`N>dwhpwV&%|DQR7`RS`SFmb5lvW^>q&EF}Mh z&*)yf`!};v>*g`fK2usDetl|{l4m>1C|f`$xpw*dJhj8YAId9PH#Pj4@HDCajQiIlMT9~G z(*P;P@ZTwh*$I{_#ohwA&-S0pV#-N4rYs;`fi=$|XR{36Bzysts4f6LOQ8rgAMWR* zW#6}J&bc9n^saE-qeOTfiRfAkz30kvQSmp;ZEe>nMZ(K;N?muET5<93_L<^-x_-;) z?)-91Oi<`Z|2J>>nJQx}5Uz~!l+cnB66Mj)LX;c{B+O}`O!*bS#R5y#Q8^uYXY8dx zcNKXM5CFcZJp96sp2Ad05{isoFkFE(lp3+tl4lI6p|*~Es%J?<;iNTy21{ceq>aH4 zNySp=jL?2?^lTC`>FAR?bQee-()DiWwXw|Py5?y8x;x6lnXxA^AtN{CY`rHmegv;^ zSJYGI0KjR=%c!zb2|D2M?p$C~x2;m}&aP!&s5@7?JXn_?m)Mp<6OLyI5AFA60A#9W z9*Xn6IhEu5BV00%{7JdOw_i_43i@4kv$Wr*Q^jc^CdV{I!Dmu#p_{w~$jGaK81t9*rhN`@c` ztlMwm`#Gg893Cz_7V-IU}fqsDOc`w=MC%chL z$)c`_8r3_F6-dumnOW%GI|pyNPx?fMUZw;wZm z#`1cB3Yn5{SFU?0G_E(JBq+5xhAAQF#{Ogs30phTRLUD03UL@dR`onWaVH>FcOwP>OR!JC7XgJ7 zs0>le4B=!slduWHp!-y;733LP*Xmz?ZBZlUp#)Xhz_5DOvRJ5lYKguQsCq!}WnS#Y z5QKU$8~~obSC5@gxFF|KhZ^Z>M!g!bt*`(3%6AA zKOCLst%&8b5>m-t($qCe;nju;H6-m!>I}TY=Tmj*P{h06X(z+SAI7F~2S*C=QpG~? zUsbCqsgee=cBchisVFZ>8I`Jo=P?Ut3mba{c%G&=HvOLGFB@jNkG~D;QCh^0FxbWV zJ6W66y5cnf^$P4gUP!!oeoJ$J$K%Y_Un-F$Hd(<1Qi=M%Q^|jYxhXjHNU;Cyh=7EW zQ4|!b&TOfZzG#ZtmTX-yt!x#~i257kuO;fW(_99MC7Az_a*3i6L1Ct7?spP3voXi4YBQCPdRSp0x^s8&oPmx=*$EpPofJ#m2|+_mcSwdt zJlG3N3*CgPdY6I?eXo};ERK60+xR4X;d=I+K|~_A>D-ZmcVaqY1VcUAO{5vV9q5QE zZ=L=2Ykcx9)7{Pbr@TUi0Hd;S#6Mg3`y8oRf6{xJ^|9R7nP`C<@-*q&%}0|*o|#nH z*lyaUF0y#cO=bn`fn|;3Ri=~ws8j?JoANz%*B^DgZZ3W*CF)hFO(je4 zOM6xN__ISV_ytCQg`=!fu4*Trw6%i?<05%Wh9^*8WvT!ngy);QIryX4h2QWmpR>mb za5*(muZcUA&;3PX$va*OYXj|7N)B%5t7~#59r3$3Lyq!#=6@60yrZf*a`K22YyST3LioGv2 zHIGgt-6Ah%lbzSPB~=vPAcI`(ly-exl5r+T6_}C2*@pDUsowTzuMg6+4iT2Y%aZfJdXa8VtKHWthmpCfU(A!4 zL3tfY2;5PrmrWqi5r$Ao6Zq%XWIZuXM7@`OTuSrskWx}cvnL@)jg0J{h)y`kD0VCO5 zXMc0AI0T&?_`1C0gQ7c)sK0Dq`>v4F`M-bsTw7IfL6`%SW-?TWcG>a z5=*8>eKk#ge74t2E`Y)+LwiC_EO^Xl^HvGJcqtmyaoO}TxN!c=PqkZq|n?l-dd$TeNCjvvgESdUv0mx)OBVJ?8 zd?B$8^9MH;)lG*TPvHLzpwASho$NGbIqLg!078;UXisG}q@lGcAt`W3Z7e9A&EN}F z@Qn78YC6~f)g7~56x}oEmudv#&WaeF8?-`rVA07Pe~1^1a-d+(-p7g7FMbkMMZ|z-jyTjsIUXsX~q~M z%$0Kd($3{ZG9z)@^(!&rWHkSc{JniRx*c)H;J#2fi23pb-6XTsY03q@Z5W-;88{G% zDI50*AD!2y`2%_s_S3(pB;HBdz_DsODG4m-ZAf20hprMX)x)@&>sxI>T^*v7K<<;H z66GDa>pK62L!j`d{&?KvlxyEduihKxUxkwV(XaS-+`Rnn3g!Qur>xW?!2wmuf3S+_ zorxBaqIR^9-CRS6-0VNoX6eZHX|j2k#s9qGBZW2pdd4ltt*+G14)-*}^uWbXp{kO<{P zk%e@HF%wWgB_*MErNS{{Zg#8$_{Ku_P=q~&Nv>I0a$#-SofEsTd`7&n98vz5>b#t~$rGo=ztGTP9lTU)D( zW$9>|d;8fsw5WQ(SFju&=wbqpWT%`-6Izu}9Em$~jJU2&)62@*?9%sfn~Ky^2k=BR zO7P$K?Rrc_Jw6=M7R0kCOEqvpT8HtZ`BtfGUBA8Cfk=lw+lIz1ExCGsVjSYz1#>fO z4Q3YAi##^3|F9xF|D@qaVBi;ThZ&k#^aW9QoS03xxOVxf$3|D+AO02aZuA?QkjQX5 zzGg`N2(X}U3JTsP;OEt6JRt7&gZr){=1Z9=Ctj8u_WH^0lgK<5)CMq7v1(H)u~KZd zF%>nhE0^IOcgFoni+!acAGTp_Ng*aeQBU(g4zhFFjw%)Cfn_6(&V3P=CobaN>KZghd#od?i6&b3{Ka9Y zoHE-l2p_l2$C~@%@O!Qw`eh6sgVVnvyLqxFfOm?q`8jhHQahLdy->pxC|hE1qZDm| zqJr4eaR`nMP@O9X+QoYXwM&8iQV?Cjcp0Li3dstI1?*j`)l}hj@QH9baf$sH`34QM zFJQ3exwu=ogu&{kRuOke`sG-1z5&n{T(gsK@eo~<;|J=K2)CZCF8Y&0s*&eVTpMih zn&%w(z684Tv8ZMMSE?3opg&K{lL(7dy8F%WdQF<}9Pa&$O}g)KGqlFYcJP#>+CtFd z(UkG(Na?f@`!}(zY;eQ5j@ibveinedp^e$&w0=gi{Xv>2+gbzXgb&=f2osY} zF4zLjnNir11cI^N>AnGcS_pHYx=Eu2CF7v__4-`|JOBZ7DYQ&Pw&zOMQ_HxXN%Qug z5`GImrMVS^EB7QrWCtqr1!?MSYO^Z~PnMFCxMrr1a7wR^Z}wL=k)w^ug7P+nIe59EO(G{2R%N&WYKxaB5o zdE)Q1`Tqz~{@eZVH2e?ub1^}YK^KXXTZvRoUYu&1Fz7Ft6icJ}!!mbo`n12RC5TpU z`FwVxP}#!wdT$^W@P&{^oA=pa>6+8-al)A>ja z;PVmc`(0_jlYUQPKn3ik@^w>y6$C4+IcnStV=^F7bCN7~+Y22YIE~~gO1KKrgdDgH z(9~B&9*|1fmAAnXmJ-1^V(SPnLa5%U3t4iK>Z5|i84Ti%NnzLRk-^#grrwv)OShL0 z==wHM7aojkx^wyjcP^VLL>6Ph znE(o>`rQTiF{)O%cB4fvt5K~Z8BSXjL)}RN(aGQ&_My6A4*d3pal6a(#@VPdUt>D$ zOcsfl6fFl1QDLicD{YzeqJGp16v4#Y=9`qYmUov4Q^e%9OJLq|7Vf-sjw5);t-eZx z|5u?T)rW2lAg{}{RXK5zG41){>Fe17*2xYcf;LN@rXb!>yt8j!m*>$nvd9^P@{3Q) zQ#o=R7BO3W^i<<>8Mx${2TSCP;PDpNl|&3xjuxZX3(PqrlWZn&!l{})H%RI1edJOY zQ*%(%qqgjEFuY7Knp1F_$*3L{M!t|7Ix|9(6j{wg+6d_Z1W9+!Yc1n}t10&Yd~^Xj z>}ajN_JjI_GOds!{09uemcdmV1K3y>uJbvk%4RM%=%Id#-_p8aT5}W%9XsG#Tyo55 z<|xNT#tm_U{WUjcD=9cuY-U6|BwBlHrK0bKryl)B_Y%h~z%ny-vA(!UsnN`0uiC{qzXjD_w4FP{vRDbT#YbqJHF7)-ias%s6G(q z-%cUFUi5Xv%0Hskor5?RQr<93hAdMjBpiPyu(BZx%-FaxLP=wCWHaTj)Sv~9cD(P% zZvFpo!MM>f!nck6;i?D%{49n;bUYQ!7rqV2&dH%wmpGtQrVNp<_WUhqKeRT z|H(Gu*dNP+HmWPPz!&us9R0}rB6tilT9OHz#rGYHcqH)5Ct|2p8Y_40 zK%$y%&D(o`-ZbBu2F&a5U&tkl^R}78p8+&B*2d6mq`}>Gp7ME9>yxhiLTtKRh|glE zWgn;N%0-$r++1*C0V%nRvnLGaxv+;F-@0Ud3)NP%-n+qCus8qVy?e;MWcgb!1Wd_UUGxo0HzN zBl?5=6|Ghpo&?~)FXv~#D(L^UBMG#=3@P8^fZxs)8f}mHp;Fxz z4i^mQ3vq$OOGth14=Su>MnX7PXw=)jZmC$J9rd^kroag!>tCV}N9~Svdi1t}-qv|$!P z?3{AjR%YY1#x`QHoZMuE{w$%tyn7Xn1x==VdCdJ&rGt@5UrG)659@$$6o`PvE-$U~MQt zi0q~zJZPz>55n4~Z$ICEkrA;aY1cv_wYm!;*4MzqH*~hy;?HG^1YB<=_2cUonw_sY zcUl9=TQDMImVFjXBycRI`O8c=SsEiokgiDm*P`v!-k^cc*uvuUApurVG|kks&q{pi z)S0gnG3QWji#t6P{+W52#iQk|s8wcMpg@+f>ol8niIT`&Am3htOm27=$SAil@16m1 zPW~S2MVXOlf&MwCDYNZ+6tW=cp&9r#0Dv^4a{u~FgiqO_MC^Za0{hmy$6hR1*BgU^ zo2Q0h`+1Iisk`tY_Ra?B>%Y8Y&o6N|Nu5aK4G~+;b~mp>>Hj?|k3W^56FsX1qBAN~ zfc_M76T{j-2dOOxYG>x-`uNZFv7x3bq@NnwZo$2;8CkwhPdK*@>zQhV@cM+E z{O9{0HyEN|1Q-{T1QZN%zN8wmowOh`7+MtdK4_zF;$NOjUGo*h6;qKQgRv8k#k&EZ z=s$b=(dZ22dvGYw3U4%fTtu;W=ASY8a&fi7MYlrU`>0|&ERlFNseN4ILIBP%4R!M4!-Nx#zs2dl2RB%y0 zjIFQB1()leu^))`y5Dni%RurP>`xydMVejeROlvnQQ)&Og83|tR~F?y%6-!h7PFCz z<2>NZXon0=jZwLV?6Yl34bf==T`FmS7P5&lR2yj<&ofiDo`pr}D2J9BwyL%>EpSF(p z=p5CQS#Lco3>Vp*gztLlxDOQo-~PKGkFG?wOp~lM`%1=l6s2#z)lC-e>F(vNmYO zYzu+nPTbz|GkclDzOV%4~D{eE15#iHeHBQU3k{X5a~Kgr@#n|`_< z5UO1NA5b;Sh6Vr6Z%+ROSX)WP&mrF0pe*g<-@(`$RZ_>_=8jwVD6pHr*whe z@OHM#A-s(msoWU%_BTMw1oEr02JHHNn(zpllttn%*-4rR#QYZGO=RY68Lr+#FO_Mp zoUNButKKWixx>)jD#LmyfE*%=!kmq!x@RWtXjvCG)Qb@M~)ke78U}Da^3p#Gnr9)(q6b0@R zSQX9oNdj!`-OeQ)XNO1t3%!X39*8zk_qqAY zJZG+!z=JORcMglGQd1iykt;DVv2lKf? zXeHDl1%YhTtK`$d4&U7)meo(8=1m#^1^j+GWCfQil%>v-o*1NnY_aji>mI=Zu-NId zoj}*DdGVsV`s$Qr#*1pEV5L}mat_f7nw5_g)b+OWH`1Fqtm|i^DUUPiUY()o_hWSF z8^H*!;ZdVgCLhTCl?PET9%Lj0)iJ+ zV>Jl5)Mr(QdiyH-u&T-g2}PxSQ1zW#ym+(MXW8V0venXx2W)@R@-O71>#@4PMfjx9H4o?2Cq zXJ`j`{jr-g+#nKsmDtBoJfJfyL$X?(^RZtRXU5F*y)U_50YZZnlei)J^h=uAnR=C! zOBDxTW684&-qkewrY)&FnO~uxiUo`84tp_k)uvTJsckev+WV(3vwkJhvSx#exUMib zg~Xuu?N*ojKwSQUBZf4asNxa(m_^DMHjn9-T;(%7<};9)+&K_>h7N6Y>{biGwrT-~}fn&9s4?(Xgmjk^VRhv1=ccc*c8cMA|KxVr>*4;mz= zzqQuh|9{Tj=VIRU)qLht{f?>{V^lr(w1kCnXUCop+D^x8;Z66q6U?01+jNU$B%ihf`7N& zucl7(`}&kWE1ByJghEITAVRZZPcnFyf~Rdya2S+@kShA1BzEnICOWQ+Ogoz}1IEdG zfjLk&vn3o~$wqf5F`OvR);kUzp4rz*OQJnbZ^JrbvLUnAz6STvXU-s*P3lS^NpF%) zAYEEyp)pRYd#DdwPybhGNQ`dRH3OST`TU#dxpiS>Ln$DG*sYzSXk}96jpm}@41UPl zfV&pM3tkaCXdU?%V&6`h+_EyZJ6Htu?|{|6#WiVv>E`tuvZ>{geB*AhTln+!+VSh8 zObmYrZqY#DTEh9p%gM_qGypr%z0A>f0weij+_#hCR8h{x$W_5m;uU23+iNKV!izU7 z5$fw;WtVgw?=TfOC!IaHEk3eWT_L%>qxbw$Epe(3%35Kbg>`|=bKmN{H)kI+sRNkW zjOp)D!mq+VZ`7IAIjbnk_sO+9+j3F&!wR3<4OdXk*U!rGFb90&a4iYM$nl-+OJ-Vu zk1}Ni5l8j-idV(2pNmpI#Z@wF@D@YXD+g3p$W+p;=v!t93v$UC)864*!{e{H%B!Wd zA6M*DrDNbJWP&3@NzsE>=A!8DVV;3x-;}*A$pijeBKXbUlBD1vud11ub5#Xj=2z`+ zYh9Y}m9wMW3<9I0872H_hN5xjy<=!zQDSvGgY(sao0>-A*lZUZIGJ20-u+ zlU3R9q>Gtob84p}9kJP~Mg`fBC8OoPtB!C^s2g;p9u`^)fKPN=PAg4+yu6-ci6K;D z98pnG3nky^{wNBi#Ym@OQj~*5?`OsrK^i-e(GC9+`3WWzFBM&?4;V~G6PBsf5XcYB zn5mjL{0&-#eRvi-?hp^AsPQe&w}9sB5*_UdqF;}VB_55O5o1*sh)LTc*F%3;?kE=H zZ@(hp8!@_hUFa~NwZOe8p<`{vkkY9Ppdq|t1@Jy@ZGYo%=yQUndg-YC+^KwjYn{quGmS#^37}o20t5RH@MXX)c2`_v< zq&x(Vdm#hB!V5xgYPLoz+<2qOxOQrT9ItDIfqEKk+#Uv~_lwoVT?5K2k(c3X^Up6f zX8Rju)<@;Df5;^}YO_+sj%C)?sd+>CyspxMvwIXKCQA?3flFLBp|mXOYlSUH8o9Ot zl)crA;2PnKl)oFS+aBc$T)ce?wxG`6Z@tTP+6MLwNQ1S}Kh@O^PY-`kgqUf8#81VR z&i%hdlmP-|;OI|QSVy1%1!!@&ApDZ31(slzKd^MPa8AQOF%gMhyz3f`RFG|#U^;6; zlt4d9Y2()c+{s5Dkwg-F<}1Zq4}=T&^8oQ$dX0U=e%Y`v^v5O*T2KHOONeU@q!I~Z zkI45hd&(*BJBN!V%q^wNTbPXBPI*T5j7=2$cCCLu$$f%!CD2mw#pZqSyD_f^v{z4Y zF*u}kd(O4~1VVz{hVF*>5vMBY{$zMbr?{r8nqGlk6VCQ(3ATQF%bNp zp#7Kfs_6tf(EmML|4Vr#psd_#ii3TE#387r4!p?lo2q|!?BzulJ?WTS(Z79y0J{B7 zBO@5b;-1yc?&FW?zT=#YoPWRy(T@an3FK#~gwk%5e&mE6V;oU4$?3*YhO^+KZ>G$} zqTs8Unw{X3$;uGrB4{E35mPkTV(6h2@zfiFM-hxoQVZn)j%*&)mG_0l|8WT(6OT9l zaS3qU!7c&+e_R67!x!4TUwfpH|8@!VQWku^%=%Dsp?kDko>*RCNqek&&(sgbE7Xa# zWKa)Ix<@ovP+q8ySIQg+%Ccg42TZx%?_1Ub1)cGI7vuULJu0(XKUy3wQ@mtqt)1=m zy*n}ydhk$Cm;xlWwvfIAXwEJMu11^UUol8ldnMCjl?%g%AAJ>^!C#*bAO4aN-?X`g zBY||6Zsg|nmPXE#v<3LbBgo`p@rqppI9+zbVwvA2e8eR;4I;M08f8~!C}8I!4O0Uz zddE3s3HCL~LwqSU(iFo_zI*2rE@^DM$~-K(EkWsTK^AuyhvECO3S#``&t+w?IL5+Ua!U8MN{|7>9>MaTtk7FQ$K~^Yn7P<~aw>O2flf#ZawxxZen8&Qum4+iv0q;x zf@RlRB)TZYl{C~2?MXi-*dYLtfn}Gu#M|Bh!fWe0a+&;pTLgWc;etA?qL`u~pYur{ zTMee?dfXAHBaI6&{w_>vIA?)z8s28|XRh|7(O`DM`(%LvxErVbsvj~-Ed-bF7$JKd zAqzWL)O=F?4;47CmK>YMt-dOw4M^r6-n#z$MpcVm^6{DEAE4@smMZvxf%?DbuKzP# z8IS)pT{koq#IQ#Cpf|)~Qw8bBaAWR^uvDFb$uyJ9($w&XN5BcV5^Pl5@@gNnq#iOs zN%$kyf9Xb>9Xnb=)k(TGa^AmvnaSJA`FMYN#p?&#N|}jll&0E0IN=e&PQehhP{xhK zkL1vdhs0UxVi+%r#l1Ep(sj${IhqBDc08c3#m|#OSIk@lz3?G6rf_sD3;i+X_*EzFw5kz5=Df;rhOH%CN=IWu=u)*KCO)DX)lL}~9G>o^#VN%@6reC}emNcN-)yqqo4O7q4H4tK~iJ4oeDzxlm zrYbIskmZ)qrh{{WBS9RBcdh%Qib1Y#Z% z@19jSCsZQLHoB6(qM}eqw*}s9L{}#6q7C0#Y5-Ac+=;a*K=4yz=gn!!f{c3}=*L6bLAj##oFlRr9 zS}#spaUs7JfqyaMnS1a()&HK`#2e#wmVuK0kYWKBrP;9s zDm4tuWN9-Qu27vVPt)P~={m`mPhej+nw4uVR!+kIGJ+#-gWKG}G8I?2{b}>p%hRU2 z^FK=N8;r^4a|(5ec?yykZwd;^y~;>x^a%=W3b_6B&`(Z^aAsl>k=bl!n@IAJKoiyQ zV0kAD4xFUq#%OqaJ)y)~Bu)t5)Pu6X%;ZVp3RydSow)aV?cG4^OYYB@2X#1FMgUxJ z_-eit!6~44NTf+Uub0+rGNmD_6vsp6oYMF_Znk4(zO9smy?VUZeO=HK-&41lKRG~c z;U1$IPSInKr6lJ<=6ma+LReZJ&>-En=ScH7E|A=Ep;-t~q5P>Tc?RX;xko4HkcYOCqy|;L?3*$y4bKp;8=xyQj%Xc!zQp|0W6OJOQXvDU0vO0 z-cK~W@*d^94i%0JS$FphpL4I;ikI)aHq;os^2GCvma7N>`u#W3vi_A7O=*V`GJy`+ zwFTZKXWVVAvhOrXfKZ})q`Po&VLAom@Gcl1D_a~{)ZWi`^b~^4qU1|+e{+vKT7QP; zw$3Da$GB3KfqgT~-5?-v;1h}I>hS2P`zGPhp7pQw)%y2(c9mpgPtfY8I1Fo9PTG#S zy7#DO=-AG;lIktL=pAL^CfwZNcN%8CvtD-~Sq|A5K;=tJ{Fe~pLE(&}7kB5#@s;DS zy1hD}$D*k3sHngduF;SjCUOTI@fTQy^Z)~5TH&V{C1PHn_YEqgEfU!a=|96eFiqDi z5RBIUg&O~hW%NKG{@*Oa!J|5jR-D8`h`(eb5k5K)5`zltmyyBDqJ8t84nk`(&9?rg$!n~gVf2^y2s8IQ^B{zu07UGbl6~2dwC%>5`_!~EU>#>C&)H%)#({Kqnx4g&Y55VK!TU@|k^DV$(6QFW zW&Qg1MtwxtTN{t#W#EPudT&+V(Nx1bh%#|A1>}_430U<;^Q1ZSKzsSI%z$ld3~Y0} zWV_Zr=9f&+zhRz9^_}z68{pVAJG=4cTYUmCo+&1@*q0SwYcw*gZ*n|{#koVv!Bnc+ z>f?I+=^C0nlBvg1TJ5}o(ukLbi& zVYEhw7@6JA%Gt|GO_JV~oYG5&#LQI@QGU;TL3)8BCOyg8#%ij)+GhP}2GsRIr!(vb5#XecKTm6>=% z#}f5L_ag8x;>~+UdBnc_xIaldS;RhpezFS2c|mlJMF5*-N9<_e%?grB)C81p>GHW1 zF-j5z!1uu9(!Zjp#MGK>i;B}0eTn=(9uod>&d@9VeWwl(j|LeBhHm)(B*+5;U{y!f zgNF03nj6ef?2plOW=d2OOr)VBnZU-+loPnJpUhc8;^50lT(fEHZ7rQCYKQEk1Gi7p zw!Kog96@D{>m|0?E%fZR+4J3i>(}qsuW|`3^tQ*B<4s9N7h`C1SGj!S zw%T1=%8biE>**nIFOc61{Ux^tU#R^r^&#Y};&;Vi?0;Ji7Eg-qNF+ z=sMjDIzZ;eAo!PjBSv4c8MX+uG+MhtBdDfk;JCzQXr)v)0^FH{{Uyar@H>!d7{&if zbgtBzJnLE6s^{*d`o#!o>1s$s>1xDl>1sm0qU$Ho)IG8Qa%s{4dTG(`^uy#FY=NYR zP{C^ZQf$eww16)8VNP3EsByxT2L!|Uj=F38%g|+vmC2;^ypD8bXY5RHu8PUCzOWf? zB&_2En`~-_Owqz?0m-1fWw*ol zzB9LF@wfD*`-(@RpU2|7@`*O6<_Ul0l*9(oD%7T$50oTl;Xu)Pk8i6mHNICPo-0>f zPQI32M~&soJW6P7lsJ4ioXH}+^pY7O3Gsczn87!fpySVS$|kq9r4n#M?u1!3_NPs~ zJSroBAMqhHm)rW+xG;5f-N7+_Nb+=De*$iw1=q3c-lR39<~d|F0ffWe)Z4I4$Yc0p z_iN1qB=q_X_);c296##EI{i3F>walG3TW5%o<$?-4|dB1$8 z_$PsK!X!aLZxTS?k6_ff_5A2-kY-W~PWcU|09PD_&bGaG?wgY@yY1$5kC|Cm7dh@0 zdzZDv&JU}u(hB7D4~!98;c?m7eZ5Xyyz?G?d^dqO+B(k5c3&Wk`+(!`s;TNPhic!- zre^JTS3lU@56Q1>5B=r_=3B~b*-fX{p{ZEBy31GY+^VvcW83j~IIL)vS@#1J(V@RX zUNM%bsV7{SFdgPg>6I&LO?gDdkcPc$?XrJp0S2?J=QogP3yG6q{$Q@_luL05Smoa5SD&y`q?@YmecOm(`N_Z`rPLx zQ?=dJ^j^Bg;;EZ3^K&t_54i(!_CCe`4nEe+?`c@iHOEyu#%YR=FX0FprdQDZl-;aD z^vAt@8T!J4hChA1NzGCD>4;g*UuO)rFId!}x}8&O$bQ|x{v4-?d1d61c|u}Bi0^q8 znz)%8)|r0_mCSV8aAwtGtvFEe~b$FXwY!)9R#q2Eh&N`gpSBoLE(5E)tg^Gty zIc(QKre$esavWb33P9n-^2eoTm~*`wX;H%-jCcenUahUN5x);}_5E5Xah$%kFe|E* zCtGP|)kIeTzgt!e*VM??S}2(CAP+pYMViU3Yo-dk6jlVZrj&7}D;K3GC`K`uPfwRh zxU!;9rx630>ek*t99r0lBgNl*Cuywa=l5KFL7EY@aKn5>8_Ma>b|)NJsv@Ul|0MGC z7TakmCUn`}$DQeG9>&8w2^BcSN-;P|AqfUVJ1GHQ5fl|me`5C~ZKuesDG1HTsenee zC>f&0>ym`gEA9^7t11xq$xl}*P`F<#b++lM2q3s({WSQE+t;q3hl|w8#xYYP#3Tgq zk-|0v^d+g>HsuU?jsGxk+FA*x({sParhL=07g1DElS)?>pA8*1emF9#P_&2tredg4 zF;of+DO`Tkm8Dw;u{tBSehuT5rFgk}tdrAqMy+OWcC&JFF)M}2qVxx6{Zq|d+kJhxhZ>e7+goQuMTId-tZQ^g&RH86k9#|}V^>hPL z&=1_W_*E~jiwct~iPuE~EL(x^6$o3=iFE?q!`SRs6@ zc;(+Q`20u52>&e;xd67c2)#RF4oSh44RUN9F|tSRm& z^Gp_XolD1ecBbPF`Y##2Y8oX6J$o6|tmb}V)1 zcgE&GUzFtHtHJk_3M@ToPQ}_%;2Z~v!$F1LElPM-cCm4|f!nEDW>tvubCQ!#GHv{+Iw+8XxphEDZ(wr=Gd8mlQiSnTKWd+%hR=K*2pA%GX{CdB-B+U#+xImPcDlh zJ#mFs#}$k>K}=uy{m)P<)NNPU!cp?x$uc55^CkdcJS@Eldoy}x3Yrz7WMayORc&Lw z@jwz(ixWqCbsYoMnQ0FQvG20+u0YQ+TUA2`cH01h%*KCgU1{b+2E4% z*RPmbS^HS_lt+coQ>(xdiPgA1i(d;F$n9!b`YtH$(`Ksu+#WE$X9^yb51ZWmSgA{_ zf1LX$t8dH(dSV;VE*I67892ILPn2%8*31Xs$d0R`)eiarp-S7NQUR@#o9G46;XqUN3tD@VPJA z|LqJ7eTFa^A~rA=9JC4c#y->w3LP5`%P7cj5!-AFmk}$ZRcGB83m}V&oo%~PP2|XI zRo?(2C$UI7A>g9SLvJ37PaAajq;8(guiuWl{2q_%uSGu{G%}cJ54oeWN?}P)#^|Co z;)&z+8g9l)^gqx|brqwzhQx@lz;xgpn9+r!`I^~`v;y(;UD9F#;0J%YvD570T*+q+ zIl?a_Qcsex?$RS*))Ow(6Q zOozc}UHdUDrw(svQ{N}MvQ^^!&Ub1))^6ai2WK`%|6tI@uF;}8y6!+zVC_rGb%K0YhF881xD6^BkfiSl)FJXX$SoLT|JqU!H8v?uNRlL4SE&rd0L% zp;(`7s#Ik9_u0_J(jw>t!n^amqQw2-GBa;dt!#j;wq4jHuA7H*Yk8cEN@6`F_zp{J z^qlq~oX{=29a6&xZem5}W#*MN6|T(8<5zV7VvlQi#V0&l6;l>@b*JfO&bj)^i#j>| zIL*YlG`9k{P(v(4I07aa+Uor~mMn;J(7n76O~5dmucd8bYMrNwRb6HpEmJ%Ty!3_! zuR}l_u0iCn>}gQEnkn{rO;&k(5P^Lrfi)ap5lQl72RVM-5%$`TtF15>^d%WB;iNua zCTIlb1rm+zd);%8q8x9aU-M`rZ!l2uE$XJ9(%8`Pj#r9<_5^7+&hXkj?0&(1 zSB*nzr#e^4_OJ!h1D;RZ5AI>eZVww6Dr5$VoZ@Pm$XaS=lp+!ohh3ZCIf%3wv?wQk zkd1sXAB8Z_nBL=K#q6BdrSa9f*^&4uRO-s_C&x}o&R~s#&adLQ*s3oB#ZW8cT221v zhW8&o;?n?5?K@c1o&Qf!S0DloO2GpERl<@Ql?U0-!v8Wlsw0#_|M^5FO{dq;)nIY{ zNi%xrlbKhF#-W*m3y9*@hS>WP3Bsn+Xpb@GMmm+G&?n9T~O!AAO+>z}9S291uo8ZluK+T8emF-(Qt0{O^Y47JyB zGyL8&99An0H~C-HB;jx!Q!w^;6tU#Z0bCE~%uBSt825LJ``oM;HC)iqP{ckw-umvl zLP=0Q*VDog2upS>h!-E9_|fjJ{-Piz`ge^{!QD$J1UT4t9{l`&&XXp+{3}!j4k+p1 z>||+b=J21~O~D7}6%iO17)cmUPZ%6e7;R4&9vPV3nxQ7C_ne^~k8j60Z#|1cB{`Zh zFrPy!fa=L>aK2P%0&Tg0m>+e`C0kp`3CwB>T$EMT=7{!Q9KEZvmtM!-2_`l<{;4(yr5m7Q;aS@S$y_qjMm^xUaeUw2BO4WPX z;sey{c#EG*wd}D&qb=u(+U#&6EGUX4VN#-`yRX-K*Y%(Dt6N(&T~XJ~f%iEdtD+T` zy^>;Kig4b_3CI4lzqgOKyngj&2y%Wr%;frg8o!AkJl=dBi~F+qycDPC^mZiUjABSP zAx`tgG;<^Y6t`ueY3xwVRWz@5-yLyFcT4}I2(3u1h^>gM$UK3Np2k$In`f#w*pyxx zUz%r0tVlWGn@*E{tOwH5*VWfs*X3z)F!Lct#kVo>VL{bfEhr+su2_w&*ChI3&+vUD zg8^k95?dC{(#K%f%lpf)nn(3B$%ZD;pa}LTx`rzRuq`^pVFxQASH^G{|3dQ?QbflU zCq4Ritf3g{L^soQ*VQ1w7h<;wWLI0IFCOkCpAvk!O+D4!`VI7J zk)rzz=t$iCP|)l4?9xrlP0?-Bkr{3-KDGE}u}rntK=Dx~CwDz3dHKhf;wT#GvSv*? z^+gx|2h0M7n>~{P+UPAUqH*vcvBWs&EyhP%qR+haZ`AtX- ze?EjYZP|9l%w3}<$!No*cKWYS(Y*k3G|pa1ATsambaM&3b(wyq{M8&@0-s=NJr-y0 zsvyUl*%dFGi2!9$ESNDjsY$nXMYRPS4ok!%Cb4ZO>|T76Q&e0H!|6Yd%M=nNh6!u%{EX`4fv%mDZ^X zP#s*yXm5IkvdP9yNC`Uc8g1pA4LGNSB(R1%c) zlWSmJEj*k$WN`QpZ73#K54)DJIAxLx3+xV-+8T@d{h5||V8i_I)rc9^Z;3=-K2?!r z3Q`z2D}aqFv#MV)%guh79_(vd4aau}O7MyfcMaT-C=I{$4KV(W7v<8cj*T>}*WRgW zm*@(%O&BuoG*=0d`1aMU8GX)R--y_EF-wEZ`X1+MH)8!cq^^H!zRgL|JtShgV58QX ztAdxHM=Kr&0hvfg+<5kEyKhaOFwXAvRLfGgG^MM&|wFrKjZwgDY}?!(lrR+^GR&>^ePO4tDttE~uRdsicUjWMA7 zXQ^^n|A7RgiPXhuh79?57q=?;(B+w7HTw#j3+Znk0k{`iO3^7J3yW|0;4x-$2yxyQ zZ5;B=wOZe-)OZ zn`vj&^y<=dpdu1<(qI(qC9pqN@e`0Dn5H}4E@W;soVY3!mRJ8~;eSp6-bsAf&bk){ zz>?(-<)`0s=gs8Tco;=izM;!E=g%7j4Sr7X|5k$VoovcY#y3hZCRg5x#mI+iAw$fx zKJbGt6o+a~9v{OCn^?Z?r)&I?e;7PdYMUrS^8)oFLa=;WW_+%=PJ6wJG6W*kvyf*3 zn_+T4QIs%wb2HT=*Q1;Ra6l_a7+S;3T2Wk(rkRuKkq;>J|xPT=Q($&flTF42q96$huJk5lc|zrOz68%ZuSE0#A^87z$#jT1Z(D zD|94S3*0T(S&Y)f8kMe-i&Uwa)zhZZW(S|CqF4$@K=Lbwo@Q-~LG&0vtBS=R%I(ku z-*|1NkPq(7vx5DO8(dDpt;eq^e~pS73bxmFMQz6X{SfvSFROkkTz)ChK-H-ze<}?6 zKq$Kd!q7ZzW^Ohw0=*DbNJe`ikVTuS88x@Pkk{I8q(l|4?va}Vh)EQxNLR`oimV48 z3-yWM(l=HTN>Nz}guVw$KGVVL&^D{;x{>;L6*_ds^@x6>=1tn~MqbJgClRJQGK9Sx zMp_39@NUZAto`deMPs>(xE{=>PdqsPy*Po_Ca8ZcPXFvvgLCiyvo19dAi{z>>zx(2|PTE;c)#U@^!)Q&6A{6l#@{@IEGYtb13tBKH+w0D~BDSqyz` zsVD|=zx+VmiPh~c^oOH*ycu!!Jf8YJdF)Io0?`pxOHqgAIu&fcTNQ?kxB{>keKs5H z4~`RJ;F-iiC~wG=SQO${UNV0)yg@e~F zo(#T8w7*Pb9}hxa4d*VC-DR3Msvs2lkPSXkMXjmcbdHBmp^50|JDfTn>VIE-kV1pG z6_zLz{QFZhRfKlYH+VQfk82x%-QKJnI!*@n(rm$Tz%YS%L7YXG@DvP8#ai7ZNBt&| zCXKFBN&5u7pz~pr(eSn~_)fZ#keVFba8oY)qbqUpnw>R$Jg(FOT7z=il~x|BTk^_t zD|@3IUis&j%XsXB5%rVr27ighoco%^SKs<}Pw|KExQe};&z7vRW#A_&LS|=c%FKOh z@F(e~iLFaQzcNjcUYs@obK<=SV(uT9Yu*G8;&|!mNx0*K1!~65pE&VeIk6tgU8+2S ze=p^PW2TnVeVb1URV0|`Vy{R*5S9+fvzbcc#IzSEn{(^aTzQ1g4cS} zSioYz&J>yUzYA|^{SzhD(IS@o`;g*?CpkxUs~PL+QyiwZy0a?#HeMA{8wQm$5d-rI zZgI}6OrBWoq?yDQ&F0j3u$)jKhMU-h|s-o)QrNkq-^l+OU$;8~l_(8VmW!J5Y z(`rMW-)UKgWYhf$e)gK5syet76HAUZdIOc|l3rB$IO(;QEz)tVLe;d+H2o#hI+Pjw)!-^A3Z!WLLdVFB2O;}1pILX6=h6#=vbIuy`^C?HWV8}QUCE#+lJy* z*cIwQ1|jGELU=cq1&|W+RWkXu@v)lo$49GmCfl7*vSI2v6}jSPaF_DL!-h3L76aA4 z@W%J@NSi>wJzJLsKfx`0`nkb|-SH_Uh_?`@17)cn*#=4^%9ry@#r%WmD4_u38Se-p zV{g+k6S)Wb1TC>OZMPR@xY!Fsxc7Bqy08O4l= z_mhsz0#DsTXJK*GoVRgU*NoRHEl{_p&VKE?{hAf)c~^D!`Bm4I{$0NNuk0U_*`|?K z_4(G@-A_606ZB7;elvp^eLipWpXicj5CUt-$cTYrE==g|@HxQylR2s@`Mq5Ew@oTn z?PNYdc?U&|?&KkAc{7#d699$^Rn%XaF%N)YO_ZqZo;l!64-+^yL<@S>!~`x4{o=ay z0{E9j0d4m3LC+a9A}d3E+_zygAH`!ypvlDpIzasLfh%Zo-Xe&m+_p(IWr16-WXueZ z4zy{~OtAzptyqN7Fmmgaj@bdGXAhJB)AI)ofa$pdL%{UHfj@v{_CNx_GJjwO$SWA5 z2Gz_Rr~~pU#`FQpwoS$<6I`8!DYUk3VKjtXo$4vH_HJo3I`(&103JKHI2x6u16z=g zQHr(Aof4pKsYD~?oNKLQ%nH!OwN?Zi(+1S7lo+H~+uk_<+H9MQQiwtKHcc8Sgj}af z78x|(Ko`pgG61Lf11o^j+yOsGNGIjj!od@_Pae(8%7He(Y2m;Va5a0N3@|PjL*({J zqVcl5GXoeGjVW^bWYMs$9AJZb)Kj{-ebQ*^mk+qdRDT9!A`M@>SkUbgj{zHE-|6s` ztyrlBD=&sEPE1&-2VoxFI6N9jX$NHF7>~a#I%0_d;S>B+gIX83 zIC#UHaIs{G@+Me8w>W(9Tc$xwo(a$+H_gn6Q^ZhZdSn?4Zk<$YIbt|LIEb;9q~tq; za0h67dHog;jPNxyE(o7#gMd7UvF6u$CPJ<3K^eLx7~K;PJZ8?tP5VBT&}BDJ5K&Oe zm@Won3B+xIxm)%@wiix_O2?Q?szV=vfjW9FZpPyMqv)CPCOAQ!h^Y+`OsX zlhz5kCgg$^LDCl@h)gu9gCwo}>W`Vw%2cWo0VY9~{;|-?X2LkS;y@#N6I5N7kzcZ6 z*wsdSz}%=My}&B`)NeONkM-4Z?XWfYx7Iz;SVx zDOiPYtXT;*^Uwa&HaGT%iB8e~;Z`2!#0r+FAxhXOi21@3!V9fq0phr!oasi`*edAv zvo3@N+yQz;TLA)Dh$GNJ^mEDTNH$YL?vV+spiNM&W?Ra8c)v4xA``30jIIf;N^}3P zrXRdtdcQkVfc%aQRDffUV&x;p*}x$ey3UP11T;=Za_)zI3{Y=0|8VF-W6D@?vqp$RA z5eD!|R1I9vI5+T6N0_=|CgefVP}(K^@Ip~y3_%VMm~bZciWO$?jrtO;iqAF>8t|!@ zHe@>rLB5aN5E7NgF&ZZPuUMatq2}Sqs?!$Zv@X1$M8x}PF*6kl;`;?4nCV>zx}*B1 zpqwQ8p@T5hcaVUdj6u$5Hx>{$5S36(@JeJRlqR&QN+vu(DeyTmJ01|LTUKou{pV03 zG&j}|K=FPJxDt#^v4X^Y4+w0ibc_m07n-0}D3FGUupcG#v>8w-sCn%>md&wsX}{GY zEd-GIh8F@zdxIYYM0*Cq4gEQ%zYpGDd`A@OLvcqD0{De|rxS`nW5>`F5ke$FOgP9A zD!sfPT(|{gOnE~N5r7fI4W&%&B7TN!rVBz4CtMBXtW?1g!}?2__MEs$^$Eoe!rR~ZrYJm-0xh2&det46p?g&zKI#|&gHV#PA+MGl+ z7_K$z)wizZjaH@c7^>Lr?n}6aa~42R6w#hSl=4$XFK~exA1;{|4CR_P6c&t zVesA~F^{CMT=M=N($_ckx-pt0zF3G!0N%T{DB-#(dh~^-`s8?{c2E9nUBkamOK1uR z{*{vG_z?diSW9REXoEe2?)^DUrbo__HL2svk&6Cs!V9tXtp+G0AANWR{w?{7Id&IPbQjbxbih@4stzq8_VU9XsnD62HknwCf+{7A-PdQ>=HZwcLGb?8~@r8Go&~ z2ym^^B012}JtPlk)%pnh<(ww`W>zgU$=W(@9FV2;h`umdSwm{ck}oElrh*Yt{`>%S z)i_RBG|Z5XZVb~;=F_urd?x_>M&jSP3fdxc1my9kz=oLectvbjH){#x4r@^lBh5QH z*444}>I-woHMe#Pc6B>YTT0hO&oj`feQiUglbc60B#i9!rRq14x*67U$;tW|#R5u&C!;9>xDAGEk#Y^0Ilrt8n)msc>{bpD=7*CotD<+_uv0 zn?K(?x%1>-Y3;PzvhLm!GgzLJveB5D)i)w$sqe0I32HVEXz#A|XuoVaq$~fpq2&bjCV}g;>kGJ*^#!KUFbx=sUuLCbNCdAWCh^ zK_`|+{d<9C<2?lGr)tlb>P8E@?yXfMRwy-avtenlTmlK>l;J?l^D7trUcLg$ToJGC zS0bHe0=-X58(Tq#bw9AL${tqR=2(gJC5+?xfJg6e^>7{*IaQICWGTPxV0h8aLMIIr zL?3%*eT=T}1+o*@Vt*PZag`lQ!;u@tS8+*YrclaHjtWs@HGc2@X0&K*ag>$163TtZ z%4=`OHq`H-!X*t{gjJHz2{%TA9Ge_o+aFv(E)^0vkfY4u=aTsLQ`~KV-x#~pAKBKy zqI3RX&I-GVW~OGON2{5hQ+(+~8$<>~9A-OmaP^7+HJ(;UVuw)TWb?9{Kp;!Us2T%1 zv%u$XrGviuyE5Sln=FBYtc+95p~dCx`Y7R1`ip9e57m7T6pW1_Z^u% zJO#3syAies+jo$>jlwSTj+}_Px;q<`g=<|LJ-l{H7P$zCFCRNM5OC-11i5iStaNlz zR<@$zTV3*2j-6Z?;EmbuWd?89|JZ%?@+h3SxcBHxdQbm>UB{x*-hP@X;2|p&@rLXn z&}+ovyz2&h2-zNu&{*R(-w{G7b+_NU%oTw~goIw50Q{=T_*Hh8z_OfwNWu`Whb>g3*@JBzzvD9ruQpj1j|k4Mk@HC&+oH1K zwbnJ@647i(CXCQ^CLYqJgEC1@1PQnbV%`el7uPhL&W%Q8C3yhs)AD zC#91`b5tfiYb*at4c}OC*xzze&Tsyo{d*M52q1|BZ$Zz)W+W;(d|YSZ%k>zXPLPR{ zM^3vtnzuQUX(%HjhRx%!lrd%MAt=k5%cEkzR+@PFXcdvLxz)sMc4X&CoK&=E?*k4V zni#DRaw$D@NVt$3|EzWGKuTQn9EqfwE~fF)YU~=zIoM{4{RA$wxHd zHIPXtq_JMIh*ChH@-;UriRzrJtQvsp6oEbw>Fv&ZlzU4VC-yPi=-h{`0`)!YdBx9H|Oej4_j?=5bfh818@iDGw4^%_kySPVME^93{hFnxADTgxl!uMF=XIE>Gh)LOx^I@?m9kJ> zN3t*A$`K=N-XyI3%Dd4H{fcQmL-3tQDJo%(;rIUbXN^DsQeh=-!5iN4csB*hXotLR zv$m<2jLk{QTrA!`x^6|xJS&4yvKcMiy<3*24nevXKrM~cD5M(vBr0eU_?XD9TcZx| zDOsRORUILQAxh&aatQpP_jP?zQkIfU9<7R9<&_IjW>TGCQpktTI7ct_w&4%XYyE)U z3wt_W_ws9f^Msj;ek>#WZe0m4p>Xbnd;3>g(`#1sR=(7i6_Cu2?Yv7H4?&` zGU?7V3HNr8DQssxqB}x)3KfqkdUR=BSz9}`(#cq6w_;B4Z>DCy=!L_+(!3)MTc{ivrhn~3h&5_6FXA2QV6eD-x*_p11;&UU&fbBk zmnO7^Ep)bPKUN?OXlSr<>3IE!VP7?Wmd-5H>6wH~tp{{-WX4(a^FN=fR2BCkRNzy~ zAPT`sGvWm_N(lGIvo~;6+%F6}Veks34Zyi`3lNXZIQ-_z#udz7)X}vicebv&&P~pp zn(7iAreOy?J(ht840LFL=za$8L)x_X6IqUR@!G%s z^xJz`b2=?z(R}Ce5lE_Tr9TLb7svR{vOWv2i%ZCzrfEk_QU}>ojq>t$``f^d0X|Tp zT&fnER}NVpy0m>G7s0E1mtdC*-r{5v4pHrgNwteMc3LYJH>M7g-l>}B6qyEa0X5J; z`dooQaRLNf3Cqyon*dE1oG@~iJo*|L<>tj(Imk$MyMi0&_~zqi4LTR}wE26ptQyvq z0nGJ-_Dle`{h(wMz*s+M-UM*k4{9<2=rxQS zUy+5SOc|q&^P~khSe63#?rAKjjT~80wS!}rB7p;J5B7}AtPJ)Hm3Hu0nk^lMrBloR za%g60Y3%cHHG8cAu2hWE_86)7h0Hq9*$ke5G?j&z*~%A1u8ftnB>;zthvP_Ia;m{1 zIiGUlM<7gS9qWxlb%lqvR+^Tk>5Rc<$)Y}@jKj&CO9OLW+mXIbOhSiuk8%?)GO6Ko z8SueZt$RT>R!uZ*c;6%z(9&p8Ox~7EPD9ScHh+r#fc|bD;e42`d{G@P9kWrTa^c4U006p?kV;d+lnTpUcCms|Vogj$ccX zU%uVC#@jcw;NclEb1Hq$o)lKy>y-bU3239)xf#+@97hY>^4RG5%A8h5K`_)$z%;)s z*d0kdO)0YbMm3l4{i9+O2KV7hofrS@sD$VrOXTFPv6&QVnmLZL>{lCK55Wx_yfP<$ zsAxHy&FvG!`V58@$Zj&2Dagx~sen^9kP2p7=DVY`IGxF>KNo!szy@qngx z{J08a7nI;`)*lKu1^yDNN=Vu(JIQlr4%x?!mrk;#5o7=aadq?SM`ru1>r!7EpKR3z_=HCNtk`nsG+;&+sm$ z^|3c}Cm9qtM``NP!{I84O7+*Z4|%0C7e@}5ydxme>UU^G2v+)M^@X`Zcc3SiY0Th> z<9>Ppv_YfEp|2tLa`DVuRv1hMUk{3<;TUmVJj7r-&2Dqyoh>mQGFOX_|8D^hZp9Mk zRI!ZFg^Iy|k)1zjFw-J#rOAqdIWPd@V>j}HA0d3Vacn*2otgQmh1Ek%Z1>D&U412* zc{&o%)!AEnlls_p6F!Eo1{{2R7M&cp+}!!OwaxP<_vFUg=7d_RhAU{Yl1h z)cUSH<-_G-Up`!wErQt}FCscE2UtW*wHpSh*&dcVgT=^OtdhwK2~=lYxs2M1j+)Y* zicY?so{ox^rjn{k`ktPa(x$4WQZrv~A3aNrsOlOhovTv@&lYG(BVe{XsCsHyx;h5n zwJ%9Acj>;ZDpwTiEjM-MIsSB=~Bpxw9INUjwd6Dngno3 zj8z;MPcnP1rH|2sh=SFFJ4F+saSelp5HPh+zF~|E}P<3MY3DZNn*oFo5oF>E?)p} zR3i&I6A{^G-CZRyLqDVpbl|o6GWvv=tiG*}_2WQt7px-P?TaJYDfqmdPzebyi@-<- zSJC{~l>6~{J;Tzto@jrl;_C;x)<}pZHY^78XXp^Jxqg3=eIVKA+fk_p?K&0R5^2;1euJnrF^Wl!hV%9;6d?%Fj(P?T4Ubr4)$KZ-Cw^Z!41N}Y zb9O?VejW<$HD|0K<>&V2#R$IHcLRM1FQj;VK9p{@Yj%#s$hrNm+U>@ z%eHB?e}K;<-x4!rcUA_`J0ew?nLHbr#PtoWF6G!$;XgnT~#)~#{sTO$KwX#W53bxzTlgx{L)Bputf zZQHhOc5Ht!zS!=LZFOvRY}>Xq`JXvAGiS|Q)LL~>H?`kgRlD|npWl-e%0Mmu9;viwgGd0eu zhb*S#!DvSB0nP(>=5Y74c47G;Z=x6}7Y7}PesFfiMiLQtYNKkoPE-UFwD{!At*xTh zt4Fr1LDO9%{N03;$kca%@Cw~>gy;*e6^up4t5Dt)a-ePF)!3DFWW;j+J7ARjl7nhT zp@`{w%3#EEO> z6o5O~o^J&Jlo>0h$y`if!McQ9M{lKx0xr(nc-B@HmKWx?6xMS$R=RpiyQq65{1VcZ zl@^yyamlgiqtew>Ry(KZjgbLPiH6Th5LP~89JBvhKZ4fYmRU?e=Z%t8*2ADfw9Q2E zBk*&Q?db%z6>VwD)fEnAFk^NoPJ;{0%A*E&XY2Sp|L?mXj-fh1m> zocNvFARunTqT(V^lmkhm4h@{@1G_LnXsQ|!EvFvqD!7&}Ivww&M=ng4cA}0=msGR~ z*Cqe~Hr_i~0`B}6bpr12;gX>1lER>UdQw9S(LsAR)--bAptg2N={Ajry^Ca@zvJWJ za^PNP7dj#vhyb`oad;Dd+nF#4UVA?(1LQOydPfDwLg zXM(o9W+saw1W#vVN>{{TVlg{alN01H=YcB$+g3Gu)f@@zChoT!yn*P~9*MVcN!ZeX za2q)tmyrB4ewM2H$7%jxIK3^K9bJ>K-~ytdT5EZac|R9Fyectl{H#j&YoB;PhFJ?h zg3?J(fg?s9W@2XgR>Harwo2UcY9yGGkxbKGQ95xGT z5wd@IFTVe`kh|m_?i}{_h!|n`KK^`pQoY$y3_bpZ_Vp(iJ%>157( zduB-cokPlLf24`uv1KvA{w$D?ZE}buN_78G$f1gK#0Sn9aud4D7c6?;AiW3BP89c+ zo9togqZlRxFGm z8+H@@`2zqmZdSJZEasFw)3O1$j>q0#I2~0>bNRVI*GEFJgXMs%O<~cP>HTU?NE>%u zb529%{ap)s_eWp62G}6>PH=4!P{j7Un9Y??W%(<{tQa4(WGYA|-a6$O)=ErU@C@?`C;{HSh`^oQP++?j6-Jw)8`!VAOwEEcj!JxTOnn^U~B3h6@xh5x%`hz)>IP=H!bz z0~_84Z-mA25z&o_q#0yG(@A*9xe8uiU627F@1Xrvdxm)X1YZt=hv8=KDrieA*dA@w zRbDjsNjLv;ee)-<+j1AB9V6>A{%h38xr6yrt#hU`J|GE|AqbDmEAoq-3A}AAo6O=w ztii?ksoyG`za$QjBLyJ{vKNt)jnDP!q@?;_?$77$2@1i2db8y;axeb*w+ANwOo=1& z(&QGT=BbYQpRO_3o170jHa4H>KOi|5{O_3I{EyfLB=1F{!JP= zTECST&RESY`UIrLW%#t>sSj4^=xLD&qj&O0Ni!=29q*q2#_;1JUK@R1CTx!Ele#RQ z)&DR$9k+}veRGqwnc5*9FSy5k?5AdAFn&jwJFT*P<#e9k_*u1GH z>cU*!jWtZtJsJvhe{jLRIdVjYaNM(AeE+MbL*!r;*ayg5_n{n%AOdBRJfi?s_YQ44 zU2**ca+DqbagM;Zu;50uY%Ju(9t@xi==^7?nJtwYwX&>Vu3x2`*!44kI5`LbIz*fk z&tTQ7jEZEWfj8oH6_*Kn;9VW*)3soDWqKjhsC$Cl>LK{@p()7yD8y@R`p0^x{PMFU z#6-K4Xu0#>g1SrW^ma)5aY0a*vk%Y(7`Q5a!(>W;1%-vpTi(j=jM6YAqjwETHjS~$ zkapoDNF$9@Ul2B@A2JR5tnm$yvkm*mBcn3Mp+^WkTr}fsFsxyJw1XzZb!mNu06tV+b-9<^5%xD ze!Eu8nVJIt;&Odg0Ol*cTI+c7aiKH*W55a^Kdmn%^H9-bYjZ4_n^?}xP20%WWpc%5p;+Zim3#tvx^Y` z)&wZr${Oen=j!I=M%-b>h*i(-qOBZb1Yggn=1u zufNfP)8=py`w8E)V5=%v`&Vp>n&kE~E6q2peql=_IM-+)?>3blD8FiWalxZXyDZX3 z3!rc;hf)=ab1jFC>`^$W7%}GGjZA6+_#{>uW+-ma2^*nRB`V@)Y?9d~Gmp_((e6tx`;#gc;GHzlMb_{c%cC~hW0HSW0`{^1VLEzH ztu5L6d(NuBs}v?&@oYqlSHVoI3r1$K-@#*U==4k7m!{x&7We_l3)ubMJC-IIlVC#u ztVZ`g8t8^JfVC;u@#5hOELmRyeNViO6aP?=E!j%SR^F-t1%`zPmk`@=)qg%s8mEdj zcO=gggY0QaB98eP<=#79@ka9lpiP%mZ6pAeOcm@FY>E?B4+L0otU4Q7SAWLYcqjL6 z*3CZeFGgVF{~3ggo|C5sO<`q3coAFB&7&*~H!JDQ4o}9il~Hq{cA8WRo3gr&md};y zyKxQjEqwizV>cp&51L2JH2MdA z@t3|LV5KS7IQgA~TgS8lIvuZ!J#VZ}KfveDLt8?&?-0B<%LE+?jxJvBUhX?m`azcQ z2{U=U)jxWcxbMA@J1=LM!)FP-KLz;zV4hx0_KZS)GOq9YBfy42jESRUQzRO9)&?jR z_E7X1ha))_#XBZR1?awTIuZ)P@Kcoue3J7`bq8UJ({zx8^{$Zs;6F)bcVL;br4xE* zUj&u$x{+kzow(e1jWf7U_MQql%RbFY;&^MGjV2uoxqEBWI!E}v?r`ae zAoAfONwcqlM9@hA&`gQImOkk8gkYWM8Bq6(Q_>@LGk&%Y+&G4&!%ao;J=dlGt0ajk zf`L?cau|plas^bkJGv7qJYAgQ`2mNQ`;W9Vq1cDTA0wm})hTh{6`oOi2BOc(ImI-n zB3Unx^QYNg+pR$cn`Q`z88`et!{H@IG&8cP#9vZH%S(R%vPwbQOT)ZOAhJm;da+G7 zT%))hu^BKtB3>M8w0=kqhx%oa4BnJTtn;fPw7D!T3f9Tr4F(3X0rl1)hM zVV8QeoN#0SH?HZ#lL0(+7&sxQlPG7~dNV)MU@`Sbb+!d&U_MOv-^E&X&EsZIT>R-h&sD@`khS_%C-oIzJ0ou7S;2BeiO>f#n5Y7CuCF;Ok4^E3+-JL3)HlF7$zvi_%rmS`lA82b9N{GH?8%aE@aMIWD~|I6waHXT?_yEme3mXzqbv8cD@D6 z!+#^!UYbkP{={G#Ue^$D&oPesXiL1#TEU${FU2mN z&%$lTVAKTR-wjJRdvx|N!a6K}vw}RBiI@g8GCpm@aB74@tfzYY=Pfy3fVSO!0KXW3 z*)diSg2T7WX34SIo%CLfoHr)XE->*zV)P|2vC2OfI1S0#$vjsBMd?1?`Rq=QFe&J7 zu#>Vr;Ph2IAa%6iP0bW*I-Q#tv@{PAgyD8CprOA zjd9~0k3E$?EFo}f$71(soaD1-2*?T0s`E^u$8cy+9pF8uyJ3*vy{fAiggjTcA;!7A zJ2$@}eT2LLeN*d*(49>44D-xT#1K6EPO}b!_v9K>!M%-T_xk6mErw_N50#j>IAcsS z;e~C0tC`pLX&2Wy30JV+ISMjK{5Bg7`dV-z%i(P4G7eh0K%NBmAenOE2LPWXPAxm{ z4y)9LS(&tnw=@mn^6O~cUvN5SgudY!j{RA7e!C`D-4|YWkh291KU6fa7m+-ivoK() z$jq80K|UU*B2~IlSSMY!ICt?g=>p@+Icf?OlITOqFx*dT^o>2&5fOA|jY zsZZ}DmtEd6r%tPnCZrtm>Z;s!F(-FOR$r&Bcqwv*=vJNFeHvYPhvp9dlAAnm81s5( zD>Irquxw=I3=)6l}-$A z(|YlFR({622MS_?eu_vP>bJlyDGGNSi>qFHloix{Vz)Rl%26p{6ZKyO&Ah4(>BpA~ z#>j|-+)VL=X{Y$@&9;vIkkB5tlTO%1t8;K4t+Q}?HQ*o4%>j^ z1!z9B322Bvu?WH(Xyz!TqK;@*~JWSnj|G$<{}AL`0nC58HF zCuGKDG*Yy60ED^)j`ehYwxOSeGcLj2#Ttlsonq(Mo*ww=4*VcZ5Z`J)Er_OCkkkh7 z=ihWm0~r+BVqEv<2A(W>M!4}t=~}oWuCqzvEU!C35*#ur+ z-Uu7r{`j%}%%?L8yKG<|c%-sUDH}*hnX7Ui^ua1<0M!RH@y;!N$0veoN;V5EpY#Ky z_1&7)t^P1&Z`1PMo1RRh%zWXGFwwp>ArQ;(lzXh?FTQrpbykTH#U*45rF1|Cxdy63 zaXrquCYq?-tboDeV+aJ)j`OH7?5opz+2ca#+$5nyxBG6ZGO`2!PTT+;)WcOG%`Bm2u%U|7N`?%(=zkXDlJJSn1 z8Z9NBzC2fLJl71hH@Z1nqF!2Cp>=$;XgzW0cEF$oP$k2sEPtpZu;@SHY9`a^#6Jqy z@0sx+z%C_l#+`h0ABpER%3p_`+Xq*o`8z94{e|+rV*4t7k0yAhmt!tCpBVbpRozV3 zrp|DJ35nP%ZyZj1i;*5;!;1|_GAI0UAR-r1EXqV1SJ3E4VqrlL5|O~cuD8It6*`Dd zKI@cRhL>Q4Hy=L7D?(-fLbT?h71s)1mUv6OH*ViAQ$BqEJu?rwIlj zg>V^u;VKDOXyZ|6PGVO4vJq_ilCmK=AmBx4m~nlmx)s*Nxl_g#Q0HfekfWh=2>nfq z$*)`ka-`kW$pCF0>xz>G?~3Toq=#qWx;g;t~VFZvuG29jwm~m`~>DP{9Jq z&!4wta*q+#609#_D$(cRcBN?0uchAU=f;sctWoW3FQp@h`@_d$lOdvx1v>R#PCqB7 zN9zh9KUan|RQ<=-9XSk%vckmyb-fnlGl4tFt1ehwY&<`{;3?l26Y()Z0}F(6=-!ad zA0XjMUc~bmzjoldL?Cch{{pt#Orq~=>=6EC+RFz>Jy>SuS3;RFAD=ojS(5RvpiJDp zXxHaE{&ZdZN{ChHod5m<_DR*w*&ewvrmm<1lvGPd3TyNk zKZ|iT(n%l0G8qmP3$bLuSdYj4QlyqMKA=&5+yW_>Q%{es*0G+XBgl(pr}@S&m6EZN z3bL20m(|QjTZkpjSLjUv(hI}YlJJfsio+|FL3+F4fcO$J8_@0_4H@UUc`Sdro?1|1 zESo2bbq+1A!l46wpJrYFfgPSHH8j)W-T0qwfxnZN0|mNKgRK~&0~khr61D}~&u7Od z!DksU*ai+xSBoG3!WsGy5{^@`@lg`%ZSF2w1s;rpsnR4$P*`SoEiK8ha&QzBy{&{# z8eZ<(G8&)PYhzEO>5hztC&Sg2fuLIu+O_!K(=$0uX*$dv+Z6F{h9Ll<_o~+FCbIye zrb)t?tVn_at5bS@srm-Z5O>VExG*i={Jbg`c)SwmK(h9iY1{`lXM%TI=t~{?%6QKq zH-h5s3oGWS0Ao5@?=&s!W|th`W@=X!F?5 z+?VWlWLY3g7P(T-VC`B(tFgM=i%0mQ1DJi83(k9dVx&$0UJcRrwBm`vF8Or~+*aZB zKQ!~tvwAWeY#$xZ5*MU8ymxN;9jE4AY}zrp>M=x7ULA0g%j4%vm<&vRM$DbN`MYz0 zhEI~DV`s&J%Qu2o4l`Z;B?g<<>;NAzC(b-aX^a(TLV#y8gn!Kt;;oQdMNYUdTFayF z6qp_q!FMhL47+O>@s2rerd3R{tymSTq!(uNa?B4~B3Hqcq^mI&PR)qu`chX8x-FZa zHZQ|ik8~sHovyWt6%2Bis3_)vcs^*`L7lw8C#fmDA^YT>mR-To5Q^%WfNVGP=- zh1K60Jbq*lHg5%m*;cWAVVS?(3tg~!fwj%JE&ihfp!Rn2*(PorZ1eVUGZAi*5GVxF zIpW&2F3eGwqAG%?ob!%z6Us(E^r}&!cs%H7R^fAbSI@-p$6MT!lHvL(n{m8SQY|Pd z>wW=QuQ{=x0g{Yw*m?+G0a-hS*HrJg8>^JbuncUb7#diSz@$MsAHP86`90_Qs{mwP zZrW>rN9AYF45G6L>Y!WI2TWPF#H+(}4ne3~V_Rgj>TQb@^K*?P+-y)rOXUGof4_E; z1g{r@H(st*MoYDF9y8%>&y1#Xzm7S^HkMZ*6YYMTnq#f8QcPF;WN8>Uo*0p)5AT8g zAEDM&ofcf(a+KfUSp)aVsK3`z6i1n)7IAn1t*vCSiYw!8`Hs zgv?;ulrM)cs{1xx?C%pb3vQ<^c4!lDT7P7V!S>+puzEnvT?*FR)Bl2pO5+8w<(6TIVGqL|DG+Gf_d z{l8gy0+G{A25dimfad%^HBjH$IQkzy(yp$cQT|I8M^6Zg2AI%_@yO#ter6ccd|1XCSMLp!H&-}yKTD{i#IiD=!R<{22eZ$VYFP(>Ee z!$L`ojP=9T$&!nON7BS45{2^X%R$yw=hHa%)BEAN%N761)Ysc*_z#WUEEuwHH&mfLBWGheDbQf$Jp=Z5I+k>`r&ZgmrkRZ%L@Ur+~Du*N(PSkXupMgS-l z4pv*!<=*LuX;cyL<`AVwc{qc(&9K^p9gQ%AFom!xIDNP^Y?PKF*7J%!He1sO7FX7D6TtGsbpNz&D9!q0uBNbX6ewyq zmVbFqpECHcBV#`)TpTN3?=>xW&5k*E&7LL9W{*CM!gg*D=MI%8>=V^9yb6r}SpQ!q zpp#Q?K7l*^|tD}POqh> zQty$c6rdLq&Tv;8{L5@b(^I&Yo3G}3S1v2x=yh_?*S2yPDNNT8Bbkthxc)yWUbo$V zkc^efV^1B=6%6Qw9dmrJ)!gFK)HZ2Ov4)VO}VV@%lG)dO*?O6SO)Zp2}V-{d^0jZJmCnVpO|+ z^JYy}Vb>|>sWfc@M@S};!uq0WwRv1-qOzF@DSw{3cjx24Ab&K?6;g0pfo`#7<2$HC zvy^soMY9g1eE%$pvZl+NLIdWrKEb0572@AJBk7jT{>Wcp5~*RrH^r?B_PV%e%>{#S zGl0A+%!x`Zg3Ug8dC&L!I$OII*M-!S$yG)<<1;6>20Q%yvdHO7qtwyHF^JRGMtial z`7loT$#{L0+2|F+9m_3UvMF;M!0xT%wX5-m1mFYuHnb{MVYOWA&m)$yqnfHLO!$Dv zM&{K_ex|}7b4uzTsw0P}!mUlfWG2+UXaHhUEDU8oD-*Y&B#OkRp;!sZw8l(H5&RSA zydt2x?9Ew#Alc>dy={P7c~`720UCO?H&fA>M}<)wYS`yGwmRT%(3f(dAM6_u%A*rk zOPnlfT~HP?Mzn12AXjMCeTFnIbv0`-#`z7noN`7VZC&4=^e^}+2uez}W3=kjX@DO6 zpM)1^ch8#cwFS7tMA`T#O(KJ;dM}efhB;oUpm}_M2HE-L10kfrE1-H-T*K}xrBM@x zS$TjmBjS@Y>z>5M{6>~_N3RSNbi?=5)K;F-wC~aa3_1$b&y3zuDVeoY9*f$zW;;6# z38-Vu%g|Ds%%qP+auToTL{x=64=6juv=;!BLePIBOBu@FrBkOWHU4qsOxVDLtk8>~ z8w6JBDgqTYm|P#lZ(}@W&SBhm=w1`)Fx_(mW61ObcyM@O$VSKxrt)hF3{n#<(B%5g zn~O}t2$r}Qe2t{i75gsk{k;D$pzNge&Hn^n%6Th|PZ2V1PTI1PBl|Y|&`XNtsn`v_UKZJrHe!=93!dB3kPy64=ZKNc zVKTV_1T=#~im3_1i+T~=f2!Iq_vJ-MX?f1*Q|+uq;UnA=-h!)hU}vrsbH8=i8Des> zh0nTK_k49)sqzT4FWszl0%E5w-QGQFcB9uCn_{mxDoJjZH4<{(5h@%@t>@%{o0YyE zohilVWKA$jKm_h;DVb8&J7mg#1s`oGI(w@NoCAAeUTVwsJl{oacZe5xeOptcdR=jG zvBbO)#y=us3V!o0Y*zucr!0*vA`X07duzjfy zbyBT^5?=(wWU46#Ea8Gb^`Vw7|CZu?D40l91cOpaWL(FdIV-a-ce5|YN7_rP1%utp zm+`4NGBi8!|MW~|=*iQ|mNZC@o)_BeDp4mHS3WL%NHgj8nR+smw3C(K0^tfMRK5tx zhe=RwH^+s!>BX?W01{~p+FUYOw!j6<#86liR5*t`=|b*m@V8yivEmAre2N zK#h~aIlWsJ`?L58as99whWVyQw|aPP7lLSPbElhTpX>%y0Je|Id^x6fFoO{*>&iWc zbRzr-aMeW)t^QcB-aOVwlR|d6?#d7k>jr7UA#0)7bbHMV!4URoX0<_UgV=OO?$!{3 zwpayCFfcnB`P1T!&87nRS|RrHrj$5!uoKJjW)A&T$9bHcEjgd*m!{lp%&&CC?Zp8M zUKyflOP+j3-@6e6pOBpqzx8@keBS;{(X>(h4dI8IvV10`;@QonK`O;lcTZ-MTlDFd z^yfi+gFI3HevjB%V`+N9ruMR%q8c8~;O0s{&J9Q>cAe(n)x0DAN0Ds!wv{9H4M0l4 z{QrH8ziDD=)4l)wHO}orf&`HLZ(VYzvaDjC5Yi_GO;HoyPeae35fH9rBtvEtMjG!Q zF(C4M3}--jnO`!@y@upBLL`yj95M6k<^fpdOA6M8yXUTzjGC)B#UF0AWm=kbc_ymj z!oi^Wl5m-^o|96OU*^s^h?ZdvkAWeMAm_;5~ZyX3oYUds1HA231CNt7(I8PSZaQ=j!j$2Qi~j4^bztu z%QD#`QK<4Q6sP!qa0Akc1)y;Mt2X*xjc>A%F0?Q1!pxrp^BVdpRLWekK4y+c6g)!a zF4J&YO6FzgZ6O;f@^AJq?MPz6zMfL`taBlxq%5=&=#Eq#H5nR8mQVnJv~1p!^qBf3 zD1VDgw?CmTYkNhLNsJl!wIBEA)y4VOL>8yRX2;(vOTm|!ejkurg0CEpU*9gRp;!Rg z0eBdH-CiHG4$a;xsuum;D;m!~L8LJLhP{MT&!XUf!fgTYC)#IIa6tZc63nMsZwZ>O zPVXP;XBTil-ZmrpdrDBZ#x*C*r$+A<^|KH7SMD}pR4$f1ZIE+I@D6#HCbog)vk$Ub z&jo{HU$?%}kH2j~fj(HD=RqVb8O8w4enkE4AW6OLFyydseJ13tgCLdXCfMSB3`{)e zB__QwN78DT z3J0e3aYxvG4?Rb;_g;9K^@}j++L)bo`*b)(CcdMv{vA6`xj=pUbeO3YV;sI?2u53X zQEXfDmZTknus5u(;qP+r57T!=g&VfT@*cYr2L~|r%-BH>Tf*8jdZz8)!us3^!@lmo zwdiw$TVnOi*?|aiV6nCQu1#USdZ)^-x0Xx#zKoh+2u)*vLhP3aM zVFAEkt{7t;QF>8Q(Lca5C;EOKbip`qIB0anp;j7$pbGU4G)v~`J@e)2UCdg!!7=P= zA;f+)^qSqjnyj{cMR1lC&**%bgEhM-=rzKBfJ^yNd76V~E$;kDHGpAMwRfILr5ju> z)j`~AGW709Fq+MXlCqYJ9g1^}8w(u9%^*8Cu~zgC8hc~ezuPx7n_*FE9r$S#Eh$xP z!8;1J8aE7l+JkR(s)P9N?v%g#gkA}HSg|IJpR<98CZRw?<5plhmOCrlo;HoZc=_YD zEjppF_@Ja$sgL#}7l1+a20MF1sNRyfbl0V{;+c=2*D(;tXrwimR-3#df^KNxVrVhk z)pu$*7y+!Qv#$BJ_!d$hfnqQuYFs>u3oAelf$mZwOTKd9VTpkBIqnHngt%wfu=(U^wKHRN}V3lO`8jD!$hn=C?h(Wp5`E3)ST z9X$`SC)rnZsu(K8Ur#met7kqUUrYx#BDJVyC3jI!hCg()Ds-9Cp~OkaXqv>?#+jNZ zoHnGr3w15|0SB{du~I^YHIQu&cah2zEGXV`XYgm*Ta@|C-d9PRHPP0;9<;L6UBf2l zTym=-o-;IPXI}3k2w-2l)btpD2yiyYF88eD{#HgtRJ}51JfcpIebMK1bLtI4#N;Mz z9P!p<(u;k0N?f^?_p*u2a#2rNof>8Z8LVswp-oD3fE?>>ZK%nA+?J|FUZwFQW7@6p zt&{&^xgjbVxCaUV-f6xZN5KNyS>c|+>Dk} zEF$7ON7; zuIC##8%Bx=%RgT0e(IsTI7;=BM85ONvFM6z5A=uvYurF&2V&+0l}GXWE!(b zT3BDBPJ2EwJ<=_1yOQ_!)}^{usbOHu{!9kL6u`ERgx*B#ZC1-wh#-;w9xTnwd5lFB zIfGUI(8=uX3>aM_505{TIVzDL`Xbgkvs}fXYoP5sLW7{!RpBysCrweHe7xI}5wAl} z(5=qE>on(Sv)@@|-eHlUGT9Q0o;KJLJVBqm2F~tY8|x85HMW?Z4B61}rG{Cv|JLX- z04SN6o~)TAQE=coBy;Q&W=o74Y;c)2`ne#@jpQ=+aD09An|`Fs_%W`*DTp))VP}B)f|}b(<}Ep~ent31+TZpdHwX0GWEY@48;HtcSJ-D&X;=^N(DVvD|IJp%P?Tfp^H|dQ8lTu*Ez* z%&!~IGz@6)2hT7@8S0-ydTL6gHS>@V{W>IXUW&kZ=5m$uiDH(IU?+KwjnVsSYjW&S zN>llNHzdTE>fD#L6WVI@P8ZQ)RCt_nTr{%n`ywqYxJO@{<7c#p0)U7T zwcjZ;hFCFSZZU{LFDaI|U9`EgrE+Cx_DRp;J9VE-TNMO~`=(Vu7iG3w(sA=Ejr$~3 zTDYV=bbor3ND7+w87UE1aT*fQ9mi0DMaX+85&_J-<+hsiUvNJ`PDpJ{EUEId(nN>I zkjHS2Zmq&^X-ko49DazmZLACI?JWAkN_%Sn%fU!9O|+sN=AW~rjtId2S9wKjz-JJ zOfNib0jvpg%x}KKxxPFjrMQ<Vf~eM{^6lzH9I z!Ux(W4w!S6wF$@*Q2>QhB$zX~!84Azs)){GMr_4SsG&4%hgX!kl_6$TQ29@5 zk34T~^g=F^p6t2fssaIl$p6iPquhD5Y@q4#sRCRBy6~ei_k9{Bi zXf5P+BB40ci{ZHkkp47O6*4CS%g_X?f!p;CI86Ou9lGk_QwW?>j5A?7s%$QHo?0!! zd20=lPk4Pl+Z!X-8~}kXvR*5L3^D;LPs8p8_8jEsG-!Q7W@03B4yB}F3vu3DJin8j zkfF%|mZ6`hOKp+BG?T_A-_%wqC+Do#6kB4X16#&(C8F^kMgcadUxSWI1I%08W@<7j z`9#cr+Ng<|9IXm6TJ%zxc!UsDS+gOLWE%8%$S=k|dqwlIZvg*5^61Xoeu{XlzNxYs z*tYS$7zR@$xHet9SiVo)*ZBMUS^KRJc>~!ms`Qkc5;c@GtnMd{z_a3 zZ6t(nm7#=@jRrg%G}{~!R1C$;yVB&?ikxW=D={9Vq6>eII9Ne}(X;E~Q5S71tGOLZ2uP?`Tu(BikJZ8o2h5xa>~@Fc^914tjbR^b%&qy^ zi+qJI0b{N+Jeo0D3CQlZc@*3s;kq?i((@5nrKY(MB`3{Y1wn}RT1K#F2zZzyf-+;D zbCfq37vPw&vx3GC%Hjg;8&_PG`*@5G`k)?NSXSi^6`MNWMW>@nt^EeHwJR!olvlpM zlfKZcU*%3*J-UGszOWsI{w`FU`)s8z(rU*K^GE5avBTocE*H3#b~hotl%x*e)fdKm zyxVLfjXJ?3j0(VXU~D9)^mi|v{2Vhi0cCbmSJochcUKUDt?;pTJWs)if8jGb zCMk0rS(80;;zaW}0a%&M!rZWT^u_|PxPJr3WUD-SnX`l+Xx@kg9ok*mOL|&Y&uy-@ zL2~ThJd=#x2Bp{PqHHzGY%SK>ZCKX22rs1tmfV+=9$=H%G~#*=5K3>*qI*Ksa76kT z0mSPFXTQ8xVPh~Z#5YB1v^UQ5zNpY*dNA>~>Bm`h;<|7HqY~p9!;)r=t*`@W@#2?u zA6q)beg5_(Y?8#a=apWaRP}UE8Es%)&%@}TR&7En7BC`ehGTZI$jWD+qH67 zkfHZ9%)x$ZcEIjuuBg#VV*ir=6W<8)r^sq%?a2dNF)y;*f{f7xl-0_evw z&{jJ_Hq>)`jJy>e?5$F6`$gSXtqY*XN^VQbe^(>7{4^$%qorT>^o_X@VpTnKOPz45 zHXtsF)o;#pmz{`KW;akBbe>(@t$eudZk-G$wftLkgjT;PZN69fyF7CiS?lb_SYfBR zZf;BQIkFB&8jn#4^*C-Ag!Ih;HK29kU$9xMOH=aNWo_%-Xe3{HhS|@D&ffF2^b4V= zhRVXeJ-o2ho=}1#@x*-Ix)qiEqOaKfN3K+N_*Xg|`iC^hqLQ|fUxH{?fkD4VV>guU zv?drz)4Z2$vUuijQrlfw9ufQXM9O+uxfX03Lz>*U$_rbb2?*?OEVTGE7XVPx(&tWG zBD;b%XYvPsrrxnp2y0h&H4|3&4oB4&*k%mbRgTl}Ggc3FKr8ff7VK?gczNn$g!oiV zKeeanatw0n<}5Et$9dwWcFQoT)-xrG%DrEid?iQ(!zGInWPgR_Ns=7xmEY_8Vjap3 z2?XBFRS^xA$nJOVB{wDwFafejl|3w`eKN(mU^|004^+guB0@2m=JOBbCzD+LjZjtH zUdd0Jj!{Zs$xn+Og)PF^=`_{ZOR{Uh$XvdHhDYB)G60=bUjb#hWKMsfh6jzg#PbbO ze@~Umgzs=gzCn~*+YpBLlR2yh*(EmNC$8d}U7AGt;L`Ms@Q|Y#3BHlWZj$*Zm<&a2UWC)s!W>w8&^z+ue*64?=At zJN;$in?tUAg;_>Rb_zh42~Z_J<$dg>P4TW4`#{nq$NWilx#3RGM5!5_X3V_xn_6i{ zP>WIFg0Ji=i#0w?dY{p>o2?eL=%ESvY@qY@XXHCabpjedgJoAgo6OUsN$i8N4gBo-!W|5K0d5jfQVm-%I3(cLc|NDJ?m6EZ5TuF`jk$_S~=OnULKj9d{tCqhL z+idL5P)BU)lOF)gY&g`DGgBPbKHK9U94=_Q_b5IMO_$0Ti&+Ays}=ZOnzJ5lfyKT# z8HN7t-JiYM${~&1M7Kp;utV7gUe8C|Mp)%V5KAj1+4HO((NpW>po9W_1w|_@|A;g9 z22jZ#HO)BwwQ-J~OJ3k*pQhD}`g=i8!tB$sCM>Lv0_p+y>0mD~##_tqA1rP*v1*?a zY*9RGuiH6Bdg=4AJ%eav?1|ox8pTJ^`nhB4+>pIhqcO6uCP*c^>o`%mBBGTAzVjNP zq>dnd@oQCjLvP$lP#qa?s~$rImdza-)0`YPoU?NhH27FwhW}{}AxS+a4=XoY*FntNyh5*STJ%V`X#v~Q@Z zp$JuQY?J1Io>xyt^&tCGggN@$d-hCGmd8giM#L~)EuUTJ|F>pXpuLC$o9KL(u51Ealp6N3e5`RSm&!h zhjCydkrUc{$W-A6J@GL(LT8_F5hva(Z+W9gi~!f%W@_o}_%AHWU&j@8V)nsp(e2}G z=kFl@38+B6!Wi*)w=LxR_&>3XX~>GOd;on}P+>Iwz08V=Vv9cxp(v-KvQyIWDM+>t zM&SI_4W@9F?OYcspB07Sp&|Z4f;4RnQsO)P>#c-GzkQCzhmY2H14M1JmYM3TO)7qX zfy2Ui!D8FN?aNXXX!YUPz)fhZdF@wUt@7Rb^^#&kJaWaZ=P{le(D^Brl0p%4kAgKvO2NMEYfWxAB_H6k5pSQ|_7xFlk1_pmN zc#h;DejyD3qq^olG>Cp<@G`S~I`KycQH|P>oV4@7EF^FKb={3rxopIYlgPB?K3_5+ zs}zf?H5o0nu%bO$^?N7Ms5b+uG_0sT*C;k4nnIlsgD_CG`WovOkhqG`KIu+ca+ zJ5b39-(%m0@8kbkf&VggtsOwZ1H{Bnzz8d&g)TjMyt6WKDyMT{|5KuN8|y*Ed%EPp zdwS(2i?i4U;Y-Ft%PZtxzF5#P^trk_@C2QfkR~FHM<0RShklTlTYLUTbs{-b>)?j= zZ*Z}^&X^ktY}18lS3Pao+LN_Ob+*)zBz!O9&i!jb!yu{TXuYijYDKwG-{@_9DGd5R zktgp}9B_nWBZ${uv;gb3-J}09oQY^OqwrAtbdP+?;T=OQ0veKD(eh?NCrm{I-C`j2 zk2>UDKD_0>A|i%wR@nP}n~wi8A_?Cp{j~2}g-%(8MgW8=k12dVy!>D(vpT|qnAP#F zAaquGvT|aB>1z5B6b!Q169cmpq7^$eWDHVdH(>8ndpQs>2}qWSt#RB>*}NVC0$ZCP z=cby&sj+tWygdIP*wQIjnB>eB@_K8P0?hM5+#uRDJi@y$J!1*ML7q&JLnnzMI}Lhr z+)082(g2WfD>@&k#9Oyu$iU}#;-yUPRj2tQxX`J*&OElcEar$Q$VAKezp$LrVatN)p%Tu=KQ1T( zu=ml&sXQyUBZEp{4$;PGJr~0;T+57()n-E8XiTKatR()j@zZo(DCKScX3N3x1~Ai2 zkiw%LFBuV48RL-RSTL6vgJ8o)VOd(qr`c`EMwx~^u3l^%-HeO%vbIjZH<+#URcetB zbKTPVPGLy&-qOf^es($D!PSiq=!SV(=qs3JJe)P!rTu|gN}3OPFxV$|6j0^br(CaX zbuqHN#7AKxM7a^->T8lSU7+`i2lxUxF=M41#-X@3?;41?_>_m~hDXBM8lsD~$PezsM861JSAljQULBL0oj6yYh6-vVsu zPCZIBRX3RC)@MKm7ac}-D*?rXiwru5fd8^mY*|a3V8T&0a;w;HFV?Cf1C(XVHz;B! zKI}?ygV9@;-C!ZLjc))Yw+!iLe`%r(`ezy+LpX^N!w(vde4y5w-2KCO+63p7&*FHZ zigZnm3p6GZQ7>)vT+GmFz*{xj-E}Y9=M`6uMpPX(aUHZ~8?6@Wtbxa$&xEiNFDWs= zk!M?`w2mjGr52(Ih*>m71eBo;S!vwc0?>7^JduN!di+{xR{MLtovDK%gMJ2De6Z>W zatO6kaoW_o9ka0;XZ(r_83F`c4AtwZm#VM700&|>M=lk*dcmR>R`v#B=CY?7aNBD} z`U}YpaHOe89%eCY3WZ5VGrv1nGgMX{JgP z0b`&0sZ4MsKZlXE#y0Q5$Nyn)*tyky)#Y>5k@0eNbp-%bis`1u;4VA{Nk>>PcJgK@`iw(9$ z)@Cas^^xYRvan+ssk++#F}G;0v$Z@iJ27d^F^3$(r}L)RK{$VnMZR*)LB8k$hhhI1R_8w%AafH5nwzhHxNp@!P=yL1BwHnr5AO7h z1xHX*?!?LmwrIOlwGbE&G|@c{G*PnxP=U{tl+h`+(egD{{^B(g`I-xnicLodJ+6^V4G2atfD7m=y+x6>=- zIi7Du=d2Z1K057pn7Ut(c*q2o&17PssTXZ8DT_5|Y{a(R4Tv8U4h-x2)?5z!5-dH| zO+y<;u&qYTkQ0hjYly7b2rMfE&_L<2u)g(0W05t99a!o;{3sjosaG1R_%;)LOSiy} zQy+#eKU$sY`-Jl^zKz*SV^20J;Ww?&s?=q&^01}gDl&_=mZMxDvXz@y>s^7P$|CZkqTP?Gb#vh=b7W&AW_>l=C7l{IFRhR;w&}4MOYCtGNsEbDYZL(Hp(_S{(#tPt zWk(gjU4Y%l{KIFo5^SF!Z5nEuvm*fC7pFX|S~y?CU$@c{cQ08x=F;s$3BM!*^M$Gv zZiFr()Lu?GseEl;nH~a)vr1XZu9cw*6;xLx_4@VT$xmXSy({xwB2+tc>3K!CwdO)g z+E4nSVNqN9yrh+3QLR(nV%!l!$O0I@QxE;zOZ3#-nW>@Ir``4L!&67{%QffyK}nU z!;HA$9${16P12=uME#|Y>$1Xr3@s%o;@|tL>cgz{io7bgu&(TzV>3;GpS5QgTL9f_ z)WBPFh63~M2=U`D>N_*XO|YU}JZh_Si8nzy;U@9U$saf_Gc zH{!gSHFxP9;^$UjCi}bWuItm+U)5i$SCYE)x^ydnADYASxd`)M=87tW$}+w#GhNNA zFrsKIJ$Q&j(#;%vHxJVLC0X7UCR$Zx7)$hQtFUbE!ILnGR~|wSl*8*0(;F9_6xi?k zXq?Sdmfjg+f^9hlcQ<9~VMIJLmC+HyzGOhek~d~P_$0@imi|Z8>O}T3VLi=i%J#?8 za`oU7YO)U1;26Y;uwJp)z${_>QYcC|x??y!AA4N|B6kVHVDoO`@+bChj(gclgNa>m zW z>Mamw8IGnrkbpRKb3rm-X0q*U^UQai%K#V(trmJB=B$;aujPWtRtEWbHmPV`8d3aXdfa zb;X{w07{X|k)`VjwDB8CNmDnRz;2*;!fbY$xbE9nUdK9!qs%Y=YTA2%FT7R?!wINg@L$kX)qemwk+oAC zh|_)YBgt7wODEoufX?`@aDWo7MQHr>jon;HzJ0#nyY@d^_(w!#r#g+imWrvagvb_2 z`GmfD;4yrmVZ0h0N(@4{BJH_5U>QP@$!lbg3MY`%>=7HRO@z+np8^pC z>{>oTgFk@|0PVPmNt7V=&{>y62sS7O#e_BXt60bbm;VU5uM0dl4Pb2EF^dlW_Ny6a zicn*k_F>*b@N>1-vuw2!W^rv^o8=AzuKy&7lBdFf_=U%V6>v9#livH-2pi zvL~vy?g$1#q(RhP@52%{O^LT0BB(x?Mx0IOpo_LhFc?sS>#kb>^bl7fr_eRa4zeAZ zkEus$(Ax38OHg+J(zQr?5nFs4p}1uZl_B`?Hua>ieOq`syAi(1?@Es1zp@olk{ymPkz z6ao!LDW*Vs{Nk@R!$ns|fd@&@MK1dzkC}#FT2M{BW)`ouCq>G38sH)>euKPf zoC1CxUIhwYqN1xmc$~*~;PtER5-bMjB?7yS^$ofE3QDWqy#1%2#xzv=yD|vV#6fT6 z@|6AA_ucj3XTJ>=9y0ABPM1FFY*c`#YkN1_WoCj_2|T#!lqE^DLhaU1>UT-)H(Ni- zB99sv)9n9-39g)#zvvFYLO&Y1s|`$(i>aVEGu1>cy``D`iuvgB$p= ziMADNQggO%@V)&jiS>CNkO(LQWm8R=@*de3=5sJ+bX#mO*)?i-y^2DQEylw>jRmHC|(UY zOQnzh6>g|Af%kXzeU8peo!IS(mntP2?GR~fudvpjb=s+)aJG!LH=LyU#ey9OD84<# z$y_#w+SwY$RaM5^yovieQInWc9#tS&FF+~ffaAf=TZ-!x_6HCIxUtc^Tc=cokwpWfK!`|yS(RSE6Yuw5p2u!3})RZizlnX)_ zS(hc2ih4q(aUhn%(FNe*J;t>?hk>Q9tz2^5N4HSt;MlJ7oIQ8#%s##N%6vF*{_yFlkZ%|KT)We0U2i=V-S(&q?wiDj)5As6}BJWX+SJu*k^3ic57FX5#Be zk{!ci8@p^^UA0R2Bida3=s3HBXRwpHktw-yX-cJ*##^_l)y6J5|HenGJZAeVl5No` zE{gqR8GQ^W-(FmAM`l;lTKn6ZxW8H6uV;O3{vf<>W3#2%HcdR-Xth-`V5E!pW~$13 z1-dl zMxCz}k0o{m8RI1)0W@BpLlE9Wle`>>V@-q9{3iE-#(mJ0Zv+#&8dWCUjUQ7<7UZ7_ z!TOjh?HrEzJA#jJ2JqUt{oF#>e46ln)^%Ot$eQ$5^Pa%*zQ>Tk$00pzqL2*5B;s*8 zRH$-6!(Nc8;mC*FG?vu!pcADkT5d(&Rw-re1WHXo^qdh1$MD~Zl!!lF8%V!6k+xA9 zuqEmMp;StV%4LLNVU0M#?c``8zHTUMJLb%IR4?2iRUGj)b?xk@ z>{HgiX4YBQR%vAOp%WY>zn4mA7@zy8(2TNxn-3GS$6wPXip& zmAC^1pB4ACVq`5t6TiMM_R>o)pTX-udu7~;wUd-z0~drJT48mv!uxhcUWUANRZ$AL z_`r+`Ed*^7h^rGj_pb>Ji6P7H?Xjc!%O@CKx@B!hQ!~F*j z%79w8{2-#xdXl0`LT2(yVHiB+UwVToKhT%7{jTJSm)#yd=?pE)Z`KdC$`o=^PSwQe zHH4BCvm@Rw#EYP?G`{<~2jvhRI*&mBR_+L!?jhUZo)H4)&NF8G(0lKi3@ixT(QUIz zVsd+c@EPyDHB7^nrvovt!?A|KyD*G!ILs*55Xpd3IttY8O&d25)<}bI)#+KvkMKC} zG#8tR9Lv*Z*=q z%cc-DvkTh&76NU=DUG7n@}dD@(pljWuZr@vq*q1~$wzpRg0^9CJD{&UGgz2?*N42V zE#J-E)naaV0*f#w5Fc5{M6;RjQ*qA+eV28LTc-{__hynnW(q|0B&lrm#1q}{hOdf+ zwzF3_s+>N6RrTgvhU7DB+$nXp$>PNjU+PYn=#|CNKgz>~^vACv>a3czyE?w42gg62 z1>=VhumYyDM_MdXs4PPfJ%M6vieLB8uG1Jg2!C zt;kH7@S1ot(QD484EA;{ko#2<5HmjF#M2R@jfVCOv>8t9H;1<=?AQ2SL%-Ei>U@xciP+Ba2Xbw^ygog{ z%!^Hp^u@pucoLuyD2XqTFqylnDED4@5*m84;fRBCPWe~mhITfUzK3x9@n^^8!de$U z+mloL6~;Ast78Mi8m?O9WYw?>X=0HqJrc+M<(`3WMJOagjsJ5k`|GRtjUBxn=mM=y zCf_wDJ8Rm_>W5bq{j!l-&d9B@tz6|F<1m+=)X!{aGT6%LnGxbvKx*WQnl^~Gt`x?y zAicw(+vKKh%D5u%Jy$I@DSJFhfzm6}7$vD$4#^N!+vGWuSw*epvX0pLL3haZKU)2m zW$h5^+dJ{jf2w_drTPq)F-%NgUFz;%*=elb+ntF?3V{mn#A7jApQAAkfgMFuT$e={ zmR0n>>>{~jW@%%hd(}*~$rK?hc=;NiP&R=nopKsjX{|Blum<2h3 z<`kP;VLh>t0V0QBt2kwnnVcl9l_@whJ$^TK_#wHYZjUt`Z`w?15J<6xw-x2oJK~sh#s?k}S#SM_E1w}*{oKbgV0~WLuvmDPDY*4TvG=(1U3mV!B0)?39Xt7`2l98{pHhr!6H>>M; z4&Q0F)(9WsfmiLuUx_B#Y*bdE!>)K=GSA*5CQl>Pls*srT+?P#{%490wr<~L69GT? zTEyFcgBO$|6#ZVXbBEpO^W_~gDyLHuM&IO7E?q?%&kCP66@dWMHAx%>u0xq#K8Yhe zi#1*WPcV?6PiL?Ntw2pbW})Zq?Ds0YQ9J3R_w8-R0fKT&yUl&6?WiieE$#4Sp>OIS zE?VN}yD163?25F)k=gKIc`ex>gUCARbrhvins3QkMwjB`2-;2TP;44qgzpgMAc1gf z>OM&y^_aYqo`6CFbi;75>p#bDTFL{0HC{nca64DPvX23Iq<;F6_)nKiYKK(c)O3(2 zXiH&Dk=}ei7>03blQu^MvIZZC;@L(~eE7b; z`o)kbx~{v79ZmZYvy-1_t{>0fb(~@2B=w=p{6O< z0oQTYVb^IB{*?Et{Gl(pA8xZuH{>)I>(t11y$~D4#lZ~EOnlztU+$7=KiKej;TKY94xZRXfnk^3mM6*; z^1pzHcLbcN5IxHcK0VN}WAPVNBN_QcsWAEIY4)hI-WOq>fS(#mww7+m)1?e1r9M_k@2hhIXtJ!58xc}4%pe2Kgp+hKWo!ngiA^Yveboc}ZLH)tFa_szmE zB@fGi3;J`%xfJ8Qd2TeA6<1=fy^V{25(4XFAS)oeyqsC(j8h;;1fP2lIrTawTvVSgVTh1d|IW4K8Y1jRoYQ(p@Huaxp;~SYUWH3))A_D^wgi+oYC46@{bA=FyLNYj{MnYgOh$@Dp za|oBym^92o2_&B#g7tatYDOZ<0S@K+F_Sa*@Dsshz`%rg4@D79D449T* zr`Y&&nz_T0DovXffQKuMPX%4PXXl>;tSt)XN!bFN666>S;WTc>?g{(MTLIIRK{-P^ z!#1Iz=7SsLp)NLP>&q`MhqHTPWvCG*p|s5Plz$X>O%WPM0{^b*WV&Oi`t6xQ z`Oh^KioCfy1}5LuWa+ACYj63l3QEDB&TAqF2nZ4gPF@JSUI<=Z2|{MB`>N#9%p8rAP`D4^~T_% z-1|+t{g~ekFTy9epZ&g{%^B|kq28OaMVsMvf3o=`6kCidL8yRDh>dN9MX=p`>Lhn( z=1NYGtQ3L4wh{T03lYuD?@J+0sm&`P)0qiS=gA0x@4@pFU(Z0}pl)umbTruuSn_ytdp*?}V&tzQU}S>saTP=^f&4WpuDn@$L5Q@GRY4Q6LL81? z9#YXSzBt8j4U?4582jYWdC$U4B=#M{(T%?I@&3pmTVXiI@tDNW60@~W)+NtUqEWAP zFURid_Z}vqJ6|d#6*z~XOOV9ru+w4)+D{u#DtkEZ`fb%%P-}WtKh=o0H@!IHJ(JMm z^cC0edG4<9VuNL;7zVdoPr^G-Pj(rTQJ#bO6Q$fz3TfTqviS14^#TO?Cl;|-nT$ye z^(AXoW)Vj&CS z5@nYn;`-N<_M-B+T2#jR;I|~FfJ8V7W(yI5uzIQkj1%l1OB+8;n161j#!%*vG$s2n zB+GInEK#BD(46s6z-S>F)BGX%dkWnBV9)#X#sw1pFS>n0<-e|sS`j3?|9v8O{P&5F z_DNU;%#InrO3NB9{&W?o3Tnh0oG;VIYLq-zeGRf#em~+#5%v3k@Dcj?{qV?yq$VT6 zZ{dQSUcudf0H8li$Kc0(*?q}t4D%mu=T}$B zyO)L3SzS7{+_cl`*NsJ#s)F>x2k`a$zxH@<9RDmvWPdZJVKiObbPS^33wbqt=cwGd zTs-Lx5B(={DIy&VM|wL<#sA;b`uFy|IRjM(|H^#1j*CvjC3SIrI6{$a!h}NlLhpST z{UVXb`OehZ9QZI(Py*A0;hBBf-_iEbNX@Gg+*#z;q-s1%P4!u!eZb90Yv?sD=LQfwfsecg)!DoQtL zNn5S~>K4kIoKvDKPGi%O9-f4f&&1EnpW6s#X zs}VhQnR48nzldis7Om)i``os@2sqiqQej}t!8Wr|^}zqSE|#^*rxVBu5LAi&uLBQ|En7bYI!VC(oe2)c;`bZ4d{Y4ThOH5ybSg!OoYE>egqYvS>vcP3569=#DEXDjx$`=v zlzZoRST#Y(nLx3gwJlZvy?1|G@AUZou-aj*YIwaW(}t!9t6+p&hyaE5!6hO>$KV4I z|9%Y7Q}Abhp9T!kE3nGrb&AMyDu(Dc@Z96|7m?>&jPE}nFCx!982zg76@zG+?C)NO zF}^@*0^f%o1ip{J0nih@E@R+b2ZHPgoh&+{-dgYA0OmMzEquTjB9GJ{(R&G6|3nCC zU?!|Bh#D|PC^F!J3Ivk@ygB_NAz;8e(~cO36DWbxKNdm``b;D;`;U0%9}QUo6qr6T zgHH&r%sUDoKL8l)LgUw%U@#zoU1;)}5S#)~;Sid+rUji4=3D<$GL8lM1mXZ?5E@ze zghMpns=1Xp{IeiT0N}uN1fYS-e+I)H;wQ>7U2rh3-~H&GhKXi=O^X5Q_Mkw6Gg zoQHl8@-p@i1y)|;8#G*N2^m7ZUB?dr1@|^lz7I>Gh{jqoJtO6pTtf>fKtivn!!ZuL z!V~s*!xPL(hQ=DzAvo6wChA7721?8nMGm%2F)ew#uQoubr={&e9?rG+;dYdV_+7Ij^UxlxcQXV;e_EmgD zt7dwQd`XHPqL6H zFwAb%wqE2{PgHBGABmZvnIbl>SShnf!MB!TumXvjw#*F%8pX^;-7N5zZs>hDitY4p z(+)i4cJsbTx?Q*YX=pFNMSubk=#u5=$=u`;_|}rex$k$4O)a!+Sq?P7`@LRUU+f z@%xsu%mrq)R~qWAG*)0_zC$-Iu2O#F!50yg!whB1;WH(gAHqfk>nUSj_7D4F!o1j6 ziY3zV8K?9H%=3&2+Sw_`c2AII7X2JkeWbj{y{M%-+r~IsBe8bgV;BvW2aXv&%_w_) zV|8G260filSlV^cclDL;tWrrV-_K%s%(`3uE}1=i(5pju@nQh1ioW4uw56LVW4L_< zC{fa|d|zo+d`;Tl?zn+5jlA(l%W|7AUpgjY$Tavh$5a(M7vy@-L7rqif2P~9%)RQf zviMztUd4d^lFyG$Aw_Vv#G?rpI1+o*iwD!dL{FEPh5GRMp(Zz$H9!9jm%m(UkvEFw z$?0NQ&1m*qz4`{oi^x#v>m9vvZjfc*Uq6TqGQr=%1KECKhDeBDDa)rfPp{V#mpnr^uN)-p~PSB$;f@AjI!>5bF$0&BU}8!+hsdMW?X7K z2imx2g*=|iDne&X&ILZ+dUU-+GahdF#xJEG$A7Ew4c?0g=SSLBB_r;*ZmYGit_NH; z!5T&W(a$jy&^qeZBxuOY>5-o*!0_7%@4#*PEOhZB@Gj}z`E}p#+3iFsZ7e!a3t4j` z&s(2i_Rse|;9Kmu>6h9$IGdTScyCjVwUQzw@3G||g&(LyCKwB-a| z_AWbs?b;qYpk!veVX;li!0?2o`sL@>ZuGU6C1`KgzU*70as|$!Ffe(@oy-zRQ;#P3 zjN*5Tq&Ocn(vb&_AV~GV0QSa3Tbm+l@8r(JfKOc)7(Cf%9Q zoC2yru?FIQ#OHtTs$W_C&79sgoobaS7k1v5_F>i9vCT0KR5vB$C?`|WMTUgd;?dG3 zg>bvMX78AEZ=>*q+Hup5(=-cfQIn{s%TbksT2#g%X69I-u#QLZ(uHZG`pb-5AHTdo zUz7mio9B(q{mT@Ut#N9zS%NeQV~G~?Vx5fobc+KE{((`V8fR(XeXfcw_)VOQoM zsO3=As%{G85hrwUqHuhvhylv#nzpLIS+k~TAHiYBeifqTH~r#3PD7By_?B~~&ZPwM zRp8+V-UX@Vyh}~VZdT|ll-)}@!abllI8j&y-bQCSp_kiS@nq%9UjCYm|I+{L`pt%7 zdGYHG=|0`q>*L_1YM{C(=M~iw=eDo6&b^7MLZZsgLf1zfb}R)9H4ivBceVD=o8vt# z;p#H;s%~kb@Z~{@bK^p6YXuYSBESoJDx{9x*5y+ zz+Y{dv#xo7uY4sw6z#=gMLI{*5I$n$&oHN= z3qA|FtU9Kw&8&W;sAZAgP}cXOca*2!1tJXP_F-uelzs8Uc^Hz%U?8O2qy0q9%%b`0 z{qAQbh0c*eTyEUU)+)IKmrN^^mBV+>H}dDoD05C}8*=0|%yRb1IZ14SXvZ@%J5EiD z7Tg5k7SnsoCCRhb-Faj(6B+Mt4ph#9Z^AUBROGzZGAya0SbHMf)g+{wx%hY{MS&?K zFc;wLz9RF&I9_K)eFD4f?vItNj^JO{P~W5YJOZv6iRMB-HzXq);{^7b-$YB z;`x=dG5%gRH(Duq*OVoF^>fLx|FA5Yr3mduq+n;epmU54B@?-huv z@6I+`yIcrWx)fRd1U+1X01x2IX{rWLvGgjV(mQ535Bq0{4wKRq_QX2wb6ufVj$~X( z^0E;{xC6kR=ISYVM=WcT(iP*jf%ND*$ybyd8c|Yg!#1g65eBtgrcVu6Rf06<>eo{t zSqtcA86AGgEba@C^f>YE;g`gofzLDvhVcE~olG_f84In+oHNazkmP_DUc%4f&5u-a zKVUz|Oq+hDz0JF}4q5Y3`11MPbGU-xx!P2IC>PD~kL!XblD-?|A!9J1s#LmxC+%~4OVFiDt0*BDAOoR)nguxK+A!EJT#_ujAG=apVL4}BTPJ6gH%>fZ8b2Xq8{3=h zYC3OY@~pw#N`JE@lF=?@xSurxtkof{&D#To3W8idA8m?VzFfO!1f<)`;eE*ffOVhl zBqaNn1%;LF#(YYOJwnN0dm6BG3v< z%MYmj^XZH_9~mmk`pUmSt$2ZA>#)384LaDJ3~cs?@=$R`DzhZ_xj5*Yo)cnE061m< zx+UJHK(5`~&&T=h&R5URFA$ud-pmZbjLgIj^sYu`YE-lwv>Yt1V*ACCdQ{XvSKa;S zVa+Ie>@`}As9H&7Dz+lIY5D1u5@x5;#ItT8b%kJBl5CW8l#Grv{sXCH8$6di{dp-OQwj%@U#IFK)u)5_ z^nz8?8a8?UT+>yw<7jw4AMrx3`OeBHB44FK+m04hqvvj@;E&CP_6KQdKrOC{4w2x( z?yuc%ZLl(mMlVZw6>i)ds6_Heudr3sq(`Q0qhR`*p1$t|3YG=Y<5)&D4eMevQ!|&q zu-JPjm>m4kA30EC=2~Wq`%N+b=!|l%A=fL%WoIDARi5tt16*#K3A0@9Bms=Iti7XqvfIqwxO}A#99u8OBc41xaDYC22xv@MK-C!A=JAMH z8t&BQ-`1MXQV^c!;OkhE#nKS|&0*!M7L$G}e%E?+QyqSr9rLBhl)1i=gDP{~y)rYleK<{@{M^#ssL|JdO~d z{j=Mank~AzH@u>Lz9V`m97Vi={o4()M?qWs65*xM-FjB|WXGM4OBDF>0iiL-GJcaZ z?v+tE^N30C*ob?C0}r-q3}_f>*7(rg=lCz>jt zn#1EqCD(QDv39Bu@sOL3RNoBRb6_#Es7*tqr&>3ioN{F^YYgG-|# z!mNPD8_ql&2v?HzO$8{olQb+&mwb437M|Rq`C+ImZH*S&GCh9&H;0xNb@GgKbD;g* zyKH`g7;c;enrAN7KnY{g&Y+~o+^F#hmrGX0I`z!h%B4eEv4uty{)e3}Sx>JAn?ymD zwL~5EK_ng4CL06M!7D~I99uECAVqcquB3rTBWa~OlY!FV!f@TK7WRk>e0+9iAZ0{t z_-|Q#pCAU*1eX{{bRJ9M&a4wzH1%NGwVi*r$NQ<7F#s1F-=sJ)`D8U!tg}y@a8dM- zdrfl@!xC9)g*-Pg0pYS~g6Z}5&>4WzoPM43ZJZQ@r$Gyt~$k1(;OT*Io6iOvr& zF+uSAcd}RHtv-dt*%v%M6hk0@Z$P;7WOby*k22%k)ZjbDMW*&>9!7yhAa4`3g!cP~ zQb)%=wDf?oyw1L{i%;qmk0TpI_NDZ>}v92P2rnu z1aaM9%uqQEXP>P{-C$|=7xt6zN8DuQ#!tvOc8AJeFk6|Olg=a;+6Z*cvFA243NSDE zMVP_dtNc}W7E+?k^xt3@`#)hQ{#Vz@XiY->_-`PL6!+D2>6X&MOp}xlkETLB4Jl=$5+#xoQ!^#YUNwrWRr_t0-+VMlYoJnATEzwX75=mMhCO|TwPRo z@wCUM=w!Ye0zwT)km(S!5%VXZeM)aVDw?58UHMuOr}QqY^NdFF~*dfcsA8@^1eiCXXEQ`V6yqV_se^a zVRh{@X(NyjlB6dW!1wc$5Te9+qEIPvh9lkzRw!ku&GYOyd$vpXas#eWujg>%RfJ~_ z;ub%4+vmkLSfUlf23tfShv<2y+OPV>qtqLISA1=f6g}0~xj?T@scO{c-PGLy>>%u5F7hpXQ?_AsWr@Ait`!H8;}&}k!_ zLa3bp+N85%*;|>>f2B8giGHPVgFfKkCrMeF^dd+*=ARy2_C2SS`qYkMT&TNi;-Opq z#5;mJZTuID8T0YgW^Y(b|933HzSYEKxEQ|?bwb{M5p_=L8&T{3<>}Y*#d^c`E!>{e zZum_W&JaFq0J14hRZ(#WA$X}JFP(O2d!}tu2N-{|fXHJM`7c;Hbkl@s-I<d`SV7%&*utQzvW zxfs~p|N3m;L+RMKtSv>iP=QL4`r17IuzrB<`FWO`Bn_1js zJkZPWQg#=AGH{KDZx?>59;J7q9(*CjCA42rVXgCv&1kePUs1?)N5xzgUU?#*`yzk- z=XIWse!hTqQ<7FTBa8H=^9pCQ@C z?Fnf@CBX7bwheOvVdhShJEjBQww#*&D}ljtyoRV7`2m(MC)Jq5rm%*dH3!b86Pt8G zsqJZCsMTxGf_c`eKCb!r>pwIN`-FaM^q-jgTc(*Yl!`?0uei~E1``z+_rC*2eFmnB z+xL3@sqHLI?E_GKvfMOq4vS&YaKnTmN}IO2oZH(D-Elt~2>%5CUxYoG&D~P`&g*Kr z^JMGuoYnD3!#iszt)_Z&ImD%=Vsp*0?D*djk<=VvVzET?F5r)V1G`Qq6DVo~T4|+e zNLy4lu*232nh1yALP=;KH7X*L_lD4$+n}i#eo=oT4`*jz(XD;dOM$+4%fHw(eNQ?| zVHW07T3C}(d%(D5Qn#`uH?C!QL^G{#$-+n>dPKEEU$>Nzp8;)(R*~}wa?LzyjeuMY zM*bgE<=KA$vaY1Zj{R}jHSZV%3TA1b0wR5ntv2--a~=87XoNio%_cXgXlx~V7KE+7@lJj{w0k$vf5QS#u!3Sc5k-s69iFhkH1FTr$^XL+eu+ccyssuYzdqq3}O4 zV0p-D_89SMZ{cXi+|*bIe;w}+OC9$ugde{1n9wy1O^%a?Ebi#(!d12W`BUen>Sa0)*Tm^>k>h>!!az#K3nwRO7wz|%V1lN& zQyrJIwfTl?x|#C!4#0?08o6{XtKc^KFYI%#CG!Q__E;LqA0FA|m7hpk`ag>P{wRoM z>v>i9eeC`kDFus9!V}>=`esZ8`z?@0*79D&FtLX^DlSVlxeT#_23gQM+Mc8#h9+B& ze`J2FuUUgeKhlEQ>-=4|)K)Y%R}#((SA8JZMbr%)2g{d{YAo#=Ib)rBW?bbDZiU!l z(~~~&_I~hXlj4PNG>6W=;zk3JAJ~5Zne^|6WS#$8-1txHjDlQLjEw1-H)~d$`lj+* z!ojd&5j_@MhUCLg774$};04FVv5<&4Uy9sHjA`y8ke8oMxs)|w5c%_6Y+xjYT@U8F zM<~(B4i!P{QnX*})^oLAR(C#s<3D*r1`#jPG0kMeZAX8kp`G&xzY;Ijk{V&SiGAPb0+8Pe-ZjHOUdmy;GI|O&vKnH8w-QC?SK(OEv+%32} z!IHN#Ggs%BlTf@`_{XLyqW4(4lgjyWaFyd1A5K0?) zIndZP()81~>mYbkGTtxLy0wt$oI_4!gcMtc+o8oDLcyKqQa$v!I%>R6vxm8~1gAaI zg8PoBU?9_yNwt2#7VY`5$3ctX>1$GZF^6?O?zspKVAkKQ=)hs8x!)fISG&d0K0N7f zsuM;$&{UMV4-FcA6a6XWx3G+@NV2lS32f`87uw+p`RG~6JSMnu+rs=QYHcwqcWsEo zy0ewbt|Rt)tXKzlgus9bI9P5vc$NwHZNl3)Ca8NEU$z@MGTaad6>BAnJ4zQW7XQdA zc2fi&DmDv^xKo3U&v98WIFyau{*;lwV8*WDd_U^MZyDYZyF|5Py1BWlVq)U}M<kHV2kKeQkngbG75{G={Ri6AW`m*4pX?0~R@c``-DmOMiG^v?H-~nj z0$^e*%IWoikpkd z#>QHDb%$hWx%pEoK=S~}KzPn+`%rm~U093B@}>~u$cvGOX4~=CsW+G{^YqIQubYI% zN8bBZLo|bFhiY~z(kF}2nLMDE@Do=hjzv3TWMq7CYd@Qh(HWNq7otAl^ljW6@3Yekw@FQ~U-*g<5JR3)<96JjqWI26a3%iWVXSL|;4-TP&Had}2Cv z`}V6q_iKs{Ck`$9``ulMcNm9&@GOZzBz%D zisIK_2!BlR;#XGUAC&!Xc=O+ar9mDNB2fMBY7!)$Ry$670)q$-GmfhR%}N&{K}Mta zl~6+bFVd%-x2K?`Q~J+Y zI;!>gCaT~->4m+nSzo+uM8tkKR-()T#Q?x$&|@nv#7Wa^E|Lxn3P8aE0v6ez3LOJTJ)8n7wvZkfs!~TkVKXo4UNpwdQ zG3|DJ7sMKgJ5-6Fzs??2$b?VzoNdaMEFpB~4kq5_Xj5szE%Y&{g{ zN2uJp=sduz=;07v!&IVQ{;1>nme?npyM+bvQDGNiWXe!lxoS5C;`W%b0tgF7#2LMJ zR|)&Els@o1jcG-2Jy!~^;P#9%-#WCYCoPz4wwMd#-`W@?X480+(a({I4hOzsM@31g1vBH~pGDwjs z0}4>q!!n1w8dZRF=@00KHf43(k*m3aC#Q@aMJyu)9)2sd&S=M{vRhYeq*G)_6l`|Z zkwy*wLWCf91L!VKsi&lvwg*;z3=>wCefEtJmh5*3G4wr9hR&(#(oasdFTcukwREmY zO{&&)O2Z*2Z&ot-B1K%suYEs(!T%go>noU;7yEwh?ys)%Z=YhcYry7B0rS5|B(b!< zWN;A)L+HvKk*h^oVsdWLGFW(m6+Lr3aJP%Av3uO3hB*|mFNBw#rFVy$NgOi z{E^V)b(e+T$Oi}9i43=?Y$wj!m+3mJcjG#o_10iA$>}dL2@Xoe%>CUPfTz!SKmzik zZ@Xh5nmCO^v`+ef0}_5QklEa2P7J~a360&t6lV^XP2!!2S>(VbFkgfoSF& znF*#fvp=GYbuQib1U!93;u6kSV?A|y$rGrZGqO5byR%I+1$wW!>_^q>CiK;o(}|U` z+VLrBN!eJP-aj;PW#o2prdzNgSKf3K-G6j~+<9qi#egnsaz7c0UoNwj?bnpn?j8b% z+bpXo?0qp?#H!*oYf`pqpAW%E#u{i;UaV>N{QE?)*h1J)Ed+SBwV`zm?6@ zcwc&U7s?wj>Sf;(Jad09wFgJrg7u2I0hw8gGD4}X@}7-59pp@A)9iR3Jw=qcp36~Jk%rq_m4bky^P6c%iD*;}36WGjzRr3aws^6o zwY|&R-fl-jy(~wX)Y4t1Tnlj}n zoKjDP)|_Mc6lVH6WMPgY=CPb6{srkzu=U^N{b}%jm8M{<)R=(8nHuu(;Cm>Yt%V&A?&BAOz7Uwp+Ig|KxHgvlw;vY&yq0+1i8R+1!P?UJ)7d zE;Yn7)88Gq$as73IuJZOJG&GSse4!bBO~-ns3wvfZhI&#?&8OA6DAP(dK7tN0w3UF zm_2-l&!&h6Grh4PT9Qzf3AiF-+Ajqg=U{IF0*6R?w&*fWL91N@XKMG9OT20Git8ZL zggswbx;}qh)#7W#4Ix}xZR;|MQ&`<2vDW1BB`5Sf=Lxn1mb>P31-#CEB{8%dIm6?U z$5sA>U(SZQ`GiV}H%k|xOr$ntAJP4fXKrxB&BvMLK?%gt!{;xpeQg$!(3RIy zH4gI1CynzZ2aXHazy%kP_#VpS+Xl`k4QZy1vFLsm?m8Hh+sG~ZK=R+l2D3Hpuxwsy zA^4@w$Uz{2qe4HIBi!a=fvd4X^5fs(GksQoe7gxE*Jmpxz5T1t$v)tVVk2}?O~xjM z{)ITirnl=_`bC%5*mLArh20d`VNFlQ;TNj?XD5A!Ks6S62kMv=Hj4@e)InDfbP*eJ z&lPa)+jS!+ITp8M$4^8u-Oxhv>gl_Hv0@|Xk3c8kQ1mQtS8mQh=~Vbt8syFMh@`_xIgvfFptY{BBbP;y0K( z0x@#gFFHp7LHgYG-<`8SYuxJvJ4z^Z-|IC83<+9o+Wim}0#D!A5O%>`zm&+aq@Us+ zf?>On!|-F-@7UKzt|UNN07pwUz%#xWGR?rd0kYsB=j?2Hew1y$KWlx zcH)4_ZPoU$9Vf(JFuygf_+CGde;q)?xX|E3xKP(Y-BK;@O9KSdI3vCq6_EB+)noS5 zsPy3hv6X_o7_w3GRQRBbikxXaY3?xnOz2-hcH5>!tlAz&JiPfT^fR*m>54WKDg8@) z0m1v)dRVYj5lU#(HXI_}?VkL$8)8}?C87t(!j2!J_RSGPyZDan&oBLY?d8rW)2eVH zAAd3rTw>P-8`B%0AZjdqNF7WVrqtPGz#Hc7*6 z$04Him*E(}iD)k)iD-BO0t&9+U)6S0f0p*2?z|wrY!AZ+2;1k(iYv`GJW-P<(i)1= zRT<7juyuJVF1h?jRwd;=3w&-ZPf%zjo9HNfH2Lkqnapr>Axq8V_qo$b#f!sSBIY^x z8koGlJl{gKB&jYfo7s6PcwX=G4f!#hHljhsym%tyJ1Mt(Y*{vR%yil}&PorPa$eJH z4lLQlO^X~h;ThCb^3s|#Yp7BOnrfA|T!poR!Cg>}i+|`~n^}WQ&1^ME zk}TseSqWw-2IIjfo1OM*BuEi;`zv>aaRXWl%7&hFJs`n zpnqz~$#&43mxAj2hUMV+S8{IRS%f{;=JOiGOF4FxG zqb&Odq|}NFHtmI-+;Nk-thaomNO)i3x+PPQA|7}(~-2Qk{6E+z{6j0eW(9!}JseaN!fQFE-mt}K~mPRng~Ty68Fk)Lw7 zWnhkSdZaU|R@me-^>G@SKkbjYx7zLZ-%OLNwRjn(6_$!Xlv}MOlWkZNZcR?>noqa= z@>Kspg|qQD%{0NS_GJS;0!cWDZ($^QTpsqKJ+hLo#IX2>fzs4C+SUk{Rp*Viu z?#eSbo*@uUG4~SiAaQ3r+2=N*_cY=2-E8u|yBM(04p)D=io>QtM`!UP; zdVTl-#bMJD?F+>g)G!Uz$vNGc;QNgdmY44c6`5CUq3(eMf}_$~+kCBubxKio6IW#~ z5;?c!Bn87=4L3_Y>5yu^`)@#J4%B^)%yu5>tXMx%j3GA*BN)}^wEYI6RM_EOkxHCD zf=y7MRk#okmn*Ll+RX(nn|mMZd=`h)3*l&ABb&s*pe4BApsc3erHZyrmL~2r2f<>L zTC{!O`4Ilzq;!2>GQBSj8!0j}=R}zB&>4kbv~L3O1y+$#!ER9Tz)XS__ne5)yrdAD zVp6Vkc$eOYk&)lluLV^_R5m508%4Eo@9jcf{m_bhQkru9X9qqow@%1*6^h>VqwPW8 z+kH~-k0^ZFa{8zajPM+Q5@N4IE{V&6=UH_qgHnY<&hc2~E%gK+Msvuhl$(H6o`4&V zHg!z;ic*wBgX~ElsC1-Mw4J=SmwaZKXdM%A|B;tFEXYzl9@n=htMhrJb|!Qh$56TS zN3wE8e4IQLR?8yrIvo4j-e7LFv_L2;bLA!s({ed_24tkB0*Q*BonSdHkXC_t8|5!U zR?Vm2m@I&N#Ik=(t67+M@n9!+!L z+S4gPPKvL%)|_q5-fNiF>US)8xWEO+M-TFn)lC$>p|$~TRQS@^{BX-q>eN4v@5;00 zN`jwvb}lV}tufc1(g^X<_B_fG2X&Nm^hP`w*|yJ5DeR9`uMolCWd(lsk4e}**v1W( ztCQm|($p@=R+R^wIoYxi5Qsm$W%0oB@`MovYY>!+v~`WEqiz+9)+CFknN;G;cak0q z#5wZQD-Qz)+*QeT=Co|tZuj@}VG+lUoH#T-lfLUTrQzw%`F!OCzM{^oCfMN!zi5~b z7Xn02?aCPs@6;`Bm7AwYAc!x|+Ah*PF-w;Ka%r`B!nbZwI;VZ@f^H$CIuUATs;KGK z-ku6YlV4v*+|1zc>KLpEf@@r-c3g*O?rb>)nWO-*!@8XdJ{uTvLJLy6#sIcjFg@~Q zx}^$FC_IWJHqb8V;l=OK&l20!-E1?WSeAI9<*6M-r$7#B-rViU`z$b|Zx9;uH8r|i z&my z+oy5yP9(pO7%?7g&>kD2ktR=UP*l`R4VgBt3PcDP#`beWAh|BG#8Y@;Z_=dC(EDhm za+;F2RC^{WtMQX=vXmQ8R;??;EiY{?7Jll>D36SZ0p*G1*vRFKgn=-_6{0m3M%7SE{xgKD+zgJ!7!|A z?M-AcP8vf1Q|%3@7MxH(h&bwswFx6(J2gRlepH0M+bn^rEUIx|Hy}OgBx-`uMEig% z&5m#A_NgmqaxTkMZY9qPR1Ph}JFL1VuAR4vrHj*2O!)&|0T5eZuRs&`2{MvkzYSa) zc_>AncfT?hnt1JUl@>J(LC0zNa0M3>?uH@!c&3XRd(_3o>xY~VbU@mNe~)J=ENV#HWFh>Qm+olbS1d`S0Zm(wb)bX zL@oGgm-gygx2JWtU%4L1)R8Dm5+p9}EIJ#&{KaYB)#Z$)t{Yjdv}Z~1{>`Xj8gmOXy~kohR-}wNrKP@a>IF5ctSS`^w80IQ;7`c&5P>kB72>`0iaJ z!@t-9aK{NZP|U-|-rU0V?{38!O+&XiDQw|Wx&-E^K?H;l0dfeU;Z(3%GibV0ah8T> zIYbl`yHz=YL^smNs4M!Syc(tyhG%rFAU^y44({&v72s!> zpe>aai5$c-{;WKpyntfer-1hrCw0yeELGKW(^HlZN~xbQ-unfaDm-WF90r*t!hH3X z;w9S*p9s&)kRlor7qtVmgbG)?279TVa}IXX;i$4=boxN2#YFm)?e)rYSWYFcc&`kK15zB^S)2Bw=|n$q8MQ3kuV z?eoJLbHkiRgeQ4S#aRL7hvc8rQMzjgEyF;G-))l!_i&v$kt>3&V!lqYVFj;$Y6fDn zMK8aOzC>7mNA;RC_F=nX607>K@aq#OLRa5^3S0VoiMJeY$na{Lhbtm(RHZ z4VO+Y?;})CI*c)&5rrpJCf@nzGAaPiP4`#{I1DHm;R9tMxB6B{oIBgYG_bu%j6=7R zlG$kvn)^vW#&;$kcX{9&EIRd|U6fu3<(^52q6(x2_DK1ZSgtXPpd?KPyM&48Jy$?) z?f7~NOdDcl;VF)(@%P~CUf_E|UhGD7=VS#XUg(#&wLht+RJ|9|EJU~naYcX%_&<)V z!jV7`!AOhn;@ZZ5AF7QXZU{5!)Sqn{jEc68tw!)hf5&<=+!GO-nbSISy^Q*K^C zMKG>+%D;}g2BU(!$=H1#g1yxm`Ifr8y7&lUnD4`={W2<1>hZ+S+O?Jg3(v(CV68<$ zWIcya9^5ezifx1>o$%gnU(WUaz+VffiIIG8f8!kdzx=wd4*?hGVCrtos^sBd;cD}d zO4iX`)(!k-VdZLS{~wdDlhz#PMA1P&K{(zB`MFfJRcJX`var<5QnGkG7x8fHWHt2A zmzbvFS|X=CJ{n64 zqnr{r+5nelOp4K^c~D?DZOYq|OV1Tx*(XNXSO+BWaXDm6w4@=T1mxN)iXy+nuC-UI z&ruU|Go3DrDGd`>(}kxnvf>PmPs;gc6a1JbDjP<`+%N4lq-#T6!Qiw%aF6MJAm{CN zppUHPrJ1L_FE~u5IQdRo>Z!2i$^@SOwWm}cE&{|RRvNI;MFn!4wRCh%ukvUj)&!2) zTi&I6naqf+w@jGLU1vfeemUjnYOfJp+ogDE+u<2KsHR`Wm-|-l&qUJ z(6gA%Rp|5I@kCj)I)LDNEJ^Uh@DAG&}wq`C4?5f@Y<)4nkt1vz~S20 zSZkEVr7=U!QclGvk%dC~L}o_-w?eIWL&)N3E`Tf`az3IrBgF{mbK};#k%Jr3copf6 zS3fGVqMzZmh>j`>&&(GIkxcmIsj?+%T5dp9q7I&IR|vfE?k505;&<`wHDwGke_Zmb zjl(9B2mC;7$a?h887HgS3jX4HlF24m+O+T-3}VvceYmxX=n7Tv-Bbv+IvHd$njzn$ zgBY}+K19??8BImA>O4x~nVDh?_4hKSndvHODi>(>lKEf;YVYkAy#6)&dx*N62UiVD z&3yYeHB;9Hre^Xdkzj!T*yi7f#{)bWC+Lr453}1@Z}nyYkL+nHIY2ck*MzH~pQyee z%>PLX#^Y#2ufcQCNfA5SK=VAdgx0=Z2&HnR5>1sW76 ztPb-AL|H|KfgWvQj|}b?myS*H>A68q-0=-m%@05Z`QRDhA#&3zLh|LO9Jhq$qh!_o zBl-0bINYRg6Pp&bjNkN_dL0dk*-qB`)C%fVF>(}1Dp1$+PufhuRWp$3ADi7)tq+L$ z$M}PEp9}8@OPrjcdc(llFuZ^KbCPc;Gscw6r0k|Mf|v)vqDpK!~Tj zAd{yZt|5ISNRD4*hyedr1Nim7TbGl-AD{0!5-K0qo0VO`iQC4}>i^JT`A-EFEfi&C zMP(`2_rD91=L;vkxTld*AmP|P?7glB4N{QFlQA|hsfy6Dvl?!R&@*y(gd}_bREvx7 zHc*L^S|U;g3*oBbOn<=yWW^)|W&~ouXMA=vGly91`wD5sUJoHl1FQ8{yES)0G-fz> zxwZdZGHEM#$)YPrFlnDxkzgeMkM$b;Ef?eYM?i)|3f2Q9?|;k2*ePKAdlSI_+{8OW zUdmv2o$_WXPU$WDYNiIN-tdGE@b>WmY%}wN*%Ot6RPi7B(fig16OsMXW9*}mnZWWf z3Zf&(chHnblu&R`jNwCHQK>Qj%YW?#up53~3f{@qzjg!u!+-vyO>7{M5&oC_4awiz zV1bo_#U+6SYvH7c62p-Kd9o(HOkfJ;+aA6qYe>ODgvj;It{2Z!NWnJzErs)ERc|Kf zSTuc_nQ4i7$?6#yX;XzGU!j{Zp+&i7jL1eI_7EU#;`yo;Jz)=G$b4|6U>#w#(2@RH z>k*3_UJ7_kE#TMxG?u>?ng(Z!gr4^M0Er0uzm^JC=lMseCyfLGnJi658yQOhzh2<2 zaK4#{%y^5vj)RFE&Jvz77*NTj5Cs12BmR8%_@bW^0&!$G*z5m13l3fb*TTvJ{Fvl_ zJ^Sz0J)aXKoV4r{Bm$t?U%5s&i*Iw#MK)6`3ZVc7SuvAvA0ZSG0nuL$s{pt70mfmX z)U~Chh_4oUm(zO(ZiG=%W}l+qW=xv=o2;lPYyWI+27eQ$+hXRW$a^46eY8H2WGv<| zrgTP8zv_N@3Hq=C^r8Ap0_RufzI^A^ci`lMf{)eIZeI8q*a6Rkwd686h+@F)!WkrR zgRHARN>)#7{Ioe@5soLXjDLZh)n#TppK|4Saz0h-4(k=iS@HvWVE1>*Y<#F0r*jDA zea{&K#c7osbYJJydM)zN7|(#c%gZ!5+9zY#o$twekwpg6#Oohtv}vX$jPF}ionRdn zfGqeNic3i~K<7Hjm#o5~@X?&&_rK$cQ-4?GDeCrY2K+338nI;Kd9A-3;pRQT_G=~T za{z@L-OnR|jz4DFY_{Km&bq`pScfSl*gNnal8c{T@kK%kJf#*}Ll6BnW}1TQDW3%c z6*H^VNe9?OMngBhZY15mAJ}nSToQd1_0wryDnEtjger;K`G5FSWor}0)ktn2;7J6A2WN`o_7uwdQTaVLa zcGDxj)j=H4aT9hX@DU~#%iCxhCJ+yX1S7p)H+>lrkF%fv**By+x8r%R@Awacj%tiFFaiziXkiwOU zf(%Ow)4}kU(@WvLhn%MB{FLfHwwF2jh(~*=7@b7(48;A?p|M^cQk##n?w1Qy#Hi@M zziv=+6D+H^uy96>a1oqJ8A|{rxaKjJz8_bMmD(vN$(QRk7PpG|(!^lQws*x4SpZX2 zJPkPam^^kulyvabJ*acDWay3qI=(s?@TiVqaNZ?{pe!c;0M$ZUx0|+54{FA63ofpr zDD@>uI)n7W^3$<|yEmX3K0@eeZKWra!uY&I8wgYoY5OqO{whcs9zj;wq4G}5?gTyy zu`jfe9<48J`jC;wsl_CZW>;It@B=e>Xb`aqe3?=QYW*a?>l8q7OL7+(jf_@{p;YJ* z8I}j^@5V2L6~4}6P!9!4K245k^S3Q0DlE=M6~dx;NCefD8#eFlONYjuHljY1?bB1| zn09I_Cl&|9_;+>&wFQQBF8DGPIB*M)or2zH&Ov1uh$)qE1fsr>ra?P%;sJxG>~&4& zRMMJYm9sXvHQoK^xTPma!?U zNFP^#n0^m3YfKj1`|g-hZk?u@2#>Y1)$k(7NNn*2QzFtF7)O8&Mz2$l3&} z`^FU8%G!LTe~T~HlRb7Uy8}`t#-=3m2$YZ}acQb`j?=$|6h|b^`j*|PCn}{R!{*FR z%lJZADOR{=yeE1!hhi3XTPBJ`dqdqiUKCtr3m#h}wC;$!5pz3m_M+RRSOj)1dcfgu zmREYLe8Rf+3Z<~1F}A$6;Mo%WZwjJQ!hrp=GHUEy2Uk=}}%cGpH(b+kP!*w-0frg!~$NV6TD@ExL3ghCZV zIF~`Js>(b;bI@EAL$A?!nDklN1ZK(=BIie;jOa< zJ_8sq-UwTaTG3AN?gooS4}7?Ie}@a-<@q_~Q^w_#dO49OmM&#T2OpnLQ=0g~-7Gu! zzT8-Ib^C!ePJGXO=Cd~aeW2^suxyH^Rkxu!<7#J!U2!}#8iWPhuU3yGB#+A7=kUYT zVWwwl6}R0Y#}WgGlmEV)3eRD7hy?Bo;Y0dYICUz8!%h2>lm9sp^-q+S2a8UHPM34F zyw_{#!GUhFHUt=Ssi~qfuymrYZ+x@K;K-TJh%OYF3Vn<^{ckzH}%G*9bh#w?uUTXI=!;o0KC-!bP=HZAl5 zNBMg?o=|?E;0uy2`JR=;t(Hk0du)B9U?km%VS=jRF{1f?ln1~$P)_icCk0PVK2W!@ z+YAzNKY8M*o0b_}R(?t!)^>~6YHDQISs?J_&=qLV=}bciKY=%lJo>7b8a3S79NFwk zqKUiwt1R;Cj)>EbbOp=K&A}Og#Wg=_A~-%36p+F*DS zW>fYf4&0zxGnONPgW%c%On11Z4{0g~(7A|PWh!rfRY`=6R*t4%6c7tea{nZ_|C}o0 z@~58fenlb!YU!&JX`=r^^~mzp>ydGj+3aPSFs zO-xAnJZN7bdGCCvG8bG1B;M%l|9f7#;`u=foj zAI=eZ>Elt{jvGuAA~K9Tq87Fe;tirbSjJ*V&-%y=_%pmfSrVWP@YRsGW)YA4VAWnK z)(0N)cw~8`sTzd}q8flhCgm{DENHBW97kR{Q3-nl2;k2Wz_Qht!nSdlJL?Y!f10?e z0KK#q?LZ7Hwf9|>fSjDw2ig>#P=g7mzHu>`=!mD1hsmOh|5ON$phwyIQ47+f5lqMX z{JEV9=v+F$VbDgYUys--f(=hCI1DpIBS@>YCtjgHJ9Rve>b?OHSPHdz9EwAEN~}8V z8*6hY;~z2asHLYy`x{qvGN>l&>C^cW2nw<%?%; z<$QjA5`aSFeVqs2<%rpLox3;G`XqmrQn@D>pdM(BRDs09KYB@S%+G}fE<=4{V6-ys zm|_i$Xk{1cG3xem<$4z&uTZtDB)4Y3k&dYm)sSSbq#qPT%U8(c7gq+IcK#3{RaSEx z^FufX5a2%3taw=}I`}B#we#*U(uozFgoTL!@)}`|hk@2Awg|%-xt$LM?v!OFATp}} zIE5>xvx!q-Amk!$a#m#3E6KL$@>LZ8*6tiV8Tk+DBvOxO(eFtkZ$b}D)hZrXmslr^ zS^uKTKi@J6tX;}YVVC#yqVww2B3_ZxOH=l0fYFQN7ld?xFg{$2{6<1~fa58refd3& zoK|~`u=P+4Jd8FD{sfYo>Ssf#RLZv1X4;*3()^4Jidm3O6Ia(~f@g^D8oq?gz^rXr z|JXySz}|3q;i55N<;0GJyS#Fw!uQK1auf_y!|KNsqZbOmbf!i2Wz2w}P6c4`Df?zc zpMZdk=vxu>ZqzU91N`Qv)FX8+`}`hiy>BZX68%}^&oE5VZUJgoWj7aXm*u1N*0%L6O9Fntv^Y>iib^#I8i2EW57!0>yUL3fn?Ro?0j z$3Lgfn>+vg>H*tmsfI5SN<}U6d|783y0Wm?ZknMoLf$R!livqmgb{^O7WEa*vF+8+ z?>|tisIR!53rx2(A^c~M`40hKQ37NdAQhFVqmv^Q``=taOv;KHxE%}h8x(>{F1D=c ziOdSorlQL~me28gB#s&Xdj)+y( z)D}<#2o-M1{24H87t1${JxnxA80!-Y6Wc~~$%jc4MI=u&-cl}R#r%2%4=f~T3%V+x zg|GF5Sl5VSxj#as2wx+=GouQDYp){tj-IPMCOgWqi8vFXpNx1 zkNhz@`<+38$iUYY8eo>`o?dq9=D3v``+F;R=@jxM9_9)G4_)m2Cj%z^F`S~KFZRv& zt7CD~sH@#*thFSOSP#p8$aR|gH4LyqP z9^yAUk7KI~TP&@ALNgoNNE)8?akvI&V&nfq7vP^{44=C42WbA|F>HUkOsM*}RPwSg_%`fuzTC(ig2p{ zk;R_5`$9N%^jFB|>6D)y#FsF}iY@1R#sL|9s9rx5O;A>tU!}XhSM>*JObTKnyLnVd z6fN0ATzrrjWh3O|7NS-lsQSqYfpQE2IMhAeX})EQ#*;p*M^pmt{Ogt+q7FIsro!8V zFJ^t4XxByX>F7qp3n20DjHAUquARRG9Xc>jw-2QY_$Sp8?CHs-2O9x_r__r4wmes zBzfj!tJNa2qXx8QswVFphGD>&Xf_kJAD#VIjVT^*_j(CBeLG7Wuy9bF;`(U z9+5+nILmHc>%kbZvOh672`|MV#`GEO%YKur+HWF2VwY0a!nSfKHu{gc zwNgnuY!Qx)qMnK7<-=B5;FqZ;PV1upf!g6LWbM~4dii3646fbWiT=(w(KqRem~8z4_Ierb(z z-t6!{WDo^Q@f?VghPQ{reAdZ7#G=Tf-`k?lJTg6DRJ%lvaUa-7F#H;a?M<8(^wl0c zeFX2@T$e;|AQFT>GdlKUo)~dS07cf&(C1EBC;h^?ClbuE^b7JQ6i#C)v&|&tcr^0I zzf_pvoffm1z_#`$vHve>?Ekhkj{k^NhHkUqp`l30?@*Fc1C|$8%|4>!$IMUT6Il_> zf5%5etNCMWYdkse8kpHTaL4MTOaUIBuD~J^M3*3$grI|+&AqdZ$Ly|^x2IFEW}2i> zq^V(GL4LW3{2w#JUe~@Ni)FLi4JX5r#%@9YhOnjOx7pwyV1<*{P(*3sEHzY9grVR& zj^$S^4@O1`>X^&er8*nd?C8Xv_i}{XuQP(8bE}AMz1Hj7{jHOkFTkd@(mO+3XR#Uv z&5Cix(>@fF%2zKjz}ed2KmxzS`1JMwIJgV!*LzEsp|0S>1V^*Q;Mf+dY4@{PYEEDu z2Y*_S1p9<#)A@BLb#$ccn3{(o>COJq!=!A%(Z@NSZrZ)uij4y0NWXQn@Z=o-J~mBu z2UQ2&)4~=;KTfLPRLx;rg2}YEw4_QK*{R+1^8~vw&-0bjF}E1hEEjUD+w4t3i604XN39xkFjz7V{97Ze`-nLoFyt#cnC=N5lRj-nPm1|q+k_XTN_Oo zTrwmtacdd%B>V}5>an`^B^;zlu|Ty{mRGv*wIp^EZ@wr$Kc&3)k9Rp2PxmiBp+!j; zRo6AIMEfXV;&7N4ybJU5O=SD#06~CQ@{j}o5ZfS%@^b@U49FaiVk#61P!op#s50cS zuECC^IK-?_)gD}p%#ZI_HAC%5%BvwXNF}bu+3~{~XRVppX#{;@qT3QMF@f_K{Rna- zSgh@_C5Tlwk$DTQ?eZ`^#}tyPS7%ocue&8q(l+p)tRF=6S`R8~8)Zx8Z>s;&B3&E= ztac;+m1@fUgE95d&{RLv&GOa8l?4e?60b#XfV)+p77qp-=*l<^d%m;~yEUqq%j)zS zmiBR0b=agM7ijurao*vC4#~%?;?_CIXo8H4tIhQdigHToZd5Q!_o9tv0sLjvDS5s0 zJu?7I;Eqq6qB`|jjhfr2Vufj9#m`RQM!EJ+dGZ*EpIso+*|tb6l(N;4`{}IiR<26nOy8>$B-z;-b4;Zq|3? zs41I(Kyo*PfvWEB>xX3)g|!>+E8M3G-BtInq(oBLVK58ZT*jhZoj?1R4zfuD*>^t- zOB56CW8ocNz|fVsiLhtbeRzxPiIZuOj-W(_zJl19`EZple;_!zJ`l~C^k8AOvR;OC zEE3-H86ofe&Ki;x+V?d@;Q~JSNQV=qHyWzUkydmO3GlpZr=#DqEeGLdxvuYSfq#)F zB)30Kw7^yw^j{(HU*yUE%(qKZ-lXUe!9+Op9O2EXAiq3pIf+@TPhD-$aN#FxL8=`b&l{rov46=vXQWm8 zUVXB;mwGB`kUbNTA) zy?I(aG1cM##=WYrA!#iqAcKvU9oqC1C_;(8Sj*y=oKpXOja|uivIr>)t^6k4amdqFw>I^b(l`?lanfuCxyA zcv|@IRH2pn9a`z-j-?D<`u(v+#j7dJucT|@>X4t7%aUK*13PT)68B*j#ofbzydqnf z%sG@EffM|*dF;P!7!C;Kb3CBE{$@qt(l6~`!Dh7d@2c&;!{dMQ=YPRtQCjjbRE92S z6w`>$jAUaB*2~CdFmhFyGhbDb5T+EGZJNv!*A4{I)sISs(3u4rX)H% zRiV_Op=ex2qshMTd7~UC#OaVp!I`vpXK15j9=vRBjmnIZ+t%qOW7JHj^#>ql{Q=09 zIJC(!9U)^g3q3<@xUR*D9c9Iob5fS-N8|l74^-5<7u+5x+SiV)`BqtSCdC^0&@NSe zk6?hTkFy$Ep{Av53$j#Z_qJ5;N-W^1#CN66Sx-0sRT78;yXAZRSnN(COW`7R#;e>e z3vWMwvYwK{nvC4N&&VuRk$>qhIMD0`l}=sT4=x?xf9hYKrtaTXg#Y+UUGE6HOoP%; zoJG;JMA7w%1pbr^&m*Um5kFk1req98<=qfRDsxh??Fj1%qu8f_ll9*0 zk}CpscSBWpAgIS~!vR!a=h8Qyi}kV{`WnPyuNiI+6^yXZG~Q3AbhOKc*m29Y%G$Mw z%sjMqNLX`0s8K(KXI^&0y9dhQPn zD^z)A8&peqlz)zT@)X$V#IncdYbcSmF>~0i&!VR<=ZMkaNwazopMp}2y-wpyLJoIU z(iPse za|;VTdZ{-rygT|B9Pd}sKefO}JWb{!*j+yU8zldCcloCRh)K~^$NGRi{2K}ZODx2> zY8{gcK}JFd;yZd!{to7#SRTk6UfzMHz3V%8{MZNoK+JIl<2RL5;SPm<*Dq%ip1+^- zE^;?V^Lj-BAkzB@(MtJ@oF({i`BM3eT$$XeoxkjQPbO^Z?Cbo{agu_1&e{P=<>AG& z?}Q?W8#%Y!Zg=`3mk;OT8Li)$0;V1|Ga`n^Y@N~dq1&Vi{=J++~F5P{`HxK znK6FiM7OBk!WtQ*1rP`>7%ccnMbXJNn~=`JWS3e{^{vw)D>H%SmM){Fct$cL#66?A zr5fWqM@BaDgFC*gb*iGqNCX3rQ*{peDKxvZte9V4#w@HruwCuV@6<00TXM6$F^}62wB(2M4NWZ};3Ij7ZYfWIa-z%?P_Dlz&9_8AP z!KshMa$33@J6P^~1zGUPGrurTo>6~m!@0s1p%!i^9dWJ;Fs^DT(?qP7hd-o@d_&bj zD}|4y)Pa<)SgU+ZRrmIK3FwUbu1{L83=sujgh9rttT?@7hbqKgc=m@*S|P{A=<0cKkj1w|LrdSR3`r|yM&>Wn`*9D%utbG z456T-WJwIrgohe&7a#=xAK8T}kZQp7{}J|;L2+)|wjnq)?(Xgu+}+*X-61#~tZ@(S z?hrJ1aCZ%^2^K6k1bCh7v+p_Y?sxC&s^&)*O%>}KW6d$=nrlps4fiRp7hd_YzoN^~ z#ZdGUAq$TgAqOEMILcu2?h@AV7`an|$&<>fg|8BF2D1Px7!TK5zvvASB57UqaWxe5 zT_{?HC2@M3jYktc+iCHYcm_w8K!L$l5{%Z9VNB4ocO0v@$e;3SOd{KrK4?{iXKDr? zf#!D58jUM7BWeo2uZ0F&Ev!C?E6Ji`M@d>yLJ*gb9Z8Uto=#U4%$eddTT7S?b>f!@ z+bAaoVl*-)W{n;X#Whi1$vUt%h3{H_`P|T@thygd`_!g(&gkkw>h&`!S*}=5O!LqV zIDJ8?k1op(lG$W)GAYu$&jJ?Q#;)_G76d$CyoDa0#}c zS%}>~w56`8=uZjuUgifSIdn_77I1J0=~c9dfKO9w+=74ot1TcqQFEz%1Wj*!hRX)J zspErhAnCmPqDUH{~RdoW-AA2rzj3@==NDfxf%#RSZkAHUUDw1k8KTp>j82UCmzvAWEe z&;n|V?2LS{E#m?KZGH96rSJxUoLEMTZL9W!?DJplskOOZUISlXSc5)034He+MEjn; zr?c-pSwB!eQQvn7q8O_ms~@U2ipTd@EL1S*5qA-Ly9`GM z#`s*K-YrlZ4q_)ub@dJ@? z$9)m72VgNq25Lb+Np-Z7%AN?ZXfYGR5k;S~oLEYdQjw=rYxz2F$y%xVPPJc2BNqst zRVp)G!Wm$kS)hQ<4>%3_wtHT4PNa(IBmJ?fa=}lXkYxNh#!MzsL#`Zw=#&JP8j{ym zotXB>jGl-;*jyvdm# zx-iTBZpQ4Kl3e1oCFs9Hi=nyAdtmx!hD^>~0A!jXIKDiG{#?8X^O}vwD`3m(@urUI zZA?+fU~4O{m~L(H;wA^&^u<`8n79mwqCI>>=PRLGlq*Y+SJ$CSe!Cvt$2UQJI)7VY zYX9E1!o;pqhA*qC&R>A4w5G{&94Zq3r~CI0Dl|XW){n;ZqEOJpx3F$F06@bN3yv7W z=qnnOhDD8G98c4#cNV1w@fAUasgkyWiEj%&9_n|YiG*w7Rc0qrufY6X_#C=?ss6b-cF zZQc~M67)c>!1!+|_(bqge+)6%WFBZ#WQNoZOy&~J9(?xEOomc>nU)xfz;h`gHuX|Ew|QMf zWjS|&ya5<-ckiA3b0+6gHB29hF(a+x61v-43-&smS_yDJ8G_+y9j;S2k*&HD!_nc# z#5L5%Le?fn*|1h=#atZCACufo+Zt~vO5RdM=x=pQN0R9}-+L-FH`s}BaQY?nawN;j zH4If)>fUdEp$wD9aLX3+xkY}cDD*VkqhCNl?#rU@Q@5{8;@H| zhQ`>+Z8dhcu@9>-O)Qv_0lGt3zi(QEDf!4X&#~>#VQMWEzuAiW^d7kp8QgL8H8UrR zCH*+A(Nhi~6}fz*DYeldgIZI(dz=Iu4siYrs2ec{oVy^p7EH9L zPnqZnl_KhMkqft~h)o_>b4DT3=MBl8@x&oVpnPPF*~?6`U!qmt^lOpldV}gCIEr3Wx$`!Gzfs ztlx{^aexKZMBO0&5{)I`Qx%D}&|m*5d!3!2spY)?2FdA=UJQBlEI;=gF3^)~c;YKrW~hpq)j&A>_Jd+xPB&C{Z0&TS0niNb04 zLuv!v0O4NlyICDMAabo43^xKE$@^vTxiWBciT}Z#-z>RA+10$uY*yU7<5@tnunf z?@?r-X8SQ{4q&_g)!xJi<&^ZLE;(Ye1#!qaur)!a zBD`)MPYFpTa+m+WDly8~-4+>Qk20lFV=kcgX!>H+iZLRW<)$qgUK)FyI+GSwa0*sr zymrT(ABXXJrB3Llkbk5{oLASq=Ypy((J;4Qr${}*+y=wxvlHnvqY0-&nTTi#*-jU1@@$NIO&U8-wKBS%V9+oR>$VzGhR*pG zTQCpl8l8P%Uj0}SL(r$l1HVWer7K0VX$dAge {dJX=wO$a)00?cOVFTDi-(q!- za-1_WDxgQXb~d0?@I^#R*FIR<=DpbR&UB%n?!paZw=?oS6bzxnTK0@|s}}RBj$9z6 z8xzZF231v-F5P`(j}KJC_8YJsGX?^t-yg69M#Rm1myUI!5pupSeF*%FeFr+Xg! z9H+ZJV(mW%&R^1&hZckz85`aa79Mp#K7LrDk9qyWm|iptlHZK6{@?Q#p8w96?Y|jA zNV$s!G5bbz#sz)gZQY-A|GleMQA(MLJW2n*J4pP*;AFqP;pT7LFn&EcJIeX>e0RnD z0j=}eEOd`fI@CPMJarEb+(sfW>>h_h$OdjDk&G*`(*Wjo;cMDl6`*SeT;z|#*1k6~ zP>n~vIlS%-w-59%ABkUUEg&`(Cf@EQiFcWfXIuYi{x&2~KvYY~ zA;>)Q6cxA#4l*2QodLguk68X|=BayU{`iW@3$0Qb%KnyG5I7KK5+Y02ljCm(RQ1Z4om zI+;FL(J**+@$R@&Qf&8HlTBD@jhSN98$iZpM zJUET<3hi?L=qMsD(?6R%VX&zIOmggQ5ji5m%FyCyp-?ZjAGup$D_|yccIWW6pd+LL zf@v!9)dw*QWM90s*@wcMy$0{C)E;B^7MH!WZLO6PrBw)GipDC zT$|eY%WZQk1w>6}N;u_)_tS^n-Bw0_j7Fp=l-VFSw2Rpvy|XAUI9Z7IZNCMZTOxOP z&IRa-o|FXjg2PM+QNYUYFaue?hYK93Cjyhk?&5ciEb=#Lc88P-%fO_mfH*$~<~FvM4+kQ5CWC6P{p({@MrKuiKZk};&b6zSHoQO1lt$)9H7uVYz_x4V*~dJ zeGO(^5V^SkJ0K7x4A*);6%q=nb;NP0hoPWjR*U9W_J!d`Yg|3^=L+rJSef)bJ>{6J zKF?%;qvjIyG+VVN`(5ISBBDB_%*aRE=}$GGiId+2>g~oP#(*w@g^wZS=p4s9O z6EPl1VU($OtL7XjU%0NUGsy9~Daz7F z)%m+|ICq8z-y`F#++uDGF5m?EiT_fczx!z?hdNJ=(j$gNmWbc>#tXn z3JIeHjRfbo&$*4MJQfV5&?GmNf5goUZR}Pj5$9p59$n@&Rm)VX;mk4M>zm}o#MwkW z8&aB6rJ$(bVG-`K|Cy(jVoP@J%PyaBnwjcIAzqm3uMfl}1kat<&mJR^|K`G$Etro4 zI6&dL1f(Gk9^I}gW!deimtJTPAe5DrEe4&YNqP!4TQan{Vdp%>3iLPtzT;zT=1z@f)>tU_2UcgER&-ht3$=zC@`jH&agOeMw?${+# z4~Tl8Lxny$7I5%tpm^@p&%!q2}NL0=Q(Um^5==d$uxDo^~Ew&85}RUeN&=8D&*8;HhA<} z*Y7r-&tJU#+UVW=`AHz)F?-&6iUj!lNbmve2ZMiUKEp@MF$85SKp0b)V;E6bR9`Ky zFE_a2LZMgf6IL#Rf31q66x3d@e(+i_e{e^zU+^8&9W+Mp9dvRqY%mHG3bZ<#4Q@0W zT5NE+2z(~gWN`X+Cn<9)sSR2a!Yb5hFx1elcgzaU_fPo4ohA_Dt`<*;SibK{-BLL-Ol4Q#jeO;U zNb_)AprR+4%IS60mf8kDqwZnxykcrc^@Hzh-HV8)pbF7-m9T0KiU!NWB1NTr(D4W3 z=P)}qmD9VbFIOZ#?;YY_fRQkFXYm#02FhInDGTPJarR%ewz;Kz7+f_?ye(4yNwz8a^(T zil$DsR+jD_|B8-U_UZ(hxPi!R4sdHwY9bW3!6r!X9?&&v`86NqqunZ?qir<}`p{FI z-W?f<2t6^}LqGSy@>9?);*|x~bG&?PF@J`RJMVyBb_g1saG&Zv@X5RCJ`ha%^}OW_ zd~-PH3nh+b5Dhd@5;N+Hf-)2n6;lml+iGy}HK+|}SMG+0on1%pIB%2;85NH24n32O&S0WC8m*s?+ z!jRX!+R$C=C@a+-tUr}`(2lf!FDr}q;HMbL(tQ{4As{-ng$-_6{lupzes*FrJy1}c zW%|7qXndc`0-BVR4+@_jX|jdiU9!l=*Dy$=(q3%aq0-(>(@-5toa<^VqG0tlU*wAH zWMa26)!lBQ5nO!tVfRW{nW_L)CUlExblK4R7Hsn!7Z%VZw@*$P238wReX^_|;5OXQ znX49M0k-%NZ{pa-y5wK@T1u+Z1*F(e8L0U;URITVT?(t-Q-66xmqAxpx$B z#VO~|V>Q!=-{@%IqF=ndf`1uWEVod^rMR^i9ymbbsb2LXkvCU6fAkW9n&UEgnuwA6 zVU~*?(~OseWV1HC?LaHP6Bc5MMnuWlqC^F^7A@~WG>L4CQ&iExenN$sROax)QkB`| z2GGg#T!RtTo@&Op>1ULlnma>gVVxr_RsZBkY0XEC@g>HRnX(wW@k)Chuc$E9SR7Ff zcLH^vOge)?gkxd&5!hw@{HK~Vi*=QpwT~Uzol^!>HS%Tr^KTu89SDM!`iq_*>0X1B zrh!gH#i!~NO0J(`Djg zY|Sk5+MSdmS-ymJLU*5?qcy-?B)OQzw<}3Ru7-J7`Bu{Kqf*8zt}DXiiBBPiw&(5i z^EvFoc=ACfk=5bBSsRPwDuZ6dez5T+?XYKZ&n`{&ldyKNjpiD0W3D62Ow+*mwg4+d z4N6FPHgB6~@^buxR7|3SQ5*}&t_TZ=IvZ2yiUyTIMpKm&50wf$IV-(>&Fj>@Sk8+s zYws^!ScD89KR>q~Jh!5-`I;Q9;LDwFXOpB5dy?2?k)&)s;vd&V5S4V3ALC2;m&Zz1 z)WL6by$LKu8eD??LXjB|g}9H#LIG@xlo1mc3KL)Q8nW&K`Rt)YsTscR=YfB=u1<`*32Ozbrr|)~YXZF04I`GkoB)8+cnU!-zI65vfC_ z70IqjwiMxM7V?#n+${Plb)^I&?GO%8z}$d+t8(obAMZ_>An30`_$M~`0rn=zo38H!cZ+PASi6l z|0(>HD{Q04G^&*Dwz`g!ai;0R!$3DQGNX%45CtM3;;wLy3#C&o$@-y{7%R-EHW`UE z-gFsMV;&9WRo92MKWa=ybqPn~4t$mW2%OdYw|3JDDgY0VVII4<(;rofn(;$etbHW2 ze>DQ5%AGjTNJ71$oa>IZtU-d!F6_PP&|0dD*)Ap6{K;!7`|18M@D<7hlp4l93^7bL zoQQ{_aY{Ms2uHbrkJD?cI6IJwB|?3-`Z*N2_{IZKyHKy zxh2jAOHd97v#eb}Sd&}I-i@KIn9=eD5Q=TYXsrhm$sGkE*WhSV;G-N5-#hVpE$vs? z5X`m`#?6*(ih}g1Qp-86t+@0Ml>^)u_>h}ZHTjO7G`}!O$I_?@p`P{^AK8+&*Lb+B zCPENA(W(xWl#8Ylzd*P{c25Q4$L%=4JNA7vTUZfdfbf)QNiV4^Hhj?JTR}g4ff;-u zxLpDsGOtjZLd5>*oQVI{gh&Iv>{s^>+mEzAExRTRV3!Y9761nD20+F}ogG2Uy7~D; zF7=~hgTn(eZ=&}FGIPx?X<0b%~!xm4?WK(1TDS*VrD8Kt-S8 z+nCj0lt_Z-kR6(`<2XXA%sFEy*aWh~c<^z1K?ZLvOYsY33m3avhk5kpt*y0TW{`Tv zKV*?}O0SlenWfSLpUCH8Qf#tuv?>cWv1`77rjImahgrrx>~kl$G^;z9X6IJy=Tnh+ zJUAJh;}h2ns9)9UJ5iX#w6&X1f49pLN{1Vc38Wj0`33E__ia{K&;`u9}Hc_zUhpWU(MRY^(r)|i4+5x0kImv{t>l}9z zZ5#(vIpM<{esA=3xa)1qMh6-qu`sHrml#=V+r+t}{EA(-Kh=zX?h=0^ z9Q;WCdX$cdlX8D<7+9bvK}$lE&fVJcRhtp%KSE5Kv{$#u@8M*}DIo_BPB7@{6#{^W4Sl8jmS0z;H`b zqew5pr;sjj)m%FSARQyJDOJ1}AO;Dadr37?MY&EwN@BIOXa}eF8s^OOU~Be8KM1 zRHQ1MD_I?}u#555|7CE0nZszrTid_Wi?wzrz-#aeL+Wj=nGn+!eqjCs@`4%(j`d^a zw|F&wZUY2^D1yv`6!O<3Lal-Y2jJ~w)CkEHXGQr3>5$=cH@{?5r6g&!We|eq zT32`rQt!^;HvF&8n_Nw^;nPBahpSumZ)5K(Ym8is!w6Wy+d0QFrNL!_CZImdUR!Rc zu=3Vm=w`o!F4cGmqE*4Slshd3ud1?8o_i9x>W_xUy&p9xu!}aVXqL9DwDubZkBY}c z)H(}QCF~dS>J5ow> zyUMsyA9yzxM`Th#`xJ2x&BTnqjEd7vPS!4>;KAJk1&VQG48s*3!9t8hL|6O;o($Z;FTB=uMAc^6jM>=ZzcKO2G%f!gpq>D5U>BFCejjL;}WCp`+ z>Yv49i{>m#4;k7>q!Mzd5#+G zW7Zn>me*C81Xcs#=3$;1YD_c26KCJ$^?0;|nLeueJUiqV&BpPSNC)+30*y_Qg*wol zhMm5)4jZD-WIx47#R%4#-oj@XR?h|wGn=&pG#yaJ;=?tdG<})zvB$lK{*5-ldS=s0 zFxq4xQ2!xM!AhPJLOe~U8ICtCkQ-n^gGM?EU7Uf-^M}DUL&p78;BU@=-~Ub5{(&hx zFBql?8+LuHgoDaWnX8Kr^^f&939E)54f)?U>Wiv;4OpN}{3;e}^rPaPi^C5ylx^|R zmnG!yoI#<%NH9q>iYF*SZZMUp#GeT%lV2aBLu5}&&^e@K4MEm2j}L$BhvfqjD>d( zB`uh~3r!vC+l(zJf`Ua9`;LtjK=Nv7bUuvHgN+eX@PTF%P;vsBZOHu2XmNZbEd4382>hST{`29x4uI{Y zeE0wWAlOXrU)oc3ROziKW;Kil6s&7M63&9?YzW4Jw<~We6(#N2ot1;f$V37lWH#?| zdnE)vb|{2oTN=+;oMasxWrO9}NK?CO;c57U-~(YPSB#B}J?=7VfvH*unk1AjCR4<4 z&kUM4!>T9cY?V1&2p{|5GFzAv$mKmXaC#;yhfd=jnw#h>Cu2}eQm2<)oH8qIUWnd! z$cF{hx=;HG1N7ohVC5Q~#f&mFxzWzxzowU(yo9yZ7{Vpyq@fHAS+s=?y@mlr5C0mG zMtH3BBFZg8Wo$3=u+AogyO}PR|b2p0YJ@Q{DYK{)Q!f=L`whdu=z9dB+yGf#{%}K8D+Wc=uA^&*h~Rm6+)= z*0ccabzG=ZimOjc2~ycp5yEtYGQ=$sho7;}d?c%kT#;@nahjn%aXe1d13-Gn^L>&F zRi(tVjjH_dUgn~CHx%f$#D6C%*`V$^7d8Lp3#soNBoh+VY|#v!C4*3wFZmc`ll8#a zYd%07iR|W&UD)>zQz}_tlz#rF`~>$7;s8CJJZv2;S=7x<9ZbO%lV+A~{{h+n4PEtl z6cJ15Z5>3^r8(! zov$oEEn9w%VmX@qejeyEuhJAyXfBaPsfsl(IV&JjhX=i)TVR((xQfv{__aIN|2w*n zAH4DgvfriAvmf@?@2tDMexGNuT{b;7W`v2?7CuD(l(V@g9O{VHDdZc}tdgOVn`i9V zjwz)3&KTa4ZIr;v`3+N%y0_NBA)zSeWlZvjhqn zimBdC&(r3I8KO4vi8pd?utDIPk1iZ9$>HT-n&UAM4vj|uA1)(Pc?8G-b299GZSGg3 z8PA-cWV-LC7e2>0i5}Mhoa`;kK2cIEM{m#g=6F2M%*5{a$x(CxKPQ8l#=cf9Z&d9Q*`_O30b%)-mZ@3JqNG*idF>AA!!W3z@=>^>H zy4BaFCAN8`dLIMwe3PFql6$OOgQa;o_kLCVYNaczG2!OHL(Ll^atQYF| zu8ziJB$&5{*wGQ6;Nk5_p*}_=ASDh^4p2ft(ki@5wlIB9owE@eS#z@@w%D??q~+jv zQq`2YRMMO^NQzP;dHQ5){3`ky-f}RJGwAU3p5keb-@V)Ktjikw!}2WrHQ+?x1NE~il0fC@5A!Dc( zGq_@aMSy&Qnk$h47MDz=4ubq3`&W89whA*|p&~PA7!3m!Q=rZul8Z*@!;v~q59^o- zYYbZqhXN}lZX7$IdObo2z19_NRed~Jtr=3sXA|Lm@K7JzGjm+{U8(hHm-HRD5aNL{ zSCxL*308mI{u6JxeuDJ5gux>GsqHKLKu`=QZ_gD8=aLI4E!{7CCvE+Ln0WSrLBYN? z2A1FpvKd28$;&e!&Y0 z$0fG%j@cmqOw*~5BaBga<{+d>9=FFItNSi2)Lb5JCKgEvbOn$JCcH;q;+d_AP#W}9 zj~{ZJFP;f?l;ocW6_q(!ckq8t2rfx%dFWcRnuBSs0mck`Z>)#knB7}Q$+V+5y~z)gtV)WEspr@l1Oqy&-N}8UsI6#YH{6`q(X=k$(26 z1VU!Z0Eh4uC}U%tv=9~``1b~4ShumPJmJXk&0RR$Rab>vP7kbPMWEvh+@ z6m*pcik@iGPS>431f{aLc#A13jx4+lpKtN^l+(}4=B!qQ7Ts{Y&gkFI(*u#+GWaGlf;pEtZXTZu? z$oaScV!#HD*YC&)cPfAHD(fy8x1?h`BiG@thy5Y)l~cTX!{&LrnD~hqfj8; z=_t=Ac*3!wE|i+>hOD3G%jFb;Wi^_g?@YJaeDB*GY>(bOJ8DnCI+el913#*2Und$C zK5%8gAd?O4)Bo8T8)+x2;V$tpl0HKd%i0sKm$w=> zS8_azw;JCmUGfQ9&baq8{CSzPd_JD$S22>p$!pD(;H;5%CJgTqyCBz4Db}KaYpgQ2 ztWM#g@6p%TByUxlc6nB;>3;e&)S18zBf8Vj$VAmBJy z?BK&pakh_z;p?`61m2%o4zb_~b9miyV0=_a$Y>F=dYc5#+IU556>0&+!GFuPmmCwH z!4H!GFH1e3;1H2vH>$xge>|iClr5!KVBeI$V^*oJ4~=3|tIz4)H@|M{BH&Kmh(Mm> z0_C(u2eMt8qku-^3NeO9lnM}05(R@gDmu-I)wqqw{qO~gjR1|sPtXkzze3H5qaf(J z*)vZqo?SjynVCRVc)wi3Xp6tsSiY1WKIaoXf-f2>D*GIaHIWqBMSrc6Ae z;17c2k$WdT0ck66E=0^#TZwE8MrQ8Z!b9 zh>X@(tr3}Ff*hkuhs6;DBs_*|RAaG{Kt6$QRO7Vr=dGrgko%Sn|E)(eHRUIpdQQJo zU-iPwG`^LvZXa1Y^jm2eu^i?w>BC#DU{8xI)Uq6(ftnKLh9?_sVsidCHXRG=!y~i? z<-~~TqyRMdPPtpf$_{r6$a}Qzvm9@JTkW42I2FRtLeTCyilaZMfd|CUydDc5G4@@w z6)hD9b+5H&Bee&9xAa(Cew;FM#`Cq4>~}f0jQFXpBirFZMdVPtGbYfI7i=?Os_!W2Xi1#I zRMBL3~bX&lwLk!I6BR*gfx^mqE_ZU7m($ZhF#}HzjR#9J&+wi z^q4U$X7Y_hZt;?7_ew05w~MMP^GIRZ^Q*nn(;xXczO*!L#-K=4VsM-0cRP_nCw>M* zS}qc+_)w~W{zU#@c|Y~nm7zX|H_`gwQ3px?O8yc4;~ejgO>mnMxK{|A$EjIb{j2sI zKmEH(GW5l9&I!NhUFK3WO2YRB?-sZ@brDp?>+Tx!uMHa_27>n*2vNJeuonfz^R(By z0KnBjmTBOYaGvp-DHn2AN>|22S-=zAZ2YV!A%)1t?KiT)m?2OUKM*Y*?(IrQnj2O% z?YWk;d}T-u86$m)EIJz!W>^#^4iU!^ z2YRBtyl}wek~PwDK;@_1X;3RxtjOW`kgGH4bL$|Gb!V|1aANc{gcD>#S&xAKbf;%P z_l1F+$~VXXFSNO+Das44!EYzbf;K*=MO{d|yQb*}rM4hmPKJ>9CHGp&>#sw$GE?-2 zpDK6|%RCHd9+C+vZEQ*Wa{+i2UlXPt6ESg46EyIaq!N_Sp=b)`u3=%lx;x`E%VjA% z4~BAp@onx-KX5C7#tEiZ=i*ag$Xw2IP2iTg8T!nE8RigPMiUU9yN096y94zb8Xe_y zf0_C#%$g?SA{~(z*e=B%fNx_*|6P=Mnt9nJolLYP=MH8a=+wa!wzkoXu$QpBW` zPdsZmjN@oD2aCK%r*%ZWuW0$g87qG!d_a{MPT)oq})ah3zIMJ!E53(eHFd$g-wgplQ zP=KJP4ZLBqD>cT(Q%|toBWWWx$doNXTxvCs}-t#nKDFdK%OIy9<;8o z1O)ohYvkQ>pz?}885b^iY9{GerqvQ})&RhFzpX$wuWM}mW6#mW1?R7a?FJ=dr>qNZ z#|;@}=BA(_JY@t?ZrhnQ2(F0}dGhJmVTl%{L~b3JXKCN&QTB$(Z5lla9pKhz*;uF) z-d3SQ`xn_;ezWC~TEoKGt`DaRQt8O;v+45P{cuw06=OfpYj;N_zB4tKTBZh1x>x!K z!S+&3F!MGZ*gW4)Hjh;!HEpZzYw}L5GX1Vu9-Z;OSpkE<6P!)`-<8MTAo%-5{U)te z2o4VD>0#@@qUh=7=;TdIw0iBiSzg-*E41W*N6;xkiuC8gZxr9_*fR3V8I!=bS z5u+TS*5E|NASoO7z-!>|$jGrR|1t{e({R~KvK`O9=_pHTOR{cpJCW^oHrd8;H@#*2 z1;XRgk?CST28um#t}l2(9I5)y%qS?lcP zW1otX*XMS?I25e48Lc;?$3VyGm`tW4Nb3nd;Hz+Q!3Hz>yYH8cnEic9xC}Kv_(8{0 z-%5W82`27eiP%2OQU%z?NeWY!&bm{(eh(Q@HvU!+0UZMT$cHf&pa~pcARpgF6AuU$ zMStHRn#;y%U9Dc23ik3M$2w#0lvNy%r0!E6U33{}{3%g2BN8Z$R&isSyQ%c zpy8-cE=HE_8ewNtjJcq&{TQ+Mxng#b&NjP+KGFYko<40m2y>@9X{RPn^V9jD=w^IcKvK zO8%Gi(eRx0V)poYRE${g4Cd>}#oh2m6&uZPD13=9k(u^WPZOgULu;~zW!6wP(`$;7zzbWl!egYzpWbiWw8>#>{-!74(CFSHh@v4AWp~IZf87CZg`^J0QC{rhzm=vvB zCUMULH}#i!5BZXLu2lESz*a~D!M(zAIq=_d>4CdgiI$fM z1wuol@ccQX)3}-HOLVH^?rT_y2wNPTN`J$`WaK|!q3^GK8y0c91zhJQs2})V{4U0Qq4suI5 zSDD|@gR5K@yEh~nzt2or;>F?Uk*Tc!eY!Lv65mFVdd~f>IgYuFS>zIBlY$OCda6|G zL)Xl<&~sJj2lADo*N^Xn)E7I_<}}f<-yO3^r@uaet|=y-(<`T}p>BOe322 z`)$G98#^`yQyIs$AnOQCb!^S|@0MFe*>bZiBd+u&?PjVOt=Ys2$r(+PSLrn)7A*G< zDi2l@`S^f)os-t(BTDoKBW}^P+Os~f1eyg|ut;52&{*>oCtPQXVP$LXk5OOug1!m6 zo?@wDOx3!?3`vC{TuW)Mqvwt?T`5|O9j^Fou`CC~6TH}M<8~kMdF*Zr8<-#3AK>}m z%?3H^Jv!gH0S0dodCyv>jJEfWk=C%gt?|hO!b>@D&RWXuuC(QW1JI48IMM~ofv}xcOvry=d9rfy`lv|*`c6QBaq#L?DfweLf58# zb#Jg1;{Ugi*WYKFpS;2K#Q%tUf9dxU%`I&LNz51Oj?tt~UpYl=`{KkqULJKl!4gA{K;iPY2Bwbon(UDcm1+-m&5ZE zbN6|`!IH488|PvI?yN8BtR>rm(Arow2v(Sf2v%4w(7Mp>2yS2(&a1#5;&bt&zRz$w zt*CVfmwIm?I>{w~Z*C3|@*!v-q0Zn@?+4$AwukbN4ah@qMwup~P9D~VN6p`OBeNj7 z$^j#Uuxzuj%w!_Q>4#jT2B9|5%?t+Da5XQHnaf8^L1%g7qo6x$$>Zs; zQc+2Ev!hOADn7A|pvDf)OB!-C7aOrE*78gB$|`5tr4DHE}QPwTEhzs zfeJwz$DQg1Nmrbw3t7(uF>LkS*4CJ6&SlhX@A~%H>qn{1>Dw_+&K7QHC-|;z!Pu| zG$<|qfajzfg~S&6`PAGTQI-DQ#IklD*UGKf;z0hEswN#neQf0G1H%!bDroz=L_DH3 zoC2m?%Tzd0Vh@Mnilx&}0mX}WLZ?Xt#*j=d#z{`0@6#Z;787?xoYbA$$fdJ7Uha?1 z{Ee7Pg{>5;*##(0MA=0_hAy;fnbtsf?9CeyR*CmHMNIF{JgIT@-kT;Wttt_j+iWK4 zJsP|(?bI8%jV!mqs&*KS`afj7b9h{R*EZbPwj0}4)7ZA%*j6WMY$uIv+sTBDZ5xf% z@S9%O{l4$>J@+4(?0w8+jvQ<4wSJ4|`EY^6+c(_+Q`nj`0IO(yi#J(#k>6|kTI2jyM zsEpjo`VmYkyD_LU&6(9~-G>8_Zkx5GjY79eO;y73LM1Y{HYI(eCslAjExT437(ilM z4$C*5?k3JYS}d?nF-4Rib@UE~U?f35()T9-j0_X}DR3XYi_vga_XbOlz%cQH>!HA3 zHaS;FG*76uKE_CbuE50=$it|~SW|0e{HJ{p)j~_srI?=~w9D8KPjU(MHka|dD>jAj zdR3m!iNv@p)J*JDdvK>#?|QUa(cP~*V(RRUus~}l1d5X- z5*!NDjnG#fw43%;s(zvsBAhl|#wy@}Z6JJR5w??S;+GggA}t6_DfWciU8;Wz3E5K$ zaua88P|Iuz56RkpodZWk(AwR z9Afr$i$%{yLx)A|!|VO?BhIJdEBV8hWO~4sc~p;Kw7HY|;1S(Lsg_z@rHS}pc?1ky zx6*Y{L`p;`@4h2dr~nfz3#1;V5|$%v?DtM;BZgS8sH6D76#=|Eftc+~|66c6a4JYS z>nB`Q{4ayl`1ZRIAzr?vJM3~crNDVd*xpxZ9g}3CZyy~UKtdDV)s{MDY% zpAMT?Tl#YOPU+E4`jH*$mBZawya7V$i2t)yVLb&G0PQ|Ok5f? zSE3=`H&0sE5HF?5d^8I3Joxt=gko#~`oTuiT!c9tlNkKU$?~-IA{3OwSYzI}p3>on zyu!9l+*GEW8@1(8g=~MD}*8#SKk%?(40;pY#C@Kwvdd z#%3e0gTvKw^aa#r=SE(+-PWpoGwpGlKo_xOOvUAc*iscx-nBL!C>Q+jyrbJw8|TU)xjgrE8RCz)?@9~HUmH+JB!J_6zz#i*<@buIi1m4dSHEy1MMNauW+eIQLZdW zwyaP)xt{bM4Abhk#A=CiRwDcoqmp$UIIY-dk``|urxW>ve_7HEq=5QKYySO9Z+v#l zyc04Wp6k44$ez${uyFl6(9z}_43y|fpA#8;$Eb`Fti=hg3vS`=uR8!X)*yY-5P>wt zBf*sNrr};yzu&4mubZDp7VhV^Z+d#RO&Lj#DrXrt)br!2h^fg!xv%~O<_DBX#0rAq z&6NVfqo?ecCqm1>lf{E~qNMV`lO6pstcq|uG3%fO#Dx#~n64FYiK$ZRewZRdsK#cdrPMvIIA@KO-;?m2vGFIBZ6-}&`I?NP1nftAI{#fB zoCDI>g@IBgDd_*7A^){we_a&JmgF`>TS&-%2K}MBK@14;ilC9j5DQ^ilAFB12*aZX zvT)5#+%OrHp!ndR?3DIhkrS?tHHr5O7=XXfFCJSkR(#-zmBpc-g}a5&-!ar}LWMb4 zlfHN;YDl`~y%MYe4gDYyP>_XppfBv7*`|17?H`X$h{!`PKr$OSYt+H?{b9;?RVU^^)dRb&p=)wtA&iYSuB?W$u;L0jr=QLYnVcAx^FS@8GG3@%9tKL)Fwir z1-l{mpX{*0;cqtm^d-h&ln|t@KV=UmE|R_?j{&;sBB&z1x{oyC5heoNSRJ8{g67Y&go)^sBvP)J ze=yi?L&m|dFpRgEw<3rGO_w>=L!*qbxDZiDEj&M^82b^|2ru5PvI;i${4-uej1;MX z?E%Vh&;F|z_OGs%dk`JL{?S#7TpuFNzn7^N{fN!~Dv%V2c<}GlU)81(4 zB2d#xcU#n=lKdEKpV5UuW%b~6ER!WPCKEh4!OG8Bj54HFzinXMl(sd56WpibU`v`W zgA1}Ok4eXm`^MXp;a=;b&0x1av>Zqcoi5f$UgOuE9k(XzAJlg?Obe!jXye6;O>o&a zq*9nQK=@`sMDlTW`vdduH)X_m5F!E$qz`Ad%eg~lWneH59Y9T3g` z06L7AY%CpuN(yoyng8EI4(i{n7qD?e>=wTfM3(^_?c5whd4rS z4y$qXv;LLD1!~{04%Jr?FM6G0_u3C*);G2ANqXMwl{fuc?eZa8yq}B^^=SpeNtNHR ziK9r(c#;b67dk&F5rOI}ymdIoksw+8-OBjCviMgNHE&4xsh^)fQ$rIVDr6857Kf1i zcp>FM3nQ2gmDOod`z8o8DQJW)4hlB>U^z>|KC3Qy8FmxCa8~B?^(O($-53JY2#mhN z^>qHDj%@yJq3_?{kb8q{`sy5wcItir@iiP9LeTNKaP5~Hs|d8Yoi|28qS+RCGL}Jw z$C-XANdgbvhg#rFAHXigZ+++3BsFgXFZ0*qY;+-Uo9Krcl$AqIn#r=hWhOLR!Uc;o zP%E))A>I5r9a-+e(y8vDbBcT`v@ECnJ7LMi=geYebCDtHo@il0ak`fF9nZ|7*;us; zUGRvHPg`t+ZV`>BD#JRST~J6E^(~)Hu+~T|?yX9JcX*uJ8SJe1`a z$pv#ri!Z{O;0Hk3D<6LOO{jKri&WwhZnLHI-2(d+e6b|c`>Re7>^gbO5MZb&j7@K2 z*=CZr9H;gup2{k*m%dQ`Yxr)4%N3M*iBHTgN)4eY*0sV|n(FPHf97aW_;)mBKx%LK zugcp0DZT#v{TWn_?;S;iN_DhHhE8SP0KIexp`ZiKKr5-3A9m99qxFq(v#X4&vs{a) z#AvcjArW=~&1$fimR34dOruaLsrnhVjZe+pTT{%wWm{&1J-MpkzoNAGO^H4Td-w|8 zFQ}9vjEF1{M|mH!UA`Y>4|lDAqYS-2I^pMr{{e4u%mel`w=gc9puRg(i8B zb(RVu3FU-zR1-zDOAsg&209@nA1CYwLlX^i*bYG8cg4+1b>LG_oXi|whEZfC2#`tF zgiP?{=3zhDEC)WwsO{v&P_s~shEZ5h@X`ZD_xPw-s!h>piMSNb9H(nc;g4MP#B?N) zQ(^=ZP90fu3;b{S3=;Ou5l>sAqGB|!N@FyOURoSwhSz2T#Hx1MNZVRud3po- z8|{n__9Popjv3@F8%*_o8c@GcAkpO`m0%%?``GW57c#YK(`UP%yOO42G!tBMFfubD@q-~-t-H^gkTH-~2YRKk<%aBX31@2`JR(UFse z$v!o@yHvpcMPEcThQ6tDJsL=FN}~z5ks`G*7mKN|(?CaIixgt7pJ9lGekPNJ~r#U4pNjMJOLUiw`V!Zb*n>B%3eyd($uR?jQ%iM+wJ! zz_&G)PQkL9q75|cnX!`v7`JK^|qs_UW*98*Azo*F1;w-n}&y)MurW%a~*lQdW z50(iugk2dR-#SBb#oe0RNOb-jf#`(U8up@L;pJCsi}iJ2Sa?)Dr@P1u*nA>-piL83 zAAdx;h~mVa$+T_X6j%bd`o+_bxErIO!AnSce5JILfYK=qzlH8jYgthJV-sP_ z=1pNu=7!nAi8Wb{(syY)rw|&nXY+C*%lI=uI@Q3?>#EI!E+j*BnVT5P^vJ9kC7+l z2-cm1))~=Gc9Ah017&+S4~yk=+0t4QX1R5qlf!Ie`U@7*Y09KQ?uqP+skbcL*3=Q$ zw1PCu77_~U3Nm4@@hmy<@4|D6`;uvKGnhL4r$7OQ9IyisLM9x)$Vxj^!^%K&`wUs{ zbzK$rGHb8VT8njn#LD-@n&^zTb5as`bB~~Tm=TZRjEqQ2=~}%eIKT$h{J^Pidhwyx z0LB*}x5bleH~k+?-G?r$n>V+F1mS0*e)kxUgwO+HIbxb;gb_^za*R>&n|-t`F@U&_U5)vw*Oi%&?WCgSHe zeD@2^K^%&78@+@0v~Al&&2|4qvD}a~>ENJbgY>_dH3KM+83FqLKp0?yBaDeiiTI0t zwF}1wx0Cr}U`-FTDnUCRs1^|+yWBf##%;?5yP(t4bv;=T!@X%~k!diGXEifv*)@X) zNZRvp2Wr+g6R|g3uK48~W!`?jSb2S$s_6kc9@OOYR2@hNiwdiR6%D2(qu%vH(kotF9|s~o9XOUM_F%Sp(tv|CXmG=D2=wn8Jm?>}O2 zQ*F38b(Z)!>tHE%c9(y$V%v9EIFOXx5{^Z-sPTP8f#GLy7Vg+V0!BfM-)#(qE<|0m z_>$ZPKu&L(-S_vQ)=lk+G;mRa(_Z(9hNYoXfgtf+?L5w1py2Sfz-LigCqwthnz>PO=RwF6FIKuDODNandfNt<#j^sg!ua z0*;zp#H<`IT*|w++RSzLOOx0dc;}`O`KOe}#*dChec4oW^FCSo!in@(WDxx(AmrIt z;!ldmCMH&U2?=>}oyGYBKA=9vFnt!}F2q8ol!9A63S^GKva=~Rnkt<731vzt)RsK! zY@Y#cUV{S`U34Opzv{A?s_vF;Av%!&1#q6MI$@P;I1eJS1PWY#^YL`QMc*MiK6(Hy& z{y=pGtcrV~1{sN@A9`UkV5hs!wNXeg*BSyyj+v7%7f`+vc9>g-k9>#8$5wg$m33+& zY_`QeXYvq!UL-K=vk=D#0_jMsoJApU29G;p%5{~GoTznDO0t$6`qnUT1h?bx<~zG(`b7ACAPhlV^vlDA@)hcDlBgd|VU< z;7{X`;)90iaox{k;2Q;RPs_bS$|s9Mlqc@6T`B1v>-EQdjl8vk=i-3C4^Y>pW*lE$ z6ErmA`S{rX^|AjzXuF4ez?&G@!y=zT-;H6`p(0+dUPiT28Ac&0gEU&)r;{KoZt|rl zMxBL;|*AY;OJng;RA zPopi4&&17MhTwVXm zqvqx_4$>Ww0vI-W+>h_n(?4H$!!cb{@Gv{s61Jvpztkz;{i=_a6$k8c{nijJ!2g5x zx`gzoDUSotmYd@gUgB-XZ2xv5^Ci%Mv>pL|$OUEQp<_-(0fCS4>@INbsFT5?@8VkOk!#0$ zWH(3*57a0ju@^;R`fdt&Q>f~|}GFc9rBk9!rYikrIO($Yfx+(L=*-&_Y_e@#dk+`3nm?!*h^%dj^UL(tLm zc}B{*qlSJ_Et$vB=vqHjH9&ACBd0C*SSA};#9Z_12&rs=+Kl*n{NyT6@aR9Q z4J=(R6Yf}`l<+U7(Eq9i;13|n0M-;Rlri;TRuyzZ_)FE9^2wV&M~PpngoOuEP!);M zr#-v#2`-8jv%4`3JR^JuGwh|mQobBgi_%L0!h_H;?7O3>oeQ|bw@csC6jo~kl z!m&OJB1g^-;m5zQZY|mA4iSN8&6Fh#xV2(2n?3Fqhf2kaDx&4flprLx0^~>0wdR@< zs`4Sqou&;1$KaV^?YoZC%?)4?aVMu}*80OiZ`A6ox$%v#Burqn{6XfnyO^JF-j*)u z9+=LWuw#|8_=vDmYx#l2aPaQN%8)H{$7t`8h`(YcyTocS&tArri8&2vFwB$K)#S_o@EHE|;~?Nnsb1 zp@_NlywF?OZg2F3Ov@=tvz{0PB8VldyQQ_k@P*4VyS}`6X8~{4ocoTUj!K=tuR(h+ zAT2h7b-=5^s>Sl#HPwEzBGxGx4}LtyucMf|f@eg*eJ?8aLK1BVYQSSJ%cZwlB6r)p zPMULxxWjeuN?5ktuZ$J#-b;8zYedm3?$jUDZH!xPT&#a`#0l9ur{T;*(@-kDZ#&Uz zA!LQ+Sm5&8D095zI9iY(B2epx^byG*5RrpGA2a&P;D)X3;1nku_D>`dT{m5Q?tZ$G z|ENf#IK$WOa(y{9g#p0BWSbDD#)@2?Y)Qr6SE)Gh)%m)<`l0+XwV?;{JD| zye|Wp3Pcp{^0yO(X}G@}Vaku=Yyuwe)oC9x)xeUT1~cIk6KP(s2*#?4xH7u3wzF9} zoZpEl0K&nKcHGr7m)%WjE#1IiHT}I*!GjVFcJVGRn|CX_;l--S!lUb9>*Mv6-e2{K zT`SH>a|nmc6|aKrV)RKRgt34W7{4r*IiQ?GQxOCdyBIao(rL37Io{2klA?zUURJM)(#i+G%o9h(*OJ68_ zij#C7JQJI5R-xT zT~&S3q`;(4^bjFQ(d{r>p;`%jaD910);wkIskwMM3^{>9IcI06-YiIkezs@qan1SS zJVnaO`utH69u7E2_PIfjO{P&^67&F6P0T23P`4*Jji#M`d()f@rKe}^Xgh9iF5@?o z%4s4T6QU+{%Uh|Ol98HhwJC;Vt!W`Lbs2J7tcvDpBMqev;+8tjX(4pU+tA|d`uSWg zf3BhzwnEwR8;@#BBT&ri@{;cUt8JdorTuBu1WU|jM>hdJg&E>T`kM7wb8siX_^e7p z^^yCi*-z2p$5FFRi(HL(K!%MgYhpSh+)c<)b5Mo3K`W#CK_L2vvqM$-TBIGo=`DZF zeNo3{Eo5aM^W2B+^efyC&9$gq`{NXLps=O%1xOspxkr zo=IAjBypa={8)$RRz3~#mHj!|{=O*%28wd&?At<)VJ`2W@_z*I|NE5wlp08ZjF%eYgM^UU(~S%Z@bSP92UTh3+(*92k``6n zkv2lg&80MAp)us&p;Z+kC8!|YH%U{OkA1`7`Y94g_JnQt6nfnu z;SQOd3rodmnkVounvzIB@scMl%LdAgT)~H93sCQUk}zxB4UXaxpf4br^4ac-B&Ok^ zA8Cb=_Z072gGF#wj?)&dhGnbqC(qla+f>RGR<8{MEJ23TwuXd(6X9Uurm4pDnL}yI z(YD4|K~a6f+;^1jldr3bx(%ejs@#@`%7K4C=%maW+`=(XOP$ZH#z*LL(w{hxpuC&0 zX39IYW{ID1%dTyVLetxZ=57*m)Z-kw$;d`bPvLGE!E!!-s5_P7H3x|E4%or@O2{Ajtt7MXl2Ak@zj?z zf|CoHh{gWg6D6Z)(#7f&e`AOnpU_)}A7aO&=ds$KTUYz>Wh=A%hDCQkNZ7*a@L9#xOTCNqlVQg&2)2s(jVf1lS z(aJGroia`O(agL2#z}O=KkPiuOBAHrZ&QW&r276x$AYMmK6#LLPA!#OTiMnkTj{R8 z>|29?mCfhtA@+@{ynxE3R7#FPPWq~Ee%h+KH%QV-=vZ)x+X`V~ zQ%n}p@ht+6(V4jTym*?b=nsxrC2htBBezN4C`O}_sw0A@H^E!ZBSm&{PavtGs1#TI zLKe+*=sB=V)(6DKr3EEnI5qF#;a>^q7&dCyFA-p?CLMRuKDs z#zBCA0*OyxMIC(`cZKYB!#x7<&8yyaMsJwyWDvu*VDo5YD3 z72!kM#2yN~f(QSnf%p-*)EQ4F0M5Y{Y0A^w;f*h5{sY=${J<{PYJT%lJJf+l(c$y4 zh?TL?HzO>4sB0vp35R@2Y+Svd4-0{&h)pCyob4`@8MNA|u3J=?7m~TgJ>ic%p^-_d z_2EZz=9u`wTuC+ZuAX&{=ILb#Q~rniNCS)8fi23-=&5V(nK6SKDfegwFc_&&&3ss1Er zqyW4B%z@8ghg9qK(S;d)ZvCn|FYWc23LPW)YCaHp7lPT$J$-TThz(WKrEir-75h@p z8GOkaX{*gqBqag!33+_dl`gpE(zy@)uVOoee=%0IP>x|v{ixhzVmXR~vjO{Ak_C$S zh4f|&K`k=DZvkO$i%9ZxCh3KAXkCnT6?|-S59;%4>UC@-zaWF$mlD!?@(1!{M9P4% zqJtN;UP*&GcB~92V-)OM=d{=BigsiW9jWQPA@QVOoy=~<$b18jr1w=<1tSqNO@bl6 zuZyYW&G^!XorQH+vr@8Ot+ZcuS}n4 zb3GE_HM258J!{){F{_@pV{mp9W~J_dWX0{lDVk)a9p6!uJ+48kU%mK);_J9hmFSWI z^>fRw5zc!A>$h;Xk2E-rNB`upjWY(m2cR_e=KrO!F#l(qmb3-b)s`kkLj(LfyB4jk zuem0H+GW$rRv#4u@=5orbnhhqoDj?$%$}PJcki2w7-~yv`nCibsgar(Y>RmCAG#z{P`xw=!YFo4{D-2YsKXntBMdCN=?^MLobPl)Vmwk{UZ?FuWXfFv%3sP&mVcESpWY(A zA*Gh-@c#-q!(Bvf3mw&8R8v*wSmSV;;Ix%xjTnrmIF!?uCNNJ`B64=F_|}5weX!3i z^vCT_ehKAEEy^_A59+W#N^9$FpSc6r+w^C7wj^9cowDr9OwznI^2WIjxY}NCBtA%7 z>+~Uh`|BRv>tD4r0%&KT4{atUZ-Av$d*J;4QAXvRJm|%C3 z!jBg#WU#Y5Y0h86E@hSc>-!gN*=tsGM41r9|GE3`~pVG z`=+w6w0Ym}a|ZS_q;1w^g}PoLOgd?yi1C>^i<*d`SIwfA8>_snkG5IOoXH-+Y)ch3 z#q%hZsOCCKs(KrF;6IyY3|E<>>eDd$J!l!sR#!I@>^e5_=AIOJ??F#_OlGuPEWb!ZyB`a7sO`_cG-H&T>;8yH<{{8D$$(xU zRN8gqL4s~6L_V!f88YIJVjClN8 zj_>B=&?crl_h$%mBb_16I5W<5urv*g9xzoNKxkxtC>E6mKmSG=0yi5vqcu5)4F5Eu zJUET^#SgIeN0FtU#VLwF(J;%rO9rpko_ypRnz2Xi$ORejo7A9cr8gZ_X&SOOGRN&a zCfZU?H~1k9M{I|IWkUA8ICE3g2ULHj&+yYZDpI zBGFEPW_K=JJ;5Dz@AE%8$vN@#>7Q%vB>Dn`dPyj$+-T_7T$~gp6CXRJ zd1@S{m_O-NDvc|Tt1>7ts4yrqtbL|g8OgKl4?{vIVk<9AW6>C5kbtXPeTM6Ep6J-+ zUPt?76;4xGIf2p zV8XM9jE^H*A*bPZ{+UTdu4ljl^RA9d0hwI$h$m`yPs9TL5X8cCPogq9Y3k+&C}ed>e-T>Y>YFrV1E%ks z2N~X67*~$dr&kPHovfsr=Y_$kxpJ0OSm(?VS5hydXe-YL5nduJx{4|s#S=T15Gdkq z`ifl}Km=M5A!q}nWy^E2W%&p)mYkojWMxJxn}k0>#;PCO*{e1Qn}ia^L3jL@aW5ag+Z zXCe;W2!2%QI<22~5V?@}yt;~9xr7(5{d_a(2B3oChKl{ddfQLrGawuBISV(Y93X3Coy9?Wc$Nw z|EH<4kbN<06T+JS83`FvihY_T0UPIl+0=B=q4olAgP?BIL}WE@W|~WFt|rlN%nAh* z3V=*w-#DMNutU@&65Zo$>^6<*X+Y*SZTVcpK%5|OeReI3sHJA@6mp5b+8F3@$rK~4PE|jW`M-}saTgh@On`GFf;4V1^dzP z)RZAuV_m+nwy3D6YEW68x(IP1d@-gPrd(>DV^vgTAYnOQwW+iuQ3Kz_ZZk4Oi|hJu zehdJnlod1TbGY~j%bewE=xdrT@#B`(+GL*N9Z&{X{caawW$?HggQbdabxtG$KL;C7&ios~+EX3>#@85H(25#NK zPv+(XNm@LE4Sqdkz5U5{3 zz=|rUV3}geO&YvOixf=;n`bh}UaCTnRPtNC?D=9NZQjyU(tsJf8=fLpg{1t;EyAnp z!it_H6iv!`-Be~e?_%5Y?JbVKU8j5P`F2qd7EBG{JYglavi}ho%F-Yqv3F_y{thp; zKxZ#Th_A~Y0Z@mXXFGH~w(ayK2hb0JwvrXyfhzh2Fj>McdsfY#|1LQcC{;1BO#@d& zVr}=|$->(vVK6l|=BC{K>KBd`H^ScL7F^*B=5Xp_N2n+bSny9Cb*N5EnFuoW@c&`# z|FLz(ugH1;bvsoIP0a81n_Sx1gP1xknBpBjKrGC)%psQI!Ccs)mNa&a4nOzNvAH|e zCaNLcsP9o+pQ|P_%fU@RDj3Pd-LmE5rlJ9( z5aDpl2AmY)Xz|!o*q#TigxCVdfALT?53@3we!3i!B4gY`l6w&D3;wJPMPj7h=e7@} zjk>Q2Wv-`Q8xo7<8(;&!=%n12+FH0x#YErwBO$P+uF&69xewYI9zRX9Hk>yk?b{2n z2i`Id0NDF<9P&Gcull#+(i$G+j1gT|XVKP7L$N()J^ccs!R=u-PO9KiEk4B;(cRw~ z3u78~Yhz5yiy<>t`Wx)F6@?v)7YPbH$p)9xKM~{ANcD|y$*|4L8Qiq+OFg-i>eT1t z1v_?OZwr%zM9RMXr1~w*x^^enWqp&as_6%;1bA!;5<3chiu3&`M-(&BO}f+~P%zpc zzP3iuCed5=>0abnL&+BbtQ;(%x{Pw`)RTjNutDYYSj{TKK7PKX5&H3vsARqhKB%QJ z{Zm2zH3GB$YmUve?*s<}utV-3?UHesvOET=V-j!vQ(;+5BqBM~S&TjYeNaZ#EfSb; z1|Z)zmYD#X`FVoQV@XT{FG1KT%O_fiWyE~V!^U?M^@(-Esa4$~Y_@-$bW(q*7#d+t zA9~c1k&7sHzXW+x`A)vUi~yT+ch^1I13snw`cEj^-OqJv&z}hHb2IwJr_|tA){C>j<*4Co?Y7d_#<%lsqsi`L+)4 z***ex_VZNHnF(6pEY1&BBX;f@>{t!$ys8auA8bA&J`}l~&8qR3x}6zW7`ksZf*P#)+intgrV8Iy=Voyzoz6ey?>k3Rtpeci zSrFlZE$gQV^Jr5s=K?&_3ZeE-T+`j(!oK>RXXipxaOhUw#k}XXEl9qmjt^iAL?jbD zvMvv$kG|mYrA8j)w}s)$h7{O!s*g&eIMpXawp>5xSC$VfP#;(9qXWpwsPbd43}8J- z#lK5XykbmV3RjngUy)mqmZEW%Qvt}bIMY`|>&e7Ac-DrFnL+^SMVXzU@4uKfr6pEz zmCeaUzF#x5R4p60<=Ye32~IJK;UkEul1E-G6x4uRjH) zkHX;oI^VtP`s6=?FRMd(8?(2*3-KU=wY8B6MoFRoO2Bg@N|q%N<38D6y=5E}9{f_<}W0^dkeN zUNp|0!vRhK9|-d-r7wKY#eYMbR>KdN_&^ozK0{z?`$9e9rYT%Zvd#)#QiHp2ZhQSn z*lUEcBdUa0m3tnexQ`g7?qyFya1F{no5@VQZ{tYp2PQ7S>DXNJB?n{<)DyE>8h2Yk zMtH9=aN49ujf>YyVk)yZt(b|M#SyY8+8*HRI%*V|?x_wPDGJ4Z|8bwtXgs&P9$0E? zdy;zWD4t{2MapeBr^ET>$FF~4t^UfBzQ6c0N1&ho)n<1hL8eZf;s8ZiKa}KU|Kb>f zz|VCHqVFqkgbPDC0)=ZLaFkK=Ki@;d{&WowNMjr=UIrB9Vq(66z0)|MLp5OvoS%>X zw z(e3OYyN`mWks67vNw8jtH;)x7pQ zjbQ{L2!CVkAY87!!>mA}IKZI8F|t0wO~K3fKtTQM4##i9EAU-J2_1)KQRXUx?9Z!w zlQN_7SbFYZXTCFgz%}BV(@>H+tnoF*-=~;_*=mXmB*^jq7Un+>F|iLaQfh}dh^xv^ zWlQldzNtr$+XkPb3mp$6K(CH|X!Z)0m32r^og*Oa$y1l*PucDk1k zJEPQSMii!kIA!|Ykzh$+y^j&)5m4K&w5M%sJ~@5=avkqUEy6hd^Ge*ALbYQ;>Heh# zckvD2!am0R_GFf{%reSGFl23(6G?j1Zx$lC#{O;k{8^zqS^0rAd?memTyO%J0%%eh zwayrWk$xE|hB22h>B}MW0~;|*n{4%(hX5HqQ71V)taTIM%F6LjPo{YA6_Ct-IOtfq zh2ixXMLh4fza(M>;Ncuwr)NAV&&J0+;g@k+Ak!)17TF-cj_L=-H^oB`tg^9lR1;h} z&QQ52-WO>JE}XKrjpor>-!i?B?b&rm6a?U~#Wq@7kp$Ogul*neD+p($lGpoS_14w^ zCiB#j3URjspu+GpRf%3!vElRlxC|4I=o=z(zzduK6v>$PdZ88pPNkAw|Gm{3c9S}s z55d;VEBM1Rc(g;`5btRp(51BxDx-uQx)M!e`sE*>Ybj$)VZAE0Fv zo;VgcXuAvfOr9xzODhpMjZW@q_uhe)UbnXm`N1D02KmfHvoZHie3`Og@lCn%>R6U+ ztI6z>4jFFRA)Cy52#R^JUcFB0dm!BykaAc*7{a28^YDh)n$A%(LXRd>+1rW3{UdhMwictiF&l(_~9e5o--!!)BqrV1q0i^`1;)tqyy6b(!r-x zC2u6W)NOtwq||N~WYqr*bG6i1l_(CMN{HcrhPhnNSIIE*Dt3b+UdB0+lQ>k_@RWOg zRYT%@_a~wJRxFN+xWB#+LgJCXSb7|$_9sQIu;jTSJm){=<8f3Dp8HC~Dbli(!?c2d zK%pAjC*Qgs-}V{P=M6^04O2QVxQ#+hqC?aolR}RGtzr%Ug^@4?S5)CS3HQ59>6K6a zhu&6$l1{OE#h=gNX=acf*d=_s81ZoC%?4vQfvJ&wQ8s;ro*V57&4midX!OKUzyV9_ zbm{W^%HO1thS#e1Y2-XxA$gkR08Hf|8}2+S2l=Ep z##OWBHXyNRIseIs#4kkRE^|Cf?aF<%C6l+7p=+i(--noAhdJ{x(NK;5%2X%pT7nWJ z(}e$WW1vimE_IR;4FeFZvIg43LKXap$fMieB z>ur=PNxYV%((uf6;LmjKOZji?BsmV3pE8!GYx9lOff*rgNKN3d@O-l)sYYL!^CH9p zUAvq_YgJo9@3M7>u^=jr5Yq{Wx!zSE|D>Eb^#Oq7fqFJ6W~=XvOM#cD*S&s_Uzf}{zUcSV;YKIQjGZ4jg zNe1pRa|&~oJ5f)}vV^iugvR~cQY-nugk?5pqQPaR)thqZAV$d(ap_aFB%Bsk zGx6A%M+Aq_)i4j$ASK|VFG+RjhHTv*u~yAuT%`vwe;NLjd)lXM#lO5RRR1OS|JgGn zPW?^-iW9tEP%!~^aWgWYa5v<`X<=e@o{JGFFgJ2OzvwKVAg_WFqUdGCt+CEYa#6di zg-$Av$=jcpj?YFoiH+rGTxg5eZoGduGqG1 z+ji11x^uGk8RLD&8T&)k52*Fjv(}pToYxfPcD?mt$_evG=RY6!K2!Ei{V@d7z`iA0 zF^nSoCNgjoCu7+-@)HP4$MFyy>84CVrL{s##?~TI{0AK3KA8n`+Q5s4G86aQ>`dM^T6jBdT1K(ind)5d$U0fE9it?%XIbks# znI*OzvD52~v_tEnBgvhhCK3?~j-c(sP6?&W>tACA0m*BQd{=i4>b5@SzvC@J;Ylc` zN=mcrGYx71qrOYwukZ=gtRw-809F#l7!gw4reRmFc=#V%JOO?7!0CLu6v-tl7BY?J z1n%CkL?fEt9p8I>*^Z7W=y6Wi!Cr0^Vp0zl_OB_{v3YcNJuiHg*sq@o6d*3i5&9xv zv0jL-S>8%fg5ukD2Uihk^~WN2MvENB^SgeFb&;U~5`@_izaj)5&|*?(uYW|{{37HK z^}uuJHm3?MPBdGzETgDCEK?y+|JD^ncBrYSe@6{sa4NICDw?~Vlsu`0CR=RNE6pZq z`anvX6c1a1EYy{Ojl(0_+`=}xWpOf>@SDMGDX6%DxS%fo8v>;Q$K?7w_{`lzEVMZ! z^+T30`i|Hr|drpW6J-RrQsS8AOKM+w(2;lh<^0` zI(3?qVC7;~5$sJFH+4lCl~8I#tXgmkXcffz9jiJwOOug9SU!`gcQIO*sruR;8 zWuKomnMfRUP$u&CyVB^x;qF3b@K(799a?I$VJlKvRW+1TLjt5>p=drrck7b2%lOgT z``m)JYL|uePw{@sX{=}H3TJIZj`|t6 zQs}*pd24LJo%W@r59%35Pjnysi)!}o`OKWxEmGw-_;8y($MS&Ha_RI@x3x4iOrW%C z(V^5ri6np|x-XVXIu2ArA^pR6Gr~uFXugkeDUg8-6)<@e|9kZ2-52 zV$Cg_?>i~ZxJhb5Z~^n0T93xtHpqGAo0dzD`&7Fd%2^#ja5!BnNq&g!KJ*< z4ShdNjc~{9xBiUP-tQd=r#u50A(%XU9{Kw`eP}?NGcNiM^x|Q@c*(h*YEtQzbweic zoLE9W{uhclhS(L}pGPr$2P~1OplrhihhNRv_mDL|+Yoa8{#zibcb4+g>hqsD^y9zt z)aW(@?0>9?Xek@m2nc}rGVS%`M$2te1SP9EqiQL{IC--maj2>avu8 zfPcj!E0x$ghw0A?8Q)XipJyM>?+Sub1LAhLO2InD%Q?nG+lAq*f7IONL{m%5ZYJmm`QT3_vZ#u zDjz7lp*ksPyOii0ns2GnnVcJJYGlr~HdeX!df4t!!$$mjFFcf!F}9e}Ex&V2XC8EE z^;z8gekC;+V0AsOScd^J@y%Q}AS=v^U*43kJ#Iq%-c#xI8<_H|$`7-eP|wL^){J?t z_Yq@U4EmaYd-t+JWuTb(2j0O2+K}oCl-Uc`Jc8r*I&KiF(ZcS#hMi^PqRA;1OALxs z3Y#3!-M=^FOs*Fx>(Bj`=l|@t|GUe6%1iVBSLJm%R6#_(NmB?nh*ro@VoJ_2y&$*F zP6Jkx?@%VaFjfJ#Mea?ST4u!*EPN&}s2|@5V2Q){)1MWsCTmh#A{aJxRg2}aO4DCo z&d)i%nDRC$G}M^*Yqyx^+3a=&L!jepd9a3l)tL$~Qs8)0s3v+~>MCVh-U4 z$Q;D>qTwt%tPk;;*&H~|i+!DEmuj)wr?*?t7@!p%nyrl^>bK}p_(KQ}78CXR83dV( zWj2D*+J~~Sm`$Nu&@jCWMQ@c|=waV=lk9)YgUlddHAFsYdDk_q?z*x=jChrOaJCbh zn|C0q8v~`B5{FaPa}rw}Bz6hCjCd#tfU>j74$KZI%k#-6m~IBI81&QX>d^5RW{j(D z%1_^AtAqofECk1mhy9&EIi$Hu%&X2nk2h>7^Jzunux9qj3N^z#mopi$JLhycOqW7@ zO!KCyluoQGB1U17;VAUPf|0+Z*3`F6Q>eg1DZ_rq%&)iH=H2CZR=e;b=0uc6*E013#@;u63FREE96HGT#w;Cl}Bb5o(kGd`g#ayprqfg3u>9M;LO z2^&c;N+=>2$qGOcyz;KS$6DQEx{&`k+T+M_R}zHz-#c!BoOiC{H`Fg*;QpK5n68HB zNQtsUfB^`9j+P77$`|Lny35eX9zrwdD?&wh)s|@n{jS*3!0z>r(W=1wzfXGDR#?==`kTQui12!$HI@g_t8v{0BJ+3WwQpA~tZ;u<@>!F^wKbInjQ$nk2hN20# zKhquWZRnw3q~!y$d|`Z5V1w(ytsJ&hEb-*$g=D$Rof}U_UeuxIgu;90kA4O7DBw(d z2;g_qeD&N{*ZfEuDEIN0s3ZSX>4jsEE6{1#AJeQIVM5_tuEa^yBEQVF%>hV1IwQO+ zM-)K}od~C>_LJel_-P3}hRNT*y4o|pnAY?K&F16l)EG&fdvXL)9oFw2>eV^-MrO|l zP)hcU`v-z+rVj#8;9pheC?*Km?dunmFP{JB#qsay{F7dTn!;F&fC2c-*NCD1ldp;J zX+~N2X)h16xT}|%Qq@)|AGf&oRI~ZC-|@xT$NSs*^_QvKlpYFZ{*+qc0zCgFv49abf*(FpEFws_8NpZjH9#A!kHk1oj^sI zy0ce_I_bdzhh<+J*8>Ia%sD(LuJakC;r^Wc`D1}|I0_1)V4%_Nvw%0(6z;GSY#rKj zS5aCR3krrHV1=Z%fN~?Kz|ldUS#8A9?97zd>TI2sVu2X|4sbJ-MTVU&ufv$x)w*ET z1+RaN@Q#W3X}ztKW39=uuE}~u&3MZ+`f5#%)3&_(h<(Xs-Ig`&s)3{UkEz{T9hqSR zcelWO$6+1Z(6Jgkck|uPn|c&il)v@e?RP4)Sw z8P3m*a&1ry6k6V?5bI`HsCo%j(sKc&SIzo&D@NNUh#lxzFBW8l$QyLrSK4aPL{1(z zutET*^hG()zy=H!%=^@R?8v4Hut`Q7=2)JEIMkP)PYSvLjtN{qJM3+6vMgMy(yC)8 zb>pdp&GLaV{kO2D(kIKL+b>rh$<{8K1*75|eAa0{P*kG3^zEpJ_4p1U+D>bWT%pK% zq4LT(gqQaq<0WS4G!9a5bVo$}%3sohjoSbdKCWEKtExuX&E&VbUq$r(VLe7u+bDY=$NBNtDAXq2a z1RKRzBS{dt=uEW(RS>;Wl9+^i=TEbuDJ<{kW$_zil1yYf%1f3+&vYMO{AVmZzRX&H zTe-?XfBz=gVPBzb0m>!W=fSR(p3(zi6%8hrq9%|u~kxAiA`FORhFPNbR;X^ zSw#=iRKLR=Mr-|rS*3W^qx*nJ3k zqF=Y9&q}WOij!NKbECkwn6|xj@a{VIlSrSA@Cy_qZ=3gwM`kHP#qbp^ExH?zJ#4|` zV?)7)H0~`;VvcG5SQ7m%WJRUAPH1TwfASb{FQg1x(c7%trDtS_pnhhX3h4;}G+^5o zM%@zh3b=yhv}~j23h?%zdC|`m%+?D{zr%>A4>b#}`yOJ6ecT}U11#Q!hcaw%(5mk8 zrh>IRsxo-^_L@=T$<>OFFsHQlK8A!&(7usN$S<9JH?bCn7+IEsQz$E){rb?3jiWHE zzW4p?C7kzX?IWzXAG(MRDPO<~;m5!4O3UaNgxAka%o9pVAE zFU6JS*BH9B2c+ZFW)x}0jx(zf}l$$@>Fgj#F#{^WzzMe zD`5xBmPF-m6_YuzULFo!B)MmE_3n^>A50t_u`28b;fC+2X$oOcWT5g%~uKQ-wQYh$&oN*XN zOud0z;qSj*Bi!BmNwLp!?d#{~zf)7}l&VU2_!NY1NIw8Y8W0{h%W;20Z2br0Q3d1n z$}|c^A_z4IrRsIdCpFJrKi0|`rUl+jedku+StDN8m z=R?C$u~y843Js%q1qxB1a+l6f1L{&QC7|b zLYtzFQ70->R?mnA<13xS3#FiWY2-Ci-iHdsP~QXeg13ri-q2ou2@$BA2t)Ig&xD}8 zh6)j=o^S>8gJu#5ICjAWggmxl^xmrons+GvtVHDe-dFQc8YvF)(Hfa9vFMF~eqkcg zNBnI{q>uD_om%Ub2@U5)7R}Z}bddZ#I(*VYZUhZY`-VoyPj#fa+~;as$WK$0r;F}@ zhngE8;tSD56!c)*-w}TKoF6WS)P<$5TKipq;s8U{Wp|&t?UoG9&__)a!B5K+BXv21 zK(Q958u|qe&HF|%Y4cSxX>)ItpUHv%6#Sk%X2(Zy;4H+?a3D(LEaWFSVqeBMP$no- zx)LK_(jx1I+aq7XNm;*syH9-;yGo>^>WB&0R&oT(fyyOdfTDtIB|pFj^5Odh>Bs*o zD;3V*3(MDwK5L;a^q%t@YwW*YKC0rPuc2Pdi9V9%Zu*|Q)Uf6o!~F4ZbnhFT5zm^)De ztX0MN6Iif@cF$V~bd}HA7FxReItymCa*4%eZNmeU&Eh@NL|qN&Te*aO!^RsTuyz6a zD411zZxpM>0C(S=X5}WYZ@=Zh?o@2(fcnUv%?N(_@>d7)nz2-^KoKc=U!JvYU>oW^ zK`h}}KMKD9I|{#;Ix3rR?xeDAU-CT=AZCy9mYvTXwMAdH<=;JXV##!(`|KUHC7T6? zSf(OyejZtzsWMjpc1FD5{?Oc<(=Z|9Gl$Irc`md$dv_$tVpL2lZaa)kYC~y=Ck?u5 zO=yiYCHE-uG#q3~K#3EMPMc(Q_r-EIx$i)mrZcXG=iy?~a59)16P3&1Y|2Cfm@XZG zt1X2ev$$P39Vl2$aHox#?-}hinT`*$l;Y-C7srPvL(R+8&7h-?4ap8i)_g=aV{#?e zGn>&T?LJ&doujDDjjh~{5V3fgZ@CRcnz$cM=&rM`D;7@3edbIw9ZFi+&HaWsnz~Z@o-?ruMZ=sqCd=IIYzZsX9Ja5e!s4QoequDdGio_YphhGgC|bG z6S1uzrLLfL#DY30LPWL-K#&lkH0tP0JbgI5Udz?6$A*q$2Tw=E&qITe6J6x=eLU)1 zUlZhHtEX@Hy@0$e(>IW;BmeY!9?sN7K1~)!O-O;egKcdQ=Hr3<;e2%CI1SV?i$-$$ zz8Sf;Ogw%YprV+^1pqjmSthvRAEEoXZRmGL60{b2LiGGNjK^aMIEF_&d&HpFIQ?Zu zu`u#B#^AFm_@_z|c9!|7Ce2{ZT~lA6HloJlS-)!kKltBj{mpe zGZ*HJroG1&+o)Os^GTU_UDmbB1JU`~;kEjddM=vT?8ofKapV(jmaL|DTHSG!V`lOG z33*alj@XVAXU?%+0YlSGV^S#1D};SA^`ObfVr0wZU&(bgfUz=jjl0yWUC2bP5p!pK z1MZ6IwK0>GZ=ll18V1MrR9yi(?x_;2?w+2cYwWME1RPhXtwb!7@wng74TIz~xSGv( zWSMm@UO36$@#uykcaE#$CXgtsJ>x6VeCfpGyCU)o6FRHwj$)^?S?l|k$c9WO4>XRy zWEhxtZt}Nl0D?Jri!c(ewSt#UgG`+edW%)QFckPmQZx`AJ4Ne+YgUS+0ieF8Xz&ZITzot{;-d}x3&Aq6=#L(j96T(UI2fH8rBpr$0R zJMc-G$3vYpng`H(xp8oFNuN}0zhMU%7@w4pU>&D1z_x2382lOsSqEU_zlDU5!Y0Xa zOP!OYGos?FWo32j;p1|h-RZcLn9R%VBZjlh9A;ypLN5hq8#wqJbEusK#_zrz>Ksr$ zN3i&-v0woGs9;SqzV~%nnAX{~EPJ92J+xYw!*LsIISZDgSv*I&giVU!-#(GiW2AN!T+)>0(1#!~kx5@qCP%ZN!N z{gsxfO`G3SE|=KCJ&N{|JHX3n;;5(TX454&R~w9BKCw|$iRQMR zRNpnX;*SQR*HbXnSKlo@iNWcaBsb6pN+Fwovdpm>czycDFYjwQW1a0+BPmNc$5nZN zxnL1levYn~ujAK}Ti95~_`i~_V<{?>W7jnQ2DwHb;|&iUNhob4HzjRx)0@)Jxu#40 zouU{#=w%BY+scV6)aILixA-IA&hpy6yg?Tgwhv@)09B~7Yq%Vz=cmmzt2c=(Z79(K z5U%Q}c$zBK5R)s_EnZp;boI@Dx+atT4pOg~f+Vc#Xt;WvJFR$6?)_pnX!eC|=1|_2 z7iMgGZ=<$=^9qC*gZ;h$y!Mwk$PkPLjPcrK^&VnuAP7XobyL>lsH)wraK$WXwEXgL z_ya39y&t^-n;kv1|FgpUhy3K8?y|!%04NwH;n|qwfcwtSZ%l5lW0MU+u1X ztvB@alWzZ2s-kjF;Bw7i$)aa6)7ib2@a#EkP8HbVx2qhXb>3GaL*3m&j>oATx;FbC zr!=%j=1i(j5a&9z%54DjQG-ofex=}4MS}ZfbO-$j3!m0Ul$ICj=qY0Y9ABmtfJTq7 z7hi$_Tf#SbGzT;6C@V67i|%*i>-h}^!l z*Ny7ETwX3jb>5Tc23b`zn6?Tt;s9w7f+-ZLsy*Z`Z5X$_F5}JtKR~ zujY9Dy15}5e46f_xinK7lYD!Vswx(Fdz3o=QC_>4(@GYPVO3ljLXHpw)qU(o8DnP! z7WU9n9G!WtZ@Nr;oPjXa6Z?cAzb_AQjKunJ`Frd!haVLS9PtC0Ivs!51G4(x${1tT zJ%-nhZtT*oV@}~%JnFt2V{?P@fDd2H9}R^QRL-?xNIj5bycHRqP1k}dHDc(q5pFzC z=`fF8K6eaHn;!GL{ks*7My!l7w;G$Icf3}}{qH4agx@RgP7nv8;pIsj*f&Uu;h%IJ zcgt=+DhF9`H1JRMS>9fv0r$04oe^za{pqau)j{4-+(5ei>%_d48w;}@MxmhHc`Uz} zA=sgt)o(-KVk-?_1O;=dyiPI(S*lO`zg2u2GP`ZrO<{c@ljR?c?IUHs_~C{RT=_eI zyF~(7o!0LSrED?d`RKQY>Y}5``RjpifdsMo7XCD2(72jbtRbnI7*K<7Y`DV&yNA_v zH2wP|rP~{$6ZWn#1rw}an6J}DxW)>epz51;`+YjWj&&E#Hr>^FAMfrGP=DY?X|_Fg zZ2Kl6fBgk=_eX#W0?*_kS+`3+y&8c73O&ZS9!*Km|9%7Z^@36y{y0CU_=5)v~#5sbdyZ zC=a>`9a0O92rWJR)vLa>XU~kIzIFZUhrd9t{?76eQVFy1+vF!}C;FKD6pTpt!jIGa zY2)s9=Cbe0*(cFa#{X#-2ku^14J`0VGn_TyHJ{V%%2SW3s6Jh z(L5uR%;h5i(bKVujtCsCKK@vE`g_UIxcqaq;W4hLHHG{9N3K4>rg-|sa^g_)u=T(} z-Q@%^P0W0)1aGm4^0m1V$njnNl%`|sIRHyQ8{0r8hu79KT6i0bOc#!nfhYga zjHxCo{6_MR5yeBoU+op{zN|}1kFgKnGyzO^730B04j2_^O5y&y`kHFol($pHGvG)v zw|bC(Y2&=-y2o-&7i!~2rp&R<)Sy%QL!C?tpFAjq*L{8qI=zV2p_|@k{(#=``|>?_ zHEJsgq;pwZl?}$^RcyN3D?074o2_TYt4ZF&T>F74Sbq_`Sv7|ZW9IZ~)d<95-3^CI zI97g^6`Ru-Mc^!%33BTk8T$`gSAgw{AiXuS$Wmud^FVmz&mhlA@(t*IE-%MN%psg& zF1AVg36ly<=c>$AT64QO&McI!>sbX0H@8OX>3wQ9o#{7+dIc&$;z^U4`94`A-$yyP&ik5MjXySI@@FffECF(Vw+j`K zKh{^>$6ZgIrSSt9wGcurrPts^oMHZbv&LWj&+}^V> z>m#uX#rJHXNB!CC(Xm<=m4I@6L~_I2dmdl{ghzYBE3CTT-~dv?91%VF-=Ftij1|QW zzU)ssyl~Zuq4KzX67TmAukm;0V1n+t4KH?)fKo0T>8~hakRhof^87^Nj(rU>p_u5S zi5}#mXQ^daou{Oq77&#FXfIUb7qu)DSy54?29;bP8BP~QCtS<| zcDj~)x1ToK)~#Sv7SztWuFnCA*W&?+g)aG-{l0fsZ(ZKk?N3t~9SncHj(xwtJ3!-(9K39nVc`O0JmSr=AoYkq*wTEpzwZU_%jqHs6_)1 zN<_3LMEa+sCs>H|{l<7EMykXQH*C_cu0t?1N=Sz|=t?|lCOFul4=6R#A_1uv_A+B) z6GFEyw?VXTmnyVDzR#~iw(qAywf}X4cwbP5W}hf>%mXuJ)B{X?+ygZ_a+r~P-w{(N za>N7K5{qF68;fC235#doOu9vPXuJg=HzcP+WSlb72`)Xf9?*odEX_%+g0o&0NvSA1 zn=q?4tn^?~9%TIctf9b4F@oa7qCwmd*Y32T?U&kROFq?XbOXM-im*N5W!U^$OP(cf zK}=bx%9EIvd1V5c(SIx# zao*Bx5Uc$Wq?N6`T=LODmk({JTca0MN8v@8AI%eb5+%{{vM`txVM7}v;Fl~$-M6Ji zpRE%z(dLfJl|i*5Y$i0^B$Bjida^H*hem7Ym8sqo*0u}`ia^_gPKZ6=ZeH+c(4AeM zD`Vd<#46o8;udmVs+S0eSW5=X;I64ybVKs#XR#$0MY?9NE2e~mC-%!)7Lg(Ql<%dN zl~0`_ca%FJWV;!sN<%i_cnQgZ=tO6{QOA$ebcd)FPj$JNL1#X`+-w7t{D#Sx&iDGR zam)8XDQ5x(Wf^%M#j-TY1p(fBgo>RzU!kT!}MMTs}

    nl>ZL>Y)roKisi+!=MMJE;s6@%rWU;AX z_a+c|fVK+Oy{c%3rgZu4QoMCEnXMp~dU9IMo0fk9g;gLg7w(PLG)TMYBsJ9wN&vPn zV@d;Rm;jVUhRZW-En%3r4gw+D_-EnQ;Vnw?WO7TO{jfmL8wQMM-ExWXAL3(EWkeib zC_oKQZuTMzIe!lbms@MVw~19aN7vH-)`Z#~MP?xH+TCb+l^1oDM+P=?fZw*k=MwB~ zQSW^s1?5Vg=VD@MOl&Hxi+mEamp`l4HZTv>G{ov0k~((zF<3;ev74RY>Ax5xHT>%a z+jaT^7Yilw#}gnDlVEoZus4&9Exm&|N}5I2r(Exip4A7ed7$3~o*rbJ!KR=pPuvjUZi?=!5>CnN%&kz+KHtFH~0WXKaFKgY- zgBu-LP!15o9xjut$Pr4FdAu0(yc=0Ss+Xl5u^F653q6)$1-cikIxiM*Qt=|bF$!p+ zR;6)-Me}JOO_fxw1|E%k_0p+jaWWuCOO1g%E5@T0;u;M;C<$lpjMY(%vm+XmdAiE% zZyz-Tk1B{O+u_tA2*ir6AUH|?X>-asWpgXjD-D>^zhtTTC z>A|?U>e>4>EZydhzFk~`+VY53qJey)h^(d2gF=Z=BUynnjD_R=&PR5%uTif!GVd;C zR&0zU`h?tcn6E8UFq1rwuRaA_QUo>0r?l~_j~4WySU`109qv*sD4cA1*ElV+&gYEU zu~GUABs-{s;_UwZ+t{)x6YcE#2^Z%^{?94&V(jx2%3%2E-OX)5fKGw=29FM~{*0%i z@(KGFjT18v5K=Da3_O|rvOp{yx1UUVJLF1Wtgombj$TD7kf2^CZnkc5BOXGOS5;<>(Fm5== zBtn)Rn0AS#q(W~wXfpD4EkkysiagZ3^a0G~ShehOsx;VWU}VR<%31&>MX}b`9qzd3 zNmBW+a=wbn5MXfp0bJADWl5-XgSGiEjt%rQkxuV^jM<;-@aiSuGXG~zj;G!0W zmLYzft;w|W0prH6E6b5zD61O?4fG7M0O8lg1$Zz1zQ2)G3@5UJ-`h{Ng711A43wl% z5IiamIqBI-hFpHoy;_?)=gg#S_NHBu|JD3s+dE+Y&&3FLW=lfv(BtW)9~o4tF_D`|D4b$e0Ezqp`tKPo}@;chB!IRHRG>0YtX>7-ra9 zCUdEM9@z6E%=f4%@|a=~jWB+rwlU>E-3Jd$HKW!U0NphYM&Dx6;Wl!=5{JgJze*H? z0+cNQ!4l-zwhLcr;g*qvog2Sb`QK6l%+l908TR~N)~Dr-Gl%%*m}Qb=kNF^y0Fo<@ z0l1I4QG=yFln#T@k^cN!R-VhSRC%q9cD5R(K!4y&2(gVa>-{ho@Iw9^wzcVdJN>s& zKF>Q(fahOjSmGj6<&|~0lJ8e%SrV5ejtoJ}u;MqMv};4x1=kqIc%)bjkLNaZDTc7@;%t+R=2yh+MU`D$u5CJJ=x5Is}_=32xWjIGH3kVR0%_M zdTG>7wg=eztOF(uAA;CV_$Mz&<&J=1k+8WsB_nW`b@~41X`kX3n+w+w7dod-G#PwM z!Q^adUO*chFoRtxwZ_nJ52g z$Hc2K{lFtB+o_ zPuAs=69(WUyI~`OjViyKNh-X;zGa|!<4Y4Ej^>9CpMuMj1j#I<{GBIc3xk|A1NG_9 z`KqybB6z(+D?fDD#sZ3G>#4nxY`gPZ6Oa$4m;I=0a+kVa8>Y7#{T3+X^A->L1I=4&p^IBeW%Uk3$tF+KtbdX#Y*jwPPI9X}WFZb=eX zPG;{AakaXC+R3jsJg>4Q^Z_5Y6u+=m6m=r7aPJji@E2mW43|wtobaB2P;MaqU0TD_v!#B{N&j|<&be{xe=nft@u4FL!mYRdg znpm6?BhVc!lBCk$8w<1?Axi*Bs;VQDC>r>LBoo7+DHXa(;!Rq(sg8Or@gST};hu9x zwGkQ@n*L5W^ml?Xjp2M~|8rQR$y>f%W>S<ZUvXpM`W#ET**fj; zjC^pC>`fN-_?!P`P%;iPvp;#j&xY~fz{k4dbT_(`j!k>x7UW;=*YtsAPSr;F<}}Z4 zA%McPjgJGmn~hVEZg&ibEkRYRNTBfzhG*1w;y;JfPC}T|Hsr>8_uh^p+p&2&xB%Veh);jUlc@)Fo^k|kk-PZkES*UeIs z$!1L?8~n2>{=6Zu)#~^ESptR2z*o?zn%*QCrh1?948{U1xBoO^g*#VVw<*WH*io-Sj2wu_&g;=K!RuB{f60rPaSf`jd z(ObC?BGEA^Q5GoSI{<3b)Ml;lwDPZaXk9_N9e< z?UF7Gxf%TzZy;nkBhc|Faoqn`;`rBWdJX{x1^9<#^r><*+m1VxhpV+xs$GD$HBlN# z%fQykOIK1c?T$`V8SCn}WppCmt4jt@#ff;b+=^pO&xkSE{@T27cbm$5nX;X#-rnx| zUb3T!d`QX)P6JI1T?Va&#;bNC(wiPo485crx(^D{rXqFZi=wnEjEl`Bq}g@DrkW@L zv;>AwqchPM>WO}prVd9=c2plEp=Bg>vcrUoAxdZHFw_`3Z}-WuW_RaaqcK)b73^fJ zFw&Sfhv-7G0G|I?1D$s?bLS0zA9vN-xC>TKv5C49YdeI{?$9PMoA*sv*Rc+Yrh=kA zsI6XwZ*MAzZ+<5ex$LRWoJyNM09tATqS1JZH`sp&)XF|(8H_}S*83cV87ej7&cZ&y z?%F!{lb=tNG+@w=rW0+t2~VTKNjr&bsgbH+?RBg-3dFGrK;Dt`zYF~S)DGyDol4*s z5M*YPTudySg;$iqKhXeH$LW1fR5gB%Ykl=Cw!T;5<=NGkgRA;pc_BYK%2DZgDm`eD2|pqyQl;X^nG59!3NSU77;@yon0-XHzt=E5+ExlIuMTuK~A zI*3$i`3>Jp(m6+Mzk!0ljneUqO)3?~rtzdi$f;9LX5#yAEJZn(Jc2I8A)^F? zbLujv-l^QUj_mTx0UQ6G9}GDImgzpXkjwujlK7zEa8d}oKRfb{)!+~T-lm@W19G_g z1Cp5{pFHc{+Cv%$sIXCRxo@r#)>_ckmPxd64dD%Ag7#jsZ=i;TnPfeMwQu0VM07>& z(${Zxc^sYcUOz3(3ma>S;%GCg|GXdSTW$&5nH^0=ZoiKIiP>~!_+`g-FnHeiT!hj6 zdyhl!T~1jxT`Rw|C|wH#D0P#iTV8;ctz1adS1xC5)Ka!)b@W94Jv)R&0wyBd-EtG4@0{|p?w9h@nmEDCJ(TproGHc_ z!!neHMDIVO25RtO-^k$E_q@64gj}Ds;Oj5mB5sX4@bquOA~3%T4eQ^C0|m~XasKA+ z8@?+HZ{L2!i`fBue(lY}ZqMFg`sLE>D7TgMnY*e+F`SO|D+GGGs76$uNEK`*rl_Ca zVClZx80#+Xv+JI=$g4ZwKomXzwxt0Lwst3qU)nE*bPZiO)X3dE3IN`?T&#tro=E$kM8<%wH!{*)5Ed3w46YUe=Dpj+XFap7KIk35fzWztZUn5L}wTW6`N4NQ6( znZ2l%oy)$yKfbe5=?D#eqCSj})P`vjLGnBxt&tilzmp{`uGsKH#;PdYZ^&R!DLr_} z6s@qN4ms9{?!8mXF@B8d$o!|gXgK&}7vKSka+D)f7fqwYhEtcHM;a{#G+=X)BO9Zq z`~v)F7bKdj5PI{`%*wlR;w`;uQZ3Z@i%r9(3(gMB;J)Sf}qwzQ%kKUr%H0*nGH>>T+@Qm# zqrtPBy#EfMDd4#on9ka=4`*S}i(1fnT`pB;$6JsIth+?)Iu;liLGW4574os@eTWZXQ<*}a0dc&cBC!`U2Xz;i~4!qp}UZH z5}-5gX|glR+w!&Zrb0!WBnxXIP38`|Hx_0>RwSJSY)IX3Co!4!f+;?}2<`$eNh_D` z_J4KKB{839J729o%CK62-Zzc&dbhepH1d3UP9pBbh4~It1|qT3QiYF*0aQGaP7@81G|BV}+-R9?#PHN=l*MR=%Gq^0 z`$N#MH^kg%MDt!_C1nV9|1LzUjFHD3*I33%5(qeP@YNX#R^*-nDA-7L}NMp4^!zMUW5seX@!egbu*~wN%TU&WTw?G z(9#i0tu&FSa1pqMrk3B{u$!ycPf^#*v2UQq%`AQdPsKYMQzT8=0Z4o~*Ngu$B0zaT zRohI=5P>EOT!bv0l29^q3-uzjC?w7>)o9;78BCe|YF?D~3VEh&JpzdA6L5jXzAkf9 zCNyl^372#2{HUul+qQBEqQc+v;C?hG8w>d1-G>aldPQb$IplQ!d@cSM6mp0ZiT(EZ zWLvw@OR`1t^>ywP0n8s|U4I+fk(PhJzS^oiF| zSN0i=MXnEXM&?UOXQ`6jwTppZ{`_za$uU#&4r{a^-z^4K1qtEJF#5_FaiuH3q%+@F z$9;g{3=pH9F}4Kb_2-+}y{Nssu0+l+?^T}Z3~klaF|S3& zgdgaI>1YRR5XF@{YHBgU6>wn*mM zrIZM65Grs;FGO*Ym|BMnWQAyh5-X@R>#T3d4c}yu0S}oSqooUQ-_Vu~b!bx$;l)PBZBVcy7s^T&f0)&&hxtz_@rR~$&Y%}&5O$0yv@8>=6J_jsbm;*!wbrMt zu(G(!}4E_u18pQj~_MZ7UpqOcvl3*IcvGSLXlF=JbMo8?G&!5hB?YxZz*! zGg}V$kfVini`>s=u^=7HlFfF!$iHq+NA2Zw1H|m_2}$3LrVPFR(l`Wp!oM*`KhAUJ zDwNF0P&?mUHutIC){a^7O6Ju-U?6uRa2#8c4nMVNqiWd``G6s`TpZf*pHJn6q0|@0 zZb`L1Bk*Uslczv!W6@&5Ap`7*08FSKDPL0v2>l*KR?*kM^HkB7`XXK@fL;kF^4a_{wkIKEC~< zQUq?ZmQ#Gj3JOU6qji6(6cQvkBWqJ;`z0!wY1YRP9PR7^}i ze5gu#V-a)AQUo!Y>nGh(a=WFF`H`dcGQtGLJc=v*PvTt&uCh9vr_;QzHeMdy#@)Zj z(pr1u=9TT{0n>kI3k!c5Eu^tCd7UV!S&9O8fzh;5+(Qde{j%TZTdew7p>+ylc5$;htH!&WfVz0itBO(`FcE!{+i(Uhibn(22*>M#)f8;JHEcI{-6Grrw@ z)_=ZML2l#L(ecK=fcZ5|wpWx590s3aT)i(^kwtyLxqwPyb7vC>;Ke7zDlyD0Y>z&TXDDOpuF|)l9A; z$4`TQ%nL5<5n_@t-}Cf&9^LPSd0%@G^SR>omP@CX75*rjo9Jd?$yx&kLRl|t%~VFE zdm;ULnD&`alp41CLx^;%=fwKqayXJ(c$BlHCZ*2jORBOp!Evw^z3|Iy6AYF)SM%3c&>IJ=+CNb&?fi1$22mBb2MQdnP z8~kjgR%sBEhYG~G3^Ju;U(#o3#K}T=PobD2o%X4rs64@k-a>2c+!|%6_DH}g5?5S)2*H%p~Uinc6y_F zIdy%*!~@l0$lr`N#0>XaI*<^GJatNw>c$YMe=d32*u%$^Kc=m?bot|iK!DUUSyHGF z=@}%=43u;dOLD|xi`d9r2ORbrn)gn1_Sx#-6fH^mi`_I#ElK?P03 zf4TmDscljtNkP2L5C!nqKwF#nVx{#JbunI5zCtt|UK%|J1tEm8Esj&{=0Yi{<~?I7 z%78BfJ)y`Hwt|X62tQd~E;(3u8ZHTH>_5fS=}Ih%4i6NES3v<5=u{2XG{xqJptJ#a z?mKb=Svz%-#23G1$!5Ry^N)H`8uj+G#+a6>r8}@?O*KIqjq)X11M>my4*d>0@8Rg0 z=c;7VIZTzACB~z*7iFS)QziZr;*g^=9}-gI3ddTM45~;5`qM+g;{{@vA}mp2*%jZ& z8Zz_l7MIi*^Lzk{$gHuk=uyALP+})YS~ysuQT$|B^#@U?yRi#bgjKCsby_v0do|T_9`+cp!O;=*d3@C4@3NY8KMqwO_;K8fY@|`kPpusRuK=#2WzpsBAOg9 z+hcTOQ{72}67Wg`SQL(#H~~XL2<;Yg^E5=DvB=%6N{9HPZAe)cYDMATnLG=`m!Nfq zJG^k0M+yfIf}rreuvQ)ftF6@RV&7V}qC+JL!Bt#)!C7BHk=xIWjIRIaNE!W2trSj@ zjL120K$%F3QiH5E@-=hd?K8rDmh4vl0T?-f$qkb~!=AG&|MYV%mT?WLJRuS06Ny&*s;0JzO0M zJG>X@9?14c9ucGStui~-9u+;yjyXLnuXl7lqC4vziaR`yr9JM~ZvL0MBA>f^c_92E z5nx|qJiE=+fCkI&VGlrR%J~2oQO;e>7~Mjc6ft%;Tz07F(o@lOIT<@OMUKgOfH&dV zqPr$}hvymoG~2m*9H?>5~SN#{P-=bF?7tl=DCNzA^h^?ue)Hdh{YR)@Hz}vu0<@Q|wC84QtJ| zQwMqyufkn&oyc*H?D|=GuTx`ZK87`U zCv~Zoh`D=A8Xc(TOGQZ4ng;<4r(TvQcFW~6$JpEMxPz%;B#;ntn=ognLy*>9MgFm= zS%j=ET*=N{u(2-3qqUMe^kV})5Lfox<;>5SP{StmbEz}a%*MMW=47;M+}(4_kLFX` zk|%izDm06;rD9}7IrHp3f zY<6at&!;4VT_vTs?k&k=C}>zW*#R0XHho61W~k_8R8wA=tp0-&g~9FJ70ohGxglyJQs;(_SZLvM1yBC0Th0@E6i?eBUkTv3cmU!n!aS640k&M-! zd%_UUsg@f-^uzos7&bxC@aq08JCDT4@yYp}KDNGKC-Q)m*i38Y5cCrDw_15%8=T=G z;t~zAT!ftU@J!`KGLt3j#+sgGd77Tag~TgrWP(k&Oo`xIHsFyZUI1~^eT&5 zz?BL{AX*UTNiMd6$~JZehfQ!_*S26~hvyoRS;tL5ZW_L+Sy9~RT{BqE5eaNlAS&5Q z44+O%zCFkKgsg-6wcDH`Z#V-u?C=NTK4rE)+RdBn@F)eTa(urC<%FKX><+8ap5}z# z6AH&=d%mhbQ{im>)-#3|#xU;QtGU}%lwfGM;iR;C=mZM4f{XX$^sXjF;!j#0SPz`BIZQpOH7g7Yrh49fbrc=;Uy%m z6|xErDtALm$R{qO9llsdAH$M}(M97AuE~Yf4wsz4Lna7ro!3MaQiagl0Oev_T?qQX zm-kba?@+&R;fJ}0!DE;uf(yo!_B}?fOT#{!(K1eGkL-ppanMEKaM%NeE2F<@HZ#R0 zL<3)1jg8@)0iG1nC|E2}$u#zuOXHIwj-(rkCq1^_p>qoC)Iec?^`Y4|MxKMa@;L?d zT&%<&b-!b_Fb>f*zI_z)rVZW;@9^JZN_%!<&w`Maw6YEw1HM@HX=5-Mex;4l*(#G6 ztEN6QYpDQd64tv&T!4yPktxl&7qcH-2QY3$1}HUTa4u$fHjZa{vMgGI0a8(bVzwM! zH_WLW92fs)iFqL|Jr8J!_4S-ZhHIel0^Ha5&Xr)bdo!yOt6tVyg&1W15_K(l>HAGl zca@kkZ=~5;uM;wq>^O{+KBLN`3RJ-#ow0lkJ^I=|51bJd%5^bdp42I4PGWBp1ZGHy zF#onnNoe*4h;4|3$3fnZQ`zCW8J*qIRPMQMMd43M!#n z_)g9%pPGlYabef|6XH$H2sMCo?C<;>3Y&56Q+USNj~~uYM?A*|6My#>(|Q6zq~NAm zd6%z*`;$T};To`Eu#Z?f78r9xQQ8wNgfw6P56Uh+al#F2WXY1delV86+d`ya5iqPPhpID~FnYk|T{wL9Oh(jQjb&WdwWc_jQQNpp^xHPH*r1GAS90X*t=s0K@ zS0m()YHiyydEJ5S*N_V+ru`Bh%r)B6h{aNE^xlwiz^0q-CLn*o9gYO5s`dW4s>z12 zaH*5Bb}&NqL~i3D3jxsnrm-gK`Hj1T@1k=A$mW!pK%qOwaJdTX)3{owr+&cVp60Ap zfovlY$Y#3i7WE<7`AJ+#)j+nLD6`Gh#`gJ%;>9_R&`CrK?Z~JN9clx9Iz#R>XbVew zI@#1{M8}c?aFM|76ipTK1E*bGwDshrsOOIuqO=kV`8$bQLf$^H`l0FpKb!^ZB)dq| zyFAU4?msa1X zgCipD7zc_Fl!4gzHC!IiWAl?Q7Ce2MoR%m#sI*R`Xpp&*3HWi`&z$iQvunv8;kQhf zyPMDP<3KU%3&ismIHO_9XrhrBQ zC;}KS=-Vjge2hyMEmj}35oE5z7?h^@InCI`(Y^pLr@-tHx-d4zp+#!c%ebkeoqh18 zSX@FJRc4TvnS$*Gl}0$(q~?o-n?|$BL*g{nrkhw2&#Me6a(;I3m}DpmB$av%2GKoe zV1fAk5ZT%DC8!g-nSh~&ZYymxDr81?u9Pl^b(t-OW-3Ul1v(PySaP+4AJAqNI;MGW z!p5{^%5FW>z9G?btqhH`$8GmgYNRX)q8UAwBdiMC4TBbzJmn6IvD{+lCv0L*d&V5W zRlosQYTRP&MpCsKmh&pP9D~b?9UEY%cVzhQe0-U|3O{o7a1+r6)({nO?(5&ScN&2P zH^qhTSWhtM|FDic`Mf1pg2GNc$ie@?fbri1O;D-uqHs{Dxo99Ft#SE2(3?b(BwZDe zABNx~00WD&VWneXWF+j8$27aysESvGr>sp3EW(i@y#{}!+{Y^wk8(smzTa}5+Gu0( z{qXPwR}>Q)ZjX_}`^HU+X9^A|fC`CIVYDO+W&*d2qDf=SHC!0rg@K76S7$UeLi0qi zPGhh0#|p_*tKPf@qPGM$#s*!UGfya<)}4wrlbD(iAZB_hZ4ef7_TaXxl|)O9@?+}Xk(+tJ<_Akrw`^b?79(37kRdC<{^AYL;cIU^%D z714UYoanOodFqtI616P4lQn=4gOTk#OyTcHzCUxfP>0D1+_HGIWv;jYthPh*aU>8W zJLNCpb3!M3u{`m%)+uE1AScYGUhs*$$F38L%bpt<*ZvsPs6~dm;2Dq7#t5@uQ0$&i)@yZP&KE|`mvcS zs`S(({pH6orT!f8F0v_zS@Q3?uH_$x_;8T(a{leSN?*|QS@bIaF_oJE2_{v~5rGUS z0xR=-X^C3>wJ#R!*Xcn^L(ntBD}{j|%0DWll~rMc z;Y#3lJnU9-*a3#BMA8DPBYA8ZOsgFc2%jb4s5v1bH@4iny%GzhrSDvWPX z5}qBh-Dq_DqG7Yq4~O&Mm_?`ttnMJ!q;F|ig)ab01b?2}Jj%9Hl@En(j-haCLUx+5 z2(=Q_8tCB2jL2fT6CONeyxsq&Mp`v>(LT?kmYgW)HJ%$BbuE9+~0?&a$bSz zHS#{F`>up0Ll=_C59i*t%@T-7yAEf3cnudHr|`f%&HnXF?|@~J;H{Kc)1V8OPSHyn znQvgzF->yJBs7Ki8qVz+&^Ov0l>4;3kBCaq@6v`$i)c?_t^e%XSIjdvHnS2|R3b(R zmobOM!^rC{QC)0@X~m5N%*=IUKH!uMsV9)35Xyvbj6vMChqPfu_U%np;h1ZT+KH-+ zx{nB-h|-A+pNmo;3amI1cbMF8B4>CS@iT(0all9|L1ZiYs#(6&_1&de50~(d!`-J< zgw`wkKi*Pk=+Rj$_$w^ULBW>S5uRM8K3&>8qQ&iAVK#RpXKwP%_(p?`pmM z(cXXsof`FjjT)zd_#ClQbLl}d@3so-W~hAdh@lH$35we4Yp&KykfF$$62nl&;KCg3 zS95dqb#9xu!{6hLC1L{G8Rq?l{lqxvgs|tRs|cv?Fyyos&v~1*M6eafIG3mIe0d)D zyFA}-Vg7X-G0}W(**ueHb z-+U(D3)|V`Y`>Dn-Tdt3Ko(Q#5kX92z&xdh1$H@z)c@i*kOwnD_4r8)^3S7W&+un6 zG+HC+UOBLeN9kT}eia8C7MP3N+#Py;B?p9g7<>b*UUyBUD@9(*M`fX}EvP4H35b)N zuG0==027Hw@3pQJj;vR?c_6bHDS)7ic5-W!L0h{@-KDX9_hxga&K)P)dW}c5q|{H9 z#?!2;bkWv#x#fH;#!gz&OyaV|yfuNZ+LN#FDsv=TacJ~#vbnu7`5v%Y;3~IFOIO2_ zGT3ms@+7@EFP(x!{;f1aeaEjkk%wk=^!eqqjNcj0C}HOoW9yM0t3OjA>dc7c}4jJ$^ee2*l zVx9VY8E^imrR7$V;}z+e#HQP|vIjAh0BP68{-?L@4M%`gLVGq1j?Vxo?zPzgau@A`3$78CI!j9CpR_n zf>qY8-bLrV6Vk#!?+&5}?)K4*#mWWXrh>1RGz>@&|?>=WTGtkHtTyU6^PM@H7Bwq=qeXV&xI@242DrQZg- zW8g`sbD$n`Z1$AF-^xGsj-j5N6x7Q6NvGdpofUEzwF5-W-ly3H^uO*vI>X0p!4^`5 zQYNnS+%Z1T+KP((5D{(0q;d^ASpu8-fGP@yl>?NJ3IkE>@eFFS24%$CKg>XMzp=Lr zB(_G}Q7&9Iy0VD&d*|%d?_^KjY#7?HOf@H4gjFbQBDY}&r{I~^dqzLaO?KG32BYCg zq#ILjK?RGPkYdQWY@0akB>qR*g|g!n%NorF`tCP1uY-?0#&QsaPds6eIv5|YTkTW(`bO>v^4 zf{WuuMn~XPy8bLyo}?@aE!nX;wbWX;WzqTdP~!VN&a<-6x38r5o;wAI4N|BQljbD(yY&<9K|kAHLbe9oG-s}7E~F&N+nUK-Eb@0m+_~= zSdO$M9LmUx{2ZHO7Tk{La*Lq$`hwEUeT{XXB!R{n6nFp3)J5X$A3Xn6Qxv>OD=-qZ zv!@<7&mZa=ji zN!SV=j1eAv15PxzkW3Y+tX|$MmB{DHGZNYRuVVK<>NzMg`X>J#dqrtT!dwL5jP`%w zjQ>-(XbOZ!WEHlzcloD#!EYB8Gy-FTCxKrS&p`qa#NT5s=QHuliVfk+`P)`e;dc z8%?;Yr_MN{sEpdA|HNV#(fi6l^mGq{kd!Sd%0-5~k-)5&LN00=93y9I9*3)PrzH;9vjg(%g_ zJhlhanhP^(SHtc!RwDzxT%W}vb4Vc#T=aGPXkoE8Kl=Uh1}^x;8wwk$8DN96!c=Qb z9GD+YyF{z1SYP`!F?Y!-Ryfsb0sl+HOTJ8;ZIk`i80`Wo-@;~z!9YH@;t93Qdtd}G zcg2~HzLlGT@-{`@ZfBe6vo#@>+8OQB_gwD109|pG&{3RU9^Ac|kMS<6al=B{Q2rdt z)ON~ZiV8RxjH6}4Ir9mDNlEdFK0{|+DAZEAY*9ft58Ff*yumT`c-tCo0?^fk1dL@z zz(?3kv;5I?h!UD7aVDx2j-$^HblmNeDgQ-)H6t$m}$~yXe6ZZXV|D(A>lRF;^d8>X&$O_4!G9Uqw`)={4DLd z?jmd4gbswBE;{@DA^gFii%Js}aiH_3Q&;JbKfV~IL$IVClVOd(TWuCienWDdPylRP)&QmU6W)a zB6F3CnH&JV;SL#Z_E`BK!>g5_$lO!u6gusT=z_pGq!fOU%%)aDOB3y}Jgao|;Fms^ zfps%utFv3T5lky=zY{2Wu2+ zE*M?iixVe_kA{Z9K$HhoI;od;uczC36;>D*M^<_+E=X2QuBQbOx!4>Ro;l3HPkAo& z+@ru00-i8c;TWloyQI=J$yj`%Hm%3+vhku6Z)*}d7P%F#@MWo&c_|e4RG*Wob6Dk& zsv49I%x`>p5&7K(NVcq*E}>1O;^Z+Rw&OuN_Y8j|4Dpa=kw%$dtox3?(puxvzY88g zhfhM)Z^I0Vn82Wfs!kseH41}oemSm{6{RE6{b!nBZ?PN49rS1z|2xf4@Pgw56G@Hs zg3FN2Wk~xz^#x3v(8SyTG67dG8&@&&t7LWslE0YPSGM_{wOjXSBZq)KN~k< zAz;AqoHm)p8*kMbqC1QR4Y{mz04-d@n z^|iqnfJO0nK6LwF6d2coTaox;JK7NDRnyXWtzVIfoPM|=bIU@NnS>F{bRCM^U%2Cm zi8p)-o+9#=9}#A4NCUgaA3BYNFXUk!nAMW_(u2n?hN;QAs)hizf4nuk{4h%e1q>-4 z$TNcdc#{AX-a@6O#G%JPhE*zPEc8vm(POQH(E~iyT<*&I=891GCB91z_${H zO>_1@Pek)@hVa-KGA8Wc%h7R*!ciOvki5 z|5(TKLTEtx*)NDIOfqUPt11;u@Rm2YqN(9~^R#moRXz71P2Ryp|Cin6*!*#*Ri|F9 zlCIyt=Y^4Dtpc{8pEza}S*m<^s_jam(R| z{h7PCz{cP4x}Z8Jp8OBJQ*H#4+!}JxdK_ZJS#pu1I-gS}3*8CnghazWB>$F8%cagM zt<3m5kM}>ALtok|cv?wsiKz+Jgm$wDXq30TIHVcaf-0Sqd#Q}X- z9IxOp`1=GyFHLVt)R4AzubRZ%iI0NzxB2l#xB}K~0F_t7_0?Pm>Yf%^R{p&02H_=L zd(}Klp(KMQ1t!lxC5WXTA&ob%-8w7(5eY+?b3pQl(pQu+1bLL!atCLWY~tajau&Tv zRBAeHiHY?O>i*FUmeWrc>P&+@%qREm{l&sP4DGfW6D@g8EP=nPBXAqxU?iZhHS({M zmD-yQn$Z!lckui#@EEPS{g*<}JNxQE!lZWYt85db^a5F)jZhUGVO3E6te|QxYvYiq*mEu^YhL`EHd>-Y*XAUM$B@Nz)1W=^yx}6k@DIvR*k1rhxeKmjaa$zTNL|@bMEJ1&~mAbOR zsm6S?tpH%Mc*!K4!iX!dd?c21@c|jII%n0JupbjR8VnZS_~~MCzGNBvuC|7~K-8XS zZQ)PV=#f1BQy=ZW7-7qm(p^m0<>jX6e37Tfs9B3!ho?H;x1PLu65Umm(wM2d>)3gg zVY+-bnfc9|*w&_^wN9foOFj3z)n5atUKMh*ll!_GF+ka|L%kt~a)t)4QV|%8wtU@o zxKJj1ot7};j|U6P(D*8RX`|OGdfa3j2P6I*m5T4){iKY*cR+>tD;vJ^gt|saZBZgL zMMA9%)u+^;KC}?;+PK$C^ib}>XYKAW{OVUi?}yWAej4UA-^D^~kc4Qa0}tXLiP0s+ z^v&Ty`$|c-WGdMt-ZB?VrVDVy-RrsvUuMB)og`B${XTO;JXb9_AtP9xNn6(1M8hG{ zj->D~wkem*ahJ7&)Z&9ShifCZ37v>9(UM4OjQ#q;In$c(4n{@k6|A^xV1Py^Vkebx z{Fro~=tvRz&nQ+DA*NcNDGuhUy%3ct3tXnXQEB61&y`B;2}`M!OtU;qO{y#zVS8GP|n+*<hzlhW$u75#2W#Mc2e17?`~0 zzdYj(Z0C?{-v8F`!X7GQEd5I&$O6RQ@gJGqRSLX3P+3<72gKHqm?m3aO(3=SY-`<; zU)Nfw_?!zZ7RNu}7pHu(f5sgjKWUSeMYpK0CPY(3$p2Ch;hy-t=qrCKBZ$ky`{GC2 z&*S^c9P3Z6m5!ATy!#$gbz|27{retT;$VsJH<-?f6Q73%C^3h0Cvx|Ukkl|_Job#R z{s5ltfYCJo>)+AF*Se4KGZOE^JDMSG-o}W(kP{U}`?eKaEz~N154^-fEFtWzJi{fg zfX@=`Eg%x(sjht`DN5cnrM$S+7r);V8D(t2_?^Nn9X)cA6C_o*`pb$@mNhAbV9t$j z5(7zepc*j3!a2#wmbamSl}E^(brR%HFqR@o0+cCe?%r^(d4ek0+zq}g2U4m^EzISju!SXg|9fgRMR%5&MoglN|_7}{c zy>zOBpW!XX2Cko3bXtLCA)t|J%AkC^TYC72lBa1BHpwcLG6!y#L2=6P9{#Ts(s84! ztQ_=`|N2*b;r~$IPm>^Ef@oMkYRkhP<~mE!^)f$*QIH%1&(FyB=;9xR2d<39kOx{; zVwk1Prsv$kwx#po!pZ$U2_#)}>2B7R$=n0!k6S*t6ciqQKRo1oV!xIZlIaD9%z(kC zUQxc1?A<5LCcS12Mzxb{qcT+*qoa~hj!?Ff(UKPtB9hY*8l+ZN1#S$7L?k5DCs9*V zw@z3o|5Cs29%|f}p*=71|3pQl+oKee%DyW6z z45fm@#bp}VZT|e1Z`=&x4msuLY1HA6M;Yc&x zx)0B*McNe4=1F2WI?%oJ@IKhugDwz$2Ahl9$bdWBw4F>_zH1fey0S&bR2ZyOE;`=^PlWjb>XEF&;HlHF3avRa9x+d6til|3c^N{)*v2Fm2d z-W>ClkORwmo0H|IiT3Nq%S1JFq^`J6tM^%vo^Qa|=RcjLucRg4rZmgJtttyCu zZ{Ct<=9p>O+yM2CMvjUscRM>kVA?UCZqc4Fxy9prG}UHqHucX4`w%$6I6p|e;!oX? zu`jj?v`*_$pA8y7qk^(cQJk&^*Y1nvz}=?@N17h$3y2*}gDYU`Xw?MfQ3g&H$+eWI z^oiarH-AF7EsKt&U5B~H(itv-5zgSy%870_O{EP+9C-X)q3uV1y?uk~!&=O)Sp4qI zv+!v)?!nmTb#@_llLFQlPcaRa8BbZt&!=HcvoFNzK?{*S@U$c5t)4%KLwzqmehBz|FMOV4BX%m73qC9SK9h0jW5n<}qc zggy?f(fmLKFL+{yJC}^aVb7GkFuAEio=TSDAG!GBr=O)*FlszAOHMQuOui>!l0{Vy z*-7K;Y7SekviW3~G>V%f)6}AwkJS&C9;~Y6RIzJ#4AA@b8-Vuw<~)&!EZs@zmE<3^ zM||{g<}LekEJ;IKfbkF7+12Oy?fxZ1y;d$!G9S($0=?YlCjnabEErb5zlW+6TKa#r zgJQ@3L%dsd1dbfYM9-+J%w*Tk2-x@ogMg<^n2_)pFJ0gkGUUsep;2DFMOb}&jjPUyPmj-`@cr#)Q}qr8YKOE&KwSO zF0Y3jjKFF^z4&q2toGKCGqebfSMuFSlGD4(xZHMSF|J-fC^bVvC3WKcfQik%ImN-s z?mJe&DXAMZko*U6>54;8y!RMFC@nhM#oV%l6W$=L^h4%Km9UnWOkoSIZ=d|yvZ-st zIvSh%6>e#;!MAX#k%2TD1@J8$Pv;Wsl9y!HQs>Qx`_v!G81DjAJ(txT_OY8B3jNO{ zmEU*q@7H%99bf+TMs{!OH&c)|O8z^N|DP!)&?Y-}svRx@JTR_Tx?d16eo&soPx-9(gGCJCv_hnXfS@xMtvmm%(Vu`F zef{s0wpO4978PW^zqo|{1K2QT!{MgN;3H83eU)@&Kw83l#)rR}i?vouut@VDZTTsL zAo-*SQ28F&f5x!YPmOTXEY^&rM&Rck`>DDtM&w%HPyKMdINaEsT7UTi`yFP9mR4J{ zzShQYu&+Eg04_udhl!JY$dG4*_|O1M1^#A)F)Dd1Riu)|CdrC!Yu~PCG1+^sDahUp zINnVcqEq%hFEGQ6sG~%rbO&BmxN01xA;7R7+D?R-i5;5+?A*Nb?7^C2D;scJxEJ;4 z?fi~?=8PBI#hC!k)XS5#lcH?mZo=8`sQi&=>5#0~R2J5upRa4T9FmV_lLDc*MX z!dn-`X&ZO;Thu~%pTV66Vt68X|Bc)-kfpk$$q~WzXF;;!5$giqDk4$;l)$1 zYnX{?6Uz9Kl}tg7fszp`dH-QC86RPI|M!7TIP0i32YKUvrJjNRJ(l;sy^l|yKtS}9 zyn?Efh*-3$jPn{JV%Kktv)ehwec^N!0t!Re%9uJGDe}VwYQlhT!};}J`@1j49OvWQ zin(^;J1Z`mkjvZGm(hzu%eTI}RfkOArr%9@A^&_E)oA$%Uro=jEw=l_&%3SnnKtSJ z2v&Bzbw(^X;oH=!VC@&XUTC+@)gn72o}60sm`scD41VUq>@Uzv_Ou`F`U&!Oel~^g>zqOk<{n+R<2S zc?K&6$8c_<_|TLZfGuFEW}IjBQ4-{y2Q1^KU|}f!ZJ8}0`~cPKznznRJmT*|i2y~S zslBaU?O$(fw; zfMa8~+01hfalf@>7kkA)apt>F;m0w7>&2uS=TrB){VjC#z=JnNZkyfh-HjO=1*ptP zpsnq&9tVL-XXna!Th8R_%U$whB2NzeNzBwOP3)xOnxJh_`sHtt9?kX3zNI zVA0s~LT>&I|I^Gj`|~W@b`224MB8UFv*&bas8;$y&6^-HMqPh-tQqAJVd}^GXzI#t zKR8x}tCr9AMRBrJhzVC>@s1mTgusNObQUzQ>HSJs5B^vKZL5*sA)2Ng?DfVlue0tJ zB&I;fM8}94mob)V4ZiKD-@Fc!K@c#I$?3zU;$)~rv_Zl@O8IxGrzrJnA*RJ*qIyzj zmvfcVO%*13lJZcwCs)Any`)-A>frN3(LN7gf|vKml*6F^sWs5YlrDBc%L7`6LE zld&5jA-Cr`7w_zCaC+L|g2%{=q6=ZQs~xIZR?AEp(2FAPK@1)s*91I6|9}RY&Q+x) z{<_#mAr1|I81I!S^B{nmW=pK6WyN@cR(6a&QX(0wlFG>&2~^AEL~%kB6oK*`K+-7z zbjv_|t!^wN0u+(O-p#Wg%o_?@-3*f)*koUGjT@tv6X)tUzpf92JD_2EC`7V|Pt~{J zlKYoCc>2BO`|0(V3nr2w)Q6g7*6#INo{&V&6J0R*>;DD%8J%jY2|!64;lGnO(8EpH zLV$pP3ubYfRG_dT^~7Ey@AL6mMtD@9 zuPTV_Q3;VRRib6Hnpb@bT~suv`YbwhgU98HO3yl81=b|iY8S`gZ60t7~H2Za< ztkQfbh=3(jM2!f;@Axz@oGu|l0X$W zD$zOVSg%&7iT0R+t8V%#-H8XjlJ~DbK216kBRLvPU1wTk=b4T&KqpuaI+7rT_3iSS z(L#xMPhAgKRV846&KHD;y~yCAYd88zKv-#IvPQ)PeV|>ctkR~FT&hO-kvf+?s3STM zbG`&45zAByjcc{VmdQfP~^w8(|y&;y=FM`H++=!TZ~nIP1n%esuNR_ z9-DDgCz#7Wn@d7(+C^3X6pp^-THH{#$n?21yO;eIt^CRrc>GQ^Ey+!#Fy$%1S7A61 zNa=GfN(STB=VNl@D?n@_4wV9^6k{%7qo{k=B`EGw(gts1{oksDD=$8|g+u}6pWU}m-&NB+5ZHnVGDxq#cggi^zk4QTK;g%e+hlHXU zAv``6i3gdPB3iJuuH4`3H0@Y^MAukr!^W-H7{#*@08`nU41?Fdg?Lp+cK8xO*sA5< z>5Ll&0yn7g@sHkpd+}sFLVdJ-b8T?hIjOL|N1kv%IG-A!Dr>j|#a#%;a<)#Wk{?>) zJ&`T4z<1EffaM{7!~2{a+F zNJ!e3Gzr8Ve06Xe13JMV2{5A|K=eseEO8c)X0Ue2<}T(*Bb%~1HUZ(ZZe*jPd*0W) zh(G^Ur@^&n#`ayt@=hE}gLgDaxiM9oqJQIBH9X;sJ=KbkHoS5=KhdT$1GIhXJ*oIR zB{G;V+ZSn?Vhym!$ASnL=PMh0@kxVuDLd&woVwY0kh5c{V~C6>0)YaGGRnp}%#K*YIyk;*zcnKp+}BneH}X7}E#hw7Z-WN`r|=Vbpq@t21o#@= zTw`Qt7Dnq0Qbwx^ey3=#o~2u-sRsQ3tkeQSI34WIU_CGaAPkZS`u+nrO8h~P8KJPiP1b4M9|%n5RU3ZK36Mi!f0WLDuR9#-%|S>DZr)7N4i#UKbr-MZ;pq2Ey^A4qM^7>l}Ycgsmt(S`wkix~-!Of1(|?Pb<9oxSr3#{ehUsk;|I&Fx&6E>J7#i?< zqzmJNutWo#FA|5%sHhbVR@m%mgQh3qY(N~Z=n%bu9w4V>uzOg#8yeNR23$fWe$us6ig5ZpCDxWZS$VF6qA1jQAcdgvmW!?26jnn4Ca zd>Wy3=O!~3xG^5NXZqL!T@Fsa;>vxe$(yg2i|XBj;we@zKTrZjW>G248fu9BQaRo5 z^}I_)F<%L_gYv~^0bisG3%ia8F6{4r4HVi#t<6q-`)IOskvqYr= z6xlAFj7fIycmYQVN1UobdrW9kX;LK>OB%`<#J_|cH6YPt)oLoH#F18s8UWTUD+Ai^ z3#ye=?DAcCodvHctBPux*0_^fv@XD70xY9-&q-n84Ok{;anXLPs3=~;Y6PqMX=tnh? zHP*&Z%js9j#th{=X7R!eoE+!rJI8AKOTRa$+8EQA7TinDEoWNC1+D<(h_u_DtG<3p za^9%k2d2RKSdP+X&4G?s>|!i&>bnGk?M|z+zoh-w~9I+gaI__v;Rw7ORfu0Mo z@v@#T3{xc>hMR7_Y|Wd_wn)h8(5^$q^pX`7;0|j$tciKd*ND;%z%Sl%7H>l7o)rIc z)si%kfp{)xg0BHw#8M!hU|!RrcgNK>Wx(m%VFHt*TQS4w@qp)BjI@dqb|s4}pJTt` z(*scQK;fZf1Z~mpqm~ARtg*elrO^X{{D{Sl|7Er*;VaK;^* z4Bj!c@ZBnJhN2{+N`Uh0!bkH3-Vgi|YJXeE3G#>%oeDey)m#w^+M1MB@jktZU}rLi zQum|8&)*5G=Kb#h_XyL{_k55G{~rVHp!osp)Eyo;yuV_bAY$;ex~)z6n)c=YN8CF< z=h=pTgKgB{j_t;_-PpEmH|E{wj&0j$Y&3S#*tU(PO|o~N=iS+Nzt7Ch?99&W57$3% zoyU0|`d|qj#u`)ar3zMs8g@{3;YnW4T(&*lmF)EUEH*UpfF&@ATUsos9ofabJ3F0l zYWV03{~P5Z{e-SoZLO8@D7YEUKm-*;6A<7R))J~IyVph~1YycCtDQzGD6YfT6z?3B ze&Q1?6SXSZY&BXgv*ZV&ykro237lkR4>6nIY&}30C$E@}T=^SH=cECbovWIz*SRPS*dJ{mR zj!>ki}$wj$44&^0G&W-^Np2F`|cHl0PQwY+7deugw|2F67 zB|neMX50fAwGo>6SP3;9sv?q)jL89%OPQ}wb>g8^r0$i`bZrjHTbi0lhDi>Fm2%bF zgbs=P@$w-8yyAP(RKg+i-9NQPjOO=WYT%y_^`EEefA4%rQW;wi5mP^_f+Zn3^O0eJ z>h`MWnqcpqZ+co-LCSW8SWWWwXt27CYWQUgC|1^Y1-G-HZU;`y&MB~Y&x)Is|Ly1a{{6DAS_Dl%5~ofsD8zD-V^7I&I0^tFAQ6vL%I0{u1MgQ? zwXssUXxbt*X{9laH6od(VLzPSB1-=}^9y({3hlyMv+F{5*IF_#EaO|Shd@tvlO6-{ zl{X@7LAfs>hPCFbH9XdJy)P<;qw4q>hWPXuiRxygli zf;T>k)o>y*$7(-E&BdEu^T-`+ZT{q}CihZ{+<%~uWY4QB(dMnF^(LgSK?zDHBV6P7 ziDNf=0xP}(|?#N}M(u}LRL z79^ykDdl{Zl(Q83&Y)pVkkw#^=^vz&edc@5)XpKC*HW1UKgT=(KZ{zNXMnkxG1lvJ zn~aCcNa$fBy()c+dv$6YFi>!|9eM}oYyE*O**X+UOtMrtf3Y}#X1J#N(qIiz6 zIltJlkW`#hh_&t$nNXsE*=CBql{vjL<8WHPDoYtap}p+PeRYRy&e5!FM2mc7Ut$RZ z&;j|m&lZJ$Q&j%9AuPX0{M9K;qL044OLDkPF2>o=mKz>g8Z7dp7^P_Eo&CyP(2 zzqyr9RLhCTa_o2b{c@(>Vq{SXKDUP=w7m^;ryo0#+{6hIRjkHQ;TB_h3V$Wy;-xSs zeFvz&4~8G%Y)TgK*N1ZDSWvn<)3UWi*2%eJm$LbA|2__*H=M=m-Ti0E=%DFHk_PU) zr~lh||1+c2%R@p2+Jb?D7y<|(jVvm#$k1X?w2eEc5fQPZW1pWX`l2u~9%S(#s}d>Y zGMG=lqZzuH4H0k0e(mg-WHvqdX?D2Y;W6D7>)HZ*dwW653u|J(EUL>I+F-w=Y5l$( z5E%Mb%2wr?JIDcGnYUZ<4fSIoL=7B{YQB+_IE<#64w&iM%4nq551@ozpI7?P34`te zfyGWAVXpim;Tu{BdzDdyF~4E0;5==-HeBGy`t@D~{wgZ+pY@uHL5FbTZ6S%3Cefp~ zzU*IpjG`=#T`jTpa}BSM4FnZ}od-G? zR!$nZdXC1c* z{!u2+Ak+yz?o^h{y}?XH?7E|Z=^?W{ra8IC<4shE&G(R!i=zG1XeUHm;AW14L`pq znYkFP+<};3AdR_)2c8-w!riUqRh{2xp9;2vp22A1IeXY$wAQ1|#LUFR-w&^EAM-*4 zAUPm)VVV)oah5saMK;CoY_VjhwU&dWmCCtb;wAk~0iuJDGn7(G_?l3X550tbMKOwl z)oiX;X{Er`=ywd^?!8}nPHr-BXY`43Osy7B8j7qH%cFlj2Ht){a3@mKqhu;SJA-i{ z7??Hn%fZmb1HGy3iB5F+1fKRSrAaIJy>+%a*M2sq$H-+aJ(G}8QG77np?K7VYAPL3 zkHLo;I|1#407NJk?va5d`coX~@;~)P*VLePDY0za2Yg?rl}$z+PU&Vt+sU0m`LVu}s(UyFSopAp+uA{Hx# zFlhhjI?>OLgXF*o{x9wPe-z2!Z39&*iZCiMc-x@M1o>Cb`nX<3_o+h+PZ_OpT8y+W z(Uul1w!xNtPR=spq-uKwc{e<6TYgtEydec;fZx}0%A?z5+{>%W=YvvjQm8R98IgoQ zlvmm{^GQ(e(@f5nPwg($q^bSf-ngrWW1)Zs7hO01F~~tThGoPv(y!wOTNwQl9>xnm z?Q+{(5$iztaxq$7+Ym z67#QJ5JBBfSiUl~$2`tYbb`Jl-cQtsA=ak|MdgJ=r$GGD5%<*`F z&&U2hMcBVYJ>*>^OrYAvU%SM22I*yAI|#U7_e_5zTBNMj!4;y&E6)B<@RZqCP-Hs)nkL>V|rT3LLm*6^${D+JWzHqHe$T z#`xHYBaZT29JLPtpbgS249i0i$LI%;DX-q3^;3)H7-#L+05&|u`vf{iopk#MHdB~m zJJ4iWr<9VwMyBMQ!%7yq>Gqg8>YmMj5MzzL(6l5aQxilk93rWo%)QJMpW_G_;E$(c zrUW%w=0!v~fQty`mBsjH=n&qq6CexB6fDb8cBv9~dHb@Ej1?;@4q2{fk(pVY3iD>l zEKg)YJyhm6h8!eac6xh#)6#qG0yRIm00N)VWXS9~pO!t-%iqh>2G2dub19fe$z$Hu z42=^UnbQ>7AigMn+GscL*IW_EmE<{NO8JKTav0PNV2i70ivt3zLb@$pxYqhA>gVZ- zX63as*68+b2UQe!kHK)JJWCC(Mz0R-1s%+$)jR>39t6r9CX+_H)#?n+!aJ-0sF*5; zvJ1qg<&g^WcHNT?P39+VO%@DgSms8suuE#DFKoK@Km9m}*39@;){DO@d-~ZPd5YGP zsn0�}Hk0nq`~SD-<)D>lSl1YpgS~_R_OA=kl?C{rR28$GN~TnZ}x#Srknc*m#1q z>mb`A)f?QB6P%xdRk(@%EsU8wqqtRFsZ!RN&S@y3 zh0%+w2@wxnO695*?&Dk|UXgMhOA_C}uTu7WAhB_asqdGX`CwkX9^t;}hFPJH&n-G>d(L17JV6=!Vvti^O`T(;g8BlAl z2dPK?5_Z@dT~fct|A= z(&%ZVi#1P_(PZ=#Mx0~fb>1*%$#@Wm^Ig@p&=r#6Y4hI%?$Vbo%RFtV zE5gNF=D!IaLF-qw&^)7d64D)ZtH$O$7qz3VX)CgZUjUaRILqRWIk%&Jk>mQyqTJhf zqP(+`w9?sJ|zsaukyT+Oj&4!*Qd7~103buz?bB_UEOVM8d=?1)A( zQh&HR2s3&w9N0D}j=v{B14ve=Df-MmOik`fB*NcTC)$>#By?qcil2fiZZfg(UY zc$c=!9Q?D_TKxr}1khI`T#TjFR9e5Wi+piSj`hfGp=G?Yn^>f_b{JcleFE;g?yT7E zv#4uOU81gxo!{|de66LO2*(|#N=Olm`f`U*e>E6 z7L1-bOeYuEFXo0X_>lyUUYHv%Nk{wFK?4T4?cgZd=!Aw|uuPOf3_f@Cd0Rh04g#~c z^)6XOUr$YNIcvJDp^#)uYvLYJWjn(h!X&72y`>9rvVGe*r1zZ==7MoB%QUlD>7N>ru;D}_67~0<0($Z*;z%)k!?ufT9cVtNt{CcLf`K;x>X$rtC#y0HOpDNaOM6fY4ELl5iF+?bMm|2hLi zqK8_-9|bU&bw*e1lRHZbC^yAga+}C6F(tY>Yo+MM8dxIiEZrkp|2czVjIQZc)o&!F zS%L?^rd6#(u98eS(b89e#qv69U<7xwSDEHK5-9fPXdO zta$?R^cTzQx%PsvZwVPwm~!;3nu(OcafroAyZJ+@{K65j$e&j zlZ_W6U3}9eskUtU;61Om=4?&kk-+Qhowers9XDk1(O&4b+2=tZb6A%x3K5X)FMdP? zzJisa91jkLKL&rGMqnh=Ufv=-g>fc)c#j;7Q;d7~){-p%n&=;?>nV}(gFco1{fiBF zN3!yA1A|O3?$&)_tBsGNLj^yW&~~n~1E0;0RcS8AYb|950BoNs$>XoLL6+aB}7&&Qy09NL~sV1YNmd&AX~N-KSKNKd(DgJFF;guzL()1-3*ld*t zN0{kMMmH>wnwT>uPI=bqU)7%+oTg$%N?(|9T{CAvbDKK8MTR~7q*>HFCW+%0h@E4y z?;-uTUdJdR5}#e3@Hq&ds<9N2-0PG-Q_B~qpMq>>x0%8nn(k5adcJ6 z68VWVCk7m=mUt0K(pyeetA?4tEO`*;r`Kaw<|vX+d&kbq0f9l%t}L!?#WK-`PE5dk z<%P8c8Hoi7P@^ud)d(4=bL~E^u7y#wF3D-*a;q$OdSa>7WF#d>NkWB*4d)}Hr370$DhE~Q8IiNMl(qgLA^+1V)A0)bxbF|mvqfnV4K9U z)??2FGpvYU>_^a-EbB4ag1wjgc`6`=2BvppZbFm#qGmB#K+FPW&|McgsL!J*Jt=>$ zvcbMbB3`J>q57l#HJ)2qdPG)3SX;eLNSK=%6;dj3k$Y zQ^oQ6N0%vOOz+{Z`it6sTLbW8ikC`rj))FiQ^A)+5f~S*)L%53U~so11;CI*Ve-}O zeIN<{_{qdMoCSpk+mZs&hHXmNy>t3{*(lAcusiuG$ zN0UT7J6&)?z^}xT{pIdz>-~chVKR5Ova(uTVV_D}_KQmI8H!MFXUV{r6KgG>$jHsQ_2N!QIL`+9~Y>C~~=iIsxx z^Xz(a?7sV%{WA{5q!|yD?L++MPB zyZqAjz?{a<72n0cFa$?I zUvD}N@3RlkN58W3wWHDihvC{3m7x-EWs*OUoKHTpx8IbZUSkVnu;2mx!l!5%mwskW z&RS_&#c^O&>?HzO34C z>khWzy)=thkPnz42JM(}4=A%Drx^Bei}3q|Asi*qZg%U43n8lORTZ3lW8WH3eHpT9 zR;Y0N@%g-nt38rmAWfX&;hw%Ycyiv(x2E+T z-t$9Kni=h187Td=Pl@gcZ625De3nxaT|MuAp5gMsVZdUNZb-4FL8gg?L&{8r!Dr@9 z6Ff#~9!fu|f@swA=<~s)=aXKQN$;ReH*mF&RJ2Y5*k7w{n?2r*p_5U=eLt;kl)ErY z0t%517pvv?m>cIcR;$JLn)}Nf!zkY< zWr(g~9a3&T{_Kn=#eiFpIKVrsoC0MdG@*#G$%$KHMsW5~KTpt1rSx9fROVQA6X?SS zP%JE>GougkkO_Vb2yAdH0BeCURI(NPb_zSp;3e;|Gi z#srlF$$!#CYSzb4dW;KF6~=Uc#@8|(X<2OngjV;43Eh6)eIaRumsDAvdjv-2w*iVE z4sybhG#-Ib`SZGV@Ysh6tn)B3wcrW^6*(AM6LlKgv<_1nchoSOj8~UI-<~kPa=3E=&C|Tg8W%W$JdE_}Z0a`ChGMKMT~px>lgYBXVX-6P{gTX?7Ix%$gh_ z4lYFrN1KFWaOZ0cjzI`~W)|(5n&G=4rj}2=Mw}%$?fHAvBoa+#Il({pFYWk$OiX=` zka$w%$^NP!#;<{YFc{u`-*~aQTDaIi>NBI^#9+`2OA0HZ%)jmD9-WY2G49Y@rfA*^ z>A3^>tYqS@i&96)jNY9t{Wjt2-v#_&WjgmGB0wF|7G@FAZa;1)D;(-8WdidZwW(k% z1;(wGM0DPw)#Hn+n7!~pw%_4>dL=IuMKok8ybUlH5%Z}?VpQTTBX9F~;}ck=Vp{ca z)QGxuT%9VytfOW-p%OAs&cj=Z(2FZ7Z+R~v8w45pOLHDIW8PvzM5)UtfJ)*#BO(sE zqYeV%ZWh5WmGG{NRX$Yo=8MsKtu7vXafoxJyi)o|mziK`ppZh&O4b*lqzL3RzhkJe z(i$KyQZfFkjbiHSLcY*1NFW@k)hl`(J*|KjzRY;8QDt>#{GuP#O^w*bBBTsyDB7VZ z1V)@l;|OA$*Rq1w+) zt8n65X5im8sS$(nifp!nOJj$cDTlEPze9Ns3rR#vlOwr#5IftswlQ^?{J&8^{$YRs z|Noc1FUD4S|ukyMfv zL|P#Gzp^{os{aZ`tu>}`BiV~GC5a$VVDMRY1V;vog#^>8i!@^3ZHJULHNffexHlR| z27fO=OB#pVO{eOHWr>LtlMSJz_Zo8HX=t$I-vavog3k+B!*bAaV(`SW!O~!;(%#0x z!*ak>9!QFYpyiZcu{H%9*=way1C62Pkg%A~ws{$OwT7aCco})LdeuL1IR_ZeJ~Tgy z8D7tq#c>t4O)EKdr2^(^o@kLnNKdZ`fr>ENt}Cx=XW2>5&(i}XGqk5lCRKcT_hQ#N zUl)k4KQAWOu0%HN;5*N2WUC<*?pkS5IxnYo^rNWjy1hdSWMnr1rv>tTfR1!^16dewOHBaAvxorLbim2 zn>CidNh9op1s-Ckm^nQ<^7x$J7J2uMKLLIY`Uo>{UA;OY>@rzmfv-_Od=&`zIRlONFrc1CnB`{G@r@eb4cO7ti{_Av`GNzgQF2Yu{{ zm7peW;uKOyiqYr|rCaP872Mvx1q?`LtqG9;!|!2hUjM-XrR^{vk%O=I<$vc!*_z0B zsW9nCM8J4`SppUe&{kPT1L1H@?n9+JCa7Mwx3-Ix8d?_>OT$*#I~R8kXOz5OPQv|F z(?H!@`RY}97grx%!^K%)Dlze{*T>`ii0K=npr_V-?Y{9!zYnslAdWBbp68w^MB+ya z6ez9UwGWkPCOD2B{w!6^KH2`L2n?PEc%>?%$iQkoO-i^HT&&es1ZSF7EXp0B0tDQS z3x(L7J=;GNCRXPII=Bbjtn(fc*zcK54rg^#{R198w7405=KIF!60Aa~@szv7Ypk&c z-4%fuBdh{u&n9=lk#Yk^36Z8mniI8#e^G8{9K6ClhA!>QEPgKp8WkY}R!t`)vo{gJCamE5j z*Q=PnGXn(e&7W82{vp7JvRpY zHjDnJB1t`$2lIJlKW{ifpP-HL_Fy14%^m3V!12-C zv~;b7@7IZM`voyjfpLk}0IFjv@)v&LunoY$LoL@7f#+X1Ij1(QD5pmNpdX z?SjkvWO7vZ7 z=SV@TOqt(n9(e?wJe zzpCJXnN+QbfC&rhZcrG)5OVU;Gw$^dj^y{}0^@M{4Z-1PzW0B&_2lcLV3rSf=K}pd zYtR23j>nt&Q4A416)XZq05mhVGj}y7XZc?;ck#Ni;IcGC(OXxyWzpNVYaX7_1XB;b znteVb!JA3Ke7V6 zAQ51&5Y-x_jM+oUzw#6rXHb(m76eU0tS`Ah5JRcza!ZVTgS~sxQ?pmpB37;_4CoBJFoD4x|OAPE(+n#c- z-QHS#kdn0*-ML>6osPT-3YW)WXaHRS2zHrYkk?Y`&V8+e>iWtZm+~cv4OQxLG*qzT z1-t9W)fMEB}*5Q8g0lOx*bohB=0bEl)Rd`y0|@;5mBKa2c8^ z23t~qe#9WLM#$HqlBFD$+25tEpsx^#hJT;FaNNmf)W3D9#DCvDa2*4L=HsBNID(9o!d8%=K66h;5)LJhR!6$}!Lf2Z^+V!`%pRT_P z+dNSWsP8B6Gqp}ue(7YPtX`II%7yHP0B6Va&~7zn)X+|sc(q5ga4=8i%s-0=e}3!Y zRngg`L@*qB>O)SqGqbOZDkMkRkCq9~%0ROL&Swjy29PAl-LrkM5@7V(`=xBw35L8&2MeLkx$Q>^{-1N;?|(F3|I9mU zMvxGI|Hit#(l}U%9!!ztV{rxqKv63ZRR@bfpkmpg;1~7v@r$9tx|>>l3y=ErC+xWN z8hWEPKeC`%Wx?UgZXwDNB?JZw*#_Uyan|p&(BUoMNBW(Ha2oVEN((t(T4Bi*h@fQX zwLq9k0OTvDP1!O||0jHOoW>heNI?Gs5afgF6YlCxI)gUMicxJ2)3v#5iBsNTzqzCE zqaWRJ%@#pO+kxQVPlKgqR;3mt?Mup?AD#ba(ex@-49#|AQ~4&S{?_ripF;YoJ9Yfh zWI0lpj6;IR+@gcUHM%G1R0RT;&_y-fT!~Gie59_J>9D@)*87o6tcloz;$uN+K;cA& zRqY|i?p40Sym&=-bMx`alog0rr72a9X8`jy3a%(W zQOKE;(P>c?Dy*anxxV+e_O15`;q)k(JO}h0?g)&1n&i6&hHm_UyMu?`(x^M`eZe_^1`C%+N&gP7x2kM0KlACYHsMk8 zH`wE-!<{Ubo`y{(m&+$&VnGO;0aNHFOnxQiVk3#(sz5JjWK4Q$PqhHnMvc#5STyaT zM#pglsu7Gdq$Ek72}C|jsx*dB0~zTDA5bu?llLSjWM*mt$!NuUBKUBWS-=9tmj#-G zIP92vt;Hq~0<@PqB@xmzB`j5>By~LZEb|gR3HT^WajYT0~Qx{QyB zdW&aru%Uk?7}LJ^sNC%PlnlBJFZwNU*u_!>n}@TvF;#Qnb%hy-G0GFh0N)h*hpbpD z5oE7OzMOqQ|IQ$H5BD2_)Rkc2HU)`(u?M0a=b4V?s-a~ zBq^@z0PHmpL$21ghUpdtA=6cDhe!r8+1g%JsztV*FAmS1FiR~0F5}yCQ#p0ba{8m1 zx$EO40y*vkvDhzSuxdH)~Kk%QiSsH&OA^hz6LHTy*c>ary{}^hZ7dsIal3Qcgs`EFAH67YHpB|F= zC!!@Bq{hu|{u%<5u`X+irpkAks^Zf5D#*?K41~u1F%pyr5IJgi-`ruQL>q%EX!(h4}3G^jrfP z1-}D+1|1@|ReUuYN!ThPVeQF1Fd*P6k0=K#^$Ihr@M|Lq8dV3B=)d}580XR+)8Md2 z{C`Z5olGk>C=)>fZ<1s5saEZUs-j3HClms0BZuI`U`;Z6<6r)wER)^*%33^bP`1kex?J_}1y z4JG}OPA`gKG;?z$D=hqEP1qG-vK<%WPq0`ADTIeJ{2gY>L?qoXiN|!Cv;)P)%eJjrdm`+~X9mLlb z>Q!KxaIdG}tgi1Y+BL=ZTCvffAQPBqQ=kgmr@GL#*wybShY9HZlJmvv?<6S#)pt{r`~67T^Lil2mkt@^61)TYX!Lrkg9 z6*FJ1o)RNHz1CIC30f^0%uBXlGGIEChwb=Lx8L?`W$B173sm(Ul22;f-+Oc3g4BFclAUVSQ6K zpOZi1BKZoB)$L9Enk@|bRoYy}m1;b#A&_6BXHcBxpUu-Y;{sEtO_a1m*`_?&97il_ z@@i%PmB4rV%eygX%c{>(0>vlZHH1SSuJGI1j3u0EbuN&7gSF(vq@p{`{=v#2DtxE8 z+FZ*2dmgR=km&(IO|$R)Hroux`l@3jTdTZOJe_^>x7BUJfRz~4d}ncduF_ueA|r1) zDNH}l%h~YDu)$KQF#V3!o>h{DgssO51B@XJ5@Xy&c@QL|rt*(z#8bkIXZ^;*xpbi! zZN})O+B4vuY?hQ=>Bc1<$l!CbUnJBk;*ygQ;?V3f?c-9dCE}~B_G;{6?Ule|w|7uL zO65q7c#ZRdv$qOWHCcMk&lXTjvHJO+q^Zt+Bi`Rg;rFx>1}$)LrA?4W@BfflX2K?_ zG{DvK`+qR+|5neb3*V7=fPXEJ!OU+2dyfwNJT>9UTM`Xw23KhDI?@Qt`~1%(!9|LP z?)qbW^=s#!gnXcFTH;>f{#QiA(xZ& zBsEqLGZ;`sy(7&D_hd8?9zZ~aMrA6DbBY9S)ZuH!VaaI70`Y;YzZ}Nm+Hm_nhms)m z!#nKd1ky`z7jdtvUw^0w#6^mY(_oHdfsSQi{G3qlr|1FYS~9!@9aR#kYnNJ1*-&ws zIhIo6r8qR*0d?O*rt+C7hht*Bh7Mok=8|T15S#qx~O2s zd%S89@B2OvNWB<{rXQUyr+;T#A}D`Q%v4^8@tsk!OH9RDfemr|`mns(Wm3M(sAmb9 zd`*uTdj$F(15wuS#)G!KVThKp4*h~x@IKca?wH}`TYtu=-qJF&>oDfeh3kt1VYQE6 zM;{604Ou;NVmwp{N}z+at1bNV)Ys;l>_2X(Z_qUk#gEnj{jtoh4ZY~Pt!q|6;%tqr zr6(LrpM&K!X5`0+^jMy_d^%_9%nz+IBV$bG)+z965-xEj*abQ9@*3U2Q^@{#ySX9* zC33e(V}5E&>}Y!x00(u9qt{SO4WpXgv~fnuzHY#`OCBfV-&SUf{EEeC;s#qX$^ur0hVBq}(AF4PE*?#)xN(7SY+}dfDH!Ni5Qm^}v7z`aD^Lnad9lE04Z>K^RW7PGl0_ei&!&RcWYGP4Pe?!5ey$vmlzwvxY@AqSR+St0<^tke4`t^Hj>m8~VTL^)h z;tRGBtf$70I|u`m>w<4w(47>(?e=f@5Sv)nP2U{7^md$+hi&-{j&5LLJv=+d_XLQp z?w-F5hw=;V-mt_fdisv-u@PB_x(mAdmWomCoTtPTuWsT5=boMbeP4rP5m+(z2GUEmT{pz25m{4s)VjnRl* z=N^6N8T36gyDceUi+YLymc3N&3}?uUWy5cTlces3quI|J@B|A$#7V@vJRpOeT!$L9 zX5}(B0nImk^G4i^MEtp`HB9KUtwUt}XvwTE1N9@DtfluS!PTsn$_p{7(JP0@`}i!9 zqBd1uOd2wA6bAtI#PPqm!pX3QIW61aO3^dTR_&y}4*0#fyq{HNH7{t%WuCX#yTxpa zC|)ij_$iBWxgDVZH$M2KOZx%oEsmAB9vGB0xcy|PK7{Umv{l*5!zfdTJ|Lu&xHu9PRt zRHHA(=A2u(v2qGK^b{ifV4J5WHQaBYpVeC7>#_*CY0S_u5SoFvUm}GD$-sjkHxjiA zr^&)Jr92&pK~2n_j_7Kt$bbyqUnNro9n_!itK$N5%f^skLaIw1t1u+~LVi;G$B9OK zybg&9U^+i5-#+l^yF^pko(P?$2R^h-8R}%7NyaI?GKMxioq=v-hZhO4c#@h(2?Bc& z?+ijvv;9mZFrU5}uZaUb>nm+(?l8NqCgQ73`?`w%_peqf$2vu<4nN;Rq%?G%7paV2 z#h3~vyJWc8Yh2tIeCp9l3a4!X-lW#+7Rp@MjoYi{sXI;EB%^0fwdUJEB^fm-n9*qT z+#1a|gE84g(j6jH_eGq^A$8rCxGHXe3ye>mn{7qZz$HEcg68i&?8hQ{9verh^0Sn3 zTmDGuwu~$`%jSelL9v1htH&LoqJ*#G=c*vVt%&VG@hp-_dVyq;*WnG5`dcW!)+~7@ zFNZi|zUc6&8j5qA_Mn)J-d*EjEiA`=p;$<10>CF(m7E>%`OxG9zU5J|-w^Hny$tt_ z!KYwJjh|K7+dFxmeauG(!)O$XBT;cBCQLeU<;r*EO#@Y0NEr%PI=_8W;f)nwl{6vp z(LCn2-r&DP{xq(-&cx`#W6C(qn%Xv@8-yvN&6y$7rp4${7_XXjVqyl0Z%C_qtD7CdENoEN;b01s29Q&ztku=p#{ReV@1BX&$5ants+o z5(3ndgY0KFW04uU;uIe>!8@2;TRz~mk=ZFhurbLip||oGQf{ahNNi7WC>CeT81~yS z(uL26v^Gs^rZ3@0Od^!A#Un$IUSF$bpDxs_kufnu^OG%ueZiU`(C{o)D_5#W_BPi= z10qv*L$9N+pVVKfxua}-k1QiWMmTOio67jw09vnN zmY({Yak5I}dHiYK?KNapd}JOm+k>yeUsUo4;&F@iTbEqnr$gZ1K~t5yo%=BO;qM3h z&$-C|IT%bW<3T0?PCBeH1Gf0fDLmfC4tx(TVx_s1 zAe)yD{%FCkuHq|{*BAr)I)~(016)nj{JmJ#lliTU+v2(xXd$+?Hr_F>Ep{jJQ55Y! z90xcc8<8hRjw*y_YKUjP>`;qqWHRrGifxih?QJ&66CWJ!*v>e@oOg3n3E(GT9y2zT z86V|wTOqAujiw7(JKLzMcv`d36Hx6@pcX=%&bdnS#&_o!*#C5_a%Xl*u!>A%dS_=V zUD`kCG-SjJ93{MqllwSt$Ni&^tavz>p3G*m?<~13Mg*;NjmkY!Sz`w6gr+NY8P1QP zLv`JutSyKby@`26!vwybRf2Ceip%nWko$pq<9n4T)lxOJp|?!-!@DQ6zuT`s$#efZ zUPmTlNI>lM*R0=$v}A2F$3E;Ar=^?wR%>j2PMj{B+%5;3l{RaA3nL~lJzbYQ!CmO@ z*q%KLYdila78w4HIZX$TOU(Ye;sQ$(;iXCwp`!h->6V|eykZ|%*dC!cLjIJ&%)>zHSryI*raWv__)51N*&;A7K{h3~Dj+^8SSH#~PKb%ri zIOiEj_SS%bngtO!i%?dyyGrU(N*w!fSaA!rv2KB$ZC?e3`2tni*qDb)c3Af&9!2fHiDCWwQJS>hFCLCZsFZ2A>4le+O4!TAUP6{~uV=r6*R`!iE`eJ*U9q%urc|!{RA7yMvU~Q=3jb>H4B;h-!29K*zo~N9N z*I#FA$RE`Cf7VtRtM}_8+Tz(Ep5ayCU0S#1N+Pj&S+!P;&tVc4rp=o3R+7b@IGZ^w ze1Q|kbO7SCMvG(0L1NY5mKja`fIozz)ffyHI8|Lk@XPgrLk`(62^3#%I8)&ug<jfV_VyCCRN6*yagGJetkR^Z1SVkd2H^WnrRwap$l9QHew5JwOYUk zFnwsmD?g@Iw|!qeR!Xl!oIVPqiiMTn@(#5VSO=Ozu{|ht?p3oFM7UIwi7=ly_2!fY zj294c5DEV(*pV++brl?PFzGN(TL#`#6f`%z?l;}dlmE;##`CJ`#dMBoUVEboq)8n* z$Bg7F#%A{-M#tIwVFeX3XCU1Jxk{OK`M8+mu9OSBbnkR3*~t#OY@pUK4+X}>%|f#w zk^y-*z$7FY@jgfk$988lWu}vNp1`&{JrXUU?`A3_cWP-fc@!mPYk#&B-!OPJGS%Dk zO~0DMO6-@I?~kpEDO(`0W`21h`9zClu!~glB!e8EY?Rme6{6lP?1oJ5CFalCf~g#l zh*01SW(Ytb?zI3`DMdN_!?YRw1I6Ki?gsr~N-T>rGgLeI7Fp{9{@;k<$NFjw07nb_ ze^r_OFB45`HY6P2pa^uuXHwWeiLMU?(U#HSEH}a4AG#dJWm1w-8Y6G&*yGbYuRr=y z?=@z`Nrjh;PF&t|y+8hb_VxMX6h#QO%sl07o|F1am?39XkIPJtuO&M51sWfwf*Q9h ze?-ZSolo@bhkR%UWJ9Iz36cWNBMrgw`cBCOPfTs{y`E|ePy&6BK{AaHm%4yfOaLZ@ zm0?=-@R#SgRAwmktF8_E6;2Hv!Z)GHX+_LM*=!lj3O1OFzB2@no@D#3H*0^d zyz|-ln`!!OpE|3)!xmqZAEdk9a3&l|u| zGXJ0*bVWK31m)lJt?s>jp`K+Eu2*LC3sGX3ZABTw_Cky8l*rJilTizBcnJHHP>oHV zd;EoAw4`90C?B(GAWw%tCkQpDFtyK+z8$;lQSb3Umy5tH*WEY2l!VW$_6>X zfJvr*GUDnLj5YWM_x{ft{81zoY!3p)3y6VvCSZY`f1Xj<00@mQP{~1XUjs9iJr_Mm zylw73+;mrCVlU~dGzaGvhZBc?0)7Mc_>oyZz#62TUUwF^-9W*f4_O5l1t$duyLo_5 z2k6N~oiw&>qm6Bw z&6)On-gDOXJ!h?%HUH1Ouj}6Xw}05B+ZVTVV~K3tgKJbQ-Q)3&NN^n>l{y;&xT}Wn z$n5S(Yl{13_snRGf!C%aSkEDfQ-7yCp0Ezti7%EG|5%?^Omj*9r534W8X#$ds>@KU zJkS?sg@8Q}SFl{O5f_i}m8yPdv6M&mLCrtlBjSyfJZj~O6EuO6e06ncHhQCJ*(FtV zeLGskPyO2V1~v)6prD_~#>ba!Gm&hg)ZhQ`;0(XC0@#6n@E-8{zg96o#ue@VBwHnb zfWv&`F*5NnA)vx0i-{x!7K;7WR}P0SNF3b7cavz{MgVEZEN?S3R;aDsI4@au;DC=R z*h!jlyT2I^wcUrB{rG~B6W@nXJ1Xj5g>)et%Vel0RZH+3Cn!^2|C+r{-3r^o*9>_z zi!os7`JT`gr)!BtZ9LJ}AwBGHoq=je86WrE&aX!c$I?0jjAw5B2$<%+K z%jeckGz1j~(3&3f*KVJ!pbw_CrVaGK#Cqw;h)M*CTZaLL#{&kzXrSiPa`k44#)Oi1WQA@nQ#@6rc_NS9>h#X zQwgmIiiYqyZzdE5VzX&q8>3loZ5_qq&f*HzM@)o#R`gm76_Dg~fq|>NWSB8=COe(u za&Q66;S&UaGx=M5UW>a@>?4KdhGs<}knz9~<$MXaDiDN%P8DsT6IKqYAkjtL_JYE6 zR00Ii5T&9GGjmHj-&LurHj_q@M0AUff ze~)8OrXjQDMVyA6pBT2qr@Aj=#SVZKh| zViYK(zrHbpCvbrhr#i}(4Zco-F~4F0mBQ)y9If|bg^d)Tvkd1Irq-f@_QY=giwq#X z_cFENQZMauJ(ahjJsB^#xU*VZl^8M=&a~0cUCF&aDK^eRA97U|Kk8gh(}wc7nMyNY z*uK8^EB1@~c|t)1^TL7Ut@(MQewj`N7p68SMCWy>k9@$vGbg_S2g54MOA)GK8DFazhX7XR_ zUqq!|aG_`=!T^>|VtP8%10z@8#yN!U!PnmZn(AVFuO!a~-p#82f~)>JvHba^{tuM_ z2_G3U6|evg`(NUqud*F51pv+O0*kgn3<5^pR3vXl-_R}WI}yd$cuAdA2!&5am6`q= zFvrKH$ABC{FF*Vx8fe+h=bnk|R4+Hh{n#OJxc7X0d#eHLzvdJd6z3F|YRZk|0*kiu zLdrw#*_2z-hrkZzBu%s-`=esW?2oWC)M4rY5ca)naT4m`o}iDlwz=PUA!sy>CDU1A zI^v8f6@3P4lxt$2b?l${EVbPZf4US*?sPMR{4xq(Fx;){Mh+&iv|qtbeO)VWCM6E+ zUV%r!at41b+!aj$W?28?Ls;W19tFit!|iNB)v0C#LFOkO&;ccFr8n?~6Bq29!|h`N zm__zf4AKw7BDUk*1(dJP9ua56I%Uo+0o!G2hm9k%&dM0 z%3zyXwfjgiTG_^TkO*=`!_Haqh%|l{B(ko(6ig+_ZNJQE(Z8gFORu1U_WW>!XGB}W zynH!50xeP1!+ecpLLiyFsVtGFglWYEoFnTmHoow75^zkqg#>@l%OO=)PsHK7frsoX z<#h=}jFN$rghz@p4$2LRk=$cFJC7t-#Oo&lcOA8?Z0T9u!9Q5xS7 zXEdgbvUXQ6$q<*@T7E;PKbB)WN)}6kVv^lzVl|W!2?;W3bSbKx)kKau6x~SBVihqI zPlYU0$H{M9{C&@}#balK0PD#A-!s!+rYZY>U{7D=Rb^BmB%rWI6FO2CEGhCE4)bRt zu|de(O5vaqqy{8sJimz6^o_tAD@#lvtnFS_;X&!g#pZEXzdJM;LVFTeohaEo1O z;};&o?eah;IPR8kA;@MFR{vE53g#U{8t6-tq&!9gs*4-`3kbf-7F_Z1n!292DNKD9 z**%?$rB1!=%-}xHu)L4_fpeZNgWIT6YO}S;(urtINgTwzW}R0)B^o5YLh)3L_b(xH zeCC!qwmeI9G_WBXW@^!xVc^VJC$mAVx>lT|)M&id2NMf8+CxzLv0$-DUUK~;xws@)) z@!J=;xSa4-Ry8Ud8BqazzdT^A&)+UBQD4MNPTa%!j6xTDZzhTi{@) zn~NVfgqZn?4!;JWa@U13Y+?R#(sfYA4C4{Am|mdf=^|=$136sXgJvMqQ~Vk_g?95F z%zd~wfJGiye8vANzW-igsdr-tD5;D_2(&;K6*(aJ3N5RlakW~fLhaN-twyq9Ypz~Q zQ3z=O5e;#p&W4WKhNS)1+zaj-^F#iQ5Z`OTKyy`CZd$+W#)Qeui1&wwC-_WHL@+n( zEe6Iv=Iwz-3*s;_(QvkqAXsnvB>n4B5m@61ShqUrnRh!Www8m>^UooLhmQ2VYJ#a`F1*kr(WxJAPn9=x6QU<58n`uwez}z@5}a=`0e0bq`SQp!=vbk-2cI zK00Pc1@6RAK zlH>$Wp9El-V^kQ3eYx9R4kpgdj^3W0&w0PGSJJi=8p`#WfiI)!YB=f$iHX5cs*%-{ z3inn*gk*)(BJHHjGsR#>*WD5W=q$-bgW?uz1Lk?8sZLtHnc-uKXbvVIpq9Am3h584 zWi2~_9=fg#pommzmOorG_xsFx_42N{^4;cLvhAa8#`paQDw`nMzbUp`A~QSReJ+9f zh3vylCt1lWC$V7^wt4#%ygzB5ZSeg;U+*c_W1XDh_7X{z?n!@N(2ofZ&>wJ5Qz}z` z;R%ehG;5r1Bv6?bogC7*7V+t`Ud z>~rPOtEb0b?$ALjoW_*^B#L$$Q^m~Lhgo!-BgmNO>&&5OZ}ZtFrd-~pzBktwXbdsA zPcV#@L)YPtgdficyGEMuxS}zGM>LJ(F!S6FVV3R_+-1%7FHlYpd?{e+L_Gf_)h6e2 z;m$3SR!m^%5N~=}GqPcl-kF!v?Dxngg5RGLp)_?L_o9-Pi+TpC5I(vF>J$%wRlthQ z&(#@Qs3b73=eQ`*hi;9pwhyV6q4XPOk!v8_=-oy*FrPUI@K@nr?q$xv04%Gff1UhP zARC2g)L^(hXdNI8I zie8$h!`F?Pj_;e(jGydmPc`Rjt4&0Pi({&2*i_-s6q${BnQ8+dgz|p1k#ku-gxsaK zeMXPHXyi2Q;mC4fMB4aaREs3tA~Z77fac9K^Mymz{#>x*_4G|W<05eX21d$&=lY^b}$ep%uP z&4G6z!VYOlPPwl{A2U@5VkSW-KXf&S$@AeYCPpPUzJLB&d;D6z+`m9UQkR(? zQ^}ISBhOcd|Nfa~@iGH?b9*Ea1TT_Ku~sB1=?(0rx)*D_$d=Iwh#Xb=mLynhk8}RzIv+%Y#2#EGr~ge_?C_~0+wu|SBA$cde;l= z9kK-)Ub@mon$w$HjJGsxbBu_2oyFtoI%B)V^ym-4O~5CtK2UTibWBxUbtjQQJ8%=2 zA#~uKtkPQ;paWBkHbK==uvH!44Rawe6XJr?$5kc-5E2$qwN}cC-|2wR%Q)w%AP$+w ziP=huiIkL@WgPq#Lo0dO5)y@g>dQmHRBnR94@WnMBtxFU7f%KIV>T1C0}cIu@_gPSJ!8KlXTi~ zh6<&E1lb-i-b-*D4Dj~lv2g{=+VXNe3SMj!CoYSAJlo&4)Ls5@R|@6 zn`V~NzZky5OM^2IjKZ6Tab_fQT!QD1SlD7C1Aw@LW9PDcU-^`H43};4_?n)nCGnuA z%qIZ0rhM=xh9F8T`OBIqh?vAj+vP*!7rA&19m=J5+-}H~>L3rjih;!b!Wn4VWNOLH z&Op`QP2AJLcH9r`=Q&;o?m1XfXZROnR-~@hv{?e>FZ!MNDj(Sx{gWRr zfJCWrayG|AGpj?Jt#}>2V8krDM z4!ZhUFaB(#0d&eiaPf-~NPGPPGT7c>6-QOA0`V2mOOKttnToH+_&-Gl;4d=|robS9RxMN4M2 zqb=8gCMKH0;fwI1m`k(R4*DXgc?n^hQgZP)q_rU?n}%;m`^G`0Y3f^?qmfvZYW!go z*2qUEtHQ3JMkIVM_01o&JDsOTzt@N!QL5yy{*z`%Qe$Fr4m=Eoe~01!>7@iDZ~Wzn z4T;DE?3+Ou>@6}nk?suQ zp-RbaEP=*KIYz{#mEoCZ+^mN;*$z{qz{&TN^Al@nahyzEnlN>+6)*@ym91l@rf|C` z#0;JmN1eUnLKMP12}`N~-+f+EY$pe$z{<58Xyzbd<+|+$ffM4!*OL_^6?N(n0F8gk z3=yo~3v-u>t&G0v z7KBQtbV-vM+NAbt3(mJYPgZZi?5K*M>Bzq$J~7~MkD;-n;Co+|3gg{J*Tv{p#o(+sSj$YZ1X{4Ru@6(Hmx zG9wbK&bq?VrQRAKNja)A#N}CXv=q!y8jDZ;>lH!)7?j&6lm|U`JGg?a?4U54YR4cHlbABxDzT7;l(w7&nGb!rz2BoI_VSAAeVhj41<* z(;SnbWF|b5+I2U=zXGBf8w-`5n! z+OD-$e}$8XfdTXtG_{CmH&)=vxf-9cE=)Fr$Qzr=76S|rI0yI5YLYjlzV+bVnJCM|@N9Xq4T`d}W}%kUs@ z-jUnLrCAoDy25_yHKs9(Cu00yIv8kZ#ywMqoLa6qf?Po zsLz@NH@cWxtY;NsC<$nu>7SvNy#Fln<20P9GQ!frNpP-@%M~f)^p2)1c+Ggr41b|~ z96aQ=IYG!pgreVuqM}&`B8^)4Evd8XQ@${WvbOzq`O%qq57f(swS?6~xIlPsHEVLd z`!eR9r|uZ{L3e6$WblI-$xVF`T3==Q=^&%RqRqSCsG4h>pTz(^#~6olkE!$cMe#=r z&GXUivD9^Vb#o0Fq07CoGcH)T>mC8^l+c7DEY!g~D3>L%%p+g8CKM%7YbCH^W@Iv5 z@vw}aK0lzBncNM=_+!1wrQ8R5V1<8vr+~A_5jC4GjGDFX$ex#wB;Y8TLKVnwgh%{O zC?J%$6p9N7+NV!R|E|#`1w+8LCTjSE7Bfe_BeoM&arr085v%pu7CjSBdzAt}j35T(m?RS?B~F_czQ zi>UUl-|459-1c(+4*89NzY7}o;WNjEW zASAcGWB97H-jgauG(Y&5T<17%;%$8Y6s=$Pu)q7w#-bKXRseovAE4ds}e{I_U5JffJM1VUb< z?$PSehN(eQ(J}#(I!Uxm4^Mcr$56+1V}EHIsyHF7}h+*g@-b%)zph1&vJ{ZVyGKm;Lf~4B|sN&YzonfPilz(;xs*6TNnYk z6EPMxx;s)JKn`5O)kJ!nuy~5-8PrcIA<4wTd!m?HVZA8e{9;qZIlblZ9v~|b0O-5o z(Iv28L@1>D9{e#3h6fr3BYuU}1`z79!usDkblR*J3tO1XUgR7htEVh3wND+bqxkx> z!B7>HzL&?0Y2vZS*q-BLQqjx2;;7)TIxy*@z;HB7PKtFkRJ}zsq|E&)(AYD#DRqmc zUho4p<&aRuonIG3fq(eR|91$(N$p}pBu{h;Nzw9-H%Oi~gZ$s@ z`9jPGz@W*f@*IK2$ir#@isVM;svmUN2I9D9eqnfRcTPMk8@ zWsx?ed)vt#6MURn-~0osDZi&DHKpsEyZXw9+IoMrh?YH3N0$?SJYV(r=S%glLa@ee z2szD7AXRA3zU+Rb`BU3AS67#njDfR^9{??5$Zl8w&bCUk3!~_FnyU9cQ#4%0Bg%?kSq7FXxx#uK=L8=5j#%Lp^0qy}T7Sr0eGT)@#a ztzU(|nH}$3v4)-?YcQ^}%)YH3O3e9hLNQttW78!O()A1*2@km}4ERJKKxec=L0BO^ z`Pg#o0QQyDp9XQgAFID7+h@I{dkDj*(I`G|hGOUBr>S~oiJi>M4wLK5cLetk4kmni z$YZQ$3MSzV>N;gh-!r}>-^;juOTeCX{tVY~t?qA3U$B`Q8|p% zF^vTh3b<1~k?3~MWINX0$j{lKNT_ZYGDRgsbPGS##b0rt@R~PKhn=1enwQU)mx7-% zptvx&1f1xcRk_u<1Tf~IgP8LoHOB` zqkkKDtma6C%;m5+3wB1C-l&LX#9e<-8eYZNW(=@YU=79aj_LZG4hX}BAkW&LG6a5T z3xD{qf{7GSLdTls2dQ&1I;a`+kg?0_jX45nPg+C=>^a$MRhHPWBc?7&gMM2i+J9m1 zM&U_@JY;)zqjWYht%7Z~Ysa0j1C=C-Vm~Z{_ELe6V_{Yhh{tW9P3JW|LxOqwhC)2A ziXJ|V(LTO2K>VzEtO9?-O-gwhO~C zKU0;Ad&HB3{foc~-flaV;u2P)|rcMY9s-`81Vv>EpuSvLTpb@VU}>qrW8- z8c~Iyn18|}?Vjl{kodF!lwW}5Xwn#|744-yO_Wf?yViB6MsclkXPGaK;%B^c5h=% zJJU?t_*20;uZEIOb>Im-@uy@V{VZC^J&n6Dc}&X=9Jcry#sXZ>k$6r~ea7o!BSh+hSf-FS_fpZh zJE4y`+?5CYo~cyA*?X`Lfkl(Fnnb!9d9J5SrwrRMa}2`S_p4afAX{8!wU*9P+elcNgo+5ei%JyhAVRXT(0QiiuF@AgarsP?cp%=ve$-u)w zkX27j5BTA~6a4sn&-u;#EpLY#eY)aFa&VYNKGyC0{|auG=g*F;hTo`<~@Yo-Rxg3XevI1l4I>J@7kdp!y@ z%5TVONCd|f&RR339ZLdRFN#2%a90SP){%3UV5l>SG)jL6G^!p)JzqowUqUTbq){_O zg(pWe!zm};>}M2Szr~(g$G4yuOrsNk2R5c_2Z+pY1=jpE&bYpW_%N>Ii3h6kuvkiG zFfzWwt`Ytv)5iX`chn;`;g_Q)q}Y=uG-~|RpL?u)Lyp)vG2?Wvc)L7Zh5&=U*=zi% z<0pdHqo>b@XqC)3yN8p^tSQXruC_hyMcMWmikzfP!Gyevn|f%TDe256o!kZh9**#Q zPSKRaQK-nlTA^h;)M^U~77_*t@N~N$<`nh6I;O>ITkd@7tbzI!{k~Xoyx~~4J!fpI z^ugez)T0Vdd^w*ox~!>RpN5k?9x$$tmLthz3)EA z@Gb{4Z<~&WHYT#_^aOZywYnz7txD9`^nF`D%dMEBEuW07_DWBlqpw@3)|YLFT}#i8 z>CW|GR5uuB7yFVivW_w=k2X#7^LK{dno3`YN6%><9M6c2H+t~5Vx&d`GJjT-9~$P| zsy9li#f;}~JGtr&1x)Jgk1IA#!FWZr5~npNk!HtaCmeU^z_p*P(c4xL*@Ercu!qn- z)<|ozrn|LNOQG-m7@6x=7hjp{))AWA-jQjUmx2M;PHjl^2BCyeA1b2vy7?942>A=L z3^J?320A8b0sVXWqMC3%U^*dvoLW=<%q2U^(U+PJ92v)Ah#G{3c9aTaO3UAVvg~_+ z*-}`Ic&E?OuKPw{?=>F1zRl#DgQ51>`29vfD5g#lrO zAnf#S0xSod%cF;Fv2$?8Ze)CviI2b9?_#7=``mS5oU-q{%INjd0ch_ehaL$7LI`fo zwjHM4BgEk`yjnMT)AR_xQZ&qm@}GQj%hH7dm8t8B$qe!;rGjBsDcUQF#It^hWSpdc z<+lY06Yi?P!Xs&lJHWwGhH~UFowqJ3#2FLTn>TIri!KABAWkDptNAR)mJ=mP$6;_c=z6@O2Gpaqwv*n}8&3pwuo>D~s4FET^^@y0SG$a(*!zRg*W^cRmb`8b(SZ zaT7jC%QB4ZRB^Os2WHQ+T=IM$ry2aPje)a>O)C5uh$CWOLQdbWgTghFuepKXu;R7Cg5XHkb_uQgEDNI^qI~$`c?df0)r}q z98t1S+LkjAm@5@PUd3hDkA8tZM6{q#c>KOA5AE=Zcrk2!-2!H)aLI zb)mskK$dxtLvlw8|9r&^h8xdQ-QfZdiVF+YlQJz7TVbvFWt?i_Nk?jK6)4^0jF>tx z!)=%r4Djz9G;W%eTE`cSrq_5o)M@NsxBKRiE9`|bKQArpul#Pgh-}XD5&3q`{N-?L zBoB%}EPa-bBeb>c8>-x|!Ut@ulgqjhFr|!)t?#qGAwwU^0vdaVW0wh)cnRf?e*9s5F~3PvDD}S6Bio zENi)qvKSs;P9dt%7U9bo;@wuWr%M~U`}n{6Z<6Tr-FN?)sN!&PGM9m`Scre^4F73q zQ?Z&6$pOj#q^B`>YSl4-AeUy)C{+Z3*NG|7D%q$}{RGqgLT72fF)F8PQIG3>sr^KF zA87Cf?o^yz*1MEGbtjX39D8!NMlOV!HtWazxZ#rH0?aDpcDVq)U9Ty>S*3NhY(xeG zkb{rIbg5rO2cUyjz$~Gysnt{ibBP_$Z2?qI%>I5y%CeEN@#if^8e`RJ-=vqyVEv_4HsdkKmN5{oWiesmpN${)vRMOpBF!qWe*c3Y*0g=jyNTK8} zZ%&3Qg-1V}<2ikm9LLoMclN$E$kd;$AE2|=fCmZ}lh=!jQdfwVUC~Fd(F4}lfYvK2 zSFVHib8su|+fSW&A2nYoeGo?a6{|Jmpoitek%t{*i(v{br?m@}*S({nu#^7GGd!Okgj9ta{MM8Qt7v#{n&Qb*JmQtD>FXi2ncV&QVGH&S0JXA z6>5YzrBpWe&uzT?z1ry|EFvm^2Yc~f;__6-9XOm+1W|a1KbA}k|3#{z|3r)nq49$j ztq$MR>=Z;0wOO}=r-WXjL?N>??-R3=J=wYQZpEG3*CPh_JwQqc4@w3Q%DpKK@vfq5 z7$)Yh^1586yI34veD4-~{3L6r-P-~TRU@GGDBfxfaK_Oak!m5?E)P0Ixk0fv?Q!G| zl!u1aO>_jX29mv~!Q~bT_vZ%j(!d?joGU6ahZ~`*P?dW`A)_no9Z}n}lCd+wv`)_$ zP^m9X!p}R(x=(7>EVbgNVDrEZnuYub9y2!#4%zq2Q(`7v{;ev2rU&o4U7gM0632eH zND?}IqqZHP=wZ*R?JH+j&dZi1DhAd`Zg- zT;Zf_Pwj%@{*}$F(3q~xVScigR#p|x8$YB6r~JV0byED}+gmMtxbG!%h|YOYvmzB1 z(_QiyM_rMfsb8)`jOH&fXL6yyGbkZz$kZ^^my+&+$4u3J&^|Ke8i`V)Qu>z|KH35K z#G=iSi-4-L+}BinYvEam48{9d3KlPcSyAMm}!u(!BhwTN(wuApB(#NY2e_lp|Y^`)Ypp%hTM#;$Zobar4t1l9%Sa z<7z)XqC5ZEW6i;~5V{%NLXTr>2nfR>)1fG|8zKDKU2RZttT({&8LDPP5kb~oY)lbu z^#}G+UoSD?t)$aC$4U&MnDOhmR+tep9-c*Qw6Tfpoi*E@`kmf4YK8)QW|s5@3jX8f zM{Mow-_KkMQAzKlNRO#@^v|iLt9JCsg58)%Ldi`&>~%9ha=(+&I;ff^%6_M zeO%)uP+iYTW;3yw=a_3ghuZB+cQPrfkmul28k-gohaPMa%~ng2Yuv1p z1usIU9juIzBJ;J!f zMJggW+T96Ng9_kXgiVyzO4jFcntf+vn-oo7k?Sm6cRdFbb5rL>*qAcwQrh@mtQ(ie zFBuqvYZPnP;iQrri-H^D+htpV&sAM0OaR2T zC$X(JdDml0U8k4;FN7ELbugO0{}Q(8a4MK*fZfZ;zW{>&b}vIv2&kz9=7>;%Af3^491UfO<>DOjjUjvXZ7j`%F@3yi zoU6WDl`2<_)6QgQ+5*l!$3g0w^U8_TRcYj8@}dRasv@;R*+m%_Kksj4g))#r?qT=( zp`cL>B>nlv5T82X-Wr#IacmjDL^EGNl)FHEgsLHi6h|_n$K`QLIsWXDZ3-pzY!S$O zky1;?1KVvKE%maS?&my{2g%y3N$!s7*M<10cH2qE$PUQ1=M<{W5BC!gkQ^0pOO4Zo z@_XZ7vlGu@g27Yv{T~KomMZ*0kM5Su(&jTuwK?L(LL0R-T7les7`4giCc;l)OJM=vHCKbT?uKO40oNAO_F(z|Fvf8Yr6ESs4zk@;mZ>ohv7rhU{lx3GC{EK3>Ro#5?u2U_Y=?0Vi6L(6 z?)YmRY(T=be-xtGo7RvgHX#0532o7XDT0R&M~wd~OdV|m#hut}L@1i7(HH-yP8bp_ zC8R+PaF`~V(%x7wZ1tIM`~hbgGf|LWo|FxmrvCiH>;Jk^;H3ADoViZlM_ zFg&vc(PlsOqOl&-Ot?c-Wp|i`hQPQfMx2ClRIg34(DssuO4RtGQE76J_q6zc03RR2 zNDxtR#P@F%9t7Cvf`cs)--%0h;9SuKhmFt~x8fo;dlj#}P1W9nY&Qpy(J!Ho3607; z2r0<`s+CJSC^(5W5HY7JUSkVY;R{zkopPKwH>U{P@cT1}9@YmKOkGtJ%xve$)cTv0 z#TgD;U(!ZZK*fcyG{~0m;%1W07*D;_oSB;|cT(yiBKsJ@5EFc6SSRS*M|^HSEc|9> zZ#{F?x4B-*CP`jdmLR(V0Y^>c5$`ms!c~=k9TYiu*N|IqmJ6-EpVxHDBi9RY=LDpF z0SlM(6@GP02gO6St8(Z2p6!M$O)fOdJ6CjsoOC&zoloGrGy+;7o81SF&r(_w@LU+~ zDzA?SPYCXtN~i66&CUqW8rh@9gb$vci4IhaW&^eLIf4r7ql0z?`-%IpRw{L}TALPt zry>IAczW#l8aVfgpmO>_v6ks6XXwi5rlDx)>Dl1tunV-7`i(~#BhT;W>Azac+OIp1 z1~f3(UQ%b49ORCRu3{ zG__#GR2M~Hv&iP5cAIIAkuj+5^=^j(9uUuDB^AZ(rbn;zLL`Y@8#LnJ7d|`vsH_Z< z)=n#-LSD}taT0KAnr#eDK zBlDrl%w@W7)_4`H3}(VFYQz5>*KhKAsm8@*K!maO&6vd>MFGfig zqwQ{KEf$sKl%Zm<8`qjJ7%(m3gxGHp1ou7=PQ!(X}w~@n?);P6dM4*{M7|`18TiC*4>oN&Uo_!>u&BngaRElU2-_qHx<+2`mvUCk0&3aW zd^%_JyVi6G6KxfT{Ov4PKuX!IrKAn}ye4JV5{q_~f{k03D{aMX(Y@F)i+IJob`*W2 znl*e~rwUZ=Rxx-yB;q{^>L$RFQdDb!)F7GaZ0Lz!YX z0(_sRidFIAF4?_a-oTT$2#*naK&!(v*ILhCDAi$RqRl6;jobBUjJ4&`xY8KIhW1AR zjb3lLI56xpb}Ii6K+_b&6e-7fkXyJsmHWCHtj-^La0jQ%j8p2Zi5{OuroK2Yi6UR1 zfi;(9?nuHW#pnsG!q$m7BbQW|)=SancOw{3QRo(}|I@`I(E~vGmUQ(!=!4`f>x%YG zO__=gPB>HHZ<}PqfYk5VULs2)rVE0>;m711vwl z4}{TMkgbW=!k%Y1WJ8;F&5x+I$*LDG^D@6SctGz45%-O(crHxuY((w|zo4rD6THc; zN%KqB<*8bd*I9o$mPMVY1QDbChlv<|lbA9A9ATZ{|2IPUPnY*8wRIE@F4fT<0V1{H zHxdRQI^l1q4A8o^xvI*B8Xe4vz2nmX_>aXOM&+(kQs7 zj8C86sCLo^Q}*CgTTsZ@rcpF z)N|DZ+OZpBM>43_^vvK?BE$}cZ;f3|(*TkP?Ukj7gnK2F%A<-GGFMd%`xGHKCKpW; ziuw@A3CqdIt^G^u2Ncjh(8Tg!f(E;6=cPPr+NQ7U9Ge}8eI7Xw6oK_- z2xNi0Xz2(s8~crLtRhAG@*LM@Z;Kc#Y?nIr8=Px^p!MgEG`q8OI)p+yxhXL;X260N z-xP&%3AFEq&sVCuwAeQjYkAD6(DQMwJOa_IjHmXwT|4yo+VE*?p08++(NfR$#m!Ly zs=IVk7IVL48cchHRvA)qR=zH_&c(PCRVAu3$%rN%aeJM{ue3!xq9zmG^;`uQvnyO6 zt%y~8qQ}gm^-%w&9yvo`4x@NzkR&>5u=+dFK5iJsYFH-|<>PBaU9*7LqKw8^~qYV?}`@C|8j0C5QDZ?Q|QGhlauX*cJxa>#vn$6fO1 z^ZNT4_8b10mQp=xnSF-(05Y^kB5Go@UaYO5H#mwKuKpyKPjTmG!<%d}?T=@ok z@lOFBmHj=W__!Xq1!3$V&&Fj=3|;~4eKxnnF!(<@$K&Wy^jU4u0NXK>E>>mhb7TWG zT3K`~;REESgCT@Xtx!St;?5<@hbe^88`tL}$!jTkw5}lgp&b;{qI=~TaD^5!(kp?ZKE zHcy}11_DGY2amsReTA@5nb)3HEtj&&Q-2he`fm)jLrxWGI%1niJ2D*BtK_|Mo4 zTn*^dGx5c|goIw&C1fm`eIUfNcN{W4KDFaN59DjY`{H)cGRO?5G`a%t{dy+Q>y}&U z-QeG0(jT2L)JK8J)5{SC6Q{He`3lxFqVMz?0>;H}(DkX^;Q(O){}RE!q@WoI2a_av0FQ@XAAs_=(b%3_MNWbMBsF`)q8&pLz-04wPH?D_o=D7O?zi`fkb zvO%eqX(2Y!?X!c{f<{HTkZhqfD*wz9GYYjG7R4k#61G*tBoWNb03eOA$A*`Lnx{r> zIsKF`Nu%f_oz@T_GeXr8^%-1a=Eqh!6Db$ONmB< zMTWKIK?2X=&$d>!z8SG?X(d*+MoVl6MxHjaa_-R843l4^`YT>VTSq6AlQJEZnH_A( zRnKvGq{*v#Y!*B|;eg1>hfcwpVDxg`=w*pl!pq3lwE=KD>hEincFDS*i{-se1B9aC z7YcqaQ!b$E1GPn0JopE z0S}z96IvN%kImv9Z;bR2#=Hs{G^?sR(_5%H($r^#yUsT^{|;!&5b|l^qD89^65&BV zS?3Q$F_>?Xs|WF zEXsO0uCn9%CIc#5`r0ouZujou1=ybpEi$%meA4rDqD1I5QwlZ?ZPdjmFQ#9=P8Ya* z^jLFg+t=n94G?XADuu4KZ@6R8wmN*FVv*Fx%#NC}^uo1sI1b##`=!awC8nsoz;onH zB{XAn2d2vZMDa=bmEaRUnrdU9lfl=w!Fjc%qS zj}Q?e_^ld=32#=Sz@Z~9tg(-aM5sI=Rs>kMBP=x`r(|mI5nqB&ut&yG_j|yMurscl zj^%sUkb}hMUm9Oskx?SfEJMJ`y2EB^eU?HWkQBe{yB>fh-AS-@!}b`UB{AP&{@Dy2 z5?4o(0gn{Lzro}G!fOJv;3 zBZ5o86v&ofnk6TnHrk=ONKX-F5bE3?h=szmoCCYW{5{ty(=}m}T<-GXVxYvK;F{wZ zk^9zQ$OluPIb{hcz;y_5`u(xBp2)gYrw)$+$(1rMR9Vo=pgo~_yuLNF21&w8;z^g1 zgn?3_#A&b#r&!t->oRCbd1#a3wZJxN1aTQDrv!AhS(L(J5-be4QYh3*# zf**E9hSEZLa`rX(M7jq^HIyhRIw$<5Y!a7pA0Cj)Mvo(EQ_OPT={dVo0p=SP5r9zIM-1Fk+ zJkk+!i9ms~P|o-GS-E%f0@EKxbjRx=7#17(Dl9I@Z?!sfYny$dZu>`^cndQrlW>22 zI6B-f;HQN==+FLQ_Vz0;j}JrR{-I^I^GS^_|2WcM_Q3l4+!LfR3Cwy>ayY(l!kVg?I!UUKBJ`h2QvBq9Hw?M;7KMY4~&&&D>Rt zQ&*cx_5f<_oOf|{HuUrJ}=+T22ZBFR1<`Q4Y$X-E~E?c8D|o{ zU#9^=bk^GVGQtlz|H8tVu(1u(w0#Q8En#0xI!+n19BK?YZ$CsDE{tI|ct%TorqWHTRf_hb=vJM^%ThsKHAg zUh>wxU5EZ914hj6&sQeWuYe+<_d))mftU-;?;c1^rBdumK5$%BHCpTHwoM}Xnt9Q7 zWfLZ+Cd?;8UG^*_+}u~2>?nrVEY!do%>EIkv+eg5Wr+wT6toF{<)kD*BA=oHOY`VB zx`>b7^)#KpNj$+?;^`f1X+zdARjZT{ryfT@UDy2?4NBo92SG0~v1W*Tb9fe_4PIj%}(Pz8pQQ#6Y4vLeN>DZvzV(z7D?XK%R zpoxXX9~#Bw4%D&X!E$@>bYO8f^Um|c=!fvi?dSAC@bW{11YF&J>O$>t1AEf_)D69L z{pkIe*)N7;jQ8?{HRZ@Jq+N! zR97_q{IIVgTh(6xs88)Z{NR76uBr(IYXVyk@&#CTOayq)8DR3DDFQ-wjFH-}1R?HI zlxsetN>5TQhXRnvsUXsGSeVf2;*6OmNkPV%0${l03<~yi(h%v_iV#{?Um&z%O%0hi z%LyZS;@8k`{9K#spU*i4f}GQz8lmGx(BhM6mU}xgJ8%YUU^q*umVagh;4l^u0_R*> z@EJ_wDnUy2A>m&36!Gk3Ppo{Cd-2kysos(UX4lZ@dVn4r<@GVg#~pg|O9cn~Y6Yf{ z=F*g*bmjGs=JFPTYNhK>n|XPDY9;DDG%9GR+CA;cJfY2H>c&4Ab|$Ft`^r_c`>a&F z;V)IMDf!AiI)g7Vd`C>KM@$_;1I<^u=)VSElwYwgm9MorBnK{+k1nr0(E&X}%CE?7 zUiLN>t$@dd@1+Cq6B(h1k(o&nX78F0F-E5c;I(cj+{yEBbXg5>fx<3hk*BHBn=^xO zSeFY<6=y%$2;7dL5x)_e6 zV{Xun+Ous5U(ZS6$(Iq2TlYa9Npx&bMb~crlwKaDr23$;%`bsyTbk!tNBp!<)`WX_ zGVoFm-=t`qN2U7kWWVWLzN!En61A-ISv$9$LA5d9)Uaf(sqxs~deSC^`ly;iUrL2) z)C^ElX`{Hx8FxcyPiS#ebb))<1IT&J!L6e$-e?rnnK3CWfXrKKqT@!s-YOzVOPwRx zRo#>pp=F8NLldn_i;s%j&xBf=W)tcbGGZs*O|u4ldNY?RHECf(NmC7X=GPhZ@S-PHrN|4XEg9|&R*Bri- zm^8u?*u=&*!3|&T>d8j!nAVzSx3y^0xuGCA6qDE-e^WEb_Y!sg2p2D#xLUbgr9WM5 zQa>y0AuokE$hxmrPjQ1~!NmzfVbUQ`kS?7-cS%tb-a?k+n4Ttn!5ToXB|0|(W+yrpSDwT>!YU_F8 zHJC_7i6z?%g=nv>QPZd3vyhd#0rxZ{Ee7aoyCD^i& zzkZ?c=op%$Ni%%rYN}Mav<@J+W2gb5E=Z5zPNmN{mgeK3EufmH5&@hR+C#SV9Bz?V ztFB4{w0%vF?jDUW#R?59N^c}QUrP$)_Y1T+!@6eDkJ2%6UShlPk4K^jH>0`3ZUsE8 zauNZC$TF};sZM&|x0y{Pzhtd#kwLh3c!hIttg8JueOdoXmZDR-VK>s>2360fY4e^2C@%btfvCP-z2g+Ujl7*@m?k@V7ea7hX z6kVPbJ!!c6!7$G%EGE=*_z@ID2cm({XSv(ywpu^4pFyu8>)u#if`8;|Lcv#}NMFZP zJB_K&^lHT}A{)*mo=^de+p9Hz&r>)up<>e1pDTxrNB zREPI>HhN+zB||?OT;omSh7i+OX-9qhMI*8Q_|mHxg~idC%S-`CWG5HU9Zn&p37kI% zMl1_3DgHscpO`fQe12Jj;gIjYjo@%(+SU@@r8Ij!`qK8IIYnWa9IptH$n{2~JKP^0 zm;om%&CcG+pCjfFQG%|{WDUA%OYS&et3%ng{rXu(V`VFH;Dubr#H5~T zM08%yL;#uR3G7v2hxAwm@uF}1BAwmN_1Dtl5QX1oS6XZJ<%Sv)gLWWF*>{+l6Z4)b zBRng(Oj|q0DT0>rmM$lTPOt5BBB9!~b_0$xS;K6ai{H75$}xakR}&;spqFHe5-K!4 zoRXo-9!+T>r%6dWl;V@N|Ya9erF6>WoX-p#4>}2HXRy zG_hU`WuyA~V|))5W5Tm{RTk!I>!7l59L;kE&E>v+f;kD;c9utNSc~0Y*yLb_{F>1j z))K7H?!iAjRxyT-df*8VxyP}V$}8h=;zaA3GJV)W-5&7*S~!Of)uCB zsvn{v_(u-*f3qC^OECU$=rjg?WMYYG(AsE+=ud)szjaXcU|e<=`7Nl9&%)CA|ytUvDtVVWG>rfE=hQ2U%HHb zv2(ZgoY87Ny>;1jJ$AmocV2crc4m6qjf6~oHs2EFpS*g)jF!Mx;^~^)#R2eijqGai zbWQAX^Wg7Z>)`*|z4pQ9-@2B<@7}ovkJh_=f{*@n^Mn`u>-GsOn*ZhrHJbnS2{yX> z<_Rad`}PSk8gP>leJJS>b;!v-1>r8c)vEt#?0Wo@>bA#c}lRz6h_6e--(G1zZN?)Efx@;6(Y7WGMdw0#xNU3T*I8=7*l1STMGoXK!N%Al3V_!b1Q z>9r*?!4QrxPb7OdN62G1M>Iz5c0(wvdugcL7CJHg&OI^xwhS>}AQN)-0538AfR>1N zRIPUyyxw3YaThQ-^ob~oVMk1Qf5<&GG<(|?IeM2KnSTgU_!Wl%mi| z*aIsv84+!N6?C%7SpfxD7>JglC{$Ne3^CMs8`=+A*+X+nZ_7i7ZHej^6_k{$4N8)Z zE6XdbCDKTAAw#x`oe#t!%Kb^Vp{ zc~M2Z5L;^oNB0m90}tX@?>F|=iTNZ2x~#8#mpa0BSp(WZ^|7^zg1Q(?03~G3eWE>L zJGWvey-Z6lZlz6ZYUyk%(fq1#S4oz#(tnw-hkl0zGu~axy#o7*l{L>j$kVk#si%{rw4rv$h zozO}BSJ`QrE^mxMqms>C`oc|U#xP^ipe2ITPB^ujsp4CmZ&0mciG74m>bYof(P#yO zb(f-V>g?F|1&N;Wi1bFb_7N zoBo;t_V^AS1N2No%nG6JNUJEgSTbYMQc9>O7SBh$IV=2x_5H(pN~PY!BxLZb@fw2S zj3uPUZjpw}DGB2UR1;aOol26(@n+)W9CEwxak7K}|19hL6_Mo4w8CqIF_aldwA~n< z6t)2xQ7dYjSqJceoX8be1tw1*y3+Sq#Q^Eh+x&5Z3GLAIN=$iK8`QCbCIm}O#U$m? zkq%So+*p)Fg(MD5)(y0fAYZN3hDfeyuDF;=Q{hI|;#t`RR#LL``&?;B)d>$tsZ8{m z9|`AxNLZ14AXTyBP(wseUKY(4xp0y7>e1q~dJ)C4<4OskCVVcI5mJFUG%rrTo_-p3 zG_xTjnRvuHd_l}lS&eM_&Fl{?1gq2VgAWby;Q zz=o*(Vo^O)`I`^>2g}`=p=w?9g$?4R0z(|Y^@hN>Mz!y%D!l-qDdv1!P=Vtl98u<6 zTCx0HH&>@6mIE86b5DE23M<4V;#F2&Oxw8G6~Ve2&ZT}~JJK`gTf-^)uyQ~Zfet6h zoZWKpAsbmRHISXHJ?%?T5<`Zf(pc7von@;e8P)=cifjycDaVF^)ER%F)Uoxw0|_ob zb&5^iFdaZJiqWJRekCYSSzvjhFIqYAoprJFJK9W*C%RGzeV)D>QIVpOwu!DB?Sa(8 zcq6=dj+|(0S6Qjw%17m;GK+Br=>RFgCtSYITF>wMjZMj%*NX0oHuOaudfHYlAcy=6 z+U-mSBeTD;xR$uh7Nxxi8AoVKHVOyO{R&>`Ca3PoCv8+Z&=y4N1L3wN%*M)M!6=RP z*mKCc%eniBmZV2*XPO zqb~J>9DQjrugMLDT{{1~0<1Owse3qm)izzzBpqh8h(5;}242>R23m?>K-Q`ru)L0w zH8@vfKOCkm5-N9yoNg`LBiK?>O@2RgOBE4kl{8kr{G)h9uUBTr6K3Z=o?~l#23wFX z711LR&yk?DaRbX*mE(@ewj7@%Thf9L33_S^3)gg7DF8u^WA(`D?xMQvPG)z@VqW=o z8FeR1ODu?0v#euid51D$&T|&~~-_Q#j1Cdxi<0mEP$JKXW9hdeUJ7oU!G2_1>XWe-fz!K9}PQyBmH( z@4Q!X53eV!CL;iZH?O<`9NHt!V1~l@XAPb)Cw_nY49f`DjmLV7BV!zF>^&b|)2Fx2 zbcLmp)Gtde$iBu(oTUpihp4O&r)~)0`+lyj^Dsmc8m#BiLatvP6#T^$2qNpRUxl_U zLUVHOhVPo3TpqlFa4)KL`$ptgSZ^3+;yncy#c0+c8~5#I3%NsmfZU?SsDbt@Oxj2+ z+UysHq*uqwV)jTA>~D;3J%}-&$9?W+^7ipBdk>RWV4+;x>M?_m7WBPf?|+Y9J_^fL zQ9^(EWQG2}^Q`0wA9#x7oEAiwWNig_tmI5u1ZaSc-5ett@5O2n?1~z@evkf2LP%3Q z)NZV-IHjSS0T}xAMT_m}NG;7!>eqKm6sUf%j@x{s$#108@Dp*DwLX`vOOGe7zu!I( z7D8vPHT&DKA$W+44pAa3Q^~;y!rj9s&lBvu+v2yQ@ zfBgL<&o@MHJJ0tg(M2dWydsGFxIt;DW5Hn6+Tq*TS?L=N^kBS)Qz#{B8b|zWb8e^p zVUwSp&GkAflgROc&@XX0KFS1XVfan~6T;)py<)W$s5s$dM_g(fP?)@tZ69%c zZb6}8eGDCApM8I+0Sg~OYJmt-vFkV41`mKhO%lw2+$hiPfzLfrK&Mu>)K;{!1t$MD z^jwicr{`FGHiklUyVBa~pEJOo@)NWfB7sM){ZJf%!Y5}Pb2kU$>bgGc(X(2soU-4M z`Y(NrXl9yg?grq?N03; z1JniL!P)KJ$b1e z@)Mi1=Lg{K%%C4s65@b}`q#euN~(t3dOs8xh{)XZT$7@i>?ZDdjN<7690Bk^Uf$}mWa-A$KPi? zxmf^%0lSGXOa0_+aSIVNY+R;UQKDjbpr#x!5wzh`vg(MZyV_Hla4qZ>Zgao%|L6+dmXGnrS!oO+xNTkld#rv8_5@7%rr)S>&3IswxwD z;9J1ZTcaMQ7ch?(3f|Cwe9F(LVoHN7N5NQmB{SwU_FbCTpK!UU9bl-^; z>1iY?^1{-8Qq7nxM9n@T7W4Ku_f>cA53S9|ndJWGOkmLdhme1jgutG>iiC&(_zR?_ z@Ie}E(9!(XOKd4(+osRZ2{cD|>-`DpUMjxAjRocS6%s>|u05rqSvKqFcG*^!l0&Q?`*;AlB zqUs+><-k7*#!3=mr!n18Ks^Eo{W*R+^9aihQGRuaBq;~=hstU&bD3U=57B;g9ydrK zGHvl^Kohd#Qo5t1Z*9fqNV036boS5X&0I0Wl)`v?`piz|R%Y5-M?u>iW8um)t_NZJ zH0oLwOum?Y6IoQvS!^rI=!o|iQl$-R(>Goi5u!W5PE&Q7E>%@;r3MBV0Bv!pWyuF0 zV1vlHztLH6H@(LJ>)fAFH5}c*g-*UEeSF2>HbmBt3-!B+=gF(Z1u4>>v4C1j#z@9T z1STxzR{Q(S1L%Q7wcB4uQOaFY7)Pzxp8bh*(S0>4#iP9g0hW{yidF4RLRPw*4gCWt zp;9;DF@tdD?%7MjssVt@4&8@P^4GN;Wf<7=qd(@M&fz0hCr4@V+l8^V#YzBHM)4V@y0+=3Wd%yF zO?{LPkRJ-Mb6b!1cekjf*@smouA%5$xjmz2bH2FnLgLaRGtv|BMThpN3zq(7xf!jY zN+)5y-~#dZ$k?x&vA6@fQWqN36`9U}3vRBdo+F*5X6p;^%#~}@qZOq)+>IZM0&Vbe zzJEuJUaq)Zfb+-g%=uT>>VH&|5BmLo%UB;xECxww3tJOXQUhB%TT>TXQb!X5rczRvmKCu50U(d+Jh|0FN+(;&FlP~F76=PG6!8Ds=3hMQ8v99lqA zOi?ve4FS?{4cZHdeOYoQWy%kvvi`CJKb~veBdnXc_EnM+Q(1qc$7h|X2Iv7MXD=l* zo8ZtdL`;Kr`to45c4Jd(cFyqvSpR7mXwL$}TPyZ~8Ss&_w-O4Ex_jofy}XHG4imi2 z2d4HhruJdLf~-kkrw0O~P&J|wV(;5z6D>(}QFi;=q_{ecIPdE=`6KOyb*c;$d9C{W zL7>N`UDP-n6>6o+mL@wbc&ZkO>v?$-cx$i2g^=Gx5@<3E=6ZMSH;;vYubou8)B9Rk z)Q%&z&2J3Y2k#4({ax>(o&n^Y9$PoAdlI=s+?1lAyU<+4=yUn6=ff4VOr+9bd9EN| zuD5Jo6nA_!kE4~Y$NLV4PQ+2afL9NsLE@V)ww=yg3g+!IsbDo)6<@$$t(k=l-^d+s#A$<7p|@K6Zo*zDG8KmS(zxY-@&kM-~; zBq=m_SaQQvU_2706Hx+fte-L!;dY@cpHNvdwHYo^;kx#F6g?T}K-t10P3MJ~6V+E- z+^P{FkS(H)GzK-X{_OTi5%ZY~UYHVdQ3N+c5A(#yBAlo!W1k>TNrP;%$B0_dt!EnW zT7EQPoF`n;MgQPj^C?~$$Azxb&DRnIIYpufET34$NClNHDnRuuY&P?G7g09N9x+FvFFnC12Dr!_W7O z@aF_!fqTAd-O#Lfg|mP;6~&|+@Cw%Pj=FdbRxC0mnihYLVl4NwDHP9SlzBp+K@xVj0 z7aI~9KQ1Xfg0^LX`zDypZK!r7D$oJ}j65DdM#Kb06i)cO^fU#T*+7ky!x@>8S+sjC zp8n=@%d5I^MT2sS9z8a+2r+0|=AA_?T&!nuwzklUq9EJvbE-7OU2z%VUgC)ew!ZQk zTmnR-hZykIon&EX*m)d{fOQ#-rGzXX42zyM5m7I6bLoBep~4kVo2gn!BMGL-Q_0I+ zg#Ml71wYByK&9P<&VI_{UVLGOc^p7S?n%TVt=)oM+pv#(ntwP7Yhw&tZ^D2?P&=Cw zZMt-7?zIp5^6XC@tYMJyDzvIfyN@H+atCsam0>{%r*x5u#xaXyBRV!j(dY!f(m(lX?7ux?#u+@)25Ci;W3H4bMj7}0nt_{E8#!%Fph3#edYM3X? zr}Be|+QxgX4ozjfu64e;6;5M|(#TAbDxSv?T(e?1Ja8W_-O&Z}d~k@k zHt07+e~t1>;3-2AudU}FT1Do%)B;6`mkd^0Q+^?>Hag5ANd2_VpyG_&**M9wmU#6P zHy+ti&p-SjObHK-EMdz@UEV2f?SNxQ;K>#rG45pcPW#@ zsG2)MGv_*bJ5Ilw6WC1ov5p2%>9c)-<;Uxh!ivK|p56HIDHiv6>5VaUmEx?D_p`Z{Qqr!w=`!}zcPzwRE z>bTc(KK#B6{g8h#UhRR4^5GvyVyCW4Z$I^pH~2*C9!G zbm_19DNyo#%6`a^5L=jtOwpd95bMy_P|ZTLI|895;{fIa92x@pb3Tx=(TljVNQL#f zs%F=nv=tQ2XbV0w?c-b{Bf)Owo_NuP-Yj81YqKl|SpwzAVwRnb;>;HW2nsm=tE zRA^fDV9O&5hOT7D#$+#jha+~yE7m-3C?o>eFl zib1j1w(Qn(wZnxZ(x*O0CB1h&ysLo7kM?Y9`j+^tX;y7~X>9AFb^+s-BF2h@?rCKi z_NNV?jUC5qnUWv6O)p~9{}P?)!Tp?MYEYyh*J!Rfx>uZD-rcJU5NULYy5hDxgz9pK zzKE~Jz^M5}DcE|_B`%S>=4K2clw z835~WE*8c@4l82jKyaA#vNb+*c#7kP;7VJewOmtUVKCO$1TQ!u`fUKll2WbaYx=Z# z<5clkR4cJFB=MGxVADbu8Eyj#kG|s$K-W|0qTKd{L74L4^@n(_EOf9g_TKO9P}DR3 znEc(~sV|}>*{~xFJE=qYI0q0#JaD_iho*EyQwO(O0tBqvWmCqK!M0M8iP?}SX}`cW zh@!z^`;C~b7(VB#Q#0*eq&CYV-!2d)b)n>*Az41c%C+iZQZHqQ*!zM-88t;20gfZ{ z?OiJkigz>2iEcGn1Vtm#M4(j?TvV4d&^c4Yq>OOX)J4eX&Wqs(^4+FcVB~;6bQ{=J zAt|{7M8$`yhp!U^T|JtAglsIc1$JToikQ`SmSsX8kA3aq^Z#p{e)`1nPYICRfrm(% zT#kke_o4P^F@XIA06(ge)Ou_e##ZPnni3}Ni?^aiiLSR|os%|8JFeK8Vb}?a*^=22 z3#(5;>gVw```{=$jd^%9u)>chX1k6o=! zTNWigo|x`UmvNb7yl2~&t(eb>U4?FU_!efW7@vFP1P|@LTT>HFls7vK$9xI>rw^4`Y_0jIO)?0Ra6Arz9y($ea5Pph6+=hQlmCK$ zcJy^~_Z6P2`du7{G+p5RZ+~zLLKlYfUuw_)gF^d{|KSlH?QbY*a*hfjGyrDK7DKa4 zaE4k_?F_Oek6{o6)kaWIQBt}TIJ9CyD%Dg!;kx*NK!_|7Mfocvx;NVd5%)_xtY^MB z9E|V%Bf!Eh<3@9+a^O29a8l#4MX5xwMcuO&Igr3dX?;aWVUH1>`~l+uX0gty&@TuV zjF}{zVE-#9SOist7QiB%NMnbUm7FfN6Ij#PWBVka2Q!p>Z~6u)^#otI2YVJUW7w<5{FRuK|3`}D=2pkXQT;dDzK-#Ikc5+KSQDyTM?DZFA~bID}P z1}HF>CuRe?Zn~ypt$;13y`J+C0v%RPzOTag|fZ5O5F643=Uh zeqIgF_z1t^mTll24s9Q84X^d z2720DFrf@wV=c2|9JuyibAO`}I|bTs6QPmd+Cd#%;BXgXLXiOqET>{AxtKx$-imtb zlM)uq?1_tweFOP!zdE}3dmMwcbYIju-0_rI)wxG+Bee#*}a7!-g z+;O|PBRvd|4nygyNP^q}7Zy-SrBAIO7d|+qCO6uDL-=?o`=f`G{{L8#U!yzHCC`3VS zzYtkJB(0#0Ui%x$Z^V;9Sp!PxG`!qoXYhaILLR{nr7cXRpev%I4|^k2{9Q}PTd0t+Ba5A6`kSJWSGF>%X})SRcpTwmXf zOj5a~2`bI-o4f)X%If$9!dQ%fDGQ5Ku3P%|O0&6DR<<;d?gKqOtc1DKow%l#9zf(h z%Jce#Dq^szw^+(3Xo{A z5>Moaji~Sk*1@bX4(2dP55^2h!%g)WJUPP^U;V_CWy$D;Wy;tuc7i*^qG#lpx+3nH zxWWTQFlifZm;jS8bxqw9@wB&#FSjF@UO#1rZ;PcwZ5KaD!|{ipgBUQwpRD0--2q%c z2;EJ?h;Bn_VE6S60o;4P(-%U*#S|u&xhwge(V_$0&y6w@rO+ml5+~uYQV3j0rh8lv z)Y(FT?vce6sliL`XZIoH*hIGyK?Dibu*hle01ZcytOoIF5$T2Ha( zfK2I{FQ4#Y;wUYb^95Y_?4ID_mtC4{vqhvuYrwv3rhPrC{GyGi%4bD(czpQK1v69F zw$*CsxH=#mrE0MpQ$mKN$La~bi!{}d>|;xs$0)t@>fWZ>z3jw#ZJ6Jv&=A#^DSH`7 zDXnm}dgq82#kAZ94Dz`$mq>t^KXGgOfEzW2m0~(GRS11IZV3920^KU}K^axTEW#h= zAL`mw4>%ighgGrb-)W7yE=P`+oFD3E&fS=066H2ke947>fAx$iMqpLU{yI15mzd83 zxndJ-8mlyS!-AxM zOqrpJ$)s{l9p)m>jZy(k+h-!l#mF~`ra?P0@wg7AHzy@;y{KxHd$;~6WQrf#C6rDQ z+SHF>6w5@Dxa4o#BVBWP#c7(brMZl-qw=uOyhu-FFgD^fScGhkfQrt)bAV0F!(+2~ zNW8_r!7xTNOo@K22>}79;Ua7FoiJ2qTR9uKH-nKGHKQBN-g!ShCe6?jdyd)aPVRaz zCz54d8Z+HXZPBu;usO{#To)QyQ;&IU8`8#`<~&%|r^`xSH<2T6ZRc~=VA8Pbm@^#` z4Xdu4jkkG?U!5esb+x*cP;$FvUR}_9cG;X|lG%{ObS1wou`L2{Mp@c9zo1dD`1aIE zJ2qO5K<9Yes0NZlwjF~ZfI6|f(T=cnHwWZErzn?c>C9f*7WuNv7a2;sYOCMw^JnP` zY`UfLjFPaw(Y54fP0L(tDqGNkb5S8~o(j+M1Zz*<@iixIdiYt+-)ep3m3Z~s9#!h% zxg2xq>ew5b>*@e(jUD<(FpyPs>{L%Zo!0;PUB_Ved}E@vn2Vz4FJwokm#+dZ@h6N{ z|CEQ%bwZ~icbYc4#-CJabU)oZ8=||_JF})wMN*g-T{1Cn9y>1!v5-7)UNbKXwvaqv zo-0omrtktvk$~{zSl&`9rY)-kL$Gu$a2{Ux&6Ki{PTU4Sc+#9CwjnsLW)LFp22pAw z+=+7H;ug~uUt%WM+4QyaT$mTy>=*+HL54jPqB*Y+q}Fdg*>TkK%^h8##} z+#XlYs{Tj7H3XpidxbMtiQbRhD;|Vl#TBcgOpt&DL@zEryr^gB__36~oo@`WjsUX) zGXO6HJjMj}ZDyyoFe0HH>YDRMa9j9?v&v3h9QT!_@@?r_zT;zJT-^eQR6SI$1=VJ-Q2k44l6jH57gyp>v!PshN0H@6?q9zyFYeEY+tqi}I*#KD!V^ zz5XHyR=DOS&r_eQr{Q>~rz4(+DaEqpF-0+a9)6H>aF#Gl)(;ellKClmU6c1h-jw?Q zo@iOX^H=<&!V}_ElhP(AjikkIu?dALBBaI^GRWME_|1%rDBP9>KfWYGaXBJS)a6L9IKs0 z&SDGGi1AySQpSBH{oheTodK#d1L{)Ur<4N{6Ras~M>iIQd3@4(L2h%54+fCv4S1-| zlQ+(OMMuUwA!r_+!QK@;YVUue1Jk>H$7OxYG&3Uo?@I8$efdbqtjNgdAAH#Vpw$>~ zc@auQbg5y8DIOryPuYW0N@9ZwRHe1x37UsA5u?{k+n3M)iI~B{sQ2;Ye<0t(9imV~ zbm?1DJ3S6kTRmKT|NOaw_yOTYhenG=mq1Iaq25$uxE&QN`BiAxgfX%L9!T>YrNx>g z%mL^4$=IN;m(-Rk1t7BNAK?Mn%yga;S&qO_gbCC}$124tf*Cl8I)c8S=_-Aj?wIHM zTuu@5Bgp1Y!R1KQwTifSoxFE1qYg(hrcgvM>bO}Y-f`-}&I%GpICu533C@nn#J#?r zE|Q0uvEu;3_b46muGnfQWZtNe4r{Q?m(ps+R_X+34uLkJ258TM0$)-vf7KZwDD7Ie zMG3>+n_M)jIi;S>(tb2@Ez?*tXXF5L-Y-ED(sbtP#V>3sT$vC5ea0M_O#%rz>o%u@ z36K%d6!#%E5IFvhfw$e6w0?Q+AZIbscvhA}@hoar{XL}|HA0=e&uebFFJIi2QB}uM z5B$&C2UP2Q3h=vTfi1yiF@HLuUFB$#UOv!(e6jCmqdnwYn{Cc&pqJ5bmTGveJJ((t zvGR5~Wn>sY?5qu4bICxHFD`ylyTiMoPr}$9&m(+%W{J|^8uVb4?h_?4jw|$N9yqsm zD79}VzL-T%pznO@d%`__6>>*-ss70}_Ghgm!c zOO0pJv2Zx4HR>>UzZ3=its%ye8V=(4As%sTe$XOojbl4_auaXwcVz}TklX&(zIh}I zSz|*qR)<5QI`?pK01<5}WO)!APT2g829sQ{$Nmk|*&h-s-p>F z@e*(3IwLBq^B7^Y&O9eKYynJSILJ((aXBV8mW`lG0{ZiZ%t~ggTiuEQN@A#L!-g%A zi(KdJy-}I#-r8Yn%N0$cK?~xSaPgNV-JBvHmFo2R%a`nV=zysx5Q0QB>om#Aplh%j z4G-ppPcT2vz`$;IMz=&8(CVLZV43kItl^^+nE$ta_hBAqPp*pln8&hF!%;!baO-)`%QTm}$+{}KJmS`H$?$(nzg z^A8Fjc&Ih~$DEM0glrbKn`>5TD@%PV;QeJN`;+LE5$>-+h46gbAyM3PxD11zy@oYd z)NlxaV_Qt#Lt8A$SYH6ZOROd=S4@1y*O(uWqNw6KX?~~BCGPD0eAnuNq&zzs*nH7g!3H_NM2zFp0B3j7wMSF(6 z@F~;;rNh9eBVV-!FgVEds(tAiYzVMpEZ*X~pk%y?4IsXOk(LHfsSg#y2#!-P_jLrR zufl^}NF9yA?XgGF2{pqPS1_Qlw_r(&w+Nq+nQv!#$24s6`{fM&7FAdRDxBk zO!dk7RG^btDBf6IGA&J8ZumVNy$i#-T0^*t>v$@gBT`>o8!?YWmf-KGmNe=|bL~O} zX|a71Ut(wOMM(e(iJTy-yJobfB&944>I28474L-b7*4Wi!HSH@yTo&@SkUq@Zgm#n zgPAH0x7V19jV;F=A)m!+k`V;Ec@O!i;97##4oU+#I&RdfbXX7?@t{^0C?jHt;?cPx*J|r(c3c+RY9{CRsay(8?1Jx{3Ts0r7JmGLMkSC zeVYBbf~>E3odB7UI(OrW$14OuRMvVmjo1nr`qty2(gYt#lPoS(C9~B1GxE;jqO9vi zWlVDx>O%f{_UVDpnyxcl$mVpaO}rPeK)5lEh%9pIDTI}R;%=4RBK%a&G#BR*FE_20 ztjc2^=m?Og(5bs3(#$-;dy&v$bTOVt)yn4<@zSNTvtnqzQdlv?n_PDurb=V~mRW(d z5aOv&er?GoHRAq4x*`u9(?UVdRj0+o^NTBrm+evD9<2lMB9jFO^9=nWHb|GTT zYQ97*^0qdZfXiQH`}c$)+Z8;Exi||Jmx!C!3NL`3X^r_ThM_`&*8ZYUNwE4- zD!X_ND6~2)5iK<>XCX=AP4m9 zlFFABN586o!GE%>1X5c`Vd$-5-Jbcks(EC@w|C;^2iyOaa8&p*G?>uc`m~={~ZPwLJwu2lN@A- zFDSK_w}60cyJ~hdVCo;Af8ej|}P{3&)8&a^$K2Ynoyk+D!; zRA8Y8vuG&C2r$|2v#w~r)GxHow4Gz@6n^fkMQOHko8W{k#>EzcgIP` zb~?6g+qUhFom6bwwr$(C&5o0flP~xEzR$P!*v~tYh3GE^FQZr&YR%0F~ZzO zdivRAFiw%%FLW#G@?BJBS80P^`n#^#KGhip_?<{FrxZ3(BLYk1Y@)ySfoR&Ra_GuHFCDA-{`QHi0uhcxa!|#Jh`u~3Y0Q>Q;V)q|!BzTJaFL*RSk-EFS(n;J`RP1AR z_Ad-jLPUbFc=o|$YZN>BXnd$)e!KkueR@<a9Df zp8PEtWq0Ym6lHh8{+G(T;7|pn0EyyQ#XM3Hma=BKVuK`?%DKXsGbOIPxSDyLAkEJv z%0|`ij#W0wM%lc_pG_1_34+)tmTKk&f?BBOisl)D8>s7myxGg2kCclg@;!pCR8BF1 zPn1sb1VVWRLGeD;!`G1RC9$s1FO9)0Zqfs=sk5`h1*V-6qn<3nH!cL%2zQ0CC4pYz zqYHw(w1*r7Ir9ifHVWK?hipNeBH|iC(0Gfp;VqV}h}Y&5|G8WQR3@@A9KNXN@>t6$^l$?_IJ2icXqgIj2sTpAsb#CSs5~ z^A@N}7fxWG3g38(g)_=`?UIUH)U5tzrmSj2hpO0%8+cBZ@l%iYDp{)=oh(}fFq!9c zS;Y6KSimyHw^g`IsF>|wf^+EBD2sK^jMHWqaOwo@leDjEm-8F#vz#fJ)m>K>X(DWcYF5{ZQkLF zr!rA8=>OTnmjSJ1N}yIBZ%SZ&57k%hOKm(X0=#ce>k9S-(+ld2$%$4J7x-ZqPHZsoGp9MR*j&#`gQPeL`|&$~fwN)52I zydtIkZF$9E9f@5R8HXF?Y!WGGfGp;N)&h90O{1PRf=8iYjbEd}bty}o9&C>&HnBr+ z&x=F|E|54FUpC&5jP!bvR&LXXDX_#)KYPv)_U8LEGjBJvAs#NuEy%zac%pZNqZ3rX3^hP zV{DLcT$r@eJT7&oUbfn4H@C4!2QZ=28#YutcS>PWp&kiZb}XbNcJZ7xdjoE`-#;9m zRph`-tu~)+1uQgT^{2Xq?;|$mF7|Nr>y{%zRs1_A$QfP0n6mHtO z)p?b$WXz!J98HLtJJMrMoDVzcU12vj`MKUd{L5e7ZNr>GCi;Pm00aHa^(wB^;tqZz zynl+VWh)anCL*Jb6iJ57IRU5_O=2Ats5M;90xyXg;+kjNdh(SEl*M8&E0G35;W8;+ ze})ts7*GSd?xUSWl9`;ioJOaz=2j|Q^T@p46y-do47v~B@Hl&dT+j_ve)e5i$}ted zaiJe;t8DY=u`*B5wf*H9V|fx($=fp7*?d}fEW)#FA01#nDdXBfJ_B4$KgtN*!#h)S z$W%Mfa@#s+7H)*y^S`3g;NeN3m-Eb?v%4~hOOP^ofJU76n{8BW1YncC)ux7>Md zBuS-h&N&@PrX|u?r&%{}*w)7fXFDzbp^AddSjL0G{XVh9yI=D^29HjzNt4y)j4qj( z+!r*dUe{89urZNLSpi165p|m^1)M(4ot8qfWNvI?_r01|zu{IF#qC?pneN<~tl)n- zNNrk^IvTlev<{7>bvsO?GKNSMY{uUQ_w!z1xuds`2XBd{kGQeHC#Q~tCmXOudy!D@ z<1)-+UMU}yj9{?3SK8%doKN16kH%SATkX_#+iVRRAG}7p6aZ`vMB(Dcy|f^ns<{(a zVN%@(Y>C%MOdr#O{29hoDY?%MhO<0%@i?L#&D~k?#~4S~3^sZKIXpo*l4|RP3RJ5T z%}_%k1%GW!-AXu;vvw6$#r&#CSCSCxUzhT9X$|Q}REt8;c;@Pp>#s@Nc?V1}nB~x@N=0AXsYV-cqw#u# zDviFJX5PZu0H7_c6^6yrlc?;CEejpkU|;)lvWSfOJzk2P*k>wuz@#dq<$cG z&mu4V-dRWw^0vDPLYse*GMxY`fuS;)X%w_b{UA>ck3T*kWwmXcU(NiXs zfqi7{#Q-?*Ey+t{DVMMX;51-taAZ=|ZqK?ESu-~`rCk!S!SAfL_Gzbdj;*ftsp7JL zE}pm1J>*`HQ2=bR57c%Wr#^>ksA+TF9Bx$mlhD2FO>tE;S|V_E206O3xS}#+`X!mn zDOc1g$kl_V%)xg>_&*Z{3w+-XMnAWt%0D{Ny#Vi8F(xR3D*}z7W@`SwOg^h3ys)&$ z-1(J97R{JY(R6}-cZ9mMggl&h_sEX7Cm-I)0o}w^127OR_L40;lcE)|& zj~0(&4kDM>kt0wN;lfsC4rBA|8eoR#B%0rh^39e@N|qY2^lq8WriBuFtPKk9Otjhn zM*y_gAx#eD^kpt4T1|kdwDkIEOb*A{8H3(*1rI?Vm`=qTv-?jYqQAM5ZGTQ!>joM6 zJnW;PIA^$@bIPGE)}S@CP{XnZm&GR}oJ2I`3_XbtdBSYslukbnJ9FI+dm@%TRu=cU zq?8*18iKnzfYRT&Y6!c{*?u*{+;HQhfdfvY^oTj5Ah=I0()zpXe={G_6tK&SbZo@yZekv_xNe@2ZtHd>HC>C~s(Ccf=aFWjxdh zg^0A9xbWLPT-0HWYO3-Bh>_3o?NB5HoKYGzZu z&NJlurXKLXcP|*>@s`=glOB{eaQz7+tL+c_ZV`fZ3Pju3qd16Wv_)6w;@n|4C`T`L zI5ebmT_(1OSMW&h7UR<|ysIgJl~H;3fk7-*0QZE(nhnQaHe@kuMPoCBAOlRvZU(mG zf?W4cx#48oz@T?6v_?@aqP;_AJu@`N|I$I}XKjb$D#dJ3f@rs(x;hMB5OU2+0iP4I zcu<1sI7QqP`8{ImfdcY?@^^j#qj1LPm-jEIX4_DoP!K$@C%5H2-R8SoGZ2^=9~e(O z!T5VCWFOTrx|l>A+PcKDLp4ARmHLcMc3}>G(=|0-M%d!=x(Qtt*y1vlY*!U(;Zo3f zx%dWU!OwXl`(I{%vXCJIy&?p8fwaeE>tb)-Hj-l54xuSof8sV~nP$7<#RBW{rjdG@ zZF@NZsavLbU(+`z4ItKE>irJOjC6oj4p{S5|C8EDNsGZ0-u zOdk6VpkqjB{1Jr%za!t}?F2%@+ne)zaen?Wd`lg@&w*nk^K=m5#$yHep=VH*v%m5} z;!b1*K-LL@z|{C%Jm3cDbNp_G3X6>aja9H$mTbO@Rb8uuVVl+8*Sc~mlazTgO|GLG zH}O53{u51fh@96>@#h5aJXxyYOa}?)6|tU_+n)gMK(V`(*PdISJ)xi-CC=c{wJ7kV z5h&QrWG-h{7yGCru3CdTW#8jCy|e_nh2FYJ%g%IVeMk@_+xg1=3;Avh!yULH)5e+b zPsJMAor*}dm?x^}id?-(#f!rtpgv~675d>dG+rP+7N#%{{^ zM)qq!+u=oS*nCCTyLb1p*1Po!F|J0C*L{#6MWeFZy(Xsu;FCtr2v+V$*CR@9X)Aht zA#Hu`!CKKVH%uSG3ReTXvQ#%v!IKS*$TM@^srw7E^X%gf?8RqLnR^MC%M>YaFCCO)cPMo+N++C5>cr3En{xk(*8E*8 zQ&esAV{dJwV;eANH<`YCyA*lkV_Y!YMa=o8G_--Vb?LY2Hly?AGwaieZZi&SN0?2mpkRlM?}oD4rK4?|1}o(D_(A={M!>_Gh+_yB%w0LQ z8&9`>SXB^L#<4yVXQSPSYr!SVOZFLa4~%q1t9>0DxeJVphPUGW8ut19mh7(U^6z%P zgFsv8kWy;_+sv`1`7?yZI?pruMtJ32crtB8w= zel4iXjYH`K-jTZ^keI{-7iq5DEa%{{ErFGlj_QjGDz6P(^~JufvuD9!k@OlKn;q97-LBsmtX72~6reB&@tV&e#gcWUhz0_@(@ z*S~&Wzl62skZ(z5?!RHP_P@R{;JLJLuqnHO-^K1ls#f1)1C%WxAyQ1zz~U+s%U{c~ z7Lc<)Y`mqdDmoWgiIvIIm)B+cW8q{?_lW~`D|pL4!2y1HRM7E7l|^2TG=PYnz9SA~ zbArfR7Za16wv)_`YmT=~_xIZsx*lMR{wG3hL={99#O%-*5$Z5?wA-*yIT0_?eJMn} zuxEhgAR8fG*t6=Ot3u$dB8q}JOiDrQjROj<8%a3EG*cvFB$Ko_0zG*itOXPVhH&Tf z4fsH4AY?GHA!c%@2|r=78m4Xn{sH9xOE3deRx3V1bOchnT^Yz_E8&5R^q3p8BjXkR zK&(mY*bUqO^*{%VJ0`eOFtCuqFt_cfn=inVQ92mx(AX;!6BBJa+2EiV24clT*-mO$ zEH$w%^o@~sq%o52`JCViI(8}Fh{FXLEKp^n0<+q;+W5DrfZpOlaY8md%|X5IfEKA# zi`1F6K_B_DOInFlwMELP7ycU8#eofwSv*&p%ZQ`3oCBNpZgdJTNQKDO{DN|bh$CR3 zZo#ze&(edxNg!BU~u3S1*A_&{!#>pIi_iI-wZW=3a+=|3)}Y)=eJQ| z>zGfv4ZNQ%nN0wq;jP2Zz^~2fLjB7A5axCPzHQ}lHAnVgOXLH{C`-3i+nM{O{tZju$p#Ear$yEHwIZpK}*X4Qm z4|_&77b>Rec+d0}3?`lRUi*{=k2J;PV%!-TEb4aOW=|oQ0pRsV$stWNyIMf5T2wL# z!}e}i)FOeV6#t-l{;<6J!#g=93y?y8CYP;>MU=Vf$V3vB$i&8*v*@eb(qS5*T zF}5BhmJj-A+%jdEFLL)-MH7tTW&LctRo zF5@f7bCFaIJd_Xe>%qX3{oJpOcH8{O3zQFD{J_)4ZINX@rI$&{=)Zs}!MrbqS68b` zP4bK!9C<-!;wC#j9dibmWzJ=S!=z|V57nDY!WyD@HAC3U>H}+#8LO=Bjs!2~QtQwf zB1v$P?cRv=Q@KGE97A?#(tcRYJP~FKFfo*}XeP}@Jt+9+K>|WuF*DsT)>*BQHs`_n z4@vPb=GghePEm`_8xTM~mH2yTAfcls89)?(&C0Lk{8D!*sUAizpod$+YIno1M70aj zPaoB5)X(KsQwCW*eC4N$*WDH7pRvKnH`?O=b-||)wZM}9XFzSEASL%5qyJPL6r}MFy@s$11%g+N z1~>F44}?Evz8fv)RixdyO9LHGeC*m*NEe|M^TXZTj&rthI1aLGC!;h!i zMErB|Wf|l#lSj)^(;n|Br~zLrll;(Tp7Ikw;jg}C-?n>~s9(*^F{#>xMkK!6rXQ7S z#MC2U!)9FB@Xeq&-ha=Z>#SU)0p7X;-Z5ao8y~i55h3Bini51P+N$1AbKa0vE|k<#;wT+sq^! zyMg;p?I_cV1(D$a$5E9JnV%mlNiD?_h7v^;3StH!5y9aH&=*%PsiKk2XO9aXxuwG( zViSI1z1CLlZ;G=k*q@cm{~g5R@e1I59%(PXgIVtW*JuhYZdQ(F`>o(p|MzIw0|-{) zyC>q`qCXmHS3Je-J~jl|!Xdjev(T{g*jQy@Xs?})YU44NCx;fXSv?X_=G%ZhCcbg@ z6;@Hh@8^>3iF@{1+T`Z@cSQt0^bAL~U8P;6eWha{&I~&%ccj5Q$Vg($&#Sjm0!aE8 z4_`5LxZ3cuA98hr@>|xK^wc)ZGlPJ((8Q>}GMcT?6NRUjiL=89^Zl8k*ZWLqty#lg z1i?oHb#{_sA!#xu$BC0~5~ME3Rgn&Kn8U6id`)c4RG7u<f5SynfG-{^XUe4!%iN%EqNMJGU*j!i$=P-qcp1a6bBjEq9bE%VB2|{ZmR20cW(nKP2 zE0CBQ6cmWu?=w^y_z(2NpAe4lz0zE|NH?+ysf+pQb;wPPIEubay&kGe}DXj z$LMfB*sTn-hrHq8xvf6TmK6kp{etm}`HAK|x#!`Fh6S{44&EBF0mNGSy{uQ=M&>pE z`1TO9eeE0J(^U8@70#=ace+e;Ns^z0lUb;7T;xuNiF1i_bWY~OWTv?k zqK=s%(ILa^yj>6j>pi6li0h{dg09)}K#QlIXE%!}6FEisrTNhr<_K*f4Pcsjbl4r_ zHuo4-#F?}<3hv5dbbnpC^FIF|cm7461W*sD6KD~^la4jp88w>){8b)(>&lwB@zC4T z)6Ul`(g1Vy`Ii>Nry|eR^S#K$|GmgRmHtBuf(C|zOHt)PLc}mAhq@Z55@{lO z??^&-#fCoUrIVlafw5r41%X5TQcpJ;VVuQX*QL!y_iN%ohs^V1{y|~T#O5OYGN1S4 zG&|nT_B{Ds^&@OwRy>c^6M`H1lRdwX_V2iYDp(grb?<=lFPH#cNsK!M(J3)_@CuAR zYg8qSs(spFKzVCS8YLEn*(t1|S&;$x3U&?V@VQmgqqjZ28==!=OO*nrF7-3tYRX=B z#VaBziVVXH{P#{!;pvkNUK zB$8BbiHWpX0BtQJvB|+c3JW%Kr|#WC!0Izh(?kfkU;rIT*QG*EKmeQ_tnr1R^uy8Z;s~P3lL+VI!Q8gxgtX`bk6Sc;)EFs$6m?w`b zeKOku%w)$6TU2ny^T9h>2^*_rMgJ6d5ZgJUzq~56O1HeBnhw}oLuI0UUYG-eKD3CD zmVA=@U1dl4(i!=altFS*MiX{c_qMix5U#5qS32`I%jlU&|DNduQ@LEU0*C0%1K9(B zeEI-wNLO%|W{6^wT`r@f5oyou0^+sDzl+0<{;O;4HB|B*6htiLaGR9c;Fay4tG@Fb zxftB{1`7SZ=ePeezV+3D#{#GtSUa2ihpn!(A&aGeqEm3%%l_!ErfE|rdsL26s-I+e zL1Q7<7^W(;1J1s`Y7h*^G^lqA{aQ{pyI7b~Li$l!{D!}XX0a}@YWCOgx@no$Z2IZf zcYl4T(MXgA8(zolV3Z1?oCr5^9qE2hC@f+IVY9Hc)ad{UJYWIo)n3nkR0;9P%?-Z~ zHU|{`9f^1|iQX3zim}+3T5ykGR7iao32#i?KvxtC2$L#wRDC2f^2Cjt9}*G+I=H^X zQYPxGTp|4ONKT_q_cHLwMv6%i^a)%j1emdJgC@_qDfis6QvaOuDBWUkJi6-GdHHP7Cg+YE;F#U#avm>**#_wGW3&5)RFeL^6m>k!kP>T;5&3k?DVa zb@Qy#0ya8DM-60_aIq}zD$`qx+!q>+uu+<4epU{hcfEu(E3W)Z|A6@BTZ2OU?o~rm zGVjFRjh!HOx;1H^ez%_+((wfbVi*^-IErxtYx5hMFx~HC%&+pNu{_OV!J0Pd>359w z8%1|J$LMJF=W+#OSm~{8X^D=b|2V&4rGHI32Y?Zi|7ANP8cY=^9dZ*EZB2TtU1{1M z_YQD-z~jxSi0*TY)D{>Q$%l4^VWIb#JHEEK^$fC#gHJt0(ZJ7$=s8Z$ODgc*1HaOC zNereSwYU^`C;MmLPo|I#4t(!>jQ^^v_}^UDzhAg1aFZ|)DZZHS;3=2paIpWYcO|L4 z0FYM@zi3Bwt-6|r4D^u&d#(9LG1l1ismUz*F!qJ@{rkei7hJk@HzG)?+B*~pY!--R z)^TQu*xEdaEfVnHV3DG*r?bztjz5j=&LUNtNjy;#^t!LR(%c<8)7;sR(>)#7q08kSuRaf)#@8!h|$4>JGJ-yyMJ_J%e2(+;YDX;vW!p z6C5DnCEKULegMTxJX2Kw&_|5lqs#y&xL>^zKnVC1?MaREct0V7*oI;(IcO|3=Xgx# zPZ4Xq(!UF#jpqFJ3WBeuiQwCt)9{Gy$}Jq0SrXA!=>esoBW_W!@m()9(H_r|3CNr{vy&HB{P5P*rfOIW==f$66;vd@{kGbZBIl9OuT zgx>&_Gou}}zZP|1F_B$bn6xcfhE&BP7EpAv>(q@6+NZgD>SLJE4c7!bhO+p@Yko@Hz=H?g|_?C)|NVpG=DytA* z@cfx}>fGSvu>;{>tXql63QWyuN*feDE@J%hzt|q2$TA4$JiMDFK$+3hFsE>@xsKQW z1lkrcs;-N748oqG`1^9=QFEyzw$O}V<^#G=5*P?~DGg!jzkF=LM_^ogucSCpfU zaXsK>=~#wrZ!OJjZAtKY1U1PyKHlyKz*qG zh&bbsoiR$M*+-{|>Ud_5F@lwt-AKZ#nwhuciF)uC@jj%f94{b?86|!edE+*)iF0;V z7}_d=1uruN*)Sg%@8y{>G$KaqVeV2#4Lhv6r*_HmL!v;s<_SouO0g)HZU*S`%1L6) z23IkWs>bs5S~8;M15%eBKoAMnE$?nwxOdY~NX=)Eu%1M@$$oQ29dj?E-3&pi8fBHJQJ&??qrS3LMn?Ia!L;a*CSlJb(}_>V0}7|PdIq#mlp8SB?YNO8v$+nA7#1b66x4FUtoP+~F;=7{ z@(Jct`j0P|Y^?Ifel&?kF5~xT<38BSzE*9>3|PMlZ&mZ59~z#W#+uL^5+~%t8?*Du zsdmfRP7HG&=X@X}n z3^U-yWV*1O&tydy*H_-fxT@ANPhqj7s6_LO%4a|=PCdB(4X*rNJIjAq5}BTqPe+fp zQ2eke&_-y(G}7v+jq-K$NkV8m?uD^4N-E^JzlMN=8I*g;UojLN@5Nx49Iq&Pc4bFZ zcK>()O8VKT?g0eBYuAIpbBbWXoz@=1xA~*&wkV)b!?RuIA&L?W!?8LahzII zM9zc(>W)N2O(V`qr$@yzLesM+k|ZwsOme3nS3!e#uN2qe(1`1ez3bas>`rVp!Cpk+ zO?1u1f-^Vw_c!5r4;y3yy>D zE)28M53UxQYRIT0T*w7hx%4AAKt71+sXUC1(Z!FiTL^X__~_>^y*?mn2!`T@JOW0((#Pk9G`H2 z0QggNeJ4?~hpAsjt=RolzMX%v8`ie2H@5z~+&_J1L000F=nM2!{mS%Z`xe+O?OCHS zIMto%Pfh10_HiP3(a@0*%;j>y&5+oG?xB8O7y#yG_7(lL+tsz@p@fVW3isheRit;5 z!{1u-_JN$xo|L%*^9TbrDC=9x4CDEv4YY@tbBfoF7^=(N{0009lUYCfkJLqNNou#| zQ~BxAmfyYf4|Uk+k2ek1;xyXyQ~r$ZB_{izj-qHrv=& z&I7oJ4%EAUx2r067@ZmY0UppC%-`jVrlyZIprcCxp}H0yX#CLsLoX}qdC!**aly&K zx8$T-0lWr8GuE35uth3WxO9*n#G7hVO*(A7Lhx(1H{~Oc&#;eBC& z)tqYFmvL+#n)F7}!^hk<-pbF+I4sOK!3<#{MGt0^Hs0fzj(x%j+ zLt^DdGsJ^)F~)6gU=%G}S}EC}{W12~i{XQ6QO*eggFpivE8#G`FNsJS`9iGK)cyr> zC3v7(S|&yt{p_fmfJd_WvFVyma0Q^3(c?m^^NPMWC$h$G1!m9F2q?|<*=55$xB%$^ zC}xYo1Qq=*XjF=5g>?=@Au^AO2ia8$5vPRcKq)ATct->yJc`9L6`A=Soxbu4F8iB+r8eyNoL0Okp)=Y_ zVUd2w?n<_+0$8O-Q$VmsRUlwnLCa6jI)nGud3~=s`)9}h4AaPqd0R-o@5kouFl$IY5D=W)%-AztkPGWjq`FMW;>4QWulHyWxx}EJNf$a3v2dF?i{*-jn z7Qj594NtXW2xow2QReCFWrCM-J2wDRg3%{m#x8*YOnJP;>)rC7@6tC&m~t&cZreO| zF{q&fxfiyZAFWY6LVOO~C<-?(J+BPWvx!euMg}~IlJ%=o%;Sl>Q-v4ByJX}wvb0A2 zIcMtIy;O9h+Ac(l;TUk!qO^~77BYx-$5873W2bJpdbI>fs-)^7Qd>|nv2&0 znTcTajYt~ygGB;JDG{lle5P}A#!Mpv|D`{T%nb%5R3SKntb8{At-*3TFy)Mlu7rreeX?1_HvnX z?epX2rz+7mMxipaJd^|RI>HTq9|UoNP*vEAd_RGFzqKDQrD6--hPi({gwalqBlIK1 z2$JxZ5ah=u_soRd8DZTHfv@}l?y9qN zZ!vzmeswco!=o+l8kY#;veIbXjdc14Seu2sE^@Ca9#Q`C{4?uii%JH^xD*n6_inc8 zPxzxZ+C`pbeRk6*84Iu9e#{Zowq?u7u2ZMwHg4LGp^M7icS>SaO0CU9mK(_nxf3H+ zRhOjz*1FZ z3;%-8tc70bAw(4dN!IKk(Xa2K-e>xo$X|(~o$$(kJAQmfyUFLppqv$*?gZ_lKUcJJ z-!tt)JbNFK4W^id@sY4gE|L?}TaZYrDhk(+VIy*s3LnKHB@9SvqoJ1Jiyfunq>zvl z6ecCcDsm9B)=qpM1NC(f8I4PY?n^Ai?2cX%%^be%qHphlGN5>XSc{ z&+aow^bBQY^5O5vr4+uEc>QI2fiVW-V?B{$X0bW=kUU!< zY*UHLrQvOVy*K>iS?LkeZ~MX+l)6SX1{`KNL{1vDv~8L?Wjy7Z_P7!@ZJ@r-%EvKZ zb>{E`Xl$ADS|Bk<0Y&oMg5~rMfx-q4kZ1ll-2Td=LD8|)+yXOdydBavul6T;|2&Xv zXc}t_zsIH||7UCp@y)pYH(K+LHk$$`3=96hid2@SJO6j@5&#~})D=S+ZFV=#Ew{R3 zll;_BT7D6O14X`Jy$~}QNHjUi89v`4qrk$kQ6{4xrE7tx*d`te+l8SqtZSZ?XQ9|) zfrz#1y5lVCZ&#YZT2>a=pI`5$$KKbk+fUnX-SBSbMI(3+zQg{4WV8;3X~-dfEVnM? zoj-$One0b{Ge6x5eA%Gf8hmTeZis+=lzSn-yu^D$z}{1ROTgX>eRGh%{aSoc&m=%S)Vpe+@8!OF$Tt_jyfnLPpzp=Le26zCz@O5+IndA0Kmdx}8qoJ9-yX!9 z7T{0W-W}*?2%sOTDb+V8m_;p>p2>A$1S{5LOE#8&Qjc zhd34$6Iz=-q&lDmHA~Tt_3Q_^WG5QfbQ38q>yy0)4Wno3rW*Ko!+^+rgEAl|y!rz^ zVmIhp9-F!03f^IQ*DA#Hj{8T8Gk9hlc*5`*zM%@%V|tI05Eh3dkN$GpJ{^i=-(55 zKx2kuYQsQ_m2nI^+m5|h8bn60B?Y5BE zF}$blv60w8{Uou&%ryJ{(`m{5E^o}BwptbX5w*u^qx%MSjR2))<_4>ib`(Nf1G2%g z!QL=$m>!s~7_XRc^f<=slXigyz6Rp`$QTBpjDr}%nAJw?w0l%`b%8bglu*C>tVq0I zb&2<^=BW0n+@b1|~47lW&~eiUMm!CWyU2l9>XDJCP5N zx)D8zy^vWY-2mttt=JdDoe(J!Zgk&2t!fqQZ`pHzlG~?*+VR6h>Q3NsL%&M8*-|Ii z^Q=|ue=-aRydJ(IS7lCZ4x%(cAV3zgT(6>9(c0aBX@uY#TOt&0`LNFk$9%IMDodC0YD5O@rHDe zywgtjz|$q)ysZw+((J*jFWs|~@42(OU;UY@$qAFB=PxOdO`HAAa3eVBy|0 zJ&guC0>ox;SFRQ1prb1HSj>lrD%CM=K zFyMs3jN6-ppy4ToAS8 zstY1{F!pvZmMJ)@At(*jcauq%e-E)Iqn`2ZMx}ku-@4VI(&QzjP-p+3z(CG z*9Vx!4~dtooiBy=teefy^6?jfE1iWWHFWQQ^NFHWr~83qtYbH}Rm{opflTlQ`)^uuK!tZvc03N-Pq?);6GZ-oZl5+4dV|BVpfR2LA zq^?eoicSJ=$Wrz@fy~6Rn>AZaV`x#1wuwnj2ki7DgQL9%>d!95Y8Wg-(TemAsg2B} z<&9DMo~bQcO>6e55w;C{l?}*s>$NV&hNvtuN-AY2T_nWq6Xa3>giPmE! zgC6q}w7MVaEmKSt92qtv^GMa*{9w%=qzw}44^JM)=qHt6Q{763w(%7_j@BqR=|qkU zxMLXw8f=trI2WT!**Sb6O=@8zg4!3(+1zFxZ~aShRLx}uMvJD<+nj(i0JEOxCff;O z%i3U7{>MEck5^wv3*OA0`{IUIjZ0@O709c{<{E)G4xsfZfGB;HSTFV;tueUVpzVOQjRpFnFRlUxtvn ze%3;KXNr7_4k3$^<=mbJfP)Z1EYZXHUD%sA5kqyL?zNV?cra8tf-yCF+VCxcZicA<+Ny63qe3fDU=w}m zL)Vf(he8aNh@0H$(Fx~tQ$t27pRFnhQud>Wj58AduSJ-7?C-@!0O&U;BcnmjN!J$=ZP`U)oj3>&^r&zJeL5{K%@Ft+S+2|c z!N~+>&;CKh_llu#C_vORrpD6jNR3nyOQ&kY<76HkFU(1%Jk9NQ{`@maO+tAgNMI^} zw;r=-a$5iC{ ziy|dtE&y9d)M1p#FjbXY(i>4xMW1gFWK6HD&RJgs2u?snSDH1RF~2!sC7JrDmh_at zE7r}nn0q~)Ss0K1B_kv#f-(@TLqxm$BN$=v(Xm9`#y`AY^NdFn0&?EJ>%jU%b#Q8vgf zE=gI3CSbN*BTjSV0mXqICTfaq%>tq{Q8S;o#8UHx+7x*t4bH{c&HKS&==8cNS*xEQMW z05b=vpI$&QbB-K|w>zOg3Pm-j@8}91Xzss)wo1;x>=SF%$*#zj-|QmB+L3aXg@&1*hdFTg7Z_c=cBTgB$u6rLQBth6sejgO7HSEJ~&=-*}#Vbx|W9tvHl3dPRw9TO_LAGqatoCpT zHzo>lZV(<}iGCdldCo4;=}f<9lubMHpOPTk#M)wm%sfzHO0_#|P+_z+G2PHxLlPL* z#ZG5DP)SO))Voz>GW}5$t;5&sU;yZ_Ba}p({E!`mqf~!VJh?tx33dlKv~1E6v0aZ= zD73T%bqL!2f?Yoab3X(eUxru$S#7BOj%apdJFY?g!>?-kk~LYxeG6503g~ONXI!(r z4@k1S(!Y&0<8gspIpSg#b<=J{A`vp%(k9*@t$S$Unm_5-mYV#OX3klZb^$9VzfFT` zhP5pn2^4Q=t)71u{C{k{V|Zm<*Zo;ZDn7Ao+qNpUU2!V5PsKJXwr$&XQZXvFDo*;` z_xpDD)BmSG?CUxo&$ZXyYmYVNoWGIm4b}21!a8g`f|eEhq=JRiGPyHH@7tc6>iiGP zt_+pbN>?1%6WRW=N++bmp~sRRzoj6DoAfTT0@l8$;{?BOcxfM~fe8OTd;PekVKooC1^`xL~X^%4=2l@Ub)!A<{zzTet*3HfGlG)vcYkyIhLDaxqdLLxt z;dyZ17e!H6e#nenD~Fq&)ig{25N3Nw!png=*-b9M8ZGT;Y^G&rE$U-frAW1wWT#+O z?^i&r`X33@HUu-(KP3513IuG49U&^~hU}2K;7LPbbPE07O$5lKVdVQ?r~AfLmE~Q)BQ9-}CRcPN759 z<&Ne{H0K?Dxc7O@^1g2J_p?d0_7~b40u8rxESGYAH`#1*&|~ITs^^=_1#zE!Ih1xD zsj_DcBS^gfJ1yxM)(<2KRa>HHXI4&CjdQkJd(Gm4AOih5KQSwZOG@?DJ^qaOmFLN zx95d_bgU^t?wKn8u(QT^R3d9jT@;D|Q++qqVTYr(xm$j+Jmgw=(f2DcyvAzq_fkNS zL6$+>J@fSPDgtwXq6Ma9_6o+(2 zk(UjHa*1)SE=*p^+_U&bKG}!Wi%_TI(8yAe*B$HDq`zBPy{~U{R+V33Kj{DMH*Not z%%%e(+YQtHp9R7H?53eh1rtX=OC{z&gacaJ8Ozxlo0>V1|1X>@S>;M`K?z;JzPCLF zydnR|1l)SA8iF>QNKTU3T;@$lgsotA|HotRbhZ774%>SLIT%a&u@u@T^}g>}6dFRf zt=HwNMN`Bo_YCDe&g^M6r z2Z`IvR}>WU&--zs4}CTEH+#p8ULwUN`)?E}FaEtQpaHYgt>5yKCP zr+|nF#izrE1fkfQut~Y;NC*N0`moy_6hxq>>0tJgf{Y>cs=24XVFT5X2f{?hAW#+D zNg_y+FL6~JVReNr93; z<0efX7)(ZH=?AJUD~1@9Z_Ra2TxU2g;O4}{d&)2bs*bs{Q(E539S~+*&hJBvpTBb| zZ5pLbyfLIN{h%0NZ3HHmXg8LK=`1})OIy1 zaZvV?a2nrAqgeb%ZvEblB?v3C-YASdBHu7C&0T-Q?Ar;HbhnPfd8+x4J$?@0^84|e zye|5iwQ_5$8P1;?;~AGmj@!d{F=Mr|>^>lS)o1KyyXxFfOf7JJ@Gg02h{{WFDz}oa zTNhQfxHCk+0;UU_@^ScYbM1s_D&{idkkf#lTIXMc3dtD~y>e1h6$L{)dq7uJw4iv^ z)iS?`Hj1ncDbGmB*KP(3`V5`fdAE?cPjAx@g&8YmHNW*sW&r*iM|p|-4EA2=D%{CJ zq05{a7yyFi%S?f70n>R*r_K`3+{HFj9+SVJ` zgB9%?!KQauNxv)k7D+RbE7F`V{wrL!=g;o16=c#>iQ&k7PZovPmda81d~2!dqd+ zg=v`n{1&u-PMPo1W?&A`NS>|O-P+nstL+#65{D&?ypl4`+pykT!6t9Rqm~;YPXDCq z^SNQh+scloP0)#Ib3PBzVNrY}I~fADOevWA$gQ@+>tD2rME#fTSuv&0=4hfGmUYjHb-`2Ef`i^dvT2au&S2Y{=i z$!<$#K$&oMJpz8T3+r&C2_O8sAKWFZwM&3Xbf2K>|2s&OD!&Q9O`V1YeO~X3;J-2w;eLLWGrCdLQB6 ze)b~4y~Xt+!M}y|#=^aI_AUSsCf&en0aFO-#K4HP)?n;LM z6LMkw82h|Dl6Ms;ly=;4K7ir@2-h>HHX*X8>;pusZ}DFle15}iCmGT44_%`UOaQi% zbEBEopXwRhp9NqyuGt4<0ZeQ@seP?CzOZY%X8<7G9Eu;ce`?olzddW*4}Ayi{DHTy z-4Q?y%Nyi?A?sVxt_1){Hs|?i5MVbdGdrO1kLm2z-$H(Dfo-%3B93MlBZ>Vq1~qgH zxL(`BR$HtIdT0dTX*K5PG5jd@ZCA+) zA`+rVHc0o;U(7Q5XmMtC6maaNl30SL0J>&!1!BY6DVyOyJTv}STh^_>P1bH4P@iqD zAI?>V+8!Z;nZZ*BcjcomHGkd4C^l4Qo5j-t16q9V1`l4l{kDW(tmDuVg z+zWO<>6Y?XSB-SEmAIV7W*ghxFb4Syk@dSX+>BNoCc1;Qrk;s$KSuS?Qa>K$E>4-g z^U4@eM#@l{cb=r&3>K}AEh$SaP{e%pQ-CzJWEOzMKa!fnls%Vz7C*p`~oP3atO4&2^(tclIJ|vTGVa9>Sq-( zXgRa{Xf)KSRrjS3Hkq89YD=2W93iIBr(7zMlkUdp_+66(I5H$Fpu)oO0JLxnV=6*uBO==})u0uKYV|z=SF#5F^Q~de}C+#}d6Q)69q^s$t#q5577on9Qcc>?FRIHd+Em8&RT%8e7xTn@>Rzs zVqP+G)Dv#hlp8}G?J`Iqr+zbuV~scARhncpFQ1A#{@8@V1v8J+<$*-DI(Tm`Xg507 zr|bjzkk=v229)fz z@Ail%!0rV-Cw6#A#+IBUVHoo&nR!I`!crj}x>W;aP&svgr*mTw7U2D-5=HVvHwR7X zq~oq|q6-||C-j$TX5YFc_692pTN6j-I=kEQRk`ym?hYmR&YXxkw z%e%`rAZbFeh0lBrbNMLI#RJi3(XI!-m+-#J!phNe0o$LRiCG@I=9M5^Y3JoqVNsog zzsXY=SOogZ8L9hi0mVLl@EROYLltgJU1ji@?kHUuP?W_$k50nd9D?8PJiMhtfR;KZ z086^G63+>htRC7KQXQzfIMOJ4BNkOo?#@FPYaQ&}CavN1Thptp;Xa|g)U2(g$xJO~ zh#P8ZA_|F)%uLnNij3dzcxh|vQGkNvm!hS5FVNF$)Cf742?C)vxphrLv8(P%DQN(7gRAMbLtx zBlB=}9Fp@clbZ`&;G5YP$A7DYQ(p#5R9jZMt#f>rFt}&!4PAS?PZ?j!J1#0NMULys z#$&;`4%}1a-{*l;3~cV}7MNTZs;}p&Qx@zE?Y8xk&;+U>HY$@x3wwu#(qS@0PLtLA?YxF5HmXzW|mLLmcdAWAF0K)b}R&2)9kI;bA- z^D`saotirnmX0Sl-e_r_=2z{agUp*ukWzPzEs8*ubC^Th(;IhoA;<_aL(= z8a%^nI08I8TlMKO?p|iTg0EDumcK*j9RFo_p_3d8#`$S5@Rkng!vJ3;Vuqkg!Yv5R z*7tK_wN{{3!|{L<0JYp^-B26(3ki3!7}96dh?Av(X2ym+EzAEw#w5F}^|YZj`r~YN5?28&^jm{kiesD+18`4!oK*ZEDhHnyn>mtC?E&c2YxA z9}|4??Ps5j?$0Ya+YkfY-YGh=sCD#5aQ=8mcO^#);(lbaFqXk&Tjw|!sAlt>7`j>f z)#_Q9qv$oy|y?gR-aL3iX5f-hR<3|TMS z6RY9j+8$I%{@$AL_YO-)lf4~cB?=v#CI6ci!cSgRxDAyfJKgUp!sIB#kMx+_pn&!R z*FC)w5qbD;g)gHf=3kVGABck1BC6qkF*j`H*G7MAMZJ9IK7N?fKuSepkJy1c=Ym}a z%8PcTCI}85y|fNdDm&b62g)n7jA_rff&j*GLT)(SYG9o&hyS{RH``$U^ITI50fXm}}tT*{A)5xeEb454a znHF!WSALwxGMis#Oh`+w@kBvTTJ>m)tVhA<8zXrgGhgL~;F(youSO2N$_^_u)Iwbr zE_Q>t|2kALofPSz^2y3L+BJnzm>THfKQo?96XL&$R<@tap)Fv~uzb-xDCx-qs6@RMtga;-i@v5L|JII&P6G@sUrOF3}2=Y=C^KU&oQNE>{GQMxt zvS(+`WG3DH_nzO88IhA4z2<_FK=W#|Juu#o5I#AyF4Lbyc%z>)=DS|w$5JVXVj)DQ z*YJr%q=L|OvEzQ7r~PDVgne^Zw^-2znoInhCsy3M9O91x2>abwR-SwLS3^gmdIV3@ zNY4nkG{s+PW+f5WM-B4Qt%!fqxg2&lCbsPcx};hZN{;x@5h!zH|7{Tf0J~(=<)Q2l z)ZA&*$=lS+%FQeNa5B|)Vb`nscUOlq96x!t z{wR-|Z1v>_)(28RPaqobm7>Rqu>{wl55r?i&JD5iW7RO9e1`-w!Tc(!2^zDuy%Nua zHH3FQ=d!Jwyn!Oots+3xsQB#iMN}}gPTnn4G&H7 zU1Wzdv+aX)aLl-12co-K8VQ=n`!-{WYHxitBbmEPqTZ4kA#jKaRf0s>WcHLUOBeI( zD~S~pi>y$bY>t6BLG%01Fa|YE#Zk$#r<#gZ51#L+<6XVnEE-KkJs2)F^r9V!X?8kU z(DifrWo1NJy)Po@pkyYhv*kpL{x#ulQaRIZs^WaYR$hDF8N5G2yHyq}0W==xU8Xgw z2VZy<>+T`p(~gCz$Oig)_ttwxFAYiDUBXdf8I?RnMka)q^^ES_)1$P98^2upJH8nB z`ClbXRq6K&5Fzu9xI?kE{Ii_XC-&zlAU`7EzuShRXArsn-!T|jCNU9gfr7P)rFpNO za!d+_y<&JeQk2-G2bsDY?`ZQsZq<`nbWiG9aav(bL=@f1C?dR(SB{67SMgUD@6V9J z5YC9j_;P${j!JIKXSoRyxbHlLaTCf(GX+7SgeKTiCa8gcGM)D%9SgGb&a{q`XCeTB*=pZ~tX@IGlf{+M z>)?7B>z{Pxe26-_D%L3`rUAS)t?VJd5ODhkGr)D(5{|)>+Me$ngv_QIA8mHvdSA69 zOxqOqwOccyb|(lkZhq|rJX{9RY6el3uUXE9v$@EG5c0aiX`?IW+Jx1Edl?gTv^0AJ z!2*1Ir_cn)55 z`tJGIe-b^t{(Fu5$^Yk-ga3a#tttOLW0PVwGbPEgcIcZMRS!n!(?QV zILRBqzR&7VcA#OWW&OUKR#9oHYBqQ%$E*_3wJ+1bptcd!na`uuyX>Ecck>*2K%;$?zzqzKO4HSRAL zY}Qkc0z}?{GgH)_NaM6!3_uIZM&GqHP9#dQ@(jH!a{OFxZBpt4HZD#&vPrEO`+(`U zF_^sv*57Y3I902=lU%ziwvi{jc?I{%Eg&0{zq75RA~tAsmw1F-ImE@g5P)A*7liJ;?~oNq$$Rh zh(9b_%&lPsP*2sSqG>k6+$busZy0HYD@MN4=mH7Y8ctW)5z%`^DF3@1Wi7pSPp5ymsj$Fq!t8GZbq4yDEv&0h6GmvO?sh6ewb4Rh zkiLPZJIdinld`gqCYLMGH^j9(k%@Y3b$YuFxX_aE!KlM8BGl5z#lPHr2WRCsw}|s> zy7*`=8uQb?7O~}(H8*ddP>8J2=hf?SHDv?UY?Nee_Jx(e?;yLA_smuA*^@zzt0fS# z&CtBPk#J&hwflwc0fudg!NHder>Y{vuflo5uy&zjWG}=muvmrYI(v2Ad+P6{JOZk< zcV(~N(l;xAkN&k9sU}OKQ5hxfKZ110VU`<}=*S#|E0fflHS8=QG+Ke}1giT8bksoZ z_<*k{KL*1w>clw^OC?Lh)$mkQL*zTwrf8PF-TwLFDV#V@vdJ7~HmY>NV4c}ttJ~00 z4m;J@(2jMwZ&!sAVfmw}j{|{DvyxB`OJ8JNRM9R!C{bQ!7g8*qM$e9OpN5M9^$_Fo zzW6+#DxH+4Fa!7=oxs|CAbAMkdZH+&~Inky_{xA_sS*B}d{cOuvdE?a{`lG0AM=*B{VR&Ll2tUuX& zJFgkW&$6=`|8174>Ur>c=z&I{en$I%328jk)-0N)B-}MJ>3bB5lNq5bhPM z9W0`m^jRlqD>prbCyAA(p|6Ywj`wF8UWvI|olm6Q7`1=tmx-1^1@aKmR2XP<99@l8nhR{;dYfKEa-71&D{H2*zq?v-Ks z)XtT%Mdwb;QI;ba5PpGrAT)NL*w@!DK!bz*R?!q0dnhu77ar3wb|N<~Xt`ItZ{L!s z>jD`!5Mnd-5c4hohn@g?2!X1W_89q!+y8HhGrPWP;lYL#Dl3|;B{c8?NJvsiAuH0z z>J|QES(#UtBH7$rDx;Vu6I)RquV%+50zhP`xhN%o+BlB3h z;(_>d44TfRj6OmLl4wHxJI^x*#6Iw2BH;PjW2iO*c-6D6+CG9Qx4cP(l5*U{t`?91 z92oP=5M#00IO6#1-ga&{N!f*!I%$omCUI<`vPo%cG%?>aXo&0*{O&8 zhQOyiHk?JoCxL=h4_=~U1avhe2tC=E{VMjYZ~if}Y<~^BdIg0^gcNKYBP>gl>7>(` zVCj>r>x6ZLP1tTUcN?%Bf8=JPs`Mkw&&zbXHrRs#3=}fxix2Z!awdRr33@X7)qume z-!oUR#dU69o!Lr(_{3nlfD}Mu^!GDtgZb@*zCC{pod^7Z2=;28a6H*#iS}c11KRjr zUS2O(+_M+9v>^*VR?D4+mAl`Lbf>!%HpF@|eoO^ie^p#&E7}vYr&g4~w#3ni2uDIU zP~qc(1$l4O&V*;Qze*`#*jVvFsBp#&!@tBfXGz?_5R$6^om*Wx0q^J!Sk7>k=ZXa& zxEJZ;Aw3XBxBHshE}@qOb%9<1c!JG&6v3vYl&SX1WR+!_q<_l@euYY#JKHI{Nk8E= zI!RM$;kg#~fL&vrJ9XHrG8emQNM9?o!>TiO`H@WSVOHd5d4c11Orp4uRKcX8H&@Pu z3xq?;(b2ecoZK;-xtIu|N;KgwC8Qt6gJRDWE-cY<56a* z#|G*{y$OAgkl;r1&ikWDv49I@QTTVvtGrn$;6Sj2$M?d=n@ zRf0c=ZXfw)=fBQ@t>~1JucpW}LWLfFQK(WAF?C5AuAAkwbS&~H6?b<1NmsoM#ndl` z@|$u%=Hc@+I9hkqDq8=`6>TfI(dyn zWx-vDCq{fmrT_=C`pu3sLiNI)S?z^1pgN2~^LD`@KqOwLk{`8ipSg}D_S~(UY-GzQ z?S=TFFYY|Kwhld9@!@PV4!%}-7Ac(7 z67ImV-6g4eyOdkKcPyFh-oMkzPs$ZJeHB$%B!+xaH-EqbJjZb+KOp~+xET|H`UfD1 ztNfp>691dS146#hroMB5V6V2SKkV@&&;|Baxc?rSr5D=ek(9wsJEw!&hKtFrk*gGS zDyb;F*t!Y0QR&Pub~GsWLw`uULccU6|12g3+a`TDx`se@52uMwOm;K>yWu^((bjOZ zx!v=DXoM0Qfh@5>y9Xb`hwsjF>F6_7H#oPWb@&vqFE|1>G9U=p@dCo~M5V@!ETO`+ zYKWq6$v8MP0b*juj|W1{l9XJPMuGw%<#-z>xB`@vIi?m`%?XwprztW`>$KAr5{I-%ErU|EV7=5dhb}k$o%}ELd&oM*yeG z4wAQ0HzE65$TZ~jogY{M(XjeW7Pm@Bfa#hU#)krd20c~45YP6C4LzkJa-$%(3>%88 zQCU}dM13yX#aA?h$&xL}Es*?nv0VWzvjG?LL*8U`g84pH`nS9sTXZVF;c%Uu6$VOq zw_+g?+kFKYp0r9WZ8^g{r>)tFisjI}$mSqwB3iRM{yeQWaX66QXO|hk^)b?ny1fnC zNHWw!rWkD11Q6@mG}nU3Z;s2m&!)*kiVemRmvIR&-=mvTtHU~(g^Je-ue2LWa;OPK zev#JA&FngH?#_AEguid>STcER)A&)pi`%x8QgnSH z4Llf>Y9vxJR}C&+#xX#Ras$k~=*omf^D%Cg$?h~h~G zz^Z=z7V2x^Jlg&qWDKOy9Ai9qRP5{V4tL`Cgr{XvssbL;+LX;oKvzJD<^&!Ykf!>^ zj6KU7GDxy<53Oz}Vv`ru0TizLfHYLtUdfpLIJofa5WqRD5volc%+eJ4se*d%n3 z4?Msp3)>?bl*QK0_?c|0GrI-z|D>$$OGbUJfDU;f|J94}Zy#D}!VDrtYV;cbEwu~< z5k57J6!cD1sjoZXNgxZDOyFu8P&K1c!HA+7!6w*P>^ra}z>ac#k7Eiy|2ncE*+FKe zU$JS{lu>`z&(%vt)XOG4^n1ym}QbaA+sFHmtyMn*aV=+)FW{-{LOX%fl< zuAL>uU^&)L;EA|0X|uIqj~p9=RJhC6)gij0Y=f>$aAv);8Dm@DvBylgWHR=$nelg* zqnv=)t9Dhr4m5XBZf*56Q?+pd<=@lJ_dF&Gpf4@AGcTab3p4f!LWtNXm`ZW_u@3dE zRv()Y7T^n&A1g=WLn!dfjb8Jxup8=WNvSfEyImMg($^baoBKZzlmVh0w3FArl*)63 z2#8df9O@wUNK}DqfuVy5pcnbQ2wh1fAJPS2a`@#s|gIqUKE-> z2hI6qSu^Q>SL;QbV&GJkn*hq}0O_8v(#(bHmVu)P7wzoDT?|%LDA@t+f4|Y-B$xr2 z;N*Dh>5S@{ih|Xls?54J07DMtvg+6`L05e1f9TifbvOUgi04rF`UWhr+Vkem)mAin zjd#lYR%kFF`X!?y9rD?y_`KNk_DAY*Gc`opp=v zmtaHk2Fy&cFPYEUEjK9Kl*ROvL1fv5epzoRz!26YX3>ch$$&6%#V1R+aqUu1van8l zx$1(>YdyebclA;C>wK(|lYhQy$;9oe8Lh%*Hxok$Jan5@UC7c z-2yq{eS}fgaT+WwO$efMB#F0H(OG{HcYFo>#A%Xv&$yCDcTN5Ac>a|x{STmdj_%9X zsV-w&sX?>*dHnomu?FU%Vj1V6aIqgu@>Sr~aVsN@c!j&ge&_xPVqDia)y?ukvr%Uy zO(}T35#2=(LsHl{-a%~p&mP@OMC)QczF0MXJb6F9V)+dlrI%>&eqf)hd+(f`XtgV9 zPnURUrC;qW`H&KLN3CS%Kf+o}Uyu~>qjt1D;cXrFiDka1=k;j;7pUg zXOB0)e}{+g4x3xD_?(Qs+;Oq9fwpkk=ffVm*n5E1&!3!K3v2`aOSA14ACc$>eFE|S zE1@v82L_~yHuz8wfo4hqiUZ8({9F`PqPay3Vq9^&!qEBCYV=^m!tvDS3e3roCb9}D zH!a9w6md^rZzP<9vWp0^SF+viCEnXTM226Ax$kEmnXj4ecXQEx_Q99liJtc(&55P< z@$mSX9#67TG|@9*Z~Shy8ab3^j{07Zb5LL3nIyUGs0-t*0W{Tf46!3^YOT6eTZL4_ z$O|3&5uc)LEr&BiITOq{a#ZNe*5mW7dKl2xRc1q3Z5Dl5^^`6Zac0J@s7XqP6N7t7 z%_cJ^7ctj=+f*wtl{1R9cKVr0+=BSq#8NrR+QP*j)8{KXfJ7dm{NYY;*Jh`iL#(2E z5Yu4;`OAlCAWPO5-x^1daZdfZ&zq8sADcQ{F z*fSteNhiu4?d@UD_RI6m?db08fk2p`gLT==W_&fJj;tX6r?H5CkOU+SHEo1egsjLA z<}JXi%NmhilUs5D%VgV`kpQ^m8~`PzQ)9q(es=hC+j(F=-`)k3{rE1qgulMAFKL0^ z-c5X(Cn%;9Uu$xn#NAUL!{Klq=ONYp%-t+HaQi-Mnek`s$%41dd`53KX@EI=rM*auV_%l+_$e_sm3 z+iC)GKM&Tkv-w*Bprya<_hYrh&mNlV=&y;TJB2Cgd|&-3(?U9~McF9so&_$W%+yKZ ztjd>7{0ROGQAX8|tx;-T^1>|-=TKa(6gwwp8*uI%kY3O(DG3q8{p4MiuQGxIoD2dU9Mh|=T|7ZR>aC)hP< zOPz6m`(Y&KCJT#ZJ}Za)CJ&1}{F1`o&~Ufa={xW9`f&pPlk-&2?xpp>WOF106y~$S*AbCe2 z)^($MW#vS#(y_@89Gd+FH>q0fk!dcY%)mx~1UG?3LCG2mSLW4^?Azw}fuPVcFP*$N zFKBwrn>a9Ow~Pje?D*wJ@fy5qciR<@KCscfxR;Af-rwmU1At%HfZVKPE?Z4R+`ZXV zIwl-|7u$k$TtcJx&x{mn#U;7 zhnhBjRqsAB`0E@K%s4*=exN(byXKLR-NwgSY!Dt?y7GO>J}+T=#;!H05;DMxg?~V{ zP&HbZsntb!wztVG(LZdCDCkuRjL?FOR6&b7PiZeLFKgBYQspQM@{1i@(v-Uk_LjIr z>~!t#(MrU!UZ?HBrBIS_T8f7|&91$`^tgsZQmJ0A(4;yXK>s7wsj&*8T0!dz^Pgh< zKL-0}CJ+#S>bmxzIOmTT`?HBP1vw@gd1?x)vsH!ES2I;OD~Hm)=mlO{IHn~pXH@&9 zErhd8TH=30Rqh~^$kkxuydcOe4TbllNDj~K5;?bu^~aoj%)7JL?tDk~L6OeyieGB> zWiYQ~T8VeXV2Px)60XC8WKq{p1(R#(^FZUJ&_Em}cxT$!uSxs~9vrdg2I%H8X_~jA zd!~dunaIUD!njOhNj;Qus>82oiFv!3QJ(CknONpv_7hLo-*K0Jd8sn`+J8D$KGDs zdqC?Ee9s_W>Cy}3ZPnM9{W6AiqKL0wv)YPv1#^%hT8AAfm;E{;nlEnT8Dn@}Omo+J zM5G`;9Iwu?px>0)_I}DRl)DYNF8hMnMG_+N?x$JC6d;nB=eb#q%#h2gW$@Uq6}$VB zG#ytuLlqUgHnV8CoFh0dAl+rbqp~FN6$7VI-;HxRDoEi6%8roDPjTu^z4Y9p!Nj_c zo~rv&<>GnVeRE@>jOE+TveBFd<2YP89*Syo?@6)F!hBw5PezjUkJk0K&+PPEe7t=Z z*EpY2xPKWt$pJ&c14-EmTU_jgpAV{(Vjx%UyT4OQtfvfxZcys+IPoWhpeVc0^7VyP`Sf;ze{eu-88 z9lR{JubDRwpcokLNgb6-62Lic&EK_92Px_i+SArTfOzhTX={9y|z-&|Migs z9kX#$iB&;g%70xK)n@&6dL7za49H;h*F2lk#Ja&jY|nBGRrZ^)Nk$Wb`bqY6?Eny4 zT?th_i3`;)=?2j@znQ-^L%{uNrOm?P>UWL3&=>YO2b9*<{k`f)Z^m*)YsOOvZffl` zy8a;wBtWXb9|m-#zmAX-g0yLmj__Q*8_l&L5+Oj*TsAV5kTUBFgnOjGNij&Vz+c+ahEuxqpeSO#)i0#zMhV%WXz5iB64b_) zsuAeNBVg8n+o+{Vy@z{9{gfKo*!#Z|k?Jv8ZF+cJ{&tZnb(uPGD)^bom% z`;DLSKfv{mXJq;X(6$Z#U)%P7PC4;>NwvX86aeb}^AvofU^X*E8exF&4W|KJU*H3N zGUSI{tLXq%gGNqR*(29C&D~OSdt?Bq;;08xpxG4u1x}|Qvy=iBpt|qI(Rt1Gb~i*? zzb-}ttrP7nEv8zhN?qw*Qv?@nNmA_pFFOs#$A*QEJy z`_8(L)Qat;XZy)o^#b=HMv%%$eZZWWaq0!jfVsjFwYpo|$Df1v3=XLnxLC0ejS%aZ6Vy6ES#PwKV)7ihI2pW5(s^#@3>nVpM}_N2eaxVR$OP2{hm}$J9gWv- zvnNO&7fw3lBfbUPwd(#-%s99hZ`}dH_*Px^<;wJ-{XHC&b8H-gLs10j+0_36h@~BU zrV5v@ewjKLae?A<&7bkzX#K4EU{*}~HQF_xD={7?dPyk|InGwma>#O5t(ftKJc2q963Pb ze3AFly1`e}DQ9g9M>d=^otahM(BTt*TqW^3u8BTc)YGhH z!%`>>cgVk^`##d@8^t$r%qQVOH|HAAF>)**c7%0;Zz68?{R>(8aDzsal;=xpP&kqW zF#i>Od>Pj6ebh8|HVcPORMGqmM(6PBO(3U1ahN;t>%WSGFS|M9pB#bx|1wTe6}>>) z8ptz&PJK>9AOO+<0A#hTPvN9Q>1HG@%!G2pl(fGW@+ot}H@v*ORw%nR6Mqr*f1rOt zV4+15K^JHOL*EcgF7i|R#8zb0SG@I?)P;b+cPJzHN6`&LN2jytv}i~82W@I# zPgEov+$jZ;V3baMos*^p-F30e8v@%V;H1OJFIkKMIVs(T&T}eSA>{D9Z|~r-Bx~+w zjOw~bXnUI4w4H1CMe+G@DrVWna;Cp%ThdSEQJ>pZ-mYxS~W*Z3k^$zy9+~@jsdqG)sq{ zsz3nJoFcLV%-A8@m1n2suDV(6umL-Lwh==OduBUj+{OdmYeVrky3*^rGb#=+gl}LU zL^XS|gEJ&I;=e|S-`=Bb+u3y2iNlUP+s3VyiRPK_2FS+3AyiTM-G+)KWD z85HxTW1c3Z8C2Xi^BsNZc#Ya4vpx21OgY?zzR1wh4_J$&ey3mdn}y(A$=ALh1pPt6 zsc&8)CsyH$x!S@P@c)Q=yb{-6!k`-s`QJBsrUsEJHMbQBE_J916rT7$GC?M^`d}ac zhL+T|&$gm6nrS#coO8D<051KSyCZ9vWWuI_ND@??v9u;<;eRcNazD0SfN2Mb1dGd~ zHXb2AzgNUue_pBJea_ISy6L(;$IyY@Qm4fTYNIXify`ija|Q>!W+m~qpdVVywU$O@ zLAtFf{kV3la@@crE&x*2-#ia0J&g7$;;5o)@ZUBg-7rBNS=K_zd3Nj-1G&P-*&SyD zGD23Yld&p>(sS|33!pLmj~D^2uD!?(<}dlCL4~1E;_>W-Xw%G48XI?yqaqAGP`sXh9Tm?6 zFVRiVqagPGD;QE;i4ZaWP2vNx{*SFvr1`9griuOm1(7HR z`}Ty;3Zp*MY*d8-|H06LLFeUA2mkc#Q^-gQfA_v)_6Q^)mgn0f&?IPtIu;NsP1tk& zlssZasOceh^AtUD#s9H$LlxWO@pchNqz`EK)r#WtrIuI?{j!FQZ|r~-xG$(yAO@P> zheT(&-c9=g<_$UDrBA%60Y(v}96$)=3(l}alM8^x#s^gy`vd@9ID?Q$`-or|t4z75 zuV2+jC}$;6pxoJjN?#XukOK?+(Dx05$TP1^U@uLOA53UoBmlkWd3$Ck7P)M-*o8R! z{X5D~v5_7KOspQ^eRQN-KPmTY`I+_zpsq-0zGEv!co5N`5RsDXk@1u4nRF8Gp>LsAIp_?gVx!++7+~CR?EA$glKL6ZeU9z`@d;6H&!tiMwbig;jD}z_3Cx;`$3oPG z!#Hnx)uq>PJ()jh^yLy_`qHu-E~E@_%S3fI*r5y}Qqmlv-s=MozZMs@ zl4#EDf0WGtQ+8)DCQUZ)HU7j%YQ(!)3O%D;P*lBmdf+!h9=-g@&!a@OiYyE2w~p}a zayIs2(^1g+Tc|!2ZdR1R3u(~>zo}nO!OeB*O!KhdcwrpknlYSvdfqc8&qt^4{Jn>h zo1vDgYJJ+)+OaCMs_#5_Bg#qn$E5m19c4*M%wYr@P)MvR#5kBY%0owWElE&a^Z%mj zt)k-Ewl&-Y3-0a^+}+*X-QC^2f)x(I9fC`6_W*^vLvVt-YarZOYwdII*=M!;Fdyoz z+8krdF5~atZ_UhU?tSZ_L3kVrk5__7=PEr(8%3a%GEpP|M@;GTcXN(8a zAaedWUXHkt2NUKuVq+R4KwJ6Tjk;*7zi-}d0PCbwX@F~0--QP5d&gX+(jDCanHC2j zgcH^ep5+_O+xuK4O>@Z>v$RxySUsd}<#XlimTU!57VgA_SZlOS#qd$h4yh=}t=UB{ zo^_*PQ^PV-%+rHHBs zrLnH%YQT)^N>2U4crJz#gF#HUa~C$^W=tu%GxpSl54tP0Iz)DKmdx8lvgto-N-!ft z&F49ei0|f#heu1L0deesMvBeUN!UNTe*G};D&?$RQ}H}OJ{M;lRH8V~%RXaBQgriKx%nEEyOpGe8TuUdyE0V^T&~kq{V;z^XqN)bkmL@l4 z^K{n9tY$voDtKmj)8sB-jzy=mE+&Ti`kLNjUTbfDGpGo|26&MrCA4e{t%*5D`>>8d zBQcs&8N6;Qd}PSJgi;K;CDPe&pl){ysM^C^BsoO(DQhSVkv!2zj<>12_)01suo8ZiUQ2 zK7A;YdBOX_4-jWQFS%3Xkm&zhzQGd0d8aS=V{r|^QrTMCJbCXLN&wFxDTYcF&DH@Q z=KqzHfw3ir7Ar#7JUEiZh*AbbD=RiT8m_@CMmx+xFQpBsu7mGXVj^r!!WnbZq zPwOp2+@rl}s8zjDETmuqB7I~E9V&%8I}iyK`6vpb2*7hlgJv6~hjSBQN|yiZ7NtR# zD*G|2CmEGo%>=@q#}DSyK7PC3YEl+ri`?Qa?5Mm&)a-txY>IR-U4>p{B8BZJZj0pJ z4cxliF>P+IBjMgyj1HNm)eK8Kk=r1}D_VsZ9f^21!zzX=)Hp5!wVK4kyaRlp0mH1# zGbV*^JwQX?V+SWtm4Z=d+F*kdQ8>JSCp5d(rv)dk+be#qcZXLjjYGwc=jGFNMl$-D z*vB_Ch$W{s_;(6;S~~k$4hLLGNAX>nB@T`_GfhyTYLyc%r~`pXzI(JG*82T#eLxar zzlJ=RU%d>rb^h<>KDc&G5B!LP^B1+B383~pDgz+uL29q$1zw%ayt!%TQkOp_>rF5;f57#UWPuYw6hf~^8;tXBFuz3b;36~Iw1 z?^R8)^Aau{)O(Hp-@yJgzKT&DTrAt)3Sm(cQr@1&v8FAniD_yLr$OaSYy^nir;L=1 zc!n~-s7V>)N>bYw!ss@p-F#2uun?Ng7dA6okJPl^vXjZzQCDJabU|ZBQo;1j7B=dl z0hcDH3mYwXRl77%f$c?VehF=W9^`kH*sfvEc*`n@a;NNiIz<+xCRNVk*P9b6?4F;p zP5D|)6DogXmP&PisqtQG*jY0YYr!0oa@rN4EuYOZPFUoCV%aTtz#rd&=RUs0Aa)BB zA$fd`Acrqbn61udI5vyZKqfMU0Wr=)sH#T<<5*0SeD#;u!nk>)#Y?8(^53fVrjUOJ zrRvN21qt-oH5H^CpY=Zx{FPZE(wyr5B{i@6cV-Een*S%WtbhdHaCxds%R`9b2JY6x zJN;>y2svs1yZremg`f+_qDw_MHLlTo-mR+YR5Sc--f>WIGz0FLXmIGGhjT_ePCoWL zzHN7R3;+0FW0q`|Y8F>0r&_d9C-Sx4gfv%@SPT1f1ReK&2)fUyo8OH6K&BK7X&`cC z7I0uthgy8uSN=2b2%xl` zH5kz?l9B0%;Sv9uI)m2@Ny$55_N&k!?3^kN`Y|1!VqG0~G=Sy3Cz99#< zOEix^5lvO*5QBdMZr-b~?&QI*{ssK}_wh-O?*OBCjXaT1(>H98kO8XTtRD4GOS7S$ z!ADxUZ?Gr9jHt|Ai76Slsh`v$4)%fc2N*Qi=sL4q9Xzt zM!$IS`NB)`(dpUd7l2B<{iok}-Rc=(%p1;w5C~q(1Ap;<l{(BMD9vdGuxP;Rry6M z({dhFM3QMoQSAEoOJY;4J3h$l&>X*=MGhUOr-99f%%On!8$j;W^6_W()a{o7Ddz;> zBwkl2MWd?_%rEYmCn|ACM3|SbzIS~r%~?9nXHLtErUvh8Tc96VYzN^O@qISqFGySP zsE7LX@;*dUrfa2gQqh7gTIa?DqIJFGoUjh-%6OAi2L7_7ILRjnQ}p?91s4jUC?&|k z>33)BCheSuaJvH;el01aTN{-o8P&kZk7pdM?7!P02#U%S^;*!M+vD6*dcHkiZQ`9A znpkJN=BT@5JERKKDSGWF&r;s->gm42yx&DHSl&>szy0l{(Xg$UKMr2?hkqZs|JO+y z!1+)99;fN34))RxVxn}AO-m1h7)(oRsaJ&LP=Pl7MiaiI2Ig^V*#@prATUsMvi>v? z=6xgj6U}EHvWR!Of-?0aP1qb853(LUWRU+tp(DJ<`ZTt)K?9?zQGy2-@r%-%V5^v9{T7l#iIS zVprT?{u-?m>f-1L_wwioirXU~T9*iB6Wd2QA_^{ejZf>Uv==K_lE^+|0ZL9@T(u!roo(AX)xC$!;GcRi z?;*@kx-K#13jPPlpS z_d7TIW9{IBtTH|&e>c}`UF>fgZ-oO^?um)6WZLUHq;YL^QeWZZG{<&z_(^u9Aq^FG z_!p%ydC8yOXA+wn+zV3VGs8Qj=;YP>9gjgs$g}=7zAwfutcfr4fZOiZVO^{a6kp4Y zh_1rzBz3xC#l%`vmYe4z%W$n#0;Q?g_#_j|bh(PABQ2h+hV`7pg!rrCW4*pTt-!sL z;}F}ngDF$2CakhLlf8@*$C8vuzwi-)i!8C^R8vQLyJ8o+>Be59n0U+E-#rg3jtBc1 z@VU)f49u*j*7lRvfW}+MX3b&#XR(&&!5l*mBz{FseCwOIB<*!$f}vW*p8jh+Z4SGY zcp@V6P~JN&+-DJPlk>{kZU;l8i|P|W7yGm4j}I1`Fxie09VG9c$mbYsvx~L;(iGw+F^>$-f>=MvbYGZN{bnS`clbl zTCVrQeB9+h8elY>=Qhs=39I{YVHEBnG=Bn0lolKg+{_y3^Z|Myh`9{7j5nFmw4$p{j9`d4FQXaK7dhy`)| z79NF2k2LAz15uXn9W?OMkf~WggH2>ZV$;Z$JY7ASM}tpS(FG0ny66`#bxn4v2h7LC zsi0NAMq3j?tTK;HZRQ^VBUkMl4(HEF){pdK?y=hr1acz}LMD zzwCFaXSw@E^h0{48qOc#>Rx{B&DP@w3gG_772lzT6pp!I*$G9qUUua@ArQ1)MO z0qTF~{4{`{N??Ty*tu*@657C3ly!i#JkA+El9@(X69a2&ZPi59F^-W50DaxZIO{Du z=3`AtRmeV?!H5fn|X!C3kv=$WPhmxSk_wjbuG05)pXe z4oR`!glHwQjYJIpR=oW-@+}XTOEB!{lk**}KkX>ZbfKf?fdrOF2v+=H+63`<=b=2j zz!ZRY$Isj&;QCoLiUWi+izCzIGLz;W&}<>A!V<+HSvs0cu_qlAdI!}`O&Bgj%1*i` zJB2|po@!m(LB5Fi7_L&%c>~&@ES1Rko?`rs%0=cIi-SkP%g{jtYDxLaB?mrSS&$zc z>P@*?P+>y??m5yd7Avw0$NS$w`_lj;PJ<74b?p9Yb^cx)aJQQpP_6SxM}HaX4+>0; zkKYh3;@(f$t`sP9htj^ZD4ZcUI1o(2$dq5?r=XUv7D%zl9+y-biu!Yc8-{DL&|zF; zL`vBgg7b+N8>Rk~g3^%kVqfN$whrHW{HNLv^Zb{(j}oab03SpIH+FgtiJn42M?|5K zJ0xb&B`Kr1XjaS10a~iLg?^IS^sA*dorOfvUUaJ^<%+XTlIPJ6G#yg2&Cz~X_q40E zM_&ptV=>1ZnMgiR;aWv-3L#%Ae9BHX#1__5y3+WSXYSaw zdoJCTdT>76tLC;d#QRKzngrc7WG-TWY}x)(niBoqsoJYV+4z-UF4HiMr(P7vaTENH zMcd|?CD^J#W>2@#Wvr)qCGB6+e1_p$#~R_nCrwIC8oT3s)i{axuX<%LuSR830mfxi z0s3VC#iw|s$)`{y#-})?>}#ytmdHB&C8xn;zUE)~e4emHkj9J0d;yAOr`I^SeAh_1 z9Rn{;I}h6teCMf_zprpVh81dR8s+05cWR?yYgV6L&;^8wtDUwYnoSmoe)CC~gr(ja;}ugyHvMUmPiYaj4B)?_mpVjVYLp zmI$4oo!<6<0Rqvjb+U@r3M|zK9UfDsXV?TaRTmR4`j7#D z0sIEB>vML^$d?C(Rfq5h9FM#s+mR)A)|it@t3hF`%CDR>2p9`Y^Uf%G*DQvaJ;i13 zxZ@6n+}RR3$?7chp?ZgCa_oYZKK-9N3tOB?FJ^8G#!;q3g>xj8JG|2ZqNi?}WjnT_ z7l@C}*Rv8etrji?SFx{9R>fyk=%~{Hnv~UBAcKO%LGBF6nuO%qJVm9o0`8M(4dL-X z`${%iUPxC`c^d1Mcbw6AgJzfOLymHx;eLm=pVS*YS6Nl7a$}bXFo9Z-_D9{YNVg?H zQ$=UxdZeGRr!rn*)J>^@@sV%KLc zp%GRa<)62HUV{y<9!+>{>HRYJRtgd}dpQf%1UVd6s;q0YSKJS{wRrT>g5Pyf$Hv5$Z!rR&{6 zL?p=wMnG8RvlR#WHs_nw%Bo|8PrgW`kC}^X6faQzs0>Pti z0k=gaS^p1>?DJd1=s*rI2>gPq1FIc__ zZkQ2Ok>Bfm)y2&NW0n`|+J8DY*e2Go23iAnwqvA{Hr-e#Hx+%@`E#slts;2s&Ov>! z%6E4Yzn1%}sG)@W8*=kR0dhV9B{aUa*P;Q}HVxhZ0i>!^UNpwPGzW}>PKz}V5n#kN zRC!N+eZzy-uMu?yxT`m&W5I}jY^i5F7G-4J!0?Vyuo64o+svdRMfRo{(@VE7baikG znX{DYl3o)p_&_p|nbzM0@O%8hvPIAUQza@pIMBt+GO=`Y92i4V0%)Y`1PPnO=|KiA z!Kw0098Z}VaBgiZ?mjmBN<~=5dByE$ONZZ?^L4`3K7sw@>%An}CJ=aH=g^e(sr7ET zolEzAld92@J%Rx}(mykq4gF|kGBE}U9h;XtSOc5JiBNYArr;MF9Rh^9oS_T2^Qq%Uz!M7Lc9$F?n2ugKLtJTU%

    CaTOpms^AZ6#am7jzAc&%HQ-zmu-s%WX!K za)IV$c^7(yqb&tDgd@{9^@PCR4j96qZU4boLQ?;x+OCEk_Mpn2q+xd(w{v^1ClvoS z5wcKkWx*tH4`_>_lIc#%2=%K1!aOq$d?kr~`i?4@5+N;`GTq2_Fe1zuzPO9aN084@ z0{==&CQ_tQzIVMu%Og<25f3)ET=w?P1?3@evflZxKOa^!4~_gl=`1T4UlEFfZ*{}z zK#1*!^N9D1%kwXfu{JQ0Z+M`;L2ZZTk1MPxOtmQ5bS2!ba5eIMvDo+ThY}n99 zZ@cCSAb{w3#F2)N#d<~{KFiyb3ls`hOHdVx4{hir0mTie6*&-=QMuerU`7i9%?Sfh zhJM`a%ZEtN5tG8n!bMbIDN}$vsNqSwJ};h{x5d40u?C~Hn4nJ^ht<&H5|?fmo#gcH zz~$rse;v=;4Y?L~TA@!AEVY>a$nMoCTx!_6m99zF3 z`PT{B%0N5Aa}*LP=15OYIGSEqaXF9>%j0`NVv{8dcbwpM4Amt?I4K|cYEFqil-rnBn5Vy9D&t54>O=v~pyX?m1SB zOn^AN{5pvHTUa%sv+Yak9nq(gL+|Q9(+dJ&--o~PAm6mxBkJpzX}3xwTNDJ-&(%)rcVemh!A+D$D|7~eo-)LQ79FQK@_{eRaBCqc2`eQwYe?z z>1y{1@8I0YQPM<0{(_1m3CGdPL+mdr&QF*j2uP)&*|}- z4|$;u^wk?*S$25TPIS-dQ^8sBN(Vb=>-wRjmOpMD74%DYKoCf2?VR>b8ZAbP8K|;y z0}_FLAqKrJd6){s8p*(Xr$^ua(It2-NrOAKAu?*jc4P4>;X{TdM;*tO@dobdXI|hQ z4EO*Dd&qR3>CkqW& z6x9%ZIbPqg#z6z!;cu6>Y3Fy&A>fOtDjf&-D&M64CHT%%+P3QK7k*l4i8P0Zd&-8@ zDM9S!?qnPa4Kb+8Uf})XsqV&8{2bYs=PvFNzsoMF)F1Y#ZskRna5F!)tK?ts*ky(G z6j?2!c4nh6>vv{wXW4s1`{@xxPeDOn@;F{<1-!%_dc*317!V8{ui__BoM`B<` zg$FY*`|T4nCOHhMx??lnN(}spmIx*@QcH8P#g|!ADkUq8v&GFzZhL)yXXAisCWNY0 zt(Oq{=Gr&GnhW;!4^+@2zs7|Q1GYAQpEXWf{ONgI%m2`_dx#njWI^jcCF?@#UnA?n z=j}{*(?PC=o`;$}T?9ATne~LS*^9KOZ85@WcUimbg5lJGRIm!n{OR zWbBAoM4VkV0p!DL!iae`CKHJvNeWAYep)A!;|u}CuIUjsY#ieWstupHmW`*cHM|w9Sv9fExu`P)j=cj@Z-ImB@)B3fyIYvEfEL`zC0^JM|AR+hn9{~Iii4zS4rZ5-Q0 zxk`h(IR8{{D$^r1`4e{Ejl89#v7< zBX2yH=*`S#imeXtH;v&%x9?|{UZqnAoy^;{gWp(n4bydZADsm%_=nr{KgzX#Z?>%$ zL&UX8;ZX$0d&GWOq@0}cG<)B<*bNj^{W$maoR>B?UodezMOreTPC+>r3ss2#S=BKjE#!q)nZ0oU@7fS%Xik-hd+zQl+guar^ddMLf73_2?2Az zB4BEq_3O>Ru=hacJ=!GuhNyp+{`8TSUaK0$TM?%C6c1OzLAp|J$FCXoi90TQOOIhr zk5v^;q#<@E>#(boa1i>$&b=A&F<%<2Z)uLan_XQF~etX3-%pSqvJS7FGP#d zucy+Je9J2mt`Q!~mlU$me2KXvPClE<><6lbz?R7!n#Rc`C6e}L__qHvSPdPlDh?EyGnW{8@6oKd~UAtb#%S zh}pLMDrZh+2DmE#YTZ=hn4hmNdmepz&t=}AH z+vZ9xE)5utQ0tyD*wqvTH!GQqb4vL1X0HfD2|BQccgE8*yx^SF>_Eg!ePb$c${bk2 zTc~7}(kfdW%5APV=e0iXX*{pm@Loy-cn_Hc3#;UoTTF_xe=VC8igQd}$wjFv8Zjf# zU85k*)7SGSkPB&6C^(VvYU&E+mi}CM~ z*�>)srl;9q_8Q`hHW$nd=aTJ{EOi{0x>0a`CjSa1&Je{7X)AbTi7Pr`W8@1pglRQPRNpjwNUeFP;=x3E;$?`XslJ}V+f?5wz1td%91{SFx1 zE@e^BO<8RbT;hVPeF)7wDny}0u9UU9Q#Riz zm-n5In%NC^VIj(EW3J({vNgS^#k{dhgaxY4u~z*llLtP{KxM}m>h{}r(Et`ck>kU@ z>tPPbI#*%Fo9=HBoP8=mj$OEOU{-QhP4PfHzph3OPc;}#NWR(qhO|0fAd_!3rJwhE zJD(UnSA<*R$l2bJUm7mp?h}EEFxc=})wERWL~gtcB%r9O-Rl zMXZw84=2KM5~i2AKWHlG<2JH5nfCV=hgXo_9JM4$&PpM-gt8HS`b@)%JBTXoWqKV} z05&rmb#9|>)=n?bk+0uJt_r>WJ;t(J3WzYs_4of%0z1KdOg%88i6B-{tZPlPuo|&m zDujjTWmq|%%Ck*oeYBA#cSfb%6B*x5qSn5^J1;aRiy3)0^!0Fgd57pYfH;N`Zd?^eU!}&CeS4dc1E14B!+Wt#al$_ zzD3@hVqA?$Gi|fai&#}BNOW0TZisXc!qOzwEOWJqqp^5YrG;KznMNbel=IIviu#nv z_CFF~)aqsj@WM#^voHw%+cV;CAYGmu3LapMB2KaBt&Z0RbwFLeZ(H|T?Fv&v=DH_p zt%LJo%uD^`hnfl$%AXH_6dt&cPyQLT+|HXUxA$+?aNAIc33&-U38DzduOAR6Q5~Da z&I-ZG?)W5V8BSqK7hn+q(B|of_Rb`GMX{i@3-yZ@%Nb>t#`A{6_e?-e zI#9o2@tW-r+WLq>Eo%IR48v-yF=XXU5g}9~oeBr1!OY_wR{N*eUfRyLv>10oOpW9q z0nPx*iH13aJwZiZdZQG~qIRXS=-K@7FItqd&9CrQ11dALI~AwnR;AWSAyC!8m2By=adCB}Ub@F-k{ zlrH!piiv+g^SV7ks6}~v;LE5Ma8f7LgL4VEr^gLpF^^mysKcz8{sL4*t_M{XIkbDi1X2FdAU0&1FrY3#5HmNp8b%2KF1|ze%nX#SYU4yH z=f)V{n9~O~uqg*6a$4Ab@?-fzZYiE}kCsp5l|MKKH#D;qq(?D<_E6JB`Urc8w&9(L z{u%Q&jPy|VAgI$Mi^aS%jbz6CJ;Eb}C1S_>uehSafE(uO4PG9Te|>oV@#(naLM8y{ zyU$Bu{z+L$GapaqS&qZd>lOJ$fs2ohE?V%FYOv2F8j|Bq%+i`j0_$gLiNcXQ$!~%T zB@#ROpeAwpXtYA8ULwP`uE#Q<{uwQ>ty7xNShytaRHrThf(_@St7!CvBfc^ z&5ruYSj_f$rmZ#TuSuuNkUch{Ym0Duig@q0klb@VKCgpJw=0C1vJ_$6W z=~!o72a5YJ!5VwUGF~*rRu0M^;N;jugxVY&lpeyX68YO?-F_dvV4Qi*outnpjL!jg2;(FDbO>(BgC1mdDI4C^7LAn@O z^ZK}dBIsw1cNwd+zKP+St|R~;W#2Tq<1DMcfOAt#F>_`qzfjMudbbC)?$%z4xgSSBd_lWkpI<5NYH!6chr3j66i z(X`j3qEUI|PtR$i5?Q6bk5QWxxhjS06NvfWnKhL6te!_#X?O2ljrwz3-MoUVYV!Cs z^Esc+ULiF9SS~Lxf^K};l@9mlb1zd^0|^IoH^X5y@bO%s=BWXnKPy=sA0MUVAv+x( z*QMntx!=&<&5a7eL0j8$Ng0umObLUwQSQY1d>hd479wA0OZm!$$%rHvcO^b>;z=<6 z^7+6?EYTe+_JI>kqFTk4ucCtV1iloqTo3i0jtU~0^N#0>+*aR*6BiY&m$Vm+ee7n} z*w^RRJ#kA@xAQSTex#sEY3UykhjY4AJrqTCtb3N9ZNy~RGnx&wf^u5({yH)V^>xJ7 zXolvMc_ylYqkeZ9d4oDRwhVnq3?(QhkbXZl%dGE=1SkD;^(=_>eiN)Ra^2x1C0u<) z5)^%h3>apEP&@$%@iELguhdEWunL5TJDrfulu#U^u&WCK1XJCikq{*wS#pujN^+6T zq~zVtR!NlUY86Vad=udeT#%Ng%rYxN*A!1cJqh}9N3lrx&aQ()OtyOd2~;q=A^O&K zUr6%POSpqwPK8^``0fdsl?ZOk2eNE+K=H+YQSy(}gMZ{H?kl}Qk%IUcZ$0z$Oh!@t z4SmN_!}M2QKuB=e(VHQiI-M?^@dYMr-&>3Jal2V@+dvU0nId(nRpThrFWWCZlJ2?a zd6cO5R-Iqwpv)U}x^V^>-4}cFBOGM|B;z|~pfJpy9@SR3`}^Mjk~cBMv2n0)BNN=G z`|s%Vzl9sbV9o6p5-@=XoP!~!zw08y08%yVl`&LN1zS~^Hqy9tL{t$F9tx@kadb4A zz7!%z(q+6j>T4M^Y&dr4hX{uQK4z(yVf@H?DvEVqqYC^W`rGuPo85hCBFFLIF`q>E z<3Qqp&tT%7&VkO3&b|+*eyG5$%H6@Gn1_pZ)7{~DCR~<;LP6FfBXPS=5soP1D_|z7 z4yv}zbvi}>+VZ_zZ*(kBc6G_xyjEku z0MtTL_L=Z!#i}ym?GAp7DZT8xmj{5+{seAu)V8#=3SCF!SJZ)8#aU0Ddz4k_M?+3n zO=Y@vV1)zrd5Dn*(ni5R@PkC95W0{u?Q8PRt-9pNTQU6oM=2(`Len8Ii-R;WyeSOfe!fGd%?P46WBpIMPoXI-@`E`{6joPu;UUkdelsXY!=A8?JoL{Y z@--s2!zQp?!pl`+qm+`KjMFn)f5P<)I5&sMDY2#Al7Sw`RNv1qWTE1jVn)9{@F9tj#P9?8}bH zvX(Qd=c{HPFgWJ1eAq2x%O%)d!c3rvXNOIogZkXw5V?hh@HIlAAYp8@kqZ-svJ2-w zdzt5QNtDqXe2y?X&;Fz9C;vr_D6!CEn_YSXj>X6&IjVc1yj5$ew z=8zj}CV-T)Vr(uIj<4=X$(I3#(o^~4B!?fkW}>fn`pv>hV8_w&Bz*M5p)pu-Pguu9 zV@(_4{e5(X5%E14zKyCl5&$m4ENQ&8gjuRQ{DxmKlrfjz_7wdX*jn5DHU?%+?v$bz zC@WZ*esaCYJaq3X;df6vlAT$Z`2P_ur$%TUcQ2%8tMN&~D~XH|KSYcbtQS8Z`nlDUUsfucU0VW-F7b?G zQYkru7yjOYzCi!Nk#2>gljWT2BTi*R=N1|ONn|3iDhb9YIj8Rq_J`6F`ti>TXN6ZN zegV+apxz@vVKe)JlJiwAwPF96@n4A_+l*>xJvi?h{CD2htAL~oh*Mj00%L0g2|7K^ zat@^COPesGX_e-d!nkF^p|E^Kx?%N#kJeUyVrpl5UX>KS!WqwK(}{`*kmiZ}DndP_ zKu5Es1m8n=9h~Ll&VuhDPPsoItuNM9?!?2$i))G-g0g@}NM10uDITZ>i-}@J5g~=h zSkOfCBX~LzKYJMg69g=*_S;@&c5SBM{2q$(dq<#xa`|KhPeUsY5{N3;Ow$;(O3%DR z@GzrYwUIBAJfj=dN2ZP=4@Ddb-)j-<^V+Qeg+QLTMCk$Aaxz4=9(Q{!8JR_uW{nc^ z<%l1sN(CrJi0hAdV}6Mc<~&e{Bp;e;WcbjD{@Ln-JD{r2C*E}oc^)UGq`p9gXcSc; zRIt2k>@8aN=O(oA%|J9m^EI>*cI&74=ClwUp6wuJG_ORH@Y5)w*N=cN@%$;sB7t$7 zqvA@E8ruqxPMHmlQSeLsN3Ra2KF%z#myh8b zkt6|i(zwVz$;~kpvYm92|1K?Fw!c>%Sr9Tmc!i*RRXkR2`}6m}`FGB5SCHUsV(_og z$aEKL6ajEc0tDVDYfD2z<<<8=&~R$>_`jMQyp+t5nKQs`X(v|d~xZ?3zi@wW}26l48v-M!f=7L8R@w1`wrNlr|#0^uk zK0rp4+{Tx&i|9wSIq{Jo8}}o%>#=o5Q`z|Tg9(#1hj*dDtZg{k*VVh+Kkvgc4{l8~ zOM%i8@is-em?~lOl3zs4G9m;sbOidLL~Sd?)YKWYv^QtHU3%UlUr!KTE|G|eFmF?W zHKQpJ1HWU-rWQD8Yv+Bf6$1G^g$gV2hLYak{B@vnl#?`r!3Wy%-v^o{4~zu>dsT1& z5Z0V%<_$?30CXq-E9j-y z>)Pkp@aI}{wBpE&XnM?bXc5oPxlqDs?FYZt03be($T(H22w!lN%z7BwcDXUwx~{#@ zc;xcexTEs;HFIIs9sBzCoWdD<*_#Tr_y0CML0E2v_J++~XK{?NnU?)2#UTrDXeWUR zZP`a)8nheU{=uf=f2;eCN0bs)1O_{6&<@;^utgQNW=Rw%v`Wlm$E#q$ns2eiGGZkU1H}W zq|RnO|8XIDPNsQG76{B_H|YG5^b{uXiqJH@W4m7Dn#2Ixd-S{>PkJhhW>Fsye8)ZT zFwJ^jq#+cpSD8a@$Txg93fl|W!1|vtits9xHfr!LTKV^R|DuobAGjRFKjGg|9n4Nq zMg61XyynzuunLd)^;B!T#j+xr8P#0EXU0($N|EK-v{SG*>D+1Fsr!{a67Cy?+4dxf zWlm!XK0OM*`@usXd-6}?A3xqdsPw7BS;CE@d%$7ADN|HYp-Sq(fXZWM;xfLXNohs; zi^Yznu+aeqe9+M>Fel9QM2RayAl%?NOtHghOy&CEtuO`p8N^%aEC@466gOBmAOkyt z1{eWxrUq?9Yx^(2Lob1dA3Ii|t+YuO(q6uuwwe(OBrCqvc|exp{$YexMK5|div&;8$Y9Gzae6@V6? z3s|7y=}Zy)#UIY3PwPL3!b~k(P=OOf*_OC3-q47&!<&!3=B<;ylGQUVkn%-I$dAS@ z!#@FtD+Gv9$}IrgSlNNlP<9674vqQy^|zYua}II6>IE3Fa}~ioIXt9(l8L>9rK61^ zT2zq`bI^s>xO{N2P-!eb<)1_z@u+fzrR^^77oo-ctz|Su`NGxU^Tn9(Rz0t&#fDe! zDX|7!glURlWT!;Pd8wQ-$uV#Xg&LspK$w7g7A1`Y+IL{3oCew)uc+mjW^HttjNPC+ z;Di z)88JvKN0>7Kymp6wwC={P{G2C#t8^l{hxOE+Nxdk<0mxB#Fh~IYgJW=YujtF!N)Uv zdy8zw&Yy)rsnC)tSCFqXu5;DQbQxbW_a-vg#SeCG!R$iqVv`YA>NKb zmB<`lB0F*$B-}#`M zH9WH&Mi4(Q4rDuf2ZT9fe^)W29^*n=lQ_GS(LRc=aK#RX^rb9h>{N6gnXPS{t;sen z`)Q#n7LE*}=&qE(A;8z?A4VM8ond=Md=5B;dzJ=?#yJEFTuHS=N3nXwMFRDl90f4m zp~H7bUiqF8OMAd&!Y9~t>id?zV3ZX9JNo^n#1+ZQ|uz}3=u&KUMaVqlP)DEO8oyYEiC+)7+vRsE+QVvYH{U35}4Y{)a z`AYQi$1}!L6DsrOxjMKD9Qq8Z?$Nk*9jxs5fUdd1K4|ru=)i}X^TvMji`nsl5?YTk zf%gV{&KT;Z;Yw5ib>K^2q9n$Mf$UO393`$H9`Uf}GH=>9mpIj1r<&yM4VKo)#gn$J z6uCXCt+J_1fsVFP-J5Rrm9)(bvl>;BW`VX^hShd!P9*E|1Yyc0?9z7LE4n)sP1fCO z4KG9CzeWh%j(0GT_YQv@Wtc(vlbT|^9$5iaBde&7Bel zfRvB=_mv*VsbORX`?;6D*{Tu{>YOt|Ag;Q{B9!+NLdXJ4T6412Ia5-NojE4tqXA-R zFq|o*y3TD-&`L=2smP!+6G_DJB0YUBrV&=wCeNcI$72pnE({eqtvDKc?j{ET@kM|F zblZdRNi?5pfs%nwE=GT%XQyKD0k8)K_=HL&ko+!Mpx;6XuQ9{vU&bh(BkPSmhE24K zI~LyMK$m^anq&VR6S|{8wBzRF`g7PPnHf68@4hJef>8V!ohQsu9=!numh+Ru1ZTym*}4R|qAMBS8B!09Po&PQop>GuQR~w&80-T+=Q*An*|s zX(HCIZ#uwQLmuX>LDN<)Hi(s)_Z3rAN)ly=!XSyIgq*5Y_6WVE@@Q}kok`wDc~IU* zTatu|lfhi8pBf#TV}0Nd-3`BIpaWP-5fGQax!xav-_uuykH}y*vilhcdct;Ppb5wb zfb*vC$yinyiiIILMI24hRN}44Z^%hWq|1ihawCG_8bKvWu?UqzOC-dfy1?y}8sf^S zo`}BGom!$0k3DkhtBad3lNF+@G`I1o26E%U0JBoCZ}}~p-P%&kc$Ob_+|$a8uFAJp zy#!YkSg+!`9JxZeh;<%-$BBN<{P)d(_Z`}&amOc&QuqC^O8@eo@NG0P~MCB{nKuQ)>=$tBT#_|DH zZ71!dDqfmRqKXGl(frivyP8aK+*8z75~tBm#`SARGiQPt>^iO68Nk(%s$NN`rKxNOvh9 z-xhuD=jM4ouIv0W^II`%t(iT0GqV;KvEihTpVQV$cb<{V@E6p*24+RT-LRlXn&6j8regyxk;RaxbK52rF`^u4 z7SjvQ?&! z0CUcBkRbQ4-hKp5(^UBIt$$%B3{$WE(tgjFEe^O3P)K;c8Bzm1nB^w*7ugImZ2|8id0O! z>w;vWY~V!<32yO6GHl4XmaQJi@Sq7Atbe-zmVC!9y$5p)4T!33Xn> zxP_z?f~<&vPq4jw-_ka(FJ$drmxWiYV`u8qxaCXE`~?m?ysmQ+ZEu=p1w?)J?tE`8JoH zl1SB#yud~Ng0#e*Nb6}zcX#D$rOZXZMHU)PL>kV-uMuubS&xmY+>2Oj)F=Zk;i&Sd zRkM=Nd#)0J@!o(GA?aNn)sc_n2`4-kXK{o)NkT1;%5zSZC>a~2M1sx9pj?mlFJx_9 z^N~;X>CYQR7d(bhR0vpQWU9UV3Q?;W7F|P<-P$`{nUe1OKd&mgmf9&wa%!?DbrBAH z!cy<%<3DQ3ELR;j^pc;&F-7;KZ;lIEDOV8=T>Nulqm$~p(zE~nF7)9Dr^6Wv0{B1) z*!A2KFVZEFvtg$_qthfJR2EZ+MUNt9nUMeb#f;S981UmsV_+{gAF5=(0LZIXXw0+NU_F}>2RuQchZJMQc*4(3JzF$)reMA`CS zB-h6ECPrbEFXC~q1rYK~4(2<7F_El?O&2;N{a?X0OlkKrhlok_=>#E%Ep6zAT z>iAx+ewb82(N)q;mqZzJv&7uzUyz5W84+Pmo$(a7O}QI(>>6jBZL1S+X1?^Tr4=G3 zG;{MDt5tKKoi-XdTK5xDQ^)dUo3*f(a8&U^8AIMOGj?mjfYYFYm4IL*_{H2QPwmbR zj@X~{-x}wC$h4JqDao1}5_uF&OlQ1-4`kx*BRyo*JP#$2>7SZe6)hs%v=OgBsfiTvekUv$~S z&oSwPi}Ff8+O6D1-`DAsMuEqfKT8{01vzD!D>ROk#Gy77us$DQbFn@}Zj$>J-2=PP z?|&AwJ`XNd-H`ImE~hBcpp%@|bf-%`DG%*=*WaHd$$qBDoX=|dGChgb-*~Bu%js~^ zI+3SGaDRC1nkDK^PwG7U(mZxsr`<8ye>Q`?xi)TTl!jMTfr?RqiosxB9af8@)W3@6 zCBtVFv-XW${!*oW@O93ZSL_9`Fx~hqfbdmvQtSIVq(ZZO3+#Y z9WG^LZ1ecmq%0X4c?`POYtc9DSlAvV1FUbq=k|Tf@i`5=_rDJ_)H_VAt^52g*=5V) z`?AN8Z-@6$>ecC9hR82CBc%2-i$)K2WfVj91xD*`jODMftwbK8> zv9h>)H#tHmHIYgdMY*^{M^IrngJ5AcfkzRRYeBddeldwJMuW@VWp%ue$m6oMrFp`%CL??Gdh$x;BIrs4QeFr(?~i`|`=mD4G@MBm1dX+z|FChkD5KpZEk|apH-Derw7! zp6Z(}UCiN=^1uL`1LBII=#vz%LF@qhus}a`?7!%Ov1Jrq$l?8ggnK%cK;w^db zEzI_3_LpX-W8Ek;xf%^hP4$&Da9I0x}td|&?0*)`3KM_z5A-Y zi1V{sDwrsBJ`s%*H*w(iO;T(=Y&O)Xd=%+)#Y&*O_fu_hXhR)&?a za`ISWR`w0*N$u=2;0~)Ohx@}{q-8K2aU$xFkkm@bMh}FQ6$ggZd9d2$?t>`g_WhLi{ zmE|6h&ATxw}jR?69X^3D~71nAYHmV`XzRRBaPB9I(4IT zoE6u3>kZ5xFkz9u(feaIc&cwpGs&fzgCF zYw!10e}Ccl#2Ij+3%qr@|IcXw7$0#+^BOW8xJwa8=g@Y<=(Wa175`4=SkQ}5FlXLVT`@bHu1=f5b#%azXw1fRwsO!un_zi2aehnl z11z3RuhOd>^d?DGnJHo1PdCd@m#K2uNbNLI%jrEkdZ;)*SxH-5Z(v$BdwWSqR-(GV z_bgG|>;XGUoC1a`b?z(P@n#DOkr+C$ezi41O%?=Qca*Of(c=(zhYx+Sd{2fCa_o=w zTuBlB^2f7C-uDp##P3wx@X(2Vl=`7YSJY>%_?qX~LmOT#8l*7g27*KiQNH@)GE zx60D%!Y2PCN;kFzR6O{US9&0>^)Jj7yXu=?f2rjMnIM^ShKb~c?rpJo-Dt&4th=g! zo$(H;6_b2+emVqnK5Y;@e0xVbHg$=_(|HUvrj!6~I`n(^9ME2=7vczfb^_9Df&S;; z)ezuYH6f(j4TK8zQUCHq?ytQ?f*Uu&p%NvE-7q_ikCe9v zd8NIiTXxfqQ#wOjGDFb~6!k7-)+y1n|6)@&Q}5moQKoNKvdVwdHcv}lwK|q@@3H#o z(E0Fp9s$D;ToLbEdZ=W}RO+)bj{ErGhM`;elos<978h{L@M_`@U5@*z;nk!c0vz{Y z!>jLqcyrtr4X;9e0Jm}chrujohJ&{tj(h1XaN>^)GZiu<4V)Zvc14^`^QDAluk}*u}6cG+X8|<%$;w9yA+N$wl`!K6LRdJMt zHmqL}#r1&;s;?WU@}*5esl?--$x%>*6;ip&nn0-B0%jhwN?NK?kQN51sjf!uS!QhqKz>!&E2&<4Wsi3-)+2E1!(Z^_z+d!4^k-+ef z-C&VfVh;1Ba;l&@Oz6FfpJNSsN_C2*CqZsnyQ^KEGOR60E1O} z!#mzn1!EP`D;Mwi3Zp`L<7vF721bRpa$wCfU6DkS5*yOPTtK_%i8gY&o7#GsZZ(ymV#j%AjB-N@Q!yGkHRtkU%_8)yYvAKxD6x;6 zaL^nhT*X$;oBbglz6_!FAVXuN?>&)9f3$@ndmdZPMOzcdLFj_-9{okTH!q9dhz|mrq zl4;9~7j~xDb-*07DnNr8(9=pqpMIGIclo_I#)Qk>c`@~a0LJTE0o}Mvtz!HhRg>a| zJ4Qo<7Q^_6l4|GY6;&gF7D;~+;owYJXpx&>7-(TbG1^uQf3_QfuFr5HWS&uLZt$b) zInNZ;#+AS_#{%#ynKZ0MA`+Re+G2fm{aI^b&^@93W!X|QHN*;prH}Et$OG`iz^q~- zU7K*QaI56kEM0PNN0_&W{;VpF8n=4>5%Y%6eg$_9pp~kvmH3pxKq^zQ^aw;~uIamu z5PrQpriUSwV)HlR5aSnv6-$7DR5>s>@<(kWUNXa%=T2(CY1LFPI-;~l`P<5TeSTR9 zGh4HW-RQrZ)fu%f2LGC(e`KE=9{f6MvMjegMkuq>exDM~ptN&#-T~p)+s-ott^jzr z2!tdCM*;5_{sMDS@Re~9Fwah#;L(il>EF_^GQUmtv4P&to?YHwedfa@hTpXXGuLB@ zVMq72M&>J_j~evSknoE&Nw|b(7HDY(P#)+9EDXy%#0>Y;s}hdTT@VBoibinqnsWdn zLODg_`!$w;7KB5}#*DR8xK)a4o-XJBDFhNqz88+kWG|2Tx?JE`(R^aoBm<5xTVEVY z1dtZpO8jDde;>ZquvBuoihlcXjWwX;)|BDy{n{#p;kWy&Nic%21X?&691o0u7>R#w zxHuM0&a-g1!tCTAVmRs-jMRAGK!xzCTQv>&%-Jxw1ex4O56oLN?5RSoDA}nZuE5N+ z)spZOD;oCCK~L4$Q&)?^5r2_&NfamdkZMX&JC$7fX{evks;l%ks7W&4`!nobo5Rf( znXud=cICm1m6(ul3umn6gv0acnvmbuFyNdiYA~&%YY+`oXa*-pX;ejmjrhZe#wxV> z5X3e6!fr7%(t5`<`w+`ZOi(&rVHY@NFc4hNs#hf$W`(Q$Sk;&qbWoAR6e$kJ`9-Wz z+QW}gpwhDU!(F5H)J3Dpe>FEeYWk2SfQLl6Z-;7QgSdE3K99Lo3fuD77P$6f(9t`l z*2lw;F0+>Zx;CLqP%t+f{5sq1OdqaAW1af&6^+qs{h5}=`lesNn8rG76jUAIT1Fv- zbTjV6Uf=DrE>5_)lZw-d;th=AZ>j;GuMdhF)1JG+w;9OTXq{AG`bc1Fo!P28p@4O; zK?x`%wrNw!D&eoZ#T#0;J{q?lRx`s_k7qSbbWLuf z*REQXu)%9>#n+Bm?H``WWU;IEP`dl~i3Xw^i)0gLH&a#_C8H=kL;MqyPq{A??I9xh0vl+?s>rwtB|&POmpMsUHnvZm?J#qbgAd@90wr^Cc&~*^j z4M3mH+$wph0)Dm^_bggKKHa+RQxbYNm%W<>w)BTIvNtfoq#w|R+)%8HJ*Y%AKPE)1 z=G|Px&Z=o0T~zTGV%zeQ<%=Hnlf9q_fSLio7&zB;S|D+yrhV5B%sDikJ;Dx8c%r$qW=6 z*Q`ofpf8v9P{=f~~74o^c0 zEoPOeRxLzgP%SPxau3zHAXZsEl9_g>C7o5)ci{e(G#!3DRW=hvt@kX zZd7ib)dI;mn%CA$v009+ZP1rjjuMN3vgdV^)r4mmY>{251Lr;&mjt_h-F->Mg)}6q z2`Y%sWL@}n5o(R%X5!O@ZHE0}$n96doL*tluN+&@x6?x#41^td{zbDEueo z3Q1czKibYquQ&PM^M}ETUishqclasw%VqFGl$eRbbi5))den(WFRkn%>jHFw08SN(5(e$+tt zs4^izSvu&toGAONW9c$khU((7cm2ve_JoB5{2Hai!`o*9)z^30nSa$0yM~#o+o&?- zL(AcPLz5UPX5c|%mnay zefsl*7qa54QlEbPio+S{Lr0$$k1;}=J8>=}OZcsu>AfkD53uo?FuX9bUtOb*eEg-vli)a^#22QI^zQ7o!IZk9W}vn|HXiuG2KixrpB=)))0 zaIsOVd~%YO_kDyv&|vJ9&quMs=U0*7Vt!<8>R*NKn_L zY;8x9V&5y-l@-o!oHX?qI}~Vv?9(tkr2}6qfaw7&GhS;oKF@;2zsa zxf2IwH!pnr@zTTyDD(5iZP0y+`L1=Li7uFOQ1|e{o066dFA7!4TMY8PDNnf+oER-8 z=G$;MPf|{8n`OTTOZuH6Hd)gaew6$ z?7S)kro(q$5?)d%V~Zrtl-!Z8tz{1o-&mu+mxM7_>49uBO5z{F^lKsPn7Kr;$ChDW zyn=1#;c9rTJ)ze@U}<#llBjOJ2)yw!$6u&unRNMGs)&>EveC(eKZf}%a<-Na#5Co%u4dHHnBU^iv%GTAJFJV?%r(X-d`a4 zi+z)B*JBc#p*OiW{8||gnF^lCwnM#kYqm)BD{Tu1%15D|-oJ<&PE0%w>H<^73oDvt z$HQ1n9~e_83cZYtEHU@lj{TscX>X^eAH8YqY)YW38B=8U&O(EGVzgSvV)C$mN>&va~h#x%{<$p#W>{mim7Dm zE?NTD4@Ifubu{sb=?&!hfO4~%@-Ha71H4|z+1kDepUH)zGyuQi(B@1|Gop2!yry;% z)=b?uvPR_2W6tA7LwVz>6K5swg2<*{E3Q1sDs4yjWYqrb*%l@fnfAMSXAjrXOi(f^ zP++Hog397WsmhCN>kFyY*QD!v)Lg>O-He8lLUfbmor|jvG9DtFvMq9S!T~&gcv5$xS^*clz5}aK4Ll8@>1=Bn-4;k9;$mAIiASX%-svIJ zKwFohfxd*R4~_abtq?suWwlT(Woe}_B4+AJDr}s^rmbW92|@H_rY~ar4PN`aP`fA| zHdRcMx6!>&h<{k{8~*3uWjZA~G*kz1C$7e5U#9x@MEm2*Z3Tmjv4{(4;;p0r7qj;a zr36kr%(c?o{lUaKOJ0UJY}HT=72On>c10RoIIqHY1YlZj+uye2(5%tseDYFat1bX8 znq+rSE^S4*nyxz4aHNTDldD(n)hewEu3HdyBS@cN4$T$r?iK=H2&E#6a}|-3=A_cN zhBG#{df!NV!ppw)NftKk+^s^D)K+4jobti2MvX|SKHC_KOXc{iy&*@gIOAR zuHCt=jH5F@ely}J7$ZPciw+Hir?lI8+dWh(^|Id0BXM(5v)RofG-Vr|S#-3vQ2+xW zWMY3?wa%FF!}KIe&fxW!ZQ1wDPZlFxoRPz?n^qofrpZ7RTzX^!_pm z3T{BcKEn~8C;F69mkBaapK&bBx13}GL&Q`YXl%XW7v8qG;pXw_Pmt7Vge$99Xe&{s zSYy3bZt4ik|9prbb^h`CZJ7wQSi|TAJ#-W=%f*=7?cznW+S2;#VH?M=Z_l^#>2rN` z2fJkOz=@!S6YxSMi7VkO9G`je(g#d(uSo3)%!*zHV~HH^NO@>j(w~uW6*!l<1ME$f z-b);BgchCLa*{?IN;hCS+Do7*%NLKj8hpTH)0>xX#`0r{o|RpyB)XefMm2Igx#x-( zD5s*BEL~sTW7j^0pHzK9d+RX76o`F)h64SOK}uapCmZZ|dw;(C#B>?=roA)Mq|p%i zNlN!VqtBB%$`)bLT+F7!2K-_C%P%}1l|t@M;KnW}8_O(|h2oJ0)-_2;O%+|fKG$3) z_rl|CL*9zH-F|;C;zS*(nyfT^F&yK)*(|E$EX3uu4rTMwb&A{b9^(`Xd1NqMy_j&F z6v=E1I4lHy4KK4J({z|#L`hiHX|#cMH9N0n<(Nahq#ysrn2|*k@x-6*gFLE9ek&td z_J}avvl+GK%G79MF_#)Q?|O&M*7$mG;KmuDpq^|aU)ha$Sv=4}|!e*Z=BHq|}HLm*w{ z$(_ktFp^+|)RlU_;bmT}auRj$Xn+BcA`eHdbVS_KVS<&=XHRNE8x82C7#8M5pQyz+ z=_ZsI;$QJgXo)xQH+g+L10l1+Av4V3WcRp+R?&TeO-J$5NnG!xmMu~le(s-&8poH7 zWttf6kdi+#m~lQRdxjSM3cKBGxj7#LlUs@H9XPp<*Ofr9EyrgffZ9dqqvY$;g zD9JW61aDf(GL(VF^Dx~e1&Ny_49;-s-%5wwmBXz%N;8wQ5|sF?1ms0L7f?va4Ixr| zw+QYLlRtVFbhZ^%JMPcEkpr89g@C7~5CoM@e?|d4MgC2i>?2QvbnKyj&acbPLlpmB zv6n_LG<#<1_x@A>*KCa*)7BXY{{RC(GDd(;mcCBK8_Vb7O(*H;SNf2~!&e5-Id z3r4KVt+jK^+y5r2)e!bsda|4}%)LQiX&_qFL@ZWwb|0Lna*DwBbJr2)yJJ=GHEbyd*d<$RvEDKLy!n{Z^pHz5#D%$2QB^yaJ zYqcvS+m29G?xZQVIwSqF5mdRm#zl3spF4D@pF0Aq))}YcsDY-Q3*4v^zrEBt=Li_b z^em>6p&(rrNpDmz{_Rz*pu~&n#%db;C7(dTsZ*446DK6DzO~{fS2_yZjlnn`(_+_2 zA*@JgIr?LP3;9vPh&h)%Joc{028&4cuxReq?g}%AwJL3;{a>(5>vFZ!iA~+q>H53% z82BpLVi2iyCRCD3u}OnZK5Wt@T2X_w^E7j@ZBff|V@AAOq3T;~yyo`{q|~MHJecOt z_ts&x;WIjIYR_du7^Bkyj@Y6`l%60UDzE#dK_u$WuBv7k^rOc54L_E-GHnMo+-QG; z(%acJKVavvgevcjr|oALuN>=)Zet7VKP+@n=Zkik-k9UwdA837w<-u@dou^#0WJfG zd5R{aRa^7US1uwe@a*$UjO*`fZ)b=DgPswV6>s(kRZtF3LO zS^0E|N7zQ1Z^X#0m^%M%#OkZs`4gM3o1bhfKTZilLDsP@s_o~T2A1#$OXi~(JT7a_ z@b`m#26BaGPF>TUL-FEoEZ)q6`-qm0Ek3LY?j}@Z=zsEGN&BJr2I_-;v@^5e=NR@n zTt7(?_>M1F3c_$tO^$7A&aIIB0y(x#s(;XBH=8i6`b36RZP-ZF@%L9s>{uFpLoibo36Z!FQ)m?b;v#)0pnS131#TQgoY&o7 z4szj=K-d^+oCzl*&7i&}iWqYmhq^IhUP37lM_$e7{O&zI$Pz%~&d61BlNn6>*3Aw| z(Nw8(XrG$zeVe~C;+@Z=&c5Z_PHE`M)=yu|6gBfqt=)W@HPM_ct+EmGC15 z-(+R>C)1~{pM9}@Er@6fI|qJyXWK{en$09-)n-TaD|*An5C;)3DN}e0Yum7RLmQFr zcX=%}bJiPGWZ}~2uCWO{_8vWV4v2BK_R@<=WJOsny)bBOqa5?>sH8BS?63tRhBVZg zE_jVf{NlKm{@f%qca4X9!b+JhsrCn!JPWqXb@I@KibP5~Rf6Kq*v`QOcX1eZI|}EP z?3Z-DLJoC5YrgD`Y0}}i{9x2vEXYJoX*Ypl$wFIR{j75{Hw@j_t???@-Hn$t$iZ|o z)z+kR=h>1US?&|F$_jVFyI89j3icDz-oRFNne%NA!8$_g zcvYF|L^4n0`nlsvTE@Gk)y_cktJNp{?Ak7sTdPpNi}R z$LNhBSQwffOWBJf3oF`w#rjUic^S!Jq7=vT>DX$%>$2x-qZ{9=fk}>$O(wx-SAFxF zVNt2W7@<+J4DXS#Q)F`#qVQTPNmnMvc^VF|gUwPG^%TDzIO3p)4(4I^CD(uqsz1yx ze=gQn93c1Hes;c=1P;u5dZS%*pqO0p9xtM%+a?_=rojDEVIPJ|(+7par}JyiBT#^U zg7Y7$UTbe9`xBBOW$UYjWU0VYC2oVAvR}vsqI@pYbB=b4c1l7OjVQtO`%YBw-e$9w zeKc$ofs0%>_Sz8Q5^t)BEbK5v)3?UC7Q3Zy)Kkx3S3!^$Ag}BHf z3oj!Pax0{6q-`-x<@c=_N*TQ(Eg^WTtEcmdb#KS=yGo)%=aUsT{sNoA(-^$g*^tf2 z{Qk89x;;@+zs1Pm;{emrs#o$({8iQ95AX4JhM@ESzy-%0nIMsA#LsyMJdwZ4(=|=5 zXp!`}vG(pxey{;!-b&1X_UmXIeWzBJY8O?XX;-lE_|N?5Nzddn zfwBXY6Cp0E10S3gu6l^G!-z{()EPX$m=C*hZg0YGhL}O)q)u>ov19 z`tTyETI)S+*p}r{UVF^fp0wF1rIn&JrN9^>8{}GjnR(hV2ZES(=Rp}-Ss@~6=S*Q%E4Wma$l$XZNVA1d09Em@j?2TQF zM9cyI@ghFl{uH@Y1$0Fl4#lM;S{-T3H%xLnL;oXifhbl(5#q1RI*t9#s34xkd}!^X z;^OiNJ4NKEbB zB=?g_@WJ|d4~)y&Xz!bjR}pxjH_Ld#rcaCMSzB#c<@;-oS<(*t+)-drNM|=*!X2Gg zaiUHO!g?ISvm!3-I4|7HW7EBjDM5i|(6`25;U(2#*Vfx`<%>v{9G!q24@(gK@{K9a zlNT03zqR9QXBw%%`3W^gFaLcseOT3w4n4Q~7}!7Lc9QdqfRQs}CmHYRd|n;;kltZx%6KE+uiRQp( z030MdV<9cLqb`r|jZWPpKXiz-wH}m@ab#`jKg7sZE9aO%a{x1dHIU|sq{OeG_OkB$ zOYSdp%d873*Gf=uQNk{E-tD0a#@fpSLK7p{l58hKe79M0F!so+2e=-i~z2w;@}s)!<~zuAlMv2oEvLnee2RS-NP$Wz1H| zx+4i28f)w;@TA6=eDM)U?IP_w6`NMsHf#bus#_;7sEB)|eYul8f8yy&N;j9|<_#7T zFNuz-*=RRu9E@kENpv!@ywJWgzjZJ9X;Yf!EQMWS>%yHinz$R;8gFdyi9poYR>cLS zgP;0|@_-|%tDSFl3HoA&UT{*Xx*M5LeC3p=#oh#0*_shy?3vsz=MA)2%jrqYS8jNy zT9@4=8m=z`a_p0BGPVicZqhHWOd^5DuDY*KCBS((YivZNkz>?lb+X}n{mP)8ocGah zLf1v}Bn|^Jh(iYA$?XyQ4VvMHLYtJ4k&Jj4g9&(^D2bV#xYnZZ-AyTK7UWa22v&*= zF<|8sU^>3QB+vLoOfGjzm5WwTditg_%8mNciyJsYQXJkzh#HCE(7QHbaNPx-ipgKZ zW>9uO{rIxDvOe!tyrh0a(>{Wcz43K%WS7TP*YL2;8CvW`y4kn$nSoE7Ofe}P#Ji$> zVzb>oIqANd0>py_228vQ>|@-=r6ogFjz7sz`gKc2?M4JwnO^Dzh<-#BX3@Tep@{1mfn6*GOk)u49bKu@_}USBJb&=VCI_ z?4gh)AW+qdn>APGZVhxHSm}cg@wBe)b1deXZy&hw9QitFt$-Qp%lTR2cG@)IqvULRAF3qqE#47l8hxGO6YE#ya~PAKcM4ES z(!IY9&4O%bMSNb8ZZW`ql{4n>Ao89{Ff!p`&y1wdA%1a(dqkM`eacT*d@HTlL_5U$ zJ$Z}Wso&~0evpwg(_(6Xn1jpM?rYXX&9S*ty5~(zUvJC@>rmvDF5RDZzk;yfkKSb% z6x!B*2fOx#lV=PrEpOc&kZEp}+djR>L_Aww-qM z=Vs)teC|CJ{>!KN;l^L*CydA41GfxZkgqU9*1^XMTz%1%gCCKw@3}r0(@ifqy%?LL z1&R5mlJ{j5ScE)bYpg&RSj#`L9)G_ps^%`J^}&Xb#=`+aqr$(Lak4H;l;$z1AGHruv@UOm8)mBP8{AFaS$!F_N0&fE9BRG2nu7WCIZxCQ zC#M#SNYlxt(BSkOKYi#F^J_;ADRM=E{Etu80W4X3P`*&ih+g~{AEE-<{bwmKJlFl; z5_vS?&eapO7Tn0sbr(KLxcX_7`$Vxa5;ph+dc7Sh?hs#YRM_FgP)mG8Swu@fFEp?G z$xBv?z=m>4dJkWS8GSxVB-1z35(7Yb}?xtSdH?S%jfa|k z5FB(OppxQeAQJ(W(3I8!p1tq1j-;bXW&PS(^TtcYXGJVF`i-f%5|O=B^7(zgg4aHc zjR(Wy`n@HkUeV?aPw#6=k4$-D7Q-?H2BJ@D|C3Vg&+zwJHBkR>OsL}c?t(6=Ak zqj9^ce9-i6H^NRpDgf!T_MEYPOo3CuA_C$kIUY_Nl`SYwYDl9J8I|s)O;9$n#+^l+ zB+0gI)|)wI^z$hCVj(rng!C~H+Ks&NH=78g)*@>{DOBAFH7LvDIi`3vE~rLxKQ7K< zQgo_^RbTIXHsVJK*ACWB@C>o}2H$ka7)gtzdq9skmdm~ihgcLNgrT}1Qf^`Vz8BmRw5P)NZY;Ob8~K~!RkCrOO9Y`#&bigtdMf8J9j@T|I#~_F7vAE9 zcIiaTOLz%4uvfBxG>`*-SFBj+OCj4*(W+B|T#c^_bdO1H?LO|N_6JH5+T7Q7i&5>o z_H#;g$c2|Hs0P0844xwYVq=bs4gK_nQSfuTOMJ$lngJEKer$llv!4z8S%H}5i!wnU zrW?C;i!ii8RvE$PWAilKo?s!STE~hNW9^%?0iMXI=@3rg-s9U2A2de=tE|A`T=A_R zRb2d4T`WS|6=x%Sa*4`GH2PV_9Eny9OXpTU&8Mp(X~-GXnT=3iS25zI9-TakP%D;G z4?9(5{pr6eGf-Ah=FBj^exDqgN_`shZDqm5?h{KB=2ZuM4J<*DQ$qU=Mkf<~=stRMB)qTHyc{T~%N#Cn44o9iopSNsQba z{Vd7tg#>GeIwF6r#9)496tYCFo;7<~;s^*_ zwUj$-%#CW36l|(M8o8-zNW|I-#tKRUk?NiTwr4$G0(sbp-zq+b8Tdm>tylLYzVj6F zOEu4UfzusE?K6!C0q8e3NMH*KKJ838mH8hk*8D0!wgC)gh5#zfIGQBdBw9ie`+#k{ zu!%&fs<8JdnJ=KTsJO+x*8dFninqY&^i!?-`ure`?JB*Rj~TwV*tA#5MK!@Oa35nv z_+ia@<&5v61(=@y|Cs*2>BS&kmLN2U!3GL0m>oE6YHDZf;$p9Q^t&#p?_$DhZ=!LW zvUiBTv`-cYSnl*pvR?{oh?3Ks+8@pB#vQzNX`|dWqz((8cyVX30c2DNrU_^q8d+*j zX}Gbl0N?#OMG?g%AK(~Zj$S14;5&3KbzG_lFzuH@a1j<9{+aBYAF1>9`D-|BQOdWX zV9rjVVa~*0X6N_pp2mCB5<>e&pJDOycyyHNcI({nHEEKQw%ls*k4mV3Ov2*GXU6xA zMBi2H>=j*WW?iIsdX{VEW6HkB63AARQa|Um{uQfUs^G^WROKDs<+1<8iK5Odd9UE5 zJywslxd6uuR{5Pbh`ruEj`~xWMRI&j9N6zOT3T6fcuQu0@eXUfidCDrn#6&pS8}1s zM;%$Kw8D4w8cDM0?E=`9K8O~EL2{P>l$-Pz0$hV@q;`XR`IO&?}wTJ~8 z?|7YC-@lsW44FR*W_x$`7Hv|SlP(g^%>BBXcoJKmK)9euwPoA3v1u~!Sh-JQfgP-u zvy9xgWFj>>^U)LmLYsvngc{@1whRBW9DKEl_v_02J0-(`sXXajsCR-7Ld2iz%30W!Tzx}pIn@Brg@HT*R=L|pN4LjQwATtxd!4j`9-ALk=vC*y<8ZQUTo2My|4vkZPGsOkiGGCx9! zGe2lxB##dnWB;o`nLU$$IdF{=5h%jQ@(8WS3L)?5-}pZve)Wzj9BJKvm4cxEcWYY{0hTF?5&vH&pxq<$aiL^9o?-4_f*<1A zYKjno_US)_WS;=%IE5srED%AoKY>X8xXkN!t84i0)`YM*e25R=f**^oVo^c6ItIYqxhW~DLiHEEYgQy{&KZ8jBusHnP+?)8PnROV%@TXaS7-)`7 z{?q(%7)1GC2pVoW<88oJB7iX-N1OYH-%znoe{{?G-No_o zVJwEH&k~gONt6Cj9BmZnWX}5A2}&ru!2aF&Alll7sPqdEJ66CPdOSO2bN><$Kt}&z zwep*lf5Cs`KuE?w%ny8ancXdr0QFaapV%X}6Ba$7S*{RZC10CCpvV7uf1)RudKCy1 zGhnrO?EUw}e<{!*J)xWfG#zAQ{O_cLVTuW&2d08O z!1(c`v#I_AErW(8gP_g)H33ij$#X71ZA`!nk4^Bj;SauK0z~kj1--KNwFv+V1lYfL zq(yq;AL!asXc7qH$yQzrz!z^D^B5EXVMF7L>71zwxXP6ar{e zi15N+JWk8}?K&{NP2gFn^hhuAu|If25@-rY$=qK&Jfjyi3=WVp4On`l9^t{$|KJfJ zUe3^n5X0HOgCF@SL>>eTycRI#^*<(!n*q$0g#wA21#v)DXFyaBY(5vT4x#~ynEgGzvCDm)mP63Ai}^{q#r?x(SJiV|H#>!1>|&N{BEIxu+IHems>Cx!5fgn z3G6B!PkSn?KTu&)Xxs;NAC@_!tt;3Y$nh+&%pv0b0o((;{)ix(2SnK5{b>eULSsWJ z#QrYHMPN5IH$aXw;MK?bg*W(r@X=P#D3FEu|3vNXZ(d47fBIM!{%(L`>+U+Zfk09M z=GEgV_>BAyG=Aal6m*9t`BV)^bO0=<{KzZ?RDYni3m~2cV>ABP?YaS=&jIM;O~)+l zZzwUuVG%_70R0L(1o>Sg`W+CNk5>#9#y`*_PiR6WWx2mxCC!vCU` zi-bmk+%Nsd)SzX+)Ppbn0Hl{egbyPn-WYdM140TPFwA44P{{s)W-k9V3ewMKYs-LG zCZMRvJXX z2ohEcjSTs?`geT^HTT-`1nl4hxbU&UHrD?T(f*kPNNWHu5xakypF&=&f!H3DqichA zb_HDf8gT7nlhVKb7k;<}8uLN92O^GZ>ETPjSigPocvt$v;~ye8pmHUM^N#{z>wp4! zZ~pX+;ln(G9Q6ZJF?Rj$V0K8U*dUC+hR_)bivCfMczXRq1cFrm_`6OD#%%6d1A{jN z+`#b&Z;}5m{M{Ke2r{?wSBSzW29G&FF#3P|##;1mq7A@#85@7&Le2xxMGdfzEwGfc zJ(BdJ{4vp=IO0fH12F)rK>)CMd%S`dH~tGR`UHj$ftY+>X{4)rt3CvSS<2Mk_!|9u6iFL}dqr2N`FD!GuI>0ydD}{W~Tl9Z3ZdV5Qu` z{NFg}MgE5f1;VBcgYw@uQj9uq{wd%ZZ9p3TBNrKC{~_u$f}wfe68%s0Jzl41F4kG_EuR6X1^U8wYpFRZ0s5uNC#ERsv=O&$7IbQ+8tOq;?Os{|a zWfT4(IzEGb0x{VByTGo-y;t%AmMVYh|JT@8fK|10VIwFU;+(S&DBUGU3kHgUiU>+r z*Dh34lv1&--HnBgih@Mua?#AwJ6#U=Wdrl1a{r|t8Z{N>{^R6{()~s2xW=-tU z_?C`3_&v$`O6`1J-n>%&60kmump8=9zLdy`3>&0hk#w+TLcO2yv%?8!gnpoqA9vmT zD88Q9zWXYeUKU_L4bba&u1x|c%8xvF$r{u-xb2qTD11N}HSdiis#nTJlCnCyg0dW< zC~;-7^p(;^`tyGJ=h(*5pZ~m4T8qW+zv$L%+b2i>Z7{yblE6I4HG$sdLD=kFUC(BK6uLyp)WYl{R^34M0oEu)fk<12rZ$)U40XpD?i+*zSjY@S#?(mOA)qZ&`k$u*1Pl z+Yl_whM(p~3t4LoUPhL_Wi2CHcK`NmsIC{};k%mEIErs5Dx_Dro4r~)#}K7~S^$e*kFDstf+>$dN#E6M{=_-?#>Z1j=g?_blm z(dJH){eK1Lq6hAS`);ajbNp-kG>QL+_sVcl>e1x`=Z>}2)4Ku1@$;kQGbJckt4P)z zeM1uG#BBdsRJ)bF;NKPe`3^a0y+qXMgVI(q%whPKzI>DP=ZqqKWtrG2t5MT(ZP0w} z$_#_KTO=}rV*N@aC4cy4fkBtBQ`~H3tA(bq*WT{q8mI{|`^-9Vc7B)X`ydDHoxK0GDv7xn*R6zzlV-x2UA1CiW5A1xHL={e9HLhv!GV=C|+F6-=g_Ip{NWazj zs;uEc1q`cixRZWZVI1_%x9gjm`lOt@0fH;cy+y3$g(;EV(BV=+cUnu`Rr#?5To2{Lv-0WqA6iymnSOmQ6Cz)2j+M5=4OPiD735l`((JFUJ?v{Viv zHmUV9MH)2l5@PWV&h;b1*8Q%Y&~^TI0HF( zM*}#z;FVykY_u1J)6y^v_RB&?#4kt9|`wN_hxWz|Xp!oh4|eB4qt3G&0L6EECX85F$dS z?s|H0ZW^cs$?IuQkxVbboP-h)(_c_SfTspvP8wGb%%$IY)8Aseq~E5e7+6Vegwo~I zbE!c4Pd$V+o?K!7`BS_liT|xMBZj|~jF6Va53E8~d$G3C8CS#Y!2eGtR)7Wc-0^qJpO> zM;*8cEnSBGn{a|JuP1R;Q`Qji5e;)vi-GHl?h?V_zeH;A)kNs?;NVrq0BZ%i=Urvz zC=C=Nb%`=a#5<;&_XvV0bD)4Ij`h4~9em?I%Bmv%qFvNiZ{WuRAIafsH_^f8|G_+v zR`s3zz6&=Y&j-kZ0Zr$*9w*^1ZZt5S8z^{-%{{lHV5@J_RQn5Y&NiqtcQfk`|JG}_ zhh{I%LUFZpcJzvp1`sH+(mZ5QfVq}(h)mnsB1slicx0B*+NT47{!OqDK$Gcg;Qnw>0kK>vseYXVtP=c|1gCMwyqFnBm zdryGaQ{V`AA6tAtgZCu#Gb88jzX9Yp->)8lk0B7^1w zQplE{N(-rZJD8(+4GMUWE%83Wf}WNU$5VeHa{qzBxEZHd-*Xa?vrBm+GRgwp*U3U~ z5ycrbtM!^h)L}I=%)8lDw>3l#WQJ3DHW{Y9Mi{m&xDVuSg(j@em6U#0L*zq-Rb)*v zaCx(;Yf;EM6v8)29PA25VbEuG~|70qMvG_m#XB`GePtLM#D>Y z;A3f`-d0S!pZQzA{0pM%Wr*@W^F$sb+K(x(eekl^mr?&#AdKbou=XoYm~Ch&vxIe8ve)gjv3ue`OiKtW6LNnHKt z5=0wOrlc#?bcogsKuIGpDV99V410JQ+gdt+8}@>mSl`vku_(K|1bIIsIr ztb^@CxWe0i#a$8}O0t+!_oSJ)O-gFLSQoTV)OqRmX)esMZ25OsdN7LZh3>?MwUe%q zhvtIKJa@r}JaA!y{h@q??^Ea^4IYADGH`mRg*R|zqfxIZJ%+SK#X>NQ@!XRhso?_I z;40XO8rR#g=T=2%d$fa|UPn&j&M(w(OENN~f(5DK#-?EPdKDdg0kc{Rvx?$q4u0im zY^3BbZP~W7f9FxJ^8mewuZ3Nan#N0%sO{P3&o{$qJn|`x>|CNo3dE#N1zk-~cZa4k zR7B+`*b(QZ6)ej1h2}qBbzXuzk^we?@Bw2FD-D@I{5=GvC`Li>#F9wp(FA(rNA*sg z628zwFd^1S6|BhK(afrj=I2}Nfkx<7%m_|1CcYY?V5yqJoipeV+Zo-1_j*Bo5_D8$ z!IX?htso;g-pmU7Z1U^%4+u8H-}9P3?9UUqiu=*TlKfpal^0Obdc3^9nG&QSQjrjK zA>E9oRlWe>gMsI3{-de}?+$6tY~#+=6#70U;9fP`?WH;)Xd9A2q|kW;0lq z2Wvj=e^gV0SCI~0f~|-b>`f<{pb;h@O5!K`KSDKlWAb}=1yeF}a0P|7C1Q=@ckF?t zoZzVV=(|riPiaMJd9&twYq2!P8dWJlhv12(*4H5tM(eius3WICe!#NBVA*`UWYU-? zGA22bV6~d!%wyv}T?17B96}o1S_L<0GcN5z$+g;lcZ0Tj!g=u8+t6CV z+s&(BO`LpLbuK*a{kj4qn~yT$IR4c;NO&I~AyicK%JF++KdUf10AJo~?Ch*zEs)be z%n)xh|I@1<)H@Z8!#BLVAH@ff&08uMX+@(GuRsUkDI&H?OnRy41>gdh(Rg4l8*qg$8KVcBsd&f9DeeK~Q9ekzgf~Ta0@aj;*uo9i$hE-<~b+aAAi6zQjfU^1E z!lWE=t|54lP7#K3ZK|dBre<$Ff!%=m@gv-5eTt738~wwbjw2_4>v~66jV*HrEi$$QlLfs}q0wpLz}j1kw?JV_!=_ z%0&p4WN1x6S~4pXGHy12p6;7T$6B=L5#|y!x0zu@>D&l4*N{D1p3w=VE!gvBVPz}f zm(~(YjFQvS{>e;6Qr?mE+1A&3U1^!XSGdOA#SkT|@e9M;52e{MS2 z(3bSZ@d!~Ti38dVlS7g`yj~oe*YOhmd>u%$bSgXs4L2l)i|T-3d{Yg;QS2K(B04Y1 zz*KQ6W*Dv^A6HsYMJ*HBj zPa7u(G|zGH<5hZSiUw~(I@eb>PkMdusySWrn-N)qzla7I=GNp~D4LmWY34oC{VcL2 z5+3$9GO%{;ov6((r~OXpqio^2HBLzYCbM!)wn*Q5oECrDLVPc=_as4eNF zZ)7>I5$i_(9=qLfFa)3*Z1{G4eqMt6ni^S?PHOzvsoN7~08dx=AdJp);Gza^LN+#H z;?zmpy_aUdDqyhT`TuuW2Y*Llq!4rMwP+{k-p;>vFBI?`ecM^52qe<38|w8=Nlr@& zs2ATNrv88)J(Bvl(|j$lDL(QsiwTgKB|K=8(+@D;!ue4}R zS?w*!8C(rQea6eje#f8wZ&s~)C1)n}&q(Z+^p^@(r+jxA`P>>#9WuH;MjM?c@%i8R zl1rmR`L1J3>V zl-IPMwY_dapnVEGJ>;8pp0iqB^mu%HVk)bVsFSLcI3;^p!D^6P!prOP{U41Wk~yj? z;7{r1xXn34g__2slQY5mhvk`gkK z`X!QOuas5+l(LS&^Y|9XUl$-_Y@;m-|B5s$MM+l@tm7!rrWJzqFEsaX^g6!B-47^5 z)I2G%ds4rcw2T3nEi(HhC8oMJNEhX3n1&8XTz(7I%_z(7k1a-4g7we^Vt$5Ix9a~d z%UJjpu_TheQm_`ik@2L6n@1pNBG|ff)!QHPzuAiPVk2%jIR3y|j9OI?-}36YSiLm; zfHczgtg@2Wh=q4++;4<33qWhlm3yd;hHgN={^|WhKPGm-dv(iYIHD2zz~cBujKlT= z$#X^}rzDj!HXo`!A@IvQZ_rJGrCM<1?QE!~jfNN5JX){}rUWf7miu)O`d&}1h9hu} zQxn7uNXbmrF*Q*mwH=!behGql&?>Ky%1zaT@dHv)bm+opfprWMlRtoNJm@^R0$Vpz z)3r#Z-kG@u_~`g_Y)k4AD_A$895#+`*Kj5%3-R)yK~D4k#i0~Ok(tqIptA zO5%{F{gQel4^Bu+YS1fD>Xf3Xi}jjM8i%h!gt*qKw8`%7_BWP)uUqFMiwDz@i5_Kg z(b0{b-iiz@m;M7XGSd5HCUuvpST2z~Tr5-)#dtsQ{nZz!)mx|->ry(;vwr`R!T-{9 z@(4S#07!ENFW;r?2CA90PEPBdKBzw=DNox>lthdx&MiLQ;0_3HLmGa5Fn3T{VzIs5 z|KDn!;`eKfE7bfLYUT~^=8*rv_3xVgcj3kwe=t%WuS>7$(0v+AP%{#pMsQXC?kOoS z!jYP*KByJi`YovZr48uzm!=;*kE5?1!st12C%czDX%f2`+R0hBbiM8G)#QDWhIEB? zlhbAgVkPCQS z@M<4xtkX}85&T8iV$)f+nfPrMUOp7C!ot?=7{OH>I3|!!CF20e#migy`3f?<{@oH& z5(AMiSzgYASi4VAW@~p+Oe(r8M0O+ZB`6zF&zl{?YZ4fdx4_X;HpLAUstPY2r`noB`2mN z|C`t~C5gn373@Uizw+I1jh^YJ8*X^JydNwhY0a5kCcPVZYC2$T;0k$Ihnh07Q4PzC z-TL|pU_OX-c$jw`8F{CM<-EAM!XJHE0sG-$VfAIizlGo=`rn<(8Nqi@33@2VldIOc zMkFXhupqUU8C4?pI!~IO7XbGIFW(h!MN9aZtBedu=l}4ZpVJsolxtY{HT2&f z5St&h55#DQOvr&&Y?saT{SQ`tMgg-RlnW8)9o%4xjqxf{UND#Krqpo8DrTCft8WYjk0!WH1V0MGLeX{y0v-*y|8q?z7w z=I}SbrP>cM zVsRC}DM&$GnF*1%WxAM|YTz{os6xChc(u>@mJ&l@pOFbEZp&o7*m{MP103c7cyoU( z82{S*j}AVz9W$8c??<$}gkR}Gk_U(P?x=%bx(kIA9AiQ?de%|E*DqVw@k&7KMp1R*=_P=59vv4;wNB1&Bn+KF43a^s znnmc}Sp;OCi0%O?XdJ<_Xg!8!VN?#g8BgH^HSGqubq{VBiIRJw=US5k1f1iLj;ls&Pd!xhreHn)~KCQ7epu?{elSug}z>oPOTOh^!sR zh+b3}b}k1*8(>;|tIOtSh_FV~nc?rhIl7u2Go||x_&R;amhkDFSu)=H-vgbqF+uhW zD&3W)&C@^yGHtNnAgZf>|JFDDftHP+Wz0%+o>41w@NWmRy>MrWe_m+@u_NFUVsy~R zF09wVw-_fgCkI9%G#oaR^(X80+fCv@xeBG#<0y?c{e@CNiiWc4x2%|{oQ~=jqxzv7 z<%z94rM1>QT-kQM4&BrL5k*25I?vwiIz(HCu}#TePKNBC1T3$NJ;9d z$8d|*_UqSy=n07UnDNAMiRjfm_^?|iSn~{s$jd&89(M^n&Xbd8&q)d2v#TH{H-@k^ zo4W6Re5nK8mgq8kXS;Y#gIAHTnlcr!{lW^lynpnlt{`$MLzH$=L*y@-^kZMoPIPFc zw@8L_@&sLxplhEYW*OyWY9sbzW9l|iUKeycN`yjsvD7skRG}+Snr&JZb;yvR2&|k@ zPrkis-Ov!(N!QR&uBYbq-?|B+oPc5R*=Xe>3GdpCdFnoMj{cwnGYwKaIH^sZh)|=Z ziTx5&5ICkbWhBw>SLpo!_Y`pF8|s5ZLejbkmZDB7c0J%gz_k#sEyq3RxyaqKMN;3t z+vv^dkJ{ISDlemysw8zf;);Z~MlK}qC2N|1mlqn*RRnW14L^FXc_mfy!h7JW^kw*J z(-(}TcY?TrA>T7C-b#4zSCLwuV3Zm^d(gA}!2TebiGp~dn(wtlhW(NTCiUx29tNqT z@iO%DgST|SZ)KSVF!`tz}GLM(odfxylWCuX~3AyJ6Zr50Et34 zm0Eq3py!gXuX)6GBYTpS#Oj`XcJaF=AQ}!L-XIFTX^5m_Tz%Fpen&U$T!SbzIsV;# zP^c>rlCc48Z!gxZe%rP&e+bs!KvSE;zc13@D~s09yh@(?S3viJHSbVEi#1SFGG8H6 zkb_H@=(+kUm6JfU3H+;b{OA7B5Sb8{RKZ#-q(!?}2Jgdro)kkh4YnbvEppP-C0Ek$VCmyHWK??N8XZJN1d z)OAYE(b$qKbaW# z^nw}%i1N~SNA38Z8clWd3InbmtBl^zIrqIOm~MDIp1H6Pr&G-fB@zsl4k z)mjI?ZZyLOopY;n5LkLqC*Ny_+i36(q-YFt2#@U@+Pwglc9j6oX`dyw8oViKJeFyy z!?!joRsuvPz`UP&Y_G*DlJIe@7D<^cldyV{m?@?a#7>hmZ?@?pkoG~5e9wIDpd}}( z$Fgv&&!p^*G&IVGjw3*)eFB{{cw-X3UWUt)UoahywwXH@8Du@$qI@{%~rsd`Xaco$!M{F|&2x}>Lm9>fnON}MG*NI8(@`rDubd-k6L4_4$U`H%jAV@$?8V;; z_pUG}0{y*QWu4!bZj)JQtn312uFv%i3o zGDET%R~U<0tFh_oDDq(rw01N|2b4R7nfbDHXM^e4aRsItyqQ}C>)?$GP{~QBm?XDU zb|*3*3EgfPkIHJFh)^A(R+nWeb9eW|wDhzd1Jc|FWY8T5-2=(!u56@hZJc3B&pM<) z0G^L;?f(xx&7v8f4OfngPlKy{iinPP>xp%FK2~IC7VA8Xca7aevtgGZ0PhNGG|(YB zJC*Gd{_550=nE9G&$zV1`4h?GO-aNw*4Y}khgG|P%-ldE9{9O$Ok)j^5h*#$@GXp{ z+AfAr*=5=|t(AnIIRmjs%yhv*Y~IGljvtnqw`&`n-*cx6(pm0RsYyAHz^f&M=9_nC zTOIt$molZcd7qNd_O+pv9;hwfEZ;lo5Dj@JGylIfuetlzV{s^)PBHmr+1Q2WW36qL zRdv>e&}i&JnP%zMU5Du1XPFs^_ynueOsB5vTZK-A$oFKW8_2l_Pid|V>|5ox82%k* zV*~%mH;}xKhDeIzyyy2aUJc1)s8XpBZJ-1VD4{bQbu#Q~H`~TEM5K`lnqTK>KUl&? z%oL=Lgpy3v{1Q9$|8qGSk#x{yUP*?!vw)%5Ca0PF_(3?Eu3he=R_Su7cnT-YMo z68$>Wzw}H*7%4$p&ti6G=TSSjsv$NuAydTr>mk!5=(<@f(XeoG;-0CHb$S^SHJ&3u zWwQlK(K@#`h;)~%^L4iLQ|H-i=lj(khkD6TU^wLConQDo33_xk%Qv`M)xK(iKI9Ik z$2ZYxY<8r|x!_=Itj$8-svMF*FR$HHrY=TnDBgn%p2H?y-4ic|A46T}1v!@oGA|OTwqlWz(4M8!9zD0Jhd;EcxU< z3UwzZ=d#wG+`1V#3WI|!={$2! zN%)j`EG!!y5xM3T8mku6$!oOQ6%jfw%-B@xY7YMRQJiju6#xvrJUr=v8eX2b5oXZ? zKj>e41^8#^f`|LO(ZWAO85@;3MtSCmOW!OQn>);#Cr$aOC5?zfy>Aj$@7d-#ySJjY z&d`;+POY)>ZbX`GMoK8RrLmlRR(t!y{jO$HogRV*;u9AC$TWC=t@G*T+s|) z$F~zC{LE!&lJJErGo4ZI@RlVI>M>rv(sxx_e8~UGGPHU*WNr6>yI(oia4N&^e|&sA+nNpVYr?qopQZ z-I=Fp*gP-eFrR*G-p;51N_E3OR3ctQnmz8=*G5lQ&M%5IGSc7rsW57jt<&F!2nC z=(!M{Xk`x#ktcZ-#MHIKop>z=wjCtneaF>a8oY`GEMq?ZrKKv69^pHH+VIiKxjs7h zrt6K%lgVnL@pY!=g+Ztg5c0E`pfsLHMLsNJ)h`@Aa?Sw|$x#yDna`)|5JmjQ=3p&r zeoCUF5uFP)(%LZo^}Mf$pSK*h)~x-H4WZ}S_{ix2mdR*BJ{@(ezZx!Gk9Wwx<37C_ zo=%$h#%@1IjkF@?x5KGu5-QuD&GV<(s|?sMpRBtvSWDx#$Jm7Qh-IoBRqVaD3d*3X zI{YBLe5go69g&LkUdb|KladPUrzo*(3zcH9)_J^!Yw!X&wvzFWcIdpBE@0Bz#_Q>z zVPVsC@Zncc5ua78Gk*GY_1*vwO#u<#J&tCTCR(_PHO0?g$tm>KVp@|1TuCcu@g<&H+-Rrb$pp2(P_uV&(Xtl(UQE|?Tz!Gur8jLYVUWMpDXrU;*kS0~ZT zFs$a;l<`ia5Pbn!aSj}n7G>by~oEkAQ}K7UbDl=!#zk`sc9;bvyWIs6bBat zW+FVe0`uYptNN5LNKQ6xVA|@F9q&UkYSv}UQvV|*!ioAiCZ^=vCRRw#U2E6pDlifu z*~uH@f=?3uU0oAw#Zcc!vu32F|2goV0Y}FB`&r1^??a5=w4;fP4B5=&xggxQVh?O{ z7%=ZvcOz}j@Ft{vyosE2*ujK)XKd8M6-4bpR4SrCB3?sOa|??_Z(3PS@c1i!kC_g> zV;j`u{7zQLiGrdE^peda6vAhE>#8(FmSpNKhMzLA!uTq{cLAOsw9X@4Eef?&kj|FJ zx^%l`2WTrm`SeIzB!xBdr}Z>3CdIoM|MsdOT|NI=i7Sp0k+6-WU*?q@Yf6=Rvn+q4 zO)3A3ZEPm_aly@VRF!3=zKV3GMwDCI*dX%3=v~}b^aOfljaPkZWDhAma66MCx%FU2 z8d<$Y=j1(LUZ4b>xt;ZylX*Y3jRv$423KAo#z+V5!Q_xbd8>a} zO@fY`gJ#qmB+XlXYU3E->zBbFsVU*REHNohPVHm{`R#$f+en7zQxdsI4pLE^cOkNm z*kod;ZTo763{U(EA{_K6-F=Y>q(n}8O)8Lx-K-Fc-2IEGP_=>Qhwhg3B>ej`x^3Ss zd}ZUIz?ZFb5R!nDf8HKu>&1yGhr58%>s5HxvPKfr=Q&uX?qy~*YHpjZ;=oo$^l+pK zDeDVg;1I-n+iAQUmn9fX2Y+lhFQtF`IEoJwqv~(%4-D#qG3<5a(oVcWQ;KpUf%};* z^bT8YCf>mtj?&{7UN-c1O|L>cke=6@&#}5%gPj2BB*zd)kxUsbqx}ng5FVrF9_}@-1JxQmhL7Ww_y!{78tY)Hl)k}1m0Nf6#T}hcy#C_Sy(^}Vh%Zk+ z&8oeCp!2T>izA`8xmKm6Fv?}Ra5BZ6oCjHBpFb3}f{smeT^KE+^Q>A;W*roqNWu|j zM?&kFeToZ6(5tJ&L?nFRl#v)rBU|eZcj);J2=nUtw4j9c3Ta)X6m{`4tENF z%4_=3ISCrk!Bo(W#wU~Vj?jSZK3-Ry)>RYMbTqr5z-QhU7f!s`n&l z|C3DMx=ttW`+*~MGrVa$en6oiq6I0KeEA%> zxEU%!kHGWwp7vCNI-gsn0DDWzi-SeXmX4Qt5;KL?;+;qe6=Gu;Yrp?wV^fL&1(D`}%e?LjRbGX4} z%2`&yYNwlwq?e}UV3g-0tlq^GT7^uR2G2%=D{(vdr8!f&wg-M1@canxt7k?MrI@v{~=8NjXpmhX68U@0;$IL8jbyer9>O|??C zmfz(JyKxcxMK5e16PceFHoY}n0C7b$!!t*~C^7G_6l$RZt4J{XQH>w;n0{Hwk64(I z!{>zv5tbX{wxJ1N6Uu0Bq$T6qi1j3EE(qo#-z0;A zQ@OB^D`f(;F76Cl?xJ8*pAI|6J=RpG^VAcNfgcvSc$u-&M5&3Hy#nHKfEPOt++)%* zWBTJiNjrd6!*|JujM=Wl%ezF{nT=JhI*LAnJm&&3wk zSx9G_L}?QG;hb?&caaNGTovSXX=SR-nr1!;q#N<_iRYJpA;&>AS{soS>HI~gB%XM+ zel+PT^&b!L+Thhb$Nb8QAkv9Sf8pQQYA(P#@Y-vM@UJcYX5_*>QwQqFe5*&2>~K(wEpJt711_y-HY!r{{)&5)i(_99j{~kebFmyG@N#w8K10b z*XO3N)S9NHnV+qyOGOv$#A58W_DoAev(apr3nz_dOacMPQ z4;2Y2;p-Y0#Sh$PqyDqcpHe@9duO;Tcdmqvh>>p#)}&k$Hs#juV$@F$(5t93--Njc zj{&V9F7K0sV3p+WW$D)!`GSB7#1nkRqB`aMZ$}pBz3^RpieAf57TPaLg4*9^P{+@Y zU(r)$W6E&-6fHrq?_F9+HJbmvk0GEp%1E5kghAC@PtEjPNY4=9gl#E3d_e$BW!2foW0?uL;Gd4stU6J6NUCHBWqWL<)~IIX-kHCjlcJLk1h2g=`?^Ih3D(K zF;OZi;vTE(`088p=`~R}vswyTjJ0~5qW;Ip0Hw*pqjN#u7N+aKSz^0hQc;?<>}K12 zjOi^4p8@4(o3&CzsEK<6dK*N`AsI~beAs@QaX=EKQr>j!_1kO%o5DRtw3YfPtScvG77O3XhDB6>H(IZU&XLd{^-H< zHS@uPUg^e1TemYQ)Y(wy7^oO>7~Ma)paCe#F3_DDC{mEsg>1XW!abwEkn}?)Z_0x|5HL+WqVMDp{8CO zb>?hR@=B;Kiu3YaZM|DS(c6ifIh02>McI>#*Mhlthl|5(#6NT7$>go@` zMScNs1Nx;7=og*XzZXdeS2>7!U)tf**1r>{&;L6yX%?w+jHi0-)2S#lvhwh8z(MN{ymnX>qQ@Du~GWGc{=qh6`> zPzs&Y@kUhczyCq}_cn5Fmj68hQk;WDVOp#6TwY0Bz6$Q*aau?D)K-I__}9=uMI8(> zo$Ji>2KVopnH&(Ch)Yvu$puU4Znddjafg{QkE{j{=`R=PU&sdX`m6e5Im&PN%hrv= z=bPXtVdC%?gOp8XB=D%2y?BYS7p?C1$5~*tfCni4xAJH3q>NAnR{XozuSVET zt^XTB{fBl#7D?yH-Al?92_90tsuY3x;XX5R_7bS+1>R!qLbq7Gm~%5Per536-S0PCzH6fTE6XhwdAs-*h@ z_OyDxKyJlXCCM6i;Mcin-CY_@u%t;DYZyMi!r0+F5sKk*-2GJ2-2hv651D)uFgwi7DsjB)T!O~ERnnab z*9A&<~oeQS0x>%-+rmnF#1`WPbfT$SJ8bYNN984=z1`t~j3MZvZ^KMT3_jDG+Lm-7C*3!XlxT@$W$nJ-zJ-N$ZUa@SEMUHNb%@8w}S3_Zuor|(a{#jJa; z+=&dnt}xeb&2S1xzTgUQ0bV}-==+{FvNbs$qLRMY6P;MGSw3LzAvn+b!3S!&cH}@q zm2^SHk?X%lbOzJ~P<}n{*C!@-vtU(4u@AMI^f8Lw4Ky1sKY7{p1=uK+HOcR

    =`i z%#eF+Gd{l);G1O>81RiL@Vp08;C#Kp1a-zYo>O4N4<^WbFO@)|_QpS8q)NIzWl`wQ z#(f}oZLsEt{s}5M?O>MV&to}0`&ymxT^>I)w=ckQ^h&;_4K3tERgC*qjhe9i-p^)? zGp7z@2=M%5K;Kf5SsuaICTGo17J=mm_yfKVUA2-EpVdlR;@^NZr~SkK&P9Q35ngAm zYLy)1;@$s*sZc6Xxyi#b)&%4(--5FFmUY4x~i*I=;Zrw&3WM8l~MHG+S*n$C&x|L zma+Ot7sL0X!e3C~aISzSb*SvVWO};1lJ*>CV$Cr(XcFKpX!w?t*B2Eu*PU@5`yga} zWi;Y)x6;Goyht+9P~{;C9QCQ!cIy7`o7Ow&w`WN3MnH2=HcBpX+a)tsI+KWXloM)idT=k+zJvm|Nm3WX5~ ztinwC=dj|rdEmblG`x#%+*K}l;U1SgtxYBVm3M(7ndXXXF?6W!miE zsgy1i5G(xX+CjTPki8o;#2Z$0f4MXoLqqPw68%S9VLNnVP8bO?QCS*P9VjQA{*`-@ z4EP0+*~QC#pXkJE12o8|V_Ofx?Zw4{)CJ^sCzo?WnWq5bl{|2WoQ!r;N#XE`6@hE0 zAMXWE!-pVGhsw#GP-dWg@8+*vheBV0V;!!IXOEN&4x^S7Q|)~k+S&)O4ydd%2VFf{ zPC{z1I4NP*(N}xHkq+Lxe9q&TqkrJTM&u=jcehUhb~UiPHF{2f!PuxgMa^su-(*TR z`cc=$ht8KL%1K0ZmAeR={bv8ho`BFrS1j-7JU)}gh|2B8%i( zI34%g0x4Gl@6BOv&X$wL)l||nT)*5QlQH;t4qkpbT9{1=3|Tn5`0437bTybRD#UQi z^5*Jx(Gjl{9^_XZ6QOF& z(9~yu++1{FB0*N`O+V>f{d7kPA7%OEFoJ{nDq9ho);isVerEjw+QEeru} z;FZw90H>k=-gGLhl)8a)J(ZM!9(;4fKzefOQW+=raTV!nq_QGo)rZ=j)e7{t1jje1 zBHud8*GpUxdzB{{v*eEAp#)JUonE-c(T+=RKjWt^fg9zdu&PS>492`O9llaW-V*%z zZuWQ!X;VSvD>7?xcg@DzU>014+3sz)Wd4oZo-DwpZA9$g)x$c|&mP_ZN8T##>|$0C zFjRqaz_?F9B#4^R^>TF(P^ab2wVv5t*3wN*BJ$B(C=h>1cD6UbbOnKr>$CTfwQVzOvurh}odRR{Gc&S3fu-pCa=`@<%yHo*TD+lXzRIUlT8`GC#15W~W)T}gg#x)5# z&6gQK=U-W`7NB6s=n}DKWHn%) z;N=JJwvXlHR|6Km&UM{h>+cqQ^$8nPA_O+5ST4=;Sd5}uLRW5Fg+@KYHiCO*?!@mm zD?VhF&0>9EM?+D(mri>rCoWr*RmjG-414nMn5yx>(sL3n9N&|#Sx-r-%OdA_cRoC! zSG0rx&rd+QzoTvCO9EWkK+)sh9{&D-(aRnA0W0`}7|S_`pYaei`!sPwVNI~5Tg&(k zxc-yqYNVYl9cEuRPS5^Kz$3%`ae zi33}j`Q}yfOrIorLM3-SSvYhg@3)d}AWsL}gHs8?2k8neRaM2>jfN^iwIkZXe=GyqS?IZ!w{^Xhz}k6%eo@BxuO7y)J|;E! z)0}zq-toQ|59)Gqm^jet`I7gJa%n;?DtFq2`sQPh-T;C1zv^y6 z0X3mNULUr;T75`o$|FCQ%%)33U7!k{w?}|vVjXY8+Pj3S%*1Q%SJe4=e+%M^y^7K^ zjqO1Sof%jC^mOuY7fbQful%}dD=3CWiTu8*HKC-yO;t;5_V1DL8@-IN&VU&VKj&*x zo1|A}bC|E6`!}ZtlIYcx^*ALTiPW(yeq+3VgzDHiK!2xJsdT*co#}|1ELVFgz4Dy zP+3)9R6t6;$GP1=?FWhvdRDa}N`EBIW~fxs-58r3dat8v;&((wVAROxQ3O1Xqqp&a;QP)BO&};P@Rkk)SM^NX@(t{Jpcvk5 z9_|K*SdGo&a`KC2FF_v)L^jEllSV%Y326&VAnE8LO5S4DyCKwW=;s`GXYnD0HFt^z&)_WTg5)1zB8+jl9L5e%nli0uz97=4#nvu!20R&JwVr=iJ&k5u7eT z-8?LO0(EZoWV#UxJ9CE&GNbaIgp=S4t~iC(&zd}_&lVL-qkFDPgN$_Y;ldTZD~mW+ zR!O_SnqU9yK|_ucK-J(-nKQ+%A%%wF|1_;}5U^Ecm`$3^+Q7RC8?pyn@@k|2j;;?! zaV+O$Q=PIQ5t&>CX;nuheR{Hhcpf_nB5OEoOOD8zkW6Vv1tLTtXTQuBMzwMj4qOm;CSp0sGoof_i-)OlVIS=P3V)yFJ z4Q_|da01?euYZ>f3Q}Fc)O7t=?Nc<6{}+P#aB3TaAI z7miW96B$v=<{sx?roW-V;cN)OJKu#@q~Y~-5x(E7xkP7c{^c3e|NRR*-`%sWNx`8_ zAhWb+Q@tzM=(!!SvcOLU6K_z~R%F`_wjCGrf^7IbsY}b2Fc?+6FrQ6K-q3rEg&I0#C0@L=V$>J{ir60{nI_xl)X*#5P<#%C{4}sNJc|sGu@Y zO6w4xhcY)8@n-Aqzt6~X&>rF*(vklved(v#~TmPp|Ig`$?mkk{p^u;`(a2ALw#q2XrZZqTjNX zQfCz$%aP7n7qci~_CKRhKoThV;P+UVQd%`y*i$KJ^QPnTn(n3%-%S)#ovZ2Mx@2?> zY0|GEe=JoNespHx-s9HLKERtoJG^~*)l=%s$QqMo5iEN{Du3Ncy)R9nHsp%9(SXT> z8FmfPq}Nz}-&7AcKWLQq)*Bn5{fluk%TyZ{b!<;}2=as&58+Pvp`aSRGeAAVD}S~a zbjbaAf{aPcr`vYND#`u3*fBI%eM!!fA44101(trBkZ&WS1c`0>XUtJMTe3Di6FlZ) zdx$#_w!()V%bJmh2(}$4e{+Wk8BhXsW4zuXTZ_KHl4SfA_#r%}Qf` zv?1;5v9wgt@OyU_LdF`XK5zN&|0V4juyxbp7dxicLb<2$^6u^O2ql?jsggdzT5HnD z`*gZOBL{wh*=3Ye>GoD^t*&i9n_;a14F#0v*nf p + + case p @ InsertIntoTable(table: MetastoreRelation, _, child) => + val childOutputDataTypes = child.output.map(_.dataType) + // Only check attributes, not partitionKeys since they are always strings. + // TODO: Fully support inserting into partitioned tables. + val tableOutputDataTypes = table.attributes.map(_.dataType) + + if (childOutputDataTypes == tableOutputDataTypes) { + p + } else { + // Only do the casting when child output data types differ from table output data types. + val castedChildOutput = child.output.zip(table.output).map { + case (input, table) if input.dataType != table.dataType => + Alias(Cast(input, table.dataType), input.name)() + case (input, _) => input + } + + p.copy(child = logical.Project(castedChildOutput, child)) + } } } } diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala similarity index 97% rename from src/main/scala/catalyst/frontend/Hive.scala rename to shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala index 706c2d8664188..c973ea5ef0efa 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala @@ -1,27 +1,17 @@ -package catalyst -package frontend -package hive +package org.apache.spark.sql +package shark import scala.collection.JavaConversions._ import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ -import analysis._ -import expressions._ -import plans._ -import plans.logical._ -import types._ -import catalyst.execution.HiveGenericUdtf - -/** - * A logical node that represents a non-query command to be executed by the system. For example, - * commands can be used by parsers to represent DDL operations. - */ -abstract class Command extends LeafNode { - self: Product => - def output = Seq.empty -} +import catalyst.analysis._ +import catalyst.expressions._ +import catalyst.plans._ +import catalyst.plans.logical +import catalyst.plans.logical._ +import catalyst.types._ /** * Used when we need to start parsing the AST before deciding that we are going to pass the command @@ -30,22 +20,12 @@ abstract class Command extends LeafNode { */ case object NativePlaceholder extends Command -/** - * Returned for commands supported by the parser, but not catalyst. In general these are DDL - * commands that are passed directly to Hive. - */ -case class NativeCommand(cmd: String) extends Command - -case class ExplainCommand(plan: LogicalPlan) extends Command - case class DfsCommand(cmd: String) extends Command case class ShellCommand(cmd: String) extends Command case class SourceCommand(filePath: String) extends Command -case class ConfigurationAssignment(cmd: String) extends Command - case class AddJar(jarPath: String) extends Command case class AddFile(filePath: String) extends Command @@ -208,7 +188,7 @@ object HiveQl { def parseSql(sql: String): LogicalPlan = { try { if (sql.toLowerCase.startsWith("set")) { - ConfigurationAssignment(sql) + NativeCommand(sql) } else if (sql.toLowerCase.startsWith("add jar")) { AddJar(sql.drop(8)) } else if (sql.toLowerCase.startsWith("add file")) { @@ -436,7 +416,11 @@ object HiveQl { val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) Some( - ScriptTransformation(inputExprs.map(nodeToExpr), unescapedScript, output, withWhere)) + logical.ScriptTransformation( + inputExprs.map(nodeToExpr), + unescapedScript, + output, + withWhere)) case _ => None } diff --git a/src/main/scala/catalyst/execution/ScriptTransformation.scala b/shark/src/main/scala/org/apache/spark/sql/shark/ScriptTransformation.scala similarity index 93% rename from src/main/scala/catalyst/execution/ScriptTransformation.scala rename to shark/src/main/scala/org/apache/spark/sql/shark/ScriptTransformation.scala index 946c5061a9e5c..f86650d3faf3d 100644 --- a/src/main/scala/catalyst/execution/ScriptTransformation.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/ScriptTransformation.scala @@ -1,9 +1,10 @@ -package catalyst -package execution +package org.apache.spark.sql +package shark import java.io.{InputStreamReader, BufferedReader} import catalyst.expressions._ +import execution._ import scala.collection.JavaConversions._ @@ -18,7 +19,7 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: SharkPlan)(@transient sc: SharkContext) + child: SparkPlan)(@transient sc: SharkContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala b/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala new file mode 100644 index 0000000000000..e118cd2d6d108 --- /dev/null +++ b/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala @@ -0,0 +1,270 @@ +package org.apache.spark.sql +package shark + +import java.io.{PrintStream, InputStreamReader, BufferedReader, File} +import java.util.{ArrayList => JArrayList} +import scala.language.implicitConversions + +import org.apache.spark.SparkContext +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.ql.processors.{CommandProcessorResponse, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.processors.CommandProcessor +import org.apache.hadoop.hive.ql.Driver +import org.apache.spark.rdd.RDD + +import catalyst.analysis.{Analyzer, OverrideCatalog} +import catalyst.expressions.GenericRow +import catalyst.plans.logical.{BaseRelation, LogicalPlan, NativeCommand, ExplainCommand} +import catalyst.types._ + +import execution._ + +import scala.collection.JavaConversions._ + +/** + * Starts up an instance of shark where metadata is stored locally. An in-process metadata data is + * created with data stored in ./metadata. Warehouse data is stored in in ./warehouse. + */ +class LocalSharkContext(sc: SparkContext) extends SharkContext(sc) { + override def warehousePath = new File("warehouse").getCanonicalPath + override def metastorePath = new File("metastore").getCanonicalPath +} + +/** + * An instance of the shark execution engine. This class is responsible for taking queries + * expressed either in SQL or as raw catalyst logical plans and optimizing them for execution + * using Spark. Additionally this class maintains the connection with the hive metadata store. + */ +abstract class SharkContext(sc: SparkContext) extends SparkSqlContext(sc) { + self => + + /** The path to the hive warehouse. */ + def warehousePath: String + /** The path to the local metastore. */ + def metastorePath: String + + override def parseSql(sql: String): LogicalPlan = HiveQl.parseSql(sql) + override def executePlan(plan: LogicalPlan): this.QueryExecution = + new this.QueryExecution { val logical = plan } + + /** Sets up the system initially or after a RESET command */ + protected def configure() { + // TODO: refactor this so we can work with other databases. + runSqlHive( + s"set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$metastorePath;create=true") + runSqlHive("set hive.metastore.warehouse.dir=" + warehousePath) + } + + // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. + // TODO: Move, or just use a properly configured logger. + val outputBuffer = new java.io.OutputStream { + var pos: Int = 0 + var buffer = new Array[Int](10240) + def write(i: Int): Unit = { + buffer(pos) = i + pos = (pos + 1) % buffer.size + } + + override def toString = { + val (end, start) = buffer.splitAt(pos) + val input = new java.io.InputStream { + val iterator = (start ++ end).iterator + + def read(): Int = if (iterator.hasNext) iterator.next else -1 + } + val reader = new BufferedReader(new InputStreamReader(input)) + val stringBuilder = new StringBuilder + var line = reader.readLine() + while(line != null) { + stringBuilder.append(line) + stringBuilder.append("\n") + line = reader.readLine() + } + stringBuilder.toString() + } + } + + @transient lazy val hiveconf = new HiveConf(classOf[SessionState]) + @transient val sessionState = new SessionState(hiveconf) + + sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") + sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") + + configure() // Must be called before initializing the catalog below. + + /* A catalyst metadata catalog that points to the Shark/Hive Metastore. */ + override val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog + + /* An analyzer that uses the Shark/Hive metastore. */ + override val analyzer = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) + + def tables: Seq[BaseRelation] = { + // TODO: Move this functionallity to Catalog. Make client protected. + val allTables = catalog.client.getAllTables("default") + allTables.map(catalog.lookupRelation(None, _, None)).collect { case b: BaseRelation => b } + } + + /** + * Runs the specified SQL query using Hive. + */ + def runSqlHive(sql: String): Seq[String] = { + val maxResults = 100000 + val results = runHive(sql, 100000) + // It is very confusing when you only get back some of the results... + if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") + results + } + + // TODO: Move this. + + SessionState.start(sessionState) + + /** + * Execute the command using Hive and return the results as a sequence. Each element + * in the sequence is one row. + */ + def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { + try { + val cmd_trimmed: String = cmd.trim() + val tokens: Array[String] = cmd_trimmed.split("\\s+") + val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() + val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) + + SessionState.start(sessionState) + + if (proc.isInstanceOf[Driver]) { + val driver: Driver = proc.asInstanceOf[Driver] + driver.init() + + val results = new JArrayList[String] + val response: CommandProcessorResponse = driver.run(cmd) + // Throw an exception if there is an error in query processing. + if (response.getResponseCode != 0) { + driver.destroy() + throw new QueryExecutionException(response.getErrorMessage) + } + driver.setMaxRows(maxRows) + driver.getResults(results) + driver.destroy() + results + } else { + sessionState.out.println(tokens(0) + " " + cmd_1) + Seq(proc.run(cmd_1).getResponseCode.toString) + } + } catch { + case e: Exception => + println( + """ + |====================== + |HIVE FAILURE OUTPUT + |====================== + """.stripMargin) + println(outputBuffer.toString) + println( + """ + |====================== + |END HIVE FAILURE OUTPUT + |====================== + """.stripMargin) + throw e + } + } + + object SharkPlanner extends SparkPlanner with SharkStrategies { + val sharkContext = self + + override val strategies: Seq[Strategy] = Seq( + PartitionPrunings, + HiveTableScans, + DataSinks, + Scripts, + PartialAggregation, + SparkEquiInnerJoin, + BasicOperators, + CartesianProduct, + BroadcastNestedLoopJoin + ) + } + + override val planner = SharkPlanner + + protected lazy val emptyResult = sparkContext.parallelize(Seq(new GenericRow(Vector()): Row), 1) + + /** Extends QueryExecution with shark specific features. */ + abstract class QueryExecution extends super.QueryExecution { + // TODO: Create mixin for the analyzer instead of overriding things here. + override lazy val optimizedPlan = + optimizer(catalog.PreInsertionCasts(catalog.CreateTables(analyzed))) + + // TODO: We are loosing schema here. + override lazy val toRdd: RDD[Row] = + analyzed match { + case NativeCommand(cmd) => + val output = runSqlHive(cmd) + + if (output.size == 0) { + emptyResult + } else { + val asRows = output.map(r => new GenericRow(r.split("\t"))) + sparkContext.parallelize(asRows, 1) + } + case _ => + executedPlan.execute() + } + + protected val primitiveTypes = + Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, + ShortType, DecimalType) + + protected def toHiveString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ))=> + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_,_], MapType(kType, vType)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "NULL" + case (other, tpe) if primitiveTypes contains tpe => other.toString + } + + /** Hive outputs fields of structs slightly differently than top level attributes. */ + protected def toHiveStructString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ))=> + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_,_], MapType(kType, vType)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "null" + case (s: String, StringType) => "\"" + s + "\"" + case (other, tpe) if primitiveTypes contains tpe => other.toString + } + + /** + * Returns the result as a hive compatible sequence of strings. For native commands, the + * execution is simply passed back to Hive. + */ + def stringResult(): Seq[String] = analyzed match { + case NativeCommand(cmd) => runSqlHive(cmd) + case ExplainCommand(plan) => new QueryExecution { val logical = plan }.toString.split("\n") + case query => + val result: Seq[Seq[Any]] = toRdd.collect().toSeq + // We need the types so we can output struct field names + val types = analyzed.output.map(_.dataType) + // Reformat to match hive tab delimited output. + val asString = result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq + asString + } + } +} diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala b/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala new file mode 100644 index 0000000000000..1cb0cdba8c5c7 --- /dev/null +++ b/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala @@ -0,0 +1,85 @@ +package org.apache.spark.sql +package shark + +import catalyst.expressions._ +import catalyst.planning._ +import catalyst.plans._ +import catalyst.plans.logical.LogicalPlan + +import execution.SparkPlan + +trait SharkStrategies { + // Possibly being too clever with types here... or not clever enough. + self: SparkSqlContext#SparkPlanner => + + val sharkContext: SharkContext + + object Scripts extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case logical.ScriptTransformation(input, script, output, child) => + ScriptTransformation(input, script, output, planLater(child))(sharkContext) :: Nil + case _ => Nil + } + } + + object DataSinks extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case logical.InsertIntoTable(table: MetastoreRelation, partition, child) => + InsertIntoHiveTable(table, partition, planLater(child))(sharkContext) :: Nil + case _ => Nil + } + } + + object HiveTableScans extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + // Push attributes into table scan when possible. + case p @ logical.Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => + HiveTableScan( + projectList.asInstanceOf[Seq[Attribute]], m, None)(sharkContext) :: Nil + case m: MetastoreRelation => + HiveTableScan(m.output, m, None)(sharkContext) :: Nil + case _ => Nil + } + + /** + * Returns true if `projectList` only performs column pruning and does not evaluate other + * complex expressions. + */ + def isSimpleProject(projectList: Seq[NamedExpression]) = { + projectList.forall(_.isInstanceOf[Attribute]) + } + } + + /** + * A strategy used to detect filtering predicates on top of a partitioned relation to help + * partition pruning. + * + * This strategy itself doesn't perform partition pruning, it just collects and combines all the + * partition pruning predicates and pass them down to the underlying [[HiveTableScan]] operator, + * which does the actual pruning work. + */ + object PartitionPrunings extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case p @ FilteredOperation(predicates, relation: MetastoreRelation) + if relation.hiveQlTable.isPartitioned => + + val partitionKeyIds = relation.partitionKeys.map(_.id).toSet + + // Filter out all predicates that only deal with partition keys + val (pruningPredicates, otherPredicates) = predicates.partition { + _.references.map(_.id).subsetOf(partitionKeyIds) + } + + val scan = HiveTableScan( + relation.output, relation, pruningPredicates.reduceLeftOption(And))(sharkContext) + + otherPredicates + .reduceLeftOption(And) + .map(execution.Filter(_, scan)) + .getOrElse(scan) :: Nil + + case _ => + Nil + } + } +} diff --git a/src/main/scala/catalyst/execution/TableReader.scala b/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala similarity index 94% rename from src/main/scala/catalyst/execution/TableReader.scala rename to shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala index 1f7221e84bfe1..5ddb8b3a1656c 100644 --- a/src/main/scala/catalyst/execution/TableReader.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala @@ -1,4 +1,5 @@ -package catalyst.execution +package org.apache.spark.sql +package shark import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ @@ -21,8 +22,7 @@ import org.apache.spark.rdd.{HadoopRDD, UnionRDD, EmptyRDD, RDD} * type of table storage: HeapTableReader for Shark tables in Spark's block manager, * TachyonTableReader for tables in Tachyon, and HadoopTableReader for Hive tables in a filesystem. */ -private[catalyst] sealed trait TableReader { - +private[shark] sealed trait TableReader { def makeRDDForTable(hiveTable: HiveTable): RDD[_] def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] @@ -34,19 +34,20 @@ private[catalyst] sealed trait TableReader { * Helper class for scanning tables stored in Hadoop - e.g., to read Hive tables that reside in the * data warehouse directory. */ -private[catalyst] -class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf: HiveConf) +private[shark] +class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: SharkContext) extends TableReader { // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless // it is smaller than what Spark suggests. private val _minSplitsPerRDD = math.max( - _localHConf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits) + sc.hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinSplits) // TODO: set aws s3 credentials. - private val _broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(_localHConf)) + private val _broadcastedHiveConf = + sc.sparkContext.broadcast(new SerializableWritable(sc.hiveconf)) def broadcastedHiveConf = _broadcastedHiveConf @@ -165,7 +166,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf }.toSeq // Even if we don't use any partitions, we still need an empty RDD if (hivePartitionRDDs.size == 0) { - new EmptyRDD[Object](SharkEnv.sc) + new EmptyRDD[Object](sc.sparkContext) } else { new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs) } @@ -178,7 +179,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { filterOpt match { case Some(filter) => - val fs = path.getFileSystem(_localHConf) + val fs = path.getFileSystem(sc.hiveconf) val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString) filteredFiles.mkString(",") case None => path.toString @@ -197,7 +198,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf val initializeJobConfFunc = HadoopTableReader.initializeLocalJobConfFunc(path, tableDesc) _ val rdd = new HadoopRDD( - SharkEnv.sc, + sc.sparkContext, _broadcastedHiveConf.asInstanceOf[Broadcast[SerializableWritable[Configuration]]], Some(initializeJobConfFunc), inputFormatClass, @@ -211,7 +212,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf } -private[catalyst] object HadoopTableReader { +private[shark] object HadoopTableReader { /** * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/shark/src/main/scala/org/apache/spark/sql/shark/TestShark.scala similarity index 89% rename from src/main/scala/catalyst/execution/TestShark.scala rename to shark/src/main/scala/org/apache/spark/sql/shark/TestShark.scala index af6264cb7afb3..829fb1948d701 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/TestShark.scala @@ -1,5 +1,5 @@ -package catalyst -package execution +package org.apache.spark.sql +package shark import java.io.File import java.util.{Set => JavaSet} @@ -16,12 +16,14 @@ import org.apache.hadoop.hive.serde2.avro.AvroSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.serde2.RegexSerDe -import analysis._ -import plans.logical.LogicalPlan -import frontend.hive._ -import util._ +import org.apache.spark.{SparkContext, SparkConf} -object TestShark extends TestSharkInstance +import catalyst.analysis._ +import catalyst.plans.logical.{LogicalPlan, NativeCommand} +import catalyst.util._ + +object TestShark + extends TestSharkContext(new SparkContext("local", "TestSqlContext", new SparkConf())) /** * A locally running test instance of spark. The lifecycle for a given query is managed by the @@ -47,26 +49,21 @@ object TestShark extends TestSharkInstance * Calling [[reset]] will delete all tables and other state in the database, leaving the database * in a "clean" state. * - * TestShark is implemented as a singleton object because instantiating multiple copies of the hive - * metastore seems to lead to weird non-deterministic failures. Therefore, the execution of + * TestShark is singleton object version of this class because instantiating multiple copies of the + * hive metastore seems to lead to weird non-deterministic failures. Therefore, the execution of * testcases that rely on TestShark must be serialized. */ -class TestSharkInstance extends SharkInstance { +class TestSharkContext(sc: SparkContext) extends SharkContext(sc) { self => - lazy val master = "local" + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests + // without restarting the JVM. + System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") + lazy val warehousePath = getTempFilePath("sharkWarehouse").getCanonicalPath lazy val metastorePath = getTempFilePath("sharkMetastore").getCanonicalPath - override protected def createContext() = { - // By clearing the port we force Spark to pick a new one. This allows us to rerun tests - // without restarting the JVM. - System.clearProperty("spark.driver.port") - System.clearProperty("spark.hostPort") - - SharkEnv.initWithSharkContext("catalyst.execution.TestShark", master) - } - /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") /** The location of the hive source code. */ @@ -75,6 +72,9 @@ class TestSharkInstance extends SharkInstance { // Override so we can intercept relative paths and rewrite them to point at hive. override def runSqlHive(sql: String): Seq[String] = super.runSqlHive(rewritePaths(sql)) + override def executePlan(plan: LogicalPlan): this.QueryExecution = + new this.QueryExecution { val logical = plan } + /** * Returns the value of specified environmental variable as a [[java.io.File]] after checking * to ensure it exists @@ -98,7 +98,7 @@ class TestSharkInstance extends SharkInstance { hiveFilesTemp.delete() hiveFilesTemp.mkdir() - val inRepoTests = new File("src/test/hive/") + val inRepoTests = new File("shark/src/test/hive/") def getHiveFile(path: String): File = { val stripped = path.replaceAll("""\.\.\/""", "") hiveDevHome @@ -109,12 +109,18 @@ class TestSharkInstance extends SharkInstance { val describedTable = "DESCRIBE (\\w+)".r + class SqlQueryExecution(sql: String) extends this.QueryExecution { + lazy val logical = HiveQl.parseSql(sql) + def hiveExec() = runSqlHive(sql) + override def toString = sql + "\n" + super.toString + } + /** - * Override SharkQuery with special debug workflow. + * Override QueryExecution with special debug workflow. */ - abstract class SharkQuery extends super.SharkQuery { + abstract class QueryExecution extends super.QueryExecution { override lazy val analyzed = { - val describedTables = parsed match { + val describedTables = logical match { case NativeCommand(describedTable(tbl)) => tbl :: Nil case _ => Nil } @@ -122,28 +128,16 @@ class TestSharkInstance extends SharkInstance { // Make sure any test tables referenced are loaded. val referencedTables = describedTables ++ - parsed.collect { case UnresolvedRelation(databaseName, name, _) => name } + logical.collect { case UnresolvedRelation(databaseName, name, _) => name } val referencedTestTables = referencedTables.filter(testTables.contains) logger.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) // Proceed with analysis. - analyze(parsed) + analyzer(logical) } - - /** - * Runs the query after interposing operators that print the result of each intermediate step. - */ - def debugExec() = DebugQuery(executedPlan).execute().collect() } - class SharkSqlQuery(sql: String) extends SharkQuery { - lazy val parsed = HiveQl.parseSql(sql) - def hiveExec() = runSqlHive(sql) - override def toString = sql + "\n" + super.toString - } - - - /* We must repeat the implicits so that we bind to the overridden versions */ + /* We must repeat the implicits so that we bind to the overridden versions implicit class stringToTestQuery(str: String) { def q = new SharkSqlQuery(str) @@ -153,9 +147,13 @@ class TestSharkInstance extends SharkInstance { val parsed = plan } + */ + case class TestTable(name: String, commands: (()=>Unit)*) - implicit class SqlCmd(sql: String) { def cmd = () => sql.q.stringResult(): Unit} + implicit class SqlCmd(sql: String) { + def cmd = () => new SqlQueryExecution(sql).stringResult(): Unit + } /** * A list of test tables and the DDL required to initialize them. A test table is loaded on diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala similarity index 97% rename from src/main/scala/catalyst/execution/hiveOperators.scala rename to shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index 492abf3fd0a76..41f69604b3d2d 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -1,5 +1,5 @@ -package catalyst -package execution +package org.apache.spark.sql +package shark import java.io.{File, IOException} import java.util.UUID @@ -21,6 +21,8 @@ import org.apache.spark.SparkContext._ import catalyst.expressions._ import catalyst.types.{BooleanType, DataType} +import execution._ + /* Implicits */ import scala.collection.JavaConversions._ @@ -53,7 +55,7 @@ case class HiveTableScan( } @transient - val hadoopReader = new HadoopTableReader(relation.tableDesc, sc.hiveconf) + val hadoopReader = new HadoopTableReader(relation.tableDesc, sc) /** * The hive object inspector for this table, which can be used to extract values from the @@ -125,7 +127,7 @@ case class HiveTableScan( * @param partitions All partitions of the relation. * @return Partitions that are involved in the query plan. */ - private[catalyst] def prunePartitions(partitions: Seq[HivePartition]) = { + private[shark] def prunePartitions(partitions: Seq[HivePartition]) = { boundPruningPred match { case None => partitions case Some(shouldKeep) => partitions.filter { part => @@ -164,7 +166,7 @@ case class HiveTableScan( } case class InsertIntoHiveTable( - table: MetastoreRelation, partition: Map[String, Option[String]], child: SharkPlan) + table: MetastoreRelation, partition: Map[String, Option[String]], child: SparkPlan) (@transient sc: SharkContext) extends UnaryNode { @@ -263,6 +265,6 @@ case class InsertIntoHiveTable( // however for now we return an empty list to simplify compatibility checks with hive, which // does not return anything for insert operations. // TODO: implement hive compatibility as rules. - sc.makeRDD(Nil, 1) + sc.sparkContext.makeRDD(Nil, 1) } } diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala similarity index 88% rename from src/main/scala/catalyst/execution/FunctionRegistry.scala rename to shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala index 57c8839fa371c..7118b16fe9055 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala @@ -1,5 +1,5 @@ -package catalyst -package execution +package org.apache.spark.sql +package shark import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer @@ -13,10 +13,14 @@ import org.apache.hadoop.hive.ql.udf.generic._ import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.{io => hadoopIo} +import catalyst.analysis import catalyst.expressions._ +import catalyst.types import catalyst.types._ -object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionFactory { +object HiveFunctionRegistry + extends analysis.FunctionRegistry with HiveFunctionFactory with HiveInspectors { + def lookupFunction(name: String, children: Seq[Expression]): Expression = { // We only look it up to see if it exists, but do not include it in the HiveUDF since it is // not always serializable. @@ -200,7 +204,6 @@ case class HiveGenericUdf( } trait HiveInspectors { - /** Converts native catalyst types to the types expected by Hive */ def wrap(a: Any): AnyRef = a match { case s: String => new hadoopIo.Text(s) @@ -258,6 +261,24 @@ trait HiveInspectors { case _: WritableByteObjectInspector => ByteType case _: JavaByteObjectInspector => ByteType } + + implicit class typeInfoConversions(dt: DataType) { + import org.apache.hadoop.hive.serde2.typeinfo._ + import TypeInfoFactory._ + + def toTypeInfo: TypeInfo = dt match { + case BooleanType => booleanTypeInfo + case ByteType => byteTypeInfo + case DoubleType => doubleTypeInfo + case FloatType => floatTypeInfo + case IntegerType => intTypeInfo + case LongType => longTypeInfo + case ShortType => shortTypeInfo + case StringType => stringTypeInfo + case DecimalType => decimalTypeInfo + case NullType => voidTypeInfo + } + } } case class HiveGenericUdaf( @@ -284,6 +305,8 @@ case class HiveGenericUdaf( def references: Set[Attribute] = children.map(_.references).flatten.toSet override def toString = s"$nodeName#$name(${children.mkString(",")})" + + def newInstance = new HiveUdafFunction(name, children, this) } /** @@ -361,3 +384,35 @@ case class HiveGenericUdtf( override def toString() = s"$nodeName#$name(${children.mkString(",")})" } + +case class HiveUdafFunction( + functionName: String, + exprs: Seq[Expression], + base: AggregateExpression) + extends AggregateFunction + with HiveInspectors + with HiveFunctionFactory { + + def this() = this(null, null, null) + + private val resolver = createFunction[AbstractGenericUDAFResolver](functionName) + + private val inspectors = exprs.map(_.dataType).map(toInspector).toArray + + private val function = { + val evaluator = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray) + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) + evaluator + } + + // Cast required to avoid type inference selecting a deprecated Hive API. + private val buffer = + function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] + + def result: Any = unwrap(function.evaluate(buffer)) + + def apply(input: Seq[Row]): Unit = { + val inputs = exprs.map(Evaluate(_, input).asInstanceOf[AnyRef]).toArray + function.iterate(buffer, inputs) + } +} diff --git a/shark/src/test/resources/log4j.properties b/shark/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..5e17e3b596ba1 --- /dev/null +++ b/shark/src/test/resources/log4j.properties @@ -0,0 +1,47 @@ +# +# 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. +# + +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootLogger=DEBUG, CA, FA + +#Console Appender +log4j.appender.CA=org.apache.log4j.ConsoleAppender +log4j.appender.CA.layout=org.apache.log4j.PatternLayout +log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n +log4j.appender.CA.Threshold = WARN + + +#File Appender +log4j.appender.FA=org.apache.log4j.FileAppender +log4j.appender.FA.append=false +log4j.appender.FA.file=target/unit-tests.log +log4j.appender.FA.layout=org.apache.log4j.PatternLayout +log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n + +# Set the logger level of File Appender to WARN +log4j.appender.FA.Threshold = INFO + +# Some packages are noisy for no good reason. +log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false +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.ql.metadata.Hive=false +log4j.logger.hive.ql.metadata.Hive=OFF + diff --git a/src/test/scala/catalyst/execution/BigDataBenchmark.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/BigDataBenchmarkSuite.scala similarity index 97% rename from src/test/scala/catalyst/execution/BigDataBenchmark.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/BigDataBenchmarkSuite.scala index 1ff9cb6491746..958fff68ad858 100644 --- a/src/test/scala/catalyst/execution/BigDataBenchmark.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/BigDataBenchmarkSuite.scala @@ -1,4 +1,5 @@ -package catalyst +package org.apache.spark.sql +package shark package execution import java.io.File @@ -7,7 +8,7 @@ import java.io.File * A set of test cases based on the big-data-benchmark. * https://amplab.cs.berkeley.edu/benchmark/ */ -class BigDataBenchmarkTests extends HiveComparisonTest { +class BigDataBenchmarkSuite extends HiveComparisonTest { import TestShark._ val testDataDirectory = new File("target/big-data-benchmark-testdata") diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/execution/ConcurrentHiveSuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/ConcurrentHiveSuite.scala new file mode 100644 index 0000000000000..3b8b9a9d577f4 --- /dev/null +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/ConcurrentHiveSuite.scala @@ -0,0 +1,21 @@ +package org.apache.spark +package sql +package shark +package execution + + +import org.scalatest.{FunSuite, BeforeAndAfterAll} + +class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll { + ignore("multiple instances not supported") { + test("Multiple Hive Instances") { + (1 to 10).map { i => + val ts = + new TestSharkContext(new SparkContext("local", s"TestSqlContext$i", new SparkConf())) + ts.executeSql("SHOW TABLES").toRdd.collect() + ts.executeSql("SELECT * FROM src").toRdd.collect() + ts.executeSql("SHOW TABLES").toRdd.collect() + } + } + } +} \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveComparisonTest.scala similarity index 92% rename from src/test/scala/catalyst/execution/HiveComparisonTest.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveComparisonTest.scala index ae7e740b2efcb..9e540c9fc7f95 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveComparisonTest.scala @@ -1,12 +1,15 @@ -package catalyst +package org.apache.spark.sql +package shark package execution import java.io._ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} -import frontend.hive.{ExplainCommand, Command} -import plans.physical._ -import util._ +import catalyst.plans.logical.{ExplainCommand, NativeCommand} +import catalyst.plans._ +import catalyst.util._ + +import org.apache.spark.sql.execution.Sort /** * Allows the creations of tests that execute the same query against both hive @@ -59,7 +62,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G .map(name => new File(targetDir, s"$suiteName.$name")) /** The local directory with cached golden answer will be stored. */ - protected val answerCache = new File("src/test/hive/golden") + protected val answerCache = new File("shark/src/test/hive/golden") if (!answerCache.exists) answerCache.mkdir() @@ -68,22 +71,22 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G /** Directory containing a file for each test case that passes. */ val passedDirectory = new File(targetDir, s"$suiteName.passed") - if(!passedDirectory.exists()) + if (!passedDirectory.exists()) passedDirectory.mkdir() // Not atomic! /** Directory containing output of tests that fail to execute with Catalyst. */ val failedDirectory = new File(targetDir, s"$suiteName.failed") - if(!failedDirectory.exists()) + if (!failedDirectory.exists()) failedDirectory.mkdir() // Not atomic! /** Directory containing output of tests where catalyst produces the wrong answer. */ val wrongDirectory = new File(targetDir, s"$suiteName.wrong") - if(!wrongDirectory.exists()) + if (!wrongDirectory.exists()) wrongDirectory.mkdir() // Not atomic! /** Directory containing output of tests where we fail to generate golden output with Hive. */ val hiveFailedDirectory = new File(targetDir, s"$suiteName.hiveFailed") - if(!hiveFailedDirectory.exists()) + if (!hiveFailedDirectory.exists()) hiveFailedDirectory.mkdir() // Not atomic! /** All directories that contain per-query output files */ @@ -101,11 +104,12 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G } protected def prepareAnswer( - sharkQuery: TestShark.type#SharkSqlQuery, + sharkQuery: TestShark.type#SqlQueryExecution, answer: Seq[String]): Seq[String] = { - val orderedAnswer = sharkQuery.parsed match { + val orderedAnswer = sharkQuery.logical match { // Clean out non-deterministic time schema info. - case _: Command => answer.filterNot(nonDeterministicLine).filterNot(_ == "") + case _: NativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "") + case _: ExplainCommand => answer case _ => // TODO: Really we only care about the final total ordering here... val isOrdered = sharkQuery.executedPlan.collect { @@ -164,7 +168,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G runOnlyDirectories .map(new File(_, testCaseName)) .filter(_.exists) - if(runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { + if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { logger.debug( s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") return @@ -186,7 +190,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G // previously broken behavior we simply filter out changes to this setting. .filterNot(_ contains "hive.outerjoin.supports.filters") - if(allQueries != queryList) + if (allQueries != queryList) logger.warn(s"Simplifications made on unsupported operations for test $testCaseName") lazy val consoleTestCase = { @@ -229,9 +233,9 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G hiveCachedResults } else { - val sharkQueries = queryList.map(new TestShark.SharkSqlQuery(_)) + val sharkQueries = queryList.map(new TestShark.SqlQueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. - sharkQueries.foreach(_.parsed) + sharkQueries.foreach(_.logical) val computedResults = (queryList.zipWithIndex, sharkQueries,hiveCacheFiles).zipped.map { case ((queryString, i), sharkQuery, cachedAnswerFile)=> try { @@ -275,7 +279,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => - val query = new TestShark.SharkSqlQuery(queryString) + val query = new TestShark.SqlQueryExecution(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { case e: Exception => val errorMessage = @@ -299,14 +303,14 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G // Check that the results match unless its an EXPLAIN query. val preparedHive = prepareAnswer(sharkQuery,hive) - if ((!sharkQuery.parsed.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) { + if ((!sharkQuery.logical.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${hive.size} row(s) ==" +: preparedHive val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") - if(recomputeCache) { + if (recomputeCache) { logger.warn(s"Clearing cache files for failed test $testCaseName") hiveCacheFiles.foreach(_.delete()) } @@ -332,7 +336,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G // When we encounter an error we check to see if the environment is still okay by running a simple query. // If this fails then we halt testing since something must have gone seriously wrong. try { - new TestShark.SharkSqlQuery("SELECT key FROM src").stringResult() + new TestShark.SqlQueryExecution("SELECT key FROM src").stringResult() TestShark.runSqlHive("SELECT key FROM src") } catch { case e: Exception => diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala similarity index 99% rename from src/test/scala/catalyst/execution/HiveCompatibility.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala index 4cca463425649..f65f219019896 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala @@ -1,6 +1,8 @@ -package catalyst +package org.apache.spark.sql +package shark package execution + import java.io._ import util._ @@ -8,7 +10,7 @@ import util._ /** * Runs the test cases that are included in the hive distribution. */ -class HiveCompatibility extends HiveQueryFileTest { +class HiveCompatibilitySuite extends HiveQueryFileTest { // TODO: bundle in jar files... get from classpath lazy val hiveQueryDir = TestShark.getHiveFile("ql/src/test/queries/clientpositive") def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) diff --git a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQueryFileTest.scala similarity index 96% rename from src/test/scala/catalyst/execution/HiveQueryFileTest.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQueryFileTest.scala index 97f97b5b34148..bffa0f80f6efa 100644 --- a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQueryFileTest.scala @@ -1,9 +1,10 @@ -package catalyst +package org.apache.spark.sql +package shark package execution import java.io._ -import util._ +import catalyst.util._ /** * A framework for running the query tests that are listed as a set of text files. diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala similarity index 94% rename from src/test/scala/catalyst/execution/HiveQueryTests.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala index 924a55923cc43..7f041167eb4e7 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala @@ -1,10 +1,12 @@ -package catalyst +package org.apache.spark.sql +package shark package execution + /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ -class HiveQueryTests extends HiveComparisonTest { +class HiveQuerySuite extends HiveComparisonTest { import TestShark._ createQueryTest("Simple Average", @@ -105,4 +107,7 @@ class HiveQueryTests extends HiveComparisonTest { createQueryTest("lateral view5", "FROM src SELECT explode(array(key+3, key+4))") + createQueryTest("lateral view6", + "SELECT * FROM src LATERAL VIEW explode(map(key+3,key+4)) D as k, v") + } \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/HiveResolutionSuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveResolutionSuite.scala similarity index 97% rename from src/test/scala/catalyst/execution/HiveResolutionSuite.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveResolutionSuite.scala index 4ab6c0dc80a1f..57b7cab424dde 100644 --- a/src/test/scala/catalyst/execution/HiveResolutionSuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveResolutionSuite.scala @@ -1,4 +1,5 @@ -package catalyst +package org.apache.spark.sql +package shark package execution /** diff --git a/src/test/scala/catalyst/execution/HiveSerDeSuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveSerDeSuite.scala similarity index 85% rename from src/test/scala/catalyst/execution/HiveSerDeSuite.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveSerDeSuite.scala index 2fea958cc7cac..59f7ec314e909 100644 --- a/src/test/scala/catalyst/execution/HiveSerDeSuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveSerDeSuite.scala @@ -1,4 +1,6 @@ -package catalyst.execution +package org.apache.spark.sql +package shark +package execution /** * A set of tests that validates support for Hive SerDe. diff --git a/src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveTypeCoercionSuite.scala similarity index 69% rename from src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveTypeCoercionSuite.scala index f94b9951bacab..b35fcbb87f833 100644 --- a/src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveTypeCoercionSuite.scala @@ -1,11 +1,9 @@ -package catalyst -package execution +package org.apache.spark.sql.shark.execution /** * A set of tests that validate type promotion rules. */ -class HiveTypeCoersionSuite extends HiveComparisonTest { - import TestShark._ +class HiveTypeCoercionSuite extends HiveComparisonTest { val baseTypes = Seq("1", "1.0", "1L", "1S", "1Y", "'1'") diff --git a/src/test/scala/catalyst/execution/PartitionPruningSuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/PartitionPruningSuite.scala similarity index 92% rename from src/test/scala/catalyst/execution/PartitionPruningSuite.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/PartitionPruningSuite.scala index f13317ddc9028..15ebc2ff332ec 100644 --- a/src/test/scala/catalyst/execution/PartitionPruningSuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/PartitionPruningSuite.scala @@ -1,4 +1,6 @@ -package catalyst.execution +package org.apache.spark.sql +package shark +package execution import scala.collection.JavaConversions._ @@ -34,7 +36,7 @@ class PartitionPruningSuite extends HiveComparisonTest { def createPruningTest(testCaseName: String, sql: String, expectedValues: Seq[Seq[String]]) = { test(testCaseName) { - val plan = sql.q.executedPlan + val plan = new TestShark.SqlQueryExecution(sql).executedPlan val prunedPartitions = plan.collect { case p @ HiveTableScan(_, relation, _) => p.prunePartitions(relation.hiveQlPartitions) diff --git a/src/main/scala/catalyst/execution/SharkContext.scala b/src/main/scala/catalyst/execution/SharkContext.scala deleted file mode 100644 index fd337d0895ff0..0000000000000 --- a/src/main/scala/catalyst/execution/SharkContext.scala +++ /dev/null @@ -1,115 +0,0 @@ -package catalyst.execution - -import java.io._ -import java.util.{ArrayList => JArrayList} - -import scala.collection.Map -import scala.collection.JavaConversions._ - -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.processors.CommandProcessor -import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse -import org.apache.hadoop.hive.ql.session.SessionState - -import org.apache.spark.{SparkContext, SparkEnv} - -class SharkContext( - master: String, - jobName: String, - sparkHome: String, - jars: Seq[String], - environment: Map[String, String]) - extends SparkContext(master, jobName, sparkHome, jars, environment) { - - @transient val sparkEnv = SparkEnv.get - @transient val hiveconf = new HiveConf(classOf[SessionState]) - - @transient val sessionState = new SessionState(hiveconf) - - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. - val outputBuffer = new java.io.OutputStream { - var pos: Int = 0 - var buffer = new Array[Int](10240) - def write(i: Int): Unit = { - buffer(pos) = i - pos = (pos + 1) % buffer.size - } - - override def toString = { - val (end, start) = buffer.splitAt(pos) - val input = new java.io.InputStream { - val iterator = (start ++ end).iterator - - def read(): Int = if(iterator.hasNext) iterator.next else -1 - } - val reader = new BufferedReader(new InputStreamReader(input)) - val stringBuilder = new StringBuilder - var line = reader.readLine() - while(line != null) { - stringBuilder.append(line) - stringBuilder.append("\n") - line = reader.readLine() - } - stringBuilder.toString() - } - } - - sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") - sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") - - /** - * Execute the command using Hive and return the results as a sequence. Each element - * in the sequence is one row. - */ - def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { - try { - SparkEnv.set(sparkEnv) - val cmd_trimmed: String = cmd.trim() - val tokens: Array[String] = cmd_trimmed.split("\\s+") - val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) - - SessionState.start(sessionState) - - if (proc.isInstanceOf[Driver]) { - val driver: Driver = proc.asInstanceOf[Driver] - driver.init() - - val results = new JArrayList[String] - val response: CommandProcessorResponse = driver.run(cmd) - // Throw an exception if there is an error in query processing. - if (response.getResponseCode != 0) { - driver.destroy() - throw new QueryExecutionException(response.getErrorMessage) - } - driver.setMaxRows(maxRows) - driver.getResults(results) - driver.destroy() - results - } else { - sessionState.out.println(tokens(0) + " " + cmd_1) - Seq(proc.run(cmd_1).getResponseCode.toString) - } - }catch { - case e: Exception => - println( - """ - |====================== - |HIVE FAILURE OUTPUT - |====================== - """.stripMargin) - println(outputBuffer.toString) - println( - """ - |====================== - |END HIVE FAILURE OUTPUT - |====================== - """.stripMargin) - throw e - } - } -} - - diff --git a/src/main/scala/catalyst/execution/SharkEnv.scala b/src/main/scala/catalyst/execution/SharkEnv.scala deleted file mode 100644 index d316d6f0c5d0f..0000000000000 --- a/src/main/scala/catalyst/execution/SharkEnv.scala +++ /dev/null @@ -1,43 +0,0 @@ -package catalyst.execution - -import scala.collection.mutable.{HashMap, HashSet} - -import org.apache.spark.SparkContext -import org.apache.spark.scheduler.StatsReportListener - - -/** A singleton object for the master program. The slaves should not access this. */ -object SharkEnv { - - def initWithSharkContext(jobName: String, master: String): SharkContext = { - if (sc != null) { - sc.stop() - } - - sc = new SharkContext( - if (master == null) "local" else master, - jobName, - System.getenv("SPARK_HOME"), - Nil, - executorEnvVars) - sc.addSparkListener(new StatsReportListener()) - sc.asInstanceOf[SharkContext] - } - - val executorEnvVars = new HashMap[String, String] - executorEnvVars.put("SCALA_HOME", getEnv("SCALA_HOME")) - executorEnvVars.put("SPARK_MEM", getEnv("SPARK_MEM")) - executorEnvVars.put("SPARK_CLASSPATH", getEnv("SPARK_CLASSPATH")) - executorEnvVars.put("HADOOP_HOME", getEnv("HADOOP_HOME")) - executorEnvVars.put("JAVA_HOME", getEnv("JAVA_HOME")) - executorEnvVars.put("MESOS_NATIVE_LIBRARY", getEnv("MESOS_NATIVE_LIBRARY")) - executorEnvVars.put("TACHYON_MASTER", getEnv("TACHYON_MASTER")) - executorEnvVars.put("TACHYON_WAREHOUSE_PATH", getEnv("TACHYON_WAREHOUSE_PATH")) - - val activeSessions = new HashSet[String] - - var sc: SparkContext = _ - - /** Return the value of an environmental variable as a string. */ - def getEnv(varname: String) = if (System.getenv(varname) == null) "" else System.getenv(varname) -} diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala deleted file mode 100644 index c5319cbcbd87c..0000000000000 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ /dev/null @@ -1,192 +0,0 @@ -package catalyst -package execution - -import java.io.File -import scala.language.implicitConversions - -import analysis.{SimpleAnalyzer, Analyzer} -import frontend.hive._ -import optimizer.Optimize -import planning.QueryPlanner -import plans.logical.LogicalPlan -import rules.RuleExecutor -import types._ - -/** - * Starts up an instance of shark where metadata is stored locally. An in-process metadata data is - * created with data stored in ./metadata. Warehouse data is stored in in ./warehouse. - */ -class LocalSharkInstance(val master: String) extends SharkInstance { - override def warehousePath = new File("warehouse").getCanonicalPath - override def metastorePath = new File("metastore").getCanonicalPath -} - -/** - * An instance of the shark execution engine. This class is responsible for taking queries - * expressed either in SQL or as raw catalyst logical plans and optimizing them for execution - * using Spark. Additionally this class maintains the connection with the hive metadata store. - */ -abstract class SharkInstance extends Logging { - self => - - /** The URL of the shark master. */ - def master: String - /** The path to the hive warehouse. */ - def warehousePath: String - /** The path to the local metastore. */ - def metastorePath: String - - /** The SharkContext */ - lazy val sc: SharkContext = createContext() - - protected def createContext(): SharkContext = { - SharkEnv.initWithSharkContext("catalyst.execution", master) - } - - /** Sets up the system initially or after a RESET command */ - protected def configure() { - // TODO: refactor this so we can work with other databases. - runSqlHive( - s"set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$metastorePath;create=true") - runSqlHive("set hive.metastore.warehouse.dir=" + warehousePath) - } - - configure() // Must be called before initializing the catalog below. - - /* A catalyst metadata catalog that points to the Shark/Hive Metastore. */ - val catalog = new HiveMetastoreCatalog(sc.hiveconf) - - /* An analyzer that uses the Shark/Hive metastore. */ - val analyze = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) - - /** - * Runs the specified SQL query using Hive. - */ - def runSqlHive(sql: String): Seq[String] = { - val maxResults = 100000 - val results = sc.runHive(sql, 100000) - // It is very confusing when you only get back some of the results... - if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") - results - } - - object TrivialPlanner extends QueryPlanner[SharkPlan] with PlanningStrategies { - val sc = self.sc - val strategies = - PartialAggregation :: - SparkEquiInnerJoin :: - PartitionPrunings :: - HiveTableScans :: - DataSinks :: - BasicOperators :: - CartesianProduct :: - BroadcastNestedLoopJoin :: Nil - } - - object PrepareForExecution extends RuleExecutor[SharkPlan] { - val batches = - Batch("Add exchange", Once, AddExchange) :: - Batch("Prepare Expressions", Once, new expressions.BindReferences[SharkPlan]) :: Nil - } - - class SharkSqlQuery(sql: String) extends SharkQuery { - lazy val parsed = HiveQl.parseSql(sql) - def hiveExec() = runSqlHive(sql) - override def toString = sql + "\n" + super.toString - } - - /** - * The primary workflow for executing queries using Shark. Designed to allow easy access to the - * intermediate phases of query execution. - */ - abstract class SharkQuery { - def parsed: LogicalPlan - - lazy val analyzed = analyze(parsed) - lazy val optimizedPlan = Optimize(catalog.CreateTables(analyzed)) - // TODO: Don't just pick the first one... - lazy val sharkPlan = TrivialPlanner(optimizedPlan).next() - lazy val executedPlan: SharkPlan = PrepareForExecution(sharkPlan) - - lazy val toRdd = executedPlan.execute() - - protected val primitiveTypes = - Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType, DecimalType) - - protected def toHiveString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ))=> - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_,_], MapType(kType, vType)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "NULL" - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - - /** Hive outputs fields of structs slightly differently than top level attributes. */ - protected def toHiveStructString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ))=> - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_,_], MapType(kType, vType)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "null" - case (s: String, StringType) => "\"" + s + "\"" - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - - /** - * Returns the result as a hive compatible sequence of strings. For native commands, the - * execution is simply passed back to Hive. - */ - def stringResult(): Seq[String] = analyzed match { - case NativeCommand(cmd) => runSqlHive(cmd) - case ConfigurationAssignment(cmd) => runSqlHive(cmd) - case ExplainCommand(plan) => new SharkQuery { val parsed = plan }.toString.split("\n") - case query => - val result: Seq[Seq[Any]] = toRdd.collect().toSeq - // We need the types so we can output struct field names - val types = analyzed.output.map(_.dataType) - // Reformat to match hive tab delimited output. - val asString = result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq - asString - } - - protected def stringOrError[A](f: => A): String = - try f.toString catch { case e: Throwable => e.toString } - - override def toString: String = - s"""== Logical Plan == - |${stringOrError(analyzed)} - |== Physical Plan == - |${stringOrError(executedPlan)} - """.stripMargin.trim - } - - /** - * A shark query workflow for plans where all relations have already been resolved (likely because - * the query was built from raw RDDs). Additionally attribute resolution is case sensitive. - */ - abstract class LogicalSharkQuery extends SharkQuery { - override lazy val analyzed = SimpleAnalyzer(parsed) - } - - implicit class stringToQuery(str: String) { - def q = new SharkSqlQuery(str) - } - - implicit def logicalToSharkQuery(plan: LogicalPlan) = new LogicalSharkQuery { val parsed = plan } -} diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala deleted file mode 100644 index e33e7fab2d5bc..0000000000000 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ /dev/null @@ -1,231 +0,0 @@ -package catalyst -package execution - -import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFEvaluator, AbstractGenericUDAFResolver} - -import catalyst.errors._ -import catalyst.expressions._ -import catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples} -import catalyst.types._ - -/* Implicits */ -import org.apache.spark.rdd.SharkPairRDDFunctions._ - -/** - * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each - * group. - * - * @param partial if true then aggregation is done partially on local data without shuffling to - * ensure all values where `groupingExpressions` are equal are present. - * @param groupingExpressions expressions that are evaluated to determine grouping. - * @param aggregateExpressions expressions that are computed for each group. - * @param child the input data source. - */ -case class Aggregate( - partial: Boolean, - groupingExpressions: Seq[Expression], - aggregateExpressions: Seq[NamedExpression], - child: SharkPlan)(@transient sc: SharkContext) - extends UnaryNode { - - override def requiredChildDistribution = - if (partial) { - UnspecifiedDistribution :: Nil - } else { - if (groupingExpressions == Nil) { - AllTuples :: Nil - } else { - ClusteredDistribution(groupingExpressions) :: Nil - } - } - - override def otherCopyArgs = sc :: Nil - - def output = aggregateExpressions.map(_.toAttribute) - - /* Replace all aggregate expressions with spark functions that will compute the result. */ - def createAggregateImplementations() = aggregateExpressions.map { agg => - val impl = agg transform { - case base @ Average(expr) => new AverageFunction(expr, base) - case base @ Sum(expr) => new SumFunction(expr, base) - case base @ SumDistinct(expr) => new SumDistinctFunction(expr, base) - case base @ Count(expr) => new CountFunction(expr, base) - // TODO: Create custom query plan node that calculates distinct values efficiently. - case base @ CountDistinct(expr) => new CountDistinctFunction(expr, base) - case base @ First(expr) => new FirstFunction(expr, base) - case base @ HiveGenericUdaf(resolver, expr) => new HiveUdafFunction(expr, base, resolver) - } - - val remainingAttributes = impl.collect { case a: Attribute => a } - // If any references exist that are not inside agg functions then the must be grouping exprs - // in this case we must rebind them to the grouping tuple. - if (remainingAttributes.nonEmpty) { - val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } - - // An exact match with a grouping expression - val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { - case -1 => None - case ordinal => Some(BoundReference(0, ordinal, Alias(impl, "AGGEXPR")().toAttribute)) - } - - exactGroupingExpr.getOrElse( - sys.error(s"$agg is not in grouping expressions: $groupingExpressions")) - } else { - impl - } - } - - def execute() = attachTree(this, "execute") { - // TODO: If the child of it is an [[catalyst.execution.Exchange]], - // do not evaluate the groupingExpressions again since we have evaluated it - // in the [[catalyst.execution.Exchange]]. - val grouped = child.execute().map { row => - (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row) - }.groupByKeyLocally() - - val result = grouped.map { case (group, rows) => - val aggImplementations = createAggregateImplementations() - - // Pull out all the functions so we can feed each row into them. - val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f }) - - rows.foreach { row => - val input = Vector(row) - aggFunctions.foreach(_.apply(input)) - } - buildRow(aggImplementations.map(Evaluate(_, Vector(group)))) - } - - // TODO: THIS DOES NOT PRESERVE LINEAGE AND BREAKS PIPELINING. - if(groupingExpressions.isEmpty && result.count == 0) { - // When there there is no output to the Aggregate operator, we still output an empty row. - val aggImplementations = createAggregateImplementations() - sc.makeRDD(buildRow(aggImplementations.map(Evaluate(_, Nil))) :: Nil) - } else { - result - } - } -} - -case class HiveUdafFunction( - exprs: Seq[Expression], - base: AggregateExpression, - functionName: String) - extends AggregateFunction - with HiveInspectors - with HiveFunctionFactory { - - def this() = this(null, null, null) - - private val resolver = createFunction[AbstractGenericUDAFResolver](functionName) - - private val inspectors = exprs.map(_.dataType).map(toInspector).toArray - - private val function = { - val evaluator = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray) - evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) - evaluator - } - - // Cast required to avoid type inference selecting a deprecated Hive API. - private val buffer = - function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] - - def result: Any = unwrap(function.evaluate(buffer)) - - def apply(input: Seq[Row]): Unit = { - val inputs = exprs.map(Evaluate(_, input).asInstanceOf[AnyRef]).toArray - function.iterate(buffer, inputs) - } -} - -// TODO: Move these default functions back to expressions. Build framework for instantiating them. -case class AverageFunction(expr: Expression, base: AggregateExpression) - extends AggregateFunction { - - def this() = this(null, null) // Required for serialization. - - var count: Long = _ - var sum: Long = _ - - def result: Any = sum.toDouble / count.toDouble - - def apply(input: Seq[Row]): Unit = { - count += 1 - // TODO: Support all types here... - sum += Evaluate(expr, input).asInstanceOf[Int] - } -} - -case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var count: Int = _ - - def apply(input: Seq[Row]): Unit = { - val evaluatedExpr = expr.map(Evaluate(_, input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { - count += 1 - } - } - - def result: Any = count -} - -case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var sum = Evaluate(Cast(Literal(0), expr.dataType), Nil) - - def apply(input: Seq[Row]): Unit = - sum = Evaluate(Add(Literal(sum), expr), input) - - def result: Any = sum -} - -case class SumDistinctFunction(expr: Expression, base: AggregateExpression) - extends AggregateFunction { - - def this() = this(null, null) // Required for serialization. - - val seen = new scala.collection.mutable.HashSet[Any]() - - def apply(input: Seq[Row]): Unit = { - val evaluatedExpr = Evaluate(expr, input) - if (evaluatedExpr != null) { - seen += evaluatedExpr - } - } - - def result: Any = - seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus) -} - -case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) - extends AggregateFunction { - - def this() = this(null, null) // Required for serialization. - - val seen = new scala.collection.mutable.HashSet[Any]() - - def apply(input: Seq[Row]): Unit = { - val evaluatedExpr = expr.map(Evaluate(_, input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { - seen += evaluatedExpr - } - } - - def result: Any = seen.size -} - -case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var result: Any = null - - def apply(input: Seq[Row]): Unit = { - if (result == null) { - result = Evaluate(expr, input) - } - } -} diff --git a/src/main/scala/catalyst/execution/package.scala b/src/main/scala/catalyst/execution/package.scala deleted file mode 100644 index 7eeea97aee53a..0000000000000 --- a/src/main/scala/catalyst/execution/package.scala +++ /dev/null @@ -1,34 +0,0 @@ -package catalyst - -import types._ - -/** - * A partial reimplementation of Shark, a Hive compatible SQL engine running on Spark. - * - * This implementation uses the hive parser, metadata catalog and serdes, but performs all - * optimization and execution using catalyst and spark. - * - * Currently functions that are not supported by this implementation are passed back to the - * original Shark implementation for execution. - */ -package object execution { - type Row = catalyst.expressions.Row - - implicit class typeInfoConversions(dt: DataType) { - import org.apache.hadoop.hive.serde2.typeinfo._ - import TypeInfoFactory._ - - def toTypeInfo: TypeInfo = dt match { - case BooleanType => booleanTypeInfo - case ByteType => byteTypeInfo - case DoubleType => doubleTypeInfo - case FloatType => floatTypeInfo - case IntegerType => intTypeInfo - case LongType => longTypeInfo - case ShortType => shortTypeInfo - case StringType => stringTypeInfo - case DecimalType => decimalTypeInfo - case NullType => voidTypeInfo - } - } -} diff --git a/src/main/scala/catalyst/frontend/package.scala b/src/main/scala/catalyst/frontend/package.scala deleted file mode 100644 index 4d70c4b420d98..0000000000000 --- a/src/main/scala/catalyst/frontend/package.scala +++ /dev/null @@ -1,7 +0,0 @@ -package catalyst - -/** - * A collection of languages that can be parsed into a [[catalyst.plans.logical.LogicalPlan]]. - * Currently only [[hive]]'s parser is supported. - */ -package object frontend diff --git a/src/main/scala/catalyst/package.scala b/src/main/scala/catalyst/package.scala deleted file mode 100644 index 52e9b39db9196..0000000000000 --- a/src/main/scala/catalyst/package.scala +++ /dev/null @@ -1,11 +0,0 @@ - -/** - * Catalyst is a framework for performing optimization on trees of dataflow operators. - */ -package object catalyst { - - protected[catalyst] def Logger(name: String) = - com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger(name)) - - protected[catalyst] type Logging = com.typesafe.scalalogging.slf4j.Logging -} diff --git a/src/test/scala/catalyst/execution/ConcurrentHiveTests.scala b/src/test/scala/catalyst/execution/ConcurrentHiveTests.scala deleted file mode 100644 index 7f45bfe4da528..0000000000000 --- a/src/test/scala/catalyst/execution/ConcurrentHiveTests.scala +++ /dev/null @@ -1,18 +0,0 @@ -package catalyst -package execution - -import org.scalatest.{FunSuite, BeforeAndAfterAll} - -class ConcurrentHiveTests extends FunSuite with BeforeAndAfterAll { - ignore("multiple instances not supported") { - test("Multiple Hive Instances") { - (1 to 10).map { i => - val ts = new TestSharkInstance - ts.runSqlHive("SHOW TABLES") - val q = ts.stringToTestQuery("SELECT * FROM src").q - q.toRdd.collect() - ts.runSqlHive("SHOW TABLES") - } - } - } -} \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/HiveAstTransformSuite.scala b/src/test/scala/catalyst/execution/HiveAstTransformSuite.scala deleted file mode 100644 index a46c316417672..0000000000000 --- a/src/test/scala/catalyst/execution/HiveAstTransformSuite.scala +++ /dev/null @@ -1,80 +0,0 @@ -package catalyst - -import org.scalatest.FunSuite - -import frontend.hive._ -import org.apache.hadoop.hive.ql.parse.{HiveParser, ASTNode} - -import collection.JavaConversions._ - -object HiveDistinctToGroupBy { - import HiveQl._ - - def apply(n: ASTNode): ASTNode = n transform { - case t @ Token(_, children) if getClauseOption("TOK_SELECTDI", children).isDefined => - val selectDistinctClause = getClause("TOK_SELECTDI", children) - val selectExprs = selectDistinctClause.getChildren.map { - case Token("TOK_SELEXPR", child :: Nil) => child - case Token("TOK_SELEXPR", child :: _ /* discard alias */ :: Nil) => child - - } - - val groupByClause = new ASTNode(new org.antlr.runtime.CommonToken(HiveParser.TOK_GROUPBY)) - .withText("TOK_GROUPBY") - .withChildren(selectExprs) - - t.withChildren(children.map { - case Token("TOK_SELECTDI", selectExprs) => - new ASTNode(new org.antlr.runtime.CommonToken(HiveParser.TOK_SELECT)) - .withText("TOK_SELECT") - .withChildren(selectExprs) - case other => other - } :+ groupByClause) - } -} - -class HiveAstTransformSuite extends FunSuite { - import HiveQl._ - test("simple transform") { - val q1 = getAst("SELECT key FROM src") - val q2 = getAst("SELECT value FROM src") - val transformed = q1 transform { - case t @ Token("key", Nil) => t.withText("value") - } - - transformed checkEquals q2 - } - - test("distinct to group by") { - val distinct = getAst("SELECT DISTINCT key FROM src") - val groupBy = getAst("SELECT key FROM src GROUP BY key") - val transformed = HiveDistinctToGroupBy(distinct) - - groupBy checkEquals transformed - } - - test("nested distinct to group by") { - val distinct = getAst("SELECT * FROM (SELECT DISTINCT key FROM src) a") - val groupBy = getAst("SELECT * FROM (SELECT key FROM src GROUP BY key) a") - val transformed = HiveDistinctToGroupBy(distinct) - - groupBy checkEquals transformed - } - - test("distinct to group by with alias") { - val distinct = getAst("SELECT DISTINCT key as k FROM src") - val groupBy = getAst("SELECT key as k FROM src GROUP BY key") - val transformed = HiveDistinctToGroupBy(distinct) - - groupBy checkEquals transformed - } - - test("distinct to group by complex exprs") { - val distinct = getAst("SELECT DISTINCT key + 1 FROM src") - val groupBy = getAst("SELECT key + 1 FROM src GROUP BY key + 1") - val transformed = HiveDistinctToGroupBy(distinct) - - groupBy checkEquals transformed - } - -} \ No newline at end of file From 678341a50b793b09658b823fa1bdc61a9293d770 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 14 Feb 2014 10:21:24 -0800 Subject: [PATCH 671/778] Replaced non-ascii text --- .../scala/org/apache/spark/sql/catalyst/trees/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index 1e1bbbc16cee2..cc7cd41833ec5 100644 --- a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -10,7 +10,7 @@ package catalyst * transform - accepts a partial function that is used to generate a new tree. When the * partial function can be applied to a given tree segment, that segment is replaced with the * result. After attempting to apply the partial function to a given node, the transform - * function recursively attempts to apply the function to that node’s children. + * function recursively attempts to apply the function to that node's children. * *

  1. debugging support - pretty printing, easy splicing of trees, etc.
  2. * From 1f6260d77223aaf23c2bbb112b52803bea061e42 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Fri, 14 Feb 2014 12:45:29 -0800 Subject: [PATCH 672/778] Fixed package name and test suite name in Makefile --- Makefile | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/Makefile b/Makefile index 2d7b073202613..7df09ce094da2 100644 --- a/Makefile +++ b/Makefile @@ -6,18 +6,18 @@ compile: # There is likely some bug here... still a good way to get a feeling if things are working in # parallel. s1: compile - sbt ${ARGS} -Dshark.hive.shard=0:8 "test-only catalyst.execution.HiveCompatibility" + sbt ${ARGS} -Dshark.hive.shard=0:8 "test-only org.apache.spark.sql.shark.execution.HiveCompatibilitySuite" s2: compile - sbt ${ARGS} -Dshark.hive.shard=1:8 "test-only catalyst.execution.HiveCompatibility" + sbt ${ARGS} -Dshark.hive.shard=1:8 "test-only org.apache.spark.sql.shark.execution.HiveCompatibilitySuite" s3: compile - sbt ${ARGS} -Dshark.hive.shard=2:8 "test-only catalyst.execution.HiveCompatibility" + sbt ${ARGS} -Dshark.hive.shard=2:8 "test-only org.apache.spark.sql.shark.execution.HiveCompatibilitySuite" s4: compile - sbt ${ARGS} -Dshark.hive.shard=3:8 "test-only catalyst.execution.HiveCompatibility" + sbt ${ARGS} -Dshark.hive.shard=3:8 "test-only org.apache.spark.sql.shark.execution.HiveCompatibilitySuite" s5: compile - sbt ${ARGS} -Dshark.hive.shard=4:8 "test-only catalyst.execution.HiveCompatibility" + sbt ${ARGS} -Dshark.hive.shard=4:8 "test-only org.apache.spark.sql.shark.execution.HiveCompatibilitySuite" s6: compile - sbt ${ARGS} -Dshark.hive.shard=5:8 "test-only catalyst.execution.HiveCompatibility" + sbt ${ARGS} -Dshark.hive.shard=5:8 "test-only org.apache.spark.sql.shark.execution.HiveCompatibilitySuite" s7: compile - sbt ${ARGS} -Dshark.hive.shard=6:8 "test-only catalyst.execution.HiveCompatibility" + sbt ${ARGS} -Dshark.hive.shard=6:8 "test-only org.apache.spark.sql.shark.execution.HiveCompatibilitySuite" s8: compile - sbt ${ARGS} -Dshark.hive.shard=7:8 "test-only catalyst.execution.HiveCompatibility" + sbt ${ARGS} -Dshark.hive.shard=7:8 "test-only org.apache.spark.sql.shark.execution.HiveCompatibilitySuite" From 7f206b5aa577bc4ca8aeb82d2438ad43316eb996 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 14 Feb 2014 14:34:23 -0800 Subject: [PATCH 673/778] Add support for hive TABLESAMPLE PERCENT. --- .../main/scala/org/apache/spark/sql/shark/HiveQl.scala | 9 +++++++++ .../spark/sql/shark/execution/HiveQuerySuite.scala | 4 ++++ 2 files changed, 13 insertions(+) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala index c973ea5ef0efa..02e0fef37e20c 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala @@ -550,11 +550,20 @@ object HiveQl { Token("TOK_ROWCOUNT", Nil) :: Token(count, Nil) :: Nil) => StopAfter(Literal(count.toInt), relation) + case Token("TOK_TABLESPLITSAMPLE", + Token("TOK_PERCENT", Nil) :: + Token(fraction, Nil) :: Nil) => + Sample(fraction.toDouble, withReplacement = false, (math.random * 1000).toInt, relation) case Token("TOK_TABLEBUCKETSAMPLE", Token(numerator, Nil) :: Token(denominator, Nil) :: Nil) => val fraction = numerator.toDouble / denominator.toDouble Sample(fraction, withReplacement = false, (math.random * 1000).toInt, relation) + case a: ASTNode => + throw new NotImplementedError( + s"""No parse rules for sampling clause: ${a.getType}, text: ${a.getText} : + |${dumpTree(a).toString}" + + """.stripMargin) }.getOrElse(relation) case Token("TOK_UNIQUEJOIN", joinArgs) => diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala index 7f041167eb4e7..258b902826748 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala @@ -110,4 +110,8 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("lateral view6", "SELECT * FROM src LATERAL VIEW explode(map(key+3,key+4)) D as k, v") + test("sampling") { + sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") + } + } \ No newline at end of file From ed3a1d15b80768817e9259e31499df53587c51b2 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 15 Feb 2014 07:45:32 +0800 Subject: [PATCH 674/778] Load data directly into Hive. --- .../catalyst/execution/MetastoreCatalog.scala | 52 +++--- .../scala/catalyst/execution/TestShark.scala | 26 ++- .../catalyst/execution/hiveOperators.scala | 142 +++++++++----- src/main/scala/catalyst/frontend/Hive.scala | 4 + .../hadoop/mapred/SharkHadoopWriter.scala | 175 ++++++++++++++++++ 5 files changed, 313 insertions(+), 86 deletions(-) create mode 100644 src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 24040527793cc..965183f1f4256 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -4,12 +4,12 @@ package execution import scala.util.parsing.combinator.RegexParsers import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient -import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table} -import org.apache.hadoop.hive.metastore.api.{StorageDescriptor, SerDeInfo} +import org.apache.hadoop.hive.metastore.api.{FieldSchema, StorageDescriptor, SerDeInfo} +import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.serde2.AbstractDeserializer +import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.mapred.InputFormat import analysis.Catalog @@ -21,7 +21,7 @@ import catalyst.types._ import scala.collection.JavaConversions._ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { - val client = new HiveMetaStoreClient(hiveConf) + val client = Hive.get(hiveConf) def lookupRelation( db: Option[String], @@ -29,17 +29,18 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { alias: Option[String]): BaseRelation = { val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase()) val table = client.getTable(databaseName, tableName) - val hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) - val partitions = - if (hiveQlTable.isPartitioned) { - // TODO: Is 255 the right number to pick? - client.listPartitions(databaseName, tableName, 255).toSeq + val partitions: Seq[Partition] = + if (table.isPartitioned) { + client.getPartitions(table) } else { Nil } // Since HiveQL is case insensitive for table names we make them all lowercase. - MetastoreRelation(databaseName.toLowerCase, tableName.toLowerCase, alias)(table, partitions) + MetastoreRelation( + databaseName.toLowerCase, + tableName.toLowerCase, + alias)(table.getTTable, partitions.map(part => part.getTPartition)) } /** @@ -53,14 +54,13 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { case InsertIntoCreatedTable(db, tableName, child) => val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase()) - val table = new Table() + val table = new Table(databaseName, tableName) val schema = child.output.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), "")) + table.setFields(schema) - table.setDbName(databaseName) - table.setTableName(tableName) val sd = new StorageDescriptor() - table.setSd(sd) + table.getTTable.setSd(sd) sd.setCols(schema) // TODO: THESE ARE ALL DEFAULTS, WE NEED TO PARSE / UNDERSTAND the output specs. @@ -139,20 +139,28 @@ object HiveMetastoreTypes extends RegexParsers { } case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String]) - (val table: Table, val partitions: Seq[Partition]) + (val table: TTable, val partitions: Seq[TPartition]) extends BaseRelation { + // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and + // use org.apache.hadoop.hive.ql.metadata.Partition as the type of elements of partitions. + // Right now, using org.apache.hadoop.hive.ql.metadata.Table and + // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException + // which indicates the SerDe we used is not Serializable. - def hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) + def hiveQlTable = new Table(table) def hiveQlPartitions = partitions.map { p => - new org.apache.hadoop.hive.ql.metadata.Partition(hiveQlTable, p) + new Partition(hiveQlTable, p) } val tableDesc = new TableDesc( - Class.forName(table.getSd.getSerdeInfo.getSerializationLib) - .asInstanceOf[Class[AbstractDeserializer]], - Class.forName(table.getSd.getInputFormat).asInstanceOf[Class[InputFormat[_,_]]], - Class.forName(table.getSd.getOutputFormat), + Class.forName(hiveQlTable.getSerializationLib).asInstanceOf[Class[Deserializer]], + hiveQlTable.getInputFormatClass, + // The class of table should be org.apache.hadoop.hive.ql.metadata.Table because + // getOutputFormatClass will use HiveFileFormatUtils.getOutputFormatSubstitute to + // substitute some output formats, e.g. substituting SequenceFileOutputFormat to + // HiveSequenceFileOutputFormat. + hiveQlTable.getOutputFormatClass, hiveQlTable.getMetadata ) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index af6264cb7afb3..5135c7be39edd 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -12,6 +12,7 @@ import org.apache.hadoop.hive.metastore.api.{SerDeInfo, StorageDescriptor} import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.io.avro.{AvroContainerOutputFormat, AvroContainerInputFormat} +import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.serde2.avro.AvroSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.serde2.RegexSerDe @@ -205,19 +206,14 @@ class TestSharkInstance extends SharkInstance { import org.apache.hadoop.mapred.SequenceFileInputFormat import org.apache.hadoop.mapred.SequenceFileOutputFormat - val srcThrift = new org.apache.hadoop.hive.metastore.api.Table() - srcThrift.setTableName("src_thrift") - srcThrift.setDbName("default") - srcThrift.setSd(new StorageDescriptor) - srcThrift.getSd.setCols(Nil) - srcThrift.getSd.setInputFormat(classOf[SequenceFileInputFormat[_,_]].getName) - srcThrift.getSd.setOutputFormat(classOf[SequenceFileOutputFormat[_,_]].getName) - srcThrift.getSd.setSerdeInfo(new SerDeInfo) - srcThrift.getSd.getSerdeInfo.setSerializationLib(classOf[ThriftDeserializer].getName) - srcThrift.getSd.getSerdeInfo.setParameters(Map( - "serialization.class" -> classOf[Complex].getName, - "serialization.format" -> classOf[TBinaryProtocol].getName)) - + val srcThrift = new Table("default", "src_thrift") + srcThrift.setFields(Nil) + srcThrift.setInputFormatClass(classOf[SequenceFileInputFormat[_,_]].getName) + // In Hive, SequenceFileOutputFormat will be substituted by HiveSequenceFileOutputFormat. + srcThrift.setOutputFormatClass(classOf[SequenceFileOutputFormat[_,_]].getName) + srcThrift.setSerializationLib(classOf[ThriftDeserializer].getName) + srcThrift.setSerdeParam("serialization.class", classOf[Complex].getName) + srcThrift.setSerdeParam("serialization.format", classOf[TBinaryProtocol].getName) catalog.client.createTable(srcThrift) @@ -317,11 +313,11 @@ class TestSharkInstance extends SharkInstance { logger.debug(s"Deleting table $t") val table = catalog.client.getTable("default", t) - catalog.client.listIndexes("default", t, 255).foreach { index => + catalog.client.getIndexes("default", t, 255).foreach { index => catalog.client.dropIndex("default", t, index.getIndexName, true) } - if (!MetaStoreUtils.isIndexTable(table)) { + if (!table.isIndexTable) { catalog.client.dropTable("default", t) } } diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 61c11ee49acdd..6155979b8d764 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -3,22 +3,28 @@ package execution import java.io.File -import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} -import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils -import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Hive} import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} import org.apache.hadoop.hive.serde2.Serializer import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector -import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.mapred.JobConf -import org.apache.spark.SparkContext._ + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred._ import catalyst.expressions._ import catalyst.types.{BooleanType, DataType} +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.spark.{TaskContext, SparkException} +import org.apache.hadoop.io.SequenceFile.CompressionType +import scala.Some +import catalyst.expressions.Cast +import org.apache.spark.rdd.RDD /* Implicits */ import scala.collection.JavaConversions._ @@ -167,23 +173,9 @@ case class InsertIntoHiveTable( (@transient sc: SharkContext) extends UnaryNode { - /** - * This file sink / record writer code is only the first step towards implementing this operator - * correctly and is not actually used yet. - */ - val desc = new FileSinkDesc("./", table.tableDesc, false) - val outputClass = newSerializer(table.tableDesc).getSerializedClass - - lazy val conf = new JobConf() - - lazy val writer = HiveFileFormatUtils.getHiveRecordWriter( - conf, - table.tableDesc, - outputClass, - desc, - new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory, "test.out"), - null) + @transient private val hiveContext = new Context(sc.hiveconf) + @transient private val db = Hive.get(sc.hiveconf) private def newSerializer(tableDesc: TableDesc): Serializer = { val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] @@ -208,6 +200,63 @@ case class InsertIntoHiveTable( case (obj, _) => obj } + def saveAsHiveFile( + rdd: RDD[Writable], + valueClass: Class[_], + fileSinkConf: FileSinkDesc, + conf: JobConf, + codec: Option[Class[_ <: CompressionCodec]] = None) { + if (valueClass == null) { + throw new SparkException("Output value class not set") + } + conf.setOutputValueClass(valueClass) + if (fileSinkConf.getTableInfo.getOutputFileFormatClassName == null) { + throw new SparkException("Output format class not set") + } + // Doesn't work in Scala 2.9 due to what may be a generics bug + // TODO: Should we uncomment this for Scala 2.10? + // conf.setOutputFormat(outputFormatClass) + conf.set("mapred.output.format.class", fileSinkConf.getTableInfo.getOutputFileFormatClassName) + for (c <- codec) { + conf.setCompressMapOutput(true) + conf.set("mapred.output.compress", "true") + conf.setMapOutputCompressorClass(c) + conf.set("mapred.output.compression.codec", c.getCanonicalName) + conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) + } + conf.setOutputCommitter(classOf[FileOutputCommitter]) + FileOutputFormat.setOutputPath( + conf, + SharkHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) + + logger.debug("Saving as hadoop file of type " + valueClass.getSimpleName) + + val writer = new SharkHadoopWriter(conf, fileSinkConf) + writer.preSetup() + + def writeToFile(context: TaskContext, iter: Iterator[Writable]) { + // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it + // around by taking a mod. We expect that no task will be attempted 2 billion times. + val attemptNumber = (context.attemptId % Int.MaxValue).toInt + + writer.setup(context.stageId, context.partitionId, attemptNumber) + writer.open() + + var count = 0 + while(iter.hasNext) { + val record = iter.next() + count += 1 + writer.write(record) + } + + writer.close() + writer.commit() + } + + sc.runJob(rdd, writeToFile _) + writer.commitJob() + } + /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the * `org.apache.hadoop.hive.serde2.SerDe` and the @@ -217,19 +266,17 @@ case class InsertIntoHiveTable( val childRdd = child.execute() assert(childRdd != null) - // TODO write directly to Hive - val tempDir = File.createTempFile("catalysthiveout", "") - tempDir.delete() - tempDir.mkdir() - // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer // instances within the closure, since Serializer is not serializable while TableDesc is. val tableDesc = table.tableDesc - childRdd.mapPartitions { iter => - val serializer = newSerializer(tableDesc) + val tableLocation = table.hiveQlTable.getDataLocation + val tmpLocation = hiveContext.getExternalTmpFileURI(tableLocation) + val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) + val rdd = childRdd.mapPartitions { iter => + val serializer = newSerializer(fileSinkConf.getTableInfo) val standardOI = ObjectInspectorUtils .getStandardObjectInspector( - tableDesc.getDeserializer.getObjectInspector, + fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, ObjectInspectorCopyOption.JAVA) .asInstanceOf[StructObjectInspector] @@ -237,27 +284,24 @@ case class InsertIntoHiveTable( // Casts Strings to HiveVarchars when necessary. val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector) val mappedRow = row.zip(fieldOIs).map(wrap) - (null, serializer.serialize(mappedRow.toArray, standardOI)) - } - }.saveAsHadoopFile( - tempDir.getCanonicalPath, - classOf[NullWritable], - outputClass, - tableDesc.getOutputFileFormatClass) - - val partitionSpec = if (partition.nonEmpty) { - partition.map { - case (k, Some(v)) => s"$k=$v" - // Dynamic partition inserts - case (k, None) => s"$k" - }.mkString(" PARTITION (", ", ", ")") - } else { - "" + + serializer.serialize(mappedRow.toArray, standardOI) } + } - val inpath = tempDir.getCanonicalPath + "/*" - sc.runHive(s"""LOAD DATA LOCAL INPATH '$inpath' INTO TABLE - ${table.databaseName}.${table.tableName}$partitionSpec""") + saveAsHiveFile( + rdd, + outputClass, + fileSinkConf, + new JobConf(sc.hiveconf)) + + // TODO: Correctly set replace and holdDDLTime. + db.loadTable( + new Path(fileSinkConf.getDirName), + // Have to construct the format of dbname.tablename. + s"${table.databaseName}.${table.tableName}", + false, + false) // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 706c2d8664188..02d3f279c996c 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -354,6 +354,7 @@ object HiveQl { case Token("TOK_CREATETABLE", children) if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty => + // TODO: Parse other properties. val (Some(tableNameParts) :: _ /* likeTable */ :: Some(query) :: Nil) = getClauses(Seq("TOK_TABNAME", "TOK_LIKETABLE", "TOK_QUERY"), children) @@ -498,6 +499,9 @@ object HiveQl { // There are two tokens for specifying where to sent the result that seem to be used almost // interchangeably. + // TOK_INSERT_INTO means to add files to the table. + // TOK_DESTINATION means to overwrite the table. + // TODO: We need to distinguish TOK_INSERT_INTO and TOK_DESTINATION. val resultDestination = (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) diff --git a/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala b/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala new file mode 100644 index 0000000000000..44acf02f0ce4a --- /dev/null +++ b/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala @@ -0,0 +1,175 @@ +package org.apache.hadoop.mapred + +import java.io.IOException +import java.text.NumberFormat +import java.util.Date + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.Writable + +import org.apache.spark.Logging +import org.apache.spark.SerializableWritable + +import org.apache.hadoop.hive.ql.exec.FileSinkOperator +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} + +/** + * Internal helper class that saves an RDD using a Hive OutputFormat. + * It is based on [[SparkHadoopWriter]]. + */ +class SharkHadoopWriter( + @transient jobConf: JobConf, + fileSinkConf: FileSinkDesc) + extends Logging + with SparkHadoopMapRedUtil + with Serializable { + + private val now = new Date() + private val conf = new SerializableWritable(jobConf) + + private var jobID = 0 + private var splitID = 0 + private var attemptID = 0 + private var jID: SerializableWritable[JobID] = null + private var taID: SerializableWritable[TaskAttemptID] = null + + @transient private var writer: FileSinkOperator.RecordWriter = null + @transient private var format: HiveOutputFormat[AnyRef, Writable] = null + @transient private var committer: OutputCommitter = null + @transient private var jobContext: JobContext = null + @transient private var taskContext: TaskAttemptContext = null + + def preSetup() { + setIDs(0, 0, 0) + setConfParams() + + val jCtxt = getJobContext() + getOutputCommitter().setupJob(jCtxt) + } + + + def setup(jobid: Int, splitid: Int, attemptid: Int) { + setIDs(jobid, splitid, attemptid) + setConfParams() + } + + def open() { + val numfmt = NumberFormat.getInstance() + numfmt.setMinimumIntegerDigits(5) + numfmt.setGroupingUsed(false) + + val outputName = "part-" + numfmt.format(splitID) + val path = FileOutputFormat.getOutputPath(conf.value) + + getOutputCommitter().setupTask(getTaskContext()) + writer = HiveFileFormatUtils.getHiveRecordWriter( + conf.value, + fileSinkConf.getTableInfo, + conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], + fileSinkConf, + new Path(path, outputName), + null) + } + + def write(value: Writable) { + if (writer != null) { + writer.write(value) + } else { + throw new IOException("Writer is null, open() has not been called") + } + } + + def close() { + // Seems the boolean value passed into close does not matter. + writer.close(false) + } + + def commit() { + val taCtxt = getTaskContext() + val cmtr = getOutputCommitter() + if (cmtr.needsTaskCommit(taCtxt)) { + try { + cmtr.commitTask(taCtxt) + logInfo (taID + ": Committed") + } catch { + case e: IOException => { + logError("Error committing the output of task: " + taID.value, e) + cmtr.abortTask(taCtxt) + throw e + } + } + } else { + logWarning ("No need to commit output of task: " + taID.value) + } + } + + def commitJob() { + // always ? Or if cmtr.needsTaskCommit ? + val cmtr = getOutputCommitter() + cmtr.commitJob(getJobContext()) + } + + // ********* Private Functions ********* + + private def getOutputFormat(): HiveOutputFormat[AnyRef,Writable] = { + if (format == null) { + format = conf.value.getOutputFormat() + .asInstanceOf[HiveOutputFormat[AnyRef,Writable]] + } + format + } + + private def getOutputCommitter(): OutputCommitter = { + if (committer == null) { + committer = conf.value.getOutputCommitter + } + committer + } + + private def getJobContext(): JobContext = { + if (jobContext == null) { + jobContext = newJobContext(conf.value, jID.value) + } + jobContext + } + + private def getTaskContext(): TaskAttemptContext = { + if (taskContext == null) { + taskContext = newTaskAttemptContext(conf.value, taID.value) + } + taskContext + } + + private def setIDs(jobid: Int, splitid: Int, attemptid: Int) { + jobID = jobid + splitID = splitid + attemptID = attemptid + + jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobid)) + taID = new SerializableWritable[TaskAttemptID]( + new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) + } + + private def setConfParams() { + conf.value.set("mapred.job.id", jID.value.toString) + conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) + conf.value.set("mapred.task.id", taID.value.toString) + conf.value.setBoolean("mapred.task.is.map", true) + conf.value.setInt("mapred.task.partition", splitID) + } +} + +object SharkHadoopWriter { + def createPathFromString(path: String, conf: JobConf): Path = { + if (path == null) { + throw new IllegalArgumentException("Output path is null") + } + val outputPath = new Path(path) + val fs = outputPath.getFileSystem(conf) + if (outputPath == null || fs == null) { + throw new IllegalArgumentException("Incorrectly formatted output path") + } + outputPath.makeQualified(fs) + } +} \ No newline at end of file From 346f828dc37df3a1681e6ebf2a5940a609ead50a Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 15 Feb 2014 08:38:52 +0800 Subject: [PATCH 675/778] Move SharkHadoopWriter to the correct location. --- .../main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala | 0 .../main/scala/org/apache/spark/sql/shark/hiveOperators.scala | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename {src => shark/src}/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala (100%) diff --git a/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala b/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala similarity index 100% rename from src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala rename to shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index 140d871b44805..bd12a3a88630b 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -255,7 +255,7 @@ case class InsertIntoHiveTable( writer.commit() } - sc.runJob(rdd, writeToFile _) + sc.sparkContext.runJob(rdd, writeToFile _) writer.commitJob() } From a9c318853d4bb02965252810656999be060682dd Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Fri, 14 Feb 2014 17:06:00 -0800 Subject: [PATCH 676/778] Fix udaf struct return --- .../spark/sql/shark/hiveOperators.scala | 22 ++------------ .../org/apache/spark/sql/shark/hiveUdfs.scala | 30 +++++++++++++++---- 2 files changed, 26 insertions(+), 26 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index 41f69604b3d2d..957de28dfc945 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -39,7 +39,8 @@ case class HiveTableScan( relation: MetastoreRelation, partitionPruningPred: Option[Expression])( @transient val sc: SharkContext) - extends LeafNode { + extends LeafNode + with HiveInspectors { require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, "Partition pruning predicates only supported for partitioned tables.") @@ -91,25 +92,6 @@ case class HiveTableScan( } } - def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { - case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) - case li: ListObjectInspector => - Option(li.getList(data)) - .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) - .orNull - case mi: MapObjectInspector => - Option(mi.getMap(data)).map( - _.map { - case (k,v) => - (unwrapData(k, mi.getMapKeyObjectInspector), - unwrapData(v, mi.getMapValueObjectInspector)) - }.toMap).orNull - case si: StructObjectInspector => - val allRefs = si.getAllStructFieldRefs - new GenericRow( - allRefs.map(r => unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector))) - } - private def castFromString(value: String, dataType: DataType) = { Evaluate(Cast(Literal(value), dataType), Nil) } diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala index 7118b16fe9055..b8d820f0b09b0 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala @@ -204,6 +204,26 @@ case class HiveGenericUdf( } trait HiveInspectors { + + def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { + case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) + case li: ListObjectInspector => + Option(li.getList(data)) + .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) + .orNull + case mi: MapObjectInspector => + Option(mi.getMap(data)).map( + _.map { + case (k,v) => + (unwrapData(k, mi.getMapKeyObjectInspector), + unwrapData(v, mi.getMapValueObjectInspector)) + }.toMap).orNull + case si: StructObjectInspector => + val allRefs = si.getAllStructFieldRefs + new GenericRow( + allRefs.map(r => unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector))) + } + /** Converts native catalyst types to the types expected by Hive */ def wrap(a: Any): AnyRef = a match { case s: String => new hadoopIo.Text(s) @@ -399,17 +419,15 @@ case class HiveUdafFunction( private val inspectors = exprs.map(_.dataType).map(toInspector).toArray - private val function = { - val evaluator = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray) - evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) - evaluator - } + private val function = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray) + + private val returnInspector = function.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) // Cast required to avoid type inference selecting a deprecated Hive API. private val buffer = function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] - def result: Any = unwrap(function.evaluate(buffer)) + def result: Any = unwrapData(function.evaluate(buffer), returnInspector) def apply(input: Seq[Row]): Unit = { val inputs = exprs.map(Evaluate(_, input).asInstanceOf[AnyRef]).toArray From 69adf7298edb74a9ecd704932276d988d1c8ba5d Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 15 Feb 2014 09:22:13 +0800 Subject: [PATCH 677/778] Set cloneRecords to false. --- .../scala/org/apache/spark/sql/shark/TableReader.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala b/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala index 9a1c0cb8a8a2a..0ede070d10df8 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala @@ -204,7 +204,12 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: SharkCo inputFormatClass, classOf[Writable], classOf[Writable], - _minSplitsPerRDD) + _minSplitsPerRDD, + // Have to set cloneRecords to false. When reading a table stored by ORC, the type of + // a row is OrcStruct which does not have a no-argument constructor. OrcStruct will + // cause an error when cloning a row. + // TODO: Will setting cloneRecords to false cause any problem? + false) // Only take the value (skip the key) because Hive works only with values. rdd.map(_._2) From 566fd6685fec88b88223f4b47af04eb39a69d28e Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Fri, 14 Feb 2014 18:09:30 -0800 Subject: [PATCH 678/778] Whitelist tests and add support for Binary type --- .../main/scala/org/apache/spark/sql/shark/hiveUdfs.scala | 2 ++ .../alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb | 2 +- .../alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 | 2 +- .../golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 | 2 +- .../golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 | 2 +- .../ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 | 2 +- ...ary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 | 2 +- ...binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 | 2 +- .../spark/sql/shark/execution/HiveCompatibilitySuite.scala | 6 ++++++ 9 files changed, 15 insertions(+), 7 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala index b8d820f0b09b0..00ee25d6ee0c6 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala @@ -253,6 +253,7 @@ trait HiveInspectors { case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector + case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector } def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { @@ -287,6 +288,7 @@ trait HiveInspectors { import TypeInfoFactory._ def toTypeInfo: TypeInfo = dt match { + case BinaryType => binaryTypeInfo case BooleanType => booleanTypeInfo case ByteType => byteTypeInfo case DoubleType => doubleTypeInfo diff --git a/shark/src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb b/shark/src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb index 43b9042d0bffe..ee76e02af3aba 100644 --- a/shark/src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb +++ b/shark/src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb @@ -2,4 +2,4 @@ foo int None bar string None ds string None -Detailed Table Information Table(tableName:view1, dbName:default, owner:marmbrus, createTime:1392059376, 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=1392059376}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW) +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/shark/src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 b/shark/src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 index 0419cd9376ce4..8603577477bfc 100644 --- a/shark/src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 +++ b/shark/src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 @@ -2,4 +2,4 @@ foo int None bar string None ds string None -Detailed Table Information Table(tableName:view2, dbName:default, owner:marmbrus, createTime:1392059376, 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=marmbrus, last_modified_time=1392059376, transient_lastDdlTime=1392059376}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW) +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/shark/src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 b/shark/src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 index 03866397e028e..d0eea8a3ca661 100644 --- a/shark/src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 +++ b/shark/src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 @@ -1,4 +1,4 @@ ba_key binary None ba_val binary None -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:marmbrus, createTime:1392059487, 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:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/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=1392059487}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +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/shark/src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 b/shark/src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 index f23b62931be46..0a3c9f383a6ef 100644 --- a/shark/src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 +++ b/shark/src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 @@ -1,4 +1,4 @@ ba_key binary from deserializer ba_val binary from deserializer -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:marmbrus, createTime:1392059487, 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:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/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=marmbrus, last_modified_time=1392059487, transient_lastDdlTime=1392059487}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +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/shark/src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 b/shark/src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 index af60a5daa50a4..66d33b789f1a7 100644 --- a/shark/src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 +++ b/shark/src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 @@ -1,4 +1,4 @@ ba_key binary None ba_val binary None -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:marmbrus, createTime:1392059489, 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:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/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=1392059489}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +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/shark/src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 b/shark/src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 index 9bf06320a9e6b..df14abbcc193f 100644 --- a/shark/src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 +++ b/shark/src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 @@ -1,4 +1,4 @@ ba_key binary from deserializer ba_val binary from deserializer -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:marmbrus, createTime:1392059490, 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:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/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=marmbrus, last_modified_time=1392059490, transient_lastDdlTime=1392059490}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +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/shark/src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 b/shark/src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 index f5117886c8b1b..0c246ae33e56f 100644 --- a/shark/src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 +++ b/shark/src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 @@ -1,4 +1,4 @@ ba_key binary from deserializer ba_val binary from deserializer -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:marmbrus, createTime:1392059491, 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:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/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=1392059491}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +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/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala index f65f219019896..47773532fedbe 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala @@ -211,6 +211,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "binary_constant", "binarysortable_1", "combine1", + "compute_stats_binary", + "compute_stats_boolean", + "compute_stats_double", + "compute_stats_table", + "compute_stats_long", + "compute_stats_string", "convert_enum_to_string", "correlationoptimizer11", "correlationoptimizer15", From 8506c176f7e18011df50e25f8ea98d30a57f0ccd Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 14 Feb 2014 19:20:41 -0800 Subject: [PATCH 679/778] Address review feedback. --- README.md | 29 ++++++++++--------- .../spark/sql/catalyst/analysis/Catalog.scala | 6 ++++ .../apache/spark/sql/execution/package.scala | 8 +---- .../apache/spark/sql/shark/SharkContext.scala | 9 ++---- .../org/apache/spark/sql/shark/hiveUdfs.scala | 6 ++-- .../shark/execution/HiveComparisonTest.scala | 15 ++++++---- 6 files changed, 38 insertions(+), 35 deletions(-) diff --git a/README.md b/README.md index 503235fb4ccb9..9ee6407078227 100644 --- a/README.md +++ b/README.md @@ -24,7 +24,7 @@ $ sbt/sbt > test:compile [info] Compiling 10 Scala sources to catalyst/target/scala-2.10/test-classes... [success] Total time: 15 s, completed Dec 20, 2013 12:00:06 PM -> test-only catalyst.execution.DslQueryTests +> core/test-only catalyst.execution.BasicQuerySuite ``` Any command that is prefixed with a `~` (e.g. `~compile`) will be run automatically in a loop each time any dependent files have changed. @@ -41,27 +41,28 @@ export HADOOP_HOME="/hadoop-1.0.4" Using the console ================= -An interactive scala console can be invoked by running `sbt/sbt console`. From here you can execute queries and inspect the various stages of query optimization. +An interactive scala console can be invoked by running `sbt/sbt shark/console`. From here you can execute queries and inspect the various stages of query optimization. ```scala -catalyst$ sbt/sbt console +catalyst$ sbt/sbt shark/console [info] Starting scala interpreter... -import catalyst.analysis._ -import catalyst.dsl._ -import catalyst.errors._ -import catalyst.expressions._ -import catalyst.frontend._ -import catalyst.plans.logical._ -import catalyst.rules._ -import catalyst.types._ -import catalyst.util._ -import catalyst.execution.TestShark._ +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.dsl._ +import org.apache.spark.sql.catalyst.errors._ +import org.apache.spark.sql.catalyst.expressions._ +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.catalyst.util._ +import org.apache.spark.sql.execution +import org.apache.spark.sql.shark._ +import org.apache.spark.sql.shark.TestShark._ Welcome to Scala version 2.10.3 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45). Type in expressions to have them evaluated. Type :help for more information. -scala> scala> val query = sql("SELECT * FROM (SELECT * FROM src) a") +scala> val query = sql("SELECT * FROM (SELECT * FROM src) a") query: org.apache.spark.sql.ExecutedQuery = SELECT * FROM (SELECT * FROM src) a === Query Plan === diff --git a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 55e50f774535c..fd753d4203dc2 100644 --- a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -15,6 +15,12 @@ trait Catalog { alias: Option[String] = None): LogicalPlan } +/** + * A trait that can be mixed in with other Catalogs allowing specific tables to be overridden with + * new logical plans. This can be used to bind query result to virtual tables, or replace tables + * with in-memory cached versions. Note that the set of overrides is stored in memory and thus + * lost when the JVM exits. + */ trait OverrideCatalog extends Catalog { // TODO: This doesn't work when the database changes... diff --git a/core/src/main/scala/org/apache/spark/sql/execution/package.scala b/core/src/main/scala/org/apache/spark/sql/execution/package.scala index 187a745e0ac95..aef8ee9ccfd05 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/package.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/package.scala @@ -1,13 +1,7 @@ package org.apache.spark.sql /** - * A partial reimplementation of Shark, a Hive compatible SQL engine running on Spark. - * - * This implementation uses the hive parser, metadata catalog and serdes, but performs all - * optimization and execution using catalyst and spark. - * - * Currently functions that are not supported by this implementation are passed back to the - * original Shark implementation for execution. + * An execution engine for relational query plans that runs on top Spark and returns RDDs. */ package object execution { type Row = catalyst.expressions.Row diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala b/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala index e118cd2d6d108..912e55c34b6af 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala @@ -154,15 +154,12 @@ abstract class SharkContext(sc: SparkContext) extends SparkSqlContext(sc) { } } catch { case e: Exception => - println( - """ + logger.error( + s""" |====================== |HIVE FAILURE OUTPUT |====================== - """.stripMargin) - println(outputBuffer.toString) - println( - """ + |${outputBuffer.toString} |====================== |END HIVE FAILURE OUTPUT |====================== diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala index 00ee25d6ee0c6..28662189e2b78 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala @@ -408,9 +408,9 @@ case class HiveGenericUdtf( } case class HiveUdafFunction( - functionName: String, - exprs: Seq[Expression], - base: AggregateExpression) + functionName: String, + exprs: Seq[Expression], + base: AggregateExpression) extends AggregateFunction with HiveInspectors with HiveFunctionFactory { diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveComparisonTest.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveComparisonTest.scala index 9e540c9fc7f95..1e640d3080cf1 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveComparisonTest.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveComparisonTest.scala @@ -63,31 +63,36 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G /** The local directory with cached golden answer will be stored. */ protected val answerCache = new File("shark/src/test/hive/golden") - if (!answerCache.exists) + if (!answerCache.exists) { answerCache.mkdir() + } /** The [[ClassLoader]] that contains test dependencies. Used to look for golden answers. */ protected val testClassLoader = this.getClass.getClassLoader /** Directory containing a file for each test case that passes. */ val passedDirectory = new File(targetDir, s"$suiteName.passed") - if (!passedDirectory.exists()) + if (!passedDirectory.exists()) { passedDirectory.mkdir() // Not atomic! + } /** Directory containing output of tests that fail to execute with Catalyst. */ val failedDirectory = new File(targetDir, s"$suiteName.failed") - if (!failedDirectory.exists()) + if (!failedDirectory.exists()) { failedDirectory.mkdir() // Not atomic! + } /** Directory containing output of tests where catalyst produces the wrong answer. */ val wrongDirectory = new File(targetDir, s"$suiteName.wrong") - if (!wrongDirectory.exists()) + if (!wrongDirectory.exists()) { wrongDirectory.mkdir() // Not atomic! + } /** Directory containing output of tests where we fail to generate golden output with Hive. */ val hiveFailedDirectory = new File(targetDir, s"$suiteName.hiveFailed") - if (!hiveFailedDirectory.exists()) + if (!hiveFailedDirectory.exists()) { hiveFailedDirectory.mkdir() // Not atomic! + } /** All directories that contain per-query output files */ val outputDirectories = Seq( From 3bb272ddc69472120bb0915308451576565cecf6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 14 Feb 2014 19:26:42 -0800 Subject: [PATCH 680/778] move org.apache.spark.sql package.scala to the correct location. --- .../main/scala/org/apache/spark/sql/{catalyst => }/package.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename catalyst/src/main/scala/org/apache/spark/sql/{catalyst => }/package.scala (100%) diff --git a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/package.scala similarity index 100% rename from catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/package.scala From 1596e1b14e8e2741758c6370bb29d32830476a7f Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 15 Feb 2014 12:09:25 +0800 Subject: [PATCH 681/778] Cleanup imports to make IntelliJ happy. --- .../scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala | 2 +- .../org/apache/spark/sql/shark/HiveMetastoreCatalog.scala | 3 --- .../org/apache/spark/sql/shark/ScriptTransformation.scala | 2 +- .../main/scala/org/apache/spark/sql/shark/SharkContext.scala | 2 +- .../scala/org/apache/spark/sql/shark/SharkStrategies.scala | 4 ++-- .../main/scala/org/apache/spark/sql/shark/TableReader.scala | 1 - .../scala/org/apache/spark/sql/shark/hiveOperators.scala | 5 +---- 7 files changed, 6 insertions(+), 13 deletions(-) diff --git a/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala b/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala index 44acf02f0ce4a..095d9eeada46b 100644 --- a/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala +++ b/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala @@ -12,7 +12,7 @@ import org.apache.spark.SerializableWritable import org.apache.hadoop.hive.ql.exec.FileSinkOperator import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.FileSinkDesc /** * Internal helper class that saves an RDD using a Hive OutputFormat. diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala index 9fb22f8a39254..4542f64956524 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala @@ -3,14 +3,12 @@ package shark import scala.util.parsing.combinator.RegexParsers -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{FieldSchema, StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.Deserializer -import org.apache.hadoop.mapred.InputFormat import catalyst.analysis.Catalog import catalyst.expressions._ @@ -18,7 +16,6 @@ import catalyst.plans.logical import catalyst.plans.logical._ import catalyst.rules._ import catalyst.types._ -import execution._ import scala.collection.JavaConversions._ diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/ScriptTransformation.scala b/shark/src/main/scala/org/apache/spark/sql/shark/ScriptTransformation.scala index f86650d3faf3d..6906588f19272 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/ScriptTransformation.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/ScriptTransformation.scala @@ -4,7 +4,7 @@ package shark import java.io.{InputStreamReader, BufferedReader} import catalyst.expressions._ -import execution._ +import org.apache.spark.sql.execution._ import scala.collection.JavaConversions._ diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala b/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala index e118cd2d6d108..58015753aa4a1 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala @@ -18,7 +18,7 @@ import catalyst.expressions.GenericRow import catalyst.plans.logical.{BaseRelation, LogicalPlan, NativeCommand, ExplainCommand} import catalyst.types._ -import execution._ +import org.apache.spark.sql.execution._ import scala.collection.JavaConversions._ diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala b/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala index 1cb0cdba8c5c7..6741b5481c17f 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala @@ -6,7 +6,7 @@ import catalyst.planning._ import catalyst.plans._ import catalyst.plans.logical.LogicalPlan -import execution.SparkPlan +import org.apache.spark.sql.execution._ trait SharkStrategies { // Possibly being too clever with types here... or not clever enough. @@ -75,7 +75,7 @@ trait SharkStrategies { otherPredicates .reduceLeftOption(And) - .map(execution.Filter(_, scan)) + .map(Filter(_, scan)) .getOrElse(scan) :: Nil case _ => diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala b/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala index 0ede070d10df8..9897e11936229 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala @@ -5,7 +5,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.io.Writable diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index bd12a3a88630b..b00106a87884f 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -1,8 +1,6 @@ package org.apache.spark.sql package shark -import java.io.File - import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Hive} @@ -25,8 +23,7 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import scala.Some import catalyst.expressions.Cast import org.apache.spark.rdd.RDD - -import execution._ +import org.apache.spark.sql.execution._ /* Implicits */ import scala.collection.JavaConversions._ From 5495faba864ee7ef1f8649bca02eacb7479a3b2a Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 15 Feb 2014 18:01:02 +0800 Subject: [PATCH 682/778] Remove cloneRecords which is no longer needed. --- .../scala/org/apache/spark/sql/shark/TableReader.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala b/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala index 9897e11936229..ca56b65bc7c9d 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/TableReader.scala @@ -203,12 +203,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: SharkCo inputFormatClass, classOf[Writable], classOf[Writable], - _minSplitsPerRDD, - // Have to set cloneRecords to false. When reading a table stored by ORC, the type of - // a row is OrcStruct which does not have a no-argument constructor. OrcStruct will - // cause an error when cloning a row. - // TODO: Will setting cloneRecords to false cause any problem? - false) + _minSplitsPerRDD) // Only take the value (skip the key) because Hive works only with values. rdd.map(_._2) From bdab5edd65140cd18c2dc29b00fa914d624dd999 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 15 Feb 2014 18:03:28 +0800 Subject: [PATCH 683/778] Add a TODO for loading data into partitioned tables. --- .../main/scala/org/apache/spark/sql/shark/hiveOperators.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index b00106a87884f..0f771a8e184de 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -295,6 +295,7 @@ case class InsertIntoHiveTable( new JobConf(sc.hiveconf)) // TODO: Correctly set replace and holdDDLTime. + // TODO: Handle loading into partitioned tables. db.loadTable( new Path(fileSinkConf.getDirName), // Have to construct the format of dbname.tablename. From 563bb22bd30b021e2bc276e2ed454f5296877a63 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 16 Feb 2014 08:26:05 +0800 Subject: [PATCH 684/778] Set compression info in FileSinkDesc. --- .../org/apache/spark/sql/shark/hiveOperators.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index 0f771a8e184de..dc047d4ac6a34 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -222,6 +222,9 @@ case class InsertIntoHiveTable( conf.setMapOutputCompressorClass(c) conf.set("mapred.output.compression.codec", c.getCanonicalName) conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) + fileSinkConf.setCompressed(true) + fileSinkConf.setCompressCodec(c.getCanonicalName) + fileSinkConf.setCompressType(CompressionType.BLOCK.toString) } conf.setOutputCommitter(classOf[FileOutputCommitter]) FileOutputFormat.setOutputPath( @@ -288,11 +291,16 @@ case class InsertIntoHiveTable( } } + // TODO: Correctly set codec. + // ORC stores compression information in table properties. + // While, there are other formats (e.g. RCFile) that rely on + // hive configurations to store compression information. saveAsHiveFile( rdd, outputClass, fileSinkConf, - new JobConf(sc.hiveconf)) + new JobConf(sc.hiveconf), + codec = None) // TODO: Correctly set replace and holdDDLTime. // TODO: Handle loading into partitioned tables. From e08962779a195b991c2478647c65923f4ddd23b4 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 16 Feb 2014 08:26:23 +0800 Subject: [PATCH 685/778] Code style. --- .../scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala b/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala index 095d9eeada46b..c2379c1b7043f 100644 --- a/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala +++ b/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala @@ -18,9 +18,10 @@ import org.apache.hadoop.hive.ql.plan.FileSinkDesc * Internal helper class that saves an RDD using a Hive OutputFormat. * It is based on [[SparkHadoopWriter]]. */ +protected class SharkHadoopWriter( - @transient jobConf: JobConf, - fileSinkConf: FileSinkDesc) + @transient jobConf: JobConf, + fileSinkConf: FileSinkDesc) extends Logging with SparkHadoopMapRedUtil with Serializable { From eea75c522fbf9ead1ef4280e3420d3a6685b7a0c Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 16 Feb 2014 19:24:15 +0800 Subject: [PATCH 686/778] Correctly set codec. --- .../spark/sql/shark/hiveOperators.scala | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index 88cf5b335cd38..758b899184da1 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -186,7 +186,7 @@ case class InsertIntoHiveTable( valueClass: Class[_], fileSinkConf: FileSinkDesc, conf: JobConf, - codec: Option[Class[_ <: CompressionCodec]] = None) { + isCompressed: Boolean) { if (valueClass == null) { throw new SparkException("Output value class not set") } @@ -198,15 +198,14 @@ case class InsertIntoHiveTable( // TODO: Should we uncomment this for Scala 2.10? // conf.setOutputFormat(outputFormatClass) conf.set("mapred.output.format.class", fileSinkConf.getTableInfo.getOutputFileFormatClassName) - for (c <- codec) { - conf.setCompressMapOutput(true) + if (isCompressed) { + // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", + // and "mapred.output.compression.type" have no impact on ORC because it uses table properties + // to store compression information. conf.set("mapred.output.compress", "true") - conf.setMapOutputCompressorClass(c) - conf.set("mapred.output.compression.codec", c.getCanonicalName) - conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) fileSinkConf.setCompressed(true) - fileSinkConf.setCompressCodec(c.getCanonicalName) - fileSinkConf.setCompressType(CompressionType.BLOCK.toString) + fileSinkConf.setCompressCodec(conf.get("mapred.output.compression.codec")) + fileSinkConf.setCompressType(conf.get("mapred.output.compression.type")) } conf.setOutputCommitter(classOf[FileOutputCommitter]) FileOutputFormat.setOutputPath( @@ -273,16 +272,14 @@ case class InsertIntoHiveTable( } } - // TODO: Correctly set codec. - // ORC stores compression information in table properties. - // While, there are other formats (e.g. RCFile) that rely on - // hive configurations to store compression information. + // ORC stores compression information in table properties. While, there are other formats + // (e.g. RCFile) that rely on hadoop configurations to store compression information. saveAsHiveFile( rdd, outputClass, fileSinkConf, new JobConf(sc.hiveconf), - codec = None) + sc.hiveconf.getBoolean("hive.exec.compress.output", false)) // TODO: Correctly set replace and holdDDLTime. // TODO: Handle loading into partitioned tables. From 428aff5f15a1954a983f049ade8986816d87e73c Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 16 Feb 2014 20:39:24 +0800 Subject: [PATCH 687/778] Distinguish `INSERT INTO` and `INSERT OVERWRITE`. --- .../sql/catalyst/plans/logical/basicOperators.scala | 3 ++- .../apache/hadoop/mapred/SharkHadoopWriter.scala | 2 +- .../spark/sql/shark/HiveMetastoreCatalog.scala | 5 +++-- .../scala/org/apache/spark/sql/shark/HiveQl.scala | 13 ++++++++----- .../apache/spark/sql/shark/SharkStrategies.scala | 4 ++-- .../org/apache/spark/sql/shark/hiveOperators.scala | 9 ++++++--- .../spark/sql/shark/execution/HiveQuerySuite.scala | 10 ++++++++++ 7 files changed, 32 insertions(+), 14 deletions(-) diff --git a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 69b85ec553662..3b8d62b4475e7 100644 --- a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -71,7 +71,8 @@ case class Join( case class InsertIntoTable( table: BaseRelation, partition: Map[String, Option[String]], - child: LogicalPlan) + child: LogicalPlan, + overwrite: Boolean) extends LogicalPlan { // The table being inserted into is a child for the purposes of transformations. def children = table :: child :: Nil diff --git a/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala b/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala index c2379c1b7043f..cffb9d2166e36 100644 --- a/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala +++ b/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala @@ -18,7 +18,7 @@ import org.apache.hadoop.hive.ql.plan.FileSinkDesc * Internal helper class that saves an RDD using a Hive OutputFormat. * It is based on [[SparkHadoopWriter]]. */ -protected +protected[apache] class SharkHadoopWriter( @transient jobConf: JobConf, fileSinkConf: FileSinkDesc) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala index 4542f64956524..b4129a45fad34 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala @@ -77,7 +77,8 @@ class HiveMetastoreCatalog(shark: SharkContext) extends Catalog with Logging { InsertIntoTable( lookupRelation(Some(databaseName), tableName, None).asInstanceOf[BaseRelation], Map.empty, - child) + child, + false) } } @@ -90,7 +91,7 @@ class HiveMetastoreCatalog(shark: SharkContext) extends Catalog with Logging { // Wait until children are resolved case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table: MetastoreRelation, _, child) => + case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => val childOutputDataTypes = child.output.map(_.dataType) // Only check attributes, not partitionKeys since they are always strings. // TODO: Fully support inserting into partitioned tables. diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala index 24afb4fa9d34a..29234fd4fa51a 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala @@ -485,13 +485,13 @@ object HiveQl { // interchangeably. // TOK_INSERT_INTO means to add files to the table. // TOK_DESTINATION means to overwrite the table. - // TODO: We need to distinguish TOK_INSERT_INTO and TOK_DESTINATION. val resultDestination = (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) - + val overwrite = if (intoClause.isEmpty) true else false nodeToDest( resultDestination, - withLimit) + withLimit, + overwrite) } // If there are multiple INSERTS just UNION them together into on query. @@ -646,7 +646,10 @@ object HiveQl { } val destinationToken = "TOK_DESTINATION|TOK_INSERT_INTO".r - protected def nodeToDest(node: Node, query: LogicalPlan): LogicalPlan = node match { + protected def nodeToDest( + node: Node, + query: LogicalPlan, + overwrite: Boolean): LogicalPlan = node match { case Token(destinationToken(), Token("TOK_DIR", Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) => @@ -670,7 +673,7 @@ object HiveQl { case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => key -> None }.toMap).getOrElse(Map.empty) - InsertIntoTable(UnresolvedRelation(db, tableName, None), partitionKeys, query) + InsertIntoTable(UnresolvedRelation(db, tableName, None), partitionKeys, query, overwrite) case a: ASTNode => throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala b/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala index 6741b5481c17f..5a1f7b621f7fe 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala @@ -24,8 +24,8 @@ trait SharkStrategies { object DataSinks extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.InsertIntoTable(table: MetastoreRelation, partition, child) => - InsertIntoHiveTable(table, partition, planLater(child))(sharkContext) :: Nil + case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => + InsertIntoHiveTable(table, partition, planLater(child), overwrite)(sharkContext) :: Nil case _ => Nil } } diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index 758b899184da1..ea557c397d27f 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -150,7 +150,10 @@ case class HiveTableScan( } case class InsertIntoHiveTable( - table: MetastoreRelation, partition: Map[String, Option[String]], child: SparkPlan) + table: MetastoreRelation, + partition: Map[String, Option[String]], + child: SparkPlan, + overwrite: Boolean) (@transient sc: SharkContext) extends UnaryNode { @@ -281,13 +284,13 @@ case class InsertIntoHiveTable( new JobConf(sc.hiveconf), sc.hiveconf.getBoolean("hive.exec.compress.output", false)) - // TODO: Correctly set replace and holdDDLTime. + // TODO: Correctly set holdDDLTime. // TODO: Handle loading into partitioned tables. db.loadTable( new Path(fileSinkConf.getDirName), // Have to construct the format of dbname.tablename. s"${table.databaseName}.${table.tableName}", - false, + overwrite, false) // It would be nice to just return the childRdd unchanged so insert operations could be chained, diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala index 258b902826748..e50565d448e36 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala @@ -70,6 +70,16 @@ class HiveQuerySuite extends HiveComparisonTest { |DROP DATABASE IF EXISTS testdb CASCADE """.stripMargin) + createQueryTest("insert into and insert overwrite", + """ + |CREATE TABLE createdtable like src; + |INSERT INTO TABLE createdtable SELECT * FROM src; + |INSERT INTO TABLE createdtable SELECT * FROM src1; + |SELECT * FROM createdtable; + |INSERT OVERWRITE TABLE createdtable SELECT * FROM src WHERE key = 86; + |SELECT * FROM createdtable; + """.stripMargin) + createQueryTest("transform", "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src") From a40d6d628384c172c1d1d7a4bd4011c3cb8f2b6b Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 16 Feb 2014 22:09:23 +0800 Subject: [PATCH 688/778] Loading the static partition specified in a INSERT INTO/OVERWRITE query. --- .../org/apache/spark/sql/shark/HiveQl.scala | 8 +++- .../spark/sql/shark/hiveOperators.scala | 47 ++++++++++++++----- 2 files changed, 42 insertions(+), 13 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala index 29234fd4fa51a..87ae8d1b10d1c 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala @@ -5,6 +5,7 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ +import org.apache.hadoop.hive.ql.plan.PlanUtils import catalyst.analysis._ import catalyst.expressions._ @@ -669,10 +670,15 @@ object HiveQl { val partitionKeys = partitionClause.map(_.getChildren.map { case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => - key -> Some(value) + key -> Some(PlanUtils.stripQuotes(value)) case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => key -> None }.toMap).getOrElse(Map.empty) + if (partitionKeys.values.exists(p => p.isEmpty)) { + throw new NotImplementedError(s"Do not support INSERT INTO/OVERWRITE with" + + s"dynamic partitioning.") + } + InsertIntoTable(UnresolvedRelation(db, tableName, None), partitionKeys, query, overwrite) case a: ASTNode => diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index ea557c397d27f..9fc6a0d11efe9 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -2,6 +2,7 @@ package org.apache.spark.sql package shark import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} +import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Hive} import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} @@ -17,14 +18,14 @@ import org.apache.hadoop.mapred._ import catalyst.expressions._ import catalyst.types.{BooleanType, DataType} -import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{TaskContext, SparkException} -import org.apache.hadoop.io.SequenceFile.CompressionType -import scala.Some import catalyst.expressions.Cast import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution._ +import scala.Some +import scala.collection.immutable.ListMap + /* Implicits */ import scala.collection.JavaConversions._ @@ -277,21 +278,43 @@ case class InsertIntoHiveTable( // ORC stores compression information in table properties. While, there are other formats // (e.g. RCFile) that rely on hadoop configurations to store compression information. + val jobConf = new JobConf(sc.hiveconf) saveAsHiveFile( rdd, outputClass, fileSinkConf, - new JobConf(sc.hiveconf), + jobConf, sc.hiveconf.getBoolean("hive.exec.compress.output", false)) - // TODO: Correctly set holdDDLTime. - // TODO: Handle loading into partitioned tables. - db.loadTable( - new Path(fileSinkConf.getDirName), - // Have to construct the format of dbname.tablename. - s"${table.databaseName}.${table.tableName}", - overwrite, - false) + // TODO: Handle dynamic partitioning. + val outputPath = FileOutputFormat.getOutputPath(jobConf) + // Have to construct the format of dbname.tablename. + val qualifiedTableName = s"${table.databaseName}.${table.tableName}" + if (partition.nonEmpty) { + val partitionSpec = ListMap(partition.toSeq.sortBy(_._1):_*).map { + case (key, Some(value)) => key -> value + case (key, None) => key -> "" // Should not reach here right now. + } + val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols(), partitionSpec) + db.validatePartitionNameCharacters(partVals) + // TODO: Correctly set holdDDLTime, inheritTableSpecs, and isSkewedStoreAsSubdir + // (the last three parameters). + db.loadPartition( + outputPath, + qualifiedTableName, + partitionSpec, + overwrite, + false, + true, + false) + } else { + // TODO: Correctly set holdDDLTime (the last parameter). + db.loadTable( + outputPath, + qualifiedTableName, + overwrite, + false) + } // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which From 334aacee2432fbc6c51644df08f4899d340a2ef4 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 16 Feb 2014 22:11:45 +0800 Subject: [PATCH 689/778] New golden files. --- ...erwrite-0-2e2de36571343c0ebaf02c0c0c61ba8c | 0 ...erwrite-1-548d5a45ef20e066df145944dce3415a | 0 ...erwrite-2-77baf70da8adf8b4b82d076fee937fe1 | 0 ...erwrite-3-b9002c1d71895be765575b62656d1928 | 525 ++++++++++++++++++ ...erwrite-4-3b8e0155870264d89d275a1d5bd345d0 | 0 ...erwrite-5-b9002c1d71895be765575b62656d1928 | 1 + 6 files changed, 526 insertions(+) create mode 100644 shark/src/test/hive/golden/insert into and insert overwrite-0-2e2de36571343c0ebaf02c0c0c61ba8c create mode 100644 shark/src/test/hive/golden/insert into and insert overwrite-1-548d5a45ef20e066df145944dce3415a create mode 100644 shark/src/test/hive/golden/insert into and insert overwrite-2-77baf70da8adf8b4b82d076fee937fe1 create mode 100644 shark/src/test/hive/golden/insert into and insert overwrite-3-b9002c1d71895be765575b62656d1928 create mode 100644 shark/src/test/hive/golden/insert into and insert overwrite-4-3b8e0155870264d89d275a1d5bd345d0 create mode 100644 shark/src/test/hive/golden/insert into and insert overwrite-5-b9002c1d71895be765575b62656d1928 diff --git a/shark/src/test/hive/golden/insert into and insert overwrite-0-2e2de36571343c0ebaf02c0c0c61ba8c b/shark/src/test/hive/golden/insert into and insert overwrite-0-2e2de36571343c0ebaf02c0c0c61ba8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/shark/src/test/hive/golden/insert into and insert overwrite-1-548d5a45ef20e066df145944dce3415a b/shark/src/test/hive/golden/insert into and insert overwrite-1-548d5a45ef20e066df145944dce3415a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/shark/src/test/hive/golden/insert into and insert overwrite-2-77baf70da8adf8b4b82d076fee937fe1 b/shark/src/test/hive/golden/insert into and insert overwrite-2-77baf70da8adf8b4b82d076fee937fe1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/shark/src/test/hive/golden/insert into and insert overwrite-3-b9002c1d71895be765575b62656d1928 b/shark/src/test/hive/golden/insert into and insert overwrite-3-b9002c1d71895be765575b62656d1928 new file mode 100644 index 0000000000000..813f70020af3e --- /dev/null +++ b/shark/src/test/hive/golden/insert into and insert overwrite-3-b9002c1d71895be765575b62656d1928 @@ -0,0 +1,525 @@ +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 +238 val_238 +NULL +311 val_311 +NULL val_27 +NULL val_165 +NULL val_409 +255 val_255 +278 val_278 +98 val_98 +NULL val_484 +NULL val_265 +NULL val_193 +401 val_401 +150 val_150 +273 val_273 +224 +369 +66 val_66 +128 +213 val_213 +146 val_146 +406 val_406 +NULL +NULL +NULL diff --git a/shark/src/test/hive/golden/insert into and insert overwrite-4-3b8e0155870264d89d275a1d5bd345d0 b/shark/src/test/hive/golden/insert into and insert overwrite-4-3b8e0155870264d89d275a1d5bd345d0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/shark/src/test/hive/golden/insert into and insert overwrite-5-b9002c1d71895be765575b62656d1928 b/shark/src/test/hive/golden/insert into and insert overwrite-5-b9002c1d71895be765575b62656d1928 new file mode 100644 index 0000000000000..e87fb81a6f043 --- /dev/null +++ b/shark/src/test/hive/golden/insert into and insert overwrite-5-b9002c1d71895be765575b62656d1928 @@ -0,0 +1 @@ +86 val_86 From d00260be188368ce943f2ffe7d087a7eff2f5f41 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 17 Feb 2014 08:26:19 +0800 Subject: [PATCH 690/778] Strips backticks from partition keys. --- shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala index 87ae8d1b10d1c..b644085ab6774 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala @@ -670,8 +670,8 @@ object HiveQl { val partitionKeys = partitionClause.map(_.getChildren.map { case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => - key -> Some(PlanUtils.stripQuotes(value)) - case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => key -> None + cleanIdentifier(key) -> Some(PlanUtils.stripQuotes(value)) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => cleanIdentifier(key) -> None }.toMap).getOrElse(Map.empty) if (partitionKeys.values.exists(p => p.isEmpty)) { From 555fb1d1e965d19c6e7dc28027361868b3492c0f Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 17 Feb 2014 14:51:16 +0800 Subject: [PATCH 691/778] Correctly set the extension for a text file. --- .../apache/hadoop/mapred/SharkHadoopWriter.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala b/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala index cffb9d2166e36..4da34815d7f36 100644 --- a/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala +++ b/shark/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala @@ -10,7 +10,7 @@ import org.apache.hadoop.io.Writable import org.apache.spark.Logging import org.apache.spark.SerializableWritable -import org.apache.hadoop.hive.ql.exec.FileSinkOperator +import org.apache.hadoop.hive.ql.exec.{Utilities, FileSinkOperator} import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} import org.apache.hadoop.hive.ql.plan.FileSinkDesc @@ -60,8 +60,13 @@ class SharkHadoopWriter( numfmt.setMinimumIntegerDigits(5) numfmt.setGroupingUsed(false) - val outputName = "part-" + numfmt.format(splitID) - val path = FileOutputFormat.getOutputPath(conf.value) + val extension = Utilities.getFileExtension( + conf.value, + fileSinkConf.getCompressed, + getOutputFormat()) + + val outputName = "part-" + numfmt.format(splitID) + extension + val path = FileOutputFormat.getTaskOutputPath(conf.value, outputName) getOutputCommitter().setupTask(getTaskContext()) writer = HiveFileFormatUtils.getHiveRecordWriter( @@ -69,7 +74,7 @@ class SharkHadoopWriter( fileSinkConf.getTableInfo, conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], fileSinkConf, - new Path(path, outputName), + path, null) } From feb022c1e77aac1f6b224cfc56bfd851762a0ca6 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 17 Feb 2014 14:51:55 +0800 Subject: [PATCH 692/778] Partitioning key should be case insensitive. --- .../scala/org/apache/spark/sql/shark/HiveQl.scala | 5 +++-- .../apache/spark/sql/shark/hiveOperators.scala | 15 ++++++++++----- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala index b644085ab6774..ee639aa0225b5 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala @@ -669,9 +669,10 @@ object HiveQl { } val partitionKeys = partitionClause.map(_.getChildren.map { + // Parse partitions. We also make keys case insensitive. case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => - cleanIdentifier(key) -> Some(PlanUtils.stripQuotes(value)) - case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => cleanIdentifier(key) -> None + cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => cleanIdentifier(key.toLowerCase) -> None }.toMap).getOrElse(Map.empty) if (partitionKeys.values.exists(p => p.isEmpty)) { diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index 9fc6a0d11efe9..2b67928de4652 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -290,30 +290,35 @@ case class InsertIntoHiveTable( val outputPath = FileOutputFormat.getOutputPath(jobConf) // Have to construct the format of dbname.tablename. val qualifiedTableName = s"${table.databaseName}.${table.tableName}" + // TODO: Correctly set holdDDLTime (the last parameter). + // In most of the time, we should have holdDDLTime = false. + // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. + val holdDDLTime = false if (partition.nonEmpty) { - val partitionSpec = ListMap(partition.toSeq.sortBy(_._1):_*).map { + val partitionSpec = partition.map { case (key, Some(value)) => key -> value case (key, None) => key -> "" // Should not reach here right now. } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols(), partitionSpec) db.validatePartitionNameCharacters(partVals) - // TODO: Correctly set holdDDLTime, inheritTableSpecs, and isSkewedStoreAsSubdir + // TODO: Correctly set isSkewedStoreAsSubdir (the last parameter). // (the last three parameters). db.loadPartition( outputPath, qualifiedTableName, partitionSpec, overwrite, - false, + holdDDLTime, + // inheritTableSpecs is set to true. It should be set to false for a IMPORT query + // which is currently considered as a Hive natime native command. true, false) } else { - // TODO: Correctly set holdDDLTime (the last parameter). db.loadTable( outputPath, qualifiedTableName, overwrite, - false) + holdDDLTime) } // It would be nice to just return the childRdd unchanged so insert operations could be chained, From a1a47760b718bfecc7e4b1adacb3a179f936825c Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 17 Feb 2014 18:46:13 +0800 Subject: [PATCH 693/778] Update comments. --- .../main/scala/org/apache/spark/sql/shark/hiveOperators.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index 2b67928de4652..56e26fdaaa11f 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -290,7 +290,7 @@ case class InsertIntoHiveTable( val outputPath = FileOutputFormat.getOutputPath(jobConf) // Have to construct the format of dbname.tablename. val qualifiedTableName = s"${table.databaseName}.${table.tableName}" - // TODO: Correctly set holdDDLTime (the last parameter). + // TODO: Correctly set holdDDLTime. // In most of the time, we should have holdDDLTime = false. // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. val holdDDLTime = false @@ -310,7 +310,7 @@ case class InsertIntoHiveTable( overwrite, holdDDLTime, // inheritTableSpecs is set to true. It should be set to false for a IMPORT query - // which is currently considered as a Hive natime native command. + // which is currently considered as a Hive native command. true, false) } else { From 017872cef3d771acab5fb3efc570dc1798e44f6d Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 17 Feb 2014 18:46:31 +0800 Subject: [PATCH 694/778] Remove stats20 from whitelist. --- .../spark/sql/shark/execution/HiveCompatibilitySuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala index 47773532fedbe..6af32f5fd9455 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala @@ -506,7 +506,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "sort_merge_join_desc_6", "sort_merge_join_desc_7", "stats0", - "stats20", "stats_empty_partition", "subq2", "tablename_with_select", From 128a9f8b8082b3ed0659dfe6c41dbd7cbf04ff71 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 18 Feb 2014 12:58:08 +0800 Subject: [PATCH 695/778] Minor changes. --- .../spark/sql/shark/HiveMetastoreCatalog.scala | 2 +- .../scala/org/apache/spark/sql/shark/HiveQl.scala | 2 -- .../org/apache/spark/sql/shark/hiveOperators.scala | 13 +++++++------ .../shark/execution/HiveCompatibilitySuite.scala | 1 + 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala index b4129a45fad34..7c24f7df971d9 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala @@ -78,7 +78,7 @@ class HiveMetastoreCatalog(shark: SharkContext) extends Catalog with Logging { lookupRelation(Some(databaseName), tableName, None).asInstanceOf[BaseRelation], Map.empty, child, - false) + overwrite = false) } } diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala index ee639aa0225b5..923f673e46b2c 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala @@ -482,8 +482,6 @@ object HiveQl { .map(StopAfter(_, withSort)) .getOrElse(withSort) - // There are two tokens for specifying where to sent the result that seem to be used almost - // interchangeably. // TOK_INSERT_INTO means to add files to the table. // TOK_DESTINATION means to overwrite the table. val resultDestination = diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index 56e26fdaaa11f..7dc8a02c5af1c 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -301,18 +301,19 @@ case class InsertIntoHiveTable( } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols(), partitionSpec) db.validatePartitionNameCharacters(partVals) - // TODO: Correctly set isSkewedStoreAsSubdir (the last parameter). - // (the last three parameters). + // inheritTableSpecs is set to true. It should be set to false for a IMPORT query + // which is currently considered as a Hive native command. + val inheritTableSpecs = true + // TODO: Correctly set isSkewedStoreAsSubdir. + val isSkewedStoreAsSubdir = false db.loadPartition( outputPath, qualifiedTableName, partitionSpec, overwrite, holdDDLTime, - // inheritTableSpecs is set to true. It should be set to false for a IMPORT query - // which is currently considered as a Hive native command. - true, - false) + inheritTableSpecs, + isSkewedStoreAsSubdir) } else { db.loadTable( outputPath, diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala index 6af32f5fd9455..9aa1dccae91b7 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala @@ -80,6 +80,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "index_bitmap_auto_partitioned", "ql_rewrite_gbtoidx", "stats1.*", + "stats20", "alter_merge_stats", // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. From f670c8c7adf6a3bc5c1e20850070b15e041f9285 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 18 Feb 2014 17:35:01 +0800 Subject: [PATCH 696/778] Throw a NotImplementedError for not supported clauses in a CTAS query. --- .../org/apache/spark/sql/shark/HiveQl.scala | 37 +++++++++++++++++-- 1 file changed, 34 insertions(+), 3 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala index 923f673e46b2c..d0508fca06b01 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala @@ -335,9 +335,40 @@ object HiveQl { case Token("TOK_CREATETABLE", children) if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty => - // TODO: Parse other properties. - val (Some(tableNameParts) :: _ /* likeTable */ :: Some(query) :: Nil) = - getClauses(Seq("TOK_TABNAME", "TOK_LIKETABLE", "TOK_QUERY"), children) + // TODO: Parse other clauses. + // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL + val ( + Some(tableNameParts) :: + _ /* likeTable */ :: + Some(query) +: + notImplemented) = + getClauses( + Seq( + "TOK_TABNAME", + "TOK_LIKETABLE", + "TOK_QUERY", + "TOK_IFNOTEXISTS", + "TOK_TABLECOMMENT", + "TOK_TABCOLLIST", + "TOK_TABLEPARTCOLS", // Partitioned by + "TOK_TABLEBUCKETS", // Clustered by + "TOK_TABLESKEWED", // Skewed by + "TOK_TABLEROWFORMAT", + "TOK_TABLESERIALIZER", + "TOK_FILEFORMAT_GENERIC", // For file formats not natively supported by Hive. + "TOK_TBLSEQUENCEFILE", // Stored as SequenceFile + "TOK_TBLTEXTFILE", // Stored as TextFile + "TOK_TBLRCFILE", // Stored as RCFile + "TOK_TBLORCFILE", // Stored as ORC File + "TOK_TABLEFILEFORMAT", // User-provided InputFormat and OutputFormat + "TOK_STORAGEHANDLER", // Storage handler + "TOK_TABLELOCATION", + "TOK_TABLEPROPERTIES"), + children) + if (notImplemented.exists(token => !token.isEmpty)) { + throw new NotImplementedError( + s"Unhandled clauses: ${notImplemented.flatten.map(dumpTree(_)).mkString("\n")}") + } val (db, tableName) = tableNameParts.getChildren.map{ case Token(part, Nil) => cleanIdentifier(part)} match { From 54f165b5f8814b9a9572f315b17505ef896b723a Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Tue, 18 Feb 2014 20:19:26 +0800 Subject: [PATCH 697/778] Fixed spelling typo in two golden answer file names --- ... => trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d} | 0 ... trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename shark/src/test/hive/golden/{trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d => trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d} (100%) rename shark/src/test/hive/golden/{trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 => trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3} (100%) diff --git a/shark/src/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d b/shark/src/test/hive/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d similarity index 100% rename from shark/src/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d rename to shark/src/test/hive/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d diff --git a/shark/src/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 b/shark/src/test/hive/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 similarity index 100% rename from shark/src/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 rename to shark/src/test/hive/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 From cf4db596d1ef8edcaa4f5e42648ddc57e4dc38e6 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Wed, 19 Feb 2014 00:32:20 +0800 Subject: [PATCH 698/778] Added golden answers for PruningSuite --- ...ry test-0-cb0737d268260da971487a8ac5e3d8be | 3 + ...ry test-0-c89dcddb8539bae211ea28cd1e695e35 | 3 + ...ry test-0-9589ac54d6de575083765ee64051ebaa | 3 + ...ry test-0-d6d2726928df920ab99078015b8ff494 | 3 + ...ery test-0-cfe78c68302bd143d989c383c3b0477 | 3 + ...ry test-0-358525951e893ed9686c46bf1f7447ee | 3 + ...ry test-0-d9d50146537344742300ce7b52c18c91 | 3 + ...ry test-0-c25cdafe7d85c42a9d9cd559d88a6049 | 3 + ...ry test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 | 0 ...ry test-0-3adc3a7f76b2abd059904ba81a595db3 | 500 +++++++++ ...ry test-0-3893ec7a86062215d9f817f1495a69bb | 500 +++++++++ ...ry test-0-75148567b91227053f728d72f7dc7c10 | 1000 +++++++++++++++++ ...ry test-0-4e4d2651e1e6e3c224242078d0201190 | 1000 +++++++++++++++++ ...ry test-0-b6e59464383591f02408f8765ac6a5d5 | 1000 +++++++++++++++++ ...ry test-0-681332efaff7b12e3ca12d9f021cd344 | 500 +++++++++ 15 files changed, 4524 insertions(+) create mode 100644 shark/src/test/hive/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be create mode 100644 shark/src/test/hive/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 create mode 100644 shark/src/test/hive/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa create mode 100644 shark/src/test/hive/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 create mode 100644 shark/src/test/hive/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 create mode 100644 shark/src/test/hive/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee create mode 100644 shark/src/test/hive/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 create mode 100644 shark/src/test/hive/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 create mode 100644 shark/src/test/hive/golden/Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 create mode 100644 shark/src/test/hive/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 create mode 100644 shark/src/test/hive/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb create mode 100644 shark/src/test/hive/golden/Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 create mode 100644 shark/src/test/hive/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 create mode 100644 shark/src/test/hive/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 create mode 100644 shark/src/test/hive/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 diff --git a/shark/src/test/hive/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be b/shark/src/test/hive/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be new file mode 100644 index 0000000000000..913a609968137 --- /dev/null +++ b/shark/src/test/hive/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be @@ -0,0 +1,3 @@ +86 +27 +98 diff --git a/shark/src/test/hive/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 b/shark/src/test/hive/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 new file mode 100644 index 0000000000000..9a276bc794c0d --- /dev/null +++ b/shark/src/test/hive/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 @@ -0,0 +1,3 @@ +476 +172 +622 diff --git a/shark/src/test/hive/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa b/shark/src/test/hive/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa new file mode 100644 index 0000000000000..1c8e28554a60e --- /dev/null +++ b/shark/src/test/hive/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa @@ -0,0 +1,3 @@ +238 +86 +311 diff --git a/shark/src/test/hive/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 b/shark/src/test/hive/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 new file mode 100644 index 0000000000000..913a609968137 --- /dev/null +++ b/shark/src/test/hive/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 @@ -0,0 +1,3 @@ +86 +27 +98 diff --git a/shark/src/test/hive/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 b/shark/src/test/hive/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 new file mode 100644 index 0000000000000..1c8e28554a60e --- /dev/null +++ b/shark/src/test/hive/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 @@ -0,0 +1,3 @@ +238 +86 +311 diff --git a/shark/src/test/hive/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee b/shark/src/test/hive/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee new file mode 100644 index 0000000000000..1c8e28554a60e --- /dev/null +++ b/shark/src/test/hive/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee @@ -0,0 +1,3 @@ +238 +86 +311 diff --git a/shark/src/test/hive/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 b/shark/src/test/hive/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 new file mode 100644 index 0000000000000..1c8e28554a60e --- /dev/null +++ b/shark/src/test/hive/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 @@ -0,0 +1,3 @@ +238 +86 +311 diff --git a/shark/src/test/hive/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 b/shark/src/test/hive/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 new file mode 100644 index 0000000000000..1c8e28554a60e --- /dev/null +++ b/shark/src/test/hive/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 @@ -0,0 +1,3 @@ +238 +86 +311 diff --git a/shark/src/test/hive/golden/Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 b/shark/src/test/hive/golden/Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/shark/src/test/hive/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 b/shark/src/test/hive/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 new file mode 100644 index 0000000000000..0fe6b905e7781 --- /dev/null +++ b/shark/src/test/hive/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 @@ -0,0 +1,500 @@ +val_238 11 +val_86 11 +val_311 11 +val_27 11 +val_165 11 +val_409 11 +val_255 11 +val_278 11 +val_98 11 +val_484 11 +val_265 11 +val_193 11 +val_401 11 +val_150 11 +val_273 11 +val_224 11 +val_369 11 +val_66 11 +val_128 11 +val_213 11 +val_146 11 +val_406 11 +val_429 11 +val_374 11 +val_152 11 +val_469 11 +val_145 11 +val_495 11 +val_37 11 +val_327 11 +val_281 11 +val_277 11 +val_209 11 +val_15 11 +val_82 11 +val_403 11 +val_166 11 +val_417 11 +val_430 11 +val_252 11 +val_292 11 +val_219 11 +val_287 11 +val_153 11 +val_193 11 +val_338 11 +val_446 11 +val_459 11 +val_394 11 +val_237 11 +val_482 11 +val_174 11 +val_413 11 +val_494 11 +val_207 11 +val_199 11 +val_466 11 +val_208 11 +val_174 11 +val_399 11 +val_396 11 +val_247 11 +val_417 11 +val_489 11 +val_162 11 +val_377 11 +val_397 11 +val_309 11 +val_365 11 +val_266 11 +val_439 11 +val_342 11 +val_367 11 +val_325 11 +val_167 11 +val_195 11 +val_475 11 +val_17 11 +val_113 11 +val_155 11 +val_203 11 +val_339 11 +val_0 11 +val_455 11 +val_128 11 +val_311 11 +val_316 11 +val_57 11 +val_302 11 +val_205 11 +val_149 11 +val_438 11 +val_345 11 +val_129 11 +val_170 11 +val_20 11 +val_489 11 +val_157 11 +val_378 11 +val_221 11 +val_92 11 +val_111 11 +val_47 11 +val_72 11 +val_4 11 +val_280 11 +val_35 11 +val_427 11 +val_277 11 +val_208 11 +val_356 11 +val_399 11 +val_169 11 +val_382 11 +val_498 11 +val_125 11 +val_386 11 +val_437 11 +val_469 11 +val_192 11 +val_286 11 +val_187 11 +val_176 11 +val_54 11 +val_459 11 +val_51 11 +val_138 11 +val_103 11 +val_239 11 +val_213 11 +val_216 11 +val_430 11 +val_278 11 +val_176 11 +val_289 11 +val_221 11 +val_65 11 +val_318 11 +val_332 11 +val_311 11 +val_275 11 +val_137 11 +val_241 11 +val_83 11 +val_333 11 +val_180 11 +val_284 11 +val_12 11 +val_230 11 +val_181 11 +val_67 11 +val_260 11 +val_404 11 +val_384 11 +val_489 11 +val_353 11 +val_373 11 +val_272 11 +val_138 11 +val_217 11 +val_84 11 +val_348 11 +val_466 11 +val_58 11 +val_8 11 +val_411 11 +val_230 11 +val_208 11 +val_348 11 +val_24 11 +val_463 11 +val_431 11 +val_179 11 +val_172 11 +val_42 11 +val_129 11 +val_158 11 +val_119 11 +val_496 11 +val_0 11 +val_322 11 +val_197 11 +val_468 11 +val_393 11 +val_454 11 +val_100 11 +val_298 11 +val_199 11 +val_191 11 +val_418 11 +val_96 11 +val_26 11 +val_165 11 +val_327 11 +val_230 11 +val_205 11 +val_120 11 +val_131 11 +val_51 11 +val_404 11 +val_43 11 +val_436 11 +val_156 11 +val_469 11 +val_468 11 +val_308 11 +val_95 11 +val_196 11 +val_288 11 +val_481 11 +val_457 11 +val_98 11 +val_282 11 +val_197 11 +val_187 11 +val_318 11 +val_318 11 +val_409 11 +val_470 11 +val_137 11 +val_369 11 +val_316 11 +val_169 11 +val_413 11 +val_85 11 +val_77 11 +val_0 11 +val_490 11 +val_87 11 +val_364 11 +val_179 11 +val_118 11 +val_134 11 +val_395 11 +val_282 11 +val_138 11 +val_238 11 +val_419 11 +val_15 11 +val_118 11 +val_72 11 +val_90 11 +val_307 11 +val_19 11 +val_435 11 +val_10 11 +val_277 11 +val_273 11 +val_306 11 +val_224 11 +val_309 11 +val_389 11 +val_327 11 +val_242 11 +val_369 11 +val_392 11 +val_272 11 +val_331 11 +val_401 11 +val_242 11 +val_452 11 +val_177 11 +val_226 11 +val_5 11 +val_497 11 +val_402 11 +val_396 11 +val_317 11 +val_395 11 +val_58 11 +val_35 11 +val_336 11 +val_95 11 +val_11 11 +val_168 11 +val_34 11 +val_229 11 +val_233 11 +val_143 11 +val_472 11 +val_322 11 +val_498 11 +val_160 11 +val_195 11 +val_42 11 +val_321 11 +val_430 11 +val_119 11 +val_489 11 +val_458 11 +val_78 11 +val_76 11 +val_41 11 +val_223 11 +val_492 11 +val_149 11 +val_449 11 +val_218 11 +val_228 11 +val_138 11 +val_453 11 +val_30 11 +val_209 11 +val_64 11 +val_468 11 +val_76 11 +val_74 11 +val_342 11 +val_69 11 +val_230 11 +val_33 11 +val_368 11 +val_103 11 +val_296 11 +val_113 11 +val_216 11 +val_367 11 +val_344 11 +val_167 11 +val_274 11 +val_219 11 +val_239 11 +val_485 11 +val_116 11 +val_223 11 +val_256 11 +val_263 11 +val_70 11 +val_487 11 +val_480 11 +val_401 11 +val_288 11 +val_191 11 +val_5 11 +val_244 11 +val_438 11 +val_128 11 +val_467 11 +val_432 11 +val_202 11 +val_316 11 +val_229 11 +val_469 11 +val_463 11 +val_280 11 +val_2 11 +val_35 11 +val_283 11 +val_331 11 +val_235 11 +val_80 11 +val_44 11 +val_193 11 +val_321 11 +val_335 11 +val_104 11 +val_466 11 +val_366 11 +val_175 11 +val_403 11 +val_483 11 +val_53 11 +val_105 11 +val_257 11 +val_406 11 +val_409 11 +val_190 11 +val_406 11 +val_401 11 +val_114 11 +val_258 11 +val_90 11 +val_203 11 +val_262 11 +val_348 11 +val_424 11 +val_12 11 +val_396 11 +val_201 11 +val_217 11 +val_164 11 +val_431 11 +val_454 11 +val_478 11 +val_298 11 +val_125 11 +val_431 11 +val_164 11 +val_424 11 +val_187 11 +val_382 11 +val_5 11 +val_70 11 +val_397 11 +val_480 11 +val_291 11 +val_24 11 +val_351 11 +val_255 11 +val_104 11 +val_70 11 +val_163 11 +val_438 11 +val_119 11 +val_414 11 +val_200 11 +val_491 11 +val_237 11 +val_439 11 +val_360 11 +val_248 11 +val_479 11 +val_305 11 +val_417 11 +val_199 11 +val_444 11 +val_120 11 +val_429 11 +val_169 11 +val_443 11 +val_323 11 +val_325 11 +val_277 11 +val_230 11 +val_478 11 +val_178 11 +val_468 11 +val_310 11 +val_317 11 +val_333 11 +val_493 11 +val_460 11 +val_207 11 +val_249 11 +val_265 11 +val_480 11 +val_83 11 +val_136 11 +val_353 11 +val_172 11 +val_214 11 +val_462 11 +val_233 11 +val_406 11 +val_133 11 +val_175 11 +val_189 11 +val_454 11 +val_375 11 +val_401 11 +val_421 11 +val_407 11 +val_384 11 +val_256 11 +val_26 11 +val_134 11 +val_67 11 +val_384 11 +val_379 11 +val_18 11 +val_462 11 +val_492 11 +val_100 11 +val_298 11 +val_9 11 +val_341 11 +val_498 11 +val_146 11 +val_458 11 +val_362 11 +val_186 11 +val_285 11 +val_348 11 +val_167 11 +val_18 11 +val_273 11 +val_183 11 +val_281 11 +val_344 11 +val_97 11 +val_469 11 +val_315 11 +val_84 11 +val_28 11 +val_37 11 +val_448 11 +val_152 11 +val_348 11 +val_307 11 +val_194 11 +val_414 11 +val_477 11 +val_222 11 +val_126 11 +val_90 11 +val_169 11 +val_403 11 +val_400 11 +val_200 11 +val_97 11 diff --git a/shark/src/test/hive/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb b/shark/src/test/hive/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb new file mode 100644 index 0000000000000..444039e75fbac --- /dev/null +++ b/shark/src/test/hive/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb @@ -0,0 +1,500 @@ +476 +172 +622 +54 +330 +818 +510 +556 +196 +968 +530 +386 +802 +300 +546 +448 +738 +132 +256 +426 +292 +812 +858 +748 +304 +938 +290 +990 +74 +654 +562 +554 +418 +30 +164 +806 +332 +834 +860 +504 +584 +438 +574 +306 +386 +676 +892 +918 +788 +474 +964 +348 +826 +988 +414 +398 +932 +416 +348 +798 +792 +494 +834 +978 +324 +754 +794 +618 +730 +532 +878 +684 +734 +650 +334 +390 +950 +34 +226 +310 +406 +678 +0 +910 +256 +622 +632 +114 +604 +410 +298 +876 +690 +258 +340 +40 +978 +314 +756 +442 +184 +222 +94 +144 +8 +560 +70 +854 +554 +416 +712 +798 +338 +764 +996 +250 +772 +874 +938 +384 +572 +374 +352 +108 +918 +102 +276 +206 +478 +426 +432 +860 +556 +352 +578 +442 +130 +636 +664 +622 +550 +274 +482 +166 +666 +360 +568 +24 +460 +362 +134 +520 +808 +768 +978 +706 +746 +544 +276 +434 +168 +696 +932 +116 +16 +822 +460 +416 +696 +48 +926 +862 +358 +344 +84 +258 +316 +238 +992 +0 +644 +394 +936 +786 +908 +200 +596 +398 +382 +836 +192 +52 +330 +654 +460 +410 +240 +262 +102 +808 +86 +872 +312 +938 +936 +616 +190 +392 +576 +962 +914 +196 +564 +394 +374 +636 +636 +818 +940 +274 +738 +632 +338 +826 +170 +154 +0 +980 +174 +728 +358 +236 +268 +790 +564 +276 +476 +838 +30 +236 +144 +180 +614 +38 +870 +20 +554 +546 +612 +448 +618 +778 +654 +484 +738 +784 +544 +662 +802 +484 +904 +354 +452 +10 +994 +804 +792 +634 +790 +116 +70 +672 +190 +22 +336 +68 +458 +466 +286 +944 +644 +996 +320 +390 +84 +642 +860 +238 +978 +916 +156 +152 +82 +446 +984 +298 +898 +436 +456 +276 +906 +60 +418 +128 +936 +152 +148 +684 +138 +460 +66 +736 +206 +592 +226 +432 +734 +688 +334 +548 +438 +478 +970 +232 +446 +512 +526 +140 +974 +960 +802 +576 +382 +10 +488 +876 +256 +934 +864 +404 +632 +458 +938 +926 +560 +4 +70 +566 +662 +470 +160 +88 +386 +642 +670 +208 +932 +732 +350 +806 +966 +106 +210 +514 +812 +818 +380 +812 +802 +228 +516 +180 +406 +524 +696 +848 +24 +792 +402 +434 +328 +862 +908 +956 +596 +250 +862 +328 +848 +374 +764 +10 +140 +794 +960 +582 +48 +702 +510 +208 +140 +326 +876 +238 +828 +400 +982 +474 +878 +720 +496 +958 +610 +834 +398 +888 +240 +858 +338 +886 +646 +650 +554 +460 +956 +356 +936 +620 +634 +666 +986 +920 +414 +498 +530 +960 +166 +272 +706 +344 +428 +924 +466 +812 +266 +350 +378 +908 +750 +802 +842 +814 +768 +512 +52 +268 +134 +768 +758 +36 +924 +984 +200 +596 +18 +682 +996 +292 +916 +724 +372 +570 +696 +334 +36 +546 +366 +562 +688 +194 +938 +630 +168 +56 +74 +896 +304 +696 +614 +388 +828 +954 +444 +252 +180 +338 +806 +800 +400 +194 diff --git a/shark/src/test/hive/golden/Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 b/shark/src/test/hive/golden/Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 new file mode 100644 index 0000000000000..d83d8b62b7095 --- /dev/null +++ b/shark/src/test/hive/golden/Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 @@ -0,0 +1,1000 @@ +val_238 11 +val_86 11 +val_311 11 +val_27 11 +val_165 11 +val_409 11 +val_255 11 +val_278 11 +val_98 11 +val_484 11 +val_265 11 +val_193 11 +val_401 11 +val_150 11 +val_273 11 +val_224 11 +val_369 11 +val_66 11 +val_128 11 +val_213 11 +val_146 11 +val_406 11 +val_429 11 +val_374 11 +val_152 11 +val_469 11 +val_145 11 +val_495 11 +val_37 11 +val_327 11 +val_281 11 +val_277 11 +val_209 11 +val_15 11 +val_82 11 +val_403 11 +val_166 11 +val_417 11 +val_430 11 +val_252 11 +val_292 11 +val_219 11 +val_287 11 +val_153 11 +val_193 11 +val_338 11 +val_446 11 +val_459 11 +val_394 11 +val_237 11 +val_482 11 +val_174 11 +val_413 11 +val_494 11 +val_207 11 +val_199 11 +val_466 11 +val_208 11 +val_174 11 +val_399 11 +val_396 11 +val_247 11 +val_417 11 +val_489 11 +val_162 11 +val_377 11 +val_397 11 +val_309 11 +val_365 11 +val_266 11 +val_439 11 +val_342 11 +val_367 11 +val_325 11 +val_167 11 +val_195 11 +val_475 11 +val_17 11 +val_113 11 +val_155 11 +val_203 11 +val_339 11 +val_0 11 +val_455 11 +val_128 11 +val_311 11 +val_316 11 +val_57 11 +val_302 11 +val_205 11 +val_149 11 +val_438 11 +val_345 11 +val_129 11 +val_170 11 +val_20 11 +val_489 11 +val_157 11 +val_378 11 +val_221 11 +val_92 11 +val_111 11 +val_47 11 +val_72 11 +val_4 11 +val_280 11 +val_35 11 +val_427 11 +val_277 11 +val_208 11 +val_356 11 +val_399 11 +val_169 11 +val_382 11 +val_498 11 +val_125 11 +val_386 11 +val_437 11 +val_469 11 +val_192 11 +val_286 11 +val_187 11 +val_176 11 +val_54 11 +val_459 11 +val_51 11 +val_138 11 +val_103 11 +val_239 11 +val_213 11 +val_216 11 +val_430 11 +val_278 11 +val_176 11 +val_289 11 +val_221 11 +val_65 11 +val_318 11 +val_332 11 +val_311 11 +val_275 11 +val_137 11 +val_241 11 +val_83 11 +val_333 11 +val_180 11 +val_284 11 +val_12 11 +val_230 11 +val_181 11 +val_67 11 +val_260 11 +val_404 11 +val_384 11 +val_489 11 +val_353 11 +val_373 11 +val_272 11 +val_138 11 +val_217 11 +val_84 11 +val_348 11 +val_466 11 +val_58 11 +val_8 11 +val_411 11 +val_230 11 +val_208 11 +val_348 11 +val_24 11 +val_463 11 +val_431 11 +val_179 11 +val_172 11 +val_42 11 +val_129 11 +val_158 11 +val_119 11 +val_496 11 +val_0 11 +val_322 11 +val_197 11 +val_468 11 +val_393 11 +val_454 11 +val_100 11 +val_298 11 +val_199 11 +val_191 11 +val_418 11 +val_96 11 +val_26 11 +val_165 11 +val_327 11 +val_230 11 +val_205 11 +val_120 11 +val_131 11 +val_51 11 +val_404 11 +val_43 11 +val_436 11 +val_156 11 +val_469 11 +val_468 11 +val_308 11 +val_95 11 +val_196 11 +val_288 11 +val_481 11 +val_457 11 +val_98 11 +val_282 11 +val_197 11 +val_187 11 +val_318 11 +val_318 11 +val_409 11 +val_470 11 +val_137 11 +val_369 11 +val_316 11 +val_169 11 +val_413 11 +val_85 11 +val_77 11 +val_0 11 +val_490 11 +val_87 11 +val_364 11 +val_179 11 +val_118 11 +val_134 11 +val_395 11 +val_282 11 +val_138 11 +val_238 11 +val_419 11 +val_15 11 +val_118 11 +val_72 11 +val_90 11 +val_307 11 +val_19 11 +val_435 11 +val_10 11 +val_277 11 +val_273 11 +val_306 11 +val_224 11 +val_309 11 +val_389 11 +val_327 11 +val_242 11 +val_369 11 +val_392 11 +val_272 11 +val_331 11 +val_401 11 +val_242 11 +val_452 11 +val_177 11 +val_226 11 +val_5 11 +val_497 11 +val_402 11 +val_396 11 +val_317 11 +val_395 11 +val_58 11 +val_35 11 +val_336 11 +val_95 11 +val_11 11 +val_168 11 +val_34 11 +val_229 11 +val_233 11 +val_143 11 +val_472 11 +val_322 11 +val_498 11 +val_160 11 +val_195 11 +val_42 11 +val_321 11 +val_430 11 +val_119 11 +val_489 11 +val_458 11 +val_78 11 +val_76 11 +val_41 11 +val_223 11 +val_492 11 +val_149 11 +val_449 11 +val_218 11 +val_228 11 +val_138 11 +val_453 11 +val_30 11 +val_209 11 +val_64 11 +val_468 11 +val_76 11 +val_74 11 +val_342 11 +val_69 11 +val_230 11 +val_33 11 +val_368 11 +val_103 11 +val_296 11 +val_113 11 +val_216 11 +val_367 11 +val_344 11 +val_167 11 +val_274 11 +val_219 11 +val_239 11 +val_485 11 +val_116 11 +val_223 11 +val_256 11 +val_263 11 +val_70 11 +val_487 11 +val_480 11 +val_401 11 +val_288 11 +val_191 11 +val_5 11 +val_244 11 +val_438 11 +val_128 11 +val_467 11 +val_432 11 +val_202 11 +val_316 11 +val_229 11 +val_469 11 +val_463 11 +val_280 11 +val_2 11 +val_35 11 +val_283 11 +val_331 11 +val_235 11 +val_80 11 +val_44 11 +val_193 11 +val_321 11 +val_335 11 +val_104 11 +val_466 11 +val_366 11 +val_175 11 +val_403 11 +val_483 11 +val_53 11 +val_105 11 +val_257 11 +val_406 11 +val_409 11 +val_190 11 +val_406 11 +val_401 11 +val_114 11 +val_258 11 +val_90 11 +val_203 11 +val_262 11 +val_348 11 +val_424 11 +val_12 11 +val_396 11 +val_201 11 +val_217 11 +val_164 11 +val_431 11 +val_454 11 +val_478 11 +val_298 11 +val_125 11 +val_431 11 +val_164 11 +val_424 11 +val_187 11 +val_382 11 +val_5 11 +val_70 11 +val_397 11 +val_480 11 +val_291 11 +val_24 11 +val_351 11 +val_255 11 +val_104 11 +val_70 11 +val_163 11 +val_438 11 +val_119 11 +val_414 11 +val_200 11 +val_491 11 +val_237 11 +val_439 11 +val_360 11 +val_248 11 +val_479 11 +val_305 11 +val_417 11 +val_199 11 +val_444 11 +val_120 11 +val_429 11 +val_169 11 +val_443 11 +val_323 11 +val_325 11 +val_277 11 +val_230 11 +val_478 11 +val_178 11 +val_468 11 +val_310 11 +val_317 11 +val_333 11 +val_493 11 +val_460 11 +val_207 11 +val_249 11 +val_265 11 +val_480 11 +val_83 11 +val_136 11 +val_353 11 +val_172 11 +val_214 11 +val_462 11 +val_233 11 +val_406 11 +val_133 11 +val_175 11 +val_189 11 +val_454 11 +val_375 11 +val_401 11 +val_421 11 +val_407 11 +val_384 11 +val_256 11 +val_26 11 +val_134 11 +val_67 11 +val_384 11 +val_379 11 +val_18 11 +val_462 11 +val_492 11 +val_100 11 +val_298 11 +val_9 11 +val_341 11 +val_498 11 +val_146 11 +val_458 11 +val_362 11 +val_186 11 +val_285 11 +val_348 11 +val_167 11 +val_18 11 +val_273 11 +val_183 11 +val_281 11 +val_344 11 +val_97 11 +val_469 11 +val_315 11 +val_84 11 +val_28 11 +val_37 11 +val_448 11 +val_152 11 +val_348 11 +val_307 11 +val_194 11 +val_414 11 +val_477 11 +val_222 11 +val_126 11 +val_90 11 +val_169 11 +val_403 11 +val_400 11 +val_200 11 +val_97 11 +val_238 11 +val_86 11 +val_311 11 +val_27 11 +val_165 11 +val_409 11 +val_255 11 +val_278 11 +val_98 11 +val_484 11 +val_265 11 +val_193 11 +val_401 11 +val_150 11 +val_273 11 +val_224 11 +val_369 11 +val_66 11 +val_128 11 +val_213 11 +val_146 11 +val_406 11 +val_429 11 +val_374 11 +val_152 11 +val_469 11 +val_145 11 +val_495 11 +val_37 11 +val_327 11 +val_281 11 +val_277 11 +val_209 11 +val_15 11 +val_82 11 +val_403 11 +val_166 11 +val_417 11 +val_430 11 +val_252 11 +val_292 11 +val_219 11 +val_287 11 +val_153 11 +val_193 11 +val_338 11 +val_446 11 +val_459 11 +val_394 11 +val_237 11 +val_482 11 +val_174 11 +val_413 11 +val_494 11 +val_207 11 +val_199 11 +val_466 11 +val_208 11 +val_174 11 +val_399 11 +val_396 11 +val_247 11 +val_417 11 +val_489 11 +val_162 11 +val_377 11 +val_397 11 +val_309 11 +val_365 11 +val_266 11 +val_439 11 +val_342 11 +val_367 11 +val_325 11 +val_167 11 +val_195 11 +val_475 11 +val_17 11 +val_113 11 +val_155 11 +val_203 11 +val_339 11 +val_0 11 +val_455 11 +val_128 11 +val_311 11 +val_316 11 +val_57 11 +val_302 11 +val_205 11 +val_149 11 +val_438 11 +val_345 11 +val_129 11 +val_170 11 +val_20 11 +val_489 11 +val_157 11 +val_378 11 +val_221 11 +val_92 11 +val_111 11 +val_47 11 +val_72 11 +val_4 11 +val_280 11 +val_35 11 +val_427 11 +val_277 11 +val_208 11 +val_356 11 +val_399 11 +val_169 11 +val_382 11 +val_498 11 +val_125 11 +val_386 11 +val_437 11 +val_469 11 +val_192 11 +val_286 11 +val_187 11 +val_176 11 +val_54 11 +val_459 11 +val_51 11 +val_138 11 +val_103 11 +val_239 11 +val_213 11 +val_216 11 +val_430 11 +val_278 11 +val_176 11 +val_289 11 +val_221 11 +val_65 11 +val_318 11 +val_332 11 +val_311 11 +val_275 11 +val_137 11 +val_241 11 +val_83 11 +val_333 11 +val_180 11 +val_284 11 +val_12 11 +val_230 11 +val_181 11 +val_67 11 +val_260 11 +val_404 11 +val_384 11 +val_489 11 +val_353 11 +val_373 11 +val_272 11 +val_138 11 +val_217 11 +val_84 11 +val_348 11 +val_466 11 +val_58 11 +val_8 11 +val_411 11 +val_230 11 +val_208 11 +val_348 11 +val_24 11 +val_463 11 +val_431 11 +val_179 11 +val_172 11 +val_42 11 +val_129 11 +val_158 11 +val_119 11 +val_496 11 +val_0 11 +val_322 11 +val_197 11 +val_468 11 +val_393 11 +val_454 11 +val_100 11 +val_298 11 +val_199 11 +val_191 11 +val_418 11 +val_96 11 +val_26 11 +val_165 11 +val_327 11 +val_230 11 +val_205 11 +val_120 11 +val_131 11 +val_51 11 +val_404 11 +val_43 11 +val_436 11 +val_156 11 +val_469 11 +val_468 11 +val_308 11 +val_95 11 +val_196 11 +val_288 11 +val_481 11 +val_457 11 +val_98 11 +val_282 11 +val_197 11 +val_187 11 +val_318 11 +val_318 11 +val_409 11 +val_470 11 +val_137 11 +val_369 11 +val_316 11 +val_169 11 +val_413 11 +val_85 11 +val_77 11 +val_0 11 +val_490 11 +val_87 11 +val_364 11 +val_179 11 +val_118 11 +val_134 11 +val_395 11 +val_282 11 +val_138 11 +val_238 11 +val_419 11 +val_15 11 +val_118 11 +val_72 11 +val_90 11 +val_307 11 +val_19 11 +val_435 11 +val_10 11 +val_277 11 +val_273 11 +val_306 11 +val_224 11 +val_309 11 +val_389 11 +val_327 11 +val_242 11 +val_369 11 +val_392 11 +val_272 11 +val_331 11 +val_401 11 +val_242 11 +val_452 11 +val_177 11 +val_226 11 +val_5 11 +val_497 11 +val_402 11 +val_396 11 +val_317 11 +val_395 11 +val_58 11 +val_35 11 +val_336 11 +val_95 11 +val_11 11 +val_168 11 +val_34 11 +val_229 11 +val_233 11 +val_143 11 +val_472 11 +val_322 11 +val_498 11 +val_160 11 +val_195 11 +val_42 11 +val_321 11 +val_430 11 +val_119 11 +val_489 11 +val_458 11 +val_78 11 +val_76 11 +val_41 11 +val_223 11 +val_492 11 +val_149 11 +val_449 11 +val_218 11 +val_228 11 +val_138 11 +val_453 11 +val_30 11 +val_209 11 +val_64 11 +val_468 11 +val_76 11 +val_74 11 +val_342 11 +val_69 11 +val_230 11 +val_33 11 +val_368 11 +val_103 11 +val_296 11 +val_113 11 +val_216 11 +val_367 11 +val_344 11 +val_167 11 +val_274 11 +val_219 11 +val_239 11 +val_485 11 +val_116 11 +val_223 11 +val_256 11 +val_263 11 +val_70 11 +val_487 11 +val_480 11 +val_401 11 +val_288 11 +val_191 11 +val_5 11 +val_244 11 +val_438 11 +val_128 11 +val_467 11 +val_432 11 +val_202 11 +val_316 11 +val_229 11 +val_469 11 +val_463 11 +val_280 11 +val_2 11 +val_35 11 +val_283 11 +val_331 11 +val_235 11 +val_80 11 +val_44 11 +val_193 11 +val_321 11 +val_335 11 +val_104 11 +val_466 11 +val_366 11 +val_175 11 +val_403 11 +val_483 11 +val_53 11 +val_105 11 +val_257 11 +val_406 11 +val_409 11 +val_190 11 +val_406 11 +val_401 11 +val_114 11 +val_258 11 +val_90 11 +val_203 11 +val_262 11 +val_348 11 +val_424 11 +val_12 11 +val_396 11 +val_201 11 +val_217 11 +val_164 11 +val_431 11 +val_454 11 +val_478 11 +val_298 11 +val_125 11 +val_431 11 +val_164 11 +val_424 11 +val_187 11 +val_382 11 +val_5 11 +val_70 11 +val_397 11 +val_480 11 +val_291 11 +val_24 11 +val_351 11 +val_255 11 +val_104 11 +val_70 11 +val_163 11 +val_438 11 +val_119 11 +val_414 11 +val_200 11 +val_491 11 +val_237 11 +val_439 11 +val_360 11 +val_248 11 +val_479 11 +val_305 11 +val_417 11 +val_199 11 +val_444 11 +val_120 11 +val_429 11 +val_169 11 +val_443 11 +val_323 11 +val_325 11 +val_277 11 +val_230 11 +val_478 11 +val_178 11 +val_468 11 +val_310 11 +val_317 11 +val_333 11 +val_493 11 +val_460 11 +val_207 11 +val_249 11 +val_265 11 +val_480 11 +val_83 11 +val_136 11 +val_353 11 +val_172 11 +val_214 11 +val_462 11 +val_233 11 +val_406 11 +val_133 11 +val_175 11 +val_189 11 +val_454 11 +val_375 11 +val_401 11 +val_421 11 +val_407 11 +val_384 11 +val_256 11 +val_26 11 +val_134 11 +val_67 11 +val_384 11 +val_379 11 +val_18 11 +val_462 11 +val_492 11 +val_100 11 +val_298 11 +val_9 11 +val_341 11 +val_498 11 +val_146 11 +val_458 11 +val_362 11 +val_186 11 +val_285 11 +val_348 11 +val_167 11 +val_18 11 +val_273 11 +val_183 11 +val_281 11 +val_344 11 +val_97 11 +val_469 11 +val_315 11 +val_84 11 +val_28 11 +val_37 11 +val_448 11 +val_152 11 +val_348 11 +val_307 11 +val_194 11 +val_414 11 +val_477 11 +val_222 11 +val_126 11 +val_90 11 +val_169 11 +val_403 11 +val_400 11 +val_200 11 +val_97 11 diff --git a/shark/src/test/hive/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 b/shark/src/test/hive/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 new file mode 100644 index 0000000000000..d83d8b62b7095 --- /dev/null +++ b/shark/src/test/hive/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 @@ -0,0 +1,1000 @@ +val_238 11 +val_86 11 +val_311 11 +val_27 11 +val_165 11 +val_409 11 +val_255 11 +val_278 11 +val_98 11 +val_484 11 +val_265 11 +val_193 11 +val_401 11 +val_150 11 +val_273 11 +val_224 11 +val_369 11 +val_66 11 +val_128 11 +val_213 11 +val_146 11 +val_406 11 +val_429 11 +val_374 11 +val_152 11 +val_469 11 +val_145 11 +val_495 11 +val_37 11 +val_327 11 +val_281 11 +val_277 11 +val_209 11 +val_15 11 +val_82 11 +val_403 11 +val_166 11 +val_417 11 +val_430 11 +val_252 11 +val_292 11 +val_219 11 +val_287 11 +val_153 11 +val_193 11 +val_338 11 +val_446 11 +val_459 11 +val_394 11 +val_237 11 +val_482 11 +val_174 11 +val_413 11 +val_494 11 +val_207 11 +val_199 11 +val_466 11 +val_208 11 +val_174 11 +val_399 11 +val_396 11 +val_247 11 +val_417 11 +val_489 11 +val_162 11 +val_377 11 +val_397 11 +val_309 11 +val_365 11 +val_266 11 +val_439 11 +val_342 11 +val_367 11 +val_325 11 +val_167 11 +val_195 11 +val_475 11 +val_17 11 +val_113 11 +val_155 11 +val_203 11 +val_339 11 +val_0 11 +val_455 11 +val_128 11 +val_311 11 +val_316 11 +val_57 11 +val_302 11 +val_205 11 +val_149 11 +val_438 11 +val_345 11 +val_129 11 +val_170 11 +val_20 11 +val_489 11 +val_157 11 +val_378 11 +val_221 11 +val_92 11 +val_111 11 +val_47 11 +val_72 11 +val_4 11 +val_280 11 +val_35 11 +val_427 11 +val_277 11 +val_208 11 +val_356 11 +val_399 11 +val_169 11 +val_382 11 +val_498 11 +val_125 11 +val_386 11 +val_437 11 +val_469 11 +val_192 11 +val_286 11 +val_187 11 +val_176 11 +val_54 11 +val_459 11 +val_51 11 +val_138 11 +val_103 11 +val_239 11 +val_213 11 +val_216 11 +val_430 11 +val_278 11 +val_176 11 +val_289 11 +val_221 11 +val_65 11 +val_318 11 +val_332 11 +val_311 11 +val_275 11 +val_137 11 +val_241 11 +val_83 11 +val_333 11 +val_180 11 +val_284 11 +val_12 11 +val_230 11 +val_181 11 +val_67 11 +val_260 11 +val_404 11 +val_384 11 +val_489 11 +val_353 11 +val_373 11 +val_272 11 +val_138 11 +val_217 11 +val_84 11 +val_348 11 +val_466 11 +val_58 11 +val_8 11 +val_411 11 +val_230 11 +val_208 11 +val_348 11 +val_24 11 +val_463 11 +val_431 11 +val_179 11 +val_172 11 +val_42 11 +val_129 11 +val_158 11 +val_119 11 +val_496 11 +val_0 11 +val_322 11 +val_197 11 +val_468 11 +val_393 11 +val_454 11 +val_100 11 +val_298 11 +val_199 11 +val_191 11 +val_418 11 +val_96 11 +val_26 11 +val_165 11 +val_327 11 +val_230 11 +val_205 11 +val_120 11 +val_131 11 +val_51 11 +val_404 11 +val_43 11 +val_436 11 +val_156 11 +val_469 11 +val_468 11 +val_308 11 +val_95 11 +val_196 11 +val_288 11 +val_481 11 +val_457 11 +val_98 11 +val_282 11 +val_197 11 +val_187 11 +val_318 11 +val_318 11 +val_409 11 +val_470 11 +val_137 11 +val_369 11 +val_316 11 +val_169 11 +val_413 11 +val_85 11 +val_77 11 +val_0 11 +val_490 11 +val_87 11 +val_364 11 +val_179 11 +val_118 11 +val_134 11 +val_395 11 +val_282 11 +val_138 11 +val_238 11 +val_419 11 +val_15 11 +val_118 11 +val_72 11 +val_90 11 +val_307 11 +val_19 11 +val_435 11 +val_10 11 +val_277 11 +val_273 11 +val_306 11 +val_224 11 +val_309 11 +val_389 11 +val_327 11 +val_242 11 +val_369 11 +val_392 11 +val_272 11 +val_331 11 +val_401 11 +val_242 11 +val_452 11 +val_177 11 +val_226 11 +val_5 11 +val_497 11 +val_402 11 +val_396 11 +val_317 11 +val_395 11 +val_58 11 +val_35 11 +val_336 11 +val_95 11 +val_11 11 +val_168 11 +val_34 11 +val_229 11 +val_233 11 +val_143 11 +val_472 11 +val_322 11 +val_498 11 +val_160 11 +val_195 11 +val_42 11 +val_321 11 +val_430 11 +val_119 11 +val_489 11 +val_458 11 +val_78 11 +val_76 11 +val_41 11 +val_223 11 +val_492 11 +val_149 11 +val_449 11 +val_218 11 +val_228 11 +val_138 11 +val_453 11 +val_30 11 +val_209 11 +val_64 11 +val_468 11 +val_76 11 +val_74 11 +val_342 11 +val_69 11 +val_230 11 +val_33 11 +val_368 11 +val_103 11 +val_296 11 +val_113 11 +val_216 11 +val_367 11 +val_344 11 +val_167 11 +val_274 11 +val_219 11 +val_239 11 +val_485 11 +val_116 11 +val_223 11 +val_256 11 +val_263 11 +val_70 11 +val_487 11 +val_480 11 +val_401 11 +val_288 11 +val_191 11 +val_5 11 +val_244 11 +val_438 11 +val_128 11 +val_467 11 +val_432 11 +val_202 11 +val_316 11 +val_229 11 +val_469 11 +val_463 11 +val_280 11 +val_2 11 +val_35 11 +val_283 11 +val_331 11 +val_235 11 +val_80 11 +val_44 11 +val_193 11 +val_321 11 +val_335 11 +val_104 11 +val_466 11 +val_366 11 +val_175 11 +val_403 11 +val_483 11 +val_53 11 +val_105 11 +val_257 11 +val_406 11 +val_409 11 +val_190 11 +val_406 11 +val_401 11 +val_114 11 +val_258 11 +val_90 11 +val_203 11 +val_262 11 +val_348 11 +val_424 11 +val_12 11 +val_396 11 +val_201 11 +val_217 11 +val_164 11 +val_431 11 +val_454 11 +val_478 11 +val_298 11 +val_125 11 +val_431 11 +val_164 11 +val_424 11 +val_187 11 +val_382 11 +val_5 11 +val_70 11 +val_397 11 +val_480 11 +val_291 11 +val_24 11 +val_351 11 +val_255 11 +val_104 11 +val_70 11 +val_163 11 +val_438 11 +val_119 11 +val_414 11 +val_200 11 +val_491 11 +val_237 11 +val_439 11 +val_360 11 +val_248 11 +val_479 11 +val_305 11 +val_417 11 +val_199 11 +val_444 11 +val_120 11 +val_429 11 +val_169 11 +val_443 11 +val_323 11 +val_325 11 +val_277 11 +val_230 11 +val_478 11 +val_178 11 +val_468 11 +val_310 11 +val_317 11 +val_333 11 +val_493 11 +val_460 11 +val_207 11 +val_249 11 +val_265 11 +val_480 11 +val_83 11 +val_136 11 +val_353 11 +val_172 11 +val_214 11 +val_462 11 +val_233 11 +val_406 11 +val_133 11 +val_175 11 +val_189 11 +val_454 11 +val_375 11 +val_401 11 +val_421 11 +val_407 11 +val_384 11 +val_256 11 +val_26 11 +val_134 11 +val_67 11 +val_384 11 +val_379 11 +val_18 11 +val_462 11 +val_492 11 +val_100 11 +val_298 11 +val_9 11 +val_341 11 +val_498 11 +val_146 11 +val_458 11 +val_362 11 +val_186 11 +val_285 11 +val_348 11 +val_167 11 +val_18 11 +val_273 11 +val_183 11 +val_281 11 +val_344 11 +val_97 11 +val_469 11 +val_315 11 +val_84 11 +val_28 11 +val_37 11 +val_448 11 +val_152 11 +val_348 11 +val_307 11 +val_194 11 +val_414 11 +val_477 11 +val_222 11 +val_126 11 +val_90 11 +val_169 11 +val_403 11 +val_400 11 +val_200 11 +val_97 11 +val_238 11 +val_86 11 +val_311 11 +val_27 11 +val_165 11 +val_409 11 +val_255 11 +val_278 11 +val_98 11 +val_484 11 +val_265 11 +val_193 11 +val_401 11 +val_150 11 +val_273 11 +val_224 11 +val_369 11 +val_66 11 +val_128 11 +val_213 11 +val_146 11 +val_406 11 +val_429 11 +val_374 11 +val_152 11 +val_469 11 +val_145 11 +val_495 11 +val_37 11 +val_327 11 +val_281 11 +val_277 11 +val_209 11 +val_15 11 +val_82 11 +val_403 11 +val_166 11 +val_417 11 +val_430 11 +val_252 11 +val_292 11 +val_219 11 +val_287 11 +val_153 11 +val_193 11 +val_338 11 +val_446 11 +val_459 11 +val_394 11 +val_237 11 +val_482 11 +val_174 11 +val_413 11 +val_494 11 +val_207 11 +val_199 11 +val_466 11 +val_208 11 +val_174 11 +val_399 11 +val_396 11 +val_247 11 +val_417 11 +val_489 11 +val_162 11 +val_377 11 +val_397 11 +val_309 11 +val_365 11 +val_266 11 +val_439 11 +val_342 11 +val_367 11 +val_325 11 +val_167 11 +val_195 11 +val_475 11 +val_17 11 +val_113 11 +val_155 11 +val_203 11 +val_339 11 +val_0 11 +val_455 11 +val_128 11 +val_311 11 +val_316 11 +val_57 11 +val_302 11 +val_205 11 +val_149 11 +val_438 11 +val_345 11 +val_129 11 +val_170 11 +val_20 11 +val_489 11 +val_157 11 +val_378 11 +val_221 11 +val_92 11 +val_111 11 +val_47 11 +val_72 11 +val_4 11 +val_280 11 +val_35 11 +val_427 11 +val_277 11 +val_208 11 +val_356 11 +val_399 11 +val_169 11 +val_382 11 +val_498 11 +val_125 11 +val_386 11 +val_437 11 +val_469 11 +val_192 11 +val_286 11 +val_187 11 +val_176 11 +val_54 11 +val_459 11 +val_51 11 +val_138 11 +val_103 11 +val_239 11 +val_213 11 +val_216 11 +val_430 11 +val_278 11 +val_176 11 +val_289 11 +val_221 11 +val_65 11 +val_318 11 +val_332 11 +val_311 11 +val_275 11 +val_137 11 +val_241 11 +val_83 11 +val_333 11 +val_180 11 +val_284 11 +val_12 11 +val_230 11 +val_181 11 +val_67 11 +val_260 11 +val_404 11 +val_384 11 +val_489 11 +val_353 11 +val_373 11 +val_272 11 +val_138 11 +val_217 11 +val_84 11 +val_348 11 +val_466 11 +val_58 11 +val_8 11 +val_411 11 +val_230 11 +val_208 11 +val_348 11 +val_24 11 +val_463 11 +val_431 11 +val_179 11 +val_172 11 +val_42 11 +val_129 11 +val_158 11 +val_119 11 +val_496 11 +val_0 11 +val_322 11 +val_197 11 +val_468 11 +val_393 11 +val_454 11 +val_100 11 +val_298 11 +val_199 11 +val_191 11 +val_418 11 +val_96 11 +val_26 11 +val_165 11 +val_327 11 +val_230 11 +val_205 11 +val_120 11 +val_131 11 +val_51 11 +val_404 11 +val_43 11 +val_436 11 +val_156 11 +val_469 11 +val_468 11 +val_308 11 +val_95 11 +val_196 11 +val_288 11 +val_481 11 +val_457 11 +val_98 11 +val_282 11 +val_197 11 +val_187 11 +val_318 11 +val_318 11 +val_409 11 +val_470 11 +val_137 11 +val_369 11 +val_316 11 +val_169 11 +val_413 11 +val_85 11 +val_77 11 +val_0 11 +val_490 11 +val_87 11 +val_364 11 +val_179 11 +val_118 11 +val_134 11 +val_395 11 +val_282 11 +val_138 11 +val_238 11 +val_419 11 +val_15 11 +val_118 11 +val_72 11 +val_90 11 +val_307 11 +val_19 11 +val_435 11 +val_10 11 +val_277 11 +val_273 11 +val_306 11 +val_224 11 +val_309 11 +val_389 11 +val_327 11 +val_242 11 +val_369 11 +val_392 11 +val_272 11 +val_331 11 +val_401 11 +val_242 11 +val_452 11 +val_177 11 +val_226 11 +val_5 11 +val_497 11 +val_402 11 +val_396 11 +val_317 11 +val_395 11 +val_58 11 +val_35 11 +val_336 11 +val_95 11 +val_11 11 +val_168 11 +val_34 11 +val_229 11 +val_233 11 +val_143 11 +val_472 11 +val_322 11 +val_498 11 +val_160 11 +val_195 11 +val_42 11 +val_321 11 +val_430 11 +val_119 11 +val_489 11 +val_458 11 +val_78 11 +val_76 11 +val_41 11 +val_223 11 +val_492 11 +val_149 11 +val_449 11 +val_218 11 +val_228 11 +val_138 11 +val_453 11 +val_30 11 +val_209 11 +val_64 11 +val_468 11 +val_76 11 +val_74 11 +val_342 11 +val_69 11 +val_230 11 +val_33 11 +val_368 11 +val_103 11 +val_296 11 +val_113 11 +val_216 11 +val_367 11 +val_344 11 +val_167 11 +val_274 11 +val_219 11 +val_239 11 +val_485 11 +val_116 11 +val_223 11 +val_256 11 +val_263 11 +val_70 11 +val_487 11 +val_480 11 +val_401 11 +val_288 11 +val_191 11 +val_5 11 +val_244 11 +val_438 11 +val_128 11 +val_467 11 +val_432 11 +val_202 11 +val_316 11 +val_229 11 +val_469 11 +val_463 11 +val_280 11 +val_2 11 +val_35 11 +val_283 11 +val_331 11 +val_235 11 +val_80 11 +val_44 11 +val_193 11 +val_321 11 +val_335 11 +val_104 11 +val_466 11 +val_366 11 +val_175 11 +val_403 11 +val_483 11 +val_53 11 +val_105 11 +val_257 11 +val_406 11 +val_409 11 +val_190 11 +val_406 11 +val_401 11 +val_114 11 +val_258 11 +val_90 11 +val_203 11 +val_262 11 +val_348 11 +val_424 11 +val_12 11 +val_396 11 +val_201 11 +val_217 11 +val_164 11 +val_431 11 +val_454 11 +val_478 11 +val_298 11 +val_125 11 +val_431 11 +val_164 11 +val_424 11 +val_187 11 +val_382 11 +val_5 11 +val_70 11 +val_397 11 +val_480 11 +val_291 11 +val_24 11 +val_351 11 +val_255 11 +val_104 11 +val_70 11 +val_163 11 +val_438 11 +val_119 11 +val_414 11 +val_200 11 +val_491 11 +val_237 11 +val_439 11 +val_360 11 +val_248 11 +val_479 11 +val_305 11 +val_417 11 +val_199 11 +val_444 11 +val_120 11 +val_429 11 +val_169 11 +val_443 11 +val_323 11 +val_325 11 +val_277 11 +val_230 11 +val_478 11 +val_178 11 +val_468 11 +val_310 11 +val_317 11 +val_333 11 +val_493 11 +val_460 11 +val_207 11 +val_249 11 +val_265 11 +val_480 11 +val_83 11 +val_136 11 +val_353 11 +val_172 11 +val_214 11 +val_462 11 +val_233 11 +val_406 11 +val_133 11 +val_175 11 +val_189 11 +val_454 11 +val_375 11 +val_401 11 +val_421 11 +val_407 11 +val_384 11 +val_256 11 +val_26 11 +val_134 11 +val_67 11 +val_384 11 +val_379 11 +val_18 11 +val_462 11 +val_492 11 +val_100 11 +val_298 11 +val_9 11 +val_341 11 +val_498 11 +val_146 11 +val_458 11 +val_362 11 +val_186 11 +val_285 11 +val_348 11 +val_167 11 +val_18 11 +val_273 11 +val_183 11 +val_281 11 +val_344 11 +val_97 11 +val_469 11 +val_315 11 +val_84 11 +val_28 11 +val_37 11 +val_448 11 +val_152 11 +val_348 11 +val_307 11 +val_194 11 +val_414 11 +val_477 11 +val_222 11 +val_126 11 +val_90 11 +val_169 11 +val_403 11 +val_400 11 +val_200 11 +val_97 11 diff --git a/shark/src/test/hive/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 b/shark/src/test/hive/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 new file mode 100644 index 0000000000000..83bc6257ba98b --- /dev/null +++ b/shark/src/test/hive/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 @@ -0,0 +1,1000 @@ +val_238 11 +val_86 11 +val_311 11 +val_27 11 +val_165 11 +val_409 11 +val_255 11 +val_278 11 +val_98 11 +val_484 11 +val_265 11 +val_193 11 +val_401 11 +val_150 11 +val_273 11 +val_224 11 +val_369 11 +val_66 11 +val_128 11 +val_213 11 +val_146 11 +val_406 11 +val_429 11 +val_374 11 +val_152 11 +val_469 11 +val_145 11 +val_495 11 +val_37 11 +val_327 11 +val_281 11 +val_277 11 +val_209 11 +val_15 11 +val_82 11 +val_403 11 +val_166 11 +val_417 11 +val_430 11 +val_252 11 +val_292 11 +val_219 11 +val_287 11 +val_153 11 +val_193 11 +val_338 11 +val_446 11 +val_459 11 +val_394 11 +val_237 11 +val_482 11 +val_174 11 +val_413 11 +val_494 11 +val_207 11 +val_199 11 +val_466 11 +val_208 11 +val_174 11 +val_399 11 +val_396 11 +val_247 11 +val_417 11 +val_489 11 +val_162 11 +val_377 11 +val_397 11 +val_309 11 +val_365 11 +val_266 11 +val_439 11 +val_342 11 +val_367 11 +val_325 11 +val_167 11 +val_195 11 +val_475 11 +val_17 11 +val_113 11 +val_155 11 +val_203 11 +val_339 11 +val_0 11 +val_455 11 +val_128 11 +val_311 11 +val_316 11 +val_57 11 +val_302 11 +val_205 11 +val_149 11 +val_438 11 +val_345 11 +val_129 11 +val_170 11 +val_20 11 +val_489 11 +val_157 11 +val_378 11 +val_221 11 +val_92 11 +val_111 11 +val_47 11 +val_72 11 +val_4 11 +val_280 11 +val_35 11 +val_427 11 +val_277 11 +val_208 11 +val_356 11 +val_399 11 +val_169 11 +val_382 11 +val_498 11 +val_125 11 +val_386 11 +val_437 11 +val_469 11 +val_192 11 +val_286 11 +val_187 11 +val_176 11 +val_54 11 +val_459 11 +val_51 11 +val_138 11 +val_103 11 +val_239 11 +val_213 11 +val_216 11 +val_430 11 +val_278 11 +val_176 11 +val_289 11 +val_221 11 +val_65 11 +val_318 11 +val_332 11 +val_311 11 +val_275 11 +val_137 11 +val_241 11 +val_83 11 +val_333 11 +val_180 11 +val_284 11 +val_12 11 +val_230 11 +val_181 11 +val_67 11 +val_260 11 +val_404 11 +val_384 11 +val_489 11 +val_353 11 +val_373 11 +val_272 11 +val_138 11 +val_217 11 +val_84 11 +val_348 11 +val_466 11 +val_58 11 +val_8 11 +val_411 11 +val_230 11 +val_208 11 +val_348 11 +val_24 11 +val_463 11 +val_431 11 +val_179 11 +val_172 11 +val_42 11 +val_129 11 +val_158 11 +val_119 11 +val_496 11 +val_0 11 +val_322 11 +val_197 11 +val_468 11 +val_393 11 +val_454 11 +val_100 11 +val_298 11 +val_199 11 +val_191 11 +val_418 11 +val_96 11 +val_26 11 +val_165 11 +val_327 11 +val_230 11 +val_205 11 +val_120 11 +val_131 11 +val_51 11 +val_404 11 +val_43 11 +val_436 11 +val_156 11 +val_469 11 +val_468 11 +val_308 11 +val_95 11 +val_196 11 +val_288 11 +val_481 11 +val_457 11 +val_98 11 +val_282 11 +val_197 11 +val_187 11 +val_318 11 +val_318 11 +val_409 11 +val_470 11 +val_137 11 +val_369 11 +val_316 11 +val_169 11 +val_413 11 +val_85 11 +val_77 11 +val_0 11 +val_490 11 +val_87 11 +val_364 11 +val_179 11 +val_118 11 +val_134 11 +val_395 11 +val_282 11 +val_138 11 +val_238 11 +val_419 11 +val_15 11 +val_118 11 +val_72 11 +val_90 11 +val_307 11 +val_19 11 +val_435 11 +val_10 11 +val_277 11 +val_273 11 +val_306 11 +val_224 11 +val_309 11 +val_389 11 +val_327 11 +val_242 11 +val_369 11 +val_392 11 +val_272 11 +val_331 11 +val_401 11 +val_242 11 +val_452 11 +val_177 11 +val_226 11 +val_5 11 +val_497 11 +val_402 11 +val_396 11 +val_317 11 +val_395 11 +val_58 11 +val_35 11 +val_336 11 +val_95 11 +val_11 11 +val_168 11 +val_34 11 +val_229 11 +val_233 11 +val_143 11 +val_472 11 +val_322 11 +val_498 11 +val_160 11 +val_195 11 +val_42 11 +val_321 11 +val_430 11 +val_119 11 +val_489 11 +val_458 11 +val_78 11 +val_76 11 +val_41 11 +val_223 11 +val_492 11 +val_149 11 +val_449 11 +val_218 11 +val_228 11 +val_138 11 +val_453 11 +val_30 11 +val_209 11 +val_64 11 +val_468 11 +val_76 11 +val_74 11 +val_342 11 +val_69 11 +val_230 11 +val_33 11 +val_368 11 +val_103 11 +val_296 11 +val_113 11 +val_216 11 +val_367 11 +val_344 11 +val_167 11 +val_274 11 +val_219 11 +val_239 11 +val_485 11 +val_116 11 +val_223 11 +val_256 11 +val_263 11 +val_70 11 +val_487 11 +val_480 11 +val_401 11 +val_288 11 +val_191 11 +val_5 11 +val_244 11 +val_438 11 +val_128 11 +val_467 11 +val_432 11 +val_202 11 +val_316 11 +val_229 11 +val_469 11 +val_463 11 +val_280 11 +val_2 11 +val_35 11 +val_283 11 +val_331 11 +val_235 11 +val_80 11 +val_44 11 +val_193 11 +val_321 11 +val_335 11 +val_104 11 +val_466 11 +val_366 11 +val_175 11 +val_403 11 +val_483 11 +val_53 11 +val_105 11 +val_257 11 +val_406 11 +val_409 11 +val_190 11 +val_406 11 +val_401 11 +val_114 11 +val_258 11 +val_90 11 +val_203 11 +val_262 11 +val_348 11 +val_424 11 +val_12 11 +val_396 11 +val_201 11 +val_217 11 +val_164 11 +val_431 11 +val_454 11 +val_478 11 +val_298 11 +val_125 11 +val_431 11 +val_164 11 +val_424 11 +val_187 11 +val_382 11 +val_5 11 +val_70 11 +val_397 11 +val_480 11 +val_291 11 +val_24 11 +val_351 11 +val_255 11 +val_104 11 +val_70 11 +val_163 11 +val_438 11 +val_119 11 +val_414 11 +val_200 11 +val_491 11 +val_237 11 +val_439 11 +val_360 11 +val_248 11 +val_479 11 +val_305 11 +val_417 11 +val_199 11 +val_444 11 +val_120 11 +val_429 11 +val_169 11 +val_443 11 +val_323 11 +val_325 11 +val_277 11 +val_230 11 +val_478 11 +val_178 11 +val_468 11 +val_310 11 +val_317 11 +val_333 11 +val_493 11 +val_460 11 +val_207 11 +val_249 11 +val_265 11 +val_480 11 +val_83 11 +val_136 11 +val_353 11 +val_172 11 +val_214 11 +val_462 11 +val_233 11 +val_406 11 +val_133 11 +val_175 11 +val_189 11 +val_454 11 +val_375 11 +val_401 11 +val_421 11 +val_407 11 +val_384 11 +val_256 11 +val_26 11 +val_134 11 +val_67 11 +val_384 11 +val_379 11 +val_18 11 +val_462 11 +val_492 11 +val_100 11 +val_298 11 +val_9 11 +val_341 11 +val_498 11 +val_146 11 +val_458 11 +val_362 11 +val_186 11 +val_285 11 +val_348 11 +val_167 11 +val_18 11 +val_273 11 +val_183 11 +val_281 11 +val_344 11 +val_97 11 +val_469 11 +val_315 11 +val_84 11 +val_28 11 +val_37 11 +val_448 11 +val_152 11 +val_348 11 +val_307 11 +val_194 11 +val_414 11 +val_477 11 +val_222 11 +val_126 11 +val_90 11 +val_169 11 +val_403 11 +val_400 11 +val_200 11 +val_97 11 +val_238 12 +val_86 12 +val_311 12 +val_27 12 +val_165 12 +val_409 12 +val_255 12 +val_278 12 +val_98 12 +val_484 12 +val_265 12 +val_193 12 +val_401 12 +val_150 12 +val_273 12 +val_224 12 +val_369 12 +val_66 12 +val_128 12 +val_213 12 +val_146 12 +val_406 12 +val_429 12 +val_374 12 +val_152 12 +val_469 12 +val_145 12 +val_495 12 +val_37 12 +val_327 12 +val_281 12 +val_277 12 +val_209 12 +val_15 12 +val_82 12 +val_403 12 +val_166 12 +val_417 12 +val_430 12 +val_252 12 +val_292 12 +val_219 12 +val_287 12 +val_153 12 +val_193 12 +val_338 12 +val_446 12 +val_459 12 +val_394 12 +val_237 12 +val_482 12 +val_174 12 +val_413 12 +val_494 12 +val_207 12 +val_199 12 +val_466 12 +val_208 12 +val_174 12 +val_399 12 +val_396 12 +val_247 12 +val_417 12 +val_489 12 +val_162 12 +val_377 12 +val_397 12 +val_309 12 +val_365 12 +val_266 12 +val_439 12 +val_342 12 +val_367 12 +val_325 12 +val_167 12 +val_195 12 +val_475 12 +val_17 12 +val_113 12 +val_155 12 +val_203 12 +val_339 12 +val_0 12 +val_455 12 +val_128 12 +val_311 12 +val_316 12 +val_57 12 +val_302 12 +val_205 12 +val_149 12 +val_438 12 +val_345 12 +val_129 12 +val_170 12 +val_20 12 +val_489 12 +val_157 12 +val_378 12 +val_221 12 +val_92 12 +val_111 12 +val_47 12 +val_72 12 +val_4 12 +val_280 12 +val_35 12 +val_427 12 +val_277 12 +val_208 12 +val_356 12 +val_399 12 +val_169 12 +val_382 12 +val_498 12 +val_125 12 +val_386 12 +val_437 12 +val_469 12 +val_192 12 +val_286 12 +val_187 12 +val_176 12 +val_54 12 +val_459 12 +val_51 12 +val_138 12 +val_103 12 +val_239 12 +val_213 12 +val_216 12 +val_430 12 +val_278 12 +val_176 12 +val_289 12 +val_221 12 +val_65 12 +val_318 12 +val_332 12 +val_311 12 +val_275 12 +val_137 12 +val_241 12 +val_83 12 +val_333 12 +val_180 12 +val_284 12 +val_12 12 +val_230 12 +val_181 12 +val_67 12 +val_260 12 +val_404 12 +val_384 12 +val_489 12 +val_353 12 +val_373 12 +val_272 12 +val_138 12 +val_217 12 +val_84 12 +val_348 12 +val_466 12 +val_58 12 +val_8 12 +val_411 12 +val_230 12 +val_208 12 +val_348 12 +val_24 12 +val_463 12 +val_431 12 +val_179 12 +val_172 12 +val_42 12 +val_129 12 +val_158 12 +val_119 12 +val_496 12 +val_0 12 +val_322 12 +val_197 12 +val_468 12 +val_393 12 +val_454 12 +val_100 12 +val_298 12 +val_199 12 +val_191 12 +val_418 12 +val_96 12 +val_26 12 +val_165 12 +val_327 12 +val_230 12 +val_205 12 +val_120 12 +val_131 12 +val_51 12 +val_404 12 +val_43 12 +val_436 12 +val_156 12 +val_469 12 +val_468 12 +val_308 12 +val_95 12 +val_196 12 +val_288 12 +val_481 12 +val_457 12 +val_98 12 +val_282 12 +val_197 12 +val_187 12 +val_318 12 +val_318 12 +val_409 12 +val_470 12 +val_137 12 +val_369 12 +val_316 12 +val_169 12 +val_413 12 +val_85 12 +val_77 12 +val_0 12 +val_490 12 +val_87 12 +val_364 12 +val_179 12 +val_118 12 +val_134 12 +val_395 12 +val_282 12 +val_138 12 +val_238 12 +val_419 12 +val_15 12 +val_118 12 +val_72 12 +val_90 12 +val_307 12 +val_19 12 +val_435 12 +val_10 12 +val_277 12 +val_273 12 +val_306 12 +val_224 12 +val_309 12 +val_389 12 +val_327 12 +val_242 12 +val_369 12 +val_392 12 +val_272 12 +val_331 12 +val_401 12 +val_242 12 +val_452 12 +val_177 12 +val_226 12 +val_5 12 +val_497 12 +val_402 12 +val_396 12 +val_317 12 +val_395 12 +val_58 12 +val_35 12 +val_336 12 +val_95 12 +val_11 12 +val_168 12 +val_34 12 +val_229 12 +val_233 12 +val_143 12 +val_472 12 +val_322 12 +val_498 12 +val_160 12 +val_195 12 +val_42 12 +val_321 12 +val_430 12 +val_119 12 +val_489 12 +val_458 12 +val_78 12 +val_76 12 +val_41 12 +val_223 12 +val_492 12 +val_149 12 +val_449 12 +val_218 12 +val_228 12 +val_138 12 +val_453 12 +val_30 12 +val_209 12 +val_64 12 +val_468 12 +val_76 12 +val_74 12 +val_342 12 +val_69 12 +val_230 12 +val_33 12 +val_368 12 +val_103 12 +val_296 12 +val_113 12 +val_216 12 +val_367 12 +val_344 12 +val_167 12 +val_274 12 +val_219 12 +val_239 12 +val_485 12 +val_116 12 +val_223 12 +val_256 12 +val_263 12 +val_70 12 +val_487 12 +val_480 12 +val_401 12 +val_288 12 +val_191 12 +val_5 12 +val_244 12 +val_438 12 +val_128 12 +val_467 12 +val_432 12 +val_202 12 +val_316 12 +val_229 12 +val_469 12 +val_463 12 +val_280 12 +val_2 12 +val_35 12 +val_283 12 +val_331 12 +val_235 12 +val_80 12 +val_44 12 +val_193 12 +val_321 12 +val_335 12 +val_104 12 +val_466 12 +val_366 12 +val_175 12 +val_403 12 +val_483 12 +val_53 12 +val_105 12 +val_257 12 +val_406 12 +val_409 12 +val_190 12 +val_406 12 +val_401 12 +val_114 12 +val_258 12 +val_90 12 +val_203 12 +val_262 12 +val_348 12 +val_424 12 +val_12 12 +val_396 12 +val_201 12 +val_217 12 +val_164 12 +val_431 12 +val_454 12 +val_478 12 +val_298 12 +val_125 12 +val_431 12 +val_164 12 +val_424 12 +val_187 12 +val_382 12 +val_5 12 +val_70 12 +val_397 12 +val_480 12 +val_291 12 +val_24 12 +val_351 12 +val_255 12 +val_104 12 +val_70 12 +val_163 12 +val_438 12 +val_119 12 +val_414 12 +val_200 12 +val_491 12 +val_237 12 +val_439 12 +val_360 12 +val_248 12 +val_479 12 +val_305 12 +val_417 12 +val_199 12 +val_444 12 +val_120 12 +val_429 12 +val_169 12 +val_443 12 +val_323 12 +val_325 12 +val_277 12 +val_230 12 +val_478 12 +val_178 12 +val_468 12 +val_310 12 +val_317 12 +val_333 12 +val_493 12 +val_460 12 +val_207 12 +val_249 12 +val_265 12 +val_480 12 +val_83 12 +val_136 12 +val_353 12 +val_172 12 +val_214 12 +val_462 12 +val_233 12 +val_406 12 +val_133 12 +val_175 12 +val_189 12 +val_454 12 +val_375 12 +val_401 12 +val_421 12 +val_407 12 +val_384 12 +val_256 12 +val_26 12 +val_134 12 +val_67 12 +val_384 12 +val_379 12 +val_18 12 +val_462 12 +val_492 12 +val_100 12 +val_298 12 +val_9 12 +val_341 12 +val_498 12 +val_146 12 +val_458 12 +val_362 12 +val_186 12 +val_285 12 +val_348 12 +val_167 12 +val_18 12 +val_273 12 +val_183 12 +val_281 12 +val_344 12 +val_97 12 +val_469 12 +val_315 12 +val_84 12 +val_28 12 +val_37 12 +val_448 12 +val_152 12 +val_348 12 +val_307 12 +val_194 12 +val_414 12 +val_477 12 +val_222 12 +val_126 12 +val_90 12 +val_169 12 +val_403 12 +val_400 12 +val_200 12 +val_97 12 diff --git a/shark/src/test/hive/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 b/shark/src/test/hive/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 new file mode 100644 index 0000000000000..7ba6caf9d8d59 --- /dev/null +++ b/shark/src/test/hive/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 @@ -0,0 +1,500 @@ +val_238 +val_86 +val_311 +val_27 +val_165 +val_409 +val_255 +val_278 +val_98 +val_484 +val_265 +val_193 +val_401 +val_150 +val_273 +val_224 +val_369 +val_66 +val_128 +val_213 +val_146 +val_406 +val_429 +val_374 +val_152 +val_469 +val_145 +val_495 +val_37 +val_327 +val_281 +val_277 +val_209 +val_15 +val_82 +val_403 +val_166 +val_417 +val_430 +val_252 +val_292 +val_219 +val_287 +val_153 +val_193 +val_338 +val_446 +val_459 +val_394 +val_237 +val_482 +val_174 +val_413 +val_494 +val_207 +val_199 +val_466 +val_208 +val_174 +val_399 +val_396 +val_247 +val_417 +val_489 +val_162 +val_377 +val_397 +val_309 +val_365 +val_266 +val_439 +val_342 +val_367 +val_325 +val_167 +val_195 +val_475 +val_17 +val_113 +val_155 +val_203 +val_339 +val_0 +val_455 +val_128 +val_311 +val_316 +val_57 +val_302 +val_205 +val_149 +val_438 +val_345 +val_129 +val_170 +val_20 +val_489 +val_157 +val_378 +val_221 +val_92 +val_111 +val_47 +val_72 +val_4 +val_280 +val_35 +val_427 +val_277 +val_208 +val_356 +val_399 +val_169 +val_382 +val_498 +val_125 +val_386 +val_437 +val_469 +val_192 +val_286 +val_187 +val_176 +val_54 +val_459 +val_51 +val_138 +val_103 +val_239 +val_213 +val_216 +val_430 +val_278 +val_176 +val_289 +val_221 +val_65 +val_318 +val_332 +val_311 +val_275 +val_137 +val_241 +val_83 +val_333 +val_180 +val_284 +val_12 +val_230 +val_181 +val_67 +val_260 +val_404 +val_384 +val_489 +val_353 +val_373 +val_272 +val_138 +val_217 +val_84 +val_348 +val_466 +val_58 +val_8 +val_411 +val_230 +val_208 +val_348 +val_24 +val_463 +val_431 +val_179 +val_172 +val_42 +val_129 +val_158 +val_119 +val_496 +val_0 +val_322 +val_197 +val_468 +val_393 +val_454 +val_100 +val_298 +val_199 +val_191 +val_418 +val_96 +val_26 +val_165 +val_327 +val_230 +val_205 +val_120 +val_131 +val_51 +val_404 +val_43 +val_436 +val_156 +val_469 +val_468 +val_308 +val_95 +val_196 +val_288 +val_481 +val_457 +val_98 +val_282 +val_197 +val_187 +val_318 +val_318 +val_409 +val_470 +val_137 +val_369 +val_316 +val_169 +val_413 +val_85 +val_77 +val_0 +val_490 +val_87 +val_364 +val_179 +val_118 +val_134 +val_395 +val_282 +val_138 +val_238 +val_419 +val_15 +val_118 +val_72 +val_90 +val_307 +val_19 +val_435 +val_10 +val_277 +val_273 +val_306 +val_224 +val_309 +val_389 +val_327 +val_242 +val_369 +val_392 +val_272 +val_331 +val_401 +val_242 +val_452 +val_177 +val_226 +val_5 +val_497 +val_402 +val_396 +val_317 +val_395 +val_58 +val_35 +val_336 +val_95 +val_11 +val_168 +val_34 +val_229 +val_233 +val_143 +val_472 +val_322 +val_498 +val_160 +val_195 +val_42 +val_321 +val_430 +val_119 +val_489 +val_458 +val_78 +val_76 +val_41 +val_223 +val_492 +val_149 +val_449 +val_218 +val_228 +val_138 +val_453 +val_30 +val_209 +val_64 +val_468 +val_76 +val_74 +val_342 +val_69 +val_230 +val_33 +val_368 +val_103 +val_296 +val_113 +val_216 +val_367 +val_344 +val_167 +val_274 +val_219 +val_239 +val_485 +val_116 +val_223 +val_256 +val_263 +val_70 +val_487 +val_480 +val_401 +val_288 +val_191 +val_5 +val_244 +val_438 +val_128 +val_467 +val_432 +val_202 +val_316 +val_229 +val_469 +val_463 +val_280 +val_2 +val_35 +val_283 +val_331 +val_235 +val_80 +val_44 +val_193 +val_321 +val_335 +val_104 +val_466 +val_366 +val_175 +val_403 +val_483 +val_53 +val_105 +val_257 +val_406 +val_409 +val_190 +val_406 +val_401 +val_114 +val_258 +val_90 +val_203 +val_262 +val_348 +val_424 +val_12 +val_396 +val_201 +val_217 +val_164 +val_431 +val_454 +val_478 +val_298 +val_125 +val_431 +val_164 +val_424 +val_187 +val_382 +val_5 +val_70 +val_397 +val_480 +val_291 +val_24 +val_351 +val_255 +val_104 +val_70 +val_163 +val_438 +val_119 +val_414 +val_200 +val_491 +val_237 +val_439 +val_360 +val_248 +val_479 +val_305 +val_417 +val_199 +val_444 +val_120 +val_429 +val_169 +val_443 +val_323 +val_325 +val_277 +val_230 +val_478 +val_178 +val_468 +val_310 +val_317 +val_333 +val_493 +val_460 +val_207 +val_249 +val_265 +val_480 +val_83 +val_136 +val_353 +val_172 +val_214 +val_462 +val_233 +val_406 +val_133 +val_175 +val_189 +val_454 +val_375 +val_401 +val_421 +val_407 +val_384 +val_256 +val_26 +val_134 +val_67 +val_384 +val_379 +val_18 +val_462 +val_492 +val_100 +val_298 +val_9 +val_341 +val_498 +val_146 +val_458 +val_362 +val_186 +val_285 +val_348 +val_167 +val_18 +val_273 +val_183 +val_281 +val_344 +val_97 +val_469 +val_315 +val_84 +val_28 +val_37 +val_448 +val_152 +val_348 +val_307 +val_194 +val_414 +val_477 +val_222 +val_126 +val_90 +val_169 +val_403 +val_400 +val_200 +val_97 From 29effadbc188c5e6604a9e3a7460d9abde2c2fce Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:30:20 -0800 Subject: [PATCH 699/778] Include alias in attributes that are produced by overridden tables. --- .../scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index fd753d4203dc2..743e480491190 100644 --- a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -2,7 +2,7 @@ package org.apache.spark.sql package catalyst package analysis -import plans.logical.LogicalPlan +import plans.logical.{LogicalPlan, Subquery} import scala.collection.mutable /** @@ -32,6 +32,7 @@ trait OverrideCatalog extends Catalog { alias: Option[String] = None): LogicalPlan = { overrides.get((databaseName, tableName)) + .map(r => alias.map(a => Subquery(a.toLowerCase, r)).getOrElse(r)) .getOrElse(super.lookupRelation(databaseName, tableName, alias)) } From c9116a6aa873e88c6b72d6ddc5d935af7c083f15 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:31:16 -0800 Subject: [PATCH 700/778] Add combiner to avoid NPE when spark performs external aggregation. --- .../main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala index 1599bb51cda4e..6e65d7dc745cf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -56,7 +56,7 @@ class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def groupByKeyLocally(): RDD[(K, Seq[V])] = { def createCombiner(v: V) = ArrayBuffer(v) def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v - val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner _, mergeValue _, null) + val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner _, mergeValue _, _ ++ _) val bufs = self.mapPartitionsWithContext((context, iter) => { new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) }, preservesPartitioning = true) From 8c01c2475ef87d589263ba215f26530346b9868d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:31:42 -0800 Subject: [PATCH 701/778] Move definition of Row out of execution to top level sql package. --- catalyst/src/main/scala/org/apache/spark/sql/package.scala | 2 ++ .../src/main/scala/org/apache/spark/sql/execution/package.scala | 1 - 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/catalyst/src/main/scala/org/apache/spark/sql/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/package.scala index de4f4d1c24769..29eb33b9e6065 100644 --- a/catalyst/src/main/scala/org/apache/spark/sql/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/package.scala @@ -12,4 +12,6 @@ package object sql { com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger(name)) protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging + + type Row = catalyst.expressions.Row } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/package.scala b/core/src/main/scala/org/apache/spark/sql/execution/package.scala index aef8ee9ccfd05..9097ec1c318d0 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/package.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/package.scala @@ -4,5 +4,4 @@ package org.apache.spark.sql * An execution engine for relational query plans that runs on top Spark and returns RDDs. */ package object execution { - type Row = catalyst.expressions.Row } From 4905b2b0b5f5cc8c123b41ccbb2daec117f73fad Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:33:17 -0800 Subject: [PATCH 702/778] Add more efficient TopK that avoids global sort for logical Sort => StopAfter. --- .../org/apache/spark/sql/SparkSqlContext.scala | 1 + .../spark/sql/execution/SparkStrategies.scala | 8 ++++++++ .../spark/sql/execution/basicOperators.scala | 16 ++++++++++++++++ .../apache/spark/sql/shark/SharkContext.scala | 1 + 4 files changed, 26 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala b/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala index f0a250a374ec0..fa7422ceefad5 100644 --- a/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala +++ b/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala @@ -60,6 +60,7 @@ class SparkSqlContext(val sparkContext: SparkContext) extends Logging { val sparkContext = self.sparkContext val strategies: Seq[Strategy] = + TopK :: PartialAggregation :: SparkEquiInnerJoin :: BasicOperators :: diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 37fa5b9823ab3..0013c54c87c1f 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -137,6 +137,14 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { protected lazy val singleRowRdd = sparkContext.parallelize(Seq(new GenericRow(IndexedSeq()): Row), 1) + object TopK extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case logical.StopAfter(limit, logical.Sort(order, child)) => + execution.TopK(Evaluate(limit, Nil).asInstanceOf[Int], order, planLater(child))(sparkContext) :: Nil + case _ => Nil + } + } + // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 9e79fdda68b9b..0b12a519b1920 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -53,6 +53,22 @@ case class StopAfter(limit: Int, child: SparkPlan)(@transient sc: SparkContext) def execute() = sc.makeRDD(executeCollect(), 1) } +case class TopK(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode { + override def otherCopyArgs = sc :: Nil + + def output = child.output + + @transient + lazy val ordering = new RowOrdering(sortOrder) + + override def executeCollect() = child.execute().takeOrdered(limit)(ordering) + + // TODO: Terminal split should be implemented differently from non-terminal split. + // TODO: Pick num splits based on |limit|. + def execute() = sc.makeRDD(executeCollect(), 1) +} + + case class Sort( sortOrder: Seq[SortOrder], global: Boolean, diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala b/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala index 1bc2d56c7eab0..8589001b8eaaf 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala @@ -172,6 +172,7 @@ abstract class SharkContext(sc: SparkContext) extends SparkSqlContext(sc) { val sharkContext = self override val strategies: Seq[Strategy] = Seq( + TopK, ColumnPrunings, PartitionPrunings, HiveTableScans, From 532dd3748c262cdeea2f9f7977ba3a875e8b73fe Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:34:06 -0800 Subject: [PATCH 703/778] Allow the local warehouse path to be specified. --- .../scala/org/apache/spark/sql/shark/SharkContext.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala b/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala index 8589001b8eaaf..2035f0db4c2d2 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala @@ -26,8 +26,11 @@ import scala.collection.JavaConversions._ * Starts up an instance of shark where metadata is stored locally. An in-process metadata data is * created with data stored in ./metadata. Warehouse data is stored in in ./warehouse. */ -class LocalSharkContext(sc: SparkContext) extends SharkContext(sc) { - override def warehousePath = new File("warehouse").getCanonicalPath +class LocalSharkContext( + sc: SparkContext, + override val warehousePath: String = new File("warehouse").getCanonicalPath) + extends SharkContext(sc) { + override def metastorePath = new File("metastore").getCanonicalPath } From a4308954350a578dae8d8d4d49ac7ec52c2d0fe7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:34:35 -0800 Subject: [PATCH 704/778] Planning for logical Repartition operators. --- .../org/apache/spark/sql/execution/SparkStrategies.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 0013c54c87c1f..5ef75030fa9fb 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -147,6 +147,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { + // TOOD: Set + val numPartitions = 200 def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Distinct(child) => execution.Aggregate( @@ -180,6 +182,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil case logical.NoRelation => execution.ExistingRdd(Nil, singleRowRdd) :: Nil + case logical.Repartition(expressions, child) => + execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil case _ => Nil } } From 5fe7de411c437d958d414d5530c56aceb6f6bfc3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:36:09 -0800 Subject: [PATCH 705/778] Move table creation out of rule into a separate function. --- .../sql/shark/HiveMetastoreCatalog.scala | 48 ++++++++++--------- 1 file changed, 26 insertions(+), 22 deletions(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala b/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala index 7c24f7df971d9..c67e4ecbb4d84 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala @@ -20,6 +20,8 @@ import catalyst.types._ import scala.collection.JavaConversions._ class HiveMetastoreCatalog(shark: SharkContext) extends Catalog with Logging { + import HiveMetastoreTypes._ + val client = Hive.get(shark.hiveconf) def lookupRelation( @@ -42,37 +44,39 @@ class HiveMetastoreCatalog(shark: SharkContext) extends Catalog with Logging { alias)(table.getTTable, partitions.map(part => part.getTPartition)) } + def createTable(databaseName: String, tableName: String, schema: Seq[Attribute]) { + val table = new Table(databaseName, tableName) + val hiveSchema = + schema.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), "")) + table.setFields(hiveSchema) + + val sd = new StorageDescriptor() + table.getTTable.setSd(sd) + sd.setCols(hiveSchema) + + // TODO: THESE ARE ALL DEFAULTS, WE NEED TO PARSE / UNDERSTAND the output specs. + sd.setCompressed(false) + sd.setParameters(Map[String, String]()) + sd.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") + sd.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") + val serDeInfo = new SerDeInfo() + serDeInfo.setName(tableName) + serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") + serDeInfo.setParameters(Map[String, String]()) + sd.setSerdeInfo(serDeInfo) + client.createTable(table) + } + /** * Creates any tables required for query execution. * For example, because of a CREATE TABLE X AS statement. */ object CreateTables extends Rule[LogicalPlan] { - import HiveMetastoreTypes._ - def apply(plan: LogicalPlan): LogicalPlan = plan transform { case InsertIntoCreatedTable(db, tableName, child) => val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase()) - val table = new Table(databaseName, tableName) - val schema = - child.output.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), "")) - table.setFields(schema) - - val sd = new StorageDescriptor() - table.getTTable.setSd(sd) - sd.setCols(schema) - - // TODO: THESE ARE ALL DEFAULTS, WE NEED TO PARSE / UNDERSTAND the output specs. - sd.setCompressed(false) - sd.setParameters(Map[String, String]()) - sd.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") - sd.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") - val serDeInfo = new SerDeInfo() - serDeInfo.setName(tableName) - serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") - serDeInfo.setParameters(Map[String, String]()) - sd.setSerdeInfo(serDeInfo) - client.createTable(table) + createTable(databaseName, tableName, child.output) InsertIntoTable( lookupRelation(Some(databaseName), tableName, None).asInstanceOf[BaseRelation], From b9225114460f9d628738b690fc0b33ba81a3c019 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:37:06 -0800 Subject: [PATCH 706/778] Fix insertion of nested types into hive tables. --- .../org/apache/spark/sql/shark/hiveOperators.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala index 7dc8a02c5af1c..7ef50051126e7 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala @@ -180,8 +180,16 @@ case class InsertIntoHiveTable( case (s: String, oi: JavaHiveVarcharObjectInspector) => new HiveVarchar(s, s.size) case (bd: BigDecimal, oi: JavaHiveDecimalObjectInspector) => new HiveDecimal(bd.underlying()) + case (row: Row, oi: StandardStructObjectInspector) => + val struct = oi.create() + row.zip(oi.getAllStructFieldRefs).foreach { + case (data, field) => + oi.setStructFieldData(struct, field, wrap(data, field.getFieldObjectInspector)) + } + struct case (s: Seq[_], oi: ListObjectInspector) => - seqAsJavaList(s.map(wrap(_, oi.getListElementObjectInspector))) + val wrappedSeq = s.map(wrap(_, oi.getListElementObjectInspector)) + seqAsJavaList(wrappedSeq) case (obj, _) => obj } From 18a861b108eb20afa1a87ee04324de829478b4d2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:38:06 -0800 Subject: [PATCH 707/778] Correctly convert nested products into nested rows when turning scala data into catalyst data. --- .../org/apache/spark/sql/execution/SparkStrategies.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5ef75030fa9fb..79a24451908ed 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -137,6 +137,12 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { protected lazy val singleRowRdd = sparkContext.parallelize(Seq(new GenericRow(IndexedSeq()): Row), 1) + def convertToCatalyst(a: Any): Any = a match { + case s: Seq[Any] => s.map(convertToCatalyst) + case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toSeq) + case other => other + } + object TopK extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.StopAfter(limit, logical.Sort(order, child)) => @@ -170,7 +176,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => val dataAsRdd = - sparkContext.parallelize(data.map(r => new GenericRow(r.productIterator.toVector): Row)) + sparkContext.parallelize(data.map(r => new GenericRow(r.productIterator.map(convertToCatalyst).toVector): Row)) execution.ExistingRdd(output, dataAsRdd) :: Nil case logical.StopAfter(limit, child) => execution.StopAfter( From df88f01e1d449433e2f149dbaea90a9611848ff9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:38:44 -0800 Subject: [PATCH 708/778] add a simple test for aggregation --- .../test/scala/org/apache/spark/sql/DslQueryTests.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala index 5f2f5ce9b8f0d..559a8e09880b1 100644 --- a/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala @@ -96,6 +96,13 @@ class BasicQuerySuite extends DslQueryTest { testData.data) } + test("agg") { + checkAnswer( + testData2.groupBy('a)('a, Sum('b)), + Seq((1,3),(2,3),(3,3)) + ) + } + test("select *") { checkAnswer( testData.select(Star(None)), From 6e04e5b944113bc2c0cb528dcac1ccf3276109e2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 13:39:14 -0800 Subject: [PATCH 709/778] Add insertIntoTable to the DSL. --- .../main/scala/org/apache/spark/sql/catalyst/dsl/package.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 6f81e92058bbb..1d7b7e61db5f6 100644 --- a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -138,6 +138,9 @@ package object dsl { alias: Option[String] = None) = Generate(generator, join, outer, None, plan) + def insertInto(tableName: String, overwrite: Boolean = false) = + InsertIntoTable(analysis.UnresolvedRelation(None, tableName), Map.empty, plan, overwrite) + def analyze = analysis.SimpleAnalyzer(plan) } } From 24eaa79764253a2771c980728037e17bbef17b50 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 14:22:06 -0800 Subject: [PATCH 710/778] fix > 100 chars --- .../org/apache/spark/sql/execution/SparkStrategies.scala | 6 ++++-- .../org/apache/spark/sql/execution/basicOperators.scala | 3 ++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 79a24451908ed..854bec6591974 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -146,7 +146,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object TopK extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.StopAfter(limit, logical.Sort(order, child)) => - execution.TopK(Evaluate(limit, Nil).asInstanceOf[Int], order, planLater(child))(sparkContext) :: Nil + execution.TopK( + Evaluate(limit, Nil).asInstanceOf[Int], order, planLater(child))(sparkContext) :: Nil case _ => Nil } } @@ -176,7 +177,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => val dataAsRdd = - sparkContext.parallelize(data.map(r => new GenericRow(r.productIterator.map(convertToCatalyst).toVector): Row)) + sparkContext.parallelize(data.map(r => + new GenericRow(r.productIterator.map(convertToCatalyst).toVector): Row)) execution.ExistingRdd(output, dataAsRdd) :: Nil case logical.StopAfter(limit, child) => execution.StopAfter( diff --git a/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 0b12a519b1920..377742925c737 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -53,7 +53,8 @@ case class StopAfter(limit: Int, child: SparkPlan)(@transient sc: SparkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class TopK(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode { +case class TopK(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) + (@transient sc: SparkContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil def output = child.output From d393d2abebc03408fc43dbd835105134fa256463 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 14:41:37 -0800 Subject: [PATCH 711/778] Review Comments: Add comment to map that adds a sub query. --- .../apache/spark/sql/catalyst/analysis/Catalog.scala | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 743e480491190..adedd1567e1f2 100644 --- a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -31,9 +31,14 @@ trait OverrideCatalog extends Catalog { tableName: String, alias: Option[String] = None): LogicalPlan = { - overrides.get((databaseName, tableName)) - .map(r => alias.map(a => Subquery(a.toLowerCase, r)).getOrElse(r)) - .getOrElse(super.lookupRelation(databaseName, tableName, alias)) + val overriddenTable = overrides.get((databaseName, tableName)) + + // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are + // properly qualified with this alias. + val withAlias = + overriddenTable.map(r => alias.map(a => Subquery(a.toLowerCase, r)).getOrElse(r)) + + withAlias.getOrElse(super.lookupRelation(databaseName, tableName, alias)) } def overrideTable(databaseName: Option[String], tableName: String, plan: LogicalPlan) = From f5e7492c267758c80b7ad3e4c74b3b20b34ec9e0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 25 Feb 2014 14:44:02 -0800 Subject: [PATCH 712/778] Add Apache license. Make naming more consistent. --- project/SparkBuild.scala | 19 +++-- sbt/sbt | 9 +++ .../catalyst/analysis/FunctionRegistry.scala | 20 ----- .../spark/sql/catalyst/analysis/package.scala | 8 -- .../spark/sql/catalyst/expressions/Cast.scala | 12 --- .../catalyst/expressions/ImplementedUdf.scala | 10 --- .../spark/sql/catalyst/expressions/Rand.scala | 12 --- .../sql/catalyst/expressions/ScalaUdf.scala | 19 ----- .../sql/catalyst/expressions/SortOrder.scala | 17 ----- .../catalyst/expressions/WrapDynamic.scala | 27 ------- .../expressions/stringOperations.scala | 12 --- .../spark/sql/catalyst/planning/package.scala | 7 -- .../catalyst/plans/logical/BaseRelation.scala | 10 --- .../plans/logical/ScriptTransformation.scala | 21 ------ .../catalyst/plans/logical/partitioning.scala | 29 -------- .../spark/sql/catalyst/plans/package.scala | 8 -- .../spark/sql/catalyst/plans/shared.scala | 9 --- .../spark/sql/catalyst/rules/Rule.scala | 16 ---- .../spark/sql/catalyst/rules/package.scala | 7 -- .../spark/sql/catalyst/types/package.scala | 7 -- .../scala/org/apache/spark/sql/package.scala | 17 ----- .../spark/sql/catalyst/AnalysisSuite.scala | 23 ------ .../execution/QueryExecutionException.scala | 4 - .../apache/spark/sql/execution/debug.scala | 29 -------- .../apache/spark/sql/execution/package.scala | 7 -- .../shark/execution/ConcurrentHiveSuite.scala | 21 ------ .../sql/shark/execution/HiveSerDeSuite.scala | 16 ---- .../execution/HiveTypeCoercionSuite.scala | 15 ---- {schema => sql}/README.md | 0 .../sql/catalyst/analysis/Analyzer.scala | 17 +++++ .../spark/sql/catalyst/analysis/Catalog.scala | 17 +++++ .../catalyst/analysis/FunctionRegistry.scala | 37 ++++++++++ .../catalyst/analysis/HiveTypeCoercion.scala | 17 +++++ .../spark/sql/catalyst/analysis/package.scala | 25 +++++++ .../sql/catalyst/analysis/unresolved.scala | 17 +++++ .../spark/sql/catalyst/dsl/package.scala | 17 +++++ .../spark/sql/catalyst/errors/package.scala | 17 +++++ .../catalyst/expressions/BoundAttribute.scala | 17 +++++ .../spark/sql/catalyst/expressions/Cast.scala | 29 ++++++++ .../sql/catalyst/expressions/Evaluate.scala | 17 +++++ .../sql/catalyst/expressions/Expression.scala | 17 +++++ .../catalyst/expressions/ImplementedUdf.scala | 27 +++++++ .../spark/sql/catalyst/expressions/Rand.scala | 29 ++++++++ .../spark/sql/catalyst/expressions/Row.scala | 17 +++++ .../sql/catalyst/expressions/ScalaUdf.scala | 36 +++++++++ .../sql/catalyst/expressions/SortOrder.scala | 34 +++++++++ .../catalyst/expressions/WrapDynamic.scala | 44 +++++++++++ .../sql/catalyst/expressions/aggregates.scala | 17 +++++ .../sql/catalyst/expressions/arithmetic.scala | 17 +++++ .../catalyst/expressions/complexTypes.scala | 17 +++++ .../sql/catalyst/expressions/generators.scala | 17 +++++ .../sql/catalyst/expressions/literals.scala | 17 +++++ .../expressions/namedExpressions.scala | 17 +++++ .../catalyst/expressions/nullFunctions.scala | 17 +++++ .../sql/catalyst/expressions/package.scala | 17 +++++ .../sql/catalyst/expressions/predicates.scala | 17 +++++ .../expressions/stringOperations.scala | 29 ++++++++ .../sql/catalyst/optimizer/Optimizer.scala | 17 +++++ .../sql/catalyst/planning/QueryPlanner.scala | 17 +++++ .../spark/sql/catalyst/planning/package.scala | 24 ++++++ .../sql/catalyst/planning/patterns.scala | 17 +++++ .../spark/sql/catalyst/plans/QueryPlan.scala | 17 +++++ .../catalyst/plans/logical/BaseRelation.scala | 27 +++++++ .../catalyst/plans/logical/LogicalPlan.scala | 17 +++++ .../plans/logical/ScriptTransformation.scala | 38 ++++++++++ .../catalyst/plans/logical/TestRelation.scala | 17 +++++ .../plans/logical/basicOperators.scala | 17 +++++ .../catalyst/plans/logical/partitioning.scala | 46 ++++++++++++ .../spark/sql/catalyst/plans/package.scala | 25 +++++++ .../plans/physical/partitioning.scala | 17 +++++ .../spark/sql/catalyst/plans/shared.scala | 26 +++++++ .../spark/sql/catalyst/rules/Rule.scala | 33 +++++++++ .../sql/catalyst/rules/RuleExecutor.scala | 17 +++++ .../spark/sql/catalyst/rules/package.scala | 24 ++++++ .../spark/sql/catalyst/trees/TreeNode.scala | 17 +++++ .../spark/sql/catalyst/trees/package.scala | 17 +++++ .../spark/sql/catalyst/types/dataTypes.scala | 17 +++++ .../spark/sql/catalyst/types/package.scala | 24 ++++++ .../spark/sql/catalyst/util/package.scala | 17 +++++ .../scala/org/apache/spark/sql/package.scala | 34 +++++++++ .../spark/sql/catalyst/AnalysisSuite.scala | 40 ++++++++++ .../sql/catalyst/DistributionSuite.scala | 18 ++++- .../catalyst/ExpressionEvaluationSuite.scala | 17 +++++ .../sql/catalyst/HiveTypeCoercionSuite.scala | 17 +++++ .../sql/catalyst/RuleExecutorSuite.scala | 17 +++++ .../spark/sql/catalyst/TreeNodeSuite.scala | 17 +++++ .../optimizer/ConstantFoldingSuite.scala | 17 +++++ .../optimizer/FilterPushdownSuite.scala | 0 .../catalyst/optimizer/OptimizerTest.scala | 0 .../rdd/PartitionLocalRDDFunctions.scala | 33 ++++++--- .../apache/spark/sql/SparkSqlContext.scala | 17 +++++ .../spark/sql/examples/SchemaRddExample.scala | 17 +++++ .../spark/sql/examples/ViewsExample.scala | 19 ++++- .../apache/spark/sql/execution/Exchange.scala | 17 +++++ .../apache/spark/sql/execution/Generate.scala | 17 +++++ .../execution/QueryExecutionException.scala | 21 ++++++ .../spark/sql/execution/SharkPlan.scala | 17 +++++ .../spark/sql/execution/SparkStrategies.scala | 17 +++++ .../spark/sql/execution/aggregates.scala | 19 ++++- .../spark/sql/execution/basicOperators.scala | 17 +++++ .../apache/spark/sql/execution/debug.scala | 46 ++++++++++++ .../apache/spark/sql/execution/joins.scala | 19 ++++- .../apache/spark/sql/execution/package.scala | 24 ++++++ .../core/src/test/resources/log4j.properties | 0 .../org/apache/spark/sql/DslQueryTests.scala | 21 +++++- .../org/apache/spark/sql/PlannerSuite.scala | 17 +++++ .../scala/org/apache/spark/sql/TgfSuite.scala | 17 +++++ .../hadoop/mapred/SharkHadoopWriter.scala | 21 +++++- .../apache/spark/sql/hive/HiveContext.scala | 54 +++++++++----- .../sql/hive}/HiveMetastoreCatalog.scala | 25 ++++++- .../org/apache/spark/sql/hive}/HiveQl.scala | 19 ++++- .../spark/sql/hive/HiveStrategies.scala | 37 +++++++--- .../sql/hive}/ScriptTransformation.scala | 21 +++++- .../apache/spark/sql/hive}/TableReader.scala | 31 ++++++-- .../apache/spark/sql/hive}/TestShark.scala | 70 ++++++++---------- .../spark/sql/hive}/hiveOperators.scala | 29 ++++++-- .../org/apache/spark/sql/hive}/hiveUdfs.scala | 19 ++++- .../hive/src/test/resources/log4j.properties | 0 .../execution/BigDataBenchmarkSuite.scala | 21 +++++- .../hive/execution/ConcurrentHiveSuite.scala | 38 ++++++++++ .../hive}/execution/HiveComparisonTest.scala | 73 ++++++++++++------- .../execution/HiveCompatibilitySuite.scala | 23 +++++- .../hive}/execution/HiveQueryFileTest.scala | 25 ++++++- .../sql/hive}/execution/HiveQuerySuite.scala | 21 +++++- .../hive}/execution/HiveResolutionSuite.scala | 21 +++++- .../sql/hive/execution/HiveSerDeSuite.scala | 33 +++++++++ .../execution/HiveTypeCoercionSuite.scala | 32 ++++++++ .../sql/hive}/execution/PruningSuite.scala | 23 +++++- 128 files changed, 2137 insertions(+), 543 deletions(-) delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala delete mode 100644 schema/catalyst/src/main/scala/org/apache/spark/sql/package.scala delete mode 100644 schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala delete mode 100644 schema/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala delete mode 100644 schema/core/src/main/scala/org/apache/spark/sql/execution/debug.scala delete mode 100644 schema/core/src/main/scala/org/apache/spark/sql/execution/package.scala delete mode 100644 schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/ConcurrentHiveSuite.scala delete mode 100644 schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveSerDeSuite.scala delete mode 100644 schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveTypeCoercionSuite.scala rename {schema => sql}/README.md (100%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala (88%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala (71%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala (93%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala (81%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala (85%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala (63%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala (72%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala (92%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala (73%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala (82%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala (90%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala (61%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala (64%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala (81%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala (54%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala (83%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala (52%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala (72%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala (79%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala (87%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala (69%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala (82%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala (81%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala (83%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala (63%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala (83%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala (89%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala (71%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala (93%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala (53%) rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala (82%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala rename {schema => sql}/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala (77%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala rename {schema => sql}/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala (84%) rename {schema => sql}/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala (76%) rename {schema => sql}/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala (72%) rename {schema => sql}/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala (56%) rename {schema => sql}/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala (70%) rename {schema => sql}/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala (85%) rename {schema => sql}/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala (100%) rename {schema => sql}/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala (100%) rename schema/core/src/main/scala/org/apache/spark/rdd/SharkPairRddFunctions.scala => sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala (67%) rename {schema => sql}/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala (82%) rename {schema => sql}/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala (70%) rename {schema => sql}/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala (67%) rename {schema => sql}/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala (85%) rename {schema => sql}/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala (64%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala rename {schema => sql}/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala (62%) rename {schema => sql}/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala (91%) rename {schema => sql}/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala (80%) rename {schema => sql}/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala (78%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala rename {schema => sql}/core/src/main/scala/org/apache/spark/sql/execution/joins.scala (83%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala rename {schema => sql}/core/src/test/resources/log4j.properties (100%) rename {schema => sql}/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala (87%) rename {schema => sql}/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala (62%) rename {schema => sql}/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala (67%) rename {schema => sql}/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala (85%) rename schema/hive/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala => sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala (83%) rename {schema/hive/src/main/scala/org/apache/spark/sql/shark => sql/hive/src/main/scala/org/apache/spark/sql/hive}/HiveMetastoreCatalog.scala (88%) rename {schema/hive/src/main/scala/org/apache/spark/sql/shark => sql/hive/src/main/scala/org/apache/spark/sql/hive}/HiveQl.scala (97%) rename schema/hive/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala => sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala (80%) rename {schema/hive/src/main/scala/org/apache/spark/sql/shark => sql/hive/src/main/scala/org/apache/spark/sql/hive}/ScriptTransformation.scala (67%) rename {schema/hive/src/main/scala/org/apache/spark/sql/shark => sql/hive/src/main/scala/org/apache/spark/sql/hive}/TableReader.scala (89%) rename {schema/hive/src/main/scala/org/apache/spark/sql/shark => sql/hive/src/main/scala/org/apache/spark/sql/hive}/TestShark.scala (88%) rename {schema/hive/src/main/scala/org/apache/spark/sql/shark => sql/hive/src/main/scala/org/apache/spark/sql/hive}/hiveOperators.scala (91%) rename {schema/hive/src/main/scala/org/apache/spark/sql/shark => sql/hive/src/main/scala/org/apache/spark/sql/hive}/hiveUdfs.scala (95%) rename {schema => sql}/hive/src/test/resources/log4j.properties (100%) rename {schema/hive/src/test/scala/org/apache/spark/sql/shark => sql/hive/src/test/scala/org/apache/spark/sql/hive}/execution/BigDataBenchmarkSuite.scala (81%) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala rename {schema/hive/src/test/scala/org/apache/spark/sql/shark => sql/hive/src/test/scala/org/apache/spark/sql/hive}/execution/HiveComparisonTest.scala (83%) rename {schema/hive/src/test/scala/org/apache/spark/sql/shark => sql/hive/src/test/scala/org/apache/spark/sql/hive}/execution/HiveCompatibilitySuite.scala (94%) rename {schema/hive/src/test/scala/org/apache/spark/sql/shark => sql/hive/src/test/scala/org/apache/spark/sql/hive}/execution/HiveQueryFileTest.scala (65%) rename {schema/hive/src/test/scala/org/apache/spark/sql/shark => sql/hive/src/test/scala/org/apache/spark/sql/hive}/execution/HiveQuerySuite.scala (82%) rename {schema/hive/src/test/scala/org/apache/spark/sql/shark => sql/hive/src/test/scala/org/apache/spark/sql/hive}/execution/HiveResolutionSuite.scala (64%) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala rename {schema/hive/src/test/scala/org/apache/spark/sql/shark => sql/hive/src/test/scala/org/apache/spark/sql/hive}/execution/PruningSuite.scala (83%) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index dace70e37ba97..63f7c3db6c1a4 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -58,11 +58,11 @@ object SparkBuild extends Build { lazy val graphx = Project("graphx", file("graphx"), settings = graphxSettings) dependsOn(core) - lazy val catalyst = Project("catalyst", file("schema/catalyst"), settings = catalystSettings) + lazy val catalyst = Project("catalyst", file("sql/catalyst"), settings = catalystSettings) - lazy val schema = Project("schema", file("schema/core"), settings = schemaCoreSettings) dependsOn(core, catalyst) + lazy val sql = Project("sql", file("sql/core"), settings = schemaCoreSettings) dependsOn(core, catalyst) - lazy val hive = Project("hive", file("schema/hive"), settings = hiveSettings) dependsOn(schema) + lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) @@ -123,7 +123,7 @@ object SparkBuild extends Build { .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) // Everything except assembly, tools and examples belong to packageProjects - lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx) ++ maybeYarnRef + lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, hive) ++ maybeYarnRef lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) @@ -365,8 +365,13 @@ object SparkBuild extends Build { "org.apache.hive" % "hive-metastore" % "0.12.0", "org.apache.hive" % "hive-exec" % "0.12.0", "org.apache.hive" % "hive-serde" % "0.12.0"), - // Multiple queries rely on the TestShark singleton. See comments there for more details. + // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, + // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings + // only for this subproject. + scalacOptions <<= scalacOptions map { currentOpts: Seq[String] => + currentOpts.filterNot(_ == "-deprecation") + }, initialCommands in console := """ |import org.apache.spark.sql.catalyst.analysis._ @@ -378,8 +383,8 @@ object SparkBuild extends Build { |import org.apache.spark.sql.catalyst.types._ |import org.apache.spark.sql.catalyst.util._ |import org.apache.spark.sql.execution - |import org.apache.spark.sql.shark._ - |import org.apache.spark.sql.shark.TestShark._""".stripMargin + |import org.apache.spark.sql.hive._ + |import org.apache.spark.sql.hive.TestHive._""".stripMargin ) def streamingSettings = sharedSettings ++ Seq( diff --git a/sbt/sbt b/sbt/sbt index 8472dce589bcc..efa575ca64f5a 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -17,6 +17,15 @@ # limitations under the License. # +# Required to correctly launch hadoop/hive for generating the golden answers. +HADOOP_CLASSPATH="" + +for i in $HIVE_HOME/lib/* +do HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$i +done + +export HADOOP_CLASSPATH + # This script launches sbt for this project. If present it uses the system # version of sbt. If there is no system version of sbt it attempts to download # sbt locally. diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala deleted file mode 100644 index c18e4f5790203..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ /dev/null @@ -1,20 +0,0 @@ -package org.apache.spark.sql -package catalyst -package analysis - -import expressions._ - -/** A catalog for looking up user defined functions, used by an [[Analyzer]]. */ -trait FunctionRegistry { - def lookupFunction(name: String, children: Seq[Expression]): Expression -} - -/** - * A trivial catalog that returns an error when a function is requested. Used for testing when all - * functions are already filled in and the analyser needs only to resolve attribute references. - */ -object EmptyFunctionRegistry extends FunctionRegistry { - def lookupFunction(name: String, children: Seq[Expression]): Expression = { - throw new UnsupportedOperationException - } -} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala deleted file mode 100644 index 050763c619702..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ /dev/null @@ -1,8 +0,0 @@ -package catalyst - -/** - * Provides a logical query plan [[Analyzer]] and supporting classes for performing analysis. - * Analysis consists of translating [[UnresolvedAttribute]]s and [[UnresolvedRelation]]s - * into fully typed objects using information in a schema [[Catalog]]. - */ -package object analysis diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala deleted file mode 100644 index bd42c662a041f..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ /dev/null @@ -1,12 +0,0 @@ -package org.apache.spark.sql -package catalyst -package expressions - -import types.DataType - -/** Cast the child expression to the target data type. */ -case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"CAST($child, $dataType)" -} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala deleted file mode 100644 index 4322326c0fd0e..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala +++ /dev/null @@ -1,10 +0,0 @@ -package org.apache.spark.sql -package catalyst -package expressions - -/** - * A UDF that has a native JVM implementation. - */ -trait ImplementedUdf { - def evaluate(evaluatedChildren: Seq[Any]): Any -} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala deleted file mode 100644 index 602b1d76f4610..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala +++ /dev/null @@ -1,12 +0,0 @@ -package org.apache.spark.sql -package catalyst -package expressions - -import types.DoubleType - -case object Rand extends LeafExpression { - def dataType = DoubleType - def nullable = false - def references = Set.empty - override def toString = "RAND()" -} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala deleted file mode 100644 index 5698fd8ad8d01..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ /dev/null @@ -1,19 +0,0 @@ -package org.apache.spark.sql -package catalyst -package expressions - -import types._ - -case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) - extends Expression with ImplementedUdf { - - def references = children.flatMap(_.references).toSet - def nullable = true - - def evaluate(evaluatedChildren: Seq[Any]): Any = { - children.size match { - case 1 => function.asInstanceOf[(Any) => Any](evaluatedChildren(0)) - case 2 => function.asInstanceOf[(Any, Any) => Any](evaluatedChildren(0), evaluatedChildren(1)) - } - } -} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala deleted file mode 100644 index c1c10cc3207a4..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ /dev/null @@ -1,17 +0,0 @@ -package org.apache.spark.sql -package catalyst -package expressions - -abstract sealed class SortDirection -case object Ascending extends SortDirection -case object Descending extends SortDirection - -/** - * An expression that can be used to sort a tuple. This class extends expression primarily so that - * transformations over expression will descend into its child. - */ -case class SortOrder(child: Expression, direction: SortDirection) extends UnaryExpression { - def dataType = child.dataType - def nullable = child.nullable - override def toString = s"$child ${if (direction == Ascending) "ASC" else "DESC"}" -} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala deleted file mode 100644 index 4ada044c390f7..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ /dev/null @@ -1,27 +0,0 @@ -package org.apache.spark.sql -package catalyst -package expressions - -import scala.language.dynamics - -import types._ - -case object DynamicType extends DataType - -case class WrapDynamic(children: Seq[Attribute]) extends Expression with ImplementedUdf { - def nullable = false - def references = children.toSet - def dataType = DynamicType - - def evaluate(evaluatedChildren: Seq[Any]): Any = - new DynamicRow(children, evaluatedChildren) -} - -class DynamicRow(val schema: Seq[Attribute], values: Seq[Any]) - extends GenericRow(values) with Dynamic { - - def selectDynamic(attributeName: String): String = { - val ordinal = schema.indexWhere(_.name == attributeName) - values(ordinal).toString - } -} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala deleted file mode 100644 index 843c6cc30f659..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ /dev/null @@ -1,12 +0,0 @@ -package org.apache.spark.sql -package catalyst -package expressions - -import catalyst.types.BooleanType - -case class Like(left: Expression, right: Expression) extends BinaryExpression { - def dataType = BooleanType - def nullable = left.nullable // Right cannot be null. - def symbol = "LIKE" -} - diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala deleted file mode 100644 index 22a15c6bd51e3..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala +++ /dev/null @@ -1,7 +0,0 @@ -package org.apache.spark.sql -package catalyst - -/** - * Contains classes for enumerating possible physical plans for a given logical query plan. - */ -package object planning diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala deleted file mode 100644 index 3c7b40d197f53..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala +++ /dev/null @@ -1,10 +0,0 @@ -package org.apache.spark.sql -package catalyst -package plans -package logical - -abstract class BaseRelation extends LeafNode { - self: Product => - - def tableName: String -} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala deleted file mode 100644 index 0d0b86ae16171..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ /dev/null @@ -1,21 +0,0 @@ -package org.apache.spark.sql -package catalyst -package plans -package logical - -import expressions._ - -/** - * Transforms the input by forking and running the specified script. - * - * @param input the set of expression that should be passed to the script. - * @param script the command that should be executed. - * @param output the attributes that are produced by the script. - */ -case class ScriptTransformation( - input: Seq[Expression], - script: String, - output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode { - def references = input.flatMap(_.references).toSet -} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala deleted file mode 100644 index 0f07c748e430b..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ /dev/null @@ -1,29 +0,0 @@ -package org.apache.spark.sql -package catalyst -package plans -package logical - -import expressions._ - -/** - * Performs a physical redistribution of the data. Used when the consumer of the query - * result have expectations about the distribution and ordering of partitioned input data. - */ -abstract class RedistributeData extends UnaryNode { - self: Product => - - def output = child.output -} - -case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan) - extends RedistributeData { - - def references = sortExpressions.flatMap(_.references).toSet -} - -case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan) - extends RedistributeData { - - def references = partitionExpressions.flatMap(_.references).toSet -} - diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala deleted file mode 100644 index e483182cdf16b..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala +++ /dev/null @@ -1,8 +0,0 @@ -package org.apache.spark.sql -package catalyst - -/** - * A a collection of common abstractions for query plans as well as - * a base logical plan representation. - */ -package object plans diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala deleted file mode 100644 index 828be70adf6b0..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala +++ /dev/null @@ -1,9 +0,0 @@ -package org.apache.spark.sql -package catalyst -package plans - -sealed abstract class JoinType -case object Inner extends JoinType -case object LeftOuter extends JoinType -case object RightOuter extends JoinType -case object FullOuter extends JoinType diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala deleted file mode 100644 index 79eccd6839c58..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala +++ /dev/null @@ -1,16 +0,0 @@ -package org.apache.spark.sql -package catalyst -package rules - -import trees._ - -abstract class Rule[TreeType <: TreeNode[_]] extends Logging { - - /** Name for this rule, automatically inferred based on class name. */ - val ruleName: String = { - val className = getClass.getName - if (className endsWith "$") className.dropRight(1) else className - } - - def apply(plan: TreeType): TreeType -} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala deleted file mode 100644 index 18d4e0eae122d..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala +++ /dev/null @@ -1,7 +0,0 @@ -package org.apache.spark.sql -package catalyst - -/** - * A framework for applying batches rewrite rules to trees, possibly to fixed point. - */ -package object rules diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala deleted file mode 100644 index 6988ca0aa54bb..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala +++ /dev/null @@ -1,7 +0,0 @@ -package org.apache.spark.sql -package catalyst -/** - * Contains a type system for attributes produced by relations, including complex types like - * structs, arrays and maps. - */ -package object types diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/package.scala b/schema/catalyst/src/main/scala/org/apache/spark/sql/package.scala deleted file mode 100644 index 29eb33b9e6065..0000000000000 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/package.scala +++ /dev/null @@ -1,17 +0,0 @@ -package org.apache.spark - -/** - * Allows the execution of relational queries, including those expressed in SQL using Spark. - * - * Note that this package is located in catalyst instead of in core so that all subprojects can - * inherit the settings from this package object. - */ -package object sql { - - protected[sql] def Logger(name: String) = - com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger(name)) - - protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging - - type Row = catalyst.expressions.Row -} diff --git a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala b/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala deleted file mode 100644 index fb35e27fbbfd2..0000000000000 --- a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala +++ /dev/null @@ -1,23 +0,0 @@ -package org.apache.spark.sql -package catalyst -package analysis - -import org.scalatest.FunSuite - -import analysis._ -import expressions._ -import plans.logical._ -import types._ - -import dsl._ - -class AnalysisSuite extends FunSuite { - val analyze = SimpleAnalyzer - - val testRelation = LocalRelation('a.int) - - test("analyze project") { - assert(analyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === Project(testRelation.output, testRelation)) - - } -} \ No newline at end of file diff --git a/schema/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala b/schema/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala deleted file mode 100644 index c21ab68cd814e..0000000000000 --- a/schema/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala +++ /dev/null @@ -1,4 +0,0 @@ -package org.apache.spark.sql -package execution - -class QueryExecutionException(message: String) extends Exception(message) diff --git a/schema/core/src/main/scala/org/apache/spark/sql/execution/debug.scala b/schema/core/src/main/scala/org/apache/spark/sql/execution/debug.scala deleted file mode 100644 index 6c9ea87eb5386..0000000000000 --- a/schema/core/src/main/scala/org/apache/spark/sql/execution/debug.scala +++ /dev/null @@ -1,29 +0,0 @@ -package org.apache.spark.sql -package execution - -object DebugQuery { - def apply(plan: SparkPlan): SparkPlan = { - val visited = new collection.mutable.HashSet[Long]() - plan transform { - case s: SparkPlan if !visited.contains(s.id) => - visited += s.id - DebugNode(s) - } - } -} - -case class DebugNode(child: SparkPlan) extends UnaryNode { - def references = Set.empty - def output = child.output - def execute() = { - val childRdd = child.execute() - println( - s""" - |========================= - |${child.simpleString} - |========================= - """.stripMargin) - childRdd.foreach(println(_)) - childRdd - } -} diff --git a/schema/core/src/main/scala/org/apache/spark/sql/execution/package.scala b/schema/core/src/main/scala/org/apache/spark/sql/execution/package.scala deleted file mode 100644 index 9097ec1c318d0..0000000000000 --- a/schema/core/src/main/scala/org/apache/spark/sql/execution/package.scala +++ /dev/null @@ -1,7 +0,0 @@ -package org.apache.spark.sql - -/** - * An execution engine for relational query plans that runs on top Spark and returns RDDs. - */ -package object execution { -} diff --git a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/ConcurrentHiveSuite.scala b/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/ConcurrentHiveSuite.scala deleted file mode 100644 index 3b8b9a9d577f4..0000000000000 --- a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/ConcurrentHiveSuite.scala +++ /dev/null @@ -1,21 +0,0 @@ -package org.apache.spark -package sql -package shark -package execution - - -import org.scalatest.{FunSuite, BeforeAndAfterAll} - -class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll { - ignore("multiple instances not supported") { - test("Multiple Hive Instances") { - (1 to 10).map { i => - val ts = - new TestSharkContext(new SparkContext("local", s"TestSqlContext$i", new SparkConf())) - ts.executeSql("SHOW TABLES").toRdd.collect() - ts.executeSql("SELECT * FROM src").toRdd.collect() - ts.executeSql("SHOW TABLES").toRdd.collect() - } - } - } -} \ No newline at end of file diff --git a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveSerDeSuite.scala b/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveSerDeSuite.scala deleted file mode 100644 index 59f7ec314e909..0000000000000 --- a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveSerDeSuite.scala +++ /dev/null @@ -1,16 +0,0 @@ -package org.apache.spark.sql -package shark -package execution - -/** - * A set of tests that validates support for Hive SerDe. - */ -class HiveSerDeSuite extends HiveComparisonTest { - createQueryTest( - "Read and write with LazySimpleSerDe (tab separated)", - "SELECT * from serdeins") - - createQueryTest("Read with RegexSerDe", "SELECT * FROM sales") - - createQueryTest("Read with AvroSerDe", "SELECT * FROM episodes") -} diff --git a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveTypeCoercionSuite.scala b/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveTypeCoercionSuite.scala deleted file mode 100644 index b35fcbb87f833..0000000000000 --- a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveTypeCoercionSuite.scala +++ /dev/null @@ -1,15 +0,0 @@ -package org.apache.spark.sql.shark.execution - -/** - * A set of tests that validate type promotion rules. - */ -class HiveTypeCoercionSuite extends HiveComparisonTest { - - val baseTypes = Seq("1", "1.0", "1L", "1S", "1Y", "'1'") - - baseTypes.foreach { i => - baseTypes.foreach { j => - createQueryTest(s"$i + $j", s"SELECT $i + $j FROM src LIMIT 1") - } - } -} \ No newline at end of file diff --git a/schema/README.md b/sql/README.md similarity index 100% rename from schema/README.md rename to sql/README.md diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala similarity index 88% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 185a62ab617b7..802fba59e8ba0 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package analysis diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala similarity index 71% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index adedd1567e1f2..e6e55a61d3700 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package analysis diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala new file mode 100644 index 0000000000000..a359eb541151d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -0,0 +1,37 @@ +/* + * 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 +package catalyst +package analysis + +import expressions._ + +/** A catalog for looking up user defined functions, used by an [[Analyzer]]. */ +trait FunctionRegistry { + def lookupFunction(name: String, children: Seq[Expression]): Expression +} + +/** + * A trivial catalog that returns an error when a function is requested. Used for testing when all + * functions are already filled in and the analyser needs only to resolve attribute references. + */ +object EmptyFunctionRegistry extends FunctionRegistry { + def lookupFunction(name: String, children: Seq[Expression]): Expression = { + throw new UnsupportedOperationException + } +} diff --git a/schema/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 similarity index 93% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index cf46ca3a95a2a..a0105cd7cf4af 100644 --- a/schema/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 @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package analysis diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala new file mode 100644 index 0000000000000..375c99f48ea84 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -0,0 +1,25 @@ +/* + * 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 catalyst + +/** + * Provides a logical query plan [[Analyzer]] and supporting classes for performing analysis. + * Analysis consists of translating [[UnresolvedAttribute]]s and [[UnresolvedRelation]]s + * into fully typed objects using information in a schema [[Catalog]]. + */ +package object analysis diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala similarity index 81% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 71b0cc90a2556..2ed2af1352a16 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package analysis diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala similarity index 85% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 1d7b7e61db5f6..93227c255f9e8 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala similarity index 63% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala index a98cebb7fb4fa..c253587f67c4e 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala similarity index 72% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 60a4cc0882c45..a7199ccb7b447 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala new file mode 100644 index 0000000000000..ba77693a88da3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -0,0 +1,29 @@ +/* + * 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 +package catalyst +package expressions + +import types.DataType + +/** Cast the child expression to the target data type. */ +case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { + override def foldable = child.foldable + def nullable = child.nullable + override def toString = s"CAST($child, $dataType)" +} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala similarity index 92% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala index e9948c3ae88e2..b58f06d20f9a8 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala similarity index 73% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index d561464160af5..0655e280b31c1 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala new file mode 100644 index 0000000000000..f11885ab3f7af --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala @@ -0,0 +1,27 @@ +/* + * 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 +package catalyst +package expressions + +/** + * A UDF that has a native JVM implementation. + */ +trait ImplementedUdf { + def evaluate(evaluatedChildren: Seq[Any]): Any +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala new file mode 100644 index 0000000000000..a5d0ecf96410b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala @@ -0,0 +1,29 @@ +/* + * 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 +package catalyst +package expressions + +import types.DoubleType + +case object Rand extends LeafExpression { + def dataType = DoubleType + def nullable = false + def references = Set.empty + override def toString = "RAND()" +} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala similarity index 82% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 1421e17d73371..d5c8785f6e884 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala new file mode 100644 index 0000000000000..6a6fef41973bb --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -0,0 +1,36 @@ +/* + * 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 +package catalyst +package expressions + +import types._ + +case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) + extends Expression with ImplementedUdf { + + def references = children.flatMap(_.references).toSet + def nullable = true + + def evaluate(evaluatedChildren: Seq[Any]): Any = { + children.size match { + case 1 => function.asInstanceOf[(Any) => Any](evaluatedChildren(0)) + case 2 => function.asInstanceOf[(Any, Any) => Any](evaluatedChildren(0), evaluatedChildren(1)) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala new file mode 100644 index 0000000000000..171997b90ecf4 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -0,0 +1,34 @@ +/* + * 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 +package catalyst +package expressions + +abstract sealed class SortDirection +case object Ascending extends SortDirection +case object Descending extends SortDirection + +/** + * An expression that can be used to sort a tuple. This class extends expression primarily so that + * transformations over expression will descend into its child. + */ +case class SortOrder(child: Expression, direction: SortDirection) extends UnaryExpression { + def dataType = child.dataType + def nullable = child.nullable + override def toString = s"$child ${if (direction == Ascending) "ASC" else "DESC"}" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala new file mode 100644 index 0000000000000..c87d3783a37ca --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -0,0 +1,44 @@ +/* + * 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 +package catalyst +package expressions + +import scala.language.dynamics + +import types._ + +case object DynamicType extends DataType + +case class WrapDynamic(children: Seq[Attribute]) extends Expression with ImplementedUdf { + def nullable = false + def references = children.toSet + def dataType = DynamicType + + def evaluate(evaluatedChildren: Seq[Any]): Any = + new DynamicRow(children, evaluatedChildren) +} + +class DynamicRow(val schema: Seq[Attribute], values: Seq[Any]) + extends GenericRow(values) with Dynamic { + + def selectDynamic(attributeName: String): String = { + val ordinal = schema.indexWhere(_.name == attributeName) + values(ordinal).toString + } +} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala similarity index 90% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 1708fed744849..8052d426cb4e4 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala similarity index 61% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 8c93bc5fafc7e..38b52ccc7cdcd 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/schema/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 similarity index 64% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 57a84f5ac7737..3bd2ffa1087b9 100644 --- a/schema/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 @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala similarity index 81% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 96971d5b26373..06524bd0600d4 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala similarity index 54% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 3c70d1328b1ac..cc78513e78fd1 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala similarity index 83% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 3886926882ee8..923108ce44906 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala similarity index 52% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index caec559b31bc4..757bbd40ac40f 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala similarity index 72% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index 3ea2b7ed85ea3..76554e160b1d1 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala similarity index 79% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index cc69667e6b766..60c22bfa97c79 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala new file mode 100644 index 0000000000000..6e585236b1b20 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -0,0 +1,29 @@ +/* + * 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 +package catalyst +package expressions + +import catalyst.types.BooleanType + +case class Like(left: Expression, right: Expression) extends BinaryExpression { + def dataType = BooleanType + def nullable = left.nullable // Right cannot be null. + def symbol = "LIKE" +} + diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala similarity index 87% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 8f4a6f51be814..3588a9d87a728 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package optimizer diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala similarity index 69% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index fc319349c9da8..22f8ea005baf1 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package planning diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala new file mode 100644 index 0000000000000..64370ec7c0cc4 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala @@ -0,0 +1,24 @@ +/* + * 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 +package catalyst + +/** + * Contains classes for enumerating possible physical plans for a given logical query plan. + */ +package object planning diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala similarity index 82% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index d5be8687dd24e..613b028ca8409 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package planning diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala similarity index 81% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index d12b05c0ac3df..20f230c5c4f2c 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package plans diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala new file mode 100644 index 0000000000000..e1bcad7bf28a5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala @@ -0,0 +1,27 @@ +/* + * 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 +package catalyst +package plans +package logical + +abstract class BaseRelation extends LeafNode { + self: Product => + + def tableName: String +} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala similarity index 83% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index dfb2db44f8843..bc7b6871df524 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package plans diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala new file mode 100644 index 0000000000000..1a1a2b9b88bd0 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -0,0 +1,38 @@ +/* + * 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 +package catalyst +package plans +package logical + +import expressions._ + +/** + * Transforms the input by forking and running the specified script. + * + * @param input the set of expression that should be passed to the script. + * @param script the command that should be executed. + * @param output the attributes that are produced by the script. + */ +case class ScriptTransformation( + input: Seq[Expression], + script: String, + output: Seq[Attribute], + child: LogicalPlan) extends UnaryNode { + def references = input.flatMap(_.references).toSet +} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala similarity index 63% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala index 1e1ffa584b9f8..fe069ff5a4273 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package plans diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala similarity index 83% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 3b8d62b4475e7..660210368847c 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package plans diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala new file mode 100644 index 0000000000000..f7fcdc5fdb950 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -0,0 +1,46 @@ +/* + * 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 +package catalyst +package plans +package logical + +import expressions._ + +/** + * Performs a physical redistribution of the data. Used when the consumer of the query + * result have expectations about the distribution and ordering of partitioned input data. + */ +abstract class RedistributeData extends UnaryNode { + self: Product => + + def output = child.output +} + +case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan) + extends RedistributeData { + + def references = sortExpressions.flatMap(_.references).toSet +} + +case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan) + extends RedistributeData { + + def references = partitionExpressions.flatMap(_.references).toSet +} + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala new file mode 100644 index 0000000000000..a40ab4bbb154b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala @@ -0,0 +1,25 @@ +/* + * 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 +package catalyst + +/** + * A a collection of common abstractions for query plans as well as + * a base logical plan representation. + */ +package object plans diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala similarity index 89% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 7f48a66b55175..2d8f3ad33517d 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package plans diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala new file mode 100644 index 0000000000000..9f2283ad4346e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala @@ -0,0 +1,26 @@ +/* + * 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 +package catalyst +package plans + +sealed abstract class JoinType +case object Inner extends JoinType +case object LeftOuter extends JoinType +case object RightOuter extends JoinType +case object FullOuter extends JoinType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala new file mode 100644 index 0000000000000..6ff4891a3f8a6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala @@ -0,0 +1,33 @@ +/* + * 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 +package catalyst +package rules + +import trees._ + +abstract class Rule[TreeType <: TreeNode[_]] extends Logging { + + /** Name for this rule, automatically inferred based on class name. */ + val ruleName: String = { + val className = getClass.getName + if (className endsWith "$") className.dropRight(1) else className + } + + def apply(plan: TreeType): TreeType +} diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala similarity index 71% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 886d8bd386b82..68ae30cde1c57 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package rules diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala new file mode 100644 index 0000000000000..26ab5430821f4 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala @@ -0,0 +1,24 @@ +/* + * 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 +package catalyst + +/** + * A framework for applying batches rewrite rules to trees, possibly to fixed point. + */ +package object rules diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala similarity index 93% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index a7edd3225b29d..76ede87e4e6c0 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package trees diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala similarity index 53% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index cc7cd41833ec5..e2da1d2439bbf 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala similarity index 82% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index bd8dd6db943e1..6eb2b62eccb48 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package types diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala new file mode 100644 index 0000000000000..b65a5617d9a7f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala @@ -0,0 +1,24 @@ +/* + * 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 +package catalyst +/** + * Contains a type system for attributes produced by relations, including complex types like + * structs, arrays and maps. + */ +package object types diff --git a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala similarity index 77% rename from schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 991da35920da4..b4ec176966826 100644 --- a/schema/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala new file mode 100644 index 0000000000000..517e792479bc4 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala @@ -0,0 +1,34 @@ +/* + * 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 + +/** + * Allows the execution of relational queries, including those expressed in SQL using Spark. + * + * Note that this package is located in catalyst instead of in core so that all subprojects can + * inherit the settings from this package object. + */ +package object sql { + + protected[sql] def Logger(name: String) = + com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger(name)) + + protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging + + type Row = catalyst.expressions.Row +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala new file mode 100644 index 0000000000000..39f22d2e37bf2 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala @@ -0,0 +1,40 @@ +/* + * 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 +package catalyst +package analysis + +import org.scalatest.FunSuite + +import analysis._ +import expressions._ +import plans.logical._ +import types._ + +import dsl._ + +class AnalysisSuite extends FunSuite { + val analyze = SimpleAnalyzer + + val testRelation = LocalRelation('a.int) + + test("analyze project") { + assert(analyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === Project(testRelation.output, testRelation)) + + } +} \ No newline at end of file diff --git a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala similarity index 84% rename from schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 17bf4e6b974da..660999f4c7ca9 100644 --- a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -1,7 +1,23 @@ +/* + * 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 package catalyst - import org.scalatest.FunSuite import plans.physical._ diff --git a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala similarity index 76% rename from schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala index e8dd9178b23a8..d5cc0724f5112 100644 --- a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package expressions diff --git a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala similarity index 72% rename from schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala index 1c1768233107c..f595bf7e44af7 100644 --- a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package analysis diff --git a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala similarity index 56% rename from schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala index c6776f26df714..ff7c15b7187b7 100644 --- a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package trees diff --git a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala similarity index 70% rename from schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala index 2f941be1becb1..98bb090c29d11 100644 --- a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package trees diff --git a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala similarity index 85% rename from schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index df3d329c5abe2..6c7f067b516c0 100644 --- a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 package catalyst package optimizer diff --git a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala similarity index 100% rename from schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala diff --git a/schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala similarity index 100% rename from schema/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala diff --git a/schema/core/src/main/scala/org/apache/spark/rdd/SharkPairRddFunctions.scala b/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala similarity index 67% rename from schema/core/src/main/scala/org/apache/spark/rdd/SharkPairRddFunctions.scala rename to sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala index 6e65d7dc745cf..4464356ff4649 100644 --- a/schema/core/src/main/scala/org/apache/spark/rdd/SharkPairRddFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import scala.language.implicitConversions @@ -11,12 +28,10 @@ import org.apache.spark.SparkContext._ import org.apache.spark.util.collection.AppendOnlyMap /** - * Extra functions for Shark available on RDDs of (key, value) pairs through - * an implicit conversion. - * Import `org.apache.spark.SharkPairRDDFunctions._` at the top of your program to - * use these functions. + * Extra functions on RDDs that perform only local operations. These can be used when data has + * already been partitioned correctly. */ -class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) +protected[spark] class PartitionLocalRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Logging with Serializable { @@ -56,7 +71,7 @@ class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def groupByKeyLocally(): RDD[(K, Seq[V])] = { def createCombiner(v: V) = ArrayBuffer(v) def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v - val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner _, mergeValue _, _ ++ _) + val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner, mergeValue, _ ++ _) val bufs = self.mapPartitionsWithContext((context, iter) => { new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) }, preservesPartitioning = true) @@ -76,9 +91,9 @@ class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } } -object SharkPairRDDFunctions { - implicit def rddToSharkPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = - new SharkPairRDDFunctions(rdd) +object PartitionLocalRDDFunctions { + implicit def rddToPartitionLocalRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = + new PartitionLocalRDDFunctions(rdd) } diff --git a/schema/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala similarity index 82% rename from schema/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala rename to sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala index fa7422ceefad5..f25de3827855c 100644 --- a/schema/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala @@ -1,3 +1,20 @@ +/* + * 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 import scala.language.implicitConversions diff --git a/schema/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala b/sql/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala similarity index 70% rename from schema/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala rename to sql/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala index 28d24dde00001..9308737146523 100644 --- a/schema/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala @@ -1,3 +1,20 @@ +/* + * 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 package examples diff --git a/schema/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala b/sql/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala similarity index 67% rename from schema/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala rename to sql/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala index eea5f92c4c69b..0b6de528fb060 100644 --- a/schema/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala @@ -1,3 +1,20 @@ +/* + * 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 package examples @@ -8,7 +25,7 @@ import catalyst.plans.logical._ /* Implicit Conversions */ import catalyst.dsl._ -import TestSqlContext._ // For .toRdd execution using locally running test Shark. +import TestSqlContext._ // For .toRdd execution using locally running sql instance. object ViewsExample { def main(args: Array[String]): Unit = { diff --git a/schema/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala similarity index 85% rename from schema/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 12f1696f64a3a..488e9d2a3e503 100644 --- a/schema/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -1,3 +1,20 @@ +/* + * 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 package execution diff --git a/schema/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala similarity index 64% rename from schema/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index ce4207e8b33c6..0c7ab8771a6a8 100644 --- a/schema/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -1,3 +1,20 @@ +/* + * 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 package execution diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala new file mode 100644 index 0000000000000..7ce8608d2061d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala @@ -0,0 +1,21 @@ +/* + * 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 +package execution + +class QueryExecutionException(message: String) extends Exception(message) diff --git a/schema/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala similarity index 62% rename from schema/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala index 700719b6de98e..85facee367d55 100644 --- a/schema/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala @@ -1,3 +1,20 @@ +/* + * 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 package execution diff --git a/schema/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala similarity index 91% rename from schema/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 854bec6591974..d7cbe1c9e2be5 100644 --- a/schema/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -1,3 +1,20 @@ +/* + * 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 package execution diff --git a/schema/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala similarity index 80% rename from schema/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala index 6ac9b42d90164..f94eb6801d230 100644 --- a/schema/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala @@ -1,3 +1,20 @@ +/* + * 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 package execution @@ -8,7 +25,7 @@ import catalyst.expressions._ import catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples} import catalyst.types._ -import org.apache.spark.rdd.SharkPairRDDFunctions._ +import org.apache.spark.rdd.PartitionLocalRDDFunctions._ /** * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each diff --git a/schema/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala similarity index 78% rename from schema/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 377742925c737..95278f646b7c0 100644 --- a/schema/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -1,3 +1,20 @@ +/* + * 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 package execution diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala new file mode 100644 index 0000000000000..db259b4c4b95b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala @@ -0,0 +1,46 @@ +/* + * 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 +package execution + +object DebugQuery { + def apply(plan: SparkPlan): SparkPlan = { + val visited = new collection.mutable.HashSet[Long]() + plan transform { + case s: SparkPlan if !visited.contains(s.id) => + visited += s.id + DebugNode(s) + } + } +} + +case class DebugNode(child: SparkPlan) extends UnaryNode { + def references = Set.empty + def output = child.output + def execute() = { + val childRdd = child.execute() + println( + s""" + |========================= + |${child.simpleString} + |========================= + """.stripMargin) + childRdd.foreach(println(_)) + childRdd + } +} diff --git a/schema/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala similarity index 83% rename from schema/core/src/main/scala/org/apache/spark/sql/execution/joins.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 885880a84762e..bb7edd6fb3c74 100644 --- a/schema/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -1,3 +1,20 @@ +/* + * 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 package execution @@ -11,7 +28,7 @@ import catalyst.expressions._ import catalyst.plans._ import catalyst.plans.physical.{ClusteredDistribution, Partitioning} -import org.apache.spark.rdd.SharkPairRDDFunctions._ +import org.apache.spark.rdd.PartitionLocalRDDFunctions._ case class SparkEquiInnerJoin( leftKeys: Seq[Expression], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala new file mode 100644 index 0000000000000..861dd25e5dca4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala @@ -0,0 +1,24 @@ +/* + * 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 + +/** + * An execution engine for relational query plans that runs on top Spark and returns RDDs. + */ +package object execution { +} diff --git a/schema/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties similarity index 100% rename from schema/core/src/test/resources/log4j.properties rename to sql/core/src/test/resources/log4j.properties diff --git a/schema/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala similarity index 87% rename from schema/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala rename to sql/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala index 559a8e09880b1..86b117ffd815e 100644 --- a/schema/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -70,7 +87,7 @@ class DslQueryTest extends FunSuite { val isSorted = plan.collect { case s: logical.Sort => s}.nonEmpty def prepareAnswer(answer: Seq[Any]) = if (!isSorted) answer.sortBy(_.toString) else answer - val sharkAnswer = try plan.toRdd.collect().toSeq catch { + val sparkAnswer = try plan.toRdd.collect().toSeq catch { case e: Exception => fail( s""" @@ -82,7 +99,7 @@ class DslQueryTest extends FunSuite { |$e """.stripMargin) } - assert(prepareAnswer(convertedAnswer) === prepareAnswer(sharkAnswer)) + assert(prepareAnswer(convertedAnswer) === prepareAnswer(sparkAnswer)) } } diff --git a/schema/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala similarity index 62% rename from schema/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala index c5287659b3de0..670519a916c4e 100644 --- a/schema/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 package execution diff --git a/schema/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala similarity index 67% rename from schema/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala index f4850f97078e8..65cb515780f90 100644 --- a/schema/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 package execution diff --git a/schema/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala similarity index 85% rename from schema/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala rename to sql/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala index 4da34815d7f36..16e4a4015d359 100644 --- a/schema/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala +++ b/sql/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala @@ -1,3 +1,20 @@ +/* + * 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.hadoop.mapred import java.io.IOException @@ -19,7 +36,7 @@ import org.apache.hadoop.hive.ql.plan.FileSinkDesc * It is based on [[SparkHadoopWriter]]. */ protected[apache] -class SharkHadoopWriter( +class SparkHiveHadoopWriter( @transient jobConf: JobConf, fileSinkConf: FileSinkDesc) extends Logging @@ -166,7 +183,7 @@ class SharkHadoopWriter( } } -object SharkHadoopWriter { +object SparkHiveHadoopWriter { def createPathFromString(path: String, conf: JobConf): Path = { if (path == null) { throw new IllegalArgumentException("Output path is null") diff --git a/schema/hive/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala similarity index 83% rename from schema/hive/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 2035f0db4c2d2..3e790ce940ba2 100644 --- a/schema/hive/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive import java.io.{PrintStream, InputStreamReader, BufferedReader, File} import java.util.{ArrayList => JArrayList} @@ -23,23 +40,21 @@ import org.apache.spark.sql.execution._ import scala.collection.JavaConversions._ /** - * Starts up an instance of shark where metadata is stored locally. An in-process metadata data is + * Starts up an instance of hive where metadata is stored locally. An in-process metadata data is * created with data stored in ./metadata. Warehouse data is stored in in ./warehouse. */ -class LocalSharkContext( +class LocalHiveContext( sc: SparkContext, override val warehousePath: String = new File("warehouse").getCanonicalPath) - extends SharkContext(sc) { + extends HiveContext(sc) { override def metastorePath = new File("metastore").getCanonicalPath } /** - * An instance of the shark execution engine. This class is responsible for taking queries - * expressed either in SQL or as raw catalyst logical plans and optimizing them for execution - * using Spark. Additionally this class maintains the connection with the hive metadata store. + * An instance of the Spark SQL execution engine that integrates with data stored in Hive. */ -abstract class SharkContext(sc: SparkContext) extends SparkSqlContext(sc) { +abstract class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { self => /** The path to the hive warehouse. */ @@ -60,8 +75,7 @@ abstract class SharkContext(sc: SparkContext) extends SparkSqlContext(sc) { } // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. - // TODO: Move, or just use a properly configured logger. - val outputBuffer = new java.io.OutputStream { + protected val outputBuffer = new java.io.OutputStream { var pos: Int = 0 var buffer = new Array[Int](10240) def write(i: Int): Unit = { @@ -88,18 +102,18 @@ abstract class SharkContext(sc: SparkContext) extends SparkSqlContext(sc) { } } - @transient lazy val hiveconf = new HiveConf(classOf[SessionState]) - @transient val sessionState = new SessionState(hiveconf) + @transient protected[hive] lazy val hiveconf = new HiveConf(classOf[SessionState]) + @transient protected[hive] val sessionState = new SessionState(hiveconf) sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") configure() // Must be called before initializing the catalog below. - /* A catalyst metadata catalog that points to the Shark/Hive Metastore. */ + /* A catalyst metadata catalog that points to the Hive Metastore. */ override val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog - /* An analyzer that uses the Shark/Hive metastore. */ + /* An analyzer that uses the Hive metastore. */ override val analyzer = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) def tables: Seq[BaseRelation] = { @@ -111,7 +125,7 @@ abstract class SharkContext(sc: SparkContext) extends SparkSqlContext(sc) { /** * Runs the specified SQL query using Hive. */ - def runSqlHive(sql: String): Seq[String] = { + protected def runSqlHive(sql: String): Seq[String] = { val maxResults = 100000 val results = runHive(sql, 100000) // It is very confusing when you only get back some of the results... @@ -127,7 +141,7 @@ abstract class SharkContext(sc: SparkContext) extends SparkSqlContext(sc) { * Execute the command using Hive and return the results as a sequence. Each element * in the sequence is one row. */ - def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { + protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { try { val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") @@ -171,8 +185,8 @@ abstract class SharkContext(sc: SparkContext) extends SparkSqlContext(sc) { } } - object SharkPlanner extends SparkPlanner with SharkStrategies { - val sharkContext = self + object HivePlanner extends SparkPlanner with HiveStrategies { + val hiveContext = self override val strategies: Seq[Strategy] = Seq( TopK, @@ -189,11 +203,11 @@ abstract class SharkContext(sc: SparkContext) extends SparkSqlContext(sc) { ) } - override val planner = SharkPlanner + override val planner = HivePlanner protected lazy val emptyResult = sparkContext.parallelize(Seq(new GenericRow(Vector()): Row), 1) - /** Extends QueryExecution with shark specific features. */ + /** Extends QueryExecution with hive specific features. */ abstract class QueryExecution extends super.QueryExecution { // TODO: Create mixin for the analyzer instead of overriding things here. override lazy val optimizedPlan = diff --git a/schema/hive/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala similarity index 88% rename from schema/hive/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index c67e4ecbb4d84..d63dd5cf87326 100644 --- a/schema/hive/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive import scala.util.parsing.combinator.RegexParsers @@ -19,16 +36,16 @@ import catalyst.types._ import scala.collection.JavaConversions._ -class HiveMetastoreCatalog(shark: SharkContext) extends Catalog with Logging { +class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { import HiveMetastoreTypes._ - val client = Hive.get(shark.hiveconf) + val client = Hive.get(hive.hiveconf) def lookupRelation( db: Option[String], tableName: String, alias: Option[String]): LogicalPlan = { - val databaseName = db.getOrElse(shark.sessionState.getCurrentDatabase()) + val databaseName = db.getOrElse(hive.sessionState.getCurrentDatabase()) val table = client.getTable(databaseName, tableName) val partitions: Seq[Partition] = if (table.isPartitioned) { diff --git a/schema/hive/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala similarity index 97% rename from schema/hive/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index d0508fca06b01..1998045c29926 100644 --- a/schema/hive/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive import scala.collection.JavaConversions._ diff --git a/schema/hive/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala similarity index 80% rename from schema/hive/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 8ec289e0f6e72..ffdc13a9f7010 100644 --- a/schema/hive/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive import catalyst.expressions._ import catalyst.planning._ @@ -8,16 +25,16 @@ import catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ -trait SharkStrategies { +trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. self: SparkSqlContext#SparkPlanner => - val sharkContext: SharkContext + val hiveContext: HiveContext object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ScriptTransformation(input, script, output, child) => - ScriptTransformation(input, script, output, planLater(child))(sharkContext) :: Nil + ScriptTransformation(input, script, output, planLater(child))(hiveContext) :: Nil case _ => Nil } } @@ -25,7 +42,7 @@ trait SharkStrategies { object DataSinks extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => - InsertIntoHiveTable(table, partition, planLater(child), overwrite)(sharkContext) :: Nil + InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case _ => Nil } } @@ -34,9 +51,9 @@ trait SharkStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { // Push attributes into table scan when possible. case p @ logical.Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => - HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m, None)(sharkContext) :: Nil + HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m, None)(hiveContext) :: Nil case m: MetastoreRelation => - HiveTableScan(m.output, m, None)(sharkContext) :: Nil + HiveTableScan(m.output, m, None)(hiveContext) :: Nil case _ => Nil } } @@ -62,7 +79,7 @@ trait SharkStrategies { } val scan = HiveTableScan( - relation.output, relation, pruningPredicates.reduceLeftOption(And))(sharkContext) + relation.output, relation, pruningPredicates.reduceLeftOption(And))(hiveContext) otherPredicates .reduceLeftOption(And) @@ -100,10 +117,10 @@ trait SharkStrategies { val filteredRelation = predicateOpt.map(logical.Filter(_, relation)).getOrElse(relation) PartitionPrunings(filteredRelation).view.map(_.transform { case scan: HiveTableScan => - scan.copy(attributes = prunedCols)(sharkContext) + scan.copy(attributes = prunedCols)(hiveContext) }) } else { - val scan = HiveTableScan(prunedCols, relation, None)(sharkContext) + val scan = HiveTableScan(prunedCols, relation, None)(hiveContext) predicateOpt.map(execution.Filter(_, scan)).getOrElse(scan) :: Nil } diff --git a/schema/hive/src/main/scala/org/apache/spark/sql/shark/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala similarity index 67% rename from schema/hive/src/main/scala/org/apache/spark/sql/shark/ScriptTransformation.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala index 6906588f19272..7c128246f72c3 100644 --- a/schema/hive/src/main/scala/org/apache/spark/sql/shark/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive import java.io.{InputStreamReader, BufferedReader} @@ -19,7 +36,7 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: SparkPlan)(@transient sc: SharkContext) + child: SparkPlan)(@transient sc: HiveContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil diff --git a/schema/hive/src/main/scala/org/apache/spark/sql/shark/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala similarity index 89% rename from schema/hive/src/main/scala/org/apache/spark/sql/shark/TableReader.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index ca56b65bc7c9d..71d751cbc424d 100644 --- a/schema/hive/src/main/scala/org/apache/spark/sql/shark/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ @@ -17,11 +34,9 @@ import org.apache.spark.rdd.{HadoopRDD, UnionRDD, EmptyRDD, RDD} /** - * A trait for subclasses that handle table scans. In Shark, there is one subclass for each - * type of table storage: HeapTableReader for Shark tables in Spark's block manager, - * TachyonTableReader for tables in Tachyon, and HadoopTableReader for Hive tables in a filesystem. + * A trait for subclasses that handle table scans. */ -private[shark] sealed trait TableReader { +private[hive] sealed trait TableReader { def makeRDDForTable(hiveTable: HiveTable): RDD[_] def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] @@ -33,8 +48,8 @@ private[shark] sealed trait TableReader { * Helper class for scanning tables stored in Hadoop - e.g., to read Hive tables that reside in the * data warehouse directory. */ -private[shark] -class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: SharkContext) +private[hive] +class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveContext) extends TableReader { // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless @@ -211,7 +226,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: SharkCo } -private[shark] object HadoopTableReader { +private[hive] object HadoopTableReader { /** * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to diff --git a/schema/hive/src/main/scala/org/apache/spark/sql/shark/TestShark.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala similarity index 88% rename from schema/hive/src/main/scala/org/apache/spark/sql/shark/TestShark.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala index 28d97226f9aca..5848127c54114 100644 --- a/schema/hive/src/main/scala/org/apache/spark/sql/shark/TestShark.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive import java.io.File import java.util.{Set => JavaSet} @@ -23,38 +40,21 @@ import catalyst.analysis._ import catalyst.plans.logical.{LogicalPlan, NativeCommand} import catalyst.util._ -object TestShark - extends TestSharkContext(new SparkContext("local", "TestSqlContext", new SparkConf())) +object TestHive + extends TestHiveContext(new SparkContext("local", "TestSqlContext", new SparkConf())) /** - * A locally running test instance of spark. The lifecycle for a given query is managed by the - * inner class [[SharkQuery]]. A [[SharkQuery]] can either be instantiated directly or using the - * implicit conversion '.q'. - * - * {{{ - * scala> val query = "SELECT key FROM src".q - * query: testShark.SharkQuery = - * SELECT key FROM src - * == Logical Plan == - * Project {key#2} - * MetastoreRelation src - * - * == Physical Plan == - * HiveTableScan {key#2}, MetastoreRelation src - * - * scala> query.execute().get.collect() - * res0: Array[IndexedSeq[Any]] = Array(Vector(238), Vector(86), Vector(311), ... - * }}} + * A locally running test instance of Spark's Hive execution engine. * * Data from [[testTables]] will be automatically loaded whenever a query is run over those tables. * Calling [[reset]] will delete all tables and other state in the database, leaving the database * in a "clean" state. * - * TestShark is singleton object version of this class because instantiating multiple copies of the + * TestHive is singleton object version of this class because instantiating multiple copies of the * hive metastore seems to lead to weird non-deterministic failures. Therefore, the execution of - * testcases that rely on TestShark must be serialized. + * testcases that rely on TestHive must be serialized. */ -class TestSharkContext(sc: SparkContext) extends SharkContext(sc) { +class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { self => // By clearing the port we force Spark to pick a new one. This allows us to rerun tests @@ -62,8 +62,8 @@ class TestSharkContext(sc: SparkContext) extends SharkContext(sc) { System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") - lazy val warehousePath = getTempFilePath("sharkWarehouse").getCanonicalPath - lazy val metastorePath = getTempFilePath("sharkMetastore").getCanonicalPath + lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath + lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") @@ -90,7 +90,9 @@ class TestSharkContext(sc: SparkContext) extends SharkContext(sc) { */ private def rewritePaths(cmd: String): String = if (cmd.toUpperCase contains "LOAD DATA") { - cmd.replaceAll("\\.\\.", TestShark.inRepoTests.getCanonicalPath) + val testDataLocation = + hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) + cmd.replaceAll("\\.\\.", testDataLocation) } else { cmd } @@ -99,7 +101,7 @@ class TestSharkContext(sc: SparkContext) extends SharkContext(sc) { hiveFilesTemp.delete() hiveFilesTemp.mkdir() - val inRepoTests = new File("shark/src/test/hive/") + val inRepoTests = new File("src/test/hive/") def getHiveFile(path: String): File = { val stripped = path.replaceAll("""\.\.\/""", "") hiveDevHome @@ -138,18 +140,6 @@ class TestSharkContext(sc: SparkContext) extends SharkContext(sc) { } } - /* We must repeat the implicits so that we bind to the overridden versions - - implicit class stringToTestQuery(str: String) { - def q = new SharkSqlQuery(str) - } - - implicit override def logicalToSharkQuery(plan: LogicalPlan) = new LogicalSharkQuery { - val parsed = plan - } - - */ - case class TestTable(name: String, commands: (()=>Unit)*) implicit class SqlCmd(sql: String) { diff --git a/schema/hive/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala similarity index 91% rename from schema/hive/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala index 7ef50051126e7..479759aa8184d 100644 --- a/schema/hive/src/main/scala/org/apache/spark/sql/shark/hiveOperators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.metastore.MetaStoreUtils @@ -41,7 +58,7 @@ case class HiveTableScan( attributes: Seq[Attribute], relation: MetastoreRelation, partitionPruningPred: Option[Expression])( - @transient val sc: SharkContext) + @transient val sc: HiveContext) extends LeafNode with HiveInspectors { @@ -112,7 +129,7 @@ case class HiveTableScan( * @param partitions All partitions of the relation. * @return Partitions that are involved in the query plan. */ - private[shark] def prunePartitions(partitions: Seq[HivePartition]) = { + private[hive] def prunePartitions(partitions: Seq[HivePartition]) = { boundPruningPred match { case None => partitions case Some(shouldKeep) => partitions.filter { part => @@ -155,7 +172,7 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], child: SparkPlan, overwrite: Boolean) - (@transient sc: SharkContext) + (@transient sc: HiveContext) extends UnaryNode { val outputClass = newSerializer(table.tableDesc).getSerializedClass @@ -222,11 +239,11 @@ case class InsertIntoHiveTable( conf.setOutputCommitter(classOf[FileOutputCommitter]) FileOutputFormat.setOutputPath( conf, - SharkHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) + SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) logger.debug("Saving as hadoop file of type " + valueClass.getSimpleName) - val writer = new SharkHadoopWriter(conf, fileSinkConf) + val writer = new SparkHiveHadoopWriter(conf, fileSinkConf) writer.preSetup() def writeToFile(context: TaskContext, iter: Iterator[Writable]) { diff --git a/schema/hive/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala similarity index 95% rename from schema/hive/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 28662189e2b78..6375744dfd06a 100644 --- a/schema/hive/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer diff --git a/schema/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties similarity index 100% rename from schema/hive/src/test/resources/log4j.properties rename to sql/hive/src/test/resources/log4j.properties diff --git a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala similarity index 81% rename from schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/BigDataBenchmarkSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 958fff68ad858..4b45e698601eb 100644 --- a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive package execution import java.io.File @@ -9,7 +26,7 @@ import java.io.File * https://amplab.cs.berkeley.edu/benchmark/ */ class BigDataBenchmarkSuite extends HiveComparisonTest { - import TestShark._ + import TestHive._ val testDataDirectory = new File("target/big-data-benchmark-testdata") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala new file mode 100644 index 0000000000000..e9527f0c38bee --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -0,0 +1,38 @@ +/* + * 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 +package sql +package hive +package execution + + +import org.scalatest.{FunSuite, BeforeAndAfterAll} + +class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll { + ignore("multiple instances not supported") { + test("Multiple Hive Instances") { + (1 to 10).map { i => + val ts = + new TestHiveContext(new SparkContext("local", s"TestSqlContext$i", new SparkConf())) + ts.executeSql("SHOW TABLES").toRdd.collect() + ts.executeSql("SELECT * FROM src").toRdd.collect() + ts.executeSql("SHOW TABLES").toRdd.collect() + } + } + } +} \ No newline at end of file diff --git a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala similarity index 83% rename from schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveComparisonTest.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 1e640d3080cf1..04a6fa56b4fc9 100644 --- a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive package execution import java.io._ @@ -29,14 +46,14 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G * tests. Also prevents the classpath being used when looking for golden answers as these are * usually stale. */ - val recomputeCache = System.getProperty("shark.hive.recomputeCache") != null + val recomputeCache = System.getProperty("spark.hive.recomputeCache") != null protected val shardRegEx = "(\\d+):(\\d+)".r /** * Allows multiple JVMs to be run in parallel, each responsible for portion of all test cases. - * Format `shardId:numShards`. Shard ids should be zero indexed. E.g. -Dshark.hive.testshard=0:4. + * Format `shardId:numShards`. Shard ids should be zero indexed. E.g. -Dspark.hive.testshard=0:4. */ - val shardInfo = Option(System.getProperty("shark.hive.shard")).map { + val shardInfo = Option(System.getProperty("spark.hive.shard")).map { case shardRegEx(id, total) => (id.toInt, total.toInt) } @@ -46,23 +63,23 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G * When set, this comma separated list is defines directories that contain the names of test cases * that should be skipped. * - * For example when `-Dshark.hive.skiptests=passed,hiveFailed` is specified and test cases listed + * For example when `-Dspark.hive.skiptests=passed,hiveFailed` is specified and test cases listed * in [[passedDirectory]] or [[hiveFailedDirectory]] will be skipped. */ val skipDirectories = - Option(System.getProperty("shark.hive.skiptests")) + Option(System.getProperty("spark.hive.skiptests")) .toSeq .flatMap(_.split(",")) .map(name => new File(targetDir, s"$suiteName.$name")) val runOnlyDirectories = - Option(System.getProperty("shark.hive.runonlytests")) + Option(System.getProperty("spark.hive.runonlytests")) .toSeq .flatMap(_.split(",")) .map(name => new File(targetDir, s"$suiteName.$name")) /** The local directory with cached golden answer will be stored. */ - protected val answerCache = new File("shark/src/test/hive/golden") + protected val answerCache = new File("src/test/golden") if (!answerCache.exists) { answerCache.mkdir() } @@ -109,15 +126,15 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G } protected def prepareAnswer( - sharkQuery: TestShark.type#SqlQueryExecution, + hiveQuery: TestHive.type#SqlQueryExecution, answer: Seq[String]): Seq[String] = { - val orderedAnswer = sharkQuery.logical match { + val orderedAnswer = hiveQuery.logical match { // Clean out non-deterministic time schema info. case _: NativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _: ExplainCommand => answer case _ => // TODO: Really we only care about the final total ordering here... - val isOrdered = sharkQuery.executedPlan.collect { + val isOrdered = hiveQuery.executedPlan.collect { case s @ Sort(_, global, _) if global => s }.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. @@ -180,7 +197,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G } test(testCaseName) { - logger.error(s"=== HIVE TEST: $testCaseName ===") + logger.debug(s"=== HIVE TEST: $testCaseName ===") // Clear old output for this testcase. outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) @@ -210,7 +227,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G } try { - TestShark.reset() + TestHive.reset() val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => @@ -238,11 +255,11 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G hiveCachedResults } else { - val sharkQueries = queryList.map(new TestShark.SqlQueryExecution(_)) + val hiveQueries = queryList.map(new TestHive.SqlQueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. - sharkQueries.foreach(_.logical) - val computedResults = (queryList.zipWithIndex, sharkQueries,hiveCacheFiles).zipped.map { - case ((queryString, i), sharkQuery, cachedAnswerFile)=> + hiveQueries.foreach(_.logical) + val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { + case ((queryString, i), hiveQuery, cachedAnswerFile)=> try { // Hooks often break the harness and don't really affect our test anyway, don't // even try running them. @@ -251,9 +268,9 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") // Analyze the query with catalyst to ensure test tables are loaded. - val answer = sharkQuery.analyzed match { + val answer = hiveQuery.analyzed match { case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. - case _ => TestShark.runSqlHive(queryString) + case _ => TestHive.runSqlHive(queryString) } // We need to add a new line to non-empty answers so we can differentiate Seq() @@ -277,14 +294,14 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G fail(errorMessage) } }.toSeq - TestShark.reset() + TestHive.reset() computedResults } // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => - val query = new TestShark.SqlQueryExecution(queryString) + val query = new TestHive.SqlQueryExecution(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { case e: Exception => val errorMessage = @@ -304,11 +321,11 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G }.toSeq (queryList, hiveResults, catalystResults).zipped.foreach { - case (query, hive, (sharkQuery, catalyst)) => + case (query, hive, (hiveQuery, catalyst)) => // Check that the results match unless its an EXPLAIN query. - val preparedHive = prepareAnswer(sharkQuery,hive) + val preparedHive = prepareAnswer(hiveQuery,hive) - if ((!sharkQuery.logical.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) { + if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${hive.size} row(s) ==" +: preparedHive val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst @@ -323,7 +340,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G val errorMessage = s""" |Results do not match for $testCaseName: - |$sharkQuery\n${sharkQuery.analyzed.output.map(_.name).mkString("\t")} + |$hiveQuery\n${hiveQuery.analyzed.output.map(_.name).mkString("\t")} |$resultComparison """.stripMargin @@ -337,12 +354,12 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G } catch { case tf: org.scalatest.exceptions.TestFailedException => throw tf case originalException: Exception => - if (System.getProperty("shark.hive.canarytest") != null) { + if (System.getProperty("spark.hive.canarytest") != null) { // When we encounter an error we check to see if the environment is still okay by running a simple query. // If this fails then we halt testing since something must have gone seriously wrong. try { - new TestShark.SqlQueryExecution("SELECT key FROM src").stringResult() - TestShark.runSqlHive("SELECT key FROM src") + new TestHive.SqlQueryExecution("SELECT key FROM src").stringResult() + TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => logger.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") diff --git a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala similarity index 94% rename from schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 9aa1dccae91b7..30bbbc8b088b7 100644 --- a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive package execution @@ -12,7 +29,7 @@ import util._ */ class HiveCompatibilitySuite extends HiveQueryFileTest { // TODO: bundle in jar files... get from classpath - lazy val hiveQueryDir = TestShark.getHiveFile("ql/src/test/queries/clientpositive") + lazy val hiveQueryDir = TestHive.getHiveFile("ql/src/test/queries/clientpositive") def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) /** A list of tests deemed out of scope currently and thus completely disregarded. */ @@ -123,7 +140,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { // No support for unpublished test udfs. "autogen_colalias", - // Shark does not support buckets. + // Hive does not support buckets. ".*bucket.*", // No window support yet diff --git a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala similarity index 65% rename from schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveQueryFileTest.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index bffa0f80f6efa..f0a4ec3c02fa4 100644 --- a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive package execution import java.io._ @@ -25,11 +42,11 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { def testCases: Seq[(String, File)] val runAll = - !(System.getProperty("shark.hive.alltests") == null) || + !(System.getProperty("spark.hive.alltests") == null) || runOnlyDirectories.nonEmpty || skipDirectories.nonEmpty - val whiteListProperty = "shark.hive.whitelist" + val whiteListProperty = "spark.hive.whitelist" // Allow the whiteList to be overridden by a system property val realWhiteList = Option(System.getProperty(whiteListProperty)).map(_.split(",").toSeq).getOrElse(whiteList) @@ -38,7 +55,7 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { testCases.sorted.foreach { case (testCaseName, testCaseFile) => if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { - logger.warn(s"Blacklisted test skipped $testCaseName") + logger.debug(s"Blacklisted test skipped $testCaseName") } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) diff --git a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala similarity index 82% rename from schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 4b242b0ff2411..28a5d260b3185 100644 --- a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -1,5 +1,22 @@ +/* + * 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 -package shark +package hive package execution @@ -7,7 +24,7 @@ package execution * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ class HiveQuerySuite extends HiveComparisonTest { - import TestShark._ + import TestHive._ createQueryTest("Simple Average", "SELECT AVG(key) FROM src") diff --git a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala similarity index 64% rename from schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveResolutionSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 57b7cab424dde..0dd79faa15396 100644 --- a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -1,12 +1,29 @@ +/* + * 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 -package shark +package hive package execution /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ class HiveResolutionSuite extends HiveComparisonTest { - import TestShark._ + import TestHive._ createQueryTest("table.attr", "SELECT src.key FROM src ORDER BY key LIMIT 1") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala new file mode 100644 index 0000000000000..c2264926f4356 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -0,0 +1,33 @@ +/* + * 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 +package hive +package execution + +/** + * A set of tests that validates support for Hive SerDe. + */ +class HiveSerDeSuite extends HiveComparisonTest { + createQueryTest( + "Read and write with LazySimpleSerDe (tab separated)", + "SELECT * from serdeins") + + createQueryTest("Read with RegexSerDe", "SELECT * FROM sales") + + createQueryTest("Read with AvroSerDe", "SELECT * FROM episodes") +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala new file mode 100644 index 0000000000000..bb33583e5f582 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -0,0 +1,32 @@ +/* + * 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.execution + +/** + * A set of tests that validate type promotion rules. + */ +class HiveTypeCoercionSuite extends HiveComparisonTest { + + val baseTypes = Seq("1", "1.0", "1L", "1S", "1Y", "'1'") + + baseTypes.foreach { i => + baseTypes.foreach { j => + createQueryTest(s"$i + $j", s"SELECT $i + $j FROM src LIMIT 1") + } + } +} \ No newline at end of file diff --git a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala similarity index 83% rename from schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/PruningSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 340dcc0ff7831..8542f42aa9f43 100644 --- a/schema/hive/src/test/scala/org/apache/spark/sql/shark/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -1,10 +1,27 @@ +/* + * 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 -package shark +package hive package execution import scala.collection.JavaConversions._ -import org.apache.spark.sql.shark.TestShark +import org.apache.spark.sql.hive.TestHive /** * A set of test cases that validate partition and column pruning. @@ -120,7 +137,7 @@ class PruningSuite extends HiveComparisonTest { expectedScannedColumns: Seq[String], expectedPartValues: Seq[Seq[String]]) = { test(s"$testCaseName - pruning test") { - val plan = new TestShark.SqlQueryExecution(sql).executedPlan + val plan = new TestHive.SqlQueryExecution(sql).executedPlan val actualOutputColumns = plan.output.map(_.name) val (actualScannedColumns, actualPartValues) = plan.collect { case p @ HiveTableScan(columns, relation, _) => From 5f2963c053f39ef4298598be918a4758c1c32a13 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 27 Feb 2014 15:20:05 -0800 Subject: [PATCH 713/778] naming and continuous compilation fixes. --- project/SparkBuild.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 63f7c3db6c1a4..d43beca16f37e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -60,7 +60,7 @@ object SparkBuild extends Build { lazy val catalyst = Project("catalyst", file("sql/catalyst"), settings = catalystSettings) - lazy val sql = Project("sql", file("sql/core"), settings = schemaCoreSettings) dependsOn(core, catalyst) + lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core, catalyst) lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql) @@ -123,7 +123,7 @@ object SparkBuild extends Build { .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) // Everything except assembly, tools and examples belong to packageProjects - lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, hive) ++ maybeYarnRef + lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql, hive) ++ maybeYarnRef lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) @@ -355,8 +355,8 @@ object SparkBuild extends Build { "com.typesafe" %% "scalalogging-slf4j" % "1.0.1") ) - def schemaCoreSettings = sharedSettings ++ Seq( - name := "spark-schema" + def sqlCoreSettings = sharedSettings ++ Seq( + name := "spark-sql" ) def hiveSettings = sharedSettings ++ Seq( From 4d57d0e7b0e929d14c9d4218d5b63a03e176d04d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 27 Feb 2014 15:37:26 -0800 Subject: [PATCH 714/778] Fix test execution on travis. --- .travis.yml | 13 +++++++++++++ dev/download-hive-tests.sh | 4 ++++ .../scala/org/apache/spark/sql/hive/TestShark.scala | 2 +- .../sql/hive/execution/HiveComparisonTest.scala | 2 +- 4 files changed, 19 insertions(+), 2 deletions(-) create mode 100644 .travis.yml create mode 100755 dev/download-hive-tests.sh diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 0000000000000..db57647e6d738 --- /dev/null +++ b/.travis.yml @@ -0,0 +1,13 @@ + language: scala + scala: + - "2.10.3" + jdk: + - oraclejdk7 + cache: + directories: + - $HOME/.m2 + - $HOME/.ivy2 + - $HOME/.sbt + script: + - "dev/download-hive-tests.sh" + - "GIT_AUTHOR_NAME=\"Michael Armbrust\" GIT_AUTHOR_EMAIL=\"michael@databricks.com\" GIT_COMMITTER_NAME=\"Michael Armbrust\" GIT_COMMITTER_EMAIL=\"michael@databricks.com\" sbt ++$TRAVIS_SCALA_VERSION 'set scalacOptions += \"-Xfatal-warnings\"' catalyst/test sql/test hive/test" diff --git a/dev/download-hive-tests.sh b/dev/download-hive-tests.sh new file mode 100755 index 0000000000000..916a9b322be23 --- /dev/null +++ b/dev/download-hive-tests.sh @@ -0,0 +1,4 @@ +#!/bin/sh + +wget -O hiveTests.tgz http://cs.berkeley.edu/~marmbrus/tmp/hiveTests.tgz +tar zxvf hiveTests.tgz \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala index 5848127c54114..65c0e767864dc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala @@ -101,7 +101,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { hiveFilesTemp.delete() hiveFilesTemp.mkdir() - val inRepoTests = new File("src/test/hive/") + val inRepoTests = new File("src/test/resources/") def getHiveFile(path: String): File = { val stripped = path.replaceAll("""\.\.\/""", "") hiveDevHome 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 04a6fa56b4fc9..ae259394df568 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 @@ -79,7 +79,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G .map(name => new File(targetDir, s"$suiteName.$name")) /** The local directory with cached golden answer will be stored. */ - protected val answerCache = new File("src/test/golden") + protected val answerCache = new File("src/test/resources/golden") if (!answerCache.exists) { answerCache.mkdir() } From 7413ac22622a991eac5fba33cbaeee2008f324f0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 27 Feb 2014 16:04:41 -0800 Subject: [PATCH 715/778] make test downloading quieter. --- dev/download-hive-tests.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/download-hive-tests.sh b/dev/download-hive-tests.sh index 916a9b322be23..6c412a8493c18 100755 --- a/dev/download-hive-tests.sh +++ b/dev/download-hive-tests.sh @@ -1,4 +1,4 @@ #!/bin/sh wget -O hiveTests.tgz http://cs.berkeley.edu/~marmbrus/tmp/hiveTests.tgz -tar zxvf hiveTests.tgz \ No newline at end of file +tar zxf hiveTests.tgz \ No newline at end of file From 608a29ea363e4093e605b2ecdcf3d55f4109e30d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 27 Feb 2014 18:22:58 -0800 Subject: [PATCH 716/778] Add hive as a repl dependency --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d43beca16f37e..0c27386621fa7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -50,7 +50,7 @@ object SparkBuild extends Build { lazy val core = Project("core", file("core"), settings = coreSettings) lazy val repl = Project("repl", file("repl"), settings = replSettings) - .dependsOn(core, graphx, bagel, mllib) + .dependsOn(core, graphx, bagel, mllib, hive) lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) From c3343868f8cc8b1054513fe6619c9bb193e8816a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Feb 2014 14:29:16 -0800 Subject: [PATCH 717/778] Initial support for generating schema's based on case classes. --- .../spark/sql/catalyst/dsl/package.scala | 47 +++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 93227c255f9e8..9405d0ae290db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql package catalyst import scala.language.implicitConversions +import scala.reflect.runtime.universe.TypeTag import analysis.UnresolvedAttribute import expressions._ @@ -53,6 +54,52 @@ import types._ * }}} */ package object dsl { + + /** + * Provides experimental support for generating catalyst schemas for scala objects. + */ + object reflect { + import scala.reflect.runtime.universe._ + + /** Returns a Sequence of attributes for the given case class type. */ + def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { + case s: StructType => + s.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) + } + + /** Returns a catalyst DataType for the given Scala Type using reflection. */ + def schemaFor[T: TypeTag]: DataType = schemaFor(typeOf[T]) + + /** Returns a catalyst DataType for the given Scala Type using reflection. */ + def schemaFor(tpe: `Type`): DataType = tpe match { + case t if t <:< typeOf[Product] => + val params = t.member("": TermName).asMethod.paramss + StructType( + params.head.map(p => StructField(p.name.toString, schemaFor(p.typeSignature), true))) + case t if t <:< typeOf[Seq[_]] => + val TypeRef(_, _, Seq(elementType)) = t + ArrayType(schemaFor(elementType)) + case t if t <:< typeOf[String] => StringType + case t if t <:< definitions.IntTpe => IntegerType + case t if t <:< definitions.LongTpe => LongType + case t if t <:< definitions.DoubleTpe => DoubleType + case t if t <:< definitions.ShortTpe => ShortType + case t if t <:< definitions.ByteTpe => ByteType + } + } + + implicit class CaseClassRelation[A <: Product : TypeTag](data: Seq[A]) { + + /** + * Implicitly added to Sequences of case class objects. Returns a catalyst logical relation + * for the the data in the sequence. + */ + def asRelation: LocalRelation = { + val output = reflect.attributesFor[A] + LocalRelation(output, data) + } + } + protected trait ImplicitOperators { def expr: Expression From b33e47ede48e9803fe213ec71d9a3ccea804b69a Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 16 Feb 2014 16:09:02 +0200 Subject: [PATCH 718/778] First commit of Parquet import of primitive column types --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 72 ++++++ .../catalyst/execution/ParquetTableScan.scala | 141 ++++++++++++ .../execution/ParquetQueryTests.scala | 209 ++++++++++++++++++ 3 files changed, 422 insertions(+) create mode 100644 src/main/scala/catalyst/execution/ParquetTableScan.scala create mode 100644 src/test/scala/catalyst/execution/ParquetQueryTests.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index d63dd5cf87326..28f50b81f8d57 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 @@ -26,6 +26,9 @@ import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.AbstractDeserializer +import org.apache.hadoop.mapred.InputFormat +import org.apache.hadoop.fs.Path import catalyst.analysis.Catalog import catalyst.expressions._ @@ -36,6 +39,11 @@ import catalyst.types._ import scala.collection.JavaConversions._ +import parquet.schema.MessageType +import parquet.schema.MessageTypeParser +import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} + + class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { import HiveMetastoreTypes._ @@ -235,3 +243,67 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt val output = attributes ++ partitionKeys } + +case class ParquetRelation(databaseName: String, tableName: String, alias: Option[String]) + (val parquetSchema: MessageType, val path: Path) + extends BaseRelation { + + // For now we do not allow partitioning. + val partitionKeys = List() + + /** Non-partitionKey attributes */ + val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) + + val output = attributes ++ partitionKeys + + /* TODO: implement low-level Parquet metadata store access */ + // val metaData: ParquetMetadata + // def getBlocks: java.util.List[BlockMetaData] = metaData.getBlocks + // def getColumns: java.util.List[ColumnDescriptor] = metaData.getFileMetaData.getSchema.getColumns + + val numberOfBlocks = 1 // TODO: see comment above +} + +object ParquetRelation { + def apply(schemaString : String, path: Path) = + new ParquetRelation("test", "test", Some("") )(ParquetTypesConverter.getSchema(schemaString), path) +} + +object ParquetTypesConverter { + def toDataType(parquetType : ParquetPrimitiveTypeName) : DataType = parquetType match { + // for now map binary to string type + // TODO: figure out how Parquet uses strings or why we can't use them in a MessageType schema + case ParquetPrimitiveTypeName.BINARY => StringType + case ParquetPrimitiveTypeName.BOOLEAN => BooleanType + case ParquetPrimitiveTypeName.DOUBLE => DoubleType + case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY => ArrayType(ByteType) + case ParquetPrimitiveTypeName.FLOAT => FloatType + case ParquetPrimitiveTypeName.INT32 => IntegerType + case ParquetPrimitiveTypeName.INT64 => LongType + case ParquetPrimitiveTypeName.INT96 => LongType // TODO: is there an equivalent? + case _ => sys.error(s"Unsupported parquet datatype") + } + + def getSchema(schemaString : String) : MessageType = MessageTypeParser.parseMessageType(schemaString) + + def convertToHiveSchema(schemaString : String) : List[FieldSchema] = { + val columns = getSchema(schemaString).getColumns + columns.map { + case (desc) => { + val ctype = toDataType(desc.getType) + val name = desc.getPath.toString + new FieldSchema(name, desc.getType.javaType.toString, "catalyst type: ctype.toString") + } + }.toList + } + + def convertToAttributes(parquetSchema: MessageType) : List[Attribute] = { + parquetSchema.getColumns.map { + case (desc) => { + val ctype = toDataType(desc.getType) + val name = desc.getPath.toString + new AttributeReference(name, ctype, false)() + } + }.toList + } +} diff --git a/src/main/scala/catalyst/execution/ParquetTableScan.scala b/src/main/scala/catalyst/execution/ParquetTableScan.scala new file mode 100644 index 0000000000000..0b4f45f1cff6f --- /dev/null +++ b/src/main/scala/catalyst/execution/ParquetTableScan.scala @@ -0,0 +1,141 @@ +package catalyst.execution + +import org.apache.spark.SerializableWritable +import org.apache.spark.rdd.RDD + +import parquet.io.api._ +import parquet.schema.MessageType +import parquet.hadoop.ParquetInputFormat +import parquet.hadoop.api.ReadSupport +import org.apache.hadoop.conf.Configuration +import parquet.hadoop.api.ReadSupport.ReadContext + +import catalyst.expressions._ +import catalyst.types._ +import catalyst.types.BinaryType +import catalyst.Logging + +import org.apache.hadoop.mapreduce.Job + +/** + * Parquet table scan operator. Only a stub currently. + */ +case class ParquetTableScan( + attributes: Seq[Attribute], + relation: ParquetRelation, + partitionPruningPred: Option[Expression] // not used + )( + @transient val sc: SharkContext) + extends LeafNode { + + private val _broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(sc.hiveconf)) + + def broadcastedHiveConf = _broadcastedHiveConf + + def hiveConf = _broadcastedHiveConf.value.value + + // TODO: currently this is not used + private val _minSplitsPerRDD = math.min( + math.max(sc.hiveconf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits), + relation.numberOfBlocks // we don't want to have more splits than blocks + ) + + /** + * Runs this query returning the result as an RDD. + */ + override def execute(): RDD[Row] = { + // TODO: for now we do not check whether the relation's schema matches the one of the + // underlying Parquet file + + val job = new Job(sc.hiveconf) + ParquetInputFormat.setReadSupportClass(job, classOf[catalyst.execution.RowReadSupport]) + + sc.newAPIHadoopFile( + relation.path.toUri.toString, + classOf[ParquetInputFormat[Row]], + classOf[Void], classOf[Row], + job.getConfiguration) + .map(_._2) + } + + override def output: Seq[Attribute] = attributes // right now we pass everything through, always +} + +class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterializer[Row] { + + def this(parquetSchema: MessageType) = this(new CatalystGroupConverter(ParquetTypesConverter.convertToAttributes(parquetSchema))) + + override def getCurrentRecord: Row = root.getCurrentRecord + + override def getRootConverter: GroupConverter = root +} + +class RowReadSupport extends ReadSupport[Row] with Logging { + override def prepareForRead( + conf: Configuration, + stringMap: java.util.Map[String, String], + fileSchema: MessageType, + readContext: ReadContext + ): RecordMaterializer[Row] = { + logger.debug(s"preparing for read with schema ${fileSchema.toString}") + new RowRecordMaterializer(fileSchema) + } + + override def init( + configuration: Configuration, + keyValueMetaData: java.util.Map[String, String], + fileSchema: MessageType + ): ReadContext = { + logger.debug(s"read support initialized for schema ${fileSchema.toString}") + new ReadContext(fileSchema, keyValueMetaData) + } +} + +class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { + var current: GenericRow = new GenericRow(Seq()) + // initialization may not strictly be required + val currentData: Array[Any] = new Array[Any](schema.length) + + val converters: Array[Converter] = schema.map { + a => a.dataType match { + case ctype: NativeType => + // note: for some reason matching for StringType fails so use this ugly if instead + if (ctype == StringType) new CatalystPrimitiveStringConverter(this, schema.indexOf(a)) + else new CatalystPrimitiveConverter(this, schema.indexOf(a)) + case _ => throw new RuntimeException("unable to convert datatype in CatalystGroupConverter") + } + }.toArray + + override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) + + def getCurrentRecord: GenericRow = current + + override def start(): Unit = { + for (i <- 0 until schema.length) { + currentData.update(i, Nil) + } + } + + override def end(): Unit = { + current = new GenericRow(currentData) + } +} + +class CatalystPrimitiveConverter(parent: CatalystGroupConverter, fieldIndex: Int) extends PrimitiveConverter { + // TODO: consider refactoring these together with ParquetTypesConverter + override def addBinary(value: Binary): Unit = parent.currentData.update(fieldIndex, value.getBytes.asInstanceOf[BinaryType.JvmType]) + + override def addBoolean(value: Boolean): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[BooleanType.JvmType]) + + override def addDouble(value: Double): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[DoubleType.JvmType]) + + override def addFloat(value: Float): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[FloatType.JvmType]) + + override def addInt(value: Int): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[IntegerType.JvmType]) + + override def addLong(value: Long): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[LongType.JvmType]) +} + +class CatalystPrimitiveStringConverter(parent: CatalystGroupConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { + override def addBinary(value: Binary): Unit = parent.currentData.update(fieldIndex, value.toStringUsingUTF8.asInstanceOf[StringType.JvmType]) +} diff --git a/src/test/scala/catalyst/execution/ParquetQueryTests.scala b/src/test/scala/catalyst/execution/ParquetQueryTests.scala new file mode 100644 index 0000000000000..a9b67a05dab7a --- /dev/null +++ b/src/test/scala/catalyst/execution/ParquetQueryTests.scala @@ -0,0 +1,209 @@ +package catalyst +package execution + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import java.io.File +import java.util.Arrays + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.rdd.RDD + +import parquet.schema.{MessageTypeParser, MessageType} +import parquet.column.ColumnDescriptor +import parquet.hadoop.metadata.{ParquetMetadata, CompressionCodecName} +import parquet.hadoop.{ParquetFileReader, ParquetFileWriter} +import parquet.bytes.BytesInput +import parquet.column.Encoding._ +import parquet.column.page.PageReadStore + +import expressions._ + +/* Implicits */ +import dsl._ + +object ParquetTestData { + + + val testSchema = """message myrecord { + required boolean myboolean; + required int32 myint; + required binary mystring; + }""" + + val testFile = new File("/tmp/testParquetFile").getAbsoluteFile + + val testData = ParquetRelation(testSchema, new Path(testFile.toURI)) + + def writeFile = { + testFile.delete + val path: Path = new Path(testFile.toURI) + val configuration: Configuration = new Configuration + + val schema: MessageType = MessageTypeParser.parseMessageType(ParquetTestData.testSchema) + val path1: Array[String] = Array("myboolean") + val c1: ColumnDescriptor = schema.getColumnDescription(path1) + val path2: Array[String] = Array("myint") + val c2: ColumnDescriptor = schema.getColumnDescription(path2) + val path3: Array[String] = Array("mystring") + val c3: ColumnDescriptor = schema.getColumnDescription(path3) + + val bytes1: Array[Byte] = Array(1, 0, 0).map(_.toByte) + val bytes2: Array[Byte] = Array(5, 0, 0, 0).map(_.toByte) + + // apparently strings are encoded as their length as int followed by the string + val bb = java.nio.ByteBuffer.allocate(4) + bb.order(java.nio.ByteOrder.LITTLE_ENDIAN) + val tmpbytes3: Array[Byte] = "abc".getBytes("UTF-8") + bb.putInt(tmpbytes3.length) + val bytes3: Array[Byte] = (bb.array().toList ::: tmpbytes3.toList).toArray + + val codec: CompressionCodecName = CompressionCodecName.UNCOMPRESSED + val w: ParquetFileWriter = new ParquetFileWriter(configuration, schema, path) + + w.start + w.startBlock(3) + w.startColumn(c1, 3, codec) + // note to myself: repetition levels cannot be PLAIN encoded + w.writeDataPage(3, 3, BytesInput.from(bytes1), RLE, RLE, PLAIN) + w.endColumn + w.startColumn(c2, 3, codec) + w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + w.endColumn + w.startColumn(c3, 3, codec) + w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) + w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) + w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) + w.endColumn() + w.endBlock + w.startBlock(3) + w.startColumn(c1, 3, codec) + w.writeDataPage(3, 3, BytesInput.from(bytes1), RLE, RLE, PLAIN) + w.endColumn + w.startColumn(c2, 3, codec) + w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + w.endColumn + w.startColumn(c3, 3, codec) + w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) + w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) + w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) + w.endColumn() + w.endBlock + w.end(new java.util.HashMap[String, String]) + } + + def readFile = { + val configuration: Configuration = new Configuration + val path = new Path(testFile.toURI) + val schema: MessageType = MessageTypeParser.parseMessageType(ParquetTestData.testSchema) + val path1: Array[String] = Array("myboolean") + val c1: ColumnDescriptor = schema.getColumnDescription(path1) + println(c1.toString) + val path2: Array[String] = Array("myint") + val readFooter: ParquetMetadata = ParquetFileReader.readFooter(configuration, path) + println("this many blocks: " + readFooter.getBlocks.size()) + println("metadata: " + readFooter.getFileMetaData.toString) + val r: ParquetFileReader = new ParquetFileReader(configuration, path, readFooter.getBlocks, Arrays.asList(schema.getColumnDescription(path1), schema.getColumnDescription(path2))) + var pages: PageReadStore = r.readNextRowGroup + println("number of rows first group" + pages.getRowCount) + var pageReader = pages.getPageReader(c1) + var page = pageReader.readPage() + assert(page != null) + } + + val complexSchema = "message m { required group a {required binary b;} required group c { required int64 d; }}" + + val complexTestFile: File = new File("/tmp/testComplexParquetFile").getAbsoluteFile + + val complexTestData = ParquetRelation(complexSchema, new Path(complexTestFile.toURI)) + + // this second test is from TestParquetFileWriter + def writeComplexFile = { + complexTestFile.delete + val path: Path = new Path(complexTestFile.toURI) + val configuration: Configuration = new Configuration + + val schema: MessageType = MessageTypeParser.parseMessageType(ParquetTestData.complexSchema) + val path1: Array[String] = Array("a", "b") + val c1: ColumnDescriptor = schema.getColumnDescription(path1) + val path2: Array[String] = Array("c", "d") + val c2: ColumnDescriptor = schema.getColumnDescription(path2) + + val bytes1: Array[Byte] = Array(0, 1, 2, 3).map(_.toByte) + val bytes2: Array[Byte] = Array(1, 2, 3, 4).map(_.toByte) + val bytes3: Array[Byte] = Array(2, 3, 4, 5).map(_.toByte) + val bytes4: Array[Byte] = Array(3, 4, 5, 6).map(_.toByte) + + val codec: CompressionCodecName = CompressionCodecName.UNCOMPRESSED + val w: ParquetFileWriter = new ParquetFileWriter(configuration, schema, path) + + w.start + w.startBlock(3) + w.startColumn(c1, 5, codec) + val c1Starts: Long = w.getPos + w.writeDataPage(2, 4, BytesInput.from(bytes1), RLE, RLE, PLAIN) + w.writeDataPage(3, 4, BytesInput.from(bytes1), RLE, RLE, PLAIN) + w.endColumn + val c1Ends: Long = w.getPos + w.startColumn(c2, 6, codec) + val c2Starts: Long = w.getPos + w.writeDataPage(2, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + w.writeDataPage(3, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + w.endColumn + val c2Ends: Long = w.getPos + w.endBlock + w.startBlock(4) + w.startColumn(c1, 7, codec) + w.writeDataPage(7, 4, BytesInput.from(bytes3), RLE, RLE, PLAIN) + w.endColumn + w.startColumn(c2, 8, codec) + w.writeDataPage(8, 4, BytesInput.from(bytes4), RLE, RLE, PLAIN) + w.endColumn + w.endBlock + w.end(new java.util.HashMap[String, String]) + } +} + +class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { + override def beforeAll() { + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests + // without restarting the JVM. + System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") + ParquetTestData.writeFile + ParquetTestData.readFile + ParquetTestData.writeComplexFile + } + + + test("Import of simple Parquet file") { + val result = getRDD(ParquetTestData.testData).collect() + val allChecks: Boolean = result.zipWithIndex.forall { + case (row, index) => { + val checkBoolean = + if (index % 3 == 0) + (row(0) == true) + else + (row(0) == false) + val checkInt = (row(1) == 5) + val checkString = (row(2) == "abc") + checkBoolean && checkInt && checkString + } + } + assert(allChecks) + } + + private def getRDD(parquetRelation: ParquetRelation): RDD[Row] = { + val sharkInstance = new TestSharkInstance + val catalystSchema: List[Attribute] = ParquetTypesConverter.convertToAttributes(parquetRelation.parquetSchema) + val scanner = new ParquetTableScan(catalystSchema, parquetRelation, Option(""))(sharkInstance.sc) + scanner.execute() + } +} \ No newline at end of file From 99a920916fa7f03669d86a9b9cf7482fedcaf318 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 16 Feb 2014 19:54:44 +0200 Subject: [PATCH 719/778] Expanding ParquetQueryTests to cover all primitive types --- .../execution/ParquetQueryTests.scala | 137 ++++++++++++++---- 1 file changed, 112 insertions(+), 25 deletions(-) diff --git a/src/test/scala/catalyst/execution/ParquetQueryTests.scala b/src/test/scala/catalyst/execution/ParquetQueryTests.scala index a9b67a05dab7a..59f5f129bbf51 100644 --- a/src/test/scala/catalyst/execution/ParquetQueryTests.scala +++ b/src/test/scala/catalyst/execution/ParquetQueryTests.scala @@ -31,6 +31,9 @@ object ParquetTestData { required boolean myboolean; required int32 myint; required binary mystring; + required int64 mylong; + required float myfloat; + required double mydouble; }""" val testFile = new File("/tmp/testParquetFile").getAbsoluteFile @@ -49,16 +52,12 @@ object ParquetTestData { val c2: ColumnDescriptor = schema.getColumnDescription(path2) val path3: Array[String] = Array("mystring") val c3: ColumnDescriptor = schema.getColumnDescription(path3) - - val bytes1: Array[Byte] = Array(1, 0, 0).map(_.toByte) - val bytes2: Array[Byte] = Array(5, 0, 0, 0).map(_.toByte) - - // apparently strings are encoded as their length as int followed by the string - val bb = java.nio.ByteBuffer.allocate(4) - bb.order(java.nio.ByteOrder.LITTLE_ENDIAN) - val tmpbytes3: Array[Byte] = "abc".getBytes("UTF-8") - bb.putInt(tmpbytes3.length) - val bytes3: Array[Byte] = (bb.array().toList ::: tmpbytes3.toList).toArray + val path4: Array[String] = Array("mylong") + val c4: ColumnDescriptor = schema.getColumnDescription(path4) + val path5: Array[String] = Array("myfloat") + val c5: ColumnDescriptor = schema.getColumnDescription(path5) + val path6: Array[String] = Array("mydouble") + val c6: ColumnDescriptor = schema.getColumnDescription(path6) val codec: CompressionCodecName = CompressionCodecName.UNCOMPRESSED val w: ParquetFileWriter = new ParquetFileWriter(configuration, schema, path) @@ -67,32 +66,80 @@ object ParquetTestData { w.startBlock(3) w.startColumn(c1, 3, codec) // note to myself: repetition levels cannot be PLAIN encoded - w.writeDataPage(3, 3, BytesInput.from(bytes1), RLE, RLE, PLAIN) + // boolean + w.writeDataPage(3, 3, BytesInput.from( + concat(serializeValue(true), serializeValue(false), serializeValue(false)) + ), RLE, RLE, PLAIN) w.endColumn w.startColumn(c2, 3, codec) - w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) - w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) - w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + // int + w.writeDataPage(3, 12, BytesInput.from( + concat(serializeValue(5), serializeValue(5), serializeValue(5)) + ), RLE, RLE, PLAIN) w.endColumn w.startColumn(c3, 3, codec) - w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) - w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) - w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) + // string + val bytes = serializeValue("abc".asInstanceOf[AnyVal]) + w.writeDataPage(3, 3*bytes.length, BytesInput.from( + concat(bytes, bytes, bytes) + ), RLE, RLE, PLAIN) + w.endColumn() + w.startColumn(c4, 3, codec) + // long + w.writeDataPage(3, 24, BytesInput.from( + concat(serializeValue(1L<<33), serializeValue(1L<<33), serializeValue(1L<<33)) + ), RLE, RLE, PLAIN) + w.endColumn() + w.startColumn(c5, 3, codec) + // float + w.writeDataPage(3, 12, BytesInput.from( + concat(serializeValue(2.5F), serializeValue(2.5F), serializeValue(2.5F)) + ), RLE, RLE, PLAIN) + w.endColumn() + w.startColumn(c6, 3, codec) + // double + w.writeDataPage(3, 24, BytesInput.from( + concat(serializeValue(4.5D), serializeValue(4.5D), serializeValue(4.5D)) + ), RLE, RLE, PLAIN) w.endColumn() w.endBlock w.startBlock(3) w.startColumn(c1, 3, codec) - w.writeDataPage(3, 3, BytesInput.from(bytes1), RLE, RLE, PLAIN) + // note to myself: repetition levels cannot be PLAIN encoded + // boolean + w.writeDataPage(3, 3, BytesInput.from( + concat(serializeValue(true), serializeValue(false), serializeValue(false)) + ), RLE, RLE, PLAIN) w.endColumn w.startColumn(c2, 3, codec) - w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) - w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) - w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) + // int + w.writeDataPage(3, 12, BytesInput.from( + concat(serializeValue(5), serializeValue(5), serializeValue(5)) + ), RLE, RLE, PLAIN) w.endColumn w.startColumn(c3, 3, codec) - w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) - w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) - w.writeDataPage(1, bytes3.length, BytesInput.from(bytes3), RLE, RLE, PLAIN) + // string + w.writeDataPage(3, 3*bytes.length, BytesInput.from( + concat(bytes, bytes, bytes) + ), RLE, RLE, PLAIN) + w.endColumn() + w.startColumn(c4, 3, codec) + // long + w.writeDataPage(3, 24, BytesInput.from( + concat(serializeValue(1L<<33), serializeValue(1L<<33), serializeValue(1L<<33)) + ), RLE, RLE, PLAIN) + w.endColumn() + w.startColumn(c5, 3, codec) + // float + w.writeDataPage(3, 12, BytesInput.from( + concat(serializeValue(2.5F), serializeValue(2.5F), serializeValue(2.5F)) + ), RLE, RLE, PLAIN) + w.endColumn() + w.startColumn(c6, 3, codec) + // double + w.writeDataPage(3, 24, BytesInput.from( + concat(serializeValue(4.5D), serializeValue(4.5D), serializeValue(4.5D)) + ), RLE, RLE, PLAIN) w.endColumn() w.endBlock w.end(new java.util.HashMap[String, String]) @@ -169,6 +216,43 @@ object ParquetTestData { w.endBlock w.end(new java.util.HashMap[String, String]) } + + private def serializeValue(value: Any) : Array[Byte] = { + value match { + case i: Int => { + val bb = java.nio.ByteBuffer.allocate(4) + bb.order(java.nio.ByteOrder.LITTLE_ENDIAN) + bb.putInt(i) + bb.array() + } + case l: Long => { + val bb = java.nio.ByteBuffer.allocate(8) + bb.order(java.nio.ByteOrder.LITTLE_ENDIAN) + bb.putLong(l) + bb.array() + } + case f: Float => serializeValue(java.lang.Float.floatToIntBits(f)) + case d: Double => serializeValue(java.lang.Double.doubleToLongBits(d)) + case s: String => { + // apparently strings are encoded as their length as int followed by the string + val bytes: Array[Byte] = s.getBytes("UTF-8") + (serializeValue(bytes.length).toList ::: bytes.toList).toArray + } + case b: Boolean => { + if(b) + Array(1.toByte) + else + Array(0.toByte) + } + } + } + + private def concat(values: Array[Byte]*) : Array[Byte] = { + var retval = List[Byte]() + for(value <- values) + retval = retval ::: value.toList + retval.toArray + } } class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { @@ -194,7 +278,10 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { (row(0) == false) val checkInt = (row(1) == 5) val checkString = (row(2) == "abc") - checkBoolean && checkInt && checkString + val checkLong = (row(3) == (1L<<33)) + val checkFloat = (row(4) == 2.5F) + val checkDouble = (row(5) == 4.5D) + checkBoolean && checkInt && checkString && checkLong && checkFloat && checkDouble } } assert(allChecks) From eb0e521572c500e79de2dc5c3aa188b222490681 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Mon, 17 Feb 2014 15:28:37 +0200 Subject: [PATCH 720/778] Fixing package names and other problems that came up after the rebase --- .../spark/sql/shark}/ParquetTableScan.scala | 53 ++++++++++++++----- .../shark}/execution/ParquetQueryTests.scala | 26 ++++++--- 2 files changed, 58 insertions(+), 21 deletions(-) rename {src/main/scala/catalyst/execution => shark/src/main/scala/org/apache/spark/sql/shark}/ParquetTableScan.scala (76%) rename {src/test/scala/catalyst => shark/src/test/scala/org/apache/spark/sql/shark}/execution/ParquetQueryTests.scala (94%) diff --git a/src/main/scala/catalyst/execution/ParquetTableScan.scala b/shark/src/main/scala/org/apache/spark/sql/shark/ParquetTableScan.scala similarity index 76% rename from src/main/scala/catalyst/execution/ParquetTableScan.scala rename to shark/src/main/scala/org/apache/spark/sql/shark/ParquetTableScan.scala index 0b4f45f1cff6f..c430e835ffb74 100644 --- a/src/main/scala/catalyst/execution/ParquetTableScan.scala +++ b/shark/src/main/scala/org/apache/spark/sql/shark/ParquetTableScan.scala @@ -1,4 +1,5 @@ -package catalyst.execution +package org.apache.spark.sql +package shark import org.apache.spark.SerializableWritable import org.apache.spark.rdd.RDD @@ -10,15 +11,14 @@ import parquet.hadoop.api.ReadSupport import org.apache.hadoop.conf.Configuration import parquet.hadoop.api.ReadSupport.ReadContext -import catalyst.expressions._ +import catalyst.expressions.{Attribute, GenericRow, Row, Expression} +import catalyst.plans.logical.LeafNode import catalyst.types._ -import catalyst.types.BinaryType -import catalyst.Logging import org.apache.hadoop.mapreduce.Job - /** - * Parquet table scan operator. Only a stub currently. + * Parquet table scan operator. Imports the file that backs the given [[org.apache.spark.sql.shark.ParquetRelation]] + * as a RDD[Row]. Only a stub currently. */ case class ParquetTableScan( attributes: Seq[Attribute], @@ -28,7 +28,8 @@ case class ParquetTableScan( @transient val sc: SharkContext) extends LeafNode { - private val _broadcastedHiveConf = SharkEnv.sc.broadcast(new SerializableWritable(sc.hiveconf)) + private val _broadcastedHiveConf = + sc.sparkContext.broadcast(new SerializableWritable(sc.hiveconf)) def broadcastedHiveConf = _broadcastedHiveConf @@ -36,21 +37,21 @@ case class ParquetTableScan( // TODO: currently this is not used private val _minSplitsPerRDD = math.min( - math.max(sc.hiveconf.getInt("mapred.map.tasks", 1), SharkEnv.sc.defaultMinSplits), + math.max(sc.hiveconf.getInt("mapred.map.tasks", 1), sc. sparkContext.defaultMinSplits), relation.numberOfBlocks // we don't want to have more splits than blocks ) /** * Runs this query returning the result as an RDD. - */ - override def execute(): RDD[Row] = { + */ + def execute(): RDD[Row] = { // TODO: for now we do not check whether the relation's schema matches the one of the // underlying Parquet file val job = new Job(sc.hiveconf) - ParquetInputFormat.setReadSupportClass(job, classOf[catalyst.execution.RowReadSupport]) - - sc.newAPIHadoopFile( + ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.shark.RowReadSupport]) + // TODO: add record filters, etc. + sc.sparkContext.newAPIHadoopFile( relation.path.toUri.toString, classOf[ParquetInputFormat[Row]], classOf[Void], classOf[Row], @@ -61,6 +62,11 @@ case class ParquetTableScan( override def output: Seq[Attribute] = attributes // right now we pass everything through, always } +/** + * A [[parquet.io.api.RecordMaterializer]] for Rows. + * + * @param root The root group converter for the record. + */ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterializer[Row] { def this(parquetSchema: MessageType) = this(new CatalystGroupConverter(ParquetTypesConverter.convertToAttributes(parquetSchema))) @@ -70,6 +76,9 @@ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterial override def getRootConverter: GroupConverter = root } +/** + * A [[parquet.hadoop.api.ReadSupport]] for Row objects. + */ class RowReadSupport extends ReadSupport[Row] with Logging { override def prepareForRead( conf: Configuration, @@ -91,6 +100,12 @@ class RowReadSupport extends ReadSupport[Row] with Logging { } } +/** + * A [[parquet.io.api.GroupConverter]] that is able to convert a Parquet record + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. + * + * @param schema The corresponding Shark schema in the form of a list of attributes. + */ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { var current: GenericRow = new GenericRow(Seq()) // initialization may not strictly be required @@ -121,6 +136,12 @@ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { } } +/** + * A [[parquet.io.api.PrimitiveConverter]] that converts Parquet types to Catalyst types. + * + * @param parent The parent group converter. + * @param fieldIndex The index inside the record. + */ class CatalystPrimitiveConverter(parent: CatalystGroupConverter, fieldIndex: Int) extends PrimitiveConverter { // TODO: consider refactoring these together with ParquetTypesConverter override def addBinary(value: Binary): Unit = parent.currentData.update(fieldIndex, value.getBytes.asInstanceOf[BinaryType.JvmType]) @@ -136,6 +157,12 @@ class CatalystPrimitiveConverter(parent: CatalystGroupConverter, fieldIndex: Int override def addLong(value: Long): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[LongType.JvmType]) } +/** + * A [[parquet.io.api.PrimitiveConverter]] that converts Parquet strings (fixed-length byte arrays) into Catalyst Strings. + * + * @param parent The parent group converter. + * @param fieldIndex The index inside the record. + */ class CatalystPrimitiveStringConverter(parent: CatalystGroupConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { override def addBinary(value: Binary): Unit = parent.currentData.update(fieldIndex, value.toStringUsingUTF8.asInstanceOf[StringType.JvmType]) } diff --git a/src/test/scala/catalyst/execution/ParquetQueryTests.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQueryTests.scala similarity index 94% rename from src/test/scala/catalyst/execution/ParquetQueryTests.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQueryTests.scala index 59f5f129bbf51..a28ee9f3abea4 100644 --- a/src/test/scala/catalyst/execution/ParquetQueryTests.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQueryTests.scala @@ -1,4 +1,5 @@ -package catalyst +package org.apache.spark.sql +package shark package execution import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -19,10 +20,8 @@ import parquet.bytes.BytesInput import parquet.column.Encoding._ import parquet.column.page.PageReadStore -import expressions._ - -/* Implicits */ -import dsl._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} +import shark.TestShark object ParquetTestData { @@ -217,6 +216,12 @@ object ParquetTestData { w.end(new java.util.HashMap[String, String]) } + /** + * Serializes a given value so that it conforms to Parquet's uncompressed primitive value encoding. + * + * @param value The value to serialize. + * @return A byte array that contains the serialized value. + */ private def serializeValue(value: Any) : Array[Byte] = { value match { case i: Int => { @@ -287,10 +292,15 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { assert(allChecks) } + /** + * Computes the given [[org.apache.spark.sql.shark.ParquetRelation]] and returns its RDD. + * + * @param parquetRelation The Parquet relation. + * @return An RDD of Rows. + */ private def getRDD(parquetRelation: ParquetRelation): RDD[Row] = { - val sharkInstance = new TestSharkInstance val catalystSchema: List[Attribute] = ParquetTypesConverter.convertToAttributes(parquetRelation.parquetSchema) - val scanner = new ParquetTableScan(catalystSchema, parquetRelation, Option(""))(sharkInstance.sc) - scanner.execute() + val scanner = new ParquetTableScan(catalystSchema, parquetRelation, None)(TestShark) + scanner.execute } } \ No newline at end of file From 6ad05b34ecf9d457fd95c8e7f8f74ed979048cb9 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 19 Feb 2014 13:06:53 +0200 Subject: [PATCH 721/778] Moving ParquetRelation to spark.sql core --- .../apache/spark/sql/ParquetRelation.scala | 69 ++++++++++++++++++ .../apache/spark/sql}/ParquetTableScan.scala | 33 +++------ .../apache/spark/sql}/ParquetQueryTests.scala | 13 ++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 72 +------------------ 4 files changed, 90 insertions(+), 97 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala rename {shark/src/main/scala/org/apache/spark/sql/shark => core/src/main/scala/org/apache/spark/sql}/ParquetTableScan.scala (88%) rename {shark/src/test/scala/org/apache/spark/sql/shark/execution => core/src/test/scala/org/apache/spark/sql}/ParquetQueryTests.scala (97%) diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala new file mode 100644 index 0000000000000..864cc4066799f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala @@ -0,0 +1,69 @@ +package org.apache.spark.sql + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.catalyst.plans.logical.BaseRelation +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.ArrayType +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute} + +import parquet.schema.{MessageTypeParser, MessageType} +import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} + +import scala.collection.JavaConversions._ + +/** + * Relation formed by underlying Parquet file that contains data stored in columnar form. + * + * @param parquetSchema The requested schema (may contain projections w.r.t. Parquet file). + * @param path The path to the Parquet file. + */ +case class ParquetRelation(val tableName: String, val parquetSchema: MessageType, val path: Path) + + extends BaseRelation { + + /** Attributes */ + val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) + + val output = attributes + + /* TODO: implement low-level Parquet metadata store access */ + // val metaData: ParquetMetadata + // def getBlocks: java.util.List[BlockMetaData] = metaData.getBlocks + // def getColumns: java.util.List[ColumnDescriptor] = metaData.getFileMetaData.getSchema.getColumns + + val numberOfBlocks = 1 // TODO: see comment above +} + +object ParquetRelation { + def apply(schemaString : String, path: Path, tableName: String = "ParquetTable") = + new ParquetRelation(tableName, ParquetTypesConverter.getSchema(schemaString), path) +} + +object ParquetTypesConverter { + def toDataType(parquetType : ParquetPrimitiveTypeName) : DataType = parquetType match { + // for now map binary to string type + // TODO: figure out how Parquet uses strings or why we can't use them in a MessageType schema + case ParquetPrimitiveTypeName.BINARY => StringType + case ParquetPrimitiveTypeName.BOOLEAN => BooleanType + case ParquetPrimitiveTypeName.DOUBLE => DoubleType + case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY => ArrayType(ByteType) + case ParquetPrimitiveTypeName.FLOAT => FloatType + case ParquetPrimitiveTypeName.INT32 => IntegerType + case ParquetPrimitiveTypeName.INT64 => LongType + case ParquetPrimitiveTypeName.INT96 => LongType // TODO: is there an equivalent? + case _ => sys.error(s"Unsupported parquet datatype") + } + + def getSchema(schemaString : String) : MessageType = MessageTypeParser.parseMessageType(schemaString) + + def convertToAttributes(parquetSchema: MessageType) : List[Attribute] = { + parquetSchema.getColumns.map { + case (desc) => { + val ctype = toDataType(desc.getType) + val name = desc.getPath.toString + new AttributeReference(name, ctype, false)() + } + }.toList + } +} diff --git a/shark/src/main/scala/org/apache/spark/sql/shark/ParquetTableScan.scala b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala similarity index 88% rename from shark/src/main/scala/org/apache/spark/sql/shark/ParquetTableScan.scala rename to core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala index c430e835ffb74..c8c9db78daea4 100644 --- a/shark/src/main/scala/org/apache/spark/sql/shark/ParquetTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala @@ -1,14 +1,9 @@ package org.apache.spark.sql -package shark - -import org.apache.spark.SerializableWritable -import org.apache.spark.rdd.RDD import parquet.io.api._ import parquet.schema.MessageType import parquet.hadoop.ParquetInputFormat import parquet.hadoop.api.ReadSupport -import org.apache.hadoop.conf.Configuration import parquet.hadoop.api.ReadSupport.ReadContext import catalyst.expressions.{Attribute, GenericRow, Row, Expression} @@ -16,8 +11,13 @@ import catalyst.plans.logical.LeafNode import catalyst.types._ import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD + /** - * Parquet table scan operator. Imports the file that backs the given [[org.apache.spark.sql.shark.ParquetRelation]] + * Parquet table scan operator. Imports the file that backs the given [[org.apache.spark.sql.ParquetRelation]] * as a RDD[Row]. Only a stub currently. */ case class ParquetTableScan( @@ -25,22 +25,9 @@ case class ParquetTableScan( relation: ParquetRelation, partitionPruningPred: Option[Expression] // not used )( - @transient val sc: SharkContext) + @transient val sc: SparkContext) extends LeafNode { - private val _broadcastedHiveConf = - sc.sparkContext.broadcast(new SerializableWritable(sc.hiveconf)) - - def broadcastedHiveConf = _broadcastedHiveConf - - def hiveConf = _broadcastedHiveConf.value.value - - // TODO: currently this is not used - private val _minSplitsPerRDD = math.min( - math.max(sc.hiveconf.getInt("mapred.map.tasks", 1), sc. sparkContext.defaultMinSplits), - relation.numberOfBlocks // we don't want to have more splits than blocks - ) - /** * Runs this query returning the result as an RDD. */ @@ -48,10 +35,10 @@ case class ParquetTableScan( // TODO: for now we do not check whether the relation's schema matches the one of the // underlying Parquet file - val job = new Job(sc.hiveconf) - ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.shark.RowReadSupport]) + val job = new Job(new Configuration()) + ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.RowReadSupport]) // TODO: add record filters, etc. - sc.sparkContext.newAPIHadoopFile( + sc.newAPIHadoopFile( relation.path.toUri.toString, classOf[ParquetInputFormat[Row]], classOf[Void], classOf[Row], diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala similarity index 97% rename from shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQueryTests.scala rename to core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala index a28ee9f3abea4..c19e304dce501 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala @@ -1,6 +1,4 @@ package org.apache.spark.sql -package shark -package execution import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -21,7 +19,6 @@ import parquet.column.Encoding._ import parquet.column.page.PageReadStore import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} -import shark.TestShark object ParquetTestData { @@ -252,6 +249,12 @@ object ParquetTestData { } } + /** + * Concatenates a sequence of byte arrays. + * + * @param values The sequence of byte arrays to be concatenated. + * @return The concatenation of the given byte arrays. + */ private def concat(values: Array[Byte]*) : Array[Byte] = { var retval = List[Byte]() for(value <- values) @@ -293,14 +296,14 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { } /** - * Computes the given [[org.apache.spark.sql.shark.ParquetRelation]] and returns its RDD. + * Computes the given [[org.apache.spark.sql.ParquetRelation]] and returns its RDD. * * @param parquetRelation The Parquet relation. * @return An RDD of Rows. */ private def getRDD(parquetRelation: ParquetRelation): RDD[Row] = { val catalystSchema: List[Attribute] = ParquetTypesConverter.convertToAttributes(parquetRelation.parquetSchema) - val scanner = new ParquetTableScan(catalystSchema, parquetRelation, None)(TestShark) + val scanner = new ParquetTableScan(catalystSchema, parquetRelation, None)(TestSqlContext.sparkContext) scanner.execute } } \ No newline at end of file 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 28f50b81f8d57..c942c790ec7be 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 @@ -23,12 +23,14 @@ import scala.util.parsing.combinator.RegexParsers import org.apache.hadoop.hive.metastore.api.{FieldSchema, StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table} +import org.apache.hadoop.hive.metastore.api.{StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.AbstractDeserializer import org.apache.hadoop.mapred.InputFormat -import org.apache.hadoop.fs.Path import catalyst.analysis.Catalog import catalyst.expressions._ @@ -39,10 +41,6 @@ import catalyst.types._ import scala.collection.JavaConversions._ -import parquet.schema.MessageType -import parquet.schema.MessageTypeParser -import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} - class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { import HiveMetastoreTypes._ @@ -243,67 +241,3 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt val output = attributes ++ partitionKeys } - -case class ParquetRelation(databaseName: String, tableName: String, alias: Option[String]) - (val parquetSchema: MessageType, val path: Path) - extends BaseRelation { - - // For now we do not allow partitioning. - val partitionKeys = List() - - /** Non-partitionKey attributes */ - val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) - - val output = attributes ++ partitionKeys - - /* TODO: implement low-level Parquet metadata store access */ - // val metaData: ParquetMetadata - // def getBlocks: java.util.List[BlockMetaData] = metaData.getBlocks - // def getColumns: java.util.List[ColumnDescriptor] = metaData.getFileMetaData.getSchema.getColumns - - val numberOfBlocks = 1 // TODO: see comment above -} - -object ParquetRelation { - def apply(schemaString : String, path: Path) = - new ParquetRelation("test", "test", Some("") )(ParquetTypesConverter.getSchema(schemaString), path) -} - -object ParquetTypesConverter { - def toDataType(parquetType : ParquetPrimitiveTypeName) : DataType = parquetType match { - // for now map binary to string type - // TODO: figure out how Parquet uses strings or why we can't use them in a MessageType schema - case ParquetPrimitiveTypeName.BINARY => StringType - case ParquetPrimitiveTypeName.BOOLEAN => BooleanType - case ParquetPrimitiveTypeName.DOUBLE => DoubleType - case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY => ArrayType(ByteType) - case ParquetPrimitiveTypeName.FLOAT => FloatType - case ParquetPrimitiveTypeName.INT32 => IntegerType - case ParquetPrimitiveTypeName.INT64 => LongType - case ParquetPrimitiveTypeName.INT96 => LongType // TODO: is there an equivalent? - case _ => sys.error(s"Unsupported parquet datatype") - } - - def getSchema(schemaString : String) : MessageType = MessageTypeParser.parseMessageType(schemaString) - - def convertToHiveSchema(schemaString : String) : List[FieldSchema] = { - val columns = getSchema(schemaString).getColumns - columns.map { - case (desc) => { - val ctype = toDataType(desc.getType) - val name = desc.getPath.toString - new FieldSchema(name, desc.getType.javaType.toString, "catalyst type: ctype.toString") - } - }.toList - } - - def convertToAttributes(parquetSchema: MessageType) : List[Attribute] = { - parquetSchema.getColumns.map { - case (desc) => { - val ctype = toDataType(desc.getType) - val name = desc.getPath.toString - new AttributeReference(name, ctype, false)() - } - }.toList - } -} From a11e36428f3ea166825cbeb39ea23e86046dd26a Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 19 Feb 2014 16:12:30 +0200 Subject: [PATCH 722/778] Adding Parquet RowWriteSupport --- .../apache/spark/sql/ParquetRelation.scala | 20 ++++++- .../apache/spark/sql/ParquetTableScan.scala | 54 ++++++++++++++++++- 2 files changed, 71 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala index 864cc4066799f..82b81d3d93fc2 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala @@ -5,12 +5,13 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.catalyst.plans.logical.BaseRelation import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.ArrayType -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Row, AttributeReference, Attribute} import parquet.schema.{MessageTypeParser, MessageType} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} import scala.collection.JavaConversions._ +import parquet.io.api.{Binary, RecordConsumer} /** * Relation formed by underlying Parquet file that contains data stored in columnar form. @@ -55,6 +56,23 @@ object ParquetTypesConverter { case _ => sys.error(s"Unsupported parquet datatype") } + def consumeType(consumer: RecordConsumer, ctype: DataType, record: Row, index: Int): Unit = { + ctype match { + case StringType => consumer.addBinary( + Binary.fromByteArray( + record(index).asInstanceOf[String].getBytes("utf-8") + ) + ) + case IntegerType => consumer.addInteger(record.getInt(index)) + case LongType => consumer.addLong(record.getLong(index)) + case DoubleType => consumer.addDouble(record.getDouble(index)) + // TODO: where is getFloat in Row?! + case FloatType => consumer.addFloat(record(index).asInstanceOf[Float]) + case BooleanType => consumer.addBoolean(record.getBoolean(index)) + case _ => sys.error(s"Unsupported datatype, cannot write to consumer") + } + } + def getSchema(schemaString : String) : MessageType = MessageTypeParser.parseMessageType(schemaString) def convertToAttributes(parquetSchema: MessageType) : List[Attribute] = { diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala index c8c9db78daea4..bb937485b6326 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala @@ -1,9 +1,9 @@ package org.apache.spark.sql import parquet.io.api._ -import parquet.schema.MessageType +import parquet.schema.{MessageTypeParser, MessageType} import parquet.hadoop.ParquetInputFormat -import parquet.hadoop.api.ReadSupport +import parquet.hadoop.api.{WriteSupport, ReadSupport} import parquet.hadoop.api.ReadSupport.ReadContext import catalyst.expressions.{Attribute, GenericRow, Row, Expression} @@ -15,6 +15,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import parquet.hadoop.api.WriteSupport /** * Parquet table scan operator. Imports the file that backs the given [[org.apache.spark.sql.ParquetRelation]] @@ -87,6 +88,55 @@ class RowReadSupport extends ReadSupport[Row] with Logging { } } +/** + * A [[parquet.hadoop.api.WriteSupport]] for Row ojects. + */ +class RowWriteSupport extends WriteSupport[Row] with Logging { + final val PARQUET_ROW_SCHEMA: String = "spark.sql.parquet.row.schema" + + def setSchema(schema: MessageType, configuration: Configuration) { + // for testing + this.schema = schema + configuration.set(PARQUET_ROW_SCHEMA, schema.toString) + } + + def getSchema(configuration: Configuration): MessageType = { + return MessageTypeParser.parseMessageType(configuration.get(PARQUET_ROW_SCHEMA)) + } + + private var schema: MessageType = null + private var writer: RecordConsumer = null + private var attributes: Seq[Attribute] = null + + override def init(configuration: Configuration): WriteSupport.WriteContext = { + schema = if(schema == null) + getSchema(configuration) + else + schema + attributes = ParquetTypesConverter.convertToAttributes(schema) + new WriteSupport.WriteContext(schema, new java.util.HashMap[java.lang.String, java.lang.String]()); + } + + override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { + writer = recordConsumer + } + + // TODO: add groups (nested fields) + override def write(record: Row): Unit = { + writer.startMessage() + attributes.zipWithIndex.foreach { + case (attribute, index) => { + if(record(index) != null) { // null values indicate optional fields but we do not check currently + writer.startField(attribute.name, index) + ParquetTypesConverter.consumeType(writer, attribute.dataType, record, index) + writer.endField(attribute.name, index) + } + } + } + writer.endMessage() + } +} + /** * A [[parquet.io.api.GroupConverter]] that is able to convert a Parquet record * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. From 0f17d7b6fcea76b991da1790cf39b97d5543eee1 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 19 Feb 2014 16:26:55 +0200 Subject: [PATCH 723/778] Rewriting ParquetRelation tests with RowWriteSupport --- .../apache/spark/sql/ParquetQueryTests.scala | 51 ++++++++++++++----- 1 file changed, 39 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala index c19e304dce501..ee83219e55c39 100644 --- a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala @@ -13,23 +13,23 @@ import org.apache.spark.rdd.RDD import parquet.schema.{MessageTypeParser, MessageType} import parquet.column.ColumnDescriptor import parquet.hadoop.metadata.{ParquetMetadata, CompressionCodecName} -import parquet.hadoop.{ParquetFileReader, ParquetFileWriter} +import parquet.hadoop.{ParquetWriter, ParquetFileReader, ParquetFileWriter} import parquet.bytes.BytesInput import parquet.column.Encoding._ import parquet.column.page.PageReadStore -import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} +import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, Row} object ParquetTestData { val testSchema = """message myrecord { - required boolean myboolean; - required int32 myint; - required binary mystring; - required int64 mylong; - required float myfloat; - required double mydouble; + optional boolean myboolean; + optional int32 myint; + optional binary mystring; + optional int64 mylong; + optional float myfloat; + optional double mydouble; }""" val testFile = new File("/tmp/testParquetFile").getAbsoluteFile @@ -40,8 +40,36 @@ object ParquetTestData { testFile.delete val path: Path = new Path(testFile.toURI) val configuration: Configuration = new Configuration + val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) + + val writeSupport = new RowWriteSupport() + writeSupport.setSchema(schema, configuration) + val writer = new ParquetWriter(path, writeSupport) + for(i <- 0 until 15) { + val data = new Array[Any](6) + if(i % 3 ==0) + data.update(0, true) + else + data.update(0, false) + if(i % 5 == 0) + data.update(1, 5) + else + data.update(1, null) // optional + data.update(2, "abc") + data.update(3, 1L<<33) + data.update(4, 2.5F) + data.update(5, 4.5D) + writer.write(new GenericRow(data)) + } + writer.close() + } + + def writeFileAsPages = { + testFile.delete + val path: Path = new Path(testFile.toURI) + val configuration: Configuration = new Configuration + val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) - val schema: MessageType = MessageTypeParser.parseMessageType(ParquetTestData.testSchema) val path1: Array[String] = Array("myboolean") val c1: ColumnDescriptor = schema.getColumnDescription(path1) val path2: Array[String] = Array("myint") @@ -154,7 +182,7 @@ object ParquetTestData { println("metadata: " + readFooter.getFileMetaData.toString) val r: ParquetFileReader = new ParquetFileReader(configuration, path, readFooter.getBlocks, Arrays.asList(schema.getColumnDescription(path1), schema.getColumnDescription(path2))) var pages: PageReadStore = r.readNextRowGroup - println("number of rows first group" + pages.getRowCount) + println("number of rows first group " + pages.getRowCount) var pageReader = pages.getPageReader(c1) var page = pageReader.readPage() assert(page != null) @@ -274,7 +302,6 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { ParquetTestData.writeComplexFile } - test("Import of simple Parquet file") { val result = getRDD(ParquetTestData.testData).collect() val allChecks: Boolean = result.zipWithIndex.forall { @@ -284,7 +311,7 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { (row(0) == true) else (row(0) == false) - val checkInt = (row(1) == 5) + val checkInt = ((index % 5) != 0) || (row(1) == 5) val checkString = (row(2) == "abc") val checkLong = (row(3) == (1L<<33)) val checkFloat = (row(4) == 2.5F) From 6a6bf9844e1c25e3f3360cc4c479f5db66e2bea7 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 19 Feb 2014 18:31:40 +0200 Subject: [PATCH 724/778] Added column projections to ParquetTableScan --- .../apache/spark/sql/ParquetRelation.scala | 34 +++++++-- .../apache/spark/sql/ParquetTableScan.scala | 72 ++++++++++++++----- .../apache/spark/sql/ParquetQueryTests.scala | 36 +++++++--- 3 files changed, 107 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala index 82b81d3d93fc2..08587d525006d 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala @@ -7,11 +7,14 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.ArrayType import org.apache.spark.sql.catalyst.expressions.{Row, AttributeReference, Attribute} -import parquet.schema.{MessageTypeParser, MessageType} +import parquet.schema.{GroupType, MessageTypeParser, MessageType} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} +import parquet.schema.{PrimitiveType => ParquetPrimitiveType} +import parquet.schema.{Type => ParquetType} import scala.collection.JavaConversions._ import parquet.io.api.{Binary, RecordConsumer} +import parquet.schema.Type.Repetition /** * Relation formed by underlying Parquet file that contains data stored in columnar form. @@ -28,7 +31,7 @@ case class ParquetRelation(val tableName: String, val parquetSchema: MessageType val output = attributes - /* TODO: implement low-level Parquet metadata store access */ + /* TODO: implement low-level Parquet metadata store access if needed */ // val metaData: ParquetMetadata // def getBlocks: java.util.List[BlockMetaData] = metaData.getBlocks // def getColumns: java.util.List[ColumnDescriptor] = metaData.getFileMetaData.getSchema.getColumns @@ -42,7 +45,7 @@ object ParquetRelation { } object ParquetTypesConverter { - def toDataType(parquetType : ParquetPrimitiveTypeName) : DataType = parquetType match { + def toDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match { // for now map binary to string type // TODO: figure out how Parquet uses strings or why we can't use them in a MessageType schema case ParquetPrimitiveTypeName.BINARY => StringType @@ -56,6 +59,17 @@ object ParquetTypesConverter { case _ => sys.error(s"Unsupported parquet datatype") } + def fromDataType(ctype: DataType): ParquetPrimitiveTypeName = ctype match { + case StringType => ParquetPrimitiveTypeName.BINARY + case BooleanType => ParquetPrimitiveTypeName.BOOLEAN + case DoubleType => ParquetPrimitiveTypeName.DOUBLE + case ArrayType(ByteType) => ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY + case FloatType => ParquetPrimitiveTypeName.FLOAT + case IntegerType => ParquetPrimitiveTypeName.INT32 + case LongType => ParquetPrimitiveTypeName.INT64 + case _ => sys.error(s"Unsupported datatype") + } + def consumeType(consumer: RecordConsumer, ctype: DataType, record: Row, index: Int): Unit = { ctype match { case StringType => consumer.addBinary( @@ -75,13 +89,21 @@ object ParquetTypesConverter { def getSchema(schemaString : String) : MessageType = MessageTypeParser.parseMessageType(schemaString) - def convertToAttributes(parquetSchema: MessageType) : List[Attribute] = { + def convertToAttributes(parquetSchema: MessageType) : Seq[Attribute] = { parquetSchema.getColumns.map { case (desc) => { val ctype = toDataType(desc.getType) - val name = desc.getPath.toString + val name: String = desc.getPath.mkString(".") new AttributeReference(name, ctype, false)() } - }.toList + } + } + + // TODO: allow nesting? + def convertFromAttributes(attributes: Seq[Attribute]): MessageType = { + val fields: Seq[ParquetType] = attributes.map { + a => new ParquetPrimitiveType(Repetition.OPTIONAL, fromDataType(a.dataType), a.name) + } + new MessageType("root", fields) } } diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala index bb937485b6326..9f09de2358157 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala @@ -5,6 +5,7 @@ import parquet.schema.{MessageTypeParser, MessageType} import parquet.hadoop.ParquetInputFormat import parquet.hadoop.api.{WriteSupport, ReadSupport} import parquet.hadoop.api.ReadSupport.ReadContext +import parquet.io.InvalidRecordException import catalyst.expressions.{Attribute, GenericRow, Row, Expression} import catalyst.plans.logical.LeafNode @@ -15,27 +16,25 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD -import parquet.hadoop.api.WriteSupport /** * Parquet table scan operator. Imports the file that backs the given [[org.apache.spark.sql.ParquetRelation]] * as a RDD[Row]. Only a stub currently. */ case class ParquetTableScan( - attributes: Seq[Attribute], - relation: ParquetRelation, - partitionPruningPred: Option[Expression] // not used - )( - @transient val sc: SparkContext) + relation: ParquetRelation, + columnPruningPred: Option[Expression])( + @transient val sc: SparkContext) extends LeafNode { + // TODO: make use of column pruning predicate + + private var prunedAttributes = relation.attributes // used for projections + /** * Runs this query returning the result as an RDD. */ def execute(): RDD[Row] = { - // TODO: for now we do not check whether the relation's schema matches the one of the - // underlying Parquet file - val job = new Job(new Configuration()) ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.RowReadSupport]) // TODO: add record filters, etc. @@ -47,7 +46,43 @@ case class ParquetTableScan( .map(_._2) } - override def output: Seq[Attribute] = attributes // right now we pass everything through, always + /** + * Applies a (candidate) projection. + * + * @param attributes The list of attributes to be used in the projection. + * @return True, if successful; false otherwise. + */ + def pruneColumns(attributes: Seq[Attribute]): Boolean = { + if(!validateProjection(attributes)) + false + else { + prunedAttributes = attributes + true + } + } + + /** + * Evaluates a candidate projection by checking whether the candidate is a subtype of the + * original type. + * + * @param projection The candidate projection. + * @return True if the projection is valid, false otherwise. + */ + private def validateProjection(projection: Seq[Attribute]): Boolean = { + val original: MessageType = relation.parquetSchema + val candidate: MessageType = ParquetTypesConverter.convertFromAttributes(projection) + var retval = true + try { + original.checkContains(candidate) + } catch { + case e: InvalidRecordException => { + retval = false + } + } + retval + } + + override def output: Seq[Attribute] = prunedAttributes } /** @@ -69,20 +104,18 @@ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterial */ class RowReadSupport extends ReadSupport[Row] with Logging { override def prepareForRead( - conf: Configuration, - stringMap: java.util.Map[String, String], - fileSchema: MessageType, - readContext: ReadContext - ): RecordMaterializer[Row] = { + conf: Configuration, + stringMap: java.util.Map[String, String], + fileSchema: MessageType, + readContext: ReadContext): RecordMaterializer[Row] = { logger.debug(s"preparing for read with schema ${fileSchema.toString}") new RowRecordMaterializer(fileSchema) } override def init( - configuration: Configuration, - keyValueMetaData: java.util.Map[String, String], - fileSchema: MessageType - ): ReadContext = { + configuration: Configuration, + keyValueMetaData: java.util.Map[String, String], + fileSchema: MessageType): ReadContext = { logger.debug(s"read support initialized for schema ${fileSchema.toString}") new ReadContext(fileSchema, keyValueMetaData) } @@ -97,6 +130,7 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { def setSchema(schema: MessageType, configuration: Configuration) { // for testing this.schema = schema + // TODO: could use Attributes themselves instead of Parquet schema? configuration.set(PARQUET_ROW_SCHEMA, schema.toString) } diff --git a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala index ee83219e55c39..7c2a4185cdcaa 100644 --- a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala @@ -23,14 +23,23 @@ import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, Row} object ParquetTestData { - val testSchema = """message myrecord { - optional boolean myboolean; - optional int32 myint; - optional binary mystring; - optional int64 mylong; - optional float myfloat; - optional double mydouble; - }""" + val testSchema = + """message myrecord { + |optional boolean myboolean; + |optional int32 myint; + |optional binary mystring; + |optional int64 mylong; + |optional float myfloat; + |optional double mydouble; + |}""".stripMargin + + val subTestSchema = + """ + |message myrecord { + |optional boolean myboolean; + |optional int64 mylong; + |} + """.stripMargin val testFile = new File("/tmp/testParquetFile").getAbsoluteFile @@ -322,6 +331,14 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { assert(allChecks) } + test("Projection of simple Parquet file") { + val scanner = new ParquetTableScan(ParquetTestData.testData, None)(TestSqlContext.sparkContext) + val first = scanner.pruneColumns(ParquetTypesConverter.convertToAttributes(MessageTypeParser.parseMessageType(ParquetTestData.subTestSchema))) + assert(first) + val rdd = scanner.execute() + assert(rdd != null) + } + /** * Computes the given [[org.apache.spark.sql.ParquetRelation]] and returns its RDD. * @@ -329,8 +346,7 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { * @return An RDD of Rows. */ private def getRDD(parquetRelation: ParquetRelation): RDD[Row] = { - val catalystSchema: List[Attribute] = ParquetTypesConverter.convertToAttributes(parquetRelation.parquetSchema) - val scanner = new ParquetTableScan(catalystSchema, parquetRelation, None)(TestSqlContext.sparkContext) + val scanner = new ParquetTableScan(parquetRelation, None)(TestSqlContext.sparkContext) scanner.execute } } \ No newline at end of file From f347273cb9d8f6e6c43eb3ef5e54507025ecc1cd Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Thu, 20 Feb 2014 19:01:37 +0200 Subject: [PATCH 725/778] Adding ParquetMetaData extraction, fixing schema projection --- .../apache/spark/sql/ParquetRelation.scala | 35 +++++++++++-------- .../apache/spark/sql/ParquetTableScan.scala | 24 +++++++++---- .../apache/spark/sql/ParquetQueryTests.scala | 19 +++++++--- .../spark/sql/catalyst/expressions/Row.scala | 6 ++++ 4 files changed, 59 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala index 08587d525006d..ac51039c1d0d1 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala @@ -1,47 +1,53 @@ package org.apache.spark.sql import org.apache.hadoop.fs.Path +import org.apache.hadoop.conf.Configuration import org.apache.spark.sql.catalyst.plans.logical.BaseRelation import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.ArrayType import org.apache.spark.sql.catalyst.expressions.{Row, AttributeReference, Attribute} -import parquet.schema.{GroupType, MessageTypeParser, MessageType} +import parquet.schema.{MessageTypeParser, MessageType} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} import parquet.schema.{PrimitiveType => ParquetPrimitiveType} import parquet.schema.{Type => ParquetType} -import scala.collection.JavaConversions._ import parquet.io.api.{Binary, RecordConsumer} import parquet.schema.Type.Repetition +import parquet.hadoop.ParquetFileReader +import parquet.hadoop.metadata.ParquetMetadata + +import scala.collection.JavaConversions._ /** * Relation formed by underlying Parquet file that contains data stored in columnar form. * - * @param parquetSchema The requested schema (may contain projections w.r.t. Parquet file). + * @param tableName The name of the relation. + * @param conf Hadoop configuration. * @param path The path to the Parquet file. */ -case class ParquetRelation(val tableName: String, val parquetSchema: MessageType, val path: Path) +case class ParquetRelation(val tableName: String, val conf: Configuration, val path: Path) extends BaseRelation { - /** Attributes */ + private val parquetMetaData: ParquetMetadata = readMetaData + + /** Schema derived from ParquetFile **/ + val parquetSchema: MessageType = parquetMetaData.getFileMetaData.getSchema + + /** Attributes **/ val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) + /** Output **/ val output = attributes - /* TODO: implement low-level Parquet metadata store access if needed */ - // val metaData: ParquetMetadata - // def getBlocks: java.util.List[BlockMetaData] = metaData.getBlocks - // def getColumns: java.util.List[ColumnDescriptor] = metaData.getFileMetaData.getSchema.getColumns - - val numberOfBlocks = 1 // TODO: see comment above + private def readMetaData: ParquetMetadata = ParquetFileReader.readFooter(conf, path) } object ParquetRelation { - def apply(schemaString : String, path: Path, tableName: String = "ParquetTable") = - new ParquetRelation(tableName, ParquetTypesConverter.getSchema(schemaString), path) + def apply(path: Path, tableName: String = "ParquetTable") = + new ParquetRelation(tableName, new Configuration(), path) } object ParquetTypesConverter { @@ -80,8 +86,7 @@ object ParquetTypesConverter { case IntegerType => consumer.addInteger(record.getInt(index)) case LongType => consumer.addLong(record.getLong(index)) case DoubleType => consumer.addDouble(record.getDouble(index)) - // TODO: where is getFloat in Row?! - case FloatType => consumer.addFloat(record(index).asInstanceOf[Float]) + case FloatType => consumer.addFloat(record.getFloat(index)) case BooleanType => consumer.addBoolean(record.getBoolean(index)) case _ => sys.error(s"Unsupported datatype, cannot write to consumer") } diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala index 9f09de2358157..71935ca1bb64c 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala @@ -1,11 +1,12 @@ package org.apache.spark.sql import parquet.io.api._ +import parquet.io.InvalidRecordException import parquet.schema.{MessageTypeParser, MessageType} -import parquet.hadoop.ParquetInputFormat +import parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat} import parquet.hadoop.api.{WriteSupport, ReadSupport} import parquet.hadoop.api.ReadSupport.ReadContext -import parquet.io.InvalidRecordException +import parquet.column.ParquetProperties import catalyst.expressions.{Attribute, GenericRow, Row, Expression} import catalyst.plans.logical.LeafNode @@ -37,6 +38,9 @@ case class ParquetTableScan( def execute(): RDD[Row] = { val job = new Job(new Configuration()) ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.RowReadSupport]) + job.getConfiguration.set( + RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, + ParquetTypesConverter.convertFromAttributes(prunedAttributes).toString) // TODO: add record filters, etc. sc.newAPIHadoopFile( relation.path.toUri.toString, @@ -103,35 +107,43 @@ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterial * A [[parquet.hadoop.api.ReadSupport]] for Row objects. */ class RowReadSupport extends ReadSupport[Row] with Logging { + override def prepareForRead( conf: Configuration, stringMap: java.util.Map[String, String], fileSchema: MessageType, readContext: ReadContext): RecordMaterializer[Row] = { logger.debug(s"preparing for read with schema ${fileSchema.toString}") - new RowRecordMaterializer(fileSchema) + new RowRecordMaterializer(readContext.getRequestedSchema) } override def init( configuration: Configuration, keyValueMetaData: java.util.Map[String, String], fileSchema: MessageType): ReadContext = { - logger.debug(s"read support initialized for schema ${fileSchema.toString}") - new ReadContext(fileSchema, keyValueMetaData) + val requested_schema_string = configuration.get(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, fileSchema.toString) + val requested_schema = MessageTypeParser.parseMessageType(requested_schema_string) + logger.debug(s"read support initialized for original schema ${requested_schema.toString}") + new ReadContext(requested_schema, keyValueMetaData) } } +object RowReadSupport { + val PARQUET_ROW_REQUESTED_SCHEMA = "org.apache.spark.sql.parquet.row.requested_schema" +} + /** * A [[parquet.hadoop.api.WriteSupport]] for Row ojects. */ class RowWriteSupport extends WriteSupport[Row] with Logging { - final val PARQUET_ROW_SCHEMA: String = "spark.sql.parquet.row.schema" + final val PARQUET_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.schema" def setSchema(schema: MessageType, configuration: Configuration) { // for testing this.schema = schema // TODO: could use Attributes themselves instead of Parquet schema? configuration.set(PARQUET_ROW_SCHEMA, schema.toString) + configuration.set(ParquetOutputFormat.WRITER_VERSION, ParquetProperties.WriterVersion.PARQUET_1_0.toString) } def getSchema(configuration: Configuration): MessageType = { diff --git a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala index 7c2a4185cdcaa..4af2c02bbda11 100644 --- a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala @@ -43,7 +43,7 @@ object ParquetTestData { val testFile = new File("/tmp/testParquetFile").getAbsoluteFile - val testData = ParquetRelation(testSchema, new Path(testFile.toURI)) + lazy val testData = ParquetRelation(new Path(testFile.toURI)) def writeFile = { testFile.delete @@ -201,7 +201,7 @@ object ParquetTestData { val complexTestFile: File = new File("/tmp/testComplexParquetFile").getAbsoluteFile - val complexTestData = ParquetRelation(complexSchema, new Path(complexTestFile.toURI)) + lazy val complexTestData = ParquetRelation(new Path(complexTestFile.toURI)) // this second test is from TestParquetFileWriter def writeComplexFile = { @@ -335,8 +335,19 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { val scanner = new ParquetTableScan(ParquetTestData.testData, None)(TestSqlContext.sparkContext) val first = scanner.pruneColumns(ParquetTypesConverter.convertToAttributes(MessageTypeParser.parseMessageType(ParquetTestData.subTestSchema))) assert(first) - val rdd = scanner.execute() - assert(rdd != null) + val result = scanner.execute().collect() + val allChecks: Boolean = result.zipWithIndex.forall { + case (row, index) => { + val checkBoolean = + if (index % 3 == 0) + (row(0) == true) + else + (row(0) == false) + val checkLong = (row(1) == (1L<<33)) + checkBoolean && checkLong && (row.size == 2) + } + } + assert(allChecks) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index d5c8785f6e884..881ef64471e68 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -36,6 +36,7 @@ abstract class Row extends Seq[Any] with Serializable { def getInt(i: Int): Int def getLong(i: Int): Long def getDouble(i: Int): Double + def getFloat(i: Int): Float def getBoolean(i: Int): Boolean def getShort(i: Int): Short def getByte(i: Int): Byte @@ -57,6 +58,7 @@ object EmptyRow extends Row { def getInt(i: Int): Int = throw new UnsupportedOperationException def getLong(i: Int): Long = throw new UnsupportedOperationException def getDouble(i: Int): Double = throw new UnsupportedOperationException + def getFloat(i: Int): Float = throw new UnsupportedOperationException def getBoolean(i: Int): Boolean = throw new UnsupportedOperationException def getShort(i: Int): Short = throw new UnsupportedOperationException def getByte(i: Int): Byte = throw new UnsupportedOperationException @@ -88,6 +90,10 @@ class GenericRow(input: Seq[Any]) extends Row { if (values(i) == null) sys.error("Failed to check null bit for primitive double value.") values(i).asInstanceOf[Double] } + def getFloat(i: Int): Float = { + if (values(i) == null) sys.error("Failed to check null bit for primitive float value.") + values(i).asInstanceOf[Float] + } def getBoolean(i: Int): Boolean = { if (values(i) == null) sys.error("Failed to check null bit for primitive boolean value.") values(i).asInstanceOf[Boolean] From 75262eec5e21400011359dbf3f2825cbd7be461d Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Mon, 24 Feb 2014 11:27:25 +0200 Subject: [PATCH 726/778] Integrating operations on Parquet files into SharkStrategies --- .../apache/spark/sql/ParquetRelation.scala | 72 +++- .../apache/spark/sql/ParquetTableScan.scala | 94 ++++-- .../apache/spark/sql/ParquetQueryTests.scala | 312 +----------------- .../spark/sql/shark/ParquetQuerySuite.scala | 48 +++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 7 +- .../spark/sql/hive/HiveStrategies.scala | 23 +- 6 files changed, 212 insertions(+), 344 deletions(-) create mode 100644 shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala index ac51039c1d0d1..f490502a6ba87 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala @@ -6,7 +6,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.sql.catalyst.plans.logical.BaseRelation import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.ArrayType -import org.apache.spark.sql.catalyst.expressions.{Row, AttributeReference, Attribute} +import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, AttributeReference, Attribute} import parquet.schema.{MessageTypeParser, MessageType} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} @@ -15,26 +15,26 @@ import parquet.schema.{Type => ParquetType} import parquet.io.api.{Binary, RecordConsumer} import parquet.schema.Type.Repetition -import parquet.hadoop.ParquetFileReader +import parquet.hadoop.{ParquetWriter, ParquetFileReader} import parquet.hadoop.metadata.ParquetMetadata import scala.collection.JavaConversions._ +import java.io.File /** * Relation formed by underlying Parquet file that contains data stored in columnar form. * * @param tableName The name of the relation. - * @param conf Hadoop configuration. * @param path The path to the Parquet file. */ -case class ParquetRelation(val tableName: String, val conf: Configuration, val path: Path) +case class ParquetRelation(val tableName: String, val path: String) extends BaseRelation { - private val parquetMetaData: ParquetMetadata = readMetaData + private def parquetMetaData: ParquetMetadata = readMetaData /** Schema derived from ParquetFile **/ - val parquetSchema: MessageType = parquetMetaData.getFileMetaData.getSchema + def parquetSchema: MessageType = parquetMetaData.getFileMetaData.getSchema /** Attributes **/ val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) @@ -42,12 +42,15 @@ case class ParquetRelation(val tableName: String, val conf: Configuration, val p /** Output **/ val output = attributes - private def readMetaData: ParquetMetadata = ParquetFileReader.readFooter(conf, path) + private def readMetaData: ParquetMetadata = ParquetFileReader.readFooter(new Configuration(), new Path(path)) + + override def isPartitioned = false // Parquet files have no concepts of keys, therefore no Partitioner + // Note: we could allow Block level access; needs to be thought through } object ParquetRelation { def apply(path: Path, tableName: String = "ParquetTable") = - new ParquetRelation(tableName, new Configuration(), path) + new ParquetRelation(tableName, path.toUri.toString) } object ParquetTypesConverter { @@ -112,3 +115,56 @@ object ParquetTypesConverter { new MessageType("root", fields) } } + +object ParquetTestData { + + val testSchema = + """message myrecord { + |optional boolean myboolean; + |optional int32 myint; + |optional binary mystring; + |optional int64 mylong; + |optional float myfloat; + |optional double mydouble; + |}""".stripMargin + + val subTestSchema = + """ + |message myrecord { + |optional boolean myboolean; + |optional int64 mylong; + |} + """.stripMargin + + val testFile = new File("/tmp/testParquetFile").getAbsoluteFile + + lazy val testData = ParquetRelation(new Path(testFile.toURI)) + + def writeFile = { + testFile.delete + val path: Path = new Path(testFile.toURI) + val configuration: Configuration = new Configuration + val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) + + val writeSupport = new RowWriteSupport() + writeSupport.setSchema(schema, configuration) + val writer = new ParquetWriter(path, writeSupport) + for(i <- 0 until 15) { + val data = new Array[Any](6) + if(i % 3 ==0) + data.update(0, true) + else + data.update(0, false) + if(i % 5 == 0) + data.update(1, 5) + else + data.update(1, null) // optional + data.update(2, "abc") + data.update(3, 1L<<33) + data.update(4, 2.5F) + data.update(5, 4.5D) + writer.write(new GenericRow(data)) + } + writer.close() + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala index 71935ca1bb64c..94f86de61e4b4 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala @@ -9,41 +9,40 @@ import parquet.hadoop.api.ReadSupport.ReadContext import parquet.column.ParquetProperties import catalyst.expressions.{Attribute, GenericRow, Row, Expression} -import catalyst.plans.logical.LeafNode +import org.apache.spark.sql.execution.{UnaryNode, LeafNode} import catalyst.types._ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{Path, FileSystem} -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.api.java.JavaPairRDD /** * Parquet table scan operator. Imports the file that backs the given [[org.apache.spark.sql.ParquetRelation]] * as a RDD[Row]. Only a stub currently. */ case class ParquetTableScan( + attributes: Seq[Attribute], relation: ParquetRelation, columnPruningPred: Option[Expression])( - @transient val sc: SparkContext) + @transient val sc: SparkSqlContext) extends LeafNode { - // TODO: make use of column pruning predicate - - private var prunedAttributes = relation.attributes // used for projections - /** * Runs this query returning the result as an RDD. */ - def execute(): RDD[Row] = { + override def execute(): RDD[Row] = { val job = new Job(new Configuration()) ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.RowReadSupport]) job.getConfiguration.set( RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, - ParquetTypesConverter.convertFromAttributes(prunedAttributes).toString) + ParquetTypesConverter.convertFromAttributes(attributes).toString) // TODO: add record filters, etc. - sc.newAPIHadoopFile( - relation.path.toUri.toString, + sc.sparkContext.newAPIHadoopFile( + relation.path, classOf[ParquetInputFormat[Row]], classOf[Void], classOf[Row], job.getConfiguration) @@ -53,16 +52,15 @@ case class ParquetTableScan( /** * Applies a (candidate) projection. * - * @param attributes The list of attributes to be used in the projection. - * @return True, if successful; false otherwise. + * @param prunedAttributes The list of attributes to be used in the projection. + * @return Pruned TableScan. */ - def pruneColumns(attributes: Seq[Attribute]): Boolean = { - if(!validateProjection(attributes)) - false - else { - prunedAttributes = attributes - true - } + def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = { + val success = validateProjection(prunedAttributes) + if(success) + ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sc) + else + ParquetTableScan(attributes, relation, columnPruningPred)(sc) } /** @@ -86,9 +84,53 @@ case class ParquetTableScan( retval } - override def output: Seq[Attribute] = prunedAttributes + override def output: Seq[Attribute] = attributes +} + +case class InsertIntoParquetTable( + relation: ParquetRelation, + child: SparkPlan)( + @transient val sc: SparkSqlContext) + extends UnaryNode { + + /** + * Inserts all the rows in the Parquet file. + */ + override def execute() = { + val childRdd = child.execute() + assert(childRdd != null) + + val job = new Job(new Configuration()) + ParquetOutputFormat.setWriteSupportClass(job, classOf[org.apache.spark.sql.RowWriteSupport]) + val conf = job.getConfiguration + // TODO: move that to function in object + conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString) + // TODO: we should pass the read schema, too, in case we insert from a select from a Parquet table? + // conf.set(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, relation.parquetSchema.toString) + + // TODO: add checks: file exists, etc. + val fs = FileSystem.get(conf) + fs.delete(new Path(relation.path), true) + + JavaPairRDD.fromRDD(childRdd.map(Tuple2(0, _))).saveAsNewAPIHadoopFile( + relation.path.toString, + classOf[Void], + classOf[org.apache.spark.sql.catalyst.expressions.GenericRow], + classOf[parquet.hadoop.ParquetOutputFormat[org.apache.spark.sql.catalyst.expressions.GenericRow]], + conf) + + // From InsertIntoHiveTable: + // It would be nice to just return the childRdd unchanged so insert operations could be chained, + // however for now we return an empty list to simplify compatibility checks with hive, which + // does not return anything for insert operations. + // TODO: implement hive compatibility as rules. + sc.sparkContext.makeRDD(Nil, 1) + } + + override def output = child.output } + /** * A [[parquet.io.api.RecordMaterializer]] for Rows. * @@ -136,18 +178,16 @@ object RowReadSupport { * A [[parquet.hadoop.api.WriteSupport]] for Row ojects. */ class RowWriteSupport extends WriteSupport[Row] with Logging { - final val PARQUET_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.schema" - def setSchema(schema: MessageType, configuration: Configuration) { // for testing this.schema = schema // TODO: could use Attributes themselves instead of Parquet schema? - configuration.set(PARQUET_ROW_SCHEMA, schema.toString) + configuration.set(RowWriteSupport.PARQUET_ROW_SCHEMA, schema.toString) configuration.set(ParquetOutputFormat.WRITER_VERSION, ParquetProperties.WriterVersion.PARQUET_1_0.toString) } def getSchema(configuration: Configuration): MessageType = { - return MessageTypeParser.parseMessageType(configuration.get(PARQUET_ROW_SCHEMA)) + return MessageTypeParser.parseMessageType(configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) } private var schema: MessageType = null @@ -183,6 +223,10 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { } } +object RowWriteSupport { + val PARQUET_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.schema" +} + /** * A [[parquet.io.api.GroupConverter]] that is able to convert a Parquet record * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. diff --git a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala index 4af2c02bbda11..279d9be176333 100644 --- a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala @@ -2,303 +2,11 @@ package org.apache.spark.sql import org.scalatest.{BeforeAndAfterAll, FunSuite} -import java.io.File -import java.util.Arrays - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.conf.Configuration - import org.apache.spark.rdd.RDD -import parquet.schema.{MessageTypeParser, MessageType} -import parquet.column.ColumnDescriptor -import parquet.hadoop.metadata.{ParquetMetadata, CompressionCodecName} -import parquet.hadoop.{ParquetWriter, ParquetFileReader, ParquetFileWriter} -import parquet.bytes.BytesInput -import parquet.column.Encoding._ -import parquet.column.page.PageReadStore - -import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, Row} - -object ParquetTestData { - - - val testSchema = - """message myrecord { - |optional boolean myboolean; - |optional int32 myint; - |optional binary mystring; - |optional int64 mylong; - |optional float myfloat; - |optional double mydouble; - |}""".stripMargin - - val subTestSchema = - """ - |message myrecord { - |optional boolean myboolean; - |optional int64 mylong; - |} - """.stripMargin - - val testFile = new File("/tmp/testParquetFile").getAbsoluteFile - - lazy val testData = ParquetRelation(new Path(testFile.toURI)) - - def writeFile = { - testFile.delete - val path: Path = new Path(testFile.toURI) - val configuration: Configuration = new Configuration - val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) - - val writeSupport = new RowWriteSupport() - writeSupport.setSchema(schema, configuration) - val writer = new ParquetWriter(path, writeSupport) - for(i <- 0 until 15) { - val data = new Array[Any](6) - if(i % 3 ==0) - data.update(0, true) - else - data.update(0, false) - if(i % 5 == 0) - data.update(1, 5) - else - data.update(1, null) // optional - data.update(2, "abc") - data.update(3, 1L<<33) - data.update(4, 2.5F) - data.update(5, 4.5D) - writer.write(new GenericRow(data)) - } - writer.close() - } - - def writeFileAsPages = { - testFile.delete - val path: Path = new Path(testFile.toURI) - val configuration: Configuration = new Configuration - val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) - - val path1: Array[String] = Array("myboolean") - val c1: ColumnDescriptor = schema.getColumnDescription(path1) - val path2: Array[String] = Array("myint") - val c2: ColumnDescriptor = schema.getColumnDescription(path2) - val path3: Array[String] = Array("mystring") - val c3: ColumnDescriptor = schema.getColumnDescription(path3) - val path4: Array[String] = Array("mylong") - val c4: ColumnDescriptor = schema.getColumnDescription(path4) - val path5: Array[String] = Array("myfloat") - val c5: ColumnDescriptor = schema.getColumnDescription(path5) - val path6: Array[String] = Array("mydouble") - val c6: ColumnDescriptor = schema.getColumnDescription(path6) - - val codec: CompressionCodecName = CompressionCodecName.UNCOMPRESSED - val w: ParquetFileWriter = new ParquetFileWriter(configuration, schema, path) - - w.start - w.startBlock(3) - w.startColumn(c1, 3, codec) - // note to myself: repetition levels cannot be PLAIN encoded - // boolean - w.writeDataPage(3, 3, BytesInput.from( - concat(serializeValue(true), serializeValue(false), serializeValue(false)) - ), RLE, RLE, PLAIN) - w.endColumn - w.startColumn(c2, 3, codec) - // int - w.writeDataPage(3, 12, BytesInput.from( - concat(serializeValue(5), serializeValue(5), serializeValue(5)) - ), RLE, RLE, PLAIN) - w.endColumn - w.startColumn(c3, 3, codec) - // string - val bytes = serializeValue("abc".asInstanceOf[AnyVal]) - w.writeDataPage(3, 3*bytes.length, BytesInput.from( - concat(bytes, bytes, bytes) - ), RLE, RLE, PLAIN) - w.endColumn() - w.startColumn(c4, 3, codec) - // long - w.writeDataPage(3, 24, BytesInput.from( - concat(serializeValue(1L<<33), serializeValue(1L<<33), serializeValue(1L<<33)) - ), RLE, RLE, PLAIN) - w.endColumn() - w.startColumn(c5, 3, codec) - // float - w.writeDataPage(3, 12, BytesInput.from( - concat(serializeValue(2.5F), serializeValue(2.5F), serializeValue(2.5F)) - ), RLE, RLE, PLAIN) - w.endColumn() - w.startColumn(c6, 3, codec) - // double - w.writeDataPage(3, 24, BytesInput.from( - concat(serializeValue(4.5D), serializeValue(4.5D), serializeValue(4.5D)) - ), RLE, RLE, PLAIN) - w.endColumn() - w.endBlock - w.startBlock(3) - w.startColumn(c1, 3, codec) - // note to myself: repetition levels cannot be PLAIN encoded - // boolean - w.writeDataPage(3, 3, BytesInput.from( - concat(serializeValue(true), serializeValue(false), serializeValue(false)) - ), RLE, RLE, PLAIN) - w.endColumn - w.startColumn(c2, 3, codec) - // int - w.writeDataPage(3, 12, BytesInput.from( - concat(serializeValue(5), serializeValue(5), serializeValue(5)) - ), RLE, RLE, PLAIN) - w.endColumn - w.startColumn(c3, 3, codec) - // string - w.writeDataPage(3, 3*bytes.length, BytesInput.from( - concat(bytes, bytes, bytes) - ), RLE, RLE, PLAIN) - w.endColumn() - w.startColumn(c4, 3, codec) - // long - w.writeDataPage(3, 24, BytesInput.from( - concat(serializeValue(1L<<33), serializeValue(1L<<33), serializeValue(1L<<33)) - ), RLE, RLE, PLAIN) - w.endColumn() - w.startColumn(c5, 3, codec) - // float - w.writeDataPage(3, 12, BytesInput.from( - concat(serializeValue(2.5F), serializeValue(2.5F), serializeValue(2.5F)) - ), RLE, RLE, PLAIN) - w.endColumn() - w.startColumn(c6, 3, codec) - // double - w.writeDataPage(3, 24, BytesInput.from( - concat(serializeValue(4.5D), serializeValue(4.5D), serializeValue(4.5D)) - ), RLE, RLE, PLAIN) - w.endColumn() - w.endBlock - w.end(new java.util.HashMap[String, String]) - } - - def readFile = { - val configuration: Configuration = new Configuration - val path = new Path(testFile.toURI) - val schema: MessageType = MessageTypeParser.parseMessageType(ParquetTestData.testSchema) - val path1: Array[String] = Array("myboolean") - val c1: ColumnDescriptor = schema.getColumnDescription(path1) - println(c1.toString) - val path2: Array[String] = Array("myint") - val readFooter: ParquetMetadata = ParquetFileReader.readFooter(configuration, path) - println("this many blocks: " + readFooter.getBlocks.size()) - println("metadata: " + readFooter.getFileMetaData.toString) - val r: ParquetFileReader = new ParquetFileReader(configuration, path, readFooter.getBlocks, Arrays.asList(schema.getColumnDescription(path1), schema.getColumnDescription(path2))) - var pages: PageReadStore = r.readNextRowGroup - println("number of rows first group " + pages.getRowCount) - var pageReader = pages.getPageReader(c1) - var page = pageReader.readPage() - assert(page != null) - } - - val complexSchema = "message m { required group a {required binary b;} required group c { required int64 d; }}" - - val complexTestFile: File = new File("/tmp/testComplexParquetFile").getAbsoluteFile - - lazy val complexTestData = ParquetRelation(new Path(complexTestFile.toURI)) +import parquet.schema.MessageTypeParser - // this second test is from TestParquetFileWriter - def writeComplexFile = { - complexTestFile.delete - val path: Path = new Path(complexTestFile.toURI) - val configuration: Configuration = new Configuration - - val schema: MessageType = MessageTypeParser.parseMessageType(ParquetTestData.complexSchema) - val path1: Array[String] = Array("a", "b") - val c1: ColumnDescriptor = schema.getColumnDescription(path1) - val path2: Array[String] = Array("c", "d") - val c2: ColumnDescriptor = schema.getColumnDescription(path2) - - val bytes1: Array[Byte] = Array(0, 1, 2, 3).map(_.toByte) - val bytes2: Array[Byte] = Array(1, 2, 3, 4).map(_.toByte) - val bytes3: Array[Byte] = Array(2, 3, 4, 5).map(_.toByte) - val bytes4: Array[Byte] = Array(3, 4, 5, 6).map(_.toByte) - - val codec: CompressionCodecName = CompressionCodecName.UNCOMPRESSED - val w: ParquetFileWriter = new ParquetFileWriter(configuration, schema, path) - - w.start - w.startBlock(3) - w.startColumn(c1, 5, codec) - val c1Starts: Long = w.getPos - w.writeDataPage(2, 4, BytesInput.from(bytes1), RLE, RLE, PLAIN) - w.writeDataPage(3, 4, BytesInput.from(bytes1), RLE, RLE, PLAIN) - w.endColumn - val c1Ends: Long = w.getPos - w.startColumn(c2, 6, codec) - val c2Starts: Long = w.getPos - w.writeDataPage(2, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) - w.writeDataPage(3, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) - w.writeDataPage(1, 4, BytesInput.from(bytes2), RLE, RLE, PLAIN) - w.endColumn - val c2Ends: Long = w.getPos - w.endBlock - w.startBlock(4) - w.startColumn(c1, 7, codec) - w.writeDataPage(7, 4, BytesInput.from(bytes3), RLE, RLE, PLAIN) - w.endColumn - w.startColumn(c2, 8, codec) - w.writeDataPage(8, 4, BytesInput.from(bytes4), RLE, RLE, PLAIN) - w.endColumn - w.endBlock - w.end(new java.util.HashMap[String, String]) - } - - /** - * Serializes a given value so that it conforms to Parquet's uncompressed primitive value encoding. - * - * @param value The value to serialize. - * @return A byte array that contains the serialized value. - */ - private def serializeValue(value: Any) : Array[Byte] = { - value match { - case i: Int => { - val bb = java.nio.ByteBuffer.allocate(4) - bb.order(java.nio.ByteOrder.LITTLE_ENDIAN) - bb.putInt(i) - bb.array() - } - case l: Long => { - val bb = java.nio.ByteBuffer.allocate(8) - bb.order(java.nio.ByteOrder.LITTLE_ENDIAN) - bb.putLong(l) - bb.array() - } - case f: Float => serializeValue(java.lang.Float.floatToIntBits(f)) - case d: Double => serializeValue(java.lang.Double.doubleToLongBits(d)) - case s: String => { - // apparently strings are encoded as their length as int followed by the string - val bytes: Array[Byte] = s.getBytes("UTF-8") - (serializeValue(bytes.length).toList ::: bytes.toList).toArray - } - case b: Boolean => { - if(b) - Array(1.toByte) - else - Array(0.toByte) - } - } - } - - /** - * Concatenates a sequence of byte arrays. - * - * @param values The sequence of byte arrays to be concatenated. - * @return The concatenation of the given byte arrays. - */ - private def concat(values: Array[Byte]*) : Array[Byte] = { - var retval = List[Byte]() - for(value <- values) - retval = retval ::: value.toList - retval.toArray - } -} +import org.apache.spark.sql.catalyst.expressions.Row class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { override def beforeAll() { @@ -307,8 +15,10 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") ParquetTestData.writeFile - ParquetTestData.readFile - ParquetTestData.writeComplexFile + } + + override def afterAll() { + ParquetTestData.testFile.delete() } test("Import of simple Parquet file") { @@ -332,10 +42,10 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { } test("Projection of simple Parquet file") { - val scanner = new ParquetTableScan(ParquetTestData.testData, None)(TestSqlContext.sparkContext) - val first = scanner.pruneColumns(ParquetTypesConverter.convertToAttributes(MessageTypeParser.parseMessageType(ParquetTestData.subTestSchema))) - assert(first) - val result = scanner.execute().collect() + val scanner = new ParquetTableScan(ParquetTestData.testData.attributes, ParquetTestData.testData, None)(TestSqlContext) + val projected = scanner.pruneColumns(ParquetTypesConverter.convertToAttributes(MessageTypeParser.parseMessageType(ParquetTestData.subTestSchema))) + assert(projected.attributes.size === 2) + val result = projected.execute().collect() val allChecks: Boolean = result.zipWithIndex.forall { case (row, index) => { val checkBoolean = @@ -357,7 +67,7 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { * @return An RDD of Rows. */ private def getRDD(parquetRelation: ParquetRelation): RDD[Row] = { - val scanner = new ParquetTableScan(parquetRelation, None)(TestSqlContext.sparkContext) + val scanner = new ParquetTableScan(parquetRelation.attributes, parquetRelation, None)(TestSqlContext) scanner.execute } } \ No newline at end of file diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala new file mode 100644 index 0000000000000..1be69df1aa603 --- /dev/null +++ b/shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala @@ -0,0 +1,48 @@ + +package org.apache.spark.sql +package shark + +import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.ParquetTestData + +class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { + + def runQuery(querystr: String): Array[Row] = { + TestShark.sql(querystr).rdd.collect() + } + + override def beforeAll() { + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests + // without restarting the JVM. + System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") + org.apache.spark.sql.ParquetTestData.testFile.delete() + ParquetTestData.writeFile + val testRel = ParquetTestData.testData + TestShark.catalog.overrideTable(Some[String]("default"), "testtable", testRel) + } + + override def afterAll() { + ParquetTestData.testFile.delete() + } + + test("SELECT on Parquet table") { + val rdd = runQuery("SELECT myboolean, mylong FROM default.testtable") + assert(rdd != null) + assert(rdd.forall(_.size == 2)) + } + + test("Filter on Parquet table") { + val rdd = runQuery("SELECT myboolean, mylong FROM default.testtable WHERE myboolean=true") + assert(rdd.size === 5) + assert(rdd.forall(_.getBoolean(0))) + } + + // TODO: fix insert into table + /*test("INSERT OVERWRITE Parquet table") { + val rdd = runQuery("INSERT OVERWRITE TABLE default.testtable SELECT * FROM default.testtable") + assert(rdd != null) + }*/ +} + 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 c942c790ec7be..bdec91641bc38 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 @@ -23,14 +23,9 @@ import scala.util.parsing.combinator.RegexParsers import org.apache.hadoop.hive.metastore.api.{FieldSchema, StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient -import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table} -import org.apache.hadoop.hive.metastore.api.{StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.Deserializer -import org.apache.hadoop.hive.serde2.AbstractDeserializer -import org.apache.hadoop.mapred.InputFormat import catalyst.analysis.Catalog import catalyst.expressions._ @@ -213,6 +208,8 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt new Partition(hiveQlTable, p) } + override def isPartitioned = hiveQlTable.isPartitioned + val tableDesc = new TableDesc( Class.forName(hiveQlTable.getSerializationLib).asInstanceOf[Class[Deserializer]], hiveQlTable.getInputFormatClass, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index ffdc13a9f7010..664033c1216b3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -21,7 +21,7 @@ package hive import catalyst.expressions._ import catalyst.planning._ import catalyst.plans._ -import catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} import org.apache.spark.sql.execution._ @@ -43,6 +43,8 @@ trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil + case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => + InsertIntoParquetTable(table, planLater(child))(hiveContext) :: Nil case _ => Nil } } @@ -52,8 +54,12 @@ trait HiveStrategies { // Push attributes into table scan when possible. case p @ logical.Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m, None)(hiveContext) :: Nil + case p @ logical.Project(projectList, r: ParquetRelation) if isSimpleProject(projectList) => + ParquetTableScan(projectList.asInstanceOf[Seq[Attribute]], r, None)(hiveContext) :: Nil case m: MetastoreRelation => HiveTableScan(m.output, m, None)(hiveContext) :: Nil + case p: ParquetRelation => + ParquetTableScan(p.output, p, None)(hiveContext) :: Nil case _ => Nil } } @@ -69,7 +75,7 @@ trait HiveStrategies { object PartitionPrunings extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case p @ FilteredOperation(predicates, relation: MetastoreRelation) - if relation.hiveQlTable.isPartitioned => + if relation.isPartitioned => val partitionKeyIds = relation.partitionKeys.map(_.id).toSet @@ -97,7 +103,7 @@ trait HiveStrategies { */ object ColumnPrunings extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) => + case PhysicalOperation(projectList, predicates, relation: BaseRelation) => val predicateOpt = predicates.reduceOption(And) val predicateRefs = predicateOpt.map(_.references).getOrElse(Set.empty) val projectRefs = projectList.flatMap(_.references) @@ -112,7 +118,7 @@ trait HiveStrategies { val prunedCols = (projectRefs ++ (predicateRefs -- projectRefs)).intersect(relation.output) val filteredScans = - if (relation.hiveQlTable.isPartitioned) { + if (relation.isPartitioned) { // from here on relation must be a [[MetaStoreRelation]] // Applies partition pruning first for partitioned table val filteredRelation = predicateOpt.map(logical.Filter(_, relation)).getOrElse(relation) PartitionPrunings(filteredRelation).view.map(_.transform { @@ -120,7 +126,14 @@ trait HiveStrategies { scan.copy(attributes = prunedCols)(hiveContext) }) } else { - val scan = HiveTableScan(prunedCols, relation, None)(hiveContext) + val scan = relation match { + case MetastoreRelation(_, _, _) => { + HiveTableScan(prunedCols, relation.asInstanceOf[MetastoreRelation], None)(hiveContext) + } + case ParquetRelation(_, _) => { + ParquetTableScan(relation.output, relation.asInstanceOf[ParquetRelation], None)(hiveContext).pruneColumns(prunedCols) + } + } predicateOpt.map(execution.Filter(_, scan)).getOrElse(scan) :: Nil } From 18fdc441ab3fc17535512f86cb77651d91596bdd Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 26 Feb 2014 12:12:15 +0200 Subject: [PATCH 727/778] Reworking Parquet metadata in relation and adding CREATE TABLE AS for Parquet tables --- .../apache/spark/sql/ParquetRelation.scala | 148 ++++++++++++++++-- .../apache/spark/sql/ParquetTableScan.scala | 37 +++-- .../apache/spark/sql/ParquetQueryTests.scala | 18 +++ .../spark/sql/shark/ParquetQuerySuite.scala | 43 +++-- .../apache/spark/sql/hive/HiveContext.scala | 2 + .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 + 6 files changed, 210 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala index f490502a6ba87..9493bdb81ff50 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala @@ -1,9 +1,9 @@ package org.apache.spark.sql -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, Path, FileSystem} import org.apache.hadoop.conf.Configuration -import org.apache.spark.sql.catalyst.plans.logical.BaseRelation +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, InsertIntoCreatedTable, BaseRelation} import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.ArrayType import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, AttributeReference, Attribute} @@ -15,11 +15,15 @@ import parquet.schema.{Type => ParquetType} import parquet.io.api.{Binary, RecordConsumer} import parquet.schema.Type.Repetition -import parquet.hadoop.{ParquetWriter, ParquetFileReader} -import parquet.hadoop.metadata.ParquetMetadata +import parquet.hadoop.{Footer, ParquetFileWriter, ParquetWriter, ParquetFileReader} +import parquet.hadoop.metadata.{BlockMetaData, FileMetaData, GlobalMetaData, ParquetMetadata} import scala.collection.JavaConversions._ -import java.io.File +import java.io.{IOException, FileNotFoundException, File} +import org.apache.hadoop.mapreduce.Job +import parquet.hadoop.util.ContextUtil +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.analysis.OverrideCatalog /** * Relation formed by underlying Parquet file that contains data stored in columnar form. @@ -31,18 +35,19 @@ case class ParquetRelation(val tableName: String, val path: String) extends BaseRelation { - private def parquetMetaData: ParquetMetadata = readMetaData - /** Schema derived from ParquetFile **/ - def parquetSchema: MessageType = parquetMetaData.getFileMetaData.getSchema + def parquetSchema: MessageType = + ParquetTypesConverter + .readMetaData(new Path(path)) + .getFileMetaData + .getSchema /** Attributes **/ - val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) + val attributes = ParquetTypesConverter + .convertToAttributes(parquetSchema) /** Output **/ - val output = attributes - - private def readMetaData: ParquetMetadata = ParquetFileReader.readFooter(new Configuration(), new Path(path)) + override val output = attributes override def isPartitioned = false // Parquet files have no concepts of keys, therefore no Partitioner // Note: we could allow Block level access; needs to be thought through @@ -53,6 +58,60 @@ object ParquetRelation { new ParquetRelation(tableName, path.toUri.toString) } +object RegisterParquetTables extends Rule[LogicalPlan] { + var catalog: OverrideCatalog = null + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case p @ InsertIntoTable(relation: ParquetRelation, _, _, _) => { + catalog.overrideTable(Some("parquet"), relation.tableName, relation) + p + } + } +} + +/** + * Creates any tables required for query execution. + * For example, because of a CREATE TABLE X AS statement. + */ +object CreateParquetTable extends Rule[LogicalPlan] { + var catalog: OverrideCatalog = null + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case InsertIntoCreatedTable(db, tableName, child) => { + if(catalog == null) + throw new NullPointerException("Catalog was not set inside CreateParquetTable") + + val databaseName = db.getOrElse("parquet") // see TODO in [[CreateTables]] + val job = new Job() + val conf = ContextUtil.getConfiguration(job) + val prefix = "tmp" + + val uri = FileSystem.getDefaultUri(conf) + val path = new Path( + new Path(uri), + new Path( + new Path(prefix), + new Path( + new Path(databaseName), + new Path( + new Path(tableName), + new Path("data"))))) + // TODO: add checking: directory exists, etc + + ParquetTypesConverter.writeMetaData(child.output, path) + val relation = new ParquetRelation(tableName, path.toString) + + catalog.overrideTable(Some("parquet"), tableName, relation) + + InsertIntoTable( + relation.asInstanceOf[BaseRelation], + Map.empty, + child, + overwrite = false) + } + } +} + object ParquetTypesConverter { def toDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match { // for now map binary to string type @@ -114,6 +173,66 @@ object ParquetTypesConverter { } new MessageType("root", fields) } + + // todo: proper exception handling, warning if path exists + def writeMetaData(attributes: Seq[Attribute], path: Path) { + val job = new Job() + val conf = ContextUtil.getConfiguration(job) + val fileSystem = FileSystem.get(conf) + + if(fileSystem.exists(path) && !fileSystem.getFileStatus(path).isDir) + throw new IOException(s"Expected to write to directory ${path.toString} but found file") + + val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) + + if(fileSystem.exists(metadataPath)) + fileSystem.delete(metadataPath, true) + + val extraMetadata = new java.util.HashMap[String, String]() + extraMetadata.put("path", path.toString) + // TODO: add table name, etc.? + + val parquetSchema: MessageType = ParquetTypesConverter.convertFromAttributes(attributes) + val metaData: FileMetaData = new FileMetaData( + parquetSchema, + new java.util.HashMap[String, String](), + "Shark") + + ParquetFileWriter.writeMetadataFile( + conf, + path, + new Footer( + path, + new ParquetMetadata( + metaData, + Nil) + ) :: Nil) + } + + /** + * Try to read Parquet metadata at the given Path. We first see if there is a summary file + * in the parent directory. If so, this is used. Else we read the actual footer at the given + * location. + * @param path The path at which we expect one (or more) Parquet files. + * @return The [[ParquetMetadata]] containing among other things the schema. + */ + def readMetaData(path: Path): ParquetMetadata = { + val job = new Job() + val conf = ContextUtil.getConfiguration(job) + val fs: FileSystem = path.getFileSystem(conf) + + val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) + + if(fs.exists(metadataPath) && fs.isFile(metadataPath)) + // TODO: improve exception handling, etc. + ParquetFileReader.readFooter(conf, metadataPath) + else { + if(!fs.exists(path) || !fs.isFile(path)) + throw new FileNotFoundException(s"Could not find file ${path.toString} when trying to read metadata") + + ParquetFileReader.readFooter(conf, path) + } + } } object ParquetTestData { @@ -143,7 +262,8 @@ object ParquetTestData { def writeFile = { testFile.delete val path: Path = new Path(testFile.toURI) - val configuration: Configuration = new Configuration + val job = new Job() + val configuration: Configuration = ContextUtil.getConfiguration(job) val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) val writeSupport = new RowWriteSupport() @@ -163,7 +283,7 @@ object ParquetTestData { data.update(3, 1L<<33) data.update(4, 2.5F) data.update(5, 4.5D) - writer.write(new GenericRow(data)) + writer.write(new GenericRow(data.toSeq)) } writer.close() } diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala index 94f86de61e4b4..8e80bcf4cd355 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala @@ -7,6 +7,7 @@ import parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat} import parquet.hadoop.api.{WriteSupport, ReadSupport} import parquet.hadoop.api.ReadSupport.ReadContext import parquet.column.ParquetProperties +import parquet.hadoop.util.ContextUtil import catalyst.expressions.{Attribute, GenericRow, Row, Expression} import org.apache.spark.sql.execution.{UnaryNode, LeafNode} @@ -14,7 +15,7 @@ import catalyst.types._ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution.SparkPlan @@ -35,9 +36,10 @@ case class ParquetTableScan( * Runs this query returning the result as an RDD. */ override def execute(): RDD[Row] = { - val job = new Job(new Configuration()) + val job = new Job() ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.RowReadSupport]) - job.getConfiguration.set( + val conf: Configuration = ContextUtil.getConfiguration(job) + conf.set( RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, ParquetTypesConverter.convertFromAttributes(attributes).toString) // TODO: add record filters, etc. @@ -45,7 +47,7 @@ case class ParquetTableScan( relation.path, classOf[ParquetInputFormat[Row]], classOf[Void], classOf[Row], - job.getConfiguration) + conf) .map(_._2) } @@ -60,7 +62,7 @@ case class ParquetTableScan( if(success) ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sc) else - ParquetTableScan(attributes, relation, columnPruningPred)(sc) + this // TODO: add warning to log that column projection was unsuccessful? } /** @@ -97,29 +99,35 @@ case class InsertIntoParquetTable( * Inserts all the rows in the Parquet file. */ override def execute() = { + + // TODO: currently we do not check whether the "schema"s are compatible + // That means if one first creates a table and then INSERTs data with + // and incompatible schema the execition will fail. It would be nice + // to catch this early one, maybe having the planner validate the schema + // before calling execute(). + val childRdd = child.execute() assert(childRdd != null) - val job = new Job(new Configuration()) + val job = new Job() ParquetOutputFormat.setWriteSupportClass(job, classOf[org.apache.spark.sql.RowWriteSupport]) - val conf = job.getConfiguration + val conf = ContextUtil.getConfiguration(job) // TODO: move that to function in object conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString) - // TODO: we should pass the read schema, too, in case we insert from a select from a Parquet table? - // conf.set(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, relation.parquetSchema.toString) // TODO: add checks: file exists, etc. - val fs = FileSystem.get(conf) - fs.delete(new Path(relation.path), true) + val fspath = new Path(relation.path) + val fs = fspath.getFileSystem(conf) + fs.delete(fspath, true) - JavaPairRDD.fromRDD(childRdd.map(Tuple2(0, _))).saveAsNewAPIHadoopFile( + JavaPairRDD.fromRDD(childRdd.map(Tuple2(null, _))).saveAsNewAPIHadoopFile( relation.path.toString, classOf[Void], classOf[org.apache.spark.sql.catalyst.expressions.GenericRow], classOf[parquet.hadoop.ParquetOutputFormat[org.apache.spark.sql.catalyst.expressions.GenericRow]], conf) - // From InsertIntoHiveTable: + // From [[InsertIntoHiveTable]]: // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which // does not return anything for insert operations. @@ -130,7 +138,6 @@ case class InsertIntoParquetTable( override def output = child.output } - /** * A [[parquet.io.api.RecordMaterializer]] for Rows. * @@ -212,7 +219,7 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { writer.startMessage() attributes.zipWithIndex.foreach { case (attribute, index) => { - if(record(index) != null) { // null values indicate optional fields but we do not check currently + if(record(index) != null && record(index) != Nil) { // null values indicate optional fields but we do not check currently writer.startField(attribute.name, index) ParquetTypesConverter.consumeType(writer, attribute.dataType, record, index) writer.endField(attribute.name, index) diff --git a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala index 279d9be176333..312c1e8868f47 100644 --- a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala @@ -7,6 +7,11 @@ import org.apache.spark.rdd.RDD import parquet.schema.MessageTypeParser import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.hadoop.conf.Configuration +import parquet.hadoop.ParquetFileWriter +import org.apache.hadoop.mapreduce.Job +import parquet.hadoop.util.ContextUtil class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { override def beforeAll() { @@ -60,6 +65,19 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { assert(allChecks) } + test("Writing metadata from scratch for table CREATE") { + val job = new Job() + val path = new Path("file:///tmp/test/mytesttable") + val fs: FileSystem = FileSystem.getLocal(ContextUtil.getConfiguration(job)) + ParquetTypesConverter.writeMetaData(ParquetTestData.testData.attributes, path) + assert(fs.exists(new Path(path.getParent, ParquetFileWriter.PARQUET_METADATA_FILE))) + val metaData = ParquetTypesConverter.readMetaData(path) + assert(metaData != null) + ParquetTestData.testData.parquetSchema.checkContains(metaData.getFileMetaData.getSchema) // throws excpetion if incompatible + metaData.getFileMetaData.getSchema.checkContains(ParquetTestData.testData.parquetSchema) // throws excpetion if incompatible + fs.delete(path.getParent, true) + } + /** * Computes the given [[org.apache.spark.sql.ParquetRelation]] and returns its RDD. * diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala index 1be69df1aa603..0d71ab133d2cf 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala @@ -9,7 +9,10 @@ import org.apache.spark.sql.ParquetTestData class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { def runQuery(querystr: String): Array[Row] = { - TestShark.sql(querystr).rdd.collect() + TestShark + .sql(querystr) + .rdd + .collect() } override def beforeAll() { @@ -17,10 +20,10 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { // without restarting the JVM. System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") - org.apache.spark.sql.ParquetTestData.testFile.delete() ParquetTestData.writeFile - val testRel = ParquetTestData.testData - TestShark.catalog.overrideTable(Some[String]("default"), "testtable", testRel) + + // Override initial Parquet test table + TestShark.catalog.overrideTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) } override def afterAll() { @@ -28,21 +31,39 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { } test("SELECT on Parquet table") { - val rdd = runQuery("SELECT myboolean, mylong FROM default.testtable") + val rdd = runQuery("SELECT myboolean, mylong FROM parquet.testsource") assert(rdd != null) assert(rdd.forall(_.size == 2)) } - test("Filter on Parquet table") { - val rdd = runQuery("SELECT myboolean, mylong FROM default.testtable WHERE myboolean=true") + test("Simple column projection on Parquet table") { + val rdd = runQuery("SELECT myboolean, mylong FROM parquet.testsource WHERE myboolean=true") assert(rdd.size === 5) assert(rdd.forall(_.getBoolean(0))) } - // TODO: fix insert into table - /*test("INSERT OVERWRITE Parquet table") { - val rdd = runQuery("INSERT OVERWRITE TABLE default.testtable SELECT * FROM default.testtable") - assert(rdd != null) + // TODO: It seems that "CREATE TABLE" is passed directly to Hive as a NativeCommand, which + // makes this test fail. One should come up with a more permanent solution first. + /*test("CREATE Parquet table") { + val result = runQuery("CREATE TABLE IF NOT EXISTS parquet.tmptable (key INT, value STRING)") + assert(result != null) }*/ + + test("CREATE TABLE AS Parquet table") { + runQuery("CREATE TABLE parquet.testdest AS SELECT * FROM src") + val rddCopy = runQuery("SELECT * FROM parquet.testdest").sortBy(_.getInt(0)) + val rddOrig = runQuery("SELECT * FROM src").sortBy(_.getInt(0)) + val allsame = (rddCopy, rddOrig).zipped.forall { (a,b) => (a,b).zipped.forall { (x,y) => x==y}} + assert(allsame) + } + + test("INSERT OVERWRITE to Parquet table") { + runQuery("CREATE TABLE parquet.testdest AS SELECT * FROM src") + runQuery("INSERT OVERWRITE TABLE parquet.testdest SELECT * FROM src") + val rddCopy = runQuery("SELECT * FROM parquet.testdest").sortBy(_.getInt(0)) + val rddOrig = runQuery("SELECT * FROM src").sortBy(_.getInt(0)) + val allsame = (rddCopy, rddOrig).zipped.forall { (a,b) => (a,b).zipped.forall { (x,y) => x==y } } + assert(allsame) + } } 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 3e790ce940ba2..458724f474962 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 @@ -112,6 +112,8 @@ abstract class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { /* A catalyst metadata catalog that points to the Hive Metastore. */ override val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog + // TODO: this is a temporary workaround to register Parquet tables, think of something more permanent + CreateParquetTable.catalog = catalog /* An analyzer that uses the Hive metastore. */ override val analyzer = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) 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 bdec91641bc38..e0205ef356899 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 @@ -91,6 +91,8 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { */ object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // TODO: this is a temporary workaround to register Parquet tables, think of something more permanent + case InsertIntoCreatedTable(Some("parquet"), tableName, child) => CreateParquetTable(plan) case InsertIntoCreatedTable(db, tableName, child) => val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase()) From 3a0a552a5950f99f80bc178818103e393cfa775c Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 26 Feb 2014 14:55:31 +0200 Subject: [PATCH 728/778] Reorganizing Parquet table operations --- .../sql/{ => execution}/ParquetRelation.scala | 8 +- .../execution/ParquetTableOperations.scala | 133 +++++++++++++++ .../ParquetTableSupport.scala} | 158 ++---------------- .../{ => execution}/ParquetQueryTests.scala | 13 +- .../{ => execution}/ParquetQuerySuite.scala | 4 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 + 6 files changed, 164 insertions(+), 154 deletions(-) rename core/src/main/scala/org/apache/spark/sql/{ => execution}/ParquetRelation.scala (97%) create mode 100644 core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala rename core/src/main/scala/org/apache/spark/sql/{ParquetTableScan.scala => execution/ParquetTableSupport.scala} (55%) rename core/src/test/scala/org/apache/spark/sql/{ => execution}/ParquetQueryTests.scala (87%) rename shark/src/test/scala/org/apache/spark/sql/shark/{ => execution}/ParquetQuerySuite.scala (96%) diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala rename to core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala index 9493bdb81ff50..adcdcd407f0ea 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala @@ -1,12 +1,12 @@ -package org.apache.spark.sql +package org.apache.spark.sql.execution -import org.apache.hadoop.fs.{FileStatus, Path, FileSystem} +import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.conf.Configuration import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, InsertIntoCreatedTable, BaseRelation} import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.ArrayType -import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, AttributeReference, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Row, GenericRow, AttributeReference, Attribute} import parquet.schema.{MessageTypeParser, MessageType} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} @@ -16,7 +16,7 @@ import parquet.schema.{Type => ParquetType} import parquet.io.api.{Binary, RecordConsumer} import parquet.schema.Type.Repetition import parquet.hadoop.{Footer, ParquetFileWriter, ParquetWriter, ParquetFileReader} -import parquet.hadoop.metadata.{BlockMetaData, FileMetaData, GlobalMetaData, ParquetMetadata} +import parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} import scala.collection.JavaConversions._ import java.io.{IOException, FileNotFoundException, File} diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala b/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala new file mode 100644 index 0000000000000..f3b4d6614c3f8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala @@ -0,0 +1,133 @@ +package org.apache.spark.sql.execution + +import parquet.io.InvalidRecordException +import parquet.schema.MessageType +import parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat} +import parquet.hadoop.util.ContextUtil + +import org.apache.spark.sql.catalyst.expressions.{Row, Attribute, Expression} + +import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.rdd.RDD +import org.apache.spark.api.java.JavaPairRDD +import org.apache.spark.sql.SparkSqlContext + +/** + * Parquet table scan operator. Imports the file that backs the given [[org.apache.spark.sql.execution.ParquetRelation]] + * as a RDD[Row]. Only a stub currently. + */ +case class ParquetTableScan( + attributes: Seq[Attribute], + relation: ParquetRelation, + columnPruningPred: Option[Expression])( + @transient val sc: SparkSqlContext) + extends LeafNode { + + /** + * Runs this query returning the result as an RDD. + */ + override def execute(): RDD[Row] = { + val job = new Job() + ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.execution.RowReadSupport]) + val conf: Configuration = ContextUtil.getConfiguration(job) + conf.set( + RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, + ParquetTypesConverter.convertFromAttributes(attributes).toString) + // TODO: add record filters, etc. + sc.sparkContext.newAPIHadoopFile( + relation.path, + classOf[ParquetInputFormat[Row]], + classOf[Void], classOf[Row], + conf) + .map(_._2) + } + + /** + * Applies a (candidate) projection. + * + * @param prunedAttributes The list of attributes to be used in the projection. + * @return Pruned TableScan. + */ + def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = { + val success = validateProjection(prunedAttributes) + if(success) + ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sc) + else + this // TODO: add warning to log that column projection was unsuccessful? + } + + /** + * Evaluates a candidate projection by checking whether the candidate is a subtype of the + * original type. + * + * @param projection The candidate projection. + * @return True if the projection is valid, false otherwise. + */ + private def validateProjection(projection: Seq[Attribute]): Boolean = { + val original: MessageType = relation.parquetSchema + val candidate: MessageType = ParquetTypesConverter.convertFromAttributes(projection) + var retval = true + try { + original.checkContains(candidate) + } catch { + case e: InvalidRecordException => { + retval = false + } + } + retval + } + + override def output: Seq[Attribute] = attributes +} + +case class InsertIntoParquetTable( + relation: ParquetRelation, + child: SparkPlan)( + @transient val sc: SparkSqlContext) + extends UnaryNode { + + /** + * Inserts all the rows in the Parquet file. + */ + override def execute() = { + + // TODO: currently we do not check whether the "schema"s are compatible + // That means if one first creates a table and then INSERTs data with + // and incompatible schema the execition will fail. It would be nice + // to catch this early one, maybe having the planner validate the schema + // before calling execute(). + + val childRdd = child.execute() + assert(childRdd != null) + + val job = new Job() + ParquetOutputFormat.setWriteSupportClass(job, classOf[org.apache.spark.sql.execution.RowWriteSupport]) + val conf = ContextUtil.getConfiguration(job) + // TODO: move that to function in object + conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString) + + // TODO: add checks: file exists, etc. + val fspath = new Path(relation.path) + val fs = fspath.getFileSystem(conf) + fs.delete(fspath, true) + + JavaPairRDD.fromRDD(childRdd.map(Tuple2(null, _))).saveAsNewAPIHadoopFile( + relation.path.toString, + classOf[Void], + classOf[org.apache.spark.sql.catalyst.expressions.GenericRow], + classOf[parquet.hadoop.ParquetOutputFormat[org.apache.spark.sql.catalyst.expressions.GenericRow]], + conf) + + // From [[InsertIntoHiveTable]]: + // It would be nice to just return the childRdd unchanged so insert operations could be chained, + // however for now we return an empty list to simplify compatibility checks with hive, which + // does not return anything for insert operations. + // TODO: implement hive compatibility as rules. + sc.sparkContext.makeRDD(Nil, 1) + } + + override def output = child.output +} diff --git a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala b/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala similarity index 55% rename from core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala rename to core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala index 8e80bcf4cd355..f28ce25a7705e 100644 --- a/core/src/main/scala/org/apache/spark/sql/ParquetTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala @@ -1,142 +1,18 @@ -package org.apache.spark.sql +package org.apache.spark.sql.execution + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.Logging import parquet.io.api._ -import parquet.io.InvalidRecordException import parquet.schema.{MessageTypeParser, MessageType} -import parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat} import parquet.hadoop.api.{WriteSupport, ReadSupport} import parquet.hadoop.api.ReadSupport.ReadContext +import parquet.hadoop.ParquetOutputFormat import parquet.column.ParquetProperties -import parquet.hadoop.util.ContextUtil - -import catalyst.expressions.{Attribute, GenericRow, Row, Expression} -import org.apache.spark.sql.execution.{UnaryNode, LeafNode} -import catalyst.types._ - -import org.apache.hadoop.mapreduce.Job -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.api.java.JavaPairRDD - -/** - * Parquet table scan operator. Imports the file that backs the given [[org.apache.spark.sql.ParquetRelation]] - * as a RDD[Row]. Only a stub currently. - */ -case class ParquetTableScan( - attributes: Seq[Attribute], - relation: ParquetRelation, - columnPruningPred: Option[Expression])( - @transient val sc: SparkSqlContext) - extends LeafNode { - - /** - * Runs this query returning the result as an RDD. - */ - override def execute(): RDD[Row] = { - val job = new Job() - ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.RowReadSupport]) - val conf: Configuration = ContextUtil.getConfiguration(job) - conf.set( - RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, - ParquetTypesConverter.convertFromAttributes(attributes).toString) - // TODO: add record filters, etc. - sc.sparkContext.newAPIHadoopFile( - relation.path, - classOf[ParquetInputFormat[Row]], - classOf[Void], classOf[Row], - conf) - .map(_._2) - } - - /** - * Applies a (candidate) projection. - * - * @param prunedAttributes The list of attributes to be used in the projection. - * @return Pruned TableScan. - */ - def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = { - val success = validateProjection(prunedAttributes) - if(success) - ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sc) - else - this // TODO: add warning to log that column projection was unsuccessful? - } - - /** - * Evaluates a candidate projection by checking whether the candidate is a subtype of the - * original type. - * - * @param projection The candidate projection. - * @return True if the projection is valid, false otherwise. - */ - private def validateProjection(projection: Seq[Attribute]): Boolean = { - val original: MessageType = relation.parquetSchema - val candidate: MessageType = ParquetTypesConverter.convertFromAttributes(projection) - var retval = true - try { - original.checkContains(candidate) - } catch { - case e: InvalidRecordException => { - retval = false - } - } - retval - } - - override def output: Seq[Attribute] = attributes -} - -case class InsertIntoParquetTable( - relation: ParquetRelation, - child: SparkPlan)( - @transient val sc: SparkSqlContext) - extends UnaryNode { - - /** - * Inserts all the rows in the Parquet file. - */ - override def execute() = { - - // TODO: currently we do not check whether the "schema"s are compatible - // That means if one first creates a table and then INSERTs data with - // and incompatible schema the execition will fail. It would be nice - // to catch this early one, maybe having the planner validate the schema - // before calling execute(). - - val childRdd = child.execute() - assert(childRdd != null) - - val job = new Job() - ParquetOutputFormat.setWriteSupportClass(job, classOf[org.apache.spark.sql.RowWriteSupport]) - val conf = ContextUtil.getConfiguration(job) - // TODO: move that to function in object - conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString) - - // TODO: add checks: file exists, etc. - val fspath = new Path(relation.path) - val fs = fspath.getFileSystem(conf) - fs.delete(fspath, true) - - JavaPairRDD.fromRDD(childRdd.map(Tuple2(null, _))).saveAsNewAPIHadoopFile( - relation.path.toString, - classOf[Void], - classOf[org.apache.spark.sql.catalyst.expressions.GenericRow], - classOf[parquet.hadoop.ParquetOutputFormat[org.apache.spark.sql.catalyst.expressions.GenericRow]], - conf) - - // From [[InsertIntoHiveTable]]: - // It would be nice to just return the childRdd unchanged so insert operations could be chained, - // however for now we return an empty list to simplify compatibility checks with hive, which - // does not return anything for insert operations. - // TODO: implement hive compatibility as rules. - sc.sparkContext.makeRDD(Nil, 1) - } - - override def output = child.output -} +import org.apache.spark.sql.catalyst.expressions.{Row, GenericRow, Attribute} +import org.apache.spark.sql.catalyst.types._ /** * A [[parquet.io.api.RecordMaterializer]] for Rows. @@ -158,21 +34,21 @@ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterial class RowReadSupport extends ReadSupport[Row] with Logging { override def prepareForRead( - conf: Configuration, - stringMap: java.util.Map[String, String], - fileSchema: MessageType, - readContext: ReadContext): RecordMaterializer[Row] = { - logger.debug(s"preparing for read with schema ${fileSchema.toString}") + conf: Configuration, + stringMap: java.util.Map[String, String], + fileSchema: MessageType, + readContext: ReadContext): RecordMaterializer[Row] = { + log.debug(s"preparing for read with schema ${fileSchema.toString}") new RowRecordMaterializer(readContext.getRequestedSchema) } override def init( - configuration: Configuration, - keyValueMetaData: java.util.Map[String, String], - fileSchema: MessageType): ReadContext = { + configuration: Configuration, + keyValueMetaData: java.util.Map[String, String], + fileSchema: MessageType): ReadContext = { val requested_schema_string = configuration.get(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, fileSchema.toString) val requested_schema = MessageTypeParser.parseMessageType(requested_schema_string) - logger.debug(s"read support initialized for original schema ${requested_schema.toString}") + log.debug(s"read support initialized for original schema ${requested_schema.toString}") new ReadContext(requested_schema, keyValueMetaData) } } diff --git a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala similarity index 87% rename from core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala rename to core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala index 312c1e8868f47..df83c319839eb 100644 --- a/core/src/test/scala/org/apache/spark/sql/ParquetQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala @@ -1,4 +1,4 @@ -package org.apache.spark.sql +package org.apache.spark.sql.execution import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -6,12 +6,11 @@ import org.apache.spark.rdd.RDD import parquet.schema.MessageTypeParser -import org.apache.spark.sql.catalyst.expressions.Row import org.apache.hadoop.fs.{Path, FileSystem} -import org.apache.hadoop.conf.Configuration import parquet.hadoop.ParquetFileWriter import org.apache.hadoop.mapreduce.Job import parquet.hadoop.util.ContextUtil +import org.apache.spark.sql.TestSqlContext class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { override def beforeAll() { @@ -70,16 +69,16 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { val path = new Path("file:///tmp/test/mytesttable") val fs: FileSystem = FileSystem.getLocal(ContextUtil.getConfiguration(job)) ParquetTypesConverter.writeMetaData(ParquetTestData.testData.attributes, path) - assert(fs.exists(new Path(path.getParent, ParquetFileWriter.PARQUET_METADATA_FILE))) + assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) val metaData = ParquetTypesConverter.readMetaData(path) assert(metaData != null) - ParquetTestData.testData.parquetSchema.checkContains(metaData.getFileMetaData.getSchema) // throws excpetion if incompatible - metaData.getFileMetaData.getSchema.checkContains(ParquetTestData.testData.parquetSchema) // throws excpetion if incompatible + ParquetTestData.testData.parquetSchema.checkContains(metaData.getFileMetaData.getSchema) // throws exception if incompatible + metaData.getFileMetaData.getSchema.checkContains(ParquetTestData.testData.parquetSchema) // throws exception if incompatible fs.delete(path.getParent, true) } /** - * Computes the given [[org.apache.spark.sql.ParquetRelation]] and returns its RDD. + * Computes the given [[org.apache.spark.sql.execution.ParquetRelation]] and returns its RDD. * * @param parquetRelation The Parquet relation. * @return An RDD of Rows. diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQuerySuite.scala similarity index 96% rename from shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala rename to shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQuerySuite.scala index 0d71ab133d2cf..fe5f3ce2d288e 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/ParquetQuerySuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQuerySuite.scala @@ -1,10 +1,10 @@ -package org.apache.spark.sql +package org.apache.spark.sql.execution package shark import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.ParquetTestData +import org.apache.spark.sql.shark.TestShark class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { 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 e0205ef356899..52796907c6b4e 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 @@ -36,6 +36,8 @@ import catalyst.types._ import scala.collection.JavaConversions._ +import org.apache.spark.sql.execution.CreateParquetTable + class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { import HiveMetastoreTypes._ From 332119573ba934e7fd8cb1f7adcd0d3bd791a1c2 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Thu, 27 Feb 2014 09:41:21 +0200 Subject: [PATCH 729/778] Fixing one import in ParquetQueryTests.scala --- .../scala/org/apache/spark/sql/execution/ParquetQueryTests.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala index df83c319839eb..bc5233cbe057e 100644 --- a/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala @@ -11,6 +11,7 @@ import parquet.hadoop.ParquetFileWriter import org.apache.hadoop.mapreduce.Job import parquet.hadoop.util.ContextUtil import org.apache.spark.sql.TestSqlContext +import org.apache.spark.sql.catalyst.expressions.Row class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { override def beforeAll() { From 61e3bfbbb2fe4894fa5c2d7c27f1da6cec903819 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 2 Mar 2014 12:45:59 +0100 Subject: [PATCH 730/778] Adding WriteToFile operator and rewriting ParquetQuerySuite --- .../spark/sql/execution/ParquetRelation.scala | 121 ++++++++---------- .../execution/ParquetTableOperations.scala | 46 +++---- .../sql/execution/ParquetQueryTests.scala | 6 +- .../shark/execution/ParquetQuerySuite.scala | 120 +++++++++++++---- .../spark/sql/catalyst/dsl/package.scala | 4 + .../plans/logical/basicOperators.scala | 8 ++ .../spark/sql/execution/SparkStrategies.scala | 3 + .../apache/spark/sql/hive/HiveContext.scala | 2 - .../spark/sql/hive/HiveMetastoreCatalog.scala | 5 - .../spark/sql/hive/HiveStrategies.scala | 8 +- 10 files changed, 190 insertions(+), 133 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala index adcdcd407f0ea..fbd6910e3dc23 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala @@ -1,39 +1,52 @@ package org.apache.spark.sql.execution +import java.io.{IOException, FileNotFoundException, File} + import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.fs.permission.FsAction -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, InsertIntoCreatedTable, BaseRelation} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, BaseRelation} import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.ArrayType import org.apache.spark.sql.catalyst.expressions.{Row, GenericRow, AttributeReference, Attribute} +import org.apache.spark.sql.catalyst.analysis.UnresolvedException import parquet.schema.{MessageTypeParser, MessageType} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} import parquet.schema.{PrimitiveType => ParquetPrimitiveType} import parquet.schema.{Type => ParquetType} - -import parquet.io.api.{Binary, RecordConsumer} import parquet.schema.Type.Repetition +import parquet.io.api.{Binary, RecordConsumer} import parquet.hadoop.{Footer, ParquetFileWriter, ParquetWriter, ParquetFileReader} import parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} +import parquet.hadoop.util.ContextUtil import scala.collection.JavaConversions._ -import java.io.{IOException, FileNotFoundException, File} -import org.apache.hadoop.mapreduce.Job -import parquet.hadoop.util.ContextUtil -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.analysis.OverrideCatalog /** * Relation formed by underlying Parquet file that contains data stored in columnar form. + * Note that there are currently two ways to import a ParquetRelation: + * a) create the Relation "manually" and register via the OverrideCatalog, e.g.: + * TestShark.catalog.overrideTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) + * and then execute the query as usual. + * b) store a relation via WriteToFile and manually resolve the corresponding ParquetRelation: + * val query = TestShark.parseSql(querystr).transform { + * case relation @ UnresolvedRelation(databaseName, name, alias) => + * if(name.equals(tableName)) + * ParquetRelation(tableName, filename) + * else + * relation + * } + * TestShark.executePlan(query) + * .toRdd + * .collect() * * @param tableName The name of the relation. * @param path The path to the Parquet file. */ -case class ParquetRelation(val tableName: String, val path: String) - - extends BaseRelation { +case class ParquetRelation(val tableName: String, val path: String) extends BaseRelation { /** Schema derived from ParquetFile **/ def parquetSchema: MessageType = @@ -43,7 +56,8 @@ case class ParquetRelation(val tableName: String, val path: String) .getSchema /** Attributes **/ - val attributes = ParquetTypesConverter + val attributes = + ParquetTypesConverter .convertToAttributes(parquetSchema) /** Output **/ @@ -54,61 +68,23 @@ case class ParquetRelation(val tableName: String, val path: String) } object ParquetRelation { - def apply(path: Path, tableName: String = "ParquetTable") = - new ParquetRelation(tableName, path.toUri.toString) -} -object RegisterParquetTables extends Rule[LogicalPlan] { - var catalog: OverrideCatalog = null + def create(pathString: String, child: LogicalPlan, conf: Configuration, tableName: Option[String]): ParquetRelation = { + if(!child.resolved) + throw new UnresolvedException[LogicalPlan](child, "Attempt to create Parquet table from unresolved child (when schema is not available)") - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p @ InsertIntoTable(relation: ParquetRelation, _, _, _) => { - catalog.overrideTable(Some("parquet"), relation.tableName, relation) - p - } + val name = s"${tableName.getOrElse(child.nodeName)}_parquet" + val path = checkPath(pathString, conf) + ParquetTypesConverter.writeMetaData(child.output, path, conf) + new ParquetRelation(name, path.toString) } -} -/** - * Creates any tables required for query execution. - * For example, because of a CREATE TABLE X AS statement. - */ -object CreateParquetTable extends Rule[LogicalPlan] { - var catalog: OverrideCatalog = null - - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case InsertIntoCreatedTable(db, tableName, child) => { - if(catalog == null) - throw new NullPointerException("Catalog was not set inside CreateParquetTable") - - val databaseName = db.getOrElse("parquet") // see TODO in [[CreateTables]] - val job = new Job() - val conf = ContextUtil.getConfiguration(job) - val prefix = "tmp" - - val uri = FileSystem.getDefaultUri(conf) - val path = new Path( - new Path(uri), - new Path( - new Path(prefix), - new Path( - new Path(databaseName), - new Path( - new Path(tableName), - new Path("data"))))) - // TODO: add checking: directory exists, etc - - ParquetTypesConverter.writeMetaData(child.output, path) - val relation = new ParquetRelation(tableName, path.toString) - - catalog.overrideTable(Some("parquet"), tableName, relation) - - InsertIntoTable( - relation.asInstanceOf[BaseRelation], - Map.empty, - child, - overwrite = false) - } + private def checkPath(pathStr: String, conf: Configuration): Path = { + val path = new Path(pathStr) + val fs = path.getFileSystem(conf) + if(fs.exists(path) && !fs.getFileStatus(path).getPermission.getUserAction.implies(FsAction.READ_WRITE)) + throw new IOException(s"Unable to create ParquetRelation: path ${path.toString} not read-writable") + path } } @@ -174,10 +150,7 @@ object ParquetTypesConverter { new MessageType("root", fields) } - // todo: proper exception handling, warning if path exists - def writeMetaData(attributes: Seq[Attribute], path: Path) { - val job = new Job() - val conf = ContextUtil.getConfiguration(job) + def writeMetaData(attributes: Seq[Attribute], path: Path, conf: Configuration) { val fileSystem = FileSystem.get(conf) if(fileSystem.exists(path) && !fileSystem.getFileStatus(path).isDir) @@ -185,12 +158,18 @@ object ParquetTypesConverter { val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) - if(fileSystem.exists(metadataPath)) - fileSystem.delete(metadataPath, true) + if(fileSystem.exists(metadataPath)) { + try { + fileSystem.delete(metadataPath, true) + } catch { + case e: IOException => + throw new IOException(s"Unable to delete previous PARQUET_METADATA_FILE:\n${e.toString}") + } + } val extraMetadata = new java.util.HashMap[String, String]() extraMetadata.put("path", path.toString) - // TODO: add table name, etc.? + // TODO: add extra data, e.g., table name, date, etc.? val parquetSchema: MessageType = ParquetTypesConverter.convertFromAttributes(attributes) val metaData: FileMetaData = new FileMetaData( @@ -257,7 +236,7 @@ object ParquetTestData { val testFile = new File("/tmp/testParquetFile").getAbsoluteFile - lazy val testData = ParquetRelation(new Path(testFile.toURI)) + lazy val testData = new ParquetRelation("testData", testFile.toURI.toString) def writeFile = { testFile.delete diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala b/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala index f3b4d6614c3f8..088e689634126 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala @@ -1,19 +1,21 @@ -package org.apache.spark.sql.execution +package org.apache.spark.sql +package execution import parquet.io.InvalidRecordException import parquet.schema.MessageType import parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat} import parquet.hadoop.util.ContextUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.api.java.JavaPairRDD +import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.expressions.{Row, Attribute, Expression} import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.rdd.RDD -import org.apache.spark.api.java.JavaPairRDD -import org.apache.spark.sql.SparkSqlContext +import java.io.IOException /** * Parquet table scan operator. Imports the file that backs the given [[org.apache.spark.sql.execution.ParquetRelation]] @@ -23,21 +25,21 @@ case class ParquetTableScan( attributes: Seq[Attribute], relation: ParquetRelation, columnPruningPred: Option[Expression])( - @transient val sc: SparkSqlContext) + @transient val sc: SparkContext) extends LeafNode { /** * Runs this query returning the result as an RDD. */ override def execute(): RDD[Row] = { - val job = new Job() + val job = new Job(sc.hadoopConfiguration) ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.execution.RowReadSupport]) val conf: Configuration = ContextUtil.getConfiguration(job) conf.set( RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, ParquetTypesConverter.convertFromAttributes(attributes).toString) - // TODO: add record filters, etc. - sc.sparkContext.newAPIHadoopFile( + // TODO: think about adding record filters + sc.newAPIHadoopFile( relation.path, classOf[ParquetInputFormat[Row]], classOf[Void], classOf[Row], @@ -86,33 +88,37 @@ case class ParquetTableScan( case class InsertIntoParquetTable( relation: ParquetRelation, child: SparkPlan)( - @transient val sc: SparkSqlContext) + @transient val sc: SparkContext) extends UnaryNode { /** - * Inserts all the rows in the Parquet file. + * Inserts all the rows in the Parquet file. Note that OVERWRITE is implicit, since Parquet files are write-once. */ override def execute() = { - // TODO: currently we do not check whether the "schema"s are compatible // That means if one first creates a table and then INSERTs data with - // and incompatible schema the execition will fail. It would be nice + // and incompatible schema the execution will fail. It would be nice // to catch this early one, maybe having the planner validate the schema // before calling execute(). val childRdd = child.execute() assert(childRdd != null) - val job = new Job() + val job = new Job(sc.hadoopConfiguration) ParquetOutputFormat.setWriteSupportClass(job, classOf[org.apache.spark.sql.execution.RowWriteSupport]) - val conf = ContextUtil.getConfiguration(job) + val conf = job.getConfiguration // TODO: move that to function in object conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString) - // TODO: add checks: file exists, etc. val fspath = new Path(relation.path) val fs = fspath.getFileSystem(conf) - fs.delete(fspath, true) + + try { + fs.delete(fspath, true) + } catch { + case e: IOException => + throw new IOException(s"Unable to clear output directory ${fspath.toString} prior to InsertIntoParquetTable:\n${e.toString}") + } JavaPairRDD.fromRDD(childRdd.map(Tuple2(null, _))).saveAsNewAPIHadoopFile( relation.path.toString, @@ -121,12 +127,8 @@ case class InsertIntoParquetTable( classOf[parquet.hadoop.ParquetOutputFormat[org.apache.spark.sql.catalyst.expressions.GenericRow]], conf) - // From [[InsertIntoHiveTable]]: - // It would be nice to just return the childRdd unchanged so insert operations could be chained, - // however for now we return an empty list to simplify compatibility checks with hive, which - // does not return anything for insert operations. - // TODO: implement hive compatibility as rules. - sc.sparkContext.makeRDD(Nil, 1) + // We return the child RDD to allow chaining (alternatively, one could return nothing). + childRdd } override def output = child.output diff --git a/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala index bc5233cbe057e..da4dbdfc08558 100644 --- a/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala @@ -47,7 +47,7 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { } test("Projection of simple Parquet file") { - val scanner = new ParquetTableScan(ParquetTestData.testData.attributes, ParquetTestData.testData, None)(TestSqlContext) + val scanner = new ParquetTableScan(ParquetTestData.testData.attributes, ParquetTestData.testData, None)(TestSqlContext.sparkContext) val projected = scanner.pruneColumns(ParquetTypesConverter.convertToAttributes(MessageTypeParser.parseMessageType(ParquetTestData.subTestSchema))) assert(projected.attributes.size === 2) val result = projected.execute().collect() @@ -69,7 +69,7 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { val job = new Job() val path = new Path("file:///tmp/test/mytesttable") val fs: FileSystem = FileSystem.getLocal(ContextUtil.getConfiguration(job)) - ParquetTypesConverter.writeMetaData(ParquetTestData.testData.attributes, path) + ParquetTypesConverter.writeMetaData(ParquetTestData.testData.attributes, path, TestSqlContext.sparkContext.hadoopConfiguration) assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) val metaData = ParquetTypesConverter.readMetaData(path) assert(metaData != null) @@ -85,7 +85,7 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { * @return An RDD of Rows. */ private def getRDD(parquetRelation: ParquetRelation): RDD[Row] = { - val scanner = new ParquetTableScan(parquetRelation.attributes, parquetRelation, None)(TestSqlContext) + val scanner = new ParquetTableScan(parquetRelation.attributes, parquetRelation, None)(TestSqlContext.sparkContext) scanner.execute } } \ No newline at end of file diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQuerySuite.scala b/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQuerySuite.scala index fe5f3ce2d288e..0ab412bb0591b 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQuerySuite.scala +++ b/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQuerySuite.scala @@ -2,26 +2,85 @@ package org.apache.spark.sql.execution package shark -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import java.io.File + +import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite} + import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.shark.TestShark +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.WriteToFile +import org.apache.spark.sql.catalyst.plans.logical.InsertIntoCreatedTable +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { +class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { - def runQuery(querystr: String): Array[Row] = { - TestShark - .sql(querystr) - .rdd + val filename = "file:///tmp/parquettest" + + // runs a SQL and optionally resolves one Parquet table + def runQuery(querystr: String, tableName: Option[String] = None, filename: Option[String] = None): Array[Row] = { + // call to resolve references in order to get CREATE TABLE AS to work + val query = TestShark + .parseSql(querystr) + val finalQuery = + if(tableName.nonEmpty && filename.nonEmpty) + resolveParquetTable(tableName.get, filename.get, query) + else + query + TestShark.executePlan(finalQuery) + .toRdd .collect() } + // stores a query output to a Parquet file + def storeQuery(querystr: String, filename: String): Unit = { + val query = WriteToFile( + filename, + TestShark.parseSql(querystr), + Some("testtable")) + TestShark + .executePlan(query) + .stringResult() + } + + /** + * TODO: This function is necessary as long as there is no notion of a Catalog for + * Parquet tables. Once such a thing exists this functionality should be moved there. + */ + def resolveParquetTable(tableName: String, filename: String, plan: LogicalPlan): LogicalPlan = { + plan.transform { + case relation @ UnresolvedRelation(databaseName, name, alias) => + if(name == tableName) + ParquetRelation(tableName, filename) + else + relation + case op @ InsertIntoCreatedTable(databaseName, name, child) => + if(name == tableName) { + // note: at this stage the plan is not yet analyzed but Parquet needs to know the schema + // and for that we need the child to be resolved + TestShark.loadTestTable("src") // may not be loaded now + val relation = ParquetRelation.create( + filename, + TestShark.analyzer(child), + TestShark.sparkContext.hadoopConfiguration, + Some(tableName)) + InsertIntoTable( + relation.asInstanceOf[BaseRelation], + Map.empty, + child, + overwrite = false) + } else + op + } + } + override def beforeAll() { // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") + // write test data ParquetTestData.writeFile - // Override initial Parquet test table TestShark.catalog.overrideTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) } @@ -30,40 +89,49 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { ParquetTestData.testFile.delete() } + override def beforeEach() { + (new File(filename)).getAbsoluteFile.delete() + } + + override def afterEach() { + (new File(filename)).getAbsoluteFile.delete() + } + test("SELECT on Parquet table") { - val rdd = runQuery("SELECT myboolean, mylong FROM parquet.testsource") + val rdd = runQuery("SELECT * FROM parquet.testsource") assert(rdd != null) - assert(rdd.forall(_.size == 2)) + assert(rdd.forall(_.size == 6)) } - test("Simple column projection on Parquet table") { + test("Simple column projection + filter on Parquet table") { val rdd = runQuery("SELECT myboolean, mylong FROM parquet.testsource WHERE myboolean=true") assert(rdd.size === 5) assert(rdd.forall(_.getBoolean(0))) } - // TODO: It seems that "CREATE TABLE" is passed directly to Hive as a NativeCommand, which - // makes this test fail. One should come up with a more permanent solution first. - /*test("CREATE Parquet table") { - val result = runQuery("CREATE TABLE IF NOT EXISTS parquet.tmptable (key INT, value STRING)") - assert(result != null) - }*/ + test("Converting Hive to Parquet Table via WriteToFile") { + storeQuery("SELECT * FROM src", filename) + val rddOne = runQuery("SELECT * FROM src").sortBy(_.getInt(0)) + val rddTwo = runQuery("SELECT * from ptable", Some("ptable"), Some(filename)).sortBy(_.getInt(0)) + val allsame = (rddOne, rddTwo).zipped.forall { (a,b) => (a,b).zipped.forall { (x,y) => x==y}} + assert(allsame) + } - test("CREATE TABLE AS Parquet table") { - runQuery("CREATE TABLE parquet.testdest AS SELECT * FROM src") - val rddCopy = runQuery("SELECT * FROM parquet.testdest").sortBy(_.getInt(0)) - val rddOrig = runQuery("SELECT * FROM src").sortBy(_.getInt(0)) - val allsame = (rddCopy, rddOrig).zipped.forall { (a,b) => (a,b).zipped.forall { (x,y) => x==y}} + test("INSERT OVERWRITE TABLE Parquet table") { + storeQuery("SELECT * FROM parquet.testsource", filename) + runQuery("INSERT OVERWRITE TABLE ptable SELECT * FROM parquet.testsource", Some("ptable"), Some(filename)) + runQuery("INSERT OVERWRITE TABLE ptable SELECT * FROM parquet.testsource", Some("ptable"), Some(filename)) + val rddCopy = runQuery("SELECT * FROM ptable", Some("ptable"), Some(filename)) + val rddOrig = runQuery("SELECT * FROM parquet.testsource") + val allsame = (rddCopy, rddOrig).zipped.forall { (a,b) => (a,b).zipped.forall { (x,y) => x==y } } assert(allsame) } - test("INSERT OVERWRITE to Parquet table") { - runQuery("CREATE TABLE parquet.testdest AS SELECT * FROM src") - runQuery("INSERT OVERWRITE TABLE parquet.testdest SELECT * FROM src") - val rddCopy = runQuery("SELECT * FROM parquet.testdest").sortBy(_.getInt(0)) + test("CREATE TABLE AS Parquet table") { + runQuery("CREATE TABLE ptable AS SELECT * FROM src", Some("ptable"), Some(filename)) + val rddCopy = runQuery("SELECT * FROM ptable", Some("ptable"), Some(filename)).sortBy(_.getInt(0)) val rddOrig = runQuery("SELECT * FROM src").sortBy(_.getInt(0)) val allsame = (rddCopy, rddOrig).zipped.forall { (a,b) => (a,b).zipped.forall { (x,y) => x==y } } assert(allsame) } } - diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 9405d0ae290db..b926fb481576f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -206,5 +206,9 @@ package object dsl { InsertIntoTable(analysis.UnresolvedRelation(None, tableName), Map.empty, plan, overwrite) def analyze = analysis.SimpleAnalyzer(plan) + + def writeToFile(path: String) = WriteToFile(path, plan, None) + + // TODO: for a loadFromFile it would be good to have a Catalog that knows how to resolve ParquetTables } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 660210368847c..69d4f16d6e9ca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -109,6 +109,14 @@ case class InsertIntoCreatedTable( def output = child.output } +case class WriteToFile( + path: String, + child: LogicalPlan, + tableName: Option[String]) extends UnaryNode { + def references = Set.empty + def output = child.output +} + case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { def output = child.output def references = order.flatMap(_.references).toSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index d7cbe1c9e2be5..959d36e13c89a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -209,6 +209,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.ExistingRdd(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil + case logical.WriteToFile(path, child, tableName) => + val relation = ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, tableName) + execution.InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil case _ => Nil } } 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 458724f474962..3e790ce940ba2 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 @@ -112,8 +112,6 @@ abstract class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { /* A catalyst metadata catalog that points to the Hive Metastore. */ override val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog - // TODO: this is a temporary workaround to register Parquet tables, think of something more permanent - CreateParquetTable.catalog = catalog /* An analyzer that uses the Hive metastore. */ override val analyzer = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) 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 52796907c6b4e..e3b883365b84a 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 @@ -36,9 +36,6 @@ import catalyst.types._ import scala.collection.JavaConversions._ -import org.apache.spark.sql.execution.CreateParquetTable - - class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { import HiveMetastoreTypes._ @@ -93,8 +90,6 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { */ object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - // TODO: this is a temporary workaround to register Parquet tables, think of something more permanent - case InsertIntoCreatedTable(Some("parquet"), tableName, child) => CreateParquetTable(plan) case InsertIntoCreatedTable(db, tableName, child) => val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase()) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 664033c1216b3..dcd421b28b6e7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -44,7 +44,7 @@ trait HiveStrategies { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => - InsertIntoParquetTable(table, planLater(child))(hiveContext) :: Nil + InsertIntoParquetTable(table, planLater(child))(hiveContext.sparkContext) :: Nil case _ => Nil } } @@ -55,11 +55,11 @@ trait HiveStrategies { case p @ logical.Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m, None)(hiveContext) :: Nil case p @ logical.Project(projectList, r: ParquetRelation) if isSimpleProject(projectList) => - ParquetTableScan(projectList.asInstanceOf[Seq[Attribute]], r, None)(hiveContext) :: Nil + ParquetTableScan(projectList.asInstanceOf[Seq[Attribute]], r, None)(hiveContext.sparkContext) :: Nil case m: MetastoreRelation => HiveTableScan(m.output, m, None)(hiveContext) :: Nil case p: ParquetRelation => - ParquetTableScan(p.output, p, None)(hiveContext) :: Nil + ParquetTableScan(p.output, p, None)(hiveContext.sparkContext) :: Nil case _ => Nil } } @@ -131,7 +131,7 @@ trait HiveStrategies { HiveTableScan(prunedCols, relation.asInstanceOf[MetastoreRelation], None)(hiveContext) } case ParquetRelation(_, _) => { - ParquetTableScan(relation.output, relation.asInstanceOf[ParquetRelation], None)(hiveContext).pruneColumns(prunedCols) + ParquetTableScan(relation.output, relation.asInstanceOf[ParquetRelation], None)(hiveContext.sparkContext).pruneColumns(prunedCols) } } predicateOpt.map(execution.Filter(_, scan)).getOrElse(scan) :: Nil From c863bed3d17abf9cd3da7cee8637d77b088a192d Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 2 Mar 2014 15:28:23 +0100 Subject: [PATCH 731/778] Codestyle checks --- .../spark/sql/execution/ParquetRelation.scala | 65 ++++++++++++------ .../execution/ParquetTableOperations.scala | 35 +++++++--- .../sql/execution/ParquetTableSupport.scala | 68 +++++++++++++------ .../spark/sql/catalyst/dsl/package.scala | 3 +- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../hadoop/mapred/SharkHadoopWriter.scala | 2 +- .../spark/sql/hive/HiveStrategies.scala | 16 ++++- 7 files changed, 134 insertions(+), 58 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala index fbd6910e3dc23..97c3002ae9525 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala @@ -29,9 +29,13 @@ import scala.collection.JavaConversions._ * Relation formed by underlying Parquet file that contains data stored in columnar form. * Note that there are currently two ways to import a ParquetRelation: * a) create the Relation "manually" and register via the OverrideCatalog, e.g.: - * TestShark.catalog.overrideTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) + * TestShark.catalog.overrideTable( + * Some[String]("parquet"), + * "testsource", + * ParquetTestData.testData) * and then execute the query as usual. - * b) store a relation via WriteToFile and manually resolve the corresponding ParquetRelation: + * b) store a relation via WriteToFile and manually resolve the corresponding + * ParquetRelation: * val query = TestShark.parseSql(querystr).transform { * case relation @ UnresolvedRelation(databaseName, name, alias) => * if(name.equals(tableName)) @@ -63,15 +67,22 @@ case class ParquetRelation(val tableName: String, val path: String) extends Base /** Output **/ override val output = attributes - override def isPartitioned = false // Parquet files have no concepts of keys, therefore no Partitioner + // Parquet files have no concepts of keys, therefore no Partitioner // Note: we could allow Block level access; needs to be thought through + override def isPartitioned = false } object ParquetRelation { - def create(pathString: String, child: LogicalPlan, conf: Configuration, tableName: Option[String]): ParquetRelation = { - if(!child.resolved) - throw new UnresolvedException[LogicalPlan](child, "Attempt to create Parquet table from unresolved child (when schema is not available)") + def create(pathString: String, + child: LogicalPlan, + conf: Configuration, + tableName: Option[String]): ParquetRelation = { + if(!child.resolved) { + throw new UnresolvedException[LogicalPlan]( + child, + "Attempt to create Parquet table from unresolved child (when schema is not available)") + } val name = s"${tableName.getOrElse(child.nodeName)}_parquet" val path = checkPath(pathString, conf) @@ -82,8 +93,14 @@ object ParquetRelation { private def checkPath(pathStr: String, conf: Configuration): Path = { val path = new Path(pathStr) val fs = path.getFileSystem(conf) - if(fs.exists(path) && !fs.getFileStatus(path).getPermission.getUserAction.implies(FsAction.READ_WRITE)) - throw new IOException(s"Unable to create ParquetRelation: path ${path.toString} not read-writable") + if(fs.exists(path) && + !fs.getFileStatus(path) + .getPermission + .getUserAction + .implies(FsAction.READ_WRITE)) { + throw new IOException( + s"Unable to create ParquetRelation: path ${path.toString} not read-writable") + } path } } @@ -130,7 +147,8 @@ object ParquetTypesConverter { } } - def getSchema(schemaString : String) : MessageType = MessageTypeParser.parseMessageType(schemaString) + def getSchema(schemaString : String) : MessageType = + MessageTypeParser.parseMessageType(schemaString) def convertToAttributes(parquetSchema: MessageType) : Seq[Attribute] = { parquetSchema.getColumns.map { @@ -153,8 +171,9 @@ object ParquetTypesConverter { def writeMetaData(attributes: Seq[Attribute], path: Path, conf: Configuration) { val fileSystem = FileSystem.get(conf) - if(fileSystem.exists(path) && !fileSystem.getFileStatus(path).isDir) + if(fileSystem.exists(path) && !fileSystem.getFileStatus(path).isDir) { throw new IOException(s"Expected to write to directory ${path.toString} but found file") + } val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) @@ -171,7 +190,8 @@ object ParquetTypesConverter { extraMetadata.put("path", path.toString) // TODO: add extra data, e.g., table name, date, etc.? - val parquetSchema: MessageType = ParquetTypesConverter.convertFromAttributes(attributes) + val parquetSchema: MessageType = + ParquetTypesConverter.convertFromAttributes(attributes) val metaData: FileMetaData = new FileMetaData( parquetSchema, new java.util.HashMap[String, String](), @@ -202,13 +222,14 @@ object ParquetTypesConverter { val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) - if(fs.exists(metadataPath) && fs.isFile(metadataPath)) + if(fs.exists(metadataPath) && fs.isFile(metadataPath)) { // TODO: improve exception handling, etc. ParquetFileReader.readFooter(conf, metadataPath) - else { - if(!fs.exists(path) || !fs.isFile(path)) - throw new FileNotFoundException(s"Could not find file ${path.toString} when trying to read metadata") - + } else { + if(!fs.exists(path) || !fs.isFile(path)) { + throw new FileNotFoundException( + s"Could not find file ${path.toString} when trying to read metadata") + } ParquetFileReader.readFooter(conf, path) } } @@ -250,14 +271,16 @@ object ParquetTestData { val writer = new ParquetWriter(path, writeSupport) for(i <- 0 until 15) { val data = new Array[Any](6) - if(i % 3 ==0) + if(i % 3 ==0) { data.update(0, true) - else + } else { data.update(0, false) - if(i % 5 == 0) + } + if(i % 5 == 0) { data.update(1, 5) - else + } else { data.update(1, null) // optional + } data.update(2, "abc") data.update(3, 1L<<33) data.update(4, 2.5F) @@ -266,4 +289,4 @@ object ParquetTestData { } writer.close() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala b/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala index 088e689634126..38da8c3bc74dd 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala @@ -18,8 +18,8 @@ import org.apache.hadoop.fs.Path import java.io.IOException /** - * Parquet table scan operator. Imports the file that backs the given [[org.apache.spark.sql.execution.ParquetRelation]] - * as a RDD[Row]. Only a stub currently. + * Parquet table scan operator. Imports the file that backs the given + * [[org.apache.spark.sql.execution.ParquetRelation]] as a RDD[Row]. */ case class ParquetTableScan( attributes: Seq[Attribute], @@ -33,7 +33,9 @@ case class ParquetTableScan( */ override def execute(): RDD[Row] = { val job = new Job(sc.hadoopConfiguration) - ParquetInputFormat.setReadSupportClass(job, classOf[org.apache.spark.sql.execution.RowReadSupport]) + ParquetInputFormat.setReadSupportClass( + job, + classOf[org.apache.spark.sql.execution.RowReadSupport]) val conf: Configuration = ContextUtil.getConfiguration(job) conf.set( RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, @@ -55,15 +57,16 @@ case class ParquetTableScan( */ def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = { val success = validateProjection(prunedAttributes) - if(success) + if(success) { ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sc) - else + } else { this // TODO: add warning to log that column projection was unsuccessful? + } } /** - * Evaluates a candidate projection by checking whether the candidate is a subtype of the - * original type. + * Evaluates a candidate projection by checking whether the candidate is a subtype + * of the original type. * * @param projection The candidate projection. * @return True if the projection is valid, false otherwise. @@ -92,7 +95,8 @@ case class InsertIntoParquetTable( extends UnaryNode { /** - * Inserts all the rows in the Parquet file. Note that OVERWRITE is implicit, since Parquet files are write-once. + * Inserts all the rows in the Parquet file. Note that OVERWRITE is implicit, since + * Parquet files are write-once. */ override def execute() = { // TODO: currently we do not check whether the "schema"s are compatible @@ -105,9 +109,13 @@ case class InsertIntoParquetTable( assert(childRdd != null) val job = new Job(sc.hadoopConfiguration) - ParquetOutputFormat.setWriteSupportClass(job, classOf[org.apache.spark.sql.execution.RowWriteSupport]) - val conf = job.getConfiguration + + ParquetOutputFormat.setWriteSupportClass( + job, + classOf[org.apache.spark.sql.execution.RowWriteSupport]) + // TODO: move that to function in object + val conf = job.getConfiguration conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString) val fspath = new Path(relation.path) @@ -117,14 +125,18 @@ case class InsertIntoParquetTable( fs.delete(fspath, true) } catch { case e: IOException => - throw new IOException(s"Unable to clear output directory ${fspath.toString} prior to InsertIntoParquetTable:\n${e.toString}") + throw new IOException( + s"Unable to clear output directory ${fspath.toString} prior" + + s" to InsertIntoParquetTable:\n${e.toString}") } JavaPairRDD.fromRDD(childRdd.map(Tuple2(null, _))).saveAsNewAPIHadoopFile( relation.path.toString, classOf[Void], classOf[org.apache.spark.sql.catalyst.expressions.GenericRow], + // scalastyle:off line.size.limit classOf[parquet.hadoop.ParquetOutputFormat[org.apache.spark.sql.catalyst.expressions.GenericRow]], + // scalastyle:on line.size.limit conf) // We return the child RDD to allow chaining (alternatively, one could return nothing). @@ -133,3 +145,4 @@ case class InsertIntoParquetTable( override def output = child.output } + diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala b/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala index f28ce25a7705e..636523c582758 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala @@ -21,7 +21,10 @@ import org.apache.spark.sql.catalyst.types._ */ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterializer[Row] { - def this(parquetSchema: MessageType) = this(new CatalystGroupConverter(ParquetTypesConverter.convertToAttributes(parquetSchema))) + def this(parquetSchema: MessageType) = + this( + new CatalystGroupConverter( + ParquetTypesConverter.convertToAttributes(parquetSchema))) override def getCurrentRecord: Row = root.getCurrentRecord @@ -46,8 +49,11 @@ class RowReadSupport extends ReadSupport[Row] with Logging { configuration: Configuration, keyValueMetaData: java.util.Map[String, String], fileSchema: MessageType): ReadContext = { - val requested_schema_string = configuration.get(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, fileSchema.toString) - val requested_schema = MessageTypeParser.parseMessageType(requested_schema_string) + val requested_schema_string = + configuration.get(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, fileSchema.toString) + val requested_schema = + MessageTypeParser.parseMessageType(requested_schema_string) + log.debug(s"read support initialized for original schema ${requested_schema.toString}") new ReadContext(requested_schema, keyValueMetaData) } @@ -65,12 +71,17 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { // for testing this.schema = schema // TODO: could use Attributes themselves instead of Parquet schema? - configuration.set(RowWriteSupport.PARQUET_ROW_SCHEMA, schema.toString) - configuration.set(ParquetOutputFormat.WRITER_VERSION, ParquetProperties.WriterVersion.PARQUET_1_0.toString) + configuration.set( + RowWriteSupport.PARQUET_ROW_SCHEMA, + schema.toString) + configuration.set( + ParquetOutputFormat.WRITER_VERSION, + ParquetProperties.WriterVersion.PARQUET_1_0.toString) } def getSchema(configuration: Configuration): MessageType = { - return MessageTypeParser.parseMessageType(configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) + return MessageTypeParser.parseMessageType( + configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) } private var schema: MessageType = null @@ -78,12 +89,15 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { private var attributes: Seq[Attribute] = null override def init(configuration: Configuration): WriteSupport.WriteContext = { - schema = if(schema == null) + schema = if(schema == null) { getSchema(configuration) - else + } else { schema + } attributes = ParquetTypesConverter.convertToAttributes(schema) - new WriteSupport.WriteContext(schema, new java.util.HashMap[java.lang.String, java.lang.String]()); + new WriteSupport.WriteContext( + schema, + new java.util.HashMap[java.lang.String, java.lang.String]()); } override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { @@ -95,7 +109,8 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { writer.startMessage() attributes.zipWithIndex.foreach { case (attribute, index) => { - if(record(index) != null && record(index) != Nil) { // null values indicate optional fields but we do not check currently + // null values indicate optional fields but we do not check currently + if(record(index) != null && record(index) != Nil) { writer.startField(attribute.name, index) ParquetTypesConverter.consumeType(writer, attribute.dataType, record, index) writer.endField(attribute.name, index) @@ -152,27 +167,40 @@ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { * @param parent The parent group converter. * @param fieldIndex The index inside the record. */ -class CatalystPrimitiveConverter(parent: CatalystGroupConverter, fieldIndex: Int) extends PrimitiveConverter { +class CatalystPrimitiveConverter( + parent: CatalystGroupConverter, + fieldIndex: Int) extends PrimitiveConverter { // TODO: consider refactoring these together with ParquetTypesConverter - override def addBinary(value: Binary): Unit = parent.currentData.update(fieldIndex, value.getBytes.asInstanceOf[BinaryType.JvmType]) + override def addBinary(value: Binary): Unit = + parent.currentData.update(fieldIndex, value.getBytes.asInstanceOf[BinaryType.JvmType]) - override def addBoolean(value: Boolean): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[BooleanType.JvmType]) + override def addBoolean(value: Boolean): Unit = + parent.currentData.update(fieldIndex, value.asInstanceOf[BooleanType.JvmType]) - override def addDouble(value: Double): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[DoubleType.JvmType]) + override def addDouble(value: Double): Unit = + parent.currentData.update(fieldIndex, value.asInstanceOf[DoubleType.JvmType]) - override def addFloat(value: Float): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[FloatType.JvmType]) + override def addFloat(value: Float): Unit = + parent.currentData.update(fieldIndex, value.asInstanceOf[FloatType.JvmType]) - override def addInt(value: Int): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[IntegerType.JvmType]) + override def addInt(value: Int): Unit = + parent.currentData.update(fieldIndex, value.asInstanceOf[IntegerType.JvmType]) - override def addLong(value: Long): Unit = parent.currentData.update(fieldIndex, value.asInstanceOf[LongType.JvmType]) + override def addLong(value: Long): Unit = + parent.currentData.update(fieldIndex, value.asInstanceOf[LongType.JvmType]) } /** - * A [[parquet.io.api.PrimitiveConverter]] that converts Parquet strings (fixed-length byte arrays) into Catalyst Strings. + * A [[parquet.io.api.PrimitiveConverter]] that converts Parquet strings (fixed-length byte arrays) + * into Catalyst Strings. * * @param parent The parent group converter. * @param fieldIndex The index inside the record. */ -class CatalystPrimitiveStringConverter(parent: CatalystGroupConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { - override def addBinary(value: Binary): Unit = parent.currentData.update(fieldIndex, value.toStringUsingUTF8.asInstanceOf[StringType.JvmType]) +class CatalystPrimitiveStringConverter( + parent: CatalystGroupConverter, + fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { + override def addBinary(value: Binary): Unit = + parent.currentData.update(fieldIndex, value.toStringUsingUTF8.asInstanceOf[StringType.JvmType]) } + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index b926fb481576f..2c9716f01ec4d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -209,6 +209,7 @@ package object dsl { def writeToFile(path: String) = WriteToFile(path, plan, None) - // TODO: for a loadFromFile it would be good to have a Catalog that knows how to resolve ParquetTables + // TODO: for a loadFromFile it would be good to have a Catalog that knows + // how to resolve ParquetTables } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 959d36e13c89a..bc24735d6ace0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -210,7 +210,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil case logical.WriteToFile(path, child, tableName) => - val relation = ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, tableName) + val relation = + ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, tableName) execution.InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil case _ => Nil } diff --git a/sql/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala index 16e4a4015d359..08d390e8871ab 100644 --- a/sql/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala +++ b/sql/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala @@ -195,4 +195,4 @@ object SparkHiveHadoopWriter { } outputPath.makeQualified(fs) } -} \ No newline at end of file +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index dcd421b28b6e7..b1fc7f8b47be4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -55,7 +55,10 @@ trait HiveStrategies { case p @ logical.Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m, None)(hiveContext) :: Nil case p @ logical.Project(projectList, r: ParquetRelation) if isSimpleProject(projectList) => - ParquetTableScan(projectList.asInstanceOf[Seq[Attribute]], r, None)(hiveContext.sparkContext) :: Nil + ParquetTableScan( + projectList.asInstanceOf[Seq[Attribute]], + r, + None)(hiveContext.sparkContext) :: Nil case m: MetastoreRelation => HiveTableScan(m.output, m, None)(hiveContext) :: Nil case p: ParquetRelation => @@ -128,10 +131,17 @@ trait HiveStrategies { } else { val scan = relation match { case MetastoreRelation(_, _, _) => { - HiveTableScan(prunedCols, relation.asInstanceOf[MetastoreRelation], None)(hiveContext) + HiveTableScan( + prunedCols, + relation.asInstanceOf[MetastoreRelation], + None)(hiveContext) } case ParquetRelation(_, _) => { - ParquetTableScan(relation.output, relation.asInstanceOf[ParquetRelation], None)(hiveContext.sparkContext).pruneColumns(prunedCols) + ParquetTableScan( + relation.output, + relation.asInstanceOf[ParquetRelation], + None)(hiveContext.sparkContext) + .pruneColumns(prunedCols) } } predicateOpt.map(execution.Filter(_, scan)).getOrElse(scan) :: Nil From 3ac9eb05d0cec3cca166503cb4dc417168694012 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 2 Mar 2014 19:23:06 +0100 Subject: [PATCH 732/778] Rebasing to new main branch --- project/SparkBuild.scala | 5 +++- .../catalyst/plans/logical/BaseRelation.scala | 1 + .../spark/sql/execution/ParquetRelation.scala | 0 .../execution/ParquetTableOperations.scala | 0 .../sql/execution/ParquetTableSupport.scala | 0 sql/core/src/test/resources/log4j.properties | 5 ++++ .../sql/execution/ParquetQueryTests.scala | 0 .../spark/sql/hive/HiveStrategies.scala | 2 +- .../hive}/execution/ParquetQuerySuite.scala | 23 ++++++++++--------- 9 files changed, 23 insertions(+), 13 deletions(-) rename {core => sql/core}/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala (100%) rename {core => sql/core}/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala (100%) rename {core => sql/core}/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala (100%) rename {core => sql/core}/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala (100%) rename {shark/src/test/scala/org/apache/spark/sql/shark => sql/hive/src/test/scala/org/apache/spark/sql/hive}/execution/ParquetQuerySuite.scala (90%) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0c27386621fa7..fed09340dfc67 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -356,7 +356,10 @@ object SparkBuild extends Build { ) def sqlCoreSettings = sharedSettings ++ Seq( - name := "spark-sql" + name := "spark-sql", + libraryDependencies ++= Seq( + "com.twitter" % "parquet-column" % "1.3.2", + "com.twitter" % "parquet-hadoop" % "1.3.2") ) def hiveSettings = sharedSettings ++ Seq( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala index e1bcad7bf28a5..48ff45c3d3ebb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala @@ -24,4 +24,5 @@ abstract class BaseRelation extends LeafNode { self: Product => def tableName: String + def isPartitioned: Boolean = false } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index 5e17e3b596ba1..9018681478fcd 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -45,3 +45,8 @@ log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF log4j.additivity.hive.ql.metadata.Hive=false log4j.logger.hive.ql.metadata.Hive=OFF +# Parquet logging +parquet.hadoop.InternalParquetRecordReader=INFO +log4j.logger.parquet.hadoop.InternalParquetRecordReader=INFO +parquet.hadoop.ParquetInputFormat=INFO +log4j.logger.parquet.hadoop.ParquetInputFormat=INFO diff --git a/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala similarity index 100% rename from core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index b1fc7f8b47be4..153c06493189b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -21,7 +21,7 @@ package hive import catalyst.expressions._ import catalyst.planning._ import catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} +import catalyst.plans.logical.{BaseRelation, LogicalPlan} import org.apache.spark.sql.execution._ diff --git a/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ParquetQuerySuite.scala similarity index 90% rename from shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQuerySuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ParquetQuerySuite.scala index 0ab412bb0591b..9d1b70e05eeea 100644 --- a/shark/src/test/scala/org/apache/spark/sql/shark/execution/ParquetQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ParquetQuerySuite.scala @@ -1,17 +1,18 @@ -package org.apache.spark.sql.execution -package shark +package org.apache.spark.sql +package hive +package execution import java.io.File import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite} import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.shark.TestShark import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.WriteToFile import org.apache.spark.sql.catalyst.plans.logical.InsertIntoCreatedTable import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.execution.{ParquetTestData, ParquetRelation} class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { @@ -20,14 +21,14 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf // runs a SQL and optionally resolves one Parquet table def runQuery(querystr: String, tableName: Option[String] = None, filename: Option[String] = None): Array[Row] = { // call to resolve references in order to get CREATE TABLE AS to work - val query = TestShark + val query = TestHive .parseSql(querystr) val finalQuery = if(tableName.nonEmpty && filename.nonEmpty) resolveParquetTable(tableName.get, filename.get, query) else query - TestShark.executePlan(finalQuery) + TestHive.executePlan(finalQuery) .toRdd .collect() } @@ -36,9 +37,9 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf def storeQuery(querystr: String, filename: String): Unit = { val query = WriteToFile( filename, - TestShark.parseSql(querystr), + TestHive.parseSql(querystr), Some("testtable")) - TestShark + TestHive .executePlan(query) .stringResult() } @@ -58,11 +59,11 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf if(name == tableName) { // note: at this stage the plan is not yet analyzed but Parquet needs to know the schema // and for that we need the child to be resolved - TestShark.loadTestTable("src") // may not be loaded now + TestHive.loadTestTable("src") // may not be loaded now val relation = ParquetRelation.create( filename, - TestShark.analyzer(child), - TestShark.sparkContext.hadoopConfiguration, + TestHive.analyzer(child), + TestHive.sparkContext.hadoopConfiguration, Some(tableName)) InsertIntoTable( relation.asInstanceOf[BaseRelation], @@ -82,7 +83,7 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf // write test data ParquetTestData.writeFile // Override initial Parquet test table - TestShark.catalog.overrideTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) + TestHive.catalog.overrideTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) } override def afterAll() { From 3bda72db9384b0f67cfbfbe22eb2674be113ceda Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 2 Mar 2014 21:59:23 +0100 Subject: [PATCH 733/778] Adding license banner to new files --- .../spark/sql/execution/ParquetRelation.scala | 17 ++++++++++++++++ .../execution/ParquetTableOperations.scala | 17 ++++++++++++++++ .../sql/execution/ParquetTableSupport.scala | 17 ++++++++++++++++ .../sql/execution/ParquetQueryTests.scala | 20 ++++++++++++++++++- .../hive/execution/ParquetQuerySuite.scala | 16 +++++++++++++++ 5 files changed, 86 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala index 97c3002ae9525..12a7d01ce3afd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.sql.execution import java.io.{IOException, FileNotFoundException, File} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala index 38da8c3bc74dd..09c48656f3c66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala @@ -1,3 +1,20 @@ +/* + * 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 package execution diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala index 636523c582758..1ffc8f814b29e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.sql.execution import org.apache.hadoop.conf.Configuration diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala index da4dbdfc08558..4e1f7fe8c0f19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.sql.execution import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -88,4 +105,5 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { val scanner = new ParquetTableScan(parquetRelation.attributes, parquetRelation, None)(TestSqlContext.sparkContext) scanner.execute } -} \ No newline at end of file +} + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ParquetQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ParquetQuerySuite.scala index 9d1b70e05eeea..2bf955590d622 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ParquetQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ParquetQuerySuite.scala @@ -1,3 +1,19 @@ +/* + * 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 package hive From d7fbc3a591110dae76121c1095a32ab4788ae005 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 26 Feb 2014 18:00:12 -0800 Subject: [PATCH 734/778] Several performance enhancements and simplifications of the expression evaluation framework. * Removed the Evaluate singleton in favor of placing expression evaluation code in each expression. * Instead of passing in a Seq of input rows we now take a single row. A mutable JoinedRow wrapper can be used in the relatively rare cases where expressions need to be evaluated on multiple input rows. * GenericRow now takes a raw Array[Any] instead of a Seq. Since GenericRow itself is a Seq wrapper, this avoids the creation of an unnecessary object. * A new concept called MutableLiteral can be used to evaluate aggregate expressions in-place, instead of needing to build new literal trees for each update. This part is more of a WIP as we still incur boxing, however this is a strict improvement over what was there before. --- .../catalyst/expressions/BoundAttribute.scala | 40 +-- .../spark/sql/catalyst/expressions/Cast.scala | 52 ++- .../sql/catalyst/expressions/Evaluate.scala | 297 ------------------ .../sql/catalyst/expressions/Expression.scala | 113 ++++++- .../catalyst/expressions/ImplementedUdf.scala | 27 -- .../sql/catalyst/expressions/Projection.scala | 66 ++++ .../spark/sql/catalyst/expressions/Row.scala | 8 +- .../sql/catalyst/expressions/ScalaUdf.scala | 13 +- .../catalyst/expressions/WrapDynamic.scala | 13 +- .../sql/catalyst/expressions/aggregates.scala | 53 ++-- .../sql/catalyst/expressions/arithmetic.scala | 24 +- .../catalyst/expressions/complexTypes.scala | 32 ++ .../sql/catalyst/expressions/generators.scala | 12 +- .../sql/catalyst/expressions/literals.scala | 19 ++ .../expressions/namedExpressions.scala | 4 + .../catalyst/expressions/nullFunctions.scala | 32 ++ .../sql/catalyst/expressions/predicates.scala | 119 ++++++- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../spark/sql/catalyst/util/package.scala | 8 + .../catalyst/ExpressionEvaluationSuite.scala | 6 +- .../apache/spark/sql/execution/Exchange.scala | 5 +- .../apache/spark/sql/execution/Generate.scala | 14 +- .../spark/sql/execution/SharkPlan.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 10 +- .../spark/sql/execution/aggregates.scala | 16 +- .../spark/sql/execution/basicOperators.scala | 9 +- .../apache/spark/sql/execution/joins.scala | 63 ++-- .../scala/org/apache/spark/sql/TgfSuite.scala | 10 +- .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../spark/sql/hive/ScriptTransformation.scala | 5 +- .../apache/spark/sql/hive/hiveOperators.scala | 8 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 33 +- 32 files changed, 634 insertions(+), 485 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index a7199ccb7b447..3b6bac16ff4e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -29,20 +29,24 @@ import catalyst.plans.QueryPlan * to be retrieved more efficiently. However, since operations like column pruning can change * the layout of intermediate tuples, BindReferences should be run after all such transformations. */ -case class BoundReference(inputTuple: Int, ordinal: Int, baseReference: Attribute) +case class BoundReference(ordinal: Int, baseReference: Attribute) extends Attribute with trees.LeafNode[Expression] { + type EvaluatedType = Any + def nullable = baseReference.nullable def dataType = baseReference.dataType def exprId = baseReference.exprId def qualifiers = baseReference.qualifiers def name = baseReference.name - def newInstance = BoundReference(inputTuple, ordinal, baseReference.newInstance) + def newInstance = BoundReference(ordinal, baseReference.newInstance) def withQualifiers(newQualifiers: Seq[String]) = - BoundReference(inputTuple, ordinal, baseReference.withQualifiers(newQualifiers)) + BoundReference(ordinal, baseReference.withQualifiers(newQualifiers)) + + override def toString = s"$baseReference:$ordinal" - override def toString = s"$baseReference:$inputTuple.$ordinal" + override def apply(input: Row): Any = input(ordinal) } class BindReferences[TreeNode <: QueryPlan[TreeNode]] extends Rule[TreeNode] { @@ -51,30 +55,28 @@ class BindReferences[TreeNode <: QueryPlan[TreeNode]] extends Rule[TreeNode] { def apply(plan: TreeNode): TreeNode = { plan.transform { case leafNode if leafNode.children.isEmpty => leafNode - case nonLeaf => nonLeaf.transformExpressions { case e => - bindReference(e, nonLeaf.children.map(_.output)) + case unaryNode if unaryNode.children.size == 1 => unaryNode.transformExpressions { case e => + bindReference(e, unaryNode.children.head.output) } } } } object BindReferences extends Logging { - def bindReference(expression: Expression, input: Seq[Seq[Attribute]]): Expression = { + def bindReference(expression: Expression, input: Seq[Attribute]): Expression = { expression.transform { case a: AttributeReference => attachTree(a, "Binding attribute") { - def inputAsString = input.map(_.mkString("{", ",", "}")).mkString(",") - - for { - (tuple, inputTuple) <- input.zipWithIndex - (attr, ordinal) <- tuple.zipWithIndex - if attr == a - } { - logger.debug(s"Binding $attr to $inputTuple.$ordinal given input $inputAsString") - return BoundReference(inputTuple, ordinal, a) + val ordinal = input.indexWhere(_.exprId == a.exprId) + if (ordinal == -1) { + // TODO: This fallback is required because some operators (such as ScriptTransform) + // produce new attributes that can't be bound. Likely the right thing to do is remove + // this rule and require all operators to explicitly bind to the input schema that + // they specify. + logger.debug(s"Couldn't find $a in ${input.mkString("[", ",", "]")}") + a + } else { + BoundReference(ordinal, a) } - - logger.debug(s"No binding found for $a given input $inputAsString") - a } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index ba77693a88da3..608656d3a90f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -19,11 +19,61 @@ package org.apache.spark.sql package catalyst package expressions -import types.DataType +import types._ /** Cast the child expression to the target data type. */ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { override def foldable = child.foldable def nullable = child.nullable override def toString = s"CAST($child, $dataType)" + + type EvaluatedType = Any + + lazy val castingFunction: Any => Any = (child.dataType, dataType) match { + case (BinaryType, StringType) => a: Any => new String(a.asInstanceOf[Array[Byte]]) + case (StringType, BinaryType) => a: Any => a.asInstanceOf[String].getBytes + case (_, StringType) => a: Any => a.toString + case (StringType, IntegerType) => a: Any => castOrNull(a, _.toInt) + case (StringType, DoubleType) => a: Any => castOrNull(a, _.toDouble) + case (StringType, FloatType) => a: Any => castOrNull(a, _.toFloat) + case (StringType, LongType) => a: Any => castOrNull(a, _.toLong) + case (StringType, ShortType) => a: Any => castOrNull(a, _.toShort) + case (StringType, ByteType) => a: Any => castOrNull(a, _.toByte) + case (StringType, DecimalType) => a: Any => castOrNull(a, BigDecimal(_)) + case (BooleanType, ByteType) => a: Any => a match { + case null => null + case true => 1.toByte + case false => 0.toByte + } + case (dt, IntegerType) => + a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toInt(a) + case (dt, DoubleType) => + a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toDouble(a) + case (dt, FloatType) => + a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toFloat(a) + case (dt, LongType) => + a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toLong(a) + case (dt, ShortType) => + a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toInt(a).toShort + case (dt, ByteType) => + a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toInt(a).toByte + case (dt, DecimalType) => + a: Any => + BigDecimal(dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toDouble(a)) + } + + @inline + protected def castOrNull[A](a: Any, f: String => A) = + try f(a.asInstanceOf[String]) catch { + case _: java.lang.NumberFormatException => null + } + + override def apply(input: Row): Any = { + val evaluated = child.apply(input) + if (evaluated == null) { + null + } else { + castingFunction(evaluated) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala deleted file mode 100644 index b58f06d20f9a8..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala +++ /dev/null @@ -1,297 +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 -package catalyst -package expressions - -import errors._ -import types._ - -/** - * Performs evaluation of an expression tree, given a set of input tuples. - */ -object Evaluate extends Logging { - def apply(e: Expression, input: Seq[Row]): Any = attachTree(e, "Expression Evaluation Failed") { - def eval(e: Expression) = Evaluate(e, input) - - /** - * A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type - * and do any casting necessary of child evaluation. - */ - @inline - def n1(e: Expression, f: ((Numeric[Any], Any) => Any)): Any = { - val evalE = eval(e) - if (evalE == null) { - null - } else { - e.dataType match { - case n: NumericType => - val castedFunction = f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType] - castedFunction(n.numeric, evalE.asInstanceOf[n.JvmType]) - case other => sys.error(s"Type $other does not support numeric operations") - } - } - } - - @inline - def n2(e1: Expression, e2: Expression, f: ((Numeric[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) { - throw new TreeNodeException(e, s"Types do not match ${e1.dataType} != ${e2.dataType}") - } - - val evalE1 = eval(e1) - val evalE2 = eval(e2) - if (evalE1 == null || evalE2 == null) { - null - } else { - e1.dataType match { - case n: NumericType => - f.asInstanceOf[(Numeric[n.JvmType], n.JvmType, n.JvmType) => Int]( - n.numeric, evalE1.asInstanceOf[n.JvmType], evalE2.asInstanceOf[n.JvmType]) - case other => sys.error(s"Type $other does not support numeric operations") - } - } - } - - @inline - def f2(e1: Expression, e2: Expression, f: ((Fractional[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) { - throw new TreeNodeException(e, s"Types do not match ${e1.dataType} != ${e2.dataType}") - } - - val evalE1 = eval(e1) - val evalE2 = eval(e2) - if (evalE1 == null || evalE2 == null) { - null - } else { - e1.dataType match { - case ft: FractionalType => - f.asInstanceOf[(Fractional[ft.JvmType], ft.JvmType, ft.JvmType) => ft.JvmType]( - ft.fractional, evalE1.asInstanceOf[ft.JvmType], evalE2.asInstanceOf[ft.JvmType]) - case other => sys.error(s"Type $other does not support fractional operations") - } - } - } - - @inline - def i2(e1: Expression, e2: Expression, f: ((Integral[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) { - throw new TreeNodeException(e, s"Types do not match ${e1.dataType} != ${e2.dataType}") - } - val evalE1 = eval(e1) - val evalE2 = eval(e2) - if (evalE1 == null || evalE2 == null) { - null - } else { - e1.dataType match { - case i: IntegralType => - f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( - i.integral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) - case other => sys.error(s"Type $other does not support numeric operations") - } - } - } - - @inline def castOrNull[A](e: Expression, f: String => A) = - try { - eval(e) match { - case null => null - case s: String => f(s) - } - } catch { case _: java.lang.NumberFormatException => null } - - val result = e match { - case Literal(v, _) => v - - /* Alias operations do not effect evaluation */ - case Alias(c, _) => eval(c) - - /* Aggregate functions are already computed so we just need to pull out the result */ - case af: AggregateFunction => af.result - - /* Arithmetic */ - case Add(l, r) => n2(l,r, _.plus(_, _)) - case Subtract(l, r) => n2(l,r, _.minus(_, _)) - case Multiply(l, r) => n2(l,r, _.times(_, _)) - // Divide implementation are different for fractional and integral dataTypes. - case Divide(l @ FractionalType(), r) => f2(l,r, _.div(_, _)) - case Divide(l @ IntegralType(), r) => i2(l,r, _.quot(_, _)) - // Remainder is only allowed on Integral types. - case Remainder(l, r) => i2(l,r, _.rem(_, _)) - case UnaryMinus(child) => n1(child, _.negate(_)) - - /* Control Flow */ - case If(e, t, f) => if (eval(e).asInstanceOf[Boolean]) eval(t) else eval(f) - - /* Comparisons */ - case Equals(l, r) => - val left = eval(l) - val right = eval(r) - if (left == null || right == null) null else left == right - - case In(value, list) => - val evaluatedValue = eval(value) - list.exists(e => eval(e) == evaluatedValue) - - // Strings - case GreaterThan(l, r) if l.dataType == StringType && r.dataType == StringType => - eval(l).asInstanceOf[String] > eval(r).asInstanceOf[String] - case GreaterThanOrEqual(l, r) if l.dataType == StringType && r.dataType == StringType => - eval(l).asInstanceOf[String] >= eval(r).asInstanceOf[String] - case LessThan(l, r) if l.dataType == StringType && r.dataType == StringType => - eval(l).asInstanceOf[String] < eval(r).asInstanceOf[String] - case LessThanOrEqual(l, r) if l.dataType == StringType && r.dataType == StringType => - eval(l).asInstanceOf[String] <= eval(r).asInstanceOf[String] - - // Numerics - case GreaterThan(l, r) => n2(l, r, _.gt(_, _)) - case GreaterThanOrEqual(l, r) => n2(l, r, _.gteq(_, _)) - case LessThan(l, r) => n2(l, r, _.lt(_, _)) - case LessThanOrEqual(l, r) => n2(l, r, _.lteq(_, _)) - - case IsNull(e) => eval(e) == null - case IsNotNull(e) => eval(e) != null - case Coalesce(exprs) => - var currentExpression: Any = null - var i = 0 - while (i < exprs.size && currentExpression == null) { - currentExpression = eval(exprs(i)) - i += 1 - } - currentExpression - - /* Casts */ - - // Binary Conversions - case Cast(e @ BinaryType(), StringType) => new String(eval(e).asInstanceOf[Array[Byte]]) - case Cast(e @ StringType(), BinaryType) => eval(e).asInstanceOf[String].getBytes - - // toString - case Cast(e, StringType) => - eval(e) match { - case null => null - case other => other.toString - } - - // String => Numeric Types - case Cast(e @ StringType(), IntegerType) => castOrNull(e, _.toInt) - case Cast(e @ StringType(), DoubleType) => castOrNull(e, _.toDouble) - case Cast(e @ StringType(), FloatType) => castOrNull(e, _.toFloat) - case Cast(e @ StringType(), LongType) => castOrNull(e, _.toLong) - case Cast(e @ StringType(), ShortType) => castOrNull(e, _.toShort) - case Cast(e @ StringType(), ByteType) => castOrNull(e, _.toByte) - case Cast(e @ StringType(), DecimalType) => castOrNull(e, BigDecimal(_)) - - // Boolean conversions - case Cast(e, ByteType) if e.dataType == BooleanType => - eval(e) match { - case null => null - case true => 1.toByte - case false => 0.toByte - } - - // Numeric Type => Numeric Type - case Cast(e, IntegerType) => n1(e, _.toInt(_)) - case Cast(e, DoubleType) => n1(e, _.toDouble(_)) - case Cast(e, FloatType) => n1(e, _.toFloat(_)) - case Cast(e, LongType) => n1(e, _.toLong(_)) - case Cast(e, ShortType) => n1(e, _.toInt(_).toShort) - case Cast(e, ByteType) => n1(e, _.toInt(_).toByte) - case Cast(e, DecimalType) => n1(e, (n,v) => BigDecimal(n.toDouble(v))) - - /* Boolean Logic */ - case Not(c) => - eval(c) match { - case null => null - case b: Boolean => !b - } - - case And(l,r) => - val left = eval(l) - val right = eval(r) - if (left == false || right == false) { - false - } else if (left == null || right == null ) { - null - } else { - true - } - case Or(l,r) => - val left = eval(l) - val right = eval(r) - if (left == true || right == true) { - true - } else if (left == null || right == null) { - null - } else { - false - } - - /* References to input tuples */ - case br @ BoundReference(inputTuple, ordinal, _) => try input(inputTuple)(ordinal) catch { - case iob: IndexOutOfBoundsException => - throw new TreeNodeException(br, s"Reference not in tuple: $input") - } - - /* Functions */ - case Rand => scala.util.Random.nextDouble() - - /* Complex Type Access */ - case g @ GetField(e, name) => - val baseValue = eval(e).asInstanceOf[Row] - if (baseValue == null) null else baseValue(g.ordinal) - - case GetItem(e, o) if e.dataType.isInstanceOf[ArrayType] => - val baseValue = eval(e).asInstanceOf[Seq[_]] - val ordinal = eval(o).asInstanceOf[Int] - if (baseValue == null) { - null - } else if (ordinal >= baseValue.size) { - null - } else { - baseValue(ordinal) - } - - case GetItem(e, k) if e.dataType.isInstanceOf[MapType] => - val baseValue = eval(e).asInstanceOf[Map[Any, _]] - val key = eval(k) - if (baseValue == null) { - null - } else { - baseValue.get(key).orNull - } - - /* UDFs */ - case implementedFunction: ImplementedUdf => - try implementedFunction.evaluate(implementedFunction.children.map(eval)) catch { - case e: Exception => - logger.error(s"UDF Evaluation failed: $e") - null - } - - case a: Attribute => - throw new TreeNodeException(a, - "Unable to evaluate unbound reference without access to the input schema.") - case other => throw new TreeNodeException(other, "evaluation not implemented") - } - - val resultType = if (result == null) "null" else result.getClass.getName - logger.debug(s"Evaluated $e => $result of type $resultType, expected: ${e.dataType}") - result - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 0655e280b31c1..68da30c47ce16 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -19,12 +19,16 @@ package org.apache.spark.sql package catalyst package expressions +import errors._ import trees._ import types._ abstract class Expression extends TreeNode[Expression] { self: Product => + /** The narrowest possible type that is produced when this expression is evaluated. */ + type EvaluatedType <: Any + def dataType: DataType /** * Returns true when an expression is a candidate for static evaluation before the query is @@ -45,18 +49,119 @@ abstract class Expression extends TreeNode[Expression] { def nullable: Boolean def references: Set[Attribute] + /** Returns the result of evaluating this expression on a given input Row */ + def apply(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + def applyBoolean(input: Row): Boolean = apply(input).asInstanceOf[Boolean] + def applyInt(input: Row): Int = apply(input).asInstanceOf[Int] + def applyDouble(input: Row): Double = apply(input).asInstanceOf[Double] + def applyString(input: Row): String = apply(input).asInstanceOf[String] + /** - * Returns true if this expression and all its children have been resolved to a specific schema - * and false if it is still contains any unresolved placeholders. Implementations of expressions - * should override this. + * Returns `true` if this expression and all its children have been resolved to a specific schema + * and `false` if it is still contains any unresolved placeholders. Implementations of expressions + * should override this if the resolution of this type of expression involves more than just + * the resolution of its children. */ lazy val resolved: Boolean = childrenResolved /** * Returns true if all the children of this expression have been resolved to a specific schema - * and false if it is still contains any unresolved placeholders. + * and false if any still contains any unresolved placeholders. */ def childrenResolved = !children.exists(!_.resolved) + + /** + * A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type + * and do any casting necessary of child evaluation. + */ + @inline + def n1(e: Expression, i: Row, f: ((Numeric[Any], Any) => Any)): Any = { + val evalE = e.apply(i) + if (evalE == null) { + null + } else { + e.dataType match { + case n: NumericType => + val castedFunction = f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType] + castedFunction(n.numeric, evalE.asInstanceOf[n.JvmType]) + case other => sys.error(s"Type $other does not support numeric operations") + } + } + } + + @inline + protected final def n2( + i: Row, + e1: Expression, + e2: Expression, + f: ((Numeric[Any], Any, Any) => Any)): Any = { + + if (e1.dataType != e2.dataType) { + throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") + } + + val evalE1 = e1.apply(i) + val evalE2 = e2.apply(i) + if (evalE1 == null || evalE2 == null) { + null + } else { + e1.dataType match { + case n: NumericType => + f.asInstanceOf[(Numeric[n.JvmType], n.JvmType, n.JvmType) => Int]( + n.numeric, evalE1.asInstanceOf[n.JvmType], evalE2.asInstanceOf[n.JvmType]) + case other => sys.error(s"Type $other does not support numeric operations") + } + } + } + + @inline + protected final def f2( + i: Row, + e1: Expression, + e2: Expression, + f: ((Fractional[Any], Any, Any) => Any)): Any = { + if (e1.dataType != e2.dataType) { + throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") + } + + val evalE1 = e1.apply(i: Row) + val evalE2 = e2.apply(i: Row) + if (evalE1 == null || evalE2 == null) { + null + } else { + e1.dataType match { + case ft: FractionalType => + f.asInstanceOf[(Fractional[ft.JvmType], ft.JvmType, ft.JvmType) => ft.JvmType]( + ft.fractional, evalE1.asInstanceOf[ft.JvmType], evalE2.asInstanceOf[ft.JvmType]) + case other => sys.error(s"Type $other does not support fractional operations") + } + } + } + + @inline + protected final def i2( + i: Row, + e1: Expression, + e2: Expression, + f: ((Integral[Any], Any, Any) => Any)): Any = { + if (e1.dataType != e2.dataType) { + throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") + } + val evalE1 = e1.apply(i) + val evalE2 = e2.apply(i) + if (evalE1 == null || evalE2 == null) { + null + } else { + e1.dataType match { + case i: IntegralType => + f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( + i.integral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) + case other => sys.error(s"Type $other does not support numeric operations") + } + } + } } abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala deleted file mode 100644 index f11885ab3f7af..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala +++ /dev/null @@ -1,27 +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 -package catalyst -package expressions - -/** - * A UDF that has a native JVM implementation. - */ -trait ImplementedUdf { - def evaluate(evaluatedChildren: Seq[Any]): Any -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala new file mode 100644 index 0000000000000..56ba040c72bd4 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -0,0 +1,66 @@ +package org.apache.spark.sql.catalyst +package expressions + +/** + * Converts a Row to another Row given a set of expressions. + * + * If the schema of the input row is specified, then the given expression will be bound to that + * schema. + */ +class Projection(expressions: Seq[Expression]) extends (Row => Row) { + def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = + this(expressions.map(BindReferences.bindReference(_, inputSchema))) + + protected val exprArray = expressions.toArray + def apply(input: Row): Row = { + val outputArray = new Array[Any](exprArray.size) + var i = 0 + while (i < exprArray.size) { + outputArray(i) = exprArray(i).apply(input) + i += 1 + } + new GenericRow(outputArray) + } +} + +/** + * A mutable wrapper that makes two rows appear appear as a single concatenated row. Designed to + * be instantiated once per thread and reused. + */ +class JoinedRow extends Row { + private var row1: Row = _ + private var row2: Row = _ + + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = apply(i) == null + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index d5c8785f6e884..a7b8662470f69 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -65,9 +65,7 @@ object EmptyRow extends Row { /** * A row implementation that uses an array of objects as the underlying storage. */ -class GenericRow(input: Seq[Any]) extends Row { - val values = input.toIndexedSeq - +class GenericRow(val values: Array[Any]) extends Row { def iterator = values.iterator def length = values.length @@ -107,8 +105,8 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { var i = 0 while (i < ordering.size) { val order = ordering(i) - val left = Evaluate(order.child, Vector(a)) - val right = Evaluate(order.child, Vector(b)) + val left = order.child.apply(a) + val right = order.child.apply(b) if (left == null && right == null) { // Both null, continue looking. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 6a6fef41973bb..a3c7ca1acdb0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -22,15 +22,20 @@ package expressions import types._ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) - extends Expression with ImplementedUdf { + extends Expression { + + type EvaluatedType = Any def references = children.flatMap(_.references).toSet def nullable = true - def evaluate(evaluatedChildren: Seq[Any]): Any = { + override def apply(input: Row): Any = { children.size match { - case 1 => function.asInstanceOf[(Any) => Any](evaluatedChildren(0)) - case 2 => function.asInstanceOf[(Any, Any) => Any](evaluatedChildren(0), evaluatedChildren(1)) + case 1 => function.asInstanceOf[(Any) => Any](children(0).apply(input)) + case 2 => + function.asInstanceOf[(Any, Any) => Any]( + children(0).apply(input), + children(1).apply(input)) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index c87d3783a37ca..2ad8d6f31d2e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -25,16 +25,21 @@ import types._ case object DynamicType extends DataType -case class WrapDynamic(children: Seq[Attribute]) extends Expression with ImplementedUdf { +case class WrapDynamic(children: Seq[Attribute]) extends Expression { + type EvaluatedType = DynamicRow + def nullable = false def references = children.toSet def dataType = DynamicType - def evaluate(evaluatedChildren: Seq[Any]): Any = - new DynamicRow(children, evaluatedChildren) + override def apply(input: Row): DynamicRow = input match { + // Avoid copy for generic rows. + case g: GenericRow => new DynamicRow(children, g.values) + case otherRowType => new DynamicRow(children, otherRowType.toArray) + } } -class DynamicRow(val schema: Seq[Attribute], values: Seq[Any]) +class DynamicRow(val schema: Seq[Attribute], values: Array[Any]) extends GenericRow(values) with Dynamic { def selectDynamic(attributeName: String): String = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 8052d426cb4e4..aec0ca1172171 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -64,14 +64,16 @@ abstract class AggregateFunction extends AggregateExpression with Serializable with trees.LeafNode[Expression] { self: Product => + type EvaluatedType = Any + /** Base should return the generic aggregate expression that this function is computing */ val base: AggregateExpression def references = base.references def nullable = base.nullable def dataType = base.dataType - def apply(input: Seq[Row]): Unit - def result: Any + def update(input: Row): Unit + override def apply(input: Row): Any // Do we really need this? def newInstance = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) @@ -168,14 +170,16 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) def this() = this(null, null) // Required for serialization. var count: Long = _ - var sum: Long = _ + val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) + val sumAsDouble = Cast(sum, DoubleType) + + val addFunction = Add(sum, expr) - def result: Any = sum.toDouble / count.toDouble + override def apply(input: Row): Any = sumAsDouble.applyDouble(null) / count.toDouble - def apply(input: Seq[Row]): Unit = { + def update(input: Row): Unit = { count += 1 - // TODO: Support all types here... - sum += Evaluate(expr, input).asInstanceOf[Int] + sum.update(addFunction, input) } } @@ -184,25 +188,28 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag var count: Int = _ - def apply(input: Seq[Row]): Unit = { - val evaluatedExpr = expr.map(Evaluate(_, input)) + def update(input: Row): Unit = { + val evaluatedExpr = expr.map(_.apply(input)) if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { count += 1 } } - def result: Any = count + override def apply(input: Row): Any = count } case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - var sum = Evaluate(Cast(Literal(0), expr.dataType), Nil) + val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) - def apply(input: Seq[Row]): Unit = - sum = Evaluate(Add(Literal(sum), expr), input) + val addFunction = Add(sum, expr) - def result: Any = sum + def update(input: Row): Unit = { + sum.update(addFunction, input) + } + + override def apply(input: Row): Any = sum.apply(null) } case class SumDistinctFunction(expr: Expression, base: AggregateExpression) @@ -212,14 +219,14 @@ case class SumDistinctFunction(expr: Expression, base: AggregateExpression) val seen = new scala.collection.mutable.HashSet[Any]() - def apply(input: Seq[Row]): Unit = { - val evaluatedExpr = Evaluate(expr, input) + def update(input: Row): Unit = { + val evaluatedExpr = expr.apply(input) if (evaluatedExpr != null) { seen += evaluatedExpr } } - def result: Any = + override def apply(input: Row): Any = seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus) } @@ -230,14 +237,14 @@ case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpressio val seen = new scala.collection.mutable.HashSet[Any]() - def apply(input: Seq[Row]): Unit = { - val evaluatedExpr = expr.map(Evaluate(_, input)) + def update(input: Row): Unit = { + val evaluatedExpr = expr.map(_.apply(input)) if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { seen += evaluatedExpr } } - def result: Any = seen.size + override def apply(input: Row): Any = seen.size } case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -245,9 +252,11 @@ case class FirstFunction(expr: Expression, base: AggregateExpression) extends Ag var result: Any = null - def apply(input: Seq[Row]): Unit = { + def update(input: Row): Unit = { if (result == null) { - result = Evaluate(expr, input) + result = expr.apply(input) } } + + override def apply(input: Row): Any = result } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 38b52ccc7cdcd..db235645cd85c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -20,18 +20,26 @@ package catalyst package expressions import catalyst.analysis.UnresolvedException - +import catalyst.types._ case class UnaryMinus(child: Expression) extends UnaryExpression { + type EvaluatedType = Any + def dataType = child.dataType override def foldable = child.foldable def nullable = child.nullable override def toString = s"-$child" + + override def apply(input: Row): Any = { + n1(child, input, _.negate(_)) + } } abstract class BinaryArithmetic extends BinaryExpression { self: Product => + type EvaluatedType = Any + def nullable = left.nullable || right.nullable override lazy val resolved = @@ -48,20 +56,34 @@ abstract class BinaryArithmetic extends BinaryExpression { case class Add(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "+" + + override def apply(input: Row): Any = n2(input, left, right, _.plus(_, _)) } case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "-" + + override def apply(input: Row): Any = n2(input, left, right, _.minus(_, _)) } case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "*" + + override def apply(input: Row): Any = n2(input, left, right, _.times(_, _)) } case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "/" + + override def apply(input: Row): Any = dataType match { + case _: FractionalType => f2(input, left, right, _.div(_, _)) + case _: IntegralType => i2(input, left , right, _.quot(_, _)) + } + } case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "%" + + override def apply(input: Row): Any = i2(input, left, right, _.rem(_, _)) } 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 3bd2ffa1087b9..2962c623d4d9a 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 @@ -25,6 +25,8 @@ import types._ * Returns the item at `ordinal` in the Array `child` or the Key `ordinal` in Map `child`. */ case class GetItem(child: Expression, ordinal: Expression) extends Expression { + type EvaluatedType = Any + val children = child :: ordinal :: Nil /** `Null` is returned for invalid ordinals. */ override def nullable = true @@ -38,12 +40,36 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) override def toString = s"$child[$ordinal]" + + override def apply(input: Row): Any = { + if (child.dataType.isInstanceOf[ArrayType]) { + val baseValue = child.apply(input).asInstanceOf[Seq[_]] + val o = ordinal.apply(input).asInstanceOf[Int] + if (baseValue == null) { + null + } else if (o >= baseValue.size) { + null + } else { + baseValue(o) + } + } else { + val baseValue = child.apply(input).asInstanceOf[Map[Any, _]] + val key = ordinal.apply(input) + if (baseValue == null) { + null + } else { + baseValue.get(key).orNull + } + } + } } /** * Returns the value of fields in the Struct `child`. */ case class GetField(child: Expression, fieldName: String) extends UnaryExpression { + type EvaluatedType = Any + def dataType = field.dataType def nullable = field.nullable @@ -60,5 +86,11 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio lazy val ordinal = structType.fields.indexOf(field) override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType] + + override def apply(input: Row): Any = { + val baseValue = child.apply(input).asInstanceOf[Row] + if (baseValue == null) null else baseValue(ordinal) + } + override def toString = s"$child.$fieldName" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 06524bd0600d4..c367de2a3e570 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -39,6 +39,8 @@ import catalyst.types._ abstract class Generator extends Expression with (Row => TraversableOnce[Row]) { self: Product => + type EvaluatedType = TraversableOnce[Row] + lazy val dataType = ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable)))) @@ -99,14 +101,14 @@ case class Explode(attributeNames: Seq[String], child: Expression) } } - def apply(input: Row): TraversableOnce[Row] = { + override def apply(input: Row): TraversableOnce[Row] = { child.dataType match { case ArrayType(_) => - val inputArray = Evaluate(child, Vector(input)).asInstanceOf[Seq[Any]] - if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Vector(v))) + val inputArray = child.apply(input).asInstanceOf[Seq[Any]] + if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Array(v))) case MapType(_, _) => - val inputMap = Evaluate(child, Vector(input)).asInstanceOf[Map[Any,Any]] - if (inputMap == null) Nil else inputMap.map { case (k,v) => new GenericRow(Vector(k,v)) } + val inputMap = child.apply(input).asInstanceOf[Map[Any,Any]] + if (inputMap == null) Nil else inputMap.map { case (k,v) => new GenericRow(Array(k,v)) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index cc78513e78fd1..229d8f7f7b763 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -46,9 +46,28 @@ object IntegerLiteral { } case class Literal(value: Any, dataType: DataType) extends LeafExpression { + override def foldable = true def nullable = value == null def references = Set.empty override def toString = if (value != null) value.toString else "null" + + type EvaluatedType = Any + override def apply(input: Row):Any = value +} + +// TODO: Specialize +case class MutableLiteral(var value: Any, nullable: Boolean = true) extends LeafExpression { + type EvaluatedType = Any + + val dataType = Literal(value).dataType + + def references = Set.empty + + def update(expression: Expression, input: Row) = { + value = expression.apply(input) + } + + override def apply(input: Row) = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 923108ce44906..0a06e85325a22 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -78,6 +78,10 @@ case class Alias(child: Expression, name: String) (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends NamedExpression with trees.UnaryNode[Expression] { + type EvaluatedType = Any + + override def apply(input: Row) = child.apply(input) + def dataType = child.dataType def nullable = child.nullable def references = child.references diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index 757bbd40ac40f..32efb746985d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -22,6 +22,7 @@ package expressions import catalyst.analysis.UnresolvedException case class Coalesce(children: Seq[Expression]) extends Expression { + type EvaluatedType = Any /** Coalesce is nullable if all of its children are nullable, or if it has no children. */ def nullable = !children.exists(!_.nullable) @@ -40,4 +41,35 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } else { throw new UnresolvedException(this, "Coalesce cannot have children of different types.") } + + override def apply(input: Row): Any = { + var i = 0 + var result: Any = null + while(i < children.size && result == null) { + result = children(i).apply(input) + i += 1 + } + result + } } + +case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { + def references = child.references + override def foldable = child.foldable + def nullable = false + + override def apply(input: Row): Any = { + child.apply(input) == null + } +} + +case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { + def references = child.references + override def foldable = child.foldable + def nullable = false + override def toString = s"IS NOT NULL $child" + + override def apply(input: Row): Any = { + child.apply(input) != null + } +} \ No newline at end of file diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 60c22bfa97c79..561396eb4373c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -26,9 +26,11 @@ trait Predicate extends Expression { self: Product => def dataType = BooleanType + + type EvaluatedType = Any } -abstract trait PredicateHelper { +trait PredicateHelper { def splitConjunctivePredicates(condition: Expression): Seq[Expression] = condition match { case And(cond1, cond2) => splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) case other => other :: Nil @@ -45,6 +47,13 @@ case class Not(child: Expression) extends Predicate with trees.UnaryNode[Express override def foldable = child.foldable def nullable = child.nullable override def toString = s"NOT $child" + + override def apply(input: Row): Any = { + child.apply(input) match { + case null => null + case b: Boolean => !b + } + } } /** @@ -55,14 +64,43 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { def references = children.flatMap(_.references).toSet def nullable = true // TODO: Figure out correct nullability semantics of IN. override def toString = s"$value IN ${list.mkString("(", ",", ")")}" + + override def apply(input: Row): Any = { + val evaluatedValue = value.apply(input) + list.exists(e => e.apply(input) == evaluatedValue) + } } case class And(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "&&" + + override def apply(input: Row): Any = { + val l = left.apply(input) + val r = right.apply(input) + if (l == false || r == false) { + false + } else if (l == null || r == null ) { + null + } else { + true + } + } } case class Or(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "||" + + override def apply(input: Row): Any = { + val l = left.apply(input) + val r = right.apply(input) + if (l == true || r == true) { + true + } else if (l == null || r == null) { + null + } else { + false + } + } } abstract class BinaryComparison extends BinaryPredicate { @@ -71,35 +109,79 @@ abstract class BinaryComparison extends BinaryPredicate { case class Equals(left: Expression, right: Expression) extends BinaryComparison { def symbol = "=" + override def apply(input: Row): Any = { + val l = left.apply(input) + val r = right.apply(input) + if (l == null || r == null) null else l == r + } } case class LessThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<" + override def apply(input: Row): Any = { + if (left.dataType == StringType && right.dataType == StringType) { + val l = left.apply(input) + val r = right.apply(input) + if(l == null || r == null) { + null + } else { + l.asInstanceOf[String] < r.asInstanceOf[String] + } + } else { + n2(input, left, right, _.lt(_, _)) + } + } } case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<=" + override def apply(input: Row): Any = { + if (left.dataType == StringType && right.dataType == StringType) { + val l = left.apply(input) + val r = right.apply(input) + if(l == null || r == null) { + null + } else { + l.asInstanceOf[String] <= r.asInstanceOf[String] + } + } else { + n2(input, left, right, _.lteq(_, _)) + } + } } case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">" + override def apply(input: Row): Any = { + if (left.dataType == StringType && right.dataType == StringType) { + val l = left.apply(input) + val r = right.apply(input) + if(l == null || r == null) { + null + } else { + l.asInstanceOf[String] > r.asInstanceOf[String] + } + } else { + n2(input, left, right, _.gt(_, _)) + } + } } case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">=" -} - -case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references - override def foldable = child.foldable - def nullable = false -} - -case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references - override def foldable = child.foldable - def nullable = false - override def toString = s"IS NOT NULL $child" + override def apply(input: Row): Any = { + if (left.dataType == StringType && right.dataType == StringType) { + val l = left.apply(input) + val r = right.apply(input) + if(l == null || r == null) { + null + } else { + l.asInstanceOf[String] >= r.asInstanceOf[String] + } + } else { + n2(input, left, right, _.gteq(_, _)) + } + } } case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) @@ -118,5 +200,14 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi trueValue.dataType } + type EvaluatedType = Any + override def apply(input: Row): Any = { + if (predicate(input).asInstanceOf[Boolean]) { + trueValue.apply(input) + } else { + falseValue.apply(input) + } + } + override def toString = s"if ($predicate) $trueValue else $falseValue" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 3588a9d87a728..4db280317345a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -60,7 +60,7 @@ object ConstantFolding extends Rule[LogicalPlan] { case q: LogicalPlan => q transformExpressionsDown { // Skip redundant folding of literals. case l: Literal => l - case e if e.foldable => Literal(Evaluate(e, Nil), e.dataType) + case e if e.foldable => Literal(e.apply(null), e.dataType) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index b4ec176966826..52adea2661363 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -105,6 +105,14 @@ package object util { def stringOrNull(a: AnyRef) = if (a == null) null else a.toString + def benchmark[A](f: => A): A = { + val startTime = System.nanoTime() + val ret = f + val endTime = System.nanoTime() + println(s"${(endTime - startTime).toDouble / 1000000}ms") + ret + } + /* FIX ME implicit class debugLogging(a: AnyRef) { def debugLogging() { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala index d5cc0724f5112..094143c9abb1d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala @@ -29,7 +29,7 @@ import dsl._ class ExpressionEvaluationSuite extends FunSuite { test("literals") { - assert(Evaluate(Literal(1) + Literal(1), Nil) === 2) + assert((Literal(1) + Literal(1)).apply(null) === 2) } /** @@ -62,7 +62,7 @@ class ExpressionEvaluationSuite extends FunSuite { notTrueTable.foreach { case (v, answer) => val expr = Not(Literal(v, BooleanType)) - val result = Evaluate(expr, Nil) + val result = expr.apply(null) if (result != answer) fail(s"$expr should not evaluate to $result, expected: $answer") } } @@ -105,7 +105,7 @@ class ExpressionEvaluationSuite extends FunSuite { truthTable.foreach { case (l,r,answer) => val expr = op(Literal(l, BooleanType), Literal(r, BooleanType)) - val result = Evaluate(expr, Nil) + val result = expr.apply(null) if (result != answer) fail(s"$expr should not evaluate to $result, expected: $answer") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 488e9d2a3e503..4226a1a85509c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -35,8 +35,9 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una newPartitioning match { case HashPartitioning(expressions, numPartitions) => { // TODO: Eliminate redundant expressions in grouping key and value. - val rdd = child.execute().map { row => - (buildRow(expressions.toSeq.map(Evaluate(_, Vector(row)))), row) + val rdd = child.execute().mapPartitions { iter => + val hashExpressions = new Projection(expressions) + iter.map(r => (hashExpressions(r), r)) } val part = new HashPartitioner(numPartitions) val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index 0c7ab8771a6a8..c1da3653c5b8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -43,14 +43,22 @@ case class Generate( def execute() = { if (join) { - val outerNulls = Seq.fill(generator.output.size)(null) child.execute().mapPartitions { iter => + val nullValues = Seq.fill(generator.output.size)(Literal(null)) + // Used to produce rows with no matches when outer = true. + val outerProjection = + new Projection(child.output ++ nullValues, child.output) + + val joinProjection = + new Projection(child.output ++ generator.output, child.output ++ generator.output) + val joinedRow = new JoinedRow + iter.flatMap {row => val outputRows = generator(row) if (outer && outputRows.isEmpty) { - new GenericRow(row ++ outerNulls) :: Nil + outerProjection(row) :: Nil } else { - outputRows.map(or => new GenericRow(row ++ or)) + outputRows.map(or => joinProjection(joinedRow(row, or))) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala index 85facee367d55..5b57408d32ffe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala @@ -44,7 +44,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { */ def executeCollect(): Array[Row] = execute().collect() - protected def buildRow(values: Seq[Any]): Row = new catalyst.expressions.GenericRow(values) + protected def buildRow(values: Seq[Any]): Row = new catalyst.expressions.GenericRow(values.toArray) } trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index d7cbe1c9e2be5..b3a4f8dd3c6ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -152,11 +152,11 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } protected lazy val singleRowRdd = - sparkContext.parallelize(Seq(new GenericRow(IndexedSeq()): Row), 1) + sparkContext.parallelize(Seq(new GenericRow(Array()): Row), 1) def convertToCatalyst(a: Any): Any = a match { case s: Seq[Any] => s.map(convertToCatalyst) - case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toSeq) + case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) case other => other } @@ -164,7 +164,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.StopAfter(limit, logical.Sort(order, child)) => execution.TopK( - Evaluate(limit, Nil).asInstanceOf[Int], order, planLater(child))(sparkContext) :: Nil + limit.applyInt(null), order, planLater(child))(sparkContext) :: Nil case _ => Nil } } @@ -195,11 +195,11 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.LocalRelation(output, data) => val dataAsRdd = sparkContext.parallelize(data.map(r => - new GenericRow(r.productIterator.map(convertToCatalyst).toVector): Row)) + new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row)) execution.ExistingRdd(output, dataAsRdd) :: Nil case logical.StopAfter(limit, child) => execution.StopAfter( - Evaluate(limit, Nil).asInstanceOf[Int], + limit.applyInt(null), planLater(child))(sparkContext) :: Nil case Unions(unionChildren) => execution.Union(unionChildren.map(planLater))(sparkContext) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala index f94eb6801d230..132576cbc4fae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala @@ -74,7 +74,7 @@ case class Aggregate( // An exact match with a grouping expression val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { case -1 => None - case ordinal => Some(BoundReference(0, ordinal, Alias(impl, "AGGEXPR")().toAttribute)) + case ordinal => Some(BoundReference(ordinal, Alias(impl, "AGGEXPR")().toAttribute)) } exactGroupingExpr.getOrElse( @@ -88,8 +88,9 @@ case class Aggregate( // TODO: If the child of it is an [[catalyst.execution.Exchange]], // do not evaluate the groupingExpressions again since we have evaluated it // in the [[catalyst.execution.Exchange]]. - val grouped = child.execute().map { row => - (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row) + val grouped = child.execute().mapPartitions { iter => + val buildGrouping = new Projection(groupingExpressions) + iter.map(row => (buildGrouping(row), row)) }.groupByKeyLocally() val result = grouped.map { case (group, rows) => @@ -99,17 +100,16 @@ case class Aggregate( val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f }) rows.foreach { row => - val input = Vector(row) - aggFunctions.foreach(_.apply(input)) + aggFunctions.foreach(_.update(row)) } - buildRow(aggImplementations.map(Evaluate(_, Vector(group)))) + buildRow(aggImplementations.map(_.apply(group))) } - // TODO: THIS DOES NOT PRESERVE LINEAGE AND BREAKS PIPELINING. + // TODO: THIS BREAKS PIPELINING, DOUBLE COMPUTES THE ANSWER, AND USES TOO MUCH MEMORY... if (groupingExpressions.isEmpty && result.count == 0) { // When there there is no output to the Aggregate operator, we still output an empty row. val aggImplementations = createAggregateImplementations() - sc.makeRDD(buildRow(aggImplementations.map(Evaluate(_, Nil))) :: Nil) + sc.makeRDD(buildRow(aggImplementations.map(_.apply(null))) :: Nil) } else { result } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 95278f646b7c0..9a61c021fceea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -29,15 +29,16 @@ import catalyst.plans.logical.LogicalPlan case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) - def execute() = child.execute().map { row => - buildRow(projectList.map(Evaluate(_, Vector(row)))) + def execute() = child.execute().mapPartitions { iter => + val buildProjection = new Projection(projectList) + iter.map(buildProjection) } } case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { def output = child.output - def execute() = child.execute().filter { row => - Evaluate(condition, Vector(row)).asInstanceOf[Boolean] + def execute() = child.execute().mapPartitions { iter => + iter.filter(condition.applyBoolean) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index bb7edd6fb3c74..d44905dc84260 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -44,17 +44,14 @@ case class SparkEquiInnerJoin( def output = left.output ++ right.output def execute() = attachTree(this, "execute") { - val leftWithKeys = left.execute().map { row => - val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) - //logger.debug(s"leftkey [${leftKeys.mkString(",")}] => ${joinKeys.mkString(",")}] from $row") - (joinKeys, row) + val leftWithKeys = left.execute().mapPartitions { iter => + val generateLeftKeys = new Projection(leftKeys, left.output) + iter.map(row => (generateLeftKeys(row), row)) } - val rightWithKeys = right.execute().map { row => - val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) - //logger.debug(s"rightkey [${leftKeys.mkString(",")}] => ${joinKeys.mkString(",")}] from $row") - - (joinKeys, row) + val rightWithKeys = right.execute().mapPartitions { iter => + val generateRightKeys = new Projection(rightKeys, right.output) + iter.map(row => (generateRightKeys(row), row)) } // Do the join. @@ -67,7 +64,7 @@ case class SparkEquiInnerJoin( * Filters any rows where the any of the join keys is null, ensuring three-valued * logic for the equi-join conditions. */ - protected def filterNulls(rdd: RDD[(Seq[Any], Row)]) = + protected def filterNulls(rdd: RDD[(Row, Row)]) = rdd.filter { case (key: Seq[_], _) => !key.exists(_ == null) } @@ -101,32 +98,35 @@ case class BroadcastNestedLoopJoin( def execute() = { val broadcastedRelation = sc.broadcast(broadcast.execute().collect().toIndexedSeq) - val streamedPlusMatches = streamed.execute().map { streamedRow => - var i = 0 + val streamedPlusMatches = streamed.execute().mapPartitions { streamedIter => val matchedRows = new mutable.ArrayBuffer[Row] val includedBroadcastTuples = new mutable.BitSet(broadcastedRelation.value.size) - - while (i < broadcastedRelation.value.size) { - // TODO: One bitset per partition instead of per row. - val broadcastedRow = broadcastedRelation.value(i) - val includeRow = condition match { - case None => true - case Some(c) => Evaluate(c, Vector(streamedRow, broadcastedRow)).asInstanceOf[Boolean] + val joinedRow = new JoinedRow + val boundCondition = + condition + .map(c => BindReferences.bindReference(c, left.output ++ right.output)) + .getOrElse(Literal(true)) + + streamedIter.foreach { streamedRow => + var i = 0 + var matched = false + + while (i < broadcastedRelation.value.size) { + // TODO: One bitset per partition instead of per row. + val broadcastedRow = broadcastedRelation.value(i) + if (boundCondition.applyBoolean(joinedRow(streamedRow, broadcastedRow))) { + matchedRows += buildRow(streamedRow ++ broadcastedRow) + matched = true + includedBroadcastTuples += i + } + i += 1 } - if (includeRow) { - matchedRows += buildRow(streamedRow ++ broadcastedRow) - includedBroadcastTuples += i + + if (!matched && (joinType == LeftOuter || joinType == FullOuter)) { + matchedRows += buildRow(streamedRow ++ Array.fill(right.output.size)(null)) } - i += 1 - } - val outputRows = if (matchedRows.size > 0) { - matchedRows - } else if (joinType == LeftOuter || joinType == FullOuter) { - Vector(buildRow(streamedRow ++ Array.fill(right.output.size)(null))) - } else { - Vector() } - (outputRows, includedBroadcastTuples) + Iterator((matchedRows, includedBroadcastTuples)) } val includedBroadcastTuples = streamedPlusMatches.map(_._2) @@ -142,6 +142,7 @@ case class BroadcastNestedLoopJoin( broadcastedRelation.value.zipWithIndex.filter { case (row, i) => !allIncludedBroadcastTuples.contains(i) }.map { + // TODO: Use projection. case (row, _) => buildRow(Vector.fill(left.output.size)(null) ++ row) } } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala index 65cb515780f90..df1a95c69c5bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala @@ -43,13 +43,13 @@ case class ExampleTGF(input: Seq[Attribute] = Seq('name, 'age)) extends Generato val Seq(nameAttr, ageAttr) = input - def apply(input: Row): TraversableOnce[Row] = { - val name = Evaluate(nameAttr, Vector(input)) - val age = Evaluate(ageAttr, Vector(input)).asInstanceOf[Int] + override def apply(input: Row): TraversableOnce[Row] = { + val name = nameAttr.apply(input) + val age = ageAttr.applyInt(input) Iterator( - new GenericRow(Vector(s"$name is $age years old")), - new GenericRow(Vector(s"Next year, $name will be ${age + 1} years old"))) + new GenericRow(Array(s"$name is $age years old")), + new GenericRow(Array(s"Next year, $name will be ${age + 1} years old"))) } } 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 3e790ce940ba2..e269c8c846700 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 @@ -205,7 +205,7 @@ abstract class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { override val planner = HivePlanner - protected lazy val emptyResult = sparkContext.parallelize(Seq(new GenericRow(Vector()): Row), 1) + protected lazy val emptyResult = sparkContext.parallelize(Seq(new GenericRow(Array()): Row), 1) /** Extends QueryExecution with hive specific features. */ abstract class QueryExecution extends super.QueryExecution { @@ -222,7 +222,7 @@ abstract class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { if (output.size == 0) { emptyResult } else { - val asRows = output.map(r => new GenericRow(r.split("\t"))) + val asRows = output.map(r => new GenericRow(r.split("\t").asInstanceOf[Array[Any]])) sparkContext.parallelize(asRows, 1) } case _ => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala index 7c128246f72c3..f20e9d4de4cf2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala @@ -57,14 +57,15 @@ case class ScriptTransformation( var curLine = reader.readLine() while (curLine != null) { // TODO: Use SerDe - outputLines += new GenericRow(curLine.split("\t")) + outputLines += new GenericRow(curLine.split("\t").asInstanceOf[Array[Any]]) curLine = reader.readLine() } } } readerThread.start() + val outputProjection = new Projection(input) iter - .map(row => input.map(Evaluate(_, Vector(row)))) + .map(outputProjection) // TODO: Use SerDe .map(_.mkString("", "\t", "\n").getBytes).foreach(outputStream.write) outputStream.close() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala index 479759aa8184d..d1ae4c4111159 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -72,7 +72,7 @@ case class HiveTableScan( pred.dataType == BooleanType, s"Data type of predicate $pred must be BooleanType rather than ${pred.dataType}.") - BindReferences.bindReference(pred, Seq(relation.partitionKeys)) + BindReferences.bindReference(pred, relation.partitionKeys) } @transient @@ -113,7 +113,7 @@ case class HiveTableScan( } private def castFromString(value: String, dataType: DataType) = { - Evaluate(Cast(Literal(value), dataType), Nil) + Cast(Literal(value), dataType).apply(null) } @transient @@ -140,8 +140,8 @@ case class HiveTableScan( // Only partitioned values are needed here, since the predicate has already been bound to // partition key attribute references. - val row = new GenericRow(castedValues) - Evaluate(shouldKeep, Seq(row)).asInstanceOf[Boolean] + val row = new GenericRow(castedValues.toArray) + shouldKeep.applyBoolean(row) } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 6375744dfd06a..89227aa306908 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -129,10 +129,12 @@ trait HiveFunctionFactory { } abstract class HiveUdf - extends Expression with ImplementedUdf with Logging with HiveFunctionFactory { + extends Expression with Logging with HiveFunctionFactory { self: Product => type UDFType + type EvaluatedType = Any + val name: String def nullable = true @@ -186,7 +188,8 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } // TODO: Finish input output types. - def evaluate(evaluatedChildren: Seq[Any]): Any = { + override def apply(input: Row): Any = { + val evaluatedChildren = children.map(_.apply(input)) // Wrap the function arguments in the expected types. val args = evaluatedChildren.zip(wrappers).map { case (arg, wrapper) => wrapper(arg) @@ -211,10 +214,13 @@ case class HiveGenericUdf( val dataType: DataType = inspectorToDataType(returnInspector) - def evaluate(evaluatedChildren: Seq[Any]): Any = { + override def apply(input: Row): Any = { returnInspector // Make sure initialized. - val args = evaluatedChildren.map(wrap).map { v => - new DeferredJavaObject(v): DeferredObject + val args = children.map { v => + new DeferredObject { + override def prepare(i: Int) = {} + override def get(): AnyRef = wrap(v.apply(input)) + } }.toArray unwrap(function.evaluate(args)) } @@ -238,7 +244,7 @@ trait HiveInspectors { case si: StructObjectInspector => val allRefs = si.getAllStructFieldRefs new GenericRow( - allRefs.map(r => unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector))) + allRefs.map(r => unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) } /** Converts native catalyst types to the types expected by Hive */ @@ -394,12 +400,14 @@ case class HiveGenericUdtf( } } - def apply(input: Row): TraversableOnce[Row] = { + override def apply(input: Row): TraversableOnce[Row] = { outputInspectors // Make sure initialized. + + val inputProjection = new Projection(children) val collector = new UDTFCollector function.setCollector(collector) - val udtInput = children.map(Evaluate(_, Vector(input))).map(wrap).toArray + val udtInput = inputProjection(input).map(wrap).toArray function.process(udtInput) collector.collectRows() } @@ -446,10 +454,13 @@ case class HiveUdafFunction( private val buffer = function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] - def result: Any = unwrapData(function.evaluate(buffer), returnInspector) + override def apply(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) + + @transient + val inputProjection = new Projection(exprs) - def apply(input: Seq[Row]): Unit = { - val inputs = exprs.map(Evaluate(_, input).asInstanceOf[AnyRef]).toArray + def update(input: Row): Unit = { + val inputs = inputProjection(input).asInstanceOf[Seq[AnyRef]].toArray function.iterate(buffer, inputs) } } From 296fe5036105b7e519501f58e0fb0204023c23f2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 27 Feb 2014 12:30:56 -0800 Subject: [PATCH 735/778] Address review feedback. --- .../sql/catalyst/expressions/Expression.scala | 58 ++++++++++++------- .../sql/catalyst/expressions/aggregates.scala | 12 ++-- .../catalyst/expressions/complexTypes.scala | 2 +- .../spark/sql/execution/SharkPlan.scala | 3 +- .../spark/sql/execution/SparkStrategies.scala | 11 ++-- .../org/apache/spark/sql/hive/hiveUdfs.scala | 3 +- 6 files changed, 52 insertions(+), 37 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 68da30c47ce16..a07045fed79f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -53,6 +53,9 @@ abstract class Expression extends TreeNode[Expression] { def apply(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + // Primitive Accessor functions that avoid boxing for performance. + // Note this is an Unstable API as it doesn't correctly handle null values yet. + def applyBoolean(input: Row): Boolean = apply(input).asInstanceOf[Boolean] def applyInt(input: Row): Int = apply(input).asInstanceOf[Int] def applyDouble(input: Row): Double = apply(input).asInstanceOf[Double] @@ -103,15 +106,19 @@ abstract class Expression extends TreeNode[Expression] { } val evalE1 = e1.apply(i) - val evalE2 = e2.apply(i) - if (evalE1 == null || evalE2 == null) { + if(evalE1 == null) { null } else { - e1.dataType match { - case n: NumericType => - f.asInstanceOf[(Numeric[n.JvmType], n.JvmType, n.JvmType) => Int]( - n.numeric, evalE1.asInstanceOf[n.JvmType], evalE2.asInstanceOf[n.JvmType]) - case other => sys.error(s"Type $other does not support numeric operations") + val evalE2 = e2.apply(i) + if (evalE2 == null) { + null + } else { + e1.dataType match { + case n: NumericType => + f.asInstanceOf[(Numeric[n.JvmType], n.JvmType, n.JvmType) => Int]( + n.numeric, evalE1.asInstanceOf[n.JvmType], evalE2.asInstanceOf[n.JvmType]) + case other => sys.error(s"Type $other does not support numeric operations") + } } } } @@ -127,15 +134,19 @@ abstract class Expression extends TreeNode[Expression] { } val evalE1 = e1.apply(i: Row) - val evalE2 = e2.apply(i: Row) - if (evalE1 == null || evalE2 == null) { + if(evalE1 == null) { null } else { - e1.dataType match { - case ft: FractionalType => - f.asInstanceOf[(Fractional[ft.JvmType], ft.JvmType, ft.JvmType) => ft.JvmType]( - ft.fractional, evalE1.asInstanceOf[ft.JvmType], evalE2.asInstanceOf[ft.JvmType]) - case other => sys.error(s"Type $other does not support fractional operations") + val evalE2 = e2.apply(i: Row) + if (evalE2 == null) { + null + } else { + e1.dataType match { + case ft: FractionalType => + f.asInstanceOf[(Fractional[ft.JvmType], ft.JvmType, ft.JvmType) => ft.JvmType]( + ft.fractional, evalE1.asInstanceOf[ft.JvmType], evalE2.asInstanceOf[ft.JvmType]) + case other => sys.error(s"Type $other does not support fractional operations") + } } } } @@ -149,16 +160,21 @@ abstract class Expression extends TreeNode[Expression] { if (e1.dataType != e2.dataType) { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } + val evalE1 = e1.apply(i) - val evalE2 = e2.apply(i) - if (evalE1 == null || evalE2 == null) { + if(evalE1 == null) { null } else { - e1.dataType match { - case i: IntegralType => - f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( - i.integral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) - case other => sys.error(s"Type $other does not support numeric operations") + val evalE2 = e2.apply(i) + if (evalE2 == null) { + null + } else { + e1.dataType match { + case i: IntegralType => + f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( + i.integral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) + case other => sys.error(s"Type $other does not support numeric operations") + } } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index aec0ca1172171..010064489c421 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -169,11 +169,11 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) def this() = this(null, null) // Required for serialization. - var count: Long = _ - val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) - val sumAsDouble = Cast(sum, DoubleType) + private var count: Long = _ + private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) + private val sumAsDouble = Cast(sum, DoubleType) - val addFunction = Add(sum, expr) + private val addFunction = Add(sum, expr) override def apply(input: Row): Any = sumAsDouble.applyDouble(null) / count.toDouble @@ -201,9 +201,9 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) + private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) - val addFunction = Add(sum, expr) + private val addFunction = Add(sum, expr) def update(input: Row): Unit = { sum.update(addFunction, input) 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 2962c623d4d9a..d3feb6c461b55 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 @@ -47,7 +47,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { val o = ordinal.apply(input).asInstanceOf[Int] if (baseValue == null) { null - } else if (o >= baseValue.size) { + } else if (o >= baseValue.size || o < 0) { null } else { baseValue(o) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala index 5b57408d32ffe..853238eae0baa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala @@ -44,7 +44,8 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { */ def executeCollect(): Array[Row] = execute().collect() - protected def buildRow(values: Seq[Any]): Row = new catalyst.expressions.GenericRow(values.toArray) + protected def buildRow(values: Seq[Any]): Row = + new catalyst.expressions.GenericRow(values.toArray) } trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index b3a4f8dd3c6ab..9a2975686110c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -162,9 +162,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object TopK extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.StopAfter(limit, logical.Sort(order, child)) => - execution.TopK( - limit.applyInt(null), order, planLater(child))(sparkContext) :: Nil + case logical.StopAfter(IntegerLiteral(limit), logical.Sort(order, child)) => + execution.TopK(limit, order, planLater(child))(sparkContext) :: Nil case _ => Nil } } @@ -197,10 +196,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { sparkContext.parallelize(data.map(r => new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row)) execution.ExistingRdd(output, dataAsRdd) :: Nil - case logical.StopAfter(limit, child) => - execution.StopAfter( - limit.applyInt(null), - planLater(child))(sparkContext) :: Nil + case logical.StopAfter(IntegerLiteral(limit), child) => + execution.StopAfter(limit, planLater(child))(sparkContext) :: Nil case Unions(unionChildren) => execution.Union(unionChildren.map(planLater))(sparkContext) :: Nil case logical.Generate(generator, join, outer, _, child) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 89227aa306908..46bd64726fd58 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -244,7 +244,8 @@ trait HiveInspectors { case si: StructObjectInspector => val allRefs = si.getAllStructFieldRefs new GenericRow( - allRefs.map(r => unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) + allRefs.map(r => + unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) } /** Converts native catalyst types to the types expected by Hive */ From 6fdefe65478d950d3f30f6591df361558886d187 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Mar 2014 12:33:45 -0800 Subject: [PATCH 736/778] Port sbt improvements from master. --- project/SparkBuild.scala | 3 + sbt/sbt | 162 ++++++++++++++++++++++------------- sbt/sbt-launch-lib.bash | 178 +++++++++++++++++++++++++++++++++++++++ 3 files changed, 283 insertions(+), 60 deletions(-) create mode 100755 sbt/sbt-launch-lib.bash diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0c27386621fa7..07bf7861c4008 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -23,6 +23,8 @@ import AssemblyKeys._ import scala.util.Properties import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} +import scala.collection.JavaConversions._ + // For Sonatype publishing //import com.jsuereth.pgp.sbtplugin.PgpKeys._ @@ -144,6 +146,7 @@ object SparkBuild extends Build { fork := true, javaOptions in Test += "-Dspark.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", + javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark").map { case (k,v) => s"-D$k=$v" }.toSeq, javaOptions += "-Xmx3g", // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), diff --git a/sbt/sbt b/sbt/sbt index efa575ca64f5a..3ffa4ed9ab5a7 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,60 +1,102 @@ -#!/bin/bash - -# -# 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. -# - -# Required to correctly launch hadoop/hive for generating the golden answers. -HADOOP_CLASSPATH="" - -for i in $HIVE_HOME/lib/* -do HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$i -done - -export HADOOP_CLASSPATH - -# This script launches sbt for this project. If present it uses the system -# version of sbt. If there is no system version of sbt it attempts to download -# sbt locally. -SBT_VERSION=`awk -F "=" '/sbt\\.version/ {print $2}' ./project/build.properties` -URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar -URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar -JAR=sbt/sbt-launch-${SBT_VERSION}.jar - -# Download sbt launch jar if it hasn't been downloaded yet -if [ ! -f ${JAR} ]; then - # Download - printf "Attempting to fetch sbt\n" - JAR_DL=${JAR}.part - if hash curl 2>/dev/null; then - (curl --progress-bar ${URL1} > ${JAR_DL} || curl --progress-bar ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} - elif hash wget 2>/dev/null; then - (wget --progress=bar ${URL1} -O ${JAR_DL} || wget --progress=bar ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} - else - printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" - exit -1 - fi -fi -if [ ! -f ${JAR} ]; then - # We failed to download - printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" - exit -1 -fi -printf "Launching sbt from ${JAR}\n" -java \ - -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \ - -jar ${JAR} \ - "$@" +#!/usr/bin/env bash + +realpath () { +( + TARGET_FILE=$1 + + cd $(dirname $TARGET_FILE) + TARGET_FILE=$(basename $TARGET_FILE) + + COUNT=0 + while [ -L "$TARGET_FILE" -a $COUNT -lt 100 ] + do + TARGET_FILE=$(readlink $TARGET_FILE) + cd $(dirname $TARGET_FILE) + TARGET_FILE=$(basename $TARGET_FILE) + COUNT=$(($COUNT + 1)) + done + + echo $(pwd -P)/$TARGET_FILE +) +} + +. $(dirname $(realpath $0))/sbt-launch-lib.bash + + +declare -r noshare_opts="-Dsbt.global.base=project/.sbtboot -Dsbt.boot.directory=project/.boot -Dsbt.ivy.home=project/.ivy" +declare -r sbt_opts_file=".sbtopts" +declare -r etc_sbt_opts_file="/etc/sbt/sbtopts" + +usage() { + cat < path to global settings/plugins directory (default: ~/.sbt) + -sbt-boot path to shared boot directory (default: ~/.sbt/boot in 0.11 series) + -ivy path to local Ivy repository (default: ~/.ivy2) + -mem set memory options (default: $sbt_mem, which is $(get_mem_opts $sbt_mem)) + -no-share use all local caches; no sharing + -no-global uses global caches, but does not use global ~/.sbt directory. + -jvm-debug Turn on JVM debugging, open at the given port. + -batch Disable interactive mode + + # sbt version (default: from project/build.properties if present, else latest release) + -sbt-version use the specified version of sbt + -sbt-jar use the specified jar as the sbt launcher + -sbt-rc use an RC version of sbt + -sbt-snapshot use a snapshot version of sbt + + # java version (default: java from PATH, currently $(java -version 2>&1 | grep version)) + -java-home alternate JAVA_HOME + + # jvm options and output control + JAVA_OPTS environment variable, if unset uses "$java_opts" + SBT_OPTS environment variable, if unset uses "$default_sbt_opts" + .sbtopts if this file exists in the current directory, it is + prepended to the runner args + /etc/sbt/sbtopts if this file exists, it is prepended to the runner args + -Dkey=val pass -Dkey=val directly to the java runtime + -J-X pass option -X directly to the java runtime + (-J is stripped) + -S-X add -X to sbt's scalacOptions (-J is stripped) + +In the case of duplicated or conflicting options, the order above +shows precedence: JAVA_OPTS lowest, command line options highest. +EOM +} + +process_my_args () { + while [[ $# -gt 0 ]]; do + case "$1" in + -no-colors) addJava "-Dsbt.log.noformat=true" && shift ;; + -no-share) addJava "$noshare_opts" && shift ;; + -no-global) addJava "-Dsbt.global.base=$(pwd)/project/.sbtboot" && shift ;; + -sbt-boot) require_arg path "$1" "$2" && addJava "-Dsbt.boot.directory=$2" && shift 2 ;; + -sbt-dir) require_arg path "$1" "$2" && addJava "-Dsbt.global.base=$2" && shift 2 ;; + -debug-inc) addJava "-Dxsbt.inc.debug=true" && shift ;; + -batch) exec &2 "$@" +} +vlog () { + [[ $verbose || $debug ]] && echoerr "$@" +} +dlog () { + [[ $debug ]] && echoerr "$@" +} + +acquire_sbt_jar () { + SBT_VERSION=`awk -F "=" '/sbt\\.version/ {print $2}' ./project/build.properties` + URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar + URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar + JAR=sbt/sbt-launch-${SBT_VERSION}.jar + + sbt_jar=$JAR + + if [[ ! -f "$sbt_jar" ]]; then + # Download sbt launch jar if it hasn't been downloaded yet + if [ ! -f ${JAR} ]; then + # Download + printf "Attempting to fetch sbt\n" + JAR_DL=${JAR}.part + if hash curl 2>/dev/null; then + (curl --progress-bar ${URL1} > ${JAR_DL} || curl --progress-bar ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} + elif hash wget 2>/dev/null; then + (wget --progress=bar ${URL1} -O ${JAR_DL} || wget --progress=bar ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} + else + printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" + exit -1 + fi + fi + if [ ! -f ${JAR} ]; then + # We failed to download + printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" + exit -1 + fi + printf "Launching sbt from ${JAR}\n" + fi +} + +execRunner () { + # print the arguments one to a line, quoting any containing spaces + [[ $verbose || $debug ]] && echo "# Executing command line:" && { + for arg; do + if printf "%s\n" "$arg" | grep -q ' '; then + printf "\"%s\"\n" "$arg" + else + printf "%s\n" "$arg" + fi + done + echo "" + } + + exec "$@" +} + +addJava () { + dlog "[addJava] arg = '$1'" + java_args=( "${java_args[@]}" "$1" ) +} +addSbt () { + dlog "[addSbt] arg = '$1'" + sbt_commands=( "${sbt_commands[@]}" "$1" ) +} +addResidual () { + dlog "[residual] arg = '$1'" + residual_args=( "${residual_args[@]}" "$1" ) +} +addDebugger () { + addJava "-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=$1" +} + +# a ham-fisted attempt to move some memory settings in concert +# so they need not be dicked around with individually. +get_mem_opts () { + local mem=${1:-2048} + local perm=$(( $mem / 4 )) + (( $perm > 256 )) || perm=256 + (( $perm < 1024 )) || perm=1024 + local codecache=$(( $perm / 2 )) + + echo "-Xms${mem}m -Xmx${mem}m -XX:MaxPermSize=${perm}m -XX:ReservedCodeCacheSize=${codecache}m" +} + +require_arg () { + local type="$1" + local opt="$2" + local arg="$3" + if [[ -z "$arg" ]] || [[ "${arg:0:1}" == "-" ]]; then + die "$opt requires <$type> argument" + fi +} + +is_function_defined() { + declare -f "$1" > /dev/null +} + +process_args () { + while [[ $# -gt 0 ]]; do + case "$1" in + -h|-help) usage; exit 1 ;; + -v|-verbose) verbose=1 && shift ;; + -d|-debug) debug=1 && shift ;; + + -ivy) require_arg path "$1" "$2" && addJava "-Dsbt.ivy.home=$2" && shift 2 ;; + -mem) require_arg integer "$1" "$2" && sbt_mem="$2" && shift 2 ;; + -jvm-debug) require_arg port "$1" "$2" && addDebugger $2 && shift 2 ;; + -batch) exec Date: Mon, 3 Mar 2014 12:43:32 -0800 Subject: [PATCH 737/778] Add byte wrappers for hive UDFS. --- .../src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 46bd64726fd58..c04bbe8d41785 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -162,7 +162,7 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd val primitiveClasses = Seq( Integer.TYPE, classOf[java.lang.Integer], classOf[java.lang.String], java.lang.Double.TYPE, classOf[java.lang.Double], java.lang.Long.TYPE, classOf[java.lang.Long], - classOf[HiveDecimal] + classOf[HiveDecimal], java.lang.Byte.TYPE, classOf[java.lang.Byte] ) val matchingConstructor = argClass.getConstructors.find { c => c.getParameterTypes.size == 1 && primitiveClasses.contains(c.getParameterTypes.head) From 7b9d14263a4cbf5d39216c86a41b546c607b4a20 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Mar 2014 11:41:35 -0800 Subject: [PATCH 738/778] Update travis to increase permgen size. --- project/SparkBuild.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index fed09340dfc67..b492f548ea502 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -364,6 +364,7 @@ object SparkBuild extends Build { def hiveSettings = sharedSettings ++ Seq( name := "spark-hive", + javaOptions += "-XX:MaxPermSize=1g", libraryDependencies ++= Seq( "org.apache.hive" % "hive-metastore" % "0.12.0", "org.apache.hive" % "hive-exec" % "0.12.0", From 8d5da5ed977b1c867b5b78f05523d89d5552b387 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 26 Feb 2014 19:02:17 -0800 Subject: [PATCH 739/778] modify compute-classpath.sh to include datanucleus jars explicitly --- bin/compute-classpath.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 278969655de48..0071a0d2175af 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -33,6 +33,10 @@ fi # Build up classpath CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" +# Datanucleus jars do not work if included in the uberjar as plugin.xml metadata is lost. +DATANUCLEUSJARS=$(JARS=("$FWDIR/lib_managed/jars"/datanucleus-*.jar); IFS=:; echo "${JARS[*]}") +CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS + # First check if we have a dependencies jar. If so, include binary classes with the deps jar if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" From 6d315bb168443eba98d978ae65c386ff27629bfc Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 5 Mar 2014 11:48:37 +0800 Subject: [PATCH 740/778] Added Row.unapplySeq to extract fields from a Row object. --- .../main/scala/org/apache/spark/sql/package.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala index 517e792479bc4..0e9af6bc29570 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala @@ -31,4 +31,19 @@ package object sql { protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging type Row = catalyst.expressions.Row + + object Row { + /** + * This method can be used to extract fields from a [[Row]] object in a patten match. Example: + * {{{ + * import org.apache.spark.sql._ + * + * val pairs = sql("SELECT key, value FROM src").rdd.map { + * case Row(key: Int, value: String) => + * key -> value + * } + * }}} + */ + def unapplySeq(row: Row): Some[Seq[Any]] = Some(row) + } } From 70e489d277470b5ed84d856af96b1167a0f892b6 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 5 Mar 2014 12:13:19 +0800 Subject: [PATCH 741/778] Fixed a spelling typo --- sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala index 0e9af6bc29570..9ec31689b5098 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala @@ -34,7 +34,7 @@ package object sql { object Row { /** - * This method can be used to extract fields from a [[Row]] object in a patten match. Example: + * This method can be used to extract fields from a [[Row]] object in a pattern match. Example: * {{{ * import org.apache.spark.sql._ * From 0040ae6d53e4298402b1ddcbcbcea6bc2b78e7d7 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 5 Mar 2014 10:11:54 +0100 Subject: [PATCH 742/778] Feedback from code review --- project/SparkBuild.scala | 16 +- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../plans/logical/basicOperators.scala | 3 +- .../spark/sql/execution/SparkStrategies.scala | 8 +- .../ParquetRelation.scala | 162 ++++++++---------- .../ParquetTableOperations.scala | 40 ++--- .../ParquetTableSupport.scala | 71 ++++---- .../spark/sql/parquet/ParquetTestData.scala | 85 +++++++++ sql/core/src/test/resources/log4j.properties | 8 +- .../ParquetQueryTests.scala | 16 +- .../spark/sql/hive/HiveStrategies.scala | 3 + .../ParquetQuerySuite.scala | 19 +- 12 files changed, 253 insertions(+), 180 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/{execution => parquet}/ParquetRelation.scala (71%) rename sql/core/src/main/scala/org/apache/spark/sql/{execution => parquet}/ParquetTableOperations.scala (81%) rename sql/core/src/main/scala/org/apache/spark/sql/{execution => parquet}/ParquetTableSupport.scala (73%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala rename sql/core/src/test/scala/org/apache/spark/sql/{execution => parquet}/ParquetQueryTests.scala (84%) rename sql/hive/src/test/scala/org/apache/spark/sql/{hive/execution => parquet}/ParquetQuerySuite.scala (93%) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b492f548ea502..715d2109dea96 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -351,15 +351,17 @@ object SparkBuild extends Build { // this non-deterministically. TODO: FIX THIS. parallelExecution in Test := false, libraryDependencies ++= Seq( - "org.scalatest" %% "scalatest" % "1.9.1" % "test", - "com.typesafe" %% "scalalogging-slf4j" % "1.0.1") + "org.scalatest" %% "scalatest" % "1.9.1" % "test", + "com.typesafe" %% "scalalogging-slf4j" % "1.0.1" + ) ) def sqlCoreSettings = sharedSettings ++ Seq( name := "spark-sql", libraryDependencies ++= Seq( - "com.twitter" % "parquet-column" % "1.3.2", - "com.twitter" % "parquet-hadoop" % "1.3.2") + "com.twitter" % "parquet-column" % "1.3.2", + "com.twitter" % "parquet-hadoop" % "1.3.2" + ) ) def hiveSettings = sharedSettings ++ Seq( @@ -368,7 +370,8 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "org.apache.hive" % "hive-metastore" % "0.12.0", "org.apache.hive" % "hive-exec" % "0.12.0", - "org.apache.hive" % "hive-serde" % "0.12.0"), + "org.apache.hive" % "hive-serde" % "0.12.0" + ), // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings @@ -388,7 +391,8 @@ object SparkBuild extends Build { |import org.apache.spark.sql.catalyst.util._ |import org.apache.spark.sql.execution |import org.apache.spark.sql.hive._ - |import org.apache.spark.sql.hive.TestHive._""".stripMargin + |import org.apache.spark.sql.hive.TestHive._ + |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin ) def streamingSettings = sharedSettings ++ Seq( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 2c9716f01ec4d..0f02b050bd499 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -207,7 +207,7 @@ package object dsl { def analyze = analysis.SimpleAnalyzer(plan) - def writeToFile(path: String) = WriteToFile(path, plan, None) + def writeToFile(path: String) = WriteToFile(path, plan) // TODO: for a loadFromFile it would be good to have a Catalog that knows // how to resolve ParquetTables diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 69d4f16d6e9ca..8e98aab736432 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -111,8 +111,7 @@ case class InsertIntoCreatedTable( case class WriteToFile( path: String, - child: LogicalPlan, - tableName: Option[String]) extends UnaryNode { + child: LogicalPlan) extends UnaryNode { def references = Set.empty def output = child.output } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index bc24735d6ace0..57f1ebd66f40c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -25,6 +25,8 @@ import catalyst.planning._ import catalyst.plans._ import catalyst.plans.logical.LogicalPlan import catalyst.plans.physical._ +import parquet.ParquetRelation +import parquet.InsertIntoParquetTable abstract class SparkStrategies extends QueryPlanner[SparkPlan] { @@ -209,10 +211,10 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.ExistingRdd(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil - case logical.WriteToFile(path, child, tableName) => + case logical.WriteToFile(path, child) => val relation = - ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, tableName) - execution.InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil + ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, None) + InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala similarity index 71% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala rename to sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 12a7d01ce3afd..945e9d654b446 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.execution +package org.apache.spark.sql.parquet -import java.io.{IOException, FileNotFoundException, File} +import java.io.{IOException, FileNotFoundException} import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.conf.Configuration @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.permission.FsAction import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, BaseRelation} import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.ArrayType -import org.apache.spark.sql.catalyst.expressions.{Row, GenericRow, AttributeReference, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Row, AttributeReference, Attribute} import org.apache.spark.sql.catalyst.analysis.UnresolvedException import parquet.schema.{MessageTypeParser, MessageType} @@ -36,7 +36,7 @@ import parquet.schema.{PrimitiveType => ParquetPrimitiveType} import parquet.schema.{Type => ParquetType} import parquet.schema.Type.Repetition import parquet.io.api.{Binary, RecordConsumer} -import parquet.hadoop.{Footer, ParquetFileWriter, ParquetWriter, ParquetFileReader} +import parquet.hadoop.{Footer, ParquetFileWriter, ParquetFileReader} import parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} import parquet.hadoop.util.ContextUtil @@ -45,30 +45,57 @@ import scala.collection.JavaConversions._ /** * Relation formed by underlying Parquet file that contains data stored in columnar form. * Note that there are currently two ways to import a ParquetRelation: - * a) create the Relation "manually" and register via the OverrideCatalog, e.g.: - * TestShark.catalog.overrideTable( - * Some[String]("parquet"), - * "testsource", - * ParquetTestData.testData) - * and then execute the query as usual. - * b) store a relation via WriteToFile and manually resolve the corresponding - * ParquetRelation: - * val query = TestShark.parseSql(querystr).transform { - * case relation @ UnresolvedRelation(databaseName, name, alias) => - * if(name.equals(tableName)) - * ParquetRelation(tableName, filename) - * else - * relation - * } - * TestShark.executePlan(query) - * .toRdd - * .collect() * - * @param tableName The name of the relation. + * {{{ + * // a) create the Relation "manually" and register via the OverrideCatalog, e.g.: + * scala> ParquetTestData.writeFile + * + * scala> TestHive.catalog.overrideTable(None, "psrc", ParquetTestData.testData) + * res1: Option[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan] = None + * + * scala> val query = sql("SELECT * FROM psrc") + * query: org.apache.spark.sql.ExecutedQuery = + * SELECT * FROM psrc + * === Query Plan === + * ParquetTableScan [myboolean#6,myint#7,mystring#8,mylong#9L,myfloat#10,mydouble#11], + * (ParquetRelation testData, file:/tmp/testParquetFile), None + * + * scala> query.collect + * res2: Array[org.apache.spark.sql.Row] = Array([true,5,abc,8589934592,2.5,4.5], ... + * + * // b) "manually" resolve the relation by modifying the logical plan + * scala> import org.apache.spark.sql.execution.ParquetRelation + * import org.apache.spark.sql.execution.ParquetRelation + * + * scala> ParquetTestData.writeFile + * + * scala> val filename = ParquetTestData.testFile.toString + * filename: String = /tmp/testParquetFile + * + * scala> val query_string = "SELECT * FROM psrc" + * query_string: String = SELECT * FROM psrc + * + * scala> val query = TestHive.parseSql(query_string).transform { + * | case relation @ UnresolvedRelation(databaseName, name, alias) => + * | if(name == "psrc") ParquetRelation(name, filename) + * | else relation + * | } + * query: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan = + * Project [*] + * ParquetRelation psrc, /tmp/testParquetFile + * + * scala> executePlan(query).toRdd.collect() + * res8: Array[org.apache.spark.sql.Row] = Array([true,5,abc,8589934592,2.5,4.5], ... + * }}} + * + * @param tableName The name of the relation that can be used in queries. * @param path The path to the Parquet file. */ case class ParquetRelation(val tableName: String, val path: String) extends BaseRelation { + // TODO: Figure out resolution of unresolved ParquetRelation (a simple MetaStore?) + // to make this more user friendly + /** Schema derived from ParquetFile **/ def parquetSchema: MessageType = ParquetTypesConverter @@ -91,6 +118,22 @@ case class ParquetRelation(val tableName: String, val path: String) extends Base object ParquetRelation { + // The element type for the RDDs that this relation maps to. + type RowType = org.apache.spark.sql.catalyst.expressions.GenericRow + + /** + * Creates a new ParquetRelation and underlying Parquetfile for the given + * LogicalPlan. Note that this is used insider [[SparkStrategies]] to + * create a resolved relation as a data sink for writing to a Parquetfile. + * The relation is empty but is initialized with ParquetMetadata and + * can be inserted into. + * + * @param pathString The directory the Parquetfile will be stored in. + * @param child The child node that will be used for extracting the schema. + * @param conf A configuration configuration to be used. + * @param tableName The name of the resulting relation. + * @return An empty ParquetRelation inferred metadata. + */ def create(pathString: String, child: LogicalPlan, conf: Configuration, @@ -111,10 +154,10 @@ object ParquetRelation { val path = new Path(pathStr) val fs = path.getFileSystem(conf) if(fs.exists(path) && - !fs.getFileStatus(path) - .getPermission - .getUserAction - .implies(FsAction.READ_WRITE)) { + !fs.getFileStatus(path) + .getPermission + .getUserAction + .implies(FsAction.READ_WRITE)) { throw new IOException( s"Unable to create ParquetRelation: path ${path.toString} not read-writable") } @@ -134,7 +177,8 @@ object ParquetTypesConverter { case ParquetPrimitiveTypeName.INT32 => IntegerType case ParquetPrimitiveTypeName.INT64 => LongType case ParquetPrimitiveTypeName.INT96 => LongType // TODO: is there an equivalent? - case _ => sys.error(s"Unsupported parquet datatype") + case _ => sys.error( + s"Unsupported parquet datatype ${parquetType.asInstanceOf[Enum[String]].toString()}") } def fromDataType(ctype: DataType): ParquetPrimitiveTypeName = ctype match { @@ -145,7 +189,7 @@ object ParquetTypesConverter { case FloatType => ParquetPrimitiveTypeName.FLOAT case IntegerType => ParquetPrimitiveTypeName.INT32 case LongType => ParquetPrimitiveTypeName.INT64 - case _ => sys.error(s"Unsupported datatype") + case _ => sys.error(s"Unsupported datatype ${ctype.toString}") } def consumeType(consumer: RecordConsumer, ctype: DataType, record: Row, index: Int): Unit = { @@ -160,7 +204,7 @@ object ParquetTypesConverter { case DoubleType => consumer.addDouble(record.getDouble(index)) case FloatType => consumer.addFloat(record.getFloat(index)) case BooleanType => consumer.addBoolean(record.getBoolean(index)) - case _ => sys.error(s"Unsupported datatype, cannot write to consumer") + case _ => sys.error(s"Unsupported datatype ${ctype.toString}, cannot write to consumer") } } @@ -230,7 +274,7 @@ object ParquetTypesConverter { * in the parent directory. If so, this is used. Else we read the actual footer at the given * location. * @param path The path at which we expect one (or more) Parquet files. - * @return The [[ParquetMetadata]] containing among other things the schema. + * @return The `ParquetMetadata` containing among other things the schema. */ def readMetaData(path: Path): ParquetMetadata = { val job = new Job() @@ -251,59 +295,3 @@ object ParquetTypesConverter { } } } - -object ParquetTestData { - - val testSchema = - """message myrecord { - |optional boolean myboolean; - |optional int32 myint; - |optional binary mystring; - |optional int64 mylong; - |optional float myfloat; - |optional double mydouble; - |}""".stripMargin - - val subTestSchema = - """ - |message myrecord { - |optional boolean myboolean; - |optional int64 mylong; - |} - """.stripMargin - - val testFile = new File("/tmp/testParquetFile").getAbsoluteFile - - lazy val testData = new ParquetRelation("testData", testFile.toURI.toString) - - def writeFile = { - testFile.delete - val path: Path = new Path(testFile.toURI) - val job = new Job() - val configuration: Configuration = ContextUtil.getConfiguration(job) - val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) - - val writeSupport = new RowWriteSupport() - writeSupport.setSchema(schema, configuration) - val writer = new ParquetWriter(path, writeSupport) - for(i <- 0 until 15) { - val data = new Array[Any](6) - if(i % 3 ==0) { - data.update(0, true) - } else { - data.update(0, false) - } - if(i % 5 == 0) { - data.update(1, 5) - } else { - data.update(1, null) // optional - } - data.update(2, "abc") - data.update(3, 1L<<33) - data.update(4, 2.5F) - data.update(5, 4.5D) - writer.write(new GenericRow(data.toSeq)) - } - writer.close() - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala similarity index 81% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala rename to sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 09c48656f3c66..7582097896882 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -16,17 +16,18 @@ */ package org.apache.spark.sql -package execution +package parquet -import parquet.io.InvalidRecordException -import parquet.schema.MessageType -import parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat} -import parquet.hadoop.util.ContextUtil +import _root_.parquet.io.InvalidRecordException +import _root_.parquet.schema.MessageType +import _root_.parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat} +import _root_.parquet.hadoop.util.ContextUtil import org.apache.spark.rdd.RDD import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.expressions.{Row, Attribute, Expression} +import org.apache.spark.sql.execution.{SparkPlan, UnaryNode, LeafNode} import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.conf.Configuration @@ -36,27 +37,24 @@ import java.io.IOException /** * Parquet table scan operator. Imports the file that backs the given - * [[org.apache.spark.sql.execution.ParquetRelation]] as a RDD[Row]. + * [[ParquetRelation]] as a RDD[Row]. */ case class ParquetTableScan( - attributes: Seq[Attribute], + output: Seq[Attribute], relation: ParquetRelation, columnPruningPred: Option[Expression])( @transient val sc: SparkContext) extends LeafNode { - /** - * Runs this query returning the result as an RDD. - */ override def execute(): RDD[Row] = { val job = new Job(sc.hadoopConfiguration) ParquetInputFormat.setReadSupportClass( job, - classOf[org.apache.spark.sql.execution.RowReadSupport]) + classOf[org.apache.spark.sql.parquet.RowReadSupport]) val conf: Configuration = ContextUtil.getConfiguration(job) conf.set( RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, - ParquetTypesConverter.convertFromAttributes(attributes).toString) + ParquetTypesConverter.convertFromAttributes(output).toString) // TODO: think about adding record filters sc.newAPIHadoopFile( relation.path, @@ -77,7 +75,8 @@ case class ParquetTableScan( if(success) { ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sc) } else { - this // TODO: add warning to log that column projection was unsuccessful? + sys.error("Warning: Could not validate Parquet schema projection in pruneColumns") + this } } @@ -91,18 +90,15 @@ case class ParquetTableScan( private def validateProjection(projection: Seq[Attribute]): Boolean = { val original: MessageType = relation.parquetSchema val candidate: MessageType = ParquetTypesConverter.convertFromAttributes(projection) - var retval = true try { original.checkContains(candidate) + true } catch { case e: InvalidRecordException => { - retval = false + false } } - retval } - - override def output: Seq[Attribute] = attributes } case class InsertIntoParquetTable( @@ -129,7 +125,7 @@ case class InsertIntoParquetTable( ParquetOutputFormat.setWriteSupportClass( job, - classOf[org.apache.spark.sql.execution.RowWriteSupport]) + classOf[org.apache.spark.sql.parquet.RowWriteSupport]) // TODO: move that to function in object val conf = job.getConfiguration @@ -150,10 +146,8 @@ case class InsertIntoParquetTable( JavaPairRDD.fromRDD(childRdd.map(Tuple2(null, _))).saveAsNewAPIHadoopFile( relation.path.toString, classOf[Void], - classOf[org.apache.spark.sql.catalyst.expressions.GenericRow], - // scalastyle:off line.size.limit - classOf[parquet.hadoop.ParquetOutputFormat[org.apache.spark.sql.catalyst.expressions.GenericRow]], - // scalastyle:on line.size.limit + classOf[ParquetRelation.RowType], + classOf[_root_.parquet.hadoop.ParquetOutputFormat[ParquetRelation.RowType]], conf) // We return the child RDD to allow chaining (alternatively, one could return nothing). diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala similarity index 73% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 1ffc8f814b29e..6fe5e4462447d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution +package org.apache.spark.sql.parquet import org.apache.hadoop.conf.Configuration @@ -28,13 +28,13 @@ import parquet.hadoop.api.ReadSupport.ReadContext import parquet.hadoop.ParquetOutputFormat import parquet.column.ParquetProperties -import org.apache.spark.sql.catalyst.expressions.{Row, GenericRow, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Row, Attribute} import org.apache.spark.sql.catalyst.types._ /** - * A [[parquet.io.api.RecordMaterializer]] for Rows. + * A `parquet.io.api.RecordMaterializer` for Rows. * - * @param root The root group converter for the record. + *@param root The root group converter for the record. */ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterializer[Row] { @@ -49,23 +49,23 @@ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterial } /** - * A [[parquet.hadoop.api.ReadSupport]] for Row objects. + * A `parquet.hadoop.api.ReadSupport` for Row objects. */ class RowReadSupport extends ReadSupport[Row] with Logging { override def prepareForRead( - conf: Configuration, - stringMap: java.util.Map[String, String], - fileSchema: MessageType, - readContext: ReadContext): RecordMaterializer[Row] = { + conf: Configuration, + stringMap: java.util.Map[String, String], + fileSchema: MessageType, + readContext: ReadContext): RecordMaterializer[Row] = { log.debug(s"preparing for read with schema ${fileSchema.toString}") new RowRecordMaterializer(readContext.getRequestedSchema) } override def init( - configuration: Configuration, - keyValueMetaData: java.util.Map[String, String], - fileSchema: MessageType): ReadContext = { + configuration: Configuration, + keyValueMetaData: java.util.Map[String, String], + fileSchema: MessageType): ReadContext = { val requested_schema_string = configuration.get(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, fileSchema.toString) val requested_schema = @@ -81,7 +81,7 @@ object RowReadSupport { } /** - * A [[parquet.hadoop.api.WriteSupport]] for Row ojects. + * A `parquet.hadoop.api.WriteSupport` for Row ojects. */ class RowWriteSupport extends WriteSupport[Row] with Logging { def setSchema(schema: MessageType, configuration: Configuration) { @@ -124,14 +124,14 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { // TODO: add groups (nested fields) override def write(record: Row): Unit = { writer.startMessage() - attributes.zipWithIndex.foreach { - case (attribute, index) => { + // TODO: compare performance of the various ways of looping over a row + for(pair <- attributes.zipWithIndex) { + val (attribute, index) = pair // null values indicate optional fields but we do not check currently - if(record(index) != null && record(index) != Nil) { - writer.startField(attribute.name, index) - ParquetTypesConverter.consumeType(writer, attribute.dataType, record, index) - writer.endField(attribute.name, index) - } + if(record(index) != null && record(index) != Nil) { + writer.startField(attribute.name, index) + ParquetTypesConverter.consumeType(writer, attribute.dataType, record, index) + writer.endField(attribute.name, index) } } writer.endMessage() @@ -143,13 +143,14 @@ object RowWriteSupport { } /** - * A [[parquet.io.api.GroupConverter]] that is able to convert a Parquet record + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. * * @param schema The corresponding Shark schema in the form of a list of attributes. */ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { - var current: GenericRow = new GenericRow(Seq()) + type RowType = ParquetRelation.RowType + var current: RowType = new RowType(Seq()) // initialization may not strictly be required val currentData: Array[Any] = new Array[Any](schema.length) @@ -159,13 +160,14 @@ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { // note: for some reason matching for StringType fails so use this ugly if instead if (ctype == StringType) new CatalystPrimitiveStringConverter(this, schema.indexOf(a)) else new CatalystPrimitiveConverter(this, schema.indexOf(a)) - case _ => throw new RuntimeException("unable to convert datatype in CatalystGroupConverter") + case _ => throw new RuntimeException( + s"unable to convert datatype ${a.dataType.toString} in CatalystGroupConverter") } }.toArray override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) - def getCurrentRecord: GenericRow = current + def getCurrentRecord: RowType = current override def start(): Unit = { for (i <- 0 until schema.length) { @@ -174,12 +176,13 @@ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { } override def end(): Unit = { - current = new GenericRow(currentData) + // TODO: think about reusing the row versus reusing the underlying array + current = new RowType(currentData) } } /** - * A [[parquet.io.api.PrimitiveConverter]] that converts Parquet types to Catalyst types. + * A `parquet.io.api.PrimitiveConverter` that converts Parquet types to Catalyst types. * * @param parent The parent group converter. * @param fieldIndex The index inside the record. @@ -189,26 +192,26 @@ class CatalystPrimitiveConverter( fieldIndex: Int) extends PrimitiveConverter { // TODO: consider refactoring these together with ParquetTypesConverter override def addBinary(value: Binary): Unit = - parent.currentData.update(fieldIndex, value.getBytes.asInstanceOf[BinaryType.JvmType]) + parent.currentData.update(fieldIndex, value.getBytes) override def addBoolean(value: Boolean): Unit = - parent.currentData.update(fieldIndex, value.asInstanceOf[BooleanType.JvmType]) + parent.currentData.update(fieldIndex, value) override def addDouble(value: Double): Unit = - parent.currentData.update(fieldIndex, value.asInstanceOf[DoubleType.JvmType]) + parent.currentData.update(fieldIndex, value) override def addFloat(value: Float): Unit = - parent.currentData.update(fieldIndex, value.asInstanceOf[FloatType.JvmType]) + parent.currentData.update(fieldIndex, value) override def addInt(value: Int): Unit = - parent.currentData.update(fieldIndex, value.asInstanceOf[IntegerType.JvmType]) + parent.currentData.update(fieldIndex, value) override def addLong(value: Long): Unit = - parent.currentData.update(fieldIndex, value.asInstanceOf[LongType.JvmType]) + parent.currentData.update(fieldIndex, value) } /** - * A [[parquet.io.api.PrimitiveConverter]] that converts Parquet strings (fixed-length byte arrays) + * A `parquet.io.api.PrimitiveConverter` that converts Parquet strings (fixed-length byte arrays) * into Catalyst Strings. * * @param parent The parent group converter. @@ -218,6 +221,6 @@ class CatalystPrimitiveStringConverter( parent: CatalystGroupConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { override def addBinary(value: Binary): Unit = - parent.currentData.update(fieldIndex, value.toStringUsingUTF8.asInstanceOf[StringType.JvmType]) + parent.currentData.update(fieldIndex, value.toStringUsingUTF8) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala new file mode 100644 index 0000000000000..48e213891459d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -0,0 +1,85 @@ +/* + * 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.parquet + +import java.io.File + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapreduce.Job + +import parquet.schema.{MessageTypeParser, MessageType} +import parquet.hadoop.util.ContextUtil +import parquet.hadoop.ParquetWriter + +object ParquetTestData { + + val testSchema = + """message myrecord { + |optional boolean myboolean; + |optional int32 myint; + |optional binary mystring; + |optional int64 mylong; + |optional float myfloat; + |optional double mydouble; + |}""".stripMargin + + val subTestSchema = + """ + |message myrecord { + |optional boolean myboolean; + |optional int64 mylong; + |} + """.stripMargin + + val testFile = new File("/tmp/testParquetFile").getAbsoluteFile + + lazy val testData = new ParquetRelation("testData", testFile.toURI.toString) + + def writeFile = { + testFile.delete + val path: Path = new Path(testFile.toURI) + val job = new Job() + val configuration: Configuration = ContextUtil.getConfiguration(job) + val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) + + val writeSupport = new RowWriteSupport() + writeSupport.setSchema(schema, configuration) + val writer = new ParquetWriter(path, writeSupport) + for(i <- 0 until 15) { + val data = new Array[Any](6) + if(i % 3 ==0) { + data.update(0, true) + } else { + data.update(0, false) + } + if(i % 5 == 0) { + data.update(1, 5) + } else { + data.update(1, null) // optional + } + data.update(2, "abc") + data.update(3, 1L<<33) + data.update(4, 2.5F) + data.update(5, 4.5D) + writer.write(new ParquetRelation.RowType(data.toSeq)) + } + writer.close() + } +} + diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index 9018681478fcd..02c7c96780638 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -46,7 +46,7 @@ log4j.additivity.hive.ql.metadata.Hive=false log4j.logger.hive.ql.metadata.Hive=OFF # Parquet logging -parquet.hadoop.InternalParquetRecordReader=INFO -log4j.logger.parquet.hadoop.InternalParquetRecordReader=INFO -parquet.hadoop.ParquetInputFormat=INFO -log4j.logger.parquet.hadoop.ParquetInputFormat=INFO +parquet.hadoop.InternalParquetRecordReader=OFF +log4j.logger.parquet.hadoop.InternalParquetRecordReader=OFF +parquet.hadoop.ParquetInputFormat=OFF +log4j.logger.parquet.hadoop.ParquetInputFormat=OFF diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala similarity index 84% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala rename to sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala index 4e1f7fe8c0f19..c095263bbb492 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ParquetQueryTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution +package org.apache.spark.sql.parquet import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -32,10 +32,6 @@ import org.apache.spark.sql.catalyst.expressions.Row class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { override def beforeAll() { - // By clearing the port we force Spark to pick a new one. This allows us to rerun tests - // without restarting the JVM. - System.clearProperty("spark.driver.port") - System.clearProperty("spark.hostPort") ParquetTestData.writeFile } @@ -64,9 +60,9 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { } test("Projection of simple Parquet file") { - val scanner = new ParquetTableScan(ParquetTestData.testData.attributes, ParquetTestData.testData, None)(TestSqlContext.sparkContext) + val scanner = new ParquetTableScan(ParquetTestData.testData.output, ParquetTestData.testData, None)(TestSqlContext.sparkContext) val projected = scanner.pruneColumns(ParquetTypesConverter.convertToAttributes(MessageTypeParser.parseMessageType(ParquetTestData.subTestSchema))) - assert(projected.attributes.size === 2) + assert(projected.output.size === 2) val result = projected.execute().collect() val allChecks: Boolean = result.zipWithIndex.forall { case (row, index) => { @@ -86,7 +82,7 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { val job = new Job() val path = new Path("file:///tmp/test/mytesttable") val fs: FileSystem = FileSystem.getLocal(ContextUtil.getConfiguration(job)) - ParquetTypesConverter.writeMetaData(ParquetTestData.testData.attributes, path, TestSqlContext.sparkContext.hadoopConfiguration) + ParquetTypesConverter.writeMetaData(ParquetTestData.testData.output, path, TestSqlContext.sparkContext.hadoopConfiguration) assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) val metaData = ParquetTypesConverter.readMetaData(path) assert(metaData != null) @@ -96,13 +92,13 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { } /** - * Computes the given [[org.apache.spark.sql.execution.ParquetRelation]] and returns its RDD. + * Computes the given [[ParquetRelation]] and returns its RDD. * * @param parquetRelation The Parquet relation. * @return An RDD of Rows. */ private def getRDD(parquetRelation: ParquetRelation): RDD[Row] = { - val scanner = new ParquetTableScan(parquetRelation.attributes, parquetRelation, None)(TestSqlContext.sparkContext) + val scanner = new ParquetTableScan(parquetRelation.output, parquetRelation, None)(TestSqlContext.sparkContext) scanner.execute } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 153c06493189b..49164f5105f16 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -24,6 +24,7 @@ import catalyst.plans._ import catalyst.plans.logical.{BaseRelation, LogicalPlan} import org.apache.spark.sql.execution._ +import org.apache.spark.sql.parquet.{ParquetRelation, InsertIntoParquetTable, ParquetTableScan} trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. @@ -106,6 +107,8 @@ trait HiveStrategies { */ object ColumnPrunings extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + // TODO(andre): the current mix of HiveRelation and ParquetRelation + // here appears artificial; try to refactor to break it into two case PhysicalOperation(projectList, predicates, relation: BaseRelation) => val predicateOpt = predicates.reduceOption(And) val predicateRefs = predicateOpt.map(_.references).getOrElse(Set.empty) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ParquetQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala similarity index 93% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ParquetQuerySuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 2bf955590d622..4d3820902ec53 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ParquetQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -15,9 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.parquet +//package hive +//package execution + +// TODO: move this into the parquet package once it can be import java.io.File @@ -28,7 +30,9 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.WriteToFile import org.apache.spark.sql.catalyst.plans.logical.InsertIntoCreatedTable import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.execution.{ParquetTestData, ParquetRelation} +import org.apache.spark.sql.hive.TestHive + +//import org.apache.spark.sql.execution.{ParquetTestData, ParquetRelation} class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { @@ -53,8 +57,7 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf def storeQuery(querystr: String, filename: String): Unit = { val query = WriteToFile( filename, - TestHive.parseSql(querystr), - Some("testtable")) + TestHive.parseSql(querystr)) TestHive .executePlan(query) .stringResult() @@ -92,10 +95,6 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf } override def beforeAll() { - // By clearing the port we force Spark to pick a new one. This allows us to rerun tests - // without restarting the JVM. - System.clearProperty("spark.driver.port") - System.clearProperty("spark.hostPort") // write test data ParquetTestData.writeFile // Override initial Parquet test table From 7d0f13e9c8a2c336a2089affaad594943573577d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Mar 2014 11:28:03 -0800 Subject: [PATCH 743/778] Update parquet support with master. --- .../spark/sql/catalyst/expressions/Projection.scala | 9 +++++++-- .../apache/spark/sql/parquet/ParquetTableSupport.scala | 2 +- .../org/apache/spark/sql/parquet/ParquetTestData.scala | 2 +- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 56ba040c72bd4..4eea200c23a1a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -28,9 +28,10 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) { * be instantiated once per thread and reused. */ class JoinedRow extends Row { - private var row1: Row = _ - private var row2: Row = _ + private[this] var row1: Row = _ + private[this] var row2: Row = _ + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ def apply(r1: Row, r2: Row): Row = { row1 = r1 row2 = r2 @@ -63,4 +64,8 @@ class JoinedRow extends Row { def getByte(i: Int): Byte = if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 6fe5e4462447d..963e92497d43f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -150,7 +150,7 @@ object RowWriteSupport { */ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { type RowType = ParquetRelation.RowType - var current: RowType = new RowType(Seq()) + var current: RowType = new RowType(Array[Any]()) // initialization may not strictly be required val currentData: Array[Any] = new Array[Any](schema.length) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 48e213891459d..6d8bbaf1b7232 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -77,7 +77,7 @@ object ParquetTestData { data.update(3, 1L<<33) data.update(4, 2.5F) data.update(5, 4.5D) - writer.write(new ParquetRelation.RowType(data.toSeq)) + writer.write(new ParquetRelation.RowType(data.toArray)) } writer.close() } From 3c3f9624a4c3041a0d8b68bc4e218ea6e0eef769 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Mar 2014 12:17:34 -0800 Subject: [PATCH 744/778] Fix a bug due to array reuse. This will need to be revisited after we merge the mutable row PR. --- .../org/apache/spark/sql/parquet/ParquetTableSupport.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 963e92497d43f..bb9c0736eae83 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -177,7 +177,7 @@ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { override def end(): Unit = { // TODO: think about reusing the row versus reusing the underlying array - current = new RowType(currentData) + current = new RowType(currentData.clone()) } } From d37139320dd35c91c22903a919aa177ae68e4cf7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 4 Mar 2014 18:54:21 -0800 Subject: [PATCH 745/778] Add a framework for dealing with mutable rows to reduce the number of object allocations that occur in the critical path. --- .../sql/catalyst/expressions/Expression.scala | 9 +-- .../sql/catalyst/expressions/Projection.scala | 47 ++++++++++-- .../spark/sql/catalyst/expressions/Row.scala | 71 ++++++++++++++++++- .../sql/catalyst/expressions/aggregates.scala | 7 +- .../apache/spark/sql/SparkSqlContext.scala | 2 +- .../apache/spark/sql/execution/Exchange.scala | 8 ++- .../spark/sql/execution/SparkStrategies.scala | 2 +- .../spark/sql/execution/aggregates.scala | 2 +- .../spark/sql/execution/basicOperators.scala | 13 ++-- .../apache/spark/sql/execution/joins.scala | 20 +++--- .../scala/org/apache/spark/sql/TgfSuite.scala | 6 +- .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../apache/spark/sql/hive/hiveOperators.scala | 2 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 4 +- 14 files changed, 152 insertions(+), 45 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index a07045fed79f9..78aaaeebbd631 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -30,6 +30,7 @@ abstract class Expression extends TreeNode[Expression] { type EvaluatedType <: Any def dataType: DataType + /** * Returns true when an expression is a candidate for static evaluation before the query is * executed. @@ -53,14 +54,6 @@ abstract class Expression extends TreeNode[Expression] { def apply(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") - // Primitive Accessor functions that avoid boxing for performance. - // Note this is an Unstable API as it doesn't correctly handle null values yet. - - def applyBoolean(input: Row): Boolean = apply(input).asInstanceOf[Boolean] - def applyInt(input: Row): Int = apply(input).asInstanceOf[Int] - def applyDouble(input: Row): Double = apply(input).asInstanceOf[Double] - def applyString(input: Row): String = apply(input).asInstanceOf[String] - /** * Returns `true` if this expression and all its children have been resolved to a specific schema * and `false` if it is still contains any unresolved placeholders. Implementations of expressions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 4eea200c23a1a..ae04be2fb0d1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -2,10 +2,9 @@ package org.apache.spark.sql.catalyst package expressions /** - * Converts a Row to another Row given a set of expressions. - * - * If the schema of the input row is specified, then the given expression will be bound to that - * schema. + * Converts a [[Row]] to another Row given a sequence of expression that define each column of the + * new row. If the schema of the input row is specified, then the given expression will be bound to + * that schema. */ class Projection(expressions: Seq[Expression]) extends (Row => Row) { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = @@ -23,6 +22,33 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) { } } +/** + * Converts a [[Row]] to another Row given a sequence of expression that define each column of th + * new row. If the schema of the input row is specified, then the given expression will be bound to + * that schema. + * + * In contrast to a normal projection, a MutableProjection reuses the same underlying row object + * each time an input row is added. This significatly reduces the cost of calcuating the + * projection, but means that it is not safe + */ +case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) { + def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = + this(expressions.map(BindReferences.bindReference(_, inputSchema))) + + private[this] val exprArray = expressions.toArray + private[this] val mutableRow = new GenericMutableRow(exprArray.size) + def currentValue: Row = mutableRow + + def apply(input: Row): Row = { + var i = 0 + while (i < exprArray.size) { + mutableRow(i) = exprArray(i).apply(input) + i += 1 + } + mutableRow + } +} + /** * A mutable wrapper that makes two rows appear appear as a single concatenated row. Designed to * be instantiated once per thread and reused. @@ -68,4 +94,17 @@ class JoinedRow extends Row { def getFloat(i: Int): Float = if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 6bb0decc0e1cf..a2e32b414839a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -28,7 +28,7 @@ import types._ * It is invalid to use the native primitive interface to retrieve a value that is null, instead a * user must check [[isNullAt]] before attempting to retrieve a value that might be null. */ -abstract class Row extends Seq[Any] with Serializable { +trait Row extends Seq[Any] with Serializable { def apply(i: Int): Any def isNullAt(i: Int): Boolean @@ -40,9 +40,39 @@ abstract class Row extends Seq[Any] with Serializable { def getBoolean(i: Int): Boolean def getShort(i: Int): Short def getByte(i: Int): Byte + def getString(i: Int): String override def toString() = s"[${this.mkString(",")}]" + + def copy(): Row +} + +/** + * An extended interface to [[Row]] that allows the values for each column to be updated. Setting + * a value through a primitive function implicitly marks that column as not null. + */ +trait MutableRow extends Row { + def setNullAt(i: Int): Unit + + def update(ordinal: Int, value: Any) + + def setInt(ordinal: Int, value: Int) + def setLong(ordinal: Int, value: Long) + def setDouble(ordinal: Int, value: Double) + def setBoolean(ordinal: Int, value: Boolean) + def setShort(ordinal: Int, value: Short) + def setByte(ordinal: Int, value: Byte) + def setFloat(ordinal: Int, value: Byte) + + /** + * EXPERIMENTAL + * + * Returns a mutable string builder for the specified column. A given row should return the + * result of any mutations made to the returned buffer next time getString is called for the same + * column. + */ + def getStringBuilder(ordinal: Int): StringBuilder } /** @@ -62,12 +92,19 @@ object EmptyRow extends Row { def getBoolean(i: Int): Boolean = throw new UnsupportedOperationException def getShort(i: Int): Short = throw new UnsupportedOperationException def getByte(i: Int): Byte = throw new UnsupportedOperationException + def getString(i: Int): String = throw new UnsupportedOperationException + + def copy() = this } /** - * A row implementation that uses an array of objects as the underlying storage. + * A row implementation that uses an array of objects as the underlying storage. Note that, while + * the array is not copied, and thus could technically be mutated after creation, this is not + * allowed. */ -class GenericRow(val values: Array[Any]) extends Row { +class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { + def this(size: Int) = this(new Array[Any](size)) + def iterator = values.iterator def length = values.length @@ -104,8 +141,36 @@ class GenericRow(val values: Array[Any]) extends Row { if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.") values(i).asInstanceOf[Byte] } + + def getString(i: Int): String = { + if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.") + values(i).asInstanceOf[String] + } + + def copy() = this +} + +class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { + + def getStringBuilder(ordinal: Int): StringBuilder = ??? + + override def setBoolean(ordinal: Int,value: Boolean): Unit = { values(ordinal) = value } + override def setByte(ordinal: Int,value: Byte): Unit = { values(ordinal) = value } + override def setDouble(ordinal: Int,value: Double): Unit = { values(ordinal) = value } + override def setFloat(ordinal: Int,value: Byte): Unit = { values(ordinal) = value } + override def setInt(ordinal: Int,value: Int): Unit = { values(ordinal) = value } + override def setLong(ordinal: Int,value: Long): Unit = { values(ordinal) = value } + + override def setNullAt(i: Int): Unit = { values(i) = null } + + override def setShort(ordinal: Int,value: Short): Unit = { values(ordinal) = value } + + override def update(ordinal: Int,value: Any): Unit = { values(ordinal) = value } + + override def copy() = new GenericRow(values.clone()) } + class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { def compare(a: Row, b: Row): Int = { var i = 0 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 010064489c421..2287a849e6831 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -170,12 +170,15 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) def this() = this(null, null) // Required for serialization. private var count: Long = _ - private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) + private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(EmptyRow)) private val sumAsDouble = Cast(sum, DoubleType) + + private val addFunction = Add(sum, expr) - override def apply(input: Row): Any = sumAsDouble.applyDouble(null) / count.toDouble + override def apply(input: Row): Any = + sumAsDouble.apply(EmptyRow).asInstanceOf[Double] / count.toDouble def update(input: Row): Unit = { count += 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala index f25de3827855c..edf912bd8fa92 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala @@ -112,7 +112,7 @@ class SparkSqlContext(val sparkContext: SparkContext) extends Logging { lazy val executedPlan: SparkPlan = PrepareForExecution(sparkPlan) // TODO: We are loosing schema here. - lazy val toRdd: RDD[Row] = executedPlan.execute() + lazy val toRdd: RDD[Row] = executedPlan.execute().map(_.copy()) protected def stringOrError[A](f: => A): String = try f.toString catch { case e: Throwable => e.toString } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 4226a1a85509c..74bbdcd4cddbb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -35,8 +35,11 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una newPartitioning match { case HashPartitioning(expressions, numPartitions) => { // TODO: Eliminate redundant expressions in grouping key and value. - val rdd = child.execute().mapPartitions { iter => + // TODO: Should avoid copy, but there seems to be a bug in Spark. + val rdd = child.execute().map(_.copy()).mapPartitions { iter => + // TODO: Should use MutableProjection val hashExpressions = new Projection(expressions) + // TODO: Should use mutable pair. iter.map(r => (hashExpressions(r), r)) } val part = new HashPartitioner(numPartitions) @@ -48,7 +51,8 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una // TODO: ShuffledRDD should take an Ordering. implicit val ordering = new RowOrdering(sortingExpressions) - val rdd = child.execute().map(row => (row, null)) + // TODO: Avoid copy and use MutablePair. + val rdd = child.execute().map(row => (row.copy(), null)) val part = new RangePartitioner(numPartitions, rdd, ascending = true) val shuffled = new ShuffledRDD[Row, Null, (Row, Null)](rdd, part) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 6c10a537f225e..828618ebd4d4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -154,7 +154,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } protected lazy val singleRowRdd = - sparkContext.parallelize(Seq(new GenericRow(Array()): Row), 1) + sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1) def convertToCatalyst(a: Any): Any = a match { case s: Seq[Any] => s.map(convertToCatalyst) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala index 132576cbc4fae..51889c1988680 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala @@ -90,7 +90,7 @@ case class Aggregate( // in the [[catalyst.execution.Exchange]]. val grouped = child.execute().mapPartitions { iter => val buildGrouping = new Projection(groupingExpressions) - iter.map(row => (buildGrouping(row), row)) + iter.map(row => (buildGrouping(row), row.copy())) }.groupByKeyLocally() val result = grouped.map { case (group, rows) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 9a61c021fceea..40d3ef8128756 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -30,15 +30,16 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends def output = projectList.map(_.toAttribute) def execute() = child.execute().mapPartitions { iter => - val buildProjection = new Projection(projectList) - iter.map(buildProjection) + @transient val resuableProjection = new MutableProjection(projectList) + iter.map(resuableProjection) } } case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { def output = child.output + def execute() = child.execute().mapPartitions { iter => - iter.filter(condition.applyBoolean) + iter.filter(condition.apply(_).asInstanceOf[Boolean]) } } @@ -64,7 +65,7 @@ case class StopAfter(limit: Int, child: SparkPlan)(@transient sc: SparkContext) def output = child.output - override def executeCollect() = child.execute().take(limit) + override def executeCollect() = child.execute().map(_.copy()).take(limit) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. @@ -80,7 +81,7 @@ case class TopK(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) @transient lazy val ordering = new RowOrdering(sortOrder) - override def executeCollect() = child.execute().takeOrdered(limit)(ordering) + override def executeCollect() = child.execute().map(_.copy()).takeOrdered(limit)(ordering) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. @@ -103,7 +104,7 @@ case class Sort( // TODO: Optimize sorting operation? child.execute() .mapPartitions( - iterator => iterator.toArray.sorted(ordering).iterator, + iterator => iterator.map(_.copy()).toArray.sorted(ordering).iterator, preservesPartitioning = true) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index d44905dc84260..f67210d5dea49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -46,12 +46,12 @@ case class SparkEquiInnerJoin( def execute() = attachTree(this, "execute") { val leftWithKeys = left.execute().mapPartitions { iter => val generateLeftKeys = new Projection(leftKeys, left.output) - iter.map(row => (generateLeftKeys(row), row)) + iter.map(row => (generateLeftKeys(row), row.copy())) } val rightWithKeys = right.execute().mapPartitions { iter => val generateRightKeys = new Projection(rightKeys, right.output) - iter.map(row => (generateRightKeys(row), row)) + iter.map(row => (generateRightKeys(row), row.copy())) } // Do the join. @@ -73,7 +73,7 @@ case class SparkEquiInnerJoin( case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { def output = left.output ++ right.output - def execute() = left.execute().cartesian(right.execute()).map { + def execute() = left.execute().map(_.copy()).cartesian(right.execute().map(_.copy())).map { case (l: Row, r: Row) => buildRow(l ++ r) } } @@ -95,17 +95,19 @@ case class BroadcastNestedLoopJoin( /** The Broadcast relation */ def right = broadcast + @transient lazy val boundCondition = + condition + .map(c => BindReferences.bindReference(c, left.output ++ right.output)) + .getOrElse(Literal(true)) + + def execute() = { - val broadcastedRelation = sc.broadcast(broadcast.execute().collect().toIndexedSeq) + val broadcastedRelation = sc.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) val streamedPlusMatches = streamed.execute().mapPartitions { streamedIter => val matchedRows = new mutable.ArrayBuffer[Row] val includedBroadcastTuples = new mutable.BitSet(broadcastedRelation.value.size) val joinedRow = new JoinedRow - val boundCondition = - condition - .map(c => BindReferences.bindReference(c, left.output ++ right.output)) - .getOrElse(Literal(true)) streamedIter.foreach { streamedRow => var i = 0 @@ -114,7 +116,7 @@ case class BroadcastNestedLoopJoin( while (i < broadcastedRelation.value.size) { // TODO: One bitset per partition instead of per row. val broadcastedRow = broadcastedRelation.value(i) - if (boundCondition.applyBoolean(joinedRow(streamedRow, broadcastedRow))) { + if (boundCondition(joinedRow(streamedRow, broadcastedRow)).asInstanceOf[Boolean]) { matchedRows += buildRow(streamedRow ++ broadcastedRow) matched = true includedBroadcastTuples += i diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala index df1a95c69c5bd..c790243082825 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala @@ -45,11 +45,11 @@ case class ExampleTGF(input: Seq[Attribute] = Seq('name, 'age)) extends Generato override def apply(input: Row): TraversableOnce[Row] = { val name = nameAttr.apply(input) - val age = ageAttr.applyInt(input) + val age = ageAttr.apply(input).asInstanceOf[Int] Iterator( - new GenericRow(Array(s"$name is $age years old")), - new GenericRow(Array(s"Next year, $name will be ${age + 1} years old"))) + new GenericRow(Array[Any](s"$name is $age years old")), + new GenericRow(Array[Any](s"Next year, $name will be ${age + 1} years old"))) } } 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 e269c8c846700..3e8e491e8758a 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 @@ -205,7 +205,7 @@ abstract class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { override val planner = HivePlanner - protected lazy val emptyResult = sparkContext.parallelize(Seq(new GenericRow(Array()): Row), 1) + protected lazy val emptyResult = sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1) /** Extends QueryExecution with hive specific features. */ abstract class QueryExecution extends super.QueryExecution { @@ -226,7 +226,7 @@ abstract class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { sparkContext.parallelize(asRows, 1) } case _ => - executedPlan.execute() + executedPlan.execute.map(_.copy()) } protected val primitiveTypes = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala index d1ae4c4111159..d20fd87f34f48 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -141,7 +141,7 @@ case class HiveTableScan( // Only partitioned values are needed here, since the predicate has already been bound to // partition key attribute references. val row = new GenericRow(castedValues.toArray) - shouldKeep.applyBoolean(row) + shouldKeep.apply(row).asInstanceOf[Boolean] } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index c04bbe8d41785..5e775d6a048de 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -80,14 +80,14 @@ object HiveFunctionRegistry case c: Class[_] if c == classOf[hadoopIo.BooleanWritable] => BooleanType case c: Class[_] if c == classOf[java.lang.String] => StringType case c: Class[_] if c == java.lang.Short.TYPE => ShortType - case c: Class[_] if c == java.lang.Integer.TYPE => ShortType + case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType case c: Class[_] if c == java.lang.Long.TYPE => LongType case c: Class[_] if c == java.lang.Double.TYPE => DoubleType case c: Class[_] if c == java.lang.Byte.TYPE => ByteType case c: Class[_] if c == java.lang.Float.TYPE => FloatType case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType case c: Class[_] if c == classOf[java.lang.Short] => ShortType - case c: Class[_] if c == classOf[java.lang.Integer] => ShortType + case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType case c: Class[_] if c == classOf[java.lang.Long] => LongType case c: Class[_] if c == classOf[java.lang.Double] => DoubleType case c: Class[_] if c == classOf[java.lang.Byte] => ByteType From 959bdf0bb5362d6387e1748dd16b62f6abfe4801 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Mar 2014 18:05:25 -0800 Subject: [PATCH 746/778] Don't silently swallow all KryoExceptions, only the one that indicates the end of a stream. --- .../main/scala/org/apache/spark/serializer/KryoSerializer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 920490f9d0d61..d2cf9efceef25 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -103,7 +103,7 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser kryo.readClassAndObject(input).asInstanceOf[T] } catch { // DeserializationStream uses the EOF exception to indicate stopping condition. - case _: KryoException => throw new EOFException + case e: KryoException if e.getMessage == "Buffer underflow." => throw new EOFException } } From 9049cf0d432662cb40c7e31688049d9a1db6e732 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Mar 2014 18:06:53 -0800 Subject: [PATCH 747/778] Extend MutablePair interface to support easy syntax for in-place updates. Also add a constructor so that it can be serialized out-of-the-box. --- .../scala/org/apache/spark/util/MutablePair.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index b053266f12748..e317882f6e856 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -25,10 +25,20 @@ package org.apache.spark.util * @param _2 Element 2 of this MutablePair */ case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T1, - @specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T2] + @specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T2] (var _1: T1, var _2: T2) extends Product2[T1, T2] { + /** No-arg constructor for serialization */ + def this() = this(null.asInstanceOf[T1], null.asInstanceOf[T2]) + + /** Updates this pair with new values and returns itself */ + def apply(n1: T1, n2: T2): MutablePair[T1, T2] = { + _1 = n1 + _2 = n2 + this + } + override def toString = "(" + _1 + "," + _2 + ")" override def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[_,_]] From d9943336fda9c31fda202ed13e5c06b074214539 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Mar 2014 18:08:15 -0800 Subject: [PATCH 748/778] Remove copies before shuffle, this required changing the default shuffle serialization. --- .../spark/sql/catalyst/expressions/Row.scala | 5 ++ .../apache/spark/sql/execution/Exchange.scala | 64 +++++++++++++++---- 2 files changed, 55 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index a2e32b414839a..565d171841cc1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -103,6 +103,9 @@ object EmptyRow extends Row { * allowed. */ class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { + /** No-arg constructor for serialization. */ + def this() = this(null) + def this(size: Int) = this(new Array[Any](size)) def iterator = values.iterator @@ -151,6 +154,8 @@ class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { } class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { + /** No-arg constructor for serialization. */ + def this() = this(0) def getStringBuilder(ordinal: Int): StringBuilder = ??? diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 74bbdcd4cddbb..e7ed1a5e1658d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -18,43 +18,79 @@ package org.apache.spark.sql package execution +import java.nio.ByteBuffer + +import com.esotericsoftware.kryo.{Kryo, Serializer} +import com.esotericsoftware.kryo.io.{Output, Input} + +import org.apache.spark.{SparkConf, RangePartitioner, HashPartitioner} +import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.util.MutablePair + import catalyst.rules.Rule import catalyst.errors._ import catalyst.expressions._ import catalyst.plans.physical._ -import org.apache.spark.{RangePartitioner, HashPartitioner} -import org.apache.spark.rdd.ShuffledRDD +class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { + override def newKryo(): Kryo = { + val kryo = new Kryo + kryo.setRegistrationRequired(true) + kryo.register(classOf[MutablePair[_,_]]) + kryo.register(classOf[Array[Any]]) + kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow]) + kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow]) + kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]]) + kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) + kryo.setReferences(false) + kryo.setClassLoader(this.getClass.getClassLoader) + kryo + } +} + +class BigDecimalSerializer extends Serializer[BigDecimal] { + def write(kryo: Kryo, output: Output, bd: math.BigDecimal) { + // TODO: There are probably more efficient representations than strings... + output.writeString(bd.toString) + } + + def read(kryo: Kryo, input: Input, tpe: Class[BigDecimal]): BigDecimal = { + BigDecimal(input.readString()) + } +} case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { override def outputPartitioning = newPartitioning + def output = child.output def execute() = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => { // TODO: Eliminate redundant expressions in grouping key and value. - // TODO: Should avoid copy, but there seems to be a bug in Spark. - val rdd = child.execute().map(_.copy()).mapPartitions { iter => - // TODO: Should use MutableProjection - val hashExpressions = new Projection(expressions) - // TODO: Should use mutable pair. - iter.map(r => (hashExpressions(r), r)) + val rdd = child.execute().mapPartitions { iter => + val hashExpressions = new MutableProjection(expressions) + val mutablePair = new MutablePair[Row, Row]() + iter.map(r => mutablePair(hashExpressions(r), r)) } val part = new HashPartitioner(numPartitions) - val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) - + val shuffled = new ShuffledRDD[Row, Row, MutablePair[Row, Row]](rdd, part) + shuffled.setSerializer(classOf[SparkSqlSerializer].getName) shuffled.map(_._2) } case RangePartitioning(sortingExpressions, numPartitions) => { - // TODO: ShuffledRDD should take an Ordering. + // TODO: RangePartitioner should take an Ordering. implicit val ordering = new RowOrdering(sortingExpressions) - // TODO: Avoid copy and use MutablePair. - val rdd = child.execute().map(row => (row.copy(), null)) + val rdd = child.execute().mapPartitions { iter => + val mutablePair = new MutablePair[Row, Null](null, null) + iter.map(row => mutablePair(row, null)) + } val part = new RangePartitioner(numPartitions, rdd, ascending = true) - val shuffled = new ShuffledRDD[Row, Null, (Row, Null)](rdd, part) + val shuffled = new ShuffledRDD[Row, Null, MutablePair[Row, Null]](rdd, part) + shuffled.setSerializer(classOf[SparkSqlSerializer].getName) shuffled.map(_._1) } From ba28849fa9ec163dc39889cd7f3d683f28692b33 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Mar 2014 18:23:05 -0800 Subject: [PATCH 749/778] code review comments. --- .../org/apache/spark/sql/catalyst/expressions/Row.scala | 6 ++++++ .../main/scala/org/apache/spark/sql/execution/joins.scala | 6 +++--- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 565d171841cc1..6d12e6c606373 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -120,26 +120,32 @@ class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { if (values(i) == null) sys.error("Failed to check null bit for primitive int value.") values(i).asInstanceOf[Int] } + def getLong(i: Int): Long = { if (values(i) == null) sys.error("Failed to check null bit for primitive long value.") values(i).asInstanceOf[Long] } + def getDouble(i: Int): Double = { if (values(i) == null) sys.error("Failed to check null bit for primitive double value.") values(i).asInstanceOf[Double] } + def getFloat(i: Int): Float = { if (values(i) == null) sys.error("Failed to check null bit for primitive float value.") values(i).asInstanceOf[Float] } + def getBoolean(i: Int): Boolean = { if (values(i) == null) sys.error("Failed to check null bit for primitive boolean value.") values(i).asInstanceOf[Boolean] } + def getShort(i: Int): Short = { if (values(i) == null) sys.error("Failed to check null bit for primitive short value.") values(i).asInstanceOf[Short] } + def getByte(i: Int): Byte = { if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.") values(i).asInstanceOf[Byte] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index f67210d5dea49..5934fd1b03bfb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -96,9 +96,9 @@ case class BroadcastNestedLoopJoin( def right = broadcast @transient lazy val boundCondition = - condition - .map(c => BindReferences.bindReference(c, left.output ++ right.output)) - .getOrElse(Literal(true)) + condition + .map(c => BindReferences.bindReference(c, left.output ++ right.output)) + .getOrElse(Literal(true)) def execute() = { From 54637ecce8ea9a9af3b41ce4a7a719249bcff2f2 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 9 Mar 2014 20:11:58 +0100 Subject: [PATCH 750/778] First part of second round of code review feedback --- .../spark/sql/parquet/ParquetRelation.scala | 43 ++++---- .../sql/parquet/ParquetTableOperations.scala | 101 ++++++++++++++---- .../sql/parquet/ParquetTableSupport.scala | 26 ++--- .../spark/sql/parquet/ParquetTestData.scala | 26 ++++- sql/core/src/test/resources/log4j.properties | 8 +- .../spark/sql/parquet/ParquetQueryTests.scala | 62 +++++++---- .../org/apache/spark/sql/hive/TestShark.scala | 3 + .../spark/sql/parquet/ParquetQuerySuite.scala | 54 ++++++---- 8 files changed, 208 insertions(+), 115 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 945e9d654b446..67d70f0af2329 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -77,7 +77,7 @@ import scala.collection.JavaConversions._ * * scala> val query = TestHive.parseSql(query_string).transform { * | case relation @ UnresolvedRelation(databaseName, name, alias) => - * | if(name == "psrc") ParquetRelation(name, filename) + * | if (name == "psrc") ParquetRelation(name, filename) * | else relation * | } * query: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan = @@ -138,7 +138,7 @@ object ParquetRelation { child: LogicalPlan, conf: Configuration, tableName: Option[String]): ParquetRelation = { - if(!child.resolved) { + if (!child.resolved) { throw new UnresolvedException[LogicalPlan]( child, "Attempt to create Parquet table from unresolved child (when schema is not available)") @@ -153,13 +153,13 @@ object ParquetRelation { private def checkPath(pathStr: String, conf: Configuration): Path = { val path = new Path(pathStr) val fs = path.getFileSystem(conf) - if(fs.exists(path) && + if (fs.exists(path) && !fs.getFileStatus(path) .getPermission .getUserAction .implies(FsAction.READ_WRITE)) { throw new IOException( - s"Unable to create ParquetRelation: path ${path.toString} not read-writable") + s"Unable to create ParquetRelation: path $path not read-writable") } path } @@ -176,9 +176,13 @@ object ParquetTypesConverter { case ParquetPrimitiveTypeName.FLOAT => FloatType case ParquetPrimitiveTypeName.INT32 => IntegerType case ParquetPrimitiveTypeName.INT64 => LongType - case ParquetPrimitiveTypeName.INT96 => LongType // TODO: is there an equivalent? + case ParquetPrimitiveTypeName.INT96 => { + // TODO: add BigInteger type? TODO(andre) use DecimalType instead???? + sys.error("Warning: potential loss of precision: converting INT96 to long") + LongType + } case _ => sys.error( - s"Unsupported parquet datatype ${parquetType.asInstanceOf[Enum[String]].toString()}") + s"Unsupported parquet datatype $parquetType") } def fromDataType(ctype: DataType): ParquetPrimitiveTypeName = ctype match { @@ -189,7 +193,7 @@ object ParquetTypesConverter { case FloatType => ParquetPrimitiveTypeName.FLOAT case IntegerType => ParquetPrimitiveTypeName.INT32 case LongType => ParquetPrimitiveTypeName.INT64 - case _ => sys.error(s"Unsupported datatype ${ctype.toString}") + case _ => sys.error(s"Unsupported datatype $ctype") } def consumeType(consumer: RecordConsumer, ctype: DataType, record: Row, index: Int): Unit = { @@ -204,7 +208,7 @@ object ParquetTypesConverter { case DoubleType => consumer.addDouble(record.getDouble(index)) case FloatType => consumer.addFloat(record.getFloat(index)) case BooleanType => consumer.addBoolean(record.getBoolean(index)) - case _ => sys.error(s"Unsupported datatype ${ctype.toString}, cannot write to consumer") + case _ => sys.error(s"Unsupported datatype $ctype, cannot write to consumer") } } @@ -232,18 +236,18 @@ object ParquetTypesConverter { def writeMetaData(attributes: Seq[Attribute], path: Path, conf: Configuration) { val fileSystem = FileSystem.get(conf) - if(fileSystem.exists(path) && !fileSystem.getFileStatus(path).isDir) { - throw new IOException(s"Expected to write to directory ${path.toString} but found file") + if (fileSystem.exists(path) && !fileSystem.getFileStatus(path).isDir) { + throw new IOException(s"Expected to write to directory $path but found file") } val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) - if(fileSystem.exists(metadataPath)) { + if (fileSystem.exists(metadataPath)) { try { fileSystem.delete(metadataPath, true) } catch { case e: IOException => - throw new IOException(s"Unable to delete previous PARQUET_METADATA_FILE:\n${e.toString}") + throw new IOException(s"Unable to delete previous PARQUET_METADATA_FILE at $metadataPath") } } @@ -255,18 +259,13 @@ object ParquetTypesConverter { ParquetTypesConverter.convertFromAttributes(attributes) val metaData: FileMetaData = new FileMetaData( parquetSchema, - new java.util.HashMap[String, String](), - "Shark") + extraMetadata, + "Spark") ParquetFileWriter.writeMetadataFile( conf, path, - new Footer( - path, - new ParquetMetadata( - metaData, - Nil) - ) :: Nil) + new Footer(path, new ParquetMetadata(metaData, Nil)) :: Nil) } /** @@ -283,11 +282,11 @@ object ParquetTypesConverter { val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) - if(fs.exists(metadataPath) && fs.isFile(metadataPath)) { + if (fs.exists(metadataPath) && fs.isFile(metadataPath)) { // TODO: improve exception handling, etc. ParquetFileReader.readFooter(conf, metadataPath) } else { - if(!fs.exists(path) || !fs.isFile(path)) { + if (!fs.exists(path) || !fs.isFile(path)) { throw new FileNotFoundException( s"Could not find file ${path.toString} when trying to read metadata") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 7582097896882..0c50fff10c998 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -15,34 +15,37 @@ * limitations under the License. */ -package org.apache.spark.sql -package parquet +package org.apache.spark.sql.parquet -import _root_.parquet.io.InvalidRecordException -import _root_.parquet.schema.MessageType -import _root_.parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat} -import _root_.parquet.hadoop.util.ContextUtil +import parquet.io.InvalidRecordException +import parquet.schema.MessageType +import parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat} +import parquet.hadoop.util.ContextUtil import org.apache.spark.rdd.RDD -import org.apache.spark.api.java.JavaPairRDD -import org.apache.spark.SparkContext +//import org.apache.spark.api.java.JavaPairRDD +import org.apache.spark.{TaskContext, SerializableWritable, SparkContext} import org.apache.spark.sql.catalyst.expressions.{Row, Attribute, Expression} import org.apache.spark.sql.execution.{SparkPlan, UnaryNode, LeafNode} -import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} +import org.apache.hadoop.mapreduce._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import java.io.IOException +import org.apache.spark.util.MutablePair +import java.text.SimpleDateFormat +import java.util.Date /** * Parquet table scan operator. Imports the file that backs the given * [[ParquetRelation]] as a RDD[Row]. */ case class ParquetTableScan( - output: Seq[Attribute], - relation: ParquetRelation, - columnPruningPred: Option[Expression])( + @transient output: Seq[Attribute], + @transient relation: ParquetRelation, + @transient columnPruningPred: Option[Expression])( @transient val sc: SparkContext) extends LeafNode { @@ -56,6 +59,12 @@ case class ParquetTableScan( RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, ParquetTypesConverter.convertFromAttributes(output).toString) // TODO: think about adding record filters + /* Comments regarding record filters: it would be nice to push down as much filtering + to Parquet as possible. However, currently it seems we cannot pass enough information + to materialize an (arbitrary) Catalyst [[Predicate]] inside Parquet's + ``FilteredRecordReader`` (via Configuration, for example). Simple + filter-rows-by-column-values however should be supported. + */ sc.newAPIHadoopFile( relation.path, classOf[ParquetInputFormat[Row]], @@ -72,7 +81,7 @@ case class ParquetTableScan( */ def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = { val success = validateProjection(prunedAttributes) - if(success) { + if (success) { ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sc) } else { sys.error("Warning: Could not validate Parquet schema projection in pruneColumns") @@ -102,10 +111,10 @@ case class ParquetTableScan( } case class InsertIntoParquetTable( - relation: ParquetRelation, - child: SparkPlan)( + @transient relation: ParquetRelation, + @transient child: SparkPlan)( @transient val sc: SparkContext) - extends UnaryNode { + extends UnaryNode with SparkHadoopMapReduceUtil { /** * Inserts all the rows in the Parquet file. Note that OVERWRITE is implicit, since @@ -142,18 +151,64 @@ case class InsertIntoParquetTable( s"Unable to clear output directory ${fspath.toString} prior" + s" to InsertIntoParquetTable:\n${e.toString}") } - - JavaPairRDD.fromRDD(childRdd.map(Tuple2(null, _))).saveAsNewAPIHadoopFile( - relation.path.toString, - classOf[Void], - classOf[ParquetRelation.RowType], - classOf[_root_.parquet.hadoop.ParquetOutputFormat[ParquetRelation.RowType]], - conf) + saveAsHadoopFile(childRdd, relation.path.toString, conf) // We return the child RDD to allow chaining (alternatively, one could return nothing). childRdd } override def output = child.output + + // based on ``saveAsNewAPIHadoopFile`` in [[PairRDDFunctions]] + // TODO: Maybe PairRDDFunctions should use MutablePair instead of Tuple2? + // .. then we could use the default one and could use [[MutablePair]] + // instead of ``Tuple2`` + private def saveAsHadoopFile( + rdd: RDD[Row], + path: String, + conf: Configuration) { + val job = new Job(conf) + val keyType = classOf[Void] + val outputFormatType = classOf[parquet.hadoop.ParquetOutputFormat[Row]] + job.setOutputKeyClass(keyType) + job.setOutputValueClass(classOf[Row]) + val wrappedConf = new SerializableWritable(job.getConfiguration) + NewFileOutputFormat.setOutputPath(job, new Path(path)) + val formatter = new SimpleDateFormat("yyyyMMddHHmm") + val jobtrackerID = formatter.format(new Date()) + val stageId = sc.newRddId() + + def writeShard(context: TaskContext, iter: Iterator[Row]): Int = { + // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it + // around by taking a mod. We expect that no task will be attempted 2 billion times. + val attemptNumber = (context.attemptId % Int.MaxValue).toInt + /* "reduce task" */ + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, + attemptNumber) + val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) + val format = outputFormatType.newInstance + val committer = format.getOutputCommitter(hadoopContext) + committer.setupTask(hadoopContext) + val writer = format.getRecordWriter(hadoopContext) + while (iter.hasNext) { + val row = iter.next() + writer.write(null, row) + } + writer.close(hadoopContext) + committer.commitTask(hadoopContext) + return 1 + } + val jobFormat = outputFormatType.newInstance + /* apparently we need a TaskAttemptID to construct an OutputCommitter; + * however we're only going to use this local OutputCommitter for + * setupJob/commitJob, so we just use a dummy "map" task. + */ + val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0) + val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) + val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) + jobCommitter.setupJob(jobTaskContext) + sc.runJob(rdd, writeShard _) + jobCommitter.commitJob(jobTaskContext) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index bb9c0736eae83..aae2b07aed6a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -39,9 +39,7 @@ import org.apache.spark.sql.catalyst.types._ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterializer[Row] { def this(parquetSchema: MessageType) = - this( - new CatalystGroupConverter( - ParquetTypesConverter.convertToAttributes(parquetSchema))) + this(new CatalystGroupConverter(ParquetTypesConverter.convertToAttributes(parquetSchema))) override def getCurrentRecord: Row = root.getCurrentRecord @@ -106,11 +104,7 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { private var attributes: Seq[Attribute] = null override def init(configuration: Configuration): WriteSupport.WriteContext = { - schema = if(schema == null) { - getSchema(configuration) - } else { - schema - } + schema = if (schema == null) getSchema(configuration) else schema attributes = ParquetTypesConverter.convertToAttributes(schema) new WriteSupport.WriteContext( schema, @@ -123,16 +117,16 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { // TODO: add groups (nested fields) override def write(record: Row): Unit = { + var index = 0 writer.startMessage() - // TODO: compare performance of the various ways of looping over a row - for(pair <- attributes.zipWithIndex) { - val (attribute, index) = pair - // null values indicate optional fields but we do not check currently - if(record(index) != null && record(index) != Nil) { - writer.startField(attribute.name, index) - ParquetTypesConverter.consumeType(writer, attribute.dataType, record, index) - writer.endField(attribute.name, index) + while(index < attributes.size) { + // null values indicate optional fields but we do not check currently + if (record(index) != null && record(index) != Nil) { + writer.startField(attributes(index).name, index) + ParquetTypesConverter.consumeType(writer, attributes(index).dataType, record, index) + writer.endField(attributes(index).name, index) } + index = index + 1 } writer.endMessage() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 6d8bbaf1b7232..28d1091fad1af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.parquet -import java.io.File - import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.Job @@ -27,6 +25,8 @@ import parquet.schema.{MessageTypeParser, MessageType} import parquet.hadoop.util.ContextUtil import parquet.hadoop.ParquetWriter +import org.apache.spark.sql.catalyst.util.getTempFilePath + object ParquetTestData { val testSchema = @@ -39,6 +39,16 @@ object ParquetTestData { |optional double mydouble; |}""".stripMargin + // field names for test assertion error messages + val testSchemaFieldNames = Seq( + "myboolean:Boolean", + "mtint:Int", + "mystring:String", + "mylong:Long", + "myfloat:Float", + "mydouble:Double" + ) + val subTestSchema = """ |message myrecord { @@ -47,7 +57,13 @@ object ParquetTestData { |} """.stripMargin - val testFile = new File("/tmp/testParquetFile").getAbsoluteFile + // field names for test assertion error messages + val subTestSchemaFieldNames = Seq( + "myboolean:Boolean", + "mylong:Long" + ) + + val testFile = getTempFilePath("testParquetFile").getCanonicalFile lazy val testData = new ParquetRelation("testData", testFile.toURI.toString) @@ -63,12 +79,12 @@ object ParquetTestData { val writer = new ParquetWriter(path, writeSupport) for(i <- 0 until 15) { val data = new Array[Any](6) - if(i % 3 ==0) { + if (i % 3 == 0) { data.update(0, true) } else { data.update(0, false) } - if(i % 5 == 0) { + if (i % 5 == 0) { data.update(1, 5) } else { data.update(1, null) // optional diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index 02c7c96780638..7bb6789bd33a5 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -46,7 +46,7 @@ log4j.additivity.hive.ql.metadata.Hive=false log4j.logger.hive.ql.metadata.Hive=OFF # Parquet logging -parquet.hadoop.InternalParquetRecordReader=OFF -log4j.logger.parquet.hadoop.InternalParquetRecordReader=OFF -parquet.hadoop.ParquetInputFormat=OFF -log4j.logger.parquet.hadoop.ParquetInputFormat=OFF +parquet.hadoop.InternalParquetRecordReader=WARN +log4j.logger.parquet.hadoop.InternalParquetRecordReader=WARN +parquet.hadoop.ParquetInputFormat=WARN +log4j.logger.parquet.hadoop.ParquetInputFormat=WARN diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala index c095263bbb492..102e97f95b59e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala @@ -20,15 +20,16 @@ package org.apache.spark.sql.parquet import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.TestSqlContext +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.util.getTempFilePath -import parquet.schema.MessageTypeParser - +import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.fs.{Path, FileSystem} + +import parquet.schema.MessageTypeParser import parquet.hadoop.ParquetFileWriter -import org.apache.hadoop.mapreduce.Job import parquet.hadoop.util.ContextUtil -import org.apache.spark.sql.TestSqlContext -import org.apache.spark.sql.catalyst.expressions.Row class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { override def beforeAll() { @@ -45,14 +46,14 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { case (row, index) => { val checkBoolean = if (index % 3 == 0) - (row(0) == true) + row(0) == true else - (row(0) == false) + row(0) == false val checkInt = ((index % 5) != 0) || (row(1) == 5) - val checkString = (row(2) == "abc") - val checkLong = (row(3) == (1L<<33)) - val checkFloat = (row(4) == 2.5F) - val checkDouble = (row(5) == 4.5D) + val checkString = row(2) == "abc" + val checkLong = row(3) == (1L<<33) + val checkFloat = row(4) == 2.5F + val checkDouble = row(5) == 4.5D checkBoolean && checkInt && checkString && checkLong && checkFloat && checkDouble } } @@ -60,18 +61,23 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { } test("Projection of simple Parquet file") { - val scanner = new ParquetTableScan(ParquetTestData.testData.output, ParquetTestData.testData, None)(TestSqlContext.sparkContext) - val projected = scanner.pruneColumns(ParquetTypesConverter.convertToAttributes(MessageTypeParser.parseMessageType(ParquetTestData.subTestSchema))) + val scanner = new ParquetTableScan( + ParquetTestData.testData.output, + ParquetTestData.testData, + None)(TestSqlContext.sparkContext) + val projected = scanner.pruneColumns(ParquetTypesConverter + .convertToAttributes(MessageTypeParser + .parseMessageType(ParquetTestData.subTestSchema))) assert(projected.output.size === 2) val result = projected.execute().collect() val allChecks: Boolean = result.zipWithIndex.forall { case (row, index) => { val checkBoolean = if (index % 3 == 0) - (row(0) == true) + row(0) == true else - (row(0) == false) - val checkLong = (row(1) == (1L<<33)) + row(0) == false + val checkLong = row(1) == (1L<<33) checkBoolean && checkLong && (row.size == 2) } } @@ -80,15 +86,24 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { test("Writing metadata from scratch for table CREATE") { val job = new Job() - val path = new Path("file:///tmp/test/mytesttable") + val path = new Path(getTempFilePath("testtable").getCanonicalFile.toURI.toString) val fs: FileSystem = FileSystem.getLocal(ContextUtil.getConfiguration(job)) - ParquetTypesConverter.writeMetaData(ParquetTestData.testData.output, path, TestSqlContext.sparkContext.hadoopConfiguration) + ParquetTypesConverter.writeMetaData( + ParquetTestData.testData.output, + path, + TestSqlContext.sparkContext.hadoopConfiguration) assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) val metaData = ParquetTypesConverter.readMetaData(path) assert(metaData != null) - ParquetTestData.testData.parquetSchema.checkContains(metaData.getFileMetaData.getSchema) // throws exception if incompatible - metaData.getFileMetaData.getSchema.checkContains(ParquetTestData.testData.parquetSchema) // throws exception if incompatible - fs.delete(path.getParent, true) + ParquetTestData + .testData + .parquetSchema + .checkContains(metaData.getFileMetaData.getSchema) // throws exception if incompatible + metaData + .getFileMetaData + .getSchema + .checkContains(ParquetTestData.testData.parquetSchema) // throws exception if incompatible + fs.delete(path, true) } /** @@ -98,7 +113,10 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { * @return An RDD of Rows. */ private def getRDD(parquetRelation: ParquetRelation): RDD[Row] = { - val scanner = new ParquetTableScan(parquetRelation.output, parquetRelation, None)(TestSqlContext.sparkContext) + val scanner = new ParquetTableScan( + parquetRelation.output, + parquetRelation, + None)(TestSqlContext.sparkContext) scanner.execute } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala index 65c0e767864dc..4e2b01b250791 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala @@ -162,6 +162,9 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { TestTable("src1", "CREATE TABLE src1 (key INT, value STRING)".cmd, s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), + TestTable("src2", + "CREATE TABLE src2 (key INT, value STRING)".cmd, + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv2.txt")}' INTO TABLE src2".cmd), TestTable("dest1", "CREATE TABLE IF NOT EXISTS dest1 (key INT, value STRING)".cmd), TestTable("dest2", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 4d3820902ec53..734cc79580710 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -16,10 +16,6 @@ */ package org.apache.spark.sql.parquet -//package hive -//package execution - -// TODO: move this into the parquet package once it can be import java.io.File @@ -27,16 +23,14 @@ import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.WriteToFile -import org.apache.spark.sql.catalyst.plans.logical.InsertIntoCreatedTable import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.hive.TestHive -//import org.apache.spark.sql.execution.{ParquetTestData, ParquetRelation} class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { - val filename = "file:///tmp/parquettest" + val filename = getTempFilePath("parquettest").getCanonicalFile.toURI.toString // runs a SQL and optionally resolves one Parquet table def runQuery(querystr: String, tableName: Option[String] = None, filename: Option[String] = None): Array[Row] = { @@ -44,7 +38,7 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf val query = TestHive .parseSql(querystr) val finalQuery = - if(tableName.nonEmpty && filename.nonEmpty) + if (tableName.nonEmpty && filename.nonEmpty) resolveParquetTable(tableName.get, filename.get, query) else query @@ -68,17 +62,19 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf * Parquet tables. Once such a thing exists this functionality should be moved there. */ def resolveParquetTable(tableName: String, filename: String, plan: LogicalPlan): LogicalPlan = { + TestHive.loadTestTable("src") // may not be loaded now + TestHive.loadTestTable("src1") + TestHive.loadTestTable("src2") plan.transform { case relation @ UnresolvedRelation(databaseName, name, alias) => - if(name == tableName) + if (name == tableName) ParquetRelation(tableName, filename) else relation case op @ InsertIntoCreatedTable(databaseName, name, child) => - if(name == tableName) { + if (name == tableName) { // note: at this stage the plan is not yet analyzed but Parquet needs to know the schema // and for that we need the child to be resolved - TestHive.loadTestTable("src") // may not be loaded now val relation = ParquetRelation.create( filename, TestHive.analyzer(child), @@ -106,11 +102,11 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf } override def beforeEach() { - (new File(filename)).getAbsoluteFile.delete() + new File(filename).getAbsoluteFile.delete() } override def afterEach() { - (new File(filename)).getAbsoluteFile.delete() + new File(filename).getAbsoluteFile.delete() } test("SELECT on Parquet table") { @@ -121,16 +117,15 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf test("Simple column projection + filter on Parquet table") { val rdd = runQuery("SELECT myboolean, mylong FROM parquet.testsource WHERE myboolean=true") - assert(rdd.size === 5) - assert(rdd.forall(_.getBoolean(0))) + assert(rdd.size === 5, "Filter returned incorrect number of rows") + assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean field value") } test("Converting Hive to Parquet Table via WriteToFile") { storeQuery("SELECT * FROM src", filename) val rddOne = runQuery("SELECT * FROM src").sortBy(_.getInt(0)) val rddTwo = runQuery("SELECT * from ptable", Some("ptable"), Some(filename)).sortBy(_.getInt(0)) - val allsame = (rddOne, rddTwo).zipped.forall { (a,b) => (a,b).zipped.forall { (x,y) => x==y}} - assert(allsame) + compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String")) } test("INSERT OVERWRITE TABLE Parquet table") { @@ -139,15 +134,28 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf runQuery("INSERT OVERWRITE TABLE ptable SELECT * FROM parquet.testsource", Some("ptable"), Some(filename)) val rddCopy = runQuery("SELECT * FROM ptable", Some("ptable"), Some(filename)) val rddOrig = runQuery("SELECT * FROM parquet.testsource") - val allsame = (rddCopy, rddOrig).zipped.forall { (a,b) => (a,b).zipped.forall { (x,y) => x==y } } - assert(allsame) + compareRDDs(rddOrig, rddCopy, "parquet.testsource", ParquetTestData.testSchemaFieldNames) } test("CREATE TABLE AS Parquet table") { runQuery("CREATE TABLE ptable AS SELECT * FROM src", Some("ptable"), Some(filename)) - val rddCopy = runQuery("SELECT * FROM ptable", Some("ptable"), Some(filename)).sortBy(_.getInt(0)) + val rddCopy = runQuery("SELECT * FROM ptable", Some("ptable"), Some(filename)) + .sortBy[Int](_.apply(0) match { + case x: Int => x + case _ => 0 + }) val rddOrig = runQuery("SELECT * FROM src").sortBy(_.getInt(0)) - val allsame = (rddCopy, rddOrig).zipped.forall { (a,b) => (a,b).zipped.forall { (x,y) => x==y } } - assert(allsame) + compareRDDs(rddOrig, rddCopy, "src (Hive)", Seq("key:Int", "value:String")) + } + + private def compareRDDs(rddOne: Array[Row], rddTwo: Array[Row], tableName: String, fieldNames: Seq[String]) { + var counter = 0 + (rddOne, rddTwo).zipped.foreach { + (a,b) => (a,b).zipped.toArray.zipWithIndex.foreach { + case ((value_1, value_2), index) => + assert(value_1 === value_2, s"table $tableName row ${counter} field ${fieldNames(index)} don't match") + } + counter = counter + 1 + } } } From 5bacdc0e5c18bc6a4aee6bc2da8ac8d2a29751a0 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 9 Mar 2014 21:35:39 +0100 Subject: [PATCH 751/778] Moving towards mutable rows inside ParquetRowSupport --- .../sql/catalyst/expressions/Projection.scala | 3 ++ .../spark/sql/catalyst/expressions/Row.scala | 15 ++++++-- .../spark/sql/parquet/ParquetRelation.scala | 2 +- .../sql/parquet/ParquetTableSupport.scala | 34 +++++++++---------- .../spark/sql/parquet/ParquetTestData.scala | 4 ++- .../spark/sql/parquet/ParquetQueryTests.scala | 17 ++++++---- .../spark/sql/parquet/ParquetQuerySuite.scala | 2 ++ 7 files changed, 49 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index ae04be2fb0d1c..9e4094611431e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -91,6 +91,9 @@ class JoinedRow extends Row { def getByte(i: Int): Byte = if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + def getBinary(i: Int): Array[Byte] = + if (i < row1.size) row1.getBinary(i) else row2.getBinary(i - row1.size) + def getFloat(i: Int): Float = if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 6d12e6c606373..c921d49491c68 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -40,6 +40,7 @@ trait Row extends Seq[Any] with Serializable { def getBoolean(i: Int): Boolean def getShort(i: Int): Short def getByte(i: Int): Byte + def getBinary(i: Int): Array[Byte] def getString(i: Int): String override def toString() = @@ -63,7 +64,9 @@ trait MutableRow extends Row { def setBoolean(ordinal: Int, value: Boolean) def setShort(ordinal: Int, value: Short) def setByte(ordinal: Int, value: Byte) - def setFloat(ordinal: Int, value: Byte) + def setBinary(ordinal: Int, value: Array[Byte]) + def setFloat(ordinal: Int, value: Float) + def setString(ordinal: Int, value: String) /** * EXPERIMENTAL @@ -92,6 +95,7 @@ object EmptyRow extends Row { def getBoolean(i: Int): Boolean = throw new UnsupportedOperationException def getShort(i: Int): Short = throw new UnsupportedOperationException def getByte(i: Int): Byte = throw new UnsupportedOperationException + def getBinary(i: Int): Array[Byte] = throw new UnsupportedOperationException def getString(i: Int): String = throw new UnsupportedOperationException def copy() = this @@ -151,6 +155,11 @@ class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { values(i).asInstanceOf[Byte] } + def getBinary(i: Int): Array[Byte] = { + if (values(i) == null) sys.error("Failed to check null bit for primitive byte array value.") + values(i).asInstanceOf[Array[Byte]] + } + def getString(i: Int): String = { if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.") values(i).asInstanceOf[String] @@ -167,10 +176,12 @@ class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { override def setBoolean(ordinal: Int,value: Boolean): Unit = { values(ordinal) = value } override def setByte(ordinal: Int,value: Byte): Unit = { values(ordinal) = value } + override def setBinary(ordinal: Int,value: Array[Byte]): Unit = { values(ordinal) = value } override def setDouble(ordinal: Int,value: Double): Unit = { values(ordinal) = value } - override def setFloat(ordinal: Int,value: Byte): Unit = { values(ordinal) = value } + override def setFloat(ordinal: Int,value: Float): Unit = { values(ordinal) = value } override def setInt(ordinal: Int,value: Int): Unit = { values(ordinal) = value } override def setLong(ordinal: Int,value: Long): Unit = { values(ordinal) = value } + override def setString(ordinal: Int,value: String): Unit = { values(ordinal) = value } override def setNullAt(i: Int): Unit = { values(i) = null } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 67d70f0af2329..baaba0096c02d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -119,7 +119,7 @@ case class ParquetRelation(val tableName: String, val path: String) extends Base object ParquetRelation { // The element type for the RDDs that this relation maps to. - type RowType = org.apache.spark.sql.catalyst.expressions.GenericRow + type RowType = org.apache.spark.sql.catalyst.expressions.GenericMutableRow /** * Creates a new ParquetRelation and underlying Parquetfile for the given diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index aae2b07aed6a6..62187a4c1b4ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -41,7 +41,8 @@ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterial def this(parquetSchema: MessageType) = this(new CatalystGroupConverter(ParquetTypesConverter.convertToAttributes(parquetSchema))) - override def getCurrentRecord: Row = root.getCurrentRecord + // Note: return a copy of the Row + override def getCurrentRecord: Row = root.getCurrentRecord.copy() override def getRootConverter: GroupConverter = root } @@ -144,9 +145,7 @@ object RowWriteSupport { */ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { type RowType = ParquetRelation.RowType - var current: RowType = new RowType(Array[Any]()) - // initialization may not strictly be required - val currentData: Array[Any] = new Array[Any](schema.length) + private val current: RowType = new RowType(schema.length) val converters: Array[Converter] = schema.map { a => a.dataType match { @@ -161,18 +160,17 @@ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) - def getCurrentRecord: RowType = current + private[parquet] def getCurrentRecord: RowType = current override def start(): Unit = { - for (i <- 0 until schema.length) { - currentData.update(i, Nil) + var i = 0 + while (i < schema.length) { + current.setNullAt(i) + i = i + 1 } } - override def end(): Unit = { - // TODO: think about reusing the row versus reusing the underlying array - current = new RowType(currentData.clone()) - } + override def end(): Unit = {} } /** @@ -186,22 +184,22 @@ class CatalystPrimitiveConverter( fieldIndex: Int) extends PrimitiveConverter { // TODO: consider refactoring these together with ParquetTypesConverter override def addBinary(value: Binary): Unit = - parent.currentData.update(fieldIndex, value.getBytes) + parent.getCurrentRecord.setBinary(fieldIndex, value.getBytes) override def addBoolean(value: Boolean): Unit = - parent.currentData.update(fieldIndex, value) + parent.getCurrentRecord.setBoolean(fieldIndex, value) override def addDouble(value: Double): Unit = - parent.currentData.update(fieldIndex, value) + parent.getCurrentRecord.setDouble(fieldIndex, value) override def addFloat(value: Float): Unit = - parent.currentData.update(fieldIndex, value) + parent.getCurrentRecord.setFloat(fieldIndex, value) override def addInt(value: Int): Unit = - parent.currentData.update(fieldIndex, value) + parent.getCurrentRecord.setInt(fieldIndex, value) override def addLong(value: Long): Unit = - parent.currentData.update(fieldIndex, value) + parent.getCurrentRecord.setLong(fieldIndex, value) } /** @@ -215,6 +213,6 @@ class CatalystPrimitiveStringConverter( parent: CatalystGroupConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { override def addBinary(value: Binary): Unit = - parent.currentData.update(fieldIndex, value.toStringUsingUTF8) + parent.getCurrentRecord.setString(fieldIndex, value.toStringUsingUTF8) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 28d1091fad1af..2686bc2551192 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -26,6 +26,8 @@ import parquet.hadoop.util.ContextUtil import parquet.hadoop.ParquetWriter import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.catalyst.expressions.GenericRow +import java.nio.charset.Charset object ParquetTestData { @@ -93,7 +95,7 @@ object ParquetTestData { data.update(3, 1L<<33) data.update(4, 2.5F) data.update(5, 4.5D) - writer.write(new ParquetRelation.RowType(data.toArray)) + writer.write(new GenericRow(data.toArray)) } writer.close() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala index 102e97f95b59e..82208dcde5f5d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala @@ -42,22 +42,26 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { test("Import of simple Parquet file") { val result = getRDD(ParquetTestData.testData).collect() - val allChecks: Boolean = result.zipWithIndex.forall { + result.zipWithIndex.foreach { case (row, index) => { val checkBoolean = if (index % 3 == 0) row(0) == true else row(0) == false + assert(checkBoolean) val checkInt = ((index % 5) != 0) || (row(1) == 5) + assert(checkInt) val checkString = row(2) == "abc" + assert(checkString) val checkLong = row(3) == (1L<<33) + assert(checkLong) val checkFloat = row(4) == 2.5F + assert(checkFloat) val checkDouble = row(5) == 4.5D - checkBoolean && checkInt && checkString && checkLong && checkFloat && checkDouble + assert(checkDouble) } } - assert(allChecks) } test("Projection of simple Parquet file") { @@ -70,18 +74,19 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { .parseMessageType(ParquetTestData.subTestSchema))) assert(projected.output.size === 2) val result = projected.execute().collect() - val allChecks: Boolean = result.zipWithIndex.forall { + result.zipWithIndex.foreach { case (row, index) => { val checkBoolean = if (index % 3 == 0) row(0) == true else row(0) == false + assert(checkBoolean) val checkLong = row(1) == (1L<<33) - checkBoolean && checkLong && (row.size == 2) + assert(checkLong) + assert(row.size === 2, "number of columns in projection is incorrect") } } - assert(allChecks) } test("Writing metadata from scratch for table CREATE") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 734cc79580710..f2697ceb987e9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -152,6 +152,8 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf var counter = 0 (rddOne, rddTwo).zipped.foreach { (a,b) => (a,b).zipped.toArray.zipWithIndex.foreach { + case ((value_1:Array[Byte], value_2:Array[Byte]), index) => + assert(new String(value_1) === new String(value_2), s"table $tableName row ${counter} field ${fieldNames(index)} don't match") case ((value_1, value_2), index) => assert(value_1 === value_2, s"table $tableName row ${counter} field ${fieldNames(index)} don't match") } From 7ca4b4e34d466fd64243b80300fab28af09936e9 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Tue, 11 Mar 2014 18:56:40 +0100 Subject: [PATCH 752/778] Improving checks in Parquet tests --- .../sql/parquet/ParquetTableOperations.scala | 6 ++-- .../sql/parquet/ParquetTableSupport.scala | 8 ++--- .../spark/sql/parquet/ParquetTestData.scala | 2 +- .../spark/sql/parquet/ParquetQueryTests.scala | 29 +++++++------------ .../org/apache/spark/sql/hive/TestShark.scala | 3 -- .../spark/sql/parquet/ParquetQuerySuite.scala | 4 +-- 6 files changed, 19 insertions(+), 33 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 0c50fff10c998..61121103cbbfa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -23,7 +23,6 @@ import parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat} import parquet.hadoop.util.ContextUtil import org.apache.spark.rdd.RDD -//import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.{TaskContext, SerializableWritable, SparkContext} import org.apache.spark.sql.catalyst.expressions.{Row, Attribute, Expression} import org.apache.spark.sql.execution.{SparkPlan, UnaryNode, LeafNode} @@ -34,7 +33,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import java.io.IOException -import org.apache.spark.util.MutablePair import java.text.SimpleDateFormat import java.util.Date @@ -70,7 +68,7 @@ case class ParquetTableScan( classOf[ParquetInputFormat[Row]], classOf[Void], classOf[Row], conf) - .map(_._2) + .map(_._2) } /** @@ -160,7 +158,7 @@ case class InsertIntoParquetTable( override def output = child.output // based on ``saveAsNewAPIHadoopFile`` in [[PairRDDFunctions]] - // TODO: Maybe PairRDDFunctions should use MutablePair instead of Tuple2? + // TODO: Maybe PairRDDFunctions should use Product2 instead of Tuple2? // .. then we could use the default one and could use [[MutablePair]] // instead of ``Tuple2`` private def saveAsHadoopFile( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 62187a4c1b4ba..0fadcfa1cca4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -143,9 +143,9 @@ object RowWriteSupport { * * @param schema The corresponding Shark schema in the form of a list of attributes. */ -class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { - type RowType = ParquetRelation.RowType - private val current: RowType = new RowType(schema.length) +class CatalystGroupConverter(schema: Seq[Attribute], protected[parquet] val current: ParquetRelation.RowType) extends GroupConverter { + + def this(schema: Seq[Attribute]) = this(schema, new ParquetRelation.RowType(schema.length)) val converters: Array[Converter] = schema.map { a => a.dataType match { @@ -160,7 +160,7 @@ class CatalystGroupConverter(schema: Seq[Attribute]) extends GroupConverter { override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) - private[parquet] def getCurrentRecord: RowType = current + private[parquet] def getCurrentRecord: ParquetRelation.RowType = current override def start(): Unit = { var i = 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 2686bc2551192..bbe409fb9c7e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -92,7 +92,7 @@ object ParquetTestData { data.update(1, null) // optional } data.update(2, "abc") - data.update(3, 1L<<33) + data.update(3, i.toLong << 33) data.update(4, 2.5F) data.update(5, 4.5D) writer.write(new GenericRow(data.toArray)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala index 82208dcde5f5d..876c94b0f6693 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala @@ -42,6 +42,7 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { test("Import of simple Parquet file") { val result = getRDD(ParquetTestData.testData).collect() + assert(result.size === 15) result.zipWithIndex.foreach { case (row, index) => { val checkBoolean = @@ -49,17 +50,12 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { row(0) == true else row(0) == false - assert(checkBoolean) - val checkInt = ((index % 5) != 0) || (row(1) == 5) - assert(checkInt) - val checkString = row(2) == "abc" - assert(checkString) - val checkLong = row(3) == (1L<<33) - assert(checkLong) - val checkFloat = row(4) == 2.5F - assert(checkFloat) - val checkDouble = row(5) == 4.5D - assert(checkDouble) + assert(checkBoolean === true, s"boolean field value in line $index did not match") + if (index % 5 == 0) assert(row(1) === 5, s"int field value in line $index did not match") + assert(row(2) === "abc", s"string field value in line $index did not match") + assert(row(3) === (index.toLong << 33), s"long value in line $index did not match") + assert(row(4) === 2.5F, s"float field value in line $index did not match") + assert(row(5) === 4.5D, s"double field value in line $index did not match") } } } @@ -76,15 +72,12 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { val result = projected.execute().collect() result.zipWithIndex.foreach { case (row, index) => { - val checkBoolean = if (index % 3 == 0) - row(0) == true + assert(row(0) === true, s"boolean field value in line $index did not match (every third row)") else - row(0) == false - assert(checkBoolean) - val checkLong = row(1) == (1L<<33) - assert(checkLong) - assert(row.size === 2, "number of columns in projection is incorrect") + assert(row(0) === false, s"boolean field value in line $index did not match") + assert(row(1) === (index.toLong << 33), s"long field value in line $index did not match") + assert(row.size === 2, s"number of columns in projection in line $index is incorrect") } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala index 4e2b01b250791..65c0e767864dc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala @@ -162,9 +162,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { TestTable("src1", "CREATE TABLE src1 (key INT, value STRING)".cmd, s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), - TestTable("src2", - "CREATE TABLE src2 (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv2.txt")}' INTO TABLE src2".cmd), TestTable("dest1", "CREATE TABLE IF NOT EXISTS dest1 (key INT, value STRING)".cmd), TestTable("dest2", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index f2697ceb987e9..7e4a7e1fa600c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -63,8 +63,6 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf */ def resolveParquetTable(tableName: String, filename: String, plan: LogicalPlan): LogicalPlan = { TestHive.loadTestTable("src") // may not be loaded now - TestHive.loadTestTable("src1") - TestHive.loadTestTable("src2") plan.transform { case relation @ UnresolvedRelation(databaseName, name, alias) => if (name == tableName) @@ -155,7 +153,7 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf case ((value_1:Array[Byte], value_2:Array[Byte]), index) => assert(new String(value_1) === new String(value_2), s"table $tableName row ${counter} field ${fieldNames(index)} don't match") case ((value_1, value_2), index) => - assert(value_1 === value_2, s"table $tableName row ${counter} field ${fieldNames(index)} don't match") + assert(value_1 === value_2, s"table $tableName row $counter field ${fieldNames(index)} don't match") } counter = counter + 1 } From aeaef544dda49dae87385f8bdd31e2a61719dfd2 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Tue, 11 Mar 2014 19:33:00 +0100 Subject: [PATCH 753/778] Removing unnecessary Row copying and reverting some changes to MutableRow --- .../spark/sql/catalyst/expressions/Projection.scala | 3 --- .../apache/spark/sql/catalyst/expressions/Row.scala | 11 +---------- .../spark/sql/parquet/ParquetTableSupport.scala | 6 +++--- .../apache/spark/sql/parquet/ParquetQueryTests.scala | 9 +++++++-- 4 files changed, 11 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 9e4094611431e..ae04be2fb0d1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -91,9 +91,6 @@ class JoinedRow extends Row { def getByte(i: Int): Byte = if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) - def getBinary(i: Int): Array[Byte] = - if (i < row1.size) row1.getBinary(i) else row2.getBinary(i - row1.size) - def getFloat(i: Int): Float = if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index c921d49491c68..352967546827d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -40,7 +40,6 @@ trait Row extends Seq[Any] with Serializable { def getBoolean(i: Int): Boolean def getShort(i: Int): Short def getByte(i: Int): Byte - def getBinary(i: Int): Array[Byte] def getString(i: Int): String override def toString() = @@ -64,7 +63,6 @@ trait MutableRow extends Row { def setBoolean(ordinal: Int, value: Boolean) def setShort(ordinal: Int, value: Short) def setByte(ordinal: Int, value: Byte) - def setBinary(ordinal: Int, value: Array[Byte]) def setFloat(ordinal: Int, value: Float) def setString(ordinal: Int, value: String) @@ -95,7 +93,6 @@ object EmptyRow extends Row { def getBoolean(i: Int): Boolean = throw new UnsupportedOperationException def getShort(i: Int): Short = throw new UnsupportedOperationException def getByte(i: Int): Byte = throw new UnsupportedOperationException - def getBinary(i: Int): Array[Byte] = throw new UnsupportedOperationException def getString(i: Int): String = throw new UnsupportedOperationException def copy() = this @@ -155,13 +152,8 @@ class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { values(i).asInstanceOf[Byte] } - def getBinary(i: Int): Array[Byte] = { - if (values(i) == null) sys.error("Failed to check null bit for primitive byte array value.") - values(i).asInstanceOf[Array[Byte]] - } - def getString(i: Int): String = { - if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.") + if (values(i) == null) sys.error("Failed to check null bit for primitive String value.") values(i).asInstanceOf[String] } @@ -176,7 +168,6 @@ class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { override def setBoolean(ordinal: Int,value: Boolean): Unit = { values(ordinal) = value } override def setByte(ordinal: Int,value: Byte): Unit = { values(ordinal) = value } - override def setBinary(ordinal: Int,value: Array[Byte]): Unit = { values(ordinal) = value } override def setDouble(ordinal: Int,value: Double): Unit = { values(ordinal) = value } override def setFloat(ordinal: Int,value: Float): Unit = { values(ordinal) = value } override def setInt(ordinal: Int,value: Int): Unit = { values(ordinal) = value } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 0fadcfa1cca4e..db35b6a42976a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -41,8 +41,7 @@ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterial def this(parquetSchema: MessageType) = this(new CatalystGroupConverter(ParquetTypesConverter.convertToAttributes(parquetSchema))) - // Note: return a copy of the Row - override def getCurrentRecord: Row = root.getCurrentRecord.copy() + override def getCurrentRecord: Row = root.getCurrentRecord override def getRootConverter: GroupConverter = root } @@ -184,7 +183,8 @@ class CatalystPrimitiveConverter( fieldIndex: Int) extends PrimitiveConverter { // TODO: consider refactoring these together with ParquetTypesConverter override def addBinary(value: Binary): Unit = - parent.getCurrentRecord.setBinary(fieldIndex, value.getBytes) + // TODO: fix this once a setBinary will become available in MutableRow + parent.getCurrentRecord.setByte(fieldIndex, value.getBytes.apply(0)) override def addBoolean(value: Boolean): Unit = parent.getCurrentRecord.setBoolean(fieldIndex, value) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala index 876c94b0f6693..63cc02c7fe988 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala @@ -69,7 +69,10 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { .convertToAttributes(MessageTypeParser .parseMessageType(ParquetTestData.subTestSchema))) assert(projected.output.size === 2) - val result = projected.execute().collect() + val result = projected + .execute() + .map(_.copy()) + .collect() result.zipWithIndex.foreach { case (row, index) => { if (index % 3 == 0) @@ -115,7 +118,9 @@ class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { parquetRelation.output, parquetRelation, None)(TestSqlContext.sparkContext) - scanner.execute + scanner + .execute + .map(_.copy()) } } From 7386a9f386298d8428055cfae5784f78cac44ada Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 11 Mar 2014 11:34:45 -0700 Subject: [PATCH 754/778] Initial example programs using spark sql. --- examples/src/main/resources/kv1.txt | 500 ++++++++++++++++++ .../spark/sql/examples/HiveFromSpark.scala | 48 ++ .../spark/sql/examples/RDDRelation.scala | 38 ++ project/SparkBuild.scala | 4 +- .../apache/spark/sql/catalyst/SqlParser.scala | 311 +++++++++++ .../spark/sql/catalyst/analysis/Catalog.scala | 30 +- .../spark/sql/catalyst/dsl/package.scala | 91 ++-- .../apache/spark/sql/SparkSqlContext.scala | 16 +- .../spark/sql/execution/SharkPlan.scala | 10 + .../spark/sql/execution/SparkStrategies.scala | 1 + .../spark/sql/execution/basicOperators.scala | 32 ++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 6 + .../spark/sql/parquet/ParquetQuerySuite.scala | 2 +- 13 files changed, 1036 insertions(+), 53 deletions(-) create mode 100644 examples/src/main/resources/kv1.txt create mode 100644 examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala create mode 100644 examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala diff --git a/examples/src/main/resources/kv1.txt b/examples/src/main/resources/kv1.txt new file mode 100644 index 0000000000000..9825414ecf8f2 --- /dev/null +++ b/examples/src/main/resources/kv1.txt @@ -0,0 +1,500 @@ +238val_238 +86val_86 +311val_311 +27val_27 +165val_165 +409val_409 +255val_255 +278val_278 +98val_98 +484val_484 +265val_265 +193val_193 +401val_401 +150val_150 +273val_273 +224val_224 +369val_369 +66val_66 +128val_128 +213val_213 +146val_146 +406val_406 +429val_429 +374val_374 +152val_152 +469val_469 +145val_145 +495val_495 +37val_37 +327val_327 +281val_281 +277val_277 +209val_209 +15val_15 +82val_82 +403val_403 +166val_166 +417val_417 +430val_430 +252val_252 +292val_292 +219val_219 +287val_287 +153val_153 +193val_193 +338val_338 +446val_446 +459val_459 +394val_394 +237val_237 +482val_482 +174val_174 +413val_413 +494val_494 +207val_207 +199val_199 +466val_466 +208val_208 +174val_174 +399val_399 +396val_396 +247val_247 +417val_417 +489val_489 +162val_162 +377val_377 +397val_397 +309val_309 +365val_365 +266val_266 +439val_439 +342val_342 +367val_367 +325val_325 +167val_167 +195val_195 +475val_475 +17val_17 +113val_113 +155val_155 +203val_203 +339val_339 +0val_0 +455val_455 +128val_128 +311val_311 +316val_316 +57val_57 +302val_302 +205val_205 +149val_149 +438val_438 +345val_345 +129val_129 +170val_170 +20val_20 +489val_489 +157val_157 +378val_378 +221val_221 +92val_92 +111val_111 +47val_47 +72val_72 +4val_4 +280val_280 +35val_35 +427val_427 +277val_277 +208val_208 +356val_356 +399val_399 +169val_169 +382val_382 +498val_498 +125val_125 +386val_386 +437val_437 +469val_469 +192val_192 +286val_286 +187val_187 +176val_176 +54val_54 +459val_459 +51val_51 +138val_138 +103val_103 +239val_239 +213val_213 +216val_216 +430val_430 +278val_278 +176val_176 +289val_289 +221val_221 +65val_65 +318val_318 +332val_332 +311val_311 +275val_275 +137val_137 +241val_241 +83val_83 +333val_333 +180val_180 +284val_284 +12val_12 +230val_230 +181val_181 +67val_67 +260val_260 +404val_404 +384val_384 +489val_489 +353val_353 +373val_373 +272val_272 +138val_138 +217val_217 +84val_84 +348val_348 +466val_466 +58val_58 +8val_8 +411val_411 +230val_230 +208val_208 +348val_348 +24val_24 +463val_463 +431val_431 +179val_179 +172val_172 +42val_42 +129val_129 +158val_158 +119val_119 +496val_496 +0val_0 +322val_322 +197val_197 +468val_468 +393val_393 +454val_454 +100val_100 +298val_298 +199val_199 +191val_191 +418val_418 +96val_96 +26val_26 +165val_165 +327val_327 +230val_230 +205val_205 +120val_120 +131val_131 +51val_51 +404val_404 +43val_43 +436val_436 +156val_156 +469val_469 +468val_468 +308val_308 +95val_95 +196val_196 +288val_288 +481val_481 +457val_457 +98val_98 +282val_282 +197val_197 +187val_187 +318val_318 +318val_318 +409val_409 +470val_470 +137val_137 +369val_369 +316val_316 +169val_169 +413val_413 +85val_85 +77val_77 +0val_0 +490val_490 +87val_87 +364val_364 +179val_179 +118val_118 +134val_134 +395val_395 +282val_282 +138val_138 +238val_238 +419val_419 +15val_15 +118val_118 +72val_72 +90val_90 +307val_307 +19val_19 +435val_435 +10val_10 +277val_277 +273val_273 +306val_306 +224val_224 +309val_309 +389val_389 +327val_327 +242val_242 +369val_369 +392val_392 +272val_272 +331val_331 +401val_401 +242val_242 +452val_452 +177val_177 +226val_226 +5val_5 +497val_497 +402val_402 +396val_396 +317val_317 +395val_395 +58val_58 +35val_35 +336val_336 +95val_95 +11val_11 +168val_168 +34val_34 +229val_229 +233val_233 +143val_143 +472val_472 +322val_322 +498val_498 +160val_160 +195val_195 +42val_42 +321val_321 +430val_430 +119val_119 +489val_489 +458val_458 +78val_78 +76val_76 +41val_41 +223val_223 +492val_492 +149val_149 +449val_449 +218val_218 +228val_228 +138val_138 +453val_453 +30val_30 +209val_209 +64val_64 +468val_468 +76val_76 +74val_74 +342val_342 +69val_69 +230val_230 +33val_33 +368val_368 +103val_103 +296val_296 +113val_113 +216val_216 +367val_367 +344val_344 +167val_167 +274val_274 +219val_219 +239val_239 +485val_485 +116val_116 +223val_223 +256val_256 +263val_263 +70val_70 +487val_487 +480val_480 +401val_401 +288val_288 +191val_191 +5val_5 +244val_244 +438val_438 +128val_128 +467val_467 +432val_432 +202val_202 +316val_316 +229val_229 +469val_469 +463val_463 +280val_280 +2val_2 +35val_35 +283val_283 +331val_331 +235val_235 +80val_80 +44val_44 +193val_193 +321val_321 +335val_335 +104val_104 +466val_466 +366val_366 +175val_175 +403val_403 +483val_483 +53val_53 +105val_105 +257val_257 +406val_406 +409val_409 +190val_190 +406val_406 +401val_401 +114val_114 +258val_258 +90val_90 +203val_203 +262val_262 +348val_348 +424val_424 +12val_12 +396val_396 +201val_201 +217val_217 +164val_164 +431val_431 +454val_454 +478val_478 +298val_298 +125val_125 +431val_431 +164val_164 +424val_424 +187val_187 +382val_382 +5val_5 +70val_70 +397val_397 +480val_480 +291val_291 +24val_24 +351val_351 +255val_255 +104val_104 +70val_70 +163val_163 +438val_438 +119val_119 +414val_414 +200val_200 +491val_491 +237val_237 +439val_439 +360val_360 +248val_248 +479val_479 +305val_305 +417val_417 +199val_199 +444val_444 +120val_120 +429val_429 +169val_169 +443val_443 +323val_323 +325val_325 +277val_277 +230val_230 +478val_478 +178val_178 +468val_468 +310val_310 +317val_317 +333val_333 +493val_493 +460val_460 +207val_207 +249val_249 +265val_265 +480val_480 +83val_83 +136val_136 +353val_353 +172val_172 +214val_214 +462val_462 +233val_233 +406val_406 +133val_133 +175val_175 +189val_189 +454val_454 +375val_375 +401val_401 +421val_421 +407val_407 +384val_384 +256val_256 +26val_26 +134val_134 +67val_67 +384val_384 +379val_379 +18val_18 +462val_462 +492val_492 +100val_100 +298val_298 +9val_9 +341val_341 +498val_498 +146val_146 +458val_458 +362val_362 +186val_186 +285val_285 +348val_348 +167val_167 +18val_18 +273val_273 +183val_183 +281val_281 +344val_344 +97val_97 +469val_469 +315val_315 +84val_84 +28val_28 +37val_37 +448val_448 +152val_152 +348val_348 +307val_307 +194val_194 +414val_414 +477val_477 +222val_222 +126val_126 +90val_90 +169val_169 +403val_403 +400val_400 +200val_200 +97val_97 diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala new file mode 100644 index 0000000000000..5f9af86267eb6 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala @@ -0,0 +1,48 @@ +package org.apache.spark.sql.hive.examples + +import org.apache.spark.SparkContext + +import org.apache.spark.sql._ +import org.apache.spark.sql.hive.LocalHiveContext + +object HiveFromSpark { + case class Record(key: Int, value: String) + + def main(args: Array[String]) { + val sc = new SparkContext("local", "HiveFromSpark") + + // A local hive context creates an instance of the Hive Metastore in process, storing the + // the warehouse data in the current directory. This location can be overridden by + // specifying a second parameter to the constructor. + val hiveContext = new LocalHiveContext(sc) + import hiveContext._ + + sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src") + + // Queries are expressed in HiveQL + println("Result of 'SELECT *': ") + sql("SELECT * FROM src").collect.foreach(println) + + // Aggregation queries are also supported. + val count = sql("SELECT COUNT(*) FROM src").collect().head.getInt(0) + println(s"COUNT(*): $count") + + // The results of SQL queries are themselves RDDs and support all normal RDD functions. The + // items in the RDD are of type Row, which allows you to access each column by ordinal. + val rddFromSql = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") + + println("Result of RDD.map:") + val rddAsStrings = rddFromSql.map { + case Row(key: Int, value: String) => s"Key: $key, Value: $value" + } + + // You can also register RDDs as temporary tables within a HiveContext. + val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) + rdd.registerAsTable("records") + + // Queries can then join RDD data with data stored in Hive. + println("Result of SELECT *:") + sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) + } +} diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala new file mode 100644 index 0000000000000..aa1aca7bcc605 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala @@ -0,0 +1,38 @@ +package org.apache.spark.sql.examples + +import org.apache.spark.SparkContext +import org.apache.spark.sql.SparkSqlContext + +// One method for defining the schema of an RDD is to make a case class with the desired column +// names and types. +case class Record(key: Int, value: String) + +object RDDRelation { + def main(args: Array[String]) { + val sc = new SparkContext("local", "RDDRelation") + val sqlContext = new SparkSqlContext(sc) + + // Importing the SQL contexts give access to all the SQL functions and implicit conversions. + import sqlContext._ + + val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) + // Any RDD containing case classes can be registered as a table. The schema of the table is + // automatically inferred using scala reflection. + rdd.registerAsTable("records") + + // Once tables have been registered, you can run SQL queries over them. + println("Result of SELECT *:") + sql("SELECT * FROM records").collect().foreach(println) + + // Aggregation queries are also supported. + val count = sql("SELECT COUNT(*) FROM records").collect().head.getInt(0) + println(s"COUNT(*): $count") + + // The results of SQL queries are themselves RDDs and support all normal RDD functions. The + // items in the RDD are of type Row, which allows you to access each column by ordinal. + val rddFromSql = sql("SELECT key, value FROM records WHERE key < 10") + + println("Result of RDD.map:") + rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect.foreach(println) + } +} \ No newline at end of file diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 97f3c086c9e04..369452d248a9d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -60,7 +60,7 @@ object SparkBuild extends Build { lazy val graphx = Project("graphx", file("graphx"), settings = graphxSettings) dependsOn(core) - lazy val catalyst = Project("catalyst", file("sql/catalyst"), settings = catalystSettings) + lazy val catalyst = Project("catalyst", file("sql/catalyst"), settings = catalystSettings) dependsOn(core) lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core, catalyst) @@ -122,7 +122,7 @@ object SparkBuild extends Build { lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) + .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter, hive) dependsOn(allExternal: _*) // Everything except assembly, tools and examples belong to packageProjects lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql, hive) ++ maybeYarnRef diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala new file mode 100644 index 0000000000000..6cadf2d7244a8 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -0,0 +1,311 @@ +package org.apache.spark.sql.catalyst + +import scala.util.matching.Regex +import scala.util.parsing.combinator._ +import scala.util.parsing.input.CharArrayReader.EofCh +import lexical._ +import syntactical._ +import token._ + +import analysis._ +import expressions._ +import plans._ +import plans.logical._ +import types._ + +/** + * A very simple SQL parser. Based loosly on: + * https://github.com/stephentu/scala-sql-parser/blob/master/src/main/scala/parser.scala + * + * Limitations: + * - Only supports a very limited subset of SQL. + * - Keywords must be capital. + * + * This is currently included mostly for illustrative purposes. Users wanting more complete support + * for a SQL like language should checkout the HiveQL support in the sql/hive subproject. + */ +class SqlParser extends StandardTokenParsers { + + def apply(input: String): LogicalPlan = { + phrase(query)(new lexical.Scanner(input)) match { + case Success(r, x) => r + case x => sys.error(x.toString) + } + } + + protected case class Keyword(str: String) + protected implicit def asParser(k: Keyword): Parser[String] = k.str + + protected class SqlLexical extends StdLexical { + case class FloatLit(chars: String) extends Token { + override def toString = chars + } + override lazy val token: Parser[Token] = ( + identChar ~ rep( identChar | digit ) ^^ + { case first ~ rest => processIdent(first :: rest mkString "") } + | rep1(digit) ~ opt('.' ~> rep(digit)) ^^ { + case i ~ None => NumericLit(i mkString "") + case i ~ Some(d) => FloatLit(i.mkString("") + "." + d.mkString("")) + } + | '\'' ~ rep( chrExcept('\'', '\n', EofCh) ) ~ '\'' ^^ + { case '\'' ~ chars ~ '\'' => StringLit(chars mkString "") } + | '\"' ~ rep( chrExcept('\"', '\n', EofCh) ) ~ '\"' ^^ + { case '\"' ~ chars ~ '\"' => StringLit(chars mkString "") } + | EofCh ^^^ EOF + | '\'' ~> failure("unclosed string literal") + | '\"' ~> failure("unclosed string literal") + | delim + | failure("illegal character") + ) + + override def identChar = letter | elem('.') | elem('_') + + override def whitespace: Parser[Any] = rep( + whitespaceChar + | '/' ~ '*' ~ comment + | '/' ~ '/' ~ rep( chrExcept(EofCh, '\n') ) + | '#' ~ rep( chrExcept(EofCh, '\n') ) + | '-' ~ '-' ~ rep( chrExcept(EofCh, '\n') ) + | '/' ~ '*' ~ failure("unclosed comment") + ) + } + + override val lexical = new SqlLexical + + protected val ALL = Keyword("ALL") + protected val AND = Keyword("AND") + protected val AS = Keyword("AS") + protected val ASC = Keyword("ASC") + protected val AVG = Keyword("AVG") + protected val BY = Keyword("BY") + protected val CAST = Keyword("CAST") + protected val COUNT = Keyword("COUNT") + protected val DESC = Keyword("DESC") + protected val DISTINCT = Keyword("DISTINCT") + protected val FALSE = Keyword("FALSE") + protected val FIRST = Keyword("FIRST") + protected val FROM = Keyword("FROM") + protected val FULL = Keyword("FULL") + protected val GROUP = Keyword("GROUP") + protected val HAVING = Keyword("HAVING") + protected val IF = Keyword("IF") + protected val IN = Keyword("IN") + protected val INNER = Keyword("INNER") + protected val IS = Keyword("IS") + protected val JOIN = Keyword("JOIN") + protected val LEFT = Keyword("LEFT") + protected val LIMIT = Keyword("LIMIT") + protected val NOT = Keyword("NOT") + protected val NULL = Keyword("NULL") + protected val ON = Keyword("ON") + protected val OR = Keyword("OR") + protected val ORDER = Keyword("ORDER") + protected val OUTER = Keyword("OUTER") + protected val RIGHT = Keyword("RIGHT") + protected val SELECT = Keyword("SELECT") + protected val STRING = Keyword("STRING") + protected val SUM = Keyword("SUM") + protected val TRUE = Keyword("TRUE") + protected val UNION = Keyword("UNION") + protected val WHERE = Keyword("WHERE") + + // Use reflection to find the reserved words defined in this class. + protected val reservedWords = + this.getClass + .getMethods + .filter(_.getReturnType == classOf[Keyword]) + .map(_.invoke(this).asInstanceOf[Keyword]) + + lexical.reserved ++= reservedWords.map(_.str) + + lexical.delimiters += ( + "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", + ",", ";", "%", "{", "}", ":" + ) + + protected def assignAliases(exprs: Seq[Expression]): Seq[NamedExpression] = { + exprs.zipWithIndex.map { + case (ne: NamedExpression, _) => ne + case (e, i) => Alias(e, s"c$i")() + } + } + + protected lazy val query: Parser[LogicalPlan] = + select * ( + UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } | + UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } + ) + + protected lazy val select: Parser[LogicalPlan] = + SELECT ~> opt(DISTINCT) ~ projections ~ + opt(from) ~ opt(filter) ~ + opt(grouping) ~ + opt(having) ~ + opt(orderBy) ~ + opt(limit) <~ opt(";") ^^ { + case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l => + val base = r.getOrElse(NoRelation) + val withFilter = f.map(f => Filter(f, base)).getOrElse(base) + val withProjection = + g.map {g => + Aggregate(assignAliases(g), assignAliases(p), withFilter) + }.getOrElse(Project(assignAliases(p), withFilter)) + val withDistinct = d.map(_ => Distinct(withProjection)).getOrElse(withProjection) + val withHaving = h.map(h => Filter(h, withDistinct)).getOrElse(withDistinct) + val withOrder = o.map(o => Sort(o, withHaving)).getOrElse(withHaving) + val withLimit = l.map { l => StopAfter(l, withOrder) }.getOrElse(withOrder) + withLimit + } + + protected lazy val projections: Parser[Seq[Expression]] = repsep(projection, ",") + + protected lazy val projection: Parser[Expression] = + expression ~ (opt(AS) ~> opt(ident)) ^^ { + case e ~ None => e + case e ~ Some(a) => Alias(e, a)() + } + + protected lazy val from: Parser[LogicalPlan] = FROM ~> relations + + // Based very loosly on the MySQL Grammar. + // http://dev.mysql.com/doc/refman/5.0/en/join.html + protected lazy val relations: Parser[LogicalPlan] = + relation ~ "," ~ relation ^^ { case r1 ~ _ ~ r2 => Join(r1, r2, Inner, None) } | + relation + + protected lazy val relation: Parser[LogicalPlan] = + joinedRelation | + relationFactor + + protected lazy val relationFactor: Parser[LogicalPlan] = + ident ~ (opt(AS) ~> opt(ident)) ^^ { + case ident ~ alias => UnresolvedRelation(alias, ident) + } | + "(" ~> query ~ ")" ~ opt(AS) ~ ident ^^ { case s ~ _ ~ _ ~ a => Subquery(a, s) } + + protected lazy val joinedRelation: Parser[LogicalPlan] = + relationFactor ~ opt(joinType) ~ JOIN ~ relationFactor ~ opt(joinConditions) ^^ { + case r1 ~ jt ~ _ ~ r2 ~ cond => + Join(r1, r2, joinType = jt.getOrElse(Inner), cond) + } + + protected lazy val joinConditions: Parser[Expression] = + ON ~> expression + + protected lazy val joinType: Parser[JoinType] = + INNER ^^^ Inner | + LEFT ~ opt(OUTER) ^^^ LeftOuter | + RIGHT ~ opt(OUTER) ^^^ RightOuter | + FULL ~ opt(OUTER) ^^^ FullOuter + + protected lazy val filter: Parser[Expression] = WHERE ~ expression ^^ { case _ ~ e => e } + + protected lazy val orderBy: Parser[Seq[SortOrder]] = + ORDER ~> BY ~> ordering + + protected lazy val ordering: Parser[Seq[SortOrder]] = + rep1sep(singleOrder, ",") | + rep1sep(expression, ",") ~ opt(direction) ^^ { + case exps ~ None => exps.map(SortOrder(_, Ascending)) + case exps ~ Some(d) => exps.map(SortOrder(_, d)) + } + + protected lazy val singleOrder: Parser[SortOrder] = + expression ~ direction ^^ { case e ~ o => SortOrder(e,o) } + + protected lazy val direction: Parser[SortDirection] = + ASC ^^^ Ascending | + DESC ^^^ Descending + + protected lazy val grouping: Parser[Seq[Expression]] = + GROUP ~> BY ~> rep1sep(expression, ",") + + protected lazy val having: Parser[Expression] = + HAVING ~> expression + + protected lazy val limit: Parser[Expression] = + LIMIT ~> expression + + protected lazy val expression: Parser[Expression] = orExpression + + protected lazy val orExpression: Parser[Expression] = + andExpression * (OR ^^^ { (e1: Expression, e2: Expression) => Or(e1,e2) }) + + protected lazy val andExpression: Parser[Expression] = + comparisionExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1,e2) }) + + protected lazy val comparisionExpression: Parser[Expression] = + termExpression ~ "=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Equals(e1, e2) } | + termExpression ~ "<" ~ termExpression ^^ { case e1 ~ _ ~ e2 => LessThan(e1, e2) } | + termExpression ~ "<=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => LessThanOrEqual(e1, e2) } | + termExpression ~ ">" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThan(e1, e2) } | + termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } | + termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(Equals(e1, e2)) } | + termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(Equals(e1, e2)) } | + termExpression ~ IN ~ "(" ~ rep1sep(termExpression, ",") <~ ")" ^^ { + case e1 ~ _ ~ _ ~ e2 => In(e1, e2) + } | + termExpression ~ NOT ~ IN ~ "(" ~ rep1sep(termExpression, ",") <~ ")" ^^ { + case e1 ~ _ ~ _ ~ _ ~ e2 => Not(In(e1, e2)) + } | + termExpression <~ IS ~ NULL ^^ { case e => IsNull(e) } | + termExpression <~ IS ~ NOT ~ NULL ^^ { case e => IsNotNull(e) } | + NOT ~> termExpression ^^ {e => Not(e)} | + termExpression + + protected lazy val termExpression: Parser[Expression] = + productExpression * ( + "+" ^^^ { (e1: Expression, e2: Expression) => Add(e1,e2) } | + "-" ^^^ { (e1: Expression, e2: Expression) => Subtract(e1,e2) } ) + + protected lazy val productExpression: Parser[Expression] = + baseExpression * ( + "*" ^^^ { (e1: Expression, e2: Expression) => Multiply(e1,e2) } | + "/" ^^^ { (e1: Expression, e2: Expression) => Divide(e1,e2) } | + "%" ^^^ { (e1: Expression, e2: Expression) => Remainder(e1,e2) } + ) + + protected lazy val function: Parser[Expression] = + SUM ~> "(" ~> expression <~ ")" ^^ { case exp => Sum(exp) } | + SUM ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => SumDistinct(exp) } | + COUNT ~> "(" ~ "*" <~ ")" ^^ { case _ => Count(Literal(1)) } | + COUNT ~> "(" ~ expression <~ ")" ^^ { case dist ~ exp => Count(exp) } | + COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => CountDistinct(exp :: Nil) } | + FIRST ~> "(" ~> expression <~ ")" ^^ { case exp => First(exp) } | + AVG ~> "(" ~> expression <~ ")" ^^ { case exp => Sum(exp) } | + IF ~> "(" ~> expression ~ "," ~ expression ~ "," ~ expression <~ ")" ^^ { + case c ~ "," ~ t ~ "," ~ f => If(c,t,f) + } | + ident ~ "(" ~ repsep(expression, ",") <~ ")" ^^ { + case udfName ~ _ ~ exprs => UnresolvedFunction(udfName, exprs) + } + + protected lazy val cast: Parser[Expression] = + CAST ~> "(" ~> expression ~ AS ~ dataType <~ ")" ^^ { case exp ~ _ ~ t => Cast(exp, t) } + + protected lazy val literal: Parser[Literal] = + numericLit ^^ { + case i if i.toLong <= Int.MaxValue => Literal(i.toLong) + case i => Literal(i.toInt) + } | + NULL ^^^ Literal(null, NullType) | + floatLit ^^ {case f => Literal(f.toDouble) } | + stringLit ^^ {case s => Literal(s, StringType) } + + protected lazy val floatLit: Parser[String] = + elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars) + + protected lazy val baseExpression: Parser[Expression] = + TRUE ^^^ Literal(true, BooleanType) | + FALSE ^^^ Literal(false, BooleanType) | + cast | + "(" ~> expression <~ ")" | + function | + "-" ~> literal ^^ UnaryMinus | + ident ^^ UnresolvedAttribute | + "*" ^^^ Star(None) | + literal + + protected lazy val dataType: Parser[DataType] = + STRING ^^^ StringType +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index e6e55a61d3700..c7404266f88df 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -30,6 +30,29 @@ trait Catalog { databaseName: Option[String], tableName: String, alias: Option[String] = None): LogicalPlan + + def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit +} + +class SimpleCatalog extends Catalog { + val tables = new mutable.HashMap[String, LogicalPlan]() + + def registerTable(databaseName: Option[String],tableName: String, plan: LogicalPlan): Unit = { + tables += ((tableName, plan)) + } + + def dropTable(tableName: String) = tables -= tableName + + def lookupRelation( + databaseName: Option[String], + tableName: String, + alias: Option[String] = None): LogicalPlan = { + val table = tables.get(tableName).getOrElse(sys.error(s"Table Not Found: $tableName")) + + // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are + // properly qualified with this alias. + alias.map(a => Subquery(a.toLowerCase, table)).getOrElse(table) + } } /** @@ -58,8 +81,9 @@ trait OverrideCatalog extends Catalog { withAlias.getOrElse(super.lookupRelation(databaseName, tableName, alias)) } - def overrideTable(databaseName: Option[String], tableName: String, plan: LogicalPlan) = + override def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = { overrides.put((databaseName, tableName), plan) + } } /** @@ -73,4 +97,8 @@ object EmptyCatalog extends Catalog { alias: Option[String] = None) = { throw new UnsupportedOperationException } + + def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = { + throw new UnsupportedOperationException + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 0f02b050bd499..ba304a27b45bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -27,6 +27,51 @@ import plans._ import plans.logical._ import types._ +/** + * Provides experimental support for generating catalyst schemas for scala objects. + */ +object ScalaReflection { + import scala.reflect.runtime.universe._ + + /** Returns a Sequence of attributes for the given case class type. */ + def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { + case s: StructType => + s.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) + } + + /** Returns a catalyst DataType for the given Scala Type using reflection. */ + def schemaFor[T: TypeTag]: DataType = schemaFor(typeOf[T]) + + /** Returns a catalyst DataType for the given Scala Type using reflection. */ + def schemaFor(tpe: `Type`): DataType = tpe match { + case t if t <:< typeOf[Product] => + val params = t.member("": TermName).asMethod.paramss + StructType( + params.head.map(p => StructField(p.name.toString, schemaFor(p.typeSignature), true))) + case t if t <:< typeOf[Seq[_]] => + val TypeRef(_, _, Seq(elementType)) = t + ArrayType(schemaFor(elementType)) + case t if t <:< typeOf[String] => StringType + case t if t <:< definitions.IntTpe => IntegerType + case t if t <:< definitions.LongTpe => LongType + case t if t <:< definitions.DoubleTpe => DoubleType + case t if t <:< definitions.ShortTpe => ShortType + case t if t <:< definitions.ByteTpe => ByteType + } + + implicit class CaseClassRelation[A <: Product : TypeTag](data: Seq[A]) { + + /** + * Implicitly added to Sequences of case class objects. Returns a catalyst logical relation + * for the the data in the sequence. + */ + def asRelation: LocalRelation = { + val output = attributesFor[A] + LocalRelation(output, data) + } + } +} + /** * A collection of implicit conversions that create a DSL for constructing catalyst data structures. * @@ -54,52 +99,6 @@ import types._ * }}} */ package object dsl { - - /** - * Provides experimental support for generating catalyst schemas for scala objects. - */ - object reflect { - import scala.reflect.runtime.universe._ - - /** Returns a Sequence of attributes for the given case class type. */ - def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { - case s: StructType => - s.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) - } - - /** Returns a catalyst DataType for the given Scala Type using reflection. */ - def schemaFor[T: TypeTag]: DataType = schemaFor(typeOf[T]) - - /** Returns a catalyst DataType for the given Scala Type using reflection. */ - def schemaFor(tpe: `Type`): DataType = tpe match { - case t if t <:< typeOf[Product] => - val params = t.member("": TermName).asMethod.paramss - StructType( - params.head.map(p => StructField(p.name.toString, schemaFor(p.typeSignature), true))) - case t if t <:< typeOf[Seq[_]] => - val TypeRef(_, _, Seq(elementType)) = t - ArrayType(schemaFor(elementType)) - case t if t <:< typeOf[String] => StringType - case t if t <:< definitions.IntTpe => IntegerType - case t if t <:< definitions.LongTpe => LongType - case t if t <:< definitions.DoubleTpe => DoubleType - case t if t <:< definitions.ShortTpe => ShortType - case t if t <:< definitions.ByteTpe => ByteType - } - } - - implicit class CaseClassRelation[A <: Product : TypeTag](data: Seq[A]) { - - /** - * Implicitly added to Sequences of case class objects. Returns a catalyst logical relation - * for the the data in the sequence. - */ - def asRelation: LocalRelation = { - val output = reflect.attributesFor[A] - LocalRelation(output, data) - } - } - protected trait ImplicitOperators { def expr: Expression diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala index edf912bd8fa92..f51fa0dfbc181 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala @@ -18,11 +18,13 @@ package org.apache.spark.sql import scala.language.implicitConversions +import scala.reflect.runtime.universe.TypeTag import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.rdd.RDD import catalyst.analysis._ +import catalyst.dsl import catalyst.expressions.BindReferences import catalyst.optimizer.Optimizer import catalyst.planning.QueryPlanner @@ -49,21 +51,29 @@ object TestSqlContext class SparkSqlContext(val sparkContext: SparkContext) extends Logging { self => - val catalog: Catalog = EmptyCatalog + val catalog: Catalog = new SimpleCatalog val analyzer: Analyzer = new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) val optimizer = Optimizer + val parser = new catalyst.SqlParser - def parseSql(sql: String): LogicalPlan = ??? + def parseSql(sql: String): LogicalPlan = parser(sql) def executeSql(sql: String): this.QueryExecution = executePlan(parseSql(sql)) def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } implicit def logicalPlanToSparkQuery(plan: LogicalPlan) = executePlan(plan) - implicit def logicalDsl(q: ExecutedQuery) = new catalyst.dsl.DslLogicalPlan(q.logicalPlan) + implicit def logicalDsl(q: ExecutedQuery) = new dsl.DslLogicalPlan(q.logicalPlan) implicit def toRdd(q: ExecutedQuery) = q.rdd + implicit class TableRdd[A <: Product: TypeTag](rdd: RDD[A]) { + def registerAsTable(tableName: String) = { + catalog.registerTable( + None, tableName, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd)) ) + } + } + def sql(sqlText: String): ExecutedQuery = { val queryWorkflow = executeSql(sqlText) val executedPlan = queryWorkflow.analyzed match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala index 853238eae0baa..9cd2bdda5aa8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala @@ -21,6 +21,7 @@ package execution import org.apache.spark.rdd.RDD import catalyst.plans.QueryPlan +import catalyst.plans.logical import catalyst.plans.physical._ import catalyst.trees @@ -48,6 +49,15 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { new catalyst.expressions.GenericRow(values.toArray) } +/** + * Allows already planned SparkQueries to be linked into logical query plans. + */ +case class SparkLogicalPlan(alreadyPlanned: SparkPlan) extends logical.LogicalPlan { + def output = alreadyPlanned.output + def references = Set.empty + def children = Nil +} + trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { self: Product => } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 828618ebd4d4a..66e57fe21363f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -212,6 +212,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { val relation = ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, None) InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil + case SparkLogicalPlan(existingPlan) => existingPlan :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 40d3ef8128756..b799c813be2b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql package execution +import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext @@ -25,6 +27,7 @@ import catalyst.errors._ import catalyst.expressions._ import catalyst.plans.physical.{UnspecifiedDistribution, OrderedDistribution} import catalyst.plans.logical.LogicalPlan +import catalyst.ScalaReflection case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -111,6 +114,35 @@ case class Sort( def output = child.output } +object ExistingRdd { + def convertToCatalyst(a: Any): Any = a match { + case s: Seq[Any] => s.map(convertToCatalyst) + case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) + case other => other + } + + def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { + data.mapPartitions { iter => + var currentCaseClass: A = null.asInstanceOf[A] + var mutableRow: MutableRow = null + + if(iter.hasNext) { + currentCaseClass = iter.next() + mutableRow = new GenericMutableRow(currentCaseClass.productIterator.size) + } + + ??? + + } + + data.map(r => new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row) + } + + def fromProductRdd[A <: Product : TypeTag](productRdd: RDD[A]) = { + ExistingRdd(ScalaReflection.attributesFor[A], productToRowRdd(productRdd)) + } +} + case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { def execute() = rdd } 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 e3b883365b84a..882c3aa2c87c5 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 @@ -132,6 +132,12 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { } } } + + /** + * UNIMPLEMENTED: It needs to be decided how we will persist in-memory tables to the metastore. + * For now, if this functionallity is desired mix in the in-memory [[OverrideCatalog]]. + */ + override def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = ??? } object HiveMetastoreTypes extends RegexParsers { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 4d3820902ec53..34b72d506df07 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -98,7 +98,7 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAf // write test data ParquetTestData.writeFile // Override initial Parquet test table - TestHive.catalog.overrideTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) + TestHive.catalog.registerTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) } override def afterAll() { From 7233a7452fc36d3a9d7e7afcd560e9aad73bbf6c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 11 Mar 2014 15:19:08 -0700 Subject: [PATCH 755/778] initial support for maven builds --- assembly/pom.xml | 5 +++ examples/pom.xml | 6 ++++ pom.xml | 5 +++ sql/catalyst/pom.xml | 66 ++++++++++++++++++++++++++++++++++++ sql/core/pom.xml | 76 +++++++++++++++++++++++++++++++++++++++++ sql/hive/pom.xml | 81 ++++++++++++++++++++++++++++++++++++++++++++ 6 files changed, 239 insertions(+) create mode 100644 sql/catalyst/pom.xml create mode 100644 sql/core/pom.xml create mode 100644 sql/hive/pom.xml diff --git a/assembly/pom.xml b/assembly/pom.xml index 22bbbc57d81d4..b5e752c6cd1f6 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -79,6 +79,11 @@ spark-graphx_${scala.binary.version} ${project.version} + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + net.sf.py4j py4j diff --git a/examples/pom.xml b/examples/pom.xml index 9f0e2d0b875b8..60226190d40ff 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -85,6 +85,12 @@ ${project.version} provided + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + provided + org.apache.spark spark-graphx_${scala.binary.version} diff --git a/pom.xml b/pom.xml index 986626f029d75..d2d601a1b44f5 100644 --- a/pom.xml +++ b/pom.xml @@ -91,6 +91,9 @@ mllib tools streaming + sql/catalyst + sql/core + sql/hive repl assembly external/twitter @@ -118,6 +121,8 @@ 2.4.1 0.23.7 0.94.6 + 0.12.0 + 1.3.2 64m 512m diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml new file mode 100644 index 0000000000000..6a5d44d012a84 --- /dev/null +++ b/sql/catalyst/pom.xml @@ -0,0 +1,66 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-catalyst_2.10 + jar + Spark Project Catalyst + http://spark-project.org/ + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + com.typesafe + scalalogging-slf4j_${scala.binary.version} + 1.0.1 + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/sql/core/pom.xml b/sql/core/pom.xml new file mode 100644 index 0000000000000..d745c62ff2439 --- /dev/null +++ b/sql/core/pom.xml @@ -0,0 +1,76 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-sql_2.10 + jar + Spark Project SQL + http://spark-project.org/ + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + + + com.twitter + parquet-column + ${parquet.version} + + + com.twitter + parquet-hadoop + ${parquet.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml new file mode 100644 index 0000000000000..c564ac47abc8f --- /dev/null +++ b/sql/hive/pom.xml @@ -0,0 +1,81 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-hive_2.10 + jar + Spark Project Hive + http://spark-project.org/ + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + + + org.apache.hive + hive-metastore + ${hive.version} + + + org.apache.hive + hive-exec + ${hive.version} + + + org.apache.hive + hive-serde + ${hive.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + From 3447c3edb7a83163a5668c68a246bc04216a0e71 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 13 Mar 2014 12:15:50 -0700 Subject: [PATCH 756/778] Don't override the metastore / warehouse in non-local/test hive context. --- .../apache/spark/sql/SparkSqlContext.scala | 4 +- .../apache/spark/sql/hive/HiveContext.scala | 41 ++++++++----------- .../org/apache/spark/sql/hive/TestShark.scala | 6 +-- .../hive/execution/HiveComparisonTest.scala | 2 + 4 files changed, 25 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala index f51fa0dfbc181..73a03e44505cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala @@ -51,8 +51,8 @@ object TestSqlContext class SparkSqlContext(val sparkContext: SparkContext) extends Logging { self => - val catalog: Catalog = new SimpleCatalog - val analyzer: Analyzer = new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) + lazy val catalog: Catalog = new SimpleCatalog + lazy val analyzer: Analyzer = new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) val optimizer = Optimizer val parser = new catalyst.SqlParser 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 3e8e491e8758a..881f861349fa3 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 @@ -43,37 +43,34 @@ import scala.collection.JavaConversions._ * Starts up an instance of hive where metadata is stored locally. An in-process metadata data is * created with data stored in ./metadata. Warehouse data is stored in in ./warehouse. */ -class LocalHiveContext( - sc: SparkContext, - override val warehousePath: String = new File("warehouse").getCanonicalPath) +class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { - override def metastorePath = new File("metastore").getCanonicalPath + lazy val metastorePath = new File("metastore").getCanonicalPath + lazy val warehousePath: String = new File("warehouse").getCanonicalPath + + /** Sets up the system initially or after a RESET command */ + protected def configure() { + // TODO: refactor this so we can work with other databases. + runSqlHive( + s"set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$metastorePath;create=true") + runSqlHive("set hive.metastore.warehouse.dir=" + warehousePath) + } + + configure() // Must be called before initializing the catalog below. } /** * An instance of the Spark SQL execution engine that integrates with data stored in Hive. + * Configuration for Hive is read from hive-site.xml on the classpath. */ -abstract class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { +class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { self => - /** The path to the hive warehouse. */ - def warehousePath: String - /** The path to the local metastore. */ - def metastorePath: String - override def parseSql(sql: String): LogicalPlan = HiveQl.parseSql(sql) override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } - /** Sets up the system initially or after a RESET command */ - protected def configure() { - // TODO: refactor this so we can work with other databases. - runSqlHive( - s"set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$metastorePath;create=true") - runSqlHive("set hive.metastore.warehouse.dir=" + warehousePath) - } - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. protected val outputBuffer = new java.io.OutputStream { var pos: Int = 0 @@ -103,18 +100,16 @@ abstract class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { } @transient protected[hive] lazy val hiveconf = new HiveConf(classOf[SessionState]) - @transient protected[hive] val sessionState = new SessionState(hiveconf) + @transient protected[hive] lazy val sessionState = new SessionState(hiveconf) sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") - configure() // Must be called before initializing the catalog below. - /* A catalyst metadata catalog that points to the Hive Metastore. */ - override val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog + override lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog /* An analyzer that uses the Hive metastore. */ - override val analyzer = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) + override lazy val analyzer = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) def tables: Seq[BaseRelation] = { // TODO: Move this functionallity to Catalog. Make client protected. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala index 65c0e767864dc..0a13981d47513 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala @@ -54,7 +54,7 @@ object TestHive * hive metastore seems to lead to weird non-deterministic failures. Therefore, the execution of * testcases that rely on TestHive must be serialized. */ -class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { +class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { self => // By clearing the port we force Spark to pick a new one. This allows us to rerun tests @@ -62,8 +62,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") - lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath - lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath + override lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath + override lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") 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 ae259394df568..8a5b97b7a05ef 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 @@ -227,6 +227,8 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G } try { + // MINOR HACK: You must run a query before calling reset the first time. + TestHive.sql("SHOW TABLES") TestHive.reset() val hiveCacheFiles = queryList.zipWithIndex.map { From f93aa39fdd3cabc3377c92bc650a6f23469c3291 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Fri, 14 Mar 2014 18:25:21 +0200 Subject: [PATCH 757/778] Better handling of path names in ParquetRelation Previously incomplete path names (with missing URI field) were passed to Parquet. Also two rules were moved from HiveStrategies to SparkStrategies. --- .../spark/sql/execution/SparkStrategies.scala | 8 +++ .../spark/sql/parquet/ParquetRelation.scala | 51 ++++++++++++------- .../spark/sql/hive/HiveStrategies.scala | 7 --- 3 files changed, 42 insertions(+), 24 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 66e57fe21363f..65ec62fed3214 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -185,6 +185,12 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // This sort only sorts tuples within a partition. Its requiredDistribution will be // an UnspecifiedDistribution. execution.Sort(sortExprs, global = false, planLater(child)) :: Nil + case logical.Project(projectList, r: ParquetRelation) if projectList.forall(_.isInstanceOf[Attribute]) => + // simple projection of data loaded from Parquet file + parquet.ParquetTableScan( + projectList.asInstanceOf[Seq[Attribute]], + r, + None)(sparkContext) :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => @@ -212,6 +218,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { val relation = ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, None) InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil + case p: parquet.ParquetRelation => + parquet.ParquetTableScan(p.output, p, None)(sparkContext) :: Nil case SparkLogicalPlan(existingPlan) => existingPlan :: Nil case _ => Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index baaba0096c02d..638b1c72e6677 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -123,7 +123,7 @@ object ParquetRelation { /** * Creates a new ParquetRelation and underlying Parquetfile for the given - * LogicalPlan. Note that this is used insider [[SparkStrategies]] to + * LogicalPlan. Note that this is used inside [[SparkStrategies]] to * create a resolved relation as a data sink for writing to a Parquetfile. * The relation is empty but is initialized with ParquetMetadata and * can be inserted into. @@ -151,8 +151,15 @@ object ParquetRelation { } private def checkPath(pathStr: String, conf: Configuration): Path = { - val path = new Path(pathStr) - val fs = path.getFileSystem(conf) + if (pathStr == null) { + throw new IllegalArgumentException("Unable to create ParquetRelation: path is null") + } + val origPath = new Path(pathStr) + val fs = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException(s"Unable to create ParquetRelation: incorrectly formatted path $pathStr") + } + val path = origPath.makeQualified(fs) if (fs.exists(path) && !fs.getFileStatus(path) .getPermission @@ -233,24 +240,27 @@ object ParquetTypesConverter { new MessageType("root", fields) } - def writeMetaData(attributes: Seq[Attribute], path: Path, conf: Configuration) { - val fileSystem = FileSystem.get(conf) - - if (fileSystem.exists(path) && !fileSystem.getFileStatus(path).isDir) { - throw new IOException(s"Expected to write to directory $path but found file") + def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration) { + if (origPath == null) { + throw new IllegalArgumentException("Unable to write Parquet metadata: path is null") + } + val fs = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException(s"Unable to write Parquet metadata: path $origPath is incorrectly formatted") + } + val path = origPath.makeQualified(fs) + if (fs.exists(path) && !fs.getFileStatus(path).isDir) { + throw new IllegalArgumentException(s"Expected to write to directory $path but found file") } - val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) - - if (fileSystem.exists(metadataPath)) { + if (fs.exists(metadataPath)) { try { - fileSystem.delete(metadataPath, true) + fs.delete(metadataPath, true) } catch { case e: IOException => throw new IOException(s"Unable to delete previous PARQUET_METADATA_FILE at $metadataPath") } } - val extraMetadata = new java.util.HashMap[String, String]() extraMetadata.put("path", path.toString) // TODO: add extra data, e.g., table name, date, etc.? @@ -275,13 +285,20 @@ object ParquetTypesConverter { * @param path The path at which we expect one (or more) Parquet files. * @return The `ParquetMetadata` containing among other things the schema. */ - def readMetaData(path: Path): ParquetMetadata = { + def readMetaData(origPath: Path): ParquetMetadata = { + if (origPath == null) { + throw new IllegalArgumentException("Unable to read Parquet metadata: path is null") + } val job = new Job() + // TODO: since this is called from ParquetRelation (LogicalPlan) we don't have access + // to SparkContext's hadoopConfig; in principle the default FileSystem may be different(?!) val conf = ContextUtil.getConfiguration(job) - val fs: FileSystem = path.getFileSystem(conf) - + val fs: FileSystem = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException(s"Incorrectly formatted Parquet metadata path $origPath") + } + val path = origPath.makeQualified(fs) val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) - if (fs.exists(metadataPath) && fs.isFile(metadataPath)) { // TODO: improve exception handling, etc. ParquetFileReader.readFooter(conf, metadataPath) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 49164f5105f16..1eca33609bf2b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -55,15 +55,8 @@ trait HiveStrategies { // Push attributes into table scan when possible. case p @ logical.Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m, None)(hiveContext) :: Nil - case p @ logical.Project(projectList, r: ParquetRelation) if isSimpleProject(projectList) => - ParquetTableScan( - projectList.asInstanceOf[Seq[Attribute]], - r, - None)(hiveContext.sparkContext) :: Nil case m: MetastoreRelation => HiveTableScan(m.output, m, None)(hiveContext) :: Nil - case p: ParquetRelation => - ParquetTableScan(p.output, p, None)(hiveContext.sparkContext) :: Nil case _ => Nil } } From 8b35e0ac28080a4470d7e7eb6d0d3145de12d4e2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 13 Mar 2014 13:53:54 -0700 Subject: [PATCH 758/778] address feedback, work on DSL --- .../spark/sql/examples/RDDRelation.scala | 4 +- .../spark/sql/catalyst/dsl/package.scala | 35 +++++++------ .../plans/{shared.scala => joinTypes.scala} | 0 .../apache/spark/sql/SparkSqlContext.scala | 52 +++++++++++++------ .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 2 + .../spark/sql/hive/HiveStrategies.scala | 2 +- 7 files changed, 62 insertions(+), 35 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/{shared.scala => joinTypes.scala} (100%) diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala index aa1aca7bcc605..e855052136fc6 100644 --- a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala @@ -1,7 +1,7 @@ package org.apache.spark.sql.examples import org.apache.spark.SparkContext -import org.apache.spark.sql.SparkSqlContext +import org.apache.spark.sql.SqlContext // One method for defining the schema of an RDD is to make a case class with the desired column // names and types. @@ -10,7 +10,7 @@ case class Record(key: Int, value: String) object RDDRelation { def main(args: Array[String]) { val sc = new SparkContext("local", "RDDRelation") - val sqlContext = new SparkSqlContext(sc) + val sqlContext = new SqlContext(sc) // Importing the SQL contexts give access to all the SQL functions and implicit conversions. import sqlContext._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index ba304a27b45bb..76ccd1627e38b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -157,56 +157,61 @@ package object dsl { def required = a.withNullability(false) } - implicit class DslLogicalPlan(plan: LogicalPlan) { - def select(exprs: NamedExpression*) = Project(exprs, plan) + implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions - def where(condition: Expression) = Filter(condition, plan) + abstract class LogicalPlanFunctions { + def logicalPlan: LogicalPlan + + def select(exprs: NamedExpression*) = Project(exprs, logicalPlan) + + def where(condition: Expression) = Filter(condition, logicalPlan) def join( otherPlan: LogicalPlan, joinType: JoinType = Inner, condition: Option[Expression] = None) = - Join(plan, otherPlan, joinType, condition) + Join(logicalPlan, otherPlan, joinType, condition) - def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, plan) + def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, logicalPlan) def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*) = { val aliasedExprs = aggregateExprs.map { case ne: NamedExpression => ne case e => Alias(e, e.toString)() } - Aggregate(groupingExprs, aliasedExprs, plan) + Aggregate(groupingExprs, aliasedExprs, logicalPlan) } - def subquery(alias: Symbol) = Subquery(alias.name, plan) + def subquery(alias: Symbol) = Subquery(alias.name, logicalPlan) - def unionAll(otherPlan: LogicalPlan) = Union(plan, otherPlan) + def unionAll(otherPlan: LogicalPlan) = Union(logicalPlan, otherPlan) def sfilter[T1](arg1: Symbol)(udf: (T1) => Boolean) = - Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), plan) + Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan) def sfilter(dynamicUdf: (DynamicRow) => Boolean) = - Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(plan.output))), plan) + Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan) def sample( fraction: Double, withReplacement: Boolean = true, seed: Int = (math.random * 1000).toInt) = - Sample(fraction, withReplacement, seed, plan) + Sample(fraction, withReplacement, seed, logicalPlan) def generate( generator: Generator, join: Boolean = false, outer: Boolean = false, alias: Option[String] = None) = - Generate(generator, join, outer, None, plan) + Generate(generator, join, outer, None, logicalPlan) def insertInto(tableName: String, overwrite: Boolean = false) = - InsertIntoTable(analysis.UnresolvedRelation(None, tableName), Map.empty, plan, overwrite) + InsertIntoTable( + analysis.UnresolvedRelation(None, tableName), Map.empty, logicalPlan, overwrite) - def analyze = analysis.SimpleAnalyzer(plan) + def analyze = analysis.SimpleAnalyzer(logicalPlan) - def writeToFile(path: String) = WriteToFile(path, plan) + def writeToFile(path: String) = WriteToFile(path, logicalPlan) // TODO: for a loadFromFile it would be good to have a Catalog that knows // how to resolve ParquetTables diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala index 73a03e44505cc..9bbb87e5d57d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala @@ -33,6 +33,11 @@ import catalyst.rules.RuleExecutor import execution._ +/** + * The result of executing a query using SparkSQL. This class acts as an RDD of the through + * implicit conversions. It also allows access to the executed plan for DML queries, similar to an + * EXPLAIN command in standard SQL. + */ case class ExecutedQuery( sql: String, logicalPlan: LogicalPlan, @@ -46,34 +51,49 @@ case class ExecutedQuery( } object TestSqlContext - extends SparkSqlContext(new SparkContext("local", "TestSqlContext", new SparkConf())) + extends SqlContext(new SparkContext("local", "TestSqlContext", new SparkConf())) -class SparkSqlContext(val sparkContext: SparkContext) extends Logging { +/** + * The entry point for running relational queries using Spark. Uses the provided spark context + * to execute relational operators. + */ +class SqlContext(val sparkContext: SparkContext) extends Logging { self => - lazy val catalog: Catalog = new SimpleCatalog - lazy val analyzer: Analyzer = new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) - val optimizer = Optimizer - val parser = new catalyst.SqlParser + protected[sql] lazy val catalog: Catalog = new SimpleCatalog + protected[sql] lazy val analyzer: Analyzer = + new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) + protected[sql] val optimizer = Optimizer + protected[sql] val parser = new catalyst.SqlParser - def parseSql(sql: String): LogicalPlan = parser(sql) - def executeSql(sql: String): this.QueryExecution = executePlan(parseSql(sql)) - def executePlan(plan: LogicalPlan): this.QueryExecution = + protected[sql] def parseSql(sql: String): LogicalPlan = parser(sql) + protected[sql] def executeSql(sql: String): this.QueryExecution = executePlan(parseSql(sql)) + protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } implicit def logicalPlanToSparkQuery(plan: LogicalPlan) = executePlan(plan) implicit def logicalDsl(q: ExecutedQuery) = new dsl.DslLogicalPlan(q.logicalPlan) + /** Allows the results of sql queries to be used as RDDs */ implicit def toRdd(q: ExecutedQuery) = q.rdd - implicit class TableRdd[A <: Product: TypeTag](rdd: RDD[A]) { + /** + * Implicitly adds a `registerAsTable` to RDDs of case classes and allows the Query DSL to be + * used on them. + */ + implicit class TableRdd[A <: Product: TypeTag](rdd: RDD[A]) extends dsl.LogicalPlanFunctions { + def logicalPlan = SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd)) + def registerAsTable(tableName: String) = { - catalog.registerTable( - None, tableName, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd)) ) + catalog.registerTable(None, tableName, logicalPlan) } } + /** + * Executes a SQL query using Spark, returning the result as an RDD as well as the plan used + * for execution. + */ def sql(sqlText: String): ExecutedQuery = { val queryWorkflow = executeSql(sqlText) val executedPlan = queryWorkflow.analyzed match { @@ -83,7 +103,7 @@ class SparkSqlContext(val sparkContext: SparkContext) extends Logging { ExecutedQuery(sqlText, queryWorkflow.analyzed, executedPlan, queryWorkflow.toRdd) } - class SparkPlanner extends SparkStrategies { + protected[sql] class SparkPlanner extends SparkStrategies { val sparkContext = self.sparkContext val strategies: Seq[Strategy] = @@ -95,13 +115,13 @@ class SparkSqlContext(val sparkContext: SparkContext) extends Logging { BroadcastNestedLoopJoin :: Nil } - val planner = new SparkPlanner + protected[sql] val planner = new SparkPlanner /** * Prepares a planned SparkPlan for execution by binding references to specific ordinals, and * inserting shuffle operations as needed. */ - object PrepareForExecution extends RuleExecutor[SparkPlan] { + protected[sql] object PrepareForExecution extends RuleExecutor[SparkPlan] { val batches = Batch("Add exchange", Once, AddExchange) :: Batch("Prepare Expressions", Once, new BindReferences[SparkPlan]) :: Nil @@ -112,7 +132,7 @@ class SparkSqlContext(val sparkContext: SparkContext) extends Logging { * access to the intermediate phases of query execution for developers. Most users should * use [[ExecutedQuery]] to interact with query results. */ - abstract class QueryExecution { + protected abstract class QueryExecution { def logical: LogicalPlan lazy val analyzed = analyzer(logical) 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 881f861349fa3..a1d1ef00a94a8 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 @@ -64,7 +64,7 @@ class LocalHiveContext(sc: SparkContext) * An instance of the Spark SQL execution engine that integrates with data stored in Hive. * Configuration for Hive is read from hive-site.xml on the classpath. */ -class HiveContext(sc: SparkContext) extends SparkSqlContext(sc) { +class HiveContext(sc: SparkContext) extends SqlContext(sc) { self => override def parseSql(sql: String): LogicalPlan = HiveQl.parseSql(sql) 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 1998045c29926..9a29cce9cf734 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 @@ -48,6 +48,7 @@ case class AddJar(jarPath: String) extends Command case class AddFile(filePath: String) extends Command +/** Provides a mapping from HiveQL statments to catalyst logical plans and expression trees. */ object HiveQl { protected val nativeCommands = Seq( "TOK_DESCFUNCTION", @@ -203,6 +204,7 @@ object HiveQl { */ def getAst(sql: String): ASTNode = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql)) + /** Returns a LogicalPlan for a given HiveQL string. */ def parseSql(sql: String): LogicalPlan = { try { if (sql.toLowerCase.startsWith("set")) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 1eca33609bf2b..2b14cbab20ad5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.parquet.{ParquetRelation, InsertIntoParquetTable, Pa trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. - self: SparkSqlContext#SparkPlanner => + self: SqlContext#SparkPlanner => val hiveContext: HiveContext From d2d9678a63ffa61d5a2abd37bb667371ce8641ba Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 13 Mar 2014 19:08:27 -0700 Subject: [PATCH 759/778] Make sure hive isn't in the assembly jar. Create a separate, optional Hive assembly that is used when present. --- bin/compute-classpath.sh | 33 ++++++++++++++++++++++++++------- project/SparkBuild.scala | 19 ++++++++++++------- 2 files changed, 38 insertions(+), 14 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 0071a0d2175af..b5c103d470853 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -33,27 +33,43 @@ fi # Build up classpath CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" -# Datanucleus jars do not work if included in the uberjar as plugin.xml metadata is lost. -DATANUCLEUSJARS=$(JARS=("$FWDIR/lib_managed/jars"/datanucleus-*.jar); IFS=:; echo "${JARS[*]}") -CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS +# Support for interacting with Hive. Since hive pulls in a lot of dependencies that might break +# existing Spark applications, it is not included in the standard spark assembly. Instead, we only +# include it in the classpath if the user has explicitly requested it by running "sbt hive/assembly" +# Hopefully we will find a way to avoid uber-jars entierly and deploy only the needed packages in +# the future. +if [ -f "$FWDIR"/sql/hive/target/scala-$SCALA_VERSION/spark-hive-assembly-*.jar ]; then + echo "Hive assembly found, including hive support. If this isn't desired run sbt hive/clean." + + # Datanucleus jars do not work if only included in the uberjar as plugin.xml metadata is lost. + DATANUCLEUSJARS=$(JARS=("$FWDIR/lib_managed/jars"/datanucleus-*.jar); IFS=:; echo "${JARS[*]}") + CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS + + ASSEMBLY_DIR="$FWDIR/sql/hive/target/scala-$SCALA_VERSION/" +else + ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION/" +fi # First check if we have a dependencies jar. If so, include binary classes with the deps jar -if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar ]; then +if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" - DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar` + DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*-deps.jar` CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR" else # Else use spark-assembly jar from either RELEASE or assembly directory if [ -f "$FWDIR/RELEASE" ]; then - ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar` + ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark*-assembly*.jar` else - ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar` + ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*.jar` fi CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi @@ -66,6 +82,9 @@ if [[ $SPARK_TESTING == 1 ]]; then CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes" CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SCALA_VERSION/test-classes" CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/test-classes" fi # Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail ! diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0da1ae4979f6e..76e41e3ba212d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -52,7 +52,7 @@ object SparkBuild extends Build { lazy val core = Project("core", file("core"), settings = coreSettings) lazy val repl = Project("repl", file("repl"), settings = replSettings) - .dependsOn(core, graphx, bagel, mllib, hive) + .dependsOn(core, graphx, bagel, mllib, sql) lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) @@ -64,14 +64,15 @@ object SparkBuild extends Build { lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core, catalyst) - lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql) + // Since hive is its own assembly, it depends on all of the modules. + lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql, graphx, bagel, mllib, streaming, repl) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) - .dependsOn(core, graphx, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) dependsOn(maybeGanglia: _*) + .dependsOn(core, graphx, bagel, mllib, streaming, repl, sql) dependsOn(maybeYarn: _*) dependsOn(maybeGanglia: _*) lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects") @@ -139,11 +140,11 @@ object SparkBuild extends Build { lazy val examples = Project("examples", file("examples"), settings = examplesSettings) .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter, hive) dependsOn(allExternal: _*) - // Everything except assembly, tools, java8Tests and examples belong to packageProjects - lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql, hive) ++ maybeYarnRef ++ maybeGangliaRef + // Everything except assembly, hive, tools, java8Tests and examples belong to packageProjects + lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql) ++ maybeYarnRef ++ maybeGangliaRef lazy val allProjects = packageProjects ++ allExternalRefs ++ - Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests + Seq[ProjectReference](examples, tools, assemblyProj, hive) ++ maybeJava8Tests def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.apache.spark", @@ -386,8 +387,12 @@ object SparkBuild extends Build { ) ) - def hiveSettings = sharedSettings ++ Seq( + // Since we don't include hive in the main assembly this project also acts as an alternative + // assembly jar. + def hiveSettings = sharedSettings ++ assemblyProjSettings ++ Seq( name := "spark-hive", + jarName in assembly <<= version map { v => "spark-hive-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" }, + jarName in packageDependency <<= version map { v => "spark-hive-assembly-" + v + "-hadoop" + hadoopVersion + "-deps.jar" }, javaOptions += "-XX:MaxPermSize=1g", libraryDependencies ++= Seq( "org.apache.hive" % "hive-metastore" % "0.12.0", From 9eb029405a8ba39fe7b40736702ce1443b9b149c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 13 Mar 2014 19:50:43 -0700 Subject: [PATCH 760/778] Bring expressions implicits into SqlContext. --- .../spark/sql/examples/RDDRelation.scala | 6 +- .../spark/sql/catalyst/dsl/package.scala | 61 ++++++++++--------- .../spark/sql/catalyst/AnalysisSuite.scala | 1 + .../sql/catalyst/DistributionSuite.scala | 1 + .../catalyst/ExpressionEvaluationSuite.scala | 1 + .../optimizer/ConstantFoldingSuite.scala | 1 + .../optimizer/FilterPushdownSuite.scala | 1 + .../apache/spark/sql/SparkSqlContext.scala | 10 ++- .../spark/sql/examples/SchemaRddExample.scala | 1 - .../spark/sql/examples/ViewsExample.scala | 2 - .../spark/sql/execution/basicOperators.scala | 14 +---- .../org/apache/spark/sql/DslQueryTests.scala | 4 +- .../org/apache/spark/sql/PlannerSuite.scala | 2 +- .../scala/org/apache/spark/sql/TgfSuite.scala | 2 +- 14 files changed, 52 insertions(+), 55 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala index e855052136fc6..92e64afa44c4c 100644 --- a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala @@ -12,7 +12,7 @@ object RDDRelation { val sc = new SparkContext("local", "RDDRelation") val sqlContext = new SqlContext(sc) - // Importing the SQL contexts give access to all the SQL functions and implicit conversions. + // Importing the SQL context gives access to all the SQL functions and implicit conversions. import sqlContext._ val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) @@ -34,5 +34,9 @@ object RDDRelation { println("Result of RDD.map:") rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect.foreach(println) + + + // Queries can also be written using a LINQ-like Scala DSL. + rdd.where('key === 1).orderBy('value.asc).select('key).toRdd.collect().foreach(println) } } \ No newline at end of file diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 76ccd1627e38b..c3d3056da0de7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -99,7 +99,7 @@ object ScalaReflection { * }}} */ package object dsl { - protected trait ImplicitOperators { + trait ImplicitOperators { def expr: Expression def + (other: Expression) = Add(expr, other) @@ -122,42 +122,44 @@ package object dsl { def as(s: Symbol) = Alias(expr, s.name)() } - implicit class DslExpression(e: Expression) extends ImplicitOperators { - def expr = e - } + trait ExpressionConversions { + implicit class DslExpression(e: Expression) extends ImplicitOperators { + def expr = e + } - implicit def intToLiteral(i: Int) = Literal(i) - implicit def longToLiteral(l: Long) = Literal(l) - implicit def floatToLiteral(f: Float) = Literal(f) - implicit def doubleToLiteral(d: Double) = Literal(d) - implicit def stringToLiteral(s: String) = Literal(s) + implicit def intToLiteral(i: Int) = Literal(i) + implicit def longToLiteral(l: Long) = Literal(l) + implicit def floatToLiteral(f: Float) = Literal(f) + implicit def doubleToLiteral(d: Double) = Literal(d) + implicit def stringToLiteral(s: String) = Literal(s) - implicit def symbolToUnresolvedAttribute(s: Symbol) = analysis.UnresolvedAttribute(s.name) + implicit def symbolToUnresolvedAttribute(s: Symbol) = analysis.UnresolvedAttribute(s.name) - implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } - implicit class DslString(val s: String) extends ImplicitAttribute + implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } + implicit class DslString(val s: String) extends ImplicitAttribute - abstract class ImplicitAttribute extends ImplicitOperators { - def s: String - def expr = attr - def attr = analysis.UnresolvedAttribute(s) + abstract class ImplicitAttribute extends ImplicitOperators { + def s: String + def expr = attr + def attr = analysis.UnresolvedAttribute(s) - /** Creates a new typed attributes of type int */ - def int = AttributeReference(s, IntegerType, nullable = false)() + /** Creates a new typed attributes of type int */ + def int = AttributeReference(s, IntegerType, nullable = false)() - /** Creates a new typed attributes of type string */ - def string = AttributeReference(s, StringType, nullable = false)() - } + /** Creates a new typed attributes of type string */ + def string = AttributeReference(s, StringType, nullable = false)() + } - implicit class DslAttribute(a: AttributeReference) { - def notNull = a.withNullability(false) - def nullable = a.withNullability(true) + implicit class DslAttribute(a: AttributeReference) { + def notNull = a.withNullability(false) + def nullable = a.withNullability(true) - // Protobuf terminology - def required = a.withNullability(false) + // Protobuf terminology + def required = a.withNullability(false) + } } - implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions + object expressions extends ExpressionConversions abstract class LogicalPlanFunctions { def logicalPlan: LogicalPlan @@ -212,8 +214,7 @@ package object dsl { def analyze = analysis.SimpleAnalyzer(logicalPlan) def writeToFile(path: String) = WriteToFile(path, logicalPlan) - - // TODO: for a loadFromFile it would be good to have a Catalog that knows - // how to resolve ParquetTables } + + implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala index 39f22d2e37bf2..1fd0d26b6f8bb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala @@ -27,6 +27,7 @@ import plans.logical._ import types._ import dsl._ +import dsl.expressions._ class AnalysisSuite extends FunSuite { val analyze = SimpleAnalyzer diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 660999f4c7ca9..5ec5b96e1b37d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.FunSuite import plans.physical._ import catalyst.dsl._ +import catalyst.dsl.expressions._ class DistributionSuite extends FunSuite { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala index 094143c9abb1d..0483a059bfe6e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala @@ -25,6 +25,7 @@ import types._ import expressions._ import dsl._ +import dsl.expressions._ class ExpressionEvaluationSuite extends FunSuite { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 6c7f067b516c0..8af4ba65d6d59 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -25,6 +25,7 @@ import plans.logical.{LogicalPlan, LocalRelation} import rules._ import expressions._ import dsl._ +import dsl.expressions._ class ConstantFoldingSuite extends OptimizerTest { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index b1ba0e0af507c..ec1c10f0f562f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -8,6 +8,7 @@ import rules._ import util._ import dsl._ +import dsl.expressions._ class FilterPushdownSuite extends OptimizerTest { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala index 9bbb87e5d57d9..072af1e156731 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.RDD import catalyst.analysis._ import catalyst.dsl -import catalyst.expressions.BindReferences +import catalyst.expressions._ import catalyst.optimizer.Optimizer import catalyst.planning.QueryPlanner import catalyst.plans.logical.{LogicalPlan, NativeCommand} @@ -57,7 +57,7 @@ object TestSqlContext * The entry point for running relational queries using Spark. Uses the provided spark context * to execute relational operators. */ -class SqlContext(val sparkContext: SparkContext) extends Logging { +class SqlContext(val sparkContext: SparkContext) extends Logging with dsl.ExpressionConversions { self => protected[sql] lazy val catalog: Catalog = new SimpleCatalog @@ -73,11 +73,15 @@ class SqlContext(val sparkContext: SparkContext) extends Logging { implicit def logicalPlanToSparkQuery(plan: LogicalPlan) = executePlan(plan) - implicit def logicalDsl(q: ExecutedQuery) = new dsl.DslLogicalPlan(q.logicalPlan) + implicit def logicalDsl(q: ExecutedQuery) = new DslLogicalPlan(q.logicalPlan) /** Allows the results of sql queries to be used as RDDs */ implicit def toRdd(q: ExecutedQuery) = q.rdd + // Expression implicits. Copied from dsl package object. + + implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends dsl.LogicalPlanFunctions + /** * Implicitly adds a `registerAsTable` to RDDs of case classes and allows the Query DSL to be * used on them. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala b/sql/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala index 9308737146523..d25d439cd4eaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala @@ -21,7 +21,6 @@ package examples import catalyst.plans.logical.LocalRelation import TestSqlContext._ -import catalyst.dsl._ object SchemaRddExample { def main(args: Array[String]): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala b/sql/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala index 0b6de528fb060..1c018f2e2a21b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala @@ -23,8 +23,6 @@ import catalyst.plans.Inner import catalyst.plans.logical._ /* Implicit Conversions */ -import catalyst.dsl._ - import TestSqlContext._ // For .toRdd execution using locally running sql instance. object ViewsExample { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index b799c813be2b0..c6d31d9abc9e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -122,19 +122,7 @@ object ExistingRdd { } def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { - data.mapPartitions { iter => - var currentCaseClass: A = null.asInstanceOf[A] - var mutableRow: MutableRow = null - - if(iter.hasNext) { - currentCaseClass = iter.next() - mutableRow = new GenericMutableRow(currentCaseClass.productIterator.size) - } - - ??? - - } - + // TODO: Reuse the row, don't use map on the product iterator. Maybe code gen? data.map(r => new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala index 86b117ffd815e..488dc0ddcdd17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala @@ -26,7 +26,7 @@ import catalyst.plans.logical.LogicalPlan import catalyst.types._ /* Implicits */ -import catalyst.dsl._ +import TestSqlContext._ object TestData { val testData = @@ -69,8 +69,6 @@ object TestData { } class DslQueryTest extends FunSuite { - import TestSqlContext._ - /** * Runs the plan and makes sure the answer matches the expected result. * @param plan the query to be executed diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala index 670519a916c4e..6a55a92bb8d4e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.FunSuite import catalyst.expressions._ import catalyst.plans.logical -import catalyst.dsl._ +import TestSqlContext._ class PlannerSuite extends FunSuite { import TestData._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala index c790243082825..365600e4468ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala @@ -26,7 +26,7 @@ import catalyst.plans._ import catalyst.plans.logical.LogicalPlan import catalyst.types._ -import catalyst.dsl._ +import TestSqlContext._ /** * This is an example TGF that uses UnresolvedAttributes 'name and 'age to access specific columns From f7d992db7ba126455069f48ce3fef2f95544095d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 13 Mar 2014 22:48:59 -0700 Subject: [PATCH 761/778] Naming / spelling. --- .../spark/sql/execution/{SharkPlan.scala => SparkPlan.scala} | 0 .../{ParquetQueryTests.scala => ParquetQuerySuite.scala} | 2 +- .../mapred/{SharkHadoopWriter.scala => SparkHadoopWriter.scala} | 0 .../apache/spark/sql/hive/{TestShark.scala => TestHive.scala} | 0 .../parquet/{ParquetQuerySuite.scala => HiveParquetSuite.scala} | 2 +- 5 files changed, 2 insertions(+), 2 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/{SharkPlan.scala => SparkPlan.scala} (100%) rename sql/core/src/test/scala/org/apache/spark/sql/parquet/{ParquetQueryTests.scala => ParquetQuerySuite.scala} (98%) rename sql/hive/src/main/scala/org/apache/hadoop/mapred/{SharkHadoopWriter.scala => SparkHadoopWriter.scala} (100%) rename sql/hive/src/main/scala/org/apache/spark/sql/hive/{TestShark.scala => TestHive.scala} (100%) rename sql/hive/src/test/scala/org/apache/spark/sql/parquet/{ParquetQuerySuite.scala => HiveParquetSuite.scala} (98%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala similarity index 100% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala similarity index 98% rename from sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala rename to sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 63cc02c7fe988..45b46d5f248a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQueryTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -31,7 +31,7 @@ import parquet.schema.MessageTypeParser import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil -class ParquetQueryTests extends FunSuite with BeforeAndAfterAll { +class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { override def beforeAll() { ParquetTestData.writeFile } diff --git a/sql/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala similarity index 100% rename from sql/hive/src/main/scala/org/apache/hadoop/mapred/SharkHadoopWriter.scala rename to sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala similarity index 100% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/TestShark.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala similarity index 98% rename from sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index c39e572e30253..ee90061c7cd0e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.hive.TestHive -class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { +class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { val filename = getTempFilePath("parquettest").getCanonicalFile.toURI.toString From ce8073b32d5a8713c5ad494baa1026c103e2882d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 13 Mar 2014 23:25:59 -0700 Subject: [PATCH 762/778] clean up implicits. --- .../scala/org/apache/spark/sql/catalyst/dsl/package.scala | 8 +++++--- .../sql/catalyst/optimizer/ConstantFoldingSuite.scala | 2 +- .../sql/catalyst/optimizer/FilterPushdownSuite.scala | 2 +- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index c3d3056da0de7..43632ac1189f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -212,9 +212,11 @@ package object dsl { analysis.UnresolvedRelation(None, tableName), Map.empty, logicalPlan, overwrite) def analyze = analysis.SimpleAnalyzer(logicalPlan) - - def writeToFile(path: String) = WriteToFile(path, logicalPlan) } - implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions + object plans { + implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions { + def writeToFile(path: String) = WriteToFile(path, logicalPlan) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 8af4ba65d6d59..7ce42b2b0a057 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -24,7 +24,7 @@ import util._ import plans.logical.{LogicalPlan, LocalRelation} import rules._ import expressions._ -import dsl._ +import dsl.plans._ import dsl.expressions._ class ConstantFoldingSuite extends OptimizerTest { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index ec1c10f0f562f..cd611b3fb3400 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -7,7 +7,7 @@ import plans.logical._ import rules._ import util._ -import dsl._ +import dsl.plans._ import dsl.expressions._ class FilterPushdownSuite extends OptimizerTest { From 2f224546a0c3e0713de359727e92d727bd41091e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 13 Mar 2014 23:26:15 -0700 Subject: [PATCH 763/778] WIP: Parquet example. --- .../spark/sql/examples/RDDRelation.scala | 39 ++++++++++++++++++- .../apache/spark/sql/SparkSqlContext.scala | 16 ++++++-- 2 files changed, 51 insertions(+), 4 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala index 92e64afa44c4c..5cfe077301eb9 100644 --- a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala @@ -35,8 +35,45 @@ object RDDRelation { println("Result of RDD.map:") rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect.foreach(println) - // Queries can also be written using a LINQ-like Scala DSL. rdd.where('key === 1).orderBy('value.asc).select('key).toRdd.collect().foreach(println) + + // Write out an RDD as a parquet file. + rdd.writeToFile("pair.parquet") + + /** + * Read in parquet file. Parquet files are self-describing so the schmema is preserved. + * + * WIP: The following fails. + * [error] Exception in thread "main" parquet.io.ParquetEncodingException: pair.parquet invalid: all the files must be contained in the root file:/Users/marmbrus/workspace/spark/examples/pair.parquet + * [error] at parquet.hadoop.ParquetFileWriter.mergeFooters(ParquetFileWriter.java:354) + * [error] at parquet.hadoop.ParquetFileWriter.writeMetadataFile(ParquetFileWriter.java:342) + * [error] at org.apache.spark.sql.parquet.ParquetTypesConverter$.writeMetaData(ParquetRelation.scala:265) + * [error] at org.apache.spark.sql.parquet.ParquetRelation$.create(ParquetRelation.scala:149) + * [error] at org.apache.spark.sql.execution.SparkStrategies$BasicOperators$.apply(SparkStrategies.scala:213) + * [error] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:60) + * [error] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:60) + * [error] at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) + * [error] at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:61) + * [error] at org.apache.spark.sql.SqlContext$QueryExecution.sparkPlan$lzycompute(SparkSqlContext.scala:155) + * [error] at org.apache.spark.sql.SqlContext$QueryExecution.sparkPlan(SparkSqlContext.scala:155) + * [error] at org.apache.spark.sql.SqlContext$QueryExecution.executedPlan$lzycompute(SparkSqlContext.scala:156) + * [error] at org.apache.spark.sql.SqlContext$QueryExecution.executedPlan(SparkSqlContext.scala:156) + * [error] at org.apache.spark.sql.SqlContext$QueryExecution.toRdd$lzycompute(SparkSqlContext.scala:159) + * [error] at org.apache.spark.sql.SqlContext$QueryExecution.toRdd(SparkSqlContext.scala:159) + * [error] at org.apache.spark.sql.SqlContext$TableRdd.writeToFile(SparkSqlContext.scala:96) + * [error] at org.apache.spark.sql.examples.RDDRelation$.main(RDDRelation.scala:41) + * [error] at org.apache.spark.sql.examples.RDDRelation.main(RDDRelation.scala) + */ + val parquetFile = sqlContext.loadFile("pair.parquet") + + // Queries can be run using the DSL on parequet files just like the original RDD. + parquetFile.where('key === 1).select('value).toRdd.collect().foreach(println) + + // These files can also be registered as tables. + parquetFile.registerAsTable("parquetFile") + sql("SELECT * FROM parquetFile").collect().foreach(println) + + } } \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala index 072af1e156731..5c5c7082fc126 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala @@ -28,7 +28,7 @@ import catalyst.dsl import catalyst.expressions._ import catalyst.optimizer.Optimizer import catalyst.planning.QueryPlanner -import catalyst.plans.logical.{LogicalPlan, NativeCommand} +import catalyst.plans.logical.{LogicalPlan, NativeCommand, WriteToFile} import catalyst.rules.RuleExecutor import execution._ @@ -80,7 +80,11 @@ class SqlContext(val sparkContext: SparkContext) extends Logging with dsl.Expres // Expression implicits. Copied from dsl package object. - implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends dsl.LogicalPlanFunctions + implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends dsl.LogicalPlanFunctions { + def registerAsTable(tableName: String): Unit = { + catalog.registerTable(None, tableName, logicalPlan) + } + } /** * Implicitly adds a `registerAsTable` to RDDs of case classes and allows the Query DSL to be @@ -89,11 +93,17 @@ class SqlContext(val sparkContext: SparkContext) extends Logging with dsl.Expres implicit class TableRdd[A <: Product: TypeTag](rdd: RDD[A]) extends dsl.LogicalPlanFunctions { def logicalPlan = SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd)) - def registerAsTable(tableName: String) = { + def writeToFile(path: String) = { + WriteToFile(path, logicalPlan).toRdd + } + + def registerAsTable(tableName: String): Unit = { catalog.registerTable(None, tableName, logicalPlan) } } + def loadFile(path: String): LogicalPlan = parquet.ParquetRelation("ParquetFile", path) + /** * Executes a SQL query using Spark, returning the result as an RDD as well as the plan used * for execution. From c01470fa14e75fbbea72b0c244515d1f2cdb26cb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 14 Mar 2014 10:07:50 -0700 Subject: [PATCH 764/778] Clean up example --- .../spark/sql/examples/RDDRelation.scala | 27 +------------------ 1 file changed, 1 insertion(+), 26 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala index 5cfe077301eb9..6f28fe46e677f 100644 --- a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala @@ -41,30 +41,7 @@ object RDDRelation { // Write out an RDD as a parquet file. rdd.writeToFile("pair.parquet") - /** - * Read in parquet file. Parquet files are self-describing so the schmema is preserved. - * - * WIP: The following fails. - * [error] Exception in thread "main" parquet.io.ParquetEncodingException: pair.parquet invalid: all the files must be contained in the root file:/Users/marmbrus/workspace/spark/examples/pair.parquet - * [error] at parquet.hadoop.ParquetFileWriter.mergeFooters(ParquetFileWriter.java:354) - * [error] at parquet.hadoop.ParquetFileWriter.writeMetadataFile(ParquetFileWriter.java:342) - * [error] at org.apache.spark.sql.parquet.ParquetTypesConverter$.writeMetaData(ParquetRelation.scala:265) - * [error] at org.apache.spark.sql.parquet.ParquetRelation$.create(ParquetRelation.scala:149) - * [error] at org.apache.spark.sql.execution.SparkStrategies$BasicOperators$.apply(SparkStrategies.scala:213) - * [error] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:60) - * [error] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:60) - * [error] at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) - * [error] at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:61) - * [error] at org.apache.spark.sql.SqlContext$QueryExecution.sparkPlan$lzycompute(SparkSqlContext.scala:155) - * [error] at org.apache.spark.sql.SqlContext$QueryExecution.sparkPlan(SparkSqlContext.scala:155) - * [error] at org.apache.spark.sql.SqlContext$QueryExecution.executedPlan$lzycompute(SparkSqlContext.scala:156) - * [error] at org.apache.spark.sql.SqlContext$QueryExecution.executedPlan(SparkSqlContext.scala:156) - * [error] at org.apache.spark.sql.SqlContext$QueryExecution.toRdd$lzycompute(SparkSqlContext.scala:159) - * [error] at org.apache.spark.sql.SqlContext$QueryExecution.toRdd(SparkSqlContext.scala:159) - * [error] at org.apache.spark.sql.SqlContext$TableRdd.writeToFile(SparkSqlContext.scala:96) - * [error] at org.apache.spark.sql.examples.RDDRelation$.main(RDDRelation.scala:41) - * [error] at org.apache.spark.sql.examples.RDDRelation.main(RDDRelation.scala) - */ + // Read in parquet file. Parquet files are self-describing so the schmema is preserved. val parquetFile = sqlContext.loadFile("pair.parquet") // Queries can be run using the DSL on parequet files just like the original RDD. @@ -73,7 +50,5 @@ object RDDRelation { // These files can also be registered as tables. parquetFile.registerAsTable("parquetFile") sql("SELECT * FROM parquetFile").collect().foreach(println) - - } } \ No newline at end of file From 013f62a2eb59e76510d06d6e8b2ab6a882bdb598 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 14 Mar 2014 10:31:34 -0700 Subject: [PATCH 765/778] Fix documentation / code style. --- sql/README.md | 48 ++++++------------- .../sql/parquet/ParquetTableSupport.scala | 6 ++- 2 files changed, 18 insertions(+), 36 deletions(-) diff --git a/sql/README.md b/sql/README.md index 9ee6407078227..4418b2fc4a4ad 100644 --- a/sql/README.md +++ b/sql/README.md @@ -1,37 +1,17 @@ -catalyst -======== -Catalyst is a functional framework for optimizing relational query plans. +Spark SQL +========= -[![Build Status](https://magnum.travis-ci.com/databricks/catalyst.png?token=sNeje9KkkWMHYrVqko4t&branch=master)](https://magnum.travis-ci.com/databricks/catalyst) +This module provides support for executing relational queries expressed in either SQL or a LINQ-like Scala DSL. -More documentation can be found in the project's [scaladoc](http://databricks.github.io/catalyst/latest/api/#catalyst.package) +Spark SQL is broken up into three subprojects: + - Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions. + - Execution (sql/core) - A query planner / execution engine for translating Catalyst’s logical query plans into Spark RDDs. This component also includes a new public interface, SqlContext, that allows users to execute SQL or LINQ statements against existing RDDs and Parquet files. + - Hive Metastore Support (sql/hive) - An extension of SqlContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allows users to run queries that include Hive UDFs, UDAFs, and UDTFs. -Getting Started -=============== -Catalyst is built using [SBT](https://github.com/harrah/xsbt). The SBT launcher is included in the distribution (bin/sbt) and is responsible for downloading all other required jars (scala compiler and other dependencies). -SBT commands can be invoked from the command line. For example, to clean and build a jar, you would run the following command: - - catalyst/$ sbt clean package - -Additionally, if you are going to be running several commands, you can use SBT from an interactive console, which amortizes the cost of starting the JVM and JITing SBT and the scala compiler. For example: - -``` -$ sbt/sbt -[info] Loading project definition from /Users/marmbrus/workspace/catalyst.clean/project -[info] Set current project to default-1207ac (in build file:/Users/marmbrus/workspace/catalyst.clean/) -> clean -> test:compile -[info] Compiling 10 Scala sources to catalyst/target/scala-2.10/test-classes... -[success] Total time: 15 s, completed Dec 20, 2013 12:00:06 PM -> core/test-only catalyst.execution.BasicQuerySuite -``` - -Any command that is prefixed with a `~` (e.g. `~compile`) will be run automatically in a loop each time any dependent files have changed. - -Other dependencies ------------------- -In order to run all of the test cases or interact with sample data, you will need to set several environmental variables. +Other dependencies for developers +--------------------------------- +In order to create new hive test cases , you will need to set several environmental variables. ``` export HIVE_HOME="/hive/build/dist" @@ -41,10 +21,10 @@ export HADOOP_HOME="/hadoop-1.0.4" Using the console ================= -An interactive scala console can be invoked by running `sbt/sbt shark/console`. From here you can execute queries and inspect the various stages of query optimization. +An interactive scala console can be invoked by running `sbt/sbt hive/console`. From here you can execute queries and inspect the various stages of query optimization. ```scala -catalyst$ sbt/sbt shark/console +catalyst$ sbt/sbt hive/console [info] Starting scala interpreter... import org.apache.spark.sql.catalyst.analysis._ @@ -56,8 +36,8 @@ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution -import org.apache.spark.sql.shark._ -import org.apache.spark.sql.shark.TestShark._ +import org.apache.spark.sql.hive._ +import org.apache.spark.sql.hive.TestHive._ Welcome to Scala version 2.10.3 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45). Type in expressions to have them evaluated. Type :help for more information. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index db35b6a42976a..c2ae18b882d19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -140,9 +140,11 @@ object RowWriteSupport { * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. * - * @param schema The corresponding Shark schema in the form of a list of attributes. + * @param schema The corresponding Catalyst schema in the form of a list of attributes. */ -class CatalystGroupConverter(schema: Seq[Attribute], protected[parquet] val current: ParquetRelation.RowType) extends GroupConverter { +class CatalystGroupConverter( + schema: Seq[Attribute], + protected[parquet] val current: ParquetRelation.RowType) extends GroupConverter { def this(schema: Seq[Attribute]) = this(schema, new ParquetRelation.RowType(schema.length)) From c2efad69d2013c4a8557874b9b1260ea7ae8dafc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 14 Mar 2014 13:14:01 -0700 Subject: [PATCH 766/778] First draft of SQL documentation. --- docs/_layouts/global.html | 9 +++ docs/_plugins/copy_api_dirs.rb | 13 ++++ docs/index.md | 1 + docs/sql-programming-guide.md | 128 +++++++++++++++++++++++++++++++++ 4 files changed, 151 insertions(+) create mode 100644 docs/sql-programming-guide.md diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 49fd78ca98655..f63e7fe189aa6 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -69,6 +69,7 @@
  3. Bagel (Pregel on Spark)
  4. GraphX (Graph Processing)
  5. +
  6. Spark SQL
  7. @@ -82,6 +83,14 @@
  8. MLlib (Machine Learning)
  9. Bagel (Pregel on Spark)
  10. GraphX (Graph Processing)
  11. +
  12. Spark in Python
  13. Spark Streaming
  14. +
  15. Spark SQL
  16. MLlib (Machine Learning)
  17. Bagel (Pregel on Spark)
  18. GraphX (Graph Processing)
  19. -
  20. Spark SQL
  21. @@ -80,17 +80,17 @@
  22. Spark Core for Python
  23. Spark Streaming
  24. -
  25. MLlib (Machine Learning)
  26. -
  27. Bagel (Pregel on Spark)
  28. -
  29. GraphX (Graph Processing)
  30. +
  31. MLlib (Machine Learning)
  32. +
  33. Bagel (Pregel on Spark)
  34. +
  35. GraphX (Graph Processing)
  36. MLlib (Machine Learning)
  37. T#C<=3 zv1~>%h-PHo1YPpE-{-zp|+0~Hl>LzK+pP2 zgV3U!2U%0qZNpKMAc6No8;pYoP70$Nl5HC<=Z?eP=g*zwOsz*BiU=QZ%LVlo4tTbk z2n)5nA8750-k$V4GHfYjOqvG4GoLaE6#)h58^7q6OS*0gPT51wn{DriPv_m1>Fei? z+oQMR){F})uS8cXa8~G|-zO*)>RVY%1l9Tw{)iR0#A0hi!4+qx)F=K?U3I?f%ry%< zz-gnE(Xot;n5mi;#lGD}jnjg_HU(_ykV<@*xEvW{DHd4pbfoh0Z2W;*zqEv015SqE zfYQl!>X<_hyFbnSG4eLTIWXgj<$@ zOi<-WVJlO9F6StQ%CAU}?e|S_nRken2uq&_!|W9%|4e9!rX+b}ft(D~7T6C~_rYP|0lOZ*^!(D|01z|6J=`_a&> z$uChggW1~Lj1`H@CWzRriVFm<+9R!ul}K&@f%OK{UO`)|k`GHyXEjvsow7hBekm}J z_?*~O%9<^14-95mWq3%GrEb=~}0lM|Ti0 zFFw80HN35_>xXF?h&7dN6*Q_RtUzSu{R zH$}LyGN^x!2Q);~HA6LZRwkm2iC?$NHce4BUNL~~fFR9l$=6<*4z)=`lv47#WKxP| zSX%v3#Bx2R`%%T@{#mx0^If#Sl8Dz`y7h)HV5EIQcnEuAS@R#8tFeDIH2UzqZ^|{Q z1fR+4BO;fg#>=An@TP#e>x3bquyp}%I4KPj<8Lb(!?5)|$h+I~2ZmmQlqT-7>>H0? zDn}%iA*qy^Y}P%%C%hX&>IlH0%d{LpQJ+~U(G1nT6BUb-F3BixI_KHzBaUm$2X_$> z7RvJrISGF?f>RI&SEVHdQ{x5#PCjp@)FI-tLs)nl^(iWS(BfhFX*s)(m*uu#Nd{`> zF9@VU++bMHt>LIcuFYZ3NmuBh71H+t=Adx=Cp!O2wbO_1t4jAZnF}6S-U%b}*@(G@ zq85SW5%ZU7!QJIZ2B-Af2PdLl_XlEOeztOgerh7?6e0}iQEr*UnteS9N?hPMP zdS~)PBGs?f0=mqFGlPREC|35wBHuDFb>NuvoU#P@cO$&}VX7FuK&X#c%jWKssCf*{)?%6c#`n*N8 zYNzU>M5Sl1FJY*C-2w26R(%CUevH(WN6Cs$M|B?z0sMuQu7DOwR0gnnM`=^>2^G|B zC-^tnO;wXu`C2ThJ#lcmm-JL~Pm`;SP`{Q@Ydwu+$t=>yN<3&;#})ffJP{+Yn7D z>6O0;)M)muQvU1!V~`huuwVJ`O7fxXf!H_25+QlVJg35s?4KO;Z7u`fJ?uOvsQcnf zW*+JAwQgmjsx(L?zDP{0Z!2`K|JpO*?C3JTtLu>lvNOK$jMOn}*>ndgQ_*1t+Z$eP zf6*{V7D<2+hu&YoeS2-OFdy?$onhg9P$!80Z4mAe0@a8iW=RJEyfWTI2;zY*G#5{fjA_@L>P^6=yMlc zB`O$uLtL_rFoSmT{1@xbhILucmynLnVD4QYmoZJU%BNqQ$Xwl9!*sWy4?3OXUe}%Z4{iV;?ryVj_yLOy$ptzDjA`+#PmwFT>(}b2QJYcCz|1 zCw;H=zoB)~&X|LY$PDSC^v~lpFRk$JIE8k< zz4@Snr|--KYP)@v@4UQ^r@Or1!Eno5^I@mCQ4F_RgDP%Y{ zf+sMpJlU^nKNx8L%GNkv4f!#J3VGw+Wp0v{->~K&Q*6=_EKS>UIL>ZX(r>BqSip_E zGP_h!6`0PbkUAT_p`=RZ51mmwn5G$i!R?(Mt}d**dEr{ncN}B~9V+ z0iy+vg_b}wZO>{)bKwlFn~ZTw$rw^|9vZa5Ny|`^O0u33N{dk&xMf>^78T<8dfd)3 zEJiJu8J)!%dc&C1^=B=Ry>&=R)Sa%Z--sR-m3ql?E68ykDmu@Om*n>?=9pZ(~{B+SxX5z}n-mURvWV+@{Hjb-& z!!9#7M}W6ObYea91H{WTV>-!_zSdlLK_+qD_0^RpZt3MEXN!%8SeWuT2P5?zt*)ky5k1x1JDf!J zo)#Ol*zB0t3hv!Bx@IwA?#{d1dL)YB0a4F&IKTN&o^RD~MmD+N%9GNtTqfK|sQUWM zWvy6(-$K5ayZ5a+E=>UtJS96{JxbXSgS%3xN(YG((0GzZ8Z#>%ynCqdlCU`yl-9hk zv$E>IwFz3Gf<5ijPdi{tYh8-<`fITXL!8a=o=a4Xp@=XkzkR*Zhdb#VvX9d?&`%t-jxm#)k5VHahYJEvZ0f|706tEKbTY2 z&|RdteV3$Hqn#OssZip%lE`Iip9`#QsHvj(OB>fmjEq)S7Pl`K`T=!4&Is#LFTPXn z&4ddraXJr<_^G3VSsO;h?oQ)5npcU>=Zt$Um{~(!s6O$2{c>W)G4zIBj9|X9Fj&0$*JeTGJO>6J^tSh75Vs<|q==G+vVk2%+Dbrs0_OiNE^va6^WJI<*=R zyeJ@tlWEligc5F zV;|SqXlTZPrzCZTmWSp)iSU1M@01AtQ-34gDIfl~^>;u!<-vcX`D^;O^*1{F{e)AH i`@OJJko!mNzo!2$tg|p-ID2ZR`xV2#Mox~?)xQ8ebNdqj literal 0 HcmV?d00001 diff --git a/src/test/hive/data/files/UserVisits.dat b/src/test/hive/data/files/UserVisits.dat new file mode 100644 index 0000000000000..f56c5a1b67b36 --- /dev/null +++ b/src/test/hive/data/files/UserVisits.dat @@ -0,0 +1,55 @@ +170.131.22.2|13rdgckzlcblruc.html|1984-8-7|336.869186722|NuSearch Spider|HUN|HUN-NL|remnants|3 +162.114.4.2|6xpirzjeytxdjsmwtmyeugkesratmpvamliekrijlgmvyyrslqwgw.html|1978-1-9|331.791153595|Superdownloads Spiderma|AUT|AUT-ZR|MHD|8 +177.110.45.18|11zvmoamsyaameokoeylbkivgquksibqbalnpmailbiyfxitbhfdroyxesixbjndkyqzl.html|1986-9-25|411.968497603|Mozilla/4.0|FLK|FLK-GB|apj@as.arizona.edu.|7 +157.111.12.37|44mvdnls.html|2002-7-3|486.660926201|PHP/4.0.|FIN|FIN-CZ|diffuse|3 +161.100.45.22|14ceyigx.html|1978-10-26|399.80234522|NP/0.1|BEN|BEN-CA|region|8 +164.118.48.16|28axfinfqwdcwoorukpwqvqoxxeuivbniclnkytavwdslrj.html|1996-12-8|298.335411612|MSNBOT_Mobile MSMOBOT Mozilla/2.0|USA|USA-IO|medium|1 +153.104.13.11|19aysprojntmnwymfdkaznbqxprxxaissjqkzhzivsvipuvuxfuxsvnqlfnigvby.html|1976-10-6|146.309480768|WebSearch.COM.AU/3.0.1|MCO|MCO-YD|state|5 +150.112.45.27|12hcaewxiswjeezfnlulkenwubaqsitpuarufosogoxls.html|1995-6-19|173.469334335|WinkBot/0.06|PHL|PHL-NN|important|2 +152.108.39.16|36umg.html|1974-3-28|269.969215988|GSiteCrawler/v1.xx rev. xxx|MNG|MNG-HI|...)|6 +174.119.41.16|60yxoboskwpyfin.html|2002-7-17|436.113482675|Infoseek SideWinder/2.0B|NIC|NIC-JP|data|1 +165.116.21.12|70difiadhmrvragggmoaufnuwwbakbjntnwzvxcdjtybufiarwbmcphzmizwkikewh.html|1984-2-6|13.099044572|WWWeasel Robot v1.00|THA|THA-FO|bubbles|6 +155.128.42.14|21brkepinqiwvtmfmebjckkhwevhxaesogkykzgyqpuligrul.html|1986-7-29|347.800952938|Mozilla/4.0 compatible ZyBorg/1.0|IRN|IRN-YS|conduction|1 +156.131.31.12|14nbaaoablhxrlvbfgrwcxktvshtkoqzddbdepegbmesxztdglzjjkc.html|2002-7-30|85.7691140217|Java1.1.xx.|BRA|BRA-BL|circumstellar|9 +159.122.42.18|4xfydvopxveeduudfzodxkbczvdlzou.html|1989-9-20|332.572440865|Metaeuro Web Crawler/0.2|LUX|LUX-SD|kinematics|7 +151.104.39.45|65psclahgvasawczpyicyxkuqzwpbowghmzkxzsdvtwwpzvfydiwbsqrrmhtbezjqyuo.html|2002-1-13|190.528735328|JobSpider_BA/1.|UGA|UGA-PY|pulsars:|7 +159.132.24.22|18vhcbzhhblfbayejcybyibwqsgzlkmswizyjzgrbrw.html|1978-1-2|182.368755789|Piffany_Web_Scraper_v0.|ITA|ITA-NJ|nonthermal|1 +170.101.17.16|40prmxavsjoizdzkgsncesndxebatfwvrmmejnacxol.html|1989-9-1|41.4163486896|Mozilla/4.01 [en]|ZAF|ZAF-AK|Scuti|6 +171.124.38.2|29nripzogexadckoiaoafxvtkrxksdqgveydtxsabpbfsltbmibrfwlqojagmr.html|1979-6-12|192.085693167|IconSurf/2.0 favicon monitor|SVN|SVN-DY|systems|5 +178.128.29.41|24tmrndfialwvkwybuspjyexlkiamebwtvilimqqncnimkgofzepximj.html|2000-7-8|276.89796127|obidos-bot|SLB|SLB-RL|(...|4 +175.101.24.43|70dcfbcotdzhfhuhquyosbcviglrkrakddmifpxzswg.html|1978-3-16|131.775726872|Mozilla/4.0|BMU|BMU-BR|spiral|6 +155.102.37.30|99cyllzbnsowifxdxsdmiseiceeriaaoucmgnlhaewxmbvqynulwmpepujhckhqfjdmxpuyt.html|1975-5-4|311.052004479|WebSearch.COM.AU/3.0.1|NLD|NLD-GX|Herbig-Haro|6 +156.105.11.18|1nczmzpivhbgn.html|1992-9-19|36.9747263531|Search/1.0|GLP|GLP-DJ|observations|3 +164.115.38.23|79bvcojctkaugbcterbzfykwvesklokgilbkalntvoocqqvuixunvekqjcburlbzxckxnyrjm.html|1991-4-20|267.047961774|Journster.com RSS/Atom aggregator 0.5|HKG|HKG-PK|radio|2 +179.133.2.36|12azizhsdhdgdpidjgmdeyzmfhdwsbezbeyjegcioforvxvfehjigiulqyhizmhargkwmmeartsnrosvvbdbkynawvi.html|1999-12-9|481.463770712|LeechGet 200x|SCG|SCG-XF|instruments|8 +178.107.45.18|45mbziaowxegkhzcmbsyrextgqjbyezodmqduqrqnwxydwaqytopxmidcsfbwfparfemvwdjtaiwxjnvcclaotdrmjs.html|1983-4-13|51.6686671965|IlTrovatore/1.2|HND|HND-AN|dynamics|2 +162.117.17.14|17tkabzxynnqswezhqmkvrlfycpmxqowlhgligihuwxmscmasylopwuozjawaotlwaxfggmack.html|2001-12-24|161.048060104|Mozilla/4.5 [en]C-CCK-MCD {TLC;RETAIL}|RWA|RWA-QE|rays|9 +178.119.40.7|48amqtmqxsjgrmjkszztfpegqzapidysnze.html|1987-4-3|492.988714137|Mozilla/4.0|AUT|AUT-ZR|cosmology:|8 +160.119.18.18|15yufqaoxpuqwb.html|1979-7-22|394.694548614|scooter-venus-3.0.vn|MCO|MCO-YD|outflows|1 +162.112.21.25|21boum.html|1991-2-6|165.368136543|LinkProver 2.|TCA|TCA-IS|spots|8 +176.112.31.17|20gblxgjcvpu.html|1991-8-5|78.2740990152|Mozilla/4.0|BMU|BMU-BR|masses|2 +166.130.12.13|9izokfebomgsiifyzrsepbbemutvj.html|2003-12-5|188.600736756|WWW-Mechanize/1.1|TGO|TGO-WB|bursts|5 +171.100.18.39|97sxfsgahjujwzlszmxkahyslcobrrlx.html|1985-11-21|143.277058506|Overture-WebCrawler/3.8/Fresh|SAU|SAU-KL|interferometric|5 +152.122.43.35|85zdszgzonsxkqbrkthtceiuzjsedwvghvkzvqzj.html|1989-12-1|315.628996565|moget/x.x|UMI|UMI-VU|Galaxy:|2 +157.133.36.37|15xnilzhtqjsxhhbzazrflznupllyhvdbsqjeqqyharfiyhhyhzdszrnpcyoktslljvqam.html|1990-3-20|426.498017786|PrivacyFinder/1.|UZB|UZB-ZJ|nebulae|7 +161.134.11.11|96kvrofepctfbesrphjiznjktygntkkubupsjvxyxrdzvwrkeasdobohauvueg.html|1984-6-6|280.039128409|Waypath development crawler - info at waypath dot co|IDN|IDN-BH|supergiants|6 +163.123.23.13|19rkrtwumqwmnnzisxyeesqacwolpypyxhipaejnvfzitzrlwqqbigblcqxrpnqmuybudkiyqhhjgzvdpleysg.html|1977-10-11|86.3390049695|Opera/5.0|LSO|LSO-PW|testing|7 +166.126.40.21|52ejufqiidwioozorbnsjxezfwaucndbihldnblvehdtwchoeuhoslnyioslbwmkdynrzymegpy.html|1990-10-20|125.582281932|Mozilla/4.0|BTN|BTN-HP|catalogs|9 +158.133.10.19|87nzdhsnzhkylakazmkvctgaaxtrafpxscxvjqijxthitrj.html|1982-10-5|481.583542862|larbin|GAB|GAB-CS|angular|8 +173.104.45.8|49sdptdphxjlbiwrbbrsebwqquadx.html|1981-5-2|41.3182727245|LECodeChecker/3.0 libgetdoc/1.|AUS|AUS-AV|bands|6 +160.101.31.43|6lrepnctlanokfhla.html|1973-9-7|133.29867101|sogou develop spide|SWE|SWE-TM|time|5 +150.127.33.8|22oeawpxhqahkvtaecwp.html|1999-3-16|398.882494477|W3C-WebCon/5.x.x libwww/5.x.|ISR|ISR-DY|history|1 +154.114.47.36|2mzzsgievabpkaoqegadbbjxwkutdisnvrmox.html|1981-7-24|332.760102125|mammoth/1.0|AUT|AUT-ZR|FUNCTION|3 +155.108.15.24|22beewtbnpw.html|1996-6-7|393.470347637|Scrubby/3.0|ABW|ABW-NB|horizontal-branch|4 +177.120.40.39|48itvyjulckeddslsuayoguojzhvqvmfgvyctiwflhj.html|1977-8-12|239.601807636|webmeasurement-bot, http://rvs.informatik.uni-leipzig.d|WSM|WSM-UF|are|3 +179.123.41.31|46eppnympstjuhivvpritvotqmivgsfmdkbtxafns.html|2001-11-26|258.55616439|Mozilla/2.0|SYR|SYR-XP|photometric|1 +175.100.9.4|32fjrnrlabonc.html|1988-10-22|344.394849153|Snapbot/1.|GUF|GUF-KP|acceleration|2 +155.126.7.17|72wufwnsdsqncftnvdcunnknzqnaiyflmcgsytkbmbpogicblew.html|1981-12-5|398.334494319|UKWizz/Nutch-0.8.1|NIC|NIC-JP|Kuiper|4 +150.118.20.31|1mbyargbxtnjtivflxzzredcfbtehxbxjcwkucmrwaaqiwvutuulzxnezhi.html|1982-8-27|168.936669894|Mozilla/4.0|IRL|IRL-NN|cataclysmic|5 +177.116.39.36|84maivbmcqggefkjtsde.html|1982-6-11|88.121669797|Mozilla/4.0|ARE|ARE-MX|instruments|1 +168.119.19.26|73vhjursdvxateuvrxsspwwfdbsoqfegeannuegyadzuitparisgfomiqfxhkcnocacxfivfmuzuopvfynmdcyl.html|1991-11-17|397.829289621|webbandit/4.xx.|NIC|NIC-JP|dust|2 +154.100.36.32|57rylepuglpfqvjwkxgrtftvqkjzjwsznjyzontuzizqdimofsfzxzuojeot.html|1999-1-5|334.714055649|RRC|GTM|GTM-VH|blue|7 +153.112.2.11|6pkwxtlgkkxoqtxpgrullqxjauquvmlkcwhzpsgzdeotymieddqpu.html|1975-8-6|348.218411093|Wotbox/alpha0.6|MNP|MNP-UD|supernovae:|6 +150.107.15.22|53gohsgrvrjgfptttlpfipgsnijsrhxsyeggwnysfhykxrdqdsvlicdwkmpcumut.html|1978-8-2|355.771603423|Mozilla/3.0|DEU|DEU-PU|stars|4 +150.126.27.44|0rgxbnwiqebsmszpkvfpxvhkleebngzxxgvzt.html|1989-5-18|467.800755054|Mozilla/3.01|ZWE|ZWE-TS|system|3 +151.101.32.3|34btbqii.html|1998-8-1|131.055972797|Orca Browser|THA|THA-FO|late-type|5 diff --git a/src/test/hive/data/files/apache.access.2.log b/src/test/hive/data/files/apache.access.2.log new file mode 100644 index 0000000000000..cd7fea4e3fb5d --- /dev/null +++ b/src/test/hive/data/files/apache.access.2.log @@ -0,0 +1 @@ +127.0.0.1 - - [26/May/2009:00:00:00 +0000] "GET /someurl/?track=Blabla(Main) HTTP/1.1" 200 5864 - "Mozilla/5.0 (Windows; U; Windows NT 6.0; en-US) AppleWebKit/525.19 (KHTML, like Gecko) Chrome/1.0.154.65 Safari/525.19" diff --git a/src/test/hive/data/files/apache.access.log b/src/test/hive/data/files/apache.access.log new file mode 100644 index 0000000000000..14c341efc16ed --- /dev/null +++ b/src/test/hive/data/files/apache.access.log @@ -0,0 +1 @@ +127.0.0.1 - frank [10/Oct/2000:13:55:36 -0700] "GET /apache_pb.gif HTTP/1.0" 200 2326 diff --git a/src/test/hive/data/files/archive_corrupt.rc b/src/test/hive/data/files/archive_corrupt.rc new file mode 100644 index 0000000000000000000000000000000000000000..ab1f2ef25f1adb309a9434c90a749db023cab591 GIT binary patch literal 1374 zcmWG`4P?{JFG|--EJ#ewNY%?oOv%qL(96gyOVumP(aX%&3vzbL%t=-8POWq*O-oBH z(kDiBSYl3TDnvaagF2=Tpt;HUxdlb3#l?E=Rhb3O`6;Q%3=9m6GGLp5(nZN>Kx=`j za!Paa^zuq`lTwQqjdW%t)yB_0@Vlq9GyA*wbsGi-4t=1mxj?P=U7f7guFzo=b1m2IwqBG8r!zob&l;Vq1?2` z-yAq)YE@nM7498jSaY#2LvHOy(QV!HAI5Oz&g!{)?Zx%3@P12#3wk-M> zBNd;x`-kMYsTY==yph{mx3)r6<6b@Y%I2r~%JtrOO%| z`+%b}`{yz;Fd%%VI^}lb5d#k9=tsQo_HUF;-T!XVRz^=}rT1!glWTXqaV>wdclGKS z-=?R&%iSm$XqkWP@uADvrI<5SQ&H8oMPaf=De6P*+efOp>Ge51}@}fihp?==p z$)9G1PkA+ceN)txds^ooUA%UG^~?>I-yWaY88vm+^TQv{ZqL+T-Yj-<%f!PhZx7#H zmm8lR;Uc+qUa0x!<@bDxO%L6AId%F`{qxrkh0K-zzR&aSUV-4kX|WxXH=UMa`&8L^ zsQuo#p(i^7xVyB6AJE_?oD?a`_3OMWV9 z9y{gLclY$U{nxJ?+q_z3(@Z-(3vmZ9C|SKDNTEd5?LH`S`U+)HX~`FSu@ce{9lf!uYEA63_1 zNw&KjFIEZq{`L5ERwGH4_j!*ax!O*ZP5wVSQ(57poVL@Y5NY$7VispBgL@673t~=~ zylF3)@X~IlIoAeT_5;)6)b74W)pJ!z_^_=iQ|m`Vq0kFASJrK(G+(UJT9;tocx=^7 z&)wb}e5F?RmkT%@+^U(nJ8R3i+a~N6-WR{9NjutT9u+>8ZmRxN;SZ%|ovwf8pXp9g;H zyl{*tbZtw>y3u&);E6}h*)Mjj-XA5f+c#^ahb<@b#)`Q~HZMCAU6-1P%7~VT?zuE? z+9jT6IwD7Rf9^Q(OfT1=sJ2c+W@@%PSGl_1A&sYO$IH)@P2RF-^AeH6Z8r6L{W#no z1_#EOmHs|vdZ{ktMyUGneTOx@x;yTaZx(74`sLiPLumD?jY$GLk-Ib*Quo_z-#4u_ zYVW$p2kpt%d0W)lo|HHTZJyWf?QugxOXa4MH{UvLu97L41>1R6zxXnp``g6LX9NEj zs;^J_-8e5)P$lV)!t+H+MH}Dxd!3eFEaq*w`|v~OUuVAs?mgmE5Zh95Y>nV%zsT6- z@1)f7@)QEEH~Sp?QXss}Sn>UjxTVcjhssKq++HZ-_Dv&oOQW{=(MB%bwS_F!0*gm_rt1uvs8)uAuYj}n?Gl)%bUALG4&UweNU4;w^Yb(<=&SbC}#C^F5f$u&&#Lk$>xMk SetP4*{dKJ;oBVhg7`Omf;eTcT literal 0 HcmV?d00001 diff --git a/src/test/hive/data/files/array_table.txt b/src/test/hive/data/files/array_table.txt new file mode 100644 index 0000000000000..463365f71b81b --- /dev/null +++ b/src/test/hive/data/files/array_table.txt @@ -0,0 +1,2 @@ +a1,a2,a3 b1,b2,b3,b4 +a21,a22,a23 b21,b22,b23,b24 diff --git a/src/test/hive/data/files/binary.txt b/src/test/hive/data/files/binary.txt new file mode 100644 index 0000000000000..c6fbdee959041 --- /dev/null +++ b/src/test/hive/data/files/binary.txt @@ -0,0 +1,10 @@ +the quick brown fox jumped over the lazy little dog +today is nice outside +the quick brown fox jumped over the lazy little dog + +wikipedia is a great source of information +the quick brown fox jumped over the lazy little dog + +estimating the number of distinct values is a hard problem + +the quick brown fox jumped over the lazy little dog diff --git a/src/test/hive/data/files/bool.txt b/src/test/hive/data/files/bool.txt new file mode 100644 index 0000000000000..065359b9f9325 --- /dev/null +++ b/src/test/hive/data/files/bool.txt @@ -0,0 +1,33 @@ +true +false +true +true +true +false +false +false +false +true +true +true +true +false + +false +true +true +false +false +false +false +false +false +false +false +true +false +false +false +true +true +false diff --git a/src/test/hive/data/files/complex.seq b/src/test/hive/data/files/complex.seq new file mode 100644 index 0000000000000000000000000000000000000000..c27d5c09b1da881d8fd6fb2aaa1f5d169d1de3ae GIT binary patch literal 1606 zcma*nziSjx5C`zLx3_zHUi3n^fD$eI(R%Ff-9rQsY+@BG>=M>wH6ejJHc>bPtK=+< zv9RAHn+>+ zcCj@cPlh|=;qB=`xqrXfIVc|NmH)9s$j{|V$InlvRp;l))Ae892d{B*hG_Fu@yrv} z!@=j>o@n8WSCyONYHP#q3){o7tyam@7ww)pXZa}9S6?^_bc8{)j0ai1r<%_Sw?}Ps z(b+Ah8`Ro)QtUl0WhstvJl7mz$jaCC4|PqTX=n^dZB5e(Xr>+bl-fw(>n2N?0ybH~ zFa>B-CMpfn58qj>n?lV*9=ZcpH-*v^vI(hS!d6n7m9wd3nnGMwCMv~C#C)HQuum5! zS3lKFk!GS_&P8-}QzT6hn~)l&E{|)48KZ0y6=9@8QE8w~Z*)G^QL%<1K0Q_oIx3c^ zm{CX#)S^PAF{9chD#l2KqEeh7<~udRPQ7^Z_gfv6XebA(&mj6bDv_v!QAiEc1s-Py zVZtiMR3(_HR8$(O)$4y|bycdV=(nTSYDrh6QkAj_siC^2RAG>^$~9FfW-1kxhU)F_ zS9j~GOjC8xaSpMpt1_v|ScTM31xgjg5vw|;D#J{rqS8@Nh~YM*GtY%NloTUNlnX1EJ+mu3l%44q~<0*V5wFrsVqoUvQjEaP0lY$ zQPNS$OUwoFQ}UBb@{5Y00>uT1$so~^)Z&uNymY;c%(7HHh(W1EDXBnJX_=`xDaA@w z(bX^$@=9})Qj366P#ZGyN`M@oxj=LLftrdG^3xOwauO?n1}cc9<1 z%PcA`i3hs}Zb)%S5l}hY9Iz4vkP?t#i6AG#jYKytCy|h8AQgBGi>+n29$9%)>}&`7 z>$N}rUYjs~z1Tb^)w7A*O$GuDi5&uxMky`!8r@0)Y+SW=*Dkg_k3Dnw!~O6tUt8O^ z8?6mUDsB-h4!qN{_V}9oBby{rew8QNXr)y*nVvo5d$V!X%jHW7)8BV$91Mw9)K;EY z`qxeSTztZ=3#Y=S%ZaVpRlxXc@7D#EtrvMCr=^D^Uf*K4Jo2fi;qGFNFGmltf4*(| Nrs2$iUySH31OSgw(98e; literal 0 HcmV?d00001 diff --git a/src/test/hive/data/files/docurl.txt b/src/test/hive/data/files/docurl.txt new file mode 100644 index 0000000000000..5862d78b4bc53 --- /dev/null +++ b/src/test/hive/data/files/docurl.txt @@ -0,0 +1,8 @@ +bonework Chad bullfrog almighty blubberers cynodictis boilersmith cosmopolitical corrie autoincrements +casings choked colpohysterotomy comedist cradleman annexa agronomical archmockery Cocles adhaka +daekon link anticrochet auricular cheeked Arbon alder-leaved +darlingness breamed company carbureted comediette condensery link +daekon link anticrochet auricular cheeked Arbon alder-leaved +darlingness breamed company carbureted comediette condensery link +daekon link anticrochet auricular cheeked Arbon alder-leaved darlingness breamed company carbureted comediette condensery link +daekon link anticrochet auricular cheeked Arbon alder-leaved darlingness breamed company carbureted comediette condensery link diff --git a/src/test/hive/data/files/double.txt b/src/test/hive/data/files/double.txt new file mode 100644 index 0000000000000..66c030bc47252 --- /dev/null +++ b/src/test/hive/data/files/double.txt @@ -0,0 +1,16 @@ +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 diff --git a/src/test/hive/data/files/employee.dat b/src/test/hive/data/files/employee.dat new file mode 100644 index 0000000000000..c9d0197539efc --- /dev/null +++ b/src/test/hive/data/files/employee.dat @@ -0,0 +1,13 @@ +16|john +17|robert +18|andrew +19|katty +21|tom +22|tim +23|james +24|paul +27|edward +29|alan +31|kerry +34|terri + diff --git a/src/test/hive/data/files/employee2.dat b/src/test/hive/data/files/employee2.dat new file mode 100644 index 0000000000000..ad2d99a112511 --- /dev/null +++ b/src/test/hive/data/files/employee2.dat @@ -0,0 +1,7 @@ +16|john +17|robert +18|andrew +19|katty +27|edward +29|alan +31|kerry diff --git a/src/test/hive/data/files/employee_part.txt b/src/test/hive/data/files/employee_part.txt new file mode 100644 index 0000000000000..8a10b9ea0298c --- /dev/null +++ b/src/test/hive/data/files/employee_part.txt @@ -0,0 +1,9 @@ +16|john|4000|USA +17|robert|2000|USA +18|andrew|4000|USA +19|katty|2000|USA +27|edward|4000|UK +29|alan|3000|UK +31|kerry|4000|UK +34|tom|3000|UK +35|zack|2000|UK diff --git a/src/test/hive/data/files/empty1.txt b/src/test/hive/data/files/empty1.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/data/files/empty2.txt b/src/test/hive/data/files/empty2.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/data/files/episodes.avro b/src/test/hive/data/files/episodes.avro new file mode 100644 index 0000000000000000000000000000000000000000..58a028ce19e6a1e964465dba8f9fbf5c84b3c3e5 GIT binary patch literal 597 zcmZ`$$w~u35Ov8x#Dj=L5s_jL1qmVRM7@a%A}%2+f)b^isW@$Vx`*!0$Pn`jp8Nt& zeudv=PZl@uSoPMfKD&P$pU7gYWL|p#h4`N7Iwpz8*>)6pQu$8K5g4X3MNCVd^l+mi z^wPBcH1bcl6SZw%Sr`PO_y|f#X$L9-g z;dAr)w);_-ea$!*mc7p@CSd|NlpVELhMh<;4y8h|knQ6Gw{;CytVP*k1x_$Y;bL~} zP%34!WeX0_W;dkQhBBN}WGK8R1;wpeZEAH#z@;EmCg2I|28{bqD#NLaMs<_Q9|e1z!KVRZ$dP6+#0&;a16W`5Bef0`;XaK>N77*YG#D;p z+>fk$v2%ZqRmP5Qwj9a zw98HR*%|Agivnx)pltv}xX}zYS$*7S1~)MS=bZs%I@KAfPoot9WhBit)LerL0c)gR zc=n4dWVhTWuto;&2v|bDFdkgw?Cq8YzX`1AETIRt6tbn>7!K<5kZ;ZV6Z;$CayJb#+fC8ZHYNM*?fQ z(bIt?jGgYkixbj~)QgPv0j$Z!fDbI8(_Mij96Tft!q4&l^2KNAm*SPd?*VIS8(r7r z#S}7(wqeieWsKQUXPYQuT&UcODAzBk2%xMTH0zMN33@N^fB3tfi3j5?P(I1PZW`Ea zXQJAlwZWZx%3KxHnlF!N%Ndz=$C-8~HE@P&mz2+-r!mh+4S=;B`q=4W+~J;j+@ICR zJ7p5D^ZLPK9CEZln-#qT^9xWW_o<|bAYPmM|ERfg)@ZGw;Ck5}e-z)^uay0g7|h zpn#&AotbJEPc&)~Sh^vzP-1~xl-{-Eq%BWvry`~wk`0@(;UFgimdet4#CC?$w_LB3 zgvyjq#Whz3)&U2JG_ZuR(H=F-*(KKo)<|b82bQp9t#(WG0&8S2S^)3ld|o0|acW}T zYlbeDz}ME)eTCKARL51w7fcC*(N=&s%rx+iL;mX692Yb`Rrtum#L&|)_nPM3wgT;M zmKDJ`D}t%VJfTumqi%t9F@B`2081E*aX=B0CdEmU<}x{3jlxLNryC$8%1jhz#tkg6{J6TY6mF zo;N+el^Ro`GOp>aKsj6+xu`JH>}(^a18e$VRRb(xe}^5dS-WLaSTM=~>pa@5n2KF? zITa?Pm?Em@)XuypR?*|EqB(8`R|(-kstlB^%m61RZkck|>Kbe}0cHA3K1=_rEo_(~=7|(#U`L_3mRZsg~Ru8s)fHmEyI$#Oi8MV)_O~X&` z5IYNo1z=4LmY2X1wtb61?gG@2xl9@~ErYrEOvMy6HPm2q2I70u_#~1}J7b0u9U5~k zu%^=A)ZLs^0a#o1k+B3=LSy;>mN0m!z!J91pfA2iN=-@W+SlL_8132o_1HsZ=g+AdjJ`u4rNV15iqaBVNosS9B36D{i0x03U=L=Pr< zU`;oc=)e-T=I11QKpDwe#m`!0-eP7dB{RVm2Cyav>s+7+yEmPd@U5KvUbJ?q7buHV zH?M?k&oNjf18ZckV*o6nGXnri*iy4LoZg|%xEZs0?5okugp?6~-1Zu)gRlD}W{JpY#|Gmp4-*Hw1oiI?Mj$h0{ohO-$OEfPi(L zhJ&`9dgo&Zur{>zD`xT|XN;3G($<E%#*>1-SV z>zl!%6R7b+rkQW7#DMk1w*9^KXsHgT#RF@4Ffs#6*p^;x`Y|dFSkv1iX{>61HL?w^gE z*8c3u=sLh!ru|XTSV|)yt#BQuU@&%Iw zunyAxGOJZ0x_}vc)e(%pz5&)Z+gun7)4&?(d~X4k(C9Nj2&Wn|PAUWKlSxaSY2u-I Ncumf8nLOid|39=r{Z;?~ literal 0 HcmV?d00001 diff --git a/src/test/hive/data/files/kv1.string-sorted.txt b/src/test/hive/data/files/kv1.string-sorted.txt new file mode 100644 index 0000000000000..5c1575d34285e --- /dev/null +++ b/src/test/hive/data/files/kv1.string-sorted.txt @@ -0,0 +1,500 @@ +0val_0 +0val_0 +0val_0 +10val_10 +100val_100 +100val_100 +103val_103 +103val_103 +104val_104 +104val_104 +105val_105 +11val_11 +111val_111 +113val_113 +113val_113 +114val_114 +116val_116 +118val_118 +118val_118 +119val_119 +119val_119 +119val_119 +12val_12 +12val_12 +120val_120 +120val_120 +125val_125 +125val_125 +126val_126 +128val_128 +128val_128 +128val_128 +129val_129 +129val_129 +131val_131 +133val_133 +134val_134 +134val_134 +136val_136 +137val_137 +137val_137 +138val_138 +138val_138 +138val_138 +138val_138 +143val_143 +145val_145 +146val_146 +146val_146 +149val_149 +149val_149 +15val_15 +15val_15 +150val_150 +152val_152 +152val_152 +153val_153 +155val_155 +156val_156 +157val_157 +158val_158 +160val_160 +162val_162 +163val_163 +164val_164 +164val_164 +165val_165 +165val_165 +166val_166 +167val_167 +167val_167 +167val_167 +168val_168 +169val_169 +169val_169 +169val_169 +169val_169 +17val_17 +170val_170 +172val_172 +172val_172 +174val_174 +174val_174 +175val_175 +175val_175 +176val_176 +176val_176 +177val_177 +178val_178 +179val_179 +179val_179 +18val_18 +18val_18 +180val_180 +181val_181 +183val_183 +186val_186 +187val_187 +187val_187 +187val_187 +189val_189 +19val_19 +190val_190 +191val_191 +191val_191 +192val_192 +193val_193 +193val_193 +193val_193 +194val_194 +195val_195 +195val_195 +196val_196 +197val_197 +197val_197 +199val_199 +199val_199 +199val_199 +2val_2 +20val_20 +200val_200 +200val_200 +201val_201 +202val_202 +203val_203 +203val_203 +205val_205 +205val_205 +207val_207 +207val_207 +208val_208 +208val_208 +208val_208 +209val_209 +209val_209 +213val_213 +213val_213 +214val_214 +216val_216 +216val_216 +217val_217 +217val_217 +218val_218 +219val_219 +219val_219 +221val_221 +221val_221 +222val_222 +223val_223 +223val_223 +224val_224 +224val_224 +226val_226 +228val_228 +229val_229 +229val_229 +230val_230 +230val_230 +230val_230 +230val_230 +230val_230 +233val_233 +233val_233 +235val_235 +237val_237 +237val_237 +238val_238 +238val_238 +239val_239 +239val_239 +24val_24 +24val_24 +241val_241 +242val_242 +242val_242 +244val_244 +247val_247 +248val_248 +249val_249 +252val_252 +255val_255 +255val_255 +256val_256 +256val_256 +257val_257 +258val_258 +26val_26 +26val_26 +260val_260 +262val_262 +263val_263 +265val_265 +265val_265 +266val_266 +27val_27 +272val_272 +272val_272 +273val_273 +273val_273 +273val_273 +274val_274 +275val_275 +277val_277 +277val_277 +277val_277 +277val_277 +278val_278 +278val_278 +28val_28 +280val_280 +280val_280 +281val_281 +281val_281 +282val_282 +282val_282 +283val_283 +284val_284 +285val_285 +286val_286 +287val_287 +288val_288 +288val_288 +289val_289 +291val_291 +292val_292 +296val_296 +298val_298 +298val_298 +298val_298 +30val_30 +302val_302 +305val_305 +306val_306 +307val_307 +307val_307 +308val_308 +309val_309 +309val_309 +310val_310 +311val_311 +311val_311 +311val_311 +315val_315 +316val_316 +316val_316 +316val_316 +317val_317 +317val_317 +318val_318 +318val_318 +318val_318 +321val_321 +321val_321 +322val_322 +322val_322 +323val_323 +325val_325 +325val_325 +327val_327 +327val_327 +327val_327 +33val_33 +331val_331 +331val_331 +332val_332 +333val_333 +333val_333 +335val_335 +336val_336 +338val_338 +339val_339 +34val_34 +341val_341 +342val_342 +342val_342 +344val_344 +344val_344 +345val_345 +348val_348 +348val_348 +348val_348 +348val_348 +348val_348 +35val_35 +35val_35 +35val_35 +351val_351 +353val_353 +353val_353 +356val_356 +360val_360 +362val_362 +364val_364 +365val_365 +366val_366 +367val_367 +367val_367 +368val_368 +369val_369 +369val_369 +369val_369 +37val_37 +37val_37 +373val_373 +374val_374 +375val_375 +377val_377 +378val_378 +379val_379 +382val_382 +382val_382 +384val_384 +384val_384 +384val_384 +386val_386 +389val_389 +392val_392 +393val_393 +394val_394 +395val_395 +395val_395 +396val_396 +396val_396 +396val_396 +397val_397 +397val_397 +399val_399 +399val_399 +4val_4 +400val_400 +401val_401 +401val_401 +401val_401 +401val_401 +401val_401 +402val_402 +403val_403 +403val_403 +403val_403 +404val_404 +404val_404 +406val_406 +406val_406 +406val_406 +406val_406 +407val_407 +409val_409 +409val_409 +409val_409 +41val_41 +411val_411 +413val_413 +413val_413 +414val_414 +414val_414 +417val_417 +417val_417 +417val_417 +418val_418 +419val_419 +42val_42 +42val_42 +421val_421 +424val_424 +424val_424 +427val_427 +429val_429 +429val_429 +43val_43 +430val_430 +430val_430 +430val_430 +431val_431 +431val_431 +431val_431 +432val_432 +435val_435 +436val_436 +437val_437 +438val_438 +438val_438 +438val_438 +439val_439 +439val_439 +44val_44 +443val_443 +444val_444 +446val_446 +448val_448 +449val_449 +452val_452 +453val_453 +454val_454 +454val_454 +454val_454 +455val_455 +457val_457 +458val_458 +458val_458 +459val_459 +459val_459 +460val_460 +462val_462 +462val_462 +463val_463 +463val_463 +466val_466 +466val_466 +466val_466 +467val_467 +468val_468 +468val_468 +468val_468 +468val_468 +469val_469 +469val_469 +469val_469 +469val_469 +469val_469 +47val_47 +470val_470 +472val_472 +475val_475 +477val_477 +478val_478 +478val_478 +479val_479 +480val_480 +480val_480 +480val_480 +481val_481 +482val_482 +483val_483 +484val_484 +485val_485 +487val_487 +489val_489 +489val_489 +489val_489 +489val_489 +490val_490 +491val_491 +492val_492 +492val_492 +493val_493 +494val_494 +495val_495 +496val_496 +497val_497 +498val_498 +498val_498 +498val_498 +5val_5 +5val_5 +5val_5 +51val_51 +51val_51 +53val_53 +54val_54 +57val_57 +58val_58 +58val_58 +64val_64 +65val_65 +66val_66 +67val_67 +67val_67 +69val_69 +70val_70 +70val_70 +70val_70 +72val_72 +72val_72 +74val_74 +76val_76 +76val_76 +77val_77 +78val_78 +8val_8 +80val_80 +82val_82 +83val_83 +83val_83 +84val_84 +84val_84 +85val_85 +86val_86 +87val_87 +9val_9 +90val_90 +90val_90 +90val_90 +92val_92 +95val_95 +95val_95 +96val_96 +97val_97 +97val_97 +98val_98 +98val_98 diff --git a/src/test/hive/data/files/kv1.txt b/src/test/hive/data/files/kv1.txt new file mode 100644 index 0000000000000..9825414ecf8f2 --- /dev/null +++ b/src/test/hive/data/files/kv1.txt @@ -0,0 +1,500 @@ +238val_238 +86val_86 +311val_311 +27val_27 +165val_165 +409val_409 +255val_255 +278val_278 +98val_98 +484val_484 +265val_265 +193val_193 +401val_401 +150val_150 +273val_273 +224val_224 +369val_369 +66val_66 +128val_128 +213val_213 +146val_146 +406val_406 +429val_429 +374val_374 +152val_152 +469val_469 +145val_145 +495val_495 +37val_37 +327val_327 +281val_281 +277val_277 +209val_209 +15val_15 +82val_82 +403val_403 +166val_166 +417val_417 +430val_430 +252val_252 +292val_292 +219val_219 +287val_287 +153val_153 +193val_193 +338val_338 +446val_446 +459val_459 +394val_394 +237val_237 +482val_482 +174val_174 +413val_413 +494val_494 +207val_207 +199val_199 +466val_466 +208val_208 +174val_174 +399val_399 +396val_396 +247val_247 +417val_417 +489val_489 +162val_162 +377val_377 +397val_397 +309val_309 +365val_365 +266val_266 +439val_439 +342val_342 +367val_367 +325val_325 +167val_167 +195val_195 +475val_475 +17val_17 +113val_113 +155val_155 +203val_203 +339val_339 +0val_0 +455val_455 +128val_128 +311val_311 +316val_316 +57val_57 +302val_302 +205val_205 +149val_149 +438val_438 +345val_345 +129val_129 +170val_170 +20val_20 +489val_489 +157val_157 +378val_378 +221val_221 +92val_92 +111val_111 +47val_47 +72val_72 +4val_4 +280val_280 +35val_35 +427val_427 +277val_277 +208val_208 +356val_356 +399val_399 +169val_169 +382val_382 +498val_498 +125val_125 +386val_386 +437val_437 +469val_469 +192val_192 +286val_286 +187val_187 +176val_176 +54val_54 +459val_459 +51val_51 +138val_138 +103val_103 +239val_239 +213val_213 +216val_216 +430val_430 +278val_278 +176val_176 +289val_289 +221val_221 +65val_65 +318val_318 +332val_332 +311val_311 +275val_275 +137val_137 +241val_241 +83val_83 +333val_333 +180val_180 +284val_284 +12val_12 +230val_230 +181val_181 +67val_67 +260val_260 +404val_404 +384val_384 +489val_489 +353val_353 +373val_373 +272val_272 +138val_138 +217val_217 +84val_84 +348val_348 +466val_466 +58val_58 +8val_8 +411val_411 +230val_230 +208val_208 +348val_348 +24val_24 +463val_463 +431val_431 +179val_179 +172val_172 +42val_42 +129val_129 +158val_158 +119val_119 +496val_496 +0val_0 +322val_322 +197val_197 +468val_468 +393val_393 +454val_454 +100val_100 +298val_298 +199val_199 +191val_191 +418val_418 +96val_96 +26val_26 +165val_165 +327val_327 +230val_230 +205val_205 +120val_120 +131val_131 +51val_51 +404val_404 +43val_43 +436val_436 +156val_156 +469val_469 +468val_468 +308val_308 +95val_95 +196val_196 +288val_288 +481val_481 +457val_457 +98val_98 +282val_282 +197val_197 +187val_187 +318val_318 +318val_318 +409val_409 +470val_470 +137val_137 +369val_369 +316val_316 +169val_169 +413val_413 +85val_85 +77val_77 +0val_0 +490val_490 +87val_87 +364val_364 +179val_179 +118val_118 +134val_134 +395val_395 +282val_282 +138val_138 +238val_238 +419val_419 +15val_15 +118val_118 +72val_72 +90val_90 +307val_307 +19val_19 +435val_435 +10val_10 +277val_277 +273val_273 +306val_306 +224val_224 +309val_309 +389val_389 +327val_327 +242val_242 +369val_369 +392val_392 +272val_272 +331val_331 +401val_401 +242val_242 +452val_452 +177val_177 +226val_226 +5val_5 +497val_497 +402val_402 +396val_396 +317val_317 +395val_395 +58val_58 +35val_35 +336val_336 +95val_95 +11val_11 +168val_168 +34val_34 +229val_229 +233val_233 +143val_143 +472val_472 +322val_322 +498val_498 +160val_160 +195val_195 +42val_42 +321val_321 +430val_430 +119val_119 +489val_489 +458val_458 +78val_78 +76val_76 +41val_41 +223val_223 +492val_492 +149val_149 +449val_449 +218val_218 +228val_228 +138val_138 +453val_453 +30val_30 +209val_209 +64val_64 +468val_468 +76val_76 +74val_74 +342val_342 +69val_69 +230val_230 +33val_33 +368val_368 +103val_103 +296val_296 +113val_113 +216val_216 +367val_367 +344val_344 +167val_167 +274val_274 +219val_219 +239val_239 +485val_485 +116val_116 +223val_223 +256val_256 +263val_263 +70val_70 +487val_487 +480val_480 +401val_401 +288val_288 +191val_191 +5val_5 +244val_244 +438val_438 +128val_128 +467val_467 +432val_432 +202val_202 +316val_316 +229val_229 +469val_469 +463val_463 +280val_280 +2val_2 +35val_35 +283val_283 +331val_331 +235val_235 +80val_80 +44val_44 +193val_193 +321val_321 +335val_335 +104val_104 +466val_466 +366val_366 +175val_175 +403val_403 +483val_483 +53val_53 +105val_105 +257val_257 +406val_406 +409val_409 +190val_190 +406val_406 +401val_401 +114val_114 +258val_258 +90val_90 +203val_203 +262val_262 +348val_348 +424val_424 +12val_12 +396val_396 +201val_201 +217val_217 +164val_164 +431val_431 +454val_454 +478val_478 +298val_298 +125val_125 +431val_431 +164val_164 +424val_424 +187val_187 +382val_382 +5val_5 +70val_70 +397val_397 +480val_480 +291val_291 +24val_24 +351val_351 +255val_255 +104val_104 +70val_70 +163val_163 +438val_438 +119val_119 +414val_414 +200val_200 +491val_491 +237val_237 +439val_439 +360val_360 +248val_248 +479val_479 +305val_305 +417val_417 +199val_199 +444val_444 +120val_120 +429val_429 +169val_169 +443val_443 +323val_323 +325val_325 +277val_277 +230val_230 +478val_478 +178val_178 +468val_468 +310val_310 +317val_317 +333val_333 +493val_493 +460val_460 +207val_207 +249val_249 +265val_265 +480val_480 +83val_83 +136val_136 +353val_353 +172val_172 +214val_214 +462val_462 +233val_233 +406val_406 +133val_133 +175val_175 +189val_189 +454val_454 +375val_375 +401val_401 +421val_421 +407val_407 +384val_384 +256val_256 +26val_26 +134val_134 +67val_67 +384val_384 +379val_379 +18val_18 +462val_462 +492val_492 +100val_100 +298val_298 +9val_9 +341val_341 +498val_498 +146val_146 +458val_458 +362val_362 +186val_186 +285val_285 +348val_348 +167val_167 +18val_18 +273val_273 +183val_183 +281val_281 +344val_344 +97val_97 +469val_469 +315val_315 +84val_84 +28val_28 +37val_37 +448val_448 +152val_152 +348val_348 +307val_307 +194val_194 +414val_414 +477val_477 +222val_222 +126val_126 +90val_90 +169val_169 +403val_403 +400val_400 +200val_200 +97val_97 diff --git a/src/test/hive/data/files/kv1.val.sorted.txt b/src/test/hive/data/files/kv1.val.sorted.txt new file mode 100644 index 0000000000000..916a0f13c70f6 --- /dev/null +++ b/src/test/hive/data/files/kv1.val.sorted.txt @@ -0,0 +1,500 @@ +val_0 +val_0 +val_0 +val_10 +val_100 +val_100 +val_103 +val_103 +val_104 +val_104 +val_105 +val_11 +val_111 +val_113 +val_113 +val_114 +val_116 +val_118 +val_118 +val_119 +val_119 +val_119 +val_12 +val_12 +val_120 +val_120 +val_125 +val_125 +val_126 +val_128 +val_128 +val_128 +val_129 +val_129 +val_131 +val_133 +val_134 +val_134 +val_136 +val_137 +val_137 +val_138 +val_138 +val_138 +val_138 +val_143 +val_145 +val_146 +val_146 +val_149 +val_149 +val_15 +val_15 +val_150 +val_152 +val_152 +val_153 +val_155 +val_156 +val_157 +val_158 +val_160 +val_162 +val_163 +val_164 +val_164 +val_165 +val_165 +val_166 +val_167 +val_167 +val_167 +val_168 +val_169 +val_169 +val_169 +val_169 +val_17 +val_170 +val_172 +val_172 +val_174 +val_174 +val_175 +val_175 +val_176 +val_176 +val_177 +val_178 +val_179 +val_179 +val_18 +val_18 +val_180 +val_181 +val_183 +val_186 +val_187 +val_187 +val_187 +val_189 +val_19 +val_190 +val_191 +val_191 +val_192 +val_193 +val_193 +val_193 +val_194 +val_195 +val_195 +val_196 +val_197 +val_197 +val_199 +val_199 +val_199 +val_2 +val_20 +val_200 +val_200 +val_201 +val_202 +val_203 +val_203 +val_205 +val_205 +val_207 +val_207 +val_208 +val_208 +val_208 +val_209 +val_209 +val_213 +val_213 +val_214 +val_216 +val_216 +val_217 +val_217 +val_218 +val_219 +val_219 +val_221 +val_221 +val_222 +val_223 +val_223 +val_224 +val_224 +val_226 +val_228 +val_229 +val_229 +val_230 +val_230 +val_230 +val_230 +val_230 +val_233 +val_233 +val_235 +val_237 +val_237 +val_238 +val_238 +val_239 +val_239 +val_24 +val_24 +val_241 +val_242 +val_242 +val_244 +val_247 +val_248 +val_249 +val_252 +val_255 +val_255 +val_256 +val_256 +val_257 +val_258 +val_26 +val_26 +val_260 +val_262 +val_263 +val_265 +val_265 +val_266 +val_27 +val_272 +val_272 +val_273 +val_273 +val_273 +val_274 +val_275 +val_277 +val_277 +val_277 +val_277 +val_278 +val_278 +val_28 +val_280 +val_280 +val_281 +val_281 +val_282 +val_282 +val_283 +val_284 +val_285 +val_286 +val_287 +val_288 +val_288 +val_289 +val_291 +val_292 +val_296 +val_298 +val_298 +val_298 +val_30 +val_302 +val_305 +val_306 +val_307 +val_307 +val_308 +val_309 +val_309 +val_310 +val_311 +val_311 +val_311 +val_315 +val_316 +val_316 +val_316 +val_317 +val_317 +val_318 +val_318 +val_318 +val_321 +val_321 +val_322 +val_322 +val_323 +val_325 +val_325 +val_327 +val_327 +val_327 +val_33 +val_331 +val_331 +val_332 +val_333 +val_333 +val_335 +val_336 +val_338 +val_339 +val_34 +val_341 +val_342 +val_342 +val_344 +val_344 +val_345 +val_348 +val_348 +val_348 +val_348 +val_348 +val_35 +val_35 +val_35 +val_351 +val_353 +val_353 +val_356 +val_360 +val_362 +val_364 +val_365 +val_366 +val_367 +val_367 +val_368 +val_369 +val_369 +val_369 +val_37 +val_37 +val_373 +val_374 +val_375 +val_377 +val_378 +val_379 +val_382 +val_382 +val_384 +val_384 +val_384 +val_386 +val_389 +val_392 +val_393 +val_394 +val_395 +val_395 +val_396 +val_396 +val_396 +val_397 +val_397 +val_399 +val_399 +val_4 +val_400 +val_401 +val_401 +val_401 +val_401 +val_401 +val_402 +val_403 +val_403 +val_403 +val_404 +val_404 +val_406 +val_406 +val_406 +val_406 +val_407 +val_409 +val_409 +val_409 +val_41 +val_411 +val_413 +val_413 +val_414 +val_414 +val_417 +val_417 +val_417 +val_418 +val_419 +val_42 +val_42 +val_421 +val_424 +val_424 +val_427 +val_429 +val_429 +val_43 +val_430 +val_430 +val_430 +val_431 +val_431 +val_431 +val_432 +val_435 +val_436 +val_437 +val_438 +val_438 +val_438 +val_439 +val_439 +val_44 +val_443 +val_444 +val_446 +val_448 +val_449 +val_452 +val_453 +val_454 +val_454 +val_454 +val_455 +val_457 +val_458 +val_458 +val_459 +val_459 +val_460 +val_462 +val_462 +val_463 +val_463 +val_466 +val_466 +val_466 +val_467 +val_468 +val_468 +val_468 +val_468 +val_469 +val_469 +val_469 +val_469 +val_469 +val_47 +val_470 +val_472 +val_475 +val_477 +val_478 +val_478 +val_479 +val_480 +val_480 +val_480 +val_481 +val_482 +val_483 +val_484 +val_485 +val_487 +val_489 +val_489 +val_489 +val_489 +val_490 +val_491 +val_492 +val_492 +val_493 +val_494 +val_495 +val_496 +val_497 +val_498 +val_498 +val_498 +val_5 +val_5 +val_5 +val_51 +val_51 +val_53 +val_54 +val_57 +val_58 +val_58 +val_64 +val_65 +val_66 +val_67 +val_67 +val_69 +val_70 +val_70 +val_70 +val_72 +val_72 +val_74 +val_76 +val_76 +val_77 +val_78 +val_8 +val_80 +val_82 +val_83 +val_83 +val_84 +val_84 +val_85 +val_86 +val_87 +val_9 +val_90 +val_90 +val_90 +val_92 +val_95 +val_95 +val_96 +val_97 +val_97 +val_98 +val_98 diff --git a/src/test/hive/data/files/kv1_broken.seq b/src/test/hive/data/files/kv1_broken.seq new file mode 100644 index 0000000000000000000000000000000000000000..49bddeb8050612b0347836b884e4a6e088d11802 GIT binary patch literal 216 zcmWG`4P?{KFG|--EJ#ewNY%?oOv%qL(96gyOVumP(MzpJP1bX&EJ+P7$}CAt%1M>P zRGXQv7m`|0!T<(R*Sû`¤qMwÎtCSGïk[E¬6’:röQDêÞßxo–õ‚4³ !&g—9™¡JýÙí/ÉÍBÒWœ ? β… 7üªKb骋†qfHrstr¨^*‹5µà¯3Ÿ»°Ã¶Ûýç’ï^Ó}®6ØüÀ\k~@èY +”Ý7ª¾?E™Õº¾}´ ã?úÍÙ \ No newline at end of file diff --git a/src/test/hive/data/files/map_table.txt b/src/test/hive/data/files/map_table.txt new file mode 100644 index 0000000000000..f3a01d8f180be --- /dev/null +++ b/src/test/hive/data/files/map_table.txt @@ -0,0 +1,2 @@ +foo1 k1:v1,k2:v2,k3:v3 +foo2 k21:v21,k22:v22,k31:v31 diff --git a/src/test/hive/data/files/nested_complex.txt b/src/test/hive/data/files/nested_complex.txt new file mode 100644 index 0000000000000..fee462c44e349 --- /dev/null +++ b/src/test/hive/data/files/nested_complex.txt @@ -0,0 +1,2 @@ +3012k1v1k2v2a102 +2032k1v1k3v3b102 diff --git a/src/test/hive/data/files/null.txt b/src/test/hive/data/files/null.txt new file mode 100644 index 0000000000000..7014a64126de4 --- /dev/null +++ b/src/test/hive/data/files/null.txt @@ -0,0 +1,10 @@ +1.01same0 +1.01same1 +1.01same2 +1.01same3 +1.01same4 +\N1same5 +\N\Nsame6 +1.0\Nsame7 +1.01same8 +1.01same9 diff --git a/src/test/hive/data/files/nullfile.txt b/src/test/hive/data/files/nullfile.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/data/files/nulls.txt b/src/test/hive/data/files/nulls.txt new file mode 100644 index 0000000000000..21cb1b5627c62 --- /dev/null +++ b/src/test/hive/data/files/nulls.txt @@ -0,0 +1,1002 @@ +1 +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N diff --git a/src/test/hive/data/files/orc_create.txt b/src/test/hive/data/files/orc_create.txt new file mode 100644 index 0000000000000..211e047346309 --- /dev/null +++ b/src/test/hive/data/files/orc_create.txt @@ -0,0 +1,3 @@ +line1|key11:value11,key12:value12,key13:value13|a,b,c|one,two +line2|key21:value21,key22:value22,key23:value23|d,e,f|three,four +line3|key31:value31,key32:value32,key33:value33|g,h,i|five,six diff --git a/src/test/hive/data/files/orc_create_people.txt b/src/test/hive/data/files/orc_create_people.txt new file mode 100644 index 0000000000000..884598981a13c --- /dev/null +++ b/src/test/hive/data/files/orc_create_people.txt @@ -0,0 +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 diff --git a/src/test/hive/data/files/over10k b/src/test/hive/data/files/over10k new file mode 100644 index 0000000000000..adc75519ee3e4 --- /dev/null +++ b/src/test/hive/data/files/over10k @@ -0,0 +1,9999 @@ +124|336|65664|4294967435|74.72|42.47|true|bob davidson|2013-03-01 09:11:58.703302|45.40|yard duty +19|442|65553|4294967380|26.43|37.77|true|alice zipper|2013-03-01 09:11:58.703217|29.62|history +35|387|65619|4294967459|96.91|18.86|false|katie davidson|2013-03-01 09:11:58.703079|27.32|history +111|372|65656|4294967312|13.01|34.95|false|xavier quirinius|2013-03-01 09:11:58.703310|23.91|topology +54|317|65547|4294967409|60.71|2.09|false|nick robinson|2013-03-01 09:11:58.703103|90.21|geology +-3|467|65575|4294967437|81.64|23.53|true|tom hernandez|2013-03-01 09:11:58.703188|32.85|study skills +53|317|65702|4294967398|35.17|30.87|false|ulysses hernandez|2013-03-01 09:11:58.703164|79.50|industrial engineering +122|356|65759|4294967379|92.61|18.82|true|priscilla ichabod|2013-03-01 09:11:58.703175|92.81|nap time +18|407|65725|4294967362|82.52|5.30|true|quinn van buren|2013-03-01 09:11:58.703282|35.86|kindergarten +66|484|65685|4294967387|45.99|6.81|false|luke laertes|2013-03-01 09:11:58.703317|63.27|mathematics +122|444|65675|4294967500|72.62|34.15|false|xavier polk|2013-03-01 09:11:58.703222|99.79|american history +115|269|65627|4294967362|91.42|1.61|false|zach falkner|2013-03-01 09:11:58.703252|49.85|american history +109|438|65779|4294967371|67.18|40.76|false|quinn davidson|2013-03-01 09:11:58.703318|92.06|quiet hour +102|436|65749|4294967542|68.89|39.01|false|priscilla van buren|2013-03-01 09:11:58.703305|82.99|mathematics +92|490|65745|4294967420|10.22|49.45|true|fred king|2013-03-01 09:11:58.703277|80.12|education +34|439|65667|4294967480|26.49|18.47|true|calvin ellison|2013-03-01 09:11:58.703135|97.87|joggying +72|383|65638|4294967398|23.60|46.73|false|sarah garcia|2013-03-01 09:11:58.703192|11.20|topology +11|505|65576|4294967297|71.32|18.19|false|zach young|2013-03-01 09:11:58.703221|92.47|chemistry +69|376|65751|4294967384|88.77|27.02|false|david underhill|2013-03-01 09:11:58.703244|9.02|mathematics +36|309|65769|4294967317|91.16|38.05|false|yuri carson|2013-03-01 09:11:58.703178|68.33|debate +51|258|65781|4294967391|1.92|39.05|false|ulysses laertes|2013-03-01 09:11:58.703248|95.52|nap time +1|482|65675|4294967314|38.33|21.67|false|jessica thompson|2013-03-01 09:11:58.703110|83.09|forestry +51|295|65716|4294967433|16.24|48.96|false|sarah johnson|2013-03-01 09:11:58.703299|9.00|linguistics +68|439|65703|4294967469|92.55|23.30|true|zach ovid|2013-03-01 09:11:58.703316|65.91|religion +104|299|65791|4294967424|34.03|27.14|false|victor nixon|2013-03-01 09:11:58.703173|48.17|topology +14|337|65611|4294967542|16.99|16.36|true|sarah ellison|2013-03-01 09:11:58.703187|64.89|biology +21|305|65664|4294967361|82.41|49.69|true|xavier davidson|2013-03-01 09:11:58.703287|75.43|mathematics +79|419|65755|4294967329|91.05|39.04|false|fred steinbeck|2013-03-01 09:11:58.703216|11.23|values clariffication +109|427|65626|4294967308|98.72|31.23|true|tom robinson|2013-03-01 09:11:58.703273|64.61|kindergarten +2|379|65707|4294967450|44.27|37.93|false|victor allen|2013-03-01 09:11:58.703076|96.97|american history +7|508|65681|4294967345|66.61|23.73|false|luke allen|2013-03-01 09:11:58.703104|75.85|history +66|300|65553|4294967447|29.02|3.69|false|fred zipper|2013-03-01 09:11:58.703307|72.16|nap time +15|347|65604|4294967484|79.83|26.24|false|alice nixon|2013-03-01 09:11:58.703124|85.91|zync studies +104|454|65579|4294967467|54.44|42.57|true|irene van buren|2013-03-01 09:11:58.703225|73.48|values clariffication +6|409|65669|4294967519|69.53|2.80|true|alice laertes|2013-03-01 09:11:58.703250|93.79|industrial engineering +121|449|65776|4294967327|60.60|45.14|true|quinn davidson|2013-03-01 09:11:58.703175|98.87|industrial engineering +51|445|65761|4294967423|10.60|0.52|false|rachel ellison|2013-03-01 09:11:58.703196|60.30|education +63|268|65652|4294967374|40.78|47.60|false|tom van buren|2013-03-01 09:11:58.703279|42.78|education +10|439|65641|4294967353|24.83|46.33|true|luke zipper|2013-03-01 09:11:58.703257|26.02|topology +17|384|65700|4294967457|20.07|33.19|true|mike carson|2013-03-01 09:11:58.703140|2.83|religion +38|351|65567|4294967316|22.78|30.04|true|jessica ellison|2013-03-01 09:11:58.703113|84.56|forestry +61|419|65783|4294967427|70.93|39.46|false|bob brown|2013-03-01 09:11:58.703207|62.58|education +95|511|65636|4294967325|71.50|2.52|false|zach thompson|2013-03-01 09:11:58.703198|67.23|quiet hour +83|316|65749|4294967342|90.51|23.55|true|holly ichabod|2013-03-01 09:11:58.703091|45.00|nap time +38|318|65560|4294967354|43.73|30.98|false|david polk|2013-03-01 09:11:58.703318|21.81|mathematics +63|473|65720|4294967324|80.74|40.60|false|holly falkner|2013-03-01 09:11:58.703111|18.80|mathematics +82|272|65699|4294967488|92.82|8.98|true|wendy van buren|2013-03-01 09:11:58.703314|41.13|mathematics +35|475|65574|4294967455|60.13|10.85|false|tom falkner|2013-03-01 09:11:58.703217|23.57|wind surfing +2|375|65608|4294967444|9.93|16.47|false|xavier thompson|2013-03-01 09:11:58.703154|16.93|topology +62|428|65758|4294967373|96.09|38.60|true|fred johnson|2013-03-01 09:11:58.703307|60.88|geology +100|391|65661|4294967317|52.72|15.01|true|victor steinbeck|2013-03-01 09:11:58.703232|61.96|debate +10|264|65688|4294967479|78.30|0.63|true|irene thompson|2013-03-01 09:11:58.703156|76.21|undecided +87|469|65580|4294967411|90.20|45.49|true|calvin brown|2013-03-01 09:11:58.703278|56.33|nap time +27|262|65787|4294967371|57.35|44.12|false|irene underhill|2013-03-01 09:11:58.703289|31.19|quiet hour +89|466|65541|4294967468|85.51|19.79|true|calvin davidson|2013-03-01 09:11:58.703113|35.38|philosophy +15|379|65597|4294967382|48.28|22.73|true|luke xylophone|2013-03-01 09:11:58.703309|37.41|topology +1|284|65572|4294967342|37.07|14.51|true|fred ichabod|2013-03-01 09:11:58.703254|43.49|quiet hour +89|444|65579|4294967419|46.02|6.06|false|victor davidson|2013-03-01 09:11:58.703077|91.42|xylophone band +-2|427|65666|4294967465|19.69|33.24|true|bob xylophone|2013-03-01 09:11:58.703219|32.73|joggying +58|360|65564|4294967402|19.00|21.16|false|oscar white|2013-03-01 09:11:58.703161|15.82|forestry +9|274|65710|4294967481|9.57|4.97|true|irene laertes|2013-03-01 09:11:58.703138|86.70|yard duty +107|271|65652|4294967329|38.30|25.36|false|tom polk|2013-03-01 09:11:58.703131|73.61|kindergarten +79|399|65664|4294967492|41.85|29.00|false|calvin garcia|2013-03-01 09:11:58.703074|97.64|religion +58|448|65591|4294967451|62.74|17.69|true|mike king|2013-03-01 09:11:58.703312|82.08|nap time +98|430|65616|4294967451|35.89|12.15|true|ulysses underhill|2013-03-01 09:11:58.703083|20.52|zync studies +83|382|65550|4294967451|35.00|21.02|true|katie hernandez|2013-03-01 09:11:58.703145|8.46|xylophone band +23|381|65661|4294967425|31.40|21.26|false|katie miller|2013-03-01 09:11:58.703251|68.98|topology +62|358|65552|4294967533|50.96|9.42|false|luke allen|2013-03-01 09:11:58.703117|82.21|kindergarten +7|487|65740|4294967416|9.22|43.64|false|nick laertes|2013-03-01 09:11:58.703272|7.49|american history +5|285|65654|4294967436|42.55|14.69|false|holly underhill|2013-03-01 09:11:58.703185|8.91|study skills +104|472|65620|4294967548|22.08|9.94|true|irene falkner|2013-03-01 09:11:58.703135|29.14|opthamology +65|374|65560|4294967516|65.43|22.48|true|oscar quirinius|2013-03-01 09:11:58.703316|16.86|mathematics +8|273|65641|4294967507|35.08|37.80|false|nick allen|2013-03-01 09:11:58.703186|91.46|biology +20|449|65663|4294967439|4.72|12.34|true|xavier ichabod|2013-03-01 09:11:58.703259|11.37|geology +85|313|65780|4294967303|71.01|0.78|false|zach xylophone|2013-03-01 09:11:58.703203|11.63|biology +27|329|65778|4294967451|6.63|7.03|true|jessica zipper|2013-03-01 09:11:58.703130|51.64|philosophy +79|451|65662|4294967487|73.63|15.46|true|alice allen|2013-03-01 09:11:58.703161|99.18|wind surfing +69|440|65720|4294967373|9.56|24.42|true|jessica johnson|2013-03-01 09:11:58.703132|2.72|mathematics +119|390|65544|4294967411|41.31|20.02|false|fred steinbeck|2013-03-01 09:11:58.703267|26.94|nap time +85|468|65643|4294967430|86.63|21.00|true|wendy ovid|2013-03-01 09:11:58.703124|48.81|forestry +-2|429|65664|4294967441|45.19|40.43|false|wendy king|2013-03-01 09:11:58.703265|11.46|industrial engineering +100|481|65750|4294967308|21.19|28.14|false|holly hernandez|2013-03-01 09:11:58.703203|85.44|chemistry +-1|417|65685|4294967492|28.89|5.19|true|mike white|2013-03-01 09:11:58.703275|90.69|forestry +77|385|65681|4294967535|18.56|12.43|true|priscilla johnson|2013-03-01 09:11:58.703176|35.45|study skills +89|478|65583|4294967522|92.40|26.71|true|gabriella ovid|2013-03-01 09:11:58.703092|50.57|biology +43|333|65549|4294967331|11.22|38.00|true|victor xylophone|2013-03-01 09:11:58.703257|58.96|nap time +11|387|65718|4294967457|89.53|16.54|true|alice laertes|2013-03-01 09:11:58.703130|63.06|industrial engineering +105|438|65623|4294967501|39.69|22.95|false|ulysses xylophone|2013-03-01 09:11:58.703148|48.51|kindergarten +111|349|65740|4294967400|17.37|43.34|true|yuri nixon|2013-03-01 09:11:58.703280|55.16|quiet hour +85|461|65654|4294967507|89.10|13.44|true|xavier johnson|2013-03-01 09:11:58.703127|19.64|zync studies +71|409|65536|4294967490|46.97|25.92|false|fred miller|2013-03-01 09:11:58.703116|33.45|history +51|398|65687|4294967304|30.37|47.31|false|luke garcia|2013-03-01 09:11:58.703156|21.79|industrial engineering +47|291|65615|4294967402|37.14|29.63|false|oscar ovid|2013-03-01 09:11:58.703231|39.40|geology +-1|268|65778|4294967418|56.33|44.73|true|calvin falkner|2013-03-01 09:11:58.703220|7.37|history +86|422|65582|4294967353|87.83|2.34|false|luke ellison|2013-03-01 09:11:58.703313|9.35|joggying +76|400|65661|4294967379|15.81|46.57|true|rachel allen|2013-03-01 09:11:58.703211|5.24|wind surfing +81|448|65550|4294967533|25.88|46.54|true|oscar king|2013-03-01 09:11:58.703161|57.63|american history +22|288|65578|4294967341|33.87|20.76|true|calvin hernandez|2013-03-01 09:11:58.703173|51.36|biology +59|410|65572|4294967370|74.42|26.22|false|nick ichabod|2013-03-01 09:11:58.703221|93.48|xylophone band +86|349|65686|4294967512|62.85|24.61|true|bob ovid|2013-03-01 09:11:58.703106|31.75|religion +27|335|65636|4294967505|37.14|2.29|false|alice quirinius|2013-03-01 09:11:58.703287|21.15|education +74|422|65682|4294967316|37.60|45.06|false|rachel carson|2013-03-01 09:11:58.703140|92.95|joggying +24|269|65644|4294967349|31.01|26.44|true|katie young|2013-03-01 09:11:58.703149|58.05|xylophone band +12|411|65560|4294967346|8.91|40.79|false|calvin thompson|2013-03-01 09:11:58.703227|71.59|religion +8|379|65574|4294967410|24.49|2.74|true|calvin young|2013-03-01 09:11:58.703283|38.54|study skills +123|477|65699|4294967340|1.21|28.71|true|quinn ovid|2013-03-01 09:11:58.703278|88.09|undecided +28|349|65711|4294967458|84.69|47.30|true|holly ichabod|2013-03-01 09:11:58.703109|50.26|zync studies +83|420|65563|4294967400|85.10|43.40|true|mike king|2013-03-01 09:11:58.703169|39.96|xylophone band +51|434|65549|4294967449|3.96|39.57|false|mike miller|2013-03-01 09:11:58.703141|76.06|kindergarten +91|346|65696|4294967355|71.31|1.29|true|alice brown|2013-03-01 09:11:58.703245|17.35|zync studies +82|393|65763|4294967452|66.51|14.44|false|quinn steinbeck|2013-03-01 09:11:58.703187|87.99|joggying +85|439|65759|4294967530|20.64|21.57|true|david quirinius|2013-03-01 09:11:58.703083|47.54|philosophy +66|475|65633|4294967498|26.64|42.03|false|gabriella ichabod|2013-03-01 09:11:58.703302|59.56|chemistry +117|311|65691|4294967335|24.28|43.67|false|quinn brown|2013-03-01 09:11:58.703295|56.02|geology +92|352|65559|4294967353|3.82|31.84|false|bob zipper|2013-03-01 09:11:58.703316|29.55|topology +31|432|65607|4294967497|40.80|35.49|false|jessica johnson|2013-03-01 09:11:58.703218|9.67|study skills +114|279|65657|4294967339|54.73|30.17|true|quinn allen|2013-03-01 09:11:58.703297|64.39|debate +28|485|65694|4294967534|79.48|8.84|false|quinn underhill|2013-03-01 09:11:58.703275|82.24|history +2|399|65571|4294967410|59.68|3.92|false|luke underhill|2013-03-01 09:11:58.703221|26.64|topology +17|395|65589|4294967464|95.33|47.46|false|wendy ovid|2013-03-01 09:11:58.703118|54.01|zync studies +71|363|65787|4294967465|59.50|20.17|true|oscar king|2013-03-01 09:11:58.703285|64.88|history +48|511|65561|4294967547|8.45|9.39|false|xavier zipper|2013-03-01 09:11:58.703297|25.67|chemistry +1|424|65712|4294967484|0.13|39.92|true|yuri johnson|2013-03-01 09:11:58.703144|94.43|zync studies +109|306|65643|4294967323|79.54|38.74|true|ulysses carson|2013-03-01 09:11:58.703263|92.24|undecided +49|349|65747|4294967306|84.40|44.93|true|katie garcia|2013-03-01 09:11:58.703226|39.77|chemistry +114|454|65542|4294967443|64.67|13.02|true|katie allen|2013-03-01 09:11:58.703275|84.63|philosophy +28|307|65606|4294967389|95.34|9.62|true|jessica davidson|2013-03-01 09:11:58.703181|57.69|american history +-1|348|65556|4294967413|35.17|9.51|false|bob young|2013-03-01 09:11:58.703280|45.81|quiet hour +1|485|65625|4294967309|41.81|15.46|false|david van buren|2013-03-01 09:11:58.703207|55.06|debate +38|291|65634|4294967438|83.57|49.63|false|david van buren|2013-03-01 09:11:58.703174|64.36|debate +50|338|65634|4294967463|85.23|32.33|false|david ellison|2013-03-01 09:11:58.703247|57.09|values clariffication +113|383|65578|4294967358|43.16|22.22|true|alice xylophone|2013-03-01 09:11:58.703250|41.10|philosophy +43|305|65572|4294967511|36.70|11.12|true|ethan van buren|2013-03-01 09:11:58.703110|70.91|study skills +110|344|65759|4294967464|27.31|46.31|true|mike davidson|2013-03-01 09:11:58.703106|14.10|philosophy +23|424|65731|4294967416|65.72|43.58|true|calvin polk|2013-03-01 09:11:58.703160|24.48|philosophy +16|281|65605|4294967493|45.34|10.36|true|bob white|2013-03-01 09:11:58.703140|14.12|american history +120|285|65649|4294967355|35.56|49.34|true|victor ovid|2013-03-01 09:11:58.703205|43.92|opthamology +98|452|65544|4294967309|73.93|18.86|true|jessica white|2013-03-01 09:11:58.703175|65.77|philosophy +41|360|65643|4294967488|76.28|46.18|false|quinn thompson|2013-03-01 09:11:58.703203|61.29|philosophy +66|462|65697|4294967452|57.29|38.07|true|ethan ovid|2013-03-01 09:11:58.703138|90.16|philosophy +99|373|65579|4294967465|24.02|47.95|true|irene polk|2013-03-01 09:11:58.703231|21.36|mathematics +15|334|65576|4294967542|75.66|11.12|true|jessica robinson|2013-03-01 09:11:58.703104|2.04|chemistry +53|466|65545|4294967514|94.66|15.12|true|wendy ellison|2013-03-01 09:11:58.703236|79.00|xylophone band +64|295|65622|4294967521|59.34|7.62|false|victor brown|2013-03-01 09:11:58.703157|28.37|undecided +8|323|65774|4294967503|21.49|5.55|false|priscilla xylophone|2013-03-01 09:11:58.703213|48.80|religion +42|368|65701|4294967410|35.68|35.84|true|gabriella polk|2013-03-01 09:11:58.703235|6.35|xylophone band +72|389|65724|4294967300|69.74|38.04|false|calvin nixon|2013-03-01 09:11:58.703161|85.52|education +23|493|65555|4294967455|77.89|25.11|true|victor brown|2013-03-01 09:11:58.703125|46.72|education +26|465|65700|4294967346|46.10|38.22|true|bob quirinius|2013-03-01 09:11:58.703177|37.61|kindergarten +77|507|65721|4294967488|78.31|33.11|true|fred davidson|2013-03-01 09:11:58.703187|72.82|biology +88|346|65654|4294967369|48.23|35.72|false|mike brown|2013-03-01 09:11:58.703197|26.73|chemistry +45|357|65755|4294967485|94.25|46.52|true|gabriella thompson|2013-03-01 09:11:58.703202|67.54|religion +11|286|65667|4294967382|48.45|22.94|true|rachel underhill|2013-03-01 09:11:58.703115|63.90|values clariffication +95|489|65711|4294967493|11.89|25.36|false|ulysses laertes|2013-03-01 09:11:58.703249|78.69|history +23|448|65637|4294967435|76.28|19.32|false|ulysses miller|2013-03-01 09:11:58.703290|16.89|mathematics +42|498|65687|4294967352|42.85|39.27|false|oscar robinson|2013-03-01 09:11:58.703139|48.73|undecided +27|490|65680|4294967347|57.46|11.02|true|wendy van buren|2013-03-01 09:11:58.703291|95.26|xylophone band +53|257|65790|4294967425|9.26|9.93|false|oscar laertes|2013-03-01 09:11:58.703113|46.91|forestry +27|457|65570|4294967464|81.58|3.78|false|ulysses underhill|2013-03-01 09:11:58.703088|17.09|quiet hour +46|480|65649|4294967458|86.23|6.76|false|quinn king|2013-03-01 09:11:58.703099|91.03|debate +57|419|65786|4294967300|6.57|36.93|false|nick allen|2013-03-01 09:11:58.703290|66.89|nap time +72|267|65537|4294967460|55.20|42.89|false|oscar carson|2013-03-01 09:11:58.703153|51.91|topology +110|346|65693|4294967505|67.12|9.14|true|luke white|2013-03-01 09:11:58.703199|45.69|quiet hour +117|346|65619|4294967321|78.73|35.68|false|holly brown|2013-03-01 09:11:58.703245|36.59|xylophone band +35|451|65763|4294967498|33.83|10.70|true|sarah robinson|2013-03-01 09:11:58.703126|53.52|nap time +87|466|65562|4294967446|35.62|30.58|true|jessica quirinius|2013-03-01 09:11:58.703246|45.94|values clariffication +116|307|65785|4294967434|72.18|44.24|false|rachel zipper|2013-03-01 09:11:58.703144|29.61|debate +36|460|65541|4294967365|55.99|38.73|false|fred hernandez|2013-03-01 09:11:58.703256|98.87|zync studies +2|383|65649|4294967537|74.19|44.57|true|ulysses king|2013-03-01 09:11:58.703118|8.69|quiet hour +11|331|65659|4294967432|68.01|7.24|true|quinn nixon|2013-03-01 09:11:58.703184|8.95|xylophone band +122|449|65751|4294967548|81.66|29.49|true|mike carson|2013-03-01 09:11:58.703258|30.68|wind surfing +9|308|65715|4294967297|12.54|27.61|false|bob king|2013-03-01 09:11:58.703309|89.40|geology +26|457|65590|4294967302|87.22|26.79|true|xavier king|2013-03-01 09:11:58.703176|67.80|forestry +59|390|65776|4294967421|97.81|40.22|false|katie king|2013-03-01 09:11:58.703132|49.73|zync studies +22|489|65717|4294967422|89.38|28.13|false|mike quirinius|2013-03-01 09:11:58.703262|5.81|chemistry +82|303|65764|4294967480|54.34|13.84|false|david xylophone|2013-03-01 09:11:58.703202|31.41|religion +27|261|65619|4294967401|88.78|18.28|false|bob ovid|2013-03-01 09:11:58.703207|36.48|wind surfing +24|333|65617|4294967370|10.26|16.29|false|yuri quirinius|2013-03-01 09:11:58.703179|10.82|geology +115|334|65653|4294967435|46.45|17.86|false|gabriella steinbeck|2013-03-01 09:11:58.703085|85.09|joggying +77|435|65578|4294967474|38.62|47.15|false|alice steinbeck|2013-03-01 09:11:58.703073|75.62|study skills +101|259|65759|4294967383|79.96|10.23|false|holly underhill|2013-03-01 09:11:58.703087|10.63|nap time +45|401|65680|4294967303|32.41|25.43|true|ulysses steinbeck|2013-03-01 09:11:58.703143|55.30|education +100|430|65570|4294967431|50.31|46.98|false|calvin laertes|2013-03-01 09:11:58.703154|76.80|yard duty +106|456|65548|4294967484|37.90|14.24|true|mike hernandez|2013-03-01 09:11:58.703294|58.51|chemistry +3|260|65659|4294967508|91.53|43.18|false|oscar ovid|2013-03-01 09:11:58.703281|99.87|chemistry +60|275|65654|4294967405|24.80|20.64|true|nick young|2013-03-01 09:11:58.703074|37.80|geology +50|346|65673|4294967549|55.51|32.20|true|alice steinbeck|2013-03-01 09:11:58.703198|33.27|yard duty +85|448|65784|4294967299|90.56|32.56|false|alice hernandez|2013-03-01 09:11:58.703278|68.18|opthamology +10|498|65549|4294967361|87.48|0.66|true|oscar carson|2013-03-01 09:11:58.703150|50.60|chemistry +16|466|65673|4294967318|87.57|45.11|true|bob garcia|2013-03-01 09:11:58.703173|87.78|history +33|508|65600|4294967372|12.02|17.08|true|jessica zipper|2013-03-01 09:11:58.703302|36.57|values clariffication +31|426|65602|4294967543|24.79|36.72|true|holly hernandez|2013-03-01 09:11:58.703306|8.13|quiet hour +61|266|65564|4294967461|95.81|3.07|false|priscilla nixon|2013-03-01 09:11:58.703269|90.34|study skills +41|329|65717|4294967463|30.36|30.37|false|rachel robinson|2013-03-01 09:11:58.703214|57.61|yard duty +114|271|65581|4294967520|61.88|23.27|false|oscar van buren|2013-03-01 09:11:58.703071|88.09|industrial engineering +22|267|65651|4294967449|68.25|17.52|false|ulysses hernandez|2013-03-01 09:11:58.703097|74.13|industrial engineering +120|406|65610|4294967538|59.99|39.78|true|quinn garcia|2013-03-01 09:11:58.703299|39.74|linguistics +5|372|65699|4294967375|80.97|31.76|false|katie ellison|2013-03-01 09:11:58.703112|13.29|american history +40|465|65713|4294967408|50.09|38.66|true|bob carson|2013-03-01 09:11:58.703273|10.94|wind surfing +115|281|65651|4294967434|80.46|35.90|true|gabriella king|2013-03-01 09:11:58.703156|84.42|joggying +5|395|65741|4294967521|29.54|40.04|true|calvin quirinius|2013-03-01 09:11:58.703131|63.81|religion +45|292|65769|4294967336|32.01|29.73|false|mike white|2013-03-01 09:11:58.703310|70.52|chemistry +116|396|65778|4294967515|30.65|49.38|true|quinn ellison|2013-03-01 09:11:58.703131|99.08|opthamology +68|264|65623|4294967481|69.97|47.03|true|zach garcia|2013-03-01 09:11:58.703235|28.60|education +18|280|65536|4294967320|32.92|45.94|false|holly white|2013-03-01 09:11:58.703086|58.86|topology +28|302|65653|4294967388|50.40|38.37|false|holly miller|2013-03-01 09:11:58.703195|3.81|philosophy +114|385|65541|4294967458|73.48|34.97|true|oscar quirinius|2013-03-01 09:11:58.703143|72.33|xylophone band +4|279|65745|4294967431|83.58|31.66|true|fred van buren|2013-03-01 09:11:58.703087|25.19|study skills +51|377|65626|4294967479|1.27|12.73|false|wendy miller|2013-03-01 09:11:58.703072|96.47|values clariffication +78|351|65755|4294967337|99.29|24.84|true|holly zipper|2013-03-01 09:11:58.703236|81.44|forestry +75|461|65760|4294967326|3.21|8.52|false|david brown|2013-03-01 09:11:58.703144|21.14|xylophone band +124|371|65687|4294967489|79.55|45.10|false|priscilla carson|2013-03-01 09:11:58.703277|3.37|philosophy +95|424|65643|4294967442|36.22|24.86|false|rachel king|2013-03-01 09:11:58.703261|71.29|yard duty +88|348|65645|4294967328|46.88|5.74|false|luke polk|2013-03-01 09:11:58.703306|3.95|yard duty +27|278|65622|4294967516|25.67|46.19|true|priscilla zipper|2013-03-01 09:11:58.703302|50.08|chemistry +23|302|65595|4294967497|22.01|5.84|true|wendy falkner|2013-03-01 09:11:58.703164|48.92|biology +30|441|65721|4294967456|40.17|28.17|false|sarah steinbeck|2013-03-01 09:11:58.703256|23.78|wind surfing +31|461|65608|4294967349|61.92|49.45|true|bob miller|2013-03-01 09:11:58.703255|48.90|biology +13|477|65680|4294967505|22.85|12.85|true|calvin steinbeck|2013-03-01 09:11:58.703277|54.11|zync studies +29|449|65596|4294967490|58.67|9.22|true|victor robinson|2013-03-01 09:11:58.703177|79.75|zync studies +13|503|65664|4294967406|52.50|41.37|true|holly laertes|2013-03-01 09:11:58.703196|34.71|yard duty +63|368|65714|4294967420|20.38|25.32|true|yuri xylophone|2013-03-01 09:11:58.703189|49.69|debate +44|437|65721|4294967376|22.27|3.27|true|yuri king|2013-03-01 09:11:58.703269|86.59|topology +35|500|65641|4294967469|68.85|13.20|true|victor ellison|2013-03-01 09:11:58.703245|41.16|undecided +49|271|65735|4294967454|59.83|22.33|false|oscar white|2013-03-01 09:11:58.703139|12.67|nap time +0|264|65670|4294967479|72.98|3.67|true|david xylophone|2013-03-01 09:11:58.703269|61.06|philosophy +47|463|65751|4294967525|71.68|30.61|true|jessica ovid|2013-03-01 09:11:58.703072|2.96|education +120|273|65724|4294967477|94.15|6.54|true|david ellison|2013-03-01 09:11:58.703091|36.05|education +6|486|65611|4294967485|74.00|27.42|false|ulysses steinbeck|2013-03-01 09:11:58.703258|21.21|education +53|311|65650|4294967299|48.11|22.64|true|wendy hernandez|2013-03-01 09:11:58.703146|89.12|kindergarten +117|396|65541|4294967434|74.78|6.62|true|katie miller|2013-03-01 09:11:58.703243|89.14|yard duty +24|454|65628|4294967337|88.55|31.61|false|victor young|2013-03-01 09:11:58.703296|30.43|undecided +116|485|65575|4294967296|95.84|14.26|true|irene polk|2013-03-01 09:11:58.703128|17.00|kindergarten +11|438|65622|4294967368|89.81|0.73|true|sarah white|2013-03-01 09:11:58.703287|88.60|biology +65|263|65671|4294967519|68.95|20.34|true|alice laertes|2013-03-01 09:11:58.703103|8.65|history +31|278|65634|4294967511|50.41|46.87|true|holly underhill|2013-03-01 09:11:58.703244|99.34|debate +98|466|65602|4294967470|8.32|1.17|false|david underhill|2013-03-01 09:11:58.703275|95.58|debate +114|384|65549|4294967361|55.39|36.12|true|holly king|2013-03-01 09:11:58.703077|55.04|forestry +35|463|65646|4294967363|50.83|34.68|true|gabriella miller|2013-03-01 09:11:58.703133|27.34|nap time +37|317|65684|4294967499|23.13|26.55|false|calvin laertes|2013-03-01 09:11:58.703162|3.51|study skills +80|390|65766|4294967360|67.26|8.82|true|xavier hernandez|2013-03-01 09:11:58.703288|66.13|biology +111|486|65620|4294967357|22.36|46.67|true|ulysses underhill|2013-03-01 09:11:58.703172|64.65|zync studies +88|322|65733|4294967316|80.58|32.61|false|quinn brown|2013-03-01 09:11:58.703272|99.80|yard duty +20|372|65587|4294967462|76.92|7.37|true|gabriella hernandez|2013-03-01 09:11:58.703182|11.86|mathematics +107|278|65550|4294967432|83.08|22.15|false|sarah zipper|2013-03-01 09:11:58.703131|1.75|forestry +50|405|65715|4294967538|79.12|16.49|false|calvin ovid|2013-03-01 09:11:58.703297|39.62|chemistry +100|310|65657|4294967393|20.67|7.45|false|gabriella king|2013-03-01 09:11:58.703290|26.40|xylophone band +66|477|65674|4294967313|8.45|4.83|false|wendy young|2013-03-01 09:11:58.703180|10.28|xylophone band +73|318|65610|4294967401|11.91|23.15|false|nick thompson|2013-03-01 09:11:58.703097|36.77|religion +-2|269|65681|4294967544|99.24|28.19|false|rachel falkner|2013-03-01 09:11:58.703089|41.57|quiet hour +85|495|65656|4294967507|90.12|3.99|true|fred polk|2013-03-01 09:11:58.703257|14.96|quiet hour +20|288|65758|4294967547|20.79|34.05|false|xavier carson|2013-03-01 09:11:58.703092|61.69|chemistry +123|299|65606|4294967525|99.36|7.66|false|sarah falkner|2013-03-01 09:11:58.703300|7.51|wind surfing +10|273|65750|4294967499|23.77|21.66|false|yuri underhill|2013-03-01 09:11:58.703177|20.91|religion +75|393|65726|4294967467|39.98|29.78|false|katie ichabod|2013-03-01 09:11:58.703238|6.00|industrial engineering +47|375|65775|4294967465|70.56|32.56|false|david robinson|2013-03-01 09:11:58.703238|7.88|linguistics +106|482|65624|4294967446|45.45|11.66|true|rachel laertes|2013-03-01 09:11:58.703119|58.02|debate +36|273|65739|4294967416|1.25|33.36|true|bob zipper|2013-03-01 09:11:58.703248|74.77|industrial engineering +92|335|65603|4294967296|80.96|18.48|false|ulysses nixon|2013-03-01 09:11:58.703198|45.63|opthamology +68|301|65579|4294967497|94.43|45.02|false|zach zipper|2013-03-01 09:11:58.703230|28.02|wind surfing +106|502|65695|4294967501|83.92|35.24|true|mike zipper|2013-03-01 09:11:58.703157|57.33|opthamology +122|396|65550|4294967350|60.02|32.13|true|sarah ovid|2013-03-01 09:11:58.703099|91.38|wind surfing +67|436|65541|4294967471|19.06|39.60|true|oscar king|2013-03-01 09:11:58.703125|14.94|opthamology +56|463|65587|4294967307|52.17|42.41|true|rachel brown|2013-03-01 09:11:58.703182|94.47|joggying +46|319|65683|4294967549|94.33|14.75|true|irene steinbeck|2013-03-01 09:11:58.703108|99.56|kindergarten +18|503|65750|4294967467|12.50|30.41|false|ulysses davidson|2013-03-01 09:11:58.703310|83.44|yard duty +43|473|65697|4294967457|79.97|11.35|false|david quirinius|2013-03-01 09:11:58.703206|50.01|undecided +120|413|65650|4294967543|48.25|10.47|false|sarah king|2013-03-01 09:11:58.703122|90.77|zync studies +91|448|65559|4294967320|90.35|6.28|true|gabriella ichabod|2013-03-01 09:11:58.703320|83.50|nap time +24|419|65755|4294967444|15.92|11.90|false|xavier davidson|2013-03-01 09:11:58.703208|38.39|yard duty +63|481|65591|4294967382|28.31|36.11|true|irene underhill|2013-03-01 09:11:58.703177|11.43|history +118|497|65738|4294967546|19.14|5.09|true|alice king|2013-03-01 09:11:58.703197|82.32|industrial engineering +25|507|65643|4294967312|52.53|6.06|false|katie van buren|2013-03-01 09:11:58.703323|93.11|opthamology +79|422|65691|4294967299|39.21|29.52|true|zach quirinius|2013-03-01 09:11:58.703136|38.23|philosophy +27|367|65675|4294967518|12.32|40.26|true|tom brown|2013-03-01 09:11:58.703104|87.36|mathematics +96|421|65692|4294967526|98.48|34.41|true|oscar falkner|2013-03-01 09:11:58.703214|54.76|zync studies +27|340|65677|4294967461|98.96|45.78|true|rachel carson|2013-03-01 09:11:58.703208|25.37|zync studies +3|469|65743|4294967428|10.66|39.84|false|victor zipper|2013-03-01 09:11:58.703181|26.60|mathematics +122|288|65695|4294967469|90.05|12.70|false|zach steinbeck|2013-03-01 09:11:58.703314|39.12|education +81|449|65726|4294967395|97.09|40.15|false|bob ovid|2013-03-01 09:11:58.703250|60.46|kindergarten +77|337|65766|4294967334|12.16|13.49|true|wendy quirinius|2013-03-01 09:11:58.703282|37.06|history +5|319|65611|4294967494|45.92|2.61|true|wendy nixon|2013-03-01 09:11:58.703209|25.92|nap time +8|469|65592|4294967535|17.79|48.89|false|yuri steinbeck|2013-03-01 09:11:58.703299|23.19|kindergarten +74|487|65563|4294967305|60.06|13.22|true|ulysses polk|2013-03-01 09:11:58.703222|38.23|yard duty +85|454|65620|4294967520|65.55|13.20|false|victor quirinius|2013-03-01 09:11:58.703152|6.85|history +104|448|65693|4294967459|80.92|36.70|false|rachel falkner|2013-03-01 09:11:58.703140|32.33|zync studies +114|279|65544|4294967383|6.67|19.87|false|xavier hernandez|2013-03-01 09:11:58.703324|48.84|geology +120|331|65539|4294967324|88.02|40.94|true|holly nixon|2013-03-01 09:11:58.703262|96.64|yard duty +84|326|65782|4294967362|17.87|38.67|true|victor ellison|2013-03-01 09:11:58.703226|34.31|joggying +45|408|65577|4294967461|59.43|36.13|false|ethan falkner|2013-03-01 09:11:58.703317|41.01|values clariffication +88|504|65652|4294967419|78.28|47.86|false|priscilla white|2013-03-01 09:11:58.703324|55.04|history +73|306|65659|4294967341|34.58|45.90|false|yuri white|2013-03-01 09:11:58.703201|75.91|nap time +97|360|65590|4294967324|68.96|49.52|false|alice miller|2013-03-01 09:11:58.703247|22.13|philosophy +89|300|65675|4294967384|49.77|12.43|true|jessica davidson|2013-03-01 09:11:58.703276|79.90|values clariffication +118|309|65578|4294967357|75.10|33.02|false|tom ellison|2013-03-01 09:11:58.703197|16.01|zync studies +34|324|65549|4294967307|17.16|38.94|false|quinn underhill|2013-03-01 09:11:58.703111|28.93|forestry +93|441|65549|4294967389|95.11|0.27|true|quinn davidson|2013-03-01 09:11:58.703278|13.30|joggying +117|323|65669|4294967316|90.25|41.90|false|alice falkner|2013-03-01 09:11:58.703221|54.25|study skills +118|374|65541|4294967528|6.72|38.04|false|priscilla ovid|2013-03-01 09:11:58.703304|9.04|undecided +120|298|65763|4294967513|3.33|21.11|false|oscar ichabod|2013-03-01 09:11:58.703094|86.43|education +100|354|65649|4294967358|7.54|35.36|true|zach zipper|2013-03-01 09:11:58.703174|79.51|history +37|379|65737|4294967314|88.17|9.21|true|alice hernandez|2013-03-01 09:11:58.703089|29.92|opthamology +115|502|65773|4294967522|37.59|0.02|true|yuri laertes|2013-03-01 09:11:58.703260|3.81|american history +73|324|65582|4294967330|59.68|32.25|false|jessica ovid|2013-03-01 09:11:58.703109|96.60|religion +51|484|65635|4294967500|2.18|25.75|true|oscar van buren|2013-03-01 09:11:58.703255|82.29|yard duty +16|445|65620|4294967517|21.61|0.86|true|mike van buren|2013-03-01 09:11:58.703110|8.16|opthamology +102|501|65710|4294967332|47.27|1.31|false|xavier underhill|2013-03-01 09:11:58.703094|57.11|geology +28|372|65578|4294967374|75.42|5.93|true|holly thompson|2013-03-01 09:11:58.703205|54.35|debate +12|372|65573|4294967345|94.31|35.23|false|calvin falkner|2013-03-01 09:11:58.703122|53.56|debate +19|492|65588|4294967445|19.06|14.83|false|tom ichabod|2013-03-01 09:11:58.703171|69.48|nap time +70|430|65539|4294967534|18.89|43.84|true|tom carson|2013-03-01 09:11:58.703182|21.93|joggying +33|321|65594|4294967495|97.71|3.40|false|fred young|2013-03-01 09:11:58.703288|60.57|zync studies +44|452|65645|4294967345|56.15|16.90|true|oscar johnson|2013-03-01 09:11:58.703257|66.76|industrial engineering +53|293|65778|4294967418|16.09|14.99|true|oscar johnson|2013-03-01 09:11:58.703235|89.51|geology +73|451|65599|4294967373|36.89|33.64|true|zach ichabod|2013-03-01 09:11:58.703315|4.93|opthamology +104|474|65773|4294967473|40.98|11.43|false|quinn garcia|2013-03-01 09:11:58.703167|76.46|values clariffication +31|356|65721|4294967481|1.08|3.03|true|rachel ovid|2013-03-01 09:11:58.703120|95.38|zync studies +23|306|65777|4294967489|13.99|21.69|true|oscar zipper|2013-03-01 09:11:58.703280|54.39|zync studies +93|342|65623|4294967405|99.67|44.90|true|ulysses underhill|2013-03-01 09:11:58.703083|31.22|biology +18|440|65773|4294967318|57.23|1.42|false|oscar xylophone|2013-03-01 09:11:58.703165|63.26|zync studies +72|489|65549|4294967424|82.56|13.88|false|bob ichabod|2013-03-01 09:11:58.703112|80.09|quiet hour +12|284|65666|4294967470|89.80|42.04|false|ulysses garcia|2013-03-01 09:11:58.703073|28.69|opthamology +0|312|65599|4294967519|78.21|45.53|false|alice xylophone|2013-03-01 09:11:58.703289|0.56|nap time +24|483|65572|4294967499|79.42|9.96|false|xavier xylophone|2013-03-01 09:11:58.703101|22.64|quiet hour +34|405|65719|4294967306|1.31|23.48|false|katie white|2013-03-01 09:11:58.703198|51.13|nap time +99|497|65656|4294967390|32.37|6.21|true|wendy polk|2013-03-01 09:11:58.703212|53.27|mathematics +83|493|65742|4294967352|76.33|28.36|true|ethan carson|2013-03-01 09:11:58.703269|43.84|philosophy +106|282|65688|4294967433|75.19|13.08|true|yuri brown|2013-03-01 09:11:58.703118|77.88|debate +81|305|65783|4294967443|39.01|33.02|true|bob king|2013-03-01 09:11:58.703114|77.40|chemistry +74|309|65662|4294967518|4.17|38.43|false|rachel thompson|2013-03-01 09:11:58.703243|76.11|undecided +63|364|65705|4294967490|73.68|4.96|true|wendy white|2013-03-01 09:11:58.703186|13.81|forestry +46|363|65735|4294967537|72.79|29.22|true|ulysses brown|2013-03-01 09:11:58.703271|61.16|joggying +84|323|65685|4294967477|61.86|14.91|false|ethan brown|2013-03-01 09:11:58.703256|38.71|biology +4|392|65665|4294967391|53.27|3.86|true|zach miller|2013-03-01 09:11:58.703296|43.66|undecided +94|474|65759|4294967317|57.08|31.36|false|zach brown|2013-03-01 09:11:58.703239|69.24|xylophone band +26|491|65683|4294967420|46.62|48.52|false|wendy laertes|2013-03-01 09:11:58.703178|97.21|values clariffication +35|488|65737|4294967502|62.52|3.15|true|xavier quirinius|2013-03-01 09:11:58.703096|42.64|linguistics +59|382|65762|4294967468|25.28|35.10|true|david robinson|2013-03-01 09:11:58.703126|5.49|mathematics +58|427|65597|4294967511|71.19|47.55|false|xavier ichabod|2013-03-01 09:11:58.703323|36.22|chemistry +18|428|65775|4294967436|74.19|48.08|true|irene xylophone|2013-03-01 09:11:58.703122|98.43|geology +69|489|65536|4294967404|33.52|17.99|false|oscar ichabod|2013-03-01 09:11:58.703247|32.68|topology +71|439|65618|4294967349|49.78|4.57|false|ethan hernandez|2013-03-01 09:11:58.703075|10.06|yard duty +60|475|65664|4294967299|44.43|25.02|false|irene laertes|2013-03-01 09:11:58.703172|55.82|quiet hour +35|280|65779|4294967322|1.87|16.04|false|luke ellison|2013-03-01 09:11:58.703180|11.87|religion +122|473|65629|4294967387|90.77|25.59|true|fred underhill|2013-03-01 09:11:58.703316|58.81|undecided +-1|423|65663|4294967380|0.79|21.33|false|bob laertes|2013-03-01 09:11:58.703278|94.16|debate +68|314|65770|4294967398|70.85|29.24|true|mike garcia|2013-03-01 09:11:58.703312|51.90|american history +77|328|65789|4294967489|81.32|19.12|false|fred ichabod|2013-03-01 09:11:58.703160|83.18|debate +60|342|65550|4294967306|90.38|45.69|false|victor brown|2013-03-01 09:11:58.703311|7.38|biology +53|422|65784|4294967450|56.04|43.76|true|calvin falkner|2013-03-01 09:11:58.703193|93.95|zync studies +103|504|65768|4294967354|53.93|16.42|true|bob davidson|2013-03-01 09:11:58.703141|78.25|kindergarten +8|272|65541|4294967325|63.90|11.20|true|oscar polk|2013-03-01 09:11:58.703136|27.89|debate +108|446|65733|4294967403|7.11|32.85|false|ethan brown|2013-03-01 09:11:58.703170|11.44|american history +42|315|65782|4294967369|4.46|11.63|false|bob garcia|2013-03-01 09:11:58.703292|30.24|industrial engineering +16|482|65736|4294967310|43.19|46.30|true|ethan garcia|2013-03-01 09:11:58.703243|28.90|wind surfing +28|454|65612|4294967480|65.02|43.00|false|ulysses polk|2013-03-01 09:11:58.703140|25.10|philosophy +58|482|65775|4294967417|68.04|19.41|false|wendy underhill|2013-03-01 09:11:58.703212|88.25|yard duty +88|437|65762|4294967385|76.71|13.83|false|david laertes|2013-03-01 09:11:58.703169|45.70|religion +83|265|65705|4294967392|19.28|5.42|true|quinn ellison|2013-03-01 09:11:58.703275|20.57|education +2|266|65551|4294967296|35.65|45.12|true|david young|2013-03-01 09:11:58.703184|73.93|education +91|415|65672|4294967316|59.45|20.80|true|mike hernandez|2013-03-01 09:11:58.703241|75.27|study skills +34|379|65750|4294967441|98.18|20.58|true|alice underhill|2013-03-01 09:11:58.703170|96.85|mathematics +86|451|65580|4294967451|8.71|47.68|true|ulysses polk|2013-03-01 09:11:58.703165|81.42|geology +46|266|65762|4294967305|2.97|19.56|false|calvin falkner|2013-03-01 09:11:58.703103|77.96|chemistry +81|338|65681|4294967519|50.66|16.10|false|katie ovid|2013-03-01 09:11:58.703259|61.30|wind surfing +17|441|65633|4294967460|89.15|21.11|true|priscilla johnson|2013-03-01 09:11:58.703139|28.49|education +112|312|65685|4294967395|33.76|21.38|true|wendy young|2013-03-01 09:11:58.703182|4.88|religion +44|270|65751|4294967301|67.48|37.12|false|oscar garcia|2013-03-01 09:11:58.703239|47.81|religion +37|411|65680|4294967353|2.92|29.66|true|ethan zipper|2013-03-01 09:11:58.703132|34.08|american history +95|329|65628|4294967396|61.70|20.75|true|david ovid|2013-03-01 09:11:58.703223|83.26|nap time +33|497|65612|4294967375|28.69|14.84|true|fred nixon|2013-03-01 09:11:58.703070|52.64|values clariffication +55|328|65703|4294967492|68.41|0.63|true|ulysses carson|2013-03-01 09:11:58.703282|37.18|geology +17|483|65713|4294967458|56.81|1.69|false|calvin xylophone|2013-03-01 09:11:58.703250|90.91|mathematics +61|510|65675|4294967391|88.07|13.44|false|ulysses young|2013-03-01 09:11:58.703281|24.19|joggying +53|279|65541|4294967441|77.97|14.72|false|xavier ellison|2013-03-01 09:11:58.703091|15.85|american history +114|304|65643|4294967457|54.75|46.53|false|ethan laertes|2013-03-01 09:11:58.703250|52.56|mathematics +61|405|65609|4294967301|16.48|48.50|false|mike falkner|2013-03-01 09:11:58.703159|83.82|wind surfing +93|433|65629|4294967439|39.83|8.56|false|katie king|2013-03-01 09:11:58.703125|72.90|values clariffication +6|348|65785|4294967326|57.37|33.90|true|ulysses underhill|2013-03-01 09:11:58.703226|68.30|opthamology +39|492|65701|4294967458|23.96|19.65|true|fred polk|2013-03-01 09:11:58.703262|1.81|xylophone band +74|256|65778|4294967487|74.15|25.49|true|david davidson|2013-03-01 09:11:58.703151|95.69|yard duty +99|435|65546|4294967535|22.85|3.12|false|tom young|2013-03-01 09:11:58.703111|91.13|history +106|441|65771|4294967386|83.93|28.44|false|xavier allen|2013-03-01 09:11:58.703226|70.88|xylophone band +124|392|65711|4294967498|15.37|18.83|false|irene garcia|2013-03-01 09:11:58.703158|71.89|industrial engineering +9|336|65658|4294967412|43.17|18.38|false|katie ichabod|2013-03-01 09:11:58.703091|68.85|mathematics +3|395|65747|4294967313|57.25|3.17|true|wendy garcia|2013-03-01 09:11:58.703074|58.47|xylophone band +123|381|65780|4294967388|5.44|35.16|true|tom carson|2013-03-01 09:11:58.703220|10.95|quiet hour +58|256|65733|4294967501|70.53|23.07|true|zach white|2013-03-01 09:11:58.703090|50.99|history +106|284|65619|4294967527|92.96|0.86|true|mike nixon|2013-03-01 09:11:58.703084|32.64|xylophone band +58|289|65604|4294967313|77.36|10.49|false|wendy falkner|2013-03-01 09:11:58.703075|23.89|kindergarten +6|342|65602|4294967365|85.49|24.46|false|zach steinbeck|2013-03-01 09:11:58.703111|85.94|study skills +89|354|65593|4294967462|97.51|21.01|true|ethan zipper|2013-03-01 09:11:58.703201|35.15|forestry +44|364|65614|4294967413|4.35|9.19|false|ethan king|2013-03-01 09:11:58.703219|1.42|nap time +43|436|65678|4294967424|50.92|3.40|true|luke van buren|2013-03-01 09:11:58.703093|33.90|values clariffication +27|335|65617|4294967381|64.87|25.03|false|david allen|2013-03-01 09:11:58.703140|64.90|nap time +44|390|65693|4294967504|1.29|28.40|false|bob davidson|2013-03-01 09:11:58.703127|84.88|history +42|315|65713|4294967544|88.48|46.90|true|ulysses underhill|2013-03-01 09:11:58.703259|39.47|education +54|327|65657|4294967334|42.42|2.89|true|jessica zipper|2013-03-01 09:11:58.703182|65.01|linguistics +68|501|65606|4294967445|5.08|39.85|true|alice robinson|2013-03-01 09:11:58.703206|87.16|topology +37|293|65762|4294967316|67.38|41.20|true|zach brown|2013-03-01 09:11:58.703155|21.70|forestry +10|471|65563|4294967316|37.72|45.06|false|tom quirinius|2013-03-01 09:11:58.703248|14.36|quiet hour +42|353|65672|4294967465|70.04|27.62|false|xavier garcia|2013-03-01 09:11:58.703153|13.28|forestry +15|271|65647|4294967392|45.06|35.36|true|quinn young|2013-03-01 09:11:58.703209|81.95|kindergarten +71|332|65717|4294967337|10.73|34.17|true|gabriella ichabod|2013-03-01 09:11:58.703288|80.05|linguistics +92|418|65706|4294967367|2.07|16.35|false|yuri hernandez|2013-03-01 09:11:58.703205|85.13|joggying +101|326|65586|4294967435|30.81|4.27|true|rachel brown|2013-03-01 09:11:58.703179|18.65|undecided +76|422|65552|4294967325|26.73|48.00|false|mike zipper|2013-03-01 09:11:58.703137|18.11|industrial engineering +119|373|65704|4294967459|27.07|27.54|false|mike polk|2013-03-01 09:11:58.703202|54.76|biology +70|283|65620|4294967355|9.13|3.42|false|zach falkner|2013-03-01 09:11:58.703252|96.95|debate +117|267|65637|4294967544|93.64|11.69|false|david brown|2013-03-01 09:11:58.703224|26.78|topology +88|328|65547|4294967517|70.35|15.13|true|alice davidson|2013-03-01 09:11:58.703262|17.71|undecided +22|447|65757|4294967539|8.79|6.52|true|bob king|2013-03-01 09:11:58.703325|9.24|joggying +-1|300|65663|4294967343|71.26|34.62|true|calvin ovid|2013-03-01 09:11:58.703262|78.56|study skills +92|297|65704|4294967403|84.83|0.21|false|gabriella brown|2013-03-01 09:11:58.703089|16.22|wind surfing +25|446|65789|4294967501|5.54|49.56|false|jessica garcia|2013-03-01 09:11:58.703286|92.74|linguistics +65|274|65619|4294967389|12.45|41.33|true|mike polk|2013-03-01 09:11:58.703219|34.72|nap time +111|287|65587|4294967421|48.22|12.74|false|yuri johnson|2013-03-01 09:11:58.703227|64.06|values clariffication +64|371|65685|4294967320|15.63|39.84|false|ethan brown|2013-03-01 09:11:58.703132|43.48|opthamology +72|421|65764|4294967458|88.80|39.49|true|rachel falkner|2013-03-01 09:11:58.703227|88.69|topology +108|301|65536|4294967357|90.05|17.59|true|ethan johnson|2013-03-01 09:11:58.703271|75.70|undecided +-2|461|65648|4294967425|58.52|24.85|false|rachel thompson|2013-03-01 09:11:58.703318|85.62|zync studies +113|395|65666|4294967447|26.49|13.44|true|tom steinbeck|2013-03-01 09:11:58.703247|83.95|industrial engineering +106|415|65644|4294967399|63.35|18.38|false|xavier davidson|2013-03-01 09:11:58.703234|44.10|study skills +117|483|65627|4294967547|21.18|49.46|true|priscilla ichabod|2013-03-01 09:11:58.703184|61.89|philosophy +44|460|65693|4294967423|58.00|23.77|false|quinn zipper|2013-03-01 09:11:58.703165|8.38|kindergarten +71|324|65681|4294967388|47.59|33.67|false|nick ichabod|2013-03-01 09:11:58.703106|71.17|xylophone band +90|269|65648|4294967392|42.31|11.27|true|holly king|2013-03-01 09:11:58.703196|0.08|american history +22|264|65537|4294967419|8.07|10.71|false|david xylophone|2013-03-01 09:11:58.703136|11.87|undecided +51|447|65791|4294967354|28.95|44.27|false|luke davidson|2013-03-01 09:11:58.703292|4.24|chemistry +6|397|65660|4294967512|0.27|24.95|false|nick young|2013-03-01 09:11:58.703156|62.42|industrial engineering +122|392|65619|4294967527|20.81|27.27|false|nick underhill|2013-03-01 09:11:58.703305|18.68|education +43|461|65686|4294967544|20.82|30.99|false|holly hernandez|2013-03-01 09:11:58.703286|98.68|history +21|280|65716|4294967393|48.08|29.62|true|gabriella ellison|2013-03-01 09:11:58.703264|85.93|american history +10|453|65745|4294967397|89.01|24.03|true|nick ellison|2013-03-01 09:11:58.703108|83.13|topology +56|392|65586|4294967410|48.15|29.76|true|alice nixon|2013-03-01 09:11:58.703223|94.25|american history +95|415|65560|4294967444|4.71|17.29|false|quinn laertes|2013-03-01 09:11:58.703267|92.59|xylophone band +13|465|65713|4294967306|26.76|6.56|true|yuri polk|2013-03-01 09:11:58.703169|47.77|american history +34|425|65660|4294967371|58.43|15.54|true|irene garcia|2013-03-01 09:11:58.703172|28.05|forestry +59|496|65576|4294967331|15.75|28.68|false|tom king|2013-03-01 09:11:58.703215|66.55|opthamology +-1|433|65581|4294967299|86.92|23.15|false|yuri ellison|2013-03-01 09:11:58.703098|21.29|history +13|260|65699|4294967496|41.87|9.26|false|sarah king|2013-03-01 09:11:58.703128|84.79|study skills +45|298|65747|4294967434|15.15|31.01|false|priscilla polk|2013-03-01 09:11:58.703113|21.80|yard duty +31|366|65557|4294967458|41.71|1.98|false|sarah miller|2013-03-01 09:11:58.703216|69.92|linguistics +63|464|65596|4294967327|44.57|0.47|true|holly allen|2013-03-01 09:11:58.703200|12.72|nap time +5|357|65736|4294967351|0.60|15.38|false|rachel ovid|2013-03-01 09:11:58.703246|86.51|xylophone band +16|483|65658|4294967350|38.88|2.89|false|oscar robinson|2013-03-01 09:11:58.703225|66.53|topology +90|487|65695|4294967313|57.93|15.18|false|yuri quirinius|2013-03-01 09:11:58.703320|77.09|education +117|302|65615|4294967382|21.32|31.46|false|ethan polk|2013-03-01 09:11:58.703274|10.79|kindergarten +75|504|65564|4294967534|7.56|19.79|false|mike steinbeck|2013-03-01 09:11:58.703148|26.97|xylophone band +86|257|65748|4294967427|32.52|26.87|true|ulysses young|2013-03-01 09:11:58.703192|29.49|values clariffication +112|348|65643|4294967388|30.61|21.27|false|oscar polk|2013-03-01 09:11:58.703076|22.05|yard duty +91|307|65544|4294967344|4.57|0.60|false|rachel robinson|2013-03-01 09:11:58.703227|79.21|undecided +49|407|65639|4294967362|27.66|34.06|true|yuri garcia|2013-03-01 09:11:58.703163|97.48|mathematics +95|459|65760|4294967524|85.74|49.16|false|mike ellison|2013-03-01 09:11:58.703074|48.20|nap time +76|258|65724|4294967365|70.00|34.95|false|irene quirinius|2013-03-01 09:11:58.703212|10.52|geology +17|406|65585|4294967401|0.43|29.19|true|priscilla young|2013-03-01 09:11:58.703235|33.69|religion +90|273|65548|4294967366|62.90|12.30|true|alice polk|2013-03-01 09:11:58.703074|53.52|study skills +41|301|65706|4294967501|51.84|7.02|true|ethan quirinius|2013-03-01 09:11:58.703265|55.94|study skills +38|376|65783|4294967507|10.19|38.28|false|tom quirinius|2013-03-01 09:11:58.703185|52.43|kindergarten +50|317|65709|4294967379|68.50|7.36|true|victor nixon|2013-03-01 09:11:58.703162|67.69|forestry +33|265|65713|4294967529|86.69|16.40|true|holly thompson|2013-03-01 09:11:58.703189|72.37|history +89|476|65728|4294967549|62.30|14.07|false|alice quirinius|2013-03-01 09:11:58.703195|55.37|zync studies +24|339|65724|4294967328|64.95|37.02|false|rachel robinson|2013-03-01 09:11:58.703182|75.06|wind surfing +124|421|65743|4294967434|3.62|21.02|true|holly polk|2013-03-01 09:11:58.703173|62.00|education +85|463|65542|4294967348|3.86|46.36|true|ethan quirinius|2013-03-01 09:11:58.703287|42.85|biology +66|264|65600|4294967388|28.71|43.96|false|priscilla nixon|2013-03-01 09:11:58.703169|92.92|yard duty +112|373|65680|4294967449|90.28|49.34|false|sarah falkner|2013-03-01 09:11:58.703085|1.01|geology +28|480|65637|4294967328|76.69|33.76|false|oscar ichabod|2013-03-01 09:11:58.703323|93.41|kindergarten +101|476|65631|4294967538|29.78|23.61|false|irene nixon|2013-03-01 09:11:58.703270|33.72|values clariffication +96|435|65658|4294967523|6.74|28.52|false|yuri falkner|2013-03-01 09:11:58.703099|11.69|joggying +54|478|65615|4294967344|9.74|21.52|false|jessica van buren|2013-03-01 09:11:58.703284|99.88|philosophy +89|354|65777|4294967509|71.68|46.03|true|ulysses polk|2013-03-01 09:11:58.703324|82.62|wind surfing +16|436|65757|4294967379|7.06|28.15|true|rachel zipper|2013-03-01 09:11:58.703267|12.22|forestry +78|275|65596|4294967299|21.70|49.32|true|calvin xylophone|2013-03-01 09:11:58.703166|85.74|kindergarten +53|489|65752|4294967329|83.27|27.20|true|holly ichabod|2013-03-01 09:11:58.703079|47.82|industrial engineering +111|288|65770|4294967544|80.84|16.79|true|mike van buren|2013-03-01 09:11:58.703220|46.71|topology +85|334|65610|4294967373|2.96|2.36|false|ulysses miller|2013-03-01 09:11:58.703310|9.24|xylophone band +71|418|65598|4294967439|80.30|13.66|true|bob garcia|2013-03-01 09:11:58.703104|63.30|values clariffication +40|299|65712|4294967352|25.37|19.72|false|ethan miller|2013-03-01 09:11:58.703207|53.65|opthamology +32|458|65783|4294967309|45.42|44.83|false|jessica underhill|2013-03-01 09:11:58.703206|46.34|undecided +5|329|65789|4294967502|56.10|47.22|true|alice robinson|2013-03-01 09:11:58.703264|52.49|quiet hour +34|450|65607|4294967328|18.20|18.79|true|victor johnson|2013-03-01 09:11:58.703233|71.75|topology +58|359|65675|4294967486|3.61|19.20|true|katie ellison|2013-03-01 09:11:58.703243|35.78|values clariffication +-2|288|65658|4294967383|53.78|19.92|false|holly nixon|2013-03-01 09:11:58.703224|42.93|industrial engineering +0|392|65634|4294967299|65.70|48.98|true|luke robinson|2013-03-01 09:11:58.703263|14.40|geology +3|277|65788|4294967403|58.08|20.55|false|xavier ovid|2013-03-01 09:11:58.703281|62.11|zync studies +29|331|65539|4294967420|73.18|28.96|true|ethan brown|2013-03-01 09:11:58.703094|58.85|zync studies +57|420|65771|4294967508|34.21|16.12|false|calvin van buren|2013-03-01 09:11:58.703243|6.15|joggying +41|266|65774|4294967365|41.68|36.86|false|victor van buren|2013-03-01 09:11:58.703163|45.97|industrial engineering +116|487|65659|4294967309|77.66|35.15|true|sarah johnson|2013-03-01 09:11:58.703294|95.71|opthamology +54|481|65755|4294967429|28.50|37.76|false|victor xylophone|2013-03-01 09:11:58.703318|64.00|xylophone band +60|301|65746|4294967381|4.41|12.30|true|wendy garcia|2013-03-01 09:11:58.703183|36.74|education +104|266|65704|4294967413|59.16|34.84|true|jessica ichabod|2013-03-01 09:11:58.703172|93.21|forestry +105|345|65582|4294967478|12.42|15.27|false|wendy miller|2013-03-01 09:11:58.703287|0.89|philosophy +97|503|65690|4294967463|77.57|27.54|true|priscilla brown|2013-03-01 09:11:58.703110|51.82|study skills +18|286|65621|4294967494|64.77|37.32|true|mike ichabod|2013-03-01 09:11:58.703285|95.14|forestry +1|446|65683|4294967500|85.49|33.18|true|oscar hernandez|2013-03-01 09:11:58.703283|17.62|undecided +12|298|65653|4294967501|42.56|4.25|true|irene nixon|2013-03-01 09:11:58.703265|52.23|nap time +5|475|65644|4294967470|70.06|18.29|true|gabriella van buren|2013-03-01 09:11:58.703088|1.97|american history +54|321|65724|4294967486|47.69|49.73|true|david ellison|2013-03-01 09:11:58.703235|50.96|religion +88|331|65682|4294967412|73.32|17.76|true|gabriella thompson|2013-03-01 09:11:58.703283|67.17|values clariffication +56|310|65556|4294967518|38.85|34.53|false|irene allen|2013-03-01 09:11:58.703217|50.42|education +71|306|65664|4294967541|34.97|44.40|false|victor van buren|2013-03-01 09:11:58.703185|38.42|religion +114|402|65622|4294967542|59.87|24.61|true|ethan polk|2013-03-01 09:11:58.703265|93.70|yard duty +43|440|65693|4294967409|38.05|7.71|true|luke ovid|2013-03-01 09:11:58.703133|69.32|xylophone band +26|296|65677|4294967419|66.89|49.04|true|sarah robinson|2013-03-01 09:11:58.703277|5.06|wind surfing +68|472|65628|4294967512|8.67|19.47|false|tom ovid|2013-03-01 09:11:58.703116|74.31|linguistics +53|373|65691|4294967371|79.75|32.39|false|irene ovid|2013-03-01 09:11:58.703124|69.80|nap time +18|509|65638|4294967455|62.92|14.62|false|victor laertes|2013-03-01 09:11:58.703075|42.47|values clariffication +53|482|65736|4294967394|74.59|36.84|true|mike young|2013-03-01 09:11:58.703301|48.54|joggying +103|494|65760|4294967301|59.21|8.72|false|ethan laertes|2013-03-01 09:11:58.703312|95.45|nap time +80|277|65671|4294967370|97.26|7.14|false|sarah ichabod|2013-03-01 09:11:58.703297|36.34|industrial engineering +28|380|65714|4294967514|0.28|48.71|false|ethan ellison|2013-03-01 09:11:58.703118|47.42|chemistry +1|414|65716|4294967551|12.19|36.57|true|nick polk|2013-03-01 09:11:58.703188|33.40|xylophone band +18|417|65766|4294967499|37.78|40.96|false|ethan nixon|2013-03-01 09:11:58.703157|66.34|american history +52|511|65658|4294967432|7.96|14.33|false|priscilla carson|2013-03-01 09:11:58.703188|46.61|geology +13|374|65553|4294967448|62.20|10.15|false|jessica carson|2013-03-01 09:11:58.703185|25.31|debate +19|462|65732|4294967478|59.70|24.13|false|calvin underhill|2013-03-01 09:11:58.703074|91.15|debate +87|325|65730|4294967516|13.15|4.44|false|wendy ichabod|2013-03-01 09:11:58.703242|67.81|joggying +99|303|65566|4294967353|98.57|40.50|true|ulysses ichabod|2013-03-01 09:11:58.703104|7.80|wind surfing +87|341|65733|4294967412|18.93|16.13|true|katie zipper|2013-03-01 09:11:58.703190|93.45|american history +83|311|65653|4294967419|61.21|13.04|false|xavier polk|2013-03-01 09:11:58.703310|55.12|study skills +65|458|65554|4294967329|69.96|33.38|false|calvin ovid|2013-03-01 09:11:58.703237|0.24|nap time +123|278|65573|4294967496|86.22|39.74|true|zach underhill|2013-03-01 09:11:58.703078|75.99|history +19|463|65767|4294967437|48.52|26.71|true|quinn underhill|2013-03-01 09:11:58.703177|95.57|forestry +108|466|65600|4294967458|90.73|8.95|false|xavier brown|2013-03-01 09:11:58.703235|75.18|biology +53|435|65687|4294967451|76.70|4.58|false|wendy xylophone|2013-03-01 09:11:58.703243|11.26|nap time +54|346|65751|4294967398|98.31|34.54|false|holly polk|2013-03-01 09:11:58.703195|78.75|history +88|334|65583|4294967468|5.88|45.10|false|irene johnson|2013-03-01 09:11:58.703166|28.07|kindergarten +52|365|65691|4294967551|99.15|21.81|false|tom robinson|2013-03-01 09:11:58.703135|76.98|values clariffication +28|259|65544|4294967398|54.31|25.80|false|yuri quirinius|2013-03-01 09:11:58.703168|32.03|values clariffication +26|347|65734|4294967499|5.24|30.92|true|irene ovid|2013-03-01 09:11:58.703192|80.48|debate +108|280|65765|4294967434|4.79|30.35|false|irene brown|2013-03-01 09:11:58.703229|88.52|american history +2|291|65789|4294967464|16.99|2.07|false|bob falkner|2013-03-01 09:11:58.703139|43.53|industrial engineering +43|476|65570|4294967398|1.12|10.52|true|yuri ellison|2013-03-01 09:11:58.703244|30.98|forestry +114|433|65572|4294967463|43.02|20.30|false|zach carson|2013-03-01 09:11:58.703316|30.86|opthamology +119|278|65773|4294967488|31.15|23.18|true|wendy xylophone|2013-03-01 09:11:58.703312|18.20|xylophone band +108|263|65787|4294967297|12.44|29.46|true|rachel quirinius|2013-03-01 09:11:58.703073|21.76|industrial engineering +37|434|65638|4294967335|55.18|45.54|false|gabriella white|2013-03-01 09:11:58.703196|19.13|zync studies +124|311|65670|4294967420|87.94|3.07|false|wendy king|2013-03-01 09:11:58.703308|61.54|values clariffication +104|331|65757|4294967330|30.71|47.91|false|katie ichabod|2013-03-01 09:11:58.703197|34.84|study skills +25|438|65782|4294967525|29.40|45.71|true|fred quirinius|2013-03-01 09:11:58.703283|81.90|nap time +71|347|65584|4294967549|99.65|21.14|false|jessica falkner|2013-03-01 09:11:58.703286|65.78|philosophy +51|345|65566|4294967519|70.38|31.33|true|wendy laertes|2013-03-01 09:11:58.703163|21.34|chemistry +2|333|65697|4294967456|39.90|16.15|false|yuri johnson|2013-03-01 09:11:58.703272|77.35|study skills +114|425|65772|4294967307|55.10|34.83|true|luke robinson|2013-03-01 09:11:58.703166|82.33|religion +109|344|65595|4294967409|42.24|5.35|false|irene polk|2013-03-01 09:11:58.703089|8.76|education +95|432|65711|4294967378|39.87|30.78|false|wendy allen|2013-03-01 09:11:58.703172|65.17|american history +121|342|65565|4294967528|52.85|20.30|true|yuri allen|2013-03-01 09:11:58.703298|8.80|undecided +22|361|65648|4294967546|91.88|17.19|false|mike white|2013-03-01 09:11:58.703167|17.66|quiet hour +81|283|65702|4294967424|24.52|6.81|false|zach king|2013-03-01 09:11:58.703136|68.04|religion +15|297|65728|4294967362|18.50|24.18|false|katie falkner|2013-03-01 09:11:58.703189|36.96|education +77|441|65727|4294967391|43.13|17.47|false|rachel young|2013-03-01 09:11:58.703313|49.19|education +24|266|65747|4294967351|32.75|36.09|true|ethan allen|2013-03-01 09:11:58.703273|52.77|zync studies +89|313|65550|4294967361|41.89|33.02|true|david thompson|2013-03-01 09:11:58.703121|96.90|biology +101|317|65756|4294967505|4.80|41.36|true|luke laertes|2013-03-01 09:11:58.703106|11.81|industrial engineering +37|472|65545|4294967351|83.54|16.90|true|mike young|2013-03-01 09:11:58.703196|34.65|nap time +50|402|65781|4294967410|65.38|28.10|false|gabriella laertes|2013-03-01 09:11:58.703126|70.59|philosophy +105|435|65637|4294967534|80.23|13.30|true|calvin carson|2013-03-01 09:11:58.703203|29.34|forestry +33|258|65762|4294967417|15.22|13.23|false|calvin steinbeck|2013-03-01 09:11:58.703322|87.50|nap time +118|486|65651|4294967472|32.23|46.39|false|fred steinbeck|2013-03-01 09:11:58.703142|24.25|mathematics +47|306|65636|4294967524|54.10|30.78|false|ulysses xylophone|2013-03-01 09:11:58.703309|86.29|mathematics +13|469|65661|4294967404|99.92|18.34|true|irene falkner|2013-03-01 09:11:58.703244|8.06|undecided +8|267|65703|4294967473|75.29|9.81|false|xavier white|2013-03-01 09:11:58.703233|40.81|nap time +18|263|65582|4294967500|78.64|36.05|false|gabriella steinbeck|2013-03-01 09:11:58.703208|1.23|industrial engineering +122|354|65709|4294967334|74.45|44.04|false|rachel allen|2013-03-01 09:11:58.703075|5.64|yard duty +43|340|65669|4294967500|23.19|20.58|true|katie nixon|2013-03-01 09:11:58.703080|1.76|biology +87|361|65662|4294967331|97.56|44.66|false|david underhill|2013-03-01 09:11:58.703303|88.64|opthamology +36|429|65723|4294967296|67.45|19.63|false|nick quirinius|2013-03-01 09:11:58.703297|39.90|biology +83|353|65538|4294967339|0.08|23.91|true|holly thompson|2013-03-01 09:11:58.703157|30.27|quiet hour +60|421|65774|4294967400|5.62|2.43|true|quinn thompson|2013-03-01 09:11:58.703128|45.74|study skills +42|323|65629|4294967324|8.45|30.67|true|luke ichabod|2013-03-01 09:11:58.703241|19.33|undecided +103|310|65603|4294967475|44.10|16.79|false|wendy ellison|2013-03-01 09:11:58.703154|34.79|mathematics +90|414|65702|4294967337|66.36|49.79|false|gabriella ichabod|2013-03-01 09:11:58.703171|37.60|biology +28|443|65635|4294967426|97.68|31.45|false|wendy falkner|2013-03-01 09:11:58.703186|64.99|yard duty +53|467|65718|4294967355|11.15|34.81|false|luke johnson|2013-03-01 09:11:58.703252|84.35|forestry +120|294|65590|4294967443|90.07|21.77|false|jessica nixon|2013-03-01 09:11:58.703086|13.51|education +44|260|65727|4294967453|60.53|19.03|true|oscar thompson|2013-03-01 09:11:58.703150|88.11|xylophone band +28|418|65756|4294967451|27.12|34.49|false|oscar laertes|2013-03-01 09:11:58.703206|63.51|biology +5|450|65594|4294967449|96.62|41.56|true|jessica white|2013-03-01 09:11:58.703323|65.61|biology +60|400|65698|4294967299|41.34|36.56|false|oscar thompson|2013-03-01 09:11:58.703073|79.91|xylophone band +0|314|65654|4294967473|12.85|35.01|false|xavier ellison|2013-03-01 09:11:58.703189|37.74|topology +31|426|65789|4294967526|58.13|15.09|false|tom zipper|2013-03-01 09:11:58.703258|89.00|american history +95|300|65568|4294967520|39.82|47.72|true|oscar zipper|2013-03-01 09:11:58.703285|58.66|quiet hour +122|384|65578|4294967358|8.45|49.21|true|quinn steinbeck|2013-03-01 09:11:58.703148|58.78|education +86|330|65676|4294967480|88.47|17.33|false|nick falkner|2013-03-01 09:11:58.703202|58.12|linguistics +71|276|65584|4294967431|8.07|20.91|true|bob brown|2013-03-01 09:11:58.703265|6.88|joggying +54|463|65701|4294967313|43.31|23.45|false|tom laertes|2013-03-01 09:11:58.703092|36.84|geology +112|372|65722|4294967529|41.44|20.22|false|irene ichabod|2013-03-01 09:11:58.703111|38.82|linguistics +65|323|65739|4294967424|4.47|25.51|false|alice johnson|2013-03-01 09:11:58.703294|92.40|forestry +39|368|65590|4294967420|27.72|30.55|false|katie brown|2013-03-01 09:11:58.703182|42.53|zync studies +25|284|65753|4294967543|15.26|36.93|false|gabriella brown|2013-03-01 09:11:58.703107|86.97|wind surfing +55|325|65594|4294967337|7.98|2.82|true|bob polk|2013-03-01 09:11:58.703283|35.28|zync studies +63|458|65537|4294967463|21.94|49.71|true|fred van buren|2013-03-01 09:11:58.703278|99.34|wind surfing +61|300|65588|4294967412|75.73|41.75|true|zach brown|2013-03-01 09:11:58.703310|75.87|religion +47|328|65784|4294967411|94.08|2.79|false|nick johnson|2013-03-01 09:11:58.703214|31.35|linguistics +4|509|65776|4294967432|78.26|35.02|false|mike king|2013-03-01 09:11:58.703231|18.70|undecided +86|425|65626|4294967374|87.40|34.90|true|oscar underhill|2013-03-01 09:11:58.703276|63.54|education +123|499|65623|4294967521|43.37|4.98|true|jessica young|2013-03-01 09:11:58.703227|97.20|mathematics +16|380|65755|4294967349|56.68|26.55|false|priscilla carson|2013-03-01 09:11:58.703217|76.75|undecided +86|391|65620|4294967337|34.73|4.16|false|katie white|2013-03-01 09:11:58.703109|81.28|history +46|463|65622|4294967479|23.44|16.92|true|ethan polk|2013-03-01 09:11:58.703273|74.31|chemistry +79|486|65694|4294967305|10.25|9.64|false|david young|2013-03-01 09:11:58.703266|58.23|topology +57|456|65606|4294967422|72.56|30.28|true|victor johnson|2013-03-01 09:11:58.703222|55.31|kindergarten +92|306|65657|4294967337|50.26|46.88|false|wendy brown|2013-03-01 09:11:58.703124|37.39|history +36|350|65717|4294967481|92.98|11.91|true|gabriella hernandez|2013-03-01 09:11:58.703182|1.23|yard duty +60|459|65655|4294967535|36.95|41.86|true|holly johnson|2013-03-01 09:11:58.703301|77.41|linguistics +26|393|65561|4294967304|90.69|6.61|true|calvin white|2013-03-01 09:11:58.703266|47.71|geology +106|370|65625|4294967333|3.00|10.38|true|victor polk|2013-03-01 09:11:58.703112|79.65|undecided +62|427|65787|4294967323|86.93|12.64|false|irene garcia|2013-03-01 09:11:58.703215|18.78|religion +48|438|65575|4294967376|25.42|46.03|false|calvin xylophone|2013-03-01 09:11:58.703158|83.40|topology +101|327|65706|4294967340|71.54|11.48|false|gabriella ellison|2013-03-01 09:11:58.703321|3.91|nap time +9|350|65704|4294967355|21.23|12.66|true|tom miller|2013-03-01 09:11:58.703210|37.12|industrial engineering +83|336|65672|4294967347|9.71|8.05|false|priscilla quirinius|2013-03-01 09:11:58.703174|58.10|debate +108|476|65587|4294967306|2.96|34.11|false|rachel brown|2013-03-01 09:11:58.703130|5.83|quiet hour +54|341|65756|4294967522|1.08|17.63|true|david davidson|2013-03-01 09:11:58.703317|1.45|religion +48|459|65777|4294967312|76.72|2.75|false|wendy garcia|2013-03-01 09:11:58.703144|7.04|kindergarten +-3|438|65618|4294967398|62.39|4.62|false|victor xylophone|2013-03-01 09:11:58.703135|88.38|values clariffication +105|376|65602|4294967472|10.67|49.84|true|bob laertes|2013-03-01 09:11:58.703142|47.16|education +15|433|65691|4294967353|49.12|14.17|false|zach brown|2013-03-01 09:11:58.703155|6.93|kindergarten +117|276|65581|4294967457|69.80|33.64|true|gabriella van buren|2013-03-01 09:11:58.703290|56.59|industrial engineering +112|444|65721|4294967401|47.88|40.91|true|victor king|2013-03-01 09:11:58.703253|55.75|industrial engineering +71|260|65589|4294967329|2.35|6.98|true|ethan polk|2013-03-01 09:11:58.703268|94.73|geology +7|486|65773|4294967390|58.86|49.46|false|irene quirinius|2013-03-01 09:11:58.703273|30.49|quiet hour +14|474|65558|4294967538|74.62|23.78|true|quinn king|2013-03-01 09:11:58.703107|16.26|linguistics +7|343|65559|4294967371|48.01|48.80|true|zach brown|2013-03-01 09:11:58.703245|32.30|study skills +-2|304|65629|4294967386|84.38|33.49|false|zach garcia|2013-03-01 09:11:58.703264|56.99|chemistry +72|503|65727|4294967376|79.99|27.39|false|wendy laertes|2013-03-01 09:11:58.703274|88.83|zync studies +32|348|65747|4294967514|37.85|45.09|true|sarah white|2013-03-01 09:11:58.703309|11.16|mathematics +103|343|65773|4294967485|46.80|27.89|false|fred young|2013-03-01 09:11:58.703230|53.09|undecided +24|329|65561|4294967526|52.87|15.98|false|fred van buren|2013-03-01 09:11:58.703178|32.86|kindergarten +27|405|65536|4294967508|82.24|29.41|true|oscar ovid|2013-03-01 09:11:58.703166|16.85|biology +3|308|65757|4294967430|49.28|38.04|false|nick zipper|2013-03-01 09:11:58.703132|1.86|kindergarten +72|430|65763|4294967324|99.91|11.46|false|david hernandez|2013-03-01 09:11:58.703294|43.80|biology +107|271|65601|4294967456|8.57|1.30|false|xavier king|2013-03-01 09:11:58.703315|48.16|values clariffication +85|283|65746|4294967384|11.68|47.37|false|katie polk|2013-03-01 09:11:58.703311|93.21|xylophone band +49|388|65574|4294967407|27.63|17.03|false|bob xylophone|2013-03-01 09:11:58.703307|31.28|joggying +1|472|65775|4294967340|74.52|15.51|false|victor hernandez|2013-03-01 09:11:58.703077|66.68|values clariffication +89|278|65570|4294967466|55.63|24.53|true|ethan underhill|2013-03-01 09:11:58.703218|18.20|values clariffication +28|294|65756|4294967487|76.74|24.59|true|tom ellison|2013-03-01 09:11:58.703147|70.00|debate +74|504|65766|4294967520|89.93|41.54|false|alice zipper|2013-03-01 09:11:58.703134|98.90|history +42|342|65628|4294967436|9.48|24.35|false|victor davidson|2013-03-01 09:11:58.703236|37.80|religion +82|324|65789|4294967492|14.92|45.29|false|tom johnson|2013-03-01 09:11:58.703179|20.44|study skills +24|434|65788|4294967389|76.05|15.09|false|ulysses thompson|2013-03-01 09:11:58.703117|21.81|topology +33|285|65689|4294967355|97.83|41.08|true|nick steinbeck|2013-03-01 09:11:58.703178|9.12|biology +51|334|65670|4294967369|70.24|29.11|false|priscilla brown|2013-03-01 09:11:58.703269|92.97|geology +90|509|65602|4294967415|7.96|36.62|false|holly white|2013-03-01 09:11:58.703195|86.23|values clariffication +83|279|65754|4294967435|5.40|33.52|false|bob garcia|2013-03-01 09:11:58.703311|54.35|forestry +124|495|65549|4294967471|99.21|44.80|true|jessica davidson|2013-03-01 09:11:58.703130|64.91|history +120|445|65670|4294967344|98.23|47.82|false|tom ellison|2013-03-01 09:11:58.703085|5.37|education +14|273|65779|4294967474|92.05|33.36|false|calvin steinbeck|2013-03-01 09:11:58.703287|45.61|linguistics +15|351|65601|4294967459|5.67|30.66|true|victor white|2013-03-01 09:11:58.703162|37.93|kindergarten +63|422|65690|4294967550|8.42|28.42|true|oscar laertes|2013-03-01 09:11:58.703144|3.43|values clariffication +2|477|65571|4294967463|79.21|18.20|true|mike garcia|2013-03-01 09:11:58.703195|82.50|mathematics +46|386|65790|4294967507|52.73|9.27|true|priscilla ovid|2013-03-01 09:11:58.703274|84.10|undecided +28|427|65559|4294967370|39.03|42.02|true|alice carson|2013-03-01 09:11:58.703226|56.06|chemistry +101|282|65738|4294967432|93.61|4.53|true|calvin falkner|2013-03-01 09:11:58.703198|95.10|xylophone band +46|410|65721|4294967362|80.30|17.03|false|bob ellison|2013-03-01 09:11:58.703319|95.12|values clariffication +103|430|65642|4294967347|84.23|14.83|true|mike king|2013-03-01 09:11:58.703255|95.61|opthamology +51|459|65641|4294967380|5.45|19.47|true|ulysses underhill|2013-03-01 09:11:58.703289|84.27|geology +70|342|65569|4294967492|64.30|43.95|true|luke ovid|2013-03-01 09:11:58.703281|52.50|values clariffication +48|329|65788|4294967378|30.25|22.19|true|holly hernandez|2013-03-01 09:11:58.703157|16.10|quiet hour +29|374|65696|4294967481|13.87|19.93|false|xavier polk|2013-03-01 09:11:58.703258|40.30|geology +30|294|65779|4294967482|36.58|18.35|false|jessica white|2013-03-01 09:11:58.703085|39.48|kindergarten +74|289|65733|4294967495|77.84|28.96|false|jessica miller|2013-03-01 09:11:58.703177|20.72|linguistics +114|499|65724|4294967334|42.51|46.09|true|nick brown|2013-03-01 09:11:58.703176|71.66|nap time +53|371|65603|4294967303|51.29|18.41|true|nick van buren|2013-03-01 09:11:58.703186|41.44|history +121|401|65779|4294967375|29.24|25.08|true|david quirinius|2013-03-01 09:11:58.703295|65.24|religion +72|408|65581|4294967453|7.82|1.50|false|mike young|2013-03-01 09:11:58.703242|45.48|geology +116|482|65550|4294967489|5.85|31.75|true|mike steinbeck|2013-03-01 09:11:58.703150|33.01|religion +94|335|65603|4294967357|68.25|13.72|true|tom miller|2013-03-01 09:11:58.703192|80.24|undecided +4|460|65625|4294967360|5.51|22.60|true|oscar laertes|2013-03-01 09:11:58.703293|42.86|nap time +110|306|65571|4294967419|43.01|31.74|true|gabriella garcia|2013-03-01 09:11:58.703280|40.68|chemistry +0|330|65673|4294967343|91.97|15.22|false|victor brown|2013-03-01 09:11:58.703097|0.90|xylophone band +31|459|65595|4294967413|40.00|17.51|false|alice nixon|2013-03-01 09:11:58.703159|72.70|topology +45|423|65673|4294967447|11.55|3.83|true|jessica white|2013-03-01 09:11:58.703140|43.81|chemistry +55|427|65722|4294967331|10.09|45.28|true|ethan brown|2013-03-01 09:11:58.703084|1.76|joggying +9|321|65658|4294967501|54.83|20.55|false|mike davidson|2013-03-01 09:11:58.703179|60.94|debate +88|277|65585|4294967398|3.97|5.58|true|nick johnson|2013-03-01 09:11:58.703111|37.34|chemistry +110|279|65604|4294967344|92.33|5.31|true|quinn garcia|2013-03-01 09:11:58.703102|63.65|linguistics +9|456|65541|4294967531|38.57|9.56|true|xavier hernandez|2013-03-01 09:11:58.703174|36.33|linguistics +73|340|65584|4294967330|21.28|15.97|false|zach miller|2013-03-01 09:11:58.703265|12.38|philosophy +76|368|65608|4294967506|16.69|37.94|true|luke laertes|2013-03-01 09:11:58.703075|87.90|history +113|445|65658|4294967530|4.32|46.28|true|priscilla young|2013-03-01 09:11:58.703192|2.63|quiet hour +61|491|65711|4294967382|11.19|42.37|false|jessica young|2013-03-01 09:11:58.703261|8.66|wind surfing +70|361|65746|4294967332|97.57|22.61|false|katie young|2013-03-01 09:11:58.703182|98.10|undecided +38|336|65556|4294967482|14.84|12.30|true|gabriella ovid|2013-03-01 09:11:58.703086|9.96|xylophone band +100|413|65547|4294967303|53.18|20.42|true|luke allen|2013-03-01 09:11:58.703319|83.21|debate +117|474|65729|4294967333|68.22|18.36|false|priscilla underhill|2013-03-01 09:11:58.703249|19.42|education +118|467|65632|4294967296|41.36|29.36|false|tom hernandez|2013-03-01 09:11:58.703216|88.51|philosophy +50|367|65663|4294967405|11.18|33.00|true|jessica ellison|2013-03-01 09:11:58.703255|5.42|opthamology +19|276|65727|4294967539|68.81|0.37|false|holly van buren|2013-03-01 09:11:58.703084|19.81|biology +48|328|65594|4294967366|59.07|11.35|true|tom miller|2013-03-01 09:11:58.703162|67.22|nap time +5|435|65731|4294967493|73.88|30.12|false|sarah johnson|2013-03-01 09:11:58.703198|97.18|nap time +38|384|65667|4294967539|81.32|39.41|false|sarah ichabod|2013-03-01 09:11:58.703184|13.96|industrial engineering +9|316|65786|4294967546|54.47|27.40|false|ulysses hernandez|2013-03-01 09:11:58.703206|80.94|xylophone band +84|496|65604|4294967329|59.45|23.18|true|rachel king|2013-03-01 09:11:58.703077|63.36|quiet hour +34|264|65564|4294967437|31.77|12.14|false|calvin davidson|2013-03-01 09:11:58.703180|30.78|study skills +79|377|65570|4294967410|22.12|0.98|true|victor miller|2013-03-01 09:11:58.703088|45.23|joggying +7|342|65786|4294967449|72.53|21.30|false|ulysses quirinius|2013-03-01 09:11:58.703157|8.99|linguistics +85|349|65573|4294967308|97.46|12.13|false|mike steinbeck|2013-03-01 09:11:58.703179|96.32|undecided +27|287|65708|4294967542|83.33|36.73|false|quinn allen|2013-03-01 09:11:58.703157|1.29|mathematics +78|421|65633|4294967410|87.67|47.98|false|irene brown|2013-03-01 09:11:58.703207|81.02|zync studies +75|499|65696|4294967405|53.59|7.71|false|zach thompson|2013-03-01 09:11:58.703234|87.89|kindergarten +107|421|65595|4294967543|57.11|46.59|true|ethan xylophone|2013-03-01 09:11:58.703085|67.56|linguistics +124|310|65548|4294967319|58.66|15.94|false|victor thompson|2013-03-01 09:11:58.703168|52.94|forestry +5|501|65585|4294967363|33.58|19.98|true|alice xylophone|2013-03-01 09:11:58.703242|31.23|geology +65|476|65542|4294967547|41.29|23.25|true|quinn laertes|2013-03-01 09:11:58.703240|73.93|forestry +70|443|65700|4294967448|52.44|21.86|true|quinn brown|2013-03-01 09:11:58.703257|44.88|xylophone band +28|378|65678|4294967355|68.32|1.45|false|sarah xylophone|2013-03-01 09:11:58.703155|81.87|values clariffication +3|322|65672|4294967508|25.55|26.28|true|jessica carson|2013-03-01 09:11:58.703120|52.60|education +82|257|65654|4294967497|9.80|20.44|true|priscilla thompson|2013-03-01 09:11:58.703308|29.85|geology +115|276|65632|4294967457|90.69|40.89|true|tom robinson|2013-03-01 09:11:58.703157|5.09|study skills +33|459|65783|4294967300|10.29|14.51|true|fred falkner|2013-03-01 09:11:58.703270|51.08|philosophy +102|359|65560|4294967482|93.03|25.17|false|fred nixon|2013-03-01 09:11:58.703120|64.38|education +40|500|65664|4294967428|16.25|7.80|true|luke ellison|2013-03-01 09:11:58.703248|27.60|topology +32|467|65547|4294967502|67.94|21.95|true|ethan robinson|2013-03-01 09:11:58.703188|62.31|values clariffication +44|475|65776|4294967510|89.55|7.92|true|luke allen|2013-03-01 09:11:58.703105|28.47|study skills +60|381|65627|4294967365|33.85|41.36|true|quinn robinson|2013-03-01 09:11:58.703258|19.65|linguistics +103|430|65591|4294967344|75.03|22.27|false|bob ellison|2013-03-01 09:11:58.703277|9.85|values clariffication +35|423|65667|4294967507|65.43|8.76|true|zach allen|2013-03-01 09:11:58.703172|3.44|wind surfing +90|406|65763|4294967450|0.15|13.95|true|priscilla xylophone|2013-03-01 09:11:58.703245|0.72|nap time +25|403|65782|4294967516|0.61|3.40|false|wendy allen|2013-03-01 09:11:58.703152|16.51|quiet hour +104|415|65758|4294967454|66.17|44.12|false|tom robinson|2013-03-01 09:11:58.703161|25.62|kindergarten +81|272|65548|4294967413|40.04|21.30|false|tom white|2013-03-01 09:11:58.703231|77.24|study skills +37|503|65562|4294967358|44.22|44.32|false|rachel laertes|2013-03-01 09:11:58.703126|40.62|linguistics +40|432|65691|4294967547|24.86|45.46|false|ulysses laertes|2013-03-01 09:11:58.703171|58.01|topology +42|345|65676|4294967447|14.92|26.67|false|yuri thompson|2013-03-01 09:11:58.703250|80.13|history +44|465|65744|4294967391|27.30|47.71|false|xavier johnson|2013-03-01 09:11:58.703295|8.58|philosophy +14|413|65623|4294967434|87.61|20.93|false|gabriella falkner|2013-03-01 09:11:58.703318|30.83|religion +48|498|65774|4294967431|59.71|30.22|true|gabriella young|2013-03-01 09:11:58.703098|94.77|zync studies +105|326|65673|4294967440|4.59|25.92|true|victor robinson|2013-03-01 09:11:58.703290|64.23|forestry +61|407|65750|4294967412|0.10|40.59|true|zach ovid|2013-03-01 09:11:58.703137|34.29|joggying +69|473|65599|4294967325|76.72|28.69|true|zach robinson|2013-03-01 09:11:58.703305|71.51|religion +26|283|65564|4294967400|46.86|12.35|true|bob ovid|2013-03-01 09:11:58.703322|9.47|forestry +50|316|65654|4294967488|9.81|27.36|true|calvin nixon|2013-03-01 09:11:58.703133|53.55|topology +112|316|65719|4294967473|51.79|43.04|false|luke brown|2013-03-01 09:11:58.703116|82.10|history +32|264|65606|4294967487|66.17|43.92|true|holly young|2013-03-01 09:11:58.703239|24.80|zync studies +31|493|65562|4294967428|38.94|24.25|true|alice van buren|2013-03-01 09:11:58.703143|26.86|geology +32|490|65658|4294967541|62.23|30.16|false|rachel johnson|2013-03-01 09:11:58.703155|92.67|wind surfing +113|302|65721|4294967526|96.68|32.74|true|holly underhill|2013-03-01 09:11:58.703306|39.19|values clariffication +44|357|65542|4294967373|31.91|4.81|true|oscar thompson|2013-03-01 09:11:58.703182|73.72|debate +0|268|65740|4294967449|32.89|28.52|false|oscar zipper|2013-03-01 09:11:58.703113|58.65|industrial engineering +121|355|65537|4294967437|85.90|10.99|true|calvin brown|2013-03-01 09:11:58.703254|4.49|yard duty +45|459|65565|4294967507|6.55|34.52|true|gabriella davidson|2013-03-01 09:11:58.703117|81.72|kindergarten +1|428|65564|4294967532|69.32|2.43|false|holly robinson|2013-03-01 09:11:58.703324|98.36|yard duty +31|351|65669|4294967441|95.38|43.95|true|calvin zipper|2013-03-01 09:11:58.703308|16.95|xylophone band +121|330|65755|4294967380|64.36|2.53|false|holly johnson|2013-03-01 09:11:58.703210|90.08|joggying +53|464|65720|4294967413|26.39|14.52|true|nick garcia|2013-03-01 09:11:58.703315|62.04|opthamology +106|318|65789|4294967381|1.02|33.60|false|fred van buren|2013-03-01 09:11:58.703156|96.90|joggying +31|281|65681|4294967368|86.00|14.29|false|yuri falkner|2013-03-01 09:11:58.703310|55.44|xylophone band +71|356|65715|4294967364|26.47|34.14|true|wendy robinson|2013-03-01 09:11:58.703178|9.18|zync studies +97|283|65589|4294967434|37.80|31.64|true|fred white|2013-03-01 09:11:58.703188|74.83|mathematics +43|418|65646|4294967385|43.92|19.31|true|priscilla king|2013-03-01 09:11:58.703244|4.47|zync studies +46|485|65739|4294967519|9.19|1.79|true|calvin zipper|2013-03-01 09:11:58.703089|22.23|nap time +45|394|65586|4294967391|63.33|44.63|true|wendy king|2013-03-01 09:11:58.703124|56.73|education +51|362|65677|4294967445|63.42|34.20|true|ethan white|2013-03-01 09:11:58.703092|48.79|chemistry +91|384|65644|4294967533|10.09|4.95|true|victor xylophone|2013-03-01 09:11:58.703320|87.31|opthamology +77|485|65651|4294967326|50.70|49.72|false|victor quirinius|2013-03-01 09:11:58.703256|33.24|opthamology +80|333|65614|4294967331|13.10|39.55|true|oscar brown|2013-03-01 09:11:58.703154|75.45|industrial engineering +31|474|65682|4294967551|30.63|2.53|false|yuri allen|2013-03-01 09:11:58.703212|35.36|forestry +71|391|65749|4294967412|41.20|20.94|true|calvin nixon|2013-03-01 09:11:58.703311|15.14|wind surfing +0|372|65720|4294967416|79.48|20.76|true|oscar quirinius|2013-03-01 09:11:58.703254|0.32|education +41|453|65677|4294967371|76.10|28.02|false|victor xylophone|2013-03-01 09:11:58.703179|69.86|religion +111|317|65703|4294967297|38.05|0.48|true|fred nixon|2013-03-01 09:11:58.703256|90.89|values clariffication +18|468|65551|4294967546|0.98|45.14|false|irene polk|2013-03-01 09:11:58.703186|39.43|wind surfing +0|486|65706|4294967466|20.82|30.55|false|mike allen|2013-03-01 09:11:58.703246|72.87|mathematics +92|329|65610|4294967521|47.08|36.57|false|irene polk|2013-03-01 09:11:58.703231|11.99|debate +35|409|65777|4294967422|93.09|11.50|false|bob brown|2013-03-01 09:11:58.703269|73.79|religion +80|455|65644|4294967460|75.35|20.56|false|nick xylophone|2013-03-01 09:11:58.703249|30.25|quiet hour +17|435|65684|4294967326|31.61|8.33|false|victor allen|2013-03-01 09:11:58.703264|77.18|mathematics +91|295|65587|4294967506|76.93|9.88|true|xavier polk|2013-03-01 09:11:58.703243|53.84|linguistics +92|363|65697|4294967310|13.89|47.40|true|katie robinson|2013-03-01 09:11:58.703237|3.18|xylophone band +17|351|65589|4294967439|54.99|44.75|false|irene van buren|2013-03-01 09:11:58.703196|49.49|american history +88|445|65778|4294967422|33.67|28.86|false|bob van buren|2013-03-01 09:11:58.703230|62.17|forestry +44|386|65692|4294967434|0.84|20.94|true|wendy polk|2013-03-01 09:11:58.703226|87.05|values clariffication +84|373|65585|4294967486|14.13|30.21|true|katie xylophone|2013-03-01 09:11:58.703084|50.38|forestry +83|395|65645|4294967509|99.62|9.50|true|irene ichabod|2013-03-01 09:11:58.703212|63.22|zync studies +28|428|65650|4294967320|85.76|22.68|false|wendy thompson|2013-03-01 09:11:58.703145|85.89|wind surfing +27|503|65628|4294967371|95.07|25.58|true|ethan laertes|2013-03-01 09:11:58.703310|14.83|topology +69|304|65695|4294967384|13.94|10.51|true|nick garcia|2013-03-01 09:11:58.703167|59.96|mathematics +15|264|65737|4294967412|9.04|32.04|false|alice ovid|2013-03-01 09:11:58.703279|20.85|kindergarten +118|399|65730|4294967309|0.52|33.72|false|luke laertes|2013-03-01 09:11:58.703098|1.35|quiet hour +84|437|65601|4294967357|49.44|8.79|false|nick davidson|2013-03-01 09:11:58.703161|26.54|undecided +1|313|65543|4294967362|45.35|47.49|true|bob white|2013-03-01 09:11:58.703144|21.59|linguistics +40|442|65659|4294967379|50.08|31.50|false|irene ellison|2013-03-01 09:11:58.703306|53.80|philosophy +60|308|65564|4294967405|36.79|6.46|true|quinn ichabod|2013-03-01 09:11:58.703298|26.06|geology +118|282|65644|4294967407|67.59|32.89|true|victor laertes|2013-03-01 09:11:58.703177|89.95|study skills +77|320|65585|4294967344|45.59|39.11|true|bob garcia|2013-03-01 09:11:58.703301|39.32|american history +17|407|65716|4294967460|14.44|16.87|false|luke johnson|2013-03-01 09:11:58.703200|71.53|linguistics +84|492|65737|4294967467|23.45|46.69|false|nick ichabod|2013-03-01 09:11:58.703127|3.98|values clariffication +97|266|65544|4294967373|32.20|9.87|true|zach garcia|2013-03-01 09:11:58.703289|1.67|quiet hour +21|348|65765|4294967366|56.62|2.75|false|nick zipper|2013-03-01 09:11:58.703269|69.76|zync studies +88|404|65548|4294967382|6.87|15.00|false|oscar carson|2013-03-01 09:11:58.703318|57.61|wind surfing +119|427|65553|4294967319|95.53|12.82|true|luke underhill|2013-03-01 09:11:58.703130|93.95|wind surfing +16|405|65591|4294967351|97.24|33.01|false|ethan quirinius|2013-03-01 09:11:58.703156|60.78|xylophone band +65|460|65570|4294967399|74.30|19.59|false|jessica white|2013-03-01 09:11:58.703111|25.57|study skills +11|300|65583|4294967382|88.22|30.13|false|tom falkner|2013-03-01 09:11:58.703275|93.49|joggying +106|446|65613|4294967369|48.45|7.18|true|alice garcia|2013-03-01 09:11:58.703193|43.72|chemistry +114|410|65578|4294967363|94.34|15.10|true|zach ovid|2013-03-01 09:11:58.703299|55.97|history +-1|372|65680|4294967490|15.45|18.09|false|ethan laertes|2013-03-01 09:11:58.703110|65.88|opthamology +113|361|65660|4294967516|23.17|38.39|true|alice king|2013-03-01 09:11:58.703246|62.82|geology +98|381|65563|4294967458|41.29|48.85|false|sarah garcia|2013-03-01 09:11:58.703243|33.29|mathematics +108|385|65730|4294967387|65.44|43.34|false|irene miller|2013-03-01 09:11:58.703157|81.44|philosophy +10|441|65723|4294967528|96.23|34.98|false|xavier brown|2013-03-01 09:11:58.703281|19.95|nap time +84|403|65656|4294967550|7.05|33.66|true|luke davidson|2013-03-01 09:11:58.703300|14.00|kindergarten +84|447|65755|4294967387|4.92|22.75|false|rachel davidson|2013-03-01 09:11:58.703178|96.80|biology +25|309|65568|4294967354|58.75|23.31|true|katie zipper|2013-03-01 09:11:58.703156|21.79|quiet hour +17|388|65784|4294967542|40.24|3.98|true|katie polk|2013-03-01 09:11:58.703163|15.70|kindergarten +5|486|65602|4294967358|71.55|3.40|true|ulysses carson|2013-03-01 09:11:58.703266|83.67|mathematics +59|494|65786|4294967483|35.80|11.44|true|zach garcia|2013-03-01 09:11:58.703125|56.82|yard duty +80|509|65553|4294967544|50.28|47.00|false|calvin white|2013-03-01 09:11:58.703071|71.07|debate +109|376|65538|4294967453|59.61|35.62|true|priscilla xylophone|2013-03-01 09:11:58.703286|9.53|study skills +40|496|65640|4294967513|93.73|25.71|true|calvin thompson|2013-03-01 09:11:58.703124|8.93|mathematics +65|508|65732|4294967409|81.47|29.25|false|ethan ellison|2013-03-01 09:11:58.703154|30.06|religion +50|478|65599|4294967338|77.81|32.31|true|holly brown|2013-03-01 09:11:58.703128|76.29|kindergarten +26|451|65716|4294967312|77.42|13.60|true|ulysses carson|2013-03-01 09:11:58.703116|38.93|history +31|382|65737|4294967301|58.09|41.34|true|wendy thompson|2013-03-01 09:11:58.703231|80.97|biology +33|430|65694|4294967346|48.37|40.46|false|fred king|2013-03-01 09:11:58.703167|88.50|forestry +6|408|65658|4294967503|99.68|44.11|false|mike polk|2013-03-01 09:11:58.703313|20.69|industrial engineering +100|264|65637|4294967340|85.00|45.86|false|fred falkner|2013-03-01 09:11:58.703148|44.32|joggying +119|316|65646|4294967405|46.27|16.82|false|gabriella allen|2013-03-01 09:11:58.703224|17.92|wind surfing +105|478|65790|4294967302|88.05|5.82|true|gabriella polk|2013-03-01 09:11:58.703236|84.87|topology +124|384|65697|4294967426|15.30|19.15|true|fred quirinius|2013-03-01 09:11:58.703139|86.79|education +109|319|65734|4294967387|52.23|20.82|false|alice king|2013-03-01 09:11:58.703262|31.71|american history +93|356|65650|4294967428|53.94|27.46|true|irene brown|2013-03-01 09:11:58.703130|59.55|religion +51|444|65662|4294967539|42.48|14.39|false|alice zipper|2013-03-01 09:11:58.703202|76.51|geology +61|432|65778|4294967469|9.57|29.88|false|mike white|2013-03-01 09:11:58.703074|64.69|wind surfing +78|419|65633|4294967299|34.35|32.06|true|bob zipper|2013-03-01 09:11:58.703081|72.54|history +67|295|65654|4294967449|59.55|1.37|true|ulysses white|2013-03-01 09:11:58.703096|11.64|topology +70|312|65742|4294967540|54.43|19.85|false|tom polk|2013-03-01 09:11:58.703206|6.61|values clariffication +56|351|65732|4294967479|8.37|34.68|true|xavier white|2013-03-01 09:11:58.703103|95.68|quiet hour +42|495|65536|4294967431|43.57|46.81|false|tom johnson|2013-03-01 09:11:58.703245|62.25|american history +0|428|65590|4294967473|94.54|28.29|true|irene carson|2013-03-01 09:11:58.703089|92.71|education +91|262|65661|4294967391|73.65|13.30|false|sarah garcia|2013-03-01 09:11:58.703325|26.80|topology +73|337|65775|4294967406|57.12|24.35|true|oscar xylophone|2013-03-01 09:11:58.703091|66.56|topology +58|404|65693|4294967345|78.89|6.57|true|sarah carson|2013-03-01 09:11:58.703218|84.12|undecided +77|461|65581|4294967449|89.55|49.44|true|jessica thompson|2013-03-01 09:11:58.703216|60.88|quiet hour +89|492|65736|4294967336|14.93|26.74|false|ulysses young|2013-03-01 09:11:58.703089|69.96|undecided +116|444|65659|4294967329|59.62|37.37|true|victor hernandez|2013-03-01 09:11:58.703093|94.69|kindergarten +114|427|65712|4294967419|32.47|10.24|true|jessica quirinius|2013-03-01 09:11:58.703116|49.69|values clariffication +43|503|65650|4294967482|96.38|6.96|true|nick nixon|2013-03-01 09:11:58.703302|23.55|american history +116|351|65771|4294967470|48.59|24.17|false|fred ellison|2013-03-01 09:11:58.703282|7.50|values clariffication +-2|261|65603|4294967332|39.18|28.98|false|fred polk|2013-03-01 09:11:58.703310|69.26|opthamology +123|439|65781|4294967448|15.86|42.96|true|ulysses xylophone|2013-03-01 09:11:58.703272|12.45|geology +83|451|65583|4294967443|10.13|44.36|true|nick falkner|2013-03-01 09:11:58.703293|57.67|values clariffication +51|390|65709|4294967548|29.59|49.59|true|oscar steinbeck|2013-03-01 09:11:58.703243|49.79|education +107|313|65699|4294967493|9.25|41.34|false|gabriella young|2013-03-01 09:11:58.703213|38.71|mathematics +93|403|65782|4294967355|74.53|26.21|true|oscar robinson|2013-03-01 09:11:58.703239|76.14|topology +7|320|65624|4294967530|41.34|11.54|true|bob ellison|2013-03-01 09:11:58.703249|93.30|nap time +38|308|65559|4294967465|95.81|13.71|true|david davidson|2013-03-01 09:11:58.703114|73.94|forestry +112|295|65662|4294967517|65.62|34.97|true|holly johnson|2013-03-01 09:11:58.703274|59.92|zync studies +101|283|65538|4294967527|84.03|27.29|true|yuri brown|2013-03-01 09:11:58.703306|44.24|biology +20|509|65686|4294967330|12.46|40.63|false|victor steinbeck|2013-03-01 09:11:58.703323|38.27|linguistics +0|367|65685|4294967491|91.61|13.80|false|priscilla van buren|2013-03-01 09:11:58.703096|87.84|topology +18|371|65552|4294967424|37.24|3.71|false|fred davidson|2013-03-01 09:11:58.703308|48.96|undecided +92|344|65618|4294967390|46.09|28.69|true|victor steinbeck|2013-03-01 09:11:58.703193|81.99|biology +34|322|65659|4294967353|78.62|21.42|false|ethan robinson|2013-03-01 09:11:58.703153|46.57|religion +27|401|65779|4294967402|97.39|43.42|false|mike zipper|2013-03-01 09:11:58.703145|64.41|quiet hour +74|290|65630|4294967436|57.89|8.33|false|oscar ellison|2013-03-01 09:11:58.703296|99.05|biology +81|344|65677|4294967530|28.11|20.56|false|calvin brown|2013-03-01 09:11:58.703159|87.67|quiet hour +109|275|65651|4294967393|96.94|17.44|false|luke underhill|2013-03-01 09:11:58.703228|58.25|opthamology +97|290|65774|4294967311|71.07|11.44|false|wendy robinson|2013-03-01 09:11:58.703240|5.10|undecided +41|338|65618|4294967304|45.68|26.17|true|xavier allen|2013-03-01 09:11:58.703138|55.68|biology +88|496|65767|4294967448|14.30|41.69|true|wendy quirinius|2013-03-01 09:11:58.703106|94.32|kindergarten +51|495|65554|4294967347|94.27|26.65|true|irene robinson|2013-03-01 09:11:58.703179|63.61|quiet hour +97|491|65618|4294967423|32.25|2.13|false|luke falkner|2013-03-01 09:11:58.703166|16.60|study skills +9|311|65719|4294967442|2.60|26.73|true|zach miller|2013-03-01 09:11:58.703223|64.21|undecided +62|262|65668|4294967468|61.94|7.31|false|priscilla johnson|2013-03-01 09:11:58.703228|48.68|topology +63|341|65704|4294967546|7.79|2.63|false|xavier brown|2013-03-01 09:11:58.703261|69.36|debate +69|492|65751|4294967324|45.10|20.65|true|sarah johnson|2013-03-01 09:11:58.703095|9.77|chemistry +17|404|65712|4294967388|71.13|45.81|false|gabriella ichabod|2013-03-01 09:11:58.703164|30.27|undecided +72|310|65780|4294967507|80.99|24.03|true|tom davidson|2013-03-01 09:11:58.703212|38.58|forestry +18|397|65611|4294967439|98.22|15.32|true|xavier allen|2013-03-01 09:11:58.703089|69.46|undecided +50|307|65596|4294967300|80.60|30.89|true|calvin falkner|2013-03-01 09:11:58.703275|54.75|values clariffication +84|305|65637|4294967402|10.16|43.04|true|yuri laertes|2013-03-01 09:11:58.703283|58.05|linguistics +75|479|65749|4294967528|80.52|5.31|false|priscilla brown|2013-03-01 09:11:58.703148|65.49|undecided +97|323|65704|4294967477|60.12|42.22|true|mike nixon|2013-03-01 09:11:58.703249|48.48|wind surfing +14|437|65612|4294967539|64.25|4.82|false|zach ichabod|2013-03-01 09:11:58.703215|51.14|forestry +22|280|65774|4294967311|77.10|40.24|false|jessica nixon|2013-03-01 09:11:58.703238|56.84|xylophone band +118|497|65536|4294967381|50.32|12.72|false|david nixon|2013-03-01 09:11:58.703285|83.48|values clariffication +27|482|65624|4294967313|78.98|43.96|true|oscar carson|2013-03-01 09:11:58.703217|70.08|joggying +99|436|65746|4294967342|39.81|15.28|false|calvin young|2013-03-01 09:11:58.703145|97.52|quiet hour +50|409|65738|4294967535|28.56|0.74|true|mike ellison|2013-03-01 09:11:58.703282|30.94|values clariffication +66|267|65738|4294967371|70.89|11.40|true|oscar thompson|2013-03-01 09:11:58.703104|51.33|undecided +107|466|65691|4294967302|9.68|40.39|false|nick ellison|2013-03-01 09:11:58.703108|28.47|american history +55|335|65699|4294967487|5.28|16.66|true|david ichabod|2013-03-01 09:11:58.703161|68.94|nap time +-2|340|65627|4294967533|76.52|21.91|true|quinn laertes|2013-03-01 09:11:58.703178|93.29|xylophone band +68|448|65597|4294967422|70.39|40.01|true|ethan laertes|2013-03-01 09:11:58.703109|69.05|quiet hour +65|505|65679|4294967338|75.88|15.87|true|yuri steinbeck|2013-03-01 09:11:58.703085|65.46|yard duty +53|315|65607|4294967496|1.58|20.02|true|victor johnson|2013-03-01 09:11:58.703202|36.58|undecided +94|475|65606|4294967503|62.14|26.00|false|mike ellison|2013-03-01 09:11:58.703115|67.94|biology +-3|275|65622|4294967302|71.78|8.49|false|wendy robinson|2013-03-01 09:11:58.703294|95.39|undecided +-1|281|65643|4294967323|15.10|45.00|false|irene nixon|2013-03-01 09:11:58.703223|80.96|undecided +72|469|65765|4294967500|60.22|32.25|false|holly young|2013-03-01 09:11:58.703253|88.56|wind surfing +23|401|65693|4294967439|34.41|46.73|false|rachel brown|2013-03-01 09:11:58.703306|71.72|geology +36|377|65697|4294967444|98.51|49.05|false|oscar carson|2013-03-01 09:11:58.703081|67.90|joggying +18|264|65595|4294967512|18.86|42.56|false|fred davidson|2013-03-01 09:11:58.703137|89.60|history +82|257|65655|4294967320|40.42|41.02|true|luke quirinius|2013-03-01 09:11:58.703205|35.80|zync studies +120|497|65595|4294967442|45.56|21.53|true|sarah young|2013-03-01 09:11:58.703167|90.24|quiet hour +94|256|65747|4294967347|32.18|8.61|true|quinn quirinius|2013-03-01 09:11:58.703217|71.87|history +112|453|65586|4294967540|42.76|3.66|true|gabriella carson|2013-03-01 09:11:58.703291|1.15|wind surfing +73|394|65657|4294967486|11.82|39.80|false|luke laertes|2013-03-01 09:11:58.703160|11.93|nap time +-3|344|65733|4294967363|0.56|11.96|true|rachel thompson|2013-03-01 09:11:58.703276|88.46|wind surfing +0|356|65581|4294967326|40.44|33.18|false|calvin robinson|2013-03-01 09:11:58.703294|29.74|zync studies +101|487|65676|4294967367|85.87|13.14|true|zach zipper|2013-03-01 09:11:58.703077|27.58|zync studies +48|491|65669|4294967492|63.51|49.85|true|tom van buren|2013-03-01 09:11:58.703247|1.29|forestry +93|504|65557|4294967500|22.68|37.23|true|bob hernandez|2013-03-01 09:11:58.703207|21.14|zync studies +35|290|65726|4294967434|18.63|7.24|true|priscilla zipper|2013-03-01 09:11:58.703195|7.42|education +114|503|65677|4294967534|64.22|15.16|true|gabriella allen|2013-03-01 09:11:58.703324|44.47|wind surfing +-3|458|65696|4294967418|45.24|8.49|false|irene ellison|2013-03-01 09:11:58.703092|54.02|american history +103|411|65579|4294967521|22.25|18.31|true|quinn zipper|2013-03-01 09:11:58.703175|33.37|forestry +100|492|65778|4294967379|13.35|23.88|false|luke garcia|2013-03-01 09:11:58.703260|59.03|undecided +119|421|65580|4294967480|27.87|46.74|true|wendy brown|2013-03-01 09:11:58.703085|59.43|study skills +20|385|65562|4294967324|71.80|5.51|false|oscar ichabod|2013-03-01 09:11:58.703194|72.04|mathematics +81|331|65745|4294967531|80.71|42.55|true|ethan laertes|2013-03-01 09:11:58.703226|66.36|yard duty +78|509|65758|4294967531|21.45|23.59|false|alice allen|2013-03-01 09:11:58.703171|56.56|geology +17|496|65624|4294967389|96.25|1.62|true|nick laertes|2013-03-01 09:11:58.703113|65.31|mathematics +43|379|65616|4294967348|46.15|12.02|true|rachel falkner|2013-03-01 09:11:58.703120|75.20|geology +43|281|65676|4294967540|87.14|21.33|true|jessica garcia|2013-03-01 09:11:58.703173|8.77|debate +22|304|65607|4294967318|82.72|2.34|false|priscilla van buren|2013-03-01 09:11:58.703185|75.86|forestry +59|431|65537|4294967326|11.34|2.90|true|oscar robinson|2013-03-01 09:11:58.703161|81.04|zync studies +6|322|65715|4294967385|82.55|8.62|false|david ichabod|2013-03-01 09:11:58.703248|81.28|topology +79|396|65686|4294967335|89.28|25.63|true|rachel polk|2013-03-01 09:11:58.703277|58.87|geology +95|482|65637|4294967342|9.70|2.16|true|bob steinbeck|2013-03-01 09:11:58.703225|35.51|undecided +71|280|65610|4294967451|33.36|25.95|true|rachel brown|2013-03-01 09:11:58.703097|7.59|opthamology +32|400|65753|4294967360|35.13|32.17|true|irene ovid|2013-03-01 09:11:58.703294|59.13|mathematics +70|373|65773|4294967502|46.18|47.06|false|zach king|2013-03-01 09:11:58.703132|87.66|mathematics +105|509|65716|4294967308|22.94|9.61|true|jessica quirinius|2013-03-01 09:11:58.703073|10.07|zync studies +20|286|65671|4294967307|28.45|25.01|true|oscar white|2013-03-01 09:11:58.703299|52.05|undecided +98|432|65669|4294967373|63.12|23.03|false|calvin allen|2013-03-01 09:11:58.703214|38.60|industrial engineering +57|304|65731|4294967315|94.68|44.22|false|xavier carson|2013-03-01 09:11:58.703208|18.20|study skills +45|433|65672|4294967506|85.03|0.37|false|tom nixon|2013-03-01 09:11:58.703154|64.09|mathematics +77|483|65784|4294967330|23.07|29.96|true|zach ovid|2013-03-01 09:11:58.703168|97.37|education +70|268|65766|4294967299|88.36|17.15|true|gabriella thompson|2013-03-01 09:11:58.703169|86.53|opthamology +34|398|65731|4294967546|66.61|46.93|true|calvin johnson|2013-03-01 09:11:58.703078|61.52|quiet hour +2|443|65552|4294967349|51.85|30.49|false|oscar white|2013-03-01 09:11:58.703125|78.52|philosophy +65|393|65625|4294967374|24.73|41.75|false|katie garcia|2013-03-01 09:11:58.703083|20.61|opthamology +83|432|65656|4294967303|26.08|39.73|true|jessica underhill|2013-03-01 09:11:58.703147|82.67|geology +32|506|65551|4294967375|19.13|3.29|true|ulysses ichabod|2013-03-01 09:11:58.703195|74.13|zync studies +13|324|65544|4294967539|84.31|37.34|true|tom young|2013-03-01 09:11:58.703271|65.85|education +11|267|65727|4294967325|33.55|17.89|true|jessica davidson|2013-03-01 09:11:58.703144|87.37|kindergarten +119|280|65615|4294967457|94.72|19.90|false|rachel white|2013-03-01 09:11:58.703163|54.47|biology +85|392|65760|4294967436|65.02|16.91|false|bob ellison|2013-03-01 09:11:58.703115|27.52|history +89|436|65623|4294967461|89.03|25.97|true|fred robinson|2013-03-01 09:11:58.703239|95.58|linguistics +51|265|65626|4294967521|94.38|1.53|true|luke thompson|2013-03-01 09:11:58.703120|75.15|yard duty +122|428|65715|4294967520|35.72|8.21|false|priscilla underhill|2013-03-01 09:11:58.703212|54.79|opthamology +114|507|65576|4294967517|15.18|34.48|true|luke allen|2013-03-01 09:11:58.703140|4.88|religion +22|257|65588|4294967316|81.17|28.77|false|nick quirinius|2013-03-01 09:11:58.703224|76.44|forestry +78|417|65552|4294967300|64.00|10.49|true|calvin van buren|2013-03-01 09:11:58.703223|3.43|debate +88|409|65607|4294967509|99.13|41.24|false|holly zipper|2013-03-01 09:11:58.703267|91.52|nap time +36|274|65705|4294967514|70.52|5.93|true|fred nixon|2013-03-01 09:11:58.703179|65.13|topology +42|338|65719|4294967488|87.99|39.29|false|nick ovid|2013-03-01 09:11:58.703302|66.85|education +18|446|65544|4294967435|93.11|14.90|true|holly xylophone|2013-03-01 09:11:58.703085|73.08|history +110|261|65600|4294967428|75.83|10.22|false|mike garcia|2013-03-01 09:11:58.703132|43.03|education +61|382|65623|4294967443|25.11|2.26|false|david ovid|2013-03-01 09:11:58.703125|39.74|education +57|277|65754|4294967357|36.26|32.98|true|gabriella zipper|2013-03-01 09:11:58.703077|10.16|topology +12|379|65698|4294967477|47.57|44.60|false|mike carson|2013-03-01 09:11:58.703291|42.98|religion +3|464|65617|4294967424|82.30|2.92|false|ethan brown|2013-03-01 09:11:58.703076|18.51|wind surfing +5|360|65543|4294967305|67.98|38.53|false|priscilla johnson|2013-03-01 09:11:58.703259|52.28|geology +37|445|65612|4294967444|92.11|0.74|true|wendy steinbeck|2013-03-01 09:11:58.703096|16.05|study skills +58|382|65644|4294967500|18.63|30.09|true|oscar allen|2013-03-01 09:11:58.703144|96.45|linguistics +115|327|65607|4294967391|28.79|0.80|false|yuri polk|2013-03-01 09:11:58.703153|47.12|study skills +93|373|65707|4294967343|95.48|11.83|false|oscar hernandez|2013-03-01 09:11:58.703245|76.20|undecided +108|478|65712|4294967522|46.43|33.12|false|nick garcia|2013-03-01 09:11:58.703166|34.70|kindergarten +-3|376|65548|4294967431|96.78|43.23|false|fred ellison|2013-03-01 09:11:58.703233|75.39|education +44|398|65691|4294967393|26.71|25.51|false|fred ellison|2013-03-01 09:11:58.703270|5.01|undecided +91|392|65708|4294967305|71.35|26.84|false|victor hernandez|2013-03-01 09:11:58.703198|70.89|philosophy +14|355|65556|4294967482|64.46|5.96|true|oscar davidson|2013-03-01 09:11:58.703239|95.12|linguistics +89|278|65593|4294967432|8.61|16.38|false|tom brown|2013-03-01 09:11:58.703266|84.60|quiet hour +11|425|65624|4294967305|16.08|41.83|true|tom carson|2013-03-01 09:11:58.703324|14.08|forestry +61|478|65575|4294967442|96.73|43.85|true|ulysses ellison|2013-03-01 09:11:58.703218|61.41|history +54|501|65722|4294967336|42.67|30.41|true|irene laertes|2013-03-01 09:11:58.703153|4.11|undecided +58|290|65596|4294967532|41.62|31.16|false|oscar nixon|2013-03-01 09:11:58.703123|94.35|chemistry +17|350|65571|4294967428|19.03|36.22|false|victor hernandez|2013-03-01 09:11:58.703295|71.57|debate +49|291|65548|4294967403|77.02|46.73|true|victor white|2013-03-01 09:11:58.703221|51.90|zync studies +17|495|65689|4294967373|41.45|42.96|true|oscar quirinius|2013-03-01 09:11:58.703197|27.51|chemistry +1|418|65670|4294967534|6.29|30.62|true|yuri carson|2013-03-01 09:11:58.703089|93.66|linguistics +49|345|65614|4294967318|50.02|2.89|false|ethan falkner|2013-03-01 09:11:58.703255|94.47|linguistics +123|419|65654|4294967442|97.87|40.84|true|luke ichabod|2013-03-01 09:11:58.703159|88.23|linguistics +66|359|65556|4294967532|86.93|23.63|true|zach king|2013-03-01 09:11:58.703181|99.94|study skills +50|290|65630|4294967304|10.20|21.01|false|oscar ellison|2013-03-01 09:11:58.703139|72.13|opthamology +118|334|65586|4294967525|38.79|31.63|true|mike king|2013-03-01 09:11:58.703307|87.57|philosophy +60|454|65706|4294967443|47.32|3.27|true|fred polk|2013-03-01 09:11:58.703278|9.74|joggying +25|291|65746|4294967407|60.26|42.76|false|wendy nixon|2013-03-01 09:11:58.703137|34.75|american history +108|391|65717|4294967336|43.71|41.73|false|rachel white|2013-03-01 09:11:58.703272|33.13|wind surfing +0|267|65608|4294967523|46.87|4.48|true|jessica quirinius|2013-03-01 09:11:58.703278|69.42|geology +46|360|65702|4294967541|46.21|20.42|false|jessica underhill|2013-03-01 09:11:58.703117|55.68|linguistics +88|404|65730|4294967335|17.74|49.67|false|katie van buren|2013-03-01 09:11:58.703198|30.60|biology +41|275|65694|4294967503|1.91|44.92|true|sarah carson|2013-03-01 09:11:58.703096|65.86|debate +93|426|65714|4294967478|83.40|20.19|false|quinn davidson|2013-03-01 09:11:58.703093|14.85|debate +78|448|65568|4294967513|74.02|10.41|true|quinn garcia|2013-03-01 09:11:58.703107|16.26|quiet hour +77|283|65711|4294967523|10.17|28.61|false|gabriella falkner|2013-03-01 09:11:58.703106|96.10|topology +45|377|65786|4294967364|92.37|38.14|false|alice steinbeck|2013-03-01 09:11:58.703299|29.32|wind surfing +117|341|65588|4294967496|51.29|28.71|true|jessica brown|2013-03-01 09:11:58.703262|98.33|philosophy +5|485|65635|4294967306|2.20|40.21|false|yuri nixon|2013-03-01 09:11:58.703256|43.80|history +96|315|65769|4294967323|94.68|10.21|true|mike king|2013-03-01 09:11:58.703109|38.98|linguistics +5|295|65621|4294967329|3.28|6.58|true|tom van buren|2013-03-01 09:11:58.703244|25.67|joggying +79|340|65744|4294967531|79.49|19.72|false|ulysses robinson|2013-03-01 09:11:58.703197|46.98|wind surfing +18|312|65648|4294967547|72.04|9.99|true|fred falkner|2013-03-01 09:11:58.703159|64.06|industrial engineering +17|382|65769|4294967351|57.64|38.92|false|fred laertes|2013-03-01 09:11:58.703242|91.49|biology +9|293|65545|4294967527|31.67|43.30|false|luke johnson|2013-03-01 09:11:58.703325|65.81|joggying +39|340|65650|4294967365|22.12|22.27|false|xavier quirinius|2013-03-01 09:11:58.703082|37.25|nap time +5|292|65598|4294967394|79.38|39.63|true|mike ellison|2013-03-01 09:11:58.703315|53.04|mathematics +71|298|65540|4294967510|91.63|9.60|false|gabriella zipper|2013-03-01 09:11:58.703140|68.14|geology +46|303|65562|4294967402|96.29|28.91|false|ethan laertes|2013-03-01 09:11:58.703147|54.94|debate +99|282|65659|4294967394|30.62|34.72|true|wendy garcia|2013-03-01 09:11:58.703283|94.30|mathematics +67|265|65562|4294967421|53.06|4.92|false|jessica xylophone|2013-03-01 09:11:58.703182|69.41|quiet hour +53|274|65658|4294967545|95.28|48.78|false|luke polk|2013-03-01 09:11:58.703095|77.66|topology +71|496|65643|4294967349|84.72|14.22|true|calvin ovid|2013-03-01 09:11:58.703252|50.79|chemistry +19|281|65542|4294967452|29.41|16.87|false|zach xylophone|2013-03-01 09:11:58.703318|78.01|quiet hour +44|330|65647|4294967339|39.34|33.30|true|katie king|2013-03-01 09:11:58.703224|58.03|education +109|283|65675|4294967429|88.91|28.77|true|ulysses white|2013-03-01 09:11:58.703244|35.33|yard duty +59|373|65589|4294967359|9.35|22.19|true|luke falkner|2013-03-01 09:11:58.703112|72.30|xylophone band +80|407|65588|4294967371|51.25|38.62|false|david allen|2013-03-01 09:11:58.703215|64.63|kindergarten +31|457|65656|4294967394|29.36|31.68|true|ulysses ovid|2013-03-01 09:11:58.703192|64.44|nap time +33|319|65617|4294967431|64.65|15.09|true|tom laertes|2013-03-01 09:11:58.703269|47.45|kindergarten +41|334|65776|4294967432|71.89|47.91|false|ulysses johnson|2013-03-01 09:11:58.703272|61.41|american history +97|349|65769|4294967386|12.90|42.00|false|irene quirinius|2013-03-01 09:11:58.703239|58.05|xylophone band +40|331|65705|4294967470|91.78|31.30|false|oscar van buren|2013-03-01 09:11:58.703250|25.89|zync studies +119|310|65679|4294967319|14.21|10.38|true|sarah carson|2013-03-01 09:11:58.703221|64.15|industrial engineering +38|493|65788|4294967427|60.85|47.66|true|ethan white|2013-03-01 09:11:58.703173|25.77|zync studies +70|285|65718|4294967353|37.10|35.85|false|mike ellison|2013-03-01 09:11:58.703167|89.67|kindergarten +93|403|65752|4294967497|97.65|47.03|false|luke miller|2013-03-01 09:11:58.703133|28.92|biology +48|501|65720|4294967525|2.79|20.39|true|alice allen|2013-03-01 09:11:58.703093|22.34|wind surfing +88|307|65747|4294967389|31.86|20.97|true|jessica carson|2013-03-01 09:11:58.703194|15.07|joggying +66|259|65767|4294967378|51.72|5.11|true|gabriella falkner|2013-03-01 09:11:58.703119|90.88|xylophone band +118|292|65637|4294967403|49.68|11.16|false|jessica polk|2013-03-01 09:11:58.703262|48.88|education +49|448|65742|4294967470|82.34|30.04|true|yuri polk|2013-03-01 09:11:58.703171|24.94|forestry +93|326|65678|4294967428|45.19|1.71|true|david white|2013-03-01 09:11:58.703101|8.72|industrial engineering +74|298|65773|4294967400|21.80|28.85|false|oscar xylophone|2013-03-01 09:11:58.703124|47.09|religion +96|263|65769|4294967532|53.02|4.04|false|calvin quirinius|2013-03-01 09:11:58.703128|11.45|forestry +11|382|65779|4294967345|62.72|40.27|false|david davidson|2013-03-01 09:11:58.703176|28.20|geology +123|284|65783|4294967301|60.26|37.08|true|victor davidson|2013-03-01 09:11:58.703227|17.65|forestry +58|478|65684|4294967439|95.53|14.16|true|ulysses van buren|2013-03-01 09:11:58.703076|67.34|values clariffication +27|256|65599|4294967383|89.55|16.61|true|xavier quirinius|2013-03-01 09:11:58.703304|41.61|linguistics +66|372|65718|4294967401|83.87|28.64|false|yuri underhill|2013-03-01 09:11:58.703117|33.35|education +82|337|65662|4294967453|11.57|45.53|false|wendy underhill|2013-03-01 09:11:58.703299|27.89|wind surfing +94|485|65735|4294967472|69.88|14.92|false|victor hernandez|2013-03-01 09:11:58.703193|28.42|topology +101|259|65757|4294967550|82.97|42.30|false|nick miller|2013-03-01 09:11:58.703190|90.09|study skills +48|443|65675|4294967322|57.67|9.40|true|nick robinson|2013-03-01 09:11:58.703228|4.19|biology +60|309|65743|4294967450|15.90|7.53|true|xavier laertes|2013-03-01 09:11:58.703289|38.73|wind surfing +16|323|65748|4294967471|6.84|17.55|true|zach ellison|2013-03-01 09:11:58.703164|77.52|religion +59|484|65743|4294967483|66.67|8.09|true|victor king|2013-03-01 09:11:58.703297|52.65|debate +2|507|65764|4294967544|72.51|30.90|false|katie young|2013-03-01 09:11:58.703223|57.10|opthamology +101|315|65749|4294967519|85.14|5.23|true|mike steinbeck|2013-03-01 09:11:58.703131|73.24|topology +38|290|65628|4294967473|56.07|37.96|false|wendy allen|2013-03-01 09:11:58.703322|2.48|philosophy +117|400|65755|4294967399|91.48|17.46|true|priscilla johnson|2013-03-01 09:11:58.703267|22.44|history +120|325|65758|4294967540|79.19|11.26|true|wendy underhill|2013-03-01 09:11:58.703226|94.90|debate +-1|274|65641|4294967476|44.56|39.79|true|fred underhill|2013-03-01 09:11:58.703265|63.70|study skills +39|482|65664|4294967343|64.82|36.45|false|mike white|2013-03-01 09:11:58.703317|3.37|xylophone band +79|326|65674|4294967348|35.52|39.97|true|sarah falkner|2013-03-01 09:11:58.703280|40.58|yard duty +26|283|65680|4294967426|1.38|24.97|false|victor quirinius|2013-03-01 09:11:58.703175|81.03|quiet hour +88|301|65540|4294967343|80.93|17.59|true|tom hernandez|2013-03-01 09:11:58.703207|39.89|nap time +96|369|65764|4294967460|81.00|20.14|true|katie falkner|2013-03-01 09:11:58.703309|24.70|topology +82|404|65585|4294967532|3.17|44.99|false|ulysses brown|2013-03-01 09:11:58.703258|35.23|yard duty +78|509|65614|4294967508|39.03|16.44|true|nick xylophone|2013-03-01 09:11:58.703165|75.70|religion +5|388|65570|4294967386|70.37|9.03|false|zach thompson|2013-03-01 09:11:58.703073|72.86|topology +118|364|65686|4294967344|80.31|14.47|false|mike steinbeck|2013-03-01 09:11:58.703269|32.01|biology +62|299|65577|4294967382|25.24|39.64|true|oscar laertes|2013-03-01 09:11:58.703286|64.52|history +69|263|65658|4294967393|58.09|28.20|true|priscilla falkner|2013-03-01 09:11:58.703189|80.24|wind surfing +17|441|65593|4294967406|60.22|46.14|false|quinn garcia|2013-03-01 09:11:58.703320|51.89|geology +88|438|65543|4294967400|73.13|48.39|true|rachel allen|2013-03-01 09:11:58.703239|41.97|nap time +36|433|65744|4294967339|97.59|33.21|false|priscilla allen|2013-03-01 09:11:58.703111|3.29|yard duty +99|384|65725|4294967362|84.90|44.61|true|wendy ichabod|2013-03-01 09:11:58.703097|61.31|american history +60|265|65654|4294967484|4.80|37.68|false|mike laertes|2013-03-01 09:11:58.703165|97.49|wind surfing +112|378|65669|4294967335|72.23|9.78|false|sarah king|2013-03-01 09:11:58.703087|39.82|forestry +13|307|65555|4294967403|20.14|13.23|false|xavier zipper|2013-03-01 09:11:58.703250|39.16|wind surfing +98|344|65725|4294967335|48.64|14.04|false|priscilla ovid|2013-03-01 09:11:58.703152|51.48|opthamology +105|420|65682|4294967330|79.54|42.95|false|alice nixon|2013-03-01 09:11:58.703306|38.04|joggying +42|458|65538|4294967351|27.04|40.79|false|holly hernandez|2013-03-01 09:11:58.703191|43.33|history +34|481|65668|4294967550|23.52|30.53|true|nick ichabod|2013-03-01 09:11:58.703140|68.00|joggying +14|260|65703|4294967547|40.07|25.09|true|mike xylophone|2013-03-01 09:11:58.703286|47.48|opthamology +5|338|65664|4294967335|23.13|8.04|false|tom white|2013-03-01 09:11:58.703292|2.87|nap time +9|329|65544|4294967549|55.30|4.55|false|xavier allen|2013-03-01 09:11:58.703180|97.10|geology +113|293|65613|4294967515|95.36|3.19|true|quinn quirinius|2013-03-01 09:11:58.703324|4.09|forestry +26|303|65571|4294967419|80.17|25.06|false|sarah quirinius|2013-03-01 09:11:58.703133|52.79|quiet hour +81|396|65728|4294967319|82.26|30.74|true|wendy robinson|2013-03-01 09:11:58.703140|9.59|opthamology +4|419|65568|4294967448|28.11|17.89|false|sarah xylophone|2013-03-01 09:11:58.703121|6.26|topology +-3|337|65658|4294967361|43.40|12.05|false|victor allen|2013-03-01 09:11:58.703155|45.20|topology +45|293|65671|4294967366|16.72|31.07|true|holly garcia|2013-03-01 09:11:58.703103|35.97|opthamology +27|354|65687|4294967485|58.63|32.78|false|bob garcia|2013-03-01 09:11:58.703253|48.20|history +45|433|65565|4294967330|89.07|6.79|false|irene davidson|2013-03-01 09:11:58.703120|43.55|undecided +2|366|65648|4294967464|57.52|22.54|false|priscilla van buren|2013-03-01 09:11:58.703188|57.79|biology +101|261|65641|4294967363|92.51|47.13|false|sarah brown|2013-03-01 09:11:58.703318|1.25|mathematics +124|482|65667|4294967439|82.74|30.64|false|nick zipper|2013-03-01 09:11:58.703152|89.94|geology +49|393|65541|4294967538|13.45|3.82|false|holly laertes|2013-03-01 09:11:58.703289|88.75|linguistics +74|488|65672|4294967413|57.77|30.34|true|yuri polk|2013-03-01 09:11:58.703082|72.59|debate +55|271|65756|4294967378|53.22|12.50|true|oscar carson|2013-03-01 09:11:58.703128|92.74|religion +120|458|65684|4294967409|64.20|8.60|false|alice white|2013-03-01 09:11:58.703184|73.51|undecided +80|496|65675|4294967317|38.29|19.61|false|quinn xylophone|2013-03-01 09:11:58.703262|84.42|education +73|287|65571|4294967454|40.52|40.95|true|mike ichabod|2013-03-01 09:11:58.703312|5.88|american history +30|414|65783|4294967458|88.59|19.69|false|victor carson|2013-03-01 09:11:58.703252|79.02|yard duty +92|450|65688|4294967397|93.03|10.35|true|nick thompson|2013-03-01 09:11:58.703201|46.29|nap time +47|464|65620|4294967392|19.18|32.39|false|luke ichabod|2013-03-01 09:11:58.703324|17.02|biology +34|279|65719|4294967497|59.87|5.46|false|mike zipper|2013-03-01 09:11:58.703163|46.48|zync studies +70|416|65653|4294967346|25.13|2.51|true|quinn allen|2013-03-01 09:11:58.703165|89.23|values clariffication +93|373|65595|4294967486|91.67|27.74|true|alice van buren|2013-03-01 09:11:58.703259|63.57|opthamology +110|443|65664|4294967507|94.65|2.81|false|david davidson|2013-03-01 09:11:58.703145|57.30|debate +111|479|65689|4294967440|6.39|47.56|false|irene zipper|2013-03-01 09:11:58.703171|73.97|american history +110|424|65681|4294967364|13.64|7.22|false|bob davidson|2013-03-01 09:11:58.703073|97.52|zync studies +48|288|65665|4294967484|42.23|27.91|false|rachel polk|2013-03-01 09:11:58.703268|73.20|debate +5|411|65573|4294967336|64.06|32.98|true|wendy garcia|2013-03-01 09:11:58.703124|18.98|chemistry +79|339|65686|4294967544|70.29|29.32|false|ethan allen|2013-03-01 09:11:58.703096|98.10|biology +88|482|65639|4294967515|80.14|22.60|false|victor ichabod|2013-03-01 09:11:58.703210|76.98|study skills +77|409|65609|4294967458|38.64|22.64|false|luke hernandez|2013-03-01 09:11:58.703208|38.67|biology +48|275|65732|4294967491|9.75|1.89|false|alice xylophone|2013-03-01 09:11:58.703190|19.73|education +69|347|65684|4294967362|1.20|15.53|true|zach underhill|2013-03-01 09:11:58.703192|34.87|topology +85|386|65658|4294967418|33.25|6.75|true|zach underhill|2013-03-01 09:11:58.703286|69.87|values clariffication +91|463|65718|4294967463|38.33|45.89|true|fred nixon|2013-03-01 09:11:58.703257|78.16|geology +34|302|65748|4294967406|19.42|21.36|false|tom hernandez|2013-03-01 09:11:58.703153|53.37|linguistics +40|467|65745|4294967453|1.63|9.23|false|gabriella xylophone|2013-03-01 09:11:58.703310|59.30|values clariffication +3|382|65592|4294967490|46.74|11.36|true|zach quirinius|2013-03-01 09:11:58.703143|45.36|geology +55|314|65707|4294967387|12.91|18.52|false|jessica white|2013-03-01 09:11:58.703279|22.40|chemistry +119|364|65791|4294967432|89.50|19.42|true|bob davidson|2013-03-01 09:11:58.703180|7.39|education +115|426|65651|4294967412|16.11|43.34|true|alice zipper|2013-03-01 09:11:58.703144|0.30|linguistics +-2|428|65673|4294967471|60.25|16.57|true|fred robinson|2013-03-01 09:11:58.703200|41.47|yard duty +79|481|65726|4294967383|22.70|41.13|true|tom garcia|2013-03-01 09:11:58.703183|19.06|history +2|321|65697|4294967442|15.45|20.43|true|xavier davidson|2013-03-01 09:11:58.703241|84.61|american history +120|510|65674|4294967376|3.38|25.74|true|sarah ovid|2013-03-01 09:11:58.703124|67.74|opthamology +100|396|65736|4294967425|66.81|1.85|true|xavier brown|2013-03-01 09:11:58.703082|12.53|geology +14|451|65617|4294967394|64.46|18.99|false|tom ichabod|2013-03-01 09:11:58.703226|36.36|yard duty +9|404|65689|4294967372|99.80|3.82|false|luke ichabod|2013-03-01 09:11:58.703278|13.10|debate +1|300|65719|4294967430|3.41|47.94|false|tom zipper|2013-03-01 09:11:58.703248|27.04|biology +50|353|65724|4294967458|80.41|41.10|true|sarah hernandez|2013-03-01 09:11:58.703276|28.81|study skills +65|483|65704|4294967370|67.13|35.00|false|calvin johnson|2013-03-01 09:11:58.703092|43.67|history +94|478|65701|4294967489|33.79|35.73|false|bob carson|2013-03-01 09:11:58.703098|53.34|linguistics +30|286|65791|4294967351|36.71|29.69|false|alice robinson|2013-03-01 09:11:58.703100|61.50|kindergarten +5|342|65638|4294967308|1.93|22.60|false|sarah miller|2013-03-01 09:11:58.703216|99.63|quiet hour +11|320|65586|4294967341|35.06|49.67|false|luke hernandez|2013-03-01 09:11:58.703269|71.86|topology +44|489|65701|4294967545|33.53|30.45|true|zach nixon|2013-03-01 09:11:58.703094|36.45|mathematics +122|355|65697|4294967369|97.70|34.52|false|wendy brown|2013-03-01 09:11:58.703251|87.52|debate +32|474|65721|4294967548|26.95|6.76|true|holly falkner|2013-03-01 09:11:58.703071|16.55|philosophy +112|366|65643|4294967541|63.44|19.11|true|oscar laertes|2013-03-01 09:11:58.703111|81.04|philosophy +25|337|65699|4294967349|36.45|10.93|false|yuri brown|2013-03-01 09:11:58.703116|92.72|zync studies +39|446|65767|4294967406|29.50|3.62|true|wendy zipper|2013-03-01 09:11:58.703254|13.15|education +114|375|65692|4294967298|63.61|47.33|false|jessica quirinius|2013-03-01 09:11:58.703281|38.13|yard duty +103|427|65772|4294967338|75.43|34.83|false|katie miller|2013-03-01 09:11:58.703312|88.55|kindergarten +96|342|65645|4294967522|88.94|0.22|false|mike nixon|2013-03-01 09:11:58.703303|54.16|quiet hour +121|326|65600|4294967393|92.14|18.16|false|tom ichabod|2013-03-01 09:11:58.703200|54.89|biology +57|440|65730|4294967472|53.97|14.17|false|bob van buren|2013-03-01 09:11:58.703097|62.66|kindergarten +55|402|65710|4294967415|55.70|1.61|true|rachel king|2013-03-01 09:11:58.703218|34.83|yard duty +60|401|65572|4294967466|81.08|38.20|true|quinn miller|2013-03-01 09:11:58.703296|40.33|geology +50|331|65573|4294967508|61.31|12.31|true|holly xylophone|2013-03-01 09:11:58.703152|53.31|philosophy +78|500|65588|4294967425|17.31|18.15|true|irene miller|2013-03-01 09:11:58.703311|60.41|opthamology +59|470|65673|4294967422|79.95|32.02|true|ulysses allen|2013-03-01 09:11:58.703124|89.97|zync studies +10|409|65548|4294967389|9.25|30.43|false|jessica davidson|2013-03-01 09:11:58.703256|61.15|religion +72|392|65686|4294967515|19.77|22.15|false|ulysses robinson|2013-03-01 09:11:58.703211|52.51|yard duty +100|280|65703|4294967471|58.55|1.27|false|oscar underhill|2013-03-01 09:11:58.703289|61.90|wind surfing +50|411|65646|4294967498|64.98|48.95|true|bob king|2013-03-01 09:11:58.703151|28.64|american history +56|447|65650|4294967424|72.52|48.41|false|oscar ellison|2013-03-01 09:11:58.703238|95.89|industrial engineering +72|360|65586|4294967478|28.52|17.69|true|rachel miller|2013-03-01 09:11:58.703245|33.64|xylophone band +24|431|65773|4294967319|10.30|17.53|true|zach falkner|2013-03-01 09:11:58.703254|23.55|yard duty +83|359|65777|4294967309|89.01|49.71|true|luke davidson|2013-03-01 09:11:58.703083|26.13|study skills +54|338|65690|4294967506|33.70|13.11|true|nick quirinius|2013-03-01 09:11:58.703189|32.28|religion +102|347|65587|4294967378|89.89|28.15|false|jessica allen|2013-03-01 09:11:58.703139|17.93|study skills +75|467|65572|4294967402|1.84|42.76|false|holly van buren|2013-03-01 09:11:58.703091|17.42|american history +44|430|65555|4294967538|56.73|48.09|true|quinn ellison|2013-03-01 09:11:58.703324|53.27|debate +39|298|65756|4294967310|0.79|6.22|true|bob carson|2013-03-01 09:11:58.703072|74.59|nap time +118|386|65747|4294967516|39.52|32.91|true|holly white|2013-03-01 09:11:58.703220|79.22|joggying +108|456|65640|4294967354|73.34|13.84|false|irene garcia|2013-03-01 09:11:58.703106|16.27|zync studies +44|377|65656|4294967345|8.87|22.92|false|alice underhill|2013-03-01 09:11:58.703314|72.95|wind surfing +82|348|65658|4294967477|9.88|15.67|false|oscar ovid|2013-03-01 09:11:58.703201|54.86|yard duty +39|306|65762|4294967362|29.56|3.01|false|mike johnson|2013-03-01 09:11:58.703292|31.32|debate +118|352|65592|4294967468|78.85|10.55|false|xavier davidson|2013-03-01 09:11:58.703273|70.62|history +34|447|65631|4294967470|0.69|14.39|true|luke davidson|2013-03-01 09:11:58.703305|65.93|quiet hour +92|346|65756|4294967468|15.22|35.12|true|ethan brown|2013-03-01 09:11:58.703145|6.16|philosophy +106|278|65683|4294967420|89.96|40.77|false|ulysses zipper|2013-03-01 09:11:58.703131|80.52|mathematics +114|365|65737|4294967516|46.18|5.50|true|ulysses robinson|2013-03-01 09:11:58.703184|99.16|values clariffication +100|331|65558|4294967495|99.17|21.94|true|priscilla white|2013-03-01 09:11:58.703146|38.76|yard duty +123|490|65581|4294967542|17.34|48.96|true|fred johnson|2013-03-01 09:11:58.703240|81.80|education +54|495|65744|4294967542|80.14|27.43|false|irene king|2013-03-01 09:11:58.703257|58.29|american history +10|488|65779|4294967357|87.19|20.37|false|irene nixon|2013-03-01 09:11:58.703217|39.14|kindergarten +12|356|65735|4294967345|73.43|19.80|false|calvin johnson|2013-03-01 09:11:58.703105|91.43|industrial engineering +116|422|65673|4294967524|85.11|47.28|false|ethan garcia|2013-03-01 09:11:58.703114|29.66|yard duty +69|306|65667|4294967427|68.09|1.30|true|calvin steinbeck|2013-03-01 09:11:58.703082|74.67|debate +75|394|65676|4294967426|76.89|38.85|true|victor white|2013-03-01 09:11:58.703123|28.04|forestry +86|395|65731|4294967534|77.96|18.94|true|bob miller|2013-03-01 09:11:58.703300|47.64|yard duty +71|505|65603|4294967393|52.61|27.72|false|xavier robinson|2013-03-01 09:11:58.703182|55.45|education +51|276|65729|4294967447|98.94|1.35|true|ulysses underhill|2013-03-01 09:11:58.703303|57.25|yard duty +81|508|65589|4294967512|44.49|39.58|true|alice xylophone|2013-03-01 09:11:58.703264|13.16|topology +60|504|65719|4294967377|53.40|16.70|false|luke zipper|2013-03-01 09:11:58.703308|75.26|philosophy +50|466|65761|4294967390|96.60|3.30|true|sarah white|2013-03-01 09:11:58.703121|33.95|philosophy +116|434|65721|4294967500|40.22|23.70|true|calvin hernandez|2013-03-01 09:11:58.703123|66.32|biology +55|485|65554|4294967419|45.05|24.54|false|nick robinson|2013-03-01 09:11:58.703321|10.54|kindergarten +39|443|65654|4294967308|94.31|37.67|true|priscilla miller|2013-03-01 09:11:58.703285|49.81|nap time +31|377|65762|4294967401|25.51|48.40|true|victor falkner|2013-03-01 09:11:58.703160|52.24|kindergarten +47|364|65733|4294967519|15.89|1.07|true|david van buren|2013-03-01 09:11:58.703127|1.17|undecided +97|372|65761|4294967543|4.80|31.00|true|holly garcia|2013-03-01 09:11:58.703298|59.76|opthamology +10|416|65585|4294967487|58.65|32.20|true|victor zipper|2013-03-01 09:11:58.703145|38.70|religion +94|312|65641|4294967542|35.85|26.86|false|zach polk|2013-03-01 09:11:58.703099|93.86|study skills +68|432|65565|4294967470|66.17|27.45|true|bob davidson|2013-03-01 09:11:58.703245|37.86|quiet hour +35|297|65632|4294967451|58.75|27.65|true|xavier quirinius|2013-03-01 09:11:58.703317|45.92|american history +-1|309|65615|4294967443|18.57|11.13|false|oscar garcia|2013-03-01 09:11:58.703325|8.90|quiet hour +19|467|65628|4294967536|16.76|9.37|true|victor johnson|2013-03-01 09:11:58.703215|97.91|values clariffication +73|426|65776|4294967529|95.39|13.18|true|david carson|2013-03-01 09:11:58.703183|63.80|history +24|256|65549|4294967529|23.72|7.97|false|zach nixon|2013-03-01 09:11:58.703318|85.63|undecided +9|421|65618|4294967485|77.04|7.96|false|yuri king|2013-03-01 09:11:58.703296|85.85|kindergarten +107|504|65694|4294967432|8.68|49.24|false|bob young|2013-03-01 09:11:58.703131|52.08|philosophy +118|438|65779|4294967409|94.88|17.74|true|xavier carson|2013-03-01 09:11:58.703322|99.13|yard duty +0|334|65686|4294967313|75.36|6.40|false|jessica king|2013-03-01 09:11:58.703120|98.04|chemistry +51|312|65628|4294967308|0.09|22.27|false|victor steinbeck|2013-03-01 09:11:58.703265|16.24|forestry +70|295|65632|4294967410|46.97|48.52|true|fred quirinius|2013-03-01 09:11:58.703186|3.53|wind surfing +39|384|65695|4294967462|53.10|20.23|true|ulysses johnson|2013-03-01 09:11:58.703092|20.39|philosophy +42|468|65693|4294967486|22.36|25.48|true|calvin nixon|2013-03-01 09:11:58.703323|7.89|study skills +20|423|65609|4294967509|39.05|10.47|true|bob polk|2013-03-01 09:11:58.703310|12.58|mathematics +37|434|65731|4294967527|46.35|8.75|false|bob polk|2013-03-01 09:11:58.703222|96.74|nap time +14|288|65714|4294967325|90.04|0.13|false|quinn zipper|2013-03-01 09:11:58.703154|26.56|xylophone band +3|501|65685|4294967496|96.24|24.46|false|quinn brown|2013-03-01 09:11:58.703275|51.58|wind surfing +25|418|65676|4294967516|42.63|2.70|false|tom quirinius|2013-03-01 09:11:58.703211|82.04|study skills +74|477|65597|4294967343|6.94|23.21|false|bob king|2013-03-01 09:11:58.703220|72.15|religion +16|436|65719|4294967408|77.98|13.10|true|holly king|2013-03-01 09:11:58.703146|12.75|xylophone band +14|264|65754|4294967428|0.20|32.48|false|luke zipper|2013-03-01 09:11:58.703132|83.15|american history +96|298|65567|4294967314|83.24|36.95|true|nick xylophone|2013-03-01 09:11:58.703245|67.50|religion +34|325|65658|4294967435|47.86|29.98|false|jessica robinson|2013-03-01 09:11:58.703104|84.71|study skills +36|314|65685|4294967349|35.87|9.47|true|david johnson|2013-03-01 09:11:58.703234|41.80|wind surfing +35|298|65633|4294967514|85.19|11.97|true|yuri zipper|2013-03-01 09:11:58.703262|1.40|mathematics +80|431|65605|4294967371|47.31|14.36|false|alice zipper|2013-03-01 09:11:58.703189|53.53|debate +28|509|65719|4294967480|99.25|7.63|true|luke white|2013-03-01 09:11:58.703228|26.22|biology +99|427|65653|4294967361|9.23|5.24|false|oscar ellison|2013-03-01 09:11:58.703158|28.93|xylophone band +119|286|65660|4294967374|46.73|10.28|false|rachel polk|2013-03-01 09:11:58.703309|44.74|yard duty +101|506|65781|4294967353|64.17|41.61|true|katie davidson|2013-03-01 09:11:58.703247|14.83|zync studies +66|289|65722|4294967404|8.42|7.98|false|alice underhill|2013-03-01 09:11:58.703235|60.56|joggying +115|412|65663|4294967496|72.87|28.10|true|calvin garcia|2013-03-01 09:11:58.703300|79.81|joggying +7|266|65717|4294967448|60.76|0.14|true|yuri miller|2013-03-01 09:11:58.703244|25.21|study skills +51|357|65563|4294967379|73.06|39.29|true|jessica white|2013-03-01 09:11:58.703258|10.15|yard duty +11|459|65785|4294967419|41.25|32.57|true|irene young|2013-03-01 09:11:58.703223|5.65|forestry +43|491|65594|4294967301|18.42|23.54|false|gabriella steinbeck|2013-03-01 09:11:58.703118|47.17|philosophy +44|301|65702|4294967445|1.88|22.12|true|katie brown|2013-03-01 09:11:58.703215|98.20|topology +75|491|65563|4294967426|52.98|7.52|true|ulysses falkner|2013-03-01 09:11:58.703256|49.23|topology +104|407|65683|4294967443|84.37|22.37|false|sarah johnson|2013-03-01 09:11:58.703281|21.29|topology +101|310|65686|4294967402|13.78|18.26|false|calvin carson|2013-03-01 09:11:58.703150|96.30|education +50|472|65614|4294967450|12.37|4.49|false|mike polk|2013-03-01 09:11:58.703126|89.32|values clariffication +61|263|65583|4294967461|64.81|2.29|false|quinn johnson|2013-03-01 09:11:58.703156|85.54|joggying +69|310|65558|4294967475|79.32|42.49|true|katie steinbeck|2013-03-01 09:11:58.703112|86.66|geology +20|285|65759|4294967447|46.70|1.09|true|calvin underhill|2013-03-01 09:11:58.703239|94.46|opthamology +117|319|65629|4294967401|15.34|48.84|true|ethan hernandez|2013-03-01 09:11:58.703224|24.90|chemistry +51|368|65687|4294967383|80.54|38.99|true|zach ovid|2013-03-01 09:11:58.703212|12.61|geology +72|439|65616|4294967506|44.30|6.18|false|holly miller|2013-03-01 09:11:58.703074|76.72|history +94|413|65581|4294967434|89.48|43.78|false|xavier miller|2013-03-01 09:11:58.703298|64.77|nap time +32|506|65635|4294967404|90.32|31.72|true|ethan hernandez|2013-03-01 09:11:58.703169|37.66|wind surfing +24|301|65717|4294967420|99.08|1.68|false|sarah thompson|2013-03-01 09:11:58.703164|66.60|industrial engineering +69|388|65694|4294967379|64.47|45.06|true|holly thompson|2013-03-01 09:11:58.703114|71.22|religion +1|507|65548|4294967507|46.99|46.57|false|sarah xylophone|2013-03-01 09:11:58.703263|69.45|values clariffication +88|327|65772|4294967528|54.71|35.71|false|yuri ovid|2013-03-01 09:11:58.703309|43.94|american history +2|396|65671|4294967437|73.17|49.38|false|luke underhill|2013-03-01 09:11:58.703128|62.89|xylophone band +111|313|65638|4294967356|56.66|44.77|false|yuri van buren|2013-03-01 09:11:58.703089|55.56|biology +10|266|65646|4294967469|75.08|6.09|true|zach young|2013-03-01 09:11:58.703276|12.92|philosophy +27|345|65760|4294967361|19.50|35.24|true|fred robinson|2013-03-01 09:11:58.703194|14.04|kindergarten +29|497|65563|4294967457|97.30|23.05|false|wendy zipper|2013-03-01 09:11:58.703232|19.45|history +114|467|65740|4294967387|39.31|28.05|false|yuri robinson|2013-03-01 09:11:58.703129|97.28|linguistics +117|456|65565|4294967381|81.41|15.50|true|yuri brown|2013-03-01 09:11:58.703195|79.85|industrial engineering +118|340|65736|4294967447|12.42|25.39|false|jessica xylophone|2013-03-01 09:11:58.703227|82.19|study skills +3|409|65774|4294967458|30.88|19.52|false|sarah allen|2013-03-01 09:11:58.703159|44.66|linguistics +90|489|65722|4294967485|63.55|29.46|true|katie steinbeck|2013-03-01 09:11:58.703310|7.27|quiet hour +94|412|65612|4294967299|46.04|19.37|true|calvin ellison|2013-03-01 09:11:58.703106|48.48|mathematics +8|439|65735|4294967361|87.11|44.27|false|mike allen|2013-03-01 09:11:58.703084|67.50|zync studies +4|434|65623|4294967361|56.60|10.99|true|luke johnson|2013-03-01 09:11:58.703313|92.36|forestry +25|472|65739|4294967333|86.25|18.58|false|mike brown|2013-03-01 09:11:58.703230|19.68|education +44|436|65787|4294967374|23.59|8.19|false|mike davidson|2013-03-01 09:11:58.703297|44.32|opthamology +57|351|65629|4294967550|61.53|1.97|false|priscilla davidson|2013-03-01 09:11:58.703172|2.56|topology +17|366|65635|4294967423|98.31|1.51|true|wendy quirinius|2013-03-01 09:11:58.703275|11.87|topology +83|291|65727|4294967410|90.90|41.15|false|david robinson|2013-03-01 09:11:58.703152|49.49|xylophone band +37|326|65593|4294967529|14.36|0.06|true|gabriella quirinius|2013-03-01 09:11:58.703192|24.61|zync studies +102|349|65617|4294967516|54.02|41.69|false|ethan king|2013-03-01 09:11:58.703163|90.98|mathematics +-2|502|65752|4294967543|48.26|23.84|true|nick laertes|2013-03-01 09:11:58.703239|45.64|values clariffication +21|367|65667|4294967323|87.17|9.78|true|zach falkner|2013-03-01 09:11:58.703119|11.74|study skills +3|443|65678|4294967299|16.25|19.45|false|bob underhill|2013-03-01 09:11:58.703207|8.71|american history +23|326|65624|4294967325|10.34|21.24|true|priscilla quirinius|2013-03-01 09:11:58.703210|40.50|kindergarten +22|345|65576|4294967344|78.13|43.20|false|ethan ichabod|2013-03-01 09:11:58.703184|52.41|chemistry +75|467|65766|4294967337|45.02|38.79|false|xavier king|2013-03-01 09:11:58.703093|11.34|chemistry +68|256|65563|4294967484|94.87|45.55|true|wendy nixon|2013-03-01 09:11:58.703276|94.61|education +77|449|65602|4294967481|40.09|15.37|true|tom johnson|2013-03-01 09:11:58.703304|43.11|opthamology +0|481|65698|4294967455|33.64|46.45|true|xavier hernandez|2013-03-01 09:11:58.703305|58.45|wind surfing +92|367|65680|4294967396|44.27|29.82|true|xavier nixon|2013-03-01 09:11:58.703213|37.60|biology +8|389|65570|4294967400|33.70|15.10|true|calvin garcia|2013-03-01 09:11:58.703139|26.88|chemistry +51|370|65763|4294967341|5.83|2.02|false|katie ellison|2013-03-01 09:11:58.703177|98.63|forestry +21|370|65571|4294967341|37.10|7.98|false|bob carson|2013-03-01 09:11:58.703150|64.71|quiet hour +8|382|65762|4294967443|45.37|27.71|false|rachel garcia|2013-03-01 09:11:58.703270|22.25|topology +124|430|65556|4294967455|51.58|27.21|true|tom davidson|2013-03-01 09:11:58.703156|6.22|philosophy +21|396|65770|4294967476|33.27|8.25|true|david garcia|2013-03-01 09:11:58.703198|3.27|xylophone band +0|455|65544|4294967396|47.80|35.96|false|rachel steinbeck|2013-03-01 09:11:58.703106|62.60|geology +48|332|65721|4294967296|12.35|28.34|true|david ovid|2013-03-01 09:11:58.703155|45.19|quiet hour +97|436|65630|4294967387|99.55|10.18|false|quinn falkner|2013-03-01 09:11:58.703178|55.89|linguistics +9|419|65625|4294967348|35.92|3.65|true|rachel zipper|2013-03-01 09:11:58.703165|49.65|mathematics +-1|458|65596|4294967444|54.63|15.25|false|katie garcia|2013-03-01 09:11:58.703257|36.18|debate +82|473|65607|4294967466|43.77|32.88|false|fred white|2013-03-01 09:11:58.703278|96.64|religion +110|319|65664|4294967439|74.15|22.62|false|ulysses miller|2013-03-01 09:11:58.703225|85.26|kindergarten +2|397|65639|4294967396|72.51|42.04|true|rachel laertes|2013-03-01 09:11:58.703149|9.78|opthamology +89|338|65676|4294967444|0.62|25.57|true|mike brown|2013-03-01 09:11:58.703123|53.31|debate +59|510|65674|4294967507|92.42|5.24|false|jessica polk|2013-03-01 09:11:58.703241|30.00|religion +91|308|65591|4294967300|9.39|23.99|true|zach white|2013-03-01 09:11:58.703121|15.65|study skills +123|359|65768|4294967536|80.58|14.55|false|bob thompson|2013-03-01 09:11:58.703176|97.41|opthamology +32|499|65702|4294967537|64.03|26.28|true|jessica garcia|2013-03-01 09:11:58.703317|73.77|nap time +101|356|65783|4294967498|79.87|2.02|true|ethan robinson|2013-03-01 09:11:58.703276|64.18|education +28|388|65728|4294967401|34.35|12.02|false|katie laertes|2013-03-01 09:11:58.703185|58.80|education +31|275|65682|4294967486|54.55|39.46|false|victor ellison|2013-03-01 09:11:58.703201|31.37|yard duty +34|457|65643|4294967474|45.53|11.33|true|zach brown|2013-03-01 09:11:58.703298|26.85|yard duty +76|430|65661|4294967337|36.51|6.88|false|holly johnson|2013-03-01 09:11:58.703266|19.46|geology +78|507|65650|4294967511|8.62|36.80|true|luke garcia|2013-03-01 09:11:58.703232|63.07|american history +113|442|65713|4294967428|61.73|27.08|true|ethan ovid|2013-03-01 09:11:58.703112|46.20|philosophy +80|346|65665|4294967549|36.21|0.68|false|bob steinbeck|2013-03-01 09:11:58.703277|36.30|linguistics +109|398|65767|4294967535|22.99|13.58|true|jessica ichabod|2013-03-01 09:11:58.703227|95.87|geology +39|326|65610|4294967367|72.69|14.17|true|quinn allen|2013-03-01 09:11:58.703155|67.74|kindergarten +15|305|65577|4294967365|14.48|20.23|true|bob miller|2013-03-01 09:11:58.703315|21.79|forestry +56|493|65602|4294967370|65.81|46.92|true|nick van buren|2013-03-01 09:11:58.703323|95.24|topology +-2|440|65595|4294967324|14.02|14.62|true|oscar van buren|2013-03-01 09:11:58.703268|28.08|geology +38|462|65565|4294967470|75.33|23.37|false|tom falkner|2013-03-01 09:11:58.703233|30.57|values clariffication +78|332|65750|4294967431|42.02|23.31|false|david garcia|2013-03-01 09:11:58.703221|28.23|biology +56|263|65648|4294967527|56.05|48.23|true|bob ichabod|2013-03-01 09:11:58.703233|18.89|geology +75|487|65720|4294967479|50.24|14.66|false|gabriella brown|2013-03-01 09:11:58.703194|75.74|study skills +17|268|65660|4294967488|65.11|36.26|false|victor hernandez|2013-03-01 09:11:58.703202|18.86|wind surfing +23|365|65702|4294967363|30.17|45.63|false|holly robinson|2013-03-01 09:11:58.703159|7.66|education +41|488|65636|4294967302|28.45|21.45|false|tom laertes|2013-03-01 09:11:58.703250|27.80|philosophy +8|359|65645|4294967419|89.17|0.80|false|rachel brown|2013-03-01 09:11:58.703223|33.29|yard duty +72|372|65692|4294967439|58.23|38.24|false|zach falkner|2013-03-01 09:11:58.703244|79.20|industrial engineering +121|501|65536|4294967513|22.40|47.08|true|yuri ellison|2013-03-01 09:11:58.703079|39.32|american history +105|296|65671|4294967443|41.32|1.41|true|victor steinbeck|2013-03-01 09:11:58.703190|39.27|mathematics +86|468|65705|4294967425|10.98|13.75|false|zach zipper|2013-03-01 09:11:58.703296|68.25|undecided +50|291|65716|4294967433|19.67|13.25|true|ethan nixon|2013-03-01 09:11:58.703219|26.32|philosophy +29|337|65618|4294967457|0.53|36.70|false|luke quirinius|2013-03-01 09:11:58.703190|59.51|history +93|481|65615|4294967428|55.64|45.30|false|bob hernandez|2013-03-01 09:11:58.703254|69.76|education +107|261|65706|4294967450|59.98|27.04|true|ethan robinson|2013-03-01 09:11:58.703099|21.93|xylophone band +17|486|65712|4294967435|19.75|27.77|true|oscar miller|2013-03-01 09:11:58.703201|16.03|forestry +71|462|65755|4294967501|88.51|37.18|true|zach xylophone|2013-03-01 09:11:58.703076|32.79|biology +120|511|65571|4294967400|54.17|33.26|true|sarah johnson|2013-03-01 09:11:58.703188|76.03|religion +114|338|65624|4294967372|27.69|3.36|true|victor garcia|2013-03-01 09:11:58.703198|31.49|debate +58|341|65580|4294967549|53.43|11.40|true|tom miller|2013-03-01 09:11:58.703271|92.64|wind surfing +97|368|65546|4294967492|47.53|41.08|true|sarah garcia|2013-03-01 09:11:58.703313|96.02|topology +38|417|65731|4294967405|63.77|18.88|false|rachel allen|2013-03-01 09:11:58.703202|87.70|nap time +80|429|65775|4294967439|8.41|9.98|true|david zipper|2013-03-01 09:11:58.703154|57.53|values clariffication +66|507|65546|4294967364|34.10|43.88|false|ethan xylophone|2013-03-01 09:11:58.703210|26.11|debate +110|373|65541|4294967330|55.94|46.56|true|yuri allen|2013-03-01 09:11:58.703160|21.59|values clariffication +113|421|65579|4294967314|55.18|45.87|false|calvin quirinius|2013-03-01 09:11:58.703175|8.91|kindergarten +30|318|65756|4294967489|83.02|37.60|true|nick ellison|2013-03-01 09:11:58.703294|25.94|religion +103|486|65569|4294967544|99.34|21.03|false|nick robinson|2013-03-01 09:11:58.703266|79.06|opthamology +38|327|65590|4294967299|92.17|23.49|false|wendy allen|2013-03-01 09:11:58.703091|71.48|history +121|464|65544|4294967372|20.09|9.52|false|gabriella robinson|2013-03-01 09:11:58.703175|92.05|philosophy +71|366|65595|4294967406|11.97|29.38|false|alice polk|2013-03-01 09:11:58.703297|18.21|history +118|264|65755|4294967441|90.49|34.62|false|victor nixon|2013-03-01 09:11:58.703144|68.89|philosophy +89|503|65776|4294967376|64.28|22.80|true|luke steinbeck|2013-03-01 09:11:58.703320|38.54|linguistics +82|292|65585|4294967500|50.06|4.31|false|alice ichabod|2013-03-01 09:11:58.703245|10.93|education +86|425|65717|4294967533|43.31|39.57|false|luke king|2013-03-01 09:11:58.703130|69.06|values clariffication +61|317|65716|4294967459|93.23|13.58|true|victor davidson|2013-03-01 09:11:58.703103|15.42|religion +24|404|65662|4294967336|54.15|4.83|false|irene ellison|2013-03-01 09:11:58.703103|96.44|american history +35|339|65739|4294967464|66.85|11.22|true|victor white|2013-03-01 09:11:58.703276|86.49|mathematics +45|300|65711|4294967524|66.56|3.57|false|oscar carson|2013-03-01 09:11:58.703211|55.36|forestry +9|505|65779|4294967487|51.40|36.50|true|jessica quirinius|2013-03-01 09:11:58.703167|89.00|linguistics +12|490|65600|4294967460|70.03|24.63|true|ethan davidson|2013-03-01 09:11:58.703112|4.89|linguistics +93|368|65782|4294967465|78.85|41.09|false|xavier young|2013-03-01 09:11:58.703118|42.54|zync studies +109|435|65543|4294967343|68.34|13.65|false|alice thompson|2013-03-01 09:11:58.703310|85.50|industrial engineering +-2|463|65565|4294967446|90.86|37.94|true|luke laertes|2013-03-01 09:11:58.703156|99.52|forestry +37|396|65702|4294967431|25.46|24.89|false|bob allen|2013-03-01 09:11:58.703251|41.53|forestry +32|315|65584|4294967519|63.10|12.58|true|quinn king|2013-03-01 09:11:58.703112|94.33|geology +123|470|65743|4294967505|70.65|7.35|false|zach carson|2013-03-01 09:11:58.703082|22.94|biology +17|364|65760|4294967548|15.93|43.48|false|nick ovid|2013-03-01 09:11:58.703265|91.56|linguistics +98|295|65560|4294967467|88.29|49.59|true|yuri quirinius|2013-03-01 09:11:58.703189|56.98|kindergarten +109|436|65754|4294967391|97.71|6.15|false|victor falkner|2013-03-01 09:11:58.703257|97.74|undecided +62|367|65726|4294967509|57.47|37.69|false|zach falkner|2013-03-01 09:11:58.703101|3.09|quiet hour +29|443|65687|4294967520|65.06|30.97|true|tom johnson|2013-03-01 09:11:58.703186|35.78|history +12|385|65585|4294967330|67.75|3.78|true|priscilla miller|2013-03-01 09:11:58.703248|74.57|chemistry +106|464|65658|4294967356|94.03|47.57|true|luke davidson|2013-03-01 09:11:58.703128|11.78|joggying +57|345|65623|4294967374|20.86|7.21|false|priscilla zipper|2013-03-01 09:11:58.703286|27.53|industrial engineering +59|311|65677|4294967388|38.24|27.87|false|holly underhill|2013-03-01 09:11:58.703173|31.94|american history +98|435|65756|4294967412|33.34|47.88|true|zach king|2013-03-01 09:11:58.703305|81.19|nap time +39|376|65708|4294967316|3.37|37.30|false|alice brown|2013-03-01 09:11:58.703182|52.08|joggying +86|339|65776|4294967528|82.07|10.06|false|tom thompson|2013-03-01 09:11:58.703323|44.41|american history +115|429|65692|4294967406|32.96|17.36|true|ethan hernandez|2013-03-01 09:11:58.703143|63.59|linguistics +100|362|65574|4294967394|83.55|26.85|false|quinn brown|2013-03-01 09:11:58.703111|80.80|joggying +108|309|65763|4294967513|97.88|4.42|true|ethan robinson|2013-03-01 09:11:58.703308|38.27|topology +87|489|65564|4294967457|68.79|38.98|false|nick king|2013-03-01 09:11:58.703156|49.12|zync studies +72|500|65775|4294967298|51.04|37.85|false|alice falkner|2013-03-01 09:11:58.703105|69.94|american history +101|335|65769|4294967324|8.01|47.30|false|david white|2013-03-01 09:11:58.703094|7.02|kindergarten +43|293|65633|4294967461|50.02|26.25|false|mike nixon|2013-03-01 09:11:58.703215|62.71|biology +107|426|65572|4294967472|54.27|26.60|true|quinn carson|2013-03-01 09:11:58.703135|64.22|wind surfing +111|356|65755|4294967380|63.29|23.69|false|priscilla brown|2013-03-01 09:11:58.703146|24.87|chemistry +91|474|65687|4294967515|11.30|28.40|true|tom allen|2013-03-01 09:11:58.703288|16.35|zync studies +111|295|65554|4294967480|60.48|17.50|true|nick ellison|2013-03-01 09:11:58.703209|33.65|mathematics +48|261|65678|4294967465|89.19|2.81|false|sarah robinson|2013-03-01 09:11:58.703310|82.53|quiet hour +100|507|65587|4294967526|44.19|46.79|false|oscar king|2013-03-01 09:11:58.703097|38.45|chemistry +91|306|65736|4294967346|91.85|14.28|false|priscilla ichabod|2013-03-01 09:11:58.703171|8.43|biology +32|345|65579|4294967547|8.91|5.05|true|jessica johnson|2013-03-01 09:11:58.703159|70.25|nap time +88|408|65769|4294967395|82.12|12.49|false|irene laertes|2013-03-01 09:11:58.703135|82.47|zync studies +7|395|65661|4294967415|13.73|31.77|true|ethan thompson|2013-03-01 09:11:58.703304|21.26|quiet hour +106|478|65628|4294967466|94.23|48.91|true|tom davidson|2013-03-01 09:11:58.703247|44.91|nap time +46|335|65722|4294967460|27.31|17.14|false|ulysses hernandez|2013-03-01 09:11:58.703210|52.38|forestry +113|464|65609|4294967348|9.50|0.61|true|ethan ellison|2013-03-01 09:11:58.703221|49.55|undecided +30|351|65776|4294967493|48.36|29.42|false|alice young|2013-03-01 09:11:58.703147|76.54|quiet hour +95|363|65611|4294967395|78.23|26.15|false|holly polk|2013-03-01 09:11:58.703102|63.34|forestry +69|416|65666|4294967507|20.44|28.12|false|gabriella brown|2013-03-01 09:11:58.703139|52.80|joggying +95|288|65600|4294967427|60.94|33.88|true|alice xylophone|2013-03-01 09:11:58.703129|63.05|geology +43|305|65574|4294967317|48.39|46.23|false|holly garcia|2013-03-01 09:11:58.703247|10.67|joggying +58|324|65658|4294967513|73.07|40.94|true|irene davidson|2013-03-01 09:11:58.703283|7.59|values clariffication +88|438|65758|4294967370|83.14|12.55|true|jessica hernandez|2013-03-01 09:11:58.703260|4.64|yard duty +38|301|65590|4294967497|50.69|40.79|false|luke garcia|2013-03-01 09:11:58.703110|31.83|study skills +62|346|65758|4294967464|73.76|39.60|true|ethan falkner|2013-03-01 09:11:58.703246|77.01|debate +100|364|65575|4294967298|10.13|29.42|true|wendy nixon|2013-03-01 09:11:58.703302|21.35|philosophy +90|506|65663|4294967439|88.87|24.77|true|victor nixon|2013-03-01 09:11:58.703160|81.70|mathematics +115|440|65631|4294967414|51.16|14.54|false|yuri ichabod|2013-03-01 09:11:58.703132|88.29|topology +106|403|65605|4294967477|85.34|1.41|false|quinn allen|2013-03-01 09:11:58.703272|93.36|xylophone band +11|458|65738|4294967392|52.53|48.68|true|zach falkner|2013-03-01 09:11:58.703257|76.01|history +72|456|65590|4294967329|49.31|32.70|false|zach van buren|2013-03-01 09:11:58.703076|94.44|values clariffication +47|293|65740|4294967398|29.41|35.44|false|nick hernandez|2013-03-01 09:11:58.703139|74.44|yard duty +59|446|65686|4294967438|53.97|14.94|true|xavier quirinius|2013-03-01 09:11:58.703179|98.42|history +62|318|65663|4294967508|64.89|9.31|false|irene white|2013-03-01 09:11:58.703274|29.68|nap time +14|451|65581|4294967434|69.58|47.16|false|david xylophone|2013-03-01 09:11:58.703295|29.39|quiet hour +73|481|65604|4294967495|92.32|32.35|true|priscilla nixon|2013-03-01 09:11:58.703139|79.09|topology +39|391|65565|4294967537|24.19|32.39|true|nick garcia|2013-03-01 09:11:58.703151|37.34|study skills +72|437|65613|4294967339|91.71|36.12|false|jessica davidson|2013-03-01 09:11:58.703269|38.55|kindergarten +35|377|65597|4294967428|48.52|0.20|true|holly hernandez|2013-03-01 09:11:58.703189|84.62|education +89|275|65760|4294967465|71.50|14.86|false|tom miller|2013-03-01 09:11:58.703083|10.11|chemistry +13|465|65745|4294967489|44.87|33.20|true|nick van buren|2013-03-01 09:11:58.703254|91.56|forestry +52|480|65769|4294967487|37.67|45.34|true|holly allen|2013-03-01 09:11:58.703266|67.29|undecided +64|366|65593|4294967397|98.96|38.90|true|victor hernandez|2013-03-01 09:11:58.703098|79.15|geology +106|330|65699|4294967448|3.55|20.60|true|alice thompson|2013-03-01 09:11:58.703317|10.78|undecided +-3|311|65569|4294967460|3.82|35.45|false|luke garcia|2013-03-01 09:11:58.703076|93.38|chemistry +23|345|65725|4294967481|68.23|38.80|false|gabriella ichabod|2013-03-01 09:11:58.703305|69.85|undecided +43|508|65637|4294967448|87.23|13.20|false|bob ellison|2013-03-01 09:11:58.703082|38.88|opthamology +98|290|65670|4294967535|86.82|34.49|true|calvin king|2013-03-01 09:11:58.703136|54.21|american history +29|302|65711|4294967356|28.55|45.40|false|bob falkner|2013-03-01 09:11:58.703257|90.30|joggying +121|333|65632|4294967475|73.85|28.05|false|yuri thompson|2013-03-01 09:11:58.703301|37.22|nap time +82|414|65662|4294967515|16.83|49.01|true|zach polk|2013-03-01 09:11:58.703190|86.86|american history +19|259|65700|4294967501|20.27|49.29|true|irene brown|2013-03-01 09:11:58.703213|74.73|american history +41|382|65542|4294967544|40.11|42.63|true|tom laertes|2013-03-01 09:11:58.703284|49.40|education +25|509|65733|4294967500|66.29|14.30|true|katie nixon|2013-03-01 09:11:58.703294|33.51|undecided +119|407|65770|4294967406|5.63|10.32|true|yuri underhill|2013-03-01 09:11:58.703237|90.68|american history +32|393|65679|4294967520|55.41|49.24|true|wendy king|2013-03-01 09:11:58.703157|20.43|zync studies +25|281|65671|4294967486|72.40|10.66|true|luke ellison|2013-03-01 09:11:58.703095|17.06|forestry +47|423|65718|4294967419|0.18|11.90|false|holly falkner|2013-03-01 09:11:58.703314|67.10|joggying +67|273|65546|4294967443|31.41|16.50|false|yuri ellison|2013-03-01 09:11:58.703252|44.59|nap time +22|296|65783|4294967390|17.26|11.22|false|xavier laertes|2013-03-01 09:11:58.703181|35.58|history +53|406|65575|4294967512|93.79|35.64|false|mike miller|2013-03-01 09:11:58.703260|74.24|kindergarten +41|433|65666|4294967461|38.45|16.29|true|irene johnson|2013-03-01 09:11:58.703158|67.93|kindergarten +105|272|65742|4294967525|45.18|43.54|false|alice davidson|2013-03-01 09:11:58.703267|89.18|history +15|279|65562|4294967535|74.94|30.14|false|ethan falkner|2013-03-01 09:11:58.703300|7.14|topology +96|462|65714|4294967546|93.28|7.90|true|xavier young|2013-03-01 09:11:58.703216|52.52|values clariffication +70|280|65771|4294967501|4.75|29.26|true|wendy robinson|2013-03-01 09:11:58.703226|67.91|wind surfing +93|495|65777|4294967417|76.99|0.40|true|tom polk|2013-03-01 09:11:58.703203|96.30|education +7|425|65729|4294967391|68.08|38.01|true|sarah carson|2013-03-01 09:11:58.703290|34.22|opthamology +26|359|65702|4294967358|18.38|24.02|true|mike allen|2013-03-01 09:11:58.703206|1.07|biology +75|398|65702|4294967380|91.36|17.21|true|alice xylophone|2013-03-01 09:11:58.703263|5.43|opthamology +3|305|65662|4294967506|83.78|34.16|true|mike falkner|2013-03-01 09:11:58.703180|72.04|quiet hour +5|393|65715|4294967371|19.56|48.45|true|irene thompson|2013-03-01 09:11:58.703088|4.49|forestry +79|324|65766|4294967400|76.66|49.19|true|rachel carson|2013-03-01 09:11:58.703125|20.00|study skills +113|457|65550|4294967348|15.90|40.54|false|tom thompson|2013-03-01 09:11:58.703094|0.91|zync studies +45|486|65688|4294967310|14.01|30.31|false|victor miller|2013-03-01 09:11:58.703107|55.37|american history +104|398|65752|4294967355|90.78|21.00|true|yuri miller|2013-03-01 09:11:58.703167|86.52|history +38|292|65748|4294967524|38.18|12.49|true|victor king|2013-03-01 09:11:58.703288|0.05|education +21|472|65756|4294967450|45.21|4.61|false|jessica robinson|2013-03-01 09:11:58.703088|74.53|american history +115|486|65684|4294967522|88.43|36.70|true|irene garcia|2013-03-01 09:11:58.703285|15.28|zync studies +79|287|65679|4294967488|60.01|2.11|true|ulysses hernandez|2013-03-01 09:11:58.703146|72.14|debate +47|499|65678|4294967350|0.94|19.92|true|nick ovid|2013-03-01 09:11:58.703258|34.10|history +113|406|65790|4294967374|29.78|45.51|true|tom ellison|2013-03-01 09:11:58.703207|2.80|xylophone band +17|423|65756|4294967361|21.89|2.49|true|xavier brown|2013-03-01 09:11:58.703189|74.14|joggying +33|327|65752|4294967453|29.95|12.01|false|xavier hernandez|2013-03-01 09:11:58.703163|27.63|values clariffication +95|402|65633|4294967516|6.34|32.67|false|alice hernandez|2013-03-01 09:11:58.703114|90.56|quiet hour +74|397|65756|4294967514|17.04|26.12|true|priscilla thompson|2013-03-01 09:11:58.703097|35.45|topology +13|371|65600|4294967494|35.54|35.84|false|fred ovid|2013-03-01 09:11:58.703140|9.97|debate +113|263|65630|4294967303|95.11|9.55|true|alice garcia|2013-03-01 09:11:58.703079|37.82|opthamology +123|267|65709|4294967518|16.85|20.70|true|irene laertes|2013-03-01 09:11:58.703170|63.94|undecided +118|474|65711|4294967501|36.91|30.63|false|tom zipper|2013-03-01 09:11:58.703231|19.04|undecided +14|312|65689|4294967366|34.28|46.39|false|holly young|2013-03-01 09:11:58.703076|15.40|biology +-2|388|65676|4294967431|25.11|29.00|true|wendy nixon|2013-03-01 09:11:58.703129|52.13|zync studies +34|400|65740|4294967422|93.02|13.92|true|ulysses allen|2013-03-01 09:11:58.703075|42.88|study skills +117|481|65717|4294967450|77.95|8.03|true|priscilla johnson|2013-03-01 09:11:58.703299|98.82|kindergarten +112|326|65759|4294967318|48.78|1.40|false|jessica davidson|2013-03-01 09:11:58.703241|16.00|forestry +81|266|65576|4294967306|44.54|45.97|true|david zipper|2013-03-01 09:11:58.703287|66.43|xylophone band +78|420|65726|4294967509|31.84|36.05|false|jessica brown|2013-03-01 09:11:58.703071|60.77|xylophone band +84|496|65630|4294967517|97.48|27.66|true|wendy robinson|2013-03-01 09:11:58.703088|95.41|chemistry +51|419|65575|4294967534|67.00|36.74|true|katie underhill|2013-03-01 09:11:58.703214|75.80|forestry +115|286|65579|4294967330|15.40|41.60|true|bob ellison|2013-03-01 09:11:58.703275|99.79|religion +73|342|65692|4294967358|53.63|18.59|false|zach xylophone|2013-03-01 09:11:58.703105|19.28|joggying +46|280|65740|4294967325|34.02|22.53|true|david van buren|2013-03-01 09:11:58.703218|77.56|religion +27|478|65575|4294967420|4.54|26.29|true|mike brown|2013-03-01 09:11:58.703178|42.68|american history +3|259|65738|4294967448|52.94|46.89|true|yuri allen|2013-03-01 09:11:58.703180|50.01|history +38|365|65571|4294967526|49.62|4.83|true|yuri ellison|2013-03-01 09:11:58.703215|0.04|history +90|470|65591|4294967368|15.45|8.28|true|nick garcia|2013-03-01 09:11:58.703144|77.77|study skills +83|376|65574|4294967298|88.55|26.17|true|xavier brown|2013-03-01 09:11:58.703203|11.28|american history +99|378|65771|4294967480|30.65|11.56|false|gabriella ellison|2013-03-01 09:11:58.703159|37.06|mathematics +1|491|65755|4294967523|27.15|16.41|true|nick ovid|2013-03-01 09:11:58.703306|4.58|philosophy +94|343|65553|4294967423|81.15|28.58|false|rachel nixon|2013-03-01 09:11:58.703159|21.98|topology +52|336|65670|4294967455|54.92|22.42|false|sarah robinson|2013-03-01 09:11:58.703316|6.40|yard duty +-1|411|65664|4294967308|55.44|4.18|true|gabriella white|2013-03-01 09:11:58.703079|68.26|debate +75|332|65683|4294967460|14.81|17.38|false|jessica hernandez|2013-03-01 09:11:58.703314|87.97|quiet hour +108|379|65786|4294967542|61.59|39.91|false|jessica robinson|2013-03-01 09:11:58.703314|70.01|quiet hour +67|431|65604|4294967323|83.09|23.25|true|quinn miller|2013-03-01 09:11:58.703319|14.35|kindergarten +52|261|65644|4294967395|18.71|13.02|false|bob carson|2013-03-01 09:11:58.703270|84.28|history +46|344|65624|4294967454|14.50|23.17|true|rachel polk|2013-03-01 09:11:58.703269|76.18|education +57|465|65768|4294967538|83.67|11.38|true|oscar king|2013-03-01 09:11:58.703302|88.12|linguistics +56|477|65569|4294967373|18.36|49.75|false|tom hernandez|2013-03-01 09:11:58.703196|15.19|history +1|474|65547|4294967392|54.32|29.39|true|quinn underhill|2013-03-01 09:11:58.703272|69.90|zync studies +18|384|65547|4294967429|12.54|14.51|true|nick white|2013-03-01 09:11:58.703126|82.54|chemistry +71|313|65545|4294967356|95.21|27.64|true|wendy xylophone|2013-03-01 09:11:58.703269|0.22|mathematics +114|348|65622|4294967423|56.13|41.12|false|jessica allen|2013-03-01 09:11:58.703224|93.46|mathematics +101|302|65647|4294967530|59.16|7.74|false|irene steinbeck|2013-03-01 09:11:58.703198|38.56|industrial engineering +2|271|65585|4294967423|87.00|28.70|true|luke xylophone|2013-03-01 09:11:58.703262|63.51|history +41|488|65605|4294967424|26.84|45.52|false|sarah underhill|2013-03-01 09:11:58.703318|89.23|quiet hour +93|299|65781|4294967396|87.79|3.22|false|alice xylophone|2013-03-01 09:11:58.703197|30.23|philosophy +66|497|65766|4294967350|57.25|9.54|true|wendy zipper|2013-03-01 09:11:58.703191|24.63|chemistry +68|434|65737|4294967316|73.26|13.19|false|xavier carson|2013-03-01 09:11:58.703074|78.92|biology +15|499|65650|4294967419|34.83|17.34|false|oscar hernandez|2013-03-01 09:11:58.703093|45.03|opthamology +1|449|65637|4294967421|32.76|41.60|false|priscilla ellison|2013-03-01 09:11:58.703092|4.16|yard duty +106|364|65645|4294967376|13.33|0.11|true|nick robinson|2013-03-01 09:11:58.703175|42.40|values clariffication +28|273|65560|4294967503|74.97|39.03|false|fred allen|2013-03-01 09:11:58.703083|46.99|biology +119|427|65668|4294967527|93.08|6.73|true|fred hernandez|2013-03-01 09:11:58.703149|80.51|chemistry +98|398|65697|4294967301|6.42|32.75|false|wendy king|2013-03-01 09:11:58.703178|54.31|history +108|476|65555|4294967374|26.61|30.25|true|katie robinson|2013-03-01 09:11:58.703105|95.84|opthamology +89|394|65740|4294967505|32.56|33.98|false|wendy hernandez|2013-03-01 09:11:58.703215|9.37|yard duty +121|350|65646|4294967440|27.63|34.77|false|quinn brown|2013-03-01 09:11:58.703253|62.20|religion +59|314|65558|4294967498|98.40|49.78|true|tom white|2013-03-01 09:11:58.703196|82.39|joggying +19|464|65659|4294967449|60.66|47.16|false|bob zipper|2013-03-01 09:11:58.703222|47.97|american history +88|326|65769|4294967500|86.30|7.89|true|nick zipper|2013-03-01 09:11:58.703309|46.36|quiet hour +99|332|65642|4294967550|12.16|7.27|false|bob garcia|2013-03-01 09:11:58.703278|26.99|chemistry +9|498|65621|4294967420|61.01|21.24|true|tom underhill|2013-03-01 09:11:58.703107|97.29|geology +97|321|65787|4294967539|90.31|41.25|true|katie robinson|2013-03-01 09:11:58.703122|84.01|philosophy +91|305|65620|4294967335|96.10|5.28|true|quinn underhill|2013-03-01 09:11:58.703272|7.61|values clariffication +72|501|65748|4294967446|67.72|36.02|false|rachel quirinius|2013-03-01 09:11:58.703230|81.36|topology +12|379|65784|4294967392|32.19|14.20|false|rachel xylophone|2013-03-01 09:11:58.703104|29.54|history +52|410|65543|4294967348|71.35|48.38|true|victor ichabod|2013-03-01 09:11:58.703107|27.12|zync studies +46|405|65787|4294967360|5.17|32.44|true|david underhill|2013-03-01 09:11:58.703166|48.54|chemistry +123|498|65769|4294967345|35.10|29.96|true|luke ichabod|2013-03-01 09:11:58.703099|4.68|values clariffication +12|470|65633|4294967411|42.83|45.56|true|tom thompson|2013-03-01 09:11:58.703222|68.46|mathematics +51|286|65768|4294967504|63.90|39.27|true|mike thompson|2013-03-01 09:11:58.703176|90.84|linguistics +38|345|65769|4294967305|59.57|19.30|true|jessica nixon|2013-03-01 09:11:58.703321|87.61|history +101|276|65791|4294967414|13.25|43.90|true|jessica davidson|2013-03-01 09:11:58.703290|3.01|xylophone band +77|430|65598|4294967390|82.93|37.42|false|irene thompson|2013-03-01 09:11:58.703169|99.66|industrial engineering +88|352|65770|4294967462|65.40|40.41|false|luke ovid|2013-03-01 09:11:58.703239|77.12|zync studies +68|303|65728|4294967422|20.85|38.19|false|bob quirinius|2013-03-01 09:11:58.703111|24.89|chemistry +94|376|65550|4294967434|81.96|35.84|false|calvin miller|2013-03-01 09:11:58.703122|84.90|american history +46|484|65778|4294967518|26.16|45.58|false|victor king|2013-03-01 09:11:58.703109|45.86|education +5|451|65675|4294967486|33.93|43.36|true|mike robinson|2013-03-01 09:11:58.703129|54.04|education +64|303|65646|4294967467|33.80|14.77|false|bob laertes|2013-03-01 09:11:58.703223|86.85|zync studies +72|444|65705|4294967501|52.37|45.57|true|quinn young|2013-03-01 09:11:58.703140|55.68|linguistics +35|487|65623|4294967399|37.63|2.18|true|ethan xylophone|2013-03-01 09:11:58.703073|38.43|biology +55|307|65609|4294967511|7.90|21.70|false|sarah xylophone|2013-03-01 09:11:58.703298|72.04|forestry +2|409|65567|4294967359|36.53|28.29|true|ethan carson|2013-03-01 09:11:58.703264|87.97|kindergarten +65|409|65705|4294967502|13.44|20.46|true|quinn ellison|2013-03-01 09:11:58.703235|51.76|kindergarten +78|425|65643|4294967345|14.29|26.19|true|holly miller|2013-03-01 09:11:58.703132|72.62|study skills +5|395|65569|4294967296|52.76|37.54|false|nick steinbeck|2013-03-01 09:11:58.703103|2.84|forestry +23|470|65708|4294967524|24.86|11.44|false|rachel allen|2013-03-01 09:11:58.703243|24.04|mathematics +-2|383|65600|4294967455|67.97|26.28|false|mike falkner|2013-03-01 09:11:58.703206|61.31|mathematics +35|369|65772|4294967299|32.61|18.07|false|jessica robinson|2013-03-01 09:11:58.703174|24.20|religion +53|342|65752|4294967448|90.76|31.92|true|alice falkner|2013-03-01 09:11:58.703087|9.87|industrial engineering +100|488|65563|4294967520|8.26|22.00|true|quinn johnson|2013-03-01 09:11:58.703223|37.71|values clariffication +95|511|65578|4294967340|76.32|43.80|false|irene young|2013-03-01 09:11:58.703262|2.96|wind surfing +23|467|65768|4294967374|3.23|17.28|false|nick king|2013-03-01 09:11:58.703240|30.11|religion +94|489|65703|4294967407|94.59|5.84|false|katie falkner|2013-03-01 09:11:58.703149|84.25|wind surfing +92|464|65773|4294967465|19.34|21.35|false|priscilla young|2013-03-01 09:11:58.703144|21.28|biology +0|281|65582|4294967387|51.31|14.21|true|irene polk|2013-03-01 09:11:58.703298|58.84|biology +10|375|65735|4294967427|57.14|13.67|false|priscilla polk|2013-03-01 09:11:58.703164|60.98|undecided +38|506|65762|4294967517|50.63|11.40|true|tom allen|2013-03-01 09:11:58.703161|5.37|nap time +76|443|65599|4294967424|64.01|7.43|false|nick brown|2013-03-01 09:11:58.703111|47.07|mathematics +42|449|65586|4294967542|72.69|8.84|true|jessica garcia|2013-03-01 09:11:58.703163|83.64|american history +43|416|65635|4294967478|12.91|16.83|false|holly white|2013-03-01 09:11:58.703100|38.22|joggying +67|311|65700|4294967320|22.88|34.76|true|gabriella miller|2013-03-01 09:11:58.703148|9.92|linguistics +54|481|65558|4294967341|24.60|11.80|false|yuri brown|2013-03-01 09:11:58.703237|40.67|geology +33|301|65677|4294967477|14.40|11.49|false|jessica white|2013-03-01 09:11:58.703150|35.96|geology +69|383|65562|4294967525|99.77|49.20|true|fred davidson|2013-03-01 09:11:58.703278|73.35|zync studies +27|386|65714|4294967323|73.24|15.33|false|sarah quirinius|2013-03-01 09:11:58.703074|78.53|values clariffication +16|368|65606|4294967423|52.71|33.60|true|david allen|2013-03-01 09:11:58.703177|37.11|education +107|451|65623|4294967538|3.10|13.71|true|oscar nixon|2013-03-01 09:11:58.703226|3.60|industrial engineering +119|465|65540|4294967430|35.96|41.95|false|tom quirinius|2013-03-01 09:11:58.703197|32.45|american history +102|464|65544|4294967502|19.10|8.04|true|ethan ovid|2013-03-01 09:11:58.703071|73.80|forestry +32|498|65739|4294967328|14.22|45.12|false|victor zipper|2013-03-01 09:11:58.703146|85.40|values clariffication +87|432|65760|4294967497|51.30|36.32|true|yuri polk|2013-03-01 09:11:58.703144|19.15|values clariffication +3|327|65710|4294967349|87.11|24.71|true|quinn carson|2013-03-01 09:11:58.703315|53.30|chemistry +111|469|65741|4294967400|34.07|7.91|false|mike carson|2013-03-01 09:11:58.703240|88.88|nap time +11|264|65685|4294967521|73.42|16.63|true|victor white|2013-03-01 09:11:58.703103|91.54|zync studies +28|391|65698|4294967483|33.41|2.58|true|zach underhill|2013-03-01 09:11:58.703132|87.56|industrial engineering +119|451|65548|4294967455|54.12|12.88|false|xavier johnson|2013-03-01 09:11:58.703164|68.04|linguistics +21|500|65696|4294967332|22.75|18.92|false|oscar polk|2013-03-01 09:11:58.703227|90.47|nap time +23|355|65552|4294967443|28.38|20.21|false|holly miller|2013-03-01 09:11:58.703290|15.26|history +22|374|65589|4294967318|98.14|1.25|true|david carson|2013-03-01 09:11:58.703075|71.84|philosophy +26|369|65654|4294967543|65.30|8.75|false|yuri johnson|2013-03-01 09:11:58.703195|9.09|religion +102|303|65661|4294967487|25.31|11.37|false|xavier falkner|2013-03-01 09:11:58.703138|65.33|study skills +101|338|65775|4294967308|34.65|35.33|true|nick van buren|2013-03-01 09:11:58.703199|60.75|linguistics +57|479|65597|4294967473|35.55|40.05|true|rachel ichabod|2013-03-01 09:11:58.703323|8.77|topology +40|287|65776|4294967464|72.20|45.10|false|oscar allen|2013-03-01 09:11:58.703115|1.00|religion +64|499|65698|4294967397|32.81|24.68|true|katie van buren|2013-03-01 09:11:58.703317|4.16|forestry +114|352|65787|4294967541|79.47|40.81|true|xavier ichabod|2013-03-01 09:11:58.703313|26.62|xylophone band +2|369|65599|4294967495|42.11|2.56|true|oscar carson|2013-03-01 09:11:58.703152|50.78|joggying +119|403|65544|4294967304|73.95|12.56|false|alice ellison|2013-03-01 09:11:58.703123|66.02|zync studies +5|268|65596|4294967547|19.15|34.88|false|katie davidson|2013-03-01 09:11:58.703123|81.89|opthamology +97|451|65783|4294967407|93.59|40.96|true|alice steinbeck|2013-03-01 09:11:58.703238|5.24|mathematics +66|270|65755|4294967546|52.14|34.28|true|david ovid|2013-03-01 09:11:58.703179|55.70|opthamology +102|411|65552|4294967356|19.55|24.37|false|fred ellison|2013-03-01 09:11:58.703250|96.13|values clariffication +29|291|65737|4294967535|20.38|18.64|false|ulysses laertes|2013-03-01 09:11:58.703278|18.46|forestry +20|422|65755|4294967486|12.67|35.53|false|gabriella robinson|2013-03-01 09:11:58.703235|46.86|forestry +59|273|65766|4294967499|29.10|45.78|false|david thompson|2013-03-01 09:11:58.703293|70.70|undecided +92|282|65658|4294967436|62.77|34.77|false|ulysses underhill|2013-03-01 09:11:58.703232|69.16|education +124|390|65638|4294967550|17.41|8.28|true|oscar king|2013-03-01 09:11:58.703098|28.91|forestry +101|396|65772|4294967425|25.60|6.24|true|david nixon|2013-03-01 09:11:58.703192|28.53|zync studies +61|463|65613|4294967316|34.83|33.68|false|mike underhill|2013-03-01 09:11:58.703305|82.04|debate +-1|352|65690|4294967527|69.28|39.80|true|victor van buren|2013-03-01 09:11:58.703121|11.95|yard duty +62|286|65671|4294967438|2.79|44.44|true|xavier white|2013-03-01 09:11:58.703190|96.82|wind surfing +58|330|65655|4294967338|71.64|4.63|true|luke laertes|2013-03-01 09:11:58.703079|93.26|zync studies +121|340|65740|4294967365|20.36|32.38|true|calvin miller|2013-03-01 09:11:58.703119|27.04|undecided +49|385|65570|4294967375|0.68|1.58|false|bob xylophone|2013-03-01 09:11:58.703091|38.02|history +76|465|65737|4294967444|37.55|45.81|true|luke johnson|2013-03-01 09:11:58.703259|30.20|forestry +3|363|65593|4294967467|73.64|38.72|true|bob hernandez|2013-03-01 09:11:58.703146|49.51|nap time +103|455|65751|4294967421|6.54|22.30|true|xavier king|2013-03-01 09:11:58.703091|89.79|debate +46|382|65547|4294967390|24.87|49.95|false|calvin ellison|2013-03-01 09:11:58.703273|12.67|debate +70|455|65746|4294967350|28.94|27.85|false|bob xylophone|2013-03-01 09:11:58.703174|83.62|yard duty +45|494|65727|4294967462|19.33|18.61|false|sarah robinson|2013-03-01 09:11:58.703180|46.32|undecided +72|501|65608|4294967490|47.81|32.55|true|david young|2013-03-01 09:11:58.703314|27.01|opthamology +33|365|65731|4294967349|27.56|27.50|false|fred hernandez|2013-03-01 09:11:58.703257|23.21|values clariffication +0|305|65673|4294967417|84.87|14.63|false|calvin zipper|2013-03-01 09:11:58.703226|55.73|geology +63|433|65592|4294967431|27.88|33.25|false|xavier ellison|2013-03-01 09:11:58.703308|64.28|forestry +92|478|65712|4294967522|97.55|38.78|false|oscar carson|2013-03-01 09:11:58.703136|95.67|geology +72|257|65756|4294967414|24.44|6.91|false|fred zipper|2013-03-01 09:11:58.703217|56.98|religion +4|414|65789|4294967492|26.00|0.85|true|calvin miller|2013-03-01 09:11:58.703128|96.05|study skills +13|508|65681|4294967376|63.79|43.50|true|luke hernandez|2013-03-01 09:11:58.703222|87.58|yard duty +15|339|65619|4294967500|18.31|19.13|false|nick hernandez|2013-03-01 09:11:58.703080|78.75|nap time +67|278|65680|4294967328|98.56|21.56|true|ulysses brown|2013-03-01 09:11:58.703107|15.01|topology +33|256|65752|4294967318|61.21|33.23|false|victor hernandez|2013-03-01 09:11:58.703259|80.08|zync studies +83|259|65740|4294967309|85.27|38.87|true|wendy hernandez|2013-03-01 09:11:58.703211|47.41|chemistry +122|256|65596|4294967401|5.35|44.00|false|victor davidson|2013-03-01 09:11:58.703214|30.66|industrial engineering +86|285|65742|4294967396|12.66|13.39|false|holly underhill|2013-03-01 09:11:58.703306|13.92|linguistics +59|471|65581|4294967320|57.47|43.12|false|bob davidson|2013-03-01 09:11:58.703215|36.65|religion +0|334|65785|4294967333|39.11|17.92|true|bob robinson|2013-03-01 09:11:58.703235|51.84|geology +57|367|65770|4294967365|16.71|37.66|true|oscar xylophone|2013-03-01 09:11:58.703305|88.79|values clariffication +20|404|65705|4294967517|70.97|14.00|true|xavier johnson|2013-03-01 09:11:58.703267|53.94|topology +53|317|65537|4294967414|59.82|24.76|true|wendy van buren|2013-03-01 09:11:58.703087|36.60|yard duty +52|341|65586|4294967319|39.33|14.81|true|nick laertes|2013-03-01 09:11:58.703305|42.62|industrial engineering +54|462|65660|4294967431|28.38|12.30|false|victor steinbeck|2013-03-01 09:11:58.703311|22.73|zync studies +94|493|65634|4294967475|30.11|37.83|false|katie thompson|2013-03-01 09:11:58.703218|55.64|history +74|442|65758|4294967350|42.48|15.90|false|zach allen|2013-03-01 09:11:58.703175|76.57|american history +122|469|65663|4294967361|91.47|6.90|true|quinn laertes|2013-03-01 09:11:58.703323|43.85|philosophy +42|420|65643|4294967318|57.70|37.60|false|bob white|2013-03-01 09:11:58.703266|17.59|joggying +2|379|65637|4294967442|83.70|28.57|false|holly quirinius|2013-03-01 09:11:58.703305|93.24|xylophone band +7|288|65611|4294967521|96.71|0.23|true|rachel davidson|2013-03-01 09:11:58.703075|17.23|joggying +17|409|65694|4294967498|49.91|0.89|true|victor nixon|2013-03-01 09:11:58.703190|93.01|forestry +42|429|65583|4294967524|23.54|21.40|false|nick young|2013-03-01 09:11:58.703270|12.86|kindergarten +27|342|65771|4294967431|70.32|12.52|true|victor king|2013-03-01 09:11:58.703297|77.10|opthamology +123|466|65539|4294967495|86.55|13.33|true|david white|2013-03-01 09:11:58.703145|59.49|biology +106|445|65776|4294967503|20.14|7.49|true|rachel quirinius|2013-03-01 09:11:58.703301|69.47|biology +82|382|65631|4294967474|36.51|39.62|false|yuri white|2013-03-01 09:11:58.703307|43.44|yard duty +64|341|65560|4294967336|79.34|25.20|true|jessica ellison|2013-03-01 09:11:58.703250|63.30|yard duty +115|317|65586|4294967466|28.15|46.61|false|katie xylophone|2013-03-01 09:11:58.703265|68.10|debate +24|432|65565|4294967395|43.09|27.05|false|sarah ellison|2013-03-01 09:11:58.703254|60.66|wind surfing +24|454|65578|4294967515|37.58|15.89|false|zach carson|2013-03-01 09:11:58.703204|54.16|xylophone band +99|377|65747|4294967445|54.34|39.71|true|oscar thompson|2013-03-01 09:11:58.703277|92.78|zync studies +62|351|65586|4294967446|73.53|5.99|true|gabriella ellison|2013-03-01 09:11:58.703313|76.30|opthamology +124|381|65585|4294967311|23.75|31.09|false|victor ichabod|2013-03-01 09:11:58.703120|31.29|industrial engineering +22|312|65597|4294967526|28.48|14.10|true|bob steinbeck|2013-03-01 09:11:58.703094|84.41|mathematics +44|376|65707|4294967355|18.71|38.31|false|alice miller|2013-03-01 09:11:58.703189|88.11|linguistics +117|425|65645|4294967480|4.71|44.71|false|katie ichabod|2013-03-01 09:11:58.703217|30.74|yard duty +48|266|65592|4294967515|91.09|37.24|true|holly van buren|2013-03-01 09:11:58.703209|85.70|study skills +36|368|65640|4294967548|62.29|19.39|true|gabriella underhill|2013-03-01 09:11:58.703288|44.26|study skills +122|323|65657|4294967399|72.91|30.74|true|mike underhill|2013-03-01 09:11:58.703106|60.29|industrial engineering +5|403|65648|4294967416|69.01|2.01|true|katie quirinius|2013-03-01 09:11:58.703284|60.46|opthamology +67|367|65693|4294967300|1.43|37.53|true|irene ichabod|2013-03-01 09:11:58.703186|38.66|undecided +72|367|65699|4294967417|87.42|45.69|false|victor laertes|2013-03-01 09:11:58.703286|28.61|undecided +4|429|65703|4294967307|38.17|35.54|false|katie thompson|2013-03-01 09:11:58.703112|3.59|american history +70|430|65621|4294967344|11.43|42.13|true|quinn falkner|2013-03-01 09:11:58.703308|79.08|joggying +123|392|65761|4294967545|22.33|2.49|false|bob brown|2013-03-01 09:11:58.703276|34.85|undecided +8|377|65607|4294967342|63.55|20.34|false|luke zipper|2013-03-01 09:11:58.703300|40.88|joggying +25|405|65595|4294967383|46.17|3.17|false|bob xylophone|2013-03-01 09:11:58.703183|80.95|kindergarten +9|279|65545|4294967338|84.92|0.43|false|priscilla thompson|2013-03-01 09:11:58.703091|54.05|industrial engineering +-2|346|65627|4294967357|59.20|45.96|false|sarah white|2013-03-01 09:11:58.703170|82.20|forestry +0|367|65667|4294967331|92.74|9.04|false|rachel thompson|2013-03-01 09:11:58.703299|22.60|yard duty +39|310|65563|4294967531|99.29|21.99|false|yuri garcia|2013-03-01 09:11:58.703129|27.87|zync studies +58|389|65560|4294967520|44.42|5.91|true|david ellison|2013-03-01 09:11:58.703245|2.06|zync studies +78|269|65568|4294967357|11.56|48.27|true|irene nixon|2013-03-01 09:11:58.703275|89.40|mathematics +117|488|65668|4294967509|34.12|6.42|true|bob young|2013-03-01 09:11:58.703319|10.59|industrial engineering +89|388|65568|4294967510|51.02|10.95|false|sarah zipper|2013-03-01 09:11:58.703124|27.34|geology +51|350|65537|4294967448|55.58|12.03|true|holly laertes|2013-03-01 09:11:58.703161|94.52|american history +58|367|65603|4294967366|71.34|47.43|true|ethan nixon|2013-03-01 09:11:58.703283|22.77|linguistics +5|438|65730|4294967374|95.53|30.67|true|rachel falkner|2013-03-01 09:11:58.703248|68.01|nap time +37|430|65774|4294967472|81.26|15.05|true|ethan thompson|2013-03-01 09:11:58.703178|33.63|philosophy +38|408|65603|4294967503|73.83|30.17|false|mike miller|2013-03-01 09:11:58.703322|82.07|religion +110|340|65545|4294967337|0.01|21.13|true|yuri thompson|2013-03-01 09:11:58.703097|22.68|religion +120|404|65638|4294967441|20.41|22.56|false|holly quirinius|2013-03-01 09:11:58.703158|60.85|industrial engineering +91|310|65709|4294967399|65.97|32.53|true|mike carson|2013-03-01 09:11:58.703265|47.77|forestry +113|314|65690|4294967326|75.01|22.49|false|katie carson|2013-03-01 09:11:58.703103|72.22|american history +17|331|65734|4294967345|46.88|44.27|true|alice garcia|2013-03-01 09:11:58.703257|44.90|industrial engineering +12|280|65745|4294967343|13.39|11.97|false|fred brown|2013-03-01 09:11:58.703268|76.40|kindergarten +97|315|65555|4294967463|65.84|14.63|false|gabriella thompson|2013-03-01 09:11:58.703100|72.23|nap time +12|466|65574|4294967426|43.00|23.60|false|gabriella ellison|2013-03-01 09:11:58.703140|18.96|zync studies +71|372|65655|4294967416|95.65|19.58|false|xavier brown|2013-03-01 09:11:58.703120|16.91|topology +6|314|65694|4294967309|72.89|27.67|false|ulysses garcia|2013-03-01 09:11:58.703212|5.09|topology +81|466|65627|4294967528|83.65|42.06|false|sarah hernandez|2013-03-01 09:11:58.703157|97.75|mathematics +40|316|65647|4294967348|45.58|22.92|false|irene ovid|2013-03-01 09:11:58.703288|8.48|linguistics +80|365|65625|4294967500|51.14|41.26|true|zach johnson|2013-03-01 09:11:58.703283|24.62|nap time +18|268|65720|4294967463|24.08|38.27|false|calvin ichabod|2013-03-01 09:11:58.703164|32.25|industrial engineering +123|417|65569|4294967434|64.82|43.98|false|gabriella davidson|2013-03-01 09:11:58.703226|99.94|opthamology +82|370|65667|4294967416|54.10|1.84|false|priscilla thompson|2013-03-01 09:11:58.703072|49.82|linguistics +100|422|65543|4294967463|3.58|14.19|false|rachel zipper|2013-03-01 09:11:58.703198|69.55|geology +16|358|65627|4294967387|70.54|24.95|true|sarah johnson|2013-03-01 09:11:58.703266|11.20|american history +57|269|65586|4294967442|40.84|38.53|true|ulysses robinson|2013-03-01 09:11:58.703311|48.37|industrial engineering +25|353|65695|4294967313|88.01|32.85|true|nick miller|2013-03-01 09:11:58.703210|4.57|biology +49|413|65764|4294967531|32.36|13.47|false|priscilla zipper|2013-03-01 09:11:58.703177|61.81|history +10|425|65557|4294967424|8.06|49.17|true|irene quirinius|2013-03-01 09:11:58.703293|14.26|chemistry +80|426|65745|4294967532|40.15|35.94|false|sarah hernandez|2013-03-01 09:11:58.703287|56.02|yard duty +52|489|65564|4294967370|90.00|9.70|false|mike carson|2013-03-01 09:11:58.703201|14.49|xylophone band +103|442|65756|4294967399|36.42|32.01|true|bob xylophone|2013-03-01 09:11:58.703126|39.12|quiet hour +60|475|65564|4294967398|68.90|40.81|true|oscar allen|2013-03-01 09:11:58.703230|83.05|wind surfing +18|462|65543|4294967359|16.10|7.96|false|xavier polk|2013-03-01 09:11:58.703169|21.63|xylophone band +98|388|65689|4294967476|62.64|40.54|false|zach steinbeck|2013-03-01 09:11:58.703151|11.20|education +122|476|65724|4294967387|3.43|25.90|true|holly carson|2013-03-01 09:11:58.703236|26.98|debate +30|452|65686|4294967409|89.79|44.88|false|oscar quirinius|2013-03-01 09:11:58.703207|24.71|kindergarten +67|356|65765|4294967481|66.58|39.83|false|david ovid|2013-03-01 09:11:58.703115|42.84|zync studies +77|324|65682|4294967307|22.46|48.37|true|mike ellison|2013-03-01 09:11:58.703192|16.70|joggying +59|465|65708|4294967410|54.80|30.20|false|yuri falkner|2013-03-01 09:11:58.703213|80.44|debate +33|461|65703|4294967538|22.16|27.44|false|quinn laertes|2013-03-01 09:11:58.703122|25.31|industrial engineering +44|365|65769|4294967408|44.97|43.57|true|xavier king|2013-03-01 09:11:58.703087|2.77|yard duty +105|282|65773|4294967531|52.12|35.66|true|katie young|2013-03-01 09:11:58.703129|68.88|philosophy +56|319|65689|4294967542|28.67|29.94|true|wendy hernandez|2013-03-01 09:11:58.703073|72.47|zync studies +79|444|65629|4294967417|27.89|3.73|true|wendy underhill|2013-03-01 09:11:58.703317|77.18|values clariffication +23|449|65769|4294967332|86.93|23.67|false|jessica nixon|2013-03-01 09:11:58.703205|57.86|american history +86|410|65558|4294967369|56.78|32.39|true|yuri polk|2013-03-01 09:11:58.703241|30.96|debate +70|452|65671|4294967351|87.97|33.98|true|oscar quirinius|2013-03-01 09:11:58.703266|19.34|debate +54|412|65736|4294967472|28.91|14.10|false|rachel thompson|2013-03-01 09:11:58.703183|41.44|education +117|266|65572|4294967444|2.78|48.98|false|holly white|2013-03-01 09:11:58.703320|79.53|opthamology +114|437|65729|4294967492|55.34|9.42|false|bob laertes|2013-03-01 09:11:58.703112|68.82|yard duty +89|328|65565|4294967381|65.69|11.49|false|gabriella brown|2013-03-01 09:11:58.703159|58.15|quiet hour +34|364|65766|4294967307|4.13|35.89|false|wendy young|2013-03-01 09:11:58.703299|92.35|mathematics +20|262|65640|4294967460|9.92|47.04|false|tom king|2013-03-01 09:11:58.703225|27.64|topology +120|265|65651|4294967378|62.55|1.56|true|sarah johnson|2013-03-01 09:11:58.703129|5.18|xylophone band +94|462|65775|4294967543|84.42|13.72|false|priscilla garcia|2013-03-01 09:11:58.703207|16.79|study skills +51|352|65584|4294967515|51.86|31.86|true|irene falkner|2013-03-01 09:11:58.703228|14.88|wind surfing +112|383|65654|4294967312|5.01|49.41|true|fred polk|2013-03-01 09:11:58.703175|58.31|history +15|471|65617|4294967531|42.72|47.01|true|oscar ellison|2013-03-01 09:11:58.703077|84.70|education +103|316|65616|4294967459|12.34|15.86|true|wendy robinson|2013-03-01 09:11:58.703324|41.54|values clariffication +42|456|65752|4294967428|61.28|48.09|false|priscilla ellison|2013-03-01 09:11:58.703266|52.05|religion +66|410|65787|4294967400|0.52|29.43|true|david hernandez|2013-03-01 09:11:58.703200|93.79|debate +93|437|65773|4294967317|40.98|12.76|false|oscar davidson|2013-03-01 09:11:58.703126|8.90|debate +53|489|65708|4294967472|42.66|26.45|false|ulysses johnson|2013-03-01 09:11:58.703138|72.55|education +47|499|65568|4294967537|61.25|37.16|false|irene quirinius|2013-03-01 09:11:58.703170|23.47|nap time +27|354|65760|4294967432|75.17|35.88|true|mike young|2013-03-01 09:11:58.703305|7.38|geology +73|389|65544|4294967431|22.89|37.48|false|calvin thompson|2013-03-01 09:11:58.703299|24.97|philosophy +27|489|65716|4294967315|8.75|24.50|false|jessica thompson|2013-03-01 09:11:58.703202|82.43|wind surfing +40|419|65616|4294967422|39.70|44.56|true|oscar nixon|2013-03-01 09:11:58.703216|61.56|chemistry +106|425|65725|4294967509|12.99|39.02|false|oscar ovid|2013-03-01 09:11:58.703167|0.80|history +75|341|65654|4294967367|78.35|36.59|false|quinn underhill|2013-03-01 09:11:58.703254|77.33|yard duty +11|357|65770|4294967417|71.97|46.47|false|luke king|2013-03-01 09:11:58.703231|18.21|mathematics +49|472|65587|4294967543|52.19|8.89|false|sarah hernandez|2013-03-01 09:11:58.703290|37.03|xylophone band +0|431|65700|4294967529|30.43|17.43|true|tom robinson|2013-03-01 09:11:58.703317|76.99|geology +31|295|65710|4294967542|70.55|25.96|true|zach white|2013-03-01 09:11:58.703213|56.04|chemistry +21|368|65609|4294967415|44.39|8.39|true|xavier ovid|2013-03-01 09:11:58.703211|11.87|topology +-3|284|65566|4294967400|62.81|39.10|false|jessica white|2013-03-01 09:11:58.703199|69.98|opthamology +43|294|65709|4294967420|16.41|27.09|true|xavier zipper|2013-03-01 09:11:58.703188|35.95|debate +5|457|65710|4294967338|35.92|33.94|false|xavier underhill|2013-03-01 09:11:58.703121|1.01|american history +88|314|65562|4294967479|33.36|10.32|false|priscilla king|2013-03-01 09:11:58.703082|49.70|american history +121|394|65774|4294967502|85.81|44.14|true|yuri thompson|2013-03-01 09:11:58.703104|48.19|nap time +60|383|65760|4294967473|34.61|6.32|true|tom polk|2013-03-01 09:11:58.703282|46.41|joggying +49|309|65578|4294967306|65.85|33.28|false|ethan van buren|2013-03-01 09:11:58.703325|45.98|topology +114|382|65581|4294967414|20.30|33.27|true|rachel miller|2013-03-01 09:11:58.703126|15.70|quiet hour +55|311|65659|4294967543|21.44|3.27|false|ulysses white|2013-03-01 09:11:58.703289|98.99|study skills +90|415|65644|4294967319|62.83|32.79|false|mike garcia|2013-03-01 09:11:58.703119|71.05|industrial engineering +51|308|65609|4294967441|26.13|37.43|false|xavier king|2013-03-01 09:11:58.703190|11.24|linguistics +56|402|65560|4294967475|10.49|0.99|true|priscilla quirinius|2013-03-01 09:11:58.703186|49.28|opthamology +60|446|65657|4294967320|4.19|27.56|true|priscilla johnson|2013-03-01 09:11:58.703319|86.38|chemistry +43|349|65618|4294967296|68.93|11.30|true|katie quirinius|2013-03-01 09:11:58.703123|47.97|joggying +88|292|65625|4294967339|94.91|18.00|true|gabriella carson|2013-03-01 09:11:58.703131|79.11|opthamology +123|386|65577|4294967524|16.30|41.56|false|alice quirinius|2013-03-01 09:11:58.703240|82.48|american history +114|282|65712|4294967523|6.92|25.65|true|quinn white|2013-03-01 09:11:58.703263|29.64|debate +118|367|65763|4294967304|47.97|33.58|false|yuri xylophone|2013-03-01 09:11:58.703254|56.21|kindergarten +10|295|65595|4294967435|25.07|12.72|false|yuri white|2013-03-01 09:11:58.703279|11.15|quiet hour +88|265|65561|4294967317|60.71|46.33|true|ulysses young|2013-03-01 09:11:58.703315|46.95|topology +28|260|65558|4294967346|69.17|49.39|true|oscar zipper|2013-03-01 09:11:58.703316|65.08|undecided +14|432|65732|4294967522|70.22|13.81|false|irene van buren|2013-03-01 09:11:58.703084|18.02|history +90|292|65657|4294967538|61.81|5.77|true|sarah ichabod|2013-03-01 09:11:58.703105|91.11|xylophone band +56|352|65752|4294967485|13.37|36.07|true|calvin van buren|2013-03-01 09:11:58.703112|65.90|history +82|381|65588|4294967420|43.04|7.38|true|calvin white|2013-03-01 09:11:58.703102|97.43|nap time +99|457|65617|4294967506|14.52|16.05|false|priscilla brown|2013-03-01 09:11:58.703176|15.55|kindergarten +10|333|65600|4294967367|71.39|24.73|false|victor ovid|2013-03-01 09:11:58.703280|2.52|quiet hour +26|301|65753|4294967485|96.24|6.82|false|sarah brown|2013-03-01 09:11:58.703276|71.33|industrial engineering +21|340|65737|4294967486|74.97|47.10|true|holly underhill|2013-03-01 09:11:58.703245|12.18|opthamology +81|454|65686|4294967388|86.77|2.26|true|ethan allen|2013-03-01 09:11:58.703105|24.87|history +10|292|65711|4294967474|61.83|11.12|true|katie ellison|2013-03-01 09:11:58.703218|75.00|study skills +32|277|65735|4294967358|21.03|40.48|true|nick allen|2013-03-01 09:11:58.703168|11.63|xylophone band +82|435|65725|4294967429|66.83|18.66|true|yuri polk|2013-03-01 09:11:58.703198|45.60|biology +57|278|65590|4294967322|48.00|42.28|false|david carson|2013-03-01 09:11:58.703290|94.83|biology +109|295|65595|4294967459|1.44|33.71|false|gabriella davidson|2013-03-01 09:11:58.703144|58.28|values clariffication +60|431|65680|4294967419|45.33|27.48|false|tom miller|2013-03-01 09:11:58.703129|39.98|history +26|266|65713|4294967354|84.62|45.68|false|quinn steinbeck|2013-03-01 09:11:58.703189|33.66|debate +78|497|65778|4294967485|1.74|39.24|true|calvin ichabod|2013-03-01 09:11:58.703279|23.67|nap time +15|415|65560|4294967408|66.04|26.59|true|bob robinson|2013-03-01 09:11:58.703214|7.73|religion +38|476|65563|4294967452|59.84|11.57|false|calvin steinbeck|2013-03-01 09:11:58.703109|98.60|values clariffication +38|494|65740|4294967531|8.17|41.54|false|calvin thompson|2013-03-01 09:11:58.703084|86.58|zync studies +94|310|65711|4294967464|15.24|36.79|false|mike johnson|2013-03-01 09:11:58.703275|11.06|zync studies +51|328|65762|4294967536|70.77|46.20|true|luke ichabod|2013-03-01 09:11:58.703300|90.72|mathematics +67|465|65581|4294967343|43.54|29.15|false|wendy thompson|2013-03-01 09:11:58.703233|39.20|nap time +50|447|65694|4294967338|40.25|21.49|true|holly white|2013-03-01 09:11:58.703109|60.35|joggying +54|315|65574|4294967469|74.15|36.82|false|fred garcia|2013-03-01 09:11:58.703211|88.24|biology +18|435|65784|4294967502|67.24|4.27|true|luke nixon|2013-03-01 09:11:58.703257|47.45|biology +104|324|65601|4294967531|49.00|28.17|true|mike underhill|2013-03-01 09:11:58.703177|44.81|topology +-3|286|65573|4294967493|18.27|23.71|false|zach young|2013-03-01 09:11:58.703191|21.69|kindergarten +53|273|65706|4294967521|95.36|18.85|false|calvin ellison|2013-03-01 09:11:58.703215|54.10|study skills +98|424|65608|4294967397|12.02|45.92|true|oscar young|2013-03-01 09:11:58.703316|65.43|study skills +89|400|65626|4294967492|5.11|13.95|true|zach zipper|2013-03-01 09:11:58.703227|43.25|linguistics +71|407|65624|4294967386|97.20|14.88|false|zach laertes|2013-03-01 09:11:58.703171|28.03|biology +105|318|65693|4294967422|47.44|14.80|false|irene laertes|2013-03-01 09:11:58.703228|69.85|industrial engineering +12|507|65657|4294967403|37.19|4.76|true|oscar quirinius|2013-03-01 09:11:58.703268|51.58|kindergarten +53|456|65566|4294967437|48.95|31.95|false|oscar robinson|2013-03-01 09:11:58.703260|27.49|undecided +77|257|65732|4294967339|96.85|18.51|true|nick zipper|2013-03-01 09:11:58.703086|68.97|forestry +16|259|65674|4294967377|53.44|5.42|false|katie zipper|2013-03-01 09:11:58.703190|90.09|nap time +87|367|65717|4294967491|79.00|29.46|false|gabriella steinbeck|2013-03-01 09:11:58.703289|10.86|biology +113|257|65720|4294967437|24.40|10.00|true|mike underhill|2013-03-01 09:11:58.703153|23.05|industrial engineering +104|370|65562|4294967457|31.94|16.31|false|quinn allen|2013-03-01 09:11:58.703071|40.60|debate +1|392|65539|4294967356|75.60|16.98|false|mike steinbeck|2013-03-01 09:11:58.703215|32.95|biology +4|446|65664|4294967340|45.20|38.39|false|zach zipper|2013-03-01 09:11:58.703140|65.50|education +63|342|65608|4294967517|9.12|44.84|true|calvin white|2013-03-01 09:11:58.703305|67.80|joggying +2|472|65728|4294967387|68.34|36.06|true|oscar falkner|2013-03-01 09:11:58.703249|49.31|industrial engineering +60|424|65705|4294967464|6.62|9.55|true|yuri steinbeck|2013-03-01 09:11:58.703246|17.01|forestry +91|471|65624|4294967469|27.25|21.93|false|gabriella allen|2013-03-01 09:11:58.703236|73.70|philosophy +104|360|65776|4294967425|57.41|30.49|true|luke quirinius|2013-03-01 09:11:58.703257|94.30|geology +76|389|65729|4294967497|73.82|1.18|false|ulysses underhill|2013-03-01 09:11:58.703278|66.52|linguistics +70|290|65547|4294967301|16.94|10.48|false|victor zipper|2013-03-01 09:11:58.703296|17.89|xylophone band +107|477|65559|4294967546|88.00|24.00|true|rachel hernandez|2013-03-01 09:11:58.703181|31.96|wind surfing +25|346|65727|4294967398|70.75|38.38|true|yuri davidson|2013-03-01 09:11:58.703119|29.93|zync studies +54|326|65707|4294967468|56.70|30.47|true|jessica thompson|2013-03-01 09:11:58.703072|80.72|xylophone band +87|374|65600|4294967484|32.94|9.31|true|holly white|2013-03-01 09:11:58.703095|22.03|american history +60|399|65609|4294967338|80.82|47.45|true|victor nixon|2013-03-01 09:11:58.703156|69.84|quiet hour +104|400|65790|4294967521|33.36|21.85|false|jessica laertes|2013-03-01 09:11:58.703110|55.10|zync studies +60|476|65592|4294967542|10.68|38.25|true|bob ovid|2013-03-01 09:11:58.703220|18.49|wind surfing +82|425|65628|4294967319|45.33|42.97|false|gabriella thompson|2013-03-01 09:11:58.703170|80.55|xylophone band +104|268|65775|4294967412|81.03|39.90|true|yuri hernandez|2013-03-01 09:11:58.703134|71.24|nap time +103|438|65712|4294967367|74.74|29.66|false|quinn davidson|2013-03-01 09:11:58.703211|84.69|industrial engineering +81|474|65757|4294967515|9.28|48.26|false|calvin van buren|2013-03-01 09:11:58.703161|30.80|biology +49|440|65592|4294967315|72.64|12.17|false|ulysses robinson|2013-03-01 09:11:58.703223|90.90|undecided +49|299|65639|4294967520|35.43|29.52|false|jessica white|2013-03-01 09:11:58.703138|81.16|xylophone band +62|460|65604|4294967476|46.37|26.35|false|tom robinson|2013-03-01 09:11:58.703251|62.99|philosophy +69|335|65676|4294967336|18.26|46.74|true|jessica polk|2013-03-01 09:11:58.703190|66.23|study skills +43|388|65568|4294967335|29.13|33.04|false|irene robinson|2013-03-01 09:11:58.703291|58.94|education +61|471|65649|4294967309|60.93|42.43|true|rachel zipper|2013-03-01 09:11:58.703210|81.93|industrial engineering +93|318|65725|4294967344|24.64|29.12|false|nick davidson|2013-03-01 09:11:58.703236|87.49|yard duty +43|448|65602|4294967324|19.30|12.85|true|quinn ellison|2013-03-01 09:11:58.703105|77.45|joggying +2|352|65628|4294967395|72.07|39.95|false|gabriella hernandez|2013-03-01 09:11:58.703172|19.31|quiet hour +66|284|65668|4294967422|47.94|46.30|false|zach davidson|2013-03-01 09:11:58.703125|47.91|mathematics +0|327|65765|4294967465|77.19|33.83|false|irene robinson|2013-03-01 09:11:58.703085|1.45|opthamology +81|334|65602|4294967377|63.41|27.00|false|mike hernandez|2013-03-01 09:11:58.703189|32.57|biology +123|435|65722|4294967419|27.62|18.45|true|irene johnson|2013-03-01 09:11:58.703162|16.75|topology +33|492|65771|4294967324|49.52|26.71|true|ulysses xylophone|2013-03-01 09:11:58.703192|12.53|study skills +101|296|65602|4294967435|54.67|15.07|true|calvin quirinius|2013-03-01 09:11:58.703273|20.88|debate +23|279|65546|4294967435|97.32|20.20|false|holly quirinius|2013-03-01 09:11:58.703279|27.76|american history +40|304|65649|4294967454|20.23|9.28|true|david zipper|2013-03-01 09:11:58.703268|76.12|forestry +115|431|65640|4294967504|62.15|15.78|false|wendy ichabod|2013-03-01 09:11:58.703235|6.87|forestry +30|500|65760|4294967517|95.08|47.72|true|tom laertes|2013-03-01 09:11:58.703080|85.83|chemistry +0|294|65624|4294967405|31.24|39.90|false|rachel ovid|2013-03-01 09:11:58.703184|22.66|xylophone band +74|436|65569|4294967353|55.02|37.48|false|bob polk|2013-03-01 09:11:58.703274|13.85|mathematics +56|307|65610|4294967394|28.91|4.88|true|alice white|2013-03-01 09:11:58.703293|53.58|values clariffication +33|496|65580|4294967352|52.84|9.98|true|ethan laertes|2013-03-01 09:11:58.703214|51.43|yard duty +121|382|65762|4294967392|41.45|20.62|true|xavier miller|2013-03-01 09:11:58.703144|45.08|undecided +58|446|65716|4294967538|27.78|32.51|false|calvin garcia|2013-03-01 09:11:58.703280|68.60|values clariffication +-2|406|65582|4294967311|20.94|35.74|false|bob van buren|2013-03-01 09:11:58.703218|25.17|opthamology +69|461|65652|4294967544|89.64|40.36|false|rachel white|2013-03-01 09:11:58.703229|58.80|topology +119|295|65668|4294967356|25.93|9.75|true|calvin white|2013-03-01 09:11:58.703121|36.42|geology +88|263|65635|4294967430|62.15|18.26|false|alice miller|2013-03-01 09:11:58.703125|53.75|yard duty +111|298|65776|4294967415|78.12|26.72|false|wendy underhill|2013-03-01 09:11:58.703086|0.70|xylophone band +12|321|65737|4294967414|89.17|33.54|true|yuri ichabod|2013-03-01 09:11:58.703197|51.14|xylophone band +52|309|65555|4294967421|56.44|11.67|false|xavier carson|2013-03-01 09:11:58.703323|34.37|joggying +38|463|65594|4294967384|98.75|14.17|false|sarah underhill|2013-03-01 09:11:58.703234|59.64|philosophy +75|383|65645|4294967512|14.72|3.75|false|wendy miller|2013-03-01 09:11:58.703243|79.83|chemistry +46|446|65609|4294967354|82.88|22.56|false|jessica zipper|2013-03-01 09:11:58.703252|25.18|forestry +93|309|65563|4294967302|95.51|25.91|false|holly steinbeck|2013-03-01 09:11:58.703077|14.24|industrial engineering +105|448|65730|4294967515|85.59|1.05|true|mike carson|2013-03-01 09:11:58.703147|58.03|zync studies +11|268|65566|4294967476|20.94|17.60|false|xavier miller|2013-03-01 09:11:58.703217|38.17|study skills +120|333|65572|4294967399|98.91|7.42|true|ethan nixon|2013-03-01 09:11:58.703170|33.27|debate +54|380|65567|4294967500|71.54|17.17|true|mike nixon|2013-03-01 09:11:58.703154|35.66|xylophone band +27|399|65723|4294967323|44.74|17.95|false|mike young|2013-03-01 09:11:58.703207|86.86|history +117|297|65726|4294967389|16.22|41.92|true|nick davidson|2013-03-01 09:11:58.703313|33.00|undecided +60|462|65750|4294967513|71.20|2.35|false|katie allen|2013-03-01 09:11:58.703152|76.27|industrial engineering +90|322|65598|4294967529|30.43|18.06|true|gabriella xylophone|2013-03-01 09:11:58.703158|65.77|linguistics +73|447|65543|4294967328|14.45|18.64|true|victor hernandez|2013-03-01 09:11:58.703227|17.68|opthamology +15|322|65645|4294967300|35.11|8.86|true|calvin xylophone|2013-03-01 09:11:58.703127|93.28|kindergarten +44|289|65635|4294967475|70.96|36.88|true|calvin polk|2013-03-01 09:11:58.703248|52.62|zync studies +79|393|65611|4294967528|7.23|27.44|false|alice falkner|2013-03-01 09:11:58.703295|75.78|philosophy +5|288|65595|4294967521|91.73|29.13|false|luke laertes|2013-03-01 09:11:58.703238|33.75|debate +14|397|65554|4294967360|51.62|1.58|true|nick nixon|2013-03-01 09:11:58.703234|52.92|mathematics +12|508|65691|4294967374|47.59|8.10|true|quinn miller|2013-03-01 09:11:58.703215|53.61|philosophy +72|303|65711|4294967368|85.89|30.06|false|rachel quirinius|2013-03-01 09:11:58.703289|32.01|education +42|506|65641|4294967303|24.71|9.66|false|quinn steinbeck|2013-03-01 09:11:58.703077|71.42|study skills +85|412|65764|4294967341|72.87|35.54|false|victor miller|2013-03-01 09:11:58.703113|11.40|linguistics +98|369|65709|4294967509|91.23|44.06|true|katie carson|2013-03-01 09:11:58.703201|20.31|biology +40|438|65706|4294967454|40.78|25.56|false|oscar polk|2013-03-01 09:11:58.703292|28.25|study skills +91|510|65564|4294967481|29.49|37.61|true|ulysses thompson|2013-03-01 09:11:58.703231|2.62|quiet hour +116|398|65633|4294967456|57.96|22.85|true|oscar miller|2013-03-01 09:11:58.703265|30.55|education +75|351|65682|4294967544|53.83|0.81|false|ethan thompson|2013-03-01 09:11:58.703077|2.19|nap time +83|412|65756|4294967483|78.10|17.40|false|ulysses polk|2013-03-01 09:11:58.703255|30.58|biology +75|351|65700|4294967336|31.18|39.95|false|zach allen|2013-03-01 09:11:58.703306|60.99|philosophy +3|466|65566|4294967299|42.80|26.54|false|quinn miller|2013-03-01 09:11:58.703149|1.69|mathematics +94|353|65553|4294967421|91.20|28.04|false|ethan robinson|2013-03-01 09:11:58.703240|82.90|forestry +14|335|65550|4294967486|20.64|24.83|false|katie brown|2013-03-01 09:11:58.703180|31.42|nap time +114|449|65559|4294967461|53.52|16.35|true|xavier allen|2013-03-01 09:11:58.703237|39.70|chemistry +99|423|65579|4294967516|76.93|24.09|false|sarah brown|2013-03-01 09:11:58.703278|41.91|values clariffication +5|498|65644|4294967461|12.77|0.70|false|yuri davidson|2013-03-01 09:11:58.703309|63.14|religion +94|280|65706|4294967319|4.30|43.42|true|nick miller|2013-03-01 09:11:58.703149|34.14|quiet hour +116|464|65717|4294967390|6.96|6.90|false|david miller|2013-03-01 09:11:58.703214|13.35|quiet hour +49|464|65543|4294967318|3.01|12.09|false|sarah white|2013-03-01 09:11:58.703240|3.60|quiet hour +108|453|65780|4294967454|51.48|26.71|false|quinn xylophone|2013-03-01 09:11:58.703177|16.14|forestry +78|393|65657|4294967479|79.60|42.37|true|victor miller|2013-03-01 09:11:58.703119|14.74|quiet hour +38|439|65683|4294967337|49.57|20.36|true|ulysses falkner|2013-03-01 09:11:58.703217|62.01|yard duty +17|307|65614|4294967426|34.98|46.24|true|katie falkner|2013-03-01 09:11:58.703097|6.18|religion +58|422|65554|4294967425|90.72|3.05|true|wendy zipper|2013-03-01 09:11:58.703258|55.83|study skills +82|323|65638|4294967309|36.64|49.01|false|victor thompson|2013-03-01 09:11:58.703306|32.11|chemistry +50|447|65712|4294967524|20.56|1.87|false|quinn robinson|2013-03-01 09:11:58.703130|99.86|debate +110|332|65707|4294967363|15.67|23.70|true|david young|2013-03-01 09:11:58.703096|23.99|wind surfing +114|341|65670|4294967446|10.49|5.35|true|zach steinbeck|2013-03-01 09:11:58.703128|82.98|yard duty +113|296|65664|4294967435|96.85|33.77|false|bob johnson|2013-03-01 09:11:58.703218|83.33|linguistics +14|472|65766|4294967385|70.85|31.23|true|rachel ellison|2013-03-01 09:11:58.703179|36.33|history +18|434|65713|4294967536|59.74|26.99|false|nick xylophone|2013-03-01 09:11:58.703154|89.39|geology +103|291|65770|4294967512|39.91|8.29|true|ulysses miller|2013-03-01 09:11:58.703194|99.50|quiet hour +41|337|65652|4294967499|39.87|21.03|false|quinn ovid|2013-03-01 09:11:58.703102|25.51|chemistry +123|468|65611|4294967476|40.53|5.51|false|calvin nixon|2013-03-01 09:11:58.703218|22.44|education +110|286|65696|4294967494|12.83|11.36|false|bob king|2013-03-01 09:11:58.703071|94.97|geology +106|352|65720|4294967544|28.17|46.94|true|david laertes|2013-03-01 09:11:58.703108|6.19|american history +4|329|65572|4294967440|58.87|14.01|true|ethan polk|2013-03-01 09:11:58.703190|10.67|study skills +4|397|65791|4294967529|57.61|22.12|false|bob nixon|2013-03-01 09:11:58.703312|60.01|zync studies +64|411|65608|4294967334|82.76|29.91|true|luke steinbeck|2013-03-01 09:11:58.703246|3.94|study skills +90|453|65633|4294967371|71.02|41.34|true|fred polk|2013-03-01 09:11:58.703219|24.21|opthamology +113|381|65587|4294967358|62.85|34.36|true|gabriella nixon|2013-03-01 09:11:58.703086|5.05|mathematics +-1|318|65663|4294967465|57.73|41.18|true|sarah king|2013-03-01 09:11:58.703238|76.60|xylophone band +31|491|65664|4294967344|23.61|3.30|false|ethan robinson|2013-03-01 09:11:58.703160|2.54|undecided +120|376|65593|4294967419|54.87|19.34|false|irene miller|2013-03-01 09:11:58.703148|5.33|zync studies +65|469|65690|4294967430|69.05|39.37|false|gabriella nixon|2013-03-01 09:11:58.703219|89.53|opthamology +50|472|65769|4294967483|49.51|18.27|false|david young|2013-03-01 09:11:58.703216|93.77|quiet hour +107|310|65769|4294967372|78.62|3.43|false|irene quirinius|2013-03-01 09:11:58.703142|53.73|biology +120|341|65595|4294967508|5.76|35.25|false|luke ovid|2013-03-01 09:11:58.703164|21.42|joggying +0|482|65684|4294967386|53.95|42.55|true|tom young|2013-03-01 09:11:58.703150|62.39|study skills +108|270|65753|4294967515|75.10|11.34|false|xavier underhill|2013-03-01 09:11:58.703130|41.52|religion +97|402|65783|4294967510|12.68|32.92|false|mike carson|2013-03-01 09:11:58.703249|35.76|history +88|404|65784|4294967429|55.14|44.08|true|sarah king|2013-03-01 09:11:58.703319|48.24|american history +73|288|65680|4294967414|56.27|25.57|false|sarah steinbeck|2013-03-01 09:11:58.703120|91.11|industrial engineering +116|339|65728|4294967411|40.44|28.02|true|david allen|2013-03-01 09:11:58.703168|96.53|wind surfing +15|256|65651|4294967395|90.32|44.66|true|victor thompson|2013-03-01 09:11:58.703090|75.03|quiet hour +74|505|65576|4294967354|59.60|4.04|true|tom nixon|2013-03-01 09:11:58.703087|83.86|biology +35|311|65762|4294967545|50.18|42.13|false|david davidson|2013-03-01 09:11:58.703276|84.60|quiet hour +91|305|65770|4294967497|15.10|41.94|false|yuri steinbeck|2013-03-01 09:11:58.703277|47.03|education +45|483|65772|4294967340|80.09|8.32|true|priscilla polk|2013-03-01 09:11:58.703217|6.79|wind surfing +115|299|65711|4294967514|36.81|42.03|false|yuri carson|2013-03-01 09:11:58.703259|66.93|zync studies +48|303|65636|4294967534|32.22|26.99|true|priscilla miller|2013-03-01 09:11:58.703224|82.34|values clariffication +75|384|65667|4294967432|25.96|19.37|true|quinn ellison|2013-03-01 09:11:58.703216|12.03|wind surfing +82|280|65747|4294967471|91.04|43.18|true|sarah underhill|2013-03-01 09:11:58.703092|23.07|mathematics +7|473|65565|4294967387|4.52|29.85|true|alice thompson|2013-03-01 09:11:58.703215|84.95|yard duty +3|260|65609|4294967493|90.22|35.52|false|sarah laertes|2013-03-01 09:11:58.703135|81.91|zync studies +49|391|65631|4294967475|94.67|9.84|true|bob brown|2013-03-01 09:11:58.703282|67.54|history +86|293|65592|4294967442|72.42|49.05|true|nick falkner|2013-03-01 09:11:58.703269|91.51|joggying +80|459|65769|4294967530|34.99|30.77|true|priscilla young|2013-03-01 09:11:58.703148|87.96|geology +42|293|65742|4294967298|47.57|2.08|false|ethan ovid|2013-03-01 09:11:58.703300|51.45|undecided +21|256|65613|4294967386|93.29|16.98|true|nick zipper|2013-03-01 09:11:58.703276|57.61|history +5|309|65623|4294967369|32.17|29.64|true|jessica king|2013-03-01 09:11:58.703115|54.28|topology +91|263|65778|4294967521|61.81|39.30|false|bob young|2013-03-01 09:11:58.703214|47.63|geology +-1|427|65672|4294967484|1.27|17.63|false|victor davidson|2013-03-01 09:11:58.703228|98.52|opthamology +46|396|65595|4294967491|17.76|28.23|false|yuri ichabod|2013-03-01 09:11:58.703234|92.94|mathematics +61|472|65658|4294967319|55.29|15.30|true|sarah hernandez|2013-03-01 09:11:58.703095|40.93|undecided +33|423|65556|4294967365|1.22|48.72|false|tom ellison|2013-03-01 09:11:58.703156|27.26|values clariffication +8|353|65665|4294967461|96.99|39.82|false|luke miller|2013-03-01 09:11:58.703202|91.70|philosophy +95|277|65652|4294967490|3.92|44.73|false|sarah thompson|2013-03-01 09:11:58.703106|64.71|undecided +109|420|65751|4294967401|27.78|4.26|true|jessica allen|2013-03-01 09:11:58.703086|54.29|nap time +41|273|65734|4294967501|75.08|44.16|true|wendy king|2013-03-01 09:11:58.703206|52.66|debate +62|494|65629|4294967483|9.69|11.01|true|bob young|2013-03-01 09:11:58.703101|56.74|zync studies +97|285|65721|4294967411|35.48|15.02|true|xavier king|2013-03-01 09:11:58.703248|26.44|linguistics +36|335|65653|4294967461|97.14|4.50|false|luke falkner|2013-03-01 09:11:58.703310|13.25|american history +47|412|65564|4294967461|47.66|31.02|true|david king|2013-03-01 09:11:58.703306|99.41|zync studies +71|441|65600|4294967506|22.95|22.38|false|ethan van buren|2013-03-01 09:11:58.703089|93.31|topology +122|265|65546|4294967465|18.80|38.79|true|bob xylophone|2013-03-01 09:11:58.703291|21.55|mathematics +22|462|65589|4294967519|25.24|20.25|true|irene steinbeck|2013-03-01 09:11:58.703313|61.59|forestry +21|478|65704|4294967405|13.19|21.21|true|ethan underhill|2013-03-01 09:11:58.703086|36.41|study skills +52|335|65575|4294967458|25.53|24.72|false|sarah miller|2013-03-01 09:11:58.703142|84.26|study skills +90|473|65616|4294967469|23.81|19.35|true|tom young|2013-03-01 09:11:58.703178|59.41|debate +103|394|65679|4294967332|14.37|40.27|true|ethan young|2013-03-01 09:11:58.703158|76.64|wind surfing +105|309|65537|4294967336|96.42|12.22|true|nick carson|2013-03-01 09:11:58.703115|91.35|philosophy +5|361|65772|4294967300|68.42|48.91|true|nick king|2013-03-01 09:11:58.703324|93.58|debate +43|485|65610|4294967330|16.02|6.16|true|nick nixon|2013-03-01 09:11:58.703279|67.60|joggying +104|459|65710|4294967354|12.39|47.05|true|david van buren|2013-03-01 09:11:58.703151|36.93|joggying +4|310|65539|4294967338|53.33|6.44|false|david ellison|2013-03-01 09:11:58.703081|27.99|kindergarten +99|479|65591|4294967428|50.82|11.80|true|sarah robinson|2013-03-01 09:11:58.703198|74.64|mathematics +63|401|65634|4294967353|28.86|30.95|true|yuri nixon|2013-03-01 09:11:58.703164|73.09|topology +97|473|65757|4294967309|27.83|28.76|true|priscilla hernandez|2013-03-01 09:11:58.703316|32.72|study skills +75|289|65785|4294967326|83.89|30.69|true|bob ichabod|2013-03-01 09:11:58.703162|50.60|quiet hour +120|289|65790|4294967424|28.31|45.42|false|gabriella quirinius|2013-03-01 09:11:58.703172|35.29|biology +29|260|65632|4294967421|51.23|3.76|false|sarah steinbeck|2013-03-01 09:11:58.703245|90.08|religion +68|379|65673|4294967334|10.82|38.50|false|zach allen|2013-03-01 09:11:58.703174|28.40|mathematics +93|389|65618|4294967490|94.23|28.45|true|fred ovid|2013-03-01 09:11:58.703112|75.97|study skills +17|507|65632|4294967403|61.71|0.31|false|xavier laertes|2013-03-01 09:11:58.703237|31.04|mathematics +20|352|65542|4294967521|29.76|47.62|false|priscilla laertes|2013-03-01 09:11:58.703080|17.31|study skills +51|462|65606|4294967530|80.59|7.36|true|katie laertes|2013-03-01 09:11:58.703166|10.78|zync studies +105|422|65683|4294967550|70.18|9.74|true|jessica ovid|2013-03-01 09:11:58.703116|37.84|nap time +78|458|65537|4294967540|49.82|35.81|false|victor xylophone|2013-03-01 09:11:58.703281|83.21|undecided +104|259|65567|4294967520|10.43|12.97|false|ulysses falkner|2013-03-01 09:11:58.703270|72.39|quiet hour +104|423|65679|4294967413|5.63|30.91|true|priscilla ichabod|2013-03-01 09:11:58.703280|82.21|religion +74|352|65687|4294967384|49.32|6.10|false|katie laertes|2013-03-01 09:11:58.703152|14.46|yard duty +47|359|65584|4294967339|14.51|23.12|true|tom allen|2013-03-01 09:11:58.703138|43.08|mathematics +109|463|65666|4294967512|69.08|25.69|false|ulysses ovid|2013-03-01 09:11:58.703318|16.15|biology +50|496|65595|4294967374|67.08|10.81|true|xavier white|2013-03-01 09:11:58.703170|82.61|nap time +85|398|65636|4294967550|3.68|40.61|true|rachel polk|2013-03-01 09:11:58.703229|94.03|values clariffication +22|354|65655|4294967518|31.56|49.06|false|sarah xylophone|2013-03-01 09:11:58.703113|48.88|values clariffication +106|387|65558|4294967523|5.54|34.77|false|mike miller|2013-03-01 09:11:58.703273|44.49|yard duty +87|432|65759|4294967386|61.47|45.91|false|calvin ichabod|2013-03-01 09:11:58.703283|49.19|wind surfing +55|312|65560|4294967528|70.13|35.47|true|katie garcia|2013-03-01 09:11:58.703151|18.69|mathematics +62|346|65641|4294967495|25.43|21.23|false|jessica brown|2013-03-01 09:11:58.703241|46.96|wind surfing +90|410|65759|4294967529|0.87|38.49|false|ulysses johnson|2013-03-01 09:11:58.703202|50.59|topology +64|446|65672|4294967476|23.51|29.70|false|gabriella garcia|2013-03-01 09:11:58.703246|48.13|opthamology +35|265|65718|4294967492|90.65|42.09|false|yuri nixon|2013-03-01 09:11:58.703304|71.94|industrial engineering +8|399|65725|4294967373|71.54|37.54|true|ulysses ichabod|2013-03-01 09:11:58.703193|73.54|xylophone band +38|329|65678|4294967463|62.42|30.27|false|tom polk|2013-03-01 09:11:58.703319|85.26|history +15|320|65689|4294967387|74.87|23.35|true|nick carson|2013-03-01 09:11:58.703272|36.96|linguistics +78|462|65758|4294967461|49.58|41.04|false|calvin robinson|2013-03-01 09:11:58.703179|0.01|american history +100|498|65574|4294967526|74.24|45.81|false|wendy nixon|2013-03-01 09:11:58.703258|18.77|xylophone band +47|391|65583|4294967518|6.25|17.81|false|irene hernandez|2013-03-01 09:11:58.703292|68.02|education +66|375|65713|4294967518|29.72|36.34|false|oscar ovid|2013-03-01 09:11:58.703180|41.71|xylophone band +82|266|65718|4294967491|19.86|22.03|true|victor young|2013-03-01 09:11:58.703094|82.16|zync studies +97|344|65625|4294967404|13.95|31.43|true|yuri ellison|2013-03-01 09:11:58.703324|19.30|undecided +120|309|65770|4294967540|51.52|31.88|true|katie davidson|2013-03-01 09:11:58.703162|26.79|education +57|393|65751|4294967406|12.99|8.89|false|priscilla brown|2013-03-01 09:11:58.703137|57.88|nap time +49|471|65664|4294967466|55.53|16.14|false|gabriella robinson|2013-03-01 09:11:58.703153|90.37|linguistics +0|399|65597|4294967516|81.74|33.76|true|holly quirinius|2013-03-01 09:11:58.703228|51.02|industrial engineering +15|477|65770|4294967470|26.57|3.37|true|zach miller|2013-03-01 09:11:58.703271|68.80|topology +8|301|65644|4294967313|72.19|3.01|false|ulysses ichabod|2013-03-01 09:11:58.703134|9.11|kindergarten +72|415|65719|4294967470|92.42|15.49|false|priscilla carson|2013-03-01 09:11:58.703234|3.51|undecided +17|384|65605|4294967397|44.80|4.27|true|rachel johnson|2013-03-01 09:11:58.703314|18.49|opthamology +82|404|65772|4294967509|11.08|49.86|false|katie allen|2013-03-01 09:11:58.703175|10.59|study skills +45|395|65666|4294967493|88.57|21.64|false|david underhill|2013-03-01 09:11:58.703138|22.59|xylophone band +54|360|65706|4294967442|44.30|27.63|true|katie ovid|2013-03-01 09:11:58.703285|66.24|opthamology +106|406|65617|4294967485|11.05|32.03|false|rachel brown|2013-03-01 09:11:58.703081|62.32|study skills +84|498|65787|4294967528|64.19|25.34|true|wendy hernandez|2013-03-01 09:11:58.703216|19.71|xylophone band +81|413|65712|4294967512|18.56|37.58|false|david ellison|2013-03-01 09:11:58.703106|5.21|topology +58|449|65767|4294967446|6.53|24.72|false|fred laertes|2013-03-01 09:11:58.703259|7.97|xylophone band +8|277|65711|4294967312|86.37|24.70|true|yuri nixon|2013-03-01 09:11:58.703264|31.27|study skills +80|325|65636|4294967347|12.93|49.46|false|calvin polk|2013-03-01 09:11:58.703238|45.86|biology +112|486|65655|4294967507|21.33|26.59|false|wendy white|2013-03-01 09:11:58.703126|29.36|industrial engineering +94|393|65776|4294967418|4.28|9.90|false|yuri xylophone|2013-03-01 09:11:58.703091|61.99|industrial engineering +48|291|65558|4294967310|42.71|45.48|false|quinn quirinius|2013-03-01 09:11:58.703233|90.31|wind surfing +50|333|65771|4294967523|0.78|44.80|false|sarah thompson|2013-03-01 09:11:58.703257|44.39|philosophy +63|344|65745|4294967461|53.34|11.56|false|mike ovid|2013-03-01 09:11:58.703274|62.44|undecided +38|280|65557|4294967327|47.48|35.87|false|zach zipper|2013-03-01 09:11:58.703229|50.04|nap time +81|415|65703|4294967538|40.49|19.27|true|david laertes|2013-03-01 09:11:58.703281|61.33|yard duty +83|322|65725|4294967544|20.85|23.56|true|ethan ellison|2013-03-01 09:11:58.703271|19.58|quiet hour +4|431|65666|4294967548|35.65|36.95|false|zach miller|2013-03-01 09:11:58.703323|99.06|study skills +80|397|65635|4294967505|79.65|23.07|true|ethan carson|2013-03-01 09:11:58.703166|22.70|debate +11|283|65622|4294967414|73.65|33.44|false|jessica ellison|2013-03-01 09:11:58.703270|17.79|biology +65|489|65719|4294967339|83.28|27.26|false|yuri laertes|2013-03-01 09:11:58.703299|40.73|religion +105|426|65594|4294967535|25.03|5.48|false|nick hernandez|2013-03-01 09:11:58.703281|79.84|quiet hour +53|320|65789|4294967392|32.58|37.48|false|holly underhill|2013-03-01 09:11:58.703204|70.52|topology +77|380|65611|4294967315|30.66|21.43|true|calvin zipper|2013-03-01 09:11:58.703278|79.93|nap time +105|502|65780|4294967337|91.56|35.76|true|katie garcia|2013-03-01 09:11:58.703263|8.63|study skills +55|469|65713|4294967413|44.54|21.34|false|ulysses zipper|2013-03-01 09:11:58.703243|5.90|joggying +6|417|65642|4294967452|24.15|0.47|false|bob carson|2013-03-01 09:11:58.703219|46.47|undecided +68|302|65562|4294967310|89.95|29.80|false|yuri thompson|2013-03-01 09:11:58.703197|82.02|linguistics +56|265|65731|4294967475|27.32|24.18|false|yuri miller|2013-03-01 09:11:58.703182|75.18|values clariffication +89|473|65593|4294967356|54.54|46.69|true|nick white|2013-03-01 09:11:58.703166|82.92|xylophone band +53|258|65679|4294967514|61.24|15.68|true|priscilla zipper|2013-03-01 09:11:58.703250|64.75|xylophone band +57|484|65759|4294967326|92.98|31.60|false|holly van buren|2013-03-01 09:11:58.703217|8.02|philosophy +78|500|65648|4294967435|68.19|9.06|true|mike zipper|2013-03-01 09:11:58.703147|43.89|industrial engineering +57|392|65659|4294967518|70.00|40.94|true|katie nixon|2013-03-01 09:11:58.703195|53.84|mathematics +10|274|65638|4294967437|34.95|10.36|false|nick van buren|2013-03-01 09:11:58.703136|66.15|zync studies +87|323|65690|4294967482|63.45|11.89|false|xavier allen|2013-03-01 09:11:58.703296|17.27|xylophone band +24|301|65661|4294967362|88.35|37.45|false|bob van buren|2013-03-01 09:11:58.703157|4.05|american history +47|280|65627|4294967451|54.91|14.51|false|victor nixon|2013-03-01 09:11:58.703251|4.14|history +119|264|65730|4294967323|20.16|1.19|true|victor falkner|2013-03-01 09:11:58.703097|65.40|debate +82|279|65636|4294967382|65.91|13.40|true|ethan carson|2013-03-01 09:11:58.703178|6.64|xylophone band +47|389|65604|4294967362|16.42|35.00|false|holly king|2013-03-01 09:11:58.703200|76.19|linguistics +62|305|65589|4294967549|49.68|41.46|false|katie king|2013-03-01 09:11:58.703151|48.68|mathematics +99|458|65680|4294967428|16.59|6.95|true|oscar nixon|2013-03-01 09:11:58.703107|20.51|xylophone band +4|453|65618|4294967321|22.19|42.83|false|nick garcia|2013-03-01 09:11:58.703219|80.55|philosophy +4|409|65559|4294967467|50.25|41.99|true|ethan thompson|2013-03-01 09:11:58.703261|71.73|study skills +42|475|65766|4294967318|12.41|37.81|false|katie nixon|2013-03-01 09:11:58.703103|34.12|nap time +69|391|65649|4294967415|93.50|43.10|true|bob white|2013-03-01 09:11:58.703092|89.54|quiet hour +81|307|65562|4294967518|28.91|1.06|true|rachel polk|2013-03-01 09:11:58.703113|25.52|industrial engineering +6|396|65618|4294967538|53.18|28.30|false|calvin white|2013-03-01 09:11:58.703265|94.34|study skills +93|362|65640|4294967358|79.85|24.97|false|katie polk|2013-03-01 09:11:58.703186|15.89|nap time +5|398|65704|4294967533|24.98|45.65|true|zach steinbeck|2013-03-01 09:11:58.703205|64.88|industrial engineering +31|349|65751|4294967425|44.64|12.64|false|zach davidson|2013-03-01 09:11:58.703265|10.52|quiet hour +5|507|65783|4294967354|81.55|41.19|false|ulysses steinbeck|2013-03-01 09:11:58.703267|35.93|opthamology +89|480|65762|4294967435|78.94|9.91|true|sarah thompson|2013-03-01 09:11:58.703292|93.62|opthamology +61|358|65710|4294967404|28.22|34.33|true|ethan allen|2013-03-01 09:11:58.703264|0.34|philosophy +119|309|65662|4294967500|39.80|42.87|false|quinn miller|2013-03-01 09:11:58.703188|59.83|education +58|328|65747|4294967527|17.85|2.02|false|xavier falkner|2013-03-01 09:11:58.703264|44.98|mathematics +1|389|65653|4294967378|44.76|26.69|false|victor van buren|2013-03-01 09:11:58.703320|14.17|debate +7|361|65761|4294967423|41.51|37.53|false|xavier robinson|2013-03-01 09:11:58.703306|71.28|opthamology +82|414|65757|4294967435|27.84|2.88|true|oscar miller|2013-03-01 09:11:58.703113|69.64|joggying +0|417|65593|4294967531|9.26|30.55|true|sarah carson|2013-03-01 09:11:58.703073|69.76|zync studies +45|398|65591|4294967421|25.77|45.92|false|victor laertes|2013-03-01 09:11:58.703159|76.09|zync studies +71|350|65578|4294967323|4.45|13.91|true|fred miller|2013-03-01 09:11:58.703176|33.07|chemistry +117|414|65595|4294967440|21.04|5.86|true|tom hernandez|2013-03-01 09:11:58.703296|65.18|zync studies +91|361|65737|4294967478|36.16|38.25|false|bob garcia|2013-03-01 09:11:58.703304|4.51|history +21|287|65547|4294967431|82.24|21.64|false|katie brown|2013-03-01 09:11:58.703150|94.60|topology +73|479|65683|4294967517|27.20|18.18|true|victor underhill|2013-03-01 09:11:58.703233|67.93|debate +21|264|65776|4294967480|79.24|0.03|false|nick steinbeck|2013-03-01 09:11:58.703246|19.51|chemistry +30|471|65768|4294967519|60.21|44.13|true|ethan thompson|2013-03-01 09:11:58.703145|90.89|american history +61|275|65633|4294967381|8.59|7.43|false|sarah zipper|2013-03-01 09:11:58.703167|11.04|education +105|396|65559|4294967411|60.97|21.02|false|ulysses hernandez|2013-03-01 09:11:58.703214|35.68|history +104|326|65676|4294967346|44.84|16.48|true|calvin allen|2013-03-01 09:11:58.703165|80.82|history +28|263|65742|4294967522|26.59|29.30|false|ethan thompson|2013-03-01 09:11:58.703276|52.89|linguistics +66|304|65786|4294967382|35.70|36.08|true|bob king|2013-03-01 09:11:58.703173|55.66|opthamology +101|327|65592|4294967448|6.96|5.63|false|nick xylophone|2013-03-01 09:11:58.703235|68.28|forestry +31|259|65637|4294967455|82.54|32.37|true|yuri xylophone|2013-03-01 09:11:58.703310|79.40|industrial engineering +87|459|65615|4294967439|20.04|9.20|false|yuri hernandez|2013-03-01 09:11:58.703104|67.54|geology +99|469|65599|4294967443|93.88|19.59|true|wendy davidson|2013-03-01 09:11:58.703263|98.06|education +62|262|65559|4294967401|25.70|25.74|false|oscar xylophone|2013-03-01 09:11:58.703086|73.75|forestry +43|472|65693|4294967505|67.03|12.61|true|luke falkner|2013-03-01 09:11:58.703166|11.02|study skills +109|487|65741|4294967300|99.34|14.35|false|nick ellison|2013-03-01 09:11:58.703078|99.42|nap time +105|306|65748|4294967448|42.43|38.99|false|jessica king|2013-03-01 09:11:58.703099|43.79|zync studies +26|497|65573|4294967310|33.96|22.71|true|oscar king|2013-03-01 09:11:58.703157|39.42|wind surfing +12|319|65538|4294967342|87.16|5.66|false|mike brown|2013-03-01 09:11:58.703181|18.02|american history +120|413|65779|4294967368|59.72|33.74|true|gabriella thompson|2013-03-01 09:11:58.703177|51.55|undecided +9|421|65639|4294967403|96.67|5.60|true|fred young|2013-03-01 09:11:58.703260|61.56|joggying +66|482|65702|4294967524|1.39|11.59|true|nick van buren|2013-03-01 09:11:58.703284|49.92|xylophone band +66|371|65620|4294967305|59.78|31.50|false|calvin brown|2013-03-01 09:11:58.703159|14.69|debate +83|419|65571|4294967538|97.95|36.66|true|jessica king|2013-03-01 09:11:58.703096|10.48|philosophy +41|455|65768|4294967535|8.23|38.31|false|mike zipper|2013-03-01 09:11:58.703127|49.73|yard duty +114|373|65680|4294967478|83.95|8.55|true|nick ellison|2013-03-01 09:11:58.703227|60.04|values clariffication +119|504|65747|4294967422|71.72|12.29|true|zach white|2013-03-01 09:11:58.703256|28.44|american history +76|503|65721|4294967390|79.65|2.05|true|gabriella robinson|2013-03-01 09:11:58.703283|88.29|nap time +88|317|65702|4294967394|79.55|13.50|true|fred nixon|2013-03-01 09:11:58.703080|76.19|debate +75|314|65685|4294967481|56.24|34.84|true|ethan ellison|2013-03-01 09:11:58.703250|87.79|kindergarten +24|503|65562|4294967444|93.23|8.96|false|fred polk|2013-03-01 09:11:58.703325|53.14|industrial engineering +93|289|65691|4294967399|33.42|46.28|true|holly miller|2013-03-01 09:11:58.703262|89.20|kindergarten +44|364|65550|4294967528|18.37|7.10|false|mike garcia|2013-03-01 09:11:58.703229|50.61|history +110|453|65620|4294967357|29.82|44.41|true|wendy polk|2013-03-01 09:11:58.703274|17.78|nap time +82|425|65724|4294967478|31.23|39.15|false|victor johnson|2013-03-01 09:11:58.703110|36.25|forestry +67|462|65698|4294967463|21.39|3.31|true|yuri falkner|2013-03-01 09:11:58.703189|53.59|linguistics +16|392|65787|4294967367|63.07|10.48|false|ethan king|2013-03-01 09:11:58.703212|13.01|debate +78|420|65536|4294967384|69.72|35.55|true|victor johnson|2013-03-01 09:11:58.703276|30.25|undecided +88|378|65555|4294967502|15.81|14.67|true|ethan zipper|2013-03-01 09:11:58.703275|46.11|forestry +90|495|65756|4294967469|60.93|23.26|true|yuri king|2013-03-01 09:11:58.703133|57.10|philosophy +120|438|65741|4294967359|34.85|14.62|true|irene nixon|2013-03-01 09:11:58.703111|47.65|education +107|372|65536|4294967308|46.13|18.68|true|oscar allen|2013-03-01 09:11:58.703310|95.99|debate +45|421|65598|4294967427|38.17|49.12|false|mike hernandez|2013-03-01 09:11:58.703211|90.93|quiet hour +53|429|65724|4294967478|96.89|18.20|false|wendy nixon|2013-03-01 09:11:58.703116|67.59|zync studies +69|303|65729|4294967343|97.39|19.41|true|wendy johnson|2013-03-01 09:11:58.703243|56.28|philosophy +88|425|65615|4294967436|83.43|25.40|true|quinn allen|2013-03-01 09:11:58.703100|2.25|chemistry +24|360|65611|4294967421|83.53|48.60|true|bob steinbeck|2013-03-01 09:11:58.703166|85.98|debate +44|491|65712|4294967496|49.20|46.36|false|alice underhill|2013-03-01 09:11:58.703115|87.69|joggying +8|406|65613|4294967526|30.07|37.23|false|mike carson|2013-03-01 09:11:58.703238|49.60|values clariffication +3|439|65704|4294967477|93.74|6.26|true|mike steinbeck|2013-03-01 09:11:58.703312|0.06|forestry +55|284|65775|4294967448|27.33|11.22|true|priscilla nixon|2013-03-01 09:11:58.703207|90.40|philosophy +32|290|65710|4294967378|85.35|38.45|true|ethan nixon|2013-03-01 09:11:58.703120|35.59|religion +65|504|65625|4294967515|45.39|22.92|false|priscilla ichabod|2013-03-01 09:11:58.703175|41.06|debate +79|434|65722|4294967525|6.97|27.21|false|sarah young|2013-03-01 09:11:58.703214|88.06|linguistics +29|457|65630|4294967410|87.96|23.54|true|david zipper|2013-03-01 09:11:58.703080|60.17|joggying +89|464|65720|4294967453|92.32|32.28|false|victor robinson|2013-03-01 09:11:58.703115|42.01|philosophy +108|295|65708|4294967338|6.73|24.21|false|nick van buren|2013-03-01 09:11:58.703210|51.91|topology +57|413|65786|4294967475|34.57|37.10|true|ethan van buren|2013-03-01 09:11:58.703082|73.12|forestry +50|260|65574|4294967497|54.58|4.31|false|calvin zipper|2013-03-01 09:11:58.703286|88.75|undecided +17|454|65642|4294967479|47.65|44.47|true|ethan robinson|2013-03-01 09:11:58.703166|65.79|wind surfing +82|354|65779|4294967306|26.96|9.70|true|alice ovid|2013-03-01 09:11:58.703311|61.44|values clariffication +102|493|65743|4294967360|29.03|41.33|true|fred allen|2013-03-01 09:11:58.703131|72.99|biology +117|304|65549|4294967418|33.16|8.86|false|victor davidson|2013-03-01 09:11:58.703132|31.74|study skills +61|274|65537|4294967462|14.24|20.38|true|nick davidson|2013-03-01 09:11:58.703119|56.13|forestry +43|385|65560|4294967451|54.11|43.45|true|xavier young|2013-03-01 09:11:58.703255|61.56|undecided +17|260|65566|4294967321|26.35|29.01|false|tom hernandez|2013-03-01 09:11:58.703104|51.21|philosophy +36|367|65635|4294967346|79.37|19.51|true|katie nixon|2013-03-01 09:11:58.703110|37.24|chemistry +-3|339|65671|4294967311|8.37|15.98|true|bob ellison|2013-03-01 09:11:58.703261|14.01|linguistics +56|275|65772|4294967367|74.69|38.31|true|ulysses underhill|2013-03-01 09:11:58.703093|99.48|religion +97|428|65671|4294967536|83.40|13.65|true|ethan robinson|2013-03-01 09:11:58.703127|99.50|education +39|472|65612|4294967319|11.35|10.27|false|luke allen|2013-03-01 09:11:58.703128|28.22|topology +119|413|65536|4294967449|15.82|27.95|false|rachel brown|2013-03-01 09:11:58.703233|30.60|forestry +51|288|65611|4294967317|90.68|32.05|false|quinn king|2013-03-01 09:11:58.703075|17.64|opthamology +120|453|65744|4294967447|47.52|9.78|true|calvin ichabod|2013-03-01 09:11:58.703178|7.89|biology +122|457|65763|4294967535|21.11|36.20|false|xavier polk|2013-03-01 09:11:58.703324|8.48|religion +68|352|65731|4294967476|55.34|0.60|true|ethan johnson|2013-03-01 09:11:58.703264|5.39|xylophone band +124|396|65600|4294967453|50.94|43.34|true|alice zipper|2013-03-01 09:11:58.703124|24.87|biology +124|448|65615|4294967303|20.19|30.52|true|nick van buren|2013-03-01 09:11:58.703193|87.80|industrial engineering +61|354|65606|4294967450|27.47|10.80|false|ethan ovid|2013-03-01 09:11:58.703198|88.65|industrial engineering +119|479|65649|4294967450|14.43|0.94|true|calvin steinbeck|2013-03-01 09:11:58.703091|71.82|philosophy +12|324|65749|4294967432|43.01|26.76|true|alice polk|2013-03-01 09:11:58.703287|57.58|opthamology +31|405|65610|4294967398|2.17|3.53|false|zach falkner|2013-03-01 09:11:58.703321|38.34|wind surfing +96|384|65774|4294967525|39.38|46.68|false|ethan steinbeck|2013-03-01 09:11:58.703106|53.74|undecided +39|300|65566|4294967464|6.80|15.49|false|victor ichabod|2013-03-01 09:11:58.703260|83.01|kindergarten +113|423|65677|4294967493|76.10|8.86|true|jessica nixon|2013-03-01 09:11:58.703316|15.30|biology +54|326|65704|4294967363|32.02|15.29|false|ulysses quirinius|2013-03-01 09:11:58.703165|70.96|american history +90|453|65748|4294967296|22.52|47.92|false|ethan ellison|2013-03-01 09:11:58.703220|18.73|kindergarten +74|445|65695|4294967398|26.18|35.21|true|tom ovid|2013-03-01 09:11:58.703299|35.48|education +12|489|65723|4294967526|11.06|49.33|false|irene polk|2013-03-01 09:11:58.703202|79.13|religion +26|294|65645|4294967444|55.11|6.26|false|xavier king|2013-03-01 09:11:58.703223|32.88|study skills +19|399|65625|4294967329|87.94|4.46|true|gabriella robinson|2013-03-01 09:11:58.703093|43.59|nap time +109|265|65575|4294967516|96.18|7.01|true|bob quirinius|2013-03-01 09:11:58.703097|33.47|values clariffication +20|345|65585|4294967359|18.35|34.42|false|zach miller|2013-03-01 09:11:58.703201|47.51|opthamology +48|372|65731|4294967321|85.40|1.68|false|bob thompson|2013-03-01 09:11:58.703138|94.34|education +57|476|65768|4294967336|45.77|1.74|true|bob nixon|2013-03-01 09:11:58.703211|4.35|biology +3|463|65675|4294967306|40.66|43.32|false|nick steinbeck|2013-03-01 09:11:58.703158|21.75|mathematics +92|448|65779|4294967497|69.72|21.74|true|luke quirinius|2013-03-01 09:11:58.703246|47.99|industrial engineering +124|511|65713|4294967348|46.91|25.76|false|katie allen|2013-03-01 09:11:58.703196|89.98|religion +106|382|65728|4294967359|92.59|21.15|false|fred quirinius|2013-03-01 09:11:58.703082|41.62|quiet hour +113|475|65767|4294967469|24.37|43.29|false|irene polk|2013-03-01 09:11:58.703259|39.42|kindergarten +28|328|65786|4294967518|94.56|18.30|true|zach hernandez|2013-03-01 09:11:58.703236|9.07|chemistry +116|478|65695|4294967395|7.09|8.92|false|oscar davidson|2013-03-01 09:11:58.703260|29.63|values clariffication +44|300|65649|4294967508|25.74|19.34|true|yuri allen|2013-03-01 09:11:58.703283|33.11|opthamology +103|389|65660|4294967440|38.32|41.78|true|quinn white|2013-03-01 09:11:58.703249|72.67|american history +124|431|65576|4294967367|56.74|36.29|false|tom ellison|2013-03-01 09:11:58.703199|1.22|mathematics +46|390|65625|4294967523|39.15|10.74|true|david young|2013-03-01 09:11:58.703171|18.67|kindergarten +80|320|65773|4294967399|2.85|37.03|true|tom allen|2013-03-01 09:11:58.703310|61.47|debate +100|306|65781|4294967335|29.64|14.99|false|katie miller|2013-03-01 09:11:58.703108|9.07|yard duty +101|411|65772|4294967365|32.71|33.18|false|calvin davidson|2013-03-01 09:11:58.703147|80.82|opthamology +85|428|65587|4294967340|75.56|43.57|false|luke robinson|2013-03-01 09:11:58.703293|53.21|mathematics +6|321|65587|4294967297|90.10|0.33|false|gabriella robinson|2013-03-01 09:11:58.703288|74.23|topology +48|406|65654|4294967345|0.39|33.84|false|zach robinson|2013-03-01 09:11:58.703272|21.26|biology +15|355|65627|4294967414|96.98|5.65|false|quinn johnson|2013-03-01 09:11:58.703296|96.28|quiet hour +78|463|65645|4294967361|39.65|44.14|true|bob quirinius|2013-03-01 09:11:58.703183|80.42|study skills +124|379|65791|4294967390|67.46|40.25|true|ethan brown|2013-03-01 09:11:58.703323|56.97|american history +122|382|65576|4294967434|25.10|18.85|false|holly carson|2013-03-01 09:11:58.703282|32.60|philosophy +15|419|65688|4294967509|45.48|19.20|true|tom brown|2013-03-01 09:11:58.703315|40.07|values clariffication +48|342|65560|4294967329|67.61|3.22|false|gabriella carson|2013-03-01 09:11:58.703108|39.05|xylophone band +13|325|65547|4294967310|84.31|18.42|true|david robinson|2013-03-01 09:11:58.703297|66.54|philosophy +22|344|65552|4294967434|61.42|12.76|true|katie laertes|2013-03-01 09:11:58.703202|42.74|values clariffication +46|355|65758|4294967333|12.98|17.65|true|luke xylophone|2013-03-01 09:11:58.703187|55.82|yard duty +65|411|65785|4294967481|57.84|25.83|false|jessica hernandez|2013-03-01 09:11:58.703183|20.54|debate +24|450|65721|4294967332|83.75|9.61|true|tom hernandez|2013-03-01 09:11:58.703150|95.54|undecided +75|473|65656|4294967431|53.77|49.45|false|fred quirinius|2013-03-01 09:11:58.703156|15.60|linguistics +118|445|65681|4294967439|65.25|19.44|true|wendy robinson|2013-03-01 09:11:58.703203|34.45|american history +75|334|65599|4294967331|9.92|6.82|false|oscar zipper|2013-03-01 09:11:58.703186|15.98|history +73|308|65595|4294967344|39.10|4.17|false|luke falkner|2013-03-01 09:11:58.703265|64.75|wind surfing +113|350|65587|4294967377|54.33|10.65|false|katie van buren|2013-03-01 09:11:58.703295|57.67|undecided +4|301|65558|4294967362|71.84|20.10|false|yuri falkner|2013-03-01 09:11:58.703243|58.15|mathematics +76|272|65549|4294967347|40.14|19.46|true|irene garcia|2013-03-01 09:11:58.703322|38.93|education +38|301|65686|4294967456|70.76|5.99|false|rachel xylophone|2013-03-01 09:11:58.703079|64.81|xylophone band +24|495|65658|4294967502|11.18|23.82|true|sarah steinbeck|2013-03-01 09:11:58.703220|39.36|linguistics +90|269|65641|4294967389|90.42|32.68|false|oscar thompson|2013-03-01 09:11:58.703079|76.49|religion +14|297|65567|4294967412|85.82|26.85|true|luke thompson|2013-03-01 09:11:58.703242|39.56|kindergarten +108|318|65557|4294967485|36.87|13.93|true|mike miller|2013-03-01 09:11:58.703277|93.85|chemistry +32|261|65597|4294967304|98.09|17.73|false|fred johnson|2013-03-01 09:11:58.703314|36.97|study skills +107|481|65694|4294967410|18.45|10.43|false|jessica laertes|2013-03-01 09:11:58.703219|23.38|kindergarten +102|436|65547|4294967339|35.51|38.65|true|quinn steinbeck|2013-03-01 09:11:58.703299|50.32|joggying +85|482|65791|4294967526|16.26|37.52|true|wendy steinbeck|2013-03-01 09:11:58.703192|62.34|joggying +77|378|65548|4294967374|85.60|14.16|false|oscar nixon|2013-03-01 09:11:58.703298|98.23|kindergarten +106|373|65746|4294967389|93.05|8.68|false|ulysses garcia|2013-03-01 09:11:58.703294|8.94|undecided +103|446|65783|4294967308|47.08|48.21|false|zach zipper|2013-03-01 09:11:58.703164|28.20|nap time +8|467|65708|4294967492|26.77|9.82|true|calvin king|2013-03-01 09:11:58.703225|61.79|religion +2|323|65548|4294967351|78.00|47.16|true|zach brown|2013-03-01 09:11:58.703152|58.99|study skills +85|495|65595|4294967388|45.03|5.17|false|ulysses falkner|2013-03-01 09:11:58.703259|45.38|linguistics +15|273|65714|4294967452|41.00|47.34|false|jessica hernandez|2013-03-01 09:11:58.703205|84.03|topology +16|257|65610|4294967430|47.58|23.18|true|yuri brown|2013-03-01 09:11:58.703284|18.70|zync studies +97|472|65687|4294967484|65.82|13.49|true|holly polk|2013-03-01 09:11:58.703119|63.85|chemistry +119|346|65609|4294967432|37.79|10.70|true|david allen|2013-03-01 09:11:58.703271|90.70|philosophy +123|447|65676|4294967433|80.72|28.91|true|luke polk|2013-03-01 09:11:58.703092|13.91|opthamology +98|468|65778|4294967415|38.73|25.87|true|ulysses polk|2013-03-01 09:11:58.703074|80.31|debate +64|357|65640|4294967455|68.45|20.36|true|rachel xylophone|2013-03-01 09:11:58.703200|68.48|chemistry +54|337|65570|4294967331|47.05|14.39|false|fred polk|2013-03-01 09:11:58.703277|19.30|biology +92|417|65644|4294967386|77.40|49.52|true|david brown|2013-03-01 09:11:58.703181|73.08|wind surfing +4|381|65756|4294967406|76.25|30.22|false|sarah johnson|2013-03-01 09:11:58.703311|3.61|undecided +116|317|65671|4294967519|66.49|9.40|true|calvin ichabod|2013-03-01 09:11:58.703084|26.34|mathematics +9|440|65751|4294967390|62.24|20.51|true|bob laertes|2013-03-01 09:11:58.703169|53.51|debate +4|336|65770|4294967370|81.02|37.45|false|victor laertes|2013-03-01 09:11:58.703257|57.42|chemistry +30|369|65740|4294967529|46.28|45.37|false|david nixon|2013-03-01 09:11:58.703325|44.00|religion +34|276|65621|4294967542|48.39|47.83|true|xavier ovid|2013-03-01 09:11:58.703297|87.52|debate +60|462|65563|4294967440|15.41|46.01|true|zach underhill|2013-03-01 09:11:58.703148|14.88|geology +10|361|65647|4294967360|45.01|43.10|false|alice zipper|2013-03-01 09:11:58.703287|82.83|opthamology +88|350|65550|4294967520|66.47|10.14|true|victor van buren|2013-03-01 09:11:58.703201|65.34|geology +82|509|65627|4294967316|21.11|40.73|false|luke carson|2013-03-01 09:11:58.703285|62.49|chemistry +32|495|65645|4294967542|68.51|10.77|true|luke nixon|2013-03-01 09:11:58.703210|79.40|study skills +9|266|65716|4294967323|76.51|42.04|true|zach quirinius|2013-03-01 09:11:58.703211|77.75|opthamology +118|303|65692|4294967445|28.15|14.23|true|alice ichabod|2013-03-01 09:11:58.703251|17.53|undecided +54|318|65742|4294967454|59.27|30.50|false|calvin xylophone|2013-03-01 09:11:58.703204|28.41|opthamology +5|456|65773|4294967411|14.96|14.89|false|wendy thompson|2013-03-01 09:11:58.703071|45.45|philosophy +77|469|65553|4294967502|9.40|13.11|false|holly underhill|2013-03-01 09:11:58.703206|29.01|history +99|491|65598|4294967540|77.43|8.23|true|david johnson|2013-03-01 09:11:58.703191|53.38|opthamology +38|360|65761|4294967451|92.50|12.28|true|fred miller|2013-03-01 09:11:58.703233|80.85|philosophy +91|493|65616|4294967298|76.22|9.70|false|bob ovid|2013-03-01 09:11:58.703117|48.69|forestry +47|261|65648|4294967314|62.08|26.49|false|ulysses johnson|2013-03-01 09:11:58.703127|46.26|forestry +87|403|65728|4294967362|78.80|15.56|true|irene carson|2013-03-01 09:11:58.703172|81.13|history +-3|337|65629|4294967521|55.59|6.54|true|luke king|2013-03-01 09:11:58.703207|58.52|industrial engineering +35|266|65709|4294967467|99.41|32.00|false|mike white|2013-03-01 09:11:58.703199|82.60|kindergarten +28|324|65773|4294967449|67.33|39.17|false|sarah xylophone|2013-03-01 09:11:58.703191|35.25|religion +15|313|65688|4294967347|29.43|21.91|false|zach allen|2013-03-01 09:11:58.703273|74.14|study skills +28|462|65555|4294967329|50.93|42.53|true|irene robinson|2013-03-01 09:11:58.703170|93.83|history +65|273|65563|4294967368|86.73|33.27|false|rachel xylophone|2013-03-01 09:11:58.703324|8.89|biology +112|341|65627|4294967301|86.14|7.93|true|luke robinson|2013-03-01 09:11:58.703205|95.14|forestry +4|509|65669|4294967467|17.66|30.35|true|nick white|2013-03-01 09:11:58.703219|84.23|chemistry +117|346|65568|4294967340|98.08|18.77|true|jessica van buren|2013-03-01 09:11:58.703170|67.05|quiet hour +31|302|65621|4294967399|12.49|48.25|false|irene laertes|2013-03-01 09:11:58.703189|66.61|wind surfing +60|467|65609|4294967332|42.74|42.12|true|sarah hernandez|2013-03-01 09:11:58.703305|80.09|industrial engineering +63|428|65557|4294967479|43.87|32.54|false|katie miller|2013-03-01 09:11:58.703162|30.08|quiet hour +54|265|65564|4294967309|69.78|19.39|true|ethan hernandez|2013-03-01 09:11:58.703268|66.68|american history +115|352|65751|4294967303|8.79|9.65|true|ethan young|2013-03-01 09:11:58.703207|14.09|education +112|382|65601|4294967326|42.92|31.27|false|irene falkner|2013-03-01 09:11:58.703106|90.38|wind surfing +32|294|65551|4294967427|47.30|9.30|true|mike brown|2013-03-01 09:11:58.703320|57.34|values clariffication +36|363|65706|4294967500|39.36|27.45|false|fred robinson|2013-03-01 09:11:58.703252|83.85|quiet hour +76|508|65593|4294967318|25.48|28.39|false|irene falkner|2013-03-01 09:11:58.703321|62.51|geology +73|483|65604|4294967451|62.75|45.03|false|yuri young|2013-03-01 09:11:58.703142|83.33|religion +92|471|65583|4294967398|7.00|35.96|true|gabriella brown|2013-03-01 09:11:58.703172|91.81|industrial engineering +7|308|65791|4294967304|88.80|38.93|false|fred ellison|2013-03-01 09:11:58.703139|93.50|xylophone band +9|381|65704|4294967505|95.46|21.15|true|alice brown|2013-03-01 09:11:58.703149|85.26|quiet hour +40|355|65758|4294967321|19.77|15.37|false|ethan hernandez|2013-03-01 09:11:58.703322|19.13|mathematics +66|460|65721|4294967361|10.36|30.74|true|mike miller|2013-03-01 09:11:58.703268|32.12|zync studies +120|492|65620|4294967464|52.30|43.70|true|mike steinbeck|2013-03-01 09:11:58.703074|0.02|chemistry +22|351|65742|4294967430|61.06|29.65|false|sarah miller|2013-03-01 09:11:58.703234|11.93|forestry +0|464|65602|4294967530|63.82|23.40|true|wendy king|2013-03-01 09:11:58.703160|81.96|opthamology +101|335|65736|4294967485|7.57|46.61|false|yuri young|2013-03-01 09:11:58.703304|53.61|religion +37|488|65640|4294967380|94.93|2.09|false|priscilla underhill|2013-03-01 09:11:58.703319|50.13|nap time +76|498|65676|4294967376|42.37|38.43|true|zach young|2013-03-01 09:11:58.703109|41.08|forestry +60|305|65600|4294967452|49.58|26.04|false|calvin zipper|2013-03-01 09:11:58.703254|41.56|study skills +101|257|65638|4294967365|95.35|19.76|true|ethan underhill|2013-03-01 09:11:58.703234|19.62|study skills +52|318|65644|4294967429|50.82|8.33|true|victor laertes|2013-03-01 09:11:58.703318|40.65|xylophone band +30|347|65604|4294967482|53.48|10.40|true|tom van buren|2013-03-01 09:11:58.703258|30.57|education +8|360|65570|4294967483|77.94|19.62|false|tom garcia|2013-03-01 09:11:58.703172|45.47|linguistics +52|477|65735|4294967463|98.29|0.32|false|nick polk|2013-03-01 09:11:58.703243|21.96|mathematics +1|416|65563|4294967378|70.73|44.49|false|zach miller|2013-03-01 09:11:58.703231|74.06|yard duty +32|311|65715|4294967428|97.31|22.20|true|alice falkner|2013-03-01 09:11:58.703228|49.60|religion +37|446|65707|4294967348|18.29|0.76|false|fred king|2013-03-01 09:11:58.703277|57.22|linguistics +108|423|65607|4294967497|41.24|3.37|false|victor robinson|2013-03-01 09:11:58.703084|96.76|quiet hour +45|303|65734|4294967361|53.38|41.26|true|ethan quirinius|2013-03-01 09:11:58.703212|39.38|linguistics +94|399|65657|4294967524|95.09|43.52|true|oscar xylophone|2013-03-01 09:11:58.703099|1.42|quiet hour +114|410|65727|4294967501|48.91|46.68|false|priscilla carson|2013-03-01 09:11:58.703222|80.64|nap time +49|416|65771|4294967526|31.05|30.75|false|alice robinson|2013-03-01 09:11:58.703131|87.44|history +119|434|65680|4294967509|60.36|24.16|false|ethan carson|2013-03-01 09:11:58.703167|21.27|american history +78|271|65771|4294967534|31.17|17.25|true|oscar thompson|2013-03-01 09:11:58.703255|50.53|zync studies +1|484|65676|4294967405|77.44|0.14|false|wendy zipper|2013-03-01 09:11:58.703244|35.70|biology +2|407|65691|4294967332|90.94|26.92|true|quinn ovid|2013-03-01 09:11:58.703170|51.04|values clariffication +19|471|65595|4294967444|39.01|36.41|false|katie steinbeck|2013-03-01 09:11:58.703272|64.98|xylophone band +13|446|65725|4294967328|19.15|9.87|true|sarah xylophone|2013-03-01 09:11:58.703130|13.68|yard duty +64|474|65573|4294967347|28.69|6.21|true|oscar van buren|2013-03-01 09:11:58.703146|75.77|linguistics +67|408|65616|4294967444|16.08|44.35|false|mike brown|2013-03-01 09:11:58.703196|29.68|geology +66|347|65578|4294967479|3.13|42.30|true|calvin davidson|2013-03-01 09:11:58.703258|49.97|xylophone band +120|415|65695|4294967336|20.15|13.84|true|zach underhill|2013-03-01 09:11:58.703231|98.25|debate +56|340|65767|4294967443|11.81|11.18|true|bob white|2013-03-01 09:11:58.703197|61.94|mathematics +7|491|65759|4294967517|43.51|6.51|true|ethan thompson|2013-03-01 09:11:58.703145|33.36|chemistry +11|320|65670|4294967509|45.88|27.05|false|xavier garcia|2013-03-01 09:11:58.703158|49.83|nap time +14|489|65772|4294967384|69.00|46.40|true|gabriella nixon|2013-03-01 09:11:58.703284|97.38|mathematics +41|450|65568|4294967540|3.88|45.67|false|ethan nixon|2013-03-01 09:11:58.703242|34.34|wind surfing +56|275|65582|4294967338|54.86|19.12|true|yuri laertes|2013-03-01 09:11:58.703132|5.68|study skills +43|366|65695|4294967333|84.68|22.09|false|rachel polk|2013-03-01 09:11:58.703145|81.38|industrial engineering +113|478|65774|4294967416|89.37|1.52|false|ethan robinson|2013-03-01 09:11:58.703252|75.53|forestry +79|353|65744|4294967439|12.19|27.27|true|katie ovid|2013-03-01 09:11:58.703077|45.83|geology +94|504|65780|4294967316|55.29|42.67|true|yuri carson|2013-03-01 09:11:58.703293|69.14|history +85|366|65545|4294967350|62.54|46.76|false|xavier miller|2013-03-01 09:11:58.703244|70.45|kindergarten +39|353|65636|4294967509|3.54|24.38|true|jessica xylophone|2013-03-01 09:11:58.703270|3.68|philosophy +86|364|65661|4294967429|2.17|29.87|false|holly carson|2013-03-01 09:11:58.703113|48.22|values clariffication +96|395|65609|4294967461|96.87|1.87|true|bob thompson|2013-03-01 09:11:58.703160|71.15|forestry +24|275|65786|4294967346|55.65|20.42|false|sarah polk|2013-03-01 09:11:58.703231|87.41|philosophy +63|449|65619|4294967495|67.77|40.31|false|tom ellison|2013-03-01 09:11:58.703215|27.96|philosophy +7|329|65638|4294967398|3.15|29.92|false|ulysses ovid|2013-03-01 09:11:58.703094|46.36|study skills +72|362|65722|4294967476|39.84|20.05|false|bob nixon|2013-03-01 09:11:58.703103|77.61|zync studies +66|320|65782|4294967309|61.62|10.82|true|victor polk|2013-03-01 09:11:58.703105|29.83|philosophy +55|374|65601|4294967305|89.07|32.33|true|calvin robinson|2013-03-01 09:11:58.703221|80.90|education +102|257|65694|4294967469|29.00|1.75|false|luke falkner|2013-03-01 09:11:58.703160|60.20|values clariffication +30|362|65686|4294967340|70.18|29.16|false|fred nixon|2013-03-01 09:11:58.703296|85.21|topology +86|294|65737|4294967346|68.06|47.76|true|bob thompson|2013-03-01 09:11:58.703317|55.30|chemistry +-1|439|65638|4294967447|4.31|24.29|false|bob robinson|2013-03-01 09:11:58.703171|69.06|topology +41|367|65748|4294967514|75.90|47.29|true|victor ellison|2013-03-01 09:11:58.703105|60.18|chemistry +30|268|65701|4294967549|16.65|18.35|true|fred quirinius|2013-03-01 09:11:58.703125|49.26|history +56|421|65629|4294967447|26.21|43.11|true|fred johnson|2013-03-01 09:11:58.703200|91.90|xylophone band +62|291|65661|4294967301|27.42|38.92|true|gabriella steinbeck|2013-03-01 09:11:58.703093|96.26|kindergarten +104|261|65740|4294967361|84.16|7.34|true|nick quirinius|2013-03-01 09:11:58.703212|66.07|kindergarten +51|309|65687|4294967510|35.89|46.88|true|rachel xylophone|2013-03-01 09:11:58.703276|48.20|philosophy +77|426|65773|4294967366|79.08|20.88|true|ulysses hernandez|2013-03-01 09:11:58.703075|79.25|joggying +117|492|65585|4294967391|95.59|12.50|false|tom underhill|2013-03-01 09:11:58.703155|6.14|education +44|274|65751|4294967465|76.32|14.48|false|oscar johnson|2013-03-01 09:11:58.703164|24.62|philosophy +28|446|65725|4294967516|58.76|9.30|false|mike johnson|2013-03-01 09:11:58.703312|31.58|religion +7|434|65696|4294967484|18.77|23.59|true|mike ichabod|2013-03-01 09:11:58.703215|17.77|linguistics +95|320|65630|4294967330|70.42|10.73|true|david young|2013-03-01 09:11:58.703261|53.29|opthamology +23|476|65741|4294967372|8.63|10.12|false|holly steinbeck|2013-03-01 09:11:58.703303|6.11|american history +-2|421|65772|4294967546|16.35|3.49|true|jessica ovid|2013-03-01 09:11:58.703250|22.73|study skills +92|473|65681|4294967480|72.60|23.70|false|ethan young|2013-03-01 09:11:58.703226|28.90|biology +121|468|65688|4294967515|74.68|2.32|true|mike allen|2013-03-01 09:11:58.703110|36.77|quiet hour +114|259|65610|4294967537|97.94|10.53|true|yuri thompson|2013-03-01 09:11:58.703077|85.17|study skills +2|265|65691|4294967441|89.11|14.82|false|oscar thompson|2013-03-01 09:11:58.703096|72.89|kindergarten +73|265|65738|4294967382|42.63|31.19|false|tom ichabod|2013-03-01 09:11:58.703094|54.05|history +52|461|65719|4294967390|70.74|9.94|true|holly falkner|2013-03-01 09:11:58.703111|54.98|yard duty +27|260|65605|4294967308|11.30|19.46|true|holly nixon|2013-03-01 09:11:58.703123|96.04|debate +114|290|65685|4294967437|47.23|41.78|true|alice hernandez|2013-03-01 09:11:58.703087|87.93|industrial engineering +51|327|65730|4294967439|48.19|1.12|false|calvin johnson|2013-03-01 09:11:58.703184|44.92|industrial engineering +26|384|65623|4294967505|11.37|20.15|false|bob nixon|2013-03-01 09:11:58.703307|66.07|wind surfing +115|456|65588|4294967347|30.08|47.18|true|ethan xylophone|2013-03-01 09:11:58.703131|77.44|biology +80|489|65753|4294967344|75.87|1.32|true|ulysses polk|2013-03-01 09:11:58.703082|74.50|philosophy +14|284|65614|4294967417|46.04|7.29|false|fred xylophone|2013-03-01 09:11:58.703288|23.08|wind surfing +30|433|65669|4294967518|18.94|46.79|true|luke polk|2013-03-01 09:11:58.703125|62.59|xylophone band +122|399|65623|4294967382|25.99|8.48|false|ethan ellison|2013-03-01 09:11:58.703098|47.45|kindergarten +1|300|65650|4294967322|3.45|14.42|false|priscilla polk|2013-03-01 09:11:58.703215|76.87|biology +71|422|65586|4294967527|27.73|10.80|true|fred robinson|2013-03-01 09:11:58.703245|11.41|undecided +23|321|65558|4294967492|73.69|4.54|false|jessica johnson|2013-03-01 09:11:58.703076|94.62|linguistics +117|448|65760|4294967414|11.03|34.51|false|ethan garcia|2013-03-01 09:11:58.703313|73.48|study skills +84|306|65655|4294967397|76.41|35.06|true|sarah ichabod|2013-03-01 09:11:58.703091|0.98|zync studies +34|462|65704|4294967355|18.84|14.63|false|irene polk|2013-03-01 09:11:58.703081|65.13|geology +101|404|65624|4294967347|17.04|12.07|true|fred hernandez|2013-03-01 09:11:58.703167|0.21|topology +22|332|65549|4294967519|48.35|46.07|false|sarah polk|2013-03-01 09:11:58.703124|27.63|linguistics +64|303|65644|4294967492|48.85|48.01|false|calvin white|2013-03-01 09:11:58.703303|47.26|quiet hour +75|416|65690|4294967437|29.62|8.01|true|priscilla ichabod|2013-03-01 09:11:58.703241|55.33|geology +7|262|65555|4294967319|0.82|6.63|true|david king|2013-03-01 09:11:58.703097|27.12|chemistry +52|448|65650|4294967311|36.34|48.13|false|wendy van buren|2013-03-01 09:11:58.703192|42.68|history +84|301|65716|4294967317|80.25|23.74|true|luke van buren|2013-03-01 09:11:58.703103|98.64|quiet hour +18|482|65634|4294967421|31.71|16.80|true|david johnson|2013-03-01 09:11:58.703263|82.48|kindergarten +87|311|65563|4294967489|38.20|14.34|false|xavier underhill|2013-03-01 09:11:58.703246|54.98|wind surfing +97|308|65538|4294967517|2.94|18.18|false|katie nixon|2013-03-01 09:11:58.703186|8.91|mathematics +3|311|65756|4294967510|59.82|30.84|true|xavier laertes|2013-03-01 09:11:58.703247|35.26|values clariffication +65|322|65606|4294967379|60.52|38.50|false|xavier allen|2013-03-01 09:11:58.703178|22.86|debate +108|276|65758|4294967326|37.25|33.06|true|nick zipper|2013-03-01 09:11:58.703191|51.68|debate +25|422|65678|4294967426|72.19|23.31|true|mike king|2013-03-01 09:11:58.703112|77.02|forestry +5|464|65756|4294967371|0.75|19.18|false|irene miller|2013-03-01 09:11:58.703111|72.30|values clariffication +76|463|65543|4294967350|67.93|3.23|false|zach xylophone|2013-03-01 09:11:58.703286|35.56|education +49|474|65706|4294967516|22.92|47.19|true|jessica polk|2013-03-01 09:11:58.703292|23.07|wind surfing +20|389|65752|4294967441|90.40|13.44|false|calvin davidson|2013-03-01 09:11:58.703163|57.39|quiet hour +42|280|65699|4294967408|72.64|49.27|true|irene robinson|2013-03-01 09:11:58.703276|42.57|wind surfing +14|399|65602|4294967473|74.74|14.30|true|zach davidson|2013-03-01 09:11:58.703155|82.74|forestry +93|380|65705|4294967469|78.35|6.19|false|holly garcia|2013-03-01 09:11:58.703152|39.94|mathematics +86|356|65650|4294967311|23.75|0.78|true|priscilla ichabod|2013-03-01 09:11:58.703227|30.84|philosophy +37|297|65675|4294967538|65.68|6.09|false|priscilla white|2013-03-01 09:11:58.703315|70.14|forestry +10|357|65645|4294967487|97.81|34.61|false|jessica allen|2013-03-01 09:11:58.703280|41.00|history +18|296|65723|4294967323|61.41|11.23|true|tom ichabod|2013-03-01 09:11:58.703103|51.29|zync studies +29|291|65772|4294967341|64.64|47.89|true|yuri quirinius|2013-03-01 09:11:58.703242|11.21|xylophone band +106|377|65659|4294967484|4.69|47.70|false|mike hernandez|2013-03-01 09:11:58.703298|6.66|opthamology +45|414|65711|4294967325|51.12|47.60|false|nick davidson|2013-03-01 09:11:58.703222|6.50|mathematics +42|476|65715|4294967432|77.18|16.72|false|wendy ellison|2013-03-01 09:11:58.703186|17.77|topology +37|268|65576|4294967382|8.78|49.30|true|priscilla xylophone|2013-03-01 09:11:58.703077|25.00|study skills +97|364|65768|4294967531|56.59|2.05|false|holly carson|2013-03-01 09:11:58.703226|26.81|debate +67|461|65774|4294967453|45.65|33.91|false|katie xylophone|2013-03-01 09:11:58.703121|21.59|debate +41|306|65702|4294967379|12.42|16.09|false|luke carson|2013-03-01 09:11:58.703096|24.45|american history +25|421|65742|4294967451|90.16|4.17|true|irene robinson|2013-03-01 09:11:58.703267|98.18|geology +73|326|65781|4294967344|99.51|37.23|false|yuri ellison|2013-03-01 09:11:58.703145|4.26|joggying +88|475|65722|4294967467|37.55|17.59|false|priscilla laertes|2013-03-01 09:11:58.703287|75.71|yard duty +22|494|65604|4294967371|62.73|43.72|false|yuri carson|2013-03-01 09:11:58.703289|56.72|values clariffication +97|327|65581|4294967458|86.13|39.99|true|irene quirinius|2013-03-01 09:11:58.703108|65.58|values clariffication +85|304|65618|4294967526|47.00|43.58|true|irene quirinius|2013-03-01 09:11:58.703153|14.80|debate +122|358|65541|4294967378|99.25|47.87|false|jessica quirinius|2013-03-01 09:11:58.703278|58.25|opthamology +122|487|65656|4294967499|8.32|37.21|false|ulysses robinson|2013-03-01 09:11:58.703192|48.43|biology +124|419|65546|4294967467|29.40|9.75|false|ethan underhill|2013-03-01 09:11:58.703194|42.94|american history +18|272|65718|4294967407|45.66|27.93|true|fred laertes|2013-03-01 09:11:58.703094|38.01|linguistics +94|475|65725|4294967398|13.07|45.38|false|nick ichabod|2013-03-01 09:11:58.703211|66.16|xylophone band +3|342|65723|4294967368|19.37|17.02|true|gabriella davidson|2013-03-01 09:11:58.703249|36.21|religion +56|469|65579|4294967379|89.47|47.45|false|oscar quirinius|2013-03-01 09:11:58.703143|26.95|quiet hour +102|333|65537|4294967450|89.47|33.33|true|luke nixon|2013-03-01 09:11:58.703079|53.92|religion +15|496|65740|4294967463|12.06|3.97|false|oscar carson|2013-03-01 09:11:58.703227|93.95|industrial engineering +88|420|65760|4294967323|96.52|2.59|true|wendy nixon|2013-03-01 09:11:58.703261|78.93|quiet hour +71|440|65701|4294967323|10.20|29.61|true|irene garcia|2013-03-01 09:11:58.703261|13.49|history +-1|467|65635|4294967340|19.42|9.70|false|xavier robinson|2013-03-01 09:11:58.703232|56.33|topology +107|332|65790|4294967542|17.51|1.81|true|luke garcia|2013-03-01 09:11:58.703215|58.96|biology +61|325|65594|4294967500|77.76|46.99|true|priscilla falkner|2013-03-01 09:11:58.703137|26.35|mathematics +71|379|65577|4294967528|15.99|2.55|true|quinn davidson|2013-03-01 09:11:58.703316|49.30|geology +45|489|65567|4294967491|1.42|23.46|true|oscar nixon|2013-03-01 09:11:58.703135|29.02|american history +100|416|65761|4294967296|47.89|37.99|false|rachel thompson|2013-03-01 09:11:58.703183|39.51|religion +117|321|65648|4294967358|67.22|34.33|true|oscar zipper|2013-03-01 09:11:58.703237|45.90|industrial engineering +113|454|65771|4294967296|23.34|48.03|true|irene nixon|2013-03-01 09:11:58.703216|57.29|american history +89|424|65728|4294967467|1.30|6.87|false|luke ovid|2013-03-01 09:11:58.703098|56.53|topology +117|377|65689|4294967543|98.73|23.40|true|holly falkner|2013-03-01 09:11:58.703195|28.67|undecided +47|349|65627|4294967390|66.96|46.70|false|bob underhill|2013-03-01 09:11:58.703134|41.54|chemistry +76|491|65744|4294967341|20.24|22.10|false|nick hernandez|2013-03-01 09:11:58.703230|0.73|nap time +61|302|65747|4294967413|99.84|10.06|false|holly ichabod|2013-03-01 09:11:58.703257|65.92|industrial engineering +14|383|65588|4294967337|18.13|16.99|true|gabriella ovid|2013-03-01 09:11:58.703133|31.49|philosophy +46|301|65716|4294967385|17.51|20.17|true|calvin davidson|2013-03-01 09:11:58.703268|23.76|joggying +85|485|65538|4294967320|0.15|14.75|false|alice robinson|2013-03-01 09:11:58.703303|76.67|yard duty +84|474|65674|4294967367|16.44|31.32|true|rachel xylophone|2013-03-01 09:11:58.703244|21.01|mathematics +33|499|65548|4294967459|27.44|11.20|false|sarah polk|2013-03-01 09:11:58.703113|58.30|topology +62|350|65669|4294967399|77.98|21.45|true|oscar carson|2013-03-01 09:11:58.703076|14.68|values clariffication +57|300|65573|4294967301|63.03|40.18|false|quinn ovid|2013-03-01 09:11:58.703071|53.61|american history +122|293|65734|4294967364|34.80|13.81|false|ethan white|2013-03-01 09:11:58.703180|15.54|values clariffication +87|426|65583|4294967338|61.69|32.80|false|rachel king|2013-03-01 09:11:58.703154|91.89|debate +97|274|65688|4294967469|1.77|39.79|true|bob robinson|2013-03-01 09:11:58.703268|97.20|education +89|409|65718|4294967413|20.19|30.31|true|rachel polk|2013-03-01 09:11:58.703283|93.84|chemistry +85|491|65667|4294967441|48.73|9.27|true|holly underhill|2013-03-01 09:11:58.703231|36.82|industrial engineering +83|452|65768|4294967342|29.59|37.71|false|zach zipper|2013-03-01 09:11:58.703321|31.29|kindergarten +10|289|65615|4294967507|88.55|0.71|true|quinn falkner|2013-03-01 09:11:58.703136|42.20|opthamology +103|496|65624|4294967391|65.39|16.28|false|quinn carson|2013-03-01 09:11:58.703197|25.58|industrial engineering +78|317|65686|4294967479|71.80|21.70|false|mike ovid|2013-03-01 09:11:58.703125|96.25|opthamology +77|486|65747|4294967445|17.36|41.14|false|holly hernandez|2013-03-01 09:11:58.703323|39.88|industrial engineering +12|386|65711|4294967357|18.45|21.11|true|fred underhill|2013-03-01 09:11:58.703278|72.82|american history +30|438|65564|4294967307|99.01|45.67|false|ethan falkner|2013-03-01 09:11:58.703216|55.22|history +123|511|65588|4294967424|78.85|6.77|true|mike davidson|2013-03-01 09:11:58.703078|48.29|quiet hour +44|450|65578|4294967402|86.36|36.78|false|bob ovid|2013-03-01 09:11:58.703150|50.51|undecided +88|342|65661|4294967323|99.34|37.43|true|nick young|2013-03-01 09:11:58.703191|49.70|chemistry +38|369|65546|4294967482|17.63|45.14|false|calvin hernandez|2013-03-01 09:11:58.703129|34.87|education +110|507|65732|4294967436|72.96|14.89|false|yuri thompson|2013-03-01 09:11:58.703304|44.35|religion +33|275|65578|4294967519|38.85|21.08|true|xavier steinbeck|2013-03-01 09:11:58.703124|5.49|joggying +109|300|65729|4294967460|83.05|1.35|false|jessica young|2013-03-01 09:11:58.703231|86.34|wind surfing +95|390|65641|4294967456|6.00|1.13|false|mike garcia|2013-03-01 09:11:58.703298|25.35|american history +26|300|65759|4294967341|96.11|28.68|false|ulysses ovid|2013-03-01 09:11:58.703196|92.95|philosophy +55|384|65718|4294967341|51.69|1.54|false|sarah hernandez|2013-03-01 09:11:58.703264|34.07|forestry +92|403|65547|4294967368|31.36|49.72|true|gabriella young|2013-03-01 09:11:58.703077|28.31|xylophone band +105|313|65696|4294967521|93.51|43.46|false|jessica davidson|2013-03-01 09:11:58.703110|20.13|undecided +121|339|65681|4294967380|26.98|7.74|true|ulysses davidson|2013-03-01 09:11:58.703090|22.88|quiet hour +15|498|65705|4294967346|2.40|30.10|false|tom robinson|2013-03-01 09:11:58.703273|95.20|joggying +111|280|65596|4294967454|37.62|18.11|false|calvin king|2013-03-01 09:11:58.703263|55.60|debate +94|414|65690|4294967454|62.34|29.42|true|zach laertes|2013-03-01 09:11:58.703156|20.11|quiet hour +106|315|65583|4294967458|34.13|16.45|false|tom johnson|2013-03-01 09:11:58.703171|70.58|topology +4|336|65653|4294967538|37.41|15.33|true|ulysses thompson|2013-03-01 09:11:58.703210|38.25|nap time +50|291|65749|4294967392|34.87|29.92|false|oscar carson|2013-03-01 09:11:58.703316|91.80|industrial engineering +88|362|65561|4294967349|19.37|38.21|true|wendy ellison|2013-03-01 09:11:58.703271|74.73|undecided +44|294|65627|4294967511|78.02|22.92|true|mike brown|2013-03-01 09:11:58.703093|38.49|xylophone band +32|452|65779|4294967360|56.91|29.68|false|katie falkner|2013-03-01 09:11:58.703178|59.55|chemistry +23|371|65772|4294967485|46.40|5.20|true|ulysses white|2013-03-01 09:11:58.703178|29.27|study skills +2|505|65626|4294967364|68.93|48.16|false|oscar thompson|2013-03-01 09:11:58.703278|29.54|topology +48|332|65557|4294967359|80.27|19.41|true|mike garcia|2013-03-01 09:11:58.703121|95.54|quiet hour +116|480|65693|4294967364|15.97|40.58|true|gabriella xylophone|2013-03-01 09:11:58.703253|57.26|wind surfing +55|289|65617|4294967497|59.42|46.10|true|fred xylophone|2013-03-01 09:11:58.703147|4.31|nap time +119|257|65771|4294967327|48.50|31.85|true|zach hernandez|2013-03-01 09:11:58.703323|45.45|opthamology +108|369|65759|4294967505|3.32|27.71|true|ethan ichabod|2013-03-01 09:11:58.703080|16.63|philosophy +107|333|65664|4294967454|66.69|6.33|false|nick hernandez|2013-03-01 09:11:58.703143|12.63|philosophy +58|349|65683|4294967513|27.93|18.43|true|jessica falkner|2013-03-01 09:11:58.703226|81.16|quiet hour +-1|447|65713|4294967479|14.54|16.45|true|jessica laertes|2013-03-01 09:11:58.703070|79.99|quiet hour +28|487|65630|4294967335|9.06|14.71|true|xavier steinbeck|2013-03-01 09:11:58.703133|82.24|geology +111|497|65694|4294967392|90.69|40.84|true|sarah nixon|2013-03-01 09:11:58.703185|50.00|history +13|354|65753|4294967322|35.40|10.85|true|mike garcia|2013-03-01 09:11:58.703136|28.92|study skills +50|399|65631|4294967336|90.46|19.23|true|jessica miller|2013-03-01 09:11:58.703180|45.67|linguistics +106|411|65790|4294967417|89.69|6.41|true|rachel polk|2013-03-01 09:11:58.703289|84.90|linguistics +52|412|65706|4294967320|96.84|13.16|true|jessica falkner|2013-03-01 09:11:58.703220|53.61|religion +64|324|65743|4294967460|73.40|16.31|true|ethan nixon|2013-03-01 09:11:58.703261|0.38|kindergarten +55|332|65781|4294967416|13.93|30.84|false|alice brown|2013-03-01 09:11:58.703104|85.82|mathematics +31|494|65662|4294967366|89.23|0.97|false|irene king|2013-03-01 09:11:58.703239|13.86|nap time +76|427|65581|4294967457|83.65|10.65|false|oscar davidson|2013-03-01 09:11:58.703125|43.75|values clariffication +-1|415|65555|4294967518|16.76|42.50|true|david falkner|2013-03-01 09:11:58.703278|46.73|quiet hour +7|486|65785|4294967337|45.56|31.21|false|katie davidson|2013-03-01 09:11:58.703084|33.04|yard duty +26|375|65648|4294967389|58.86|43.82|true|holly zipper|2013-03-01 09:11:58.703142|14.54|undecided +13|322|65697|4294967433|91.06|6.96|false|calvin robinson|2013-03-01 09:11:58.703220|40.11|history +33|325|65757|4294967544|1.36|20.52|false|xavier van buren|2013-03-01 09:11:58.703225|27.07|xylophone band +22|498|65669|4294967479|26.10|35.50|false|zach davidson|2013-03-01 09:11:58.703233|40.50|education +114|261|65550|4294967395|14.78|40.47|true|fred ellison|2013-03-01 09:11:58.703249|44.21|values clariffication +40|256|65789|4294967333|91.12|38.53|true|sarah brown|2013-03-01 09:11:58.703080|64.88|education +21|367|65766|4294967496|35.20|43.78|false|ethan carson|2013-03-01 09:11:58.703133|96.83|education +10|331|65633|4294967328|94.32|30.17|false|quinn zipper|2013-03-01 09:11:58.703127|1.50|undecided +62|359|65774|4294967313|91.58|45.00|true|wendy underhill|2013-03-01 09:11:58.703089|70.27|forestry +77|335|65730|4294967479|92.23|5.81|true|alice ellison|2013-03-01 09:11:58.703222|63.65|values clariffication +112|281|65702|4294967381|98.79|31.38|true|jessica garcia|2013-03-01 09:11:58.703277|83.93|quiet hour +51|491|65727|4294967305|2.57|21.59|true|calvin xylophone|2013-03-01 09:11:58.703089|97.15|yard duty +29|472|65664|4294967494|9.75|25.02|false|sarah zipper|2013-03-01 09:11:58.703127|82.89|biology +103|434|65684|4294967354|56.58|27.95|false|ulysses zipper|2013-03-01 09:11:58.703222|46.42|education +8|387|65673|4294967514|15.58|22.76|true|quinn ovid|2013-03-01 09:11:58.703086|86.86|yard duty +16|407|65617|4294967338|92.17|32.11|true|alice polk|2013-03-01 09:11:58.703300|37.88|industrial engineering +80|294|65617|4294967337|0.59|15.98|true|wendy ichabod|2013-03-01 09:11:58.703077|18.17|philosophy +21|439|65746|4294967469|55.37|42.14|false|nick underhill|2013-03-01 09:11:58.703176|76.95|chemistry +50|262|65706|4294967309|98.41|23.13|false|nick garcia|2013-03-01 09:11:58.703210|53.40|wind surfing +110|271|65737|4294967298|15.81|41.04|true|gabriella van buren|2013-03-01 09:11:58.703324|88.58|values clariffication +45|462|65673|4294967533|56.85|40.48|false|bob ovid|2013-03-01 09:11:58.703234|13.56|geology +43|366|65744|4294967303|11.33|1.29|false|alice garcia|2013-03-01 09:11:58.703178|62.65|undecided +89|435|65575|4294967542|2.70|31.25|true|nick davidson|2013-03-01 09:11:58.703280|24.30|chemistry +42|467|65548|4294967317|19.02|30.64|false|holly nixon|2013-03-01 09:11:58.703188|92.88|religion +-3|455|65570|4294967304|2.48|30.76|false|alice king|2013-03-01 09:11:58.703140|41.87|forestry +31|390|65695|4294967317|3.36|36.34|true|xavier underhill|2013-03-01 09:11:58.703277|23.24|philosophy +43|357|65595|4294967457|42.41|44.15|true|david robinson|2013-03-01 09:11:58.703304|50.55|yard duty +74|441|65683|4294967332|17.70|9.41|false|katie allen|2013-03-01 09:11:58.703085|36.74|topology +96|422|65727|4294967508|99.93|4.83|true|victor young|2013-03-01 09:11:58.703305|95.46|opthamology +110|380|65668|4294967546|32.65|11.81|false|oscar brown|2013-03-01 09:11:58.703314|40.59|quiet hour +92|334|65682|4294967405|49.44|45.51|true|gabriella quirinius|2013-03-01 09:11:58.703167|18.51|study skills +83|466|65747|4294967325|34.11|47.70|false|calvin allen|2013-03-01 09:11:58.703226|16.49|undecided +113|336|65578|4294967296|38.23|9.36|false|david ovid|2013-03-01 09:11:58.703160|16.94|geology +29|369|65672|4294967317|75.10|15.88|false|wendy ichabod|2013-03-01 09:11:58.703173|17.52|biology +5|285|65682|4294967509|82.49|46.98|true|xavier allen|2013-03-01 09:11:58.703179|41.69|quiet hour +71|367|65697|4294967491|78.06|23.16|false|priscilla ichabod|2013-03-01 09:11:58.703096|1.68|nap time +15|377|65771|4294967369|70.28|4.98|true|holly thompson|2013-03-01 09:11:58.703102|98.12|linguistics +27|402|65735|4294967411|69.54|38.66|true|mike carson|2013-03-01 09:11:58.703248|98.51|linguistics +88|312|65683|4294967533|16.16|14.44|false|ethan xylophone|2013-03-01 09:11:58.703084|3.36|history +105|316|65766|4294967362|85.78|35.95|true|calvin quirinius|2013-03-01 09:11:58.703290|7.90|yard duty +72|390|65641|4294967364|92.73|32.01|false|jessica ovid|2013-03-01 09:11:58.703220|93.40|history +111|318|65639|4294967339|16.34|4.91|false|calvin young|2013-03-01 09:11:58.703210|29.73|opthamology +108|446|65588|4294967531|66.15|23.33|true|calvin hernandez|2013-03-01 09:11:58.703119|58.68|religion +122|442|65569|4294967349|53.28|16.09|true|irene robinson|2013-03-01 09:11:58.703184|19.87|history +8|429|65682|4294967514|86.73|7.17|false|gabriella ellison|2013-03-01 09:11:58.703323|19.30|yard duty +66|309|65628|4294967345|86.10|39.85|true|jessica ichabod|2013-03-01 09:11:58.703114|61.36|topology +29|329|65611|4294967412|31.46|0.08|false|jessica white|2013-03-01 09:11:58.703119|30.70|topology +78|409|65670|4294967491|26.44|8.06|true|fred robinson|2013-03-01 09:11:58.703104|29.88|topology +72|350|65545|4294967333|63.03|14.50|false|gabriella nixon|2013-03-01 09:11:58.703134|36.52|debate +86|395|65634|4294967438|63.01|9.39|false|mike miller|2013-03-01 09:11:58.703251|19.91|xylophone band +1|401|65578|4294967449|59.41|18.86|false|sarah young|2013-03-01 09:11:58.703135|58.90|american history +36|442|65724|4294967537|35.03|30.02|false|alice van buren|2013-03-01 09:11:58.703107|82.08|history +30|328|65588|4294967500|79.62|15.93|true|david allen|2013-03-01 09:11:58.703074|36.06|nap time +76|312|65739|4294967300|72.40|38.30|false|fred carson|2013-03-01 09:11:58.703092|3.11|philosophy +54|382|65574|4294967459|64.33|35.50|true|wendy ichabod|2013-03-01 09:11:58.703230|84.44|linguistics +105|465|65662|4294967354|82.51|17.61|false|katie laertes|2013-03-01 09:11:58.703320|37.55|linguistics +52|407|65740|4294967334|25.47|19.64|true|calvin xylophone|2013-03-01 09:11:58.703318|75.80|religion +9|357|65566|4294967437|67.22|22.75|true|bob falkner|2013-03-01 09:11:58.703267|75.33|history +67|318|65692|4294967337|55.82|2.95|false|david van buren|2013-03-01 09:11:58.703218|81.02|opthamology +9|504|65713|4294967309|29.04|24.84|false|tom laertes|2013-03-01 09:11:58.703204|0.64|quiet hour +52|302|65721|4294967462|88.55|6.81|false|sarah king|2013-03-01 09:11:58.703230|39.15|geology +84|474|65682|4294967355|72.97|15.78|true|irene ichabod|2013-03-01 09:11:58.703118|66.10|mathematics +8|367|65773|4294967307|57.48|25.10|true|ulysses laertes|2013-03-01 09:11:58.703220|65.37|xylophone band +64|423|65708|4294967510|29.52|16.92|false|zach young|2013-03-01 09:11:58.703280|71.65|education +-1|257|65631|4294967395|51.61|29.85|true|jessica laertes|2013-03-01 09:11:58.703322|12.12|education +73|317|65758|4294967405|15.74|26.98|false|bob young|2013-03-01 09:11:58.703287|99.53|religion +76|373|65578|4294967373|21.78|18.27|false|david van buren|2013-03-01 09:11:58.703121|46.36|topology +67|487|65720|4294967444|13.96|2.69|true|bob laertes|2013-03-01 09:11:58.703220|16.17|mathematics +123|465|65694|4294967499|84.01|7.64|false|xavier allen|2013-03-01 09:11:58.703286|0.46|opthamology +110|495|65542|4294967381|82.89|40.00|true|irene davidson|2013-03-01 09:11:58.703167|7.36|joggying +18|395|65630|4294967532|72.17|12.26|false|bob davidson|2013-03-01 09:11:58.703192|96.66|mathematics +117|427|65562|4294967354|22.25|16.41|false|ethan brown|2013-03-01 09:11:58.703301|66.27|biology +8|492|65589|4294967487|22.99|5.92|false|jessica hernandez|2013-03-01 09:11:58.703231|33.14|mathematics +77|441|65789|4294967373|91.12|14.29|true|luke falkner|2013-03-01 09:11:58.703085|3.34|biology +12|291|65715|4294967334|28.18|35.26|false|victor quirinius|2013-03-01 09:11:58.703081|56.04|education +81|425|65570|4294967370|30.30|21.48|true|alice brown|2013-03-01 09:11:58.703266|93.78|religion +43|443|65613|4294967468|62.77|3.44|true|gabriella steinbeck|2013-03-01 09:11:58.703325|23.84|forestry +33|430|65620|4294967411|61.40|49.57|true|ethan nixon|2013-03-01 09:11:58.703206|21.78|xylophone band +24|421|65769|4294967517|94.25|39.43|true|ethan polk|2013-03-01 09:11:58.703094|0.11|religion +94|295|65550|4294967419|29.95|20.99|true|fred polk|2013-03-01 09:11:58.703131|20.97|study skills +57|310|65608|4294967405|94.75|6.85|true|tom falkner|2013-03-01 09:11:58.703234|86.85|philosophy +56|470|65621|4294967431|25.42|4.32|true|mike ellison|2013-03-01 09:11:58.703268|47.83|philosophy +81|428|65755|4294967541|18.43|36.54|true|luke laertes|2013-03-01 09:11:58.703302|33.77|kindergarten +47|389|65732|4294967368|49.43|22.64|false|quinn underhill|2013-03-01 09:11:58.703205|58.71|chemistry +104|396|65583|4294967551|67.22|40.02|true|katie king|2013-03-01 09:11:58.703320|28.92|debate +80|405|65577|4294967306|13.17|22.10|true|gabriella nixon|2013-03-01 09:11:58.703209|92.38|philosophy +115|319|65782|4294967533|51.54|25.51|false|katie brown|2013-03-01 09:11:58.703258|93.03|mathematics +-2|389|65706|4294967488|26.68|17.93|false|alice underhill|2013-03-01 09:11:58.703122|87.47|forestry +48|405|65620|4294967368|58.01|45.84|false|priscilla nixon|2013-03-01 09:11:58.703256|63.41|linguistics +96|489|65543|4294967315|96.37|31.72|true|fred underhill|2013-03-01 09:11:58.703283|89.25|topology +78|435|65782|4294967301|62.77|6.11|true|jessica king|2013-03-01 09:11:58.703090|64.41|undecided +30|408|65574|4294967546|72.68|48.10|true|david young|2013-03-01 09:11:58.703151|21.41|wind surfing +46|481|65780|4294967443|78.34|1.11|false|nick brown|2013-03-01 09:11:58.703200|18.37|chemistry +74|483|65657|4294967386|87.05|0.18|false|nick davidson|2013-03-01 09:11:58.703094|17.39|industrial engineering +72|299|65605|4294967475|92.62|35.72|false|alice xylophone|2013-03-01 09:11:58.703257|55.87|education +27|428|65675|4294967550|86.64|41.73|false|zach nixon|2013-03-01 09:11:58.703129|20.65|linguistics +72|376|65547|4294967413|51.19|7.48|false|david steinbeck|2013-03-01 09:11:58.703312|35.06|topology +33|506|65581|4294967406|16.31|29.38|true|gabriella underhill|2013-03-01 09:11:58.703130|75.88|chemistry +112|489|65729|4294967361|48.89|48.50|true|yuri carson|2013-03-01 09:11:58.703104|78.15|geology +120|323|65655|4294967300|10.53|21.59|false|priscilla underhill|2013-03-01 09:11:58.703158|42.11|joggying +98|411|65677|4294967343|32.73|34.24|false|jessica ichabod|2013-03-01 09:11:58.703321|78.14|linguistics +81|300|65783|4294967327|42.49|0.50|false|victor falkner|2013-03-01 09:11:58.703235|64.40|values clariffication +97|487|65763|4294967498|76.83|18.75|false|alice quirinius|2013-03-01 09:11:58.703295|20.61|philosophy +53|471|65778|4294967391|34.67|40.57|true|oscar young|2013-03-01 09:11:58.703090|44.09|linguistics +56|283|65723|4294967520|70.01|0.92|true|zach hernandez|2013-03-01 09:11:58.703156|4.39|chemistry +0|498|65767|4294967420|63.57|32.10|false|david underhill|2013-03-01 09:11:58.703276|46.90|forestry +51|353|65646|4294967548|58.94|12.81|false|nick zipper|2013-03-01 09:11:58.703174|30.49|kindergarten +57|459|65734|4294967422|91.84|19.18|true|alice miller|2013-03-01 09:11:58.703260|84.80|study skills +104|281|65626|4294967551|13.30|0.17|false|katie brown|2013-03-01 09:11:58.703147|61.05|undecided +74|395|65698|4294967304|33.44|30.82|false|david van buren|2013-03-01 09:11:58.703109|1.36|mathematics +118|335|65587|4294967367|25.71|36.38|false|tom ichabod|2013-03-01 09:11:58.703144|29.68|history +23|360|65771|4294967488|1.76|19.64|true|calvin davidson|2013-03-01 09:11:58.703126|80.11|nap time +1|321|65764|4294967298|26.47|24.35|false|irene nixon|2013-03-01 09:11:58.703162|10.66|industrial engineering +93|294|65624|4294967537|6.44|25.74|false|victor hernandez|2013-03-01 09:11:58.703156|83.10|education +67|501|65674|4294967406|82.79|46.56|true|fred van buren|2013-03-01 09:11:58.703220|27.81|mathematics +9|439|65787|4294967325|53.18|3.99|false|calvin zipper|2013-03-01 09:11:58.703208|10.72|opthamology +100|469|65554|4294967323|63.81|20.97|true|xavier robinson|2013-03-01 09:11:58.703139|3.52|mathematics +96|484|65545|4294967410|59.98|40.13|true|bob miller|2013-03-01 09:11:58.703325|54.64|xylophone band +18|270|65677|4294967448|85.69|10.43|true|luke van buren|2013-03-01 09:11:58.703109|15.79|mathematics +75|299|65536|4294967510|35.69|36.12|true|ethan underhill|2013-03-01 09:11:58.703277|94.09|opthamology +124|280|65611|4294967422|6.52|15.49|false|gabriella miller|2013-03-01 09:11:58.703230|44.47|american history +114|445|65536|4294967478|69.79|16.96|false|mike quirinius|2013-03-01 09:11:58.703247|59.67|linguistics +34|301|65591|4294967386|17.71|4.53|true|yuri steinbeck|2013-03-01 09:11:58.703323|48.76|industrial engineering +8|359|65568|4294967473|58.25|0.31|true|nick white|2013-03-01 09:11:58.703075|61.20|xylophone band +122|493|65756|4294967445|77.35|1.74|true|sarah carson|2013-03-01 09:11:58.703076|25.51|history +82|455|65699|4294967512|37.34|26.64|false|yuri brown|2013-03-01 09:11:58.703237|43.95|chemistry +104|351|65777|4294967382|52.12|12.10|false|bob white|2013-03-01 09:11:58.703297|52.33|zync studies +43|416|65773|4294967408|16.72|21.04|true|ethan carson|2013-03-01 09:11:58.703266|3.46|geology +55|450|65536|4294967307|76.69|13.26|false|gabriella underhill|2013-03-01 09:11:58.703168|20.50|zync studies +96|361|65539|4294967508|94.18|48.41|true|quinn carson|2013-03-01 09:11:58.703173|50.85|yard duty +16|305|65591|4294967403|4.29|37.37|false|jessica miller|2013-03-01 09:11:58.703075|11.28|study skills +22|354|65636|4294967436|97.34|31.85|true|luke thompson|2013-03-01 09:11:58.703296|80.08|industrial engineering +64|274|65567|4294967458|8.51|0.99|true|wendy xylophone|2013-03-01 09:11:58.703322|63.72|religion +46|362|65753|4294967443|32.63|28.93|false|wendy nixon|2013-03-01 09:11:58.703213|44.26|industrial engineering +67|402|65615|4294967437|4.69|18.72|true|irene laertes|2013-03-01 09:11:58.703131|24.49|religion +78|433|65753|4294967392|28.36|2.78|true|ulysses steinbeck|2013-03-01 09:11:58.703184|0.97|mathematics +107|273|65747|4294967344|89.44|5.57|false|tom thompson|2013-03-01 09:11:58.703152|58.54|kindergarten +17|511|65739|4294967388|37.62|34.42|true|tom underhill|2013-03-01 09:11:58.703122|36.58|philosophy +-2|327|65617|4294967492|48.74|35.18|true|ulysses robinson|2013-03-01 09:11:58.703129|35.32|nap time +87|475|65640|4294967370|16.92|49.31|false|bob carson|2013-03-01 09:11:58.703272|25.53|mathematics +42|355|65669|4294967368|59.44|24.61|false|luke underhill|2013-03-01 09:11:58.703287|87.86|history +65|310|65581|4294967378|19.12|24.97|true|ulysses ichabod|2013-03-01 09:11:58.703257|24.65|opthamology +107|478|65765|4294967533|4.09|12.86|false|katie davidson|2013-03-01 09:11:58.703262|71.14|chemistry +24|360|65687|4294967420|33.23|7.39|true|holly white|2013-03-01 09:11:58.703170|42.52|linguistics +78|478|65735|4294967377|37.37|38.27|true|yuri ichabod|2013-03-01 09:11:58.703105|22.68|topology +29|357|65687|4294967505|69.88|42.24|true|yuri thompson|2013-03-01 09:11:58.703087|61.11|mathematics +75|342|65671|4294967350|81.52|0.93|false|alice steinbeck|2013-03-01 09:11:58.703226|51.32|undecided +58|504|65618|4294967411|29.26|22.21|false|irene carson|2013-03-01 09:11:58.703263|21.25|topology +38|435|65715|4294967338|89.78|18.48|true|tom carson|2013-03-01 09:11:58.703102|50.29|kindergarten +111|377|65544|4294967505|98.07|14.29|false|jessica quirinius|2013-03-01 09:11:58.703220|14.18|linguistics +36|502|65584|4294967410|0.05|48.79|false|david davidson|2013-03-01 09:11:58.703113|47.35|mathematics +35|282|65640|4294967372|3.96|10.68|false|yuri polk|2013-03-01 09:11:58.703256|49.35|zync studies +110|405|65779|4294967439|6.33|10.16|true|oscar garcia|2013-03-01 09:11:58.703171|68.74|zync studies +7|344|65643|4294967444|9.35|43.76|true|bob thompson|2013-03-01 09:11:58.703125|93.49|topology +-3|500|65704|4294967480|2.26|28.79|true|mike polk|2013-03-01 09:11:58.703190|4.33|nap time +58|325|65573|4294967429|52.15|19.93|false|alice garcia|2013-03-01 09:11:58.703286|97.26|industrial engineering +53|429|65595|4294967517|0.29|34.74|true|zach hernandez|2013-03-01 09:11:58.703250|18.74|american history +61|419|65714|4294967386|38.15|8.04|false|rachel brown|2013-03-01 09:11:58.703196|44.59|chemistry +27|499|65782|4294967548|0.69|11.42|false|ethan quirinius|2013-03-01 09:11:58.703136|57.64|debate +79|402|65734|4294967498|59.23|17.01|false|irene allen|2013-03-01 09:11:58.703225|74.81|xylophone band +116|356|65628|4294967542|77.23|29.26|false|zach carson|2013-03-01 09:11:58.703117|49.87|debate +3|398|65768|4294967506|52.53|12.73|true|fred johnson|2013-03-01 09:11:58.703220|40.12|wind surfing +75|381|65704|4294967423|61.05|12.27|false|yuri davidson|2013-03-01 09:11:58.703209|3.87|opthamology +35|490|65750|4294967518|28.72|7.05|false|xavier garcia|2013-03-01 09:11:58.703272|70.02|forestry +59|368|65594|4294967374|40.50|46.64|true|holly xylophone|2013-03-01 09:11:58.703313|42.26|history +72|423|65765|4294967425|82.48|2.09|true|yuri miller|2013-03-01 09:11:58.703209|20.08|joggying +114|501|65759|4294967379|53.24|42.52|false|alice johnson|2013-03-01 09:11:58.703197|9.95|zync studies +37|428|65711|4294967506|54.86|39.93|false|jessica thompson|2013-03-01 09:11:58.703140|85.44|education +110|405|65602|4294967419|26.77|5.05|true|ethan quirinius|2013-03-01 09:11:58.703076|30.37|wind surfing +19|432|65670|4294967404|46.06|42.53|true|jessica zipper|2013-03-01 09:11:58.703140|59.19|education +37|431|65634|4294967376|26.00|7.38|true|rachel nixon|2013-03-01 09:11:58.703252|69.49|philosophy +18|279|65715|4294967301|77.28|27.96|true|bob zipper|2013-03-01 09:11:58.703295|12.11|yard duty +60|268|65589|4294967483|89.97|37.13|true|victor laertes|2013-03-01 09:11:58.703281|0.42|mathematics +107|498|65681|4294967455|73.15|8.19|true|jessica hernandez|2013-03-01 09:11:58.703080|63.94|xylophone band +64|287|65577|4294967334|86.00|9.36|false|mike laertes|2013-03-01 09:11:58.703130|40.29|philosophy +99|494|65788|4294967465|79.94|9.45|true|quinn king|2013-03-01 09:11:58.703130|6.61|industrial engineering +122|398|65634|4294967379|76.99|2.01|false|ethan van buren|2013-03-01 09:11:58.703225|57.71|industrial engineering +34|426|65716|4294967406|51.33|9.39|true|oscar laertes|2013-03-01 09:11:58.703186|27.06|religion +12|346|65563|4294967429|87.25|25.52|true|gabriella davidson|2013-03-01 09:11:58.703297|22.02|history +112|295|65672|4294967504|21.31|5.67|false|mike ellison|2013-03-01 09:11:58.703260|61.24|chemistry +93|355|65699|4294967488|63.57|35.92|false|alice ellison|2013-03-01 09:11:58.703222|85.67|geology +84|373|65592|4294967442|43.46|18.09|true|bob ovid|2013-03-01 09:11:58.703130|92.22|opthamology +36|338|65774|4294967408|61.74|40.77|false|fred allen|2013-03-01 09:11:58.703251|58.07|industrial engineering +97|411|65589|4294967336|8.09|31.19|false|jessica nixon|2013-03-01 09:11:58.703073|82.59|nap time +-2|511|65712|4294967313|43.61|29.15|false|fred king|2013-03-01 09:11:58.703268|42.74|chemistry +1|473|65630|4294967413|51.72|6.17|true|ethan johnson|2013-03-01 09:11:58.703141|6.63|study skills +101|395|65641|4294967355|47.12|31.36|true|rachel ellison|2013-03-01 09:11:58.703223|70.51|debate +51|256|65540|4294967543|49.44|29.95|true|wendy garcia|2013-03-01 09:11:58.703191|18.85|mathematics +80|257|65712|4294967533|90.44|30.33|true|ethan polk|2013-03-01 09:11:58.703194|68.90|philosophy +72|338|65660|4294967361|27.58|25.64|true|zach young|2013-03-01 09:11:58.703261|64.29|topology +85|304|65662|4294967508|92.30|5.53|false|sarah miller|2013-03-01 09:11:58.703198|62.20|debate +20|441|65650|4294967304|53.80|22.20|false|priscilla white|2013-03-01 09:11:58.703322|1.00|education +68|437|65538|4294967321|33.21|40.77|true|luke white|2013-03-01 09:11:58.703152|98.11|opthamology +103|397|65622|4294967463|52.34|21.60|true|rachel robinson|2013-03-01 09:11:58.703129|67.66|industrial engineering +31|456|65766|4294967355|46.79|20.56|true|calvin johnson|2013-03-01 09:11:58.703242|44.94|zync studies +28|296|65776|4294967421|55.61|9.63|true|katie robinson|2013-03-01 09:11:58.703222|88.06|biology +-1|335|65717|4294967415|16.91|24.69|true|luke steinbeck|2013-03-01 09:11:58.703319|98.24|zync studies +82|412|65606|4294967383|37.91|31.97|true|bob carson|2013-03-01 09:11:58.703129|60.39|forestry +66|496|65616|4294967403|64.68|8.16|true|katie thompson|2013-03-01 09:11:58.703298|77.83|forestry +116|411|65721|4294967506|83.79|48.64|true|xavier thompson|2013-03-01 09:11:58.703291|31.83|xylophone band +30|408|65582|4294967526|39.29|23.30|false|jessica hernandez|2013-03-01 09:11:58.703084|84.93|biology +60|361|65574|4294967457|22.38|40.07|false|oscar underhill|2013-03-01 09:11:58.703189|9.01|religion +82|471|65594|4294967303|79.14|2.39|true|yuri ichabod|2013-03-01 09:11:58.703235|24.55|american history +42|275|65692|4294967420|9.25|14.31|false|calvin king|2013-03-01 09:11:58.703163|1.11|undecided +39|439|65761|4294967406|83.13|23.36|true|gabriella davidson|2013-03-01 09:11:58.703074|80.99|chemistry +36|428|65565|4294967495|59.97|8.65|true|calvin falkner|2013-03-01 09:11:58.703268|36.10|quiet hour +43|369|65544|4294967425|28.26|27.11|false|calvin nixon|2013-03-01 09:11:58.703235|63.83|linguistics +123|343|65641|4294967530|33.52|40.05|false|rachel van buren|2013-03-01 09:11:58.703128|30.84|debate +105|337|65577|4294967455|87.07|43.30|true|wendy johnson|2013-03-01 09:11:58.703238|17.58|undecided +1|280|65728|4294967528|30.08|36.91|true|ulysses ichabod|2013-03-01 09:11:58.703230|32.89|debate +51|389|65789|4294967463|38.04|12.19|true|yuri davidson|2013-03-01 09:11:58.703138|11.90|geology +27|305|65568|4294967512|5.52|30.15|false|xavier carson|2013-03-01 09:11:58.703311|86.25|debate +44|302|65696|4294967316|23.59|12.46|false|bob carson|2013-03-01 09:11:58.703301|98.89|xylophone band +103|412|65562|4294967441|14.87|43.81|false|ethan young|2013-03-01 09:11:58.703197|3.81|nap time +85|287|65554|4294967488|93.54|20.30|true|nick allen|2013-03-01 09:11:58.703239|40.96|topology +7|446|65568|4294967358|78.72|5.57|true|quinn ellison|2013-03-01 09:11:58.703243|57.39|education +37|280|65675|4294967422|77.19|31.59|true|ulysses falkner|2013-03-01 09:11:58.703280|76.73|industrial engineering +120|477|65692|4294967391|20.21|44.19|false|fred brown|2013-03-01 09:11:58.703227|20.97|opthamology +1|257|65771|4294967451|53.89|28.77|true|yuri nixon|2013-03-01 09:11:58.703253|2.70|religion +31|299|65748|4294967369|45.66|5.05|false|nick hernandez|2013-03-01 09:11:58.703301|24.10|debate +101|425|65578|4294967473|41.47|15.15|true|oscar allen|2013-03-01 09:11:58.703087|6.41|history +55|333|65540|4294967363|39.35|46.32|false|sarah hernandez|2013-03-01 09:11:58.703293|91.07|american history +41|284|65615|4294967377|92.10|23.09|false|nick steinbeck|2013-03-01 09:11:58.703092|90.74|biology +88|330|65716|4294967322|12.04|26.80|false|ulysses polk|2013-03-01 09:11:58.703090|56.83|religion +99|305|65554|4294967428|78.19|28.49|true|gabriella robinson|2013-03-01 09:11:58.703117|80.59|undecided +98|380|65714|4294967357|84.17|11.98|false|victor steinbeck|2013-03-01 09:11:58.703288|62.59|debate +65|311|65774|4294967428|65.52|21.69|true|xavier johnson|2013-03-01 09:11:58.703316|1.20|chemistry +12|479|65687|4294967377|97.43|8.03|true|sarah garcia|2013-03-01 09:11:58.703223|43.43|quiet hour +30|506|65748|4294967437|6.82|45.57|false|zach brown|2013-03-01 09:11:58.703196|62.39|zync studies +15|398|65551|4294967487|33.42|32.99|false|yuri brown|2013-03-01 09:11:58.703148|26.00|kindergarten +122|308|65627|4294967404|47.86|13.57|true|tom ellison|2013-03-01 09:11:58.703149|75.02|kindergarten +108|389|65698|4294967444|74.75|2.41|true|fred young|2013-03-01 09:11:58.703174|57.21|undecided +71|391|65691|4294967302|17.88|15.20|false|oscar ellison|2013-03-01 09:11:58.703204|64.38|values clariffication +9|504|65761|4294967547|72.67|40.19|false|ulysses van buren|2013-03-01 09:11:58.703201|84.73|kindergarten +0|440|65661|4294967548|78.43|3.34|true|bob white|2013-03-01 09:11:58.703264|31.44|education +90|404|65623|4294967437|85.60|27.86|false|oscar young|2013-03-01 09:11:58.703286|6.43|values clariffication +15|403|65677|4294967401|28.50|36.46|false|jessica laertes|2013-03-01 09:11:58.703101|44.85|education +95|474|65734|4294967490|25.57|23.74|true|bob falkner|2013-03-01 09:11:58.703168|99.54|wind surfing +49|276|65766|4294967515|9.79|20.29|false|quinn johnson|2013-03-01 09:11:58.703162|26.27|mathematics +-2|361|65677|4294967330|93.54|19.61|false|quinn ovid|2013-03-01 09:11:58.703246|30.55|religion +103|505|65543|4294967427|23.45|6.37|false|rachel allen|2013-03-01 09:11:58.703268|17.92|education +91|434|65587|4294967508|0.95|5.85|false|ethan steinbeck|2013-03-01 09:11:58.703122|34.13|topology +86|430|65742|4294967327|59.32|11.02|true|priscilla nixon|2013-03-01 09:11:58.703287|94.12|kindergarten +13|488|65630|4294967521|62.52|36.03|true|holly ovid|2013-03-01 09:11:58.703194|33.39|wind surfing +7|441|65554|4294967318|50.07|15.35|false|luke steinbeck|2013-03-01 09:11:58.703272|98.91|chemistry +29|303|65684|4294967467|5.54|19.76|false|wendy van buren|2013-03-01 09:11:58.703169|54.72|geology +26|343|65698|4294967527|95.85|47.34|true|quinn thompson|2013-03-01 09:11:58.703131|58.56|kindergarten +91|312|65598|4294967500|72.38|27.25|false|bob underhill|2013-03-01 09:11:58.703209|68.49|quiet hour +58|464|65694|4294967374|60.48|28.69|true|katie ovid|2013-03-01 09:11:58.703145|31.50|american history +81|347|65678|4294967348|96.64|10.97|false|tom davidson|2013-03-01 09:11:58.703240|61.22|geology +118|335|65635|4294967369|16.99|43.81|false|zach johnson|2013-03-01 09:11:58.703270|19.14|kindergarten +108|499|65764|4294967318|8.78|38.84|true|victor falkner|2013-03-01 09:11:58.703146|16.87|chemistry +22|408|65630|4294967444|19.38|4.39|true|quinn garcia|2013-03-01 09:11:58.703133|68.09|topology +116|380|65647|4294967549|74.69|39.75|false|calvin zipper|2013-03-01 09:11:58.703203|52.89|geology +34|463|65609|4294967359|98.25|41.83|false|luke young|2013-03-01 09:11:58.703236|49.26|topology +96|339|65702|4294967422|89.45|35.77|true|nick underhill|2013-03-01 09:11:58.703279|22.44|religion +95|472|65635|4294967436|20.76|10.72|false|holly johnson|2013-03-01 09:11:58.703310|99.96|xylophone band +99|320|65605|4294967480|40.10|25.82|false|holly carson|2013-03-01 09:11:58.703298|56.07|american history +-2|360|65635|4294967420|19.54|29.40|false|sarah allen|2013-03-01 09:11:58.703304|63.93|chemistry +47|295|65642|4294967367|13.97|43.86|false|zach ichabod|2013-03-01 09:11:58.703256|75.32|education +103|455|65729|4294967369|82.08|18.85|true|xavier miller|2013-03-01 09:11:58.703234|58.62|joggying +109|302|65590|4294967347|63.97|39.13|false|david brown|2013-03-01 09:11:58.703156|66.84|industrial engineering +52|411|65583|4294967373|0.14|28.74|false|ulysses falkner|2013-03-01 09:11:58.703229|41.39|education +77|383|65641|4294967298|49.20|18.87|false|gabriella davidson|2013-03-01 09:11:58.703278|58.19|linguistics +-1|422|65757|4294967550|8.93|8.81|true|ulysses robinson|2013-03-01 09:11:58.703173|34.88|nap time +82|366|65551|4294967328|47.49|2.81|false|david van buren|2013-03-01 09:11:58.703138|86.27|study skills +87|352|65617|4294967298|97.90|45.71|true|jessica johnson|2013-03-01 09:11:58.703089|7.45|wind surfing +111|361|65615|4294967480|87.43|15.60|false|gabriella van buren|2013-03-01 09:11:58.703070|96.13|philosophy +12|431|65595|4294967490|48.94|39.24|true|ethan young|2013-03-01 09:11:58.703099|87.10|wind surfing +66|335|65570|4294967360|69.38|37.19|true|nick carson|2013-03-01 09:11:58.703304|50.97|education +124|397|65783|4294967414|53.50|38.18|false|mike ichabod|2013-03-01 09:11:58.703221|16.77|xylophone band +63|301|65649|4294967313|39.40|41.96|false|katie van buren|2013-03-01 09:11:58.703230|74.89|american history +-2|390|65753|4294967327|77.93|44.13|false|quinn ovid|2013-03-01 09:11:58.703168|35.10|yard duty +117|308|65763|4294967504|46.24|35.80|true|wendy king|2013-03-01 09:11:58.703106|86.79|religion +79|340|65680|4294967477|85.72|22.49|true|quinn polk|2013-03-01 09:11:58.703167|89.84|chemistry +15|478|65594|4294967440|49.22|45.49|false|sarah laertes|2013-03-01 09:11:58.703082|18.14|mathematics +15|353|65665|4294967437|86.95|32.03|false|ethan young|2013-03-01 09:11:58.703290|61.28|forestry +36|449|65769|4294967388|55.12|27.40|false|tom van buren|2013-03-01 09:11:58.703134|38.29|values clariffication +87|331|65746|4294967509|68.46|49.96|false|luke quirinius|2013-03-01 09:11:58.703151|89.77|debate +-1|309|65788|4294967309|43.40|42.32|true|katie king|2013-03-01 09:11:58.703090|38.61|xylophone band +73|406|65686|4294967314|63.33|25.87|false|ethan laertes|2013-03-01 09:11:58.703317|65.33|xylophone band +94|374|65592|4294967365|93.25|39.97|false|tom quirinius|2013-03-01 09:11:58.703174|51.74|education +68|430|65605|4294967427|45.52|47.71|false|katie zipper|2013-03-01 09:11:58.703250|47.85|kindergarten +109|508|65723|4294967533|51.03|10.99|false|bob quirinius|2013-03-01 09:11:58.703277|3.64|quiet hour +107|334|65596|4294967452|84.30|44.03|false|sarah laertes|2013-03-01 09:11:58.703137|44.09|wind surfing +38|274|65537|4294967446|74.24|45.57|true|alice ellison|2013-03-01 09:11:58.703163|14.48|mathematics +34|458|65747|4294967519|63.86|30.65|false|david falkner|2013-03-01 09:11:58.703148|71.82|american history +119|401|65720|4294967512|57.28|36.08|false|david white|2013-03-01 09:11:58.703260|80.21|nap time +119|384|65781|4294967521|66.42|35.56|false|yuri robinson|2013-03-01 09:11:58.703289|93.03|history +14|355|65585|4294967319|85.57|27.16|true|zach robinson|2013-03-01 09:11:58.703205|78.17|education +18|466|65539|4294967450|31.27|3.95|false|mike miller|2013-03-01 09:11:58.703153|95.93|education +7|322|65721|4294967519|77.24|49.14|true|tom johnson|2013-03-01 09:11:58.703165|49.75|topology +66|368|65791|4294967489|39.18|45.41|false|zach davidson|2013-03-01 09:11:58.703217|50.83|american history +32|363|65675|4294967449|56.89|21.12|false|zach xylophone|2013-03-01 09:11:58.703233|18.88|wind surfing +51|385|65762|4294967422|29.27|37.93|true|luke carson|2013-03-01 09:11:58.703080|56.63|topology +5|388|65635|4294967376|62.29|26.91|false|jessica brown|2013-03-01 09:11:58.703214|7.84|opthamology +3|359|65558|4294967440|36.21|47.51|true|mike robinson|2013-03-01 09:11:58.703292|33.50|values clariffication +107|505|65568|4294967540|13.71|1.76|true|quinn allen|2013-03-01 09:11:58.703209|11.69|religion +5|328|65749|4294967498|37.18|9.53|true|alice johnson|2013-03-01 09:11:58.703128|3.04|history +102|319|65569|4294967486|76.06|19.20|true|luke brown|2013-03-01 09:11:58.703183|89.32|chemistry +10|340|65642|4294967475|22.37|46.37|true|ulysses young|2013-03-01 09:11:58.703222|29.62|undecided +63|276|65627|4294967357|16.19|36.33|true|sarah thompson|2013-03-01 09:11:58.703294|22.38|geology +60|498|65721|4294967317|48.14|17.38|false|luke young|2013-03-01 09:11:58.703251|76.06|history +105|457|65641|4294967524|2.18|41.50|true|xavier xylophone|2013-03-01 09:11:58.703218|83.69|forestry +87|381|65764|4294967417|0.17|10.40|true|victor van buren|2013-03-01 09:11:58.703120|53.10|american history +84|491|65788|4294967325|7.51|2.06|false|sarah zipper|2013-03-01 09:11:58.703215|44.21|yard duty +19|505|65564|4294967445|82.93|39.64|true|david xylophone|2013-03-01 09:11:58.703209|15.15|history +4|317|65628|4294967403|70.73|29.67|true|ulysses allen|2013-03-01 09:11:58.703248|31.29|zync studies +111|403|65547|4294967335|63.73|12.34|false|oscar white|2013-03-01 09:11:58.703302|74.98|philosophy +-2|275|65714|4294967384|21.58|18.57|true|ulysses garcia|2013-03-01 09:11:58.703217|32.31|biology +75|503|65659|4294967296|32.51|16.62|false|quinn steinbeck|2013-03-01 09:11:58.703268|76.77|wind surfing +17|308|65553|4294967507|15.76|16.77|true|rachel carson|2013-03-01 09:11:58.703132|15.37|religion +118|436|65573|4294967428|93.88|15.02|false|xavier xylophone|2013-03-01 09:11:58.703322|80.76|geology +124|345|65589|4294967530|24.56|12.16|true|quinn robinson|2013-03-01 09:11:58.703125|13.49|religion +33|493|65723|4294967410|85.12|2.31|false|fred robinson|2013-03-01 09:11:58.703246|73.45|quiet hour +115|309|65546|4294967526|0.77|4.47|true|bob zipper|2013-03-01 09:11:58.703188|90.39|geology +41|306|65646|4294967423|12.44|2.97|true|jessica xylophone|2013-03-01 09:11:58.703108|24.70|xylophone band +81|357|65550|4294967372|89.86|40.57|false|alice polk|2013-03-01 09:11:58.703310|74.28|geology +59|339|65631|4294967547|18.81|41.87|true|oscar miller|2013-03-01 09:11:58.703075|67.50|nap time +11|327|65701|4294967432|74.42|13.87|true|zach king|2013-03-01 09:11:58.703289|11.62|kindergarten +118|303|65649|4294967462|39.43|23.07|true|calvin white|2013-03-01 09:11:58.703242|77.64|xylophone band +83|492|65644|4294967492|31.27|19.34|false|yuri ovid|2013-03-01 09:11:58.703211|54.73|biology +7|280|65568|4294967492|65.17|44.24|false|sarah allen|2013-03-01 09:11:58.703231|78.28|xylophone band +124|373|65665|4294967500|43.39|13.47|true|ethan king|2013-03-01 09:11:58.703128|56.45|education +18|362|65767|4294967429|21.12|9.62|true|rachel ovid|2013-03-01 09:11:58.703280|89.81|religion +65|333|65711|4294967509|43.73|3.70|true|victor van buren|2013-03-01 09:11:58.703095|0.97|american history +71|281|65778|4294967551|75.23|31.82|true|gabriella nixon|2013-03-01 09:11:58.703184|87.11|debate +73|331|65579|4294967445|74.80|33.90|true|luke polk|2013-03-01 09:11:58.703213|52.55|kindergarten +18|452|65703|4294967351|22.71|32.19|false|xavier king|2013-03-01 09:11:58.703229|15.02|biology +61|440|65686|4294967339|26.46|41.95|true|irene falkner|2013-03-01 09:11:58.703322|54.04|opthamology +30|429|65694|4294967545|43.41|11.52|false|katie miller|2013-03-01 09:11:58.703282|48.71|american history +10|381|65682|4294967350|64.89|42.20|true|alice robinson|2013-03-01 09:11:58.703208|53.07|forestry +34|346|65674|4294967475|3.73|16.55|true|alice king|2013-03-01 09:11:58.703099|67.87|biology +51|321|65584|4294967457|87.95|29.23|true|holly carson|2013-03-01 09:11:58.703084|77.85|quiet hour +-2|334|65644|4294967311|70.20|23.98|true|katie ovid|2013-03-01 09:11:58.703223|43.26|religion +19|343|65555|4294967507|36.52|10.40|false|xavier carson|2013-03-01 09:11:58.703187|24.30|wind surfing +20|316|65615|4294967448|19.53|2.05|false|yuri thompson|2013-03-01 09:11:58.703211|10.35|zync studies +93|396|65788|4294967352|57.53|2.83|false|gabriella zipper|2013-03-01 09:11:58.703233|53.96|yard duty +23|378|65693|4294967394|69.63|40.32|false|gabriella white|2013-03-01 09:11:58.703158|27.90|undecided +2|409|65554|4294967415|52.18|25.89|true|rachel carson|2013-03-01 09:11:58.703255|35.75|topology +52|290|65759|4294967446|17.39|10.74|false|sarah davidson|2013-03-01 09:11:58.703142|89.35|opthamology +30|490|65751|4294967327|53.03|32.65|true|luke nixon|2013-03-01 09:11:58.703290|95.90|geology +106|347|65689|4294967417|52.33|38.33|false|oscar ellison|2013-03-01 09:11:58.703283|71.87|values clariffication +71|476|65659|4294967329|54.98|3.65|true|mike brown|2013-03-01 09:11:58.703155|68.15|forestry +83|424|65661|4294967464|20.47|16.60|true|priscilla nixon|2013-03-01 09:11:58.703125|71.57|yard duty +41|271|65706|4294967327|79.23|23.23|true|holly thompson|2013-03-01 09:11:58.703246|60.85|religion +85|461|65567|4294967356|39.88|2.29|false|nick polk|2013-03-01 09:11:58.703220|84.08|forestry +103|436|65703|4294967355|74.40|22.68|false|ethan young|2013-03-01 09:11:58.703312|79.46|forestry +52|486|65737|4294967483|17.93|15.29|true|ethan thompson|2013-03-01 09:11:58.703088|47.29|topology +113|451|65784|4294967314|56.58|5.89|false|mike white|2013-03-01 09:11:58.703198|61.24|biology +16|458|65754|4294967549|91.65|3.41|true|mike johnson|2013-03-01 09:11:58.703312|67.22|xylophone band +69|283|65643|4294967362|59.23|21.64|false|katie davidson|2013-03-01 09:11:58.703184|94.11|study skills +12|373|65690|4294967434|4.72|7.48|true|jessica robinson|2013-03-01 09:11:58.703305|99.57|linguistics +15|309|65718|4294967346|5.00|12.22|false|luke nixon|2013-03-01 09:11:58.703119|4.25|religion +15|357|65563|4294967395|49.59|30.04|true|sarah steinbeck|2013-03-01 09:11:58.703128|57.33|american history +120|479|65631|4294967433|24.54|1.17|false|alice davidson|2013-03-01 09:11:58.703106|5.76|geology +9|317|65564|4294967471|39.29|21.54|false|jessica garcia|2013-03-01 09:11:58.703232|60.16|forestry +101|458|65548|4294967400|32.15|32.70|false|calvin robinson|2013-03-01 09:11:58.703077|11.71|kindergarten +33|455|65718|4294967416|32.16|49.76|true|alice falkner|2013-03-01 09:11:58.703260|32.80|biology +52|290|65694|4294967395|24.96|42.85|false|tom zipper|2013-03-01 09:11:58.703314|8.90|opthamology +74|404|65710|4294967363|12.60|13.47|true|alice carson|2013-03-01 09:11:58.703139|98.90|geology +100|405|65785|4294967532|0.91|26.61|true|irene falkner|2013-03-01 09:11:58.703292|98.25|nap time +117|488|65576|4294967357|33.08|47.34|false|ulysses xylophone|2013-03-01 09:11:58.703276|1.94|industrial engineering +12|284|65671|4294967331|91.26|17.19|true|luke xylophone|2013-03-01 09:11:58.703143|99.24|linguistics +118|389|65652|4294967439|46.80|44.75|false|victor johnson|2013-03-01 09:11:58.703205|17.83|forestry +-1|466|65764|4294967506|75.70|11.92|false|david quirinius|2013-03-01 09:11:58.703167|10.45|zync studies +13|492|65673|4294967404|5.74|45.75|false|alice brown|2013-03-01 09:11:58.703074|2.81|study skills +97|444|65611|4294967479|92.50|15.78|false|gabriella garcia|2013-03-01 09:11:58.703172|8.48|study skills +45|343|65690|4294967451|66.97|1.20|true|victor young|2013-03-01 09:11:58.703263|46.66|zync studies +109|272|65700|4294967407|72.11|26.41|true|wendy davidson|2013-03-01 09:11:58.703164|54.70|chemistry +36|355|65541|4294967380|39.34|29.28|true|alice ovid|2013-03-01 09:11:58.703181|74.61|education +107|466|65606|4294967498|10.65|4.82|false|holly garcia|2013-03-01 09:11:58.703197|63.47|biology +47|417|65717|4294967464|14.22|28.44|false|calvin van buren|2013-03-01 09:11:58.703127|60.93|nap time +5|486|65668|4294967445|59.64|13.77|true|wendy garcia|2013-03-01 09:11:58.703184|66.41|nap time +59|469|65727|4294967301|57.31|0.79|false|mike hernandez|2013-03-01 09:11:58.703177|84.40|religion +7|470|65653|4294967486|39.98|10.25|true|yuri allen|2013-03-01 09:11:58.703280|84.97|undecided +65|377|65735|4294967517|93.52|21.92|true|ethan ichabod|2013-03-01 09:11:58.703216|69.58|philosophy +37|343|65601|4294967318|91.95|0.75|false|holly steinbeck|2013-03-01 09:11:58.703199|98.51|geology +23|341|65648|4294967327|94.17|17.32|false|ethan ovid|2013-03-01 09:11:58.703129|18.23|geology +106|391|65667|4294967521|48.19|6.70|true|priscilla laertes|2013-03-01 09:11:58.703199|66.46|chemistry +101|419|65736|4294967473|74.68|22.36|false|gabriella thompson|2013-03-01 09:11:58.703239|43.16|undecided +22|334|65571|4294967334|64.43|44.23|true|priscilla nixon|2013-03-01 09:11:58.703291|31.76|chemistry +41|258|65573|4294967310|31.45|10.95|false|gabriella young|2013-03-01 09:11:58.703264|70.64|debate +43|486|65602|4294967466|61.49|45.43|true|ulysses king|2013-03-01 09:11:58.703313|97.99|values clariffication +47|450|65738|4294967522|89.76|29.36|true|alice thompson|2013-03-01 09:11:58.703138|79.34|forestry +38|456|65681|4294967410|23.58|39.34|true|holly carson|2013-03-01 09:11:58.703317|34.78|mathematics +99|447|65750|4294967423|98.66|14.20|true|mike brown|2013-03-01 09:11:58.703288|91.47|topology +1|338|65675|4294967538|60.68|18.55|true|wendy davidson|2013-03-01 09:11:58.703318|25.47|philosophy +11|477|65543|4294967301|95.36|9.10|false|gabriella ovid|2013-03-01 09:11:58.703151|27.07|quiet hour +60|418|65744|4294967300|77.78|42.33|true|katie brown|2013-03-01 09:11:58.703073|26.37|wind surfing +53|377|65752|4294967379|70.20|21.91|true|oscar van buren|2013-03-01 09:11:58.703122|74.12|forestry +96|452|65747|4294967392|0.36|19.63|true|holly allen|2013-03-01 09:11:58.703072|60.55|kindergarten +3|482|65662|4294967531|92.99|33.85|false|victor steinbeck|2013-03-01 09:11:58.703232|16.06|debate +75|286|65747|4294967388|84.24|24.46|false|victor xylophone|2013-03-01 09:11:58.703320|18.00|geology +123|453|65618|4294967402|13.92|33.73|false|wendy johnson|2013-03-01 09:11:58.703258|93.97|chemistry +34|379|65788|4294967441|88.36|23.01|true|bob nixon|2013-03-01 09:11:58.703102|81.35|kindergarten +16|472|65765|4294967423|26.09|4.32|true|david young|2013-03-01 09:11:58.703157|62.84|opthamology +40|386|65595|4294967373|11.44|35.52|true|priscilla king|2013-03-01 09:11:58.703263|39.12|biology +75|443|65594|4294967542|72.33|34.34|false|ethan ovid|2013-03-01 09:11:58.703300|31.30|study skills +30|382|65572|4294967458|87.54|16.90|true|alice robinson|2013-03-01 09:11:58.703224|89.31|education +75|497|65558|4294967504|96.93|17.45|false|ethan johnson|2013-03-01 09:11:58.703182|50.03|debate +117|390|65782|4294967420|2.03|18.37|false|yuri polk|2013-03-01 09:11:58.703195|73.06|kindergarten +56|332|65670|4294967369|82.25|30.46|true|rachel laertes|2013-03-01 09:11:58.703186|37.31|zync studies +16|481|65659|4294967338|27.01|37.67|true|irene ellison|2013-03-01 09:11:58.703147|12.30|topology +76|442|65604|4294967390|71.70|5.63|true|ethan davidson|2013-03-01 09:11:58.703121|0.33|zync studies +91|347|65572|4294967454|85.49|27.97|false|wendy miller|2013-03-01 09:11:58.703163|56.22|quiet hour +52|377|65677|4294967350|63.81|26.66|false|mike zipper|2013-03-01 09:11:58.703192|27.57|kindergarten +6|421|65647|4294967509|34.99|15.30|true|wendy laertes|2013-03-01 09:11:58.703124|69.33|history +70|271|65690|4294967547|67.35|35.24|true|xavier laertes|2013-03-01 09:11:58.703167|91.56|quiet hour +12|387|65572|4294967507|15.31|21.34|true|victor ellison|2013-03-01 09:11:58.703139|18.35|geology +101|299|65741|4294967432|34.56|34.26|false|david ovid|2013-03-01 09:11:58.703271|11.66|debate +110|406|65701|4294967327|30.20|2.43|true|holly carson|2013-03-01 09:11:58.703263|37.07|linguistics +107|399|65542|4294967480|38.27|15.08|false|wendy young|2013-03-01 09:11:58.703167|84.40|education +-1|464|65776|4294967468|30.32|46.98|true|rachel laertes|2013-03-01 09:11:58.703130|66.53|quiet hour +87|408|65683|4294967333|29.71|20.14|true|gabriella johnson|2013-03-01 09:11:58.703071|93.01|forestry +53|405|65686|4294967527|34.20|22.01|true|ulysses falkner|2013-03-01 09:11:58.703270|83.29|study skills +114|444|65637|4294967485|6.42|27.85|true|holly ovid|2013-03-01 09:11:58.703294|59.17|kindergarten +31|305|65595|4294967499|64.66|25.11|true|sarah white|2013-03-01 09:11:58.703203|40.76|topology +27|356|65587|4294967509|34.15|14.78|true|david white|2013-03-01 09:11:58.703299|98.11|topology +86|296|65741|4294967512|91.53|31.03|false|alice ellison|2013-03-01 09:11:58.703170|57.69|geology +6|257|65612|4294967305|3.12|10.29|true|sarah hernandez|2013-03-01 09:11:58.703221|83.31|biology +48|309|65761|4294967331|56.13|22.30|false|mike underhill|2013-03-01 09:11:58.703261|93.73|forestry +19|509|65742|4294967505|57.36|11.43|true|bob ovid|2013-03-01 09:11:58.703276|18.79|history +4|307|65553|4294967517|9.81|10.15|false|ulysses ellison|2013-03-01 09:11:58.703305|55.09|quiet hour +44|332|65554|4294967403|15.89|19.95|true|bob robinson|2013-03-01 09:11:58.703099|55.84|industrial engineering +71|347|65705|4294967362|27.42|36.86|false|katie miller|2013-03-01 09:11:58.703183|72.60|kindergarten +23|476|65751|4294967436|6.79|0.72|false|katie robinson|2013-03-01 09:11:58.703121|76.76|zync studies +35|314|65791|4294967336|57.08|1.93|true|alice young|2013-03-01 09:11:58.703277|91.51|mathematics +58|498|65778|4294967363|58.73|39.25|true|ethan underhill|2013-03-01 09:11:58.703082|0.45|biology +112|419|65648|4294967359|39.97|23.51|true|nick hernandez|2013-03-01 09:11:58.703302|5.51|yard duty +12|415|65747|4294967387|40.87|47.45|false|quinn hernandez|2013-03-01 09:11:58.703259|44.37|values clariffication +90|419|65761|4294967416|78.22|32.36|true|katie quirinius|2013-03-01 09:11:58.703111|57.99|american history +36|268|65627|4294967501|9.16|29.98|false|zach falkner|2013-03-01 09:11:58.703295|3.58|history +25|495|65577|4294967321|44.83|48.75|false|ulysses davidson|2013-03-01 09:11:58.703142|20.51|chemistry +94|301|65550|4294967522|7.18|29.11|true|priscilla allen|2013-03-01 09:11:58.703168|80.45|undecided +71|443|65729|4294967453|17.81|13.38|true|jessica steinbeck|2013-03-01 09:11:58.703117|42.85|values clariffication +114|391|65723|4294967297|80.42|13.67|true|zach falkner|2013-03-01 09:11:58.703136|19.14|undecided +14|298|65623|4294967475|80.44|30.44|true|zach van buren|2013-03-01 09:11:58.703116|49.72|mathematics +23|357|65581|4294967428|96.98|17.68|true|wendy polk|2013-03-01 09:11:58.703160|65.37|philosophy +101|383|65542|4294967427|61.36|36.17|false|xavier nixon|2013-03-01 09:11:58.703291|37.16|topology +110|399|65652|4294967396|22.12|16.04|false|gabriella steinbeck|2013-03-01 09:11:58.703249|86.11|joggying +114|330|65773|4294967490|91.69|27.93|false|zach underhill|2013-03-01 09:11:58.703158|96.34|wind surfing +104|497|65768|4294967341|71.66|49.36|true|katie steinbeck|2013-03-01 09:11:58.703093|44.21|quiet hour +85|368|65567|4294967333|32.11|27.80|false|calvin ellison|2013-03-01 09:11:58.703206|64.01|joggying +36|277|65715|4294967355|67.84|26.06|false|yuri king|2013-03-01 09:11:58.703286|30.76|topology +56|408|65716|4294967361|69.93|11.47|true|xavier young|2013-03-01 09:11:58.703087|67.42|wind surfing +11|333|65623|4294967335|42.22|38.78|false|nick underhill|2013-03-01 09:11:58.703185|12.08|philosophy +111|506|65791|4294967358|5.51|6.10|false|irene ovid|2013-03-01 09:11:58.703119|96.66|industrial engineering +101|411|65624|4294967550|72.84|32.79|true|ethan davidson|2013-03-01 09:11:58.703261|49.80|education +66|477|65701|4294967380|38.68|1.98|true|luke ovid|2013-03-01 09:11:58.703194|99.92|history +15|285|65670|4294967535|7.36|7.25|false|mike van buren|2013-03-01 09:11:58.703265|0.46|geology +10|290|65660|4294967455|28.42|30.35|false|victor ichabod|2013-03-01 09:11:58.703269|55.76|biology +77|379|65603|4294967315|10.39|15.38|false|david king|2013-03-01 09:11:58.703305|96.01|zync studies +121|345|65750|4294967457|34.40|35.01|false|nick thompson|2013-03-01 09:11:58.703293|32.95|opthamology +67|345|65619|4294967331|81.41|6.70|true|victor white|2013-03-01 09:11:58.703217|48.75|debate +17|395|65738|4294967405|55.21|7.52|false|sarah carson|2013-03-01 09:11:58.703245|0.97|debate +36|338|65673|4294967317|20.08|24.61|true|wendy nixon|2013-03-01 09:11:58.703077|12.81|industrial engineering +49|425|65772|4294967447|57.19|15.04|true|xavier quirinius|2013-03-01 09:11:58.703216|3.87|industrial engineering +5|490|65732|4294967303|67.54|33.80|false|victor polk|2013-03-01 09:11:58.703102|69.82|mathematics +31|274|65702|4294967495|37.88|26.83|false|katie miller|2013-03-01 09:11:58.703127|57.23|kindergarten +93|353|65668|4294967349|26.34|6.31|false|holly brown|2013-03-01 09:11:58.703194|89.35|opthamology +114|499|65583|4294967325|90.80|38.86|false|katie ichabod|2013-03-01 09:11:58.703129|70.91|zync studies +118|458|65610|4294967355|57.49|41.03|false|rachel brown|2013-03-01 09:11:58.703304|43.60|biology +2|385|65772|4294967428|32.20|7.27|false|calvin laertes|2013-03-01 09:11:58.703309|40.81|xylophone band +88|491|65732|4294967433|70.68|29.84|false|holly laertes|2013-03-01 09:11:58.703165|32.41|history +74|277|65700|4294967453|16.76|29.26|true|zach king|2013-03-01 09:11:58.703177|90.26|geology +14|442|65719|4294967449|63.38|13.99|true|oscar quirinius|2013-03-01 09:11:58.703296|97.76|industrial engineering +9|288|65709|4294967304|7.13|5.80|false|yuri falkner|2013-03-01 09:11:58.703275|75.40|american history +89|489|65593|4294967361|25.89|47.24|true|ulysses polk|2013-03-01 09:11:58.703164|76.77|chemistry +109|407|65771|4294967418|72.05|32.62|false|tom young|2013-03-01 09:11:58.703285|0.08|religion +74|285|65734|4294967393|12.67|46.88|false|victor robinson|2013-03-01 09:11:58.703096|59.40|philosophy +74|287|65789|4294967414|69.51|46.03|false|mike robinson|2013-03-01 09:11:58.703291|13.60|chemistry +39|429|65588|4294967332|85.67|21.12|true|calvin polk|2013-03-01 09:11:58.703247|80.50|chemistry +0|417|65703|4294967376|25.10|46.85|false|tom ellison|2013-03-01 09:11:58.703247|89.35|education +71|403|65670|4294967328|62.02|10.41|false|fred van buren|2013-03-01 09:11:58.703325|86.78|values clariffication +7|301|65543|4294967462|41.16|39.95|true|yuri carson|2013-03-01 09:11:58.703260|3.85|chemistry +122|317|65547|4294967499|47.95|21.80|true|wendy young|2013-03-01 09:11:58.703117|76.58|joggying +50|332|65710|4294967447|69.57|8.11|false|ulysses young|2013-03-01 09:11:58.703303|54.32|industrial engineering +54|347|65542|4294967545|10.92|19.02|false|katie steinbeck|2013-03-01 09:11:58.703177|14.94|chemistry +17|423|65591|4294967464|50.57|37.51|true|tom ovid|2013-03-01 09:11:58.703095|68.21|linguistics +112|385|65675|4294967412|35.31|6.92|false|victor quirinius|2013-03-01 09:11:58.703081|77.39|philosophy +48|490|65641|4294967432|60.66|27.69|false|bob nixon|2013-03-01 09:11:58.703145|70.14|american history +46|413|65553|4294967357|20.99|16.73|true|tom ichabod|2013-03-01 09:11:58.703121|74.46|forestry +37|312|65644|4294967506|26.18|1.77|false|quinn ellison|2013-03-01 09:11:58.703078|81.10|nap time +121|399|65648|4294967405|20.87|9.02|false|holly xylophone|2013-03-01 09:11:58.703290|53.56|history +67|311|65777|4294967382|43.43|32.80|true|david miller|2013-03-01 09:11:58.703285|30.47|education +34|292|65756|4294967428|71.76|13.47|false|ulysses falkner|2013-03-01 09:11:58.703309|67.03|religion +74|481|65679|4294967424|3.44|19.27|false|nick nixon|2013-03-01 09:11:58.703306|64.29|history +28|436|65692|4294967408|52.93|19.17|false|gabriella underhill|2013-03-01 09:11:58.703314|33.68|undecided +7|397|65787|4294967329|47.82|27.29|true|rachel xylophone|2013-03-01 09:11:58.703250|32.97|opthamology +50|294|65775|4294967462|8.88|16.15|true|ulysses king|2013-03-01 09:11:58.703104|7.09|chemistry +1|429|65541|4294967324|82.69|18.41|false|jessica ovid|2013-03-01 09:11:58.703134|38.37|topology +91|509|65739|4294967397|96.88|17.26|true|luke zipper|2013-03-01 09:11:58.703111|96.23|undecided +63|377|65674|4294967356|50.02|1.23|false|katie thompson|2013-03-01 09:11:58.703232|13.08|american history +90|277|65709|4294967366|65.43|11.46|true|priscilla falkner|2013-03-01 09:11:58.703078|63.33|history +0|411|65743|4294967316|0.08|30.43|true|david ovid|2013-03-01 09:11:58.703163|95.94|biology +77|354|65765|4294967409|37.55|12.06|true|sarah ovid|2013-03-01 09:11:58.703209|50.45|biology +99|356|65780|4294967442|24.62|30.41|true|fred brown|2013-03-01 09:11:58.703280|54.10|education +7|439|65606|4294967513|62.45|19.60|true|nick carson|2013-03-01 09:11:58.703233|59.10|nap time +14|374|65622|4294967492|98.51|40.87|true|sarah robinson|2013-03-01 09:11:58.703136|86.75|joggying +111|479|65605|4294967365|10.46|16.57|false|oscar garcia|2013-03-01 09:11:58.703265|27.35|opthamology +65|290|65748|4294967392|37.39|19.49|true|quinn miller|2013-03-01 09:11:58.703198|26.20|debate +55|483|65786|4294967380|19.32|0.79|true|yuri ovid|2013-03-01 09:11:58.703287|15.80|history +-1|423|65587|4294967430|10.75|41.06|false|alice quirinius|2013-03-01 09:11:58.703284|29.95|religion +26|267|65676|4294967298|55.39|46.87|true|rachel thompson|2013-03-01 09:11:58.703310|27.59|forestry +13|406|65726|4294967427|53.09|5.12|true|bob laertes|2013-03-01 09:11:58.703260|32.09|wind surfing +10|426|65775|4294967535|29.99|8.98|true|yuri brown|2013-03-01 09:11:58.703253|48.08|history +22|320|65649|4294967407|91.93|2.42|true|tom king|2013-03-01 09:11:58.703122|93.89|opthamology +63|261|65748|4294967357|10.44|44.85|false|ulysses white|2013-03-01 09:11:58.703167|97.60|religion +102|453|65539|4294967307|58.07|47.60|false|zach ichabod|2013-03-01 09:11:58.703129|62.09|nap time +32|464|65543|4294967451|85.81|49.51|false|calvin carson|2013-03-01 09:11:58.703107|44.93|chemistry +30|453|65556|4294967406|63.39|8.35|true|ulysses thompson|2013-03-01 09:11:58.703308|8.40|chemistry +45|461|65625|4294967539|90.18|38.11|true|katie davidson|2013-03-01 09:11:58.703186|30.37|zync studies +2|260|65732|4294967524|22.15|2.38|true|sarah polk|2013-03-01 09:11:58.703084|86.33|linguistics +93|465|65656|4294967340|50.99|10.12|true|bob carson|2013-03-01 09:11:58.703241|95.63|kindergarten +55|374|65775|4294967484|55.00|41.23|false|victor falkner|2013-03-01 09:11:58.703176|0.10|forestry +63|336|65786|4294967399|16.02|12.54|true|alice hernandez|2013-03-01 09:11:58.703175|12.54|kindergarten +64|286|65559|4294967464|49.52|25.04|true|mike brown|2013-03-01 09:11:58.703259|36.87|wind surfing +62|480|65553|4294967374|21.34|47.06|true|fred thompson|2013-03-01 09:11:58.703084|44.57|debate +102|277|65548|4294967542|58.21|34.53|true|rachel robinson|2013-03-01 09:11:58.703258|65.74|religion +110|457|65561|4294967371|92.33|35.61|true|sarah miller|2013-03-01 09:11:58.703090|64.15|joggying +5|402|65553|4294967307|93.10|26.10|false|nick thompson|2013-03-01 09:11:58.703084|57.37|mathematics +21|411|65650|4294967482|22.26|20.75|false|tom robinson|2013-03-01 09:11:58.703292|83.82|joggying +39|381|65579|4294967352|26.77|12.35|true|holly allen|2013-03-01 09:11:58.703225|23.91|topology +109|318|65723|4294967471|46.56|31.03|false|sarah nixon|2013-03-01 09:11:58.703224|42.62|opthamology +123|343|65779|4294967399|32.16|4.16|false|oscar johnson|2013-03-01 09:11:58.703163|23.14|kindergarten +17|345|65693|4294967379|26.90|37.74|false|irene davidson|2013-03-01 09:11:58.703140|17.58|philosophy +87|376|65576|4294967311|87.28|1.57|false|alice carson|2013-03-01 09:11:58.703155|10.36|study skills +8|307|65537|4294967429|76.83|17.86|false|zach nixon|2013-03-01 09:11:58.703256|7.36|topology +66|448|65734|4294967372|37.66|48.77|true|oscar davidson|2013-03-01 09:11:58.703194|59.28|joggying +43|337|65696|4294967319|0.20|18.30|false|ethan ovid|2013-03-01 09:11:58.703249|57.38|quiet hour +106|421|65604|4294967348|46.75|13.86|true|katie zipper|2013-03-01 09:11:58.703312|7.08|xylophone band +42|482|65605|4294967498|75.19|39.59|true|sarah brown|2013-03-01 09:11:58.703141|66.57|joggying +113|353|65570|4294967341|74.96|16.67|false|fred ichabod|2013-03-01 09:11:58.703171|9.56|values clariffication +40|267|65624|4294967314|99.80|21.59|true|yuri ellison|2013-03-01 09:11:58.703103|32.08|nap time +11|366|65693|4294967365|9.17|7.62|true|zach quirinius|2013-03-01 09:11:58.703280|20.53|study skills +7|395|65666|4294967451|20.12|21.41|false|zach xylophone|2013-03-01 09:11:58.703276|52.67|history +64|292|65789|4294967440|18.62|47.54|false|yuri hernandez|2013-03-01 09:11:58.703124|33.95|study skills +113|459|65644|4294967475|89.58|22.57|true|rachel xylophone|2013-03-01 09:11:58.703112|21.62|geology +84|335|65635|4294967297|38.31|2.34|false|rachel davidson|2013-03-01 09:11:58.703194|49.07|biology +9|440|65742|4294967484|56.34|6.06|true|luke polk|2013-03-01 09:11:58.703224|97.88|philosophy +106|487|65713|4294967359|78.63|48.13|true|quinn davidson|2013-03-01 09:11:58.703143|37.34|nap time +72|485|65554|4294967359|46.78|47.65|true|gabriella van buren|2013-03-01 09:11:58.703094|48.48|mathematics +-3|493|65662|4294967482|28.75|30.21|false|xavier garcia|2013-03-01 09:11:58.703194|4.94|education +69|375|65549|4294967400|5.69|31.51|false|luke laertes|2013-03-01 09:11:58.703204|6.35|study skills +102|319|65681|4294967418|63.05|35.81|true|quinn robinson|2013-03-01 09:11:58.703166|57.32|mathematics +81|317|65598|4294967359|68.49|38.31|false|bob allen|2013-03-01 09:11:58.703237|85.99|industrial engineering +-3|465|65735|4294967298|72.30|22.58|false|bob underhill|2013-03-01 09:11:58.703176|80.99|joggying +81|496|65574|4294967405|76.96|26.05|false|victor nixon|2013-03-01 09:11:58.703157|44.50|history +75|349|65731|4294967338|26.33|45.42|true|gabriella quirinius|2013-03-01 09:11:58.703107|63.73|undecided +50|333|65607|4294967316|8.42|24.57|true|calvin carson|2013-03-01 09:11:58.703115|67.18|geology +37|431|65788|4294967487|31.99|41.19|false|quinn king|2013-03-01 09:11:58.703210|77.48|topology +121|469|65623|4294967513|68.52|8.74|false|yuri brown|2013-03-01 09:11:58.703204|95.81|biology +64|284|65605|4294967421|78.12|15.41|true|zach white|2013-03-01 09:11:58.703212|47.61|yard duty +16|386|65570|4294967342|99.75|2.67|true|rachel davidson|2013-03-01 09:11:58.703159|3.45|wind surfing +-1|405|65558|4294967342|60.59|26.52|true|priscilla davidson|2013-03-01 09:11:58.703242|28.95|quiet hour +-3|408|65667|4294967509|81.68|45.90|true|david hernandez|2013-03-01 09:11:58.703252|51.79|topology +118|423|65591|4294967419|52.07|32.25|false|sarah quirinius|2013-03-01 09:11:58.703296|2.46|education +85|453|65730|4294967414|44.95|23.68|false|ethan ellison|2013-03-01 09:11:58.703260|46.93|chemistry +58|377|65705|4294967437|25.91|48.46|true|alice underhill|2013-03-01 09:11:58.703072|96.69|wind surfing +15|356|65671|4294967401|88.40|26.85|true|oscar brown|2013-03-01 09:11:58.703141|35.40|philosophy +103|349|65756|4294967423|59.40|24.13|false|luke allen|2013-03-01 09:11:58.703159|25.41|religion +91|377|65602|4294967530|78.52|12.55|true|oscar zipper|2013-03-01 09:11:58.703283|55.41|industrial engineering +39|488|65649|4294967394|29.86|6.47|true|calvin ellison|2013-03-01 09:11:58.703171|16.87|debate +109|310|65785|4294967350|21.72|34.15|true|mike robinson|2013-03-01 09:11:58.703254|78.60|history +19|467|65788|4294967459|76.56|7.66|false|ethan carson|2013-03-01 09:11:58.703237|91.09|education +31|345|65749|4294967441|33.29|49.32|true|fred thompson|2013-03-01 09:11:58.703132|44.02|biology +81|480|65598|4294967456|33.55|39.46|true|alice steinbeck|2013-03-01 09:11:58.703091|19.53|forestry +36|424|65758|4294967420|82.01|46.79|true|mike hernandez|2013-03-01 09:11:58.703269|0.35|education +38|272|65668|4294967414|41.90|3.94|false|rachel falkner|2013-03-01 09:11:58.703313|75.88|geology +84|484|65722|4294967343|66.39|6.81|false|oscar quirinius|2013-03-01 09:11:58.703150|32.48|philosophy +113|270|65744|4294967331|25.33|17.62|false|fred zipper|2013-03-01 09:11:58.703263|58.82|undecided +14|325|65585|4294967525|77.72|26.71|false|tom miller|2013-03-01 09:11:58.703213|56.63|chemistry +15|374|65676|4294967469|60.72|2.52|false|david quirinius|2013-03-01 09:11:58.703312|35.38|quiet hour +62|342|65696|4294967395|25.52|1.71|true|nick polk|2013-03-01 09:11:58.703158|22.02|xylophone band +24|256|65643|4294967351|94.05|1.37|false|calvin young|2013-03-01 09:11:58.703265|19.61|education +78|362|65622|4294967544|43.84|1.74|true|priscilla polk|2013-03-01 09:11:58.703073|58.97|philosophy +53|487|65592|4294967434|43.51|14.58|true|ulysses robinson|2013-03-01 09:11:58.703126|62.73|undecided +109|339|65594|4294967333|61.63|28.35|false|oscar robinson|2013-03-01 09:11:58.703078|60.87|philosophy +118|259|65569|4294967351|94.61|14.72|true|irene quirinius|2013-03-01 09:11:58.703217|90.70|quiet hour +55|286|65740|4294967333|6.31|20.67|true|nick ovid|2013-03-01 09:11:58.703212|82.93|opthamology +3|284|65541|4294967344|26.77|19.05|true|xavier laertes|2013-03-01 09:11:58.703316|80.54|undecided +1|453|65780|4294967482|18.58|7.04|false|bob carson|2013-03-01 09:11:58.703321|82.33|chemistry +113|434|65575|4294967401|17.79|43.12|true|ethan king|2013-03-01 09:11:58.703312|19.95|geology +-1|354|65714|4294967492|86.03|18.38|false|gabriella xylophone|2013-03-01 09:11:58.703254|13.03|chemistry +69|323|65662|4294967346|51.19|17.74|false|irene johnson|2013-03-01 09:11:58.703248|44.25|values clariffication +71|405|65540|4294967458|55.91|41.15|false|mike brown|2013-03-01 09:11:58.703153|24.48|philosophy +34|383|65708|4294967461|1.69|28.15|true|fred brown|2013-03-01 09:11:58.703161|91.46|linguistics +4|462|65537|4294967446|60.53|41.72|false|bob carson|2013-03-01 09:11:58.703095|95.64|linguistics +76|335|65645|4294967546|96.82|28.38|false|ulysses carson|2013-03-01 09:11:58.703246|59.49|yard duty +35|310|65610|4294967323|85.38|40.07|true|ethan falkner|2013-03-01 09:11:58.703277|65.37|education +47|435|65543|4294967406|46.63|44.15|true|alice hernandez|2013-03-01 09:11:58.703243|90.13|undecided +63|407|65668|4294967428|26.90|49.22|false|katie thompson|2013-03-01 09:11:58.703308|96.31|industrial engineering +18|410|65543|4294967519|99.58|6.66|true|victor allen|2013-03-01 09:11:58.703096|41.33|industrial engineering +82|314|65723|4294967329|27.26|27.07|true|quinn carson|2013-03-01 09:11:58.703291|32.78|wind surfing +99|427|65596|4294967442|20.89|4.10|true|fred nixon|2013-03-01 09:11:58.703102|43.73|philosophy +94|330|65776|4294967475|34.12|10.18|false|victor davidson|2013-03-01 09:11:58.703112|82.94|xylophone band +19|445|65688|4294967308|99.14|1.02|true|nick zipper|2013-03-01 09:11:58.703112|68.05|religion +54|340|65609|4294967335|17.19|14.06|true|holly ovid|2013-03-01 09:11:58.703217|72.54|religion +41|336|65564|4294967410|69.17|4.86|false|yuri white|2013-03-01 09:11:58.703321|57.88|education +121|467|65707|4294967349|78.39|41.17|false|priscilla steinbeck|2013-03-01 09:11:58.703275|81.42|chemistry +80|443|65725|4294967446|92.31|6.36|true|sarah robinson|2013-03-01 09:11:58.703273|7.28|geology +123|405|65670|4294967524|19.41|37.64|false|sarah davidson|2013-03-01 09:11:58.703202|84.29|zync studies +25|278|65680|4294967425|61.01|34.11|false|calvin nixon|2013-03-01 09:11:58.703228|83.92|linguistics +-1|410|65571|4294967519|79.57|30.73|false|david ovid|2013-03-01 09:11:58.703287|75.14|joggying +1|499|65587|4294967368|91.96|45.46|false|priscilla thompson|2013-03-01 09:11:58.703207|17.07|wind surfing +0|440|65643|4294967527|27.38|13.06|false|irene laertes|2013-03-01 09:11:58.703297|58.82|values clariffication +116|510|65745|4294967458|41.21|23.95|false|david white|2013-03-01 09:11:58.703274|85.19|nap time +37|469|65728|4294967313|82.32|15.08|false|bob white|2013-03-01 09:11:58.703127|44.78|study skills +25|354|65713|4294967323|15.81|19.27|false|jessica white|2013-03-01 09:11:58.703257|49.80|philosophy +31|458|65550|4294967508|57.43|25.69|true|alice ichabod|2013-03-01 09:11:58.703173|75.81|topology +5|351|65677|4294967414|69.47|48.82|false|alice underhill|2013-03-01 09:11:58.703156|91.75|opthamology +44|350|65743|4294967427|50.09|30.70|true|sarah hernandez|2013-03-01 09:11:58.703176|70.06|topology +100|306|65654|4294967396|54.81|29.72|false|ulysses allen|2013-03-01 09:11:58.703140|11.56|forestry +115|401|65580|4294967434|66.48|41.63|true|priscilla ichabod|2013-03-01 09:11:58.703301|46.09|industrial engineering +121|376|65618|4294967489|61.15|27.37|true|holly polk|2013-03-01 09:11:58.703202|58.82|american history +3|411|65791|4294967371|45.48|31.58|false|holly hernandez|2013-03-01 09:11:58.703255|39.76|forestry +48|425|65641|4294967316|95.74|41.42|true|oscar young|2013-03-01 09:11:58.703165|54.75|nap time +-1|362|65737|4294967424|49.66|31.87|true|sarah king|2013-03-01 09:11:58.703224|7.34|religion +116|279|65715|4294967410|66.56|1.05|true|luke white|2013-03-01 09:11:58.703310|69.27|mathematics +99|333|65599|4294967494|15.29|22.04|true|victor underhill|2013-03-01 09:11:58.703178|41.56|chemistry +-3|477|65785|4294967464|97.51|10.84|true|tom hernandez|2013-03-01 09:11:58.703108|6.71|history +18|402|65623|4294967322|98.56|16.91|false|ulysses miller|2013-03-01 09:11:58.703183|7.17|quiet hour +37|455|65785|4294967540|50.55|24.15|false|tom miller|2013-03-01 09:11:58.703223|45.25|quiet hour +42|354|65568|4294967316|23.51|17.91|false|holly robinson|2013-03-01 09:11:58.703108|28.17|chemistry +89|471|65567|4294967469|45.99|24.14|false|irene falkner|2013-03-01 09:11:58.703087|61.94|debate +4|269|65567|4294967516|26.11|1.03|false|victor brown|2013-03-01 09:11:58.703073|32.73|yard duty +106|350|65635|4294967516|59.80|12.99|true|oscar allen|2013-03-01 09:11:58.703190|59.32|study skills +67|268|65791|4294967365|0.80|11.90|true|sarah robinson|2013-03-01 09:11:58.703160|81.29|opthamology +70|511|65583|4294967517|36.70|47.53|false|quinn garcia|2013-03-01 09:11:58.703175|90.90|opthamology +123|415|65762|4294967452|88.94|28.12|true|quinn ovid|2013-03-01 09:11:58.703316|70.57|education +-1|467|65596|4294967326|89.74|13.90|false|zach hernandez|2013-03-01 09:11:58.703277|60.18|nap time +55|452|65704|4294967415|3.65|31.02|true|gabriella allen|2013-03-01 09:11:58.703232|19.77|topology +112|511|65626|4294967366|60.26|6.21|true|zach nixon|2013-03-01 09:11:58.703176|42.85|nap time +-1|385|65764|4294967341|62.98|3.50|false|nick quirinius|2013-03-01 09:11:58.703292|6.02|yard duty +12|444|65623|4294967362|88.29|12.82|true|rachel miller|2013-03-01 09:11:58.703319|53.41|wind surfing +103|505|65583|4294967407|1.67|36.87|true|oscar carson|2013-03-01 09:11:58.703142|23.61|biology +93|366|65544|4294967306|89.95|20.69|true|rachel davidson|2013-03-01 09:11:58.703249|45.69|quiet hour +87|468|65762|4294967330|53.26|38.35|true|fred polk|2013-03-01 09:11:58.703071|95.89|opthamology +84|479|65698|4294967400|58.51|21.14|true|tom falkner|2013-03-01 09:11:58.703207|2.32|kindergarten +74|395|65578|4294967430|36.33|31.68|false|katie garcia|2013-03-01 09:11:58.703161|70.10|study skills +30|511|65651|4294967444|96.19|41.63|false|priscilla carson|2013-03-01 09:11:58.703098|30.35|education +54|505|65699|4294967455|43.70|36.29|false|holly laertes|2013-03-01 09:11:58.703185|73.47|opthamology +7|426|65779|4294967547|47.25|35.49|false|yuri zipper|2013-03-01 09:11:58.703298|39.12|chemistry +123|349|65686|4294967522|19.56|45.81|false|mike allen|2013-03-01 09:11:58.703095|3.78|chemistry +100|511|65582|4294967399|37.44|37.01|true|bob polk|2013-03-01 09:11:58.703115|32.68|joggying +112|327|65558|4294967377|17.83|3.84|false|xavier brown|2013-03-01 09:11:58.703200|69.98|industrial engineering +76|393|65598|4294967397|31.92|31.31|false|katie brown|2013-03-01 09:11:58.703252|1.81|industrial engineering +30|414|65542|4294967381|65.50|44.78|false|priscilla white|2013-03-01 09:11:58.703217|86.51|philosophy +80|340|65655|4294967503|2.11|47.77|false|quinn ellison|2013-03-01 09:11:58.703256|50.30|topology +122|461|65605|4294967317|6.19|29.10|true|ethan carson|2013-03-01 09:11:58.703171|61.26|undecided +58|353|65739|4294967345|42.55|12.72|true|bob white|2013-03-01 09:11:58.703296|36.85|yard duty +68|388|65769|4294967395|73.82|6.35|true|luke van buren|2013-03-01 09:11:58.703260|99.29|linguistics +11|348|65659|4294967328|1.24|40.57|false|tom garcia|2013-03-01 09:11:58.703169|57.41|philosophy +50|282|65665|4294967509|41.36|35.56|true|quinn davidson|2013-03-01 09:11:58.703213|73.80|philosophy +21|442|65580|4294967387|81.11|3.61|false|victor laertes|2013-03-01 09:11:58.703287|32.96|american history +117|423|65555|4294967365|7.91|30.79|true|oscar zipper|2013-03-01 09:11:58.703186|92.30|mathematics +73|330|65788|4294967386|36.34|49.11|true|calvin young|2013-03-01 09:11:58.703074|5.14|linguistics +120|313|65671|4294967464|26.46|30.26|true|sarah brown|2013-03-01 09:11:58.703130|9.02|biology +83|338|65784|4294967495|25.36|27.41|false|oscar garcia|2013-03-01 09:11:58.703181|76.17|american history +97|319|65726|4294967523|36.39|9.99|true|yuri king|2013-03-01 09:11:58.703323|4.44|nap time +11|495|65787|4294967366|92.17|7.41|false|gabriella garcia|2013-03-01 09:11:58.703152|60.18|linguistics +72|339|65692|4294967366|47.94|28.23|false|david ellison|2013-03-01 09:11:58.703075|69.92|religion +108|281|65688|4294967331|29.23|35.57|false|tom quirinius|2013-03-01 09:11:58.703230|41.51|linguistics +25|471|65732|4294967334|31.15|43.30|false|tom xylophone|2013-03-01 09:11:58.703116|99.20|industrial engineering +117|259|65608|4294967408|29.06|41.49|false|ulysses white|2013-03-01 09:11:58.703106|59.49|history +33|431|65732|4294967377|88.57|47.54|false|nick king|2013-03-01 09:11:58.703209|98.90|american history +60|488|65707|4294967452|56.87|7.12|true|ulysses underhill|2013-03-01 09:11:58.703284|61.46|nap time +3|339|65726|4294967319|40.48|0.90|false|quinn nixon|2013-03-01 09:11:58.703134|88.05|debate +81|354|65577|4294967443|74.20|48.14|false|tom laertes|2013-03-01 09:11:58.703128|83.80|geology +47|510|65696|4294967299|21.45|31.70|false|nick falkner|2013-03-01 09:11:58.703222|60.83|kindergarten +76|361|65733|4294967352|12.49|4.35|false|nick steinbeck|2013-03-01 09:11:58.703233|36.34|industrial engineering +60|341|65603|4294967461|80.48|47.69|true|yuri falkner|2013-03-01 09:11:58.703286|37.95|undecided +87|468|65649|4294967377|36.82|1.80|false|alice young|2013-03-01 09:11:58.703177|77.57|joggying +71|299|65692|4294967373|90.91|18.87|true|calvin johnson|2013-03-01 09:11:58.703135|30.45|american history +117|501|65679|4294967483|53.08|31.96|true|nick xylophone|2013-03-01 09:11:58.703076|77.13|linguistics +88|332|65671|4294967324|13.14|3.25|false|wendy garcia|2013-03-01 09:11:58.703312|84.50|yard duty +18|322|65732|4294967392|96.48|43.57|false|xavier brown|2013-03-01 09:11:58.703231|68.30|education +94|282|65636|4294967412|11.20|39.68|true|luke van buren|2013-03-01 09:11:58.703278|67.83|debate +118|368|65691|4294967391|96.19|9.84|true|jessica laertes|2013-03-01 09:11:58.703241|95.59|topology +100|473|65787|4294967434|4.59|26.01|true|fred nixon|2013-03-01 09:11:58.703231|97.11|biology +59|389|65745|4294967406|39.53|18.25|true|priscilla underhill|2013-03-01 09:11:58.703325|62.70|yard duty +80|276|65766|4294967467|38.77|37.89|true|jessica ellison|2013-03-01 09:11:58.703241|91.19|chemistry +58|353|65634|4294967517|59.69|31.66|true|nick van buren|2013-03-01 09:11:58.703257|82.25|nap time +111|313|65732|4294967476|81.62|22.25|true|zach davidson|2013-03-01 09:11:58.703292|67.82|undecided +22|302|65740|4294967403|42.90|38.34|false|ethan ovid|2013-03-01 09:11:58.703089|22.66|industrial engineering +100|458|65606|4294967369|99.00|20.92|true|victor falkner|2013-03-01 09:11:58.703168|80.73|undecided +25|460|65595|4294967407|95.92|44.51|true|wendy brown|2013-03-01 09:11:58.703108|37.28|quiet hour +49|495|65565|4294967374|30.12|42.89|false|rachel steinbeck|2013-03-01 09:11:58.703270|20.09|values clariffication +-1|479|65642|4294967316|30.14|16.11|false|gabriella white|2013-03-01 09:11:58.703077|48.68|industrial engineering +36|292|65665|4294967438|73.30|9.41|true|oscar davidson|2013-03-01 09:11:58.703221|72.83|nap time +29|397|65629|4294967515|7.03|46.88|false|fred white|2013-03-01 09:11:58.703208|44.60|biology +84|471|65788|4294967440|3.56|43.80|true|oscar allen|2013-03-01 09:11:58.703100|59.13|chemistry +102|300|65576|4294967500|49.25|17.49|false|ulysses white|2013-03-01 09:11:58.703094|23.40|study skills +36|341|65724|4294967327|77.42|17.72|false|priscilla allen|2013-03-01 09:11:58.703198|28.18|philosophy +85|321|65558|4294967482|53.81|14.98|true|jessica ellison|2013-03-01 09:11:58.703099|4.36|education +64|353|65686|4294967415|35.00|47.65|false|bob quirinius|2013-03-01 09:11:58.703286|10.35|topology +37|279|65705|4294967333|46.78|3.96|false|alice steinbeck|2013-03-01 09:11:58.703281|93.16|values clariffication +40|476|65647|4294967456|86.74|9.77|false|sarah miller|2013-03-01 09:11:58.703103|83.26|topology +106|497|65762|4294967411|95.10|29.88|false|yuri carson|2013-03-01 09:11:58.703124|64.20|forestry +32|413|65680|4294967334|71.00|7.80|false|quinn xylophone|2013-03-01 09:11:58.703084|17.15|forestry +54|398|65641|4294967486|37.98|41.17|true|bob nixon|2013-03-01 09:11:58.703300|90.15|religion +119|364|65767|4294967327|89.55|36.86|true|ethan zipper|2013-03-01 09:11:58.703206|18.76|topology +100|267|65546|4294967458|19.81|40.41|false|david steinbeck|2013-03-01 09:11:58.703159|14.23|quiet hour +87|333|65789|4294967377|70.65|37.54|false|luke ovid|2013-03-01 09:11:58.703250|94.86|wind surfing +117|405|65555|4294967358|97.55|15.48|true|katie zipper|2013-03-01 09:11:58.703187|22.95|industrial engineering +0|503|65745|4294967348|74.03|27.07|true|quinn garcia|2013-03-01 09:11:58.703240|4.21|linguistics +122|370|65768|4294967545|58.71|9.32|false|yuri ellison|2013-03-01 09:11:58.703311|36.33|joggying +62|311|65656|4294967456|38.28|30.00|false|rachel ovid|2013-03-01 09:11:58.703270|62.02|philosophy +48|499|65595|4294967450|20.51|12.22|false|quinn laertes|2013-03-01 09:11:58.703145|88.23|undecided +122|310|65737|4294967303|79.93|7.85|false|luke garcia|2013-03-01 09:11:58.703232|2.70|topology +27|409|65541|4294967484|88.12|2.59|true|priscilla falkner|2013-03-01 09:11:58.703088|22.68|philosophy +56|351|65781|4294967407|83.01|7.07|true|ethan carson|2013-03-01 09:11:58.703306|95.70|american history +110|400|65617|4294967535|27.93|21.70|true|nick steinbeck|2013-03-01 09:11:58.703159|81.28|zync studies +78|377|65579|4294967454|70.68|6.26|true|holly carson|2013-03-01 09:11:58.703194|44.60|joggying +86|502|65710|4294967397|62.60|44.42|true|wendy ellison|2013-03-01 09:11:58.703301|3.95|biology +45|487|65575|4294967387|23.69|47.63|false|jessica thompson|2013-03-01 09:11:58.703243|82.25|american history +121|482|65782|4294967451|17.09|1.88|false|victor steinbeck|2013-03-01 09:11:58.703149|8.12|yard duty +60|426|65672|4294967338|6.90|33.66|true|ulysses brown|2013-03-01 09:11:58.703267|4.40|nap time +12|335|65707|4294967333|38.67|22.83|true|quinn van buren|2013-03-01 09:11:58.703256|52.32|undecided +59|443|65665|4294967427|82.38|8.52|true|wendy miller|2013-03-01 09:11:58.703278|59.92|chemistry +71|262|65574|4294967401|80.62|38.79|true|rachel hernandez|2013-03-01 09:11:58.703154|26.99|quiet hour +71|432|65676|4294967349|27.07|24.15|false|yuri xylophone|2013-03-01 09:11:58.703213|39.60|linguistics +75|420|65594|4294967357|76.79|43.32|false|gabriella steinbeck|2013-03-01 09:11:58.703237|49.57|wind surfing +111|335|65591|4294967408|34.96|13.51|false|priscilla nixon|2013-03-01 09:11:58.703185|74.95|topology +29|379|65745|4294967523|44.32|8.40|true|ethan nixon|2013-03-01 09:11:58.703086|71.87|joggying +27|433|65606|4294967516|66.28|8.73|false|quinn polk|2013-03-01 09:11:58.703231|75.57|religion +5|439|65667|4294967404|33.64|40.57|true|priscilla allen|2013-03-01 09:11:58.703175|35.46|mathematics +10|266|65747|4294967297|12.06|48.57|false|mike steinbeck|2013-03-01 09:11:58.703291|97.53|religion +59|393|65739|4294967351|98.16|6.32|false|gabriella van buren|2013-03-01 09:11:58.703241|35.75|history +64|468|65787|4294967433|68.96|19.27|false|katie ichabod|2013-03-01 09:11:58.703324|45.46|xylophone band +65|369|65555|4294967529|81.84|20.68|true|jessica polk|2013-03-01 09:11:58.703302|76.65|quiet hour +120|297|65587|4294967510|68.66|13.92|true|ulysses xylophone|2013-03-01 09:11:58.703134|69.14|mathematics +45|456|65604|4294967354|20.19|23.73|false|sarah robinson|2013-03-01 09:11:58.703157|16.54|wind surfing +-3|400|65557|4294967503|76.31|29.44|false|alice allen|2013-03-01 09:11:58.703323|19.11|debate +49|435|65574|4294967464|19.14|18.78|true|irene polk|2013-03-01 09:11:58.703158|21.41|american history +44|309|65544|4294967319|65.91|46.02|true|sarah allen|2013-03-01 09:11:58.703296|77.76|nap time +99|360|65575|4294967404|17.91|11.12|false|calvin allen|2013-03-01 09:11:58.703221|53.14|biology +10|414|65735|4294967376|94.43|1.64|false|fred quirinius|2013-03-01 09:11:58.703232|79.92|philosophy +21|335|65732|4294967523|54.34|19.64|true|bob xylophone|2013-03-01 09:11:58.703107|30.32|yard duty +78|349|65580|4294967440|33.29|35.00|false|ulysses davidson|2013-03-01 09:11:58.703143|82.41|opthamology +75|281|65540|4294967319|36.67|33.20|false|rachel zipper|2013-03-01 09:11:58.703169|4.55|chemistry +54|281|65657|4294967528|73.96|21.95|true|priscilla quirinius|2013-03-01 09:11:58.703310|33.69|american history +19|291|65783|4294967332|81.88|1.11|true|ethan thompson|2013-03-01 09:11:58.703319|98.25|linguistics +44|322|65637|4294967416|4.64|25.71|true|irene allen|2013-03-01 09:11:58.703195|33.53|wind surfing +105|436|65655|4294967518|9.80|31.51|true|quinn johnson|2013-03-01 09:11:58.703258|49.73|study skills +83|369|65568|4294967415|57.07|17.41|false|yuri van buren|2013-03-01 09:11:58.703156|58.41|biology +42|427|65704|4294967367|82.36|22.92|false|calvin davidson|2013-03-01 09:11:58.703212|7.71|zync studies +31|270|65652|4294967407|46.34|2.87|true|yuri quirinius|2013-03-01 09:11:58.703194|60.67|religion +98|268|65576|4294967395|49.97|23.17|true|zach brown|2013-03-01 09:11:58.703186|10.93|geology +88|297|65537|4294967549|72.62|12.36|false|sarah ichabod|2013-03-01 09:11:58.703210|29.01|wind surfing +36|386|65759|4294967348|10.98|45.78|true|alice king|2013-03-01 09:11:58.703110|79.08|education +18|483|65658|4294967411|21.00|23.54|false|rachel hernandez|2013-03-01 09:11:58.703157|24.03|chemistry +53|404|65748|4294967510|6.54|27.17|true|fred hernandez|2013-03-01 09:11:58.703102|28.14|forestry +74|399|65682|4294967538|85.55|43.37|false|irene falkner|2013-03-01 09:11:58.703170|75.95|yard duty +123|403|65654|4294967367|61.07|28.24|true|holly carson|2013-03-01 09:11:58.703287|7.59|mathematics +19|257|65557|4294967524|55.07|22.33|true|oscar young|2013-03-01 09:11:58.703252|51.18|american history +80|413|65785|4294967429|38.30|21.78|true|ethan miller|2013-03-01 09:11:58.703208|37.63|linguistics +29|492|65654|4294967303|31.02|14.34|false|wendy allen|2013-03-01 09:11:58.703184|61.82|education +73|261|65672|4294967462|14.83|11.27|false|xavier ichabod|2013-03-01 09:11:58.703317|2.68|history +116|293|65734|4294967516|3.70|27.62|true|katie steinbeck|2013-03-01 09:11:58.703240|5.63|mathematics +1|382|65570|4294967504|76.46|49.42|false|ulysses davidson|2013-03-01 09:11:58.703089|31.98|wind surfing +79|462|65734|4294967476|89.52|36.38|false|quinn steinbeck|2013-03-01 09:11:58.703224|84.72|linguistics +114|286|65554|4294967550|18.35|13.01|true|fred robinson|2013-03-01 09:11:58.703171|69.58|yard duty +12|511|65657|4294967394|31.17|39.20|false|calvin laertes|2013-03-01 09:11:58.703070|58.68|wind surfing +61|473|65758|4294967330|77.23|47.58|false|wendy van buren|2013-03-01 09:11:58.703288|4.70|quiet hour +70|365|65566|4294967366|83.52|9.07|false|luke johnson|2013-03-01 09:11:58.703274|42.58|joggying +113|424|65703|4294967386|43.04|18.69|true|fred steinbeck|2013-03-01 09:11:58.703213|98.67|joggying +98|465|65609|4294967371|50.32|30.55|true|ethan young|2013-03-01 09:11:58.703220|60.90|history +103|451|65762|4294967468|88.88|47.08|false|ulysses garcia|2013-03-01 09:11:58.703221|6.81|american history +100|347|65576|4294967383|1.77|12.63|false|victor davidson|2013-03-01 09:11:58.703213|17.33|study skills +48|395|65574|4294967332|20.57|13.18|true|rachel quirinius|2013-03-01 09:11:58.703132|52.70|education +36|394|65740|4294967504|89.26|1.28|true|alice miller|2013-03-01 09:11:58.703152|34.90|opthamology +96|421|65775|4294967340|82.62|41.30|true|tom hernandez|2013-03-01 09:11:58.703201|68.45|linguistics +23|490|65686|4294967508|81.79|42.64|false|oscar van buren|2013-03-01 09:11:58.703114|62.86|wind surfing +8|443|65734|4294967410|63.46|16.01|true|alice polk|2013-03-01 09:11:58.703241|67.07|zync studies +94|492|65681|4294967332|78.86|17.84|true|rachel carson|2013-03-01 09:11:58.703256|35.83|study skills +72|497|65638|4294967530|22.13|42.77|false|priscilla brown|2013-03-01 09:11:58.703258|63.01|philosophy +40|263|65746|4294967297|6.81|31.85|true|ulysses garcia|2013-03-01 09:11:58.703201|5.86|kindergarten +34|408|65791|4294967336|8.38|41.52|true|alice davidson|2013-03-01 09:11:58.703182|3.51|chemistry +30|402|65678|4294967443|79.16|43.02|true|xavier garcia|2013-03-01 09:11:58.703167|51.04|chemistry +116|290|65716|4294967468|71.18|5.52|false|luke king|2013-03-01 09:11:58.703317|16.60|biology +60|374|65725|4294967338|97.67|15.75|true|fred nixon|2013-03-01 09:11:58.703295|93.43|quiet hour +15|361|65551|4294967468|44.29|18.95|true|david polk|2013-03-01 09:11:58.703120|22.34|education +56|491|65717|4294967546|88.28|17.48|false|oscar ellison|2013-03-01 09:11:58.703218|35.37|opthamology +67|403|65779|4294967397|64.09|4.78|true|wendy brown|2013-03-01 09:11:58.703192|90.65|quiet hour +25|414|65625|4294967527|5.73|11.33|true|priscilla young|2013-03-01 09:11:58.703188|46.50|geology +11|404|65580|4294967326|12.59|19.98|true|victor white|2013-03-01 09:11:58.703131|63.55|religion +17|383|65758|4294967454|2.05|18.70|true|ethan davidson|2013-03-01 09:11:58.703322|3.69|wind surfing +26|295|65662|4294967495|88.53|27.40|true|irene davidson|2013-03-01 09:11:58.703233|90.28|philosophy +87|355|65590|4294967320|37.55|40.84|false|xavier robinson|2013-03-01 09:11:58.703109|3.04|xylophone band +14|432|65699|4294967383|91.24|3.71|false|oscar quirinius|2013-03-01 09:11:58.703164|51.90|geology +116|262|65739|4294967477|17.77|35.60|true|sarah white|2013-03-01 09:11:58.703120|27.00|religion +111|460|65598|4294967309|55.34|39.43|true|tom young|2013-03-01 09:11:58.703199|76.25|wind surfing +89|445|65770|4294967317|67.36|44.86|true|victor garcia|2013-03-01 09:11:58.703298|59.99|study skills +79|421|65641|4294967329|0.16|1.44|true|mike underhill|2013-03-01 09:11:58.703237|66.36|kindergarten +1|307|65780|4294967324|0.44|9.34|false|sarah falkner|2013-03-01 09:11:58.703173|3.75|debate +105|341|65778|4294967442|54.90|9.43|true|ulysses thompson|2013-03-01 09:11:58.703137|83.12|biology +111|498|65540|4294967467|17.90|41.59|false|ethan white|2013-03-01 09:11:58.703270|47.96|philosophy +41|349|65777|4294967406|89.80|26.39|false|bob young|2013-03-01 09:11:58.703077|39.04|education +13|314|65616|4294967322|54.86|3.33|false|mike ellison|2013-03-01 09:11:58.703234|2.66|philosophy +53|320|65745|4294967333|24.83|28.87|false|yuri ovid|2013-03-01 09:11:58.703195|71.54|philosophy +60|448|65592|4294967513|15.83|1.58|true|tom hernandez|2013-03-01 09:11:58.703230|97.68|topology +63|465|65603|4294967426|0.23|5.61|true|mike steinbeck|2013-03-01 09:11:58.703277|73.88|mathematics +57|363|65627|4294967335|71.92|28.22|false|nick johnson|2013-03-01 09:11:58.703140|98.72|study skills +110|325|65575|4294967321|78.35|33.13|true|calvin steinbeck|2013-03-01 09:11:58.703200|67.18|biology +11|463|65713|4294967425|52.25|19.85|false|priscilla steinbeck|2013-03-01 09:11:58.703231|82.12|industrial engineering +16|372|65689|4294967372|2.59|44.21|false|ulysses steinbeck|2013-03-01 09:11:58.703203|0.35|geology +82|370|65542|4294967317|42.41|9.70|true|calvin ellison|2013-03-01 09:11:58.703255|48.76|study skills +101|485|65755|4294967425|68.23|39.52|false|irene van buren|2013-03-01 09:11:58.703167|64.18|education +95|305|65735|4294967394|4.41|29.66|false|priscilla davidson|2013-03-01 09:11:58.703176|43.37|quiet hour +30|308|65554|4294967401|60.27|44.29|true|katie thompson|2013-03-01 09:11:58.703253|2.35|yard duty +20|271|65619|4294967313|56.82|49.98|true|calvin ichabod|2013-03-01 09:11:58.703216|78.05|linguistics +94|331|65536|4294967341|85.65|30.84|true|priscilla young|2013-03-01 09:11:58.703135|95.88|yard duty +51|470|65681|4294967322|99.41|14.84|true|xavier thompson|2013-03-01 09:11:58.703223|38.09|opthamology +14|416|65606|4294967437|71.74|13.50|false|quinn steinbeck|2013-03-01 09:11:58.703232|87.57|topology +13|347|65732|4294967389|34.37|45.75|false|luke white|2013-03-01 09:11:58.703209|74.49|geology +107|291|65758|4294967360|18.79|25.90|true|sarah xylophone|2013-03-01 09:11:58.703097|0.38|xylophone band +-3|313|65540|4294967316|25.67|39.88|false|ulysses robinson|2013-03-01 09:11:58.703227|60.80|religion +45|503|65704|4294967373|82.24|10.83|false|holly white|2013-03-01 09:11:58.703283|87.60|mathematics +1|488|65775|4294967484|49.66|38.44|true|sarah ichabod|2013-03-01 09:11:58.703085|11.81|nap time +42|344|65557|4294967416|20.64|40.45|true|calvin carson|2013-03-01 09:11:58.703103|33.90|wind surfing +18|414|65690|4294967435|23.13|29.84|false|rachel brown|2013-03-01 09:11:58.703187|28.63|mathematics +122|272|65568|4294967427|76.41|39.38|true|sarah zipper|2013-03-01 09:11:58.703122|3.87|kindergarten +81|347|65731|4294967527|50.40|34.75|false|katie white|2013-03-01 09:11:58.703142|48.28|philosophy +70|460|65731|4294967440|60.74|5.03|true|gabriella brown|2013-03-01 09:11:58.703143|9.52|xylophone band +85|429|65722|4294967308|8.00|44.13|true|katie ellison|2013-03-01 09:11:58.703217|36.86|kindergarten +106|381|65611|4294967501|88.93|8.82|false|gabriella underhill|2013-03-01 09:11:58.703071|39.86|wind surfing +-3|320|65644|4294967434|84.39|48.00|false|sarah robinson|2013-03-01 09:11:58.703288|72.48|wind surfing +82|370|65745|4294967366|60.27|18.15|true|wendy laertes|2013-03-01 09:11:58.703268|64.01|forestry +5|337|65615|4294967340|12.94|1.25|false|rachel van buren|2013-03-01 09:11:58.703098|19.01|linguistics +62|336|65585|4294967509|95.36|2.26|true|sarah polk|2013-03-01 09:11:58.703264|84.63|debate +15|316|65748|4294967462|79.30|23.55|false|zach polk|2013-03-01 09:11:58.703280|92.47|american history +21|372|65749|4294967533|35.37|26.30|false|quinn quirinius|2013-03-01 09:11:58.703178|80.71|wind surfing +79|271|65620|4294967375|48.45|23.71|false|david davidson|2013-03-01 09:11:58.703110|70.52|forestry +90|333|65657|4294967409|70.25|45.15|false|irene johnson|2013-03-01 09:11:58.703130|38.00|joggying +60|490|65585|4294967376|43.50|23.66|false|mike hernandez|2013-03-01 09:11:58.703177|90.01|opthamology +27|321|65577|4294967453|98.10|34.22|true|ethan nixon|2013-03-01 09:11:58.703148|20.84|undecided +76|330|65597|4294967477|16.46|34.95|false|fred hernandez|2013-03-01 09:11:58.703268|20.06|religion +73|415|65629|4294967485|24.58|36.10|true|calvin thompson|2013-03-01 09:11:58.703274|92.63|philosophy +76|498|65560|4294967331|81.30|10.05|true|zach zipper|2013-03-01 09:11:58.703228|86.19|chemistry +110|417|65569|4294967395|89.64|11.89|true|holly brown|2013-03-01 09:11:58.703077|41.32|xylophone band +45|294|65659|4294967348|31.87|2.62|false|victor steinbeck|2013-03-01 09:11:58.703305|57.40|undecided +8|300|65770|4294967492|59.72|12.16|false|calvin garcia|2013-03-01 09:11:58.703283|13.19|quiet hour +87|308|65552|4294967533|40.99|24.70|true|luke robinson|2013-03-01 09:11:58.703252|90.01|xylophone band +120|319|65669|4294967304|45.52|26.79|false|sarah nixon|2013-03-01 09:11:58.703202|1.62|opthamology +38|509|65685|4294967370|32.94|35.14|true|alice laertes|2013-03-01 09:11:58.703210|51.84|linguistics +19|314|65619|4294967394|0.02|33.38|false|luke steinbeck|2013-03-01 09:11:58.703105|77.78|undecided +42|279|65696|4294967393|39.11|31.84|true|katie polk|2013-03-01 09:11:58.703163|26.42|xylophone band +70|328|65723|4294967373|17.31|42.67|false|david thompson|2013-03-01 09:11:58.703135|87.41|values clariffication +123|410|65668|4294967412|86.42|0.66|true|quinn johnson|2013-03-01 09:11:58.703194|20.15|xylophone band +6|507|65625|4294967384|90.98|34.23|false|irene young|2013-03-01 09:11:58.703161|72.07|undecided +33|326|65591|4294967519|29.10|47.00|true|xavier young|2013-03-01 09:11:58.703266|65.20|values clariffication +95|502|65600|4294967472|0.43|38.63|true|zach miller|2013-03-01 09:11:58.703146|3.88|biology +14|426|65619|4294967541|18.19|0.53|true|alice laertes|2013-03-01 09:11:58.703304|28.09|chemistry +9|291|65552|4294967518|75.41|1.85|false|nick brown|2013-03-01 09:11:58.703300|51.13|religion +6|391|65680|4294967441|7.92|40.90|true|jessica ovid|2013-03-01 09:11:58.703314|97.32|values clariffication +40|313|65635|4294967326|42.75|14.60|true|irene carson|2013-03-01 09:11:58.703198|16.19|study skills +14|283|65589|4294967411|25.39|41.39|false|irene carson|2013-03-01 09:11:58.703204|34.42|wind surfing +45|451|65682|4294967326|84.36|3.47|true|tom hernandez|2013-03-01 09:11:58.703316|9.33|opthamology +0|350|65620|4294967415|78.88|18.98|true|quinn quirinius|2013-03-01 09:11:58.703164|30.07|undecided +81|478|65623|4294967339|33.52|11.97|false|victor allen|2013-03-01 09:11:58.703273|40.11|philosophy +99|341|65666|4294967325|50.79|12.87|true|ulysses van buren|2013-03-01 09:11:58.703234|11.37|biology +19|476|65624|4294967384|5.14|48.09|false|luke van buren|2013-03-01 09:11:58.703208|23.53|values clariffication +111|426|65577|4294967325|14.06|39.54|false|oscar zipper|2013-03-01 09:11:58.703124|97.69|education +119|461|65561|4294967457|97.25|40.67|false|tom ichabod|2013-03-01 09:11:58.703188|10.34|debate +67|395|65632|4294967355|25.33|40.09|true|priscilla zipper|2013-03-01 09:11:58.703312|0.64|philosophy +92|391|65609|4294967469|13.55|29.79|false|bob davidson|2013-03-01 09:11:58.703213|59.77|american history +113|386|65648|4294967319|84.65|46.16|true|sarah ichabod|2013-03-01 09:11:58.703131|8.68|biology +72|337|65723|4294967443|42.19|16.25|false|gabriella zipper|2013-03-01 09:11:58.703260|79.33|quiet hour +54|393|65544|4294967392|9.66|42.94|true|rachel brown|2013-03-01 09:11:58.703219|88.83|nap time +117|416|65732|4294967495|82.21|12.72|true|rachel davidson|2013-03-01 09:11:58.703206|58.64|american history +49|486|65619|4294967422|52.64|2.38|false|sarah robinson|2013-03-01 09:11:58.703307|76.64|history +2|381|65549|4294967358|37.97|23.31|false|nick laertes|2013-03-01 09:11:58.703116|24.38|mathematics +74|444|65745|4294967454|90.53|46.44|false|jessica johnson|2013-03-01 09:11:58.703254|11.37|education +114|322|65582|4294967520|2.40|30.12|true|fred nixon|2013-03-01 09:11:58.703156|39.77|joggying +9|280|65536|4294967299|42.20|29.36|true|sarah thompson|2013-03-01 09:11:58.703142|5.72|philosophy +98|284|65730|4294967389|22.02|33.94|true|ulysses zipper|2013-03-01 09:11:58.703219|65.66|kindergarten +120|358|65687|4294967399|12.95|48.86|false|zach carson|2013-03-01 09:11:58.703305|18.80|chemistry +89|403|65735|4294967510|81.26|32.46|false|fred nixon|2013-03-01 09:11:58.703093|90.36|xylophone band +60|439|65558|4294967396|33.97|9.49|true|fred underhill|2013-03-01 09:11:58.703237|27.53|philosophy +37|459|65655|4294967495|59.41|22.35|false|victor davidson|2013-03-01 09:11:58.703209|22.94|topology +50|298|65554|4294967315|53.84|17.23|true|alice davidson|2013-03-01 09:11:58.703291|64.52|wind surfing +76|421|65742|4294967387|1.67|32.01|false|tom falkner|2013-03-01 09:11:58.703100|28.29|xylophone band +47|478|65709|4294967362|50.90|31.57|false|gabriella king|2013-03-01 09:11:58.703154|10.74|biology +103|289|65607|4294967408|5.72|29.41|false|oscar ellison|2013-03-01 09:11:58.703267|63.81|nap time +92|370|65579|4294967332|68.25|41.79|true|oscar ovid|2013-03-01 09:11:58.703139|97.98|yard duty +45|364|65601|4294967396|56.72|3.94|false|fred ovid|2013-03-01 09:11:58.703211|54.82|opthamology +24|499|65673|4294967476|28.72|15.71|true|rachel robinson|2013-03-01 09:11:58.703275|90.02|debate +58|261|65748|4294967349|47.48|43.42|false|ethan robinson|2013-03-01 09:11:58.703274|20.98|study skills +-2|330|65667|4294967489|63.77|43.88|false|sarah steinbeck|2013-03-01 09:11:58.703092|72.13|zync studies +121|271|65715|4294967404|96.02|16.35|true|katie quirinius|2013-03-01 09:11:58.703091|37.25|linguistics +68|472|65609|4294967315|89.57|18.39|false|alice allen|2013-03-01 09:11:58.703312|85.65|quiet hour +17|411|65766|4294967544|19.86|25.66|true|victor falkner|2013-03-01 09:11:58.703272|81.57|wind surfing +16|447|65687|4294967307|84.15|44.07|false|jessica falkner|2013-03-01 09:11:58.703134|99.40|forestry +37|278|65554|4294967340|45.97|24.17|true|calvin underhill|2013-03-01 09:11:58.703277|86.71|forestry +56|440|65719|4294967506|62.08|31.00|true|bob ovid|2013-03-01 09:11:58.703206|10.42|xylophone band +21|355|65548|4294967546|26.18|21.98|true|calvin steinbeck|2013-03-01 09:11:58.703191|75.97|nap time +3|474|65609|4294967354|7.97|9.84|true|victor garcia|2013-03-01 09:11:58.703229|14.91|education +123|312|65614|4294967508|90.93|24.46|true|yuri white|2013-03-01 09:11:58.703151|21.37|values clariffication +8|413|65664|4294967406|16.25|47.13|false|ethan thompson|2013-03-01 09:11:58.703268|85.28|zync studies +52|373|65764|4294967430|8.95|14.24|true|bob white|2013-03-01 09:11:58.703181|19.17|linguistics +39|456|65716|4294967473|93.10|14.67|false|quinn van buren|2013-03-01 09:11:58.703303|31.07|values clariffication +25|460|65592|4294967301|27.63|32.94|true|irene johnson|2013-03-01 09:11:58.703161|30.17|wind surfing +97|384|65785|4294967482|6.58|37.04|true|irene robinson|2013-03-01 09:11:58.703301|10.14|wind surfing +82|505|65780|4294967366|46.81|27.70|false|quinn hernandez|2013-03-01 09:11:58.703156|82.06|quiet hour +72|444|65681|4294967399|23.01|2.89|true|alice nixon|2013-03-01 09:11:58.703240|92.50|chemistry +51|285|65567|4294967528|29.77|4.40|true|bob laertes|2013-03-01 09:11:58.703260|77.53|wind surfing +114|403|65726|4294967322|41.68|29.58|true|xavier polk|2013-03-01 09:11:58.703297|61.34|yard duty +28|317|65590|4294967524|77.29|23.05|false|ulysses hernandez|2013-03-01 09:11:58.703268|95.69|linguistics +104|425|65691|4294967396|35.60|17.06|false|yuri brown|2013-03-01 09:11:58.703228|65.28|nap time +5|300|65768|4294967373|84.93|31.79|false|david allen|2013-03-01 09:11:58.703182|14.04|wind surfing +100|480|65607|4294967409|23.84|14.90|true|wendy underhill|2013-03-01 09:11:58.703282|12.41|opthamology +48|268|65727|4294967503|69.77|14.46|false|yuri nixon|2013-03-01 09:11:58.703272|16.11|forestry +47|477|65705|4294967334|40.51|7.32|true|tom allen|2013-03-01 09:11:58.703079|19.42|yard duty +93|411|65709|4294967450|90.99|21.51|true|priscilla king|2013-03-01 09:11:58.703286|83.52|forestry +71|401|65735|4294967508|26.78|43.35|false|ulysses quirinius|2013-03-01 09:11:58.703248|99.38|history +87|302|65583|4294967349|98.29|17.15|false|quinn xylophone|2013-03-01 09:11:58.703161|2.37|quiet hour +61|276|65658|4294967343|55.76|4.44|true|xavier garcia|2013-03-01 09:11:58.703278|88.69|debate +99|454|65590|4294967535|24.09|4.88|false|fred king|2013-03-01 09:11:58.703272|41.61|quiet hour +118|366|65781|4294967363|5.13|8.81|false|luke miller|2013-03-01 09:11:58.703152|77.25|opthamology +61|304|65737|4294967458|88.97|40.84|false|irene polk|2013-03-01 09:11:58.703131|85.80|opthamology +38|415|65640|4294967319|19.56|1.00|false|nick miller|2013-03-01 09:11:58.703184|16.68|linguistics +66|335|65720|4294967520|15.28|8.24|true|yuri hernandez|2013-03-01 09:11:58.703272|26.22|zync studies +21|362|65667|4294967299|53.79|16.87|true|bob laertes|2013-03-01 09:11:58.703303|39.12|industrial engineering +48|469|65625|4294967500|41.38|7.70|false|jessica miller|2013-03-01 09:11:58.703104|32.85|joggying +20|406|65654|4294967508|58.14|47.42|false|victor brown|2013-03-01 09:11:58.703119|23.07|linguistics +38|417|65727|4294967433|97.89|16.57|true|jessica white|2013-03-01 09:11:58.703193|44.84|industrial engineering +86|338|65650|4294967377|60.72|1.15|true|victor young|2013-03-01 09:11:58.703195|94.47|mathematics +10|467|65622|4294967404|48.78|36.09|false|luke laertes|2013-03-01 09:11:58.703312|29.59|mathematics +25|340|65653|4294967327|15.50|32.33|false|david young|2013-03-01 09:11:58.703316|49.17|geology +103|294|65588|4294967548|12.90|28.23|true|luke brown|2013-03-01 09:11:58.703168|69.33|biology +-3|485|65669|4294967428|21.34|13.07|false|priscilla zipper|2013-03-01 09:11:58.703321|27.89|quiet hour +19|460|65767|4294967371|66.53|43.75|false|ethan ichabod|2013-03-01 09:11:58.703226|40.41|geology +90|415|65583|4294967342|75.91|28.98|false|sarah polk|2013-03-01 09:11:58.703278|66.71|debate +20|315|65747|4294967351|84.20|10.69|false|calvin falkner|2013-03-01 09:11:58.703159|52.98|kindergarten +15|256|65653|4294967536|8.10|23.50|true|calvin johnson|2013-03-01 09:11:58.703291|85.08|xylophone band +54|398|65701|4294967496|3.58|36.62|true|ethan thompson|2013-03-01 09:11:58.703288|18.68|quiet hour +112|315|65757|4294967317|35.02|2.06|true|david falkner|2013-03-01 09:11:58.703096|58.70|geology +1|293|65677|4294967337|1.04|29.66|false|holly ellison|2013-03-01 09:11:58.703161|10.47|education +92|406|65727|4294967550|13.32|36.57|true|bob falkner|2013-03-01 09:11:58.703233|57.28|linguistics +37|275|65623|4294967428|36.34|12.13|true|rachel robinson|2013-03-01 09:11:58.703254|72.17|quiet hour +99|471|65612|4294967443|0.42|31.86|false|mike allen|2013-03-01 09:11:58.703221|6.64|zync studies +61|384|65707|4294967421|69.66|36.72|true|gabriella quirinius|2013-03-01 09:11:58.703206|25.68|industrial engineering +15|411|65562|4294967390|9.00|28.87|false|holly ovid|2013-03-01 09:11:58.703193|60.78|mathematics +-2|475|65782|4294967308|69.87|18.00|false|ethan nixon|2013-03-01 09:11:58.703294|10.91|industrial engineering +83|391|65660|4294967333|27.60|24.98|false|calvin polk|2013-03-01 09:11:58.703102|42.74|education +98|502|65705|4294967329|21.28|12.82|true|katie white|2013-03-01 09:11:58.703202|77.75|values clariffication +59|298|65779|4294967373|91.57|39.66|true|sarah allen|2013-03-01 09:11:58.703303|33.03|geology +74|381|65717|4294967442|95.68|20.36|false|holly steinbeck|2013-03-01 09:11:58.703301|30.20|kindergarten +-3|316|65696|4294967445|22.00|43.41|false|priscilla laertes|2013-03-01 09:11:58.703250|51.43|values clariffication +83|317|65568|4294967504|88.46|27.58|true|ethan underhill|2013-03-01 09:11:58.703224|0.56|history +83|383|65665|4294967361|5.29|0.02|false|holly polk|2013-03-01 09:11:58.703134|91.66|opthamology +41|418|65770|4294967324|20.99|47.46|false|luke davidson|2013-03-01 09:11:58.703111|6.61|study skills +113|500|65728|4294967394|63.24|46.55|false|xavier laertes|2013-03-01 09:11:58.703314|28.91|philosophy +40|413|65590|4294967307|87.12|36.97|false|nick zipper|2013-03-01 09:11:58.703131|15.83|topology +49|402|65618|4294967425|62.41|1.77|true|fred falkner|2013-03-01 09:11:58.703131|28.11|debate +84|440|65791|4294967485|68.94|37.49|false|rachel ichabod|2013-03-01 09:11:58.703308|9.58|topology +94|393|65645|4294967490|32.14|2.65|true|victor van buren|2013-03-01 09:11:58.703182|34.82|chemistry +25|392|65641|4294967368|66.11|4.24|true|oscar xylophone|2013-03-01 09:11:58.703277|5.20|undecided +112|371|65622|4294967420|3.62|18.95|false|nick steinbeck|2013-03-01 09:11:58.703103|32.15|nap time +114|281|65747|4294967479|74.37|31.93|true|rachel white|2013-03-01 09:11:58.703287|56.43|nap time +15|353|65601|4294967308|46.58|39.36|true|calvin quirinius|2013-03-01 09:11:58.703137|95.11|education +101|502|65716|4294967369|11.60|41.49|true|nick davidson|2013-03-01 09:11:58.703279|11.37|linguistics +116|416|65749|4294967523|66.99|7.69|false|david zipper|2013-03-01 09:11:58.703085|68.97|history +43|366|65590|4294967450|24.84|18.01|true|alice ichabod|2013-03-01 09:11:58.703312|63.48|debate +66|354|65579|4294967505|36.98|14.90|true|david zipper|2013-03-01 09:11:58.703299|39.90|linguistics +63|420|65626|4294967525|18.36|32.10|true|holly young|2013-03-01 09:11:58.703295|83.63|joggying +36|268|65748|4294967507|58.95|22.48|false|calvin underhill|2013-03-01 09:11:58.703213|55.14|topology +77|390|65631|4294967497|21.41|36.53|false|katie zipper|2013-03-01 09:11:58.703262|71.08|values clariffication +35|496|65703|4294967551|17.27|4.50|false|holly thompson|2013-03-01 09:11:58.703127|85.43|religion +15|349|65745|4294967459|64.96|6.17|true|bob ellison|2013-03-01 09:11:58.703105|7.99|forestry +108|389|65737|4294967448|45.46|49.33|false|rachel steinbeck|2013-03-01 09:11:58.703190|5.05|chemistry +20|510|65677|4294967532|58.85|33.76|false|oscar allen|2013-03-01 09:11:58.703127|45.13|kindergarten +108|463|65653|4294967483|32.16|8.65|true|calvin zipper|2013-03-01 09:11:58.703110|94.13|forestry +74|480|65711|4294967299|55.17|4.13|false|holly falkner|2013-03-01 09:11:58.703320|87.19|history +82|430|65722|4294967518|6.93|19.06|true|yuri laertes|2013-03-01 09:11:58.703247|44.30|nap time +53|340|65582|4294967359|49.89|48.62|true|victor brown|2013-03-01 09:11:58.703214|83.06|opthamology +40|311|65645|4294967520|86.73|2.77|false|katie steinbeck|2013-03-01 09:11:58.703181|71.66|biology +100|288|65660|4294967382|12.43|25.92|true|bob allen|2013-03-01 09:11:58.703129|60.67|yard duty +119|448|65551|4294967371|25.03|26.17|false|tom young|2013-03-01 09:11:58.703177|11.43|quiet hour +121|340|65600|4294967496|43.10|38.84|false|bob white|2013-03-01 09:11:58.703104|76.75|chemistry +78|378|65662|4294967372|88.94|9.90|false|luke quirinius|2013-03-01 09:11:58.703155|85.02|history +109|487|65749|4294967400|39.78|25.70|true|ethan polk|2013-03-01 09:11:58.703121|79.96|biology +2|448|65723|4294967509|95.02|41.54|true|xavier robinson|2013-03-01 09:11:58.703305|94.63|chemistry +76|256|65738|4294967514|9.00|12.34|true|rachel ellison|2013-03-01 09:11:58.703072|10.32|education +9|501|65782|4294967404|61.23|40.87|true|calvin van buren|2013-03-01 09:11:58.703193|10.63|wind surfing +21|280|65606|4294967429|82.07|24.59|false|tom brown|2013-03-01 09:11:58.703108|79.56|values clariffication +32|494|65682|4294967403|81.72|19.31|true|rachel steinbeck|2013-03-01 09:11:58.703305|74.75|study skills +78|333|65683|4294967417|94.02|42.52|true|oscar garcia|2013-03-01 09:11:58.703297|95.45|opthamology +22|318|65700|4294967458|1.73|8.82|false|rachel young|2013-03-01 09:11:58.703073|46.47|kindergarten +113|275|65725|4294967412|52.82|26.27|true|yuri ichabod|2013-03-01 09:11:58.703123|8.55|kindergarten +65|384|65753|4294967457|62.55|11.27|false|mike brown|2013-03-01 09:11:58.703113|71.21|forestry +3|356|65672|4294967425|66.93|8.66|true|victor ichabod|2013-03-01 09:11:58.703190|71.90|study skills +72|460|65653|4294967429|1.27|37.31|true|david young|2013-03-01 09:11:58.703241|47.12|opthamology +106|266|65671|4294967434|12.66|33.13|true|rachel miller|2013-03-01 09:11:58.703249|13.32|study skills +19|264|65652|4294967425|26.12|14.18|false|xavier ellison|2013-03-01 09:11:58.703275|12.26|zync studies +115|322|65646|4294967459|7.38|15.16|false|yuri white|2013-03-01 09:11:58.703179|26.10|topology +102|373|65683|4294967321|46.83|43.87|true|holly garcia|2013-03-01 09:11:58.703240|65.85|quiet hour +6|306|65585|4294967535|11.54|22.73|true|tom ovid|2013-03-01 09:11:58.703185|86.38|study skills +22|263|65599|4294967494|19.50|38.18|false|xavier ichabod|2013-03-01 09:11:58.703192|76.09|nap time +26|474|65737|4294967449|11.03|5.89|true|oscar king|2013-03-01 09:11:58.703222|5.46|joggying +45|399|65583|4294967400|67.32|43.41|true|rachel robinson|2013-03-01 09:11:58.703175|12.59|undecided +39|310|65732|4294967461|31.78|7.03|false|alice miller|2013-03-01 09:11:58.703205|21.83|geology +6|396|65646|4294967303|62.81|46.87|false|irene quirinius|2013-03-01 09:11:58.703201|74.16|forestry +57|292|65572|4294967339|11.07|2.16|true|tom allen|2013-03-01 09:11:58.703093|31.11|opthamology +108|406|65660|4294967347|2.66|28.22|true|zach xylophone|2013-03-01 09:11:58.703170|61.83|history +45|325|65582|4294967297|94.10|9.61|false|bob johnson|2013-03-01 09:11:58.703190|42.39|undecided +75|498|65545|4294967455|74.18|32.10|true|gabriella underhill|2013-03-01 09:11:58.703170|73.48|joggying +36|433|65567|4294967515|40.55|5.81|true|yuri quirinius|2013-03-01 09:11:58.703164|36.88|industrial engineering +61|399|65673|4294967448|1.85|43.99|false|nick hernandez|2013-03-01 09:11:58.703271|69.52|yard duty +108|424|65756|4294967443|13.78|7.92|true|calvin king|2013-03-01 09:11:58.703077|88.45|religion +83|262|65761|4294967395|73.73|49.10|false|alice ovid|2013-03-01 09:11:58.703184|79.04|american history +93|434|65761|4294967390|85.32|20.87|true|wendy hernandez|2013-03-01 09:11:58.703198|87.19|undecided +35|412|65770|4294967525|81.12|43.60|false|gabriella polk|2013-03-01 09:11:58.703237|82.78|debate +3|444|65735|4294967487|18.91|0.96|true|sarah miller|2013-03-01 09:11:58.703283|84.26|opthamology +83|310|65670|4294967399|59.72|0.09|true|victor davidson|2013-03-01 09:11:58.703219|23.09|study skills +99|392|65763|4294967339|1.97|10.19|false|sarah nixon|2013-03-01 09:11:58.703112|51.66|xylophone band +56|372|65658|4294967542|97.72|13.07|true|wendy steinbeck|2013-03-01 09:11:58.703264|51.54|philosophy +93|442|65664|4294967419|68.02|24.51|false|wendy laertes|2013-03-01 09:11:58.703237|21.60|opthamology +18|398|65574|4294967397|75.31|16.79|false|ethan king|2013-03-01 09:11:58.703144|32.63|biology +49|332|65590|4294967510|0.62|15.58|true|oscar ichabod|2013-03-01 09:11:58.703104|39.06|values clariffication +1|432|65692|4294967513|20.50|26.52|false|calvin steinbeck|2013-03-01 09:11:58.703160|76.04|religion +40|423|65541|4294967526|29.09|49.03|false|wendy ovid|2013-03-01 09:11:58.703099|26.01|wind surfing +124|414|65640|4294967441|9.36|5.08|true|katie steinbeck|2013-03-01 09:11:58.703156|71.66|geology +44|498|65696|4294967536|16.92|32.88|false|ethan ichabod|2013-03-01 09:11:58.703135|18.43|joggying +91|289|65726|4294967439|2.91|49.40|false|david laertes|2013-03-01 09:11:58.703204|49.19|forestry +59|341|65633|4294967499|7.07|7.76|true|yuri van buren|2013-03-01 09:11:58.703325|95.94|forestry +95|418|65674|4294967514|99.44|27.31|false|priscilla falkner|2013-03-01 09:11:58.703309|79.65|wind surfing +44|472|65614|4294967464|23.13|45.45|false|holly davidson|2013-03-01 09:11:58.703112|80.10|undecided +64|420|65543|4294967335|89.49|21.89|true|gabriella underhill|2013-03-01 09:11:58.703290|77.75|geology +8|351|65624|4294967338|32.03|14.69|true|rachel allen|2013-03-01 09:11:58.703206|30.08|yard duty +68|364|65652|4294967544|30.64|10.43|true|bob ovid|2013-03-01 09:11:58.703130|52.72|study skills +28|258|65771|4294967406|52.63|32.42|false|zach zipper|2013-03-01 09:11:58.703151|14.28|topology +44|289|65652|4294967447|11.51|16.86|true|wendy garcia|2013-03-01 09:11:58.703218|91.38|quiet hour +71|361|65718|4294967397|45.04|45.31|false|oscar carson|2013-03-01 09:11:58.703147|82.90|philosophy +30|504|65664|4294967477|48.30|30.35|true|luke xylophone|2013-03-01 09:11:58.703173|38.98|quiet hour +50|327|65612|4294967367|37.71|30.46|false|victor ichabod|2013-03-01 09:11:58.703323|10.00|nap time +22|434|65755|4294967543|52.73|29.00|true|holly hernandez|2013-03-01 09:11:58.703179|73.46|yard duty +65|303|65790|4294967470|43.92|14.62|false|jessica underhill|2013-03-01 09:11:58.703235|13.92|linguistics +118|428|65564|4294967389|28.31|35.83|false|jessica underhill|2013-03-01 09:11:58.703250|44.58|joggying +94|296|65570|4294967455|85.01|40.08|true|jessica ovid|2013-03-01 09:11:58.703169|2.14|kindergarten +10|280|65660|4294967321|91.15|14.80|true|sarah young|2013-03-01 09:11:58.703295|72.67|yard duty +30|284|65657|4294967487|75.93|0.20|false|tom xylophone|2013-03-01 09:11:58.703260|20.69|opthamology +119|425|65680|4294967361|72.98|37.37|false|katie polk|2013-03-01 09:11:58.703150|71.90|american history +72|301|65603|4294967478|38.50|30.42|true|gabriella steinbeck|2013-03-01 09:11:58.703071|19.27|american history +17|258|65579|4294967447|47.52|14.88|true|jessica ichabod|2013-03-01 09:11:58.703207|79.45|debate +101|366|65790|4294967324|89.50|20.23|true|wendy falkner|2013-03-01 09:11:58.703239|70.92|undecided +109|424|65684|4294967473|26.19|49.17|true|ulysses underhill|2013-03-01 09:11:58.703270|36.41|quiet hour +55|486|65675|4294967313|4.28|8.52|false|sarah zipper|2013-03-01 09:11:58.703279|10.03|mathematics +66|452|65594|4294967506|96.62|3.21|false|zach allen|2013-03-01 09:11:58.703253|76.09|forestry +80|307|65772|4294967376|82.16|38.76|false|wendy carson|2013-03-01 09:11:58.703102|73.79|forestry +60|337|65648|4294967528|25.94|34.79|true|irene king|2013-03-01 09:11:58.703168|33.63|xylophone band +112|506|65632|4294967408|37.13|31.93|false|calvin davidson|2013-03-01 09:11:58.703090|50.80|quiet hour +88|387|65699|4294967424|21.96|20.06|false|victor xylophone|2013-03-01 09:11:58.703233|94.69|geology +93|403|65565|4294967518|6.76|33.41|true|priscilla allen|2013-03-01 09:11:58.703280|55.90|debate +34|266|65755|4294967481|37.76|19.63|false|alice miller|2013-03-01 09:11:58.703257|52.69|education +44|484|65629|4294967361|13.17|5.84|false|luke steinbeck|2013-03-01 09:11:58.703238|62.37|mathematics +112|257|65574|4294967490|19.96|7.80|false|zach hernandez|2013-03-01 09:11:58.703253|41.76|history +2|312|65767|4294967538|23.05|22.83|true|fred king|2013-03-01 09:11:58.703166|31.87|nap time +23|406|65752|4294967309|37.31|45.70|true|alice nixon|2013-03-01 09:11:58.703191|58.05|debate +124|321|65711|4294967444|38.43|18.35|false|oscar robinson|2013-03-01 09:11:58.703140|88.35|topology +20|381|65619|4294967509|84.37|31.73|false|priscilla allen|2013-03-01 09:11:58.703277|73.58|study skills +46|458|65630|4294967541|6.81|13.93|true|yuri johnson|2013-03-01 09:11:58.703179|85.01|american history +85|293|65592|4294967417|35.59|6.90|true|quinn van buren|2013-03-01 09:11:58.703295|85.73|topology +110|468|65574|4294967544|67.75|30.25|false|bob ichabod|2013-03-01 09:11:58.703086|13.12|joggying +98|345|65715|4294967533|26.76|29.92|true|mike hernandez|2013-03-01 09:11:58.703203|31.22|values clariffication +64|467|65642|4294967447|53.51|41.28|true|rachel allen|2013-03-01 09:11:58.703174|98.15|geology +32|399|65677|4294967411|65.00|15.28|true|quinn nixon|2013-03-01 09:11:58.703236|25.63|wind surfing +13|417|65545|4294967510|33.80|47.84|false|wendy thompson|2013-03-01 09:11:58.703142|21.92|forestry +66|380|65681|4294967491|14.07|49.22|true|david brown|2013-03-01 09:11:58.703279|69.30|mathematics +83|376|65665|4294967541|38.79|33.72|true|calvin hernandez|2013-03-01 09:11:58.703305|0.03|debate +13|487|65710|4294967547|69.70|11.48|true|rachel davidson|2013-03-01 09:11:58.703143|86.43|quiet hour +88|445|65571|4294967338|10.75|20.18|false|wendy davidson|2013-03-01 09:11:58.703107|30.96|religion +63|416|65546|4294967503|28.87|36.45|true|yuri thompson|2013-03-01 09:11:58.703228|37.39|debate +85|481|65774|4294967524|22.69|5.63|true|xavier carson|2013-03-01 09:11:58.703286|17.25|education +50|418|65562|4294967481|68.65|44.81|false|jessica xylophone|2013-03-01 09:11:58.703275|64.49|kindergarten +-3|454|65705|4294967468|62.12|14.32|true|mike white|2013-03-01 09:11:58.703087|40.18|joggying +37|509|65648|4294967394|83.45|16.94|false|irene nixon|2013-03-01 09:11:58.703095|98.01|study skills +-3|318|65553|4294967452|9.86|32.77|false|holly underhill|2013-03-01 09:11:58.703219|47.25|wind surfing +45|502|65759|4294967518|84.59|11.17|false|luke robinson|2013-03-01 09:11:58.703242|63.28|wind surfing +59|271|65771|4294967508|47.57|34.86|true|quinn van buren|2013-03-01 09:11:58.703273|86.41|undecided +63|272|65565|4294967543|43.36|4.47|false|calvin young|2013-03-01 09:11:58.703285|90.34|joggying +98|446|65538|4294967342|2.80|35.13|true|luke davidson|2013-03-01 09:11:58.703083|92.44|joggying +51|318|65734|4294967391|57.42|49.33|true|mike falkner|2013-03-01 09:11:58.703139|34.80|opthamology +121|507|65762|4294967358|88.18|16.48|false|oscar ellison|2013-03-01 09:11:58.703132|99.77|undecided +37|350|65611|4294967420|41.42|49.64|false|calvin miller|2013-03-01 09:11:58.703104|73.54|study skills +116|497|65645|4294967447|10.26|1.92|false|quinn nixon|2013-03-01 09:11:58.703154|20.68|history +93|368|65777|4294967354|85.25|8.67|true|priscilla carson|2013-03-01 09:11:58.703111|39.52|geology +53|281|65557|4294967391|14.75|30.56|true|rachel brown|2013-03-01 09:11:58.703322|90.85|nap time +29|376|65674|4294967396|87.93|45.89|true|ethan young|2013-03-01 09:11:58.703071|8.64|nap time +114|337|65707|4294967382|97.94|5.92|true|alice brown|2013-03-01 09:11:58.703087|42.43|biology +19|264|65591|4294967531|61.58|13.95|false|ulysses davidson|2013-03-01 09:11:58.703082|46.26|chemistry +15|483|65658|4294967446|48.84|9.84|false|mike thompson|2013-03-01 09:11:58.703248|21.61|joggying +80|421|65765|4294967499|8.13|25.08|false|sarah white|2013-03-01 09:11:58.703266|63.03|linguistics +76|271|65627|4294967508|93.47|17.33|false|david davidson|2013-03-01 09:11:58.703094|19.77|religion +19|479|65751|4294967378|1.86|15.29|true|calvin allen|2013-03-01 09:11:58.703136|18.52|undecided +97|344|65581|4294967551|91.11|7.13|false|luke zipper|2013-03-01 09:11:58.703083|37.56|chemistry +21|494|65753|4294967296|62.50|36.92|false|mike xylophone|2013-03-01 09:11:58.703132|46.54|undecided +74|326|65545|4294967300|83.04|39.55|true|holly miller|2013-03-01 09:11:58.703281|47.49|joggying +65|491|65619|4294967437|77.92|23.75|true|mike ellison|2013-03-01 09:11:58.703319|58.82|biology +98|442|65585|4294967446|16.59|16.76|false|irene thompson|2013-03-01 09:11:58.703293|64.12|kindergarten +70|384|65696|4294967335|31.05|19.10|true|ulysses hernandez|2013-03-01 09:11:58.703297|13.05|zync studies +64|301|65696|4294967304|61.43|1.73|true|jessica davidson|2013-03-01 09:11:58.703155|71.57|kindergarten +102|457|65692|4294967454|3.44|45.84|true|luke steinbeck|2013-03-01 09:11:58.703242|7.36|chemistry +6|288|65695|4294967448|1.42|0.23|true|gabriella white|2013-03-01 09:11:58.703182|94.71|undecided +46|395|65672|4294967520|67.22|27.46|false|priscilla allen|2013-03-01 09:11:58.703189|86.71|nap time +62|266|65725|4294967499|22.91|5.74|true|katie ichabod|2013-03-01 09:11:58.703143|35.14|philosophy +89|404|65741|4294967522|39.05|42.34|false|holly ichabod|2013-03-01 09:11:58.703194|16.66|forestry +83|448|65763|4294967334|30.81|19.44|false|mike underhill|2013-03-01 09:11:58.703161|23.45|zync studies +68|287|65605|4294967328|52.40|8.04|true|jessica king|2013-03-01 09:11:58.703086|96.57|opthamology +111|473|65560|4294967502|4.85|37.40|false|mike steinbeck|2013-03-01 09:11:58.703091|67.37|philosophy +-1|371|65568|4294967454|10.24|49.63|true|zach underhill|2013-03-01 09:11:58.703166|48.59|history +36|276|65788|4294967453|62.66|43.97|false|david steinbeck|2013-03-01 09:11:58.703173|0.21|linguistics +-3|433|65654|4294967455|6.83|5.33|false|bob van buren|2013-03-01 09:11:58.703199|28.85|yard duty +60|510|65691|4294967461|21.29|14.39|true|david allen|2013-03-01 09:11:58.703289|55.25|quiet hour +71|467|65773|4294967360|56.48|14.08|false|calvin king|2013-03-01 09:11:58.703174|65.27|chemistry +84|304|65748|4294967337|46.58|11.00|true|jessica young|2013-03-01 09:11:58.703102|10.56|industrial engineering +34|325|65763|4294967440|37.84|13.55|false|holly laertes|2013-03-01 09:11:58.703211|30.58|yard duty +44|376|65758|4294967520|48.58|5.97|true|ulysses johnson|2013-03-01 09:11:58.703116|63.77|xylophone band +98|327|65732|4294967298|55.00|2.58|false|mike van buren|2013-03-01 09:11:58.703191|10.29|philosophy +3|261|65578|4294967418|22.87|29.07|false|quinn quirinius|2013-03-01 09:11:58.703130|8.64|values clariffication +65|348|65556|4294967425|31.22|40.38|false|luke xylophone|2013-03-01 09:11:58.703169|23.84|chemistry +91|349|65566|4294967380|50.05|21.81|true|xavier thompson|2013-03-01 09:11:58.703250|94.53|quiet hour +28|365|65546|4294967320|87.50|14.85|false|wendy johnson|2013-03-01 09:11:58.703138|1.85|mathematics +46|342|65700|4294967477|51.04|29.52|true|calvin miller|2013-03-01 09:11:58.703079|55.38|kindergarten +103|307|65608|4294967452|16.57|14.80|true|oscar thompson|2013-03-01 09:11:58.703136|14.39|zync studies +55|416|65771|4294967298|52.09|29.94|false|priscilla hernandez|2013-03-01 09:11:58.703071|98.16|education +70|361|65684|4294967456|43.07|35.50|true|nick ichabod|2013-03-01 09:11:58.703160|84.17|yard duty +23|417|65786|4294967541|42.56|31.57|true|ethan polk|2013-03-01 09:11:58.703171|40.07|american history +111|499|65790|4294967473|62.51|33.57|false|alice brown|2013-03-01 09:11:58.703152|3.74|religion +42|357|65563|4294967332|8.43|26.31|false|ethan garcia|2013-03-01 09:11:58.703168|80.61|yard duty +98|459|65595|4294967525|75.96|26.55|false|bob brown|2013-03-01 09:11:58.703235|8.78|geology +110|319|65633|4294967300|97.35|0.09|true|holly falkner|2013-03-01 09:11:58.703081|87.83|opthamology +83|462|65669|4294967459|82.69|3.06|true|quinn steinbeck|2013-03-01 09:11:58.703086|38.70|joggying +17|360|65733|4294967549|15.01|40.91|true|priscilla hernandez|2013-03-01 09:11:58.703170|68.76|industrial engineering +73|493|65631|4294967491|61.23|23.46|false|jessica johnson|2013-03-01 09:11:58.703229|42.02|study skills +116|382|65690|4294967318|68.72|36.96|true|holly thompson|2013-03-01 09:11:58.703114|80.81|opthamology +26|306|65621|4294967363|58.27|45.82|false|ethan xylophone|2013-03-01 09:11:58.703271|57.33|history +120|305|65607|4294967438|62.35|27.91|true|zach carson|2013-03-01 09:11:58.703157|25.41|geology +35|288|65727|4294967343|16.84|10.17|true|holly king|2013-03-01 09:11:58.703090|20.43|study skills +78|381|65606|4294967473|6.24|9.90|false|david zipper|2013-03-01 09:11:58.703234|45.82|geology +6|408|65551|4294967456|7.94|3.67|true|david laertes|2013-03-01 09:11:58.703093|15.29|education +44|300|65709|4294967364|37.70|49.69|false|rachel carson|2013-03-01 09:11:58.703090|29.00|debate +62|317|65606|4294967359|85.51|11.38|false|sarah quirinius|2013-03-01 09:11:58.703151|0.30|religion +10|510|65655|4294967312|51.36|34.95|false|ulysses carson|2013-03-01 09:11:58.703141|86.15|nap time +86|395|65663|4294967385|61.18|31.79|true|sarah underhill|2013-03-01 09:11:58.703151|1.55|forestry +96|461|65776|4294967355|13.87|27.43|true|oscar xylophone|2013-03-01 09:11:58.703260|58.14|geology +96|362|65679|4294967326|94.63|2.52|false|luke nixon|2013-03-01 09:11:58.703317|73.07|linguistics +60|393|65685|4294967465|85.10|41.29|true|alice falkner|2013-03-01 09:11:58.703074|60.24|debate +17|492|65778|4294967422|15.73|40.38|true|mike johnson|2013-03-01 09:11:58.703164|77.51|forestry +61|345|65751|4294967481|87.03|43.01|false|ethan quirinius|2013-03-01 09:11:58.703149|46.43|undecided +71|292|65605|4294967521|37.71|0.62|false|wendy garcia|2013-03-01 09:11:58.703304|42.91|kindergarten +119|393|65536|4294967460|52.70|9.61|true|priscilla xylophone|2013-03-01 09:11:58.703310|35.07|joggying +95|413|65716|4294967297|85.00|3.56|true|holly king|2013-03-01 09:11:58.703180|98.46|wind surfing +77|266|65676|4294967530|50.59|39.78|true|jessica miller|2013-03-01 09:11:58.703112|97.56|biology +75|488|65718|4294967323|45.58|11.56|false|calvin ovid|2013-03-01 09:11:58.703078|88.03|history +91|315|65569|4294967400|14.68|37.80|true|tom zipper|2013-03-01 09:11:58.703179|66.85|biology +43|263|65690|4294967515|66.50|17.06|true|gabriella falkner|2013-03-01 09:11:58.703176|1.37|xylophone band +105|413|65568|4294967315|17.07|32.15|true|priscilla thompson|2013-03-01 09:11:58.703321|76.06|undecided +14|262|65576|4294967463|40.83|21.83|true|mike white|2013-03-01 09:11:58.703230|52.23|geology +33|420|65735|4294967442|98.89|0.09|false|quinn falkner|2013-03-01 09:11:58.703314|86.85|values clariffication +81|304|65756|4294967363|86.84|49.20|true|alice miller|2013-03-01 09:11:58.703131|68.01|religion +69|382|65668|4294967476|25.31|46.49|true|gabriella white|2013-03-01 09:11:58.703229|13.89|history +107|371|65744|4294967305|93.75|35.41|true|nick quirinius|2013-03-01 09:11:58.703284|38.08|industrial engineering +48|476|65782|4294967541|53.36|28.36|false|oscar carson|2013-03-01 09:11:58.703150|95.12|education +84|341|65709|4294967486|4.46|41.88|false|luke robinson|2013-03-01 09:11:58.703125|97.39|nap time +23|292|65744|4294967392|94.99|28.20|false|rachel miller|2013-03-01 09:11:58.703305|98.13|wind surfing +82|441|65659|4294967379|23.23|18.83|true|david polk|2013-03-01 09:11:58.703186|98.32|industrial engineering +97|354|65685|4294967423|24.29|6.89|false|calvin johnson|2013-03-01 09:11:58.703126|89.67|religion +30|310|65613|4294967512|75.77|31.13|false|sarah davidson|2013-03-01 09:11:58.703117|46.48|undecided +43|334|65730|4294967347|15.59|12.56|true|katie allen|2013-03-01 09:11:58.703186|29.99|history +1|319|65605|4294967431|10.86|29.97|false|sarah young|2013-03-01 09:11:58.703271|46.92|topology +57|338|65701|4294967319|9.02|34.27|true|gabriella polk|2013-03-01 09:11:58.703230|84.10|joggying +16|397|65668|4294967486|15.59|18.73|false|calvin carson|2013-03-01 09:11:58.703103|23.19|study skills +25|320|65756|4294967347|81.97|19.15|true|calvin brown|2013-03-01 09:11:58.703264|70.36|joggying +11|403|65775|4294967491|21.73|18.73|false|priscilla davidson|2013-03-01 09:11:58.703218|36.28|zync studies +-1|423|65751|4294967518|58.41|18.52|true|calvin king|2013-03-01 09:11:58.703126|91.16|biology +13|350|65746|4294967330|7.79|39.53|true|jessica nixon|2013-03-01 09:11:58.703189|40.64|zync studies +66|287|65674|4294967307|13.52|42.50|true|holly quirinius|2013-03-01 09:11:58.703211|12.15|quiet hour +91|389|65718|4294967497|50.08|0.16|true|fred nixon|2013-03-01 09:11:58.703121|72.03|linguistics +58|483|65726|4294967382|28.42|34.37|true|irene hernandez|2013-03-01 09:11:58.703291|75.15|philosophy +25|275|65707|4294967319|46.99|31.15|false|david garcia|2013-03-01 09:11:58.703119|66.67|forestry +117|275|65584|4294967524|19.22|23.71|true|jessica king|2013-03-01 09:11:58.703298|44.67|mathematics +99|277|65712|4294967460|33.47|13.07|false|ethan allen|2013-03-01 09:11:58.703191|61.96|wind surfing +58|459|65780|4294967395|5.32|33.75|false|quinn miller|2013-03-01 09:11:58.703074|0.86|forestry +-2|420|65684|4294967492|17.09|25.44|true|tom ellison|2013-03-01 09:11:58.703227|27.57|linguistics +49|463|65558|4294967376|95.47|0.70|true|alice quirinius|2013-03-01 09:11:58.703178|0.23|study skills +99|415|65568|4294967422|3.50|39.80|false|xavier van buren|2013-03-01 09:11:58.703307|20.56|zync studies +115|273|65761|4294967348|54.25|13.17|true|mike thompson|2013-03-01 09:11:58.703162|31.68|quiet hour +45|289|65539|4294967344|37.60|43.55|false|rachel laertes|2013-03-01 09:11:58.703156|50.02|debate +21|426|65663|4294967425|69.05|46.89|false|holly king|2013-03-01 09:11:58.703079|98.28|quiet hour +69|455|65590|4294967536|66.64|25.17|false|mike nixon|2013-03-01 09:11:58.703162|75.65|yard duty +27|275|65545|4294967403|65.73|4.39|false|sarah miller|2013-03-01 09:11:58.703118|54.75|quiet hour +58|374|65609|4294967413|33.62|38.89|false|quinn garcia|2013-03-01 09:11:58.703255|50.25|yard duty +67|349|65654|4294967378|22.03|20.94|false|jessica robinson|2013-03-01 09:11:58.703233|27.27|values clariffication +35|511|65591|4294967373|33.52|45.16|false|quinn ellison|2013-03-01 09:11:58.703316|48.54|debate +4|275|65539|4294967342|99.68|16.89|true|quinn ichabod|2013-03-01 09:11:58.703139|35.82|philosophy +69|316|65775|4294967324|10.18|44.90|true|fred underhill|2013-03-01 09:11:58.703075|4.35|history +5|281|65553|4294967505|61.05|12.33|false|yuri robinson|2013-03-01 09:11:58.703299|29.38|nap time +73|441|65540|4294967492|39.94|38.58|true|fred hernandez|2013-03-01 09:11:58.703173|4.23|biology +97|388|65675|4294967532|51.60|23.49|false|alice garcia|2013-03-01 09:11:58.703134|31.56|geology +2|475|65607|4294967427|37.06|46.86|false|katie van buren|2013-03-01 09:11:58.703287|58.79|quiet hour +37|400|65588|4294967368|17.88|30.34|false|quinn hernandez|2013-03-01 09:11:58.703091|78.59|geology +-1|408|65646|4294967385|28.05|27.06|false|rachel zipper|2013-03-01 09:11:58.703230|93.22|mathematics +68|352|65611|4294967546|90.53|42.68|false|priscilla robinson|2013-03-01 09:11:58.703273|12.05|xylophone band +114|369|65567|4294967418|74.67|9.73|false|irene king|2013-03-01 09:11:58.703265|65.05|biology +33|268|65566|4294967301|93.34|46.43|true|zach white|2013-03-01 09:11:58.703179|67.09|undecided +44|304|65761|4294967549|81.03|11.69|false|victor ichabod|2013-03-01 09:11:58.703130|31.27|forestry +46|398|65608|4294967396|43.33|34.12|false|rachel falkner|2013-03-01 09:11:58.703081|71.20|topology +45|291|65758|4294967426|97.06|6.13|true|mike allen|2013-03-01 09:11:58.703086|47.41|opthamology +98|466|65601|4294967385|81.90|12.38|true|ethan falkner|2013-03-01 09:11:58.703220|40.35|american history +99|303|65599|4294967390|29.57|46.71|false|alice steinbeck|2013-03-01 09:11:58.703266|88.68|american history +69|474|65564|4294967430|25.00|35.70|true|jessica davidson|2013-03-01 09:11:58.703097|27.80|chemistry +37|477|65639|4294967384|92.84|1.55|false|rachel ellison|2013-03-01 09:11:58.703095|37.85|geology +86|332|65582|4294967426|38.85|39.19|true|luke xylophone|2013-03-01 09:11:58.703113|46.09|yard duty +38|504|65544|4294967339|73.67|3.44|false|victor king|2013-03-01 09:11:58.703124|85.55|joggying +79|270|65624|4294967373|39.54|33.65|true|quinn nixon|2013-03-01 09:11:58.703224|45.02|forestry +-1|421|65684|4294967477|49.82|15.95|false|rachel davidson|2013-03-01 09:11:58.703243|37.07|history +99|457|65699|4294967435|77.24|34.28|true|quinn garcia|2013-03-01 09:11:58.703167|87.47|geology +47|274|65566|4294967519|87.91|0.06|true|xavier robinson|2013-03-01 09:11:58.703200|14.56|religion +66|272|65733|4294967355|1.01|19.27|false|tom thompson|2013-03-01 09:11:58.703206|20.95|debate +21|506|65585|4294967417|54.33|43.63|false|zach johnson|2013-03-01 09:11:58.703178|85.59|yard duty +72|458|65781|4294967372|6.75|6.80|true|yuri johnson|2013-03-01 09:11:58.703128|95.48|wind surfing +85|485|65731|4294967388|3.73|35.18|true|rachel ovid|2013-03-01 09:11:58.703071|37.63|kindergarten +71|306|65636|4294967431|49.44|48.48|false|yuri thompson|2013-03-01 09:11:58.703322|28.10|geology +23|508|65709|4294967452|61.00|25.73|true|xavier robinson|2013-03-01 09:11:58.703096|53.89|religion +38|296|65642|4294967311|51.50|13.65|true|sarah hernandez|2013-03-01 09:11:58.703304|61.11|mathematics +18|443|65701|4294967521|86.98|25.93|false|priscilla polk|2013-03-01 09:11:58.703072|28.09|quiet hour +65|274|65613|4294967546|61.09|33.03|false|yuri garcia|2013-03-01 09:11:58.703178|4.36|mathematics +20|311|65539|4294967448|13.92|32.19|false|priscilla polk|2013-03-01 09:11:58.703143|85.68|topology +14|405|65598|4294967446|8.94|31.06|true|gabriella young|2013-03-01 09:11:58.703147|92.23|american history +-1|500|65544|4294967337|54.09|13.65|true|calvin laertes|2013-03-01 09:11:58.703314|54.72|education +116|423|65761|4294967379|77.77|4.37|false|sarah allen|2013-03-01 09:11:58.703220|23.45|nap time +26|349|65674|4294967358|24.96|22.48|true|irene ellison|2013-03-01 09:11:58.703301|53.16|wind surfing +120|308|65640|4294967411|37.01|49.33|true|ulysses van buren|2013-03-01 09:11:58.703208|85.58|religion +29|475|65656|4294967480|95.29|12.62|true|ulysses hernandez|2013-03-01 09:11:58.703274|29.23|values clariffication +34|362|65772|4294967398|4.81|4.12|false|fred laertes|2013-03-01 09:11:58.703138|97.28|linguistics +5|345|65691|4294967312|40.22|16.96|true|alice xylophone|2013-03-01 09:11:58.703208|8.30|american history +89|338|65621|4294967541|81.55|0.48|false|rachel carson|2013-03-01 09:11:58.703158|29.07|wind surfing +103|415|65664|4294967459|98.23|40.12|true|oscar white|2013-03-01 09:11:58.703249|63.66|quiet hour +120|432|65701|4294967358|52.26|17.18|true|gabriella nixon|2013-03-01 09:11:58.703180|88.20|linguistics +98|357|65632|4294967454|50.34|23.20|true|calvin falkner|2013-03-01 09:11:58.703195|39.16|wind surfing +109|391|65585|4294967339|81.18|41.53|false|fred white|2013-03-01 09:11:58.703192|74.63|yard duty +81|333|65624|4294967398|90.26|28.26|true|david johnson|2013-03-01 09:11:58.703092|9.08|forestry +91|423|65705|4294967523|18.89|26.30|false|priscilla robinson|2013-03-01 09:11:58.703205|33.76|religion +60|296|65562|4294967424|25.65|24.11|true|xavier brown|2013-03-01 09:11:58.703315|65.21|joggying +45|428|65657|4294967463|11.65|28.15|true|gabriella underhill|2013-03-01 09:11:58.703150|46.78|american history +122|468|65674|4294967428|42.82|30.66|false|ulysses garcia|2013-03-01 09:11:58.703228|72.22|religion +56|394|65640|4294967454|43.77|29.21|false|katie xylophone|2013-03-01 09:11:58.703302|11.70|philosophy +-3|507|65728|4294967525|81.95|47.14|true|rachel davidson|2013-03-01 09:11:58.703316|30.59|study skills +82|396|65760|4294967306|56.91|26.25|false|gabriella ellison|2013-03-01 09:11:58.703252|77.29|zync studies +93|371|65779|4294967308|6.37|49.74|false|holly underhill|2013-03-01 09:11:58.703247|31.44|american history +36|511|65688|4294967515|74.79|22.16|true|ulysses steinbeck|2013-03-01 09:11:58.703246|98.00|wind surfing +0|477|65738|4294967512|7.75|46.57|true|gabriella zipper|2013-03-01 09:11:58.703253|79.39|american history +3|309|65746|4294967437|18.88|21.44|true|alice garcia|2013-03-01 09:11:58.703195|80.98|religion +24|278|65713|4294967358|68.99|22.48|false|holly johnson|2013-03-01 09:11:58.703204|98.50|joggying +41|431|65702|4294967550|28.96|15.97|false|oscar laertes|2013-03-01 09:11:58.703201|95.61|forestry +42|437|65656|4294967476|8.98|42.94|false|katie thompson|2013-03-01 09:11:58.703235|21.81|kindergarten +106|308|65563|4294967308|3.09|48.89|false|sarah polk|2013-03-01 09:11:58.703116|13.10|kindergarten +88|381|65756|4294967531|57.22|15.33|false|jessica hernandez|2013-03-01 09:11:58.703303|97.66|yard duty +83|399|65600|4294967494|43.93|13.31|true|wendy robinson|2013-03-01 09:11:58.703077|36.66|joggying +122|483|65787|4294967387|51.81|26.77|false|priscilla hernandez|2013-03-01 09:11:58.703146|25.31|american history +1|333|65646|4294967354|42.31|28.36|false|irene allen|2013-03-01 09:11:58.703314|73.35|values clariffication +45|325|65556|4294967374|22.49|41.49|false|gabriella white|2013-03-01 09:11:58.703116|58.60|forestry +34|366|65638|4294967325|45.49|38.51|false|luke nixon|2013-03-01 09:11:58.703079|85.90|history +31|332|65605|4294967350|9.60|45.65|false|xavier brown|2013-03-01 09:11:58.703262|89.66|zync studies +23|349|65766|4294967311|45.20|2.18|false|irene king|2013-03-01 09:11:58.703282|46.84|study skills +46|278|65608|4294967411|81.29|25.84|true|rachel davidson|2013-03-01 09:11:58.703072|44.58|joggying +31|435|65703|4294967408|80.48|23.52|false|irene laertes|2013-03-01 09:11:58.703230|98.07|debate +67|345|65644|4294967313|44.82|6.70|true|bob ellison|2013-03-01 09:11:58.703155|89.87|nap time +22|376|65723|4294967458|82.24|35.73|false|sarah young|2013-03-01 09:11:58.703221|97.31|topology +95|407|65607|4294967403|2.72|5.95|false|katie allen|2013-03-01 09:11:58.703087|62.42|xylophone band +10|357|65643|4294967521|89.89|16.24|false|tom allen|2013-03-01 09:11:58.703263|6.89|topology +88|405|65756|4294967379|81.76|38.64|true|rachel robinson|2013-03-01 09:11:58.703257|96.07|religion +82|382|65713|4294967522|45.91|3.34|true|tom hernandez|2013-03-01 09:11:58.703072|52.65|american history +5|508|65780|4294967529|61.60|19.48|false|xavier nixon|2013-03-01 09:11:58.703244|25.72|history +29|499|65665|4294967296|21.70|39.99|false|calvin allen|2013-03-01 09:11:58.703092|48.51|philosophy +95|483|65745|4294967338|50.82|20.37|true|xavier king|2013-03-01 09:11:58.703279|4.14|forestry +49|431|65554|4294967542|47.22|44.11|true|fred ichabod|2013-03-01 09:11:58.703136|48.17|debate +27|304|65580|4294967551|0.51|41.25|true|calvin ovid|2013-03-01 09:11:58.703313|67.78|study skills +14|325|65602|4294967504|7.95|45.26|false|victor johnson|2013-03-01 09:11:58.703084|47.11|linguistics +43|383|65737|4294967458|10.47|45.37|true|katie ovid|2013-03-01 09:11:58.703264|36.36|forestry +25|257|65584|4294967355|35.88|35.51|true|gabriella ellison|2013-03-01 09:11:58.703256|17.67|topology +88|259|65733|4294967321|69.22|45.38|false|rachel van buren|2013-03-01 09:11:58.703088|25.76|nap time +34|270|65541|4294967363|55.91|7.49|true|priscilla ichabod|2013-03-01 09:11:58.703189|15.31|american history +92|315|65772|4294967506|77.46|3.72|false|alice ichabod|2013-03-01 09:11:58.703132|95.72|opthamology +89|304|65726|4294967316|55.99|44.46|true|fred johnson|2013-03-01 09:11:58.703196|47.69|philosophy +50|486|65586|4294967411|34.76|30.83|true|katie falkner|2013-03-01 09:11:58.703259|42.60|values clariffication +35|380|65723|4294967432|27.68|37.15|true|oscar falkner|2013-03-01 09:11:58.703087|47.25|zync studies +95|422|65696|4294967424|21.32|26.12|false|tom davidson|2013-03-01 09:11:58.703299|83.21|values clariffication +29|480|65660|4294967311|74.53|28.43|false|fred davidson|2013-03-01 09:11:58.703132|35.08|nap time +11|510|65694|4294967452|80.55|10.65|true|jessica nixon|2013-03-01 09:11:58.703291|58.04|nap time +89|414|65766|4294967476|8.00|30.63|true|tom garcia|2013-03-01 09:11:58.703315|6.99|linguistics +86|282|65722|4294967541|3.99|46.79|true|david laertes|2013-03-01 09:11:58.703114|11.82|study skills +102|316|65549|4294967324|44.66|21.59|true|luke johnson|2013-03-01 09:11:58.703163|13.15|topology +74|437|65697|4294967502|37.93|28.24|false|mike white|2013-03-01 09:11:58.703099|85.41|forestry +25|311|65607|4294967485|84.89|3.40|false|xavier hernandez|2013-03-01 09:11:58.703260|39.97|quiet hour +15|480|65545|4294967521|70.33|4.46|true|oscar xylophone|2013-03-01 09:11:58.703173|76.37|american history +109|494|65775|4294967493|90.97|3.35|false|fred allen|2013-03-01 09:11:58.703200|36.66|quiet hour +46|270|65764|4294967339|18.45|26.14|false|nick carson|2013-03-01 09:11:58.703284|95.46|linguistics +67|462|65780|4294967496|92.53|9.03|true|luke underhill|2013-03-01 09:11:58.703268|60.32|joggying +34|473|65587|4294967417|20.53|13.40|false|priscilla polk|2013-03-01 09:11:58.703242|63.11|joggying +13|345|65762|4294967469|28.84|9.35|true|xavier johnson|2013-03-01 09:11:58.703296|40.82|wind surfing +112|337|65601|4294967329|40.31|5.32|true|yuri hernandez|2013-03-01 09:11:58.703189|53.98|american history +2|304|65718|4294967346|91.32|10.19|true|oscar carson|2013-03-01 09:11:58.703310|91.69|study skills +19|354|65558|4294967373|20.14|48.76|true|luke nixon|2013-03-01 09:11:58.703257|8.06|nap time +103|362|65612|4294967504|54.91|7.28|false|quinn white|2013-03-01 09:11:58.703318|14.68|linguistics +66|485|65602|4294967433|67.23|25.83|true|nick ovid|2013-03-01 09:11:58.703199|84.42|values clariffication +109|433|65755|4294967543|84.38|30.58|true|luke ovid|2013-03-01 09:11:58.703266|64.35|american history +100|361|65572|4294967392|28.67|3.12|false|jessica van buren|2013-03-01 09:11:58.703120|38.15|linguistics +29|388|65615|4294967351|70.90|22.57|true|quinn steinbeck|2013-03-01 09:11:58.703211|53.91|yard duty +118|362|65548|4294967368|3.20|11.52|false|mike davidson|2013-03-01 09:11:58.703219|52.20|zync studies +27|487|65759|4294967349|95.36|28.75|true|ulysses xylophone|2013-03-01 09:11:58.703305|96.35|joggying +40|480|65659|4294967440|52.55|33.48|false|luke garcia|2013-03-01 09:11:58.703235|54.04|education +29|314|65709|4294967334|27.76|11.38|false|priscilla king|2013-03-01 09:11:58.703117|83.42|forestry +40|356|65560|4294967418|11.42|34.83|true|oscar zipper|2013-03-01 09:11:58.703142|92.08|linguistics +8|430|65682|4294967310|57.04|47.68|true|alice king|2013-03-01 09:11:58.703131|11.38|forestry +67|264|65693|4294967393|53.41|30.67|false|bob falkner|2013-03-01 09:11:58.703167|38.65|kindergarten +8|487|65714|4294967543|2.35|40.78|false|nick young|2013-03-01 09:11:58.703191|64.98|history +107|273|65607|4294967419|50.79|19.35|true|katie xylophone|2013-03-01 09:11:58.703238|25.13|kindergarten +17|424|65708|4294967325|85.34|6.27|true|victor davidson|2013-03-01 09:11:58.703302|71.12|topology +42|320|65556|4294967452|67.38|5.37|true|irene steinbeck|2013-03-01 09:11:58.703076|0.45|debate +-3|362|65712|4294967325|43.73|48.74|false|oscar garcia|2013-03-01 09:11:58.703282|29.97|chemistry +20|302|65575|4294967371|48.37|11.02|false|calvin nixon|2013-03-01 09:11:58.703245|36.75|debate +61|393|65764|4294967353|38.43|11.48|true|holly nixon|2013-03-01 09:11:58.703205|24.87|xylophone band +13|333|65644|4294967413|16.44|47.68|false|priscilla white|2013-03-01 09:11:58.703120|85.34|study skills +20|377|65753|4294967314|51.18|18.68|true|yuri johnson|2013-03-01 09:11:58.703217|2.76|nap time +108|464|65684|4294967340|51.02|30.95|false|rachel brown|2013-03-01 09:11:58.703084|12.27|quiet hour +74|493|65544|4294967369|85.39|32.26|true|oscar miller|2013-03-01 09:11:58.703077|4.03|kindergarten +9|340|65596|4294967319|17.59|29.55|false|gabriella hernandez|2013-03-01 09:11:58.703284|16.29|study skills +-2|352|65648|4294967329|71.03|16.29|true|tom xylophone|2013-03-01 09:11:58.703203|4.60|mathematics +38|410|65770|4294967403|24.77|35.34|true|ulysses thompson|2013-03-01 09:11:58.703082|51.54|industrial engineering +106|381|65700|4294967381|34.28|39.12|true|nick johnson|2013-03-01 09:11:58.703268|14.35|forestry +120|494|65594|4294967499|20.69|31.86|true|ulysses ellison|2013-03-01 09:11:58.703093|44.58|religion +110|292|65707|4294967373|24.43|48.34|true|oscar ichabod|2013-03-01 09:11:58.703181|53.53|linguistics +10|480|65662|4294967484|95.13|20.85|true|oscar allen|2013-03-01 09:11:58.703168|9.06|philosophy +4|412|65590|4294967328|93.90|25.40|false|jessica ichabod|2013-03-01 09:11:58.703143|3.58|joggying +79|278|65669|4294967327|30.87|40.52|true|fred young|2013-03-01 09:11:58.703256|21.67|yard duty +124|333|65724|4294967506|3.17|16.08|true|gabriella xylophone|2013-03-01 09:11:58.703070|0.50|mathematics +12|407|65577|4294967449|46.83|25.68|false|mike carson|2013-03-01 09:11:58.703150|31.13|undecided +8|503|65541|4294967546|69.18|39.01|true|xavier ichabod|2013-03-01 09:11:58.703178|7.23|linguistics +84|472|65715|4294967481|96.31|18.51|true|gabriella brown|2013-03-01 09:11:58.703192|25.71|opthamology +82|480|65746|4294967534|95.69|48.20|false|victor davidson|2013-03-01 09:11:58.703119|66.78|study skills +1|274|65778|4294967424|26.14|39.77|false|alice ovid|2013-03-01 09:11:58.703157|86.49|opthamology +123|408|65548|4294967438|14.56|15.95|false|luke underhill|2013-03-01 09:11:58.703320|0.31|education +102|506|65593|4294967490|67.38|22.97|false|priscilla brown|2013-03-01 09:11:58.703283|1.66|kindergarten +38|332|65560|4294967298|30.31|10.71|true|quinn laertes|2013-03-01 09:11:58.703112|73.08|religion +75|499|65603|4294967322|14.44|49.43|false|yuri thompson|2013-03-01 09:11:58.703151|11.87|american history +6|359|65689|4294967508|95.09|19.09|true|xavier garcia|2013-03-01 09:11:58.703316|7.59|forestry +89|273|65607|4294967316|57.84|49.47|true|tom robinson|2013-03-01 09:11:58.703110|99.00|education +22|322|65776|4294967347|47.82|3.81|false|luke falkner|2013-03-01 09:11:58.703089|75.51|zync studies +17|472|65583|4294967438|66.08|4.64|true|rachel quirinius|2013-03-01 09:11:58.703285|19.07|forestry +104|305|65679|4294967432|29.91|48.67|true|gabriella zipper|2013-03-01 09:11:58.703097|50.31|yard duty +100|442|65588|4294967449|64.01|47.52|false|rachel king|2013-03-01 09:11:58.703117|15.96|xylophone band +23|458|65538|4294967313|40.67|18.13|false|quinn garcia|2013-03-01 09:11:58.703185|48.61|kindergarten +8|399|65703|4294967437|29.11|9.64|false|tom zipper|2013-03-01 09:11:58.703105|34.94|zync studies +87|423|65644|4294967471|17.51|3.09|false|ethan garcia|2013-03-01 09:11:58.703157|31.92|study skills +98|477|65791|4294967422|22.87|14.38|true|nick carson|2013-03-01 09:11:58.703111|84.41|linguistics +87|269|65748|4294967543|20.61|35.33|false|bob ovid|2013-03-01 09:11:58.703070|80.82|mathematics +36|510|65715|4294967531|92.13|49.70|true|fred zipper|2013-03-01 09:11:58.703310|70.81|quiet hour +2|489|65658|4294967314|67.62|31.20|true|oscar hernandez|2013-03-01 09:11:58.703130|90.69|xylophone band +34|315|65689|4294967336|14.78|36.35|true|ethan van buren|2013-03-01 09:11:58.703156|54.20|opthamology +0|410|65789|4294967338|2.12|20.59|true|zach allen|2013-03-01 09:11:58.703078|72.37|study skills +1|499|65543|4294967492|25.68|29.87|true|rachel hernandez|2013-03-01 09:11:58.703316|42.81|geology +96|389|65749|4294967499|89.20|29.86|true|calvin brown|2013-03-01 09:11:58.703244|71.83|forestry +78|432|65770|4294967332|41.88|9.42|false|fred davidson|2013-03-01 09:11:58.703192|61.03|topology +-2|307|65592|4294967529|13.62|16.78|false|katie carson|2013-03-01 09:11:58.703271|51.09|chemistry +12|393|65679|4294967421|11.51|13.72|true|tom hernandez|2013-03-01 09:11:58.703078|89.58|undecided +88|277|65576|4294967353|9.16|2.96|false|nick miller|2013-03-01 09:11:58.703158|96.98|education +88|489|65747|4294967390|37.74|33.69|false|ulysses xylophone|2013-03-01 09:11:58.703225|86.95|linguistics +-2|329|65755|4294967527|73.67|12.95|false|nick white|2013-03-01 09:11:58.703179|46.23|forestry +3|477|65725|4294967347|77.98|12.57|true|fred ovid|2013-03-01 09:11:58.703324|58.90|religion +35|469|65622|4294967538|21.47|10.61|true|priscilla ellison|2013-03-01 09:11:58.703231|33.66|linguistics +69|361|65687|4294967406|89.50|38.30|false|nick miller|2013-03-01 09:11:58.703124|47.16|education +106|470|65663|4294967541|94.07|22.03|false|zach brown|2013-03-01 09:11:58.703191|85.52|debate +27|319|65727|4294967377|85.24|38.45|false|sarah ovid|2013-03-01 09:11:58.703288|59.59|chemistry +51|469|65631|4294967456|91.18|49.62|true|holly van buren|2013-03-01 09:11:58.703108|92.52|joggying +122|302|65647|4294967500|78.03|6.22|true|tom quirinius|2013-03-01 09:11:58.703201|62.47|xylophone band +-1|294|65748|4294967502|97.23|15.14|false|nick ovid|2013-03-01 09:11:58.703299|5.76|quiet hour +18|290|65767|4294967409|28.16|46.21|false|yuri underhill|2013-03-01 09:11:58.703301|0.38|opthamology +78|431|65592|4294967499|63.07|19.24|true|ethan polk|2013-03-01 09:11:58.703073|5.10|linguistics +37|338|65584|4294967327|67.09|30.80|true|tom brown|2013-03-01 09:11:58.703236|1.13|forestry +112|447|65789|4294967360|59.75|22.61|true|alice young|2013-03-01 09:11:58.703124|92.16|education +79|345|65769|4294967479|50.73|3.06|true|calvin miller|2013-03-01 09:11:58.703257|62.01|zync studies +75|483|65668|4294967298|49.43|37.98|false|ulysses king|2013-03-01 09:11:58.703168|59.79|yard duty +37|357|65612|4294967315|97.43|48.32|true|holly underhill|2013-03-01 09:11:58.703281|99.12|kindergarten +94|392|65758|4294967347|19.32|23.99|true|alice underhill|2013-03-01 09:11:58.703125|86.64|geology +9|396|65717|4294967370|90.00|35.93|true|tom steinbeck|2013-03-01 09:11:58.703149|74.22|values clariffication +2|310|65593|4294967536|96.66|40.64|true|quinn quirinius|2013-03-01 09:11:58.703106|98.76|forestry +85|368|65692|4294967395|66.57|3.52|true|calvin garcia|2013-03-01 09:11:58.703256|92.89|yard duty +3|461|65602|4294967368|7.14|12.21|false|alice zipper|2013-03-01 09:11:58.703141|30.38|biology +85|475|65563|4294967463|93.17|19.84|true|tom young|2013-03-01 09:11:58.703241|93.97|xylophone band +6|449|65720|4294967353|55.25|46.42|true|xavier davidson|2013-03-01 09:11:58.703215|3.65|linguistics +76|475|65696|4294967545|0.14|17.22|false|gabriella robinson|2013-03-01 09:11:58.703283|23.51|xylophone band +120|458|65737|4294967550|92.91|29.21|true|priscilla brown|2013-03-01 09:11:58.703144|30.70|study skills +109|279|65650|4294967518|75.97|32.91|false|victor ichabod|2013-03-01 09:11:58.703270|6.47|nap time +-1|390|65769|4294967365|2.80|2.02|false|ethan zipper|2013-03-01 09:11:58.703172|22.62|undecided +65|407|65604|4294967382|45.70|38.15|true|nick robinson|2013-03-01 09:11:58.703174|96.20|joggying +54|478|65771|4294967470|54.46|33.00|false|tom garcia|2013-03-01 09:11:58.703243|67.81|quiet hour +35|283|65787|4294967296|15.80|15.95|true|zach nixon|2013-03-01 09:11:58.703242|95.56|opthamology +7|444|65613|4294967364|94.63|15.86|true|wendy xylophone|2013-03-01 09:11:58.703257|16.63|yard duty +106|347|65612|4294967475|44.87|23.03|true|rachel carson|2013-03-01 09:11:58.703130|4.30|religion +12|386|65699|4294967496|9.29|18.74|false|ethan nixon|2013-03-01 09:11:58.703099|81.26|biology +63|280|65722|4294967380|98.09|37.36|true|ulysses young|2013-03-01 09:11:58.703299|54.66|quiet hour +7|301|65578|4294967314|41.62|12.20|false|nick young|2013-03-01 09:11:58.703199|27.57|debate +55|455|65764|4294967410|64.57|2.03|false|nick xylophone|2013-03-01 09:11:58.703194|51.56|zync studies +90|435|65652|4294967316|69.88|28.64|true|mike miller|2013-03-01 09:11:58.703282|1.72|undecided +102|490|65771|4294967310|58.93|26.35|true|zach carson|2013-03-01 09:11:58.703296|35.19|american history +81|325|65585|4294967505|39.90|46.21|false|ethan underhill|2013-03-01 09:11:58.703234|96.07|religion +4|445|65552|4294967524|93.35|48.66|true|gabriella davidson|2013-03-01 09:11:58.703120|31.79|chemistry +16|363|65761|4294967529|10.13|48.24|true|alice xylophone|2013-03-01 09:11:58.703223|18.03|history +-3|459|65644|4294967456|92.71|0.08|false|jessica king|2013-03-01 09:11:58.703279|53.48|joggying +40|455|65621|4294967384|1.97|33.97|false|victor garcia|2013-03-01 09:11:58.703080|81.71|mathematics +112|337|65777|4294967415|33.53|36.17|false|quinn king|2013-03-01 09:11:58.703141|49.88|values clariffication +0|329|65684|4294967461|54.83|49.06|false|xavier hernandez|2013-03-01 09:11:58.703296|94.88|forestry +10|491|65698|4294967531|17.85|22.83|true|tom johnson|2013-03-01 09:11:58.703070|3.07|history +47|300|65709|4294967495|63.91|13.31|false|oscar ellison|2013-03-01 09:11:58.703318|63.27|linguistics +36|261|65547|4294967497|45.12|24.62|true|alice zipper|2013-03-01 09:11:58.703086|93.10|mathematics +71|405|65734|4294967507|29.46|49.93|false|bob xylophone|2013-03-01 09:11:58.703249|81.40|opthamology +27|467|65740|4294967512|19.06|47.72|false|irene quirinius|2013-03-01 09:11:58.703307|43.61|education +58|306|65669|4294967534|68.20|14.43|false|ethan nixon|2013-03-01 09:11:58.703236|21.84|nap time +-3|333|65562|4294967359|22.34|35.58|false|ulysses steinbeck|2013-03-01 09:11:58.703259|86.78|xylophone band +20|305|65692|4294967484|32.28|32.11|true|jessica ellison|2013-03-01 09:11:58.703096|75.79|industrial engineering +23|402|65723|4294967339|9.57|4.18|true|sarah polk|2013-03-01 09:11:58.703190|62.93|chemistry +49|295|65559|4294967373|46.88|31.41|false|oscar steinbeck|2013-03-01 09:11:58.703312|34.58|zync studies +40|389|65554|4294967315|8.59|33.70|true|sarah ichabod|2013-03-01 09:11:58.703136|93.18|zync studies +87|316|65691|4294967395|15.87|5.64|false|calvin robinson|2013-03-01 09:11:58.703186|36.24|kindergarten +50|280|65631|4294967446|81.58|6.64|false|victor robinson|2013-03-01 09:11:58.703314|33.82|philosophy +73|470|65665|4294967514|54.52|48.31|false|quinn quirinius|2013-03-01 09:11:58.703206|31.68|xylophone band +92|462|65624|4294967296|80.73|47.04|true|bob steinbeck|2013-03-01 09:11:58.703197|37.71|quiet hour +92|344|65627|4294967336|90.25|36.79|false|priscilla garcia|2013-03-01 09:11:58.703287|53.48|history +37|329|65720|4294967383|38.35|22.55|true|bob falkner|2013-03-01 09:11:58.703319|11.91|opthamology +118|322|65763|4294967514|25.77|36.15|true|alice ovid|2013-03-01 09:11:58.703322|94.46|chemistry +11|396|65629|4294967544|58.54|20.51|true|priscilla king|2013-03-01 09:11:58.703277|71.39|nap time +28|491|65568|4294967425|84.69|9.15|false|tom ichabod|2013-03-01 09:11:58.703149|25.08|linguistics +43|406|65783|4294967537|26.21|0.43|true|holly robinson|2013-03-01 09:11:58.703123|46.19|joggying +45|393|65750|4294967408|56.91|49.36|false|priscilla steinbeck|2013-03-01 09:11:58.703218|97.18|wind surfing +110|359|65559|4294967434|20.23|9.19|false|sarah falkner|2013-03-01 09:11:58.703135|3.60|xylophone band +14|465|65541|4294967445|30.26|3.34|true|victor ellison|2013-03-01 09:11:58.703245|96.71|topology +35|322|65788|4294967489|10.06|20.83|false|jessica underhill|2013-03-01 09:11:58.703249|98.67|nap time +5|409|65686|4294967486|36.23|13.04|true|katie ichabod|2013-03-01 09:11:58.703273|20.44|wind surfing +1|284|65548|4294967539|29.85|30.95|true|tom young|2013-03-01 09:11:58.703085|63.92|wind surfing +121|328|65742|4294967466|50.32|42.73|true|wendy van buren|2013-03-01 09:11:58.703156|60.80|education +23|398|65651|4294967431|60.38|46.02|false|gabriella laertes|2013-03-01 09:11:58.703134|94.58|chemistry +22|399|65676|4294967466|72.63|4.67|false|fred underhill|2013-03-01 09:11:58.703274|88.74|wind surfing +30|372|65776|4294967531|7.41|10.10|true|mike johnson|2013-03-01 09:11:58.703115|77.38|joggying +81|275|65701|4294967450|70.95|15.12|true|katie garcia|2013-03-01 09:11:58.703084|27.55|undecided +61|370|65591|4294967329|45.44|6.26|false|jessica polk|2013-03-01 09:11:58.703300|49.73|opthamology +113|500|65570|4294967472|97.19|46.45|true|yuri ichabod|2013-03-01 09:11:58.703179|10.11|history +13|299|65786|4294967477|97.65|6.54|false|xavier carson|2013-03-01 09:11:58.703073|61.77|values clariffication +107|289|65614|4294967519|88.57|43.02|false|yuri ichabod|2013-03-01 09:11:58.703270|8.76|biology +8|424|65728|4294967313|46.88|23.47|false|david garcia|2013-03-01 09:11:58.703248|46.33|chemistry +59|300|65537|4294967371|37.31|37.09|true|zach ovid|2013-03-01 09:11:58.703174|20.39|industrial engineering +100|430|65661|4294967303|18.31|37.13|true|yuri white|2013-03-01 09:11:58.703224|81.15|undecided +76|303|65676|4294967515|56.72|37.17|false|rachel ovid|2013-03-01 09:11:58.703090|79.57|wind surfing +12|302|65739|4294967433|42.06|35.15|true|yuri steinbeck|2013-03-01 09:11:58.703126|53.98|philosophy +17|443|65576|4294967519|87.92|14.12|true|nick miller|2013-03-01 09:11:58.703285|60.00|philosophy +103|489|65642|4294967492|93.67|8.63|false|wendy miller|2013-03-01 09:11:58.703292|14.57|education +21|308|65738|4294967400|79.15|31.11|false|victor white|2013-03-01 09:11:58.703186|8.39|values clariffication +21|376|65661|4294967315|21.59|44.37|false|yuri xylophone|2013-03-01 09:11:58.703197|17.09|history +3|341|65653|4294967352|29.58|0.57|false|alice hernandez|2013-03-01 09:11:58.703122|35.90|undecided +116|396|65726|4294967461|4.36|44.18|true|nick quirinius|2013-03-01 09:11:58.703258|23.95|history +33|392|65611|4294967515|21.47|6.52|false|sarah falkner|2013-03-01 09:11:58.703306|70.32|study skills +106|509|65698|4294967331|35.26|33.69|true|victor steinbeck|2013-03-01 09:11:58.703158|55.39|linguistics +4|445|65694|4294967548|67.78|2.14|true|katie king|2013-03-01 09:11:58.703175|46.45|mathematics +102|403|65546|4294967374|9.09|47.46|false|david steinbeck|2013-03-01 09:11:58.703121|30.01|forestry +14|297|65644|4294967301|78.17|31.88|true|priscilla ichabod|2013-03-01 09:11:58.703190|84.18|geology +98|491|65762|4294967312|84.97|43.83|false|priscilla brown|2013-03-01 09:11:58.703137|48.90|nap time +90|504|65703|4294967456|93.36|33.56|false|victor brown|2013-03-01 09:11:58.703228|71.91|wind surfing +82|483|65684|4294967399|19.50|11.43|false|holly ellison|2013-03-01 09:11:58.703249|30.42|history +2|373|65712|4294967524|57.45|49.38|false|zach underhill|2013-03-01 09:11:58.703142|38.41|history +16|264|65785|4294967370|7.75|29.95|true|calvin underhill|2013-03-01 09:11:58.703168|3.93|values clariffication +63|345|65621|4294967504|88.45|15.48|false|xavier johnson|2013-03-01 09:11:58.703304|58.39|biology +45|334|65733|4294967472|33.55|41.17|false|mike ichabod|2013-03-01 09:11:58.703144|19.00|undecided +48|330|65718|4294967546|70.75|7.62|true|quinn xylophone|2013-03-01 09:11:58.703289|96.20|topology +82|295|65621|4294967514|17.44|21.23|true|priscilla young|2013-03-01 09:11:58.703103|21.49|quiet hour +107|351|65578|4294967380|27.63|45.41|false|zach ichabod|2013-03-01 09:11:58.703325|47.89|opthamology +14|346|65748|4294967353|58.72|22.31|false|jessica young|2013-03-01 09:11:58.703207|90.01|debate +35|361|65607|4294967446|39.14|49.94|true|ulysses white|2013-03-01 09:11:58.703188|94.90|religion +87|439|65700|4294967401|98.22|21.96|true|david steinbeck|2013-03-01 09:11:58.703219|14.73|chemistry +117|377|65773|4294967500|48.84|9.05|false|rachel nixon|2013-03-01 09:11:58.703198|32.15|zync studies +97|396|65779|4294967334|80.59|29.68|false|oscar garcia|2013-03-01 09:11:58.703105|36.54|wind surfing +28|373|65556|4294967416|53.74|30.92|false|victor robinson|2013-03-01 09:11:58.703192|80.05|topology +52|324|65592|4294967406|68.55|4.43|false|xavier laertes|2013-03-01 09:11:58.703244|46.74|study skills +25|291|65745|4294967536|93.83|10.75|false|sarah miller|2013-03-01 09:11:58.703228|33.21|linguistics +101|289|65640|4294967468|57.83|32.94|true|katie van buren|2013-03-01 09:11:58.703223|76.95|yard duty +72|270|65762|4294967446|52.21|46.01|true|calvin quirinius|2013-03-01 09:11:58.703214|39.27|yard duty +66|506|65605|4294967348|58.62|7.00|true|ethan zipper|2013-03-01 09:11:58.703275|85.37|values clariffication +0|489|65703|4294967320|42.48|14.15|true|wendy steinbeck|2013-03-01 09:11:58.703099|38.14|forestry +39|370|65618|4294967346|49.38|18.38|false|mike ovid|2013-03-01 09:11:58.703227|68.32|xylophone band +75|449|65586|4294967540|98.45|21.74|false|ethan allen|2013-03-01 09:11:58.703279|26.21|wind surfing +45|383|65641|4294967521|69.97|12.76|false|mike ellison|2013-03-01 09:11:58.703159|96.36|philosophy +87|328|65737|4294967354|90.87|46.92|false|ulysses zipper|2013-03-01 09:11:58.703307|41.02|history +7|366|65548|4294967492|4.67|8.97|true|jessica van buren|2013-03-01 09:11:58.703285|90.95|religion +62|475|65701|4294967514|77.61|26.42|true|tom miller|2013-03-01 09:11:58.703129|89.77|education +44|440|65782|4294967467|10.35|46.92|true|wendy underhill|2013-03-01 09:11:58.703143|21.45|undecided +32|390|65692|4294967466|47.54|35.59|true|jessica nixon|2013-03-01 09:11:58.703303|69.53|wind surfing +42|472|65635|4294967350|72.81|26.78|false|katie steinbeck|2013-03-01 09:11:58.703144|12.92|geology +79|352|65698|4294967502|43.53|23.86|true|ethan underhill|2013-03-01 09:11:58.703105|48.91|opthamology +13|328|65784|4294967520|22.38|43.56|false|yuri falkner|2013-03-01 09:11:58.703303|12.79|nap time +56|482|65747|4294967366|47.63|49.39|true|luke falkner|2013-03-01 09:11:58.703194|99.19|quiet hour +72|334|65566|4294967482|11.55|4.75|true|priscilla ellison|2013-03-01 09:11:58.703209|68.54|undecided +96|417|65557|4294967325|80.68|5.10|false|bob ellison|2013-03-01 09:11:58.703209|78.22|industrial engineering +65|484|65770|4294967413|49.29|44.68|true|rachel johnson|2013-03-01 09:11:58.703121|20.30|biology +75|455|65760|4294967514|9.45|44.17|true|holly ichabod|2013-03-01 09:11:58.703105|82.85|opthamology +93|317|65675|4294967478|1.87|17.03|false|david ichabod|2013-03-01 09:11:58.703178|9.88|yard duty +26|317|65590|4294967527|67.83|45.47|true|yuri king|2013-03-01 09:11:58.703182|94.45|debate +98|450|65682|4294967359|6.08|7.38|true|victor xylophone|2013-03-01 09:11:58.703196|53.28|industrial engineering +106|478|65577|4294967519|83.13|10.16|true|ethan garcia|2013-03-01 09:11:58.703110|48.13|opthamology +115|309|65538|4294967334|99.10|9.89|true|calvin allen|2013-03-01 09:11:58.703311|15.17|values clariffication +11|401|65714|4294967532|55.36|47.07|true|calvin johnson|2013-03-01 09:11:58.703194|59.33|geology +20|452|65666|4294967302|49.57|6.97|true|alice brown|2013-03-01 09:11:58.703192|62.93|industrial engineering +66|289|65538|4294967541|66.01|39.78|false|xavier davidson|2013-03-01 09:11:58.703320|40.63|forestry +81|343|65691|4294967508|76.37|27.30|true|victor carson|2013-03-01 09:11:58.703218|63.28|debate +6|461|65713|4294967442|93.48|39.11|false|yuri underhill|2013-03-01 09:11:58.703272|38.88|religion +1|295|65571|4294967321|65.51|36.83|false|zach falkner|2013-03-01 09:11:58.703224|17.17|values clariffication +96|350|65660|4294967462|84.21|44.22|false|ulysses johnson|2013-03-01 09:11:58.703071|15.60|history +5|469|65688|4294967336|41.36|38.25|false|bob carson|2013-03-01 09:11:58.703133|7.97|religion +38|321|65677|4294967518|73.47|21.01|false|alice davidson|2013-03-01 09:11:58.703198|84.37|nap time +12|468|65626|4294967319|16.30|17.98|true|victor polk|2013-03-01 09:11:58.703185|64.20|geology +64|443|65655|4294967383|65.20|9.95|true|alice steinbeck|2013-03-01 09:11:58.703294|5.84|debate +76|415|65565|4294967415|32.02|35.03|true|katie miller|2013-03-01 09:11:58.703187|11.21|values clariffication +85|413|65584|4294967396|68.90|45.79|true|nick falkner|2013-03-01 09:11:58.703204|25.28|philosophy +66|407|65575|4294967300|17.57|44.01|false|priscilla young|2013-03-01 09:11:58.703251|6.42|debate +105|304|65702|4294967423|50.83|48.97|true|luke white|2013-03-01 09:11:58.703177|18.47|religion +20|461|65617|4294967538|22.31|34.26|true|priscilla steinbeck|2013-03-01 09:11:58.703223|19.26|xylophone band +13|289|65646|4294967391|89.11|3.68|true|holly underhill|2013-03-01 09:11:58.703237|89.41|mathematics +99|481|65735|4294967539|37.01|49.34|false|ulysses quirinius|2013-03-01 09:11:58.703257|31.10|industrial engineering +110|264|65633|4294967423|25.46|8.90|false|fred laertes|2013-03-01 09:11:58.703288|38.44|nap time +98|362|65791|4294967465|93.75|25.14|false|jessica miller|2013-03-01 09:11:58.703155|90.79|values clariffication +112|277|65756|4294967404|81.99|28.42|false|zach carson|2013-03-01 09:11:58.703262|90.39|debate +57|467|65594|4294967505|33.78|1.63|false|katie steinbeck|2013-03-01 09:11:58.703288|69.27|undecided +122|480|65653|4294967478|58.91|21.69|false|wendy hernandez|2013-03-01 09:11:58.703316|41.41|topology +13|399|65611|4294967376|53.59|21.64|false|zach van buren|2013-03-01 09:11:58.703180|10.72|philosophy +27|493|65777|4294967468|48.09|11.16|true|xavier nixon|2013-03-01 09:11:58.703303|47.91|debate +-3|448|65610|4294967314|81.97|31.11|true|mike xylophone|2013-03-01 09:11:58.703308|79.37|opthamology +106|432|65610|4294967466|88.61|32.15|false|xavier falkner|2013-03-01 09:11:58.703135|43.78|joggying +90|272|65555|4294967457|5.50|38.03|false|gabriella garcia|2013-03-01 09:11:58.703248|49.00|joggying +45|508|65638|4294967362|53.36|19.51|true|nick hernandez|2013-03-01 09:11:58.703254|30.71|kindergarten +26|346|65618|4294967393|46.24|22.74|true|katie young|2013-03-01 09:11:58.703275|77.16|yard duty +97|477|65570|4294967400|90.81|9.98|false|luke underhill|2013-03-01 09:11:58.703178|42.58|linguistics +46|283|65551|4294967451|3.69|19.52|false|tom hernandez|2013-03-01 09:11:58.703251|48.25|geology +16|412|65592|4294967396|52.43|31.68|false|mike johnson|2013-03-01 09:11:58.703320|37.82|biology +92|310|65660|4294967427|3.33|8.98|true|priscilla polk|2013-03-01 09:11:58.703227|58.23|topology +29|342|65589|4294967403|15.32|11.57|true|katie nixon|2013-03-01 09:11:58.703199|23.01|nap time +28|402|65756|4294967548|46.19|22.49|true|katie polk|2013-03-01 09:11:58.703205|49.15|biology +72|298|65724|4294967427|59.66|8.06|false|priscilla robinson|2013-03-01 09:11:58.703214|58.05|history +34|370|65587|4294967309|27.44|14.06|true|xavier king|2013-03-01 09:11:58.703126|84.83|mathematics +43|320|65772|4294967470|83.33|20.21|false|luke steinbeck|2013-03-01 09:11:58.703233|36.23|religion +83|509|65629|4294967455|88.35|0.91|true|luke ichabod|2013-03-01 09:11:58.703232|87.65|joggying +22|319|65685|4294967359|30.00|35.38|false|mike hernandez|2013-03-01 09:11:58.703140|73.13|opthamology +93|389|65773|4294967468|83.13|12.55|true|jessica carson|2013-03-01 09:11:58.703090|33.29|mathematics +28|440|65691|4294967492|73.18|5.99|true|oscar carson|2013-03-01 09:11:58.703151|75.39|quiet hour +92|341|65599|4294967474|82.13|18.55|true|jessica king|2013-03-01 09:11:58.703258|23.06|zync studies +101|509|65684|4294967330|36.90|40.77|false|mike laertes|2013-03-01 09:11:58.703287|94.60|chemistry +43|416|65695|4294967316|93.92|28.02|false|ulysses quirinius|2013-03-01 09:11:58.703171|23.63|american history +117|419|65620|4294967303|43.30|9.03|true|nick miller|2013-03-01 09:11:58.703193|2.26|biology +120|280|65674|4294967533|72.38|18.59|false|fred ellison|2013-03-01 09:11:58.703123|80.81|xylophone band +38|356|65729|4294967396|59.16|35.72|true|yuri hernandez|2013-03-01 09:11:58.703192|49.84|joggying +27|291|65654|4294967509|80.04|9.80|false|fred hernandez|2013-03-01 09:11:58.703170|77.80|wind surfing +92|418|65744|4294967366|67.33|12.32|false|irene ellison|2013-03-01 09:11:58.703298|2.61|yard duty +84|344|65655|4294967341|5.68|45.14|true|victor hernandez|2013-03-01 09:11:58.703139|19.01|philosophy +68|511|65577|4294967331|14.02|12.47|false|tom carson|2013-03-01 09:11:58.703198|57.34|debate +59|256|65706|4294967467|83.67|38.37|false|quinn hernandez|2013-03-01 09:11:58.703321|78.05|values clariffication +109|405|65665|4294967428|80.74|32.82|false|xavier ovid|2013-03-01 09:11:58.703227|72.73|geology +22|439|65536|4294967336|32.75|23.35|false|calvin thompson|2013-03-01 09:11:58.703131|52.78|nap time +12|358|65555|4294967523|98.55|27.67|true|sarah quirinius|2013-03-01 09:11:58.703114|56.45|topology +28|284|65665|4294967308|25.01|22.62|false|irene falkner|2013-03-01 09:11:58.703127|21.92|undecided +115|326|65603|4294967465|63.69|22.12|true|yuri brown|2013-03-01 09:11:58.703215|65.03|religion +65|386|65772|4294967325|10.29|17.65|false|alice ovid|2013-03-01 09:11:58.703167|95.28|undecided +4|333|65741|4294967347|26.37|20.86|false|irene ovid|2013-03-01 09:11:58.703214|82.00|linguistics +74|265|65750|4294967308|57.51|14.40|false|oscar zipper|2013-03-01 09:11:58.703147|95.37|yard duty +67|300|65609|4294967350|9.68|30.20|false|zach allen|2013-03-01 09:11:58.703182|45.38|chemistry +48|359|65600|4294967479|33.01|49.64|true|fred white|2013-03-01 09:11:58.703133|99.46|study skills +69|321|65596|4294967503|71.92|27.70|true|david falkner|2013-03-01 09:11:58.703098|98.80|mathematics +27|411|65736|4294967504|15.75|28.53|true|nick robinson|2013-03-01 09:11:58.703318|10.36|debate +35|350|65585|4294967401|54.71|48.74|false|nick falkner|2013-03-01 09:11:58.703187|39.89|wind surfing +104|364|65623|4294967415|51.84|20.40|false|katie steinbeck|2013-03-01 09:11:58.703206|9.50|joggying +18|303|65720|4294967458|59.33|10.43|false|luke brown|2013-03-01 09:11:58.703149|36.43|forestry +1|320|65622|4294967335|74.14|18.89|true|alice ichabod|2013-03-01 09:11:58.703236|78.65|study skills +69|291|65635|4294967385|66.40|12.49|false|victor underhill|2013-03-01 09:11:58.703271|74.00|forestry +63|287|65550|4294967410|59.07|43.58|true|jessica carson|2013-03-01 09:11:58.703268|55.96|nap time +0|346|65552|4294967412|58.02|40.47|false|calvin brown|2013-03-01 09:11:58.703212|89.46|undecided +39|283|65546|4294967318|59.98|23.54|true|calvin zipper|2013-03-01 09:11:58.703236|18.08|mathematics +123|435|65581|4294967530|17.80|49.96|false|quinn allen|2013-03-01 09:11:58.703212|31.43|opthamology +94|316|65731|4294967335|46.34|8.99|true|irene ovid|2013-03-01 09:11:58.703198|53.60|topology +40|366|65692|4294967354|72.07|30.70|false|wendy steinbeck|2013-03-01 09:11:58.703121|73.41|zync studies +37|327|65548|4294967382|89.11|4.63|false|oscar miller|2013-03-01 09:11:58.703113|12.83|yard duty +110|288|65585|4294967494|95.43|37.25|true|sarah underhill|2013-03-01 09:11:58.703277|85.86|debate +76|297|65566|4294967484|66.30|8.35|false|yuri king|2013-03-01 09:11:58.703227|85.79|nap time +-2|445|65737|4294967351|35.62|27.31|true|oscar zipper|2013-03-01 09:11:58.703127|51.55|forestry +89|451|65580|4294967322|17.56|21.06|true|bob miller|2013-03-01 09:11:58.703094|74.32|philosophy +65|375|65564|4294967442|16.23|21.07|true|irene quirinius|2013-03-01 09:11:58.703280|83.03|forestry +94|265|65749|4294967538|53.90|17.49|true|sarah hernandez|2013-03-01 09:11:58.703098|84.61|xylophone band +-1|483|65664|4294967426|51.26|33.70|true|mike hernandez|2013-03-01 09:11:58.703119|72.96|nap time +5|485|65725|4294967464|27.59|34.52|true|jessica polk|2013-03-01 09:11:58.703256|74.01|industrial engineering +31|392|65729|4294967330|52.96|39.77|false|tom quirinius|2013-03-01 09:11:58.703231|76.00|mathematics +104|344|65766|4294967436|87.94|42.01|false|ulysses ichabod|2013-03-01 09:11:58.703184|40.58|biology +14|346|65621|4294967439|33.57|1.46|true|mike davidson|2013-03-01 09:11:58.703154|86.94|xylophone band +96|362|65628|4294967426|91.41|12.03|false|calvin robinson|2013-03-01 09:11:58.703307|44.75|opthamology +107|504|65548|4294967401|92.69|33.56|true|rachel young|2013-03-01 09:11:58.703255|41.26|philosophy +89|318|65788|4294967305|99.09|45.55|false|priscilla nixon|2013-03-01 09:11:58.703105|0.97|chemistry +0|443|65633|4294967506|44.99|35.71|false|quinn miller|2013-03-01 09:11:58.703299|24.04|linguistics +24|361|65782|4294967361|80.99|0.45|false|rachel carson|2013-03-01 09:11:58.703162|72.04|joggying +72|370|65636|4294967375|77.97|47.60|true|victor ellison|2013-03-01 09:11:58.703115|28.54|forestry +81|268|65786|4294967528|54.84|42.64|false|wendy zipper|2013-03-01 09:11:58.703169|40.68|wind surfing +100|437|65578|4294967458|35.95|25.05|true|luke brown|2013-03-01 09:11:58.703127|59.02|geology +33|455|65696|4294967380|17.30|11.18|false|ethan robinson|2013-03-01 09:11:58.703276|21.84|nap time +108|305|65651|4294967489|5.36|44.25|true|jessica garcia|2013-03-01 09:11:58.703265|21.89|geology +29|408|65762|4294967505|77.60|0.39|false|zach garcia|2013-03-01 09:11:58.703294|14.74|wind surfing +62|504|65555|4294967514|26.66|18.23|true|irene johnson|2013-03-01 09:11:58.703087|95.80|topology +18|509|65576|4294967363|77.37|15.63|true|calvin quirinius|2013-03-01 09:11:58.703254|76.21|undecided +66|303|65647|4294967453|80.82|3.08|true|bob van buren|2013-03-01 09:11:58.703263|14.92|yard duty +123|371|65669|4294967385|76.54|5.90|false|nick nixon|2013-03-01 09:11:58.703311|26.32|xylophone band +25|408|65562|4294967301|92.29|32.85|false|ethan hernandez|2013-03-01 09:11:58.703104|39.85|chemistry +52|322|65629|4294967351|2.44|14.35|false|rachel laertes|2013-03-01 09:11:58.703195|73.46|kindergarten +6|429|65620|4294967375|56.65|49.95|false|tom garcia|2013-03-01 09:11:58.703193|27.84|yard duty +51|280|65733|4294967378|99.21|18.67|false|david robinson|2013-03-01 09:11:58.703289|69.33|chemistry +93|429|65776|4294967542|39.04|43.25|false|yuri nixon|2013-03-01 09:11:58.703317|48.13|yard duty +52|357|65661|4294967502|75.71|5.72|false|xavier brown|2013-03-01 09:11:58.703093|74.17|joggying +23|494|65705|4294967356|99.53|18.84|false|zach polk|2013-03-01 09:11:58.703288|19.08|topology +65|416|65617|4294967487|33.69|39.96|true|quinn robinson|2013-03-01 09:11:58.703292|29.53|quiet hour +-1|307|65713|4294967322|82.89|7.78|false|david underhill|2013-03-01 09:11:58.703200|71.76|opthamology +112|505|65617|4294967439|62.89|42.01|true|fred johnson|2013-03-01 09:11:58.703104|99.63|biology +118|329|65737|4294967382|64.39|12.59|false|bob robinson|2013-03-01 09:11:58.703177|69.14|values clariffication +77|402|65558|4294967525|17.14|43.24|true|sarah garcia|2013-03-01 09:11:58.703131|53.10|xylophone band +75|361|65729|4294967543|68.33|7.76|true|mike carson|2013-03-01 09:11:58.703084|31.83|values clariffication +6|411|65751|4294967435|34.01|23.50|false|zach young|2013-03-01 09:11:58.703173|14.12|debate +113|505|65740|4294967412|73.52|41.38|true|fred polk|2013-03-01 09:11:58.703238|39.61|mathematics +119|483|65765|4294967335|31.75|22.82|false|katie king|2013-03-01 09:11:58.703188|16.81|opthamology +24|314|65686|4294967443|89.16|33.17|true|zach thompson|2013-03-01 09:11:58.703179|93.43|history +83|356|65670|4294967521|9.82|43.87|true|irene polk|2013-03-01 09:11:58.703183|47.57|american history +33|330|65649|4294967369|25.73|29.97|false|gabriella falkner|2013-03-01 09:11:58.703214|34.77|undecided +116|309|65780|4294967430|93.67|29.76|false|alice xylophone|2013-03-01 09:11:58.703160|58.54|biology +102|360|65653|4294967488|98.34|12.51|true|david quirinius|2013-03-01 09:11:58.703205|28.15|opthamology +90|484|65683|4294967376|1.17|12.17|true|zach ellison|2013-03-01 09:11:58.703279|80.09|biology +4|264|65728|4294967318|85.03|47.63|false|rachel van buren|2013-03-01 09:11:58.703269|18.86|values clariffication +47|491|65599|4294967305|66.10|27.03|true|alice thompson|2013-03-01 09:11:58.703258|52.41|chemistry +110|386|65647|4294967436|19.15|46.78|true|david ellison|2013-03-01 09:11:58.703238|80.43|chemistry +-3|298|65720|4294967305|34.60|39.70|false|ethan steinbeck|2013-03-01 09:11:58.703079|34.82|kindergarten +29|337|65601|4294967406|23.53|26.83|true|victor garcia|2013-03-01 09:11:58.703264|66.47|history +72|421|65787|4294967391|52.13|8.23|true|wendy ichabod|2013-03-01 09:11:58.703298|37.22|undecided +112|498|65561|4294967365|94.74|20.04|true|quinn allen|2013-03-01 09:11:58.703128|77.75|philosophy +54|448|65650|4294967377|82.76|42.31|false|wendy young|2013-03-01 09:11:58.703151|50.54|history +28|305|65771|4294967474|18.08|49.44|true|irene falkner|2013-03-01 09:11:58.703261|48.97|chemistry +83|308|65686|4294967417|48.37|36.70|false|yuri allen|2013-03-01 09:11:58.703074|32.40|quiet hour +3|394|65744|4294967510|38.04|21.56|false|xavier allen|2013-03-01 09:11:58.703267|50.17|joggying +5|408|65708|4294967498|42.70|21.15|false|quinn falkner|2013-03-01 09:11:58.703237|30.67|zync studies +62|420|65599|4294967306|25.35|41.64|true|bob polk|2013-03-01 09:11:58.703223|84.58|philosophy +116|285|65771|4294967296|65.17|24.37|false|holly allen|2013-03-01 09:11:58.703147|0.40|mathematics +113|379|65682|4294967410|69.40|26.95|false|gabriella underhill|2013-03-01 09:11:58.703163|49.18|industrial engineering +124|333|65606|4294967389|30.62|47.51|false|zach davidson|2013-03-01 09:11:58.703291|90.02|quiet hour +31|493|65579|4294967313|84.95|31.24|false|sarah polk|2013-03-01 09:11:58.703240|4.91|kindergarten +118|313|65726|4294967498|44.21|32.08|true|calvin xylophone|2013-03-01 09:11:58.703189|39.37|forestry +36|394|65702|4294967507|53.76|48.23|false|alice white|2013-03-01 09:11:58.703292|81.35|study skills +96|326|65540|4294967329|86.40|39.84|true|holly garcia|2013-03-01 09:11:58.703226|45.77|education +120|338|65694|4294967424|25.06|33.73|true|ulysses laertes|2013-03-01 09:11:58.703143|15.69|education +62|275|65691|4294967500|29.46|35.94|true|bob ellison|2013-03-01 09:11:58.703270|15.38|religion +106|503|65701|4294967406|74.77|25.45|true|quinn allen|2013-03-01 09:11:58.703226|96.80|topology +6|395|65730|4294967478|6.19|33.53|true|holly xylophone|2013-03-01 09:11:58.703210|24.23|opthamology +61|440|65786|4294967402|70.39|45.54|true|sarah laertes|2013-03-01 09:11:58.703234|54.37|quiet hour +0|504|65690|4294967327|72.29|16.30|true|rachel xylophone|2013-03-01 09:11:58.703319|67.87|debate +93|368|65710|4294967422|7.70|44.54|true|ulysses johnson|2013-03-01 09:11:58.703173|97.85|geology +35|356|65791|4294967541|46.42|6.49|true|ulysses davidson|2013-03-01 09:11:58.703086|79.88|xylophone band +78|430|65729|4294967297|76.93|36.01|false|calvin van buren|2013-03-01 09:11:58.703317|72.39|undecided +73|501|65738|4294967315|76.57|38.86|false|mike underhill|2013-03-01 09:11:58.703192|47.55|quiet hour +116|490|65723|4294967484|78.41|29.16|false|xavier king|2013-03-01 09:11:58.703079|58.10|industrial engineering +121|271|65555|4294967537|67.79|36.20|true|fred miller|2013-03-01 09:11:58.703152|48.07|values clariffication +10|462|65584|4294967516|43.20|41.53|true|fred falkner|2013-03-01 09:11:58.703174|94.77|biology +7|305|65571|4294967408|14.70|9.65|false|ulysses xylophone|2013-03-01 09:11:58.703071|70.57|geology +67|352|65651|4294967313|20.48|16.31|false|fred falkner|2013-03-01 09:11:58.703227|67.04|joggying +87|301|65719|4294967443|12.25|26.23|false|david johnson|2013-03-01 09:11:58.703208|77.93|values clariffication +13|422|65698|4294967349|72.43|28.20|false|sarah quirinius|2013-03-01 09:11:58.703076|34.88|topology +108|344|65605|4294967388|70.32|48.28|true|calvin king|2013-03-01 09:11:58.703156|98.10|topology +47|431|65775|4294967492|67.50|38.18|true|fred quirinius|2013-03-01 09:11:58.703173|40.57|geology +17|414|65695|4294967405|31.68|2.89|true|sarah nixon|2013-03-01 09:11:58.703254|17.05|forestry +78|374|65773|4294967464|85.59|1.68|false|alice steinbeck|2013-03-01 09:11:58.703244|67.02|linguistics +12|510|65668|4294967424|88.65|36.20|true|ulysses brown|2013-03-01 09:11:58.703282|75.09|forestry +102|294|65606|4294967495|42.79|45.29|false|irene robinson|2013-03-01 09:11:58.703079|62.59|zync studies +43|264|65614|4294967415|11.39|45.03|true|calvin carson|2013-03-01 09:11:58.703138|44.20|industrial engineering +23|370|65563|4294967390|9.49|5.50|true|mike brown|2013-03-01 09:11:58.703293|84.57|philosophy +77|327|65753|4294967400|40.34|9.50|false|fred xylophone|2013-03-01 09:11:58.703093|87.48|education +29|375|65757|4294967354|33.50|18.35|true|calvin garcia|2013-03-01 09:11:58.703140|47.91|xylophone band +31|301|65660|4294967512|56.91|22.63|true|rachel johnson|2013-03-01 09:11:58.703113|71.84|xylophone band +63|370|65575|4294967466|89.97|11.35|false|quinn thompson|2013-03-01 09:11:58.703257|96.43|xylophone band +94|295|65747|4294967361|29.50|27.79|false|nick nixon|2013-03-01 09:11:58.703243|21.55|linguistics +26|290|65734|4294967366|61.34|28.69|false|bob underhill|2013-03-01 09:11:58.703227|41.71|kindergarten +49|462|65681|4294967537|37.40|32.23|true|yuri falkner|2013-03-01 09:11:58.703269|64.70|nap time +-2|283|65695|4294967436|4.40|17.87|false|ethan polk|2013-03-01 09:11:58.703249|39.60|debate +120|266|65699|4294967389|2.06|13.81|false|wendy hernandez|2013-03-01 09:11:58.703312|34.58|debate +80|390|65723|4294967522|86.89|19.89|false|irene xylophone|2013-03-01 09:11:58.703199|95.79|philosophy +-2|462|65574|4294967366|4.06|29.84|false|wendy ellison|2013-03-01 09:11:58.703139|46.01|industrial engineering +103|259|65632|4294967449|73.42|36.35|false|jessica zipper|2013-03-01 09:11:58.703248|10.40|joggying +101|451|65744|4294967342|35.08|20.39|true|bob robinson|2013-03-01 09:11:58.703143|70.80|american history +100|306|65642|4294967335|9.84|45.50|false|katie king|2013-03-01 09:11:58.703307|86.84|mathematics +123|443|65681|4294967308|26.80|24.29|true|calvin allen|2013-03-01 09:11:58.703153|87.84|linguistics +45|345|65735|4294967364|96.29|4.74|true|victor polk|2013-03-01 09:11:58.703248|42.36|zync studies +78|256|65586|4294967460|12.97|44.15|false|ethan falkner|2013-03-01 09:11:58.703171|81.51|chemistry +4|292|65589|4294967359|51.94|10.19|true|tom zipper|2013-03-01 09:11:58.703217|66.99|mathematics +71|262|65754|4294967385|17.76|6.77|true|david steinbeck|2013-03-01 09:11:58.703243|7.03|industrial engineering +-1|259|65789|4294967424|59.65|33.60|false|david garcia|2013-03-01 09:11:58.703092|83.21|quiet hour +20|427|65606|4294967312|21.45|17.87|false|jessica miller|2013-03-01 09:11:58.703164|33.41|debate +64|268|65676|4294967328|35.12|24.59|false|gabriella falkner|2013-03-01 09:11:58.703322|61.82|linguistics +79|389|65711|4294967384|34.73|4.55|true|bob miller|2013-03-01 09:11:58.703204|13.21|history +92|336|65620|4294967346|94.14|0.24|true|yuri zipper|2013-03-01 09:11:58.703130|48.96|history +106|510|65788|4294967550|15.18|25.69|true|jessica steinbeck|2013-03-01 09:11:58.703249|14.21|xylophone band +2|454|65675|4294967445|91.59|45.89|false|nick hernandez|2013-03-01 09:11:58.703132|56.08|topology +107|265|65585|4294967545|83.09|9.45|false|ethan brown|2013-03-01 09:11:58.703313|80.40|zync studies +114|414|65733|4294967549|78.61|19.71|true|victor quirinius|2013-03-01 09:11:58.703083|57.66|biology +61|293|65663|4294967479|39.83|48.16|false|victor underhill|2013-03-01 09:11:58.703199|69.29|undecided +15|410|65662|4294967370|68.26|39.56|true|mike davidson|2013-03-01 09:11:58.703112|32.45|geology +37|290|65699|4294967539|59.51|44.24|false|irene van buren|2013-03-01 09:11:58.703288|38.60|industrial engineering +25|416|65549|4294967376|16.43|33.95|false|quinn white|2013-03-01 09:11:58.703216|70.58|chemistry +87|438|65710|4294967363|14.42|6.32|true|fred miller|2013-03-01 09:11:58.703211|28.68|debate +1|506|65656|4294967513|12.41|9.26|false|fred polk|2013-03-01 09:11:58.703093|53.41|geology +22|353|65646|4294967325|5.76|42.40|false|fred young|2013-03-01 09:11:58.703075|7.03|kindergarten +84|286|65536|4294967490|33.64|22.65|false|david johnson|2013-03-01 09:11:58.703279|93.68|zync studies +19|357|65601|4294967337|5.00|36.77|true|david brown|2013-03-01 09:11:58.703305|8.06|wind surfing +85|461|65569|4294967314|50.86|5.31|false|nick van buren|2013-03-01 09:11:58.703246|1.59|forestry +79|432|65775|4294967406|66.02|29.56|true|rachel young|2013-03-01 09:11:58.703169|15.23|history +24|339|65539|4294967328|73.35|2.20|false|oscar johnson|2013-03-01 09:11:58.703184|16.87|religion +73|427|65734|4294967379|45.23|41.98|true|yuri johnson|2013-03-01 09:11:58.703161|92.97|linguistics +64|381|65628|4294967547|99.64|33.77|true|priscilla carson|2013-03-01 09:11:58.703164|18.74|american history +123|386|65538|4294967337|66.10|6.89|true|ulysses davidson|2013-03-01 09:11:58.703222|83.82|quiet hour +4|342|65734|4294967504|86.23|41.34|false|david laertes|2013-03-01 09:11:58.703303|93.99|opthamology +10|340|65781|4294967478|84.96|22.70|true|luke falkner|2013-03-01 09:11:58.703243|40.12|joggying +36|351|65666|4294967360|66.34|1.42|true|katie young|2013-03-01 09:11:58.703133|8.65|mathematics +102|268|65614|4294967300|57.82|49.88|false|holly davidson|2013-03-01 09:11:58.703155|61.16|nap time +23|287|65725|4294967377|41.52|34.76|false|irene ellison|2013-03-01 09:11:58.703129|91.22|values clariffication +26|297|65553|4294967484|72.56|11.87|false|victor xylophone|2013-03-01 09:11:58.703256|54.80|american history +35|278|65582|4294967314|32.26|43.95|true|bob quirinius|2013-03-01 09:11:58.703258|94.16|joggying +92|443|65724|4294967383|96.99|47.82|false|calvin king|2013-03-01 09:11:58.703117|37.46|joggying +62|288|65764|4294967551|8.29|38.33|true|ethan zipper|2013-03-01 09:11:58.703209|56.54|joggying +104|286|65614|4294967432|1.90|41.73|true|wendy ovid|2013-03-01 09:11:58.703276|28.40|forestry +91|288|65789|4294967465|26.35|47.27|false|jessica brown|2013-03-01 09:11:58.703122|43.55|religion +84|399|65686|4294967447|48.27|22.15|true|bob thompson|2013-03-01 09:11:58.703290|82.94|forestry +17|387|65735|4294967416|81.41|32.20|true|katie davidson|2013-03-01 09:11:58.703071|88.08|wind surfing +84|458|65647|4294967439|79.73|38.59|false|xavier ellison|2013-03-01 09:11:58.703219|44.96|american history +15|421|65706|4294967532|28.23|34.56|false|sarah thompson|2013-03-01 09:11:58.703147|6.54|geology +9|288|65598|4294967475|35.46|24.35|false|nick laertes|2013-03-01 09:11:58.703100|81.35|quiet hour +90|378|65778|4294967352|52.09|40.45|false|sarah falkner|2013-03-01 09:11:58.703267|4.08|wind surfing +46|265|65556|4294967329|94.56|2.07|true|yuri xylophone|2013-03-01 09:11:58.703275|90.83|yard duty +10|300|65635|4294967544|62.24|3.88|true|mike garcia|2013-03-01 09:11:58.703295|25.13|history +60|368|65675|4294967309|80.28|26.32|true|irene robinson|2013-03-01 09:11:58.703248|55.43|nap time +120|321|65574|4294967362|47.24|48.58|false|bob zipper|2013-03-01 09:11:58.703096|42.13|religion +122|261|65654|4294967522|92.59|38.10|false|ulysses laertes|2013-03-01 09:11:58.703185|26.36|history +64|489|65618|4294967418|69.72|13.88|false|ethan underhill|2013-03-01 09:11:58.703214|9.45|kindergarten +17|461|65560|4294967342|98.28|28.99|true|ulysses miller|2013-03-01 09:11:58.703094|5.83|mathematics +123|439|65545|4294967551|87.70|22.43|false|david king|2013-03-01 09:11:58.703282|40.54|values clariffication +55|280|65660|4294967365|91.85|27.50|true|sarah brown|2013-03-01 09:11:58.703087|3.54|industrial engineering +51|419|65703|4294967329|71.61|27.56|false|jessica young|2013-03-01 09:11:58.703191|94.03|linguistics +105|347|65584|4294967408|45.05|27.03|true|luke johnson|2013-03-01 09:11:58.703074|9.84|joggying +73|426|65584|4294967459|39.49|44.82|true|tom allen|2013-03-01 09:11:58.703241|79.44|nap time +95|317|65641|4294967434|90.16|24.62|true|holly white|2013-03-01 09:11:58.703176|38.81|topology +101|391|65705|4294967456|58.46|36.43|false|holly thompson|2013-03-01 09:11:58.703233|14.47|mathematics +114|290|65568|4294967348|31.22|30.50|true|fred thompson|2013-03-01 09:11:58.703273|51.56|joggying +109|329|65696|4294967474|58.07|48.62|false|wendy ichabod|2013-03-01 09:11:58.703128|84.47|forestry +50|500|65687|4294967517|47.89|42.70|true|ulysses hernandez|2013-03-01 09:11:58.703258|74.68|religion +37|342|65679|4294967536|69.97|0.30|false|priscilla zipper|2013-03-01 09:11:58.703079|70.01|geology +33|419|65541|4294967399|49.51|6.32|false|katie laertes|2013-03-01 09:11:58.703104|79.93|xylophone band +35|328|65612|4294967438|50.00|37.43|false|alice miller|2013-03-01 09:11:58.703232|20.65|topology +58|486|65787|4294967426|51.47|45.38|false|david xylophone|2013-03-01 09:11:58.703124|89.50|industrial engineering +48|449|65589|4294967526|51.19|49.81|true|bob young|2013-03-01 09:11:58.703169|62.00|study skills +117|344|65661|4294967397|82.26|15.27|false|rachel thompson|2013-03-01 09:11:58.703324|16.09|values clariffication +112|429|65546|4294967312|1.64|47.62|true|jessica ovid|2013-03-01 09:11:58.703085|29.36|american history +75|416|65625|4294967364|88.24|26.17|true|ulysses xylophone|2013-03-01 09:11:58.703183|39.05|american history +61|493|65766|4294967353|97.42|11.93|true|irene quirinius|2013-03-01 09:11:58.703090|23.72|philosophy +9|421|65722|4294967427|82.73|32.53|true|xavier nixon|2013-03-01 09:11:58.703183|56.53|opthamology +52|352|65556|4294967465|49.63|17.87|true|gabriella laertes|2013-03-01 09:11:58.703254|85.97|nap time +76|300|65654|4294967426|88.83|20.26|false|wendy brown|2013-03-01 09:11:58.703072|39.18|nap time +46|360|65738|4294967437|25.09|30.84|true|tom ovid|2013-03-01 09:11:58.703292|81.53|nap time +9|270|65767|4294967504|48.60|2.43|false|luke ichabod|2013-03-01 09:11:58.703262|43.66|yard duty +48|480|65564|4294967423|92.55|45.00|true|fred quirinius|2013-03-01 09:11:58.703073|80.97|education +50|371|65710|4294967436|17.60|5.39|true|alice falkner|2013-03-01 09:11:58.703190|23.34|debate +62|267|65726|4294967341|30.45|18.99|true|sarah underhill|2013-03-01 09:11:58.703136|12.92|joggying +93|468|65759|4294967505|31.00|40.35|true|jessica johnson|2013-03-01 09:11:58.703115|0.08|history +51|457|65632|4294967429|68.53|17.31|true|fred ellison|2013-03-01 09:11:58.703295|52.29|debate +67|413|65706|4294967368|17.35|11.31|true|irene thompson|2013-03-01 09:11:58.703153|34.02|linguistics +28|400|65688|4294967419|23.45|36.03|false|katie davidson|2013-03-01 09:11:58.703234|91.31|opthamology +80|307|65695|4294967366|47.42|7.58|true|bob nixon|2013-03-01 09:11:58.703093|48.21|religion +48|509|65658|4294967441|48.41|28.97|false|katie quirinius|2013-03-01 09:11:58.703230|21.56|undecided +79|479|65582|4294967339|38.77|19.41|true|katie polk|2013-03-01 09:11:58.703280|32.13|education +17|483|65578|4294967368|48.79|5.40|false|ethan johnson|2013-03-01 09:11:58.703241|33.18|linguistics +71|491|65791|4294967368|91.87|46.94|true|alice miller|2013-03-01 09:11:58.703183|61.58|opthamology +39|421|65711|4294967508|18.93|41.99|false|mike underhill|2013-03-01 09:11:58.703252|12.07|philosophy +40|364|65590|4294967318|10.85|47.16|false|oscar hernandez|2013-03-01 09:11:58.703228|33.22|nap time +113|363|65659|4294967444|31.24|41.17|true|alice zipper|2013-03-01 09:11:58.703174|87.52|zync studies +34|477|65682|4294967358|60.61|30.09|true|bob davidson|2013-03-01 09:11:58.703261|19.55|chemistry +98|359|65543|4294967375|55.71|24.08|false|luke underhill|2013-03-01 09:11:58.703137|51.09|forestry +88|384|65754|4294967458|32.90|23.85|false|quinn steinbeck|2013-03-01 09:11:58.703248|4.59|opthamology +22|461|65543|4294967404|7.40|19.91|false|victor robinson|2013-03-01 09:11:58.703128|32.99|wind surfing +1|458|65719|4294967429|36.92|29.22|true|ethan king|2013-03-01 09:11:58.703077|61.57|biology +60|362|65660|4294967328|10.55|4.03|true|victor xylophone|2013-03-01 09:11:58.703162|84.35|opthamology +18|430|65704|4294967515|94.05|27.31|false|priscilla ovid|2013-03-01 09:11:58.703077|82.28|opthamology +119|499|65554|4294967532|30.92|45.31|false|victor brown|2013-03-01 09:11:58.703203|72.90|biology +77|400|65601|4294967495|3.16|39.12|true|holly king|2013-03-01 09:11:58.703182|62.80|xylophone band +40|282|65783|4294967487|62.70|47.93|false|wendy steinbeck|2013-03-01 09:11:58.703147|55.42|quiet hour +23|496|65658|4294967437|51.17|13.21|true|quinn van buren|2013-03-01 09:11:58.703296|91.85|chemistry +96|337|65603|4294967441|63.44|15.24|true|quinn white|2013-03-01 09:11:58.703237|43.37|undecided +22|376|65600|4294967382|73.46|37.32|false|sarah ovid|2013-03-01 09:11:58.703198|77.39|kindergarten +107|455|65780|4294967432|8.61|22.76|false|zach allen|2013-03-01 09:11:58.703171|27.62|undecided +98|292|65774|4294967439|21.08|31.32|true|yuri quirinius|2013-03-01 09:11:58.703275|82.02|religion +17|440|65569|4294967391|88.19|13.91|false|oscar king|2013-03-01 09:11:58.703278|74.79|quiet hour +66|353|65742|4294967311|65.55|33.22|false|oscar allen|2013-03-01 09:11:58.703112|20.94|kindergarten +14|388|65544|4294967344|89.96|25.86|false|irene laertes|2013-03-01 09:11:58.703083|51.39|kindergarten +5|489|65543|4294967471|67.72|35.67|false|fred laertes|2013-03-01 09:11:58.703217|5.10|opthamology +11|297|65583|4294967452|8.43|14.02|true|tom underhill|2013-03-01 09:11:58.703213|71.90|study skills +120|473|65770|4294967447|54.94|19.08|true|rachel nixon|2013-03-01 09:11:58.703099|88.67|wind surfing +43|484|65684|4294967543|84.35|21.58|true|rachel davidson|2013-03-01 09:11:58.703104|4.62|american history +66|276|65695|4294967361|90.42|7.81|true|sarah king|2013-03-01 09:11:58.703284|3.22|philosophy +17|376|65708|4294967302|17.92|36.29|true|calvin quirinius|2013-03-01 09:11:58.703151|2.66|philosophy +-1|396|65736|4294967542|79.84|48.56|false|victor king|2013-03-01 09:11:58.703086|30.68|values clariffication +102|389|65640|4294967491|8.47|38.74|true|luke zipper|2013-03-01 09:11:58.703300|58.89|undecided +29|302|65711|4294967391|43.90|25.14|false|alice brown|2013-03-01 09:11:58.703081|84.37|linguistics +109|449|65614|4294967512|55.36|26.51|true|luke ovid|2013-03-01 09:11:58.703102|11.20|industrial engineering +44|381|65557|4294967520|90.28|30.81|true|victor young|2013-03-01 09:11:58.703315|30.96|nap time +113|377|65576|4294967395|8.18|45.29|false|gabriella king|2013-03-01 09:11:58.703318|49.18|study skills +43|423|65688|4294967385|46.01|18.47|true|zach davidson|2013-03-01 09:11:58.703308|97.09|philosophy +56|485|65633|4294967308|80.09|22.33|false|katie brown|2013-03-01 09:11:58.703144|95.06|quiet hour +110|498|65577|4294967351|59.63|40.83|true|irene king|2013-03-01 09:11:58.703142|61.43|history +31|259|65710|4294967480|35.29|28.62|false|wendy zipper|2013-03-01 09:11:58.703111|49.91|philosophy +14|446|65772|4294967543|53.92|25.84|true|sarah brown|2013-03-01 09:11:58.703273|38.05|biology +108|426|65588|4294967438|31.00|45.02|true|luke miller|2013-03-01 09:11:58.703293|11.90|zync studies +71|292|65558|4294967432|2.01|9.03|true|luke brown|2013-03-01 09:11:58.703309|7.61|opthamology +67|319|65715|4294967427|35.42|22.97|false|ethan king|2013-03-01 09:11:58.703106|84.96|biology +54|329|65644|4294967424|14.83|11.82|false|wendy young|2013-03-01 09:11:58.703108|51.29|values clariffication +76|295|65730|4294967405|55.88|13.40|true|irene xylophone|2013-03-01 09:11:58.703092|6.06|kindergarten +122|504|65548|4294967455|63.88|15.72|false|ulysses carson|2013-03-01 09:11:58.703166|87.50|chemistry +2|407|65676|4294967549|96.82|37.50|false|holly van buren|2013-03-01 09:11:58.703115|24.48|geology +15|461|65582|4294967462|21.78|34.12|false|nick underhill|2013-03-01 09:11:58.703219|3.11|chemistry +45|399|65762|4294967354|23.23|40.13|false|sarah hernandez|2013-03-01 09:11:58.703081|70.02|joggying +12|360|65685|4294967406|83.19|28.47|true|jessica quirinius|2013-03-01 09:11:58.703320|9.82|quiet hour +25|311|65721|4294967446|34.53|1.32|true|jessica white|2013-03-01 09:11:58.703200|46.13|mathematics +109|416|65580|4294967497|30.72|1.94|true|david zipper|2013-03-01 09:11:58.703299|41.81|quiet hour +87|411|65686|4294967307|53.74|32.62|false|xavier ichabod|2013-03-01 09:11:58.703180|81.13|opthamology +26|347|65677|4294967510|34.48|33.09|true|david carson|2013-03-01 09:11:58.703270|43.56|industrial engineering +29|462|65687|4294967415|9.62|4.10|true|ulysses nixon|2013-03-01 09:11:58.703098|63.60|opthamology +37|264|65698|4294967358|19.49|32.77|true|sarah young|2013-03-01 09:11:58.703288|75.95|industrial engineering +99|277|65573|4294967531|33.42|33.67|true|oscar thompson|2013-03-01 09:11:58.703090|78.83|wind surfing +90|272|65760|4294967329|3.21|21.15|false|zach ovid|2013-03-01 09:11:58.703098|94.91|nap time +69|336|65649|4294967448|8.06|23.57|true|david zipper|2013-03-01 09:11:58.703271|50.03|chemistry +102|385|65542|4294967420|94.58|27.08|true|xavier brown|2013-03-01 09:11:58.703096|30.42|values clariffication +115|324|65568|4294967465|95.63|49.26|true|priscilla ellison|2013-03-01 09:11:58.703191|72.46|nap time +21|355|65648|4294967351|57.76|9.75|false|priscilla zipper|2013-03-01 09:11:58.703268|13.26|chemistry +38|401|65725|4294967444|28.85|43.56|true|rachel nixon|2013-03-01 09:11:58.703322|9.15|debate +1|422|65666|4294967401|76.72|42.66|true|ulysses white|2013-03-01 09:11:58.703095|64.04|religion +-1|323|65669|4294967356|83.97|23.40|true|nick king|2013-03-01 09:11:58.703242|95.16|philosophy +45|391|65715|4294967505|12.41|31.59|false|luke white|2013-03-01 09:11:58.703257|72.79|american history +48|481|65622|4294967431|56.05|28.82|true|luke brown|2013-03-01 09:11:58.703254|71.80|geology +103|369|65583|4294967325|20.85|13.06|true|quinn nixon|2013-03-01 09:11:58.703097|57.91|forestry +77|381|65687|4294967368|33.29|32.55|true|calvin steinbeck|2013-03-01 09:11:58.703128|43.74|education +56|259|65771|4294967360|99.71|45.84|true|bob hernandez|2013-03-01 09:11:58.703200|0.99|geology +0|275|65756|4294967453|14.22|35.26|true|holly allen|2013-03-01 09:11:58.703180|20.67|linguistics +86|441|65667|4294967514|51.68|16.77|false|priscilla ichabod|2013-03-01 09:11:58.703225|8.25|philosophy +-1|474|65578|4294967404|30.83|34.56|false|david nixon|2013-03-01 09:11:58.703316|70.15|nap time +1|261|65782|4294967438|95.52|33.13|true|yuri laertes|2013-03-01 09:11:58.703256|50.67|zync studies +40|381|65704|4294967330|32.05|46.97|true|irene white|2013-03-01 09:11:58.703272|70.89|undecided +72|305|65711|4294967322|20.36|16.70|true|quinn robinson|2013-03-01 09:11:58.703243|83.00|american history +43|347|65639|4294967394|44.61|19.60|true|sarah johnson|2013-03-01 09:11:58.703288|14.40|values clariffication +72|356|65586|4294967301|65.13|49.89|false|zach johnson|2013-03-01 09:11:58.703211|36.98|industrial engineering +57|491|65788|4294967360|63.57|43.54|false|tom brown|2013-03-01 09:11:58.703241|59.21|linguistics +66|336|65753|4294967522|13.39|1.62|true|tom ellison|2013-03-01 09:11:58.703109|25.83|wind surfing +18|295|65623|4294967331|31.20|23.04|false|victor van buren|2013-03-01 09:11:58.703138|94.47|nap time +63|474|65770|4294967394|23.89|30.48|true|fred johnson|2013-03-01 09:11:58.703289|85.88|history +117|415|65606|4294967465|41.76|14.09|true|sarah ellison|2013-03-01 09:11:58.703289|19.63|biology +104|467|65658|4294967480|69.47|24.01|false|calvin steinbeck|2013-03-01 09:11:58.703249|16.93|mathematics +44|336|65718|4294967436|60.04|22.67|false|ulysses underhill|2013-03-01 09:11:58.703144|93.29|study skills +102|341|65587|4294967366|30.52|37.48|true|nick brown|2013-03-01 09:11:58.703194|65.49|american history +47|440|65717|4294967530|64.03|46.51|true|zach polk|2013-03-01 09:11:58.703121|48.02|values clariffication +48|478|65605|4294967299|85.23|41.07|true|irene king|2013-03-01 09:11:58.703085|24.15|opthamology +103|465|65753|4294967369|16.22|5.41|true|priscilla van buren|2013-03-01 09:11:58.703320|7.98|nap time +7|357|65620|4294967474|56.65|23.27|false|bob johnson|2013-03-01 09:11:58.703300|96.98|religion +3|437|65651|4294967331|34.20|33.34|false|quinn davidson|2013-03-01 09:11:58.703138|83.65|chemistry +91|467|65630|4294967456|15.62|28.31|false|priscilla underhill|2013-03-01 09:11:58.703081|3.83|linguistics +10|387|65706|4294967534|69.13|0.68|true|irene quirinius|2013-03-01 09:11:58.703252|51.83|joggying +44|295|65540|4294967432|0.85|28.56|true|david ellison|2013-03-01 09:11:58.703075|66.25|forestry +40|485|65587|4294967381|10.71|47.80|true|rachel young|2013-03-01 09:11:58.703255|60.25|xylophone band +79|365|65734|4294967533|99.89|47.48|false|bob ichabod|2013-03-01 09:11:58.703120|35.10|industrial engineering +86|284|65542|4294967434|18.78|40.69|true|rachel polk|2013-03-01 09:11:58.703324|23.85|quiet hour +18|307|65561|4294967414|40.56|5.84|false|ethan laertes|2013-03-01 09:11:58.703106|83.08|xylophone band +83|505|65536|4294967451|35.46|6.36|false|oscar steinbeck|2013-03-01 09:11:58.703319|4.34|opthamology +42|447|65610|4294967467|69.06|13.24|false|fred white|2013-03-01 09:11:58.703081|96.68|debate +28|353|65789|4294967335|57.13|19.93|false|tom ichabod|2013-03-01 09:11:58.703163|86.61|nap time +62|368|65674|4294967489|40.18|18.41|true|priscilla van buren|2013-03-01 09:11:58.703113|59.10|industrial engineering +22|263|65636|4294967518|43.05|46.40|false|ethan ichabod|2013-03-01 09:11:58.703275|83.64|industrial engineering +1|277|65620|4294967400|15.55|15.32|false|fred van buren|2013-03-01 09:11:58.703171|20.24|geology +34|298|65729|4294967539|6.41|38.84|false|calvin polk|2013-03-01 09:11:58.703146|17.13|geology +102|457|65737|4294967490|58.84|28.09|false|tom miller|2013-03-01 09:11:58.703220|1.24|yard duty +5|434|65629|4294967533|75.03|10.25|true|holly falkner|2013-03-01 09:11:58.703089|33.18|industrial engineering +82|485|65639|4294967382|31.08|15.64|false|yuri thompson|2013-03-01 09:11:58.703118|79.10|study skills +77|487|65611|4294967309|21.63|29.18|false|fred king|2013-03-01 09:11:58.703130|21.43|chemistry +23|311|65751|4294967329|26.04|47.15|false|gabriella ichabod|2013-03-01 09:11:58.703303|41.83|chemistry +75|336|65732|4294967318|48.65|39.40|false|xavier underhill|2013-03-01 09:11:58.703247|12.03|biology +99|322|65591|4294967483|43.47|2.07|false|katie young|2013-03-01 09:11:58.703259|96.60|religion +35|433|65786|4294967349|69.25|21.02|false|jessica laertes|2013-03-01 09:11:58.703099|27.76|geology +32|401|65707|4294967323|78.67|42.54|true|rachel van buren|2013-03-01 09:11:58.703197|52.24|opthamology +41|316|65541|4294967428|54.78|44.91|false|yuri polk|2013-03-01 09:11:58.703165|68.09|linguistics +78|394|65673|4294967405|8.42|43.32|false|wendy polk|2013-03-01 09:11:58.703184|82.72|linguistics +98|297|65712|4294967503|19.34|23.85|false|gabriella brown|2013-03-01 09:11:58.703087|98.79|opthamology +69|269|65720|4294967384|77.67|43.03|true|tom falkner|2013-03-01 09:11:58.703316|11.19|yard duty +93|415|65625|4294967326|84.74|32.86|false|fred ellison|2013-03-01 09:11:58.703318|14.01|geology +64|317|65647|4294967545|36.88|1.60|false|bob ovid|2013-03-01 09:11:58.703145|33.25|kindergarten +6|426|65599|4294967301|13.07|6.67|false|wendy carson|2013-03-01 09:11:58.703107|28.08|xylophone band +29|377|65717|4294967329|47.67|12.40|true|nick carson|2013-03-01 09:11:58.703245|24.44|undecided +30|377|65759|4294967460|10.37|26.05|true|luke van buren|2013-03-01 09:11:58.703168|41.70|joggying +98|302|65647|4294967397|64.11|22.69|false|rachel van buren|2013-03-01 09:11:58.703260|50.34|forestry +44|395|65568|4294967541|92.90|24.63|true|victor garcia|2013-03-01 09:11:58.703125|10.88|opthamology +96|392|65784|4294967550|8.13|14.17|true|ethan young|2013-03-01 09:11:58.703325|17.21|mathematics +115|419|65609|4294967354|67.24|35.58|true|xavier polk|2013-03-01 09:11:58.703150|28.35|religion +45|390|65715|4294967490|91.17|34.85|false|oscar brown|2013-03-01 09:11:58.703262|68.48|religion +113|380|65735|4294967332|7.38|2.22|true|irene van buren|2013-03-01 09:11:58.703181|4.74|industrial engineering +46|438|65787|4294967307|89.09|5.94|false|rachel xylophone|2013-03-01 09:11:58.703102|45.89|forestry +69|274|65780|4294967472|15.38|48.61|true|david king|2013-03-01 09:11:58.703152|11.58|religion +115|334|65629|4294967524|54.61|1.33|true|nick underhill|2013-03-01 09:11:58.703104|94.78|topology +6|313|65549|4294967494|27.79|45.21|false|luke carson|2013-03-01 09:11:58.703295|19.93|philosophy +119|430|65760|4294967431|50.17|14.12|true|luke falkner|2013-03-01 09:11:58.703321|0.55|yard duty +70|417|65692|4294967337|77.34|10.95|true|jessica young|2013-03-01 09:11:58.703083|36.51|wind surfing +9|362|65669|4294967548|5.47|32.94|true|mike robinson|2013-03-01 09:11:58.703185|97.77|quiet hour +68|285|65674|4294967420|38.66|16.70|true|david nixon|2013-03-01 09:11:58.703151|5.07|kindergarten +44|300|65557|4294967410|45.89|1.84|false|zach quirinius|2013-03-01 09:11:58.703154|94.61|kindergarten +60|278|65557|4294967540|77.96|45.71|false|yuri steinbeck|2013-03-01 09:11:58.703209|2.45|geology +37|320|65678|4294967549|3.21|30.35|true|yuri carson|2013-03-01 09:11:58.703094|3.39|nap time +76|340|65742|4294967440|89.33|24.35|true|gabriella nixon|2013-03-01 09:11:58.703107|18.12|education +54|450|65631|4294967361|5.96|5.91|false|holly johnson|2013-03-01 09:11:58.703321|63.25|mathematics +14|326|65698|4294967487|35.71|11.75|true|zach steinbeck|2013-03-01 09:11:58.703306|62.01|opthamology +35|373|65752|4294967534|55.51|37.65|false|zach nixon|2013-03-01 09:11:58.703159|82.44|american history +54|431|65755|4294967532|18.62|29.02|false|ulysses carson|2013-03-01 09:11:58.703317|97.96|values clariffication +27|263|65782|4294967532|23.63|30.74|false|victor ichabod|2013-03-01 09:11:58.703324|17.09|wind surfing +109|362|65763|4294967405|80.02|18.10|true|priscilla garcia|2013-03-01 09:11:58.703072|50.26|linguistics +119|391|65776|4294967454|59.88|1.75|false|nick ichabod|2013-03-01 09:11:58.703199|38.30|xylophone band +19|349|65611|4294967364|89.72|47.17|false|rachel ichabod|2013-03-01 09:11:58.703173|76.88|geology +123|260|65652|4294967534|70.66|49.82|false|jessica johnson|2013-03-01 09:11:58.703083|22.49|nap time +32|287|65655|4294967359|94.27|23.04|true|quinn zipper|2013-03-01 09:11:58.703210|46.81|history +71|280|65743|4294967389|27.81|44.29|false|quinn white|2013-03-01 09:11:58.703166|28.96|history +21|273|65541|4294967337|97.51|32.01|true|alice thompson|2013-03-01 09:11:58.703077|99.10|philosophy +39|506|65728|4294967544|82.19|27.96|false|bob steinbeck|2013-03-01 09:11:58.703193|5.05|undecided +41|414|65720|4294967316|9.10|9.15|false|jessica davidson|2013-03-01 09:11:58.703108|98.36|education +47|316|65731|4294967470|97.37|15.54|false|victor ichabod|2013-03-01 09:11:58.703264|82.37|geology +89|336|65645|4294967303|35.19|30.77|false|alice underhill|2013-03-01 09:11:58.703222|35.65|nap time +40|461|65603|4294967399|79.80|39.72|true|wendy van buren|2013-03-01 09:11:58.703231|97.74|undecided +117|289|65593|4294967540|95.53|37.94|false|gabriella underhill|2013-03-01 09:11:58.703101|54.65|debate +114|295|65553|4294967311|65.22|11.94|true|katie allen|2013-03-01 09:11:58.703249|89.02|xylophone band +97|495|65648|4294967368|32.62|28.27|true|bob xylophone|2013-03-01 09:11:58.703294|10.76|values clariffication +61|421|65536|4294967489|26.32|22.25|true|priscilla white|2013-03-01 09:11:58.703203|80.26|history +18|398|65785|4294967336|32.69|0.32|false|alice ichabod|2013-03-01 09:11:58.703267|95.46|xylophone band +75|505|65662|4294967384|93.63|1.83|true|ethan ovid|2013-03-01 09:11:58.703219|80.83|forestry +0|260|65753|4294967315|3.23|48.98|true|katie ichabod|2013-03-01 09:11:58.703283|80.09|quiet hour +109|415|65637|4294967508|30.48|13.07|true|rachel miller|2013-03-01 09:11:58.703322|80.93|religion +101|460|65624|4294967440|49.52|41.56|true|ethan ovid|2013-03-01 09:11:58.703292|51.29|mathematics +111|411|65581|4294967361|73.76|16.06|false|xavier steinbeck|2013-03-01 09:11:58.703181|30.25|education +8|456|65740|4294967398|61.74|2.73|false|victor laertes|2013-03-01 09:11:58.703311|25.95|chemistry +109|418|65774|4294967337|45.02|35.81|true|holly polk|2013-03-01 09:11:58.703263|37.73|religion +18|361|65617|4294967549|65.26|2.12|true|fred carson|2013-03-01 09:11:58.703168|45.69|kindergarten +50|467|65689|4294967311|50.61|22.80|true|irene king|2013-03-01 09:11:58.703316|28.63|xylophone band +56|313|65637|4294967427|99.84|21.46|false|sarah white|2013-03-01 09:11:58.703284|59.78|yard duty +62|409|65773|4294967539|26.42|9.61|true|luke van buren|2013-03-01 09:11:58.703115|39.98|industrial engineering +80|364|65678|4294967493|78.28|27.80|true|calvin brown|2013-03-01 09:11:58.703237|11.73|xylophone band +19|284|65688|4294967439|40.85|18.55|false|zach falkner|2013-03-01 09:11:58.703120|27.72|biology +123|363|65723|4294967535|31.75|46.97|false|victor zipper|2013-03-01 09:11:58.703238|47.12|mathematics +114|453|65558|4294967466|49.74|20.26|false|yuri falkner|2013-03-01 09:11:58.703204|97.01|biology +35|389|65604|4294967497|77.75|15.65|true|yuri falkner|2013-03-01 09:11:58.703209|60.52|nap time +115|469|65695|4294967483|99.31|29.48|true|priscilla ichabod|2013-03-01 09:11:58.703265|53.50|mathematics +53|486|65758|4294967419|46.99|42.08|false|victor carson|2013-03-01 09:11:58.703257|95.33|chemistry +79|469|65692|4294967538|96.85|3.55|true|rachel johnson|2013-03-01 09:11:58.703090|5.46|nap time +112|490|65790|4294967464|77.97|24.24|true|nick brown|2013-03-01 09:11:58.703167|4.41|kindergarten +106|470|65728|4294967504|14.78|30.11|true|fred king|2013-03-01 09:11:58.703107|26.06|wind surfing +108|371|65671|4294967309|32.30|10.33|false|mike ichabod|2013-03-01 09:11:58.703194|69.22|wind surfing +1|505|65788|4294967433|61.43|3.98|false|mike white|2013-03-01 09:11:58.703190|11.65|wind surfing +35|405|65719|4294967533|99.72|39.54|true|quinn hernandez|2013-03-01 09:11:58.703229|62.67|quiet hour +73|395|65562|4294967453|41.29|48.50|false|jessica xylophone|2013-03-01 09:11:58.703199|44.72|forestry +90|318|65676|4294967347|49.98|21.45|true|ethan thompson|2013-03-01 09:11:58.703251|24.94|kindergarten +56|301|65744|4294967383|39.15|34.52|true|nick quirinius|2013-03-01 09:11:58.703307|90.42|wind surfing +34|446|65784|4294967371|48.50|0.04|false|tom young|2013-03-01 09:11:58.703216|2.33|opthamology +-2|277|65710|4294967388|95.54|40.00|false|jessica zipper|2013-03-01 09:11:58.703292|22.32|geology +96|307|65626|4294967467|84.54|2.77|false|ulysses zipper|2013-03-01 09:11:58.703214|77.26|wind surfing +62|336|65588|4294967511|96.35|25.32|false|alice laertes|2013-03-01 09:11:58.703176|87.01|xylophone band +122|435|65744|4294967374|64.04|47.50|false|gabriella davidson|2013-03-01 09:11:58.703221|91.98|kindergarten +31|277|65749|4294967391|76.35|23.64|true|david brown|2013-03-01 09:11:58.703270|77.55|kindergarten +44|492|65787|4294967395|95.04|18.83|true|jessica polk|2013-03-01 09:11:58.703070|40.48|american history +25|493|65703|4294967420|35.46|12.09|false|oscar brown|2013-03-01 09:11:58.703296|93.49|nap time +85|375|65757|4294967326|40.80|31.80|false|nick white|2013-03-01 09:11:58.703242|71.15|undecided +19|299|65735|4294967373|87.06|38.25|true|fred zipper|2013-03-01 09:11:58.703254|19.53|american history +98|422|65656|4294967467|34.95|24.43|true|katie white|2013-03-01 09:11:58.703176|8.34|geology +-3|432|65646|4294967492|0.83|27.18|true|oscar davidson|2013-03-01 09:11:58.703071|56.11|linguistics +75|452|65730|4294967326|44.15|11.32|false|bob xylophone|2013-03-01 09:11:58.703134|63.18|religion +102|319|65779|4294967514|66.66|37.38|true|gabriella van buren|2013-03-01 09:11:58.703291|83.39|nap time +55|267|65676|4294967441|26.90|38.20|true|ulysses garcia|2013-03-01 09:11:58.703259|62.93|religion +111|491|65751|4294967323|92.95|18.96|false|nick van buren|2013-03-01 09:11:58.703113|35.17|geology +112|347|65714|4294967444|71.61|2.45|false|calvin underhill|2013-03-01 09:11:58.703230|84.17|wind surfing +124|290|65560|4294967329|60.62|5.16|true|jessica falkner|2013-03-01 09:11:58.703082|0.99|opthamology +29|461|65541|4294967466|4.22|47.04|true|nick van buren|2013-03-01 09:11:58.703147|28.09|zync studies +25|440|65783|4294967492|84.73|37.56|false|ethan king|2013-03-01 09:11:58.703240|63.80|industrial engineering +115|439|65791|4294967470|33.97|44.40|false|fred ovid|2013-03-01 09:11:58.703186|25.71|debate +102|294|65623|4294967305|87.14|37.17|false|victor ichabod|2013-03-01 09:11:58.703313|15.70|education +60|341|65608|4294967481|56.50|13.63|true|victor brown|2013-03-01 09:11:58.703105|7.34|industrial engineering +101|394|65717|4294967401|58.04|37.69|true|priscilla allen|2013-03-01 09:11:58.703123|77.27|chemistry +94|415|65738|4294967422|55.86|8.18|true|david brown|2013-03-01 09:11:58.703092|14.38|philosophy +94|431|65724|4294967395|79.76|43.27|false|yuri ichabod|2013-03-01 09:11:58.703287|75.22|chemistry +5|271|65737|4294967541|41.25|22.04|false|mike xylophone|2013-03-01 09:11:58.703228|71.62|linguistics +45|324|65587|4294967300|3.84|11.32|false|wendy miller|2013-03-01 09:11:58.703156|9.10|religion +28|282|65605|4294967493|20.27|40.77|false|fred xylophone|2013-03-01 09:11:58.703267|75.39|chemistry +93|484|65767|4294967472|12.22|41.75|false|mike polk|2013-03-01 09:11:58.703213|61.50|nap time +91|270|65660|4294967525|48.74|8.96|true|bob nixon|2013-03-01 09:11:58.703203|87.21|opthamology +85|292|65788|4294967367|54.43|26.22|true|alice ichabod|2013-03-01 09:11:58.703148|75.66|history +13|449|65571|4294967476|6.71|48.33|false|victor xylophone|2013-03-01 09:11:58.703310|53.39|linguistics +25|504|65753|4294967472|36.35|26.60|false|priscilla miller|2013-03-01 09:11:58.703147|41.61|forestry +109|270|65663|4294967316|32.19|40.43|false|rachel xylophone|2013-03-01 09:11:58.703313|98.83|mathematics +82|281|65666|4294967473|40.98|8.05|true|gabriella quirinius|2013-03-01 09:11:58.703193|95.67|industrial engineering +22|393|65570|4294967534|97.71|34.63|true|katie brown|2013-03-01 09:11:58.703113|68.09|study skills +33|453|65564|4294967343|88.90|2.21|false|holly ichabod|2013-03-01 09:11:58.703231|48.69|forestry +119|288|65685|4294967314|40.34|6.60|false|mike zipper|2013-03-01 09:11:58.703256|8.35|study skills +66|346|65729|4294967548|94.88|16.88|false|holly underhill|2013-03-01 09:11:58.703168|16.05|religion +31|361|65758|4294967413|53.03|23.31|true|xavier thompson|2013-03-01 09:11:58.703194|7.30|joggying +21|412|65709|4294967480|48.15|42.41|false|ulysses underhill|2013-03-01 09:11:58.703261|26.83|history +55|369|65570|4294967421|14.58|19.41|true|ethan xylophone|2013-03-01 09:11:58.703112|14.27|wind surfing +69|322|65577|4294967437|36.08|31.06|true|katie robinson|2013-03-01 09:11:58.703297|40.63|values clariffication +72|378|65606|4294967377|76.44|5.39|false|priscilla ovid|2013-03-01 09:11:58.703101|85.38|opthamology +10|393|65651|4294967312|46.16|43.08|true|nick ovid|2013-03-01 09:11:58.703258|99.00|education +32|509|65732|4294967307|37.73|10.74|true|holly underhill|2013-03-01 09:11:58.703152|2.17|religion +14|474|65565|4294967307|16.95|14.09|false|gabriella underhill|2013-03-01 09:11:58.703128|83.20|study skills +-1|378|65634|4294967312|24.54|15.51|false|victor xylophone|2013-03-01 09:11:58.703296|19.57|wind surfing +57|387|65586|4294967475|67.56|0.26|true|katie hernandez|2013-03-01 09:11:58.703215|41.89|values clariffication +89|304|65733|4294967521|46.54|40.83|false|gabriella brown|2013-03-01 09:11:58.703109|1.43|debate +22|398|65697|4294967387|14.95|2.55|true|bob garcia|2013-03-01 09:11:58.703266|91.44|mathematics +106|473|65616|4294967427|32.25|3.26|true|luke ovid|2013-03-01 09:11:58.703258|2.10|quiet hour +100|295|65575|4294967479|90.67|8.11|false|rachel davidson|2013-03-01 09:11:58.703319|76.70|linguistics +52|429|65769|4294967443|45.09|39.71|false|mike steinbeck|2013-03-01 09:11:58.703119|48.43|opthamology +29|506|65550|4294967438|92.43|15.71|true|ethan thompson|2013-03-01 09:11:58.703268|13.11|study skills +50|422|65696|4294967363|54.57|28.26|true|bob johnson|2013-03-01 09:11:58.703144|70.80|debate +67|258|65753|4294967483|51.82|33.42|true|mike underhill|2013-03-01 09:11:58.703186|9.92|opthamology +17|466|65647|4294967461|81.11|16.27|true|ethan garcia|2013-03-01 09:11:58.703230|21.43|chemistry +53|445|65764|4294967443|66.98|49.87|false|nick steinbeck|2013-03-01 09:11:58.703223|1.07|mathematics +74|264|65650|4294967523|49.32|20.25|false|ulysses carson|2013-03-01 09:11:58.703144|5.86|values clariffication +34|415|65731|4294967508|38.83|36.15|false|calvin johnson|2013-03-01 09:11:58.703079|82.94|chemistry +4|416|65595|4294967419|72.41|32.67|true|yuri thompson|2013-03-01 09:11:58.703315|37.43|education +54|463|65548|4294967466|60.68|33.01|true|rachel robinson|2013-03-01 09:11:58.703315|78.57|nap time +9|298|65536|4294967530|50.28|7.77|true|nick davidson|2013-03-01 09:11:58.703132|5.53|education +76|439|65752|4294967459|44.39|17.78|true|gabriella ichabod|2013-03-01 09:11:58.703226|14.65|kindergarten +109|296|65658|4294967534|95.54|3.99|true|katie hernandez|2013-03-01 09:11:58.703210|26.20|topology +22|463|65580|4294967539|21.59|27.90|true|fred hernandez|2013-03-01 09:11:58.703152|11.34|joggying +106|430|65662|4294967532|5.54|31.81|true|katie ichabod|2013-03-01 09:11:58.703281|18.55|mathematics +30|341|65554|4294967544|35.20|41.75|true|bob laertes|2013-03-01 09:11:58.703298|45.86|linguistics +51|428|65602|4294967315|73.64|45.49|false|oscar garcia|2013-03-01 09:11:58.703267|21.70|xylophone band +85|291|65749|4294967444|35.81|34.21|true|luke ichabod|2013-03-01 09:11:58.703209|57.87|zync studies +15|285|65646|4294967393|80.54|23.04|false|rachel laertes|2013-03-01 09:11:58.703262|35.89|geology +47|447|65676|4294967478|17.85|33.00|false|mike brown|2013-03-01 09:11:58.703161|19.61|debate +112|507|65785|4294967513|25.81|2.60|true|fred steinbeck|2013-03-01 09:11:58.703230|36.17|undecided +36|263|65707|4294967379|73.58|2.63|false|victor allen|2013-03-01 09:11:58.703131|94.48|topology +32|486|65754|4294967312|50.44|33.84|true|holly robinson|2013-03-01 09:11:58.703104|69.75|religion +26|372|65749|4294967520|7.16|25.70|true|mike van buren|2013-03-01 09:11:58.703248|56.36|xylophone band +24|467|65672|4294967429|64.92|10.95|false|priscilla steinbeck|2013-03-01 09:11:58.703305|7.08|xylophone band +59|466|65766|4294967455|37.62|36.78|true|zach johnson|2013-03-01 09:11:58.703209|36.63|industrial engineering +86|475|65554|4294967344|31.43|28.87|false|calvin ovid|2013-03-01 09:11:58.703227|72.31|kindergarten +26|421|65652|4294967537|27.93|41.19|true|bob garcia|2013-03-01 09:11:58.703237|8.00|biology +30|318|65695|4294967375|56.10|46.50|true|alice carson|2013-03-01 09:11:58.703183|32.27|mathematics +83|266|65710|4294967384|7.24|20.79|false|calvin falkner|2013-03-01 09:11:58.703141|2.86|yard duty +16|325|65751|4294967441|25.63|0.33|true|wendy allen|2013-03-01 09:11:58.703264|72.17|philosophy +77|471|65756|4294967545|5.94|40.46|false|yuri king|2013-03-01 09:11:58.703144|99.64|history +94|313|65687|4294967426|72.92|38.17|false|calvin hernandez|2013-03-01 09:11:58.703273|33.30|nap time +105|280|65593|4294967529|96.43|23.26|true|david falkner|2013-03-01 09:11:58.703236|92.99|kindergarten +4|257|65572|4294967411|79.15|36.77|false|ethan king|2013-03-01 09:11:58.703190|31.10|american history +3|427|65595|4294967329|29.01|7.85|true|rachel polk|2013-03-01 09:11:58.703072|93.30|geology +30|298|65713|4294967547|13.44|28.01|false|victor hernandez|2013-03-01 09:11:58.703236|12.55|debate +4|433|65730|4294967406|69.36|40.67|true|irene ichabod|2013-03-01 09:11:58.703322|76.97|education +45|308|65770|4294967514|77.88|36.41|true|tom underhill|2013-03-01 09:11:58.703190|48.90|nap time +50|510|65770|4294967426|84.04|35.49|false|irene young|2013-03-01 09:11:58.703303|32.32|xylophone band +28|466|65726|4294967520|63.21|26.26|false|victor king|2013-03-01 09:11:58.703256|93.99|values clariffication +95|323|65701|4294967304|81.69|5.77|true|luke white|2013-03-01 09:11:58.703117|15.87|education +63|399|65651|4294967430|30.71|18.97|false|irene van buren|2013-03-01 09:11:58.703084|69.03|opthamology +64|460|65630|4294967421|0.89|40.24|true|bob king|2013-03-01 09:11:58.703152|50.57|undecided +64|482|65658|4294967525|17.33|21.36|false|fred van buren|2013-03-01 09:11:58.703265|63.12|chemistry +-1|256|65604|4294967443|40.97|49.08|true|fred quirinius|2013-03-01 09:11:58.703280|61.56|opthamology +10|303|65618|4294967517|14.75|1.98|false|xavier ellison|2013-03-01 09:11:58.703248|62.81|zync studies +95|399|65779|4294967541|58.89|7.40|false|alice brown|2013-03-01 09:11:58.703208|55.41|biology +52|335|65556|4294967436|98.71|17.32|true|calvin garcia|2013-03-01 09:11:58.703106|46.14|quiet hour +69|334|65665|4294967384|4.49|14.12|true|nick brown|2013-03-01 09:11:58.703100|6.48|topology +81|407|65537|4294967409|82.09|20.41|false|yuri white|2013-03-01 09:11:58.703118|94.87|philosophy +14|273|65760|4294967341|36.91|18.16|true|calvin ichabod|2013-03-01 09:11:58.703105|18.42|values clariffication +33|263|65729|4294967330|34.56|33.40|true|victor king|2013-03-01 09:11:58.703094|93.64|geology +73|451|65771|4294967423|58.02|39.37|false|ulysses falkner|2013-03-01 09:11:58.703133|80.02|quiet hour +61|333|65691|4294967533|52.83|30.82|true|xavier johnson|2013-03-01 09:11:58.703179|97.69|nap time +35|473|65583|4294967313|12.85|30.02|true|mike ichabod|2013-03-01 09:11:58.703177|32.84|education +44|445|65603|4294967400|78.30|13.62|true|ulysses underhill|2013-03-01 09:11:58.703217|59.06|nap time +85|485|65780|4294967375|43.78|32.60|false|ethan brown|2013-03-01 09:11:58.703225|13.04|quiet hour +54|269|65617|4294967444|43.94|3.68|true|zach king|2013-03-01 09:11:58.703323|7.81|study skills +57|429|65688|4294967361|43.35|42.07|true|quinn zipper|2013-03-01 09:11:58.703272|21.73|joggying +40|454|65738|4294967349|88.99|6.01|true|jessica laertes|2013-03-01 09:11:58.703126|87.17|values clariffication +7|349|65732|4294967434|30.83|25.66|false|nick nixon|2013-03-01 09:11:58.703096|16.92|wind surfing +9|432|65675|4294967454|8.49|20.16|false|sarah nixon|2013-03-01 09:11:58.703100|41.55|philosophy +42|483|65664|4294967324|79.61|39.41|true|irene ichabod|2013-03-01 09:11:58.703125|5.82|industrial engineering +72|356|65692|4294967423|5.35|34.05|false|victor carson|2013-03-01 09:11:58.703076|52.70|linguistics +50|342|65666|4294967397|46.57|10.32|false|oscar falkner|2013-03-01 09:11:58.703212|48.71|values clariffication +92|299|65746|4294967319|84.97|2.04|false|quinn van buren|2013-03-01 09:11:58.703275|12.95|philosophy +40|350|65683|4294967451|0.23|23.62|false|calvin white|2013-03-01 09:11:58.703319|65.06|wind surfing +41|481|65626|4294967430|76.51|45.35|true|sarah allen|2013-03-01 09:11:58.703222|14.55|chemistry +65|309|65576|4294967541|12.34|18.18|false|luke robinson|2013-03-01 09:11:58.703260|58.81|linguistics +34|325|65778|4294967327|81.27|15.89|false|mike xylophone|2013-03-01 09:11:58.703075|21.69|zync studies +102|392|65709|4294967506|66.82|48.61|true|david ovid|2013-03-01 09:11:58.703165|52.83|religion +30|347|65768|4294967333|36.53|34.87|false|mike davidson|2013-03-01 09:11:58.703115|75.53|debate +-2|483|65603|4294967313|51.80|20.31|true|yuri hernandez|2013-03-01 09:11:58.703257|42.71|yard duty +78|496|65779|4294967445|42.85|5.43|true|luke hernandez|2013-03-01 09:11:58.703161|63.64|kindergarten +24|414|65546|4294967368|33.37|43.55|false|ulysses king|2013-03-01 09:11:58.703125|46.78|undecided +100|305|65593|4294967419|34.99|2.84|true|zach miller|2013-03-01 09:11:58.703262|63.21|debate +55|341|65569|4294967546|65.23|34.72|true|luke ellison|2013-03-01 09:11:58.703193|43.18|forestry +50|351|65757|4294967454|79.85|18.09|true|calvin ellison|2013-03-01 09:11:58.703207|41.88|religion +5|280|65780|4294967438|65.12|17.43|false|luke garcia|2013-03-01 09:11:58.703091|45.32|education +108|342|65671|4294967406|60.49|19.66|false|irene white|2013-03-01 09:11:58.703312|71.79|history +99|265|65627|4294967378|88.19|42.12|true|tom white|2013-03-01 09:11:58.703300|74.60|yard duty +14|316|65626|4294967440|52.66|4.50|true|xavier steinbeck|2013-03-01 09:11:58.703234|80.83|education +51|276|65643|4294967498|47.86|1.94|false|wendy ichabod|2013-03-01 09:11:58.703266|43.80|topology +28|286|65734|4294967481|19.40|32.87|true|luke laertes|2013-03-01 09:11:58.703215|30.43|mathematics +12|428|65733|4294967342|77.90|23.24|false|victor carson|2013-03-01 09:11:58.703278|65.62|xylophone band +120|428|65715|4294967335|40.54|32.26|false|sarah falkner|2013-03-01 09:11:58.703070|71.35|wind surfing +34|510|65647|4294967317|93.48|2.02|false|luke ellison|2013-03-01 09:11:58.703184|49.13|wind surfing +111|299|65613|4294967438|56.80|13.56|false|victor ovid|2013-03-01 09:11:58.703300|88.23|mathematics +79|287|65564|4294967370|95.76|43.12|false|calvin young|2013-03-01 09:11:58.703203|41.88|mathematics +98|504|65728|4294967327|43.01|17.56|false|priscilla quirinius|2013-03-01 09:11:58.703073|99.35|industrial engineering +87|474|65788|4294967513|72.73|35.96|false|jessica young|2013-03-01 09:11:58.703170|17.19|study skills +27|406|65542|4294967549|31.90|2.28|false|mike brown|2013-03-01 09:11:58.703120|31.03|zync studies +6|381|65548|4294967477|79.19|20.59|false|sarah allen|2013-03-01 09:11:58.703308|95.20|forestry +47|258|65717|4294967429|88.43|40.91|true|nick king|2013-03-01 09:11:58.703187|35.85|quiet hour +7|506|65745|4294967458|30.84|9.36|true|calvin hernandez|2013-03-01 09:11:58.703146|21.61|yard duty +112|401|65702|4294967447|84.84|26.78|true|oscar thompson|2013-03-01 09:11:58.703321|70.32|values clariffication +42|498|65637|4294967404|21.35|33.44|true|jessica garcia|2013-03-01 09:11:58.703182|10.65|kindergarten +74|491|65782|4294967394|49.64|25.68|true|holly thompson|2013-03-01 09:11:58.703071|91.12|forestry +63|510|65682|4294967304|67.11|29.82|true|tom van buren|2013-03-01 09:11:58.703219|88.31|wind surfing +66|502|65721|4294967324|57.13|26.66|false|luke young|2013-03-01 09:11:58.703074|69.54|industrial engineering +46|373|65683|4294967498|1.99|22.40|true|xavier johnson|2013-03-01 09:11:58.703071|48.08|undecided +4|392|65629|4294967457|91.00|20.73|true|katie quirinius|2013-03-01 09:11:58.703308|60.60|study skills +50|346|65778|4294967447|84.11|31.62|false|xavier ovid|2013-03-01 09:11:58.703114|54.42|forestry +91|414|65665|4294967315|52.83|32.33|true|nick king|2013-03-01 09:11:58.703104|97.86|joggying +69|442|65715|4294967482|37.81|19.53|true|tom king|2013-03-01 09:11:58.703107|99.57|study skills +100|404|65672|4294967348|96.67|40.69|false|wendy zipper|2013-03-01 09:11:58.703168|15.42|debate +78|472|65750|4294967531|37.56|42.30|true|bob ovid|2013-03-01 09:11:58.703242|35.80|industrial engineering +9|368|65785|4294967458|52.45|8.29|false|jessica polk|2013-03-01 09:11:58.703132|26.85|debate +120|489|65754|4294967315|45.68|39.07|true|quinn garcia|2013-03-01 09:11:58.703266|69.26|nap time +96|433|65582|4294967534|54.49|30.28|false|david johnson|2013-03-01 09:11:58.703223|15.22|values clariffication +95|448|65720|4294967397|42.51|35.44|false|yuri ellison|2013-03-01 09:11:58.703308|12.08|debate +9|371|65661|4294967305|16.91|33.06|true|katie johnson|2013-03-01 09:11:58.703130|10.11|kindergarten +61|508|65638|4294967442|37.12|31.33|false|nick ovid|2013-03-01 09:11:58.703085|47.02|joggying +6|428|65674|4294967454|95.39|31.23|false|quinn thompson|2013-03-01 09:11:58.703304|45.96|biology +76|418|65706|4294967329|16.88|26.49|true|irene thompson|2013-03-01 09:11:58.703161|63.15|quiet hour +18|361|65755|4294967442|43.38|0.94|false|ulysses white|2013-03-01 09:11:58.703223|11.24|history +77|448|65651|4294967370|87.35|12.85|true|oscar ellison|2013-03-01 09:11:58.703296|72.58|religion +55|408|65628|4294967365|12.63|43.11|true|luke robinson|2013-03-01 09:11:58.703207|10.56|nap time +89|496|65742|4294967331|68.17|31.01|false|oscar king|2013-03-01 09:11:58.703213|16.18|kindergarten +19|425|65759|4294967515|60.36|42.87|true|mike king|2013-03-01 09:11:58.703301|70.29|geology +16|479|65727|4294967440|47.30|48.11|false|irene johnson|2013-03-01 09:11:58.703160|11.40|yard duty +62|437|65546|4294967436|10.65|31.77|false|nick king|2013-03-01 09:11:58.703162|24.99|study skills +62|475|65561|4294967511|54.09|17.04|false|david allen|2013-03-01 09:11:58.703105|87.06|geology +30|337|65682|4294967495|74.01|26.40|false|luke johnson|2013-03-01 09:11:58.703222|23.94|biology +39|386|65704|4294967474|91.91|21.21|false|ulysses van buren|2013-03-01 09:11:58.703220|88.93|xylophone band +45|304|65765|4294967501|58.12|1.50|true|ethan hernandez|2013-03-01 09:11:58.703157|97.59|xylophone band +81|297|65716|4294967385|73.14|46.13|true|priscilla steinbeck|2013-03-01 09:11:58.703235|25.31|industrial engineering +88|501|65735|4294967431|2.78|7.31|false|fred ichabod|2013-03-01 09:11:58.703230|19.64|zync studies +47|344|65545|4294967549|23.15|47.90|true|alice ichabod|2013-03-01 09:11:58.703271|54.68|forestry +4|304|65700|4294967366|44.78|9.75|true|jessica xylophone|2013-03-01 09:11:58.703115|94.39|wind surfing +46|500|65634|4294967348|65.87|9.70|false|luke ovid|2013-03-01 09:11:58.703113|96.45|opthamology +34|371|65694|4294967415|62.58|17.73|true|victor davidson|2013-03-01 09:11:58.703306|89.05|nap time +2|279|65558|4294967524|70.99|41.12|true|david van buren|2013-03-01 09:11:58.703100|72.59|religion +74|463|65669|4294967396|66.63|15.58|false|zach ovid|2013-03-01 09:11:58.703145|18.12|kindergarten +1|389|65699|4294967412|32.70|30.11|true|alice falkner|2013-03-01 09:11:58.703313|60.70|kindergarten +16|389|65584|4294967500|89.47|38.30|true|luke robinson|2013-03-01 09:11:58.703262|70.81|zync studies +71|464|65673|4294967378|60.84|5.61|false|calvin falkner|2013-03-01 09:11:58.703098|59.17|topology +95|476|65726|4294967332|89.19|15.53|true|jessica zipper|2013-03-01 09:11:58.703188|53.65|linguistics +14|452|65688|4294967363|99.26|8.34|false|victor hernandez|2013-03-01 09:11:58.703072|32.20|yard duty +55|318|65657|4294967379|12.08|14.30|true|tom king|2013-03-01 09:11:58.703175|98.06|quiet hour +104|420|65725|4294967401|40.67|49.67|true|oscar van buren|2013-03-01 09:11:58.703187|15.90|opthamology +117|400|65558|4294967299|63.35|33.08|false|victor robinson|2013-03-01 09:11:58.703187|57.58|education +78|483|65734|4294967415|84.95|12.52|false|alice xylophone|2013-03-01 09:11:58.703255|27.53|topology +12|403|65586|4294967468|10.81|38.20|false|luke underhill|2013-03-01 09:11:58.703286|90.83|zync studies +120|425|65677|4294967370|86.28|14.73|true|alice young|2013-03-01 09:11:58.703198|27.43|philosophy +45|409|65766|4294967300|93.42|40.56|true|katie allen|2013-03-01 09:11:58.703228|94.82|debate +16|328|65619|4294967487|55.71|14.23|true|quinn white|2013-03-01 09:11:58.703274|37.60|quiet hour +46|391|65611|4294967504|25.94|33.61|true|wendy miller|2013-03-01 09:11:58.703099|18.64|study skills +81|285|65539|4294967456|28.04|35.69|true|luke davidson|2013-03-01 09:11:58.703275|21.99|education +5|276|65743|4294967466|82.12|4.51|true|oscar ichabod|2013-03-01 09:11:58.703137|49.67|undecided +50|473|65607|4294967378|66.80|41.86|true|xavier young|2013-03-01 09:11:58.703098|81.99|zync studies +8|439|65547|4294967504|1.76|38.10|true|priscilla allen|2013-03-01 09:11:58.703286|23.42|chemistry +78|303|65603|4294967302|67.38|7.70|true|irene thompson|2013-03-01 09:11:58.703195|15.47|linguistics +101|357|65754|4294967304|1.86|8.42|true|priscilla van buren|2013-03-01 09:11:58.703127|64.43|xylophone band +22|296|65785|4294967386|26.46|32.84|false|ulysses ellison|2013-03-01 09:11:58.703152|29.48|forestry +124|325|65578|4294967306|78.82|44.81|false|david thompson|2013-03-01 09:11:58.703322|18.06|linguistics +23|503|65583|4294967427|76.59|47.53|true|irene zipper|2013-03-01 09:11:58.703167|85.83|topology +31|271|65774|4294967363|67.23|18.86|false|yuri brown|2013-03-01 09:11:58.703086|83.00|industrial engineering +49|302|65714|4294967409|36.27|29.15|true|yuri polk|2013-03-01 09:11:58.703138|94.04|quiet hour +41|365|65637|4294967377|71.41|25.37|false|rachel quirinius|2013-03-01 09:11:58.703230|0.83|philosophy +39|369|65721|4294967490|0.07|49.88|false|calvin quirinius|2013-03-01 09:11:58.703323|49.05|industrial engineering +4|486|65710|4294967309|8.02|23.91|true|yuri young|2013-03-01 09:11:58.703236|2.37|opthamology +21|391|65566|4294967538|89.21|22.89|true|priscilla polk|2013-03-01 09:11:58.703201|40.04|debate +74|323|65617|4294967496|92.43|19.03|false|ethan polk|2013-03-01 09:11:58.703232|50.90|joggying +68|392|65752|4294967469|0.48|2.04|true|rachel ichabod|2013-03-01 09:11:58.703072|33.41|mathematics +29|321|65604|4294967316|67.66|34.85|false|wendy young|2013-03-01 09:11:58.703114|59.69|quiet hour +18|355|65582|4294967543|31.06|21.29|true|mike johnson|2013-03-01 09:11:58.703223|55.81|industrial engineering +6|344|65650|4294967386|12.63|35.49|false|victor thompson|2013-03-01 09:11:58.703160|95.11|debate +83|391|65689|4294967450|18.22|32.18|false|katie davidson|2013-03-01 09:11:58.703275|73.28|undecided +99|284|65694|4294967396|76.38|5.14|true|ulysses xylophone|2013-03-01 09:11:58.703291|7.67|topology +21|415|65648|4294967351|16.19|12.90|true|xavier brown|2013-03-01 09:11:58.703284|79.41|xylophone band +96|338|65771|4294967333|66.51|31.44|true|oscar xylophone|2013-03-01 09:11:58.703247|61.36|biology +119|396|65735|4294967355|37.50|29.58|false|rachel garcia|2013-03-01 09:11:58.703319|69.17|opthamology +76|309|65573|4294967334|74.01|32.61|false|ethan van buren|2013-03-01 09:11:58.703287|55.94|linguistics +26|443|65544|4294967343|13.52|29.45|true|tom young|2013-03-01 09:11:58.703246|85.76|geology +107|439|65593|4294967348|41.79|33.12|true|quinn thompson|2013-03-01 09:11:58.703129|58.79|geology +46|443|65557|4294967523|77.35|37.29|true|tom ichabod|2013-03-01 09:11:58.703118|85.34|debate +69|417|65789|4294967378|81.79|23.68|false|tom carson|2013-03-01 09:11:58.703073|81.30|values clariffication +29|415|65734|4294967361|43.49|10.04|false|irene miller|2013-03-01 09:11:58.703097|68.89|industrial engineering +46|361|65750|4294967387|14.78|40.08|false|fred allen|2013-03-01 09:11:58.703146|49.92|linguistics +28|346|65745|4294967547|51.04|30.25|true|ethan thompson|2013-03-01 09:11:58.703235|81.58|study skills +51|356|65600|4294967346|47.94|10.95|false|fred young|2013-03-01 09:11:58.703269|34.45|education +23|344|65558|4294967468|51.94|37.19|true|tom carson|2013-03-01 09:11:58.703162|78.55|industrial engineering +26|412|65718|4294967337|18.53|7.40|false|alice ichabod|2013-03-01 09:11:58.703087|5.12|debate +107|284|65673|4294967440|62.14|4.60|false|gabriella ellison|2013-03-01 09:11:58.703319|0.51|yard duty +76|263|65750|4294967499|16.51|27.60|true|irene king|2013-03-01 09:11:58.703112|95.90|xylophone band +68|345|65718|4294967449|51.75|26.38|false|calvin ellison|2013-03-01 09:11:58.703122|94.92|debate +57|328|65783|4294967334|56.06|16.45|true|katie miller|2013-03-01 09:11:58.703106|54.50|zync studies +62|310|65572|4294967405|30.70|16.17|true|wendy falkner|2013-03-01 09:11:58.703182|57.01|wind surfing +41|306|65752|4294967439|94.23|5.09|true|nick falkner|2013-03-01 09:11:58.703204|58.97|biology +104|468|65648|4294967446|88.11|13.60|true|mike laertes|2013-03-01 09:11:58.703180|50.60|yard duty +75|342|65595|4294967511|95.68|6.58|false|wendy king|2013-03-01 09:11:58.703121|77.63|joggying +86|259|65623|4294967339|67.81|18.71|false|ulysses garcia|2013-03-01 09:11:58.703245|1.25|values clariffication +29|463|65696|4294967534|14.78|22.65|false|sarah polk|2013-03-01 09:11:58.703281|42.42|debate +38|311|65612|4294967346|8.19|47.58|true|victor underhill|2013-03-01 09:11:58.703120|33.72|values clariffication +32|273|65671|4294967379|10.46|2.82|true|ethan king|2013-03-01 09:11:58.703231|96.06|chemistry +43|348|65672|4294967316|41.93|15.01|false|rachel johnson|2013-03-01 09:11:58.703198|51.68|xylophone band +121|314|65767|4294967456|25.38|49.92|false|zach young|2013-03-01 09:11:58.703180|60.62|mathematics +81|495|65783|4294967469|41.27|6.21|false|mike garcia|2013-03-01 09:11:58.703221|31.29|joggying +47|407|65683|4294967375|49.17|36.83|false|luke van buren|2013-03-01 09:11:58.703139|59.24|opthamology +82|496|65649|4294967521|5.37|46.54|true|bob robinson|2013-03-01 09:11:58.703123|19.64|yard duty +122|374|65703|4294967496|44.96|36.37|true|yuri ichabod|2013-03-01 09:11:58.703234|3.79|xylophone band +7|498|65611|4294967507|19.92|18.85|false|sarah xylophone|2013-03-01 09:11:58.703169|96.63|history +2|342|65776|4294967529|50.53|21.36|false|ethan garcia|2013-03-01 09:11:58.703234|38.67|history +56|293|65565|4294967447|74.29|9.52|false|tom xylophone|2013-03-01 09:11:58.703258|71.94|undecided +91|383|65694|4294967365|68.83|13.73|false|alice van buren|2013-03-01 09:11:58.703207|6.29|wind surfing +84|443|65593|4294967424|98.37|38.89|false|fred carson|2013-03-01 09:11:58.703317|49.99|linguistics +39|463|65647|4294967466|38.09|0.37|true|gabriella carson|2013-03-01 09:11:58.703156|70.82|industrial engineering +72|356|65564|4294967323|80.54|48.18|false|holly hernandez|2013-03-01 09:11:58.703199|2.93|quiet hour +0|287|65684|4294967505|19.28|40.19|false|mike falkner|2013-03-01 09:11:58.703288|5.47|nap time +81|418|65675|4294967409|48.65|23.45|false|victor johnson|2013-03-01 09:11:58.703265|46.23|wind surfing +4|294|65703|4294967415|57.70|10.00|false|victor johnson|2013-03-01 09:11:58.703170|13.97|religion +76|477|65789|4294967350|38.71|38.27|true|luke polk|2013-03-01 09:11:58.703218|12.65|study skills +112|494|65752|4294967495|66.74|27.49|false|luke underhill|2013-03-01 09:11:58.703306|18.94|xylophone band +81|477|65577|4294967369|40.64|39.53|true|quinn ichabod|2013-03-01 09:11:58.703283|68.68|education +5|325|65682|4294967409|44.80|47.79|false|victor young|2013-03-01 09:11:58.703288|9.10|mathematics +77|273|65639|4294967338|96.72|36.25|true|gabriella garcia|2013-03-01 09:11:58.703286|12.31|xylophone band +90|339|65685|4294967468|71.19|4.66|false|quinn ellison|2013-03-01 09:11:58.703310|93.00|opthamology +68|386|65771|4294967548|36.65|28.40|true|zach ichabod|2013-03-01 09:11:58.703155|33.03|values clariffication +52|398|65678|4294967550|33.58|10.01|false|gabriella falkner|2013-03-01 09:11:58.703292|60.20|zync studies +18|449|65641|4294967309|63.46|32.52|true|priscilla thompson|2013-03-01 09:11:58.703085|3.84|industrial engineering +22|444|65760|4294967503|18.32|34.64|false|jessica brown|2013-03-01 09:11:58.703082|99.87|education +44|331|65571|4294967303|56.51|22.65|false|mike underhill|2013-03-01 09:11:58.703325|10.33|opthamology +3|358|65609|4294967405|45.09|19.72|true|alice nixon|2013-03-01 09:11:58.703317|74.17|quiet hour +65|258|65752|4294967429|31.58|8.97|true|priscilla zipper|2013-03-01 09:11:58.703093|4.87|wind surfing +34|346|65715|4294967467|50.18|12.49|false|katie young|2013-03-01 09:11:58.703076|86.52|topology +4|273|65619|4294967325|94.09|22.68|true|holly van buren|2013-03-01 09:11:58.703309|3.01|biology +8|279|65630|4294967299|16.49|30.41|true|gabriella zipper|2013-03-01 09:11:58.703171|25.57|debate +118|420|65750|4294967465|39.04|14.21|true|luke polk|2013-03-01 09:11:58.703242|15.54|topology +72|283|65580|4294967508|14.78|36.34|true|sarah thompson|2013-03-01 09:11:58.703217|2.90|american history +56|287|65747|4294967360|90.86|1.94|true|alice davidson|2013-03-01 09:11:58.703155|32.97|history +62|463|65701|4294967380|58.47|2.68|true|fred xylophone|2013-03-01 09:11:58.703246|36.38|values clariffication +35|291|65740|4294967317|37.62|31.85|true|sarah ellison|2013-03-01 09:11:58.703198|48.84|american history +47|431|65736|4294967444|84.34|4.82|true|ulysses zipper|2013-03-01 09:11:58.703192|62.14|undecided +82|494|65617|4294967393|97.71|22.84|false|sarah garcia|2013-03-01 09:11:58.703294|25.71|yard duty +67|444|65678|4294967355|53.00|14.07|false|david brown|2013-03-01 09:11:58.703271|86.14|history +62|451|65664|4294967410|61.82|30.07|true|luke young|2013-03-01 09:11:58.703200|70.78|xylophone band +73|461|65650|4294967442|16.83|7.27|false|mike brown|2013-03-01 09:11:58.703104|10.60|zync studies +45|483|65639|4294967548|80.55|7.58|false|fred davidson|2013-03-01 09:11:58.703317|99.44|values clariffication +110|397|65781|4294967512|28.50|24.21|true|xavier white|2013-03-01 09:11:58.703176|92.61|opthamology +92|436|65549|4294967316|16.50|42.99|true|tom johnson|2013-03-01 09:11:58.703220|21.94|topology +19|270|65689|4294967326|60.33|35.29|true|wendy nixon|2013-03-01 09:11:58.703129|10.41|history +42|262|65734|4294967387|50.79|20.85|false|ethan davidson|2013-03-01 09:11:58.703083|93.19|debate +112|468|65611|4294967389|74.54|30.62|true|katie zipper|2013-03-01 09:11:58.703216|28.15|linguistics +119|309|65765|4294967387|26.56|46.89|true|mike ovid|2013-03-01 09:11:58.703117|16.87|wind surfing +100|265|65705|4294967546|24.81|6.95|true|david xylophone|2013-03-01 09:11:58.703219|62.35|opthamology +96|505|65689|4294967399|26.37|33.53|true|luke davidson|2013-03-01 09:11:58.703236|92.92|debate +25|279|65550|4294967451|83.21|12.26|false|katie falkner|2013-03-01 09:11:58.703135|62.16|chemistry +64|297|65566|4294967550|17.17|7.03|false|fred robinson|2013-03-01 09:11:58.703192|42.93|kindergarten +3|403|65688|4294967485|71.07|7.30|false|holly underhill|2013-03-01 09:11:58.703162|21.62|xylophone band +11|387|65581|4294967348|54.44|49.50|false|jessica ellison|2013-03-01 09:11:58.703305|39.56|quiet hour +63|382|65764|4294967442|29.18|21.08|false|tom young|2013-03-01 09:11:58.703288|44.77|mathematics +122|405|65726|4294967411|17.71|32.90|true|tom white|2013-03-01 09:11:58.703084|4.82|study skills +115|317|65642|4294967341|91.30|11.49|false|oscar white|2013-03-01 09:11:58.703133|84.45|forestry +1|382|65547|4294967471|51.23|39.65|true|sarah davidson|2013-03-01 09:11:58.703266|28.04|wind surfing +11|387|65744|4294967419|25.64|43.85|false|quinn hernandez|2013-03-01 09:11:58.703267|71.94|values clariffication +5|494|65724|4294967314|7.92|42.76|true|xavier van buren|2013-03-01 09:11:58.703158|86.50|study skills +118|382|65739|4294967333|80.92|19.11|true|priscilla ovid|2013-03-01 09:11:58.703154|48.12|nap time +28|265|65779|4294967336|40.28|40.08|false|quinn falkner|2013-03-01 09:11:58.703307|31.24|industrial engineering +3|511|65617|4294967303|18.31|19.29|true|luke davidson|2013-03-01 09:11:58.703135|97.35|philosophy +72|324|65770|4294967477|49.61|0.30|true|ethan miller|2013-03-01 09:11:58.703121|26.96|debate +122|297|65594|4294967532|26.40|40.97|false|ulysses young|2013-03-01 09:11:58.703224|72.16|yard duty +38|314|65665|4294967432|94.70|33.76|false|nick allen|2013-03-01 09:11:58.703199|7.82|history +69|374|65655|4294967500|4.36|7.99|false|katie van buren|2013-03-01 09:11:58.703198|81.18|debate +114|405|65551|4294967521|72.18|49.26|false|david laertes|2013-03-01 09:11:58.703162|31.16|nap time +41|266|65619|4294967310|58.47|42.11|false|nick young|2013-03-01 09:11:58.703202|93.57|values clariffication +3|446|65636|4294967421|14.37|47.47|false|ethan steinbeck|2013-03-01 09:11:58.703283|77.99|wind surfing +11|409|65577|4294967330|88.63|25.03|false|david johnson|2013-03-01 09:11:58.703183|91.10|religion +121|485|65738|4294967485|5.47|33.88|false|wendy brown|2013-03-01 09:11:58.703257|62.84|undecided +16|470|65590|4294967318|45.55|7.19|true|xavier ovid|2013-03-01 09:11:58.703313|73.64|chemistry +86|285|65637|4294967532|93.95|8.49|true|xavier polk|2013-03-01 09:11:58.703253|76.20|opthamology +49|321|65579|4294967525|14.15|7.81|false|oscar polk|2013-03-01 09:11:58.703293|95.62|linguistics +6|494|65562|4294967412|78.14|5.85|true|katie xylophone|2013-03-01 09:11:58.703243|90.51|linguistics +90|470|65568|4294967491|26.87|6.96|false|priscilla king|2013-03-01 09:11:58.703179|79.17|kindergarten +29|405|65713|4294967397|62.53|17.62|false|katie brown|2013-03-01 09:11:58.703177|14.59|quiet hour +25|355|65707|4294967509|29.00|3.98|true|katie thompson|2013-03-01 09:11:58.703267|15.20|linguistics +7|371|65690|4294967480|25.13|30.88|true|priscilla van buren|2013-03-01 09:11:58.703279|83.24|history +82|474|65704|4294967450|9.92|7.06|true|gabriella ellison|2013-03-01 09:11:58.703218|25.91|religion +89|446|65573|4294967426|0.88|8.13|false|jessica ovid|2013-03-01 09:11:58.703297|16.40|values clariffication +84|458|65623|4294967449|76.75|27.01|true|luke ovid|2013-03-01 09:11:58.703315|21.09|industrial engineering +76|396|65748|4294967421|72.28|20.46|true|katie ellison|2013-03-01 09:11:58.703311|77.61|kindergarten +105|373|65603|4294967524|81.85|22.83|true|tom allen|2013-03-01 09:11:58.703102|36.77|wind surfing +90|360|65671|4294967375|66.95|22.37|true|sarah falkner|2013-03-01 09:11:58.703107|3.18|industrial engineering +61|400|65609|4294967443|22.49|39.35|false|gabriella laertes|2013-03-01 09:11:58.703202|97.42|american history +40|391|65590|4294967352|91.79|26.82|true|jessica underhill|2013-03-01 09:11:58.703091|62.43|values clariffication +44|391|65785|4294967415|9.42|36.38|false|xavier davidson|2013-03-01 09:11:58.703289|92.73|wind surfing +26|361|65771|4294967356|85.72|0.26|true|alice king|2013-03-01 09:11:58.703278|44.83|quiet hour +53|413|65609|4294967306|51.64|35.53|true|priscilla laertes|2013-03-01 09:11:58.703211|6.57|education +54|294|65667|4294967328|46.63|1.82|false|priscilla zipper|2013-03-01 09:11:58.703293|52.59|yard duty +50|400|65641|4294967538|27.16|4.58|true|xavier van buren|2013-03-01 09:11:58.703176|79.04|chemistry +4|339|65725|4294967343|84.12|36.75|false|irene underhill|2013-03-01 09:11:58.703120|6.42|opthamology +72|434|65590|4294967396|3.52|10.37|true|mike nixon|2013-03-01 09:11:58.703125|23.14|chemistry +8|332|65758|4294967376|67.40|32.65|true|mike allen|2013-03-01 09:11:58.703251|48.63|philosophy +41|354|65682|4294967344|38.22|49.23|true|quinn polk|2013-03-01 09:11:58.703287|54.48|geology +63|306|65566|4294967464|76.87|27.32|true|holly laertes|2013-03-01 09:11:58.703175|50.84|education +82|444|65679|4294967394|26.97|9.50|false|sarah robinson|2013-03-01 09:11:58.703103|26.74|wind surfing +29|373|65726|4294967491|29.93|30.18|true|sarah quirinius|2013-03-01 09:11:58.703154|3.46|topology +73|282|65785|4294967326|77.70|33.04|true|rachel underhill|2013-03-01 09:11:58.703129|96.50|philosophy +43|482|65685|4294967346|34.18|33.56|true|priscilla robinson|2013-03-01 09:11:58.703183|81.23|biology +30|379|65656|4294967458|69.98|22.44|true|ethan ellison|2013-03-01 09:11:58.703219|68.84|geology +19|396|65637|4294967423|27.01|7.83|false|ethan brown|2013-03-01 09:11:58.703292|42.68|undecided +21|382|65564|4294967371|12.91|37.78|true|zach ellison|2013-03-01 09:11:58.703289|99.86|values clariffication +88|403|65602|4294967401|87.10|13.05|true|gabriella ichabod|2013-03-01 09:11:58.703118|32.10|religion +71|432|65698|4294967527|40.49|40.42|true|oscar laertes|2013-03-01 09:11:58.703103|53.91|xylophone band +71|486|65712|4294967417|52.60|1.67|true|irene quirinius|2013-03-01 09:11:58.703187|10.12|zync studies +111|299|65624|4294967501|80.23|23.16|true|alice nixon|2013-03-01 09:11:58.703259|31.56|biology +119|387|65556|4294967399|51.43|20.38|false|irene miller|2013-03-01 09:11:58.703275|43.33|yard duty +46|369|65540|4294967499|0.52|14.11|false|alice ovid|2013-03-01 09:11:58.703207|0.06|linguistics +21|380|65627|4294967540|41.87|44.75|true|tom miller|2013-03-01 09:11:58.703312|34.88|nap time +47|301|65700|4294967374|83.21|7.69|true|wendy quirinius|2013-03-01 09:11:58.703190|69.76|forestry +68|276|65707|4294967400|76.04|48.78|true|luke young|2013-03-01 09:11:58.703302|75.71|zync studies +121|323|65612|4294967544|80.63|8.63|false|calvin steinbeck|2013-03-01 09:11:58.703095|69.47|mathematics +23|451|65546|4294967416|35.50|21.34|false|zach xylophone|2013-03-01 09:11:58.703158|41.57|philosophy +115|492|65745|4294967404|1.53|25.32|false|jessica quirinius|2013-03-01 09:11:58.703243|79.49|xylophone band +34|445|65775|4294967511|46.36|43.96|true|nick steinbeck|2013-03-01 09:11:58.703208|52.41|mathematics +63|286|65627|4294967550|26.91|15.77|true|mike johnson|2013-03-01 09:11:58.703286|83.79|education +114|413|65710|4294967449|61.41|39.10|true|yuri hernandez|2013-03-01 09:11:58.703071|50.06|biology +39|392|65579|4294967481|85.27|23.94|true|fred young|2013-03-01 09:11:58.703217|2.84|linguistics +47|376|65666|4294967352|99.09|34.71|false|xavier white|2013-03-01 09:11:58.703288|34.98|american history +0|386|65630|4294967396|17.59|46.00|true|holly steinbeck|2013-03-01 09:11:58.703113|64.76|industrial engineering +57|325|65576|4294967331|52.59|19.78|true|mike laertes|2013-03-01 09:11:58.703251|36.33|nap time +58|280|65703|4294967548|34.35|34.54|false|ethan carson|2013-03-01 09:11:58.703176|66.33|geology +85|440|65677|4294967436|28.82|36.99|false|jessica nixon|2013-03-01 09:11:58.703309|68.08|chemistry +97|311|65581|4294967443|11.39|35.10|true|david underhill|2013-03-01 09:11:58.703281|65.32|american history +58|336|65589|4294967417|44.00|21.51|false|zach nixon|2013-03-01 09:11:58.703092|50.86|education +44|338|65551|4294967433|87.52|2.23|true|gabriella polk|2013-03-01 09:11:58.703280|33.27|quiet hour +69|505|65697|4294967332|92.53|0.89|false|holly davidson|2013-03-01 09:11:58.703147|51.95|kindergarten +9|365|65570|4294967462|14.59|12.94|false|jessica thompson|2013-03-01 09:11:58.703172|81.39|education +8|257|65781|4294967331|17.33|18.39|true|alice underhill|2013-03-01 09:11:58.703108|16.06|values clariffication +66|417|65727|4294967517|70.71|32.74|false|ethan young|2013-03-01 09:11:58.703131|53.79|mathematics +48|392|65691|4294967308|42.71|32.25|true|luke carson|2013-03-01 09:11:58.703300|26.41|study skills +61|320|65664|4294967383|79.21|44.34|true|tom johnson|2013-03-01 09:11:58.703241|59.56|topology +73|404|65727|4294967535|86.56|25.24|false|ethan underhill|2013-03-01 09:11:58.703266|49.21|undecided +69|434|65769|4294967476|79.14|20.83|true|victor underhill|2013-03-01 09:11:58.703094|90.30|values clariffication +115|262|65735|4294967411|45.10|33.29|true|sarah laertes|2013-03-01 09:11:58.703238|75.61|forestry +1|490|65560|4294967422|69.09|14.57|false|tom polk|2013-03-01 09:11:58.703186|41.02|biology +98|336|65782|4294967314|43.00|18.33|false|xavier ichabod|2013-03-01 09:11:58.703173|82.39|yard duty +74|485|65666|4294967395|92.98|39.00|false|fred ellison|2013-03-01 09:11:58.703155|21.08|topology +52|452|65722|4294967492|61.82|19.16|false|alice white|2013-03-01 09:11:58.703322|87.30|joggying +-3|289|65757|4294967528|56.20|44.24|true|luke ichabod|2013-03-01 09:11:58.703294|6.79|yard duty +69|362|65550|4294967341|3.01|15.03|false|jessica carson|2013-03-01 09:11:58.703195|75.97|philosophy +62|329|65601|4294967337|51.08|28.02|true|gabriella underhill|2013-03-01 09:11:58.703179|49.82|kindergarten +118|271|65688|4294967385|5.76|29.59|false|xavier van buren|2013-03-01 09:11:58.703250|25.80|opthamology +53|358|65651|4294967411|34.20|31.27|false|david thompson|2013-03-01 09:11:58.703104|14.63|linguistics +60|279|65556|4294967349|26.78|38.01|true|oscar nixon|2013-03-01 09:11:58.703107|46.94|philosophy +106|510|65618|4294967462|32.38|42.81|false|luke young|2013-03-01 09:11:58.703078|73.70|mathematics +90|486|65725|4294967526|97.12|17.66|true|alice garcia|2013-03-01 09:11:58.703147|2.48|education +27|321|65541|4294967476|61.39|29.78|true|quinn laertes|2013-03-01 09:11:58.703131|42.85|debate +50|349|65605|4294967434|1.56|2.09|false|victor ichabod|2013-03-01 09:11:58.703128|58.84|industrial engineering +96|290|65709|4294967507|19.66|19.93|false|gabriella van buren|2013-03-01 09:11:58.703302|87.89|nap time +96|473|65790|4294967328|77.90|9.91|true|fred underhill|2013-03-01 09:11:58.703211|52.84|philosophy +57|442|65759|4294967464|70.11|38.37|true|quinn thompson|2013-03-01 09:11:58.703134|81.14|values clariffication +35|371|65689|4294967500|7.68|24.29|false|fred quirinius|2013-03-01 09:11:58.703286|44.52|values clariffication +94|431|65582|4294967312|45.17|45.39|true|luke xylophone|2013-03-01 09:11:58.703262|29.15|kindergarten +8|496|65791|4294967346|65.31|44.58|true|victor davidson|2013-03-01 09:11:58.703286|54.19|quiet hour +75|393|65699|4294967316|4.77|32.73|false|bob quirinius|2013-03-01 09:11:58.703325|33.84|joggying +89|449|65618|4294967343|10.74|20.21|true|wendy davidson|2013-03-01 09:11:58.703294|58.42|nap time +58|497|65754|4294967326|42.77|14.00|true|luke steinbeck|2013-03-01 09:11:58.703084|86.18|joggying +116|477|65618|4294967546|49.81|7.48|false|quinn robinson|2013-03-01 09:11:58.703263|42.82|geology +66|447|65546|4294967462|75.57|40.18|true|calvin carson|2013-03-01 09:11:58.703161|25.64|opthamology +123|476|65761|4294967406|48.30|12.29|false|katie steinbeck|2013-03-01 09:11:58.703097|79.63|american history +60|357|65787|4294967349|56.31|33.94|true|oscar underhill|2013-03-01 09:11:58.703112|29.65|history +50|481|65616|4294967351|68.76|47.23|true|oscar ellison|2013-03-01 09:11:58.703262|14.69|geology +63|497|65677|4294967307|40.05|15.85|false|tom carson|2013-03-01 09:11:58.703200|25.23|history +71|267|65736|4294967432|4.41|38.03|true|yuri thompson|2013-03-01 09:11:58.703318|12.51|geology +50|316|65778|4294967412|78.61|42.84|false|bob polk|2013-03-01 09:11:58.703107|39.40|industrial engineering +123|294|65770|4294967305|11.50|14.10|true|victor thompson|2013-03-01 09:11:58.703080|18.66|education +75|292|65650|4294967408|49.53|43.24|true|tom ichabod|2013-03-01 09:11:58.703274|26.89|xylophone band +50|291|65598|4294967543|90.44|8.82|false|mike van buren|2013-03-01 09:11:58.703099|56.04|philosophy +88|466|65774|4294967395|20.86|13.95|false|zach laertes|2013-03-01 09:11:58.703095|31.79|american history +97|363|65775|4294967426|36.70|29.54|false|luke hernandez|2013-03-01 09:11:58.703308|76.26|joggying +20|507|65538|4294967339|22.61|43.91|false|victor garcia|2013-03-01 09:11:58.703089|55.41|wind surfing +36|475|65715|4294967307|75.43|32.38|false|quinn thompson|2013-03-01 09:11:58.703114|38.60|history +46|459|65770|4294967366|37.04|43.71|false|yuri polk|2013-03-01 09:11:58.703083|14.69|chemistry +107|329|65643|4294967319|7.86|5.51|true|calvin laertes|2013-03-01 09:11:58.703134|86.82|wind surfing +51|500|65785|4294967437|32.79|43.84|false|yuri polk|2013-03-01 09:11:58.703241|84.46|geology +40|479|65684|4294967433|34.47|24.46|true|xavier quirinius|2013-03-01 09:11:58.703210|39.63|education +22|421|65692|4294967417|52.77|24.47|true|jessica underhill|2013-03-01 09:11:58.703233|23.11|values clariffication +97|378|65574|4294967356|77.82|45.51|true|sarah nixon|2013-03-01 09:11:58.703236|2.15|linguistics +16|424|65647|4294967312|22.38|6.48|false|irene van buren|2013-03-01 09:11:58.703162|83.75|kindergarten +33|306|65751|4294967508|8.30|47.84|true|yuri brown|2013-03-01 09:11:58.703079|44.53|forestry +6|510|65590|4294967314|73.49|18.39|false|tom johnson|2013-03-01 09:11:58.703076|12.92|philosophy +10|470|65657|4294967419|78.12|21.72|false|priscilla davidson|2013-03-01 09:11:58.703303|79.69|philosophy +20|350|65728|4294967396|79.73|4.80|false|holly miller|2013-03-01 09:11:58.703081|35.16|values clariffication +15|455|65580|4294967327|92.64|29.25|true|nick hernandez|2013-03-01 09:11:58.703218|4.47|wind surfing +114|468|65583|4294967422|71.35|11.46|false|calvin davidson|2013-03-01 09:11:58.703320|15.65|linguistics +82|508|65770|4294967486|94.12|34.98|false|mike brown|2013-03-01 09:11:58.703195|48.02|philosophy +11|365|65600|4294967304|41.95|17.46|false|yuri hernandez|2013-03-01 09:11:58.703293|83.03|history +56|326|65778|4294967436|0.54|17.32|true|oscar white|2013-03-01 09:11:58.703216|97.42|american history +2|376|65634|4294967525|75.50|38.65|false|victor zipper|2013-03-01 09:11:58.703311|81.96|philosophy +98|474|65600|4294967480|31.66|27.06|true|gabriella king|2013-03-01 09:11:58.703320|43.81|study skills +36|305|65727|4294967425|10.19|46.47|true|katie miller|2013-03-01 09:11:58.703100|9.01|mathematics +1|364|65589|4294967336|66.09|7.05|false|quinn ovid|2013-03-01 09:11:58.703167|12.04|linguistics +104|263|65606|4294967412|15.38|34.08|true|victor young|2013-03-01 09:11:58.703194|66.49|study skills +99|381|65569|4294967441|6.17|26.91|true|zach carson|2013-03-01 09:11:58.703128|4.21|nap time +47|410|65574|4294967376|89.64|16.33|false|quinn allen|2013-03-01 09:11:58.703089|46.42|biology +7|392|65637|4294967330|82.92|18.94|false|luke steinbeck|2013-03-01 09:11:58.703095|0.75|kindergarten +47|415|65550|4294967527|91.70|22.20|false|zach ichabod|2013-03-01 09:11:58.703216|31.96|religion +48|492|65608|4294967364|10.13|25.33|true|ulysses underhill|2013-03-01 09:11:58.703129|23.60|study skills +74|399|65708|4294967352|91.31|27.08|true|sarah garcia|2013-03-01 09:11:58.703237|74.69|philosophy +13|487|65664|4294967488|95.77|33.48|true|yuri king|2013-03-01 09:11:58.703241|73.66|religion +27|406|65773|4294967539|74.60|29.01|true|bob laertes|2013-03-01 09:11:58.703190|75.14|chemistry +114|507|65779|4294967391|39.61|34.93|true|calvin hernandez|2013-03-01 09:11:58.703104|8.69|opthamology +89|441|65655|4294967422|57.72|46.25|true|mike zipper|2013-03-01 09:11:58.703219|58.83|yard duty +1|338|65759|4294967515|88.55|43.84|false|holly king|2013-03-01 09:11:58.703290|29.65|american history +55|379|65745|4294967393|53.58|4.13|false|xavier van buren|2013-03-01 09:11:58.703134|2.48|mathematics +38|315|65571|4294967299|85.91|4.59|false|wendy nixon|2013-03-01 09:11:58.703285|91.00|mathematics +59|350|65698|4294967338|99.58|37.22|true|rachel young|2013-03-01 09:11:58.703146|24.34|religion +122|480|65776|4294967539|37.19|2.58|true|luke polk|2013-03-01 09:11:58.703273|46.19|biology +92|417|65609|4294967297|89.02|7.81|false|sarah van buren|2013-03-01 09:11:58.703120|8.04|history +83|405|65713|4294967388|23.05|20.36|false|alice ellison|2013-03-01 09:11:58.703115|80.68|forestry +90|367|65790|4294967425|55.67|33.81|false|yuri ellison|2013-03-01 09:11:58.703318|82.74|forestry +49|441|65650|4294967335|20.74|0.30|false|irene falkner|2013-03-01 09:11:58.703312|74.41|biology +21|364|65662|4294967494|39.62|28.84|false|tom falkner|2013-03-01 09:11:58.703220|31.66|opthamology +39|299|65649|4294967542|46.94|37.07|true|ethan garcia|2013-03-01 09:11:58.703216|70.63|xylophone band +73|496|65546|4294967339|12.86|33.70|false|priscilla hernandez|2013-03-01 09:11:58.703133|47.91|quiet hour +3|403|65562|4294967439|8.95|26.97|true|calvin zipper|2013-03-01 09:11:58.703290|10.83|forestry +108|450|65628|4294967469|65.61|12.21|true|yuri laertes|2013-03-01 09:11:58.703231|99.85|zync studies +-1|378|65747|4294967448|47.27|47.55|true|katie white|2013-03-01 09:11:58.703232|24.97|biology +78|333|65764|4294967500|48.54|45.61|false|quinn miller|2013-03-01 09:11:58.703236|8.55|quiet hour +91|270|65698|4294967509|69.77|28.87|true|gabriella brown|2013-03-01 09:11:58.703324|42.22|religion +74|310|65622|4294967361|32.15|37.59|false|yuri underhill|2013-03-01 09:11:58.703223|77.31|education +86|506|65724|4294967447|17.75|8.76|true|holly zipper|2013-03-01 09:11:58.703117|10.77|wind surfing +64|352|65617|4294967301|82.51|37.60|true|quinn quirinius|2013-03-01 09:11:58.703257|46.93|yard duty +66|300|65702|4294967307|65.14|36.79|true|mike quirinius|2013-03-01 09:11:58.703168|85.22|philosophy +34|477|65655|4294967527|62.68|2.52|true|victor miller|2013-03-01 09:11:58.703161|81.78|biology +99|414|65677|4294967493|35.74|6.63|true|luke thompson|2013-03-01 09:11:58.703224|57.03|values clariffication +99|285|65544|4294967320|50.38|46.60|true|priscilla king|2013-03-01 09:11:58.703165|19.24|undecided +45|390|65771|4294967310|28.38|3.69|true|zach quirinius|2013-03-01 09:11:58.703110|67.02|history +105|491|65682|4294967354|16.72|26.38|true|mike hernandez|2013-03-01 09:11:58.703287|39.92|yard duty +29|497|65691|4294967400|66.99|4.04|true|alice hernandez|2013-03-01 09:11:58.703190|50.00|education +114|307|65756|4294967464|35.68|21.55|false|ulysses nixon|2013-03-01 09:11:58.703298|37.69|linguistics +27|274|65749|4294967296|41.54|2.15|false|jessica steinbeck|2013-03-01 09:11:58.703318|18.68|quiet hour +69|380|65741|4294967327|91.81|3.91|false|calvin nixon|2013-03-01 09:11:58.703143|86.92|joggying +33|450|65650|4294967534|76.49|28.65|true|mike nixon|2013-03-01 09:11:58.703315|98.68|study skills +41|296|65654|4294967379|22.74|35.11|true|victor young|2013-03-01 09:11:58.703278|19.63|nap time +74|389|65591|4294967461|56.17|44.97|true|david xylophone|2013-03-01 09:11:58.703302|77.64|debate +50|297|65759|4294967400|74.10|49.50|true|quinn king|2013-03-01 09:11:58.703081|37.70|religion +11|402|65725|4294967347|51.14|20.08|true|gabriella allen|2013-03-01 09:11:58.703181|29.57|quiet hour +79|419|65732|4294967335|57.51|11.39|false|fred quirinius|2013-03-01 09:11:58.703190|93.96|wind surfing +90|419|65735|4294967469|25.68|4.31|true|xavier johnson|2013-03-01 09:11:58.703209|20.68|history +86|337|65709|4294967329|87.33|35.86|false|gabriella van buren|2013-03-01 09:11:58.703208|82.36|education +107|312|65714|4294967456|65.27|18.07|false|victor ichabod|2013-03-01 09:11:58.703114|94.63|joggying +82|504|65607|4294967328|28.26|29.72|true|mike young|2013-03-01 09:11:58.703267|80.00|geology +106|286|65743|4294967350|42.84|17.53|true|victor allen|2013-03-01 09:11:58.703084|74.13|biology +116|310|65588|4294967518|14.74|43.49|false|mike ichabod|2013-03-01 09:11:58.703222|83.52|joggying +43|354|65597|4294967415|44.65|36.18|false|calvin thompson|2013-03-01 09:11:58.703117|1.43|chemistry +50|303|65751|4294967343|3.57|26.92|true|wendy garcia|2013-03-01 09:11:58.703289|79.09|history +66|456|65782|4294967314|0.68|30.38|true|yuri brown|2013-03-01 09:11:58.703230|72.50|forestry +2|332|65756|4294967474|93.43|49.44|false|irene garcia|2013-03-01 09:11:58.703085|78.48|history +78|510|65646|4294967299|10.10|27.68|false|mike falkner|2013-03-01 09:11:58.703217|99.53|values clariffication +7|269|65695|4294967435|89.34|32.85|false|holly white|2013-03-01 09:11:58.703180|44.19|philosophy +81|305|65593|4294967450|20.16|40.49|true|mike carson|2013-03-01 09:11:58.703226|62.26|opthamology +6|367|65690|4294967446|5.99|46.66|false|katie ellison|2013-03-01 09:11:58.703242|13.78|chemistry +39|423|65740|4294967440|6.34|13.02|false|alice davidson|2013-03-01 09:11:58.703306|81.98|topology +47|479|65628|4294967364|92.30|37.05|true|xavier polk|2013-03-01 09:11:58.703323|17.60|debate +92|261|65632|4294967323|12.98|16.82|false|holly brown|2013-03-01 09:11:58.703096|72.10|chemistry +10|456|65691|4294967407|18.80|47.75|true|bob carson|2013-03-01 09:11:58.703191|64.98|religion +30|406|65691|4294967395|47.08|0.84|false|wendy miller|2013-03-01 09:11:58.703205|85.94|xylophone band +31|272|65566|4294967371|1.86|46.07|true|priscilla king|2013-03-01 09:11:58.703139|52.17|opthamology +94|441|65684|4294967336|72.89|33.19|false|alice hernandez|2013-03-01 09:11:58.703302|96.24|study skills +8|410|65607|4294967297|41.74|39.09|false|katie xylophone|2013-03-01 09:11:58.703126|19.32|quiet hour +114|452|65588|4294967301|32.16|46.67|false|xavier allen|2013-03-01 09:11:58.703237|1.55|forestry +38|449|65567|4294967415|27.53|4.16|false|xavier ellison|2013-03-01 09:11:58.703224|56.18|education +53|354|65768|4294967384|13.84|16.39|false|mike johnson|2013-03-01 09:11:58.703089|62.21|study skills +85|339|65743|4294967406|8.87|14.30|false|wendy nixon|2013-03-01 09:11:58.703233|24.05|yard duty +75|305|65791|4294967467|17.58|6.21|true|xavier miller|2013-03-01 09:11:58.703322|22.80|zync studies +113|507|65635|4294967351|54.03|12.26|false|sarah ovid|2013-03-01 09:11:58.703075|27.05|biology +106|446|65762|4294967365|98.41|22.06|false|quinn ovid|2013-03-01 09:11:58.703197|79.65|topology +56|385|65713|4294967324|4.70|28.11|true|calvin ichabod|2013-03-01 09:11:58.703181|28.17|education +59|396|65708|4294967434|44.30|2.66|true|zach young|2013-03-01 09:11:58.703071|99.58|american history +70|289|65601|4294967536|1.10|42.66|true|oscar young|2013-03-01 09:11:58.703147|64.24|wind surfing +86|457|65648|4294967497|24.71|13.83|false|wendy zipper|2013-03-01 09:11:58.703123|91.58|geology +112|400|65740|4294967327|85.64|24.36|false|calvin steinbeck|2013-03-01 09:11:58.703181|70.31|education +92|409|65728|4294967378|56.72|39.71|false|alice johnson|2013-03-01 09:11:58.703320|74.76|yard duty +13|449|65542|4294967370|0.27|9.42|false|wendy polk|2013-03-01 09:11:58.703137|76.52|history +73|372|65738|4294967503|65.31|33.82|false|nick zipper|2013-03-01 09:11:58.703164|74.59|joggying +17|375|65786|4294967420|71.28|4.38|true|nick ellison|2013-03-01 09:11:58.703132|90.38|quiet hour +90|296|65670|4294967354|66.56|4.76|false|fred brown|2013-03-01 09:11:58.703259|15.42|religion +97|503|65609|4294967511|25.54|16.89|false|gabriella hernandez|2013-03-01 09:11:58.703076|23.79|chemistry +49|324|65573|4294967358|76.40|15.20|false|gabriella falkner|2013-03-01 09:11:58.703108|59.49|biology +65|434|65607|4294967531|5.05|14.42|false|ethan allen|2013-03-01 09:11:58.703145|67.90|philosophy +80|363|65547|4294967353|43.21|19.00|true|fred johnson|2013-03-01 09:11:58.703202|72.09|philosophy +7|485|65704|4294967514|40.20|21.36|true|jessica davidson|2013-03-01 09:11:58.703090|58.79|geology +23|256|65615|4294967499|20.66|49.68|true|victor johnson|2013-03-01 09:11:58.703303|68.95|philosophy +123|302|65601|4294967523|8.98|43.39|false|yuri carson|2013-03-01 09:11:58.703128|14.66|debate +22|260|65708|4294967415|36.14|3.80|true|david johnson|2013-03-01 09:11:58.703176|24.98|opthamology +46|272|65539|4294967337|46.84|3.27|true|katie van buren|2013-03-01 09:11:58.703149|58.85|xylophone band +70|425|65662|4294967488|72.09|35.77|false|oscar ovid|2013-03-01 09:11:58.703186|93.41|zync studies +33|269|65774|4294967324|81.99|34.57|true|bob johnson|2013-03-01 09:11:58.703302|74.36|topology +38|282|65615|4294967542|60.51|45.27|false|calvin steinbeck|2013-03-01 09:11:58.703168|1.39|industrial engineering +43|401|65655|4294967392|52.67|10.00|true|katie hernandez|2013-03-01 09:11:58.703160|62.95|debate +33|361|65788|4294967539|96.24|45.70|false|mike xylophone|2013-03-01 09:11:58.703119|52.02|study skills +71|353|65635|4294967507|16.48|27.51|true|luke polk|2013-03-01 09:11:58.703206|7.12|education +100|326|65652|4294967302|1.64|1.22|false|calvin laertes|2013-03-01 09:11:58.703249|72.95|yard duty +82|370|65690|4294967524|87.29|41.62|false|zach falkner|2013-03-01 09:11:58.703312|54.31|philosophy +64|304|65652|4294967444|24.17|11.62|true|katie van buren|2013-03-01 09:11:58.703212|9.88|biology +47|322|65569|4294967419|32.81|8.89|true|holly xylophone|2013-03-01 09:11:58.703246|61.52|linguistics +74|310|65673|4294967317|19.28|7.44|true|tom falkner|2013-03-01 09:11:58.703162|66.98|xylophone band +76|422|65734|4294967405|34.92|14.58|true|gabriella ichabod|2013-03-01 09:11:58.703102|53.77|quiet hour +33|302|65541|4294967428|63.52|44.31|false|priscilla xylophone|2013-03-01 09:11:58.703323|48.17|education +62|482|65681|4294967458|56.14|13.50|true|alice ellison|2013-03-01 09:11:58.703133|61.85|study skills +111|328|65615|4294967505|57.53|2.63|false|ethan underhill|2013-03-01 09:11:58.703292|50.78|american history +104|371|65660|4294967320|68.40|0.87|true|nick robinson|2013-03-01 09:11:58.703184|14.19|industrial engineering +64|363|65643|4294967370|14.94|9.78|false|luke laertes|2013-03-01 09:11:58.703141|8.62|philosophy +7|388|65622|4294967354|85.47|28.95|false|victor brown|2013-03-01 09:11:58.703189|63.40|chemistry +46|386|65580|4294967495|91.25|3.73|true|irene underhill|2013-03-01 09:11:58.703157|72.91|debate +112|320|65564|4294967351|96.78|2.50|true|victor thompson|2013-03-01 09:11:58.703215|31.01|opthamology +22|268|65753|4294967536|87.48|1.97|false|nick ichabod|2013-03-01 09:11:58.703110|58.81|debate +123|376|65623|4294967440|29.22|20.61|false|xavier brown|2013-03-01 09:11:58.703262|53.50|geology +44|421|65729|4294967523|64.84|26.16|true|priscilla davidson|2013-03-01 09:11:58.703306|70.38|chemistry +68|273|65668|4294967386|63.91|33.90|false|bob nixon|2013-03-01 09:11:58.703295|9.43|kindergarten +43|305|65621|4294967386|31.02|12.13|false|rachel ellison|2013-03-01 09:11:58.703166|49.81|undecided +44|378|65695|4294967447|7.25|5.15|false|ethan polk|2013-03-01 09:11:58.703291|0.96|yard duty +63|333|65559|4294967429|4.28|5.60|false|zach davidson|2013-03-01 09:11:58.703294|75.82|chemistry +24|313|65621|4294967376|59.94|47.02|false|sarah ellison|2013-03-01 09:11:58.703079|53.13|forestry +107|353|65715|4294967503|29.29|7.91|false|luke steinbeck|2013-03-01 09:11:58.703254|62.56|values clariffication +84|286|65671|4294967388|43.62|14.13|false|oscar miller|2013-03-01 09:11:58.703213|81.81|nap time +46|327|65595|4294967312|8.19|40.79|true|oscar steinbeck|2013-03-01 09:11:58.703304|83.49|religion +54|507|65769|4294967319|32.86|46.77|false|irene miller|2013-03-01 09:11:58.703208|65.08|mathematics +38|490|65669|4294967514|21.76|47.81|false|oscar carson|2013-03-01 09:11:58.703224|64.60|study skills +101|302|65743|4294967491|0.02|1.24|false|fred zipper|2013-03-01 09:11:58.703312|33.88|values clariffication +14|411|65575|4294967328|48.89|39.13|false|mike underhill|2013-03-01 09:11:58.703225|58.34|forestry +116|290|65600|4294967542|12.35|6.64|true|victor steinbeck|2013-03-01 09:11:58.703263|75.46|study skills +58|453|65684|4294967472|47.79|19.04|false|rachel zipper|2013-03-01 09:11:58.703144|23.53|education +16|271|65665|4294967519|24.19|9.45|false|gabriella garcia|2013-03-01 09:11:58.703092|18.11|mathematics +3|509|65601|4294967505|39.22|9.45|true|katie polk|2013-03-01 09:11:58.703299|96.76|zync studies +10|472|65644|4294967363|82.76|7.04|true|ethan van buren|2013-03-01 09:11:58.703325|36.96|linguistics +8|457|65722|4294967489|31.35|49.36|true|calvin xylophone|2013-03-01 09:11:58.703283|48.77|undecided +61|420|65746|4294967436|13.14|26.73|true|oscar brown|2013-03-01 09:11:58.703210|89.26|kindergarten +81|398|65752|4294967493|46.04|8.71|false|mike davidson|2013-03-01 09:11:58.703101|73.37|philosophy +31|306|65601|4294967429|67.65|31.26|false|calvin ellison|2013-03-01 09:11:58.703216|52.84|topology +102|327|65593|4294967342|49.77|6.02|true|quinn ichabod|2013-03-01 09:11:58.703137|57.95|study skills +47|314|65639|4294967327|15.65|21.63|false|wendy carson|2013-03-01 09:11:58.703315|3.73|quiet hour +88|295|65638|4294967502|56.26|12.09|false|xavier nixon|2013-03-01 09:11:58.703081|32.55|xylophone band +10|326|65618|4294967540|12.78|16.53|true|gabriella ichabod|2013-03-01 09:11:58.703308|27.75|yard duty +10|360|65757|4294967351|54.78|7.40|false|ulysses king|2013-03-01 09:11:58.703248|89.67|quiet hour +1|384|65691|4294967363|23.82|12.34|true|oscar zipper|2013-03-01 09:11:58.703286|17.29|history +25|419|65661|4294967408|26.23|38.15|false|nick quirinius|2013-03-01 09:11:58.703259|8.88|linguistics +82|438|65664|4294967442|79.70|20.16|false|david ovid|2013-03-01 09:11:58.703146|56.74|quiet hour +71|461|65671|4294967522|39.19|35.38|false|jessica young|2013-03-01 09:11:58.703145|8.86|topology +1|388|65719|4294967405|28.05|35.50|true|holly falkner|2013-03-01 09:11:58.703124|50.39|wind surfing +37|472|65699|4294967452|70.38|23.58|true|david ichabod|2013-03-01 09:11:58.703184|85.40|wind surfing +60|274|65698|4294967381|87.13|23.20|false|rachel johnson|2013-03-01 09:11:58.703170|92.59|biology +101|344|65783|4294967420|67.71|49.03|true|ethan quirinius|2013-03-01 09:11:58.703143|76.33|quiet hour +91|328|65601|4294967401|35.25|43.48|false|ulysses falkner|2013-03-01 09:11:58.703238|23.85|philosophy +95|301|65679|4294967380|69.97|4.18|true|yuri johnson|2013-03-01 09:11:58.703238|23.58|zync studies +33|369|65556|4294967325|62.16|6.62|true|victor laertes|2013-03-01 09:11:58.703120|56.74|chemistry +47|294|65768|4294967453|15.07|12.84|false|oscar ichabod|2013-03-01 09:11:58.703209|79.08|linguistics +68|357|65732|4294967549|74.85|21.26|false|luke ovid|2013-03-01 09:11:58.703081|92.57|linguistics +26|270|65661|4294967336|7.77|21.55|false|victor nixon|2013-03-01 09:11:58.703080|24.24|xylophone band +30|295|65572|4294967418|88.39|47.36|false|bob quirinius|2013-03-01 09:11:58.703284|84.05|american history +45|272|65753|4294967519|80.01|14.82|true|zach steinbeck|2013-03-01 09:11:58.703156|42.70|industrial engineering +108|448|65726|4294967447|42.11|26.74|true|bob young|2013-03-01 09:11:58.703314|30.10|history +75|416|65680|4294967468|59.57|37.31|false|holly polk|2013-03-01 09:11:58.703082|3.13|religion +35|380|65722|4294967401|63.18|44.41|false|rachel van buren|2013-03-01 09:11:58.703073|16.57|opthamology +6|323|65564|4294967414|68.32|12.79|false|quinn brown|2013-03-01 09:11:58.703135|72.41|industrial engineering +27|354|65688|4294967426|53.53|20.56|true|priscilla hernandez|2013-03-01 09:11:58.703193|41.75|quiet hour +26|371|65663|4294967385|98.39|49.05|true|holly ovid|2013-03-01 09:11:58.703126|90.31|kindergarten +34|444|65705|4294967453|1.02|47.11|true|sarah steinbeck|2013-03-01 09:11:58.703216|86.65|opthamology +29|458|65785|4294967363|63.33|9.40|true|david robinson|2013-03-01 09:11:58.703202|94.26|american history +8|477|65744|4294967339|13.77|14.30|false|ethan nixon|2013-03-01 09:11:58.703093|8.96|zync studies +74|292|65789|4294967430|92.21|15.09|true|victor polk|2013-03-01 09:11:58.703215|87.24|religion +53|283|65675|4294967541|45.20|20.96|true|nick polk|2013-03-01 09:11:58.703107|54.53|study skills +61|274|65542|4294967339|27.38|18.79|false|wendy young|2013-03-01 09:11:58.703085|92.94|yard duty +10|477|65722|4294967373|18.84|28.11|false|alice falkner|2013-03-01 09:11:58.703213|92.71|philosophy +117|286|65682|4294967389|1.05|7.12|false|rachel garcia|2013-03-01 09:11:58.703221|40.70|religion +41|304|65556|4294967345|74.88|48.05|true|yuri miller|2013-03-01 09:11:58.703323|90.74|forestry +30|455|65543|4294967479|2.31|0.76|true|quinn young|2013-03-01 09:11:58.703116|67.65|education +107|466|65669|4294967349|45.07|11.52|true|alice quirinius|2013-03-01 09:11:58.703141|48.64|joggying +97|495|65764|4294967547|4.42|22.76|false|katie ovid|2013-03-01 09:11:58.703258|17.29|study skills +24|268|65596|4294967314|19.69|22.35|true|sarah thompson|2013-03-01 09:11:58.703165|18.42|undecided +86|397|65769|4294967377|2.66|0.31|true|sarah davidson|2013-03-01 09:11:58.703223|52.63|study skills +73|447|65629|4294967395|98.57|43.81|false|victor carson|2013-03-01 09:11:58.703180|31.70|philosophy +47|486|65695|4294967337|14.72|2.25|true|ethan allen|2013-03-01 09:11:58.703173|87.72|philosophy +56|299|65711|4294967392|18.87|12.22|true|sarah van buren|2013-03-01 09:11:58.703121|55.92|yard duty +67|382|65718|4294967455|19.72|43.16|true|nick steinbeck|2013-03-01 09:11:58.703125|73.64|topology +15|438|65766|4294967335|24.63|42.35|true|jessica zipper|2013-03-01 09:11:58.703263|93.01|study skills +87|490|65658|4294967381|57.81|6.23|true|nick white|2013-03-01 09:11:58.703269|54.76|education +55|463|65633|4294967458|87.18|25.00|false|rachel carson|2013-03-01 09:11:58.703254|24.84|joggying +65|360|65686|4294967508|79.84|0.55|false|holly king|2013-03-01 09:11:58.703312|37.10|values clariffication +30|376|65640|4294967332|1.90|20.38|true|irene carson|2013-03-01 09:11:58.703323|96.62|philosophy +7|443|65546|4294967471|26.21|23.73|false|xavier allen|2013-03-01 09:11:58.703102|83.86|history +50|370|65596|4294967397|74.62|21.35|true|fred king|2013-03-01 09:11:58.703194|13.80|education +120|306|65589|4294967534|25.43|48.58|false|katie carson|2013-03-01 09:11:58.703171|73.21|linguistics +1|325|65614|4294967488|70.74|47.86|false|irene thompson|2013-03-01 09:11:58.703118|9.02|debate +10|284|65628|4294967468|36.17|42.53|false|priscilla ichabod|2013-03-01 09:11:58.703153|20.37|quiet hour +61|363|65588|4294967408|32.14|39.13|false|wendy miller|2013-03-01 09:11:58.703301|13.06|mathematics +83|495|65563|4294967370|11.93|10.44|true|ulysses garcia|2013-03-01 09:11:58.703139|79.58|opthamology +8|477|65780|4294967526|70.42|31.48|false|luke zipper|2013-03-01 09:11:58.703185|39.07|linguistics +12|389|65674|4294967354|44.18|25.85|false|irene hernandez|2013-03-01 09:11:58.703250|2.51|debate +78|499|65617|4294967364|20.30|49.32|false|bob ellison|2013-03-01 09:11:58.703094|60.38|wind surfing +48|371|65765|4294967537|39.16|31.96|false|david allen|2013-03-01 09:11:58.703264|36.91|chemistry +85|307|65754|4294967485|84.89|42.77|true|david ellison|2013-03-01 09:11:58.703280|31.02|kindergarten +48|355|65746|4294967407|3.92|2.56|true|rachel robinson|2013-03-01 09:11:58.703217|3.84|mathematics +88|289|65636|4294967470|23.16|29.03|true|irene xylophone|2013-03-01 09:11:58.703289|63.23|joggying +21|393|65624|4294967447|60.30|3.68|true|katie ellison|2013-03-01 09:11:58.703152|93.09|religion +-2|346|65698|4294967318|21.56|35.32|false|sarah steinbeck|2013-03-01 09:11:58.703169|38.48|chemistry +117|271|65734|4294967430|69.50|39.99|false|gabriella underhill|2013-03-01 09:11:58.703312|64.40|joggying +8|395|65659|4294967403|54.43|48.91|false|david zipper|2013-03-01 09:11:58.703305|87.67|wind surfing +65|262|65726|4294967337|37.09|4.93|true|calvin brown|2013-03-01 09:11:58.703214|37.35|study skills +22|419|65679|4294967500|72.81|4.94|true|priscilla garcia|2013-03-01 09:11:58.703276|32.02|values clariffication +54|496|65716|4294967395|75.81|31.28|false|david garcia|2013-03-01 09:11:58.703152|94.44|history +79|421|65634|4294967311|67.99|35.04|false|quinn zipper|2013-03-01 09:11:58.703296|29.09|undecided +31|359|65557|4294967392|34.34|8.03|true|zach davidson|2013-03-01 09:11:58.703119|10.81|wind surfing +116|259|65627|4294967384|34.79|8.88|false|nick polk|2013-03-01 09:11:58.703269|18.08|opthamology +120|382|65622|4294967300|51.87|41.64|true|alice falkner|2013-03-01 09:11:58.703223|59.68|debate +85|266|65693|4294967470|9.88|27.76|true|david polk|2013-03-01 09:11:58.703190|5.00|undecided +55|438|65774|4294967483|72.31|43.28|false|wendy underhill|2013-03-01 09:11:58.703125|24.98|philosophy +87|301|65685|4294967357|83.13|22.93|false|holly underhill|2013-03-01 09:11:58.703141|71.99|values clariffication +11|275|65632|4294967436|48.01|20.25|false|xavier zipper|2013-03-01 09:11:58.703177|82.20|undecided +67|330|65727|4294967303|73.86|22.28|false|bob falkner|2013-03-01 09:11:58.703303|35.20|religion +89|472|65547|4294967506|80.16|37.65|false|zach van buren|2013-03-01 09:11:58.703286|84.69|undecided +114|504|65777|4294967341|30.22|27.50|true|jessica ovid|2013-03-01 09:11:58.703192|12.87|yard duty +121|457|65679|4294967502|82.82|47.25|false|quinn carson|2013-03-01 09:11:58.703213|15.76|philosophy +83|488|65562|4294967333|47.81|38.68|false|wendy ichabod|2013-03-01 09:11:58.703261|99.55|topology +104|399|65573|4294967530|19.04|2.99|false|zach king|2013-03-01 09:11:58.703275|36.95|nap time +64|258|65568|4294967540|13.57|26.49|true|zach steinbeck|2013-03-01 09:11:58.703241|72.99|joggying +15|456|65599|4294967437|40.55|47.57|true|quinn ellison|2013-03-01 09:11:58.703106|79.80|biology +62|418|65540|4294967470|62.72|31.17|false|rachel robinson|2013-03-01 09:11:58.703143|88.46|linguistics +68|457|65624|4294967349|67.92|11.59|false|quinn carson|2013-03-01 09:11:58.703317|92.18|undecided +124|420|65583|4294967443|89.94|15.06|false|zach quirinius|2013-03-01 09:11:58.703250|39.55|zync studies +45|450|65708|4294967502|75.06|15.97|true|alice laertes|2013-03-01 09:11:58.703269|60.82|values clariffication +59|310|65610|4294967405|81.84|9.09|false|gabriella nixon|2013-03-01 09:11:58.703180|86.51|religion +110|260|65568|4294967338|90.76|9.92|true|jessica polk|2013-03-01 09:11:58.703223|89.32|religion +64|423|65661|4294967323|34.80|25.13|false|yuri ichabod|2013-03-01 09:11:58.703248|71.35|values clariffication +106|402|65643|4294967544|0.99|10.18|false|nick xylophone|2013-03-01 09:11:58.703317|15.27|geology +114|293|65680|4294967480|91.65|4.66|false|gabriella nixon|2013-03-01 09:11:58.703261|54.27|philosophy +37|494|65782|4294967513|28.16|33.43|false|calvin polk|2013-03-01 09:11:58.703210|1.29|education +5|342|65730|4294967339|71.79|39.68|false|jessica falkner|2013-03-01 09:11:58.703320|18.38|chemistry +13|291|65731|4294967448|78.09|31.31|false|zach ovid|2013-03-01 09:11:58.703143|33.04|american history +76|332|65700|4294967495|82.17|34.84|false|wendy allen|2013-03-01 09:11:58.703304|98.64|philosophy +102|379|65762|4294967354|82.09|48.14|false|bob robinson|2013-03-01 09:11:58.703110|2.35|opthamology +48|461|65728|4294967326|81.74|29.69|false|quinn king|2013-03-01 09:11:58.703231|96.77|linguistics +8|290|65759|4294967309|19.55|23.59|true|yuri ichabod|2013-03-01 09:11:58.703136|45.24|biology +71|317|65543|4294967326|28.33|17.32|true|fred zipper|2013-03-01 09:11:58.703308|46.99|geology +63|278|65696|4294967494|86.82|7.05|false|victor zipper|2013-03-01 09:11:58.703279|43.71|opthamology +49|436|65677|4294967389|65.83|29.87|true|calvin ellison|2013-03-01 09:11:58.703078|97.03|nap time +82|294|65560|4294967406|80.20|37.60|true|rachel nixon|2013-03-01 09:11:58.703302|85.55|study skills +120|427|65599|4294967451|9.85|13.20|false|irene miller|2013-03-01 09:11:58.703254|65.21|nap time +31|447|65573|4294967516|84.58|21.91|true|irene hernandez|2013-03-01 09:11:58.703170|9.05|yard duty +37|443|65692|4294967383|62.59|31.85|false|zach van buren|2013-03-01 09:11:58.703132|28.33|religion +-2|318|65730|4294967403|5.43|13.93|true|tom ichabod|2013-03-01 09:11:58.703165|38.98|history +22|354|65714|4294967513|31.54|31.06|false|oscar zipper|2013-03-01 09:11:58.703076|84.97|opthamology +100|286|65705|4294967502|49.20|13.53|true|alice young|2013-03-01 09:11:58.703300|46.25|opthamology +42|388|65764|4294967299|43.29|36.20|true|jessica johnson|2013-03-01 09:11:58.703122|26.32|debate +48|292|65598|4294967550|46.74|6.48|false|katie davidson|2013-03-01 09:11:58.703102|37.02|zync studies +16|410|65547|4294967516|31.12|40.17|false|katie ichabod|2013-03-01 09:11:58.703254|30.49|forestry +-3|485|65661|4294967441|26.21|16.19|false|alice xylophone|2013-03-01 09:11:58.703129|96.95|topology +51|472|65757|4294967310|35.32|16.73|true|irene brown|2013-03-01 09:11:58.703071|77.55|values clariffication +18|376|65604|4294967345|74.60|27.44|false|nick brown|2013-03-01 09:11:58.703235|83.19|chemistry +51|410|65657|4294967374|86.83|2.66|false|priscilla king|2013-03-01 09:11:58.703314|59.93|forestry +37|283|65615|4294967393|82.63|24.16|true|ethan johnson|2013-03-01 09:11:58.703273|70.41|education +58|336|65573|4294967305|48.20|13.55|true|wendy thompson|2013-03-01 09:11:58.703273|37.97|history +29|450|65684|4294967368|44.78|8.08|true|fred young|2013-03-01 09:11:58.703222|18.03|yard duty +44|506|65618|4294967494|61.77|47.01|false|oscar zipper|2013-03-01 09:11:58.703255|61.68|mathematics +89|272|65716|4294967533|84.87|30.97|true|nick falkner|2013-03-01 09:11:58.703144|33.03|philosophy +90|459|65544|4294967339|1.82|33.44|false|tom garcia|2013-03-01 09:11:58.703322|48.33|quiet hour +43|300|65685|4294967315|98.76|32.30|true|calvin zipper|2013-03-01 09:11:58.703283|57.38|zync studies +38|256|65744|4294967497|47.17|18.11|false|tom allen|2013-03-01 09:11:58.703264|70.87|biology +73|293|65791|4294967485|48.12|24.61|false|yuri miller|2013-03-01 09:11:58.703103|10.20|chemistry +-2|353|65598|4294967450|81.49|48.77|false|jessica steinbeck|2013-03-01 09:11:58.703299|74.29|quiet hour +90|289|65676|4294967359|4.08|12.26|false|quinn ellison|2013-03-01 09:11:58.703286|63.08|wind surfing +21|473|65541|4294967434|91.68|1.02|true|nick miller|2013-03-01 09:11:58.703307|96.21|philosophy +20|418|65753|4294967549|25.55|14.03|false|xavier falkner|2013-03-01 09:11:58.703090|21.84|wind surfing +99|431|65694|4294967346|69.53|48.70|false|fred ichabod|2013-03-01 09:11:58.703321|23.84|debate +108|330|65639|4294967518|65.78|37.87|true|sarah allen|2013-03-01 09:11:58.703177|30.02|philosophy +51|445|65591|4294967463|28.75|28.64|true|fred steinbeck|2013-03-01 09:11:58.703188|72.43|mathematics +107|329|65550|4294967523|30.68|25.24|true|victor falkner|2013-03-01 09:11:58.703123|98.13|opthamology +96|407|65682|4294967370|27.75|8.89|true|holly falkner|2013-03-01 09:11:58.703124|19.83|linguistics +7|410|65620|4294967440|64.34|44.61|true|holly robinson|2013-03-01 09:11:58.703227|98.38|xylophone band +105|307|65603|4294967374|39.34|45.63|false|fred laertes|2013-03-01 09:11:58.703246|78.62|american history +29|344|65669|4294967355|11.37|17.47|true|rachel hernandez|2013-03-01 09:11:58.703131|46.93|yard duty +110|492|65673|4294967545|30.78|8.37|true|bob quirinius|2013-03-01 09:11:58.703168|30.41|philosophy +42|351|65683|4294967531|1.39|6.31|true|victor quirinius|2013-03-01 09:11:58.703165|25.40|philosophy +11|510|65716|4294967452|11.83|13.53|false|quinn zipper|2013-03-01 09:11:58.703104|87.62|chemistry +99|301|65638|4294967535|18.57|32.51|true|gabriella falkner|2013-03-01 09:11:58.703279|69.42|wind surfing +37|406|65734|4294967486|74.53|34.86|false|ulysses carson|2013-03-01 09:11:58.703151|84.93|education +-3|386|65611|4294967331|58.81|22.43|true|sarah miller|2013-03-01 09:11:58.703160|75.23|mathematics +66|378|65710|4294967320|86.53|4.67|false|victor ichabod|2013-03-01 09:11:58.703110|36.77|industrial engineering +55|458|65744|4294967441|38.51|2.84|true|oscar white|2013-03-01 09:11:58.703156|45.33|forestry +107|287|65577|4294967465|9.60|32.12|true|calvin falkner|2013-03-01 09:11:58.703135|28.31|philosophy +25|374|65605|4294967302|2.18|31.56|false|sarah king|2013-03-01 09:11:58.703268|11.17|biology +90|354|65756|4294967450|96.65|44.66|true|priscilla hernandez|2013-03-01 09:11:58.703192|35.13|zync studies +75|466|65710|4294967340|81.35|6.55|true|rachel ovid|2013-03-01 09:11:58.703159|34.40|quiet hour +68|459|65777|4294967386|74.21|5.76|true|ethan ichabod|2013-03-01 09:11:58.703307|91.30|yard duty +42|379|65745|4294967497|84.31|45.49|true|ulysses underhill|2013-03-01 09:11:58.703171|99.19|american history +22|331|65536|4294967312|43.00|26.00|true|zach hernandez|2013-03-01 09:11:58.703134|47.60|linguistics +43|472|65670|4294967539|44.92|19.84|false|bob white|2013-03-01 09:11:58.703241|65.37|industrial engineering +67|410|65669|4294967413|26.15|15.96|false|katie brown|2013-03-01 09:11:58.703096|79.60|forestry +76|309|65607|4294967366|29.97|10.72|true|zach ovid|2013-03-01 09:11:58.703298|56.59|study skills +109|346|65751|4294967512|64.45|27.86|false|irene miller|2013-03-01 09:11:58.703213|20.97|philosophy +89|262|65630|4294967434|96.75|29.85|false|victor thompson|2013-03-01 09:11:58.703247|9.14|values clariffication +78|468|65719|4294967401|98.51|48.54|true|oscar carson|2013-03-01 09:11:58.703221|9.49|joggying +56|453|65762|4294967451|7.39|16.74|true|victor ichabod|2013-03-01 09:11:58.703142|4.97|joggying +30|483|65638|4294967473|91.28|39.28|false|david ellison|2013-03-01 09:11:58.703195|42.28|debate +57|383|65539|4294967301|42.83|33.77|true|zach ichabod|2013-03-01 09:11:58.703226|57.58|topology +113|266|65578|4294967449|28.72|17.90|true|oscar zipper|2013-03-01 09:11:58.703312|42.74|joggying +112|459|65756|4294967380|17.53|32.84|true|mike underhill|2013-03-01 09:11:58.703106|11.45|forestry +63|273|65777|4294967420|93.56|45.98|true|wendy van buren|2013-03-01 09:11:58.703101|44.65|yard duty +79|256|65616|4294967514|76.38|12.08|false|alice ovid|2013-03-01 09:11:58.703176|54.26|history +42|431|65726|4294967376|35.48|18.25|false|ulysses davidson|2013-03-01 09:11:58.703181|98.38|american history +59|289|65557|4294967434|57.02|17.92|false|wendy ichabod|2013-03-01 09:11:58.703261|26.94|wind surfing +114|455|65571|4294967369|61.58|3.08|true|irene van buren|2013-03-01 09:11:58.703110|0.84|religion +118|410|65615|4294967382|25.13|26.52|true|victor hernandez|2013-03-01 09:11:58.703072|48.94|forestry +69|419|65571|4294967318|43.32|1.57|true|luke hernandez|2013-03-01 09:11:58.703073|44.36|debate +93|289|65713|4294967460|35.54|10.02|false|katie hernandez|2013-03-01 09:11:58.703186|55.78|debate +31|319|65582|4294967396|80.56|28.75|true|tom laertes|2013-03-01 09:11:58.703320|69.78|industrial engineering +64|414|65600|4294967396|40.26|6.06|false|sarah carson|2013-03-01 09:11:58.703256|71.93|chemistry +84|343|65552|4294967353|79.16|23.81|false|wendy steinbeck|2013-03-01 09:11:58.703142|96.53|debate +31|444|65635|4294967458|53.49|14.97|false|rachel king|2013-03-01 09:11:58.703290|15.38|undecided +42|304|65778|4294967403|86.69|16.26|true|ulysses allen|2013-03-01 09:11:58.703155|86.27|undecided +109|257|65771|4294967463|95.58|46.19|true|zach zipper|2013-03-01 09:11:58.703096|50.17|debate +30|373|65695|4294967475|90.83|12.82|false|rachel allen|2013-03-01 09:11:58.703260|77.86|religion +83|473|65636|4294967328|70.95|25.97|true|victor thompson|2013-03-01 09:11:58.703223|70.16|mathematics +55|378|65631|4294967444|16.54|24.36|false|fred king|2013-03-01 09:11:58.703079|9.13|wind surfing +113|468|65735|4294967362|90.18|9.86|false|ethan thompson|2013-03-01 09:11:58.703256|11.12|quiet hour +114|294|65763|4294967489|61.76|24.26|false|luke brown|2013-03-01 09:11:58.703282|15.58|nap time +37|411|65608|4294967418|23.63|18.42|true|fred quirinius|2013-03-01 09:11:58.703285|48.49|opthamology +60|415|65715|4294967479|87.07|18.50|false|david polk|2013-03-01 09:11:58.703138|68.00|biology +44|417|65747|4294967494|72.85|30.43|true|zach hernandez|2013-03-01 09:11:58.703189|46.35|study skills +71|341|65717|4294967485|21.77|14.37|true|quinn davidson|2013-03-01 09:11:58.703190|78.98|nap time +37|405|65569|4294967488|66.69|26.91|false|david thompson|2013-03-01 09:11:58.703283|76.83|nap time +48|387|65600|4294967486|68.36|47.81|true|luke thompson|2013-03-01 09:11:58.703174|74.10|xylophone band +13|446|65602|4294967325|31.89|29.99|false|bob laertes|2013-03-01 09:11:58.703184|30.99|history +122|417|65601|4294967341|78.43|20.50|false|wendy hernandez|2013-03-01 09:11:58.703131|85.55|history +35|280|65723|4294967388|52.44|23.60|true|rachel ichabod|2013-03-01 09:11:58.703072|89.99|industrial engineering +85|318|65754|4294967433|35.93|5.24|true|mike xylophone|2013-03-01 09:11:58.703215|84.96|education +11|495|65588|4294967420|18.31|28.31|true|holly polk|2013-03-01 09:11:58.703202|20.69|kindergarten +75|471|65752|4294967302|18.57|3.46|true|wendy johnson|2013-03-01 09:11:58.703150|28.33|topology +-2|261|65744|4294967452|72.59|29.88|true|ethan falkner|2013-03-01 09:11:58.703211|1.40|chemistry +51|262|65564|4294967412|15.13|31.03|false|yuri polk|2013-03-01 09:11:58.703093|38.73|topology +106|479|65594|4294967463|96.12|30.80|true|oscar quirinius|2013-03-01 09:11:58.703111|72.56|undecided +31|273|65559|4294967448|54.64|17.60|false|katie robinson|2013-03-01 09:11:58.703261|2.07|mathematics +69|488|65657|4294967443|20.99|5.73|false|tom steinbeck|2013-03-01 09:11:58.703181|42.54|industrial engineering +118|358|65571|4294967300|54.39|31.72|false|victor steinbeck|2013-03-01 09:11:58.703274|97.94|wind surfing +108|334|65624|4294967305|37.79|45.89|true|luke young|2013-03-01 09:11:58.703076|61.50|zync studies +119|495|65632|4294967465|15.49|37.52|false|jessica thompson|2013-03-01 09:11:58.703288|30.93|topology +17|317|65624|4294967523|57.78|23.39|false|bob falkner|2013-03-01 09:11:58.703144|5.35|xylophone band +16|345|65649|4294967409|89.62|49.79|false|nick garcia|2013-03-01 09:11:58.703270|68.12|biology +49|261|65719|4294967418|8.44|44.95|true|ethan nixon|2013-03-01 09:11:58.703200|88.50|industrial engineering +83|379|65579|4294967462|6.11|32.56|false|tom thompson|2013-03-01 09:11:58.703248|25.65|linguistics +33|291|65569|4294967416|80.65|44.95|false|ulysses underhill|2013-03-01 09:11:58.703248|46.77|opthamology +14|497|65547|4294967299|86.80|31.43|false|oscar miller|2013-03-01 09:11:58.703294|81.54|topology +43|472|65609|4294967473|7.03|40.40|true|priscilla johnson|2013-03-01 09:11:58.703115|2.36|study skills +68|419|65735|4294967305|86.09|42.77|false|alice young|2013-03-01 09:11:58.703311|22.59|mathematics +121|476|65666|4294967372|34.30|35.47|false|alice ovid|2013-03-01 09:11:58.703236|39.21|joggying +58|431|65564|4294967436|74.87|2.27|true|mike van buren|2013-03-01 09:11:58.703141|61.23|mathematics +18|347|65547|4294967477|57.89|20.02|false|alice ichabod|2013-03-01 09:11:58.703290|81.21|kindergarten +17|367|65652|4294967353|41.28|22.59|false|alice nixon|2013-03-01 09:11:58.703220|82.79|industrial engineering +68|480|65709|4294967459|13.67|6.15|true|fred underhill|2013-03-01 09:11:58.703161|76.09|history +76|434|65681|4294967399|6.07|23.68|true|jessica white|2013-03-01 09:11:58.703283|5.49|quiet hour +15|271|65702|4294967512|71.04|38.06|false|irene robinson|2013-03-01 09:11:58.703247|45.54|industrial engineering +122|338|65538|4294967451|48.28|30.94|false|alice ichabod|2013-03-01 09:11:58.703154|53.35|philosophy +111|366|65759|4294967411|42.11|44.16|true|quinn brown|2013-03-01 09:11:58.703234|66.39|study skills +102|329|65679|4294967495|5.77|14.48|false|ulysses nixon|2013-03-01 09:11:58.703319|58.27|forestry +80|288|65687|4294967548|14.53|26.29|false|zach robinson|2013-03-01 09:11:58.703204|99.00|geology +62|306|65623|4294967456|56.45|8.93|false|xavier garcia|2013-03-01 09:11:58.703219|19.18|chemistry +13|458|65722|4294967508|30.06|48.77|false|katie white|2013-03-01 09:11:58.703209|77.13|zync studies +123|440|65718|4294967368|99.04|35.55|true|bob xylophone|2013-03-01 09:11:58.703174|55.45|zync studies +101|258|65582|4294967355|67.28|0.51|false|david garcia|2013-03-01 09:11:58.703178|18.57|forestry +53|286|65667|4294967527|36.95|48.93|true|quinn steinbeck|2013-03-01 09:11:58.703181|97.78|topology +49|421|65616|4294967469|45.36|5.30|false|calvin ovid|2013-03-01 09:11:58.703310|7.82|joggying +10|283|65665|4294967551|25.81|47.41|false|wendy hernandez|2013-03-01 09:11:58.703309|62.45|nap time +106|393|65786|4294967352|45.74|0.62|true|gabriella steinbeck|2013-03-01 09:11:58.703115|40.58|zync studies +81|444|65613|4294967510|73.45|21.14|false|ethan carson|2013-03-01 09:11:58.703320|54.67|xylophone band +15|339|65575|4294967308|50.83|41.44|false|irene laertes|2013-03-01 09:11:58.703131|5.93|study skills +19|447|65608|4294967432|82.05|13.00|false|ulysses allen|2013-03-01 09:11:58.703078|43.26|values clariffication +27|328|65663|4294967460|22.65|14.90|false|quinn steinbeck|2013-03-01 09:11:58.703080|14.03|education +36|488|65629|4294967372|94.18|36.49|false|oscar allen|2013-03-01 09:11:58.703297|42.35|history +82|469|65698|4294967493|90.10|22.72|false|ethan king|2013-03-01 09:11:58.703205|57.17|study skills +119|363|65707|4294967489|48.84|30.91|false|priscilla laertes|2013-03-01 09:11:58.703101|26.21|history +18|329|65601|4294967480|81.40|1.47|false|rachel underhill|2013-03-01 09:11:58.703104|17.68|mathematics +10|278|65641|4294967343|69.83|36.92|true|oscar quirinius|2013-03-01 09:11:58.703109|90.40|wind surfing +55|475|65675|4294967383|27.54|6.44|false|luke quirinius|2013-03-01 09:11:58.703245|9.82|joggying +79|345|65646|4294967363|96.49|46.27|true|jessica brown|2013-03-01 09:11:58.703226|56.65|linguistics +14|376|65678|4294967495|74.26|32.61|true|fred falkner|2013-03-01 09:11:58.703129|62.53|linguistics +49|360|65552|4294967309|27.00|16.92|true|tom hernandez|2013-03-01 09:11:58.703141|99.62|religion +36|310|65719|4294967343|73.15|35.88|false|quinn white|2013-03-01 09:11:58.703145|16.87|history +82|493|65546|4294967482|75.95|49.39|true|gabriella robinson|2013-03-01 09:11:58.703108|51.88|study skills +67|329|65669|4294967318|77.84|0.04|true|priscilla quirinius|2013-03-01 09:11:58.703291|25.90|zync studies +0|397|65607|4294967316|51.73|42.40|false|xavier johnson|2013-03-01 09:11:58.703195|17.72|american history +120|367|65785|4294967492|48.17|19.20|true|ethan polk|2013-03-01 09:11:58.703194|5.05|education +120|346|65660|4294967463|34.43|34.59|true|wendy brown|2013-03-01 09:11:58.703242|98.39|zync studies +80|380|65614|4294967454|64.16|31.60|true|xavier miller|2013-03-01 09:11:58.703310|88.94|linguistics +83|475|65658|4294967314|82.79|45.74|false|ethan brown|2013-03-01 09:11:58.703228|99.50|geology +24|372|65744|4294967340|72.48|49.80|false|bob brown|2013-03-01 09:11:58.703318|4.78|topology +50|468|65568|4294967371|17.45|4.19|false|gabriella quirinius|2013-03-01 09:11:58.703118|56.86|topology +34|511|65663|4294967410|30.71|32.61|true|oscar carson|2013-03-01 09:11:58.703153|73.75|history +51|316|65756|4294967356|19.28|38.02|false|priscilla ovid|2013-03-01 09:11:58.703269|79.14|biology +63|394|65699|4294967318|79.36|37.51|true|xavier van buren|2013-03-01 09:11:58.703288|99.98|biology +60|499|65674|4294967459|44.41|15.07|false|ethan van buren|2013-03-01 09:11:58.703167|64.91|religion +29|339|65556|4294967519|57.78|27.36|true|mike allen|2013-03-01 09:11:58.703102|1.78|study skills +91|463|65641|4294967543|92.21|42.43|true|oscar steinbeck|2013-03-01 09:11:58.703079|7.36|geology +56|260|65630|4294967390|88.79|32.44|true|quinn polk|2013-03-01 09:11:58.703088|81.60|chemistry +16|351|65730|4294967491|69.81|27.80|false|wendy king|2013-03-01 09:11:58.703185|53.14|study skills +5|503|65682|4294967429|86.57|38.88|false|ethan miller|2013-03-01 09:11:58.703209|19.93|quiet hour +111|371|65711|4294967523|74.58|14.80|true|jessica ichabod|2013-03-01 09:11:58.703219|31.22|american history +108|351|65737|4294967449|94.88|35.76|false|oscar ellison|2013-03-01 09:11:58.703201|14.16|history +58|263|65614|4294967297|67.18|30.87|true|calvin thompson|2013-03-01 09:11:58.703289|14.35|wind surfing +86|443|65571|4294967299|36.22|7.54|true|luke robinson|2013-03-01 09:11:58.703220|65.82|kindergarten +99|385|65623|4294967506|29.70|30.14|true|holly hernandez|2013-03-01 09:11:58.703125|19.19|religion +13|456|65781|4294967371|48.47|4.42|true|mike white|2013-03-01 09:11:58.703108|48.93|debate +99|345|65765|4294967470|97.85|24.77|false|calvin hernandez|2013-03-01 09:11:58.703265|91.43|xylophone band +63|256|65737|4294967541|3.38|11.59|false|fred hernandez|2013-03-01 09:11:58.703074|68.76|joggying +69|374|65783|4294967428|22.22|26.26|true|luke robinson|2013-03-01 09:11:58.703090|59.41|study skills +22|305|65670|4294967477|40.59|18.92|false|mike brown|2013-03-01 09:11:58.703265|43.00|wind surfing +12|325|65731|4294967386|51.67|13.68|true|holly van buren|2013-03-01 09:11:58.703174|74.08|undecided +33|325|65788|4294967381|12.41|15.21|false|ulysses hernandez|2013-03-01 09:11:58.703144|7.87|american history +49|347|65734|4294967358|41.60|8.38|true|mike laertes|2013-03-01 09:11:58.703070|83.64|kindergarten +45|373|65740|4294967384|17.88|46.15|true|xavier carson|2013-03-01 09:11:58.703080|68.09|forestry +14|281|65779|4294967508|71.86|14.41|false|calvin garcia|2013-03-01 09:11:58.703261|19.55|debate +20|388|65612|4294967433|9.81|32.18|false|wendy van buren|2013-03-01 09:11:58.703167|73.84|kindergarten +72|277|65559|4294967449|12.48|30.70|false|yuri steinbeck|2013-03-01 09:11:58.703290|94.85|american history +79|303|65765|4294967468|85.55|41.54|true|zach laertes|2013-03-01 09:11:58.703125|18.14|mathematics +22|358|65545|4294967409|0.90|5.38|true|ulysses garcia|2013-03-01 09:11:58.703076|37.67|american history +14|332|65755|4294967458|21.07|5.71|true|quinn brown|2013-03-01 09:11:58.703263|5.83|education +74|263|65736|4294967337|91.80|33.80|true|victor robinson|2013-03-01 09:11:58.703272|1.04|quiet hour +42|327|65620|4294967464|78.96|47.36|false|fred brown|2013-03-01 09:11:58.703188|49.62|philosophy +121|378|65569|4294967369|9.88|0.24|false|yuri garcia|2013-03-01 09:11:58.703098|5.45|linguistics +116|391|65737|4294967464|13.36|17.59|false|yuri xylophone|2013-03-01 09:11:58.703292|99.95|zync studies +95|410|65545|4294967505|76.99|29.09|true|oscar garcia|2013-03-01 09:11:58.703130|83.10|joggying +50|345|65754|4294967303|9.58|6.56|false|katie xylophone|2013-03-01 09:11:58.703246|13.59|xylophone band +80|312|65558|4294967400|98.06|44.00|false|fred falkner|2013-03-01 09:11:58.703278|1.34|wind surfing +104|354|65767|4294967529|32.48|2.17|false|wendy quirinius|2013-03-01 09:11:58.703316|10.88|xylophone band +113|412|65571|4294967520|75.05|33.87|false|sarah thompson|2013-03-01 09:11:58.703094|49.39|american history +46|376|65639|4294967420|16.16|23.93|true|sarah garcia|2013-03-01 09:11:58.703210|63.46|philosophy +-2|326|65618|4294967341|75.05|41.20|false|zach underhill|2013-03-01 09:11:58.703198|74.60|linguistics +89|466|65613|4294967463|7.30|3.27|true|fred polk|2013-03-01 09:11:58.703082|93.05|mathematics +101|403|65574|4294967380|7.05|23.43|false|rachel hernandez|2013-03-01 09:11:58.703193|54.85|topology +120|497|65669|4294967353|77.10|18.96|false|oscar ichabod|2013-03-01 09:11:58.703255|47.49|debate +75|472|65699|4294967386|62.59|38.97|true|quinn falkner|2013-03-01 09:11:58.703248|89.79|joggying +1|406|65694|4294967463|40.15|24.67|true|luke davidson|2013-03-01 09:11:58.703149|52.88|forestry +83|357|65699|4294967516|47.71|28.68|false|nick ichabod|2013-03-01 09:11:58.703291|87.29|joggying +120|267|65670|4294967415|64.71|27.20|false|ulysses davidson|2013-03-01 09:11:58.703274|19.89|philosophy +120|418|65780|4294967426|77.99|49.62|true|katie king|2013-03-01 09:11:58.703197|11.81|forestry +114|451|65567|4294967405|26.48|37.18|true|holly brown|2013-03-01 09:11:58.703162|8.60|values clariffication +82|505|65755|4294967418|24.51|20.66|true|tom quirinius|2013-03-01 09:11:58.703219|48.60|study skills +82|501|65616|4294967534|47.33|19.43|false|ulysses thompson|2013-03-01 09:11:58.703155|29.66|xylophone band +58|421|65762|4294967514|86.45|10.51|true|luke thompson|2013-03-01 09:11:58.703074|21.55|zync studies +55|303|65776|4294967347|72.18|17.97|true|xavier allen|2013-03-01 09:11:58.703090|79.04|religion +38|445|65546|4294967507|26.22|35.65|true|quinn brown|2013-03-01 09:11:58.703074|83.78|history +6|339|65712|4294967387|70.19|20.62|false|katie robinson|2013-03-01 09:11:58.703306|71.09|yard duty +55|492|65648|4294967465|34.32|40.91|true|priscilla miller|2013-03-01 09:11:58.703112|77.56|study skills +105|332|65791|4294967523|1.90|10.09|false|wendy ichabod|2013-03-01 09:11:58.703306|48.02|religion +59|291|65719|4294967443|95.17|6.98|false|yuri carson|2013-03-01 09:11:58.703305|35.21|industrial engineering +75|285|65631|4294967441|70.76|22.78|true|katie garcia|2013-03-01 09:11:58.703158|4.79|yard duty +58|365|65658|4294967496|0.35|18.45|false|irene zipper|2013-03-01 09:11:58.703188|63.78|history +106|332|65679|4294967459|76.64|2.98|false|jessica king|2013-03-01 09:11:58.703318|75.69|mathematics +114|394|65576|4294967427|3.08|32.33|false|victor polk|2013-03-01 09:11:58.703242|25.22|undecided +26|458|65616|4294967473|9.50|48.78|true|ulysses underhill|2013-03-01 09:11:58.703095|83.50|opthamology +3|408|65678|4294967548|27.21|10.11|false|priscilla davidson|2013-03-01 09:11:58.703258|88.75|joggying +1|405|65639|4294967479|64.20|46.87|false|bob hernandez|2013-03-01 09:11:58.703101|53.36|xylophone band +94|291|65644|4294967326|34.89|17.76|false|david king|2013-03-01 09:11:58.703188|94.49|xylophone band +110|490|65749|4294967420|21.99|36.30|true|xavier johnson|2013-03-01 09:11:58.703309|41.35|quiet hour +17|343|65607|4294967316|20.13|5.16|true|nick king|2013-03-01 09:11:58.703287|26.46|forestry +47|489|65788|4294967501|30.59|26.75|false|sarah steinbeck|2013-03-01 09:11:58.703106|23.33|biology +77|461|65689|4294967499|30.40|2.12|true|sarah allen|2013-03-01 09:11:58.703237|49.06|linguistics +116|436|65723|4294967528|23.05|23.83|false|mike king|2013-03-01 09:11:58.703109|30.82|industrial engineering +19|385|65644|4294967313|25.05|10.06|false|fred xylophone|2013-03-01 09:11:58.703291|13.93|wind surfing +72|454|65597|4294967539|45.90|34.00|false|victor polk|2013-03-01 09:11:58.703202|10.16|industrial engineering +115|495|65579|4294967316|58.10|27.99|false|holly carson|2013-03-01 09:11:58.703300|93.73|topology +116|309|65648|4294967392|18.93|33.02|false|fred van buren|2013-03-01 09:11:58.703279|16.21|kindergarten +51|257|65626|4294967510|23.18|27.79|true|katie garcia|2013-03-01 09:11:58.703131|46.12|zync studies +110|471|65726|4294967505|23.04|36.17|true|katie nixon|2013-03-01 09:11:58.703150|34.87|debate +65|460|65702|4294967413|21.04|38.09|false|wendy nixon|2013-03-01 09:11:58.703241|54.56|education +103|260|65734|4294967392|8.36|4.27|false|mike ellison|2013-03-01 09:11:58.703104|57.71|philosophy +11|312|65654|4294967311|4.17|6.45|true|irene ellison|2013-03-01 09:11:58.703324|32.94|opthamology +30|391|65656|4294967520|32.50|8.18|true|luke robinson|2013-03-01 09:11:58.703196|8.57|undecided +26|486|65768|4294967457|19.07|38.10|false|bob ovid|2013-03-01 09:11:58.703144|18.15|opthamology +61|463|65784|4294967382|78.25|4.79|true|zach allen|2013-03-01 09:11:58.703225|44.96|philosophy +66|345|65736|4294967378|4.98|4.80|false|zach underhill|2013-03-01 09:11:58.703307|8.98|biology +71|273|65621|4294967481|61.73|0.27|true|priscilla young|2013-03-01 09:11:58.703166|27.04|yard duty +34|339|65739|4294967374|17.37|29.76|true|quinn garcia|2013-03-01 09:11:58.703257|40.52|quiet hour +26|268|65777|4294967530|24.99|41.14|true|sarah underhill|2013-03-01 09:11:58.703276|64.21|zync studies +107|454|65760|4294967477|26.61|34.52|false|ethan brown|2013-03-01 09:11:58.703153|11.02|forestry +86|403|65596|4294967485|12.38|31.08|false|mike white|2013-03-01 09:11:58.703123|12.61|yard duty +16|307|65541|4294967536|95.60|30.44|false|sarah young|2013-03-01 09:11:58.703229|3.29|zync studies +66|422|65542|4294967337|57.20|20.32|false|xavier johnson|2013-03-01 09:11:58.703273|7.92|values clariffication +81|428|65717|4294967378|65.68|1.71|false|sarah miller|2013-03-01 09:11:58.703263|63.52|religion +96|259|65630|4294967462|53.70|6.16|true|luke zipper|2013-03-01 09:11:58.703082|75.04|joggying +22|405|65637|4294967542|77.52|7.86|true|luke miller|2013-03-01 09:11:58.703149|19.87|history +48|378|65700|4294967347|60.72|16.60|true|tom hernandez|2013-03-01 09:11:58.703211|60.89|mathematics +23|363|65572|4294967414|23.07|32.63|false|holly underhill|2013-03-01 09:11:58.703290|37.44|opthamology +43|267|65771|4294967495|75.60|25.22|true|irene ichabod|2013-03-01 09:11:58.703161|87.65|values clariffication +113|390|65758|4294967401|65.42|21.18|true|luke brown|2013-03-01 09:11:58.703304|19.43|topology +98|498|65690|4294967375|31.77|13.03|true|zach underhill|2013-03-01 09:11:58.703315|21.89|chemistry +27|375|65577|4294967406|76.76|31.09|false|quinn carson|2013-03-01 09:11:58.703141|72.54|biology +110|310|65652|4294967310|9.68|3.59|false|calvin johnson|2013-03-01 09:11:58.703093|0.37|education +5|283|65791|4294967546|29.29|22.17|false|quinn xylophone|2013-03-01 09:11:58.703248|61.34|biology +19|281|65541|4294967466|9.15|49.86|true|victor ovid|2013-03-01 09:11:58.703117|44.68|debate +53|471|65552|4294967356|30.81|27.49|false|oscar ellison|2013-03-01 09:11:58.703088|78.45|xylophone band +79|481|65706|4294967310|11.45|45.98|true|katie miller|2013-03-01 09:11:58.703111|36.11|forestry +76|416|65727|4294967545|25.06|32.71|true|yuri falkner|2013-03-01 09:11:58.703127|57.89|american history +34|494|65549|4294967308|61.19|23.83|false|victor white|2013-03-01 09:11:58.703211|7.57|history +7|355|65577|4294967460|77.81|41.90|true|oscar laertes|2013-03-01 09:11:58.703146|54.66|opthamology +57|486|65592|4294967443|22.38|23.19|true|victor white|2013-03-01 09:11:58.703200|41.90|religion +62|340|65740|4294967461|41.62|27.29|false|victor falkner|2013-03-01 09:11:58.703092|7.94|quiet hour +39|478|65765|4294967372|8.58|45.52|true|calvin white|2013-03-01 09:11:58.703113|45.35|religion +56|346|65736|4294967370|36.71|46.23|false|jessica xylophone|2013-03-01 09:11:58.703234|94.80|yard duty +76|501|65758|4294967459|54.85|14.76|true|zach young|2013-03-01 09:11:58.703123|91.37|history +67|325|65722|4294967514|1.36|11.64|true|holly ichabod|2013-03-01 09:11:58.703119|69.30|joggying +59|409|65749|4294967377|88.66|23.78|true|xavier davidson|2013-03-01 09:11:58.703309|2.68|joggying +27|264|65681|4294967524|53.33|22.93|false|holly garcia|2013-03-01 09:11:58.703122|65.28|nap time +74|257|65566|4294967517|68.54|43.27|true|priscilla laertes|2013-03-01 09:11:58.703156|60.05|biology +5|321|65727|4294967320|63.58|1.48|false|bob young|2013-03-01 09:11:58.703299|22.53|industrial engineering +37|445|65590|4294967528|92.22|9.30|true|alice davidson|2013-03-01 09:11:58.703263|92.81|religion +63|393|65628|4294967485|75.42|30.11|true|mike ovid|2013-03-01 09:11:58.703284|73.67|biology +122|341|65559|4294967440|31.60|38.23|true|victor young|2013-03-01 09:11:58.703206|19.84|nap time +-2|503|65764|4294967382|5.59|17.19|true|ethan quirinius|2013-03-01 09:11:58.703197|3.88|philosophy +55|510|65692|4294967490|28.57|41.59|true|zach ellison|2013-03-01 09:11:58.703323|70.79|american history +62|479|65595|4294967535|39.65|45.66|true|nick underhill|2013-03-01 09:11:58.703251|66.98|topology +34|271|65631|4294967314|84.84|30.31|false|ethan davidson|2013-03-01 09:11:58.703180|24.32|zync studies +84|427|65739|4294967333|81.02|23.49|false|gabriella robinson|2013-03-01 09:11:58.703273|10.22|wind surfing +91|455|65703|4294967379|99.75|6.11|true|david johnson|2013-03-01 09:11:58.703202|54.98|xylophone band +19|488|65629|4294967347|32.47|25.17|false|tom van buren|2013-03-01 09:11:58.703307|25.04|industrial engineering +88|304|65785|4294967330|52.46|38.78|false|bob robinson|2013-03-01 09:11:58.703194|81.27|topology +37|283|65658|4294967316|80.01|5.14|true|mike ovid|2013-03-01 09:11:58.703283|3.54|study skills +69|391|65583|4294967414|89.86|21.06|true|fred robinson|2013-03-01 09:11:58.703311|55.90|biology +105|499|65765|4294967323|11.92|21.31|false|jessica hernandez|2013-03-01 09:11:58.703171|39.37|yard duty +19|385|65714|4294967385|67.60|28.05|true|holly brown|2013-03-01 09:11:58.703272|33.89|study skills +9|389|65737|4294967395|40.75|32.37|true|holly davidson|2013-03-01 09:11:58.703316|67.82|education +19|273|65733|4294967305|0.16|35.54|false|david laertes|2013-03-01 09:11:58.703277|96.48|geology +11|508|65778|4294967311|2.66|24.45|false|fred xylophone|2013-03-01 09:11:58.703137|92.53|values clariffication +105|485|65682|4294967303|6.01|21.04|false|alice nixon|2013-03-01 09:11:58.703305|97.25|joggying +50|487|65637|4294967391|87.19|32.34|false|alice thompson|2013-03-01 09:11:58.703284|96.09|philosophy +28|473|65727|4294967520|85.92|33.39|true|priscilla laertes|2013-03-01 09:11:58.703286|32.61|chemistry +32|455|65629|4294967418|1.15|47.09|true|mike hernandez|2013-03-01 09:11:58.703231|78.43|opthamology +36|493|65572|4294967533|46.37|20.27|false|nick thompson|2013-03-01 09:11:58.703313|21.46|opthamology +61|475|65665|4294967309|92.32|45.31|true|wendy van buren|2013-03-01 09:11:58.703086|10.20|religion +96|484|65600|4294967494|75.98|49.73|false|alice allen|2013-03-01 09:11:58.703229|7.97|philosophy +9|360|65788|4294967519|53.29|37.05|false|priscilla zipper|2013-03-01 09:11:58.703170|28.69|nap time +118|450|65710|4294967484|0.39|49.57|true|nick miller|2013-03-01 09:11:58.703102|77.93|undecided +70|440|65705|4294967537|91.77|6.65|true|victor robinson|2013-03-01 09:11:58.703195|41.30|joggying +41|441|65790|4294967406|80.61|19.30|false|irene hernandez|2013-03-01 09:11:58.703115|2.15|chemistry +103|503|65647|4294967521|91.37|16.81|true|calvin young|2013-03-01 09:11:58.703151|35.27|yard duty +23|433|65726|4294967365|81.30|42.08|false|irene quirinius|2013-03-01 09:11:58.703292|90.37|wind surfing +22|474|65624|4294967320|97.32|8.52|false|sarah xylophone|2013-03-01 09:11:58.703187|23.57|history +18|438|65718|4294967432|18.81|38.63|false|rachel ovid|2013-03-01 09:11:58.703166|56.15|joggying +81|371|65719|4294967396|42.70|9.03|false|fred robinson|2013-03-01 09:11:58.703281|84.75|values clariffication +96|497|65684|4294967496|3.84|12.64|false|calvin thompson|2013-03-01 09:11:58.703088|56.90|debate +87|304|65588|4294967484|84.85|7.38|true|oscar steinbeck|2013-03-01 09:11:58.703194|78.97|education +56|378|65668|4294967384|20.74|36.18|true|yuri van buren|2013-03-01 09:11:58.703115|83.95|debate +119|275|65572|4294967377|46.84|20.69|true|priscilla zipper|2013-03-01 09:11:58.703071|44.32|topology +119|380|65609|4294967496|34.35|15.01|false|oscar garcia|2013-03-01 09:11:58.703294|34.80|philosophy +60|375|65572|4294967316|56.39|6.02|true|gabriella carson|2013-03-01 09:11:58.703102|95.81|values clariffication +16|364|65694|4294967433|86.17|7.59|true|mike polk|2013-03-01 09:11:58.703087|4.55|american history +11|403|65637|4294967478|43.69|20.87|false|mike allen|2013-03-01 09:11:58.703289|64.81|education +77|256|65731|4294967375|34.09|47.24|true|gabriella falkner|2013-03-01 09:11:58.703175|84.35|zync studies +104|449|65685|4294967356|0.97|1.34|false|holly white|2013-03-01 09:11:58.703093|28.87|education +107|370|65561|4294967325|4.85|34.95|false|ulysses johnson|2013-03-01 09:11:58.703133|53.04|industrial engineering +107|305|65718|4294967354|19.36|9.45|false|victor quirinius|2013-03-01 09:11:58.703281|91.03|history +84|326|65750|4294967447|64.35|30.57|true|irene falkner|2013-03-01 09:11:58.703105|93.27|xylophone band +63|302|65703|4294967363|3.65|1.58|false|ethan xylophone|2013-03-01 09:11:58.703275|73.65|mathematics +89|360|65734|4294967463|69.66|22.40|true|quinn allen|2013-03-01 09:11:58.703171|55.00|industrial engineering +23|362|65660|4294967318|22.16|36.91|true|nick van buren|2013-03-01 09:11:58.703074|94.50|undecided +99|417|65566|4294967408|46.55|0.08|true|zach white|2013-03-01 09:11:58.703277|28.31|industrial engineering +16|509|65561|4294967487|77.64|29.28|true|tom ovid|2013-03-01 09:11:58.703316|92.75|zync studies +58|505|65605|4294967352|38.31|41.95|true|irene johnson|2013-03-01 09:11:58.703096|73.07|nap time +54|439|65747|4294967339|85.61|47.60|true|zach polk|2013-03-01 09:11:58.703276|69.00|quiet hour +115|302|65653|4294967526|95.83|7.57|false|holly van buren|2013-03-01 09:11:58.703117|85.29|religion +47|338|65577|4294967421|18.81|38.54|false|luke falkner|2013-03-01 09:11:58.703290|96.85|history +109|470|65760|4294967481|24.54|46.70|true|yuri white|2013-03-01 09:11:58.703135|59.72|joggying +5|448|65750|4294967502|20.75|31.36|false|ulysses carson|2013-03-01 09:11:58.703223|54.04|kindergarten +99|266|65551|4294967543|71.35|49.86|true|nick polk|2013-03-01 09:11:58.703074|34.27|american history +12|337|65735|4294967349|7.48|37.92|true|priscilla king|2013-03-01 09:11:58.703182|8.22|philosophy +48|373|65730|4294967421|24.70|20.62|true|irene ichabod|2013-03-01 09:11:58.703074|75.11|education +16|482|65699|4294967388|64.66|31.10|false|mike laertes|2013-03-01 09:11:58.703074|76.54|mathematics +59|445|65698|4294967396|11.77|48.00|false|david young|2013-03-01 09:11:58.703310|81.23|american history +35|385|65726|4294967370|6.84|22.68|false|priscilla davidson|2013-03-01 09:11:58.703170|59.36|biology +87|466|65708|4294967391|10.48|46.72|true|ulysses young|2013-03-01 09:11:58.703106|54.23|linguistics +110|496|65644|4294967492|97.59|16.20|false|katie quirinius|2013-03-01 09:11:58.703272|25.88|quiet hour +85|383|65567|4294967520|49.20|15.56|false|wendy steinbeck|2013-03-01 09:11:58.703135|92.79|undecided +39|391|65738|4294967298|32.88|32.44|false|gabriella garcia|2013-03-01 09:11:58.703275|4.60|mathematics +55|487|65591|4294967471|43.66|38.09|false|luke carson|2013-03-01 09:11:58.703267|32.05|philosophy +31|388|65554|4294967443|65.45|44.19|false|ethan laertes|2013-03-01 09:11:58.703083|2.72|undecided +24|281|65768|4294967351|20.06|16.55|true|ulysses carson|2013-03-01 09:11:58.703276|55.40|undecided +103|425|65708|4294967493|95.18|13.17|true|calvin robinson|2013-03-01 09:11:58.703081|47.35|xylophone band +38|454|65587|4294967481|93.82|38.45|true|quinn underhill|2013-03-01 09:11:58.703322|60.90|zync studies +103|345|65556|4294967329|70.11|45.76|false|quinn nixon|2013-03-01 09:11:58.703301|6.61|biology +12|392|65666|4294967337|16.08|25.98|true|zach van buren|2013-03-01 09:11:58.703163|65.53|forestry +40|436|65721|4294967369|33.33|24.51|true|yuri polk|2013-03-01 09:11:58.703079|10.74|industrial engineering +41|396|65745|4294967487|64.86|11.69|true|gabriella nixon|2013-03-01 09:11:58.703255|78.83|topology +42|376|65759|4294967545|14.40|30.46|true|nick robinson|2013-03-01 09:11:58.703167|52.83|american history +14|414|65590|4294967304|43.93|18.81|false|quinn white|2013-03-01 09:11:58.703140|72.33|values clariffication +33|301|65656|4294967483|98.59|46.37|true|tom falkner|2013-03-01 09:11:58.703077|4.32|nap time +3|416|65684|4294967443|61.01|2.20|false|fred xylophone|2013-03-01 09:11:58.703293|13.20|study skills +83|313|65726|4294967456|13.97|23.09|true|fred brown|2013-03-01 09:11:58.703124|33.50|mathematics +65|334|65561|4294967412|86.76|25.27|false|luke miller|2013-03-01 09:11:58.703136|19.28|forestry +25|496|65605|4294967364|55.24|0.77|true|david polk|2013-03-01 09:11:58.703200|64.77|wind surfing +99|504|65584|4294967366|48.50|10.28|false|xavier robinson|2013-03-01 09:11:58.703221|28.25|mathematics +0|340|65778|4294967363|13.16|3.27|true|gabriella king|2013-03-01 09:11:58.703078|63.35|joggying +71|350|65646|4294967455|86.07|3.05|false|katie robinson|2013-03-01 09:11:58.703223|56.23|history +31|409|65560|4294967438|1.50|17.82|true|david underhill|2013-03-01 09:11:58.703301|73.02|american history +55|325|65774|4294967430|93.45|39.52|false|oscar underhill|2013-03-01 09:11:58.703124|50.71|industrial engineering +86|402|65728|4294967551|66.99|33.10|true|rachel nixon|2013-03-01 09:11:58.703293|75.36|yard duty +5|418|65639|4294967300|87.15|39.72|true|wendy zipper|2013-03-01 09:11:58.703111|18.39|education +15|363|65745|4294967313|34.36|36.94|false|calvin van buren|2013-03-01 09:11:58.703117|75.84|forestry +35|331|65570|4294967464|82.02|35.48|true|ethan garcia|2013-03-01 09:11:58.703152|32.56|study skills +97|467|65651|4294967491|76.30|20.37|true|yuri hernandez|2013-03-01 09:11:58.703106|56.33|philosophy +91|269|65700|4294967439|16.66|10.63|true|david underhill|2013-03-01 09:11:58.703312|93.71|american history +25|379|65736|4294967384|8.06|47.10|true|gabriella young|2013-03-01 09:11:58.703211|64.42|kindergarten +81|467|65751|4294967423|53.65|47.49|true|xavier quirinius|2013-03-01 09:11:58.703265|34.71|linguistics +39|376|65696|4294967304|88.24|27.40|true|nick davidson|2013-03-01 09:11:58.703181|37.95|debate +69|298|65537|4294967419|85.45|30.64|false|zach ichabod|2013-03-01 09:11:58.703097|25.20|biology +77|355|65639|4294967431|60.84|32.27|true|david xylophone|2013-03-01 09:11:58.703266|87.76|topology +107|481|65723|4294967402|3.17|7.86|true|bob white|2013-03-01 09:11:58.703252|40.74|study skills +35|368|65580|4294967508|99.18|47.75|false|zach davidson|2013-03-01 09:11:58.703280|83.22|quiet hour +8|469|65684|4294967419|35.21|0.11|false|xavier steinbeck|2013-03-01 09:11:58.703236|19.60|nap time +24|457|65548|4294967501|9.58|0.75|false|calvin ovid|2013-03-01 09:11:58.703295|3.61|religion +49|302|65773|4294967366|32.72|35.46|false|calvin young|2013-03-01 09:11:58.703099|34.64|study skills +121|355|65764|4294967495|22.14|35.99|true|luke garcia|2013-03-01 09:11:58.703194|33.77|history +44|317|65769|4294967541|72.17|8.33|true|ethan davidson|2013-03-01 09:11:58.703135|58.88|topology +110|358|65688|4294967436|36.78|39.65|true|rachel hernandez|2013-03-01 09:11:58.703154|94.63|debate +44|274|65598|4294967353|59.38|32.87|false|yuri brown|2013-03-01 09:11:58.703093|1.49|industrial engineering +94|459|65704|4294967342|5.89|37.82|true|oscar young|2013-03-01 09:11:58.703300|8.17|yard duty +34|402|65744|4294967389|56.24|19.90|true|quinn polk|2013-03-01 09:11:58.703310|36.13|kindergarten +88|364|65606|4294967395|19.76|48.20|true|fred allen|2013-03-01 09:11:58.703262|2.63|mathematics +67|337|65786|4294967401|9.16|9.05|true|xavier garcia|2013-03-01 09:11:58.703085|35.14|linguistics +79|501|65643|4294967383|17.73|30.61|true|katie ovid|2013-03-01 09:11:58.703287|66.37|topology +19|480|65640|4294967537|38.97|32.24|true|yuri nixon|2013-03-01 09:11:58.703189|62.28|opthamology +96|373|65728|4294967401|76.42|16.13|true|calvin carson|2013-03-01 09:11:58.703304|58.44|religion +114|380|65627|4294967333|53.53|21.76|true|alice ovid|2013-03-01 09:11:58.703263|61.85|mathematics +75|307|65695|4294967449|27.23|46.77|false|calvin nixon|2013-03-01 09:11:58.703071|80.81|religion +-3|505|65565|4294967407|68.73|4.65|true|holly nixon|2013-03-01 09:11:58.703262|15.13|debate +78|344|65671|4294967351|27.35|25.74|false|quinn quirinius|2013-03-01 09:11:58.703234|10.32|quiet hour +21|487|65581|4294967324|69.02|46.67|false|ethan ellison|2013-03-01 09:11:58.703165|13.82|kindergarten +99|442|65732|4294967512|52.88|14.91|true|victor hernandez|2013-03-01 09:11:58.703302|79.03|quiet hour +82|372|65749|4294967431|6.31|22.24|false|priscilla hernandez|2013-03-01 09:11:58.703244|27.14|debate +111|387|65582|4294967456|98.85|19.01|true|quinn quirinius|2013-03-01 09:11:58.703230|78.56|kindergarten +95|335|65607|4294967433|25.98|42.38|false|sarah garcia|2013-03-01 09:11:58.703195|11.65|mathematics +39|505|65572|4294967418|58.80|16.28|false|priscilla carson|2013-03-01 09:11:58.703194|48.42|nap time +12|293|65671|4294967319|19.51|20.14|false|rachel nixon|2013-03-01 09:11:58.703082|33.58|geology +34|458|65778|4294967344|16.43|33.37|true|oscar hernandez|2013-03-01 09:11:58.703200|89.00|american history +30|440|65769|4294967439|37.32|43.33|true|luke nixon|2013-03-01 09:11:58.703200|39.75|debate +113|355|65553|4294967551|13.72|16.39|true|irene underhill|2013-03-01 09:11:58.703204|17.98|study skills +28|392|65754|4294967415|49.92|6.04|true|wendy thompson|2013-03-01 09:11:58.703217|27.39|chemistry +25|450|65712|4294967493|63.40|22.37|true|priscilla falkner|2013-03-01 09:11:58.703236|63.25|geology +40|299|65566|4294967332|22.94|24.80|false|zach allen|2013-03-01 09:11:58.703273|23.55|values clariffication +89|500|65556|4294967500|49.15|44.39|false|irene quirinius|2013-03-01 09:11:58.703247|57.60|joggying +113|273|65712|4294967404|70.34|31.93|false|katie brown|2013-03-01 09:11:58.703117|92.31|geology +84|273|65615|4294967532|4.29|39.86|true|gabriella hernandez|2013-03-01 09:11:58.703169|68.25|education +96|335|65704|4294967345|70.93|46.90|true|sarah underhill|2013-03-01 09:11:58.703252|42.66|education +32|502|65771|4294967499|79.05|5.01|true|yuri zipper|2013-03-01 09:11:58.703081|60.00|xylophone band +57|341|65697|4294967423|76.13|27.69|true|calvin carson|2013-03-01 09:11:58.703071|10.75|linguistics +121|339|65544|4294967507|97.47|29.18|false|bob underhill|2013-03-01 09:11:58.703275|53.07|opthamology +84|257|65575|4294967528|1.21|46.88|false|oscar thompson|2013-03-01 09:11:58.703135|38.84|kindergarten +13|446|65616|4294967537|68.00|20.68|true|priscilla ichabod|2013-03-01 09:11:58.703094|95.40|religion +79|376|65739|4294967500|78.93|22.57|true|gabriella brown|2013-03-01 09:11:58.703273|67.87|philosophy +87|378|65598|4294967528|46.86|20.63|false|alice polk|2013-03-01 09:11:58.703125|25.50|philosophy +60|373|65652|4294967392|62.61|11.64|true|nick miller|2013-03-01 09:11:58.703074|2.82|geology +76|376|65742|4294967513|59.90|13.66|true|sarah davidson|2013-03-01 09:11:58.703302|54.54|zync studies +84|355|65668|4294967437|96.01|42.27|true|calvin laertes|2013-03-01 09:11:58.703254|98.42|quiet hour +120|458|65725|4294967436|67.07|3.88|false|tom underhill|2013-03-01 09:11:58.703325|46.29|debate +11|381|65636|4294967541|19.17|24.65|true|rachel brown|2013-03-01 09:11:58.703218|48.94|linguistics +22|301|65593|4294967532|4.58|0.74|true|zach nixon|2013-03-01 09:11:58.703263|19.84|philosophy +6|413|65746|4294967546|53.73|32.47|true|calvin white|2013-03-01 09:11:58.703255|47.66|joggying +77|467|65683|4294967399|61.76|3.72|false|david allen|2013-03-01 09:11:58.703162|43.72|quiet hour +37|498|65650|4294967312|6.51|44.27|true|ethan allen|2013-03-01 09:11:58.703247|11.62|yard duty +102|408|65665|4294967518|44.08|35.35|false|xavier laertes|2013-03-01 09:11:58.703159|30.26|nap time +34|472|65707|4294967350|48.34|21.72|false|jessica brown|2013-03-01 09:11:58.703297|43.09|history +18|271|65761|4294967370|39.82|0.46|true|gabriella laertes|2013-03-01 09:11:58.703129|76.24|undecided +36|380|65785|4294967363|64.20|32.67|true|alice carson|2013-03-01 09:11:58.703268|22.59|chemistry +71|390|65777|4294967409|46.00|44.78|true|holly zipper|2013-03-01 09:11:58.703235|29.96|zync studies +45|386|65646|4294967389|32.58|49.38|true|zach van buren|2013-03-01 09:11:58.703202|98.22|geology +65|260|65621|4294967456|18.84|48.79|false|calvin xylophone|2013-03-01 09:11:58.703173|95.94|xylophone band +-3|275|65543|4294967522|74.92|17.29|false|mike king|2013-03-01 09:11:58.703214|52.90|opthamology +61|452|65557|4294967483|95.89|22.12|true|holly falkner|2013-03-01 09:11:58.703255|15.99|industrial engineering +28|497|65562|4294967305|21.41|46.71|false|jessica steinbeck|2013-03-01 09:11:58.703298|95.07|study skills +104|274|65577|4294967524|90.64|11.08|true|oscar van buren|2013-03-01 09:11:58.703202|90.55|wind surfing +60|266|65773|4294967432|66.33|29.16|true|katie brown|2013-03-01 09:11:58.703075|28.74|industrial engineering +20|481|65762|4294967397|85.74|45.68|true|irene polk|2013-03-01 09:11:58.703150|25.29|values clariffication +3|511|65559|4294967488|35.17|21.53|true|rachel xylophone|2013-03-01 09:11:58.703298|16.74|zync studies +2|326|65757|4294967463|48.40|34.10|true|nick allen|2013-03-01 09:11:58.703227|6.76|opthamology +86|382|65552|4294967340|56.41|24.77|true|jessica miller|2013-03-01 09:11:58.703265|54.92|opthamology +94|322|65747|4294967476|60.88|10.76|false|irene ovid|2013-03-01 09:11:58.703295|26.19|history +14|286|65728|4294967341|23.65|9.78|true|zach young|2013-03-01 09:11:58.703302|26.65|values clariffication +94|479|65785|4294967300|99.94|10.15|true|yuri garcia|2013-03-01 09:11:58.703143|82.46|linguistics +22|361|65717|4294967487|48.28|11.11|false|zach xylophone|2013-03-01 09:11:58.703241|5.15|philosophy +37|337|65547|4294967344|12.16|21.10|false|calvin davidson|2013-03-01 09:11:58.703171|4.73|xylophone band +16|329|65764|4294967337|39.30|19.37|false|priscilla underhill|2013-03-01 09:11:58.703273|15.81|mathematics +120|361|65700|4294967454|16.94|26.46|true|oscar nixon|2013-03-01 09:11:58.703239|40.87|xylophone band +93|461|65707|4294967404|5.22|5.44|true|ethan white|2013-03-01 09:11:58.703168|63.05|wind surfing +86|452|65731|4294967373|25.69|0.26|true|jessica steinbeck|2013-03-01 09:11:58.703118|60.87|biology +69|336|65779|4294967505|53.06|4.50|true|bob johnson|2013-03-01 09:11:58.703267|64.71|industrial engineering +14|387|65740|4294967430|94.42|18.97|false|ethan zipper|2013-03-01 09:11:58.703269|23.99|zync studies +-3|415|65571|4294967536|61.81|24.24|true|victor robinson|2013-03-01 09:11:58.703305|22.72|american history +51|320|65677|4294967415|27.17|45.00|false|zach allen|2013-03-01 09:11:58.703195|55.00|topology +69|355|65687|4294967370|31.05|20.99|true|gabriella garcia|2013-03-01 09:11:58.703190|91.28|wind surfing +29|452|65582|4294967454|20.47|9.27|true|bob hernandez|2013-03-01 09:11:58.703083|68.43|yard duty +38|370|65563|4294967479|52.06|8.26|true|tom thompson|2013-03-01 09:11:58.703209|83.21|biology +17|487|65576|4294967362|82.94|6.04|false|quinn garcia|2013-03-01 09:11:58.703227|69.91|wind surfing +10|489|65761|4294967533|15.56|25.80|true|ethan ichabod|2013-03-01 09:11:58.703232|81.41|wind surfing +65|340|65670|4294967384|80.43|15.86|true|xavier underhill|2013-03-01 09:11:58.703302|51.08|biology +110|270|65563|4294967453|60.39|2.80|false|alice davidson|2013-03-01 09:11:58.703238|95.46|biology +97|381|65538|4294967485|44.89|43.32|false|nick quirinius|2013-03-01 09:11:58.703284|78.41|quiet hour +75|496|65621|4294967537|78.69|46.15|false|sarah steinbeck|2013-03-01 09:11:58.703087|23.44|philosophy +110|299|65696|4294967530|45.45|43.50|false|zach polk|2013-03-01 09:11:58.703270|57.49|forestry +37|356|65596|4294967440|26.21|25.28|false|david brown|2013-03-01 09:11:58.703164|45.98|history +27|317|65678|4294967329|51.25|15.92|true|luke hernandez|2013-03-01 09:11:58.703179|31.91|american history +59|470|65655|4294967508|41.99|6.49|false|ethan carson|2013-03-01 09:11:58.703300|93.24|mathematics +62|264|65577|4294967383|19.91|16.77|false|quinn robinson|2013-03-01 09:11:58.703087|50.73|study skills +75|408|65705|4294967398|58.21|14.29|true|jessica allen|2013-03-01 09:11:58.703310|86.21|values clariffication +73|403|65562|4294967382|20.47|29.01|true|sarah steinbeck|2013-03-01 09:11:58.703174|10.15|history +76|498|65695|4294967321|80.45|13.88|false|victor hernandez|2013-03-01 09:11:58.703141|61.28|kindergarten +2|397|65649|4294967517|47.26|36.68|false|rachel robinson|2013-03-01 09:11:58.703251|43.29|religion +9|276|65556|4294967474|38.91|16.97|true|mike nixon|2013-03-01 09:11:58.703163|41.37|chemistry +120|323|65744|4294967303|0.28|22.84|true|yuri davidson|2013-03-01 09:11:58.703324|97.38|yard duty +49|343|65602|4294967321|54.17|14.77|false|priscilla robinson|2013-03-01 09:11:58.703314|18.14|industrial engineering +32|298|65648|4294967474|98.97|1.42|true|sarah king|2013-03-01 09:11:58.703188|0.03|biology +82|458|65761|4294967388|25.53|34.13|true|priscilla falkner|2013-03-01 09:11:58.703239|58.06|chemistry +25|455|65782|4294967369|58.92|1.20|false|fred miller|2013-03-01 09:11:58.703271|75.06|forestry +119|507|65778|4294967299|36.17|48.84|true|quinn white|2013-03-01 09:11:58.703203|40.06|forestry +23|418|65743|4294967323|96.24|44.98|false|oscar allen|2013-03-01 09:11:58.703110|24.56|topology +63|303|65771|4294967375|78.09|4.85|true|alice laertes|2013-03-01 09:11:58.703257|74.36|xylophone band +98|467|65713|4294967296|30.90|9.35|true|gabriella steinbeck|2013-03-01 09:11:58.703110|46.53|quiet hour +73|488|65723|4294967361|29.59|38.04|false|gabriella brown|2013-03-01 09:11:58.703109|40.90|linguistics +6|362|65739|4294967368|38.93|9.97|true|tom allen|2013-03-01 09:11:58.703160|7.19|kindergarten +25|395|65570|4294967430|70.31|25.04|true|wendy ellison|2013-03-01 09:11:58.703287|21.49|opthamology +97|350|65746|4294967439|54.06|33.76|true|david zipper|2013-03-01 09:11:58.703180|33.76|xylophone band +108|308|65704|4294967524|92.69|11.23|true|david young|2013-03-01 09:11:58.703109|82.05|education +71|307|65562|4294967426|41.41|33.17|false|katie ichabod|2013-03-01 09:11:58.703134|36.77|kindergarten +91|418|65671|4294967355|64.52|6.55|false|katie underhill|2013-03-01 09:11:58.703086|91.12|nap time +35|405|65648|4294967425|60.41|8.87|true|zach ichabod|2013-03-01 09:11:58.703260|11.38|quiet hour +57|477|65682|4294967498|15.02|27.74|true|sarah hernandez|2013-03-01 09:11:58.703234|36.75|yard duty +65|323|65606|4294967439|0.03|35.17|true|quinn miller|2013-03-01 09:11:58.703281|4.50|opthamology +87|477|65566|4294967318|56.21|46.48|false|alice van buren|2013-03-01 09:11:58.703110|27.13|wind surfing +99|404|65639|4294967340|7.53|28.00|true|sarah ovid|2013-03-01 09:11:58.703267|68.35|wind surfing +115|481|65671|4294967458|13.10|25.30|false|quinn carson|2013-03-01 09:11:58.703126|30.40|debate +50|388|65552|4294967392|41.61|17.70|false|fred ichabod|2013-03-01 09:11:58.703183|52.28|philosophy +39|367|65779|4294967435|19.15|38.25|false|jessica polk|2013-03-01 09:11:58.703313|25.94|joggying +77|477|65763|4294967432|62.86|18.69|true|priscilla king|2013-03-01 09:11:58.703231|12.40|mathematics +1|283|65639|4294967297|38.95|30.35|true|oscar robinson|2013-03-01 09:11:58.703176|3.13|xylophone band +99|374|65748|4294967409|27.72|34.99|true|holly hernandez|2013-03-01 09:11:58.703196|43.26|study skills +43|290|65603|4294967505|6.93|18.91|false|ethan van buren|2013-03-01 09:11:58.703323|54.34|forestry +37|459|65698|4294967475|77.90|4.69|false|fred davidson|2013-03-01 09:11:58.703238|57.48|american history +24|447|65673|4294967333|19.87|31.59|false|david king|2013-03-01 09:11:58.703130|66.36|wind surfing +78|329|65555|4294967395|14.35|17.71|true|rachel ichabod|2013-03-01 09:11:58.703272|57.55|opthamology +29|452|65632|4294967330|91.85|9.50|false|oscar ichabod|2013-03-01 09:11:58.703077|18.83|joggying +14|401|65689|4294967519|14.41|39.72|false|luke laertes|2013-03-01 09:11:58.703308|59.57|wind surfing +48|389|65661|4294967336|72.83|29.50|true|xavier white|2013-03-01 09:11:58.703118|49.67|education +77|488|65657|4294967480|91.45|40.84|true|fred white|2013-03-01 09:11:58.703252|40.62|nap time +48|360|65780|4294967332|46.84|17.84|false|bob king|2013-03-01 09:11:58.703273|45.40|philosophy +-2|378|65553|4294967461|9.81|10.36|true|bob king|2013-03-01 09:11:58.703236|90.88|opthamology +6|468|65619|4294967474|61.75|37.73|true|mike underhill|2013-03-01 09:11:58.703200|59.53|industrial engineering +48|444|65568|4294967504|52.16|39.11|false|zach falkner|2013-03-01 09:11:58.703183|57.19|history +-1|482|65690|4294967476|1.49|35.42|false|quinn miller|2013-03-01 09:11:58.703301|53.85|biology +105|379|65664|4294967545|93.78|18.05|true|bob brown|2013-03-01 09:11:58.703272|67.91|wind surfing +67|393|65757|4294967409|91.16|24.26|false|victor nixon|2013-03-01 09:11:58.703209|96.39|religion +19|339|65760|4294967406|66.12|36.22|false|gabriella ichabod|2013-03-01 09:11:58.703253|28.43|study skills +18|325|65560|4294967407|27.93|22.95|true|katie garcia|2013-03-01 09:11:58.703073|35.45|geology +99|333|65694|4294967348|93.25|12.77|true|jessica thompson|2013-03-01 09:11:58.703241|73.39|topology +86|290|65770|4294967522|54.49|42.92|true|oscar underhill|2013-03-01 09:11:58.703193|1.49|opthamology +124|336|65761|4294967418|56.59|16.62|true|sarah quirinius|2013-03-01 09:11:58.703293|55.03|mathematics +95|323|65679|4294967477|37.34|41.25|true|priscilla laertes|2013-03-01 09:11:58.703196|68.16|zync studies +7|412|65679|4294967544|87.84|13.17|true|holly allen|2013-03-01 09:11:58.703135|37.48|geology +82|353|65650|4294967362|80.38|18.53|true|gabriella carson|2013-03-01 09:11:58.703291|30.11|linguistics +112|330|65746|4294967492|6.06|30.92|true|nick thompson|2013-03-01 09:11:58.703140|54.03|yard duty +112|434|65661|4294967549|93.18|46.90|false|jessica nixon|2013-03-01 09:11:58.703125|28.14|education +45|261|65550|4294967523|2.89|10.76|false|ethan johnson|2013-03-01 09:11:58.703191|68.40|chemistry +23|457|65559|4294967453|45.43|10.98|true|calvin ovid|2013-03-01 09:11:58.703231|72.47|chemistry +66|369|65640|4294967542|39.70|38.83|true|priscilla robinson|2013-03-01 09:11:58.703141|62.11|american history +68|405|65645|4294967483|71.74|34.38|false|katie nixon|2013-03-01 09:11:58.703146|81.74|kindergarten +106|498|65786|4294967374|52.18|18.05|false|irene quirinius|2013-03-01 09:11:58.703261|47.77|undecided +98|367|65687|4294967383|46.62|46.57|true|ulysses van buren|2013-03-01 09:11:58.703212|69.01|joggying +2|295|65555|4294967548|61.61|5.68|false|tom van buren|2013-03-01 09:11:58.703148|82.96|geology +66|377|65683|4294967343|26.11|19.18|false|bob king|2013-03-01 09:11:58.703287|66.91|topology +45|393|65727|4294967438|67.44|33.20|true|mike polk|2013-03-01 09:11:58.703203|30.77|yard duty +74|269|65788|4294967475|51.91|17.09|true|sarah ichabod|2013-03-01 09:11:58.703170|91.35|topology +19|490|65550|4294967445|43.17|31.30|true|luke hernandez|2013-03-01 09:11:58.703270|81.66|debate +61|502|65753|4294967508|34.19|18.62|true|xavier zipper|2013-03-01 09:11:58.703085|97.84|kindergarten +62|317|65633|4294967424|76.81|5.93|false|tom zipper|2013-03-01 09:11:58.703232|16.59|forestry +-3|335|65696|4294967333|72.26|9.66|true|nick nixon|2013-03-01 09:11:58.703083|85.48|philosophy +26|317|65575|4294967504|88.29|2.45|true|bob johnson|2013-03-01 09:11:58.703311|53.63|opthamology +-1|422|65784|4294967384|36.80|49.72|true|ulysses xylophone|2013-03-01 09:11:58.703120|52.10|joggying +39|443|65589|4294967369|19.20|2.39|false|ulysses allen|2013-03-01 09:11:58.703229|79.15|yard duty +18|455|65644|4294967419|23.50|43.14|true|katie xylophone|2013-03-01 09:11:58.703228|98.77|religion +123|346|65787|4294967454|25.05|22.60|true|holly hernandez|2013-03-01 09:11:58.703269|20.99|joggying +27|504|65717|4294967365|96.68|26.00|true|victor laertes|2013-03-01 09:11:58.703158|24.07|forestry +7|315|65678|4294967440|6.96|28.84|false|jessica nixon|2013-03-01 09:11:58.703175|73.46|american history +110|264|65743|4294967327|71.34|48.89|true|jessica steinbeck|2013-03-01 09:11:58.703093|37.06|study skills +91|500|65653|4294967512|98.49|35.06|false|tom underhill|2013-03-01 09:11:58.703224|34.01|xylophone band +85|388|65626|4294967497|34.15|47.49|true|sarah johnson|2013-03-01 09:11:58.703314|53.64|forestry +35|379|65762|4294967468|73.58|36.53|true|oscar nixon|2013-03-01 09:11:58.703325|57.86|history +101|448|65638|4294967308|78.02|46.75|true|mike steinbeck|2013-03-01 09:11:58.703087|23.53|history +44|326|65741|4294967362|63.97|18.45|true|gabriella quirinius|2013-03-01 09:11:58.703084|80.38|study skills +7|503|65789|4294967363|20.45|40.84|false|irene miller|2013-03-01 09:11:58.703204|64.32|biology +112|474|65586|4294967421|68.91|42.95|false|ethan nixon|2013-03-01 09:11:58.703320|97.32|debate +76|492|65604|4294967353|93.74|29.10|false|calvin robinson|2013-03-01 09:11:58.703278|55.69|mathematics +97|309|65624|4294967419|5.83|45.44|true|victor miller|2013-03-01 09:11:58.703199|30.15|religion +29|414|65587|4294967437|34.50|18.86|false|bob falkner|2013-03-01 09:11:58.703324|69.52|undecided +44|361|65722|4294967538|88.74|16.60|false|irene van buren|2013-03-01 09:11:58.703108|38.08|study skills +100|368|65623|4294967423|61.47|28.81|true|sarah brown|2013-03-01 09:11:58.703181|82.10|american history +29|328|65767|4294967520|31.00|8.76|false|xavier van buren|2013-03-01 09:11:58.703102|68.87|education +75|301|65595|4294967319|13.97|41.26|false|luke davidson|2013-03-01 09:11:58.703242|31.24|topology +120|483|65580|4294967406|50.93|35.61|false|gabriella zipper|2013-03-01 09:11:58.703190|68.83|study skills +26|419|65634|4294967377|28.27|8.92|true|victor hernandez|2013-03-01 09:11:58.703094|45.12|kindergarten +62|437|65675|4294967510|12.00|30.37|true|irene miller|2013-03-01 09:11:58.703117|96.81|quiet hour +94|337|65743|4294967386|56.56|21.00|false|tom carson|2013-03-01 09:11:58.703268|79.92|education +85|478|65545|4294967434|55.32|42.36|false|fred steinbeck|2013-03-01 09:11:58.703093|8.65|biology +55|433|65663|4294967497|81.71|24.83|true|xavier quirinius|2013-03-01 09:11:58.703180|61.42|education +78|463|65767|4294967545|79.11|32.79|true|holly hernandez|2013-03-01 09:11:58.703137|97.03|values clariffication +-3|280|65548|4294967350|52.30|33.06|true|calvin white|2013-03-01 09:11:58.703295|29.54|quiet hour +56|424|65742|4294967545|89.27|29.20|true|irene ellison|2013-03-01 09:11:58.703140|87.04|american history +72|469|65573|4294967510|54.52|22.00|true|nick nixon|2013-03-01 09:11:58.703253|36.49|yard duty +0|397|65667|4294967336|86.24|13.67|true|nick thompson|2013-03-01 09:11:58.703186|99.87|chemistry +16|349|65590|4294967345|52.71|44.99|false|rachel quirinius|2013-03-01 09:11:58.703306|47.06|opthamology +60|419|65550|4294967414|66.25|2.76|true|nick young|2013-03-01 09:11:58.703178|6.17|xylophone band +25|459|65564|4294967454|66.03|44.75|true|bob johnson|2013-03-01 09:11:58.703162|34.84|religion +3|507|65722|4294967408|0.12|27.23|true|irene thompson|2013-03-01 09:11:58.703270|96.72|topology +76|417|65600|4294967496|7.80|0.98|false|calvin polk|2013-03-01 09:11:58.703192|59.90|nap time +18|282|65710|4294967308|9.27|36.70|true|tom white|2013-03-01 09:11:58.703208|0.57|zync studies +63|262|65777|4294967372|93.59|44.28|true|luke johnson|2013-03-01 09:11:58.703268|52.19|industrial engineering +11|447|65579|4294967363|87.39|38.99|true|yuri robinson|2013-03-01 09:11:58.703160|80.83|history +105|269|65629|4294967340|26.58|3.96|false|jessica ichabod|2013-03-01 09:11:58.703191|10.77|nap time +74|468|65755|4294967325|16.59|35.47|true|ulysses hernandez|2013-03-01 09:11:58.703265|34.67|opthamology +91|378|65725|4294967535|92.48|36.48|false|nick robinson|2013-03-01 09:11:58.703266|61.30|philosophy +39|281|65759|4294967434|99.32|32.31|false|zach van buren|2013-03-01 09:11:58.703077|17.73|nap time +59|404|65542|4294967332|42.13|2.35|false|tom young|2013-03-01 09:11:58.703235|93.63|history +117|488|65654|4294967544|71.50|44.00|false|wendy carson|2013-03-01 09:11:58.703296|3.16|forestry +72|283|65624|4294967325|7.58|22.53|true|calvin ellison|2013-03-01 09:11:58.703302|27.82|debate +122|510|65662|4294967415|78.52|8.04|false|calvin miller|2013-03-01 09:11:58.703215|73.20|religion +111|344|65586|4294967551|86.97|1.02|true|fred falkner|2013-03-01 09:11:58.703135|29.75|values clariffication +110|399|65789|4294967535|83.53|0.74|true|rachel ichabod|2013-03-01 09:11:58.703325|24.68|linguistics +31|336|65618|4294967411|45.11|48.07|false|xavier robinson|2013-03-01 09:11:58.703225|17.68|biology +38|299|65605|4294967336|37.62|30.97|true|yuri underhill|2013-03-01 09:11:58.703224|51.40|education +122|473|65719|4294967478|9.60|28.10|true|fred nixon|2013-03-01 09:11:58.703318|68.06|quiet hour +5|268|65601|4294967325|73.03|44.06|false|wendy ovid|2013-03-01 09:11:58.703227|43.78|study skills +10|257|65610|4294967355|98.00|38.84|false|alice xylophone|2013-03-01 09:11:58.703273|42.48|quiet hour +86|351|65686|4294967425|84.11|39.33|false|gabriella robinson|2013-03-01 09:11:58.703282|20.88|wind surfing +19|403|65539|4294967472|88.14|2.12|false|alice quirinius|2013-03-01 09:11:58.703310|18.50|topology +34|458|65636|4294967412|36.37|44.50|true|irene allen|2013-03-01 09:11:58.703307|81.74|industrial engineering +7|261|65645|4294967487|42.97|13.47|true|gabriella garcia|2013-03-01 09:11:58.703187|14.41|linguistics +103|276|65625|4294967500|91.30|45.96|true|tom young|2013-03-01 09:11:58.703147|31.27|biology +49|256|65713|4294967515|21.83|6.21|false|rachel ovid|2013-03-01 09:11:58.703166|52.58|biology +94|345|65778|4294967473|53.72|47.73|false|david robinson|2013-03-01 09:11:58.703096|1.10|opthamology +94|261|65609|4294967316|52.94|36.45|false|victor ovid|2013-03-01 09:11:58.703182|7.14|study skills +88|351|65677|4294967296|39.82|22.46|true|quinn miller|2013-03-01 09:11:58.703265|48.41|study skills +14|307|65707|4294967539|6.35|38.77|true|oscar thompson|2013-03-01 09:11:58.703141|30.01|biology +95|493|65554|4294967355|26.67|25.75|true|rachel hernandez|2013-03-01 09:11:58.703253|74.44|education +31|305|65591|4294967410|92.96|36.96|true|gabriella white|2013-03-01 09:11:58.703131|44.20|debate +72|473|65708|4294967509|79.29|19.98|true|priscilla polk|2013-03-01 09:11:58.703141|60.53|american history +103|392|65733|4294967344|23.73|22.43|false|holly steinbeck|2013-03-01 09:11:58.703073|35.40|kindergarten +56|378|65562|4294967348|57.65|46.35|false|ulysses davidson|2013-03-01 09:11:58.703259|72.47|xylophone band +59|334|65542|4294967379|93.08|31.53|false|fred nixon|2013-03-01 09:11:58.703257|61.30|nap time +119|442|65746|4294967522|84.87|42.87|false|zach falkner|2013-03-01 09:11:58.703258|33.31|education +72|263|65571|4294967492|2.80|15.63|false|wendy brown|2013-03-01 09:11:58.703317|34.11|linguistics +94|416|65554|4294967330|8.41|33.12|true|holly hernandez|2013-03-01 09:11:58.703251|11.78|forestry +58|496|65698|4294967540|16.54|1.19|false|oscar davidson|2013-03-01 09:11:58.703073|14.07|undecided +55|376|65735|4294967430|8.11|3.88|false|wendy allen|2013-03-01 09:11:58.703304|16.57|nap time +81|384|65726|4294967499|30.88|33.56|true|quinn van buren|2013-03-01 09:11:58.703193|81.03|mathematics +42|498|65759|4294967547|99.55|1.17|true|david hernandez|2013-03-01 09:11:58.703317|27.33|quiet hour +116|261|65785|4294967546|66.55|36.20|true|katie xylophone|2013-03-01 09:11:58.703282|27.79|undecided +12|423|65626|4294967382|46.96|2.15|false|holly ovid|2013-03-01 09:11:58.703111|30.23|xylophone band +5|395|65760|4294967466|22.86|48.04|false|tom van buren|2013-03-01 09:11:58.703245|41.55|undecided +75|343|65584|4294967432|38.27|15.81|false|zach white|2013-03-01 09:11:58.703093|60.13|yard duty +3|488|65604|4294967365|0.19|23.86|true|priscilla falkner|2013-03-01 09:11:58.703141|10.47|xylophone band +64|479|65701|4294967364|84.31|27.75|true|rachel white|2013-03-01 09:11:58.703313|82.39|debate +80|481|65657|4294967533|78.13|47.90|true|oscar carson|2013-03-01 09:11:58.703084|8.44|biology +71|398|65636|4294967389|86.41|34.43|false|yuri white|2013-03-01 09:11:58.703312|75.09|chemistry +18|423|65648|4294967365|42.57|17.65|true|nick falkner|2013-03-01 09:11:58.703265|75.84|xylophone band +60|487|65596|4294967492|69.80|8.80|false|alice davidson|2013-03-01 09:11:58.703273|97.15|american history +26|444|65673|4294967343|81.65|4.37|false|mike quirinius|2013-03-01 09:11:58.703271|23.79|kindergarten +66|272|65784|4294967516|5.97|3.68|false|xavier king|2013-03-01 09:11:58.703304|49.54|yard duty +12|321|65647|4294967382|19.80|38.05|true|rachel young|2013-03-01 09:11:58.703153|47.72|xylophone band +80|372|65611|4294967455|44.53|30.15|false|ulysses quirinius|2013-03-01 09:11:58.703149|55.02|xylophone band +74|315|65562|4294967452|29.61|48.28|false|sarah miller|2013-03-01 09:11:58.703133|39.94|linguistics +96|471|65600|4294967303|44.08|47.28|false|ulysses miller|2013-03-01 09:11:58.703146|97.42|forestry +1|503|65672|4294967524|54.01|18.40|false|wendy underhill|2013-03-01 09:11:58.703300|42.29|religion +116|298|65581|4294967357|33.15|14.54|false|oscar steinbeck|2013-03-01 09:11:58.703186|73.51|joggying +96|499|65663|4294967303|85.91|28.30|true|wendy garcia|2013-03-01 09:11:58.703112|78.41|religion +64|464|65657|4294967312|44.70|41.64|true|xavier allen|2013-03-01 09:11:58.703308|19.55|joggying +13|510|65790|4294967422|88.82|21.15|true|priscilla davidson|2013-03-01 09:11:58.703098|49.60|quiet hour +91|444|65544|4294967442|53.13|20.24|true|irene ovid|2013-03-01 09:11:58.703106|80.84|zync studies +49|296|65738|4294967481|0.03|32.31|false|calvin van buren|2013-03-01 09:11:58.703148|70.74|values clariffication +31|501|65647|4294967532|98.99|39.42|true|irene van buren|2013-03-01 09:11:58.703183|95.07|xylophone band +84|294|65706|4294967551|38.66|12.70|true|yuri king|2013-03-01 09:11:58.703229|61.05|philosophy +55|340|65637|4294967539|15.56|21.11|true|wendy polk|2013-03-01 09:11:58.703274|60.25|education +61|341|65724|4294967400|42.27|15.92|false|david johnson|2013-03-01 09:11:58.703265|90.36|linguistics +92|341|65613|4294967396|71.61|13.35|true|priscilla carson|2013-03-01 09:11:58.703277|54.43|zync studies +29|329|65620|4294967417|80.22|47.45|true|jessica allen|2013-03-01 09:11:58.703115|14.09|industrial engineering +57|411|65760|4294967434|17.41|47.99|true|quinn ovid|2013-03-01 09:11:58.703226|67.91|joggying +110|265|65684|4294967547|37.32|45.17|false|fred underhill|2013-03-01 09:11:58.703105|9.29|mathematics +66|401|65615|4294967381|64.93|5.48|true|luke falkner|2013-03-01 09:11:58.703081|99.05|forestry +96|408|65777|4294967444|29.43|24.01|false|zach allen|2013-03-01 09:11:58.703286|50.73|debate +116|278|65637|4294967339|49.54|32.57|false|victor young|2013-03-01 09:11:58.703191|51.89|american history +101|313|65672|4294967314|1.75|22.79|true|calvin hernandez|2013-03-01 09:11:58.703134|38.83|quiet hour +117|389|65640|4294967331|8.37|29.06|false|gabriella king|2013-03-01 09:11:58.703119|57.90|nap time +29|281|65551|4294967419|72.36|39.34|true|rachel carson|2013-03-01 09:11:58.703097|11.40|zync studies +108|422|65681|4294967473|70.36|7.36|true|holly polk|2013-03-01 09:11:58.703297|25.21|undecided +116|392|65745|4294967399|16.61|29.62|true|zach miller|2013-03-01 09:11:58.703101|38.16|chemistry +56|304|65568|4294967448|84.91|33.23|false|irene young|2013-03-01 09:11:58.703080|96.34|zync studies +112|476|65549|4294967454|43.32|28.70|true|wendy hernandez|2013-03-01 09:11:58.703272|10.48|mathematics +49|353|65575|4294967357|19.46|28.16|false|irene hernandez|2013-03-01 09:11:58.703154|70.45|biology +38|299|65667|4294967438|90.68|25.84|true|wendy king|2013-03-01 09:11:58.703166|99.78|xylophone band +40|273|65578|4294967467|31.88|2.84|false|nick falkner|2013-03-01 09:11:58.703093|17.27|forestry +33|466|65782|4294967503|68.62|42.19|false|luke king|2013-03-01 09:11:58.703071|65.18|forestry +27|280|65733|4294967537|71.08|47.51|false|xavier young|2013-03-01 09:11:58.703293|78.84|study skills +53|362|65712|4294967469|95.50|18.52|true|irene miller|2013-03-01 09:11:58.703157|45.83|education +89|386|65673|4294967527|78.16|28.45|false|quinn johnson|2013-03-01 09:11:58.703274|20.02|study skills +72|323|65556|4294967441|19.24|20.82|false|holly miller|2013-03-01 09:11:58.703211|74.07|yard duty +63|309|65582|4294967299|51.88|30.57|true|gabriella king|2013-03-01 09:11:58.703078|51.43|american history +1|488|65664|4294967318|77.87|15.14|true|gabriella underhill|2013-03-01 09:11:58.703183|94.30|history +61|470|65589|4294967403|4.94|43.93|false|ethan davidson|2013-03-01 09:11:58.703101|7.06|industrial engineering +9|322|65685|4294967440|40.87|47.79|true|zach xylophone|2013-03-01 09:11:58.703202|66.91|nap time +115|470|65768|4294967359|72.64|16.96|true|mike johnson|2013-03-01 09:11:58.703106|15.20|kindergarten +69|292|65607|4294967296|80.71|44.66|false|ulysses xylophone|2013-03-01 09:11:58.703252|79.02|education +121|305|65697|4294967498|42.06|41.62|false|jessica davidson|2013-03-01 09:11:58.703282|53.86|joggying +73|428|65743|4294967464|54.62|13.01|true|ulysses ellison|2013-03-01 09:11:58.703313|72.11|forestry +62|321|65791|4294967384|48.57|44.83|false|irene ellison|2013-03-01 09:11:58.703252|31.70|education +64|493|65721|4294967474|28.21|40.61|true|gabriella nixon|2013-03-01 09:11:58.703191|32.55|education +86|371|65676|4294967348|37.11|37.50|false|quinn white|2013-03-01 09:11:58.703103|82.06|mathematics +20|396|65553|4294967530|46.78|41.47|true|ethan hernandez|2013-03-01 09:11:58.703300|73.47|american history +62|278|65569|4294967460|7.70|15.89|false|holly quirinius|2013-03-01 09:11:58.703227|1.10|forestry +93|441|65736|4294967517|56.01|43.03|true|nick xylophone|2013-03-01 09:11:58.703210|65.79|quiet hour +37|416|65536|4294967476|59.33|21.37|false|alice robinson|2013-03-01 09:11:58.703231|1.33|forestry +97|492|65783|4294967424|90.96|48.56|false|ethan falkner|2013-03-01 09:11:58.703114|52.51|history +50|488|65658|4294967314|12.10|42.61|true|quinn johnson|2013-03-01 09:11:58.703076|89.29|values clariffication +85|267|65752|4294967303|92.46|38.54|false|mike davidson|2013-03-01 09:11:58.703207|3.48|wind surfing +28|412|65650|4294967408|25.57|39.52|false|bob allen|2013-03-01 09:11:58.703226|9.86|quiet hour +36|376|65718|4294967408|70.99|47.86|true|priscilla zipper|2013-03-01 09:11:58.703223|38.01|nap time +100|279|65774|4294967433|41.50|31.27|false|holly brown|2013-03-01 09:11:58.703182|30.81|american history +7|360|65598|4294967353|21.52|20.41|false|jessica zipper|2013-03-01 09:11:58.703074|59.37|values clariffication +47|302|65618|4294967351|60.17|19.69|true|fred steinbeck|2013-03-01 09:11:58.703101|69.13|religion +26|361|65768|4294967432|98.02|49.61|true|victor laertes|2013-03-01 09:11:58.703246|9.17|zync studies +101|276|65697|4294967546|13.88|17.09|true|ethan ichabod|2013-03-01 09:11:58.703131|85.26|geology +72|388|65617|4294967378|83.50|13.02|false|yuri steinbeck|2013-03-01 09:11:58.703253|60.23|study skills +43|416|65683|4294967484|26.13|11.02|true|zach nixon|2013-03-01 09:11:58.703197|92.79|xylophone band +4|434|65710|4294967302|40.00|0.27|false|katie ovid|2013-03-01 09:11:58.703297|23.91|history +29|373|65773|4294967491|42.63|30.15|true|david white|2013-03-01 09:11:58.703224|50.09|philosophy +49|411|65640|4294967473|94.27|17.40|false|luke steinbeck|2013-03-01 09:11:58.703071|52.59|philosophy +103|360|65774|4294967392|89.50|23.80|true|ulysses underhill|2013-03-01 09:11:58.703107|15.14|debate +123|346|65610|4294967487|7.05|26.46|false|jessica white|2013-03-01 09:11:58.703314|68.80|chemistry +87|362|65591|4294967372|54.98|33.32|false|xavier white|2013-03-01 09:11:58.703262|57.36|biology +94|305|65689|4294967325|88.83|47.32|false|david king|2013-03-01 09:11:58.703135|51.19|yard duty +124|433|65772|4294967390|4.20|16.48|true|zach white|2013-03-01 09:11:58.703278|29.37|american history +85|368|65657|4294967509|61.03|8.67|true|david king|2013-03-01 09:11:58.703075|52.02|xylophone band +18|406|65599|4294967519|21.22|35.88|true|victor van buren|2013-03-01 09:11:58.703269|7.19|linguistics +102|417|65568|4294967323|20.21|37.76|true|tom robinson|2013-03-01 09:11:58.703302|13.16|nap time +41|442|65783|4294967472|29.33|39.95|true|holly ellison|2013-03-01 09:11:58.703231|10.76|wind surfing +-1|301|65585|4294967365|42.68|22.99|false|david laertes|2013-03-01 09:11:58.703179|86.02|nap time +124|334|65747|4294967347|69.36|49.09|true|zach white|2013-03-01 09:11:58.703228|15.81|nap time +78|307|65703|4294967322|69.41|20.01|true|yuri falkner|2013-03-01 09:11:58.703157|46.87|values clariffication +12|345|65536|4294967326|16.02|11.58|false|wendy polk|2013-03-01 09:11:58.703239|65.17|quiet hour +2|508|65684|4294967510|34.61|1.35|true|holly robinson|2013-03-01 09:11:58.703180|31.07|yard duty +37|435|65562|4294967343|84.81|12.36|false|ulysses garcia|2013-03-01 09:11:58.703292|80.73|industrial engineering +107|436|65764|4294967319|47.40|20.27|false|david king|2013-03-01 09:11:58.703277|89.52|nap time +15|279|65661|4294967346|26.16|3.85|false|zach nixon|2013-03-01 09:11:58.703269|67.85|undecided +103|415|65580|4294967355|3.71|23.31|true|jessica johnson|2013-03-01 09:11:58.703149|96.59|values clariffication +8|331|65738|4294967497|19.75|31.29|true|luke ichabod|2013-03-01 09:11:58.703202|90.91|zync studies +63|429|65619|4294967481|94.86|1.13|false|xavier falkner|2013-03-01 09:11:58.703091|12.34|topology +4|438|65737|4294967302|69.77|45.19|true|irene falkner|2013-03-01 09:11:58.703079|83.15|nap time +91|427|65748|4294967401|91.02|15.20|false|david laertes|2013-03-01 09:11:58.703191|21.29|philosophy +54|498|65787|4294967324|61.73|31.27|true|priscilla garcia|2013-03-01 09:11:58.703273|34.61|linguistics +8|455|65767|4294967550|10.67|27.76|true|luke ovid|2013-03-01 09:11:58.703220|37.48|philosophy +124|282|65575|4294967354|73.46|42.88|true|gabriella allen|2013-03-01 09:11:58.703301|79.96|history +27|500|65743|4294967415|42.76|44.41|true|tom white|2013-03-01 09:11:58.703233|78.95|opthamology +14|333|65669|4294967313|36.13|42.80|true|calvin ovid|2013-03-01 09:11:58.703251|82.64|yard duty +70|511|65544|4294967527|64.95|3.08|false|victor miller|2013-03-01 09:11:58.703099|75.50|industrial engineering +30|480|65783|4294967414|16.88|0.53|false|ulysses garcia|2013-03-01 09:11:58.703076|83.44|zync studies +51|449|65788|4294967430|0.58|9.63|false|xavier polk|2013-03-01 09:11:58.703115|81.21|linguistics +7|289|65788|4294967346|42.12|47.35|false|irene xylophone|2013-03-01 09:11:58.703323|49.83|debate +51|411|65597|4294967511|77.48|28.77|true|fred johnson|2013-03-01 09:11:58.703180|17.52|zync studies +75|287|65758|4294967362|99.29|15.07|false|jessica king|2013-03-01 09:11:58.703160|28.11|forestry +106|474|65581|4294967432|94.66|40.05|false|rachel thompson|2013-03-01 09:11:58.703103|73.80|geology +42|306|65729|4294967425|46.71|24.87|false|yuri polk|2013-03-01 09:11:58.703189|62.26|american history +47|283|65607|4294967429|67.43|11.54|true|holly ellison|2013-03-01 09:11:58.703167|40.94|topology +10|280|65772|4294967336|23.44|0.72|true|katie zipper|2013-03-01 09:11:58.703283|2.57|values clariffication +42|439|65537|4294967536|7.75|29.08|false|jessica hernandez|2013-03-01 09:11:58.703216|47.75|xylophone band +26|391|65745|4294967413|86.46|6.69|false|gabriella falkner|2013-03-01 09:11:58.703105|83.29|joggying +98|411|65692|4294967472|5.19|46.37|true|zach falkner|2013-03-01 09:11:58.703135|12.78|study skills +88|474|65536|4294967394|8.86|49.89|false|xavier davidson|2013-03-01 09:11:58.703071|1.05|philosophy +93|377|65743|4294967431|3.76|40.73|true|katie falkner|2013-03-01 09:11:58.703145|12.07|geology +118|317|65575|4294967415|51.57|0.06|true|katie falkner|2013-03-01 09:11:58.703189|75.77|biology +75|333|65666|4294967304|22.20|21.15|false|fred zipper|2013-03-01 09:11:58.703217|99.58|zync studies +13|288|65536|4294967431|71.55|3.51|false|gabriella garcia|2013-03-01 09:11:58.703108|22.15|kindergarten +121|363|65671|4294967305|39.06|39.21|true|quinn king|2013-03-01 09:11:58.703323|99.37|quiet hour +54|332|65772|4294967407|51.76|12.97|false|mike underhill|2013-03-01 09:11:58.703226|3.51|study skills +70|266|65587|4294967526|59.65|49.52|false|rachel garcia|2013-03-01 09:11:58.703207|17.84|industrial engineering +46|480|65686|4294967541|32.34|15.68|true|katie quirinius|2013-03-01 09:11:58.703300|79.85|quiet hour +91|298|65683|4294967359|99.98|4.81|true|oscar king|2013-03-01 09:11:58.703166|0.92|yard duty +21|472|65747|4294967369|88.83|20.56|true|oscar ellison|2013-03-01 09:11:58.703242|99.42|quiet hour +4|290|65744|4294967506|61.82|1.18|true|irene garcia|2013-03-01 09:11:58.703259|84.01|forestry +40|346|65678|4294967540|40.90|10.24|true|sarah zipper|2013-03-01 09:11:58.703170|17.05|study skills +105|448|65555|4294967470|34.85|11.71|true|tom white|2013-03-01 09:11:58.703321|52.56|history +96|510|65602|4294967430|85.23|26.73|true|sarah allen|2013-03-01 09:11:58.703107|53.35|biology +43|495|65554|4294967532|45.09|11.72|false|nick johnson|2013-03-01 09:11:58.703079|48.06|xylophone band +9|427|65553|4294967505|27.62|38.95|true|luke zipper|2013-03-01 09:11:58.703183|37.05|linguistics +108|337|65551|4294967330|73.80|5.75|true|ethan nixon|2013-03-01 09:11:58.703209|18.08|american history +8|310|65599|4294967356|43.42|37.20|false|bob polk|2013-03-01 09:11:58.703112|97.11|yard duty +119|455|65773|4294967482|98.28|15.98|true|tom laertes|2013-03-01 09:11:58.703202|19.47|history +112|278|65601|4294967392|20.35|14.15|false|zach miller|2013-03-01 09:11:58.703136|92.93|forestry +72|284|65648|4294967458|54.91|32.75|false|fred ovid|2013-03-01 09:11:58.703119|41.45|nap time +-1|430|65590|4294967310|35.62|22.75|false|zach laertes|2013-03-01 09:11:58.703128|89.62|xylophone band +58|339|65705|4294967539|95.36|12.10|false|rachel garcia|2013-03-01 09:11:58.703324|19.78|philosophy +3|412|65547|4294967470|88.22|0.38|false|mike brown|2013-03-01 09:11:58.703251|0.27|zync studies +80|493|65757|4294967441|99.92|30.50|false|sarah ichabod|2013-03-01 09:11:58.703298|64.34|forestry +16|330|65551|4294967313|48.42|13.82|true|zach thompson|2013-03-01 09:11:58.703191|29.75|geology +88|377|65557|4294967358|26.20|25.38|false|luke xylophone|2013-03-01 09:11:58.703218|20.61|geology +115|411|65574|4294967505|99.58|12.87|true|calvin van buren|2013-03-01 09:11:58.703110|63.36|kindergarten +19|344|65738|4294967315|7.84|10.01|false|bob garcia|2013-03-01 09:11:58.703280|30.65|opthamology +109|288|65581|4294967455|32.08|30.23|false|priscilla laertes|2013-03-01 09:11:58.703256|49.71|chemistry +108|321|65759|4294967307|76.47|3.86|true|wendy thompson|2013-03-01 09:11:58.703105|51.10|joggying +20|368|65677|4294967329|6.20|32.57|true|quinn ovid|2013-03-01 09:11:58.703162|30.16|yard duty +40|322|65633|4294967540|59.93|14.00|true|oscar laertes|2013-03-01 09:11:58.703247|24.19|topology +101|409|65606|4294967430|28.95|12.18|true|holly ovid|2013-03-01 09:11:58.703290|66.62|industrial engineering +91|381|65589|4294967536|12.25|39.98|false|yuri ellison|2013-03-01 09:11:58.703249|83.89|undecided +78|369|65541|4294967360|90.85|10.22|true|luke carson|2013-03-01 09:11:58.703319|16.26|education +18|407|65536|4294967463|54.33|3.43|false|luke allen|2013-03-01 09:11:58.703094|11.31|industrial engineering +63|414|65598|4294967336|80.64|31.20|true|yuri xylophone|2013-03-01 09:11:58.703124|95.12|study skills +90|275|65594|4294967352|50.51|25.10|false|oscar brown|2013-03-01 09:11:58.703120|33.42|topology +56|433|65626|4294967491|40.28|44.60|false|katie miller|2013-03-01 09:11:58.703233|62.95|quiet hour +-3|314|65670|4294967330|13.67|34.86|false|wendy xylophone|2013-03-01 09:11:58.703191|84.76|mathematics +115|358|65627|4294967391|38.71|22.44|false|fred robinson|2013-03-01 09:11:58.703089|48.61|undecided +93|291|65774|4294967438|21.01|6.27|true|rachel robinson|2013-03-01 09:11:58.703071|25.74|american history +78|345|65664|4294967392|30.86|32.19|true|luke ellison|2013-03-01 09:11:58.703245|19.88|undecided +52|387|65702|4294967457|34.20|4.54|true|jessica xylophone|2013-03-01 09:11:58.703141|30.35|undecided +111|477|65683|4294967372|28.21|41.77|false|rachel miller|2013-03-01 09:11:58.703204|81.98|yard duty +55|392|65692|4294967536|50.84|27.04|false|wendy polk|2013-03-01 09:11:58.703137|23.67|topology +113|413|65725|4294967538|38.88|48.32|true|oscar ellison|2013-03-01 09:11:58.703317|67.30|topology +85|482|65772|4294967455|69.47|47.09|true|oscar ovid|2013-03-01 09:11:58.703162|62.89|xylophone band +108|434|65724|4294967402|39.01|14.10|false|sarah laertes|2013-03-01 09:11:58.703096|53.47|forestry +32|332|65771|4294967374|27.06|46.67|true|quinn young|2013-03-01 09:11:58.703289|16.38|forestry +68|387|65653|4294967484|12.61|32.29|false|katie robinson|2013-03-01 09:11:58.703151|18.23|religion +86|371|65602|4294967441|67.33|39.42|true|bob brown|2013-03-01 09:11:58.703258|2.89|zync studies +64|381|65765|4294967412|44.21|1.50|true|priscilla van buren|2013-03-01 09:11:58.703089|8.48|chemistry +48|289|65561|4294967439|18.66|44.60|true|katie quirinius|2013-03-01 09:11:58.703220|7.28|yard duty +103|268|65632|4294967476|68.35|11.36|false|holly falkner|2013-03-01 09:11:58.703239|65.78|mathematics +39|412|65549|4294967331|3.22|23.05|false|nick underhill|2013-03-01 09:11:58.703122|90.45|mathematics +104|411|65576|4294967308|70.48|32.41|true|david garcia|2013-03-01 09:11:58.703163|12.99|forestry +65|508|65555|4294967510|63.45|29.45|true|zach zipper|2013-03-01 09:11:58.703306|59.35|history +97|382|65573|4294967490|97.40|11.07|false|mike underhill|2013-03-01 09:11:58.703267|65.89|debate +74|257|65594|4294967319|78.39|20.30|false|holly robinson|2013-03-01 09:11:58.703199|57.94|debate +9|418|65698|4294967339|69.78|45.67|false|wendy davidson|2013-03-01 09:11:58.703151|89.34|values clariffication +118|329|65630|4294967356|50.86|34.43|false|quinn hernandez|2013-03-01 09:11:58.703144|81.76|yard duty +107|457|65705|4294967359|96.25|11.94|true|luke polk|2013-03-01 09:11:58.703124|70.63|debate +96|367|65748|4294967544|76.27|32.49|false|priscilla white|2013-03-01 09:11:58.703321|7.83|forestry +77|402|65663|4294967493|67.22|15.39|true|mike nixon|2013-03-01 09:11:58.703300|32.16|kindergarten +61|378|65547|4294967404|56.48|43.52|false|luke ovid|2013-03-01 09:11:58.703300|66.79|debate +73|350|65695|4294967330|56.91|24.91|false|yuri white|2013-03-01 09:11:58.703315|52.38|quiet hour +79|381|65783|4294967358|95.37|20.79|true|sarah white|2013-03-01 09:11:58.703303|67.23|study skills +48|295|65590|4294967349|84.51|35.69|false|gabriella young|2013-03-01 09:11:58.703073|7.11|nap time +38|454|65581|4294967486|9.21|5.20|true|katie robinson|2013-03-01 09:11:58.703188|11.10|history +11|484|65552|4294967452|79.68|20.47|true|irene young|2013-03-01 09:11:58.703246|84.08|geology +61|266|65586|4294967464|2.13|32.70|true|gabriella xylophone|2013-03-01 09:11:58.703073|15.63|debate +78|488|65640|4294967361|63.69|6.24|true|rachel underhill|2013-03-01 09:11:58.703075|60.22|forestry +57|369|65674|4294967506|96.84|3.10|true|calvin falkner|2013-03-01 09:11:58.703166|80.31|american history +43|416|65780|4294967323|95.31|9.12|false|rachel quirinius|2013-03-01 09:11:58.703281|65.85|values clariffication +65|472|65728|4294967513|71.98|49.80|false|yuri laertes|2013-03-01 09:11:58.703261|7.27|kindergarten +24|485|65788|4294967534|40.82|22.42|false|gabriella garcia|2013-03-01 09:11:58.703239|90.51|opthamology +98|493|65651|4294967488|40.95|15.41|false|oscar davidson|2013-03-01 09:11:58.703118|0.77|mathematics +27|389|65550|4294967538|8.74|32.82|false|wendy thompson|2013-03-01 09:11:58.703257|13.09|linguistics +69|490|65562|4294967418|12.05|44.88|true|ulysses king|2013-03-01 09:11:58.703184|34.63|linguistics +52|307|65639|4294967389|44.04|36.36|false|jessica young|2013-03-01 09:11:58.703309|63.72|philosophy +82|414|65548|4294967382|97.03|7.32|true|quinn nixon|2013-03-01 09:11:58.703173|87.71|xylophone band +17|319|65709|4294967327|1.52|40.23|true|rachel white|2013-03-01 09:11:58.703249|46.40|kindergarten +66|360|65736|4294967424|83.86|48.98|true|katie zipper|2013-03-01 09:11:58.703179|4.62|yard duty +55|412|65612|4294967396|15.87|17.48|false|oscar robinson|2013-03-01 09:11:58.703243|69.72|chemistry +70|299|65642|4294967306|60.57|13.48|false|zach white|2013-03-01 09:11:58.703324|43.51|religion +69|428|65790|4294967530|55.00|30.40|false|gabriella xylophone|2013-03-01 09:11:58.703230|48.55|education +13|295|65663|4294967363|69.36|41.50|false|calvin carson|2013-03-01 09:11:58.703201|23.82|kindergarten +54|278|65740|4294967478|84.11|3.81|false|zach carson|2013-03-01 09:11:58.703287|51.54|mathematics +49|394|65613|4294967438|85.13|29.62|false|holly zipper|2013-03-01 09:11:58.703111|85.31|industrial engineering +55|397|65543|4294967387|81.45|12.59|true|ulysses hernandez|2013-03-01 09:11:58.703195|78.66|study skills +21|289|65775|4294967446|48.64|38.87|false|mike nixon|2013-03-01 09:11:58.703259|14.35|chemistry +32|496|65595|4294967473|31.09|47.69|true|jessica brown|2013-03-01 09:11:58.703274|58.58|quiet hour +16|332|65628|4294967338|43.05|45.30|false|alice miller|2013-03-01 09:11:58.703114|85.75|zync studies +8|374|65677|4294967483|75.10|42.03|true|alice ellison|2013-03-01 09:11:58.703074|58.92|undecided +26|371|65729|4294967300|82.71|19.95|false|katie polk|2013-03-01 09:11:58.703095|6.78|opthamology +32|466|65560|4294967351|35.93|22.30|true|calvin white|2013-03-01 09:11:58.703077|1.82|linguistics +40|452|65576|4294967374|84.92|12.10|true|david zipper|2013-03-01 09:11:58.703159|35.97|quiet hour +23|311|65647|4294967459|20.42|4.20|true|alice white|2013-03-01 09:11:58.703129|73.98|debate +113|436|65724|4294967521|19.75|38.70|false|yuri davidson|2013-03-01 09:11:58.703224|81.68|joggying +53|504|65654|4294967327|89.21|17.39|false|yuri carson|2013-03-01 09:11:58.703119|13.77|wind surfing +33|272|65646|4294967305|78.35|4.06|true|sarah xylophone|2013-03-01 09:11:58.703209|64.77|biology +76|334|65584|4294967377|19.45|32.50|false|sarah xylophone|2013-03-01 09:11:58.703241|48.34|mathematics +27|440|65547|4294967516|68.07|37.22|false|david nixon|2013-03-01 09:11:58.703087|56.15|linguistics +7|319|65589|4294967337|4.90|15.73|false|calvin garcia|2013-03-01 09:11:58.703162|40.84|values clariffication +19|300|65596|4294967363|32.28|39.73|true|sarah steinbeck|2013-03-01 09:11:58.703244|72.90|xylophone band +119|273|65742|4294967356|25.61|6.80|false|irene laertes|2013-03-01 09:11:58.703197|91.96|chemistry +14|274|65547|4294967349|4.21|34.75|false|wendy carson|2013-03-01 09:11:58.703157|36.90|study skills +82|503|65728|4294967426|78.97|3.81|true|zach nixon|2013-03-01 09:11:58.703277|20.94|quiet hour +44|391|65547|4294967545|37.26|47.62|true|wendy garcia|2013-03-01 09:11:58.703103|66.58|biology +93|424|65559|4294967327|95.89|42.91|false|mike young|2013-03-01 09:11:58.703290|29.19|zync studies +49|484|65586|4294967501|9.62|30.73|false|ulysses ellison|2013-03-01 09:11:58.703301|86.39|xylophone band +79|402|65568|4294967357|69.44|33.11|false|david underhill|2013-03-01 09:11:58.703287|23.94|debate +68|456|65774|4294967500|77.97|19.46|false|ulysses white|2013-03-01 09:11:58.703092|99.08|opthamology +59|270|65551|4294967302|98.57|45.24|false|gabriella van buren|2013-03-01 09:11:58.703101|51.52|zync studies +36|477|65772|4294967470|54.07|22.43|false|xavier ichabod|2013-03-01 09:11:58.703081|73.41|industrial engineering +25|410|65699|4294967384|97.33|10.51|true|jessica brown|2013-03-01 09:11:58.703182|57.82|debate +77|443|65629|4294967470|98.55|8.08|false|tom falkner|2013-03-01 09:11:58.703121|54.46|topology +123|416|65582|4294967298|38.62|7.82|true|bob garcia|2013-03-01 09:11:58.703291|59.08|study skills +25|353|65659|4294967498|69.42|0.34|false|holly steinbeck|2013-03-01 09:11:58.703097|4.70|forestry +61|390|65648|4294967330|93.28|9.19|false|priscilla miller|2013-03-01 09:11:58.703229|68.25|zync studies +49|408|65603|4294967416|26.72|39.16|true|quinn laertes|2013-03-01 09:11:58.703081|32.43|history +46|506|65651|4294967437|87.85|11.04|false|holly carson|2013-03-01 09:11:58.703277|81.37|undecided +33|325|65596|4294967548|81.81|46.88|true|ulysses xylophone|2013-03-01 09:11:58.703175|59.01|forestry +56|353|65716|4294967341|39.30|3.71|true|sarah falkner|2013-03-01 09:11:58.703072|84.20|zync studies +67|280|65629|4294967467|87.52|18.87|false|tom garcia|2013-03-01 09:11:58.703142|18.24|debate +79|295|65695|4294967509|31.98|24.06|true|yuri davidson|2013-03-01 09:11:58.703076|92.32|quiet hour +97|427|65760|4294967456|95.94|26.41|true|david zipper|2013-03-01 09:11:58.703300|70.12|values clariffication +84|342|65609|4294967315|80.34|45.37|false|oscar allen|2013-03-01 09:11:58.703295|87.52|chemistry +41|257|65542|4294967486|62.59|31.51|true|zach hernandez|2013-03-01 09:11:58.703217|46.70|topology +13|361|65565|4294967407|71.65|35.34|true|yuri laertes|2013-03-01 09:11:58.703086|59.95|industrial engineering +90|451|65651|4294967360|49.50|17.33|true|priscilla garcia|2013-03-01 09:11:58.703237|30.34|philosophy +88|404|65545|4294967483|5.32|6.73|true|fred quirinius|2013-03-01 09:11:58.703219|8.98|nap time +15|301|65788|4294967455|75.74|34.18|false|mike ichabod|2013-03-01 09:11:58.703211|94.23|education +124|281|65621|4294967324|39.53|46.39|false|katie falkner|2013-03-01 09:11:58.703307|95.87|american history +-3|414|65608|4294967338|81.39|49.09|true|tom steinbeck|2013-03-01 09:11:58.703251|10.88|xylophone band +74|280|65580|4294967531|22.13|36.32|false|ulysses ovid|2013-03-01 09:11:58.703156|78.80|geology +82|306|65739|4294967394|10.77|17.72|true|holly van buren|2013-03-01 09:11:58.703179|57.99|religion +16|352|65655|4294967475|20.65|25.96|false|jessica falkner|2013-03-01 09:11:58.703320|94.83|nap time +109|330|65569|4294967545|81.83|20.00|false|victor ellison|2013-03-01 09:11:58.703312|20.65|study skills +122|345|65618|4294967372|76.27|27.41|true|irene johnson|2013-03-01 09:11:58.703225|84.07|philosophy +61|292|65752|4294967346|92.30|15.75|false|yuri johnson|2013-03-01 09:11:58.703099|5.96|biology +17|469|65548|4294967380|88.05|34.88|false|luke young|2013-03-01 09:11:58.703117|79.63|quiet hour +109|482|65581|4294967382|17.93|33.65|true|bob ovid|2013-03-01 09:11:58.703305|49.95|mathematics +66|486|65588|4294967351|87.77|16.95|true|calvin van buren|2013-03-01 09:11:58.703231|11.12|xylophone band +45|288|65602|4294967306|85.63|6.19|true|mike ichabod|2013-03-01 09:11:58.703290|7.10|topology +48|309|65733|4294967411|90.80|43.81|false|rachel king|2013-03-01 09:11:58.703165|91.51|debate +27|342|65703|4294967427|52.94|4.69|false|nick thompson|2013-03-01 09:11:58.703295|70.57|zync studies +7|350|65549|4294967520|5.95|43.47|false|rachel thompson|2013-03-01 09:11:58.703287|75.30|undecided +21|405|65551|4294967493|96.23|12.07|false|holly laertes|2013-03-01 09:11:58.703262|75.48|quiet hour +22|458|65632|4294967395|56.65|29.02|false|tom laertes|2013-03-01 09:11:58.703145|91.34|nap time +57|258|65603|4294967420|2.61|47.01|true|calvin johnson|2013-03-01 09:11:58.703180|88.73|forestry +25|364|65586|4294967500|6.60|5.01|true|wendy brown|2013-03-01 09:11:58.703119|66.11|undecided +94|384|65633|4294967431|75.09|20.98|false|priscilla nixon|2013-03-01 09:11:58.703242|87.41|wind surfing +110|302|65683|4294967408|96.24|43.53|true|ethan polk|2013-03-01 09:11:58.703265|61.81|nap time +92|363|65698|4294967465|97.05|19.93|true|zach xylophone|2013-03-01 09:11:58.703193|72.32|philosophy +72|414|65648|4294967441|81.28|21.00|true|zach carson|2013-03-01 09:11:58.703221|60.26|joggying +107|272|65699|4294967456|60.33|4.95|false|holly miller|2013-03-01 09:11:58.703204|55.49|forestry +120|385|65636|4294967467|21.06|23.65|true|rachel polk|2013-03-01 09:11:58.703099|25.83|wind surfing +10|431|65673|4294967433|41.27|20.39|false|victor garcia|2013-03-01 09:11:58.703198|99.05|yard duty +2|326|65584|4294967388|93.31|14.11|false|ulysses ellison|2013-03-01 09:11:58.703098|80.56|linguistics +123|339|65767|4294967374|11.38|35.21|false|irene van buren|2013-03-01 09:11:58.703274|68.64|xylophone band +91|333|65620|4294967296|12.35|3.38|true|alice zipper|2013-03-01 09:11:58.703254|21.50|wind surfing +99|381|65673|4294967490|93.02|21.87|true|ulysses underhill|2013-03-01 09:11:58.703132|96.89|religion +20|448|65641|4294967481|99.98|40.87|false|alice davidson|2013-03-01 09:11:58.703207|3.14|biology +96|256|65778|4294967319|16.29|48.48|false|ulysses young|2013-03-01 09:11:58.703277|32.09|quiet hour +24|407|65597|4294967351|2.85|21.57|false|sarah allen|2013-03-01 09:11:58.703178|98.70|forestry +17|428|65623|4294967414|1.13|23.04|true|bob brown|2013-03-01 09:11:58.703163|73.27|debate +45|447|65757|4294967436|65.74|12.29|true|katie ichabod|2013-03-01 09:11:58.703182|40.99|quiet hour +37|369|65703|4294967454|78.09|34.35|false|oscar robinson|2013-03-01 09:11:58.703108|66.19|linguistics +18|485|65751|4294967526|58.77|19.40|false|ulysses carson|2013-03-01 09:11:58.703072|27.63|linguistics +96|443|65542|4294967524|72.64|31.69|false|holly falkner|2013-03-01 09:11:58.703251|54.62|study skills +101|398|65607|4294967382|15.84|8.14|true|tom allen|2013-03-01 09:11:58.703302|56.41|forestry +62|335|65733|4294967322|11.19|25.37|false|mike van buren|2013-03-01 09:11:58.703133|48.27|undecided +52|317|65600|4294967444|73.81|38.01|true|rachel garcia|2013-03-01 09:11:58.703095|34.57|biology +8|476|65755|4294967443|62.33|22.18|false|jessica miller|2013-03-01 09:11:58.703101|82.64|mathematics +64|349|65620|4294967307|5.15|1.10|false|irene ichabod|2013-03-01 09:11:58.703203|80.09|mathematics +37|423|65594|4294967521|3.51|34.36|true|fred robinson|2013-03-01 09:11:58.703081|6.24|nap time +3|347|65782|4294967423|12.60|38.86|true|oscar quirinius|2013-03-01 09:11:58.703106|5.07|industrial engineering +51|328|65658|4294967465|2.74|16.75|true|ethan king|2013-03-01 09:11:58.703089|38.84|philosophy +33|260|65683|4294967352|1.61|19.55|false|katie king|2013-03-01 09:11:58.703151|39.61|philosophy +69|336|65654|4294967476|25.28|45.68|false|alice quirinius|2013-03-01 09:11:58.703097|98.13|forestry +107|423|65563|4294967491|65.43|31.74|false|yuri davidson|2013-03-01 09:11:58.703162|19.69|debate +16|261|65679|4294967330|18.22|4.04|false|katie quirinius|2013-03-01 09:11:58.703189|93.84|topology +96|470|65746|4294967339|9.14|40.15|false|holly falkner|2013-03-01 09:11:58.703258|25.15|quiet hour +77|474|65657|4294967471|90.39|29.76|false|priscilla underhill|2013-03-01 09:11:58.703287|36.51|kindergarten +46|423|65548|4294967517|1.61|7.86|false|tom xylophone|2013-03-01 09:11:58.703262|12.72|history +84|350|65604|4294967297|91.25|20.25|false|wendy ellison|2013-03-01 09:11:58.703163|2.83|yard duty +78|376|65656|4294967396|79.41|13.96|true|sarah xylophone|2013-03-01 09:11:58.703282|44.96|yard duty +22|380|65697|4294967304|1.08|12.91|false|katie underhill|2013-03-01 09:11:58.703206|41.70|history +44|503|65659|4294967499|33.37|38.04|true|quinn davidson|2013-03-01 09:11:58.703234|68.99|american history +23|432|65570|4294967491|74.51|31.01|false|calvin garcia|2013-03-01 09:11:58.703229|93.77|biology +1|462|65744|4294967419|87.16|42.25|false|fred johnson|2013-03-01 09:11:58.703134|48.50|history +102|430|65615|4294967315|31.94|46.49|true|ethan garcia|2013-03-01 09:11:58.703137|32.80|philosophy +32|263|65717|4294967352|27.56|8.35|false|rachel falkner|2013-03-01 09:11:58.703143|99.01|yard duty +61|340|65582|4294967447|66.95|14.66|false|xavier ellison|2013-03-01 09:11:58.703213|19.85|yard duty +121|353|65641|4294967362|68.77|30.82|true|zach polk|2013-03-01 09:11:58.703261|19.97|geology +13|364|65787|4294967340|33.88|32.50|true|holly van buren|2013-03-01 09:11:58.703128|90.59|opthamology +55|289|65723|4294967414|27.11|28.35|true|nick young|2013-03-01 09:11:58.703208|57.68|linguistics +36|443|65624|4294967347|97.24|9.94|true|calvin king|2013-03-01 09:11:58.703105|72.46|undecided +74|506|65784|4294967519|48.52|13.93|true|tom laertes|2013-03-01 09:11:58.703130|69.80|american history +93|320|65632|4294967411|64.22|24.77|true|ethan robinson|2013-03-01 09:11:58.703262|51.15|study skills +116|376|65614|4294967367|69.58|9.22|true|victor garcia|2013-03-01 09:11:58.703226|96.92|biology +121|277|65691|4294967366|83.10|3.68|true|alice steinbeck|2013-03-01 09:11:58.703166|76.51|topology +4|299|65622|4294967303|2.26|1.62|true|tom young|2013-03-01 09:11:58.703232|77.06|debate +56|422|65710|4294967432|73.47|37.69|true|calvin miller|2013-03-01 09:11:58.703081|14.28|kindergarten +0|331|65565|4294967458|14.60|21.76|true|david allen|2013-03-01 09:11:58.703167|67.20|zync studies +46|270|65627|4294967512|59.57|46.61|true|holly white|2013-03-01 09:11:58.703304|14.24|industrial engineering +76|278|65677|4294967471|33.07|16.59|true|xavier carson|2013-03-01 09:11:58.703084|60.35|quiet hour +123|350|65548|4294967473|4.83|6.77|false|ethan young|2013-03-01 09:11:58.703235|75.73|opthamology +13|344|65723|4294967544|17.66|6.85|false|tom brown|2013-03-01 09:11:58.703298|8.49|history +56|377|65761|4294967328|94.94|47.86|false|gabriella quirinius|2013-03-01 09:11:58.703317|49.26|chemistry +43|432|65558|4294967516|58.57|49.40|false|victor underhill|2013-03-01 09:11:58.703081|80.01|study skills +60|432|65746|4294967317|87.27|7.82|false|victor young|2013-03-01 09:11:58.703317|17.09|american history +54|288|65790|4294967481|12.03|23.16|false|ulysses nixon|2013-03-01 09:11:58.703289|95.67|education +80|508|65541|4294967456|37.40|11.24|true|priscilla steinbeck|2013-03-01 09:11:58.703081|47.13|mathematics +59|311|65710|4294967377|34.37|23.12|false|katie carson|2013-03-01 09:11:58.703307|99.33|philosophy +101|467|65593|4294967544|27.91|4.19|false|tom xylophone|2013-03-01 09:11:58.703176|12.48|history +76|460|65779|4294967302|86.19|29.73|false|priscilla ellison|2013-03-01 09:11:58.703169|28.71|chemistry +20|266|65633|4294967432|60.96|38.87|true|victor underhill|2013-03-01 09:11:58.703131|14.10|mathematics +114|449|65644|4294967380|63.38|21.23|false|calvin underhill|2013-03-01 09:11:58.703195|57.49|debate +94|487|65655|4294967324|34.76|3.87|true|oscar falkner|2013-03-01 09:11:58.703078|35.90|wind surfing +32|417|65683|4294967506|68.66|13.23|false|jessica young|2013-03-01 09:11:58.703142|82.83|joggying +17|460|65696|4294967478|3.87|40.26|true|priscilla brown|2013-03-01 09:11:58.703308|0.37|mathematics +2|498|65536|4294967509|65.54|44.95|true|zach allen|2013-03-01 09:11:58.703210|32.03|xylophone band +28|308|65662|4294967487|25.50|20.69|false|ethan garcia|2013-03-01 09:11:58.703222|78.36|linguistics +16|348|65744|4294967541|57.55|32.47|true|priscilla nixon|2013-03-01 09:11:58.703208|58.90|undecided +74|436|65738|4294967512|18.01|47.42|true|alice ichabod|2013-03-01 09:11:58.703113|39.48|undecided +67|338|65593|4294967442|7.61|28.67|true|tom xylophone|2013-03-01 09:11:58.703100|93.46|study skills +114|423|65742|4294967513|3.27|46.58|false|zach brown|2013-03-01 09:11:58.703233|8.57|education +32|491|65715|4294967520|56.74|15.12|false|gabriella ichabod|2013-03-01 09:11:58.703313|99.32|values clariffication +3|492|65684|4294967421|72.57|49.09|false|quinn ovid|2013-03-01 09:11:58.703117|20.21|wind surfing +95|479|65603|4294967320|17.57|4.74|false|david garcia|2013-03-01 09:11:58.703126|79.95|biology +75|441|65568|4294967420|5.25|4.23|false|zach ellison|2013-03-01 09:11:58.703287|36.41|industrial engineering +37|327|65702|4294967533|5.89|4.00|false|xavier white|2013-03-01 09:11:58.703312|59.54|chemistry +47|473|65562|4294967324|80.28|19.72|true|ulysses xylophone|2013-03-01 09:11:58.703238|12.48|joggying +122|435|65608|4294967483|33.35|21.33|false|zach falkner|2013-03-01 09:11:58.703227|44.56|quiet hour +110|310|65749|4294967451|55.01|11.81|false|david nixon|2013-03-01 09:11:58.703265|63.01|zync studies +13|379|65560|4294967446|16.48|12.99|true|zach carson|2013-03-01 09:11:58.703086|50.56|philosophy +122|334|65682|4294967397|39.73|31.14|true|ulysses garcia|2013-03-01 09:11:58.703278|93.87|undecided +76|501|65603|4294967519|60.32|29.51|true|david underhill|2013-03-01 09:11:58.703301|52.97|study skills +23|411|65632|4294967426|74.36|4.34|true|quinn xylophone|2013-03-01 09:11:58.703258|26.43|forestry +0|272|65563|4294967540|50.81|15.05|false|gabriella underhill|2013-03-01 09:11:58.703239|92.56|study skills +25|338|65743|4294967388|12.03|47.72|true|irene robinson|2013-03-01 09:11:58.703177|48.05|linguistics +34|361|65613|4294967336|37.37|18.35|false|rachel zipper|2013-03-01 09:11:58.703285|34.42|mathematics +107|504|65578|4294967497|35.57|18.31|true|quinn davidson|2013-03-01 09:11:58.703227|77.03|history +52|360|65710|4294967391|6.17|47.18|false|calvin quirinius|2013-03-01 09:11:58.703271|44.96|quiet hour +21|361|65712|4294967312|75.11|17.90|false|bob ichabod|2013-03-01 09:11:58.703124|31.73|geology +-1|321|65714|4294967417|36.00|39.21|false|oscar carson|2013-03-01 09:11:58.703273|13.36|opthamology +55|391|65697|4294967446|59.97|9.13|false|david ichabod|2013-03-01 09:11:58.703223|41.39|study skills +109|456|65737|4294967307|91.31|29.90|false|priscilla underhill|2013-03-01 09:11:58.703157|3.07|geology +121|473|65740|4294967545|24.22|37.83|false|quinn white|2013-03-01 09:11:58.703178|31.59|debate +28|290|65662|4294967506|42.16|22.91|true|ethan young|2013-03-01 09:11:58.703242|74.97|education +71|373|65641|4294967396|93.59|32.92|true|wendy zipper|2013-03-01 09:11:58.703113|58.83|wind surfing +50|276|65639|4294967547|7.28|11.20|false|rachel carson|2013-03-01 09:11:58.703296|76.65|nap time +96|358|65784|4294967363|11.76|25.14|false|tom xylophone|2013-03-01 09:11:58.703168|57.68|xylophone band +45|365|65644|4294967395|95.33|5.97|true|wendy van buren|2013-03-01 09:11:58.703279|41.81|nap time +16|393|65767|4294967479|15.82|37.36|true|ulysses hernandez|2013-03-01 09:11:58.703104|99.55|history +7|391|65568|4294967468|69.69|7.34|false|ulysses white|2013-03-01 09:11:58.703266|90.55|linguistics +121|346|65656|4294967449|44.95|3.38|true|sarah miller|2013-03-01 09:11:58.703157|44.69|values clariffication +121|316|65621|4294967431|84.59|44.77|false|xavier robinson|2013-03-01 09:11:58.703219|64.70|philosophy +41|411|65696|4294967406|61.90|18.09|true|rachel davidson|2013-03-01 09:11:58.703102|98.24|zync studies +107|474|65551|4294967361|34.69|25.54|false|ethan laertes|2013-03-01 09:11:58.703185|72.65|study skills +84|507|65781|4294967343|84.65|10.33|false|xavier carson|2013-03-01 09:11:58.703075|57.74|study skills +100|430|65684|4294967396|26.17|8.65|true|jessica xylophone|2013-03-01 09:11:58.703240|55.39|nap time +27|270|65567|4294967439|65.19|10.26|false|tom falkner|2013-03-01 09:11:58.703099|62.31|study skills +37|429|65554|4294967338|15.54|15.98|false|ulysses nixon|2013-03-01 09:11:58.703112|19.47|mathematics +66|429|65679|4294967333|13.70|5.68|true|victor ovid|2013-03-01 09:11:58.703212|24.02|values clariffication +62|363|65781|4294967390|72.14|5.81|true|katie polk|2013-03-01 09:11:58.703180|13.78|biology +57|462|65627|4294967500|49.88|46.85|true|gabriella brown|2013-03-01 09:11:58.703268|38.61|philosophy +102|306|65707|4294967417|2.04|13.10|true|priscilla johnson|2013-03-01 09:11:58.703138|54.47|linguistics +92|358|65702|4294967426|68.09|16.13|true|victor quirinius|2013-03-01 09:11:58.703277|36.81|zync studies +49|382|65573|4294967479|24.82|8.10|true|quinn carson|2013-03-01 09:11:58.703256|27.48|nap time +83|359|65742|4294967323|30.27|28.29|false|tom carson|2013-03-01 09:11:58.703267|47.68|opthamology +11|334|65537|4294967405|19.89|38.68|false|priscilla white|2013-03-01 09:11:58.703311|19.44|values clariffication +62|313|65573|4294967383|49.42|18.68|false|oscar johnson|2013-03-01 09:11:58.703120|3.47|xylophone band +93|496|65724|4294967529|56.65|3.56|false|david steinbeck|2013-03-01 09:11:58.703321|46.30|yard duty +48|329|65732|4294967318|97.21|10.08|false|alice miller|2013-03-01 09:11:58.703142|19.46|biology +64|464|65751|4294967518|92.05|1.51|true|katie nixon|2013-03-01 09:11:58.703122|64.19|religion +67|279|65654|4294967509|61.94|24.96|false|xavier brown|2013-03-01 09:11:58.703212|33.67|industrial engineering +82|439|65626|4294967467|59.85|17.75|true|gabriella white|2013-03-01 09:11:58.703212|59.63|wind surfing +96|343|65666|4294967302|97.79|27.79|true|quinn brown|2013-03-01 09:11:58.703159|89.54|zync studies +22|351|65648|4294967446|89.32|41.38|false|yuri nixon|2013-03-01 09:11:58.703119|24.69|chemistry +25|262|65637|4294967388|42.47|42.34|true|ulysses ichabod|2013-03-01 09:11:58.703247|89.55|religion +22|499|65643|4294967464|75.78|7.75|true|oscar allen|2013-03-01 09:11:58.703237|29.27|philosophy +23|314|65620|4294967506|38.94|41.27|true|holly ellison|2013-03-01 09:11:58.703291|39.80|values clariffication +4|317|65711|4294967490|82.04|27.10|false|oscar underhill|2013-03-01 09:11:58.703144|5.00|debate +47|385|65693|4294967517|76.75|49.85|true|gabriella underhill|2013-03-01 09:11:58.703218|44.18|linguistics +61|387|65669|4294967372|78.48|44.86|true|mike garcia|2013-03-01 09:11:58.703139|55.41|american history +87|302|65701|4294967338|70.98|42.47|true|gabriella hernandez|2013-03-01 09:11:58.703143|92.56|industrial engineering +44|278|65790|4294967297|30.27|31.11|false|tom king|2013-03-01 09:11:58.703227|3.12|education +38|429|65662|4294967298|20.33|39.54|true|nick underhill|2013-03-01 09:11:58.703172|19.84|zync studies +7|366|65667|4294967363|11.91|14.90|false|calvin xylophone|2013-03-01 09:11:58.703318|78.75|chemistry +107|307|65698|4294967465|31.85|33.55|true|quinn young|2013-03-01 09:11:58.703169|92.14|topology +30|354|65720|4294967499|74.40|15.72|false|ethan robinson|2013-03-01 09:11:58.703276|46.14|joggying +112|406|65661|4294967548|67.62|36.27|false|zach brown|2013-03-01 09:11:58.703299|2.64|chemistry +45|424|65733|4294967429|5.66|27.15|false|victor laertes|2013-03-01 09:11:58.703186|46.14|american history +48|486|65568|4294967505|57.91|44.81|true|xavier davidson|2013-03-01 09:11:58.703227|32.83|values clariffication +37|271|65572|4294967331|13.32|28.25|true|sarah ichabod|2013-03-01 09:11:58.703130|24.73|study skills +15|440|65611|4294967546|37.19|21.68|true|rachel laertes|2013-03-01 09:11:58.703253|54.82|joggying +49|309|65596|4294967296|18.67|35.81|false|irene van buren|2013-03-01 09:11:58.703136|92.82|linguistics +102|349|65722|4294967306|92.21|24.15|false|ulysses brown|2013-03-01 09:11:58.703297|17.27|mathematics +43|407|65614|4294967515|82.29|15.26|true|sarah van buren|2013-03-01 09:11:58.703076|42.39|quiet hour +109|358|65650|4294967444|66.95|0.19|true|zach carson|2013-03-01 09:11:58.703246|71.43|mathematics +5|427|65574|4294967372|85.55|40.01|false|katie xylophone|2013-03-01 09:11:58.703071|18.68|topology +84|464|65622|4294967324|46.06|42.38|false|ethan garcia|2013-03-01 09:11:58.703312|54.20|quiet hour +114|409|65589|4294967509|67.63|28.10|false|sarah miller|2013-03-01 09:11:58.703201|19.54|xylophone band +106|492|65579|4294967471|37.35|7.92|true|mike xylophone|2013-03-01 09:11:58.703140|52.74|chemistry +37|407|65703|4294967544|65.10|41.23|true|david ichabod|2013-03-01 09:11:58.703083|61.55|quiet hour +49|500|65688|4294967482|94.77|21.32|true|victor miller|2013-03-01 09:11:58.703190|29.64|education +110|418|65568|4294967313|20.27|4.95|false|ethan miller|2013-03-01 09:11:58.703082|17.87|biology +100|369|65683|4294967478|66.04|27.25|false|jessica miller|2013-03-01 09:11:58.703215|88.80|opthamology +60|332|65696|4294967368|15.73|49.23|true|bob robinson|2013-03-01 09:11:58.703191|95.05|yard duty +97|321|65773|4294967326|13.35|3.95|false|mike king|2013-03-01 09:11:58.703245|10.32|philosophy +73|456|65574|4294967408|55.39|26.65|true|mike van buren|2013-03-01 09:11:58.703137|74.93|industrial engineering +19|505|65679|4294967331|42.43|33.01|true|holly robinson|2013-03-01 09:11:58.703233|77.84|religion +33|277|65647|4294967442|47.29|40.35|false|zach falkner|2013-03-01 09:11:58.703254|3.88|chemistry +55|402|65579|4294967511|83.18|48.47|false|priscilla laertes|2013-03-01 09:11:58.703108|63.55|yard duty +26|306|65569|4294967408|50.10|6.39|true|katie miller|2013-03-01 09:11:58.703122|64.46|linguistics +64|279|65777|4294967457|95.40|8.02|true|sarah brown|2013-03-01 09:11:58.703094|24.83|quiet hour +25|475|65663|4294967418|23.93|34.18|true|katie laertes|2013-03-01 09:11:58.703164|14.14|opthamology +1|470|65783|4294967339|25.36|40.58|false|xavier hernandez|2013-03-01 09:11:58.703292|2.36|chemistry +9|447|65694|4294967387|76.43|35.66|true|irene king|2013-03-01 09:11:58.703312|22.48|education +111|276|65731|4294967401|12.48|28.80|true|priscilla davidson|2013-03-01 09:11:58.703086|91.68|philosophy +91|435|65736|4294967486|17.69|17.48|false|gabriella underhill|2013-03-01 09:11:58.703282|81.54|biology +101|447|65652|4294967505|41.67|30.85|false|calvin laertes|2013-03-01 09:11:58.703089|89.95|values clariffication +67|343|65678|4294967460|31.59|21.41|false|gabriella white|2013-03-01 09:11:58.703220|4.55|opthamology +67|442|65728|4294967431|54.07|34.46|false|oscar ovid|2013-03-01 09:11:58.703101|95.69|industrial engineering +3|487|65714|4294967399|37.31|6.52|false|priscilla carson|2013-03-01 09:11:58.703308|65.23|mathematics +0|431|65764|4294967507|69.00|16.51|false|ulysses ovid|2013-03-01 09:11:58.703111|62.48|kindergarten +32|429|65730|4294967404|52.80|11.96|true|nick davidson|2013-03-01 09:11:58.703271|64.23|religion +114|503|65633|4294967337|30.80|4.33|true|luke ichabod|2013-03-01 09:11:58.703263|77.66|zync studies +63|360|65784|4294967406|81.93|1.55|true|oscar white|2013-03-01 09:11:58.703252|39.82|chemistry +17|324|65594|4294967504|78.08|49.50|false|quinn xylophone|2013-03-01 09:11:58.703248|54.03|biology +42|362|65647|4294967402|28.53|21.56|true|katie young|2013-03-01 09:11:58.703112|68.26|wind surfing +80|261|65536|4294967349|37.60|32.47|true|bob robinson|2013-03-01 09:11:58.703172|86.04|linguistics +21|455|65656|4294967366|94.35|49.48|true|ulysses polk|2013-03-01 09:11:58.703298|23.59|nap time +109|325|65748|4294967313|72.42|15.03|false|katie falkner|2013-03-01 09:11:58.703204|32.96|linguistics +44|266|65617|4294967474|43.54|26.02|false|bob carson|2013-03-01 09:11:58.703140|88.38|yard duty +18|467|65540|4294967479|62.68|49.61|false|nick ovid|2013-03-01 09:11:58.703121|48.84|chemistry +84|284|65553|4294967506|13.27|43.79|true|gabriella johnson|2013-03-01 09:11:58.703269|25.36|wind surfing +60|471|65742|4294967302|68.22|37.67|true|sarah johnson|2013-03-01 09:11:58.703148|83.42|quiet hour +39|501|65568|4294967318|19.67|14.51|true|ulysses ichabod|2013-03-01 09:11:58.703323|35.21|wind surfing +-2|474|65553|4294967546|14.93|5.59|true|rachel polk|2013-03-01 09:11:58.703317|29.59|forestry +75|376|65787|4294967468|49.35|2.52|false|ulysses miller|2013-03-01 09:11:58.703246|37.94|wind surfing +124|315|65715|4294967341|61.29|34.11|true|rachel nixon|2013-03-01 09:11:58.703084|91.08|education +34|390|65657|4294967323|39.46|44.80|true|mike carson|2013-03-01 09:11:58.703188|24.47|values clariffication +117|412|65705|4294967355|37.95|37.98|false|ethan nixon|2013-03-01 09:11:58.703072|88.87|quiet hour +24|457|65603|4294967510|34.56|28.31|false|bob miller|2013-03-01 09:11:58.703076|30.85|xylophone band +11|380|65643|4294967421|11.56|44.56|true|quinn van buren|2013-03-01 09:11:58.703202|17.17|chemistry +113|276|65632|4294967487|64.87|12.11|true|ethan ellison|2013-03-01 09:11:58.703076|90.58|values clariffication +76|365|65732|4294967405|4.15|14.82|true|ethan laertes|2013-03-01 09:11:58.703280|97.99|religion +117|343|65606|4294967349|19.44|16.62|true|gabriella thompson|2013-03-01 09:11:58.703240|19.93|zync studies +121|449|65719|4294967497|1.40|37.03|false|yuri ichabod|2013-03-01 09:11:58.703184|69.62|undecided +39|311|65536|4294967428|88.01|3.81|true|ethan ovid|2013-03-01 09:11:58.703241|88.04|debate +23|337|65606|4294967401|7.33|2.17|true|fred van buren|2013-03-01 09:11:58.703275|76.17|mathematics +1|303|65632|4294967392|17.89|40.74|true|luke hernandez|2013-03-01 09:11:58.703142|16.76|philosophy +27|320|65700|4294967498|48.24|41.45|false|alice hernandez|2013-03-01 09:11:58.703102|5.42|zync studies +8|461|65634|4294967433|76.19|25.19|false|gabriella ichabod|2013-03-01 09:11:58.703079|45.77|biology +41|263|65627|4294967420|96.96|19.57|false|jessica steinbeck|2013-03-01 09:11:58.703177|47.81|history +62|309|65578|4294967480|24.19|5.02|true|nick king|2013-03-01 09:11:58.703259|85.50|topology +113|505|65703|4294967461|77.61|32.81|false|oscar underhill|2013-03-01 09:11:58.703239|23.17|wind surfing +120|384|65569|4294967451|98.23|43.46|true|priscilla underhill|2013-03-01 09:11:58.703319|64.65|chemistry +33|409|65562|4294967369|22.65|8.88|false|rachel nixon|2013-03-01 09:11:58.703231|41.28|undecided +118|421|65786|4294967500|51.68|19.82|true|irene carson|2013-03-01 09:11:58.703248|59.91|joggying +-2|436|65644|4294967357|87.39|45.26|false|quinn carson|2013-03-01 09:11:58.703307|29.72|american history +122|477|65640|4294967436|44.19|10.22|true|katie white|2013-03-01 09:11:58.703100|73.45|study skills +13|497|65539|4294967374|82.09|4.76|true|mike underhill|2013-03-01 09:11:58.703167|75.75|philosophy +2|321|65664|4294967411|67.86|14.48|true|victor garcia|2013-03-01 09:11:58.703082|98.29|chemistry +117|493|65659|4294967441|59.05|11.55|true|rachel polk|2013-03-01 09:11:58.703124|90.05|yard duty +-2|297|65724|4294967347|75.78|34.68|true|ulysses steinbeck|2013-03-01 09:11:58.703096|83.47|geology +58|348|65581|4294967448|90.86|7.14|false|nick hernandez|2013-03-01 09:11:58.703182|7.46|linguistics +111|381|65710|4294967345|25.47|17.08|true|zach johnson|2013-03-01 09:11:58.703299|79.09|quiet hour +32|433|65588|4294967544|96.84|34.45|false|sarah brown|2013-03-01 09:11:58.703296|94.13|history +70|394|65630|4294967528|69.22|42.64|true|sarah polk|2013-03-01 09:11:58.703142|7.04|biology +116|261|65599|4294967414|17.45|19.06|false|katie robinson|2013-03-01 09:11:58.703084|32.08|joggying +44|334|65621|4294967486|14.13|42.03|true|rachel ichabod|2013-03-01 09:11:58.703085|64.91|zync studies +17|391|65751|4294967310|0.76|9.71|true|wendy king|2013-03-01 09:11:58.703236|0.01|values clariffication +112|316|65604|4294967401|38.07|18.77|false|yuri steinbeck|2013-03-01 09:11:58.703105|30.21|debate +19|437|65670|4294967341|51.87|25.18|true|calvin ovid|2013-03-01 09:11:58.703080|64.44|forestry +7|393|65551|4294967516|82.21|39.45|false|oscar xylophone|2013-03-01 09:11:58.703179|87.59|kindergarten +83|492|65685|4294967422|4.10|32.29|true|fred ichabod|2013-03-01 09:11:58.703136|36.95|biology +35|477|65787|4294967400|47.81|41.97|false|gabriella quirinius|2013-03-01 09:11:58.703244|54.72|mathematics +96|370|65737|4294967546|1.17|49.23|true|ulysses laertes|2013-03-01 09:11:58.703156|42.28|nap time +104|492|65655|4294967481|3.47|22.25|true|quinn van buren|2013-03-01 09:11:58.703235|32.22|philosophy +90|275|65745|4294967335|31.57|30.15|true|yuri allen|2013-03-01 09:11:58.703243|77.57|chemistry +45|270|65776|4294967352|59.44|36.50|true|nick garcia|2013-03-01 09:11:58.703220|49.62|wind surfing +93|469|65692|4294967545|30.27|40.97|true|mike garcia|2013-03-01 09:11:58.703110|25.42|american history +63|308|65720|4294967448|88.16|49.93|false|ethan brown|2013-03-01 09:11:58.703294|22.67|religion +63|457|65563|4294967341|58.68|32.56|false|nick nixon|2013-03-01 09:11:58.703100|13.20|opthamology +93|486|65771|4294967386|64.19|23.67|true|david garcia|2013-03-01 09:11:58.703295|77.78|industrial engineering +66|283|65643|4294967461|91.45|5.61|true|yuri davidson|2013-03-01 09:11:58.703136|86.86|religion +102|486|65774|4294967425|63.08|10.05|true|katie quirinius|2013-03-01 09:11:58.703242|36.53|history +51|463|65682|4294967540|78.37|42.16|true|rachel underhill|2013-03-01 09:11:58.703182|91.09|quiet hour +19|270|65725|4294967524|96.40|21.54|false|david king|2013-03-01 09:11:58.703145|87.32|history +4|337|65650|4294967317|10.81|1.45|true|victor robinson|2013-03-01 09:11:58.703317|17.26|xylophone band +40|435|65538|4294967339|70.35|1.25|false|quinn laertes|2013-03-01 09:11:58.703129|2.03|yard duty +122|510|65695|4294967340|5.73|42.61|true|jessica brown|2013-03-01 09:11:58.703267|97.81|mathematics +26|365|65657|4294967543|36.78|2.80|true|zach ovid|2013-03-01 09:11:58.703269|83.54|geology +107|300|65611|4294967341|25.19|43.74|false|holly garcia|2013-03-01 09:11:58.703302|89.38|opthamology +99|394|65744|4294967395|59.38|33.97|true|david underhill|2013-03-01 09:11:58.703199|54.80|nap time +37|511|65614|4294967375|76.62|36.32|true|irene laertes|2013-03-01 09:11:58.703188|97.55|kindergarten +21|451|65700|4294967498|43.01|33.01|true|wendy carson|2013-03-01 09:11:58.703246|21.19|zync studies +85|341|65773|4294967391|87.73|15.31|true|luke quirinius|2013-03-01 09:11:58.703252|53.82|kindergarten +67|467|65761|4294967455|71.67|27.53|true|oscar laertes|2013-03-01 09:11:58.703317|57.17|zync studies +9|382|65714|4294967374|21.07|18.80|false|yuri white|2013-03-01 09:11:58.703223|59.48|industrial engineering +53|416|65599|4294967543|69.90|24.40|true|mike quirinius|2013-03-01 09:11:58.703225|82.67|american history +95|335|65574|4294967472|85.56|47.14|false|bob xylophone|2013-03-01 09:11:58.703087|62.70|undecided +70|343|65537|4294967439|43.14|44.95|true|gabriella ichabod|2013-03-01 09:11:58.703075|7.72|linguistics +121|268|65545|4294967360|17.32|9.29|true|rachel ichabod|2013-03-01 09:11:58.703268|33.03|zync studies +86|393|65696|4294967315|66.25|2.47|false|tom laertes|2013-03-01 09:11:58.703303|32.15|debate +99|316|65766|4294967339|37.96|44.43|true|xavier brown|2013-03-01 09:11:58.703282|42.60|education +75|478|65639|4294967492|73.46|18.96|false|jessica robinson|2013-03-01 09:11:58.703155|93.79|wind surfing +-1|299|65701|4294967390|57.44|19.29|true|luke steinbeck|2013-03-01 09:11:58.703278|55.23|opthamology +124|319|65632|4294967489|60.40|45.62|false|ulysses quirinius|2013-03-01 09:11:58.703197|60.94|values clariffication +54|273|65548|4294967539|68.28|46.18|true|jessica garcia|2013-03-01 09:11:58.703123|63.43|undecided +15|379|65624|4294967504|37.11|35.68|false|luke underhill|2013-03-01 09:11:58.703220|64.82|religion +83|299|65722|4294967441|80.19|3.41|false|mike hernandez|2013-03-01 09:11:58.703239|98.88|kindergarten +121|385|65785|4294967491|71.61|19.98|true|luke steinbeck|2013-03-01 09:11:58.703133|64.39|values clariffication +91|284|65633|4294967430|68.09|18.99|true|wendy steinbeck|2013-03-01 09:11:58.703098|74.52|geology +48|279|65635|4294967413|35.72|4.47|true|katie white|2013-03-01 09:11:58.703174|29.04|philosophy +13|467|65698|4294967394|52.21|49.31|false|nick miller|2013-03-01 09:11:58.703227|95.74|topology +12|436|65726|4294967484|9.51|44.64|false|david underhill|2013-03-01 09:11:58.703120|15.43|debate +88|485|65543|4294967301|96.79|48.09|true|ethan thompson|2013-03-01 09:11:58.703125|4.56|xylophone band +74|431|65701|4294967487|16.29|5.34|true|xavier steinbeck|2013-03-01 09:11:58.703202|1.48|philosophy +30|269|65595|4294967322|3.23|26.23|false|katie robinson|2013-03-01 09:11:58.703224|93.60|zync studies +116|323|65616|4294967467|48.56|34.57|false|alice miller|2013-03-01 09:11:58.703186|25.12|mathematics +22|436|65673|4294967425|85.18|1.16|false|victor king|2013-03-01 09:11:58.703171|46.87|topology +1|339|65715|4294967330|4.92|17.44|false|david steinbeck|2013-03-01 09:11:58.703151|55.81|quiet hour +53|500|65626|4294967373|46.26|23.70|false|luke xylophone|2013-03-01 09:11:58.703134|31.27|american history +12|437|65726|4294967366|89.69|31.32|true|calvin allen|2013-03-01 09:11:58.703160|5.28|undecided +78|353|65599|4294967456|36.23|3.14|false|holly white|2013-03-01 09:11:58.703183|76.31|philosophy +93|489|65567|4294967549|30.72|45.00|true|bob ichabod|2013-03-01 09:11:58.703193|57.93|yard duty +119|281|65685|4294967353|38.59|24.96|false|oscar zipper|2013-03-01 09:11:58.703286|40.43|forestry +38|447|65775|4294967490|46.29|43.87|true|zach allen|2013-03-01 09:11:58.703170|73.69|undecided +27|317|65749|4294967324|41.46|22.45|true|yuri ichabod|2013-03-01 09:11:58.703157|69.58|xylophone band +0|296|65710|4294967318|39.34|19.38|true|katie ichabod|2013-03-01 09:11:58.703129|83.92|undecided +93|459|65737|4294967455|64.36|21.89|true|calvin zipper|2013-03-01 09:11:58.703201|8.38|chemistry +5|464|65544|4294967490|16.55|34.62|true|wendy zipper|2013-03-01 09:11:58.703299|66.76|forestry +110|276|65542|4294967534|56.28|31.91|true|zach falkner|2013-03-01 09:11:58.703197|25.18|debate +89|363|65719|4294967532|20.93|18.38|true|rachel ellison|2013-03-01 09:11:58.703176|50.15|values clariffication +96|434|65567|4294967299|44.81|40.69|true|mike underhill|2013-03-01 09:11:58.703113|60.64|philosophy +97|478|65764|4294967379|29.39|48.04|false|mike polk|2013-03-01 09:11:58.703204|50.22|linguistics +90|438|65771|4294967348|2.74|19.71|true|jessica thompson|2013-03-01 09:11:58.703236|4.58|nap time +39|312|65598|4294967296|2.01|6.65|false|sarah miller|2013-03-01 09:11:58.703208|8.28|values clariffication +18|297|65675|4294967423|6.73|10.28|false|mike falkner|2013-03-01 09:11:58.703099|47.34|kindergarten +24|376|65753|4294967378|75.92|41.80|false|quinn allen|2013-03-01 09:11:58.703073|42.87|kindergarten +18|395|65644|4294967308|92.79|38.73|true|bob miller|2013-03-01 09:11:58.703164|41.19|chemistry +34|423|65742|4294967397|75.89|13.24|true|victor laertes|2013-03-01 09:11:58.703166|82.79|biology +124|509|65771|4294967449|65.33|17.63|false|sarah ellison|2013-03-01 09:11:58.703141|42.52|american history +60|345|65629|4294967446|63.71|42.98|true|xavier hernandez|2013-03-01 09:11:58.703267|93.66|study skills +96|260|65716|4294967305|90.77|44.69|true|ulysses carson|2013-03-01 09:11:58.703174|88.39|biology +98|269|65558|4294967358|71.26|25.18|false|rachel steinbeck|2013-03-01 09:11:58.703270|54.00|wind surfing +45|407|65615|4294967482|2.03|6.96|true|luke ichabod|2013-03-01 09:11:58.703197|90.16|linguistics +85|267|65610|4294967470|55.90|14.31|false|rachel laertes|2013-03-01 09:11:58.703135|77.40|zync studies +52|260|65718|4294967534|58.21|36.20|true|sarah robinson|2013-03-01 09:11:58.703147|3.36|joggying +33|381|65746|4294967542|83.57|38.17|false|victor robinson|2013-03-01 09:11:58.703296|14.00|american history +12|440|65556|4294967357|43.36|42.10|true|oscar robinson|2013-03-01 09:11:58.703166|56.06|joggying +93|332|65736|4294967334|82.21|27.73|true|ethan brown|2013-03-01 09:11:58.703121|99.28|study skills +63|394|65570|4294967511|49.37|22.36|false|ulysses falkner|2013-03-01 09:11:58.703306|87.97|american history +41|284|65680|4294967397|18.77|18.16|true|jessica van buren|2013-03-01 09:11:58.703267|48.51|american history +28|316|65748|4294967356|71.60|13.83|false|mike xylophone|2013-03-01 09:11:58.703208|69.45|wind surfing +111|333|65577|4294967444|34.09|5.52|true|nick polk|2013-03-01 09:11:58.703118|23.41|joggying +19|320|65621|4294967305|50.96|5.92|false|yuri davidson|2013-03-01 09:11:58.703319|33.37|biology +70|339|65600|4294967323|24.09|39.99|false|xavier ichabod|2013-03-01 09:11:58.703134|22.22|opthamology +123|409|65624|4294967434|23.09|36.54|false|yuri miller|2013-03-01 09:11:58.703170|63.39|debate +9|506|65593|4294967514|80.95|1.17|false|victor laertes|2013-03-01 09:11:58.703086|65.69|kindergarten +104|420|65584|4294967544|84.35|32.70|false|victor allen|2013-03-01 09:11:58.703219|39.70|study skills +88|339|65718|4294967440|5.31|31.81|true|yuri polk|2013-03-01 09:11:58.703191|70.48|forestry +25|271|65541|4294967416|55.12|41.11|false|calvin laertes|2013-03-01 09:11:58.703242|47.88|kindergarten +15|478|65742|4294967403|69.79|5.16|false|bob ovid|2013-03-01 09:11:58.703121|16.18|study skills +62|263|65742|4294967381|13.52|20.24|true|mike white|2013-03-01 09:11:58.703187|7.39|american history +24|388|65671|4294967546|55.41|16.56|false|quinn carson|2013-03-01 09:11:58.703158|24.92|undecided +39|436|65673|4294967400|87.19|33.25|true|zach brown|2013-03-01 09:11:58.703166|74.80|industrial engineering +42|343|65688|4294967420|95.12|22.99|true|victor davidson|2013-03-01 09:11:58.703246|81.22|industrial engineering +73|395|65685|4294967328|27.35|41.29|false|nick king|2013-03-01 09:11:58.703146|21.60|philosophy +107|456|65764|4294967533|43.91|4.08|true|fred steinbeck|2013-03-01 09:11:58.703092|25.89|yard duty +-3|469|65698|4294967357|47.51|49.22|true|david falkner|2013-03-01 09:11:58.703305|78.40|joggying +2|301|65662|4294967408|51.41|35.52|false|oscar davidson|2013-03-01 09:11:58.703287|60.59|education +33|466|65558|4294967324|89.36|0.30|false|tom robinson|2013-03-01 09:11:58.703184|76.17|yard duty +22|430|65624|4294967333|41.66|0.20|false|jessica quirinius|2013-03-01 09:11:58.703164|99.33|values clariffication +56|359|65624|4294967339|80.66|3.17|false|zach laertes|2013-03-01 09:11:58.703206|0.64|zync studies +24|447|65552|4294967522|5.18|13.93|true|victor polk|2013-03-01 09:11:58.703252|99.82|forestry +78|290|65710|4294967329|73.44|3.41|true|holly miller|2013-03-01 09:11:58.703305|8.74|debate +111|257|65691|4294967305|80.76|11.43|false|david brown|2013-03-01 09:11:58.703286|0.91|mathematics +6|354|65538|4294967425|97.18|33.07|false|oscar white|2013-03-01 09:11:58.703153|35.14|mathematics +15|346|65739|4294967324|48.80|24.44|false|priscilla brown|2013-03-01 09:11:58.703158|83.01|zync studies +-3|280|65769|4294967324|28.78|35.05|true|xavier ovid|2013-03-01 09:11:58.703148|43.31|kindergarten +32|336|65747|4294967505|73.98|36.53|false|david xylophone|2013-03-01 09:11:58.703088|55.07|chemistry +57|345|65771|4294967396|71.02|14.41|true|bob quirinius|2013-03-01 09:11:58.703183|48.77|industrial engineering +98|362|65604|4294967531|23.07|20.16|false|nick falkner|2013-03-01 09:11:58.703112|56.36|history +46|304|65559|4294967498|42.55|16.80|false|mike xylophone|2013-03-01 09:11:58.703130|61.89|study skills +88|335|65778|4294967389|43.60|48.56|true|tom xylophone|2013-03-01 09:11:58.703083|41.78|wind surfing +56|422|65643|4294967451|15.67|23.68|false|irene ovid|2013-03-01 09:11:58.703151|64.99|values clariffication +30|412|65657|4294967317|20.82|29.99|true|yuri ellison|2013-03-01 09:11:58.703231|97.56|joggying +107|463|65676|4294967415|93.46|36.48|true|victor young|2013-03-01 09:11:58.703162|51.71|geology +55|324|65564|4294967325|33.98|33.84|true|tom steinbeck|2013-03-01 09:11:58.703318|8.73|xylophone band +58|501|65746|4294967524|94.82|45.75|false|xavier steinbeck|2013-03-01 09:11:58.703307|17.75|kindergarten +81|286|65739|4294967514|8.48|17.12|true|ethan thompson|2013-03-01 09:11:58.703304|71.45|forestry +27|382|65665|4294967374|14.78|48.12|true|yuri allen|2013-03-01 09:11:58.703070|99.31|mathematics +22|456|65580|4294967461|7.70|16.18|true|wendy xylophone|2013-03-01 09:11:58.703281|11.65|geology +41|263|65659|4294967406|48.52|32.24|false|quinn underhill|2013-03-01 09:11:58.703191|9.99|joggying +11|410|65650|4294967377|27.85|42.01|true|oscar thompson|2013-03-01 09:11:58.703205|36.92|wind surfing +54|398|65678|4294967548|9.96|5.98|false|xavier hernandez|2013-03-01 09:11:58.703073|28.28|biology +69|411|65751|4294967501|49.76|8.02|true|priscilla falkner|2013-03-01 09:11:58.703073|10.51|philosophy +57|506|65741|4294967409|40.46|29.97|true|priscilla brown|2013-03-01 09:11:58.703086|35.83|linguistics +108|403|65646|4294967440|65.18|27.41|false|luke ellison|2013-03-01 09:11:58.703305|95.65|zync studies +54|349|65590|4294967490|89.48|7.96|true|mike thompson|2013-03-01 09:11:58.703188|1.79|geology +5|345|65784|4294967401|29.44|41.77|true|wendy quirinius|2013-03-01 09:11:58.703092|74.38|history +119|378|65578|4294967300|57.55|11.53|false|jessica davidson|2013-03-01 09:11:58.703111|50.26|quiet hour +76|319|65587|4294967505|99.48|35.28|true|ethan xylophone|2013-03-01 09:11:58.703206|97.50|industrial engineering +78|474|65783|4294967346|21.94|36.82|false|xavier ichabod|2013-03-01 09:11:58.703100|36.77|american history +53|322|65543|4294967308|94.53|47.92|false|calvin ichabod|2013-03-01 09:11:58.703131|72.63|undecided +11|396|65765|4294967462|17.35|2.12|true|jessica king|2013-03-01 09:11:58.703302|95.48|kindergarten +61|274|65709|4294967341|67.51|28.10|false|gabriella underhill|2013-03-01 09:11:58.703296|43.18|industrial engineering +116|286|65753|4294967336|18.17|49.49|true|luke allen|2013-03-01 09:11:58.703317|0.87|education +28|365|65743|4294967380|97.43|8.37|true|quinn brown|2013-03-01 09:11:58.703133|55.24|religion +34|498|65759|4294967323|64.43|9.69|false|xavier garcia|2013-03-01 09:11:58.703239|17.74|study skills +61|387|65604|4294967422|7.21|49.58|false|david falkner|2013-03-01 09:11:58.703234|49.12|debate +20|287|65649|4294967496|91.28|27.77|true|alice robinson|2013-03-01 09:11:58.703109|88.92|nap time +29|270|65745|4294967475|77.78|26.30|false|zach carson|2013-03-01 09:11:58.703299|49.88|linguistics +24|336|65627|4294967439|61.42|27.58|false|katie white|2013-03-01 09:11:58.703146|33.76|study skills +14|462|65545|4294967438|66.89|22.19|true|alice allen|2013-03-01 09:11:58.703252|65.02|geology +57|487|65746|4294967439|61.35|15.83|false|alice polk|2013-03-01 09:11:58.703092|0.29|linguistics +124|436|65790|4294967492|51.25|16.35|true|sarah robinson|2013-03-01 09:11:58.703105|28.65|values clariffication +7|395|65564|4294967335|5.58|41.46|true|yuri zipper|2013-03-01 09:11:58.703101|30.65|opthamology +74|436|65588|4294967465|92.26|21.42|false|bob ichabod|2013-03-01 09:11:58.703287|59.12|chemistry +54|457|65760|4294967490|97.99|32.83|false|jessica laertes|2013-03-01 09:11:58.703241|63.65|quiet hour +49|410|65703|4294967430|79.05|31.17|false|victor hernandez|2013-03-01 09:11:58.703205|22.00|study skills +15|274|65777|4294967403|26.90|35.11|false|priscilla van buren|2013-03-01 09:11:58.703174|8.35|values clariffication +109|377|65681|4294967339|28.35|49.39|true|victor xylophone|2013-03-01 09:11:58.703156|11.56|wind surfing +108|378|65784|4294967487|19.03|10.93|false|katie allen|2013-03-01 09:11:58.703267|80.61|xylophone band +64|379|65746|4294967328|17.78|3.21|true|alice garcia|2013-03-01 09:11:58.703235|82.79|opthamology +72|263|65592|4294967353|48.39|1.67|true|ulysses steinbeck|2013-03-01 09:11:58.703323|98.58|undecided +49|434|65690|4294967467|98.47|42.54|true|mike van buren|2013-03-01 09:11:58.703243|80.96|values clariffication +102|277|65782|4294967302|88.96|22.13|true|ulysses steinbeck|2013-03-01 09:11:58.703205|84.65|chemistry +76|381|65620|4294967549|74.27|42.80|false|ulysses underhill|2013-03-01 09:11:58.703213|28.05|quiet hour +107|374|65790|4294967331|14.53|29.73|true|jessica xylophone|2013-03-01 09:11:58.703168|19.24|biology +37|394|65594|4294967512|47.29|11.13|true|david underhill|2013-03-01 09:11:58.703170|34.59|history +61|354|65773|4294967547|85.17|19.17|false|zach xylophone|2013-03-01 09:11:58.703077|37.05|topology +37|291|65635|4294967404|15.00|46.39|false|holly quirinius|2013-03-01 09:11:58.703218|11.23|geology +94|274|65723|4294967400|18.79|46.26|true|yuri white|2013-03-01 09:11:58.703148|48.31|quiet hour +43|422|65692|4294967336|60.24|20.21|false|fred davidson|2013-03-01 09:11:58.703159|57.20|study skills +39|506|65592|4294967474|25.58|28.93|false|xavier ichabod|2013-03-01 09:11:58.703181|5.99|american history +58|351|65650|4294967303|4.66|31.07|true|oscar white|2013-03-01 09:11:58.703277|2.75|chemistry +-3|299|65763|4294967542|85.96|10.45|true|jessica miller|2013-03-01 09:11:58.703245|26.16|mathematics +-1|452|65647|4294967531|96.16|34.84|true|katie underhill|2013-03-01 09:11:58.703128|78.86|geology +108|263|65705|4294967375|75.94|48.56|true|ethan quirinius|2013-03-01 09:11:58.703245|0.56|values clariffication +115|303|65683|4294967453|73.53|24.06|true|zach van buren|2013-03-01 09:11:58.703137|81.44|philosophy +5|459|65729|4294967323|90.17|15.58|false|oscar robinson|2013-03-01 09:11:58.703179|20.35|american history +48|438|65657|4294967406|95.57|14.22|false|fred ichabod|2013-03-01 09:11:58.703078|87.02|chemistry +112|459|65784|4294967493|80.09|7.78|false|fred miller|2013-03-01 09:11:58.703131|56.05|industrial engineering +35|354|65727|4294967378|38.26|48.46|true|yuri steinbeck|2013-03-01 09:11:58.703207|64.75|biology +55|305|65612|4294967504|14.60|17.96|true|luke underhill|2013-03-01 09:11:58.703219|11.76|religion +121|368|65656|4294967349|51.33|2.47|true|wendy underhill|2013-03-01 09:11:58.703268|12.99|geology +55|302|65732|4294967544|81.75|22.59|true|irene hernandez|2013-03-01 09:11:58.703226|92.01|geology +77|363|65696|4294967350|25.62|8.94|true|victor miller|2013-03-01 09:11:58.703159|30.33|american history +91|285|65787|4294967504|83.54|25.10|false|ulysses van buren|2013-03-01 09:11:58.703241|86.20|nap time +21|296|65689|4294967417|54.19|25.71|true|quinn ovid|2013-03-01 09:11:58.703250|8.12|kindergarten +99|499|65731|4294967453|30.85|28.79|false|ethan ovid|2013-03-01 09:11:58.703206|89.61|values clariffication +52|358|65756|4294967441|73.96|43.49|false|katie van buren|2013-03-01 09:11:58.703171|6.85|geology +1|414|65734|4294967511|22.76|37.13|true|holly davidson|2013-03-01 09:11:58.703145|40.39|religion +66|469|65557|4294967409|49.97|18.29|true|xavier thompson|2013-03-01 09:11:58.703265|49.88|xylophone band +75|352|65774|4294967375|93.20|45.67|false|mike thompson|2013-03-01 09:11:58.703085|59.69|undecided +45|428|65770|4294967447|11.78|20.13|false|ulysses davidson|2013-03-01 09:11:58.703158|36.68|values clariffication +92|344|65732|4294967499|29.10|48.53|true|sarah underhill|2013-03-01 09:11:58.703155|36.25|american history +80|393|65699|4294967323|70.76|12.68|true|katie carson|2013-03-01 09:11:58.703091|43.11|geology +99|360|65737|4294967482|6.40|33.05|false|victor young|2013-03-01 09:11:58.703074|98.15|debate +97|420|65777|4294967365|38.21|19.62|true|irene hernandez|2013-03-01 09:11:58.703303|38.22|geology +89|370|65586|4294967457|55.05|29.44|false|holly underhill|2013-03-01 09:11:58.703146|62.65|forestry +48|360|65770|4294967425|27.06|40.94|true|bob robinson|2013-03-01 09:11:58.703266|6.60|mathematics +103|444|65566|4294967505|50.77|17.27|true|yuri ichabod|2013-03-01 09:11:58.703189|49.93|biology +83|422|65618|4294967453|54.95|7.94|true|tom johnson|2013-03-01 09:11:58.703164|20.60|values clariffication +15|415|65732|4294967357|39.15|18.28|false|zach steinbeck|2013-03-01 09:11:58.703242|88.42|chemistry +24|327|65557|4294967482|73.11|7.31|true|nick robinson|2013-03-01 09:11:58.703118|60.36|opthamology +50|316|65717|4294967398|28.34|24.17|true|xavier van buren|2013-03-01 09:11:58.703118|28.35|biology +113|292|65714|4294967430|24.19|14.77|true|calvin garcia|2013-03-01 09:11:58.703301|28.51|xylophone band +111|501|65669|4294967464|38.24|20.27|true|katie falkner|2013-03-01 09:11:58.703151|15.67|opthamology +121|302|65579|4294967317|82.42|4.73|false|rachel laertes|2013-03-01 09:11:58.703231|53.68|religion +117|433|65782|4294967475|68.41|19.64|false|sarah quirinius|2013-03-01 09:11:58.703158|60.97|american history +98|356|65739|4294967546|52.29|46.71|false|victor polk|2013-03-01 09:11:58.703235|19.84|quiet hour +12|400|65717|4294967384|42.23|18.33|true|nick brown|2013-03-01 09:11:58.703135|67.52|nap time +31|324|65764|4294967356|95.63|10.01|true|irene brown|2013-03-01 09:11:58.703319|99.28|wind surfing +119|474|65657|4294967446|3.07|20.94|true|tom garcia|2013-03-01 09:11:58.703269|58.80|undecided +22|397|65610|4294967376|23.65|26.29|true|zach davidson|2013-03-01 09:11:58.703169|3.93|debate +20|281|65633|4294967394|77.33|16.09|false|victor thompson|2013-03-01 09:11:58.703179|69.01|values clariffication +102|484|65546|4294967349|27.88|5.21|false|oscar laertes|2013-03-01 09:11:58.703252|3.87|xylophone band +90|286|65760|4294967433|80.89|4.16|false|yuri ovid|2013-03-01 09:11:58.703242|30.02|american history +107|460|65665|4294967501|34.52|10.71|false|victor polk|2013-03-01 09:11:58.703118|93.11|philosophy +53|452|65710|4294967508|28.67|44.47|false|priscilla allen|2013-03-01 09:11:58.703126|5.31|education +50|484|65698|4294967438|47.68|35.24|false|oscar quirinius|2013-03-01 09:11:58.703219|10.08|geology +72|373|65688|4294967473|12.32|46.18|false|yuri van buren|2013-03-01 09:11:58.703192|84.92|values clariffication +92|459|65712|4294967497|1.29|6.36|false|alice garcia|2013-03-01 09:11:58.703187|45.31|debate +36|357|65537|4294967442|37.99|26.51|false|yuri quirinius|2013-03-01 09:11:58.703289|66.30|debate +70|337|65650|4294967332|64.49|23.98|true|sarah hernandez|2013-03-01 09:11:58.703198|90.38|geology +61|355|65621|4294967419|65.03|1.39|true|bob underhill|2013-03-01 09:11:58.703157|3.49|biology +36|457|65587|4294967300|42.86|7.91|true|yuri underhill|2013-03-01 09:11:58.703129|0.40|mathematics +79|398|65680|4294967401|57.46|6.96|false|alice ichabod|2013-03-01 09:11:58.703320|11.21|debate +111|278|65698|4294967367|87.09|34.08|true|ulysses xylophone|2013-03-01 09:11:58.703244|57.91|religion +30|271|65624|4294967453|52.55|26.65|false|ethan allen|2013-03-01 09:11:58.703190|62.04|opthamology +30|271|65754|4294967315|61.75|3.07|false|irene thompson|2013-03-01 09:11:58.703233|38.11|chemistry +61|499|65787|4294967365|65.35|16.25|false|calvin ovid|2013-03-01 09:11:58.703313|12.65|xylophone band +96|354|65664|4294967340|39.81|17.77|false|nick brown|2013-03-01 09:11:58.703082|84.55|nap time +6|463|65766|4294967435|84.74|29.66|false|alice nixon|2013-03-01 09:11:58.703104|40.50|values clariffication +15|389|65577|4294967537|73.51|46.93|true|irene brown|2013-03-01 09:11:58.703295|10.16|mathematics +68|389|65790|4294967473|74.45|0.49|true|priscilla brown|2013-03-01 09:11:58.703273|63.77|linguistics +102|460|65709|4294967543|48.77|18.60|true|david polk|2013-03-01 09:11:58.703168|41.58|yard duty +76|440|65651|4294967453|73.13|29.16|false|ethan allen|2013-03-01 09:11:58.703130|56.74|mathematics +78|427|65626|4294967317|59.09|21.85|true|sarah falkner|2013-03-01 09:11:58.703116|24.36|values clariffication +124|490|65623|4294967304|89.12|11.92|false|luke polk|2013-03-01 09:11:58.703262|57.65|geology +38|363|65604|4294967494|35.28|46.80|false|zach van buren|2013-03-01 09:11:58.703270|2.97|topology +79|262|65649|4294967413|9.89|27.35|true|fred nixon|2013-03-01 09:11:58.703253|99.51|values clariffication +83|318|65616|4294967470|39.74|11.85|true|victor young|2013-03-01 09:11:58.703192|83.48|industrial engineering +69|346|65650|4294967306|91.81|16.62|false|alice xylophone|2013-03-01 09:11:58.703276|72.84|philosophy +70|379|65711|4294967437|78.36|12.90|true|rachel hernandez|2013-03-01 09:11:58.703152|76.65|chemistry +94|368|65722|4294967428|63.39|7.72|false|xavier zipper|2013-03-01 09:11:58.703104|56.37|education +73|321|65756|4294967503|98.99|32.49|true|rachel xylophone|2013-03-01 09:11:58.703297|6.54|linguistics +-1|262|65659|4294967490|64.95|38.79|true|rachel polk|2013-03-01 09:11:58.703275|24.14|joggying +97|310|65606|4294967462|33.69|17.35|true|yuri quirinius|2013-03-01 09:11:58.703106|54.03|linguistics +13|447|65551|4294967440|59.02|33.80|true|irene johnson|2013-03-01 09:11:58.703125|58.21|geology +-2|312|65536|4294967484|1.55|20.12|false|ulysses polk|2013-03-01 09:11:58.703120|85.25|kindergarten +32|375|65760|4294967511|78.93|23.54|false|victor hernandez|2013-03-01 09:11:58.703303|62.41|zync studies +-3|264|65776|4294967398|20.95|5.97|false|bob polk|2013-03-01 09:11:58.703128|92.91|joggying +-1|303|65577|4294967480|38.46|36.97|true|victor davidson|2013-03-01 09:11:58.703142|86.75|yard duty +-3|465|65551|4294967457|83.39|46.64|true|mike allen|2013-03-01 09:11:58.703292|53.46|values clariffication +112|429|65643|4294967500|2.26|12.34|true|yuri white|2013-03-01 09:11:58.703212|32.66|biology +31|313|65618|4294967332|16.71|13.63|true|david robinson|2013-03-01 09:11:58.703221|74.02|xylophone band +55|379|65593|4294967375|15.82|43.85|false|ethan falkner|2013-03-01 09:11:58.703265|16.59|zync studies +122|322|65762|4294967388|12.93|38.43|true|victor king|2013-03-01 09:11:58.703163|72.19|education +109|478|65653|4294967489|14.54|38.22|true|oscar van buren|2013-03-01 09:11:58.703100|24.16|chemistry +102|435|65621|4294967412|14.96|31.32|true|sarah hernandez|2013-03-01 09:11:58.703252|96.51|kindergarten +69|414|65785|4294967428|73.99|6.55|true|holly zipper|2013-03-01 09:11:58.703127|88.14|debate +24|283|65646|4294967332|47.78|40.82|true|zach young|2013-03-01 09:11:58.703266|58.08|values clariffication +80|265|65762|4294967477|28.27|9.75|true|tom ovid|2013-03-01 09:11:58.703320|71.66|forestry +31|400|65542|4294967437|42.48|43.86|true|mike zipper|2013-03-01 09:11:58.703180|2.55|wind surfing +23|411|65689|4294967495|69.55|10.98|true|holly ovid|2013-03-01 09:11:58.703234|33.20|nap time +108|345|65662|4294967355|36.64|31.60|false|victor underhill|2013-03-01 09:11:58.703321|7.95|philosophy +111|481|65765|4294967394|50.25|15.97|false|nick allen|2013-03-01 09:11:58.703263|91.76|opthamology +97|380|65564|4294967314|40.98|48.33|true|luke polk|2013-03-01 09:11:58.703214|46.32|philosophy +4|495|65566|4294967358|39.65|1.80|true|xavier quirinius|2013-03-01 09:11:58.703289|99.31|education +13|482|65642|4294967542|41.07|49.97|true|holly quirinius|2013-03-01 09:11:58.703323|97.24|topology +81|426|65721|4294967345|38.46|46.27|true|quinn davidson|2013-03-01 09:11:58.703270|63.30|nap time +124|302|65702|4294967499|53.31|25.68|false|nick johnson|2013-03-01 09:11:58.703075|47.53|geology +26|354|65563|4294967509|29.15|24.13|true|jessica polk|2013-03-01 09:11:58.703174|44.66|religion +72|480|65772|4294967495|76.62|19.51|true|victor zipper|2013-03-01 09:11:58.703325|55.04|topology +81|275|65743|4294967518|35.35|32.37|false|katie white|2013-03-01 09:11:58.703105|85.82|american history +-1|379|65705|4294967531|75.75|0.38|false|katie laertes|2013-03-01 09:11:58.703116|94.88|zync studies +3|362|65726|4294967481|81.47|45.45|false|jessica allen|2013-03-01 09:11:58.703117|99.90|industrial engineering +30|378|65672|4294967310|98.02|6.46|false|bob van buren|2013-03-01 09:11:58.703168|43.41|education +114|305|65575|4294967492|83.26|31.45|false|david nixon|2013-03-01 09:11:58.703291|75.07|biology +87|450|65721|4294967547|63.46|35.74|true|katie young|2013-03-01 09:11:58.703271|89.52|linguistics +83|350|65708|4294967360|61.33|33.11|false|ulysses thompson|2013-03-01 09:11:58.703225|18.32|industrial engineering +75|502|65577|4294967519|77.98|16.97|true|ethan garcia|2013-03-01 09:11:58.703273|93.81|nap time +13|270|65724|4294967542|52.97|32.55|false|gabriella carson|2013-03-01 09:11:58.703098|79.55|philosophy +25|441|65742|4294967438|1.27|24.71|true|sarah davidson|2013-03-01 09:11:58.703224|43.81|religion +67|277|65565|4294967438|3.02|25.51|true|david johnson|2013-03-01 09:11:58.703150|60.96|chemistry +65|320|65745|4294967311|54.25|49.60|false|alice ellison|2013-03-01 09:11:58.703296|78.33|chemistry +33|395|65625|4294967446|41.15|30.27|false|priscilla quirinius|2013-03-01 09:11:58.703162|27.04|geology +58|476|65762|4294967379|76.94|48.63|false|jessica robinson|2013-03-01 09:11:58.703230|75.64|mathematics +122|447|65586|4294967426|2.31|47.70|false|victor johnson|2013-03-01 09:11:58.703191|88.13|geology +41|350|65685|4294967339|44.53|31.67|true|mike laertes|2013-03-01 09:11:58.703169|18.35|nap time +64|377|65578|4294967303|17.58|22.91|false|yuri brown|2013-03-01 09:11:58.703071|35.14|topology +121|510|65563|4294967325|18.87|39.07|true|holly thompson|2013-03-01 09:11:58.703227|63.54|xylophone band +30|274|65663|4294967475|95.43|34.28|false|holly quirinius|2013-03-01 09:11:58.703140|53.69|education +3|346|65590|4294967327|53.82|30.76|true|rachel polk|2013-03-01 09:11:58.703239|58.64|topology +37|299|65755|4294967547|28.05|4.14|false|zach garcia|2013-03-01 09:11:58.703294|44.81|kindergarten +4|454|65673|4294967549|85.50|39.96|true|fred ovid|2013-03-01 09:11:58.703103|52.78|zync studies +23|276|65589|4294967441|93.58|35.33|true|ulysses brown|2013-03-01 09:11:58.703120|23.78|opthamology +45|393|65555|4294967497|23.89|45.08|false|bob ovid|2013-03-01 09:11:58.703286|59.29|biology +26|480|65585|4294967412|83.75|30.19|true|quinn falkner|2013-03-01 09:11:58.703205|60.85|chemistry +106|278|65722|4294967451|18.45|30.14|true|fred young|2013-03-01 09:11:58.703193|27.68|education +119|286|65658|4294967523|66.18|43.45|false|rachel van buren|2013-03-01 09:11:58.703092|66.34|nap time +75|354|65616|4294967478|99.39|38.84|true|rachel quirinius|2013-03-01 09:11:58.703125|83.48|geology +14|466|65620|4294967340|93.03|6.81|false|wendy ichabod|2013-03-01 09:11:58.703169|8.27|topology +9|409|65727|4294967362|96.55|28.80|true|ethan carson|2013-03-01 09:11:58.703113|90.45|mathematics +46|492|65728|4294967365|91.34|42.38|true|zach young|2013-03-01 09:11:58.703120|82.95|linguistics +-3|303|65617|4294967473|10.26|1.41|false|ulysses quirinius|2013-03-01 09:11:58.703189|83.66|chemistry +58|457|65600|4294967311|16.36|2.91|false|luke ovid|2013-03-01 09:11:58.703080|38.77|yard duty +97|273|65604|4294967397|52.77|18.64|false|nick van buren|2013-03-01 09:11:58.703265|11.69|religion +74|443|65605|4294967383|16.80|40.43|true|yuri steinbeck|2013-03-01 09:11:58.703234|79.42|philosophy +32|343|65547|4294967550|95.02|21.40|false|david hernandez|2013-03-01 09:11:58.703312|13.97|joggying +109|356|65564|4294967472|50.40|10.13|false|bob thompson|2013-03-01 09:11:58.703143|45.89|mathematics +52|264|65774|4294967452|41.31|34.34|false|wendy carson|2013-03-01 09:11:58.703150|42.89|forestry +29|490|65783|4294967492|99.83|14.20|false|victor miller|2013-03-01 09:11:58.703321|96.79|religion +86|263|65710|4294967386|14.63|29.51|true|wendy allen|2013-03-01 09:11:58.703100|21.60|values clariffication +49|416|65637|4294967525|72.02|30.24|true|tom ellison|2013-03-01 09:11:58.703260|73.32|xylophone band +13|258|65569|4294967358|66.81|26.25|false|oscar van buren|2013-03-01 09:11:58.703253|6.06|debate +76|389|65652|4294967375|19.32|0.73|false|victor ellison|2013-03-01 09:11:58.703292|48.98|quiet hour +64|375|65713|4294967328|31.41|26.19|false|irene quirinius|2013-03-01 09:11:58.703252|94.75|zync studies +15|314|65575|4294967542|98.05|34.69|false|tom steinbeck|2013-03-01 09:11:58.703094|24.86|opthamology +110|359|65740|4294967343|12.91|21.78|true|katie steinbeck|2013-03-01 09:11:58.703228|0.80|xylophone band +109|443|65759|4294967434|78.50|38.66|false|irene polk|2013-03-01 09:11:58.703319|38.70|xylophone band +114|282|65613|4294967331|52.71|22.88|true|sarah polk|2013-03-01 09:11:58.703165|70.10|philosophy +107|483|65721|4294967301|41.85|3.17|false|ethan laertes|2013-03-01 09:11:58.703147|10.43|xylophone band +119|425|65779|4294967431|32.20|19.33|false|nick thompson|2013-03-01 09:11:58.703279|93.98|joggying +97|299|65712|4294967537|41.62|21.48|false|tom davidson|2013-03-01 09:11:58.703235|48.20|philosophy +71|362|65758|4294967333|63.89|38.22|true|bob quirinius|2013-03-01 09:11:58.703283|27.35|geology +2|395|65725|4294967457|72.18|10.41|false|bob allen|2013-03-01 09:11:58.703157|86.85|american history +121|361|65552|4294967322|84.66|12.52|true|tom hernandez|2013-03-01 09:11:58.703129|82.39|yard duty +98|375|65758|4294967419|72.97|38.40|true|david steinbeck|2013-03-01 09:11:58.703310|26.12|geology +93|297|65738|4294967548|0.31|2.57|false|wendy miller|2013-03-01 09:11:58.703272|23.65|xylophone band +59|279|65614|4294967471|5.66|43.19|false|nick ovid|2013-03-01 09:11:58.703156|77.43|kindergarten +18|419|65658|4294967316|36.79|4.28|true|tom young|2013-03-01 09:11:58.703292|43.39|education +46|410|65758|4294967537|71.54|41.11|true|bob young|2013-03-01 09:11:58.703087|17.13|nap time +36|325|65750|4294967497|32.81|37.40|true|tom steinbeck|2013-03-01 09:11:58.703096|45.64|education +30|341|65684|4294967302|88.32|41.89|false|irene nixon|2013-03-01 09:11:58.703091|79.51|zync studies +15|281|65649|4294967340|58.36|33.60|false|katie allen|2013-03-01 09:11:58.703169|14.01|undecided +123|489|65649|4294967430|19.81|36.70|false|quinn underhill|2013-03-01 09:11:58.703170|5.72|opthamology +124|433|65553|4294967504|66.41|38.14|true|calvin white|2013-03-01 09:11:58.703322|81.85|nap time +84|265|65605|4294967425|47.35|23.22|true|fred ellison|2013-03-01 09:11:58.703165|93.65|education +90|398|65669|4294967328|79.19|49.75|true|bob quirinius|2013-03-01 09:11:58.703121|63.55|wind surfing +121|452|65596|4294967440|89.86|20.17|true|alice falkner|2013-03-01 09:11:58.703178|70.66|xylophone band +29|491|65727|4294967514|76.70|1.57|true|mike nixon|2013-03-01 09:11:58.703133|43.13|nap time +85|339|65715|4294967384|18.73|3.79|true|mike falkner|2013-03-01 09:11:58.703227|33.70|wind surfing +1|393|65638|4294967421|92.67|31.37|true|wendy garcia|2013-03-01 09:11:58.703120|50.10|mathematics +41|327|65683|4294967388|75.26|21.92|true|tom xylophone|2013-03-01 09:11:58.703216|71.67|religion +123|258|65761|4294967344|33.73|1.70|false|jessica falkner|2013-03-01 09:11:58.703293|83.96|quiet hour +96|474|65645|4294967300|81.72|6.00|false|priscilla laertes|2013-03-01 09:11:58.703237|1.68|debate +91|421|65638|4294967528|18.51|10.60|false|david falkner|2013-03-01 09:11:58.703194|76.00|chemistry +41|415|65788|4294967405|42.12|8.26|true|holly robinson|2013-03-01 09:11:58.703201|14.66|topology +32|324|65610|4294967441|87.80|30.14|true|tom king|2013-03-01 09:11:58.703161|2.26|xylophone band +15|375|65710|4294967300|56.05|40.95|false|irene king|2013-03-01 09:11:58.703274|36.32|wind surfing +16|443|65581|4294967398|61.59|24.44|true|tom garcia|2013-03-01 09:11:58.703283|82.70|undecided +-3|454|65627|4294967481|17.60|35.72|false|bob underhill|2013-03-01 09:11:58.703188|67.16|religion +114|505|65643|4294967486|16.91|29.82|true|calvin ichabod|2013-03-01 09:11:58.703243|20.46|philosophy +8|331|65786|4294967353|42.55|24.85|false|ethan xylophone|2013-03-01 09:11:58.703173|69.42|opthamology +84|343|65644|4294967391|29.08|28.57|true|nick white|2013-03-01 09:11:58.703230|50.04|linguistics +24|447|65730|4294967456|59.23|22.07|false|david van buren|2013-03-01 09:11:58.703255|20.34|zync studies +33|461|65583|4294967408|60.37|13.17|true|calvin johnson|2013-03-01 09:11:58.703276|75.68|philosophy +64|373|65726|4294967494|13.34|48.53|false|gabriella van buren|2013-03-01 09:11:58.703224|48.95|zync studies +75|329|65686|4294967380|93.00|20.88|true|victor carson|2013-03-01 09:11:58.703209|90.36|debate +17|422|65617|4294967480|73.48|9.74|true|bob carson|2013-03-01 09:11:58.703145|19.11|chemistry +112|457|65694|4294967516|88.27|18.54|false|ethan garcia|2013-03-01 09:11:58.703100|71.51|xylophone band +25|479|65746|4294967441|82.07|31.79|false|ethan underhill|2013-03-01 09:11:58.703320|12.77|values clariffication +55|341|65759|4294967432|10.27|34.77|true|xavier allen|2013-03-01 09:11:58.703141|52.60|joggying +119|291|65708|4294967467|76.89|9.62|true|holly allen|2013-03-01 09:11:58.703170|6.83|wind surfing +78|258|65568|4294967324|81.41|21.02|true|nick falkner|2013-03-01 09:11:58.703102|69.44|kindergarten +18|464|65569|4294967526|10.45|6.89|false|david quirinius|2013-03-01 09:11:58.703137|22.16|topology +32|314|65556|4294967307|37.32|24.70|true|katie zipper|2013-03-01 09:11:58.703071|76.54|chemistry +98|284|65780|4294967486|18.06|26.01|true|calvin miller|2013-03-01 09:11:58.703110|56.08|yard duty +63|352|65684|4294967319|21.15|26.39|true|luke white|2013-03-01 09:11:58.703162|22.35|industrial engineering +20|314|65773|4294967506|31.68|38.08|false|katie ichabod|2013-03-01 09:11:58.703176|88.99|debate +74|413|65540|4294967540|1.47|28.88|false|gabriella hernandez|2013-03-01 09:11:58.703177|62.62|opthamology +16|423|65597|4294967486|3.96|25.57|true|irene garcia|2013-03-01 09:11:58.703318|81.96|chemistry +71|287|65687|4294967489|14.08|39.44|true|tom thompson|2013-03-01 09:11:58.703166|78.11|forestry +77|500|65626|4294967340|96.67|46.47|true|fred ichabod|2013-03-01 09:11:58.703224|55.50|opthamology +123|409|65682|4294967480|16.15|6.57|true|tom ellison|2013-03-01 09:11:58.703205|47.37|chemistry +95|425|65784|4294967411|98.91|26.02|true|katie nixon|2013-03-01 09:11:58.703088|59.42|values clariffication +13|490|65702|4294967511|0.79|10.34|false|xavier allen|2013-03-01 09:11:58.703112|5.71|joggying +102|338|65614|4294967413|74.08|22.11|false|irene nixon|2013-03-01 09:11:58.703309|10.86|wind surfing +35|502|65702|4294967513|52.00|10.10|true|rachel ellison|2013-03-01 09:11:58.703214|99.82|chemistry +31|413|65737|4294967496|89.49|42.69|true|rachel carson|2013-03-01 09:11:58.703185|91.19|industrial engineering +46|444|65544|4294967402|23.62|25.23|true|holly xylophone|2013-03-01 09:11:58.703189|21.94|religion +73|470|65729|4294967548|59.74|17.52|true|bob ovid|2013-03-01 09:11:58.703083|53.22|geology +23|326|65680|4294967409|1.10|13.77|true|tom underhill|2013-03-01 09:11:58.703231|79.03|history +118|350|65721|4294967374|33.83|43.32|true|zach king|2013-03-01 09:11:58.703111|97.56|wind surfing +68|484|65694|4294967505|52.83|19.87|false|holly van buren|2013-03-01 09:11:58.703265|21.13|history +0|480|65549|4294967533|76.67|29.69|true|jessica quirinius|2013-03-01 09:11:58.703278|31.48|wind surfing +70|370|65697|4294967373|96.26|34.93|true|oscar young|2013-03-01 09:11:58.703082|18.02|values clariffication +117|276|65743|4294967511|38.68|24.47|true|mike xylophone|2013-03-01 09:11:58.703222|24.82|undecided +62|454|65537|4294967530|95.89|7.37|true|holly quirinius|2013-03-01 09:11:58.703313|35.86|joggying +36|415|65767|4294967348|71.20|39.67|false|jessica young|2013-03-01 09:11:58.703270|25.66|quiet hour +80|376|65789|4294967441|72.65|44.44|true|xavier ovid|2013-03-01 09:11:58.703178|24.40|quiet hour +82|382|65633|4294967344|32.42|34.84|true|david king|2013-03-01 09:11:58.703149|93.54|study skills +92|338|65693|4294967477|39.41|23.94|false|ethan carson|2013-03-01 09:11:58.703314|81.24|education +31|421|65784|4294967401|36.32|41.73|true|nick king|2013-03-01 09:11:58.703116|49.78|opthamology +32|345|65781|4294967337|83.81|26.62|true|irene allen|2013-03-01 09:11:58.703308|12.77|industrial engineering +67|324|65739|4294967378|27.67|33.85|true|tom underhill|2013-03-01 09:11:58.703183|33.37|geology +53|323|65701|4294967477|98.48|8.66|false|gabriella hernandez|2013-03-01 09:11:58.703277|60.56|forestry +3|482|65620|4294967335|18.02|6.41|true|zach underhill|2013-03-01 09:11:58.703205|86.41|mathematics +60|384|65634|4294967415|9.06|11.08|false|gabriella hernandez|2013-03-01 09:11:58.703165|9.29|philosophy +16|276|65604|4294967438|91.54|35.82|true|david davidson|2013-03-01 09:11:58.703295|75.76|joggying +74|390|65556|4294967457|72.78|46.07|true|bob falkner|2013-03-01 09:11:58.703311|35.03|zync studies +113|494|65604|4294967419|85.41|34.24|false|ethan thompson|2013-03-01 09:11:58.703125|86.72|american history +35|284|65758|4294967298|14.63|11.44|false|mike steinbeck|2013-03-01 09:11:58.703316|93.57|linguistics +46|420|65537|4294967419|67.13|3.07|true|katie robinson|2013-03-01 09:11:58.703224|1.12|study skills +77|385|65553|4294967370|34.37|12.87|true|oscar johnson|2013-03-01 09:11:58.703248|85.39|study skills +20|470|65556|4294967384|70.63|19.50|false|priscilla robinson|2013-03-01 09:11:58.703214|95.08|undecided +54|319|65735|4294967328|55.39|18.32|false|tom miller|2013-03-01 09:11:58.703300|68.09|linguistics +43|494|65777|4294967478|14.53|13.34|true|rachel underhill|2013-03-01 09:11:58.703319|61.88|zync studies +108|259|65648|4294967508|96.50|32.29|true|katie carson|2013-03-01 09:11:58.703162|18.00|xylophone band +104|486|65761|4294967353|9.26|5.85|true|fred quirinius|2013-03-01 09:11:58.703143|82.23|geology +57|435|65645|4294967498|29.43|6.68|false|ethan zipper|2013-03-01 09:11:58.703118|70.49|industrial engineering +2|300|65712|4294967491|64.66|39.70|false|ulysses robinson|2013-03-01 09:11:58.703283|27.95|debate +-3|275|65575|4294967441|38.22|2.43|true|sarah xylophone|2013-03-01 09:11:58.703112|93.32|wind surfing +25|282|65695|4294967390|37.20|28.19|false|zach allen|2013-03-01 09:11:58.703153|95.87|chemistry +47|473|65762|4294967340|22.60|29.87|false|sarah johnson|2013-03-01 09:11:58.703150|68.84|mathematics +96|383|65658|4294967444|4.36|43.54|true|gabriella xylophone|2013-03-01 09:11:58.703248|26.01|quiet hour +28|506|65766|4294967331|84.31|27.28|true|victor king|2013-03-01 09:11:58.703087|79.33|values clariffication +109|304|65741|4294967372|24.85|47.54|false|luke van buren|2013-03-01 09:11:58.703187|62.84|geology +95|260|65583|4294967441|5.02|35.40|false|fred laertes|2013-03-01 09:11:58.703230|5.21|study skills +51|313|65612|4294967416|36.06|43.73|false|alice ellison|2013-03-01 09:11:58.703173|69.28|industrial engineering +85|296|65594|4294967488|65.21|41.59|true|quinn garcia|2013-03-01 09:11:58.703155|57.80|chemistry +61|392|65678|4294967421|54.34|48.34|true|oscar robinson|2013-03-01 09:11:58.703125|1.74|zync studies +-2|291|65548|4294967484|66.75|49.86|true|jessica ichabod|2013-03-01 09:11:58.703233|77.77|biology +70|303|65646|4294967451|80.46|37.43|true|fred allen|2013-03-01 09:11:58.703324|98.99|biology +53|478|65758|4294967467|6.46|33.36|true|jessica ellison|2013-03-01 09:11:58.703098|35.98|religion +65|331|65557|4294967406|70.90|29.90|true|alice ellison|2013-03-01 09:11:58.703311|29.99|zync studies +24|436|65609|4294967522|4.76|42.97|false|jessica garcia|2013-03-01 09:11:58.703210|76.57|geology +1|418|65563|4294967351|15.42|10.86|false|rachel carson|2013-03-01 09:11:58.703259|84.29|biology +24|443|65570|4294967364|22.41|30.08|true|nick hernandez|2013-03-01 09:11:58.703220|41.59|debate +104|394|65712|4294967406|63.99|36.47|true|priscilla allen|2013-03-01 09:11:58.703086|74.39|geology +105|321|65658|4294967476|98.20|43.28|false|victor steinbeck|2013-03-01 09:11:58.703251|13.34|forestry +95|396|65542|4294967343|69.57|40.34|false|zach ichabod|2013-03-01 09:11:58.703196|51.95|history +68|402|65680|4294967369|13.64|27.54|false|rachel ellison|2013-03-01 09:11:58.703186|3.73|linguistics +61|319|65747|4294967520|59.23|32.68|false|nick young|2013-03-01 09:11:58.703306|19.07|topology +2|502|65710|4294967372|34.14|40.33|false|luke johnson|2013-03-01 09:11:58.703287|10.52|religion +98|368|65752|4294967479|13.99|46.90|false|gabriella johnson|2013-03-01 09:11:58.703292|42.30|industrial engineering +117|292|65590|4294967331|92.84|2.31|false|nick garcia|2013-03-01 09:11:58.703246|47.52|opthamology +95|378|65769|4294967513|26.82|43.65|true|xavier steinbeck|2013-03-01 09:11:58.703252|49.88|xylophone band +50|260|65592|4294967518|70.96|20.86|true|zach thompson|2013-03-01 09:11:58.703123|39.67|biology +92|494|65551|4294967340|20.20|30.00|false|calvin white|2013-03-01 09:11:58.703299|13.08|education +39|434|65594|4294967378|88.61|11.65|false|quinn johnson|2013-03-01 09:11:58.703305|55.18|nap time +61|271|65749|4294967365|37.51|41.76|true|victor carson|2013-03-01 09:11:58.703157|79.16|chemistry +70|475|65737|4294967423|51.35|49.03|false|luke robinson|2013-03-01 09:11:58.703079|24.60|education +18|451|65724|4294967396|77.96|18.30|true|luke garcia|2013-03-01 09:11:58.703103|37.38|wind surfing +0|372|65702|4294967398|2.00|47.09|true|quinn carson|2013-03-01 09:11:58.703300|71.25|mathematics +106|399|65645|4294967549|21.78|24.24|false|zach ovid|2013-03-01 09:11:58.703155|59.94|debate +96|388|65645|4294967404|2.12|13.02|false|ulysses nixon|2013-03-01 09:11:58.703319|49.43|biology +92|505|65540|4294967540|91.13|38.30|true|ulysses polk|2013-03-01 09:11:58.703173|67.54|wind surfing +95|367|65667|4294967304|15.20|20.34|true|mike robinson|2013-03-01 09:11:58.703283|54.75|debate +112|376|65611|4294967418|13.26|38.51|false|sarah falkner|2013-03-01 09:11:58.703082|23.44|philosophy +45|399|65583|4294967444|82.99|23.05|true|irene nixon|2013-03-01 09:11:58.703149|36.76|mathematics +114|350|65668|4294967371|95.01|13.77|false|sarah robinson|2013-03-01 09:11:58.703243|61.12|wind surfing +69|310|65728|4294967297|77.06|22.77|false|ulysses allen|2013-03-01 09:11:58.703213|5.93|zync studies +63|458|65741|4294967311|82.11|24.62|true|sarah brown|2013-03-01 09:11:58.703281|99.02|mathematics +8|362|65635|4294967392|71.28|8.25|true|rachel davidson|2013-03-01 09:11:58.703119|86.23|yard duty +94|405|65752|4294967447|63.56|21.59|true|bob laertes|2013-03-01 09:11:58.703243|90.66|linguistics +76|397|65658|4294967380|15.10|12.19|true|priscilla quirinius|2013-03-01 09:11:58.703302|20.51|zync studies +81|378|65555|4294967513|89.28|6.52|false|irene brown|2013-03-01 09:11:58.703294|0.36|opthamology +10|276|65763|4294967536|94.86|22.33|false|gabriella zipper|2013-03-01 09:11:58.703123|88.74|geology +123|346|65775|4294967509|99.68|42.13|true|priscilla van buren|2013-03-01 09:11:58.703140|83.66|undecided +37|259|65686|4294967485|48.55|43.03|true|gabriella white|2013-03-01 09:11:58.703235|48.86|history +116|374|65627|4294967494|17.59|26.69|true|nick davidson|2013-03-01 09:11:58.703310|87.05|geology +81|378|65745|4294967375|50.18|30.58|true|fred polk|2013-03-01 09:11:58.703309|56.44|industrial engineering +45|450|65791|4294967428|27.21|4.71|true|zach underhill|2013-03-01 09:11:58.703282|64.23|education +25|496|65737|4294967362|2.06|18.82|true|wendy young|2013-03-01 09:11:58.703291|63.14|topology +1|295|65554|4294967405|60.30|30.70|true|alice xylophone|2013-03-01 09:11:58.703105|35.46|joggying +74|506|65675|4294967372|67.79|13.05|false|gabriella quirinius|2013-03-01 09:11:58.703172|33.87|industrial engineering +15|394|65575|4294967508|27.63|41.23|true|xavier van buren|2013-03-01 09:11:58.703293|92.90|mathematics +61|369|65674|4294967456|44.68|8.02|false|priscilla young|2013-03-01 09:11:58.703272|60.04|biology +55|429|65634|4294967445|73.20|21.31|true|calvin miller|2013-03-01 09:11:58.703198|77.87|mathematics +122|467|65777|4294967303|12.26|31.46|true|quinn zipper|2013-03-01 09:11:58.703207|80.40|linguistics +123|476|65572|4294967377|90.21|23.59|true|jessica ellison|2013-03-01 09:11:58.703314|26.11|yard duty +22|331|65658|4294967480|3.75|26.95|true|katie polk|2013-03-01 09:11:58.703131|51.68|debate +75|316|65580|4294967450|21.03|13.77|true|rachel ichabod|2013-03-01 09:11:58.703076|93.93|values clariffication +96|391|65592|4294967384|28.60|19.76|false|ethan ellison|2013-03-01 09:11:58.703082|39.25|american history +91|507|65702|4294967394|68.64|22.97|false|ethan xylophone|2013-03-01 09:11:58.703186|41.87|industrial engineering +35|473|65600|4294967385|28.25|6.13|true|sarah young|2013-03-01 09:11:58.703146|22.76|nap time +14|352|65711|4294967326|74.01|36.65|false|nick king|2013-03-01 09:11:58.703162|63.36|opthamology +31|361|65745|4294967449|72.47|20.81|true|holly polk|2013-03-01 09:11:58.703153|33.22|zync studies +-3|458|65563|4294967315|62.77|41.50|false|alice king|2013-03-01 09:11:58.703247|2.98|mathematics +68|352|65778|4294967542|1.24|18.55|true|holly quirinius|2013-03-01 09:11:58.703311|56.32|philosophy +18|258|65606|4294967472|35.52|35.18|false|xavier zipper|2013-03-01 09:11:58.703174|13.86|education +82|483|65584|4294967433|86.06|4.17|true|gabriella hernandez|2013-03-01 09:11:58.703138|52.52|industrial engineering +15|297|65554|4294967312|68.55|29.54|false|ulysses nixon|2013-03-01 09:11:58.703247|54.41|undecided +48|377|65653|4294967346|26.08|11.25|true|david young|2013-03-01 09:11:58.703227|56.10|forestry +94|374|65786|4294967375|6.51|46.30|true|david quirinius|2013-03-01 09:11:58.703182|23.32|nap time +121|310|65651|4294967361|6.98|12.35|true|irene carson|2013-03-01 09:11:58.703119|24.87|joggying +54|398|65663|4294967490|64.78|13.95|false|calvin xylophone|2013-03-01 09:11:58.703156|76.28|philosophy +71|354|65669|4294967376|35.92|43.88|true|irene allen|2013-03-01 09:11:58.703078|18.13|quiet hour +96|507|65766|4294967545|17.84|27.31|false|rachel underhill|2013-03-01 09:11:58.703211|60.17|zync studies +29|434|65595|4294967423|22.02|45.72|false|gabriella king|2013-03-01 09:11:58.703117|82.98|geology +50|345|65670|4294967528|29.89|25.71|false|ethan robinson|2013-03-01 09:11:58.703295|38.91|chemistry +17|365|65591|4294967539|1.09|0.42|true|alice johnson|2013-03-01 09:11:58.703262|29.31|debate +114|356|65686|4294967475|86.14|32.44|true|mike hernandez|2013-03-01 09:11:58.703195|71.01|nap time +94|427|65580|4294967472|3.13|8.18|true|katie king|2013-03-01 09:11:58.703204|69.88|industrial engineering +58|457|65680|4294967350|7.12|27.18|false|david hernandez|2013-03-01 09:11:58.703315|31.41|joggying +17|485|65590|4294967469|34.14|2.71|false|ulysses underhill|2013-03-01 09:11:58.703121|95.71|values clariffication +89|470|65763|4294967468|74.12|44.19|true|katie white|2013-03-01 09:11:58.703136|49.31|undecided +66|433|65781|4294967438|87.55|48.96|true|mike allen|2013-03-01 09:11:58.703098|47.16|biology +71|408|65752|4294967483|61.49|2.33|false|bob xylophone|2013-03-01 09:11:58.703233|60.37|xylophone band +115|259|65545|4294967386|16.90|17.18|true|yuri van buren|2013-03-01 09:11:58.703131|28.30|chemistry +84|473|65536|4294967412|61.28|28.75|true|fred miller|2013-03-01 09:11:58.703319|44.41|joggying +66|460|65737|4294967347|89.32|47.63|false|bob miller|2013-03-01 09:11:58.703318|96.38|industrial engineering +43|409|65721|4294967339|96.90|35.35|false|calvin johnson|2013-03-01 09:11:58.703091|40.06|joggying +118|498|65564|4294967365|99.37|8.34|false|luke hernandez|2013-03-01 09:11:58.703198|38.88|education +98|296|65708|4294967388|55.82|47.44|false|victor ichabod|2013-03-01 09:11:58.703298|94.56|topology +39|449|65704|4294967532|54.19|2.61|true|calvin quirinius|2013-03-01 09:11:58.703144|28.93|religion +9|487|65616|4294967364|26.77|15.88|false|sarah xylophone|2013-03-01 09:11:58.703305|20.63|linguistics +112|482|65639|4294967429|45.01|5.26|true|nick hernandez|2013-03-01 09:11:58.703325|23.95|nap time +33|497|65729|4294967436|3.49|18.27|true|david allen|2013-03-01 09:11:58.703228|93.17|forestry +114|407|65783|4294967364|95.09|45.37|false|priscilla brown|2013-03-01 09:11:58.703213|44.61|yard duty +39|372|65625|4294967525|16.46|21.29|true|fred nixon|2013-03-01 09:11:58.703169|25.84|kindergarten +107|429|65764|4294967330|21.90|9.83|false|mike carson|2013-03-01 09:11:58.703271|39.20|topology +28|379|65697|4294967376|74.49|34.77|true|priscilla king|2013-03-01 09:11:58.703283|22.13|yard duty +34|504|65575|4294967548|70.27|15.45|true|yuri davidson|2013-03-01 09:11:58.703263|3.98|undecided +3|276|65617|4294967455|40.98|39.22|false|mike ovid|2013-03-01 09:11:58.703274|96.76|religion +23|404|65582|4294967524|64.62|37.89|false|xavier polk|2013-03-01 09:11:58.703249|92.42|xylophone band +44|363|65569|4294967468|34.06|11.12|false|david davidson|2013-03-01 09:11:58.703107|11.29|geology +83|277|65658|4294967540|91.19|43.74|false|xavier young|2013-03-01 09:11:58.703241|4.97|topology +110|341|65590|4294967505|26.24|5.91|false|tom allen|2013-03-01 09:11:58.703315|62.99|wind surfing +62|277|65537|4294967521|56.19|47.08|false|mike hernandez|2013-03-01 09:11:58.703164|7.52|american history +20|273|65543|4294967375|27.71|35.70|true|victor polk|2013-03-01 09:11:58.703285|7.93|joggying +84|289|65585|4294967353|72.30|3.90|false|priscilla allen|2013-03-01 09:11:58.703271|78.78|industrial engineering +55|431|65665|4294967433|67.21|48.43|true|oscar nixon|2013-03-01 09:11:58.703207|87.73|study skills +60|479|65647|4294967340|4.94|17.52|false|wendy thompson|2013-03-01 09:11:58.703100|37.41|biology +35|371|65588|4294967373|36.96|27.11|true|fred thompson|2013-03-01 09:11:58.703235|65.46|history +88|425|65690|4294967479|59.79|4.43|true|sarah quirinius|2013-03-01 09:11:58.703158|45.41|chemistry +6|488|65579|4294967498|41.45|31.31|false|sarah brown|2013-03-01 09:11:58.703210|58.41|kindergarten +111|352|65536|4294967348|15.73|14.02|true|priscilla garcia|2013-03-01 09:11:58.703174|28.49|values clariffication +84|265|65547|4294967390|86.42|49.77|true|bob carson|2013-03-01 09:11:58.703278|85.00|chemistry +50|311|65752|4294967368|1.46|6.43|true|katie garcia|2013-03-01 09:11:58.703186|94.01|study skills +33|442|65754|4294967542|61.21|20.83|false|priscilla polk|2013-03-01 09:11:58.703180|19.19|nap time +82|382|65568|4294967493|80.18|21.16|true|david laertes|2013-03-01 09:11:58.703307|64.42|mathematics +121|359|65720|4294967414|8.06|8.23|false|ethan carson|2013-03-01 09:11:58.703095|69.16|topology +0|286|65631|4294967320|44.76|14.30|false|tom laertes|2013-03-01 09:11:58.703184|78.32|mathematics +6|271|65671|4294967395|32.59|42.38|false|irene van buren|2013-03-01 09:11:58.703170|66.83|american history +3|364|65659|4294967459|35.58|4.75|false|tom hernandez|2013-03-01 09:11:58.703201|42.00|american history +26|308|65628|4294967535|93.60|2.11|false|katie ovid|2013-03-01 09:11:58.703189|71.72|american history +100|348|65645|4294967460|75.16|49.64|true|calvin king|2013-03-01 09:11:58.703268|42.40|yard duty +75|366|65592|4294967417|15.08|19.07|false|tom garcia|2013-03-01 09:11:58.703320|56.55|joggying +31|432|65592|4294967361|67.01|16.11|true|zach garcia|2013-03-01 09:11:58.703194|56.85|history +95|307|65612|4294967475|90.73|36.04|true|bob zipper|2013-03-01 09:11:58.703219|52.42|mathematics +8|484|65552|4294967489|31.96|30.52|false|luke polk|2013-03-01 09:11:58.703116|16.89|american history +106|343|65702|4294967495|99.78|42.16|true|gabriella brown|2013-03-01 09:11:58.703088|89.18|religion +0|461|65589|4294967313|52.33|17.70|false|tom steinbeck|2013-03-01 09:11:58.703092|1.56|philosophy +116|328|65758|4294967551|24.23|41.36|false|xavier quirinius|2013-03-01 09:11:58.703181|70.67|joggying +61|455|65648|4294967408|63.63|41.22|false|wendy thompson|2013-03-01 09:11:58.703313|51.08|kindergarten +64|445|65756|4294967546|49.78|36.40|false|nick carson|2013-03-01 09:11:58.703268|76.20|history +119|289|65706|4294967378|74.65|21.33|false|gabriella hernandez|2013-03-01 09:11:58.703095|4.17|history +118|430|65658|4294967487|47.68|28.93|false|quinn falkner|2013-03-01 09:11:58.703233|77.29|study skills +39|268|65655|4294967411|9.73|39.99|true|luke falkner|2013-03-01 09:11:58.703181|70.93|forestry +14|470|65594|4294967463|70.93|45.81|true|gabriella quirinius|2013-03-01 09:11:58.703290|4.18|forestry +90|495|65758|4294967426|51.81|42.82|true|tom hernandez|2013-03-01 09:11:58.703290|59.64|study skills +12|464|65769|4294967460|56.63|4.98|true|holly zipper|2013-03-01 09:11:58.703138|88.19|forestry +34|299|65545|4294967527|21.69|14.53|true|ulysses underhill|2013-03-01 09:11:58.703124|78.36|opthamology +13|493|65627|4294967359|6.44|28.08|false|alice quirinius|2013-03-01 09:11:58.703260|44.94|undecided +67|424|65741|4294967482|57.59|39.14|false|quinn davidson|2013-03-01 09:11:58.703129|31.99|xylophone band +22|382|65554|4294967385|19.23|44.21|true|victor robinson|2013-03-01 09:11:58.703138|49.01|zync studies +-3|381|65640|4294967379|59.34|7.97|false|ulysses ellison|2013-03-01 09:11:58.703197|32.06|undecided +39|496|65739|4294967484|73.71|8.12|false|yuri van buren|2013-03-01 09:11:58.703115|83.16|xylophone band +10|473|65781|4294967457|34.23|6.78|true|jessica miller|2013-03-01 09:11:58.703120|30.01|xylophone band +73|358|65701|4294967441|5.03|27.01|false|tom nixon|2013-03-01 09:11:58.703279|24.69|quiet hour +-3|279|65661|4294967536|25.50|0.02|false|wendy quirinius|2013-03-01 09:11:58.703266|74.92|undecided +29|485|65680|4294967493|86.55|3.84|false|gabriella steinbeck|2013-03-01 09:11:58.703102|57.65|nap time +31|359|65695|4294967494|20.78|32.65|false|david ovid|2013-03-01 09:11:58.703127|84.07|topology +81|269|65635|4294967502|17.85|10.21|true|irene polk|2013-03-01 09:11:58.703102|72.92|joggying +8|478|65552|4294967327|25.59|40.98|false|priscilla underhill|2013-03-01 09:11:58.703226|98.48|industrial engineering +115|389|65775|4294967388|31.43|49.78|false|bob miller|2013-03-01 09:11:58.703200|38.03|history +23|508|65622|4294967428|47.03|1.12|false|mike polk|2013-03-01 09:11:58.703212|76.95|quiet hour +43|443|65787|4294967304|63.63|13.10|true|irene nixon|2013-03-01 09:11:58.703222|85.60|undecided +30|463|65590|4294967373|64.44|48.41|true|holly ovid|2013-03-01 09:11:58.703156|59.52|kindergarten +43|415|65783|4294967401|90.90|6.62|true|mike king|2013-03-01 09:11:58.703265|83.10|religion +62|442|65652|4294967490|67.49|23.47|false|bob quirinius|2013-03-01 09:11:58.703315|6.30|kindergarten +1|258|65728|4294967477|9.05|13.13|false|tom laertes|2013-03-01 09:11:58.703159|7.03|opthamology +86|447|65669|4294967487|17.45|39.00|true|ulysses ichabod|2013-03-01 09:11:58.703289|89.08|history +94|266|65712|4294967447|32.63|7.23|true|holly white|2013-03-01 09:11:58.703177|52.48|philosophy +80|428|65766|4294967350|35.17|18.46|false|katie johnson|2013-03-01 09:11:58.703260|81.50|nap time +-3|485|65684|4294967483|11.83|8.04|false|david garcia|2013-03-01 09:11:58.703190|63.10|wind surfing +47|287|65730|4294967541|38.05|30.61|true|irene ovid|2013-03-01 09:11:58.703272|59.12|debate +8|325|65538|4294967398|7.43|2.29|false|tom polk|2013-03-01 09:11:58.703111|68.02|values clariffication +100|390|65676|4294967541|39.67|35.13|true|jessica ellison|2013-03-01 09:11:58.703325|82.68|undecided +27|448|65760|4294967387|28.56|13.86|false|sarah davidson|2013-03-01 09:11:58.703212|60.29|american history +53|374|65569|4294967484|88.81|11.86|false|fred laertes|2013-03-01 09:11:58.703305|97.07|forestry +87|267|65709|4294967405|24.59|31.09|true|calvin miller|2013-03-01 09:11:58.703083|46.39|joggying +110|419|65771|4294967424|59.16|21.73|false|oscar polk|2013-03-01 09:11:58.703157|34.59|education +52|371|65651|4294967329|30.87|30.55|true|david laertes|2013-03-01 09:11:58.703154|38.47|philosophy +13|422|65629|4294967301|8.06|39.36|false|nick johnson|2013-03-01 09:11:58.703166|63.49|history +-3|376|65766|4294967326|97.88|5.58|true|sarah zipper|2013-03-01 09:11:58.703289|49.38|study skills +39|349|65540|4294967447|1.00|6.83|false|xavier underhill|2013-03-01 09:11:58.703250|73.13|quiet hour +8|373|65680|4294967389|85.89|19.62|false|quinn underhill|2013-03-01 09:11:58.703199|86.60|industrial engineering +32|309|65565|4294967320|49.30|34.91|false|nick ovid|2013-03-01 09:11:58.703116|33.06|industrial engineering +44|401|65648|4294967444|80.63|8.30|false|jessica ichabod|2013-03-01 09:11:58.703251|23.42|undecided +108|292|65669|4294967412|30.49|21.37|false|gabriella johnson|2013-03-01 09:11:58.703295|7.16|undecided +57|465|65698|4294967483|81.77|0.58|false|calvin johnson|2013-03-01 09:11:58.703098|55.19|yard duty +61|283|65669|4294967509|55.47|17.70|true|nick falkner|2013-03-01 09:11:58.703085|44.21|history +123|315|65619|4294967354|33.21|17.18|false|david ovid|2013-03-01 09:11:58.703204|24.78|philosophy +68|508|65718|4294967375|12.37|2.88|false|oscar polk|2013-03-01 09:11:58.703288|3.41|religion +74|304|65596|4294967543|4.19|35.56|true|xavier robinson|2013-03-01 09:11:58.703305|24.53|nap time +43|479|65775|4294967549|55.14|19.00|false|xavier thompson|2013-03-01 09:11:58.703235|7.52|joggying +10|506|65652|4294967374|91.24|32.90|true|yuri ovid|2013-03-01 09:11:58.703303|51.85|history +69|312|65739|4294967350|14.78|35.55|true|david white|2013-03-01 09:11:58.703127|47.23|education +79|368|65655|4294967498|94.65|33.90|false|tom ovid|2013-03-01 09:11:58.703270|12.76|history +64|263|65545|4294967478|92.78|39.17|false|zach white|2013-03-01 09:11:58.703185|88.19|undecided +108|268|65713|4294967446|27.96|46.87|true|alice carson|2013-03-01 09:11:58.703180|31.18|geology +68|422|65660|4294967465|50.08|42.44|false|fred brown|2013-03-01 09:11:58.703110|56.85|values clariffication +30|376|65738|4294967427|46.46|21.15|false|sarah zipper|2013-03-01 09:11:58.703313|32.65|mathematics +52|270|65783|4294967516|30.17|19.65|false|ethan ellison|2013-03-01 09:11:58.703136|2.82|joggying +-3|431|65635|4294967500|29.06|0.34|false|calvin ichabod|2013-03-01 09:11:58.703213|28.54|undecided +103|384|65554|4294967429|52.45|8.38|false|ethan hernandez|2013-03-01 09:11:58.703246|53.94|study skills +107|308|65767|4294967543|19.28|29.12|true|ethan davidson|2013-03-01 09:11:58.703248|31.17|nap time +9|382|65572|4294967489|81.19|14.71|false|jessica thompson|2013-03-01 09:11:58.703282|95.63|history +81|421|65600|4294967429|23.34|47.85|false|ulysses miller|2013-03-01 09:11:58.703230|81.70|xylophone band +36|463|65615|4294967322|16.52|24.97|false|mike ovid|2013-03-01 09:11:58.703157|18.15|debate +108|506|65612|4294967465|78.04|26.54|true|fred laertes|2013-03-01 09:11:58.703280|70.70|kindergarten +5|470|65765|4294967395|66.84|10.51|true|alice nixon|2013-03-01 09:11:58.703076|84.60|forestry +43|334|65551|4294967469|37.63|0.51|false|tom white|2013-03-01 09:11:58.703218|76.19|xylophone band +19|289|65746|4294967441|46.97|11.20|false|holly falkner|2013-03-01 09:11:58.703165|78.48|joggying +39|439|65655|4294967400|87.07|22.04|true|priscilla ellison|2013-03-01 09:11:58.703293|36.11|forestry +72|336|65563|4294967383|2.96|1.42|false|wendy garcia|2013-03-01 09:11:58.703268|92.39|chemistry +122|485|65721|4294967513|69.19|37.22|true|bob carson|2013-03-01 09:11:58.703179|10.74|mathematics +94|427|65757|4294967542|93.46|26.06|false|david miller|2013-03-01 09:11:58.703100|19.04|undecided +60|436|65545|4294967484|15.47|33.75|false|priscilla zipper|2013-03-01 09:11:58.703103|76.35|kindergarten +67|452|65646|4294967511|57.37|46.07|true|mike allen|2013-03-01 09:11:58.703198|2.68|american history +118|438|65681|4294967549|94.07|34.44|true|sarah brown|2013-03-01 09:11:58.703227|23.73|joggying +109|334|65717|4294967423|70.67|1.76|false|mike thompson|2013-03-01 09:11:58.703124|2.07|forestry +38|454|65593|4294967503|34.29|32.47|true|holly davidson|2013-03-01 09:11:58.703250|14.89|debate +94|300|65760|4294967481|38.85|38.02|true|xavier young|2013-03-01 09:11:58.703295|6.84|biology +26|353|65762|4294967372|54.76|27.55|false|jessica underhill|2013-03-01 09:11:58.703256|77.21|values clariffication +109|268|65774|4294967538|87.94|38.79|false|quinn quirinius|2013-03-01 09:11:58.703247|35.94|topology +111|305|65624|4294967503|11.69|27.72|true|tom ichabod|2013-03-01 09:11:58.703149|87.71|chemistry +80|397|65725|4294967372|4.92|38.63|false|luke white|2013-03-01 09:11:58.703101|31.36|wind surfing +104|380|65581|4294967384|95.46|29.48|false|wendy ellison|2013-03-01 09:11:58.703141|19.95|biology +116|483|65614|4294967357|38.19|16.53|true|quinn miller|2013-03-01 09:11:58.703138|40.79|opthamology +24|432|65636|4294967515|36.68|28.36|false|ulysses polk|2013-03-01 09:11:58.703208|93.03|xylophone band +44|325|65592|4294967362|0.55|32.02|false|zach falkner|2013-03-01 09:11:58.703091|61.19|study skills +103|290|65684|4294967471|38.34|44.56|false|irene zipper|2013-03-01 09:11:58.703202|76.02|quiet hour +48|288|65601|4294967459|20.72|35.64|false|holly ellison|2013-03-01 09:11:58.703123|89.60|linguistics +-1|326|65743|4294967544|27.51|22.35|false|sarah king|2013-03-01 09:11:58.703257|61.58|education +117|258|65584|4294967364|64.92|24.54|false|nick xylophone|2013-03-01 09:11:58.703302|99.23|chemistry +5|333|65634|4294967439|19.18|28.29|true|wendy van buren|2013-03-01 09:11:58.703173|30.33|forestry +46|415|65769|4294967314|97.45|20.92|true|priscilla garcia|2013-03-01 09:11:58.703254|27.44|religion +39|479|65538|4294967469|5.38|32.31|true|holly falkner|2013-03-01 09:11:58.703250|81.59|opthamology +124|492|65773|4294967493|68.32|23.80|true|alice robinson|2013-03-01 09:11:58.703088|27.25|study skills +24|261|65718|4294967405|25.69|6.05|false|luke robinson|2013-03-01 09:11:58.703139|7.74|topology +80|389|65604|4294967424|7.29|25.86|true|calvin ellison|2013-03-01 09:11:58.703242|96.60|industrial engineering +9|481|65610|4294967360|39.48|16.29|true|katie white|2013-03-01 09:11:58.703070|37.30|history +3|436|65752|4294967341|66.38|44.66|true|ethan robinson|2013-03-01 09:11:58.703314|58.39|history +93|439|65713|4294967398|6.96|26.35|true|calvin miller|2013-03-01 09:11:58.703287|67.04|industrial engineering +70|371|65549|4294967405|88.19|42.58|true|fred hernandez|2013-03-01 09:11:58.703096|20.33|joggying +7|307|65580|4294967468|94.15|45.55|true|nick robinson|2013-03-01 09:11:58.703176|45.15|linguistics +77|418|65657|4294967432|83.59|2.23|true|irene davidson|2013-03-01 09:11:58.703150|96.61|american history +121|334|65716|4294967520|47.29|36.79|false|mike young|2013-03-01 09:11:58.703212|56.49|philosophy +120|309|65587|4294967519|81.48|33.39|false|victor quirinius|2013-03-01 09:11:58.703289|22.93|industrial engineering +92|345|65609|4294967499|34.02|3.72|true|victor garcia|2013-03-01 09:11:58.703297|96.42|values clariffication +53|464|65595|4294967353|63.36|14.13|true|quinn xylophone|2013-03-01 09:11:58.703157|43.13|american history +18|474|65688|4294967521|22.85|33.11|false|irene davidson|2013-03-01 09:11:58.703157|2.21|kindergarten +79|367|65566|4294967417|18.15|12.65|false|quinn robinson|2013-03-01 09:11:58.703132|88.99|study skills +117|429|65635|4294967383|67.98|14.17|true|nick zipper|2013-03-01 09:11:58.703100|40.57|philosophy +64|292|65732|4294967397|46.69|47.27|true|nick zipper|2013-03-01 09:11:58.703109|15.32|nap time +80|474|65758|4294967542|84.46|44.70|false|sarah young|2013-03-01 09:11:58.703185|26.04|american history +35|464|65563|4294967421|11.66|20.23|true|luke thompson|2013-03-01 09:11:58.703239|89.40|zync studies +73|463|65562|4294967468|37.93|14.91|true|xavier ichabod|2013-03-01 09:11:58.703179|67.99|undecided +17|447|65762|4294967328|66.48|6.35|true|priscilla falkner|2013-03-01 09:11:58.703286|68.09|opthamology +18|385|65605|4294967456|34.94|31.85|true|mike ellison|2013-03-01 09:11:58.703288|14.32|quiet hour +-3|451|65696|4294967532|6.80|40.07|false|luke young|2013-03-01 09:11:58.703182|26.91|biology +81|479|65655|4294967424|82.97|12.85|true|zach thompson|2013-03-01 09:11:58.703256|76.39|industrial engineering +92|499|65627|4294967430|69.69|24.17|false|nick polk|2013-03-01 09:11:58.703208|33.05|forestry +98|383|65611|4294967464|94.27|33.09|false|luke ichabod|2013-03-01 09:11:58.703138|84.03|values clariffication +40|438|65605|4294967548|59.47|30.38|false|quinn young|2013-03-01 09:11:58.703192|79.08|topology +2|271|65634|4294967531|35.19|11.79|false|tom hernandez|2013-03-01 09:11:58.703243|29.60|quiet hour +11|498|65569|4294967363|4.48|26.25|true|nick king|2013-03-01 09:11:58.703072|15.83|biology +76|298|65660|4294967504|71.01|13.05|false|sarah robinson|2013-03-01 09:11:58.703325|17.86|kindergarten +8|346|65702|4294967456|46.60|43.85|false|zach white|2013-03-01 09:11:58.703159|41.14|chemistry +39|421|65674|4294967421|27.57|20.92|true|quinn allen|2013-03-01 09:11:58.703142|42.94|american history +119|461|65714|4294967369|34.25|4.51|true|irene xylophone|2013-03-01 09:11:58.703279|16.58|study skills +69|279|65700|4294967488|83.31|24.00|true|mike nixon|2013-03-01 09:11:58.703075|64.55|debate +103|389|65669|4294967531|31.19|29.28|true|bob ichabod|2013-03-01 09:11:58.703322|96.69|american history +58|385|65653|4294967357|20.15|13.27|true|nick white|2013-03-01 09:11:58.703233|54.07|kindergarten +30|367|65712|4294967461|65.95|47.81|false|quinn young|2013-03-01 09:11:58.703086|63.93|quiet hour +24|368|65585|4294967337|97.87|6.55|true|ethan allen|2013-03-01 09:11:58.703324|26.17|forestry +2|402|65762|4294967398|31.89|18.37|true|rachel underhill|2013-03-01 09:11:58.703097|68.90|values clariffication +93|427|65699|4294967495|75.14|15.80|false|david steinbeck|2013-03-01 09:11:58.703196|63.21|nap time +123|427|65774|4294967484|4.58|38.94|true|yuri zipper|2013-03-01 09:11:58.703109|77.88|linguistics +75|429|65645|4294967410|80.23|17.78|true|zach king|2013-03-01 09:11:58.703124|78.73|history +54|360|65770|4294967498|71.66|13.68|false|nick nixon|2013-03-01 09:11:58.703316|29.89|yard duty +79|415|65544|4294967533|36.93|25.57|false|mike garcia|2013-03-01 09:11:58.703283|51.94|undecided +44|404|65667|4294967380|71.04|40.83|false|tom thompson|2013-03-01 09:11:58.703122|1.43|education +24|354|65624|4294967423|74.66|37.18|true|ulysses underhill|2013-03-01 09:11:58.703228|97.48|education +89|390|65755|4294967449|5.39|6.71|false|nick quirinius|2013-03-01 09:11:58.703111|71.15|chemistry +69|311|65619|4294967435|97.83|8.83|false|ulysses underhill|2013-03-01 09:11:58.703256|14.32|opthamology +73|427|65704|4294967363|45.49|6.73|false|zach young|2013-03-01 09:11:58.703074|66.59|forestry +74|347|65600|4294967367|57.19|13.67|false|david garcia|2013-03-01 09:11:58.703169|83.71|philosophy +102|447|65626|4294967441|50.74|17.13|false|tom falkner|2013-03-01 09:11:58.703205|62.59|joggying +54|378|65711|4294967414|55.31|2.29|false|fred falkner|2013-03-01 09:11:58.703084|59.53|education +87|405|65635|4294967461|23.29|11.95|true|mike steinbeck|2013-03-01 09:11:58.703246|84.80|american history +76|318|65644|4294967397|29.90|29.12|true|victor van buren|2013-03-01 09:11:58.703260|14.14|american history +68|378|65734|4294967546|72.17|6.95|true|oscar underhill|2013-03-01 09:11:58.703158|53.08|zync studies +85|444|65789|4294967471|91.82|48.49|false|bob garcia|2013-03-01 09:11:58.703300|69.81|quiet hour +123|379|65606|4294967517|95.53|17.05|true|mike ovid|2013-03-01 09:11:58.703147|76.93|joggying +115|333|65656|4294967507|50.07|16.61|false|jessica underhill|2013-03-01 09:11:58.703319|59.71|study skills +15|425|65676|4294967352|92.42|35.89|false|ethan van buren|2013-03-01 09:11:58.703170|90.15|kindergarten +113|444|65725|4294967372|32.78|9.52|true|luke van buren|2013-03-01 09:11:58.703261|22.85|zync studies +26|304|65749|4294967519|91.54|34.97|true|mike young|2013-03-01 09:11:58.703149|83.37|chemistry +33|289|65575|4294967393|12.38|33.89|false|katie van buren|2013-03-01 09:11:58.703291|45.08|mathematics +103|469|65672|4294967361|31.33|36.49|true|mike underhill|2013-03-01 09:11:58.703235|61.57|chemistry +74|445|65566|4294967486|63.84|16.02|false|wendy nixon|2013-03-01 09:11:58.703258|12.55|industrial engineering +68|448|65609|4294967425|45.13|37.04|false|katie ovid|2013-03-01 09:11:58.703200|19.08|nap time +0|363|65754|4294967491|27.57|5.19|false|ethan laertes|2013-03-01 09:11:58.703306|45.33|philosophy +115|473|65773|4294967485|96.45|0.41|true|mike allen|2013-03-01 09:11:58.703211|86.60|debate +75|451|65770|4294967384|13.07|20.70|true|gabriella king|2013-03-01 09:11:58.703219|27.07|wind surfing +98|284|65729|4294967482|71.95|45.38|false|ethan ellison|2013-03-01 09:11:58.703252|64.17|debate +51|271|65777|4294967337|65.94|6.95|false|sarah zipper|2013-03-01 09:11:58.703124|41.46|debate +92|319|65624|4294967296|75.12|29.40|true|ethan ichabod|2013-03-01 09:11:58.703126|9.60|study skills +112|369|65600|4294967379|29.07|38.34|true|luke thompson|2013-03-01 09:11:58.703071|68.36|philosophy +51|489|65578|4294967337|69.92|21.56|true|jessica king|2013-03-01 09:11:58.703213|5.03|philosophy +-2|304|65712|4294967483|59.04|35.60|true|priscilla brown|2013-03-01 09:11:58.703227|81.04|kindergarten +39|337|65766|4294967343|2.58|17.29|false|xavier polk|2013-03-01 09:11:58.703322|73.75|values clariffication +27|309|65631|4294967461|10.86|16.44|false|bob davidson|2013-03-01 09:11:58.703244|21.81|biology +101|435|65628|4294967365|79.98|24.14|true|irene quirinius|2013-03-01 09:11:58.703080|66.27|linguistics +6|500|65672|4294967425|77.93|7.89|true|nick garcia|2013-03-01 09:11:58.703285|15.54|quiet hour +115|337|65676|4294967345|5.76|41.72|true|katie steinbeck|2013-03-01 09:11:58.703102|77.62|education +76|435|65775|4294967403|70.29|43.49|false|gabriella polk|2013-03-01 09:11:58.703099|10.40|religion +11|266|65594|4294967545|69.06|19.79|true|yuri zipper|2013-03-01 09:11:58.703242|98.86|joggying +103|296|65746|4294967424|31.27|6.91|true|irene johnson|2013-03-01 09:11:58.703210|10.90|study skills +78|365|65542|4294967354|47.37|6.70|false|jessica thompson|2013-03-01 09:11:58.703271|42.30|mathematics +74|426|65669|4294967486|32.97|14.19|true|sarah van buren|2013-03-01 09:11:58.703317|92.03|study skills +118|334|65756|4294967472|22.62|45.79|false|yuri hernandez|2013-03-01 09:11:58.703183|14.49|linguistics +121|256|65620|4294967322|51.72|49.38|false|xavier ovid|2013-03-01 09:11:58.703148|52.70|nap time +88|351|65543|4294967324|19.79|39.85|false|ulysses ellison|2013-03-01 09:11:58.703139|41.86|biology +111|444|65616|4294967341|8.47|33.32|false|irene xylophone|2013-03-01 09:11:58.703233|84.17|history +4|292|65540|4294967296|58.69|33.54|false|irene garcia|2013-03-01 09:11:58.703294|81.51|wind surfing +86|407|65580|4294967544|63.98|12.32|false|mike ellison|2013-03-01 09:11:58.703160|92.48|values clariffication +39|461|65688|4294967470|63.58|10.43|false|sarah zipper|2013-03-01 09:11:58.703151|74.85|values clariffication +99|471|65591|4294967355|85.76|2.81|true|priscilla laertes|2013-03-01 09:11:58.703132|24.76|philosophy +25|484|65645|4294967410|0.57|15.76|true|priscilla king|2013-03-01 09:11:58.703309|22.69|education +41|505|65782|4294967308|52.79|19.60|true|rachel miller|2013-03-01 09:11:58.703157|6.74|history +113|477|65737|4294967529|9.96|46.90|false|irene white|2013-03-01 09:11:58.703089|79.59|education +54|294|65736|4294967451|34.60|48.64|true|priscilla van buren|2013-03-01 09:11:58.703134|3.31|yard duty +34|369|65717|4294967334|70.39|2.31|true|oscar johnson|2013-03-01 09:11:58.703216|79.94|wind surfing +20|460|65595|4294967416|74.24|1.61|true|xavier nixon|2013-03-01 09:11:58.703206|64.95|philosophy +33|402|65558|4294967441|72.12|45.31|false|bob king|2013-03-01 09:11:58.703269|20.94|philosophy +108|454|65732|4294967297|18.92|7.03|false|oscar quirinius|2013-03-01 09:11:58.703166|21.36|industrial engineering +18|345|65773|4294967375|90.49|21.93|false|irene johnson|2013-03-01 09:11:58.703093|36.38|education +17|493|65681|4294967388|78.85|3.41|true|tom ichabod|2013-03-01 09:11:58.703095|28.31|american history +34|477|65582|4294967430|34.76|22.02|false|mike thompson|2013-03-01 09:11:58.703172|1.38|wind surfing +38|435|65589|4294967492|52.87|1.74|false|zach young|2013-03-01 09:11:58.703215|65.98|undecided +112|287|65551|4294967476|4.83|40.14|false|zach thompson|2013-03-01 09:11:58.703260|10.80|american history +53|405|65779|4294967526|39.65|29.73|true|fred zipper|2013-03-01 09:11:58.703208|25.41|values clariffication +68|381|65540|4294967335|16.62|27.23|true|xavier nixon|2013-03-01 09:11:58.703274|73.99|history +60|404|65602|4294967476|55.55|6.74|false|ethan carson|2013-03-01 09:11:58.703106|5.64|history +57|269|65773|4294967495|57.49|25.47|false|rachel garcia|2013-03-01 09:11:58.703097|64.90|mathematics +28|463|65592|4294967320|3.70|27.12|false|jessica king|2013-03-01 09:11:58.703167|76.61|nap time +68|290|65606|4294967359|5.50|47.78|true|ethan white|2013-03-01 09:11:58.703102|60.24|chemistry +52|359|65604|4294967508|85.18|1.25|true|alice miller|2013-03-01 09:11:58.703172|42.09|philosophy +90|263|65622|4294967546|90.40|49.20|true|katie carson|2013-03-01 09:11:58.703130|88.51|philosophy +16|358|65704|4294967337|10.39|16.53|true|luke ellison|2013-03-01 09:11:58.703223|63.39|mathematics +36|349|65714|4294967428|90.50|40.64|false|ethan quirinius|2013-03-01 09:11:58.703190|0.17|debate +36|314|65728|4294967511|58.60|36.24|false|priscilla carson|2013-03-01 09:11:58.703141|77.59|zync studies +74|284|65655|4294967387|47.45|15.95|true|victor carson|2013-03-01 09:11:58.703241|52.95|education +114|385|65573|4294967419|44.89|1.87|true|katie nixon|2013-03-01 09:11:58.703190|64.24|quiet hour +11|340|65755|4294967452|40.94|0.59|true|quinn underhill|2013-03-01 09:11:58.703080|87.76|history +53|353|65621|4294967458|87.00|44.47|false|katie zipper|2013-03-01 09:11:58.703304|81.11|joggying +92|483|65739|4294967457|96.93|35.13|true|katie thompson|2013-03-01 09:11:58.703233|47.86|quiet hour +76|419|65634|4294967405|20.06|42.54|false|fred brown|2013-03-01 09:11:58.703285|3.59|topology +41|345|65632|4294967323|6.72|28.55|false|quinn nixon|2013-03-01 09:11:58.703165|69.29|topology +18|481|65727|4294967327|83.64|38.29|false|zach quirinius|2013-03-01 09:11:58.703105|62.83|chemistry +109|480|65556|4294967357|9.65|5.69|true|wendy king|2013-03-01 09:11:58.703128|41.67|quiet hour +53|447|65557|4294967469|5.27|40.67|true|calvin van buren|2013-03-01 09:11:58.703309|85.26|joggying +22|449|65689|4294967304|39.59|29.03|false|rachel laertes|2013-03-01 09:11:58.703161|7.21|mathematics +49|257|65654|4294967339|24.54|1.99|true|irene young|2013-03-01 09:11:58.703127|90.64|topology +96|374|65578|4294967313|54.57|49.15|true|luke ellison|2013-03-01 09:11:58.703136|30.97|nap time +69|403|65578|4294967475|45.38|6.00|false|quinn king|2013-03-01 09:11:58.703126|6.82|kindergarten +92|412|65609|4294967501|60.05|28.07|false|bob van buren|2013-03-01 09:11:58.703082|37.60|study skills +6|311|65680|4294967413|67.91|48.15|true|david robinson|2013-03-01 09:11:58.703208|7.80|american history +71|294|65699|4294967514|99.34|38.36|false|quinn young|2013-03-01 09:11:58.703091|69.06|yard duty +122|381|65632|4294967438|38.35|24.09|true|david falkner|2013-03-01 09:11:58.703070|76.50|american history +115|323|65751|4294967465|19.46|15.29|false|tom allen|2013-03-01 09:11:58.703209|73.55|mathematics +91|474|65780|4294967422|63.22|19.74|false|luke quirinius|2013-03-01 09:11:58.703284|35.68|industrial engineering +104|300|65704|4294967425|27.62|31.93|true|alice ichabod|2013-03-01 09:11:58.703250|88.78|joggying +7|383|65743|4294967353|64.72|15.94|true|zach laertes|2013-03-01 09:11:58.703246|79.07|geology +13|457|65553|4294967547|31.90|26.36|false|wendy quirinius|2013-03-01 09:11:58.703192|32.56|education +4|347|65785|4294967342|5.76|32.38|false|alice hernandez|2013-03-01 09:11:58.703142|70.60|xylophone band +64|414|65639|4294967498|10.43|36.85|false|david falkner|2013-03-01 09:11:58.703274|60.01|industrial engineering +55|293|65609|4294967312|87.47|39.53|true|gabriella nixon|2013-03-01 09:11:58.703315|62.03|history +78|443|65678|4294967305|75.51|44.51|true|ethan carson|2013-03-01 09:11:58.703278|90.43|geology +121|488|65537|4294967448|17.54|13.44|true|sarah falkner|2013-03-01 09:11:58.703221|8.63|study skills +104|444|65536|4294967334|93.56|34.20|true|rachel ichabod|2013-03-01 09:11:58.703214|46.97|joggying +28|377|65687|4294967339|65.44|46.45|true|nick garcia|2013-03-01 09:11:58.703123|38.91|zync studies +-2|473|65565|4294967320|87.78|12.26|true|alice carson|2013-03-01 09:11:58.703074|89.54|xylophone band +82|448|65718|4294967533|60.38|7.60|false|mike underhill|2013-03-01 09:11:58.703270|57.58|philosophy +81|259|65764|4294967452|88.25|15.41|true|victor underhill|2013-03-01 09:11:58.703204|55.61|education +104|325|65785|4294967463|48.83|19.16|true|calvin nixon|2013-03-01 09:11:58.703093|30.81|geology +17|415|65772|4294967541|4.56|7.27|true|oscar garcia|2013-03-01 09:11:58.703287|54.15|joggying +100|400|65773|4294967475|12.17|30.37|false|victor xylophone|2013-03-01 09:11:58.703178|19.88|nap time +111|312|65785|4294967296|77.09|39.14|false|victor garcia|2013-03-01 09:11:58.703242|86.29|joggying +119|327|65647|4294967343|50.04|27.00|true|oscar miller|2013-03-01 09:11:58.703321|92.97|yard duty +98|455|65605|4294967359|17.27|48.26|true|luke steinbeck|2013-03-01 09:11:58.703142|90.50|wind surfing +61|426|65618|4294967485|61.88|14.15|false|sarah quirinius|2013-03-01 09:11:58.703174|55.59|mathematics +8|382|65788|4294967302|72.41|31.03|false|katie ovid|2013-03-01 09:11:58.703225|32.02|biology +37|286|65721|4294967369|8.28|1.49|false|oscar young|2013-03-01 09:11:58.703236|97.62|forestry +46|297|65552|4294967329|47.92|9.95|true|yuri ovid|2013-03-01 09:11:58.703143|74.51|religion +-1|493|65654|4294967429|4.51|3.59|true|priscilla ichabod|2013-03-01 09:11:58.703226|59.28|american history +124|289|65786|4294967500|89.22|11.52|true|bob white|2013-03-01 09:11:58.703197|87.43|chemistry +47|258|65546|4294967351|91.19|44.01|true|sarah zipper|2013-03-01 09:11:58.703206|69.72|industrial engineering +108|408|65658|4294967362|66.68|38.05|true|katie allen|2013-03-01 09:11:58.703212|54.61|american history +32|425|65739|4294967508|41.64|19.54|false|zach johnson|2013-03-01 09:11:58.703318|9.99|topology +12|447|65748|4294967545|19.71|49.05|false|oscar van buren|2013-03-01 09:11:58.703091|58.79|education +124|423|65758|4294967416|2.03|41.55|true|gabriella steinbeck|2013-03-01 09:11:58.703162|97.11|topology +93|413|65646|4294967312|1.44|18.16|false|luke quirinius|2013-03-01 09:11:58.703163|22.71|wind surfing +66|481|65709|4294967533|18.45|23.51|false|alice falkner|2013-03-01 09:11:58.703294|78.13|quiet hour +36|486|65569|4294967442|21.57|48.60|false|quinn thompson|2013-03-01 09:11:58.703088|90.26|wind surfing +87|465|65656|4294967340|85.55|28.44|true|zach polk|2013-03-01 09:11:58.703077|61.38|mathematics +37|318|65733|4294967493|14.14|14.62|true|priscilla young|2013-03-01 09:11:58.703100|30.81|quiet hour +115|419|65683|4294967440|53.12|26.89|true|calvin laertes|2013-03-01 09:11:58.703286|98.28|kindergarten +104|339|65785|4294967310|57.14|48.11|false|alice falkner|2013-03-01 09:11:58.703200|46.58|zync studies +47|368|65767|4294967532|51.13|44.77|true|rachel zipper|2013-03-01 09:11:58.703316|61.14|linguistics +6|291|65617|4294967479|43.28|8.20|false|priscilla thompson|2013-03-01 09:11:58.703157|56.12|religion +62|369|65775|4294967451|39.50|1.84|false|wendy brown|2013-03-01 09:11:58.703308|42.02|linguistics +101|364|65627|4294967388|45.57|7.74|false|luke carson|2013-03-01 09:11:58.703097|17.57|topology +36|344|65708|4294967526|56.42|4.58|false|gabriella white|2013-03-01 09:11:58.703171|24.06|history +90|465|65583|4294967506|37.42|23.76|false|ethan hernandez|2013-03-01 09:11:58.703094|42.38|undecided +40|481|65639|4294967326|28.82|13.16|true|david ellison|2013-03-01 09:11:58.703145|49.54|linguistics +72|429|65618|4294967517|14.97|39.28|true|tom quirinius|2013-03-01 09:11:58.703229|34.40|xylophone band +58|493|65616|4294967355|19.81|49.37|false|priscilla robinson|2013-03-01 09:11:58.703142|29.21|religion +25|452|65541|4294967395|29.15|1.49|true|zach garcia|2013-03-01 09:11:58.703102|70.06|industrial engineering +59|492|65650|4294967322|21.31|2.93|true|nick young|2013-03-01 09:11:58.703305|4.29|forestry +76|261|65543|4294967430|66.87|35.89|false|tom garcia|2013-03-01 09:11:58.703230|13.25|forestry +52|327|65667|4294967543|55.24|24.97|true|xavier underhill|2013-03-01 09:11:58.703261|86.13|zync studies +80|422|65553|4294967408|24.18|28.17|true|nick ellison|2013-03-01 09:11:58.703128|33.36|joggying +-1|427|65651|4294967380|23.72|22.34|true|zach brown|2013-03-01 09:11:58.703174|59.44|kindergarten +124|370|65617|4294967468|89.87|24.47|true|jessica thompson|2013-03-01 09:11:58.703170|66.22|topology +65|437|65698|4294967332|47.58|32.51|true|wendy ellison|2013-03-01 09:11:58.703160|42.22|history +20|499|65581|4294967396|5.19|6.98|false|ulysses brown|2013-03-01 09:11:58.703222|26.46|history +62|344|65770|4294967514|53.41|17.50|false|alice white|2013-03-01 09:11:58.703140|50.22|nap time +61|350|65687|4294967344|84.78|25.68|false|xavier underhill|2013-03-01 09:11:58.703256|44.94|linguistics +33|412|65714|4294967496|11.91|16.39|true|irene zipper|2013-03-01 09:11:58.703210|14.78|values clariffication +14|437|65602|4294967484|74.94|7.05|false|tom nixon|2013-03-01 09:11:58.703112|67.04|quiet hour +50|492|65619|4294967359|94.98|38.50|true|bob van buren|2013-03-01 09:11:58.703133|51.12|geology +121|369|65769|4294967304|88.21|19.10|true|tom robinson|2013-03-01 09:11:58.703295|7.10|values clariffication +66|286|65581|4294967516|5.26|15.00|false|mike polk|2013-03-01 09:11:58.703084|69.34|history +121|315|65713|4294967541|88.42|46.68|true|sarah van buren|2013-03-01 09:11:58.703121|9.58|zync studies +62|424|65731|4294967453|40.35|3.75|true|holly underhill|2013-03-01 09:11:58.703191|61.93|linguistics +29|496|65727|4294967387|6.81|23.41|true|xavier young|2013-03-01 09:11:58.703315|21.06|yard duty +52|259|65588|4294967425|83.91|12.10|true|ulysses davidson|2013-03-01 09:11:58.703100|90.71|kindergarten +108|486|65717|4294967298|48.51|26.14|false|jessica miller|2013-03-01 09:11:58.703247|26.33|quiet hour +24|346|65637|4294967518|67.97|13.67|false|sarah polk|2013-03-01 09:11:58.703228|31.60|debate +21|478|65738|4294967480|99.30|14.09|true|wendy johnson|2013-03-01 09:11:58.703186|48.79|wind surfing +50|457|65791|4294967546|24.78|39.66|false|holly young|2013-03-01 09:11:58.703276|18.12|linguistics +117|384|65755|4294967525|64.76|39.04|false|irene quirinius|2013-03-01 09:11:58.703130|58.59|philosophy +51|316|65640|4294967524|42.19|20.83|true|wendy hernandez|2013-03-01 09:11:58.703130|53.94|forestry +23|354|65698|4294967317|68.23|13.68|false|alice ellison|2013-03-01 09:11:58.703264|72.44|undecided +101|486|65701|4294967369|54.27|45.06|false|mike laertes|2013-03-01 09:11:58.703292|41.16|topology +61|425|65774|4294967383|0.07|3.32|false|calvin thompson|2013-03-01 09:11:58.703190|66.08|study skills +27|429|65777|4294967439|38.61|25.66|true|calvin steinbeck|2013-03-01 09:11:58.703070|94.02|undecided +72|322|65638|4294967445|48.82|1.93|false|priscilla van buren|2013-03-01 09:11:58.703291|54.69|quiet hour +85|403|65675|4294967396|52.78|48.32|true|tom allen|2013-03-01 09:11:58.703266|77.79|undecided +109|344|65556|4294967426|54.60|15.52|true|jessica underhill|2013-03-01 09:11:58.703159|27.13|debate +87|302|65582|4294967485|26.19|29.72|true|nick young|2013-03-01 09:11:58.703216|29.49|religion +62|356|65667|4294967383|6.78|5.88|false|irene van buren|2013-03-01 09:11:58.703204|48.21|history +123|423|65562|4294967382|28.40|27.45|false|fred johnson|2013-03-01 09:11:58.703294|81.03|quiet hour +62|370|65766|4294967352|61.81|44.76|true|irene carson|2013-03-01 09:11:58.703245|67.40|religion +46|349|65700|4294967471|62.11|40.76|false|rachel underhill|2013-03-01 09:11:58.703117|15.52|yard duty +35|422|65589|4294967488|82.31|42.27|false|calvin hernandez|2013-03-01 09:11:58.703239|94.35|forestry +18|369|65674|4294967517|78.10|32.46|false|holly falkner|2013-03-01 09:11:58.703094|45.78|biology +68|257|65738|4294967364|96.01|44.95|false|fred brown|2013-03-01 09:11:58.703274|19.87|nap time +67|478|65603|4294967440|80.03|36.49|false|mike ichabod|2013-03-01 09:11:58.703152|7.57|religion +80|393|65666|4294967549|42.43|47.36|false|bob underhill|2013-03-01 09:11:58.703262|76.35|industrial engineering +123|380|65765|4294967367|79.69|41.18|true|alice underhill|2013-03-01 09:11:58.703119|12.23|opthamology +19|391|65748|4294967456|16.15|26.48|true|tom robinson|2013-03-01 09:11:58.703132|43.88|yard duty +7|358|65766|4294967529|77.96|6.32|true|victor ichabod|2013-03-01 09:11:58.703325|0.85|zync studies +77|372|65728|4294967485|28.65|41.49|false|calvin hernandez|2013-03-01 09:11:58.703187|17.38|mathematics +114|440|65674|4294967451|83.07|32.43|true|zach young|2013-03-01 09:11:58.703260|87.43|history +8|366|65783|4294967485|95.46|41.14|false|ethan robinson|2013-03-01 09:11:58.703299|55.93|zync studies +62|339|65603|4294967380|36.15|2.94|false|quinn xylophone|2013-03-01 09:11:58.703203|70.54|undecided +91|403|65753|4294967386|12.19|38.41|false|calvin polk|2013-03-01 09:11:58.703272|38.50|undecided +51|379|65749|4294967387|93.97|14.61|false|ethan davidson|2013-03-01 09:11:58.703190|31.31|forestry +28|425|65619|4294967338|11.50|46.48|true|calvin miller|2013-03-01 09:11:58.703298|49.84|philosophy +103|409|65661|4294967476|21.29|48.16|true|priscilla underhill|2013-03-01 09:11:58.703261|21.65|undecided +57|404|65779|4294967531|93.75|21.77|false|jessica garcia|2013-03-01 09:11:58.703205|16.85|quiet hour +30|326|65721|4294967405|68.64|10.37|false|tom ellison|2013-03-01 09:11:58.703140|78.92|wind surfing +80|491|65744|4294967363|96.20|28.28|true|gabriella hernandez|2013-03-01 09:11:58.703121|9.35|geology +104|475|65744|4294967482|57.56|49.61|true|fred ellison|2013-03-01 09:11:58.703131|82.61|philosophy +121|447|65566|4294967545|10.06|23.94|true|irene carson|2013-03-01 09:11:58.703248|9.37|industrial engineering +23|333|65642|4294967533|62.80|10.38|true|sarah ovid|2013-03-01 09:11:58.703165|95.13|values clariffication +77|282|65655|4294967472|70.67|22.37|false|quinn nixon|2013-03-01 09:11:58.703281|40.42|geology +35|504|65547|4294967434|70.56|47.95|false|fred white|2013-03-01 09:11:58.703276|1.50|education +96|319|65586|4294967518|34.22|4.24|true|priscilla robinson|2013-03-01 09:11:58.703144|32.10|education +18|286|65636|4294967545|3.28|21.76|false|mike johnson|2013-03-01 09:11:58.703126|83.37|chemistry +11|478|65559|4294967524|14.12|39.50|false|luke laertes|2013-03-01 09:11:58.703303|76.45|geology +16|377|65745|4294967492|34.93|26.17|false|mike brown|2013-03-01 09:11:58.703150|28.94|debate +109|386|65546|4294967539|81.99|37.00|true|victor steinbeck|2013-03-01 09:11:58.703239|10.52|industrial engineering +53|261|65657|4294967495|75.57|49.39|true|bob ellison|2013-03-01 09:11:58.703132|94.91|zync studies +93|371|65599|4294967526|41.79|9.49|true|nick ovid|2013-03-01 09:11:58.703137|33.73|values clariffication +54|400|65757|4294967365|81.52|25.42|true|tom miller|2013-03-01 09:11:58.703284|96.60|yard duty +50|271|65715|4294967326|61.15|31.48|true|gabriella ellison|2013-03-01 09:11:58.703093|40.99|biology +69|484|65699|4294967427|45.81|10.52|true|gabriella nixon|2013-03-01 09:11:58.703190|10.16|history +118|285|65682|4294967311|91.26|23.35|false|rachel white|2013-03-01 09:11:58.703073|20.18|religion +115|412|65719|4294967381|44.98|24.24|false|bob hernandez|2013-03-01 09:11:58.703185|25.38|opthamology +100|323|65611|4294967404|25.97|18.98|true|fred robinson|2013-03-01 09:11:58.703218|28.62|xylophone band +6|428|65739|4294967497|60.45|45.89|false|oscar garcia|2013-03-01 09:11:58.703225|88.39|linguistics +-2|479|65779|4294967424|41.86|14.56|false|victor zipper|2013-03-01 09:11:58.703199|24.90|geology +24|362|65684|4294967349|57.61|0.43|true|yuri young|2013-03-01 09:11:58.703128|22.16|nap time +14|451|65695|4294967486|23.99|4.27|false|sarah nixon|2013-03-01 09:11:58.703292|91.72|chemistry +9|453|65735|4294967498|91.99|2.85|true|bob young|2013-03-01 09:11:58.703122|7.49|debate +11|353|65577|4294967435|16.87|33.33|false|irene miller|2013-03-01 09:11:58.703225|41.38|undecided +27|414|65632|4294967334|20.30|7.52|false|xavier ellison|2013-03-01 09:11:58.703150|49.10|history +13|406|65742|4294967475|47.16|41.61|false|priscilla garcia|2013-03-01 09:11:58.703112|48.18|chemistry +76|486|65584|4294967385|24.16|28.70|false|nick thompson|2013-03-01 09:11:58.703299|40.18|joggying +103|289|65757|4294967373|42.85|29.84|true|nick nixon|2013-03-01 09:11:58.703315|2.07|xylophone band +26|510|65563|4294967397|6.56|10.01|true|nick underhill|2013-03-01 09:11:58.703100|74.58|education +116|322|65651|4294967392|24.46|7.55|true|quinn hernandez|2013-03-01 09:11:58.703109|23.93|quiet hour +97|333|65776|4294967415|94.91|24.64|true|xavier quirinius|2013-03-01 09:11:58.703290|38.08|mathematics +24|336|65574|4294967333|66.07|25.42|true|ethan garcia|2013-03-01 09:11:58.703323|27.02|geology +46|396|65549|4294967482|80.31|34.38|true|holly nixon|2013-03-01 09:11:58.703312|39.69|nap time +83|348|65567|4294967421|18.14|5.71|false|xavier ichabod|2013-03-01 09:11:58.703123|48.70|joggying +77|334|65666|4294967337|40.53|23.61|true|zach laertes|2013-03-01 09:11:58.703075|20.35|kindergarten +109|258|65781|4294967519|50.36|12.74|true|holly johnson|2013-03-01 09:11:58.703228|14.39|biology +16|427|65661|4294967440|33.42|5.16|true|fred thompson|2013-03-01 09:11:58.703237|99.22|undecided +50|337|65583|4294967312|10.61|0.73|false|jessica thompson|2013-03-01 09:11:58.703307|29.40|nap time +9|416|65635|4294967448|9.30|17.50|false|holly young|2013-03-01 09:11:58.703229|88.10|undecided +64|412|65567|4294967463|6.07|38.63|true|calvin nixon|2013-03-01 09:11:58.703317|25.43|joggying +86|438|65772|4294967480|73.79|47.76|true|irene laertes|2013-03-01 09:11:58.703110|21.19|undecided +119|372|65598|4294967494|94.20|41.68|true|wendy thompson|2013-03-01 09:11:58.703171|16.27|forestry +31|499|65571|4294967505|38.00|21.84|true|wendy ovid|2013-03-01 09:11:58.703115|11.25|kindergarten +100|396|65762|4294967341|24.90|36.03|true|david ovid|2013-03-01 09:11:58.703213|41.92|geology +118|451|65541|4294967367|19.88|2.66|false|david laertes|2013-03-01 09:11:58.703289|53.92|zync studies +88|391|65726|4294967426|0.84|47.11|false|victor hernandez|2013-03-01 09:11:58.703274|2.10|history +-1|413|65549|4294967470|9.62|28.51|false|wendy hernandez|2013-03-01 09:11:58.703315|59.32|wind surfing +56|454|65536|4294967431|19.94|33.49|true|david laertes|2013-03-01 09:11:58.703284|98.42|xylophone band +32|394|65719|4294967355|82.52|7.50|true|jessica thompson|2013-03-01 09:11:58.703146|94.60|american history +67|336|65665|4294967503|33.72|36.79|true|xavier carson|2013-03-01 09:11:58.703135|19.68|philosophy +85|384|65644|4294967472|35.77|9.69|true|gabriella falkner|2013-03-01 09:11:58.703192|72.45|undecided +25|417|65776|4294967477|95.76|33.52|true|oscar falkner|2013-03-01 09:11:58.703292|17.50|industrial engineering +102|403|65593|4294967497|39.87|16.67|false|quinn hernandez|2013-03-01 09:11:58.703241|18.27|industrial engineering +35|499|65584|4294967352|88.15|7.73|false|ethan thompson|2013-03-01 09:11:58.703086|27.80|linguistics +56|362|65729|4294967505|31.92|48.47|false|yuri nixon|2013-03-01 09:11:58.703247|94.44|yard duty +101|492|65721|4294967518|98.03|13.64|false|irene xylophone|2013-03-01 09:11:58.703241|0.34|values clariffication +113|310|65766|4294967441|36.38|40.04|false|zach garcia|2013-03-01 09:11:58.703309|51.33|biology +37|275|65536|4294967491|73.64|14.18|true|david nixon|2013-03-01 09:11:58.703244|0.93|kindergarten +53|399|65686|4294967466|78.99|34.31|true|nick underhill|2013-03-01 09:11:58.703283|13.95|study skills +16|370|65631|4294967460|17.37|16.16|true|calvin xylophone|2013-03-01 09:11:58.703210|50.60|philosophy +12|257|65578|4294967308|61.60|45.27|false|victor xylophone|2013-03-01 09:11:58.703240|98.48|linguistics +44|262|65580|4294967551|74.34|24.36|false|calvin xylophone|2013-03-01 09:11:58.703097|62.98|linguistics +82|493|65701|4294967389|89.64|23.91|false|sarah johnson|2013-03-01 09:11:58.703097|85.65|linguistics +49|316|65559|4294967329|46.90|5.41|false|alice carson|2013-03-01 09:11:58.703133|34.22|joggying +112|407|65789|4294967536|89.74|30.52|false|yuri laertes|2013-03-01 09:11:58.703314|57.12|debate +91|412|65783|4294967412|9.99|38.63|false|gabriella nixon|2013-03-01 09:11:58.703214|30.18|zync studies +78|420|65680|4294967342|22.57|30.58|true|calvin thompson|2013-03-01 09:11:58.703238|27.58|values clariffication +88|324|65690|4294967418|18.32|33.83|false|alice xylophone|2013-03-01 09:11:58.703307|3.04|quiet hour +107|294|65688|4294967412|85.80|36.83|true|katie xylophone|2013-03-01 09:11:58.703301|78.07|industrial engineering +16|301|65589|4294967380|84.36|16.77|false|yuri hernandez|2013-03-01 09:11:58.703226|53.39|geology +120|423|65624|4294967477|13.17|17.66|true|sarah davidson|2013-03-01 09:11:58.703217|16.34|american history +73|374|65600|4294967313|65.86|20.33|true|quinn falkner|2013-03-01 09:11:58.703132|34.70|study skills +10|492|65639|4294967442|9.16|32.19|true|victor zipper|2013-03-01 09:11:58.703218|56.83|xylophone band +12|274|65724|4294967492|33.51|24.13|true|holly young|2013-03-01 09:11:58.703076|75.21|religion +32|331|65618|4294967314|13.97|34.16|false|ulysses davidson|2013-03-01 09:11:58.703278|26.57|values clariffication +42|280|65752|4294967478|92.96|7.45|false|katie king|2013-03-01 09:11:58.703070|66.80|geology +26|383|65640|4294967490|70.41|11.31|false|calvin johnson|2013-03-01 09:11:58.703190|64.14|quiet hour +91|421|65572|4294967333|10.71|19.30|false|holly polk|2013-03-01 09:11:58.703116|41.87|zync studies +64|385|65614|4294967447|63.21|43.07|true|nick zipper|2013-03-01 09:11:58.703135|86.10|chemistry +14|273|65615|4294967419|41.10|25.73|true|fred miller|2013-03-01 09:11:58.703112|28.00|quiet hour +103|466|65644|4294967501|22.44|18.32|true|irene white|2013-03-01 09:11:58.703268|49.90|wind surfing +116|360|65715|4294967331|70.37|18.13|true|mike ellison|2013-03-01 09:11:58.703255|35.32|joggying +28|491|65569|4294967486|24.77|5.62|true|rachel ichabod|2013-03-01 09:11:58.703168|43.41|mathematics +24|418|65693|4294967544|85.51|6.74|false|sarah thompson|2013-03-01 09:11:58.703140|93.65|american history +87|280|65783|4294967467|91.47|11.40|true|quinn falkner|2013-03-01 09:11:58.703105|79.48|debate +111|268|65654|4294967418|37.92|31.63|false|nick brown|2013-03-01 09:11:58.703260|15.75|yard duty +62|503|65756|4294967420|68.06|46.84|true|ethan falkner|2013-03-01 09:11:58.703137|68.80|study skills +114|370|65691|4294967337|7.34|48.34|true|jessica brown|2013-03-01 09:11:58.703184|18.88|nap time +-2|271|65717|4294967444|79.01|1.40|true|jessica hernandez|2013-03-01 09:11:58.703217|86.40|wind surfing +120|340|65735|4294967482|77.89|37.64|true|ulysses ichabod|2013-03-01 09:11:58.703272|89.62|wind surfing +62|440|65607|4294967334|81.02|40.77|true|victor quirinius|2013-03-01 09:11:58.703250|35.77|religion +8|292|65696|4294967343|16.83|48.40|false|gabriella underhill|2013-03-01 09:11:58.703214|26.78|xylophone band +19|334|65719|4294967465|90.49|8.62|false|david nixon|2013-03-01 09:11:58.703259|50.25|forestry +45|294|65751|4294967415|97.12|8.86|false|ulysses quirinius|2013-03-01 09:11:58.703164|43.65|philosophy +15|309|65663|4294967441|34.64|15.57|true|sarah young|2013-03-01 09:11:58.703314|54.02|forestry +4|445|65567|4294967458|27.17|35.83|true|wendy young|2013-03-01 09:11:58.703155|54.43|xylophone band +20|417|65775|4294967534|71.61|44.76|false|bob carson|2013-03-01 09:11:58.703281|33.99|american history +77|323|65770|4294967468|45.43|32.01|true|quinn polk|2013-03-01 09:11:58.703287|9.84|nap time +117|337|65732|4294967322|42.50|42.07|false|nick ichabod|2013-03-01 09:11:58.703119|50.76|yard duty +64|410|65687|4294967362|3.68|39.84|false|mike brown|2013-03-01 09:11:58.703142|84.44|undecided +79|400|65707|4294967410|58.00|39.20|false|ethan zipper|2013-03-01 09:11:58.703237|39.47|forestry +23|356|65648|4294967362|0.67|15.80|true|xavier garcia|2013-03-01 09:11:58.703290|51.78|joggying +23|410|65749|4294967483|83.39|30.81|true|bob falkner|2013-03-01 09:11:58.703245|21.55|religion +106|449|65676|4294967332|37.82|1.47|false|victor brown|2013-03-01 09:11:58.703309|14.94|yard duty +93|405|65785|4294967514|69.32|41.57|false|david brown|2013-03-01 09:11:58.703282|65.35|biology +11|462|65560|4294967296|50.33|47.48|true|luke robinson|2013-03-01 09:11:58.703229|87.60|topology +39|394|65648|4294967396|80.75|26.19|true|bob falkner|2013-03-01 09:11:58.703183|97.34|quiet hour +66|449|65642|4294967484|20.67|43.38|false|ethan white|2013-03-01 09:11:58.703242|45.45|quiet hour +121|427|65587|4294967471|3.50|44.90|false|xavier xylophone|2013-03-01 09:11:58.703320|76.13|history +50|405|65669|4294967377|6.89|20.55|true|sarah johnson|2013-03-01 09:11:58.703267|89.60|american history +57|290|65736|4294967343|56.94|47.86|false|oscar davidson|2013-03-01 09:11:58.703231|99.18|quiet hour +10|378|65656|4294967387|15.62|26.68|true|gabriella quirinius|2013-03-01 09:11:58.703081|81.67|industrial engineering +12|298|65580|4294967338|11.94|36.19|false|rachel robinson|2013-03-01 09:11:58.703176|16.53|wind surfing +81|277|65631|4294967467|18.87|18.55|true|holly ichabod|2013-03-01 09:11:58.703239|26.05|nap time +16|472|65554|4294967350|5.95|42.73|true|fred thompson|2013-03-01 09:11:58.703108|76.92|wind surfing +50|376|65608|4294967543|54.66|36.02|true|fred steinbeck|2013-03-01 09:11:58.703211|86.70|study skills +18|295|65708|4294967381|61.76|43.73|true|katie ovid|2013-03-01 09:11:58.703287|30.06|xylophone band +111|351|65734|4294967434|64.85|2.36|false|katie johnson|2013-03-01 09:11:58.703121|35.05|zync studies +68|319|65652|4294967318|52.73|23.11|false|nick steinbeck|2013-03-01 09:11:58.703104|53.65|mathematics +36|436|65597|4294967395|8.03|46.67|true|nick brown|2013-03-01 09:11:58.703109|31.54|values clariffication +124|448|65594|4294967492|15.65|12.99|true|wendy robinson|2013-03-01 09:11:58.703296|65.82|geology +87|423|65649|4294967348|42.41|11.05|true|david davidson|2013-03-01 09:11:58.703234|46.09|wind surfing +46|495|65779|4294967402|92.64|28.98|true|nick van buren|2013-03-01 09:11:58.703135|22.14|geology +11|505|65572|4294967370|1.85|39.68|true|luke johnson|2013-03-01 09:11:58.703325|22.60|xylophone band +91|287|65642|4294967351|85.43|5.50|true|wendy brown|2013-03-01 09:11:58.703181|82.48|opthamology +66|489|65786|4294967505|89.25|48.66|false|irene polk|2013-03-01 09:11:58.703320|89.14|chemistry +86|399|65598|4294967498|29.31|15.11|false|priscilla nixon|2013-03-01 09:11:58.703092|64.84|undecided +21|311|65777|4294967430|83.19|44.06|false|calvin king|2013-03-01 09:11:58.703118|8.84|nap time +80|261|65566|4294967523|40.03|27.43|false|bob hernandez|2013-03-01 09:11:58.703172|83.46|kindergarten +10|342|65585|4294967426|98.51|41.94|true|jessica ellison|2013-03-01 09:11:58.703272|56.90|opthamology +7|256|65627|4294967318|54.23|25.97|false|ulysses white|2013-03-01 09:11:58.703310|93.97|forestry +105|366|65555|4294967481|57.38|46.92|true|fred zipper|2013-03-01 09:11:58.703202|96.48|history +70|304|65665|4294967369|73.89|7.30|true|priscilla king|2013-03-01 09:11:58.703091|42.26|religion +73|318|65630|4294967414|15.37|16.45|true|luke nixon|2013-03-01 09:11:58.703249|10.04|quiet hour +120|269|65698|4294967326|36.56|34.19|true|bob allen|2013-03-01 09:11:58.703292|33.11|values clariffication +11|288|65546|4294967351|47.05|25.50|false|xavier van buren|2013-03-01 09:11:58.703197|23.82|philosophy +42|504|65695|4294967484|66.41|4.63|true|mike ellison|2013-03-01 09:11:58.703144|1.97|chemistry +29|393|65747|4294967366|27.20|0.98|false|david allen|2013-03-01 09:11:58.703293|41.05|linguistics +10|297|65676|4294967531|16.36|30.48|false|tom steinbeck|2013-03-01 09:11:58.703288|86.85|xylophone band +25|344|65554|4294967458|47.52|17.21|true|ulysses king|2013-03-01 09:11:58.703116|73.98|history +111|486|65767|4294967314|88.60|43.84|true|mike nixon|2013-03-01 09:11:58.703098|57.89|nap time +36|495|65616|4294967545|84.11|32.88|true|ulysses miller|2013-03-01 09:11:58.703089|36.61|nap time +46|279|65589|4294967353|19.73|15.74|true|sarah brown|2013-03-01 09:11:58.703139|69.89|wind surfing +107|450|65576|4294967509|78.19|6.31|true|luke van buren|2013-03-01 09:11:58.703202|28.33|xylophone band +81|300|65778|4294967321|37.54|30.28|true|oscar underhill|2013-03-01 09:11:58.703253|57.25|undecided +39|391|65777|4294967431|12.29|49.03|true|quinn underhill|2013-03-01 09:11:58.703152|3.98|debate +66|389|65708|4294967531|39.12|8.32|false|tom zipper|2013-03-01 09:11:58.703113|92.17|american history +60|344|65712|4294967535|54.49|18.69|true|irene garcia|2013-03-01 09:11:58.703136|88.30|forestry +94|313|65643|4294967502|4.63|17.67|true|alice white|2013-03-01 09:11:58.703271|75.10|values clariffication +84|355|65729|4294967485|7.06|10.12|true|ethan quirinius|2013-03-01 09:11:58.703107|51.40|study skills +59|495|65626|4294967426|47.27|42.14|true|gabriella steinbeck|2013-03-01 09:11:58.703075|21.44|forestry +34|426|65664|4294967466|30.07|28.12|false|calvin van buren|2013-03-01 09:11:58.703251|18.30|yard duty +-2|472|65765|4294967401|8.56|8.90|false|zach zipper|2013-03-01 09:11:58.703149|50.26|quiet hour +42|313|65584|4294967504|17.77|18.16|true|holly davidson|2013-03-01 09:11:58.703306|37.98|nap time +30|363|65578|4294967531|38.03|43.65|true|tom white|2013-03-01 09:11:58.703298|69.85|debate +59|456|65567|4294967366|83.76|40.48|false|ethan carson|2013-03-01 09:11:58.703269|87.52|zync studies +73|399|65789|4294967493|48.59|34.31|true|sarah king|2013-03-01 09:11:58.703313|90.29|education +83|365|65769|4294967314|61.75|49.49|false|alice laertes|2013-03-01 09:11:58.703293|33.07|debate +101|279|65586|4294967324|12.53|38.30|true|wendy hernandez|2013-03-01 09:11:58.703141|75.87|study skills +35|315|65647|4294967473|50.95|34.51|false|luke xylophone|2013-03-01 09:11:58.703113|13.27|quiet hour +13|303|65572|4294967479|59.37|14.55|true|sarah king|2013-03-01 09:11:58.703114|79.60|zync studies +49|488|65769|4294967422|29.29|38.58|false|xavier ovid|2013-03-01 09:11:58.703229|50.59|geology +21|342|65552|4294967400|53.92|49.51|false|fred steinbeck|2013-03-01 09:11:58.703317|78.24|values clariffication +65|438|65573|4294967486|24.02|0.39|true|oscar nixon|2013-03-01 09:11:58.703227|33.77|linguistics +39|459|65746|4294967375|72.84|27.19|true|bob falkner|2013-03-01 09:11:58.703276|49.54|values clariffication +47|403|65727|4294967334|66.88|9.80|false|oscar falkner|2013-03-01 09:11:58.703201|48.29|linguistics +99|286|65592|4294967412|59.24|15.86|false|fred thompson|2013-03-01 09:11:58.703254|3.27|opthamology +86|345|65712|4294967441|71.35|29.72|true|xavier carson|2013-03-01 09:11:58.703203|62.81|quiet hour +82|387|65562|4294967327|18.25|8.89|true|sarah steinbeck|2013-03-01 09:11:58.703278|94.58|education +83|368|65620|4294967469|50.71|17.23|true|victor xylophone|2013-03-01 09:11:58.703307|7.46|joggying +93|498|65750|4294967347|21.56|14.41|false|holly white|2013-03-01 09:11:58.703240|12.07|mathematics +12|431|65658|4294967437|49.66|23.29|false|ethan johnson|2013-03-01 09:11:58.703196|4.43|values clariffication +60|479|65749|4294967306|16.11|37.60|false|wendy brown|2013-03-01 09:11:58.703162|43.42|yard duty +96|398|65719|4294967380|87.64|15.01|false|yuri nixon|2013-03-01 09:11:58.703175|33.07|linguistics +76|341|65565|4294967393|80.91|40.96|false|victor ovid|2013-03-01 09:11:58.703228|79.13|opthamology +55|448|65716|4294967402|9.11|45.24|false|victor king|2013-03-01 09:11:58.703188|29.92|mathematics +88|425|65752|4294967335|46.88|1.57|true|david garcia|2013-03-01 09:11:58.703196|19.17|quiet hour +80|366|65718|4294967372|5.71|43.14|false|bob quirinius|2013-03-01 09:11:58.703168|39.71|xylophone band +119|468|65560|4294967390|87.23|10.09|false|ethan ichabod|2013-03-01 09:11:58.703268|98.76|undecided +73|492|65536|4294967365|15.92|14.83|true|tom steinbeck|2013-03-01 09:11:58.703200|45.25|american history +118|368|65777|4294967491|0.36|24.39|false|quinn brown|2013-03-01 09:11:58.703077|47.45|quiet hour +33|389|65608|4294967378|21.19|26.14|false|wendy falkner|2013-03-01 09:11:58.703221|13.58|opthamology +55|490|65614|4294967379|43.91|0.75|false|wendy ovid|2013-03-01 09:11:58.703193|40.91|education +96|352|65708|4294967507|42.03|39.55|true|katie robinson|2013-03-01 09:11:58.703218|1.38|values clariffication +76|310|65695|4294967441|36.59|28.12|false|sarah king|2013-03-01 09:11:58.703186|96.52|history +16|492|65562|4294967497|94.55|44.65|false|alice miller|2013-03-01 09:11:58.703251|65.66|geology +85|352|65750|4294967311|24.97|35.42|false|jessica white|2013-03-01 09:11:58.703081|2.80|topology +62|337|65644|4294967297|96.48|5.80|true|fred brown|2013-03-01 09:11:58.703227|25.39|opthamology +79|502|65717|4294967437|14.02|28.63|false|oscar young|2013-03-01 09:11:58.703303|58.28|nap time +-3|386|65716|4294967496|12.12|2.37|false|zach thompson|2013-03-01 09:11:58.703252|16.39|linguistics +81|496|65719|4294967313|84.54|43.42|true|jessica hernandez|2013-03-01 09:11:58.703255|27.86|geology +67|351|65701|4294967461|98.04|5.78|true|calvin allen|2013-03-01 09:11:58.703298|75.04|education +61|496|65655|4294967327|66.05|20.13|false|oscar zipper|2013-03-01 09:11:58.703226|4.28|history +62|507|65550|4294967529|55.83|23.26|true|ulysses ellison|2013-03-01 09:11:58.703308|77.07|american history +85|331|65689|4294967441|1.79|21.43|true|irene miller|2013-03-01 09:11:58.703183|88.46|undecided +4|453|65738|4294967327|13.77|41.91|true|victor johnson|2013-03-01 09:11:58.703210|44.51|religion +110|500|65729|4294967437|31.52|18.15|false|victor steinbeck|2013-03-01 09:11:58.703125|75.58|study skills +108|416|65683|4294967486|48.01|26.50|false|zach zipper|2013-03-01 09:11:58.703293|18.18|debate +46|457|65591|4294967440|29.80|16.92|false|rachel polk|2013-03-01 09:11:58.703287|53.97|debate +82|304|65625|4294967548|13.04|12.38|false|katie davidson|2013-03-01 09:11:58.703216|74.11|geology +25|285|65629|4294967489|70.23|42.83|true|victor steinbeck|2013-03-01 09:11:58.703090|90.56|biology +62|463|65757|4294967352|83.35|39.27|false|rachel ichabod|2013-03-01 09:11:58.703129|39.86|geology +82|344|65555|4294967363|41.65|22.43|true|bob white|2013-03-01 09:11:58.703245|90.56|biology +14|364|65647|4294967545|11.75|44.11|false|ethan falkner|2013-03-01 09:11:58.703266|41.68|education +92|430|65683|4294967536|19.20|36.90|false|zach thompson|2013-03-01 09:11:58.703155|79.60|mathematics +28|364|65704|4294967409|85.16|19.57|true|nick allen|2013-03-01 09:11:58.703264|84.88|industrial engineering +11|389|65612|4294967380|92.09|31.35|false|katie robinson|2013-03-01 09:11:58.703251|64.22|linguistics +88|321|65562|4294967301|6.20|1.85|true|nick ichabod|2013-03-01 09:11:58.703182|38.33|xylophone band +-2|265|65674|4294967480|70.67|25.08|true|fred zipper|2013-03-01 09:11:58.703122|80.63|religion +79|466|65752|4294967451|48.07|24.29|true|gabriella carson|2013-03-01 09:11:58.703272|59.18|kindergarten +12|427|65717|4294967507|38.26|18.34|true|xavier xylophone|2013-03-01 09:11:58.703211|43.58|wind surfing +64|300|65733|4294967322|21.34|14.46|true|sarah underhill|2013-03-01 09:11:58.703286|56.38|education +111|490|65572|4294967454|77.51|34.31|false|alice ellison|2013-03-01 09:11:58.703099|54.51|opthamology +69|264|65656|4294967304|54.24|19.82|false|mike xylophone|2013-03-01 09:11:58.703236|50.31|topology +75|413|65647|4294967519|87.63|19.88|true|sarah allen|2013-03-01 09:11:58.703138|23.93|topology +61|454|65645|4294967301|22.95|22.43|true|gabriella hernandez|2013-03-01 09:11:58.703233|42.14|yard duty +60|354|65607|4294967341|17.12|6.70|true|calvin robinson|2013-03-01 09:11:58.703173|4.31|quiet hour +119|303|65568|4294967435|75.14|45.31|false|yuri polk|2013-03-01 09:11:58.703143|59.03|nap time +38|343|65719|4294967367|66.73|3.09|false|mike garcia|2013-03-01 09:11:58.703193|18.90|opthamology +85|280|65671|4294967419|68.39|12.40|true|nick xylophone|2013-03-01 09:11:58.703284|91.33|religion +74|320|65754|4294967308|69.52|49.39|false|rachel zipper|2013-03-01 09:11:58.703258|7.33|topology +54|385|65573|4294967467|96.15|0.84|false|fred davidson|2013-03-01 09:11:58.703263|84.20|chemistry +105|415|65706|4294967486|28.82|23.39|true|yuri falkner|2013-03-01 09:11:58.703170|91.41|kindergarten +123|282|65725|4294967326|95.49|17.82|false|victor laertes|2013-03-01 09:11:58.703088|89.80|biology +59|473|65755|4294967478|90.51|28.26|false|mike king|2013-03-01 09:11:58.703198|94.03|philosophy +34|339|65652|4294967397|25.96|45.70|true|yuri robinson|2013-03-01 09:11:58.703193|26.31|debate +-2|390|65563|4294967390|44.04|34.14|true|sarah van buren|2013-03-01 09:11:58.703319|20.23|american history +78|300|65757|4294967352|74.24|26.64|false|katie davidson|2013-03-01 09:11:58.703102|27.38|forestry +28|363|65617|4294967508|85.84|24.83|false|rachel davidson|2013-03-01 09:11:58.703278|32.65|mathematics +99|481|65554|4294967413|77.90|4.03|true|luke young|2013-03-01 09:11:58.703173|99.76|values clariffication +31|321|65760|4294967447|40.32|23.71|false|rachel ovid|2013-03-01 09:11:58.703127|23.74|forestry +-2|315|65726|4294967393|70.87|49.91|false|yuri ichabod|2013-03-01 09:11:58.703076|37.48|education +3|388|65551|4294967369|17.17|38.13|false|rachel nixon|2013-03-01 09:11:58.703306|3.69|zync studies +17|495|65670|4294967397|35.41|37.90|false|quinn nixon|2013-03-01 09:11:58.703171|69.29|philosophy +18|466|65561|4294967484|88.35|18.24|true|alice polk|2013-03-01 09:11:58.703134|87.29|education +57|485|65755|4294967379|98.86|17.49|false|holly underhill|2013-03-01 09:11:58.703125|40.28|chemistry +103|296|65656|4294967406|95.75|25.92|true|alice ovid|2013-03-01 09:11:58.703093|73.06|linguistics +47|374|65735|4294967507|38.59|10.28|false|tom van buren|2013-03-01 09:11:58.703225|50.60|study skills +4|350|65615|4294967509|49.09|0.51|true|holly hernandez|2013-03-01 09:11:58.703206|65.77|wind surfing +56|474|65776|4294967448|56.66|29.48|true|rachel laertes|2013-03-01 09:11:58.703207|92.22|industrial engineering +51|411|65734|4294967331|44.52|42.05|false|tom underhill|2013-03-01 09:11:58.703149|98.08|nap time +109|491|65619|4294967485|54.10|22.84|true|gabriella thompson|2013-03-01 09:11:58.703221|40.56|zync studies +74|327|65678|4294967402|27.10|8.18|false|rachel ellison|2013-03-01 09:11:58.703251|57.69|yard duty +-2|305|65767|4294967529|76.54|4.72|true|calvin xylophone|2013-03-01 09:11:58.703083|69.43|quiet hour +12|283|65666|4294967454|34.77|39.37|false|gabriella ellison|2013-03-01 09:11:58.703160|25.17|linguistics +57|472|65694|4294967477|15.43|25.96|false|yuri king|2013-03-01 09:11:58.703175|67.18|debate +100|342|65724|4294967391|98.08|36.79|true|nick xylophone|2013-03-01 09:11:58.703234|72.39|topology +32|399|65734|4294967375|34.25|29.96|false|priscilla allen|2013-03-01 09:11:58.703321|37.97|opthamology +22|487|65575|4294967333|27.92|19.97|false|sarah thompson|2013-03-01 09:11:58.703089|63.17|chemistry +58|470|65536|4294967375|45.66|11.16|false|katie ellison|2013-03-01 09:11:58.703077|74.47|geology +73|395|65751|4294967335|3.98|35.78|true|alice polk|2013-03-01 09:11:58.703245|36.93|values clariffication +-3|315|65671|4294967412|94.22|25.96|true|oscar johnson|2013-03-01 09:11:58.703133|88.55|nap time +99|324|65580|4294967323|78.50|27.04|false|quinn robinson|2013-03-01 09:11:58.703316|41.51|american history +95|290|65717|4294967342|99.88|8.08|false|sarah ovid|2013-03-01 09:11:58.703243|82.69|biology +11|355|65762|4294967517|33.96|48.02|true|xavier zipper|2013-03-01 09:11:58.703164|13.88|chemistry +96|407|65783|4294967463|89.64|9.19|true|zach laertes|2013-03-01 09:11:58.703166|38.46|xylophone band +37|399|65760|4294967318|79.01|35.35|true|luke steinbeck|2013-03-01 09:11:58.703150|86.93|biology +102|336|65727|4294967428|89.22|21.22|true|quinn carson|2013-03-01 09:11:58.703087|88.90|industrial engineering +35|432|65701|4294967304|34.73|5.96|true|jessica falkner|2013-03-01 09:11:58.703313|26.77|linguistics +65|392|65553|4294967431|62.35|40.49|false|wendy thompson|2013-03-01 09:11:58.703281|28.41|opthamology +16|366|65756|4294967513|52.76|9.48|false|katie miller|2013-03-01 09:11:58.703216|77.97|kindergarten +74|488|65694|4294967528|73.85|3.80|false|oscar falkner|2013-03-01 09:11:58.703228|98.35|values clariffication +66|329|65676|4294967319|18.46|9.95|true|david allen|2013-03-01 09:11:58.703200|14.55|topology +110|444|65645|4294967392|11.86|14.13|true|yuri johnson|2013-03-01 09:11:58.703304|67.14|linguistics +45|420|65787|4294967397|25.24|11.80|true|oscar nixon|2013-03-01 09:11:58.703243|61.00|zync studies +54|487|65591|4294967500|36.84|28.73|false|oscar ichabod|2013-03-01 09:11:58.703073|1.65|geology +27|305|65603|4294967530|61.72|26.95|true|irene laertes|2013-03-01 09:11:58.703212|61.59|nap time +115|503|65724|4294967441|28.33|3.74|true|calvin king|2013-03-01 09:11:58.703310|1.36|undecided +107|315|65550|4294967441|30.16|4.43|true|gabriella ellison|2013-03-01 09:11:58.703195|37.02|nap time +48|358|65633|4294967303|76.01|29.13|false|xavier carson|2013-03-01 09:11:58.703269|38.24|undecided +82|508|65765|4294967516|13.73|9.98|true|bob brown|2013-03-01 09:11:58.703162|74.60|geology +93|450|65632|4294967470|8.96|43.67|true|rachel robinson|2013-03-01 09:11:58.703124|13.94|xylophone band +89|498|65739|4294967499|30.83|2.11|false|oscar white|2013-03-01 09:11:58.703227|52.95|forestry +85|502|65776|4294967350|76.12|43.33|true|oscar underhill|2013-03-01 09:11:58.703272|11.14|debate +70|404|65747|4294967351|57.41|1.67|false|zach robinson|2013-03-01 09:11:58.703175|3.16|religion +108|279|65696|4294967320|82.52|37.07|true|katie johnson|2013-03-01 09:11:58.703075|77.92|chemistry +32|294|65695|4294967538|62.87|45.18|false|jessica miller|2013-03-01 09:11:58.703123|56.65|undecided +27|355|65588|4294967521|48.10|11.93|false|fred garcia|2013-03-01 09:11:58.703267|43.47|mathematics +39|418|65670|4294967296|91.41|47.22|true|calvin young|2013-03-01 09:11:58.703265|25.10|industrial engineering +18|258|65551|4294967366|91.56|44.35|true|bob falkner|2013-03-01 09:11:58.703271|19.14|philosophy +101|420|65644|4294967408|29.00|24.68|false|irene johnson|2013-03-01 09:11:58.703215|99.37|linguistics +60|500|65604|4294967472|47.85|36.76|false|tom quirinius|2013-03-01 09:11:58.703126|91.67|philosophy +57|313|65537|4294967419|26.62|5.16|false|mike garcia|2013-03-01 09:11:58.703112|43.45|american history +11|333|65553|4294967550|6.90|18.23|true|fred ichabod|2013-03-01 09:11:58.703312|65.30|zync studies +100|285|65632|4294967302|1.84|3.76|true|tom robinson|2013-03-01 09:11:58.703224|94.95|joggying +105|341|65670|4294967324|11.40|24.18|false|oscar nixon|2013-03-01 09:11:58.703178|55.94|study skills +2|396|65705|4294967527|15.53|6.16|false|xavier miller|2013-03-01 09:11:58.703122|66.23|chemistry +47|399|65703|4294967346|74.05|14.18|true|sarah zipper|2013-03-01 09:11:58.703137|84.02|kindergarten +58|299|65623|4294967536|15.56|42.31|true|alice garcia|2013-03-01 09:11:58.703204|69.48|yard duty +62|453|65624|4294967394|68.71|47.89|false|mike falkner|2013-03-01 09:11:58.703163|53.02|joggying +19|464|65703|4294967364|57.60|23.26|true|katie ovid|2013-03-01 09:11:58.703298|65.75|biology +25|354|65766|4294967364|16.19|17.26|false|alice robinson|2013-03-01 09:11:58.703239|37.84|chemistry +101|281|65677|4294967368|52.23|44.11|false|oscar davidson|2013-03-01 09:11:58.703137|60.48|chemistry +77|311|65732|4294967469|23.22|28.42|false|ethan xylophone|2013-03-01 09:11:58.703264|41.43|kindergarten +75|395|65570|4294967332|49.32|28.96|true|gabriella quirinius|2013-03-01 09:11:58.703179|38.74|education +103|377|65732|4294967523|43.23|39.39|false|ulysses king|2013-03-01 09:11:58.703268|8.64|american history +44|492|65677|4294967519|53.78|8.39|false|rachel white|2013-03-01 09:11:58.703207|49.67|study skills +112|378|65689|4294967474|9.09|39.48|true|holly garcia|2013-03-01 09:11:58.703200|14.64|yard duty +29|504|65772|4294967519|71.99|27.30|true|victor xylophone|2013-03-01 09:11:58.703077|13.94|topology +5|480|65567|4294967498|41.66|2.89|true|bob garcia|2013-03-01 09:11:58.703265|69.47|opthamology +70|327|65681|4294967529|15.29|30.06|false|mike quirinius|2013-03-01 09:11:58.703203|38.23|zync studies +15|410|65642|4294967355|63.52|13.55|true|jessica quirinius|2013-03-01 09:11:58.703148|2.20|chemistry +106|313|65663|4294967464|35.80|11.45|true|victor xylophone|2013-03-01 09:11:58.703095|78.55|wind surfing +4|427|65570|4294967301|44.69|39.27|false|gabriella hernandez|2013-03-01 09:11:58.703103|6.39|education +33|319|65753|4294967349|40.67|5.48|true|ulysses robinson|2013-03-01 09:11:58.703224|98.53|industrial engineering +47|273|65592|4294967391|75.98|31.67|true|david carson|2013-03-01 09:11:58.703119|44.17|history +63|391|65618|4294967387|23.53|10.03|true|sarah underhill|2013-03-01 09:11:58.703100|19.99|yard duty +35|457|65655|4294967447|7.44|44.44|false|zach laertes|2013-03-01 09:11:58.703219|18.04|joggying +100|307|65692|4294967499|66.45|13.19|false|tom xylophone|2013-03-01 09:11:58.703225|61.38|american history +74|372|65748|4294967385|1.60|44.86|false|luke ellison|2013-03-01 09:11:58.703139|29.06|history +76|415|65770|4294967296|2.14|47.61|false|ulysses ichabod|2013-03-01 09:11:58.703325|4.60|industrial engineering +90|476|65768|4294967444|36.62|43.90|false|sarah ellison|2013-03-01 09:11:58.703111|45.70|debate +92|303|65695|4294967343|84.51|1.23|true|priscilla zipper|2013-03-01 09:11:58.703196|22.71|religion +4|433|65590|4294967364|50.90|37.26|true|jessica garcia|2013-03-01 09:11:58.703299|10.50|industrial engineering +15|510|65596|4294967394|95.69|7.58|false|holly allen|2013-03-01 09:11:58.703283|39.61|study skills +67|298|65621|4294967323|49.45|34.30|true|yuri allen|2013-03-01 09:11:58.703122|11.66|linguistics +24|387|65684|4294967461|19.83|25.63|false|rachel van buren|2013-03-01 09:11:58.703192|78.73|zync studies +107|286|65627|4294967341|92.77|3.26|true|zach garcia|2013-03-01 09:11:58.703084|42.88|nap time +103|467|65586|4294967337|37.95|47.86|true|calvin miller|2013-03-01 09:11:58.703133|89.92|joggying +109|290|65573|4294967474|61.32|30.01|true|bob van buren|2013-03-01 09:11:58.703280|68.49|education +51|391|65738|4294967329|43.75|46.72|true|wendy miller|2013-03-01 09:11:58.703315|23.48|nap time +43|465|65642|4294967357|32.98|23.75|false|wendy steinbeck|2013-03-01 09:11:58.703266|1.22|quiet hour +49|415|65642|4294967395|48.03|32.67|false|sarah white|2013-03-01 09:11:58.703272|46.21|biology +77|260|65700|4294967545|26.90|5.84|false|zach young|2013-03-01 09:11:58.703325|87.00|undecided +75|439|65754|4294967519|29.03|2.70|true|xavier quirinius|2013-03-01 09:11:58.703278|67.83|mathematics +37|277|65580|4294967538|55.27|38.84|true|gabriella nixon|2013-03-01 09:11:58.703232|75.48|debate +109|496|65683|4294967404|97.35|5.85|true|ethan xylophone|2013-03-01 09:11:58.703090|65.82|philosophy +123|407|65764|4294967485|84.22|39.79|false|bob king|2013-03-01 09:11:58.703196|35.73|geology +103|422|65609|4294967476|77.21|42.21|true|wendy falkner|2013-03-01 09:11:58.703294|63.90|wind surfing +115|490|65557|4294967461|37.07|27.10|false|ulysses underhill|2013-03-01 09:11:58.703324|54.55|american history +81|436|65767|4294967437|61.46|43.83|false|tom ichabod|2013-03-01 09:11:58.703232|32.92|wind surfing +55|274|65555|4294967501|76.37|21.85|true|yuri miller|2013-03-01 09:11:58.703095|29.49|linguistics +8|368|65633|4294967453|74.29|10.41|false|priscilla allen|2013-03-01 09:11:58.703243|43.80|philosophy +48|494|65649|4294967512|77.11|13.67|false|tom davidson|2013-03-01 09:11:58.703131|40.12|xylophone band +122|259|65714|4294967513|74.85|38.93|false|rachel xylophone|2013-03-01 09:11:58.703230|9.39|geology +62|407|65647|4294967338|50.46|7.69|true|jessica allen|2013-03-01 09:11:58.703075|49.89|xylophone band +101|416|65698|4294967327|55.64|45.18|true|quinn nixon|2013-03-01 09:11:58.703271|6.82|debate +60|276|65651|4294967551|96.88|42.31|true|priscilla quirinius|2013-03-01 09:11:58.703131|55.75|geology +117|376|65774|4294967320|32.31|25.85|true|ulysses ovid|2013-03-01 09:11:58.703308|72.97|undecided +-1|484|65587|4294967410|58.42|8.47|false|bob white|2013-03-01 09:11:58.703161|65.50|debate +88|454|65694|4294967417|85.88|2.93|false|xavier ellison|2013-03-01 09:11:58.703146|13.90|industrial engineering +2|261|65587|4294967501|3.10|8.82|true|zach king|2013-03-01 09:11:58.703314|41.30|history +120|343|65610|4294967491|5.09|27.28|false|bob ovid|2013-03-01 09:11:58.703139|64.12|biology +28|284|65587|4294967448|9.14|20.28|true|irene quirinius|2013-03-01 09:11:58.703319|4.33|biology +46|289|65563|4294967493|84.61|45.85|true|luke johnson|2013-03-01 09:11:58.703155|73.48|forestry +92|290|65754|4294967351|23.34|15.70|true|ulysses garcia|2013-03-01 09:11:58.703248|47.94|religion +95|303|65604|4294967395|65.21|6.32|true|nick brown|2013-03-01 09:11:58.703150|88.11|study skills +121|466|65537|4294967374|46.87|9.72|false|wendy garcia|2013-03-01 09:11:58.703141|56.52|debate +39|288|65737|4294967410|53.95|34.84|true|calvin young|2013-03-01 09:11:58.703214|27.71|education +64|357|65677|4294967458|51.88|20.69|true|nick young|2013-03-01 09:11:58.703222|74.09|philosophy +117|434|65553|4294967514|36.40|26.64|true|fred zipper|2013-03-01 09:11:58.703324|43.77|xylophone band +-1|429|65739|4294967472|55.04|34.07|false|victor brown|2013-03-01 09:11:58.703229|9.71|forestry +33|415|65781|4294967503|88.66|35.06|false|oscar nixon|2013-03-01 09:11:58.703109|72.87|education +24|440|65749|4294967307|67.89|13.96|false|rachel johnson|2013-03-01 09:11:58.703127|7.37|topology +82|486|65672|4294967394|14.82|33.28|true|irene falkner|2013-03-01 09:11:58.703184|75.40|zync studies +41|336|65618|4294967325|10.59|37.40|true|katie polk|2013-03-01 09:11:58.703177|45.21|religion +104|285|65781|4294967429|74.64|3.34|false|alice laertes|2013-03-01 09:11:58.703234|73.16|wind surfing +55|360|65684|4294967310|13.08|21.56|true|calvin young|2013-03-01 09:11:58.703192|62.92|undecided +56|505|65615|4294967374|28.17|29.67|true|mike zipper|2013-03-01 09:11:58.703147|69.41|study skills +35|363|65708|4294967358|37.99|26.35|true|mike carson|2013-03-01 09:11:58.703104|47.25|quiet hour +114|341|65602|4294967344|6.88|39.44|true|ethan king|2013-03-01 09:11:58.703229|51.76|xylophone band +2|470|65735|4294967314|40.93|44.99|true|david polk|2013-03-01 09:11:58.703096|33.22|wind surfing +92|340|65779|4294967513|26.36|39.07|false|gabriella steinbeck|2013-03-01 09:11:58.703135|42.04|chemistry +112|272|65701|4294967481|37.89|41.60|false|nick king|2013-03-01 09:11:58.703267|3.47|philosophy +71|509|65661|4294967425|82.49|45.28|false|katie garcia|2013-03-01 09:11:58.703138|68.10|american history +13|401|65689|4294967321|74.53|9.41|false|alice johnson|2013-03-01 09:11:58.703204|89.77|biology +70|439|65712|4294967431|22.10|0.94|false|luke young|2013-03-01 09:11:58.703107|89.71|american history +12|437|65789|4294967306|89.79|4.07|false|priscilla king|2013-03-01 09:11:58.703199|99.04|mathematics +5|387|65738|4294967450|49.76|49.15|false|wendy king|2013-03-01 09:11:58.703083|15.46|philosophy +116|354|65690|4294967522|35.31|5.45|false|fred ovid|2013-03-01 09:11:58.703289|1.72|values clariffication +6|370|65556|4294967535|25.00|15.26|false|rachel davidson|2013-03-01 09:11:58.703313|98.54|religion +66|390|65750|4294967405|53.47|26.67|true|xavier steinbeck|2013-03-01 09:11:58.703223|32.41|mathematics +30|283|65607|4294967426|19.21|14.15|true|zach underhill|2013-03-01 09:11:58.703123|97.70|yard duty +119|286|65561|4294967397|29.55|22.02|true|priscilla steinbeck|2013-03-01 09:11:58.703123|61.84|chemistry +29|279|65732|4294967531|37.19|10.59|false|irene ellison|2013-03-01 09:11:58.703177|34.38|philosophy +112|485|65546|4294967544|18.71|45.68|true|jessica white|2013-03-01 09:11:58.703251|5.24|nap time +106|375|65673|4294967530|67.32|12.78|true|victor king|2013-03-01 09:11:58.703270|69.28|biology +104|289|65766|4294967516|4.95|30.14|false|rachel falkner|2013-03-01 09:11:58.703251|34.87|religion +19|301|65789|4294967435|4.09|23.71|true|priscilla ichabod|2013-03-01 09:11:58.703151|60.01|geology +44|320|65598|4294967352|77.98|37.71|false|xavier thompson|2013-03-01 09:11:58.703139|64.33|wind surfing +112|490|65627|4294967460|88.85|11.86|false|ethan johnson|2013-03-01 09:11:58.703160|47.46|opthamology +36|471|65543|4294967510|14.34|40.27|false|bob xylophone|2013-03-01 09:11:58.703288|68.72|industrial engineering +106|446|65718|4294967355|38.63|41.60|false|tom johnson|2013-03-01 09:11:58.703181|50.84|zync studies +73|418|65768|4294967526|21.60|1.03|false|zach xylophone|2013-03-01 09:11:58.703213|98.29|opthamology +73|486|65597|4294967422|78.71|24.82|false|xavier davidson|2013-03-01 09:11:58.703245|30.67|study skills +112|438|65576|4294967315|48.49|18.03|false|oscar garcia|2013-03-01 09:11:58.703095|33.51|education +23|462|65635|4294967305|62.15|18.79|false|gabriella falkner|2013-03-01 09:11:58.703283|5.68|quiet hour +106|409|65709|4294967462|80.13|42.58|true|katie thompson|2013-03-01 09:11:58.703211|16.61|nap time +48|258|65611|4294967396|49.13|0.49|true|alice nixon|2013-03-01 09:11:58.703325|82.03|education +47|397|65739|4294967325|25.42|5.79|false|katie young|2013-03-01 09:11:58.703271|78.91|topology +117|277|65728|4294967543|30.52|45.65|true|yuri johnson|2013-03-01 09:11:58.703073|74.08|zync studies +97|344|65715|4294967498|63.50|24.20|true|david nixon|2013-03-01 09:11:58.703277|63.99|forestry +112|262|65681|4294967296|12.99|30.41|true|jessica ellison|2013-03-01 09:11:58.703214|62.09|linguistics +12|268|65659|4294967515|48.00|10.15|true|sarah steinbeck|2013-03-01 09:11:58.703283|24.84|quiet hour +37|388|65593|4294967480|38.73|5.73|true|zach johnson|2013-03-01 09:11:58.703073|46.62|debate +77|323|65751|4294967448|71.42|34.41|true|bob xylophone|2013-03-01 09:11:58.703291|47.54|education +2|426|65786|4294967347|89.27|18.86|false|wendy steinbeck|2013-03-01 09:11:58.703269|19.61|philosophy +75|478|65681|4294967459|67.19|27.74|false|mike johnson|2013-03-01 09:11:58.703279|25.88|industrial engineering +-3|374|65731|4294967388|22.35|22.71|true|bob johnson|2013-03-01 09:11:58.703204|79.77|biology +124|308|65782|4294967440|74.91|14.43|true|bob white|2013-03-01 09:11:58.703315|79.64|quiet hour +66|501|65622|4294967330|11.32|22.52|true|jessica robinson|2013-03-01 09:11:58.703071|74.55|mathematics +32|457|65616|4294967409|12.86|44.19|true|calvin miller|2013-03-01 09:11:58.703147|74.52|mathematics +79|404|65595|4294967474|99.16|10.10|true|bob underhill|2013-03-01 09:11:58.703222|51.69|kindergarten +43|301|65688|4294967450|93.34|29.59|true|ulysses zipper|2013-03-01 09:11:58.703077|64.19|linguistics +82|379|65581|4294967479|44.02|29.03|false|katie steinbeck|2013-03-01 09:11:58.703263|52.12|geology +115|423|65622|4294967383|79.48|3.98|true|jessica underhill|2013-03-01 09:11:58.703223|33.59|religion +82|455|65703|4294967411|19.41|49.17|false|gabriella ichabod|2013-03-01 09:11:58.703203|70.51|wind surfing +2|336|65724|4294967358|16.63|39.98|false|fred white|2013-03-01 09:11:58.703148|79.02|religion +89|486|65619|4294967496|40.45|6.44|true|holly quirinius|2013-03-01 09:11:58.703090|51.91|study skills +70|296|65718|4294967371|78.59|10.85|false|david falkner|2013-03-01 09:11:58.703148|13.04|xylophone band +72|447|65776|4294967420|17.99|33.37|true|quinn ovid|2013-03-01 09:11:58.703317|81.61|zync studies +31|322|65700|4294967343|47.33|36.34|false|victor ellison|2013-03-01 09:11:58.703319|32.10|religion +104|381|65655|4294967364|94.13|17.62|false|gabriella zipper|2013-03-01 09:11:58.703111|61.11|biology +78|437|65595|4294967517|95.30|31.43|false|xavier zipper|2013-03-01 09:11:58.703146|31.34|wind surfing +100|506|65763|4294967461|86.91|29.63|false|ulysses thompson|2013-03-01 09:11:58.703171|45.29|forestry +72|357|65726|4294967509|62.08|20.08|true|priscilla johnson|2013-03-01 09:11:58.703247|94.42|topology +1|412|65577|4294967334|67.57|6.78|false|victor falkner|2013-03-01 09:11:58.703201|45.81|forestry +62|477|65692|4294967430|10.89|36.98|false|calvin brown|2013-03-01 09:11:58.703070|26.68|study skills +77|319|65695|4294967549|11.40|19.73|false|alice van buren|2013-03-01 09:11:58.703175|81.37|joggying +90|379|65569|4294967463|17.12|18.66|false|sarah white|2013-03-01 09:11:58.703309|37.07|industrial engineering +65|343|65787|4294967550|3.03|27.11|false|alice ellison|2013-03-01 09:11:58.703151|79.28|xylophone band +27|303|65700|4294967447|78.58|30.67|false|gabriella davidson|2013-03-01 09:11:58.703110|8.26|nap time +3|507|65724|4294967411|38.05|47.37|true|wendy polk|2013-03-01 09:11:58.703211|2.22|topology +82|420|65594|4294967412|47.36|39.78|false|katie allen|2013-03-01 09:11:58.703197|25.56|industrial engineering +102|427|65746|4294967376|46.41|41.72|false|priscilla xylophone|2013-03-01 09:11:58.703256|91.37|history +68|356|65696|4294967539|64.66|20.44|true|holly underhill|2013-03-01 09:11:58.703172|62.89|topology +52|509|65727|4294967365|68.32|8.89|false|ulysses nixon|2013-03-01 09:11:58.703226|80.69|chemistry +65|467|65687|4294967538|33.81|30.96|true|calvin ichabod|2013-03-01 09:11:58.703220|48.77|philosophy +121|261|65662|4294967343|39.95|41.90|false|bob brown|2013-03-01 09:11:58.703125|47.79|religion +91|310|65679|4294967325|92.00|9.82|false|zach robinson|2013-03-01 09:11:58.703158|57.09|forestry +31|382|65729|4294967468|95.79|38.38|true|jessica underhill|2013-03-01 09:11:58.703230|24.37|topology +66|285|65775|4294967353|38.74|22.91|false|luke brown|2013-03-01 09:11:58.703156|85.52|zync studies +27|256|65580|4294967477|64.81|39.73|false|victor robinson|2013-03-01 09:11:58.703147|93.55|mathematics +117|340|65759|4294967370|48.13|41.76|false|ethan steinbeck|2013-03-01 09:11:58.703325|0.06|values clariffication +66|358|65618|4294967458|30.47|0.12|true|irene king|2013-03-01 09:11:58.703316|52.80|industrial engineering +23|395|65750|4294967539|33.41|48.06|false|ethan laertes|2013-03-01 09:11:58.703247|36.38|religion +109|405|65618|4294967378|39.27|4.36|true|katie young|2013-03-01 09:11:58.703127|19.21|study skills +114|323|65727|4294967428|44.93|42.75|true|alice hernandez|2013-03-01 09:11:58.703253|57.50|debate +52|323|65778|4294967490|78.59|14.07|false|oscar polk|2013-03-01 09:11:58.703137|98.90|quiet hour +46|327|65728|4294967404|67.80|37.97|false|david robinson|2013-03-01 09:11:58.703325|73.52|education +87|307|65567|4294967314|3.43|23.65|false|oscar garcia|2013-03-01 09:11:58.703162|34.23|kindergarten +39|292|65611|4294967545|42.08|27.52|false|priscilla brown|2013-03-01 09:11:58.703071|31.85|debate +39|365|65727|4294967437|71.18|2.39|true|gabriella white|2013-03-01 09:11:58.703106|81.37|biology +123|337|65591|4294967510|57.04|15.73|false|holly ovid|2013-03-01 09:11:58.703250|86.19|forestry +113|365|65686|4294967369|46.27|35.73|true|jessica robinson|2013-03-01 09:11:58.703200|56.20|debate +14|435|65566|4294967320|71.42|36.39|true|xavier ovid|2013-03-01 09:11:58.703296|48.84|wind surfing +55|310|65569|4294967466|59.85|7.63|false|oscar xylophone|2013-03-01 09:11:58.703295|19.30|religion +46|284|65653|4294967390|84.16|12.11|false|xavier brown|2013-03-01 09:11:58.703092|24.79|philosophy +8|298|65752|4294967377|55.97|37.12|false|tom nixon|2013-03-01 09:11:58.703156|49.99|history +76|490|65542|4294967318|24.15|40.52|true|rachel garcia|2013-03-01 09:11:58.703233|15.89|history +68|309|65689|4294967517|23.05|45.64|true|calvin davidson|2013-03-01 09:11:58.703198|85.65|biology +110|437|65728|4294967527|43.67|46.77|false|wendy brown|2013-03-01 09:11:58.703282|71.07|american history +67|415|65609|4294967366|99.21|36.86|true|ulysses robinson|2013-03-01 09:11:58.703214|75.90|joggying +11|287|65776|4294967531|88.29|12.43|true|katie johnson|2013-03-01 09:11:58.703105|42.34|opthamology +48|257|65541|4294967501|51.26|18.91|false|xavier brown|2013-03-01 09:11:58.703073|20.95|history +85|339|65580|4294967444|75.75|28.03|true|fred ovid|2013-03-01 09:11:58.703300|87.29|debate +30|372|65683|4294967520|60.58|38.44|true|alice laertes|2013-03-01 09:11:58.703248|30.59|kindergarten +65|408|65603|4294967541|48.16|21.67|true|alice hernandez|2013-03-01 09:11:58.703087|36.95|quiet hour +14|488|65763|4294967510|4.41|28.17|true|ethan hernandez|2013-03-01 09:11:58.703249|53.83|topology +45|383|65720|4294967401|65.44|37.61|true|tom brown|2013-03-01 09:11:58.703090|22.04|linguistics +11|451|65714|4294967529|97.64|43.88|true|nick van buren|2013-03-01 09:11:58.703181|90.02|nap time +18|471|65601|4294967496|83.28|28.25|false|zach polk|2013-03-01 09:11:58.703260|22.76|kindergarten +80|387|65672|4294967433|59.91|33.12|false|priscilla van buren|2013-03-01 09:11:58.703243|99.76|wind surfing +86|300|65762|4294967472|6.66|28.94|true|jessica brown|2013-03-01 09:11:58.703077|75.44|philosophy +41|256|65668|4294967531|92.71|36.78|true|irene polk|2013-03-01 09:11:58.703073|88.16|joggying +52|296|65576|4294967513|28.60|8.67|false|luke johnson|2013-03-01 09:11:58.703170|34.61|industrial engineering +70|301|65600|4294967426|94.67|25.19|false|holly ichabod|2013-03-01 09:11:58.703114|34.55|kindergarten +24|350|65558|4294967535|55.53|30.31|true|xavier falkner|2013-03-01 09:11:58.703275|51.59|biology +27|469|65693|4294967303|7.18|13.74|true|rachel nixon|2013-03-01 09:11:58.703092|90.62|debate +59|296|65568|4294967305|83.28|12.06|false|ulysses hernandez|2013-03-01 09:11:58.703077|89.53|debate +60|433|65734|4294967497|69.35|37.99|true|luke underhill|2013-03-01 09:11:58.703133|64.09|study skills +25|347|65709|4294967446|22.61|6.52|false|mike laertes|2013-03-01 09:11:58.703112|50.75|values clariffication +49|398|65609|4294967322|69.18|45.82|true|alice nixon|2013-03-01 09:11:58.703325|47.09|education +111|336|65597|4294967370|80.12|6.43|false|alice laertes|2013-03-01 09:11:58.703141|45.30|study skills +71|465|65697|4294967519|8.45|33.50|false|bob king|2013-03-01 09:11:58.703283|36.73|history +99|360|65622|4294967476|78.72|45.24|true|alice johnson|2013-03-01 09:11:58.703109|79.20|nap time +60|420|65779|4294967465|50.50|14.63|false|zach davidson|2013-03-01 09:11:58.703183|31.60|topology +104|463|65561|4294967502|52.64|37.36|false|quinn underhill|2013-03-01 09:11:58.703296|87.28|yard duty +81|352|65597|4294967334|94.30|14.64|true|zach xylophone|2013-03-01 09:11:58.703174|69.90|wind surfing +1|397|65556|4294967384|73.95|46.88|false|luke thompson|2013-03-01 09:11:58.703240|79.45|quiet hour +103|372|65661|4294967341|13.64|29.67|false|fred ovid|2013-03-01 09:11:58.703307|4.62|american history +55|286|65748|4294967437|62.15|34.28|true|victor falkner|2013-03-01 09:11:58.703236|8.89|geology +46|279|65759|4294967384|18.54|27.25|true|ulysses zipper|2013-03-01 09:11:58.703224|87.42|history +93|507|65595|4294967413|94.37|9.08|false|calvin carson|2013-03-01 09:11:58.703094|8.54|zync studies +62|327|65564|4294967482|34.74|46.31|true|irene carson|2013-03-01 09:11:58.703213|55.90|philosophy +39|302|65655|4294967311|71.58|11.61|true|fred van buren|2013-03-01 09:11:58.703090|75.88|topology +25|373|65721|4294967327|59.34|27.10|false|priscilla polk|2013-03-01 09:11:58.703227|88.96|yard duty +31|507|65549|4294967393|6.24|9.55|false|bob xylophone|2013-03-01 09:11:58.703143|17.08|history +0|291|65752|4294967323|33.40|16.33|true|irene ovid|2013-03-01 09:11:58.703319|12.13|biology +24|282|65542|4294967480|78.58|24.87|false|rachel thompson|2013-03-01 09:11:58.703070|89.64|chemistry +15|300|65746|4294967323|77.73|49.95|false|zach ellison|2013-03-01 09:11:58.703120|98.07|nap time +1|375|65769|4294967376|62.09|3.50|true|rachel miller|2013-03-01 09:11:58.703144|6.90|xylophone band +16|281|65727|4294967524|65.15|15.25|false|yuri ichabod|2013-03-01 09:11:58.703200|18.38|nap time +73|434|65740|4294967320|44.62|4.17|false|priscilla ichabod|2013-03-01 09:11:58.703301|89.87|linguistics +58|263|65651|4294967520|51.39|35.41|false|alice steinbeck|2013-03-01 09:11:58.703220|60.46|study skills +16|422|65570|4294967393|68.48|14.41|true|calvin nixon|2013-03-01 09:11:58.703248|69.97|study skills +122|507|65638|4294967312|57.46|3.36|false|victor davidson|2013-03-01 09:11:58.703312|27.18|xylophone band +66|344|65699|4294967533|82.51|49.92|false|xavier robinson|2013-03-01 09:11:58.703271|61.67|xylophone band +15|328|65625|4294967473|89.38|17.46|false|tom davidson|2013-03-01 09:11:58.703251|88.80|industrial engineering +20|331|65779|4294967325|39.96|9.22|false|zach steinbeck|2013-03-01 09:11:58.703213|36.32|education +24|446|65618|4294967329|95.15|39.73|false|wendy young|2013-03-01 09:11:58.703163|54.45|quiet hour +-2|503|65595|4294967521|89.18|38.53|false|sarah falkner|2013-03-01 09:11:58.703252|37.45|philosophy +31|477|65760|4294967317|95.93|7.41|true|jessica miller|2013-03-01 09:11:58.703164|75.66|zync studies +79|416|65591|4294967418|88.81|11.76|false|quinn white|2013-03-01 09:11:58.703115|83.73|forestry +3|485|65620|4294967531|86.02|31.08|true|nick thompson|2013-03-01 09:11:58.703314|86.12|wind surfing +76|270|65717|4294967345|36.09|47.64|false|alice hernandez|2013-03-01 09:11:58.703183|8.84|kindergarten +4|504|65751|4294967410|76.93|7.25|true|gabriella falkner|2013-03-01 09:11:58.703193|21.02|geology +23|479|65727|4294967500|25.30|20.63|true|yuri steinbeck|2013-03-01 09:11:58.703129|86.87|philosophy +33|441|65539|4294967417|62.66|9.80|false|luke ovid|2013-03-01 09:11:58.703296|44.36|kindergarten +47|293|65544|4294967493|54.15|23.75|true|irene brown|2013-03-01 09:11:58.703266|88.79|linguistics +15|481|65727|4294967387|7.20|4.09|false|holly xylophone|2013-03-01 09:11:58.703165|45.84|american history +51|498|65660|4294967340|67.88|10.21|true|ethan ichabod|2013-03-01 09:11:58.703082|8.69|undecided +73|314|65745|4294967463|1.18|48.05|true|oscar davidson|2013-03-01 09:11:58.703191|47.61|forestry +19|277|65539|4294967400|75.43|26.94|true|david polk|2013-03-01 09:11:58.703188|89.14|joggying +123|274|65702|4294967364|2.17|8.68|true|mike robinson|2013-03-01 09:11:58.703265|88.18|industrial engineering +67|471|65755|4294967363|88.08|5.95|true|quinn van buren|2013-03-01 09:11:58.703072|61.77|topology +124|509|65650|4294967416|62.79|27.10|false|katie young|2013-03-01 09:11:58.703159|23.20|philosophy +78|258|65776|4294967329|18.65|0.20|true|wendy king|2013-03-01 09:11:58.703313|3.90|xylophone band +1|500|65747|4294967474|62.09|13.65|false|wendy falkner|2013-03-01 09:11:58.703072|3.40|education +90|414|65788|4294967430|39.71|37.80|true|calvin white|2013-03-01 09:11:58.703155|92.28|industrial engineering +112|363|65670|4294967346|89.07|43.99|true|holly xylophone|2013-03-01 09:11:58.703150|56.64|biology +123|495|65560|4294967534|30.61|23.45|false|rachel young|2013-03-01 09:11:58.703097|13.45|chemistry +75|460|65790|4294967344|82.32|9.53|true|zach white|2013-03-01 09:11:58.703233|14.76|opthamology +82|304|65712|4294967361|46.82|37.82|true|fred hernandez|2013-03-01 09:11:58.703275|64.49|kindergarten +55|274|65733|4294967499|6.82|49.58|false|rachel falkner|2013-03-01 09:11:58.703083|86.60|geology +117|324|65728|4294967371|20.74|24.52|true|tom hernandez|2013-03-01 09:11:58.703279|49.89|education +103|431|65745|4294967438|47.93|9.16|false|nick ellison|2013-03-01 09:11:58.703151|86.19|joggying +69|306|65624|4294967318|50.25|10.89|false|alice xylophone|2013-03-01 09:11:58.703104|48.32|debate +110|508|65566|4294967352|23.27|25.06|true|yuri garcia|2013-03-01 09:11:58.703072|79.38|debate +70|446|65615|4294967381|50.71|32.21|false|ulysses hernandez|2013-03-01 09:11:58.703117|88.83|history +-1|390|65747|4294967363|56.02|43.55|false|alice carson|2013-03-01 09:11:58.703261|13.40|nap time +96|434|65567|4294967481|37.16|19.45|false|calvin young|2013-03-01 09:11:58.703079|37.84|american history +17|321|65672|4294967448|17.58|16.11|true|jessica davidson|2013-03-01 09:11:58.703313|42.17|values clariffication +116|402|65721|4294967410|66.36|18.84|false|priscilla ichabod|2013-03-01 09:11:58.703201|57.73|forestry +41|395|65553|4294967532|34.11|4.82|false|luke xylophone|2013-03-01 09:11:58.703108|55.05|nap time +97|381|65569|4294967320|3.38|36.29|false|irene ellison|2013-03-01 09:11:58.703128|53.37|debate +14|321|65642|4294967394|59.53|6.55|false|victor white|2013-03-01 09:11:58.703100|58.69|joggying +86|443|65639|4294967490|84.00|42.18|false|ulysses thompson|2013-03-01 09:11:58.703187|76.21|topology +91|434|65572|4294967363|48.04|24.22|false|holly white|2013-03-01 09:11:58.703149|50.33|history +13|319|65660|4294967351|22.93|31.30|false|david thompson|2013-03-01 09:11:58.703300|4.95|xylophone band +-3|324|65773|4294967296|11.07|25.95|true|oscar miller|2013-03-01 09:11:58.703320|56.60|opthamology +34|457|65583|4294967513|68.20|9.33|true|calvin white|2013-03-01 09:11:58.703072|68.40|undecided +29|499|65632|4294967303|17.51|35.93|true|priscilla thompson|2013-03-01 09:11:58.703167|94.51|zync studies +65|471|65626|4294967401|24.59|21.20|true|oscar davidson|2013-03-01 09:11:58.703171|19.91|american history +89|358|65683|4294967427|91.88|47.87|false|mike garcia|2013-03-01 09:11:58.703307|40.33|kindergarten +2|414|65559|4294967403|63.15|47.47|false|katie laertes|2013-03-01 09:11:58.703100|12.60|american history +53|365|65606|4294967478|82.93|39.60|false|mike thompson|2013-03-01 09:11:58.703299|68.91|joggying +54|376|65653|4294967488|99.87|23.17|false|rachel johnson|2013-03-01 09:11:58.703102|62.25|linguistics +95|264|65755|4294967383|14.47|48.15|false|irene xylophone|2013-03-01 09:11:58.703204|48.50|joggying +52|497|65782|4294967546|64.75|28.89|false|irene steinbeck|2013-03-01 09:11:58.703124|18.57|opthamology +52|450|65688|4294967311|91.72|37.38|true|tom zipper|2013-03-01 09:11:58.703239|46.75|religion +65|388|65697|4294967484|92.20|16.30|false|holly miller|2013-03-01 09:11:58.703317|96.58|philosophy +39|320|65767|4294967345|20.00|5.07|true|oscar quirinius|2013-03-01 09:11:58.703087|19.90|quiet hour +78|390|65583|4294967476|45.52|10.36|true|david ellison|2013-03-01 09:11:58.703294|47.62|geology +97|318|65691|4294967477|97.42|33.32|false|katie zipper|2013-03-01 09:11:58.703205|21.44|yard duty +-2|318|65629|4294967481|41.85|26.95|false|nick underhill|2013-03-01 09:11:58.703244|74.38|debate +96|491|65618|4294967431|70.43|35.75|false|mike davidson|2013-03-01 09:11:58.703216|76.89|opthamology +72|368|65554|4294967296|69.39|9.20|true|ethan garcia|2013-03-01 09:11:58.703175|15.67|quiet hour +28|415|65635|4294967417|90.17|45.92|false|bob young|2013-03-01 09:11:58.703121|28.72|opthamology +96|435|65657|4294967434|86.15|16.78|true|mike underhill|2013-03-01 09:11:58.703090|45.29|zync studies +19|359|65720|4294967386|77.99|25.58|false|jessica steinbeck|2013-03-01 09:11:58.703206|60.42|joggying +68|470|65715|4294967411|58.57|1.06|true|oscar garcia|2013-03-01 09:11:58.703106|44.67|american history +-2|384|65604|4294967317|1.65|46.75|true|nick falkner|2013-03-01 09:11:58.703206|59.16|kindergarten +94|394|65620|4294967367|39.75|14.70|false|gabriella van buren|2013-03-01 09:11:58.703320|21.10|opthamology +121|274|65699|4294967338|77.78|19.97|false|tom allen|2013-03-01 09:11:58.703319|75.42|xylophone band +113|504|65575|4294967340|99.55|36.95|true|ethan underhill|2013-03-01 09:11:58.703214|52.45|debate +24|467|65718|4294967395|53.05|0.92|true|fred nixon|2013-03-01 09:11:58.703088|82.49|study skills +124|397|65593|4294967368|4.38|46.35|true|mike nixon|2013-03-01 09:11:58.703117|45.34|kindergarten +27|282|65743|4294967362|74.41|10.45|true|fred falkner|2013-03-01 09:11:58.703234|30.72|quiet hour +42|396|65670|4294967502|0.32|39.24|false|katie underhill|2013-03-01 09:11:58.703277|96.39|undecided +73|494|65609|4294967392|39.30|47.23|true|jessica ellison|2013-03-01 09:11:58.703244|30.37|study skills +12|285|65761|4294967481|82.85|27.40|true|alice polk|2013-03-01 09:11:58.703079|46.05|american history +11|315|65540|4294967506|18.44|9.18|false|jessica hernandez|2013-03-01 09:11:58.703180|82.36|geology +10|418|65584|4294967444|35.99|49.42|false|wendy garcia|2013-03-01 09:11:58.703313|10.98|history +81|506|65666|4294967423|68.01|18.85|false|oscar xylophone|2013-03-01 09:11:58.703219|8.78|linguistics +76|489|65575|4294967392|96.79|49.36|false|quinn garcia|2013-03-01 09:11:58.703135|74.48|undecided +32|363|65782|4294967409|93.64|18.93|true|quinn ovid|2013-03-01 09:11:58.703158|14.42|zync studies +81|404|65555|4294967357|44.85|12.27|false|ulysses nixon|2013-03-01 09:11:58.703106|4.67|zync studies +89|485|65713|4294967325|96.44|17.45|false|quinn underhill|2013-03-01 09:11:58.703106|36.92|mathematics +104|399|65640|4294967402|32.84|49.11|false|priscilla davidson|2013-03-01 09:11:58.703099|57.58|mathematics +68|419|65773|4294967298|20.74|17.81|true|holly nixon|2013-03-01 09:11:58.703219|16.17|zync studies +124|470|65541|4294967495|62.06|44.19|true|oscar nixon|2013-03-01 09:11:58.703237|6.09|industrial engineering +92|289|65562|4294967322|34.78|16.38|true|wendy ovid|2013-03-01 09:11:58.703113|33.71|mathematics +61|272|65581|4294967395|66.79|15.83|false|katie hernandez|2013-03-01 09:11:58.703123|16.86|linguistics +18|308|65560|4294967322|8.66|40.43|true|alice davidson|2013-03-01 09:11:58.703205|39.69|religion +75|398|65659|4294967346|2.96|45.32|false|alice ichabod|2013-03-01 09:11:58.703129|75.04|nap time +97|449|65553|4294967531|9.31|31.05|false|quinn underhill|2013-03-01 09:11:58.703245|98.73|study skills +63|300|65541|4294967466|40.39|1.64|false|priscilla young|2013-03-01 09:11:58.703150|31.46|study skills +124|360|65753|4294967386|9.07|0.15|true|zach steinbeck|2013-03-01 09:11:58.703276|46.84|religion +70|342|65555|4294967506|33.20|28.25|false|sarah quirinius|2013-03-01 09:11:58.703173|18.30|history +54|464|65740|4294967465|3.54|7.07|false|priscilla steinbeck|2013-03-01 09:11:58.703145|58.75|xylophone band +0|283|65724|4294967302|90.38|14.52|false|yuri zipper|2013-03-01 09:11:58.703093|48.07|forestry +1|338|65650|4294967402|55.05|8.86|false|nick davidson|2013-03-01 09:11:58.703234|79.66|history +59|399|65741|4294967511|87.84|48.07|false|alice laertes|2013-03-01 09:11:58.703222|75.62|religion +88|492|65558|4294967532|20.89|39.20|true|mike steinbeck|2013-03-01 09:11:58.703247|76.63|geology +22|308|65616|4294967338|54.31|34.81|false|holly garcia|2013-03-01 09:11:58.703245|65.57|history +104|262|65741|4294967410|89.83|13.99|false|holly garcia|2013-03-01 09:11:58.703104|12.98|quiet hour +11|279|65771|4294967328|26.36|31.25|false|bob xylophone|2013-03-01 09:11:58.703136|66.16|nap time +122|379|65583|4294967412|37.16|46.75|true|david white|2013-03-01 09:11:58.703307|43.87|forestry +9|370|65616|4294967382|2.87|9.02|true|mike carson|2013-03-01 09:11:58.703183|46.98|history +81|434|65627|4294967375|16.37|34.35|false|david carson|2013-03-01 09:11:58.703145|73.59|topology +3|354|65729|4294967414|35.87|13.36|true|victor young|2013-03-01 09:11:58.703152|10.40|undecided +37|275|65654|4294967296|36.23|43.45|false|david zipper|2013-03-01 09:11:58.703184|3.36|undecided +76|498|65669|4294967389|73.25|37.10|true|mike white|2013-03-01 09:11:58.703137|36.54|industrial engineering +69|327|65595|4294967523|23.70|39.20|true|mike ellison|2013-03-01 09:11:58.703288|34.50|quiet hour +83|318|65545|4294967493|19.56|8.08|true|tom brown|2013-03-01 09:11:58.703234|68.07|quiet hour +6|474|65706|4294967491|47.24|2.48|true|wendy van buren|2013-03-01 09:11:58.703079|73.89|zync studies +57|471|65561|4294967478|3.14|31.86|false|calvin polk|2013-03-01 09:11:58.703171|84.64|religion +90|412|65572|4294967339|70.86|27.42|false|calvin johnson|2013-03-01 09:11:58.703132|18.69|forestry +21|439|65649|4294967550|32.43|42.42|true|ulysses johnson|2013-03-01 09:11:58.703161|57.03|debate +56|260|65766|4294967383|26.75|29.63|false|sarah young|2013-03-01 09:11:58.703122|72.76|topology +63|296|65680|4294967404|90.05|41.90|true|victor johnson|2013-03-01 09:11:58.703223|8.33|chemistry +87|336|65730|4294967307|34.27|26.10|false|fred allen|2013-03-01 09:11:58.703137|71.07|undecided +81|491|65645|4294967458|51.57|5.67|true|mike thompson|2013-03-01 09:11:58.703127|43.55|study skills +95|429|65705|4294967339|54.04|20.22|false|ethan xylophone|2013-03-01 09:11:58.703226|58.06|american history +-2|407|65612|4294967318|25.48|41.56|true|david laertes|2013-03-01 09:11:58.703076|40.10|forestry +43|462|65673|4294967546|49.48|40.39|true|nick steinbeck|2013-03-01 09:11:58.703143|82.70|study skills +25|464|65699|4294967342|92.88|21.97|true|holly king|2013-03-01 09:11:58.703166|19.11|quiet hour +34|439|65658|4294967495|33.23|37.92|true|quinn underhill|2013-03-01 09:11:58.703213|37.17|zync studies +76|341|65660|4294967324|78.61|4.24|false|katie robinson|2013-03-01 09:11:58.703138|42.78|values clariffication +100|301|65583|4294967541|3.00|48.85|false|holly brown|2013-03-01 09:11:58.703091|76.31|industrial engineering +105|297|65639|4294967531|11.60|19.06|true|calvin ovid|2013-03-01 09:11:58.703104|3.98|yard duty +7|361|65552|4294967390|20.14|16.01|false|irene polk|2013-03-01 09:11:58.703257|12.77|american history +22|329|65684|4294967541|29.03|37.78|false|calvin van buren|2013-03-01 09:11:58.703166|93.16|geology +51|480|65593|4294967403|55.26|1.34|false|david johnson|2013-03-01 09:11:58.703215|87.07|debate +61|301|65553|4294967510|48.51|28.05|true|luke steinbeck|2013-03-01 09:11:58.703256|42.85|opthamology +23|507|65536|4294967429|17.72|44.07|true|mike white|2013-03-01 09:11:58.703220|63.08|nap time +82|346|65740|4294967353|73.32|23.01|false|sarah miller|2013-03-01 09:11:58.703282|62.12|xylophone band +121|357|65642|4294967296|0.44|38.58|false|victor xylophone|2013-03-01 09:11:58.703082|97.57|opthamology +47|427|65598|4294967446|9.10|11.77|false|david king|2013-03-01 09:11:58.703088|82.71|biology +8|329|65627|4294967321|8.00|15.92|true|fred davidson|2013-03-01 09:11:58.703114|77.57|forestry +71|429|65581|4294967445|54.78|9.40|false|ethan king|2013-03-01 09:11:58.703259|44.10|forestry +21|363|65776|4294967398|94.00|15.33|true|tom underhill|2013-03-01 09:11:58.703277|10.99|biology +92|420|65589|4294967497|87.69|31.60|false|xavier zipper|2013-03-01 09:11:58.703074|48.21|linguistics +-3|339|65737|4294967453|14.23|26.66|true|ethan underhill|2013-03-01 09:11:58.703138|94.91|xylophone band +11|497|65589|4294967305|91.19|33.71|true|wendy thompson|2013-03-01 09:11:58.703272|13.41|joggying +60|283|65699|4294967461|3.90|21.50|false|zach ovid|2013-03-01 09:11:58.703099|71.55|topology +40|429|65541|4294967323|73.66|25.32|false|wendy xylophone|2013-03-01 09:11:58.703245|54.75|nap time +123|337|65782|4294967449|11.35|19.53|false|yuri brown|2013-03-01 09:11:58.703078|14.32|education +74|293|65566|4294967298|0.12|40.67|false|luke falkner|2013-03-01 09:11:58.703241|10.37|biology +91|383|65708|4294967430|66.25|8.04|false|fred carson|2013-03-01 09:11:58.703305|66.01|kindergarten +72|383|65562|4294967518|1.97|30.78|true|nick polk|2013-03-01 09:11:58.703263|24.54|undecided +78|374|65649|4294967496|28.27|45.63|false|calvin thompson|2013-03-01 09:11:58.703249|77.07|mathematics +115|381|65598|4294967348|46.36|46.67|false|ethan brown|2013-03-01 09:11:58.703074|84.91|american history +119|310|65640|4294967301|51.31|36.87|false|ethan white|2013-03-01 09:11:58.703296|65.50|history +9|417|65612|4294967335|34.91|3.11|false|holly xylophone|2013-03-01 09:11:58.703295|95.83|quiet hour +64|432|65612|4294967332|10.34|12.10|false|jessica garcia|2013-03-01 09:11:58.703299|39.52|history +26|317|65570|4294967369|61.07|0.30|true|irene carson|2013-03-01 09:11:58.703111|50.22|nap time +88|483|65648|4294967299|70.68|36.82|true|katie laertes|2013-03-01 09:11:58.703209|91.37|forestry +106|264|65725|4294967458|87.38|35.10|true|ulysses ichabod|2013-03-01 09:11:58.703227|40.60|wind surfing +29|506|65557|4294967309|35.20|6.83|false|ulysses robinson|2013-03-01 09:11:58.703229|84.12|xylophone band +119|442|65608|4294967482|7.39|5.86|false|yuri ellison|2013-03-01 09:11:58.703098|24.49|quiet hour +54|278|65788|4294967440|81.53|9.81|true|rachel ovid|2013-03-01 09:11:58.703234|14.23|undecided +55|265|65536|4294967460|32.62|3.55|true|irene johnson|2013-03-01 09:11:58.703179|30.91|chemistry +47|276|65543|4294967337|0.62|0.33|true|luke brown|2013-03-01 09:11:58.703085|39.95|mathematics +123|475|65706|4294967423|91.95|38.64|false|alice johnson|2013-03-01 09:11:58.703230|63.23|philosophy +105|406|65569|4294967436|68.74|44.10|false|zach laertes|2013-03-01 09:11:58.703074|20.12|philosophy +16|280|65629|4294967330|50.70|26.26|false|quinn davidson|2013-03-01 09:11:58.703273|19.10|nap time +100|428|65661|4294967476|27.44|37.76|true|zach steinbeck|2013-03-01 09:11:58.703242|9.25|undecided +61|448|65650|4294967389|47.00|43.11|true|victor davidson|2013-03-01 09:11:58.703148|9.93|topology +36|328|65684|4294967409|10.88|4.05|true|calvin king|2013-03-01 09:11:58.703218|2.75|chemistry +63|272|65571|4294967462|6.73|27.36|true|holly nixon|2013-03-01 09:11:58.703175|22.19|religion +1|351|65545|4294967396|46.16|46.81|false|nick brown|2013-03-01 09:11:58.703137|21.40|opthamology +105|377|65675|4294967548|32.73|17.79|false|calvin nixon|2013-03-01 09:11:58.703080|17.95|education +100|288|65783|4294967444|60.67|28.14|false|nick zipper|2013-03-01 09:11:58.703124|65.85|forestry +6|279|65642|4294967384|70.90|20.75|true|quinn underhill|2013-03-01 09:11:58.703316|25.62|yard duty +112|265|65729|4294967306|60.46|49.42|false|quinn nixon|2013-03-01 09:11:58.703320|60.87|religion +50|478|65775|4294967319|34.41|2.91|false|calvin davidson|2013-03-01 09:11:58.703139|21.15|religion +116|464|65757|4294967326|96.10|29.56|false|rachel ellison|2013-03-01 09:11:58.703242|44.61|wind surfing +17|340|65721|4294967481|17.62|34.24|false|oscar steinbeck|2013-03-01 09:11:58.703088|46.55|forestry +43|333|65687|4294967458|5.94|28.03|false|luke robinson|2013-03-01 09:11:58.703185|40.65|religion +57|259|65748|4294967394|53.91|49.76|false|alice johnson|2013-03-01 09:11:58.703283|56.27|study skills +100|301|65537|4294967440|22.79|6.28|false|yuri garcia|2013-03-01 09:11:58.703278|80.14|biology +97|507|65633|4294967478|65.60|9.27|false|nick hernandez|2013-03-01 09:11:58.703151|22.08|nap time +36|356|65757|4294967416|47.64|9.31|true|oscar van buren|2013-03-01 09:11:58.703196|17.48|debate +39|467|65590|4294967306|20.30|12.68|true|ulysses king|2013-03-01 09:11:58.703182|64.45|american history +11|372|65633|4294967455|33.58|29.97|false|yuri robinson|2013-03-01 09:11:58.703251|6.02|nap time +59|375|65672|4294967312|46.31|23.73|true|holly davidson|2013-03-01 09:11:58.703185|12.43|wind surfing +115|378|65595|4294967435|84.45|41.10|true|priscilla miller|2013-03-01 09:11:58.703096|70.67|values clariffication +39|457|65599|4294967514|69.02|17.65|false|xavier nixon|2013-03-01 09:11:58.703322|24.96|zync studies +69|490|65601|4294967459|56.21|45.43|false|fred quirinius|2013-03-01 09:11:58.703196|30.79|kindergarten +24|441|65600|4294967388|12.59|47.80|true|quinn ovid|2013-03-01 09:11:58.703127|34.55|opthamology +80|315|65727|4294967399|53.97|0.54|false|gabriella van buren|2013-03-01 09:11:58.703239|90.66|american history +67|477|65693|4294967412|64.32|23.54|true|luke young|2013-03-01 09:11:58.703143|98.67|quiet hour +52|396|65592|4294967299|44.91|25.05|false|calvin nixon|2013-03-01 09:11:58.703325|25.58|yard duty +97|475|65556|4294967315|24.83|5.32|true|rachel nixon|2013-03-01 09:11:58.703174|66.15|religion +-3|346|65752|4294967298|56.05|34.03|false|tom polk|2013-03-01 09:11:58.703217|49.02|zync studies +20|267|65743|4294967327|64.62|4.77|false|ethan nixon|2013-03-01 09:11:58.703296|84.02|kindergarten +0|434|65752|4294967357|96.76|35.05|true|fred davidson|2013-03-01 09:11:58.703199|50.63|chemistry +70|440|65778|4294967395|76.54|25.54|false|calvin carson|2013-03-01 09:11:58.703318|85.61|forestry +91|298|65607|4294967415|38.49|42.10|true|yuri robinson|2013-03-01 09:11:58.703279|46.34|yard duty +-2|288|65753|4294967332|55.17|46.64|true|xavier robinson|2013-03-01 09:11:58.703091|13.32|quiet hour +24|491|65681|4294967434|98.77|32.86|false|luke xylophone|2013-03-01 09:11:58.703124|79.80|opthamology +46|285|65631|4294967505|46.67|32.88|false|victor king|2013-03-01 09:11:58.703189|37.78|biology +8|404|65655|4294967443|13.11|34.72|true|yuri ovid|2013-03-01 09:11:58.703321|59.40|opthamology +49|414|65553|4294967483|97.98|6.88|false|jessica davidson|2013-03-01 09:11:58.703158|42.01|philosophy +49|381|65744|4294967343|36.34|27.46|false|wendy steinbeck|2013-03-01 09:11:58.703117|7.13|kindergarten +72|436|65609|4294967503|90.64|34.51|false|calvin thompson|2013-03-01 09:11:58.703096|41.13|debate +12|269|65721|4294967437|95.68|45.24|true|yuri garcia|2013-03-01 09:11:58.703306|34.23|zync studies +105|492|65754|4294967535|73.26|9.94|true|xavier zipper|2013-03-01 09:11:58.703230|8.65|forestry +67|316|65685|4294967549|19.21|46.98|false|alice laertes|2013-03-01 09:11:58.703306|44.22|joggying +13|461|65785|4294967507|7.21|28.28|true|katie robinson|2013-03-01 09:11:58.703079|76.31|forestry +43|327|65609|4294967414|88.01|3.48|false|zach davidson|2013-03-01 09:11:58.703094|96.45|industrial engineering +89|297|65666|4294967317|12.27|2.32|true|david allen|2013-03-01 09:11:58.703175|68.40|geology +4|419|65774|4294967460|5.53|20.55|true|quinn zipper|2013-03-01 09:11:58.703287|42.67|values clariffication +117|419|65780|4294967458|43.40|15.99|false|david van buren|2013-03-01 09:11:58.703294|21.45|nap time +37|461|65582|4294967331|45.58|31.00|true|mike steinbeck|2013-03-01 09:11:58.703083|75.45|mathematics +10|336|65694|4294967463|45.98|10.54|true|irene underhill|2013-03-01 09:11:58.703180|27.68|nap time +100|377|65559|4294967478|30.12|43.48|false|wendy underhill|2013-03-01 09:11:58.703137|0.34|xylophone band +93|468|65733|4294967499|24.37|19.33|false|ethan polk|2013-03-01 09:11:58.703185|24.40|yard duty +63|444|65622|4294967456|99.00|34.30|true|bob carson|2013-03-01 09:11:58.703281|63.78|xylophone band +88|382|65690|4294967472|19.32|5.20|true|alice falkner|2013-03-01 09:11:58.703159|32.80|xylophone band +65|346|65577|4294967453|67.56|7.88|false|ethan white|2013-03-01 09:11:58.703231|35.55|opthamology +60|265|65540|4294967438|20.58|8.66|true|zach allen|2013-03-01 09:11:58.703142|27.12|quiet hour +17|337|65647|4294967418|22.81|33.85|true|rachel davidson|2013-03-01 09:11:58.703110|58.41|chemistry +6|404|65621|4294967312|45.49|37.47|true|gabriella ellison|2013-03-01 09:11:58.703305|90.19|zync studies +30|485|65697|4294967515|86.18|42.84|false|sarah zipper|2013-03-01 09:11:58.703137|38.19|linguistics +91|287|65555|4294967467|89.22|16.90|false|rachel allen|2013-03-01 09:11:58.703166|82.53|opthamology +67|506|65647|4294967397|69.79|29.67|false|gabriella hernandez|2013-03-01 09:11:58.703245|55.13|nap time +103|390|65786|4294967355|0.02|26.65|false|luke king|2013-03-01 09:11:58.703163|21.15|yard duty +76|391|65573|4294967372|73.64|21.13|false|ethan xylophone|2013-03-01 09:11:58.703248|31.63|wind surfing +92|332|65645|4294967500|52.04|23.55|false|yuri robinson|2013-03-01 09:11:58.703121|28.98|wind surfing +109|283|65739|4294967313|14.33|16.49|true|oscar king|2013-03-01 09:11:58.703240|92.82|forestry +80|258|65656|4294967386|79.17|40.06|false|fred king|2013-03-01 09:11:58.703079|78.62|geology +28|264|65612|4294967370|40.05|1.74|false|luke ichabod|2013-03-01 09:11:58.703163|4.60|history +28|510|65651|4294967448|96.72|6.38|false|irene ellison|2013-03-01 09:11:58.703083|12.61|industrial engineering +-2|456|65733|4294967342|73.06|24.06|true|quinn hernandez|2013-03-01 09:11:58.703279|68.79|joggying +10|336|65602|4294967506|72.25|18.97|false|yuri allen|2013-03-01 09:11:58.703209|78.92|study skills +63|343|65617|4294967532|29.52|40.00|false|sarah xylophone|2013-03-01 09:11:58.703169|96.76|history +119|387|65556|4294967438|13.83|20.65|true|tom zipper|2013-03-01 09:11:58.703098|72.80|nap time +43|269|65631|4294967540|96.16|20.77|true|david xylophone|2013-03-01 09:11:58.703149|71.97|biology +55|398|65599|4294967454|22.55|16.74|true|sarah miller|2013-03-01 09:11:58.703303|26.01|chemistry +42|324|65784|4294967478|95.87|9.75|false|katie miller|2013-03-01 09:11:58.703211|47.23|xylophone band +50|439|65745|4294967360|6.57|28.35|true|nick ovid|2013-03-01 09:11:58.703299|31.03|study skills +7|409|65734|4294967481|47.78|31.04|false|nick allen|2013-03-01 09:11:58.703084|5.19|forestry +44|430|65548|4294967311|76.38|27.88|true|ethan underhill|2013-03-01 09:11:58.703205|31.20|wind surfing +111|285|65553|4294967428|23.50|44.37|true|mike steinbeck|2013-03-01 09:11:58.703269|39.84|religion +79|309|65755|4294967445|35.28|12.43|false|luke miller|2013-03-01 09:11:58.703313|48.26|mathematics +-1|474|65626|4294967383|18.78|29.73|true|victor ichabod|2013-03-01 09:11:58.703268|93.17|religion +112|466|65790|4294967505|44.91|49.38|true|yuri garcia|2013-03-01 09:11:58.703300|55.80|biology +19|354|65619|4294967412|99.18|15.85|false|calvin zipper|2013-03-01 09:11:58.703248|4.22|kindergarten +95|474|65628|4294967314|84.04|23.51|true|zach laertes|2013-03-01 09:11:58.703310|41.19|linguistics +22|478|65693|4294967414|40.30|39.01|true|oscar underhill|2013-03-01 09:11:58.703202|97.72|opthamology +26|497|65671|4294967374|28.40|27.73|false|david johnson|2013-03-01 09:11:58.703174|29.75|religion +27|507|65669|4294967328|65.69|34.00|true|sarah laertes|2013-03-01 09:11:58.703102|56.64|zync studies +82|407|65588|4294967403|61.16|18.97|true|luke nixon|2013-03-01 09:11:58.703203|72.50|industrial engineering +113|406|65704|4294967301|91.45|15.81|false|ulysses underhill|2013-03-01 09:11:58.703227|86.44|kindergarten +0|485|65764|4294967493|77.39|8.18|true|fred van buren|2013-03-01 09:11:58.703246|23.63|history +14|433|65587|4294967416|13.09|9.89|false|sarah laertes|2013-03-01 09:11:58.703204|89.62|values clariffication +0|460|65700|4294967505|71.00|16.90|true|jessica ovid|2013-03-01 09:11:58.703230|55.47|geology +61|318|65782|4294967398|19.38|30.57|true|quinn van buren|2013-03-01 09:11:58.703128|26.74|joggying +58|320|65598|4294967486|90.55|14.70|false|katie ovid|2013-03-01 09:11:58.703150|77.07|kindergarten +123|466|65626|4294967541|88.59|40.51|false|wendy hernandez|2013-03-01 09:11:58.703263|44.30|wind surfing +83|454|65725|4294967480|18.95|49.93|false|gabriella van buren|2013-03-01 09:11:58.703079|38.03|yard duty +61|257|65699|4294967328|15.36|0.14|true|priscilla miller|2013-03-01 09:11:58.703317|1.32|geology +38|401|65700|4294967460|73.69|1.98|true|ethan thompson|2013-03-01 09:11:58.703276|11.11|undecided +103|305|65739|4294967304|30.77|33.92|false|jessica white|2013-03-01 09:11:58.703260|24.48|topology +115|262|65678|4294967316|36.91|16.31|false|xavier hernandez|2013-03-01 09:11:58.703181|56.65|nap time +95|320|65649|4294967322|14.90|0.59|true|quinn steinbeck|2013-03-01 09:11:58.703140|66.80|american history +40|511|65665|4294967470|9.63|33.11|true|priscilla allen|2013-03-01 09:11:58.703140|77.34|history +-3|260|65595|4294967545|59.07|6.75|false|bob falkner|2013-03-01 09:11:58.703280|36.50|chemistry +25|370|65631|4294967359|41.18|38.26|false|gabriella miller|2013-03-01 09:11:58.703129|75.14|linguistics +75|488|65580|4294967473|15.56|48.00|true|sarah young|2013-03-01 09:11:58.703205|1.97|study skills +30|340|65662|4294967404|13.43|49.18|false|wendy garcia|2013-03-01 09:11:58.703106|21.50|biology +50|318|65585|4294967449|74.37|48.37|false|zach nixon|2013-03-01 09:11:58.703290|63.66|quiet hour +66|404|65663|4294967320|60.00|0.04|false|katie brown|2013-03-01 09:11:58.703297|48.78|industrial engineering +29|470|65606|4294967349|30.56|47.76|true|david quirinius|2013-03-01 09:11:58.703075|17.03|undecided +3|476|65776|4294967446|72.79|9.58|false|priscilla hernandez|2013-03-01 09:11:58.703265|64.33|xylophone band +95|368|65617|4294967408|51.60|8.57|false|holly falkner|2013-03-01 09:11:58.703201|9.42|zync studies +13|392|65643|4294967541|92.83|0.68|false|katie laertes|2013-03-01 09:11:58.703220|78.76|american history +124|313|65726|4294967433|21.76|25.44|true|mike zipper|2013-03-01 09:11:58.703161|20.44|linguistics +109|456|65587|4294967536|97.95|44.06|true|oscar nixon|2013-03-01 09:11:58.703306|27.35|education +76|329|65719|4294967394|30.12|28.13|true|nick hernandez|2013-03-01 09:11:58.703101|68.30|topology +76|326|65620|4294967426|88.42|18.86|true|jessica johnson|2013-03-01 09:11:58.703164|47.51|study skills +118|342|65703|4294967307|80.25|14.94|true|bob ichabod|2013-03-01 09:11:58.703283|64.79|topology +44|487|65711|4294967403|30.96|20.36|true|ethan young|2013-03-01 09:11:58.703192|56.32|linguistics +121|324|65625|4294967520|32.85|43.99|false|irene garcia|2013-03-01 09:11:58.703108|8.30|geology +76|340|65609|4294967482|53.84|49.87|true|yuri thompson|2013-03-01 09:11:58.703110|49.75|topology +21|321|65782|4294967539|47.92|41.91|true|zach underhill|2013-03-01 09:11:58.703304|18.14|study skills +122|480|65620|4294967508|88.78|43.68|false|nick brown|2013-03-01 09:11:58.703241|20.14|debate +85|285|65698|4294967328|69.90|13.56|true|priscilla miller|2013-03-01 09:11:58.703108|74.09|nap time +37|292|65689|4294967412|31.55|37.19|true|holly steinbeck|2013-03-01 09:11:58.703147|59.89|joggying +97|342|65615|4294967497|98.40|3.03|false|zach xylophone|2013-03-01 09:11:58.703285|32.47|forestry +48|337|65729|4294967470|39.83|16.53|false|irene young|2013-03-01 09:11:58.703286|87.91|linguistics +105|275|65744|4294967297|7.64|14.75|true|rachel young|2013-03-01 09:11:58.703246|44.75|education +79|410|65566|4294967357|6.66|33.49|false|yuri ovid|2013-03-01 09:11:58.703195|0.67|philosophy +118|398|65552|4294967478|20.71|4.23|false|fred allen|2013-03-01 09:11:58.703308|5.50|history +62|374|65665|4294967314|38.69|0.31|false|katie polk|2013-03-01 09:11:58.703229|11.83|philosophy +123|292|65707|4294967430|9.03|41.31|false|zach thompson|2013-03-01 09:11:58.703249|5.13|opthamology +1|397|65678|4294967437|31.39|38.67|true|david nixon|2013-03-01 09:11:58.703122|3.58|forestry +30|506|65738|4294967475|74.01|33.53|false|oscar nixon|2013-03-01 09:11:58.703132|57.55|linguistics +31|310|65790|4294967481|68.42|32.30|true|ethan king|2013-03-01 09:11:58.703261|68.81|biology +98|432|65715|4294967318|76.12|25.16|true|priscilla robinson|2013-03-01 09:11:58.703111|14.81|yard duty +120|297|65542|4294967409|78.94|24.87|false|victor ichabod|2013-03-01 09:11:58.703100|75.99|american history +109|511|65755|4294967535|6.76|6.46|false|calvin garcia|2013-03-01 09:11:58.703202|69.85|education +54|446|65545|4294967415|65.26|14.26|false|rachel polk|2013-03-01 09:11:58.703108|38.35|debate +74|320|65753|4294967526|50.46|30.05|true|fred xylophone|2013-03-01 09:11:58.703102|83.60|chemistry +78|412|65640|4294967367|63.91|20.82|true|calvin thompson|2013-03-01 09:11:58.703076|15.55|quiet hour +118|289|65778|4294967534|58.66|34.71|false|priscilla van buren|2013-03-01 09:11:58.703237|55.77|american history +25|269|65594|4294967510|25.21|1.29|true|gabriella hernandez|2013-03-01 09:11:58.703319|19.58|kindergarten +108|348|65620|4294967358|52.42|23.49|false|ulysses xylophone|2013-03-01 09:11:58.703194|87.63|quiet hour +63|360|65604|4294967343|91.18|40.58|false|zach brown|2013-03-01 09:11:58.703123|36.76|debate +98|373|65689|4294967342|88.78|31.09|false|yuri xylophone|2013-03-01 09:11:58.703098|61.52|biology +114|268|65642|4294967462|17.11|32.33|false|tom johnson|2013-03-01 09:11:58.703222|15.09|study skills +59|422|65590|4294967426|65.23|46.12|true|bob thompson|2013-03-01 09:11:58.703314|85.01|joggying +93|320|65627|4294967350|20.65|4.10|false|luke laertes|2013-03-01 09:11:58.703175|0.23|opthamology +62|258|65565|4294967391|98.19|26.40|false|katie allen|2013-03-01 09:11:58.703319|17.64|linguistics +65|294|65547|4294967440|51.90|5.79|false|priscilla underhill|2013-03-01 09:11:58.703294|34.32|values clariffication +60|434|65735|4294967339|94.52|10.59|false|xavier laertes|2013-03-01 09:11:58.703124|14.91|education +54|347|65537|4294967414|59.62|9.60|false|xavier underhill|2013-03-01 09:11:58.703261|59.70|industrial engineering +18|439|65634|4294967411|40.45|36.25|true|xavier van buren|2013-03-01 09:11:58.703235|88.10|linguistics +98|367|65596|4294967326|82.60|16.72|false|victor davidson|2013-03-01 09:11:58.703109|94.30|quiet hour +84|321|65744|4294967318|59.33|23.99|true|alice polk|2013-03-01 09:11:58.703109|89.28|biology +60|396|65767|4294967460|20.28|30.13|false|priscilla garcia|2013-03-01 09:11:58.703133|67.08|industrial engineering +70|434|65672|4294967323|38.35|28.46|false|irene carson|2013-03-01 09:11:58.703143|88.01|forestry +98|458|65716|4294967492|34.65|34.08|false|gabriella miller|2013-03-01 09:11:58.703270|20.32|quiet hour +1|484|65627|4294967514|95.46|40.98|true|ulysses carson|2013-03-01 09:11:58.703245|10.99|study skills +47|417|65550|4294967495|4.07|19.40|true|yuri ellison|2013-03-01 09:11:58.703288|0.81|forestry +43|507|65647|4294967410|50.21|4.10|false|mike ovid|2013-03-01 09:11:58.703216|66.95|forestry +67|431|65538|4294967486|91.22|9.44|false|sarah ichabod|2013-03-01 09:11:58.703112|74.48|study skills +8|292|65682|4294967310|76.81|16.66|true|xavier davidson|2013-03-01 09:11:58.703213|77.44|study skills +21|451|65776|4294967317|0.27|24.24|true|irene miller|2013-03-01 09:11:58.703092|7.67|geology +1|267|65646|4294967525|54.17|17.82|true|bob laertes|2013-03-01 09:11:58.703152|78.93|philosophy +4|366|65716|4294967547|83.04|20.71|false|luke brown|2013-03-01 09:11:58.703223|64.65|nap time +77|444|65720|4294967466|85.94|41.30|false|katie miller|2013-03-01 09:11:58.703096|53.72|philosophy +70|491|65575|4294967340|10.87|13.69|true|ethan zipper|2013-03-01 09:11:58.703246|73.38|philosophy +114|404|65763|4294967466|43.79|25.37|true|holly ovid|2013-03-01 09:11:58.703098|85.87|nap time +74|284|65625|4294967368|57.50|17.23|false|oscar steinbeck|2013-03-01 09:11:58.703175|11.70|forestry +31|382|65656|4294967533|89.27|0.95|true|jessica robinson|2013-03-01 09:11:58.703311|24.08|linguistics +78|410|65553|4294967337|38.65|47.21|false|luke ovid|2013-03-01 09:11:58.703308|62.16|topology +41|392|65627|4294967360|53.26|9.21|true|xavier quirinius|2013-03-01 09:11:58.703303|65.72|kindergarten +47|385|65663|4294967421|26.44|44.82|false|david carson|2013-03-01 09:11:58.703110|77.72|quiet hour +106|491|65789|4294967411|98.20|21.13|false|nick underhill|2013-03-01 09:11:58.703320|55.97|american history +-1|365|65601|4294967380|0.95|16.92|true|calvin brown|2013-03-01 09:11:58.703188|90.46|education +73|332|65669|4294967508|55.69|44.95|true|yuri allen|2013-03-01 09:11:58.703153|22.01|opthamology +107|439|65557|4294967400|17.47|42.04|false|zach quirinius|2013-03-01 09:11:58.703297|82.87|opthamology +100|416|65675|4294967402|21.66|12.60|false|mike white|2013-03-01 09:11:58.703136|35.43|mathematics +29|313|65726|4294967388|55.82|11.34|true|ethan thompson|2013-03-01 09:11:58.703281|25.24|quiet hour +18|385|65581|4294967439|2.80|34.48|true|tom thompson|2013-03-01 09:11:58.703077|16.15|philosophy +22|337|65752|4294967463|78.12|11.44|true|jessica davidson|2013-03-01 09:11:58.703266|63.78|industrial engineering +84|284|65674|4294967304|41.21|1.24|true|luke davidson|2013-03-01 09:11:58.703204|57.87|values clariffication +77|455|65625|4294967388|52.96|10.62|false|jessica brown|2013-03-01 09:11:58.703077|21.79|quiet hour +74|492|65774|4294967319|99.33|43.72|true|zach carson|2013-03-01 09:11:58.703185|68.95|philosophy +103|311|65719|4294967484|7.79|31.90|true|nick white|2013-03-01 09:11:58.703204|50.98|biology +91|385|65604|4294967481|64.40|36.35|false|irene thompson|2013-03-01 09:11:58.703208|83.27|zync studies +106|361|65698|4294967341|46.47|35.70|false|ethan falkner|2013-03-01 09:11:58.703281|41.58|xylophone band +119|338|65621|4294967499|30.53|45.61|true|gabriella quirinius|2013-03-01 09:11:58.703278|2.97|forestry +-1|401|65613|4294967301|16.80|40.84|false|calvin carson|2013-03-01 09:11:58.703200|68.39|forestry +23|338|65737|4294967462|37.26|34.13|false|xavier miller|2013-03-01 09:11:58.703189|89.73|topology +47|487|65606|4294967471|31.43|36.39|false|holly johnson|2013-03-01 09:11:58.703218|15.19|industrial engineering +52|355|65711|4294967445|86.19|31.46|true|wendy ovid|2013-03-01 09:11:58.703079|22.03|quiet hour +104|360|65752|4294967392|37.35|27.46|false|nick davidson|2013-03-01 09:11:58.703171|82.28|nap time +62|339|65710|4294967429|21.88|25.80|true|irene nixon|2013-03-01 09:11:58.703315|84.40|opthamology +29|397|65741|4294967325|91.57|15.73|false|nick quirinius|2013-03-01 09:11:58.703281|40.66|chemistry +-2|495|65725|4294967363|49.42|16.24|false|mike allen|2013-03-01 09:11:58.703115|49.50|mathematics +7|490|65633|4294967467|35.89|40.83|true|luke thompson|2013-03-01 09:11:58.703258|52.80|linguistics +90|270|65694|4294967515|69.79|42.06|false|holly quirinius|2013-03-01 09:11:58.703251|55.70|values clariffication +50|355|65561|4294967436|70.05|0.75|false|rachel miller|2013-03-01 09:11:58.703316|32.41|geology +115|287|65773|4294967498|43.67|25.85|false|katie laertes|2013-03-01 09:11:58.703161|71.79|linguistics +113|348|65770|4294967407|1.54|28.87|false|bob xylophone|2013-03-01 09:11:58.703304|65.42|philosophy +122|415|65780|4294967326|47.18|28.29|false|david hernandez|2013-03-01 09:11:58.703211|75.92|nap time +62|449|65560|4294967326|0.43|31.97|false|yuri van buren|2013-03-01 09:11:58.703246|61.08|yard duty +49|345|65782|4294967539|56.25|16.98|false|ulysses brown|2013-03-01 09:11:58.703173|58.50|zync studies +38|309|65748|4294967373|10.23|37.12|false|victor miller|2013-03-01 09:11:58.703146|45.36|yard duty +70|453|65759|4294967387|92.60|47.39|true|irene falkner|2013-03-01 09:11:58.703193|88.10|joggying +13|465|65655|4294967468|60.04|6.65|true|yuri xylophone|2013-03-01 09:11:58.703083|49.38|chemistry +-2|266|65769|4294967351|81.79|42.69|true|zach quirinius|2013-03-01 09:11:58.703225|55.76|philosophy +89|506|65766|4294967317|89.04|8.31|true|rachel quirinius|2013-03-01 09:11:58.703191|82.53|education +-2|361|65714|4294967383|13.24|39.48|true|xavier quirinius|2013-03-01 09:11:58.703093|36.18|kindergarten +3|445|65631|4294967503|32.81|23.21|false|fred garcia|2013-03-01 09:11:58.703141|14.33|geology +48|390|65544|4294967530|53.96|39.24|true|gabriella johnson|2013-03-01 09:11:58.703152|74.25|kindergarten +19|447|65680|4294967450|30.47|36.66|true|holly nixon|2013-03-01 09:11:58.703072|19.79|american history +34|272|65644|4294967365|50.42|20.34|true|quinn davidson|2013-03-01 09:11:58.703157|22.47|wind surfing +28|398|65663|4294967437|93.57|36.84|true|alice robinson|2013-03-01 09:11:58.703263|71.00|undecided +70|433|65545|4294967521|58.77|22.22|true|david robinson|2013-03-01 09:11:58.703168|18.31|education +17|363|65630|4294967449|31.20|42.93|true|quinn xylophone|2013-03-01 09:11:58.703088|82.29|mathematics +6|478|65700|4294967443|70.30|12.98|true|bob ichabod|2013-03-01 09:11:58.703232|1.37|debate +40|499|65776|4294967339|51.18|48.74|true|ulysses ichabod|2013-03-01 09:11:58.703273|38.04|study skills +85|297|65672|4294967359|47.92|42.74|true|mike brown|2013-03-01 09:11:58.703192|27.76|quiet hour +100|305|65725|4294967387|54.94|49.01|false|tom white|2013-03-01 09:11:58.703277|50.50|philosophy +71|309|65617|4294967388|60.69|19.10|true|ulysses zipper|2013-03-01 09:11:58.703227|43.32|quiet hour +-2|394|65714|4294967507|12.27|13.91|false|holly thompson|2013-03-01 09:11:58.703094|10.30|topology +7|278|65745|4294967325|95.42|16.52|true|alice king|2013-03-01 09:11:58.703319|28.76|yard duty +2|465|65651|4294967524|44.01|43.33|false|mike ichabod|2013-03-01 09:11:58.703175|4.89|history +49|265|65575|4294967430|39.63|49.78|true|sarah steinbeck|2013-03-01 09:11:58.703124|27.96|american history +61|346|65742|4294967311|53.01|14.99|false|oscar nixon|2013-03-01 09:11:58.703321|88.61|religion +19|489|65620|4294967324|65.95|1.66|false|nick falkner|2013-03-01 09:11:58.703294|15.30|nap time +13|395|65660|4294967499|29.21|40.66|false|luke ellison|2013-03-01 09:11:58.703112|94.74|industrial engineering +67|368|65667|4294967466|6.82|47.36|false|rachel steinbeck|2013-03-01 09:11:58.703260|50.57|joggying +12|390|65636|4294967384|10.81|12.12|true|irene polk|2013-03-01 09:11:58.703300|82.12|philosophy +35|421|65614|4294967351|88.28|13.10|true|holly white|2013-03-01 09:11:58.703297|7.76|nap time +47|489|65582|4294967319|12.66|2.59|true|alice underhill|2013-03-01 09:11:58.703321|9.02|study skills +31|278|65554|4294967333|93.66|42.58|true|jessica johnson|2013-03-01 09:11:58.703122|50.66|yard duty +57|387|65578|4294967333|33.13|41.29|true|alice ovid|2013-03-01 09:11:58.703243|23.35|history +74|381|65681|4294967378|52.14|24.37|false|priscilla van buren|2013-03-01 09:11:58.703256|60.02|religion +31|445|65732|4294967483|23.15|47.45|false|jessica garcia|2013-03-01 09:11:58.703118|51.88|biology +95|391|65718|4294967327|92.68|4.20|true|victor miller|2013-03-01 09:11:58.703075|96.74|opthamology +111|329|65577|4294967377|45.55|46.03|false|wendy steinbeck|2013-03-01 09:11:58.703220|61.18|debate +89|477|65655|4294967389|71.74|19.65|false|yuri garcia|2013-03-01 09:11:58.703124|11.71|wind surfing +109|507|65734|4294967441|68.64|40.74|false|fred king|2013-03-01 09:11:58.703219|50.73|history +81|405|65631|4294967367|72.42|49.99|true|jessica carson|2013-03-01 09:11:58.703288|86.26|industrial engineering +10|339|65710|4294967468|26.16|40.07|false|david ellison|2013-03-01 09:11:58.703122|22.07|topology +80|310|65621|4294967304|84.61|21.87|true|tom robinson|2013-03-01 09:11:58.703262|6.83|industrial engineering +100|338|65764|4294967529|35.57|19.92|false|wendy ellison|2013-03-01 09:11:58.703322|49.43|chemistry +82|295|65727|4294967449|8.42|1.36|false|katie thompson|2013-03-01 09:11:58.703141|44.75|kindergarten +27|342|65746|4294967325|98.33|10.80|false|holly robinson|2013-03-01 09:11:58.703212|54.67|education +22|278|65547|4294967396|78.82|34.82|false|irene ovid|2013-03-01 09:11:58.703130|25.56|yard duty +100|419|65597|4294967302|9.45|19.74|true|calvin robinson|2013-03-01 09:11:58.703290|64.35|geology +21|338|65582|4294967416|86.27|25.62|false|rachel polk|2013-03-01 09:11:58.703183|33.03|quiet hour +93|268|65540|4294967462|79.12|36.10|true|zach allen|2013-03-01 09:11:58.703102|98.11|biology +113|363|65673|4294967536|67.03|49.82|true|luke van buren|2013-03-01 09:11:58.703199|0.47|topology +58|442|65576|4294967337|16.51|26.71|true|david king|2013-03-01 09:11:58.703252|6.21|biology +76|393|65553|4294967517|57.65|22.91|true|zach garcia|2013-03-01 09:11:58.703312|96.55|biology +94|400|65644|4294967342|45.25|49.34|false|katie thompson|2013-03-01 09:11:58.703236|39.22|philosophy +23|503|65624|4294967516|50.91|30.24|true|fred van buren|2013-03-01 09:11:58.703287|13.72|opthamology +39|502|65778|4294967304|10.87|4.07|true|bob hernandez|2013-03-01 09:11:58.703143|95.21|topology +51|300|65690|4294967420|54.99|27.08|true|david laertes|2013-03-01 09:11:58.703233|15.90|mathematics +112|507|65577|4294967525|3.94|0.19|false|gabriella davidson|2013-03-01 09:11:58.703279|48.81|topology +41|341|65667|4294967449|43.93|49.18|false|rachel hernandez|2013-03-01 09:11:58.703131|96.81|forestry +79|428|65581|4294967486|56.99|40.93|true|zach king|2013-03-01 09:11:58.703291|22.34|forestry +59|483|65738|4294967302|95.95|2.30|true|zach ovid|2013-03-01 09:11:58.703077|42.33|chemistry +88|356|65721|4294967450|80.55|21.12|false|bob carson|2013-03-01 09:11:58.703133|12.39|education +34|276|65661|4294967539|46.28|18.62|false|calvin allen|2013-03-01 09:11:58.703133|79.16|industrial engineering +13|387|65675|4294967456|81.82|41.68|false|quinn miller|2013-03-01 09:11:58.703099|34.49|philosophy +35|376|65765|4294967300|33.20|29.83|false|luke johnson|2013-03-01 09:11:58.703196|27.36|chemistry +25|363|65790|4294967413|60.23|22.00|false|zach thompson|2013-03-01 09:11:58.703178|95.45|biology +80|461|65741|4294967368|89.92|10.51|false|nick zipper|2013-03-01 09:11:58.703280|57.29|education +19|283|65612|4294967425|96.46|34.59|false|katie davidson|2013-03-01 09:11:58.703147|46.24|values clariffication +101|288|65760|4294967519|30.70|41.74|true|fred polk|2013-03-01 09:11:58.703203|80.51|joggying +10|481|65728|4294967492|45.65|47.86|false|quinn van buren|2013-03-01 09:11:58.703180|8.59|nap time +27|306|65700|4294967465|97.41|32.95|false|rachel davidson|2013-03-01 09:11:58.703177|80.76|zync studies +38|422|65739|4294967384|11.78|40.56|false|gabriella robinson|2013-03-01 09:11:58.703284|5.88|industrial engineering +14|368|65653|4294967317|72.09|20.90|true|mike carson|2013-03-01 09:11:58.703228|13.97|religion +81|334|65698|4294967535|17.99|47.20|true|luke garcia|2013-03-01 09:11:58.703235|62.68|geology +-2|380|65707|4294967526|22.68|36.95|false|ethan allen|2013-03-01 09:11:58.703107|76.64|chemistry +21|338|65714|4294967361|97.74|5.99|true|nick garcia|2013-03-01 09:11:58.703188|68.57|opthamology +52|352|65748|4294967476|32.21|49.09|false|priscilla hernandez|2013-03-01 09:11:58.703247|58.40|industrial engineering +70|335|65750|4294967473|13.16|10.86|true|oscar ellison|2013-03-01 09:11:58.703267|29.41|zync studies +39|443|65706|4294967514|67.18|41.95|true|calvin hernandez|2013-03-01 09:11:58.703298|58.83|study skills +119|440|65742|4294967383|60.84|15.76|false|priscilla underhill|2013-03-01 09:11:58.703212|54.26|undecided +75|294|65581|4294967451|67.50|47.56|true|zach robinson|2013-03-01 09:11:58.703321|58.67|biology +121|287|65557|4294967336|50.50|3.85|true|luke thompson|2013-03-01 09:11:58.703289|47.30|industrial engineering +102|491|65744|4294967423|89.80|30.72|true|wendy hernandez|2013-03-01 09:11:58.703229|93.74|topology +20|407|65696|4294967529|39.58|15.16|false|luke zipper|2013-03-01 09:11:58.703153|93.13|education +96|450|65669|4294967311|6.22|16.96|false|david nixon|2013-03-01 09:11:58.703279|83.61|education +62|444|65708|4294967409|76.00|25.16|true|rachel zipper|2013-03-01 09:11:58.703115|38.50|quiet hour +43|466|65731|4294967441|53.85|26.51|true|alice xylophone|2013-03-01 09:11:58.703149|18.88|values clariffication +70|410|65569|4294967535|42.24|31.95|true|gabriella allen|2013-03-01 09:11:58.703301|86.34|topology +44|389|65682|4294967475|44.31|45.78|false|calvin carson|2013-03-01 09:11:58.703301|8.76|chemistry +33|379|65684|4294967487|7.18|43.01|true|katie zipper|2013-03-01 09:11:58.703098|26.62|study skills +104|258|65734|4294967506|79.48|28.35|true|yuri johnson|2013-03-01 09:11:58.703116|35.73|philosophy +117|501|65607|4294967521|76.81|2.25|false|david xylophone|2013-03-01 09:11:58.703224|34.72|forestry +74|504|65681|4294967529|77.71|19.37|true|irene brown|2013-03-01 09:11:58.703269|56.07|american history +115|259|65770|4294967352|91.66|15.43|true|rachel brown|2013-03-01 09:11:58.703171|70.78|forestry +100|405|65677|4294967347|94.92|42.33|false|zach young|2013-03-01 09:11:58.703132|23.53|linguistics +53|273|65612|4294967350|47.57|16.47|false|mike young|2013-03-01 09:11:58.703290|13.23|opthamology +78|472|65614|4294967333|27.15|23.66|true|calvin johnson|2013-03-01 09:11:58.703249|99.55|kindergarten +13|502|65609|4294967311|16.67|18.70|false|holly johnson|2013-03-01 09:11:58.703184|56.16|quiet hour +23|362|65683|4294967411|4.95|9.54|false|nick robinson|2013-03-01 09:11:58.703145|45.29|chemistry +114|407|65742|4294967356|18.00|13.99|false|holly falkner|2013-03-01 09:11:58.703226|93.90|undecided +50|291|65749|4294967498|36.02|0.64|false|sarah allen|2013-03-01 09:11:58.703102|70.84|philosophy +62|461|65735|4294967436|31.45|18.61|false|katie steinbeck|2013-03-01 09:11:58.703208|6.21|zync studies +78|264|65562|4294967545|19.77|43.76|true|oscar garcia|2013-03-01 09:11:58.703282|42.66|philosophy +108|392|65588|4294967322|64.52|17.87|true|mike ichabod|2013-03-01 09:11:58.703095|44.94|forestry +46|345|65667|4294967433|47.48|24.68|true|zach zipper|2013-03-01 09:11:58.703194|23.76|biology +56|423|65713|4294967303|69.82|31.98|true|victor underhill|2013-03-01 09:11:58.703091|6.89|history +121|354|65546|4294967548|12.88|4.54|false|ethan young|2013-03-01 09:11:58.703220|41.11|biology +48|322|65536|4294967317|70.79|13.13|false|rachel xylophone|2013-03-01 09:11:58.703178|58.80|study skills +28|374|65651|4294967389|91.36|31.77|false|nick carson|2013-03-01 09:11:58.703070|41.63|debate +60|269|65577|4294967317|84.87|38.36|true|bob quirinius|2013-03-01 09:11:58.703143|57.97|undecided +37|299|65650|4294967352|1.14|43.62|false|holly quirinius|2013-03-01 09:11:58.703241|11.77|joggying +15|411|65775|4294967417|37.16|35.02|false|yuri ichabod|2013-03-01 09:11:58.703141|86.90|xylophone band +73|331|65650|4294967461|36.96|45.85|false|wendy brown|2013-03-01 09:11:58.703231|96.62|kindergarten +20|496|65622|4294967380|32.94|14.45|false|ulysses ellison|2013-03-01 09:11:58.703311|91.87|philosophy +58|370|65624|4294967500|65.46|25.35|true|mike carson|2013-03-01 09:11:58.703177|61.64|topology +5|356|65561|4294967485|4.73|45.86|true|bob underhill|2013-03-01 09:11:58.703251|83.79|philosophy +22|297|65789|4294967536|92.53|9.14|true|wendy johnson|2013-03-01 09:11:58.703199|94.89|religion +85|363|65739|4294967449|69.07|26.17|true|luke miller|2013-03-01 09:11:58.703101|25.06|xylophone band +66|432|65545|4294967547|55.46|34.28|false|xavier ovid|2013-03-01 09:11:58.703204|48.49|forestry +71|268|65647|4294967483|18.90|41.67|false|wendy carson|2013-03-01 09:11:58.703279|88.38|wind surfing +124|287|65787|4294967416|57.16|35.79|true|yuri laertes|2013-03-01 09:11:58.703166|66.62|study skills +74|504|65601|4294967424|40.93|25.84|false|zach hernandez|2013-03-01 09:11:58.703073|32.37|american history +26|396|65599|4294967523|46.42|45.35|true|luke ellison|2013-03-01 09:11:58.703080|41.96|forestry +49|314|65596|4294967331|81.73|42.32|true|katie xylophone|2013-03-01 09:11:58.703271|95.15|yard duty +40|298|65576|4294967404|56.15|20.64|true|calvin thompson|2013-03-01 09:11:58.703314|84.81|linguistics +38|310|65765|4294967435|93.37|14.36|false|yuri white|2013-03-01 09:11:58.703121|59.43|american history +5|488|65698|4294967414|41.98|44.84|true|oscar ichabod|2013-03-01 09:11:58.703143|77.40|opthamology +18|385|65695|4294967404|67.95|31.50|false|tom brown|2013-03-01 09:11:58.703228|92.86|religion +113|285|65647|4294967415|15.58|47.59|false|victor thompson|2013-03-01 09:11:58.703105|22.03|industrial engineering +58|360|65721|4294967391|20.71|41.01|true|luke white|2013-03-01 09:11:58.703091|13.43|undecided +74|282|65681|4294967358|40.81|2.73|false|sarah white|2013-03-01 09:11:58.703271|73.89|chemistry +63|406|65549|4294967549|4.07|25.00|true|fred brown|2013-03-01 09:11:58.703182|73.68|study skills +42|290|65692|4294967349|56.99|4.13|true|david garcia|2013-03-01 09:11:58.703214|78.84|american history +2|326|65759|4294967491|28.86|25.68|true|wendy laertes|2013-03-01 09:11:58.703148|97.49|history +25|331|65590|4294967482|10.75|9.41|true|gabriella robinson|2013-03-01 09:11:58.703262|12.92|opthamology +46|423|65629|4294967376|35.34|8.77|true|bob nixon|2013-03-01 09:11:58.703151|40.39|undecided +77|281|65589|4294967415|18.02|21.68|false|holly johnson|2013-03-01 09:11:58.703308|43.10|mathematics +120|471|65733|4294967405|94.48|1.09|false|alice brown|2013-03-01 09:11:58.703146|78.85|education +11|363|65618|4294967379|38.66|2.46|false|xavier davidson|2013-03-01 09:11:58.703267|99.27|xylophone band +87|441|65773|4294967347|30.00|0.27|true|victor steinbeck|2013-03-01 09:11:58.703305|64.30|undecided +63|314|65620|4294967429|29.85|23.34|false|mike young|2013-03-01 09:11:58.703077|64.00|education +91|345|65666|4294967509|33.07|34.84|true|fred brown|2013-03-01 09:11:58.703172|6.10|debate +84|305|65600|4294967370|49.42|21.54|true|mike hernandez|2013-03-01 09:11:58.703238|50.50|zync studies +65|297|65754|4294967431|95.62|49.40|true|zach van buren|2013-03-01 09:11:58.703183|71.88|philosophy +107|434|65743|4294967504|7.32|23.48|false|yuri king|2013-03-01 09:11:58.703192|49.91|industrial engineering +5|377|65774|4294967424|13.32|3.95|true|alice nixon|2013-03-01 09:11:58.703091|15.70|history +52|276|65562|4294967316|12.42|41.92|true|mike falkner|2013-03-01 09:11:58.703239|86.54|yard duty +68|453|65634|4294967309|6.94|7.80|true|rachel carson|2013-03-01 09:11:58.703241|62.51|linguistics +55|306|65562|4294967473|32.62|36.08|true|gabriella ichabod|2013-03-01 09:11:58.703203|23.90|topology +66|359|65584|4294967433|79.73|6.64|true|sarah underhill|2013-03-01 09:11:58.703143|93.05|industrial engineering +103|310|65547|4294967546|78.86|49.82|true|tom ichabod|2013-03-01 09:11:58.703202|15.99|linguistics +36|511|65764|4294967333|70.02|7.54|true|priscilla allen|2013-03-01 09:11:58.703091|85.36|xylophone band +83|407|65698|4294967340|38.31|10.96|false|rachel ichabod|2013-03-01 09:11:58.703190|96.05|education +11|380|65746|4294967394|70.63|8.92|false|calvin johnson|2013-03-01 09:11:58.703300|78.24|values clariffication +1|434|65577|4294967306|18.26|35.14|true|wendy xylophone|2013-03-01 09:11:58.703314|40.39|debate +92|331|65564|4294967450|72.18|3.11|false|bob ovid|2013-03-01 09:11:58.703151|27.92|joggying +25|310|65677|4294967519|20.69|29.94|false|mike robinson|2013-03-01 09:11:58.703220|36.43|chemistry +73|375|65701|4294967542|43.64|35.98|true|ulysses quirinius|2013-03-01 09:11:58.703287|31.93|values clariffication +37|446|65647|4294967451|47.59|39.56|true|nick brown|2013-03-01 09:11:58.703258|23.18|quiet hour +104|408|65675|4294967502|92.30|42.76|true|holly young|2013-03-01 09:11:58.703205|94.14|mathematics +71|344|65571|4294967427|39.20|7.13|false|oscar xylophone|2013-03-01 09:11:58.703197|57.83|mathematics +84|422|65774|4294967500|45.82|30.71|false|gabriella ellison|2013-03-01 09:11:58.703164|1.27|linguistics +70|408|65707|4294967545|71.20|10.74|false|alice davidson|2013-03-01 09:11:58.703145|94.68|xylophone band +39|371|65573|4294967527|71.27|48.03|false|holly zipper|2013-03-01 09:11:58.703152|74.64|kindergarten +18|263|65609|4294967474|82.80|0.27|true|quinn van buren|2013-03-01 09:11:58.703191|42.13|philosophy +2|352|65720|4294967481|90.51|14.52|false|irene thompson|2013-03-01 09:11:58.703172|51.48|zync studies +34|299|65662|4294967551|75.13|25.62|true|calvin quirinius|2013-03-01 09:11:58.703204|89.60|xylophone band +121|348|65721|4294967512|67.05|42.60|true|nick xylophone|2013-03-01 09:11:58.703165|37.01|linguistics +6|315|65746|4294967507|9.25|2.36|true|holly van buren|2013-03-01 09:11:58.703103|69.36|forestry +66|429|65591|4294967362|27.98|47.66|false|bob laertes|2013-03-01 09:11:58.703124|52.96|values clariffication +103|486|65721|4294967457|44.04|8.41|true|jessica xylophone|2013-03-01 09:11:58.703158|38.61|debate +97|313|65715|4294967396|43.91|34.20|true|oscar steinbeck|2013-03-01 09:11:58.703301|88.69|nap time +8|488|65687|4294967396|88.03|45.56|true|nick zipper|2013-03-01 09:11:58.703162|33.38|undecided +92|273|65699|4294967507|22.03|49.22|true|luke davidson|2013-03-01 09:11:58.703233|39.42|geology +108|310|65718|4294967500|94.06|31.39|false|quinn steinbeck|2013-03-01 09:11:58.703087|60.45|xylophone band +9|265|65569|4294967400|70.70|15.77|false|tom steinbeck|2013-03-01 09:11:58.703223|90.33|philosophy +33|421|65673|4294967550|2.79|11.56|true|wendy garcia|2013-03-01 09:11:58.703236|53.24|education +5|449|65778|4294967371|69.68|49.20|false|holly nixon|2013-03-01 09:11:58.703219|77.16|undecided +61|396|65672|4294967298|40.13|49.07|true|ethan miller|2013-03-01 09:11:58.703242|25.99|religion +63|322|65567|4294967324|27.41|14.60|false|nick ovid|2013-03-01 09:11:58.703239|22.30|debate +29|465|65598|4294967501|94.29|5.29|true|jessica steinbeck|2013-03-01 09:11:58.703318|14.14|joggying +37|454|65617|4294967457|25.91|7.52|false|ethan johnson|2013-03-01 09:11:58.703242|10.70|kindergarten +18|503|65554|4294967454|40.94|28.87|true|mike falkner|2013-03-01 09:11:58.703203|90.82|nap time +8|359|65563|4294967315|45.01|30.71|true|bob king|2013-03-01 09:11:58.703097|32.07|mathematics +15|379|65700|4294967389|49.52|11.37|true|irene laertes|2013-03-01 09:11:58.703318|96.96|debate +76|464|65622|4294967438|20.81|47.60|true|fred thompson|2013-03-01 09:11:58.703119|75.11|topology +56|428|65621|4294967481|90.01|3.72|false|fred thompson|2013-03-01 09:11:58.703113|17.79|debate +19|468|65669|4294967330|27.80|18.05|false|yuri davidson|2013-03-01 09:11:58.703178|15.22|geology +75|366|65759|4294967525|68.03|15.05|false|ethan zipper|2013-03-01 09:11:58.703106|80.47|debate +94|445|65585|4294967425|81.00|14.34|false|irene johnson|2013-03-01 09:11:58.703244|90.17|forestry +75|267|65700|4294967548|46.15|34.94|true|irene garcia|2013-03-01 09:11:58.703132|36.04|yard duty +113|466|65767|4294967424|33.20|24.83|true|priscilla carson|2013-03-01 09:11:58.703139|7.83|nap time +80|286|65658|4294967433|61.20|32.05|true|holly allen|2013-03-01 09:11:58.703118|25.00|quiet hour +59|270|65647|4294967380|92.56|28.98|false|ulysses zipper|2013-03-01 09:11:58.703284|76.63|history +117|498|65658|4294967538|21.79|35.77|true|tom falkner|2013-03-01 09:11:58.703207|7.88|opthamology +35|346|65664|4294967491|8.43|21.74|false|nick garcia|2013-03-01 09:11:58.703281|92.95|xylophone band +112|363|65646|4294967340|65.36|4.08|false|ethan thompson|2013-03-01 09:11:58.703267|71.78|opthamology +56|484|65544|4294967430|67.57|25.86|false|victor garcia|2013-03-01 09:11:58.703231|59.82|wind surfing +107|373|65562|4294967452|72.41|18.69|true|tom brown|2013-03-01 09:11:58.703290|0.25|chemistry +27|296|65738|4294967434|8.63|9.09|false|ulysses white|2013-03-01 09:11:58.703303|48.85|undecided +80|355|65711|4294967501|74.64|29.41|false|quinn polk|2013-03-01 09:11:58.703151|16.72|education +90|295|65735|4294967521|1.36|18.51|true|jessica robinson|2013-03-01 09:11:58.703255|50.80|history +5|333|65607|4294967390|55.47|42.18|false|david allen|2013-03-01 09:11:58.703135|81.93|wind surfing +118|265|65685|4294967545|46.24|6.79|false|xavier steinbeck|2013-03-01 09:11:58.703094|91.93|undecided +-2|387|65593|4294967435|24.40|10.27|true|wendy ichabod|2013-03-01 09:11:58.703244|42.76|geology +111|508|65619|4294967520|32.56|4.98|true|mike steinbeck|2013-03-01 09:11:58.703230|20.35|nap time +117|403|65717|4294967420|52.26|44.93|true|victor robinson|2013-03-01 09:11:58.703218|93.24|values clariffication +101|383|65694|4294967393|46.03|43.21|true|nick miller|2013-03-01 09:11:58.703141|91.85|zync studies +79|282|65750|4294967501|81.01|11.53|false|holly white|2013-03-01 09:11:58.703242|19.78|zync studies +67|433|65624|4294967352|43.05|34.96|false|calvin xylophone|2013-03-01 09:11:58.703117|5.98|xylophone band +56|402|65536|4294967497|70.96|18.11|false|oscar garcia|2013-03-01 09:11:58.703234|59.58|nap time +64|286|65728|4294967457|5.53|21.76|false|holly ichabod|2013-03-01 09:11:58.703220|61.70|topology +18|471|65546|4294967330|19.03|37.39|false|zach falkner|2013-03-01 09:11:58.703301|5.93|history +58|439|65702|4294967382|60.01|23.60|false|gabriella robinson|2013-03-01 09:11:58.703157|17.23|philosophy +2|461|65600|4294967365|74.27|28.82|false|tom ellison|2013-03-01 09:11:58.703152|45.36|topology +104|327|65596|4294967348|85.39|6.30|false|xavier falkner|2013-03-01 09:11:58.703195|50.97|values clariffication +21|412|65774|4294967518|91.46|45.42|false|rachel zipper|2013-03-01 09:11:58.703171|9.47|kindergarten +84|348|65595|4294967481|47.01|9.47|true|fred johnson|2013-03-01 09:11:58.703127|29.43|chemistry +13|392|65695|4294967476|0.04|20.24|false|holly steinbeck|2013-03-01 09:11:58.703175|69.75|undecided +79|302|65696|4294967533|79.34|29.20|false|tom hernandez|2013-03-01 09:11:58.703200|93.98|study skills +48|280|65754|4294967431|23.99|29.86|true|xavier steinbeck|2013-03-01 09:11:58.703289|95.07|industrial engineering +40|339|65679|4294967404|96.49|16.66|true|xavier brown|2013-03-01 09:11:58.703083|81.27|study skills +57|355|65650|4294967430|68.16|26.64|true|sarah robinson|2013-03-01 09:11:58.703094|30.77|education +121|308|65617|4294967505|38.33|38.09|true|bob steinbeck|2013-03-01 09:11:58.703310|91.57|american history +43|482|65582|4294967439|89.15|18.49|false|oscar robinson|2013-03-01 09:11:58.703244|43.56|zync studies +23|334|65697|4294967331|60.43|43.53|false|quinn brown|2013-03-01 09:11:58.703171|16.27|geology +43|481|65729|4294967433|92.12|3.75|false|gabriella xylophone|2013-03-01 09:11:58.703140|9.20|geology +20|280|65743|4294967451|48.42|10.24|true|calvin garcia|2013-03-01 09:11:58.703171|28.74|opthamology +102|492|65690|4294967546|41.20|39.80|false|ethan johnson|2013-03-01 09:11:58.703315|99.14|linguistics +11|291|65670|4294967457|81.66|12.89|true|fred van buren|2013-03-01 09:11:58.703263|33.55|nap time +39|490|65608|4294967360|38.66|37.67|false|xavier falkner|2013-03-01 09:11:58.703120|48.80|zync studies +56|395|65560|4294967502|87.13|25.57|true|priscilla steinbeck|2013-03-01 09:11:58.703119|56.00|forestry +88|423|65707|4294967522|17.52|30.00|true|priscilla garcia|2013-03-01 09:11:58.703105|62.96|values clariffication +49|327|65684|4294967417|12.61|31.31|false|ulysses young|2013-03-01 09:11:58.703210|37.95|history +0|407|65611|4294967517|3.75|7.87|true|yuri king|2013-03-01 09:11:58.703090|34.27|debate +42|269|65760|4294967405|7.75|2.12|true|alice laertes|2013-03-01 09:11:58.703272|87.17|forestry +22|295|65682|4294967299|37.55|11.50|false|priscilla xylophone|2013-03-01 09:11:58.703098|17.12|american history +83|416|65624|4294967380|33.85|17.35|true|katie quirinius|2013-03-01 09:11:58.703301|4.90|zync studies +37|491|65751|4294967353|98.30|42.26|true|priscilla davidson|2013-03-01 09:11:58.703243|13.56|xylophone band +119|327|65650|4294967472|26.29|10.20|true|bob steinbeck|2013-03-01 09:11:58.703288|91.33|quiet hour +39|275|65617|4294967478|54.92|37.32|false|david quirinius|2013-03-01 09:11:58.703275|88.08|topology +6|498|65782|4294967363|9.39|21.20|true|oscar miller|2013-03-01 09:11:58.703305|0.15|philosophy +97|307|65604|4294967360|62.07|15.14|false|xavier johnson|2013-03-01 09:11:58.703324|35.84|mathematics +3|299|65655|4294967391|10.67|43.79|false|irene king|2013-03-01 09:11:58.703237|81.69|undecided +39|329|65751|4294967305|43.87|8.26|false|alice steinbeck|2013-03-01 09:11:58.703231|83.38|debate +121|359|65697|4294967498|40.82|20.75|false|fred laertes|2013-03-01 09:11:58.703129|19.72|forestry +25|310|65552|4294967471|48.94|0.92|false|irene davidson|2013-03-01 09:11:58.703222|1.77|chemistry +33|405|65774|4294967444|96.69|40.63|false|zach xylophone|2013-03-01 09:11:58.703292|29.06|debate +50|362|65711|4294967522|25.57|32.07|false|ulysses miller|2013-03-01 09:11:58.703156|60.81|yard duty +104|502|65611|4294967474|7.63|41.55|false|katie van buren|2013-03-01 09:11:58.703088|40.14|values clariffication +92|382|65639|4294967332|52.04|9.25|false|katie ichabod|2013-03-01 09:11:58.703123|47.45|opthamology +6|315|65560|4294967329|80.55|4.48|false|ethan ellison|2013-03-01 09:11:58.703323|52.81|xylophone band +91|451|65613|4294967377|65.38|6.63|true|yuri nixon|2013-03-01 09:11:58.703091|13.81|american history +77|435|65730|4294967502|5.47|10.48|true|holly ellison|2013-03-01 09:11:58.703113|94.39|linguistics +78|336|65784|4294967503|62.14|39.70|true|gabriella ovid|2013-03-01 09:11:58.703213|8.67|nap time +123|293|65580|4294967509|62.30|45.90|true|xavier underhill|2013-03-01 09:11:58.703294|66.74|industrial engineering +115|327|65562|4294967428|96.85|21.22|true|ethan robinson|2013-03-01 09:11:58.703290|82.31|chemistry +118|316|65732|4294967339|19.78|15.81|false|ulysses ichabod|2013-03-01 09:11:58.703110|13.61|forestry +4|286|65762|4294967375|77.46|6.52|true|katie robinson|2013-03-01 09:11:58.703242|36.69|study skills +88|509|65542|4294967377|40.08|20.26|false|irene underhill|2013-03-01 09:11:58.703126|83.03|debate +13|392|65738|4294967547|53.31|33.02|true|calvin brown|2013-03-01 09:11:58.703199|22.73|geology +105|470|65760|4294967420|96.71|8.42|true|gabriella polk|2013-03-01 09:11:58.703162|99.99|forestry +29|406|65753|4294967428|8.46|46.56|true|tom quirinius|2013-03-01 09:11:58.703079|62.35|yard duty +108|481|65580|4294967355|79.52|4.13|false|david young|2013-03-01 09:11:58.703311|1.56|joggying +95|487|65702|4294967549|12.92|37.95|false|mike ovid|2013-03-01 09:11:58.703317|96.45|industrial engineering +6|497|65552|4294967461|98.50|41.77|false|rachel ichabod|2013-03-01 09:11:58.703090|20.90|values clariffication +109|403|65748|4294967395|45.88|36.01|false|gabriella xylophone|2013-03-01 09:11:58.703169|29.74|undecided +27|412|65754|4294967541|8.11|13.72|true|tom young|2013-03-01 09:11:58.703259|37.65|quiet hour +22|362|65735|4294967391|88.01|17.73|true|fred young|2013-03-01 09:11:58.703214|60.37|topology +123|289|65645|4294967465|55.42|10.50|true|rachel ichabod|2013-03-01 09:11:58.703275|39.61|geology +56|389|65763|4294967318|85.41|3.94|false|ulysses white|2013-03-01 09:11:58.703324|46.06|yard duty +27|364|65789|4294967334|9.03|31.78|false|luke robinson|2013-03-01 09:11:58.703157|65.87|biology +5|496|65653|4294967327|14.66|7.55|false|luke ellison|2013-03-01 09:11:58.703168|68.92|linguistics +72|448|65651|4294967417|51.39|10.17|false|fred ichabod|2013-03-01 09:11:58.703311|22.77|philosophy +40|298|65784|4294967540|27.32|43.30|true|luke polk|2013-03-01 09:11:58.703254|19.10|yard duty +84|425|65588|4294967482|14.08|21.46|false|tom robinson|2013-03-01 09:11:58.703127|82.52|religion +19|436|65575|4294967403|23.25|41.24|true|irene allen|2013-03-01 09:11:58.703096|21.15|nap time +44|462|65705|4294967395|5.40|3.06|false|holly underhill|2013-03-01 09:11:58.703076|89.97|quiet hour +57|421|65735|4294967322|94.43|32.46|false|zach johnson|2013-03-01 09:11:58.703136|78.86|yard duty +19|263|65622|4294967349|20.30|18.39|false|jessica van buren|2013-03-01 09:11:58.703265|14.75|geology +113|330|65748|4294967310|95.94|43.04|false|sarah hernandez|2013-03-01 09:11:58.703275|12.14|values clariffication +101|451|65700|4294967303|35.79|37.53|false|tom polk|2013-03-01 09:11:58.703170|95.51|american history +108|330|65546|4294967472|36.79|42.85|false|victor johnson|2013-03-01 09:11:58.703133|4.64|undecided +69|317|65553|4294967403|80.00|4.80|false|ulysses white|2013-03-01 09:11:58.703108|28.80|quiet hour +91|412|65681|4294967398|41.91|48.08|true|zach ichabod|2013-03-01 09:11:58.703230|72.62|kindergarten +111|349|65721|4294967334|19.53|30.89|true|sarah ovid|2013-03-01 09:11:58.703307|57.66|religion +92|425|65732|4294967341|16.48|20.78|true|david king|2013-03-01 09:11:58.703127|4.44|study skills +60|330|65679|4294967547|38.59|31.78|true|mike laertes|2013-03-01 09:11:58.703201|15.36|philosophy +27|383|65716|4294967374|46.21|45.77|false|sarah thompson|2013-03-01 09:11:58.703160|5.75|american history +99|432|65768|4294967484|83.65|38.22|true|zach carson|2013-03-01 09:11:58.703144|75.64|nap time +58|441|65629|4294967318|90.68|44.05|true|jessica ichabod|2013-03-01 09:11:58.703232|17.68|biology +111|428|65613|4294967348|20.06|18.37|true|wendy ichabod|2013-03-01 09:11:58.703083|54.58|wind surfing +83|500|65767|4294967468|73.04|2.90|true|zach xylophone|2013-03-01 09:11:58.703113|55.12|mathematics +99|494|65715|4294967323|90.86|42.11|false|quinn ichabod|2013-03-01 09:11:58.703245|99.64|biology +2|300|65703|4294967309|36.68|16.48|false|oscar steinbeck|2013-03-01 09:11:58.703146|38.14|biology +98|309|65666|4294967336|84.58|18.84|true|priscilla johnson|2013-03-01 09:11:58.703117|88.45|zync studies +21|292|65784|4294967431|29.45|35.90|false|katie brown|2013-03-01 09:11:58.703138|62.91|quiet hour +-2|446|65790|4294967302|6.49|10.81|false|alice underhill|2013-03-01 09:11:58.703127|43.92|undecided +60|319|65578|4294967361|86.93|25.81|false|xavier white|2013-03-01 09:11:58.703088|78.39|undecided +11|345|65732|4294967327|7.80|11.70|false|irene nixon|2013-03-01 09:11:58.703318|53.57|wind surfing +110|426|65606|4294967512|20.14|14.02|true|holly allen|2013-03-01 09:11:58.703089|2.15|industrial engineering +32|272|65591|4294967307|58.90|34.73|false|irene underhill|2013-03-01 09:11:58.703135|58.40|topology +80|416|65608|4294967414|67.61|3.22|true|yuri falkner|2013-03-01 09:11:58.703239|22.16|geology +5|286|65743|4294967481|56.60|26.76|false|katie carson|2013-03-01 09:11:58.703146|51.62|values clariffication +76|456|65554|4294967416|49.40|37.74|true|ulysses steinbeck|2013-03-01 09:11:58.703113|92.24|biology +80|450|65604|4294967445|30.42|13.69|true|irene carson|2013-03-01 09:11:58.703169|98.93|undecided +13|309|65555|4294967366|59.08|34.92|false|victor polk|2013-03-01 09:11:58.703255|36.37|wind surfing +43|278|65688|4294967381|60.79|38.91|false|katie ichabod|2013-03-01 09:11:58.703237|60.66|kindergarten +16|392|65573|4294967518|50.88|49.35|true|calvin miller|2013-03-01 09:11:58.703126|31.42|chemistry +19|397|65552|4294967399|40.92|30.03|false|tom steinbeck|2013-03-01 09:11:58.703319|61.39|history +30|479|65777|4294967382|21.51|38.96|false|tom nixon|2013-03-01 09:11:58.703276|38.98|kindergarten +60|499|65546|4294967467|5.37|19.81|false|katie davidson|2013-03-01 09:11:58.703176|14.69|history +-1|497|65661|4294967447|51.17|44.94|false|xavier polk|2013-03-01 09:11:58.703211|5.50|religion +96|262|65706|4294967440|11.24|4.90|false|wendy hernandez|2013-03-01 09:11:58.703145|70.02|history +75|386|65623|4294967449|54.12|32.90|true|oscar nixon|2013-03-01 09:11:58.703252|44.44|zync studies +74|430|65570|4294967453|93.01|1.32|false|holly johnson|2013-03-01 09:11:58.703320|4.45|values clariffication +24|458|65542|4294967447|98.05|19.68|false|irene ellison|2013-03-01 09:11:58.703094|56.59|linguistics +72|429|65618|4294967544|18.36|47.69|false|alice white|2013-03-01 09:11:58.703267|35.01|opthamology +11|493|65603|4294967508|81.31|30.27|false|nick polk|2013-03-01 09:11:58.703159|33.00|industrial engineering +105|343|65605|4294967467|51.37|16.62|false|ethan zipper|2013-03-01 09:11:58.703126|76.95|linguistics +75|415|65669|4294967528|9.14|40.66|true|fred ellison|2013-03-01 09:11:58.703297|20.93|mathematics +6|344|65764|4294967326|79.68|21.27|true|mike laertes|2013-03-01 09:11:58.703286|19.78|philosophy +10|375|65600|4294967311|84.15|16.59|true|zach young|2013-03-01 09:11:58.703108|15.57|biology +16|503|65790|4294967454|84.73|48.11|true|priscilla allen|2013-03-01 09:11:58.703084|36.73|religion +84|499|65594|4294967359|21.95|4.33|true|david miller|2013-03-01 09:11:58.703157|69.79|history +75|369|65666|4294967465|28.30|7.63|true|fred polk|2013-03-01 09:11:58.703189|83.04|zync studies +83|345|65548|4294967456|20.46|17.87|false|luke laertes|2013-03-01 09:11:58.703233|90.38|zync studies +116|285|65672|4294967366|94.78|22.93|false|wendy nixon|2013-03-01 09:11:58.703093|76.34|forestry +117|474|65767|4294967428|51.18|47.51|false|priscilla xylophone|2013-03-01 09:11:58.703311|12.36|nap time +65|468|65639|4294967510|66.59|21.79|true|rachel nixon|2013-03-01 09:11:58.703129|85.92|quiet hour +16|288|65604|4294967531|1.93|37.32|false|priscilla brown|2013-03-01 09:11:58.703249|13.13|debate +-3|266|65736|4294967397|19.94|10.01|false|quinn ellison|2013-03-01 09:11:58.703232|88.94|forestry +22|364|65594|4294967394|83.98|44.24|false|zach nixon|2013-03-01 09:11:58.703113|44.76|debate +99|330|65604|4294967546|10.00|33.64|true|zach van buren|2013-03-01 09:11:58.703240|79.46|opthamology +79|463|65683|4294967328|21.62|48.23|true|bob underhill|2013-03-01 09:11:58.703236|27.03|american history +94|400|65705|4294967330|67.98|34.90|false|irene thompson|2013-03-01 09:11:58.703229|40.68|debate +62|451|65735|4294967373|69.25|32.44|true|zach brown|2013-03-01 09:11:58.703247|53.94|study skills +77|486|65698|4294967439|58.33|20.26|true|xavier garcia|2013-03-01 09:11:58.703120|12.59|zync studies +124|371|65627|4294967420|50.17|43.72|true|tom laertes|2013-03-01 09:11:58.703113|24.20|undecided +70|269|65779|4294967462|41.45|40.90|true|ethan zipper|2013-03-01 09:11:58.703106|5.58|chemistry +53|370|65560|4294967482|81.65|27.42|false|quinn underhill|2013-03-01 09:11:58.703123|10.70|undecided +119|454|65545|4294967440|56.56|33.80|true|bob xylophone|2013-03-01 09:11:58.703264|25.86|yard duty +63|337|65749|4294967522|44.13|6.34|true|victor davidson|2013-03-01 09:11:58.703238|34.91|linguistics +87|292|65547|4294967469|80.64|37.90|true|yuri johnson|2013-03-01 09:11:58.703232|90.75|education +124|439|65657|4294967381|66.29|41.53|false|yuri xylophone|2013-03-01 09:11:58.703186|10.62|industrial engineering +58|256|65543|4294967313|32.21|32.15|false|ethan thompson|2013-03-01 09:11:58.703108|21.23|religion +42|496|65547|4294967499|34.73|23.17|false|tom garcia|2013-03-01 09:11:58.703233|66.23|study skills +36|280|65610|4294967360|73.52|42.54|true|priscilla zipper|2013-03-01 09:11:58.703070|4.52|history +48|455|65669|4294967426|94.91|41.74|false|wendy robinson|2013-03-01 09:11:58.703133|46.92|debate +72|438|65606|4294967395|21.62|32.60|true|quinn thompson|2013-03-01 09:11:58.703202|23.09|linguistics +47|409|65601|4294967526|90.40|34.56|false|fred ellison|2013-03-01 09:11:58.703121|36.43|philosophy +45|496|65728|4294967508|7.41|25.85|true|oscar johnson|2013-03-01 09:11:58.703106|45.38|linguistics +58|503|65697|4294967417|89.42|39.57|true|quinn robinson|2013-03-01 09:11:58.703231|24.36|wind surfing +65|406|65616|4294967332|21.94|10.76|false|oscar falkner|2013-03-01 09:11:58.703229|70.76|philosophy +5|292|65702|4294967520|71.60|16.89|false|mike johnson|2013-03-01 09:11:58.703234|76.79|chemistry +75|437|65690|4294967430|62.00|47.25|true|alice davidson|2013-03-01 09:11:58.703241|14.33|american history +56|431|65766|4294967479|93.73|20.13|true|quinn nixon|2013-03-01 09:11:58.703172|48.02|linguistics +75|338|65713|4294967315|31.15|29.85|false|bob zipper|2013-03-01 09:11:58.703269|7.62|debate +50|286|65629|4294967352|21.00|12.70|true|tom brown|2013-03-01 09:11:58.703280|97.10|joggying +69|397|65681|4294967355|68.14|15.95|true|yuri quirinius|2013-03-01 09:11:58.703280|78.46|kindergarten +32|341|65573|4294967466|99.53|29.64|true|jessica nixon|2013-03-01 09:11:58.703239|46.00|wind surfing +21|451|65680|4294967430|63.21|47.66|true|calvin falkner|2013-03-01 09:11:58.703213|57.33|values clariffication +30|411|65600|4294967311|57.73|33.62|true|oscar falkner|2013-03-01 09:11:58.703287|36.56|chemistry +81|371|65785|4294967494|82.22|33.57|false|katie underhill|2013-03-01 09:11:58.703291|38.50|american history +103|434|65543|4294967493|57.15|3.24|false|priscilla king|2013-03-01 09:11:58.703113|9.98|yard duty +104|462|65790|4294967514|50.30|11.34|false|zach laertes|2013-03-01 09:11:58.703258|2.11|wind surfing +53|334|65769|4294967385|68.70|24.14|true|zach garcia|2013-03-01 09:11:58.703097|46.37|debate +96|456|65646|4294967411|50.69|34.08|true|gabriella nixon|2013-03-01 09:11:58.703177|77.95|nap time +95|367|65657|4294967396|27.13|27.49|true|oscar ellison|2013-03-01 09:11:58.703221|62.81|xylophone band +120|308|65664|4294967358|69.41|12.30|true|ulysses falkner|2013-03-01 09:11:58.703320|3.46|opthamology +40|437|65756|4294967520|15.68|37.01|true|rachel king|2013-03-01 09:11:58.703277|75.57|american history +75|281|65589|4294967436|64.25|49.97|true|victor white|2013-03-01 09:11:58.703213|96.71|philosophy +83|432|65756|4294967343|53.50|44.33|true|ulysses robinson|2013-03-01 09:11:58.703309|86.76|philosophy +79|495|65701|4294967459|55.37|36.27|true|ulysses ichabod|2013-03-01 09:11:58.703153|30.60|mathematics +35|360|65777|4294967548|1.24|33.12|false|xavier king|2013-03-01 09:11:58.703198|56.70|study skills +118|432|65725|4294967474|26.39|45.59|true|irene johnson|2013-03-01 09:11:58.703234|56.79|industrial engineering +98|500|65720|4294967521|64.57|15.83|true|calvin white|2013-03-01 09:11:58.703280|71.83|history +70|451|65692|4294967501|87.30|5.99|false|wendy miller|2013-03-01 09:11:58.703187|86.66|biology +57|489|65606|4294967549|62.08|31.59|true|irene hernandez|2013-03-01 09:11:58.703131|12.56|xylophone band +76|494|65771|4294967343|85.91|21.59|true|yuri ichabod|2013-03-01 09:11:58.703109|92.93|chemistry +78|435|65753|4294967459|32.07|31.67|false|jessica davidson|2013-03-01 09:11:58.703153|67.73|joggying +45|279|65581|4294967479|60.82|26.41|false|ulysses zipper|2013-03-01 09:11:58.703126|48.11|study skills +40|383|65626|4294967440|83.01|48.16|false|katie miller|2013-03-01 09:11:58.703172|30.80|study skills +86|375|65555|4294967438|7.96|6.63|false|nick ellison|2013-03-01 09:11:58.703083|65.81|study skills +14|363|65564|4294967404|44.58|3.51|true|nick white|2013-03-01 09:11:58.703205|48.83|joggying +72|337|65660|4294967498|82.85|5.02|true|fred king|2013-03-01 09:11:58.703071|57.47|xylophone band +120|494|65578|4294967483|49.96|22.34|true|david zipper|2013-03-01 09:11:58.703146|77.60|kindergarten +29|413|65730|4294967325|83.27|14.91|false|calvin garcia|2013-03-01 09:11:58.703176|60.58|industrial engineering +15|272|65632|4294967547|76.02|3.27|true|bob robinson|2013-03-01 09:11:58.703082|25.77|religion +10|323|65659|4294967455|51.37|32.04|true|katie ichabod|2013-03-01 09:11:58.703299|34.63|linguistics +68|298|65629|4294967328|85.49|46.61|true|tom zipper|2013-03-01 09:11:58.703083|47.08|philosophy +92|437|65743|4294967350|0.11|11.01|true|ulysses zipper|2013-03-01 09:11:58.703166|58.54|quiet hour +62|269|65603|4294967414|55.12|14.55|true|katie falkner|2013-03-01 09:11:58.703125|63.00|nap time +-3|343|65783|4294967378|7.10|18.16|true|ulysses carson|2013-03-01 09:11:58.703253|96.99|mathematics +9|332|65758|4294967471|34.89|22.99|false|fred van buren|2013-03-01 09:11:58.703186|67.89|debate +95|299|65605|4294967391|28.73|35.47|true|bob ellison|2013-03-01 09:11:58.703222|34.54|joggying +121|338|65731|4294967479|66.71|29.02|false|katie zipper|2013-03-01 09:11:58.703217|86.02|forestry +114|402|65699|4294967441|69.19|49.64|false|luke van buren|2013-03-01 09:11:58.703152|3.65|philosophy +119|350|65661|4294967549|96.58|35.36|false|fred white|2013-03-01 09:11:58.703254|13.33|opthamology +113|382|65566|4294967469|92.65|7.54|true|ethan ovid|2013-03-01 09:11:58.703283|58.03|linguistics +75|285|65740|4294967506|40.62|32.14|false|mike zipper|2013-03-01 09:11:58.703313|87.38|history +41|427|65674|4294967527|92.19|33.31|true|ulysses miller|2013-03-01 09:11:58.703286|63.87|wind surfing +105|475|65631|4294967507|14.89|37.86|false|gabriella underhill|2013-03-01 09:11:58.703162|86.78|religion +90|400|65772|4294967532|26.72|33.65|true|holly allen|2013-03-01 09:11:58.703148|0.25|debate +79|363|65629|4294967395|9.26|35.14|true|quinn quirinius|2013-03-01 09:11:58.703138|65.45|chemistry +83|434|65734|4294967308|6.16|16.37|true|zach hernandez|2013-03-01 09:11:58.703256|75.10|joggying +64|489|65646|4294967490|78.70|22.63|true|alice young|2013-03-01 09:11:58.703234|78.56|zync studies +31|468|65758|4294967432|46.22|12.92|true|mike steinbeck|2013-03-01 09:11:58.703306|75.59|education +90|340|65738|4294967461|39.43|22.38|true|yuri brown|2013-03-01 09:11:58.703264|54.93|philosophy +93|268|65664|4294967327|27.34|28.49|true|gabriella polk|2013-03-01 09:11:58.703244|80.69|topology +3|423|65539|4294967489|94.29|42.38|true|tom nixon|2013-03-01 09:11:58.703135|16.00|topology +64|353|65728|4294967345|76.95|43.36|false|ulysses xylophone|2013-03-01 09:11:58.703224|69.60|joggying +15|431|65596|4294967406|69.52|46.13|true|holly white|2013-03-01 09:11:58.703175|33.00|industrial engineering +37|469|65644|4294967315|51.18|6.22|false|nick underhill|2013-03-01 09:11:58.703325|62.72|industrial engineering +59|288|65654|4294967331|25.76|25.92|false|bob allen|2013-03-01 09:11:58.703139|53.06|values clariffication +106|492|65769|4294967468|20.58|45.03|true|jessica allen|2013-03-01 09:11:58.703118|31.50|opthamology +49|459|65672|4294967428|7.73|13.03|false|priscilla johnson|2013-03-01 09:11:58.703236|10.32|chemistry +63|468|65640|4294967496|65.03|22.02|true|mike garcia|2013-03-01 09:11:58.703271|9.89|undecided +61|443|65634|4294967415|21.32|37.23|false|rachel polk|2013-03-01 09:11:58.703178|71.02|american history +22|281|65581|4294967393|55.22|28.79|true|luke hernandez|2013-03-01 09:11:58.703270|33.92|religion +45|279|65764|4294967538|72.31|19.00|false|xavier falkner|2013-03-01 09:11:58.703238|40.51|nap time +61|407|65768|4294967429|42.60|3.04|true|zach xylophone|2013-03-01 09:11:58.703100|25.98|kindergarten +11|303|65721|4294967308|68.93|41.13|true|gabriella garcia|2013-03-01 09:11:58.703150|86.70|education +66|468|65611|4294967432|95.47|1.90|false|oscar falkner|2013-03-01 09:11:58.703178|27.62|industrial engineering +81|506|65579|4294967368|79.66|2.09|true|nick brown|2013-03-01 09:11:58.703288|7.49|nap time +81|369|65607|4294967515|7.15|37.01|false|holly ichabod|2013-03-01 09:11:58.703090|69.02|biology +59|349|65781|4294967350|49.53|1.15|false|oscar van buren|2013-03-01 09:11:58.703111|89.75|forestry +21|372|65650|4294967301|61.86|14.19|true|alice quirinius|2013-03-01 09:11:58.703180|43.84|nap time +-1|336|65702|4294967539|11.03|44.94|true|yuri robinson|2013-03-01 09:11:58.703167|74.33|zync studies +-2|352|65600|4294967320|75.72|25.21|true|katie thompson|2013-03-01 09:11:58.703138|36.31|biology +97|406|65545|4294967445|81.27|47.73|true|katie laertes|2013-03-01 09:11:58.703077|55.39|debate +62|496|65743|4294967386|34.07|11.01|true|zach quirinius|2013-03-01 09:11:58.703301|22.07|study skills +20|307|65691|4294967315|13.88|21.29|false|rachel young|2013-03-01 09:11:58.703241|2.82|debate +120|278|65555|4294967515|21.33|5.07|false|sarah ellison|2013-03-01 09:11:58.703135|35.63|zync studies +21|297|65765|4294967529|73.12|17.27|false|alice king|2013-03-01 09:11:58.703278|4.28|topology +53|361|65594|4294967368|87.69|21.61|true|sarah steinbeck|2013-03-01 09:11:58.703110|66.09|undecided +2|272|65544|4294967428|18.56|43.34|true|zach polk|2013-03-01 09:11:58.703176|77.62|debate +33|444|65564|4294967413|15.37|27.16|true|alice polk|2013-03-01 09:11:58.703318|7.90|kindergarten +91|329|65692|4294967365|31.16|25.15|false|wendy ovid|2013-03-01 09:11:58.703154|5.73|zync studies +101|457|65663|4294967532|39.03|24.25|true|bob thompson|2013-03-01 09:11:58.703084|55.71|religion +27|305|65616|4294967547|76.34|29.37|true|wendy nixon|2013-03-01 09:11:58.703198|94.05|biology +-1|282|65777|4294967314|37.27|34.32|true|oscar garcia|2013-03-01 09:11:58.703093|91.40|forestry +73|412|65683|4294967458|85.99|48.25|false|jessica steinbeck|2013-03-01 09:11:58.703150|45.98|values clariffication +35|316|65751|4294967436|47.62|41.68|false|fred xylophone|2013-03-01 09:11:58.703219|67.34|values clariffication +83|310|65737|4294967399|49.52|23.24|false|gabriella ellison|2013-03-01 09:11:58.703262|93.39|history +98|307|65588|4294967542|63.52|2.59|false|yuri allen|2013-03-01 09:11:58.703295|42.90|history +113|260|65600|4294967336|34.54|5.22|true|priscilla brown|2013-03-01 09:11:58.703115|7.92|chemistry +69|449|65665|4294967381|51.06|33.25|true|ulysses quirinius|2013-03-01 09:11:58.703094|80.72|chemistry +53|299|65639|4294967546|9.51|10.41|true|mike brown|2013-03-01 09:11:58.703209|54.88|chemistry +87|415|65725|4294967341|73.19|16.63|true|mike ovid|2013-03-01 09:11:58.703083|55.89|american history +31|389|65623|4294967465|14.71|34.87|false|bob white|2013-03-01 09:11:58.703271|33.69|yard duty +107|293|65701|4294967482|31.76|12.35|false|luke zipper|2013-03-01 09:11:58.703079|74.91|industrial engineering +5|305|65780|4294967448|23.84|37.07|false|gabriella steinbeck|2013-03-01 09:11:58.703226|78.18|mathematics +77|273|65738|4294967430|32.03|14.47|false|wendy quirinius|2013-03-01 09:11:58.703228|41.11|linguistics +65|317|65665|4294967344|56.07|35.85|true|david laertes|2013-03-01 09:11:58.703255|66.81|quiet hour +31|276|65757|4294967359|40.15|30.34|true|tom ichabod|2013-03-01 09:11:58.703111|41.01|undecided +26|494|65685|4294967418|37.81|1.87|true|david quirinius|2013-03-01 09:11:58.703233|15.35|education +94|485|65617|4294967484|42.32|49.09|false|ethan davidson|2013-03-01 09:11:58.703266|59.68|wind surfing +68|482|65772|4294967444|75.25|43.37|false|bob laertes|2013-03-01 09:11:58.703236|95.13|religion +17|279|65555|4294967510|24.16|30.74|true|rachel thompson|2013-03-01 09:11:58.703189|64.46|topology +34|306|65676|4294967373|81.70|1.92|false|ulysses polk|2013-03-01 09:11:58.703180|36.50|chemistry +94|426|65703|4294967323|70.46|26.79|true|wendy steinbeck|2013-03-01 09:11:58.703135|24.04|linguistics +112|322|65608|4294967408|9.00|30.71|true|zach xylophone|2013-03-01 09:11:58.703275|41.21|zync studies +90|441|65699|4294967334|39.28|39.93|false|nick ichabod|2013-03-01 09:11:58.703173|54.68|joggying +33|391|65646|4294967329|59.47|27.63|false|rachel allen|2013-03-01 09:11:58.703268|37.78|chemistry +86|482|65557|4294967311|89.55|44.80|true|nick white|2013-03-01 09:11:58.703290|6.92|forestry +97|417|65562|4294967549|3.61|36.42|false|yuri polk|2013-03-01 09:11:58.703272|71.59|history +80|280|65739|4294967377|39.11|22.88|false|luke laertes|2013-03-01 09:11:58.703093|24.65|education +17|420|65612|4294967522|99.69|13.02|true|david thompson|2013-03-01 09:11:58.703139|25.97|forestry +81|413|65559|4294967387|98.95|13.09|false|ulysses xylophone|2013-03-01 09:11:58.703290|23.55|debate +93|263|65638|4294967456|3.91|30.13|false|rachel underhill|2013-03-01 09:11:58.703103|21.39|philosophy +8|414|65788|4294967327|70.78|45.44|false|oscar allen|2013-03-01 09:11:58.703212|48.38|education +68|492|65658|4294967363|74.64|10.88|true|ulysses quirinius|2013-03-01 09:11:58.703265|38.54|geology +107|391|65615|4294967547|8.03|7.88|false|fred van buren|2013-03-01 09:11:58.703226|77.44|kindergarten +97|463|65554|4294967355|10.18|42.62|true|fred carson|2013-03-01 09:11:58.703134|55.76|joggying +85|441|65669|4294967455|25.01|46.53|true|ulysses underhill|2013-03-01 09:11:58.703139|48.73|industrial engineering +35|371|65783|4294967356|18.42|24.74|true|sarah zipper|2013-03-01 09:11:58.703102|83.13|xylophone band +64|395|65653|4294967515|96.15|16.73|true|zach johnson|2013-03-01 09:11:58.703239|84.83|kindergarten +111|273|65693|4294967416|96.37|48.16|false|nick hernandez|2013-03-01 09:11:58.703290|14.71|zync studies +54|436|65695|4294967454|15.29|36.76|true|ethan davidson|2013-03-01 09:11:58.703192|93.62|values clariffication +42|314|65701|4294967444|99.91|41.83|true|yuri ellison|2013-03-01 09:11:58.703214|5.50|opthamology +106|324|65730|4294967544|85.98|47.75|true|oscar young|2013-03-01 09:11:58.703321|18.19|industrial engineering +48|429|65784|4294967481|8.03|49.80|false|yuri hernandez|2013-03-01 09:11:58.703316|12.71|geology +27|288|65785|4294967438|85.86|1.98|true|irene young|2013-03-01 09:11:58.703092|3.07|linguistics +6|405|65735|4294967321|31.43|23.93|true|katie van buren|2013-03-01 09:11:58.703235|19.96|history +107|447|65713|4294967426|92.32|37.88|false|victor laertes|2013-03-01 09:11:58.703144|33.88|quiet hour +55|277|65692|4294967382|83.25|30.65|true|zach ichabod|2013-03-01 09:11:58.703292|89.58|undecided +82|349|65570|4294967482|73.17|3.12|true|bob allen|2013-03-01 09:11:58.703107|95.06|values clariffication +112|495|65670|4294967415|19.07|25.44|true|yuri nixon|2013-03-01 09:11:58.703268|61.12|debate +-2|331|65540|4294967334|33.49|7.64|false|fred allen|2013-03-01 09:11:58.703191|74.25|philosophy +52|300|65585|4294967379|64.96|46.74|false|wendy steinbeck|2013-03-01 09:11:58.703070|34.22|opthamology +61|310|65665|4294967514|70.23|25.88|true|quinn young|2013-03-01 09:11:58.703108|22.72|quiet hour +96|350|65639|4294967505|85.35|20.58|true|calvin johnson|2013-03-01 09:11:58.703267|41.13|geology +89|356|65637|4294967435|17.40|22.36|false|yuri brown|2013-03-01 09:11:58.703235|59.46|opthamology +43|441|65641|4294967483|98.02|49.56|false|priscilla allen|2013-03-01 09:11:58.703161|27.52|industrial engineering +87|431|65728|4294967412|68.66|46.60|false|victor carson|2013-03-01 09:11:58.703242|80.77|yard duty +3|440|65781|4294967439|20.27|33.89|false|calvin carson|2013-03-01 09:11:58.703270|21.28|philosophy +41|509|65574|4294967470|39.42|32.40|false|irene carson|2013-03-01 09:11:58.703158|9.18|religion +56|348|65747|4294967370|29.40|38.78|false|bob quirinius|2013-03-01 09:11:58.703299|76.16|linguistics +120|352|65745|4294967433|77.94|43.03|false|irene ellison|2013-03-01 09:11:58.703108|83.81|biology +76|290|65546|4294967306|41.47|31.00|true|jessica davidson|2013-03-01 09:11:58.703265|31.93|kindergarten +22|411|65764|4294967466|54.20|27.15|false|ulysses white|2013-03-01 09:11:58.703285|59.46|philosophy +52|256|65741|4294967317|54.80|8.40|true|oscar steinbeck|2013-03-01 09:11:58.703132|33.47|zync studies +15|270|65781|4294967429|46.48|41.04|false|holly garcia|2013-03-01 09:11:58.703259|44.33|quiet hour +59|324|65659|4294967545|69.45|39.01|false|rachel thompson|2013-03-01 09:11:58.703090|99.86|study skills +62|287|65560|4294967307|8.41|0.54|false|xavier allen|2013-03-01 09:11:58.703239|64.94|topology +14|308|65759|4294967371|80.15|11.78|false|luke garcia|2013-03-01 09:11:58.703188|29.19|education +20|345|65635|4294967423|34.21|1.25|false|ulysses falkner|2013-03-01 09:11:58.703223|3.00|kindergarten +7|410|65553|4294967516|48.63|28.21|false|katie nixon|2013-03-01 09:11:58.703115|66.91|forestry +87|288|65753|4294967505|46.71|37.70|false|xavier nixon|2013-03-01 09:11:58.703257|1.75|values clariffication +87|482|65575|4294967387|91.24|33.71|false|zach underhill|2013-03-01 09:11:58.703276|54.29|biology +26|421|65672|4294967328|11.09|11.53|false|mike hernandez|2013-03-01 09:11:58.703079|89.09|chemistry +123|411|65554|4294967480|70.16|49.58|true|wendy xylophone|2013-03-01 09:11:58.703126|8.97|topology +18|319|65625|4294967466|27.90|45.55|false|gabriella van buren|2013-03-01 09:11:58.703256|11.16|forestry +44|341|65691|4294967550|49.94|19.99|false|irene thompson|2013-03-01 09:11:58.703239|72.58|education +4|424|65784|4294967516|81.44|14.22|false|gabriella xylophone|2013-03-01 09:11:58.703150|13.81|mathematics +119|405|65735|4294967526|77.39|34.87|true|nick zipper|2013-03-01 09:11:58.703123|89.24|religion +51|471|65764|4294967318|13.43|12.00|false|ethan garcia|2013-03-01 09:11:58.703221|49.44|joggying +47|392|65601|4294967547|60.26|27.80|false|sarah ovid|2013-03-01 09:11:58.703107|55.45|philosophy +15|308|65656|4294967362|58.45|9.34|false|sarah young|2013-03-01 09:11:58.703109|24.14|joggying +28|392|65726|4294967506|59.66|28.53|false|priscilla hernandez|2013-03-01 09:11:58.703316|4.85|nap time +55|472|65638|4294967413|15.34|41.44|false|mike brown|2013-03-01 09:11:58.703149|33.92|chemistry +122|292|65715|4294967471|72.77|22.30|false|victor ichabod|2013-03-01 09:11:58.703075|61.29|kindergarten +110|297|65744|4294967445|38.59|37.28|false|oscar polk|2013-03-01 09:11:58.703101|69.98|xylophone band +98|364|65720|4294967509|65.58|33.49|false|fred thompson|2013-03-01 09:11:58.703318|10.71|debate +66|476|65726|4294967354|15.80|26.36|true|xavier miller|2013-03-01 09:11:58.703154|0.66|geology +3|434|65675|4294967450|20.56|7.54|true|zach ellison|2013-03-01 09:11:58.703305|6.00|religion +100|435|65609|4294967545|50.49|7.44|false|ethan steinbeck|2013-03-01 09:11:58.703194|62.43|joggying +88|388|65661|4294967456|91.34|14.96|false|katie zipper|2013-03-01 09:11:58.703081|29.59|quiet hour +108|420|65647|4294967481|95.32|27.58|true|rachel king|2013-03-01 09:11:58.703139|13.96|biology +24|476|65666|4294967432|5.20|42.35|false|zach robinson|2013-03-01 09:11:58.703083|96.22|mathematics +101|302|65784|4294967505|78.03|1.33|false|quinn steinbeck|2013-03-01 09:11:58.703141|25.71|values clariffication +8|462|65556|4294967512|25.16|19.11|false|mike miller|2013-03-01 09:11:58.703115|82.53|nap time +89|396|65635|4294967379|31.70|48.06|true|holly hernandez|2013-03-01 09:11:58.703284|92.74|undecided +46|460|65631|4294967327|76.06|4.27|true|holly underhill|2013-03-01 09:11:58.703289|21.59|philosophy +100|345|65601|4294967349|81.98|41.46|true|priscilla underhill|2013-03-01 09:11:58.703210|70.75|philosophy +6|427|65540|4294967502|53.54|26.12|false|calvin young|2013-03-01 09:11:58.703110|98.41|values clariffication +15|441|65752|4294967535|56.94|47.78|false|rachel garcia|2013-03-01 09:11:58.703070|44.38|joggying +94|256|65558|4294967506|71.32|27.07|false|alice robinson|2013-03-01 09:11:58.703135|27.67|religion +17|447|65603|4294967429|70.12|29.68|true|ethan miller|2013-03-01 09:11:58.703100|42.88|chemistry +99|320|65709|4294967435|60.39|19.22|false|fred carson|2013-03-01 09:11:58.703260|86.75|undecided +29|511|65724|4294967313|84.06|9.60|false|calvin nixon|2013-03-01 09:11:58.703308|20.85|opthamology +32|290|65698|4294967446|14.05|49.94|true|holly miller|2013-03-01 09:11:58.703214|15.87|industrial engineering +120|368|65788|4294967441|90.15|21.18|true|mike polk|2013-03-01 09:11:58.703150|21.66|xylophone band +79|318|65752|4294967432|74.62|14.22|true|quinn zipper|2013-03-01 09:11:58.703207|82.75|geology +78|459|65779|4294967526|41.15|28.60|false|luke zipper|2013-03-01 09:11:58.703127|37.74|industrial engineering +77|442|65645|4294967505|81.66|29.02|false|ethan ichabod|2013-03-01 09:11:58.703096|42.02|debate +3|369|65642|4294967505|45.26|12.90|true|irene young|2013-03-01 09:11:58.703180|75.25|american history +81|508|65545|4294967503|47.35|6.26|true|alice carson|2013-03-01 09:11:58.703268|98.44|undecided +65|388|65626|4294967340|74.51|9.84|true|zach king|2013-03-01 09:11:58.703168|21.11|topology +56|456|65650|4294967393|92.24|24.26|false|bob laertes|2013-03-01 09:11:58.703101|9.94|nap time +-2|395|65778|4294967460|56.76|37.52|true|luke xylophone|2013-03-01 09:11:58.703071|97.42|chemistry +56|300|65595|4294967392|64.37|3.17|true|calvin zipper|2013-03-01 09:11:58.703259|39.04|xylophone band +87|314|65674|4294967499|8.21|21.99|false|mike young|2013-03-01 09:11:58.703234|72.64|topology +59|418|65626|4294967473|75.88|37.25|false|tom ichabod|2013-03-01 09:11:58.703222|81.58|study skills +124|338|65742|4294967456|23.53|16.75|true|mike ellison|2013-03-01 09:11:58.703082|79.57|yard duty +94|387|65543|4294967316|24.79|27.31|true|irene polk|2013-03-01 09:11:58.703194|68.50|mathematics +26|401|65700|4294967407|72.10|16.84|true|jessica xylophone|2013-03-01 09:11:58.703211|68.15|debate +45|415|65590|4294967357|24.63|38.08|true|irene carson|2013-03-01 09:11:58.703313|50.13|biology +103|400|65624|4294967377|52.82|40.10|true|calvin king|2013-03-01 09:11:58.703291|95.83|industrial engineering +35|289|65643|4294967390|41.80|30.11|true|jessica xylophone|2013-03-01 09:11:58.703089|33.12|debate +69|326|65704|4294967449|35.24|37.10|false|luke robinson|2013-03-01 09:11:58.703308|29.79|zync studies +113|364|65709|4294967548|39.26|31.84|true|rachel laertes|2013-03-01 09:11:58.703286|47.70|american history +94|370|65631|4294967306|61.99|47.99|false|david underhill|2013-03-01 09:11:58.703109|3.68|history +122|398|65693|4294967445|46.28|26.30|true|luke van buren|2013-03-01 09:11:58.703206|77.30|forestry +83|428|65566|4294967522|7.16|47.46|false|gabriella nixon|2013-03-01 09:11:58.703098|37.12|joggying +82|271|65710|4294967437|16.20|27.84|true|ulysses johnson|2013-03-01 09:11:58.703194|83.89|history +113|284|65774|4294967445|75.88|10.17|false|xavier robinson|2013-03-01 09:11:58.703169|88.05|linguistics +39|419|65721|4294967489|99.21|16.98|false|xavier underhill|2013-03-01 09:11:58.703121|10.44|geology +8|270|65575|4294967396|93.77|36.42|true|yuri thompson|2013-03-01 09:11:58.703217|67.41|wind surfing +7|259|65544|4294967333|43.12|37.09|false|irene polk|2013-03-01 09:11:58.703117|36.99|geology +17|389|65577|4294967536|69.41|3.35|false|tom quirinius|2013-03-01 09:11:58.703281|1.52|geology +86|434|65572|4294967297|43.73|33.56|true|katie white|2013-03-01 09:11:58.703130|8.86|topology +1|465|65571|4294967446|20.17|48.74|true|gabriella white|2013-03-01 09:11:58.703169|80.15|debate +117|481|65651|4294967434|96.41|36.20|true|rachel young|2013-03-01 09:11:58.703095|43.93|history +31|450|65579|4294967386|29.86|4.20|false|ulysses robinson|2013-03-01 09:11:58.703207|78.35|joggying +94|437|65779|4294967548|52.62|46.79|true|victor ovid|2013-03-01 09:11:58.703163|61.89|biology +22|443|65673|4294967299|57.50|21.12|false|oscar laertes|2013-03-01 09:11:58.703088|86.28|education +45|394|65773|4294967468|58.95|48.27|false|zach allen|2013-03-01 09:11:58.703153|60.86|xylophone band +14|285|65542|4294967395|38.74|26.38|true|victor steinbeck|2013-03-01 09:11:58.703149|25.14|debate +32|356|65729|4294967323|52.21|47.71|true|irene hernandez|2013-03-01 09:11:58.703127|95.89|chemistry +83|459|65711|4294967450|15.77|5.97|false|gabriella thompson|2013-03-01 09:11:58.703270|67.24|industrial engineering +12|454|65649|4294967468|1.56|3.77|true|holly polk|2013-03-01 09:11:58.703289|90.53|linguistics +82|416|65619|4294967496|97.50|42.89|true|rachel zipper|2013-03-01 09:11:58.703176|15.82|wind surfing +8|385|65702|4294967377|28.45|19.92|false|nick allen|2013-03-01 09:11:58.703280|35.42|opthamology +15|364|65675|4294967460|90.03|30.72|true|david laertes|2013-03-01 09:11:58.703313|34.04|forestry +50|454|65659|4294967364|96.43|7.12|true|calvin underhill|2013-03-01 09:11:58.703079|98.15|religion +96|505|65780|4294967362|47.04|47.38|false|victor nixon|2013-03-01 09:11:58.703191|43.65|industrial engineering +113|431|65630|4294967495|46.08|34.97|false|victor ellison|2013-03-01 09:11:58.703070|85.40|chemistry +38|268|65709|4294967425|71.75|2.67|true|david young|2013-03-01 09:11:58.703269|41.68|biology +-2|262|65741|4294967546|55.95|46.97|true|yuri laertes|2013-03-01 09:11:58.703201|62.11|wind surfing +29|463|65776|4294967451|55.36|20.69|true|oscar xylophone|2013-03-01 09:11:58.703314|8.08|yard duty +43|404|65712|4294967472|36.68|15.20|false|quinn ichabod|2013-03-01 09:11:58.703303|43.26|values clariffication +37|338|65545|4294967482|62.70|31.92|true|alice ichabod|2013-03-01 09:11:58.703114|99.08|study skills +-2|371|65642|4294967444|15.26|0.84|false|tom van buren|2013-03-01 09:11:58.703291|31.06|chemistry +82|503|65683|4294967460|73.06|31.31|false|jessica garcia|2013-03-01 09:11:58.703082|39.09|biology +25|324|65721|4294967307|36.53|20.89|true|calvin ichabod|2013-03-01 09:11:58.703298|75.11|chemistry +45|377|65605|4294967322|53.27|7.12|true|priscilla ovid|2013-03-01 09:11:58.703093|39.81|geology +64|475|65738|4294967454|1.90|47.07|false|ulysses ovid|2013-03-01 09:11:58.703154|83.94|mathematics +58|299|65585|4294967540|81.41|10.89|true|victor van buren|2013-03-01 09:11:58.703070|86.85|history +124|276|65706|4294967352|80.15|30.99|false|victor young|2013-03-01 09:11:58.703139|14.91|quiet hour +6|402|65696|4294967500|52.52|21.78|true|quinn thompson|2013-03-01 09:11:58.703160|43.82|wind surfing +65|257|65711|4294967307|60.88|9.86|false|rachel robinson|2013-03-01 09:11:58.703171|14.21|chemistry +10|278|65602|4294967414|82.34|8.72|false|jessica quirinius|2013-03-01 09:11:58.703077|29.30|industrial engineering +16|335|65547|4294967419|80.31|49.40|false|nick johnson|2013-03-01 09:11:58.703310|70.54|zync studies +7|436|65715|4294967482|81.98|10.83|true|nick underhill|2013-03-01 09:11:58.703163|25.36|geology +12|494|65566|4294967369|28.59|25.29|false|sarah garcia|2013-03-01 09:11:58.703107|85.09|wind surfing +62|502|65707|4294967347|5.22|41.12|true|zach laertes|2013-03-01 09:11:58.703188|75.56|history +81|398|65707|4294967541|50.37|30.98|false|sarah young|2013-03-01 09:11:58.703263|45.86|nap time +122|340|65684|4294967463|70.18|20.12|false|nick zipper|2013-03-01 09:11:58.703158|27.47|topology +32|314|65636|4294967354|73.88|29.09|false|irene ichabod|2013-03-01 09:11:58.703212|26.93|study skills +81|275|65641|4294967527|29.91|14.69|false|holly steinbeck|2013-03-01 09:11:58.703282|67.70|chemistry +88|397|65725|4294967453|49.94|34.26|true|luke polk|2013-03-01 09:11:58.703110|12.73|nap time +98|344|65570|4294967478|10.37|43.65|false|victor young|2013-03-01 09:11:58.703323|88.11|religion +103|389|65594|4294967541|49.66|29.09|true|rachel underhill|2013-03-01 09:11:58.703137|73.56|topology +103|464|65539|4294967367|50.68|14.46|true|katie xylophone|2013-03-01 09:11:58.703287|8.85|philosophy +88|453|65750|4294967365|59.51|20.04|true|priscilla van buren|2013-03-01 09:11:58.703187|91.90|nap time +12|401|65665|4294967410|68.11|26.14|true|zach miller|2013-03-01 09:11:58.703239|68.24|biology +13|334|65622|4294967372|23.39|37.98|true|tom quirinius|2013-03-01 09:11:58.703175|58.78|opthamology +80|289|65633|4294967305|30.64|10.76|false|nick ellison|2013-03-01 09:11:58.703242|70.08|nap time +122|309|65722|4294967481|15.14|27.91|true|fred hernandez|2013-03-01 09:11:58.703263|26.80|debate +12|438|65548|4294967407|51.15|40.72|false|rachel brown|2013-03-01 09:11:58.703205|57.12|yard duty +72|468|65549|4294967399|79.77|4.58|false|xavier thompson|2013-03-01 09:11:58.703221|19.68|education +49|399|65787|4294967308|21.33|0.97|true|zach allen|2013-03-01 09:11:58.703283|54.09|mathematics +82|267|65606|4294967415|58.82|24.82|false|oscar thompson|2013-03-01 09:11:58.703185|59.26|opthamology +101|279|65610|4294967330|78.63|14.39|false|irene king|2013-03-01 09:11:58.703138|94.99|values clariffication +33|461|65748|4294967298|14.18|44.02|false|luke robinson|2013-03-01 09:11:58.703082|78.22|chemistry +113|313|65666|4294967359|47.01|14.86|true|oscar miller|2013-03-01 09:11:58.703242|18.25|forestry +112|294|65745|4294967456|85.63|32.70|false|nick van buren|2013-03-01 09:11:58.703257|41.08|study skills +23|391|65697|4294967435|11.61|36.09|true|fred ellison|2013-03-01 09:11:58.703127|78.36|american history +13|508|65720|4294967311|59.87|30.06|true|irene white|2013-03-01 09:11:58.703223|97.56|education +124|355|65637|4294967326|78.28|24.78|false|ethan nixon|2013-03-01 09:11:58.703142|84.78|industrial engineering +21|418|65743|4294967315|93.32|4.33|true|zach young|2013-03-01 09:11:58.703232|40.39|biology +50|332|65750|4294967500|32.21|44.89|false|katie xylophone|2013-03-01 09:11:58.703115|38.31|quiet hour +36|267|65580|4294967503|17.68|25.04|false|zach steinbeck|2013-03-01 09:11:58.703319|34.54|mathematics +57|387|65758|4294967521|44.64|14.32|false|tom thompson|2013-03-01 09:11:58.703112|90.78|yard duty +53|477|65707|4294967515|65.28|29.47|true|xavier laertes|2013-03-01 09:11:58.703169|27.01|chemistry +98|275|65707|4294967495|83.25|8.20|false|oscar nixon|2013-03-01 09:11:58.703093|6.76|history +9|301|65717|4294967474|81.34|35.65|false|bob miller|2013-03-01 09:11:58.703206|34.95|study skills +4|487|65655|4294967479|60.66|21.72|false|priscilla falkner|2013-03-01 09:11:58.703127|55.94|yard duty +30|428|65544|4294967383|42.82|15.35|true|wendy davidson|2013-03-01 09:11:58.703143|45.43|philosophy +65|290|65542|4294967389|5.77|12.59|false|xavier brown|2013-03-01 09:11:58.703185|11.15|biology +15|355|65630|4294967414|76.78|45.02|false|nick davidson|2013-03-01 09:11:58.703207|42.93|industrial engineering +75|346|65556|4294967466|65.24|36.77|true|mike johnson|2013-03-01 09:11:58.703256|70.53|biology +54|438|65759|4294967506|67.49|48.58|false|oscar thompson|2013-03-01 09:11:58.703134|9.76|philosophy +63|348|65669|4294967536|36.30|2.02|false|quinn ovid|2013-03-01 09:11:58.703108|62.77|xylophone band +106|374|65784|4294967401|58.98|34.17|true|fred miller|2013-03-01 09:11:58.703182|21.11|quiet hour +97|371|65547|4294967302|98.70|42.32|true|quinn brown|2013-03-01 09:11:58.703157|31.48|mathematics +87|284|65597|4294967397|99.34|7.70|false|gabriella nixon|2013-03-01 09:11:58.703263|79.41|philosophy +25|457|65657|4294967345|51.08|11.07|true|david ichabod|2013-03-01 09:11:58.703324|28.78|education +69|496|65673|4294967490|34.44|18.62|true|holly garcia|2013-03-01 09:11:58.703300|2.82|opthamology +91|353|65746|4294967390|67.24|4.16|true|zach ichabod|2013-03-01 09:11:58.703190|79.29|joggying +117|499|65677|4294967395|42.07|8.23|true|oscar underhill|2013-03-01 09:11:58.703303|99.50|zync studies +118|436|65674|4294967414|69.43|3.04|false|xavier robinson|2013-03-01 09:11:58.703126|3.39|zync studies +54|444|65666|4294967406|2.82|37.99|true|nick young|2013-03-01 09:11:58.703185|23.03|quiet hour +90|351|65600|4294967435|69.37|25.50|true|jessica miller|2013-03-01 09:11:58.703170|13.47|religion +118|307|65701|4294967460|30.76|24.51|false|katie falkner|2013-03-01 09:11:58.703272|15.16|debate +93|407|65686|4294967304|53.20|35.46|false|david zipper|2013-03-01 09:11:58.703255|18.28|quiet hour +66|292|65670|4294967502|36.15|49.31|false|victor polk|2013-03-01 09:11:58.703079|99.37|debate +77|317|65657|4294967355|83.98|44.72|false|wendy johnson|2013-03-01 09:11:58.703071|89.25|zync studies +5|274|65561|4294967403|63.25|45.27|true|victor underhill|2013-03-01 09:11:58.703140|93.41|study skills +104|345|65677|4294967339|64.53|37.90|false|alice king|2013-03-01 09:11:58.703276|57.56|topology +26|348|65770|4294967357|38.71|37.16|true|luke young|2013-03-01 09:11:58.703292|59.77|chemistry +70|395|65788|4294967541|44.58|17.63|true|gabriella garcia|2013-03-01 09:11:58.703102|23.37|opthamology +23|265|65655|4294967498|80.82|32.19|false|oscar allen|2013-03-01 09:11:58.703091|11.91|xylophone band +31|429|65617|4294967520|69.05|49.63|false|nick ellison|2013-03-01 09:11:58.703225|67.46|study skills +1|466|65593|4294967388|56.26|16.09|false|gabriella johnson|2013-03-01 09:11:58.703142|38.20|quiet hour +69|352|65637|4294967459|49.42|37.94|true|wendy polk|2013-03-01 09:11:58.703241|61.50|joggying +52|484|65656|4294967478|29.18|13.28|true|david van buren|2013-03-01 09:11:58.703251|98.35|opthamology +19|385|65756|4294967443|96.50|26.94|false|priscilla hernandez|2013-03-01 09:11:58.703230|76.07|values clariffication +84|498|65625|4294967425|27.78|13.58|false|zach ovid|2013-03-01 09:11:58.703210|85.01|debate +57|460|65759|4294967551|81.97|12.16|false|luke allen|2013-03-01 09:11:58.703113|85.27|joggying +97|414|65614|4294967331|55.54|5.45|false|wendy steinbeck|2013-03-01 09:11:58.703177|29.57|values clariffication +-2|467|65601|4294967345|96.04|42.73|false|zach johnson|2013-03-01 09:11:58.703321|63.24|quiet hour +84|279|65569|4294967484|98.09|33.93|true|ethan king|2013-03-01 09:11:58.703296|63.29|biology +104|365|65627|4294967433|41.51|17.59|false|priscilla johnson|2013-03-01 09:11:58.703182|61.44|joggying +29|500|65679|4294967538|22.68|23.13|true|priscilla underhill|2013-03-01 09:11:58.703225|24.33|kindergarten +44|356|65597|4294967399|48.20|13.32|true|rachel miller|2013-03-01 09:11:58.703119|45.96|mathematics +114|461|65664|4294967385|34.78|16.35|true|ethan steinbeck|2013-03-01 09:11:58.703076|27.83|values clariffication +67|432|65545|4294967500|8.30|34.74|false|calvin zipper|2013-03-01 09:11:58.703165|61.04|yard duty +114|492|65642|4294967421|45.84|42.14|false|holly thompson|2013-03-01 09:11:58.703079|88.78|mathematics +56|267|65650|4294967299|20.75|49.80|true|quinn xylophone|2013-03-01 09:11:58.703274|69.47|nap time +122|434|65675|4294967498|17.82|12.74|false|victor falkner|2013-03-01 09:11:58.703105|8.09|biology +5|444|65690|4294967330|93.08|7.70|false|luke johnson|2013-03-01 09:11:58.703097|1.19|topology +92|434|65580|4294967298|83.31|0.88|true|xavier falkner|2013-03-01 09:11:58.703295|98.56|nap time +34|442|65660|4294967467|80.43|44.07|true|jessica nixon|2013-03-01 09:11:58.703071|19.26|religion +38|392|65789|4294967399|95.11|23.23|true|david carson|2013-03-01 09:11:58.703320|41.69|values clariffication +122|278|65624|4294967368|45.39|48.87|true|mike johnson|2013-03-01 09:11:58.703109|87.38|religion +75|400|65595|4294967373|36.56|32.92|true|jessica garcia|2013-03-01 09:11:58.703218|26.57|religion +4|351|65667|4294967407|97.75|11.85|true|wendy hernandez|2013-03-01 09:11:58.703112|94.23|yard duty +39|327|65549|4294967410|22.36|40.10|true|rachel nixon|2013-03-01 09:11:58.703199|29.90|values clariffication +115|363|65558|4294967383|20.36|16.58|true|holly quirinius|2013-03-01 09:11:58.703106|58.86|education +43|490|65550|4294967372|40.80|38.04|true|quinn white|2013-03-01 09:11:58.703282|50.30|biology +56|315|65779|4294967530|6.98|27.78|true|david miller|2013-03-01 09:11:58.703288|6.29|opthamology +118|337|65717|4294967351|30.80|13.23|true|sarah johnson|2013-03-01 09:11:58.703179|11.61|topology +23|291|65663|4294967308|28.64|7.00|true|rachel garcia|2013-03-01 09:11:58.703144|64.95|kindergarten +16|369|65738|4294967427|51.80|10.90|false|irene quirinius|2013-03-01 09:11:58.703146|97.81|zync studies +99|359|65664|4294967541|8.39|0.01|false|calvin miller|2013-03-01 09:11:58.703262|96.54|industrial engineering +66|261|65704|4294967380|90.86|12.50|true|irene white|2013-03-01 09:11:58.703147|77.32|chemistry +86|405|65619|4294967407|18.99|13.84|true|wendy laertes|2013-03-01 09:11:58.703274|24.92|forestry +37|261|65672|4294967413|59.82|9.27|false|victor zipper|2013-03-01 09:11:58.703309|16.32|xylophone band +3|396|65695|4294967319|24.83|44.11|true|oscar white|2013-03-01 09:11:58.703248|83.36|mathematics +121|343|65709|4294967407|65.31|25.38|false|wendy zipper|2013-03-01 09:11:58.703137|69.52|joggying +92|274|65669|4294967351|68.15|22.23|false|luke van buren|2013-03-01 09:11:58.703249|49.34|education +61|500|65652|4294967437|86.58|6.11|true|ulysses ovid|2013-03-01 09:11:58.703282|72.31|quiet hour +50|269|65550|4294967362|49.83|29.08|false|xavier thompson|2013-03-01 09:11:58.703207|82.57|joggying +26|311|65571|4294967530|70.21|40.26|true|wendy underhill|2013-03-01 09:11:58.703177|41.04|zync studies +28|279|65637|4294967431|77.71|15.41|true|irene white|2013-03-01 09:11:58.703310|25.38|opthamology +107|396|65592|4294967309|46.03|32.39|false|ulysses white|2013-03-01 09:11:58.703091|17.20|wind surfing +-2|303|65588|4294967431|49.32|30.03|false|fred young|2013-03-01 09:11:58.703240|64.94|linguistics +116|475|65737|4294967384|87.04|28.54|false|oscar robinson|2013-03-01 09:11:58.703206|67.85|debate +13|345|65577|4294967514|2.39|35.42|false|sarah johnson|2013-03-01 09:11:58.703243|94.06|american history +50|310|65658|4294967312|74.84|29.87|false|wendy hernandez|2013-03-01 09:11:58.703115|77.63|yard duty +61|510|65613|4294967494|3.98|35.29|false|fred young|2013-03-01 09:11:58.703176|41.27|philosophy +86|320|65571|4294967497|94.26|1.12|false|oscar carson|2013-03-01 09:11:58.703212|37.74|topology +99|378|65690|4294967363|59.56|1.58|false|sarah underhill|2013-03-01 09:11:58.703090|71.99|education +-3|384|65613|4294967470|63.49|45.85|false|holly steinbeck|2013-03-01 09:11:58.703242|54.18|chemistry +10|488|65726|4294967441|4.49|17.81|true|jessica white|2013-03-01 09:11:58.703233|90.27|american history +8|373|65550|4294967345|11.84|27.43|false|luke hernandez|2013-03-01 09:11:58.703074|61.36|chemistry +52|423|65708|4294967379|94.45|2.87|true|victor king|2013-03-01 09:11:58.703100|95.43|geology +56|473|65763|4294967496|43.73|14.05|false|victor king|2013-03-01 09:11:58.703163|37.64|mathematics +13|355|65548|4294967482|36.62|43.24|true|victor xylophone|2013-03-01 09:11:58.703107|69.46|linguistics +83|386|65641|4294967409|76.93|17.42|true|zach nixon|2013-03-01 09:11:58.703200|77.22|religion +8|358|65604|4294967362|0.71|43.03|false|zach davidson|2013-03-01 09:11:58.703107|89.40|zync studies +103|492|65767|4294967305|13.39|28.56|true|tom quirinius|2013-03-01 09:11:58.703105|76.81|values clariffication +-3|280|65597|4294967377|18.44|49.80|true|alice falkner|2013-03-01 09:11:58.703304|74.42|zync studies +55|508|65742|4294967513|94.21|29.79|true|wendy ellison|2013-03-01 09:11:58.703222|47.03|opthamology +121|321|65663|4294967353|20.93|29.50|false|sarah nixon|2013-03-01 09:11:58.703126|89.49|forestry +90|457|65594|4294967333|4.28|30.19|true|gabriella hernandez|2013-03-01 09:11:58.703120|56.70|study skills +55|330|65720|4294967551|76.57|47.23|true|david laertes|2013-03-01 09:11:58.703104|36.41|forestry +66|420|65746|4294967335|29.11|31.90|true|irene steinbeck|2013-03-01 09:11:58.703325|62.79|religion +51|317|65578|4294967507|76.79|9.93|true|ethan miller|2013-03-01 09:11:58.703196|69.26|american history +36|337|65552|4294967401|24.13|37.02|true|calvin polk|2013-03-01 09:11:58.703169|71.27|quiet hour +30|408|65742|4294967303|73.81|44.95|false|ethan nixon|2013-03-01 09:11:58.703205|73.86|history +3|402|65613|4294967367|37.40|49.57|true|zach nixon|2013-03-01 09:11:58.703297|12.36|kindergarten +62|300|65622|4294967353|3.30|22.53|false|xavier underhill|2013-03-01 09:11:58.703105|50.79|education +51|412|65690|4294967316|28.38|13.52|false|victor king|2013-03-01 09:11:58.703262|99.60|biology +103|325|65688|4294967390|24.97|31.77|true|sarah polk|2013-03-01 09:11:58.703221|37.79|values clariffication +51|463|65670|4294967406|49.42|7.18|false|oscar laertes|2013-03-01 09:11:58.703202|74.73|xylophone band +5|328|65566|4294967400|86.90|38.91|false|yuri quirinius|2013-03-01 09:11:58.703221|1.25|opthamology +94|330|65727|4294967331|2.68|34.99|true|irene xylophone|2013-03-01 09:11:58.703236|19.44|study skills +51|394|65747|4294967448|63.32|42.11|true|ulysses van buren|2013-03-01 09:11:58.703221|16.39|joggying +94|357|65634|4294967377|52.33|21.39|false|zach polk|2013-03-01 09:11:58.703248|41.68|education +90|446|65568|4294967429|31.92|40.63|true|irene johnson|2013-03-01 09:11:58.703100|43.18|american history +23|493|65682|4294967412|62.04|16.85|true|fred underhill|2013-03-01 09:11:58.703316|17.23|undecided +107|460|65740|4294967466|44.54|6.34|false|priscilla falkner|2013-03-01 09:11:58.703200|89.16|geology +94|326|65758|4294967426|38.77|23.83|true|gabriella brown|2013-03-01 09:11:58.703253|87.12|nap time +53|510|65591|4294967380|5.10|33.20|false|rachel quirinius|2013-03-01 09:11:58.703318|31.12|kindergarten +90|263|65661|4294967414|97.32|40.37|false|victor robinson|2013-03-01 09:11:58.703133|78.55|study skills +118|412|65704|4294967475|0.37|45.00|true|calvin ovid|2013-03-01 09:11:58.703142|45.58|mathematics +54|397|65663|4294967367|15.28|1.05|true|irene davidson|2013-03-01 09:11:58.703258|30.80|study skills +14|256|65640|4294967349|32.64|13.67|false|sarah white|2013-03-01 09:11:58.703229|72.55|study skills +50|328|65633|4294967491|25.09|19.38|false|ethan ellison|2013-03-01 09:11:58.703303|67.87|industrial engineering +-1|467|65672|4294967370|79.05|29.47|true|jessica brown|2013-03-01 09:11:58.703071|8.86|opthamology +96|502|65659|4294967405|88.76|2.12|true|katie ovid|2013-03-01 09:11:58.703281|67.71|xylophone band +102|481|65585|4294967355|49.56|20.67|true|calvin young|2013-03-01 09:11:58.703297|95.61|joggying +69|263|65555|4294967392|21.66|14.63|true|rachel ichabod|2013-03-01 09:11:58.703257|31.71|quiet hour +38|337|65618|4294967316|34.73|44.07|true|jessica davidson|2013-03-01 09:11:58.703310|29.67|xylophone band +85|413|65724|4294967523|74.50|27.18|true|katie white|2013-03-01 09:11:58.703096|59.77|yard duty +28|361|65679|4294967348|25.88|7.89|true|fred carson|2013-03-01 09:11:58.703299|91.51|values clariffication +114|285|65791|4294967419|47.08|15.32|false|fred garcia|2013-03-01 09:11:58.703136|84.23|debate +17|276|65572|4294967443|52.79|46.99|false|quinn allen|2013-03-01 09:11:58.703202|40.40|chemistry +45|349|65560|4294967341|93.91|22.33|true|katie nixon|2013-03-01 09:11:58.703232|11.15|linguistics +96|379|65650|4294967502|19.22|44.93|true|sarah xylophone|2013-03-01 09:11:58.703121|84.37|xylophone band +48|418|65747|4294967523|90.05|19.46|false|fred polk|2013-03-01 09:11:58.703323|58.99|mathematics +116|440|65725|4294967403|23.36|40.03|true|alice ichabod|2013-03-01 09:11:58.703249|84.90|history +84|339|65611|4294967548|55.58|19.06|true|zach white|2013-03-01 09:11:58.703098|73.91|joggying +31|371|65659|4294967331|28.46|11.60|true|oscar garcia|2013-03-01 09:11:58.703207|6.03|undecided +1|312|65666|4294967451|89.58|7.49|true|victor underhill|2013-03-01 09:11:58.703156|36.22|forestry +73|410|65634|4294967450|0.08|19.63|true|nick zipper|2013-03-01 09:11:58.703151|64.22|biology +120|364|65678|4294967398|37.74|18.77|false|katie steinbeck|2013-03-01 09:11:58.703186|55.26|quiet hour +-3|494|65589|4294967369|48.09|14.40|false|jessica johnson|2013-03-01 09:11:58.703319|78.85|nap time +106|471|65786|4294967351|14.23|15.96|false|zach miller|2013-03-01 09:11:58.703255|79.26|quiet hour +18|291|65788|4294967392|6.19|44.14|false|jessica zipper|2013-03-01 09:11:58.703145|63.29|study skills +107|509|65770|4294967453|15.70|35.51|false|zach garcia|2013-03-01 09:11:58.703279|23.01|opthamology +120|396|65569|4294967500|47.81|16.43|true|bob steinbeck|2013-03-01 09:11:58.703200|34.01|biology +118|332|65626|4294967529|4.82|30.69|true|ethan ovid|2013-03-01 09:11:58.703174|39.37|biology +17|258|65669|4294967503|75.01|14.80|false|priscilla underhill|2013-03-01 09:11:58.703300|49.02|zync studies +89|456|65676|4294967335|2.59|22.40|true|yuri steinbeck|2013-03-01 09:11:58.703242|30.67|topology +5|415|65621|4294967334|27.06|34.41|false|zach robinson|2013-03-01 09:11:58.703148|18.10|yard duty +2|487|65691|4294967411|11.43|31.08|false|victor johnson|2013-03-01 09:11:58.703283|12.78|biology +33|338|65543|4294967355|37.12|30.33|true|priscilla miller|2013-03-01 09:11:58.703171|54.90|forestry +86|505|65734|4294967517|96.63|20.01|false|jessica quirinius|2013-03-01 09:11:58.703308|8.73|debate +8|476|65652|4294967432|18.94|19.03|false|nick davidson|2013-03-01 09:11:58.703111|65.85|joggying +21|507|65750|4294967388|80.62|30.07|false|ethan laertes|2013-03-01 09:11:58.703218|59.00|industrial engineering +83|499|65700|4294967309|91.06|37.70|true|victor thompson|2013-03-01 09:11:58.703254|92.41|values clariffication +93|497|65669|4294967365|53.43|27.06|true|david brown|2013-03-01 09:11:58.703150|93.52|values clariffication +52|378|65606|4294967456|43.36|47.32|true|calvin quirinius|2013-03-01 09:11:58.703295|41.52|undecided +55|309|65710|4294967329|73.65|16.52|true|gabriella polk|2013-03-01 09:11:58.703134|14.42|opthamology +55|409|65674|4294967468|19.97|13.96|true|jessica white|2013-03-01 09:11:58.703096|54.49|xylophone band +9|492|65621|4294967311|26.65|6.81|false|ethan van buren|2013-03-01 09:11:58.703176|90.22|history +118|356|65599|4294967337|33.83|18.31|true|victor johnson|2013-03-01 09:11:58.703243|76.93|joggying +90|494|65675|4294967359|21.63|19.29|false|katie hernandez|2013-03-01 09:11:58.703294|60.05|american history +62|338|65732|4294967422|40.96|26.59|false|rachel miller|2013-03-01 09:11:58.703264|1.16|history +88|478|65621|4294967485|45.31|4.37|false|ethan nixon|2013-03-01 09:11:58.703103|91.45|biology +51|336|65705|4294967402|87.56|5.61|true|ulysses falkner|2013-03-01 09:11:58.703258|58.63|xylophone band +3|499|65702|4294967371|48.39|24.99|false|ethan quirinius|2013-03-01 09:11:58.703233|27.59|kindergarten +78|387|65586|4294967349|99.61|32.43|true|sarah laertes|2013-03-01 09:11:58.703323|42.74|history +8|278|65697|4294967493|21.21|13.35|true|luke laertes|2013-03-01 09:11:58.703183|97.94|mathematics +72|507|65627|4294967536|13.53|33.19|false|priscilla johnson|2013-03-01 09:11:58.703210|42.09|topology +-2|373|65548|4294967423|16.98|43.60|true|alice nixon|2013-03-01 09:11:58.703321|52.61|debate +70|429|65598|4294967528|46.53|37.45|true|mike young|2013-03-01 09:11:58.703183|60.75|chemistry +37|373|65559|4294967476|4.77|42.24|true|ethan xylophone|2013-03-01 09:11:58.703260|12.76|yard duty +65|460|65549|4294967347|78.11|3.03|false|jessica van buren|2013-03-01 09:11:58.703263|95.69|industrial engineering +21|276|65575|4294967404|80.65|28.87|false|oscar falkner|2013-03-01 09:11:58.703201|8.95|history +69|357|65789|4294967475|58.11|2.56|true|katie falkner|2013-03-01 09:11:58.703138|31.78|philosophy +7|344|65673|4294967328|32.54|43.65|true|tom xylophone|2013-03-01 09:11:58.703097|5.65|debate +10|414|65749|4294967538|72.77|15.85|true|irene davidson|2013-03-01 09:11:58.703191|1.99|undecided +80|306|65648|4294967471|44.17|22.83|true|tom falkner|2013-03-01 09:11:58.703102|83.26|biology +36|352|65759|4294967509|88.65|42.76|false|david ellison|2013-03-01 09:11:58.703181|44.51|topology +36|374|65714|4294967417|35.04|32.03|false|nick steinbeck|2013-03-01 09:11:58.703149|68.80|joggying +67|384|65614|4294967392|53.58|38.69|false|luke laertes|2013-03-01 09:11:58.703173|69.38|religion +113|268|65610|4294967523|66.30|6.21|true|xavier white|2013-03-01 09:11:58.703150|73.73|quiet hour +77|372|65557|4294967434|65.94|29.62|true|zach polk|2013-03-01 09:11:58.703208|25.27|zync studies +46|350|65665|4294967537|97.92|27.92|false|jessica van buren|2013-03-01 09:11:58.703175|10.69|undecided +60|411|65648|4294967436|27.12|35.45|true|priscilla thompson|2013-03-01 09:11:58.703134|31.49|american history +8|325|65594|4294967351|59.36|49.19|true|victor miller|2013-03-01 09:11:58.703236|21.93|history +17|499|65622|4294967326|67.30|19.63|true|tom brown|2013-03-01 09:11:58.703257|12.11|religion +40|328|65694|4294967332|95.07|29.33|true|gabriella underhill|2013-03-01 09:11:58.703130|59.49|religion +7|402|65571|4294967437|82.31|39.22|true|ulysses nixon|2013-03-01 09:11:58.703107|37.36|topology +113|468|65784|4294967368|66.70|19.16|true|holly ellison|2013-03-01 09:11:58.703278|11.78|debate +104|458|65707|4294967536|38.38|43.24|false|ulysses brown|2013-03-01 09:11:58.703241|42.93|xylophone band +39|370|65588|4294967341|36.92|23.31|true|victor falkner|2013-03-01 09:11:58.703088|72.33|history +90|279|65552|4294967382|92.83|7.47|true|zach carson|2013-03-01 09:11:58.703110|59.95|philosophy +108|396|65545|4294967357|23.91|15.03|false|alice hernandez|2013-03-01 09:11:58.703252|94.63|american history +2|304|65634|4294967388|41.64|2.18|true|nick quirinius|2013-03-01 09:11:58.703233|47.57|history +32|307|65624|4294967364|2.34|13.36|true|jessica nixon|2013-03-01 09:11:58.703175|72.16|study skills +123|376|65602|4294967377|39.78|13.93|false|jessica miller|2013-03-01 09:11:58.703099|75.80|zync studies +110|439|65756|4294967355|34.73|9.74|true|holly zipper|2013-03-01 09:11:58.703200|24.39|quiet hour +79|494|65627|4294967515|46.82|34.99|false|victor falkner|2013-03-01 09:11:58.703125|59.47|industrial engineering +75|380|65709|4294967346|47.43|0.72|true|oscar davidson|2013-03-01 09:11:58.703216|58.46|chemistry +42|362|65672|4294967311|29.12|43.39|true|wendy white|2013-03-01 09:11:58.703138|93.52|study skills +115|356|65702|4294967474|17.34|45.44|false|mike quirinius|2013-03-01 09:11:58.703244|11.80|chemistry +39|372|65606|4294967474|0.23|23.13|false|alice quirinius|2013-03-01 09:11:58.703093|86.69|mathematics +50|386|65552|4294967318|9.61|20.07|false|zach white|2013-03-01 09:11:58.703193|63.50|opthamology +6|341|65724|4294967482|3.50|17.93|true|mike ellison|2013-03-01 09:11:58.703201|33.89|forestry +88|369|65640|4294967324|2.12|38.05|false|sarah van buren|2013-03-01 09:11:58.703164|31.70|mathematics +117|335|65680|4294967312|35.31|22.79|true|yuri robinson|2013-03-01 09:11:58.703325|8.03|forestry +38|424|65643|4294967388|50.45|42.85|false|tom white|2013-03-01 09:11:58.703076|50.40|xylophone band +28|473|65619|4294967313|73.17|46.06|false|wendy king|2013-03-01 09:11:58.703140|89.31|wind surfing +51|373|65764|4294967328|86.52|10.73|true|sarah laertes|2013-03-01 09:11:58.703237|42.36|mathematics +93|437|65540|4294967418|11.26|19.69|false|jessica robinson|2013-03-01 09:11:58.703276|83.95|nap time +18|268|65544|4294967512|5.37|23.74|false|gabriella polk|2013-03-01 09:11:58.703241|74.73|undecided +93|362|65770|4294967318|15.47|48.28|false|yuri thompson|2013-03-01 09:11:58.703120|15.26|opthamology +105|264|65645|4294967363|78.08|37.82|true|xavier laertes|2013-03-01 09:11:58.703203|0.88|values clariffication +100|311|65652|4294967504|92.08|16.99|false|luke falkner|2013-03-01 09:11:58.703131|98.03|nap time +8|305|65772|4294967402|33.84|28.27|true|alice ovid|2013-03-01 09:11:58.703152|55.90|opthamology +119|287|65700|4294967494|17.60|25.41|false|nick quirinius|2013-03-01 09:11:58.703138|31.60|undecided +81|266|65746|4294967449|81.29|43.85|false|ulysses nixon|2013-03-01 09:11:58.703238|83.51|opthamology +106|456|65755|4294967387|2.94|28.66|true|xavier johnson|2013-03-01 09:11:58.703274|68.81|values clariffication +76|428|65580|4294967458|88.21|42.37|true|mike miller|2013-03-01 09:11:58.703226|30.00|wind surfing +123|291|65679|4294967411|62.23|48.22|true|holly underhill|2013-03-01 09:11:58.703077|36.38|industrial engineering +68|507|65744|4294967315|9.31|14.31|true|alice polk|2013-03-01 09:11:58.703156|45.05|religion +59|443|65757|4294967467|26.18|13.31|false|zach white|2013-03-01 09:11:58.703277|97.50|undecided +121|480|65584|4294967352|44.74|26.70|false|sarah steinbeck|2013-03-01 09:11:58.703081|80.45|xylophone band +23|348|65646|4294967453|80.57|9.81|false|bob white|2013-03-01 09:11:58.703322|59.93|religion +63|366|65739|4294967395|26.31|37.57|true|xavier carson|2013-03-01 09:11:58.703075|76.18|religion +51|343|65673|4294967348|23.87|45.90|true|sarah garcia|2013-03-01 09:11:58.703256|41.60|industrial engineering +68|282|65630|4294967390|64.97|41.58|true|mike ichabod|2013-03-01 09:11:58.703259|68.00|education +15|316|65687|4294967398|22.50|44.26|false|calvin laertes|2013-03-01 09:11:58.703174|11.93|forestry +57|308|65734|4294967445|51.29|9.70|true|ulysses brown|2013-03-01 09:11:58.703184|42.79|american history +16|291|65734|4294967533|99.18|37.44|true|fred nixon|2013-03-01 09:11:58.703088|29.86|industrial engineering +58|424|65706|4294967364|80.97|30.40|true|tom xylophone|2013-03-01 09:11:58.703244|6.64|xylophone band +118|466|65731|4294967548|18.42|27.83|false|wendy quirinius|2013-03-01 09:11:58.703083|99.21|wind surfing +123|465|65739|4294967504|79.31|46.70|false|ulysses zipper|2013-03-01 09:11:58.703163|16.23|kindergarten +44|280|65750|4294967406|33.57|24.89|false|ulysses robinson|2013-03-01 09:11:58.703229|58.67|chemistry +49|434|65779|4294967361|22.74|12.23|true|tom young|2013-03-01 09:11:58.703250|56.70|philosophy +22|385|65789|4294967440|32.36|41.43|true|holly zipper|2013-03-01 09:11:58.703218|17.07|joggying +117|323|65590|4294967535|0.92|10.75|false|holly robinson|2013-03-01 09:11:58.703101|28.48|geology +104|329|65737|4294967350|61.99|16.48|false|tom carson|2013-03-01 09:11:58.703280|35.13|opthamology +33|499|65586|4294967434|31.13|46.69|true|fred falkner|2013-03-01 09:11:58.703232|31.87|forestry +69|259|65562|4294967481|30.98|35.57|true|zach polk|2013-03-01 09:11:58.703240|41.72|nap time +19|320|65708|4294967495|25.65|20.06|false|jessica quirinius|2013-03-01 09:11:58.703275|11.32|opthamology +82|441|65759|4294967346|73.08|39.69|true|sarah laertes|2013-03-01 09:11:58.703175|57.15|undecided +70|418|65771|4294967393|24.69|46.60|false|nick hernandez|2013-03-01 09:11:58.703152|41.40|history +97|414|65620|4294967394|18.10|1.45|true|rachel steinbeck|2013-03-01 09:11:58.703094|76.72|chemistry +77|359|65697|4294967392|40.51|23.91|false|katie quirinius|2013-03-01 09:11:58.703229|3.25|zync studies +18|369|65567|4294967461|46.68|2.07|false|katie hernandez|2013-03-01 09:11:58.703282|45.63|history +10|277|65628|4294967339|13.66|26.59|true|oscar davidson|2013-03-01 09:11:58.703193|14.16|mathematics +88|416|65536|4294967441|9.33|14.49|false|alice ichabod|2013-03-01 09:11:58.703141|31.09|opthamology +53|278|65709|4294967420|59.18|37.42|true|yuri johnson|2013-03-01 09:11:58.703266|33.49|industrial engineering +86|260|65707|4294967547|65.55|37.11|true|katie laertes|2013-03-01 09:11:58.703285|69.11|zync studies +90|260|65625|4294967510|79.71|38.23|false|quinn polk|2013-03-01 09:11:58.703214|75.88|american history +81|408|65657|4294967469|46.48|28.92|true|jessica van buren|2013-03-01 09:11:58.703085|31.76|yard duty +124|469|65577|4294967322|43.03|42.77|true|xavier allen|2013-03-01 09:11:58.703301|16.67|geology +30|286|65698|4294967351|63.48|20.35|false|bob davidson|2013-03-01 09:11:58.703191|26.09|chemistry +32|359|65748|4294967407|64.55|4.64|true|ethan carson|2013-03-01 09:11:58.703086|76.20|industrial engineering +20|475|65706|4294967327|76.23|14.71|false|luke carson|2013-03-01 09:11:58.703258|98.26|study skills +-2|389|65738|4294967520|99.45|26.26|true|bob falkner|2013-03-01 09:11:58.703071|17.47|nap time +80|449|65664|4294967305|97.03|3.09|true|fred young|2013-03-01 09:11:58.703124|75.15|debate +68|349|65668|4294967535|35.89|30.43|true|irene van buren|2013-03-01 09:11:58.703087|86.84|biology +119|335|65766|4294967393|42.78|40.52|true|katie underhill|2013-03-01 09:11:58.703217|32.18|philosophy +-3|469|65577|4294967451|88.78|32.96|true|katie ichabod|2013-03-01 09:11:58.703139|69.13|undecided +56|347|65629|4294967473|52.76|23.59|false|yuri xylophone|2013-03-01 09:11:58.703321|28.28|quiet hour +58|399|65572|4294967431|24.08|24.77|true|calvin quirinius|2013-03-01 09:11:58.703280|17.61|zync studies +61|392|65768|4294967512|69.10|18.87|false|xavier steinbeck|2013-03-01 09:11:58.703172|99.33|industrial engineering +93|413|65656|4294967355|95.27|33.46|false|quinn xylophone|2013-03-01 09:11:58.703132|54.66|undecided +44|459|65665|4294967353|27.28|18.42|true|wendy carson|2013-03-01 09:11:58.703192|47.46|linguistics +44|264|65594|4294967541|47.83|38.39|true|wendy johnson|2013-03-01 09:11:58.703251|21.49|geology +109|421|65581|4294967326|26.25|42.25|false|jessica robinson|2013-03-01 09:11:58.703299|7.20|nap time +87|289|65758|4294967539|43.32|9.35|false|david nixon|2013-03-01 09:11:58.703180|84.97|xylophone band +27|257|65560|4294967485|42.14|18.65|false|holly white|2013-03-01 09:11:58.703262|9.98|zync studies +19|364|65649|4294967476|72.25|27.42|true|victor robinson|2013-03-01 09:11:58.703308|85.43|undecided +68|411|65544|4294967428|15.57|1.44|true|luke johnson|2013-03-01 09:11:58.703261|5.46|industrial engineering +42|357|65571|4294967417|64.55|26.87|true|victor underhill|2013-03-01 09:11:58.703071|32.78|zync studies +24|480|65552|4294967482|80.62|28.66|false|bob thompson|2013-03-01 09:11:58.703118|32.36|history +16|263|65753|4294967301|67.77|19.88|false|fred ellison|2013-03-01 09:11:58.703246|28.34|topology +20|437|65560|4294967548|13.21|39.63|true|bob xylophone|2013-03-01 09:11:58.703279|62.35|zync studies +1|268|65712|4294967414|46.74|30.67|true|fred thompson|2013-03-01 09:11:58.703182|6.24|philosophy +99|370|65640|4294967456|58.68|47.18|true|priscilla nixon|2013-03-01 09:11:58.703142|68.54|yard duty +16|300|65715|4294967343|63.07|22.66|false|ethan carson|2013-03-01 09:11:58.703261|52.69|debate +104|390|65552|4294967480|51.64|8.63|false|holly falkner|2013-03-01 09:11:58.703302|94.59|mathematics +83|372|65556|4294967467|20.49|1.71|true|calvin king|2013-03-01 09:11:58.703090|71.37|philosophy +48|430|65650|4294967334|66.95|3.36|false|ulysses underhill|2013-03-01 09:11:58.703272|22.24|linguistics +73|450|65678|4294967407|10.57|19.70|true|jessica allen|2013-03-01 09:11:58.703238|21.24|chemistry +7|424|65768|4294967320|37.78|5.94|true|gabriella johnson|2013-03-01 09:11:58.703276|53.49|topology +57|503|65670|4294967398|64.29|36.54|true|alice steinbeck|2013-03-01 09:11:58.703089|57.21|study skills +5|337|65745|4294967313|46.15|42.55|false|zach king|2013-03-01 09:11:58.703125|38.82|joggying +104|274|65755|4294967404|7.70|24.54|true|irene carson|2013-03-01 09:11:58.703322|55.38|industrial engineering +107|481|65771|4294967508|32.36|33.69|false|quinn king|2013-03-01 09:11:58.703256|95.23|history +69|411|65566|4294967349|34.14|16.92|false|gabriella laertes|2013-03-01 09:11:58.703111|2.77|history +37|404|65598|4294967417|82.76|9.47|true|irene thompson|2013-03-01 09:11:58.703260|67.00|industrial engineering +57|296|65607|4294967351|42.45|12.61|true|priscilla laertes|2013-03-01 09:11:58.703165|76.58|undecided +-2|420|65603|4294967391|40.07|33.44|true|quinn miller|2013-03-01 09:11:58.703150|87.75|religion +-2|393|65715|4294967305|48.30|1.85|true|alice xylophone|2013-03-01 09:11:58.703105|29.58|values clariffication +29|434|65626|4294967475|37.33|38.56|true|katie steinbeck|2013-03-01 09:11:58.703312|5.52|quiet hour +88|344|65649|4294967336|56.40|21.55|false|david quirinius|2013-03-01 09:11:58.703129|37.43|xylophone band +11|506|65775|4294967297|85.14|12.80|true|gabriella robinson|2013-03-01 09:11:58.703266|32.17|chemistry +103|276|65700|4294967340|83.22|36.33|true|jessica davidson|2013-03-01 09:11:58.703111|49.03|industrial engineering +42|412|65547|4294967406|42.26|24.09|true|victor white|2013-03-01 09:11:58.703076|74.95|mathematics +-3|447|65755|4294967320|43.69|20.03|false|victor hernandez|2013-03-01 09:11:58.703176|13.70|forestry +85|335|65600|4294967327|72.03|42.67|false|nick polk|2013-03-01 09:11:58.703287|1.49|mathematics +108|309|65780|4294967477|61.43|33.52|true|nick garcia|2013-03-01 09:11:58.703099|38.36|geology +92|319|65592|4294967471|31.45|4.93|true|zach johnson|2013-03-01 09:11:58.703157|80.11|topology +52|457|65573|4294967500|29.32|8.19|false|gabriella ellison|2013-03-01 09:11:58.703122|54.02|wind surfing +20|434|65616|4294967470|55.64|16.43|false|tom brown|2013-03-01 09:11:58.703269|24.11|quiet hour +78|441|65656|4294967404|37.27|49.81|true|wendy zipper|2013-03-01 09:11:58.703238|44.33|debate +87|308|65764|4294967476|88.61|11.45|false|wendy hernandez|2013-03-01 09:11:58.703188|53.08|linguistics +76|284|65770|4294967452|95.00|25.29|false|mike king|2013-03-01 09:11:58.703171|8.53|debate +14|369|65558|4294967506|70.90|24.61|false|alice van buren|2013-03-01 09:11:58.703134|65.18|quiet hour +65|391|65548|4294967366|11.29|34.63|true|bob robinson|2013-03-01 09:11:58.703273|98.51|history +91|475|65777|4294967533|26.67|34.51|false|david ovid|2013-03-01 09:11:58.703090|92.13|zync studies +13|372|65755|4294967452|46.10|2.67|false|tom young|2013-03-01 09:11:58.703157|83.06|philosophy +59|318|65780|4294967312|96.17|11.27|false|wendy thompson|2013-03-01 09:11:58.703110|83.27|undecided +0|454|65546|4294967513|29.22|49.17|true|oscar brown|2013-03-01 09:11:58.703070|53.25|industrial engineering +77|279|65545|4294967439|35.15|39.07|false|gabriella van buren|2013-03-01 09:11:58.703175|36.89|study skills +79|320|65771|4294967503|37.85|12.55|false|priscilla thompson|2013-03-01 09:11:58.703133|90.38|mathematics +122|298|65786|4294967472|50.13|41.02|true|yuri thompson|2013-03-01 09:11:58.703246|65.22|linguistics +79|442|65645|4294967437|46.36|11.03|false|tom brown|2013-03-01 09:11:58.703140|72.24|yard duty +3|318|65675|4294967414|57.67|7.24|false|oscar king|2013-03-01 09:11:58.703082|44.88|american history +43|281|65698|4294967359|4.33|25.81|true|priscilla allen|2013-03-01 09:11:58.703099|94.82|debate +4|311|65583|4294967330|46.77|17.56|true|jessica thompson|2013-03-01 09:11:58.703297|66.94|debate +-3|423|65646|4294967378|63.19|34.04|false|priscilla quirinius|2013-03-01 09:11:58.703228|34.66|xylophone band +98|502|65638|4294967379|86.81|8.34|true|bob carson|2013-03-01 09:11:58.703140|59.62|biology +51|355|65781|4294967537|58.53|4.34|false|oscar white|2013-03-01 09:11:58.703074|72.00|study skills +83|323|65689|4294967427|58.02|7.70|false|zach underhill|2013-03-01 09:11:58.703300|12.72|zync studies +10|279|65655|4294967337|63.20|45.16|true|david hernandez|2013-03-01 09:11:58.703296|37.00|undecided +90|261|65599|4294967400|88.27|5.92|false|katie polk|2013-03-01 09:11:58.703164|85.70|industrial engineering +33|430|65579|4294967312|56.79|15.97|true|gabriella thompson|2013-03-01 09:11:58.703249|29.87|study skills +0|395|65760|4294967464|58.78|37.00|false|alice quirinius|2013-03-01 09:11:58.703177|31.02|history +77|325|65721|4294967507|8.63|22.63|true|priscilla hernandez|2013-03-01 09:11:58.703308|1.34|opthamology +121|343|65630|4294967359|91.54|24.17|true|wendy zipper|2013-03-01 09:11:58.703172|10.91|debate +45|420|65725|4294967499|46.11|22.71|false|tom johnson|2013-03-01 09:11:58.703248|62.80|opthamology +3|280|65720|4294967347|53.79|35.71|true|rachel king|2013-03-01 09:11:58.703101|29.26|values clariffication +1|411|65743|4294967383|75.08|12.44|false|victor nixon|2013-03-01 09:11:58.703202|24.26|linguistics +109|448|65582|4294967327|9.17|8.69|true|ethan ellison|2013-03-01 09:11:58.703202|40.63|study skills +54|413|65580|4294967492|37.56|5.20|false|ulysses ovid|2013-03-01 09:11:58.703163|35.09|history +19|448|65654|4294967470|11.05|26.20|false|bob allen|2013-03-01 09:11:58.703083|35.48|education +81|362|65580|4294967376|30.85|42.28|false|luke hernandez|2013-03-01 09:11:58.703187|25.29|history +107|305|65617|4294967360|71.78|29.08|false|alice ichabod|2013-03-01 09:11:58.703272|34.96|chemistry +53|295|65603|4294967302|74.20|49.09|false|holly xylophone|2013-03-01 09:11:58.703226|15.28|forestry +92|411|65660|4294967453|51.37|5.73|true|zach xylophone|2013-03-01 09:11:58.703140|59.56|xylophone band +3|364|65607|4294967339|91.84|14.89|true|fred laertes|2013-03-01 09:11:58.703292|78.85|yard duty +24|376|65692|4294967531|90.77|33.31|true|quinn laertes|2013-03-01 09:11:58.703075|63.49|kindergarten +99|285|65596|4294967520|86.94|31.45|true|mike thompson|2013-03-01 09:11:58.703119|8.01|nap time +7|277|65626|4294967311|68.68|40.55|true|ulysses robinson|2013-03-01 09:11:58.703139|51.74|history +37|416|65726|4294967480|7.82|25.05|true|sarah robinson|2013-03-01 09:11:58.703220|38.75|kindergarten +33|439|65644|4294967309|86.11|26.38|false|katie davidson|2013-03-01 09:11:58.703113|18.16|american history +61|277|65723|4294967327|56.53|7.67|true|irene thompson|2013-03-01 09:11:58.703281|54.64|zync studies +-3|498|65751|4294967331|80.65|0.28|true|gabriella brown|2013-03-01 09:11:58.703288|61.15|opthamology +30|401|65763|4294967494|61.12|7.22|true|mike brown|2013-03-01 09:11:58.703094|89.14|undecided +107|298|65589|4294967540|41.03|8.13|false|tom davidson|2013-03-01 09:11:58.703138|49.46|religion +42|319|65701|4294967426|25.23|12.02|true|gabriella nixon|2013-03-01 09:11:58.703211|99.28|topology +51|508|65597|4294967415|34.36|23.62|false|quinn steinbeck|2013-03-01 09:11:58.703219|22.11|study skills +57|293|65651|4294967468|83.31|33.01|false|holly nixon|2013-03-01 09:11:58.703193|80.96|nap time +55|490|65597|4294967339|70.91|17.75|true|xavier davidson|2013-03-01 09:11:58.703307|16.06|study skills +21|266|65578|4294967397|26.36|47.50|false|mike young|2013-03-01 09:11:58.703087|41.67|education +36|296|65689|4294967432|16.40|19.10|false|ulysses ovid|2013-03-01 09:11:58.703078|20.95|undecided +82|373|65637|4294967535|20.98|40.31|false|priscilla ellison|2013-03-01 09:11:58.703162|37.86|debate +114|451|65745|4294967485|18.42|44.90|true|katie laertes|2013-03-01 09:11:58.703306|63.16|american history +4|283|65705|4294967543|25.24|2.98|true|nick polk|2013-03-01 09:11:58.703299|23.35|geology +112|361|65739|4294967403|42.30|17.17|false|nick robinson|2013-03-01 09:11:58.703141|21.04|xylophone band +0|350|65547|4294967402|6.38|38.58|true|ulysses thompson|2013-03-01 09:11:58.703240|40.54|quiet hour +82|342|65671|4294967371|12.05|29.96|false|tom quirinius|2013-03-01 09:11:58.703162|69.93|undecided +77|365|65583|4294967430|89.72|48.22|true|alice king|2013-03-01 09:11:58.703258|18.75|yard duty +48|384|65697|4294967421|24.67|2.36|true|fred allen|2013-03-01 09:11:58.703203|48.81|kindergarten +34|435|65791|4294967466|13.72|46.97|false|gabriella quirinius|2013-03-01 09:11:58.703282|23.60|joggying +24|267|65596|4294967489|50.01|38.27|true|gabriella falkner|2013-03-01 09:11:58.703179|31.03|topology +32|432|65724|4294967461|33.79|19.92|true|wendy laertes|2013-03-01 09:11:58.703146|27.39|philosophy +18|383|65603|4294967501|44.74|16.52|true|nick white|2013-03-01 09:11:58.703255|68.45|study skills +93|346|65673|4294967397|7.16|34.65|true|gabriella king|2013-03-01 09:11:58.703263|43.66|joggying +1|406|65769|4294967439|96.82|5.94|true|irene young|2013-03-01 09:11:58.703274|62.70|biology +77|302|65682|4294967354|2.52|24.35|true|yuri carson|2013-03-01 09:11:58.703172|89.11|american history +21|469|65726|4294967422|76.91|37.08|false|yuri thompson|2013-03-01 09:11:58.703228|98.06|philosophy +76|427|65674|4294967351|11.13|45.24|false|alice garcia|2013-03-01 09:11:58.703205|3.08|wind surfing +102|275|65676|4294967484|4.53|17.10|true|gabriella ovid|2013-03-01 09:11:58.703094|48.37|debate +65|481|65590|4294967494|91.86|44.63|false|irene carson|2013-03-01 09:11:58.703146|68.35|industrial engineering +66|271|65651|4294967394|40.97|46.48|true|ulysses ellison|2013-03-01 09:11:58.703214|44.96|xylophone band +22|477|65543|4294967311|59.02|49.60|false|mike carson|2013-03-01 09:11:58.703292|37.86|zync studies +84|258|65717|4294967343|95.76|35.07|false|sarah polk|2013-03-01 09:11:58.703087|64.36|xylophone band +94|363|65779|4294967403|21.39|3.27|false|yuri miller|2013-03-01 09:11:58.703158|79.30|biology +119|419|65707|4294967402|81.49|21.37|true|ulysses miller|2013-03-01 09:11:58.703323|62.91|kindergarten +111|354|65760|4294967298|1.03|33.90|false|xavier davidson|2013-03-01 09:11:58.703239|4.07|industrial engineering +104|404|65648|4294967306|70.60|22.49|true|luke carson|2013-03-01 09:11:58.703152|50.62|zync studies +106|294|65779|4294967412|3.71|43.07|true|bob ichabod|2013-03-01 09:11:58.703211|48.27|undecided +85|379|65695|4294967519|99.05|32.89|false|victor polk|2013-03-01 09:11:58.703189|59.02|study skills +15|368|65638|4294967382|76.99|42.22|false|yuri falkner|2013-03-01 09:11:58.703117|21.56|topology +113|292|65696|4294967544|24.65|47.71|true|gabriella van buren|2013-03-01 09:11:58.703276|16.84|kindergarten +113|387|65604|4294967423|78.47|1.88|false|katie ellison|2013-03-01 09:11:58.703239|34.78|study skills +56|278|65717|4294967384|82.95|11.79|true|oscar ovid|2013-03-01 09:11:58.703243|2.92|opthamology +73|293|65726|4294967311|85.13|5.54|false|david allen|2013-03-01 09:11:58.703271|59.03|forestry +104|458|65549|4294967473|89.99|37.58|false|jessica robinson|2013-03-01 09:11:58.703165|99.27|values clariffication +39|351|65649|4294967425|4.76|19.44|true|alice steinbeck|2013-03-01 09:11:58.703255|57.82|american history +124|357|65730|4294967535|19.98|44.48|false|david allen|2013-03-01 09:11:58.703266|16.15|mathematics +18|508|65618|4294967376|2.55|1.53|true|wendy king|2013-03-01 09:11:58.703109|69.37|kindergarten +114|348|65707|4294967352|36.61|49.00|false|bob nixon|2013-03-01 09:11:58.703096|8.28|chemistry +87|273|65639|4294967342|29.92|26.76|true|victor garcia|2013-03-01 09:11:58.703164|98.84|study skills +56|453|65669|4294967347|78.78|28.73|false|victor carson|2013-03-01 09:11:58.703286|44.31|yard duty +110|382|65733|4294967368|38.27|1.85|false|bob davidson|2013-03-01 09:11:58.703183|9.44|linguistics +81|332|65759|4294967425|95.98|44.13|false|ulysses carson|2013-03-01 09:11:58.703139|32.12|quiet hour +43|340|65757|4294967465|78.30|31.26|false|ulysses white|2013-03-01 09:11:58.703120|77.95|biology +-2|450|65727|4294967487|94.57|30.40|false|david miller|2013-03-01 09:11:58.703238|39.90|religion +79|370|65562|4294967446|24.78|26.43|false|irene king|2013-03-01 09:11:58.703297|2.98|joggying +48|322|65748|4294967426|94.22|16.95|false|ethan davidson|2013-03-01 09:11:58.703078|86.86|quiet hour +46|334|65669|4294967395|28.31|23.08|false|tom johnson|2013-03-01 09:11:58.703084|24.64|opthamology +121|376|65677|4294967445|3.70|44.68|true|quinn nixon|2013-03-01 09:11:58.703249|8.31|american history +20|491|65550|4294967366|78.89|36.41|true|luke xylophone|2013-03-01 09:11:58.703244|30.89|study skills +100|340|65560|4294967523|6.77|30.81|false|wendy young|2013-03-01 09:11:58.703151|28.03|values clariffication +-2|347|65687|4294967440|71.04|40.80|false|tom miller|2013-03-01 09:11:58.703084|89.84|forestry +66|448|65775|4294967432|2.75|28.18|true|holly falkner|2013-03-01 09:11:58.703139|95.02|history +115|502|65753|4294967512|87.75|10.28|false|fred miller|2013-03-01 09:11:58.703122|57.79|industrial engineering +94|320|65552|4294967509|71.73|39.75|true|mike steinbeck|2013-03-01 09:11:58.703192|63.94|yard duty +79|347|65699|4294967477|68.88|47.11|true|david steinbeck|2013-03-01 09:11:58.703222|70.43|yard duty +-2|395|65693|4294967395|31.66|30.63|false|ethan king|2013-03-01 09:11:58.703121|16.43|linguistics +82|476|65740|4294967302|0.93|9.80|true|yuri white|2013-03-01 09:11:58.703320|28.76|undecided +27|266|65549|4294967482|24.01|28.29|false|victor laertes|2013-03-01 09:11:58.703235|36.11|biology +14|424|65705|4294967310|57.50|27.44|true|zach white|2013-03-01 09:11:58.703287|42.99|study skills +105|261|65689|4294967451|5.62|17.22|false|wendy van buren|2013-03-01 09:11:58.703152|62.36|wind surfing +122|279|65663|4294967320|34.36|5.33|true|katie carson|2013-03-01 09:11:58.703277|95.34|study skills +25|483|65537|4294967463|37.36|17.35|false|fred johnson|2013-03-01 09:11:58.703289|56.83|philosophy +102|504|65760|4294967536|59.52|37.31|false|priscilla quirinius|2013-03-01 09:11:58.703157|28.07|biology +16|366|65565|4294967467|45.29|35.46|false|bob ovid|2013-03-01 09:11:58.703305|22.66|quiet hour +-1|391|65678|4294967317|17.03|42.12|false|zach white|2013-03-01 09:11:58.703092|54.74|joggying +90|319|65570|4294967536|86.39|46.08|false|bob ovid|2013-03-01 09:11:58.703180|19.37|forestry +74|456|65719|4294967378|3.21|36.26|false|priscilla young|2013-03-01 09:11:58.703238|31.56|linguistics +49|458|65743|4294967411|79.92|42.75|true|priscilla carson|2013-03-01 09:11:58.703236|15.88|nap time +8|395|65566|4294967324|36.36|42.08|false|xavier davidson|2013-03-01 09:11:58.703114|11.27|kindergarten +54|363|65788|4294967369|65.70|2.27|false|holly xylophone|2013-03-01 09:11:58.703108|20.81|linguistics +4|289|65775|4294967340|64.67|9.15|false|victor ichabod|2013-03-01 09:11:58.703173|34.65|undecided +51|444|65640|4294967358|23.03|23.23|false|wendy brown|2013-03-01 09:11:58.703250|22.66|undecided +122|499|65677|4294967461|62.98|47.86|false|priscilla nixon|2013-03-01 09:11:58.703147|93.93|american history +77|297|65727|4294967327|28.27|36.09|false|priscilla falkner|2013-03-01 09:11:58.703113|62.84|geology +28|450|65760|4294967378|76.72|37.78|true|fred xylophone|2013-03-01 09:11:58.703307|28.84|wind surfing +112|334|65577|4294967356|45.20|35.36|false|xavier miller|2013-03-01 09:11:58.703138|51.53|philosophy +18|511|65568|4294967438|60.85|28.45|true|rachel laertes|2013-03-01 09:11:58.703209|89.16|history +20|266|65738|4294967355|30.12|4.75|true|yuri hernandez|2013-03-01 09:11:58.703277|20.80|geology +111|276|65602|4294967543|73.01|48.49|false|sarah brown|2013-03-01 09:11:58.703185|17.42|geology +13|357|65744|4294967444|93.73|43.64|true|oscar xylophone|2013-03-01 09:11:58.703191|89.93|religion +24|476|65682|4294967322|13.62|28.25|true|ulysses polk|2013-03-01 09:11:58.703225|72.75|values clariffication +25|402|65732|4294967516|93.46|8.03|false|david polk|2013-03-01 09:11:58.703214|96.10|history +118|355|65557|4294967309|60.51|23.78|false|ethan king|2013-03-01 09:11:58.703223|30.05|geology +81|289|65775|4294967400|33.50|1.02|true|nick quirinius|2013-03-01 09:11:58.703170|18.11|nap time +4|351|65729|4294967379|22.98|3.91|false|zach carson|2013-03-01 09:11:58.703232|49.91|chemistry +40|357|65719|4294967501|25.34|37.06|false|wendy underhill|2013-03-01 09:11:58.703072|92.34|education +95|324|65649|4294967314|38.99|28.60|true|david garcia|2013-03-01 09:11:58.703202|21.79|nap time +118|378|65737|4294967399|8.95|10.37|true|david robinson|2013-03-01 09:11:58.703281|88.80|wind surfing +120|468|65780|4294967424|59.98|5.81|true|david quirinius|2013-03-01 09:11:58.703134|37.45|undecided +55|257|65547|4294967333|54.01|21.47|false|katie ichabod|2013-03-01 09:11:58.703196|80.30|debate +53|305|65642|4294967328|10.62|14.29|false|nick ellison|2013-03-01 09:11:58.703071|53.16|yard duty +43|455|65571|4294967486|51.60|6.55|true|gabriella young|2013-03-01 09:11:58.703303|0.07|industrial engineering +46|312|65641|4294967432|75.07|23.42|true|holly ovid|2013-03-01 09:11:58.703189|46.99|biology +8|458|65627|4294967478|70.00|22.12|true|yuri hernandez|2013-03-01 09:11:58.703252|42.09|quiet hour +118|430|65611|4294967354|48.18|25.85|true|fred polk|2013-03-01 09:11:58.703170|72.65|forestry +-3|268|65710|4294967448|82.74|12.48|true|holly polk|2013-03-01 09:11:58.703273|14.55|undecided +123|309|65576|4294967422|97.81|26.49|true|jessica carson|2013-03-01 09:11:58.703132|53.41|biology +31|354|65640|4294967402|84.13|29.72|true|bob ichabod|2013-03-01 09:11:58.703224|9.63|topology +27|396|65745|4294967339|15.21|0.65|false|oscar laertes|2013-03-01 09:11:58.703092|48.14|values clariffication +33|310|65771|4294967403|85.95|35.67|false|wendy allen|2013-03-01 09:11:58.703292|2.29|geology +63|411|65619|4294967484|82.02|46.32|true|nick underhill|2013-03-01 09:11:58.703287|57.22|chemistry +56|417|65720|4294967545|39.41|41.38|true|ulysses ellison|2013-03-01 09:11:58.703106|37.76|linguistics +44|285|65575|4294967428|56.45|8.99|false|rachel ovid|2013-03-01 09:11:58.703071|18.85|education +22|423|65722|4294967503|49.04|33.81|false|fred miller|2013-03-01 09:11:58.703166|10.38|history +44|344|65634|4294967343|77.35|4.04|true|nick brown|2013-03-01 09:11:58.703129|75.10|geology +34|413|65708|4294967489|39.16|0.46|true|victor brown|2013-03-01 09:11:58.703321|77.21|philosophy +35|366|65749|4294967424|1.91|49.83|false|sarah polk|2013-03-01 09:11:58.703160|64.16|forestry +80|432|65774|4294967444|0.01|40.55|true|bob ichabod|2013-03-01 09:11:58.703165|86.78|wind surfing +-2|261|65551|4294967507|33.00|5.08|false|xavier miller|2013-03-01 09:11:58.703089|17.41|geology +111|402|65544|4294967480|78.24|46.91|false|alice davidson|2013-03-01 09:11:58.703080|37.53|forestry +120|467|65760|4294967504|52.99|27.09|true|ulysses brown|2013-03-01 09:11:58.703166|10.86|undecided +55|486|65693|4294967389|86.64|5.41|false|david polk|2013-03-01 09:11:58.703138|11.32|undecided +66|407|65750|4294967408|64.54|31.86|false|gabriella robinson|2013-03-01 09:11:58.703151|36.75|biology +19|429|65593|4294967471|89.62|38.13|false|oscar ovid|2013-03-01 09:11:58.703177|55.49|debate +101|353|65733|4294967473|43.95|44.72|false|sarah steinbeck|2013-03-01 09:11:58.703111|25.25|philosophy +79|364|65541|4294967531|74.34|13.80|true|quinn nixon|2013-03-01 09:11:58.703206|11.37|nap time +15|471|65624|4294967326|39.61|28.88|true|katie underhill|2013-03-01 09:11:58.703124|56.57|history +36|491|65767|4294967396|43.41|12.86|false|zach carson|2013-03-01 09:11:58.703151|8.85|undecided +102|289|65555|4294967488|97.13|21.09|false|sarah thompson|2013-03-01 09:11:58.703214|61.68|philosophy +41|411|65589|4294967522|61.79|15.95|true|oscar white|2013-03-01 09:11:58.703145|65.16|joggying +16|407|65538|4294967328|63.48|37.14|true|gabriella nixon|2013-03-01 09:11:58.703137|13.71|geology +27|431|65677|4294967326|60.34|33.03|true|alice davidson|2013-03-01 09:11:58.703273|41.05|geology +85|276|65748|4294967396|15.21|35.72|true|oscar johnson|2013-03-01 09:11:58.703267|63.93|xylophone band +54|420|65757|4294967487|39.15|6.05|false|bob brown|2013-03-01 09:11:58.703278|20.29|history +123|472|65725|4294967327|70.19|47.16|true|nick nixon|2013-03-01 09:11:58.703224|73.95|forestry +13|436|65580|4294967500|27.00|1.00|true|nick quirinius|2013-03-01 09:11:58.703207|86.78|nap time +104|350|65537|4294967505|79.01|34.04|true|zach falkner|2013-03-01 09:11:58.703121|40.92|philosophy +65|281|65777|4294967344|38.23|5.11|true|victor davidson|2013-03-01 09:11:58.703103|80.27|wind surfing +27|345|65686|4294967503|50.23|2.49|false|fred allen|2013-03-01 09:11:58.703230|4.54|yard duty +100|362|65623|4294967528|24.69|12.24|false|ulysses laertes|2013-03-01 09:11:58.703077|13.89|opthamology +26|284|65711|4294967547|44.01|28.28|false|xavier brown|2013-03-01 09:11:58.703239|63.24|philosophy +1|372|65788|4294967322|49.46|13.99|false|nick polk|2013-03-01 09:11:58.703172|36.80|american history +66|310|65694|4294967431|70.58|48.00|true|oscar van buren|2013-03-01 09:11:58.703325|12.66|values clariffication +105|265|65563|4294967439|9.27|15.72|false|bob ovid|2013-03-01 09:11:58.703257|21.71|topology +35|465|65719|4294967539|54.09|44.88|true|sarah van buren|2013-03-01 09:11:58.703071|52.08|philosophy +72|270|65623|4294967540|5.00|15.07|true|luke falkner|2013-03-01 09:11:58.703238|50.45|mathematics +19|408|65643|4294967393|67.97|29.85|false|rachel king|2013-03-01 09:11:58.703143|83.80|forestry +98|460|65688|4294967440|3.85|17.28|false|calvin hernandez|2013-03-01 09:11:58.703113|61.13|biology +53|352|65543|4294967436|15.80|31.78|true|nick laertes|2013-03-01 09:11:58.703242|56.82|history +114|277|65700|4294967373|52.89|13.96|false|yuri miller|2013-03-01 09:11:58.703076|7.67|geology +48|413|65538|4294967515|21.19|16.38|false|rachel young|2013-03-01 09:11:58.703291|79.10|quiet hour +65|294|65610|4294967406|91.69|23.10|false|ulysses carson|2013-03-01 09:11:58.703093|66.58|zync studies +102|511|65757|4294967360|1.47|0.93|false|yuri laertes|2013-03-01 09:11:58.703297|43.05|industrial engineering +90|380|65582|4294967447|71.64|24.90|false|ulysses garcia|2013-03-01 09:11:58.703230|6.62|zync studies +121|496|65553|4294967325|33.44|21.86|true|katie laertes|2013-03-01 09:11:58.703296|75.52|chemistry +3|487|65599|4294967437|21.17|33.51|false|priscilla garcia|2013-03-01 09:11:58.703079|15.22|linguistics +67|425|65681|4294967472|74.63|44.34|true|quinn quirinius|2013-03-01 09:11:58.703173|37.27|linguistics +76|497|65573|4294967440|99.93|15.98|false|sarah falkner|2013-03-01 09:11:58.703262|49.71|wind surfing +59|459|65727|4294967375|80.35|27.35|false|bob young|2013-03-01 09:11:58.703158|57.99|linguistics +105|423|65754|4294967299|60.15|9.39|false|david davidson|2013-03-01 09:11:58.703323|53.53|undecided +38|482|65731|4294967358|56.01|23.98|false|jessica davidson|2013-03-01 09:11:58.703079|99.84|biology +101|330|65741|4294967532|81.10|43.04|false|oscar ichabod|2013-03-01 09:11:58.703319|45.58|nap time +35|326|65654|4294967474|4.77|5.57|false|alice steinbeck|2013-03-01 09:11:58.703246|97.37|study skills +115|480|65741|4294967496|61.65|46.30|false|alice ovid|2013-03-01 09:11:58.703325|56.56|philosophy +26|484|65787|4294967447|73.47|48.92|false|tom ovid|2013-03-01 09:11:58.703172|22.72|debate +26|268|65550|4294967317|46.89|13.33|true|gabriella white|2013-03-01 09:11:58.703138|48.15|study skills +37|311|65656|4294967382|64.03|19.06|true|xavier laertes|2013-03-01 09:11:58.703165|73.23|forestry +0|399|65643|4294967376|28.15|3.65|false|ethan hernandez|2013-03-01 09:11:58.703289|10.03|forestry +35|394|65569|4294967344|64.34|15.02|false|nick hernandez|2013-03-01 09:11:58.703218|73.42|education +38|488|65553|4294967319|90.19|32.74|true|sarah miller|2013-03-01 09:11:58.703173|85.64|joggying +107|262|65562|4294967470|60.58|23.55|true|ulysses robinson|2013-03-01 09:11:58.703160|25.58|yard duty +68|347|65666|4294967464|86.42|48.97|false|tom underhill|2013-03-01 09:11:58.703151|30.59|yard duty +112|387|65622|4294967549|23.34|45.46|false|jessica miller|2013-03-01 09:11:58.703234|31.58|forestry +67|336|65732|4294967332|1.34|2.68|false|oscar ovid|2013-03-01 09:11:58.703096|57.07|education +41|315|65777|4294967385|11.88|20.71|false|tom young|2013-03-01 09:11:58.703190|44.32|biology +47|377|65626|4294967481|10.15|39.69|false|rachel xylophone|2013-03-01 09:11:58.703168|64.40|opthamology +15|425|65547|4294967296|14.66|32.59|false|katie xylophone|2013-03-01 09:11:58.703136|89.87|wind surfing +69|407|65656|4294967348|67.56|45.53|false|alice ovid|2013-03-01 09:11:58.703246|99.24|religion +72|416|65631|4294967515|40.41|17.38|true|mike ichabod|2013-03-01 09:11:58.703096|30.54|chemistry +54|482|65674|4294967373|52.37|27.44|false|nick falkner|2013-03-01 09:11:58.703229|29.32|debate +103|396|65649|4294967392|91.31|44.19|false|alice hernandez|2013-03-01 09:11:58.703273|15.47|mathematics +42|294|65788|4294967476|60.19|49.63|false|irene steinbeck|2013-03-01 09:11:58.703162|51.70|nap time +106|327|65589|4294967333|89.59|10.31|true|katie nixon|2013-03-01 09:11:58.703242|88.60|debate +101|396|65728|4294967516|74.92|32.26|true|wendy nixon|2013-03-01 09:11:58.703228|90.87|chemistry +96|385|65685|4294967343|90.36|19.80|false|irene miller|2013-03-01 09:11:58.703237|93.70|quiet hour +113|449|65754|4294967367|59.32|37.70|true|yuri young|2013-03-01 09:11:58.703113|72.55|religion +7|462|65642|4294967325|96.55|28.51|true|yuri quirinius|2013-03-01 09:11:58.703274|33.04|linguistics +88|445|65713|4294967398|86.83|46.73|false|ulysses polk|2013-03-01 09:11:58.703313|87.34|forestry +34|464|65716|4294967313|97.25|45.08|false|calvin hernandez|2013-03-01 09:11:58.703265|0.39|chemistry +38|494|65696|4294967305|90.34|24.84|true|zach steinbeck|2013-03-01 09:11:58.703173|82.78|education +104|417|65555|4294967438|77.59|27.39|false|david brown|2013-03-01 09:11:58.703109|19.14|religion +83|420|65726|4294967490|29.05|42.49|false|tom white|2013-03-01 09:11:58.703160|44.04|kindergarten +37|453|65721|4294967393|97.52|4.94|false|fred robinson|2013-03-01 09:11:58.703180|74.29|religion +25|492|65622|4294967522|72.21|44.77|false|priscilla robinson|2013-03-01 09:11:58.703324|41.93|nap time +111|301|65564|4294967348|7.12|41.54|false|oscar nixon|2013-03-01 09:11:58.703185|31.34|industrial engineering +76|463|65566|4294967339|67.32|14.09|true|irene van buren|2013-03-01 09:11:58.703301|42.25|joggying +60|461|65760|4294967444|59.50|28.38|true|gabriella nixon|2013-03-01 09:11:58.703076|86.12|kindergarten +124|468|65684|4294967499|1.86|35.63|true|quinn brown|2013-03-01 09:11:58.703074|82.73|topology +66|300|65650|4294967483|96.07|15.59|true|wendy steinbeck|2013-03-01 09:11:58.703317|88.31|debate +102|309|65666|4294967550|80.48|7.61|false|jessica davidson|2013-03-01 09:11:58.703176|32.78|opthamology +8|475|65766|4294967536|16.14|0.73|true|gabriella brown|2013-03-01 09:11:58.703103|25.70|linguistics +31|459|65746|4294967320|24.38|38.97|false|priscilla steinbeck|2013-03-01 09:11:58.703219|57.46|yard duty +6|401|65693|4294967544|0.50|44.83|true|calvin ovid|2013-03-01 09:11:58.703228|48.80|joggying +105|479|65677|4294967537|56.36|44.03|true|nick xylophone|2013-03-01 09:11:58.703172|9.59|chemistry +78|440|65620|4294967504|39.33|31.33|true|quinn nixon|2013-03-01 09:11:58.703130|53.72|mathematics +53|370|65760|4294967339|63.80|28.11|false|rachel young|2013-03-01 09:11:58.703206|73.90|industrial engineering +117|435|65782|4294967463|16.33|20.07|true|holly xylophone|2013-03-01 09:11:58.703193|27.12|joggying +-1|358|65690|4294967333|32.11|9.87|true|luke king|2013-03-01 09:11:58.703135|83.63|biology +74|266|65734|4294967319|25.31|27.23|true|calvin ellison|2013-03-01 09:11:58.703323|89.50|yard duty +53|478|65709|4294967335|13.68|44.75|false|ethan quirinius|2013-03-01 09:11:58.703278|23.34|education +69|432|65790|4294967421|41.09|33.51|true|quinn ovid|2013-03-01 09:11:58.703128|52.36|geology +122|481|65688|4294967531|41.44|49.96|true|ulysses robinson|2013-03-01 09:11:58.703203|22.14|biology +78|299|65593|4294967437|34.18|36.03|false|gabriella zipper|2013-03-01 09:11:58.703236|31.35|kindergarten +114|473|65729|4294967419|12.57|34.78|false|holly johnson|2013-03-01 09:11:58.703184|28.71|yard duty +108|327|65660|4294967379|8.44|14.90|false|wendy young|2013-03-01 09:11:58.703315|30.23|values clariffication +44|486|65548|4294967333|55.52|9.84|true|alice white|2013-03-01 09:11:58.703176|43.62|mathematics +118|275|65727|4294967343|52.16|33.37|true|david young|2013-03-01 09:11:58.703145|30.94|american history +45|310|65553|4294967399|53.58|37.27|true|holly falkner|2013-03-01 09:11:58.703177|65.26|yard duty +110|392|65645|4294967475|26.42|29.94|false|fred king|2013-03-01 09:11:58.703145|91.59|mathematics +91|445|65656|4294967424|29.97|37.75|false|sarah ichabod|2013-03-01 09:11:58.703313|83.76|chemistry +6|261|65584|4294967501|69.80|43.34|true|priscilla nixon|2013-03-01 09:11:58.703274|68.21|geology +32|363|65565|4294967503|2.89|42.67|false|wendy van buren|2013-03-01 09:11:58.703321|31.75|undecided +14|285|65771|4294967335|75.27|35.22|true|ethan quirinius|2013-03-01 09:11:58.703276|36.61|topology +24|505|65710|4294967303|94.75|18.78|false|oscar young|2013-03-01 09:11:58.703156|72.54|topology +7|429|65701|4294967410|94.58|42.09|false|luke laertes|2013-03-01 09:11:58.703271|98.26|mathematics +110|459|65644|4294967547|26.70|48.94|false|oscar underhill|2013-03-01 09:11:58.703099|74.75|values clariffication +67|282|65671|4294967533|45.58|46.44|false|alice young|2013-03-01 09:11:58.703086|8.52|xylophone band +56|257|65752|4294967497|49.35|25.89|true|nick davidson|2013-03-01 09:11:58.703204|87.67|forestry +95|491|65770|4294967401|41.18|8.05|false|jessica ichabod|2013-03-01 09:11:58.703316|66.74|study skills +47|268|65762|4294967327|51.30|6.99|false|xavier johnson|2013-03-01 09:11:58.703112|93.85|opthamology +19|380|65754|4294967305|22.82|39.39|true|yuri quirinius|2013-03-01 09:11:58.703316|56.01|yard duty +5|322|65604|4294967432|50.15|35.49|true|wendy ellison|2013-03-01 09:11:58.703075|74.72|quiet hour +77|370|65566|4294967520|67.88|44.13|true|ulysses robinson|2013-03-01 09:11:58.703218|15.22|quiet hour +97|493|65674|4294967491|5.81|8.90|true|wendy davidson|2013-03-01 09:11:58.703315|80.48|biology +110|487|65583|4294967515|48.91|15.45|false|quinn johnson|2013-03-01 09:11:58.703081|68.17|american history +37|284|65572|4294967471|2.16|17.50|true|wendy falkner|2013-03-01 09:11:58.703167|49.10|nap time +67|504|65595|4294967432|52.34|10.07|false|ethan ellison|2013-03-01 09:11:58.703308|22.99|kindergarten +25|308|65726|4294967401|22.26|35.29|false|fred carson|2013-03-01 09:11:58.703282|68.09|education +105|301|65675|4294967453|16.86|25.94|false|holly garcia|2013-03-01 09:11:58.703321|91.05|opthamology +124|446|65768|4294967315|92.33|22.21|true|oscar carson|2013-03-01 09:11:58.703072|20.14|history +61|428|65599|4294967459|25.21|15.48|false|nick ellison|2013-03-01 09:11:58.703102|56.83|zync studies +11|334|65752|4294967307|92.52|40.55|true|holly king|2013-03-01 09:11:58.703175|96.46|philosophy +71|442|65674|4294967422|79.15|12.15|true|sarah johnson|2013-03-01 09:11:58.703193|59.74|forestry +91|387|65759|4294967335|26.33|49.10|false|katie quirinius|2013-03-01 09:11:58.703112|2.51|history +86|274|65570|4294967395|4.89|36.79|false|tom carson|2013-03-01 09:11:58.703210|25.18|american history +-2|337|65738|4294967374|6.86|17.06|false|mike laertes|2013-03-01 09:11:58.703100|17.75|biology +83|505|65724|4294967360|75.95|30.47|false|sarah miller|2013-03-01 09:11:58.703132|5.04|history +123|384|65586|4294967466|56.16|44.92|false|nick white|2013-03-01 09:11:58.703135|5.16|geology +-2|406|65762|4294967443|1.79|33.42|false|david falkner|2013-03-01 09:11:58.703254|58.25|opthamology +14|293|65614|4294967423|23.51|42.13|true|rachel king|2013-03-01 09:11:58.703235|97.25|yard duty +72|311|65673|4294967425|13.80|44.33|false|tom white|2013-03-01 09:11:58.703255|18.56|philosophy +78|348|65666|4294967313|10.47|41.09|false|jessica carson|2013-03-01 09:11:58.703301|13.18|xylophone band +70|362|65604|4294967519|85.48|1.61|false|rachel ovid|2013-03-01 09:11:58.703206|60.51|xylophone band +4|268|65678|4294967485|44.08|23.96|true|zach johnson|2013-03-01 09:11:58.703315|86.26|linguistics +20|431|65737|4294967335|50.75|27.37|true|katie polk|2013-03-01 09:11:58.703287|85.49|joggying +107|417|65702|4294967321|64.89|2.84|true|sarah van buren|2013-03-01 09:11:58.703150|2.56|philosophy +111|379|65571|4294967363|33.04|16.19|true|yuri ellison|2013-03-01 09:11:58.703073|7.00|values clariffication +101|489|65651|4294967523|94.28|25.21|true|yuri carson|2013-03-01 09:11:58.703211|70.13|mathematics +94|438|65670|4294967431|68.85|34.71|true|gabriella laertes|2013-03-01 09:11:58.703320|18.69|zync studies +101|281|65628|4294967358|14.28|25.03|false|yuri polk|2013-03-01 09:11:58.703305|53.31|values clariffication +113|417|65625|4294967406|2.04|23.56|false|wendy falkner|2013-03-01 09:11:58.703096|80.13|mathematics +81|476|65780|4294967315|34.59|5.46|true|wendy white|2013-03-01 09:11:58.703324|19.03|religion +101|307|65704|4294967359|39.65|11.48|true|jessica allen|2013-03-01 09:11:58.703237|53.37|philosophy +7|492|65768|4294967330|34.28|22.18|true|ulysses zipper|2013-03-01 09:11:58.703166|51.08|linguistics +95|324|65700|4294967328|96.45|11.36|false|quinn hernandez|2013-03-01 09:11:58.703099|68.21|topology +1|297|65560|4294967550|50.57|36.26|false|sarah laertes|2013-03-01 09:11:58.703323|94.17|study skills +106|484|65656|4294967535|13.86|20.69|true|zach ovid|2013-03-01 09:11:58.703075|86.24|forestry +13|400|65781|4294967305|43.95|22.63|false|wendy garcia|2013-03-01 09:11:58.703161|92.50|study skills +34|288|65560|4294967502|18.86|43.04|false|ulysses carson|2013-03-01 09:11:58.703293|72.36|nap time +71|379|65685|4294967359|97.23|4.99|false|luke davidson|2013-03-01 09:11:58.703202|87.32|chemistry +106|275|65592|4294967344|90.75|34.57|true|oscar young|2013-03-01 09:11:58.703318|68.02|undecided +13|448|65586|4294967542|6.90|39.21|false|oscar brown|2013-03-01 09:11:58.703245|34.47|nap time +57|494|65784|4294967329|27.37|7.74|true|irene king|2013-03-01 09:11:58.703239|93.23|kindergarten +29|503|65570|4294967389|97.33|36.21|true|sarah brown|2013-03-01 09:11:58.703261|96.28|religion +36|437|65600|4294967427|48.35|46.36|false|sarah hernandez|2013-03-01 09:11:58.703143|83.46|xylophone band +112|267|65661|4294967324|32.77|1.57|false|quinn allen|2013-03-01 09:11:58.703280|52.75|forestry +34|268|65548|4294967308|40.51|48.75|false|yuri polk|2013-03-01 09:11:58.703204|94.03|undecided +94|431|65784|4294967404|17.95|21.71|true|david van buren|2013-03-01 09:11:58.703282|63.99|biology +112|300|65635|4294967319|40.18|21.78|true|david xylophone|2013-03-01 09:11:58.703184|77.75|topology +103|432|65688|4294967453|66.41|47.47|true|zach white|2013-03-01 09:11:58.703260|99.89|forestry +111|363|65738|4294967436|58.55|46.00|true|david thompson|2013-03-01 09:11:58.703194|61.34|forestry +3|367|65553|4294967358|75.20|13.02|true|katie thompson|2013-03-01 09:11:58.703254|34.22|mathematics +32|275|65667|4294967448|89.49|22.05|true|zach van buren|2013-03-01 09:11:58.703290|13.38|kindergarten +3|468|65724|4294967511|76.67|33.01|true|yuri van buren|2013-03-01 09:11:58.703227|77.92|geology +99|363|65768|4294967394|94.55|40.70|false|mike thompson|2013-03-01 09:11:58.703273|10.14|mathematics +60|494|65580|4294967463|41.41|42.60|true|luke king|2013-03-01 09:11:58.703232|1.24|study skills +50|464|65689|4294967410|36.72|42.88|false|nick johnson|2013-03-01 09:11:58.703220|66.24|wind surfing +20|340|65667|4294967480|82.00|34.38|false|sarah falkner|2013-03-01 09:11:58.703309|65.87|study skills +85|356|65589|4294967365|33.55|2.84|true|yuri young|2013-03-01 09:11:58.703136|4.70|industrial engineering +117|473|65686|4294967375|54.67|2.25|true|wendy van buren|2013-03-01 09:11:58.703284|56.58|american history +70|476|65696|4294967335|21.18|26.09|true|holly quirinius|2013-03-01 09:11:58.703307|95.32|chemistry +58|432|65538|4294967504|74.99|25.90|false|ulysses robinson|2013-03-01 09:11:58.703076|67.55|study skills +1|501|65735|4294967395|63.14|4.38|false|yuri davidson|2013-03-01 09:11:58.703125|91.60|quiet hour +104|344|65609|4294967303|55.13|20.24|false|luke falkner|2013-03-01 09:11:58.703284|60.29|quiet hour +105|503|65565|4294967484|2.74|13.42|true|quinn zipper|2013-03-01 09:11:58.703177|44.53|history +18|475|65790|4294967332|32.87|32.00|false|victor steinbeck|2013-03-01 09:11:58.703310|69.60|mathematics +53|371|65751|4294967349|43.98|23.94|false|rachel king|2013-03-01 09:11:58.703259|66.53|philosophy +12|481|65757|4294967460|97.76|40.62|false|holly underhill|2013-03-01 09:11:58.703114|89.66|values clariffication +100|432|65592|4294967370|54.39|24.11|true|gabriella hernandez|2013-03-01 09:11:58.703086|61.02|zync studies +24|484|65703|4294967543|26.20|13.00|true|xavier van buren|2013-03-01 09:11:58.703143|25.25|undecided +85|258|65770|4294967541|13.38|21.04|false|alice nixon|2013-03-01 09:11:58.703111|21.29|undecided +66|402|65626|4294967510|16.21|21.00|true|luke young|2013-03-01 09:11:58.703166|43.01|opthamology +89|259|65703|4294967352|81.53|0.81|false|david carson|2013-03-01 09:11:58.703291|38.75|opthamology +2|467|65598|4294967299|62.77|7.10|false|priscilla nixon|2013-03-01 09:11:58.703202|3.84|yard duty +85|306|65554|4294967357|40.53|30.34|false|xavier falkner|2013-03-01 09:11:58.703320|42.44|biology +71|359|65780|4294967412|37.28|27.62|true|zach davidson|2013-03-01 09:11:58.703280|1.35|wind surfing +42|504|65740|4294967477|48.89|30.59|false|rachel young|2013-03-01 09:11:58.703071|21.43|religion +87|289|65670|4294967522|28.79|13.73|true|priscilla falkner|2013-03-01 09:11:58.703219|71.49|study skills +102|300|65637|4294967455|76.56|39.14|true|sarah steinbeck|2013-03-01 09:11:58.703242|58.20|philosophy +22|495|65575|4294967357|70.67|18.73|true|david thompson|2013-03-01 09:11:58.703196|75.55|opthamology +42|377|65758|4294967406|20.88|10.24|false|luke brown|2013-03-01 09:11:58.703130|23.63|chemistry +78|424|65784|4294967455|14.65|15.47|true|oscar zipper|2013-03-01 09:11:58.703106|93.08|linguistics +60|455|65541|4294967355|67.89|23.62|false|ulysses xylophone|2013-03-01 09:11:58.703102|54.96|undecided +2|365|65755|4294967310|23.68|20.13|true|fred underhill|2013-03-01 09:11:58.703314|91.94|topology +82|455|65773|4294967451|46.26|37.94|false|ulysses quirinius|2013-03-01 09:11:58.703141|54.33|quiet hour +123|502|65540|4294967469|37.15|48.72|true|calvin underhill|2013-03-01 09:11:58.703237|70.78|biology +105|280|65601|4294967462|26.21|31.91|false|gabriella ichabod|2013-03-01 09:11:58.703213|9.91|chemistry +109|379|65641|4294967535|15.30|9.03|false|sarah laertes|2013-03-01 09:11:58.703320|93.83|quiet hour +74|343|65570|4294967515|43.01|25.46|true|fred garcia|2013-03-01 09:11:58.703167|51.58|study skills +13|349|65779|4294967478|95.41|45.47|true|tom allen|2013-03-01 09:11:58.703323|59.35|opthamology +103|332|65724|4294967459|73.42|14.64|false|mike van buren|2013-03-01 09:11:58.703130|11.36|zync studies +34|411|65722|4294967451|89.49|34.97|true|calvin falkner|2013-03-01 09:11:58.703233|61.90|xylophone band +41|501|65670|4294967429|64.24|37.79|true|alice allen|2013-03-01 09:11:58.703135|62.52|study skills +118|306|65559|4294967305|22.63|31.12|true|gabriella ellison|2013-03-01 09:11:58.703205|23.46|mathematics +30|483|65687|4294967424|4.96|16.01|false|mike robinson|2013-03-01 09:11:58.703148|63.14|study skills +-3|344|65756|4294967378|52.13|18.95|true|victor thompson|2013-03-01 09:11:58.703299|81.06|topology +17|361|65585|4294967378|50.91|29.51|false|fred miller|2013-03-01 09:11:58.703082|27.57|kindergarten +100|259|65641|4294967453|20.65|25.90|false|ethan laertes|2013-03-01 09:11:58.703261|27.33|industrial engineering +53|417|65554|4294967333|39.74|0.94|true|xavier white|2013-03-01 09:11:58.703226|94.59|american history +41|471|65717|4294967320|81.31|34.56|true|rachel garcia|2013-03-01 09:11:58.703286|6.59|xylophone band +62|273|65719|4294967452|2.44|9.81|false|yuri young|2013-03-01 09:11:58.703104|52.81|chemistry +57|457|65746|4294967519|89.60|9.73|false|yuri hernandez|2013-03-01 09:11:58.703282|71.77|yard duty +61|446|65565|4294967313|50.26|6.41|true|bob van buren|2013-03-01 09:11:58.703131|22.62|values clariffication +89|507|65726|4294967475|85.51|37.24|true|mike laertes|2013-03-01 09:11:58.703142|7.22|wind surfing +110|293|65564|4294967433|20.85|30.31|true|sarah underhill|2013-03-01 09:11:58.703283|79.67|american history +39|350|65697|4294967503|89.58|23.08|true|irene ellison|2013-03-01 09:11:58.703274|19.02|geology +76|305|65737|4294967527|91.05|17.70|false|sarah falkner|2013-03-01 09:11:58.703121|37.42|history +0|386|65772|4294967316|9.38|20.50|false|alice van buren|2013-03-01 09:11:58.703120|87.96|topology +124|400|65733|4294967518|9.36|2.84|true|ethan quirinius|2013-03-01 09:11:58.703150|85.65|kindergarten +84|424|65789|4294967327|96.62|38.77|false|nick falkner|2013-03-01 09:11:58.703115|70.50|religion +66|301|65702|4294967357|68.92|32.30|true|fred miller|2013-03-01 09:11:58.703200|8.07|biology +59|445|65581|4294967535|11.66|40.11|true|fred polk|2013-03-01 09:11:58.703152|20.40|debate +86|408|65554|4294967387|31.38|16.08|false|quinn ovid|2013-03-01 09:11:58.703159|75.33|american history +119|337|65625|4294967381|39.94|2.14|true|calvin falkner|2013-03-01 09:11:58.703129|87.78|zync studies +50|275|65625|4294967502|23.37|37.47|true|alice ovid|2013-03-01 09:11:58.703200|29.84|chemistry +39|451|65568|4294967472|64.78|10.24|false|jessica ichabod|2013-03-01 09:11:58.703311|33.15|undecided +100|459|65701|4294967500|60.68|33.40|true|zach thompson|2013-03-01 09:11:58.703242|45.49|industrial engineering +2|261|65578|4294967336|32.67|49.54|false|zach zipper|2013-03-01 09:11:58.703124|72.00|industrial engineering +69|376|65566|4294967369|10.44|39.20|true|david zipper|2013-03-01 09:11:58.703150|59.42|kindergarten +22|326|65775|4294967330|50.95|37.48|false|wendy thompson|2013-03-01 09:11:58.703192|40.22|nap time +66|470|65668|4294967358|78.55|15.71|false|victor miller|2013-03-01 09:11:58.703078|33.33|linguistics +31|374|65649|4294967441|33.07|27.69|false|rachel robinson|2013-03-01 09:11:58.703130|41.90|yard duty +97|281|65774|4294967384|8.52|43.40|false|david thompson|2013-03-01 09:11:58.703214|9.64|mathematics +30|365|65655|4294967389|54.12|21.69|true|gabriella polk|2013-03-01 09:11:58.703232|10.08|debate +53|329|65682|4294967298|98.97|26.64|false|ulysses robinson|2013-03-01 09:11:58.703142|81.88|kindergarten +71|432|65747|4294967380|98.48|48.76|false|nick steinbeck|2013-03-01 09:11:58.703245|75.76|religion +50|440|65652|4294967443|89.24|2.20|false|wendy ovid|2013-03-01 09:11:58.703319|0.26|wind surfing +57|338|65561|4294967517|72.61|14.74|true|nick xylophone|2013-03-01 09:11:58.703096|3.08|mathematics +108|370|65558|4294967395|27.49|38.99|false|bob ichabod|2013-03-01 09:11:58.703091|81.43|study skills +115|377|65621|4294967533|18.98|28.67|false|ulysses hernandez|2013-03-01 09:11:58.703141|7.54|history +10|335|65571|4294967434|22.03|1.17|true|victor laertes|2013-03-01 09:11:58.703168|86.15|xylophone band +48|262|65771|4294967518|1.89|40.87|true|bob van buren|2013-03-01 09:11:58.703085|30.01|religion +56|455|65601|4294967365|84.71|24.58|true|fred zipper|2013-03-01 09:11:58.703070|71.16|debate +124|460|65784|4294967338|42.94|18.66|false|luke thompson|2013-03-01 09:11:58.703112|92.29|geology +15|283|65788|4294967313|5.62|25.30|true|calvin hernandez|2013-03-01 09:11:58.703124|9.47|wind surfing +63|333|65568|4294967377|62.15|45.02|true|rachel young|2013-03-01 09:11:58.703080|9.49|values clariffication +21|391|65776|4294967307|50.34|21.68|true|priscilla robinson|2013-03-01 09:11:58.703103|9.73|undecided +103|418|65721|4294967526|31.57|2.42|true|fred johnson|2013-03-01 09:11:58.703252|81.23|education +84|345|65773|4294967404|44.77|4.55|true|alice falkner|2013-03-01 09:11:58.703207|47.26|american history +120|347|65755|4294967307|70.35|47.56|true|yuri davidson|2013-03-01 09:11:58.703290|27.01|xylophone band +107|407|65571|4294967423|8.55|36.84|false|david falkner|2013-03-01 09:11:58.703200|49.01|opthamology +106|405|65741|4294967461|4.56|43.89|true|quinn steinbeck|2013-03-01 09:11:58.703287|71.61|study skills +35|454|65735|4294967335|15.69|1.90|true|gabriella miller|2013-03-01 09:11:58.703143|51.01|linguistics +15|334|65741|4294967303|81.22|46.98|false|xavier ovid|2013-03-01 09:11:58.703243|23.23|xylophone band +20|447|65752|4294967440|0.50|0.58|true|alice robinson|2013-03-01 09:11:58.703321|69.79|chemistry +42|292|65548|4294967550|63.89|33.11|true|calvin young|2013-03-01 09:11:58.703107|21.70|history +82|315|65596|4294967444|88.49|19.56|true|sarah brown|2013-03-01 09:11:58.703113|99.56|debate +78|311|65602|4294967507|46.98|23.95|true|sarah young|2013-03-01 09:11:58.703172|64.40|education +96|446|65605|4294967342|15.02|8.65|true|bob ovid|2013-03-01 09:11:58.703126|63.57|xylophone band +74|400|65761|4294967336|8.02|40.12|true|mike ellison|2013-03-01 09:11:58.703312|46.04|mathematics +40|487|65641|4294967376|99.96|11.43|false|priscilla underhill|2013-03-01 09:11:58.703164|11.99|undecided +40|480|65685|4294967437|70.19|13.12|true|quinn king|2013-03-01 09:11:58.703319|72.53|debate +99|492|65708|4294967445|73.16|43.32|false|ulysses quirinius|2013-03-01 09:11:58.703094|69.33|quiet hour +123|413|65595|4294967304|73.44|49.35|false|yuri miller|2013-03-01 09:11:58.703131|31.25|mathematics +108|274|65743|4294967369|95.13|2.31|false|oscar davidson|2013-03-01 09:11:58.703102|91.77|biology +97|428|65555|4294967550|49.56|21.34|false|yuri xylophone|2013-03-01 09:11:58.703154|18.14|wind surfing +118|347|65593|4294967489|87.50|31.34|true|xavier brown|2013-03-01 09:11:58.703232|99.31|values clariffication +106|442|65617|4294967438|65.82|9.45|true|xavier van buren|2013-03-01 09:11:58.703115|37.60|kindergarten +95|362|65732|4294967318|80.88|27.10|true|oscar miller|2013-03-01 09:11:58.703256|98.90|zync studies +-1|304|65753|4294967474|43.17|33.51|false|luke hernandez|2013-03-01 09:11:58.703316|3.22|education +2|422|65703|4294967539|2.21|8.17|false|gabriella quirinius|2013-03-01 09:11:58.703116|12.76|joggying +2|333|65586|4294967386|67.68|46.08|false|jessica thompson|2013-03-01 09:11:58.703143|76.58|yard duty +2|303|65733|4294967472|60.03|15.41|false|jessica nixon|2013-03-01 09:11:58.703214|71.30|history +72|377|65591|4294967349|98.78|9.76|false|victor underhill|2013-03-01 09:11:58.703177|87.21|religion +66|483|65693|4294967404|44.51|20.11|false|tom quirinius|2013-03-01 09:11:58.703218|21.66|religion +33|303|65659|4294967513|34.98|47.54|false|nick garcia|2013-03-01 09:11:58.703295|79.74|biology +43|479|65587|4294967507|39.20|9.80|true|alice white|2013-03-01 09:11:58.703117|42.06|biology +44|266|65675|4294967477|25.77|40.16|false|priscilla garcia|2013-03-01 09:11:58.703131|33.77|biology +77|370|65539|4294967506|43.51|8.68|false|xavier underhill|2013-03-01 09:11:58.703185|37.35|industrial engineering +99|398|65701|4294967393|19.43|45.73|false|mike garcia|2013-03-01 09:11:58.703117|98.97|forestry +-1|285|65621|4294967438|28.88|18.11|true|mike van buren|2013-03-01 09:11:58.703231|24.04|undecided +56|267|65721|4294967493|8.46|35.72|true|mike brown|2013-03-01 09:11:58.703085|80.88|forestry +86|402|65719|4294967524|97.36|34.28|true|gabriella polk|2013-03-01 09:11:58.703294|92.21|joggying +116|459|65727|4294967300|78.83|27.50|false|katie polk|2013-03-01 09:11:58.703116|3.55|quiet hour +22|257|65722|4294967508|79.05|13.12|false|zach underhill|2013-03-01 09:11:58.703163|58.23|values clariffication +30|312|65712|4294967550|85.24|46.81|true|victor miller|2013-03-01 09:11:58.703303|62.33|biology +44|498|65646|4294967405|94.37|8.18|false|tom xylophone|2013-03-01 09:11:58.703089|95.09|values clariffication +80|256|65693|4294967311|62.52|23.63|true|zach underhill|2013-03-01 09:11:58.703126|64.05|yard duty +46|269|65610|4294967321|18.57|4.95|false|holly robinson|2013-03-01 09:11:58.703238|47.38|american history +47|398|65674|4294967467|43.31|12.84|true|yuri xylophone|2013-03-01 09:11:58.703074|32.96|opthamology +55|440|65695|4294967445|19.14|14.88|false|ulysses zipper|2013-03-01 09:11:58.703302|4.77|industrial engineering +9|324|65720|4294967417|83.90|35.61|false|alice hernandez|2013-03-01 09:11:58.703277|38.50|forestry +94|361|65703|4294967357|92.00|24.48|true|bob thompson|2013-03-01 09:11:58.703126|21.48|nap time +37|434|65619|4294967510|3.39|9.24|false|mike robinson|2013-03-01 09:11:58.703200|75.73|topology +55|417|65742|4294967312|99.48|19.18|false|luke polk|2013-03-01 09:11:58.703133|30.27|geology +90|491|65614|4294967475|17.41|22.64|false|zach quirinius|2013-03-01 09:11:58.703132|0.92|kindergarten +122|305|65707|4294967387|11.83|18.14|true|zach miller|2013-03-01 09:11:58.703174|6.39|religion +72|418|65653|4294967380|94.33|20.60|false|sarah polk|2013-03-01 09:11:58.703304|93.05|yard duty +51|324|65669|4294967490|74.97|14.59|false|holly polk|2013-03-01 09:11:58.703220|92.01|zync studies +68|314|65671|4294967459|81.80|33.67|true|bob carson|2013-03-01 09:11:58.703313|77.52|quiet hour +90|493|65570|4294967364|69.84|17.45|false|nick van buren|2013-03-01 09:11:58.703161|92.27|xylophone band +11|299|65621|4294967454|41.92|31.10|true|bob young|2013-03-01 09:11:58.703168|10.39|religion +49|403|65763|4294967495|93.10|32.71|false|luke robinson|2013-03-01 09:11:58.703180|2.82|chemistry +69|355|65705|4294967468|96.95|32.13|false|katie white|2013-03-01 09:11:58.703236|80.77|topology +106|383|65558|4294967463|75.69|8.54|true|alice young|2013-03-01 09:11:58.703282|23.22|mathematics +3|290|65713|4294967468|21.73|35.99|true|rachel young|2013-03-01 09:11:58.703303|44.65|values clariffication +95|286|65672|4294967323|63.67|45.13|false|fred ovid|2013-03-01 09:11:58.703138|52.85|history +57|450|65688|4294967331|26.32|23.20|true|calvin carson|2013-03-01 09:11:58.703266|56.60|study skills +-1|292|65594|4294967363|16.22|40.15|false|yuri white|2013-03-01 09:11:58.703232|54.62|industrial engineering +118|375|65703|4294967306|50.45|40.25|false|yuri underhill|2013-03-01 09:11:58.703093|15.22|xylophone band +-1|450|65594|4294967347|27.60|2.43|false|luke ellison|2013-03-01 09:11:58.703142|7.24|study skills +17|413|65595|4294967422|42.16|6.11|false|ethan ellison|2013-03-01 09:11:58.703254|84.95|chemistry +20|340|65628|4294967452|53.82|20.26|true|sarah johnson|2013-03-01 09:11:58.703297|49.70|american history +57|390|65693|4294967527|15.44|23.27|false|victor white|2013-03-01 09:11:58.703104|0.31|quiet hour +1|448|65701|4294967363|20.67|16.96|false|priscilla robinson|2013-03-01 09:11:58.703077|50.61|values clariffication +74|495|65652|4294967335|75.50|36.41|false|oscar underhill|2013-03-01 09:11:58.703312|98.86|debate +47|403|65763|4294967310|7.15|14.82|true|katie hernandez|2013-03-01 09:11:58.703209|11.67|history +90|332|65630|4294967513|67.03|10.61|true|oscar hernandez|2013-03-01 09:11:58.703123|66.45|opthamology +104|353|65778|4294967325|38.34|8.74|true|nick robinson|2013-03-01 09:11:58.703252|37.34|xylophone band +12|256|65744|4294967496|38.16|45.71|true|alice ellison|2013-03-01 09:11:58.703282|12.47|debate +69|397|65731|4294967355|82.67|23.90|false|ethan king|2013-03-01 09:11:58.703295|73.16|geology +-3|350|65566|4294967434|23.22|6.68|true|nick robinson|2013-03-01 09:11:58.703147|23.68|education +21|424|65591|4294967492|32.31|46.45|false|priscilla johnson|2013-03-01 09:11:58.703297|31.70|chemistry +19|457|65558|4294967394|28.79|28.84|true|yuri laertes|2013-03-01 09:11:58.703302|72.99|yard duty +42|262|65641|4294967335|50.26|23.48|true|nick robinson|2013-03-01 09:11:58.703299|89.23|values clariffication +62|467|65601|4294967443|61.34|4.49|false|sarah davidson|2013-03-01 09:11:58.703113|84.43|kindergarten +93|445|65756|4294967347|94.83|27.67|false|katie allen|2013-03-01 09:11:58.703225|46.60|nap time +23|366|65625|4294967455|60.10|20.85|false|ethan ovid|2013-03-01 09:11:58.703322|77.00|values clariffication +30|383|65672|4294967438|81.36|23.44|true|bob king|2013-03-01 09:11:58.703283|5.46|biology +47|359|65570|4294967351|96.35|1.16|false|fred brown|2013-03-01 09:11:58.703096|28.55|opthamology +69|261|65681|4294967540|11.46|5.34|false|calvin nixon|2013-03-01 09:11:58.703178|15.23|forestry +1|327|65600|4294967330|69.34|48.84|true|ulysses miller|2013-03-01 09:11:58.703086|16.25|chemistry +73|400|65751|4294967353|92.94|4.78|true|alice laertes|2013-03-01 09:11:58.703136|97.74|joggying +11|415|65571|4294967492|70.34|10.47|true|katie miller|2013-03-01 09:11:58.703208|2.41|study skills +7|287|65640|4294967415|88.29|29.72|false|rachel ovid|2013-03-01 09:11:58.703255|54.30|debate +118|328|65684|4294967492|99.22|6.17|true|nick hernandez|2013-03-01 09:11:58.703121|43.26|philosophy +107|351|65553|4294967393|65.59|11.76|false|xavier robinson|2013-03-01 09:11:58.703173|5.14|chemistry +121|436|65737|4294967486|2.85|26.10|false|priscilla ovid|2013-03-01 09:11:58.703266|4.78|zync studies +-1|306|65576|4294967379|84.41|37.66|true|victor quirinius|2013-03-01 09:11:58.703148|3.13|religion +65|444|65627|4294967305|61.30|8.78|true|xavier white|2013-03-01 09:11:58.703248|71.83|biology +93|437|65754|4294967404|29.31|36.93|false|xavier robinson|2013-03-01 09:11:58.703254|2.29|quiet hour +43|372|65718|4294967352|76.61|1.77|true|victor brown|2013-03-01 09:11:58.703138|37.15|undecided +91|320|65780|4294967520|5.47|33.59|true|david thompson|2013-03-01 09:11:58.703128|47.31|chemistry +88|424|65537|4294967356|19.00|6.33|true|katie thompson|2013-03-01 09:11:58.703133|52.90|joggying +109|398|65622|4294967434|35.13|49.74|false|victor garcia|2013-03-01 09:11:58.703285|82.91|xylophone band +0|379|65717|4294967524|58.34|27.07|true|wendy ichabod|2013-03-01 09:11:58.703168|21.28|values clariffication +117|379|65752|4294967503|39.20|18.98|false|holly xylophone|2013-03-01 09:11:58.703132|61.11|wind surfing +83|322|65629|4294967531|52.30|32.99|true|david underhill|2013-03-01 09:11:58.703310|90.73|joggying +92|313|65678|4294967300|56.00|31.76|true|calvin van buren|2013-03-01 09:11:58.703173|50.86|values clariffication +53|360|65593|4294967486|79.19|40.08|true|wendy garcia|2013-03-01 09:11:58.703180|11.69|topology +114|446|65759|4294967469|38.83|39.71|true|alice garcia|2013-03-01 09:11:58.703179|27.58|american history +87|345|65686|4294967503|91.32|45.37|true|mike garcia|2013-03-01 09:11:58.703131|89.86|kindergarten +96|335|65786|4294967361|85.37|44.08|false|rachel thompson|2013-03-01 09:11:58.703184|40.21|philosophy +-2|493|65784|4294967415|24.44|30.58|true|wendy young|2013-03-01 09:11:58.703177|55.86|religion +67|427|65716|4294967424|85.15|33.07|false|luke brown|2013-03-01 09:11:58.703278|20.23|linguistics +117|409|65581|4294967364|30.84|7.52|true|luke king|2013-03-01 09:11:58.703156|48.02|kindergarten +36|289|65735|4294967507|26.73|15.02|false|david robinson|2013-03-01 09:11:58.703143|10.86|xylophone band +105|480|65675|4294967523|7.06|50.00|true|nick young|2013-03-01 09:11:58.703310|79.85|opthamology +111|287|65791|4294967524|83.08|18.77|true|holly johnson|2013-03-01 09:11:58.703096|55.40|biology +72|257|65719|4294967521|62.79|45.53|false|wendy brown|2013-03-01 09:11:58.703209|35.11|study skills +110|360|65702|4294967311|92.54|48.70|true|david brown|2013-03-01 09:11:58.703100|91.09|zync studies +14|434|65585|4294967326|24.77|6.86|true|gabriella thompson|2013-03-01 09:11:58.703321|2.06|xylophone band +124|482|65785|4294967379|73.80|40.91|false|irene nixon|2013-03-01 09:11:58.703263|49.24|history +46|258|65551|4294967513|88.97|6.60|true|luke ovid|2013-03-01 09:11:58.703203|23.67|undecided +27|337|65616|4294967296|55.81|17.63|true|calvin falkner|2013-03-01 09:11:58.703270|63.74|forestry +61|486|65578|4294967347|25.14|38.90|true|gabriella laertes|2013-03-01 09:11:58.703247|2.87|study skills +33|477|65768|4294967432|67.97|30.02|false|ulysses young|2013-03-01 09:11:58.703089|38.26|opthamology +118|266|65786|4294967343|1.04|20.16|false|fred van buren|2013-03-01 09:11:58.703221|60.92|nap time +99|301|65541|4294967492|22.83|14.76|true|bob miller|2013-03-01 09:11:58.703166|17.19|forestry +102|377|65604|4294967398|28.47|15.60|false|david allen|2013-03-01 09:11:58.703146|2.95|philosophy +90|306|65671|4294967332|31.20|17.17|false|calvin polk|2013-03-01 09:11:58.703097|47.56|values clariffication +63|400|65691|4294967537|39.85|20.86|true|quinn young|2013-03-01 09:11:58.703187|97.70|chemistry +92|353|65691|4294967443|20.32|3.23|true|irene king|2013-03-01 09:11:58.703238|7.92|zync studies +50|338|65677|4294967419|82.89|37.31|false|xavier thompson|2013-03-01 09:11:58.703264|27.88|zync studies +55|504|65665|4294967550|77.47|44.52|false|sarah thompson|2013-03-01 09:11:58.703163|15.50|nap time +39|449|65576|4294967533|71.25|6.03|true|jessica allen|2013-03-01 09:11:58.703319|14.52|education +81|364|65544|4294967304|15.32|14.34|true|fred brown|2013-03-01 09:11:58.703192|5.97|religion +63|290|65600|4294967447|51.42|9.73|false|ethan white|2013-03-01 09:11:58.703298|61.88|nap time +91|393|65662|4294967504|51.40|28.95|true|zach ellison|2013-03-01 09:11:58.703277|9.05|kindergarten +41|377|65713|4294967476|40.53|12.39|true|tom underhill|2013-03-01 09:11:58.703286|9.89|nap time +70|265|65638|4294967398|11.30|6.77|false|sarah polk|2013-03-01 09:11:58.703238|19.82|nap time +62|342|65645|4294967439|55.53|34.85|true|ulysses allen|2013-03-01 09:11:58.703317|18.37|undecided +48|402|65659|4294967330|5.25|48.30|true|tom ellison|2013-03-01 09:11:58.703212|88.60|history +33|301|65542|4294967492|92.57|9.78|false|tom ichabod|2013-03-01 09:11:58.703182|41.27|chemistry +107|462|65751|4294967545|88.75|12.77|false|fred miller|2013-03-01 09:11:58.703075|22.18|kindergarten +42|256|65763|4294967496|24.89|22.22|true|holly xylophone|2013-03-01 09:11:58.703077|80.11|opthamology +0|353|65632|4294967468|1.44|31.99|false|fred ellison|2013-03-01 09:11:58.703120|6.37|forestry +19|471|65780|4294967342|69.73|21.43|true|sarah ovid|2013-03-01 09:11:58.703079|33.83|geology +24|460|65754|4294967503|83.62|9.26|false|ulysses davidson|2013-03-01 09:11:58.703198|23.20|chemistry +1|472|65641|4294967301|27.78|36.62|true|gabriella zipper|2013-03-01 09:11:58.703203|86.61|wind surfing +86|278|65551|4294967435|96.54|0.29|true|jessica ichabod|2013-03-01 09:11:58.703275|47.23|debate +98|293|65641|4294967345|24.78|43.71|false|nick robinson|2013-03-01 09:11:58.703122|97.22|debate +90|279|65776|4294967383|22.07|14.55|true|calvin zipper|2013-03-01 09:11:58.703226|62.48|debate +22|438|65606|4294967471|81.90|10.88|true|alice johnson|2013-03-01 09:11:58.703287|71.56|opthamology +113|443|65625|4294967548|87.53|9.22|true|holly miller|2013-03-01 09:11:58.703297|46.15|mathematics +86|361|65613|4294967403|10.74|49.90|true|wendy robinson|2013-03-01 09:11:58.703102|15.60|mathematics +103|310|65688|4294967474|54.72|23.66|false|david van buren|2013-03-01 09:11:58.703220|16.59|industrial engineering +8|303|65692|4294967427|70.17|15.14|true|ethan nixon|2013-03-01 09:11:58.703085|78.05|joggying +4|358|65695|4294967431|26.22|20.25|false|fred white|2013-03-01 09:11:58.703323|40.97|forestry +102|278|65538|4294967496|34.95|25.78|true|gabriella johnson|2013-03-01 09:11:58.703101|1.20|chemistry +69|470|65547|4294967493|58.91|7.92|false|calvin quirinius|2013-03-01 09:11:58.703083|43.25|kindergarten +69|301|65583|4294967480|11.05|36.13|false|jessica steinbeck|2013-03-01 09:11:58.703257|4.30|history +76|482|65547|4294967497|32.63|26.75|true|fred white|2013-03-01 09:11:58.703223|44.80|xylophone band +94|454|65775|4294967352|89.36|46.70|false|alice johnson|2013-03-01 09:11:58.703222|26.67|quiet hour +45|342|65602|4294967377|2.24|16.87|true|sarah van buren|2013-03-01 09:11:58.703078|9.02|opthamology +98|433|65670|4294967548|58.03|32.77|false|jessica thompson|2013-03-01 09:11:58.703250|57.59|biology +106|402|65693|4294967359|63.32|31.51|false|holly van buren|2013-03-01 09:11:58.703219|46.72|chemistry +27|266|65660|4294967508|94.16|18.90|true|jessica young|2013-03-01 09:11:58.703123|76.29|philosophy +80|503|65573|4294967452|85.79|33.07|true|xavier zipper|2013-03-01 09:11:58.703220|71.66|nap time +108|285|65669|4294967517|28.00|46.81|false|gabriella xylophone|2013-03-01 09:11:58.703167|19.18|kindergarten +88|375|65662|4294967486|15.33|32.15|false|quinn ichabod|2013-03-01 09:11:58.703167|75.04|kindergarten +46|403|65638|4294967305|16.94|19.71|false|oscar white|2013-03-01 09:11:58.703083|81.49|industrial engineering +41|463|65779|4294967419|10.38|42.91|true|rachel allen|2013-03-01 09:11:58.703194|13.91|study skills +64|417|65774|4294967422|82.91|9.63|true|tom xylophone|2013-03-01 09:11:58.703213|47.91|kindergarten +28|411|65623|4294967444|71.82|33.66|true|holly falkner|2013-03-01 09:11:58.703221|81.85|biology +100|356|65623|4294967515|54.36|34.44|false|luke zipper|2013-03-01 09:11:58.703070|30.77|biology +84|452|65605|4294967496|34.19|49.00|true|gabriella ellison|2013-03-01 09:11:58.703233|49.93|opthamology +68|445|65646|4294967480|14.51|49.03|false|katie king|2013-03-01 09:11:58.703108|75.29|religion +93|325|65692|4294967392|20.07|20.19|false|luke hernandez|2013-03-01 09:11:58.703257|33.12|industrial engineering +70|296|65569|4294967415|88.54|49.58|false|alice hernandez|2013-03-01 09:11:58.703228|69.33|study skills +88|294|65585|4294967427|15.16|12.19|false|ulysses garcia|2013-03-01 09:11:58.703123|51.14|yard duty +99|495|65561|4294967538|6.70|12.09|false|gabriella ellison|2013-03-01 09:11:58.703121|69.68|philosophy +42|441|65545|4294967337|97.81|30.03|true|yuri steinbeck|2013-03-01 09:11:58.703133|13.04|opthamology +76|468|65547|4294967423|55.27|25.05|false|xavier robinson|2013-03-01 09:11:58.703139|28.55|education +11|460|65657|4294967359|56.47|23.94|true|wendy underhill|2013-03-01 09:11:58.703231|62.73|geology +74|504|65673|4294967391|75.06|32.96|true|bob hernandez|2013-03-01 09:11:58.703085|72.44|industrial engineering +106|353|65730|4294967366|48.70|40.96|false|holly ellison|2013-03-01 09:11:58.703108|76.80|values clariffication +29|307|65548|4294967528|86.07|30.28|true|mike davidson|2013-03-01 09:11:58.703156|50.72|joggying +74|422|65742|4294967437|98.49|21.89|false|quinn quirinius|2013-03-01 09:11:58.703317|7.39|xylophone band +31|341|65774|4294967402|8.27|36.66|true|priscilla nixon|2013-03-01 09:11:58.703233|33.73|chemistry +121|431|65743|4294967324|92.88|35.13|false|gabriella garcia|2013-03-01 09:11:58.703270|10.78|topology +67|263|65775|4294967365|11.34|41.68|false|priscilla falkner|2013-03-01 09:11:58.703220|99.03|history +113|311|65666|4294967427|25.58|24.50|true|victor thompson|2013-03-01 09:11:58.703148|97.86|wind surfing +106|464|65752|4294967420|60.99|32.68|false|alice johnson|2013-03-01 09:11:58.703088|0.58|kindergarten +51|271|65773|4294967484|43.32|25.31|false|wendy davidson|2013-03-01 09:11:58.703219|83.94|religion +7|469|65704|4294967508|36.57|11.40|false|xavier ichabod|2013-03-01 09:11:58.703133|68.28|study skills +1|376|65606|4294967463|81.04|22.55|true|gabriella underhill|2013-03-01 09:11:58.703115|4.60|undecided +35|337|65663|4294967401|72.71|14.52|false|alice underhill|2013-03-01 09:11:58.703134|35.21|philosophy +36|426|65766|4294967541|23.98|36.94|false|wendy laertes|2013-03-01 09:11:58.703306|99.67|debate +16|400|65639|4294967481|28.84|29.89|true|bob ichabod|2013-03-01 09:11:58.703127|36.38|kindergarten +34|507|65724|4294967480|8.64|28.39|false|irene van buren|2013-03-01 09:11:58.703098|50.14|values clariffication +95|330|65738|4294967335|12.35|44.00|true|luke xylophone|2013-03-01 09:11:58.703082|66.11|mathematics +102|499|65778|4294967384|70.51|2.72|true|wendy allen|2013-03-01 09:11:58.703147|81.87|chemistry +47|435|65578|4294967336|75.61|17.20|true|gabriella davidson|2013-03-01 09:11:58.703204|21.97|wind surfing +53|355|65547|4294967357|63.99|45.72|true|oscar laertes|2013-03-01 09:11:58.703297|26.70|yard duty +87|364|65650|4294967467|73.10|11.42|true|mike garcia|2013-03-01 09:11:58.703114|99.69|linguistics +39|287|65733|4294967363|19.49|39.11|true|gabriella zipper|2013-03-01 09:11:58.703203|54.32|debate +72|390|65749|4294967415|3.00|40.44|false|luke johnson|2013-03-01 09:11:58.703303|38.34|debate +102|386|65714|4294967496|64.28|35.79|false|sarah brown|2013-03-01 09:11:58.703092|58.38|quiet hour +122|377|65764|4294967550|67.17|25.33|false|wendy king|2013-03-01 09:11:58.703319|45.29|american history +27|267|65623|4294967551|16.85|36.59|true|tom ellison|2013-03-01 09:11:58.703236|97.41|opthamology +114|392|65566|4294967378|65.65|17.45|true|wendy carson|2013-03-01 09:11:58.703124|58.90|debate +37|433|65609|4294967375|69.54|17.79|true|gabriella van buren|2013-03-01 09:11:58.703320|54.53|history +6|359|65748|4294967538|94.96|48.20|false|calvin robinson|2013-03-01 09:11:58.703245|31.17|forestry +15|318|65674|4294967403|1.67|7.99|false|fred ellison|2013-03-01 09:11:58.703280|91.00|wind surfing +2|351|65741|4294967441|45.21|29.08|true|zach laertes|2013-03-01 09:11:58.703311|34.44|forestry +60|308|65776|4294967321|13.92|4.35|false|alice young|2013-03-01 09:11:58.703280|25.73|kindergarten +29|294|65691|4294967301|88.44|6.53|true|irene laertes|2013-03-01 09:11:58.703291|14.40|quiet hour +48|498|65587|4294967518|45.78|1.80|true|gabriella brown|2013-03-01 09:11:58.703132|77.66|zync studies +93|422|65556|4294967396|19.63|24.99|true|luke miller|2013-03-01 09:11:58.703088|27.45|religion +60|447|65694|4294967304|30.33|34.91|false|nick ellison|2013-03-01 09:11:58.703121|58.92|mathematics +19|442|65745|4294967382|53.15|16.78|false|bob zipper|2013-03-01 09:11:58.703085|12.10|study skills +108|358|65592|4294967333|55.45|49.61|false|bob underhill|2013-03-01 09:11:58.703309|5.02|xylophone band +7|389|65556|4294967510|90.34|20.64|false|oscar hernandez|2013-03-01 09:11:58.703103|53.64|undecided +19|366|65669|4294967385|36.74|32.82|true|xavier johnson|2013-03-01 09:11:58.703303|73.09|opthamology +33|392|65737|4294967490|58.59|30.74|true|oscar falkner|2013-03-01 09:11:58.703307|7.06|topology +72|485|65713|4294967504|65.76|29.79|true|quinn garcia|2013-03-01 09:11:58.703087|31.43|forestry +4|504|65736|4294967520|79.46|45.07|true|jessica johnson|2013-03-01 09:11:58.703072|45.99|industrial engineering +91|443|65609|4294967320|77.60|24.07|true|holly ellison|2013-03-01 09:11:58.703194|2.04|mathematics +4|494|65740|4294967435|51.47|16.59|false|nick hernandez|2013-03-01 09:11:58.703166|5.31|study skills +80|295|65621|4294967344|83.53|41.47|false|bob steinbeck|2013-03-01 09:11:58.703311|3.75|forestry +69|285|65760|4294967425|34.08|12.37|true|katie davidson|2013-03-01 09:11:58.703094|81.51|quiet hour +-1|349|65723|4294967508|38.60|18.34|true|ulysses ichabod|2013-03-01 09:11:58.703193|75.78|geology +110|413|65612|4294967320|91.57|4.30|true|david steinbeck|2013-03-01 09:11:58.703176|48.36|topology +53|287|65692|4294967481|59.83|45.89|false|tom johnson|2013-03-01 09:11:58.703317|41.37|yard duty +122|506|65626|4294967459|99.77|1.42|true|katie carson|2013-03-01 09:11:58.703230|35.94|zync studies +104|256|65613|4294967401|78.27|15.35|true|xavier van buren|2013-03-01 09:11:58.703073|96.89|education +20|331|65633|4294967478|34.72|36.66|false|katie king|2013-03-01 09:11:58.703279|7.95|linguistics +15|326|65616|4294967473|97.76|44.28|false|oscar steinbeck|2013-03-01 09:11:58.703111|85.39|yard duty +59|399|65753|4294967484|50.43|41.55|true|zach hernandez|2013-03-01 09:11:58.703176|56.73|xylophone band +37|329|65668|4294967516|44.71|12.31|false|mike steinbeck|2013-03-01 09:11:58.703127|33.45|yard duty +-2|410|65609|4294967401|12.52|29.60|true|rachel underhill|2013-03-01 09:11:58.703259|15.98|forestry +3|497|65672|4294967376|26.42|24.18|true|quinn quirinius|2013-03-01 09:11:58.703086|96.89|religion +40|297|65540|4294967405|86.59|14.12|true|yuri allen|2013-03-01 09:11:58.703155|69.33|nap time +-1|421|65542|4294967456|70.77|0.21|true|yuri zipper|2013-03-01 09:11:58.703227|75.02|linguistics +26|364|65737|4294967352|83.47|8.04|true|tom allen|2013-03-01 09:11:58.703312|7.31|biology +13|340|65603|4294967495|96.96|18.56|false|priscilla thompson|2013-03-01 09:11:58.703090|13.44|mathematics +42|324|65644|4294967443|39.22|42.31|false|holly thompson|2013-03-01 09:11:58.703103|57.21|topology +27|448|65564|4294967495|66.16|49.95|true|irene laertes|2013-03-01 09:11:58.703195|23.09|values clariffication +16|496|65684|4294967414|55.01|45.96|true|katie white|2013-03-01 09:11:58.703138|78.30|geology +111|300|65552|4294967510|81.52|34.54|true|holly xylophone|2013-03-01 09:11:58.703221|97.65|linguistics +64|369|65749|4294967441|41.22|4.22|false|rachel thompson|2013-03-01 09:11:58.703222|80.81|industrial engineering +116|433|65721|4294967537|49.73|0.99|false|calvin zipper|2013-03-01 09:11:58.703303|12.50|yard duty +120|284|65575|4294967297|1.26|1.88|false|zach allen|2013-03-01 09:11:58.703178|88.65|debate +70|325|65676|4294967502|80.96|21.07|true|xavier polk|2013-03-01 09:11:58.703184|36.85|nap time +60|281|65744|4294967485|53.70|16.75|true|xavier miller|2013-03-01 09:11:58.703087|25.52|values clariffication +-3|307|65634|4294967546|90.30|28.44|false|irene underhill|2013-03-01 09:11:58.703298|85.38|forestry +74|365|65723|4294967420|25.84|3.54|true|irene ichabod|2013-03-01 09:11:58.703164|59.44|xylophone band +-2|330|65695|4294967367|72.19|7.31|true|tom steinbeck|2013-03-01 09:11:58.703225|77.87|history +19|265|65786|4294967358|59.09|0.07|false|victor johnson|2013-03-01 09:11:58.703264|94.13|yard duty +114|376|65681|4294967490|49.51|26.49|false|alice nixon|2013-03-01 09:11:58.703267|63.99|wind surfing +92|278|65588|4294967522|44.55|47.90|false|jessica robinson|2013-03-01 09:11:58.703260|44.95|quiet hour +108|404|65716|4294967507|51.03|6.23|false|quinn ellison|2013-03-01 09:11:58.703077|5.89|history +115|278|65637|4294967496|58.26|45.34|false|victor white|2013-03-01 09:11:58.703207|83.83|chemistry +-2|371|65632|4294967428|44.88|8.76|true|fred ichabod|2013-03-01 09:11:58.703315|94.09|study skills +9|505|65593|4294967322|87.45|45.23|false|wendy underhill|2013-03-01 09:11:58.703114|81.20|xylophone band +5|493|65630|4294967413|51.38|40.12|true|gabriella steinbeck|2013-03-01 09:11:58.703109|1.31|topology +-2|344|65786|4294967538|33.74|20.46|true|jessica white|2013-03-01 09:11:58.703317|13.04|yard duty +102|415|65691|4294967456|76.55|38.78|true|tom nixon|2013-03-01 09:11:58.703206|55.12|wind surfing +20|284|65749|4294967387|45.04|48.47|false|mike ellison|2013-03-01 09:11:58.703173|38.64|forestry +62|370|65584|4294967481|46.63|7.33|true|holly xylophone|2013-03-01 09:11:58.703201|33.72|kindergarten +16|460|65733|4294967340|66.16|11.49|false|victor ovid|2013-03-01 09:11:58.703293|88.34|american history +41|418|65680|4294967501|86.07|16.07|false|irene underhill|2013-03-01 09:11:58.703153|58.85|religion +34|359|65613|4294967413|7.89|21.58|true|irene allen|2013-03-01 09:11:58.703213|32.06|debate +72|282|65703|4294967447|64.10|22.26|true|wendy young|2013-03-01 09:11:58.703151|76.87|religion +63|324|65677|4294967428|15.92|9.50|false|irene nixon|2013-03-01 09:11:58.703246|26.66|history +9|338|65582|4294967306|40.52|5.98|false|sarah van buren|2013-03-01 09:11:58.703205|82.96|biology +37|487|65555|4294967412|74.77|13.12|true|luke king|2013-03-01 09:11:58.703084|65.24|zync studies +110|377|65655|4294967444|68.44|22.65|false|mike van buren|2013-03-01 09:11:58.703092|53.65|philosophy +56|259|65576|4294967480|7.41|15.49|false|rachel steinbeck|2013-03-01 09:11:58.703151|73.07|xylophone band +31|258|65617|4294967510|60.34|13.81|true|jessica laertes|2013-03-01 09:11:58.703312|10.94|forestry +72|481|65782|4294967318|28.37|13.70|true|nick steinbeck|2013-03-01 09:11:58.703288|13.06|values clariffication +17|458|65689|4294967493|94.85|1.34|false|fred ovid|2013-03-01 09:11:58.703240|45.06|quiet hour +96|384|65637|4294967383|77.65|48.26|false|alice quirinius|2013-03-01 09:11:58.703157|79.28|industrial engineering +64|275|65613|4294967424|34.34|16.34|false|gabriella ichabod|2013-03-01 09:11:58.703169|74.84|undecided +124|364|65789|4294967310|3.68|38.59|true|ethan thompson|2013-03-01 09:11:58.703110|69.02|undecided +72|363|65771|4294967464|37.81|31.49|true|yuri allen|2013-03-01 09:11:58.703110|22.15|opthamology +33|265|65668|4294967521|76.66|2.63|false|wendy ovid|2013-03-01 09:11:58.703260|33.20|opthamology +6|307|65554|4294967431|90.06|26.38|true|alice robinson|2013-03-01 09:11:58.703291|22.45|mathematics +112|363|65682|4294967316|98.37|39.16|true|oscar steinbeck|2013-03-01 09:11:58.703264|10.63|debate +93|310|65596|4294967532|26.50|17.39|false|nick robinson|2013-03-01 09:11:58.703236|84.99|linguistics +9|344|65674|4294967339|31.49|18.16|false|bob allen|2013-03-01 09:11:58.703200|54.69|geology +12|453|65661|4294967411|13.01|28.34|false|calvin thompson|2013-03-01 09:11:58.703225|55.23|topology +102|260|65764|4294967352|99.78|9.37|false|xavier thompson|2013-03-01 09:11:58.703213|60.57|joggying +103|377|65560|4294967467|42.43|15.03|true|bob king|2013-03-01 09:11:58.703265|17.49|religion +109|335|65784|4294967417|11.21|30.57|false|holly miller|2013-03-01 09:11:58.703262|97.82|chemistry +112|347|65713|4294967441|33.04|35.49|true|nick davidson|2013-03-01 09:11:58.703323|8.57|quiet hour +86|263|65701|4294967544|28.37|15.98|true|irene hernandez|2013-03-01 09:11:58.703074|46.39|study skills +0|373|65718|4294967469|60.38|29.28|true|alice king|2013-03-01 09:11:58.703167|92.22|undecided +100|365|65710|4294967313|54.80|41.12|true|luke garcia|2013-03-01 09:11:58.703136|24.84|zync studies +91|333|65677|4294967418|21.26|31.98|false|ethan van buren|2013-03-01 09:11:58.703311|70.90|zync studies +29|477|65676|4294967305|21.89|43.80|true|katie van buren|2013-03-01 09:11:58.703217|86.29|kindergarten +36|408|65736|4294967465|18.42|7.13|false|david allen|2013-03-01 09:11:58.703097|84.13|joggying +21|432|65783|4294967357|49.12|1.06|false|holly davidson|2013-03-01 09:11:58.703218|42.09|chemistry +124|435|65683|4294967388|65.84|25.12|true|calvin robinson|2013-03-01 09:11:58.703180|63.02|yard duty +117|484|65605|4294967523|10.15|9.53|true|priscilla brown|2013-03-01 09:11:58.703137|53.93|nap time +94|487|65569|4294967470|47.79|8.61|true|holly white|2013-03-01 09:11:58.703148|77.18|topology +111|497|65557|4294967545|69.12|21.49|false|priscilla zipper|2013-03-01 09:11:58.703083|13.07|religion +99|365|65754|4294967389|20.04|46.17|false|calvin garcia|2013-03-01 09:11:58.703082|24.26|opthamology +27|344|65714|4294967410|54.95|10.60|true|bob zipper|2013-03-01 09:11:58.703081|99.90|values clariffication +27|339|65644|4294967325|18.70|49.99|false|xavier robinson|2013-03-01 09:11:58.703268|72.83|nap time +5|506|65727|4294967496|17.30|30.78|true|rachel garcia|2013-03-01 09:11:58.703169|88.95|biology +105|508|65769|4294967442|40.91|0.07|true|ethan carson|2013-03-01 09:11:58.703288|76.83|religion +49|309|65703|4294967418|54.62|37.49|false|rachel ovid|2013-03-01 09:11:58.703234|32.52|mathematics +25|474|65638|4294967518|82.72|25.87|true|fred robinson|2013-03-01 09:11:58.703160|34.42|american history +96|471|65652|4294967401|24.60|14.69|true|gabriella polk|2013-03-01 09:11:58.703262|21.72|opthamology +95|317|65663|4294967409|61.84|27.27|true|jessica xylophone|2013-03-01 09:11:58.703309|64.95|biology +3|428|65666|4294967452|37.98|40.97|false|nick ovid|2013-03-01 09:11:58.703286|85.88|history +106|437|65624|4294967490|17.70|29.66|true|tom falkner|2013-03-01 09:11:58.703099|26.40|forestry +64|298|65674|4294967404|99.80|36.58|false|jessica thompson|2013-03-01 09:11:58.703168|50.21|xylophone band +97|359|65717|4294967465|67.58|3.26|true|nick xylophone|2013-03-01 09:11:58.703169|90.19|american history +109|277|65620|4294967305|97.25|7.80|true|nick quirinius|2013-03-01 09:11:58.703226|27.72|undecided +93|263|65725|4294967341|6.06|4.12|false|calvin king|2013-03-01 09:11:58.703299|32.44|values clariffication +108|383|65629|4294967510|39.55|47.67|false|jessica zipper|2013-03-01 09:11:58.703133|74.23|nap time +89|463|65537|4294967493|64.82|13.79|true|ethan white|2013-03-01 09:11:58.703243|89.52|nap time +88|372|65645|4294967358|34.48|11.18|true|quinn thompson|2013-03-01 09:11:58.703168|84.86|forestry +123|432|65626|4294967435|2.39|16.49|true|david white|2013-03-01 09:11:58.703136|61.24|joggying +57|486|65551|4294967397|36.11|9.88|true|katie xylophone|2013-03-01 09:11:58.703142|57.10|zync studies +59|343|65787|4294967312|66.89|6.54|true|mike laertes|2013-03-01 09:11:58.703209|27.56|xylophone band +74|267|65671|4294967409|21.14|14.64|true|priscilla miller|2013-03-01 09:11:58.703197|89.06|undecided +25|336|65587|4294967336|71.01|14.90|true|tom ichabod|2013-03-01 09:11:58.703127|74.32|zync studies +48|346|65712|4294967315|45.01|16.08|true|zach brown|2013-03-01 09:11:58.703108|21.68|zync studies +84|385|65776|4294967452|35.80|32.13|false|xavier zipper|2013-03-01 09:11:58.703311|99.46|education +58|389|65766|4294967416|95.55|20.62|false|sarah miller|2013-03-01 09:11:58.703215|70.92|history +22|403|65565|4294967381|99.65|35.42|false|yuri johnson|2013-03-01 09:11:58.703154|94.47|geology +55|428|65733|4294967535|99.54|5.35|false|jessica king|2013-03-01 09:11:58.703233|30.30|forestry +117|410|65706|4294967391|50.15|0.21|false|quinn johnson|2013-03-01 09:11:58.703248|65.99|yard duty +95|423|65573|4294967378|47.59|17.37|true|alice robinson|2013-03-01 09:11:58.703133|54.57|linguistics +87|332|65748|4294967320|19.83|41.67|false|fred ellison|2013-03-01 09:11:58.703289|79.02|mathematics +114|263|65674|4294967405|84.44|33.18|true|victor van buren|2013-03-01 09:11:58.703092|63.74|linguistics +5|369|65780|4294967488|92.02|38.59|true|zach polk|2013-03-01 09:11:58.703271|67.29|yard duty +-3|430|65667|4294967469|65.50|40.46|true|yuri xylophone|2013-03-01 09:11:58.703258|30.94|american history +120|264|65769|4294967486|89.97|41.18|false|xavier hernandez|2013-03-01 09:11:58.703140|66.89|philosophy +107|317|65634|4294967488|5.68|18.89|false|priscilla ichabod|2013-03-01 09:11:58.703196|39.42|joggying +29|386|65723|4294967328|71.48|6.13|false|ulysses ichabod|2013-03-01 09:11:58.703215|86.65|xylophone band +22|434|65768|4294967543|44.25|27.56|false|tom polk|2013-03-01 09:11:58.703306|12.30|kindergarten +-1|274|65755|4294967300|22.01|35.52|false|oscar king|2013-03-01 09:11:58.703141|33.35|chemistry +6|365|65603|4294967522|18.51|5.60|false|gabriella king|2013-03-01 09:11:58.703104|34.20|geology +97|414|65757|4294967325|31.82|22.37|false|rachel nixon|2013-03-01 09:11:58.703127|61.00|nap time +72|448|65538|4294967524|80.09|7.73|true|luke brown|2013-03-01 09:11:58.703090|95.81|american history +51|280|65589|4294967486|57.46|23.35|false|zach xylophone|2013-03-01 09:11:58.703299|11.54|education +12|447|65583|4294967389|0.98|29.79|true|yuri polk|2013-03-01 09:11:58.703305|1.89|wind surfing +-1|360|65539|4294967464|4.08|39.51|false|oscar davidson|2013-03-01 09:11:58.703144|59.47|nap time +0|380|65569|4294967425|0.94|28.93|false|sarah robinson|2013-03-01 09:11:58.703176|88.81|xylophone band +66|478|65669|4294967339|23.66|38.34|true|yuri carson|2013-03-01 09:11:58.703228|64.68|opthamology +12|322|65771|4294967545|84.87|10.76|false|sarah allen|2013-03-01 09:11:58.703271|0.79|joggying +79|308|65563|4294967347|4.06|44.84|false|nick underhill|2013-03-01 09:11:58.703097|76.53|industrial engineering +4|382|65719|4294967329|7.26|39.92|true|fred polk|2013-03-01 09:11:58.703073|73.64|mathematics +10|448|65675|4294967392|26.20|16.30|true|rachel laertes|2013-03-01 09:11:58.703200|18.01|xylophone band +45|281|65685|4294967513|81.33|32.22|true|oscar allen|2013-03-01 09:11:58.703285|71.38|religion +57|288|65599|4294967422|90.33|44.25|false|bob young|2013-03-01 09:11:58.703185|11.16|biology +77|452|65706|4294967512|22.90|5.35|true|bob van buren|2013-03-01 09:11:58.703290|14.58|debate +103|492|65773|4294967404|58.29|48.28|false|yuri thompson|2013-03-01 09:11:58.703249|84.38|undecided +84|411|65737|4294967486|63.13|1.10|true|katie ichabod|2013-03-01 09:11:58.703086|29.57|american history +28|378|65589|4294967511|26.41|39.79|true|yuri polk|2013-03-01 09:11:58.703267|28.62|values clariffication +88|478|65752|4294967364|80.59|45.13|true|victor garcia|2013-03-01 09:11:58.703081|34.90|chemistry +37|388|65608|4294967350|32.94|39.06|false|mike polk|2013-03-01 09:11:58.703273|42.48|quiet hour +25|264|65648|4294967402|90.83|30.96|false|tom ichabod|2013-03-01 09:11:58.703268|65.58|history +17|455|65738|4294967508|15.73|27.01|false|david young|2013-03-01 09:11:58.703254|26.24|american history +62|438|65655|4294967511|91.77|1.90|false|sarah steinbeck|2013-03-01 09:11:58.703150|16.41|chemistry +65|298|65669|4294967328|68.89|2.75|true|david miller|2013-03-01 09:11:58.703077|51.86|values clariffication +25|491|65641|4294967387|94.82|10.04|false|ulysses thompson|2013-03-01 09:11:58.703124|63.75|linguistics +25|497|65708|4294967497|2.45|49.99|false|ethan laertes|2013-03-01 09:11:58.703320|49.72|yard duty +117|288|65591|4294967530|75.18|2.71|false|fred quirinius|2013-03-01 09:11:58.703221|99.58|geology +62|404|65706|4294967549|86.06|40.01|true|irene zipper|2013-03-01 09:11:58.703139|13.38|kindergarten +99|362|65709|4294967399|50.48|26.34|false|jessica white|2013-03-01 09:11:58.703294|83.53|kindergarten +62|395|65685|4294967446|56.73|14.87|false|victor johnson|2013-03-01 09:11:58.703194|31.42|history +62|386|65615|4294967359|44.03|43.78|true|luke underhill|2013-03-01 09:11:58.703099|86.73|nap time +15|302|65698|4294967526|91.38|3.59|true|wendy carson|2013-03-01 09:11:58.703111|9.46|religion +92|507|65699|4294967512|8.44|34.72|false|calvin xylophone|2013-03-01 09:11:58.703198|66.89|study skills +3|279|65756|4294967439|87.65|24.72|false|david white|2013-03-01 09:11:58.703233|47.19|study skills +114|330|65754|4294967500|76.20|39.35|true|rachel quirinius|2013-03-01 09:11:58.703145|76.16|undecided +24|500|65717|4294967535|60.96|21.51|false|victor falkner|2013-03-01 09:11:58.703318|82.83|nap time +-2|331|65707|4294967335|67.12|13.51|false|bob ovid|2013-03-01 09:11:58.703285|62.32|joggying +101|463|65740|4294967425|52.27|11.58|true|priscilla robinson|2013-03-01 09:11:58.703078|13.09|yard duty +106|269|65577|4294967524|17.11|38.45|true|rachel falkner|2013-03-01 09:11:58.703197|79.89|xylophone band +121|500|65690|4294967517|49.31|9.85|false|luke robinson|2013-03-01 09:11:58.703074|37.91|topology +37|351|65587|4294967410|99.66|20.51|false|quinn falkner|2013-03-01 09:11:58.703221|80.69|history +6|340|65612|4294967345|54.08|3.53|true|oscar white|2013-03-01 09:11:58.703279|68.67|debate +115|366|65785|4294967330|90.00|25.79|true|jessica carson|2013-03-01 09:11:58.703143|2.72|xylophone band +124|307|65649|4294967368|81.66|19.35|true|wendy ichabod|2013-03-01 09:11:58.703254|73.76|opthamology +11|286|65752|4294967355|72.33|20.94|false|xavier carson|2013-03-01 09:11:58.703109|23.28|history +15|320|65716|4294967505|49.25|27.53|false|fred carson|2013-03-01 09:11:58.703263|18.08|industrial engineering +76|316|65706|4294967460|12.99|35.53|true|rachel davidson|2013-03-01 09:11:58.703300|85.43|quiet hour +-2|485|65788|4294967510|9.99|22.75|false|luke carson|2013-03-01 09:11:58.703217|82.56|mathematics +87|482|65612|4294967327|16.51|22.21|true|katie nixon|2013-03-01 09:11:58.703083|47.09|xylophone band +21|400|65777|4294967354|4.05|11.10|false|david quirinius|2013-03-01 09:11:58.703205|25.69|geology +97|343|65764|4294967427|47.79|18.94|true|ethan miller|2013-03-01 09:11:58.703308|39.81|topology +2|292|65783|4294967420|38.86|12.14|true|wendy robinson|2013-03-01 09:11:58.703239|72.70|wind surfing +48|440|65570|4294967438|41.44|13.11|true|bob thompson|2013-03-01 09:11:58.703122|57.67|american history +87|333|65592|4294967296|71.77|8.28|false|yuri nixon|2013-03-01 09:11:58.703302|87.58|quiet hour +-1|344|65616|4294967444|29.44|19.94|false|oscar falkner|2013-03-01 09:11:58.703203|28.22|geology +1|425|65625|4294967531|51.83|38.18|false|holly xylophone|2013-03-01 09:11:58.703198|0.31|geology +108|363|65715|4294967467|99.69|17.10|true|yuri xylophone|2013-03-01 09:11:58.703177|44.91|geology +93|500|65778|4294967442|82.52|38.24|true|xavier falkner|2013-03-01 09:11:58.703277|25.41|history +112|260|65612|4294967500|51.90|24.53|false|rachel falkner|2013-03-01 09:11:58.703211|65.45|american history +89|294|65754|4294967450|94.21|35.55|true|gabriella falkner|2013-03-01 09:11:58.703156|18.36|topology +32|389|65700|4294967525|42.65|32.59|true|yuri king|2013-03-01 09:11:58.703253|1.70|undecided +13|395|65715|4294967317|64.24|36.77|false|fred ovid|2013-03-01 09:11:58.703168|74.25|yard duty +5|262|65726|4294967543|8.85|12.89|true|rachel garcia|2013-03-01 09:11:58.703222|45.65|yard duty +65|324|65569|4294967315|93.15|41.46|false|alice brown|2013-03-01 09:11:58.703110|77.23|topology +73|477|65764|4294967542|27.96|44.68|false|bob steinbeck|2013-03-01 09:11:58.703173|90.95|undecided +6|337|65616|4294967456|38.34|34.04|true|rachel hernandez|2013-03-01 09:11:58.703223|60.63|debate +51|384|65649|4294967423|14.62|5.33|true|oscar king|2013-03-01 09:11:58.703232|21.96|history +87|369|65626|4294967403|20.94|26.46|true|ulysses hernandez|2013-03-01 09:11:58.703076|35.79|values clariffication +48|365|65558|4294967361|66.17|6.28|true|alice xylophone|2013-03-01 09:11:58.703081|51.13|study skills +12|388|65642|4294967298|58.26|34.09|false|jessica brown|2013-03-01 09:11:58.703081|92.61|linguistics +12|353|65703|4294967414|54.55|5.92|true|jessica johnson|2013-03-01 09:11:58.703289|91.71|chemistry +117|499|65566|4294967328|32.18|19.59|true|priscilla king|2013-03-01 09:11:58.703214|66.88|philosophy +116|363|65719|4294967513|18.59|48.19|false|priscilla johnson|2013-03-01 09:11:58.703237|55.47|history +21|433|65551|4294967366|84.35|34.09|false|oscar thompson|2013-03-01 09:11:58.703291|7.99|values clariffication +-2|409|65717|4294967343|39.62|9.79|true|irene ichabod|2013-03-01 09:11:58.703315|64.80|joggying +23|495|65785|4294967473|30.91|21.95|true|fred robinson|2013-03-01 09:11:58.703240|66.34|nap time +30|507|65673|4294967453|83.51|40.92|true|oscar thompson|2013-03-01 09:11:58.703281|65.25|values clariffication +13|365|65594|4294967446|13.41|34.03|true|irene white|2013-03-01 09:11:58.703084|52.53|topology +92|419|65771|4294967310|64.82|3.01|false|yuri brown|2013-03-01 09:11:58.703271|18.05|undecided +81|351|65781|4294967473|48.46|15.80|false|bob nixon|2013-03-01 09:11:58.703254|99.35|debate +105|490|65543|4294967334|32.91|42.91|false|yuri steinbeck|2013-03-01 09:11:58.703233|42.19|xylophone band +25|402|65619|4294967340|6.28|49.92|true|victor xylophone|2013-03-01 09:11:58.703210|84.32|philosophy +88|485|65557|4294967391|95.95|46.22|true|irene xylophone|2013-03-01 09:11:58.703141|63.31|mathematics +81|285|65758|4294967338|37.83|38.23|true|irene ichabod|2013-03-01 09:11:58.703322|43.31|quiet hour +96|316|65764|4294967442|86.76|32.89|false|wendy miller|2013-03-01 09:11:58.703190|10.35|geology +43|321|65538|4294967422|81.78|6.07|false|zach van buren|2013-03-01 09:11:58.703273|26.02|topology +60|496|65614|4294967376|34.40|45.59|true|jessica steinbeck|2013-03-01 09:11:58.703076|81.95|xylophone band +44|395|65611|4294967443|15.58|1.53|false|gabriella thompson|2013-03-01 09:11:58.703295|11.00|values clariffication +73|409|65767|4294967371|36.93|36.16|true|quinn ellison|2013-03-01 09:11:58.703105|82.70|religion +121|330|65772|4294967508|70.46|44.50|true|quinn zipper|2013-03-01 09:11:58.703272|11.31|philosophy +61|421|65541|4294967410|34.59|27.52|false|calvin johnson|2013-03-01 09:11:58.703299|3.52|history +65|370|65674|4294967474|6.94|4.38|false|tom falkner|2013-03-01 09:11:58.703142|63.24|wind surfing +41|462|65699|4294967391|58.03|17.26|false|calvin xylophone|2013-03-01 09:11:58.703322|92.60|study skills +97|460|65591|4294967515|46.39|2.16|false|mike carson|2013-03-01 09:11:58.703265|97.16|values clariffication +-1|435|65624|4294967377|73.60|45.63|true|irene hernandez|2013-03-01 09:11:58.703208|31.35|study skills +22|282|65782|4294967318|75.19|40.78|false|quinn ichabod|2013-03-01 09:11:58.703122|44.85|topology +46|487|65748|4294967318|67.01|24.13|false|victor zipper|2013-03-01 09:11:58.703273|95.40|linguistics +18|275|65757|4294967307|80.45|18.92|false|bob hernandez|2013-03-01 09:11:58.703307|38.25|education +103|264|65587|4294967306|97.65|11.36|false|david ovid|2013-03-01 09:11:58.703265|42.76|wind surfing +86|466|65642|4294967333|40.96|26.06|true|david young|2013-03-01 09:11:58.703155|2.99|kindergarten +119|437|65637|4294967494|18.93|31.04|true|calvin brown|2013-03-01 09:11:58.703241|30.45|debate +62|285|65593|4294967518|83.43|2.05|false|rachel xylophone|2013-03-01 09:11:58.703084|45.21|quiet hour +1|283|65752|4294967528|95.01|1.76|false|ethan ichabod|2013-03-01 09:11:58.703072|16.68|history +8|333|65732|4294967503|22.43|21.80|false|mike polk|2013-03-01 09:11:58.703160|71.80|industrial engineering +90|425|65648|4294967323|50.68|40.41|false|victor allen|2013-03-01 09:11:58.703146|58.75|kindergarten +110|319|65620|4294967332|32.36|35.17|true|ethan davidson|2013-03-01 09:11:58.703269|73.03|history +111|313|65711|4294967418|70.04|10.88|true|priscilla nixon|2013-03-01 09:11:58.703206|66.32|mathematics +96|399|65719|4294967401|52.35|4.01|true|rachel hernandez|2013-03-01 09:11:58.703076|32.45|values clariffication +83|353|65714|4294967384|10.12|15.81|false|rachel miller|2013-03-01 09:11:58.703110|16.39|philosophy +11|475|65747|4294967303|98.29|32.30|false|yuri king|2013-03-01 09:11:58.703285|11.06|forestry +84|295|65682|4294967463|17.75|23.28|true|alice zipper|2013-03-01 09:11:58.703306|79.77|industrial engineering +8|348|65626|4294967373|52.54|31.29|false|bob underhill|2013-03-01 09:11:58.703189|82.40|undecided +0|339|65603|4294967356|32.42|31.31|false|katie young|2013-03-01 09:11:58.703238|49.14|forestry +82|280|65688|4294967427|19.11|0.10|false|holly young|2013-03-01 09:11:58.703256|71.39|chemistry +119|465|65781|4294967467|23.83|0.95|false|yuri zipper|2013-03-01 09:11:58.703094|96.06|history +10|356|65586|4294967339|71.96|32.54|true|oscar zipper|2013-03-01 09:11:58.703091|73.01|quiet hour +25|364|65682|4294967449|50.96|34.46|true|sarah steinbeck|2013-03-01 09:11:58.703139|18.28|philosophy +47|270|65652|4294967393|85.46|33.87|true|luke zipper|2013-03-01 09:11:58.703173|96.68|philosophy +89|470|65676|4294967314|39.34|37.35|false|ulysses miller|2013-03-01 09:11:58.703303|69.67|values clariffication +105|393|65703|4294967359|19.00|45.80|false|oscar johnson|2013-03-01 09:11:58.703086|99.42|linguistics +120|415|65785|4294967498|54.68|32.92|true|calvin hernandez|2013-03-01 09:11:58.703086|93.09|linguistics +94|486|65649|4294967549|33.47|35.42|false|jessica carson|2013-03-01 09:11:58.703089|34.30|mathematics +38|288|65634|4294967304|5.10|44.83|false|ethan white|2013-03-01 09:11:58.703083|0.94|xylophone band +91|268|65578|4294967501|43.98|2.77|false|jessica white|2013-03-01 09:11:58.703195|51.68|joggying +123|409|65629|4294967431|29.23|27.30|false|ulysses garcia|2013-03-01 09:11:58.703141|70.01|philosophy +7|454|65697|4294967394|62.25|3.38|false|tom underhill|2013-03-01 09:11:58.703121|47.97|values clariffication +13|488|65662|4294967457|25.08|4.01|false|quinn van buren|2013-03-01 09:11:58.703272|35.40|history +118|388|65642|4294967438|52.78|15.67|true|rachel falkner|2013-03-01 09:11:58.703158|61.13|opthamology +1|315|65713|4294967509|43.80|24.95|false|nick brown|2013-03-01 09:11:58.703287|83.95|mathematics +11|416|65658|4294967433|19.94|8.97|false|jessica nixon|2013-03-01 09:11:58.703117|63.58|joggying +42|457|65669|4294967534|13.45|16.47|true|calvin polk|2013-03-01 09:11:58.703257|59.51|yard duty +119|467|65639|4294967304|57.17|35.89|false|nick nixon|2013-03-01 09:11:58.703088|0.98|history +5|383|65629|4294967302|70.92|32.41|false|rachel young|2013-03-01 09:11:58.703314|1.72|opthamology +108|304|65557|4294967498|26.30|33.01|true|tom nixon|2013-03-01 09:11:58.703189|70.64|opthamology +60|447|65778|4294967546|65.11|14.36|true|yuri robinson|2013-03-01 09:11:58.703284|45.69|joggying +65|406|65613|4294967522|93.10|16.27|false|xavier laertes|2013-03-01 09:11:58.703178|25.19|philosophy +113|482|65739|4294967311|51.17|36.29|true|priscilla steinbeck|2013-03-01 09:11:58.703084|13.07|kindergarten +58|453|65780|4294967484|25.45|1.99|false|alice ichabod|2013-03-01 09:11:58.703307|25.71|nap time +24|320|65759|4294967315|23.99|43.22|false|irene robinson|2013-03-01 09:11:58.703095|24.36|chemistry +112|438|65622|4294967483|62.47|21.21|false|tom laertes|2013-03-01 09:11:58.703257|54.45|nap time +89|382|65708|4294967459|40.10|45.17|false|luke ovid|2013-03-01 09:11:58.703325|59.38|yard duty +63|410|65561|4294967330|86.99|24.01|false|fred underhill|2013-03-01 09:11:58.703288|29.48|religion +103|462|65658|4294967533|48.98|46.63|true|wendy laertes|2013-03-01 09:11:58.703272|85.64|philosophy +97|279|65563|4294967322|79.42|41.65|false|yuri thompson|2013-03-01 09:11:58.703308|43.37|mathematics +122|375|65717|4294967513|99.32|27.37|true|rachel falkner|2013-03-01 09:11:58.703095|65.37|philosophy +25|481|65672|4294967454|98.90|37.58|false|oscar ovid|2013-03-01 09:11:58.703293|73.85|biology +71|409|65667|4294967420|1.98|44.05|true|alice brown|2013-03-01 09:11:58.703117|38.55|religion +86|399|65568|4294967404|26.97|34.10|true|priscilla ichabod|2013-03-01 09:11:58.703283|87.92|yard duty +114|348|65752|4294967368|18.90|42.15|false|irene zipper|2013-03-01 09:11:58.703154|63.92|debate +31|464|65683|4294967364|20.61|48.84|false|irene garcia|2013-03-01 09:11:58.703219|80.62|american history +30|302|65688|4294967477|7.75|5.34|false|quinn polk|2013-03-01 09:11:58.703085|80.36|geology +72|423|65665|4294967353|54.78|15.57|false|fred quirinius|2013-03-01 09:11:58.703219|56.86|philosophy +78|408|65609|4294967534|83.25|24.25|false|quinn falkner|2013-03-01 09:11:58.703074|29.42|quiet hour +35|308|65659|4294967371|89.52|45.35|true|luke carson|2013-03-01 09:11:58.703276|78.07|wind surfing +13|310|65558|4294967399|60.05|38.39|false|priscilla polk|2013-03-01 09:11:58.703194|53.92|mathematics +80|450|65537|4294967548|74.10|8.87|true|ulysses falkner|2013-03-01 09:11:58.703139|56.48|nap time +30|295|65743|4294967359|17.51|44.20|true|bob hernandez|2013-03-01 09:11:58.703242|59.71|quiet hour +25|372|65606|4294967412|99.40|36.98|false|yuri quirinius|2013-03-01 09:11:58.703242|87.18|zync studies +-3|454|65733|4294967544|73.83|18.42|false|bob ichabod|2013-03-01 09:11:58.703240|95.56|debate +9|440|65773|4294967362|30.46|44.91|true|xavier falkner|2013-03-01 09:11:58.703098|62.35|religion +105|289|65576|4294967342|76.65|29.47|false|ulysses garcia|2013-03-01 09:11:58.703282|71.95|chemistry +116|263|65757|4294967525|94.04|37.06|false|priscilla hernandez|2013-03-01 09:11:58.703072|13.75|linguistics +124|458|65726|4294967483|7.96|0.29|false|zach laertes|2013-03-01 09:11:58.703281|1.46|study skills +-3|507|65671|4294967305|60.28|41.50|false|quinn polk|2013-03-01 09:11:58.703244|77.17|industrial engineering +-3|458|65679|4294967331|64.29|43.80|true|irene young|2013-03-01 09:11:58.703084|2.61|american history +17|435|65739|4294967438|44.39|9.29|false|alice thompson|2013-03-01 09:11:58.703241|68.01|undecided +33|390|65564|4294967305|8.20|17.36|false|calvin laertes|2013-03-01 09:11:58.703176|65.07|zync studies +73|474|65789|4294967421|62.00|40.44|true|alice quirinius|2013-03-01 09:11:58.703101|98.80|geology +46|313|65692|4294967310|93.40|34.70|true|fred hernandez|2013-03-01 09:11:58.703196|26.80|geology +50|302|65581|4294967387|2.73|18.54|false|jessica carson|2013-03-01 09:11:58.703282|58.24|study skills +115|311|65651|4294967423|44.94|33.29|true|ethan laertes|2013-03-01 09:11:58.703116|63.49|biology +88|368|65556|4294967428|37.79|47.21|true|tom laertes|2013-03-01 09:11:58.703149|7.26|topology +59|476|65560|4294967341|26.00|21.70|true|irene ovid|2013-03-01 09:11:58.703224|37.32|wind surfing +33|489|65723|4294967491|52.08|36.13|false|quinn robinson|2013-03-01 09:11:58.703174|29.70|chemistry +69|329|65580|4294967527|45.37|25.36|true|irene ichabod|2013-03-01 09:11:58.703267|95.34|joggying +8|342|65542|4294967486|86.51|30.05|true|ulysses johnson|2013-03-01 09:11:58.703164|4.89|kindergarten +47|327|65660|4294967329|53.96|10.07|false|fred white|2013-03-01 09:11:58.703313|48.34|zync studies +77|296|65771|4294967420|94.25|12.67|true|ulysses underhill|2013-03-01 09:11:58.703080|45.67|biology +63|451|65581|4294967493|44.66|40.63|true|alice miller|2013-03-01 09:11:58.703071|97.98|geology +103|303|65605|4294967540|54.00|47.97|true|fred davidson|2013-03-01 09:11:58.703087|68.42|zync studies +68|300|65577|4294967395|8.00|27.76|false|quinn quirinius|2013-03-01 09:11:58.703124|14.35|values clariffication +41|424|65684|4294967396|44.97|44.01|false|calvin polk|2013-03-01 09:11:58.703161|31.72|linguistics +84|448|65649|4294967425|5.81|28.49|true|ulysses ichabod|2013-03-01 09:11:58.703317|96.87|history +30|398|65577|4294967306|71.32|39.24|false|katie zipper|2013-03-01 09:11:58.703310|97.22|wind surfing +70|361|65695|4294967371|6.97|45.29|false|oscar falkner|2013-03-01 09:11:58.703268|79.32|opthamology +92|371|65702|4294967518|29.30|18.48|false|david ellison|2013-03-01 09:11:58.703192|30.01|topology +10|298|65666|4294967460|82.71|16.06|true|irene white|2013-03-01 09:11:58.703198|64.62|quiet hour +109|496|65699|4294967536|36.99|14.91|true|holly hernandez|2013-03-01 09:11:58.703123|66.43|geology +68|383|65597|4294967334|84.64|1.14|true|holly falkner|2013-03-01 09:11:58.703210|96.35|kindergarten +95|433|65738|4294967363|95.88|45.88|false|rachel steinbeck|2013-03-01 09:11:58.703308|34.85|history +37|262|65773|4294967482|26.04|4.86|true|oscar hernandez|2013-03-01 09:11:58.703285|92.63|linguistics +24|421|65676|4294967355|23.99|14.11|true|ulysses ovid|2013-03-01 09:11:58.703281|19.16|forestry +91|485|65607|4294967315|55.90|17.62|false|zach nixon|2013-03-01 09:11:58.703305|83.23|joggying +67|387|65790|4294967318|93.14|31.43|false|irene king|2013-03-01 09:11:58.703188|6.25|industrial engineering +82|262|65571|4294967465|56.70|30.18|true|irene van buren|2013-03-01 09:11:58.703167|3.00|study skills +98|505|65582|4294967365|17.40|40.51|false|sarah polk|2013-03-01 09:11:58.703121|56.65|history +22|268|65612|4294967462|9.69|4.64|false|xavier ichabod|2013-03-01 09:11:58.703304|3.86|linguistics +10|332|65685|4294967332|76.12|20.13|true|priscilla laertes|2013-03-01 09:11:58.703170|82.71|opthamology +36|317|65641|4294967471|56.22|36.78|true|tom johnson|2013-03-01 09:11:58.703296|53.38|biology +60|501|65555|4294967313|13.57|11.68|true|yuri davidson|2013-03-01 09:11:58.703183|10.42|religion +123|267|65560|4294967438|40.69|11.41|true|ethan allen|2013-03-01 09:11:58.703086|91.03|undecided +-2|482|65558|4294967487|36.92|49.78|true|nick johnson|2013-03-01 09:11:58.703204|39.91|industrial engineering +59|270|65726|4294967372|48.94|37.15|false|oscar polk|2013-03-01 09:11:58.703221|12.67|quiet hour +119|385|65595|4294967373|36.66|15.82|true|jessica nixon|2013-03-01 09:11:58.703127|5.26|zync studies +122|306|65751|4294967471|56.79|48.37|true|bob hernandez|2013-03-01 09:11:58.703186|50.61|kindergarten +64|402|65777|4294967481|77.49|13.11|false|nick carson|2013-03-01 09:11:58.703264|66.64|study skills +48|465|65758|4294967485|75.39|30.96|false|ethan allen|2013-03-01 09:11:58.703076|10.00|joggying +117|458|65603|4294967342|53.32|32.59|true|ethan garcia|2013-03-01 09:11:58.703204|47.35|yard duty +23|283|65557|4294967415|24.61|14.57|false|fred white|2013-03-01 09:11:58.703082|12.44|chemistry +56|507|65538|4294967507|67.82|42.13|false|alice king|2013-03-01 09:11:58.703297|54.64|american history +96|436|65737|4294967528|81.66|27.09|false|tom zipper|2013-03-01 09:11:58.703199|85.16|debate +88|292|65578|4294967546|91.57|37.42|false|nick zipper|2013-03-01 09:11:58.703294|96.08|religion +73|481|65717|4294967391|40.07|27.66|true|yuri xylophone|2013-03-01 09:11:58.703120|18.21|history +80|280|65620|4294967482|58.09|40.39|false|fred polk|2013-03-01 09:11:58.703136|23.61|xylophone band +96|464|65659|4294967493|74.22|21.71|true|jessica ichabod|2013-03-01 09:11:58.703226|92.72|undecided +103|485|65707|4294967436|94.57|21.16|true|zach van buren|2013-03-01 09:11:58.703313|3.93|study skills +31|410|65566|4294967518|36.11|16.72|true|nick ellison|2013-03-01 09:11:58.703305|61.53|biology +-3|270|65702|4294967512|38.05|1.07|true|david carson|2013-03-01 09:11:58.703136|28.07|philosophy +3|404|65709|4294967473|14.86|48.87|true|mike quirinius|2013-03-01 09:11:58.703099|37.99|xylophone band +124|473|65644|4294967314|65.16|19.33|false|oscar white|2013-03-01 09:11:58.703194|33.17|debate +103|321|65572|4294967353|64.79|0.22|false|david robinson|2013-03-01 09:11:58.703187|20.31|linguistics +41|395|65686|4294967428|61.99|11.61|false|sarah steinbeck|2013-03-01 09:11:58.703278|17.45|biology +-3|469|65752|4294967350|55.41|32.11|true|oscar johnson|2013-03-01 09:11:58.703110|47.32|philosophy +98|336|65641|4294967519|82.11|7.91|true|tom davidson|2013-03-01 09:11:58.703320|83.43|debate +54|422|65655|4294967551|15.74|34.11|true|bob garcia|2013-03-01 09:11:58.703086|46.93|yard duty +70|462|65671|4294967385|82.68|7.94|false|fred white|2013-03-01 09:11:58.703167|45.89|joggying +62|325|65751|4294967342|36.71|28.42|true|priscilla garcia|2013-03-01 09:11:58.703239|0.56|mathematics +56|504|65635|4294967318|93.88|34.87|true|holly polk|2013-03-01 09:11:58.703227|89.14|american history +50|275|65697|4294967322|58.10|27.56|false|priscilla johnson|2013-03-01 09:11:58.703096|6.19|biology +114|428|65680|4294967498|62.68|3.90|true|yuri nixon|2013-03-01 09:11:58.703086|53.28|xylophone band +100|277|65739|4294967382|1.61|18.22|true|wendy garcia|2013-03-01 09:11:58.703137|78.35|industrial engineering +7|494|65601|4294967403|20.76|19.41|false|david underhill|2013-03-01 09:11:58.703164|70.81|topology +79|448|65744|4294967479|18.18|36.26|true|david xylophone|2013-03-01 09:11:58.703310|76.40|joggying +19|289|65562|4294967344|56.25|33.81|true|sarah van buren|2013-03-01 09:11:58.703301|64.05|forestry +10|508|65589|4294967473|96.49|7.56|false|priscilla brown|2013-03-01 09:11:58.703134|2.08|education +89|451|65686|4294967396|21.20|13.22|true|oscar king|2013-03-01 09:11:58.703127|49.12|undecided +45|323|65540|4294967436|29.79|5.69|false|tom falkner|2013-03-01 09:11:58.703102|53.85|nap time +34|319|65780|4294967523|80.40|9.05|true|sarah falkner|2013-03-01 09:11:58.703179|75.06|yard duty +30|510|65632|4294967373|60.94|21.31|true|gabriella steinbeck|2013-03-01 09:11:58.703146|69.16|undecided +72|350|65742|4294967491|3.33|30.48|false|katie johnson|2013-03-01 09:11:58.703315|55.83|topology +96|402|65620|4294967320|19.38|49.45|false|oscar steinbeck|2013-03-01 09:11:58.703303|25.84|yard duty +95|405|65536|4294967338|18.26|1.46|false|sarah thompson|2013-03-01 09:11:58.703073|29.27|education +80|396|65675|4294967379|30.21|28.41|false|rachel white|2013-03-01 09:11:58.703316|11.37|topology +5|507|65715|4294967297|87.39|16.09|true|sarah xylophone|2013-03-01 09:11:58.703321|0.46|nap time +52|322|65635|4294967296|13.25|10.02|false|wendy falkner|2013-03-01 09:11:58.703094|2.51|industrial engineering +64|345|65744|4294967316|23.26|29.25|true|sarah brown|2013-03-01 09:11:58.703245|96.45|kindergarten +97|502|65654|4294967405|0.09|3.10|false|victor robinson|2013-03-01 09:11:58.703141|29.03|religion +25|424|65599|4294967303|49.92|33.86|true|calvin miller|2013-03-01 09:11:58.703095|76.80|study skills +115|298|65599|4294967457|78.69|11.89|false|luke steinbeck|2013-03-01 09:11:58.703245|22.81|geology +49|496|65722|4294967407|17.46|33.62|false|ethan underhill|2013-03-01 09:11:58.703158|7.67|forestry +77|315|65592|4294967532|28.72|38.15|false|nick robinson|2013-03-01 09:11:58.703296|78.69|debate +33|258|65780|4294967448|5.78|19.07|true|calvin davidson|2013-03-01 09:11:58.703133|18.12|study skills +98|390|65592|4294967397|36.40|29.61|false|sarah young|2013-03-01 09:11:58.703314|74.60|wind surfing +41|415|65618|4294967426|2.23|46.43|true|nick van buren|2013-03-01 09:11:58.703225|14.78|yard duty +62|427|65671|4294967359|75.01|38.93|false|bob ovid|2013-03-01 09:11:58.703195|17.17|values clariffication +-2|294|65588|4294967301|8.51|2.16|false|zach zipper|2013-03-01 09:11:58.703208|35.15|debate +94|309|65653|4294967447|6.14|5.65|false|yuri van buren|2013-03-01 09:11:58.703279|94.47|study skills +120|377|65615|4294967364|24.99|12.26|true|oscar nixon|2013-03-01 09:11:58.703250|71.62|industrial engineering +3|500|65756|4294967445|98.38|39.43|true|luke nixon|2013-03-01 09:11:58.703243|29.49|yard duty +-1|505|65611|4294967338|75.26|22.98|false|mike allen|2013-03-01 09:11:58.703123|95.80|linguistics +124|466|65612|4294967456|72.76|15.57|false|calvin polk|2013-03-01 09:11:58.703235|37.15|biology +1|490|65591|4294967329|69.89|40.29|false|luke laertes|2013-03-01 09:11:58.703104|58.27|quiet hour +70|385|65553|4294967506|69.14|44.05|false|ethan xylophone|2013-03-01 09:11:58.703150|93.69|chemistry +68|330|65573|4294967506|66.87|17.31|true|jessica hernandez|2013-03-01 09:11:58.703124|30.57|zync studies +82|421|65699|4294967550|84.77|40.40|false|gabriella white|2013-03-01 09:11:58.703292|29.99|history +9|346|65646|4294967449|66.32|24.07|false|jessica xylophone|2013-03-01 09:11:58.703084|94.86|undecided +116|336|65638|4294967327|64.45|11.24|true|jessica falkner|2013-03-01 09:11:58.703087|60.05|study skills +19|376|65770|4294967536|79.12|20.11|false|victor carson|2013-03-01 09:11:58.703243|72.69|industrial engineering +27|433|65767|4294967395|22.53|18.81|false|bob polk|2013-03-01 09:11:58.703097|52.68|linguistics +31|468|65654|4294967361|33.08|29.95|false|bob young|2013-03-01 09:11:58.703210|16.48|philosophy +84|411|65564|4294967493|49.25|7.84|true|oscar nixon|2013-03-01 09:11:58.703274|47.54|american history +37|409|65769|4294967384|25.89|42.27|false|katie underhill|2013-03-01 09:11:58.703172|66.93|zync studies +10|356|65628|4294967475|98.07|13.86|false|david carson|2013-03-01 09:11:58.703222|7.37|nap time +105|437|65664|4294967535|2.05|17.01|true|holly laertes|2013-03-01 09:11:58.703144|5.69|industrial engineering +117|508|65788|4294967319|66.86|25.25|false|ulysses davidson|2013-03-01 09:11:58.703283|85.22|industrial engineering +108|322|65697|4294967529|20.24|40.23|true|mike carson|2013-03-01 09:11:58.703083|6.04|philosophy +80|426|65735|4294967533|73.85|41.99|false|quinn hernandez|2013-03-01 09:11:58.703098|69.55|mathematics +49|434|65692|4294967336|89.33|14.24|true|yuri underhill|2013-03-01 09:11:58.703127|3.91|quiet hour +74|501|65657|4294967451|88.85|11.09|true|bob king|2013-03-01 09:11:58.703175|51.36|quiet hour +8|380|65734|4294967369|84.11|10.24|false|victor underhill|2013-03-01 09:11:58.703291|78.90|opthamology +89|364|65735|4294967334|12.41|24.02|false|nick nixon|2013-03-01 09:11:58.703272|34.80|debate +53|479|65579|4294967303|7.50|43.05|false|rachel ellison|2013-03-01 09:11:58.703148|48.50|yard duty +67|493|65626|4294967489|98.74|32.74|false|katie thompson|2013-03-01 09:11:58.703263|87.95|geology +56|390|65676|4294967456|42.59|1.64|true|wendy king|2013-03-01 09:11:58.703307|39.31|joggying +13|431|65624|4294967330|94.05|30.76|false|quinn ichabod|2013-03-01 09:11:58.703180|1.72|biology +85|366|65627|4294967356|37.14|35.57|true|alice king|2013-03-01 09:11:58.703170|6.78|yard duty +-2|286|65549|4294967493|9.20|1.23|true|ulysses king|2013-03-01 09:11:58.703218|93.35|study skills +51|344|65698|4294967309|83.66|6.12|false|zach ellison|2013-03-01 09:11:58.703158|29.28|yard duty +89|489|65610|4294967353|64.70|8.13|true|katie polk|2013-03-01 09:11:58.703120|56.34|education +95|327|65747|4294967522|1.16|12.00|true|bob van buren|2013-03-01 09:11:58.703284|3.45|opthamology +50|508|65541|4294967451|37.38|46.94|true|quinn steinbeck|2013-03-01 09:11:58.703081|20.90|forestry +6|301|65693|4294967454|89.07|41.96|true|alice ichabod|2013-03-01 09:11:58.703297|16.13|religion +7|322|65719|4294967434|1.02|29.24|false|quinn carson|2013-03-01 09:11:58.703293|47.99|forestry +99|469|65751|4294967356|10.10|42.47|false|wendy young|2013-03-01 09:11:58.703180|63.14|opthamology +18|269|65751|4294967544|87.84|0.60|true|mike steinbeck|2013-03-01 09:11:58.703167|36.04|religion +22|361|65729|4294967328|67.51|15.52|false|zach ovid|2013-03-01 09:11:58.703317|26.96|quiet hour +114|455|65723|4294967481|4.94|33.44|false|alice van buren|2013-03-01 09:11:58.703074|72.22|philosophy +-3|384|65676|4294967453|71.97|31.52|false|alice davidson|2013-03-01 09:11:58.703226|14.28|xylophone band +37|334|65775|4294967518|17.88|45.96|false|zach ellison|2013-03-01 09:11:58.703260|9.92|nap time +28|427|65648|4294967309|45.65|3.90|true|bob robinson|2013-03-01 09:11:58.703308|89.89|chemistry +86|469|65780|4294967466|64.61|24.76|true|david steinbeck|2013-03-01 09:11:58.703241|0.68|linguistics +61|455|65567|4294967315|84.80|25.83|false|alice robinson|2013-03-01 09:11:58.703127|26.03|zync studies +-3|387|65550|4294967355|84.75|22.75|true|holly thompson|2013-03-01 09:11:58.703073|52.01|biology +14|492|65690|4294967388|98.07|15.98|true|david miller|2013-03-01 09:11:58.703096|15.69|forestry +8|318|65687|4294967551|44.02|14.70|false|quinn thompson|2013-03-01 09:11:58.703205|23.43|joggying +117|502|65789|4294967441|55.39|8.22|false|tom allen|2013-03-01 09:11:58.703129|74.48|xylophone band +20|285|65783|4294967424|99.34|21.19|false|alice thompson|2013-03-01 09:11:58.703223|9.55|opthamology +4|478|65538|4294967312|21.90|0.85|false|sarah thompson|2013-03-01 09:11:58.703089|79.07|xylophone band diff --git a/src/test/hive/data/files/over1k b/src/test/hive/data/files/over1k new file mode 100644 index 0000000000000..1b3a052be97e8 --- /dev/null +++ b/src/test/hive/data/files/over1k @@ -0,0 +1,1049 @@ +124|336|65664|4294967435|74.72|42.47|true|bob davidson|2013-03-01 09:11:58.703302|45.40|yard duty +19|442|65553|4294967380|26.43|37.77|true|alice zipper|2013-03-01 09:11:58.703217|29.62|history +35|387|65619|4294967459|96.91|18.86|false|katie davidson|2013-03-01 09:11:58.703079|27.32|history +111|372|65656|4294967312|13.01|34.95|false|xavier quirinius|2013-03-01 09:11:58.703310|23.91|topology +54|317|65547|4294967409|60.71|2.09|false|nick robinson|2013-03-01 09:11:58.703103|90.21|geology +-3|467|65575|4294967437|81.64|23.53|true|tom hernandez|2013-03-01 09:11:58.703188|32.85|study skills +53|317|65702|4294967398|35.17|30.87|false|ulysses hernandez|2013-03-01 09:11:58.703164|79.50|industrial engineering +122|356|65759|4294967379|92.61|18.82|true|priscilla ichabod|2013-03-01 09:11:58.703175|92.81|nap time +18|407|65725|4294967362|82.52|5.30|true|quinn van buren|2013-03-01 09:11:58.703282|35.86|kindergarten +66|484|65685|4294967387|45.99|6.81|false|luke laertes|2013-03-01 09:11:58.703317|63.27|mathematics +122|444|65675|4294967500|72.62|34.15|false|xavier polk|2013-03-01 09:11:58.703222|99.79|american history +115|269|65627|4294967362|91.42|1.61|false|zach falkner|2013-03-01 09:11:58.703252|49.85|american history +109|438|65779|4294967371|67.18|40.76|false|quinn davidson|2013-03-01 09:11:58.703318|92.06|quiet hour +102|436|65749|4294967542|68.89|39.01|false|priscilla van buren|2013-03-01 09:11:58.703305|82.99|mathematics +92|490|65745|4294967420|10.22|49.45|true|fred king|2013-03-01 09:11:58.703277|80.12|education +34|439|65667|4294967480|26.49|18.47|true|calvin ellison|2013-03-01 09:11:58.703135|97.87|joggying +72|383|65638|4294967398|23.60|46.73|false|sarah garcia|2013-03-01 09:11:58.703192|11.20|topology +11|505|65576|4294967297|71.32|18.19|false|zach young|2013-03-01 09:11:58.703221|92.47|chemistry +69|376|65751|4294967384|88.77|27.02|false|david underhill|2013-03-01 09:11:58.703244|9.02|mathematics +36|309|65769|4294967317|91.16|38.05|false|yuri carson|2013-03-01 09:11:58.703178|68.33|debate +51|258|65781|4294967391|1.92|39.05|false|ulysses laertes|2013-03-01 09:11:58.703248|95.52|nap time +1|482|65675|4294967314|38.33|21.67|false|jessica thompson|2013-03-01 09:11:58.703110|83.09|forestry +51|295|65716|4294967433|16.24|48.96|false|sarah johnson|2013-03-01 09:11:58.703299|9.00|linguistics +68|439|65703|4294967469|92.55|23.30|true|zach ovid|2013-03-01 09:11:58.703316|65.91|religion +104|299|65791|4294967424|34.03|27.14|false|victor nixon|2013-03-01 09:11:58.703173|48.17|topology +14|337|65611|4294967542|16.99|16.36|true|sarah ellison|2013-03-01 09:11:58.703187|64.89|biology +21|305|65664|4294967361|82.41|49.69|true|xavier davidson|2013-03-01 09:11:58.703287|75.43|mathematics +79|419|65755|4294967329|91.05|39.04|false|fred steinbeck|2013-03-01 09:11:58.703216|11.23|values clariffication +109|427|65626|4294967308|98.72|31.23|true|tom robinson|2013-03-01 09:11:58.703273|64.61|kindergarten +2|379|65707|4294967450|44.27|37.93|false|victor allen|2013-03-01 09:11:58.703076|96.97|american history +7|508|65681|4294967345|66.61|23.73|false|luke allen|2013-03-01 09:11:58.703104|75.85|history +66|300|65553|4294967447|29.02|3.69|false|fred zipper|2013-03-01 09:11:58.703307|72.16|nap time +15|347|65604|4294967484|79.83|26.24|false|alice nixon|2013-03-01 09:11:58.703124|85.91|zync studies +104|454|65579|4294967467|54.44|42.57|true|irene van buren|2013-03-01 09:11:58.703225|73.48|values clariffication +6||65669|4294967519|69.53|2.80|true|alice laertes|2013-03-01 09:11:58.703250|93.79|industrial engineering +121|449|65776|4294967327|60.60|45.14|true|quinn davidson|2013-03-01 09:11:58.703175|98.87|industrial engineering +51|445|65761|4294967423|10.60|0.52|false|rachel ellison|2013-03-01 09:11:58.703196|60.30|education +63|268|65652|4294967374|40.78|47.60|false|tom van buren|2013-03-01 09:11:58.703279|42.78|education +10|439|65641|4294967353|24.83|46.33|true|luke zipper|2013-03-01 09:11:58.703257|26.02|topology +17|384|65700|4294967457|20.07|33.19|true|mike carson|2013-03-01 09:11:58.703140|2.83|religion +38|351|65567|4294967316|22.78|30.04|true|jessica ellison|2013-03-01 09:11:58.703113|84.56|forestry +61|419|65783|4294967427|70.93|39.46|false|bob brown|2013-03-01 09:11:58.703207|62.58|education +95|511|65636|4294967325|71.50|2.52|false|zach thompson|2013-03-01 09:11:58.703198|67.23|quiet hour +83|316|65749|4294967342|90.51|23.55|true|holly ichabod|2013-03-01 09:11:58.703091|45.00|nap time +38|318|65560|4294967354|43.73|30.98|false||2013-03-01 09:11:58.703318|21.81|mathematics +|473|65720|4294967324|80.74|40.60|false|holly falkner|2013-03-01 09:11:58.703111|18.80|mathematics +82|272|65699|4294967488|92.82|8.98|true|wendy van buren|2013-03-01 09:11:58.703314|41.13|mathematics +35|475|65574|4294967455|60.13|10.85|false|tom falkner|2013-03-01 09:11:58.703217|23.57|wind surfing +2|375|65608|4294967444|9.93|16.47|false|xavier thompson|2013-03-01 09:11:58.703154|16.93|topology +62|428|65758|4294967373|96.09|38.60|true|fred johnson|2013-03-01 09:11:58.703307|60.88|geology +100|391|65661|4294967317|52.72|15.01|true|victor steinbeck|2013-03-01 09:11:58.703232|61.96|debate +10|264|65688|4294967479|78.30|0.63|true|irene thompson|2013-03-01 09:11:58.703156|76.21|undecided +87|469|65580|4294967411|90.20|45.49|true|calvin brown|2013-03-01 09:11:58.703278|56.33|nap time +27|262|65787|4294967371|57.35|44.12|false|irene underhill|2013-03-01 09:11:58.703289|31.19|quiet hour +89|466|65541|4294967468|85.51|19.79|true|calvin davidson|2013-03-01 09:11:58.703113|35.38|philosophy +15|379|65597|4294967382|48.28|22.73|true|luke xylophone|2013-03-01 09:11:58.703309|37.41|topology +1|284|65572|4294967342|37.07|14.51|true|fred ichabod|2013-03-01 09:11:58.703254|43.49|quiet hour +89||65579|4294967419|46.02|6.06|false|victor davidson|2013-03-01 09:11:58.703077|91.42|xylophone band +-2|427|65666|4294967465|19.69|33.24|true|bob xylophone|2013-03-01 09:11:58.703219|32.73|joggying +58|360|65564|4294967402|19.00|21.16|false|oscar white|2013-03-01 09:11:58.703161|15.82|forestry +9|274|65710|4294967481|9.57|4.97|true|irene laertes|2013-03-01 09:11:58.703138|86.70|yard duty +107|271|65652|4294967329|38.30|25.36|false|tom polk|2013-03-01 09:11:58.703131|73.61|kindergarten +79|399|65664|4294967492|41.85|29.00|false|calvin garcia|2013-03-01 09:11:58.703074|97.64|religion +58|448|65591|4294967451|62.74|17.69|true|mike king|2013-03-01 09:11:58.703312|82.08|nap time +98|430|65616|4294967451|35.89|12.15|true|ulysses underhill|2013-03-01 09:11:58.703083|20.52|zync studies +83|382|65550|4294967451|35.00|21.02|true|katie hernandez|2013-03-01 09:11:58.703145|8.46|xylophone band +23|381|65661|4294967425|31.40|21.26|false|katie miller|2013-03-01 09:11:58.703251|68.98|topology +62|358|65552|4294967533|50.96|9.42|false|luke allen|2013-03-01 09:11:58.703117|82.21|kindergarten +7|487|65740|4294967416|9.22|43.64|false||2013-03-01 09:11:58.703272|7.49|american history +5|285|65654|4294967436|42.55|14.69|false|holly underhill|2013-03-01 09:11:58.703185|8.91|study skills +104|472|65620|4294967548|22.08|9.94|true|irene falkner|2013-03-01 09:11:58.703135|29.14|opthamology +|374|65560|4294967516|65.43|22.48|true|oscar quirinius|2013-03-01 09:11:58.703316|16.86|mathematics +8|273|65641|4294967507|35.08|37.80|false|nick allen|2013-03-01 09:11:58.703186|91.46|biology +20|449|65663|4294967439|4.72|12.34|true|xavier ichabod|2013-03-01 09:11:58.703259|11.37|geology +85|313|65780|4294967303|71.01|0.78|false|zach xylophone|2013-03-01 09:11:58.703203|11.63|biology +27|329|65778|4294967451|6.63|7.03|true|jessica zipper|2013-03-01 09:11:58.703130|51.64|philosophy +79|451|65662|4294967487|73.63|15.46|true|alice allen|2013-03-01 09:11:58.703161|99.18|wind surfing +69|440|65720|4294967373|9.56|24.42|true|jessica johnson|2013-03-01 09:11:58.703132|2.72|mathematics +119|390|65544|4294967411|41.31|20.02|false|fred steinbeck|2013-03-01 09:11:58.703267|26.94|nap time +85|468|65643|4294967430|86.63|21.00|true|wendy ovid|2013-03-01 09:11:58.703124|48.81|forestry +-2|429|65664|4294967441|45.19|40.43|false|wendy king|2013-03-01 09:11:58.703265|11.46|industrial engineering +100|481|65750|4294967308|21.19|28.14|false|holly hernandez|2013-03-01 09:11:58.703203|85.44|chemistry +-1|417|65685|4294967492|28.89|5.19|true|mike white|2013-03-01 09:11:58.703275|90.69|forestry +77||65681|4294967535|18.56|12.43|true|priscilla johnson|2013-03-01 09:11:58.703176|35.45|study skills +89|478|65583|4294967522|92.40|26.71|true|gabriella ovid|2013-03-01 09:11:58.703092|50.57|biology +43|333|65549|4294967331|11.22|38.00|true|victor xylophone|2013-03-01 09:11:58.703257|58.96|nap time +11|387|65718|4294967457|89.53|16.54|true||2013-03-01 09:11:58.703130|63.06|industrial engineering +105|438|65623|4294967501|39.69|22.95|false|ulysses xylophone|2013-03-01 09:11:58.703148|48.51|kindergarten +111|349|65740|4294967400|17.37|43.34|true|yuri nixon|2013-03-01 09:11:58.703280|55.16|quiet hour +85|461|65654|4294967507|89.10|13.44|true|xavier johnson|2013-03-01 09:11:58.703127|19.64|zync studies +|409|65536|4294967490|46.97|25.92|false|fred miller|2013-03-01 09:11:58.703116|33.45|history +51|398|65687|4294967304|30.37|47.31|false|luke garcia|2013-03-01 09:11:58.703156|21.79|industrial engineering +47|291|65615|4294967402|37.14|29.63|false|oscar ovid|2013-03-01 09:11:58.703231|39.40|geology +-1|268|65778|4294967418|56.33|44.73|true|calvin falkner|2013-03-01 09:11:58.703220|7.37|history +86|422|65582|4294967353|87.83|2.34|false|luke ellison|2013-03-01 09:11:58.703313|9.35|joggying +76|400|65661|4294967379|15.81|46.57|true|rachel allen|2013-03-01 09:11:58.703211|5.24|wind surfing +81|448|65550|4294967533|25.88|46.54|true|oscar king|2013-03-01 09:11:58.703161|57.63|american history +22|288|65578|4294967341|33.87|20.76|true|calvin hernandez|2013-03-01 09:11:58.703173|51.36|biology +59|410|65572|4294967370|74.42|26.22|false|nick ichabod|2013-03-01 09:11:58.703221|93.48|xylophone band +86|349|65686|4294967512|62.85|24.61|true|bob ovid|2013-03-01 09:11:58.703106|31.75|religion +27|335|65636|4294967505|37.14|2.29|false|alice quirinius|2013-03-01 09:11:58.703287|21.15|education +74|422|65682|4294967316|37.60|45.06|false|rachel carson|2013-03-01 09:11:58.703140|92.95|joggying +24|269|65644|4294967349|31.01|26.44|true|katie young|2013-03-01 09:11:58.703149|58.05|xylophone band +12|411|65560|4294967346|8.91|40.79|false|calvin thompson|2013-03-01 09:11:58.703227|71.59|religion +8|379|65574|4294967410|24.49|2.74|true|calvin young|2013-03-01 09:11:58.703283|38.54|study skills +123|477|65699|4294967340|1.21|28.71|true|quinn ovid|2013-03-01 09:11:58.703278|88.09|undecided +28|349|65711|4294967458|84.69|47.30|true|holly ichabod|2013-03-01 09:11:58.703109|50.26|zync studies +83|420|65563|4294967400|85.10|43.40|true|mike king|2013-03-01 09:11:58.703169|39.96|xylophone band +51|434|65549|4294967449|3.96|39.57|false|mike miller|2013-03-01 09:11:58.703141|76.06|kindergarten +91|346|65696|4294967355|71.31|1.29|true|alice brown|2013-03-01 09:11:58.703245|17.35|zync studies +82|393|65763|4294967452|66.51|14.44|false|quinn steinbeck|2013-03-01 09:11:58.703187|87.99|joggying +85|439|65759|4294967530|20.64|21.57|true|david quirinius|2013-03-01 09:11:58.703083|47.54|philosophy +66|475|65633|4294967498|26.64|42.03|false|gabriella ichabod|2013-03-01 09:11:58.703302|59.56|chemistry +117|311|65691|4294967335|24.28|43.67|false|quinn brown|2013-03-01 09:11:58.703295|56.02|geology +92|352|65559|4294967353|3.82|31.84|false|bob zipper|2013-03-01 09:11:58.703316|29.55|topology +31|432|65607|4294967497|40.80|35.49|false|jessica johnson|2013-03-01 09:11:58.703218|9.67|study skills +114|279|65657|4294967339|54.73|30.17|true|quinn allen|2013-03-01 09:11:58.703297|64.39|debate +28|485|65694|4294967534|79.48|8.84|false|quinn underhill|2013-03-01 09:11:58.703275|82.24|history +2|399|65571|4294967410|59.68|3.92|false|luke underhill|2013-03-01 09:11:58.703221|26.64|topology +17|395|65589|4294967464|95.33|47.46|false|wendy ovid|2013-03-01 09:11:58.703118|54.01|zync studies +71|363|65787|4294967465|59.50|20.17|true|oscar king|2013-03-01 09:11:58.703285|64.88|history +48|511|65561|4294967547|8.45|9.39|false|xavier zipper|2013-03-01 09:11:58.703297|25.67|chemistry +1|424|65712|4294967484|0.13|39.92|true|yuri johnson|2013-03-01 09:11:58.703144|94.43|zync studies +109|306|65643|4294967323|79.54|38.74|true|ulysses carson|2013-03-01 09:11:58.703263|92.24|undecided +49|349|65747|4294967306|84.40|44.93|true|katie garcia|2013-03-01 09:11:58.703226|39.77|chemistry +114|454|65542|4294967443|64.67|13.02|true|katie allen|2013-03-01 09:11:58.703275|84.63|philosophy +28|307|65606|4294967389|95.34|9.62|true|jessica davidson|2013-03-01 09:11:58.703181|57.69|american history +-1|348|65556|4294967413|35.17|9.51|false|bob young|2013-03-01 09:11:58.703280|45.81|quiet hour +1|485|65625|4294967309|41.81|15.46|false|david van buren|2013-03-01 09:11:58.703207|55.06|debate +38|291|65634|4294967438|83.57|49.63|false|david van buren|2013-03-01 09:11:58.703174|64.36|debate +50|338|65634|4294967463|85.23|32.33|false|david ellison|2013-03-01 09:11:58.703247|57.09|values clariffication +113|383|65578|4294967358|43.16|22.22|true|alice xylophone|2013-03-01 09:11:58.703250|41.10|philosophy +43|305|65572|4294967511|36.70|11.12|true|ethan van buren|2013-03-01 09:11:58.703110|70.91|study skills +110|344|65759|4294967464|27.31|46.31|true|mike davidson|2013-03-01 09:11:58.703106|14.10|philosophy +23|424|65731|4294967416|65.72|43.58|true|calvin polk|2013-03-01 09:11:58.703160|24.48|philosophy +16|281|65605|4294967493|45.34|10.36|true|bob white|2013-03-01 09:11:58.703140|14.12|american history +120|285|65649|4294967355|35.56|49.34|true|victor ovid|2013-03-01 09:11:58.703205|43.92|opthamology +98|452|65544|4294967309|73.93|18.86|true|jessica white|2013-03-01 09:11:58.703175|65.77|philosophy +41|360|65643|4294967488|76.28|46.18|false|quinn thompson|2013-03-01 09:11:58.703203|61.29|philosophy +66|462|65697|4294967452|57.29|38.07|true|ethan ovid|2013-03-01 09:11:58.703138|90.16|philosophy +99|373|65579|4294967465|24.02|47.95|true|irene polk|2013-03-01 09:11:58.703231|21.36|mathematics +15|334|65576|4294967542|75.66|11.12|true|jessica robinson|2013-03-01 09:11:58.703104|2.04|chemistry +53|466|65545|4294967514|94.66|15.12|true|wendy ellison|2013-03-01 09:11:58.703236|79.00|xylophone band +64|295|65622|4294967521|59.34|7.62|false|victor brown|2013-03-01 09:11:58.703157|28.37|undecided +8|323|65774|4294967503|21.49|5.55|false|priscilla xylophone|2013-03-01 09:11:58.703213|48.80|religion +42|368|65701|4294967410|35.68|35.84|true|gabriella polk|2013-03-01 09:11:58.703235|6.35|xylophone band +72|389|65724|4294967300|69.74|38.04|false|calvin nixon|2013-03-01 09:11:58.703161|85.52|education +23|493|65555|4294967455|77.89|25.11|true|victor brown|2013-03-01 09:11:58.703125|46.72|education +26|465|65700|4294967346|46.10|38.22|true|bob quirinius|2013-03-01 09:11:58.703177|37.61|kindergarten +77|507|65721|4294967488|78.31|33.11|true|fred davidson|2013-03-01 09:11:58.703187|72.82|biology +88|346|65654|4294967369|48.23|35.72|false|mike brown|2013-03-01 09:11:58.703197|26.73|chemistry +45|357|65755|4294967485|94.25|46.52|true|gabriella thompson|2013-03-01 09:11:58.703202|67.54|religion +11|286|65667|4294967382|48.45|22.94|true|rachel underhill|2013-03-01 09:11:58.703115|63.90|values clariffication +95|489|65711|4294967493|11.89|25.36|false|ulysses laertes|2013-03-01 09:11:58.703249|78.69|history +23|448|65637|4294967435|76.28|19.32|false|ulysses miller|2013-03-01 09:11:58.703290|16.89|mathematics +42|498|65687|4294967352|42.85|39.27|false|oscar robinson|2013-03-01 09:11:58.703139|48.73|undecided +27|490|65680|4294967347|57.46|11.02|true|wendy van buren|2013-03-01 09:11:58.703291|95.26|xylophone band +53|257|65790|4294967425|9.26|9.93|false|oscar laertes|2013-03-01 09:11:58.703113|46.91|forestry +27|457|65570|4294967464|81.58|3.78|false|ulysses underhill|2013-03-01 09:11:58.703088|17.09|quiet hour +46|480|65649|4294967458|86.23|6.76|false|quinn king|2013-03-01 09:11:58.703099|91.03|debate +57|419|65786|4294967300|6.57|36.93|false|nick allen|2013-03-01 09:11:58.703290|66.89|nap time +72|267|65537|4294967460|55.20|42.89|false|oscar carson|2013-03-01 09:11:58.703153|51.91|topology +110|346|65693|4294967505|67.12|9.14|true|luke white|2013-03-01 09:11:58.703199|45.69|quiet hour +117|346|65619|4294967321|78.73|35.68|false|holly brown|2013-03-01 09:11:58.703245|36.59|xylophone band +35|451|65763|4294967498|33.83|10.70|true|sarah robinson|2013-03-01 09:11:58.703126|53.52|nap time +87|466|65562|4294967446|35.62|30.58|true|jessica quirinius|2013-03-01 09:11:58.703246|45.94|values clariffication +116|307|65785|4294967434|72.18|44.24|false|rachel zipper|2013-03-01 09:11:58.703144|29.61|debate +36|460|65541|4294967365|55.99|38.73|false|fred hernandez|2013-03-01 09:11:58.703256|98.87|zync studies +2|383|65649|4294967537|74.19|44.57|true|ulysses king|2013-03-01 09:11:58.703118|8.69|quiet hour +11|331|65659|4294967432|68.01|7.24|true|quinn nixon|2013-03-01 09:11:58.703184|8.95|xylophone band +122|449|65751|4294967548|81.66|29.49|true|mike carson|2013-03-01 09:11:58.703258|30.68|wind surfing +9|308|65715|4294967297|12.54|27.61|false|bob king|2013-03-01 09:11:58.703309|89.40|geology +26|457|65590|4294967302|87.22|26.79|true|xavier king|2013-03-01 09:11:58.703176|67.80|forestry +59|390|65776|4294967421|97.81|40.22|false|katie king|2013-03-01 09:11:58.703132|49.73|zync studies +22|489|65717|4294967422|89.38|28.13|false|mike quirinius|2013-03-01 09:11:58.703262|5.81|chemistry +82|303|65764|4294967480|54.34|13.84|false|david xylophone|2013-03-01 09:11:58.703202|31.41|religion +27|261|65619|4294967401|88.78|18.28|false|bob ovid|2013-03-01 09:11:58.703207|36.48|wind surfing +24|333|65617|4294967370|10.26|16.29|false|yuri quirinius|2013-03-01 09:11:58.703179|10.82|geology +115|334|65653|4294967435|46.45|17.86|false|gabriella steinbeck|2013-03-01 09:11:58.703085|85.09|joggying +77|435|65578|4294967474|38.62|47.15|false|alice steinbeck|2013-03-01 09:11:58.703073|75.62|study skills +101|259|65759|4294967383|79.96|10.23|false|holly underhill|2013-03-01 09:11:58.703087|10.63|nap time +45|401|65680|4294967303|32.41|25.43|true|ulysses steinbeck|2013-03-01 09:11:58.703143|55.30|education +100|430|65570|4294967431|50.31|46.98|false|calvin laertes|2013-03-01 09:11:58.703154|76.80|yard duty +106|456|65548|4294967484|37.90|14.24|true|mike hernandez|2013-03-01 09:11:58.703294|58.51|chemistry +3|260|65659|4294967508|91.53|43.18|false|oscar ovid|2013-03-01 09:11:58.703281|99.87|chemistry +60|275|65654|4294967405|24.80|20.64|true|nick young|2013-03-01 09:11:58.703074|37.80|geology +50|346|65673|4294967549|55.51|32.20|true|alice steinbeck|2013-03-01 09:11:58.703198|33.27|yard duty +85|448|65784|4294967299|90.56|32.56|false|alice hernandez|2013-03-01 09:11:58.703278|68.18|opthamology +10|498|65549|4294967361|87.48|0.66|true|oscar carson|2013-03-01 09:11:58.703150|50.60|chemistry +16|466|65673|4294967318|87.57|45.11|true|bob garcia|2013-03-01 09:11:58.703173|87.78|history +33|508|65600|4294967372|12.02|17.08|true|jessica zipper|2013-03-01 09:11:58.703302|36.57|values clariffication +31|426|65602|4294967543|24.79|36.72|true|holly hernandez|2013-03-01 09:11:58.703306|8.13|quiet hour +61|266|65564|4294967461|95.81|3.07|false|priscilla nixon|2013-03-01 09:11:58.703269|90.34|study skills +41|329|65717|4294967463|30.36|30.37|false|rachel robinson|2013-03-01 09:11:58.703214|57.61|yard duty +114|271|65581|4294967520|61.88|23.27|false|oscar van buren|2013-03-01 09:11:58.703071|88.09|industrial engineering +22|267|65651|4294967449|68.25|17.52|false|ulysses hernandez|2013-03-01 09:11:58.703097|74.13|industrial engineering +120|406|65610|4294967538|59.99|39.78|true|quinn garcia|2013-03-01 09:11:58.703299|39.74|linguistics +5|372|65699|4294967375|80.97|31.76|false|katie ellison|2013-03-01 09:11:58.703112|13.29|american history +40|465|65713|4294967408|50.09|38.66|true|bob carson|2013-03-01 09:11:58.703273|10.94|wind surfing +115|281|65651|4294967434|80.46|35.90|true|gabriella king|2013-03-01 09:11:58.703156|84.42|joggying +5|395|65741|4294967521|29.54|40.04|true|calvin quirinius|2013-03-01 09:11:58.703131|63.81|religion +45|292|65769|4294967336|32.01|29.73|false|mike white|2013-03-01 09:11:58.703310|70.52|chemistry +116|396|65778|4294967515|30.65|49.38|true|quinn ellison|2013-03-01 09:11:58.703131|99.08|opthamology +68|264|65623|4294967481|69.97|47.03|true|zach garcia|2013-03-01 09:11:58.703235|28.60|education +18|280|65536|4294967320|32.92|45.94|false|holly white|2013-03-01 09:11:58.703086|58.86|topology +28|302|65653|4294967388|50.40|38.37|false|holly miller|2013-03-01 09:11:58.703195|3.81|philosophy +114|385|65541|4294967458|73.48|34.97|true|oscar quirinius|2013-03-01 09:11:58.703143|72.33|xylophone band +4|279|65745|4294967431|83.58|31.66|true|fred van buren|2013-03-01 09:11:58.703087|25.19|study skills +51|377|65626|4294967479|1.27|12.73|false|wendy miller|2013-03-01 09:11:58.703072|96.47|values clariffication +78|351|65755|4294967337|99.29|24.84|true|holly zipper|2013-03-01 09:11:58.703236|81.44|forestry +75|461|65760|4294967326|3.21|8.52|false|david brown|2013-03-01 09:11:58.703144|21.14|xylophone band +124|371|65687|4294967489|79.55|45.10|false|priscilla carson|2013-03-01 09:11:58.703277|3.37|philosophy +95|424|65643|4294967442|36.22|24.86|false|rachel king|2013-03-01 09:11:58.703261|71.29|yard duty +88|348|65645|4294967328|46.88|5.74|false|luke polk|2013-03-01 09:11:58.703306|3.95|yard duty +27|278|65622|4294967516|25.67|46.19|true|priscilla zipper|2013-03-01 09:11:58.703302|50.08|chemistry +23|302|65595|4294967497|22.01|5.84|true|wendy falkner|2013-03-01 09:11:58.703164|48.92|biology +30|441|65721|4294967456|40.17|28.17|false|sarah steinbeck|2013-03-01 09:11:58.703256|23.78|wind surfing +31|461|65608|4294967349|61.92|49.45|true|bob miller|2013-03-01 09:11:58.703255|48.90|biology +13|477|65680|4294967505|22.85|12.85|true|calvin steinbeck|2013-03-01 09:11:58.703277|54.11|zync studies +29|449|65596|4294967490|58.67|9.22|true|victor robinson|2013-03-01 09:11:58.703177|79.75|zync studies +13|503|65664|4294967406|52.50|41.37|true|holly laertes|2013-03-01 09:11:58.703196|34.71|yard duty +63|368|65714|4294967420|20.38|25.32|true|yuri xylophone|2013-03-01 09:11:58.703189|49.69|debate +44|437|65721|4294967376|22.27|3.27|true|yuri king|2013-03-01 09:11:58.703269|86.59|topology +35|500|65641|4294967469|68.85|13.20|true|victor ellison|2013-03-01 09:11:58.703245|41.16|undecided +49|271|65735|4294967454|59.83|22.33|false|oscar white|2013-03-01 09:11:58.703139|12.67|nap time +0|264|65670|4294967479|72.98|3.67|true|david xylophone|2013-03-01 09:11:58.703269|61.06|philosophy +47|463|65751|4294967525|71.68|30.61|true|jessica ovid|2013-03-01 09:11:58.703072|2.96|education +120|273|65724|4294967477|94.15|6.54|true|david ellison|2013-03-01 09:11:58.703091|36.05|education +6|486|65611|4294967485|74.00|27.42|false|ulysses steinbeck|2013-03-01 09:11:58.703258|21.21|education +53|311|65650|4294967299|48.11|22.64|true|wendy hernandez|2013-03-01 09:11:58.703146|89.12|kindergarten +117|396|65541|4294967434|74.78|6.62|true|katie miller|2013-03-01 09:11:58.703243|89.14|yard duty +24|454|65628|4294967337|88.55|31.61|false|victor young|2013-03-01 09:11:58.703296|30.43|undecided +116|485|65575|4294967296|95.84|14.26|true|irene polk|2013-03-01 09:11:58.703128|17.00|kindergarten +11|438|65622|4294967368|89.81|0.73|true|sarah white|2013-03-01 09:11:58.703287|88.60|biology +65|263|65671|4294967519|68.95|20.34|true|alice laertes|2013-03-01 09:11:58.703103|8.65|history +31|278|65634|4294967511|50.41|46.87|true|holly underhill|2013-03-01 09:11:58.703244|99.34|debate +98|466|65602|4294967470|8.32|1.17|false|david underhill|2013-03-01 09:11:58.703275|95.58|debate +114|384|65549|4294967361|55.39|36.12|true|holly king|2013-03-01 09:11:58.703077|55.04|forestry +35|463|65646|4294967363|50.83|34.68|true|gabriella miller|2013-03-01 09:11:58.703133|27.34|nap time +37|317|65684|4294967499|23.13|26.55|false|calvin laertes|2013-03-01 09:11:58.703162|3.51|study skills +80|390|65766|4294967360|67.26|8.82|true|xavier hernandez|2013-03-01 09:11:58.703288|66.13|biology +111|486|65620|4294967357|22.36|46.67|true|ulysses underhill|2013-03-01 09:11:58.703172|64.65|zync studies +88|322|65733|4294967316|80.58|32.61|false|quinn brown|2013-03-01 09:11:58.703272|99.80|yard duty +20|372|65587|4294967462|76.92|7.37|true|gabriella hernandez|2013-03-01 09:11:58.703182|11.86|mathematics +107|278|65550|4294967432|83.08|22.15|false|sarah zipper|2013-03-01 09:11:58.703131|1.75|forestry +50|405|65715|4294967538|79.12|16.49|false|calvin ovid|2013-03-01 09:11:58.703297|39.62|chemistry +100|310|65657|4294967393|20.67|7.45|false|gabriella king|2013-03-01 09:11:58.703290|26.40|xylophone band +66|477|65674|4294967313|8.45|4.83|false|wendy young|2013-03-01 09:11:58.703180|10.28|xylophone band +73|318|65610|4294967401|11.91|23.15|false|nick thompson|2013-03-01 09:11:58.703097|36.77|religion +-2|269|65681|4294967544|99.24|28.19|false|rachel falkner|2013-03-01 09:11:58.703089|41.57|quiet hour +85|495|65656|4294967507|90.12|3.99|true|fred polk|2013-03-01 09:11:58.703257|14.96|quiet hour +20|288|65758|4294967547|20.79|34.05|false|xavier carson|2013-03-01 09:11:58.703092|61.69|chemistry +123|299|65606|4294967525|99.36|7.66|false|sarah falkner|2013-03-01 09:11:58.703300|7.51|wind surfing +10|273|65750|4294967499|23.77|21.66|false|yuri underhill|2013-03-01 09:11:58.703177|20.91|religion +75|393|65726|4294967467|39.98|29.78|false|katie ichabod|2013-03-01 09:11:58.703238|6.00|industrial engineering +47|375|65775|4294967465|70.56|32.56|false|david robinson|2013-03-01 09:11:58.703238|7.88|linguistics +106|482|65624|4294967446|45.45|11.66|true|rachel laertes|2013-03-01 09:11:58.703119|58.02|debate +36|273|65739|4294967416|1.25|33.36|true|bob zipper|2013-03-01 09:11:58.703248|74.77|industrial engineering +92|335|65603|4294967296|80.96|18.48|false|ulysses nixon|2013-03-01 09:11:58.703198|45.63|opthamology +68|301|65579|4294967497|94.43|45.02|false|zach zipper|2013-03-01 09:11:58.703230|28.02|wind surfing +106|502|65695|4294967501|83.92|35.24|true|mike zipper|2013-03-01 09:11:58.703157|57.33|opthamology +122|396|65550|4294967350|60.02|32.13|true|sarah ovid|2013-03-01 09:11:58.703099|91.38|wind surfing +67|436|65541|4294967471|19.06|39.60|true|oscar king|2013-03-01 09:11:58.703125|14.94|opthamology +56|463|65587|4294967307|52.17|42.41|true|rachel brown|2013-03-01 09:11:58.703182|94.47|joggying +46|319|65683|4294967549|94.33|14.75|true|irene steinbeck|2013-03-01 09:11:58.703108|99.56|kindergarten +18|503|65750|4294967467|12.50|30.41|false|ulysses davidson|2013-03-01 09:11:58.703310|83.44|yard duty +43|473|65697|4294967457|79.97|11.35|false|david quirinius|2013-03-01 09:11:58.703206|50.01|undecided +120|413|65650|4294967543|48.25|10.47|false|sarah king|2013-03-01 09:11:58.703122|90.77|zync studies +91|448|65559|4294967320|90.35|6.28|true|gabriella ichabod|2013-03-01 09:11:58.703320|83.50|nap time +24|419|65755|4294967444|15.92|11.90|false|xavier davidson|2013-03-01 09:11:58.703208|38.39|yard duty +63|481|65591|4294967382|28.31|36.11|true|irene underhill|2013-03-01 09:11:58.703177|11.43|history +118|497|65738|4294967546|19.14|5.09|true|alice king|2013-03-01 09:11:58.703197|82.32|industrial engineering +25|507|65643|4294967312|52.53|6.06|false|katie van buren|2013-03-01 09:11:58.703323|93.11|opthamology +79|422|65691|4294967299|39.21|29.52|true|zach quirinius|2013-03-01 09:11:58.703136|38.23|philosophy +27|367|65675|4294967518|12.32|40.26|true|tom brown|2013-03-01 09:11:58.703104|87.36|mathematics +96|421|65692|4294967526|98.48|34.41|true|oscar falkner|2013-03-01 09:11:58.703214|54.76|zync studies +27|340|65677|4294967461|98.96|45.78|true|rachel carson|2013-03-01 09:11:58.703208|25.37|zync studies +3|469|65743|4294967428|10.66|39.84|false|victor zipper|2013-03-01 09:11:58.703181|26.60|mathematics +122|288|65695|4294967469|90.05|12.70|false|zach steinbeck|2013-03-01 09:11:58.703314|39.12|education +81|449|65726|4294967395|97.09|40.15|false|bob ovid|2013-03-01 09:11:58.703250|60.46|kindergarten +77|337|65766|4294967334|12.16|13.49|true|wendy quirinius|2013-03-01 09:11:58.703282|37.06|history +5|319|65611|4294967494|45.92|2.61|true|wendy nixon|2013-03-01 09:11:58.703209|25.92|nap time +8|469|65592|4294967535|17.79|48.89|false|yuri steinbeck|2013-03-01 09:11:58.703299|23.19|kindergarten +74|487|65563|4294967305|60.06|13.22|true|ulysses polk|2013-03-01 09:11:58.703222|38.23|yard duty +85|454|65620|4294967520|65.55|13.20|false|victor quirinius|2013-03-01 09:11:58.703152|6.85|history +104|448|65693|4294967459|80.92|36.70|false|rachel falkner|2013-03-01 09:11:58.703140|32.33|zync studies +114|279|65544|4294967383|6.67|19.87|false|xavier hernandez|2013-03-01 09:11:58.703324|48.84|geology +120|331|65539|4294967324|88.02|40.94|true|holly nixon|2013-03-01 09:11:58.703262|96.64|yard duty +84|326|65782|4294967362|17.87|38.67|true|victor ellison|2013-03-01 09:11:58.703226|34.31|joggying +45|408|65577|4294967461|59.43|36.13|false|ethan falkner|2013-03-01 09:11:58.703317|41.01|values clariffication +88|504|65652|4294967419|78.28|47.86|false|priscilla white|2013-03-01 09:11:58.703324|55.04|history +73|306|65659|4294967341|34.58|45.90|false|yuri white|2013-03-01 09:11:58.703201|75.91|nap time +97|360|65590|4294967324|68.96|49.52|false|alice miller|2013-03-01 09:11:58.703247|22.13|philosophy +89|300|65675|4294967384|49.77|12.43|true|jessica davidson|2013-03-01 09:11:58.703276|79.90|values clariffication +118|309|65578|4294967357|75.10|33.02|false|tom ellison|2013-03-01 09:11:58.703197|16.01|zync studies +34|324|65549|4294967307|17.16|38.94|false|quinn underhill|2013-03-01 09:11:58.703111|28.93|forestry +93|441|65549|4294967389|95.11|0.27|true|quinn davidson|2013-03-01 09:11:58.703278|13.30|joggying +117|323|65669|4294967316|90.25|41.90|false|alice falkner|2013-03-01 09:11:58.703221|54.25|study skills +118|374|65541|4294967528|6.72|38.04|false|priscilla ovid|2013-03-01 09:11:58.703304|9.04|undecided +120|298|65763|4294967513|3.33|21.11|false|oscar ichabod|2013-03-01 09:11:58.703094|86.43|education +100|354|65649|4294967358|7.54|35.36|true|zach zipper|2013-03-01 09:11:58.703174|79.51|history +37|379|65737|4294967314|88.17|9.21|true|alice hernandez|2013-03-01 09:11:58.703089|29.92|opthamology +115|502|65773|4294967522|37.59|0.02|true|yuri laertes|2013-03-01 09:11:58.703260|3.81|american history +73|324|65582|4294967330|59.68|32.25|false|jessica ovid|2013-03-01 09:11:58.703109|96.60|religion +51|484|65635|4294967500|2.18|25.75|true|oscar van buren|2013-03-01 09:11:58.703255|82.29|yard duty +16|445|65620|4294967517|21.61|0.86|true|mike van buren|2013-03-01 09:11:58.703110|8.16|opthamology +102|501|65710|4294967332|47.27|1.31|false|xavier underhill|2013-03-01 09:11:58.703094|57.11|geology +28|372|65578|4294967374|75.42|5.93|true|holly thompson|2013-03-01 09:11:58.703205|54.35|debate +12|372|65573|4294967345|94.31|35.23|false|calvin falkner|2013-03-01 09:11:58.703122|53.56|debate +19|492|65588|4294967445|19.06|14.83|false|tom ichabod|2013-03-01 09:11:58.703171|69.48|nap time +70|430|65539|4294967534|18.89|43.84|true|tom carson|2013-03-01 09:11:58.703182|21.93|joggying +33|321|65594|4294967495|97.71|3.40|false|fred young|2013-03-01 09:11:58.703288|60.57|zync studies +44|452|65645|4294967345|56.15|16.90|true|oscar johnson|2013-03-01 09:11:58.703257|66.76|industrial engineering +53|293|65778|4294967418|16.09|14.99|true|oscar johnson|2013-03-01 09:11:58.703235|89.51|geology +73|451|65599|4294967373|36.89|33.64|true|zach ichabod|2013-03-01 09:11:58.703315|4.93|opthamology +104|474|65773|4294967473|40.98|11.43|false|quinn garcia|2013-03-01 09:11:58.703167|76.46|values clariffication +31|356|65721|4294967481|1.08|3.03|true|rachel ovid|2013-03-01 09:11:58.703120|95.38|zync studies +23|306|65777|4294967489|13.99|21.69|true|oscar zipper|2013-03-01 09:11:58.703280|54.39|zync studies +93|342|65623|4294967405|99.67|44.90|true|ulysses underhill|2013-03-01 09:11:58.703083|31.22|biology +18|440|65773|4294967318|57.23|1.42|false|oscar xylophone|2013-03-01 09:11:58.703165|63.26|zync studies +72|489|65549|4294967424|82.56|13.88|false|bob ichabod|2013-03-01 09:11:58.703112|80.09|quiet hour +12|284|65666|4294967470|89.80|42.04|false|ulysses garcia|2013-03-01 09:11:58.703073|28.69|opthamology +0|312|65599|4294967519|78.21|45.53|false|alice xylophone|2013-03-01 09:11:58.703289|0.56|nap time +24|483|65572|4294967499|79.42|9.96|false|xavier xylophone|2013-03-01 09:11:58.703101|22.64|quiet hour +34|405|65719|4294967306|1.31|23.48|false|katie white|2013-03-01 09:11:58.703198|51.13|nap time +99|497|65656|4294967390|32.37|6.21|true|wendy polk|2013-03-01 09:11:58.703212|53.27|mathematics +83|493|65742|4294967352|76.33|28.36|true|ethan carson|2013-03-01 09:11:58.703269|43.84|philosophy +106|282|65688|4294967433|75.19|13.08|true|yuri brown|2013-03-01 09:11:58.703118|77.88|debate +81|305|65783|4294967443|39.01|33.02|true|bob king|2013-03-01 09:11:58.703114|77.40|chemistry +74|309|65662|4294967518|4.17|38.43|false|rachel thompson|2013-03-01 09:11:58.703243|76.11|undecided +63|364|65705|4294967490|73.68|4.96|true|wendy white|2013-03-01 09:11:58.703186|13.81|forestry +46|363|65735|4294967537|72.79|29.22|true|ulysses brown|2013-03-01 09:11:58.703271|61.16|joggying +84|323|65685|4294967477|61.86|14.91|false|ethan brown|2013-03-01 09:11:58.703256|38.71|biology +4|392|65665|4294967391|53.27|3.86|true|zach miller|2013-03-01 09:11:58.703296|43.66|undecided +94|474|65759|4294967317|57.08|31.36|false|zach brown|2013-03-01 09:11:58.703239|69.24|xylophone band +26|491|65683|4294967420|46.62|48.52|false|wendy laertes|2013-03-01 09:11:58.703178|97.21|values clariffication +35|488|65737|4294967502|62.52|3.15|true|xavier quirinius|2013-03-01 09:11:58.703096|42.64|linguistics +59|382|65762|4294967468|25.28|35.10|true|david robinson|2013-03-01 09:11:58.703126|5.49|mathematics +58|427|65597|4294967511|71.19|47.55|false|xavier ichabod|2013-03-01 09:11:58.703323|36.22|chemistry +18|428|65775|4294967436|74.19|48.08|true|irene xylophone|2013-03-01 09:11:58.703122|98.43|geology +69|489|65536|4294967404|33.52|17.99|false|oscar ichabod|2013-03-01 09:11:58.703247|32.68|topology +71|439|65618|4294967349|49.78|4.57|false|ethan hernandez|2013-03-01 09:11:58.703075|10.06|yard duty +60|475|65664|4294967299|44.43|25.02|false|irene laertes|2013-03-01 09:11:58.703172|55.82|quiet hour +35|280|65779|4294967322|1.87|16.04|false|luke ellison|2013-03-01 09:11:58.703180|11.87|religion +122|473|65629|4294967387|90.77|25.59|true|fred underhill|2013-03-01 09:11:58.703316|58.81|undecided +-1|423|65663|4294967380|0.79|21.33|false|bob laertes|2013-03-01 09:11:58.703278|94.16|debate +68|314|65770|4294967398|70.85|29.24|true|mike garcia|2013-03-01 09:11:58.703312|51.90|american history +77|328|65789|4294967489|81.32|19.12|false|fred ichabod|2013-03-01 09:11:58.703160|83.18|debate +60|342|65550|4294967306|90.38|45.69|false|victor brown|2013-03-01 09:11:58.703311|7.38|biology +53|422|65784|4294967450|56.04|43.76|true|calvin falkner|2013-03-01 09:11:58.703193|93.95|zync studies +103|504|65768|4294967354|53.93|16.42|true|bob davidson|2013-03-01 09:11:58.703141|78.25|kindergarten +8|272|65541|4294967325|63.90|11.20|true|oscar polk|2013-03-01 09:11:58.703136|27.89|debate +108|446|65733|4294967403|7.11|32.85|false|ethan brown|2013-03-01 09:11:58.703170|11.44|american history +42|315|65782|4294967369|4.46|11.63|false|bob garcia|2013-03-01 09:11:58.703292|30.24|industrial engineering +16|482|65736|4294967310|43.19|46.30|true|ethan garcia|2013-03-01 09:11:58.703243|28.90|wind surfing +28|454|65612|4294967480|65.02|43.00|false|ulysses polk|2013-03-01 09:11:58.703140|25.10|philosophy +58|482|65775|4294967417|68.04|19.41|false|wendy underhill|2013-03-01 09:11:58.703212|88.25|yard duty +88|437|65762|4294967385|76.71|13.83|false|david laertes|2013-03-01 09:11:58.703169|45.70|religion +83|265|65705|4294967392|19.28|5.42|true|quinn ellison|2013-03-01 09:11:58.703275|20.57|education +2|266|65551|4294967296|35.65|45.12|true|david young|2013-03-01 09:11:58.703184|73.93|education +91|415|65672|4294967316|59.45|20.80|true|mike hernandez|2013-03-01 09:11:58.703241|75.27|study skills +34|379|65750|4294967441|98.18|20.58|true|alice underhill|2013-03-01 09:11:58.703170|96.85|mathematics +86|451|65580|4294967451|8.71|47.68|true|ulysses polk|2013-03-01 09:11:58.703165|81.42|geology +46|266|65762|4294967305|2.97|19.56|false|calvin falkner|2013-03-01 09:11:58.703103|77.96|chemistry +81|338|65681|4294967519|50.66|16.10|false|katie ovid|2013-03-01 09:11:58.703259|61.30|wind surfing +17|441|65633|4294967460|89.15|21.11|true|priscilla johnson|2013-03-01 09:11:58.703139|28.49|education +112|312|65685|4294967395|33.76|21.38|true|wendy young|2013-03-01 09:11:58.703182|4.88|religion +44|270|65751|4294967301|67.48|37.12|false|oscar garcia|2013-03-01 09:11:58.703239|47.81|religion +37|411|65680|4294967353|2.92|29.66|true|ethan zipper|2013-03-01 09:11:58.703132|34.08|american history +95|329|65628|4294967396|61.70|20.75|true|david ovid|2013-03-01 09:11:58.703223|83.26|nap time +33|497|65612|4294967375|28.69|14.84|true|fred nixon|2013-03-01 09:11:58.703070|52.64|values clariffication +55|328|65703|4294967492|68.41|0.63|true|ulysses carson|2013-03-01 09:11:58.703282|37.18|geology +17|483|65713|4294967458|56.81|1.69|false|calvin xylophone|2013-03-01 09:11:58.703250|90.91|mathematics +61|510|65675|4294967391|88.07|13.44|false|ulysses young|2013-03-01 09:11:58.703281|24.19|joggying +53|279|65541|4294967441|77.97|14.72|false|xavier ellison|2013-03-01 09:11:58.703091|15.85|american history +114|304|65643|4294967457|54.75|46.53|false|ethan laertes|2013-03-01 09:11:58.703250|52.56|mathematics +61|405|65609|4294967301|16.48|48.50|false|mike falkner|2013-03-01 09:11:58.703159|83.82|wind surfing +93|433|65629|4294967439|39.83|8.56|false|katie king|2013-03-01 09:11:58.703125|72.90|values clariffication +6|348|65785|4294967326|57.37|33.90|true|ulysses underhill|2013-03-01 09:11:58.703226|68.30|opthamology +39|492|65701|4294967458|23.96|19.65|true|fred polk|2013-03-01 09:11:58.703262|1.81|xylophone band +74|256|65778|4294967487|74.15|25.49|true|david davidson|2013-03-01 09:11:58.703151|95.69|yard duty +99|435|65546|4294967535|22.85|3.12|false|tom young|2013-03-01 09:11:58.703111|91.13|history +106|441|65771|4294967386|83.93|28.44|false|xavier allen|2013-03-01 09:11:58.703226|70.88|xylophone band +124|392|65711|4294967498|15.37|18.83|false|irene garcia|2013-03-01 09:11:58.703158|71.89|industrial engineering +9|336|65658|4294967412|43.17|18.38|false|katie ichabod|2013-03-01 09:11:58.703091|68.85|mathematics +3|395|65747|4294967313|57.25|3.17|true|wendy garcia|2013-03-01 09:11:58.703074|58.47|xylophone band +123|381|65780|4294967388|5.44|35.16|true|tom carson|2013-03-01 09:11:58.703220|10.95|quiet hour +58|256|65733|4294967501|70.53|23.07|true|zach white|2013-03-01 09:11:58.703090|50.99|history +106|284|65619|4294967527|92.96|0.86|true|mike nixon|2013-03-01 09:11:58.703084|32.64|xylophone band +58|289|65604|4294967313|77.36|10.49|false|wendy falkner|2013-03-01 09:11:58.703075|23.89|kindergarten +6|342|65602|4294967365|85.49|24.46|false|zach steinbeck|2013-03-01 09:11:58.703111|85.94|study skills +89|354|65593|4294967462|97.51|21.01|true|ethan zipper|2013-03-01 09:11:58.703201|35.15|forestry +44|364|65614|4294967413|4.35|9.19|false|ethan king|2013-03-01 09:11:58.703219|1.42|nap time +43|436|65678|4294967424|50.92|3.40|true|luke van buren|2013-03-01 09:11:58.703093|33.90|values clariffication +27|335|65617|4294967381|64.87|25.03|false|david allen|2013-03-01 09:11:58.703140|64.90|nap time +44|390|65693|4294967504|1.29|28.40|false|bob davidson|2013-03-01 09:11:58.703127|84.88|history +42|315|65713|4294967544|88.48|46.90|true|ulysses underhill|2013-03-01 09:11:58.703259|39.47|education +54|327|65657|4294967334|42.42|2.89|true|jessica zipper|2013-03-01 09:11:58.703182|65.01|linguistics +68|501|65606|4294967445|5.08|39.85|true|alice robinson|2013-03-01 09:11:58.703206|87.16|topology +37|293|65762|4294967316|67.38|41.20|true|zach brown|2013-03-01 09:11:58.703155|21.70|forestry +10|471|65563|4294967316|37.72|45.06|false|tom quirinius|2013-03-01 09:11:58.703248|14.36|quiet hour +42|353|65672|4294967465|70.04|27.62|false|xavier garcia|2013-03-01 09:11:58.703153|13.28|forestry +15|271|65647|4294967392|45.06|35.36|true|quinn young|2013-03-01 09:11:58.703209|81.95|kindergarten +71|332|65717|4294967337|10.73|34.17|true|gabriella ichabod|2013-03-01 09:11:58.703288|80.05|linguistics +92|418|65706|4294967367|2.07|16.35|false|yuri hernandez|2013-03-01 09:11:58.703205|85.13|joggying +101|326|65586|4294967435|30.81|4.27|true|rachel brown|2013-03-01 09:11:58.703179|18.65|undecided +76|422|65552|4294967325|26.73|48.00|false|mike zipper|2013-03-01 09:11:58.703137|18.11|industrial engineering +119|373|65704|4294967459|27.07|27.54|false|mike polk|2013-03-01 09:11:58.703202|54.76|biology +70|283|65620|4294967355|9.13|3.42|false|zach falkner|2013-03-01 09:11:58.703252|96.95|debate +117|267|65637|4294967544|93.64|11.69|false|david brown|2013-03-01 09:11:58.703224|26.78|topology +88|328|65547|4294967517|70.35|15.13|true|alice davidson|2013-03-01 09:11:58.703262|17.71|undecided +22|447|65757|4294967539|8.79|6.52|true|bob king|2013-03-01 09:11:58.703325|9.24|joggying +-1|300|65663|4294967343|71.26|34.62|true|calvin ovid|2013-03-01 09:11:58.703262|78.56|study skills +92|297|65704|4294967403|84.83|0.21|false|gabriella brown|2013-03-01 09:11:58.703089|16.22|wind surfing +25|446|65789|4294967501|5.54|49.56|false|jessica garcia|2013-03-01 09:11:58.703286|92.74|linguistics +65|274|65619|4294967389|12.45|41.33|true|mike polk|2013-03-01 09:11:58.703219|34.72|nap time +111|287|65587|4294967421|48.22|12.74|false|yuri johnson|2013-03-01 09:11:58.703227|64.06|values clariffication +64|371|65685|4294967320|15.63|39.84|false|ethan brown|2013-03-01 09:11:58.703132|43.48|opthamology +72|421|65764|4294967458|88.80|39.49|true|rachel falkner|2013-03-01 09:11:58.703227|88.69|topology +108|301|65536|4294967357|90.05|17.59|true|ethan johnson|2013-03-01 09:11:58.703271|75.70|undecided +-2|461|65648|4294967425|58.52|24.85|false|rachel thompson|2013-03-01 09:11:58.703318|85.62|zync studies +113|395|65666|4294967447|26.49|13.44|true|tom steinbeck|2013-03-01 09:11:58.703247|83.95|industrial engineering +106|415|65644|4294967399|63.35|18.38|false|xavier davidson|2013-03-01 09:11:58.703234|44.10|study skills +117|483|65627|4294967547|21.18|49.46|true|priscilla ichabod|2013-03-01 09:11:58.703184|61.89|philosophy +44|460|65693|4294967423|58.00|23.77|false|quinn zipper|2013-03-01 09:11:58.703165|8.38|kindergarten +71|324|65681|4294967388|47.59|33.67|false|nick ichabod|2013-03-01 09:11:58.703106|71.17|xylophone band +90|269|65648|4294967392|42.31|11.27|true|holly king|2013-03-01 09:11:58.703196|0.08|american history +22|264|65537|4294967419|8.07|10.71|false|david xylophone|2013-03-01 09:11:58.703136|11.87|undecided +51|447|65791|4294967354|28.95|44.27|false|luke davidson|2013-03-01 09:11:58.703292|4.24|chemistry +6|397|65660|4294967512|0.27|24.95|false|nick young|2013-03-01 09:11:58.703156|62.42|industrial engineering +122|392|65619|4294967527|20.81|27.27|false|nick underhill|2013-03-01 09:11:58.703305|18.68|education +43|461|65686|4294967544|20.82|30.99|false|holly hernandez|2013-03-01 09:11:58.703286|98.68|history +21|280|65716|4294967393|48.08|29.62|true|gabriella ellison|2013-03-01 09:11:58.703264|85.93|american history +10|453|65745|4294967397|89.01|24.03|true|nick ellison|2013-03-01 09:11:58.703108|83.13|topology +56|392|65586|4294967410|48.15|29.76|true|alice nixon|2013-03-01 09:11:58.703223|94.25|american history +95|415|65560|4294967444|4.71|17.29|false|quinn laertes|2013-03-01 09:11:58.703267|92.59|xylophone band +13|465|65713|4294967306|26.76|6.56|true|yuri polk|2013-03-01 09:11:58.703169|47.77|american history +34|425|65660|4294967371|58.43|15.54|true|irene garcia|2013-03-01 09:11:58.703172|28.05|forestry +59|496|65576|4294967331|15.75|28.68|false|tom king|2013-03-01 09:11:58.703215|66.55|opthamology +-1|433|65581|4294967299|86.92|23.15|false|yuri ellison|2013-03-01 09:11:58.703098|21.29|history +13|260|65699|4294967496|41.87|9.26|false|sarah king|2013-03-01 09:11:58.703128|84.79|study skills +45|298|65747|4294967434|15.15|31.01|false|priscilla polk|2013-03-01 09:11:58.703113|21.80|yard duty +31|366|65557|4294967458|41.71|1.98|false|sarah miller|2013-03-01 09:11:58.703216|69.92|linguistics +63|464|65596|4294967327|44.57|0.47|true|holly allen|2013-03-01 09:11:58.703200|12.72|nap time +5|357|65736|4294967351|0.60|15.38|false|rachel ovid|2013-03-01 09:11:58.703246|86.51|xylophone band +16|483|65658|4294967350|38.88|2.89|false|oscar robinson|2013-03-01 09:11:58.703225|66.53|topology +90|487|65695|4294967313|57.93|15.18|false|yuri quirinius|2013-03-01 09:11:58.703320|77.09|education +117|302|65615|4294967382|21.32|31.46|false|ethan polk|2013-03-01 09:11:58.703274|10.79|kindergarten +75|504|65564|4294967534|7.56|19.79|false|mike steinbeck|2013-03-01 09:11:58.703148|26.97|xylophone band +86|257|65748|4294967427|32.52|26.87|true|ulysses young|2013-03-01 09:11:58.703192|29.49|values clariffication +112|348|65643|4294967388|30.61|21.27|false|oscar polk|2013-03-01 09:11:58.703076|22.05|yard duty +91|307|65544|4294967344|4.57|0.60|false|rachel robinson|2013-03-01 09:11:58.703227|79.21|undecided +49|407|65639|4294967362|27.66|34.06|true|yuri garcia|2013-03-01 09:11:58.703163|97.48|mathematics +95|459|65760|4294967524|85.74|49.16|false|mike ellison|2013-03-01 09:11:58.703074|48.20|nap time +76|258|65724|4294967365|70.00|34.95|false|irene quirinius|2013-03-01 09:11:58.703212|10.52|geology +17|406|65585|4294967401|0.43|29.19|true|priscilla young|2013-03-01 09:11:58.703235|33.69|religion +90|273|65548|4294967366|62.90|12.30|true|alice polk|2013-03-01 09:11:58.703074|53.52|study skills +41|301|65706|4294967501|51.84|7.02|true|ethan quirinius|2013-03-01 09:11:58.703265|55.94|study skills +38|376|65783|4294967507|10.19|38.28|false|tom quirinius|2013-03-01 09:11:58.703185|52.43|kindergarten +50|317|65709|4294967379|68.50|7.36|true|victor nixon|2013-03-01 09:11:58.703162|67.69|forestry +33|265|65713|4294967529|86.69|16.40|true|holly thompson|2013-03-01 09:11:58.703189|72.37|history +89|476|65728|4294967549|62.30|14.07|false|alice quirinius|2013-03-01 09:11:58.703195|55.37|zync studies +24|339|65724|4294967328|64.95|37.02|false|rachel robinson|2013-03-01 09:11:58.703182|75.06|wind surfing +124|421|65743|4294967434|3.62|21.02|true|holly polk|2013-03-01 09:11:58.703173|62.00|education +85|463|65542|4294967348|3.86|46.36|true|ethan quirinius|2013-03-01 09:11:58.703287|42.85|biology +66|264|65600|4294967388|28.71|43.96|false|priscilla nixon|2013-03-01 09:11:58.703169|92.92|yard duty +112|373|65680|4294967449|90.28|49.34|false|sarah falkner|2013-03-01 09:11:58.703085|1.01|geology +28|480|65637|4294967328|76.69|33.76|false|oscar ichabod|2013-03-01 09:11:58.703323|93.41|kindergarten +101|476|65631|4294967538|29.78|23.61|false|irene nixon|2013-03-01 09:11:58.703270|33.72|values clariffication +96|435|65658|4294967523|6.74|28.52|false|yuri falkner|2013-03-01 09:11:58.703099|11.69|joggying +54|478|65615|4294967344|9.74|21.52|false|jessica van buren|2013-03-01 09:11:58.703284|99.88|philosophy +89|354|65777|4294967509|71.68|46.03|true|ulysses polk|2013-03-01 09:11:58.703324|82.62|wind surfing +16|436|65757|4294967379|7.06|28.15|true|rachel zipper|2013-03-01 09:11:58.703267|12.22|forestry +78|275|65596|4294967299|21.70|49.32|true|calvin xylophone|2013-03-01 09:11:58.703166|85.74|kindergarten +53|489|65752|4294967329|83.27|27.20|true|holly ichabod|2013-03-01 09:11:58.703079|47.82|industrial engineering +111|288|65770|4294967544|80.84|16.79|true|mike van buren|2013-03-01 09:11:58.703220|46.71|topology +85|334|65610|4294967373|2.96|2.36|false|ulysses miller|2013-03-01 09:11:58.703310|9.24|xylophone band +71|418|65598|4294967439|80.30|13.66|true|bob garcia|2013-03-01 09:11:58.703104|63.30|values clariffication +40|299|65712|4294967352|25.37|19.72|false|ethan miller|2013-03-01 09:11:58.703207|53.65|opthamology +32|458|65783|4294967309|45.42|44.83|false|jessica underhill|2013-03-01 09:11:58.703206|46.34|undecided +5|329|65789|4294967502|56.10|47.22|true|alice robinson|2013-03-01 09:11:58.703264|52.49|quiet hour +34|450|65607|4294967328|18.20|18.79|true|victor johnson|2013-03-01 09:11:58.703233|71.75|topology +58|359|65675|4294967486|3.61|19.20|true|katie ellison|2013-03-01 09:11:58.703243|35.78|values clariffication +-2|288|65658|4294967383|53.78|19.92|false|holly nixon|2013-03-01 09:11:58.703224|42.93|industrial engineering +0|392|65634|4294967299|65.70|48.98|true|luke robinson|2013-03-01 09:11:58.703263|14.40|geology +3|277|65788|4294967403|58.08|20.55|false|xavier ovid|2013-03-01 09:11:58.703281|62.11|zync studies +29|331|65539|4294967420|73.18|28.96|true|ethan brown|2013-03-01 09:11:58.703094|58.85|zync studies +57|420|65771|4294967508|34.21|16.12|false|calvin van buren|2013-03-01 09:11:58.703243|6.15|joggying +41|266|65774|4294967365|41.68|36.86|false|victor van buren|2013-03-01 09:11:58.703163|45.97|industrial engineering +116|487|65659|4294967309|77.66|35.15|true|sarah johnson|2013-03-01 09:11:58.703294|95.71|opthamology +54|481|65755|4294967429|28.50|37.76|false|victor xylophone|2013-03-01 09:11:58.703318|64.00|xylophone band +60|301|65746|4294967381|4.41|12.30|true|wendy garcia|2013-03-01 09:11:58.703183|36.74|education +104|266|65704|4294967413|59.16|34.84|true|jessica ichabod|2013-03-01 09:11:58.703172|93.21|forestry +105|345|65582|4294967478|12.42|15.27|false|wendy miller|2013-03-01 09:11:58.703287|0.89|philosophy +97|503|65690|4294967463|77.57|27.54|true|priscilla brown|2013-03-01 09:11:58.703110|51.82|study skills +18|286|65621|4294967494|64.77|37.32|true|mike ichabod|2013-03-01 09:11:58.703285|95.14|forestry +1|446|65683|4294967500|85.49|33.18|true|oscar hernandez|2013-03-01 09:11:58.703283|17.62|undecided +12|298|65653|4294967501|42.56|4.25|true|irene nixon|2013-03-01 09:11:58.703265|52.23|nap time +5|475|65644|4294967470|70.06|18.29|true|gabriella van buren|2013-03-01 09:11:58.703088|1.97|american history +54|321|65724|4294967486|47.69|49.73|true|david ellison|2013-03-01 09:11:58.703235|50.96|religion +88|331|65682|4294967412|73.32|17.76|true|gabriella thompson|2013-03-01 09:11:58.703283|67.17|values clariffication +56|310|65556|4294967518|38.85|34.53|false|irene allen|2013-03-01 09:11:58.703217|50.42|education +71|306|65664|4294967541|34.97|44.40|false|victor van buren|2013-03-01 09:11:58.703185|38.42|religion +114|402|65622|4294967542|59.87|24.61|true|ethan polk|2013-03-01 09:11:58.703265|93.70|yard duty +43|440|65693|4294967409|38.05|7.71|true|luke ovid|2013-03-01 09:11:58.703133|69.32|xylophone band +26|296|65677|4294967419|66.89|49.04|true|sarah robinson|2013-03-01 09:11:58.703277|5.06|wind surfing +68|472|65628|4294967512|8.67|19.47|false|tom ovid|2013-03-01 09:11:58.703116|74.31|linguistics +53|373|65691|4294967371|79.75|32.39|false|irene ovid|2013-03-01 09:11:58.703124|69.80|nap time +18|509|65638|4294967455|62.92|14.62|false|victor laertes|2013-03-01 09:11:58.703075|42.47|values clariffication +53|482|65736|4294967394|74.59|36.84|true|mike young|2013-03-01 09:11:58.703301|48.54|joggying +103|494|65760|4294967301|59.21|8.72|false|ethan laertes|2013-03-01 09:11:58.703312|95.45|nap time +80|277|65671|4294967370|97.26|7.14|false|sarah ichabod|2013-03-01 09:11:58.703297|36.34|industrial engineering +28|380|65714|4294967514|0.28|48.71|false|ethan ellison|2013-03-01 09:11:58.703118|47.42|chemistry +1|414|65716|4294967551|12.19|36.57|true|nick polk|2013-03-01 09:11:58.703188|33.40|xylophone band +18|417|65766|4294967499|37.78|40.96|false|ethan nixon|2013-03-01 09:11:58.703157|66.34|american history +52|511|65658|4294967432|7.96|14.33|false|priscilla carson|2013-03-01 09:11:58.703188|46.61|geology +13|374|65553|4294967448|62.20|10.15|false|jessica carson|2013-03-01 09:11:58.703185|25.31|debate +19|462|65732|4294967478|59.70|24.13|false|calvin underhill|2013-03-01 09:11:58.703074|91.15|debate +87|325|65730|4294967516|13.15|4.44|false|wendy ichabod|2013-03-01 09:11:58.703242|67.81|joggying +99|303|65566|4294967353|98.57|40.50|true|ulysses ichabod|2013-03-01 09:11:58.703104|7.80|wind surfing +87|341|65733|4294967412|18.93|16.13|true|katie zipper|2013-03-01 09:11:58.703190|93.45|american history +83|311|65653|4294967419|61.21|13.04|false|xavier polk|2013-03-01 09:11:58.703310|55.12|study skills +65|458|65554|4294967329|69.96|33.38|false|calvin ovid|2013-03-01 09:11:58.703237|0.24|nap time +123|278|65573|4294967496|86.22|39.74|true|zach underhill|2013-03-01 09:11:58.703078|75.99|history +19|463|65767|4294967437|48.52|26.71|true|quinn underhill|2013-03-01 09:11:58.703177|95.57|forestry +108|466|65600|4294967458|90.73|8.95|false|xavier brown|2013-03-01 09:11:58.703235|75.18|biology +53|435|65687|4294967451|76.70|4.58|false|wendy xylophone|2013-03-01 09:11:58.703243|11.26|nap time +54|346|65751|4294967398|98.31|34.54|false|holly polk|2013-03-01 09:11:58.703195|78.75|history +88|334|65583|4294967468|5.88|45.10|false|irene johnson|2013-03-01 09:11:58.703166|28.07|kindergarten +52|365|65691|4294967551|99.15|21.81|false|tom robinson|2013-03-01 09:11:58.703135|76.98|values clariffication +28|259|65544|4294967398|54.31|25.80|false|yuri quirinius|2013-03-01 09:11:58.703168|32.03|values clariffication +26|347|65734|4294967499|5.24|30.92|true|irene ovid|2013-03-01 09:11:58.703192|80.48|debate +108|280|65765|4294967434|4.79|30.35|false|irene brown|2013-03-01 09:11:58.703229|88.52|american history +2|291|65789|4294967464|16.99|2.07|false|bob falkner|2013-03-01 09:11:58.703139|43.53|industrial engineering +43|476|65570|4294967398|1.12|10.52|true|yuri ellison|2013-03-01 09:11:58.703244|30.98|forestry +114|433|65572|4294967463|43.02|20.30|false|zach carson|2013-03-01 09:11:58.703316|30.86|opthamology +119|278|65773|4294967488|31.15|23.18|true|wendy xylophone|2013-03-01 09:11:58.703312|18.20|xylophone band +108|263|65787|4294967297|12.44|29.46|true|rachel quirinius|2013-03-01 09:11:58.703073|21.76|industrial engineering +37|434|65638|4294967335|55.18|45.54|false|gabriella white|2013-03-01 09:11:58.703196|19.13|zync studies +124|311|65670|4294967420|87.94|3.07|false|wendy king|2013-03-01 09:11:58.703308|61.54|values clariffication +104|331|65757|4294967330|30.71|47.91|false|katie ichabod|2013-03-01 09:11:58.703197|34.84|study skills +25|438|65782|4294967525|29.40|45.71|true|fred quirinius|2013-03-01 09:11:58.703283|81.90|nap time +71|347|65584|4294967549|99.65|21.14|false|jessica falkner|2013-03-01 09:11:58.703286|65.78|philosophy +51|345|65566|4294967519|70.38|31.33|true|wendy laertes|2013-03-01 09:11:58.703163|21.34|chemistry +2|333|65697|4294967456|39.90|16.15|false|yuri johnson|2013-03-01 09:11:58.703272|77.35|study skills +114|425|65772|4294967307|55.10|34.83|true|luke robinson|2013-03-01 09:11:58.703166|82.33|religion +109|344|65595|4294967409|42.24|5.35|false|irene polk|2013-03-01 09:11:58.703089|8.76|education +95|432|65711|4294967378|39.87|30.78|false|wendy allen|2013-03-01 09:11:58.703172|65.17|american history +121|342|65565|4294967528|52.85|20.30|true|yuri allen|2013-03-01 09:11:58.703298|8.80|undecided +22|361|65648|4294967546|91.88|17.19|false|mike white|2013-03-01 09:11:58.703167|17.66|quiet hour +81|283|65702|4294967424|24.52|6.81|false|zach king|2013-03-01 09:11:58.703136|68.04|religion +15|297|65728|4294967362|18.50|24.18|false|katie falkner|2013-03-01 09:11:58.703189|36.96|education +77|441|65727|4294967391|43.13|17.47|false|rachel young|2013-03-01 09:11:58.703313|49.19|education +24|266|65747|4294967351|32.75|36.09|true|ethan allen|2013-03-01 09:11:58.703273|52.77|zync studies +89|313|65550|4294967361|41.89|33.02|true|david thompson|2013-03-01 09:11:58.703121|96.90|biology +101|317|65756|4294967505|4.80|41.36|true|luke laertes|2013-03-01 09:11:58.703106|11.81|industrial engineering +37|472|65545|4294967351|83.54|16.90|true|mike young|2013-03-01 09:11:58.703196|34.65|nap time +50|402|65781|4294967410|65.38|28.10|false|gabriella laertes|2013-03-01 09:11:58.703126|70.59|philosophy +105|435|65637|4294967534|80.23|13.30|true|calvin carson|2013-03-01 09:11:58.703203|29.34|forestry +33|258|65762|4294967417|15.22|13.23|false|calvin steinbeck|2013-03-01 09:11:58.703322|87.50|nap time +118|486|65651|4294967472|32.23|46.39|false|fred steinbeck|2013-03-01 09:11:58.703142|24.25|mathematics +47|306|65636|4294967524|54.10|30.78|false|ulysses xylophone|2013-03-01 09:11:58.703309|86.29|mathematics +13|469|65661|4294967404|99.92|18.34|true|irene falkner|2013-03-01 09:11:58.703244|8.06|undecided +8|267|65703|4294967473|75.29|9.81|false|xavier white|2013-03-01 09:11:58.703233|40.81|nap time +18|263|65582|4294967500|78.64|36.05|false|gabriella steinbeck|2013-03-01 09:11:58.703208|1.23|industrial engineering +122|354|65709|4294967334|74.45|44.04|false|rachel allen|2013-03-01 09:11:58.703075|5.64|yard duty +43|340|65669|4294967500|23.19|20.58|true|katie nixon|2013-03-01 09:11:58.703080|1.76|biology +87|361|65662|4294967331|97.56|44.66|false|david underhill|2013-03-01 09:11:58.703303|88.64|opthamology +36|429|65723|4294967296|67.45|19.63|false|nick quirinius|2013-03-01 09:11:58.703297|39.90|biology +83|353|65538|4294967339|0.08|23.91|true|holly thompson|2013-03-01 09:11:58.703157|30.27|quiet hour +60|421|65774|4294967400|5.62|2.43|true|quinn thompson|2013-03-01 09:11:58.703128|45.74|study skills +42|323|65629|4294967324|8.45|30.67|true|luke ichabod|2013-03-01 09:11:58.703241|19.33|undecided +103|310|65603|4294967475|44.10|16.79|false|wendy ellison|2013-03-01 09:11:58.703154|34.79|mathematics +90|414|65702|4294967337|66.36|49.79|false|gabriella ichabod|2013-03-01 09:11:58.703171|37.60|biology +28|443|65635|4294967426|97.68|31.45|false|wendy falkner|2013-03-01 09:11:58.703186|64.99|yard duty +53|467|65718|4294967355|11.15|34.81|false|luke johnson|2013-03-01 09:11:58.703252|84.35|forestry +120|294|65590|4294967443|90.07|21.77|false|jessica nixon|2013-03-01 09:11:58.703086|13.51|education +44|260|65727|4294967453|60.53|19.03|true|oscar thompson|2013-03-01 09:11:58.703150|88.11|xylophone band +28|418|65756|4294967451|27.12|34.49|false|oscar laertes|2013-03-01 09:11:58.703206|63.51|biology +5|450|65594|4294967449|96.62|41.56|true|jessica white|2013-03-01 09:11:58.703323|65.61|biology +60|400|65698|4294967299|41.34|36.56|false|oscar thompson|2013-03-01 09:11:58.703073|79.91|xylophone band +0|314|65654|4294967473|12.85|35.01|false|xavier ellison|2013-03-01 09:11:58.703189|37.74|topology +31|426|65789|4294967526|58.13|15.09|false|tom zipper|2013-03-01 09:11:58.703258|89.00|american history +95|300|65568|4294967520|39.82|47.72|true|oscar zipper|2013-03-01 09:11:58.703285|58.66|quiet hour +122|384|65578|4294967358|8.45|49.21|true|quinn steinbeck|2013-03-01 09:11:58.703148|58.78|education +86|330|65676|4294967480|88.47|17.33|false|nick falkner|2013-03-01 09:11:58.703202|58.12|linguistics +71|276|65584|4294967431|8.07|20.91|true|bob brown|2013-03-01 09:11:58.703265|6.88|joggying +54|463|65701|4294967313|43.31|23.45|false|tom laertes|2013-03-01 09:11:58.703092|36.84|geology +112|372|65722|4294967529|41.44|20.22|false|irene ichabod|2013-03-01 09:11:58.703111|38.82|linguistics +65|323|65739|4294967424|4.47|25.51|false|alice johnson|2013-03-01 09:11:58.703294|92.40|forestry +39|368|65590|4294967420|27.72|30.55|false|katie brown|2013-03-01 09:11:58.703182|42.53|zync studies +25|284|65753|4294967543|15.26|36.93|false|gabriella brown|2013-03-01 09:11:58.703107|86.97|wind surfing +55|325|65594|4294967337|7.98|2.82|true|bob polk|2013-03-01 09:11:58.703283|35.28|zync studies +63|458|65537|4294967463|21.94|49.71|true|fred van buren|2013-03-01 09:11:58.703278|99.34|wind surfing +61|300|65588|4294967412|75.73|41.75|true|zach brown|2013-03-01 09:11:58.703310|75.87|religion +47|328|65784|4294967411|94.08|2.79|false|nick johnson|2013-03-01 09:11:58.703214|31.35|linguistics +4|509|65776|4294967432|78.26|35.02|false|mike king|2013-03-01 09:11:58.703231|18.70|undecided +86|425|65626|4294967374|87.40|34.90|true|oscar underhill|2013-03-01 09:11:58.703276|63.54|education +123|499|65623|4294967521|43.37|4.98|true|jessica young|2013-03-01 09:11:58.703227|97.20|mathematics +16|380|65755|4294967349|56.68|26.55|false|priscilla carson|2013-03-01 09:11:58.703217|76.75|undecided +86|391|65620|4294967337|34.73|4.16|false|katie white|2013-03-01 09:11:58.703109|81.28|history +46|463|65622|4294967479|23.44|16.92|true|ethan polk|2013-03-01 09:11:58.703273|74.31|chemistry +79|486|65694|4294967305|10.25|9.64|false|david young|2013-03-01 09:11:58.703266|58.23|topology +57|456|65606|4294967422|72.56|30.28|true|victor johnson|2013-03-01 09:11:58.703222|55.31|kindergarten +92|306|65657|4294967337|50.26|46.88|false|wendy brown|2013-03-01 09:11:58.703124|37.39|history +36|350|65717|4294967481|92.98|11.91|true|gabriella hernandez|2013-03-01 09:11:58.703182|1.23|yard duty +60|459|65655|4294967535|36.95|41.86|true|holly johnson|2013-03-01 09:11:58.703301|77.41|linguistics +26|393|65561|4294967304|90.69|6.61|true|calvin white|2013-03-01 09:11:58.703266|47.71|geology +106|370|65625|4294967333|3.00|10.38|true|victor polk|2013-03-01 09:11:58.703112|79.65|undecided +62|427|65787|4294967323|86.93|12.64|false|irene garcia|2013-03-01 09:11:58.703215|18.78|religion +48|438|65575|4294967376|25.42|46.03|false|calvin xylophone|2013-03-01 09:11:58.703158|83.40|topology +101|327|65706|4294967340|71.54|11.48|false|gabriella ellison|2013-03-01 09:11:58.703321|3.91|nap time +9|350|65704|4294967355|21.23|12.66|true|tom miller|2013-03-01 09:11:58.703210|37.12|industrial engineering +83|336|65672|4294967347|9.71|8.05|false|priscilla quirinius|2013-03-01 09:11:58.703174|58.10|debate +108|476|65587|4294967306|2.96|34.11|false|rachel brown|2013-03-01 09:11:58.703130|5.83|quiet hour +54|341|65756|4294967522|1.08|17.63|true|david davidson|2013-03-01 09:11:58.703317|1.45|religion +48|459|65777|4294967312|76.72|2.75|false|wendy garcia|2013-03-01 09:11:58.703144|7.04|kindergarten +-3|438|65618|4294967398|62.39|4.62|false|victor xylophone|2013-03-01 09:11:58.703135|88.38|values clariffication +105|376|65602|4294967472|10.67|49.84|true|bob laertes|2013-03-01 09:11:58.703142|47.16|education +15|433|65691|4294967353|49.12|14.17|false|zach brown|2013-03-01 09:11:58.703155|6.93|kindergarten +117|276|65581|4294967457|69.80|33.64|true|gabriella van buren|2013-03-01 09:11:58.703290|56.59|industrial engineering +112|444|65721|4294967401|47.88|40.91|true|victor king|2013-03-01 09:11:58.703253|55.75|industrial engineering +71|260|65589|4294967329|2.35|6.98|true|ethan polk|2013-03-01 09:11:58.703268|94.73|geology +7|486|65773|4294967390|58.86|49.46|false|irene quirinius|2013-03-01 09:11:58.703273|30.49|quiet hour +14|474|65558|4294967538|74.62|23.78|true|quinn king|2013-03-01 09:11:58.703107|16.26|linguistics +7|343|65559|4294967371|48.01|48.80|true|zach brown|2013-03-01 09:11:58.703245|32.30|study skills +-2|304|65629|4294967386|84.38|33.49|false|zach garcia|2013-03-01 09:11:58.703264|56.99|chemistry +72|503|65727|4294967376|79.99|27.39|false|wendy laertes|2013-03-01 09:11:58.703274|88.83|zync studies +32|348|65747|4294967514|37.85|45.09|true|sarah white|2013-03-01 09:11:58.703309|11.16|mathematics +103|343|65773|4294967485|46.80|27.89|false|fred young|2013-03-01 09:11:58.703230|53.09|undecided +24|329|65561|4294967526|52.87|15.98|false|fred van buren|2013-03-01 09:11:58.703178|32.86|kindergarten +27|405|65536|4294967508|82.24|29.41|true|oscar ovid|2013-03-01 09:11:58.703166|16.85|biology +3|308|65757|4294967430|49.28|38.04|false|nick zipper|2013-03-01 09:11:58.703132|1.86|kindergarten +72|430|65763|4294967324|99.91|11.46|false|david hernandez|2013-03-01 09:11:58.703294|43.80|biology +107|271|65601|4294967456|8.57|1.30|false|xavier king|2013-03-01 09:11:58.703315|48.16|values clariffication +85|283|65746|4294967384|11.68|47.37|false|katie polk|2013-03-01 09:11:58.703311|93.21|xylophone band +49|388|65574|4294967407|27.63|17.03|false|bob xylophone|2013-03-01 09:11:58.703307|31.28|joggying +1|472|65775|4294967340|74.52|15.51|false|victor hernandez|2013-03-01 09:11:58.703077|66.68|values clariffication +89|278|65570|4294967466|55.63|24.53|true|ethan underhill|2013-03-01 09:11:58.703218|18.20|values clariffication +28|294|65756|4294967487|76.74|24.59|true|tom ellison|2013-03-01 09:11:58.703147|70.00|debate +74|504|65766|4294967520|89.93|41.54|false|alice zipper|2013-03-01 09:11:58.703134|98.90|history +42|342|65628|4294967436|9.48|24.35|false|victor davidson|2013-03-01 09:11:58.703236|37.80|religion +82|324|65789|4294967492|14.92|45.29|false|tom johnson|2013-03-01 09:11:58.703179|20.44|study skills +24|434|65788|4294967389|76.05|15.09|false|ulysses thompson|2013-03-01 09:11:58.703117|21.81|topology +33|285|65689|4294967355|97.83|41.08|true|nick steinbeck|2013-03-01 09:11:58.703178|9.12|biology +51|334|65670|4294967369|70.24|29.11|false|priscilla brown|2013-03-01 09:11:58.703269|92.97|geology +90|509|65602|4294967415|7.96|36.62|false|holly white|2013-03-01 09:11:58.703195|86.23|values clariffication +83|279|65754|4294967435|5.40|33.52|false|bob garcia|2013-03-01 09:11:58.703311|54.35|forestry +124|495|65549|4294967471|99.21|44.80|true|jessica davidson|2013-03-01 09:11:58.703130|64.91|history +120|445|65670|4294967344|98.23|47.82|false|tom ellison|2013-03-01 09:11:58.703085|5.37|education +14|273|65779|4294967474|92.05|33.36|false|calvin steinbeck|2013-03-01 09:11:58.703287|45.61|linguistics +15|351|65601|4294967459|5.67|30.66|true|victor white|2013-03-01 09:11:58.703162|37.93|kindergarten +63|422|65690|4294967550|8.42|28.42|true|oscar laertes|2013-03-01 09:11:58.703144|3.43|values clariffication +2|477|65571|4294967463|79.21|18.20|true|mike garcia|2013-03-01 09:11:58.703195|82.50|mathematics +46|386|65790|4294967507|52.73|9.27|true|priscilla ovid|2013-03-01 09:11:58.703274|84.10|undecided +28|427|65559|4294967370|39.03|42.02|true|alice carson|2013-03-01 09:11:58.703226|56.06|chemistry +101|282|65738|4294967432|93.61|4.53|true|calvin falkner|2013-03-01 09:11:58.703198|95.10|xylophone band +46|410|65721|4294967362|80.30|17.03|false|bob ellison|2013-03-01 09:11:58.703319|95.12|values clariffication +103|430|65642|4294967347|84.23|14.83|true|mike king|2013-03-01 09:11:58.703255|95.61|opthamology +51|459|65641|4294967380|5.45|19.47|true|ulysses underhill|2013-03-01 09:11:58.703289|84.27|geology +70|342|65569|4294967492|64.30|43.95|true|luke ovid|2013-03-01 09:11:58.703281|52.50|values clariffication +48|329|65788|4294967378|30.25|22.19|true|holly hernandez|2013-03-01 09:11:58.703157|16.10|quiet hour +29|374|65696|4294967481|13.87|19.93|false|xavier polk|2013-03-01 09:11:58.703258|40.30|geology +30|294|65779|4294967482|36.58|18.35|false|jessica white|2013-03-01 09:11:58.703085|39.48|kindergarten +74|289|65733|4294967495|77.84|28.96|false|jessica miller|2013-03-01 09:11:58.703177|20.72|linguistics +114|499|65724|4294967334|42.51|46.09|true|nick brown|2013-03-01 09:11:58.703176|71.66|nap time +53|371|65603|4294967303|51.29|18.41|true|nick van buren|2013-03-01 09:11:58.703186|41.44|history +121|401|65779|4294967375|29.24|25.08|true|david quirinius|2013-03-01 09:11:58.703295|65.24|religion +72|408|65581|4294967453|7.82|1.50|false|mike young|2013-03-01 09:11:58.703242|45.48|geology +116|482|65550|4294967489|5.85|31.75|true|mike steinbeck|2013-03-01 09:11:58.703150|33.01|religion +94|335|65603|4294967357|68.25|13.72|true|tom miller|2013-03-01 09:11:58.703192|80.24|undecided +4|460|65625|4294967360|5.51|22.60|true|oscar laertes|2013-03-01 09:11:58.703293|42.86|nap time +110|306|65571|4294967419|43.01|31.74|true|gabriella garcia|2013-03-01 09:11:58.703280|40.68|chemistry +0|330|65673|4294967343|91.97|15.22|false|victor brown|2013-03-01 09:11:58.703097|0.90|xylophone band +31|459|65595|4294967413|40.00|17.51|false|alice nixon|2013-03-01 09:11:58.703159|72.70|topology +45|423|65673|4294967447|11.55|3.83|true|jessica white|2013-03-01 09:11:58.703140|43.81|chemistry +55|427|65722|4294967331|10.09|45.28|true|ethan brown|2013-03-01 09:11:58.703084|1.76|joggying +9|321|65658|4294967501|54.83|20.55|false|mike davidson|2013-03-01 09:11:58.703179|60.94|debate +88|277|65585|4294967398|3.97|5.58|true|nick johnson|2013-03-01 09:11:58.703111|37.34|chemistry +110|279|65604|4294967344|92.33|5.31|true|quinn garcia|2013-03-01 09:11:58.703102|63.65|linguistics +9|456|65541|4294967531|38.57|9.56|true|xavier hernandez|2013-03-01 09:11:58.703174|36.33|linguistics +73|340|65584|4294967330|21.28|15.97|false|zach miller|2013-03-01 09:11:58.703265|12.38|philosophy +76|368|65608|4294967506|16.69|37.94|true|luke laertes|2013-03-01 09:11:58.703075|87.90|history +113|445|65658|4294967530|4.32|46.28|true|priscilla young|2013-03-01 09:11:58.703192|2.63|quiet hour +61|491|65711|4294967382|11.19|42.37|false|jessica young|2013-03-01 09:11:58.703261|8.66|wind surfing +70|361|65746|4294967332|97.57|22.61|false|katie young|2013-03-01 09:11:58.703182|98.10|undecided +38|336|65556|4294967482|14.84|12.30|true|gabriella ovid|2013-03-01 09:11:58.703086|9.96|xylophone band +100|413|65547|4294967303|53.18|20.42|true|luke allen|2013-03-01 09:11:58.703319|83.21|debate +117|474|65729|4294967333|68.22|18.36|false|priscilla underhill|2013-03-01 09:11:58.703249|19.42|education +118|467|65632|4294967296|41.36|29.36|false|tom hernandez|2013-03-01 09:11:58.703216|88.51|philosophy +50|367|65663|4294967405|11.18|33.00|true|jessica ellison|2013-03-01 09:11:58.703255|5.42|opthamology +19|276|65727|4294967539|68.81|0.37|false|holly van buren|2013-03-01 09:11:58.703084|19.81|biology +48|328|65594|4294967366|59.07|11.35|true|tom miller|2013-03-01 09:11:58.703162|67.22|nap time +5|435|65731|4294967493|73.88|30.12|false|sarah johnson|2013-03-01 09:11:58.703198|97.18|nap time +38|384|65667|4294967539|81.32|39.41|false|sarah ichabod|2013-03-01 09:11:58.703184|13.96|industrial engineering +9|316|65786|4294967546|54.47|27.40|false|ulysses hernandez|2013-03-01 09:11:58.703206|80.94|xylophone band +84|496|65604|4294967329|59.45|23.18|true|rachel king|2013-03-01 09:11:58.703077|63.36|quiet hour +34|264|65564|4294967437|31.77|12.14|false|calvin davidson|2013-03-01 09:11:58.703180|30.78|study skills +79|377|65570|4294967410|22.12|0.98|true|victor miller|2013-03-01 09:11:58.703088|45.23|joggying +7|342|65786|4294967449|72.53|21.30|false|ulysses quirinius|2013-03-01 09:11:58.703157|8.99|linguistics +85|349|65573|4294967308|97.46|12.13|false|mike steinbeck|2013-03-01 09:11:58.703179|96.32|undecided +27|287|65708|4294967542|83.33|36.73|false|quinn allen|2013-03-01 09:11:58.703157|1.29|mathematics +78|421|65633|4294967410|87.67|47.98|false|irene brown|2013-03-01 09:11:58.703207|81.02|zync studies +75|499|65696|4294967405|53.59|7.71|false|zach thompson|2013-03-01 09:11:58.703234|87.89|kindergarten +107|421|65595|4294967543|57.11|46.59|true|ethan xylophone|2013-03-01 09:11:58.703085|67.56|linguistics +124|310|65548|4294967319|58.66|15.94|false|victor thompson|2013-03-01 09:11:58.703168|52.94|forestry +5|501|65585|4294967363|33.58|19.98|true|alice xylophone|2013-03-01 09:11:58.703242|31.23|geology +65|476|65542|4294967547|41.29|23.25|true|quinn laertes|2013-03-01 09:11:58.703240|73.93|forestry +70|443|65700|4294967448|52.44|21.86|true|quinn brown|2013-03-01 09:11:58.703257|44.88|xylophone band +28|378|65678|4294967355|68.32|1.45|false|sarah xylophone|2013-03-01 09:11:58.703155|81.87|values clariffication +3|322|65672|4294967508|25.55|26.28|true|jessica carson|2013-03-01 09:11:58.703120|52.60|education +82|257|65654|4294967497|9.80|20.44|true|priscilla thompson|2013-03-01 09:11:58.703308|29.85|geology +115|276|65632|4294967457|90.69|40.89|true|tom robinson|2013-03-01 09:11:58.703157|5.09|study skills +33|459|65783|4294967300|10.29|14.51|true|fred falkner|2013-03-01 09:11:58.703270|51.08|philosophy +102|359|65560|4294967482|93.03|25.17|false|fred nixon|2013-03-01 09:11:58.703120|64.38|education +40|500|65664|4294967428|16.25|7.80|true|luke ellison|2013-03-01 09:11:58.703248|27.60|topology +32|467|65547|4294967502|67.94|21.95|true|ethan robinson|2013-03-01 09:11:58.703188|62.31|values clariffication +44|475|65776|4294967510|89.55|7.92|true|luke allen|2013-03-01 09:11:58.703105|28.47|study skills +60|381|65627|4294967365|33.85|41.36|true|quinn robinson|2013-03-01 09:11:58.703258|19.65|linguistics +103|430|65591|4294967344|75.03|22.27|false|bob ellison|2013-03-01 09:11:58.703277|9.85|values clariffication +35|423|65667|4294967507|65.43|8.76|true|zach allen|2013-03-01 09:11:58.703172|3.44|wind surfing +90|406|65763|4294967450|0.15|13.95|true|priscilla xylophone|2013-03-01 09:11:58.703245|0.72|nap time +25|403|65782|4294967516|0.61|3.40|false|wendy allen|2013-03-01 09:11:58.703152|16.51|quiet hour +104|415|65758|4294967454|66.17|44.12|false|tom robinson|2013-03-01 09:11:58.703161|25.62|kindergarten +81|272|65548|4294967413|40.04|21.30|false|tom white|2013-03-01 09:11:58.703231|77.24|study skills +37|503|65562|4294967358|44.22|44.32|false|rachel laertes|2013-03-01 09:11:58.703126|40.62|linguistics +40|432|65691|4294967547|24.86|45.46|false|ulysses laertes|2013-03-01 09:11:58.703171|58.01|topology +42|345|65676|4294967447|14.92|26.67|false|yuri thompson|2013-03-01 09:11:58.703250|80.13|history +44|465|65744|4294967391|27.30|47.71|false|xavier johnson|2013-03-01 09:11:58.703295|8.58|philosophy +14|413|65623|4294967434|87.61|20.93|false|gabriella falkner|2013-03-01 09:11:58.703318|30.83|religion +48|498|65774|4294967431|59.71|30.22|true|gabriella young|2013-03-01 09:11:58.703098|94.77|zync studies +105|326|65673|4294967440|4.59|25.92|true|victor robinson|2013-03-01 09:11:58.703290|64.23|forestry +61|407|65750|4294967412|0.10|40.59|true|zach ovid|2013-03-01 09:11:58.703137|34.29|joggying +69|473|65599|4294967325|76.72|28.69|true|zach robinson|2013-03-01 09:11:58.703305|71.51|religion +26|283|65564|4294967400|46.86|12.35|true|bob ovid|2013-03-01 09:11:58.703322|9.47|forestry +50|316|65654|4294967488|9.81|27.36|true|calvin nixon|2013-03-01 09:11:58.703133|53.55|topology +112|316|65719|4294967473|51.79|43.04|false|luke brown|2013-03-01 09:11:58.703116|82.10|history +32|264|65606|4294967487|66.17|43.92|true|holly young|2013-03-01 09:11:58.703239|24.80|zync studies +31|493|65562|4294967428|38.94|24.25|true|alice van buren|2013-03-01 09:11:58.703143|26.86|geology +32|490|65658|4294967541|62.23|30.16|false|rachel johnson|2013-03-01 09:11:58.703155|92.67|wind surfing +113|302|65721|4294967526|96.68|32.74|true|holly underhill|2013-03-01 09:11:58.703306|39.19|values clariffication +44|357|65542|4294967373|31.91|4.81|true|oscar thompson|2013-03-01 09:11:58.703182|73.72|debate +0|268|65740|4294967449|32.89|28.52|false|oscar zipper|2013-03-01 09:11:58.703113|58.65|industrial engineering +121|355|65537|4294967437|85.90|10.99|true|calvin brown|2013-03-01 09:11:58.703254|4.49|yard duty +45|459|65565|4294967507|6.55|34.52|true|gabriella davidson|2013-03-01 09:11:58.703117|81.72|kindergarten +1|428|65564|4294967532|69.32|2.43|false|holly robinson|2013-03-01 09:11:58.703324|98.36|yard duty +31|351|65669|4294967441|95.38|43.95|true|calvin zipper|2013-03-01 09:11:58.703308|16.95|xylophone band +121|330|65755|4294967380|64.36|2.53|false|holly johnson|2013-03-01 09:11:58.703210|90.08|joggying +53|464|65720|4294967413|26.39|14.52|true|nick garcia|2013-03-01 09:11:58.703315|62.04|opthamology +106|318|65789|4294967381|1.02|33.60|false|fred van buren|2013-03-01 09:11:58.703156|96.90|joggying +31|281|65681|4294967368|86.00|14.29|false|yuri falkner|2013-03-01 09:11:58.703310|55.44|xylophone band +71|356|65715|4294967364|26.47|34.14|true|wendy robinson|2013-03-01 09:11:58.703178|9.18|zync studies +97|283|65589|4294967434|37.80|31.64|true|fred white|2013-03-01 09:11:58.703188|74.83|mathematics +43|418|65646|4294967385|43.92|19.31|true|priscilla king|2013-03-01 09:11:58.703244|4.47|zync studies +46|485|65739|4294967519|9.19|1.79|true|calvin zipper|2013-03-01 09:11:58.703089|22.23|nap time +45|394|65586|4294967391|63.33|44.63|true|wendy king|2013-03-01 09:11:58.703124|56.73|education +51|362|65677|4294967445|63.42|34.20|true|ethan white|2013-03-01 09:11:58.703092|48.79|chemistry +91|384|65644|4294967533|10.09|4.95|true|victor xylophone|2013-03-01 09:11:58.703320|87.31|opthamology +77|485|65651|4294967326|50.70|49.72|false|victor quirinius|2013-03-01 09:11:58.703256|33.24|opthamology +80|333|65614|4294967331|13.10|39.55|true|oscar brown|2013-03-01 09:11:58.703154|75.45|industrial engineering +31|474|65682|4294967551|30.63|2.53|false|yuri allen|2013-03-01 09:11:58.703212|35.36|forestry +71|391|65749|4294967412|41.20|20.94|true|calvin nixon|2013-03-01 09:11:58.703311|15.14|wind surfing +0|372|65720|4294967416|79.48|20.76|true|oscar quirinius|2013-03-01 09:11:58.703254|0.32|education +41|453|65677|4294967371|76.10|28.02|false|victor xylophone|2013-03-01 09:11:58.703179|69.86|religion +111|317|65703|4294967297|38.05|0.48|true|fred nixon|2013-03-01 09:11:58.703256|90.89|values clariffication +18|468|65551|4294967546|0.98|45.14|false|irene polk|2013-03-01 09:11:58.703186|39.43|wind surfing +0|486|65706|4294967466|20.82|30.55|false|mike allen|2013-03-01 09:11:58.703246|72.87|mathematics +92|329|65610|4294967521|47.08|36.57|false|irene polk|2013-03-01 09:11:58.703231|11.99|debate +35|409|65777|4294967422|93.09|11.50|false|bob brown|2013-03-01 09:11:58.703269|73.79|religion +80|455|65644|4294967460|75.35|20.56|false|nick xylophone|2013-03-01 09:11:58.703249|30.25|quiet hour +17|435|65684|4294967326|31.61|8.33|false|victor allen|2013-03-01 09:11:58.703264|77.18|mathematics +91|295|65587|4294967506|76.93|9.88|true|xavier polk|2013-03-01 09:11:58.703243|53.84|linguistics +92|363|65697|4294967310|13.89|47.40|true|katie robinson|2013-03-01 09:11:58.703237|3.18|xylophone band +17|351|65589|4294967439|54.99|44.75|false|irene van buren|2013-03-01 09:11:58.703196|49.49|american history +88|445|65778|4294967422|33.67|28.86|false|bob van buren|2013-03-01 09:11:58.703230|62.17|forestry +44|386|65692|4294967434|0.84|20.94|true|wendy polk|2013-03-01 09:11:58.703226|87.05|values clariffication +84|373|65585|4294967486|14.13|30.21|true|katie xylophone|2013-03-01 09:11:58.703084|50.38|forestry +83|395|65645|4294967509|99.62|9.50|true|irene ichabod|2013-03-01 09:11:58.703212|63.22|zync studies +28|428|65650|4294967320|85.76|22.68|false|wendy thompson|2013-03-01 09:11:58.703145|85.89|wind surfing +27|503|65628|4294967371|95.07|25.58|true|ethan laertes|2013-03-01 09:11:58.703310|14.83|topology +69|304|65695|4294967384|13.94|10.51|true|nick garcia|2013-03-01 09:11:58.703167|59.96|mathematics +15|264|65737|4294967412|9.04|32.04|false|alice ovid|2013-03-01 09:11:58.703279|20.85|kindergarten +118|399|65730|4294967309|0.52|33.72|false|luke laertes|2013-03-01 09:11:58.703098|1.35|quiet hour +84|437|65601|4294967357|49.44|8.79|false|nick davidson|2013-03-01 09:11:58.703161|26.54|undecided +1|313|65543|4294967362|45.35|47.49|true|bob white|2013-03-01 09:11:58.703144|21.59|linguistics +40|442|65659|4294967379|50.08|31.50|false|irene ellison|2013-03-01 09:11:58.703306|53.80|philosophy +60|308|65564|4294967405|36.79|6.46|true|quinn ichabod|2013-03-01 09:11:58.703298|26.06|geology +118|282|65644|4294967407|67.59|32.89|true|victor laertes|2013-03-01 09:11:58.703177|89.95|study skills +77|320|65585|4294967344|45.59|39.11|true|bob garcia|2013-03-01 09:11:58.703301|39.32|american history +17|407|65716|4294967460|14.44|16.87|false|luke johnson|2013-03-01 09:11:58.703200|71.53|linguistics +84|492|65737|4294967467|23.45|46.69|false|nick ichabod|2013-03-01 09:11:58.703127|3.98|values clariffication +97|266|65544|4294967373|32.20|9.87|true|zach garcia|2013-03-01 09:11:58.703289|1.67|quiet hour +21|348|65765|4294967366|56.62|2.75|false|nick zipper|2013-03-01 09:11:58.703269|69.76|zync studies +88|404|65548|4294967382|6.87|15.00|false|oscar carson|2013-03-01 09:11:58.703318|57.61|wind surfing +119|427|65553|4294967319|95.53|12.82|true|luke underhill|2013-03-01 09:11:58.703130|93.95|wind surfing +16|405|65591|4294967351|97.24|33.01|false|ethan quirinius|2013-03-01 09:11:58.703156|60.78|xylophone band +65|460|65570|4294967399|74.30|19.59|false|jessica white|2013-03-01 09:11:58.703111|25.57|study skills +11|300|65583|4294967382|88.22|30.13|false|tom falkner|2013-03-01 09:11:58.703275|93.49|joggying +106|446|65613|4294967369|48.45|7.18|true|alice garcia|2013-03-01 09:11:58.703193|43.72|chemistry +114|410|65578|4294967363|94.34|15.10|true|zach ovid|2013-03-01 09:11:58.703299|55.97|history +-1|372|65680|4294967490|15.45|18.09|false|ethan laertes|2013-03-01 09:11:58.703110|65.88|opthamology +113|361|65660|4294967516|23.17|38.39|true|alice king|2013-03-01 09:11:58.703246|62.82|geology +98|381|65563|4294967458|41.29|48.85|false|sarah garcia|2013-03-01 09:11:58.703243|33.29|mathematics +108|385|65730|4294967387|65.44|43.34|false|irene miller|2013-03-01 09:11:58.703157|81.44|philosophy +10|441|65723|4294967528|96.23|34.98|false|xavier brown|2013-03-01 09:11:58.703281|19.95|nap time +84|403|65656|4294967550|7.05|33.66|true|luke davidson|2013-03-01 09:11:58.703300|14.00|kindergarten +84|447|65755|4294967387|4.92|22.75|false|rachel davidson|2013-03-01 09:11:58.703178|96.80|biology +25|309|65568|4294967354|58.75|23.31|true|katie zipper|2013-03-01 09:11:58.703156|21.79|quiet hour +17|388|65784|4294967542|40.24|3.98|true|katie polk|2013-03-01 09:11:58.703163|15.70|kindergarten +5|486|65602|4294967358|71.55|3.40|true|ulysses carson|2013-03-01 09:11:58.703266|83.67|mathematics +59|494|65786|4294967483|35.80|11.44|true|zach garcia|2013-03-01 09:11:58.703125|56.82|yard duty +80|509|65553|4294967544|50.28|47.00|false|calvin white|2013-03-01 09:11:58.703071|71.07|debate +109|376|65538|4294967453|59.61|35.62|true|priscilla xylophone|2013-03-01 09:11:58.703286|9.53|study skills +40|496|65640|4294967513|93.73|25.71|true|calvin thompson|2013-03-01 09:11:58.703124|8.93|mathematics +65|508|65732|4294967409|81.47|29.25|false|ethan ellison|2013-03-01 09:11:58.703154|30.06|religion +50|478|65599|4294967338|77.81|32.31|true|holly brown|2013-03-01 09:11:58.703128|76.29|kindergarten +26|451|65716|4294967312|77.42|13.60|true|ulysses carson|2013-03-01 09:11:58.703116|38.93|history +31|382|65737|4294967301|58.09|41.34|true|wendy thompson|2013-03-01 09:11:58.703231|80.97|biology +33|430|65694|4294967346|48.37|40.46|false|fred king|2013-03-01 09:11:58.703167|88.50|forestry +6|408|65658|4294967503|99.68|44.11|false|mike polk|2013-03-01 09:11:58.703313|20.69|industrial engineering +100|264|65637|4294967340|85.00|45.86|false|fred falkner|2013-03-01 09:11:58.703148|44.32|joggying +119|316|65646|4294967405|46.27|16.82|false|gabriella allen|2013-03-01 09:11:58.703224|17.92|wind surfing +105|478|65790|4294967302|88.05|5.82|true|gabriella polk|2013-03-01 09:11:58.703236|84.87|topology +124|384|65697|4294967426|15.30|19.15|true|fred quirinius|2013-03-01 09:11:58.703139|86.79|education +109|319|65734|4294967387|52.23|20.82|false|alice king|2013-03-01 09:11:58.703262|31.71|american history +93|356|65650|4294967428|53.94|27.46|true|irene brown|2013-03-01 09:11:58.703130|59.55|religion +51|444|65662|4294967539|42.48|14.39|false|alice zipper|2013-03-01 09:11:58.703202|76.51|geology +61|432|65778|4294967469|9.57|29.88|false|mike white|2013-03-01 09:11:58.703074|64.69|wind surfing +78|419|65633|4294967299|34.35|32.06|true|bob zipper|2013-03-01 09:11:58.703081|72.54|history +67|295|65654|4294967449|59.55|1.37|true|ulysses white|2013-03-01 09:11:58.703096|11.64|topology +70|312|65742|4294967540|54.43|19.85|false|tom polk|2013-03-01 09:11:58.703206|6.61|values clariffication +56|351|65732|4294967479|8.37|34.68|true|xavier white|2013-03-01 09:11:58.703103|95.68|quiet hour +42|495|65536|4294967431|43.57|46.81|false|tom johnson|2013-03-01 09:11:58.703245|62.25|american history +0|428|65590|4294967473|94.54|28.29|true|irene carson|2013-03-01 09:11:58.703089|92.71|education +91|262|65661|4294967391|73.65|13.30|false|sarah garcia|2013-03-01 09:11:58.703325|26.80|topology +73|337|65775|4294967406|57.12|24.35|true|oscar xylophone|2013-03-01 09:11:58.703091|66.56|topology +58|404|65693|4294967345|78.89|6.57|true|sarah carson|2013-03-01 09:11:58.703218|84.12|undecided +77|461|65581|4294967449|89.55|49.44|true|jessica thompson|2013-03-01 09:11:58.703216|60.88|quiet hour +89|492|65736|4294967336|14.93|26.74|false|ulysses young|2013-03-01 09:11:58.703089|69.96|undecided +116|444|65659|4294967329|59.62|37.37|true|victor hernandez|2013-03-01 09:11:58.703093|94.69|kindergarten +114|427|65712|4294967419|32.47|10.24|true|jessica quirinius|2013-03-01 09:11:58.703116|49.69|values clariffication +43|503|65650|4294967482|96.38|6.96|true|nick nixon|2013-03-01 09:11:58.703302|23.55|american history +116|351|65771|4294967470|48.59|24.17|false|fred ellison|2013-03-01 09:11:58.703282|7.50|values clariffication +-2|261|65603|4294967332|39.18|28.98|false|fred polk|2013-03-01 09:11:58.703310|69.26|opthamology +123|439|65781|4294967448|15.86|42.96|true|ulysses xylophone|2013-03-01 09:11:58.703272|12.45|geology +83|451|65583|4294967443|10.13|44.36|true|nick falkner|2013-03-01 09:11:58.703293|57.67|values clariffication +51|390|65709|4294967548|29.59|49.59|true|oscar steinbeck|2013-03-01 09:11:58.703243|49.79|education +107|313|65699|4294967493|9.25|41.34|false|gabriella young|2013-03-01 09:11:58.703213|38.71|mathematics +93|403|65782|4294967355|74.53|26.21|true|oscar robinson|2013-03-01 09:11:58.703239|76.14|topology +7|320|65624|4294967530|41.34|11.54|true|bob ellison|2013-03-01 09:11:58.703249|93.30|nap time +38|308|65559|4294967465|95.81|13.71|true|david davidson|2013-03-01 09:11:58.703114|73.94|forestry +112|295|65662|4294967517|65.62|34.97|true|holly johnson|2013-03-01 09:11:58.703274|59.92|zync studies +101|283|65538|4294967527|84.03|27.29|true|yuri brown|2013-03-01 09:11:58.703306|44.24|biology +20|509|65686|4294967330|12.46|40.63|false|victor steinbeck|2013-03-01 09:11:58.703323|38.27|linguistics +0|367|65685|4294967491|91.61|13.80|false|priscilla van buren|2013-03-01 09:11:58.703096|87.84|topology +18|371|65552|4294967424|37.24|3.71|false|fred davidson|2013-03-01 09:11:58.703308|48.96|undecided +92|344|65618|4294967390|46.09|28.69|true|victor steinbeck|2013-03-01 09:11:58.703193|81.99|biology +34|322|65659|4294967353|78.62|21.42|false|ethan robinson|2013-03-01 09:11:58.703153|46.57|religion +27|401|65779|4294967402|97.39|43.42|false|mike zipper|2013-03-01 09:11:58.703145|64.41|quiet hour +74|290|65630|4294967436|57.89|8.33|false|oscar ellison|2013-03-01 09:11:58.703296|99.05|biology +81|344|65677|4294967530|28.11|20.56|false|calvin brown|2013-03-01 09:11:58.703159|87.67|quiet hour +109|275|65651|4294967393|96.94|17.44|false|luke underhill|2013-03-01 09:11:58.703228|58.25|opthamology +97|290|65774|4294967311|71.07|11.44|false|wendy robinson|2013-03-01 09:11:58.703240|5.10|undecided +41|338|65618|4294967304|45.68|26.17|true|xavier allen|2013-03-01 09:11:58.703138|55.68|biology +88|496|65767|4294967448|14.30|41.69|true|wendy quirinius|2013-03-01 09:11:58.703106|94.32|kindergarten +51|495|65554|4294967347|94.27|26.65|true|irene robinson|2013-03-01 09:11:58.703179|63.61|quiet hour +97|491|65618|4294967423|32.25|2.13|false|luke falkner|2013-03-01 09:11:58.703166|16.60|study skills +9|311|65719|4294967442|2.60|26.73|true|zach miller|2013-03-01 09:11:58.703223|64.21|undecided +62|262|65668|4294967468|61.94|7.31|false|priscilla johnson|2013-03-01 09:11:58.703228|48.68|topology +63|341|65704|4294967546|7.79|2.63|false|xavier brown|2013-03-01 09:11:58.703261|69.36|debate +69|492|65751|4294967324|45.10|20.65|true|sarah johnson|2013-03-01 09:11:58.703095|9.77|chemistry +17|404|65712|4294967388|71.13|45.81|false|gabriella ichabod|2013-03-01 09:11:58.703164|30.27|undecided +72|310|65780|4294967507|80.99|24.03|true|tom davidson|2013-03-01 09:11:58.703212|38.58|forestry +18|397|65611|4294967439|98.22|15.32|true|xavier allen|2013-03-01 09:11:58.703089|69.46|undecided +50|307|65596|4294967300|80.60|30.89|true|calvin falkner|2013-03-01 09:11:58.703275|54.75|values clariffication +84|305|65637|4294967402|10.16|43.04|true|yuri laertes|2013-03-01 09:11:58.703283|58.05|linguistics +75|479|65749|4294967528|80.52|5.31|false|priscilla brown|2013-03-01 09:11:58.703148|65.49|undecided +97|323|65704|4294967477|60.12|42.22|true|mike nixon|2013-03-01 09:11:58.703249|48.48|wind surfing +14|437|65612|4294967539|64.25|4.82|false|zach ichabod|2013-03-01 09:11:58.703215|51.14|forestry +22|280|65774|4294967311|77.10|40.24|false|jessica nixon|2013-03-01 09:11:58.703238|56.84|xylophone band +118|497|65536|4294967381|50.32|12.72|false|david nixon|2013-03-01 09:11:58.703285|83.48|values clariffication +27|482|65624|4294967313|78.98|43.96|true|oscar carson|2013-03-01 09:11:58.703217|70.08|joggying +99|436|65746|4294967342|39.81|15.28|false|calvin young|2013-03-01 09:11:58.703145|97.52|quiet hour +50|409|65738|4294967535|28.56|0.74|true|mike ellison|2013-03-01 09:11:58.703282|30.94|values clariffication +66|267|65738|4294967371|70.89|11.40|true|oscar thompson|2013-03-01 09:11:58.703104|51.33|undecided +107|466|65691|4294967302|9.68|40.39|false|nick ellison|2013-03-01 09:11:58.703108|28.47|american history +55|335|65699|4294967487|5.28|16.66|true|david ichabod|2013-03-01 09:11:58.703161|68.94|nap time +-2|340|65627|4294967533|76.52|21.91|true|quinn laertes|2013-03-01 09:11:58.703178|93.29|xylophone band +68|448|65597|4294967422|70.39|40.01|true|ethan laertes|2013-03-01 09:11:58.703109|69.05|quiet hour +65|505|65679|4294967338|75.88|15.87|true|yuri steinbeck|2013-03-01 09:11:58.703085|65.46|yard duty +53|315|65607|4294967496|1.58|20.02|true|victor johnson|2013-03-01 09:11:58.703202|36.58|undecided +94|475|65606|4294967503|62.14|26.00|false|mike ellison|2013-03-01 09:11:58.703115|67.94|biology +-3|275|65622|4294967302|71.78|8.49|false|wendy robinson|2013-03-01 09:11:58.703294|95.39|undecided +-1|281|65643|4294967323|15.10|45.00|false|irene nixon|2013-03-01 09:11:58.703223|80.96|undecided +72|469|65765|4294967500|60.22|32.25|false|holly young|2013-03-01 09:11:58.703253|88.56|wind surfing +23|401|65693|4294967439|34.41|46.73|false|rachel brown|2013-03-01 09:11:58.703306|71.72|geology +36|377|65697|4294967444|98.51|49.05|false|oscar carson|2013-03-01 09:11:58.703081|67.90|joggying +18|264|65595|4294967512|18.86|42.56|false|fred davidson|2013-03-01 09:11:58.703137|89.60|history +82|257|65655|4294967320|40.42|41.02|true|luke quirinius|2013-03-01 09:11:58.703205|35.80|zync studies +120|497|65595|4294967442|45.56|21.53|true|sarah young|2013-03-01 09:11:58.703167|90.24|quiet hour +94|256|65747|4294967347|32.18|8.61|true|quinn quirinius|2013-03-01 09:11:58.703217|71.87|history +112|453|65586|4294967540|42.76|3.66|true|gabriella carson|2013-03-01 09:11:58.703291|1.15|wind surfing +73|394|65657|4294967486|11.82|39.80|false|luke laertes|2013-03-01 09:11:58.703160|11.93|nap time +-3|344|65733|4294967363|0.56|11.96|true|rachel thompson|2013-03-01 09:11:58.703276|88.46|wind surfing +0|356|65581|4294967326|40.44|33.18|false|calvin robinson|2013-03-01 09:11:58.703294|29.74|zync studies +101|487|65676|4294967367|85.87|13.14|true|zach zipper|2013-03-01 09:11:58.703077|27.58|zync studies +48|491|65669|4294967492|63.51|49.85|true|tom van buren|2013-03-01 09:11:58.703247|1.29|forestry +93|504|65557|4294967500|22.68|37.23|true|bob hernandez|2013-03-01 09:11:58.703207|21.14|zync studies +35|290|65726|4294967434|18.63|7.24|true|priscilla zipper|2013-03-01 09:11:58.703195|7.42|education +114|503|65677|4294967534|64.22|15.16|true|gabriella allen|2013-03-01 09:11:58.703324|44.47|wind surfing +-3|458|65696|4294967418|45.24|8.49|false|irene ellison|2013-03-01 09:11:58.703092|54.02|american history +103|411|65579|4294967521|22.25|18.31|true|quinn zipper|2013-03-01 09:11:58.703175|33.37|forestry +100|492|65778|4294967379|13.35|23.88|false|luke garcia|2013-03-01 09:11:58.703260|59.03|undecided +119|421|65580|4294967480|27.87|46.74|true|wendy brown|2013-03-01 09:11:58.703085|59.43|study skills +20|385|65562|4294967324|71.80|5.51|false|oscar ichabod|2013-03-01 09:11:58.703194|72.04|mathematics +81|331|65745|4294967531|80.71|42.55|true|ethan laertes|2013-03-01 09:11:58.703226|66.36|yard duty +78|509|65758|4294967531|21.45|23.59|false|alice allen|2013-03-01 09:11:58.703171|56.56|geology +17|496|65624|4294967389|96.25|1.62|true|nick laertes|2013-03-01 09:11:58.703113|65.31|mathematics +43|379|65616|4294967348|46.15|12.02|true|rachel falkner|2013-03-01 09:11:58.703120|75.20|geology +43|281|65676|4294967540|87.14|21.33|true|jessica garcia|2013-03-01 09:11:58.703173|8.77|debate +22|304|65607|4294967318|82.72|2.34|false|priscilla van buren|2013-03-01 09:11:58.703185|75.86|forestry +59|431|65537|4294967326|11.34|2.90|true|oscar robinson|2013-03-01 09:11:58.703161|81.04|zync studies +6|322|65715|4294967385|82.55|8.62|false|david ichabod|2013-03-01 09:11:58.703248|81.28|topology +79|396|65686|4294967335|89.28|25.63|true|rachel polk|2013-03-01 09:11:58.703277|58.87|geology +95|482|65637|4294967342|9.70|2.16|true|bob steinbeck|2013-03-01 09:11:58.703225|35.51|undecided +71|280|65610|4294967451|33.36|25.95|true|rachel brown|2013-03-01 09:11:58.703097|7.59|opthamology +32|400|65753|4294967360|35.13|32.17|true|irene ovid|2013-03-01 09:11:58.703294|59.13|mathematics +70|373|65773|4294967502|46.18|47.06|false|zach king|2013-03-01 09:11:58.703132|87.66|mathematics +105|509|65716|4294967308|22.94|9.61|true|jessica quirinius|2013-03-01 09:11:58.703073|10.07|zync studies +20|286|65671|4294967307|28.45|25.01|true|oscar white|2013-03-01 09:11:58.703299|52.05|undecided +98|432|65669|4294967373|63.12|23.03|false|calvin allen|2013-03-01 09:11:58.703214|38.60|industrial engineering +57|304|65731|4294967315|94.68|44.22|false|xavier carson|2013-03-01 09:11:58.703208|18.20|study skills +45|433|65672|4294967506|85.03|0.37|false|tom nixon|2013-03-01 09:11:58.703154|64.09|mathematics +77|483|65784|4294967330|23.07|29.96|true|zach ovid|2013-03-01 09:11:58.703168|97.37|education +70|268|65766|4294967299|88.36|17.15|true|gabriella thompson|2013-03-01 09:11:58.703169|86.53|opthamology +34|398|65731|4294967546|66.61|46.93|true|calvin johnson|2013-03-01 09:11:58.703078|61.52|quiet hour +2|443|65552|4294967349|51.85|30.49|false|oscar white|2013-03-01 09:11:58.703125|78.52|philosophy +65|393|65625|4294967374|24.73|41.75|false|katie garcia|2013-03-01 09:11:58.703083|20.61|opthamology +83|432|65656|4294967303|26.08|39.73|true|jessica underhill|2013-03-01 09:11:58.703147|82.67|geology +32|506|65551|4294967375|19.13|3.29|true|ulysses ichabod|2013-03-01 09:11:58.703195|74.13|zync studies +13|324|65544|4294967539|84.31|37.34|true|tom young|2013-03-01 09:11:58.703271|65.85|education +11|267|65727|4294967325|33.55|17.89|true|jessica davidson|2013-03-01 09:11:58.703144|87.37|kindergarten +119|280|65615|4294967457|94.72|19.90|false|rachel white|2013-03-01 09:11:58.703163|54.47|biology +85|392|65760|4294967436|65.02|16.91|false|bob ellison|2013-03-01 09:11:58.703115|27.52|history +89|436|65623|4294967461|89.03|25.97|true|fred robinson|2013-03-01 09:11:58.703239|95.58|linguistics +51|265|65626|4294967521|94.38|1.53|true|luke thompson|2013-03-01 09:11:58.703120|75.15|yard duty +122|428|65715|4294967520|35.72|8.21|false|priscilla underhill|2013-03-01 09:11:58.703212|54.79|opthamology +114|507|65576|4294967517|15.18|34.48|true|luke allen|2013-03-01 09:11:58.703140|4.88|religion +22|257|65588|4294967316|81.17|28.77|false|nick quirinius|2013-03-01 09:11:58.703224|76.44|forestry +78|417|65552|4294967300|64.00|10.49|true|calvin van buren|2013-03-01 09:11:58.703223|3.43|debate +88|409|65607|4294967509|99.13|41.24|false|holly zipper|2013-03-01 09:11:58.703267|91.52|nap time +36|274|65705|4294967514|70.52|5.93|true|fred nixon|2013-03-01 09:11:58.703179|65.13|topology +42|338|65719|4294967488|87.99|39.29|false|nick ovid|2013-03-01 09:11:58.703302|66.85|education +18|446|65544|4294967435|93.11|14.90|true|holly xylophone|2013-03-01 09:11:58.703085|73.08|history +110|261|65600|4294967428|75.83|10.22|false|mike garcia|2013-03-01 09:11:58.703132|43.03|education +61|382|65623|4294967443|25.11|2.26|false|david ovid|2013-03-01 09:11:58.703125|39.74|education +57|277|65754|4294967357|36.26|32.98|true|gabriella zipper|2013-03-01 09:11:58.703077|10.16|topology +12|379|65698|4294967477|47.57|44.60|false|mike carson|2013-03-01 09:11:58.703291|42.98|religion +3|464|65617|4294967424|82.30|2.92|false|ethan brown|2013-03-01 09:11:58.703076|18.51|wind surfing +5|360|65543|4294967305|67.98|38.53|false|priscilla johnson|2013-03-01 09:11:58.703259|52.28|geology +37|445|65612|4294967444|92.11|0.74|true|wendy steinbeck|2013-03-01 09:11:58.703096|16.05|study skills +58|382|65644|4294967500|18.63|30.09|true|oscar allen|2013-03-01 09:11:58.703144|96.45|linguistics +115|327|65607|4294967391|28.79|0.80|false|yuri polk|2013-03-01 09:11:58.703153|47.12|study skills +93|373|65707|4294967343|95.48|11.83|false|oscar hernandez|2013-03-01 09:11:58.703245|76.20|undecided +108|478|65712|4294967522|46.43|33.12|false|nick garcia|2013-03-01 09:11:58.703166|34.70|kindergarten +-3|376|65548|4294967431|96.78|43.23|false|fred ellison|2013-03-01 09:11:58.703233|75.39|education +44|398|65691|4294967393|26.71|25.51|false|fred ellison|2013-03-01 09:11:58.703270|5.01|undecided +91|392|65708|4294967305|71.35|26.84|false|victor hernandez|2013-03-01 09:11:58.703198|70.89|philosophy +14|355|65556|4294967482|64.46|5.96|true|oscar davidson|2013-03-01 09:11:58.703239|95.12|linguistics +89|278|65593|4294967432|8.61|16.38|false|tom brown|2013-03-01 09:11:58.703266|84.60|quiet hour +11|425|65624|4294967305|16.08|41.83|true|tom carson|2013-03-01 09:11:58.703324|14.08|forestry +61|478|65575|4294967442|96.73|43.85|true|ulysses ellison|2013-03-01 09:11:58.703218|61.41|history +54|501|65722|4294967336|42.67|30.41|true|irene laertes|2013-03-01 09:11:58.703153|4.11|undecided +58|290|65596|4294967532|41.62|31.16|false|oscar nixon|2013-03-01 09:11:58.703123|94.35|chemistry +17|350|65571|4294967428|19.03|36.22|false|victor hernandez|2013-03-01 09:11:58.703295|71.57|debate +49|291|65548|4294967403|77.02|46.73|true|victor white|2013-03-01 09:11:58.703221|51.90|zync studies +17|495|65689|4294967373|41.45|42.96|true|oscar quirinius|2013-03-01 09:11:58.703197|27.51|chemistry +1|418|65670|4294967534|6.29|30.62|true|yuri carson|2013-03-01 09:11:58.703089|93.66|linguistics +49|345|65614|4294967318|50.02|2.89|false|ethan falkner|2013-03-01 09:11:58.703255|94.47|linguistics +123|419|65654|4294967442|97.87|40.84|true|luke ichabod|2013-03-01 09:11:58.703159|88.23|linguistics +66|359|65556|4294967532|86.93|23.63|true|zach king|2013-03-01 09:11:58.703181|99.94|study skills +50|290|65630|4294967304|10.20|21.01|false|oscar ellison|2013-03-01 09:11:58.703139|72.13|opthamology +118|334|65586|4294967525|38.79|31.63|true|mike king|2013-03-01 09:11:58.703307|87.57|philosophy +60|454|65706|4294967443|47.32|3.27|true|fred polk|2013-03-01 09:11:58.703278|9.74|joggying +25|291|65746|4294967407|60.26|42.76|false|wendy nixon|2013-03-01 09:11:58.703137|34.75|american history +108|391|65717|4294967336|43.71|41.73|false|rachel white|2013-03-01 09:11:58.703272|33.13|wind surfing +0|267|65608|4294967523|46.87|4.48|true|jessica quirinius|2013-03-01 09:11:58.703278|69.42|geology +46|360|65702|4294967541|46.21|20.42|false|jessica underhill|2013-03-01 09:11:58.703117|55.68|linguistics +88|404|65730|4294967335|17.74|49.67|false|katie van buren|2013-03-01 09:11:58.703198|30.60|biology +41|275|65694|4294967503|1.91|44.92|true|sarah carson|2013-03-01 09:11:58.703096|65.86|debate +93|426|65714|4294967478|83.40|20.19|false|quinn davidson|2013-03-01 09:11:58.703093|14.85|debate +78|448|65568|4294967513|74.02|10.41|true|quinn garcia|2013-03-01 09:11:58.703107|16.26|quiet hour +77|283|65711|4294967523|10.17|28.61|false|gabriella falkner|2013-03-01 09:11:58.703106|96.10|topology +45|377|65786|4294967364|92.37|38.14|false|alice steinbeck|2013-03-01 09:11:58.703299|29.32|wind surfing +117|341|65588|4294967496|51.29|28.71|true|jessica brown|2013-03-01 09:11:58.703262|98.33|philosophy +5|485|65635|4294967306|2.20|40.21|false|yuri nixon|2013-03-01 09:11:58.703256|43.80|history +96|315|65769|4294967323|94.68|10.21|true|mike king|2013-03-01 09:11:58.703109|38.98|linguistics +5|295|65621|4294967329|3.28|6.58|true|tom van buren|2013-03-01 09:11:58.703244|25.67|joggying +79|340|65744|4294967531|79.49|19.72|false|ulysses robinson|2013-03-01 09:11:58.703197|46.98|wind surfing +18|312|65648|4294967547|72.04|9.99|true|fred falkner|2013-03-01 09:11:58.703159|64.06|industrial engineering +17|382|65769|4294967351|57.64|38.92|false|fred laertes|2013-03-01 09:11:58.703242|91.49|biology +9|293|65545|4294967527|31.67|43.30|false|luke johnson|2013-03-01 09:11:58.703325|65.81|joggying +39|340|65650|4294967365|22.12|22.27|false|xavier quirinius|2013-03-01 09:11:58.703082|37.25|nap time +5|292|65598|4294967394|79.38|39.63|true|mike ellison|2013-03-01 09:11:58.703315|53.04|mathematics +71|298|65540|4294967510|91.63|9.60|false|gabriella zipper|2013-03-01 09:11:58.703140|68.14|geology +46|303|65562|4294967402|96.29|28.91|false|ethan laertes|2013-03-01 09:11:58.703147|54.94|debate +99|282|65659|4294967394|30.62|34.72|true|wendy garcia|2013-03-01 09:11:58.703283|94.30|mathematics +67|265|65562|4294967421|53.06|4.92|false|jessica xylophone|2013-03-01 09:11:58.703182|69.41|quiet hour +53|274|65658|4294967545|95.28|48.78|false|luke polk|2013-03-01 09:11:58.703095|77.66|topology +71|496|65643|4294967349|84.72|14.22|true|calvin ovid|2013-03-01 09:11:58.703252|50.79|chemistry +19|281|65542|4294967452|29.41|16.87|false|zach xylophone|2013-03-01 09:11:58.703318|78.01|quiet hour +44|330|65647|4294967339|39.34|33.30|true|katie king|2013-03-01 09:11:58.703224|58.03|education +109|283|65675|4294967429|88.91|28.77|true|ulysses white|2013-03-01 09:11:58.703244|35.33|yard duty +59|373|65589|4294967359|9.35|22.19|true|luke falkner|2013-03-01 09:11:58.703112|72.30|xylophone band +80|407|65588|4294967371|51.25|38.62|false|david allen|2013-03-01 09:11:58.703215|64.63|kindergarten +31|457|65656|4294967394|29.36|31.68|true|ulysses ovid|2013-03-01 09:11:58.703192|64.44|nap time +33|319|65617|4294967431|64.65|15.09|true|tom laertes|2013-03-01 09:11:58.703269|47.45|kindergarten +41|334|65776|4294967432|71.89|47.91|false|ulysses johnson|2013-03-01 09:11:58.703272|61.41|american history +97|349|65769|4294967386|12.90|42.00|false|irene quirinius|2013-03-01 09:11:58.703239|58.05|xylophone band +40|331|65705|4294967470|91.78|31.30|false|oscar van buren|2013-03-01 09:11:58.703250|25.89|zync studies +119|310|65679|4294967319|14.21|10.38|true|sarah carson|2013-03-01 09:11:58.703221|64.15|industrial engineering +38|493|65788|4294967427|60.85|47.66|true|ethan white|2013-03-01 09:11:58.703173|25.77|zync studies +70|285|65718|4294967353|37.10|35.85|false|mike ellison|2013-03-01 09:11:58.703167|89.67|kindergarten +93|403|65752|4294967497|97.65|47.03|false|luke miller|2013-03-01 09:11:58.703133|28.92|biology +48|501|65720|4294967525|2.79|20.39|true|alice allen|2013-03-01 09:11:58.703093|22.34|wind surfing +88|307|65747|4294967389|31.86|20.97|true|jessica carson|2013-03-01 09:11:58.703194|15.07|joggying +66|259|65767|4294967378|51.72|5.11|true|gabriella falkner|2013-03-01 09:11:58.703119|90.88|xylophone band +118|292|65637|4294967403|49.68|11.16|false|jessica polk|2013-03-01 09:11:58.703262|48.88|education +49|448|65742|4294967470|82.34|30.04|true|yuri polk|2013-03-01 09:11:58.703171|24.94|forestry +93|326|65678|4294967428|45.19|1.71|true|david white|2013-03-01 09:11:58.703101|8.72|industrial engineering +74|298|65773|4294967400|21.80|28.85|false|oscar xylophone|2013-03-01 09:11:58.703124|47.09|religion +96|263|65769|4294967532|53.02|4.04|false|calvin quirinius|2013-03-01 09:11:58.703128|11.45|forestry +11|382|65779|4294967345|62.72|40.27|false|david davidson|2013-03-01 09:11:58.703176|28.20|geology +123|284|65783|4294967301|60.26|37.08|true|victor davidson|2013-03-01 09:11:58.703227|17.65|forestry +58|478|65684|4294967439|95.53|14.16|true|ulysses van buren|2013-03-01 09:11:58.703076|67.34|values clariffication +27|256|65599|4294967383|89.55|16.61|true|xavier quirinius|2013-03-01 09:11:58.703304|41.61|linguistics +66|372|65718|4294967401|83.87|28.64|false|yuri underhill|2013-03-01 09:11:58.703117|33.35|education +82|337|65662|4294967453|11.57|45.53|false|wendy underhill|2013-03-01 09:11:58.703299|27.89|wind surfing +94|485|65735|4294967472|69.88|14.92|false|victor hernandez|2013-03-01 09:11:58.703193|28.42|topology +101|259|65757|4294967550|82.97|42.30|false|nick miller|2013-03-01 09:11:58.703190|90.09|study skills +48|443|65675|4294967322|57.67|9.40|true|nick robinson|2013-03-01 09:11:58.703228|4.19|biology +60|309|65743|4294967450|15.90|7.53|true|xavier laertes|2013-03-01 09:11:58.703289|38.73|wind surfing +16|323|65748|4294967471|6.84|17.55|true|zach ellison|2013-03-01 09:11:58.703164|77.52|religion +59|484|65743|4294967483|66.67|8.09|true|victor king|2013-03-01 09:11:58.703297|52.65|debate +2|507|65764|4294967544|72.51|30.90|false|katie young|2013-03-01 09:11:58.703223|57.10|opthamology +101|315|65749|4294967519|85.14|5.23|true|mike steinbeck|2013-03-01 09:11:58.703131|73.24|topology +38|290|65628|4294967473|56.07|37.96|false|wendy allen|2013-03-01 09:11:58.703322|2.48|philosophy +117|400|65755|4294967399|91.48|17.46|true|priscilla johnson|2013-03-01 09:11:58.703267|22.44|history +120|325|65758|4294967540|79.19|11.26|true|wendy underhill|2013-03-01 09:11:58.703226|94.90|debate diff --git a/src/test/hive/data/files/part.rc b/src/test/hive/data/files/part.rc new file mode 100644 index 0000000000000000000000000000000000000000..e39e72ea58bb2346bf165633f7e440a0db4564cf GIT binary patch literal 3214 zcmcIm+in|07_Nh2*AZ=L(_imq#_O>?c)gqKA+5RO zo>)SH7b#DGgeX_^2Jr@c0j^N175|>>*eOUv!dlPlKlA~o6=reg(TTd;$1m$^CPvWrS~6$+i%f-YkE38 zy_pYJ^Q&S04gBHeO*Dk3@BIYV)=v1~(|h5Ad*SxZ_We7zKfZ^yKiR$$-oC$cnzP_q zx%ztgYWiyB^6S%;8*puX5Uy`*-r5K^Z>`_Fv2klN9&c{k*j#`3a6K2Uw@65M7nwuc)d9rm8F0z?x`aSvBBZ zBC4ulLs3Q55F-zbXcERmGIUuqbb@7>7?Ny&v#iTRlNF)@$N-ciDH=A2YG^W$7+6tY z4J%lbG+owJYK z39~Hc*%F->+vp4)(y%7a9H*Bdj{`cUF1OES*tMuRLY~i|mAD0>D2$LcZL zoi{fn%N{mzTxt&(>hbYXxG{8bf?PNqjs>st(r&zlOz^glO;6}o%h}GV$ohwpC$Pkm zK9(`JA!!E3Guxl{uW!Xs_=)3AP?vjfbkBp5vxa(3ZvZDRqRTK}2QEW&PIp9)=!CvI zo2~_~MW4H@>vA}5XK7}A>+xWWydyZD0~9}pEKrK@MiX?z+#v02wmVNC%`rJNvo>Y8 zI9)jHwi&&c=8*UMaBi}rR;PB*rzRXu=HA8sgGClNFKS5(EEl!p1(u6i$^y%Ou*&de z^zLGLX(p+yq;qDcGg+Rc1V>UHlKM$FS(h?9^^>wW#mgy9PSBoaLiDz?(QGf=g3O4)vC6um1d(^{{pp|wS@RAb{;MPcj~2DZ4O<^ zVO~!fYVJO(ah4~noI{n{rA}v-P|}?=a8~DHnn^xcpJMm4l?&CRX(xnneOZvqbt%ff zW(k?k+P+ZXLYReG=CC;W;^U2xgh;YVKtq6}5D8JRjzLp^7?C5%0D?jT)qzz4AnEa8 zz1&3Q=F{0tr;f^{y~J!i+55a*YRwlarA`%EjVh(6{}Qu?62JyBB@GTy`9wAsq}X z<{=Y4;ueDmJQ+wh#7)#=U?HFbC))Sn(;5ZwK!Z8p4)>{tiTcj-n8QXM9n8}}wCe;Q zyG$preGj>TjgABOwhkFEa@VmT2uRL2Sx_^cfFN{YnWGruaqx?CoYf|vt_xprb3_La RbI4=0nUKW1U5op}zX1)n6e$1z literal 0 HcmV?d00001 diff --git a/src/test/hive/data/files/part.seq b/src/test/hive/data/files/part.seq new file mode 100644 index 0000000000000000000000000000000000000000..b919eb28db6975f8793837c1bdf309bd5aaf86a3 GIT binary patch literal 3598 zcmd5g5w7-l5IjSDw!BUVBjkmVVFACop(x|=j5X|;Svh>1N(hB%(oA8E6@u&^K& zBv6n1AjI$Br@(7_#!k{&T#1xcYO2gL&)55UI{5UR4`jM9*oK8mZY)_O<;D=w=(6)p`0BkcKm0nd{`%-|a`G|!@fEz^EqGiaLLj<$Dub1cJiQOf5<9K+$i zX#jUW6Jk2SJw38YAH0VI6pD8@QbrvZ9HmMBYI1&7p*TM~c?vLGrOd@s0OFd|Z~}xu zm3+zZyhw#CvbZGVdyVOZ@7lB_<+@;NmWpKGB(s=>E1cz=#~pAenqKzKPkWP7+&dqi zo_*2K0$=@X(z~dMv59TiaQm8>9R>`)Y8-bzx{m95jXEt5u+<8$Wf-zdBybUtBAqwnD}{2eD)* zW0rFaak(y}$c`{kl^;scXmEOVHEO9j8DCrsCJ2%RpWu_;IUov51H2jU@CGa>k_Arr zbpcpjYCJb2Af6d?z-d6>Iov4cN5Vk;6+gdDjsj*bbL|0x1k}|Ws;jraGrHDE$8F1msD^Lv>&Mq~n zx^+VW$2A{1KZDxb;c&+iykH{9W-^r>0Gu{Bd-Y2El7}k7o=FTBi4aGn5PRkfOxGvP z<&`)08f3L(D}is8B0ut_rF?0T`!bZ>PP|40;(Bm8ltaof)CQQ7&2y;7vP`nF*}l_c z?1)D_>ln^lraS{n=vrOKIO0iq+-eOX-D*vTRCw%z=n@DDEo(wTMgRyPc&J%18&R zk!trftTH&}@KFZ@s+Xd}76n)61|yc|l+$j>AvYqG34~YxfVRMFhq^5lZGX+3C1}j@93e{h=Vd^G_OI0Jc9(B}o4eFw*K8rH5Edj&}_6TUtueTo@$!Gy#FRAD_ zo-~ftN+`=#6;kz=Z`%eeJkUa!f%&$KYYV`#ET^6L&tdJvtc9_Gy?Dp|(+S#~howOe zIDpQuoO&Sx05|ZiDm*0tb%PdxUXNqgk>?`ivlKk@00E9NB5c-bw6}dxRu)t%MD}!Y zHG($v1H%oV!B>?H4(@Nm0J}HP*E|x%8fP2W!WOuy@k9EX(;)`>@`vZU)%2?O3IYx8 zqD(*xs(B6oS_cxx@vM8*kgdusBrQ~5cARdh5UROVXHe0uukIT#6q>cuz_XOW`M+!a z@7@rHnC;uxe_+u1Rl{M*!skL4r_&8cRCOqKO!h{s>MgG!Y-`Beq9c;Smeup0UoLe ABme*a literal 0 HcmV?d00001 diff --git a/src/test/hive/data/files/part_tiny.txt b/src/test/hive/data/files/part_tiny.txt new file mode 100644 index 0000000000000..f6973fd90a226 --- /dev/null +++ b/src/test/hive/data/files/part_tiny.txt @@ -0,0 +1,26 @@ +121152almond antique burnished rose metallicManufacturer#1Brand#14PROMO PLATED TIN2JUMBO BOX1173.15e pinto beans h +121152almond antique burnished rose metallicManufacturer#1Brand#14PROMO PLATED TIN2JUMBO BOX1173.15e pinto beans h +85768almond antique chartreuse lavender yellowManufacturer#1Brand#12LARGE BRUSHED STEEL34SM BAG1753.76refull +110592almond antique salmon chartreuse burlywoodManufacturer#1Brand#15PROMO BURNISHED NICKEL6JUMBO PKG1602.59 to the furiously +86428almond aquamarine burnished black steelManufacturer#1Brand#12STANDARD ANODIZED STEEL28WRAP BAG1414.42arefully +65667almond aquamarine pink moccasin thistleManufacturer#1Brand#12LARGE BURNISHED STEEL42JUMBO CASE1632.66e across the expr +105685almond antique violet chocolate turquoiseManufacturer#2Brand#22MEDIUM ANODIZED COPPER14MED CAN1690.68ly pending requ +191709almond antique violet turquoise frostedManufacturer#2Brand#22ECONOMY POLISHED STEEL40MED BOX1800.7 haggle +146985almond aquamarine midnight light salmonManufacturer#2Brand#23MEDIUM BURNISHED COPPER2SM CASE2031.98s cajole caref +132666almond aquamarine rose maroon antiqueManufacturer#2Brand#24SMALL POLISHED NICKEL25MED BOX1698.66even +195606almond aquamarine sandy cyan gainsboroManufacturer#2Brand#25STANDARD PLATED TIN18SM PKG1701.6ic de +90681almond antique chartreuse khaki whiteManufacturer#3Brand#31MEDIUM BURNISHED TIN17SM CASE1671.68are slyly after the sl +17273almond antique forest lavender goldenrodManufacturer#3Brand#35PROMO ANODIZED TIN14JUMBO CASE1190.27along the +112398almond antique metallic orange dimManufacturer#3Brand#32MEDIUM BURNISHED BRASS19JUMBO JAR1410.39ole car +40982almond antique misty red oliveManufacturer#3Brand#32ECONOMY PLATED COPPER1LG PKG1922.98c foxes can s +144293almond antique olive coral navajoManufacturer#3Brand#34STANDARD POLISHED STEEL45JUMBO CAN1337.29ag furiously about +49671almond antique gainsboro frosted violetManufacturer#4Brand#41SMALL BRUSHED BRASS10SM BOX1620.67ccounts run quick +48427almond antique violet mint lemonManufacturer#4Brand#42PROMO POLISHED STEEL39SM CASE1375.42hely ironic i +45261almond aquamarine floral ivory bisqueManufacturer#4Brand#42SMALL PLATED STEEL27WRAP CASE1206.26careful +17927almond aquamarine yellow dodger mintManufacturer#4Brand#41ECONOMY BRUSHED COPPER7SM PKG1844.92ites. eve +33357almond azure aquamarine papaya violetManufacturer#4Brand#41STANDARD ANODIZED TIN12WRAP CASE1290.35reful +192697almond antique blue firebrick mintManufacturer#5Brand#52MEDIUM BURNISHED TIN31LG DRUM1789.69ickly ir +42669almond antique medium spring khakiManufacturer#5Brand#51STANDARD BURNISHED TIN6MED CAN1611.66sits haggl +155733almond antique sky peru orangeManufacturer#5Brand#53SMALL PLATED BRASS2WRAP DRUM1788.73furiously. bra +15103almond aquamarine dodger light gainsboroManufacturer#5Brand#53ECONOMY BURNISHED STEEL46LG PACK1018.1packages hinder carefu +78486almond azure blanched chiffon midnightManufacturer#5Brand#52LARGE BRUSHED BRASS23MED BAG1464.48hely blith diff --git a/src/test/hive/data/files/person age.txt b/src/test/hive/data/files/person age.txt new file mode 100644 index 0000000000000..c902284c11155 --- /dev/null +++ b/src/test/hive/data/files/person age.txt @@ -0,0 +1,5 @@ +John 23 +Tom 17 +Jim 31 +Boby 9 +Paul 51 \ No newline at end of file diff --git a/src/test/hive/data/files/primitive_type_arrays.txt b/src/test/hive/data/files/primitive_type_arrays.txt new file mode 100644 index 0000000000000..2ab5f42c55979 --- /dev/null +++ b/src/test/hive/data/files/primitive_type_arrays.txt @@ -0,0 +1 @@ +54321987216432168410001003572461truefalse3.1411.6182.7181.4143.141591.618032.718281.41421portosathosaramis1970-01-16 12:50:35.2421970-01-07 00:54:54.4421970-01-05 13:51:04.042 diff --git a/src/test/hive/data/files/pw17.txt b/src/test/hive/data/files/pw17.txt new file mode 100644 index 0000000000000..041944a1b022b --- /dev/null +++ b/src/test/hive/data/files/pw17.txt @@ -0,0 +1,4 @@ +1 12 1 +2 23 2 +4 56 3 +6 8 4 \ No newline at end of file diff --git a/src/test/hive/data/files/sales.txt b/src/test/hive/data/files/sales.txt new file mode 100644 index 0000000000000..6407b72051b51 --- /dev/null +++ b/src/test/hive/data/files/sales.txt @@ -0,0 +1,2 @@ +Joe 2 +Hank 2 diff --git a/src/test/hive/data/files/sample-queryplan-in-history.txt b/src/test/hive/data/files/sample-queryplan-in-history.txt new file mode 100644 index 0000000000000..79c034dabed01 --- /dev/null +++ b/src/test/hive/data/files/sample-queryplan-in-history.txt @@ -0,0 +1 @@ +Counters plan="{"queryId":"rmurthy_20090806105454","queryType":null,"queryAttributes":{"queryString":"select count(1) from nectar_fbalias_r where ds = '2009-08-03'"},"queryCounters":"null","stageGraph":{"nodeType":"1","roots":"null","adjacencyList":"]"},"stageList":[{"stageId":"Stage-1","stageType":"3","stageAttributes":"null","stageCounters":"null","taskList":[{"taskId":"Stage-1_map","taskType":"0","taskAttributes":"null","taskCounters":"null","operatorGraph":{"nodeType":"0","roots":"null","adjacencyList":[{"node":"TS_0","children":["FIL_8"],"adjacencyType":"0"},{"node":"FIL_8","children":["FIL_1"],"adjacencyType":"0"},{"node":"FIL_1","children":["SEL_2"],"adjacencyType":"0"},{"node":"SEL_2","children":["GBY_3"],"adjacencyType":"0"},{"node":"GBY_3","children":["RS_4"],"adjacencyType":"0"}]},"operatorList":[{"operatorId":"TS_0","operatorType":"9","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_TS_0_TIME_TAKEN":"5048","CNTR_NAME_TS_0_NUM_INPUT_ROWS":"52482","CNTR_NAME_TS_0_NUM_OUTPUT_ROWS":"52482"}},{"operatorId":"FIL_8","operatorType":"3","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_FIL_8_NUM_INPUT_ROWS":"52482","CNTR_NAME_FIL_8_NUM_OUTPUT_ROWS":"52482","CNTR_NAME_FIL_8_TIME_TAKEN":"4860"}},{"operatorId":"FIL_1","operatorType":"3","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_FIL_1_TIME_TAKEN":"1368","CNTR_NAME_FIL_1_NUM_INPUT_ROWS":"52482","CNTR_NAME_FIL_1_NUM_OUTPUT_ROWS":"52482"}},{"operatorId":"SEL_2","operatorType":"8","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_SEL_2_TIME_TAKEN":"604","CNTR_NAME_SEL_2_NUM_OUTPUT_ROWS":"52482","CNTR_NAME_SEL_2_NUM_INPUT_ROWS":"52482"}},{"operatorId":"GBY_3","operatorType":"5","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_GBY_3_NUM_INPUT_ROWS":"52482","CNTR_NAME_GBY_3_TIME_TAKEN":"404","CNTR_NAME_GBY_3_NUM_OUTPUT_ROWS":"64"}},{"operatorId":"RS_4","operatorType":"11","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_RS_4_NUM_INPUT_ROWS":"64","CNTR_NAME_RS_4_TIME_TAKEN":"119","CNTR_NAME_RS_4_NUM_OUTPUT_ROWS":"0"}}],"done":"false"},{"taskId":"Stage-1_reduce","taskType":"1","taskAttributes":"null","taskCounters":"null","operatorGraph":{"nodeType":"0","roots":"null","adjacencyList":[{"node":"GBY_5","children":["SEL_6"],"adjacencyType":"0"},{"node":"SEL_6","children":["FS_7"],"adjacencyType":"0"}]},"operatorList":[{"operatorId":"GBY_5","operatorType":"5","operatorAttributes":"null","operatorCounters":"null"},{"operatorId":"SEL_6","operatorType":"8","operatorAttributes":"null","operatorCounters":"null"},{"operatorId":"FS_7","operatorType":"10","operatorAttributes":"null","operatorCounters":"null"}],"done":"false"}],"done":"false"}],"done":"false"}" TIME="1249581360419" diff --git a/src/test/hive/data/files/sample-queryplan.txt b/src/test/hive/data/files/sample-queryplan.txt new file mode 100644 index 0000000000000..3b9167b4ac718 --- /dev/null +++ b/src/test/hive/data/files/sample-queryplan.txt @@ -0,0 +1 @@ +{"queryId":"rmurthy_20090806105454","queryType":null,"queryAttributes":{"queryString":"select count(1) from nectar_fbalias_r where ds = '2009-08-03'"},"queryCounters":"null","stageGraph":{"nodeType":"1","roots":"null","adjacencyList":"]"},"stageList":[{"stageId":"Stage-1","stageType":"3","stageAttributes":"null","stageCounters":"null","taskList":[{"taskId":"Stage-1_map","taskType":"0","taskAttributes":"null","taskCounters":"null","operatorGraph":{"nodeType":"0","roots":"null","adjacencyList":[{"node":"TS_0","children":["FIL_8"],"adjacencyType":"0"},{"node":"FIL_8","children":["FIL_1"],"adjacencyType":"0"},{"node":"FIL_1","children":["SEL_2"],"adjacencyType":"0"},{"node":"SEL_2","children":["GBY_3"],"adjacencyType":"0"},{"node":"GBY_3","children":["RS_4"],"adjacencyType":"0"}]},"operatorList":[{"operatorId":"TS_0","operatorType":"9","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_TS_0_TIME_TAKEN":"4420","CNTR_NAME_TS_0_NUM_INPUT_ROWS":"47303","CNTR_NAME_TS_0_NUM_OUTPUT_ROWS":"47303"}},{"operatorId":"FIL_8","operatorType":"3","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_FIL_8_NUM_INPUT_ROWS":"47303","CNTR_NAME_FIL_8_NUM_OUTPUT_ROWS":"47303","CNTR_NAME_FIL_8_TIME_TAKEN":"4251"}},{"operatorId":"FIL_1","operatorType":"3","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_FIL_1_TIME_TAKEN":"1249","CNTR_NAME_FIL_1_NUM_INPUT_ROWS":"47303","CNTR_NAME_FIL_1_NUM_OUTPUT_ROWS":"47303"}},{"operatorId":"SEL_2","operatorType":"8","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_SEL_2_TIME_TAKEN":"550","CNTR_NAME_SEL_2_NUM_OUTPUT_ROWS":"47303","CNTR_NAME_SEL_2_NUM_INPUT_ROWS":"47303"}},{"operatorId":"GBY_3","operatorType":"5","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_GBY_3_NUM_INPUT_ROWS":"47303","CNTR_NAME_GBY_3_TIME_TAKEN":"368","CNTR_NAME_GBY_3_NUM_OUTPUT_ROWS":"57"}},{"operatorId":"RS_4","operatorType":"11","operatorAttributes":"null","operatorCounters":{"CNTR_NAME_RS_4_NUM_INPUT_ROWS":"57","CNTR_NAME_RS_4_TIME_TAKEN":"105","CNTR_NAME_RS_4_NUM_OUTPUT_ROWS":"0"}}],"done":"false"},{"taskId":"Stage-1_reduce","taskType":"1","taskAttributes":"null","taskCounters":"null","operatorGraph":{"nodeType":"0","roots":"null","adjacencyList":[{"node":"GBY_5","children":["SEL_6"],"adjacencyType":"0"},{"node":"SEL_6","children":["FS_7"],"adjacencyType":"0"}]},"operatorList":[{"operatorId":"GBY_5","operatorType":"5","operatorAttributes":"null","operatorCounters":"null"},{"operatorId":"SEL_6","operatorType":"8","operatorAttributes":"null","operatorCounters":"null"},{"operatorId":"FS_7","operatorType":"10","operatorAttributes":"null","operatorCounters":"null"}],"done":"false"}],"done":"false"}],"done":"false"} diff --git a/src/test/hive/data/files/smallsrcsortbucket1outof4.txt b/src/test/hive/data/files/smallsrcsortbucket1outof4.txt new file mode 100644 index 0000000000000..1175eae354dc9 --- /dev/null +++ b/src/test/hive/data/files/smallsrcsortbucket1outof4.txt @@ -0,0 +1,5 @@ +0val_0 +103val_103 +169val_169 +172val_172 +374val_374 diff --git a/src/test/hive/data/files/smallsrcsortbucket2outof4.txt b/src/test/hive/data/files/smallsrcsortbucket2outof4.txt new file mode 100644 index 0000000000000..59347ff9a9170 --- /dev/null +++ b/src/test/hive/data/files/smallsrcsortbucket2outof4.txt @@ -0,0 +1,5 @@ +180val_180 +221val_221 +379val_379 +478val_478 +74val_74 diff --git a/src/test/hive/data/files/smallsrcsortbucket3outof4.txt b/src/test/hive/data/files/smallsrcsortbucket3outof4.txt new file mode 100644 index 0000000000000..354f45da73862 --- /dev/null +++ b/src/test/hive/data/files/smallsrcsortbucket3outof4.txt @@ -0,0 +1,5 @@ +233val_233 +424val_424 +468val_468 +53val_53 +97val_97 diff --git a/src/test/hive/data/files/smallsrcsortbucket4outof4.txt b/src/test/hive/data/files/smallsrcsortbucket4outof4.txt new file mode 100644 index 0000000000000..6aecf76a4022e --- /dev/null +++ b/src/test/hive/data/files/smallsrcsortbucket4outof4.txt @@ -0,0 +1,5 @@ +146val_146 +193val_193 +432val_432 +65val_65 +83val_83 diff --git a/src/test/hive/data/files/smb_bucket_input.rc b/src/test/hive/data/files/smb_bucket_input.rc new file mode 100644 index 0000000000000000000000000000000000000000..61814607246d1db098a84f6f16d2c4615a873922 GIT binary patch literal 253 zcmWG`4P?{JFG|--EJ#ewNY%?oOv%qL(96gyOVumP(aX%&3vzbL%t=-8POWq*O-oBH z(kDiBSYl3TDnva42r$ZkO#~WOl$-`MPA@q>r!+TDFRwH=DYb~vXysz3dBLmRtW1|q zin_Zc?+8#!6c7u5FuQ~V3nMejZzCgC_J17ge@sj)ObiVS42*!-7>G@P*c6D%5_95> YP0c`*p_wI^f{MbpP*EdOu%wYG09%buQ~&?~ literal 0 HcmV?d00001 diff --git a/src/test/hive/data/files/smb_bucket_input.txt b/src/test/hive/data/files/smb_bucket_input.txt new file mode 100644 index 0000000000000..e3a7e2977bc68 --- /dev/null +++ b/src/test/hive/data/files/smb_bucket_input.txt @@ -0,0 +1,7 @@ +4val_356 +484val_169 +1000val_1000 +2000val_169 +3000val_169 +4000val_125 +5000val_125 diff --git a/src/test/hive/data/files/smbbucket_1.rc b/src/test/hive/data/files/smbbucket_1.rc new file mode 100644 index 0000000000000000000000000000000000000000..51dbe23ae8631652ab1fda281f2cc5ae18ca49db GIT binary patch literal 208 zcmWG`4P?{JFG|--EJ#ewNY%?oOv%qL(96gyOVumP(aX%&3vzbL%t=-8POWq*O-oBH z(kDiBSYl3TDnva42r$ZkO#~WOl$-`MPA@q>r!+TDFRwH=DYb~vhr!+TDFRwH=DYb~vNTSthM}61vC)Y&F y-@Xv#T?N#l4aD3a%)-IJ#Pml(f{E>qk%5u1k*TpkSz=DS5ri}bvrR#yu>k-P|3X~= literal 0 HcmV?d00001 diff --git a/src/test/hive/data/files/smbbucket_2.txt b/src/test/hive/data/files/smbbucket_2.txt new file mode 100644 index 0000000000000..430a0ca887730 --- /dev/null +++ b/src/test/hive/data/files/smbbucket_2.txt @@ -0,0 +1,4 @@ +20val_20 +23val_23 +25val_25 +30val_30 diff --git a/src/test/hive/data/files/smbbucket_3.rc b/src/test/hive/data/files/smbbucket_3.rc new file mode 100644 index 0000000000000000000000000000000000000000..c0e8a590de75328a655ab1a05c9aa12c76b0968b GIT binary patch literal 222 zcmWG`4P?{JFG|--EJ#ewNY%?oOv%qL(96gyOVumP(aX%&3vzbL%t=-8POWq*O-oBH z(kDiBSYl3TDnva42r$ZkO#~WOl$-`MPA@q>r!+TDFRwH=DYb~vNNCNn+d>+(!8=4< zu5ISN@DZrR8i;v8n2np8nUU$YvNAI(+iw#?14DB|OCtj#5O$ao4TLdj_LZO~$NpjL>o5;6b) literal 0 HcmV?d00001 diff --git a/src/test/hive/data/files/symlink1.txt b/src/test/hive/data/files/symlink1.txt new file mode 100644 index 0000000000000..dc1a7c5682a26 --- /dev/null +++ b/src/test/hive/data/files/symlink1.txt @@ -0,0 +1,2 @@ +../data/files/T1.txt +../data/files/T3.txt diff --git a/src/test/hive/data/files/symlink2.txt b/src/test/hive/data/files/symlink2.txt new file mode 100644 index 0000000000000..8436a30adf366 --- /dev/null +++ b/src/test/hive/data/files/symlink2.txt @@ -0,0 +1 @@ +../data/files/T2.txt diff --git a/src/test/hive/data/files/tbl.txt b/src/test/hive/data/files/tbl.txt new file mode 100644 index 0000000000000..10287c34423ca --- /dev/null +++ b/src/test/hive/data/files/tbl.txt @@ -0,0 +1,10 @@ +1|1997 +2|1997 +2|1994 +2|1998 +3|1997 +3|1998 +4|1996 +4|1997 +6|1997 +7|1997 diff --git a/src/test/hive/data/files/test.dat b/src/test/hive/data/files/test.dat new file mode 100644 index 0000000000000..cf0389a5afee1 --- /dev/null +++ b/src/test/hive/data/files/test.dat @@ -0,0 +1,6 @@ +1 +2 +3 +4 +5 +6 \ No newline at end of file diff --git a/src/test/hive/data/files/test2.dat b/src/test/hive/data/files/test2.dat new file mode 100644 index 0000000000000..4b284c13ffafd --- /dev/null +++ b/src/test/hive/data/files/test2.dat @@ -0,0 +1,6 @@ +101 +202 +303 +404 +505 +606 \ No newline at end of file diff --git a/src/test/hive/data/files/text-en.txt b/src/test/hive/data/files/text-en.txt new file mode 100644 index 0000000000000..480c818a875fa --- /dev/null +++ b/src/test/hive/data/files/text-en.txt @@ -0,0 +1,95 @@ +One morning, when Gregor Samsa woke from troubled dreams, he found himself transformed in his bed into a horrible vermin. He lay on his armour-like back, and if he lifted his head a little he could see his brown belly, slightly domed and divided by arches into stiff sections. The bedding was hardly able to cover it and seemed ready to slide off any moment. His many legs, pitifully thin compared with the size of the rest of him, waved about helplessly as he looked. +"What's happened to me?" he thought. It wasn't a dream. His room, a proper human room although a little too small, lay peacefully between its four familiar walls. +A collection of textile samples lay spread out on the table - Samsa was a travelling salesman - and above it there hung a picture that he had recently cut out of an illustrated magazine and housed in a nice, gilded frame. +It showed a lady fitted out with a fur hat and fur boa who sat upright, raising a heavy fur muff that covered the whole of her lower arm towards the viewer. +Gregor then turned to look out the window at the dull weather. +Drops of rain could be heard hitting the pane, which made him feel quite sad. "How about if I sleep a little bit longer and forget all this nonsense", he thought, but that was something he was unable to do because he was used to sleeping on his right, and in his present state couldn't get into that position. However hard he threw himself onto his right, he always rolled back to where he was. +He must have tried it a hundred times, shut his eyes so that he wouldn't have to look at the floundering legs, and only stopped when he began to feel a mild, dull pain there that he had never felt before. +"Oh, God", he thought, "what a strenuous career it is that I've +chosen! Travelling day in and day out. Doing business like this +takes much more effort than doing your own business at home, and on +top of that there's the curse of travelling, worries about making +train connections, bad and irregular food, contact with different +people all the time so that you can never get to know anyone or +become friendly with them. It can all go to Hell!" He felt a +slight itch up on his belly; pushed himself slowly up on his back +towards the headboard so that he could lift his head better; found +where the itch was, and saw that it was covered with lots of little +white spots which he didn't know what to make of; and when he tried +to feel the place with one of his legs he drew it quickly back because as soon as he touched it he was overcome by a cold shudder. He slid back into his former position. "Getting up early all the time", he thought, "it makes you stupid. You've got to get enough sleep. Other travelling salesmen live a life of luxury. For instance, whenever I go back to the guest house during the morning to copy out the contract, these gentlemen are always still sitting there eating their breakfasts. I ought to just try that with my +boss; I'd get kicked out on the spot. But who knows, maybe that +would be the best thing for me. If I didn't have my parents to +think about I'd have given in my notice a long time ago, I'd have +gone up to the boss and told him just what I think, tell him +everything I would, let him know just what I feel. He'd fall right +off his desk! And it's a funny sort of business to be sitting up +there at your desk, talking down at your subordinates from up there, +especially when you have to go right up close because the boss is +hard of hearing. Well, there's still some hope; once I've got the +money together to pay off my parents' debt to him - another five or +six years I suppose - that's definitely what I'll do. That's when +I'll make the big change. First of all though, I've got to get up, +my train leaves at five." +One morning, when Gregor Samsa woke from troubled dreams, he found himself transformed in his bed into a horrible vermin. He lay on his armour-like back, and if he lifted his head a little he could see his brown belly, slightly domed and divided by arches into stiff sections. The bedding was hardly able to cover it and seemed ready to slide off any moment. His many legs, pitifully thin compared with the size of the rest of him, waved about helplessly as he looked. +"What's happened to me?" he thought. It wasn't a dream. His room, a proper human room although a little too small, lay peacefully between its four familiar walls. +A collection of textile samples lay spread out on the table - Samsa was a travelling salesman - and above it there hung a picture that he had recently cut out of an illustrated magazine and housed in a nice, gilded frame. +It showed a lady fitted out with a fur hat and fur boa who sat upright, raising a heavy fur muff that covered the whole of her lower arm towards the viewer. +Gregor then turned to look out the window at the dull weather. +Drops of rain could be heard hitting the pane, which made him feel quite sad. "How about if I sleep a little bit longer and forget all this nonsense", he thought, but that was something he was unable to do because he was used to sleeping on his right, and in his present state couldn't get into that position. However hard he threw himself onto his right, he always rolled back to where he was. +He must have tried it a hundred times, shut his eyes so that he wouldn't have to look at the floundering legs, and only stopped when he began to feel a mild, dull pain there that he had never felt before. +"Oh, God", he thought, "what a strenuous career it is that I've +chosen! Travelling day in and day out. Doing business like this +takes much more effort than doing your own business at home, and on +top of that there's the curse of travelling, worries about making +train connections, bad and irregular food, contact with different +people all the time so that you can never get to know anyone or +become friendly with them. It can all go to Hell!" He felt a +slight itch up on his belly; pushed himself slowly up on his back +towards the headboard so that he could lift his head better; found +where the itch was, and saw that it was covered with lots of little +white spots which he didn't know what to make of; and when he tried +to feel the place with one of his legs he drew it quickly back +because as soon as he touched it he was overcome by a cold shudder. +He slid back into his former position. "Getting up early all the +time", he thought, "it makes you stupid. You've got to get enough +sleep. Other travelling salesmen live a life of luxury. For +instance, whenever I go back to the guest house during the morning +to copy out the contract, these gentlemen are always still sitting +there eating their breakfasts. I ought to just try that with my +boss; I'd get kicked out on the spot. But who knows, maybe that +would be the best thing for me. If I didn't have my parents to +think about I'd have given in my notice a long time ago, I'd have +gone up to the boss and told him just what I think, tell him +everything I would, let him know just what I feel. He'd fall right +off his desk! And it's a funny sort of business to be sitting up +there at your desk, talking down at your subordinates from up there, +especially when you have to go right up close because the boss is +hard of hearing. Well, there's still some hope; once I've got the +money together to pay off my parents' debt to him - another five or +six years I suppose - that's definitely what I'll do. That's when +I'll make the big change. First of all though, I've got to get up, +my train leaves at five." +And he looked over at the alarm clock, ticking on the chest of drawers. "God in Heaven!" he thought. It was half past six and the hands were quietly moving forwards, it was even later than half past, more like quarter to seven. Had the alarm clock not rung? He could see from the bed that it had been set for four o'clock as it should have been; it certainly must have rung. Yes, but was it +possible to quietly sleep through that furniture-rattling noise? True, he had not slept peacefully, but probably all the more deeply because of that. What should he do now? The next train went at seven; if he were to catch that he would have to rush like mad and +the collection of samples was still not packed, and he did not at all feel particularly fresh and lively. And even if he did catch the train he would not avoid his boss's anger as the office assistant would have been there to see the five o'clock train go, he +would have put in his report about Gregor's not being there a long time ago. The office assistant was the boss's man, spineless, and with no understanding. What about if he reported sick? But that would be extremely strained and suspicious as in fifteen years of +service Gregor had never once yet been ill. His boss would certainly come round with the doctor from the medical insurance company, accuse his parents of having a lazy son, and accept the doctor's recommendation not to make any claim as the doctor believed +that no-one was ever ill but that many were workshy. And what's more, would he have been entirely wrong in this case? Gregor did in fact, apart from excessive sleepiness after sleeping for so long, +feel completely well and even felt much hungrier than usual. +One morning, as Gregor Samsa was waking up from anxious dreams, he discovered that in his bed he had been changed into a monstrous verminous bug. He lay on his armour-hard back and saw, as he lifted his head up a little, his brown, arched abdomen divided up into rigid bow-like sections. From this height the blanket, just about ready to slide off completely, could hardly stay in place. His numerous legs, pitifully thin in comparison to the rest of his circumference, flickered helplessly before his eyes. +“What’s happened to me,†he thought. It was no dream. His room, a proper room for a human being, only somewhat too small, lay quietly between the four well-known walls. Above the table, on which an unpacked collection of sample cloth goods was spread out—Samsa was a travelling salesman—hung the picture which he had cut out of an illustrated magazine a little while ago and set in a pretty gilt frame. It was a picture of a woman with a fur hat and a fur boa. She sat erect there, lifting up in the direction of the viewer a solid fur muff into which her entire forearm had disappeared. +“O God,†he thought, “what a demanding job I’ve chosen! Day in, day out, on the road. The stresses of selling are much greater than the actual work going on at head office, and, in addition to that, I still have to cope with the problems of travelling, the worries about train connections, irregular bad food, temporary and constantly changing human relationships, which never come from the heart. To hell with it all!†He felt a slight itching on the top of his abdomen. He slowly pushed himself on his back closer to the bed post so that he could lift his head more easily, found the itchy part, which was entirely covered with small white spots—he did not know what to make of them and wanted to feel the place with a leg. But he retracted it immediately, for the contact felt like a cold shower all over him. +He slid back again into his earlier position. “This getting up early,†he thought, “makes a man quite idiotic. A man must have his sleep. Other travelling salesmen live like harem women. For instance, when I come back to the inn during the course of the morning to write up the necessary orders, these gentlemen are just sitting down to breakfast. If I were to try that with my boss, I’d be thrown out on the spot. Still, who knows whether that mightn’t be really good for me. If I didn’t hold back for my parents’ sake, I’d have quit ages ago. I would’ve gone to the boss and told him just what I think from the bottom of my heart. He would’ve fallen right off his desk! How weird it is to sit up at that desk and talk down to the employee from way up there. What’s more, the boss has trouble hearing, so the employee has to step up quite close to him. Anyway, I haven’t completely given up that hope yet. Once I’ve got together the money to pay off my parents’ debt to him—that should take another five or six years—I’ll do it for sure. Then I’ll make the big break. In any case, right now I have to get up. My train leaves at five o’clock.†+He looked over at the alarm clock ticking away by the chest of drawers. “Good God!†he thought. It was half past six, and the hands were going quietly on. It was even past the half hour, already nearly quarter to. Could the alarm have failed to ring? One saw from the bed that it was properly set for four o’clock. Certainly it had rung. Yes, but was it possible to sleep peacefully through that noise which made the furniture shake? Now, it is true he had not slept peacefully, but evidently he had slept all the more deeply. Still, what should he do now? The next train left at seven o’clock. To catch that one, he would have to go in a mad rush. The sample collection was not packed up yet, and he really did not feel particularly fresh and active. And even if he caught the train, there was no avoiding a blow-up with the boss, because the firm’s errand boy would have waited for the five o’clock train and reported the news of his absence long ago. He was the boss’s minion, without backbone and intelligence. Well then, what if he reported in sick? But that would be extremely embarrassing and suspicious, because during his five years’ service Gregor had not been sick even once. The boss would certainly come with the doctor from the health insurance company and would reproach his parents for their lazy son and cut short all objections with the insurance doctor’s comments; for him everyone was completely healthy but really lazy about work. And besides, would the doctor in this case be totally wrong? Apart from a really excessive drowsiness after the long sleep, Gregor, in fact, felt quite well and even had a really strong appetite. +As he was thinking all this over in the greatest haste, without being able to make the decision to get out of bed—the alarm clock was indicating exactly quarter to seven—there was a cautious knock on the door by the head of the bed. “Gregor,†a voice called—it was his mother—“it’s quarter to seven. Don’t you want to be on your way?†The soft voice! Gregor was startled when he heard his voice answering. It was clearly and unmistakably his earlier voice, but in it was intermingled, as if from below, an irrepressible, painful squeaking, which left the words positively distinct only in the first moment and distorted them in the reverberation, so that one did not know if one had heard correctly. Gregor wanted to answer in detail and explain everything, but in these circumstances he confined himself to saying, “Yes, yes, thank you mother. I’m getting up right away.†Because of the wooden door the change in Gregor’s voice was not really noticeable outside, so his mother calmed down with this explanation and shuffled off. However, as a result of the short conversation, the other family members became aware that Gregor was unexpectedly still at home, and already his father was knocking on one side door, weakly but with his fist. “Gregor, Gregor,†he called out, “what’s going on?†And, after a short while, he urged him on again in a deeper voice: “Gregor! Gregor!†At the other side door, however, his sister knocked lightly. “Gregor? Are you all right? Do you need anything?†Gregor directed answers in both directions, “I’ll be ready right away.†He made an effort with the most careful articulation and inserted long pauses between the individual words to remove everything remarkable from his voice. His father turned back to his breakfast. However, the sister whispered, “Gregor, open the door—I beg you.†Gregor had no intention of opening the door, but congratulated himself on his precaution, acquired from travelling, of locking all doors during the night, even at home. +First he wanted to stand up quietly and undisturbed, get dressed, above all have breakfast, and only then consider further action, for—he noticed this clearly—by thinking things over in bed he would not reach a reasonable conclusion. He remembered that he had already often felt some light pain or other in bed, perhaps the result of an awkward lying position, which later, once he stood up, turned out to be purely imaginary, and he was eager to see how his present fantasies would gradually dissipate. That the change in his voice was nothing other than the onset of a real chill, an occupational illness of commercial travellers, of that he had not the slightest doubt. +It was very easy to throw aside the blanket. He needed only to push himself up a little, and it fell by itself. But to continue was difficult, particularly because he was so unusually wide. He needed arms and hands to push himself upright. Instead of these, however, he had only many small limbs, which were incessantly moving with very different motions and which, in addition, he was unable to control. If he wanted to bend one of them, then it was the first to extend itself, and if he finally succeeded doing what he wanted with this limb, in the meantime all the others, as if left free, moved around in an excessively painful agitation. “But I must not stay in bed uselessly,†said Gregor to himself. +At first he wanted to get out of bed with the lower part of his body, but this lower part—which, by the way, he had not yet looked at and which he also could not picture clearly—proved itself too difficult to move. The attempt went so slowly. When, having become almost frantic, he finally hurled himself forward with all his force and without thinking, he chose his direction incorrectly, and he hit the lower bedpost hard. The violent pain he felt revealed to him that the lower part of his body was at the moment probably the most sensitive. +Thus, he tried to get his upper body out of the bed first and turned his head carefully toward the edge of the bed. He managed to do this easily, and in spite of its width and weight his body mass at last slowly followed the turning of his head. But as he finally raised his head outside the bed in the open air, he became anxious about moving forward any further in this manner, for if he allowed himself eventually to fall by this process, it would really take a miracle to prevent his head from getting injured. And at all costs he must not lose consciousness right now. He preferred to remain in bed. +However, after a similar effort, while he lay there again, sighing as before, and once again saw his small limbs fighting one another, if anything even worse than earlier, and did not see any chance of imposing quiet and order on this arbitrary movement, he told himself again that he could not possibly remain in bed and that it might be the most reasonable thing to sacrifice everything if there was even the slightest hope of getting himself out of bed in the process. At the same moment, however, he did not forget to remind himself from time to time of the fact that calm—indeed the calmest—reflection might be much better than confused decisions. At such moments, he directed his gaze as precisely as he could toward the window, but unfortunately there was little confident cheer to be had from a glance at the morning mist, which concealed even the other side of the narrow street. “It’s already seven o’clock,†he told himself at the latest sounds from the alarm clock, “already seven o’clock and still such a fog.†And for a little while longer he lay quietly with weak breathing, as if perhaps waiting for normal and natural conditions to re-emerge out of the complete stillness. +But then he said to himself, “Before it strikes a quarter past seven, whatever happens I must be completely out of bed. Besides, by then someone from the office will arrive to inquire about me, because the office will open before seven o’clock.†And he made an effort then to rock his entire body length out of the bed with a uniform motion. If he let himself fall out of the bed in this way, his head, which in the course of the fall he intended to lift up sharply, would probably remain uninjured. His back seemed to be hard; nothing would really happen to that as a result of the fall onto the carpet. His greatest reservation was a worry about the loud noise which the fall must create and which presumably would arouse, if not fright, then at least concern on the other side of all the doors. However, he had to take that chance. +As Gregor was already in the process of lifting himself half out of bed—the new method was more of a game than an effort; he needed only to rock with a series of jerks—it struck him how easy all this would be if someone were to come to his aid. Two strong people—he thought of his father and the servant girl—would have been quite sufficient. They would only have had to push their arms under his arched back to get him out of the bed, to bend down with their load, and then merely to exercise patience so that he could complete the flip onto the floor, where his diminutive legs would then, he hoped, acquire a purpose. Now, quite apart from the fact that the doors were locked, should he really call out for help? In spite of all his distress, he was unable to suppress a smile at this idea. +He had already got to the point where, by rocking more strongly, he maintained his equilibrium with difficulty, and very soon he would finally have to make a final decision, for in five minutes it would be a quarter past seven. Then there was a ring at the door of the apartment. “That’s someone from the office,†he told himself, and he almost froze, while his small limbs only danced around all the faster. For one moment everything remained still. “They aren’t opening,†Gregor said to himself, caught up in some absurd hope. But of course then, as usual, the servant girl with her firm tread went to the door and opened it. Gregor needed to hear only the first word of the visitor’s greeting to recognize immediately who it was, the manager himself. Why was Gregor the only one condemned to work in a firm where, at the slightest lapse, someone at once attracted the greatest suspicion? Were all the employees then collectively, one and all, scoundrels? Among them was there then no truly devoted person who, if he failed to use just a couple of hours in the morning for office work, would become abnormal from pangs of conscience and really be in no state to get out of bed? Was it really not enough to let an apprentice make inquiries, if such questioning was even generally necessary? Must the manager himself come, and in the process must it be demonstrated to the entire innocent family that the investigation of this suspicious circumstance could be entrusted only to the intelligence of the manager? And more as a consequence of the excited state in which this idea put Gregor than as a result of an actual decision, he swung himself with all his might out of the bed. There was a loud thud, but not a real crash. The fall was absorbed somewhat by the carpet and, in addition, his back was more elastic than Gregor had thought. For that reason the dull noise was not quite so conspicuous. But he had not held his head up with sufficient care and had hit it. He turned his head, irritated and in pain, and rubbed it on the carpet. +“Something has fallen in there,†said the manager in the next room on the left. Gregor tried to imagine to himself whether anything similar to what was happening to him today could have also happened at some point to the manager. At least one had to concede the possibility of such a thing. However, as if to give a rough answer to this question, the manager now, with a squeak of his polished boots, took a few determined steps in the next room. From the neighbouring room on the right the sister was whispering to inform Gregor: “Gregor, the manager is here.†“I know,†said Gregor to himself. But he did not dare make his voice loud enough so that his sister could hear. +“Gregor,†his father now said from the neighbouring room on the left, “Mr. Manager has come and is asking why you have not left on the early train. We don’t know what we should tell him. Besides, he also wants to speak to you personally. So please open the door. He will be good enough to forgive the mess in your room.†In the middle of all this, the manager called out in a friendly way, “Good morning, Mr. Samsa.†“He is not well,†said his mother to the manager, while his father was still talking at the door, “He is not well, believe me, Mr. Manager. Otherwise how would Gregor miss a train? The young man has nothing in his head except business. I’m almost angry that he never goes out in the evening. Right now he’s been in the city eight days, but he’s been at home every evening. He sits here with us at the table and reads the newspaper quietly or studies his travel schedules. It’s a quite a diversion for him to busy himself with fretwork. For instance, he cut out a small frame over the course of two or three evenings. You’d be amazed how pretty it is. It’s hanging right inside the room. You’ll see it immediately, as soon as Gregor opens the door. Anyway, I’m happy that you’re here, Mr. Manager. By ourselves, we would never have made Gregor open the door. He’s so stubborn, and he’s certainly not well, although he denied that this morning.†“I’m coming right away,†said Gregor slowly and deliberately and didn’t move, so as not to lose one word of the conversation. “My dear lady, I cannot explain it to myself in any other way,†said the manager; “I hope it is nothing serious. On the other hand, I must also say that we business people, luckily or unluckily, however one looks at it, very often simply have to overcome a slight indisposition for business reasons.†“So can Mr. Manager come in to see you now?†asked his father impatiently and knocked once again on the door. “No,†said Gregor. In the neighbouring room on the left an awkward stillness descended. In the neighbouring room on the right the sister began to sob. +Why did his sister not go to the others? She had probably just got up out of bed now and had not even started to get dressed yet. Then why was she crying? Because he was not getting up and letting the manager in, because he was in danger of losing his position, and because then his boss would badger his parents once again with the old demands? Those were probably unnecessary worries right now. Gregor was still here and was not thinking at all about abandoning his family. At the moment he was lying right there on the carpet, and no one who knew about his condition would have seriously demanded that he let the manager in. But Gregor would not be casually dismissed right way because of this small discourtesy, for which he would find an easy and suitable excuse later on. It seemed to Gregor that it might be far more reasonable to leave him in peace at the moment, instead of disturbing him with crying and conversation. But it was the very uncertainty which distressed the others and excused their behaviour. diff --git a/src/test/hive/data/files/things.txt b/src/test/hive/data/files/things.txt new file mode 100644 index 0000000000000..7eedbcfae2066 --- /dev/null +++ b/src/test/hive/data/files/things.txt @@ -0,0 +1 @@ +2 Tie diff --git a/src/test/hive/data/files/things2.txt b/src/test/hive/data/files/things2.txt new file mode 100644 index 0000000000000..7eedbcfae2066 --- /dev/null +++ b/src/test/hive/data/files/things2.txt @@ -0,0 +1 @@ +2 Tie diff --git a/src/test/hive/data/files/tiny_a.txt b/src/test/hive/data/files/tiny_a.txt new file mode 100644 index 0000000000000..c7867e6c62e00 --- /dev/null +++ b/src/test/hive/data/files/tiny_a.txt @@ -0,0 +1 @@ +103200920268924913312 diff --git a/src/test/hive/data/files/tiny_b.txt b/src/test/hive/data/files/tiny_b.txt new file mode 100644 index 0000000000000..c8cdef26d7ba2 --- /dev/null +++ b/src/test/hive/data/files/tiny_b.txt @@ -0,0 +1,2 @@ +10320092002467760023374853555556606672767778808187889090919092971001031041071081081091101131131131131131131141161161161171161171171171151151171171171211201311311311251251241241281281311311321331341341341342626\N\N\N\N116 +1032009202689249102591061311421592443203984174335536167108269179711046105110931112114212151220122612321267136415491646194821702272232524332534285229252992311932073279332334123637364536343450347336383688373637583812386238733868388341184134412741704216\N\N\N\N3139 diff --git a/src/test/hive/data/files/types/primitives/090101.txt b/src/test/hive/data/files/types/primitives/090101.txt new file mode 100644 index 0000000000000..35041d5be9a46 --- /dev/null +++ b/src/test/hive/data/files/types/primitives/090101.txt @@ -0,0 +1,25 @@ +0,true,0,0,0,0,0.0,0.0,01/01/09,0,2009-01-01 00:00:00.0 +1,\N,1,1,1,10,1.1,10.1,01/01/09,1,2009-01-01 00:01:00.0 +2,true,\N,2,2,20,2.2,20.2,01/01/09,2,2009-01-01 00:02:00.10 +3,false,3,\N,3,30,3.3,30.299999999999997,01/01/09,3,2009-01-01 00:03:00.30 +4,true,4,4,\N,40,4.4,40.4,01/01/09,4,2009-01-01 00:04:00.60 +5,false,5,5,5,\N,5.5,50.5,01/01/09,5,2009-01-01 00:05:00.100 +6,true,6,6,6,60,\N,60.599999999999994,01/01/09,6,2009-01-01 00:06:00.150 +7,false,7,7,7,70,7.7,\N,01/01/09,7,2009-01-01 00:07:00.210 +8,true,8,8,8,80,8.8,80.8,\N,8,2009-01-01 00:08:00.280 +9,false,9,9,9,90,9.9,90.89999999999999,01/01/09,\N,2009-01-01 00:09:00.360 +10,true,0,0,0,0,0.0,0.0,01/02/09,0,\N +11,\N,1,1,1,10,1.1,10.1,01/02/09,1,2009-01-02 00:11:00.450 +12,true,\N,2,2,20,2.2,20.2,01/02/09,2,2009-01-02 00:12:00.460 +13,false,3,\N,3,30,3.3,30.299999999999997,01/02/09,3,2009-01-02 00:13:00.480 +14,true,4,4,\N,40,4.4,40.4,01/02/09,4,2009-01-02 00:14:00.510 +15,false,5,5,5,\N,5.5,50.5,01/02/09,5,2009-01-02 00:15:00.550 +16,true,6,6,6,60,\N,60.599999999999994,01/02/09,6,2009-01-02 00:16:00.600 +17,false,7,7,7,70,7.7,\N,01/02/09,7,2009-01-02 00:17:00.660 +18,true,8,8,8,80,8.8,80.8,\N,8,2009-01-02 00:18:00.730 +19,false,9,9,9,90,9.9,90.89999999999999,01/02/09,\N,2009-01-02 00:19:00.810 +20,true,0,0,0,0,0.0,0.0,01/03/09,0,\N +21,\N,1,1,1,10,1.1,10.1,01/03/09,1,2009-01-03 00:21:00.900 +22,true,\N,2,2,20,2.2,20.2,01/03/09,2,2009-01-03 00:22:00.910 +23,false,3,\N,3,30,3.3,30.299999999999997,01/03/09,3,2009-01-03 00:23:00.930 +24,true,4,4,\N,40,4.4,40.4,01/03/09,4,2009-01-03 00:24:00.960 diff --git a/src/test/hive/data/files/types/primitives/090201.txt b/src/test/hive/data/files/types/primitives/090201.txt new file mode 100644 index 0000000000000..f2a21f77b1028 --- /dev/null +++ b/src/test/hive/data/files/types/primitives/090201.txt @@ -0,0 +1,25 @@ +25,false,0,0,0,\N,0.0,0.0,02/01/09,0,2009-02-01 00:00:00.0 +26,true,1,1,1,10,\N,10.1,02/01/09,1,2009-02-01 00:01:00.0 +27,false,2,2,2,20,2.2,\N,02/01/09,2,2009-02-01 00:02:00.10 +28,true,3,3,3,30,3.3,\N,02/01/09,3,2009-02-01 00:03:00.30 +29,false,4,4,4,40,4.4,40.4,\N,4,2009-02-01 00:04:00.60 +30,true,5,5,5,50,5.5,50.5,\N,5,2009-02-01 00:05:00.100 +31,false,6,6,6,60,6.6,60.599999999999994,02/01/09,\N,2009-02-01 00:06:00.150 +32,true,7,7,7,70,7.7,70.7,02/01/09,7,\N +33,\N,8,8,8,80,8.8,80.8,02/01/09,8,2009-02-01 00:08:00.280 +34,true,\N,9,9,90,9.9,90.89999999999999,02/01/09,9,2009-02-01 00:09:00.360 +35,false,0,\N,0,0,0.0,0.0,02/02/09,0,2009-02-02 00:10:00.450 +36,true,1,1,\N,10,1.1,10.1,02/02/09,1,2009-02-02 00:11:00.450 +37,false,2,2,2,\N,2.2,20.2,02/02/09,2,2009-02-02 00:12:00.460 +38,true,3,3,3,30,\N,30.299999999999997,02/02/09,3,2009-02-02 00:13:00.480 +39,false,4,4,4,40,4.4,\N,02/02/09,4,2009-02-02 00:14:00.510 +40,true,5,5,5,50,5.5,50.5,\N,5,2009-02-02 00:15:00.550 +41,false,6,6,6,60,6.6,60.599999999999994,02/02/09,\N,2009-02-02 00:16:00.600 +42,true,7,7,7,70,7.7,70.7,02/02/09,7,\N +43,\N,8,8,8,80,8.8,80.8,02/02/09,8,2009-02-02 00:18:00.730 +44,true,\N,9,9,90,9.9,90.89999999999999,02/02/09,9,2009-02-02 00:19:00.810 +45,false,0,\N,0,0,0.0,0.0,02/03/09,0,2009-02-03 00:20:00.900 +46,true,1,1,\N,10,1.1,10.1,02/03/09,1,2009-02-03 00:21:00.900 +47,false,2,2,2,\N,2.2,20.2,02/03/09,2,2009-02-03 00:22:00.910 +48,true,3,3,3,30,\N,30.299999999999997,02/03/09,3,2009-02-03 00:23:00.930 +49,false,4,4,4,40,4.4,\N,02/03/09,4,2009-02-03 00:24:00.960 diff --git a/src/test/hive/data/files/types/primitives/090301.txt b/src/test/hive/data/files/types/primitives/090301.txt new file mode 100644 index 0000000000000..7f475f9295f24 --- /dev/null +++ b/src/test/hive/data/files/types/primitives/090301.txt @@ -0,0 +1,25 @@ +50,true,0,0,0,0,0.0,0.0,\N,0,2009-03-01 00:00:00.0 +51,false,1,1,1,10,1.1,10.1,03/01/09,\N,2009-03-01 00:01:00.0 +52,true,2,2,2,20,2.2,20.2,03/01/09,2,\N +53,\N,3,3,3,30,3.3,30.299999999999997,03/01/09,3,2009-03-01 00:03:00.30 +54,true,\N,4,4,40,4.4,40.4,03/01/09,4,2009-03-01 00:04:00.60 +55,false,5,\N,5,50,5.5,50.5,03/01/09,5,2009-03-01 00:05:00.100 +56,true,6,6,\N,60,6.6,60.599999999999994,03/01/09,6,2009-03-01 00:06:00.150 +57,false,7,7,7,\N,7.7,70.7,03/01/09,7,2009-03-01 00:07:00.210 +58,true,8,8,8,80,\N,80.8,03/01/09,8,2009-03-01 00:08:00.280 +59,false,9,9,9,90,9.9,\N,03/01/09,9,2009-03-01 00:09:00.360 +60,true,0,0,0,0,0.0,0.0,\N,0,2009-03-02 00:10:00.450 +61,false,1,1,1,10,1.1,10.1,03/02/09,\N,2009-03-02 00:11:00.450 +62,true,2,2,2,20,2.2,20.2,03/02/09,2,\N +63,\N,3,3,3,30,3.3,30.299999999999997,03/02/09,3,2009-03-02 00:13:00.480 +64,true,\N,4,4,40,4.4,40.4,03/02/09,4,2009-03-02 00:14:00.510 +65,false,5,\N,5,50,5.5,50.5,03/02/09,5,2009-03-02 00:15:00.550 +66,true,6,6,\N,60,6.6,60.599999999999994,03/02/09,6,2009-03-02 00:16:00.600 +67,false,7,7,7,\N,7.7,70.7,03/02/09,7,2009-03-02 00:17:00.660 +68,true,8,8,8,80,\N,80.8,03/02/09,8,2009-03-02 00:18:00.730 +69,false,9,9,9,90,9.9,\N,03/02/09,9,2009-03-02 00:19:00.810 +70,true,0,0,0,0,0.0,0.0,\N,0,2009-03-03 00:20:00.900 +71,false,1,1,1,10,1.1,10.1,03/03/09,\N,2009-03-03 00:21:00.900 +72,true,2,2,2,20,2.2,20.2,03/03/09,2,\N +73,\N,3,3,3,30,3.3,30.299999999999997,03/03/09,3,2009-03-03 00:23:00.930 +74,true,\N,4,4,40,4.4,40.4,03/03/09,4,2009-03-03 00:24:00.960 diff --git a/src/test/hive/data/files/types/primitives/090401.txt b/src/test/hive/data/files/types/primitives/090401.txt new file mode 100644 index 0000000000000..a3a0d69a22a46 --- /dev/null +++ b/src/test/hive/data/files/types/primitives/090401.txt @@ -0,0 +1,25 @@ +75,false,0,\N,0,0,0.0,0.0,04/01/09,0,2009-04-01 00:00:00.0 +76,true,1,1,\N,10,1.1,10.1,04/01/09,1,2009-04-01 00:01:00.0 +77,false,2,2,2,\N,2.2,20.2,04/01/09,2,2009-04-01 00:02:00.10 +78,true,3,3,3,30,\N,30.299999999999997,04/01/09,3,2009-04-01 00:03:00.30 +79,false,4,4,4,40,4.4,\N,04/01/09,4,2009-04-01 00:04:00.60 +80,true,5,5,5,50,5.5,50.5,\N,5,2009-04-01 00:05:00.100 +81,false,6,6,6,60,6.6,60.599999999999994,04/01/09,\N,2009-04-01 00:06:00.150 +82,true,7,7,7,70,7.7,70.7,04/01/09,7,\N +83,\N,8,8,8,80,8.8,80.8,04/01/09,8,2009-04-01 00:08:00.280 +84,true,\N,9,9,90,9.9,90.89999999999999,04/01/09,9,2009-04-01 00:09:00.360 +85,false,0,\N,0,0,0.0,0.0,04/02/09,0,2009-04-02 00:10:00.450 +86,true,1,1,\N,10,1.1,10.1,04/02/09,1,2009-04-02 00:11:00.450 +87,false,2,2,2,\N,2.2,20.2,04/02/09,2,2009-04-02 00:12:00.460 +88,true,3,3,3,30,\N,30.299999999999997,04/02/09,3,2009-04-02 00:13:00.480 +89,false,4,4,4,40,4.4,\N,04/02/09,4,2009-04-02 00:14:00.510 +90,true,5,5,5,50,5.5,50.5,\N,5,2009-04-02 00:15:00.550 +91,false,6,6,6,60,6.6,60.599999999999994,04/02/09,\N,2009-04-02 00:16:00.600 +92,true,7,7,7,70,7.7,70.7,04/02/09,7,\N +93,\N,8,8,8,80,8.8,80.8,04/02/09,8,2009-04-02 00:18:00.730 +94,true,\N,9,9,90,9.9,90.89999999999999,04/02/09,9,2009-04-02 00:19:00.810 +95,false,0,\N,0,0,0.0,0.0,04/03/09,0,2009-04-03 00:20:00.900 +96,true,1,1,\N,10,1.1,10.1,04/03/09,1,2009-04-03 00:21:00.900 +97,false,2,2,2,\N,2.2,20.2,04/03/09,2,2009-04-03 00:22:00.910 +98,true,3,3,3,30,\N,30.299999999999997,04/03/09,3,2009-04-03 00:23:00.930 +99,false,4,4,4,40,4.4,\N,04/03/09,4,2009-04-03 00:24:00.960 diff --git a/src/test/hive/data/files/union_input.txt b/src/test/hive/data/files/union_input.txt new file mode 100644 index 0000000000000..e3020be086b86 --- /dev/null +++ b/src/test/hive/data/files/union_input.txt @@ -0,0 +1,8 @@ +011oneone +12.02twotwo +2threefour3threefour +35five5fivefive +2sixseven6sixseven +38eight8eighteight +099ninenine +110.010tenten diff --git a/src/test/hive/data/files/v1.txt b/src/test/hive/data/files/v1.txt new file mode 100644 index 0000000000000..1ce88332978db --- /dev/null +++ b/src/test/hive/data/files/v1.txt @@ -0,0 +1,4 @@ +1111fooabc2013-10-10 12:12:12xyzfunbar2013-10-10 12:12:12lmn2013-11-11 12:12:1292222 +2222fooabc2013-10-10 12:12:12xyzfunbar2013-10-10 12:12:12lmn2013-11-11 12:12:1294444 +3333fooabc2013-10-10 12:12:12xyzfunbar2013-10-10 12:12:12lmn2013-11-11 12:12:1296666 +4444fooabc2013-10-10 12:12:12xyzfunbar2013-10-10 12:12:12lmn2013-11-11 12:12:1298888 diff --git a/src/test/hive/data/files/v2.txt b/src/test/hive/data/files/v2.txt new file mode 100644 index 0000000000000..5e4c94c2e6079 --- /dev/null +++ b/src/test/hive/data/files/v2.txt @@ -0,0 +1,9 @@ +111199999 +222299999 +222299999 +444499999 +555599999 +666699999 +666699999 +666699999 +888899999 diff --git a/src/test/hive/data/files/vc1.txt b/src/test/hive/data/files/vc1.txt new file mode 100644 index 0000000000000..d0f99523dc2fd --- /dev/null +++ b/src/test/hive/data/files/vc1.txt @@ -0,0 +1,3 @@ +1abc +2abc +3 abc diff --git a/src/test/hive/data/files/x.txt b/src/test/hive/data/files/x.txt new file mode 100644 index 0000000000000..6407b72051b51 --- /dev/null +++ b/src/test/hive/data/files/x.txt @@ -0,0 +1,2 @@ +Joe 2 +Hank 2 diff --git a/src/test/hive/data/files/y.txt b/src/test/hive/data/files/y.txt new file mode 100644 index 0000000000000..7eedbcfae2066 --- /dev/null +++ b/src/test/hive/data/files/y.txt @@ -0,0 +1 @@ +2 Tie diff --git a/src/test/hive/data/files/z.txt b/src/test/hive/data/files/z.txt new file mode 100644 index 0000000000000..7eedbcfae2066 --- /dev/null +++ b/src/test/hive/data/files/z.txt @@ -0,0 +1 @@ +2 Tie diff --git a/src/test/hive/data/metadb/.gitignore b/src/test/hive/data/metadb/.gitignore new file mode 100644 index 0000000000000..0dd98905045cd --- /dev/null +++ b/src/test/hive/data/metadb/.gitignore @@ -0,0 +1 @@ +# Dummy file to make Git recognize this empty directory diff --git a/src/test/hive/data/scripts/cat.py b/src/test/hive/data/scripts/cat.py new file mode 100644 index 0000000000000..2395b2cdeb391 --- /dev/null +++ b/src/test/hive/data/scripts/cat.py @@ -0,0 +1,29 @@ +# +# 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, re +import datetime +import os + +table_name=None +if os.environ.has_key('hive_streaming_tablename'): + table_name=os.environ['hive_streaming_tablename'] + +for line in sys.stdin: + print line + print >> sys.stderr, "dummy" diff --git a/src/test/hive/data/scripts/cat_error.py b/src/test/hive/data/scripts/cat_error.py new file mode 100644 index 0000000000000..9642efec8ecb4 --- /dev/null +++ b/src/test/hive/data/scripts/cat_error.py @@ -0,0 +1,24 @@ +# +# 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 + +for line in sys.stdin: + print line + +sys.exit(1) diff --git a/src/test/hive/data/scripts/doubleescapedtab.py b/src/test/hive/data/scripts/doubleescapedtab.py new file mode 100644 index 0000000000000..d373067baed2c --- /dev/null +++ b/src/test/hive/data/scripts/doubleescapedtab.py @@ -0,0 +1,24 @@ +# +# 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 + +for line in sys.stdin: + print "1\\\\\\t2" + print "1\\\\\\\\t2" + diff --git a/src/test/hive/data/scripts/dumpdata_script.py b/src/test/hive/data/scripts/dumpdata_script.py new file mode 100644 index 0000000000000..c96c9e529bbb1 --- /dev/null +++ b/src/test/hive/data/scripts/dumpdata_script.py @@ -0,0 +1,27 @@ +# +# 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 + +for i in xrange(50): + for j in xrange(5): + for k in xrange(20022): + print 20000 * i + k + +for line in sys.stdin: + pass diff --git a/src/test/hive/data/scripts/error_script b/src/test/hive/data/scripts/error_script new file mode 100755 index 0000000000000..8d86b62f0f744 --- /dev/null +++ b/src/test/hive/data/scripts/error_script @@ -0,0 +1,26 @@ +#! /bin/bash +# 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. + +exit 1 +ret=0 +while [ "$ret" = "0" ]; +do + read -t 1 -a v + ret=$? +done + +exit 1 diff --git a/src/test/hive/data/scripts/escapedcarriagereturn.py b/src/test/hive/data/scripts/escapedcarriagereturn.py new file mode 100644 index 0000000000000..475928a2430f6 --- /dev/null +++ b/src/test/hive/data/scripts/escapedcarriagereturn.py @@ -0,0 +1,23 @@ +# +# 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 + +for line in sys.stdin: + print "1\\\\r2" + diff --git a/src/test/hive/data/scripts/escapednewline.py b/src/test/hive/data/scripts/escapednewline.py new file mode 100644 index 0000000000000..0d5751454bed7 --- /dev/null +++ b/src/test/hive/data/scripts/escapednewline.py @@ -0,0 +1,23 @@ +# +# 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 + +for line in sys.stdin: + print "1\\\\n2" + diff --git a/src/test/hive/data/scripts/escapedtab.py b/src/test/hive/data/scripts/escapedtab.py new file mode 100644 index 0000000000000..549c91e444632 --- /dev/null +++ b/src/test/hive/data/scripts/escapedtab.py @@ -0,0 +1,23 @@ +# +# 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 + +for line in sys.stdin: + print "1\\\\t2" + diff --git a/src/test/hive/data/scripts/input20_script b/src/test/hive/data/scripts/input20_script new file mode 100755 index 0000000000000..e8e41189c17cf --- /dev/null +++ b/src/test/hive/data/scripts/input20_script @@ -0,0 +1,20 @@ +#! /bin/bash +# 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. + +# This script outputs a row of the following format +# <# of values for the key> _ +uniq -c | sed "s@^ *@@" | sed "s@ @_@" | sed "s@ @ @" diff --git a/src/test/hive/data/scripts/newline.py b/src/test/hive/data/scripts/newline.py new file mode 100644 index 0000000000000..6500d900dd8ab --- /dev/null +++ b/src/test/hive/data/scripts/newline.py @@ -0,0 +1,24 @@ +# +# 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 + +for line in sys.stdin: + print "1\\n2" + print "1\\r2" + print "1\\t2" diff --git a/src/test/hive/data/scripts/q_test_cleanup.sql b/src/test/hive/data/scripts/q_test_cleanup.sql new file mode 100644 index 0000000000000..31bd7205d8591 --- /dev/null +++ b/src/test/hive/data/scripts/q_test_cleanup.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS src; +DROP TABLE IF EXISTS src1; +DROP TABLE IF EXISTS src_json; +DROP TABLE IF EXISTS src_sequencefile; +DROP TABLE IF EXISTS src_thrift; +DROP TABLE IF EXISTS srcbucket; +DROP TABLE IF EXISTS srcbucket2; +DROP TABLE IF EXISTS srcpart; +DROP TABLE IF EXISTS primitives; + diff --git a/src/test/hive/data/scripts/q_test_init.sql b/src/test/hive/data/scripts/q_test_init.sql new file mode 100644 index 0000000000000..12afdf391132e --- /dev/null +++ b/src/test/hive/data/scripts/q_test_init.sql @@ -0,0 +1,132 @@ +-- +-- Table src +-- +DROP TABLE IF EXISTS src; + +CREATE TABLE src (key STRING, value STRING) STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" INTO TABLE src; + +-- +-- Table src1 +-- +DROP TABLE IF EXISTS src1; + +CREATE TABLE src1 (key STRING, value STRING) STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv3.txt" INTO TABLE src1; + +-- +-- Table src_json +-- +DROP TABLE IF EXISTS src_json; + +CREATE TABLE src_json (json STRING) STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/json.txt" INTO TABLE src_json; + + +-- +-- Table src_sequencefile +-- +DROP TABLE IF EXISTS src_sequencefile; + +CREATE TABLE src_sequencefile (key STRING, value STRING) STORED AS SEQUENCEFILE; + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.seq" INTO TABLE src_sequencefile; + + +-- +-- Table src_thrift +-- +DROP TABLE IF EXISTS src_thrift; + +CREATE TABLE src_thrift +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer' +WITH SERDEPROPERTIES ( + 'serialization.class' = 'org.apache.hadoop.hive.serde2.thrift.test.Complex', + 'serialization.format' = 'com.facebook.thrift.protocol.TBinaryProtocol') +STORED AS SEQUENCEFILE; + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/complex.seq" INTO TABLE src_thrift; + + +-- +-- Table srcbucket +-- +DROP TABLE IF EXISTS srcbucket; + +CREATE TABLE srcbucket (key INT, value STRING) +CLUSTERED BY (key) INTO 2 BUCKETS +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket0.txt" INTO TABLE srcbucket; +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket1.txt" INTO TABLE srcbucket; + + +-- +-- Table srcbucket2 +-- +DROP TABLE IF EXISTS srcbucket2; + +CREATE TABLE srcbucket2 (key INT, value STRING) +CLUSTERED BY (key) INTO 4 BUCKETS +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket20.txt" INTO TABLE srcbucket2; +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket21.txt" INTO TABLE srcbucket2; + + +-- +-- Table srcpart +-- +DROP TABLE IF EXISTS srcpart; + +CREATE TABLE srcpart (key STRING, value STRING) +PARTITIONED BY (ds STRING, hr STRING) +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" +OVERWRITE INTO TABLE srcpart PARTITION (ds="2008-04-08", hr="11"); + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" +OVERWRITE INTO TABLE srcpart PARTITION (ds="2008-04-08", hr="12"); + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" +OVERWRITE INTO TABLE srcpart PARTITION (ds="2008-04-09", hr="11"); + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" +OVERWRITE INTO TABLE srcpart PARTITION (ds="2008-04-09", hr="12"); + + +DROP TABLE IF EXISTS primitives; +CREATE TABLE primitives ( + id INT, + bool_col BOOLEAN, + tinyint_col TINYINT, + smallint_col SMALLINT, + int_col INT, + bigint_col BIGINT, + float_col FLOAT, + double_col DOUBLE, + date_string_col STRING, + string_col STRING, + timestamp_col TIMESTAMP) +PARTITIONED BY (year INT, month INT) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + ESCAPED BY '\\' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/types/primitives/090101.txt" +OVERWRITE INTO TABLE primitives PARTITION(year=2009, month=1); + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/types/primitives/090201.txt" +OVERWRITE INTO TABLE primitives PARTITION(year=2009, month=2); + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/types/primitives/090301.txt" +OVERWRITE INTO TABLE primitives PARTITION(year=2009, month=3); + +LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/types/primitives/090401.txt" +OVERWRITE INTO TABLE primitives PARTITION(year=2009, month=4); + diff --git a/src/test/hive/data/scripts/test_init_file.sql b/src/test/hive/data/scripts/test_init_file.sql new file mode 100644 index 0000000000000..776a46be084c8 --- /dev/null +++ b/src/test/hive/data/scripts/test_init_file.sql @@ -0,0 +1 @@ +create table tbl_created_by_init(i int); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q b/src/test/hive/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q new file mode 100644 index 0000000000000..8f0a60b713ab8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q @@ -0,0 +1,8 @@ +SET hive.metastore.partition.name.whitelist.pattern=[\\x20-\\x7E&&[^,]]* ; +-- This pattern matches all printable ASCII characters (disallow unicode) and disallows commas + +CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS part_whitelist_test; + +ALTER TABLE part_whitelist_test ADD PARTITION (ds='1,2,3,4'); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/addpart1.q b/src/test/hive/ql/src/test/queries/clientnegative/addpart1.q new file mode 100644 index 0000000000000..a7c9fe91f6cd9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/addpart1.q @@ -0,0 +1,11 @@ + +create table addpart1 (a int) partitioned by (b string, c string); + +alter table addpart1 add partition (b='f', c='s'); + +show partitions addpart1; + +alter table addpart1 add partition (b='f', c=''); + +show prtitions addpart1; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q new file mode 100644 index 0000000000000..4881757a4613a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q @@ -0,0 +1,16 @@ +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; + +show table extended like `src_rc_concatenate_test`; + +select count(1) from src_rc_concatenate_test; +select sum(hash(key)), sum(hash(value)) from src_rc_concatenate_test; + +create index src_rc_concatenate_test_index on table src_rc_concatenate_test(key) as 'compact' WITH DEFERRED REBUILD IDXPROPERTIES ("prop1"="val1", "prop2"="val2"); +show indexes on src_rc_concatenate_test; + +alter table src_rc_concatenate_test concatenate; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_non_native.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_non_native.q new file mode 100644 index 0000000000000..73ae853778832 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_non_native.q @@ -0,0 +1,6 @@ + +CREATE TABLE non_native1(key int, value string) +STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; + +-- we do not support ALTER TABLE on non-native tables yet +ALTER TABLE non_native1 RENAME TO new_non_native; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q new file mode 100644 index 0000000000000..e10f77cf3f161 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q @@ -0,0 +1,11 @@ +-- create testing table +create table alter_coltype(key string, value string) partitioned by (dt string, ts string); + +-- insert and create a partition +insert overwrite table alter_coltype partition(dt='100x', ts='6:30pm') select * from src1; + +desc alter_coltype; + +-- alter partition change multiple keys at same time +alter table alter_coltype partition column (dt int, ts int); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q new file mode 100644 index 0000000000000..66eba75d4084d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q @@ -0,0 +1,12 @@ +-- create testing table +create table alter_coltype(key string, value string) partitioned by (dt string, ts string); + +-- insert and create a partition +insert overwrite table alter_coltype partition(dt='100x', ts='6:30pm') select * from src1; + +desc alter_coltype; + +-- alter partition key column with invalid column name +alter table alter_coltype partition column (dd int); + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q new file mode 100644 index 0000000000000..ad016c5f3a76e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q @@ -0,0 +1,11 @@ +-- create testing table +create table alter_coltype(key string, value string) partitioned by (dt string, ts string); + +-- insert and create a partition +insert overwrite table alter_coltype partition(dt='100x', ts='6:30pm') select * from src1; + +desc alter_coltype; + +-- alter partition key column data type for ts column to a wrong type +alter table alter_coltype partition column (ts time); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q new file mode 100644 index 0000000000000..5f9d5ef9ca09e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q @@ -0,0 +1,8 @@ +-- Create table +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'); + +alter table alter_part_invalidspec partition (year='1997') enable no_drop; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop.q new file mode 100644 index 0000000000000..92af30b6b5a15 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop.q @@ -0,0 +1,9 @@ +-- Create table +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'); + +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/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q new file mode 100644 index 0000000000000..135411fd320ef --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q @@ -0,0 +1,9 @@ +-- Create table +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'); + +alter table alter_part_nodrop_table partition (year='1996') enable no_drop; +drop table alter_part_nodrop_table; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_offline.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_offline.q new file mode 100644 index 0000000000000..899145deaf8c5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_offline.q @@ -0,0 +1,11 @@ +-- create table +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'); + +alter table alter_part_offline partition (year='1996') disable offline; +select * from alter_part_offline where year = '1996'; +alter table alter_part_offline partition (year='1996') enable offline; +select * from alter_part_offline where year = '1996'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q new file mode 100644 index 0000000000000..6e33bc0782d29 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q @@ -0,0 +1,9 @@ +SET hive.metastore.partition.name.whitelist.pattern=[\\x20-\\x7E&&[^,]]* ; +-- This pattern matches all printable ASCII characters (disallow unicode) and disallows commas + +CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS part_whitelist_test; + +ALTER TABLE part_whitelist_test ADD PARTITION (ds='1'); + +ALTER TABLE part_whitelist_test PARTITION (ds='1') rename to partition (ds='1,2,3'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q new file mode 100644 index 0000000000000..26ba2878903fe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q @@ -0,0 +1,6 @@ +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 ; +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 ; + +alter table alter_rename_partition partition (pCol1='nonexist_part1:', pcol2='nonexist_part2:') rename to partition (pCol1='new_part1:', pcol2='new_part2:'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q new file mode 100644 index 0000000000000..6e51c2f7629c9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q @@ -0,0 +1,6 @@ +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 ; +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 ; + +alter table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') rename to partition (pCol1='old_part1:', pcol2='old_part2:'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q new file mode 100644 index 0000000000000..2d4ce0b9f6051 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q @@ -0,0 +1,6 @@ +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 ; +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 ; + +alter table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') rename to partition (pCol1='old_part1:', pcol2='old_part2:', pcol3='old_part3:'); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_table_add_partition.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_table_add_partition.q new file mode 100644 index 0000000000000..2427c3b2a45f8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_table_add_partition.q @@ -0,0 +1,5 @@ +create table mp (a int) partitioned by (b int); + +-- should fail +alter table mp add partition (b='1', c='1'); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q new file mode 100644 index 0000000000000..fad194d016ec9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q @@ -0,0 +1,7 @@ +drop table aa; +create table aa ( test STRING ) + ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.RegexSerDe' + WITH SERDEPROPERTIES ("input.regex" = "(.*)", "output.format.string" = "$1s"); + +alter table aa set serdeproperties ("input.regex" = "[^\\](.*)", "output.format.string" = "$1s"); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q new file mode 100644 index 0000000000000..30fe4d9916ab3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q @@ -0,0 +1,4 @@ +DROP VIEW testView; + +-- Cannot ALTER VIEW AS SELECT if view currently does not exist +ALTER VIEW testView AS SELECT * FROM srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q new file mode 100644 index 0000000000000..dca6770b1b17e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q @@ -0,0 +1,12 @@ +CREATE VIEW testViewPart PARTITIONED ON (value) +AS +SELECT key, value +FROM src +WHERE key=86; + +ALTER VIEW testViewPart +ADD PARTITION (value='val_86') PARTITION (value='val_xyz'); +DESCRIBE FORMATTED testViewPart; + +-- If a view has partition, could not replace it with ALTER VIEW AS SELECT +ALTER VIEW testViewPart as SELECT * FROM srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure.q new file mode 100644 index 0000000000000..705b985095fa3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure.q @@ -0,0 +1,3 @@ +DROP VIEW xxx3; +CREATE VIEW xxx3 AS SELECT * FROM src; +ALTER TABLE xxx3 REPLACE COLUMNS (xyz int); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure2.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure2.q new file mode 100644 index 0000000000000..26d2c4f3ad2fb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure2.q @@ -0,0 +1,8 @@ +DROP VIEW xxx4; +CREATE VIEW xxx4 +PARTITIONED ON (value) +AS +SELECT * FROM src; + +-- should fail: need to use ALTER VIEW, not ALTER TABLE +ALTER TABLE xxx4 ADD PARTITION (value='val_86'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure3.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure3.q new file mode 100644 index 0000000000000..49c17a8b573c9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure3.q @@ -0,0 +1,2 @@ +-- should fail: can't use ALTER VIEW on a table +ALTER VIEW srcpart ADD PARTITION (ds='2012-12-31', hr='23'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure4.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure4.q new file mode 100644 index 0000000000000..e2fad270b1d89 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure4.q @@ -0,0 +1,8 @@ +DROP VIEW xxx5; +CREATE VIEW xxx5 +PARTITIONED ON (value) +AS +SELECT * FROM src; + +-- should fail: LOCATION clause is illegal +ALTER VIEW xxx5 ADD PARTITION (value='val_86') LOCATION '/foo/bar/baz'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure5.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure5.q new file mode 100644 index 0000000000000..e44766e113063 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure5.q @@ -0,0 +1,8 @@ +DROP VIEW xxx6; +CREATE VIEW xxx6 +PARTITIONED ON (value) +AS +SELECT * FROM src; + +-- should fail: partition column name does not match +ALTER VIEW xxx6 ADD PARTITION (v='val_86'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure6.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure6.q new file mode 100644 index 0000000000000..dab7b145f7c4e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure6.q @@ -0,0 +1,11 @@ +DROP VIEW xxx7; +CREATE VIEW xxx7 +PARTITIONED ON (key) +AS +SELECT hr,key FROM srcpart; + +SET hive.mapred.mode=strict; + +-- strict mode should cause this to fail since view partition +-- predicate does not correspond to an underlying table partition predicate +ALTER VIEW xxx7 ADD PARTITION (key=10); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure7.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure7.q new file mode 100644 index 0000000000000..eff04c5b47dee --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure7.q @@ -0,0 +1,8 @@ +DROP VIEW xxx8; +CREATE VIEW xxx8 +PARTITIONED ON (ds,hr) +AS +SELECT key,ds,hr FROM srcpart; + +-- should fail: need to fill in all partition columns +ALTER VIEW xxx8 ADD PARTITION (ds='2011-01-01'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure8.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure8.q new file mode 100644 index 0000000000000..9dff784250616 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure8.q @@ -0,0 +1,3 @@ +-- should fail: can't use ALTER VIEW on a table +CREATE TABLE invites (foo INT, bar STRING) PARTITIONED BY (ds STRING); +ALTER VIEW invites RENAME TO invites2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure9.q b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure9.q new file mode 100644 index 0000000000000..0f40fad90d974 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure9.q @@ -0,0 +1,7 @@ +DROP VIEW xxx4; +CREATE VIEW xxx4 +AS +SELECT * FROM src; + +-- should fail: need to use ALTER VIEW, not ALTER TABLE +ALTER TABLE xxx4 RENAME TO xxx4a; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/altern1.q b/src/test/hive/ql/src/test/queries/clientnegative/altern1.q new file mode 100644 index 0000000000000..60414c1f3a7a5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/altern1.q @@ -0,0 +1,4 @@ + +create table altern1(a int, b int) partitioned by (ds string); +alter table altern1 replace columns(a int, b int, ds string); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col.q b/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col.q new file mode 100644 index 0000000000000..866cec126f782 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col.q @@ -0,0 +1 @@ +FROM (SELECT key, concat(value) AS key FROM src) a SELECT a.key; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col0.q b/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col0.q new file mode 100644 index 0000000000000..46349c60bc79b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col0.q @@ -0,0 +1,2 @@ +-- TOK_ALLCOLREF +explain select * from (select * from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col1.q b/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col1.q new file mode 100644 index 0000000000000..fdf20f850e8f8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col1.q @@ -0,0 +1,2 @@ +-- 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/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col2.q b/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col2.q new file mode 100644 index 0000000000000..de59bc579a0fe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col2.q @@ -0,0 +1,2 @@ +-- 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/src/test/hive/ql/src/test/queries/clientnegative/analyze.q b/src/test/hive/ql/src/test/queries/clientnegative/analyze.q new file mode 100644 index 0000000000000..874f5bfc14125 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/analyze.q @@ -0,0 +1 @@ +analyze table srcpart compute statistics; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/analyze1.q b/src/test/hive/ql/src/test/queries/clientnegative/analyze1.q new file mode 100644 index 0000000000000..057a1a0b482ea --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/analyze1.q @@ -0,0 +1 @@ +analyze table srcpart partition (key) compute statistics; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/analyze_view.q b/src/test/hive/ql/src/test/queries/clientnegative/analyze_view.q new file mode 100644 index 0000000000000..af4970f52e8be --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/analyze_view.q @@ -0,0 +1,6 @@ +DROP VIEW av; + +CREATE VIEW av AS SELECT * FROM src; + +-- should fail: can't analyze a view...yet +ANALYZE TABLE av COMPUTE STATISTICS; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive1.q b/src/test/hive/ql/src/test/queries/clientnegative/archive1.q new file mode 100644 index 0000000000000..a4b50f5e14107 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive1.q @@ -0,0 +1,11 @@ +set hive.archive.enabled = true; +-- Tests trying to archive a partition twice. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE srcpart_archived LIKE srcpart; + +INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; + +ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', hr='12'); +ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive2.q b/src/test/hive/ql/src/test/queries/clientnegative/archive2.q new file mode 100644 index 0000000000000..ff8dcb248568c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive2.q @@ -0,0 +1,10 @@ +set hive.archive.enabled = true; +-- Tests trying to unarchive a non-archived partition +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +drop table tstsrcpart; +create table tstsrcpart like srcpart; +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12') +select key, value from srcpart where ds='2008-04-08' and hr='12'; + +ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive3.q b/src/test/hive/ql/src/test/queries/clientnegative/archive3.q new file mode 100644 index 0000000000000..53057daa1b622 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive3.q @@ -0,0 +1,5 @@ +set hive.archive.enabled = true; +-- Tests archiving a table +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +ALTER TABLE srcpart ARCHIVE; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive4.q b/src/test/hive/ql/src/test/queries/clientnegative/archive4.q new file mode 100644 index 0000000000000..56d6f1798debe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive4.q @@ -0,0 +1,5 @@ +set hive.archive.enabled = true; +-- Tests archiving multiple partitions +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +ALTER TABLE srcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12') PARTITION (ds='2008-04-08', hr='11'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive5.q b/src/test/hive/ql/src/test/queries/clientnegative/archive5.q new file mode 100644 index 0000000000000..4f6dc8d72ceed --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive5.q @@ -0,0 +1,5 @@ +set hive.archive.enabled = true; +-- Tests creating a partition where the partition value will collide with the +-- a intermediate directory + +ALTER TABLE srcpart ADD PARTITION (ds='2008-04-08', hr='14_INTERMEDIATE_ORIGINAL') diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_corrupt.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_corrupt.q new file mode 100644 index 0000000000000..bea25391628a6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_corrupt.q @@ -0,0 +1,18 @@ +USE default; + +set hive.archive.enabled = true; +set hive.enforce.bucketing = true; + +drop table tstsrcpart; + +create table tstsrcpart like srcpart; + +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20) +-- The version of GzipCodec that is provided in Hadoop 0.20 silently ignores +-- file format errors. However, versions of Hadoop that include +-- HADOOP-6835 (e.g. 0.23 and 1.x) cause a Wrong File Format exception +-- 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'); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_insert1.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_insert1.q new file mode 100644 index 0000000000000..deaff63d673a7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_insert1.q @@ -0,0 +1,13 @@ +set hive.archive.enabled = true; +-- Tests trying to insert into archived partition. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE tstsrcpart LIKE srcpart; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); + +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/src/test/hive/ql/src/test/queries/clientnegative/archive_insert2.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_insert2.q new file mode 100644 index 0000000000000..d744f2487694c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_insert2.q @@ -0,0 +1,13 @@ +set hive.archive.enabled = true; +-- Tests trying to insert into archived partition. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE tstsrcpart LIKE srcpart; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); + +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/src/test/hive/ql/src/test/queries/clientnegative/archive_insert3.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_insert3.q new file mode 100644 index 0000000000000..c6cb142824c8d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_insert3.q @@ -0,0 +1,13 @@ +set hive.archive.enabled = true; +-- Tests trying to create partition inside of archived directory. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE tstsrcpart LIKE srcpart; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_insert4.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_insert4.q new file mode 100644 index 0000000000000..c36f3ef9e8775 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_insert4.q @@ -0,0 +1,15 @@ +set hive.archive.enabled = true; +-- Tests trying to (possible) dynamic insert into archived partition. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE tstsrcpart LIKE srcpart; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); + +SET hive.exec.dynamic.partition=true; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr) +SELECT key, value, hr FROM srcpart WHERE ds='2008-04-08' AND hr='12'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi1.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi1.q new file mode 100644 index 0000000000000..8c702ed008bfa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi1.q @@ -0,0 +1,13 @@ +set hive.archive.enabled = true; +-- Tests trying to archive a partition twice. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE tstsrcpart LIKE srcpart; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND 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'; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi2.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi2.q new file mode 100644 index 0000000000000..d3cfb89c98749 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi2.q @@ -0,0 +1,12 @@ +set hive.archive.enabled = true; +-- Tests trying to unarchive a non-archived partition group +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +drop table tstsrcpart; +create table tstsrcpart like srcpart; +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND 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'; + +ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi3.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi3.q new file mode 100644 index 0000000000000..75f5dfad47b37 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi3.q @@ -0,0 +1,13 @@ +set hive.archive.enabled = true; +-- Tests trying to archive outer partition group containing other partition inside. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE tstsrcpart LIKE srcpart; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND 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'; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi4.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi4.q new file mode 100644 index 0000000000000..abe0647ae6ee1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi4.q @@ -0,0 +1,13 @@ +set hive.archive.enabled = true; +-- Tests trying to archive inner partition contained in archived partition group. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE tstsrcpart LIKE srcpart; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND 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'; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi5.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi5.q new file mode 100644 index 0000000000000..71635e054a1e9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi5.q @@ -0,0 +1,13 @@ +set hive.archive.enabled = true; +-- Tests trying to unarchive outer partition group containing other partition inside. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE tstsrcpart LIKE srcpart; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND 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'; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); +ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi6.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi6.q new file mode 100644 index 0000000000000..5bb1474fdc383 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi6.q @@ -0,0 +1,13 @@ +set hive.archive.enabled = true; +-- Tests trying to unarchive inner partition contained in archived partition group. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE tstsrcpart LIKE srcpart; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND 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'; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); +ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi7.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi7.q new file mode 100644 index 0000000000000..db7f392737e97 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_multi7.q @@ -0,0 +1,12 @@ +set hive.archive.enabled = true; +-- Tests trying to archive a partition group with custom locations. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE tstsrcpart LIKE srcpart; + +INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; +ALTER TABLE tstsrcpart ADD PARTITION (ds='2008-04-08', hr='12') +LOCATION "${system:test.tmp.dir}/tstsrc"; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec1.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec1.q new file mode 100644 index 0000000000000..d83b19d9fe31a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec1.q @@ -0,0 +1,10 @@ +set hive.archive.enabled = true; +-- Tests trying to archive a partition twice. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE srcpart_archived LIKE srcpart; + +INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; + +ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', nonexistingpart='12'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec2.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec2.q new file mode 100644 index 0000000000000..ed14bbf688d52 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec2.q @@ -0,0 +1,10 @@ +set hive.archive.enabled = true; +-- Tests trying to archive a partition twice. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE srcpart_archived LIKE srcpart; + +INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; + +ALTER TABLE srcpart_archived ARCHIVE PARTITION (hr='12'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec3.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec3.q new file mode 100644 index 0000000000000..f27ad6d63b08e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec3.q @@ -0,0 +1,10 @@ +set hive.archive.enabled = true; +-- Tests trying to archive a partition twice. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE srcpart_archived LIKE srcpart; + +INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; + +ALTER TABLE srcpart_archived ARCHIVE PARTITION (); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec4.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec4.q new file mode 100644 index 0000000000000..491c2ac4596fc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec4.q @@ -0,0 +1,10 @@ +set hive.archive.enabled = true; +-- Tests trying to archive a partition twice. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE srcpart_archived LIKE srcpart; + +INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; + +ALTER TABLE srcpart_archived ARCHIVE PARTITION (hr='12', ds='2008-04-08'); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec5.q b/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec5.q new file mode 100644 index 0000000000000..bb25ef2c7e0f1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec5.q @@ -0,0 +1,10 @@ +set hive.archive.enabled = true; +-- Tests trying to archive a partition twice. +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +CREATE TABLE srcpart_archived (key string, value string) partitioned by (ds string, hr int, min int); + +INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12', min='00') +SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; + +ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', min='00'); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_1.q b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_1.q new file mode 100644 index 0000000000000..c38dab5eb702f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_1.q @@ -0,0 +1,7 @@ +create table authorization_fail_1 (key int, value string); +set hive.security.authorization.enabled=true; + +grant Create on table authorization_fail_1 to user hive_test_user; +grant Create on table authorization_fail_1 to user hive_test_user; + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_2.q b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_2.q new file mode 100644 index 0000000000000..341e44774d9c2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_2.q @@ -0,0 +1,7 @@ +create table authorization_fail_2 (key int, value string) partitioned by (ds string); + +set hive.security.authorization.enabled=true; + +alter table authorization_fail_2 add partition (ds='2010'); + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_3.q b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_3.q new file mode 100644 index 0000000000000..6a56daa05fee8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_3.q @@ -0,0 +1,12 @@ +-- SORT_BEFORE_DIFF + +create table authorization_fail_3 (key int, value string) partitioned by (ds string); +set hive.security.authorization.enabled=true; + +grant Create on table authorization_fail_3 to user hive_test_user; +alter table authorization_fail_3 add partition (ds='2010'); + +show grant user hive_test_user on table authorization_fail_3; +show grant user hive_test_user on table authorization_fail_3 partition (ds='2010'); + +select key from authorization_fail_3 where ds='2010'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_4.q b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_4.q new file mode 100644 index 0000000000000..f0cb6459a255b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_4.q @@ -0,0 +1,15 @@ +-- SORT_BEFORE_DIFF + +create table authorization_fail_4 (key int, value string) partitioned by (ds string); + +set hive.security.authorization.enabled=true; +grant Alter on table authorization_fail_4 to user hive_test_user; +ALTER TABLE authorization_fail_4 SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); + +grant Create on table authorization_fail_4 to user hive_test_user; +alter table authorization_fail_4 add partition (ds='2010'); + +show grant user hive_test_user on table authorization_fail_4; +show grant user hive_test_user on table authorization_fail_4 partition (ds='2010'); + +select key from authorization_fail_4 where ds='2010'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_5.q b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_5.q new file mode 100644 index 0000000000000..b4efab5667f6c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_5.q @@ -0,0 +1,20 @@ +-- SORT_BEFORE_DIFF + +create table authorization_fail (key int, value string) partitioned by (ds string); +set hive.security.authorization.enabled=true; + +grant Alter on table authorization_fail to user hive_test_user; +ALTER TABLE authorization_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); + +grant Create on table authorization_fail to user hive_test_user; +grant Select on table authorization_fail to user hive_test_user; +alter table authorization_fail add partition (ds='2010'); + +show grant user hive_test_user on table authorization_fail; +show grant user hive_test_user on table authorization_fail partition (ds='2010'); + +revoke Select on table authorization_fail partition (ds='2010') from user hive_test_user; + +show grant user hive_test_user on table authorization_fail partition (ds='2010'); + +select key from authorization_fail where ds='2010'; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_6.q b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_6.q new file mode 100644 index 0000000000000..977246948cad0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_6.q @@ -0,0 +1,6 @@ +-- SORT_BEFORE_DIFF + +create table authorization_part_fail (key int, value string) partitioned by (ds string); +set hive.security.authorization.enabled=true; + +ALTER TABLE authorization_part_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_7.q b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_7.q new file mode 100644 index 0000000000000..492deed10bfeb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_7.q @@ -0,0 +1,17 @@ +-- SORT_BEFORE_DIFF + +create table authorization_fail (key int, value string); + +set hive.security.authorization.enabled=true; + +create role hive_test_role_fail; + +grant role hive_test_role_fail to user hive_test_user; +grant select on table authorization_fail to role hive_test_role_fail; +show role grant user hive_test_user; + +show grant role hive_test_role_fail on table authorization_fail; + +drop role hive_test_role_fail; + +select key from authorization_fail; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_part.q b/src/test/hive/ql/src/test/queries/clientnegative/authorization_part.q new file mode 100644 index 0000000000000..a654a2380c75c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/authorization_part.q @@ -0,0 +1,37 @@ +-- SORT_BEFORE_DIFF + +create table authorization_part_fail (key int, value string) partitioned by (ds string); +ALTER TABLE authorization_part_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); +create table src_auth as select * from src; +set hive.security.authorization.enabled=true; + +grant Create on table authorization_part_fail to user hive_test_user; +grant Update on table authorization_part_fail to user hive_test_user; +grant Drop on table authorization_part_fail to user hive_test_user; +grant select on table src_auth to user hive_test_user; + +-- column grant to group + +grant select(key) on table authorization_part_fail to group hive_test_group1; +grant select on table authorization_part_fail to group hive_test_group1; + +show grant group hive_test_group1 on table authorization_part_fail; + +insert overwrite table authorization_part_fail partition (ds='2010') select key, value from src_auth; +show grant group hive_test_group1 on table authorization_part_fail(key) partition (ds='2010'); +show grant group hive_test_group1 on table authorization_part_fail partition (ds='2010'); +select key, value from authorization_part_fail where ds='2010' order by key limit 20; + +insert overwrite table authorization_part_fail partition (ds='2011') select key, value from src_auth; +show grant group hive_test_group1 on table authorization_part_fail(key) partition (ds='2011'); +show grant group hive_test_group1 on table authorization_part_fail partition (ds='2011'); +select key, value from authorization_part_fail where ds='2011' order by key limit 20; + +select key,value, ds from authorization_part_fail where ds>='2010' order by key, ds limit 20; + +revoke select on table authorization_part_fail partition (ds='2010') from group hive_test_group1; + +select key,value, ds from authorization_part_fail where ds>='2010' order by key, ds limit 20; + +drop table authorization_part_fail; +drop table src_auth; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/autolocal1.q b/src/test/hive/ql/src/test/queries/clientnegative/autolocal1.q new file mode 100644 index 0000000000000..bd1c9d6e15a76 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/autolocal1.q @@ -0,0 +1,15 @@ +set mapred.job.tracker=abracadabra; +set hive.exec.mode.local.auto.inputbytes.max=1; +set hive.exec.mode.local.auto=true; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20) +-- hadoop0.23 changes the behavior of JobClient initialization +-- in hadoop0.20, JobClient initialization tries to get JobTracker's address +-- this throws the expected IllegalArgumentException +-- in hadoop0.23, JobClient initialization only initializes cluster +-- and get user group information +-- not attempts to get JobTracker's address +-- no IllegalArgumentException thrown in JobClient Initialization +-- an exception is thrown when JobClient submitJob + +SELECT key FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bad_exec_hooks.q b/src/test/hive/ql/src/test/queries/clientnegative/bad_exec_hooks.q new file mode 100644 index 0000000000000..709d8d9c85442 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/bad_exec_hooks.q @@ -0,0 +1,6 @@ +set hive.exec.pre.hooks="org.this.is.a.bad.class"; + +EXPLAIN +SELECT x.* FROM SRC x LIMIT 20; + +SELECT x.* FROM SRC x LIMIT 20; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bad_indextype.q b/src/test/hive/ql/src/test/queries/clientnegative/bad_indextype.q new file mode 100644 index 0000000000000..8f5bf42664b96 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/bad_indextype.q @@ -0,0 +1 @@ +CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) AS 'UNKNOWN' WITH DEFERRED REBUILD; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bad_sample_clause.q b/src/test/hive/ql/src/test/queries/clientnegative/bad_sample_clause.q new file mode 100644 index 0000000000000..fd6769827b829 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/bad_sample_clause.q @@ -0,0 +1,6 @@ +CREATE TABLE dest1(key INT, value STRING, dt STRING, hr STRING) STORED AS TEXTFILE; + +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 2) s +WHERE s.ds='2008-04-08' and s.hr='11'; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q b/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q new file mode 100644 index 0000000000000..048a8fd5cfb8e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q @@ -0,0 +1,42 @@ +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'); + +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'); + +-- The number of buckets in the 2 tables above (being joined later) dont match. +-- Throw an error if the user requested a bucketed mapjoin to be enforced. +-- In the default case (hive.enforce.bucketmapjoin=false), the query succeeds +-- even though mapjoin is not being performed + +explain +select a.key, a.value, b.value +from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b +on a.key=b.key and a.ds="2008-04-08" and b.ds="2008-04-08"; + +set hive.optimize.bucketmapjoin = true; + +explain +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b +on a.key=b.key and a.ds="2008-04-08" and b.ds="2008-04-08"; + +set hive.enforce.bucketmapjoin=true; + +explain +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b +on a.key=b.key and a.ds="2008-04-08" and b.ds="2008-04-08"; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q b/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q new file mode 100644 index 0000000000000..9478a2f1b989c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q @@ -0,0 +1,20 @@ +-- Although the user has specified a bucketed map-join, the number of buckets in the table +-- do not match the number of files +drop table table1; +drop table table2; + +create table table1(key string, value string) clustered by (key, value) +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 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; + +select /*+ mapjoin(b) */ count(*) from table1 a join table2 b on a.key=b.key and a.value=b.value; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q b/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q new file mode 100644 index 0000000000000..69afe0ae700ed --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q @@ -0,0 +1,24 @@ +-- Although the user has specified a bucketed map-join, the number of buckets in the table +-- do not match the number of files +drop table table1; +drop table table2; + +create table table1(key string, value string) partitioned by (ds string) clustered by (key, value) +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='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; + +set hive.optimize.bucketmapjoin = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +select /*+ mapjoin(b) */ count(*) from table1 a join table2 b +on a.key=b.key and a.value=b.value and a.ds is not null; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/cachingprintstream.q b/src/test/hive/ql/src/test/queries/clientnegative/cachingprintstream.q new file mode 100644 index 0000000000000..d57a4517f00fb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/cachingprintstream.q @@ -0,0 +1,8 @@ +set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifyCachingPrintStreamHook; +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyCachingPrintStreamHook; + +SELECT count(*) FROM src; +FROM src SELECT TRANSFORM (key, value) USING 'FAKE_SCRIPT_SHOULD_NOT_EXIST' AS key, value; + +set hive.exec.failure.hooks=; +set hive.exec.post.hooks=; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q b/src/test/hive/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q new file mode 100644 index 0000000000000..bc980448a9e22 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q @@ -0,0 +1,6 @@ +-- TaskLog retrieval upon Null Pointer Exception in Cluster + +CREATE TEMPORARY FUNCTION evaluate_npe AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFEvaluateNPE'; + +FROM src +SELECT evaluate_npe(src.key) LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clusterbydistributeby.q b/src/test/hive/ql/src/test/queries/clientnegative/clusterbydistributeby.q new file mode 100644 index 0000000000000..4c6a9b38d785d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/clusterbydistributeby.q @@ -0,0 +1,8 @@ +CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +CLUSTER BY tvalue, tkey +DISTRIBUTE BY tvalue, tkey; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clusterbyorderby.q b/src/test/hive/ql/src/test/queries/clientnegative/clusterbyorderby.q new file mode 100644 index 0000000000000..d9ee9b9d262d0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/clusterbyorderby.q @@ -0,0 +1,5 @@ +FROM src +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +CLUSTER BY tvalue, tkey +ORDER BY ten, one; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clusterbysortby.q b/src/test/hive/ql/src/test/queries/clientnegative/clusterbysortby.q new file mode 100644 index 0000000000000..7b4e744ba66df --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/clusterbysortby.q @@ -0,0 +1,8 @@ +CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +CLUSTER BY tvalue, tkey +SORT BY ten, one; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clustern1.q b/src/test/hive/ql/src/test/queries/clientnegative/clustern1.q new file mode 100644 index 0000000000000..0ff44779657ee --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/clustern1.q @@ -0,0 +1,2 @@ +EXPLAIN +SELECT x.key, x.value as key FROM SRC x CLUSTER BY key; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clustern2.q b/src/test/hive/ql/src/test/queries/clientnegative/clustern2.q new file mode 100644 index 0000000000000..9ed8944d2bb6c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/clustern2.q @@ -0,0 +1,3 @@ +EXPLAIN +SELECT x.key, x.value as v1, y.* FROM SRC x JOIN SRC y ON (x.key = y.key) CLUSTER BY key; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clustern3.q b/src/test/hive/ql/src/test/queries/clientnegative/clustern3.q new file mode 100644 index 0000000000000..23f73667edf5f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/clustern3.q @@ -0,0 +1,2 @@ +EXPLAIN +SELECT x.key as k1, x.value FROM SRC x CLUSTER BY x.key; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clustern4.q b/src/test/hive/ql/src/test/queries/clientnegative/clustern4.q new file mode 100644 index 0000000000000..3a9b45ca60576 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/clustern4.q @@ -0,0 +1,2 @@ +EXPLAIN +SELECT x.key as k1, x.value FROM SRC x CLUSTER BY key; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q b/src/test/hive/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q new file mode 100644 index 0000000000000..9a3e0b2efe693 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q @@ -0,0 +1,5 @@ +set hive.mapred.supports.subdirectories=true; + +CREATE TABLE skewedtable (key STRING, value STRING) SKEWED BY (key) ON (1,5,6); + +ALTER TABLE skewedtable CHANGE key key INT; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_rename1.q b/src/test/hive/ql/src/test/queries/clientnegative/column_rename1.q new file mode 100644 index 0000000000000..d99b821802df7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/column_rename1.q @@ -0,0 +1,6 @@ +drop table tstsrc; +create table tstsrc like src; +insert overwrite table tstsrc +select key, value from src; + +alter table tstsrc change src_not_exist key_value string; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_rename2.q b/src/test/hive/ql/src/test/queries/clientnegative/column_rename2.q new file mode 100644 index 0000000000000..cccc8ad54e300 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/column_rename2.q @@ -0,0 +1,6 @@ +drop table tstsrc; +create table tstsrc like src; +insert overwrite table tstsrc +select key, value from src; + +alter table tstsrc change key value string; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_rename3.q b/src/test/hive/ql/src/test/queries/clientnegative/column_rename3.q new file mode 100644 index 0000000000000..91c9537a99ad5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/column_rename3.q @@ -0,0 +1 @@ +alter table src change key key; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_rename4.q b/src/test/hive/ql/src/test/queries/clientnegative/column_rename4.q new file mode 100644 index 0000000000000..dd89a5a10b227 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/column_rename4.q @@ -0,0 +1,6 @@ +drop table tstsrc; +create table tstsrc like src; +insert overwrite table tstsrc +select key, value from src; + +alter table tstsrc change key key2 string after key_value; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_rename5.q b/src/test/hive/ql/src/test/queries/clientnegative/column_rename5.q new file mode 100644 index 0000000000000..3827b83361fbb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/column_rename5.q @@ -0,0 +1,6 @@ +set hive.mapred.supports.subdirectories=true; + +CREATE TABLE skewedtable (key STRING, value STRING) SKEWED BY (key) ON (1,5,6); + +ALTER TABLE skewedtable CHANGE key key_new STRING; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q new file mode 100644 index 0000000000000..af923504c8c4f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q @@ -0,0 +1,16 @@ +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'); + +-- dynamic partitioning syntax +explain +analyze table Employee_Part partition (employeeSalary='4000.0', country) compute statistics for columns employeeName, employeeID; +analyze table Employee_Part partition (employeeSalary='4000.0', country) compute statistics for columns employeeName, employeeID; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q new file mode 100644 index 0000000000000..d9725ddc7fdf6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q @@ -0,0 +1,16 @@ +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'); + +-- don't specify all partitioning keys +explain +analyze table Employee_Part partition (employeeSalary='2000.0') compute statistics for columns employeeID; +analyze table Employee_Part partition (employeeSalary='2000.0') compute statistics for columns employeeID; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q new file mode 100644 index 0000000000000..eb73962ba133c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q @@ -0,0 +1,16 @@ +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'); + +-- specify invalid values for the partitioning keys +explain +analyze table Employee_Part partition (employeeSalary='4000.0', country='Canada') compute statistics for columns employeeName, employeeID; +analyze table Employee_Part partition (employeeSalary='4000.0', country='Canada') compute statistics for columns employeeName, employeeID; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q new file mode 100644 index 0000000000000..dbfaaecbdf614 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q @@ -0,0 +1,16 @@ +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'); + +-- specify partitioning clause multiple times +explain +analyze table Employee_Part partition (employeeSalary='4000.0', country='USA') partition(employeeSalary='2000.0', country='USA') compute statistics for columns employeeName, employeeID; +analyze table Employee_Part partition (employeeSalary='4000.0', country='USA') partition(employeeSalary='2000.0', country='USA') compute statistics for columns employeeName, employeeID; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl.q b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl.q new file mode 100644 index 0000000000000..ca8548958fcd6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl.q @@ -0,0 +1,22 @@ + +DROP TABLE IF EXISTS UserVisits_web_text_none; + +CREATE TABLE UserVisits_web_text_none ( + sourceIP string, + destURL string, + visitDate string, + adRevenue float, + userAgent string, + cCode string, + lCode string, + sKeyword string, + 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; + +explain +analyze table UserVisits_web_text_none compute statistics for columns destIP; + +analyze table UserVisits_web_text_none compute statistics for columns destIP; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q new file mode 100644 index 0000000000000..5bbd70d86b297 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q @@ -0,0 +1,17 @@ + +DROP TABLE IF EXISTS table_complex_type; + +CREATE TABLE table_complex_type ( + a STRING, + b ARRAY, + c ARRAY>, + d MAP> + ) STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table_complex_type; + + +explain +analyze table table_complex_type compute statistics for columns d; + +analyze table table_complex_type compute statistics for columns d; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q new file mode 100644 index 0000000000000..ca8548958fcd6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q @@ -0,0 +1,22 @@ + +DROP TABLE IF EXISTS UserVisits_web_text_none; + +CREATE TABLE UserVisits_web_text_none ( + sourceIP string, + destURL string, + visitDate string, + adRevenue float, + userAgent string, + cCode string, + lCode string, + sKeyword string, + 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; + +explain +analyze table UserVisits_web_text_none compute statistics for columns destIP; + +analyze table UserVisits_web_text_none compute statistics for columns destIP; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/compare_double_bigint.q b/src/test/hive/ql/src/test/queries/clientnegative/compare_double_bigint.q new file mode 100644 index 0000000000000..8ee4b277cbf72 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/compare_double_bigint.q @@ -0,0 +1,5 @@ +set hive.mapred.mode=strict; + +-- This should fail until we fix the issue with precision when casting a bigint to a double + +select * from src where cast(1 as bigint) = 1.0 limit 10; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/compare_string_bigint.q b/src/test/hive/ql/src/test/queries/clientnegative/compare_string_bigint.q new file mode 100644 index 0000000000000..810f65d4d2b44 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/compare_string_bigint.q @@ -0,0 +1,5 @@ +set hive.mapred.mode=strict; + +--This should fail until we fix the issue with precision when casting a bigint to a double + +select * from src where cast(1 as bigint) = '1' limit 10; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_insert_outputformat.q b/src/test/hive/ql/src/test/queries/clientnegative/create_insert_outputformat.q new file mode 100644 index 0000000000000..a052663055ef6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_insert_outputformat.q @@ -0,0 +1,11 @@ + + +CREATE TABLE table_test_output_format(key INT, value STRING) STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.mapred.MapFileOutputFormat'; + +FROM src +INSERT OVERWRITE TABLE table_test_output_format SELECT src.key, src.value LIMIT 10; + +describe table_test_output_format; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view1.q b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view1.q new file mode 100644 index 0000000000000..c332278b84f68 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view1.q @@ -0,0 +1,6 @@ +-- Cannot add or drop partition columns with CREATE OR REPLACE VIEW if partitions currently exist (must specify partition columns) + +drop view v; +create view v partitioned on (ds, hr) as select * from srcpart; +alter view v add partition (ds='1',hr='2'); +create or replace view v as select * from srcpart; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view2.q b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view2.q new file mode 100644 index 0000000000000..b53dd07ce8ae6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view2.q @@ -0,0 +1,6 @@ +-- Cannot add or drop partition columns with CREATE OR REPLACE VIEW if partitions currently exist + +drop view v; +create view v partitioned on (ds, hr) as select * from srcpart; +alter view v add partition (ds='1',hr='2'); +create or replace view v partitioned on (hr) as select * from srcpart; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view3.q b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view3.q new file mode 100644 index 0000000000000..d6fa7785dfa95 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view3.q @@ -0,0 +1,3 @@ +-- Existing table is not a view + +create or replace view src as select ds, hr from srcpart; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view4.q b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view4.q new file mode 100644 index 0000000000000..12b6059b9e3ee --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view4.q @@ -0,0 +1,5 @@ +-- View must have at least one non-partition column. + +drop view v; +create view v partitioned on (ds, hr) as select * from srcpart; +create or replace view v partitioned on (ds, hr) as select ds, hr from srcpart; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view5.q b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view5.q new file mode 100644 index 0000000000000..4eb9c94896d8f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view5.q @@ -0,0 +1,5 @@ +-- Can't combine IF NOT EXISTS and OR REPLACE. + +drop view v; +create view v partitioned on (ds, hr) as select * from srcpart; +create or replace view if not exists v as select * from srcpart; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view6.q b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view6.q new file mode 100644 index 0000000000000..a2f916fb26528 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view6.q @@ -0,0 +1,5 @@ +-- Can't update view to have an invalid definition + +drop view v; +create view v partitioned on (ds, hr) as select * from srcpart; +create or replace view v partitioned on (ds, hr) as blah; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view7.q b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view7.q new file mode 100644 index 0000000000000..765a96572a04f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view7.q @@ -0,0 +1,7 @@ +-- Can't update view to have a view cycle (1) + +drop view v; +create view v1 partitioned on (ds, hr) as select * from srcpart; +create view v2 partitioned on (ds, hr) as select * from v1; +create view v3 partitioned on (ds, hr) as select * from v2; +create or replace view v1 partitioned on (ds, hr) as select * from v3; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view8.q b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view8.q new file mode 100644 index 0000000000000..f3a59b1d07be7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view8.q @@ -0,0 +1,5 @@ +-- Can't update view to have a view cycle (2) + +drop view v; +create view v1 partitioned on (ds, hr) as select * from srcpart; +create or replace view v1 partitioned on (ds, hr) as select * from v1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q b/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q new file mode 100644 index 0000000000000..1d6574e739602 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q @@ -0,0 +1,3 @@ +set hive.mapred.supports.subdirectories=true; + +CREATE TABLE skewed_table (key STRING, value STRING) SKEWED BY (key) ON ((1),(5,8),(6)); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q b/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q new file mode 100644 index 0000000000000..726f6dd1dfcf6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q @@ -0,0 +1,3 @@ +set hive.mapred.supports.subdirectories=true; + +CREATE TABLE skewed_table (key STRING, value STRING) SKEWED BY (key,key) ON ((1),(5),(6)); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q b/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q new file mode 100644 index 0000000000000..30dd4181653d5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q @@ -0,0 +1,4 @@ +set hive.mapred.supports.subdirectories=true; + +CREATE TABLE skewed_table (key STRING, value STRING) SKEWED BY (key_non) ON ((1),(5),(6)); + \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure1.q b/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure1.q new file mode 100644 index 0000000000000..e87c12b8a1fe6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure1.q @@ -0,0 +1 @@ +create table table_in_database_creation_not_exist.test as select * from src limit 1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure2.q b/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure2.q new file mode 100644 index 0000000000000..0bddae066450e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure2.q @@ -0,0 +1 @@ +create table `table_in_database_creation_not_exist.test` as select * from src limit 1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure3.q b/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure3.q new file mode 100644 index 0000000000000..9f9f5f64dfd94 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure3.q @@ -0,0 +1 @@ +create table table_in_database_creation_not_exist.test (a string); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure4.q b/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure4.q new file mode 100644 index 0000000000000..67745e011141a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure4.q @@ -0,0 +1 @@ +create table `table_in_database_creation_not_exist.test` (a string); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_table_wrong_regex.q b/src/test/hive/ql/src/test/queries/clientnegative/create_table_wrong_regex.q new file mode 100644 index 0000000000000..dc91c9c9ef05e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_table_wrong_regex.q @@ -0,0 +1,4 @@ +drop table aa; +create table aa ( test STRING ) + ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.RegexSerDe' + WITH SERDEPROPERTIES ("input.regex" = "[^\\](.*)", "output.format.string" = "$1s"); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_udaf_failure.q b/src/test/hive/ql/src/test/queries/clientnegative/create_udaf_failure.q new file mode 100644 index 0000000000000..e0bb408a64f2f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_udaf_failure.q @@ -0,0 +1,6 @@ +CREATE TEMPORARY FUNCTION test_udaf AS 'org.apache.hadoop.hive.ql.udf.UDAFWrongArgLengthForTestCase'; + +EXPLAIN +SELECT test_udaf(length(src.value)) FROM src; + +SELECT test_udaf(length(src.value)) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_genericudf.q b/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_genericudf.q new file mode 100644 index 0000000000000..07010c11c7d40 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_genericudf.q @@ -0,0 +1 @@ +CREATE TEMPORARY FUNCTION dummy_genericudf AS 'org.apache.hadoop.hive.ql.udf.generic.DummyGenericUDF'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q b/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q new file mode 100644 index 0000000000000..a243fff033c4d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q @@ -0,0 +1 @@ +CREATE TEMPORARY FUNCTION dummy_function AS 'org.apache.hadoop.hive.ql.udf.DummyFunction'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure1.q b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure1.q new file mode 100644 index 0000000000000..c9060c6766491 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure1.q @@ -0,0 +1,6 @@ + +DROP VIEW xxx12; + +-- views and tables share the same namespace +CREATE TABLE xxx12(key int); +CREATE VIEW xxx12 AS SELECT key FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure2.q b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure2.q new file mode 100644 index 0000000000000..6fdcd4a9d377d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure2.q @@ -0,0 +1,6 @@ + +DROP VIEW xxx4; + +-- views and tables share the same namespace +CREATE VIEW xxx4 AS SELECT key FROM src; +CREATE TABLE xxx4(key int); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure3.q b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure3.q new file mode 100644 index 0000000000000..ad5fc499edf94 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure3.q @@ -0,0 +1,5 @@ +DROP VIEW xxx13; + +-- number of explicit view column defs must match underlying SELECT +CREATE VIEW xxx13(x,y,z) AS +SELECT key FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure4.q b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure4.q new file mode 100644 index 0000000000000..eecde65e1137e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure4.q @@ -0,0 +1,5 @@ +DROP VIEW xxx5; + +-- duplicate column names are illegal +CREATE VIEW xxx5(x,x) AS +SELECT key,value FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure5.q b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure5.q new file mode 100644 index 0000000000000..f720899168735 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure5.q @@ -0,0 +1,9 @@ +DROP VIEW xxx14; + +-- Ideally (and according to SQL:200n), this should actually be legal, +-- but since internally we impose the new column descriptors by +-- reference to underlying name rather than position, we have to make +-- it illegal. There's an easy workaround (provide the unique names +-- via direct column aliases, e.g. SELECT key AS x, key AS y) +CREATE VIEW xxx14(x,y) AS +SELECT key,key FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure6.q b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure6.q new file mode 100644 index 0000000000000..57f52a8af149d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure6.q @@ -0,0 +1,6 @@ +DROP VIEW xxx15; + +-- should fail: baz is not a column +CREATE VIEW xxx15 +PARTITIONED ON (baz) +AS SELECT key FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure7.q b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure7.q new file mode 100644 index 0000000000000..00d7f9fbf4eda --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure7.q @@ -0,0 +1,6 @@ +DROP VIEW xxx16; + +-- should fail: must have at least one non-partitioning column +CREATE VIEW xxx16 +PARTITIONED ON (key) +AS SELECT key FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure8.q b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure8.q new file mode 100644 index 0000000000000..08291826d978b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure8.q @@ -0,0 +1,6 @@ +DROP VIEW xxx17; + +-- should fail: partitioning key must be at end +CREATE VIEW xxx17 +PARTITIONED ON (key) +AS SELECT key,value FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure9.q b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure9.q new file mode 100644 index 0000000000000..d7d44a49c393a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure9.q @@ -0,0 +1,6 @@ +DROP VIEW xxx18; + +-- should fail: partitioning columns out of order +CREATE VIEW xxx18 +PARTITIONED ON (value,key) +AS SELECT key+1 as k2,key,value FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ctas.q b/src/test/hive/ql/src/test/queries/clientnegative/ctas.q new file mode 100644 index 0000000000000..507a7a76b1ee7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ctas.q @@ -0,0 +1,5 @@ + + +create external table nzhang_ctas4 as select key, value from src; + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_create_already_exists.q b/src/test/hive/ql/src/test/queries/clientnegative/database_create_already_exists.q new file mode 100644 index 0000000000000..3af7607739a54 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/database_create_already_exists.q @@ -0,0 +1,5 @@ +SHOW DATABASES; + +-- Try to create a database that already exists +CREATE DATABASE test_db; +CREATE DATABASE test_db; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_create_invalid_name.q b/src/test/hive/ql/src/test/queries/clientnegative/database_create_invalid_name.q new file mode 100644 index 0000000000000..5d6749542b470 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/database_create_invalid_name.q @@ -0,0 +1,4 @@ +SHOW DATABASES; + +-- Try to create a database with an invalid name +CREATE DATABASE `test.db`; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q b/src/test/hive/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q new file mode 100644 index 0000000000000..66a940e63dea6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q @@ -0,0 +1,4 @@ +SHOW DATABASES; + +-- Try to drop a database that does not exist +DROP DATABASE does_not_exist; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty.q b/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty.q new file mode 100644 index 0000000000000..ae5a443f10627 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty.q @@ -0,0 +1,8 @@ +SHOW DATABASES; + +-- Try to drop a non-empty database +CREATE DATABASE test_db; +USE test_db; +CREATE TABLE t(a INT); +USE default; +DROP DATABASE test_db; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q b/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q new file mode 100644 index 0000000000000..e1cb81c93f27b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q @@ -0,0 +1,8 @@ +SHOW DATABASES; + +-- Try to drop a non-empty database in restrict mode +CREATE DATABASE db_drop_non_empty_restrict; +USE db_drop_non_empty_restrict; +CREATE TABLE t(a INT); +USE default; +DROP DATABASE db_drop_non_empty_restrict; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q b/src/test/hive/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q new file mode 100644 index 0000000000000..5cd469769e0aa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q @@ -0,0 +1,4 @@ +SHOW DATABASES; + +-- Try to switch to a database that does not exist +USE does_not_exist; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/date_literal1.q b/src/test/hive/ql/src/test/queries/clientnegative/date_literal1.q new file mode 100644 index 0000000000000..b7fac0d3df346 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/date_literal1.q @@ -0,0 +1,2 @@ +-- Not in YYYY-MM-DD format +SELECT DATE '2001-1-1' FROM src LIMIT 2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/date_literal2.q b/src/test/hive/ql/src/test/queries/clientnegative/date_literal2.q new file mode 100644 index 0000000000000..711dc9e0fd357 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/date_literal2.q @@ -0,0 +1,2 @@ +-- Not in YYYY-MM-DD format +SELECT DATE '2001/01/01' FROM src LIMIT 2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/date_literal3.q b/src/test/hive/ql/src/test/queries/clientnegative/date_literal3.q new file mode 100644 index 0000000000000..9483509b6bb79 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/date_literal3.q @@ -0,0 +1,2 @@ +-- Invalid date value +SELECT DATE '2001-01-32' FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ddltime.q b/src/test/hive/ql/src/test/queries/clientnegative/ddltime.q new file mode 100644 index 0000000000000..3517a6046de14 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ddltime.q @@ -0,0 +1,6 @@ + +create table T2 like srcpart; + +insert overwrite table T2 partition (ds = '2010-06-21', hr='1') select /*+ HOLD_DDLTIME */ key, value from src where key > 10; + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision.q b/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision.q new file mode 100644 index 0000000000000..f49649837e214 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision.q @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS DECIMAL_PRECISION; + +CREATE TABLE DECIMAL_PRECISION(dec decimal) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +SELECT dec * 123456789012345678901234567890.123456789bd FROM DECIMAL_PRECISION; + +DROP TABLE DECIMAL_PRECISION; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision_1.q b/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision_1.q new file mode 100644 index 0000000000000..036ff1facc0a1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision_1.q @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS DECIMAL_PRECISION; + +CREATE TABLE DECIMAL_PRECISION(dec decimal) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +SELECT * from DECIMAL_PRECISION WHERE dec > 1234567890123456789.0123456789bd; + +DROP TABLE DECIMAL_PRECISION; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/default_partition_name.q b/src/test/hive/ql/src/test/queries/clientnegative/default_partition_name.q new file mode 100644 index 0000000000000..816b6cb80a964 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/default_partition_name.q @@ -0,0 +1,3 @@ +create table default_partition_name (key int, value string) partitioned by (ds string); + +alter table default_partition_name add partition(ds='__HIVE_DEFAULT_PARTITION__'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/deletejar.q b/src/test/hive/ql/src/test/queries/clientnegative/deletejar.q new file mode 100644 index 0000000000000..7b0c92311a11a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/deletejar.q @@ -0,0 +1,4 @@ + +ADD JAR ../data/files/TestSerDe.jar; +DELETE JAR ../data/files/TestSerDe.jar; +CREATE TABLE DELETEJAR(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' STORED AS TEXTFILE; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/desc_failure1.q b/src/test/hive/ql/src/test/queries/clientnegative/desc_failure1.q new file mode 100644 index 0000000000000..f7304b12e65f6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/desc_failure1.q @@ -0,0 +1 @@ +DESC NonExistentTable; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/desc_failure2.q b/src/test/hive/ql/src/test/queries/clientnegative/desc_failure2.q new file mode 100644 index 0000000000000..f28b610466499 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/desc_failure2.q @@ -0,0 +1,2 @@ +DESC srcpart; +DESC srcpart PARTITION(ds='2012-04-08', hr='15'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/desc_failure3.q b/src/test/hive/ql/src/test/queries/clientnegative/desc_failure3.q new file mode 100644 index 0000000000000..bee0ea5788b4e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/desc_failure3.q @@ -0,0 +1,5 @@ +CREATE DATABASE db1; +CREATE TABLE db1.t1(key1 INT, value1 STRING) PARTITIONED BY (ds STRING, part STRING); + +-- describe database.table.column +DESCRIBE db1.t1.key1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath1.q b/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath1.q new file mode 100644 index 0000000000000..ea72f83e1d585 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath1.q @@ -0,0 +1 @@ +describe src_thrift.$elem$; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath2.q b/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath2.q new file mode 100644 index 0000000000000..f1fee1ac444de --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath2.q @@ -0,0 +1 @@ +describe src_thrift.$key$; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath3.q b/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath3.q new file mode 100644 index 0000000000000..4a11f6845f396 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath3.q @@ -0,0 +1 @@ +describe src_thrift.lint.abc; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath4.q b/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath4.q new file mode 100644 index 0000000000000..0912bf1cd9ddd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath4.q @@ -0,0 +1 @@ +describe src_thrift.mStringString.abc; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q b/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q new file mode 100644 index 0000000000000..d0d748cf4ffd7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q @@ -0,0 +1,17 @@ +SET hive.metastore.disallow.incompatible.col.type.changes=true; +SELECT * FROM src LIMIT 1; +CREATE TABLE test_table123 (a INT, b MAP) PARTITIONED BY (ds STRING) STORED AS SEQUENCEFILE; +INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, MAP("a1", "b1") FROM src LIMIT 1; +SELECT * from test_table123 WHERE ds="foo1"; +ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b MAP); +ALTER TABLE test_table123 REPLACE COLUMNS (a BIGINT, b MAP); +ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b MAP); +ALTER TABLE test_table123 REPLACE COLUMNS (a DOUBLE, b MAP); +ALTER TABLE test_table123 REPLACE COLUMNS (a TINYINT, b MAP); +ALTER TABLE test_table123 REPLACE COLUMNS (a BOOLEAN, b MAP); +ALTER TABLE test_table123 REPLACE COLUMNS (a TINYINT, b MAP); +ALTER TABLE test_table123 CHANGE COLUMN a a_new BOOLEAN; +-- All the above ALTERs will succeed since they are between compatible types. +-- The following ALTER will fail as MAP and STRING are not +-- compatible. +ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b STRING); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q b/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q new file mode 100644 index 0000000000000..4460c3edd7e4b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q @@ -0,0 +1,6 @@ +SET hive.metastore.disallow.incompatible.col.type.changes=true; +SELECT * FROM src LIMIT 1; +CREATE TABLE test_table123 (a INT, b STRING) PARTITIONED BY (ds STRING) STORED AS SEQUENCEFILE; +INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, "one" FROM src LIMIT 1; +SELECT * from test_table123 WHERE ds="foo1"; +ALTER TABLE test_table123 CHANGE COLUMN b b MAP; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_function_failure.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_function_failure.q new file mode 100644 index 0000000000000..51dc5e9d8e32c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_function_failure.q @@ -0,0 +1,3 @@ +set hive.exec.drop.ignorenonexistent=false; +-- Can't use DROP TEMPORARY FUNCTION if the function doesn't exist and IF EXISTS isn't specified +DROP TEMPORARY FUNCTION UnknownFunction; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_index_failure.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_index_failure.q new file mode 100644 index 0000000000000..6e907dfa99b24 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_index_failure.q @@ -0,0 +1,3 @@ +set hive.exec.drop.ignorenonexistent=false; +-- Can't use DROP INDEX if the index doesn't exist and IF EXISTS isn't specified +DROP INDEX UnknownIndex ON src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_native_udf.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_native_udf.q new file mode 100644 index 0000000000000..ae047bbc1780f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_native_udf.q @@ -0,0 +1 @@ +DROP TEMPORARY FUNCTION max; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_failure.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_failure.q new file mode 100644 index 0000000000000..c2074f69cbf36 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_failure.q @@ -0,0 +1,11 @@ +create table mp (a string) partitioned by (b string, c string); + +alter table mp add partition (b='1', c='1'); +alter table mp add partition (b='1', c='2'); +alter table mp add partition (b='2', c='2'); + +show partitions mp; + +set hive.exec.drop.ignorenonexistent=false; +-- Can't use DROP PARTITION if the partition doesn't exist and IF EXISTS isn't specified +alter table mp drop partition (b='3'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q new file mode 100644 index 0000000000000..df476ed7c4634 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q @@ -0,0 +1,8 @@ +create table ptestfilter1 (a string, b int) partitioned by (c string, d string); + +alter table ptestfilter1 add partition (c='US', d=1); +show partitions ptestfilter1; + +set hive.exec.drop.ignorenonexistent=false; +alter table ptestfilter1 drop partition (c='US', d<1); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q new file mode 100644 index 0000000000000..4d238d73a9116 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q @@ -0,0 +1,11 @@ +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/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure1.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure1.q new file mode 100644 index 0000000000000..d47c08b876fca --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure1.q @@ -0,0 +1,3 @@ +set hive.exec.drop.ignorenonexistent=false; +-- Can't use DROP TABLE if the table doesn't exist and IF EXISTS isn't specified +DROP TABLE UnknownTable; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure2.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure2.q new file mode 100644 index 0000000000000..631e4ffba7a42 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure2.q @@ -0,0 +1,3 @@ +CREATE VIEW xxx6 AS SELECT key FROM src; +-- Can't use DROP TABLE on a view +DROP TABLE xxx6; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure3.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure3.q new file mode 100644 index 0000000000000..534ce0b0324af --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure3.q @@ -0,0 +1,12 @@ +create database dtf3; +use dtf3; + +create table drop_table_failure_temp(col STRING) partitioned by (p STRING); + +alter table drop_table_failure_temp add partition (p ='p1'); +alter table drop_table_failure_temp add partition (p ='p2'); +alter table drop_table_failure_temp add partition (p ='p3'); + +alter table drop_table_failure_temp partition (p ='p3') ENABLE NO_DROP; + +drop table drop_table_failure_temp; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure1.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure1.q new file mode 100644 index 0000000000000..79cb4e445b05c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure1.q @@ -0,0 +1,6 @@ + + +CREATE TABLE xxx1(key int); + +-- Can't use DROP VIEW on a base table +DROP VIEW xxx1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure2.q b/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure2.q new file mode 100644 index 0000000000000..93bb16232d57d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure2.q @@ -0,0 +1,3 @@ +SET hive.exec.drop.ignorenonexistent=false; +-- Can't use DROP VIEW if the view doesn't exist and IF EXISTS isn't specified +DROP VIEW UnknownView; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q b/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q new file mode 100644 index 0000000000000..b2e8567f09e16 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q @@ -0,0 +1 @@ +FROM src SELECT TRANSFORM (key, value) USING "awk -F'\001' '{print $0}'" AS (foo, foo); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q b/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q new file mode 100644 index 0000000000000..dabbc351bc386 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q @@ -0,0 +1 @@ +FROM src SELECT TRANSFORM (key, value) USING "awk -F'\001' '{print $0}'" AS (foo STRING, foo STRING); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert1.q b/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert1.q new file mode 100644 index 0000000000000..fcbc7d5444a4d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert1.q @@ -0,0 +1,7 @@ + +create table dest1_din1(key int, value string); + +from src +insert overwrite table dest1_din1 select key, value +insert overwrite table dest1_din1 select key, value; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert2.q b/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert2.q new file mode 100644 index 0000000000000..4f79a0352f21c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert2.q @@ -0,0 +1,6 @@ + +create table dest1_din2(key int, value string) partitioned by (ds string); + +from src +insert overwrite table dest1_din2 partition (ds='1') select key, value +insert overwrite table dest1_din2 partition (ds='1') select key, value; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert3.q b/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert3.q new file mode 100644 index 0000000000000..7b271a56d1844 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert3.q @@ -0,0 +1,4 @@ + +from src +insert overwrite directory '${system:test.tmp.dir}/dest1' select key, value +insert overwrite directory '${system:test.tmp.dir}/dest1' select key, value; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part1.q b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part1.q new file mode 100644 index 0000000000000..9f0b6c7a0cc88 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part1.q @@ -0,0 +1,11 @@ +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nostrict; +set hive.exec.max.dynamic.partitions=2; + + +create table dynamic_partition (key string) partitioned by (value string); + +insert overwrite table dynamic_partition partition(hr) select key, value from src; + + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part2.q b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part2.q new file mode 100644 index 0000000000000..00a92783c0548 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part2.q @@ -0,0 +1,11 @@ + +create table nzhang_part1 (key string, value string) partitioned by (ds string, hr string); + +set hive.exec.dynamic.partition=true; + +insert overwrite table nzhang_part1 partition(ds='11', hr) select key, value from srcpart where ds is not null; + +show partitions nzhang_part1; + + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part3.q b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part3.q new file mode 100644 index 0000000000000..7a8c58a6b255b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part3.q @@ -0,0 +1,9 @@ +set hive.exec.max.dynamic.partitions=600; +set hive.exec.max.dynamic.partitions.pernode=600; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; +set hive.exec.max.created.files=100; + +create table nzhang_part( key string) partitioned by (value string); + +insert overwrite table nzhang_part partition(value) select key, value from src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part4.q b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part4.q new file mode 100644 index 0000000000000..9aff7aa6310d2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part4.q @@ -0,0 +1,7 @@ +create table nzhang_part4 (key string) partitioned by (ds string, hr string, value string); + +set hive.exec.dynamic.partition=true; + +insert overwrite table nzhang_part4 partition(value = 'aaa', ds='11', hr) select key, hr from srcpart where ds is not null; + +drop table nzhang_part4; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled new file mode 100644 index 0000000000000..a8fce595005d2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled @@ -0,0 +1,24 @@ +-- 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. + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.stats.autogether=false; +set hive.error.on.empty.partition=true; + +create table dyn_err(key string, value string) partitioned by (ds string); + +insert overwrite table dyn_err partition(ds) select key, value, ds from srcpart where ds is not null and key = 'no exists'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max.q b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max.q new file mode 100644 index 0000000000000..6a7a6255b959b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max.q @@ -0,0 +1,16 @@ +USE default; + +-- Test of hive.exec.max.dynamic.partitions +-- Set hive.exec.max.dynamic.partitions.pernode to a large value so it will be ignored + +CREATE TABLE max_parts(key STRING) PARTITIONED BY (value STRING); + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.max.dynamic.partitions=10; +set hive.exec.max.dynamic.partitions.pernode=1000; + +INSERT OVERWRITE TABLE max_parts PARTITION(value) +SELECT key, value +FROM src +LIMIT 50; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q new file mode 100644 index 0000000000000..a411ec520b6d0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q @@ -0,0 +1,15 @@ +USE default; + +-- Test of hive.exec.max.dynamic.partitions.pernode + +CREATE TABLE max_parts(key STRING) PARTITIONED BY (value STRING); + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.max.dynamic.partitions=1000; +set hive.exec.max.dynamic.partitions.pernode=10; + +INSERT OVERWRITE TABLE max_parts PARTITION(value) +SELECT key, value +FROM src +LIMIT 50; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q b/src/test/hive/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q new file mode 100644 index 0000000000000..0be2e71c94dff --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q @@ -0,0 +1,17 @@ +SET hive.metastore.partition.name.whitelist.pattern=[^9]*; +set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifyTableDirectoryIsEmptyHook; + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +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); + +-- 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 +-- This shows that no dynamic partitions were created and left behind or had directories created + +insert overwrite table dest_table partition (ds, hr) select key, hr, ds, value from source_table where ds='2008-04-08' order by value asc; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q new file mode 100644 index 0000000000000..e4f0daca92bd1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q @@ -0,0 +1,12 @@ +CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); +CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); +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'); +SHOW PARTITIONS exchange_part_test1; +SHOW PARTITIONS exchange_part_test2; + +-- for exchange_part_test1 the value of ds is not given and the value of hr is given, thus this query will fail +alter table exchange_part_test1 exchange partition (hr='h1') with table exchange_part_test2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q new file mode 100644 index 0000000000000..4d1e0a62a431f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q @@ -0,0 +1,12 @@ +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; +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; + +-- 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/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q new file mode 100644 index 0000000000000..23777db3ea93d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q @@ -0,0 +1,13 @@ +CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); +CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); +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_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 +alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q new file mode 100644 index 0000000000000..350bf248acc9c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q @@ -0,0 +1,13 @@ +CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); +CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); +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_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 +alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q new file mode 100644 index 0000000000000..81944b3330853 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q @@ -0,0 +1,6 @@ +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 +alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q new file mode 100644 index 0000000000000..7b926a3a8a51c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q @@ -0,0 +1,2 @@ +-- t1 does not exist and the query fails +alter table t1 exchange partition (ds='2013-04-05') with table t2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q new file mode 100644 index 0000000000000..48fcd74a6f22d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q @@ -0,0 +1,8 @@ +CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); +SHOW PARTITIONS exchange_part_test1; + +ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05'); +SHOW PARTITIONS exchange_part_test1; + +-- exchange_part_test2 table 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/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q new file mode 100644 index 0000000000000..23e86e96ca4bc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q @@ -0,0 +1,11 @@ +CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); +CREATE TABLE exchange_part_test2 (f1 string, f2 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'); +SHOW PARTITIONS exchange_part_test1; +SHOW PARTITIONS exchange_part_test2; + +-- exchange_part_test1 and exchange_part_test2 do not have the same scheme and thus they fail +ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE exchange_part_test2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q new file mode 100644 index 0000000000000..d86ecd5785d02 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q @@ -0,0 +1,12 @@ +set hive.test.mode=true; +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; +export table exim_department to 'nosuchschema://nosuchauthority/ql/test/data/exports/exim_department'; +drop table exim_department; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q new file mode 100644 index 0000000000000..5f3223152f766 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q @@ -0,0 +1,24 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +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; +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q new file mode 100644 index 0000000000000..d7204dc478d25 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q @@ -0,0 +1,38 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; + +create table exim_employee ( emp_id int comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; + +create table exim_employee ( emp_id int comment "employee id") + comment "table of employees" + 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" + 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; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q new file mode 100644 index 0000000000000..6cd7eda455ee6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q @@ -0,0 +1,23 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_key int comment "department id") + stored as textfile + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q new file mode 100644 index 0000000000000..7f3f577c433b3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q @@ -0,0 +1,23 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id", dep_name string) + stored as textfile + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q new file mode 100644 index 0000000000000..d3ec9fff82bf9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q @@ -0,0 +1,23 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id bigint comment "department id") + stored as textfile + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q new file mode 100644 index 0000000000000..1cc691fc2912d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q @@ -0,0 +1,23 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id") + stored as rcfile + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q new file mode 100644 index 0000000000000..27830ad5f93dd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q @@ -0,0 +1,26 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id") + stored as inputformat "org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat" + outputformat "org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat" + inputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileInputDriver" + outputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileOutputDriver" + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q new file mode 100644 index 0000000000000..d85048a97a876 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q @@ -0,0 +1,24 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id") + row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" + stored as textfile + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q new file mode 100644 index 0000000000000..84b3786a161c6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q @@ -0,0 +1,28 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id") + row format serde "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe" + with serdeproperties ("serialization.format"="0") + stored as inputformat "org.apache.hadoop.mapred.TextInputFormat" + outputformat "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat" + inputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileInputDriver" + outputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileOutputDriver" + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q new file mode 100644 index 0000000000000..eaf9c579d51d4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q @@ -0,0 +1,24 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id") + clustered by (dep_id) into 10 buckets + stored as textfile + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q new file mode 100644 index 0000000000000..092fd779541c3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q @@ -0,0 +1,25 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; + +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id") + clustered by (dep_id) sorted by (dep_id asc) into 10 buckets + stored as textfile + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q new file mode 100644 index 0000000000000..289bcf001fded --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q @@ -0,0 +1,9 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; + +create table exim_department ( dep_id int comment "department id") + clustered by (dep_id) sorted by (dep_id desc) into 10 buckets + stored by "org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler" + tblproperties("creator"="krishna"); +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q new file mode 100644 index 0000000000000..05de3d77b07b8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q @@ -0,0 +1,24 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id") + stored by "org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler" + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; + \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q new file mode 100644 index 0000000000000..dc194ca814ee4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q @@ -0,0 +1,25 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id") + partitioned by (dep_org string) + stored as textfile + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; + \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q new file mode 100644 index 0000000000000..e233707cc4db2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q @@ -0,0 +1,25 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; + +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id") + stored as textfile + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; + \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q new file mode 100644 index 0000000000000..a10788e3e3b6b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q @@ -0,0 +1,26 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; + +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department id") + partitioned by (dep_mgr string) + stored as textfile + tblproperties("creator"="krishna"); +import from 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/exports/exim_department; + +drop database importer; + \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q new file mode 100644 index 0000000000000..cc4a56ca34b88 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q @@ -0,0 +1,30 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; + +create table exim_employee ( emp_id int comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; +import table exim_employee partition (emp_country="us") 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; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q new file mode 100644 index 0000000000000..140e3bb3b1bbc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q @@ -0,0 +1,30 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; + +create table exim_employee ( emp_id int comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; +import table exim_employee partition (emp_country="us", emp_state="kl") 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; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q new file mode 100644 index 0000000000000..048befe4d3d43 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q @@ -0,0 +1,23 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +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; +drop table exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q new file mode 100644 index 0000000000000..89cbb9ecd8086 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q @@ -0,0 +1,30 @@ +set hive.test.mode=true; +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; +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; + +create table exim_department ( dep_id int comment "department id") + stored as textfile + location 'ql/test/data/tablestore/exim_department' + 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; +drop table exim_department; +dfs -rmr ../build/ql/test/data/tablestore/exim_department; + + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q new file mode 100644 index 0000000000000..0cbfc85258d2c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q @@ -0,0 +1,35 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; + +create table exim_employee ( emp_id int comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; + +create table exim_employee ( emp_id int comment "employee id") + comment "employee table" + partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") + stored as textfile + 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; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_22_export_authfail.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_22_export_authfail.q new file mode 100644 index 0000000000000..d9ab0cf0e4e50 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_22_export_authfail.q @@ -0,0 +1,14 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; + +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; +export table exim_department to 'ql/test/data/exports/exim_department'; + +set hive.security.authorization.enabled=false; +drop table exim_department; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q new file mode 100644 index 0000000000000..2dbd534074fa8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q @@ -0,0 +1,22 @@ +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int) stored as textfile; +set hive.security.authorization.enabled=true; +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; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q new file mode 100644 index 0000000000000..ccbcee3698dae --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q @@ -0,0 +1,31 @@ +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 comment "employee id") + comment "employee table" + 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" + 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; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; +create table exim_employee ( emp_id int comment "employee id") + comment "employee table" + partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") + stored as textfile + tblproperties("creator"="krishna"); + +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; +drop table exim_employee; +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q b/src/test/hive/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q new file mode 100644 index 0000000000000..50bfe005c4278 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q @@ -0,0 +1,23 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +set hive.security.authorization.enabled=true; +import from 'ql/test/data/exports/exim_department'; + +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; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/external1.q b/src/test/hive/ql/src/test/queries/clientnegative/external1.q new file mode 100644 index 0000000000000..d56c955050bc5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/external1.q @@ -0,0 +1,3 @@ + +create external table external1(a int, b int) location 'invalidscheme://data.s3ndemo.hive/kv'; +describe external1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/external2.q b/src/test/hive/ql/src/test/queries/clientnegative/external2.q new file mode 100644 index 0000000000000..0df85a09afdd5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/external2.q @@ -0,0 +1,4 @@ + +create external table external2(a int, b int) partitioned by (ds string); +alter table external2 add partition (ds='2008-01-01') location 'invalidscheme://data.s3ndemo.hive/pkv/2008-01-01'; +describe external2 partition (ds='2008-01-01'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fetchtask_ioexception.q b/src/test/hive/ql/src/test/queries/clientnegative/fetchtask_ioexception.q new file mode 100644 index 0000000000000..9f44f225e955a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/fetchtask_ioexception.q @@ -0,0 +1,7 @@ +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; + +SELECT * FROM fetchtask_ioexception; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fileformat_bad_class.q b/src/test/hive/ql/src/test/queries/clientnegative/fileformat_bad_class.q new file mode 100644 index 0000000000000..33dd4fa614f05 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/fileformat_bad_class.q @@ -0,0 +1,3 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS + INPUTFORMAT 'ClassDoesNotExist' + OUTPUTFORMAT 'java.lang.Void'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_input.q b/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_input.q new file mode 100644 index 0000000000000..c514562b24160 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_input.q @@ -0,0 +1,8 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS + INPUTFORMAT 'java.lang.Void' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_output.q b/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_output.q new file mode 100644 index 0000000000000..a9cef1eada16a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_output.q @@ -0,0 +1,6 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + OUTPUTFORMAT 'java.lang.Void'; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name1.q b/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name1.q new file mode 100644 index 0000000000000..f50369b138573 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name1.q @@ -0,0 +1,2 @@ +set fs.default.name='http://www.example.com; +show tables; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name2.q b/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name2.q new file mode 100644 index 0000000000000..485c3db06823f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name2.q @@ -0,0 +1,2 @@ +set fs.default.name='http://www.example.com; +SELECT * FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/genericFileFormat.q b/src/test/hive/ql/src/test/queries/clientnegative/genericFileFormat.q new file mode 100644 index 0000000000000..bd633b9760ab6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/genericFileFormat.q @@ -0,0 +1 @@ +create table testFail (a int) stored as foo; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q new file mode 100644 index 0000000000000..cecd9c6bd807c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q @@ -0,0 +1,14 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; + +EXPLAIN +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); + +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; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q new file mode 100644 index 0000000000000..e3b0066112c5c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q @@ -0,0 +1,13 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; + +EXPLAIN +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); + +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; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q new file mode 100644 index 0000000000000..168aeb1261b33 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q @@ -0,0 +1,36 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)), + sum(DISTINCT substr(src.value, 5)), + count(DISTINCT substr(src.value, 5)); + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)), + sum(DISTINCT substr(src.value, 5)), + count(DISTINCT substr(src.value, 5)); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q new file mode 100644 index 0000000000000..1a28477918c8e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q @@ -0,0 +1,36 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)), + sum(DISTINCT substr(src.value, 5)), + count(DISTINCT substr(src.value, 5)); + + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)), + sum(DISTINCT substr(src.value, 5)), + count(DISTINCT substr(src.value, 5)); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube1.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube1.q new file mode 100644 index 0000000000000..a0bc177ad6351 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube1.q @@ -0,0 +1,4 @@ +set hive.map.aggr=false; + +SELECT key, count(distinct value) FROM src GROUP BY key with cube; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube2.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube2.q new file mode 100644 index 0000000000000..f8ecb6a2d4347 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube2.q @@ -0,0 +1,4 @@ +set hive.map.aggr=true; + +SELECT key, value, count(distinct value) FROM src GROUP BY key, value with cube; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_id1.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_id1.q new file mode 100644 index 0000000000000..ac5b6f7b03056 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_id1.q @@ -0,0 +1,4 @@ +CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; + +SELECT GROUPING__ID FROM T1; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q new file mode 100644 index 0000000000000..ec6b16bfb28c6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q @@ -0,0 +1,5 @@ +CREATE TABLE T1(a STRING, b STRING, c STRING); + +-- Check for empty grouping set +SELECT * FROM T1 GROUP BY a GROUPING SETS (()); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q new file mode 100644 index 0000000000000..c988e04e74fa2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q @@ -0,0 +1,4 @@ +CREATE TABLE T1(a STRING, b STRING, c STRING); + +-- Check for mupltiple empty grouping sets +SELECT * FROM T1 GROUP BY b GROUPING SETS ((), (), ()); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q new file mode 100644 index 0000000000000..3e73552422956 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q @@ -0,0 +1,4 @@ +CREATE TABLE T1(a STRING, b STRING, c STRING); + +-- Grouping sets expression is not in GROUP BY clause +SELECT a FROM T1 GROUP BY a GROUPING SETS (a, b); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q new file mode 100644 index 0000000000000..cf6352c47d7eb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q @@ -0,0 +1,4 @@ +CREATE TABLE T1(a STRING, b STRING, c STRING); + +-- Expression 'a' is not in GROUP BY clause +SELECT a FROM T1 GROUP BY b GROUPING SETS (b); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q new file mode 100644 index 0000000000000..7df3318a644c9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q @@ -0,0 +1,5 @@ +CREATE TABLE T1(a STRING, b STRING, c STRING); + +-- Alias in GROUPING SETS +SELECT a as c, count(*) FROM T1 GROUP BY c GROUPING SETS (c); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q new file mode 100644 index 0000000000000..2783047698e78 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q @@ -0,0 +1,8 @@ +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; + +-- Since 4 grouping sets would be generated for the query below, an additional MR job should be created +-- This is not allowed with distincts. +SELECT a, b, count(distinct c) from T1 group by a, b with cube; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q new file mode 100644 index 0000000000000..6c9d5133ad7ee --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q @@ -0,0 +1,10 @@ +set hive.new.job.grouping.set.cardinality=2; +set hive.map.aggr=true; +set hive.groupby.skewindata=true; + +CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; + +-- Since 4 grouping sets would be generated for the query below, an additional MR job should be created +-- This is not allowed with map-side aggregation and skew +SELECT a, b, count(1) from T1 group by a, b with cube; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_invalid_position.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_invalid_position.q new file mode 100644 index 0000000000000..173a752e351a8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_invalid_position.q @@ -0,0 +1,4 @@ +set hive.groupby.orderby.position.alias=true; + +-- invalid position alias in group by +SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY 3; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_key.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_key.q new file mode 100644 index 0000000000000..20970152c33cc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_key.q @@ -0,0 +1 @@ +SELECT concat(value, concat(value)) FROM src GROUP BY concat(value); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup1.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup1.q new file mode 100644 index 0000000000000..6366744276077 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup1.q @@ -0,0 +1,4 @@ +set hive.map.aggr=false; + +SELECT key, value, count(1) FROM src GROUP BY key, value with rollup; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup2.q b/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup2.q new file mode 100644 index 0000000000000..aa19b523e9d92 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup2.q @@ -0,0 +1,4 @@ +set hive.map.aggr=true; + +SELECT key, value, count(key) FROM src GROUP BY key, value with rollup; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/having1.q b/src/test/hive/ql/src/test/queries/clientnegative/having1.q new file mode 100644 index 0000000000000..71f4fd13a0a0e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/having1.q @@ -0,0 +1,2 @@ +EXPLAIN SELECT * FROM src HAVING key > 300; +SELECT * FROM src HAVING key > 300; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type.q b/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type.q new file mode 100644 index 0000000000000..1cdaffd1f31a3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type.q @@ -0,0 +1,7 @@ +-- 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'); + +select * from tab1; +drop table tab1; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type2.q b/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type2.q new file mode 100644 index 0000000000000..2438288209896 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type2.q @@ -0,0 +1,3 @@ +create table tab1 (id1 int, id2 string) PARTITIONED BY(month string,day int) row format delimited fields terminated by ','; +alter table tab1 add partition (month='June', day='second'); +drop table tab1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q b/src/test/hive/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q new file mode 100644 index 0000000000000..a17cd1fec5366 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q @@ -0,0 +1,7 @@ +EXPLAIN +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.map.aggr=false; +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX src1_index ON src REBUILD; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/index_compact_entry_limit.q b/src/test/hive/ql/src/test/queries/clientnegative/index_compact_entry_limit.q new file mode 100644 index 0000000000000..7d003e3e4b440 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/index_compact_entry_limit.q @@ -0,0 +1,11 @@ +drop index src_index on src; + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , `_offsets` FROM default__src_src_index__ WHERE key<1000; +SET hive.index.compact.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; +SET hive.index.compact.query.max.entries=5; +SELECT key, value FROM src WHERE key=100 ORDER BY key; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/index_compact_size_limit.q b/src/test/hive/ql/src/test/queries/clientnegative/index_compact_size_limit.q new file mode 100644 index 0000000000000..d79674539a1b2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/index_compact_size_limit.q @@ -0,0 +1,12 @@ +drop index src_index on src; + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , `_offsets` FROM default__src_src_index__ WHERE key<1000; +SET hive.index.compact.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; +SET hive.index.compact.query.max.size=1024; +SELECT key, value FROM src WHERE key=100 ORDER BY key; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/input1.q b/src/test/hive/ql/src/test/queries/clientnegative/input1.q new file mode 100644 index 0000000000000..92a6791acb652 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/input1.q @@ -0,0 +1 @@ +SELECT a.* FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/input2.q b/src/test/hive/ql/src/test/queries/clientnegative/input2.q new file mode 100644 index 0000000000000..0fe907d9d8ae7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/input2.q @@ -0,0 +1 @@ +SELECT a.key FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/input4.q b/src/test/hive/ql/src/test/queries/clientnegative/input4.q new file mode 100644 index 0000000000000..60aea3208c4ed --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/input4.q @@ -0,0 +1,5 @@ +set hive.mapred.mode=strict; + +select * from srcpart a join + (select b.key, count(1) as count from srcpart b where b.ds = '2008-04-08' and b.hr = '14' group by b.key) subq + where a.ds = '2008-04-08' and a.hr = '11' limit 10; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/input41.q b/src/test/hive/ql/src/test/queries/clientnegative/input41.q new file mode 100644 index 0000000000000..872ab1014874d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/input41.q @@ -0,0 +1,5 @@ +select * from + (select * from src + union all + select * from srcpart where ds = '2009-08-09' + )x; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/input_part0_neg.q b/src/test/hive/ql/src/test/queries/clientnegative/input_part0_neg.q new file mode 100644 index 0000000000000..4656693d4838c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/input_part0_neg.q @@ -0,0 +1,3 @@ +set hive.mapred.mode=strict; + +SELECT x.* FROM SRCPART x WHERE key = '2008-04-08'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insert_into1.q b/src/test/hive/ql/src/test/queries/clientnegative/insert_into1.q new file mode 100644 index 0000000000000..8c197670211bf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/insert_into1.q @@ -0,0 +1,11 @@ +set hive.lock.numretries=5; +set hive.lock.sleep.between.retries=5; + +DROP TABLE insert_into1_neg; + +CREATE TABLE insert_into1_neg (key int, value string); + +LOCK TABLE insert_into1_neg SHARED; +INSERT INTO TABLE insert_into1_neg SELECT * FROM src LIMIT 100; + +DROP TABLE insert_into1_neg; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insert_into2.q b/src/test/hive/ql/src/test/queries/clientnegative/insert_into2.q new file mode 100644 index 0000000000000..73a3b6ff13705 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/insert_into2.q @@ -0,0 +1,10 @@ +set hive.lock.numretries=5; +set hive.lock.sleep.between.retries=5; + +DROP TABLE insert_into1_neg; +CREATE TABLE insert_into1_neg (key int, value string); + +LOCK TABLE insert_into1_neg EXCLUSIVE; +INSERT INTO TABLE insert_into1_neg SELECT * FROM src LIMIT 100; + +DROP TABLE insert_into1_neg; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insert_into3.q b/src/test/hive/ql/src/test/queries/clientnegative/insert_into3.q new file mode 100644 index 0000000000000..4d048b337ec45 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/insert_into3.q @@ -0,0 +1,16 @@ +set hive.lock.numretries=5; +set hive.lock.sleep.between.retries=5; + +DROP TABLE insert_into3_neg; + +CREATE TABLE insert_into3_neg (key int, value string) + PARTITIONED BY (ds string); + +INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') + SELECT * FROM src LIMIT 100; + +LOCK TABLE insert_into3_neg PARTITION (ds='1') SHARED; +INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') + SELECT * FROM src LIMIT 100; + +DROP TABLE insert_into3_neg; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insert_into4.q b/src/test/hive/ql/src/test/queries/clientnegative/insert_into4.q new file mode 100644 index 0000000000000..b8944e742b4da --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/insert_into4.q @@ -0,0 +1,16 @@ +set hive.lock.numretries=5; +set hive.lock.sleep.between.retries=5; + +DROP TABLE insert_into3_neg; + +CREATE TABLE insert_into3_neg (key int, value string) + PARTITIONED BY (ds string); + +INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') + SELECT * FROM src LIMIT 100; + +LOCK TABLE insert_into3_neg PARTITION (ds='1') EXCLUSIVE; +INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') + SELECT * FROM src LIMIT 100; + +DROP TABLE insert_into3_neg; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insert_view_failure.q b/src/test/hive/ql/src/test/queries/clientnegative/insert_view_failure.q new file mode 100644 index 0000000000000..1f5e13906259a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/insert_view_failure.q @@ -0,0 +1,5 @@ +DROP VIEW xxx2; +CREATE VIEW xxx2 AS SELECT * FROM src; +INSERT OVERWRITE TABLE xxx2 +SELECT key, value +FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insertexternal1.q b/src/test/hive/ql/src/test/queries/clientnegative/insertexternal1.q new file mode 100644 index 0000000000000..01ebae1022324 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/insertexternal1.q @@ -0,0 +1,8 @@ +set hive.insert.into.external.tables=false; + + +create external table texternal(key string, val string) partitioned by (insertdate string); + +alter table texternal add partition (insertdate='2008-01-01') location 'pfile://${system:test.tmp.dir}/texternal/2008-01-01'; +from src insert overwrite table texternal partition (insertdate='2008-01-01') select *; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q b/src/test/hive/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q new file mode 100644 index 0000000000000..cbf65c4ac69fe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q @@ -0,0 +1,9 @@ +set hive.exec.dynamic.partition=true; + +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); + +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/src/test/hive/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q new file mode 100644 index 0000000000000..ad37cff79b587 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q @@ -0,0 +1,3 @@ + +select timestamp('2001-01-01 00:00:01') - timestamp('2000-01-01 00:00:01') from src; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_avg_syntax.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_avg_syntax.q new file mode 100644 index 0000000000000..d5b58e0765536 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_avg_syntax.q @@ -0,0 +1 @@ +SELECT avg(*) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q new file mode 100644 index 0000000000000..73e4729aa0fc1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q @@ -0,0 +1,2 @@ +create table tbl (a binary); +select cast (a as int) from tbl limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q new file mode 100644 index 0000000000000..50ec48152548c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q @@ -0,0 +1,2 @@ +create table tbl (a binary); +select cast (a as tinyint) from tbl limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q new file mode 100644 index 0000000000000..16f56ec5d3403 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q @@ -0,0 +1,2 @@ +create table tbl (a binary); +select cast (a as smallint) from tbl limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q new file mode 100644 index 0000000000000..bd222f14b469a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q @@ -0,0 +1,2 @@ +create table tbl (a binary); +select cast (a as bigint) from tbl limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q new file mode 100644 index 0000000000000..594fd2bb6f625 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q @@ -0,0 +1,2 @@ +create table tbl (a binary); +select cast (a as float) from tbl limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q new file mode 100644 index 0000000000000..40ff801460ef8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q @@ -0,0 +1,2 @@ +create table tbl (a binary); +select cast (a as double) from tbl limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q new file mode 100644 index 0000000000000..00cd98ed13b7d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q @@ -0,0 +1 @@ +select cast (2 as binary) from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q new file mode 100644 index 0000000000000..f31344f835bb2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q @@ -0,0 +1 @@ +select cast(cast (2 as smallint) as binary) from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q new file mode 100644 index 0000000000000..af23d29f4e984 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q @@ -0,0 +1 @@ +select cast(cast (2 as tinyint) as binary) from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q new file mode 100644 index 0000000000000..91abe1e6b8a27 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q @@ -0,0 +1 @@ +select cast(cast (2 as bigint) as binary) from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q new file mode 100644 index 0000000000000..afd99be9765a3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q @@ -0,0 +1 @@ +select cast(cast (2 as float) as binary) from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q new file mode 100644 index 0000000000000..c2143c5c9e955 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q @@ -0,0 +1 @@ +select cast(cast (2 as double) as binary) from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_columns.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_columns.q new file mode 100644 index 0000000000000..14b3409cb4cc4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_columns.q @@ -0,0 +1,4 @@ +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/src/test/hive/ql/src/test/queries/clientnegative/invalid_config1.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_config1.q new file mode 100644 index 0000000000000..c49ac8a69086e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_config1.q @@ -0,0 +1,3 @@ +set mapred.input.dir.recursive=true; + +CREATE TABLE skewedtable (key STRING, value STRING) SKEWED BY (key) ON (1,5,6); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_config2.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_config2.q new file mode 100644 index 0000000000000..fa023c8c4b5f2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_config2.q @@ -0,0 +1,4 @@ +set hive.mapred.supports.subdirectories=false; +set hive.optimize.union.remove=true; + +select count(1) from src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl1.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl1.q new file mode 100644 index 0000000000000..2e1ea6b005618 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl1.q @@ -0,0 +1,9 @@ + +CREATE TABLE inv_valid_tbl1 COMMENT 'This is a thrift based table' + PARTITIONED BY(aint DATETIME, country STRING) + CLUSTERED BY(aint) SORTED BY(lint) INTO 32 BUCKETS + ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer' + WITH SERDEPROPERTIES ('serialization.class' = 'org.apache.hadoop.hive.serde2.thrift.test.Complex', + 'serialization.format' = 'org.apache.thrift.protocol.TBinaryProtocol') + STORED AS SEQUENCEFILE; +DESCRIBE EXTENDED inv_valid_tbl1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl2.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl2.q new file mode 100644 index 0000000000000..408919ee2d633 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl2.q @@ -0,0 +1 @@ +create tabl tmp_zshao_22 (id int, name strin; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_mapjoin1.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_mapjoin1.q new file mode 100644 index 0000000000000..56d9211d28ebb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_mapjoin1.q @@ -0,0 +1 @@ +select /*+ MAPJOIN(a) ,MAPJOIN(b)*/ * from src a join src b on (a.key=b.key and a.value=b.value); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_max_syntax.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_max_syntax.q new file mode 100644 index 0000000000000..20033734090f3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_max_syntax.q @@ -0,0 +1 @@ +SELECT max(*) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_min_syntax.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_min_syntax.q new file mode 100644 index 0000000000000..584283a08a9e7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_min_syntax.q @@ -0,0 +1 @@ +SELECT min(*) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column.q new file mode 100644 index 0000000000000..106ba42213197 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column.q @@ -0,0 +1,4 @@ +-- Create table +create table if not exists test_invalid_column(key string, value string ) partitioned by (year string, month string) stored as textfile ; + +select * from test_invalid_column where column1=123; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q new file mode 100644 index 0000000000000..bc70dbca20772 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q @@ -0,0 +1,4 @@ +-- Create table +create table if not exists test_invalid_column(key string, value string ) partitioned by (year string, month string) stored as textfile ; + +select * from (select * from test_invalid_column) subq where subq = 123; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q new file mode 100644 index 0000000000000..b821e6129a7bf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q @@ -0,0 +1,4 @@ +-- Create table +create table if not exists test_invalid_column(key string, value string ) partitioned by (year string, month string) stored as textfile ; + +select * from test_invalid_column where test_invalid_column=123; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_expression.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_expression.q new file mode 100644 index 0000000000000..01617f9363b58 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_expression.q @@ -0,0 +1 @@ +select foo from a a where foo > .foo; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_std_syntax.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_std_syntax.q new file mode 100644 index 0000000000000..13104198a6db0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_std_syntax.q @@ -0,0 +1 @@ +SELECT std(*) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q new file mode 100644 index 0000000000000..c6a12526559e0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q @@ -0,0 +1 @@ +SELECT stddev_samp(*) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_sum_syntax.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_sum_syntax.q new file mode 100644 index 0000000000000..2d591baa24eb7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_sum_syntax.q @@ -0,0 +1 @@ +SELECT sum(*) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter1.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter1.q new file mode 100644 index 0000000000000..bb19cff8a93ee --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter1.q @@ -0,0 +1,2 @@ +CREATE TABLE alter_test (d STRING); +ALTER TABLE alter_test CHANGE d d DATETIME; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter2.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter2.q new file mode 100644 index 0000000000000..aa01b358727b1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter2.q @@ -0,0 +1,2 @@ +CREATE TABLE alter_test (d STRING); +ALTER TABLE alter_test ADD COLUMNS (ds DATETIME); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_create2.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_create2.q new file mode 100644 index 0000000000000..978f4244a6ba0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_create2.q @@ -0,0 +1 @@ +CREATE TABLE datetime_test (d DATETIME); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_transform.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_transform.q new file mode 100644 index 0000000000000..dfc4864acf43f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_transform.q @@ -0,0 +1 @@ +SELECT TRANSFORM(*) USING 'cat' AS (key DATETIME) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_tbl_name.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_tbl_name.q new file mode 100644 index 0000000000000..09394e71ada96 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_tbl_name.q @@ -0,0 +1 @@ +create table invalid-name(a int, b string); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q new file mode 100644 index 0000000000000..ce2a8c4769111 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q @@ -0,0 +1 @@ +SELECT var_samp(*) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q new file mode 100644 index 0000000000000..43de018c9f14c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q @@ -0,0 +1,2 @@ +drop table if exists invalid_varchar_length_1; +create table invalid_varchar_length_1 (c1 varchar(1000000)); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q new file mode 100644 index 0000000000000..3c199d31e7ffc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q @@ -0,0 +1 @@ +select cast(value as varchar(100000)) from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q new file mode 100644 index 0000000000000..fed04764a9440 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q @@ -0,0 +1,3 @@ +drop table if exists invalid_varchar_length_3; +create table invalid_varchar_length_3 (c1 varchar(0)); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_variance_syntax.q b/src/test/hive/ql/src/test/queries/clientnegative/invalid_variance_syntax.q new file mode 100644 index 0000000000000..5b478299317aa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalid_variance_syntax.q @@ -0,0 +1 @@ +SELECT variance(*) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalidate_view1.q b/src/test/hive/ql/src/test/queries/clientnegative/invalidate_view1.q new file mode 100644 index 0000000000000..dd39c5eb4a4fe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/invalidate_view1.q @@ -0,0 +1,11 @@ +DROP VIEW xxx8; +DROP VIEW xxx9; + +-- create two levels of view reference, then invalidate intermediate view +-- by dropping a column from underlying table, and verify that +-- querying outermost view results in full error context +CREATE TABLE xxx10 (key int, value int); +CREATE VIEW xxx9 AS SELECT * FROM xxx10; +CREATE VIEW xxx8 AS SELECT * FROM xxx9 xxx; +ALTER TABLE xxx10 REPLACE COLUMNS (key int); +SELECT * FROM xxx8 yyy; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join2.q b/src/test/hive/ql/src/test/queries/clientnegative/join2.q new file mode 100644 index 0000000000000..98a5f1e6629c7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/join2.q @@ -0,0 +1,5 @@ +SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value +FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key); + + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join28.q b/src/test/hive/ql/src/test/queries/clientnegative/join28.q new file mode 100644 index 0000000000000..32ff105c2e450 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/join28.q @@ -0,0 +1,15 @@ +CREATE TABLE dest_j1(key STRING, value STRING) STORED AS TEXTFILE; + +-- Mapjoin followed by mapjoin is not supported. +-- The same query would work fine without the hint. +-- Note that there is a positive test with the same name in clientpositive +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(z) */ subq.key1, z.value +FROM +(SELECT /*+ MAPJOIN(x) */ x.key as key1, x.value as value1, y.key as key2, y.value as value2 + FROM src1 x JOIN src y ON (x.key = y.key)) subq + JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11); + + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join29.q b/src/test/hive/ql/src/test/queries/clientnegative/join29.q new file mode 100644 index 0000000000000..53a1652d25b20 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/join29.q @@ -0,0 +1,10 @@ +CREATE TABLE dest_j1(key STRING, cnt1 INT, cnt2 INT); + +-- Mapjoin followed by group by is not supported. +-- The same query would work without the hint +-- Note that there is a positive test with the same name in clientpositive +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(subq1) */ subq1.key, subq1.cnt, subq2.cnt +FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN + (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join32.q b/src/test/hive/ql/src/test/queries/clientnegative/join32.q new file mode 100644 index 0000000000000..54a4dcd9afe2a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/join32.q @@ -0,0 +1,14 @@ +CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE; + +-- Mapjoin followed by Mapjoin is not supported. +-- The same query would work without the hint +-- Note that there is a positive test with the same name in clientpositive +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x,z) */ x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11); + + + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join35.q b/src/test/hive/ql/src/test/queries/clientnegative/join35.q new file mode 100644 index 0000000000000..fc8f77ca1232c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/join35.q @@ -0,0 +1,18 @@ +CREATE TABLE dest_j1(key STRING, value STRING, val2 INT) STORED AS TEXTFILE; + +-- Mapjoin followed by union is not supported. +-- The same query would work without the hint +-- Note that there is a positive test with the same name in clientpositive +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x) */ x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key); + + + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join_nonexistent_part.q b/src/test/hive/ql/src/test/queries/clientnegative/join_nonexistent_part.q new file mode 100644 index 0000000000000..b4a4757d22147 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/join_nonexistent_part.q @@ -0,0 +1,4 @@ +SET hive.security.authorization.enabled = true; +SELECT * +FROM srcpart s1 join src s2 on s1.key == s2.key +WHERE s1.ds='non-existent'; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/joinneg.q b/src/test/hive/ql/src/test/queries/clientnegative/joinneg.q new file mode 100644 index 0000000000000..a4967fd5dfb4a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/joinneg.q @@ -0,0 +1,6 @@ +EXPLAIN FROM +(SELECT src.* FROM src) x +JOIN +(SELECT src.* FROM src) Y +ON (x.key = b.key) +SELECT Y.*; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_alias.q b/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_alias.q new file mode 100644 index 0000000000000..50d535e6e1ecf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_alias.q @@ -0,0 +1,3 @@ +-- Check alias count for LATERAL VIEW syntax: +-- explode returns a table with only 1 col - should be an error if query specifies >1 col aliases +SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol1, myCol2 LIMIT 3; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_join.q b/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_join.q new file mode 100644 index 0000000000000..818754ecbf05b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_join.q @@ -0,0 +1 @@ +SELECT src.key FROM src LATERAL VIEW explode(array(1,2,3)) AS myTable JOIN src b ON src.key; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/line_terminator.q b/src/test/hive/ql/src/test/queries/clientnegative/line_terminator.q new file mode 100644 index 0000000000000..ad3542c40ace4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/line_terminator.q @@ -0,0 +1,3 @@ +CREATE TABLE mytable (col1 STRING, col2 INT) +ROW FORMAT DELIMITED +LINES TERMINATED BY ','; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_exist_part_authfail.q b/src/test/hive/ql/src/test/queries/clientnegative/load_exist_part_authfail.q new file mode 100644 index 0000000000000..f86cd92d9d4c7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_exist_part_authfail.q @@ -0,0 +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'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_non_native.q b/src/test/hive/ql/src/test/queries/clientnegative/load_non_native.q new file mode 100644 index 0000000000000..387aaed9a1e50 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_non_native.q @@ -0,0 +1,5 @@ + +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; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_nonpart_authfail.q b/src/test/hive/ql/src/test/queries/clientnegative/load_nonpart_authfail.q new file mode 100644 index 0000000000000..d807c698777f1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_nonpart_authfail.q @@ -0,0 +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 ; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_part_authfail.q b/src/test/hive/ql/src/test/queries/clientnegative/load_part_authfail.q new file mode 100644 index 0000000000000..c409d5a94a9ce --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_part_authfail.q @@ -0,0 +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'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_part_nospec.q b/src/test/hive/ql/src/test/queries/clientnegative/load_part_nospec.q new file mode 100644 index 0000000000000..356c16a664386 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_part_nospec.q @@ -0,0 +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; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_stored_as_dirs.q b/src/test/hive/ql/src/test/queries/clientnegative/load_stored_as_dirs.q new file mode 100644 index 0000000000000..eed5651cbf20b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_stored_as_dirs.q @@ -0,0 +1,7 @@ +set hive.mapred.supports.subdirectories=true; + +-- Load data can't work with table with stored as directories +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; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_view_failure.q b/src/test/hive/ql/src/test/queries/clientnegative/load_view_failure.q new file mode 100644 index 0000000000000..927f02e82bf91 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_view_failure.q @@ -0,0 +1,3 @@ +DROP VIEW xxx11; +CREATE VIEW xxx11 AS SELECT * FROM src; +LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE xxx11; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat.q b/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat.q new file mode 100644 index 0000000000000..16feeca22649a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat.q @@ -0,0 +1,6 @@ +-- test for loading into tables with the correct file format +-- test for loading into partitions with the correct file format + + +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; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q b/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q new file mode 100644 index 0000000000000..7e589fbfde620 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q @@ -0,0 +1,6 @@ +-- test for loading into tables with the correct file format +-- test for loading into partitions with the correct file format + + +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 diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q b/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q new file mode 100644 index 0000000000000..ff5ed4e2e3107 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q @@ -0,0 +1,6 @@ +-- test for loading into tables with the correct file format +-- test for loading into partitions with the correct file format + + +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 diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_noof_part.q b/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_noof_part.q new file mode 100644 index 0000000000000..ffb64ed643b14 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_noof_part.q @@ -0,0 +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'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/local_mapred_error_cache.q b/src/test/hive/ql/src/test/queries/clientnegative/local_mapred_error_cache.q new file mode 100644 index 0000000000000..8f4b37a9d49f6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/local_mapred_error_cache.q @@ -0,0 +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); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lockneg1.q b/src/test/hive/ql/src/test/queries/clientnegative/lockneg1.q new file mode 100644 index 0000000000000..e1b58fca80af3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/lockneg1.q @@ -0,0 +1,10 @@ +drop table tstsrc; +create table tstsrc like src; +insert overwrite table tstsrc select key, value from src; + +set hive.lock.numretries=0; +set hive.unlock.numretries=0; + +LOCK TABLE tstsrc SHARED; +LOCK TABLE tstsrc SHARED; +LOCK TABLE tstsrc EXCLUSIVE; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lockneg2.q b/src/test/hive/ql/src/test/queries/clientnegative/lockneg2.q new file mode 100644 index 0000000000000..a4604cd470658 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/lockneg2.q @@ -0,0 +1,6 @@ +drop table tstsrc; +create table tstsrc like src; +insert overwrite table tstsrc select key, value from src; + +set hive.unlock.numretries=0; +UNLOCK TABLE tstsrc; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lockneg3.q b/src/test/hive/ql/src/test/queries/clientnegative/lockneg3.q new file mode 100644 index 0000000000000..f2252f7bdf4da --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/lockneg3.q @@ -0,0 +1,9 @@ +drop table tstsrcpart; +create table tstsrcpart like srcpart; + +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') +select key, value from srcpart where ds='2008-04-08' and hr='11'; + +set hive.lock.numretries=0; +set hive.unlock.numretries=0; +UNLOCK TABLE tstsrcpart PARTITION(ds='2008-04-08', hr='11'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lockneg4.q b/src/test/hive/ql/src/test/queries/clientnegative/lockneg4.q new file mode 100644 index 0000000000000..b47644cca362c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/lockneg4.q @@ -0,0 +1,12 @@ +drop table tstsrcpart; +create table tstsrcpart like srcpart; + +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') +select key, value from srcpart where ds='2008-04-08' and hr='11'; + +set hive.lock.numretries=0; +set hive.unlock.numretries=0; + +LOCK TABLE tstsrcpart PARTITION(ds='2008-04-08', hr='11') EXCLUSIVE; +SHOW LOCKS tstsrcpart PARTITION(ds='2008-04-08', hr='12'); + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lockneg5.q b/src/test/hive/ql/src/test/queries/clientnegative/lockneg5.q new file mode 100644 index 0000000000000..19c1ce28c2422 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/lockneg5.q @@ -0,0 +1,2 @@ +drop table tstsrcpart; +show locks tstsrcpart extended; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/macro_unused_parameter.q b/src/test/hive/ql/src/test/queries/clientnegative/macro_unused_parameter.q new file mode 100644 index 0000000000000..523710ddf3a5b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/macro_unused_parameter.q @@ -0,0 +1 @@ +CREATE TEMPORARY MACRO BAD_MACRO (x INT, y INT) x; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q b/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q new file mode 100644 index 0000000000000..76c7ae94d4b63 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q @@ -0,0 +1,13 @@ +set hive.exec.mode.local.auto=false; +set hive.exec.job.debug.capture.stacktraces=true; +set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; + +FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Hadoop 0.23 changes the getTaskDiagnostics behavior +-- The Error Code of hive failure MapReduce job changes +-- In Hadoop 0.20 +-- Hive failure MapReduce job gets 20000 as Error Code +-- In Hadoop 0.23 +-- Hive failure MapReduce job gets 2 as Error Code diff --git a/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q b/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q new file mode 100644 index 0000000000000..9d0548cc10f55 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q @@ -0,0 +1,13 @@ +set hive.exec.mode.local.auto=false; +set hive.exec.job.debug.capture.stacktraces=true; +set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; + +FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); + +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Hadoop 0.23 changes the getTaskDiagnostics behavior +-- The Error Code of hive failure MapReduce job changes +-- In Hadoop 0.20 +-- Hive failure MapReduce job gets 20000 as Error Code +-- In Hadoop 0.23 +-- Hive failure MapReduce job gets 2 as Error Code diff --git a/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q b/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q new file mode 100644 index 0000000000000..c93aedb3137bf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q @@ -0,0 +1,13 @@ +set hive.exec.mode.local.auto=false; +set hive.exec.job.debug.capture.stacktraces=false; +set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; + +FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Hadoop 0.23 changes the getTaskDiagnostics behavior +-- The Error Code of hive failure MapReduce job changes +-- In Hadoop 0.20 +-- Hive failure MapReduce job gets 20000 as Error Code +-- In Hadoop 0.23 +-- Hive failure MapReduce job gets 2 as Error Code diff --git a/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q b/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q new file mode 100644 index 0000000000000..e319944958c2a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q @@ -0,0 +1,13 @@ +set hive.exec.mode.local.auto=false; +set hive.exec.job.debug.capture.stacktraces=false; +set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; + +FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); + +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Hadoop 0.23 changes the getTaskDiagnostics behavior +-- The Error Code of hive failure MapReduce job changes +-- In Hadoop 0.20 +-- Hive failure MapReduce job gets 20000 as Error Code +-- In Hadoop 0.23 +-- Hive failure MapReduce job gets 2 as Error Code diff --git a/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_1.q b/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_1.q new file mode 100644 index 0000000000000..0a48c01546ec5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_1.q @@ -0,0 +1,3 @@ +create table src2 like src; +CREATE INDEX src_index_merge_test ON TABLE src2(key) as 'COMPACT' WITH DEFERRED REBUILD; +alter table src2 concatenate; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_2.q b/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_2.q new file mode 100644 index 0000000000000..a4fab1c8b804d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_2.q @@ -0,0 +1,3 @@ +create table srcpart2 (key int, value string) partitioned by (ds string); +insert overwrite table srcpart2 partition (ds='2011') select * from src; +alter table srcpart2 concatenate; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_3.q b/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_3.q new file mode 100644 index 0000000000000..6bc645e4c2378 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_3.q @@ -0,0 +1,6 @@ +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; + +create table srcpart2 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 2 buckets stored as RCFILE; +insert overwrite table srcpart2 partition (ds='2011') select * from src; +alter table srcpart2 partition (ds = '2011') concatenate; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/minimr_broken_pipe.q b/src/test/hive/ql/src/test/queries/clientnegative/minimr_broken_pipe.q new file mode 100644 index 0000000000000..8dda9cdf4a37d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/minimr_broken_pipe.q @@ -0,0 +1,4 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +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/src/test/hive/ql/src/test/queries/clientnegative/nested_complex_neg.q b/src/test/hive/ql/src/test/queries/clientnegative/nested_complex_neg.q new file mode 100644 index 0000000000000..ac6c4ee549d85 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/nested_complex_neg.q @@ -0,0 +1,15 @@ + +create table nestedcomplex ( +simple_int int, +max_nested_array array>>>>>>>>>>>>>>>>>>>>>>, +max_nested_map array>>>>>>>>>>>>>>>>>>>>>, +max_nested_struct array>>>>>>>>>>>>>>>>>>>>>>, +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; + +select * from nestedcomplex sort by simple_int; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/no_matching_udf.q b/src/test/hive/ql/src/test/queries/clientnegative/no_matching_udf.q new file mode 100644 index 0000000000000..0c24b1626a532 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/no_matching_udf.q @@ -0,0 +1 @@ +SELECT percentile(3.5, 0.99) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/nonkey_groupby.q b/src/test/hive/ql/src/test/queries/clientnegative/nonkey_groupby.q new file mode 100644 index 0000000000000..431e04efd9342 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/nonkey_groupby.q @@ -0,0 +1 @@ +EXPLAIN SELECT key, count(1) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/nopart_insert.q b/src/test/hive/ql/src/test/queries/clientnegative/nopart_insert.q new file mode 100644 index 0000000000000..4841f9e11c84e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/nopart_insert.q @@ -0,0 +1,7 @@ + +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) +FROM src; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/nopart_load.q b/src/test/hive/ql/src/test/queries/clientnegative/nopart_load.q new file mode 100644 index 0000000000000..6e5ad6eb41a8b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/nopart_load.q @@ -0,0 +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 ; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/notable_alias3.q b/src/test/hive/ql/src/test/queries/clientnegative/notable_alias3.q new file mode 100644 index 0000000000000..6cc3e87288a89 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/notable_alias3.q @@ -0,0 +1,4 @@ +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/src/test/hive/ql/src/test/queries/clientnegative/notable_alias4.q b/src/test/hive/ql/src/test/queries/clientnegative/notable_alias4.q new file mode 100644 index 0000000000000..e7ad6b79d3ed9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/notable_alias4.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT key from src JOIN src1 on src1.key=src.key; + +SELECT key from src JOIN src1 on src1.key=src.key; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/orderby_invalid_position.q b/src/test/hive/ql/src/test/queries/clientnegative/orderby_invalid_position.q new file mode 100644 index 0000000000000..4dbf2a6d56a2e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/orderby_invalid_position.q @@ -0,0 +1,4 @@ +set hive.groupby.orderby.position.alias=true; + +-- invalid position alias in order by +SELECT src.key, src.value FROM src ORDER BY 0; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/orderby_position_unsupported.q b/src/test/hive/ql/src/test/queries/clientnegative/orderby_position_unsupported.q new file mode 100644 index 0000000000000..a490c2306ec4a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/orderby_position_unsupported.q @@ -0,0 +1,4 @@ +set hive.groupby.orderby.position.alias=true; + +-- position alias is not supported when SELECT * +SELECT src.* FROM src ORDER BY 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/orderbysortby.q b/src/test/hive/ql/src/test/queries/clientnegative/orderbysortby.q new file mode 100644 index 0000000000000..5dff69fdbb78d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/orderbysortby.q @@ -0,0 +1,8 @@ +CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +ORDER BY tvalue, tkey +SORT BY ten, one; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/part_col_complex_type.q b/src/test/hive/ql/src/test/queries/clientnegative/part_col_complex_type.q new file mode 100644 index 0000000000000..4b9eb847db542 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/part_col_complex_type.q @@ -0,0 +1 @@ +create table t (a string) partitioned by (b map); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part.q new file mode 100644 index 0000000000000..541599915afc0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part.q @@ -0,0 +1,15 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode3; + +create table tbl_protectmode3 (col string) partitioned by (p string); +alter table tbl_protectmode3 add partition (p='p1'); +alter table tbl_protectmode3 add partition (p='p2'); + +select * from tbl_protectmode3 where p='p1'; +select * from tbl_protectmode3 where p='p2'; + +alter table tbl_protectmode3 partition (p='p1') enable offline; + +select * from tbl_protectmode3 where p='p2'; +select * from tbl_protectmode3 where p='p1'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part1.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part1.q new file mode 100644 index 0000000000000..99256da285c1a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part1.q @@ -0,0 +1,21 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode5; + +create table tbl_protectmode5_1 (col string); + +create table tbl_protectmode5 (col string) partitioned by (p string); +alter table tbl_protectmode5 add partition (p='p1'); +alter table tbl_protectmode5 add partition (p='p2'); + +insert overwrite table tbl_protectmode5_1 +select col from tbl_protectmode5 where p='p1'; +insert overwrite table tbl_protectmode5_1 +select col from tbl_protectmode5 where p='p2'; + +alter table tbl_protectmode5 partition (p='p1') enable offline; + +insert overwrite table tbl_protectmode5_1 +select col from tbl_protectmode5 where p='p2'; +insert overwrite table tbl_protectmode5_1 +select col from tbl_protectmode5 where p='p1'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part2.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part2.q new file mode 100644 index 0000000000000..72b55ea25d8e9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part2.q @@ -0,0 +1,9 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +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'); +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'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q new file mode 100644 index 0000000000000..b4e508ff98180 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q @@ -0,0 +1,10 @@ +-- protect mode: syntax to change protect mode works and queries to drop partitions are blocked if it is marked no drop + +drop table tbl_protectmode_no_drop; + +create table tbl_protectmode_no_drop (c1 string,c2 string) partitioned by (p string); +alter table tbl_protectmode_no_drop add partition (p='p1'); +alter table tbl_protectmode_no_drop partition (p='p1') enable no_drop; +desc extended tbl_protectmode_no_drop partition (p='p1'); + +alter table tbl_protectmode_no_drop drop partition (p='p1'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl1.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl1.q new file mode 100644 index 0000000000000..236129902c07c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl1.q @@ -0,0 +1,8 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode_1; + +create table tbl_protectmode_1 (col string); +select * from tbl_protectmode_1; +alter table tbl_protectmode_1 enable offline; +select * from tbl_protectmode_1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl2.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl2.q new file mode 100644 index 0000000000000..05964c35e9e08 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl2.q @@ -0,0 +1,12 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode2; + +create table tbl_protectmode2 (col string) partitioned by (p string); +alter table tbl_protectmode2 add partition (p='p1'); +alter table tbl_protectmode2 enable no_drop; +alter table tbl_protectmode2 enable offline; +alter table tbl_protectmode2 disable no_drop; +desc extended tbl_protectmode2; + +select * from tbl_protectmode2 where p='p1'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl3.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl3.q new file mode 100644 index 0000000000000..bbaa2670875b6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl3.q @@ -0,0 +1,10 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode_4; + +create table tbl_protectmode_4 (col string); +select col from tbl_protectmode_4; +alter table tbl_protectmode_4 enable offline; +desc extended tbl_protectmode_4; + +select col from tbl_protectmode_4; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl4.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl4.q new file mode 100644 index 0000000000000..c7880de6d8aec --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl4.q @@ -0,0 +1,15 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode_tbl4; +drop table tbl_protectmode_tbl4_src; + +create table tbl_protectmode_tbl4_src (col string); + +create table tbl_protectmode_tbl4 (col string) partitioned by (p string); +alter table tbl_protectmode_tbl4 add partition (p='p1'); +alter table tbl_protectmode_tbl4 enable no_drop; +alter table tbl_protectmode_tbl4 enable offline; +alter table tbl_protectmode_tbl4 disable no_drop; +desc extended tbl_protectmode_tbl4; + +select col from tbl_protectmode_tbl4 where p='not_exist'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl5.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl5.q new file mode 100644 index 0000000000000..cd848fd4a1b9f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl5.q @@ -0,0 +1,15 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode_tbl5; +drop table tbl_protectmode_tbl5_src; + +create table tbl_protectmode_tbl5_src (col string); + +create table tbl_protectmode_tbl5 (col string) partitioned by (p string); +alter table tbl_protectmode_tbl5 add partition (p='p1'); +alter table tbl_protectmode_tbl5 enable no_drop; +alter table tbl_protectmode_tbl5 enable offline; +alter table tbl_protectmode_tbl5 disable no_drop; +desc extended tbl_protectmode_tbl5; + +insert overwrite table tbl_protectmode_tbl5 partition (p='not_exist') select col from tbl_protectmode_tbl5_src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl6.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl6.q new file mode 100644 index 0000000000000..26248cc6b4877 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl6.q @@ -0,0 +1,8 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode_tbl6; + +create table tbl_protectmode_tbl6 (col string); +alter table tbl_protectmode_tbl6 enable no_drop cascade; + +drop table tbl_protectmode_tbl6; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl7.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl7.q new file mode 100644 index 0000000000000..afff8404edc01 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl7.q @@ -0,0 +1,13 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode_tbl7; +create table tbl_protectmode_tbl7 (col string) partitioned by (p string); +alter table tbl_protectmode_tbl7 add partition (p='p1'); +alter table tbl_protectmode_tbl7 enable no_drop; + +alter table tbl_protectmode_tbl7 drop partition (p='p1'); + +alter table tbl_protectmode_tbl7 add partition (p='p1'); +alter table tbl_protectmode_tbl7 enable no_drop cascade; + +alter table tbl_protectmode_tbl7 drop partition (p='p1'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl8.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl8.q new file mode 100644 index 0000000000000..809c287fc502a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl8.q @@ -0,0 +1,13 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode_tbl8; +create table tbl_protectmode_tbl8 (col string) partitioned by (p string); +alter table tbl_protectmode_tbl8 add partition (p='p1'); +alter table tbl_protectmode_tbl8 enable no_drop; + +alter table tbl_protectmode_tbl8 drop partition (p='p1'); + +alter table tbl_protectmode_tbl8 enable no_drop cascade; + +alter table tbl_protectmode_tbl8 add partition (p='p1'); +alter table tbl_protectmode_tbl8 drop partition (p='p1'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q new file mode 100644 index 0000000000000..a4ef2acbfd406 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q @@ -0,0 +1,9 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl_protectmode__no_drop; + +create table tbl_protectmode__no_drop (col string); +select * from tbl_protectmode__no_drop; +alter table tbl_protectmode__no_drop enable no_drop; +desc extended tbl_protectmode__no_drop; +drop table tbl_protectmode__no_drop; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q new file mode 100644 index 0000000000000..542367ace22e0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q @@ -0,0 +1,20 @@ +DROP TABLE part; + +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; + +-- testAggrFuncsWithNoGBYNoPartDef +select p_mfgr, +sum(p_retailprice) as s1 +from part; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q new file mode 100644 index 0000000000000..95b35113e3cf0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q @@ -0,0 +1,28 @@ +DROP TABLE part; + +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; + +-- testAmbiguousWindowDefn +select p_mfgr, p_name, p_size, +sum(p_size) over (w1) as s1, +sum(p_size) over (w2) as s2, +sum(p_size) over (w3) as s3 +from part +distribute by p_mfgr +sort by p_mfgr +window w1 as (rows between 2 preceding and 2 following), + w2 as (rows between unbounded preceding and current row), + w3 as w3; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q new file mode 100644 index 0000000000000..caebebf8eaa4c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q @@ -0,0 +1,19 @@ +DROP TABLE part; + +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 +); + +-- testPartitonBySortBy +select p_mfgr, p_name, p_size, +sum(p_retailprice) over (distribute by p_mfgr order by p_mfgr) as s1 +from part +; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q new file mode 100644 index 0000000000000..3a0304188d2ac --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q @@ -0,0 +1,22 @@ +DROP TABLE part; + +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 +); + +-- testDuplicateWindowAlias +select p_mfgr, p_name, p_size, +sum(p_size) over (w1) as s1, +sum(p_size) over (w2) as s2 +from part +window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), + w2 as w1, + w2 as (rows between unbounded preceding and current row); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q new file mode 100644 index 0000000000000..f351a1448b150 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q @@ -0,0 +1,20 @@ +DROP TABLE part; + +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 +); + +-- testHavingLeadWithNoGBYNoWindowing +select p_mfgr,p_name, p_size +from part +having lead(p_size, 1) over() <= p_size +distribute by p_mfgr +sort by p_name; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q new file mode 100644 index 0000000000000..d0d3d3fae23f4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q @@ -0,0 +1,22 @@ +DROP TABLE part; + +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 +); + +-- testHavingLeadWithPTF +select p_mfgr,p_name, p_size +from noop(on part +partition by p_mfgr +order by p_name) +having lead(p_size, 1) over() <= p_size +distribute by p_mfgr +sort by p_name; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q new file mode 100644 index 0000000000000..40a39cb68b5e3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q @@ -0,0 +1,21 @@ +DROP TABLE part; + +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, + p_complex array +); + +-- testInvalidValueBoundary +select p_mfgr,p_name, p_size, +sum(p_size) over (w1) as s , +dense_rank() over(w1) as dr +from part +window w1 as (partition by p_mfgr order by p_complex range between 2 preceding and current row); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q new file mode 100644 index 0000000000000..80441e4f571f8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q @@ -0,0 +1,20 @@ +DROP TABLE part; + +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 +); + +-- testJoinWithAmbigousAlias +select abc.* +from noop(on part +partition by p_mfgr +order by p_name +) abc join part on abc.p_partkey = p1.p_partkey; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q new file mode 100644 index 0000000000000..1c98b8743cd72 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q @@ -0,0 +1,19 @@ +DROP TABLE part; + +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 +); + +-- testPartitonBySortBy +select p_mfgr, p_name, p_size, +sum(p_retailprice) over (partition by p_mfgr sort by p_mfgr) as s1 +from part +; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q new file mode 100644 index 0000000000000..8f4a21bd6c966 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q @@ -0,0 +1,21 @@ +DROP TABLE part; + +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 +); + +-- testWhereWithRankCond +select p_mfgr,p_name, p_size, +rank() over() as r +from part +where r < 4 +distribute by p_mfgr +sort by p_mfgr; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries.q new file mode 100644 index 0000000000000..ddab4367bb66b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries.q @@ -0,0 +1,17 @@ +-- 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 +); + +select p_mfgr, p_name, p_size, + sum(p_retailprice) over (rows unbounded following) as s1 + from part distribute by p_mfgr sort by p_name; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q b/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q new file mode 100644 index 0000000000000..16cb52ca8414e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q @@ -0,0 +1,17 @@ +-- 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 +); + +select p_mfgr, p_name, p_size, + sum(p_retailprice) over (range unbounded following) as s1 + from part distribute by p_mfgr sort by p_name; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/recursive_view.q b/src/test/hive/ql/src/test/queries/clientnegative/recursive_view.q new file mode 100644 index 0000000000000..590523e9b6259 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/recursive_view.q @@ -0,0 +1,15 @@ +-- Can't have recursive views + +drop table t; +drop view r0; +drop view r1; +drop view r2; +drop view r3; +create table t (id int); +create view r0 as select * from t; +create view r1 as select * from r0; +create view r2 as select * from r1; +create view r3 as select * from r2; +drop view r0; +alter view r3 rename to r0; +select * from r0; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/regex_col_1.q b/src/test/hive/ql/src/test/queries/clientnegative/regex_col_1.q new file mode 100644 index 0000000000000..8333ddc948419 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/regex_col_1.q @@ -0,0 +1,2 @@ +EXPLAIN +SELECT `+++` FROM srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/regex_col_2.q b/src/test/hive/ql/src/test/queries/clientnegative/regex_col_2.q new file mode 100644 index 0000000000000..d1aa1f1a9542d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/regex_col_2.q @@ -0,0 +1,2 @@ +EXPLAIN +SELECT `.a.` FROM srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/regex_col_groupby.q b/src/test/hive/ql/src/test/queries/clientnegative/regex_col_groupby.q new file mode 100644 index 0000000000000..53971916e6c96 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/regex_col_groupby.q @@ -0,0 +1,2 @@ +EXPLAIN +SELECT `..`, count(1) FROM srcpart GROUP BY `..`; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/sa_fail_hook3.q b/src/test/hive/ql/src/test/queries/clientnegative/sa_fail_hook3.q new file mode 100644 index 0000000000000..e54201c09e6f5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/sa_fail_hook3.q @@ -0,0 +1,4 @@ +create table mp2 (a string) partitioned by (b string); +alter table mp2 add partition (b='1'); +alter table mp2 partition (b='1') enable NO_DROP; +alter table mp2 drop partition (b='1'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/sample.q b/src/test/hive/ql/src/test/queries/clientnegative/sample.q new file mode 100644 index 0000000000000..0086352f8c47c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/sample.q @@ -0,0 +1 @@ +explain extended SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 5 OUT OF 4 on key) s \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe1.q b/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe1.q new file mode 100644 index 0000000000000..6b1c09decfb35 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe1.q @@ -0,0 +1,3 @@ +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/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe2.q b/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe2.q new file mode 100644 index 0000000000000..1c3093c0e7023 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe2.q @@ -0,0 +1,3 @@ +set hive.exec.script.allow.partial.consumption = false; +-- Tests exception in ScriptOperator.processOp() by passing extra data needed to fill pipe buffer +SELECT TRANSFORM(key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value) USING 'true' as a,b,c,d FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe3.q b/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe3.q new file mode 100644 index 0000000000000..60f93d209802a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe3.q @@ -0,0 +1,3 @@ +set hive.exec.script.allow.partial.consumption = true; +-- Test to ensure that a script with a bad error code still fails even with partial consumption +SELECT TRANSFORM(*) USING 'false' AS a, b FROM (SELECT TRANSFORM(*) USING 'echo' AS a, b FROM src LIMIT 1) tmp; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/script_error.q b/src/test/hive/ql/src/test/queries/clientnegative/script_error.q new file mode 100644 index 0000000000000..e46aed03b147f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/script_error.q @@ -0,0 +1,7 @@ +EXPLAIN +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) +FROM src; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/select_charliteral.q b/src/test/hive/ql/src/test/queries/clientnegative/select_charliteral.q new file mode 100644 index 0000000000000..1e4c70e663f05 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/select_charliteral.q @@ -0,0 +1,3 @@ +-- Check that charSetLiteral syntax conformance +-- Check that a sane error message with correct line/column numbers is emitted with helpful context tokens. +select _c17, count(1) from tmp_tl_foo group by _c17 diff --git a/src/test/hive/ql/src/test/queries/clientnegative/select_udtf_alias.q b/src/test/hive/ql/src/test/queries/clientnegative/select_udtf_alias.q new file mode 100644 index 0000000000000..8ace4414fc14e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/select_udtf_alias.q @@ -0,0 +1,3 @@ +-- Check alias count for SELECT UDTF() syntax: +-- explode returns a table with only 1 col - should be an error if query specifies >1 col aliases +SELECT explode(array(1,2,3)) AS (myCol1, myCol2) LIMIT 3; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/semijoin1.q b/src/test/hive/ql/src/test/queries/clientnegative/semijoin1.q new file mode 100644 index 0000000000000..06e6cad34b4d8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/semijoin1.q @@ -0,0 +1,2 @@ +-- reference rhs of semijoin in select-clause +select b.value from src a left semi join src b on (b.key = a.key and b.key = '100'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/semijoin2.q b/src/test/hive/ql/src/test/queries/clientnegative/semijoin2.q new file mode 100644 index 0000000000000..46faae641640d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/semijoin2.q @@ -0,0 +1,2 @@ +-- rhs table reference in the where clause +select a.value from src a left semi join src b on a.key = b.key where b.value = 'val_18'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/semijoin3.q b/src/test/hive/ql/src/test/queries/clientnegative/semijoin3.q new file mode 100644 index 0000000000000..35b455a7292d1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/semijoin3.q @@ -0,0 +1,2 @@ +-- rhs table reference in group by +select * from src a left semi join src b on a.key = b.key group by b.value; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/semijoin4.q b/src/test/hive/ql/src/test/queries/clientnegative/semijoin4.q new file mode 100644 index 0000000000000..4e52ebfb3cdeb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/semijoin4.q @@ -0,0 +1,3 @@ +-- rhs table is a view and reference the view in where clause +select a.value from src a left semi join (select key , value from src where key > 100) b on a.key = b.key where b.value = 'val_108' ; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/serde_regex.q b/src/test/hive/ql/src/test/queries/clientnegative/serde_regex.q new file mode 100644 index 0000000000000..13b3f165b9684 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/serde_regex.q @@ -0,0 +1,17 @@ +USE default; +-- This should fail because Regex SerDe doesn't support STRUCT +CREATE TABLE serde_regex( + host STRING, + identity STRING, + user STRING, + time TIMESTAMP, + request STRING, + status INT, + size INT, + referer STRING, + agent STRING, + strct STRUCT) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' +WITH SERDEPROPERTIES ( + "input.regex" = "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?") +STORED AS TEXTFILE; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/serde_regex2.q b/src/test/hive/ql/src/test/queries/clientnegative/serde_regex2.q new file mode 100644 index 0000000000000..a3955744221a4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/serde_regex2.q @@ -0,0 +1,23 @@ +USE default; +-- Mismatch between the number of matching groups and columns, throw run time exception. Ideally this should throw a compile time exception. See JIRA-3023 for more details. + CREATE TABLE serde_regex( + host STRING, + identity STRING, + user STRING, + time STRING, + request STRING, + status STRING, + size STRING, + referer STRING, + agent STRING) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' +WITH SERDEPROPERTIES ( + "input.regex" = "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)" +) +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; + +-- raise an exception +SELECT * FROM serde_regex ORDER BY time; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/serde_regex3.q b/src/test/hive/ql/src/test/queries/clientnegative/serde_regex3.q new file mode 100644 index 0000000000000..5a0295c971c26 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/serde_regex3.q @@ -0,0 +1,14 @@ +USE default; +-- null input.regex, raise an exception + CREATE TABLE serde_regex( + host STRING, + identity STRING, + user STRING, + time STRING, + request STRING, + status STRING, + size STRING, + referer STRING, + agent STRING) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' +STORED AS TEXTFILE; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q b/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q new file mode 100644 index 0000000000000..4cb48664b602f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q @@ -0,0 +1,5 @@ +-- should fail: hive.join.cache.size accepts int type +desc src; + +set hive.conf.validation=true; +set hive.join.cache.size=test; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q b/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q new file mode 100644 index 0000000000000..330aafd19858c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q @@ -0,0 +1,5 @@ +-- should fail: hive.map.aggr.hash.min.reduction accepts float type +desc src; + +set hive.conf.validation=true; +set hive.map.aggr.hash.min.reduction=false; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/set_table_property.q b/src/test/hive/ql/src/test/queries/clientnegative/set_table_property.q new file mode 100644 index 0000000000000..d582aaeb386c2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/set_table_property.q @@ -0,0 +1,4 @@ +create table testTable(col1 int, col2 int); + +-- set a table property = null, it should be caught by the grammar +alter table testTable set tblproperties ('a'=); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_columns1.q b/src/test/hive/ql/src/test/queries/clientnegative/show_columns1.q new file mode 100644 index 0000000000000..25705dc3d5275 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_columns1.q @@ -0,0 +1,2 @@ +SHOW COLUMNS from shcol_test; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_columns2.q b/src/test/hive/ql/src/test/queries/clientnegative/show_columns2.q new file mode 100644 index 0000000000000..c55b449a0b5ff --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_columns2.q @@ -0,0 +1,2 @@ +SHOW COLUMNS from shcol_test foo; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_columns3.q b/src/test/hive/ql/src/test/queries/clientnegative/show_columns3.q new file mode 100644 index 0000000000000..508a786609d8b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_columns3.q @@ -0,0 +1,7 @@ +CREATE DATABASE test_db; +USE test_db; +CREATE TABLE foo(a INT); + +use default; +SHOW COLUMNS from test_db.foo from test_db; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q b/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q new file mode 100644 index 0000000000000..83e5093aa1f21 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q @@ -0,0 +1,2 @@ +SHOW CREATE TABLE tmp_nonexist; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_index.q b/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_index.q new file mode 100644 index 0000000000000..0dd0ef9a255bf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_index.q @@ -0,0 +1,6 @@ +CREATE TABLE tmp_showcrt (key int, value string); +CREATE INDEX tmp_index on table tmp_showcrt(key) as 'compact' WITH DEFERRED REBUILD; +SHOW CREATE TABLE default__tmp_showcrt_tmp_index__; +DROP INDEX tmp_index on tmp_showcrt; +DROP TABLE tmp_showcrt; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_partitions1.q b/src/test/hive/ql/src/test/queries/clientnegative/show_partitions1.q new file mode 100644 index 0000000000000..71f68c894f2a0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_partitions1.q @@ -0,0 +1 @@ +SHOW PARTITIONS NonExistentTable; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tableproperties1.q b/src/test/hive/ql/src/test/queries/clientnegative/show_tableproperties1.q new file mode 100644 index 0000000000000..254a1d3a5ac3c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_tableproperties1.q @@ -0,0 +1 @@ +SHOW TBLPROPERTIES NonExistentTable; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad1.q b/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad1.q new file mode 100644 index 0000000000000..1bc94d6392c67 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad1.q @@ -0,0 +1 @@ +SHOW TABLES JOIN; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad2.q b/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad2.q new file mode 100644 index 0000000000000..5e828b647ac38 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad2.q @@ -0,0 +1 @@ +SHOW TABLES FROM default LIKE a b; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db1.q b/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db1.q new file mode 100644 index 0000000000000..d0141f6c291cb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db1.q @@ -0,0 +1 @@ +SHOW TABLES FROM nonexistent; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db2.q b/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db2.q new file mode 100644 index 0000000000000..ee0deba87a948 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db2.q @@ -0,0 +1 @@ +SHOW TABLES FROM nonexistent LIKE 'test'; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus.q b/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus.q new file mode 100644 index 0000000000000..283b5836e27fa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus.q @@ -0,0 +1 @@ +SHOW TABLE EXTENDED LIKE `srcpar*` PARTITION(ds='2008-04-08', hr=11); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q b/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q new file mode 100644 index 0000000000000..242e165285547 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q @@ -0,0 +1 @@ +SHOW TABLE EXTENDED LIKE `srcpart` PARTITION(ds='2008-14-08', hr=11); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q b/src/test/hive/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q new file mode 100644 index 0000000000000..880323c604b66 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q @@ -0,0 +1,23 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + + +CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; + + +CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; + +insert overwrite table smb_bucket4_1 +select * from src; + +insert overwrite table smb_bucket4_2 +select * from src; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +select /*+mapjoin(a)*/ * from smb_bucket4_1 a left outer join smb_bucket4_2 b on a.key = b.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/smb_mapjoin_14.q b/src/test/hive/ql/src/test/queries/clientnegative/smb_mapjoin_14.q new file mode 100644 index 0000000000000..54bfba03d82d7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/smb_mapjoin_14.q @@ -0,0 +1,38 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +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; + +insert overwrite table tbl1 +select * from src where key < 10; + +insert overwrite table tbl2 +select * from src where key < 10; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +-- A join is being performed across different sub-queries, where a mapjoin is being performed in each of them. +-- Each sub-query should be converted to a sort-merge join. +-- A join followed by mapjoin is not allowed, so this query should fail. +-- Once HIVE-3403 is in, this should be automatically converted to a sort-merge join without the hint +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select /*+mapjoin(a)*/ 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 /*+mapjoin(a)*/ 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/src/test/hive/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q b/src/test/hive/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q new file mode 100644 index 0000000000000..7d11f450edfde --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q @@ -0,0 +1,28 @@ +create table table_asc(key int, value string) CLUSTERED BY (key) SORTED BY (key asc) +INTO 1 BUCKETS STORED AS RCFILE; +create table table_desc(key int, value string) CLUSTERED BY (key) SORTED BY (key desc) +INTO 1 BUCKETS STORED AS RCFILE; + +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; + +insert overwrite table table_asc select key, value from src; +insert overwrite table table_desc select key, value from src; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +-- If the user asked for sort merge join to be enforced (by setting +-- hive.enforce.sortmergebucketmapjoin to true), an error should be thrown, since +-- one of the tables is in ascending order and the other is in descending order, +-- and sort merge bucket mapjoin cannot be performed. In the default mode, the +-- query would succeed, although a regular map-join would be performed instead of +-- what the user asked. + +explain +select /*+mapjoin(a)*/ * from table_asc a join table_desc b on a.key = b.key; + +set hive.enforce.sortmergebucketmapjoin=true; + +explain +select /*+mapjoin(a)*/ * from table_asc a join table_desc b on a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/split_sample_out_of_range.q b/src/test/hive/ql/src/test/queries/clientnegative/split_sample_out_of_range.q new file mode 100644 index 0000000000000..66af1fd7da682 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/split_sample_out_of_range.q @@ -0,0 +1,3 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +select key from src tablesample(105 percent); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format.q b/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format.q new file mode 100644 index 0000000000000..f71cc4487910a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format.q @@ -0,0 +1,3 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +select key from src tablesample(1 percent); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q b/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q new file mode 100644 index 0000000000000..1a13c0ff4cb28 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q @@ -0,0 +1,3 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +select key from src tablesample(1K); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q b/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q new file mode 100644 index 0000000000000..401cc37f67dd0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q @@ -0,0 +1,18 @@ +-- In this test, there is a dummy stats aggregator which throws an error when the +-- method connect is called (as indicated by the parameter hive.test.dummystats.aggregator) +-- If stats need not be reliable, the statement succeeds. However, if stats are supposed +-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails +-- because stats cannot be collected for this statement + +create table tmptable(key string, value string); + +set hive.stats.dbclass=dummy; +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; + +set hive.stats.reliable=false; +INSERT OVERWRITE TABLE tmptable select * from src; + +set hive.stats.reliable=true; +INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q b/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q new file mode 100644 index 0000000000000..c7e63591adde0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q @@ -0,0 +1,16 @@ +-- In this test, the stats aggregator does not exists. +-- If stats need not be reliable, the statement succeeds. However, if stats are supposed +-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails +-- because stats cannot be collected for this statement + +create table tmptable(key string, value string); + +set hive.stats.dbclass=dummy; +set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; +set hive.stats.default.aggregator=""; + +set hive.stats.reliable=false; +INSERT OVERWRITE TABLE tmptable select * from src; + +set hive.stats.reliable=true; +INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_noscan_non_native.q b/src/test/hive/ql/src/test/queries/clientnegative/stats_noscan_non_native.q new file mode 100644 index 0000000000000..bde66278360c5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/stats_noscan_non_native.q @@ -0,0 +1,6 @@ + +CREATE TABLE non_native1(key int, value string) +STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; + +-- we do not support analyze table ... noscan on non-native tables yet +analyze table non_native1 compute statistics noscan; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q b/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q new file mode 100644 index 0000000000000..47a8148e0869c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q @@ -0,0 +1,31 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set mapred.min.split.size=256; +set mapred.min.split.size.per.node=256; +set mapred.min.split.size.per.rack=256; +set mapred.max.split.size=256; + +-- test analyze table ... compute statistics partialscan + +-- 1. prepare data +CREATE table analyze_srcpart_partial_scan (key STRING, value STRING) +partitioned by (ds string, hr string) +stored as rcfile; +insert overwrite table analyze_srcpart_partial_scan partition (ds, hr) select * from srcpart where ds is not null; +describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11); + + +-- 2. partialscan +explain +analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; +analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; + +-- 3. confirm result +describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11); +describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-09',hr=11); +drop table analyze_srcpart_partial_scan; + + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q b/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q new file mode 100644 index 0000000000000..c206b8b5d765e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q @@ -0,0 +1,5 @@ + +CREATE EXTERNAL TABLE external_table (key int, value string); + +-- we do not support analyze table ... partialscan on EXTERNAL tables yet +analyze table external_table compute statistics partialscan; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q b/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q new file mode 100644 index 0000000000000..8e02ced85e708 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q @@ -0,0 +1,6 @@ + +CREATE TABLE non_native1(key int, value string) +STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; + +-- we do not support analyze table ... partialscan on non-native tables yet +analyze table non_native1 compute statistics partialscan; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q b/src/test/hive/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q new file mode 100644 index 0000000000000..56d93d08aa697 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q @@ -0,0 +1,12 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +-- test analyze table ... compute statistics partialscan + +create table analyze_srcpart_partial_scan like srcpart; +insert overwrite table analyze_srcpart_partial_scan partition (ds, hr) select * from srcpart where ds is not null; +analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_1.q b/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_1.q new file mode 100644 index 0000000000000..7fa0f55f2a450 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_1.q @@ -0,0 +1,18 @@ +-- In this test, there is a dummy stats publisher which throws an error when the +-- method connect is called (as indicated by the parameter hive.test.dummystats.publisher) +-- If stats need not be reliable, the statement succeeds. However, if stats are supposed +-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails +-- because stats cannot be collected for this statement + +create table tmptable(key string, value string); + +set hive.stats.dbclass=dummy; +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; + +set hive.stats.reliable=false; +INSERT OVERWRITE TABLE tmptable select * from src; + +set hive.stats.reliable=true; +INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_2.q b/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_2.q new file mode 100644 index 0000000000000..f82d4b54b697b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_2.q @@ -0,0 +1,16 @@ +-- In this test, the stats publisher does not exists. +-- If stats need not be reliable, the statement succeeds. However, if stats are supposed +-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails +-- because stats cannot be collected for this statement + +create table tmptable(key string, value string); + +set hive.stats.dbclass=dummy; +set hive.stats.default.publisher=""; +set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; + +set hive.stats.reliable=false; +INSERT OVERWRITE TABLE tmptable select * from src; + +set hive.stats.reliable=true; +INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/strict_join.q b/src/test/hive/ql/src/test/queries/clientnegative/strict_join.q new file mode 100644 index 0000000000000..d618ee28fdb2e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/strict_join.q @@ -0,0 +1,3 @@ +set hive.mapred.mode=strict; + +SELECT * FROM src src1 JOIN src src2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/strict_orderby.q b/src/test/hive/ql/src/test/queries/clientnegative/strict_orderby.q new file mode 100644 index 0000000000000..781cdbb05088b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/strict_orderby.q @@ -0,0 +1,7 @@ +set hive.mapred.mode=strict; + +EXPLAIN +SELECT src.key, src.value from src order by src.key; + +SELECT src.key, src.value from src order by src.key; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/strict_pruning.q b/src/test/hive/ql/src/test/queries/clientnegative/strict_pruning.q new file mode 100644 index 0000000000000..270ab2f593ac2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/strict_pruning.q @@ -0,0 +1,6 @@ +set hive.mapred.mode=strict; + +EXPLAIN +SELECT count(1) FROM srcPART; + +SELECT count(1) FROM srcPART; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/subq_insert.q b/src/test/hive/ql/src/test/queries/clientnegative/subq_insert.q new file mode 100644 index 0000000000000..0bc9e24e4828c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/subq_insert.q @@ -0,0 +1,2 @@ +EXPLAIN +SELECT * FROM (INSERT OVERWRITE TABLE src1 SELECT * FROM src ) y; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/touch1.q b/src/test/hive/ql/src/test/queries/clientnegative/touch1.q new file mode 100644 index 0000000000000..9efbba0082b6f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/touch1.q @@ -0,0 +1 @@ +ALTER TABLE srcpart TOUCH PARTITION (ds='2008-04-08', hr='13'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/touch2.q b/src/test/hive/ql/src/test/queries/clientnegative/touch2.q new file mode 100644 index 0000000000000..923a171e0482e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/touch2.q @@ -0,0 +1 @@ +ALTER TABLE src TOUCH PARTITION (ds='2008-04-08', hr='12'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_bucketed_column.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_bucketed_column.q new file mode 100644 index 0000000000000..e53665695a39d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_bucketed_column.q @@ -0,0 +1,7 @@ +-- Tests truncating a bucketed column + +CREATE TABLE test_tab (key STRING, value STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS RCFILE; + +INSERT OVERWRITE TABLE test_tab SELECT * FROM src; + +TRUNCATE TABLE test_tab COLUMNS (key); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q new file mode 100644 index 0000000000000..13f32c8968a16 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q @@ -0,0 +1,9 @@ +-- Tests truncating a column from an indexed table + +CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; + +INSERT OVERWRITE TABLE test_tab SELECT * FROM src; + +CREATE INDEX test_tab_index ON TABLE test_tab (key) as 'COMPACT' WITH DEFERRED REBUILD; + +TRUNCATE TABLE test_tab COLUMNS (value); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q new file mode 100644 index 0000000000000..0ece6007f7b61 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q @@ -0,0 +1,14 @@ +set hive.mapred.supports.subdirectories=true; +set mapred.input.dir.recursive=true; + +-- Tests truncating a column on which a table is list bucketed + +CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; + +ALTER TABLE test_tab +SKEWED BY (key) ON ("484") +STORED AS DIRECTORIES; + +INSERT OVERWRITE TABLE test_tab SELECT * FROM src; + +TRUNCATE TABLE test_tab COLUMNS (key); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_seqfile.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_seqfile.q new file mode 100644 index 0000000000000..903540dae8985 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_seqfile.q @@ -0,0 +1,7 @@ +-- Tests truncating a column from a table stored as a sequence file + +CREATE TABLE test_tab (key STRING, value STRING) STORED AS SEQUENCEFILE; + +INSERT OVERWRITE TABLE test_tab SELECT * FROM src; + +TRUNCATE TABLE test_tab COLUMNS (key); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q new file mode 100644 index 0000000000000..5509552811b09 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q @@ -0,0 +1,7 @@ +-- Tests attempting to truncate a column in a table that doesn't exist + +CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; + +INSERT OVERWRITE TABLE test_tab SELECT * FROM src; + +TRUNCATE TABLE test_tab COLUMNS (doesnt_exist); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column.q new file mode 100644 index 0000000000000..134743ac13a5d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column.q @@ -0,0 +1,7 @@ +-- Tests truncating a partition column + +CREATE TABLE test_tab (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE; + +INSERT OVERWRITE TABLE test_tab PARTITION (part = '1') SELECT * FROM src; + +TRUNCATE TABLE test_tab COLUMNS (part); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column2.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column2.q new file mode 100644 index 0000000000000..47635208a7810 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column2.q @@ -0,0 +1,7 @@ +-- Tests truncating a partition column + +CREATE TABLE test_tab (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE; + +INSERT OVERWRITE TABLE test_tab PARTITION (part = '1') SELECT * FROM src; + +TRUNCATE TABLE test_tab PARTITION (part = '1') COLUMNS (part); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure1.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure1.q new file mode 100644 index 0000000000000..f6cfa44bbb126 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure1.q @@ -0,0 +1,2 @@ +-- partition spec for non-partitioned table +TRUNCATE TABLE src partition (ds='2008-04-08', hr='11'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure2.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure2.q new file mode 100644 index 0000000000000..1137d893eb0e1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure2.q @@ -0,0 +1,2 @@ +-- full partition spec for not existing partition +TRUNCATE TABLE srcpart partition (ds='2012-12-17', hr='15'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure3.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure3.q new file mode 100644 index 0000000000000..c5cf58775b309 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure3.q @@ -0,0 +1,4 @@ +create external table external1 (a int, b int) partitioned by (ds string); + +-- trucate for non-managed table +TRUNCATE TABLE external1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure4.q b/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure4.q new file mode 100644 index 0000000000000..a7f1e92d55980 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure4.q @@ -0,0 +1,5 @@ +CREATE TABLE non_native(key int, value string) +STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; + +-- trucate for non-native table +TRUNCATE TABLE non_native; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udaf_invalid_place.q b/src/test/hive/ql/src/test/queries/clientnegative/udaf_invalid_place.q new file mode 100644 index 0000000000000..f37ce72ae4197 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udaf_invalid_place.q @@ -0,0 +1 @@ +select distinct key, sum(key) from src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q new file mode 100644 index 0000000000000..c2a132d4db05e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q @@ -0,0 +1,2 @@ +-- invalid first argument +SELECT array_contains(1, 2) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q new file mode 100644 index 0000000000000..36f85d34a6e0b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q @@ -0,0 +1,2 @@ +-- invalid second argument +SELECT array_contains(array(1, 2, 3), '2') FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true.q new file mode 100644 index 0000000000000..73b3f9654f1c3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true.q @@ -0,0 +1,7 @@ +DESCRIBE FUNCTION ASSERT_TRUE; + +EXPLAIN SELECT ASSERT_TRUE(x > 0) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; +SELECT ASSERT_TRUE(x > 0) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; + +EXPLAIN SELECT ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; +SELECT ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true2.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true2.q new file mode 100644 index 0000000000000..4b62220764bb9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true2.q @@ -0,0 +1,2 @@ +EXPLAIN SELECT 1 + ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; +SELECT 1 + ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong.q new file mode 100644 index 0000000000000..2fb5ff74cc3ce --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong.q @@ -0,0 +1,6 @@ +SELECT CASE '1' + WHEN 1 THEN 2 + WHEN 3 THEN 4 + ELSE 5 + END +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q new file mode 100644 index 0000000000000..5772dc1a95c9d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q @@ -0,0 +1,6 @@ +SELECT CASE 1 + WHEN 1 THEN '2' + WHEN 3 THEN 4 + ELSE 5 + END +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q new file mode 100644 index 0000000000000..5aaf0188eb9c6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q @@ -0,0 +1,6 @@ +SELECT CASE 1 + WHEN 1 THEN NULL + WHEN 3 THEN '2' + ELSE 7 + END +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_coalesce.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_coalesce.q new file mode 100644 index 0000000000000..7405e387caf70 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_coalesce.q @@ -0,0 +1 @@ +SELECT COALESCE(array('a', 'b'), '2.0') FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q new file mode 100644 index 0000000000000..8c2017bc636cf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q @@ -0,0 +1,2 @@ +-- invalid argument number +SELECT concat_ws('-') FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q new file mode 100644 index 0000000000000..c49e7868bbb5e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q @@ -0,0 +1,2 @@ +-- invalid argument type +SELECT concat_ws('[]', array(100, 200, 50)) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q new file mode 100644 index 0000000000000..72b86271f5ea6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q @@ -0,0 +1,2 @@ +-- invalid argument type +SELECT concat_ws(1234, array('www', 'facebook', 'com')) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q new file mode 100644 index 0000000000000..fbe4902d644cc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q @@ -0,0 +1 @@ +SELECT elt(3) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q new file mode 100644 index 0000000000000..bb1fdbf789e3e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q @@ -0,0 +1,3 @@ +FROM src_thrift +SELECT elt(1, src_thrift.lintstring) +WHERE src_thrift.lintstring IS NOT NULL; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q new file mode 100644 index 0000000000000..9703c82d8a4d9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q @@ -0,0 +1 @@ +SELECT field(3) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_type.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_type.q new file mode 100644 index 0000000000000..61b2cd06496e2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_type.q @@ -0,0 +1,3 @@ +FROM src_thrift +SELECT field(1, src_thrift.lintstring) +WHERE src_thrift.lintstring IS NOT NULL; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q new file mode 100644 index 0000000000000..18c985c606849 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q @@ -0,0 +1,2 @@ +-- invalid argument length +SELECT format_number(12332.123456) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q new file mode 100644 index 0000000000000..7959c20b28e56 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q @@ -0,0 +1,2 @@ +-- invalid argument length +SELECT format_number(12332.123456, 2, 3) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q new file mode 100644 index 0000000000000..7d90ef86da7b7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q @@ -0,0 +1,2 @@ +-- invalid argument(second argument should be >= 0) +SELECT format_number(12332.123456, -4) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q new file mode 100644 index 0000000000000..e545f4aa1420c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q @@ -0,0 +1,2 @@ +-- invalid argument type +SELECT format_number(12332.123456, 4.01) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q new file mode 100644 index 0000000000000..a6f71778f143b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q @@ -0,0 +1,2 @@ +-- invalid argument type +SELECT format_number(array(12332.123456, 321.23), 5) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q new file mode 100644 index 0000000000000..e5b11b9b71ee7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q @@ -0,0 +1,2 @@ +-- invalid argument type +SELECT format_number(12332.123456, "4") FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q new file mode 100644 index 0000000000000..aa4a3a44751c4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q @@ -0,0 +1,2 @@ +-- invalid argument type(format_number returns the result as a string) +SELECT format_number(format_number(12332.123456, 4), 2) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q new file mode 100644 index 0000000000000..21ca6e7d36255 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q @@ -0,0 +1 @@ +CREATE TEMPORARY FUNCTION moo AS 'org.apache.hadoop.hive.ql.Driver'; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_if_not_bool.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_if_not_bool.q new file mode 100644 index 0000000000000..74458d0c3db2c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_if_not_bool.q @@ -0,0 +1 @@ +SELECT IF('STRING', 1, 1) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q new file mode 100644 index 0000000000000..ad19364c33076 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q @@ -0,0 +1 @@ +SELECT IF(TRUE, 1) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_in.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_in.q new file mode 100644 index 0000000000000..ce9ce54fac680 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_in.q @@ -0,0 +1 @@ +SELECT 3 IN (array(1,2,3)) FROM src; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q new file mode 100644 index 0000000000000..ac8253fb1e941 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q @@ -0,0 +1 @@ +SELECT instr('abcd') FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q new file mode 100644 index 0000000000000..9ac3ed6614897 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q @@ -0,0 +1,3 @@ +FROM src_thrift +SELECT instr('abcd', src_thrift.lintstring) +WHERE src_thrift.lintstring IS NOT NULL; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q new file mode 100644 index 0000000000000..ca7caad54d646 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q @@ -0,0 +1 @@ +SELECT locate('a', 'b', 1, 2) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q new file mode 100644 index 0000000000000..4bbf79a310b01 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q @@ -0,0 +1,3 @@ +FROM src_thrift +SELECT locate('abcd', src_thrift.lintstring) +WHERE src_thrift.lintstring IS NOT NULL; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q new file mode 100644 index 0000000000000..ebb6c2ab418eb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q @@ -0,0 +1 @@ +SELECT map_keys(map("a", "1"), map("b", "2")) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q new file mode 100644 index 0000000000000..0757d1494f3cd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q @@ -0,0 +1 @@ +SELECT map_keys(array(1, 2, 3)) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q new file mode 100644 index 0000000000000..c97476a1263ee --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q @@ -0,0 +1 @@ +SELECT map_values(map("a", "1"), map("b", "2")) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q new file mode 100644 index 0000000000000..cc060ea0f0ec7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q @@ -0,0 +1 @@ +SELECT map_values(array(1, 2, 3, 4)) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_max.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_max.q new file mode 100644 index 0000000000000..7282e07596032 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_max.q @@ -0,0 +1,2 @@ +SELECT max(map("key", key, "value", value)) +FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_min.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_min.q new file mode 100644 index 0000000000000..b9528fa6dafef --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_min.q @@ -0,0 +1,2 @@ +SELECT min(map("key", key, "value", value)) +FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong1.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong1.q new file mode 100644 index 0000000000000..88ca4fefc3055 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong1.q @@ -0,0 +1,2 @@ +-- invalid argument length +SELECT printf() FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong2.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong2.q new file mode 100644 index 0000000000000..01ed2ffcf017f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong2.q @@ -0,0 +1,2 @@ +-- invalid argument type +SELECT printf(100) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong3.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong3.q new file mode 100644 index 0000000000000..71f118b8dc0dd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong3.q @@ -0,0 +1,2 @@ +-- invalid argument type +SELECT printf("Hello World %s", array("invalid", "argument")) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong4.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong4.q new file mode 100644 index 0000000000000..71f118b8dc0dd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong4.q @@ -0,0 +1,2 @@ +-- invalid argument type +SELECT printf("Hello World %s", array("invalid", "argument")) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_reflect_neg.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_reflect_neg.q new file mode 100644 index 0000000000000..67efb64505d9e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_reflect_neg.q @@ -0,0 +1,9 @@ +SELECT reflect("java.lang.StringClassThatDoesNotExist", "valueOf", 1), + reflect("java.lang.String", "methodThatDoesNotExist"), + reflect("java.lang.Math", "max", "overloadthatdoesnotexist", 3), + reflect("java.lang.Math", "min", 2, 3), + reflect("java.lang.Math", "round", 2.5), + reflect("java.lang.Math", "exp", 1.0), + reflect("java.lang.Math", "floor", 1.9) +FROM src LIMIT 1; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q new file mode 100644 index 0000000000000..c628ff8aa1972 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q @@ -0,0 +1,5 @@ +FROM src_thrift +SELECT size(src_thrift.lint, src_thrift.lintstring), + size() +WHERE src_thrift.lint IS NOT NULL + AND NOT (src_thrift.mstringstring IS NULL) LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_type.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_type.q new file mode 100644 index 0000000000000..16695f6adc3f4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_type.q @@ -0,0 +1 @@ +SELECT SIZE('wrong type: string') FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q new file mode 100644 index 0000000000000..9954f4ab4d3c5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q @@ -0,0 +1,2 @@ +-- invalid argument number +SELECT sort_array(array(2, 5, 4), 3) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q new file mode 100644 index 0000000000000..32c2645519497 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q @@ -0,0 +1,2 @@ +-- invalid argument type +SELECT sort_array("Invalid") FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q new file mode 100644 index 0000000000000..034de06b8e391 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q @@ -0,0 +1,2 @@ +-- invalid argument type +SELECT sort_array(array(array(10, 20), array(5, 15), array(3, 13))) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error.q new file mode 100644 index 0000000000000..846f87c2e51b1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error.q @@ -0,0 +1,3 @@ +CREATE TEMPORARY FUNCTION test_error AS 'org.apache.hadoop.hive.ql.udf.UDFTestErrorOnFalse'; + +SELECT test_error(key < 125 OR key > 130) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error_reduce.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error_reduce.q new file mode 100644 index 0000000000000..b1a06f2a07af3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error_reduce.q @@ -0,0 +1,11 @@ +CREATE TEMPORARY FUNCTION test_error AS 'org.apache.hadoop.hive.ql.udf.UDFTestErrorOnFalse'; + + +SELECT test_error(key < 125 OR key > 130) +FROM ( + SELECT * + FROM src + DISTRIBUTE BY rand() +) map_output; + + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong.q new file mode 100644 index 0000000000000..d4d2d2e48517f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong.q @@ -0,0 +1,6 @@ +SELECT CASE + WHEN TRUE THEN 2 + WHEN '1' THEN 4 + ELSE 5 + END +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q new file mode 100644 index 0000000000000..79fa65f63da56 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q @@ -0,0 +1,6 @@ +SELECT CASE + WHEN 1=2 THEN '2' + WHEN 3=4 THEN 4 + ELSE 5 + END +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q b/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q new file mode 100644 index 0000000000000..8bb5fdd7ea377 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q @@ -0,0 +1,6 @@ +SELECT CASE + WHEN 1=2 THEN '2' + WHEN 3=4 THEN '5' + ELSE 5.3 + END +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udfnull.q b/src/test/hive/ql/src/test/queries/clientnegative/udfnull.q new file mode 100644 index 0000000000000..3c4204f780004 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udfnull.q @@ -0,0 +1,6 @@ + +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/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q b/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q new file mode 100644 index 0000000000000..942ae5d8315f5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q @@ -0,0 +1 @@ +SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal) FROM src GROUP BY key; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q b/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q new file mode 100644 index 0000000000000..00d359a75ce0c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q @@ -0,0 +1 @@ +SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal,myVal2) FROM src; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q b/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q new file mode 100644 index 0000000000000..51df8fa862e1a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q @@ -0,0 +1 @@ +select explode(array(1),array(2)) as myCol from src; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q b/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q new file mode 100644 index 0000000000000..ae8dff7bad8d8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q @@ -0,0 +1 @@ +SELECT explode(null) as myNull FROM src GROUP BY key; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_invalid_place.q b/src/test/hive/ql/src/test/queries/clientnegative/udtf_invalid_place.q new file mode 100644 index 0000000000000..ab84a801e9ed6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udtf_invalid_place.q @@ -0,0 +1 @@ +select distinct key, explode(key) from src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported1.q b/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported1.q new file mode 100644 index 0000000000000..04e98d52c548a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported1.q @@ -0,0 +1 @@ +SELECT explode(array(1,2,3)) as myCol, key FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported3.q b/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported3.q new file mode 100644 index 0000000000000..f4fe0dde3e62c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported3.q @@ -0,0 +1 @@ +SELECT explode(array(1,2,3)) as myCol FROM src GROUP BY key; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/union.q b/src/test/hive/ql/src/test/queries/clientnegative/union.q new file mode 100644 index 0000000000000..e3c5c830897ea --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/union.q @@ -0,0 +1,4 @@ +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/src/test/hive/ql/src/test/queries/clientnegative/union2.q b/src/test/hive/ql/src/test/queries/clientnegative/union2.q new file mode 100644 index 0000000000000..38db488eaf68e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/union2.q @@ -0,0 +1,13 @@ + + +create table if not exists union2_t1(r string, c string, v array); +create table if not exists union2_t2(s string, c string, v string); + +explain +SELECT s.r, s.c, sum(s.v) +FROM ( + SELECT a.r AS r, a.c AS c, a.v AS v FROM union2_t1 a + UNION ALL + SELECT b.s AS r, b.c AS c, 0 + b.v AS v FROM union2_t2 b +) s +GROUP BY s.r, s.c; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/union22.q b/src/test/hive/ql/src/test/queries/clientnegative/union22.q new file mode 100644 index 0000000000000..72f3314bdac96 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/union22.q @@ -0,0 +1,26 @@ +create table dst_union22(k1 string, k2 string, k3 string, k4 string) partitioned by (ds string); +create table dst_union22_delta(k0 string, k1 string, k2 string, k3 string, k4 string, k5 string) partitioned by (ds string); + +insert overwrite table dst_union22 partition (ds='1') +select key, value, key , value from src; + +insert overwrite table dst_union22_delta partition (ds='1') +select key, key, value, key, value, value from src; + +set hive.merge.mapfiles=false; + +-- Union followed by Mapjoin is not supported. +-- The same query would work without the hint +-- Note that there is a positive test with the same name in clientpositive +explain extended +insert overwrite table dst_union22 partition (ds='2') +select * from +( +select k1 as k1, k2 as k2, k3 as k3, k4 as k4 from dst_union22_delta where ds = '1' and k0 <= 50 +union all +select /*+ MAPJOIN(b) */ a.k1 as k1, a.k2 as k2, b.k3 as k3, b.k4 as k4 +from dst_union22 a left outer join (select * from dst_union22_delta where ds = '1' and k0 > 50) b on +a.k1 = b.k1 and a.ds='1' +where a.k1 > 20 +) +subq; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/union3.q b/src/test/hive/ql/src/test/queries/clientnegative/union3.q new file mode 100644 index 0000000000000..ce657478c150d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/union3.q @@ -0,0 +1,5 @@ +-- Ensure that UNION ALL columns are in the correct order on both sides +-- Ensure that the appropriate error message is propagated +CREATE TABLE IF NOT EXISTS union3 (bar int, baz int); +SELECT * FROM ( SELECT f.bar, f.baz FROM union3 f UNION ALL SELECT b.baz, b.bar FROM union3 b ) c; +DROP TABLE union3; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin.q b/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin.q new file mode 100644 index 0000000000000..d6a19c397d80d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin.q @@ -0,0 +1,3 @@ +FROM UNIQUEJOIN (SELECT src.key from src WHERE src.key<4) a (a.key), PRESERVE src b(b.key) +SELECT a.key, b.key; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin2.q b/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin2.q new file mode 100644 index 0000000000000..6e9a08251407d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin2.q @@ -0,0 +1,3 @@ +FROM UNIQUEJOIN src a (a.key), PRESERVE src b (b.key, b.val) +SELECT a.key, b.key; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin3.q b/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin3.q new file mode 100644 index 0000000000000..89a8f1b2aaa84 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin3.q @@ -0,0 +1,3 @@ +FROM UNIQUEJOIN src a (a.key), PRESERVE src b (b.key) JOIN src c ON c.key +SELECT a.key; + diff --git a/src/test/hive/ql/src/test/queries/clientnegative/unset_table_property.q b/src/test/hive/ql/src/test/queries/clientnegative/unset_table_property.q new file mode 100644 index 0000000000000..7a24e652b46f2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/unset_table_property.q @@ -0,0 +1,6 @@ +CREATE TABLE testTable(col1 INT, col2 INT); +ALTER TABLE testTable SET TBLPROPERTIES ('a'='1', 'c'='3'); +SHOW TBLPROPERTIES testTable; + +-- unset a subset of the properties and some non-existed properties without if exists +ALTER TABLE testTable UNSET TBLPROPERTIES ('c', 'x', 'y', 'z'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/unset_view_property.q b/src/test/hive/ql/src/test/queries/clientnegative/unset_view_property.q new file mode 100644 index 0000000000000..11131006e998a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/unset_view_property.q @@ -0,0 +1,6 @@ +CREATE VIEW testView AS SELECT value FROM src WHERE key=86; +ALTER VIEW testView SET TBLPROPERTIES ('propA'='100', 'propB'='200'); +SHOW TBLPROPERTIES testView; + +-- unset a subset of the properties and some non-existed properties without if exists +ALTER VIEW testView UNSET TBLPROPERTIES ('propB', 'propX', 'propY', 'propZ'); diff --git a/src/test/hive/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q b/src/test/hive/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q new file mode 100644 index 0000000000000..b54b7a532176f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q @@ -0,0 +1,15 @@ +DROP TABLE part; + +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 sum(lead(p_retailprice,1)) as s1 from part; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/windowing_ll_no_over.q b/src/test/hive/ql/src/test/queries/clientnegative/windowing_ll_no_over.q new file mode 100644 index 0000000000000..3ca1104b0158d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/windowing_ll_no_over.q @@ -0,0 +1,17 @@ +DROP TABLE part; + +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 p_mfgr, +lead(p_retailprice,1) as s1 +from part; diff --git a/src/test/hive/ql/src/test/queries/clientnegative/wrong_column_type.q b/src/test/hive/ql/src/test/queries/clientnegative/wrong_column_type.q new file mode 100644 index 0000000000000..490f0c3b4d110 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientnegative/wrong_column_type.q @@ -0,0 +1,4 @@ +CREATE TABLE dest1(a float); + +INSERT OVERWRITE TABLE dest1 +SELECT array(1.0,2.0) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/add_part_exist.q b/src/test/hive/ql/src/test/queries/clientpositive/add_part_exist.q new file mode 100644 index 0000000000000..d176661ecb8ea --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/add_part_exist.q @@ -0,0 +1,37 @@ +CREATE TABLE add_part_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD PARTITION (ds='2010-01-01'); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01'); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-02'); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01') PARTITION (ds='2010-01-02') PARTITION (ds='2010-01-03'); +SHOW PARTITIONS add_part_test; + +DROP TABLE add_part_test; +SHOW TABLES; + +-- Test ALTER TABLE ADD PARTITION in non-default Database +CREATE DATABASE add_part_test_db; +USE add_part_test_db; +SHOW TABLES; + +CREATE TABLE add_part_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD PARTITION (ds='2010-01-01'); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01'); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-02'); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01') PARTITION (ds='2010-01-02') PARTITION (ds='2010-01-03'); +SHOW PARTITIONS add_part_test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/add_part_multiple.q b/src/test/hive/ql/src/test/queries/clientpositive/add_part_multiple.q new file mode 100644 index 0000000000000..9248a7f3b94b8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/add_part_multiple.q @@ -0,0 +1,24 @@ +-- HIVE-5122 locations for 2nd, 3rd... partition are ignored + +CREATE TABLE add_part_test (key STRING, value STRING) PARTITIONED BY (ds STRING); + +explain +ALTER TABLE add_part_test ADD IF NOT EXISTS +PARTITION (ds='2010-01-01') location 'A' +PARTITION (ds='2010-02-01') location 'B' +PARTITION (ds='2010-03-01') +PARTITION (ds='2010-04-01') location 'C'; + +ALTER TABLE add_part_test ADD IF NOT EXISTS +PARTITION (ds='2010-01-01') location 'A' +PARTITION (ds='2010-02-01') location 'B' +PARTITION (ds='2010-03-01') +PARTITION (ds='2010-04-01') location 'C'; + +from src TABLESAMPLE (1 ROWS) +insert into table add_part_test PARTITION (ds='2010-01-01') select 100,100 +insert into table add_part_test PARTITION (ds='2010-02-01') select 200,200 +insert into table add_part_test PARTITION (ds='2010-03-01') select 400,300 +insert into table add_part_test PARTITION (ds='2010-04-01') select 500,400; + +select * from add_part_test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q b/src/test/hive/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q new file mode 100644 index 0000000000000..f51c53c2ff627 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q @@ -0,0 +1,7 @@ +SET hive.metastore.partition.name.whitelist.pattern=; +-- Test with no partition name whitelist pattern + +CREATE TABLE part_nowhitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS part_nowhitelist_test; + +ALTER TABLE part_nowhitelist_test ADD PARTITION (ds='1,2,3,4'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q b/src/test/hive/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q new file mode 100644 index 0000000000000..009c7610ef917 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q @@ -0,0 +1,9 @@ +SET hive.metastore.partition.name.whitelist.pattern=[A-Za-z]*; +-- This pattern matches only letters. + +CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS part_whitelist_test; + +ALTER TABLE part_whitelist_test ADD PARTITION (ds='Part'); + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alias_casted_column.q b/src/test/hive/ql/src/test/queries/clientpositive/alias_casted_column.q new file mode 100644 index 0000000000000..f1fc91a5f2d0c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alias_casted_column.q @@ -0,0 +1,5 @@ +-- HIVE-2477 Use name of original expression for name of CAST output +explain select key from (select cast(key as int) from src )t; + +--backward +explain select key2 from (select cast(key as int) key2 from src )t; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/allcolref_in_udf.q b/src/test/hive/ql/src/test/queries/clientpositive/allcolref_in_udf.q new file mode 100644 index 0000000000000..020975cc69b94 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/allcolref_in_udf.q @@ -0,0 +1,21 @@ +explain +select concat(*),array(*) from src where key < 100 limit 10; + +select concat(*),array(*) from src where key < 100 limit 10; + +-- The order of columns is decided by row schema of prev operator +-- Like join which has two or more aliases, it's from left most aias to right aliases. + +explain +select stack(2, *) as (e1,e2,e3) from ( + select concat(*), concat(a.*), concat(b.*), concat(a.*, b.key), concat(a.key, b.*) + from src a join src b on a.key+1=b.key where a.key < 100) x limit 10; + +select stack(2, *) as (e1,e2,e3) from ( + select concat(*), concat(a.*), concat(b.*), concat(a.*, b.key), concat(a.key, b.*) + from src a join src b on a.key+1=b.key where a.key < 100) x limit 10; + +-- HIVE-4181 TOK_FUNCTIONSTAR for UDTF +create table allcolref as select array(key, value) from src; +explain select explode(*) as x from allcolref limit 10; +select explode(*) as x from allcolref limit 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter1.q b/src/test/hive/ql/src/test/queries/clientpositive/alter1.q new file mode 100644 index 0000000000000..5fd19455f93cd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter1.q @@ -0,0 +1,71 @@ +create table alter1(a int, b int); +describe extended alter1; +alter table alter1 set tblproperties ('a'='1', 'c'='3'); +describe extended alter1; +alter table alter1 set tblproperties ('a'='1', 'c'='4', 'd'='3'); +describe extended alter1; + +alter table alter1 set tblproperties ('EXTERNAL'='TRUE'); +describe extended alter1; +alter table alter1 set tblproperties ('EXTERNAL'='FALSE'); +describe extended alter1; + +alter table alter1 set serdeproperties('s1'='9'); +describe extended alter1; +alter table alter1 set serdeproperties('s1'='10', 's2' ='20'); +describe extended alter1; + +add jar ../data/files/TestSerDe.jar; +alter table alter1 set serde 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties('s1'='9'); +describe extended alter1; + +alter table alter1 set serde 'org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe'; +describe extended alter1; + +alter table alter1 replace columns (a int, b int, c string); +describe alter1; + +-- Cleanup +DROP TABLE alter1; +SHOW TABLES; + +-- With non-default Database + +CREATE DATABASE alter1_db; +USE alter1_db; +SHOW TABLES; + +CREATE TABLE alter1(a INT, b INT); +DESCRIBE EXTENDED alter1; + +ALTER TABLE alter1 SET TBLPROPERTIES ('a'='1', 'c'='3'); +DESCRIBE EXTENDED alter1; + +ALTER TABLE alter1 SET TBLPROPERTIES ('a'='1', 'c'='4', 'd'='3'); +DESCRIBE EXTENDED alter1; + +ALTER TABLE alter1 SET TBLPROPERTIES ('EXTERNAL'='TRUE'); +DESCRIBE EXTENDED alter1; + +ALTER TABLE alter1 SET TBLPROPERTIES ('EXTERNAL'='FALSE'); +DESCRIBE EXTENDED alter1; + +ALTER TABLE alter1 SET SERDEPROPERTIES('s1'='9'); +DESCRIBE EXTENDED alter1; + +ALTER TABLE alter1 SET SERDEPROPERTIES('s1'='10', 's2' ='20'); +DESCRIBE EXTENDED alter1; + +add jar ../data/files/TestSerDe.jar; +ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' WITH SERDEPROPERTIES ('s1'='9'); +DESCRIBE EXTENDED alter1; + +ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe'; +DESCRIBE EXTENDED alter1; + +ALTER TABLE alter1 REPLACE COLUMNS (a int, b int, c string); +DESCRIBE alter1; + +DROP TABLE alter1; +USE default; +DROP DATABASE alter1_db; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter2.q b/src/test/hive/ql/src/test/queries/clientpositive/alter2.q new file mode 100644 index 0000000000000..ddf57873a548c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter2.q @@ -0,0 +1,55 @@ +create table alter2(a int, b int) partitioned by (insertdate string); +describe extended alter2; +show partitions alter2; +alter table alter2 add partition (insertdate='2008-01-01') location '2008/01/01'; +describe extended alter2; +show partitions alter2; +alter table alter2 add partition (insertdate='2008-01-02') location '2008/01/02'; +describe extended alter2; +show partitions alter2; +drop table alter2; + +create external table alter2(a int, b int) partitioned by (insertdate string); +describe extended alter2; +show partitions alter2; +alter table alter2 add partition (insertdate='2008-01-01') location '2008/01/01'; +describe extended alter2; +show partitions alter2; +alter table alter2 add partition (insertdate='2008-01-02') location '2008/01/02'; +describe extended alter2; +show partitions alter2; + +-- Cleanup +DROP TABLE alter2; +SHOW TABLES; + +-- Using non-default Database + +CREATE DATABASE alter2_db; +USE alter2_db; +SHOW TABLES; + +CREATE TABLE alter2(a int, b int) PARTITIONED BY (insertdate string); +DESCRIBE EXTENDED alter2; +SHOW PARTITIONS alter2; +ALTER TABLE alter2 ADD PARTITION (insertdate='2008-01-01') LOCATION '2008/01/01'; +DESCRIBE EXTENDED alter2; +SHOW PARTITIONS alter2; +ALTER TABLE alter2 ADD PARTITION (insertdate='2008-01-02') LOCATION '2008/01/02'; +DESCRIBE EXTENDED alter2; +SHOW PARTITIONS alter2; +DROP TABLE alter2; + +CREATE EXTERNAL TABLE alter2(a int, b int) PARTITIONED BY (insertdate string); +DESCRIBE EXTENDED alter2; +SHOW PARTITIONS alter2; +ALTER TABLE alter2 ADD PARTITION (insertdate='2008-01-01') LOCATION '2008/01/01'; +DESCRIBE EXTENDED alter2; +SHOW PARTITIONS alter2; +ALTER TABLE alter2 ADD PARTITION (insertdate='2008-01-02') LOCATION '2008/01/02'; +DESCRIBE EXTENDED alter2; +SHOW PARTITIONS alter2; + +DROP TABLE alter2; +USE default; +DROP DATABASE alter2_db; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter3.q b/src/test/hive/ql/src/test/queries/clientpositive/alter3.q new file mode 100644 index 0000000000000..3cde00748b98e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter3.q @@ -0,0 +1,56 @@ +create table alter3_src ( col1 string ) stored as textfile ; +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; + +create table alter3_like like alter3; + +insert overwrite table alter3 partition (pCol1='test_part:', pcol2='test_part:') select col1 from alter3_src ; +select * from alter3 where pcol1='test_part:' and pcol2='test_part:'; + + +alter table alter3 rename to alter3_renamed; +describe extended alter3_renamed; +describe extended alter3_renamed partition (pCol1='test_part:', pcol2='test_part:'); +select * from alter3_renamed where pcol1='test_part:' and pcol2='test_part:'; + +insert overwrite table alter3_like +partition (pCol1='test_part:', pcol2='test_part:') +select col1 from alter3_src; +alter table alter3_like rename to alter3_like_renamed; + +describe extended alter3_like_renamed; + +-- Cleanup +DROP TABLE alter3_src; +DROP TABLE alter3_renamed; +DROP TABLE alter3_like_renamed; +SHOW TABLES; + +-- With non-default Database + +CREATE DATABASE alter3_db; +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 ; + +CREATE TABLE alter3 (col1 STRING) PARTITIONED BY (pcol1 STRING, pcol2 STRING) STORED AS SEQUENCEFILE; + +CREATE TABLE alter3_like LIKE alter3; + +INSERT OVERWRITE TABLE alter3 PARTITION (pCol1='test_part:', pcol2='test_part:') SELECT col1 FROM alter3_src ; +SELECT * FROM alter3 WHERE pcol1='test_part:' AND pcol2='test_part:'; + +ALTER TABLE alter3 RENAME TO alter3_renamed; +DESCRIBE EXTENDED alter3_renamed; +DESCRIBE EXTENDED alter3_renamed PARTITION (pCol1='test_part:', pcol2='test_part:'); +SELECT * FROM alter3_renamed WHERE pcol1='test_part:' AND pcol2='test_part:'; + +INSERT OVERWRITE TABLE alter3_like +PARTITION (pCol1='test_part:', pcol2='test_part:') +SELECT col1 FROM alter3_src; +ALTER TABLE alter3_like RENAME TO alter3_like_renamed; + +DESCRIBE EXTENDED alter3_like_renamed; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter4.q b/src/test/hive/ql/src/test/queries/clientpositive/alter4.q new file mode 100644 index 0000000000000..542ff019b2595 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter4.q @@ -0,0 +1,26 @@ +CREATE TABLE set_bucketing_test (key INT, value STRING) CLUSTERED BY (key) INTO 10 BUCKETS; +DESCRIBE EXTENDED set_bucketing_test; + +ALTER TABLE set_bucketing_test NOT CLUSTERED; +DESCRIBE EXTENDED set_bucketing_test; + +-- Cleanup +DROP TABLE set_bucketing_test; +SHOW TABLES; + +-- with non-default Database + +CREATE DATABASE alter4_db; +USE alter4_db; +SHOW TABLES; + +CREATE TABLE set_bucketing_test (key INT, value STRING) CLUSTERED BY (key) INTO 10 BUCKETS; +DESCRIBE EXTENDED set_bucketing_test; + +ALTER TABLE set_bucketing_test NOT CLUSTERED; +DESCRIBE EXTENDED set_bucketing_test; + +DROP TABLE set_bucketing_test; +USE default; +DROP DATABASE alter4_db; +SHOW DATABASES; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter5.q b/src/test/hive/ql/src/test/queries/clientpositive/alter5.q new file mode 100644 index 0000000000000..0d14f228d1c30 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter5.q @@ -0,0 +1,45 @@ +-- +-- Added to validate the fix for HIVE-2117 - explicit partition location +-- + +create table alter5_src ( col1 string ) stored as textfile ; +load data local inpath '../data/files/test.dat' overwrite into table alter5_src ; + +create table alter5 ( col1 string ) partitioned by (dt string); + +-- +-- Here's the interesting bit for HIVE-2117 - partition subdir should be +-- named "parta". +-- +alter table alter5 add partition (dt='a') location 'parta'; + +describe extended alter5 partition (dt='a'); + +insert overwrite table alter5 partition (dt='a') select col1 from alter5_src ; +select * from alter5 where dt='a'; + +describe extended alter5 partition (dt='a'); + +-- Cleanup +DROP TABLE alter5_src; +DROP TABLE alter5; +SHOW TABLES; + +-- With non-default Database + +CREATE DATABASE alter5_db; +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 ; + +create table alter5 ( col1 string ) partitioned by (dt string); +alter table alter5 add partition (dt='a') location 'parta'; + +describe extended alter5 partition (dt='a'); + +insert overwrite table alter5 partition (dt='a') select col1 from alter5_src ; +select * from alter5 where dt='a'; + +describe extended alter5 partition (dt='a'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q new file mode 100644 index 0000000000000..807ef539c8643 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q @@ -0,0 +1,48 @@ +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; + +show table extended like `src_rc_concatenate_test`; + +select count(1) from src_rc_concatenate_test; +select sum(hash(key)), sum(hash(value)) from src_rc_concatenate_test; + +create index src_rc_concatenate_test_index on table src_rc_concatenate_test(key) as 'compact' WITH DEFERRED REBUILD IDXPROPERTIES ("prop1"="val1", "prop2"="val2"); +show indexes on src_rc_concatenate_test; + +alter table src_rc_concatenate_test concatenate; + +show table extended like `src_rc_concatenate_test`; + +select count(1) from src_rc_concatenate_test; +select sum(hash(key)), sum(hash(value)) from src_rc_concatenate_test; + +drop index src_rc_concatenate_test_index on src_rc_concatenate_test; + +create table src_rc_concatenate_test_part(key int, value string) partitioned by (ds string) stored as rcfile; + +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'); + +show table extended like `src_rc_concatenate_test_part` partition (ds='2011'); + +select count(1) from src_rc_concatenate_test_part; +select sum(hash(key)), sum(hash(value)) from src_rc_concatenate_test_part; + +create index src_rc_concatenate_test_part_index on table src_rc_concatenate_test_part(key) as 'compact' WITH DEFERRED REBUILD IDXPROPERTIES ("prop1"="val1", "prop2"="val2"); +show indexes on src_rc_concatenate_test_part; + +alter table src_rc_concatenate_test_part partition (ds='2011') concatenate; + +show table extended like `src_rc_concatenate_test_part` partition (ds='2011'); + +select count(1) from src_rc_concatenate_test_part; +select sum(hash(key)), sum(hash(value)) from src_rc_concatenate_test_part; + +drop index src_rc_concatenate_test_part_index on src_rc_concatenate_test_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_index.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_index.q new file mode 100644 index 0000000000000..2aa13da46961f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_index.q @@ -0,0 +1,11 @@ +drop index src_index_8 on src; + +create index src_index_8 on table src(key) as 'compact' WITH DEFERRED REBUILD IDXPROPERTIES ("prop1"="val1", "prop2"="val2"); +desc extended default__src_src_index_8__; + +alter index src_index_8 on src set IDXPROPERTIES ("prop1"="val1_new", "prop3"="val3"); +desc extended default__src_src_index_8__; + +drop index src_index_8 on src; + +show tables; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_merge.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_merge.q new file mode 100644 index 0000000000000..c3502739d54a9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_merge.q @@ -0,0 +1,41 @@ +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; + +show table extended like `src_rc_merge_test`; + +select count(1) from src_rc_merge_test; +select sum(hash(key)), sum(hash(value)) from src_rc_merge_test; + +alter table src_rc_merge_test concatenate; + +show table extended like `src_rc_merge_test`; + +select count(1) from src_rc_merge_test; +select sum(hash(key)), sum(hash(value)) from src_rc_merge_test; + + +create table src_rc_merge_test_part(key int, value string) partitioned by (ds string) stored as rcfile; + +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'); + +show table extended like `src_rc_merge_test_part` partition (ds='2011'); + +select count(1) from src_rc_merge_test_part; +select sum(hash(key)), sum(hash(value)) from src_rc_merge_test_part; + +alter table src_rc_merge_test_part partition (ds='2011') concatenate; + +show table extended like `src_rc_merge_test_part` partition (ds='2011'); + +select count(1) from src_rc_merge_test_part; +select sum(hash(key)), sum(hash(value)) from src_rc_merge_test_part; + +drop table src_rc_merge_test; +drop table src_rc_merge_test_part; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_2.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_2.q new file mode 100644 index 0000000000000..65ddfed13dac4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_2.q @@ -0,0 +1,19 @@ +create table src_rc_merge_test_part(key int, value string) partitioned by (ds string, ts string) stored as rcfile; + +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'); + +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'; + +alter table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31') concatenate; + + +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'; + +drop table src_rc_merge_test_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_stats.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_stats.q new file mode 100644 index 0000000000000..23bae55ab2add --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_stats.q @@ -0,0 +1,41 @@ +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; + +show table extended like `src_rc_merge_test_stat`; +desc extended src_rc_merge_test_stat; + +analyze table src_rc_merge_test_stat compute statistics; + +desc extended src_rc_merge_test_stat; + +alter table src_rc_merge_test_stat concatenate; + +show table extended like `src_rc_merge_test_stat`; +desc extended src_rc_merge_test_stat; + + +create table src_rc_merge_test_part_stat(key int, value string) partitioned by (ds string) stored as rcfile; + +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'); + +show table extended like `src_rc_merge_test_part_stat` partition (ds='2011'); +desc extended src_rc_merge_test_part_stat; + +analyze table src_rc_merge_test_part_stat partition(ds='2011') compute statistics; + +desc extended src_rc_merge_test_part_stat; + +alter table src_rc_merge_test_part_stat partition (ds='2011') concatenate; + +show table extended like `src_rc_merge_test_part_stat` partition (ds='2011'); +desc extended src_rc_merge_test_part_stat; + +drop table src_rc_merge_test_stat; +drop table src_rc_merge_test_part_stat; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q new file mode 100644 index 0000000000000..b6d1eb8f2d4b8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q @@ -0,0 +1,59 @@ + +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/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q new file mode 100644 index 0000000000000..c6a4ad24fcdde --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q @@ -0,0 +1,84 @@ +-- 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/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q new file mode 100644 index 0000000000000..7481c651bfb7a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q @@ -0,0 +1,23 @@ +create table alter_table_partition_clusterby_sortby (a int, b int) partitioned by (c string) clustered by (a, b) sorted by (a desc, b asc) into 4 buckets; +alter table alter_table_partition_clusterby_sortby add partition(c='abc'); + +-- Turn off sorting for a partition + +alter table alter_table_partition_clusterby_sortby partition(c='abc') not sorted; +desc formatted alter_table_partition_clusterby_sortby partition(c='abc'); + +-- Modify clustering for a partition + +alter table alter_table_partition_clusterby_sortby partition(c='abc') clustered by (b) sorted by (b desc) into 4 buckets; +desc formatted alter_table_partition_clusterby_sortby partition(c='abc'); + +-- Turn off clustering for a partition + +alter table alter_table_partition_clusterby_sortby partition(c='abc') not clustered; +desc formatted alter_table_partition_clusterby_sortby partition(c='abc'); + +-- Table properties should be unchanged + +desc formatted alter_table_partition_clusterby_sortby; + +drop table alter_table_partition_clusterby_sortby; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_coltype.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_coltype.q new file mode 100644 index 0000000000000..5479afbbd5ab0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_coltype.q @@ -0,0 +1,57 @@ +-- create testing table. +create table alter_coltype(key string, value string) partitioned by (dt string, ts string); + +-- insert and create a partition. +insert overwrite table alter_coltype partition(dt='100x', ts='6:30pm') select * from src1; + +desc alter_coltype; + +-- select with paritition predicate. +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); + +-- load a new partition using new data type. +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 = 100; + +-- alter partition key column data type for ts column. +alter table alter_coltype partition column (ts double); + +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; + +-- 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'; + +-- 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; + +-- 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 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'; + +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/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_format_loc.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_format_loc.q new file mode 100644 index 0000000000000..bef6865833960 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_format_loc.q @@ -0,0 +1,32 @@ +create table alter_partition_format_test (key int, value string); +desc extended alter_partition_format_test; + +alter table alter_partition_format_test set fileformat rcfile; +desc extended alter_partition_format_test; + +alter table alter_partition_format_test set location "file:/test/test/"; +desc extended alter_partition_format_test; + +drop table alter_partition_format_test; + +--partitioned table +create table alter_partition_format_test (key int, value string) partitioned by (ds string); + +alter table alter_partition_format_test add partition(ds='2010'); +desc extended alter_partition_format_test partition(ds='2010'); + +alter table alter_partition_format_test partition(ds='2010') set fileformat rcfile; +desc extended alter_partition_format_test partition(ds='2010'); + +alter table alter_partition_format_test partition(ds='2010') set location "file:/test/test/ds=2010"; +desc extended alter_partition_format_test partition(ds='2010'); + +desc extended alter_partition_format_test; + +alter table alter_partition_format_test set fileformat rcfile; +desc extended alter_partition_format_test; + +alter table alter_partition_format_test set location "file:/test/test/"; +desc extended alter_partition_format_test; + +drop table alter_partition_format_test; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q new file mode 100644 index 0000000000000..7bcb9f071c415 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q @@ -0,0 +1,26 @@ +-- Create table +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'); + +-- offline +alter table alter_part_protect_mode partition (year='1996') disable offline; +select * from alter_part_protect_mode where year = '1996'; +alter table alter_part_protect_mode partition (year='1995') enable offline; +alter table alter_part_protect_mode partition (year='1995') disable offline; +select * from alter_part_protect_mode where year = '1995'; + +-- no_drop +alter table alter_part_protect_mode partition (year='1996') enable no_drop; +alter table alter_part_protect_mode partition (year='1995') disable no_drop; +alter table alter_part_protect_mode drop partition (year='1995'); +alter table alter_part_protect_mode partition (year='1994', month='07') disable no_drop; +alter table alter_part_protect_mode drop partition (year='1994'); + +-- Cleanup +alter table alter_part_protect_mode partition (year='1996') disable no_drop; +drop table alter_part_protect_mode; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q new file mode 100644 index 0000000000000..301362a881456 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q @@ -0,0 +1,9 @@ +SET hive.metastore.partition.name.whitelist.pattern=[A-Za-z]*; +-- This pattern matches only letters. + +CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS part_whitelist_test; + +ALTER TABLE part_whitelist_test ADD PARTITION (ds='Part'); + +ALTER TABLE part_whitelist_test PARTITION (ds='Part') rename to partition (ds='Apart'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition.q new file mode 100644 index 0000000000000..d498cd52a5ffb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition.q @@ -0,0 +1,41 @@ +-- Cleanup +DROP TABLE alter_rename_partition_src; +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 ; + +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 ; +select * from alter_rename_partition where pcol1='old_part1:' and pcol2='old_part2:'; + +alter table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') rename to partition (pCol1='new_part1:', pcol2='new_part2:'); +SHOW PARTITIONS alter_rename_partition; +select * from alter_rename_partition where pcol1='old_part1:' and pcol2='old_part2:'; +select * from alter_rename_partition where pcol1='new_part1:' and pcol2='new_part2:'; + +-- Cleanup +DROP TABLE alter_rename_partition_src; +DROP TABLE alter_rename_partition; +SHOW TABLES; + +-- With non-default Database + +CREATE DATABASE alter_rename_partition_db; +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 ; + +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 ; +SELECT * FROM alter_rename_partition WHERE pcol1='old_part1:' AND pcol2='old_part2:'; + +ALTER TABLE alter_rename_partition PARTITION (pCol1='old_part1:', pcol2='old_part2:') RENAME TO PARTITION (pCol1='new_part1:', pcol2='new_part2:'); +SHOW PARTITIONS alter_rename_partition; +SELECT * FROM alter_rename_partition WHERE pcol1='old_part1:' and pcol2='old_part2:'; +SELECT * FROM alter_rename_partition WHERE pcol1='new_part1:' and pcol2='new_part2:'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q new file mode 100644 index 0000000000000..cdefc2dcbe7b4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q @@ -0,0 +1,22 @@ +-- SORT_BEFORE_DIFF + +create table src_auth_tmp as select * from src; + +create table authorization_part (key int, value string) partitioned by (ds string); +ALTER TABLE authorization_part SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); +set hive.security.authorization.enabled=true; +grant select on table src_auth_tmp to user hive_test_user; + +-- column grant to user +grant Create on table 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; + +show grant user hive_test_user on table authorization_part; +grant select(key) on table authorization_part to user hive_test_user; +insert overwrite table authorization_part partition (ds='2010') select key, value from src_auth_tmp; +show grant user hive_test_user on table authorization_part(key) partition (ds='2010'); +alter table authorization_part partition (ds='2010') rename to partition (ds='2010_tmp'); +show grant user hive_test_user on table authorization_part(key) partition (ds='2010_tmp'); + +drop table authorization_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_skewed_table.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_skewed_table.q new file mode 100644 index 0000000000000..216bbb58cdf3f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_skewed_table.q @@ -0,0 +1,32 @@ +set hive.mapred.supports.subdirectories=true; + +create table original (key STRING, value STRING); + +describe formatted original; + +alter table original SKEWED BY (key) ON (1,5,6); + +describe formatted original; + +drop table original; + +create table original2 (key STRING, value STRING) ; + +describe formatted original2; + +alter table original2 SKEWED BY (key, value) ON ((1,1),(5,6)); + +describe formatted original2; + +drop table original2; + +create table original3 (key STRING, value STRING) SKEWED BY (key, value) ON ((1,1),(5,6)); + +describe formatted original3; + +alter table original3 not skewed; + +describe formatted original3; + +drop table original3; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_table_not_sorted.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_table_not_sorted.q new file mode 100644 index 0000000000000..7aff3288cf646 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_table_not_sorted.q @@ -0,0 +1,7 @@ +create table alter_table_not_sorted (a int, b int) clustered by (a) sorted by (a) into 4 buckets; +desc formatted alter_table_not_sorted; + +alter table alter_table_not_sorted not sorted; +desc formatted alter_table_not_sorted; + +drop table alter_table_not_sorted; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde.q new file mode 100644 index 0000000000000..a80693e7c5153 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde.q @@ -0,0 +1,33 @@ +-- test table +create table test_table (id int, query string, name string); +describe extended test_table; + +alter table test_table set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; +describe extended test_table; + +alter table test_table set serdeproperties ('field.delim' = ','); +describe extended test_table; + +drop table test_table; + +--- test partitioned table +create table test_table (id int, query string, name string) partitioned by (dt string); + +alter table test_table add partition (dt = '2011'); +describe extended test_table partition (dt='2011'); + +alter table test_table set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; +describe extended test_table partition (dt='2011'); + +alter table test_table set serdeproperties ('field.delim' = ','); +describe extended test_table partition (dt='2011'); + +-- test partitions + +alter table test_table partition(dt='2011') set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; +describe extended test_table partition (dt='2011'); + +alter table test_table partition(dt='2011') set serdeproperties ('field.delim' = ','); +describe extended test_table partition (dt='2011'); + +drop table test_table diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde2.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde2.q new file mode 100644 index 0000000000000..6ca3be3139e45 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde2.q @@ -0,0 +1,20 @@ +-- Tests that when overwriting a partition in a table after altering the serde properties +-- the partition metadata is updated as well. + +CREATE TABLE tst1(key STRING, value STRING) PARTITIONED BY (ds STRING); + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test altering the serde properties + +ALTER TABLE tst1 SET SERDEPROPERTIES ('field.delim' = ','); + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar1.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar1.q new file mode 100644 index 0000000000000..781cbda735158 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar1.q @@ -0,0 +1,32 @@ +drop table alter_varchar_1; + +create table alter_varchar_1 (key string, value string); +insert overwrite table alter_varchar_1 + select key, value from src order by key limit 5; + +select * from alter_varchar_1 order by key; + +-- change column to varchar +alter table alter_varchar_1 change column value value varchar(20); +-- contents should still look the same +select * from alter_varchar_1 order by key; + +-- change column to smaller varchar +alter table alter_varchar_1 change column value value varchar(3); +-- value column should be truncated now +select * from alter_varchar_1 order by key; + +-- change back to bigger varchar +alter table alter_varchar_1 change column value value varchar(20); +-- column values should be full size again +select * from alter_varchar_1 order by key; + +-- add varchar column +alter table alter_varchar_1 add columns (key2 int, value2 varchar(10)); +select * from alter_varchar_1 order by key; + +insert overwrite table alter_varchar_1 + select key, value, key, value from src order by key limit 5; +select * from alter_varchar_1 order by key; + +drop table alter_varchar_1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar2.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar2.q new file mode 100644 index 0000000000000..5a481e7f8ebfe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar2.q @@ -0,0 +1,22 @@ + +-- alter column type, with partitioned table +drop table if exists alter_varchar2; + +create table alter_varchar2 ( + c1 varchar(255) +) partitioned by (hr int); + +insert overwrite table alter_varchar2 partition (hr=1) + select value from src limit 1; + +select c1, length(c1) from alter_varchar2; + +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 hr, c1, length(c1) from alter_varchar2 where hr = 1; +select hr, c1, length(c1) from alter_varchar2 where hr = 2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_view_as_select.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_view_as_select.q new file mode 100644 index 0000000000000..dcab3cab7e9ac --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_view_as_select.q @@ -0,0 +1,13 @@ +DROP VIEW testView; +CREATE VIEW testView as SELECT * FROM srcpart; +DESCRIBE FORMATTED testView; + +ALTER VIEW testView AS SELECT value FROM src WHERE key=86; +DESCRIBE FORMATTED testView; + +ALTER VIEW testView AS +SELECT * FROM src +WHERE key > 80 AND key < 100 +ORDER BY key, value +LIMIT 10; +DESCRIBE FORMATTED testView; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_view_rename.q b/src/test/hive/ql/src/test/queries/clientpositive/alter_view_rename.q new file mode 100644 index 0000000000000..68cf9d6e0a974 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/alter_view_rename.q @@ -0,0 +1,10 @@ +CREATE TABLE invites (foo INT, bar STRING) PARTITIONED BY (ds STRING); +CREATE VIEW view1 as SELECT * FROM invites; +DESCRIBE EXTENDED view1; + +ALTER VIEW view1 RENAME TO view2; +DESCRIBE EXTENDED view2; +SELECT * FROM view2; + +DROP TABLE invites; +DROP VIEW view2; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ambiguous_col.q b/src/test/hive/ql/src/test/queries/clientpositive/ambiguous_col.q new file mode 100644 index 0000000000000..e7053c1c8eb55 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ambiguous_col.q @@ -0,0 +1,6 @@ +-- 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 +explain select * from (select a.key, a.`[k].*` from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; +-- EXPRESSION +explain select * from (select a.key, a.key from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/archive.q b/src/test/hive/ql/src/test/queries/clientpositive/archive.q new file mode 100644 index 0000000000000..a928a81023ff6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/archive.q @@ -0,0 +1,69 @@ +set hive.archive.enabled = true; +set hive.enforce.bucketing = true; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20) + +drop table tstsrc; +drop table tstsrcpart; + +create table tstsrc like src; +insert overwrite table tstsrc select key, value from src; + +create table tstsrcpart (key string, value string) partitioned by (ds string, hr string) clustered by (key) into 10 buckets; + +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') +select key, value from srcpart where ds='2008-04-08' and 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'; + +insert overwrite table tstsrcpart partition (ds='2008-04-09', hr='11') +select key, value from srcpart where ds='2008-04-09' and hr='11'; + +insert overwrite table tstsrcpart partition (ds='2008-04-09', hr='12') +select key, value from srcpart where ds='2008-04-09' and hr='12'; + +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); + +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2; + +SELECT key, count(1) FROM tstsrcpart WHERE ds='2008-04-08' AND hr='12' AND key='0' GROUP BY key; + +SELECT * FROM tstsrcpart a JOIN tstsrc b ON a.key=b.key +WHERE a.ds='2008-04-08' AND a.hr='12' AND a.key='0'; + +ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); + +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2; + +CREATE TABLE harbucket(key INT) +PARTITIONED by (ds STRING) +CLUSTERED BY (key) INTO 10 BUCKETS; + +INSERT OVERWRITE TABLE harbucket PARTITION(ds='1') SELECT CAST(key AS INT) AS a FROM tstsrc WHERE key < 50; + +SELECT key FROM harbucket TABLESAMPLE(BUCKET 1 OUT OF 10) SORT BY key; +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); +SELECT key FROM harbucket TABLESAMPLE(BUCKET 1 OUT OF 10) SORT BY key; +ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); +SELECT key FROM harbucket TABLESAMPLE(BUCKET 1 OUT OF 10) SORT BY key; + + +CREATE TABLE old_name(key INT) +PARTITIONED by (ds STRING); + +INSERT OVERWRITE TABLE old_name PARTITION(ds='1') SELECT CAST(key AS INT) AS a FROM tstsrc WHERE key < 50; +ALTER TABLE old_name ARCHIVE PARTITION (ds='1'); +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM old_name WHERE ds='1') subq1) subq2; +ALTER TABLE old_name RENAME TO new_name; +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM new_name WHERE ds='1') subq1) subq2; + +drop table tstsrc; +drop table tstsrcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/archive_corrupt.q b/src/test/hive/ql/src/test/queries/clientpositive/archive_corrupt.q new file mode 100644 index 0000000000000..b83eab5d38137 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/archive_corrupt.q @@ -0,0 +1,35 @@ +USE default; + +set hive.archive.enabled = true; +set hive.enforce.bucketing = true; + +drop table tstsrcpart; + +create table tstsrcpart like srcpart; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20) +-- The version of GzipCodec provided in Hadoop 0.20 silently ignores +-- file format errors. However, versions of Hadoop that include +-- HADOOP-6835 (e.g. 0.23 and 1.x) cause a Wrong File Format exception +-- 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'); + +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12') +select key, value from srcpart where ds='2008-04-08' and hr='12'; + +insert overwrite table tstsrcpart partition (ds='2008-04-09', hr='11') +select key, value from srcpart where ds='2008-04-09' and hr='11'; + +insert overwrite table tstsrcpart partition (ds='2008-04-09', hr='12') +select key, value from srcpart where ds='2008-04-09' and hr='12'; + +describe extended tstsrcpart partition (ds='2008-04-08', hr='11'); + +alter table tstsrcpart archive partition (ds='2008-04-08', hr='11'); + +describe extended tstsrcpart partition (ds='2008-04-08', hr='11'); + +alter table tstsrcpart unarchive partition (ds='2008-04-08', hr='11'); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q b/src/test/hive/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q new file mode 100644 index 0000000000000..ddc06a99bb5bc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q @@ -0,0 +1,69 @@ +set hive.archive.enabled = true; +set hive.enforce.bucketing = true; + +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +drop table tstsrc; +drop table tstsrcpart; + +create table tstsrc like src; +insert overwrite table tstsrc select key, value from src; + +create table tstsrcpart (key string, value string) partitioned by (ds string, hr string) clustered by (key) into 10 buckets; + +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') +select key, value from srcpart where ds='2008-04-08' and 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'; + +insert overwrite table tstsrcpart partition (ds='2008-04-09', hr='11') +select key, value from srcpart where ds='2008-04-09' and hr='11'; + +insert overwrite table tstsrcpart partition (ds='2008-04-09', hr='12') +select key, value from srcpart where ds='2008-04-09' and hr='12'; + +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); + +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2; + +SELECT key, count(1) FROM tstsrcpart WHERE ds='2008-04-08' AND hr='12' AND key='0' GROUP BY key; + +SELECT * FROM tstsrcpart a JOIN tstsrc b ON a.key=b.key +WHERE a.ds='2008-04-08' AND a.hr='12' AND a.key='0'; + +ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); + +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2; + +CREATE TABLE harbucket(key INT) +PARTITIONED by (ds STRING) +CLUSTERED BY (key) INTO 10 BUCKETS; + +INSERT OVERWRITE TABLE harbucket PARTITION(ds='1') SELECT CAST(key AS INT) AS a FROM tstsrc WHERE key < 50; + +SELECT key FROM harbucket TABLESAMPLE(BUCKET 1 OUT OF 10) SORT BY key; +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); +SELECT key FROM harbucket TABLESAMPLE(BUCKET 1 OUT OF 10) SORT BY key; +ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); +SELECT key FROM harbucket TABLESAMPLE(BUCKET 1 OUT OF 10) SORT BY key; + + +CREATE TABLE old_name(key INT) +PARTITIONED by (ds STRING); + +INSERT OVERWRITE TABLE old_name PARTITION(ds='1') SELECT CAST(key AS INT) AS a FROM tstsrc WHERE key < 50; +ALTER TABLE old_name ARCHIVE PARTITION (ds='1'); +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM old_name WHERE ds='1') subq1) subq2; +ALTER TABLE old_name RENAME TO new_name; +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM new_name WHERE ds='1') subq1) subq2; + +drop table tstsrc; +drop table tstsrcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/archive_multi.q b/src/test/hive/ql/src/test/queries/clientpositive/archive_multi.q new file mode 100644 index 0000000000000..2c1a6d83a35c9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/archive_multi.q @@ -0,0 +1,42 @@ +set hive.archive.enabled = true; +set hive.enforce.bucketing = true; + +drop table tstsrc; +drop table tstsrcpart; + +create table tstsrc like src; +insert overwrite table tstsrc select key, value from src; + +create table tstsrcpart like srcpart; + +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') +select key, value from srcpart where ds='2008-04-08' and 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'; + +insert overwrite table tstsrcpart partition (ds='2008-04-09', hr='11') +select key, value from srcpart where ds='2008-04-09' and hr='11'; + +insert overwrite table tstsrcpart partition (ds='2008-04-09', hr='12') +select key, value from srcpart where ds='2008-04-09' and hr='12'; + +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2; + +ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); + +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2; + +SELECT key, count(1) FROM tstsrcpart WHERE ds='2008-04-08' AND hr='12' AND key='0' GROUP BY key; + +SELECT * FROM tstsrcpart a JOIN tstsrc b ON a.key=b.key +WHERE a.ds='2008-04-08' AND a.hr='12' AND a.key='0'; + +ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08'); + +SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col +FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_1.q b/src/test/hive/ql/src/test/queries/clientpositive/authorization_1.q new file mode 100644 index 0000000000000..25c99187f3c00 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/authorization_1.q @@ -0,0 +1,89 @@ +-- SORT_BEFORE_DIFF + +create table src_autho_test as select * from src; + +set hive.security.authorization.enabled=true; + +--table grant to user + +grant select on table src_autho_test to user hive_test_user; + +show grant user hive_test_user on table src_autho_test; +show grant user hive_test_user on table src_autho_test(key); + +select key from src_autho_test order by key limit 20; + +revoke select on table src_autho_test from user hive_test_user; +show grant user hive_test_user on table src_autho_test; +show grant user hive_test_user on table src_autho_test(key); + +--column grant to user + +grant select(key) on table src_autho_test to user hive_test_user; + +show grant user hive_test_user on table src_autho_test; +show grant user hive_test_user on table src_autho_test(key); + +select key from src_autho_test order by key limit 20; + +revoke select(key) on table src_autho_test from user hive_test_user; +show grant user hive_test_user on table src_autho_test; +show grant user hive_test_user on table src_autho_test(key); + +--table grant to group + +grant select on table src_autho_test to group hive_test_group1; + +show grant group hive_test_group1 on table src_autho_test; +show grant group hive_test_group1 on table src_autho_test(key); + +select key from src_autho_test order by key limit 20; + +revoke select on table src_autho_test from group hive_test_group1; +show grant group hive_test_group1 on table src_autho_test; +show grant group hive_test_group1 on table src_autho_test(key); + +--column grant to group + +grant select(key) on table src_autho_test to group hive_test_group1; + +show grant group hive_test_group1 on table src_autho_test; +show grant group hive_test_group1 on table src_autho_test(key); + +select key from src_autho_test order by key limit 20; + +revoke select(key) on table src_autho_test from group hive_test_group1; +show grant group hive_test_group1 on table src_autho_test; +show grant group hive_test_group1 on table src_autho_test(key); + +--role +create role src_role; +grant role src_role to user hive_test_user; +show role grant user hive_test_user; + +--column grant to role + +grant select(key) on table src_autho_test to role src_role; + +show grant role src_role on table src_autho_test; +show grant role src_role on table src_autho_test(key); + +select key from src_autho_test order by key limit 20; + +revoke select(key) on table src_autho_test from role src_role; + +--table grant to role + +grant select on table src_autho_test to role src_role; + +select key from src_autho_test order by key limit 20; + +show grant role src_role on table src_autho_test; +show grant role src_role on table src_autho_test(key); +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; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_2.q b/src/test/hive/ql/src/test/queries/clientpositive/authorization_2.q new file mode 100644 index 0000000000000..4fc79b9ed294a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/authorization_2.q @@ -0,0 +1,111 @@ +-- SORT_BEFORE_DIFF + +create table authorization_part (key int, value string) partitioned by (ds string); +create table src_auth_tmp as select * from src; +ALTER TABLE authorization_part SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); +set hive.security.authorization.enabled=true; + +-- column grant to user +grant Create on table 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; + +show grant user hive_test_user on table authorization_part; + +alter table authorization_part add partition (ds='2010'); +show grant user hive_test_user on table authorization_part partition (ds='2010'); + +grant select(key) on table authorization_part to user hive_test_user; +alter table authorization_part drop partition (ds='2010'); +insert overwrite table authorization_part partition (ds='2010') select key, value from src_auth_tmp; +show grant user hive_test_user on table authorization_part(key) partition (ds='2010'); +show grant user hive_test_user on table authorization_part(key); +select key from authorization_part where ds='2010' order by key limit 20; + +revoke select(key) on table authorization_part from user hive_test_user; +show grant user hive_test_user on table authorization_part(key); +show grant user hive_test_user on table authorization_part(key) partition (ds='2010'); + +select key from authorization_part where ds='2010' order by key limit 20; + +revoke select(key) on table authorization_part partition (ds='2010') from user hive_test_user; +show grant user hive_test_user on table authorization_part(key) partition (ds='2010'); + +alter table authorization_part drop partition (ds='2010'); + +-- table grant to user +show grant user hive_test_user on table authorization_part; + +alter table authorization_part add partition (ds='2010'); +show grant user hive_test_user on table authorization_part partition (ds='2010'); + +grant select on table authorization_part to user hive_test_user; +alter table authorization_part drop partition (ds='2010'); +insert overwrite table authorization_part partition (ds='2010') select key, value from src_auth_tmp; +show grant user hive_test_user on table authorization_part partition (ds='2010'); +show grant user hive_test_user on table authorization_part; +select key from authorization_part where ds='2010' order by key limit 20; + +revoke select on table authorization_part from user hive_test_user; +show grant user hive_test_user on table authorization_part; +show grant user hive_test_user on table authorization_part partition (ds='2010'); + +select key from authorization_part where ds='2010' order by key limit 20; + +revoke select on table authorization_part partition (ds='2010') from user hive_test_user; +show grant user hive_test_user on table authorization_part partition (ds='2010'); + +alter table authorization_part drop partition (ds='2010'); + +-- column grant to group + +show grant group hive_test_group1 on table authorization_part; + +alter table authorization_part add partition (ds='2010'); +show grant group hive_test_group1 on table authorization_part partition (ds='2010'); + +grant select(key) on table authorization_part to group hive_test_group1; +alter table authorization_part drop partition (ds='2010'); +insert overwrite table authorization_part partition (ds='2010') select key, value from src_auth_tmp; +show grant group hive_test_group1 on table authorization_part(key) partition (ds='2010'); +show grant group hive_test_group1 on table authorization_part(key); +select key from authorization_part where ds='2010' order by key limit 20; + +revoke select(key) on table authorization_part from group hive_test_group1; +show grant group hive_test_group1 on table authorization_part(key); +show grant group hive_test_group1 on table authorization_part(key) partition (ds='2010'); + +select key from authorization_part where ds='2010' order by key limit 20; + +revoke select(key) on table authorization_part partition (ds='2010') from group hive_test_group1; +show grant group hive_test_group1 on table authorization_part(key) partition (ds='2010'); + +alter table authorization_part drop partition (ds='2010'); + +-- table grant to group +show grant group hive_test_group1 on table authorization_part; + +alter table authorization_part add partition (ds='2010'); +show grant group hive_test_group1 on table authorization_part partition (ds='2010'); + +grant select on table authorization_part to group hive_test_group1; +alter table authorization_part drop partition (ds='2010'); +insert overwrite table authorization_part partition (ds='2010') select key, value from src_auth_tmp; +show grant group hive_test_group1 on table authorization_part partition (ds='2010'); +show grant group hive_test_group1 on table authorization_part; +select key from authorization_part where ds='2010' order by key limit 20; + +revoke select on table authorization_part from group hive_test_group1; +show grant group hive_test_group1 on table authorization_part; +show grant group hive_test_group1 on table authorization_part partition (ds='2010'); + +select key from authorization_part where ds='2010' order by key limit 20; + +revoke select on table authorization_part partition (ds='2010') from group hive_test_group1; +show grant group hive_test_group1 on table authorization_part partition (ds='2010'); + + +revoke select on table src_auth_tmp from user hive_test_user; +set hive.security.authorization.enabled=false; +drop table authorization_part; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_3.q b/src/test/hive/ql/src/test/queries/clientpositive/authorization_3.q new file mode 100644 index 0000000000000..ba76b0002fde8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/authorization_3.q @@ -0,0 +1,19 @@ +-- SORT_BEFORE_DIFF + +create table src_autho_test as select * from src; + +grant drop on table src_autho_test to user hive_test_user; +grant select on table src_autho_test to user hive_test_user; + +show grant user hive_test_user on table src_autho_test; + +revoke select on table src_autho_test from user hive_test_user; +revoke drop on table src_autho_test from user hive_test_user; + +grant drop,select on table src_autho_test to user hive_test_user; +show grant user hive_test_user on table src_autho_test; +revoke drop,select on table src_autho_test from user hive_test_user; + +grant drop,select(key), select(value) on table src_autho_test to user hive_test_user; +show grant user hive_test_user on table src_autho_test; +revoke drop,select(key), select(value) on table src_autho_test from user hive_test_user; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_4.q b/src/test/hive/ql/src/test/queries/clientpositive/authorization_4.q new file mode 100644 index 0000000000000..152c8e5cf305a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/authorization_4.q @@ -0,0 +1,13 @@ +-- SORT_BEFORE_DIFF + +create table src_autho_test as select * from src; + +grant All on table src_autho_test to user hive_test_user; + +set hive.security.authorization.enabled=true; + +show grant user hive_test_user on table src_autho_test; + +select key from src_autho_test order by key limit 20; + +drop table src_autho_test; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_5.q b/src/test/hive/ql/src/test/queries/clientpositive/authorization_5.q new file mode 100644 index 0000000000000..8869edce2f4ea --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/authorization_5.q @@ -0,0 +1,20 @@ +-- SORT_BEFORE_DIFF + +CREATE DATABASE IF NOT EXISTS test_db COMMENT 'Hive test database'; +SHOW DATABASES; + +GRANT drop ON DATABASE test_db TO USER hive_test_user; +GRANT select ON DATABASE test_db TO USER hive_test_user; + +SHOW GRANT USER hive_test_user ON DATABASE test_db; + +CREATE ROLE db_test_role; +GRANT ROLE db_test_role TO USER hive_test_user; +SHOW ROLE GRANT USER hive_test_user; + +GRANT drop ON DATABASE test_db TO ROLE db_test_role; +GRANT select ON DATABASE test_db TO ROLE db_test_role; + +SHOW GRANT ROLE db_test_role ON DATABASE test_db; + +DROP DATABASE IF EXISTS test_db; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_6.q b/src/test/hive/ql/src/test/queries/clientpositive/authorization_6.q new file mode 100644 index 0000000000000..258ada4988362 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/authorization_6.q @@ -0,0 +1,42 @@ +-- SORT_BEFORE_DIFF + +create table src_auth_tmp as select * from src; + +create table authorization_part (key int, value string) partitioned by (ds string); +ALTER TABLE authorization_part SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); +set hive.security.authorization.enabled=true; +grant select on table src_auth_tmp to user hive_test_user; + +-- column grant to user +grant Create on table 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; + +show grant user hive_test_user on table authorization_part; +grant select(key) on table authorization_part to user hive_test_user; +insert overwrite table authorization_part partition (ds='2010') select key, value from src_auth_tmp; +insert overwrite table authorization_part partition (ds='2011') select key, value from src_auth_tmp; +show grant user hive_test_user on table authorization_part(key) partition (ds='2010'); +show grant user hive_test_user on table authorization_part(key) partition (ds='2011'); +show grant user hive_test_user on table authorization_part(key); +select key from authorization_part where ds>='2010' order by key limit 20; + +drop table authorization_part; + +set hive.security.authorization.enabled=false; +create table authorization_part (key int, value string) partitioned by (ds string); +ALTER TABLE authorization_part SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="FALSE"); + +set hive.security.authorization.enabled=true; +grant Create on table authorization_part to user hive_test_user; +grant Update on table authorization_part to user hive_test_user; + +show grant user hive_test_user on table authorization_part; + +grant select(key) on table authorization_part to user hive_test_user; +insert overwrite table authorization_part partition (ds='2010') select key, value from src_auth_tmp; +insert overwrite table authorization_part partition (ds='2011') select key, value from src_auth_tmp; +show grant user hive_test_user on table authorization_part(key) partition (ds='2010'); +show grant user hive_test_user on table authorization_part(key) partition (ds='2011'); +show grant user hive_test_user on table authorization_part(key); +select key from authorization_part where ds>='2010' order by key limit 20; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_7.q b/src/test/hive/ql/src/test/queries/clientpositive/authorization_7.q new file mode 100644 index 0000000000000..ae49000442a1f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/authorization_7.q @@ -0,0 +1,15 @@ +GRANT ALL TO USER hive_test_user; +SET hive.security.authorization.enabled=true; +CREATE TABLE src_authorization_7 (key int, value string); +DESCRIBE src_authorization_7; +DROP TABLE src_authorization_7; +REVOKE ALL FROM USER hive_test_user; + +SET hive.security.authorization.enabled=false; + +GRANT ALL TO GROUP hive_test_group1; +SET hive.security.authorization.enabled=true; +CREATE TABLE src_authorization_7 (key int, value string); +DESCRIBE src_authorization_7; +DROP TABLE src_authorization_7; +REVOKE ALL FROM GROUP hive_test_group1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_8.q b/src/test/hive/ql/src/test/queries/clientpositive/authorization_8.q new file mode 100644 index 0000000000000..67fcf3162dd56 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/authorization_8.q @@ -0,0 +1,8 @@ +set hive.security.authorization.enabled=true; +GRANT ALL TO USER hive_test_user; +CREATE TABLE tbl_j5jbymsx8e (key INT, value STRING) PARTITIONED BY (ds STRING); +CREATE VIEW view_j5jbymsx8e_1 as SELECT * FROM tbl_j5jbymsx8e; +DESCRIBE view_j5jbymsx8e_1; +ALTER VIEW view_j5jbymsx8e_1 RENAME TO view_j5jbymsx8e_2; +REVOKE ALL FROM USER hive_test_user; +set hive.security.authorization.enabled=false; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join0.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join0.q new file mode 100644 index 0000000000000..84731dbce4a03 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join0.q @@ -0,0 +1,23 @@ + +set hive.auto.convert.join = true; + +explain +select sum(hash(a.k1,a.v1,a.k2, a.v2)) +from ( +SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +) a; + +select sum(hash(a.k1,a.v1,a.k2, a.v2)) +from ( +SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 +) a; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join1.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join1.q new file mode 100644 index 0000000000000..2062a28b46e21 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join1.q @@ -0,0 +1,12 @@ +set hive.auto.convert.join =true; + +CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE; + +explain +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value; + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value; + +SELECT sum(hash(dest_j1.key,dest_j1.value)) FROM dest_j1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join10.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join10.q new file mode 100644 index 0000000000000..2dbdcf75f3c24 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join10.q @@ -0,0 +1,17 @@ +set hive.auto.convert.join = true; + +explain +FROM +(SELECT src.* FROM src) x +JOIN +(SELECT src.* FROM src) Y +ON (x.key = Y.key) +select sum(hash(Y.key,Y.value)); + +FROM +(SELECT src.* FROM src) x +JOIN +(SELECT src.* FROM src) Y +ON (x.key = Y.key) +select sum(hash(Y.key,Y.value)); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join11.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join11.q new file mode 100644 index 0000000000000..d8045b1f86592 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join11.q @@ -0,0 +1,16 @@ +set hive.auto.convert.join = true; + +explain +SELECT sum(hash(src1.c1, src2.c4)) +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100; + +SELECT sum(hash(src1.c1, src2.c4)) +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join12.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join12.q new file mode 100644 index 0000000000000..b31e4f7f4d97a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join12.q @@ -0,0 +1,25 @@ + + +set hive.auto.convert.join = true; + + +explain +SELECT sum(hash(src1.c1, src2.c4)) +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100 +JOIN +(SELECT src.key as c5, src.value as c6 from src) src3 +ON src1.c1 = src3.c5 AND src3.c5 < 80; + +SELECT sum(hash(src1.c1, src2.c4)) +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100 +JOIN +(SELECT src.key as c5, src.value as c6 from src) src3 +ON src1.c1 = src3.c5 AND src3.c5 < 80; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join13.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join13.q new file mode 100644 index 0000000000000..c116eba6b7597 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join13.q @@ -0,0 +1,23 @@ + +set hive.auto.convert.join = true; + +explain +SELECT sum(hash(src1.c1, src2.c4)) +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100 +JOIN +(SELECT src.key as c5, src.value as c6 from src) src3 +ON src1.c1 + src2.c3 = src3.c5 AND src3.c5 < 200; + +SELECT sum(hash(src1.c1, src2.c4)) +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100 +JOIN +(SELECT src.key as c5, src.value as c6 from src) src3 +ON src1.c1 + src2.c3 = src3.c5 AND src3.c5 < 200; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join14.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join14.q new file mode 100644 index 0000000000000..b282fb9a1aba0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join14.q @@ -0,0 +1,19 @@ + +set hive.auto.convert.join = true; + +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +CREATE TABLE dest1(c1 INT, c2 STRING) STORED AS TEXTFILE; + +set mapreduce.framework.name=yarn; +set mapreduce.jobtracker.address=localhost:58; +set hive.exec.mode.local.auto=true; + +explain +FROM src JOIN srcpart ON src.key = srcpart.key AND srcpart.ds = '2008-04-08' and src.key > 100 +INSERT OVERWRITE TABLE dest1 SELECT src.key, srcpart.value; + +FROM src JOIN srcpart ON src.key = srcpart.key AND srcpart.ds = '2008-04-08' and src.key > 100 +INSERT OVERWRITE TABLE dest1 SELECT src.key, srcpart.value; + +SELECT sum(hash(dest1.c1,dest1.c2)) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q new file mode 100644 index 0000000000000..235b7c1b3fcd2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q @@ -0,0 +1,18 @@ + +set hive.auto.convert.join = true; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +CREATE TABLE dest1(c1 INT, c2 STRING) STORED AS TEXTFILE; + +set mapred.job.tracker=localhost:58; +set hive.exec.mode.local.auto=true; + +explain +FROM src JOIN srcpart ON src.key = srcpart.key AND srcpart.ds = '2008-04-08' and src.key > 100 +INSERT OVERWRITE TABLE dest1 SELECT src.key, srcpart.value; + +FROM src JOIN srcpart ON src.key = srcpart.key AND srcpart.ds = '2008-04-08' and src.key > 100 +INSERT OVERWRITE TABLE dest1 SELECT src.key, srcpart.value; + +SELECT sum(hash(dest1.c1,dest1.c2)) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join15.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join15.q new file mode 100644 index 0000000000000..66b745f62ead9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join15.q @@ -0,0 +1,19 @@ + +set hive.auto.convert.join = true; + +explain +select sum(hash(a.k1,a.v1,a.k2, a.v2)) +from ( +SELECT src1.key as k1, src1.value as v1, src2.key as k2, src2.value as v2 +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +SORT BY k1, v1, k2, v2 +) a; + + +select sum(hash(a.k1,a.v1,a.k2, a.v2)) +from ( +SELECT src1.key as k1, src1.value as v1, src2.key as k2, src2.value as v2 +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +SORT BY k1, v1, k2, v2 +) a; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join16.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join16.q new file mode 100644 index 0000000000000..0ade940892f42 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join16.q @@ -0,0 +1,18 @@ + +set hive.auto.convert.join = true; + + +explain +SELECT sum(hash(subq.key, tab.value)) +FROM +(select a.key, a.value from src a where a.key > 10 ) subq +JOIN src tab +ON (subq.key = tab.key and subq.key > 20 and subq.value = tab.value) +where tab.value < 200; + +SELECT sum(hash(subq.key, tab.value)) +FROM +(select a.key, a.value from src a where a.key > 10 ) subq +JOIN src tab +ON (subq.key = tab.key and subq.key > 20 and subq.value = tab.value) +where tab.value < 200; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join17.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join17.q new file mode 100644 index 0000000000000..4bb804e3e7f38 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join17.q @@ -0,0 +1,14 @@ + +set hive.auto.convert.join = true; + +CREATE TABLE dest1(key1 INT, value1 STRING, key2 INT, value2 STRING) STORED AS TEXTFILE; + +explain +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.*; + + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.*; + +SELECT sum(hash(dest1.key1,dest1.value1,dest1.key2,dest1.value2)) FROM dest1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join18.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join18.q new file mode 100644 index 0000000000000..7a3d6c9a4448b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join18.q @@ -0,0 +1,27 @@ + +set hive.auto.convert.join = true; +explain + SELECT sum(hash(a.key, a.value, b.key, b.value)) + FROM + ( + SELECT src1.key as key, count(src1.value) AS value FROM src src1 group by src1.key + ) a + FULL OUTER JOIN + ( + SELECT src2.key as key, count(distinct(src2.value)) AS value + FROM src1 src2 group by src2.key + ) b + ON (a.key = b.key); + + + SELECT sum(hash(a.key, a.value, b.key, b.value)) + FROM + ( + SELECT src1.key as key, count(src1.value) AS value FROM src src1 group by src1.key + ) a + FULL OUTER JOIN + ( + SELECT src2.key as key, count(distinct(src2.value)) AS value + FROM src1 src2 group by src2.key + ) b + ON (a.key = b.key); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q new file mode 100644 index 0000000000000..7465bc9b23bd2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q @@ -0,0 +1,29 @@ +set hive.auto.convert.join = true; + +explain + SELECT sum(hash(a.key, a.value, b.key, b.value1, b.value2)) + FROM + ( + SELECT src1.key as key, count(src1.value) AS value FROM src src1 group by src1.key + ) a + FULL OUTER JOIN + ( + SELECT src2.key as key, count(distinct(src2.value)) AS value1, + count(distinct(src2.key)) AS value2 + FROM src1 src2 group by src2.key + ) b + ON (a.key = b.key); + + + SELECT sum(hash(a.key, a.value, b.key, b.value1, b.value2)) + FROM + ( + SELECT src1.key as key, count(src1.value) AS value FROM src src1 group by src1.key + ) a + FULL OUTER JOIN + ( + SELECT src2.key as key, count(distinct(src2.value)) AS value1, + count(distinct(src2.key)) AS value2 + FROM src1 src2 group by src2.key + ) b + ON (a.key = b.key); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join19.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join19.q new file mode 100644 index 0000000000000..32935f1d89cf5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join19.q @@ -0,0 +1,16 @@ +set hive.auto.convert.join = true; + +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +explain +FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value +where (src1.ds = '2008-04-08' or src1.ds = '2008-04-09' )and (src1.hr = '12' or src1.hr = '11'); + + +FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value +where (src1.ds = '2008-04-08' or src1.ds = '2008-04-09' )and (src1.hr = '12' or src1.hr = '11'); + + +SELECT sum(hash(dest1.key,dest1.value)) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join2.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join2.q new file mode 100644 index 0000000000000..f598ccb4da234 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join2.q @@ -0,0 +1,13 @@ +set hive.auto.convert.join = true; + +CREATE TABLE dest_j2(key INT, value STRING) STORED AS TEXTFILE; + +explain +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) +INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value; + + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) +INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value; + +SELECT sum(hash(dest_j2.key,dest_j2.value)) FROM dest_j2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join20.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join20.q new file mode 100644 index 0000000000000..e61958414294c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join20.q @@ -0,0 +1,31 @@ +set hive.auto.convert.join = true; + +explain +select sum(hash(a.k1,a.v1,a.k2,a.v2,a.k3,a.v3)) +from ( +SELECT src1.key as k1, src1.value as v1, src2.key as k2, src2.value as v2 , src3.key as k3, src3.value as v3 +FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY k1,v1,k2,v2,k3,v3 +)a; + +select sum(hash(a.k1,a.v1,a.k2,a.v2,a.k3,a.v3)) +from ( +SELECT src1.key as k1, src1.value as v1, src2.key as k2, src2.value as v2 , src3.key as k3, src3.value as v3 +FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY k1,v1,k2,v2,k3,v3 +)a; + +explain +select sum(hash(a.k1,a.v1,a.k2,a.v2,a.k3,a.v3)) +from ( +SELECT src1.key as k1, src1.value as v1, src2.key as k2, src2.value as v2 , src3.key as k3, src3.value as v3 +FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key < 15) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY k1,v1,k2,v2,k3,v3 +)a; + +select sum(hash(a.k1,a.v1,a.k2,a.v2,a.k3,a.v3)) +from ( +SELECT src1.key as k1, src1.value as v1, src2.key as k2, src2.value as v2 , src3.key as k3, src3.value as v3 +FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key < 15) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY k1,v1,k2,v2,k3,v3 +)a; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join21.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join21.q new file mode 100644 index 0000000000000..e77f922aaada0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join21.q @@ -0,0 +1,5 @@ +set hive.auto.convert.join = true; +explain +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join22.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join22.q new file mode 100644 index 0000000000000..b5eec90d09de1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join22.q @@ -0,0 +1,5 @@ +set hive.auto.convert.join = true; +explain +SELECT sum(hash(src5.src1_value)) FROM (SELECT src3.*, src4.value as src4_value, src4.key as src4_key FROM src src4 JOIN (SELECT src2.*, src1.key as src1_key, src1.value as src1_value FROM src src1 JOIN src src2 ON src1.key = src2.key) src3 ON src3.src1_key = src4.key) src5; + +SELECT sum(hash(src5.src1_value)) FROM (SELECT src3.*, src4.value as src4_value, src4.key as src4_key FROM src src4 JOIN (SELECT src2.*, src1.key as src1_key, src1.value as src1_value FROM src src1 JOIN src src2 ON src1.key = src2.key) src3 ON src3.src1_key = src4.key) src5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join23.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join23.q new file mode 100644 index 0000000000000..191d269ba840c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join23.q @@ -0,0 +1,6 @@ +set hive.auto.convert.join = true; + +explain +SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value; + +SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join24.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join24.q new file mode 100644 index 0000000000000..e8b26481024ba --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join24.q @@ -0,0 +1,13 @@ +set hive.auto.convert.join = true; + +create table tst1(key STRING, cnt INT); + +INSERT OVERWRITE TABLE tst1 +SELECT a.key, count(1) FROM src a group by a.key; + +explain +SELECT sum(a.cnt) FROM tst1 a JOIN tst1 b ON a.key = b.key; + +SELECT sum(a.cnt) FROM tst1 a JOIN tst1 b ON a.key = b.key; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join25.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join25.q new file mode 100644 index 0000000000000..eaf7489a17025 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join25.q @@ -0,0 +1,30 @@ +set hive.auto.convert.join = true; +set hive.mapjoin.localtask.max.memory.usage = 0.0001; +set hive.mapjoin.check.memory.rows = 2; +set hive.auto.convert.join.noconditionaltask = false; + +-- This test tests the scenario when the mapper dies. So, create a conditional task for the mapjoin +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value +where (src1.ds = '2008-04-08' or src1.ds = '2008-04-09' )and (src1.hr = '12' or src1.hr = '11'); + +SELECT sum(hash(dest1.key,dest1.value)) FROM dest1; + + + +CREATE TABLE dest_j2(key INT, value STRING) STORED AS TEXTFILE; + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) +INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value; + +SELECT sum(hash(dest_j2.key,dest_j2.value)) FROM dest_j2; + +CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE; + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value; + +SELECT sum(hash(dest_j1.key,dest_j1.value)) FROM dest_j1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join26.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join26.q new file mode 100644 index 0000000000000..16fbfe6cfbe53 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join26.q @@ -0,0 +1,10 @@ +CREATE TABLE dest_j1(key INT, cnt INT); +set hive.auto.convert.join = true; +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, count(1) FROM src1 x JOIN src y ON (x.key = y.key) group by x.key; + +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, count(1) FROM src1 x JOIN src y ON (x.key = y.key) group by x.key; + +select * from dest_j1 x order by x.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join27.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join27.q new file mode 100644 index 0000000000000..2dc5845ed3b82 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join27.q @@ -0,0 +1,25 @@ +set hive.auto.convert.join = true; + +explain +SELECT count(1) +FROM +( +SELECT src.key, src.value from src +UNION ALL +SELECT DISTINCT src.key, src.value from src +) src_12 +JOIN +(SELECT src.key as k, src.value as v from src) src3 +ON src_12.key = src3.k AND src3.k < 200; + + +SELECT count(1) +FROM +( +SELECT src.key, src.value from src +UNION ALL +SELECT DISTINCT src.key, src.value from src +) src_12 +JOIN +(SELECT src.key as k, src.value as v from src) src3 +ON src_12.key = src3.k AND src3.k < 200; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join28.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join28.q new file mode 100644 index 0000000000000..61b9d84291eda --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join28.q @@ -0,0 +1,14 @@ +set hive.mapjoin.smalltable.filesize = 1; + +set hive.auto.convert.join = true; +explain +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) LEFT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) LEFT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join29.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join29.q new file mode 100644 index 0000000000000..c9f9191258231 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join29.q @@ -0,0 +1,45 @@ +set hive.auto.convert.join = true; +explain +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) LEFT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) LEFT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) LEFT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) LEFT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) LEFT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) LEFT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join3.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join3.q new file mode 100644 index 0000000000000..954765444c4ea --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join3.q @@ -0,0 +1,12 @@ +set hive.auto.convert.join = true; + +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +explain +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value; + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value; + +SELECT sum(hash(dest1.key,dest1.value)) FROM dest1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join30.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join30.q new file mode 100644 index 0000000000000..9e31f0ffe2aeb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join30.q @@ -0,0 +1,151 @@ +set hive.auto.convert.join = true; + +explain +FROM +(SELECT src.* FROM src sort by key) x +JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +select sum(hash(Y.key,Y.value)); + +FROM +(SELECT src.* FROM src sort by key) x +JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +select sum(hash(Y.key,Y.value)); + +explain +FROM +(SELECT src.* FROM src sort by key) x +LEFT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +select sum(hash(Y.key,Y.value)); + +FROM +(SELECT src.* FROM src sort by key) x +LEFT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +select sum(hash(Y.key,Y.value)); + +explain +FROM +(SELECT src.* FROM src sort by key) x +RIGHT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +select sum(hash(Y.key,Y.value)); + +FROM +(SELECT src.* FROM src sort by key) x +RIGHT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +select sum(hash(Y.key,Y.value)); + +explain +FROM +(SELECT src.* FROM src sort by key) x +JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); + +FROM +(SELECT src.* FROM src sort by key) x +JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); + +explain +FROM +(SELECT src.* FROM src sort by key) x +JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +LEFT OUTER JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); + +FROM +(SELECT src.* FROM src sort by key) x +JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +LEFT OUTER JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); + +explain +FROM +(SELECT src.* FROM src sort by key) x +LEFT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +LEFT OUTER JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); + +FROM +(SELECT src.* FROM src sort by key) x +LEFT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +LEFT OUTER JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); + +explain +FROM +(SELECT src.* FROM src sort by key) x +LEFT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +RIGHT OUTER JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); + +FROM +(SELECT src.* FROM src sort by key) x +LEFT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +RIGHT OUTER JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); + +explain +FROM +(SELECT src.* FROM src sort by key) x +RIGHT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +RIGHT OUTER JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); + +FROM +(SELECT src.* FROM src sort by key) x +RIGHT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +RIGHT OUTER JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join31.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join31.q new file mode 100644 index 0000000000000..cfa5eba17569c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join31.q @@ -0,0 +1,22 @@ +set hive.auto.convert.join = true; + +explain +FROM +(SELECT src.* FROM src sort by key) x +RIGHT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); + +FROM +(SELECT src.* FROM src sort by key) x +RIGHT OUTER JOIN +(SELECT src.* FROM src sort by value) Y +ON (x.key = Y.key) +JOIN +(SELECT src.* FROM src sort by value) Z +ON (x.key = Z.key) +select sum(hash(Y.key,Y.value)); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join32.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join32.q new file mode 100644 index 0000000000000..289bfbc6da917 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join32.q @@ -0,0 +1,78 @@ +set hive.auto.convert.join=true; + +-- empty tables +create table studenttab10k (name string, age int, gpa double); +create table votertab10k (name string, age int, registration string, contributions float); + +explain select s.name, count(distinct registration) +from studenttab10k s join votertab10k v +on (s.name = v.name) +group by s.name; + +select s.name, count(distinct registration) +from studenttab10k s join votertab10k v +on (s.name = v.name) +group by s.name; + +set hive.optimize.bucketmapjoin=true; +set hive.optimize.bucketmapjoin.sortedmerge=true; +set hive.auto.convert.sortmerge.join=true; + +-- smb +create table studenttab10k_smb (name string, age int, gpa double) clustered by (name) sorted by (name) into 2 buckets; +create table votertab10k_smb (name string, age int, registration string, contributions float) clustered by (name) sorted by (name) into 2 buckets; + +explain select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name; + +select s.name, count(distinct registration) +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; + +explain select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name; + +select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name; + +-- smb + partitions +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'); + +explain select s.name, count(distinct registration) +from studenttab10k_part s join votertab10k_part v +on (s.name = v.name) +where s.p = 'bar' +and v.p = 'bar' +group by s.name; + +select s.name, count(distinct registration) +from studenttab10k_part s join votertab10k_part v +on (s.name = v.name) +where s.p = 'bar' +and v.p = 'bar' +group by s.name; + +drop table studenttab10k; +drop table votertab10k; +drop table studenttab10k_smb; +drop table votertab10k_smb; +drop table studenttab10k_part; +drop table votertab10k_part; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join4.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join4.q new file mode 100644 index 0000000000000..1c2ad5f60b3e1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join4.q @@ -0,0 +1,34 @@ +set hive.auto.convert.join = true; + +CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING) STORED AS TEXTFILE; + +explain +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + LEFT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + LEFT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + +SELECT sum(hash(dest1.c1,dest1.c2,dest1.c3,dest1.c4)) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join5.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join5.q new file mode 100644 index 0000000000000..945629242329f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join5.q @@ -0,0 +1,34 @@ +set hive.auto.convert.join = true; + +CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING) STORED AS TEXTFILE; + +explain +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + RIGHT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + RIGHT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + +SELECT sum(hash(dest1.c1,dest1.c2,dest1.c3,dest1.c4)) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join6.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join6.q new file mode 100644 index 0000000000000..a0f8e55651db7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join6.q @@ -0,0 +1,35 @@ +set hive.auto.convert.join = true; + +CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING) STORED AS TEXTFILE; + +explain +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + FULL OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + FULL OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + + +SELECT sum(hash(dest1.c1,dest1.c2,dest1.c3,dest1.c4)) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join7.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join7.q new file mode 100644 index 0000000000000..ca105601d08bc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join7.q @@ -0,0 +1,46 @@ +set hive.auto.convert.join = true; + +CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING, c5 INT, c6 STRING) STORED AS TEXTFILE; + + +explain +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + FULL OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + LEFT OUTER JOIN + ( + FROM src src3 SELECT src3.key AS c5, src3.value AS c6 WHERE src3.key > 20 and src3.key < 25 + ) c + ON (a.c1 = c.c5) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4, c.c5 AS c5, c.c6 AS c6 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4, c.c5, c.c6; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + FULL OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + LEFT OUTER JOIN + ( + FROM src src3 SELECT src3.key AS c5, src3.value AS c6 WHERE src3.key > 20 and src3.key < 25 + ) c + ON (a.c1 = c.c5) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4, c.c5 AS c5, c.c6 AS c6 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4, c.c5, c.c6; + + +SELECT sum(hash(dest1.c1,dest1.c2,dest1.c3,dest1.c4,dest1.c5,dest1.c6)) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join8.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join8.q new file mode 100644 index 0000000000000..553fc25ba7df4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join8.q @@ -0,0 +1,34 @@ +set hive.auto.convert.join = true; + +CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING) STORED AS TEXTFILE; + +explain +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + LEFT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4 where c.c3 IS NULL AND c.c1 IS NOT NULL; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + LEFT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4 where c.c3 IS NULL AND c.c1 IS NOT NULL; + +SELECT sum(hash(dest1.c1,dest1.c2,dest1.c3,dest1.c4)) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join9.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join9.q new file mode 100644 index 0000000000000..fb3eafb441d70 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join9.q @@ -0,0 +1,14 @@ +set hive.auto.convert.join = true; + +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +explain +FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value where src1.ds = '2008-04-08' and src1.hr = '12'; + +FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value where src1.ds = '2008-04-08' and src1.hr = '12'; + + + +SELECT sum(hash(dest1.key,dest1.value)) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join_filters.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join_filters.q new file mode 100644 index 0000000000000..458504cdc3d04 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join_filters.q @@ -0,0 +1,82 @@ +set hive.auto.convert.join = true; + +CREATE TABLE myinput1(key int, value int); +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; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a RIGHT 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; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a FULL 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; + +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b ON a.key = b.value AND 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 JOIN myinput1 b ON a.key = b.key AND 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 JOIN myinput1 b ON a.value = b.value AND 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 JOIN myinput1 b ON a.value = b.value and a.key=b.key AND 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 = b.value AND 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.value = b.value AND 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 = b.key AND 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 = b.key and a.value=b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key = b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key = b.key AND 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 RIGHT OUTER JOIN myinput1 b ON a.value = b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key=b.key and a.value = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.key = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.key = b.key AND 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 FULL OUTER JOIN myinput1 b ON a.value = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.value = b.value and a.key=b.key AND 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.value=b.value AND 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) RIGHT OUTER JOIN myinput1 c ON (b.value=c.value AND c.key > 40 AND c.value > 50 AND c.key = c.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 RIGHT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) LEFT OUTER JOIN myinput1 c ON (b.value=c.value AND c.key > 40 AND c.value > 50 AND c.key = c.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 RIGHT OUTER JOIN myinput1 c ON a.value = b.value and b.value = c.value AND 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 AND c.key > 40 AND c.value > 50 AND c.key = c.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) from myinput1 a LEFT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) RIGHT OUTER JOIN myinput1 c ON (b.key=c.key AND c.key > 40 AND c.value > 50 AND c.key = c.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 RIGHT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) LEFT OUTER JOIN myinput1 c ON (b.key=c.key AND c.key > 40 AND c.value > 50 AND c.key = c.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 RIGHT OUTER JOIN myinput1 c ON a.value = b.value and b.key = c.key AND 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 AND c.key > 40 AND c.value > 50 AND c.key = c.value; + + +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; + +SET hive.optimize.bucketmapjoin = true; +SET hive.optimize.bucketmapjoin.sortedmerge = true; +SET hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +SET hive.outerjoin.supports.filters = false; + +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; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a RIGHT 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; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a FULL 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; + +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b ON a.key = b.value AND 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 JOIN myinput1 b ON a.key = b.key AND 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 JOIN myinput1 b ON a.value = b.value AND 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 JOIN myinput1 b ON a.value = b.value and a.key=b.key AND 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 = b.value AND 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.value = b.value AND 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 = b.key AND 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 = b.key and a.value=b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key = b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key = b.key AND 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 RIGHT OUTER JOIN myinput1 b ON a.value = b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key=b.key and a.value = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.key = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.key = b.key AND 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 FULL OUTER JOIN myinput1 b ON a.value = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.value = b.value and a.key=b.key AND 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.value=b.value AND 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) RIGHT OUTER JOIN myinput1 c ON (b.value=c.value AND c.key > 40 AND c.value > 50 AND c.key = c.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 RIGHT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) LEFT OUTER JOIN myinput1 c ON (b.value=c.value AND c.key > 40 AND c.value > 50 AND c.key = c.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 RIGHT OUTER JOIN myinput1 c ON a.value = b.value and b.value = c.value AND 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 AND c.key > 40 AND c.value > 50 AND c.key = c.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) from myinput1 a LEFT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) RIGHT OUTER JOIN myinput1 c ON (b.key=c.key AND c.key > 40 AND c.value > 50 AND c.key = c.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 RIGHT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) LEFT OUTER JOIN myinput1 c ON (b.key=c.key AND c.key > 40 AND c.value > 50 AND c.key = c.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 RIGHT OUTER JOIN myinput1 c ON a.value = b.value and b.key = c.key AND 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 AND c.key > 40 AND c.value > 50 AND c.key = c.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join_nulls.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join_nulls.q new file mode 100644 index 0000000000000..766348d7c001e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join_nulls.q @@ -0,0 +1,29 @@ +set hive.auto.convert.join = true; + +CREATE TABLE myinput1(key int, value int); +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; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a RIGHT OUTER JOIN myinput1 b; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b ON a.key = b.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b ON a.key = b.key; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b ON a.value = b.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b ON a.value = b.value and a.key=b.key; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key = b.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.value = b.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key = b.key; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key = b.key and a.value=b.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key = b.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key = b.key; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.value = b.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key=b.key and a.value = b.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.key = b.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.key = b.key; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.value = b.value; +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.value = b.value and a.key=b.key; + +SELECT sum(hash(a.key,a.value,b.key,b.value)) from myinput1 a LEFT OUTER JOIN myinput1 b ON (a.value=b.value) RIGHT OUTER JOIN myinput1 c ON (b.value=c.value); +SELECT sum(hash(a.key,a.value,b.key,b.value)) from myinput1 a RIGHT OUTER JOIN myinput1 b ON (a.value=b.value) LEFT OUTER JOIN myinput1 c ON (b.value=c.value); +SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b RIGHT OUTER JOIN myinput1 c ON a.value = b.value and b.value = c.value; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join_reordering_values.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_join_reordering_values.q new file mode 100644 index 0000000000000..46a4a0d5339a7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_join_reordering_values.q @@ -0,0 +1,31 @@ +-- 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; +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; +drop table if exists user_small; +create table user_small( userid int); +insert overwrite table user_small select key from testsrc limit 100; + +set hive.auto.convert.join.noconditionaltask.size = 200; +explain extended SELECT + `dim_pay_date`.`date` + , `deal`.`dealid` +FROM `orderpayment_small` `orderpayment` +JOIN `orderpayment_small` `dim_pay_date` ON `dim_pay_date`.`date` = `orderpayment`.`date` +JOIN `orderpayment_small` `deal` ON `deal`.`dealid` = `orderpayment`.`dealid` +JOIN `orderpayment_small` `order_city` ON `order_city`.`cityid` = `orderpayment`.`cityid` +JOIN `user_small` `user` ON `user`.`userid` = `orderpayment`.`userid` +limit 5; + +SELECT + `dim_pay_date`.`date` + , `deal`.`dealid` +FROM `orderpayment_small` `orderpayment` +JOIN `orderpayment_small` `dim_pay_date` ON `dim_pay_date`.`date` = `orderpayment`.`date` +JOIN `orderpayment_small` `deal` ON `deal`.`dealid` = `orderpayment`.`dealid` +JOIN `orderpayment_small` `order_city` ON `order_city`.`cityid` = `orderpayment`.`cityid` +JOIN `user_small` `user` ON `user`.`userid` = `orderpayment`.`userid` +limit 5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q new file mode 100644 index 0000000000000..5911c49afea7b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q @@ -0,0 +1,296 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +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; + +insert overwrite table tbl1 +select * from src where key < 10; + +insert overwrite table tbl2 +select * from src where key < 10; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +set hive.auto.convert.sortmerge.join=true; + +-- The join is being performed as part of sub-query. It should be converted to a sort-merge join +explain +select count(*) 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; + +select count(*) 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; + +-- The join is being performed as part of more than one sub-query. It should be converted to a sort-merge join +explain +select count(*) from +( + select key, count(*) 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 +) subq2; + +select count(*) from +( + select key, count(*) 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 +) subq2; + +-- A join is being performed across different sub-queries, where a join is being performed in each of them. +-- Each sub-query should be converted to a sort-merge join. +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; + +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; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join, although there is more than one level of sub-query +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key; + +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key; + +-- Both the tables are nested sub-queries i.e more then 1 level of sub-query. +-- The join should be converted to a sort-merge join +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key; + +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters and the join key +-- is not getting modified, it should be converted to a sort-merge join. Note that the sub-query modifies one +-- item, but that is not part of the join key. +explain +select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key; + +-- Since the join key is modified by the sub-query, neither sort-merge join not bucketized map-side +-- join should be performed +explain +select count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key; + +-- One of the tables is a sub-query and the other is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +-- There are more than 2 inputs to the join, all of them being sub-queries. +-- It should be converted to to a sort-merge join +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on (subq1.key = subq2.key) + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key); + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key); + +-- The join is being performed on a nested sub-query, and an aggregation is performed after that. +-- The join should be converted to a sort-merge join +explain +select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a; + +select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a; + +CREATE TABLE dest1(key int, value string); +CREATE TABLE dest2(key int, val1 string, val2 string); + +-- The join is followed by a multi-table insert. It should be converted to +-- a sort-merge join +explain +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 +insert overwrite table dest1 select key, val1 +insert overwrite table dest2 select key, val1, val2; + +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 +insert overwrite table dest1 select key, val1 +insert overwrite table dest2 select key, val1, val2; + +select * from dest1 order by key, value; +select * from dest2 order by key, val1, val2; + +DROP TABLE dest2; +CREATE TABLE dest2(key int, cnt int); + +-- The join is followed by a multi-table insert, and one of the inserts involves a reducer. +-- It should be converted to a sort-merge join +explain +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 +insert overwrite table dest1 select key, val1 +insert overwrite table dest2 select key, count(*) group by key; + +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 +insert overwrite table dest1 select key, val1 +insert overwrite table dest2 select key, count(*) group by key; + +select * from dest1 order by key, value; +select * from dest2 order by key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q new file mode 100644 index 0000000000000..e76b560f41899 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q @@ -0,0 +1,36 @@ +-- 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/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'); + +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; + +-- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q new file mode 100644 index 0000000000000..83e1cec536100 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q @@ -0,0 +1,57 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +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; + +insert overwrite table tbl1 +select * from src where key < 10; + +insert overwrite table tbl2 +select * from src where key < 10; + +set hive.auto.convert.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +set hive.auto.convert.sortmerge.join=true; +set hive.auto.convert.sortmerge.join.to.mapjoin=false; +-- One of the subqueries contains a union, so it should not be converted to a sort-merge join. +explain +select count(*) from + ( + select * from + (select a.key as key, a.value as value from tbl1 a where key < 6 + union all + select a.key as key, a.value as value from tbl1 a where key < 6 + ) usubq1 ) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +select count(*) from + ( + select * from + (select a.key as key, a.value as value from tbl1 a where key < 6 + union all + select a.key as key, a.value as value from tbl1 a where key < 6 + ) usubq1 ) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +-- One of the subqueries contains a groupby, so it should not be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, count(*) as value from tbl1 a where key < 6 group by a.key) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key as key, count(*) as value from tbl1 a where key < 6 group by a.key) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q new file mode 100644 index 0000000000000..f9fa1e4d4e1db --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q @@ -0,0 +1,36 @@ +-- 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'); + +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'); + +set hive.auto.convert.join=true; + +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.auto.convert.sortmerge.join=true; +set hive.optimize.bucketmapjoin=true; +set hive.optimize.bucketmapjoin.sortedmerge=true; + +-- Since size is being used to find the big table, the order of the tables in the join does not matter +-- The tables are only bucketed and not sorted, the join should not be converted +-- Currenly, a join is only converted to a sort-merge join without a hint, automatic conversion to +-- bucketized mapjoin is not done +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +-- The join is converted to a bucketed mapjoin with a mapjoin hint +explain extended select /*+ mapjoin(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /*+ mapjoin(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q new file mode 100644 index 0000000000000..db53a656a850a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q @@ -0,0 +1,31 @@ +-- 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/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'); + +set hive.auto.convert.join=true; +set hive.auto.convert.sortmerge.join=true; +set hive.optimize.bucketmapjoin = true; +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'); + +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/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q new file mode 100644 index 0000000000000..28bbc2d8f1a3e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q @@ -0,0 +1,81 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +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; + +insert overwrite table tbl1 select * from src where key < 10; +insert overwrite table tbl2 select * from src where key < 10; + +CREATE TABLE dest1(k1 int, k2 int); +CREATE TABLE dest2(k1 string, k2 string); + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.auto.convert.sortmerge.join.to.mapjoin=false; +set hive.auto.convert.sortmerge.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.auto.convert.join=true; + +-- A SMB join followed by a mutli-insert +explain +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2; + +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2; + +select * from dest1 order by k1, k2; +select * from dest2 order by k1, k2; + +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=200; + +-- A SMB join followed by a mutli-insert +explain +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2; + +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2; + +select * from dest1 order by k1, k2; +select * from dest2 order by k1, k2; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; +-- A SMB join followed by a mutli-insert +explain +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2; + +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2; + +select * from dest1 order by k1, k2; +select * from dest2 order by k1, k2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q new file mode 100644 index 0000000000000..4cda4df62ec35 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q @@ -0,0 +1,29 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +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; + +insert overwrite table tbl1 select * from src where key < 20; +insert overwrite table tbl2 select * from src where key < 10; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.auto.convert.sortmerge.join.to.mapjoin=true; +set hive.auto.convert.sortmerge.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.auto.convert.join=true; + +-- Since tbl1 is the bigger table, tbl1 Left Outer Join tbl2 can be performed +explain +select count(*) FROM tbl1 a LEFT OUTER JOIN tbl2 b ON a.key = b.key; +select count(*) FROM tbl1 a LEFT OUTER JOIN tbl2 b ON a.key = b.key; + +insert overwrite table tbl2 select * from src where key < 200; + +-- Since tbl2 is the bigger table, tbl1 Right Outer Join tbl2 can be performed +explain +select count(*) FROM tbl1 a RIGHT OUTER JOIN tbl2 b ON a.key = b.key; +select count(*) FROM tbl1 a RIGHT OUTER JOIN tbl2 b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q new file mode 100644 index 0000000000000..c7bcae64b38d5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q @@ -0,0 +1,23 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +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; + +insert overwrite table tbl1 select * from src where key < 20; +insert overwrite table tbl2 select * from src where key < 10; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.auto.convert.sortmerge.join.to.mapjoin=true; +set hive.auto.convert.sortmerge.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.auto.convert.join=true; + +explain +select count(*) FROM tbl1 a LEFT OUTER JOIN tbl2 b ON a.key = b.key; + +explain +select count(*) FROM tbl1 a RIGHT OUTER JOIN tbl2 b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q new file mode 100644 index 0000000000000..75339778af802 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q @@ -0,0 +1,32 @@ +-- 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'); + +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-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; +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.LeftmostBigTableSelectorForAutoSMJ; + +-- Since the leftmost table is assumed as the big table, arrange the tables in the join accordingly +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; +set hive.mapjoin.localtask.max.memory.usage = 0.0001; +set hive.mapjoin.check.memory.rows = 2; + +-- The mapjoin should fail resulting in the sort-merge join +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q new file mode 100644 index 0000000000000..39a695fdf039d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q @@ -0,0 +1,31 @@ +-- 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-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'); + +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; + +-- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q new file mode 100644 index 0000000000000..6072272c478a3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q @@ -0,0 +1,33 @@ +-- 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-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'); + +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.AvgPartitionSizeBasedBigTableSelectorForAutoSMJ; + +-- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q new file mode 100644 index 0000000000000..a28ce3de5fc63 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q @@ -0,0 +1,28 @@ +-- 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; + +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; + +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.AvgPartitionSizeBasedBigTableSelectorForAutoSMJ; + +-- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; +set hive.auto.convert.join=true; +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q new file mode 100644 index 0000000000000..309987b8f7711 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q @@ -0,0 +1,73 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +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; +CREATE TABLE tbl3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE tbl4(key int, value string) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; + +insert overwrite table tbl1 select * from src; +insert overwrite table tbl2 select * from src; +insert overwrite table tbl3 select * from src; +insert overwrite table tbl4 select * from src; + +set hive.auto.convert.sortmerge.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=200; +set hive.auto.convert.sortmerge.join.to.mapjoin=false; + +-- A SMB join is being followed by a regular join on a non-bucketed table on a different key + +-- Three tests below are all the same query with different alias, which changes dispatch order of GenMapRedWalker +-- This is dependent to iteration order of HashMap, so can be meaningless in non-sun jdk +-- b = TS[0]-OP[13]-MAPJOIN[11]-RS[6]-JOIN[8]-SEL[9]-FS[10] +-- c = TS[1]-RS[7]-JOIN[8] +-- a = TS[2]-MAPJOIN[11] +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value; +select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value; + +-- d = TS[0]-RS[7]-JOIN[8]-SEL[9]-FS[10] +-- b = TS[1]-OP[13]-MAPJOIN[11]-RS[6]-JOIN[8] +-- a = TS[2]-MAPJOIN[11] +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src d on d.value = a.value; +select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src d on d.value = a.value; + +-- b = TS[0]-OP[13]-MAPJOIN[11]-RS[6]-JOIN[8]-SEL[9]-FS[10] +-- a = TS[1]-MAPJOIN[11] +-- h = TS[2]-RS[7]-JOIN[8] +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src h on h.value = a.value; +select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src h on h.value = a.value; + +-- A SMB join is being followed by a regular join on a non-bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.key = a.key; +select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.key = a.key; + +-- A SMB join is being followed by a regular join on a bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.key = a.key; +select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.key = a.key; + +-- A SMB join is being followed by a regular join on a bucketed table on a different key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value; +select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; + +-- A SMB join is being followed by a regular join on a non-bucketed table on a different key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value; +select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value; + +-- A SMB join is being followed by a regular join on a non-bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.key = a.key; +select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.key = a.key; + +-- A SMB join is being followed by a regular join on a bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.key = a.key; +select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.key = a.key; + +-- A SMB join is being followed by a regular join on a bucketed table on a different key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value; +select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q new file mode 100644 index 0000000000000..d62f6377218c6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q @@ -0,0 +1,36 @@ +-- 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-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-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; +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.AvgPartitionSizeBasedBigTableSelectorForAutoSMJ; + +-- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q new file mode 100644 index 0000000000000..6302a1be1477c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q @@ -0,0 +1,40 @@ +-- 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-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'); + +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; + +-- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; +set hive.mapjoin.localtask.max.memory.usage = 0.0001; +set hive.mapjoin.check.memory.rows = 2; + +-- The mapjoin should fail resulting in the sort-merge join +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q new file mode 100644 index 0000000000000..ef63dfed5267a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q @@ -0,0 +1,536 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +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; + +insert overwrite table tbl1 +select * from src where key < 10; + +insert overwrite table tbl2 +select * from src where key < 10; + +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; +-- The join is being performed as part of sub-query. It should be converted to a sort-merge join +explain +select count(*) 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; + +select count(*) 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; + +-- The join is being performed as part of sub-query. It should be converted to a sort-merge join +-- Add a order by at the end to make the results deterministic. +explain +select key, count(*) 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 +order by key; + +select key, count(*) 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 +order by key; + +-- The join is being performed as part of more than one sub-query. It should be converted to a sort-merge join +explain +select count(*) from +( + select key, count(*) 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 +) subq2; + +select count(*) from +( + select key, count(*) 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 +) subq2; + +-- A join is being performed across different sub-queries, where a join is being performed in each of them. +-- Each sub-query should be converted to a sort-merge join. +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; + +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; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join, although there is more than one level of sub-query +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key; + +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key; + +-- Both the tables are nested sub-queries i.e more then 1 level of sub-query. +-- The join should be converted to a sort-merge join +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key; + +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters and the join key +-- is not getting modified, it should be converted to a sort-merge join. Note that the sub-query modifies one +-- item, but that is not part of the join key. +explain +select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key; + +-- Since the join key is modified by the sub-query, neither sort-merge join not bucketized mapside +-- join should be performed +explain +select count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key; + +-- The left table is a sub-query and the right table is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +-- The right table is a sub-query and the left table is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.key; + +select count(*) from tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.key; + +-- There are more than 2 inputs to the join, all of them being sub-queries. +-- It should be converted to to a sort-merge join +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on (subq1.key = subq2.key) + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key); + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key); + +-- The join is being performed on a nested sub-query, and an aggregation is performed after that. +-- The join should be converted to a sort-merge join +explain +select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a; + +select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; + +-- The join is being performed as part of sub-query. It should be converted to a sort-merge join +explain +select count(*) 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; + +select count(*) 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; + +-- The join is being performed as part of sub-query. It should be converted to a sort-merge join +-- Add a order by at the end to make the results deterministic. +explain +select key, count(*) 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 +order by key; + +select key, count(*) 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 +order by key; + +-- The join is being performed as part of more than one sub-query. It should be converted to a sort-merge join +explain +select count(*) from +( + select key, count(*) 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 +) subq2; + +select count(*) from +( + select key, count(*) 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 +) subq2; + +-- A join is being performed across different sub-queries, where a join is being performed in each of them. +-- Each sub-query should be converted to a sort-merge join. +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; + +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; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join, although there is more than one level of sub-query +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key; + +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key; + +-- Both the tables are nested sub-queries i.e more then 1 level of sub-query. +-- The join should be converted to a sort-merge join +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key; + +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters and the join key +-- is not getting modified, it should be converted to a sort-merge join. Note that the sub-query modifies one +-- item, but that is not part of the join key. +explain +select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key; + +-- The left table is a sub-query and the right table is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +-- The right table is a sub-query and the left table is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.key; + +select count(*) from tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.key; + +-- There are more than 2 inputs to the join, all of them being sub-queries. +-- It should be converted to to a sort-merge join +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on (subq1.key = subq2.key) + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key); + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key); + +-- The join is being performed on a nested sub-query, and an aggregation is performed after that. +-- The join should be converted to a sort-merge join +explain +select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a; + +select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/autogen_colalias.q b/src/test/hive/ql/src/test/queries/clientpositive/autogen_colalias.q new file mode 100644 index 0000000000000..e247a733ae60a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/autogen_colalias.q @@ -0,0 +1,25 @@ +CREATE TEMPORARY FUNCTION test_max AS 'org.apache.hadoop.hive.ql.udf.UDAFTestMax'; + +create table dest_grouped_old1 as select 1+1, 2+2 as zz, src.key, test_max(length(src.value)), count(src.value), sin(count(src.value)), count(sin(src.value)), unix_timestamp(), CAST(SUM(IF(value > 10, value, 1)) AS INT), if(src.key > 1, +1, +0) + from src group by src.key; +describe dest_grouped_old1; + +create table dest_grouped_old2 as select distinct src.key from src; +describe dest_grouped_old2; + +set hive.autogen.columnalias.prefix.label=column_; +set hive.autogen.columnalias.prefix.includefuncname=true; + +create table dest_grouped_new1 as select 1+1, 2+2 as zz, ((src.key % 2)+2)/2, test_max(length(src.value)), count(src.value), sin(count(src.value)), count(sin(src.value)), unix_timestamp(), CAST(SUM(IF(value > 10, value, 1)) AS INT), if(src.key > 10, + (src.key +5) % 2, +0) +from src group by src.key; +describe dest_grouped_new1; + +create table dest_grouped_new2 as select distinct src.key from src; +describe dest_grouped_new2; + +-- Drop the temporary function at the end till HIVE-3160 gets fixed +DROP TEMPORARY FUNCTION test_max; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_change_schema.q b/src/test/hive/ql/src/test/queries/clientpositive/avro_change_schema.q new file mode 100644 index 0000000000000..08e72c408efde --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/avro_change_schema.q @@ -0,0 +1,28 @@ +-- verify that we can update the table properties +CREATE TABLE avro2 +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' +TBLPROPERTIES ('avro.schema.literal'='{ "namespace": "org.apache.hive", + "name": "first_schema", + "type": "record", + "fields": [ + { "name":"string1", "type":"string" }, + { "name":"string2", "type":"string" } + ] }'); + +DESCRIBE avro2; + +ALTER TABLE avro2 SET TBLPROPERTIES ('avro.schema.literal'='{ "namespace": "org.apache.hive", + "name": "second_schema", + "type": "record", + "fields": [ + { "name":"int1", "type":"int" }, + { "name":"float1", "type":"float" }, + { "name":"double1", "type":"double" } + ] }'); + +DESCRIBE avro2; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_compression_enabled.q b/src/test/hive/ql/src/test/queries/clientpositive/avro_compression_enabled.q new file mode 100644 index 0000000000000..8367206231974 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/avro_compression_enabled.q @@ -0,0 +1,43 @@ +-- verify that new joins bring in correct schemas (including evolved schemas) + +CREATE TABLE doctors4 +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "testing.hive.avro.serde", + "name": "doctors", + "type": "record", + "fields": [ + { + "name":"number", + "type":"int", + "doc":"Order of playing the role" + }, + { + "name":"first_name", + "type":"string", + "doc":"first name of actor playing role" + }, + { + "name":"last_name", + "type":"string", + "doc":"last name of actor playing role" + }, + { + "name":"extra_field", + "type":"string", + "doc:":"an extra field not in the original file", + "default":"fishfingers and custard" + } + ] +}'); + +LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors4; + +set hive.exec.compress.output=true; + +select count(*) from src; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_evolved_schemas.q b/src/test/hive/ql/src/test/queries/clientpositive/avro_evolved_schemas.q new file mode 100644 index 0000000000000..3fe8ff12b4506 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/avro_evolved_schemas.q @@ -0,0 +1,42 @@ +-- verify that new fields in schema get propagated to table scans +CREATE TABLE doctors_with_new_field +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "testing.hive.avro.serde", + "name": "doctors", + "type": "record", + "fields": [ + { + "name":"number", + "type":"int", + "doc":"Order of playing the role" + }, + { + "name":"first_name", + "type":"string", + "doc":"first name of actor playing role" + }, + { + "name":"last_name", + "type":"string", + "doc":"last name of actor playing role" + }, + { + "name":"extra_field", + "type":"string", + "doc:":"an extra field not in the original file", + "default":"fishfingers and custard" + } + ] +}'); + +DESCRIBE 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/src/test/hive/ql/src/test/queries/clientpositive/avro_joins.q b/src/test/hive/ql/src/test/queries/clientpositive/avro_joins.q new file mode 100644 index 0000000000000..25b77c06270d0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/avro_joins.q @@ -0,0 +1,79 @@ +-- verify that new joins bring in correct schemas (including evolved schemas) + +CREATE TABLE doctors4 +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "testing.hive.avro.serde", + "name": "doctors", + "type": "record", + "fields": [ + { + "name":"number", + "type":"int", + "doc":"Order of playing the role" + }, + { + "name":"first_name", + "type":"string", + "doc":"first name of actor playing role" + }, + { + "name":"last_name", + "type":"string", + "doc":"last name of actor playing role" + }, + { + "name":"extra_field", + "type":"string", + "doc:":"an extra field not in the original file", + "default":"fishfingers and custard" + } + ] +}'); + +DESCRIBE doctors4; + +LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors4; + +CREATE TABLE episodes +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "testing.hive.avro.serde", + "name": "episodes", + "type": "record", + "fields": [ + { + "name":"title", + "type":"string", + "doc":"episode title" + }, + { + "name":"air_date", + "type":"string", + "doc":"initial date" + }, + { + "name":"doctor", + "type":"int", + "doc":"main actor playing the Doctor in episode" + } + ] +}'); + +DESCRIBE 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) +ORDER BY d.last_name, e.title; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_nullable_fields.q b/src/test/hive/ql/src/test/queries/clientpositive/avro_nullable_fields.q new file mode 100644 index 0000000000000..584c6f740ba91 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/avro_nullable_fields.q @@ -0,0 +1,59 @@ +-- Verify that nullable fields properly work +CREATE TABLE test_serializer(string1 STRING, + int1 INT, + tinyint1 TINYINT, + smallint1 SMALLINT, + bigint1 BIGINT, + boolean1 BOOLEAN, + float1 FLOAT, + double1 DOUBLE, + list1 ARRAY, + map1 MAP, + struct1 STRUCT, + enum1 STRING, + nullableint INT, + bytes1 BINARY, + fixed1 BINARY) + 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; + +CREATE TABLE as_avro + ROW FORMAT + SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + TBLPROPERTIES ( + 'avro.schema.literal'='{ + "namespace": "com.howdy", + "name": "some_schema", + "type": "record", + "fields": [ + { "name": "string1", "type": ["null", "string"] }, + { "name": "int1", "type": ["null", "int"] }, + { "name": "tinyint1", "type": ["null", "int"] }, + { "name": "smallint1", "type": ["null", "int"] }, + { "name": "bigint1", "type": ["null", "long"] }, + { "name": "boolean1", "type": ["null", "boolean"] }, + { "name": "float1", "type": ["null", "float"] }, + { "name": "double1", "type": ["null", "double"] }, + { "name": "list1", "type": ["null", {"type": "array", "items": "string"}] }, + { "name": "map1", "type": ["null", {"type": "map", "values": "int"}] }, + { "name": "struct1", "type": ["null", {"type": "record", "name": "struct1_name", "fields": [ + { "name": "sInt", "type": "int" }, + { "name": "sBoolean", "type": "boolean" }, + { "name": "sString", "type": "string" } + ]}] }, + { "name": "enum1", "type": ["null", {"type": "enum", "name": "enum1_values", "symbols": ["BLUE", "RED", "GREEN"]}] }, + { "name": "nullableint", "type": ["null", "int"] }, + { "name": "bytes1", "type": ["null", "bytes"] }, + { "name": "fixed1", "type": ["null", {"type": "fixed", "name": "threebytes", "size": 3}] } + ] + }' + ) +; + +INSERT OVERWRITE TABLE as_avro SELECT * FROM test_serializer; +SELECT * FROM as_avro; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_partitioned.q b/src/test/hive/ql/src/test/queries/clientpositive/avro_partitioned.q new file mode 100644 index 0000000000000..8e4d40f2bd59b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/avro_partitioned.q @@ -0,0 +1,73 @@ +-- verify that new joins bring in correct schemas (including evolved schemas) +CREATE TABLE episodes +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "testing.hive.avro.serde", + "name": "episodes", + "type": "record", + "fields": [ + { + "name":"title", + "type":"string", + "doc":"episode title" + }, + { + "name":"air_date", + "type":"string", + "doc":"initial date" + }, + { + "name":"doctor", + "type":"int", + "doc":"main actor playing the Doctor in episode" + } + ] +}'); + +LOAD DATA LOCAL INPATH '../data/files/episodes.avro' INTO TABLE episodes; + +CREATE TABLE episodes_partitioned +PARTITIONED BY (doctor_pt INT) +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "testing.hive.avro.serde", + "name": "episodes", + "type": "record", + "fields": [ + { + "name":"title", + "type":"string", + "doc":"episode title" + }, + { + "name":"air_date", + "type":"string", + "doc":"initial date" + }, + { + "name":"doctor", + "type":"int", + "doc":"main actor playing the Doctor in episode" + } + ] +}'); + +SET hive.exec.dynamic.partition.mode=nonstrict; +INSERT OVERWRITE TABLE episodes_partitioned PARTITION (doctor_pt) SELECT title, air_date, doctor, doctor as doctor_pt FROM episodes; + +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; +-- Fetch w/filter to specific partition +SELECT * FROM episodes_partitioned WHERE doctor_pt = 6; +-- Fetch w/non-existant partition +SELECT * FROM episodes_partitioned WHERE doctor_pt = 7 LIMIT 5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_sanity_test.q b/src/test/hive/ql/src/test/queries/clientpositive/avro_sanity_test.q new file mode 100644 index 0000000000000..e3f8b07b30c59 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/avro_sanity_test.q @@ -0,0 +1,36 @@ +-- verify that we can actually read avro files +CREATE TABLE doctors +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "testing.hive.avro.serde", + "name": "doctors", + "type": "record", + "fields": [ + { + "name":"number", + "type":"int", + "doc":"Order of playing the role" + }, + { + "name":"first_name", + "type":"string", + "doc":"first name of actor playing role" + }, + { + "name":"last_name", + "type":"string", + "doc":"last name of actor playing role" + } + ] +}'); + +DESCRIBE doctors; + +LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors; + +SELECT * FROM doctors ORDER BY number; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_error_message.q b/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_error_message.q new file mode 100644 index 0000000000000..cf1fda1d53409 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_error_message.q @@ -0,0 +1,11 @@ +-- verify we get the sentinel schema if we don't provide one + +CREATE TABLE avro_with_no_schema +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat'; + +DESCRIBE avro_with_no_schema; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_literal.q b/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_literal.q new file mode 100644 index 0000000000000..d77f31077a40f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_literal.q @@ -0,0 +1,32 @@ +CREATE TABLE avro1 +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "org.apache.hive", + "name": "big_old_schema", + "type": "record", + "fields": [ + { "name":"string1", "type":"string" }, + { "name":"int1", "type":"int" }, + { "name":"tinyint1", "type":"int" }, + { "name":"smallint1", "type":"int" }, + { "name":"bigint1", "type":"long" }, + { "name":"boolean1", "type":"boolean" }, + { "name":"float1", "type":"float" }, + { "name":"double1", "type":"double" }, + { "name":"list1", "type":{"type":"array", "items":"string"} }, + { "name":"map1", "type":{"type":"map", "values":"int"} }, + { "name":"struct1", "type":{"type":"record", "name":"struct1_name", "fields": [ + { "name":"sInt", "type":"int" }, { "name":"sBoolean", "type":"boolean" }, { "name":"sString", "type":"string" } ] } }, + { "name":"union1", "type":["float", "boolean", "string"] }, + { "name":"enum1", "type":{"type":"enum", "name":"enum1_values", "symbols":["BLUE","RED", "GREEN"]} }, + { "name":"nullableint", "type":["int", "null"] }, + { "name":"bytes1", "type":"bytes" }, + { "name":"fixed1", "type":{"type":"fixed", "name":"threebytes", "size":3} } + ] }'); + +DESCRIBE avro1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ba_table1.q b/src/test/hive/ql/src/test/queries/clientpositive/ba_table1.q new file mode 100644 index 0000000000000..6587f2e929f99 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ba_table1.q @@ -0,0 +1,13 @@ +drop table ba_test; + +-- This query tests a) binary type works correctly in grammar b) string can be cast into binary c) binary can be stored in a table d) binary data can be loaded back again and queried d) order-by on a binary key + +create table ba_test (ba_key binary, ba_val binary) ; + +describe extended ba_test; + +from src insert overwrite table ba_test select cast (src.key as binary), cast (src.value as binary); + +select ba_key,ba_val from ba_test order by ba_key limit 10; + +drop table ba_test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ba_table2.q b/src/test/hive/ql/src/test/queries/clientpositive/ba_table2.q new file mode 100644 index 0000000000000..d3c3e0807f08c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ba_table2.q @@ -0,0 +1,16 @@ +drop table ba_test; + +-- All the test in ba_test1.q + using LazyBinarySerde instead of LazySimpleSerde + +create table ba_test (ba_key binary, ba_val binary) ; +alter table ba_test set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe'; + +describe extended ba_test; + +from src insert overwrite table ba_test select cast (src.key as binary), cast (src.value as binary); + +select ba_key, ba_val from ba_test order by ba_val limit 10; + +drop table ba_test; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ba_table3.q b/src/test/hive/ql/src/test/queries/clientpositive/ba_table3.q new file mode 100644 index 0000000000000..8137722965f6c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ba_table3.q @@ -0,0 +1,13 @@ +drop table ba_test; + +-- All the tests of ba_table1.q + test for a group-by and aggregation on a binary key. + +create table ba_test (ba_key binary, ba_val binary) ; + +from src insert overwrite table ba_test select cast (src.key as binary), cast (src.value as binary); + +select ba_test.ba_key, count(ba_test.ba_val) from ba_test group by ba_test.ba_key order by ba_key limit 5; + +drop table ba_test; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ba_table_udfs.q b/src/test/hive/ql/src/test/queries/clientpositive/ba_table_udfs.q new file mode 100644 index 0000000000000..7a3d3f7e4180f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ba_table_udfs.q @@ -0,0 +1,36 @@ +USE default; + +CREATE TABLE dest1(bytes1 BINARY, + bytes2 BINARY, + string STRING); + +FROM src INSERT OVERWRITE TABLE dest1 +SELECT + CAST(key AS BINARY), + CAST(value AS BINARY), + value +ORDER BY value +LIMIT 100; + +--Add in a null row for good measure +INSERT INTO TABLE dest1 SELECT NULL, NULL, NULL FROM dest1 LIMIT 1; + +-- this query tests all the udfs provided to work with binary types + +SELECT + bytes1, + bytes2, + string, + LENGTH(bytes1), + CONCAT(bytes1, bytes2), + SUBSTR(bytes2, 1, 4), + SUBSTR(bytes2, 3), + SUBSTR(bytes2, -4, 3), + HEX(bytes1), + UNHEX(HEX(bytes1)), + BASE64(bytes1), + UNBASE64(BASE64(bytes1)), + HEX(ENCODE(string, 'US-ASCII')), + DECODE(ENCODE(string, 'US-ASCII'), 'US-ASCII') +FROM dest1 +ORDER BY bytes2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ba_table_union.q b/src/test/hive/ql/src/test/queries/clientpositive/ba_table_union.q new file mode 100644 index 0000000000000..9804659d36625 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ba_table_union.q @@ -0,0 +1,14 @@ +drop table ba_test; + +-- this query tests ba_table1.q + nested queries with multiple operations on binary data types + union on binary types +create table ba_test (ba_key binary, ba_val binary) ; + +describe extended ba_test; + +from src insert overwrite table ba_test select cast (src.key as binary), cast (src.value as binary); + +select * from ( select key from src where key < 50 union all select cast(ba_key as string) as key from ba_test limit 50) unioned order by key limit 10; + +drop table ba_test; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/binary_constant.q b/src/test/hive/ql/src/test/queries/clientpositive/binary_constant.q new file mode 100644 index 0000000000000..e0a8b95401d0b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/binary_constant.q @@ -0,0 +1 @@ +select cast(cast('a' as binary) as string) from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/binary_output_format.q b/src/test/hive/ql/src/test/queries/clientpositive/binary_output_format.q new file mode 100644 index 0000000000000..c5d9abd062c4f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/binary_output_format.q @@ -0,0 +1,39 @@ +-- Create a table with binary output format +CREATE TABLE dest1(mydata STRING) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +WITH SERDEPROPERTIES ( + 'serialization.last.column.takes.rest'='true' +) +STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat'; + +-- Insert into that table using transform +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest1 +SELECT TRANSFORM(*) + USING 'cat' + AS mydata STRING + ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + WITH SERDEPROPERTIES ( + 'serialization.last.column.takes.rest'='true' + ) + RECORDREADER 'org.apache.hadoop.hive.ql.exec.BinaryRecordReader' +FROM src; + +INSERT OVERWRITE TABLE dest1 +SELECT TRANSFORM(*) + USING 'cat' + AS mydata STRING + ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + WITH SERDEPROPERTIES ( + 'serialization.last.column.takes.rest'='true' + ) + RECORDREADER 'org.apache.hadoop.hive.ql.exec.BinaryRecordReader' +FROM src; + +-- Test the result +SELECT * FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/binary_table_bincolserde.q b/src/test/hive/ql/src/test/queries/clientpositive/binary_table_bincolserde.q new file mode 100644 index 0000000000000..47a560c494aae --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/binary_table_bincolserde.q @@ -0,0 +1,16 @@ +drop table ba_test; + +-- Tests everything in binary_table_colserde.q + uses LazyBinaryColumnarSerde + +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.LazyBinaryColumnarSerDe'; + +describe extended ba_test; + +from src insert overwrite table ba_test select cast (src.key as binary), cast (src.value as binary); + +select ba_key, ba_val from ba_test order by ba_key limit 10; + +drop table ba_test; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/binary_table_colserde.q b/src/test/hive/ql/src/test/queries/clientpositive/binary_table_colserde.q new file mode 100644 index 0000000000000..eadf07d14d756 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/binary_table_colserde.q @@ -0,0 +1,15 @@ +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; + +describe extended ba_test; + +from src insert overwrite table ba_test select cast (src.key as binary), cast (src.value as binary); + +select ba_key, ba_val from ba_test order by ba_key limit 10; + +drop table ba_test; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/binarysortable_1.q b/src/test/hive/ql/src/test/queries/clientpositive/binarysortable_1.q new file mode 100644 index 0000000000000..a98a2305cfcaf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/binarysortable_1.q @@ -0,0 +1,21 @@ +CREATE TABLE mytable(key STRING, value STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '9' +STORED AS TEXTFILE; + +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 +FROM ( + SELECT key, sum(value) as value + FROM mytable + GROUP BY key +) a; + +SELECT REGEXP_REPLACE(REGEXP_REPLACE(REGEXP_REPLACE(key, '\001', '^A'), '\0', '^@'), '\002', '^B'), value +FROM ( + SELECT key, sum(value) as value + FROM mytable + GROUP BY key +) a; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket1.q b/src/test/hive/ql/src/test/queries/clientpositive/bucket1.q new file mode 100644 index 0000000000000..ca9b9025e270f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucket1.q @@ -0,0 +1,13 @@ +set hive.enforce.bucketing = true; +set hive.exec.reducers.max = 200; + +CREATE TABLE bucket1_1(key int, value string) CLUSTERED BY (key) INTO 100 BUCKETS; + +explain extended +insert overwrite table bucket1_1 +select * from src; + +insert overwrite table bucket1_1 +select * from src; + +select * from bucket1_1 order by key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket2.q b/src/test/hive/ql/src/test/queries/clientpositive/bucket2.q new file mode 100644 index 0000000000000..d4bddf72ed74d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucket2.q @@ -0,0 +1,16 @@ +set hive.enforce.bucketing = true; +set hive.exec.reducers.max = 1; + +CREATE TABLE bucket2_1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; + +explain extended +insert overwrite table bucket2_1 +select * from src; + +insert overwrite table bucket2_1 +select * from src; + +explain +select * from bucket2_1 tablesample (bucket 1 out of 2) s order by key; + +select * from bucket2_1 tablesample (bucket 1 out of 2) s order by key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket3.q b/src/test/hive/ql/src/test/queries/clientpositive/bucket3.q new file mode 100644 index 0000000000000..d891b90b7b379 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucket3.q @@ -0,0 +1,19 @@ +set hive.enforce.bucketing = true; +set hive.exec.reducers.max = 1; + +CREATE TABLE bucket3_1(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS; + +explain extended +insert overwrite table bucket3_1 partition (ds='1') +select * from src; + +insert overwrite table bucket3_1 partition (ds='1') +select * from src; + +insert overwrite table bucket3_1 partition (ds='2') +select * from src; + +explain +select * from bucket3_1 tablesample (bucket 1 out of 2) s where ds = '1' order by key; + +select * from bucket3_1 tablesample (bucket 1 out of 2) s where ds = '1' order by key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket4.q b/src/test/hive/ql/src/test/queries/clientpositive/bucket4.q new file mode 100644 index 0000000000000..2b3f80543a224 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucket4.q @@ -0,0 +1,18 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +CREATE TABLE bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +explain extended +insert overwrite table bucket4_1 +select * from src; + +insert overwrite table bucket4_1 +select * from src; + +explain +select * from bucket4_1 tablesample (bucket 1 out of 2) s; + +select * from bucket4_1 tablesample (bucket 1 out of 2) s; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket5.q b/src/test/hive/ql/src/test/queries/clientpositive/bucket5.q new file mode 100644 index 0000000000000..877f8a50a0e35 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucket5.q @@ -0,0 +1,33 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles = true; +set hive.merge.mapredfiles = true; +set mapred.reduce.tasks = 2; + +-- Tests that when a multi insert inserts into a bucketed table and a table which is not bucketed +-- the bucketed table is not merged and the table which is not bucketed is + +CREATE TABLE bucketed_table(key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE unbucketed_table(key INT, value STRING); + +EXPLAIN EXTENDED +FROM src +INSERT OVERWRITE TABLE bucketed_table SELECT key, value +INSERT OVERWRITE TABLE unbucketed_table SELECT key, value cluster by key; + +FROM src +INSERT OVERWRITE TABLE bucketed_table SELECT key, value +INSERT OVERWRITE TABLE unbucketed_table SELECT key, value cluster by key; + +DESC FORMATTED bucketed_table; + +SELECT * FROM bucketed_table TABLESAMPLE (BUCKET 1 OUT OF 2) s LIMIT 10; +SELECT * FROM bucketed_table TABLESAMPLE (BUCKET 2 OUT OF 2) s LIMIT 10; + +-- Should be 2 (not merged) +SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM bucketed_table; + +-- Should be 1 (merged) +SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM unbucketed_table; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket_groupby.q b/src/test/hive/ql/src/test/queries/clientpositive/bucket_groupby.q new file mode 100644 index 0000000000000..a531ba70a7481 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucket_groupby.q @@ -0,0 +1,75 @@ +create table clustergroupby(key string, value string) partitioned by(ds string); +describe extended clustergroupby; +alter table clustergroupby clustered by (key) into 1 buckets; + +insert overwrite table clustergroupby partition (ds='100') select key, value from src sort by key; + +explain +select key, count(1) from clustergroupby where ds='100' group by key limit 10; +select key, count(1) from clustergroupby where ds='100' group by key limit 10; + +describe extended clustergroupby; +insert overwrite table clustergroupby partition (ds='101') select key, value from src distribute by key; + +--normal-- +explain +select key, count(1) from clustergroupby where ds='101' group by key limit 10; +select key, count(1) from clustergroupby where ds='101' group by key limit 10; + +--function-- +explain +select length(key), count(1) from clustergroupby where ds='101' group by length(key) limit 10; +select length(key), count(1) from clustergroupby where ds='101' group by length(key) limit 10; +explain +select abs(length(key)), count(1) from clustergroupby where ds='101' group by abs(length(key)) limit 10; +select abs(length(key)), count(1) from clustergroupby where ds='101' group by abs(length(key)) limit 10; + +--constant-- +explain +select key, count(1) from clustergroupby where ds='101' group by key,3 limit 10; +select key, count(1) from clustergroupby where ds='101' group by key,3 limit 10; + +--subquery-- +explain +select key, count(1) from (select value as key, key as value from clustergroupby where ds='101')subq group by key limit 10; +select key, count(1) from (select value as key, key as value from clustergroupby where ds='101')subq group by key limit 10; + +explain +select key, count(1) from clustergroupby group by key; +select key, count(1) from clustergroupby group by key; + +explain +select key, count(1) from clustergroupby group by key, 3; + +-- number of buckets cannot be changed, so drop the table +drop table clustergroupby; +create table clustergroupby(key string, value string) partitioned by(ds string); + +--sort columns-- +alter table clustergroupby clustered by (value) sorted by (key, value) into 1 buckets; +describe extended clustergroupby; +insert overwrite table clustergroupby partition (ds='102') select key, value from src distribute by value sort by key, value; + +explain +select key, count(1) from clustergroupby where ds='102' group by key limit 10; +select key, count(1) from clustergroupby where ds='102' group by key limit 10; +explain +select value, count(1) from clustergroupby where ds='102' group by value limit 10; +select value, count(1) from clustergroupby where ds='102' group by value limit 10; +explain +select key, count(1) from clustergroupby where ds='102' group by key, value limit 10; +select key, count(1) from clustergroupby where ds='102' group by key, value limit 10; + +-- number of buckets cannot be changed, so drop the table +drop table clustergroupby; +create table clustergroupby(key string, value string) partitioned by(ds string); + +alter table clustergroupby clustered by (value, key) sorted by (key) into 1 buckets; +describe extended clustergroupby; +insert overwrite table clustergroupby partition (ds='103') select key, value from src distribute by value, key sort by key; +explain +select key, count(1) from clustergroupby where ds='103' group by key limit 10; +select key, count(1) from clustergroupby where ds='103' group by key limit 10; +explain +select key, count(1) from clustergroupby where ds='103' group by value, key limit 10; +select key, count(1) from clustergroupby where ds='103' group by value, key limit 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_1.q b/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_1.q new file mode 100644 index 0000000000000..33dd5d5cd2e0a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_1.q @@ -0,0 +1,26 @@ +drop table table1; +drop table table2; + +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; + +create table table1(key string, value string) clustered by (key, value) +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; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +-- The tables are bucketed in same columns in different order, +-- but sorted in different column orders +-- Neither bucketed map-join, nor sort-merge join should be performed + +explain extended +select /*+ mapjoin(b) */ count(*) from table1 a join table2 b on a.key=b.key and a.value=b.value; + +select /*+ mapjoin(b) */ count(*) from table1 a join table2 b on a.key=b.key and a.value=b.value; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_2.q b/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_2.q new file mode 100644 index 0000000000000..d1097e70a9880 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_2.q @@ -0,0 +1,26 @@ +drop table table1; +drop table table2; + +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; + +create table table1(key string, value string) clustered by (key, value) +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; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +-- The tables are bucketed in same columns in different order, +-- but sorted in different column orders +-- Neither bucketed map-join, nor sort-merge join should be performed + +explain extended +select /*+ mapjoin(b) */ count(*) from table1 a join table2 b on a.key=b.key and a.value=b.value; + +select /*+ mapjoin(b) */ count(*) from table1 a join table2 b on a.key=b.key and a.value=b.value; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers.q b/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers.q new file mode 100644 index 0000000000000..37ae6cc7adeae --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers.q @@ -0,0 +1,16 @@ +set hive.enforce.bucketing = true; +set hive.exec.mode.local.auto=false; +set mapred.reduce.tasks = 10; + +-- This test sets number of mapred tasks to 10 for a database with 50 buckets, +-- and uses a post-hook to confirm that 10 tasks were created + +CREATE TABLE bucket_nr(key int, value string) CLUSTERED BY (key) INTO 50 BUCKETS; +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyNumReducersHook; +set VerifyNumReducersHook.num.reducers=10; + +insert overwrite table bucket_nr +select * from src; + +set hive.exec.post.hooks=; +drop table bucket_nr; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers2.q b/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers2.q new file mode 100644 index 0000000000000..8c64d60c5ac42 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers2.q @@ -0,0 +1,13 @@ +set hive.enforce.bucketing = true; +set hive.exec.mode.local.auto=false; +set hive.exec.reducers.max = 2; + +-- This test sets the maximum number of reduce tasks to 2 for overwriting a +-- table with 3 buckets, and uses a post-hook to confirm that 1 reducer was used + +CREATE TABLE test_table(key int, value string) CLUSTERED BY (key) INTO 3 BUCKETS; +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyNumReducersHook; +set VerifyNumReducersHook.num.reducers=1; + +insert overwrite table test_table +select * from src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_1.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_1.q new file mode 100644 index 0000000000000..5b013995d5ba4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_1.q @@ -0,0 +1,23 @@ +-- 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'); + +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'); + +set hive.optimize.bucketmapjoin = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_2.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_2.q new file mode 100644 index 0000000000000..f952f2ee6d445 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_2.q @@ -0,0 +1,21 @@ +-- 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'); + +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-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; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_3.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_3.q new file mode 100644 index 0000000000000..461fbb196e758 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_3.q @@ -0,0 +1,21 @@ +-- 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-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'); + +set hive.optimize.bucketmapjoin = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_4.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_4.q new file mode 100644 index 0000000000000..366da4473b36d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_4.q @@ -0,0 +1,23 @@ +-- 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-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'); + +set hive.optimize.bucketmapjoin = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_5.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_5.q new file mode 100644 index 0000000000000..411fed392a45e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_5.q @@ -0,0 +1,18 @@ +-- 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; + +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; + +set hive.optimize.bucketmapjoin = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_6.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_6.q new file mode 100644 index 0000000000000..204d1e9010d5d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_6.q @@ -0,0 +1,21 @@ +-- 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; + +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-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; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_7.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_7.q new file mode 100644 index 0000000000000..b0bca460f303f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_7.q @@ -0,0 +1,26 @@ +-- 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-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-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; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_8.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_8.q new file mode 100644 index 0000000000000..9533c5512f3dd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_8.q @@ -0,0 +1,26 @@ +-- 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-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-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; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q new file mode 100644 index 0000000000000..4c7f0c9595488 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q @@ -0,0 +1,33 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +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; + +CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE; + +EXPLAIN INSERT OVERWRITE TABLE T2 SELECT * FROM ( +SELECT tmp1.name as name FROM ( + SELECT name, 'MMM' AS n FROM T1) tmp1 + JOIN (SELECT 'MMM' AS n FROM T1) tmp2 + JOIN (SELECT 'MMM' AS n FROM T1) tmp3 + ON tmp1.n = tmp2.n AND tmp1.n = tmp3.n) ttt LIMIT 5000000; + + +INSERT OVERWRITE TABLE T2 SELECT * FROM ( +SELECT tmp1.name as name FROM ( + SELECT name, 'MMM' AS n FROM T1) tmp1 + JOIN (SELECT 'MMM' AS n FROM T1) tmp2 + JOIN (SELECT 'MMM' AS n FROM T1) tmp3 + ON tmp1.n = tmp2.n AND tmp1.n = tmp3.n) ttt LIMIT 5000000; + +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; + +EXPLAIN SELECT COUNT(1) FROM T3; +SELECT COUNT(1) FROM T3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q new file mode 100644 index 0000000000000..9cdfe8e6e9f6e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q @@ -0,0 +1,23 @@ +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'); + +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'); + +set hive.optimize.bucketmapjoin = true; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; + +set hive.input.format = org.apache.hadoop.hive.ql.io.HiveInputFormat; +select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin1.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin1.q new file mode 100644 index 0000000000000..2bd8e1e2e5d98 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin1.q @@ -0,0 +1,107 @@ +CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 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; + +CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; + +set hive.optimize.bucketmapjoin = true; + +-- empty partitions (HIVE-3205) +explain extended +select /*+mapjoin(b)*/ 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"; + +select /*+mapjoin(b)*/ 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"; + +explain extended +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"; + +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_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'); + +create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); +create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); + +set hive.optimize.bucketmapjoin = true; +create table bucketmapjoin_tmp_result (key string , value1 string, value2 string); + +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; + +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; + + +set hive.optimize.bucketmapjoin = true; +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; + + +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin10.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin10.q new file mode 100644 index 0000000000000..cf4222bdd781e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin10.q @@ -0,0 +1,36 @@ +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'); + +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'); + +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'); + +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'); + +ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 3 BUCKETS; + +set hive.optimize.bucketmapjoin=true; + +-- The table bucketing metadata matches but the partition metadata does not, bucket map join should not be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part IS NOT NULL AND b.part IS NOT NULL; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part IS NOT NULL AND b.part IS NOT NULL; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin11.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin11.q new file mode 100644 index 0000000000000..e10ab522a296d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin11.q @@ -0,0 +1,47 @@ +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'); + +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'); + +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'); + +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'); + + +set hive.optimize.bucketmapjoin=true; + +-- The table and partition bucketing metadata doesn't match but the bucket numbers of all partitions is +-- a power of 2 and the bucketing columns match so bucket map join should be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part IS NOT NULL AND b.part IS NOT NULL; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part IS NOT NULL AND b.part IS NOT NULL; + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = b.part AND a.part IS NOT NULL AND b.part IS NOT NULL; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = b.part AND a.part IS NOT NULL AND b.part IS NOT NULL; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin12.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin12.q new file mode 100644 index 0000000000000..8139000f4a733 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin12.q @@ -0,0 +1,44 @@ +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'); + +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'); + +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'); + +ALTER TABLE srcbucket_mapjoin_part_3 CLUSTERED BY (key) INTO 2 BUCKETS; + +set hive.optimize.bucketmapjoin=true; + +-- The partition bucketing metadata match but one table is not bucketed, bucket map join should still be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' and b.part = '1'; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' and b.part = '1'; + +-- The table bucketing metadata match but one partition is not bucketed, bucket map join should not be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_3 b +ON a.key = b.key AND a.part = '1' and b.part = '1'; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_3 b +ON a.key = b.key AND a.part = '1' and b.part = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin13.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin13.q new file mode 100644 index 0000000000000..f01c43e5694f9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin13.q @@ -0,0 +1,74 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max=1; + +CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (value) INTO 2 BUCKETS; + +-- part=1 partition for srcbucket_mapjoin_part_1 is bucketed by 'value' +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part='1') +SELECT * FROM src; + +ALTER TABLE srcbucket_mapjoin_part_1 CLUSTERED BY (key) INTO 2 BUCKETS; + +-- part=2 partition for srcbucket_mapjoin_part_1 is bucketed by 'key' +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part='2') +SELECT * FROM src; + +CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (key) INTO 2 BUCKETS; + +-- part=1 partition for srcbucket_mapjoin_part_2 is bucketed by 'key' +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_2 PARTITION (part='1') +SELECT * FROM src; + +set hive.optimize.bucketmapjoin=true; + +-- part=1 partition for srcbucket_mapjoin_part_1 is bucketed by 'value' +-- and it is also being joined. So, bucketed map-join cannot be performed +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key; + +-- part=2 partition for srcbucket_mapjoin_part_1 is bucketed by 'key' +-- and it is being joined. So, bucketed map-join can be performed +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key and a.part = '2'; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key and a.part = '2'; + +ALTER TABLE srcbucket_mapjoin_part_1 drop partition (part = '1'); + +-- part=2 partition for srcbucket_mapjoin_part_1 is bucketed by 'key' +-- and it is being joined. So, bucketed map-join can be performed +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key; + +ALTER TABLE srcbucket_mapjoin_part_1 CLUSTERED BY (value) INTO 2 BUCKETS; + +-- part=2 partition for srcbucket_mapjoin_part_1 is bucketed by 'key' +-- and it is being joined. So, bucketed map-join can be performed +-- The fact that the table is being bucketed by 'value' does not matter +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin2.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin2.q new file mode 100644 index 0000000000000..fdbadfcb04c9c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin2.q @@ -0,0 +1,112 @@ +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'); + +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'); + +create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); +create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); + +set hive.optimize.bucketmapjoin = true; +create table bucketmapjoin_tmp_result (key string , value1 string, value2 string); + +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b +on a.key=b.key and b.ds="2008-04-08"; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b +on a.key=b.key and b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b +on a.key=b.key and b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; + +set hive.optimize.bucketmapjoin = true; +explain extended +insert overwrite table bucketmapjoin_tmp_result +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 and b.ds="2008-04-08"; + +insert overwrite table bucketmapjoin_tmp_result +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 and b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +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 and b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +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'); + +set hive.optimize.bucketmapjoin = true; + +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b +on a.key=b.key; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b +on a.key=b.key; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b +on a.key=b.key; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin3.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin3.q new file mode 100644 index 0000000000000..8fda80231522e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin3.q @@ -0,0 +1,78 @@ +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; + +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'); + +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'); + +create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); +create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); + +set hive.optimize.bucketmapjoin = true; +create table bucketmapjoin_tmp_result (key string , value1 string, value2 string); + +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part_2 a join srcbucket_mapjoin_part b +on a.key=b.key and b.ds="2008-04-08" and a.ds="2008-04-08"; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part_2 a join srcbucket_mapjoin_part b +on a.key=b.key and b.ds="2008-04-08" and a.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part_2 a join srcbucket_mapjoin_part b +on a.key=b.key and b.ds="2008-04-08" and a.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; + +set hive.optimize.bucketmapjoin = true; +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part_2 a join srcbucket_mapjoin_part b +on a.key=b.key and b.ds="2008-04-08" and a.ds="2008-04-08"; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part_2 a join srcbucket_mapjoin_part b +on a.key=b.key and b.ds="2008-04-08" and a.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin_part_2 a join srcbucket_mapjoin_part b +on a.key=b.key and b.ds="2008-04-08" and a.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin4.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin4.q new file mode 100644 index 0000000000000..c1a8f2ab7bfa0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin4.q @@ -0,0 +1,80 @@ +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; + +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'); + +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'); + +create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); +create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); + +set hive.optimize.bucketmapjoin = true; +create table bucketmapjoin_tmp_result (key string , value1 string, value2 string); + +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin b +on a.key=b.key; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin b +on a.key=b.key; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin b +on a.key=b.key; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; + + +set hive.optimize.bucketmapjoin = true; +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin b +on a.key=b.key; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin b +on a.key=b.key; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin b +on a.key=b.key; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin5.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin5.q new file mode 100644 index 0000000000000..2df49b64f3227 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin5.q @@ -0,0 +1,85 @@ +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; + +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'); + +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'); + +create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); +create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); + +set hive.optimize.bucketmapjoin = true; +create table bucketmapjoin_tmp_result (key string , value1 string, value2 string); + +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; + + +set hive.optimize.bucketmapjoin = true; +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part_2 b +on a.key=b.key; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part_2 b +on a.key=b.key; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part_2 b +on a.key=b.key; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin6.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin6.q new file mode 100644 index 0000000000000..2998d7ccf1961 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin6.q @@ -0,0 +1,25 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +create table tmp1 (a string, b string) clustered by (a) sorted by (a) into 10 buckets; + +create table tmp2 (a string, b string) clustered by (a) sorted by (a) into 10 buckets; + + +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max=1; + + +insert overwrite table tmp1 select * from src where key < 50; +insert overwrite table tmp2 select * from src where key < 50; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.merge.mapfiles=false; +create table tmp3 (a string, b string, c string) clustered by (a) sorted by (a) into 10 buckets; + + +insert overwrite table tmp3 + select /*+ MAPJOIN(l) */ i.a, i.b, l.b + from tmp1 i join tmp2 l ON i.a = l.a; + +select * from tmp3 order by a, b, c; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin7.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin7.q new file mode 100644 index 0000000000000..3a96c642b3e4c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin7.q @@ -0,0 +1,24 @@ +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'); + +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'); + +set hive.optimize.bucketmapjoin=true; + +-- Tests that bucket map join works with a table with more than one level of partitioning + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' LIMIT 1; + +SELECT /*+ MAPJOIN(b) */ a.key, b.value +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.ds = '2008-04-08' AND b.ds = '2008-04-08' LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin8.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin8.q new file mode 100644 index 0000000000000..5e8daa5f8f125 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin8.q @@ -0,0 +1,39 @@ +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'); + +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'); + +ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 3 BUCKETS; + +set hive.optimize.bucketmapjoin=true; + +-- The partition bucketing metadata match but the tables have different numbers of buckets, bucket map join should still be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' and b.part = '1'; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' and b.part = '1'; + +ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (value) INTO 2 BUCKETS; + +-- The partition bucketing metadata match but the tables are bucketed on different columns, bucket map join should still be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' and b.part = '1'; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' and b.part = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin9.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin9.q new file mode 100644 index 0000000000000..86344a53f5a95 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin9.q @@ -0,0 +1,45 @@ +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'); + +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'); + +ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS; + +set hive.optimize.bucketmapjoin=true; + +-- The table bucketing metadata matches but the partitions have different numbers of buckets, bucket map join should not be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' and b.part = '1'; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +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'); + +ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS; + +-- The table bucketing metadata matches but the partitions are bucketed on different columns, bucket map join should not be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' AND b.part = '1'; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' AND b.part = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q new file mode 100644 index 0000000000000..d7634333e2fee --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q @@ -0,0 +1,27 @@ + + + + +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; + +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'); + + + +set hive.optimize.bucketmapjoin = true; +create table bucketmapjoin_tmp_result (key string , value1 string, value2 string); + +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q new file mode 100644 index 0000000000000..901f0567590ec --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q @@ -0,0 +1,18 @@ +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; + +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'); + +set hive.optimize.bucketmapjoin = true; +create table bucketmapjoin_tmp_result (key string , value1 string, value2 string); + +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part_2 b +on a.key=b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q new file mode 100644 index 0000000000000..d66e1238bb79b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q @@ -0,0 +1,39 @@ +drop table test1; +drop table test2; +drop table test3; +drop table test4; + +create table test1 (key string, value string) clustered by (key) sorted by (key) into 3 buckets; +create table test2 (key string, value string) clustered by (value) sorted by (value) into 3 buckets; +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 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 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 +explain extended select /* + MAPJOIN(R) */ * from test1 L join test1 R on L.key=R.key AND L.value=R.value; +explain extended select /* + MAPJOIN(R) */ * from test2 L join test2 R on L.key=R.key AND L.value=R.value; + +-- should not apply bucket mapjoin +explain extended select /* + MAPJOIN(R) */ * from test1 L join test1 R on L.key+L.key=R.key; +explain extended select /* + MAPJOIN(R) */ * from test1 L join test2 R on L.key=R.key AND L.value=R.value; +explain extended select /* + MAPJOIN(R) */ * from test1 L join test3 R on L.key=R.key AND L.value=R.value; +explain extended select /* + MAPJOIN(R) */ * from test1 L join test4 R on L.key=R.key AND L.value=R.value; +explain extended select /* + MAPJOIN(R) */ * from test2 L join test3 R on L.key=R.key AND L.value=R.value; +explain extended select /* + MAPJOIN(R) */ * from test2 L join test4 R on L.key=R.key AND L.value=R.value; +explain extended select /* + MAPJOIN(R) */ * from test3 L join test4 R on L.key=R.key AND L.value=R.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q new file mode 100644 index 0000000000000..8cc308ffb1156 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q @@ -0,0 +1,78 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT *; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT x.key, x.value from +( +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' +)x; + +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT x.key, x.value from +( +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' +)x; + +select count(*) from test_table2 where ds = '1'; +select count(*) from test_table2 tablesample (bucket 1 out of 2) s where ds = '1'; +select count(*) from test_table2 tablesample (bucket 2 out of 2) s where ds = '1'; + +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT * from +( +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' +)x; + +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT * from +( +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' +)x; + +select count(*) from test_table2 where ds = '1'; +select count(*) from test_table2 tablesample (bucket 1 out of 2) s where ds = '1'; +select count(*) from test_table2 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- it should be a map-only job +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT x.key, concat(x.value, x.value) from +( +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' +)x; + +-- it should be a map-reduce job +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT x.key+x.key, x.value from +( +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' +)x; + +-- it should be a map-only job +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT x.k1, concat(x.v1, x.v1) from +( +SELECT a.key as k1, a.value as v1 FROM test_table1 a WHERE a.ds = '1' +)x; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q new file mode 100644 index 0000000000000..9ecd2c48ba764 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q @@ -0,0 +1,143 @@ +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.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.auto.convert.sortmerge.join.bigtable.selection.policy=org.apache.hadoop.hive.ql.optimizer.LeftmostBigTableSelectorForAutoSMJ; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table3 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * where key < 10; + +FROM src +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT * where key < 100; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '2') SELECT * where key < 10; + +FROM src +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '2') SELECT * where key < 100; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- Since more than one partition of 'a' (the big table) is being selected, +-- it should be a map-reduce job +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds is not null and b.ds = '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds is not null and b.ds = '1'; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- Since a single partition of the big table ('a') is being selected, it should be a map-only +-- job even though multiple partitions of 'b' are being selected +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds is not null; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds is not null; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- This should be a map-only job +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM +(select key, value from test_table1 where ds = '1') a +JOIN +(select key, value from test_table2 where ds = '1') b +ON a.key = b.key; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM +(select key, value from test_table1 where ds = '1') a +JOIN +(select key, value from test_table2 where ds = '1') b +ON a.key = b.key; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- This should be a map-only job +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.v1, b.v2) +FROM +(select key, concat(value, value) as v1 from test_table1 where ds = '1') a +JOIN +(select key, concat(value, value) as v2 from test_table2 where ds = '1') b +ON a.key = b.key; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.v1, b.v2) +FROM +(select key, concat(value, value) as v1 from test_table1 where ds = '1') a +JOIN +(select key, concat(value, value) as v2 from test_table2 where ds = '1') b +ON a.key = b.key; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- This should be a map-reduce job +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key+a.key, concat(a.value, b.value) +FROM +(select key, value from test_table1 where ds = '1') a +JOIN +(select key, value from test_table2 where ds = '1') b +ON a.key = b.key; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key+a.key, concat(a.value, b.value) +FROM +(select key, value from test_table1 where ds = '1') a +JOIN +(select key, value from test_table2 where ds = '1') b +ON a.key = b.key; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q new file mode 100644 index 0000000000000..91e97de62c82f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q @@ -0,0 +1,52 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (value STRING, key INT) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT *; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- The bucketing positions dont match - although the actual bucketing do. +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT x.value, x.key from +(SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1')x; + +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT x.value, x.key from +(SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1')x; + +select count(*) from test_table2 where ds = '1'; +select count(*) from test_table2 tablesample (bucket 1 out of 2) s where ds = '1'; +select count(*) from test_table2 tablesample (bucket 2 out of 2) s where ds = '1'; + +CREATE TABLE test_table3 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- The bucketing positions dont match - this should be a map-reduce operation +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT x.key, x.value from +(SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1')x; + +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT x.key, x.value from +(SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1')x; + +select count(*) from test_table2 where ds = '1'; +select count(*) from test_table2 tablesample (bucket 1 out of 2) s where ds = '1'; +select count(*) from test_table2 tablesample (bucket 2 out of 2) s where ds = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q new file mode 100644 index 0000000000000..623b22bfe494c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q @@ -0,0 +1,65 @@ +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.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.auto.convert.sortmerge.join.bigtable.selection.policy=org.apache.hadoop.hive.ql.optimizer.LeftmostBigTableSelectorForAutoSMJ; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table3 (key INT, key2 INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key2) SORTED BY (key2) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * where key < 10; + +FROM src +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT * where key < 100; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation, since the insert is happening on the bucketing position +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +DROP TABLE test_table3; + +CREATE TABLE test_table3 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-reduce operation, since the insert is happening on a non-bucketing position +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, a.value +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, a.value +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +DROP TABLE test_table3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q new file mode 100644 index 0000000000000..205a45064b4e3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q @@ -0,0 +1,63 @@ +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.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.auto.convert.sortmerge.join.bigtable.selection.policy=org.apache.hadoop.hive.ql.optimizer.LeftmostBigTableSelectorForAutoSMJ; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table3 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key desc) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * where key < 10; + +FROM src +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT * where key < 100; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-reduce operation, since the sort-order does not match +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- This should be a map-reduce job since the sort order does not match +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM +(select key, value from test_table1 where ds = '1') a +JOIN +(select key, value from test_table2 where ds = '1') b +ON a.key = b.key; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM +(select key, value from test_table1 where ds = '1') a +JOIN +(select key, value from test_table2 where ds = '1') b +ON a.key = b.key; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q new file mode 100644 index 0000000000000..a4e84f85e1664 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q @@ -0,0 +1,156 @@ +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.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.auto.convert.sortmerge.join.bigtable.selection.policy=org.apache.hadoop.hive.ql.optimizer.LeftmostBigTableSelectorForAutoSMJ; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, key2 INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key, key2) SORTED BY (key ASC, key2 DESC) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, key2 INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key, key2) SORTED BY (key ASC, key2 DESC) INTO 2 BUCKETS; +CREATE TABLE test_table3 (key INT, key2 INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key, key2) SORTED BY (key ASC, key2 DESC) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT key, key+1, value where key < 10; + +FROM src +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT key, key+1, value where key < 100; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation, since the sort-order matches +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, a.key2, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, a.key2, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1'; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation, since the sort-order matches +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT subq1.key, subq1.key2, subq1.value from +( +SELECT a.key, a.key2, concat(a.value, b.value) as value +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' +)subq1; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT subq1.key, subq1.key2, subq1.value from +( +SELECT a.key, a.key2, concat(a.value, b.value) as value +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' +)subq1; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-reduce operation +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key2, a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1'; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-reduce operation +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT subq1.key2, subq1.key, subq1.value from +( +SELECT a.key, a.key2, concat(a.value, b.value) as value +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' +)subq1; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT subq2.key, subq2.key2, subq2.value from +( +SELECT subq1.key2, subq1.key, subq1.value from +( +SELECT a.key, a.key2, concat(a.value, b.value) as value +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' +)subq1 +)subq2; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT subq2.key, subq2.key2, subq2.value from +( +SELECT subq1.key2, subq1.key, subq1.value from +( +SELECT a.key, a.key2, concat(a.value, b.value) as value +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' +)subq1 +)subq2; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT subq2.k2, subq2.k1, subq2.value from +( +SELECT subq1.key2 as k1, subq1.key as k2, subq1.value from +( +SELECT a.key, a.key2, concat(a.value, b.value) as value +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' +)subq1 +)subq2; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT subq2.k2, subq2.k1, subq2.value from +( +SELECT subq1.key2 as k1, subq1.key as k2, subq1.value from +( +SELECT a.key, a.key2, concat(a.value, b.value) as value +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' +)subq1 +)subq2; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +CREATE TABLE test_table4 (key INT, key2 INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key, key2) SORTED BY (key DESC, key2 DESC) INTO 2 BUCKETS; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-reduce operation +EXPLAIN +INSERT OVERWRITE TABLE test_table4 PARTITION (ds = '1') +SELECT subq2.k2, subq2.k1, subq2.value from +( +SELECT subq1.key2 as k1, subq1.key as k2, subq1.value from +( +SELECT a.key, a.key2, concat(a.value, b.value) as value +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' +)subq1 +)subq2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q new file mode 100644 index 0000000000000..f597884bb7645 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q @@ -0,0 +1,88 @@ +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.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.auto.convert.sortmerge.join.bigtable.selection.policy=org.apache.hadoop.hive.ql.optimizer.LeftmostBigTableSelectorForAutoSMJ; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table3 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * where key < 10; + +FROM src +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT * where key < 100; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1' +and (a.key = 0 or a.key = 5); + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1' +and (a.key = 0 or a.key = 5); + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- This should be a map-only job +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM +(select key, value from test_table1 where ds = '1' and (key = 0 or key = 5)) a +JOIN +(select key, value from test_table2 where ds = '1' and (key = 0 or key = 5)) b +ON a.key = b.key; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM +(select key, value from test_table1 where ds = '1' and (key = 0 or key = 5)) a +JOIN +(select key, value from test_table2 where ds = '1' and (key = 0 or key = 5)) b +ON a.key = b.key; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- This should be a map-only job +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM +(select key, value from test_table1 where ds = '1' and key < 8) a +JOIN +(select key, value from test_table2 where ds = '1' and key < 8) b +ON a.key = b.key +WHERE a.key = 0 or a.key = 5; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, concat(a.value, b.value) +FROM +(select key, value from test_table1 where ds = '1' and key < 8) a +JOIN +(select key, value from test_table2 where ds = '1' and key < 8) b +ON a.key = b.key +WHERE a.key = 0 or a.key = 5; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q new file mode 100644 index 0000000000000..95a9a6481a4a6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q @@ -0,0 +1,58 @@ +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.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.auto.convert.sortmerge.join.bigtable.selection.policy=org.apache.hadoop.hive.ql.optimizer.LeftmostBigTableSelectorForAutoSMJ; + +set hive.auto.convert.sortmerge.join.to.mapjoin=true; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table3 (key INT, key2 INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * where key < 10; + +FROM src +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT * where key < 100; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, b.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.key, b.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT b.key, a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT b.key, a.key, concat(a.value, b.value) +FROM test_table1 a JOIN test_table2 b +ON a.key = b.key WHERE a.ds = '1' and b.ds = '1'; + +select * from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select * from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/case_sensitivity.q b/src/test/hive/ql/src/test/queries/clientpositive/case_sensitivity.q new file mode 100644 index 0000000000000..a4b73f0a87e95 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/case_sensitivity.q @@ -0,0 +1,10 @@ +CREATE TABLE DEST1(Key INT, VALUE STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM SRC_THRIFT +INSERT OVERWRITE TABLE dest1 SELECT src_Thrift.LINT[1], src_thrift.lintstring[0].MYSTRING where src_thrift.liNT[0] > 0; + +FROM SRC_THRIFT +INSERT OVERWRITE TABLE dest1 SELECT src_Thrift.LINT[1], src_thrift.lintstring[0].MYSTRING where src_thrift.liNT[0] > 0; + +SELECT DEST1.* FROM Dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/cast1.q b/src/test/hive/ql/src/test/queries/clientpositive/cast1.q new file mode 100644 index 0000000000000..7f5f8a58deecf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/cast1.q @@ -0,0 +1,9 @@ +CREATE TABLE dest1(c1 INT, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 INT, c6 STRING, c7 INT) STORED AS TEXTFILE; + +EXPLAIN +FROM src INSERT OVERWRITE TABLE dest1 SELECT 3 + 2, 3.0 + 2, 3 + 2.0, 3.0 + 2.0, 3 + CAST(2.0 AS INT) + CAST(CAST(0 AS SMALLINT) AS INT), CAST(1 AS BOOLEAN), CAST(TRUE AS INT) WHERE src.key = 86; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT 3 + 2, 3.0 + 2, 3 + 2.0, 3.0 + 2.0, 3 + CAST(2.0 AS INT) + CAST(CAST(0 AS SMALLINT) AS INT), CAST(1 AS BOOLEAN), CAST(TRUE AS INT) WHERE src.key = 86; + +select dest1.* FROM dest1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/cast_to_int.q b/src/test/hive/ql/src/test/queries/clientpositive/cast_to_int.q new file mode 100644 index 0000000000000..729ffdc8687b5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/cast_to_int.q @@ -0,0 +1,30 @@ +-- cast string floats to integer types +select + cast('1' as float), + cast('1.4' as float), + cast('1.6' as float), + cast('1' as int), + cast('1.4' as int), + cast('1.6' as int), + cast('1' as tinyint), + cast('1.4' as tinyint), + cast('1.6' as tinyint), + cast('1' as smallint), + cast('1.4' as smallint), + cast('1.6' as smallint), + cast('1' as bigint), + cast('1.4' as bigint), + cast('1.6' as bigint), + cast (cast('1' as float) as int), + cast(cast ('1.4' as float) as int), + cast(cast ('1.6' as float) as int), + cast('+1e5' as int), + cast('2147483647' as int), + cast('-2147483648' as int), + cast('32767' as smallint), + cast('-32768' as smallint), + cast('-128' as tinyint), + cast('127' as tinyint), + cast('1.0a' as int), + cast('-1.-1' as int) +from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/cluster.q b/src/test/hive/ql/src/test/queries/clientpositive/cluster.q new file mode 100644 index 0000000000000..24819707181a8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/cluster.q @@ -0,0 +1,65 @@ +EXPLAIN +SELECT * FROM SRC x where x.key = 10 CLUSTER BY x.key; +SELECT * FROM SRC x where x.key = 10 CLUSTER BY x.key; + +EXPLAIN +SELECT * FROM SRC x where x.key = 20 CLUSTER BY key ; +SELECT * FROM SRC x where x.key = 20 CLUSTER BY key ; + +EXPLAIN +SELECT x.* FROM SRC x where x.key = 20 CLUSTER BY key; +SELECT x.* FROM SRC x where x.key = 20 CLUSTER BY key; + +EXPLAIN +SELECT x.* FROM SRC x where x.key = 20 CLUSTER BY x.key; +SELECT x.* FROM SRC x where x.key = 20 CLUSTER BY x.key; + +EXPLAIN +SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY key ; +SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY key ; + +EXPLAIN +SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY x.key; +SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY x.key; + +EXPLAIN +SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY v1; +SELECT x.key, x.value as v1 FROM SRC x where x.key = 20 CLUSTER BY v1; + +EXPLAIN +SELECT y.* from (SELECT x.* FROM SRC x CLUSTER BY x.key) y where y.key = 20; +SELECT y.* from (SELECT x.* FROM SRC x CLUSTER BY x.key) y where y.key = 20; + + +EXPLAIN +SELECT x.key, x.value as v1, y.key FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1;; +SELECT x.key, x.value as v1, y.key FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1; + +EXPLAIN +SELECT x.key, x.value as v1, y.* FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1; +SELECT x.key, x.value as v1, y.* FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1; + +EXPLAIN +SELECT x.key, x.value as v1, y.* FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY x.key ; +SELECT x.key, x.value as v1, y.* FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY x.key ; + +EXPLAIN +SELECT x.key, x.value as v1, y.key as yk FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY key ; +SELECT x.key, x.value as v1, y.key as yk FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY key ; + +EXPLAIN +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 +CLUSTER BY unioninput.key; + +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 +CLUSTER BY unioninput.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/column_access_stats.q b/src/test/hive/ql/src/test/queries/clientpositive/column_access_stats.q new file mode 100644 index 0000000000000..3c8a309991d75 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/column_access_stats.q @@ -0,0 +1,162 @@ +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.CheckColumnAccessHook; +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; + +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) PARTITIONED BY (p STRING); + +-- Simple select queries +SELECT key FROM T1 ORDER BY key; +SELECT key, val FROM T1 ORDER BY key, val; +SELECT 1 FROM T1; +SELECT key, val from T4 where p=1; +SELECT val FROM T4 where p=1; +SELECT p, val FROM T4 where p=1; + +-- More complicated select queries +EXPLAIN SELECT key FROM (SELECT key, val FROM T1) subq1 ORDER BY key; +SELECT key FROM (SELECT key, val FROM T1) subq1 ORDER BY key; +EXPLAIN SELECT k FROM (SELECT key as k, val as v FROM T1) subq1 ORDER BY k; +SELECT k FROM (SELECT key as k, val as v FROM T1) subq1 ORDER BY k; +SELECT key + 1 as k FROM T1 ORDER BY k; +SELECT key + val as k FROM T1 ORDER BY k; + +-- Work with union +EXPLAIN +SELECT * FROM ( +SELECT key as c FROM T1 + UNION ALL +SELECT val as c FROM T1 +) subq1 ORDER BY c; + +SELECT * FROM ( +SELECT key as c FROM T1 + UNION ALL +SELECT val as c FROM T1 +) subq1 ORDER BY c; + +EXPLAIN +SELECT * FROM ( +SELECT key as c FROM T1 + UNION ALL +SELECT key as c FROM T1 +) subq1 ORDER BY c; + +SELECT * FROM ( +SELECT key as c FROM T1 + UNION ALL +SELECT key as c FROM T1 +) subq1 ORDER BY c; + +-- Work with insert overwrite +FROM T1 +INSERT OVERWRITE TABLE T2 SELECT key, count(1) GROUP BY key +INSERT OVERWRITE TABLE T3 SELECT key, sum(val) GROUP BY key; + +-- Simple joins +SELECT * +FROM T1 JOIN T2 +ON T1.key = T2.key +ORDER BY T1.key, T1.val, T2.key, T2.val; + +EXPLAIN +SELECT T1.key +FROM T1 JOIN T2 +ON T1.key = T2.key +ORDER BY T1.key; + +SELECT T1.key +FROM T1 JOIN T2 +ON T1.key = T2.key +ORDER BY T1.key; + +SELECT * +FROM T1 JOIN T2 +ON T1.key = T2.key AND T1.val = T2.val +ORDER BY T1.key, T1.val; + +-- Map join +SELECT /*+ MAPJOIN(a) */ * +FROM T1 a JOIN T2 b +ON a.key = b.key +ORDER BY a.key, a.val, b.key, b.val; + +-- More joins +EXPLAIN +SELECT * +FROM T1 JOIN T2 +ON T1.key = T2.key AND T1.val = 3 and T2.val = 3 +ORDER BY T1.key, T1.val; + +SELECT * +FROM T1 JOIN T2 +ON T1.key = T2.key AND T1.val = 3 and T2.val = 3 +ORDER BY T1.key, T1.val; + +EXPLAIN +SELECT subq1.val +FROM +( + SELECT val FROM T1 WHERE key = 5 +) subq1 +JOIN +( + SELECT val FROM T2 WHERE key = 6 +) subq2 +ON subq1.val = subq2.val +ORDER BY subq1.val; + +SELECT subq1.val +FROM +( + SELECT val FROM T1 WHERE key = 5 +) subq1 +JOIN +( + SELECT val FROM T2 WHERE key = 6 +) subq2 +ON subq1.val = subq2.val +ORDER BY subq1.val; + +-- Join followed by join +EXPLAIN +SELECT * +FROM +( + SELECT subq1.key as key + FROM + ( + SELECT key, val FROM T1 + ) subq1 + JOIN + ( + SELECT key, 'teststring' as val FROM T2 + ) subq2 + ON subq1.key = subq2.key +) T4 +JOIN T3 +ON T3.key = T4.key +ORDER BY T3.key, T4.key; + +SELECT * +FROM +( + SELECT subq1.key as key + FROM + ( + SELECT key, val FROM T1 + ) subq1 + JOIN + ( + SELECT key, 'teststring' as val FROM T2 + ) subq2 + ON subq1.key = subq2.key +) T4 +JOIN T3 +ON T3.key = T4.key +ORDER BY T3.key, T4.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q b/src/test/hive/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q new file mode 100644 index 0000000000000..4c3b44ae14af6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q @@ -0,0 +1,24 @@ +CREATE TABLE columnarserde_create_shortcut(a array, b array, c map, d int, e string) STORED AS RCFILE; + +EXPLAIN +FROM src_thrift +INSERT OVERWRITE TABLE columnarserde_create_shortcut SELECT src_thrift.lint, src_thrift.lstring, src_thrift.mstringstring, src_thrift.aint, src_thrift.astring DISTRIBUTE BY 1; + +FROM src_thrift +INSERT OVERWRITE TABLE columnarserde_create_shortcut SELECT src_thrift.lint, src_thrift.lstring, src_thrift.mstringstring, src_thrift.aint, src_thrift.astring DISTRIBUTE BY 1; + +SELECT columnarserde_create_shortcut.* FROM columnarserde_create_shortcut CLUSTER BY 1; + +SELECT columnarserde_create_shortcut.a[0], columnarserde_create_shortcut.b[0], columnarserde_create_shortcut.c['key2'], columnarserde_create_shortcut.d, columnarserde_create_shortcut.e FROM columnarserde_create_shortcut CLUSTER BY 1; + +CREATE table columnShortcutTable (key STRING, value STRING) STORED AS RCFILE; + +FROM src +INSERT OVERWRITE TABLE columnShortcutTable SELECT src.key, src.value LIMIT 10; +describe columnShortcutTable; +SELECT columnShortcutTable.* FROM columnShortcutTable ORDER BY key ASC, value ASC; + +ALTER TABLE columnShortcutTable ADD COLUMNS (c string); +SELECT columnShortcutTable.* FROM columnShortcutTable ORDER BY key ASC, value ASC; +ALTER TABLE columnShortcutTable REPLACE COLUMNS (key int); +SELECT columnShortcutTable.* FROM columnShortcutTable ORDER BY key ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/columnstats_partlvl.q b/src/test/hive/ql/src/test/queries/clientpositive/columnstats_partlvl.q new file mode 100644 index 0000000000000..98627776d3163 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/columnstats_partlvl.q @@ -0,0 +1,20 @@ + +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); + +explain +analyze table Employee_Part partition (employeeSalary=2000.0) compute statistics for columns employeeID; +explain extended +analyze table Employee_Part partition (employeeSalary=2000.0) compute statistics for columns employeeID; +analyze table Employee_Part partition (employeeSalary=2000.0) compute statistics for columns employeeID; + +explain +analyze table Employee_Part partition (employeeSalary=4000.0) compute statistics for columns employeeID; +explain extended +analyze table Employee_Part partition (employeeSalary=4000.0) compute statistics for columns employeeID; +analyze table Employee_Part partition (employeeSalary=4000.0) compute statistics for columns employeeID; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/columnstats_tbllvl.q b/src/test/hive/ql/src/test/queries/clientpositive/columnstats_tbllvl.q new file mode 100644 index 0000000000000..72d88a67b5212 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/columnstats_tbllvl.q @@ -0,0 +1,38 @@ + +DROP TABLE IF EXISTS UserVisits_web_text_none; + +CREATE TABLE UserVisits_web_text_none ( + sourceIP string, + destURL string, + visitDate string, + adRevenue float, + userAgent string, + cCode string, + lCode string, + sKeyword string, + 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; + +explain +analyze table UserVisits_web_text_none compute statistics for columns sourceIP, avgTimeOnSite, adRevenue; + +explain extended +analyze table UserVisits_web_text_none compute statistics for columns sourceIP, avgTimeOnSite, adRevenue; + +analyze table UserVisits_web_text_none compute statistics for columns sourceIP, avgTimeOnSite, adRevenue; + +CREATE TABLE empty_tab( + a int, + b double, + c string, + d boolean, + e binary) +row format delimited fields terminated by '|' stored as textfile; + +explain +analyze table empty_tab compute statistics for columns a,b,c,d,e; + +analyze table empty_tab compute statistics for columns a,b,c,d,e; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/combine1.q b/src/test/hive/ql/src/test/queries/clientpositive/combine1.q new file mode 100644 index 0000000000000..86abf0996057b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/combine1.q @@ -0,0 +1,17 @@ +set hive.exec.compress.output = true; +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set mapred.min.split.size=256; +set mapred.min.split.size.per.node=256; +set mapred.min.split.size.per.rack=256; +set mapred.max.split.size=256; + +set mapred.output.compression.codec=org.apache.hadoop.io.compress.GzipCodec; + +create table combine1_1(key string, value string) stored as textfile; + +insert overwrite table combine1_1 +select * from src; + + +select key, value from combine1_1 ORDER BY key ASC, value ASC; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/combine2.q b/src/test/hive/ql/src/test/queries/clientpositive/combine2.q new file mode 100644 index 0000000000000..cfd9856f0868a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/combine2.q @@ -0,0 +1,51 @@ +USE default; + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set mapred.min.split.size=256; +set mapred.min.split.size.per.node=256; +set mapred.min.split.size.per.rack=256; +set mapred.max.split.size=256; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set mapred.cache.shared.enabled=false; +set hive.merge.smallfiles.avgsize=0; + +-- EXCLUDE_OS_WINDOWS +-- excluded on windows because of difference in file name encoding logic + + + +create table combine2(key string) partitioned by (value string); + +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) +-- This test sets mapred.max.split.size=256 and hive.merge.smallfiles.avgsize=0 +-- in an attempt to force the generation of multiple splits and multiple output files. +-- However, Hadoop 0.20 is incapable of generating splits smaller than the block size +-- when using CombineFileInputFormat, so only one split is generated. This has a +-- significant impact on the results results of this test. +-- This issue was fixed in MAPREDUCE-2046 which is included in 0.22. + +insert overwrite table combine2 partition(value) +select * from ( + select key, value from src where key < 10 + union all + select key, '|' as value from src where key = 11 + union all + select key, '2010-04-21 09:45:00' value from src where key = 19) s; + +show partitions combine2; + +explain +select key, value from combine2 where value is not null order by key; + +select key, value from combine2 where value is not null order by key; + +explain extended +select count(1) from combine2 where value is not null; + +select count(1) from combine2 where value is not null; + +explain +select ds, count(1) from srcpart where ds is not null group by ds; + +select ds, count(1) from srcpart where ds is not null group by ds; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/combine2_hadoop20.q b/src/test/hive/ql/src/test/queries/clientpositive/combine2_hadoop20.q new file mode 100644 index 0000000000000..8f9a59d497536 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/combine2_hadoop20.q @@ -0,0 +1,50 @@ +USE default; + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set mapred.min.split.size=256; +set mapred.min.split.size.per.node=256; +set mapred.min.split.size.per.rack=256; +set mapred.max.split.size=256; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set mapred.cache.shared.enabled=false; +set hive.merge.smallfiles.avgsize=0; + +-- EXCLUDE_OS_WINDOWS +-- excluded on windows because of difference in file name encoding logic + + +create table combine2(key string) partitioned by (value string); + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) +-- This test sets mapred.max.split.size=256 and hive.merge.smallfiles.avgsize=0 +-- in an attempt to force the generation of multiple splits and multiple output files. +-- However, Hadoop 0.20 is incapable of generating splits smaller than the block size +-- when using CombineFileInputFormat, so only one split is generated. This has a +-- significant impact on the results results of this test. +-- This issue was fixed in MAPREDUCE-2046 which is included in 0.22. + +insert overwrite table combine2 partition(value) +select * from ( + select key, value from src where key < 10 + union all + select key, '|' as value from src where key = 11 + union all + select key, '2010-04-21 09:45:00' value from src where key = 19) s; + +show partitions combine2; + +explain +select key, value from combine2 where value is not null order by key; + +select key, value from combine2 where value is not null order by key; + +explain extended +select count(1) from combine2 where value is not null; + +select count(1) from combine2 where value is not null; + +explain +select ds, count(1) from srcpart where ds is not null group by ds; + +select ds, count(1) from srcpart where ds is not null group by ds; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/combine2_win.q b/src/test/hive/ql/src/test/queries/clientpositive/combine2_win.q new file mode 100644 index 0000000000000..f6090bb99b29a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/combine2_win.q @@ -0,0 +1,41 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set mapred.min.split.size=256; +set mapred.min.split.size.per.node=256; +set mapred.min.split.size.per.rack=256; +set mapred.max.split.size=256; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set mapred.cache.shared.enabled=false; +set hive.merge.smallfiles.avgsize=0; + +-- INCLUDE_OS_WINDOWS +-- included only on windows because of difference in file name encoding logic + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +create table combine2(key string) partitioned by (value string); + +insert overwrite table combine2 partition(value) +select * from ( + select key, value from src where key < 10 + union all + select key, '|' as value from src where key = 11 + union all + select key, '2010-04-21 09:45:00' value from src where key = 19) s; + +show partitions combine2; + +explain +select key, value from combine2 where value is not null order by key; + +select key, value from combine2 where value is not null order by key; + +explain extended +select count(1) from combine2 where value is not null; + +select count(1) from combine2 where value is not null; + +explain +select ds, count(1) from srcpart where ds is not null group by ds; + +select ds, count(1) from srcpart where ds is not null group by ds; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/combine3.q b/src/test/hive/ql/src/test/queries/clientpositive/combine3.q new file mode 100644 index 0000000000000..c9afc91bb4561 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/combine3.q @@ -0,0 +1,41 @@ +set hive.exec.compress.output = true; +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set mapred.min.split.size=256; +set mapred.min.split.size.per.node=256; +set mapred.min.split.size.per.rack=256; +set mapred.max.split.size=256; + + +drop table combine_3_srcpart_seq_rc; + +create table combine_3_srcpart_seq_rc (key int , value string) partitioned by (ds string, hr string) stored as sequencefile; + +insert overwrite table combine_3_srcpart_seq_rc partition (ds="2010-08-03", hr="00") select * from src; + +alter table combine_3_srcpart_seq_rc set fileformat rcfile; +insert overwrite table combine_3_srcpart_seq_rc partition (ds="2010-08-03", hr="001") select * from src; + +desc extended combine_3_srcpart_seq_rc partition(ds="2010-08-03", hr="00"); +desc extended combine_3_srcpart_seq_rc partition(ds="2010-08-03", hr="001"); + +select key, value, ds, hr from combine_3_srcpart_seq_rc where ds="2010-08-03" order by key, hr limit 30; + +set hive.enforce.bucketing = true; +set hive.exec.reducers.max = 1; + +drop table bucket3_1; +CREATE TABLE combine_3_srcpart_seq_rc_bucket(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS stored as sequencefile; + +insert overwrite table combine_3_srcpart_seq_rc_bucket partition (ds='1') +select * from src; + +alter table combine_3_srcpart_seq_rc_bucket set fileformat rcfile; + +insert overwrite table combine_3_srcpart_seq_rc_bucket partition (ds='11') +select * from src; + +select key, ds from combine_3_srcpart_seq_rc_bucket tablesample (bucket 1 out of 2) s where ds = '1' or ds= '11' order by key, ds limit 30; + +drop table combine_3_srcpart_seq_rc_bucket; + +drop table combine_3_srcpart_seq_rc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_binary.q b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_binary.q new file mode 100644 index 0000000000000..c19813602782f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_binary.q @@ -0,0 +1,9 @@ +create table tab_binary(a binary); + +-- insert some data +LOAD DATA LOCAL INPATH "../data/files/binary.txt" INTO TABLE tab_binary; + +select count(*) from tab_binary; + +-- compute statistical summary of data +select compute_stats(a, 16) from tab_binary; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_boolean.q b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_boolean.q new file mode 100644 index 0000000000000..dc76f7c7d57e1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_boolean.q @@ -0,0 +1,9 @@ +create table tab_bool(a boolean); + +-- insert some data +LOAD DATA LOCAL INPATH "../data/files/bool.txt" INTO TABLE tab_bool; + +select count(*) from tab_bool; + +-- compute statistical summary of data +select compute_stats(a, 16) from tab_bool; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_double.q b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_double.q new file mode 100644 index 0000000000000..6c6dc47e67701 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_double.q @@ -0,0 +1,9 @@ +create table tab_double(a double); + +-- insert some data +LOAD DATA LOCAL INPATH "../data/files/double.txt" INTO TABLE tab_double; + +select count(*) from tab_double; + +-- compute statistical summary of data +select compute_stats(a, 16) from tab_double; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_empty_table.q b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_empty_table.q new file mode 100644 index 0000000000000..d4ed93fa63987 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_empty_table.q @@ -0,0 +1,12 @@ +create table tab_empty(a boolean, b int, c double, d string, e binary); + +select count(*) from tab_empty; + +-- compute statistical summary of data +select compute_stats(a, 16) from tab_empty; +select compute_stats(b, 16) from tab_empty; +select compute_stats(c, 16) from tab_empty; +select compute_stats(d, 16) from tab_empty; +select compute_stats(e, 16) from tab_empty; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_long.q b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_long.q new file mode 100644 index 0000000000000..7d0a1584a28cc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_long.q @@ -0,0 +1,9 @@ +create table tab_int(a int); + +-- insert some data +LOAD DATA LOCAL INPATH "../data/files/int.txt" INTO TABLE tab_int; + +select count(*) from tab_int; + +-- compute statistical summary of data +select compute_stats(a, 16) from tab_int; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_string.q b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_string.q new file mode 100644 index 0000000000000..f146f6b95b544 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_string.q @@ -0,0 +1,9 @@ +create table tab_string(a string); + +-- insert some data +LOAD DATA LOCAL INPATH "../data/files/string.txt" INTO TABLE tab_string; + +select count(*) from tab_string; + +-- compute statistical summary of data +select compute_stats(a, 16) from tab_string; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q b/src/test/hive/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q new file mode 100644 index 0000000000000..1ae194338ce8c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q @@ -0,0 +1,21 @@ +CREATE TABLE citl_table (key STRING, value STRING) PARTITIONED BY (part STRING) +STORED AS RCFILE +LOCATION 'pfile:${system:test.tmp.dir}/citl_table'; + +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyPartitionIsSubdirectoryOfTableHook; + +INSERT OVERWRITE TABLE citl_table PARTITION (part = '1') SELECT * FROM src; + +SET hive.exec.post.hooks=; + +ALTER TABLE citl_table SET LOCATION 'file:${system:test.tmp.dir}/citl_table'; + +ALTER TABLE citl_table PARTITION (part = '1') CONCATENATE; + +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyPartitionIsSubdirectoryOfTableHook; + +SELECT count(*) FROM citl_table where part = '1'; + +SET hive.exec.post.hooks=; + +DROP TABLE citl_table; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/constant_prop.q b/src/test/hive/ql/src/test/queries/clientpositive/constant_prop.q new file mode 100644 index 0000000000000..ced72d60445c2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/constant_prop.q @@ -0,0 +1,20 @@ +EXPLAIN +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 + ), + 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; + +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 + ), + 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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/convert_enum_to_string.q b/src/test/hive/ql/src/test/queries/clientpositive/convert_enum_to_string.q new file mode 100644 index 0000000000000..3dc4a6ad42523 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/convert_enum_to_string.q @@ -0,0 +1,10 @@ +-- Ensure Enum fields are converted to strings (instead of struct) + +create table convert_enum_to_string + partitioned by (b string) + row format serde "org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer" + with serdeproperties ( + "serialization.class"="org.apache.hadoop.hive.serde2.thrift.test.MegaStruct", + "serialization.format"="org.apache.thrift.protocol.TBinaryProtocol"); + +describe convert_enum_to_string; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer1.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer1.q new file mode 100644 index 0000000000000..b3fd3f760eaf0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer1.q @@ -0,0 +1,264 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- This query has a GroupByOperator folling JoinOperator and they share the same keys. +-- When Correlation Optimizer is turned off, three MR jobs will be generated. +-- When Correlation Optimizer is turned on, two MR jobs will be generated +-- and JoinOperator (on the column of key) and GroupByOperator (also on the column +-- of key) will be executed in the first MR job. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.auto.convert.join=true; +set hive.optimize.correlation=true; +-- Enable hive.auto.convert.join. +-- Correlation Optimizer will detect that the join will be converted to a Map-join, +-- so it will not try to optimize this query. +-- We should generate 1 MR job for subquery tmp. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- If the key of a GroupByOperator is the left table's key in +-- a Left Semi Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- If the key of a GroupByOperator is the left table's key in +-- a Left Outer Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + + +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 +-- operators because those keys with a null value are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +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; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +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=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +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; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +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 the key of a GroupByOperator is the right table's key in +-- a Right Outer Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + + +set hive.optimize.correlation=false; +-- If the key of a GroupByOperator is the left table's key in +-- a Right 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)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=false; +-- This query has a Full Outer Join followed by a GroupByOperator and +-- they share the same key. Because those keys with a null value are not grouped +-- in the output of the Full Outer Join, we cannot use a single MR to execute +-- these two operators. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- Currently, we only handle exactly same keys, this query will not be optimized +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp; + +set hive.optimize.correlation=false; +-- Currently, we only handle exactly same keys, this query will not be optimized +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer10.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer10.q new file mode 100644 index 0000000000000..4369125a77a14 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer10.q @@ -0,0 +1,130 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +-- This case is used to test LEFT SEMI JOIN since Hive will +-- introduce a GroupByOperator before the ReduceSinkOperator of +-- the right table (yy in queries below) +-- of LEFT SEMI JOIN. +EXPLAIN +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt; + +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt; + +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +-- This case is used to test LEFT SEMI JOIN since Hive will +-- introduce a GroupByOperator before the ReduceSinkOperator of +-- the right table (yy in queries below) +-- of LEFT SEMI JOIN. +EXPLAIN +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- This test is used to test if we can use shared scan for +-- xx, yy:x, and yy:y. +EXPLAIN +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer11.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer11.q new file mode 100644 index 0000000000000..412baa77f549f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer11.q @@ -0,0 +1,73 @@ +set hive.auto.convert.join=false; +-- Tests in this file are used to make sure Correlation Optimizer +-- can correctly handle tables with partitions + +CREATE TABLE part_table(key string, value string) PARTITIONED BY (partitionId int); +INSERT OVERWRITE TABLE part_table PARTITION (partitionId=1) + SELECT key, value FROM src ORDER BY key, value LIMIT 100; +INSERT OVERWRITE TABLE part_table PARTITION (partitionId=2) + SELECT key, value FROM src1 ORDER BY key, value; + +set hive.optimize.correlation=false; +-- In this case, we should not do shared scan on part_table +-- because left and right tables of JOIN use different partitions +-- of part_table. With Correlation Optimizer we will generate +-- 1 MR job. +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key; + +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key; + +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key; + +set hive.optimize.correlation=false; +-- In this case, we should do shared scan on part_table +-- because left and right tables of JOIN use the same partition +-- of part_table. With Correlation Optimizer we will generate +-- 1 MR job. +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key; + +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key; + +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer12.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer12.q new file mode 100644 index 0000000000000..409cdb4c5b38c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer12.q @@ -0,0 +1,9 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=true; +-- Currently, correlation optimizer does not support PTF operator +EXPLAIN SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(x.value) OVER (PARTITION BY x.key) AS cnt FROM src x) xx +JOIN +(SELECT y.key as key, count(y.value) OVER (PARTITION BY y.key) AS cnt FROM src1 y) yy +ON (xx.key=yy.key); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer13.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer13.q new file mode 100644 index 0000000000000..e979465d28af9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer13.q @@ -0,0 +1,18 @@ +CREATE TABLE tmp(c1 INT, c2 INT, c3 STRING, c4 STRING); +INSERT OVERWRITE TABLE tmp +SELECT x.key, y.key, x.value, y.value FROM src x JOIN src y ON (x.key = y.key); + +set hive.optimize.correlation=true; +-- The query in this file have operators with same set of keys +-- but having different sorting orders. +-- Correlation optimizer currently do not optimize this case. +-- This case will be optimized latter (need a follow-up jira). + +EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c3, x.c1) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c3, x1.c1) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer14.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer14.q new file mode 100644 index 0000000000000..1322b793780a9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer14.q @@ -0,0 +1,149 @@ +set hive.optimize.reducededuplication=true; +set hive.optimize.reducededuplication.min.reducer=1; +set hive.optimize.correlation=true; +-- This file is used to show plans of queries involving cluster by, distribute by, +-- order by, and sort by. +-- Right now, Correlation optimizer check the most restrictive condition +-- when determining if a ReduceSinkOperator is not necessary. +-- This condition is that two ReduceSinkOperators should have same sorting columns, +-- same partitioning columns, same sorting orders and no conflict on the numbers of reducers. + +-- Distribute by will not be optimized because distribute by does not introduce +-- sorting columns. +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key) yy +ON (xx.key=yy.key); + +-- Sort by will not be optimized because sort by does not introduce partitioning columns +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y SORT BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=false; +-- Distribute by and sort by on the same key(s) should be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=true; +-- Because for join we use ascending order, if sort by uses descending order, +-- this query will not be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key DESC) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key DESC) yy +ON (xx.key=yy.key); + +-- Even if hive.optimize.reducededuplication.min.reducer=1, order by will not be optimized +-- because order by does not introduce partitioning columns +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x ORDER BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y ORDER BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=false; +-- Cluster by will be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=false; +-- If hive.optimize.reducededuplication.min.reducer=1, +-- group by and then order by should be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer15.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer15.q new file mode 100644 index 0000000000000..dd6143c402c20 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer15.q @@ -0,0 +1,32 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +set hive.optimize.correlation=true; +set hive.join.emit.interval=1; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer2.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer2.q new file mode 100644 index 0000000000000..af97f16e3629a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer2.q @@ -0,0 +1,188 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- In this query, subquery a and b both have a GroupByOperator and the a and b will be +-- joined. The key of JoinOperator is the same with both keys of GroupByOperators in subquery +-- a and b. When Correlation Optimizer is turned off, we have four MR jobs. +-- When Correlation Optimizer is turned on, 2 MR jobs will be generated. +-- The first job will evaluate subquery tmp (including subquery a, b, and the JoinOperator on a +-- and b). +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=false; +-- Left Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=false; +-- Right Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=false; +-- Full Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=false; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp; + +set hive.optimize.correlation=true; +-- After FULL OUTER JOIN, keys with null values are not grouped, right now, +-- we have to generate 2 MR jobs for tmp, 1 MR job for a join b and another for the +-- GroupByOperator on key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, we need 4 MR jobs. +-- When Correlation Optimizer is turned on, the subquery of tmp will be evaluated in +-- a single MR job (including the subquery a, the subquery b, and a join b). So, we +-- will have 2 MR jobs. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer3.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer3.q new file mode 100644 index 0000000000000..89b7c15690ab1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer3.q @@ -0,0 +1,96 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 5 MR jobs will be generated. +-- When Correlation Optimizer is turned on, the subquery tmp will be evalauted +-- in a single MR job (including the subquery b, the subquery d, and b join d). +-- At the reduce side of the MR job evaluating tmp, two operation paths +-- (for subquery b and d) have different depths. The path starting from subquery b +-- is JOIN->GBY->JOIN, which has a depth of 3. While, the path starting from subquery d +-- is JOIN->JOIN. We should be able to handle this case. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +-- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +-- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer4.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer4.q new file mode 100644 index 0000000000000..70fcdfc0b4288 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer4.q @@ -0,0 +1,158 @@ +CREATE TABLE T1(key INT, val STRING); +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; +CREATE TABLE T3(key INT, val STRING); +LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T3; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, this query will be evaluated +-- by 3 MR jobs. +-- When Correlation Optimizer is turned on, this query will be evaluated by +-- 2 MR jobs. The subquery tmp will be evaluated in a single MR job. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +-- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- This case should be optimized, since the key of GroupByOperator is from the leftmost table +-- of a chain of LEFT OUTER JOINs. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +-- This query will not be optimized by correlation optimizer because +-- GroupByOperator uses y.key (a right table of a left outer join) +-- as the key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=false; +-- This case should be optimized, since the key of GroupByOperator is from the rightmost table +-- of a chain of RIGHT OUTER JOINs. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp; + +set hive.optimize.correlation=true; +-- This query will not be optimized by correlation optimizer because +-- GroupByOperator uses y.key (a left table of a right outer join) +-- as the key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=false; +-- This case should not be optimized because afer the FULL OUTER JOIN, rows with null keys +-- are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer5.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer5.q new file mode 100644 index 0000000000000..ac836c0bfdc91 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer5.q @@ -0,0 +1,79 @@ +CREATE TABLE T1(key INT, val STRING); +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; +CREATE TABLE T3(key INT, val STRING); +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; + +CREATE TABLE dest_co1(key INT, val STRING); +CREATE TABLE dest_co2(key INT, val STRING); +CREATE TABLE dest_co3(key INT, val STRING); + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 3 MR jobs are needed. +-- When Correlation Optimizer is turned on, only a single MR job is needed. +EXPLAIN +INSERT OVERWRITE TABLE dest_co1 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +INSERT OVERWRITE TABLE dest_co1 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +set hive.optimize.correlation=true; +EXPLAIN +INSERT OVERWRITE TABLE dest_co2 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +INSERT OVERWRITE TABLE dest_co2 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask.size=10000000000; +-- Enable hive.auto.convert.join. +EXPLAIN +INSERT OVERWRITE TABLE dest_co3 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +INSERT OVERWRITE TABLE dest_co3 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +-- dest_co1, dest_co2 and dest_co3 should be same +-- SELECT * FROM dest_co1 x ORDER BY x.key, x.val; +-- SELECT * FROM dest_co2 x ORDER BY x.key, x.val; +SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co1; +SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co2; +SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer6.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer6.q new file mode 100644 index 0000000000000..89c060919c205 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer6.q @@ -0,0 +1,322 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx, subquery yy, and xx join yy. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +-- Enable hive.auto.convert.join. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 3 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery tmp and tmp join z. +EXPLAIN +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value; + +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value; + +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 4 MR jobs are needed. +-- 2 MR jobs are used to evaluate yy, 1 MR is used to evaluate xx and xx join yy. +-- The last MR is used for ordering. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer7.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer7.q new file mode 100644 index 0000000000000..3ba83f489ccc3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer7.q @@ -0,0 +1,77 @@ +set hive.auto.convert.join=true; + +set hive.optimize.correlation=false; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000000000; + +set hive.optimize.correlation=false; +-- Without correlation optimizer, we will have 3 MR jobs. +-- The first one is a MapJoin and Aggregation (in the Reduce Phase). +-- The second one is another MapJoin. The third one is for ordering. +-- With the correlation optimizer, right now, we have +-- 2 MR jobs. The first one will evaluate the sub-query xx and the join of +-- xx and yy. The second one will do the ORDER BY. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer8.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer8.q new file mode 100644 index 0000000000000..4e79d80f11bbb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer8.q @@ -0,0 +1,110 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When the Correlation Optimizer is turned off, this query will be evaluated by +-- 4 MR jobs. +-- When the Correlation Optimizer is turned on, because both inputs of the +-- UnionOperator are correlated, we can use 2 MR jobs to evaluate this query. +-- The first MR job will evaluate subquery subq1 and subq1 join x. The second +-- MR is for ordering. +EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt; + +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt; + +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt; + +set hive.optimize.correlation=false; +-- When the Correlation Optimizer is turned off, this query will be evaluated by +-- 4 MR jobs. +-- When the Correlation Optimizer is turned on, because both inputs of the +-- UnionOperator are correlated, we can use 2 MR jobs to evaluate this query. +-- The first MR job will evaluate subquery subq1 and subq1 join x. The second +-- MR is for ordering. +EXPLAIN +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value; + +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value; + +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value; + +set hive.optimize.correlation=true; +-- When the Correlation Optimizer is turned on, because a input of UnionOperator is +-- not correlated, we cannot handle this case right now. So, this query will not be +-- optimized. +EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key, x1.value +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt; + +set hive.optimize.correlation=true; +-- When the Correlation Optimizer is turned on, because a input of UnionOperator is +-- not correlated, we cannot handle this case right now. So, this query will not be +-- optimized. +EXPLAIN +SELECT subq1.key, subq1.value, x.key, x.value +FROM +( SELECT cast(x.key as INT) as key, count(1) as value from src x where x.key < 20 group by x.key + UNION ALL + SELECT count(1) as key, cast(x1.key as INT) as value from src x1 where x1.key > 100 group by x1.key +) subq1 +FULL OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.value, x.key, x.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer9.q b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer9.q new file mode 100644 index 0000000000000..6169a30c05146 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer9.q @@ -0,0 +1,71 @@ +CREATE TABLE tmp(c1 INT, c2 INT, c3 STRING, c4 STRING); + +set hive.auto.convert.join=false; + +INSERT OVERWRITE TABLE tmp +SELECT x.key, y.key, x.value, y.value FROM src x JOIN src y ON (x.key = y.key); + +set hive.optimize.correlation=false; +EXPLAIN +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt; + +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt; + +set hive.optimize.correlation=true; +-- The merged table scan should be able to load both c1 and c2 +EXPLAIN +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt; + +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt; + +set hive.optimize.correlation=false; +EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt; + +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt; + +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/count.q b/src/test/hive/ql/src/test/queries/clientpositive/count.q new file mode 100644 index 0000000000000..0d66a5ec345e6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/count.q @@ -0,0 +1,17 @@ +create table abcd (a int, b int, c int, d int); +LOAD DATA LOCAL INPATH '../data/files/in4.txt' INTO TABLE abcd; + +select * from abcd; +set hive.map.aggr=true; +explain select a, count(distinct b), count(distinct c), sum(d) from abcd group by a; +select a, count(distinct b), count(distinct c), sum(d) from abcd group by a; + +explain select count(1), count(*), count(a), count(b), count(c), count(d), count(distinct a), count(distinct b), count(distinct c), count(distinct d), count(distinct a,b), count(distinct b,c), count(distinct c,d), count(distinct a,d), count(distinct a,c), count(distinct b,d), count(distinct a,b,c), count(distinct b,c,d), count(distinct a,c,d), count(distinct a,b,d), count(distinct a,b,c,d) from abcd; +select count(1), count(*), count(a), count(b), count(c), count(d), count(distinct a), count(distinct b), count(distinct c), count(distinct d), count(distinct a,b), count(distinct b,c), count(distinct c,d), count(distinct a,d), count(distinct a,c), count(distinct b,d), count(distinct a,b,c), count(distinct b,c,d), count(distinct a,c,d), count(distinct a,b,d), count(distinct a,b,c,d) from abcd; + +set hive.map.aggr=false; +explain select a, count(distinct b), count(distinct c), sum(d) from abcd group by a; +select a, count(distinct b), count(distinct c), sum(d) from abcd group by a; + +explain select count(1), count(*), count(a), count(b), count(c), count(d), count(distinct a), count(distinct b), count(distinct c), count(distinct d), count(distinct a,b), count(distinct b,c), count(distinct c,d), count(distinct a,d), count(distinct a,c), count(distinct b,d), count(distinct a,b,c), count(distinct b,c,d), count(distinct a,c,d), count(distinct a,b,d), count(distinct a,b,c,d) from abcd; +select count(1), count(*), count(a), count(b), count(c), count(d), count(distinct a), count(distinct b), count(distinct c), count(distinct d), count(distinct a,b), count(distinct b,c), count(distinct c,d), count(distinct a,d), count(distinct a,c), count(distinct b,d), count(distinct a,b,c), count(distinct b,c,d), count(distinct a,c,d), count(distinct a,b,d), count(distinct a,b,c,d) from abcd; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/cp_mj_rc.q b/src/test/hive/ql/src/test/queries/clientpositive/cp_mj_rc.q new file mode 100644 index 0000000000000..62d38e3caae58 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/cp_mj_rc.q @@ -0,0 +1,7 @@ +create table src_six_columns (k1 string, v1 string, k2 string, v2 string, k3 string, v3 string) stored as rcfile; +insert overwrite table src_six_columns select value, value, key, value, value, value from src; +create table src_two_columns (k1 string, v1 string) stored as rcfile; +insert overwrite table src_two_columns select key, value from src; +SELECT /*+ MAPJOIN(six) */ six.*, two.k1 from src_six_columns six join src_two_columns two on (six.k3=two.k1); + +SELECT /*+ MAPJOIN(two) */ two.*, six.k3 from src_six_columns six join src_two_columns two on (six.k3=two.k1); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_1.q b/src/test/hive/ql/src/test/queries/clientpositive/create_1.q new file mode 100644 index 0000000000000..f348e5902263a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_1.q @@ -0,0 +1,29 @@ +set fs.default.name=invalidscheme:///; + +CREATE TABLE table1 (a STRING, b STRING) STORED AS TEXTFILE; +DESCRIBE table1; +DESCRIBE EXTENDED table1; + +CREATE TABLE IF NOT EXISTS table1 (a STRING, b STRING) STORED AS TEXTFILE; + +CREATE TABLE IF NOT EXISTS table2 (a STRING, b INT) STORED AS TEXTFILE; +DESCRIBE table2; +DESCRIBE EXTENDED table2; + +CREATE TABLE table3 (a STRING, b STRING) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +STORED AS TEXTFILE; +DESCRIBE table3; +DESCRIBE EXTENDED table3; + +CREATE TABLE table4 (a STRING, b STRING) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +STORED AS SEQUENCEFILE; +DESCRIBE table4; +DESCRIBE EXTENDED table4; + +CREATE TABLE table5 (a STRING, b STRING) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +STORED AS RCFILE; +DESCRIBE table5; +DESCRIBE EXTENDED table5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q b/src/test/hive/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q new file mode 100644 index 0000000000000..b51b238a2d0e2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q @@ -0,0 +1,40 @@ +set hive.mapred.supports.subdirectories=true; + +-- Test stored as directories +-- it covers a few cases + +-- 1. create a table with stored as directories +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; +describe formatted stored_as_dirs_multiple; + +-- 2. turn off stored as directories but table is still a skewed table +alter table stored_as_dirs_multiple not stored as DIRECTORIES; +describe formatted stored_as_dirs_multiple; + +-- 3. turn off skewed +alter table stored_as_dirs_multiple not skewed; +describe formatted stored_as_dirs_multiple; + +-- 4. alter a table to stored as directories +CREATE TABLE stored_as_dirs_single (key STRING, value STRING); +alter table stored_as_dirs_single SKEWED BY (key) ON ('1','5','6') +stored as DIRECTORIES; +describe formatted stored_as_dirs_single; + +-- 5. turn off skewed should turn off stored as directories too +alter table stored_as_dirs_single not skewed; +describe formatted stored_as_dirs_single; + +-- 6. turn on stored as directories again +alter table stored_as_dirs_single SKEWED BY (key) ON ('1','5','6') +stored as DIRECTORIES; +describe formatted stored_as_dirs_single; + +-- 7. create table like +create table stored_as_dirs_single_like like stored_as_dirs_single; +describe formatted stored_as_dirs_single_like; + +-- cleanup +drop table stored_as_dirs_single; +drop table stored_as_dirs_multiple; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_big_view.q b/src/test/hive/ql/src/test/queries/clientpositive/create_big_view.q new file mode 100644 index 0000000000000..49caacc3e2019 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_big_view.q @@ -0,0 +1,246 @@ +DROP VIEW big_view; + +-- Define a view with long SQL text to test metastore and other limits. + +CREATE VIEW big_view AS SELECT +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' AS a, +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', +'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' +FROM src; + +SELECT a FROM big_view +LIMIT 1; + +DROP VIEW big_view; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_default_prop.q b/src/test/hive/ql/src/test/queries/clientpositive/create_default_prop.q new file mode 100644 index 0000000000000..26d7332ecb88c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_default_prop.q @@ -0,0 +1,10 @@ +set hive.table.parameters.default=p1=v1,P2=v21=v22=v23; +CREATE TABLE table_p1 (a STRING); +DESC EXTENDED table_p1; + +set hive.table.parameters.default=p3=v3; +CREATE TABLE table_p2 LIKE table_p1; +DESC EXTENDED table_p2; + +CREATE TABLE table_p3 AS SELECT * FROM table_p1; +DESC EXTENDED table_p3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_escape.q b/src/test/hive/ql/src/test/queries/clientpositive/create_escape.q new file mode 100644 index 0000000000000..49f7c5fe601aa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_escape.q @@ -0,0 +1,10 @@ +CREATE TABLE table1 (a STRING, b STRING) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' ESCAPED BY '\\' +STORED AS TEXTFILE; + +DESCRIBE table1; +DESCRIBE EXTENDED table1; + +INSERT OVERWRITE TABLE table1 SELECT key, '\\\t\\' FROM src WHERE key = 86; + +SELECT * FROM table1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_genericudaf.q b/src/test/hive/ql/src/test/queries/clientpositive/create_genericudaf.q new file mode 100644 index 0000000000000..b2bc0423e8b2a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_genericudaf.q @@ -0,0 +1,17 @@ +EXPLAIN +CREATE TEMPORARY FUNCTION test_avg AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'; + +CREATE TEMPORARY FUNCTION test_avg AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'; + +EXPLAIN +SELECT + test_avg(1), + test_avg(substr(value,5)) +FROM src; + +SELECT + test_avg(1), + test_avg(substr(value,5)) +FROM src; + +DROP TEMPORARY FUNCTIOn test_avg; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_genericudf.q b/src/test/hive/ql/src/test/queries/clientpositive/create_genericudf.q new file mode 100644 index 0000000000000..1d43f30e8e150 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_genericudf.q @@ -0,0 +1,21 @@ +EXPLAIN +CREATE TEMPORARY FUNCTION test_translate AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestTranslate'; + +CREATE TEMPORARY FUNCTION test_translate AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestTranslate'; + +CREATE TABLE dest1(c1 STRING, c2 STRING, c3 STRING, c4 STRING, c5 STRING, c6 STRING, c7 STRING); + +FROM src +INSERT OVERWRITE TABLE dest1 +SELECT + test_translate('abc', 'a', 'b'), + test_translate('abc', 'ab', 'bc'), + test_translate(NULL, 'a', 'b'), + test_translate('a', NULL, 'b'), + test_translate('a', 'a', NULL), + test_translate('abc', 'ab', 'b'), + test_translate('abc', 'a', 'ab'); + +SELECT dest1.* FROM dest1 LIMIT 1; + +DROP TEMPORARY FUNCTION test_translate; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_insert_outputformat.q b/src/test/hive/ql/src/test/queries/clientpositive/create_insert_outputformat.q new file mode 100644 index 0000000000000..f18d68866a8c6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_insert_outputformat.q @@ -0,0 +1,30 @@ + + +CREATE TABLE table_test_output_format(key INT, value STRING) STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'; + +FROM src +INSERT OVERWRITE TABLE table_test_output_format SELECT src.key, src.value LIMIT 10; +describe table_test_output_format; + + + +CREATE TABLE table_test_output_format_sequencefile(key INT, value STRING) STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileOutputFormat'; + +FROM src +INSERT OVERWRITE TABLE table_test_output_format_sequencefile SELECT src.key, src.value LIMIT 10; +describe table_test_output_format_sequencefile; + + + +CREATE TABLE table_test_output_format_hivesequencefile(key INT, value STRING) STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat'; + +FROM src +INSERT OVERWRITE TABLE table_test_output_format_hivesequencefile SELECT src.key, src.value LIMIT 10; +describe table_test_output_format_hivesequencefile; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_like.q b/src/test/hive/ql/src/test/queries/clientpositive/create_like.q new file mode 100644 index 0000000000000..cb4d6578af476 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_like.q @@ -0,0 +1,35 @@ + + + + +CREATE TABLE table1 (a STRING, b STRING) STORED AS TEXTFILE; +DESCRIBE FORMATTED table1; + +CREATE TABLE table2 LIKE table1; +DESCRIBE FORMATTED table2; + +CREATE TABLE IF NOT EXISTS table2 LIKE table1; + +CREATE EXTERNAL TABLE IF NOT EXISTS table2 LIKE table1; + +CREATE EXTERNAL TABLE IF NOT EXISTS table3 LIKE table1; +DESCRIBE FORMATTED table3; + +INSERT OVERWRITE TABLE table1 SELECT key, value FROM src WHERE key = 86; +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'; + +SELECT * FROM table4; +SELECT * FROM table5; + +DROP TABLE table5; +SELECT * FROM table4; +DROP TABLE table4; + +CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:test.src.data.dir}/files/ext_test'; +SELECT * FROM table4; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_like2.q b/src/test/hive/ql/src/test/queries/clientpositive/create_like2.q new file mode 100644 index 0000000000000..146467b2cd2c9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_like2.q @@ -0,0 +1,9 @@ +-- Tests the copying over of Table Parameters according to a HiveConf setting +-- when doing a CREATE TABLE LIKE. + +CREATE TABLE table1(a INT, b STRING); +ALTER TABLE table1 SET TBLPROPERTIES ('a'='1', 'b'='2', 'c'='3', 'd' = '4'); + +SET hive.ddl.createtablelike.properties.whitelist=a,c,D; +CREATE TABLE table2 LIKE table1; +DESC FORMATTED table2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_like_tbl_props.q b/src/test/hive/ql/src/test/queries/clientpositive/create_like_tbl_props.q new file mode 100644 index 0000000000000..8f951d1dda446 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_like_tbl_props.q @@ -0,0 +1,35 @@ +-- Test that CREATE TABLE LIKE commands can take explicit table properties + +CREATE TABLE test_table LIKE src TBLPROPERTIES('key'='value'); + +DESC FORMATTED test_table; + +set hive.table.parameters.default=key1=value1; + +--Test that CREATE TABLE LIKE commands can take default table properties + +CREATE TABLE test_table1 LIKE src; + +DESC FORMATTED test_table1; + +-- Test that CREATE TABLE LIKE commands can take default and explicit table properties + +CREATE TABLE test_table2 LIKE src TBLPROPERTIES('key2' = 'value2'); + +DESC FORMATTED test_table2; + +set hive.ddl.createtablelike.properties.whitelist=key2; + +-- Test that properties inherited are overwritten by explicitly set ones + +CREATE TABLE test_table3 LIKE test_table2 TBLPROPERTIES('key2' = 'value3'); + +DESC FORMATTED test_table3; + +--Test that CREATE TALBE LIKE on a view can take explicit table properties + +CREATE VIEW test_view (key, value) AS SELECT * FROM src; + +CREATE TABLE test_table4 LIKE test_view TBLPROPERTIES('key'='value'); + +DESC FORMATTED test_table4; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_like_view.q b/src/test/hive/ql/src/test/queries/clientpositive/create_like_view.q new file mode 100644 index 0000000000000..f0bf58596256e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_like_view.q @@ -0,0 +1,39 @@ +DROP TABLE IF EXISTS table1; +DROP TABLE IF EXISTS table2; +DROP TABLE IF EXISTS table3; +DROP VIEW IF EXISTS view1; + +CREATE TABLE table1 (a STRING, b STRING) STORED AS TEXTFILE; +DESCRIBE table1; +DESCRIBE FORMATTED table1; + +CREATE VIEW view1 AS SELECT * FROM table1; + +CREATE TABLE table2 LIKE view1; +DESCRIBE table2; +DESCRIBE FORMATTED table2; + +CREATE TABLE IF NOT EXISTS table2 LIKE view1; + +CREATE EXTERNAL TABLE IF NOT EXISTS table2 LIKE view1; + +CREATE EXTERNAL TABLE IF NOT EXISTS table3 LIKE view1; +DESCRIBE table3; +DESCRIBE FORMATTED table3; + +INSERT OVERWRITE TABLE table1 SELECT key, value FROM src WHERE key = 86; +INSERT OVERWRITE TABLE table2 SELECT key, value FROM src WHERE key = 100; + +SELECT * FROM table1 order by a, b; +SELECT * FROM table2 order by a, b; + +DROP TABLE table1; +DROP TABLE table2; +DROP VIEW view1; + +-- check partitions +create view view1 partitioned on (ds, hr) as select * from srcpart; +create table table1 like view1; +describe formatted table1; +DROP TABLE table1; +DROP VIEW view1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_merge_compressed.q b/src/test/hive/ql/src/test/queries/clientpositive/create_merge_compressed.q new file mode 100644 index 0000000000000..4418b3430affc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_merge_compressed.q @@ -0,0 +1,24 @@ +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; + +set hive.exec.compress.output = true; + +create table tgt_rc_merge_test(key int, value string) stored as rcfile; +insert into table tgt_rc_merge_test select * from src_rc_merge_test; +insert into table tgt_rc_merge_test select * from src_rc_merge_test; + +show table extended like `tgt_rc_merge_test`; + +select count(1) from tgt_rc_merge_test; +select sum(hash(key)), sum(hash(value)) from tgt_rc_merge_test; + +alter table tgt_rc_merge_test concatenate; + +show table extended like `tgt_rc_merge_test`; + +select count(1) from tgt_rc_merge_test; +select sum(hash(key)), sum(hash(value)) from tgt_rc_merge_test; + +drop table src_rc_merge_test; +drop table tgt_rc_merge_test; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_nested_type.q b/src/test/hive/ql/src/test/queries/clientpositive/create_nested_type.q new file mode 100644 index 0000000000000..2debd0d71d224 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_nested_type.q @@ -0,0 +1,16 @@ + + +CREATE TABLE table1 ( + a STRING, + b ARRAY, + c ARRAY>, + d MAP> + ) STORED AS TEXTFILE; +DESCRIBE table1; +DESCRIBE EXTENDED table1; + +LOAD DATA LOCAL INPATH '../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table1; + +SELECT * from table1; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_or_replace_view.q b/src/test/hive/ql/src/test/queries/clientpositive/create_or_replace_view.q new file mode 100644 index 0000000000000..a8f59b7f05354 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_or_replace_view.q @@ -0,0 +1,32 @@ +drop view v; +create view v as select * from srcpart; +describe formatted v; + +-- modifying definition of unpartitioned view +create or replace view v partitioned on (ds, hr) as select * from srcpart; +alter view v add partition (ds='2008-04-08',hr='11'); +alter view v add partition (ds='2008-04-08',hr='12'); +select * from v where value='val_409' and ds='2008-04-08' and hr='11'; +describe formatted v; +show partitions v; + +-- altering partitioned view 1 +create or replace view v partitioned on (ds, hr) as select value, ds, hr from srcpart; +select * from v where value='val_409' and ds='2008-04-08' and hr='11'; +describe formatted v; +show partitions v; + +-- altering partitioned view 2 +create or replace view v partitioned on (ds, hr) as select key, value, ds, hr from srcpart; +select * from v where value='val_409' and ds='2008-04-08' and hr='11'; +describe formatted v; +show partitions v; +drop view v; + +-- updating to fix view with invalid definition +create table srcpart_temp like srcpart; +create view v partitioned on (ds, hr) as select * from srcpart_temp; +drop table srcpart_temp; -- v is now invalid +create or replace view v partitioned on (ds, hr) as select * from srcpart; +describe formatted v; +drop view v; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_skewed_table1.q b/src/test/hive/ql/src/test/queries/clientpositive/create_skewed_table1.q new file mode 100644 index 0000000000000..5d3a48acb4b3a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_skewed_table1.q @@ -0,0 +1,10 @@ +set hive.mapred.supports.subdirectories=true; +CREATE TABLE list_bucket_single (key STRING, value STRING) SKEWED BY (key) ON ('1','5','6'); +CREATE TABLE list_bucket_single_2 (key STRING, value STRING) SKEWED BY (key) ON ((1),(5),(6)); +CREATE TABLE list_bucket_multiple (col1 STRING, col2 int, col3 STRING) SKEWED BY (col1, col2) ON (('s1',1), ('s3',3), ('s13',13), ('s78',78)); +describe formatted list_bucket_single_2; +describe formatted list_bucket_single; +describe formatted list_bucket_multiple; +drop table list_bucket_single; +drop table list_bucket_multiple; +drop table list_bucket_single_2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_struct_table.q b/src/test/hive/ql/src/test/queries/clientpositive/create_struct_table.q new file mode 100644 index 0000000000000..dd5aa63e4578a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_struct_table.q @@ -0,0 +1,12 @@ + +create table abc(strct struct) +row format delimited + fields terminated by '\t' + collection items terminated by '\001'; + +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/src/test/hive/ql/src/test/queries/clientpositive/create_udaf.q b/src/test/hive/ql/src/test/queries/clientpositive/create_udaf.q new file mode 100644 index 0000000000000..a6bd7a0846da9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_udaf.q @@ -0,0 +1,19 @@ +EXPLAIN +CREATE TEMPORARY FUNCTION test_max AS 'org.apache.hadoop.hive.ql.udf.UDAFTestMax'; + +CREATE TEMPORARY FUNCTION test_max AS 'org.apache.hadoop.hive.ql.udf.UDAFTestMax'; + +CREATE TABLE dest1(col INT); + +FROM src INSERT OVERWRITE TABLE dest1 SELECT test_max(length(src.value)); + +SELECT dest1.* FROM dest1; + +-- cover all the other value types: +SELECT test_max(CAST(length(src.value) AS SMALLINT)) FROM src; +SELECT test_max(CAST(length(src.value) AS BIGINT)) FROM src; +SELECT test_max(CAST(length(src.value) AS DOUBLE)) FROM src; +SELECT test_max(CAST(length(src.value) AS FLOAT)) FROM src; +SELECT test_max(substr(src.value,5)) FROM src; + +DROP TEMPORARY FUNCTION test_max; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_union_table.q b/src/test/hive/ql/src/test/queries/clientpositive/create_union_table.q new file mode 100644 index 0000000000000..bb0e5b989d54d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_union_table.q @@ -0,0 +1,10 @@ +explain create table abc(mydata uniontype,struct>, +strct struct); + +create table abc(mydata uniontype,struct>, +strct struct); + +load data local inpath '../data/files/union_input.txt' +overwrite into table abc; + +SELECT * FROM abc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_view.q b/src/test/hive/ql/src/test/queries/clientpositive/create_view.q new file mode 100644 index 0000000000000..e08274c9e994f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_view.q @@ -0,0 +1,247 @@ +DROP VIEW view1; +DROP VIEW view2; +DROP VIEW view3; +DROP VIEW view4; +DROP VIEW view5; +DROP VIEW view6; +DROP VIEW view7; +DROP VIEW view8; +DROP VIEW view9; +DROP VIEW view10; +DROP VIEW view11; +DROP VIEW view12; +DROP VIEW view13; +DROP VIEW view14; +DROP VIEW view15; +DROP VIEW view16; +DROP TEMPORARY FUNCTION test_translate; +DROP TEMPORARY FUNCTION test_max; +DROP TEMPORARY FUNCTION test_explode; + + +SELECT * FROM src WHERE key=86; +CREATE VIEW view1 AS SELECT value FROM src WHERE key=86; +CREATE VIEW view2 AS SELECT * FROM src; +CREATE VIEW view3(valoo) +TBLPROPERTIES ("fear" = "factor") +AS SELECT upper(value) FROM src WHERE key=86; +SELECT * from view1; +SELECT * from view2 where key=18; +SELECT * from view3; + +-- test EXPLAIN output for CREATE VIEW +EXPLAIN +CREATE VIEW view0(valoo) AS SELECT upper(value) FROM src WHERE key=86; + +-- make sure EXPLAIN works with a query which references a view +EXPLAIN +SELECT * from view2 where key=18; + +SHOW TABLES 'view.*'; +DESCRIBE view1; +DESCRIBE EXTENDED view1; +DESCRIBE FORMATTED view1; +DESCRIBE view2; +DESCRIBE EXTENDED view2; +DESCRIBE FORMATTED view2; +DESCRIBE view3; +DESCRIBE EXTENDED view3; +DESCRIBE FORMATTED view3; + +ALTER VIEW view3 SET TBLPROPERTIES ("biggest" = "loser"); +DESCRIBE EXTENDED view3; +DESCRIBE FORMATTED view3; + +CREATE TABLE table1 (key int); + +-- use DESCRIBE EXTENDED on a base table and an external table as points +-- of comparison for view descriptions +DESCRIBE EXTENDED table1; +DESCRIBE EXTENDED src1; + +-- use DESCRIBE EXTENDED on a base table as a point of comparison for +-- view descriptions +DESCRIBE EXTENDED table1; + + +INSERT OVERWRITE TABLE table1 SELECT key FROM src WHERE key = 86; + +SELECT * FROM table1; +CREATE VIEW view4 AS SELECT * FROM table1; +SELECT * FROM view4; +DESCRIBE view4; +ALTER TABLE table1 ADD COLUMNS (value STRING); +SELECT * FROM table1; +SELECT * FROM view4; +DESCRIBE table1; +DESCRIBE view4; + +CREATE VIEW view5 AS SELECT v1.key as key1, v2.key as key2 +FROM view4 v1 join view4 v2; +SELECT * FROM view5; +DESCRIBE view5; + +-- verify that column name and comment in DDL portion +-- overrides column alias in SELECT +CREATE VIEW view6(valoo COMMENT 'I cannot spell') AS +SELECT upper(value) as blarg FROM src WHERE key=86; +DESCRIBE view6; + +-- verify that ORDER BY and LIMIT are both supported in view def +CREATE VIEW view7 AS +SELECT * FROM src +WHERE key > 80 AND key < 100 +ORDER BY key, value +LIMIT 10; + +SELECT * FROM view7; + +-- top-level ORDER BY should override the one inside the view +-- (however, the inside ORDER BY should still influence the evaluation +-- of the limit) +SELECT * FROM view7 ORDER BY key DESC, value; + +-- top-level LIMIT should override if lower +SELECT * FROM view7 LIMIT 5; + +-- but not if higher +SELECT * FROM view7 LIMIT 20; + +-- test usage of a function within a view +CREATE TEMPORARY FUNCTION test_translate AS +'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestTranslate'; +CREATE VIEW view8(c) AS +SELECT test_translate('abc', 'a', 'b') +FROM table1; +DESCRIBE EXTENDED view8; +DESCRIBE FORMATTED view8; +SELECT * FROM view8; + +-- test usage of a UDAF within a view +CREATE TEMPORARY FUNCTION test_max AS +'org.apache.hadoop.hive.ql.udf.UDAFTestMax'; +set hive.map.aggr=false; +-- disable map-side aggregation +CREATE VIEW view9(m) AS +SELECT test_max(length(value)) +FROM src; +DESCRIBE EXTENDED view9; +DESCRIBE FORMATTED view9; +SELECT * FROM view9; +DROP VIEW view9; +set hive.map.aggr=true; +-- enable map-side aggregation +CREATE VIEW view9(m) AS +SELECT test_max(length(value)) +FROM src; +DESCRIBE EXTENDED view9; +DESCRIBE FORMATTED view9; +SELECT * FROM view9; + +-- test usage of a subselect within a view +CREATE VIEW view10 AS +SELECT slurp.* FROM (SELECT * FROM src WHERE key=86) slurp; +DESCRIBE EXTENDED view10; +DESCRIBE FORMATTED view10; +SELECT * FROM view10; + +-- test usage of a UDTF within a view +CREATE TEMPORARY FUNCTION test_explode AS +'org.apache.hadoop.hive.ql.udf.generic.GenericUDTFExplode'; +CREATE VIEW view11 AS +SELECT test_explode(array(1,2,3)) AS (boom) +FROM table1; +DESCRIBE EXTENDED view11; +DESCRIBE FORMATTED view11; +SELECT * FROM view11; + +-- test usage of LATERAL within a view +CREATE VIEW view12 AS +SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol; +DESCRIBE EXTENDED view12; +DESCRIBE FORMATTED view12; +SELECT * FROM view12 +ORDER BY key ASC, myCol ASC LIMIT 1; + +-- test usage of LATERAL with a view as the LHS +SELECT * FROM view2 LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +ORDER BY key ASC, myCol ASC LIMIT 1; + +-- test usage of TABLESAMPLE within a view +CREATE VIEW view13 AS +SELECT s.key +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 ON key) s; +DESCRIBE EXTENDED view13; +DESCRIBE FORMATTED view13; +SELECT * FROM view13 +ORDER BY key LIMIT 12; + +-- test usage of JOIN+UNION+AGG all within same view +CREATE VIEW view14 AS +SELECT unionsrc1.key as k1, unionsrc1.value as v1, + unionsrc2.key as k2, unionsrc2.value as v2 +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2 where s2.key < 10) unionsrc1 +JOIN + (select 'tst1' as key, cast(count(1) as string) as value from src s3 + UNION ALL + select s4.key as key, s4.value as value from src s4 where s4.key < 10) unionsrc2 +ON (unionsrc1.key = unionsrc2.key); +DESCRIBE EXTENDED view14; +DESCRIBE FORMATTED view14; +SELECT * FROM view14 +ORDER BY k1; + +-- test usage of GROUP BY within view +CREATE VIEW view15 AS +SELECT key,COUNT(value) AS value_count +FROM src +GROUP BY key; +DESCRIBE EXTENDED view15; +DESCRIBE FORMATTED view15; +SELECT * FROM view15 +ORDER BY value_count DESC, key +LIMIT 10; + +-- test usage of DISTINCT within view +CREATE VIEW view16 AS +SELECT DISTINCT value +FROM src; +DESCRIBE EXTENDED view16; +DESCRIBE FORMATTED view16; +SELECT * FROM view16 +ORDER BY value +LIMIT 10; + +-- HIVE-2133: DROP TABLE IF EXISTS should ignore a matching view name +DROP TABLE IF EXISTS view16; +DESCRIBE view16; + +-- Likewise, DROP VIEW IF EXISTS should ignore a matching table name +DROP VIEW IF EXISTS table1; +DESCRIBE table1; + +-- this should work since currently we don't track view->table +-- dependencies for implementing RESTRICT + + +DROP VIEW view1; +DROP VIEW view2; +DROP VIEW view3; +DROP VIEW view4; +DROP VIEW view5; +DROP VIEW view6; +DROP VIEW view7; +DROP VIEW view8; +DROP VIEW view9; +DROP VIEW view10; +DROP VIEW view11; +DROP VIEW view12; +DROP VIEW view13; +DROP VIEW view14; +DROP VIEW view15; +DROP VIEW view16; +DROP TEMPORARY FUNCTION test_translate; +DROP TEMPORARY FUNCTION test_max; +DROP TEMPORARY FUNCTION test_explode; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_view_partitioned.q b/src/test/hive/ql/src/test/queries/clientpositive/create_view_partitioned.q new file mode 100644 index 0000000000000..5e30f91f4eb78 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_view_partitioned.q @@ -0,0 +1,83 @@ +DROP VIEW vp1; +DROP VIEW vp2; +DROP VIEW vp3; + +-- test partitioned view definition +-- (underlying table is not actually partitioned) +CREATE VIEW vp1 +PARTITIONED ON (value) +AS +SELECT key, value +FROM src +WHERE key=86; +DESCRIBE EXTENDED vp1; +DESCRIBE FORMATTED vp1; + +SELECT * FROM vp1; + +SELECT key FROM vp1; + +SELECT value FROM vp1; + +ALTER VIEW vp1 +ADD PARTITION (value='val_86') PARTITION (value='val_xyz'); + +-- should work since we use IF NOT EXISTS +ALTER VIEW vp1 +ADD IF NOT EXISTS PARTITION (value='val_xyz'); + +SHOW PARTITIONS vp1; + +SHOW PARTITIONS vp1 PARTITION(value='val_86'); + +SHOW TABLE EXTENDED LIKE vp1; + +SHOW TABLE EXTENDED LIKE vp1 PARTITION(value='val_86'); + +ALTER VIEW vp1 +DROP PARTITION (value='val_xyz'); + +SET hive.exec.drop.ignorenonexistent=false; + +-- should work since we use IF EXISTS +ALTER VIEW vp1 +DROP IF EXISTS PARTITION (value='val_xyz'); + +SHOW PARTITIONS vp1; + +SET hive.mapred.mode=strict; + +-- Even though no partition predicate is specified in the next query, +-- the WHERE clause inside of the view should satisfy strict mode. +-- In other words, strict only applies to underlying tables +-- (regardless of whether or not the view is partitioned). +SELECT * FROM vp1; + +SET hive.mapred.mode=nonstrict; + +-- test a partitioned view on top of an underlying partitioned table, +-- but with only a suffix of the partitioning columns +CREATE VIEW vp2 +PARTITIONED ON (hr) +AS SELECT * FROM srcpart WHERE key < 10; +DESCRIBE FORMATTED vp2; + +ALTER VIEW vp2 ADD PARTITION (hr='11') PARTITION (hr='12'); +SELECT key FROM vp2 WHERE hr='12' ORDER BY key; + +-- test a partitioned view where the PARTITIONED ON clause references +-- an imposed column name +CREATE VIEW vp3(k,v) +PARTITIONED ON (v) +AS +SELECT key, value +FROM src +WHERE key=86; +DESCRIBE FORMATTED vp3; + +ALTER VIEW vp3 +ADD PARTITION (v='val_86'); + +DROP VIEW vp1; +DROP VIEW vp2; +DROP VIEW vp3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_view_translate.q b/src/test/hive/ql/src/test/queries/clientpositive/create_view_translate.q new file mode 100644 index 0000000000000..2199750f42f7e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/create_view_translate.q @@ -0,0 +1,13 @@ +drop view if exists v; +drop view if exists w; + +create view v as select cast(key as string) from src; +describe formatted v; + +create view w as select key, value from ( + select key, value from src +) a; +describe formatted w; + +drop view v; +drop view w; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/cross_join.q b/src/test/hive/ql/src/test/queries/clientpositive/cross_join.q new file mode 100644 index 0000000000000..1f888dd47b9e9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/cross_join.q @@ -0,0 +1,6 @@ +-- current +explain select src.key from src join src src2; +-- ansi cross join +explain select src.key from src cross join src src2; +-- appending condition is allowed +explain select src.key from src cross join src src2 on src.key=src2.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ct_case_insensitive.q b/src/test/hive/ql/src/test/queries/clientpositive/ct_case_insensitive.q new file mode 100644 index 0000000000000..00e9722087414 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ct_case_insensitive.q @@ -0,0 +1,3 @@ +CREATE TABLE tmp_pyang_bucket3 (userId INT) CLUSTERED BY (userid) INTO 32 BUCKETS; +DROP TABLE tmp_pyang_bucket3; +CREATE TABLE tmp_pyang_bucket3 (userId INT) CLUSTERED BY (userid) SORTED BY (USERID) INTO 32 BUCKETS; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas.q b/src/test/hive/ql/src/test/queries/clientpositive/ctas.q new file mode 100644 index 0000000000000..e595904b412fa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ctas.q @@ -0,0 +1,69 @@ +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +create table nzhang_Tmp(a int, b string); +select * from nzhang_Tmp; + +explain create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10; + +create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10; + +select * from nzhang_CTAS1; + +describe formatted nzhang_CTAS1; + + +explain create table nzhang_ctas2 as select * from src sort by key, value limit 10; + +create table nzhang_ctas2 as select * from src sort by key, value limit 10; + +select * from nzhang_ctas2; + +describe formatted nzhang_CTAS2; + + +explain create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb from src sort by half_key, conb limit 10; + +create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb from src sort by half_key, conb limit 10; + +select * from nzhang_ctas3; + +describe formatted nzhang_CTAS3; + + +explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2; + +create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2; + +select * from nzhang_ctas3; + +describe formatted nzhang_CTAS3; + + +explain create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10; + +create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10; + +select * from nzhang_ctas4; + +describe formatted nzhang_CTAS4; + +explain extended 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; + +set mapreduce.framework.name=yarn; +set mapreduce.jobtracker.address=localhost:58; +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; +create table nzhang_ctas7 as select key, `to` from nzhang_ctas6; + + + + + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas_colname.q b/src/test/hive/ql/src/test/queries/clientpositive/ctas_colname.q new file mode 100644 index 0000000000000..80b8b4f06487d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ctas_colname.q @@ -0,0 +1,48 @@ +-- HIVE-4392, column aliases from expressionRR (GBY, etc.) are not valid name for table +-- use internal name as column name instead + +-- group by +explain +create table summary as select *, sum(key), count(value) from src; +create table summary as select *, sum(key), count(value) from src; +describe formatted summary; +select * from summary order by `_col0`, `_col1`, `_c1`, `_c2`; + +-- window functions +explain +create table x4 as select *, rank() over(partition by key order by value) as rr from src1; +create table x4 as select *, rank() over(partition by key order by value) as rr from src1; +describe formatted x4; +select * from x4 order by key, value, rr; + +explain +create table x5 as select *, lead(key,1) over(partition by key order by value) as lead1 from src limit 20; +create table x5 as select *, lead(key,1) over(partition by key order by value) as lead1 from src limit 20; +describe formatted x5; +select * from x5 order by key, value, lead1; + +-- sub queries +explain +create table x6 as select * from (select *, max(key) from src1) a; +create table x6 as select * from (select *, max(key) from src1) a; +describe formatted x6; +select * from x6 order by `_col0`, `_c1`; + +explain +create table x7 as select * from (select * from src group by key) a; +create table x7 as select * from (select * from src group by key) a; +describe formatted x7; +select * from x7 order by `_col0`; + +explain +create table x8 as select * from (select * from src group by key having key < 9) a; +create table x8 as select * from (select * from src group by key having key < 9) a; +describe formatted x8; +select * from x8 order by `_col0`; + +explain +create table x9 as select * from (select max(value),key from src group by key having key < 9 AND max(value) IS NOT NULL) a; +create table x9 as select * from (select max(value),key from src group by key having key < 9 AND max(value) IS NOT NULL) a; +describe formatted x9; +select * from x9 order by key, `_c0`; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas_date.q b/src/test/hive/ql/src/test/queries/clientpositive/ctas_date.q new file mode 100644 index 0000000000000..03af69fd3bddf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ctas_date.q @@ -0,0 +1,26 @@ +drop table ctas_date_1; +drop table ctas_date_2; +drop view ctas_date_3; +drop view ctas_date_4; + +create table ctas_date_1 (key int, value string, dd date); +insert overwrite table ctas_date_1 + select key, value, date '2012-01-01' from src sort by key, value limit 5; + +-- create table as with date column +create table ctas_date_2 as select key, value, dd, date '1980-12-12' from ctas_date_1; + +-- view with date column +create view ctas_date_3 as select * from ctas_date_2 where dd > date '2000-01-01'; +create view ctas_date_4 as select * from ctas_date_2 where dd < date '2000-01-01'; + +select key, value, dd, date '1980-12-12' from ctas_date_1; +select * from ctas_date_2; +select * from ctas_date_3; +select count(*) from ctas_date_4; + + +drop table ctas_date_1; +drop table ctas_date_2; +drop view ctas_date_3; +drop view ctas_date_4; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas_hadoop20.q b/src/test/hive/ql/src/test/queries/clientpositive/ctas_hadoop20.q new file mode 100644 index 0000000000000..4961b971dbdec --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ctas_hadoop20.q @@ -0,0 +1,68 @@ +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +create table nzhang_Tmp(a int, b string); +select * from nzhang_Tmp; + +explain create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10; + +create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10; + +select * from nzhang_CTAS1; + +describe formatted nzhang_CTAS1; + + +explain create table nzhang_ctas2 as select * from src sort by key, value limit 10; + +create table nzhang_ctas2 as select * from src sort by key, value limit 10; + +select * from nzhang_ctas2; + +describe formatted nzhang_CTAS2; + + +explain create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb from src sort by half_key, conb limit 10; + +create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb from src sort by half_key, conb limit 10; + +select * from nzhang_ctas3; + +describe formatted nzhang_CTAS3; + + +explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2; + +create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2; + +select * from nzhang_ctas3; + +describe formatted nzhang_CTAS3; + + +explain create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10; + +create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10; + +select * from nzhang_ctas4; + +describe formatted nzhang_CTAS4; + +explain extended 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; + +set mapred.job.tracker=localhost:58; +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; +create table nzhang_ctas7 as select key, `to` from nzhang_ctas6; + + + + + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas_uses_database_location.q b/src/test/hive/ql/src/test/queries/clientpositive/ctas_uses_database_location.q new file mode 100644 index 0000000000000..86c940a25b291 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ctas_uses_database_location.q @@ -0,0 +1,14 @@ +set hive.metastore.warehouse.dir=invalid_scheme://${system:test.tmp.dir}; + +-- Tests that CTAS queries in non-default databases use the location of the database +-- not the hive.metastore.warehouse.dir for intermediate files (FileSinkOperator output). +-- If hive.metastore.warehouse.dir were used this would fail because the scheme is invalid. + +CREATE DATABASE db1 +LOCATION 'pfile://${system:test.tmp.dir}/db1'; + +USE db1; +EXPLAIN CREATE TABLE table_db1 AS SELECT * FROM default.src; +CREATE TABLE table_db1 AS SELECT * FROM default.src; + +DESCRIBE FORMATTED table_db1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas_varchar.q b/src/test/hive/ql/src/test/queries/clientpositive/ctas_varchar.q new file mode 100644 index 0000000000000..f646b36c8a6f9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ctas_varchar.q @@ -0,0 +1,22 @@ +drop table ctas_varchar_1; +drop table ctas_varchar_2; +drop view ctas_varchar_3; + +create table ctas_varchar_1 (key varchar(10), value string); +insert overwrite table ctas_varchar_1 + select key, value from src sort by key, value limit 5; + +-- create table as with varchar column +create table ctas_varchar_2 as select key, value from ctas_varchar_1; + +-- view with varchar column +create view ctas_varchar_3 as select key, value from ctas_varchar_2; + +select key, value from ctas_varchar_1; +select * from ctas_varchar_2; +select * from ctas_varchar_3; + + +drop table ctas_varchar_1; +drop table ctas_varchar_2; +drop view ctas_varchar_3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/custom_input_output_format.q b/src/test/hive/ql/src/test/queries/clientpositive/custom_input_output_format.q new file mode 100644 index 0000000000000..ff5e86dc5a6b4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/custom_input_output_format.q @@ -0,0 +1,6 @@ +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'; +INSERT OVERWRITE TABLE src1_rot13_iof SELECT * FROM src1; +SELECT * FROM src1_rot13_iof; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/database.q b/src/test/hive/ql/src/test/queries/clientpositive/database.q new file mode 100644 index 0000000000000..9140a42b650e8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/database.q @@ -0,0 +1,216 @@ +set hive.support.concurrency = true; + +SHOW DATABASES; + +-- CREATE with comment +CREATE DATABASE test_db COMMENT 'Hive test database'; +SHOW DATABASES; + +-- CREATE INE already exists +CREATE DATABASE IF NOT EXISTS test_db; +SHOW DATABASES; + +-- SHOW DATABASES synonym +SHOW SCHEMAS; + +-- DROP +DROP DATABASE test_db; +SHOW DATABASES; + +-- CREATE INE doesn't exist +CREATE DATABASE IF NOT EXISTS test_db COMMENT 'Hive test database'; +SHOW DATABASES; + +-- DROP IE exists +DROP DATABASE IF EXISTS test_db; +SHOW DATABASES; + +-- DROP IE doesn't exist +DROP DATABASE IF EXISTS test_db; + +-- SHOW +CREATE DATABASE test_db; +SHOW DATABASES; + +-- SHOW pattern +SHOW DATABASES LIKE 'test*'; + +-- SHOW pattern +SHOW DATABASES LIKE '*ef*'; + + +USE test_db; +SHOW DATABASES; + +-- CREATE table in non-default DB +CREATE TABLE test_table (col1 STRING) STORED AS TEXTFILE; +SHOW TABLES; + +-- DESCRIBE table in non-default DB +DESCRIBE test_table; + +-- DESCRIBE EXTENDED in non-default DB +DESCRIBE EXTENDED test_table; + +-- CREATE LIKE in non-default DB +CREATE TABLE test_table_like LIKE test_table; +SHOW TABLES; +DESCRIBE EXTENDED test_table_like; + +-- LOAD and SELECT +LOAD DATA LOCAL INPATH '../data/files/test.dat' +OVERWRITE INTO TABLE test_table; +SELECT * FROM test_table; + +-- DROP and CREATE w/o LOAD +DROP TABLE test_table; +SHOW TABLES; + +CREATE TABLE test_table (col1 STRING) STORED AS TEXTFILE; +SHOW TABLES; + +SELECT * FROM test_table; + +-- CREATE table that already exists in DEFAULT +USE test_db; +CREATE TABLE src (col1 STRING) STORED AS TEXTFILE; +SHOW TABLES; + +SELECT * FROM src LIMIT 10; + +USE default; +SELECT * FROM src LIMIT 10; + +-- DROP DATABASE +USE test_db; + +DROP TABLE src; +DROP TABLE test_table; +DROP TABLE test_table_like; +SHOW TABLES; + +USE default; +DROP DATABASE test_db; +SHOW DATABASES; + +-- DROP EMPTY DATABASE CASCADE +CREATE DATABASE to_drop_db1; +SHOW DATABASES; +USE default; +DROP DATABASE to_drop_db1 CASCADE; +SHOW DATABASES; + +-- DROP NON-EMPTY DATABASE CASCADE +CREATE DATABASE to_drop_db2; +SHOW DATABASES; +USE to_drop_db2; +CREATE TABLE temp_tbl (c STRING); +CREATE TABLE temp_tbl2 LIKE temp_tbl; +INSERT OVERWRITE TABLE temp_tbl2 SELECT COUNT(*) FROM temp_tbl; +USE default; +DROP DATABASE to_drop_db2 CASCADE; +SHOW DATABASES; + +-- DROP NON-EMPTY DATABASE CASCADE IF EXISTS +CREATE DATABASE to_drop_db3; +SHOW DATABASES; +USE to_drop_db3; +CREATE TABLE temp_tbl (c STRING); +USE default; +DROP DATABASE IF EXISTS to_drop_db3 CASCADE; +SHOW DATABASES; + +-- DROP NON-EXISTING DATABASE CASCADE IF EXISTS +DROP DATABASE IF EXISTS non_exists_db3 CASCADE; +SHOW DATABASES; + +-- DROP NON-EXISTING DATABASE RESTRICT IF EXISTS +DROP DATABASE IF EXISTS non_exists_db3 RESTRICT; + +-- DROP EMPTY DATABASE RESTRICT +CREATE DATABASE to_drop_db4; +SHOW DATABASES; +DROP DATABASE to_drop_db4 RESTRICT; +SHOW DATABASES; + + +-- +-- Canonical Name Tests +-- + +CREATE DATABASE db1; +CREATE DATABASE db2; + +-- CREATE foreign table +CREATE TABLE db1.src(key STRING, value STRING) +STORED AS TEXTFILE; + +-- LOAD into foreign table +LOAD DATA LOCAL INPATH '../data/files/kv1.txt' +OVERWRITE INTO TABLE db1.src; + +-- SELECT from foreign table +SELECT * FROM db1.src; + +-- CREATE Partitioned foreign table +CREATE TABLE db1.srcpart(key STRING, value STRING) +PARTITIONED BY (ds STRING, hr STRING) +STORED AS TEXTFILE; + +-- LOAD data into Partitioned foreign table +LOAD DATA LOCAL INPATH '../data/files/kv1.txt' +OVERWRITE INTO TABLE db1.srcpart +PARTITION (ds='2008-04-08', hr='11'); + +-- SELECT from Partitioned foreign table +SELECT key, value FROM db1.srcpart +WHERE key < 100 AND ds='2008-04-08' AND hr='11'; + +-- SELECT JOINed product of two foreign tables +USE db2; +SELECT a.* FROM db1.src a JOIN default.src1 b +ON (a.key = b.key); + +-- CREATE TABLE AS SELECT from foreign table +CREATE TABLE conflict_name AS +SELECT value FROM default.src WHERE key = 66; + +-- CREATE foreign table +CREATE TABLE db1.conflict_name AS +SELECT value FROM db1.src WHERE key = 8; + +-- query tables with the same names in different DBs +SELECT * FROM ( + SELECT value FROM db1.conflict_name +UNION ALL + SELECT value FROM conflict_name +) subq ORDER BY value; + +USE default; +SELECT * FROM ( + SELECT value FROM db1.conflict_name +UNION ALL + SELECT value FROM db2.conflict_name +) subq ORDER BY value; + +-- TABLESAMPLES +CREATE TABLE bucketized_src (key INT, value STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; + +INSERT OVERWRITE TABLE bucketized_src +SELECT key, value FROM src WHERE key=66; + +SELECT key FROM bucketized_src TABLESAMPLE(BUCKET 1 out of 1); + +-- CREATE TABLE LIKE +CREATE TABLE db2.src1 LIKE default.src; + +USE db2; +DESC EXTENDED src1; + +-- character escaping +SELECT key FROM `default`.src ORDER BY key LIMIT 1; +SELECT key FROM `default`.`src` ORDER BY key LIMIT 1; +SELECT key FROM default.`src` ORDER BY key LIMIT 1; + +USE default; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/database_drop.q b/src/test/hive/ql/src/test/queries/clientpositive/database_drop.q new file mode 100644 index 0000000000000..4e17c7ad5ac10 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/database_drop.q @@ -0,0 +1,81 @@ +-- create database with multiple tables, indexes and views. +-- Use both partitioned and non-partitioned tables, as well as +-- tables and indexes with specific storage locations +-- verify the drop the database with cascade works and that the directories +-- outside the database's default storage are removed as part of the drop + +CREATE DATABASE db5; +SHOW DATABASES; +USE db5; + +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; +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; + +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; +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'); +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; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/part_tab2; +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'); +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; +ALTER INDEX idx4 ON part_tab2 PARTITION (ds='2009-04-09') REBUILD; + + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/part_tab3; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/part_tab3_p1; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/part_tab3_idx5; +-- add a partitioned table, index and view with a different storage location +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'); +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; +ALTER INDEX idx5 ON part_tab3 PARTITION (ds='2009-04-09') REBUILD; + + + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/extab1; +dfs -touchz ${system:test.tmp.dir}/dbcascade/extab1/file1.txt; +-- add an external table +CREATE EXTERNAL TABLE extab1(id INT, name STRING) ROW FORMAT + DELIMITED FIELDS TERMINATED BY '' + LINES TERMINATED BY '\n' + STORED AS TEXTFILE + LOCATION 'file:${system:test.tmp.dir}/dbcascade/extab1'; + +-- drop the database with cascade +DROP DATABASE db5 CASCADE; + +dfs -test -d ${system:test.tmp.dir}/dbcascade/extab1; +dfs -rmr ${system:test.tmp.dir}/dbcascade; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/database_location.q b/src/test/hive/ql/src/test/queries/clientpositive/database_location.q new file mode 100644 index 0000000000000..e131b5ff02e67 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/database_location.q @@ -0,0 +1,20 @@ +CREATE DATABASE db1; +DESCRIBE DATABASE EXTENDED db1; + +USE db1; +CREATE TABLE table_db1 (name STRING, value INT); + +DESCRIBE FORMATTED table_db1; +SHOW TABLES; + +CREATE DATABASE db2 +COMMENT 'database 2' +LOCATION '${hiveconf:hive.metastore.warehouse.dir}/db2'; + +DESCRIBE DATABASE EXTENDED db2; + +USE db2; +CREATE TABLE table_db2 (name STRING, value INT); + +DESCRIBE FORMATTED table_db2; +SHOW TABLES; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/database_properties.q b/src/test/hive/ql/src/test/queries/clientpositive/database_properties.q new file mode 100644 index 0000000000000..7f3b239d1836d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/database_properties.q @@ -0,0 +1,26 @@ +set datanucleus.cache.collections=false; +set datanucleus.cache.collections.lazy=false; + +create database db1; + +show databases; + +create database db2 with dbproperties ( + 'mapred.jobtracker.url'='http://my.jobtracker.com:53000', + 'hive.warehouse.dir' = '/user/hive/warehouse', + 'mapred.scratch.dir' = 'hdfs://tmp.dfs.com:50029/tmp'); + +describe database db2; + +describe database extended db2; + + +set datanucleus.cache.collections=false; +set datanucleus.cache.collections.lazy=false; + +alter database db2 set dbproperties ( + 'new.property' = 'some new props', + 'hive.warehouse.dir' = 'new/warehouse/dir'); + +describe database extended db2; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_1.q b/src/test/hive/ql/src/test/queries/clientpositive/date_1.q new file mode 100644 index 0000000000000..a2322fc1e45a0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/date_1.q @@ -0,0 +1,87 @@ +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 * 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 * 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 * from date_1 limit 1; +select d, count(d) from date_1 group by d; + +-- Valid casts +select + cast('2012-01-01' as string), + cast(d as string), + cast(d as timestamp), + cast(cast(d as timestamp) as date), + cast(d as date) +from date_1 limit 1; + +-- Invalid casts. +select + cast(d as boolean), + cast(d as tinyint), + cast(d as smallint), + cast(d as int), + cast(d as bigint), + cast(d as float), + cast(d as double) +from date_1 limit 1; + +-- These comparisons should all be true +select + date '2011-01-01' = date '2011-01-01', + unix_timestamp(date '2011-01-01') = unix_timestamp(date '2011-01-01'), + unix_timestamp(date '2011-01-01') = unix_timestamp(cast(date '2011-01-01' as timestamp)), + unix_timestamp(date '2011-01-01') = unix_timestamp(cast(cast('2011-01-01 12:13:14' as timestamp) as date)), + unix_timestamp(date '2011-01-01') < unix_timestamp(cast('2011-01-01 00:00:01' as timestamp)), + unix_timestamp(date '2011-01-01') = unix_timestamp(cast('2011-01-01 00:00:00' as timestamp)), + unix_timestamp(date '2011-01-01') > unix_timestamp(cast('2010-12-31 23:59:59' as timestamp)), + date '2011-01-01' = cast(timestamp('2011-01-01 23:24:25') as date), + '2011-01-01' = cast(d as string), + '2011-01-01' = cast(date '2011-01-01' as string) +from date_1 limit 1; + +select + date('2001-01-28'), + date('2001-02-28'), + date('2001-03-28'), + date('2001-04-28'), + date('2001-05-28'), + date('2001-06-28'), + date('2001-07-28'), + date('2001-08-28'), + date('2001-09-28'), + date('2001-10-28'), + date('2001-11-28'), + date('2001-12-28') +from date_1 limit 1; + +select + unix_timestamp(date('2001-01-28')) = unix_timestamp(cast('2001-01-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-02-28')) = unix_timestamp(cast('2001-02-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-03-28')) = unix_timestamp(cast('2001-03-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-04-28')) = unix_timestamp(cast('2001-04-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-05-28')) = unix_timestamp(cast('2001-05-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-06-28')) = unix_timestamp(cast('2001-06-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-07-28')) = unix_timestamp(cast('2001-07-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-08-28')) = unix_timestamp(cast('2001-08-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-09-28')) = unix_timestamp(cast('2001-09-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-10-28')) = unix_timestamp(cast('2001-10-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-11-28')) = unix_timestamp(cast('2001-11-28 0:0:0' as timestamp)), + unix_timestamp(date('2001-12-28')) = unix_timestamp(cast('2001-12-28 0:0:0' as timestamp)) +from date_1 limit 1; + +drop table date_1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_2.q b/src/test/hive/ql/src/test/queries/clientpositive/date_2.q new file mode 100644 index 0000000000000..0821e012ba26b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/date_2.q @@ -0,0 +1,18 @@ +drop table if exists date_2; + +create table date_2 ( + 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_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; + +select fl_date, count(*) from date_2 group by fl_date; + +drop table date_2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_3.q b/src/test/hive/ql/src/test/queries/clientpositive/date_3.q new file mode 100644 index 0000000000000..be251484459b6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/date_3.q @@ -0,0 +1,14 @@ +drop table date_3; + +create table date_3 ( + c1 int +); + +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 * from date_3; + +drop table date_3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_4.q b/src/test/hive/ql/src/test/queries/clientpositive/date_4.q new file mode 100644 index 0000000000000..4801a79b8795b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/date_4.q @@ -0,0 +1,11 @@ +drop table date_4; + +create table date_4 (d date); +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 d, date '2011-01-01' from date_4 limit 1; + +drop table date_4; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_comparison.q b/src/test/hive/ql/src/test/queries/clientpositive/date_comparison.q new file mode 100644 index 0000000000000..bdcb6c1b6e447 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/date_comparison.q @@ -0,0 +1,38 @@ +-- Comparisons against same value +select cast('2011-05-06' as date) > + cast('2011-05-06' as date) from src limit 1; + +select cast('2011-05-06' as date) < + cast('2011-05-06' as date) from src limit 1; + +select cast('2011-05-06' as date) = + cast('2011-05-06' as date) from src limit 1; + +select cast('2011-05-06' as date) <> + cast('2011-05-06' as date) from src limit 1; + +select cast('2011-05-06' as date) >= + cast('2011-05-06' as date) from src limit 1; + +select cast('2011-05-06' as date) <= + cast('2011-05-06' as date) from src limit 1; + +-- Now try with differing values +select cast('2011-05-05' as date) > + cast('2011-05-06' as date) from src limit 1; + +select cast('2011-05-05' as date) < + cast('2011-05-06' as date) from src limit 1; + +select cast('2011-05-05' as date) = + cast('2011-05-06' as date) from src limit 1; + +select cast('2011-05-05' as date) <> + cast('2011-05-06' as date) from src limit 1; + +select cast('2011-05-05' as date) >= + cast('2011-05-06' as date) from src limit 1; + +select cast('2011-05-05' as date) <= + cast('2011-05-06' as date) from src limit 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_join1.q b/src/test/hive/ql/src/test/queries/clientpositive/date_join1.q new file mode 100644 index 0000000000000..a5844b76e9ef8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/date_join1.q @@ -0,0 +1,19 @@ +drop table date_join1; + +create table date_join1 ( + ORIGIN_CITY_NAME string, + DEST_CITY_NAME string, + FL_DATE date, + ARR_DELAY float, + FL_NUM int +); + +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 + from date_join1 t1 + join date_join1 t2 + on (t1.fl_date = t2.fl_date); + +drop table date_join1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_serde.q b/src/test/hive/ql/src/test/queries/clientpositive/date_serde.q new file mode 100644 index 0000000000000..ffc06d270d53a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/date_serde.q @@ -0,0 +1,112 @@ +drop table if exists date_serde_regex; +drop table date_serde_lb; +drop table date_serde_ls; +drop table date_serde_c; +drop table date_serde_lbc; +drop table date_serde_orc; + + +-- +-- RegexSerDe +-- +create table date_serde_regex ( + ORIGIN_CITY_NAME string, + DEST_CITY_NAME string, + FL_DATE date, + ARR_DELAY float, + FL_NUM int +) +row format serde 'org.apache.hadoop.hive.serde2.RegexSerDe' +with serdeproperties ( + "input.regex" = "([^]*)([^]*)([^]*)([^]*)([0-9]*)" +) +stored as textfile; + +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; + +-- +-- LazyBinary +-- +create table date_serde_lb ( + c1 date, + c2 int +); +alter table date_serde_lb set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe'; + +insert overwrite table date_serde_lb + select fl_date, fl_num from date_serde_regex limit 1; + +select * from date_serde_lb; +select c1, sum(c2) from date_serde_lb group by c1; + +-- +-- LazySimple +-- +create table date_serde_ls ( + c1 date, + c2 int +); +alter table date_serde_ls set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; + +insert overwrite table date_serde_ls + select c1, c2 from date_serde_lb limit 1; + +select * from date_serde_ls; +select c1, sum(c2) from date_serde_ls group by c1; + +-- +-- Columnar +-- +create table date_serde_c ( + c1 date, + c2 int +) stored as rcfile; +alter table date_serde_c set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; + +insert overwrite table date_serde_c + select c1, c2 from date_serde_ls limit 1; + +select * from date_serde_c; +select c1, sum(c2) from date_serde_c group by c1; + +-- +-- LazyBinaryColumnar +-- +create table date_serde_lbc ( + c1 date, + c2 int +) stored as rcfile; +alter table date_serde_lbc set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; + +insert overwrite table date_serde_lbc + select c1, c2 from date_serde_c limit 1; + +select * from date_serde_lbc; +select c1, sum(c2) from date_serde_lbc group by c1; + +-- +-- ORC +-- +create table date_serde_orc ( + c1 date, + c2 int +) stored as orc; +alter table date_serde_orc set serde 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'; + +insert overwrite table date_serde_orc + select c1, c2 from date_serde_lbc limit 1; + +select * from date_serde_orc; +select c1, sum(c2) from date_serde_orc group by c1; + + + +drop table date_serde_regex; +drop table date_serde_lb; +drop table date_serde_ls; +drop table date_serde_c; +drop table date_serde_lbc; +drop table date_serde_orc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_udf.q b/src/test/hive/ql/src/test/queries/clientpositive/date_udf.q new file mode 100644 index 0000000000000..9696320a85fcd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/date_udf.q @@ -0,0 +1,69 @@ +drop table date_udf; +drop table date_udf_string; +drop table date_udf_flight; + +create table date_udf (d date); +create table date_udf_string (d string); +from src + insert overwrite table date_udf + select '2011-05-06' limit 1 + insert overwrite table date_udf_string + select '2011-05-06' limit 1; + +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; + +-- Test UDFs with date input +select unix_timestamp(d), year(d), month(d), day(d), dayofmonth(d), + weekofyear(d), to_date(d) + from date_udf; + +select date_add(d, 5), date_sub(d, 10) + from date_udf; + +select datediff(d, d), datediff(d, '2002-03-21'), datediff('2002-03-21', d), + datediff(cast ('2002-03-21 00:00:00' as timestamp), d), + datediff(d, cast ('2002-03-21 00:00:00' as timestamp)) + from date_udf; + +-- Test UDFs with string input +select unix_timestamp(d), year(d), month(d), day(d), dayofmonth(d), + weekofyear(d), to_date(d) + from date_udf_string; + +select date_add(d, 5), date_sub(d, 10) from date_udf_string; + +select datediff(d, d), datediff(d, '2002-03-21'), datediff('2002-03-21', d), + datediff('2002-03-21 00:00:00', d), + datediff(d, '2002-03-21 00:00:00') + from date_udf_string; + +select + to_utc_timestamp(date '1970-01-01', 'America/Los_Angeles'), + from_utc_timestamp(date '1970-01-01', 'America/Los_Angeles'), + to_utc_timestamp(date '2013-06-19', 'America/Los_Angeles'), + from_utc_timestamp(date '2013-06-19', 'America/Los_Angeles') + from date_udf; + +-- should all be true +select + to_utc_timestamp(date '1970-01-01', 'America/Los_Angeles') = to_utc_timestamp(timestamp('1970-01-01 00:00:00'), 'America/Los_Angeles'), + from_utc_timestamp(date '1970-01-01', 'America/Los_Angeles') = from_utc_timestamp(timestamp('1970-01-01 00:00:00'), 'America/Los_Angeles'), + to_utc_timestamp(date '2013-06-19', 'America/Los_Angeles') = to_utc_timestamp(timestamp('2013-06-19 00:00:00'), 'America/Los_Angeles'), + from_utc_timestamp(date '2013-06-19', 'America/Los_Angeles') = from_utc_timestamp(timestamp('2013-06-19 00:00:00'), 'America/Los_Angeles') + from date_udf; + +-- Aggregation functions (min/max) +select min(fl_date) from date_udf_flight; +select max(fl_date) from date_udf_flight; + + +drop table date_udf; +drop table date_udf_string; +drop table date_udf_flight; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ddltime.q b/src/test/hive/ql/src/test/queries/clientpositive/ddltime.q new file mode 100644 index 0000000000000..3eead6f29dfa6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ddltime.q @@ -0,0 +1,45 @@ +create table T1 like src; + +desc extended T1; + +!sleep 1; +insert overwrite table T1 select * from src; + +desc extended T1; + +!sleep 1; + +insert overwrite table T1 select /*+ HOLD_DDLTIME*/ * from src; + +desc extended T1; + +!sleep 1; + +insert overwrite table T1 select * from src; + +desc extended T1; + + + +create table if not exists T2 like srcpart; +desc extended T2; + +!sleep 1; + +insert overwrite table T2 partition (ds = '2010-06-21', hr = '1') select key, value from src where key > 10; + +desc extended T2 partition (ds = '2010-06-21', hr = '1'); + +!sleep 1; + +insert overwrite table T2 partition (ds = '2010-06-21', hr='1') select /*+ HOLD_DDLTIME */ key, value from src where key > 10; + +desc extended T2 partition (ds = '2010-06-21', hr = '1'); + +!sleep 1; + +insert overwrite table T2 partition (ds='2010-06-01', hr='1') select key, value from src where key > 10; + +desc extended T2 partition(ds='2010-06-01', hr='1'); + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_1.q b/src/test/hive/ql/src/test/queries/clientpositive/decimal_1.q new file mode 100644 index 0000000000000..6c689e188a908 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/decimal_1.q @@ -0,0 +1,18 @@ +drop table decimal_1; + +create table decimal_1 (t decimal); +alter table decimal_1 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; + +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; + +drop table decimal_1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_2.q b/src/test/hive/ql/src/test/queries/clientpositive/decimal_2.q new file mode 100644 index 0000000000000..4890618a0dc32 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/decimal_2.q @@ -0,0 +1,40 @@ +drop table decimal_2; + +create table decimal_2 (t decimal); +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(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; + +insert overwrite table decimal_2 + select cast('3404045.5044003' as decimal) from src limit 1; + +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(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; +drop table decimal_2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_3.q b/src/test/hive/ql/src/test/queries/clientpositive/decimal_3.q new file mode 100644 index 0000000000000..28211e3f14cc1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/decimal_3.q @@ -0,0 +1,28 @@ +DROP TABLE IF EXISTS DECIMAL_3; + +CREATE TABLE DECIMAL_3(key decimal, value int) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_3; + +SELECT * FROM DECIMAL_3 ORDER BY key, value; + +SELECT * FROM DECIMAL_3 ORDER BY key DESC, value DESC; + +SELECT * FROM DECIMAL_3 ORDER BY key, value; + +SELECT DISTINCT key FROM DECIMAL_3 ORDER BY key; + +SELECT key, sum(value) FROM DECIMAL_3 GROUP BY key ORDER BY key; + +SELECT value, sum(key) FROM DECIMAL_3 GROUP BY value ORDER BY value; + +SELECT * FROM DECIMAL_3 a JOIN DECIMAL_3 b ON (a.key = b.key) ORDER BY a.key, a.value, b.value; + +SELECT * FROM DECIMAL_3 WHERE key=3.14 ORDER BY key, value; + +SELECT * FROM DECIMAL_3 WHERE key=3.140 ORDER BY key, value; + +DROP TABLE DECIMAL_3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_4.q b/src/test/hive/ql/src/test/queries/clientpositive/decimal_4.q new file mode 100644 index 0000000000000..e8a89c131cd85 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/decimal_4.q @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS DECIMAL_4_1; +DROP TABLE IF EXISTS DECIMAL_4_2; + +CREATE TABLE DECIMAL_4_1(key decimal, value int) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +CREATE TABLE DECIMAL_4_2(key decimal, value decimal) +STORED AS ORC; + +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; + +SELECT * FROM DECIMAL_4_1 ORDER BY key, value; + +SELECT * FROM DECIMAL_4_2 ORDER BY key; + +DROP TABLE DECIMAL_4_1; +DROP TABLE DECIMAL_4_2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_join.q b/src/test/hive/ql/src/test/queries/clientpositive/decimal_join.q new file mode 100644 index 0000000000000..589fc6597dfa4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/decimal_join.q @@ -0,0 +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; + +select * from src_dec a join src_dec b on a.key=b.key+450; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_precision.q b/src/test/hive/ql/src/test/queries/clientpositive/decimal_precision.q new file mode 100644 index 0000000000000..403c2be3fbc10 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/decimal_precision.q @@ -0,0 +1,27 @@ +DROP TABLE IF EXISTS DECIMAL_PRECISION; + +CREATE TABLE DECIMAL_PRECISION(dec decimal) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/kv8.txt' INTO TABLE DECIMAL_PRECISION; + +SELECT * FROM DECIMAL_PRECISION ORDER BY dec; + +SELECT dec, dec + 1, dec - 1 FROM DECIMAL_PRECISION ORDER BY dec; +SELECT dec, dec * 2, dec / 3 FROM DECIMAL_PRECISION ORDER BY dec; +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; + +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 MIN(cast('123456789012345678901234567890.123456789' as decimal)) FROM DECIMAL_PRECISION; +SELECT COUNT(cast('123456789012345678901234567890.123456789' as decimal)) FROM DECIMAL_PRECISION; + +DROP TABLE DECIMAL_PRECISION; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_serde.q b/src/test/hive/ql/src/test/queries/clientpositive/decimal_serde.q new file mode 100644 index 0000000000000..3556807705ae5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/decimal_serde.q @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS DECIMAL_TEXT; +DROP TABLE IF EXISTS DECIMAL_RC; +DROP TABLE IF EXISTS DECIMAL_LAZY_COL; +DROP TABLE IF EXISTS DECIMAL_SEQUENCE; + +CREATE TABLE DECIMAL_TEXT (key decimal, value int) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_TEXT; + +SELECT * FROM DECIMAL_TEXT ORDER BY key, value; + +CREATE TABLE DECIMAL_RC +STORED AS RCFile AS +SELECT * FROM DECIMAL_TEXT; + +CREATE TABLE DECIMAL_LAZY_COL +ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" +STORED AS RCFile AS +SELECT * FROM DECIMAL_RC; + +CREATE TABLE DECIMAL_SEQUENCE +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '\001' +COLLECTION ITEMS TERMINATED BY '\002' +MAP KEYS TERMINATED BY '\003' +STORED AS SEQUENCEFILE AS +SELECT * FROM DECIMAL_LAZY_COL ORDER BY key; + +SELECT * FROM DECIMAL_SEQUENCE ORDER BY key, value; + +DROP TABLE IF EXISTS DECIMAL_TEXT; +DROP TABLE IF EXISTS DECIMAL_RC; +DROP TABLE IF EXISTS DECIMAL_LAZY_COL; +DROP TABLE IF EXISTS DECIMAL_SEQUENCE; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_udf.q b/src/test/hive/ql/src/test/queries/clientpositive/decimal_udf.q new file mode 100644 index 0000000000000..b5ff088d1613a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/decimal_udf.q @@ -0,0 +1,128 @@ +DROP TABLE IF EXISTS DECIMAL_UDF; + +CREATE TABLE DECIMAL_UDF (key decimal, value int) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_UDF; + +-- addition +EXPLAIN SELECT key + key FROM DECIMAL_UDF; +SELECT key + key FROM DECIMAL_UDF; + +EXPLAIN SELECT key + value FROM DECIMAL_UDF; +SELECT key + value FROM DECIMAL_UDF; + +EXPLAIN SELECT key + (value/2) FROM DECIMAL_UDF; +SELECT key + (value/2) FROM DECIMAL_UDF; + +EXPLAIN SELECT key + '1.0' FROM DECIMAL_UDF; +SELECT key + '1.0' FROM DECIMAL_UDF; + +-- substraction +EXPLAIN SELECT key - key FROM DECIMAL_UDF; +SELECT key - key FROM DECIMAL_UDF; + +EXPLAIN SELECT key - value FROM DECIMAL_UDF; +SELECT key - value FROM DECIMAL_UDF; + +EXPLAIN SELECT key - (value/2) FROM DECIMAL_UDF; +SELECT key - (value/2) FROM DECIMAL_UDF; + +EXPLAIN SELECT key - '1.0' FROM DECIMAL_UDF; +SELECT key - '1.0' FROM DECIMAL_UDF; + +-- multiplication +EXPLAIN SELECT key * key FROM DECIMAL_UDF; +SELECT key * key FROM DECIMAL_UDF; + +EXPLAIN SELECT key * value FROM DECIMAL_UDF; +SELECT key * value FROM DECIMAL_UDF; + +EXPLAIN SELECT key * (value/2) FROM DECIMAL_UDF; +SELECT key * (value/2) FROM DECIMAL_UDF; + +EXPLAIN SELECT key * '2.0' FROM DECIMAL_UDF; +SELECT key * '2.0' FROM DECIMAL_UDF; + +-- division +EXPLAIN SELECT key / 0 FROM DECIMAL_UDF limit 1; +SELECT key / 0 FROM DECIMAL_UDF limit 1; + +EXPLAIN SELECT key / NULL FROM DECIMAL_UDF limit 1; +SELECT key / NULL FROM DECIMAL_UDF limit 1; + +EXPLAIN SELECT key / key FROM DECIMAL_UDF WHERE key is not null and key <> 0; +SELECT key / key FROM DECIMAL_UDF WHERE key is not null and key <> 0; + +EXPLAIN SELECT key / value FROM DECIMAL_UDF WHERE value is not null and value <> 0; +SELECT key / value FROM DECIMAL_UDF WHERE value is not null and value <> 0; + +EXPLAIN SELECT key / (value/2) FROM DECIMAL_UDF WHERE value is not null and value <> 0; +SELECT key / (value/2) FROM DECIMAL_UDF WHERE value is not null and value <> 0; + +EXPLAIN SELECT key / '2.0' FROM DECIMAL_UDF; +SELECT key / '2.0' FROM DECIMAL_UDF; + +-- abs +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; + +-- negative +EXPLAIN SELECT -key FROM DECIMAL_UDF; +SELECT -key FROM DECIMAL_UDF; + +-- positive +EXPLAIN SELECT +key FROM DECIMAL_UDF; +SELECT +key FROM DECIMAL_UDF; + +-- ceiling +EXPlAIN SELECT CEIL(key) FROM DECIMAL_UDF; +SELECT CEIL(key) FROM DECIMAL_UDF; + +-- floor +EXPLAIN SELECT FLOOR(key) FROM DECIMAL_UDF; +SELECT FLOOR(key) FROM DECIMAL_UDF; + +-- round +EXPLAIN SELECT ROUND(key, 2) FROM DECIMAL_UDF; +SELECT ROUND(key, 2) FROM DECIMAL_UDF; + +-- power +EXPLAIN SELECT POWER(key, 2) FROM DECIMAL_UDF; +SELECT POWER(key, 2) FROM DECIMAL_UDF; + +-- modulo +EXPLAIN SELECT (key + 1) % (key / 2) FROM DECIMAL_UDF; +SELECT (key + 1) % (key / 2) FROM DECIMAL_UDF; + +-- stddev, var +EXPLAIN SELECT value, stddev(key), variance(key) FROM DECIMAL_UDF GROUP BY value; +SELECT value, stddev(key), variance(key) FROM DECIMAL_UDF GROUP BY value; + +-- stddev_samp, var_samp +EXPLAIN SELECT value, stddev_samp(key), var_samp(key) FROM DECIMAL_UDF GROUP BY value; +SELECT value, stddev_samp(key), var_samp(key) FROM DECIMAL_UDF GROUP BY value; + +-- histogram +EXPLAIN SELECT histogram_numeric(key, 3) FROM DECIMAL_UDF; +SELECT histogram_numeric(key, 3) FROM DECIMAL_UDF; + +-- min +EXPLAIN SELECT MIN(key) FROM DECIMAL_UDF; +SELECT MIN(key) FROM DECIMAL_UDF; + +-- max +EXPLAIN SELECT MAX(key) FROM DECIMAL_UDF; +SELECT MAX(key) FROM DECIMAL_UDF; + +-- count +EXPLAIN SELECT COUNT(key) FROM DECIMAL_UDF; +SELECT COUNT(key) FROM DECIMAL_UDF; + +DROP TABLE IF EXISTS DECIMAL_UDF; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/default_partition_name.q b/src/test/hive/ql/src/test/queries/clientpositive/default_partition_name.q new file mode 100644 index 0000000000000..6a82318787979 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/default_partition_name.q @@ -0,0 +1,7 @@ +create table default_partition_name (key int, value string) partitioned by (ds string); + +set hive.exec.default.partition.name='some_other_default_partition_name'; + +alter table default_partition_name add partition(ds='__HIVE_DEFAULT_PARTITION__'); + +show partitions default_partition_name; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/delimiter.q b/src/test/hive/ql/src/test/queries/clientpositive/delimiter.q new file mode 100644 index 0000000000000..112ac57c3be5b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/delimiter.q @@ -0,0 +1,12 @@ +create table impressions (imp string, msg string) +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; + +select * from impressions; + +select imp,msg from impressions; + +drop table impressions; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q b/src/test/hive/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_indent.q b/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_indent.q new file mode 100644 index 0000000000000..310b694874f8d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_indent.q @@ -0,0 +1,14 @@ +-- test comment indent processing for multi-line comments + +CREATE TABLE test_table( + col1 INT COMMENT 'col1 one line comment', + col2 STRING COMMENT 'col2 +two lines comment', + col3 STRING COMMENT 'col3 +three lines +comment') +COMMENT 'table comment +two lines'; + +DESCRIBE test_table; +DESCRIBE FORMATTED test_table; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_nonascii.q b/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_nonascii.q new file mode 100644 index 0000000000000..9027d1ee46511 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_nonascii.q @@ -0,0 +1,9 @@ +-- HIVE-2905 showing non-ascii comments + +create table dummy (col1 string, col2 string, col3 string); + +alter table dummy change col1 col1 string comment '한글_col1'; +alter table dummy change col2 col2 string comment '漢字_col2'; +alter table dummy change col3 col3 string comment 'ã‚ã”_col3'; + +DESCRIBE FORMATTED dummy; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_database_json.q b/src/test/hive/ql/src/test/queries/clientpositive/describe_database_json.q new file mode 100644 index 0000000000000..2f4cedcdc4ff7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/describe_database_json.q @@ -0,0 +1,23 @@ +set hive.ddl.output.format=json; + +CREATE DATABASE IF NOT EXISTS jsondb1 COMMENT 'Test database' LOCATION '${hiveconf:hive.metastore.warehouse.dir}/jsondb1' WITH DBPROPERTIES ('id' = 'jsondb1'); + +DESCRIBE DATABASE jsondb1; + +DESCRIBE DATABASE EXTENDED jsondb1; + +SHOW DATABASES; + +SHOW DATABASES LIKE 'json*'; + +DROP DATABASE jsondb1; + +CREATE DATABASE jsondb1; + +DESCRIBE DATABASE jsondb1; + +DESCRIBE DATABASE EXTENDED jsondb1; + +DROP DATABASE jsondb1; + +set hive.ddl.output.format=text; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q b/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q new file mode 100644 index 0000000000000..241ff1113286e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q @@ -0,0 +1,15 @@ +DROP VIEW view_partitioned; + +CREATE VIEW view_partitioned +PARTITIONED ON (value) +AS +SELECT key, value +FROM src +WHERE key=86; + +ALTER VIEW view_partitioned +ADD PARTITION (value='val_86'); + +DESCRIBE FORMATTED view_partitioned PARTITION (value='val_86'); + +DROP VIEW view_partitioned; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q b/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q new file mode 100644 index 0000000000000..150ff70641198 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q @@ -0,0 +1,17 @@ +set hive.ddl.output.format=json; + +DROP VIEW view_partitioned; + +CREATE VIEW view_partitioned +PARTITIONED ON (value) +AS +SELECT key, value +FROM src +WHERE key=86; + +ALTER VIEW view_partitioned +ADD PARTITION (value='val_86'); + +DESCRIBE FORMATTED view_partitioned PARTITION (value='val_86'); + +DROP VIEW view_partitioned; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_pretty.q b/src/test/hive/ql/src/test/queries/clientpositive/describe_pretty.q new file mode 100644 index 0000000000000..89db075fd6d1b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/describe_pretty.q @@ -0,0 +1,80 @@ +-- test comment indent processing for multi-line comments + +CREATE TABLE test_table( + col1 INT COMMENT 'col1 one line comment', + col2 STRING COMMENT 'col2 +two lines comment', + col3 STRING COMMENT 'col3 +three lines +comment', + col4 STRING COMMENT 'col4 very long comment that is greater than 80 chars and is likely to spill into multiple lines', + col5 STRING COMMENT '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 COMMENT '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) +COMMENT 'table comment +two lines'; + +SET hive.cli.pretty.output.num.cols=80; + +-- There will be an extra tab at the end of each comment line in the output. +-- This is because DESCRIBE command separates the column, type and +-- comment field using a \t. DESCRIBE PRETTY uses spaces instead +-- of \t to separate columns. Hive gets confused when it parses the string +-- table description constructed in MetaDataPrettyFormatUtils, and adds a tab +-- at the end of each line. +-- There are three ways to address this: +-- 1. Pad each row to the full terminal width with extra spaces. +-- 2. Assume a maximum tab width of 8, and subtract 2 * 8 spaces from the +-- available line width. This approach wastes upto 2 * 8 - 2 columns. +-- 3. Since the pretty output is meant only for human consumption, do nothing. +-- Just add a comment to the unit test file explaining what is happening. +-- This is the approach chosen. + +DESCRIBE PRETTY test_table; + +SET hive.cli.pretty.output.num.cols=200; +DESCRIBE PRETTY test_table; + +SET hive.cli.pretty.output.num.cols=50; +DESCRIBE PRETTY test_table; + +SET hive.cli.pretty.output.num.cols=60; +DESCRIBE PRETTY test_table; + +CREATE TABLE test_table_very_long_column_name( + col1 INT COMMENT 'col1 one line comment', + col2_abcdefghiklmnopqrstuvxyz STRING COMMENT 'col2 +two lines comment', + col3 STRING COMMENT 'col3 +three lines +comment', + col4 STRING COMMENT 'col4 very long comment that is greater than 80 chars and is likely to spill into multiple lines') +; + +SET hive.cli.pretty.output.num.cols=80; +DESCRIBE PRETTY test_table_very_long_column_name; + +SET hive.cli.pretty.output.num.cols=20; +DESCRIBE PRETTY test_table_very_long_column_name; + +CREATE TABLE test_table_partitioned( + col1 INT COMMENT 'col1 one line comment', + col2 STRING COMMENT 'col2 +two lines comment', + col3 STRING COMMENT 'col3 +three lines +comment', + col4 STRING COMMENT 'col4 very long comment that is greater than 80 chars and is likely to spill into multiple lines', + col5 STRING COMMENT '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 COMMENT '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) +COMMENT 'table comment +two lines' +PARTITIONED BY (ds STRING); + +SET hive.cli.pretty.output.num.cols=60; +DESCRIBE PRETTY test_table_partitioned; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_syntax.q b/src/test/hive/ql/src/test/queries/clientpositive/describe_syntax.q new file mode 100644 index 0000000000000..c9255b5d3ceac --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/describe_syntax.q @@ -0,0 +1,46 @@ + +CREATE DATABASE db1; +CREATE TABLE db1.t1(key1 INT, value1 STRING) PARTITIONED BY (ds STRING, part STRING); + +use db1; + +ALTER TABLE t1 ADD PARTITION (ds='3', part='3'); +ALTER TABLE t1 ADD PARTITION (ds='4', part='4'); +ALTER TABLE t1 ADD PARTITION (ds='4', part='5'); + +-- describe table +DESCRIBE t1; +DESCRIBE EXTENDED t1; +DESCRIBE FORMATTED t1; + +-- describe database.table +DESCRIBE db1.t1; +DESCRIBE EXTENDED db1.t1; +DESCRIBE FORMATTED db1.t1; + +-- describe table column +DESCRIBE t1 key1; +DESCRIBE EXTENDED t1 key1; +DESCRIBE FORMATTED t1 key1; + +-- describe database.tabe column +DESCRIBE db1.t1 key1; +DESCRIBE EXTENDED db1.t1 key1; +DESCRIBE FORMATTED db1.t1 key1; + +-- describe table.column +-- after first checking t1.key1 for database.table not valid +-- fall back to the old syntax table.column +DESCRIBE t1.key1; +DESCRIBE EXTENDED t1.key1; +DESCRIBE FORMATTED t1.key1; + +-- describe table partition +DESCRIBE t1 PARTITION(ds='4', part='5'); +DESCRIBE EXTENDED t1 PARTITION(ds='4', part='5'); +DESCRIBE FORMATTED t1 PARTITION(ds='4', part='5'); + +-- describe database.table partition +DESCRIBE db1.t1 PARTITION(ds='4', part='5'); +DESCRIBE EXTENDED db1.t1 PARTITION(ds='4', part='5'); +DESCRIBE FORMATTED db1.t1 PARTITION(ds='4', part='5'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_table.q b/src/test/hive/ql/src/test/queries/clientpositive/describe_table.q new file mode 100644 index 0000000000000..f72cae9d8003f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/describe_table.q @@ -0,0 +1,19 @@ +describe srcpart; +describe srcpart.key; +describe srcpart PARTITION(ds='2008-04-08', hr='12'); + +describe extended srcpart; +describe extended srcpart.key; +describe extended srcpart PARTITION(ds='2008-04-08', hr='12'); + +describe formatted srcpart; +describe formatted srcpart.key; +describe formatted srcpart PARTITION(ds='2008-04-08', hr='12'); + +create table srcpart_serdeprops like srcpart; +alter table srcpart_serdeprops set serdeproperties('xyz'='0'); +alter table srcpart_serdeprops set serdeproperties('pqrs'='1'); +alter table srcpart_serdeprops set serdeproperties('abcd'='2'); +alter table srcpart_serdeprops set serdeproperties('A1234'='3'); +describe formatted srcpart_serdeprops; +drop table srcpart_serdeprops; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_table_json.q b/src/test/hive/ql/src/test/queries/clientpositive/describe_table_json.q new file mode 100644 index 0000000000000..f19e98d7985ce --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/describe_table_json.q @@ -0,0 +1,19 @@ +set hive.ddl.output.format=json; + +CREATE TABLE IF NOT EXISTS jsontable (key INT, value STRING) COMMENT 'json table' STORED AS TEXTFILE; + +SHOW TABLES; + +SHOW TABLES LIKE 'json*'; + +SHOW TABLE EXTENDED LIKE 'json*'; + +ALTER TABLE jsontable SET TBLPROPERTIES ('id' = 'jsontable'); + +DESCRIBE jsontable; + +DESCRIBE extended jsontable; + +DROP TABLE jsontable; + +set hive.ddl.output.format=text; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_xpath.q b/src/test/hive/ql/src/test/queries/clientpositive/describe_xpath.q new file mode 100644 index 0000000000000..469cbf43c1141 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/describe_xpath.q @@ -0,0 +1,17 @@ +-- Describe a list structure in a thrift table +describe src_thrift.lint; + +-- Describe the element of a list +describe src_thrift.lint.$elem$; + +-- Describe the key of a map +describe src_thrift.mStringString.$key$; + +-- Describe the value of a map +describe src_thrift.mStringString.$value$; + +-- Describe a complex element of a list +describe src_thrift.lintString.$elem$; + +-- Describe a member of an element of a list +describe src_thrift.lintString.$elem$.myint; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/diff_part_input_formats.q b/src/test/hive/ql/src/test/queries/clientpositive/diff_part_input_formats.q new file mode 100644 index 0000000000000..c5741d005010d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/diff_part_input_formats.q @@ -0,0 +1,9 @@ +-- Tests the case where a table is changed from sequence file to a RC file, +-- resulting in partitions in both file formats. If no valid partitions are +-- selected, then it should still use RC file for reading the dummy partition. +CREATE TABLE part_test (key STRING, value STRING) PARTITIONED BY (ds STRING) STORED AS SEQUENCEFILE; +ALTER TABLE part_test ADD PARTITION(ds='1'); +ALTER TABLE part_test SET FILEFORMAT RCFILE; +ALTER TABLE part_test ADD PARTITION(ds='2'); +SELECT count(1) FROM part_test WHERE ds='3'; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/disable_file_format_check.q b/src/test/hive/ql/src/test/queries/clientpositive/disable_file_format_check.q new file mode 100644 index 0000000000000..6ea4156b3493e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/disable_file_format_check.q @@ -0,0 +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; + +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; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q b/src/test/hive/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q new file mode 100644 index 0000000000000..471d296b8d2d0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q @@ -0,0 +1,21 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.enforce.bucketing = true; +set hive.exec.reducers.max = 1; +set hive.merge.mapredfiles=true; + + +CREATE TABLE bucket2_1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; + +explain extended +insert overwrite table bucket2_1 +select * from src; + +insert overwrite table bucket2_1 +select * from src; + +explain +select * from bucket2_1 tablesample (bucket 1 out of 2) s order by key; + +select * from bucket2_1 tablesample (bucket 1 out of 2) s order by key; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q b/src/test/hive/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q new file mode 100644 index 0000000000000..2a1e7276cacfe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q @@ -0,0 +1,7 @@ +SET hive.metastore.disallow.incompatible.col.type.changes=false; +SELECT * FROM src LIMIT 1; +CREATE TABLE test_table123 (a INT, b MAP) PARTITIONED BY (ds STRING) STORED AS SEQUENCEFILE; +INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, MAP("a1", "b1") FROM src LIMIT 1; +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/src/test/hive/ql/src/test/queries/clientpositive/driverhook.q b/src/test/hive/ql/src/test/queries/clientpositive/driverhook.q new file mode 100644 index 0000000000000..191c15436b614 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/driverhook.q @@ -0,0 +1,6 @@ +SET hive.exec.driver.run.hooks=org.apache.hadoop.hive.ql.hooks.DriverTestHook; + +-- This query should appear in the Hive CLI output. +-- We test DriverTestHook, which does exactly that. +-- This should not break. +SELECT * FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q new file mode 100644 index 0000000000000..a5f5189491fed --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q @@ -0,0 +1,27 @@ +-- This test verifies that if a partition exists outside a table's current location when the +-- database is dropped the partition's location is dropped as well. + +CREATE DATABASE test_database; + +USE test_database; + +CREATE TABLE test_table (key STRING, value STRING) +PARTITIONED BY (part STRING) +STORED AS RCFILE +LOCATION 'file:${system:test.tmp.dir}/drop_database_removes_partition_dirs_table'; + +ALTER TABLE test_table ADD PARTITION (part = '1') +LOCATION 'file:${system:test.tmp.dir}/drop_database_removes_partition_dirs_table2/part=1'; + +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT * FROM default.src; + +dfs -ls ${system:test.tmp.dir}/drop_database_removes_partition_dirs_table2; + +USE default; + +DROP DATABASE test_database CASCADE; + +dfs -ls ${system:test.tmp.dir}/drop_database_removes_partition_dirs_table2; + +dfs -rmr ${system:test.tmp.dir}/drop_database_removes_partition_dirs_table2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_function.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_function.q new file mode 100644 index 0000000000000..18c428ce20a55 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_function.q @@ -0,0 +1,2 @@ +SET hive.exec.drop.ignorenonexistent=false; +DROP TEMPORARY FUNCTION IF EXISTS UnknownFunction; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_index.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_index.q new file mode 100644 index 0000000000000..df23f8d33f0c2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_index.q @@ -0,0 +1,2 @@ +SET hive.exec.drop.ignorenonexistent=false; +DROP INDEX IF EXISTS UnknownIndex ON src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q new file mode 100644 index 0000000000000..ed471db2cffb7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q @@ -0,0 +1,22 @@ +-- This test verifies that if a partition exists outside an index table's current location when the +-- index is dropped the partition's location is dropped as well. + +CREATE TABLE test_table (key STRING, value STRING) +PARTITIONED BY (part STRING) +STORED AS RCFILE +LOCATION 'file:${system:test.tmp.dir}/drop_database_removes_partition_dirs_table'; + +CREATE INDEX test_index ON +TABLE test_table(key) AS 'compact' WITH DEFERRED REBUILD +IN TABLE test_index_table; + +ALTER TABLE test_index_table ADD PARTITION (part = '1') +LOCATION 'file:${system:test.tmp.dir}/drop_index_removes_partition_dirs_index_table2/part=1'; + +dfs -ls ${system:test.tmp.dir}/drop_index_removes_partition_dirs_index_table2; + +DROP INDEX test_index ON test_table; + +dfs -ls ${system:test.tmp.dir}/drop_index_removes_partition_dirs_index_table2; + +dfs -rmr ${system:test.tmp.dir}/drop_index_removes_partition_dirs_index_table2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_multi_partitions.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_multi_partitions.q new file mode 100644 index 0000000000000..14e235660929f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_multi_partitions.q @@ -0,0 +1,17 @@ +create table mp (a string) partitioned by (b string, c string); + +alter table mp add partition (b='1', c='1'); +alter table mp add partition (b='1', c='2'); +alter table mp add partition (b='2', c='2'); + +show partitions mp; + +explain extended alter table mp drop partition (b='1'); +alter table mp drop partition (b='1'); + +show partitions mp; + +set hive.exec.drop.ignorenonexistent=false; +alter table mp drop if exists partition (b='3'); + +show partitions mp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter.q new file mode 100644 index 0000000000000..04fbcae0a6586 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter.q @@ -0,0 +1,37 @@ +create table ptestfilter (a string, b int) partitioned by (c string, d string); +describe ptestfilter; + +alter table ptestfilter add partition (c='US', d=1); +alter table ptestfilter add partition (c='US', d=2); +alter table ptestFilter add partition (c='Uganda', d=2); +alter table ptestfilter add partition (c='Germany', d=2); +alter table ptestfilter add partition (c='Canada', d=3); +alter table ptestfilter add partition (c='Russia', d=3); +alter table ptestfilter add partition (c='Greece', d=2); +alter table ptestfilter add partition (c='India', d=3); +alter table ptestfilter add partition (c='France', d=4); +show partitions ptestfilter; + +alter table ptestfilter drop partition (c='US', d<'2'); +show partitions ptestfilter; + +alter table ptestfilter drop partition (c>='US', d<='2'); +show partitions ptestfilter; + +alter table ptestfilter drop partition (c >'India'); +show partitions ptestfilter; + +alter table ptestfilter drop partition (c >='India'), + partition (c='Greece', d='2'); +show partitions ptestfilter; + +alter table ptestfilter drop partition (c != 'France'); +show partitions ptestfilter; + +set hive.exec.drop.ignorenonexistent=false; +alter table ptestfilter drop if exists partition (c='US'); +show partitions ptestfilter; + +drop table ptestfilter; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter2.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter2.q new file mode 100644 index 0000000000000..798aa6d51a185 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter2.q @@ -0,0 +1,20 @@ +create table ptestfilter (a string, b int) partitioned by (c int, d int); +describe ptestfilter; + +alter table ptestfilter add partition (c=1, d=1); +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); +show partitions ptestfilter; + +alter table ptestfilter drop partition (c=1, d=1); +show partitions ptestfilter; + +alter table ptestfilter drop partition (c=2); +show partitions ptestfilter; + +drop table ptestfilter; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter3.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter3.q new file mode 100644 index 0000000000000..2d41da4627bdd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter3.q @@ -0,0 +1,20 @@ +create table ptestfilter (a string, b int) partitioned by (c string, d int); +describe ptestfilter; + +alter table ptestfilter add partition (c='1', d=1); +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); +show partitions ptestfilter; + +alter table ptestfilter drop partition (c='1', d=1); +show partitions ptestfilter; + +alter table ptestfilter drop partition (c='2'); +show partitions ptestfilter; + +drop table ptestfilter; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q new file mode 100644 index 0000000000000..e825df9e5b5b4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q @@ -0,0 +1,10 @@ +create table tbl_protectmode_no_drop (c1 string,c2 string) partitioned by (p string); +alter table tbl_protectmode_no_drop add partition (p='p1'); +alter table tbl_protectmode_no_drop partition (p='p1') enable no_drop; +desc extended tbl_protectmode_no_drop partition (p='p1'); + +-- The partition will be dropped, even though we have enabled no_drop +-- as 'ignore protection' has been specified in the command predicate +alter table tbl_protectmode_no_drop drop partition (p='p1') ignore protection; +drop table tbl_protectmode_no_drop; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_table.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_table.q new file mode 100644 index 0000000000000..6d189fc43c6b7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_table.q @@ -0,0 +1,2 @@ +SET hive.exec.drop.ignorenonexistent=false; +DROP TABLE IF EXISTS UnknownTable; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_table2.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_table2.q new file mode 100644 index 0000000000000..a3e8c5c3bd5eb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_table2.q @@ -0,0 +1,15 @@ +SET hive.metastore.batch.retrieve.max=1; +create table if not exists temp(col STRING) partitioned by (p STRING); +alter table temp add if not exists partition (p ='p1'); +alter table temp add if not exists partition (p ='p2'); +alter table temp add if not exists partition (p ='p3'); + +show partitions temp; + +drop table temp; + +create table if not exists temp(col STRING) partitioned by (p STRING); + +show partitions temp; + +drop table temp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q new file mode 100644 index 0000000000000..102b3ef5f5e3a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q @@ -0,0 +1,21 @@ +-- This test verifies that if a partition exists outside the table's current location when the +-- table is dropped the partition's location is dropped as well. + +CREATE TABLE test_table (key STRING, value STRING) +PARTITIONED BY (part STRING) +STORED AS RCFILE +LOCATION 'file:${system:test.tmp.dir}/drop_table_removes_partition_dirs_table'; + +ALTER TABLE test_table ADD PARTITION (part = '1') +LOCATION 'file:${system:test.tmp.dir}/drop_table_removes_partition_dirs_table2/part=1'; + +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT * FROM src; + +dfs -ls ${system:test.tmp.dir}/drop_table_removes_partition_dirs_table2; + +DROP TABLE test_table; + +dfs -ls ${system:test.tmp.dir}/drop_table_removes_partition_dirs_table2; + +dfs -rmr ${system:test.tmp.dir}/drop_table_removes_partition_dirs_table2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_udf.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_udf.q new file mode 100644 index 0000000000000..8aaa66e3e694d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_udf.q @@ -0,0 +1,6 @@ +CREATE TEMPORARY FUNCTION test_translate AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestTranslate'; + +EXPLAIN +DROP TEMPORARY FUNCTION test_translate; + +DROP TEMPORARY FUNCTION test_translate; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_view.q b/src/test/hive/ql/src/test/queries/clientpositive/drop_view.q new file mode 100644 index 0000000000000..4ff097541ee99 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/drop_view.q @@ -0,0 +1,2 @@ +SET hive.exec.drop.ignorenonexistent=false; +DROP VIEW IF EXISTS UnknownView; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q b/src/test/hive/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q new file mode 100644 index 0000000000000..397a2200a8972 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q @@ -0,0 +1,19 @@ +create table dynamic_part_table(intcol int) partitioned by (partcol1 int, partcol2 int); + +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, 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, NULL, NULL from src where key=150; + +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)or (partcol1=1 and partcol2='__HIVE_DEFAULT_PARTITION__'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/enforce_order.q b/src/test/hive/ql/src/test/queries/clientpositive/enforce_order.q new file mode 100644 index 0000000000000..6a303c3ad2139 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/enforce_order.q @@ -0,0 +1,13 @@ +drop table table_asc; +drop table table_desc; + +set hive.enforce.sorting = true; + +create table table_asc(key string, value string) clustered by (key) sorted by (key ASC) into 1 BUCKETS; +create table table_desc(key string, value string) clustered by (key) sorted by (key DESC) into 1 BUCKETS; + +insert overwrite table table_asc select key, value from src; +insert overwrite table table_desc select key, value from src; + +select * from table_asc limit 10; +select * from table_desc limit 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape1.q b/src/test/hive/ql/src/test/queries/clientpositive/escape1.q new file mode 100644 index 0000000000000..a7f4cf79c980f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/escape1.q @@ -0,0 +1,28 @@ +set hive.exec.dynamic.partition=true; +set hive.exec.max.dynamic.partitions.pernode=200; + +-- EXCLUDE_OS_WINDOWS +-- excluded on windows because of difference in file name encoding logic + +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; + +SELECT count(*) from escape_raw; +SELECT * from escape_raw; + +CREATE TABLE escape1 (a STRING) PARTITIONED BY (ds STRING, part STRING); +INSERT OVERWRITE TABLE escape1 PARTITION (ds='1', part) SELECT '1', s from +escape_raw; + +SELECT count(*) from escape1; +SELECT * from escape1; +SHOW PARTITIONS escape1; + +ALTER TABLE escape1 DROP PARTITION (ds='1'); +SHOW PARTITIONS escape1; + +DROP TABLE escape1; +DROP TABLE escape_raw; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape2.q b/src/test/hive/ql/src/test/queries/clientpositive/escape2.q new file mode 100644 index 0000000000000..473cbf8c94e4e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/escape2.q @@ -0,0 +1,122 @@ +set hive.exec.dynamic.partition=true; +set hive.exec.max.dynamic.partitions.pernode=200; +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set hive.default.fileformat=RCFILE; + +-- EXCLUDE_OS_WINDOWS +-- excluded on windows because of difference in file name encoding logic + +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; + +SELECT count(*) from escape_raw; +SELECT * from escape_raw; + +CREATE TABLE escape2(a STRING) PARTITIONED BY (ds STRING, part STRING); +INSERT OVERWRITE TABLE escape2 PARTITION (ds='1', part) SELECT '1', s from +escape_raw; + +SELECT count(*) from escape2; +SELECT * from escape2; +SHOW PARTITIONS escape2; + +-- ASCII values 1-31, 59, 92, 127 were not included in the below commands + +ALTER table escape2 PARTITION (ds='1', part=' ') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='!') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='"') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='#') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='$') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='%') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='&') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part="'") CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='(') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part=')') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='*') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='+') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part=',') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='-') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='.') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='/') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='0') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='1') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='2') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='3') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='4') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='5') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='6') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='7') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='8') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='9') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part=':') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='<') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='=') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='>') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='?') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='@') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='A') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='B') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='C') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='D') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='E') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='F') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='G') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='H') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='I') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='J') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='K') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='L') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='M') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='N') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='O') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='P') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='Q') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='R') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='S') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='T') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='U') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='V') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='W') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='X') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='Y') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='Z') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='[') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part=']') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='_') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='`') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='a') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='b') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='c') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='d') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='e') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='f') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='g') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='h') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='i') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='j') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='k') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='l') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='m') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='n') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='o') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='p') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='q') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='r') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='s') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='t') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='u') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='v') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='w') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='x') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='y') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='z') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='{') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='|') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='}') CONCATENATE; +ALTER TABLE escape2 PARTITION (ds='1', part='~') CONCATENATE; + +DROP TABLE escape2; +DROP TABLE escape_raw; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape_clusterby1.q b/src/test/hive/ql/src/test/queries/clientpositive/escape_clusterby1.q new file mode 100644 index 0000000000000..13ea15153909a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/escape_clusterby1.q @@ -0,0 +1,6 @@ +-- escaped column names in cluster by are not working jira 3267 +explain +select key, value from src cluster by key, value; + +explain +select `key`, value from src cluster by `key`, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape_distributeby1.q b/src/test/hive/ql/src/test/queries/clientpositive/escape_distributeby1.q new file mode 100644 index 0000000000000..40ed2de635367 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/escape_distributeby1.q @@ -0,0 +1,6 @@ +-- escaped column names in distribute by by are not working jira 3267 +explain +select key, value from src distribute by key, value; + +explain +select `key`, value from src distribute by `key`, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape_orderby1.q b/src/test/hive/ql/src/test/queries/clientpositive/escape_orderby1.q new file mode 100644 index 0000000000000..39a1c4c787e6a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/escape_orderby1.q @@ -0,0 +1,6 @@ +-- escaped column names in order by are not working jira 3267 +explain +select key, value from src order by key, value; + +explain +select `key`, value from src order by `key`, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape_sortby1.q b/src/test/hive/ql/src/test/queries/clientpositive/escape_sortby1.q new file mode 100644 index 0000000000000..6b487c8a47d21 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/escape_sortby1.q @@ -0,0 +1,6 @@ +-- escaped column names in sort by are not working jira 3267 +explain +select key, value from src sort by key, value; + +explain +select `key`, value from src sort by `key`, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition.q b/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition.q new file mode 100644 index 0000000000000..6e8bf8ebc1fa2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition.q @@ -0,0 +1,12 @@ +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; +SHOW PARTITIONS exchange_part_test2; + +ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05'); +SHOW PARTITIONS exchange_part_test1; +SHOW PARTITIONS exchange_part_test2; + +ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE exchange_part_test2; +SHOW PARTITIONS exchange_part_test1; +SHOW PARTITIONS exchange_part_test2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition2.q b/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition2.q new file mode 100644 index 0000000000000..27b335a3d7844 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition2.q @@ -0,0 +1,12 @@ +CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); +CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); +SHOW PARTITIONS exchange_part_test1; +SHOW PARTITIONS exchange_part_test2; + +ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='1'); +SHOW PARTITIONS exchange_part_test1; +SHOW PARTITIONS exchange_part_test2; + +ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05', hr='1') WITH TABLE exchange_part_test2; +SHOW PARTITIONS exchange_part_test1; +SHOW PARTITIONS exchange_part_test2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition3.q b/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition3.q new file mode 100644 index 0000000000000..7b9060d420a1e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition3.q @@ -0,0 +1,14 @@ +CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); +CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); +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'); +SHOW PARTITIONS exchange_part_test1; +SHOW PARTITIONS exchange_part_test2; + +-- This will exchange both partitions hr=1 and hr=2 +ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE exchange_part_test2; +SHOW PARTITIONS exchange_part_test1; +SHOW PARTITIONS exchange_part_test2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q new file mode 100644 index 0000000000000..7fa96b629a0fa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q @@ -0,0 +1,23 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +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; +select * from exim_department; +drop table exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_01_nonpart.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_01_nonpart.q new file mode 100644 index 0000000000000..9920e778d18a6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_01_nonpart.q @@ -0,0 +1,24 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +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; +select * from exim_department; +drop table exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q new file mode 100644 index 0000000000000..4017c83aa3dd4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q @@ -0,0 +1,25 @@ +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 comment "employee id") + comment "employee table" + 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; +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 extended exim_employee; +show table extended like exim_employee; +dfs -rmr ../build/ql/test/data/exports/exim_employee; +select * from exim_employee; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_02_part.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_02_part.q new file mode 100644 index 0000000000000..21138f0263418 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_02_part.q @@ -0,0 +1,27 @@ +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 comment "employee id") + comment "employee table" + 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" + 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; +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 extended exim_employee; +show table extended like exim_employee; +dfs -rmr ../build/ql/test/data/exports/exim_employee; +select * from exim_employee; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q new file mode 100644 index 0000000000000..5f6bdee83cc90 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q @@ -0,0 +1,26 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int comment "department identifier") + stored as textfile + tblproperties("maker"="krishna"); +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; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_04_all_part.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_04_all_part.q new file mode 100644 index 0000000000000..69c6faa30a07c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_04_all_part.q @@ -0,0 +1,33 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +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 extended exim_employee; +show table extended like exim_employee; +dfs -rmr ../build/ql/test/data/exports/exim_employee; +select * from exim_employee; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q new file mode 100644 index 0000000000000..cdc02fa25c169 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q @@ -0,0 +1,39 @@ +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 comment 'employee id', emp_name string, emp_dob string comment 'employee date of birth', emp_sex string comment 'M/F') + comment 'employee table' + partitioned by (emp_country string comment '2-char code', emp_state string comment '2-char code') + clustered by (emp_sex) sorted by (emp_id ASC) into 10 buckets + row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" with serdeproperties ('serialization.format'='1') + stored as rcfile; + +alter table exim_employee add columns (emp_dept int); +alter table exim_employee clustered by (emp_sex, emp_dept) sorted by (emp_id desc) into 5 buckets; +alter table exim_employee add partition (emp_country='in', emp_state='tn'); + +alter table exim_employee set serde "org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe" with serdeproperties ('serialization.format'='2'); +alter table exim_employee set fileformat + inputformat "org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat" + 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; +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 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; +select * from exim_employee; +drop table exim_employee; + +drop database importer; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_05_some_part.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_05_some_part.q new file mode 100644 index 0000000000000..50a59463b1870 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_05_some_part.q @@ -0,0 +1,33 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +export table exim_employee partition (emp_state="ka") 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 extended exim_employee; +show table extended like exim_employee; +dfs -rmr ../build/ql/test/data/exports/exim_employee; +select * from exim_employee; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_06_one_part.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_06_one_part.q new file mode 100644 index 0000000000000..5136090929fc5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_06_one_part.q @@ -0,0 +1,33 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +export table exim_employee partition (emp_country="in",emp_state="ka") 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 extended exim_employee; +show table extended like exim_employee; +dfs -rmr ../build/ql/test/data/exports/exim_employee; +select * from exim_employee; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q new file mode 100644 index 0000000000000..5b9d4ddc03c59 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q @@ -0,0 +1,39 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; + +create table exim_employee ( emp_id int comment "employee id") + comment "table of employees" + 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" + 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; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q new file mode 100644 index 0000000000000..173f1569c501a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q @@ -0,0 +1,28 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; +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"); +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; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q new file mode 100644 index 0000000000000..178b76674d862 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q @@ -0,0 +1,40 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; +create table exim_employee ( emp_id int comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_10_external_managed.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_10_external_managed.q new file mode 100644 index 0000000000000..413f2aa4762f5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_10_external_managed.q @@ -0,0 +1,27 @@ +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; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; +dfs -rmr ../build/ql/test/data/tablestore/exim_department; + +create database importer; +use importer; + +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; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_11_managed_external.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_11_managed_external.q new file mode 100644 index 0000000000000..f3b2896a322a5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_11_managed_external.q @@ -0,0 +1,24 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +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; +select * from exim_department; +drop table exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_12_external_location.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_12_external_location.q new file mode 100644 index 0000000000000..37d063432e16a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_12_external_location.q @@ -0,0 +1,29 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; +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; +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; + +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; +select * from exim_department; +dfs -rmr ../build/ql/test/data/tablestore/exim_department; +select * from exim_department; +drop table exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_13_managed_location.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_13_managed_location.q new file mode 100644 index 0000000000000..fb5058b840bae --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_13_managed_location.q @@ -0,0 +1,29 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; +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; +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; + +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; +select * from exim_department; +dfs -rmr ../build/ql/test/data/tablestore/exim_department; +select * from exim_department; +drop table exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q new file mode 100644 index 0000000000000..031b6bda6cece --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q @@ -0,0 +1,33 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; +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; +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; + +create table exim_department ( dep_id int comment "department id") + stored as textfile + location 'ql/test/data/tablestore/exim_department' + tblproperties("creator"="krishna"); +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; +select * from exim_department; +dfs -rmr ../build/ql/test/data/tablestore/exim_department; +select * from exim_department; +drop table exim_department; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_15_external_part.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_15_external_part.q new file mode 100644 index 0000000000000..ff088c70d7ffb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_15_external_part.q @@ -0,0 +1,50 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +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; + +create external table exim_employee ( emp_id int comment "employee id") + comment "employee table" + partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") + stored as textfile + location 'ql/test/data/tablestore/exim_employee' + tblproperties("creator"="krishna"); +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" + 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; +select * from exim_employee; +dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +select * from exim_employee; +drop table exim_employee; + + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_16_part_external.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_16_part_external.q new file mode 100644 index 0000000000000..6f4ee7a01c256 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_16_part_external.q @@ -0,0 +1,49 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +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; + +create external table exim_employee ( emp_id int comment "employee id") + comment "employee table" + partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") + stored as textfile + location 'ql/test/data/tablestore2/exim_employee' + tblproperties("creator"="krishna"); +import table exim_employee partition (emp_country="us", emp_state="tn") + from 'ql/test/data/exports/exim_employee' + 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; +select * from exim_employee; +dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +select * from exim_employee; +drop table exim_employee; +dfs -rmr ../build/ql/test/data/tablestore2/exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_17_part_managed.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_17_part_managed.q new file mode 100644 index 0000000000000..56ec152948aed --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_17_part_managed.q @@ -0,0 +1,48 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +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; + +create table exim_employee ( emp_id int comment "employee id") + comment "employee table" + partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") + stored as textfile + tblproperties("creator"="krishna"); +import table exim_employee partition (emp_country="us", emp_state="tn") + from 'ql/test/data/exports/exim_employee' + location 'ql/test/data/tablestore/exim_employee'; +alter table exim_employee add partition (emp_country="us", emp_state="ap") + location 'ql/test/data/tablestore2/exim_employee'; +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; +select * from exim_employee; +dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +select * from exim_employee; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_18_part_external.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_18_part_external.q new file mode 100644 index 0000000000000..7aa1297dc7d02 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_18_part_external.q @@ -0,0 +1,36 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; + +import external table exim_employee partition (emp_country="us", emp_state="tn") + from 'ql/test/data/exports/exim_employee'; +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; +select * from exim_employee; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q new file mode 100644 index 0000000000000..cb9f8efc087da --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q @@ -0,0 +1,38 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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; +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; + +import external table exim_employee + from 'ql/test/data/exports/exim_employee' + location 'ql/test/data/tablestore/exim_employee'; +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; +select * from exim_employee; +dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +select * from exim_employee; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_19_part_external_location.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_19_part_external_location.q new file mode 100644 index 0000000000000..bdbd19df70a21 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_19_part_external_location.q @@ -0,0 +1,41 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +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; + +import external table exim_employee partition (emp_country="us", emp_state="tn") + from 'ql/test/data/exports/exim_employee' + location 'ql/test/data/tablestore/exim_employee'; +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; +select * from exim_employee; +dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +select * from exim_employee; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q new file mode 100644 index 0000000000000..eb44961a9b7ca --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q @@ -0,0 +1,41 @@ +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 comment "employee id") + comment "employee table" + 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" + into table exim_employee partition (emp_country="in", emp_state="tn"); +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" + into table exim_employee partition (emp_country="us", emp_state="tn"); +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; +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; + +import table exim_employee partition (emp_country="us", emp_state="tn") + from 'ql/test/data/exports/exim_employee' + location 'ql/test/data/tablestore/exim_employee'; +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; +select * from exim_employee; +dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +select * from exim_employee; +drop table exim_employee; + +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q new file mode 100644 index 0000000000000..822ed70a38c0c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q @@ -0,0 +1,15 @@ +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; + +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; +export table exim_department to 'ql/test/data/exports/exim_department'; + +set hive.security.authorization.enabled=false; +drop table exim_department; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q new file mode 100644 index 0000000000000..440d08d2dc7b5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q @@ -0,0 +1,26 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +create table exim_department ( dep_id int) stored as textfile; +set hive.security.authorization.enabled=true; +grant Alter on table exim_department to user hive_test_user; +grant Update on table exim_department to user hive_test_user; +import from 'ql/test/data/exports/exim_department'; + +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; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q new file mode 100644 index 0000000000000..30fc343dd8f9c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q @@ -0,0 +1,34 @@ +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 comment "employee id") + comment "employee table" + 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" + 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; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; +create table exim_employee ( emp_id int comment "employee id") + comment "employee table" + partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") + stored as textfile + tblproperties("creator"="krishna"); + +set hive.security.authorization.enabled=true; +grant Alter on table exim_employee to user hive_test_user; +grant Update on table exim_employee to user hive_test_user; +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; +drop table exim_employee; +drop database importer; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q b/src/test/hive/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q new file mode 100644 index 0000000000000..2dc5af6ce4774 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q @@ -0,0 +1,24 @@ +set hive.test.mode=true; +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; +export table exim_department to 'ql/test/data/exports/exim_department'; +drop table exim_department; + +create database importer; +use importer; + +set hive.security.authorization.enabled=true; +grant Create on database importer to user hive_test_user; +import from 'ql/test/data/exports/exim_department'; + +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; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency.q b/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency.q new file mode 100644 index 0000000000000..eecf2a6bbef04 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency.q @@ -0,0 +1,34 @@ +-- This test is used for testing EXPLAIN DEPENDENCY command + +-- Create some views +CREATE VIEW V1 AS SELECT key, value from src; +CREATE VIEW V2 AS SELECT ds, key, value FROM srcpart WHERE ds IS NOT NULL; +CREATE VIEW V3 AS + SELECT src1.key, src2.value FROM V2 src1 + JOIN src src2 ON src1.key = src2.key WHERE src1.ds IS NOT NULL; +CREATE VIEW V4 AS + SELECT src1.key, src2.value as value1, src3.value as value2 + FROM V1 src1 JOIN V2 src2 on src1.key = src2.key JOIN src src3 ON src2.key = src3.key; + +-- Simple select queries, union queries and join queries +EXPLAIN DEPENDENCY + SELECT key, count(1) FROM srcpart WHERE ds IS NOT NULL GROUP BY key; +EXPLAIN DEPENDENCY + SELECT key, count(1) FROM (SELECT key, value FROM src) subq1 GROUP BY key; +EXPLAIN DEPENDENCY + SELECT * FROM ( + SELECT key, value FROM src UNION ALL SELECT key, value FROM srcpart WHERE ds IS NOT NULL + ) S1; +EXPLAIN DEPENDENCY + SELECT S1.key, S2.value FROM src S1 JOIN srcpart S2 ON S1.key = S2.key WHERE ds IS NOT NULL; + +-- With views +EXPLAIN DEPENDENCY SELECT * FROM V1; +EXPLAIN DEPENDENCY SELECT * FROM V2; +EXPLAIN DEPENDENCY SELECT * FROM V3; +EXPLAIN DEPENDENCY SELECT * FROM V4; + +-- The table should show up in the explain dependency even if none +-- of the partitions are selected. +CREATE VIEW V5 as SELECT * FROM srcpart where ds = '10'; +EXPLAIN DEPENDENCY SELECT * FROM V5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency2.q b/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency2.q new file mode 100644 index 0000000000000..88d4464a91487 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency2.q @@ -0,0 +1,31 @@ +-- This test is used for testing EXPLAIN DEPENDENCY command + +-- select from a table which does not involve a map-reduce job +EXPLAIN DEPENDENCY SELECT * FROM src; + +-- select from a table which involves a map-reduce job +EXPLAIN DEPENDENCY SELECT count(*) FROM src; + +-- select from a partitioned table which does not involve a map-reduce job +-- and some partitions are being selected +EXPLAIN DEPENDENCY SELECT * FROM srcpart where ds is not null; + +-- select from a partitioned table which does not involve a map-reduce job +-- and none of the partitions are being selected +EXPLAIN DEPENDENCY SELECT * FROM srcpart where ds = '1'; + +-- select from a partitioned table which involves a map-reduce job +-- and some partitions are being selected +EXPLAIN DEPENDENCY SELECT count(*) FROM srcpart where ds is not null; + +-- select from a partitioned table which involves a map-reduce job +-- and none of the partitions are being selected +EXPLAIN DEPENDENCY SELECT count(*) FROM srcpart where ds = '1'; + +create table tstsrcpart like srcpart; + +-- select from a partitioned table with no partitions which does not involve a map-reduce job +EXPLAIN DEPENDENCY SELECT * FROM tstsrcpart where ds is not null; + +-- select from a partitioned table with no partitions which involves a map-reduce job +EXPLAIN DEPENDENCY SELECT count(*) FROM tstsrcpart where ds is not null; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/explain_logical.q b/src/test/hive/ql/src/test/queries/clientpositive/explain_logical.q new file mode 100644 index 0000000000000..4b0fa5b8932d7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/explain_logical.q @@ -0,0 +1,36 @@ +-- This test is used for testing EXPLAIN LOGICAL command + +-- Create some views +CREATE VIEW V1 AS SELECT key, value from src; +CREATE VIEW V2 AS SELECT ds, key, value FROM srcpart WHERE ds IS NOT NULL; +CREATE VIEW V3 AS + SELECT src1.key, src2.value FROM V2 src1 + JOIN src src2 ON src1.key = src2.key WHERE src1.ds IS NOT NULL; +CREATE VIEW V4 AS + SELECT src1.key, src2.value as value1, src3.value as value2 + FROM V1 src1 JOIN V2 src2 on src1.key = src2.key JOIN src src3 ON src2.key = src3.key; + +-- Simple select queries, union queries and join queries +EXPLAIN LOGICAL + SELECT key, count(1) FROM srcpart WHERE ds IS NOT NULL GROUP BY key; +EXPLAIN LOGICAL + SELECT key, count(1) FROM (SELECT key, value FROM src) subq1 GROUP BY key; +EXPLAIN LOGICAL + SELECT * FROM ( + SELECT key, value FROM src UNION ALL SELECT key, value FROM srcpart WHERE ds IS NOT NULL + ) S1; +EXPLAIN LOGICAL + SELECT S1.key, S2.value FROM src S1 JOIN srcpart S2 ON S1.key = S2.key WHERE ds IS NOT NULL; + +-- With views +EXPLAIN LOGICAL SELECT * FROM V1; +EXPLAIN LOGICAL SELECT * FROM V2; +EXPLAIN LOGICAL SELECT * FROM V3; +EXPLAIN LOGICAL SELECT * FROM V4; + +-- The table should show up in the explain logical even if none +-- of the partitions are selected. +CREATE VIEW V5 as SELECT * FROM srcpart where ds = '10'; +EXPLAIN LOGICAL SELECT * FROM V5; + +EXPLAIN LOGICAL SELECT s1.key, s1.cnt, s2.value FROM (SELECT key, count(value) as cnt FROM src GROUP BY key) s1 JOIN src s2 ON (s1.key = s2.key) ORDER BY s1.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/explode_null.q b/src/test/hive/ql/src/test/queries/clientpositive/explode_null.q new file mode 100644 index 0000000000000..76e453525eeba --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/explode_null.q @@ -0,0 +1,10 @@ +SELECT explode(col) AS myCol FROM + (SELECT array(1,2,3) AS col FROM src LIMIT 1 + UNION ALL + SELECT IF(false, array(1,2,3), NULL) AS col FROM src LIMIT 1) a; + +SELECT explode(col) AS (myCol1,myCol2) FROM + (SELECT map(1,'one',2,'two',3,'three') AS col FROM src LIMIT 1 + UNION ALL + SELECT IF(false, map(1,'one',2,'two',3,'three'), NULL) AS col FROM src LIMIT 1) a; + \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/fetch_aggregation.q b/src/test/hive/ql/src/test/queries/clientpositive/fetch_aggregation.q new file mode 100644 index 0000000000000..618fea158aa65 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/fetch_aggregation.q @@ -0,0 +1,6 @@ +set hive.fetch.task.aggr=true; + +explain +select count(key),sum(key),avg(key),min(key),max(key),std(key),variance(key) from src; + +select count(key),sum(key),avg(key),min(key),max(key),std(key),variance(key) from src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/fileformat_mix.q b/src/test/hive/ql/src/test/queries/clientpositive/fileformat_mix.q new file mode 100644 index 0000000000000..1e0c164733fbe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/fileformat_mix.q @@ -0,0 +1,16 @@ + + +create table fileformat_mix_test (src int, value string) partitioned by (ds string); +alter table fileformat_mix_test set fileformat Sequencefile; + +insert overwrite table fileformat_mix_test partition (ds='1') +select key, value from src; + +alter table fileformat_mix_test add partition (ds='2'); + +alter table fileformat_mix_test set fileformat rcfile; + +select count(1) from fileformat_mix_test; + +select src from fileformat_mix_test; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/fileformat_sequencefile.q b/src/test/hive/ql/src/test/queries/clientpositive/fileformat_sequencefile.q new file mode 100644 index 0000000000000..895c70f8af4c2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/fileformat_sequencefile.q @@ -0,0 +1,17 @@ +EXPLAIN +CREATE TABLE dest1(key INT, value STRING) STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileOutputFormat'; + +CREATE TABLE dest1(key INT, value STRING) STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileOutputFormat'; + +DESCRIBE EXTENDED dest1; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10; + +SELECT dest1.* FROM dest1; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/fileformat_text.q b/src/test/hive/ql/src/test/queries/clientpositive/fileformat_text.q new file mode 100644 index 0000000000000..8233aded970c9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/fileformat_text.q @@ -0,0 +1,17 @@ +EXPLAIN +CREATE TABLE dest1(key INT, value STRING) STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'; + +CREATE TABLE dest1(key INT, value STRING) STORED AS + INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'; + +DESCRIBE EXTENDED dest1; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10; + +SELECT dest1.* FROM dest1; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask.q b/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask.q new file mode 100644 index 0000000000000..fe24da744112a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask.q @@ -0,0 +1,16 @@ + +CREATE TABLE filter_join_breaktask(key int, value string) partitioned by (ds string); + +INSERT OVERWRITE TABLE filter_join_breaktask PARTITION(ds='2008-04-08') +SELECT key, value from src1; + + +EXPLAIN EXTENDED +SELECT f.key, g.value +FROM filter_join_breaktask f JOIN filter_join_breaktask m ON( f.key = m.key AND f.ds='2008-04-08' AND m.ds='2008-04-08' AND f.key is not null) +JOIN filter_join_breaktask g ON(g.value = m.value AND g.ds='2008-04-08' AND m.ds='2008-04-08' AND m.value is not null AND m.value !=''); + +SELECT f.key, g.value +FROM filter_join_breaktask f JOIN filter_join_breaktask m ON( f.key = m.key AND f.ds='2008-04-08' AND m.ds='2008-04-08' AND f.key is not null) +JOIN filter_join_breaktask g ON(g.value = m.value AND g.ds='2008-04-08' AND m.ds='2008-04-08' AND m.value is not null AND m.value !=''); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask2.q b/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask2.q new file mode 100644 index 0000000000000..f8d855b25948a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask2.q @@ -0,0 +1,40 @@ + + + + + +create table T1(c1 string, c2 string, c3 string, c4 string, c5 string, c6 string, c7 string) +partitioned by (ds string); + +create table T2(c1 string, c2 string, c3 string, c0 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) partitioned by (ds string); + +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 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 T3 partition (ds='2010-04-17') select 4,5,0 from src limit 1; + +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; + +select * from T2; +select * from T1; +select * from T3; +select * from T4; + +SELECT a.c1 as a_c1, b.c1 b_c1, d.c0 as d_c0 +FROM T1 a JOIN T2 b + ON (a.c1 = b.c1 AND a.ds='2010-04-17' AND b.ds='2010-04-17') + JOIN T3 c + ON (a.c1 = c.c1 AND a.ds='2010-04-17' AND c.ds='2010-04-17') + JOIN T4 d + ON (c.c0 = d.c0 AND c.ds='2010-04-17' AND d.ds='2010-04-17'); + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/global_limit.q b/src/test/hive/ql/src/test/queries/clientpositive/global_limit.q new file mode 100644 index 0000000000000..b76cf34120701 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/global_limit.q @@ -0,0 +1,68 @@ +set hive.limit.optimize.enable=true; +set hive.limit.optimize.limit.file=2; + +drop table gl_tgt; +drop table gl_src1; +drop table gl_src2; +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; + + + + +set hive.limit.row.max.size=100; + +-- need one file +create table gl_tgt as select key from gl_src1 limit 1; +select * from gl_tgt ORDER BY key ASC; +-- need two files +select 'x' as key_new , split(value,',') as value_new from gl_src1 ORDER BY key_new ASC, value_new[0] ASC limit 20; +-- no sufficient files +select key, value, split(value,',') as value_new from gl_src1 ORDER BY key ASC, value ASC, value_new[0] ASC limit 30; +-- need all files +select key from gl_src1 ORDER BY key ASC limit 100; +set hive.limit.optimize.limit.file=4; +select key from gl_src1 ORDER BY key ASC limit 30; + +-- not qualified cases +select key, count(1) from gl_src1 group by key ORDER BY key ASC limit 5; +select distinct key from gl_src1 ORDER BY key ASC limit 10; +select count(1) from gl_src1 limit 1; +select transform(*) using "tr _ \n" as t from +(select "a_a_a_a_a_a_" from gl_src1 limit 100) subq ORDER BY t; +select key from (select * from (select key,value from gl_src1)t1 limit 10)t2 ORDER BY key ASC limit 2000; + +-- complicated queries +select key from (select * from (select key,value from gl_src1 limit 10)t1 )t2 ORDER BY key ASC; +select key from (select * from (select key,value from gl_src1)t1 limit 10)t2 ORDER BY key ASC; +insert overwrite table gl_tgt select key+1 from (select * from (select key,value from gl_src1)t1)t2 limit 10; +select * from gl_tgt ORDER BY key ASC; + +-- empty table +create table gl_src2 (key int, value string) stored as textfile; +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'); + +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; +select key from gl_src_part1 where p='12' ORDER BY key ASC limit 10; +select key from gl_src_part1 where p='13' ORDER BY key ASC limit 10; +alter table gl_src_part1 add partition (p='13'); +select key from gl_src_part1 where p='13' ORDER BY key ASC limit 10; +select key from gl_src_part1 where p='12' ORDER BY key ASC limit 1000; + +drop table gl_src1; +drop table gl_src2; +drop table gl_src_part1; +drop table gl_tgt; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby1.q new file mode 100755 index 0000000000000..1275eab281f42 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby1.q @@ -0,0 +1,15 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +CREATE TABLE dest_g1(key INT, value DOUBLE) STORED AS TEXTFILE; + +set fs.default.name=invalidscheme:///; + +EXPLAIN +FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; + +set fs.default.name=file:///; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby10.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby10.q new file mode 100644 index 0000000000000..db38d43fe4354 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby10.q @@ -0,0 +1,48 @@ +set hive.map.aggr=false; +set hive.multigroupby.singlereducer=false; +set hive.groupby.skewindata=true; + +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; + +EXPLAIN +FROM INPUT +INSERT OVERWRITE TABLE dest1 SELECT INPUT.key, count(substr(INPUT.value,5)), count(distinct substr(INPUT.value,5)) GROUP BY INPUT.key +INSERT OVERWRITE TABLE dest2 SELECT INPUT.key, sum(substr(INPUT.value,5)), sum(distinct substr(INPUT.value,5)) GROUP BY INPUT.key; + +FROM INPUT +INSERT OVERWRITE TABLE dest1 SELECT INPUT.key, count(substr(INPUT.value,5)), count(distinct substr(INPUT.value,5)) GROUP BY INPUT.key +INSERT OVERWRITE TABLE dest2 SELECT INPUT.key, sum(substr(INPUT.value,5)), sum(distinct substr(INPUT.value,5)) GROUP BY INPUT.key; + +SELECT * from dest1; +SELECT * from dest2; + +set hive.multigroupby.singlereducer=true; + +EXPLAIN +FROM INPUT +INSERT OVERWRITE TABLE dest1 SELECT INPUT.key, count(substr(INPUT.value,5)), count(distinct substr(INPUT.value,5)) GROUP BY INPUT.key +INSERT OVERWRITE TABLE dest2 SELECT INPUT.key, sum(substr(INPUT.value,5)), sum(distinct substr(INPUT.value,5)) GROUP BY INPUT.key; + +FROM INPUT +INSERT OVERWRITE TABLE dest1 SELECT INPUT.key, count(substr(INPUT.value,5)), count(distinct substr(INPUT.value,5)) GROUP BY INPUT.key +INSERT OVERWRITE TABLE dest2 SELECT INPUT.key, sum(substr(INPUT.value,5)), sum(distinct substr(INPUT.value,5)) GROUP BY INPUT.key; + +SELECT * from dest1; +SELECT * from dest2; + +-- HIVE-3852 Multi-groupby optimization fails when same distinct column is used twice or more +EXPLAIN +FROM INPUT +INSERT OVERWRITE TABLE dest1 SELECT INPUT.key, sum(distinct substr(INPUT.value,5)), count(distinct substr(INPUT.value,5)) GROUP BY INPUT.key +INSERT OVERWRITE TABLE dest2 SELECT INPUT.key, sum(distinct substr(INPUT.value,5)), avg(distinct substr(INPUT.value,5)) GROUP BY INPUT.key; + +FROM INPUT +INSERT OVERWRITE TABLE dest1 SELECT INPUT.key, sum(distinct substr(INPUT.value,5)), count(distinct substr(INPUT.value,5)) GROUP BY INPUT.key +INSERT OVERWRITE TABLE dest2 SELECT INPUT.key, sum(distinct substr(INPUT.value,5)), avg(distinct substr(INPUT.value,5)) GROUP BY INPUT.key; + +SELECT * from dest1; +SELECT * from dest2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby11.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby11.q new file mode 100644 index 0000000000000..0bf92ac4483d2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby11.q @@ -0,0 +1,27 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + + + + +CREATE TABLE dest1(key STRING, val1 INT, val2 INT) partitioned by (ds string); +CREATE TABLE dest2(key STRING, val1 INT, val2 INT) partitioned by (ds string); + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 partition(ds='111') + SELECT src.value, count(src.key), count(distinct src.key) GROUP BY src.value +INSERT OVERWRITE TABLE dest2 partition(ds='111') + SELECT substr(src.value, 5), count(src.key), count(distinct src.key) GROUP BY substr(src.value, 5); + +FROM src +INSERT OVERWRITE TABLE dest1 partition(ds='111') + SELECT src.value, count(src.key), count(distinct src.key) GROUP BY src.value +INSERT OVERWRITE TABLE dest2 partition(ds='111') + SELECT substr(src.value, 5), count(src.key), count(distinct src.key) GROUP BY substr(src.value, 5); + +SELECT * from dest1; +SELECT * from dest2; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1_limit.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby1_limit.q new file mode 100644 index 0000000000000..1b6891e33a37e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby1_limit.q @@ -0,0 +1,10 @@ +set mapred.reduce.tasks=31; + +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; + +SELECT dest1.* FROM dest1 ORDER BY dest1.key ASC , dest1.value ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map.q new file mode 100644 index 0000000000000..82cff36422e62 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map.q @@ -0,0 +1,12 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +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; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_nomap.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_nomap.q new file mode 100644 index 0000000000000..eb09a9c70e0c1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_nomap.q @@ -0,0 +1,12 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set hive.groupby.mapaggr.checkinterval=20; + +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; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_skew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_skew.q new file mode 100644 index 0000000000000..874995888b447 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_skew.q @@ -0,0 +1,12 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +set mapred.reduce.tasks=31; + +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; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1_noskew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby1_noskew.q new file mode 100644 index 0000000000000..1b10f1e024819 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby1_noskew.q @@ -0,0 +1,12 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest_g1(key INT, value DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby2.q new file mode 100755 index 0000000000000..80b50c39bb7cf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby2.q @@ -0,0 +1,14 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE; + +EXPLAIN +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); + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_limit.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_limit.q new file mode 100644 index 0000000000000..4bc263c77f1b4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_limit.q @@ -0,0 +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 LIMIT 5; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map.q new file mode 100644 index 0000000000000..c3cf598fb1c90 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map.q @@ -0,0 +1,14 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE; + +EXPLAIN +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); + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q new file mode 100644 index 0000000000000..25e6789b63ef2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q @@ -0,0 +1,14 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; + +EXPLAIN +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); + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_skew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_skew.q new file mode 100644 index 0000000000000..39a2a178e3a5e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_skew.q @@ -0,0 +1,14 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE; + +EXPLAIN +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); + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew.q new file mode 100644 index 0000000000000..c3c82d51749fe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew.q @@ -0,0 +1,14 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE; + +EXPLAIN +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); + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q new file mode 100644 index 0000000000000..b80c271afe5dc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q @@ -0,0 +1,14 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; + +EXPLAIN +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); + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby3.q new file mode 100755 index 0000000000000..8f24584a58e50 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby3.q @@ -0,0 +1,33 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)); + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)); + +SELECT dest1.* FROM dest1; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map.q new file mode 100644 index 0000000000000..7ecc71dfab64a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map.q @@ -0,0 +1,34 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)); + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)); + +SELECT dest1.* FROM dest1; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q new file mode 100644 index 0000000000000..50243beca9efa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q @@ -0,0 +1,36 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)), + sum(DISTINCT substr(src.value, 5)), + count(DISTINCT substr(src.value, 5)); + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)), + sum(DISTINCT substr(src.value, 5)), + count(DISTINCT substr(src.value, 5)); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_skew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_skew.q new file mode 100644 index 0000000000000..07d10c2d741d8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_skew.q @@ -0,0 +1,34 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)); + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)); + +SELECT dest1.* FROM dest1; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew.q new file mode 100644 index 0000000000000..d33f12c5744e9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew.q @@ -0,0 +1,36 @@ +set hive.map.aggr=false; + +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)); + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)); + +SELECT dest1.* FROM dest1; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q new file mode 100644 index 0000000000000..86d8986f1df7d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q @@ -0,0 +1,38 @@ +set hive.map.aggr=false; + +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)), + sum(DISTINCT substr(src.value, 5)), + count(DISTINCT substr(src.value, 5)); + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT + sum(substr(src.value,5)), + avg(substr(src.value,5)), + avg(DISTINCT substr(src.value,5)), + max(substr(src.value,5)), + min(substr(src.value,5)), + std(substr(src.value,5)), + stddev_samp(substr(src.value,5)), + variance(substr(src.value,5)), + var_samp(substr(src.value,5)), + sum(DISTINCT substr(src.value, 5)), + count(DISTINCT substr(src.value, 5)); + +SELECT dest1.* FROM dest1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby4.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby4.q new file mode 100755 index 0000000000000..ae5cae5b73e74 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby4.q @@ -0,0 +1,14 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1); + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1); + +SELECT dest1.* FROM dest1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map.q new file mode 100644 index 0000000000000..8ecce23eb8321 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map.q @@ -0,0 +1,12 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key INT) STORED AS TEXTFILE; + +EXPLAIN +FROM src INSERT OVERWRITE TABLE dest1 SELECT count(1); + +FROM src INSERT OVERWRITE TABLE dest1 SELECT count(1); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map_skew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map_skew.q new file mode 100644 index 0000000000000..eb2001c6b21b0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map_skew.q @@ -0,0 +1,12 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key INT) STORED AS TEXTFILE; + +EXPLAIN +FROM src INSERT OVERWRITE TABLE dest1 SELECT count(1); + +FROM src INSERT OVERWRITE TABLE dest1 SELECT count(1); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby4_noskew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby4_noskew.q new file mode 100644 index 0000000000000..99c2d2d5a1784 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby4_noskew.q @@ -0,0 +1,16 @@ +set hive.map.aggr=false; + +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1); + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1); + +SELECT dest1.* FROM dest1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby5.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby5.q new file mode 100755 index 0000000000000..0909c90bc4137 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby5.q @@ -0,0 +1,18 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +INSERT OVERWRITE TABLE dest1 +SELECT src.key, sum(substr(src.value,5)) +FROM src +GROUP BY src.key; + +INSERT OVERWRITE TABLE dest1 +SELECT src.key, sum(substr(src.value,5)) +FROM src +GROUP BY src.key; + +SELECT dest1.* FROM dest1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map.q new file mode 100644 index 0000000000000..4fd6445d7927c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map.q @@ -0,0 +1,12 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key INT) STORED AS TEXTFILE; + +EXPLAIN +FROM src INSERT OVERWRITE TABLE dest1 SELECT sum(src.key); + +FROM src INSERT OVERWRITE TABLE dest1 SELECT sum(src.key); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map_skew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map_skew.q new file mode 100644 index 0000000000000..eccd45dd5b422 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map_skew.q @@ -0,0 +1,12 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key INT) STORED AS TEXTFILE; + +EXPLAIN +FROM src INSERT OVERWRITE TABLE dest1 SELECT sum(src.key); + +FROM src INSERT OVERWRITE TABLE dest1 SELECT sum(src.key); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby5_noskew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby5_noskew.q new file mode 100644 index 0000000000000..be60785d87ea5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby5_noskew.q @@ -0,0 +1,20 @@ +set hive.map.aggr=false; + +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +INSERT OVERWRITE TABLE dest1 +SELECT src.key, sum(substr(src.value,5)) +FROM src +GROUP BY src.key; + +INSERT OVERWRITE TABLE dest1 +SELECT src.key, sum(substr(src.value,5)) +FROM src +GROUP BY src.key; + +SELECT dest1.* FROM dest1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby6.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby6.q new file mode 100755 index 0000000000000..3a3cc58e0787f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby6.q @@ -0,0 +1,15 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +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; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map.q new file mode 100644 index 0000000000000..fbf761c3aea55 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map.q @@ -0,0 +1,16 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +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; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map_skew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map_skew.q new file mode 100644 index 0000000000000..ac79a286055d5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map_skew.q @@ -0,0 +1,16 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +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; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby6_noskew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby6_noskew.q new file mode 100644 index 0000000000000..2c99d362ffff7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby6_noskew.q @@ -0,0 +1,17 @@ +set hive.map.aggr=false; + +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +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; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby7.q new file mode 100644 index 0000000000000..1235e3c0a0ba5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby7.q @@ -0,0 +1,15 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; + +SET hive.exec.compress.intermediate=true; +SET hive.exec.compress.output=true; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map.q new file mode 100644 index 0000000000000..b1457d9349aee --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map.q @@ -0,0 +1,22 @@ +set hive.map.aggr=true; +set hive.multigroupby.singlereducer=false; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; + +SET hive.exec.compress.intermediate=true; +SET hive.exec.compress.output=true; + +EXPLAIN +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q new file mode 100644 index 0000000000000..481b3cd084f16 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q @@ -0,0 +1,21 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; + +SET hive.exec.compress.intermediate=true; +SET hive.exec.compress.output=true; + +EXPLAIN +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_skew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_skew.q new file mode 100644 index 0000000000000..a34ac8f64b117 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_skew.q @@ -0,0 +1,21 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +set mapred.reduce.tasks=31; + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; + +SET hive.exec.compress.intermediate=true; +SET hive.exec.compress.output=true; + +EXPLAIN +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew.q new file mode 100644 index 0000000000000..94a3dcf7ec3f2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew.q @@ -0,0 +1,22 @@ +set hive.map.aggr=false; +set hive.multigroupby.singlereducer=false; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; + +SET hive.exec.compress.intermediate=true; +SET hive.exec.compress.output=true; + +EXPLAIN +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q new file mode 100644 index 0000000000000..802aea244da9e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q @@ -0,0 +1,21 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; + +SET hive.exec.compress.intermediate=true; +SET hive.exec.compress.output=true; + +EXPLAIN +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; + +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; + +SELECT DEST1.* FROM DEST1 ORDER BY key ASC, value ASC; +SELECT DEST2.* FROM DEST2 ORDER BY key ASC, value ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby8.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby8.q new file mode 100644 index 0000000000000..e7b08b274d82c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby8.q @@ -0,0 +1,31 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +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; + +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; + +set hive.multigroupby.singlereducer=false; + +EXPLAIN +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map.q new file mode 100644 index 0000000000000..62b6ff5ddebb6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map.q @@ -0,0 +1,19 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +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; + +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; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map_skew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map_skew.q new file mode 100644 index 0000000000000..846fd01017172 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map_skew.q @@ -0,0 +1,19 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +set mapred.reduce.tasks=31; + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +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; + +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; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby8_noskew.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby8_noskew.q new file mode 100644 index 0000000000000..20c8bef34223a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby8_noskew.q @@ -0,0 +1,19 @@ +set hive.map.aggr=false; + +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby9.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby9.q new file mode 100644 index 0000000000000..afd40c9ddbaa1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby9.q @@ -0,0 +1,67 @@ + +CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key INT, val1 STRING, val2 STRING) STORED AS TEXTFILE; + +EXPLAIN +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, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key, SRC.value; + +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, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key, SRC.value; + +SELECT DEST1.* FROM DEST1; +SELECT DEST2.* FROM DEST2; + +EXPLAIN +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, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.value, SRC.key; + +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, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.value, SRC.key; + +SELECT DEST1.* FROM DEST1; +SELECT DEST2.* FROM DEST2; + +set hive.multigroupby.singlereducer=false; + +EXPLAIN +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, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key, SRC.value; + +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, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key, SRC.value; + +SELECT DEST1.* FROM DEST1; +SELECT DEST2.* FROM DEST2; + +EXPLAIN +FROM SRC +INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(SUBSTR(SRC.value,5)) GROUP BY SRC.key +INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, SRC.value, COUNT(SUBSTR(SRC.value,5)) GROUP BY SRC.key, SRC.value; + +FROM SRC +INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(SUBSTR(SRC.value,5)) GROUP BY SRC.key +INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, SRC.value, COUNT(SUBSTR(SRC.value,5)) GROUP BY SRC.key, SRC.value; + +SELECT DEST1.* FROM DEST1; +SELECT DEST2.* FROM DEST2; + +EXPLAIN +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, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.value, SRC.key; + +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, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.value, SRC.key; + +SELECT DEST1.* FROM DEST1; +SELECT DEST2.* FROM DEST2; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_bigdata.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_bigdata.q new file mode 100644 index 0000000000000..7e97f75cecc92 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_bigdata.q @@ -0,0 +1,7 @@ +set hive.map.aggr.hash.percentmemory = 0.3; +set hive.mapred.local.mem = 384; + +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/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types.q new file mode 100644 index 0000000000000..bb1e6d269710a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types.q @@ -0,0 +1,19 @@ +CREATE TABLE DEST1(key ARRAY, value BIGINT) STORED AS TEXTFILE; +CREATE TABLE DEST2(key MAP, value BIGINT) STORED AS TEXTFILE; +CREATE TABLE DEST3(key STRUCT, value BIGINT) STORED AS TEXTFILE; + +EXPLAIN +FROM SRC +INSERT OVERWRITE TABLE DEST1 SELECT ARRAY(SRC.key), COUNT(1) GROUP BY ARRAY(SRC.key) +INSERT OVERWRITE TABLE DEST2 SELECT MAP(SRC.key, SRC.value), COUNT(1) GROUP BY MAP(SRC.key, SRC.value) +INSERT OVERWRITE TABLE DEST3 SELECT STRUCT(SRC.key, SRC.value), COUNT(1) GROUP BY STRUCT(SRC.key, SRC.value); + +FROM SRC +INSERT OVERWRITE TABLE DEST1 SELECT ARRAY(SRC.key), COUNT(1) GROUP BY ARRAY(SRC.key) +INSERT OVERWRITE TABLE DEST2 SELECT MAP(SRC.key, SRC.value), COUNT(1) GROUP BY MAP(SRC.key, SRC.value) +INSERT OVERWRITE TABLE DEST3 SELECT STRUCT(SRC.key, SRC.value), COUNT(1) GROUP BY STRUCT(SRC.key, SRC.value); + +SELECT DEST1.* FROM DEST1; +SELECT DEST2.* FROM DEST2; +SELECT DEST3.* FROM DEST3; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q new file mode 100644 index 0000000000000..22c5a389a957a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q @@ -0,0 +1,17 @@ +set hive.multigroupby.singlereducer=true; + +CREATE TABLE DEST1(key ARRAY, value BIGINT) STORED AS TEXTFILE; +CREATE TABLE DEST2(key MAP, value BIGINT) STORED AS TEXTFILE; + +EXPLAIN +FROM SRC +INSERT OVERWRITE TABLE DEST1 SELECT ARRAY(SRC.key), COUNT(1) GROUP BY ARRAY(SRC.key) limit 10 +INSERT OVERWRITE TABLE DEST2 SELECT MAP(SRC.key, SRC.value), COUNT(1) GROUP BY MAP(SRC.key, SRC.value) limit 10; + +FROM SRC +INSERT OVERWRITE TABLE DEST1 SELECT ARRAY(SRC.key), COUNT(1) GROUP BY ARRAY(SRC.key) limit 10 +INSERT OVERWRITE TABLE DEST2 SELECT MAP(SRC.key, SRC.value), COUNT(1) GROUP BY MAP(SRC.key, SRC.value) limit 10; + +SELECT DEST1.* FROM DEST1 ORDER BY key[0] ASC, value ASC; +SELECT DEST2.* FROM DEST2 ORDER BY 1 ASC, value ASC; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_cube1.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_cube1.q new file mode 100644 index 0000000000000..46e1f00d0f224 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_cube1.q @@ -0,0 +1,49 @@ +set hive.map.aggr=true; +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; + +EXPLAIN +SELECT key, val, count(1) FROM T1 GROUP BY key, val with cube; + +SELECT key, val, count(1) FROM T1 GROUP BY key, val with cube +ORDER BY key, val; + +EXPLAIN +SELECT key, count(distinct val) FROM T1 GROUP BY key with cube; + +SELECT key, count(distinct val) FROM T1 GROUP BY key with cube +ORDER BY key; + +set hive.groupby.skewindata=true; + +EXPLAIN +SELECT key, val, count(1) FROM T1 GROUP BY key, val with cube; + +SELECT key, val, count(1) FROM T1 GROUP BY key, val with cube +ORDER BY key, val; + +EXPLAIN +SELECT key, count(distinct val) FROM T1 GROUP BY key with cube; + +SELECT key, count(distinct val) FROM T1 GROUP BY key with cube +ORDER BY key; + + +set hive.multigroupby.singlereducer=true; + +CREATE TABLE T2(key1 STRING, key2 STRING, val INT) STORED AS TEXTFILE; +CREATE TABLE T3(key1 STRING, key2 STRING, val INT) STORED AS TEXTFILE; + +EXPLAIN +FROM T1 +INSERT OVERWRITE TABLE T2 SELECT key, val, count(1) group by key, val with cube +INSERT OVERWRITE TABLE T3 SELECT key, val, sum(1) group by key, val with cube; + + +FROM T1 +INSERT OVERWRITE TABLE T2 SELECT key, val, count(1) group by key, val with cube +INSERT OVERWRITE TABLE T3 SELECT key, val, sum(1) group by key, val with cube; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q new file mode 100644 index 0000000000000..8219a5efb10de --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q @@ -0,0 +1,15 @@ +-- This test covers HIVE-2332 + +create table t1 (int1 int, int2 int, str1 string, str2 string); + +set hive.optimize.reducededuplication=false; +--disabled RS-dedup for keeping intention of test + +insert into table t1 select cast(key as int), cast(key as int), value, value from src where key < 6; +explain select Q1.int1, sum(distinct Q1.int1) from (select * from t1 order by int1) Q1 group by Q1.int1; +explain select int1, sum(distinct int1) from t1 group by int1; + +select Q1.int1, sum(distinct Q1.int1) from (select * from t1 order by int1) Q1 group by Q1.int1; +select int1, sum(distinct int1) from t1 group by int1; + +drop table t1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id1.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id1.q new file mode 100644 index 0000000000000..bced21f9e494a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id1.q @@ -0,0 +1,10 @@ +CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; + +SELECT key, val, GROUPING__ID from T1 group by key, val with cube; + +SELECT GROUPING__ID, key, val from T1 group by key, val with rollup; + +SELECT key, val, GROUPING__ID, CASE WHEN GROUPING__ID == 0 THEN "0" WHEN GROUPING__ID == 1 THEN "1" WHEN GROUPING__ID == 2 THEN "2" WHEN GROUPING__ID == 3 THEN "3" ELSE "nothing" END from T1 group by key, val with cube; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id2.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id2.q new file mode 100644 index 0000000000000..ffc627c82eaff --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id2.q @@ -0,0 +1,41 @@ +CREATE TABLE T1(key INT, value INT) STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/groupby_groupingid.txt' INTO TABLE T1; + +set hive.groupby.skewindata = true; + +SELECT key, value, GROUPING__ID, count(*) from T1 GROUP BY key, value WITH ROLLUP; + +SELECT GROUPING__ID, count(*) +FROM +( +SELECT key, value, GROUPING__ID, count(*) from T1 GROUP BY key, value WITH ROLLUP +) t +GROUP BY GROUPING__ID; + +SELECT t1.GROUPING__ID, t2.GROUPING__ID FROM (SELECT GROUPING__ID FROM T1 GROUP BY key,value WITH ROLLUP) t1 +JOIN +(SELECT GROUPING__ID FROM T1 GROUP BY key, value WITH ROLLUP) t2 +ON t1.GROUPING__ID = t2.GROUPING__ID; + + + + + +set hive.groupby.skewindata = false; + +SELECT key, value, GROUPING__ID, count(*) from T1 GROUP BY key, value WITH ROLLUP; + +SELECT GROUPING__ID, count(*) +FROM +( +SELECT key, value, GROUPING__ID, count(*) from T1 GROUP BY key, value WITH ROLLUP +) t +GROUP BY GROUPING__ID; + +SELECT t1.GROUPING__ID, t2.GROUPING__ID FROM (SELECT GROUPING__ID FROM T1 GROUP BY key,value WITH ROLLUP) t1 +JOIN +(SELECT GROUPING__ID FROM T1 GROUP BY key, value WITH ROLLUP) t2 +ON t1.GROUPING__ID = t2.GROUPING__ID; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q new file mode 100644 index 0000000000000..4fba7338f1d0a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q @@ -0,0 +1,18 @@ +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; + +SELECT * FROM T1; + +SELECT a, b, count(*) from T1 group by a, b with cube; + +SELECT a, b, count(*) FROM T1 GROUP BY a, b GROUPING SETS (a, (a, b), b, ()); + +SELECT a, b, count(*) FROM T1 GROUP BY a, b GROUPING SETS (a, (a, b)); + +SELECT a FROM T1 GROUP BY a, b, c GROUPING SETS (a, b, c); + +SELECT a FROM T1 GROUP BY a GROUPING SETS ((a), (a)); + +SELECT a + b, count(*) FROM T1 GROUP BY a + b GROUPING SETS (a+b); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q new file mode 100644 index 0000000000000..9f2286cc9be31 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q @@ -0,0 +1,23 @@ +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; + +-- Since 4 grouping sets would be generated for the query below, an additional MR job should be created +EXPLAIN +SELECT a, b, count(*) from T1 group by a, b with cube; +SELECT a, b, count(*) from T1 group by a, b with cube; + +EXPLAIN +SELECT a, b, sum(c) from T1 group by a, b with cube; +SELECT a, b, sum(c) from T1 group by a, b with cube; + +CREATE TABLE T2(a STRING, b STRING, c int, d int); + +INSERT OVERWRITE TABLE T2 +SELECT a, b, c, c from T1; + +EXPLAIN +SELECT a, b, sum(c+d) from T2 group by a, b with cube; +SELECT a, b, sum(c+d) from T2 group by a, b with cube; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q new file mode 100644 index 0000000000000..9a00d0a7aa77f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q @@ -0,0 +1,27 @@ +-- In this test, 2 files are loaded into table T1. The data contains rows with the same value of a and b, +-- with different number of rows for a and b in each file. Since bucketizedHiveInputFormat is used, +-- this tests that the aggregate function stores the partial aggregate state correctly even if an +-- 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; + +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.new.job.grouping.set.cardinality = 30; + +-- The query below will execute in a single MR job, since 4 rows are generated per input row +-- (cube of a,b will lead to (a,b), (a, null), (null, b) and (null, null) and +-- hive.new.job.grouping.set.cardinality is more than 4. +EXPLAIN +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube; +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube; + +set hive.new.job.grouping.set.cardinality=2; + +-- The query below will execute in 2 MR jobs, since hive.new.job.grouping.set.cardinality is set to 2. +-- The partial aggregation state should be maintained correctly across MR jobs. +EXPLAIN +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube; +SELECT a, b, avg(c), count(*) from T1 group by a, b with cube; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q new file mode 100644 index 0000000000000..25f1fcd793a74 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q @@ -0,0 +1,39 @@ +set hive.merge.mapfiles = false; +set hive.merge.mapredfiles = false; +-- Set merging to false above to make the explain more readable + +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; + +-- This tests that cubes and rollups work fine inside sub-queries. +EXPLAIN +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a order by subq1.a, subq1.b, subq2.a, subq2.b; + +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a order by subq1.a, subq1.b, subq2.a, subq2.b; + +set hive.new.job.grouping.set.cardinality=2; + +-- Since 4 grouping sets would be generated for each sub-query, an additional MR job should be created +-- for each of them +EXPLAIN +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a order by subq1.a, subq1.b, subq2.a, subq2.b; + +SELECT * FROM +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq1 +join +(SELECT a, b, count(*) from T1 where a < 3 group by a, b with cube) subq2 +on subq1.a = subq2.a order by subq1.a, subq1.b, subq2.a, subq2.b; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q new file mode 100644 index 0000000000000..fb0c5913fa07f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q @@ -0,0 +1,25 @@ +set hive.merge.mapfiles = false; +set hive.merge.mapredfiles = false; +-- Set merging to false above to make the explain more readable + +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; + +-- This tests that cubes and rollups work fine where the source is a sub-query +EXPLAIN +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube; + +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube; + +set hive.new.job.grouping.set.cardinality=2; + +-- Since 4 grouping sets would be generated for the cube, an additional MR job should be created +EXPLAIN +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube; + +SELECT a, b, count(*) FROM +(SELECT a, b, count(1) from T1 group by a, b) subq1 group by a, b with cube; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr.q new file mode 100644 index 0000000000000..f0a8b72b0c83e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr.q @@ -0,0 +1,20 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE; + +EXPLAIN EXTENDED +FROM srcpart 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))) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1); + +FROM srcpart 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))) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q new file mode 100644 index 0000000000000..b863344485d9f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q @@ -0,0 +1,20 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +set mapred.reduce.tasks=31; + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, C3 INT, c4 INT) STORED AS TEXTFILE; + +EXPLAIN EXTENDED +FROM srcpart 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(DISTINCT src.value) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1); + +FROM srcpart 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(DISTINCT src.value) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q new file mode 100644 index 0000000000000..aa1c48939a766 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q @@ -0,0 +1,32 @@ +set hive.map.aggr=true; + +create table dest1(key int, cnt int); +create table dest2(key int, cnt int); + +explain +from src +insert overwrite table dest1 select key, count(distinct value) group by key +insert overwrite table dest2 select key+key, count(distinct value) group by key+key; + +from src +insert overwrite table dest1 select key, count(distinct value) group by key +insert overwrite table dest2 select key+key, count(distinct value) group by key+key; + + +select * from dest1 where key < 10 order by key; +select * from dest2 where key < 20 order by key limit 10; + +set hive.optimize.multigroupby.common.distincts=false; + +-- no need to spray by distinct key first +explain +from src +insert overwrite table dest1 select key, count(distinct value) group by key +insert overwrite table dest2 select key+key, count(distinct value) group by key+key; + +from src +insert overwrite table dest1 select key, count(distinct value) group by key +insert overwrite table dest2 select key+key, count(distinct value) group by key+key; + +select * from dest1 where key < 10 order by key; +select * from dest2 where key < 20 order by key limit 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q new file mode 100644 index 0000000000000..145aab87f1877 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q @@ -0,0 +1,49 @@ +set hive.multigroupby.singlereducer=true; + +CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; +CREATE TABLE dest_g3(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; +CREATE TABLE dest_g4(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; +CREATE TABLE dest_h2(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; +CREATE TABLE dest_h3(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; + +EXPLAIN +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) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1) +INSERT OVERWRITE TABLE dest_g3 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) WHERE substr(src.key,1,1) < 5 GROUP BY substr(src.key,1,1) +INSERT OVERWRITE TABLE dest_g4 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); + +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) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1) +INSERT OVERWRITE TABLE dest_g3 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) WHERE substr(src.key,1,1) < 5 GROUP BY substr(src.key,1,1) +INSERT OVERWRITE TABLE dest_g4 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 * FROM dest_g2 ORDER BY key ASC, c1 ASC, c2 ASC, c3 ASC, c4 ASC; +SELECT * FROM dest_g3 ORDER BY key ASC, c1 ASC, c2 ASC, c3 ASC, c4 ASC; +SELECT * FROM dest_g4 ORDER BY key ASC, c1 ASC, c2 ASC, c3 ASC, c4 ASC; + +EXPLAIN +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) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1) +INSERT OVERWRITE TABLE dest_g3 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) WHERE substr(src.key,1,1) < 5 GROUP BY substr(src.key,1,1) +INSERT OVERWRITE TABLE dest_g4 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) +INSERT OVERWRITE TABLE dest_h2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1), substr(src.key,2,1) LIMIT 10 +INSERT OVERWRITE TABLE dest_h3 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(substr(src.value, 5)), count(src.value) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1), substr(src.key,2,1); + +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) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1) +INSERT OVERWRITE TABLE dest_g3 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) WHERE substr(src.key,1,1) < 5 GROUP BY substr(src.key,1,1) +INSERT OVERWRITE TABLE dest_g4 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) +INSERT OVERWRITE TABLE dest_h2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1), substr(src.key,2,1) LIMIT 10 +INSERT OVERWRITE TABLE dest_h3 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(substr(src.value, 5)), count(src.value) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1), substr(src.key,2,1); + +SELECT * FROM dest_g2 ORDER BY key ASC, c1 ASC, c2 ASC, c3 ASC, c4 ASC; +SELECT * FROM dest_g3 ORDER BY key ASC, c1 ASC, c2 ASC, c3 ASC, c4 ASC; +SELECT * FROM dest_g4 ORDER BY key ASC, c1 ASC, c2 ASC, c3 ASC, c4 ASC; +SELECT * FROM dest_h2 ORDER BY key ASC, c1 ASC, c2 ASC, c3 ASC, c4 ASC; +SELECT * FROM dest_h3 ORDER BY key ASC, c1 ASC, c2 ASC, c3 ASC, c4 ASC; + +DROP TABLE dest_g2; +DROP TABLE dest_g3; +DROP TABLE dest_g4; +DROP TABLE dest_h2; +DROP TABLE dest_h3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q new file mode 100644 index 0000000000000..b58f9d3ade441 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q @@ -0,0 +1,19 @@ +set hive.multigroupby.singlereducer=true; + +CREATE TABLE dest_g2(key STRING, c1 INT) STORED AS TEXTFILE; +CREATE TABLE dest_g3(key STRING, c1 INT, c2 INT) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT src.key) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1) +INSERT OVERWRITE TABLE dest_g3 SELECT substr(src.key,1,1), count(DISTINCT src.key), count(src.value) WHERE substr(src.key,1,1) < 5 GROUP BY substr(src.key,1,1); + +FROM src +INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT src.key) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1) +INSERT OVERWRITE TABLE dest_g3 SELECT substr(src.key,1,1), count(DISTINCT src.key), count(src.value) WHERE substr(src.key,1,1) < 5 GROUP BY substr(src.key,1,1); + +SELECT * FROM dest_g2; +SELECT * FROM dest_g3; + +DROP TABLE dest_g2; +DROP TABLE dest_g3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q new file mode 100644 index 0000000000000..1253ddfed52a7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q @@ -0,0 +1,101 @@ +-- HIVE-3849 Aliased column in where clause for multi-groupby single reducer cannot be resolved +create table e1 (key string, count int); +create table e2 (key string, count int); + +explain +from src +insert overwrite table e1 +select key, count(*) +where src.value in ('val_100', 'val_200', 'val_300') AND key in (100, 150, 200) +group by key +insert overwrite table e2 +select key, count(*) +where src.value in ('val_400', 'val_500') AND key in (400, 450) +group by key; + +from src +insert overwrite table e1 +select key, count(*) +where src.value in ('val_100', 'val_200', 'val_300') AND key in (100, 150, 200) +group by key +insert overwrite table e2 +select key, count(*) +where src.value in ('val_400', 'val_500') AND key in (400, 450) +group by key; + +select * from e1; +select * from e2; + +explain +from src +insert overwrite table e1 +select value, count(*) +where src.key + src.key = 200 or src.key - 100 = 100 or src.key = 300 AND VALUE IS NOT NULL +group by value +insert overwrite table e2 +select value, count(*) +where src.key + src.key = 400 or src.key - 100 = 500 AND VALUE IS NOT NULL +group by value; + +from src +insert overwrite table e1 +select value, count(*) +where src.key + src.key = 200 or src.key - 100 = 100 or src.key = 300 AND VALUE IS NOT NULL +group by value +insert overwrite table e2 +select value, count(*) +where src.key + src.key = 400 or src.key - 100 = 500 AND VALUE IS NOT NULL +group by value; + +select * from e1; +select * from e2; + +set hive.optimize.ppd=false; + +explain +from src +insert overwrite table e1 +select key, count(*) +where src.value in ('val_100', 'val_200', 'val_300') AND key in (100, 150, 200) +group by key +insert overwrite table e2 +select key, count(*) +where src.value in ('val_400', 'val_500') AND key in (400, 450) +group by key; + +from src +insert overwrite table e1 +select key, count(*) +where src.value in ('val_100', 'val_200', 'val_300') AND key in (100, 150, 200) +group by key +insert overwrite table e2 +select key, count(*) +where src.value in ('val_400', 'val_500') AND key in (400, 450) +group by key; + +select * from e1; +select * from e2; + +explain +from src +insert overwrite table e1 +select value, count(*) +where src.key + src.key = 200 or src.key - 100 = 100 or src.key = 300 AND VALUE IS NOT NULL +group by value +insert overwrite table e2 +select value, count(*) +where src.key + src.key = 400 or src.key - 100 = 500 AND VALUE IS NOT NULL +group by value; + +from src +insert overwrite table e1 +select value, count(*) +where src.key + src.key = 200 or src.key - 100 = 100 or src.key = 300 AND VALUE IS NOT NULL +group by value +insert overwrite table e2 +select value, count(*) +where src.key + src.key = 400 or src.key - 100 = 500 AND VALUE IS NOT NULL +group by value; + +select * from e1; +select * from e2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_neg_float.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_neg_float.q new file mode 100644 index 0000000000000..0e41131861b7d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_neg_float.q @@ -0,0 +1,10 @@ +FROM src +SELECT cast('-30.33' as DOUBLE) +GROUP BY cast('-30.33' as DOUBLE) +LIMIT 1; + + +FROM src +SELECT '-30.33' +GROUP BY '-30.33' +LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_position.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_position.q new file mode 100644 index 0000000000000..4316d4a63ab82 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_position.q @@ -0,0 +1,72 @@ +set hive.groupby.orderby.position.alias=true; + +CREATE TABLE testTable1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE testTable2(key INT, val1 STRING, val2 STRING) STORED AS TEXTFILE; + +-- Position Alias in GROUP BY and ORDER BY + +EXPLAIN +FROM SRC +INSERT OVERWRITE TABLE testTable1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 1 +INSERT OVERWRITE TABLE testTable2 SELECT SRC.key, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 1, 2; + +FROM SRC +INSERT OVERWRITE TABLE testTable1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 1 +INSERT OVERWRITE TABLE testTable2 SELECT SRC.key, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 1, 2; + +SELECT key, value FROM testTable1 ORDER BY 1, 2; +SELECT key, val1, val2 FROM testTable2 ORDER BY 1, 2, 3; + +EXPLAIN +FROM SRC +INSERT OVERWRITE TABLE testTable1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 1 +INSERT OVERWRITE TABLE testTable2 SELECT SRC.key, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 2, 1; + +FROM SRC +INSERT OVERWRITE TABLE testTable1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 1 +INSERT OVERWRITE TABLE testTable2 SELECT SRC.key, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 2, 1; + +SELECT key, value FROM testTable1 ORDER BY 1, 2; +SELECT key, val1, val2 FROM testTable2 ORDER BY 1, 2, 3; + +-- Position Alias in subquery + +EXPLAIN +SELECT t.key, t.value +FROM (SELECT b.key as key, count(1) as value FROM src b WHERE b.key <= 20 GROUP BY 1) t +ORDER BY 2 DESC, 1 ASC; + +SELECT t.key, t.value +FROM (SELECT b.key as key, count(1) as value FROM src b WHERE b.key <= 20 GROUP BY 1) t +ORDER BY 2 DESC, 1 ASC; + +EXPLAIN +SELECT c1, c2, c3, c4 +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2, COUNT(DISTINCT SUBSTR(src1.value,5)) AS c3 WHERE src1.key > 10 and src1.key < 20 GROUP BY 1, 2 + ) a + JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 GROUP BY 1, 2 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +ORDER BY 1 DESC, 2 DESC, 3 ASC, 4 ASC; + +SELECT c1, c2, c3, c4 +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2, COUNT(DISTINCT SUBSTR(src1.value,5)) AS c3 WHERE src1.key > 10 and src1.key < 20 GROUP BY 1, 2 + ) a + JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 GROUP BY 1, 2 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +ORDER BY 1 DESC, 2 DESC, 3 ASC, 4 ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppd.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppd.q new file mode 100644 index 0000000000000..4cda623b80af8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppd.q @@ -0,0 +1,4 @@ +-- see HIVE-2382 +create table invites (id int, foo int, bar int); +explain select * from (select foo, bar from (select bar, foo from invites c union all select bar, foo from invites d) b) a group by bar, foo having bar=1; +drop table invites; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr.q new file mode 100644 index 0000000000000..1f588eec2477c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr.q @@ -0,0 +1,19 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=false; + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE; + +EXPLAIN EXTENDED +FROM srcpart 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))) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1); + +FROM srcpart 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))) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q new file mode 100644 index 0000000000000..20c73bd306648 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q @@ -0,0 +1,19 @@ +set hive.map.aggr=false; +set hive.groupby.skewindata=false; + +CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; + +EXPLAIN EXTENDED +FROM srcpart 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(DISTINCT src.value) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1); + +FROM srcpart 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(DISTINCT src.value) +WHERE src.ds = '2008-04-08' +GROUP BY substr(src.key,1,1); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_rollup1.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_rollup1.q new file mode 100644 index 0000000000000..f79b0c472ebb1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_rollup1.q @@ -0,0 +1,49 @@ +set hive.map.aggr=true; +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; + +EXPLAIN +SELECT key, val, count(1) FROM T1 GROUP BY key, val with rollup; + +SELECT key, val, count(1) FROM T1 GROUP BY key, val with rollup +ORDER BY key, val; + +EXPLAIN +SELECT key, count(distinct val) FROM T1 GROUP BY key with rollup; + +SELECT key, count(distinct val) FROM T1 GROUP BY key with rollup +ORDER BY key; + +set hive.groupby.skewindata=true; + +EXPLAIN +SELECT key, val, count(1) FROM T1 GROUP BY key, val with rollup; + +SELECT key, val, count(1) FROM T1 GROUP BY key, val with rollup +ORDER BY key, val; + +EXPLAIN +SELECT key, count(distinct val) FROM T1 GROUP BY key with rollup; + +SELECT key, count(distinct val) FROM T1 GROUP BY key with rollup +ORDER BY key; + + +set hive.multigroupby.singlereducer=true; + +CREATE TABLE T2(key1 STRING, key2 STRING, val INT) STORED AS TEXTFILE; +CREATE TABLE T3(key1 STRING, key2 STRING, val INT) STORED AS TEXTFILE; + +EXPLAIN +FROM T1 +INSERT OVERWRITE TABLE T2 SELECT key, val, count(1) group by key, val with rollup +INSERT OVERWRITE TABLE T3 SELECT key, val, sum(1) group by key, val with rollup; + + +FROM T1 +INSERT OVERWRITE TABLE T2 SELECT key, val, count(1) group by key, val with rollup +INSERT OVERWRITE TABLE T3 SELECT key, val, sum(1) group by key, val with rollup; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_1.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_1.q new file mode 100644 index 0000000000000..911a11ae8990f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_1.q @@ -0,0 +1,282 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 select key, val from T1; + +CREATE TABLE outputTbl1(key int, cnt int); + +-- The plan should be converted to a map-side group by if the group by key +-- matches the sorted key +-- addind a order by at the end to make the test results deterministic +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 GROUP BY key; + +SELECT * FROM outputTbl1 ORDER BY key; + +CREATE TABLE outputTbl2(key1 int, key2 string, cnt int); + +-- no map-side group by even if the group by key is a superset of sorted key +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl2 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +INSERT OVERWRITE TABLE outputTbl2 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +SELECT * FROM outputTbl2 ORDER BY key1, key2; + +-- It should work for sub-queries +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM (SELECT key, val FROM T1) subq1 GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM (SELECT key, val FROM T1) subq1 GROUP BY key; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- It should work for sub-queries with column aliases +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT k, count(1) FROM (SELECT key as k, val as v FROM T1) subq1 GROUP BY k; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT k, count(1) FROM (SELECT key as k, val as v FROM T1) subq1 GROUP BY k; + +SELECT * FROM outputTbl1 ORDER BY key; + +CREATE TABLE outputTbl3(key1 int, key2 int, cnt int); + +-- The plan should be converted to a map-side group by if the group by key contains a constant followed +-- by a match to the sorted key +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl3 +SELECT 1, key, count(1) FROM T1 GROUP BY 1, key; + +INSERT OVERWRITE TABLE outputTbl3 +SELECT 1, key, count(1) FROM T1 GROUP BY 1, key; + +SELECT * FROM outputTbl3 ORDER BY key1, key2; + +CREATE TABLE outputTbl4(key1 int, key2 int, key3 string, cnt int); + +-- no map-side group by if the group by key contains a constant followed by another column +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val; + +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val; + +SELECT * FROM outputTbl4 ORDER BY key1, key2, key3; + +-- no map-side group by if the group by key contains a function +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl3 +SELECT key, key + 1, count(1) FROM T1 GROUP BY key, key + 1; + +INSERT OVERWRITE TABLE outputTbl3 +SELECT key, key + 1, count(1) FROM T1 GROUP BY key, key + 1; + +SELECT * FROM outputTbl3 ORDER BY key1, key2; + +-- it should not matter what follows the group by +-- test various cases + +-- group by followed by another group by +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key + key, sum(cnt) from +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 +group by key + key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key + key, sum(cnt) from +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 +group by key + key; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- group by followed by a union +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM ( +SELECT key, count(1) FROM T1 GROUP BY key + UNION ALL +SELECT key, count(1) FROM T1 GROUP BY key +) subq1; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM ( +SELECT key, count(1) FROM T1 GROUP BY key + UNION ALL +SELECT key, count(1) FROM T1 GROUP BY key +) subq1; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- group by followed by a union where one of the sub-queries is map-side group by +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM ( +SELECT key, count(1) FROM T1 GROUP BY key + UNION ALL +SELECT key + key as key, count(1) FROM T1 GROUP BY key + key +) subq1; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM ( +SELECT key, count(1) as cnt FROM T1 GROUP BY key + UNION ALL +SELECT key + key as key, count(1) as cnt FROM T1 GROUP BY key + key +) subq1; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- group by followed by a join +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT subq1.key, subq1.cnt+subq2.cnt FROM +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 +JOIN +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq2 +ON subq1.key = subq2.key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT subq1.key, subq1.cnt+subq2.cnt FROM +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 +JOIN +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq2 +ON subq1.key = subq2.key; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- group by followed by a join where one of the sub-queries can be performed in the mapper +EXPLAIN EXTENDED +SELECT * FROM +(SELECT key, count(1) FROM T1 GROUP BY key) subq1 +JOIN +(SELECT key, val, count(1) FROM T1 GROUP BY key, val) subq2 +ON subq1.key = subq2.key; + +CREATE TABLE T2(key STRING, val STRING) +CLUSTERED BY (key, val) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T2 select key, val from T1; + +-- no mapside sort group by if the group by is a prefix of the sorted key +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T2 GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T2 GROUP BY key; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- The plan should be converted to a map-side group by if the group by key contains a constant in between the +-- sorted keys +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val; + +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val; + +SELECT * FROM outputTbl4 ORDER BY key1, key2, key3; + +CREATE TABLE outputTbl5(key1 int, key2 int, key3 string, key4 int, cnt int); + +-- The plan should be converted to a map-side group by if the group by key contains a constant in between the +-- sorted keys followed by anything +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl5 +SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2; + +INSERT OVERWRITE TABLE outputTbl5 +SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2; + +SELECT * FROM outputTbl5 +ORDER BY key1, key2, key3, key4; + +-- contants from sub-queries should work fine +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, constant, val, count(1) from +(SELECT key, 1 as constant, val from T2)subq +group by key, constant, val; + +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, constant, val, count(1) from +(SELECT key, 1 as constant, val from T2)subq +group by key, constant, val; + +SELECT * FROM outputTbl4 ORDER BY key1, key2, key3; + +-- multiple levels of contants from sub-queries should work fine +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl4 +select key, constant3, val, count(1) from +( +SELECT key, constant as constant2, val, 2 as constant3 from +(SELECT key, 1 as constant, val from T2)subq +)subq2 +group by key, constant3, val; + +INSERT OVERWRITE TABLE outputTbl4 +select key, constant3, val, count(1) from +( +SELECT key, constant as constant2, val, 2 as constant3 from +(SELECT key, 1 as constant, val from T2)subq +)subq2 +group by key, constant3, val; + +SELECT * FROM outputTbl4 ORDER BY key1, key2, key3; + +set hive.map.aggr=true; +set hive.multigroupby.singlereducer=false; +set mapred.reduce.tasks=31; + +CREATE TABLE DEST1(key INT, cnt INT); +CREATE TABLE DEST2(key INT, val STRING, cnt INT); + +SET hive.exec.compress.intermediate=true; +SET hive.exec.compress.output=true; + +EXPLAIN +FROM T2 +INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key +INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val; + +FROM T2 +INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key +INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val; + +select * from DEST1 ORDER BY key, cnt; +select * from DEST2 ORDER BY key, val, val; + +-- multi-table insert with a sub-query +EXPLAIN +FROM (select key, val from T2 where key = 8) x +INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key +INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val; + +FROM (select key, val from T2 where key = 8) x +INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key +INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val; + +select * from DEST1 ORDER BY key, cnt; +select * from DEST2 ORDER BY key, val, cnt; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_10.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_10.q new file mode 100644 index 0000000000000..b3ddd42dceed6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_10.q @@ -0,0 +1,26 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 PARTITION (ds='1') +SELECT * from src where key = 0 or key = 11; + +-- The plan is converted to a map-side plan +EXPLAIN select distinct key from T1; +select distinct key from T1; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 PARTITION (ds='2') +SELECT * from src where key = 0 or key = 11; + +-- The plan is not converted to a map-side, since although the sorting columns and grouping +-- columns match, the user is querying multiple input partitions +EXPLAIN select distinct key from T1; +select distinct key from T1; + +DROP TABLE T1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_11.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_11.q new file mode 100644 index 0000000000000..19063f667898e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_11.q @@ -0,0 +1,40 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 PARTITION (ds='1') +SELECT * from src where key < 10; + +-- The plan is optimized to perform partial aggregation on the mapper +EXPLAIN select count(distinct key) from T1; +select count(distinct key) from T1; + +-- The plan is optimized to perform partial aggregation on the mapper +EXPLAIN select count(distinct key), count(1), count(key), sum(distinct key) from T1; +select count(distinct key), count(1), count(key), sum(distinct key) from T1; + +-- The plan is not changed in the presence of a grouping key +EXPLAIN select count(distinct key), count(1), count(key), sum(distinct key) from T1 group by key; +select count(distinct key), count(1), count(key), sum(distinct key) from T1 group by key; + +-- The plan is not changed in the presence of a grouping key +EXPLAIN select key, count(distinct key), count(1), count(key), sum(distinct key) from T1 group by key; +select key, count(distinct key), count(1), count(key), sum(distinct key) from T1 group by key; + +-- The plan is not changed in the presence of a grouping key expression +EXPLAIN select count(distinct key+key) from T1; +select count(distinct key+key) from T1; + +EXPLAIN select count(distinct 1) from T1; +select count(distinct 1) from T1; + +set hive.map.aggr=false; + +-- no plan change if map aggr is turned off +EXPLAIN select count(distinct key) from T1; +select count(distinct key) from T1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_2.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_2.q new file mode 100644 index 0000000000000..31b4ec5c74dc2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_2.q @@ -0,0 +1,25 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 select key, val from T1; + +CREATE TABLE outputTbl1(val string, cnt int); + +-- The plan should not be converted to a map-side group by even though the group by key +-- matches the sorted key. Adding a order by at the end to make the test results deterministic +EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT val, count(1) FROM T1 GROUP BY val; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT val, count(1) FROM T1 GROUP BY val; + +SELECT * FROM outputTbl1 ORDER BY val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_3.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_3.q new file mode 100644 index 0000000000000..103c57a123576 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_3.q @@ -0,0 +1,36 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 select key, val from T1; + +CREATE TABLE outputTbl1(key string, val string, cnt int); + +-- The plan should be converted to a map-side group by +EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +SELECT * FROM outputTbl1 ORDER BY key, val; + +CREATE TABLE outputTbl2(key string, cnt int); + +-- The plan should be converted to a map-side group by +EXPLAIN +INSERT OVERWRITE TABLE outputTbl2 +SELECT key, count(1) FROM T1 GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl2 +SELECT key, count(1) FROM T1 GROUP BY key; + +SELECT * FROM outputTbl2 ORDER BY key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_4.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_4.q new file mode 100644 index 0000000000000..e43da3c93225f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_4.q @@ -0,0 +1,38 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 select key, val from T1; + +CREATE TABLE outputTbl1(key STRING, cnt INT); + +-- The plan should not be converted to a map-side group by. +-- However, there should no hash-based aggregation on the map-side +EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 GROUP BY key; + +SELECT * FROM outputTbl1 ORDER BY key; + +CREATE TABLE outputTbl2(key STRING, val STRING, cnt INT); + +-- The plan should not be converted to a map-side group by. +-- Hash-based aggregations should be performed on the map-side +EXPLAIN +INSERT OVERWRITE TABLE outputTbl2 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +INSERT OVERWRITE TABLE outputTbl2 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +SELECT * FROM outputTbl2 ORDER BY key, val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_5.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_5.q new file mode 100644 index 0000000000000..bef5e5d2d547a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_5.q @@ -0,0 +1,75 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 select key, val from T1; + +CREATE TABLE outputTbl1(key STRING, val STRING, cnt INT); + +-- The plan should be converted to a map-side group by, since the +-- sorting columns and grouping columns match, and all the bucketing columns +-- are part of sorting columns +EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +SELECT * FROM outputTbl1 ORDER BY key, val; + +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 select key, val from T1; + +-- The plan should be converted to a map-side group by, since the +-- sorting columns and grouping columns match, and all the bucketing columns +-- are part of sorting columns +EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +SELECT * FROM outputTbl1 ORDER BY key, val; + +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 select key, val from T1; + +CREATE TABLE outputTbl2(key STRING, cnt INT); + +-- The plan should not be converted to a map-side group by, since although the +-- sorting columns and grouping columns match, all the bucketing columns +-- are not part of sorting columns. However, no hash map aggregation is required +-- on the mapside. +EXPLAIN +INSERT OVERWRITE TABLE outputTbl2 +SELECT key, count(1) FROM T1 GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl2 +SELECT key, count(1) FROM T1 GROUP BY key; + +SELECT * FROM outputTbl2 ORDER BY key; + +DROP TABLE T1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_6.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_6.q new file mode 100644 index 0000000000000..cf076e8125612 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_6.q @@ -0,0 +1,41 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +set hive.map.groupby.sorted=true; + +CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string); + +CREATE TABLE outputTbl1(key int, cnt int); + +-- The plan should not be converted to a map-side group since no partition is being accessed +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 where ds = '1' GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl1 +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'); + +-- The plan should not be converted to a map-side group since no partition is being accessed +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 where ds = '1' GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 where ds = '1' GROUP BY key; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- The plan should not be converted to a map-side group since the partition being accessed +-- is neither bucketed not sorted +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 where ds = '2' GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 where ds = '2' GROUP BY key; + +SELECT * FROM outputTbl1 ORDER BY key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_7.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_7.q new file mode 100644 index 0000000000000..c2d42154e516c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_7.q @@ -0,0 +1,28 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +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'); + +-- 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'; + +CREATE TABLE outputTbl1(key STRING, val STRING, cnt INT); + +-- The plan should be converted to a map-side group by, since the +-- sorting columns and grouping columns match, and all the bucketing columns +-- are part of sorting columns +EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, val, count(1) FROM T1 where ds = '1' GROUP BY key, val; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, val, count(1) FROM T1 where ds = '1' GROUP BY key, val; + +SELECT * FROM outputTbl1 ORDER BY key, val; + +DROP TABLE T1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_8.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_8.q new file mode 100644 index 0000000000000..121804e60a9da --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_8.q @@ -0,0 +1,27 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +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'); + +-- 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'; + +-- The plan is not converted to a map-side, since although the sorting columns and grouping +-- columns match, the user is issueing a distinct. +-- However, after HIVE-4310, partial aggregation is performed on the mapper +EXPLAIN +select count(distinct key) from T1; +select count(distinct key) from T1; + +set hive.map.groupby.sorted.testmode=true; +-- In testmode, the plan is not changed +EXPLAIN +select count(distinct key) from T1; +select count(distinct key) from T1; + +DROP TABLE T1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_9.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_9.q new file mode 100644 index 0000000000000..1c3d1cdcc265d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_9.q @@ -0,0 +1,21 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +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'); + +-- 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'; +INSERT OVERWRITE TABLE T1 PARTITION (ds='2') select key, val from T1 where ds = '1'; + +-- The plan is not converted to a map-side, since although the sorting columns and grouping +-- columns match, the user is querying multiple input partitions +EXPLAIN +select key, count(1) from T1 group by key; +select key, count(1) from T1 group by key; + +DROP TABLE T1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q new file mode 100644 index 0000000000000..068c26a9c2b4b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q @@ -0,0 +1,283 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +set hive.map.groupby.sorted=true; +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 select key, val from T1; + +CREATE TABLE outputTbl1(key int, cnt int); + +-- The plan should be converted to a map-side group by if the group by key +-- matches the sorted key +-- addind a order by at the end to make the test results deterministic +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 GROUP BY key; + +SELECT * FROM outputTbl1 ORDER BY key; + +CREATE TABLE outputTbl2(key1 int, key2 string, cnt int); + +-- no map-side group by even if the group by key is a superset of sorted key +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl2 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +INSERT OVERWRITE TABLE outputTbl2 +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +SELECT * FROM outputTbl2 ORDER BY key1, key2; + +-- It should work for sub-queries +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM (SELECT key, val FROM T1) subq1 GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM (SELECT key, val FROM T1) subq1 GROUP BY key; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- It should work for sub-queries with column aliases +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT k, count(1) FROM (SELECT key as k, val as v FROM T1) subq1 GROUP BY k; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT k, count(1) FROM (SELECT key as k, val as v FROM T1) subq1 GROUP BY k; + +SELECT * FROM outputTbl1 ORDER BY key; + +CREATE TABLE outputTbl3(key1 int, key2 int, cnt int); + +-- The plan should be converted to a map-side group by if the group by key contains a constant followed +-- by a match to the sorted key +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl3 +SELECT 1, key, count(1) FROM T1 GROUP BY 1, key; + +INSERT OVERWRITE TABLE outputTbl3 +SELECT 1, key, count(1) FROM T1 GROUP BY 1, key; + +SELECT * FROM outputTbl3 ORDER BY key1, key2; + +CREATE TABLE outputTbl4(key1 int, key2 int, key3 string, cnt int); + +-- no map-side group by if the group by key contains a constant followed by another column +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val; + +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val; + +SELECT * FROM outputTbl4 ORDER BY key1, key2, key3; + +-- no map-side group by if the group by key contains a function +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl3 +SELECT key, key + 1, count(1) FROM T1 GROUP BY key, key + 1; + +INSERT OVERWRITE TABLE outputTbl3 +SELECT key, key + 1, count(1) FROM T1 GROUP BY key, key + 1; + +SELECT * FROM outputTbl3 ORDER BY key1, key2; + +-- it should not matter what follows the group by +-- test various cases + +-- group by followed by another group by +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key + key, sum(cnt) from +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 +group by key + key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key + key, sum(cnt) from +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 +group by key + key; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- group by followed by a union +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM ( +SELECT key, count(1) FROM T1 GROUP BY key + UNION ALL +SELECT key, count(1) FROM T1 GROUP BY key +) subq1; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM ( +SELECT key, count(1) FROM T1 GROUP BY key + UNION ALL +SELECT key, count(1) FROM T1 GROUP BY key +) subq1; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- group by followed by a union where one of the sub-queries is map-side group by +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM ( +SELECT key, count(1) FROM T1 GROUP BY key + UNION ALL +SELECT key + key as key, count(1) FROM T1 GROUP BY key + key +) subq1; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM ( +SELECT key, count(1) as cnt FROM T1 GROUP BY key + UNION ALL +SELECT key + key as key, count(1) as cnt FROM T1 GROUP BY key + key +) subq1; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- group by followed by a join +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT subq1.key, subq1.cnt+subq2.cnt FROM +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 +JOIN +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq2 +ON subq1.key = subq2.key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT subq1.key, subq1.cnt+subq2.cnt FROM +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 +JOIN +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq2 +ON subq1.key = subq2.key; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- group by followed by a join where one of the sub-queries can be performed in the mapper +EXPLAIN EXTENDED +SELECT * FROM +(SELECT key, count(1) FROM T1 GROUP BY key) subq1 +JOIN +(SELECT key, val, count(1) FROM T1 GROUP BY key, val) subq2 +ON subq1.key = subq2.key; + +CREATE TABLE T2(key STRING, val STRING) +CLUSTERED BY (key, val) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T2 select key, val from T1; + +-- no mapside sort group by if the group by is a prefix of the sorted key +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T2 GROUP BY key; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T2 GROUP BY key; + +SELECT * FROM outputTbl1 ORDER BY key; + +-- The plan should be converted to a map-side group by if the group by key contains a constant in between the +-- sorted keys +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val; + +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val; + +SELECT * FROM outputTbl4 ORDER BY key1, key2, key3; + +CREATE TABLE outputTbl5(key1 int, key2 int, key3 string, key4 int, cnt int); + +-- The plan should be converted to a map-side group by if the group by key contains a constant in between the +-- sorted keys followed by anything +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl5 +SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2; + +INSERT OVERWRITE TABLE outputTbl5 +SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2; + +SELECT * FROM outputTbl5 +ORDER BY key1, key2, key3, key4; + +-- contants from sub-queries should work fine +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, constant, val, count(1) from +(SELECT key, 1 as constant, val from T2)subq +group by key, constant, val; + +INSERT OVERWRITE TABLE outputTbl4 +SELECT key, constant, val, count(1) from +(SELECT key, 1 as constant, val from T2)subq +group by key, constant, val; + +SELECT * FROM outputTbl4 ORDER BY key1, key2, key3; + +-- multiple levels of contants from sub-queries should work fine +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE outputTbl4 +select key, constant3, val, count(1) from +( +SELECT key, constant as constant2, val, 2 as constant3 from +(SELECT key, 1 as constant, val from T2)subq +)subq2 +group by key, constant3, val; + +INSERT OVERWRITE TABLE outputTbl4 +select key, constant3, val, count(1) from +( +SELECT key, constant as constant2, val, 2 as constant3 from +(SELECT key, 1 as constant, val from T2)subq +)subq2 +group by key, constant3, val; + +SELECT * FROM outputTbl4 ORDER BY key1, key2, key3; + +set hive.map.aggr=true; +set hive.multigroupby.singlereducer=false; +set mapred.reduce.tasks=31; + +CREATE TABLE DEST1(key INT, cnt INT); +CREATE TABLE DEST2(key INT, val STRING, cnt INT); + +SET hive.exec.compress.intermediate=true; +SET hive.exec.compress.output=true; + +EXPLAIN +FROM T2 +INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key +INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val; + +FROM T2 +INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key +INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val; + +select * from DEST1 ORDER BY key, cnt; +select * from DEST2 ORDER BY key, val, val; + +-- multi-table insert with a sub-query +EXPLAIN +FROM (select key, val from T2 where key = 8) x +INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key +INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val; + +FROM (select key, val from T2 where key = 8) x +INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key +INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val; + +select * from DEST1 ORDER BY key, cnt; +select * from DEST2 ORDER BY key, val, cnt; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_test_1.q b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_test_1.q new file mode 100644 index 0000000000000..8efa05e2544d4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_test_1.q @@ -0,0 +1,21 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 10; +set hive.map.groupby.sorted=true; +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; + +-- perform an insert to make sure there are 2 files +INSERT OVERWRITE TABLE T1 select key, val from T1; + +CREATE TABLE outputTbl1(key int, cnt int); + +-- The plan should be converted to a map-side group by if the group by key +-- matches the sorted key. However, in test mode, the group by wont be converted. +EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT key, count(1) FROM T1 GROUP BY key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/having.q b/src/test/hive/ql/src/test/queries/clientpositive/having.q new file mode 100644 index 0000000000000..5b1aa69d9d82b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/having.q @@ -0,0 +1,14 @@ +EXPLAIN SELECT count(value) AS c FROM src GROUP BY key HAVING c > 3; +SELECT count(value) AS c FROM src GROUP BY key HAVING c > 3; + +EXPLAIN SELECT key, max(value) AS c FROM src GROUP BY key HAVING key != 302; +SELECT key, max(value) AS c FROM src GROUP BY key HAVING key != 302; + +EXPLAIN SELECT key FROM src GROUP BY key HAVING max(value) > "val_255"; +SELECT key FROM src GROUP BY key HAVING max(value) > "val_255"; + +EXPLAIN SELECT key FROM src where key > 300 GROUP BY key HAVING max(value) > "val_255"; +SELECT key FROM src where key > 300 GROUP BY key HAVING max(value) > "val_255"; + +EXPLAIN SELECT key, max(value) FROM src GROUP BY key HAVING max(value) > "val_255"; +SELECT key, max(value) FROM src GROUP BY key HAVING max(value) > "val_255"; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/hook_context_cs.q b/src/test/hive/ql/src/test/queries/clientpositive/hook_context_cs.q new file mode 100644 index 0000000000000..94ba14802f015 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/hook_context_cs.q @@ -0,0 +1,14 @@ +drop table vcsc; +CREATE TABLE vcsc (c STRING) PARTITIONED BY (ds STRING); +ALTER TABLE vcsc ADD partition (ds='dummy') location '${system:test.tmp.dir}/VerifyContentSummaryCacheHook'; + +set hive.exec.pre.hooks=org.apache.hadoop.hive.ql.hooks.VerifyContentSummaryCacheHook; +SELECT a.c, b.c FROM vcsc a JOIN vcsc b ON a.ds = 'dummy' AND b.ds = 'dummy' AND a.c = b.c; + +set mapred.job.tracker=local; +set hive.exec.pre.hooks = ; +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyContentSummaryCacheHook; +SELECT a.c, b.c FROM vcsc a JOIN vcsc b ON a.ds = 'dummy' AND b.ds = 'dummy' AND a.c = b.c; + +set hive.exec.post.hooks=; +drop table vcsc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/hook_order.q b/src/test/hive/ql/src/test/queries/clientpositive/hook_order.q new file mode 100644 index 0000000000000..b6753b308b5f1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/hook_order.q @@ -0,0 +1,11 @@ +SET hive.exec.pre.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunFirst,org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunSecond; +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunFirst,org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunSecond; +SET hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunFirstSemanticAnalysisHook,org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunSecondSemanticAnalysisHook; +SET hive.exec.driver.run.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunFirstDriverRunHook,org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunSecondDriverRunHook; + +SELECT count(*) FROM src; + +SET hive.exec.pre.hooks=; +SET hive.exec.post.hooks=; +SET hive.semantic.analyzer.hook=; +SET hive.exec.driver.run.hooks=; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/implicit_cast1.q b/src/test/hive/ql/src/test/queries/clientpositive/implicit_cast1.q new file mode 100644 index 0000000000000..37fb8568c94f4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/implicit_cast1.q @@ -0,0 +1,13 @@ +CREATE TABLE implicit_test1(a BIGINT, b STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe' WITH SERDEPROPERTIES('serialization.format'= 'org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol') STORED AS TEXTFILE; + +EXPLAIN +SELECT implicit_test1.* +FROM implicit_test1 +WHERE implicit_test1.a <> 0; + +SELECT implicit_test1.* +FROM implicit_test1 +WHERE implicit_test1.a <> 0; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auth.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auth.q new file mode 100644 index 0000000000000..33a1fc581ed13 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auth.q @@ -0,0 +1,15 @@ +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; +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; +set hive.security.authorization.enabled=true; + +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/src/test/hive/ql/src/test/queries/clientpositive/index_auto.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auto.q new file mode 100644 index 0000000000000..cb8a1d6293f8d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auto.q @@ -0,0 +1,28 @@ +-- try the query without indexing, with manual indexing, and with automatic indexing + +-- without indexing +SELECT key, value FROM src WHERE key > 80 AND key < 100 ORDER BY key; + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +-- manual indexing +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_where" SELECT `_bucketname` , `_offsets` FROM default__src_src_index__ WHERE key > 80 AND key < 100; +SET hive.index.compact.file=${system:test.tmp.dir}/index_where; +SET hive.optimize.index.filter=false; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; + +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; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +-- automatic indexing +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 diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_empty.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_empty.q new file mode 100644 index 0000000000000..cb32162d40a06 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_empty.q @@ -0,0 +1,21 @@ +-- Test to ensure that an empty index result is propagated correctly + +-- Create temp, and populate it with some values in src. +CREATE TABLE temp(key STRING, val STRING) STORED AS TEXTFILE; + +-- 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; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +-- query should not return any values +SELECT * FROM default__temp_temp_index__ WHERE key = 86; +EXPLAIN SELECT * FROM temp WHERE key = 86; +SELECT * FROM temp WHERE key = 86; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=false; +DROP table temp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_file_format.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_file_format.q new file mode 100644 index 0000000000000..790e6c223f746 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_file_format.q @@ -0,0 +1,19 @@ +-- 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; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +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; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +SET hive.optimize.index.filter=true; +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 diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables.q new file mode 100644 index 0000000000000..2bf8481f1d6e2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables.q @@ -0,0 +1,22 @@ +-- try the query without indexing, with manual indexing, and with automatic indexing + +-- without indexing +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; + + +CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +CREATE INDEX srcpart_index ON TABLE srcpart(key) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX srcpart_index ON srcpart REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +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; + +DROP INDEX src_index on src; +DROP INDEX srcpart_index on src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q new file mode 100644 index 0000000000000..808a04cc360a6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q @@ -0,0 +1,23 @@ +-- try the query without indexing, with manual indexing, and with automatic indexing + +-- without indexing +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; + + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +CREATE INDEX srcpart_index ON TABLE srcpart(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX srcpart_index ON srcpart REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +-- automatic indexing +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; + +DROP INDEX src_index on src; +DROP INDEX srcpart_index on src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_multiple.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_multiple.q new file mode 100644 index 0000000000000..06e97fa76bc97 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_multiple.q @@ -0,0 +1,16 @@ +-- 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; +CREATE INDEX src_val_index ON TABLE src(value) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_key_index ON src REBUILD; +ALTER INDEX src_val_index ON src REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +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_key_index ON src; +DROP INDEX src_val_index ON src; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_partitioned.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_partitioned.q new file mode 100644 index 0000000000000..5013d29e732c4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_partitioned.q @@ -0,0 +1,12 @@ +-- 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; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +EXPLAIN SELECT key, value FROM srcpart WHERE key=86 AND ds='2008-04-09' ORDER BY key; +SELECT key, value FROM srcpart WHERE key=86 AND ds='2008-04-09' ORDER BY key; + +DROP INDEX src_part_index ON srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_self_join.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_self_join.q new file mode 100644 index 0000000000000..0984a4a21ba17 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_self_join.q @@ -0,0 +1,16 @@ +-- try the query without indexing, with manual indexing, and with automatic indexing + +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; + +CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +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; + +DROP INDEX src_index on src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_unused.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_unused.q new file mode 100644 index 0000000000000..d8f3eda1813c8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_unused.q @@ -0,0 +1,60 @@ +-- test cases where the index should not be used automatically + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=5368709120; +SET hive.optimize.index.filter.compact.maxsize=-1; + +-- min size too large (src is less than 5G) +EXPLAIN SELECT * FROM src WHERE key > 80 AND key < 100 ORDER BY key; +SELECT * FROM src WHERE key > 80 AND key < 100 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; +SET hive.optimize.index.filter.compact.maxsize=1; + +-- max size too small +EXPLAIN SELECT * FROM src WHERE key > 80 AND key < 100 ORDER BY key; +SELECT * FROM src WHERE key > 80 AND key < 100 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; +SET hive.optimize.index.filter.compact.maxsize=-1; + +-- OR predicate not supported by compact indexes +EXPLAIN SELECT * FROM src WHERE key < 10 OR key > 480 ORDER BY key; +SELECT * FROM src WHERE key < 10 OR key > 480 ORDER BY key; + + SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; +SET hive.optimize.index.filter.compact.maxsize=-1; + +-- columns are not covered by indexes +DROP INDEX src_index on src; +CREATE INDEX src_val_index ON TABLE src(value) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_val_index ON src REBUILD; + +EXPLAIN SELECT * FROM src WHERE key > 80 AND key < 100 ORDER BY key; +SELECT * FROM src WHERE key > 80 AND key < 100 ORDER BY key; + +DROP INDEX src_val_index on src; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; +SET hive.optimize.index.filter.compact.maxsize=-1; + +-- required partitions have not been built yet +CREATE INDEX src_part_index ON TABLE srcpart(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_part_index ON srcpart PARTITION (ds='2008-04-08', hr=11) REBUILD; + +EXPLAIN SELECT * FROM srcpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 ORDER BY key; +SELECT * FROM srcpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 ORDER BY key; + +DROP INDEX src_part_index on srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_update.q b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_update.q new file mode 100644 index 0000000000000..976de5c67a0d3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_auto_update.q @@ -0,0 +1,26 @@ +-- Test if index is actually being used. + +-- Create temp, and populate it with some values in src. +CREATE TABLE temp(key STRING, val STRING) STORED AS TEXTFILE; +INSERT OVERWRITE TABLE temp SELECT * FROM src WHERE key < 50; + +-- 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; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.autoupdate=true; +SET hive.optimize.index.filter.compact.minsize=0; + +-- overwrite temp table so index is out of date +EXPLAIN INSERT OVERWRITE TABLE temp SELECT * FROM src; +INSERT OVERWRITE TABLE temp SELECT * FROM src; + +-- query should return indexed values +EXPLAIN SELECT * FROM temp WHERE key = 86; +SELECT * FROM temp WHERE key = 86; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=false; +DROP table temp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap.q b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap.q new file mode 100644 index 0000000000000..f9deb2883960c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap.q @@ -0,0 +1,47 @@ +DROP INDEX srcpart_index_proj on srcpart; + +EXPLAIN +CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) as 'BITMAP' WITH DEFERRED REBUILD; +CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX srcpart_index_proj ON srcpart REBUILD; +SELECT x.* FROM default__srcpart_srcpart_index_proj__ x WHERE x.ds = '2008-04-08' and x.hr = 11 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_test_index_result" SELECT `_bucketname`, +COLLECT_SET(`_offset`) as `_offsets` FROM default__srcpart_srcpart_index_proj__ +x WHERE NOT EWAH_BITMAP_EMPTY(`_bitmaps`) AND x.key=100 AND x.ds = '2008-04-08' GROUP BY `_bucketname`; +SET hive.index.blockfilter.file=${system:test.tmp.dir}/index_test_index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat; +SELECT key, value FROM srcpart WHERE key=100 AND ds = '2008-04-08' ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_test_index_result" SELECT `_bucketname` , +COLLECT_SET(`_offset`) as `_offsets` FROM default__srcpart_srcpart_index_proj__ +x WHERE NOT EWAH_BITMAP_EMPTY(`_bitmaps`) AND x.key=100 AND x.ds = '2008-04-08' and x.hr = 11 GROUP BY `_bucketname`; +SET hive.index.blockfilter.file=${system:test.tmp.dir}/index_test_index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat; +SELECT key, value FROM srcpart WHERE key=100 AND ds = '2008-04-08' and hr = 11 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT key, value FROM srcpart WHERE key=100 AND ds = '2008-04-08' and hr = 11 ORDER BY key; + +DROP INDEX srcpart_index_proj on srcpart; + +EXPLAIN +CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) as 'BITMAP' WITH DEFERRED REBUILD; +CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX srcpart_index_proj ON srcpart REBUILD; +SELECT x.* FROM default__srcpart_srcpart_index_proj__ x; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , +COLLECT_SET(`_offset`) as `_offsets` FROM default__srcpart_srcpart_index_proj__ +WHERE NOT EWAH_BITMAP_EMPTY(`_bitmaps`) AND key=100 GROUP BY `_bucketname`; +SET hive.index.blockfilter.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat; +SELECT key, value FROM srcpart WHERE key=100 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT key, value FROM srcpart WHERE key=100 ORDER BY key; + +DROP INDEX srcpart_index_proj on srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap1.q b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap1.q new file mode 100644 index 0000000000000..55633d9225fa7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap1.q @@ -0,0 +1,18 @@ +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; +ALTER INDEX src_index ON src REBUILD; +SELECT x.* FROM default__src_src_index__ x ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname`, +COLLECT_SET(`_offset`) as `_offsets` FROM default__src_src_index__ WHERE NOT +EWAH_BITMAP_EMPTY(`_bitmaps`) AND key=100 GROUP BY `_bucketname`; +SET hive.index.blockfilter.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat; +SELECT key, value FROM src WHERE key=100 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT key, value FROM src WHERE key=100 ORDER BY key; + +DROP INDEX src_index ON src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap2.q b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap2.q new file mode 100644 index 0000000000000..bd15a21fa4200 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap2.q @@ -0,0 +1,35 @@ +EXPLAIN +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; +EXPLAIN +CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD; + +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; +CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX src1_index ON src REBUILD; +ALTER INDEX src2_index ON src REBUILD; +SELECT * FROM default__src_src1_index__ ORDER BY key; +SELECT * FROM default__src_src2_index__ ORDER BY value; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" +SELECT t.bucketname as `_bucketname`, COLLECT_SET(t.offset) AS `_offsets` FROM + (SELECT `_bucketname` AS bucketname, `_offset` AS offset + FROM default__src_src1_index__ + WHERE key = 0 AND NOT EWAH_BITMAP_EMPTY(`_bitmaps`) UNION ALL + SELECT `_bucketname` AS bucketname, `_offset` AS offset + FROM default__src_src2_index__ + WHERE value = "val2" AND NOT EWAH_BITMAP_EMPTY(`_bitmaps`)) t +GROUP BY t.bucketname; + +SET hive.index.blockfilter.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat; + +SELECT key, value FROM src WHERE key=0 OR value = "val_2" ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT key, value FROM src WHERE key=0 OR value = "val_2" ORDER BY key; + +DROP INDEX src1_index ON src; +DROP INDEX src2_index ON src; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap3.q b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap3.q new file mode 100644 index 0000000000000..0d2c811459dcb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap3.q @@ -0,0 +1,47 @@ +EXPLAIN +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; +EXPLAIN +CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD; + +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; +CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX src1_index ON src REBUILD; +ALTER INDEX src2_index ON src REBUILD; +SELECT * FROM default__src_src1_index__ ORDER BY key; +SELECT * FROM default__src_src2_index__ ORDER BY value; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +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 +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname; + +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" +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 +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname; + +SET hive.index.blockfilter.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat; + +SELECT key, value FROM src WHERE key=0 AND value = "val_0" ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT key, value FROM src WHERE key=0 AND value = "val_0" ORDER BY key; + +DROP INDEX src1_index ON src; +DROP INDEX src2_index ON src; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto.q b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto.q new file mode 100644 index 0000000000000..672ce29f1bb47 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto.q @@ -0,0 +1,52 @@ +-- 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; + +-- create indices +EXPLAIN +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; +EXPLAIN +CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; +CREATE INDEX src2_index ON TABLE src(value) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX src1_index ON src REBUILD; +ALTER INDEX src2_index ON src REBUILD; +SELECT * FROM default__src_src1_index__ ORDER BY key; +SELECT * FROM default__src_src2_index__ ORDER BY value; + + +-- manual indexing +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 +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname; + +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" +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 +EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname; + +SELECT key, value FROM src WHERE key=0 AND value = "val_0" ORDER BY key; + + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SELECT key, value FROM src WHERE key=0 AND value = "val_0" ORDER BY key; + +DROP INDEX src1_index ON src; +DROP INDEX src2_index ON src; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q new file mode 100644 index 0000000000000..90d7987594bac --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q @@ -0,0 +1,11 @@ +-- 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; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; + +EXPLAIN SELECT key, value FROM srcpart WHERE key=86 AND ds='2008-04-09' ORDER BY key; +SELECT key, value FROM srcpart WHERE key=86 AND ds='2008-04-09' ORDER BY key; + +DROP INDEX src_part_index ON srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_compression.q b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_compression.q new file mode 100644 index 0000000000000..2f5e5d4fb231b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_compression.q @@ -0,0 +1,13 @@ +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; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +-- automatic indexing +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_rc.q b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_rc.q new file mode 100644 index 0000000000000..054df51c32180 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_rc.q @@ -0,0 +1,53 @@ +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; +INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-08', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 12; +INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-09', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 11; +INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-09', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 12; + +EXPLAIN +CREATE INDEX srcpart_rc_index ON TABLE srcpart_rc(key) as 'BITMAP' WITH DEFERRED REBUILD; +CREATE INDEX srcpart_rc_index ON TABLE srcpart_rc(key) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX srcpart_rc_index ON srcpart_rc REBUILD; +SELECT x.* FROM default__srcpart_rc_srcpart_rc_index__ x WHERE x.ds = '2008-04-08' and x.hr = 11 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_test_index_result" SELECT `_bucketname`, +COLLECT_SET(`_offset`) as `_offsets` FROM default__srcpart_rc_srcpart_rc_index__ +x WHERE NOT EWAH_BITMAP_EMPTY(`_bitmaps`) AND x.key=100 AND x.ds = '2008-04-08' GROUP BY `_bucketname`; +SET hive.index.blockfilter.file=${system:test.tmp.dir}/index_test_index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat; +SELECT key, value FROM srcpart_rc WHERE key=100 AND ds = '2008-04-08' ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_test_index_result" SELECT `_bucketname` , +COLLECT_SET(`_offset`) as `_offsets` FROM default__srcpart_rc_srcpart_rc_index__ +x WHERE NOT EWAH_BITMAP_EMPTY(`_bitmaps`) AND x.key=100 AND x.ds = '2008-04-08' and x.hr = 11 GROUP BY `_bucketname`; +SET hive.index.blockfilter.file=${system:test.tmp.dir}/index_test_index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat; +SELECT key, value FROM srcpart_rc WHERE key=100 AND ds = '2008-04-08' and hr = 11 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT key, value FROM srcpart_rc WHERE key=100 AND ds = '2008-04-08' and hr = 11 ORDER BY key; + +DROP INDEX srcpart_rc_index on srcpart_rc; + +EXPLAIN +CREATE INDEX srcpart_rc_index ON TABLE srcpart_rc(key) as 'BITMAP' WITH DEFERRED REBUILD; +CREATE INDEX srcpart_rc_index ON TABLE srcpart_rc(key) as 'BITMAP' WITH DEFERRED REBUILD; +ALTER INDEX srcpart_rc_index ON srcpart_rc REBUILD; +SELECT x.* FROM default__srcpart_rc_srcpart_rc_index__ x WHERE x.key = 100; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , +COLLECT_SET(`_offset`) as `_offsets` FROM default__srcpart_rc_srcpart_rc_index__ +WHERE NOT EWAH_BITMAP_EMPTY(`_bitmaps`) AND key=100 GROUP BY `_bucketname`; +SET hive.index.blockfilter.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.HiveIndexedInputFormat; +SELECT key, value FROM srcpart_rc WHERE key=100 ORDER BY key; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compact.q b/src/test/hive/ql/src/test/queries/clientpositive/index_compact.q new file mode 100644 index 0000000000000..a936f1127f690 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_compact.q @@ -0,0 +1,41 @@ +DROP INDEX srcpart_index_proj on srcpart; + +EXPLAIN +CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) as 'COMPACT' WITH DEFERRED REBUILD; +CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX srcpart_index_proj ON srcpart REBUILD; +SELECT x.* FROM default__srcpart_srcpart_index_proj__ x WHERE x.ds = '2008-04-08' and x.hr = 11 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_test_index_result" SELECT `_bucketname` , `_offsets` FROM default__srcpart_srcpart_index_proj__ x WHERE x.key=100 AND x.ds = '2008-04-08'; +SET hive.index.compact.file=${system:test.tmp.dir}/index_test_index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; +SELECT key, value FROM srcpart WHERE key=100 AND ds = '2008-04-08' ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_test_index_result" SELECT `_bucketname` , `_offsets` FROM default__srcpart_srcpart_index_proj__ x WHERE x.key=100 AND x.ds = '2008-04-08' and x.hr = 11; +SET hive.index.compact.file=${system:test.tmp.dir}/index_test_index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; +SELECT key, value FROM srcpart WHERE key=100 AND ds = '2008-04-08' and hr = 11 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT key, value FROM srcpart WHERE key=100 AND ds = '2008-04-08' and hr = 11 ORDER BY key; + +DROP INDEX srcpart_index_proj on srcpart; + +EXPLAIN +CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) as 'COMPACT' WITH DEFERRED REBUILD; +CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX srcpart_index_proj ON srcpart REBUILD; +SELECT x.* FROM default__srcpart_srcpart_index_proj__ x; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , `_offsets` FROM default__srcpart_srcpart_index_proj__ WHERE key=100; +SET hive.index.compact.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; +SELECT key, value FROM srcpart WHERE key=100 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT key, value FROM srcpart WHERE key=100 ORDER BY key; + +DROP INDEX srcpart_index_proj on srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compact_1.q b/src/test/hive/ql/src/test/queries/clientpositive/index_compact_1.q new file mode 100644 index 0000000000000..837033be403a9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_compact_1.q @@ -0,0 +1,16 @@ +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; +ALTER INDEX src_index ON src REBUILD; +SELECT x.* FROM default__src_src_index__ x ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , `_offsets` FROM default__src_src_index__ WHERE key=100; +SET hive.index.compact.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; +SELECT key, value FROM src WHERE key=100 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT key, value FROM src WHERE key=100 ORDER BY key; + +DROP INDEX src_index on src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compact_2.q b/src/test/hive/ql/src/test/queries/clientpositive/index_compact_2.q new file mode 100644 index 0000000000000..56119ac11f32f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_compact_2.q @@ -0,0 +1,45 @@ +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; +INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-08', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 12; +INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-09', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 11; +INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-09', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 12; + +CREATE INDEX srcpart_rc_index ON TABLE srcpart_rc(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX srcpart_rc_index ON srcpart_rc REBUILD; +SELECT x.* FROM default__srcpart_rc_srcpart_rc_index__ x WHERE x.ds = '2008-04-08' and x.hr = 11 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_test_index_result" SELECT `_bucketname` , `_offsets` FROM default__srcpart_rc_srcpart_rc_index__ x WHERE x.key=100 AND x.ds = '2008-04-08'; +SET hive.index.compact.file=${system:test.tmp.dir}/index_test_index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; +SELECT key, value FROM srcpart_rc WHERE key=100 AND ds = '2008-04-08' ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_test_index_result" SELECT `_bucketname` , `_offsets` FROM default__srcpart_rc_srcpart_rc_index__ x WHERE x.key=100 AND x.ds = '2008-04-08' and x.hr = 11; +SET hive.index.compact.file=${system:test.tmp.dir}/index_test_index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; +SELECT key, value FROM srcpart_rc WHERE key=100 AND ds = '2008-04-08' and hr = 11 ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT key, value FROM srcpart_rc WHERE key=100 AND ds = '2008-04-08' and hr = 11 ORDER BY key; + +DROP INDEX srcpart_rc_index on srcpart_rc; + +EXPLAIN +CREATE INDEX srcpart_rc_index ON TABLE srcpart_rc(key) as 'COMPACT' WITH DEFERRED REBUILD; +CREATE INDEX srcpart_rc_index ON TABLE srcpart_rc(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX srcpart_rc_index ON srcpart_rc REBUILD; +SELECT x.* FROM default__srcpart_rc_srcpart_rc_index__ x ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , `_offsets` FROM default__srcpart_rc_srcpart_rc_index__ WHERE key=100; +SET hive.index.compact.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; +SELECT key, value FROM srcpart_rc WHERE key=100 ORDER BY key; + +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 diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compact_3.q b/src/test/hive/ql/src/test/queries/clientpositive/index_compact_3.q new file mode 100644 index 0000000000000..f3fcb4af3d5e6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_compact_3.q @@ -0,0 +1,19 @@ +CREATE TABLE src_index_test_rc (key int, value string) STORED AS RCFILE; + +INSERT OVERWRITE TABLE src_index_test_rc SELECT * FROM src; + +CREATE INDEX src_index ON TABLE src_index_test_rc(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src_index_test_rc REBUILD; +SELECT x.* FROM default__src_index_test_rc_src_index__ x ORDER BY key; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , `_offsets` FROM default__src_index_test_rc_src_index__ WHERE key=100; +SET hive.index.compact.file=${system:test.tmp.dir}/index_result; +SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; +SELECT key, value FROM src_index_test_rc WHERE key=100 ORDER BY key; + +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 diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compact_binary_search.q b/src/test/hive/ql/src/test/queries/clientpositive/index_compact_binary_search.q new file mode 100644 index 0000000000000..d0d9a32adcba1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_compact_binary_search.q @@ -0,0 +1,132 @@ +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.default.fileformat=TextFile; + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=1; +SET hive.index.compact.binary.search=true; + +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHiveSortedInputFormatUsedHook; + +SELECT * FROM src WHERE key = '0'; + +SELECT * FROM src WHERE key < '1'; + +SELECT * FROM src WHERE key <= '0'; + +SELECT * FROM src WHERE key > '8'; + +SELECT * FROM src WHERE key >= '9'; + +SET hive.exec.post.hooks=; + +DROP INDEX src_index ON src; + +SET hive.default.fileformat=RCFILE; + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHiveSortedInputFormatUsedHook; + +SELECT * FROM src WHERE key = '0'; + +SELECT * FROM src WHERE key < '1'; + +SELECT * FROM src WHERE key <= '0'; + +SELECT * FROM src WHERE key > '8'; + +SELECT * FROM src WHERE key >= '9'; + +SET hive.exec.post.hooks=; + +DROP INDEX src_index ON src; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +SET hive.default.fileformat=TextFile; + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHiveSortedInputFormatUsedHook; + +SELECT * FROM src WHERE key = '0'; + +SELECT * FROM src WHERE key < '1'; + +SELECT * FROM src WHERE key <= '0'; + +SELECT * FROM src WHERE key > '8'; + +SELECT * FROM src WHERE key >= '9'; + +SET hive.exec.post.hooks=; + +DROP INDEX src_index ON src; + +SET hive.default.fileformat=RCFILE; + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHiveSortedInputFormatUsedHook; + +SELECT * FROM src WHERE key = '0'; + +SELECT * FROM src WHERE key < '1'; + +SELECT * FROM src WHERE key <= '0'; + +SELECT * FROM src WHERE key > '8'; + +SELECT * FROM src WHERE key >= '9'; + +SET hive.exec.post.hooks=; + +DROP INDEX src_index ON src; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +SET hive.default.fileformat=TextFile; + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHiveSortedInputFormatUsedHook; + +SELECT * FROM src WHERE key = '0'; + +SELECT * FROM src WHERE key < '1'; + +SELECT * FROM src WHERE key <= '0'; + +SELECT * FROM src WHERE key > '8'; + +SELECT * FROM src WHERE key >= '9'; + +SET hive.exec.post.hooks=; + +DROP INDEX src_index ON src; + +SET hive.default.fileformat=RCFILE; + +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHiveSortedInputFormatUsedHook; + +SELECT * FROM src WHERE key = '0'; + +SELECT * FROM src WHERE key < '1'; + +SELECT * FROM src WHERE key <= '0'; + +SELECT * FROM src WHERE key > '8'; + +SELECT * FROM src WHERE key >= '9'; + +SET hive.exec.post.hooks=; + +DROP INDEX src_index ON src; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compression.q b/src/test/hive/ql/src/test/queries/clientpositive/index_compression.q new file mode 100644 index 0000000000000..84ed3cc932e8c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_compression.q @@ -0,0 +1,13 @@ +SET hive.exec.compress.result=true; +CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX src_index ON src REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +-- automatic indexing +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_creation.q b/src/test/hive/ql/src/test/queries/clientpositive/index_creation.q new file mode 100644 index 0000000000000..062821e870dce --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_creation.q @@ -0,0 +1,53 @@ +drop index src_index_2 on src; +drop index src_index_3 on src; +drop index src_index_4 on src; +drop index src_index_5 on src; +drop index src_index_6 on src; +drop index src_index_7 on src; +drop index src_index_8 on src; +drop index src_index_9 on src; +drop table `_t`; + +create index src_index_2 on table src(key) as 'compact' WITH DEFERRED REBUILD; +desc extended default__src_src_index_2__; + +create index src_index_3 on table src(key) as 'compact' WITH DEFERRED REBUILD in table src_idx_src_index_3; +desc extended src_idx_src_index_3; + +create index src_index_4 on table src(key) as 'compact' WITH DEFERRED REBUILD ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; +desc extended default__src_src_index_4__; + +create index src_index_5 on table src(key) as 'compact' WITH DEFERRED REBUILD ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' ESCAPED BY '\\'; +desc extended default__src_src_index_5__; + +create index src_index_6 on table src(key) as 'compact' WITH DEFERRED REBUILD STORED AS RCFILE; +desc extended default__src_src_index_6__; + +create index src_index_7 on table src(key) as 'compact' WITH DEFERRED REBUILD in table src_idx_src_index_7 STORED AS RCFILE; +desc extended src_idx_src_index_7; + +create index src_index_8 on table src(key) as 'compact' WITH DEFERRED REBUILD IDXPROPERTIES ("prop1"="val1", "prop2"="val2"); +desc extended default__src_src_index_8__; + +create index src_index_9 on table src(key) as 'compact' WITH DEFERRED REBUILD TBLPROPERTIES ("prop1"="val1", "prop2"="val2"); +desc extended default__src_src_index_9__; + +create table `_t`(`_i` int, `_j` int); +create index x on table `_t`(`_j`) as 'compact' WITH DEFERRED REBUILD; +alter index x on `_t` rebuild; + +create index x2 on table `_t`(`_i`,`_j`) as 'compact' WITH DEFERRED +REBUILD; +alter index x2 on `_t` rebuild; + +drop index src_index_2 on src; +drop index src_index_3 on src; +drop index src_index_4 on src; +drop index src_index_5 on src; +drop index src_index_6 on src; +drop index src_index_7 on src; +drop index src_index_8 on src; +drop index src_index_9 on src; +drop table `_t`; + +show tables; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_serde.q b/src/test/hive/ql/src/test/queries/clientpositive/index_serde.q new file mode 100644 index 0000000000000..a6fe16ba3b809 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_serde.q @@ -0,0 +1,49 @@ +-- Want to ensure we can build and use indices on tables stored with SerDes +-- Build the (Avro backed) table +CREATE TABLE doctors +ROW FORMAT +SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' +STORED AS +INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' +OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "testing.hive.avro.serde", + "name": "doctors", + "type": "record", + "fields": [ + { + "name":"number", + "type":"int", + "doc":"Order of playing the role" + }, + { + "name":"first_name", + "type":"string", + "doc":"first name of actor playing role" + }, + { + "name":"last_name", + "type":"string", + "doc":"last name of actor playing role" + } + ] +}'); + +DESCRIBE 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; +DESCRIBE EXTENDED default__doctors_doctors_index__; +ALTER INDEX doctors_index ON doctors REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +EXPLAIN SELECT * FROM doctors WHERE number > 6 ORDER BY number; +SELECT * FROM doctors WHERE number > 6 ORDER BY number; + +DROP INDEX doctors_index ON doctors; +DROP TABLE doctors; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_stale.q b/src/test/hive/ql/src/test/queries/clientpositive/index_stale.q new file mode 100644 index 0000000000000..82e15b97b6ac4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_stale.q @@ -0,0 +1,20 @@ +-- test that stale indexes are not used + +CREATE TABLE temp(key STRING, val STRING) STORED AS TEXTFILE; +INSERT OVERWRITE TABLE temp SELECT * FROM src WHERE key < 50; + +-- 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; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +-- overwrite temp table so index is out of date +INSERT OVERWRITE TABLE temp SELECT * FROM src; + +-- should return correct results bypassing index +EXPLAIN SELECT * FROM temp WHERE key = 86; +SELECT * FROM temp WHERE key = 86; +DROP table temp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_stale_partitioned.q b/src/test/hive/ql/src/test/queries/clientpositive/index_stale_partitioned.q new file mode 100644 index 0000000000000..e7cfeff31f6dc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/index_stale_partitioned.q @@ -0,0 +1,26 @@ +-- Test if index is actually being used. + +-- Create temp, and populate it with some values in src. +CREATE TABLE temp(key STRING, val STRING) PARTITIONED BY (foo string) STORED AS TEXTFILE; +ALTER TABLE temp ADD PARTITION (foo = 'bar'); +INSERT OVERWRITE TABLE temp PARTITION (foo = 'bar') SELECT * FROM src WHERE key < 50; + +-- Build an index on temp. +CREATE INDEX temp_index ON TABLE temp(key) as 'COMPACT' WITH DEFERRED REBUILD; +ALTER INDEX temp_index ON temp PARTITION (foo = 'bar') REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +-- overwrite temp table so index is out of date +INSERT OVERWRITE TABLE temp PARTITION (foo = 'bar') SELECT * FROM src; + +-- query should not return any values +SELECT * FROM default__temp_temp_index__ WHERE key = 86 AND foo='bar'; +EXPLAIN SELECT * FROM temp WHERE key = 86 AND foo = 'bar'; +SELECT * FROM temp WHERE key = 86 AND foo = 'bar'; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET hive.optimize.index.filter=false; +DROP table temp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort.q new file mode 100644 index 0000000000000..07466fd871ae2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort.q @@ -0,0 +1,158 @@ +set hive.exec.infer.bucket.sort=true; + +-- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata + +CREATE TABLE test_table (key STRING, value STRING) PARTITIONED BY (part STRING); + +-- Test group by, should be bucketed and sorted by group by key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, count(*) FROM src GROUP BY key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by where a key isn't selected, should not be bucketed or sorted +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, count(*) FROM src GROUP BY key, value; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join, should be bucketed and sorted by join key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, a.value FROM src a JOIN src b ON a.key = b.key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join with two keys, should be bucketed and sorted by join keys +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, a.value FROM src a JOIN src b ON a.key = b.key AND a.value = b.value; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join with two keys and only one selected, should not be bucketed or sorted +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, '1' FROM src a JOIN src b ON a.key = b.key AND a.value = b.value; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join on three tables on same key, should be bucketed and sorted by join key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, c.value FROM src a JOIN src b ON (a.key = b.key) JOIN src c ON (b.key = c.key); + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join on three tables on different keys, should be bucketed and sorted by latter key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, c.value FROM src a JOIN src b ON (a.key = b.key) JOIN src c ON (b.value = c.value); + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test distribute by, should only be bucketed by key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value FROM src DISTRIBUTE BY key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test sort by, should be sorted by key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value FROM src SORT BY key ASC; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test sort by desc, should be sorted by key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value FROM src SORT BY key DESC; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test cluster by, should be bucketed and sorted by key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value FROM src CLUSTER BY key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test distribute by and sort by different keys, should be bucketed by one key sorted by the other +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value FROM src DISTRIBUTE BY key SORT BY value; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join in simple subquery, should be bucketed and sorted on key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value from (SELECT a.key, b.value FROM src a JOIN src b ON (a.key = b.key)) subq; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join in simple subquery renaming key column, should be bucketed and sorted on key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT k, value FROM (SELECT a.key as k, b.value FROM src a JOIN src b ON (a.key = b.key)) subq; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in simple subquery, should be bucketed and sorted on key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, cnt from (SELECT key, count(*) as cnt FROM src GROUP BY key) subq; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in simple subquery renaming key column, should be bucketed and sorted on key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT k, cnt FROM (SELECT key as k, count(*) as cnt FROM src GROUP BY key) subq; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in subquery with where outside, should still be bucketed and sorted on key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value FROM (SELECT key, count(1) AS value FROM src group by key) a where key < 10; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in subquery with expression on value, should still be bucketed and sorted on key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value + 1 FROM (SELECT key, count(1) AS value FROM src group by key) a where key < 10; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in subquery with lateral view outside, should still be bucketed and sorted +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value FROM (SELECT key FROM src group by key) a lateral view explode(array(1, 2)) value as value; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in subquery with another group by outside, should be bucketed and sorted by the +-- key of the outer group by +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT count(1), value FROM (SELECT key, count(1) as value FROM src group by key) a group by value; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in subquery with select on outside reordering the columns, should be bucketed and +-- sorted by the column the group by key ends up in +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT value, key FROM (SELECT key, count(1) as value FROM src group by key) a; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in subquery followed by distribute by, should only be bucketed by the distribute key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value FROM (SELECT key, count(1) as value FROM src group by key) a distribute by key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in subquery followed by sort by, should only be sorted by the sort key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value FROM (SELECT key, count(1) as value FROM src group by key) a sort by key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in subquery followed by transform script, should not be bucketed or sorted +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT TRANSFORM (a.key, a.value) USING 'cat' AS (key, value) FROM (SELECT key, count(1) AS value FROM src GROUP BY KEY) a; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by on function, should be bucketed and sorted by key and value because the function is applied in the mapper +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, value FROM (SELECT concat(key, "a") AS key, value, count(*) FROM src GROUP BY concat(key, "a"), value) a; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q new file mode 100644 index 0000000000000..d69f49f66747c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q @@ -0,0 +1,26 @@ +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.infer.bucket.sort=true; + +-- Test writing to a bucketed table, the output should be bucketed by the bucketing key into the +-- a number of files equal to the number of buckets +CREATE TABLE test_table_bucketed (key STRING, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (value) SORTED BY (value) INTO 3 BUCKETS; + +-- Despite the fact that normally inferring would say this table is bucketed and sorted on key, +-- this should be bucketed and sorted by value into 3 buckets +INSERT OVERWRITE TABLE test_table_bucketed PARTITION (part = '1') +SELECT key, count(1) FROM src GROUP BY KEY; + +DESCRIBE FORMATTED test_table_bucketed PARTITION (part = '1'); + +-- If the count(*) from sampling the buckets matches the count(*) from each file, the table is +-- bucketed +SELECT COUNT(*) FROM test_table_bucketed TABLESAMPLE (BUCKET 1 OUT OF 3) WHERE part = '1'; + +SELECT COUNT(*) FROM test_table_bucketed TABLESAMPLE (BUCKET 2 OUT OF 3) WHERE part = '1'; + +SELECT COUNT(*) FROM test_table_bucketed TABLESAMPLE (BUCKET 3 OUT OF 3) WHERE part = '1'; + +SELECT cnt FROM (SELECT INPUT__FILE__NAME, COUNT(*) cnt FROM test_table_bucketed WHERE part = '1' +GROUP BY INPUT__FILE__NAME ORDER BY INPUT__FILE__NAME ASC LIMIT 3) a; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q new file mode 100644 index 0000000000000..6b656ffde745e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q @@ -0,0 +1,27 @@ +set hive.exec.infer.bucket.sort=true; +set hive.exec.infer.bucket.sort.num.buckets.power.two=true; +set hive.auto.convert.join=true; + +-- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata. In particular, those cases +-- where joins may be auto converted to map joins. + +CREATE TABLE test_table (key STRING, value STRING) PARTITIONED BY (part STRING); + +-- Tests a join which is converted to a map join, the output should be neither bucketed nor sorted +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +set hive.mapjoin.check.memory.rows=1; +set hive.mapjoin.localtask.max.memory.usage = 0.0001; +set hive.auto.convert.join.noconditionaltask = false; + +-- This test tests the scenario when the mapper dies. So, create a conditional task for the mapjoin. +-- Tests a join which is not converted to a map join, the output should be bucketed and sorted. + +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q new file mode 100644 index 0000000000000..119994e91b056 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q @@ -0,0 +1,87 @@ +set hive.exec.infer.bucket.sort=true; +set hive.exec.infer.bucket.sort.num.buckets.power.two=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata. In particular, those cases +-- where dynamic partitioning is used. + +CREATE TABLE test_table LIKE srcpart; +ALTER TABLE test_table SET FILEFORMAT RCFILE; + +-- Simple case, this should not be bucketed or sorted + +INSERT OVERWRITE TABLE test_table PARTITION (ds, hr) +SELECT key, value, ds, hr FROM srcpart +WHERE ds = '2008-04-08'; + +DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11'); +DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12'); + +-- This should not be bucketed or sorted since the partition keys are in the set of bucketed +-- and sorted columns for the output + +INSERT OVERWRITE TABLE test_table PARTITION (ds, hr) +SELECT key, COUNT(*), ds, hr FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key, ds, hr; + +DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11'); +DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12'); + +-- Both partitions should be bucketed and sorted by key + +INSERT OVERWRITE TABLE test_table PARTITION (ds, hr) +SELECT key, value, '2008-04-08', IF (key % 2 == 0, '11', '12') FROM +(SELECT key, COUNT(*) AS value FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key) a; + +DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11'); +DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='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=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'; + +set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=200; +set hive.exec.compress.output=false; +set hive.exec.dynamic.partition=true; +set mapred.reduce.tasks=2; + +-- Tests dynamic partitions where bucketing/sorting can be inferred, but some partitions are +-- merged and some are moved. Currently neither should be bucketed or sorted, in the future, +-- (ds='2008-04-08', hr='12') may be bucketed and sorted, (ds='2008-04-08', hr='11') should +-- definitely not be. + +EXPLAIN +INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key, value, IF (key % 100 == 0, '11', '12') FROM +(SELECT key, COUNT(*) AS value FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key) a; + +INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key, value, IF (key % 100 == 0, '11', '12') FROM +(SELECT key, COUNT(*) AS value FROM srcpart +WHERE ds = '2008-04-08' +GROUP BY key) a; + +DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='11'); +DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q new file mode 100644 index 0000000000000..e3b8a17c83daa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q @@ -0,0 +1,57 @@ +set hive.exec.infer.bucket.sort=true; + +-- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata, in particular, this tests +-- the grouping operators rollup/cube/grouping sets + +CREATE TABLE test_table_out (key STRING, value STRING, agg STRING) PARTITIONED BY (part STRING); + +CREATE TABLE test_table_out_2 (key STRING, value STRING, grouping_key STRING, agg STRING) PARTITIONED BY (part STRING); + +-- Test rollup, should not be bucketed or sorted because its missing the grouping ID +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, value, count(1) FROM src GROUP BY key, value WITH ROLLUP; + +INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, value, count(1) FROM src GROUP BY key, value WITH ROLLUP; + +DESCRIBE FORMATTED test_table_out PARTITION (part = '1'); + +-- Test rollup, should be bucketed and sorted on key, value, grouping_key + +INSERT OVERWRITE TABLE test_table_out_2 PARTITION (part = '1') +SELECT key, value, GROUPING__ID, count(1) FROM src GROUP BY key, value WITH ROLLUP; + +DESCRIBE FORMATTED test_table_out_2 PARTITION (part = '1'); + +-- Test cube, should not be bucketed or sorted because its missing the grouping ID +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, value, count(1) FROM src GROUP BY key, value WITH CUBE; + +INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, value, count(1) FROM src GROUP BY key, value WITH CUBE; + +DESCRIBE FORMATTED test_table_out PARTITION (part = '1'); + +-- Test cube, should be bucketed and sorted on key, value, grouping_key + +INSERT OVERWRITE TABLE test_table_out_2 PARTITION (part = '1') +SELECT key, value, GROUPING__ID, count(1) FROM src GROUP BY key, value WITH CUBE; + +DESCRIBE FORMATTED test_table_out_2 PARTITION (part = '1'); + +-- Test grouping sets, should not be bucketed or sorted because its missing the grouping ID +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, value, count(1) FROM src GROUP BY key, value GROUPING SETS (key, value); + +INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, value, count(1) FROM src GROUP BY key, value GROUPING SETS (key, value); + +DESCRIBE FORMATTED test_table_out PARTITION (part = '1'); + +-- Test grouping sets, should be bucketed and sorted on key, value, grouping_key + +INSERT OVERWRITE TABLE test_table_out_2 PARTITION (part = '1') +SELECT key, value, GROUPING__ID, count(1) FROM src GROUP BY key, value GROUPING SETS (key, value); + +DESCRIBE FORMATTED test_table_out_2 PARTITION (part = '1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q new file mode 100644 index 0000000000000..7d3c0dc7d5859 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q @@ -0,0 +1,33 @@ +set hive.mapred.supports.subdirectories=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This tests that bucketing/sorting metadata is not inferred for tables with list bucketing + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- create a skewed table +CREATE TABLE list_bucketing_table (key STRING, value STRING) +PARTITIONED BY (part STRING) +SKEWED BY (key) ON ("484") +STORED AS DIRECTORIES; + +-- Tests group by, the output should neither be bucketed nor sorted + +INSERT OVERWRITE TABLE list_bucketing_table PARTITION (part = '1') +SELECT key, count(*) FROM src GROUP BY key; + +DESC FORMATTED list_bucketing_table PARTITION (part = '1'); + +-- create a table skewed on a key which doesnt exist in the data +CREATE TABLE list_bucketing_table2 (key STRING, value STRING) +PARTITIONED BY (part STRING) +SKEWED BY (key) ON ("abc") +STORED AS DIRECTORIES; + +-- should not be bucketed or sorted +INSERT OVERWRITE TABLE list_bucketing_table2 PARTITION (part = '1') +SELECT key, count(*) FROM src GROUP BY key; + +DESC FORMATTED list_bucketing_table2 PARTITION (part = '1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q new file mode 100644 index 0000000000000..16e8715a7f3d3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q @@ -0,0 +1,74 @@ +set hive.exec.infer.bucket.sort=true; +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; + +-- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata, in particular, this tests +-- that operators in the mapper have no effect + +CREATE TABLE test_table1 (key STRING, value STRING) +CLUSTERED BY (key) SORTED BY (key DESC) INTO 2 BUCKETS; + +CREATE TABLE test_table2 (key STRING, value STRING) +CLUSTERED BY (key) SORTED BY (key DESC) INTO 2 BUCKETS; + +INSERT OVERWRITE TABLE test_table1 SELECT key, value FROM src; + +INSERT OVERWRITE TABLE test_table2 SELECT key, value FROM src; + +CREATE TABLE test_table_out (key STRING, value STRING) PARTITIONED BY (part STRING); + +set hive.map.groupby.sorted=true; + +-- Test map group by doesn't affect inference, should not be bucketed or sorted +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, count(*) FROM test_table1 GROUP BY key; + +INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT key, count(*) FROM test_table1 GROUP BY key; + +DESCRIBE FORMATTED test_table_out PARTITION (part = '1'); + +-- Test map group by doesn't affect inference, should be bucketed and sorted by value +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT a.key, a.value FROM ( + SELECT key, count(*) AS value FROM test_table1 GROUP BY key +) a JOIN ( + SELECT key, value FROM src +) b +ON (a.value = b.value); + +INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT a.key, a.value FROM ( + SELECT key, cast(count(*) AS STRING) AS value FROM test_table1 GROUP BY key +) a JOIN ( + SELECT key, value FROM src +) b +ON (a.value = b.value); + +DESCRIBE FORMATTED test_table_out PARTITION (part = '1'); + +set hive.map.groupby.sorted=false; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +-- Test SMB join doesn't affect inference, should not be bucketed or sorted +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ a.key, b.value FROM test_table1 a JOIN test_table2 b ON a.key = b.key; + +INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ a.key, b.value FROM test_table1 a JOIN test_table2 b ON a.key = b.key; + +DESCRIBE FORMATTED test_table_out PARTITION (part = '1'); + +-- Test SMB join doesn't affect inference, should be bucketed and sorted by key +EXPLAIN INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ b.value, count(*) FROM test_table1 a JOIN test_table2 b ON a.key = b.key +GROUP BY b.value; + +INSERT OVERWRITE TABLE test_table_out PARTITION (part = '1') +SELECT /*+ MAPJOIN(a) */ b.value, count(*) FROM test_table1 a JOIN test_table2 b ON a.key = b.key +GROUP BY b.value; + +DESCRIBE FORMATTED test_table_out PARTITION (part = '1'); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q new file mode 100644 index 0000000000000..41c1a13980cfe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q @@ -0,0 +1,25 @@ +set hive.exec.infer.bucket.sort=true; +set hive.exec.infer.bucket.sort.num.buckets.power.two=true; +set hive.merge.mapredfiles=true; +set mapred.reduce.tasks=2; + +-- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata. In particular, those cases +-- where where merging may or may not be used. + +CREATE TABLE test_table (key STRING, value STRING) PARTITIONED BY (part STRING); + +-- Tests a reduce task followed by a merge. The output should be neither bucketed nor sorted. +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +set hive.merge.smallfiles.avgsize=2; +set hive.exec.compress.output=false; + +-- Tests a reduce task followed by a move. The output should be bucketed and sorted. +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q new file mode 100644 index 0000000000000..e3992b8da07b8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q @@ -0,0 +1,46 @@ +set hive.exec.infer.bucket.sort=true; +set hive.exec.infer.bucket.sort.num.buckets.power.two=true; + +-- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata. In particular, those cases +-- where multi insert is used. + +CREATE TABLE test_table (key STRING, value STRING) PARTITIONED BY (part STRING); + +-- Simple case, neither partition should be bucketed or sorted + +FROM src +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') SELECT key, value +INSERT OVERWRITE TABLE test_table PARTITION (part = '2') SELECT value, key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); +DESCRIBE FORMATTED test_table PARTITION (part = '2'); + +-- The partitions should be bucketed and sorted by different keys + +FROM src +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') SELECT key, COUNT(*) GROUP BY key +INSERT OVERWRITE TABLE test_table PARTITION (part = '2') SELECT COUNT(*), value GROUP BY value; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); +DESCRIBE FORMATTED test_table PARTITION (part = '2'); + +-- The first partition should be bucketed and sorted, the second should not + +FROM src +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') SELECT key, COUNT(*) GROUP BY key +INSERT OVERWRITE TABLE test_table PARTITION (part = '2') SELECT key, value; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); +DESCRIBE FORMATTED test_table PARTITION (part = '2'); + +set hive.multigroupby.singlereducer=true; + +-- Test the multi group by single reducer optimization +-- Both partitions should be bucketed by key +FROM src +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') SELECT key, COUNT(*) GROUP BY key +INSERT OVERWRITE TABLE test_table PARTITION (part = '2') SELECT key, SUM(SUBSTR(value, 5)) GROUP BY key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); +DESCRIBE FORMATTED test_table PARTITION (part = '2'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q new file mode 100644 index 0000000000000..2255bdb34913d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q @@ -0,0 +1,37 @@ +set hive.exec.infer.bucket.sort=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.reduce.tasks=2; + +CREATE TABLE test_table (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING); + +-- Tests dynamic partitions where bucketing/sorting can be inferred, but not all reducers write +-- all partitions. The subquery produces rows as follows +-- key = 0: +-- 0, , 0 +-- key = 1: +-- 0, , 1 +-- key = 2: +-- 1, , 0 +-- This means that by distributing by the first column into two reducers, and using the third +-- columns as a dynamic partition, the dynamic partition for 0 will get written in both reducers +-- and the partition for 1 will get written in one reducer. So hr=0 should be bucketed by key +-- and hr=1 should not. + +EXPLAIN +INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key2, value, cast(hr as int) FROM +(SELECT if ((key % 3) < 2, 0, 1) as key2, value, (key % 2) as hr +FROM srcpart +WHERE ds = '2008-04-08') a +DISTRIBUTE BY key2; + +INSERT OVERWRITE TABLE test_table PARTITION (ds = '2008-04-08', hr) +SELECT key2, value, cast(hr as int) FROM +(SELECT if ((key % 3) < 2, 0, 1) as key2, value, (key % 3 % 2) as hr +FROM srcpart +WHERE ds = '2008-04-08') a +DISTRIBUTE BY key2; + +DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='0'); +DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q new file mode 100644 index 0000000000000..090f04f4b94c9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q @@ -0,0 +1,46 @@ +set hive.exec.infer.bucket.sort=true; +set hive.exec.infer.bucket.sort.num.buckets.power.two=true; +set hive.exec.reducers.bytes.per.reducer=2500; + +-- This tests inferring how data is bucketed/sorted from the operators in the reducer +-- and populating that information in partitions' metadata, it also verifies that the +-- number of reducers chosen will be a power of two + +CREATE TABLE test_table (key STRING, value STRING) PARTITIONED BY (part STRING); + +-- Test group by, should be bucketed and sorted by group by key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT key, count(*) FROM src GROUP BY key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join, should be bucketed and sorted by join key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, a.value FROM src a JOIN src b ON a.key = b.key; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join with two keys, should be bucketed and sorted by join keys +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, a.value FROM src a JOIN src b ON a.key = b.key AND a.value = b.value; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join on three tables on same key, should be bucketed and sorted by join key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, c.value FROM src a JOIN src b ON (a.key = b.key) JOIN src c ON (b.key = c.key); + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test join on three tables on different keys, should be bucketed and sorted by latter key +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT a.key, c.value FROM src a JOIN src b ON (a.key = b.key) JOIN src c ON (b.value = c.value); + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); + +-- Test group by in subquery with another group by outside, should be bucketed and sorted by the +-- key of the outer group by +INSERT OVERWRITE TABLE test_table PARTITION (part = '1') +SELECT count(1), value FROM (SELECT key, count(1) as value FROM src group by key) a group by value; + +DESCRIBE FORMATTED test_table PARTITION (part = '1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_const_type.q b/src/test/hive/ql/src/test/queries/clientpositive/infer_const_type.q new file mode 100644 index 0000000000000..a039dc5a2a040 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/infer_const_type.q @@ -0,0 +1,66 @@ +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; + +SELECT * FROM infertypes; + +EXPLAIN SELECT * FROM infertypes WHERE + ti = '127' AND + si = 32767 AND + i = '12345' AND + bi = '-12345' AND + fl = '0906' AND + db = '-307' AND + str = 1234; + +SELECT * FROM infertypes WHERE + ti = '127' AND + si = 32767 AND + i = '12345' AND + bi = '-12345' AND + fl = '0906' AND + db = '-307' AND + str = 1234; + +-- all should return false as all numbers exceeed the largest number +-- which could be represented by the corresponding type +-- and string_col = long_const should return false +EXPLAIN SELECT * FROM infertypes WHERE + ti = '128' OR + si = 32768 OR + i = '2147483648' OR + bi = '9223372036854775808' OR + fl = 'float' OR + db = 'double'; + +SELECT * FROM infertypes WHERE + ti = '128' OR + si = 32768 OR + i = '2147483648' OR + bi = '9223372036854775808' OR + fl = 'float' OR + db = 'double'; + +-- for the query like: int_col = double, should return false +EXPLAIN SELECT * FROM infertypes WHERE + ti = '127.0' OR + si = 327.0 OR + i = '-100.0'; + +SELECT * FROM infertypes WHERE + ti = '127.0' OR + si = 327.0 OR + i = '-100.0'; + +EXPLAIN SELECT * FROM infertypes WHERE + ti < '127.0' AND + i > '100.0' AND + str = 1.57; + +SELECT * FROM infertypes WHERE + ti < '127.0' AND + i > '100.0' AND + str = 1.57; + +DROP TABLE infertypes; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/init_file.q b/src/test/hive/ql/src/test/queries/clientpositive/init_file.q new file mode 100644 index 0000000000000..f69a88d0366e5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/init_file.q @@ -0,0 +1,5 @@ +-- tbl_created_by_init is supposed to have been created for us +-- automatically by test_init_file.sql + +select * from tbl_created_by_init; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/innerjoin.q b/src/test/hive/ql/src/test/queries/clientpositive/innerjoin.q new file mode 100644 index 0000000000000..b384268dbbcff --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/innerjoin.q @@ -0,0 +1,23 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src src1 INNER JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value ORDER BY src1.key, src2.value; + +FROM src src1 INNER JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value ORDER BY src1.key, src2.value; + +SELECT dest_j1.* FROM dest_j1; + +-- verify that INNER is a non-reserved word for backwards compatibility +create table inner(i int); + +select i from inner; + +create table i(inner int); + +select inner from i; + +explain select * from (select * from src) inner left outer join src +on inner.key=src.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inoutdriver.q b/src/test/hive/ql/src/test/queries/clientpositive/inoutdriver.q new file mode 100644 index 0000000000000..34b0e0b28ea52 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/inoutdriver.q @@ -0,0 +1,2 @@ +create table test (a int) stored as inputformat 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' outputformat 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' inputdriver 'RCFileInDriver' outputdriver 'RCFileOutDriver'; +desc extended test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input.q b/src/test/hive/ql/src/test/queries/clientpositive/input.q new file mode 100644 index 0000000000000..567d3b07f7ecc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT x.* FROM SRC x; + +SELECT x.* FROM SRC x; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input0.q b/src/test/hive/ql/src/test/queries/clientpositive/input0.q new file mode 100644 index 0000000000000..63857c779173b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input0.q @@ -0,0 +1,5 @@ +EXPLAIN +SELECT * FROM src; + +SELECT * FROM src; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input1.q b/src/test/hive/ql/src/test/queries/clientpositive/input1.q new file mode 100644 index 0000000000000..3f2cd96de9cbe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input1.q @@ -0,0 +1,9 @@ +CREATE TABLE TEST1(A INT, B DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +DESCRIBE TEST1; + +DESCRIBE TEST1; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input10.q b/src/test/hive/ql/src/test/queries/clientpositive/input10.q new file mode 100644 index 0000000000000..aef5c9bfe4b0a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input10.q @@ -0,0 +1,9 @@ +CREATE TABLE TEST10(key INT, value STRING) PARTITIONED BY(ds STRING, hr STRING) STORED AS TEXTFILE; + +EXPLAIN +DESCRIBE TEST10; + +DESCRIBE TEST10; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input11.q b/src/test/hive/ql/src/test/queries/clientpositive/input11.q new file mode 100644 index 0000000000000..5aa6ee785d682 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input11.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input11_limit.q b/src/test/hive/ql/src/test/queries/clientpositive/input11_limit.q new file mode 100644 index 0000000000000..f68398e3159b2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input11_limit.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100 LIMIT 10; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100 LIMIT 10; + +SELECT dest1.* FROM dest1 ORDER BY dest1.key ASC, dest1.value ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input12.q b/src/test/hive/ql/src/test/queries/clientpositive/input12.q new file mode 100644 index 0000000000000..d4bc409cce7d0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input12.q @@ -0,0 +1,24 @@ +set mapreduce.framework.name=yarn; +set mapreduce.jobtracker.address=localhost:58; +set hive.exec.mode.local.auto=true; + +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE dest2(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE dest3(key INT) PARTITIONED BY(ds STRING, hr STRING) STORED AS TEXTFILE; + +EXPLAIN +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; + +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; + +SELECT dest1.* FROM dest1; +SELECT dest2.* FROM dest2; +SELECT dest3.* FROM dest3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input12_hadoop20.q b/src/test/hive/ql/src/test/queries/clientpositive/input12_hadoop20.q new file mode 100644 index 0000000000000..318cd378db137 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input12_hadoop20.q @@ -0,0 +1,23 @@ +set mapred.job.tracker=localhost:58; +set hive.exec.mode.local.auto=true; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE dest2(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE dest3(key INT) PARTITIONED BY(ds STRING, hr STRING) STORED AS TEXTFILE; + +EXPLAIN +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; + +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; + +SELECT dest1.* FROM dest1; +SELECT dest2.* FROM dest2; +SELECT dest3.* FROM dest3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input13.q b/src/test/hive/ql/src/test/queries/clientpositive/input13.q new file mode 100644 index 0000000000000..40fbc84a981f1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input13.q @@ -0,0 +1,21 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE dest2(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE dest3(key INT) PARTITIONED BY(ds STRING, hr STRING) STORED AS TEXTFILE; + +EXPLAIN +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; + +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; + +SELECT dest1.* FROM dest1; +SELECT dest2.* FROM dest2; +SELECT dest3.* FROM dest3; +dfs -cat ../build/ql/test/data/warehouse/dest4.out/*; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input14.q b/src/test/hive/ql/src/test/queries/clientpositive/input14.q new file mode 100644 index 0000000000000..750c0b82aa3f1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input14.q @@ -0,0 +1,20 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input14_limit.q b/src/test/hive/ql/src/test/queries/clientpositive/input14_limit.q new file mode 100644 index 0000000000000..7316752a6df65 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input14_limit.q @@ -0,0 +1,20 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey LIMIT 20 +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey LIMIT 20 +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input15.q b/src/test/hive/ql/src/test/queries/clientpositive/input15.q new file mode 100644 index 0000000000000..2b76497fe5a7a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input15.q @@ -0,0 +1,7 @@ +EXPLAIN +CREATE TABLE TEST15(key INT, value STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; + +CREATE TABLE TEST15(key INT, value STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; + +DESCRIBE TEST15; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input16.q b/src/test/hive/ql/src/test/queries/clientpositive/input16.q new file mode 100644 index 0000000000000..82e6d81426efc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input16.q @@ -0,0 +1,6 @@ +-- TestSerDe is a user defined serde where the default delimiter is Ctrl-B +DROP TABLE INPUT16; +ADD JAR ../data/files/TestSerDe.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; +SELECT INPUT16.VALUE, INPUT16.KEY FROM INPUT16; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input16_cc.q b/src/test/hive/ql/src/test/queries/clientpositive/input16_cc.q new file mode 100644 index 0000000000000..5dab4103d8dda --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input16_cc.q @@ -0,0 +1,11 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +-- TestSerDe is a user defined serde where the default delimiter is Ctrl-B +-- the user is overwriting it with ctrlC + +DROP TABLE INPUT16_CC; +ADD JAR ../data/files/TestSerDe.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; +SELECT INPUT16_CC.VALUE, INPUT16_CC.KEY FROM INPUT16_CC; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input17.q b/src/test/hive/ql/src/test/queries/clientpositive/input17.q new file mode 100644 index 0000000000000..14bfb640067bb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input17.q @@ -0,0 +1,20 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM src_thrift + SELECT TRANSFORM(src_thrift.aint + src_thrift.lint[0], src_thrift.lintstring[0]) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue; + +FROM ( + FROM src_thrift + SELECT TRANSFORM(src_thrift.aint + src_thrift.lint[0], src_thrift.lintstring[0]) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input18.q b/src/test/hive/ql/src/test/queries/clientpositive/input18.q new file mode 100644 index 0000000000000..4761dd6cac34a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input18.q @@ -0,0 +1,20 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value, 1+2, 3+4) + USING 'cat' + CLUSTER BY key +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.key, regexp_replace(tmap.value,'\t','+') WHERE tmap.key < 100; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value, 1+2, 3+4) + USING 'cat' + CLUSTER BY key +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.key, regexp_replace(tmap.value,'\t','+') WHERE tmap.key < 100; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input19.q b/src/test/hive/ql/src/test/queries/clientpositive/input19.q new file mode 100644 index 0000000000000..fec44e97669dd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input19.q @@ -0,0 +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; +SELECT a.* FROM apachelog a; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input1_limit.q b/src/test/hive/ql/src/test/queries/clientpositive/input1_limit.q new file mode 100644 index 0000000000000..71a9b8ff46a25 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input1_limit.q @@ -0,0 +1,18 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE dest2(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100 LIMIT 10 +INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key < 100 LIMIT 5; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100 LIMIT 10 +INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key < 100 LIMIT 5; + +SELECT dest1.* FROM dest1 ORDER BY dest1.key ASC, dest1.value ASC; +SELECT dest2.* FROM dest2 ORDER BY dest2.key ASC, dest2.value ASC; + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input2.q b/src/test/hive/ql/src/test/queries/clientpositive/input2.q new file mode 100644 index 0000000000000..00693851ef1a7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input2.q @@ -0,0 +1,12 @@ +CREATE TABLE TEST2a(A INT, B DOUBLE) STORED AS TEXTFILE; +DESCRIBE TEST2a; +DESC TEST2a; +CREATE TABLE TEST2b(A ARRAY, B DOUBLE, C MAP) STORED AS TEXTFILE; +DESCRIBE TEST2b; +SHOW TABLES; +DROP TABLE TEST2a; +SHOW TABLES; +DROP TABLE TEST2b; + +EXPLAIN +SHOW TABLES; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input20.q b/src/test/hive/ql/src/test/queries/clientpositive/input20.q new file mode 100644 index 0000000000000..0566ab18c2537 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input20.q @@ -0,0 +1,30 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +ADD FILE ../data/scripts/input20_script; + +EXPLAIN +FROM ( + FROM src + MAP src.key, src.key + USING 'cat' + DISTRIBUTE BY key + SORT BY key, value +) tmap +INSERT OVERWRITE TABLE dest1 +REDUCE tmap.key, tmap.value +USING 'input20_script' +AS key, value; + +FROM ( + FROM src + MAP src.key, src.key + USING 'cat' + DISTRIBUTE BY key + SORT BY key, value +) tmap +INSERT OVERWRITE TABLE dest1 +REDUCE tmap.key, tmap.value +USING 'input20_script' +AS key, value; + +SELECT * FROM dest1 SORT BY key, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input21.q b/src/test/hive/ql/src/test/queries/clientpositive/input21.q new file mode 100644 index 0000000000000..d7c814e58061c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input21.q @@ -0,0 +1,10 @@ + + +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; + +EXPLAIN SELECT * FROM src_null DISTRIBUTE BY c SORT BY d; + +SELECT * FROM src_null DISTRIBUTE BY c SORT BY d; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input22.q b/src/test/hive/ql/src/test/queries/clientpositive/input22.q new file mode 100644 index 0000000000000..853947be57a1f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input22.q @@ -0,0 +1,15 @@ +CREATE TABLE INPUT4(KEY STRING, VALUE STRING) STORED AS TEXTFILE; +LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4; + +EXPLAIN +SELECT a.KEY2 +FROM (SELECT INPUT4.*, INPUT4.KEY as KEY2 + FROM INPUT4) a +ORDER BY KEY2 LIMIT 10; + +SELECT a.KEY2 +FROM (SELECT INPUT4.*, INPUT4.KEY as KEY2 + FROM INPUT4) a +ORDER BY KEY2 LIMIT 10; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input23.q b/src/test/hive/ql/src/test/queries/clientpositive/input23.q new file mode 100644 index 0000000000000..634699a00c4a0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input23.q @@ -0,0 +1,5 @@ +explain extended + select * from srcpart a join srcpart b where a.ds = '2008-04-08' and a.hr = '11' and b.ds = '2008-04-08' and b.hr = '14' limit 5; + +select * from srcpart a join srcpart b where a.ds = '2008-04-08' and a.hr = '11' and b.ds = '2008-04-08' and b.hr = '14' limit 5; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input24.q b/src/test/hive/ql/src/test/queries/clientpositive/input24.q new file mode 100644 index 0000000000000..95b2377f51b9d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input24.q @@ -0,0 +1,9 @@ + +create table tst(a int, b int) partitioned by (d string); +alter table tst add partition (d='2009-01-01'); +explain +select count(1) from tst x where x.d='2009-01-01'; + +select count(1) from tst x where x.d='2009-01-01'; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input25.q b/src/test/hive/ql/src/test/queries/clientpositive/input25.q new file mode 100644 index 0000000000000..e48368ff2a494 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input25.q @@ -0,0 +1,19 @@ + +create table tst(a int, b int) partitioned by (d string); +alter table tst add partition (d='2009-01-01'); +alter table tst add partition (d='2009-02-02'); + +explain +select * from ( + select * from tst x where x.d='2009-01-01' limit 10 + union all + select * from tst x where x.d='2009-02-02' limit 10 +) subq; + +select * from ( + select * from tst x where x.d='2009-01-01' limit 10 + union all + select * from tst x where x.d='2009-02-02' limit 10 +) subq; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input26.q b/src/test/hive/ql/src/test/queries/clientpositive/input26.q new file mode 100644 index 0000000000000..642a7db60ebdf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input26.q @@ -0,0 +1,12 @@ +explain +select * from ( + select * from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5 + union all + select * from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5 +)subq; + +select * from ( + select * from srcpart a where a.ds = '2008-04-08' and a.hr = '11' order by a.key limit 5 + union all + select * from srcpart b where b.ds = '2008-04-08' and b.hr = '14' limit 5 +)subq; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input28.q b/src/test/hive/ql/src/test/queries/clientpositive/input28.q new file mode 100644 index 0000000000000..12e857df8a325 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input28.q @@ -0,0 +1,10 @@ + +create table tst(a string, b string) partitioned by (d string); +alter table tst add partition (d='2009-01-01'); + +insert overwrite table tst partition(d='2009-01-01') +select tst.a, src.value from tst join src ON (tst.a = src.key); + +select * from tst where tst.d='2009-01-01'; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input2_limit.q b/src/test/hive/ql/src/test/queries/clientpositive/input2_limit.q new file mode 100644 index 0000000000000..81045100d380c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input2_limit.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT x.* FROM SRC x WHERE x.key < 300 LIMIT 5; + +SELECT x.* FROM SRC x WHERE x.key < 300 LIMIT 5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input3.q b/src/test/hive/ql/src/test/queries/clientpositive/input3.q new file mode 100644 index 0000000000000..2efa7a4d3d6bc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input3.q @@ -0,0 +1,26 @@ + + + + +CREATE TABLE TEST3a(A INT, B DOUBLE) STORED AS TEXTFILE; +DESCRIBE TEST3a; +CREATE TABLE TEST3b(A ARRAY, B DOUBLE, C MAP) STORED AS TEXTFILE; +DESCRIBE TEST3b; +SHOW TABLES; +EXPLAIN +ALTER TABLE TEST3b ADD COLUMNS (X DOUBLE); +ALTER TABLE TEST3b ADD COLUMNS (X DOUBLE); +DESCRIBE TEST3b; +EXPLAIN +ALTER TABLE TEST3b RENAME TO TEST3c; +ALTER TABLE TEST3b RENAME TO TEST3c; +DESCRIBE TEST3c; +SHOW TABLES; +EXPLAIN +ALTER TABLE TEST3c REPLACE COLUMNS (R1 INT, R2 DOUBLE); +ALTER TABLE TEST3c REPLACE COLUMNS (R1 INT, R2 DOUBLE); +DESCRIBE EXTENDED TEST3c; + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input30.q b/src/test/hive/ql/src/test/queries/clientpositive/input30.q new file mode 100644 index 0000000000000..c84c131082ce0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input30.q @@ -0,0 +1,23 @@ + + + + +create table dest30(a int); +create table tst_dest30(a int); + +set hive.test.mode=true; +set hive.test.mode.prefix=tst_; + +explain +insert overwrite table dest30 +select count(1) from src; + +insert overwrite table dest30 +select count(1) from src; + +set hive.test.mode=false; + +select * from tst_dest30; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input31.q b/src/test/hive/ql/src/test/queries/clientpositive/input31.q new file mode 100644 index 0000000000000..ece940903a521 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input31.q @@ -0,0 +1,24 @@ + + + +set hive.test.mode=true; +set hive.test.mode.prefix=tst_; + +create table tst_dest31(a int); +create table dest31(a int); + +explain +insert overwrite table dest31 +select count(1) from srcbucket; + +insert overwrite table dest31 +select count(1) from srcbucket; + +set hive.test.mode=false; + +select * from tst_dest31; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input32.q b/src/test/hive/ql/src/test/queries/clientpositive/input32.q new file mode 100644 index 0000000000000..cdc04b8c05b85 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input32.q @@ -0,0 +1,25 @@ + + + +set hive.test.mode=true; +set hive.test.mode.prefix=tst_; +set hive.test.mode.nosamplelist=src,srcbucket; + +create table dest32(a int); +create table tst_dest32(a int); + +explain +insert overwrite table dest32 +select count(1) from srcbucket; + +insert overwrite table dest32 +select count(1) from srcbucket; + +set hive.test.mode=false; + +select * from tst_dest32; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input33.q b/src/test/hive/ql/src/test/queries/clientpositive/input33.q new file mode 100644 index 0000000000000..7ab17515af5c7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input33.q @@ -0,0 +1,30 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +ADD FILE ../data/scripts/input20_script; + +EXPLAIN +FROM ( + FROM src + MAP src.key, src.key + USING 'cat' + DISTRIBUTE BY key + SORT BY key, value +) tmap +INSERT OVERWRITE TABLE dest1 +REDUCE tmap.key, tmap.value +USING 'input20_script' +AS (key STRING, value STRING); + +FROM ( + FROM src + MAP src.key, src.key + USING 'cat' + DISTRIBUTE BY key + SORT BY key, value +) tmap +INSERT OVERWRITE TABLE dest1 +REDUCE tmap.key, tmap.value +USING 'input20_script' +AS (key STRING, value STRING); + +SELECT * FROM dest1 SORT BY key, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input34.q b/src/test/hive/ql/src/test/queries/clientpositive/input34.q new file mode 100644 index 0000000000000..c600bf57e49d3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input34.q @@ -0,0 +1,20 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + USING 'cat' + AS (tkey, tvalue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tkey, tvalue; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + USING 'cat' + AS (tkey, tvalue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tkey, tvalue; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input35.q b/src/test/hive/ql/src/test/queries/clientpositive/input35.q new file mode 100644 index 0000000000000..5e37ea370a2ef --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input35.q @@ -0,0 +1,20 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' + USING 'cat' + AS (tkey, tvalue) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tkey, tvalue; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' + USING 'cat' + AS (tkey, tvalue) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tkey, tvalue; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input36.q b/src/test/hive/ql/src/test/queries/clientpositive/input36.q new file mode 100644 index 0000000000000..fd1968f683550 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input36.q @@ -0,0 +1,20 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' + USING 'cat' + AS (tkey, tvalue) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\003' +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tkey, tvalue; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' + USING 'cat' + AS (tkey, tvalue) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\003' +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tkey, tvalue; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input37.q b/src/test/hive/ql/src/test/queries/clientpositive/input37.q new file mode 100644 index 0000000000000..6fd136afec444 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input37.q @@ -0,0 +1,16 @@ +create table documents(contents string) stored as textfile; + +LOAD DATA LOCAL INPATH '../data/files/docurl.txt' INTO TABLE documents; + + +select url, count(1) +FROM +( + FROM documents + MAP documents.contents + USING 'java -cp ../build/ql/test/classes org.apache.hadoop.hive.scripts.extracturl' AS (url, count) +) subq +group by url; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input38.q b/src/test/hive/ql/src/test/queries/clientpositive/input38.q new file mode 100644 index 0000000000000..1ccc5f26a603a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input38.q @@ -0,0 +1,22 @@ + +CREATE TABLE dest1(key STRING, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value, 1+2, 3+4) + USING 'cat' +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.key, tmap.value; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value, 1+2, 3+4) + USING 'cat' +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.key, tmap.value; + + +SELECT dest1.* FROM dest1; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input39.q b/src/test/hive/ql/src/test/queries/clientpositive/input39.q new file mode 100644 index 0000000000000..04201dd108b35 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input39.q @@ -0,0 +1,32 @@ +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + + +create table t1(key string, value string) partitioned by (ds string); +create table t2(key string, value string) partitioned by (ds string); + +insert overwrite table t1 partition (ds='1') +select key, value from src; + +insert overwrite table t1 partition (ds='2') +select key, value from src; + +insert overwrite table t2 partition (ds='1') +select key, value from src; + +set hive.test.mode=true; +set hive.mapred.mode=strict; +set mapreduce.framework.name=yarn; +set mapreduce.jobtracker.address=localhost:58; +set hive.exec.mode.local.auto=true; + +explain +select count(1) from t1 join t2 on t1.key=t2.key where t1.ds='1' and t2.ds='1'; + +select count(1) from t1 join t2 on t1.key=t2.key where t1.ds='1' and t2.ds='1'; + +set hive.test.mode=false; +set mapreduce.framework.name; +set mapreduce.jobtracker.address; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input39_hadoop20.q b/src/test/hive/ql/src/test/queries/clientpositive/input39_hadoop20.q new file mode 100644 index 0000000000000..29e9fae1da9e3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input39_hadoop20.q @@ -0,0 +1,30 @@ +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + + +create table t1(key string, value string) partitioned by (ds string); +create table t2(key string, value string) partitioned by (ds string); + +insert overwrite table t1 partition (ds='1') +select key, value from src; + +insert overwrite table t1 partition (ds='2') +select key, value from src; + +insert overwrite table t2 partition (ds='1') +select key, value from src; + +set hive.test.mode=true; +set hive.mapred.mode=strict; +set mapred.job.tracker=localhost:58; +set hive.exec.mode.local.auto=true; + +explain +select count(1) from t1 join t2 on t1.key=t2.key where t1.ds='1' and t2.ds='1'; + +select count(1) from t1 join t2 on t1.key=t2.key where t1.ds='1' and t2.ds='1'; + +set hive.test.mode=false; +set mapred.job.tracker; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input3_limit.q b/src/test/hive/ql/src/test/queries/clientpositive/input3_limit.q new file mode 100644 index 0000000000000..3584820acaf4a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input3_limit.q @@ -0,0 +1,17 @@ + +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; + + +CREATE TABLE T2(key STRING, value STRING); + +EXPLAIN +INSERT OVERWRITE TABLE T2 SELECT * FROM (SELECT * FROM T1 DISTRIBUTE BY key SORT BY key, value) T LIMIT 20; + +INSERT OVERWRITE TABLE T2 SELECT * FROM (SELECT * FROM T1 DISTRIBUTE BY key SORT BY key, value) T LIMIT 20; + +SELECT * FROM T2 SORT BY key, value; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input4.q b/src/test/hive/ql/src/test/queries/clientpositive/input4.q new file mode 100644 index 0000000000000..08d6d97603045 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input4.q @@ -0,0 +1,8 @@ +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; +EXPLAIN FORMATTED +SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias; +SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input40.q b/src/test/hive/ql/src/test/queries/clientpositive/input40.q new file mode 100644 index 0000000000000..4166cb5f94939 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input40.q @@ -0,0 +1,16 @@ + + + +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; +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'); +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'); +select * from tmp_insert_test_p where ds= '2009-08-01' +order by key, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input41.q b/src/test/hive/ql/src/test/queries/clientpositive/input41.q new file mode 100644 index 0000000000000..5bc396e311f69 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input41.q @@ -0,0 +1,14 @@ +set hive.mapred.mode=strict; + +create table dest_sp (cnt int); + +insert overwrite table dest_sp +select * from + (select count(1) as cnt from src + union all + select count(1) as cnt from srcpart where ds = '2009-08-09' + )x; + +select * from dest_sp x order by x.cnt limit 2; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input42.q b/src/test/hive/ql/src/test/queries/clientpositive/input42.q new file mode 100644 index 0000000000000..8682eb7592fa6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input42.q @@ -0,0 +1,16 @@ +explain extended +select * from srcpart a where a.ds='2008-04-08' order by a.key, a.hr; + +select * from srcpart a where a.ds='2008-04-08' order by a.key, a.hr; + + +explain extended +select * from srcpart a where a.ds='2008-04-08' and key < 200 order by a.key, a.hr; + +select * from srcpart a where a.ds='2008-04-08' and key < 200 order by a.key, a.hr; + + +explain extended +select * from srcpart a where a.ds='2008-04-08' and rand(100) < 0.1 order by a.key, a.hr; + +select * from srcpart a where a.ds='2008-04-08' and rand(100) < 0.1 order by a.key, a.hr; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input43.q b/src/test/hive/ql/src/test/queries/clientpositive/input43.q new file mode 100644 index 0000000000000..5512dc328065d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input43.q @@ -0,0 +1,7 @@ +drop table tst_src1; +create table tst_src1 like 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 ; +select count(1) from tst_src1; +drop table tst_src1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input44.q b/src/test/hive/ql/src/test/queries/clientpositive/input44.q new file mode 100644 index 0000000000000..4557edc178b7f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input44.q @@ -0,0 +1,6 @@ +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 diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input45.q b/src/test/hive/ql/src/test/queries/clientpositive/input45.q new file mode 100644 index 0000000000000..633a8c6edb5cc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input45.q @@ -0,0 +1,9 @@ +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; + +dfs -cat ../build/ql/test/data/x/y/z/*.txt; + +dfs -rmr ../build/ql/test/data/x; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input46.q b/src/test/hive/ql/src/test/queries/clientpositive/input46.q new file mode 100644 index 0000000000000..06291d0d66cc5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input46.q @@ -0,0 +1,6 @@ +create database if not exists table_in_database_creation; +create table table_in_database_creation.test1 as select * from src limit 1; +create table `table_in_database_creation.test2` as select * from src limit 1; +create table table_in_database_creation.test3 (a string); +create table `table_in_database_creation.test4` (a string); +drop database table_in_database_creation cascade; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input49.q b/src/test/hive/ql/src/test/queries/clientpositive/input49.q new file mode 100644 index 0000000000000..53fa51a6d3e4e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input49.q @@ -0,0 +1,4 @@ +create table intable (b boolean, d double, f float, i int, l bigint, s string, t tinyint); +insert overwrite table intable select 0, 29098519.0, 1410.0, 996, 40408519555, "test_string", 12 from src limit 1; +select * from intable where d in (29098519.0) and f in (1410.0) and i in (996) and l in (40408519555) and s in ('test_string') and t in (12); +drop table intable; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input4_cb_delim.q b/src/test/hive/ql/src/test/queries/clientpositive/input4_cb_delim.q new file mode 100644 index 0000000000000..8c57dd3f25aa6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input4_cb_delim.q @@ -0,0 +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; +SELECT INPUT4_CB.VALUE, INPUT4_CB.KEY FROM INPUT4_CB; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input4_limit.q b/src/test/hive/ql/src/test/queries/clientpositive/input4_limit.q new file mode 100644 index 0000000000000..e6e3c467fc763 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input4_limit.q @@ -0,0 +1,5 @@ +explain +select * from src sort by key limit 10; + + +select * from src sort by key limit 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input5.q b/src/test/hive/ql/src/test/queries/clientpositive/input5.q new file mode 100644 index 0000000000000..2af9163da82c4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input5.q @@ -0,0 +1,20 @@ +CREATE TABLE dest1(key STRING, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM src_thrift + SELECT TRANSFORM(src_thrift.lint, src_thrift.lintstring) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue; + +FROM ( + FROM src_thrift + SELECT TRANSFORM(src_thrift.lint, src_thrift.lintstring) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input6.q b/src/test/hive/ql/src/test/queries/clientpositive/input6.q new file mode 100644 index 0000000000000..4f2f17e49be6e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input6.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(key STRING, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src1.value WHERE src1.key is null; + +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src1.value WHERE src1.key is null; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input7.q b/src/test/hive/ql/src/test/queries/clientpositive/input7.q new file mode 100644 index 0000000000000..f2e1ed0d3d79d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input7.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(c1 DOUBLE, c2 INT) STORED AS TEXTFILE; + +EXPLAIN +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key; + +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input8.q b/src/test/hive/ql/src/test/queries/clientpositive/input8.q new file mode 100644 index 0000000000000..66252a9c5e100 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input8.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(c1 STRING, c2 INT, c3 DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT 4 + NULL, src1.key - NULL, NULL + NULL; + +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT 4 + NULL, src1.key - NULL, NULL + NULL; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input9.q b/src/test/hive/ql/src/test/queries/clientpositive/input9.q new file mode 100644 index 0000000000000..ded20a81c6511 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input9.q @@ -0,0 +1,11 @@ +CREATE TABLE dest1(value STRING, key INT) STORED AS TEXTFILE; + +EXPLAIN +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key where NULL = NULL; + +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key where NULL = NULL; + +SELECT dest1.* FROM dest1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_columnarserde.q b/src/test/hive/ql/src/test/queries/clientpositive/input_columnarserde.q new file mode 100644 index 0000000000000..9394382b2b347 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_columnarserde.q @@ -0,0 +1,19 @@ + +CREATE TABLE input_columnarserde(a array, b array, c map, d int, e 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'; + +EXPLAIN +FROM src_thrift +INSERT OVERWRITE TABLE input_columnarserde SELECT src_thrift.lint, src_thrift.lstring, src_thrift.mstringstring, src_thrift.aint, src_thrift.astring DISTRIBUTE BY 1; + +FROM src_thrift +INSERT OVERWRITE TABLE input_columnarserde SELECT src_thrift.lint, src_thrift.lstring, src_thrift.mstringstring, src_thrift.aint, src_thrift.astring DISTRIBUTE BY 1; + +SELECT input_columnarserde.* FROM input_columnarserde DISTRIBUTE BY 1; + +SELECT input_columnarserde.a[0], input_columnarserde.b[0], input_columnarserde.c['key2'], input_columnarserde.d, input_columnarserde.e FROM input_columnarserde DISTRIBUTE BY 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_dfs.q b/src/test/hive/ql/src/test/queries/clientpositive/input_dfs.q new file mode 100644 index 0000000000000..4f5824df5c9d5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_dfs.q @@ -0,0 +1,2 @@ +dfs -cat ../data/files/kv1.txt; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_dynamicserde.q b/src/test/hive/ql/src/test/queries/clientpositive/input_dynamicserde.q new file mode 100644 index 0000000000000..14377429fda8d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_dynamicserde.q @@ -0,0 +1,18 @@ +CREATE TABLE dest1(a array, b array, c map, d int, e string) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '1' +COLLECTION ITEMS TERMINATED BY '2' +MAP KEYS TERMINATED BY '3' +LINES TERMINATED BY '10' +STORED AS TEXTFILE; + +EXPLAIN +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint, src_thrift.lstring, src_thrift.mstringstring, src_thrift.aint, src_thrift.astring; + +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint, src_thrift.lstring, src_thrift.mstringstring, src_thrift.aint, src_thrift.astring; + +SELECT dest1.* FROM dest1; + +SELECT dest1.a[0], dest1.b[0], dest1.c['key2'], dest1.d, dest1.e FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_lazyserde.q b/src/test/hive/ql/src/test/queries/clientpositive/input_lazyserde.q new file mode 100644 index 0000000000000..65e678bee8872 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_lazyserde.q @@ -0,0 +1,30 @@ +CREATE TABLE dest1(a array, b array, c map, d int, e string) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '1' +COLLECTION ITEMS TERMINATED BY '2' +MAP KEYS TERMINATED BY '3' +LINES TERMINATED BY '10' +STORED AS TEXTFILE; + +EXPLAIN +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint, src_thrift.lstring, src_thrift.mstringstring, src_thrift.aint, src_thrift.astring DISTRIBUTE BY 1; + +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint, src_thrift.lstring, src_thrift.mstringstring, src_thrift.aint, src_thrift.astring DISTRIBUTE BY 1; + +SELECT dest1.* FROM dest1 CLUSTER BY 1; + +SELECT dest1.a[0], dest1.b[0], dest1.c['key2'], dest1.d, dest1.e FROM dest1 CLUSTER BY 1; + +DROP TABLE dest1; + +CREATE TABLE dest1(a array) ROW FORMAT DELIMITED FIELDS TERMINATED BY '1' ESCAPED BY '\\'; +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint FROM src_thrift DISTRIBUTE BY 1; +SELECT * from dest1 ORDER BY 1 ASC; +DROP TABLE dest1; + +CREATE TABLE dest1(a map) ROW FORMAT DELIMITED FIELDS TERMINATED BY '1' ESCAPED BY '\\'; +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.mstringstring FROM src_thrift DISTRIBUTE BY 1; +SELECT * from dest1 ORDER BY 1 ASC; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_limit.q b/src/test/hive/ql/src/test/queries/clientpositive/input_limit.q new file mode 100644 index 0000000000000..6d4363e590059 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_limit.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT x.* FROM SRC x LIMIT 20; + +SELECT x.* FROM SRC x LIMIT 20; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part0.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part0.q new file mode 100644 index 0000000000000..e5526e55414ef --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part0.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT x.* FROM SRCPART x WHERE x.ds = '2008-04-08'; + +SELECT x.* FROM SRCPART x WHERE x.ds = '2008-04-08'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part1.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part1.q new file mode 100644 index 0000000000000..b9a210c93daa0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part1.q @@ -0,0 +1,11 @@ +CREATE TABLE dest1(key INT, value STRING, hr STRING, ds STRING) STORED AS TEXTFILE; + +EXPLAIN EXTENDED +FROM srcpart +INSERT OVERWRITE TABLE dest1 SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-08' and srcpart.hr = '12'; + +FROM srcpart +INSERT OVERWRITE TABLE dest1 SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-08' and srcpart.hr = '12'; + +SELECT dest1.* FROM dest1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part10.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part10.q new file mode 100644 index 0000000000000..b0698b991041b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part10.q @@ -0,0 +1,23 @@ +-- EXCLUDE_OS_WINDOWS +-- excluded on windows because of difference in file name encoding logic + +CREATE TABLE part_special ( + a STRING, + b STRING +) PARTITIONED BY ( + ds STRING, + ts STRING +); + +EXPLAIN +INSERT OVERWRITE TABLE part_special PARTITION(ds='2008 04 08', ts = '10:11:12=455') +SELECT 1, 2 FROM src LIMIT 1; + +INSERT OVERWRITE TABLE part_special PARTITION(ds='2008 04 08', ts = '10:11:12=455') +SELECT 1, 2 FROM src LIMIT 1; + +DESCRIBE EXTENDED part_special PARTITION(ds='2008 04 08', ts = '10:11:12=455'); + +SELECT * FROM part_special WHERE ds='2008 04 08' AND ts = '10:11:12=455'; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part10_win.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part10_win.q new file mode 100644 index 0000000000000..0f179ec25b1c1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part10_win.q @@ -0,0 +1,23 @@ +-- INCLUDE_OS_WINDOWS +-- included only on windows because of difference in file name encoding logic + +CREATE TABLE part_special ( + a STRING, + b STRING +) PARTITIONED BY ( + ds STRING, + ts STRING +); + +EXPLAIN +INSERT OVERWRITE TABLE part_special PARTITION(ds='2008 04 08', ts = '10:11:12=455') +SELECT 1, 2 FROM src LIMIT 1; + +INSERT OVERWRITE TABLE part_special PARTITION(ds='2008 04 08', ts = '10:11:12=455') +SELECT 1, 2 FROM src LIMIT 1; + +DESCRIBE EXTENDED part_special PARTITION(ds='2008 04 08', ts = '10:11:12=455'); + +SELECT * FROM part_special WHERE ds='2008 04 08' AND ts = '10:11:12=455'; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part2.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part2.q new file mode 100644 index 0000000000000..c9aaf25363e41 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part2.q @@ -0,0 +1,16 @@ +CREATE TABLE dest1(key INT, value STRING, hr STRING, ds STRING) STORED AS TEXTFILE; +CREATE TABLE dest2(key INT, value STRING, hr STRING, ds STRING) STORED AS TEXTFILE; + +EXPLAIN EXTENDED +FROM srcpart +INSERT OVERWRITE TABLE dest1 SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-08' and srcpart.hr = '12' +INSERT OVERWRITE TABLE dest2 SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-09' and srcpart.hr = '12'; + +FROM srcpart +INSERT OVERWRITE TABLE dest1 SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-08' and srcpart.hr = '12' +INSERT OVERWRITE TABLE dest2 SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-09' and srcpart.hr = '12'; + +SELECT dest1.* FROM dest1 sort by key,value,ds,hr; +SELECT dest2.* FROM dest2 sort by key,value,ds,hr; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part3.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part3.q new file mode 100644 index 0000000000000..ea1d2b2408934 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part3.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT x.* FROM SRCPART x WHERE x.ds = '2008-04-08' and x.hr = 11; + +SELECT x.* FROM SRCPART x WHERE x.ds = '2008-04-08' and x.hr = 11; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part4.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part4.q new file mode 100644 index 0000000000000..71f27befee696 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part4.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT x.* FROM SRCPART x WHERE x.ds = '2008-04-08' and x.hr = 15; + +SELECT x.* FROM SRCPART x WHERE x.ds = '2008-04-08' and x.hr = 15; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part5.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part5.q new file mode 100644 index 0000000000000..ea9f70bdcb4fe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part5.q @@ -0,0 +1,12 @@ + +create table tmptable(key string, value string, hr string, ds string); + +EXPLAIN +insert overwrite table tmptable +SELECT x.* FROM SRCPART x WHERE x.ds = '2008-04-08' and x.key < 100; + +insert overwrite table tmptable +SELECT x.* FROM SRCPART x WHERE x.ds = '2008-04-08' and x.key < 100; + +select * from tmptable x sort by x.key,x.value,x.ds,x.hr; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part6.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part6.q new file mode 100644 index 0000000000000..a60c7ac3a0284 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part6.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT x.* FROM SRCPART x WHERE x.ds = 2008-04-08 LIMIT 10; + +SELECT x.* FROM SRCPART x WHERE x.ds = 2008-04-08 LIMIT 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part7.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part7.q new file mode 100644 index 0000000000000..10f9336adcea6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part7.q @@ -0,0 +1,14 @@ +EXPLAIN EXTENDED +SELECT * FROM ( + SELECT X.* FROM SRCPART X WHERE X.ds = '2008-04-08' and X.key < 100 + UNION ALL + SELECT Y.* FROM SRCPART Y WHERE Y.ds = '2008-04-08' and Y.key < 100 +) A +SORT BY A.key, A.value, A.ds, A.hr; + +SELECT * FROM ( + SELECT X.* FROM SRCPART X WHERE X.ds = '2008-04-08' and X.key < 100 + UNION ALL + SELECT Y.* FROM SRCPART Y WHERE Y.ds = '2008-04-08' and Y.key < 100 +) A +SORT BY A.key, A.value, A.ds, A.hr; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part8.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part8.q new file mode 100644 index 0000000000000..6f1c9ec88f174 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part8.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT x.* FROM SRCPART x WHERE ds = '2008-04-08' LIMIT 10; + +SELECT x.* FROM SRCPART x WHERE ds = '2008-04-08' LIMIT 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part9.q b/src/test/hive/ql/src/test/queries/clientpositive/input_part9.q new file mode 100644 index 0000000000000..30008f0f5a964 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_part9.q @@ -0,0 +1,5 @@ +EXPLAIN EXTENDED +SELECT x.* FROM SRCPART x WHERE key IS NOT NULL AND ds = '2008-04-08' order by x.key, x.hr; + +SELECT x.* FROM SRCPART x WHERE key IS NOT NULL AND ds = '2008-04-08' order by x.key, x.hr; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q b/src/test/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q new file mode 100755 index 0000000000000..d9926888cef9c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q @@ -0,0 +1,14 @@ +set mapred.output.compress=true; +set mapred.output.compression.type=BLOCK; + +CREATE TABLE dest4_sequencefile(key INT, value STRING) STORED AS SEQUENCEFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value; + +FROM src +INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value; + +set mapred.output.compress=false; +SELECT dest4_sequencefile.* FROM dest4_sequencefile; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath.q b/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath.q new file mode 100755 index 0000000000000..ed22404500811 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(key INT, value STRING, mapvalue STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[1], src_thrift.lintstring[0].mystring, src_thrift.mstringstring['key_2']; + +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[1], src_thrift.lintstring[0].mystring, src_thrift.mstringstring['key_2']; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath2.q b/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath2.q new file mode 100644 index 0000000000000..131105b763d0e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath2.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(lint_size INT, lintstring_size INT, mstringstring_size INT) STORED AS TEXTFILE; + +EXPLAIN +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT size(src_thrift.lint), size(src_thrift.lintstring), size(src_thrift.mstringstring) where src_thrift.lint IS NOT NULL AND NOT (src_thrift.mstringstring IS NULL); + +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT size(src_thrift.lint), size(src_thrift.lintstring), size(src_thrift.mstringstring) where src_thrift.lint IS NOT NULL AND NOT (src_thrift.mstringstring IS NULL); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath3.q b/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath3.q new file mode 100644 index 0000000000000..1b57ead8225f5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath3.q @@ -0,0 +1,6 @@ +EXPLAIN +FROM src_thrift +SELECT src_thrift.mstringstring['key_9'], src_thrift.lintstring.myint; + +FROM src_thrift +SELECT src_thrift.mstringstring['key_9'], src_thrift.lintstring.myint; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath4.q b/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath4.q new file mode 100644 index 0000000000000..e8e8d2ef52b39 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath4.q @@ -0,0 +1,29 @@ +set hive.optimize.ppd=false; + +EXPLAIN +FROM src_thrift +SELECT src_thrift.mstringstring['key_9'], lintstring.myint +WHERE src_thrift.mstringstring['key_9'] IS NOT NULL + AND lintstring.myint IS NOT NULL + AND lintstring IS NOT NULL; + +FROM src_thrift +SELECT src_thrift.mstringstring['key_9'], lintstring.myint +WHERE src_thrift.mstringstring['key_9'] IS NOT NULL + OR lintstring.myint IS NOT NULL + OR lintstring IS NOT NULL; + +set hive.optimize.ppd=true; + +EXPLAIN +FROM src_thrift +SELECT src_thrift.mstringstring['key_9'], lintstring.myint +WHERE src_thrift.mstringstring['key_9'] IS NOT NULL + AND lintstring.myint IS NOT NULL + AND lintstring IS NOT NULL; + +FROM src_thrift +SELECT src_thrift.mstringstring['key_9'], lintstring.myint +WHERE src_thrift.mstringstring['key_9'] IS NOT NULL + OR lintstring.myint IS NOT NULL + OR lintstring IS NOT NULL; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl1.q b/src/test/hive/ql/src/test/queries/clientpositive/inputddl1.q new file mode 100644 index 0000000000000..b41f0b887cd29 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/inputddl1.q @@ -0,0 +1,6 @@ +EXPLAIN +CREATE TABLE INPUTDDL1(key INT, value STRING) STORED AS TEXTFILE; + +CREATE TABLE INPUTDDL1(key INT, value STRING) STORED AS TEXTFILE; + +SELECT INPUTDDL1.* from INPUTDDL1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl2.q b/src/test/hive/ql/src/test/queries/clientpositive/inputddl2.q new file mode 100644 index 0000000000000..e34779112454e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/inputddl2.q @@ -0,0 +1,6 @@ +EXPLAIN +CREATE TABLE INPUTDDL2(key INT, value STRING) PARTITIONED BY(ds STRING, country STRING) STORED AS TEXTFILE; +CREATE TABLE INPUTDDL2(key INT, value STRING) PARTITIONED BY(ds STRING, country STRING) STORED AS TEXTFILE; +DESCRIBE INPUTDDL2; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl3.q b/src/test/hive/ql/src/test/queries/clientpositive/inputddl3.q new file mode 100644 index 0000000000000..946cf54e40445 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/inputddl3.q @@ -0,0 +1,5 @@ +EXPLAIN +CREATE TABLE INPUTDDL3(key INT, value STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; +CREATE TABLE INPUTDDL3(key INT, value STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; +DESCRIBE INPUTDDL3; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl4.q b/src/test/hive/ql/src/test/queries/clientpositive/inputddl4.q new file mode 100644 index 0000000000000..0e07d8da6b9a1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/inputddl4.q @@ -0,0 +1,12 @@ +-- a simple test to test sorted/clustered syntax + +CREATE TABLE INPUTDDL4(viewTime STRING, userid INT, + page_url STRING, referrer_url STRING, + friends ARRAY, properties MAP, + ip STRING COMMENT 'IP Address of the User') + COMMENT 'This is the page view table' + PARTITIONED BY(ds STRING, country STRING) + CLUSTERED BY(userid) SORTED BY(viewTime) INTO 32 BUCKETS; +DESCRIBE INPUTDDL4; +DESCRIBE EXTENDED INPUTDDL4; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl5.q b/src/test/hive/ql/src/test/queries/clientpositive/inputddl5.q new file mode 100644 index 0000000000000..9a7ca5da1277f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/inputddl5.q @@ -0,0 +1,8 @@ +-- 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; +DESCRIBE INPUTDDL5; +SELECT INPUTDDL5.name from INPUTDDL5; +SELECT count(1) FROM INPUTDDL5 WHERE INPUTDDL5.name = _UTF-8 0xE982B5E993AE; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl6.q b/src/test/hive/ql/src/test/queries/clientpositive/inputddl6.q new file mode 100644 index 0000000000000..d33ab8d9a78f4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/inputddl6.q @@ -0,0 +1,15 @@ +-- test for describe extended table +-- 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'); +DESCRIBE EXTENDED INPUTDDL6; +DESCRIBE EXTENDED INPUTDDL6 PARTITION (ds='2008-04-08'); +SHOW PARTITIONS INPUTDDL6; +ALTER TABLE INPUTDDL6 DROP PARTITION (ds='2008-04-08'); +SHOW PARTITIONS INPUTDDL6; +EXPLAIN +DESCRIBE EXTENDED INPUTDDL6 PARTITION (ds='2008-04-09'); +DROP TABLE INPUTDDL6; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl7.q b/src/test/hive/ql/src/test/queries/clientpositive/inputddl7.q new file mode 100644 index 0000000000000..8a73935feec51 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/inputddl7.q @@ -0,0 +1,33 @@ +-- test for loading into tables with the correct file format +-- test for loading into partitions with the correct file format + + +CREATE TABLE T1(name STRING) STORED AS TEXTFILE; +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; +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'); +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'); +SELECT COUNT(1) FROM T4 where T4.ds='2008-04-09'; + +DESCRIBE EXTENDED T1; +DESCRIBE EXTENDED T2; +DESCRIBE EXTENDED T3 PARTITION (ds='2008-04-09'); +DESCRIBE EXTENDED T4 PARTITION (ds='2008-04-09'); + + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl8.q b/src/test/hive/ql/src/test/queries/clientpositive/inputddl8.q new file mode 100644 index 0000000000000..66717a158bd8a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/inputddl8.q @@ -0,0 +1,10 @@ + +CREATE TABLE INPUTDDL8 COMMENT 'This is a thrift based table' + PARTITIONED BY(ds STRING, country STRING) + CLUSTERED BY(aint) SORTED BY(lint) INTO 32 BUCKETS + ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer' + WITH SERDEPROPERTIES ('serialization.class' = 'org.apache.hadoop.hive.serde2.thrift.test.Complex', + 'serialization.format' = 'com.facebook.thrift.protocol.TBinaryProtocol') + STORED AS SEQUENCEFILE; +DESCRIBE EXTENDED INPUTDDL8; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert1.q b/src/test/hive/ql/src/test/queries/clientpositive/insert1.q new file mode 100644 index 0000000000000..5ac234db2069c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert1.q @@ -0,0 +1,36 @@ + + +create table insert1(key int, value string) stored as textfile; +create table insert2(key int, value string) stored as textfile; +insert overwrite table insert1 select a.key, a.value from insert2 a WHERE (a.key=-1); + +explain insert into table insert1 select a.key, a.value from insert2 a WHERE (a.key=-1); +explain insert into table INSERT1 select a.key, a.value from insert2 a WHERE (a.key=-1); + +-- HIVE-3465 +create database x; +create table x.insert1(key int, value string) stored as textfile; + +explain insert into table x.INSERT1 select a.key, a.value from insert2 a WHERE (a.key=-1); + +explain insert into table default.INSERT1 select a.key, a.value from insert2 a WHERE (a.key=-1); + +explain +from insert2 +insert into table insert1 select * where key < 10 +insert overwrite table x.insert1 select * where key > 10 and key < 20; + +-- HIVE-3676 +CREATE DATABASE db2; +USE db2; +CREATE TABLE result(col1 STRING); +INSERT OVERWRITE TABLE result SELECT 'db2_insert1' FROM default.src LIMIT 1; +INSERT INTO TABLE result SELECT 'db2_insert2' FROM default.src LIMIT 1; +SELECT * FROM result order by col1; + +USE default; +CREATE DATABASE db1; +CREATE TABLE db1.result(col1 STRING); +INSERT OVERWRITE TABLE db1.result SELECT 'db1_insert1' FROM src LIMIT 1; +INSERT INTO TABLE db1.result SELECT 'db1_insert2' FROM src LIMIT 1; +SELECT * FROM db1.result order by col1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q b/src/test/hive/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q new file mode 100644 index 0000000000000..6ad70b5673f30 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q @@ -0,0 +1,35 @@ +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/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); + +EXPLAIN INSERT OVERWRITE TABLE destinTable PARTITION (ds='2011-11-11', hr='11') if not exists +SELECT one,two FROM sourceTable WHERE ds='2011-11-11' AND hr='11' order by one desc, two desc limit 5; + +INSERT OVERWRITE TABLE destinTable PARTITION (ds='2011-11-11', hr='11') if not exists +SELECT one,two FROM sourceTable WHERE ds='2011-11-11' AND hr='11' order by one desc, two desc limit 5; + +select one,two from destinTable order by one desc, two desc; + +EXPLAIN INSERT OVERWRITE TABLE destinTable PARTITION (ds='2011-11-11', hr='11') if not exists +SELECT one,two FROM sourceTable WHERE ds='2011-11-11' AND hr='12' order by one desc, two desc limit 5; + +INSERT OVERWRITE TABLE destinTable PARTITION (ds='2011-11-11', hr='11') if not exists +SELECT one,two FROM sourceTable WHERE ds='2011-11-11' AND hr='12' order by one desc, two desc limit 5; + +select one,two from destinTable order by one desc, two desc; + +drop table destinTable; + +CREATE TABLE destinTable (one string,two string); + +EXPLAIN INSERT OVERWRITE TABLE destinTable SELECT one,two FROM sourceTable WHERE ds='2011-11-11' AND hr='11' order by one desc, two desc limit 5; + +INSERT OVERWRITE TABLE destinTable SELECT one,two FROM sourceTable WHERE ds='2011-11-11' AND hr='11' order by one desc, two desc limit 5; + +drop table destinTable; + +drop table sourceTable; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q b/src/test/hive/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q new file mode 100644 index 0000000000000..598d30eaebba2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q @@ -0,0 +1,35 @@ +CREATE DATABASE db1; + +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/kv3.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11'); + +CREATE TABLE db2.destinTable (one string,two string) PARTITIONED BY (ds string); + +EXPLAIN INSERT OVERWRITE TABLE db2.destinTable PARTITION (ds='2011-11-11') +SELECT one,two FROM db1.sourceTable WHERE ds='2011-11-11' order by one desc, two desc limit 5; + +INSERT OVERWRITE TABLE db2.destinTable PARTITION (ds='2011-11-11') +SELECT one,two FROM db1.sourceTable WHERE ds='2011-11-11' order by one desc, two desc limit 5; + +select one,two from db2.destinTable order by one desc, two desc; + +EXPLAIN INSERT OVERWRITE TABLE db2.destinTable PARTITION (ds='2011-11-11') +SELECT one,two FROM db1.sourceTable WHERE ds='2011-11-11' order by one desc, two desc limit 5; + +INSERT OVERWRITE TABLE db2.destinTable PARTITION (ds='2011-11-11') +SELECT one,two FROM db1.sourceTable WHERE ds='2011-11-11' order by one desc, two desc limit 5; + +select one,two from db2.destinTable order by one desc, two desc; + +drop table db2.destinTable; + +drop table db1.sourceTable; + +DROP DATABASE db1; + +DROP DATABASE db2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_compressed.q b/src/test/hive/ql/src/test/queries/clientpositive/insert_compressed.q new file mode 100644 index 0000000000000..8f1201cd6fb91 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert_compressed.q @@ -0,0 +1,15 @@ +set hive.exec.compress.output=true; + +drop table insert_compressed; +create table insert_compressed (key int, value string); + +insert overwrite table insert_compressed select * from src; +select count(*) from insert_compressed; + +insert into table insert_compressed select * from src; +select count(*) from insert_compressed; + +insert into table insert_compressed select * from src; +select count(*) from insert_compressed; + +drop table insert_compressed; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into1.q b/src/test/hive/ql/src/test/queries/clientpositive/insert_into1.q new file mode 100644 index 0000000000000..edc65a442d098 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert_into1.q @@ -0,0 +1,26 @@ +DROP TABLE insert_into1; + +CREATE TABLE insert_into1 (key int, value string); + +EXPLAIN INSERT INTO TABLE insert_into1 SELECT * from src LIMIT 100; +INSERT INTO TABLE insert_into1 SELECT * from src LIMIT 100; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into1 +) t; + +EXPLAIN INSERT INTO TABLE insert_into1 SELECT * FROM src LIMIT 100; +INSERT INTO TABLE insert_into1 SELECT * FROM src LIMIT 100; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into1 +) t; + +SELECT COUNT(*) FROM insert_into1; + +EXPLAIN INSERT OVERWRITE TABLE insert_into1 SELECT * FROM src LIMIT 10; +INSERT OVERWRITE TABLE insert_into1 SELECT * FROM src LIMIT 10; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into1 +) t; + + +DROP TABLE insert_into1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into2.q b/src/test/hive/ql/src/test/queries/clientpositive/insert_into2.q new file mode 100644 index 0000000000000..0cce9585a89e7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert_into2.q @@ -0,0 +1,31 @@ +DROP TABLE insert_into2; +CREATE TABLE insert_into2 (key int, value string) + PARTITIONED BY (ds string); + +EXPLAIN INSERT INTO TABLE insert_into2 PARTITION (ds='1') + SELECT * FROM src LIMIT 100; +INSERT INTO TABLE insert_into2 PARTITION (ds='1') SELECT * FROM src limit 100; +INSERT INTO TABLE insert_into2 PARTITION (ds='1') SELECT * FROM src limit 100; +SELECT COUNT(*) FROM insert_into2 WHERE ds='1'; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into2 +) t; + +EXPLAIN INSERT OVERWRITE TABLE insert_into2 PARTITION (ds='2') + SELECT * FROM src LIMIT 100; +INSERT OVERWRITE TABLE insert_into2 PARTITION (ds='2') + SELECT * FROM src LIMIT 100; + +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into2 +) t; + +EXPLAIN INSERT OVERWRITE TABLE insert_into2 PARTITION (ds='2') + SELECT * FROM src LIMIT 50; +INSERT OVERWRITE TABLE insert_into2 PARTITION (ds='2') + SELECT * FROM src LIMIT 50; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into2 +) t; + +DROP TABLE insert_into2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into3.q b/src/test/hive/ql/src/test/queries/clientpositive/insert_into3.q new file mode 100644 index 0000000000000..e58b212e190bb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert_into3.q @@ -0,0 +1,30 @@ +DROP TABLE insert_into3a; +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; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into3a +) t; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into3b +) t; + +EXPLAIN FROM src INSERT OVERWRITE TABLE insert_into3a SELECT * LIMIT 10 + INSERT INTO TABLE insert_into3b SELECT * LIMIT 10; +FROM src INSERT OVERWRITE TABLE insert_into3a SELECT * LIMIT 10 + INSERT INTO TABLE insert_into3b SELECT * LIMIT 10; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into3a +) t; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into3b +) t; + +DROP TABLE insert_into3a; +DROP TABLE insert_into3b; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into4.q b/src/test/hive/ql/src/test/queries/clientpositive/insert_into4.q new file mode 100644 index 0000000000000..2afe41d668bda --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert_into4.q @@ -0,0 +1,30 @@ +set hive.merge.smallfiles.avgsize=16000000; + +DROP TABLE insert_into4a; +DROP TABLE insert_into4b; + +CREATE TABLE insert_into4a (key int, value string); +CREATE TABLE insert_into4b (key int, value string); + +EXPLAIN INSERT INTO TABLE insert_into4a SELECT * FROM src LIMIT 10; +INSERT INTO TABLE insert_into4a SELECT * FROM src LIMIT 10; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into4a +) t; + +EXPLAIN INSERT INTO TABLE insert_into4a SELECT * FROM src LIMIT 10; +INSERT INTO TABLE insert_into4a SELECT * FROM src LIMIT 10; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into4a +) t; + +--At this point insert_into4a has 2 files (if INSERT INTO merges isn't fixed) + +EXPLAIN INSERT INTO TABLE insert_into4b SELECT * FROM insert_into4a; +INSERT INTO TABLE insert_into4b SELECT * FROM insert_into4a; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into4b +) t; + +DROP TABLE insert_into4a; +DROP TABLE insert_into4b; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into5.q b/src/test/hive/ql/src/test/queries/clientpositive/insert_into5.q new file mode 100644 index 0000000000000..07e80ea4e30fe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert_into5.q @@ -0,0 +1,34 @@ +DROP TABLE insert_into5a; +DROP TABLE insert_into5b; + +CREATE TABLE insert_into5a (key int, value string); +CREATE TABLE insert_into5b (key int, value string) PARTITIONED BY (ds string); + +EXPLAIN INSERT INTO TABLE insert_into5a SELECT 1, 'one' FROM src LIMIT 10; +INSERT INTO TABLE insert_into5a SELECT 1, 'one' FROM src LIMIT 10; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into5a +) t; + +EXPLAIN INSERT INTO TABLE insert_into5a SELECT * FROM insert_into5a; +INSERT INTO TABLE insert_into5a SELECT * FROM insert_into5a; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into5a +) t; + +EXPLAIN INSERT INTO TABLE insert_into5b PARTITION (ds='1') + SELECT * FROM insert_into5a; +INSERT INTO TABLE insert_into5b PARTITION (ds='1') SELECT * FROM insert_into5a; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into5b +) t; + +EXPLAIN INSERT INTO TABLE insert_into5b PARTITION (ds='1') + SELECT key, value FROM insert_into5b; +INSERT INTO TABLE insert_into5b PARTITION (ds='1') + SELECT key, value FROM insert_into5b; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into5b +) t; + +DROP TABLE insert_into5a; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into6.q b/src/test/hive/ql/src/test/queries/clientpositive/insert_into6.q new file mode 100644 index 0000000000000..2b7988ec6d27a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert_into6.q @@ -0,0 +1,28 @@ +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +DROP TABLE insert_into6a; +DROP TABLE insert_into6b; +CREATE TABLE insert_into6a (key int, value string) PARTITIONED BY (ds string); +CREATE TABLE insert_into6b (key int, value string) PARTITIONED BY (ds string); + +EXPLAIN INSERT INTO TABLE insert_into6a PARTITION (ds='1') + SELECT * FROM src LIMIT 150; +INSERT INTO TABLE insert_into6a PARTITION (ds='1') SELECT * FROM src LIMIT 150; +INSERT INTO TABLE insert_into6a PARTITION (ds='2') SELECT * FROM src LIMIT 100; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into6a +) t; + +EXPLAIN INSERT INTO TABLE insert_into6b PARTITION (ds) + SELECT * FROM insert_into6a; +INSERT INTO TABLE insert_into6b PARTITION (ds) SELECT * FROM insert_into6a; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into6b +) t; + +SHOW PARTITIONS insert_into6b; + +DROP TABLE insert_into6a; +DROP TABLE insert_into6b; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q b/src/test/hive/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q new file mode 100644 index 0000000000000..25c127f67f980 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q @@ -0,0 +1,109 @@ +insert overwrite local directory '../data/files/local_src_table_1' +select * from src ; +dfs -cat ../data/files/local_src_table_1/000000_0; + +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; + +create table array_table (a array, b array) +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; + +insert overwrite local directory '../data/files/local_array_table_1' +select * from array_table; +dfs -cat ../data/files/local_array_table_1/000000_0; + +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; + +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; + + +create table map_table (foo STRING , bar MAP) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '\t' +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; + +insert overwrite local directory '../data/files/local_map_table_1' +select * from map_table; +dfs -cat ../data/files/local_map_table_1/000000_0; + +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; + +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; + +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; + +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; + +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; + +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' +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '\t' +select key,value from local_rctable; + +dfs -cat ../data/files/local_rctable_out/000000_0; + +drop table local_rctable; +drop table array_table; +drop table map_table; +dfs -rmr ${system:test.tmp.dir}/local_rctable; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insertexternal1.q b/src/test/hive/ql/src/test/queries/clientpositive/insertexternal1.q new file mode 100644 index 0000000000000..9004a0cce87bb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/insertexternal1.q @@ -0,0 +1,14 @@ + + +create table texternal(key string, val string) partitioned by (insertdate string); + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/texternal/temp; +dfs -rmr ${system:test.tmp.dir}/texternal; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/texternal/2008-01-01; + +alter table texternal add partition (insertdate='2008-01-01') location 'pfile://${system:test.tmp.dir}/texternal/2008-01-01'; +from src insert overwrite table texternal partition (insertdate='2008-01-01') select *; + +select * from texternal where insertdate='2008-01-01'; + +dfs -rmr ${system:test.tmp.dir}/texternal; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join0.q b/src/test/hive/ql/src/test/queries/clientpositive/join0.q new file mode 100644 index 0000000000000..feac6d85714cb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join0.q @@ -0,0 +1,23 @@ +EXPLAIN +SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2; + +EXPLAIN FORMATTED +SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2; + +SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join1.q b/src/test/hive/ql/src/test/queries/clientpositive/join1.q new file mode 100644 index 0000000000000..886cba7d30d33 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join1.q @@ -0,0 +1,11 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value; + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value; + +SELECT dest_j1.* FROM dest_j1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join10.q b/src/test/hive/ql/src/test/queries/clientpositive/join10.q new file mode 100644 index 0000000000000..f424618034760 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join10.q @@ -0,0 +1,13 @@ +EXPLAIN FROM +(SELECT src.* FROM src) x +JOIN +(SELECT src.* FROM src) Y +ON (x.key = Y.key) +SELECT Y.*; + +FROM +(SELECT src.* FROM src) x +JOIN +(SELECT src.* FROM src) Y +ON (x.key = Y.key) +SELECT Y.*; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join11.q b/src/test/hive/ql/src/test/queries/clientpositive/join11.q new file mode 100644 index 0000000000000..a40f0cc99ed2c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join11.q @@ -0,0 +1,14 @@ +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100; + +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join12.q b/src/test/hive/ql/src/test/queries/clientpositive/join12.q new file mode 100644 index 0000000000000..7053a06f8e2dc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join12.q @@ -0,0 +1,20 @@ +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100 +JOIN +(SELECT src.key as c5, src.value as c6 from src) src3 +ON src1.c1 = src3.c5 AND src3.c5 < 80; + +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100 +JOIN +(SELECT src.key as c5, src.value as c6 from src) src3 +ON src1.c1 = src3.c5 AND src3.c5 < 80; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join13.q b/src/test/hive/ql/src/test/queries/clientpositive/join13.q new file mode 100644 index 0000000000000..914fb44860cb4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join13.q @@ -0,0 +1,20 @@ +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100 +JOIN +(SELECT src.key as c5, src.value as c6 from src) src3 +ON src1.c1 + src2.c3 = src3.c5 AND src3.c5 < 200; + +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100 +JOIN +(SELECT src.key as c5, src.value as c6 from src) src3 +ON src1.c1 + src2.c3 = src3.c5 AND src3.c5 < 200; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join14.q b/src/test/hive/ql/src/test/queries/clientpositive/join14.q new file mode 100644 index 0000000000000..83346b4c3466b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join14.q @@ -0,0 +1,17 @@ +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +CREATE TABLE dest1(c1 INT, c2 STRING) STORED AS TEXTFILE; + +set mapreduce.framework.name=yarn; +set mapreduce.jobtracker.address=localhost:58; +set hive.exec.mode.local.auto=true; +set hive.exec.mode.local.auto.input.files.max=6; + +EXPLAIN +FROM src JOIN srcpart ON src.key = srcpart.key AND srcpart.ds = '2008-04-08' and src.key > 100 +INSERT OVERWRITE TABLE dest1 SELECT src.key, srcpart.value; + +FROM src JOIN srcpart ON src.key = srcpart.key AND srcpart.ds = '2008-04-08' and src.key > 100 +INSERT OVERWRITE TABLE dest1 SELECT src.key, srcpart.value; + +select dest1.* from dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join14_hadoop20.q b/src/test/hive/ql/src/test/queries/clientpositive/join14_hadoop20.q new file mode 100644 index 0000000000000..a12ef1afb055f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join14_hadoop20.q @@ -0,0 +1,15 @@ +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +CREATE TABLE dest1(c1 INT, c2 STRING) STORED AS TEXTFILE; + +set mapred.job.tracker=localhost:58; +set hive.exec.mode.local.auto=true; + +EXPLAIN +FROM src JOIN srcpart ON src.key = srcpart.key AND srcpart.ds = '2008-04-08' and src.key > 100 +INSERT OVERWRITE TABLE dest1 SELECT src.key, srcpart.value; + +FROM src JOIN srcpart ON src.key = srcpart.key AND srcpart.ds = '2008-04-08' and src.key > 100 +INSERT OVERWRITE TABLE dest1 SELECT src.key, srcpart.value; + +select dest1.* from dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join15.q b/src/test/hive/ql/src/test/queries/clientpositive/join15.q new file mode 100644 index 0000000000000..b86716c9be261 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join15.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key) SORT BY src1.key, src1.value, src2.key, src2.value; + +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key) SORT BY src1.key, src1.value, src2.key, src2.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join16.q b/src/test/hive/ql/src/test/queries/clientpositive/join16.q new file mode 100644 index 0000000000000..4c9c9619f3e9a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join16.q @@ -0,0 +1 @@ +EXPLAIN SELECT subq.key, tab.value FROM (select a.key, a.value from src a where a.key > 10 ) subq JOIN src tab ON (subq.key = tab.key and subq.key > 20 and subq.value = tab.value) where tab.value < 200; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join17.q b/src/test/hive/ql/src/test/queries/clientpositive/join17.q new file mode 100644 index 0000000000000..5fbb620e218f3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join17.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(key1 INT, value1 STRING, key2 INT, value2 STRING) STORED AS TEXTFILE; + +EXPLAIN EXTENDED +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.*; + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.*; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join18.q b/src/test/hive/ql/src/test/queries/clientpositive/join18.q new file mode 100644 index 0000000000000..0ec121874cb5c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join18.q @@ -0,0 +1,24 @@ +EXPLAIN + SELECT a.key, a.value, b.key, b.value + FROM + ( + SELECT src1.key as key, count(src1.value) AS value FROM src src1 group by src1.key + ) a + FULL OUTER JOIN + ( + SELECT src2.key as key, count(distinct(src2.value)) AS value + FROM src1 src2 group by src2.key + ) b + ON (a.key = b.key); + + SELECT a.key, a.value, b.key, b.value + FROM + ( + SELECT src1.key as key, count(src1.value) AS value FROM src src1 group by src1.key + ) a + FULL OUTER JOIN + ( + SELECT src2.key as key, count(distinct(src2.value)) AS value + FROM src1 src2 group by src2.key + ) b + ON (a.key = b.key); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join18_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/join18_multi_distinct.q new file mode 100644 index 0000000000000..eb4e46af421a3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join18_multi_distinct.q @@ -0,0 +1,26 @@ +EXPLAIN + SELECT a.key, a.value, b.key, b.value1, b.value2 + FROM + ( + SELECT src1.key as key, count(src1.value) AS value FROM src src1 group by src1.key + ) a + FULL OUTER JOIN + ( + SELECT src2.key as key, count(distinct(src2.value)) AS value1, + count(distinct(src2.key)) AS value2 + FROM src1 src2 group by src2.key + ) b + ON (a.key = b.key) ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value1 ASC, b.value2 ASC; + + SELECT a.key, a.value, b.key, b.value1, b.value2 + FROM + ( + SELECT src1.key as key, count(src1.value) AS value FROM src src1 group by src1.key + ) a + FULL OUTER JOIN + ( + SELECT src2.key as key, count(distinct(src2.value)) AS value1, + count(distinct(src2.key)) AS value2 + FROM src1 src2 group by src2.key + ) b + ON (a.key = b.key) ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value1 ASC, b.value2 ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join19.q b/src/test/hive/ql/src/test/queries/clientpositive/join19.q new file mode 100644 index 0000000000000..5a6b741f198d7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join19.q @@ -0,0 +1,58 @@ +CREATE TABLE triples (foo string, subject string, predicate string, object string, foo2 string); + +EXPLAIN +SELECT t11.subject, t22.object , t33.subject , t55.object, t66.object +FROM +( +SELECT t1.subject +FROM triples t1 +WHERE +t1.predicate='http://sofa.semanticweb.org/sofa/v1.0/system#__INSTANCEOF_REL' +AND +t1.object='http://ontos/OntosMiner/Common.English/ontology#Citation' +) t11 +JOIN +( +SELECT t2.subject , t2.object +FROM triples t2 +WHERE +t2.predicate='http://sofa.semanticweb.org/sofa/v1.0/system#__LABEL_REL' +) t22 +ON (t11.subject=t22.subject) +JOIN +( +SELECT t3.subject , t3.object +FROM triples t3 +WHERE +t3.predicate='http://www.ontosearch.com/2007/12/ontosofa-ns#_from' + +) t33 +ON (t11.subject=t33.object) +JOIN +( +SELECT t4.subject +FROM triples t4 +WHERE +t4.predicate='http://sofa.semanticweb.org/sofa/v1.0/system#__INSTANCEOF_REL' +AND +t4.object='http://ontos/OntosMiner/Common.English/ontology#Author' + +) t44 +ON (t44.subject=t33.subject) +JOIN +( +SELECT t5.subject, t5.object +FROM triples t5 +WHERE +t5.predicate='http://www.ontosearch.com/2007/12/ontosofa-ns#_to' +) t55 +ON (t55.subject=t44.subject) +JOIN +( +SELECT t6.subject, t6.object +FROM triples t6 +WHERE +t6.predicate='http://sofa.semanticweb.org/sofa/v1.0/system#__LABEL_REL' +) t66 +ON (t66.subject=t55.object); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join2.q b/src/test/hive/ql/src/test/queries/clientpositive/join2.q new file mode 100644 index 0000000000000..7bb547bcb9473 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join2.q @@ -0,0 +1,10 @@ +CREATE TABLE dest_j2(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) +INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value; + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) +INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value; + +SELECT dest_j2.* FROM dest_j2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join20.q b/src/test/hive/ql/src/test/queries/clientpositive/join20.q new file mode 100644 index 0000000000000..b534a14031207 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join20.q @@ -0,0 +1,14 @@ +EXPLAIN +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + + +EXPLAIN +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key < 15) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key < 15) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join21.q b/src/test/hive/ql/src/test/queries/clientpositive/join21.q new file mode 100644 index 0000000000000..50eb73ab85f2f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join21.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) RIGHT OUTER JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join22.q b/src/test/hive/ql/src/test/queries/clientpositive/join22.q new file mode 100644 index 0000000000000..4a67d74e83abc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join22.q @@ -0,0 +1,2 @@ +explain +SELECT src5.src1_value FROM (SELECT src3.*, src4.value as src4_value, src4.key as src4_key FROM src src4 JOIN (SELECT src2.*, src1.key as src1_key, src1.value as src1_value FROM src src1 JOIN src src2 ON src1.key = src2.key) src3 ON src3.src1_key = src4.key) src5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join23.q b/src/test/hive/ql/src/test/queries/clientpositive/join23.q new file mode 100644 index 0000000000000..2db69d394171d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join23.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value; + +SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join24.q b/src/test/hive/ql/src/test/queries/clientpositive/join24.q new file mode 100644 index 0000000000000..65c9256d6f264 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join24.q @@ -0,0 +1,8 @@ +create table tst1(key STRING, cnt INT); + +INSERT OVERWRITE TABLE tst1 +SELECT a.key, count(1) FROM src a group by a.key; + +SELECT sum(a.cnt) FROM tst1 a JOIN tst1 b ON a.key = b.key; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join25.q b/src/test/hive/ql/src/test/queries/clientpositive/join25.q new file mode 100644 index 0000000000000..c0e4cbb93a386 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join25.q @@ -0,0 +1,19 @@ +set hive.mapjoin.numrows = 2; + + + +CREATE TABLE dest_j1(key INT, value STRING, val2 STRING) STORED AS TEXTFILE; + +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key); + +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key); + +select * from dest_j1 x order by x.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join26.q b/src/test/hive/ql/src/test/queries/clientpositive/join26.q new file mode 100644 index 0000000000000..cecf8da533d8f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join26.q @@ -0,0 +1,17 @@ +CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE; + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x,y) */ x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.key = z.key and z.ds='2008-04-08' and z.hr=11); + +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x,y) */ x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.key = z.key and z.ds='2008-04-08' and z.hr=11); + +select * from dest_j1 x order by x.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join27.q b/src/test/hive/ql/src/test/queries/clientpositive/join27.q new file mode 100644 index 0000000000000..2c6e95765752f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join27.q @@ -0,0 +1,17 @@ + + +CREATE TABLE dest_j1(key INT, value STRING, val2 STRING) STORED AS TEXTFILE; + +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value +FROM src1 x JOIN src y ON (x.value = y.value); + +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value +FROM src1 x JOIN src y ON (x.value = y.value); + +select * from dest_j1 x order by x.key, x.value; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join28.q b/src/test/hive/ql/src/test/queries/clientpositive/join28.q new file mode 100644 index 0000000000000..1d652b3a51843 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join28.q @@ -0,0 +1,27 @@ +CREATE TABLE dest_j1(key STRING, value STRING) STORED AS TEXTFILE; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT subq.key1, z.value +FROM +(SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 + FROM src1 x JOIN src y ON (x.key = y.key)) subq + JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11); + +INSERT OVERWRITE TABLE dest_j1 +SELECT subq.key1, z.value +FROM +(SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 + FROM src1 x JOIN src y ON (x.key = y.key)) subq + JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11); + +select * from dest_j1 x order by x.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join29.q b/src/test/hive/ql/src/test/queries/clientpositive/join29.q new file mode 100644 index 0000000000000..7fa4815486c8c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join29.q @@ -0,0 +1,20 @@ +CREATE TABLE dest_j1(key STRING, cnt1 INT, cnt2 INT); + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT subq1.key, subq1.cnt, subq2.cnt +FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN + (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key); + +INSERT OVERWRITE TABLE dest_j1 +SELECT subq1.key, subq1.cnt, subq2.cnt +FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN + (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key); + +select * from dest_j1 x order by x.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join3.q b/src/test/hive/ql/src/test/queries/clientpositive/join3.q new file mode 100644 index 0000000000000..f38fe215e8767 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join3.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value; + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join30.q b/src/test/hive/ql/src/test/queries/clientpositive/join30.q new file mode 100644 index 0000000000000..036a16d3b1eec --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join30.q @@ -0,0 +1,10 @@ +CREATE TABLE dest_j1(key INT, cnt INT); + +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x) */ x.key, count(1) FROM src1 x JOIN src y ON (x.key = y.key) group by x.key; + +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x) */ x.key, count(1) FROM src1 x JOIN src y ON (x.key = y.key) group by x.key; + +select * from dest_j1 x order by x.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join31.q b/src/test/hive/ql/src/test/queries/clientpositive/join31.q new file mode 100644 index 0000000000000..0ba143aa76b41 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join31.q @@ -0,0 +1,22 @@ +CREATE TABLE dest_j1(key STRING, cnt INT); + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT subq1.key, count(1) as cnt +FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN + (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key) +group by subq1.key; + +INSERT OVERWRITE TABLE dest_j1 +SELECT subq1.key, count(1) as cnt +FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN + (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key) +group by subq1.key; + +select * from dest_j1 x order by x.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join32.q b/src/test/hive/ql/src/test/queries/clientpositive/join32.q new file mode 100644 index 0000000000000..82fde0e965878 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join32.q @@ -0,0 +1,23 @@ +CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11); + +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11); + +select * from dest_j1 x order by x.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join32_lessSize.q b/src/test/hive/ql/src/test/queries/clientpositive/join32_lessSize.q new file mode 100644 index 0000000000000..58a26c5bb3fa5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join32_lessSize.q @@ -0,0 +1,88 @@ +CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE; +CREATE TABLE dest_j2(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=6000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11); + +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11); + +select * from dest_j1 x order by x.value; + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, z.value, y.value +FROM src w JOIN src1 x ON (x.value = w.value) +JOIN src y ON (x.key = y.key) +JOIN src1 z ON (x.key = z.key); + +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, z.value, y.value +FROM src w JOIN src1 x ON (x.value = w.value) +JOIN src y ON (x.key = y.key) +JOIN src1 z ON (x.key = z.key); + +select * from dest_j1 x order by x.value; + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j2 +SELECT res.key, z.value, res.value +FROM (select x.key, x.value from src1 x JOIN src y ON (x.key = y.key)) res +JOIN srcpart z ON (res.value = z.value and z.ds='2008-04-08' and z.hr=11); + +INSERT OVERWRITE TABLE dest_j2 +SELECT res.key, z.value, res.value +FROM (select x.key, x.value from src1 x JOIN src y ON (x.key = y.key)) res +JOIN srcpart z ON (res.value = z.value and z.ds='2008-04-08' and z.hr=11); + +select * from dest_j2 x order by x.value; + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j2 +SELECT res.key, z.value, res.value +FROM (select x.key, x.value from src1 x LEFT OUTER JOIN src y ON (x.key = y.key)) res +JOIN srcpart z ON (res.value = z.value and z.ds='2008-04-08' and z.hr=11); + +INSERT OVERWRITE TABLE dest_j2 +SELECT res.key, z.value, res.value +FROM (select x.key, x.value from src1 x LEFT OUTER JOIN src y ON (x.key = y.key)) res +JOIN srcpart z ON (res.value = z.value and z.ds='2008-04-08' and z.hr=11); + +select * from dest_j2 x order by x.value; + +EXPLAIN +INSERT OVERWRITE TABLE dest_j2 +SELECT res.key, x.value, res.value +FROM (select x.key, x.value from src1 x JOIN src y ON (x.key = y.key)) res +JOIN srcpart x ON (res.value = x.value and x.ds='2008-04-08' and x.hr=11); + +INSERT OVERWRITE TABLE dest_j2 +SELECT res.key, x.value, res.value +FROM (select x.key, x.value from src1 x JOIN src y ON (x.key = y.key)) res +JOIN srcpart x ON (res.value = x.value and x.ds='2008-04-08' and x.hr=11); + +select * from dest_j2 x order by x.value; + +EXPLAIN +INSERT OVERWRITE TABLE dest_j2 +SELECT res.key, y.value, res.value +FROM (select x.key, x.value from src1 x JOIN src y ON (x.key = y.key)) res +JOIN srcpart y ON (res.value = y.value and y.ds='2008-04-08' and y.hr=11); + +INSERT OVERWRITE TABLE dest_j2 +SELECT res.key, y.value, res.value +FROM (select x.key, x.value from src1 x JOIN src y ON (x.key = y.key)) res +JOIN srcpart y ON (res.value = y.value and y.ds='2008-04-08' and y.hr=11); + +select * from dest_j2 x order by x.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join33.q b/src/test/hive/ql/src/test/queries/clientpositive/join33.q new file mode 100644 index 0000000000000..82fde0e965878 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join33.q @@ -0,0 +1,23 @@ +CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11); + +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11); + +select * from dest_j1 x order by x.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join34.q b/src/test/hive/ql/src/test/queries/clientpositive/join34.q new file mode 100644 index 0000000000000..85adcfd232955 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join34.q @@ -0,0 +1,31 @@ +CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, x.value, subq1.value +FROM +( SELECT x.key as key, x.value as value from src x where x.key < 20 + UNION ALL + SELECT x1.key as key, x1.value as value from src x1 where x1.key > 100 +) subq1 +JOIN src1 x ON (x.key = subq1.key); + +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, x.value, subq1.value +FROM +( SELECT x.key as key, x.value as value from src x where x.key < 20 + UNION ALL + SELECT x1.key as key, x1.value as value from src x1 where x1.key > 100 +) subq1 +JOIN src1 x ON (x.key = subq1.key); + +select * from dest_j1 x order by x.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join35.q b/src/test/hive/ql/src/test/queries/clientpositive/join35.q new file mode 100644 index 0000000000000..d66616c3cd5ad --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join35.q @@ -0,0 +1,31 @@ +CREATE TABLE dest_j1(key STRING, value STRING, val2 INT) STORED AS TEXTFILE; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key); + +INSERT OVERWRITE TABLE dest_j1 +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key); + +select * from dest_j1 x order by x.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join36.q b/src/test/hive/ql/src/test/queries/clientpositive/join36.q new file mode 100644 index 0000000000000..9912610da9667 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join36.q @@ -0,0 +1,29 @@ +set hive.mapjoin.numrows = 2; + + + + + +CREATE TABLE tmp1(key INT, cnt INT); +CREATE TABLE tmp2(key INT, cnt INT); +CREATE TABLE dest_j1(key INT, value INT, val2 INT); + +INSERT OVERWRITE TABLE tmp1 +SELECT key, count(1) from src group by key; + +INSERT OVERWRITE TABLE tmp2 +SELECT key, count(1) from src group by key; + +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x) */ x.key, x.cnt, y.cnt +FROM tmp1 x JOIN tmp2 y ON (x.key = y.key); + +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x) */ x.key, x.cnt, y.cnt +FROM tmp1 x JOIN tmp2 y ON (x.key = y.key); + +select * from dest_j1 x order by x.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join37.q b/src/test/hive/ql/src/test/queries/clientpositive/join37.q new file mode 100644 index 0000000000000..a971f4ae07ebb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join37.q @@ -0,0 +1,19 @@ +set hive.mapjoin.numrows = 2; + + + +CREATE TABLE dest_j1(key INT, value STRING, val2 STRING) STORED AS TEXTFILE; + +EXPLAIN +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(X) */ x.key, x.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key); + +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(X) */ x.key, x.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key); + +select * from dest_j1 x order by x.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join38.q b/src/test/hive/ql/src/test/queries/clientpositive/join38.q new file mode 100644 index 0000000000000..7fbe377585917 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join38.q @@ -0,0 +1,20 @@ + + +create table tmp(col0 string, col1 string,col2 string,col3 string,col4 string,col5 string,col6 string,col7 string,col8 string,col9 string,col10 string,col11 string); + +insert overwrite table tmp select key, cast(key + 1 as int), key +2, key+3, key+4, cast(key+5 as int), key+6, key+7, key+8, key+9, key+10, cast(key+11 as int) from src where key = 100; + +select * from tmp; + +explain +FROM src a JOIN tmp b ON (a.key = b.col11) +SELECT /*+ MAPJOIN(a) */ a.value, b.col5, count(1) as count +where b.col11 = 111 +group by a.value, b.col5; + +FROM src a JOIN tmp b ON (a.key = b.col11) +SELECT /*+ MAPJOIN(a) */ a.value, b.col5, count(1) as count +where b.col11 = 111 +group by a.value, b.col5; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join39.q b/src/test/hive/ql/src/test/queries/clientpositive/join39.q new file mode 100644 index 0000000000000..d0edac50ef544 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join39.q @@ -0,0 +1,16 @@ +CREATE TABLE dest_j1(key STRING, value STRING, key1 string, val2 STRING) STORED AS TEXTFILE; + +explain +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(y) */ x.key, x.value, y.key, y.value +FROM src x left outer JOIN (select * from src where key <= 100) y ON (x.key = y.key); + + +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(y) */ x.key, x.value, y.key, y.value +FROM src x left outer JOIN (select * from src where key <= 100) y ON (x.key = y.key); + +select * from dest_j1 x order by x.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join4.q b/src/test/hive/ql/src/test/queries/clientpositive/join4.q new file mode 100644 index 0000000000000..100076e6cf1b5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join4.q @@ -0,0 +1,32 @@ +CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + LEFT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + LEFT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join40.q b/src/test/hive/ql/src/test/queries/clientpositive/join40.q new file mode 100644 index 0000000000000..e93f08289ac27 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join40.q @@ -0,0 +1,43 @@ +set hive.join.cache.size=1; + +EXPLAIN SELECT x.key, x.value, y.key, y.value +FROM src x left outer JOIN (select * from src where key <= 100) y ON (x.key = y.key); + +SELECT x.key, x.value, y.key, y.value +FROM src x left outer JOIN (select * from src where key <= 100) y ON (x.key = y.key); + + +EXPLAIN select src1.key, src2.value +FROM src src1 JOIN src src2 ON (src1.key = src2.key); + +select src1.key, src2.value +FROM src src1 JOIN src src2 ON (src1.key = src2.key); + + +EXPLAIN +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + + +EXPLAIN +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key < 15) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src src1 JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key < 15) RIGHT OUTER JOIN src src3 ON (src1.key = src3.key AND src3.key < 20) +SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + + +EXPLAIN +SELECT /*+ MAPJOIN(y) */ x.key, x.value, y.key, y.value +FROM src x left outer JOIN (select * from src where key <= 100) y ON (x.key = y.key); + +SELECT /*+ MAPJOIN(y) */ x.key, x.value, y.key, y.value +FROM src x left outer JOIN (select * from src where key <= 100) y ON (x.key = y.key); + +EXPLAIN +SELECT COUNT(1) FROM SRC A JOIN SRC B ON (A.KEY=B.KEY); + +SELECT COUNT(1) FROM SRC A JOIN SRC B ON (A.KEY=B.KEY); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join41.q b/src/test/hive/ql/src/test/queries/clientpositive/join41.q new file mode 100644 index 0000000000000..b8d9397fd6569 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join41.q @@ -0,0 +1,16 @@ +create table s1 as select * from src where key = 0; + +set hive.auto.convert.join.noconditionaltask=false; +EXPLAIN +SELECT * FROM s1 src1 LEFT OUTER JOIN s1 src2 ON (src1.key = src2.key AND src2.key > 10); +SELECT * FROM s1 src1 LEFT OUTER JOIN s1 src2 ON (src1.key = src2.key AND src2.key > 10); + +set hive.auto.convert.join.noconditionaltask=true; + +-- Make sure the big table is chosen correctly as part of HIVE-4146 +EXPLAIN +SELECT * FROM s1 src1 LEFT OUTER JOIN s1 src2 ON (src1.key = src2.key AND src2.key > 10); +SELECT * FROM s1 src1 LEFT OUTER JOIN s1 src2 ON (src1.key = src2.key AND src2.key > 10); + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join5.q b/src/test/hive/ql/src/test/queries/clientpositive/join5.q new file mode 100644 index 0000000000000..405a0a57448a5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join5.q @@ -0,0 +1,32 @@ +CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + RIGHT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + RIGHT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join6.q b/src/test/hive/ql/src/test/queries/clientpositive/join6.q new file mode 100644 index 0000000000000..f55bc08580eb4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join6.q @@ -0,0 +1,33 @@ +CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + FULL OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + FULL OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4; + + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join7.q b/src/test/hive/ql/src/test/queries/clientpositive/join7.q new file mode 100644 index 0000000000000..34f502fa23cc8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join7.q @@ -0,0 +1,42 @@ +CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING, c5 INT, c6 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + FULL OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + LEFT OUTER JOIN + ( + FROM src src3 SELECT src3.key AS c5, src3.value AS c6 WHERE src3.key > 20 and src3.key < 25 + ) c + ON (a.c1 = c.c5) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4, c.c5 AS c5, c.c6 AS c6 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4, c.c5, c.c6; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + FULL OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + LEFT OUTER JOIN + ( + FROM src src3 SELECT src3.key AS c5, src3.value AS c6 WHERE src3.key > 20 and src3.key < 25 + ) c + ON (a.c1 = c.c5) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4, c.c5 AS c5, c.c6 AS c6 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4, c.c5, c.c6; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join8.q b/src/test/hive/ql/src/test/queries/clientpositive/join8.q new file mode 100644 index 0000000000000..a7fdf4cd14ad9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join8.q @@ -0,0 +1,32 @@ +CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + LEFT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4 where c.c3 IS NULL AND c.c1 IS NOT NULL; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + LEFT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4 where c.c3 IS NULL AND c.c1 IS NOT NULL; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join9.q b/src/test/hive/ql/src/test/queries/clientpositive/join9.q new file mode 100644 index 0000000000000..80add736c2a9e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join9.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN EXTENDED +FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value where src1.ds = '2008-04-08' and src1.hr = '12'; + +FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value where src1.ds = '2008-04-08' and src1.hr = '12'; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_1to1.q b/src/test/hive/ql/src/test/queries/clientpositive/join_1to1.q new file mode 100644 index 0000000000000..b403814adb4c9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_1to1.q @@ -0,0 +1,58 @@ + +CREATE TABLE join_1to1_1(key1 int, key2 int, value int); +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; + + +set hive.outerjoin.supports.filters=false; + +set hive.join.emit.interval=5; + +SELECT * FROM join_1to1_1 a join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; + +set hive.join.emit.interval=2; +SELECT * FROM join_1to1_1 a join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; + +set hive.join.emit.interval=1; +SELECT * FROM join_1to1_1 a join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; + + + +set hive.outerjoin.supports.filters=true; + +set hive.join.emit.interval=5; + +SELECT * FROM join_1to1_1 a join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; + +set hive.join.emit.interval=2; +SELECT * FROM join_1to1_1 a join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; + +set hive.join.emit.interval=1; +SELECT * FROM join_1to1_1 a join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; +SELECT * FROM join_1to1_1 a full outer join join_1to1_2 b on a.key1 = b.key1 and a.key2 = b.key2 and a.value = 66 and b.value = 66 ORDER BY a.key1 ASC, a.key2 ASC, a.value ASC, b.key1 ASC, b.key2 ASC, b.value ASC; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_array.q b/src/test/hive/ql/src/test/queries/clientpositive/join_array.q new file mode 100644 index 0000000000000..e4d95a51fa927 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_array.q @@ -0,0 +1,10 @@ +create table tinyA(a bigint, b bigint) stored as textfile; +create table tinyB(a bigint, bList array) 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; + +select * from tinyA; +select * from tinyB; + +select tinyB.a, tinyB.bList from tinyB full outer join tinyA on tinyB.a = tinyA.a; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_casesensitive.q b/src/test/hive/ql/src/test/queries/clientpositive/join_casesensitive.q new file mode 100644 index 0000000000000..a9b69c5b1c448 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_casesensitive.q @@ -0,0 +1,8 @@ + +CREATE TABLE joinone(key1 int, key2 int, value int); +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; + +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/src/test/hive/ql/src/test/queries/clientpositive/join_empty.q b/src/test/hive/ql/src/test/queries/clientpositive/join_empty.q new file mode 100644 index 0000000000000..1982487c84b9d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_empty.q @@ -0,0 +1,10 @@ +create table srcpart_empty(key int, value string) partitioned by (ds string); +create table src2_empty (key int, value string); + +select /*+mapjoin(a)*/ a.key, b.value from srcpart_empty a join src b on a.key=b.key; +select /*+mapjoin(a)*/ a.key, b.value from src2_empty a join src b on a.key=b.key; + +set hive.mapred.mode=nonstrict; +set hive.auto.convert.join = true; +select a.key, b.value from srcpart_empty a join src b on a.key=b.key; +select a.key, b.value from src2_empty a join src b on a.key=b.key; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_filters.q b/src/test/hive/ql/src/test/queries/clientpositive/join_filters.q new file mode 100644 index 0000000000000..d54aa950a1ec2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_filters.q @@ -0,0 +1,155 @@ +CREATE TABLE myinput1(key int, value int); +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; +SELECT * FROM myinput1 a RIGHT 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; +SELECT * FROM myinput1 a FULL 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; + +SELECT * FROM myinput1 a JOIN myinput1 b ON a.key = b.value AND 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 JOIN myinput1 b ON a.key = b.key AND 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 JOIN myinput1 b ON a.value = b.value AND 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 JOIN myinput1 b ON a.value = b.value and a.key=b.key AND 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 = b.value AND 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.value = b.value AND 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 = b.key AND 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 = b.key and a.value=b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key = b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key = b.key AND 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 RIGHT OUTER JOIN myinput1 b ON a.value = b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key=b.key and a.value = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.key = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.key = b.key AND 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 FULL OUTER JOIN myinput1 b ON a.value = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.value = b.value and a.key=b.key AND 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.value=b.value AND 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) RIGHT OUTER JOIN myinput1 c ON (b.value=c.value AND c.key > 40 AND c.value > 50 AND c.key = c.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, c.key ASC, c.value ASC; +SELECT * from myinput1 a RIGHT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) LEFT OUTER JOIN myinput1 c ON (b.value=c.value AND c.key > 40 AND c.value > 50 AND c.key = c.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, c.key ASC, c.value ASC; +SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b RIGHT OUTER JOIN myinput1 c ON a.value = b.value and b.value = c.value AND 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 AND c.key > 40 AND c.value > 50 AND c.key = c.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC, c.key ASC, c.value ASC; +SELECT * from myinput1 a LEFT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) RIGHT OUTER JOIN myinput1 c ON (b.key=c.key AND c.key > 40 AND c.value > 50 AND c.key = c.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, c.key ASC, c.value ASC; +SELECT * from myinput1 a RIGHT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) LEFT OUTER JOIN myinput1 c ON (b.key=c.key AND c.key > 40 AND c.value > 50 AND c.key = c.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, c.key ASC, c.value ASC; +SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b RIGHT OUTER JOIN myinput1 c ON a.value = b.value and b.key = c.key AND 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 AND c.key > 40 AND c.value > 50 AND c.key = c.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC, c.key ASC, c.value ASC; + +SELECT /*+ MAPJOIN(a) */ * 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 /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.value AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.key AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value and a.key = b.key AND 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 /*+ MAPJOIN(b) */ * 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 /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.value AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.key AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value and a.key = b.key AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key = b.value AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key = b.key AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.value = b.value AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key = b.value AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key = b.key AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.value = b.value AND 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; + +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; + +SET hive.optimize.bucketmapjoin = true; +SET hive.optimize.bucketmapjoin.sortedmerge = true; +SET hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key = b.key AND 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 /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input2 b ON a.key = b.value AND 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 /*+ MAPJOIN(a) */ * FROM smb_input2 a JOIN smb_input2 b ON a.value = b.value AND 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 /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key = b.key AND a.value = b.value AND 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 /*+ MAPJOIN(b) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key = b.key AND 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 /*+ MAPJOIN(b) */ * FROM smb_input1 a JOIN smb_input2 b ON a.key = b.value AND 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 /*+ MAPJOIN(b) */ * FROM smb_input2 a JOIN smb_input2 b ON a.value = b.value AND 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 /*+ MAPJOIN(b) */ * FROM smb_input2 a JOIN smb_input2 b ON a.key = b.key AND a.value = b.value AND 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 /*+ MAPJOIN(b) */ * FROM smb_input1 a LEFT OUTER JOIN smb_input1 b ON a.key = b.key AND 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 /*+ MAPJOIN(b) */ * FROM smb_input1 a LEFT OUTER JOIN smb_input2 b ON a.key = b.value AND 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 /*+ MAPJOIN(b) */ * FROM smb_input2 a LEFT OUTER JOIN smb_input2 b ON a.value = b.value AND 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 /*+ MAPJOIN(a) */ * FROM smb_input1 a RIGHT OUTER JOIN smb_input1 b ON a.key = b.key AND 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 /*+ MAPJOIN(a) */ * FROM smb_input1 a RIGHT OUTER JOIN smb_input2 b ON a.key = b.value AND 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 /*+ MAPJOIN(a) */ * FROM smb_input2 a RIGHT OUTER JOIN smb_input2 b ON a.value = b.value AND 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; + +SET hive.outerjoin.supports.filters = false; + +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; +SELECT * FROM myinput1 a RIGHT 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; +SELECT * FROM myinput1 a FULL 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; + +SELECT * FROM myinput1 a JOIN myinput1 b ON a.key = b.value AND 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 JOIN myinput1 b ON a.key = b.key AND 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 JOIN myinput1 b ON a.value = b.value AND 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 JOIN myinput1 b ON a.value = b.value and a.key=b.key AND 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 = b.value AND 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.value = b.value AND 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 = b.key AND 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 = b.key and a.value=b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key = b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key = b.key AND 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 RIGHT OUTER JOIN myinput1 b ON a.value = b.value AND 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 RIGHT OUTER JOIN myinput1 b ON a.key=b.key and a.value = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.key = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.key = b.key AND 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 FULL OUTER JOIN myinput1 b ON a.value = b.value AND 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 FULL OUTER JOIN myinput1 b ON a.value = b.value and a.key=b.key AND 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.value=b.value AND 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) RIGHT OUTER JOIN myinput1 c ON (b.value=c.value AND c.key > 40 AND c.value > 50 AND c.key = c.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, c.key ASC, c.value ASC; +SELECT * from myinput1 a RIGHT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) LEFT OUTER JOIN myinput1 c ON (b.value=c.value AND c.key > 40 AND c.value > 50 AND c.key = c.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, c.key ASC, c.value ASC; +SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b RIGHT OUTER JOIN myinput1 c ON a.value = b.value and b.value = c.value AND 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 AND c.key > 40 AND c.value > 50 AND c.key = c.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC, c.key ASC, c.value ASC; +SELECT * from myinput1 a LEFT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) RIGHT OUTER JOIN myinput1 c ON (b.key=c.key AND c.key > 40 AND c.value > 50 AND c.key = c.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, c.key ASC, c.value ASC; +SELECT * from myinput1 a RIGHT OUTER JOIN myinput1 b ON (a.value=b.value AND 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) LEFT OUTER JOIN myinput1 c ON (b.key=c.key AND c.key > 40 AND c.value > 50 AND c.key = c.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, c.key ASC, c.value ASC; +SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b RIGHT OUTER JOIN myinput1 c ON a.value = b.value and b.key = c.key AND 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 AND c.key > 40 AND c.value > 50 AND c.key = c.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC, c.key ASC, c.value ASC; + +SELECT /*+ MAPJOIN(a) */ * 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 /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.value AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.key AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value and a.key = b.key AND 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 /*+ MAPJOIN(b) */ * 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 /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.value AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.key AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value and a.key = b.key AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key = b.value AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key = b.key AND 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 /*+ MAPJOIN(b) */ * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.value = b.value AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key = b.value AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key = b.key AND 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 /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.value = b.value AND 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 /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key = b.key AND 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 /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input2 b ON a.key = b.value AND 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 /*+ MAPJOIN(a) */ * FROM smb_input2 a JOIN smb_input2 b ON a.value = b.value AND 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 /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key = b.key AND a.value = b.value AND 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 /*+ MAPJOIN(b) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key = b.key AND 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 /*+ MAPJOIN(b) */ * FROM smb_input1 a JOIN smb_input2 b ON a.key = b.value AND 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 /*+ MAPJOIN(b) */ * FROM smb_input2 a JOIN smb_input2 b ON a.value = b.value AND 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 /*+ MAPJOIN(b) */ * FROM smb_input2 a JOIN smb_input2 b ON a.key = b.key AND a.value = b.value AND 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 /*+ MAPJOIN(b) */ * FROM smb_input1 a LEFT OUTER JOIN smb_input1 b ON a.key = b.key AND 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 /*+ MAPJOIN(b) */ * FROM smb_input1 a LEFT OUTER JOIN smb_input2 b ON a.key = b.value AND 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 /*+ MAPJOIN(b) */ * FROM smb_input2 a LEFT OUTER JOIN smb_input2 b ON a.value = b.value AND 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 /*+ MAPJOIN(a) */ * FROM smb_input1 a RIGHT OUTER JOIN smb_input1 b ON a.key = b.key AND 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 /*+ MAPJOIN(a) */ * FROM smb_input1 a RIGHT OUTER JOIN smb_input2 b ON a.key = b.value AND 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 /*+ MAPJOIN(a) */ * FROM smb_input2 a RIGHT OUTER JOIN smb_input2 b ON a.value = b.value AND 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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_filters_overlap.q b/src/test/hive/ql/src/test/queries/clientpositive/join_filters_overlap.q new file mode 100644 index 0000000000000..4394396790d9f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_filters_overlap.q @@ -0,0 +1,27 @@ +-- HIVE-3411 Filter predicates on outer join overlapped on single alias is not handled properly + +create table a as SELECT 100 as key, a.value as value FROM src LATERAL VIEW explode(array(40, 50, 60)) a as value limit 3; + +-- overlap on a +explain extended select * from a left outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (a.key=c.key AND a.value=60 AND c.value=60); +select * from a left outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (a.key=c.key AND a.value=60 AND c.value=60); +select /*+ MAPJOIN(b,c)*/ * from a left outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (a.key=c.key AND a.value=60 AND c.value=60) order by a.key ASC, a.value ASC; + +-- overlap on b +explain extended select * from a right outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (b.key=c.key AND b.value=60 AND c.value=60); +select * from a right outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (b.key=c.key AND b.value=60 AND c.value=60); +select /*+ MAPJOIN(a,c)*/ * from a right outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (b.key=c.key AND b.value=60 AND c.value=60) order by b.key ASC, b.value ASC; + +-- overlap on b with two filters for each +explain extended select * from a right outer join a b on (a.key=b.key AND a.value=50 AND b.value=50 AND b.value>10) left outer join a c on (b.key=c.key AND b.value=60 AND b.value>20 AND c.value=60); +select * from a right outer join a b on (a.key=b.key AND a.value=50 AND b.value=50 AND b.value>10) left outer join a c on (b.key=c.key AND b.value=60 AND b.value>20 AND c.value=60); +select /*+ MAPJOIN(a,c)*/ * from a right outer join a b on (a.key=b.key AND a.value=50 AND b.value=50 AND b.value>10) left outer join a c on (b.key=c.key AND b.value=60 AND b.value>20 AND c.value=60) order by b.key ASC, b.value ASC; + +-- overlap on a, b +explain extended select * from a full outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (b.key=c.key AND b.value=60 AND c.value=60) left outer join a d on (a.key=d.key AND a.value=40 AND d.value=40); +select * from a full outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (b.key=c.key AND b.value=60 AND c.value=60) left outer join a d on (a.key=d.key AND a.value=40 AND d.value=40); + +-- triple overlap on a +explain extended select * from a left outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (a.key=c.key AND a.value=60 AND c.value=60) left outer join a d on (a.key=d.key AND a.value=40 AND d.value=40); +select * from a left outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (a.key=c.key AND a.value=60 AND c.value=60) left outer join a d on (a.key=d.key AND a.value=40 AND d.value=40); +select /*+ MAPJOIN(b,c, d)*/ * from a left outer join a b on (a.key=b.key AND a.value=50 AND b.value=50) left outer join a c on (a.key=c.key AND a.value=60 AND c.value=60) left outer join a d on (a.key=d.key AND a.value=40 AND d.value=40) order by a.key ASC, a.value ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_hive_626.q b/src/test/hive/ql/src/test/queries/clientpositive/join_hive_626.q new file mode 100644 index 0000000000000..31b0c8c91c10b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_hive_626.q @@ -0,0 +1,30 @@ + + + + +create table hive_foo (foo_id int, foo_name string, foo_a string, foo_b string, +foo_c string, foo_d string) row format delimited fields terminated by ',' +stored as textfile; + +create table hive_bar (bar_id int, bar_0 int, foo_id int, bar_1 int, bar_name +string, bar_a string, bar_b string, bar_c string, bar_d string) row format +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; + +explain +select hive_foo.foo_name, hive_bar.bar_name, n from hive_foo join hive_bar on hive_foo.foo_id = +hive_bar.foo_id join hive_count on hive_count.bar_id = hive_bar.bar_id; + +select hive_foo.foo_name, hive_bar.bar_name, n from hive_foo join hive_bar on hive_foo.foo_id = +hive_bar.foo_id join hive_count on hive_count.bar_id = hive_bar.bar_id; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_literals.q b/src/test/hive/ql/src/test/queries/clientpositive/join_literals.q new file mode 100644 index 0000000000000..250bf839a4cdd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_literals.q @@ -0,0 +1,9 @@ +-- Test Joins with a variety of literals in the on clause + +SELECT COUNT(*) FROM src a JOIN src b ON a.key = b.key AND a.key = 0L; + +SELECT COUNT(*) FROM src a JOIN src b ON a.key = b.key AND a.key = 0S; + +SELECT COUNT(*) FROM src a JOIN src b ON a.key = b.key AND a.key = 0Y; + +SELECT COUNT(*) FROM src a JOIN src b ON a.key = b.key AND a.key = 0BD; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_map_ppr.q b/src/test/hive/ql/src/test/queries/clientpositive/join_map_ppr.q new file mode 100644 index 0000000000000..d8943c4204adb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_map_ppr.q @@ -0,0 +1,41 @@ +CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE; + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x,y) */ x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.key = z.key) +WHERE z.ds='2008-04-08' and z.hr=11; + +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x,y) */ x.key, z.value, y.value +FROM src1 x JOIN src y ON (x.key = y.key) +JOIN srcpart z ON (x.key = z.key) +WHERE z.ds='2008-04-08' and z.hr=11; + +select * from dest_j1 x order by x.key; + +CREATE TABLE src_copy(key int, value string); +CREATE TABLE src1_copy(key string, value string); +INSERT OVERWRITE TABLE src_copy select key, value from src; +INSERT OVERWRITE TABLE src1_copy select key, value from src1; + +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x,y) */ x.key, z.value, y.value +FROM src1_copy x JOIN src_copy y ON (x.key = y.key) +JOIN srcpart z ON (x.key = z.key) +WHERE z.ds='2008-04-08' and z.hr=11; + +INSERT OVERWRITE TABLE dest_j1 +SELECT /*+ MAPJOIN(x,y) */ x.key, z.value, y.value +FROM src1_copy x JOIN src_copy y ON (x.key = y.key) +JOIN srcpart z ON (x.key = z.key) +WHERE z.ds='2008-04-08' and z.hr=11; + +select * from dest_j1 x order by x.key; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_nulls.q b/src/test/hive/ql/src/test/queries/clientpositive/join_nulls.q new file mode 100644 index 0000000000000..4ff60713d6b44 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_nulls.q @@ -0,0 +1,68 @@ +CREATE TABLE myinput1(key int, value int); +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; +SELECT * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a JOIN myinput1 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a JOIN myinput1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a JOIN myinput1 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a JOIN myinput1 b ON a.value = b.value and a.key=b.key 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 = 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.value = 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 = b.key 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 = b.key and a.value=b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key=b.key and a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.value = b.value and a.key=b.key 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.value=b.value) RIGHT OUTER JOIN myinput1 c ON (b.value=c.value) ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * from myinput1 a RIGHT OUTER JOIN myinput1 b ON (a.value=b.value) LEFT OUTER JOIN myinput1 c ON (b.value=c.value) ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b RIGHT OUTER JOIN myinput1 c ON a.value = b.value and b.value = c.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; + +SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.value = b.value and a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; + +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; + +SET hive.optimize.bucketmapJOIN = true; +SET hive.optimize.bucketmapJOIN.sortedmerge = true; +SET hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key = b.key AND a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a RIGHT OUTER JOIN smb_input1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input1 a LEFT OUTER JOIN smb_input1 b ON a.key = b.key ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; + +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input2 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input1 a JOIN smb_input2 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input1 a LEFT OUTER JOIN smb_input2 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a RIGHT OUTER JOIN smb_input2 b ON a.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; + +SELECT /*+ MAPJOIN(a) */ * FROM smb_input2 a JOIN smb_input2 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(a) */ * FROM smb_input2 a RIGHT OUTER JOIN smb_input2 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input2 a JOIN smb_input2 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input2 a LEFT OUTER JOIN smb_input2 b ON a.value = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_nullsafe.q b/src/test/hive/ql/src/test/queries/clientpositive/join_nullsafe.q new file mode 100644 index 0000000000000..05b57bce202d2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_nullsafe.q @@ -0,0 +1,61 @@ +set hive.nullsafe.equijoin=true; + +CREATE TABLE myinput1(key int, value int); +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; +select * from myinput1 a join myinput1 b on a.key<=>b.value ORDER BY a.key, a.value, b.key, b.value; + +explain select * from myinput1 a join myinput1 b on a.key<=>b.value join myinput1 c on a.key=c.key ORDER BY a.key, a.value, b.key, b.value, c.key, c.value; +select * from myinput1 a join myinput1 b on a.key<=>b.value join myinput1 c on a.key=c.key ORDER BY a.key, a.value, b.key, b.value, c.key, c.value; + +explain select * from myinput1 a join myinput1 b on a.key<=>b.value join myinput1 c on a.key<=>c.key ORDER BY a.key, a.value, b.key, b.value, c.key, c.value; +select * from myinput1 a join myinput1 b on a.key<=>b.value join myinput1 c on a.key<=>c.key ORDER BY a.key, a.value, b.key, b.value, c.key, c.value; + +explain select * from myinput1 a join myinput1 b on a.key<=>b.value AND a.value=b.key join myinput1 c on a.key<=>c.key AND a.value=c.value ORDER BY a.key, a.value, b.key, b.value, c.key, c.value; +select * from myinput1 a join myinput1 b on a.key<=>b.value AND a.value=b.key join myinput1 c on a.key<=>c.key AND a.value=c.value ORDER BY a.key, a.value, b.key, b.value, c.key, c.value; + +explain select * from myinput1 a join myinput1 b on a.key<=>b.value AND a.value<=>b.key join myinput1 c on a.key<=>c.key AND a.value<=>c.value ORDER BY a.key, a.value, b.key, b.value, c.key, c.value; +select * from myinput1 a join myinput1 b on a.key<=>b.value AND a.value<=>b.key join myinput1 c on a.key<=>c.key AND a.value<=>c.value ORDER BY a.key, a.value, b.key, b.value, c.key, c.value; + +-- outer joins +SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b ON a.key<=>b.value ORDER BY a.key, a.value, b.key, b.value; +SELECT * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.key<=>b.value ORDER BY a.key, a.value, b.key, b.value; +SELECT * FROM myinput1 a FULL OUTER JOIN myinput1 b ON a.key<=>b.value ORDER BY a.key, a.value, b.key, b.value; + +-- map joins +SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ON a.key<=>b.value ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key<=>b.value ORDER BY a.key, a.value, b.key, b.value; + +-- 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; + +SET hive.optimize.bucketmapJOIN = true; +SET hive.optimize.bucketmapJOIN.sortedmerge = true; +SET hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key <=> b.key ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key <=> b.key AND a.value <=> b.value ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a RIGHT OUTER JOIN smb_input1 b ON a.key <=> b.key ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input1 a JOIN smb_input1 b ON a.key <=> b.key ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input1 a LEFT OUTER JOIN smb_input1 b ON a.key <=> b.key ORDER BY a.key, a.value, b.key, b.value; + +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a JOIN smb_input2 b ON a.key <=> b.value ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input1 a JOIN smb_input2 b ON a.key <=> b.value ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input1 a LEFT OUTER JOIN smb_input2 b ON a.key <=> b.value ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(a) */ * FROM smb_input1 a RIGHT OUTER JOIN smb_input2 b ON a.key <=> b.value ORDER BY a.key, a.value, b.key, b.value; + +SELECT /*+ MAPJOIN(a) */ * FROM smb_input2 a JOIN smb_input2 b ON a.value <=> b.value ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(a) */ * FROM smb_input2 a RIGHT OUTER JOIN smb_input2 b ON a.value <=> b.value ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input2 a JOIN smb_input2 b ON a.value <=> b.value ORDER BY a.key, a.value, b.key, b.value; +SELECT /*+ MAPJOIN(b) */ * FROM smb_input2 a LEFT OUTER JOIN smb_input2 b ON a.value <=> b.value ORDER BY a.key, a.value, b.key, b.value; + +--HIVE-3315 join predicate transitive +explain select * from myinput1 a join myinput1 b on a.key<=>b.value AND a.key is NULL; +select * from myinput1 a join myinput1 b on a.key<=>b.value AND a.key is NULL order by a.value ASC, b.key ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_rc.q b/src/test/hive/ql/src/test/queries/clientpositive/join_rc.q new file mode 100644 index 0000000000000..8a7c2d3eebeb8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_rc.q @@ -0,0 +1,16 @@ + + +create table join_rc1(key string, value string) stored as RCFile; +create table join_rc2(key string, value string) stored as RCFile; +insert overwrite table join_rc1 select * from src; +insert overwrite table join_rc2 select * from src; + +explain +select join_rc1.key, join_rc2.value +FROM join_rc1 JOIN join_rc2 ON join_rc1.key = join_rc2.key; + +select join_rc1.key, join_rc2.value +FROM join_rc1 JOIN join_rc2 ON join_rc1.key = join_rc2.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_reorder.q b/src/test/hive/ql/src/test/queries/clientpositive/join_reorder.q new file mode 100644 index 0000000000000..b92a79ba07ab2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_reorder.q @@ -0,0 +1,71 @@ + + + + +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; + +EXPLAIN FROM T1 a JOIN src c ON c.key+1=a.key +SELECT a.key, a.val, c.key; + +EXPLAIN FROM T1 a JOIN src c ON c.key+1=a.key +SELECT /*+ STREAMTABLE(a) */ a.key, a.val, c.key; + +FROM T1 a JOIN src c ON c.key+1=a.key +SELECT a.key, a.val, c.key; + +FROM T1 a JOIN src c ON c.key+1=a.key +SELECT /*+ STREAMTABLE(a) */ a.key, a.val, c.key; + +EXPLAIN FROM T1 a + LEFT OUTER JOIN T2 b ON (b.key=a.key) + RIGHT OUTER JOIN T3 c ON (c.val = a.val) +SELECT a.key, b.key, a.val, c.val; + +EXPLAIN FROM T1 a + LEFT OUTER JOIN T2 b ON (b.key=a.key) + RIGHT OUTER JOIN T3 c ON (c.val = a.val) +SELECT /*+ STREAMTABLE(a) */ a.key, b.key, a.val, c.val; + +FROM T1 a + LEFT OUTER JOIN T2 b ON (b.key=a.key) + RIGHT OUTER JOIN T3 c ON (c.val = a.val) +SELECT a.key, b.key, a.val, c.val; + +FROM T1 a + LEFT OUTER JOIN T2 b ON (b.key=a.key) + RIGHT OUTER JOIN T3 c ON (c.val = a.val) +SELECT /*+ STREAMTABLE(a) */ a.key, b.key, a.val, c.val; + +EXPLAIN FROM UNIQUEJOIN + PRESERVE T1 a (a.key, a.val), + PRESERVE T2 b (b.key, b.val), + PRESERVE T3 c (c.key, c.val) +SELECT a.key, b.key, c.key; + +EXPLAIN FROM UNIQUEJOIN + PRESERVE T1 a (a.key, a.val), + PRESERVE T2 b (b.key, b.val), + PRESERVE T3 c (c.key, c.val) +SELECT /*+ STREAMTABLE(b) */ a.key, b.key, c.key; + +FROM UNIQUEJOIN + PRESERVE T1 a (a.key, a.val), + PRESERVE T2 b (b.key, b.val), + PRESERVE T3 c (c.key, c.val) +SELECT a.key, b.key, c.key; + +FROM UNIQUEJOIN + PRESERVE T1 a (a.key, a.val), + PRESERVE T2 b (b.key, b.val), + PRESERVE T3 c (c.key, c.val) +SELECT /*+ STREAMTABLE(b) */ a.key, b.key, c.key; + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_reorder2.q b/src/test/hive/ql/src/test/queries/clientpositive/join_reorder2.q new file mode 100644 index 0000000000000..238c0adad3128 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_reorder2.q @@ -0,0 +1,44 @@ + + + + + +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; +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; + +EXPLAIN +SELECT /*+ STREAMTABLE(a) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key; + +SELECT /*+ STREAMTABLE(a) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key; + + +EXPLAIN +SELECT /*+ STREAMTABLE(a) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON a.val = c.val + JOIN T4 d ON a.key + 1 = d.key + 1; + + +SELECT /*+ STREAMTABLE(a) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON a.val = c.val + JOIN T4 d ON a.key + 1 = d.key + 1; + + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_reorder3.q b/src/test/hive/ql/src/test/queries/clientpositive/join_reorder3.q new file mode 100644 index 0000000000000..1bda28fbc3d5f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_reorder3.q @@ -0,0 +1,44 @@ + + + + + +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; +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; + +EXPLAIN +SELECT /*+ STREAMTABLE(a,c) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key; + +SELECT /*+ STREAMTABLE(a,c) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key; + + +EXPLAIN +SELECT /*+ STREAMTABLE(a,c) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON a.val = c.val + JOIN T4 d ON a.key + 1 = d.key + 1; + + +SELECT /*+ STREAMTABLE(a,c) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON a.val = c.val + JOIN T4 d ON a.key + 1 = d.key + 1; + + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_reorder4.q b/src/test/hive/ql/src/test/queries/clientpositive/join_reorder4.q new file mode 100644 index 0000000000000..126f356ef785a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_reorder4.q @@ -0,0 +1,18 @@ +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; + +set hive.auto.convert.join=true; + +explain select /*+ STREAMTABLE(a) */ a.*, b.*, c.* from T1 a join T2 b on a.key1=b.key2 join T3 c on a.key1=c.key3; +select /*+ STREAMTABLE(a) */ a.*, b.*, c.* from T1 a join T2 b on a.key1=b.key2 join T3 c on a.key1=c.key3; + +explain select /*+ STREAMTABLE(b) */ a.*, b.*, c.* from T1 a join T2 b on a.key1=b.key2 join T3 c on a.key1=c.key3; +select /*+ STREAMTABLE(b) */ a.*, b.*, c.* from T1 a join T2 b on a.key1=b.key2 join T3 c on a.key1=c.key3; + +explain select /*+ STREAMTABLE(c) */ a.*, b.*, c.* from T1 a join T2 b on a.key1=b.key2 join T3 c on a.key1=c.key3; +select /*+ STREAMTABLE(c) */ a.*, b.*, c.* from T1 a join T2 b on a.key1=b.key2 join T3 c on a.key1=c.key3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_star.q b/src/test/hive/ql/src/test/queries/clientpositive/join_star.q new file mode 100644 index 0000000000000..8314161975761 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_star.q @@ -0,0 +1,54 @@ +create table fact(m1 int, m2 int, d1 int, d2 int); +create table dim1(f1 int, f2 int); +create table dim2(f3 int, f4 int); +create table dim3(f5 int, f6 int); +create table dim4(f7 int, f8 int); +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; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=5000; + +explain select m1, m2, f2 from fact join dim1 on fact.d1=dim1.f1; +select m1, m2, f2 from fact join dim1 on fact.d1=dim1.f1; + +explain select m1, m2, f2, f4 from fact join dim1 on fact.d1=dim1.f1 join dim2 on fact.d2=dim2.f3; +select m1, m2, f2, f4 from fact join dim1 on fact.d1=dim1.f1 join dim2 on fact.d2=dim2.f3; + +explain select m1, m2, f2, f4 from fact join dim1 on fact.d1= dim1.f1 join dim2 on dim1.f2 = dim2.f3; +select m1, m2, f2, f4 from fact join dim1 on fact.d1= dim1.f1 join dim2 on dim1.f2 = dim2.f3; + +explain select m1, m2, f2, f4 from fact Left outer join dim1 on fact.d1= dim1.f1 Left outer join dim2 on dim1.f2 = dim2.f3; +select m1, m2, f2, f4 from fact Left outer join dim1 on fact.d1= dim1.f1 Left outer join dim2 on dim1.f2 = dim2.f3; + +explain Select m1, m2, f2, f4, f6, f8, f10, f12, f14 + from fact + Left outer join dim1 on fact.d1= dim1.f1 + Left outer join dim2 on dim1.f2 = dim2.f3 + Left outer Join dim3 on fact.d2= dim3.f5 + Left outer Join dim4 on dim3.f6= dim4.f7 + Left outer join dim5 on dim4.f8= dim5.f9 + Left outer Join dim6 on dim3.f6= dim6.f11 + Left outer Join dim7 on dim6.f12 = dim7.f13; + +Select m1, m2, f2, f4, f6, f8, f10, f12, f14 + from fact + Left outer join dim1 on fact.d1= dim1.f1 + Left outer join dim2 on dim1.f2 = dim2.f3 + Left outer Join dim3 on fact.d2= dim3.f5 + Left outer Join dim4 on dim3.f6= dim4.f7 + Left outer join dim5 on dim4.f8= dim5.f9 + Left outer Join dim6 on dim3.f6= dim6.f11 + Left outer Join dim7 on dim6.f12 = dim7.f13; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_thrift.q b/src/test/hive/ql/src/test/queries/clientpositive/join_thrift.q new file mode 100644 index 0000000000000..1b4d491207b8d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_thrift.q @@ -0,0 +1,12 @@ +DESCRIBE src_thrift; + +EXPLAIN +SELECT s1.aint, s2.lintstring +FROM src_thrift s1 +JOIN src_thrift s2 +ON s1.aint = s2.aint; + +SELECT s1.aint, s2.lintstring +FROM src_thrift s1 +JOIN src_thrift s2 +ON s1.aint = s2.aint; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_vc.q b/src/test/hive/ql/src/test/queries/clientpositive/join_vc.q new file mode 100644 index 0000000000000..63b3da7190f5c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_vc.q @@ -0,0 +1,5 @@ +-- see HIVE-4033 earlier a flag named hasVC was not initialized correctly in MapOperator.java, resulting in NPE for following query. order by and limit in the query is not relevant, problem would be evident even without those. They are there to keep .q.out file small and sorted. + +explain select t3.BLOCK__OFFSET__INSIDE__FILE,t3.key,t3.value from src t1 join src t2 on t1.key = t2.key join src t3 on t2.value = t3.value order by t3.BLOCK__OFFSET__INSIDE__FILE,t3.key,t3.value limit 3; + +select t3.BLOCK__OFFSET__INSIDE__FILE,t3.key,t3.value from src t1 join src t2 on t1.key = t2.key join src t3 on t2.value = t3.value order by t3.BLOCK__OFFSET__INSIDE__FILE,t3.key,t3.value limit 3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_view.q b/src/test/hive/ql/src/test/queries/clientpositive/join_view.q new file mode 100644 index 0000000000000..16b6816f0c861 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/join_view.q @@ -0,0 +1,16 @@ +drop table invites; +drop table invites2; +create table invites (foo int, bar string) partitioned by (ds string); +create table invites2 (foo int, bar string) partitioned by (ds string); + +set hive.mapred.mode=strict; + +-- test join views: see HIVE-1989 + +create view v as select invites.bar, invites2.foo, invites2.ds from invites join invites2 on invites.ds=invites2.ds; + +explain select * from v where ds='2011-09-01'; + +drop view v; +drop table invites; +drop table invites2; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/keyword_1.q b/src/test/hive/ql/src/test/queries/clientpositive/keyword_1.q new file mode 100644 index 0000000000000..2e996af28e39a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/keyword_1.q @@ -0,0 +1,19 @@ +-- SORT_BEFORE_DIFF + +create table test_user (user string, `group` string); +grant select on table test_user to user hive_test; + +explain select user from test_user; + +show grant user hive_test on table test_user; + +drop table test_user; + +create table test_user (role string, `group` string); +grant select on table test_user to user hive_test; + +explain select role from test_user; + +show grant user hive_test on table test_user; + +drop table test_user; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lateral_view.q b/src/test/hive/ql/src/test/queries/clientpositive/lateral_view.q new file mode 100644 index 0000000000000..f0c094fae3775 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/lateral_view.q @@ -0,0 +1,55 @@ + + + +CREATE TABLE tmp_pyang_lv (inputs string) STORED AS RCFILE; +INSERT OVERWRITE TABLE tmp_pyang_lv SELECT key FROM src; + +EXPLAIN SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol SORT BY key ASC, myCol ASC LIMIT 1; +EXPLAIN SELECT myTable.* FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LIMIT 3; +EXPLAIN SELECT myTable.myCol, myTable2.myCol2 FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LATERAL VIEW explode(array('a', 'b', 'c')) myTable2 AS myCol2 LIMIT 9; +EXPLAIN SELECT myTable2.* FROM src LATERAL VIEW explode(array(array(1,2,3))) myTable AS myCol LATERAL VIEW explode(myTable.myCol) myTable2 AS myCol2 LIMIT 3; + +-- Verify that * selects columns from both tables +SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol SORT BY key ASC, myCol ASC LIMIT 1; +-- TABLE.* should be supported +SELECT myTable.* FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LIMIT 3; +-- Multiple lateral views should result in a Cartesian product +SELECT myTable.myCol, myTable2.myCol2 FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LATERAL VIEW explode(array('a', 'b', 'c')) myTable2 AS myCol2 LIMIT 9; +-- Should be able to reference tables generated earlier +SELECT myTable2.* FROM src LATERAL VIEW explode(array(array(1,2,3))) myTable AS myCol LATERAL VIEW explode(myTable.myCol) myTable2 AS myCol2 LIMIT 3; + +EXPLAIN +SELECT myCol from tmp_pyang_lv LATERAL VIEW explode(array(1,2,3)) myTab as myCol limit 3; + +SELECT myCol from tmp_PYANG_lv LATERAL VIEW explode(array(1,2,3)) myTab as myCol limit 3; + +CREATE TABLE tmp_pyang_src_rcfile (key string, value array) STORED AS RCFILE; +INSERT OVERWRITE TABLE tmp_pyang_src_rcfile SELECT key, array(value) FROM src ORDER BY key LIMIT 20; + +SELECT key,value from tmp_pyang_src_rcfile LATERAL VIEW explode(value) myTable AS myCol; +SELECT myCol from tmp_pyang_src_rcfile LATERAL VIEW explode(value) myTable AS myCol; +SELECT * from tmp_pyang_src_rcfile LATERAL VIEW explode(value) myTable AS myCol; + +SELECT subq.key,subq.value +FROM ( +SELECT * from tmp_pyang_src_rcfile LATERAL VIEW explode(value) myTable AS myCol +)subq; + +SELECT subq.myCol +FROM ( +SELECT * from tmp_pyang_src_rcfile LATERAL VIEW explode(value) myTable AS myCol +)subq; + +SELECT subq.key +FROM ( +SELECT key, value from tmp_pyang_src_rcfile LATERAL VIEW explode(value) myTable AS myCol +)subq; + +EXPLAIN SELECT value, myCol from (SELECT key, array(value[0]) AS value FROM tmp_pyang_src_rcfile GROUP BY value[0], key) a +LATERAL VIEW explode(value) myTable AS myCol; + +SELECT value, myCol from (SELECT key, array(value[0]) AS value FROM tmp_pyang_src_rcfile GROUP BY value[0], key) a +LATERAL VIEW explode(value) myTable AS myCol; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_cp.q b/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_cp.q new file mode 100644 index 0000000000000..5e0b6fa20ebe9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_cp.q @@ -0,0 +1,8 @@ +--HIVE 3226 +drop table array_valued_src; +create table array_valued_src (key string, value array); +insert overwrite table array_valued_src select key, array(value) from src; + +-- replace sel(*) to sel(exprs) for reflecting CP result properly +explain select count(val) from (select a.key as key, b.value as array_val from src a join array_valued_src b on a.key=b.key) i lateral view explode (array_val) c as val; +select count(val) from (select a.key as key, b.value as array_val from src a join array_valued_src b on a.key=b.key) i lateral view explode (array_val) c as val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_noalias.q b/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_noalias.q new file mode 100644 index 0000000000000..11e54a5b04c07 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_noalias.q @@ -0,0 +1,12 @@ +--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; + +EXPLAIN SELECT explode(map('key1', 100, 'key2', 200)) from src limit 2; +SELECT explode(map('key1', 100, 'key2', 200)) from src limit 2; + +-- view +create view lv_noalias as SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2; + +explain select * from lv_noalias a join lv_noalias b on a.key=b.key; +select * from lv_noalias a join lv_noalias b on a.key=b.key; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_outer.q b/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_outer.q new file mode 100644 index 0000000000000..d65e79df7b3d2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_outer.q @@ -0,0 +1,15 @@ +-- UDTF forwards nothing, OUTER LV add null for that +explain +select * from src LATERAL VIEW OUTER explode(array()) C AS a limit 10; +select * from src LATERAL VIEW OUTER explode(array()) C AS a limit 10; + +-- backward compatible (UDTF forwards something for OUTER LV) +explain +select * from src LATERAL VIEW OUTER explode(array(4,5)) C AS a limit 10; +select * from src LATERAL VIEW OUTER explode(array(4,5)) C AS a limit 10; + +create table array_valued as select key, if (key > 300, array(value, value), null) as value from src; + +explain +select * from array_valued LATERAL VIEW OUTER explode(value) C AS a limit 10; +select * from array_valued LATERAL VIEW OUTER explode(value) C AS a limit 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_ppd.q b/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_ppd.q new file mode 100644 index 0000000000000..7be86a6f10ea2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_ppd.q @@ -0,0 +1,13 @@ +set hive.optimize.ppd=true; + +EXPLAIN SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE key='0'; +SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE key='0'; + +EXPLAIN SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE key='0' AND myCol=1; +SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE key='0' AND myCol=1; + +EXPLAIN SELECT value, myCol FROM (SELECT * FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE ds='2008-04-08' AND hr="12" LIMIT 12; +SELECT value, myCol FROM (SELECT * FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE ds='2008-04-08' AND hr="12" LIMIT 12; + +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'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/leadlag.q b/src/test/hive/ql/src/test/queries/clientpositive/leadlag.q new file mode 100644 index 0000000000000..f49766771067b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/leadlag.q @@ -0,0 +1,82 @@ +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; + +--1. testLagWithPTFWindowing +select p_mfgr, p_name, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_retailprice, sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +); + +-- 2. testLagWithWindowingNoPTF +select p_mfgr, p_name, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_retailprice, sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from part +; + +-- 3. testJoinWithLag +select p1.p_mfgr, p1.p_name, +p1.p_size, p1.p_size - lag(p1.p_size,1,p1.p_size) over( distribute by p1.p_mfgr sort by p1.p_name) as deltaSz +from part p1 join part p2 on p1.p_partkey = p2.p_partkey + ; + +-- 4. testLagInSum +select p_mfgr,p_name, p_size, +sum(p_size - lag(p_size,1)) over(distribute by p_mfgr sort by p_name ) as deltaSum +from part +window w1 as (rows between 2 preceding and 2 following) ; + +-- 5. testLagInSumOverWindow +select p_mfgr,p_name, p_size, +sum(p_size - lag(p_size,1)) over w1 as deltaSum +from part +window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) ; + +-- 6. testRankInLead +select p_mfgr, p_name, p_size, r1, +lead(r1,1,r1) over (distribute by p_mfgr sort by p_name) as deltaRank +from ( +select p_mfgr, p_name, p_size, +rank() over(distribute by p_mfgr sort by p_name) as r1 +from part +) a; + +-- 7. testLeadWithPTF +select p_mfgr, p_name, +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 - lead(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) +; + +-- 8. testOverNoPartitionMultipleAggregate +select p_name, p_retailprice, +lead(p_retailprice) over() as l1 , +lag(p_retailprice) over() as l2 +from part +where p_retailprice = 1173.15; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/leadlag_queries.q b/src/test/hive/ql/src/test/queries/clientpositive/leadlag_queries.q new file mode 100644 index 0000000000000..6ef3bdb107a12 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/leadlag_queries.q @@ -0,0 +1,53 @@ +-- 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; + +-- 1. testLeadUDAF +select p_mfgr, p_retailprice, +lead(p_retailprice) over (partition by p_mfgr order by p_name) as l1, +lead(p_retailprice,1) over (partition by p_mfgr order by p_name) as l2, +lead(p_retailprice,1,10) over (partition by p_mfgr order by p_name) as l3, +lead(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name) as l4, +p_retailprice - lead(p_retailprice,1,p_retailprice) over (partition by p_mfgr order by p_name) +from part; + +-- 2.testLeadUDAFPartSz1 +select p_mfgr, p_name, p_retailprice, +lead(p_retailprice,1) over (partition by p_mfgr, p_name ), +p_retailprice - lead(p_retailprice,1,p_retailprice) over (partition by p_mfgr, p_name) +from part; + +-- 3.testLagUDAF +select p_mfgr, p_retailprice, +lag(p_retailprice,1) over (partition by p_mfgr order by p_name) as l1, +lag(p_retailprice) over (partition by p_mfgr order by p_name) as l2, +lag(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name) as l3, +lag(p_retailprice,1,10) over (partition by p_mfgr order by p_name) as l4, +p_retailprice - lag(p_retailprice,1,p_retailprice) over (partition by p_mfgr order by p_name) +from part; + +-- 4.testLagUDAFPartSz1 +select p_mfgr, p_name, p_retailprice, +lag(p_retailprice,1) over (partition by p_mfgr, p_name ), +p_retailprice - lag(p_retailprice,1,p_retailprice) over (partition by p_mfgr, p_name) +from part; + +-- 5.testLeadLagUDAF +select p_mfgr, p_retailprice, +lead(p_retailprice,1) over (partition by p_mfgr order by p_name) as l1, +lead(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name) as l2, +p_retailprice - lead(p_retailprice,1,p_retailprice) over (partition by p_mfgr order by p_name), +lag(p_retailprice,1) over (partition by p_mfgr order by p_name) as l3, +lag(p_retailprice,1, p_retailprice) over (partition by p_mfgr order by p_name) as l4 +from part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin.q b/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin.q new file mode 100644 index 0000000000000..abe3d3317157c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin.q @@ -0,0 +1,23 @@ +drop table sales; +drop table things; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +CREATE TABLE sales (name STRING, id INT) +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'); + +SELECT name,id FROM sales ORDER BY name ASC, id ASC; + +SELECT id,name FROM things ORDER BY id ASC, name ASC; + +SELECT name,id FROM sales LEFT SEMI JOIN things ON (sales.id = things.id) ORDER BY name ASC, id ASC; + +drop table sales; +drop table things; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin_mr.q b/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin_mr.q new file mode 100644 index 0000000000000..5813ca3c619cc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin_mr.q @@ -0,0 +1,20 @@ +CREATE TABLE T1(key INT); +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; + +-- Run this query using TestMinimrCliDriver + +SELECT * FROM T1; +SELECT * FROM T2; + +set hive.auto.convert.join=false; +set mapred.reduce.tasks=2; + +set hive.join.emit.interval=100; + +SELECT T1.key FROM T1 LEFT SEMI JOIN (SELECT key FROM T2 SORT BY key) tmp ON (T1.key=tmp.key); + +set hive.join.emit.interval=1; + +SELECT T1.key FROM T1 LEFT SEMI JOIN (SELECT key FROM T2 SORT BY key) tmp ON (T1.key=tmp.key); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown.q b/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown.q new file mode 100644 index 0000000000000..e4d0aa06bde87 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown.q @@ -0,0 +1,66 @@ +set hive.limit.pushdown.memory.usage=0.3f; +set hive.optimize.reducededuplication.min.reducer=1; + +-- HIVE-3562 Some limit can be pushed down to map stage + +explain +select key,value from src order by key limit 20; +select key,value from src order by key limit 20; + +explain +select key,value from src order by key desc limit 20; +select key,value from src order by key desc limit 20; + +explain +select value, sum(key + 1) as sum from src group by value limit 20; +select value, sum(key + 1) as sum from src group by value limit 20; + +-- deduped RS +explain +select value,avg(key + 1) from src group by value order by value limit 20; +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; + +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; + +-- limit zero +explain +select key,value from src order by key limit 0; +select key,value from src order by key limit 0; + +-- 2MR (applied to last RS) +explain +select value, sum(key) as sum from src group by value order by sum limit 20; +select value, sum(key) as sum from src group by value order by sum limit 20; + +-- subqueries +explain +select * from +(select key, count(1) from src group by key order by key limit 2) subq +join +(select key, count(1) from src group by key limit 3) subq2 +on subq.key=subq2.key limit 4; + +set hive.map.aggr=false; +-- map aggregation disabled +explain +select value, sum(key) as sum from src group by value limit 20; +select value, sum(key) as sum from src group by value limit 20; + +set hive.limit.pushdown.memory.usage=0.00002f; + +-- flush for order-by +explain +select key,value,value,value,value,value,value,value,value from src order by key limit 100; +select key,value,value,value,value,value,value,value,value from src order by key limit 100; + +-- flush for group-by +explain +select sum(key) as sum from src group by concat(key,value,value,value,value,value,value,value,value,value) limit 100; +select sum(key) as sum from src group by concat(key,value,value,value,value,value,value,value,value,value) limit 100; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown_negative.q b/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown_negative.q new file mode 100644 index 0000000000000..a86ddf14047ef --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown_negative.q @@ -0,0 +1,22 @@ +set hive.limit.pushdown.memory.usage=0.3f; + +-- negative, RS + join +explain select * from src a join src b on a.key=b.key limit 20; + +-- negative, RS + filter +explain select value, sum(key) as sum from src group by value having sum > 100 limit 20; + +-- negative, RS + lateral view +explain select key, L.* from (select * from src order by key) a lateral view explode(array(value, value)) L as v limit 10; + +-- negative, RS + forward + multi-groupby +CREATE TABLE dest_2(key STRING, c1 INT); +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/src/test/hive/ql/src/test/queries/clientpositive/lineage1.q b/src/test/hive/ql/src/test/queries/clientpositive/lineage1.q new file mode 100644 index 0000000000000..4743b7f413ab9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/lineage1.q @@ -0,0 +1,30 @@ + + +CREATE TABLE dest_l1(key INT, value STRING) STORED AS TEXTFILE; + + +EXPLAIN +INSERT OVERWRITE TABLE dest_l1 +SELECT j.* +FROM (SELECT t1.key, p1.value + FROM src1 t1 + LEFT OUTER JOIN src p1 + ON (t1.key = p1.key) + UNION ALL + SELECT t2.key, p2.value + FROM src1 t2 + LEFT OUTER JOIN src p2 + ON (t2.key = p2.key)) j; + +INSERT OVERWRITE TABLE dest_l1 +SELECT j.* +FROM (SELECT t1.key, p1.value + FROM src1 t1 + LEFT OUTER JOIN src p1 + ON (t1.key = p1.key) + UNION ALL + SELECT t2.key, p2.value + FROM src1 t2 + LEFT OUTER JOIN src p2 + ON (t2.key = p2.key)) j; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_1.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_1.q new file mode 100644 index 0000000000000..1062e5503fd43 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_1.q @@ -0,0 +1,40 @@ +set hive.mapred.supports.subdirectories=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +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; + +-- list bucketing DML : dynamic partition and 2 stage query plan. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- create a skewed table +create table list_bucketing_dynamic_part (key String, value String) +partitioned by (ds String, hr String) +skewed by (key) on ("484") +stored as DIRECTORIES +; + +-- list bucketing DML +explain extended +insert overwrite table list_bucketing_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart where ds='2008-04-08'; +insert overwrite table list_bucketing_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart where ds='2008-04-08'; + +-- check DML result +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='11'); +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='12'); + +select count(1) from srcpart where ds='2008-04-08'; +select count(1) from list_bucketing_dynamic_part where ds='2008-04-08'; + +select key, value from srcpart where ds='2008-04-08' and hr='11' and key = "484"; +set hive.optimize.listbucketing=true; +explain extended +select key, value from list_bucketing_dynamic_part where ds='2008-04-08' and hr='11' and key = "484"; +select key, value from list_bucketing_dynamic_part where ds='2008-04-08' and hr='11' and key = "484"; + +-- clean up resources +drop table list_bucketing_dynamic_part; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_10.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_10.q new file mode 100644 index 0000000000000..2fbafc20169b5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_10.q @@ -0,0 +1,35 @@ +set hive.mapred.supports.subdirectories=true; + +set mapred.input.dir.recursive=true; + +-- run this test case in minimr to ensure it works in cluster + +-- list bucketing DML: static partition. multiple skewed columns. +-- ds=2008-04-08/hr=11/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME: +-- 5263 000000_0 +-- 5263 000001_0 +-- ds=2008-04-08/hr=11/key=103/value=val_103: +-- 99 000000_0 +-- 99 000001_0 +-- ds=2008-04-08/hr=11/key=484/value=val_484: +-- 87 000000_0 +-- 87 000001_0 + +-- create a skewed table +create table list_bucketing_static_part (key String, value String) + partitioned by (ds String, hr String) + skewed by (key) on ('484','51','103') + stored as DIRECTORIES + STORED AS RCFILE; + +-- list bucketing DML without merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from src; + +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from src; + +-- check DML result +show partitions list_bucketing_static_part; +desc formatted list_bucketing_static_part partition (ds='2008-04-08', hr='11'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_11.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_11.q new file mode 100644 index 0000000000000..8260459725efe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_11.q @@ -0,0 +1,36 @@ +set hive.mapred.supports.subdirectories=true; +set mapred.input.dir.recursive=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- Ensure it works if skewed column is not the first column in the table columns + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- list bucketing DML: static partition. multiple skewed columns. + +-- create a skewed table +create table list_bucketing_static_part (key String, value String) + partitioned by (ds String, hr String) + skewed by (value) on ('val_466','val_287','val_82') + stored as DIRECTORIES + STORED AS RCFILE; + +-- list bucketing DML without merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from src; + +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from src; + +-- check DML result +show partitions list_bucketing_static_part; +desc formatted list_bucketing_static_part partition (ds='2008-04-08', hr='11'); + +set hive.optimize.listbucketing=true; +explain extended +select key, value from list_bucketing_static_part where ds='2008-04-08' and hr='11' and value = "val_466"; +select key, value from list_bucketing_static_part where ds='2008-04-08' and hr='11' and value = "val_466"; + +drop table list_bucketing_static_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_12.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_12.q new file mode 100644 index 0000000000000..3606e3b8b77c3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_12.q @@ -0,0 +1,42 @@ +set hive.mapred.supports.subdirectories=true; +set mapred.input.dir.recursive=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- Ensure it works if skewed column is not the first column in the table columns + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- test where the skewed values are more than 1 say columns no. 2 and 4 in a table with 5 columns +create table list_bucketing_mul_col (col1 String, col2 String, col3 String, col4 String, col5 string) + partitioned by (ds String, hr String) + skewed by (col2, col4) on (('466','val_466'),('287','val_287'),('82','val_82')) + stored as DIRECTORIES + STORED AS RCFILE; + +-- list bucketing DML +explain extended +insert overwrite table list_bucketing_mul_col partition (ds = '2008-04-08', hr = '11') +select 1, key, 1, value, 1 from src; + +insert overwrite table list_bucketing_mul_col partition (ds = '2008-04-08', hr = '11') +select 1, key, 1, value, 1 from src; + +-- check DML result +show partitions list_bucketing_mul_col; +desc formatted list_bucketing_mul_col partition (ds='2008-04-08', hr='11'); + +set hive.optimize.listbucketing=true; +explain extended +select * from list_bucketing_mul_col +where ds='2008-04-08' and hr='11' and col2 = "466" and col4 = "val_466" ORDER BY col2, col4, ds, hr; +select * from list_bucketing_mul_col +where ds='2008-04-08' and hr='11' and col2 = "466" and col4 = "val_466" ORDER BY col2, col4, ds, hr; + +explain extended +select * from list_bucketing_mul_col +where ds='2008-04-08' and hr='11' and col2 = "382" and col4 = "val_382" ORDER BY col2, col4, ds, hr; +select * from list_bucketing_mul_col +where ds='2008-04-08' and hr='11' and col2 = "382" and col4 = "val_382" ORDER BY col2, col4, ds, hr; + +drop table list_bucketing_mul_col; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_13.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_13.q new file mode 100644 index 0000000000000..4903604bd0063 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_13.q @@ -0,0 +1,36 @@ +set hive.mapred.supports.subdirectories=true; +set mapred.input.dir.recursive=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- Ensure skewed value map has escaped directory name + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- test where the skewed values are more than 1 say columns no. 2 and 4 in a table with 5 columns +create table list_bucketing_mul_col (col1 String, col2 String, col3 String, col4 String, col5 string) + partitioned by (ds String, hr String) + skewed by (col2, col4) on (('466','val_466'),('287','val_287'),('82','val_82')) + stored as DIRECTORIES + STORED AS RCFILE; + +-- list bucketing DML +explain extended +insert overwrite table list_bucketing_mul_col partition (ds = '2008-04-08', hr = '2013-01-23+18:00:99') +select 1, key, 1, value, 1 from src; + +insert overwrite table list_bucketing_mul_col partition (ds = '2008-04-08', hr = '2013-01-23+18:00:99') +select 1, key, 1, value, 1 from src; + +-- check DML result +show partitions list_bucketing_mul_col; +desc formatted list_bucketing_mul_col partition (ds='2008-04-08', hr='2013-01-23+18:00:99'); + +set hive.optimize.listbucketing=true; +explain extended +select * from list_bucketing_mul_col +where ds='2008-04-08' and hr='2013-01-23+18:00:99' and col2 = "466" and col4 = "val_466" ORDER BY col2, col4, ds, hr; +select * from list_bucketing_mul_col +where ds='2008-04-08' and hr='2013-01-23+18:00:99' and col2 = "466" and col4 = "val_466" ORDER BY col2, col4, ds, hr; + +drop table list_bucketing_mul_col; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_2.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_2.q new file mode 100644 index 0000000000000..3a39f42b2a77d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_2.q @@ -0,0 +1,71 @@ +set hive.mapred.supports.subdirectories=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.merge.smallfiles.avgsize=200; +set mapred.input.dir.recursive=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.stats.reliable=true; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- list bucketing DML: static partition. multiple skewed columns. +-- ds=2008-04-08/hr=11/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME: +-- 5263 000000_0 +-- 5263 000001_0 +-- ds=2008-04-08/hr=11/key=103/value=val_103: +-- 99 000000_0 +-- 99 000001_0 +-- ds=2008-04-08/hr=11/key=484/value=val_484: +-- 87 000000_0 +-- 87 000001_0 + +-- create a skewed table +create table list_bucketing_static_part (key String, value String) + partitioned by (ds String, hr String) + skewed by (key, value) on (('484','val_484'),('51','val_14'),('103','val_103')) + stored as DIRECTORIES + STORED AS RCFILE; + +-- list bucketing DML without merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from srcpart where ds = '2008-04-08'; + +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from srcpart where ds = '2008-04-08'; + +-- check DML result +show partitions list_bucketing_static_part; +desc formatted list_bucketing_static_part partition (ds='2008-04-08', hr='11'); + +select count(1) from srcpart where ds = '2008-04-08'; +select count(*) from list_bucketing_static_part; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +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; + +-- 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 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 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 +select count(1) from srcpart where ds = '2008-04-08' and key < '51'; +select count(1) from list_bucketing_static_part where key < '51'; +select count(1) from srcpart where ds = '2008-04-08' and key <= '51'; +select count(1) from list_bucketing_static_part where key <= '51'; +select count(1) from srcpart where ds = '2008-04-08' and key > '51'; +select count(1) from list_bucketing_static_part where key > '51'; +select count(1) from srcpart where ds = '2008-04-08' and key >= '51'; +select count(1) from list_bucketing_static_part where key >= '51'; + +-- clean up +drop table list_bucketing_static_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_3.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_3.q new file mode 100644 index 0000000000000..a43613cad5e8a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_3.q @@ -0,0 +1,33 @@ +set hive.mapred.supports.subdirectories=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +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; + +-- list bucketing DML : static partition and 2 stage query plan. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- create a skewed table +create table list_bucketing_static_part (key String, value String) partitioned by (ds String, hr String) skewed by (key) on ("484") stored as DIRECTORIES; + +-- list bucketing DML +explain extended +insert overwrite table list_bucketing_static_part partition (ds='2008-04-08', hr='11') select key, value from srcpart where ds='2008-04-08'; +insert overwrite table list_bucketing_static_part partition (ds='2008-04-08', hr='11') select key, value from srcpart where ds='2008-04-08'; + +-- check DML result +desc formatted list_bucketing_static_part partition (ds='2008-04-08', hr='11'); + +select count(1) from srcpart where ds='2008-04-08'; +select count(1) from list_bucketing_static_part where ds='2008-04-08'; + +select key, value from srcpart where ds='2008-04-08' and hr='11' and key = "484"; +set hive.optimize.listbucketing=true; +explain extended +select key, value from list_bucketing_static_part where ds='2008-04-08' and hr='11' and key = "484"; +select key, value from list_bucketing_static_part where ds='2008-04-08' and hr='11' and key = "484"; +-- clean up resources +drop table list_bucketing_static_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_4.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_4.q new file mode 100644 index 0000000000000..918c817e49eff --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_4.q @@ -0,0 +1,71 @@ +set hive.mapred.supports.subdirectories=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.merge.smallfiles.avgsize=200; +set mapred.input.dir.recursive=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- list bucketing DML: static partition. multiple skewed columns. merge. +-- ds=2008-04-08/hr=11/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME: +-- 5263 000000_0 +-- 5263 000001_0 +-- ds=2008-04-08/hr=11/key=103/value=val_103: +-- 99 000000_0 +-- 99 000001_0 +-- after merge +-- 142 000000_0 +-- ds=2008-04-08/hr=11/key=484/value=val_484: +-- 87 000000_0 +-- 87 000001_0 +-- after merge +-- 118 000001_0 + +-- create a skewed table +create table list_bucketing_static_part (key String, value String) + partitioned by (ds String, hr String) + skewed by (key, value) on (('484','val_484'),('51','val_14'),('103','val_103')) + stored as DIRECTORIES + STORED AS RCFILE; + +-- list bucketing DML without merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from srcpart where ds = '2008-04-08'; + +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from srcpart where ds = '2008-04-08'; + +-- check DML result +show partitions list_bucketing_static_part; +desc formatted list_bucketing_static_part partition (ds='2008-04-08', hr='11'); + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +-- list bucketing DML with merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from srcpart where ds = '2008-04-08'; + +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from srcpart where ds = '2008-04-08'; + +-- check DML result +show partitions list_bucketing_static_part; +desc formatted list_bucketing_static_part partition (ds='2008-04-08', hr='11'); + +select count(1) from srcpart where ds = '2008-04-08'; +select count(*) from list_bucketing_static_part; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +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; + +-- clean up +drop table list_bucketing_static_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_5.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_5.q new file mode 100644 index 0000000000000..5ea2e62551a66 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_5.q @@ -0,0 +1,38 @@ +set hive.mapred.supports.subdirectories=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +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; + +-- list bucketing DML: multiple skewed columns. 2 stages + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- create a skewed table +create table list_bucketing_dynamic_part (key String, value String) +partitioned by (ds String, hr String) +skewed by (key, value) on (('484','val_484'),('51','val_14'),('103','val_103')) +stored as DIRECTORIES; + +-- list bucketing DML +explain extended +insert overwrite table list_bucketing_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart where ds='2008-04-08'; +insert overwrite table list_bucketing_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart where ds='2008-04-08'; + +-- check DML result +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='11'); +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='12'); + +select count(1) from srcpart where ds='2008-04-08'; +select count(1) from list_bucketing_dynamic_part where ds='2008-04-08'; + +select key, value from srcpart where ds='2008-04-08' and key = "103" and value ="val_103" ORDER BY key, value; +set hive.optimize.listbucketing=true; +explain extended +select key, value, ds, hr from list_bucketing_dynamic_part where ds='2008-04-08' and key = "103" and value ="val_103" ORDER BY key, value, ds, hr; +select key, value, ds, hr from list_bucketing_dynamic_part where ds='2008-04-08' and key = "103" and value ="val_103" ORDER BY key, value, ds, hr; + +-- clean up resources +drop table list_bucketing_dynamic_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_6.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_6.q new file mode 100644 index 0000000000000..d50e159277eb8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_6.q @@ -0,0 +1,97 @@ +set hive.mapred.supports.subdirectories=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.merge.smallfiles.avgsize=200; +set mapred.input.dir.recursive=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- list bucketing DML: dynamic partition. multiple skewed columns. merge. +-- The following explains merge example used in this test case +-- DML will generated 2 partitions +-- ds=2008-04-08/hr=a1 +-- ds=2008-04-08/hr=b1 +-- without merge, each partition has more files +-- ds=2008-04-08/hr=a1 has 2 files +-- ds=2008-04-08/hr=b1 has 6 files +-- with merge each partition has more files +-- ds=2008-04-08/hr=a1 has 1 files +-- ds=2008-04-08/hr=b1 has 4 files +-- The following shows file size and name in each directory +-- hr=a1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME: +-- without merge +-- 155 000000_0 +-- 155 000001_0 +-- with merge +-- 254 000000_0 +-- hr=b1/key=103/value=val_103: +-- without merge +-- 99 000000_0 +-- 99 000001_0 +-- with merge +-- 142 000001_0 +-- hr=b1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME: +-- without merge +-- 5181 000000_0 +-- 5181 000001_0 +-- with merge +-- 5181 000000_0 +-- 5181 000001_0 +-- hr=b1/key=484/value=val_484 +-- without merge +-- 87 000000_0 +-- 87 000001_0 +-- with merge +-- 118 000002_0 + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- create a skewed table +create table list_bucketing_dynamic_part (key String, value String) + partitioned by (ds String, hr String) + skewed by (key, value) on (('484','val_484'),('51','val_14'),('103','val_103')) + stored as DIRECTORIES + STORED AS RCFILE; + +-- list bucketing DML without merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') from srcpart where ds = '2008-04-08'; + +insert overwrite table list_bucketing_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') from srcpart where ds = '2008-04-08'; + +-- check DML result +show partitions list_bucketing_dynamic_part; +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='a1'); +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='b1'); + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +-- list bucketing DML with merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') from srcpart where ds = '2008-04-08'; + +insert overwrite table list_bucketing_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') from srcpart where ds = '2008-04-08'; + +-- check DML result +show partitions list_bucketing_dynamic_part; +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='a1'); +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='b1'); + +select count(1) from srcpart where ds = '2008-04-08'; +select count(*) from list_bucketing_dynamic_part; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.optimize.listbucketing=true; +explain extended +select * from list_bucketing_dynamic_part where key = '484' and value = 'val_484'; +select * from list_bucketing_dynamic_part where key = '484' and value = 'val_484'; +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_dynamic_part; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_7.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_7.q new file mode 100644 index 0000000000000..f71e80fa2a099 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_7.q @@ -0,0 +1,70 @@ +set hive.mapred.supports.subdirectories=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.merge.smallfiles.avgsize=200; +set mapred.input.dir.recursive=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.merge.rcfile.block.level=true; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- list bucketing DML : dynamic partition (one level) , merge , one skewed column +-- DML without merge files mixed with small and big files: +-- ds=2008-04-08/hr=a1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/ +-- 155 000000_0 +-- ds=2008-04-08/hr=b1/key=484 +-- 87 000000_0 +-- 87 000001_0 +-- ds=2008-04-08/hr=b1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/ +-- 5201 000000_0 +-- 5201 000001_0 +-- DML with merge will merge small files + +-- skewed table +CREATE TABLE list_bucketing_dynamic_part (key String, value STRING) + PARTITIONED BY (ds string, hr string) + skewed by (key) on ('484') + stored as DIRECTORIES + STORED AS RCFILE; + +-- list bucketing DML without merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') from srcpart where ds = '2008-04-08'; + +insert overwrite table list_bucketing_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') from srcpart where ds = '2008-04-08'; + +-- check DML result +show partitions list_bucketing_dynamic_part; +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='a1'); +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='b1'); + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +-- list bucketing DML with merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') from srcpart where ds = '2008-04-08'; + +insert overwrite table list_bucketing_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') from srcpart where ds = '2008-04-08'; + +-- check DML result +show partitions list_bucketing_dynamic_part; +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='a1'); +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='b1'); + +select count(1) from srcpart where ds = '2008-04-08'; +select count(*) from list_bucketing_dynamic_part; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +explain extended +select * from list_bucketing_dynamic_part where key = '484' and value = 'val_484'; +select * from list_bucketing_dynamic_part where key = '484' and value = 'val_484'; +select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' order by hr; + +-- clean up +drop table list_bucketing_dynamic_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_8.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_8.q new file mode 100644 index 0000000000000..9e81b8d585e6f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_8.q @@ -0,0 +1,90 @@ +set hive.mapred.supports.subdirectories=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.merge.smallfiles.avgsize=200; +set mapred.input.dir.recursive=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- list bucketing alter table ... concatenate: +-- Use list bucketing DML to generate mutilple files in partitions by turning off merge +-- dynamic partition. multiple skewed columns. merge. +-- The following explains merge example used in this test case +-- DML will generated 2 partitions +-- ds=2008-04-08/hr=a1 +-- ds=2008-04-08/hr=b1 +-- without merge, each partition has more files +-- ds=2008-04-08/hr=a1 has 2 files +-- ds=2008-04-08/hr=b1 has 6 files +-- with merge each partition has more files +-- ds=2008-04-08/hr=a1 has 1 files +-- ds=2008-04-08/hr=b1 has 4 files +-- The following shows file size and name in each directory +-- hr=a1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME: +-- without merge +-- 155 000000_0 +-- 155 000001_0 +-- with merge +-- 254 000000_0 +-- hr=b1/key=103/value=val_103: +-- without merge +-- 99 000000_0 +-- 99 000001_0 +-- with merge +-- 142 000001_0 +-- hr=b1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME: +-- without merge +-- 5181 000000_0 +-- 5181 000001_0 +-- with merge +-- 5181 000000_0 +-- 5181 000001_0 +-- hr=b1/key=484/value=val_484 +-- without merge +-- 87 000000_0 +-- 87 000001_0 +-- with merge +-- 118 000002_0 + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- create a skewed table +create table list_bucketing_dynamic_part (key String, value String) + partitioned by (ds String, hr String) + skewed by (key, value) on (('484','val_484'),('51','val_14'),('103','val_103')) + stored as DIRECTORIES + STORED AS RCFILE; + +-- list bucketing DML without merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') from srcpart where ds = '2008-04-08'; + +insert overwrite table list_bucketing_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') from srcpart where ds = '2008-04-08'; + +-- check DML result +show partitions list_bucketing_dynamic_part; +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='a1'); +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='b1'); + +set hive.merge.current.job.concatenate.list.bucketing=true; +-- concatenate the partition and it will merge files +alter table list_bucketing_dynamic_part partition (ds='2008-04-08', hr='b1') concatenate; + +desc formatted list_bucketing_dynamic_part partition (ds='2008-04-08', hr='b1'); + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select count(1) from srcpart where ds = '2008-04-08'; +select count(*) from list_bucketing_dynamic_part; +explain extended +select * from list_bucketing_dynamic_part where key = '484' and value = 'val_484'; +select * from list_bucketing_dynamic_part where key = '484' and value = 'val_484'; +select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' order by hr; + +-- clean up +drop table list_bucketing_dynamic_part; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_9.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_9.q new file mode 100644 index 0000000000000..1b69a9a0d39ef --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_9.q @@ -0,0 +1,71 @@ +set hive.mapred.supports.subdirectories=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.merge.smallfiles.avgsize=200; +set mapred.input.dir.recursive=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- list bucketing DML: static partition. multiple skewed columns. merge. +-- ds=2008-04-08/hr=11/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME: +-- 5263 000000_0 +-- 5263 000001_0 +-- ds=2008-04-08/hr=11/key=103: +-- 99 000000_0 +-- 99 000001_0 +-- after merge +-- 142 000000_0 +-- ds=2008-04-08/hr=11/key=484: +-- 87 000000_0 +-- 87 000001_0 +-- after merge +-- 118 000001_0 + +-- create a skewed table +create table list_bucketing_static_part (key String, value String) + partitioned by (ds String, hr String) + skewed by (key) on ('484','103') + stored as DIRECTORIES + STORED AS RCFILE; + +-- list bucketing DML without merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from srcpart where ds = '2008-04-08'; + +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from srcpart where ds = '2008-04-08'; + +-- check DML result +show partitions list_bucketing_static_part; +desc formatted list_bucketing_static_part partition (ds='2008-04-08', hr='11'); + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +-- list bucketing DML with merge. use bucketize to generate a few small files. +explain extended +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from srcpart where ds = '2008-04-08'; + +insert overwrite table list_bucketing_static_part partition (ds = '2008-04-08', hr = '11') +select key, value from srcpart where ds = '2008-04-08'; + +-- check DML result +show partitions list_bucketing_static_part; +desc formatted list_bucketing_static_part partition (ds='2008-04-08', hr='11'); + +select count(1) from srcpart where ds = '2008-04-08'; +select count(*) from list_bucketing_static_part; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +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, ds, hr; + +-- clean up +drop table list_bucketing_static_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q new file mode 100644 index 0000000000000..48b1ff3ac81a7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q @@ -0,0 +1,56 @@ +set hive.mapred.supports.subdirectories=true; +set hive.optimize.listbucketing=true; +set mapred.input.dir.recursive=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- List bucketing query logic test case. We simulate the directory structure by DML here. +-- Test condition: +-- 1. where clause has multiple skewed columns +-- 2. where clause doesn't have non-skewed column +-- 3. where clause has one and operator +-- Test focus: +-- 1. basic list bucketing query work +-- Test result: +-- 1. pruner only pick up right directory +-- 2. query result is right + +-- create a skewed table +create table fact_daily (key String, value String) +partitioned by (ds String, hr String) +skewed by (key, value) on (('484','val_484'),('238','val_238')) +stored as DIRECTORIES; + +insert overwrite table fact_daily partition (ds = '1', hr = '4') +select key, value from src; + +describe formatted fact_daily PARTITION (ds = '1', hr='4'); + +SELECT count(1) FROM fact_daily WHERE ds='1' and hr='4'; + +-- pruner only pick up skewed-value directory +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended SELECT key FROM fact_daily WHERE ( ds='1' and hr='4') and (key='484' and value= 'val_484'); +-- List Bucketing Query +SELECT key FROM fact_daily WHERE ( ds='1' and hr='4') and (key='484' and value= 'val_484'); + +-- pruner only pick up skewed-value directory +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended SELECT key,value FROM fact_daily WHERE ( ds='1' and hr='4') and (key='238' and value= 'val_238'); +-- List Bucketing Query +SELECT key,value FROM fact_daily WHERE ( ds='1' and hr='4') and (key='238' and value= 'val_238'); + +-- pruner only pick up default directory +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended SELECT key FROM fact_daily WHERE ( ds='1' and hr='4') and (value = "3"); +-- List Bucketing Query +SELECT key FROM fact_daily WHERE ( ds='1' and hr='4') and (value = "3"); + +-- pruner only pick up default directory +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended SELECT key,value FROM fact_daily WHERE ( ds='1' and hr='4') and key = '495'; +-- List Bucketing Query +SELECT key,value FROM fact_daily WHERE ( ds='1' and hr='4') and key = '369'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q new file mode 100644 index 0000000000000..a364168a6e3be --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q @@ -0,0 +1,50 @@ + set hive.mapred.supports.subdirectories=true; +set hive.optimize.listbucketing=true; +set mapred.input.dir.recursive=true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- List bucketing query logic test case. We simulate the directory structure by DML here. +-- Test condition: +-- 1. where clause has multiple skewed columns and non-skewed columns +-- 3. where clause has a few operators +-- Test focus: +-- 1. basic list bucketing query work +-- Test result: +-- 1. pruner only pick up right directory +-- 2. query result is right + +-- create a skewed table +create table fact_daily (key String, value String) +partitioned by (ds String, hr String) +skewed by (key, value) on (('484','val_484'),('238','val_238')) +stored as DIRECTORIES; + +insert overwrite table fact_daily partition (ds = '1', hr = '4') +select key, value from src; + +describe formatted fact_daily PARTITION (ds = '1', hr='4'); + +SELECT count(1) FROM fact_daily WHERE ds='1' and hr='4'; + +-- pruner only pick up default directory +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended SELECT key, value FROM fact_daily WHERE ds='1' and hr='4' and value= 'val_484' ORDER BY key, value; +-- List Bucketing Query +SELECT key, value FROM fact_daily WHERE ds='1' and hr='4' and value= 'val_484' ORDER BY key, value; + +-- pruner only pick up default directory +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended SELECT key FROM fact_daily WHERE ds='1' and hr='4' and key= '406' ORDER BY key; +-- List Bucketing Query +SELECT key, value FROM fact_daily WHERE ds='1' and hr='4' and key= '406' ORDER BY key; + +-- pruner only pick up skewed-value directory +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended SELECT key, value FROM fact_daily WHERE ds='1' and hr='4' and ( (key='484' and value ='val_484') or (key='238' and value= 'val_238')) ORDER BY key, value; +-- List Bucketing Query +SELECT key, value FROM fact_daily WHERE ds='1' and hr='4' and ( (key='484' and value ='val_484') or (key='238' and value= 'val_238')) ORDER BY key, value; + +-- clean up +drop table fact_daily; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q new file mode 100644 index 0000000000000..2d1193deff0cc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q @@ -0,0 +1,57 @@ +set hive.mapred.supports.subdirectories=true; +set hive.optimize.listbucketing=true; +set mapred.input.dir.recursive=true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- List bucketing query logic test case. We simulate the directory structure by DML here. +-- Test condition: +-- 1. where clause has multiple skewed columns and non-skewed columns +-- 3. where clause has a few operators +-- Test focus: +-- 1. query works for on partition level. +-- A table can mix up non-skewed partition and skewed partition +-- Even for skewed partition, it can have different skewed information. +-- Test result: +-- 1. pruner only pick up right directory +-- 2. query result is right + +-- create a skewed table +create table fact_daily (key String, value String) +partitioned by (ds String, hr String) ; + +-- partition no skew +insert overwrite table fact_daily partition (ds = '1', hr = '1') +select key, value from src; +describe formatted fact_daily PARTITION (ds = '1', hr='1'); + +-- partition. skewed value is 484/238 +alter table fact_daily skewed by (key, value) on (('484','val_484'),('238','val_238')) stored as DIRECTORIES; +insert overwrite table fact_daily partition (ds = '1', hr = '2') +select key, value from src; +describe formatted fact_daily PARTITION (ds = '1', hr='2'); + +-- another partition. skewed value is 327 +alter table fact_daily skewed by (key, value) on (('327','val_327')) stored as DIRECTORIES; +insert overwrite table fact_daily partition (ds = '1', hr = '3') +select key, value from src; +describe formatted fact_daily PARTITION (ds = '1', hr='3'); + +-- query non-skewed partition +explain extended +select * from fact_daily where ds = '1' and hr='1' and key='145'; +select * from fact_daily where ds = '1' and hr='1' and key='145'; +explain extended +select count(*) from fact_daily where ds = '1' and hr='1'; +select count(*) from fact_daily where ds = '1' and hr='1'; + +-- query skewed partition +explain extended +SELECT * FROM fact_daily WHERE ds='1' and hr='2' and (key='484' and value='val_484'); +SELECT * FROM fact_daily WHERE ds='1' and hr='2' and (key='484' and value='val_484'); + +-- query another skewed partition +explain extended +SELECT * FROM fact_daily WHERE ds='1' and hr='3' and (key='327' and value='val_327'); +SELECT * FROM fact_daily WHERE ds='1' and hr='3' and (key='327' and value='val_327'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q new file mode 100644 index 0000000000000..7769fb7af75d2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q @@ -0,0 +1,61 @@ +set hive.mapred.supports.subdirectories=true; +set hive.optimize.listbucketing=true; +set mapred.input.dir.recursive=true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- List bucketing query logic test case. +-- Test condition: +-- 1. where clause has only one skewed column +-- 2. where clause doesn't have non-skewed column +-- 3. where clause has one and operator +-- Test result: +-- 1. pruner only pick up right directory +-- 2. query result is right + +-- create 2 tables: fact_daily and fact_tz +-- fact_daily will be used for list bucketing query +-- fact_tz is a table used to prepare data and test directories +CREATE TABLE fact_daily(x int) PARTITIONED BY (ds STRING); +CREATE TABLE fact_tz(x int) PARTITIONED BY (ds STRING, hr STRING) +LOCATION '${hiveconf:hive.metastore.warehouse.dir}/fact_tz'; + +-- create /fact_tz/ds=1/hr=1 directory +INSERT OVERWRITE TABLE fact_tz PARTITION (ds='1', hr='1') +SELECT key FROM src WHERE key=484; + +-- create /fact_tz/ds=1/hr=2 directory +INSERT OVERWRITE TABLE fact_tz PARTITION (ds='1', hr='2') +SELECT key+11 FROM src WHERE key=484; + +dfs -lsr ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1; +dfs -mv ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/hr=1 ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/x=484; +dfs -mv ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/hr=2 ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME; +dfs -lsr ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1; + +-- switch fact_daily to skewed table and point its location to /fact_tz/ds=1 +alter table fact_daily skewed by (x) on (484); +ALTER TABLE fact_daily SET TBLPROPERTIES('EXTERNAL'='TRUE'); +ALTER TABLE fact_daily ADD PARTITION (ds='1') +LOCATION '${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1'; + +-- set List Bucketing location map +alter table fact_daily PARTITION (ds = '1') set skewed location (484='${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/x=484','HIVE_DEFAULT_LIST_BUCKETING_KEY'='${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME'); +describe formatted fact_daily PARTITION (ds = '1'); + +SELECT * FROM fact_daily WHERE ds='1' ORDER BY x; + +-- pruner only pick up skewed-value directory +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended SELECT x FROM fact_daily WHERE ds='1' and x=484; +-- List Bucketing Query +SELECT x FROM fact_daily WHERE ds='1' and x=484; + +-- pruner only pick up default directory since x equal to non-skewed value +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended SELECT x FROM fact_daily WHERE ds='1' and x=495; +-- List Bucketing Query +SELECT x FROM fact_daily WHERE ds='1' and x=495; +explain extended SELECT x FROM fact_daily WHERE ds='1' and x=1; +SELECT x FROM fact_daily WHERE ds='1' and x=1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q new file mode 100644 index 0000000000000..6a6945d4fdf9e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q @@ -0,0 +1,73 @@ +set hive.mapred.supports.subdirectories=true; +set hive.optimize.listbucketing=true; +set mapred.input.dir.recursive=true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- List bucketing query logic test case. +-- Test condition: +-- 1. where clause has only one skewed column +-- 2. where clause doesn't have non-skewed column +-- Test focus: +-- 1. list bucketing query logic works fine for subquery +-- Test result: +-- 1. pruner only pick up right directory +-- 2. query result is right + +-- create 2 tables: fact_daily and fact_tz +-- fact_daily will be used for list bucketing query +-- fact_tz is a table used to prepare data and test directories +CREATE TABLE fact_daily(x int, y STRING) PARTITIONED BY (ds STRING); +CREATE TABLE fact_tz(x int, y STRING) PARTITIONED BY (ds STRING, hr STRING) +LOCATION '${hiveconf:hive.metastore.warehouse.dir}/fact_tz'; + +-- create /fact_tz/ds=1/hr=1 directory +INSERT OVERWRITE TABLE fact_tz PARTITION (ds='1', hr='1') +SELECT key, value FROM src WHERE key=484; + +-- create /fact_tz/ds=1/hr=2 directory +INSERT OVERWRITE TABLE fact_tz PARTITION (ds='1', hr='2') +SELECT key+11, value FROM src WHERE key=484; + +dfs -lsr ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1; +dfs -mv ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/hr=1 ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/x=484; +dfs -mv ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/hr=2 ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME; +dfs -lsr ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1; + +-- switch fact_daily to skewed table and point its location to /fact_tz/ds=1 +alter table fact_daily skewed by (x) on (484); +ALTER TABLE fact_daily SET TBLPROPERTIES('EXTERNAL'='TRUE'); +ALTER TABLE fact_daily ADD PARTITION (ds='1') +LOCATION '${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1'; + +-- set List Bucketing location map +alter table fact_daily PARTITION (ds = '1') set skewed location (484='${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/x=484','HIVE_DEFAULT_LIST_BUCKETING_KEY'='${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME'); +describe formatted fact_daily PARTITION (ds = '1'); + +SELECT * FROM fact_daily WHERE ds='1' ORDER BY x, y; + +-- The first subquery +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended select x from (select x from fact_daily where ds = '1') subq where x = 484; +-- List Bucketing Query +select x from (select * from fact_daily where ds = '1') subq where x = 484; + +-- The second subquery +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended select x1, y1 from(select x as x1, y as y1 from fact_daily where ds ='1') subq where x1 = 484 ORDER BY x1, y1; +-- List Bucketing Query +select x1, y1 from(select x as x1, y as y1 from fact_daily where ds ='1') subq where x1 = 484 ORDER BY x1, y1; + + +-- The third subquery +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended select y, count(1) from fact_daily where ds ='1' and x = 484 group by y; +-- List Bucketing Query +select y, count(1) from fact_daily where ds ='1' and x = 484 group by y; + +-- The fourth subquery +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended select x, c from (select x, count(1) as c from fact_daily where ds = '1' group by x) subq where x = 484;; +-- List Bucketing Query +select x, c from (select x, count(1) as c from fact_daily where ds = '1' group by x) subq where x = 484; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q new file mode 100644 index 0000000000000..892b4c5d3d7cc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q @@ -0,0 +1,61 @@ +set hive.optimize.listbucketing=true; +set mapred.input.dir.recursive=true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.mapred.supports.subdirectories=true; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +-- List bucketing query logic test case. +-- Test condition: +-- 1. where clause has single skewed columns and non-skewed columns +-- 3. where clause has a few operators +-- Test focus: +-- 1. basic list bucketing query works for not (equal) case +-- Test result: +-- 1. pruner only pick up right directory +-- 2. query result is right + +-- create 2 tables: fact_daily and fact_tz +-- fact_daily will be used for list bucketing query +-- fact_tz is a table used to prepare data and test directories +CREATE TABLE fact_daily(x int, y STRING, z STRING) PARTITIONED BY (ds STRING); +CREATE TABLE fact_tz(x int, y STRING, z STRING) PARTITIONED BY (ds STRING, hr STRING) +LOCATION '${hiveconf:hive.metastore.warehouse.dir}/fact_tz'; + +-- create /fact_tz/ds=1/hr=1 directory +INSERT OVERWRITE TABLE fact_tz PARTITION (ds='1', hr='1') +SELECT key, value, value FROM src WHERE key=484; + +-- create /fact_tz/ds=1/hr=2 directory +INSERT OVERWRITE TABLE fact_tz PARTITION (ds='1', hr='2') +SELECT key, value, value FROM src WHERE key=278 or key=86; + +-- create /fact_tz/ds=1/hr=3 directory +INSERT OVERWRITE TABLE fact_tz PARTITION (ds='1', hr='3') +SELECT key, value, value FROM src WHERE key=238; + +dfs -lsr ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1; +dfs -mv ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/hr=1 ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/x=484; +dfs -mv ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/hr=2 ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME; +dfs -mv ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/hr=3 ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/x=238; +dfs -lsr ${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1; + +-- switch fact_daily to skewed table and point its location to /fact_tz/ds=1 +alter table fact_daily skewed by (x) on (484,238); +ALTER TABLE fact_daily SET TBLPROPERTIES('EXTERNAL'='TRUE'); +ALTER TABLE fact_daily ADD PARTITION (ds='1') +LOCATION '${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1'; + +-- set List Bucketing location map +alter table fact_daily PARTITION (ds = '1') set skewed location (484='${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/x=484', +238='${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/x=238', +'HIVE_DEFAULT_LIST_BUCKETING_KEY'='${hiveconf:hive.metastore.warehouse.dir}/fact_tz/ds=1/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME'); +describe formatted fact_daily PARTITION (ds = '1'); + +SELECT * FROM fact_daily WHERE ds='1' ORDER BY x, y, z; + +-- pruner pick up right directory +-- explain plan shows which directory selected: Truncated Path -> Alias +explain extended SELECT x FROM fact_daily WHERE ds='1' and not (x = 86) ORDER BY x; +-- List Bucketing Query +SELECT x FROM fact_daily WHERE ds='1' and not (x = 86) ORDER BY x; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/literal_decimal.q b/src/test/hive/ql/src/test/queries/clientpositive/literal_decimal.q new file mode 100644 index 0000000000000..a6ad4b8485a26 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/literal_decimal.q @@ -0,0 +1,3 @@ +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/src/test/hive/ql/src/test/queries/clientpositive/literal_double.q b/src/test/hive/ql/src/test/queries/clientpositive/literal_double.q new file mode 100644 index 0000000000000..08836127b9958 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/literal_double.q @@ -0,0 +1,3 @@ +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/src/test/hive/ql/src/test/queries/clientpositive/literal_ints.q b/src/test/hive/ql/src/test/queries/clientpositive/literal_ints.q new file mode 100644 index 0000000000000..9da622e246793 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/literal_ints.q @@ -0,0 +1,3 @@ +EXPLAIN SELECT 100, 100Y, 100S, 100L FROM src LIMIT 1; + +SELECT 100, 100Y, 100S, 100L FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/literal_string.q b/src/test/hive/ql/src/test/queries/clientpositive/literal_string.q new file mode 100644 index 0000000000000..21f0890ada168 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/literal_string.q @@ -0,0 +1,13 @@ +EXPLAIN SELECT 'face''book', 'face' 'book', 'face' + 'book', + "face""book", "face" "book", "face" + "book", + 'face' 'bo' 'ok', 'face'"book", + "face"'book', 'facebook' FROM src LIMIT 1; + +SELECT 'face''book', 'face' 'book', 'face' + 'book', + "face""book", "face" "book", "face" + "book", + 'face' 'bo' 'ok', 'face'"book", + "face"'book', 'facebook' FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_binary_data.q b/src/test/hive/ql/src/test/queries/clientpositive/load_binary_data.q new file mode 100644 index 0000000000000..7da363183ccf6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_binary_data.q @@ -0,0 +1,13 @@ +CREATE TABLE mytable(key binary, value int) +ROW FORMAT DELIMITED +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; + +create table dest1 (key binary, value int); + +insert overwrite table dest1 select transform(*) using 'cat' as key binary, value int from mytable; + +select key, value, length (key) from dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part1.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part1.q new file mode 100644 index 0000000000000..52b4937d4a9dd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part1.q @@ -0,0 +1,30 @@ +show partitions srcpart; + + + + +create table if not exists nzhang_part1 like srcpart; +create table if not exists nzhang_part2 like srcpart; +describe extended nzhang_part1; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +explain +from srcpart +insert overwrite table nzhang_part1 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08' +insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08'; + +from srcpart +insert overwrite table nzhang_part1 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08' +insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08'; + + +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; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part10.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part10.q new file mode 100644 index 0000000000000..9517664675d69 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part10.q @@ -0,0 +1,24 @@ +show partitions srcpart; + + + +create table if not exists nzhang_part10 like srcpart; +describe extended nzhang_part10; + +set hive.merge.mapfiles=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +explain +from srcpart +insert overwrite table nzhang_part10 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08'; + +from srcpart +insert overwrite table nzhang_part10 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08'; + + +show partitions nzhang_part10; + +select * from nzhang_part10 where ds is not null and hr is not null; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part11.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part11.q new file mode 100644 index 0000000000000..4cfbfe8eac003 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part11.q @@ -0,0 +1,17 @@ +show partitions srcpart; + + +create table if not exists nzhang_part like srcpart; +describe extended nzhang_part; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.exec.compress.output=true; +set hive.exec.dynamic.partition=true; + +insert overwrite table nzhang_part partition (ds="2010-03-03", hr) select key, value, hr from srcpart where ds is not null and hr is not null; + +select * from nzhang_part where ds = '2010-03-03' and hr = '11'; +select * from nzhang_part where ds = '2010-03-03' and hr = '12'; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part12.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part12.q new file mode 100644 index 0000000000000..dd8bd530af43c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part12.q @@ -0,0 +1,19 @@ +show partitions srcpart; + + + +create table if not exists nzhang_part12 like srcpart; +describe extended nzhang_part12; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.exec.dynamic.partition=true; + + +insert overwrite table nzhang_part12 partition (ds="2010-03-03", hr) select key, value, cast(hr*2 as int) from srcpart where ds is not null and hr is not null; + +show partitions nzhang_part12; + +select * from nzhang_part12 where ds is not null and hr is not null; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part13.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part13.q new file mode 100644 index 0000000000000..0b8d44f7cf597 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part13.q @@ -0,0 +1,37 @@ +show partitions srcpart; + + + +create table if not exists nzhang_part13 like srcpart; +describe extended nzhang_part13; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.exec.dynamic.partition=true; + +explain +insert overwrite table nzhang_part13 partition (ds="2010-03-03", hr) +select * from ( + select key, value, '22' + from src + where key < 20 + union all + select key, value, '33' + from src + where key > 20 and key < 40) s; + +insert overwrite table nzhang_part13 partition (ds="2010-03-03", hr) +select * from ( + select key, value, '22' + from src + where key < 20 + union all + select key, value, '33' + from src + where key > 20 and key < 40) s; + +show partitions nzhang_part13; + +select * from nzhang_part13 where ds is not null and hr is not null; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14.q new file mode 100644 index 0000000000000..ba3ce39cf7920 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14.q @@ -0,0 +1,37 @@ +-- EXCLUDE_OS_WINDOWS +-- excluded on windows because of difference in file name encoding logic + +create table if not exists nzhang_part14 (key string) + partitioned by (value string); + +describe extended nzhang_part14; + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +explain +insert overwrite table nzhang_part14 partition(value) +select key, value from ( + select 'k1' as key, cast(null as string) as value from src limit 2 + union all + select 'k2' as key, '' as value from src limit 2 + union all + select 'k3' as key, ' ' as value from src limit 2 +) T; + +insert overwrite table nzhang_part14 partition(value) +select key, value from ( + select 'k1' as key, cast(null as string) as value from src limit 2 + union all + select 'k2' as key, '' as value from src limit 2 + union all + select 'k3' as key, ' ' as value from src limit 2 +) T; + + +show partitions nzhang_part14; + +select * from nzhang_part14 where value <> 'a' +order by key, value; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14_win.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14_win.q new file mode 100644 index 0000000000000..d32b539ae96ab --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14_win.q @@ -0,0 +1,38 @@ +-- INCLUDE_OS_WINDOWS +-- included only on windows because of difference in file name encoding logic + + +create table if not exists nzhang_part14 (key string) + partitioned by (value string); + +describe extended nzhang_part14; + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +explain +insert overwrite table nzhang_part14 partition(value) +select key, value from ( + select 'k1' as key, cast(null as string) as value from src limit 2 + union all + select 'k2' as key, '' as value from src limit 2 + union all + select 'k3' as key, ' ' as value from src limit 2 +) T; + +insert overwrite table nzhang_part14 partition(value) +select key, value from ( + select 'k1' as key, cast(null as string) as value from src limit 2 + union all + select 'k2' as key, '' as value from src limit 2 + union all + select 'k3' as key, ' ' as value from src limit 2 +) T; + + +show partitions nzhang_part14; + +select * from nzhang_part14 where value <> 'a' +order by key, value; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part15.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part15.q new file mode 100644 index 0000000000000..3c3310882a617 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part15.q @@ -0,0 +1,13 @@ + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +create table if not exists load_dyn_part15_test (key string) + partitioned by (part_key string); + +show partitions load_dyn_part15_test; + +INSERT OVERWRITE TABLE load_dyn_part15_test PARTITION(part_key) +SELECT key, part_key FROM src LATERAL VIEW explode(array("1","{2","3]")) myTable AS part_key; + +show partitions load_dyn_part15_test; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part2.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part2.q new file mode 100644 index 0000000000000..03aa5df03501f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part2.q @@ -0,0 +1,23 @@ + +create table if not exists nzhang_part_bucket (key string, value string) + partitioned by (ds string, hr string) + clustered by (key) into 10 buckets; + +describe extended nzhang_part_bucket; + +set hive.merge.mapfiles=false; +set hive.enforce.bucketing=true; +set hive.exec.dynamic.partition=true; + +explain +insert overwrite table nzhang_part_bucket partition (ds='2010-03-23', hr) select key, value, hr from srcpart where ds is not null and hr is not null; + +insert overwrite table nzhang_part_bucket partition (ds='2010-03-23', hr) select key, value, hr from srcpart where ds is not null and hr is not null; + +show partitions nzhang_part_bucket; + +select * from nzhang_part_bucket where ds='2010-03-23' and hr='11' order by key; +select * from nzhang_part_bucket where ds='2010-03-23' and hr='12' order by key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part3.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part3.q new file mode 100644 index 0000000000000..e4c8c17f63048 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part3.q @@ -0,0 +1,19 @@ +show partitions srcpart; + + + +create table if not exists nzhang_part3 like srcpart; +describe extended nzhang_part3; + +set hive.merge.mapfiles=false; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +explain +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; + +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; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part4.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part4.q new file mode 100644 index 0000000000000..3f3a0c8d51b41 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part4.q @@ -0,0 +1,24 @@ +show partitions srcpart; + + + +create table if not exists nzhang_part4 like srcpart; +describe extended nzhang_part4; + +set hive.merge.mapfiles=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +insert overwrite table nzhang_part4 partition (ds='2008-04-08', hr='existing_value') select key, value from src; + +explain +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; + +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 is not null and hr is not null; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part5.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part5.q new file mode 100644 index 0000000000000..5780f5d43911b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part5.q @@ -0,0 +1,22 @@ + + +create table if not exists nzhang_part5 (key string) partitioned by (value string); +describe extended nzhang_part5; + +set hive.merge.mapfiles=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.max.dynamic.partitions=2000; +set hive.exec.max.dynamic.partitions.pernode=2000; + +explain +insert overwrite table nzhang_part5 partition (value) select key, value from src; + +insert overwrite table nzhang_part5 partition (value) select key, value from src; + +show partitions nzhang_part5; + +select * from nzhang_part5 where value='val_0'; +select * from nzhang_part5 where value='val_2'; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part6.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part6.q new file mode 100644 index 0000000000000..b5e85aebf4ade --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part6.q @@ -0,0 +1,16 @@ +show partitions srcpart; + + + +create table if not exists nzhang_part6 like srcpart; +describe extended nzhang_part6; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.exec.dynamic.partition=true; + +insert overwrite table nzhang_part6 partition (ds="2010-03-03", hr) select key, value, hr from srcpart where ds is not null and hr is not null; + +select * from nzhang_part6 where ds = '2010-03-03' and hr = '11'; +select * from nzhang_part6 where ds = '2010-03-03' and hr = '12'; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part7.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part7.q new file mode 100644 index 0000000000000..b9e1da032660b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part7.q @@ -0,0 +1,14 @@ +show partitions srcpart; + + + +create table if not exists nzhang_part7 like srcpart; +describe extended nzhang_part7; + + +insert overwrite table nzhang_part7 partition (ds='2010-03-03', hr='12') select key, value from srcpart where ds = '2008-04-08' and hr = '12'; + +show partitions nzhang_part7; + +select * from nzhang_part7 where ds is not null and hr is not null; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part8.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part8.q new file mode 100644 index 0000000000000..8073500c0bf07 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part8.q @@ -0,0 +1,24 @@ +show partitions srcpart; + + + +create table if not exists nzhang_part8 like srcpart; +describe extended nzhang_part8; + +set hive.merge.mapfiles=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +explain extended +from srcpart +insert overwrite table nzhang_part8 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08' +insert overwrite table nzhang_part8 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08'; + +from srcpart +insert overwrite table nzhang_part8 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08' +insert overwrite table nzhang_part8 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08'; + +show partitions nzhang_part8; + +select * from nzhang_part8 where ds is not null and hr is not null; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part9.q b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part9.q new file mode 100644 index 0000000000000..01fa596cdf04a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part9.q @@ -0,0 +1,23 @@ +show partitions srcpart; + + + +create table if not exists nzhang_part9 like srcpart; +describe extended nzhang_part9; + +set hive.merge.mapfiles=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +explain +from srcpart +insert overwrite table nzhang_part9 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08'; + +from srcpart +insert overwrite table nzhang_part9 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08'; + + +show partitions nzhang_part9; + +select * from nzhang_part9 where ds is not null and hr is not null; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q b/src/test/hive/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q new file mode 100644 index 0000000000000..6d2a8b82d33c1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q @@ -0,0 +1,5 @@ +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; +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'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q b/src/test/hive/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q new file mode 100644 index 0000000000000..3b8951a1a782f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q @@ -0,0 +1,5 @@ +-- test for loading into tables with the file with space in the name + + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_fs.q b/src/test/hive/ql/src/test/queries/clientpositive/load_fs.q new file mode 100644 index 0000000000000..c1ac29c172f60 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_fs.q @@ -0,0 +1,21 @@ + +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; + +show table extended like load_overwrite; +desc extended load_overwrite; +select count(*) from load_overwrite; + +load data inpath '${system:test.tmp.dir}/load_overwrite/kv*.txt' overwrite into table load_overwrite2; + +show table extended like load_overwrite2; +desc extended load_overwrite2; +select count(*) from load_overwrite2; + +load data inpath '${system:test.tmp.dir}/load2_*' overwrite into table load_overwrite; +show table extended like load_overwrite; +select count(*) from load_overwrite; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_fs2.q b/src/test/hive/ql/src/test/queries/clientpositive/load_fs2.q new file mode 100644 index 0000000000000..7255324d1653d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_fs2.q @@ -0,0 +1,20 @@ +-- HIVE-3300 [jira] LOAD DATA INPATH fails if a hdfs file with same name is added to table +-- 'loader' table is used only for uploading kv1.txt to HDFS (!hdfs -put is not working on minMRDriver) + +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 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 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 inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result; +show table extended like result; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q b/src/test/hive/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q new file mode 100644 index 0000000000000..cce297cca46db --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q @@ -0,0 +1,9 @@ +dfs -mkdir hdfs:///tmp/test/; + +dfs -copyFromLocal ../data/files hdfs:///tmp/test/.; + +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; + +dfs -rmr hdfs:///tmp/test; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q b/src/test/hive/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q new file mode 100644 index 0000000000000..40d8210e57d2a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q @@ -0,0 +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 ; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_overwrite.q b/src/test/hive/ql/src/test/queries/clientpositive/load_overwrite.q new file mode 100644 index 0000000000000..73853f15a94bb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_overwrite.q @@ -0,0 +1,15 @@ +create table load_overwrite like src; + +insert overwrite table load_overwrite select * from src; +show table extended like load_overwrite; +select count(*) from 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; +show table extended like load_overwrite; +select count(*) from load_overwrite; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_part_authsuccess.q b/src/test/hive/ql/src/test/queries/clientpositive/load_part_authsuccess.q new file mode 100644 index 0000000000000..ff54324a5a50c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/load_part_authsuccess.q @@ -0,0 +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'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/loadpart1.q b/src/test/hive/ql/src/test/queries/clientpositive/loadpart1.q new file mode 100644 index 0000000000000..0813bb23c3746 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/loadpart1.q @@ -0,0 +1,19 @@ + + + +create table hive_test_src ( col1 string ) stored as textfile ; +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 ; +select * from hive_test_dst where pcol1='test_part' and pcol2='test_Part'; + +insert overwrite table hive_test_dst partition ( pCol1='test_part', pcol2='test_Part') select col1 from hive_test_src ; +select * from hive_test_dst where pcol1='test_part' and pcol2='test_part'; + +select * from hive_test_dst where pcol1='test_part'; +select * from hive_test_dst where pcol1='test_part' and pcol2='test_part'; +select * from hive_test_dst where pcol1='test_Part'; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/loadpart_err.q b/src/test/hive/ql/src/test/queries/clientpositive/loadpart_err.q new file mode 100644 index 0000000000000..6e4df215479bb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/loadpart_err.q @@ -0,0 +1,20 @@ +set hive.cli.errors.ignore=true; + +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) + +CREATE TABLE loadpart1(a STRING, b STRING) PARTITIONED BY (ds STRING); + +INSERT OVERWRITE TABLE loadpart1 PARTITION (ds='2009-01-01') +SELECT TRANSFORM(src.key, src.value) USING 'error_script' AS (tkey, tvalue) +FROM src; + +DESCRIBE loadpart1; +SHOW PARTITIONS loadpart1; + +LOAD DATA LOCAL INPATH '../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05'); +SHOW PARTITIONS loadpart1; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lock1.q b/src/test/hive/ql/src/test/queries/clientpositive/lock1.q new file mode 100644 index 0000000000000..326cf4910fb49 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/lock1.q @@ -0,0 +1,29 @@ +drop table tstsrc; +create table tstsrc like src; +insert overwrite table tstsrc select key, value from src; + +SHOW LOCKS; +SHOW LOCKS tstsrc; + +LOCK TABLE tstsrc shared; +SHOW LOCKS; +SHOW LOCKS tstsrc; +SHOW LOCKS tstsrc extended; + +UNLOCK TABLE tstsrc; +SHOW LOCKS; +SHOW LOCKS extended; +SHOW LOCKS tstsrc; +lock TABLE tstsrc SHARED; +SHOW LOCKS; +SHOW LOCKS extended; +SHOW LOCKS tstsrc; +LOCK TABLE tstsrc SHARED; +SHOW LOCKS; +SHOW LOCKS extended; +SHOW LOCKS tstsrc; +UNLOCK TABLE tstsrc; +SHOW LOCKS; +SHOW LOCKS extended; +SHOW LOCKS tstsrc; +drop table tstsrc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lock2.q b/src/test/hive/ql/src/test/queries/clientpositive/lock2.q new file mode 100644 index 0000000000000..c8bff56adabe9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/lock2.q @@ -0,0 +1,36 @@ +drop table tstsrc; +create table tstsrc like src; +insert overwrite table tstsrc select key, value from src; + +drop table tstsrcpart; +create table tstsrcpart like srcpart; + +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') +select key, value from srcpart where ds='2008-04-08' and hr='11'; + +LOCK TABLE tstsrc SHARED; +LOCK TABLE tstsrcpart SHARED; +LOCK TABLE tstsrcpart PARTITION(ds='2008-04-08', hr='11') EXCLUSIVE; +SHOW LOCKS; +SHOW LOCKS tstsrcpart; +SHOW LOCKS tstsrcpart PARTITION(ds='2008-04-08', hr='11'); +SHOW LOCKS tstsrcpart PARTITION(ds='2008-04-08', hr='11') extended; + +UNLOCK TABLE tstsrc; +SHOW LOCKS; +SHOW LOCKS tstsrcpart; +SHOW LOCKS tstsrcpart PARTITION(ds='2008-04-08', hr='11'); + +UNLOCK TABLE tstsrcpart; +SHOW LOCKS; +SHOW LOCKS tstsrcpart; +SHOW LOCKS tstsrcpart PARTITION(ds='2008-04-08', hr='11'); + +UNLOCK TABLE tstsrcpart PARTITION(ds='2008-04-08', hr='11'); +SHOW LOCKS; +SHOW LOCKS tstsrcpart; +SHOW LOCKS tstsrcpart PARTITION(ds='2008-04-08', hr='11'); + + +drop table tstsrc; +drop table tstsrcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lock3.q b/src/test/hive/ql/src/test/queries/clientpositive/lock3.q new file mode 100644 index 0000000000000..7f2178f5408d6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/lock3.q @@ -0,0 +1,32 @@ +drop table tstsrcpart; +create table tstsrcpart like srcpart; + +from srcpart +insert overwrite table tstsrcpart partition (ds='2008-04-08',hr='11') +select key, value where ds='2008-04-08' and hr='11'; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + + +from srcpart +insert overwrite table tstsrcpart partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08'; + +from srcpart +insert overwrite table tstsrcpart partition (ds ='2008-04-08', hr) select key, value, hr where ds = '2008-04-08'; + + +SHOW LOCKS; +SHOW LOCKS tstsrcpart; + +drop table tstsrcpart; + +drop table tst1; +create table tst1 (key string, value string) partitioned by (a string, b string, c string, d string); + + +from srcpart +insert overwrite table tst1 partition (a='1', b='2', c, d) select key, value, ds, hr where ds = '2008-04-08'; + + +drop table tst1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lock4.q b/src/test/hive/ql/src/test/queries/clientpositive/lock4.q new file mode 100644 index 0000000000000..e07bfe867d03f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/lock4.q @@ -0,0 +1,33 @@ +set hive.lock.mapred.only.operation=true; +drop table tstsrcpart; +create table tstsrcpart like srcpart; + +from srcpart +insert overwrite table tstsrcpart partition (ds='2008-04-08',hr='11') +select key, value where ds='2008-04-08' and hr='11'; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + + +from srcpart +insert overwrite table tstsrcpart partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08'; + +from srcpart +insert overwrite table tstsrcpart partition (ds ='2008-04-08', hr) select key, value, hr where ds = '2008-04-08'; + + +SHOW LOCKS; +SHOW LOCKS tstsrcpart; + +drop table tstsrcpart; + +drop table tst1; +create table tst1 (key string, value string) partitioned by (a string, b string, c string, d string); + + +from srcpart +insert overwrite table tst1 partition (a='1', b='2', c, d) select key, value, ds, hr where ds = '2008-04-08'; + + +drop table tst1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/louter_join_ppr.q b/src/test/hive/ql/src/test/queries/clientpositive/louter_join_ppr.q new file mode 100644 index 0000000000000..c4e25eb470196 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/louter_join_ppr.q @@ -0,0 +1,71 @@ +set hive.optimize.ppd=true; + +EXPLAIN EXTENDED + FROM + src a + LEFT OUTER JOIN + srcpart b + ON (a.key = b.key AND b.ds = '2008-04-08') + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + + FROM + src a + LEFT OUTER JOIN + srcpart b + ON (a.key = b.key AND b.ds = '2008-04-08') + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + +EXPLAIN EXTENDED + FROM + srcpart a + LEFT OUTER JOIN + src b + ON (a.key = b.key AND a.ds = '2008-04-08') + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + + FROM + srcpart a + LEFT OUTER JOIN + src b + ON (a.key = b.key AND a.ds = '2008-04-08') + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + + +EXPLAIN EXTENDED + FROM + src a + LEFT OUTER JOIN + srcpart b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25 AND b.ds = '2008-04-08'; + + FROM + src a + LEFT OUTER JOIN + srcpart b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25 AND b.ds = '2008-04-08'; + +EXPLAIN EXTENDED + FROM + srcpart a + LEFT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25 AND a.ds = '2008-04-08'; + + FROM + srcpart a + LEFT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25 AND a.ds = '2008-04-08'; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/macro.q b/src/test/hive/ql/src/test/queries/clientpositive/macro.q new file mode 100644 index 0000000000000..fd0f7f2b0cdd4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/macro.q @@ -0,0 +1,26 @@ +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; +EXPLAIN EXTENDED SELECT SIGMOID(2) FROM src LIMIT 1; +DROP TEMPORARY MACRO SIGMOID; + +CREATE TEMPORARY MACRO FIXED_NUMBER() 1; +SELECT FIXED_NUMBER() + 1 FROM src LIMIT 1; +EXPLAIN SELECT FIXED_NUMBER() + 1 FROM src LIMIT 1; +EXPLAIN EXTENDED SELECT FIXED_NUMBER() + 1 FROM src LIMIT 1; +DROP TEMPORARY MACRO FIXED_NUMBER; + +set macrotest=1; +CREATE TEMPORARY MACRO CONF_TEST() "${hiveconf:macrotest}"; +SELECT CONF_TEST() FROM src LIMIT 1; +DROP TEMPORARY MACRO CONF_TEST; + +CREATE TEMPORARY MACRO SIMPLE_ADD (x INT, y INT) x + y; +CREATE TEMPORARY MACRO SIMPLE_ADD (x INT, y INT) x + y; +SELECT SIMPLE_ADD(1, 9) FROM src LIMIT 1; +EXPLAIN SELECT SIMPLE_ADD(1, 9) FROM src LIMIT 1; +EXPLAIN EXTENDED SELECT SIMPLE_ADD(1, 9) FROM src LIMIT 1; +DROP TEMPORARY MACRO SIMPLE_ADD; +DROP TEMPORARY MACRO SIMPLE_ADD; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin1.q b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin1.q new file mode 100644 index 0000000000000..49de907948f6d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin1.q @@ -0,0 +1,38 @@ +SELECT /*+ MAPJOIN(b) */ sum(a.key) as sum_a + FROM srcpart a + JOIN src b ON a.key = b.key where a.ds is not null; + +set hive.outerjoin.supports.filters=true; + +-- const filter on outer join +EXPLAIN +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN src b on a.key=b.key AND true limit 10; +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN src b on a.key=b.key AND true limit 10; + +-- func filter on outer join +EXPLAIN +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN src b on a.key=b.key AND b.key * 10 < '1000' limit 10; +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN src b on a.key=b.key AND b.key * 10 < '1000' limit 10; + +-- field filter on outer join +EXPLAIN +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN + (select key, named_struct('key', key, 'value', value) as kv from src) b on a.key=b.key AND b.kv.key > 200 limit 10; +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN + (select key, named_struct('key', key, 'value', value) as kv from src) b on a.key=b.key AND b.kv.key > 200 limit 10; + +set hive.outerjoin.supports.filters=false; + +EXPLAIN +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN src b on a.key=b.key AND true limit 10; +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN src b on a.key=b.key AND true limit 10; + +EXPLAIN +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN src b on a.key=b.key AND b.key * 10 < '1000' limit 10; +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN src b on a.key=b.key AND b.key * 10 < '1000' limit 10; + +EXPLAIN +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN + (select key, named_struct('key', key, 'value', value) as kv from src) b on a.key=b.key AND b.kv.key > 200 limit 10; +SELECT /*+ MAPJOIN(a) */ * FROM src a RIGHT OUTER JOIN + (select key, named_struct('key', key, 'value', value) as kv from src) b on a.key=b.key AND b.kv.key > 200 limit 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_distinct.q new file mode 100644 index 0000000000000..d021c631d1ea5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_distinct.q @@ -0,0 +1,55 @@ +set hive.map.aggr = true; +set hive.groupby.skewindata = true; +explain +FROM srcpart c +JOIN srcpart d +ON ( c.key=d.key AND c.ds='2008-04-08' AND d.ds='2008-04-08') +SELECT /*+ MAPJOIN(d) */ DISTINCT c.value; + +FROM srcpart c +JOIN srcpart d +ON ( c.key=d.key AND c.ds='2008-04-08' AND d.ds='2008-04-08') +SELECT /*+ MAPJOIN(d) */ DISTINCT c.value as value order by value limit 10; + +set hive.map.aggr = true; +set hive.groupby.skewindata = false; +explain +FROM srcpart c +JOIN srcpart d +ON ( c.key=d.key AND c.ds='2008-04-08' AND d.ds='2008-04-08') +SELECT /*+ MAPJOIN(d) */ DISTINCT c.value; + +FROM srcpart c +JOIN srcpart d +ON ( c.key=d.key AND c.ds='2008-04-08' AND d.ds='2008-04-08') +SELECT /*+ MAPJOIN(d) */ DISTINCT c.value as value order by value limit 10; + + +set hive.map.aggr = false; +set hive.groupby.skewindata = true; +explain +FROM srcpart c +JOIN srcpart d +ON ( c.key=d.key AND c.ds='2008-04-08' AND d.ds='2008-04-08') +SELECT /*+ MAPJOIN(d) */ DISTINCT c.value; + +FROM srcpart c +JOIN srcpart d +ON ( c.key=d.key AND c.ds='2008-04-08' AND d.ds='2008-04-08') +SELECT /*+ MAPJOIN(d) */ DISTINCT c.value as value order by value limit 10; + + +set hive.map.aggr = false; +set hive.groupby.skewindata = false; +explain +FROM srcpart c +JOIN srcpart d +ON ( c.key=d.key AND c.ds='2008-04-08' AND d.ds='2008-04-08') +SELECT /*+ MAPJOIN(d) */ DISTINCT c.value; + +FROM srcpart c +JOIN srcpart d +ON ( c.key=d.key AND c.ds='2008-04-08' AND d.ds='2008-04-08') +SELECT /*+ MAPJOIN(d) */ DISTINCT c.value as value order by value limit 10; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q new file mode 100644 index 0000000000000..a43d298a4ed57 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q @@ -0,0 +1,31 @@ +set hive.auto.convert.join = false; +--HIVE-2101 mapjoin sometimes gives wrong results if there is a filter in the on condition + +SELECT * FROM src1 + RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) + JOIN src src3 ON (src2.key = src3.key AND src3.key < 300) + SORT BY src1.key, src2.key, src3.key; + +explain +SELECT /*+ mapjoin(src1, src2) */ * FROM src1 + RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) + JOIN src src3 ON (src2.key = src3.key AND src3.key < 300) + SORT BY src1.key, src2.key, src3.key; + +SELECT /*+ mapjoin(src1, src2) */ * FROM src1 + RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) + JOIN src src3 ON (src2.key = src3.key AND src3.key < 300) + SORT BY src1.key, src2.key, src3.key; + +set hive.auto.convert.join = true; + +explain +SELECT * FROM src1 + RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) + JOIN src src3 ON (src2.key = src3.key AND src3.key < 300) + SORT BY src1.key, src2.key, src3.key; + +SELECT * FROM src1 + RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) + JOIN src src3 ON (src2.key = src3.key AND src3.key < 300) + SORT BY src1.key, src2.key, src3.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_hook.q b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_hook.q new file mode 100644 index 0000000000000..d6811d493263f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_hook.q @@ -0,0 +1,31 @@ +set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.MapJoinCounterHook ; +drop table dest1; +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +set hive.auto.convert.join = true; + +INSERT OVERWRITE TABLE dest1 +SELECT /*+ MAPJOIN(x) */ x.key, count(1) FROM src1 x JOIN src y ON (x.key = y.key) group by x.key; + + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value; + + + +set hive.mapjoin.localtask.max.memory.usage = 0.0001; +set hive.mapjoin.check.memory.rows = 2; +set hive.auto.convert.join.noconditionaltask = false; + + +FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value +where (src1.ds = '2008-04-08' or src1.ds = '2008-04-09' )and (src1.hr = '12' or src1.hr = '11'); + + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value; + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q new file mode 100644 index 0000000000000..3f87db28ed2e2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q @@ -0,0 +1,11 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +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; + +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/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery.q b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery.q new file mode 100644 index 0000000000000..81cf940bd01c3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery.q @@ -0,0 +1,34 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +EXPLAIN +SELECT subq.key1, z.value +FROM +(SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 + FROM src1 x JOIN src y ON (x.key = y.key)) subq + JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11); + +SELECT subq.key1, z.value +FROM +(SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 + FROM src1 x JOIN src y ON (x.key = y.key)) subq + JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11) +ORDER BY subq.key1, z.value; + +EXPLAIN +SELECT subq.key1, z.value +FROM +(SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 + FROM src1 x JOIN src y ON (x.key = y.key)) subq + JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11) + order by subq.key1, z.value; + +SELECT subq.key1, z.value +FROM +(SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 + FROM src1 x JOIN src y ON (x.key = y.key)) subq + JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11) + order by subq.key1, z.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery2.q b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery2.q new file mode 100644 index 0000000000000..9980946057fe1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery2.q @@ -0,0 +1,39 @@ +drop table x; +drop table y; +drop table z; + +CREATE TABLE x (name STRING, id INT) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; + +CREATE TABLE y (id INT, name STRING) +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; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +EXPLAIN +SELECT subq.key1, subq.value1, subq.key2, subq.value2, z.id, z.name +FROM +(SELECT x.id as key1, x.name as value1, y.id as key2, y.name as value2 + FROM y JOIN x ON (x.id = y.id)) subq + JOIN z ON (subq.key1 = z.id); + +SELECT subq.key1, subq.value1, subq.key2, subq.value2, z.id, z.name +FROM +(SELECT x.id as key1, x.name as value1, y.id as key2, y.name as value2 + FROM y JOIN x ON (x.id = y.id)) subq + JOIN z ON (subq.key1 = z.id); + +drop table x; +drop table y; +drop table z; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_test_outer.q b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_test_outer.q new file mode 100644 index 0000000000000..8d31590186514 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_test_outer.q @@ -0,0 +1,54 @@ +set hive.auto.convert.join = false; +--HIVE-2101 mapjoin sometimes gives wrong results if there is a filter in the on condition + +create table dest_1 (key STRING, value STRING) stored as textfile; +insert overwrite table dest_1 select * from src1 order by src1.value limit 8; +insert into table dest_1 select "333444","555666" from src1 limit 1; + +create table dest_2 (key STRING, value STRING) stored as textfile; + +insert into table dest_2 select * from dest_1; + +SELECT * FROM src1 + RIGHT OUTER JOIN dest_1 src2 ON (src1.key = src2.key) + JOIN dest_2 src3 ON (src2.key = src3.key) + SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT /*+ mapjoin(src1, src2) */ * FROM src1 + RIGHT OUTER JOIN dest_1 src2 ON (src1.key = src2.key) + JOIN dest_2 src3 ON (src2.key = src3.key) + SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT /*+ mapjoin(src1, src2) */ * FROM src1 + RIGHT OUTER JOIN dest_1 src2 ON (src1.key = src2.key) + JOIN dest_2 src3 ON (src2.key = src3.key) + SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT /*+ mapjoin(src1, src2) */ * FROM src1 + RIGHT OUTER JOIN dest_1 src2 ON (src1.key = src2.key) + JOIN dest_2 src3 ON (src1.key = src3.key) + SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +set hive.auto.convert.join = true; + +SELECT * FROM src1 + LEFT OUTER JOIN dest_1 src2 ON (src1.key = src2.key) + JOIN dest_2 src3 ON (src1.key = src3.key) + SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src1 + LEFT OUTER JOIN dest_1 src2 ON (src1.key = src2.key) + JOIN dest_2 src3 ON (src2.key = src3.key) + SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +explain +SELECT * FROM src1 + RIGHT OUTER JOIN dest_1 src2 ON (src1.key = src2.key) + JOIN dest_2 src3 ON (src2.key = src3.key) + SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; + +SELECT * FROM src1 + RIGHT OUTER JOIN dest_1 src2 ON (src1.key = src2.key) + JOIN dest_2 src3 ON (src2.key = src3.key) + SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce1.q b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce1.q new file mode 100644 index 0000000000000..83328f1f83cfc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce1.q @@ -0,0 +1,20 @@ +CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +DISTRIBUTE BY tvalue, tkey +SORT BY ten, one; + + +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +DISTRIBUTE BY tvalue, tkey +SORT BY ten, one; + + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce2.q b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce2.q new file mode 100644 index 0000000000000..ef6f8e5766ffa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce2.q @@ -0,0 +1,17 @@ +CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +DISTRIBUTE BY tvalue, tkey; + + +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +DISTRIBUTE BY tvalue, tkey; + +SELECT * FROM (SELECT dest1.* FROM dest1 DISTRIBUTE BY key SORT BY key, ten, one, value) T; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce3.q b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce3.q new file mode 100644 index 0000000000000..c31ede77d8718 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce3.q @@ -0,0 +1,17 @@ +CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +SORT BY tvalue, tkey; + + +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +SORT BY tvalue, tkey; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce4.q b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce4.q new file mode 100644 index 0000000000000..0c15f1fb55174 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce4.q @@ -0,0 +1,20 @@ +CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +DISTRIBUTE BY tvalue, tkey +SORT BY ten DESC, one ASC; + + +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (tkey, ten, one, tvalue) +DISTRIBUTE BY tvalue, tkey +SORT BY ten DESC, one ASC; + + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce5.q b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce5.q new file mode 100644 index 0000000000000..d7c53d2889c0e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce5.q @@ -0,0 +1,17 @@ +CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 +SELECT src.key as c1, CAST(src.key / 10 AS INT) as c2, CAST(src.key % 10 AS INT) as c3, src.value as c4 +DISTRIBUTE BY c4, c1 +SORT BY c2 DESC, c3 ASC; + + +FROM src +INSERT OVERWRITE TABLE dest1 +SELECT src.key as c1, CAST(src.key / 10 AS INT) as c2, CAST(src.key % 10 AS INT) as c3, src.value as c4 +DISTRIBUTE BY c4, c1 +SORT BY c2 DESC, c3 ASC; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce6.q b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce6.q new file mode 100644 index 0000000000000..c37181d351068 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce6.q @@ -0,0 +1,17 @@ +CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 +SELECT src.key, CAST(src.key / 10 AS INT) as c2, CAST(src.key % 10 AS INT) as c3, src.value +DISTRIBUTE BY value, key +SORT BY c2 DESC, c3 ASC; + + +FROM src +INSERT OVERWRITE TABLE dest1 +SELECT src.key, CAST(src.key / 10 AS INT) as c2, CAST(src.key % 10 AS INT) as c3, src.value +DISTRIBUTE BY value, key +SORT BY c2 DESC, c3 ASC; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce7.q b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce7.q new file mode 100644 index 0000000000000..0da41d2ecde53 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce7.q @@ -0,0 +1,17 @@ +CREATE TABLE dest1(k STRING, v STRING, key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.*, src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (k, v, tkey, ten, one, tvalue) +SORT BY tvalue, tkey; + + +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.*, src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (k, v, tkey, ten, one, tvalue) +SORT BY tvalue, tkey; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce8.q b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce8.q new file mode 100644 index 0000000000000..35d9ed90e77e5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mapreduce8.q @@ -0,0 +1,19 @@ +CREATE TABLE dest1(k STRING, v STRING, key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.*, src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (k, v, tkey, ten, one, tvalue) +DISTRIBUTE BY rand(3) +SORT BY tvalue, tkey; + + +FROM src +INSERT OVERWRITE TABLE dest1 +MAP src.*, src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value +USING 'cat' AS (k, v, tkey, ten, one, tvalue) +DISTRIBUTE BY rand(3) +SORT BY tvalue, tkey; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge1.q b/src/test/hive/ql/src/test/queries/clientpositive/merge1.q new file mode 100644 index 0000000000000..834f2ce374608 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/merge1.q @@ -0,0 +1,29 @@ +set hive.merge.mapredfiles=true; + +create table dest1(key int, val int); + +explain +insert overwrite table dest1 +select key, count(1) from src group by key; + +insert overwrite table dest1 +select key, count(1) from src group by key; + +select * from dest1; + +drop table dest1; + +create table test_src(key string, value string) partitioned by (ds string); +create table dest1(key string); + +insert overwrite table test_src partition(ds='101') select * from src; +insert overwrite table test_src partition(ds='102') select * from src; + +explain +insert overwrite table dest1 select key from test_src; +insert overwrite table dest1 select key from test_src; + +set hive.merge.smallfiles.avgsize=16; +explain +insert overwrite table dest1 select key from test_src; +insert overwrite table dest1 select key from test_src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge2.q b/src/test/hive/ql/src/test/queries/clientpositive/merge2.q new file mode 100644 index 0000000000000..8b77bd2fe19ba --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/merge2.q @@ -0,0 +1,35 @@ +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set mapred.min.split.size=256; +set mapred.min.split.size.per.node=256; +set mapred.min.split.size.per.rack=256; +set mapred.max.split.size=256; + +create table test1(key int, val int); + +explain +insert overwrite table test1 +select key, count(1) from src group by key; + +insert overwrite table test1 +select key, count(1) from src group by key; + +select * from test1; + +drop table test1; + + +create table test_src(key string, value string) partitioned by (ds string); +create table test1(key string); + +insert overwrite table test_src partition(ds='101') select * from src; +insert overwrite table test_src partition(ds='102') select * from src; + +explain +insert overwrite table test1 select key from test_src; +insert overwrite table test1 select key from test_src; + +set hive.merge.smallfiles.avgsize=16; +explain +insert overwrite table test1 select key from test_src; +insert overwrite table test1 select key from test_src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge3.q b/src/test/hive/ql/src/test/queries/clientpositive/merge3.q new file mode 100644 index 0000000000000..aacd0cd68fc50 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/merge3.q @@ -0,0 +1,57 @@ +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +create table merge_src as +select key, value from srcpart where ds is not null; + +create table merge_src_part (key string, value string) partitioned by (ds string); +insert overwrite table merge_src_part partition(ds) select key, value, ds from srcpart where ds is not null; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain extended +create table merge_src2 as +select key, value from merge_src; + +create table merge_src2 as +select key, value from merge_src; + +select * from merge_src2 ORDER BY key ASC, value ASC; +describe formatted merge_src2; + +create table merge_src_part2 like merge_src_part; + + +explain extended +insert overwrite table merge_src_part2 partition(ds) +select key, value, ds from merge_src_part +where ds is not null; + +insert overwrite table merge_src_part2 partition(ds) +select key, value, ds from merge_src_part +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; + +drop table merge_src_part2; + +create table merge_src_part2 like merge_src_part; + +explain extended +from (select * from merge_src_part where ds is not null distribute by ds) s +insert overwrite table merge_src_part2 partition(ds) +select key, value, ds; + +from (select * from merge_src_part where ds is not null distribute by ds) s +insert overwrite table merge_src_part2 partition(ds) +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge4.q b/src/test/hive/ql/src/test/queries/clientpositive/merge4.q new file mode 100644 index 0000000000000..744783bd62f8d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/merge4.q @@ -0,0 +1,40 @@ +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +create table nzhang_part like srcpart; + +explain +insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, value, hr from srcpart where ds='2008-04-08'; + +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; + +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; + +explain +insert overwrite table nzhang_part partition (ds='2010-08-15', hr) +select * from ( + select key, value, hr from srcpart where ds='2008-04-08' + union all + select '1' as key, '1' as value, 'file,' as hr from src limit 1) s; + +insert overwrite table nzhang_part partition (ds='2010-08-15', hr) +select * from ( + select key, value, hr from srcpart where ds='2008-04-08' + union all + select '1' as key, '1' as value, 'file,' as hr from src limit 1) s; + +show partitions nzhang_part; + +select * from nzhang_part where hr = 'file,'; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition.q b/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition.q new file mode 100644 index 0000000000000..1379426b601f0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition.q @@ -0,0 +1,46 @@ +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +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); + +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; +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'; + +select * from merge_dynamic_part order by key, value; +show table extended like `merge_dynamic_part`; + + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1000000000; +explain +insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr=11) select key, value from srcpart_merge_dp where ds='2008-04-08'; +insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr=11) select key, value from srcpart_merge_dp where ds='2008-04-08'; + +select * from merge_dynamic_part order by key, value; +show table extended like `merge_dynamic_part`; + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1000000000; +explain +insert overwrite table merge_dynamic_part partition (ds, hr) select key, value, ds, hr from srcpart_merge_dp where ds='2008-04-08' and hr=11; +insert overwrite table merge_dynamic_part partition (ds, hr) select key, value, ds, hr from srcpart_merge_dp where ds='2008-04-08' and hr=11;; + +select * from merge_dynamic_part order by key, value; +show table extended like `merge_dynamic_part`; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q b/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q new file mode 100644 index 0000000000000..b51c70ed03a46 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q @@ -0,0 +1,27 @@ +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +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); + + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=3000; +set hive.exec.compress.output=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'; + +show table extended like `merge_dynamic_part`; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q b/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q new file mode 100644 index 0000000000000..b3bcf01ea043d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q @@ -0,0 +1,38 @@ +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +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); + +show partitions srcpart_merge_dp; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=3000; +set hive.exec.compress.output=false; + +explain +insert overwrite table merge_dynamic_part partition (ds, hr) select key, value, ds, hr from srcpart_merge_dp where ds>='2008-04-08'; + +insert overwrite table merge_dynamic_part partition (ds, hr) select key, value, ds, hr from srcpart_merge_dp where ds>='2008-04-08'; + +select ds, hr, count(1) from merge_dynamic_part where ds>='2008-04-08' group by ds, hr order by ds, hr; + +show table extended like `merge_dynamic_part`; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q b/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q new file mode 100644 index 0000000000000..ef769a042d7ce --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q @@ -0,0 +1,42 @@ +-- this test verifies that the block merge task that can follow a query to generate dynamic +-- partitions does not produce incorrect results by dropping partitions + +create table srcpart_merge_dp like srcpart; + +create table srcpart_merge_dp_rc like srcpart; +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); + +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'; + +set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=10000000000000; +set hive.exec.compress.output=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +explain +insert overwrite table merge_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 2 == 0, 'a1', 'b1') as hr from srcpart_merge_dp_rc where ds = '2008-04-08'; + +insert overwrite table merge_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 2 == 0, 'a1', 'b1') as hr from srcpart_merge_dp_rc where ds = '2008-04-08'; + +show partitions merge_dynamic_part; + +select count(*) from merge_dynamic_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q b/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q new file mode 100644 index 0000000000000..a196fa05288b7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q @@ -0,0 +1,38 @@ +-- this is to test the case where some dynamic partitions are merged and some are moved + +create table srcpart_merge_dp like srcpart; + +create table srcpart_merge_dp_rc like srcpart; +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); + +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'; + +set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=200; +set hive.exec.compress.output=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +explain +insert overwrite table merge_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') as hr from srcpart_merge_dp_rc where ds = '2008-04-08'; + +insert overwrite table merge_dynamic_part partition (ds = '2008-04-08', hr) +select key, value, if(key % 100 == 0, 'a1', 'b1') as hr from srcpart_merge_dp_rc where ds = '2008-04-08'; + +show partitions merge_dynamic_part; + +select count(*) from merge_dynamic_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mergejoins.q b/src/test/hive/ql/src/test/queries/clientpositive/mergejoins.q new file mode 100644 index 0000000000000..2857c8612aeb0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mergejoins.q @@ -0,0 +1,10 @@ +create table a (val1 int, val2 int); +create table b (val1 int, val2 int); +create table c (val1 int, val2 int); +create table d (val1 int, val2 int); +create table e (val1 int, val2 int); + +explain select * from a join b on a.val1=b.val1 join c on a.val1=c.val1 join d on a.val1=d.val1 join e on a.val2=e.val2; + +--HIVE-3070 filter on outer join condition removed while merging join tree +explain select * from src a join src b on a.key=b.key left outer join src c on b.key=c.key and b.key<10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mergejoins_mixed.q b/src/test/hive/ql/src/test/queries/clientpositive/mergejoins_mixed.q new file mode 100644 index 0000000000000..a7df4e3d2dc65 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mergejoins_mixed.q @@ -0,0 +1,43 @@ +-- HIVE-3464 + +create table a (key string, value string); + +-- (a-b-c-d) +explain +select * from a join a b on (a.key=b.key) left outer join a c on (b.key=c.key) left outer join a d on (a.key=d.key); + +explain +select * from a join a b on (a.key=b.key) left outer join a c on (b.key=c.key) right outer join a d on (a.key=d.key); + +explain +select * from a join a b on (a.key=b.key) right outer join a c on (b.key=c.key) left outer join a d on (a.key=d.key); + +explain +select * from a join a b on (a.key=b.key) right outer join a c on (b.key=c.key) right outer join a d on (a.key=d.key); + +-- ((a-b-d)-c) (reordered) +explain +select * from a join a b on (a.key=b.key) left outer join a c on (b.value=c.key) left outer join a d on (a.key=d.key); + +explain +select * from a join a b on (a.key=b.key) right outer join a c on (b.value=c.key) right outer join a d on (a.key=d.key); + +explain +select * from a join a b on (a.key=b.key) full outer join a c on (b.value=c.key) full outer join a d on (a.key=d.key); + +-- (((a-b)-c)-d) +explain +select * from a join a b on (a.key=b.key) left outer join a c on (b.value=c.key) right outer join a d on (a.key=d.key); + +explain +select * from a join a b on (a.key=b.key) left outer join a c on (b.value=c.key) full outer join a d on (a.key=d.key); + +explain +select * from a join a b on (a.key=b.key) right outer join a c on (b.value=c.key) left outer join a d on (a.key=d.key); + +explain +select * from a join a b on (a.key=b.key) right outer join a c on (b.value=c.key) full outer join a d on (a.key=d.key); + +-- ((a-b)-c-d) +explain +select * from a join a b on (a.key=b.key) left outer join a c on (b.value=c.key) left outer join a d on (c.key=d.key); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/metadata_export_drop.q b/src/test/hive/ql/src/test/queries/clientpositive/metadata_export_drop.q new file mode 100644 index 0000000000000..41be152e7871d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/metadata_export_drop.q @@ -0,0 +1,8 @@ +create table tmp_meta_export_listener_drop_test (foo string); +dfs ${system:test.dfs.mkdir} ../build/ql/test/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.move.exported.metadata.to.trash=false; +drop table tmp_meta_export_listener_drop_test; +dfs -rmr ../build/ql/test/data/exports/HIVE-3427; +set hive.metastore.pre.event.listeners=; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/metadataonly1.q b/src/test/hive/ql/src/test/queries/clientpositive/metadataonly1.q new file mode 100644 index 0000000000000..eb595298497f1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/metadataonly1.q @@ -0,0 +1,45 @@ +CREATE TABLE TEST1(A INT, B DOUBLE) partitioned by (ds string); +explain extended select max(ds) from TEST1; +select max(ds) from TEST1; + +alter table TEST1 add partition (ds='1'); +explain extended select max(ds) from TEST1; +select max(ds) from TEST1; + +explain extended select count(distinct ds) from TEST1; +select count(distinct ds) from TEST1; + +explain extended select count(ds) from TEST1; +select count(ds) from TEST1; + +alter table TEST1 add partition (ds='2'); +explain extended +select count(*) from TEST1 a2 join (select max(ds) m from TEST1) b on a2.ds=b.m; +select count(*) from TEST1 a2 join (select max(ds) m from TEST1) b on a2.ds=b.m; + + +CREATE TABLE TEST2(A INT, B DOUBLE) partitioned by (ds string, hr string); +alter table TEST2 add partition (ds='1', hr='1'); +alter table TEST2 add partition (ds='1', hr='2'); +alter table TEST2 add partition (ds='1', hr='3'); + +explain extended select ds, count(distinct hr) from TEST2 group by ds; +select ds, count(distinct hr) from TEST2 group by ds; + +explain extended select ds, count(hr) from TEST2 group by ds; +select ds, count(hr) from TEST2 group by ds; + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +explain extended select max(ds) from TEST1; +select max(ds) from TEST1; + +select distinct ds from srcpart; +select min(ds),max(ds) from srcpart; + +-- HIVE-3594 URI encoding for temporary path +alter table TEST2 add partition (ds='01:10:10', hr='01'); +alter table TEST2 add partition (ds='01:10:20', hr='02'); + +explain extended select ds, count(distinct hr) from TEST2 group by ds; +select ds, count(distinct hr) from TEST2 group by ds; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mi.q b/src/test/hive/ql/src/test/queries/clientpositive/mi.q new file mode 100644 index 0000000000000..067c143c0e684 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/mi.q @@ -0,0 +1,21 @@ +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +create table nzhang_t1 like srcpart; +create table nzhang_t2 like srcpart; + +FROM srcpart +INSERT OVERWRITE TABLE nzhang_t1 PARTITION (ds, hr) +SELECT key, value, ds, hr +WHERE ds = '2008-04-08' AND hr = '11' +INSERT OVERWRITE TABLE nzhang_t2 PARTITION (ds, hr) +SELECT key, value, ds, hr +WHERE ds = '2008-04-08' and hr = '12' +GROUP BY key, value, ds, hr; + +show partitions nzhang_t1; +show partitions nzhang_t2; + +select * from nzhang_t1; +select * from nzhang_t2; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/misc_json.q b/src/test/hive/ql/src/test/queries/clientpositive/misc_json.q new file mode 100644 index 0000000000000..541e36933a6d2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/misc_json.q @@ -0,0 +1,13 @@ +set hive.ddl.output.format=json; + +CREATE TABLE IF NOT EXISTS jsontable (key INT, value STRING) COMMENT 'json table' STORED AS TEXTFILE; + +ALTER TABLE jsontable ADD COLUMNS (name STRING COMMENT 'a new column'); + +ALTER TABLE jsontable RENAME TO jsontable2; + +SHOW TABLE EXTENDED LIKE jsontable2; + +DROP TABLE jsontable2; + +set hive.ddl.output.format=text; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin1.q b/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin1.q new file mode 100644 index 0000000000000..9a0a792a91897 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin1.q @@ -0,0 +1,401 @@ +create table smallTbl1(key string, value string); +insert overwrite table smallTbl1 select * from src where key < 10; + +create table smallTbl2(key string, value string); +insert overwrite table smallTbl2 select * from src where key < 10; + +create table smallTbl3(key string, value string); +insert overwrite table smallTbl3 select * from src where key < 10; + +create table smallTbl4(key string, value string); +insert overwrite table smallTbl4 select * from src where key < 10; + +create table bigTbl(key string, value string); +insert overwrite table bigTbl +select * from +( + select * from src + union all + select * from src + union all + select * from src + union all + select * from src + union all + select * from src + union all + select * from src + union all + select * from src + union all + select * from src + union all + select * from src + union all + select * from src +) subq; + +set hive.auto.convert.join=true; + +explain +select count(*) FROM +(select bigTbl.key as key, bigTbl.value as value1, + bigTbl.value as value2 FROM bigTbl JOIN smallTbl1 + on (bigTbl.key = smallTbl1.key) +) firstjoin +JOIN +smallTbl2 on (firstjoin.value1 = smallTbl2.value); + +select count(*) FROM +(select bigTbl.key as key, bigTbl.value as value1, + bigTbl.value as value2 FROM bigTbl JOIN smallTbl1 + on (bigTbl.key = smallTbl1.key) +) firstjoin +JOIN +smallTbl2 on (firstjoin.value1 = smallTbl2.value); + +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Now run a query with two-way join, which should be converted into a +-- map-join followed by groupby - two MR jobs overall +explain +select count(*) FROM +(select bigTbl.key as key, bigTbl.value as value1, + bigTbl.value as value2 FROM bigTbl JOIN smallTbl1 + on (bigTbl.key = smallTbl1.key) +) firstjoin +JOIN +smallTbl2 on (firstjoin.value1 = smallTbl2.value); + +select count(*) FROM +(select bigTbl.key as key, bigTbl.value as value1, + bigTbl.value as value2 FROM bigTbl JOIN smallTbl1 + on (bigTbl.key = smallTbl1.key) +) firstjoin +JOIN +smallTbl2 on (firstjoin.value1 = smallTbl2.value); + +-- Now run a query with two-way join, which should first be converted into a +-- map-join followed by groupby and then finally into a single MR job. + +explain +select count(*) FROM +(select bigTbl.key as key, bigTbl.value as value1, + bigTbl.value as value2 FROM bigTbl JOIN smallTbl1 + on (bigTbl.key = smallTbl1.key) +) firstjoin +JOIN +smallTbl2 on (firstjoin.value1 = smallTbl2.value) +group by smallTbl2.key; + +select count(*) FROM +(select bigTbl.key as key, bigTbl.value as value1, + bigTbl.value as value2 FROM bigTbl JOIN smallTbl1 + on (bigTbl.key = smallTbl1.key) +) firstjoin +JOIN +smallTbl2 on (firstjoin.value1 = smallTbl2.value) +group by smallTbl2.key; + +drop table bigTbl; + +create table bigTbl(key1 string, key2 string, value string); +insert overwrite table bigTbl +select * from +( + select key as key1, key as key2, value from src + union all + select key as key1, key as key2, value from src + union all + select key as key1, key as key2, value from src + union all + select key as key1, key as key2, value from src + union all + select key as key1, key as key2, value from src + union all + select key as key1, key as key2, value from src + union all + select key as key1, key as key2, value from src + union all + select key as key1, key as key2, value from src + union all + select key as key1, key as key2, value from src + union all + select key as key1, key as key2, value from src +) subq; + +set hive.auto.convert.join.noconditionaltask=false; +-- First disable noconditionaltask +EXPLAIN +SELECT SUM(HASH(join3.key1)), + SUM(HASH(join3.key2)), + SUM(HASH(join3.key3)), + SUM(HASH(join3.key4)), + SUM(HASH(join3.key5)), + SUM(HASH(smallTbl4.key)), + SUM(HASH(join3.value1)), + SUM(HASH(join3.value2)) +FROM (SELECT join2.key1 as key1, + join2.key2 as key2, + join2.key3 as key3, + join2.key4 as key4, + smallTbl3.key as key5, + join2.value1 as value1, + join2.value2 as value2 + FROM (SELECT join1.key1 as key1, + join1.key2 as key2, + join1.key3 as key3, + smallTbl2.key as key4, + join1.value1 as value1, + join1.value2 as value2 + FROM (SELECT bigTbl.key1 as key1, + bigTbl.key2 as key2, + smallTbl1.key as key3, + bigTbl.value as value1, + bigTbl.value as value2 + FROM bigTbl + JOIN smallTbl1 ON (bigTbl.key1 = smallTbl1.key)) join1 + JOIN smallTbl2 ON (join1.value1 = smallTbl2.value)) join2 + JOIN smallTbl3 ON (join2.key2 = smallTbl3.key)) join3 +JOIN smallTbl4 ON (join3.key3 = smallTbl4.key); + +SELECT SUM(HASH(join3.key1)), + SUM(HASH(join3.key2)), + SUM(HASH(join3.key3)), + SUM(HASH(join3.key4)), + SUM(HASH(join3.key5)), + SUM(HASH(smallTbl4.key)), + SUM(HASH(join3.value1)), + SUM(HASH(join3.value2)) +FROM (SELECT join2.key1 as key1, + join2.key2 as key2, + join2.key3 as key3, + join2.key4 as key4, + smallTbl3.key as key5, + join2.value1 as value1, + join2.value2 as value2 + FROM (SELECT join1.key1 as key1, + join1.key2 as key2, + join1.key3 as key3, + smallTbl2.key as key4, + join1.value1 as value1, + join1.value2 as value2 + FROM (SELECT bigTbl.key1 as key1, + bigTbl.key2 as key2, + smallTbl1.key as key3, + bigTbl.value as value1, + bigTbl.value as value2 + FROM bigTbl + JOIN smallTbl1 ON (bigTbl.key1 = smallTbl1.key)) join1 + JOIN smallTbl2 ON (join1.value1 = smallTbl2.value)) join2 + JOIN smallTbl3 ON (join2.key2 = smallTbl3.key)) join3 +JOIN smallTbl4 ON (join3.key3 = smallTbl4.key); + +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; +-- Enable noconditionaltask and set the size of hive.auto.convert.join.noconditionaltask.size +-- to 10000, which is large enough to fit all four small tables (smallTbl1 to smallTbl4). +-- We will use a single MR job to evaluate this query. +EXPLAIN +SELECT SUM(HASH(join3.key1)), + SUM(HASH(join3.key2)), + SUM(HASH(join3.key3)), + SUM(HASH(join3.key4)), + SUM(HASH(join3.key5)), + SUM(HASH(smallTbl4.key)), + SUM(HASH(join3.value1)), + SUM(HASH(join3.value2)) +FROM (SELECT join2.key1 as key1, + join2.key2 as key2, + join2.key3 as key3, + join2.key4 as key4, + smallTbl3.key as key5, + join2.value1 as value1, + join2.value2 as value2 + FROM (SELECT join1.key1 as key1, + join1.key2 as key2, + join1.key3 as key3, + smallTbl2.key as key4, + join1.value1 as value1, + join1.value2 as value2 + FROM (SELECT bigTbl.key1 as key1, + bigTbl.key2 as key2, + smallTbl1.key as key3, + bigTbl.value as value1, + bigTbl.value as value2 + FROM bigTbl + JOIN smallTbl1 ON (bigTbl.key1 = smallTbl1.key)) join1 + JOIN smallTbl2 ON (join1.value1 = smallTbl2.value)) join2 + JOIN smallTbl3 ON (join2.key2 = smallTbl3.key)) join3 +JOIN smallTbl4 ON (join3.key3 = smallTbl4.key); + +SELECT SUM(HASH(join3.key1)), + SUM(HASH(join3.key2)), + SUM(HASH(join3.key3)), + SUM(HASH(join3.key4)), + SUM(HASH(join3.key5)), + SUM(HASH(smallTbl4.key)), + SUM(HASH(join3.value1)), + SUM(HASH(join3.value2)) +FROM (SELECT join2.key1 as key1, + join2.key2 as key2, + join2.key3 as key3, + join2.key4 as key4, + smallTbl3.key as key5, + join2.value1 as value1, + join2.value2 as value2 + FROM (SELECT join1.key1 as key1, + join1.key2 as key2, + join1.key3 as key3, + smallTbl2.key as key4, + join1.value1 as value1, + join1.value2 as value2 + FROM (SELECT bigTbl.key1 as key1, + bigTbl.key2 as key2, + smallTbl1.key as key3, + bigTbl.value as value1, + bigTbl.value as value2 + FROM bigTbl + JOIN smallTbl1 ON (bigTbl.key1 = smallTbl1.key)) join1 + JOIN smallTbl2 ON (join1.value1 = smallTbl2.value)) join2 + JOIN smallTbl3 ON (join2.key2 = smallTbl3.key)) join3 +JOIN smallTbl4 ON (join3.key3 = smallTbl4.key); + +set hive.auto.convert.join.noconditionaltask.size=200; +-- Enable noconditionaltask and set the size of hive.auto.convert.join.noconditionaltask.size +-- to 200, which is large enough to fit two small tables. We will have two jobs to evaluate this +-- query. The first job is a Map-only job to evaluate join1 and join2. +-- The second job will evaluate the rest of this query. +EXPLAIN +SELECT SUM(HASH(join3.key1)), + SUM(HASH(join3.key2)), + SUM(HASH(join3.key3)), + SUM(HASH(join3.key4)), + SUM(HASH(join3.key5)), + SUM(HASH(smallTbl4.key)), + SUM(HASH(join3.value1)), + SUM(HASH(join3.value2)) +FROM (SELECT join2.key1 as key1, + join2.key2 as key2, + join2.key3 as key3, + join2.key4 as key4, + smallTbl3.key as key5, + join2.value1 as value1, + join2.value2 as value2 + FROM (SELECT join1.key1 as key1, + join1.key2 as key2, + join1.key3 as key3, + smallTbl2.key as key4, + join1.value1 as value1, + join1.value2 as value2 + FROM (SELECT bigTbl.key1 as key1, + bigTbl.key2 as key2, + smallTbl1.key as key3, + bigTbl.value as value1, + bigTbl.value as value2 + FROM bigTbl + JOIN smallTbl1 ON (bigTbl.key1 = smallTbl1.key)) join1 + JOIN smallTbl2 ON (join1.value1 = smallTbl2.value)) join2 + JOIN smallTbl3 ON (join2.key2 = smallTbl3.key)) join3 +JOIN smallTbl4 ON (join3.key3 = smallTbl4.key); + +SELECT SUM(HASH(join3.key1)), + SUM(HASH(join3.key2)), + SUM(HASH(join3.key3)), + SUM(HASH(join3.key4)), + SUM(HASH(join3.key5)), + SUM(HASH(smallTbl4.key)), + SUM(HASH(join3.value1)), + SUM(HASH(join3.value2)) +FROM (SELECT join2.key1 as key1, + join2.key2 as key2, + join2.key3 as key3, + join2.key4 as key4, + smallTbl3.key as key5, + join2.value1 as value1, + join2.value2 as value2 + FROM (SELECT join1.key1 as key1, + join1.key2 as key2, + join1.key3 as key3, + smallTbl2.key as key4, + join1.value1 as value1, + join1.value2 as value2 + FROM (SELECT bigTbl.key1 as key1, + bigTbl.key2 as key2, + smallTbl1.key as key3, + bigTbl.value as value1, + bigTbl.value as value2 + FROM bigTbl + JOIN smallTbl1 ON (bigTbl.key1 = smallTbl1.key)) join1 + JOIN smallTbl2 ON (join1.value1 = smallTbl2.value)) join2 + JOIN smallTbl3 ON (join2.key2 = smallTbl3.key)) join3 +JOIN smallTbl4 ON (join3.key3 = smallTbl4.key); + +set hive.auto.convert.join.noconditionaltask.size=0; +-- Enable noconditionaltask and but set the size of hive.auto.convert.join.noconditionaltask.size +-- to 0. The plan will be the same as the one with a disabled nonconditionaltask. +EXPLAIN +SELECT SUM(HASH(join3.key1)), + SUM(HASH(join3.key2)), + SUM(HASH(join3.key3)), + SUM(HASH(join3.key4)), + SUM(HASH(join3.key5)), + SUM(HASH(smallTbl4.key)), + SUM(HASH(join3.value1)), + SUM(HASH(join3.value2)) +FROM (SELECT join2.key1 as key1, + join2.key2 as key2, + join2.key3 as key3, + join2.key4 as key4, + smallTbl3.key as key5, + join2.value1 as value1, + join2.value2 as value2 + FROM (SELECT join1.key1 as key1, + join1.key2 as key2, + join1.key3 as key3, + smallTbl2.key as key4, + join1.value1 as value1, + join1.value2 as value2 + FROM (SELECT bigTbl.key1 as key1, + bigTbl.key2 as key2, + smallTbl1.key as key3, + bigTbl.value as value1, + bigTbl.value as value2 + FROM bigTbl + JOIN smallTbl1 ON (bigTbl.key1 = smallTbl1.key)) join1 + JOIN smallTbl2 ON (join1.value1 = smallTbl2.value)) join2 + JOIN smallTbl3 ON (join2.key2 = smallTbl3.key)) join3 +JOIN smallTbl4 ON (join3.key3 = smallTbl4.key); + +SELECT SUM(HASH(join3.key1)), + SUM(HASH(join3.key2)), + SUM(HASH(join3.key3)), + SUM(HASH(join3.key4)), + SUM(HASH(join3.key5)), + SUM(HASH(smallTbl4.key)), + SUM(HASH(join3.value1)), + SUM(HASH(join3.value2)) +FROM (SELECT join2.key1 as key1, + join2.key2 as key2, + join2.key3 as key3, + join2.key4 as key4, + smallTbl3.key as key5, + join2.value1 as value1, + join2.value2 as value2 + FROM (SELECT join1.key1 as key1, + join1.key2 as key2, + join1.key3 as key3, + smallTbl2.key as key4, + join1.value1 as value1, + join1.value2 as value2 + FROM (SELECT bigTbl.key1 as key1, + bigTbl.key2 as key2, + smallTbl1.key as key3, + bigTbl.value as value1, + bigTbl.value as value2 + FROM bigTbl + JOIN smallTbl1 ON (bigTbl.key1 = smallTbl1.key)) join1 + JOIN smallTbl2 ON (join1.value1 = smallTbl2.value)) join2 + JOIN smallTbl3 ON (join2.key2 = smallTbl3.key)) join3 +JOIN smallTbl4 ON (join3.key3 = smallTbl4.key); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin2.q b/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin2.q new file mode 100644 index 0000000000000..ce6cf6d8d6c0d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin2.q @@ -0,0 +1,189 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=6000; + +-- we will generate one MR job. +EXPLAIN +SELECT tmp.key +FROM (SELECT x1.key AS key FROM src x1 JOIN src1 y1 ON (x1.key = y1.key) + UNION ALL + SELECT x2.key AS key FROM src x2 JOIN src1 y2 ON (x2.key = y2.key)) tmp +ORDER BY tmp.key; + +SELECT tmp.key +FROM (SELECT x1.key AS key FROM src x1 JOIN src1 y1 ON (x1.key = y1.key) + UNION ALL + SELECT x2.key AS key FROM src x2 JOIN src1 y2 ON (x2.key = y2.key)) tmp +ORDER BY tmp.key; + +set hive.auto.convert.join.noconditionaltask.size=400; +-- Check if the total size of local tables will be +-- larger than the limit that +-- we set through hive.auto.convert.join.noconditionaltask.size (right now, it is +-- 400 bytes). If so, do not merge. +-- For this query, we will merge the MapJoin of x2 and y2 into the MR job +-- for UNION ALL and ORDER BY. But, the MapJoin of x1 and y2 will not be merged +-- into that MR job. +EXPLAIN +SELECT tmp.key +FROM (SELECT x1.key AS key FROM src x1 JOIN src1 y1 ON (x1.key = y1.key) + UNION ALL + SELECT x2.key AS key FROM src x2 JOIN src1 y2 ON (x2.key = y2.key)) tmp +ORDER BY tmp.key; + +SELECT tmp.key +FROM (SELECT x1.key AS key FROM src x1 JOIN src1 y1 ON (x1.key = y1.key) + UNION ALL + SELECT x2.key AS key FROM src x2 JOIN src1 y2 ON (x2.key = y2.key)) tmp +ORDER BY tmp.key; + +set hive.auto.convert.join.noconditionaltask.size=6000; +-- We will use two jobs. +-- We will generate one MR job for GROUP BY +-- on x1, one MR job for both the MapJoin of x2 and y2, the UNION ALL, and the +-- ORDER BY. +EXPLAIN +SELECT tmp.key +FROM (SELECT x1.key AS key FROM src1 x1 GROUP BY x1.key + UNION ALL + SELECT x2.key AS key FROM src x2 JOIN src1 y2 ON (x2.key = y2.key)) tmp +ORDER BY tmp.key; + +SELECT tmp.key +FROM (SELECT x1.key AS key FROM src1 x1 GROUP BY x1.key + UNION ALL + SELECT x2.key AS key FROM src x2 JOIN src1 y2 ON (x2.key = y2.key)) tmp +ORDER BY tmp.key; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is disabled, +-- we will use 5 jobs. +-- We will generate one MR job to evaluate the sub-query tmp1, +-- one MR job to evaluate the sub-query tmp2, +-- one MR job for the Join of tmp1 and tmp2, +-- one MR job for aggregation on the result of the Join of tmp1 and tmp2, +-- and one MR job for the ORDER BY. +EXPLAIN +SELECT tmp1.key as key, count(*) as cnt +FROM (SELECT x1.key AS key + FROM src x1 JOIN src1 y1 ON (x1.key = y1.key) + GROUP BY x1.key) tmp1 +JOIN (SELECT x2.key AS key + FROM src x2 JOIN src1 y2 ON (x2.key = y2.key) + GROUP BY x2.key) tmp2 +ON (tmp1.key = tmp2.key) +GROUP BY tmp1.key +ORDER BY key, cnt; + +SELECT tmp1.key as key, count(*) as cnt +FROM (SELECT x1.key AS key + FROM src x1 JOIN src1 y1 ON (x1.key = y1.key) + GROUP BY x1.key) tmp1 +JOIN (SELECT x2.key AS key + FROM src x2 JOIN src1 y2 ON (x2.key = y2.key) + GROUP BY x2.key) tmp2 +ON (tmp1.key = tmp2.key) +GROUP BY tmp1.key +ORDER BY key, cnt; + +set hive.optimize.correlation=true; +-- When Correlation Optimizer is enabled, +-- we will use two jobs. This first MR job will evaluate sub-queries of tmp1, tmp2, +-- the Join of tmp1 and tmp2, and the aggregation on the result of the Join of +-- tmp1 and tmp2. The second job will do the ORDER BY. +EXPLAIN +SELECT tmp1.key as key, count(*) as cnt +FROM (SELECT x1.key AS key + FROM src x1 JOIN src1 y1 ON (x1.key = y1.key) + GROUP BY x1.key) tmp1 +JOIN (SELECT x2.key AS key + FROM src x2 JOIN src1 y2 ON (x2.key = y2.key) + GROUP BY x2.key) tmp2 +ON (tmp1.key = tmp2.key) +GROUP BY tmp1.key +ORDER BY key, cnt; + +SELECT tmp1.key as key, count(*) as cnt +FROM (SELECT x1.key AS key + FROM src x1 JOIN src1 y1 ON (x1.key = y1.key) + GROUP BY x1.key) tmp1 +JOIN (SELECT x2.key AS key + FROM src x2 JOIN src1 y2 ON (x2.key = y2.key) + GROUP BY x2.key) tmp2 +ON (tmp1.key = tmp2.key) +GROUP BY tmp1.key +ORDER BY key, cnt; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is disabled, +-- we will use five jobs. +-- We will generate one MR job to evaluate the sub-query tmp1, +-- one MR job to evaluate the sub-query tmp2, +-- one MR job for the Join of tmp1 and tmp2, +-- one MR job for aggregation on the result of the Join of tmp1 and tmp2, +-- and one MR job for the ORDER BY. +EXPLAIN +SELECT tmp1.key as key, count(*) as cnt +FROM (SELECT x1.key AS key + FROM src1 x1 + GROUP BY x1.key) tmp1 +JOIN (SELECT x2.key AS key + FROM src x2 JOIN src1 y2 ON (x2.key = y2.key) + GROUP BY x2.key) tmp2 +ON (tmp1.key = tmp2.key) +GROUP BY tmp1.key +ORDER BY key, cnt; + +SELECT tmp1.key as key, count(*) as cnt +FROM (SELECT x1.key AS key + FROM src1 x1 + GROUP BY x1.key) tmp1 +JOIN (SELECT x2.key AS key + FROM src x2 JOIN src1 y2 ON (x2.key = y2.key) + GROUP BY x2.key) tmp2 +ON (tmp1.key = tmp2.key) +GROUP BY tmp1.key +ORDER BY key, cnt; + +set hive.optimize.correlation=true; +-- When Correlation Optimizer is enabled, +-- we will use two job. This first MR job will evaluate sub-queries of tmp1, tmp2, +-- the Join of tmp1 and tmp2, and the aggregation on the result of the Join of +-- tmp1 and tmp2. The second job will do the ORDER BY. +EXPLAIN +SELECT tmp1.key as key, count(*) as cnt +FROM (SELECT x1.key AS key + FROM src1 x1 + GROUP BY x1.key) tmp1 +JOIN (SELECT x2.key AS key + FROM src x2 JOIN src1 y2 ON (x2.key = y2.key) + GROUP BY x2.key) tmp2 +ON (tmp1.key = tmp2.key) +GROUP BY tmp1.key +ORDER BY key, cnt; + +SELECT tmp1.key as key, count(*) as cnt +FROM (SELECT x1.key AS key + FROM src1 x1 + GROUP BY x1.key) tmp1 +JOIN (SELECT x2.key AS key + FROM src x2 JOIN src1 y2 ON (x2.key = y2.key) + GROUP BY x2.key) tmp2 +ON (tmp1.key = tmp2.key) +GROUP BY tmp1.key +ORDER BY key, cnt; + +-- Check if we can correctly handle partitioned table. +CREATE TABLE part_table(key string, value string) PARTITIONED BY (partitionId int); +INSERT OVERWRITE TABLE part_table PARTITION (partitionId=1) + SELECT key, value FROM src ORDER BY key, value LIMIT 100; +INSERT OVERWRITE TABLE part_table PARTITION (partitionId=2) + SELECT key, value FROM src1 ORDER BY key, value; + +EXPLAIN +SELECT count(*) +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); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert.q b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert.q new file mode 100644 index 0000000000000..718211b968ccd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert.q @@ -0,0 +1,266 @@ + + +create table src_multi1 like src; +create table src_multi2 like src; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + + + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +explain +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=false; + +explain +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=true; + +explain +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/hive_test/multiins_local/temp; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=true; + + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby.q b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby.q new file mode 100644 index 0000000000000..3aac82ec5a95c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby.q @@ -0,0 +1,33 @@ +--HIVE-3699 Multiple insert overwrite into multiple tables query stores same results in all tables +create table e1 (key string, count int); +create table e2 (key string, count int); + +explain FROM src +INSERT OVERWRITE TABLE e1 + SELECT key, COUNT(*) WHERE key>450 GROUP BY key ORDER BY key +INSERT OVERWRITE TABLE e2 + SELECT key, COUNT(*) WHERE key>500 GROUP BY key ORDER BY key; + +FROM src +INSERT OVERWRITE TABLE e1 + SELECT key, COUNT(*) WHERE key>450 GROUP BY key ORDER BY key +INSERT OVERWRITE TABLE e2 + SELECT key, COUNT(*) WHERE key>500 GROUP BY key ORDER BY key; + +select * from e1; +select * from e2; + +explain FROM src +INSERT OVERWRITE TABLE e1 + SELECT key, COUNT(*) WHERE key>450 GROUP BY key ORDER BY key +INSERT OVERWRITE TABLE e2 + SELECT key, COUNT(*) GROUP BY key ORDER BY key; + +FROM src +INSERT OVERWRITE TABLE e1 + SELECT key, COUNT(*) WHERE key>450 GROUP BY key ORDER BY key +INSERT OVERWRITE TABLE e2 + SELECT key, COUNT(*) GROUP BY key ORDER BY key; + +select * from e1; +select * from e2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby2.q b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby2.q new file mode 100644 index 0000000000000..46e2b19576856 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby2.q @@ -0,0 +1,19 @@ +--HIVE-3699 Multiple insert overwrite into multiple tables query stores same results in all tables +create table e1 (count int); +create table e2 (percentile double); + +explain +FROM (select key, cast(key as double) as value from src order by key) a +INSERT OVERWRITE TABLE e1 + SELECT COUNT(*) +INSERT OVERWRITE TABLE e2 + SELECT percentile_approx(value, 0.5); + +FROM (select key, cast(key as double) as value from src order by key) a +INSERT OVERWRITE TABLE e1 + SELECT COUNT(*) +INSERT OVERWRITE TABLE e2 + SELECT percentile_approx(value, 0.5); + +select * from e1; +select * from e2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby3.q b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby3.q new file mode 100644 index 0000000000000..040adca5c2e26 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby3.q @@ -0,0 +1,52 @@ +create table e1 (key string, keyD double); +create table e2 (key string, keyD double, value string); +create table e3 (key string, keyD double); + +explain +FROM (select key, cast(key as double) as keyD, value from src order by key) a +INSERT OVERWRITE TABLE e1 + SELECT key, COUNT(distinct value) group by key +INSERT OVERWRITE TABLE e2 + SELECT key, sum(keyD), value group by key, value; + +explain +FROM (select key, cast(key as double) as keyD, value from src order by key) a +INSERT OVERWRITE TABLE e2 + SELECT key, sum(keyD), value group by key, value +INSERT OVERWRITE TABLE e1 + SELECT key, COUNT(distinct value) group by key; + +FROM (select key, cast(key as double) as keyD, value from src order by key) a +INSERT OVERWRITE TABLE e1 + SELECT key, COUNT(distinct value) group by key +INSERT OVERWRITE TABLE e2 + SELECT key, sum(keyD), value group by key, value; + +select * from e1; +select * from e2; + +FROM (select key, cast(key as double) as keyD, value from src order by key) a +INSERT OVERWRITE TABLE e2 + SELECT key, sum(keyD), value group by key, value +INSERT OVERWRITE TABLE e1 + SELECT key, COUNT(distinct value) group by key; + +select * from e1; +select * from e2; + +explain +from src +insert overwrite table e1 +select key, count(distinct value) group by key +insert overwrite table e3 +select value, count(distinct key) group by value; + + +explain +FROM (select key, cast(key as double) as keyD, value from src order by key) a +INSERT OVERWRITE TABLE e1 + SELECT key, COUNT(distinct value) group by key +INSERT OVERWRITE TABLE e2 + SELECT key, sum(keyD), value group by key, value +INSERT overwrite table e3 + SELECT key, COUNT(distinct keyD) group by key, keyD, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q new file mode 100644 index 0000000000000..eb1b9ed3b437a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q @@ -0,0 +1,102 @@ +create table src_10 as select * from src limit 10; + +create table src_lv1 (key string, value string); +create table src_lv2 (key string, value string); +create table src_lv3 (key string, value string); + +-- 2LV +-- TS[0]-LVF[1]-SEL[2]-LVJ[5]-SEL[11]-FS[12] +-- -SEL[3]-UDTF[4]-LVJ[5] +-- -LVF[6]-SEL[7]-LVJ[10]-SEL[13]-FS[14] +-- -SEL[8]-UDTF[9]-LVJ[10] +explain +from src_10 +insert overwrite table src_lv1 select key, C lateral view explode(array(key+1, key+2)) A as C +insert overwrite table src_lv2 select key, C lateral view explode(array(key+3, key+4)) A as C; + +from src_10 +insert overwrite table src_lv1 select key, C lateral view explode(array(key+1, key+2)) A as C +insert overwrite table src_lv2 select key, C lateral view explode(array(key+3, key+4)) A as C; + +select * from src_lv1 order by key, value; +select * from src_lv2 order by key, value; + +-- 2(LV+GBY) +-- TS[0]-LVF[1]-SEL[2]-LVJ[5]-SEL[11]-GBY[12]-RS[13]-GBY[14]-SEL[15]-FS[16] +-- -SEL[3]-UDTF[4]-LVJ[5] +-- -LVF[6]-SEL[7]-LVJ[10]-SEL[17]-GBY[18]-RS[19]-GBY[20]-SEL[21]-FS[22] +-- -SEL[8]-UDTF[9]-LVJ[10] +explain +from src_10 +insert overwrite table src_lv1 select key, sum(C) lateral view explode(array(key+1, key+2)) A as C group by key +insert overwrite table src_lv2 select key, sum(C) lateral view explode(array(key+3, key+4)) A as C group by key; + +from src_10 +insert overwrite table src_lv1 select key, sum(C) lateral view explode(array(key+1, key+2)) A as C group by key +insert overwrite table src_lv2 select key, sum(C) lateral view explode(array(key+3, key+4)) A as C group by key; + +select * from src_lv1 order by key, value; +select * from src_lv2 order by key, value; + +-- (LV+GBY) + RS:2GBY +-- TS[0]-LVF[1]-SEL[2]-LVJ[5]-SEL[6]-GBY[7]-RS[8]-GBY[9]-SEL[10]-FS[11] +-- -SEL[3]-UDTF[4]-LVJ[5] +-- -FIL[12]-SEL[13]-RS[14]-FOR[15]-FIL[16]-GBY[17]-SEL[18]-FS[19] +-- -FIL[20]-GBY[21]-SEL[22]-FS[23] +explain +from src_10 +insert overwrite table src_lv1 select key, sum(C) lateral view explode(array(key+1, key+2)) A as C group by key +insert overwrite table src_lv2 select key, count(value) where key > 200 group by key +insert overwrite table src_lv3 select key, count(value) where key < 200 group by key; + +from src_10 +insert overwrite table src_lv1 select key, sum(C) lateral view explode(array(key+1, key+2)) A as C group by key +insert overwrite table src_lv2 select key, count(value) where key > 200 group by key +insert overwrite table src_lv3 select key, count(value) where key < 200 group by key; + +select * from src_lv1 order by key, value; +select * from src_lv2 order by key, value; +select * from src_lv3 order by key, value; + +-- todo: shared distinct columns (should work with hive.optimize.multigroupby.common.distincts) +-- 2(LV+GBY) + RS:2GBY +-- TS[0]-LVF[1]-SEL[2]-LVJ[5]-SEL[11]-GBY[12]-RS[13]-GBY[14]-SEL[15]-FS[16] +-- -SEL[3]-UDTF[4]-LVJ[5] +-- -LVF[6]-SEL[7]-LVJ[10]-SEL[17]-GBY[18]-RS[19]-GBY[20]-SEL[21]-FS[22] +-- -SEL[8]-UDTF[9]-LVJ[10] +-- -SEL[23]-GBY[24]-RS[25]-GBY[26]-SEL[27]-FS[28] +explain +from src_10 +insert overwrite table src_lv1 select C, sum(distinct key) lateral view explode(array(key+1, key+2)) A as C group by C +insert overwrite table src_lv2 select C, sum(distinct key) lateral view explode(array(key+3, key+4)) A as C group by C +insert overwrite table src_lv3 select value, sum(distinct key) group by value; + +from src_10 +insert overwrite table src_lv1 select C, sum(distinct key) lateral view explode(array(key+1, key+2)) A as C group by C +insert overwrite table src_lv2 select C, sum(distinct key) lateral view explode(array(key+3, key+4)) A as C group by C +insert overwrite table src_lv3 select value, sum(distinct key) group by value; + +select * from src_lv1 order by key, value; +select * from src_lv2 order by key, value; +select * from src_lv3 order by key, value; + +create table src_lv4 (key string, value string); + +-- Common distincts optimization works across non-lateral view queries, but not across lateral view multi inserts +explain +from src_10 +insert overwrite table src_lv1 select key, sum(distinct C) lateral view explode(array(key+1, key+2)) A as C group by key +insert overwrite table src_lv2 select key, sum(distinct C) lateral view explode(array(key+3, key+4)) A as C group by key +insert overwrite table src_lv3 select value, sum(distinct key) where key > 200 group by value +insert overwrite table src_lv4 select value, sum(distinct key) where key < 200 group by value; + +from src_10 +insert overwrite table src_lv1 select key, sum(distinct C) lateral view explode(array(key+1, key+2)) A as C group by key +insert overwrite table src_lv2 select key, sum(distinct C) lateral view explode(array(key+3, key+4)) A as C group by key +insert overwrite table src_lv3 select value, sum(distinct key) where key > 200 group by value +insert overwrite table src_lv4 select value, sum(distinct key) where key < 200 group by value; + +select * from src_lv1 order by key, value; +select * from src_lv2 order by key, value; +select * from src_lv3 order by key, value; +select * from src_lv4 order by key, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q new file mode 100644 index 0000000000000..e30992b2cde6b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q @@ -0,0 +1,414 @@ +set hive.multi.insert.move.tasks.share.dependencies=true; + +create table src_multi1 like src; +create table src_multi2 like src; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +from src +insert overwrite table src_multi1 select * where key < 10 group by key, value +insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + + + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +explain +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=false; + +explain +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=true; + +explain +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +from (select * from src union all select * from src) s +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + + + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/hive_test/multiins_local/temp; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=true; + + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/0' select * where key = 0 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key = 2 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/4' select * where key = 4; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +from src +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=false; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +explain +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +from src +insert overwrite table src_multi1 select * where key < 10 +insert overwrite table src_multi2 select * where key > 10 and key < 20 +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/1' select * where key < 10 group by key, value cluster by key +insert overwrite local directory '${system:test.tmp.dir}/hive_test/multiins_local/2' select * where key > 10 and key < 20 group by key, value cluster by value; + +select * from src_multi1 order by key, value; +select * from src_multi2 order by key, value; + +dfs -ls ${system:test.tmp.dir}/hive_test/multiins_local; +dfs -rmr ${system:test.tmp.dir}/hive_test/multiins_local; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_join_union.q b/src/test/hive/ql/src/test/queries/clientpositive/multi_join_union.q new file mode 100644 index 0000000000000..39650fd4a1ef1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multi_join_union.q @@ -0,0 +1,13 @@ + +set hive.auto.convert.join=true; + +CREATE TABLE src11 as SELECT * FROM src; +CREATE TABLE src12 as SELECT * FROM src; +CREATE TABLE src13 as SELECT * FROM src; +CREATE TABLE src14 as SELECT * FROM src; + + +EXPLAIN SELECT * FROM +src11 a JOIN +src12 b ON (a.key = b.key) JOIN +(SELECT * FROM (SELECT * FROM src13 UNION ALL SELECT * FROM src14)a )c ON c.value = b.value; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_sahooks.q b/src/test/hive/ql/src/test/queries/clientpositive/multi_sahooks.q new file mode 100644 index 0000000000000..a6cd1ef940018 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multi_sahooks.q @@ -0,0 +1,30 @@ +set hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.metadata.DummySemanticAnalyzerHook1; + +drop table tbl_sahook; +create table tbl_sahook (c string); +desc extended tbl_sahook; +drop table tbl_sahook; + +set hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.metadata.DummySemanticAnalyzerHook1,org.apache.hadoop.hive.ql.metadata.DummySemanticAnalyzerHook; + +drop table tbl_sahooks; +create table tbl_sahooks (c string); +desc extended tbl_sahooks; +drop table tbl_sahooks; + +set hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.metadata.DummySemanticAnalyzerHook,org.apache.hadoop.hive.ql.metadata.DummySemanticAnalyzerHook1; + +drop table tbl_sahooks; +create table tbl_sahooks (c string); +desc extended tbl_sahooks; +drop table tbl_sahooks; + +set hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.metadata.DummySemanticAnalyzerHook1,org.apache.hadoop.hive.ql.metadata.DummySemanticAnalyzerHook1; + +drop table tbl_sahooks; +create table tbl_sahooks (c string); +desc extended tbl_sahooks; + +set hive.semantic.analyzer.hook=; +drop table tbl_sahooks; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multigroupby_singlemr.q b/src/test/hive/ql/src/test/queries/clientpositive/multigroupby_singlemr.q new file mode 100644 index 0000000000000..a4e86ade476a7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/multigroupby_singlemr.q @@ -0,0 +1,33 @@ +CREATE TABLE TBL(C1 INT, C2 INT, C3 INT, C4 INT); + +CREATE TABLE DEST1(d1 INT, d2 INT) STORED AS TEXTFILE; +CREATE TABLE DEST2(d1 INT, d2 INT, d3 INT) STORED AS TEXTFILE; +CREATE TABLE DEST3(d1 INT, d2 INT, d3 INT, d4 INT) STORED AS TEXTFILE; +CREATE TABLE DEST4(d1 INT, d2 INT, d3 INT, d4 INT) STORED AS TEXTFILE; + +EXPLAIN +FROM TBL +INSERT OVERWRITE TABLE DEST1 SELECT TBL.C1, COUNT(TBL.C2) GROUP BY TBL.C1 +INSERT OVERWRITE TABLE DEST2 SELECT TBL.C1, TBL.C2, COUNT(TBL.C3) GROUP BY TBL.C1, TBL.C2; + +EXPLAIN +FROM TBL +INSERT OVERWRITE TABLE DEST1 SELECT TBL.C1, COUNT(TBL.C2) GROUP BY TBL.C1 +INSERT OVERWRITE TABLE DEST2 SELECT TBL.C1, TBL.C2, COUNT(TBL.C3) GROUP BY TBL.C2, TBL.C1; + +EXPLAIN +FROM TBL +INSERT OVERWRITE TABLE DEST3 SELECT TBL.C1, TBL.C2, TBL.C3, COUNT(TBL.C4) GROUP BY TBL.C1, TBL.C2, TBL.C3 +INSERT OVERWRITE TABLE DEST2 SELECT TBL.C1, TBL.C2, COUNT(TBL.C3) GROUP BY TBL.C1, TBL.C2; + +EXPLAIN +FROM TBL +INSERT OVERWRITE TABLE DEST3 SELECT TBL.C1, TBL.C2, TBL.C3, COUNT(TBL.C4) GROUP BY TBL.C1, TBL.C2, TBL.C3 +INSERT OVERWRITE TABLE DEST4 SELECT TBL.C1, TBL.C2, TBL.C3, COUNT(TBL.C4) GROUP BY TBL.C1, TBL.C3, TBL.C2; + + +EXPLAIN +FROM TBL +INSERT OVERWRITE TABLE DEST3 SELECT TBL.C1, TBL.C2, TBL.C3, COUNT(TBL.C4) GROUP BY TBL.C1, TBL.C2, TBL.C3 +INSERT OVERWRITE TABLE DEST2 SELECT TBL.C1, TBL.C2, COUNT(TBL.C3) GROUP BY TBL.C1, TBL.C2 +INSERT OVERWRITE TABLE DEST1 SELECT TBL.C1, COUNT(TBL.C2) GROUP BY TBL.C1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nested_complex.q b/src/test/hive/ql/src/test/queries/clientpositive/nested_complex.q new file mode 100644 index 0000000000000..b94fbb7b8be43 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nested_complex.q @@ -0,0 +1,22 @@ + +create table nestedcomplex ( +simple_int int, +max_nested_array array>>>>>>>>>>>>>>>>>>>>>>, +max_nested_map array>>>>>>>>>>>>>>>>>>>>>, +max_nested_struct array>>>>>>>>>>>>>>>>>>>>>>, +simple_string string) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +WITH SERDEPROPERTIES ( + 'hive.serialization.extend.nesting.levels'='true', + 'line.delim'='\n' +) +; + +describe nestedcomplex; +describe extended nestedcomplex; + + +load data local inpath '../data/files/nested_complex.txt' overwrite into table nestedcomplex; + +select * from nestedcomplex sort by simple_int; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nestedvirtual.q b/src/test/hive/ql/src/test/queries/clientpositive/nestedvirtual.q new file mode 100644 index 0000000000000..7b80941b8c396 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nestedvirtual.q @@ -0,0 +1,27 @@ +CREATE TABLE pokes (foo INT, bar STRING); +create table pokes2(foo INT, bar STRING); + +create table jssarma_nilzma_bad as select a.val, a.filename, a.offset from (select hash(foo) as val, INPUT__FILE__NAME as filename, BLOCK__OFFSET__INSIDE__FILE as offset from pokes) a join pokes2 b on (a.val = b.foo); + +drop table jssarma_nilzma_bad; + +drop table pokes; +drop table pokes2; +CREATE TABLE pokes (foo INT, bar STRING); +create table pokes2(foo INT, bar STRING); + +create table jssarma_nilzma_bad as select a.val, a.filename, a.offset from (select hash(foo) as val, INPUT__FILE__NAME as filename, BLOCK__OFFSET__INSIDE__FILE as offset from pokes) a join pokes2 b on (a.val = b.foo); + +drop table jssarma_nilzma_bad; + +drop table pokes; +drop table pokes2; +CREATE TABLE pokes (foo INT, bar STRING); +create table pokes2(foo INT, bar STRING); + +create table jssarma_nilzma_bad as select a.val, a.filename, a.offset from (select hash(foo) as val, INPUT__FILE__NAME as filename, BLOCK__OFFSET__INSIDE__FILE as offset from pokes) a join pokes2 b on (a.val = b.foo); + +drop table jssarma_nilzma_bad; + +drop table pokes; +drop table pokes2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/newline.q b/src/test/hive/ql/src/test/queries/clientpositive/newline.q new file mode 100644 index 0000000000000..722ecf6d972ca --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/newline.q @@ -0,0 +1,57 @@ +add file ../data/scripts/newline.py; +set hive.transform.escape.input=true; + +create table tmp_tmp(key string, value string) stored as rcfile; +insert overwrite table tmp_tmp +SELECT TRANSFORM(key, value) USING +'python newline.py' AS key, value FROM src limit 6; + +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; + +create table tmp_tmp(key string, value string) stored as rcfile; +insert overwrite table tmp_tmp +SELECT TRANSFORM(key, value) USING +'python escapednewline.py' AS key, value FROM src limit 5; + +select * from tmp_tmp ORDER BY key ASC, value ASC; + +SELECT TRANSFORM(key, value) USING +'cat' AS (key, value) FROM tmp_tmp; + +insert overwrite table tmp_tmp +SELECT TRANSFORM(key, value) USING +'python escapedcarriagereturn.py' AS key, value FROM src limit 5; + +select * from tmp_tmp ORDER BY key ASC, value ASC; + +SELECT TRANSFORM(key, value) USING +'cat' AS (key, value) FROM tmp_tmp; + +insert overwrite table tmp_tmp +SELECT TRANSFORM(key, value) USING +'python escapedtab.py' AS key, value FROM src limit 5; + +select * from tmp_tmp ORDER BY key ASC, value ASC; + +SELECT TRANSFORM(key, value) USING +'cat' AS (key, value) FROM tmp_tmp; + +insert overwrite table tmp_tmp +SELECT TRANSFORM(key, value) USING +'python doubleescapedtab.py' AS key, value FROM src limit 5; + +select * from tmp_tmp ORDER BY key ASC, value ASC; + +SELECT TRANSFORM(key, value) USING +'cat' AS (key, value) FROM tmp_tmp; + +SELECT key FROM (SELECT TRANSFORM ('a\tb', 'c') USING 'cat' AS (key, value) FROM src limit 1)a ORDER BY key ASC; + +SELECT value FROM (SELECT TRANSFORM ('a\tb', 'c') USING 'cat' AS (key, value) FROM src limit 1)a ORDER BY value ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/no_hooks.q b/src/test/hive/ql/src/test/queries/clientpositive/no_hooks.q new file mode 100644 index 0000000000000..bbd1feaa37533 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/no_hooks.q @@ -0,0 +1,6 @@ +set hive.exec.pre.hooks=; + +EXPLAIN +SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value; + +SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/noalias_subq1.q b/src/test/hive/ql/src/test/queries/clientpositive/noalias_subq1.q new file mode 100644 index 0000000000000..dca8a42badb89 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/noalias_subq1.q @@ -0,0 +1,5 @@ +EXPLAIN +SELECT c1 FROM (select value as c1, key as c2 from src) x where c2 < 100; + +SELECT c1 FROM (select value as c1, key as c2 from src) x where c2 < 100; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q b/src/test/hive/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q new file mode 100644 index 0000000000000..6795475699583 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q @@ -0,0 +1,9 @@ +-- was negative/ambiguous_table_col.q + +drop table ambiguous; +create table ambiguous (key string, value string); + +FROM src key +INSERT OVERWRITE TABLE ambiguous SELECT key.key, key.value WHERE key.value < 'val_100'; + +drop table ambiguous; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q b/src/test/hive/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q new file mode 100644 index 0000000000000..a9cb7d0217540 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q @@ -0,0 +1,45 @@ +-- negative, references twice for result of funcion +explain select nkey, nkey + 1 from (select key + 1 as nkey, value from src) a; + +set hive.auto.convert.join=false; +-- This test query is introduced for HIVE-4968. +-- First, we do not convert the join to MapJoin. +EXPLAIN +SELECT tmp4.key as key, tmp4.value as value, tmp4.count as count +FROM (SELECT tmp2.key as key, tmp2.value as value, tmp3.count as count + FROM (SELECT * + FROM (SELECT key, value + FROM src1) tmp1 ) tmp2 + JOIN (SELECT count(*) as count + FROM src1) tmp3 + ) tmp4 order by key, value, count; + +SELECT tmp4.key as key, tmp4.value as value, tmp4.count as count +FROM (SELECT tmp2.key as key, tmp2.value as value, tmp3.count as count + FROM (SELECT * + FROM (SELECT key, value + FROM src1) tmp1 ) tmp2 + JOIN (SELECT count(*) as count + FROM src1) tmp3 + ) tmp4 order by key, value, count; + +set hive.auto.convert.join=true; +-- Then, we convert the join to MapJoin. +EXPLAIN +SELECT tmp4.key as key, tmp4.value as value, tmp4.count as count +FROM (SELECT tmp2.key as key, tmp2.value as value, tmp3.count as count + FROM (SELECT * + FROM (SELECT key, value + FROM src1) tmp1 ) tmp2 + JOIN (SELECT count(*) as count + FROM src1) tmp3 + ) tmp4 order by key, value, count; + +SELECT tmp4.key as key, tmp4.value as value, tmp4.count as count +FROM (SELECT tmp2.key as key, tmp2.value as value, tmp3.count as count + FROM (SELECT * + FROM (SELECT key, value + FROM src1) tmp1 ) tmp2 + JOIN (SELECT count(*) as count + FROM src1) tmp3 + ) tmp4 order by key, value, count; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nonmr_fetch.q b/src/test/hive/ql/src/test/queries/clientpositive/nonmr_fetch.q new file mode 100644 index 0000000000000..e961e93a18790 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nonmr_fetch.q @@ -0,0 +1,83 @@ +set hive.fetch.task.conversion=minimal; + +-- backward compatible (minimal) +explain select * from src limit 10; +select * from src limit 10; + +explain select * from srcpart where ds='2008-04-08' AND hr='11' limit 10; +select * from srcpart where ds='2008-04-08' AND hr='11' limit 10; + +-- negative, select expression +explain select key from src limit 10; +select key from src limit 10; + +-- negative, filter on non-partition column +explain select * from srcpart where key > 100 limit 10; +select * from srcpart where key > 100 limit 10; + +-- negative, table sampling +explain select * from src TABLESAMPLE (0.25 PERCENT) limit 10; +select * from src TABLESAMPLE (0.25 PERCENT) limit 10; + +set hive.fetch.task.conversion=more; + +-- backward compatible (more) +explain select * from src limit 10; +select * from src limit 10; + +explain select * from srcpart where ds='2008-04-08' AND hr='11' limit 10; +select * from srcpart where ds='2008-04-08' AND hr='11' limit 10; + +-- select expression +explain select cast(key as int) * 10, upper(value) from src limit 10; +select cast(key as int) * 10, upper(value) from src limit 10; + +-- filter on non-partition column +explain select key from src where key < 100 limit 10; +select key from src where key < 100 limit 10; + +-- select expr for partitioned table +explain select key from srcpart where ds='2008-04-08' AND hr='11' limit 10; +select key from srcpart where ds='2008-04-08' AND hr='11' limit 10; + +-- virtual columns +explain select *, BLOCK__OFFSET__INSIDE__FILE from src where key < 10 limit 10; +select *, BLOCK__OFFSET__INSIDE__FILE from src where key < 100 limit 10; + +-- virtual columns on partitioned table +explain select *, BLOCK__OFFSET__INSIDE__FILE from srcpart where key < 10 limit 30; +select *, BLOCK__OFFSET__INSIDE__FILE from srcpart where key < 10 limit 30; + +-- bucket sampling +explain select *, BLOCK__OFFSET__INSIDE__FILE from src TABLESAMPLE (BUCKET 1 OUT OF 40 ON key); +select *, BLOCK__OFFSET__INSIDE__FILE from src TABLESAMPLE (BUCKET 1 OUT OF 40 ON key); +explain select *, BLOCK__OFFSET__INSIDE__FILE from srcpart TABLESAMPLE (BUCKET 1 OUT OF 40 ON key); +select *, BLOCK__OFFSET__INSIDE__FILE from srcpart TABLESAMPLE (BUCKET 1 OUT OF 40 ON key); + +-- split sampling +explain select * from src TABLESAMPLE (0.25 PERCENT); +select * from src TABLESAMPLE (0.25 PERCENT); +explain select *, BLOCK__OFFSET__INSIDE__FILE from srcpart TABLESAMPLE (0.25 PERCENT); +select *, BLOCK__OFFSET__INSIDE__FILE from srcpart TABLESAMPLE (0.25 PERCENT); + +-- non deterministic func +explain select key, value, BLOCK__OFFSET__INSIDE__FILE from srcpart where ds="2008-04-09" AND rand() > 1; +select key, value, BLOCK__OFFSET__INSIDE__FILE from srcpart where ds="2008-04-09" AND rand() > 1; + +-- negative, groupby +explain select key, count(value) from src group by key; + +-- negative, distinct +explain select distinct key, value from src; + +-- negative, CTAS +explain create table srcx as select distinct key, value from src; + +-- negative, analyze +explain analyze table src compute statistics; + +-- negative, subq +explain select a.* from (select * from src) a; + +-- negative, join +explain select * from src join src src2 on src.key=src2.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q b/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q new file mode 100644 index 0000000000000..9cb89da373d29 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q @@ -0,0 +1,12 @@ +CREATE TABLE table(string string) STORED AS TEXTFILE; + +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) +) subq +GROUP BY table; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q b/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q new file mode 100644 index 0000000000000..144cfeee6b14b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q @@ -0,0 +1,26 @@ +DROP TABLE insert; + +CREATE TABLE insert (key INT, as STRING); + +EXPLAIN INSERT INTO TABLE insert SELECT * FROM src LIMIT 100; +INSERT INTO TABLE insert SELECT * FROM src LIMIT 100; +SELECT SUM(HASH(hash)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (hash) FROM insert +) t; + +EXPLAIN INSERT INTO TABLE insert SELECT * FROM src LIMIT 100; +INSERT INTO TABLE insert SELECT * FROM src LIMIT 100; +SELECT SUM(HASH(sum)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (sum) FROM insert +) t; + +SELECT COUNT(*) FROM insert; + +EXPLAIN INSERT OVERWRITE TABLE insert SELECT * FROM src LIMIT 10; +INSERT OVERWRITE TABLE insert SELECT * FROM src LIMIT 10; +SELECT SUM(HASH(add)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (add) FROM insert +) t; + + +DROP TABLE insert; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/notable_alias1.q b/src/test/hive/ql/src/test/queries/clientpositive/notable_alias1.q new file mode 100644 index 0000000000000..ed7e31883af33 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/notable_alias1.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(dummy STRING, key INT, value DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', key, count(1) WHERE src.key < 100 group by key; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', key, count(1) WHERE src.key < 100 group by key; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/notable_alias2.q b/src/test/hive/ql/src/test/queries/clientpositive/notable_alias2.q new file mode 100644 index 0000000000000..b9c963c97a044 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/notable_alias2.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(dummy STRING, key INT, value DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.key, count(1) WHERE key < 100 group by src.key; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.key, count(1) WHERE key < 100 group by src.key; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/null_cast.q b/src/test/hive/ql/src/test/queries/clientpositive/null_cast.q new file mode 100644 index 0000000000000..48c39b81fdd8f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/null_cast.q @@ -0,0 +1,11 @@ +EXPLAIN SELECT ARRAY(NULL, 0), + ARRAY(NULL, ARRAY()), + ARRAY(NULL, MAP()), + ARRAY(NULL, STRUCT(0)) + FROM src LIMIT 1; + +SELECT ARRAY(NULL, 0), + ARRAY(NULL, ARRAY()), + ARRAY(NULL, MAP()), + ARRAY(NULL, STRUCT(0)) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/null_column.q b/src/test/hive/ql/src/test/queries/clientpositive/null_column.q new file mode 100644 index 0000000000000..fa4a8639446e5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/null_column.q @@ -0,0 +1,29 @@ + + + + +create table temp_null(a int) stored as textfile; +load data local inpath '../data/files/test.dat' overwrite into table temp_null; + +select null, null from temp_null; + +create table tt(a int, b string); +insert overwrite table tt select null, null from temp_null; +select * from tt; + +create table tt_b(a int, b string) row format serde "org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"; +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/*; + + +create table temp_null2 (key string, value string) partitioned by (ds string); +insert overwrite table temp_null2 partition(ds='2010-04-01') select '1',NULL from src limit 1; +select * from temp_null2 where ds is not null; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup.q b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup.q new file mode 100644 index 0000000000000..5ec67e560513e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup.q @@ -0,0 +1,31 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; + +explain +select count(1) from src x where x.key > 9999; + +select count(1) from src x where x.key > 9999; + +set hive.map.aggr=true; +set hive.groupby.skewindata=false; + +explain +select count(1) from src x where x.key > 9999; + +select count(1) from src x where x.key > 9999; + +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +explain +select count(1) from src x where x.key > 9999; + +select count(1) from src x where x.key > 9999; + +set hive.map.aggr=false; +set hive.groupby.skewindata=false; + +explain +select count(1) from src x where x.key > 9999; + +select count(1) from src x where x.key > 9999; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup2.q b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup2.q new file mode 100644 index 0000000000000..fecd6d4bd8cc8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup2.q @@ -0,0 +1,31 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; + +explain +select x.key, count(1) from src x where x.key > 9999 group by x.key; + +select x.key, count(1) from src x where x.key > 9999 group by x.key; + +set hive.map.aggr=true; +set hive.groupby.skewindata=false; + +explain +select x.key, count(1) from src x where x.key > 9999 group by x.key; + +select x.key, count(1) from src x where x.key > 9999 group by x.key; + +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +explain +select x.key, count(1) from src x where x.key > 9999 group by x.key; + +select x.key, count(1) from src x where x.key > 9999 group by x.key; + +set hive.map.aggr=false; +set hive.groupby.skewindata=false; + +explain +select x.key, count(1) from src x where x.key > 9999 group by x.key; + +select x.key, count(1) from src x where x.key > 9999 group by x.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup3.q b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup3.q new file mode 100644 index 0000000000000..a5bc9ff1f36ff --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup3.q @@ -0,0 +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'); +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'); +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'); +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'); +explain +select count(1) from tstparttbl2; +select count(1) from tstparttbl2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4.q b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4.q new file mode 100644 index 0000000000000..917d1a78ee696 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4.q @@ -0,0 +1,31 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=true; + +explain +select count(1), count(distinct x.value) from src x where x.key = 9999; + +select count(1), count(distinct x.value) from src x where x.key = 9999; + +set hive.map.aggr=true; +set hive.groupby.skewindata=false; + +explain +select count(1), count(distinct x.value) from src x where x.key = 9999; + +select count(1), count(distinct x.value) from src x where x.key = 9999; + +set hive.map.aggr=false; +set hive.groupby.skewindata=true; + +explain +select count(1), count(distinct x.value) from src x where x.key = 9999; + +select count(1), count(distinct x.value) from src x where x.key = 9999; + +set hive.map.aggr=false; +set hive.groupby.skewindata=false; + +explain +select count(1), count(distinct x.value) from src x where x.key = 9999; + +select count(1), count(distinct x.value) from src x where x.key = 9999; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q new file mode 100644 index 0000000000000..fcee62929e466 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q @@ -0,0 +1,15 @@ +set hive.map.aggr=true; +set hive.groupby.skewindata=false; + +explain +select count(1), count(distinct x.value), count(distinct substr(x.value, 5)) from src x where x.key = 9999; + +select count(1), count(distinct x.value), count(distinct substr(x.value, 5)) from src x where x.key = 9999; + +set hive.map.aggr=false; +set hive.groupby.skewindata=false; + +explain +select count(1), count(distinct x.value), count(distinct substr(x.value, 5)) from src x where x.key = 9999; + +select count(1), count(distinct x.value), count(distinct substr(x.value, 5)) from src x where x.key = 9999; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup5.q b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup5.q new file mode 100644 index 0000000000000..12773b6159a5d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nullgroup5.q @@ -0,0 +1,26 @@ + +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'); + + +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'); + +explain +select u.* from +( + select key, value from tstparttbl x where x.ds='2009-04-05' + union all + select key, value from tstparttbl2 y where y.ds='2009-04-09' +)u; + +select u.* from +( + select key, value from tstparttbl x where x.ds='2009-04-05' + union all + select key, value from tstparttbl2 y where y.ds='2009-04-09' +)u; + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullinput.q b/src/test/hive/ql/src/test/queries/clientpositive/nullinput.q new file mode 100644 index 0000000000000..4a58ed514ab17 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nullinput.q @@ -0,0 +1,4 @@ +create table tstnullinut(a string, b string); +select x.* from tstnullinut x; +select x.a, count(1) from tstnullinut x group by x.a; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullinput2.q b/src/test/hive/ql/src/test/queries/clientpositive/nullinput2.q new file mode 100644 index 0000000000000..f5c0af8fa5efa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nullinput2.q @@ -0,0 +1,8 @@ + + +create table nulltbl(key int) partitioned by (ds string); +select key from nulltbl where ds='101'; + +select count(1) from nulltbl where ds='101'; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullscript.q b/src/test/hive/ql/src/test/queries/clientpositive/nullscript.q new file mode 100644 index 0000000000000..95c9e1df37679 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/nullscript.q @@ -0,0 +1,10 @@ + +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; +explain +select transform(key) using 'cat' as key1 from nullscript; +select transform(key) using 'cat' as key1 from nullscript; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/num_op_type_conv.q b/src/test/hive/ql/src/test/queries/clientpositive/num_op_type_conv.q new file mode 100644 index 0000000000000..7f858d3e6f136 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/num_op_type_conv.q @@ -0,0 +1,9 @@ +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), + 9 % "3" FROM src LIMIT 1; + +SELECT null + 7, 1.0 - null, null + null, + CAST(21 AS BIGINT) % CAST(5 AS TINYINT), + CAST(21 AS BIGINT) % CAST(21 AS BIGINT), + 9 % "3" FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ops_comparison.q b/src/test/hive/ql/src/test/queries/clientpositive/ops_comparison.q new file mode 100644 index 0000000000000..b685ae6c4e53f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ops_comparison.q @@ -0,0 +1,23 @@ + +select 1.0 < 2.0 from src limit 1; +select 2.0 < 2.0 from src limit 1; +select 2.0 > 1.0 from src limit 1; +select 2.0 > 2.0 from src limit 1; + +select 'NaN' < 2.0 from src limit 1; +select 1.0 < 'NaN' from src limit 1; +select 1.0 > 'NaN' from src limit 1; +select 'NaN' > 2.0 from src limit 1; +select 'NaN' > 'NaN' from src limit 1; +select 'NaN' < 'NaN' from src limit 1; + +select 'NaN' = 2.0 from src limit 1; +select 1.0 = 'NaN' from src limit 1; +select 'NaN' = 2.0 from src limit 1; +select 'NaN' = 'NaN' from src limit 1; + +select 'NaN' <> 2.0 from src limit 1; +select 1.0 <> 'NaN' from src limit 1; +select 'NaN' <> 2.0 from src limit 1; +select 'NaN' <> 'NaN' from src limit 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/optional_outer.q b/src/test/hive/ql/src/test/queries/clientpositive/optional_outer.q new file mode 100644 index 0000000000000..10971ab51e328 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/optional_outer.q @@ -0,0 +1,8 @@ +EXPLAIN SELECT * FROM src a LEFT OUTER JOIN src b on (a.key=b.key); +EXPLAIN SELECT * FROM src a LEFT JOIN src b on (a.key=b.key); + +EXPLAIN SELECT * FROM src a RIGHT OUTER JOIN src b on (a.key=b.key); +EXPLAIN SELECT * FROM src a RIGHT JOIN src b on (a.key=b.key); + +EXPLAIN SELECT * FROM src a FULL OUTER JOIN src b on (a.key=b.key); +EXPLAIN SELECT * FROM src a FULL JOIN src b on (a.key=b.key); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/optrstat_groupby.q b/src/test/hive/ql/src/test/queries/clientpositive/optrstat_groupby.q new file mode 100644 index 0000000000000..5993041405ed8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/optrstat_groupby.q @@ -0,0 +1,6 @@ +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/src/test/hive/ql/src/test/queries/clientpositive/orc_create.q b/src/test/hive/ql/src/test/queries/clientpositive/orc_create.q new file mode 100644 index 0000000000000..6aca5486445c8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/orc_create.q @@ -0,0 +1,103 @@ +DROP TABLE orc_create; +DROP TABLE orc_create_complex; +DROP TABLE orc_create_staging; +DROP TABLE orc_create_people_staging; +DROP TABLE orc_create_people; + +CREATE TABLE orc_create_staging ( + str STRING, + mp MAP, + lst ARRAY, + strct STRUCT +) ROW FORMAT DELIMITED + FIELDS TERMINATED BY '|' + COLLECTION ITEMS TERMINATED BY ',' + MAP KEYS TERMINATED BY ':'; + +DESCRIBE FORMATTED orc_create_staging; + +CREATE TABLE orc_create (key INT, value STRING) + PARTITIONED BY (ds string) + STORED AS ORC; + +DESCRIBE FORMATTED orc_create; + +DROP TABLE orc_create; + +CREATE TABLE orc_create (key INT, value STRING) + PARTITIONED BY (ds string); + +DESCRIBE FORMATTED orc_create; + +ALTER TABLE orc_create SET FILEFORMAT ORC; + +DESCRIBE FORMATTED orc_create; + +DROP TABLE orc_create; + +set hive.default.fileformat=orc; + +CREATE TABLE orc_create (key INT, value STRING) + PARTITIONED BY (ds string); + +set hive.default.fileformat=text; + +DESCRIBE FORMATTED orc_create; + +CREATE TABLE orc_create_complex ( + str STRING, + mp MAP, + lst ARRAY, + strct STRUCT +) STORED AS ORC; + +DESCRIBE FORMATTED orc_create_complex; + +LOAD DATA LOCAL INPATH '../data/files/orc_create.txt' OVERWRITE INTO TABLE orc_create_staging; + +SELECT * from orc_create_staging; + +INSERT OVERWRITE TABLE orc_create_complex SELECT * FROM orc_create_staging; + +SELECT * from orc_create_complex; +SELECT str from orc_create_complex; +SELECT mp from orc_create_complex; +SELECT lst from orc_create_complex; +SELECT strct from orc_create_complex; + +CREATE TABLE orc_create_people_staging ( + id int, + first_name string, + last_name string, + address string, + state string); + +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) +PARTITIONED BY (state string) +STORED AS orc; + +set hive.exec.dynamic.partition.mode=nonstrict; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging; + +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 with no column projection +SELECT id, first_name, last_name, address + FROM orc_create_people WHERE id > 90; + +DROP TABLE orc_create; +DROP TABLE orc_create_complex; +DROP TABLE orc_create_staging; +DROP TABLE orc_create_people_staging; +DROP TABLE orc_create_people; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_createas1.q b/src/test/hive/ql/src/test/queries/clientpositive/orc_createas1.q new file mode 100644 index 0000000000000..872692567b37d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/orc_createas1.q @@ -0,0 +1,48 @@ +set mapred.max.split.size=100; +set mapred.min.split.size=1; + +DROP TABLE orc_createas1a; +DROP TABLE orc_createas1b; +DROP TABLE orc_createas1c; + +CREATE TABLE orc_createas1a (key INT, value STRING) + PARTITIONED BY (ds string); +INSERT OVERWRITE TABLE orc_createas1a PARTITION (ds='1') + SELECT * FROM src; +INSERT OVERWRITE TABLE orc_createas1a PARTITION (ds='2') + SELECT * FROM src; + +EXPLAIN CREATE TABLE orc_createas1b + STORED AS ORC AS + SELECT * FROM src; + +CREATE TABLE orc_createas1b + STORED AS ORC AS + SELECT * FROM src; + +EXPLAIN SELECT * FROM orc_createas1b ORDER BY key LIMIT 5; + +SELECT * FROM orc_createas1b ORDER BY key LIMIT 5; + +EXPLAIN + CREATE TABLE orc_createas1c + STORED AS ORC AS + SELECT key, value, PMOD(HASH(key), 50) as part + FROM orc_createas1a; +CREATE TABLE orc_createas1c + STORED AS ORC AS + SELECT key, value, PMOD(HASH(key), 50) as part + FROM orc_createas1a; + +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(key, value) USING 'tr \t _' AS (c) + FROM orc_createas1a +) t; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(key, value) USING 'tr \t _' AS (c) + FROM orc_createas1c +) t; + +DROP TABLE orc_createas1a; +DROP TABLE orc_createas1b; +DROP TABLE orc_createas1c; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q b/src/test/hive/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q new file mode 100644 index 0000000000000..f916012b8365e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q @@ -0,0 +1,60 @@ +set hive.exec.orc.dictionary.key.size.threshold=-1; + +-- Set the threshold to -1 to guarantee dictionary encoding is turned off +-- Tests that the data can be read back correctly when a string column is stored +-- without dictionary encoding + +CREATE TABLE test_orc (key STRING) +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'; + +-- should be single split +INSERT OVERWRITE TABLE test_orc SELECT key FROM src TABLESAMPLE (10 ROWS); + +-- Test reading the column back + +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' + INTO TABLE src_thousand; + +set hive.exec.orc.dictionary.key.size.threshold=0.5; + +-- Add data to the table in such a way that alternate stripes encode the column +-- differently. Setting orc.stripe.size = 1 guarantees the stripes each have +-- 5000 rows. The first stripe will have 5 * 630 distinct rows and thus be +-- above the cutoff of 50% and will be direct encoded. The second stripe +-- will have 5 * 1 distinct rows and thus be under the cutoff and will be +-- dictionary encoded. The final stripe will have 630 out of 1000 and be +-- direct encoded. + +INSERT OVERWRITE TABLE test_orc +SELECT key FROM ( +SELECT CONCAT("a", key) AS key FROM src_thousand +UNION ALL +SELECT CONCAT("b", key) AS key FROM src_thousand +UNION ALL +SELECT CONCAT("c", key) AS key FROM src_thousand +UNION ALL +SELECT CONCAT("d", key) AS key FROM src_thousand +UNION ALL +SELECT CONCAT("e", key) AS key FROM src_thousand +UNION ALL +SELECT CONCAT("f", 1) AS key FROM src_thousand +UNION ALL +SELECT CONCAT("g", 1) AS key FROM src_thousand +UNION ALL +SELECT CONCAT("h", 1) AS key FROM src_thousand +UNION ALL +SELECT CONCAT("i", 1) AS key FROM src_thousand +UNION ALL +SELECT CONCAT("j", 1) AS key FROM src_thousand +UNION ALL +SELECT CONCAT("k", key) AS key FROM src_thousand +) a ORDER BY key LIMIT 11000; + +SELECT SUM(HASH(key)) FROM test_orc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_diff_part_cols.q b/src/test/hive/ql/src/test/queries/clientpositive/orc_diff_part_cols.q new file mode 100644 index 0000000000000..cbfd7b359f878 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/orc_diff_part_cols.q @@ -0,0 +1,19 @@ +CREATE TABLE test_orc (key STRING) +PARTITIONED BY (part STRING) +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'; + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +-- Create a table with one column write to a partition, then add an additional column and write +-- to another partition +-- This can produce unexpected results with CombineHiveInputFormat + +INSERT OVERWRITE TABLE test_orc PARTITION (part = '1') SELECT key FROM src LIMIT 5; + +ALTER TABLE test_orc ADD COLUMNS (cnt INT); + +INSERT OVERWRITE TABLE test_orc PARTITION (part = '2') SELECT key, count(*) FROM src GROUP BY key LIMIT 5; + +SELECT * FROM test_orc ORDER BY key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_files.q b/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_files.q new file mode 100644 index 0000000000000..d3cbc5a372955 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_files.q @@ -0,0 +1,18 @@ +CREATE TABLE test_orc (key STRING, cnt INT) +CLUSTERED BY (key) INTO 3 BUCKETS +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'; + +set hive.enforce.bucketing=true; +set hive.exec.reducers.max = 1; +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +-- Creates a table bucketed into 3 buckets, but only one contains data, specifically bucket 1, +-- buckets 0 and 2 are empty, so this tests reading from and empty file followed by a file +-- containing data and a file containing data followed by an empty file. +-- This can produce unexpected results with CombineHiveInputFormat + +INSERT OVERWRITE TABLE test_orc SELECT one, COUNT(*) FROM (SELECT 1 AS one FROM src) a GROUP BY one; + +SELECT count(*) FROM test_orc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_strings.q b/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_strings.q new file mode 100644 index 0000000000000..0ef57d18ccaf7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_strings.q @@ -0,0 +1,16 @@ +CREATE TABLE test_orc (key STRING) +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; + +-- 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; + +-- Test reading a column which has some empty strings + +SELECT * FROM test_orc ORDER BY key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q b/src/test/hive/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q new file mode 100644 index 0000000000000..6685da7a82245 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q @@ -0,0 +1,17 @@ +CREATE TABLE test_orc (key STRING) +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'; + +ALTER TABLE test_orc SET SERDEPROPERTIES ('orc.row.index.stride' = '1000'); + +-- nulls.txt is a file containing a non-null string row followed by 1000 null string rows +-- this produces the effect that the number of non-null rows between the last and second +-- 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; + +INSERT OVERWRITE TABLE test_orc SELECT a FROM src_null; + +SELECT * FROM test_orc LIMIT 5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q b/src/test/hive/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q new file mode 100644 index 0000000000000..f5f25f00c951a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q @@ -0,0 +1,248 @@ +CREATE TABLE orc_pred(t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary) +STORED AS ORC; + +ALTER TABLE orc_pred SET SERDEPROPERTIES ('orc.row.index.stride' = '1000'); + +CREATE TABLE staging(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 '|' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../data/files/over1k' OVERWRITE INTO TABLE staging; + +INSERT INTO TABLE orc_pred select * from staging; + +-- no predicate case. the explain plan should not have filter expression in table scan operator + +SELECT SUM(HASH(t)) FROM orc_pred; + +SET hive.optimize.index.filter=true; +SELECT SUM(HASH(t)) FROM orc_pred; +SET hive.optimize.index.filter=false; + +EXPLAIN SELECT SUM(HASH(t)) FROM orc_pred; + +SET hive.optimize.index.filter=true; +EXPLAIN SELECT SUM(HASH(t)) FROM orc_pred; +SET hive.optimize.index.filter=false; + +-- all the following queries have predicates which are pushed down to table scan operator if +-- hive.optimize.index.filter is set to true. the explain plan should show filter expression +-- in table scan operator. + +SELECT * FROM orc_pred WHERE t<2 limit 1; +SET hive.optimize.index.filter=true; +SELECT * FROM orc_pred WHERE t<2 limit 1; +SET hive.optimize.index.filter=false; + +SELECT * FROM orc_pred WHERE t>2 limit 1; +SET hive.optimize.index.filter=true; +SELECT * FROM orc_pred WHERE t>2 limit 1; +SET hive.optimize.index.filter=false; + +SELECT SUM(HASH(t)) FROM orc_pred + WHERE t IS NOT NULL + AND t < 0 + AND t > -2; + +SET hive.optimize.index.filter=true; +SELECT SUM(HASH(t)) FROM orc_pred + WHERE t IS NOT NULL + AND t < 0 + AND t > -2; +SET hive.optimize.index.filter=false; + +EXPLAIN SELECT SUM(HASH(t)) FROM orc_pred + WHERE t IS NOT NULL + AND t < 0 + AND t > -2; + +SET hive.optimize.index.filter=true; +EXPLAIN SELECT SUM(HASH(t)) FROM orc_pred + WHERE t IS NOT NULL + AND t < 0 + AND t > -2; +SET hive.optimize.index.filter=false; + +SELECT t, s FROM orc_pred + WHERE t <=> -1 + AND s IS NOT NULL + AND s LIKE 'bob%' + ORDER BY s; + +SET hive.optimize.index.filter=true; +SELECT t, s FROM orc_pred + WHERE t <=> -1 + AND s IS NOT NULL + AND s LIKE 'bob%' + ORDER BY s; +SET hive.optimize.index.filter=false; + +EXPLAIN SELECT t, s FROM orc_pred + WHERE t <=> -1 + AND s IS NOT NULL + AND s LIKE 'bob%' + ORDER BY s; + +SET hive.optimize.index.filter=true; +EXPLAIN SELECT t, s FROM orc_pred + WHERE t <=> -1 + AND s IS NOT NULL + AND s LIKE 'bob%' + ORDER BY s; +SET hive.optimize.index.filter=false; + +SELECT t, s FROM orc_pred + WHERE s IS NOT NULL + AND s LIKE 'bob%' + AND t NOT IN (-1,-2,-3) + AND t BETWEEN 25 AND 30 + SORT BY t,s; + +set hive.optimize.index.filter=true; +SELECT t, s FROM orc_pred + WHERE s IS NOT NULL + AND s LIKE 'bob%' + AND t NOT IN (-1,-2,-3) + AND t BETWEEN 25 AND 30 + SORT BY t,s; +set hive.optimize.index.filter=false; + +EXPLAIN SELECT t, s FROM orc_pred + WHERE s IS NOT NULL + AND s LIKE 'bob%' + AND t NOT IN (-1,-2,-3) + AND t BETWEEN 25 AND 30 + SORT BY t,s; + +SET hive.optimize.index.filter=true; +EXPLAIN SELECT t, s FROM orc_pred + WHERE s IS NOT NULL + AND s LIKE 'bob%' + AND t NOT IN (-1,-2,-3) + AND t BETWEEN 25 AND 30 + SORT BY t,s; +SET hive.optimize.index.filter=false; + +SELECT t, si, d, s FROM orc_pred + WHERE d >= ROUND(9.99) + AND d < 12 + AND t IS NOT NULL + AND s LIKE '%son' + AND s NOT LIKE '%car%' + AND t > 0 + AND si BETWEEN 300 AND 400 + ORDER BY s DESC + LIMIT 3; + +SET hive.optimize.index.filter=true; +SELECT t, si, d, s FROM orc_pred + WHERE d >= ROUND(9.99) + AND d < 12 + AND t IS NOT NULL + AND s LIKE '%son' + AND s NOT LIKE '%car%' + AND t > 0 + AND si BETWEEN 300 AND 400 + ORDER BY s DESC + LIMIT 3; +SET hive.optimize.index.filter=false; + +EXPLAIN SELECT t, si, d, s FROM orc_pred + WHERE d >= ROUND(9.99) + AND d < 12 + AND t IS NOT NULL + AND s LIKE '%son' + AND s NOT LIKE '%car%' + AND t > 0 + AND si BETWEEN 300 AND 400 + ORDER BY s DESC + LIMIT 3; + +SET hive.optimize.index.filter=true; +EXPLAIN SELECT t, si, d, s FROM orc_pred + WHERE d >= ROUND(9.99) + AND d < 12 + AND t IS NOT NULL + AND s LIKE '%son' + AND s NOT LIKE '%car%' + AND t > 0 + AND si BETWEEN 300 AND 400 + ORDER BY s DESC + LIMIT 3; +SET hive.optimize.index.filter=false; + +SELECT t, si, d, s FROM orc_pred + WHERE t > 10 + AND t <> 101 + AND d >= ROUND(9.99) + AND d < 12 + AND t IS NOT NULL + AND s LIKE '%son' + AND s NOT LIKE '%car%' + AND t > 0 + AND si BETWEEN 300 AND 400 + SORT BY s DESC + LIMIT 3; + +SET hive.optimize.index.filter=true; +SELECT t, si, d, s FROM orc_pred + WHERE t > 10 + AND t <> 101 + AND d >= ROUND(9.99) + AND d < 12 + AND t IS NOT NULL + AND s LIKE '%son' + AND s NOT LIKE '%car%' + AND t > 0 + AND si BETWEEN 300 AND 400 + SORT BY s DESC + LIMIT 3; +SET hive.optimize.index.filter=false; + +EXPLAIN SELECT t, si, d, s FROM orc_pred + WHERE t > 10 + AND t <> 101 + AND d >= ROUND(9.99) + AND d < 12 + AND t IS NOT NULL + AND s LIKE '%son' + AND s NOT LIKE '%car%' + AND t > 0 + AND si BETWEEN 300 AND 400 + SORT BY s DESC + LIMIT 3; + +SET hive.optimize.index.filter=true; +EXPLAIN SELECT t, si, d, s FROM orc_pred + WHERE t > 10 + AND t <> 101 + AND d >= ROUND(9.99) + AND d < 12 + AND t IS NOT NULL + AND s LIKE '%son' + AND s NOT LIKE '%car%' + AND t > 0 + AND si BETWEEN 300 AND 400 + SORT BY s DESC + LIMIT 3; +SET hive.optimize.index.filter=false; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/order.q b/src/test/hive/ql/src/test/queries/clientpositive/order.q new file mode 100644 index 0000000000000..d389892eda48c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/order.q @@ -0,0 +1,9 @@ +EXPLAIN +SELECT x.* FROM SRC x ORDER BY key limit 10; + +SELECT x.* FROM SRC x ORDER BY key limit 10; + +EXPLAIN +SELECT x.* FROM SRC x ORDER BY key desc limit 10; + +SELECT x.* FROM SRC x ORDER BY key desc limit 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/order2.q b/src/test/hive/ql/src/test/queries/clientpositive/order2.q new file mode 100644 index 0000000000000..6c16fe30fa7ec --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/order2.q @@ -0,0 +1,10 @@ +set hive.optimize.ppd=true; + +EXPLAIN +SELECT subq.key, subq.value FROM +(SELECT x.* FROM SRC x ORDER BY key limit 10) subq +where subq.key < 10; + +SELECT subq.key, subq.value FROM +(SELECT x.* FROM SRC x ORDER BY key limit 10) subq +where subq.key < 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/outer_join_ppr.q b/src/test/hive/ql/src/test/queries/clientpositive/outer_join_ppr.q new file mode 100644 index 0000000000000..38007cab5f679 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/outer_join_ppr.q @@ -0,0 +1,35 @@ +set hive.optimize.ppd=true; + +EXPLAIN EXTENDED + FROM + src a + FULL OUTER JOIN + srcpart b + ON (a.key = b.key AND b.ds = '2008-04-08') + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + + FROM + src a + FULL OUTER JOIN + srcpart b + ON (a.key = b.key AND b.ds = '2008-04-08') + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + +EXPLAIN EXTENDED + FROM + src a + FULL OUTER JOIN + srcpart b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25 AND b.ds = '2008-04-08'; + + FROM + src a + FULL OUTER JOIN + srcpart b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25 AND b.ds = '2008-04-08'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/overridden_confs.q b/src/test/hive/ql/src/test/queries/clientpositive/overridden_confs.q new file mode 100644 index 0000000000000..9dcaed6acb12f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/overridden_confs.q @@ -0,0 +1,4 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyOverriddenConfigsHook; +set hive.config.doesnt.exit=abc; + +select count(*) from src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/parallel.q b/src/test/hive/ql/src/test/queries/clientpositive/parallel.q new file mode 100644 index 0000000000000..03edeaadeef51 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/parallel.q @@ -0,0 +1,28 @@ +set mapred.job.name='test_parallel'; +set hive.exec.parallel=true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +create table if not exists src_a like src; +create table if not exists src_b like src; + +explain +from (select key, value from src group by key, value) s +insert overwrite table src_a select s.key, s.value group by s.key, s.value +insert overwrite table src_b select s.key, s.value group by s.key, s.value; + +from (select key, value from src group by key, value) s +insert overwrite table src_a select s.key, s.value group by s.key, s.value +insert overwrite table src_b select s.key, s.value group by s.key, s.value; + +select * from src_a order by key, value; +select * from src_b order by key, value; + + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +from (select key, value from src group by key, value) s +insert overwrite table src_a select s.key, s.value group by s.key, s.value +insert overwrite table src_b select s.key, s.value group by s.key, s.value; + +select * from src_a order by key, value; +select * from src_b order by key, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/parallel_orderby.q b/src/test/hive/ql/src/test/queries/clientpositive/parallel_orderby.q new file mode 100644 index 0000000000000..5e09395901c08 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/parallel_orderby.q @@ -0,0 +1,23 @@ +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; + +set mapred.reduce.tasks = 4; +set hive.optimize.sampling.orderby=true; +set hive.optimize.sampling.orderby.percent=0.66f; + +explain +create table total_ordered as select * from src5 order by key, value; +create table total_ordered as select * from src5 order by key, value; + +desc formatted total_ordered; +select * from total_ordered; + +set hive.optimize.sampling.orderby.percent=0.0001f; +-- rolling back to single task in case that the number of sample is not enough + +drop table total_ordered; +create table total_ordered as select * from src5 order by key, value; + +desc formatted total_ordered; +select * from total_ordered; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/parenthesis_star_by.q b/src/test/hive/ql/src/test/queries/clientpositive/parenthesis_star_by.q new file mode 100644 index 0000000000000..9e036c1a91d3b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/parenthesis_star_by.q @@ -0,0 +1,10 @@ +SELECT key, value FROM src CLUSTER BY key, value; +SELECT key, value FROM src ORDER BY key ASC, value ASC; +SELECT key, value FROM src SORT BY key, value; +SELECT * FROM (SELECT key, value FROM src DISTRIBUTE BY key, value)t ORDER BY key, value; + + +SELECT key, value FROM src CLUSTER BY (key, value); +SELECT key, value FROM src ORDER BY (key ASC, value ASC); +SELECT key, value FROM src SORT BY (key, value); +SELECT * FROM (SELECT key, value FROM src DISTRIBUTE BY (key, value))t ORDER BY key, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partInit.q b/src/test/hive/ql/src/test/queries/clientpositive/partInit.q new file mode 100644 index 0000000000000..c492b87e0e96e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partInit.q @@ -0,0 +1,7 @@ +CREATE TABLE empty (c INT) PARTITIONED BY (p INT); +SELECT MAX(c) FROM empty; +SELECT MAX(p) FROM empty; + +ALTER TABLE empty ADD PARTITION (p=1); +SELECT MAX(p) FROM empty; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q b/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q new file mode 100644 index 0000000000000..3ee1b4ac80e32 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q @@ -0,0 +1,8 @@ +set hive.metastore.partition.inherit.table.properties=a,b; +-- The property needs to be unset at the end of the test till HIVE-3109/HIVE-3112 is fixed + +create table mytbl (c1 tinyint) partitioned by (c2 string) tblproperties ('a'='myval','b'='yourval','c'='noval'); +alter table mytbl add partition (c2 = 'v1'); +describe formatted mytbl partition (c2='v1'); + +set hive.metastore.partition.inherit.table.properties=; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q b/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q new file mode 100644 index 0000000000000..f3f0335c883d3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q @@ -0,0 +1,4 @@ +set hive.metastore.partition.inherit.table.properties=""; +create table mytbl (c1 tinyint) partitioned by (c2 string) tblproperties ('a'='myval','b'='yourval','c'='noval'); +alter table mytbl add partition (c2 = 'v1'); +describe formatted mytbl partition (c2='v1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q b/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q new file mode 100644 index 0000000000000..8dac6111ec059 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q @@ -0,0 +1,8 @@ +set hive.metastore.partition.inherit.table.properties=key1,*; +-- The property needs to be unset at the end of the test till HIVE-3109/HIVE-3112 is fixed + +create table mytbl (c1 tinyint) partitioned by (c2 string) tblproperties ('a'='myval','b'='yourval','c'='noval'); +alter table mytbl add partition (c2 = 'v1'); +describe formatted mytbl partition (c2='v1'); + +set hive.metastore.partition.inherit.table.properties=; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partcols1.q b/src/test/hive/ql/src/test/queries/clientpositive/partcols1.q new file mode 100644 index 0000000000000..b7f8c64d4261f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partcols1.q @@ -0,0 +1,18 @@ + +create table test1(col1 string) partitioned by (partitionId int); +insert overwrite table test1 partition (partitionId=1) + select key from src limit 10; + + FROM ( + FROM test1 + SELECT partitionId, 111 as col2, 222 as col3, 333 as col4 + WHERE partitionId = 1 + DISTRIBUTE BY partitionId + SORT BY partitionId + ) b + +SELECT TRANSFORM( + b.partitionId,b.col2,b.col3,b.col4 + ) + + USING 'cat' as (a,b,c,d); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_date.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_date.q new file mode 100644 index 0000000000000..8738afdfa099a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_date.q @@ -0,0 +1,45 @@ +drop table partition_date_1; + +create table partition_date_1 (key string, value string) partitioned by (dt date, region int); + +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; + +-- 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; +-- 11 +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; +-- 0 +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; +-- 10 +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; +-- 10 +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; + +drop table partition_date_1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_date2.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_date2.q new file mode 100644 index 0000000000000..9b84b59608503 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_date2.q @@ -0,0 +1,55 @@ +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 +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 *; + +select distinct dt from partition_date2_1; +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 * from partition_date2_1; + +-- truncate +truncate table partition_date2_1 partition(dt=date '2000-01-01', region=2); +select distinct dt from partition_date2_1; +select * from partition_date2_1; + +-- alter table add partition +alter table partition_date2_1 add partition (dt=date '1980-01-02', region=3); +select distinct dt from partition_date2_1; +select * from partition_date2_1; + +-- alter table drop +alter table partition_date2_1 drop partition (dt=date '1999-01-01', region=2); +select distinct dt from partition_date2_1; +select * from partition_date2_1; + +-- alter table set serde +alter table partition_date2_1 partition(dt=date '1980-01-02', region=3) + set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; + +-- alter table set fileformat +alter table partition_date2_1 partition(dt=date '1980-01-02', region=3) + set fileformat rcfile; +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 partition_date2_1 order by key,value,dt,region; + +-- alter table set location +alter table partition_date2_1 partition(dt=date '1980-01-02', region=3) + set location "file:///tmp/partition_date2_1"; +describe extended partition_date2_1 partition(dt=date '1980-01-02', region=3); + +-- alter table touch +alter table partition_date2_1 touch partition(dt=date '1980-01-02', region=3); + +drop table partition_date2_1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_decode_name.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_decode_name.q new file mode 100644 index 0000000000000..ba193cd51a26d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_decode_name.q @@ -0,0 +1,21 @@ +create table sc as select * +from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 + union all + select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 + union all + select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s; + +create table sc_part (key string) partitioned by (ts string) stored as rcfile; + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +set hive.decode.partition.name=false; +insert overwrite table sc_part partition(ts) select * from sc; +show partitions sc_part; +select count(*) from sc_part where ts is not null; + +set hive.decode.partition.name=true; +insert overwrite table sc_part partition(ts) select * from sc; +show partitions sc_part; +select count(*) from sc_part where ts is not null; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_schema1.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_schema1.q new file mode 100644 index 0000000000000..550ea47beb0db --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_schema1.q @@ -0,0 +1,12 @@ + +create table partition_schema1(key string, value string) partitioned by (dt string); + +insert overwrite table partition_schema1 partition(dt='100') select * from src1; +desc partition_schema1 partition(dt='100'); + +alter table partition_schema1 add columns (x string); + +desc partition_schema1; +desc partition_schema1 partition (dt='100'); + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_serde_format.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_serde_format.q new file mode 100644 index 0000000000000..9f7620c148236 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_serde_format.q @@ -0,0 +1,4 @@ +create table src_part_serde (key int, value string) partitioned by (ds string) stored as sequencefile; +insert overwrite table src_part_serde partition (ds='2011') select * from src; +alter table src_part_serde set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' with SERDEPROPERTIES ('serialization.format'='\t'); +select key, value from src_part_serde where ds='2011' order by key, value limit 20; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_special_char.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_special_char.q new file mode 100644 index 0000000000000..81344334dfe01 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_special_char.q @@ -0,0 +1,19 @@ +create table sc as select * +from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 + union all + select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 + union all + select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s; + +create table sc_part (key string) partitioned by (ts string) stored as rcfile; + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +insert overwrite table sc_part partition(ts) select * from sc; +show partitions sc_part; +select count(*) from sc_part where ts is not null; + +insert overwrite table sc_part partition(ts) select * from sc; +show partitions sc_part; +select count(*) from sc_part where ts is not null; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_type_check.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_type_check.q new file mode 100644 index 0000000000000..7f1accadac6ea --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_type_check.q @@ -0,0 +1,24 @@ +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); + +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'); + +select * from tab1; +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'); + +select id1, id2, day from tab1 where day='2008-01-01'; +drop table tab1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_varchar1.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_varchar1.q new file mode 100644 index 0000000000000..d700b1cbf8566 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_varchar1.q @@ -0,0 +1,43 @@ +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; +insert overwrite table partition_varchar_1 partition(dt='2000-01-01', region=2) + select * from src limit 5; +insert overwrite table partition_varchar_1 partition(dt='2013-08-08', region=1) + select * from src limit 20; +insert overwrite table partition_varchar_1 partition(dt='2013-08-08', region=10) + select * from src limit 11; + +select distinct dt from partition_varchar_1; +select * from partition_varchar_1 where dt = '2000-01-01' and region = 2 order by key,value; + +-- 15 +select count(*) from partition_varchar_1 where dt = '2000-01-01'; +-- 5 +select count(*) from partition_varchar_1 where dt = '2000-01-01' and region = 2; +-- 11 +select count(*) from partition_varchar_1 where dt = '2013-08-08' and region = 10; +-- 30 +select count(*) from partition_varchar_1 where region = 1; +-- 0 +select count(*) from partition_varchar_1 where dt = '2000-01-01' and region = 3; +-- 0 +select count(*) from partition_varchar_1 where dt = '1999-01-01'; + +-- Try other comparison operations + +-- 20 +select count(*) from partition_varchar_1 where dt > '2000-01-01' and region = 1; +-- 10 +select count(*) from partition_varchar_1 where dt < '2000-01-02' and region = 1; +-- 20 +select count(*) from partition_varchar_1 where dt >= '2000-01-02' and region = 1; +-- 10 +select count(*) from partition_varchar_1 where dt <= '2000-01-01' and region = 1; +-- 20 +select count(*) from partition_varchar_1 where dt <> '2000-01-01' and region = 1; + +drop table partition_varchar_1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q new file mode 100644 index 0000000000000..9f493e561f1f8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q @@ -0,0 +1,13 @@ + + +create table partition_vs_table(key string, value string) partitioned by (ds string); + +insert overwrite table partition_vs_table partition(ds='100') select key, value from src; + +alter table partition_vs_table add columns (newcol string); + +insert overwrite table partition_vs_table partition(ds='101') select key, value, key from src; + +select key, value, newcol from partition_vs_table +order by key, value, newcol; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat.q new file mode 100644 index 0000000000000..f0d58cf61d714 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat.q @@ -0,0 +1,32 @@ + + +create table partition_test_partitioned(key string, value string) partitioned by (dt string); + +insert overwrite table partition_test_partitioned partition(dt=100) select * from src1; +show table extended like partition_test_partitioned; +show table extended like partition_test_partitioned partition(dt=100); +select key from partition_test_partitioned where dt=100; +select key from partition_test_partitioned; + +alter table partition_test_partitioned set fileformat rcfile; +insert overwrite table partition_test_partitioned partition(dt=101) select * from src1; +show table extended like partition_test_partitioned; +show table extended like partition_test_partitioned partition(dt=100); +show table extended like partition_test_partitioned partition(dt=101); +select key from partition_test_partitioned where dt=100; +select key from partition_test_partitioned where dt=101; +select key from partition_test_partitioned; + +alter table partition_test_partitioned set fileformat Sequencefile; +insert overwrite table partition_test_partitioned partition(dt=102) select * from src1; +show table extended like partition_test_partitioned; +show table extended like partition_test_partitioned partition(dt=100); +show table extended like partition_test_partitioned partition(dt=101); +show table extended like partition_test_partitioned partition(dt=102); +select key from partition_test_partitioned where dt=100; +select key from partition_test_partitioned where dt=101; +select key from partition_test_partitioned where dt=102; +select key from partition_test_partitioned; + +select key from partition_test_partitioned where dt >=100 and dt <= 102; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q new file mode 100644 index 0000000000000..f15f72c08eb34 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q @@ -0,0 +1,13 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +-- This tests that the schema can be changed for binary serde data +create table prt(key string, value string) partitioned by (dt string); +insert overwrite table prt partition(dt='1') select * from src where key = 238; + +select * from prt where dt is not null; +select key+key, value from prt where dt is not null; + +alter table prt add columns (value2 string); + +select key+key, value from prt where dt is not null; +select * from prt where dt is not null; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q new file mode 100644 index 0000000000000..1a4291fe64f28 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q @@ -0,0 +1,19 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +-- This tests that the schema can be changed for binary serde data +create table partition_test_partitioned(key string, value string) partitioned by (dt string) stored as rcfile; +alter table partition_test_partitioned set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; +insert overwrite table partition_test_partitioned partition(dt='1') select * from src where key = 238; + +select * from partition_test_partitioned where dt is not null; +select key+key, value from partition_test_partitioned where dt is not null; + +alter table partition_test_partitioned change key key int; + +select key+key, value from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; + +alter table partition_test_partitioned add columns (value2 string); + +select key+key, value from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q new file mode 100644 index 0000000000000..bc51cb5587b16 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q @@ -0,0 +1,26 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +-- This tests that the schema can be changed for binary serde data +create table partition_test_partitioned(key string, value string) partitioned by (dt string) stored as rcfile; +alter table partition_test_partitioned set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; +insert overwrite table partition_test_partitioned partition(dt='1') select * from src where key = 238; + +select * from partition_test_partitioned where dt is not null; +select key+key, value from partition_test_partitioned where dt is not null; + +alter table partition_test_partitioned change key key int; + +select key+key, value from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; + +insert overwrite table partition_test_partitioned partition(dt='2') select * from src where key = 97; + +alter table partition_test_partitioned add columns (value2 string); + +select key+key, value from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; + +insert overwrite table partition_test_partitioned partition(dt='3') select key, value, value from src where key = 200; + +select key+key, value, value2 from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q new file mode 100644 index 0000000000000..2e4ae6954bcb5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q @@ -0,0 +1,17 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +-- This tests that the schema can be changed for partitioned tables for binary serde data for joins +create table T1(key string, value string) partitioned by (dt string) stored as rcfile; +alter table T1 set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; +insert overwrite table T1 partition (dt='1') select * from src where key = 238 or key = 97; + +alter table T1 change key key int; +insert overwrite table T1 partition (dt='2') select * from src where key = 238 or key = 97; + +alter table T1 change key key string; + +create table T2(key string, value string) partitioned by (dt string) stored as rcfile; +insert overwrite table T2 partition (dt='1') select * from src where key = 238 or key = 97; + +select /* + MAPJOIN(a) */ count(*) FROM T1 a JOIN T2 b ON a.key = b.key; +select count(*) FROM T1 a JOIN T2 b ON a.key = b.key; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q new file mode 100644 index 0000000000000..f4d4d73745bf5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q @@ -0,0 +1,57 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +CREATE TABLE tbl1(key int, value string) PARTITIONED by (ds string) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS rcfile; +CREATE TABLE tbl2(key int, value string) PARTITIONED by (ds string) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS rcfile; + +alter table tbl1 set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; +alter table tbl2 set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; + +insert overwrite table tbl1 partition (ds='1') select * from src where key < 10; +insert overwrite table tbl2 partition (ds='1') select * from src where key < 10; + +alter table tbl1 change key key int; +insert overwrite table tbl1 partition (ds='2') select * from src where key < 10; + +alter table tbl1 change key key string; + +-- The subquery itself is being map-joined. Multiple partitions of tbl1 with different schemas are being read for tbl2 +select /*+mapjoin(subq1)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +set hive.optimize.bucketmapjoin = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +-- The subquery itself is being map-joined. Since the sub-query only contains selects and filters, it should +-- be converted to a bucketized mapside join. Multiple partitions of tbl1 with different schemas are being read for each +-- bucket of tbl2 +select /*+mapjoin(subq1)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +set hive.optimize.bucketmapjoin.sortedmerge = true; + +-- The subquery itself is being map-joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join. Multiple partitions of tbl1 with different schemas are being read for a +-- given file of tbl2 +select /*+mapjoin(subq1)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +-- Since the join key is modified by the sub-query, neither sort-merge join not bucketized map-side +-- join should be performed. Multiple partitions of tbl1 with different schemas are being read for tbl2 +select /*+mapjoin(subq1)*/ count(*) from + (select a.key+1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key+1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q new file mode 100644 index 0000000000000..6fce1e0c77a4f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q @@ -0,0 +1,26 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +-- This tests that the schema can be changed for binary serde data +create table partition_test_partitioned(key string, value string) +partitioned by (dt string) stored as rcfile; +insert overwrite table partition_test_partitioned partition(dt='1') +select * from src where key = 238; + +select * from partition_test_partitioned where dt is not null; +select key+key, value from partition_test_partitioned where dt is not null; + +alter table partition_test_partitioned change key key int; + +select key+key, value from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; + +alter table partition_test_partitioned add columns (value2 string); + +select key+key, value from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; + +insert overwrite table partition_test_partitioned partition(dt='2') +select key, value, value from src where key = 86; + +select key+key, value, value2, dt from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q new file mode 100644 index 0000000000000..37bb1a76b258e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q @@ -0,0 +1,26 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +-- This tests that the schema can be changed for binary serde data +create table partition_test_partitioned(key string, value string) +partitioned by (dt string) stored as textfile; +insert overwrite table partition_test_partitioned partition(dt='1') +select * from src where key = 238; + +select * from partition_test_partitioned where dt is not null; +select key+key, value from partition_test_partitioned where dt is not null; + +alter table partition_test_partitioned change key key int; + +select key+key, value from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; + +alter table partition_test_partitioned add columns (value2 string); + +select key+key, value from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; + +insert overwrite table partition_test_partitioned partition(dt='2') +select key, value, value from src where key = 86; + +select key+key, value, value2, dt from partition_test_partitioned where dt is not null; +select * from partition_test_partitioned where dt is not null; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q new file mode 100644 index 0000000000000..e9b574c1ca380 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q @@ -0,0 +1,34 @@ +-- HIVE-5199, HIVE-5285 : CustomSerDe(1, 2, 3) are used here. +-- The final results should be all NULL columns deserialized using +-- CustomSerDe(1, 2, 3) irrespective of the inserted values + +DROP TABLE PW17; +ADD JAR ../build/ql/test/test-serdes.jar; +CREATE TABLE PW17(USER STRING, COMPLEXDT ARRAY) 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'); +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 +SELECT * FROM PW17; + +-- Test for non-parititioned table. +DROP TABLE PW17_2; +CREATE TABLE PW17_2(USER STRING, COMPLEXDT ARRAY) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe1'; +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 >) 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'); +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 +SELECT * FROM PW17; + +DROP TABLE PW17_4; +CREATE TABLE PW17_4(USER STRING, COMPLEXDT ARRAY >) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe3'; +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/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q new file mode 100644 index 0000000000000..5501a653a30a8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q @@ -0,0 +1,18 @@ + + +create table partition_test_partitioned(key string, value string) partitioned by (dt string); + +insert overwrite table partition_test_partitioned partition(dt=100) select * from src1; +alter table partition_test_partitioned set fileformat rcfile; +insert overwrite table partition_test_partitioned partition(dt=101) select * from src1; +alter table partition_test_partitioned set fileformat Sequencefile; +insert overwrite table partition_test_partitioned partition(dt=102) select * from src1; + +set hive.fetch.task.conversion=minimal; +explain select *, BLOCK__OFFSET__INSIDE__FILE from partition_test_partitioned where dt >=100 and dt <= 102; +select * from partition_test_partitioned where dt >=100 and dt <= 102; + +set hive.fetch.task.conversion=more; +explain select *, BLOCK__OFFSET__INSIDE__FILE from partition_test_partitioned where dt >=100 and dt <= 102; +select * from partition_test_partitioned where dt >=100 and dt <= 102; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q new file mode 100644 index 0000000000000..b85fec538fdd9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q @@ -0,0 +1,18 @@ + + +create table partition_test_partitioned(key string, value string) partitioned by (dt string); + +alter table partition_test_partitioned set fileformat rcfile; +insert overwrite table partition_test_partitioned partition(dt=101) select * from src1; +show table extended like partition_test_partitioned partition(dt=101); + +alter table partition_test_partitioned set fileformat Sequencefile; +insert overwrite table partition_test_partitioned partition(dt=102) select * from src1; +show table extended like partition_test_partitioned partition(dt=102); +select key from partition_test_partitioned where dt=102; + +insert overwrite table partition_test_partitioned partition(dt=101) select * from src1; +show table extended like partition_test_partitioned partition(dt=101); +select key from partition_test_partitioned where dt=101; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q new file mode 100644 index 0000000000000..33decec6eda6f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q @@ -0,0 +1,8 @@ +create table partition_test_partitioned(key string, value string) partitioned by (dt string); +alter table partition_test_partitioned set fileformat sequencefile; +insert overwrite table partition_test_partitioned partition(dt='1') select * from src1; +alter table partition_test_partitioned partition (dt='1') set fileformat sequencefile; + +alter table partition_test_partitioned add partition (dt='2'); +alter table partition_test_partitioned drop partition (dt='2'); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q new file mode 100644 index 0000000000000..fc3bb8458b3e6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q @@ -0,0 +1,14 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +create table partition_test_partitioned(key string, value string) partitioned by (dt string); + +alter table partition_test_partitioned set fileformat rcfile; +insert overwrite table partition_test_partitioned partition(dt=101) select * from src1; +alter table partition_test_partitioned set fileformat Sequencefile; +insert overwrite table partition_test_partitioned partition(dt=102) select * from src1; + +select dt, count(1) from partition_test_partitioned where dt is not null group by dt; + +insert overwrite table partition_test_partitioned partition(dt=103) select * from src1; + +select dt, count(1) from partition_test_partitioned where dt is not null group by dt; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q new file mode 100644 index 0000000000000..dac5ef213e7c4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q @@ -0,0 +1,19 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +create table partition_test_partitioned(key string, value string) partitioned by (dt string); + +alter table partition_test_partitioned set fileformat rcfile; +insert overwrite table partition_test_partitioned partition(dt=101) select * from src1; +alter table partition_test_partitioned set fileformat Sequencefile; + +insert overwrite table partition_test_partitioned partition(dt=102) select * from src1; + +select count(1) from +(select key, value from partition_test_partitioned where dt=101 and key < 100 + union all +select key, value from partition_test_partitioned where dt=101 and key < 20)s; + +select count(1) from +(select key, value from partition_test_partitioned where dt=101 and key < 100 + union all +select key, value from partition_test_partitioned where dt=102 and key < 20)s; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q new file mode 100644 index 0000000000000..c4a19ada5ff2f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q @@ -0,0 +1,12 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +create table partition_test_partitioned(key string, value string) partitioned by (dt string); + +alter table partition_test_partitioned set fileformat rcfile; +insert overwrite table partition_test_partitioned partition(dt=101) select * from src1; + +select count(1) from partition_test_partitioned a join partition_test_partitioned b on a.key = b.key +where a.dt = '101' and b.dt = '101'; + +select count(1) from partition_test_partitioned a join partition_test_partitioned b on a.key = b.key +where a.dt = '101' and b.dt = '101' and a.key < 100; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q new file mode 100644 index 0000000000000..46ea10f4b7a2f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q @@ -0,0 +1,13 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +-- This tests that a query can span multiple partitions which can not only have different file formats, but +-- also different serdes +create table partition_test_partitioned(key string, value string) partitioned by (dt string) stored as rcfile; +insert overwrite table partition_test_partitioned partition(dt='1') select * from src; +alter table partition_test_partitioned set fileformat sequencefile; +insert overwrite table partition_test_partitioned partition(dt='2') select * from src; +alter table partition_test_partitioned set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; +insert overwrite table partition_test_partitioned partition(dt='3') select * from src; + +select * from partition_test_partitioned where dt is not null order by key, value, dt limit 20; +select key+key as key, value, dt from partition_test_partitioned where dt is not null order by key, value, dt limit 20; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q new file mode 100644 index 0000000000000..5205585853140 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q @@ -0,0 +1,12 @@ +set hive.input.format = org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +-- This tests that a query can span multiple partitions which can not only have different file formats, but +-- also different serdes +create table partition_test_partitioned(key string, value string) partitioned by (dt string) stored as rcfile; +insert overwrite table partition_test_partitioned partition(dt='1') select * from src; +alter table partition_test_partitioned set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; +insert overwrite table partition_test_partitioned partition(dt='2') select * from src; + +select * from partition_test_partitioned where dt is not null order by key, value, dt limit 20; +select key+key as key, value, dt from partition_test_partitioned where dt is not null order by key, value, dt limit 20; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partitions_json.q b/src/test/hive/ql/src/test/queries/clientpositive/partitions_json.q new file mode 100644 index 0000000000000..825e3f600464f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/partitions_json.q @@ -0,0 +1,21 @@ +set hive.ddl.output.format=json; + +CREATE TABLE add_part_test (key STRING, value STRING) PARTITIONED BY (ds STRING); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD PARTITION (ds='2010-01-01'); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01'); +SHOW PARTITIONS add_part_test; + +ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-02'); +SHOW PARTITIONS add_part_test; + +SHOW TABLE EXTENDED LIKE add_part_test PARTITION (ds='2010-01-02'); + +ALTER TABLE add_part_test DROP PARTITION (ds='2010-01-02'); + +DROP TABLE add_part_test; + +set hive.ddl.output.format=text; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/pcr.q b/src/test/hive/ql/src/test/queries/clientpositive/pcr.q new file mode 100644 index 0000000000000..09a39ae4e4476 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/pcr.q @@ -0,0 +1,141 @@ +drop table pcr_t1; +drop table pcr_t2; +drop table pcr_t3; + +create table pcr_t1 (key int, value string) partitioned by (ds string); + +insert overwrite table pcr_t1 partition (ds='2000-04-08') select * from src where key < 20 order by key; +insert overwrite table pcr_t1 partition (ds='2000-04-09') select * from src where key < 20 order by key; +insert overwrite table pcr_t1 partition (ds='2000-04-10') select * from src where key < 20 order by key; + +explain extended select key, value, ds from pcr_t1 where ds<='2000-04-09' and key<5 order by key, ds; +select key, value, ds from pcr_t1 where ds<='2000-04-09' and key<5 order by key, ds; + +explain extended select key, value from pcr_t1 where ds<='2000-04-09' or key<5 order by key; +select key, value from pcr_t1 where ds<='2000-04-09' or key<5 order by key; + +explain extended select key, value, ds from pcr_t1 where ds<='2000-04-09' and key<5 and value != 'val_2' order by key, ds; +select key, value, ds from pcr_t1 where ds<='2000-04-09' and key<5 and value != 'val_2' order by key, ds; + + +explain extended +select key, value, ds from pcr_t1 +where (ds < '2000-04-09' and key < 5) or (ds > '2000-04-09' and value == 'val_5') order by key, ds; + +select key, value, ds from pcr_t1 +where (ds < '2000-04-09' and key < 5) or (ds > '2000-04-09' and value == 'val_5') order by key, ds; + + +explain extended +select key, value, ds from pcr_t1 +where (ds < '2000-04-10' and key < 5) or (ds > '2000-04-08' and value == 'val_5') order by key, ds; + +select key, value, ds from pcr_t1 +where (ds < '2000-04-10' and key < 5) or (ds > '2000-04-08' and value == 'val_5') order by key, ds; + + +explain extended +select key, value, ds from pcr_t1 +where (ds < '2000-04-10' or key < 5) and (ds > '2000-04-08' or value == 'val_5') order by key, ds; + +select key, value, ds from pcr_t1 +where (ds < '2000-04-10' or key < 5) and (ds > '2000-04-08' or value == 'val_5') order by key, ds; + + +explain extended select key, value from pcr_t1 where (ds='2000-04-08' or ds='2000-04-09') and key=14 order by key, value; +select key, value from pcr_t1 where (ds='2000-04-08' or ds='2000-04-09') and key=14 order by key, value; + +explain extended select key, value from pcr_t1 where ds='2000-04-08' or ds='2000-04-09' order by key, value; +select key, value from pcr_t1 where ds='2000-04-08' or ds='2000-04-09' order by key, value; + +explain extended select key, value from pcr_t1 where ds>='2000-04-08' or ds<'2000-04-10' order by key, value; +select key, value from pcr_t1 where ds>='2000-04-08' or ds<'2000-04-10' order by key, value; + +explain extended select key, value, ds from pcr_t1 where (ds='2000-04-08' and key=1) or (ds='2000-04-09' and key=2) order by key, value, ds; +select key, value, ds from pcr_t1 where (ds='2000-04-08' and key=1) or (ds='2000-04-09' and key=2) order by key, value, ds; + +explain extended select * from pcr_t1 t1 join pcr_t1 t2 on t1.key=t2.key and t1.ds='2000-04-08' and t2.ds='2000-04-08' order by t1.key; +select * from pcr_t1 t1 join pcr_t1 t2 on t1.key=t2.key and t1.ds='2000-04-08' and t2.ds='2000-04-08' order by t1.key; + +explain extended select * from pcr_t1 t1 join pcr_t1 t2 on t1.key=t2.key and t1.ds='2000-04-08' and t2.ds='2000-04-09' order by t1.key; +select * from pcr_t1 t1 join pcr_t1 t2 on t1.key=t2.key and t1.ds='2000-04-08' and t2.ds='2000-04-09' order by t1.key; + +insert overwrite table pcr_t1 partition (ds='2000-04-11') select * from src where key < 20 order by key; + +explain extended select key, value, ds from pcr_t1 where (ds>'2000-04-08' and ds<'2000-04-11') or (ds>='2000-04-08' and ds<='2000-04-11' and key=2) order by key, value, ds; +select key, value, ds from pcr_t1 where (ds>'2000-04-08' and ds<'2000-04-11') or (ds>='2000-04-08' and ds<='2000-04-11' and key=2) order by key, value, ds; + +explain extended select key, value, ds from pcr_t1 where (ds>'2000-04-08' and ds<'2000-04-11') or (ds<='2000-04-09' and key=2) order by key, value, ds; +select key, value, ds from pcr_t1 where (ds>'2000-04-08' and ds<'2000-04-11') or (ds<='2000-04-09' and key=2) order by key, value, ds; + +create table pcr_t2 (key int, value string); +create table pcr_t3 (key int, value string); + +explain extended +from pcr_t1 +insert overwrite table pcr_t2 select key, value where ds='2000-04-08' +insert overwrite table pcr_t3 select key, value where ds='2000-04-08'; + +from pcr_t1 +insert overwrite table pcr_t2 select key, value where ds='2000-04-08' +insert overwrite table pcr_t3 select key, value where ds='2000-04-08'; + +explain extended +from pcr_t1 +insert overwrite table pcr_t2 select key, value where ds='2000-04-08' and key=2 +insert overwrite table pcr_t3 select key, value where ds='2000-04-08' and key=3; + +from pcr_t1 +insert overwrite table pcr_t2 select key, value where ds='2000-04-08' and key=2 +insert overwrite table pcr_t3 select key, value where ds='2000-04-08' and key=3; + + +explain extended select key, value from srcpart where ds='2008-04-08' and hr=11 order by key limit 10; +select key, value from srcpart where ds='2008-04-04' and hr=11 order by key limit 10; + +explain extended select key, value, ds, hr from srcpart where ds='2008-04-08' and (hr='11' or hr='12') and key=11 order by key, ds, hr; +select key, value, ds, hr from srcpart where ds='2008-04-08' and (hr='11' or hr='12') and key=11 order by key, ds, hr; + +explain extended select key, value, ds, hr from srcpart where hr='11' and key=11 order by key, ds, hr; +select key, value, ds, hr from srcpart where hr='11' and key=11 order by key, ds, hr; + +drop table pcr_t1; +drop table pcr_t2; +drop table pcr_t3; + + +-- Test cases when a non-boolean ds expression has same and different values for all possible ds values: +drop table pcr_foo; +create table pcr_foo (key int, value string) partitioned by (ds int); +insert overwrite table pcr_foo partition (ds=3) select * from src where key < 10 order by key; +insert overwrite table pcr_foo partition (ds=5) select * from src where key < 10 order by key; +insert overwrite table pcr_foo partition (ds=7) select * from src where key < 10 order by key; + +-- the condition is 'true' for all the 3 partitions (ds=3,5,7): +select key, value, ds from pcr_foo where (ds % 2 == 1); + +-- the condition is 'true' for partitions (ds=3,5) but 'false' of partition ds=7: +select key, value, ds from pcr_foo where (ds / 3 < 2); + +drop table pcr_foo; + + + +-- Cover org.apache.hadoop.hive.ql.optimizer.pcr.PcrExprProcFactory.FieldExprProcessor. +-- Create a table with a struct data: +create table ab(strct struct) +row format delimited + fields terminated by '\t' + collection items terminated by '\001'; +load data local inpath '../data/files/kv1.txt' +overwrite into table ab; + +-- Create partitioned table with struct data: +drop table foo_field; +create table foo_field (s struct) partitioned by (ds int); +insert overwrite table foo_field partition (ds=5) select strct from ab where strct.a < 10 limit 2; +insert overwrite table foo_field partition (ds=7) select strct from ab where strct.a > 190 limit 2; +select s,ds from foo_field where ((ds + s.a) > 0) order by ds,s; + +drop table foo_field; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/plan_json.q b/src/test/hive/ql/src/test/queries/clientpositive/plan_json.q new file mode 100644 index 0000000000000..0243c995412b0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/plan_json.q @@ -0,0 +1,3 @@ +-- explain plan json: the query gets the formatted json output of the query plan of the hive query + +EXPLAIN FORMATTED SELECT count(1) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd1.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd1.q new file mode 100644 index 0000000000000..a5a4db0544ed3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd1.q @@ -0,0 +1,14 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT src.key as c3 from src where src.key > '2'; + +SELECT src.key as c3 from src where src.key > '2'; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT src.key as c3 from src where src.key > '2'; + +SELECT src.key as c3 from src where src.key > '2'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd2.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd2.q new file mode 100644 index 0000000000000..536eb989dff95 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd2.q @@ -0,0 +1,74 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=true; + +explain +select b.key,b.cc +from ( + select a.* + from ( + select key, count(value) as cc + from srcpart a + where a.ds = '2008-04-08' and a.hr = '11' + group by key + )a + distribute by a.key + sort by a.key,a.cc desc) b +where b.cc>1; + +select b.key,b.cc +from ( + select a.* + from ( + select key, count(value) as cc + from srcpart a + where a.ds = '2008-04-08' and a.hr = '11' + group by key + )a + distribute by a.key + sort by a.key,a.cc desc) b +where b.cc>1; + +EXPLAIN +SELECT user_id +FROM ( + SELECT + CAST(key AS INT) AS user_id + ,CASE WHEN (value LIKE 'aaa%' OR value LIKE 'vvv%') + THEN 1 + ELSE 0 END AS tag_student + FROM srcpart +) sub +WHERE sub.tag_student > 0; + +EXPLAIN +SELECT x.key, x.value as v1, y.key FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1; + +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +explain +select b.key,b.cc +from ( + select a.* + from ( + select key, count(value) as cc + from srcpart a + where a.ds = '2008-04-08' and a.hr = '11' + group by key + )a + distribute by a.key + sort by a.key,a.cc desc) b +where b.cc>1; + +select b.key,b.cc +from ( + select a.* + from ( + select key, count(value) as cc + from srcpart a + where a.ds = '2008-04-08' and a.hr = '11' + group by key + )a + distribute by a.key + sort by a.key,a.cc desc) b +where b.cc>1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_clusterby.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_clusterby.q new file mode 100644 index 0000000000000..0c8b245d81320 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_clusterby.q @@ -0,0 +1,20 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT * FROM SRC x where x.key = 10 CLUSTER BY x.key; +SELECT * FROM SRC x where x.key = 10 CLUSTER BY x.key; + +EXPLAIN +SELECT x.key, x.value as v1, y.key FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1;; +SELECT x.key, x.value as v1, y.key FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT * FROM SRC x where x.key = 10 CLUSTER BY x.key; +SELECT * FROM SRC x where x.key = 10 CLUSTER BY x.key; + +EXPLAIN +SELECT x.key, x.value as v1, y.key FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1;; +SELECT x.key, x.value as v1, y.key FROM SRC x JOIN SRC y ON (x.key = y.key) where x.key = 20 CLUSTER BY v1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_expr.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_expr.q new file mode 100644 index 0000000000000..969c7e4f0cf28 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_expr.q @@ -0,0 +1,24 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +CREATE TABLE ppd_constant_expr(c1 STRING, c2 INT, c3 DOUBLE) STORED AS TEXTFILE; + +EXPLAIN +FROM src1 +INSERT OVERWRITE TABLE ppd_constant_expr SELECT 4 + NULL, src1.key - NULL, NULL + NULL; + +FROM src1 +INSERT OVERWRITE TABLE ppd_constant_expr SELECT 4 + NULL, src1.key - NULL, NULL + NULL; + +SELECT ppd_constant_expr.* FROM ppd_constant_expr; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +FROM src1 +INSERT OVERWRITE TABLE ppd_constant_expr SELECT 4 + NULL, src1.key - NULL, NULL + NULL; + +FROM src1 +INSERT OVERWRITE TABLE ppd_constant_expr SELECT 4 + NULL, src1.key - NULL, NULL + NULL; + +SELECT ppd_constant_expr.* FROM ppd_constant_expr; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_where.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_where.q new file mode 100644 index 0000000000000..0c102354fed7a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_where.q @@ -0,0 +1,5 @@ +-- Test that the partition pruner does not fail when there is a constant expression in the filter + +EXPLAIN SELECT COUNT(*) FROM srcpart WHERE ds = '2008-04-08' and 'a' = 'a'; + +SELECT COUNT(*) FROM srcpart WHERE ds = '2008-04-08' and 'a' = 'a'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby.q new file mode 100644 index 0000000000000..65c0b01198491 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby.q @@ -0,0 +1,26 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT src1.c1 +FROM +(SELECT src.value as c1, count(src.key) as c2 from src where src.value > 'val_10' group by src.value) src1 +WHERE src1.c1 > 'val_200' and (src1.c2 > 30 or src1.c1 < 'val_400'); + +SELECT src1.c1 +FROM +(SELECT src.value as c1, count(src.key) as c2 from src where src.value > 'val_10' group by src.value) src1 +WHERE src1.c1 > 'val_200' and (src1.c2 > 30 or src1.c1 < 'val_400'); + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT src1.c1 +FROM +(SELECT src.value as c1, count(src.key) as c2 from src where src.value > 'val_10' group by src.value) src1 +WHERE src1.c1 > 'val_200' and (src1.c2 > 30 or src1.c1 < 'val_400'); + +SELECT src1.c1 +FROM +(SELECT src.value as c1, count(src.key) as c2 from src where src.value > 'val_10' group by src.value) src1 +WHERE src1.c1 > 'val_200' and (src1.c2 > 30 or src1.c1 < 'val_400'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby2.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby2.q new file mode 100644 index 0000000000000..ae288eada2610 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby2.q @@ -0,0 +1,30 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT max(src1.c1), src1.c2 +FROM +(SELECT src.value AS c1, count(src.key) AS c2 FROM src WHERE src.value > 'val_10' GROUP BY src.value) src1 +WHERE src1.c1 > 'val_200' AND (src1.c2 > 30 OR src1.c1 < 'val_400') +GROUP BY src1.c2; + +SELECT max(src1.c1), src1.c2 +FROM +(SELECT src.value AS c1, count(src.key) AS c2 FROM src WHERE src.value > 'val_10' GROUP BY src.value) src1 +WHERE src1.c1 > 'val_200' AND (src1.c2 > 30 OR src1.c1 < 'val_400') +GROUP BY src1.c2; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT max(src1.c1), src1.c2 +FROM +(SELECT src.value AS c1, count(src.key) AS c2 FROM src WHERE src.value > 'val_10' GROUP BY src.value) src1 +WHERE src1.c1 > 'val_200' AND (src1.c2 > 30 OR src1.c1 < 'val_400') +GROUP BY src1.c2; + +SELECT max(src1.c1), src1.c2 +FROM +(SELECT src.value AS c1, count(src.key) AS c2 FROM src WHERE src.value > 'val_10' GROUP BY src.value) src1 +WHERE src1.c1 > 'val_200' AND (src1.c2 > 30 OR src1.c1 < 'val_400') +GROUP BY src1.c2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby_join.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby_join.q new file mode 100644 index 0000000000000..82f358b0d6ca1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby_join.q @@ -0,0 +1,24 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT src1.c1, count(1) +FROM +(SELECT src.key AS c1, src.value AS c2 from src where src.key > '1' ) src1 +JOIN +(SELECT src.key AS c3, src.value AS c4 from src where src.key > '2' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +WHERE src1.c1 > '20' AND (src1.c2 < 'val_50' OR src1.c1 > '2') AND (src2.c3 > '50' OR src1.c1 < '50') AND (src2.c3 <> '4') +GROUP BY src1.c1; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT src1.c1, count(1) +FROM +(SELECT src.key AS c1, src.value AS c2 from src where src.key > '1' ) src1 +JOIN +(SELECT src.key AS c3, src.value AS c4 from src where src.key > '2' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +WHERE src1.c1 > '20' AND (src1.c2 < 'val_50' OR src1.c1 > '2') AND (src2.c3 > '50' OR src1.c1 < '50') AND (src2.c3 <> '4') +GROUP BY src1.c1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_join.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_join.q new file mode 100644 index 0000000000000..09323e965b846 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_join.q @@ -0,0 +1,38 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key > '1' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key > '2' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +WHERE src1.c1 > '20' and (src1.c2 < 'val_50' or src1.c1 > '2') and (src2.c3 > '50' or src1.c1 < '50') and (src2.c3 <> '4'); + +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key > '1' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key > '2' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +WHERE src1.c1 > '20' and (src1.c2 < 'val_50' or src1.c1 > '2') and (src2.c3 > '50' or src1.c1 < '50') and (src2.c3 <> '4'); + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key > '1' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key > '2' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +WHERE src1.c1 > '20' and (src1.c2 < 'val_50' or src1.c1 > '2') and (src2.c3 > '50' or src1.c1 < '50') and (src2.c3 <> '4'); + +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key > '1' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key > '2' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +WHERE src1.c1 > '20' and (src1.c2 < 'val_50' or src1.c1 > '2') and (src2.c3 > '50' or src1.c1 < '50') and (src2.c3 <> '4'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_join2.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_join2.q new file mode 100644 index 0000000000000..8bd4cae3f96c2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_join2.q @@ -0,0 +1,50 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key <> '302' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key <> '305' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +JOIN +(SELECT src.key as c5, src.value as c6 from src where src.key <> '306' ) src3 +ON src1.c2 = src3.c6 +WHERE src1.c1 <> '311' and (src1.c2 <> 'val_50' or src1.c1 > '1') and (src2.c3 <> '10' or src1.c1 <> '10') and (src2.c3 <> '14') and (sqrt(src3.c5) <> 13); + +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key <> '302' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key <> '305' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +JOIN +(SELECT src.key as c5, src.value as c6 from src where src.key <> '306' ) src3 +ON src1.c2 = src3.c6 +WHERE src1.c1 <> '311' and (src1.c2 <> 'val_50' or src1.c1 > '1') and (src2.c3 <> '10' or src1.c1 <> '10') and (src2.c3 <> '14') and (sqrt(src3.c5) <> 13); + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key <> '302' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key <> '305' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +JOIN +(SELECT src.key as c5, src.value as c6 from src where src.key <> '306' ) src3 +ON src1.c2 = src3.c6 +WHERE src1.c1 <> '311' and (src1.c2 <> 'val_50' or src1.c1 > '1') and (src2.c3 <> '10' or src1.c1 <> '10') and (src2.c3 <> '14') and (sqrt(src3.c5) <> 13); + +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key <> '302' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key <> '305' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +JOIN +(SELECT src.key as c5, src.value as c6 from src where src.key <> '306' ) src3 +ON src1.c2 = src3.c6 +WHERE src1.c1 <> '311' and (src1.c2 <> 'val_50' or src1.c1 > '1') and (src2.c3 <> '10' or src1.c1 <> '10') and (src2.c3 <> '14') and (sqrt(src3.c5) <> 13); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_join3.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_join3.q new file mode 100644 index 0000000000000..7abf41098618e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_join3.q @@ -0,0 +1,50 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key <> '11' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key <> '12' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +JOIN +(SELECT src.key as c5, src.value as c6 from src where src.key <> '13' ) src3 +ON src1.c1 = src3.c5 +WHERE src1.c1 > '0' and (src1.c2 <> 'val_500' or src1.c1 > '1') and (src2.c3 > '10' or src1.c1 <> '10') and (src2.c3 <> '4') and (src3.c5 <> '1'); + +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key <> '11' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key <> '12' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +JOIN +(SELECT src.key as c5, src.value as c6 from src where src.key <> '13' ) src3 +ON src1.c1 = src3.c5 +WHERE src1.c1 > '0' and (src1.c2 <> 'val_500' or src1.c1 > '1') and (src2.c3 > '10' or src1.c1 <> '10') and (src2.c3 <> '4') and (src3.c5 <> '1'); + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key <> '11' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key <> '12' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +JOIN +(SELECT src.key as c5, src.value as c6 from src where src.key <> '13' ) src3 +ON src1.c1 = src3.c5 +WHERE src1.c1 > '0' and (src1.c2 <> 'val_500' or src1.c1 > '1') and (src2.c3 > '10' or src1.c1 <> '10') and (src2.c3 <> '4') and (src3.c5 <> '1'); + +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src where src.key <> '11' ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key <> '12' ) src2 +ON src1.c1 = src2.c3 AND src1.c1 < '400' +JOIN +(SELECT src.key as c5, src.value as c6 from src where src.key <> '13' ) src3 +ON src1.c1 = src3.c5 +WHERE src1.c1 > '0' and (src1.c2 <> 'val_500' or src1.c1 > '1') and (src2.c3 > '10' or src1.c1 <> '10') and (src2.c3 <> '4') and (src3.c5 <> '1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_join_filter.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_join_filter.q new file mode 100644 index 0000000000000..9d42ef20860d1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_join_filter.q @@ -0,0 +1,116 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +explain extended select a.key, b.k2, b.k3 +from src a +join ( +select key, +min(key) as k, +min(key)+1 as k1, +min(key)+2 as k2, +min(key)+3 as k3 +from src +group by key +) b +on a.key=b.key and b.k1 < 5; + +select a.key, b.k2, b.k3 +from src a +join ( +select key, +min(key) as k, +min(key)+1 as k1, +min(key)+2 as k2, +min(key)+3 as k3 +from src +group by key +) b +on a.key=b.key and b.k1 < 5; + +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=true; + +explain extended select a.key, b.k2, b.k3 +from src a +join ( +select key, +min(key) as k, +min(key)+1 as k1, +min(key)+2 as k2, +min(key)+3 as k3 +from src +group by key +) b +on a.key=b.key and b.k1 < 5; + +select a.key, b.k2, b.k3 +from src a +join ( +select key, +min(key) as k, +min(key)+1 as k1, +min(key)+2 as k2, +min(key)+3 as k3 +from src +group by key +) b +on a.key=b.key and b.k1 < 5; + +set hive.optimize.ppd=false; +set hive.ppd.remove.duplicatefilters=false; + +explain extended select a.key, b.k2, b.k3 +from src a +join ( +select key, +min(key) as k, +min(key)+1 as k1, +min(key)+2 as k2, +min(key)+3 as k3 +from src +group by key +) b +on a.key=b.key and b.k1 < 5; + +select a.key, b.k2, b.k3 +from src a +join ( +select key, +min(key) as k, +min(key)+1 as k1, +min(key)+2 as k2, +min(key)+3 as k3 +from src +group by key +) b +on a.key=b.key and b.k1 < 5; + +set hive.optimize.ppd=faluse; +set hive.ppd.remove.duplicatefilters=true; + +explain extended select a.key, b.k2, b.k3 +from src a +join ( +select key, +min(key) as k, +min(key)+1 as k1, +min(key)+2 as k2, +min(key)+3 as k3 +from src +group by key +) b +on a.key=b.key and b.k1 < 5; + +select a.key, b.k2, b.k3 +from src a +join ( +select key, +min(key) as k, +min(key)+1 as k1, +min(key)+2 as k2, +min(key)+3 as k3 +from src +group by key +) b +on a.key=b.key and b.k1 < 5; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_multi_insert.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_multi_insert.q new file mode 100644 index 0000000000000..a802df1b98819 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_multi_insert.q @@ -0,0 +1,45 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +CREATE TABLE mi1(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE mi2(key INT, value STRING) STORED AS TEXTFILE; +CREATE TABLE mi3(key INT) PARTITIONED BY(ds STRING, hr STRING) STORED AS TEXTFILE; + +EXPLAIN +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; + +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; + +SELECT mi1.* FROM mi1; +SELECT mi2.* FROM mi2; +SELECT mi3.* FROM mi3; +dfs -cat ../build/ql/test/data/warehouse/mi4.out/*; + + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +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; + +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; + +SELECT mi1.* FROM mi1; +SELECT mi2.* FROM mi2; +SELECT mi3.* FROM mi3; +dfs -cat ../build/ql/test/data/warehouse/mi4.out/*; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join1.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join1.q new file mode 100644 index 0000000000000..752176c4b7e4b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join1.q @@ -0,0 +1,38 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN + FROM + src a + LEFT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + + FROM + src a + LEFT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN + FROM + src a + LEFT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + + FROM + src a + LEFT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join2.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join2.q new file mode 100644 index 0000000000000..e22892b45f2d0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join2.q @@ -0,0 +1,38 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN + FROM + src a + RIGHT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25'; + + FROM + src a + RIGHT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25'; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN + FROM + src a + RIGHT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25'; + + FROM + src a + RIGHT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join3.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join3.q new file mode 100644 index 0000000000000..d756b90e84316 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join3.q @@ -0,0 +1,38 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN + FROM + src a + FULL OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25'; + + FROM + src a + FULL OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25'; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN + FROM + src a + FULL OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25'; + + FROM + src a + FULL OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join4.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join4.q new file mode 100644 index 0000000000000..e465eda68e258 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join4.q @@ -0,0 +1,50 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN + FROM + src a + LEFT OUTER JOIN + src b + ON (a.key = b.key) + RIGHT OUTER JOIN + src c + ON (a.key = c.key) + SELECT a.key, a.value, b.key, b.value, c.key + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25' AND sqrt(c.key) <> 13 ; + + FROM + src a + LEFT OUTER JOIN + src b + ON (a.key = b.key) + RIGHT OUTER JOIN + src c + ON (a.key = c.key) + SELECT a.key, a.value, b.key, b.value, c.key + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25' AND sqrt(c.key) <> 13 ; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN + FROM + src a + LEFT OUTER JOIN + src b + ON (a.key = b.key) + RIGHT OUTER JOIN + src c + ON (a.key = c.key) + SELECT a.key, a.value, b.key, b.value, c.key + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25' AND sqrt(c.key) <> 13 ; + + FROM + src a + LEFT OUTER JOIN + src b + ON (a.key = b.key) + RIGHT OUTER JOIN + src c + ON (a.key = c.key) + SELECT a.key, a.value, b.key, b.value, c.key + WHERE a.key > '10' AND a.key < '20' AND b.key > '15' AND b.key < '25' AND sqrt(c.key) <> 13 ; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join5.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join5.q new file mode 100644 index 0000000000000..382b9494cf7b9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join5.q @@ -0,0 +1,16 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=true; + +create table t1 (id int, key string, value string); +create table t2 (id int, key string, value string); +create table t3 (id int, key string, value string); +create table t4 (id int, key string, value string); + +explain select * from t1 full outer join t2 on t1.id=t2.id join t3 on t2.id=t3.id where t3.id=20; +explain select * from t1 join t2 on (t1.id=t2.id) left outer join t3 on (t2.id=t3.id) where t2.id=20; +explain select * from t1 join t2 on (t1.id=t2.id) left outer join t3 on (t1.id=t3.id) where t2.id=20; + +drop table t1; +drop table t2; +drop table t3; +drop table t4; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_random.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_random.q new file mode 100644 index 0000000000000..de0c067f353b9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_random.q @@ -0,0 +1,22 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key > '2' ) src2 +ON src1.c1 = src2.c3 +WHERE rand() > 0.5; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT src1.c1, src2.c4 +FROM +(SELECT src.key as c1, src.value as c2 from src ) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src where src.key > '2' ) src2 +ON src1.c1 = src2.c3 +WHERE rand() > 0.5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_repeated_alias.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_repeated_alias.q new file mode 100644 index 0000000000000..732671afbe3da --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_repeated_alias.q @@ -0,0 +1,33 @@ +drop table pokes; +drop table pokes2; +create table pokes (foo int, bar int, blah int); +create table pokes2 (foo int, bar int, blah int); + +-- Q1: predicate should not be pushed on the right side of a left outer join +explain +SELECT a.foo as foo1, b.foo as foo2, b.bar +FROM pokes a LEFT OUTER JOIN pokes2 b +ON a.foo=b.foo +WHERE b.bar=3; + +-- Q2: predicate should not be pushed on the right side of a left outer join +explain +SELECT * FROM + (SELECT a.foo as foo1, b.foo as foo2, b.bar + FROM pokes a LEFT OUTER JOIN pokes2 b + ON a.foo=b.foo) a +WHERE a.bar=3; + +-- Q3: predicate should be pushed +explain +SELECT * FROM + (SELECT a.foo as foo1, b.foo as foo2, a.bar + FROM pokes a JOIN pokes2 b + ON a.foo=b.foo) a +WHERE a.bar=3; + +-- Q4: here, the filter c.bar should be created under the first join but above the second +explain select c.foo, d.bar from (select c.foo, b.bar, c.blah from pokes c left outer join pokes b on c.foo=b.foo) c left outer join pokes d where d.foo=1 and c.bar=2; + +drop table pokes; +drop table pokes2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_transform.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_transform.q new file mode 100644 index 0000000000000..65a498d021f77 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_transform.q @@ -0,0 +1,38 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_case.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_case.q new file mode 100644 index 0000000000000..b7a6b15bfd1c2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_case.q @@ -0,0 +1,55 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT * +FROM srcpart a JOIN srcpart b +ON a.key = b.key +WHERE a.ds = '2008-04-08' AND + b.ds = '2008-04-08' AND + CASE a.key + WHEN '27' THEN TRUE + WHEN '38' THEN FALSE + ELSE NULL + END +ORDER BY a.key, a.value, a.ds, a.hr, b.key, b.value, b.ds, b.hr; + +SELECT * +FROM srcpart a JOIN srcpart b +ON a.key = b.key +WHERE a.ds = '2008-04-08' AND + b.ds = '2008-04-08' AND + CASE a.key + WHEN '27' THEN TRUE + WHEN '38' THEN FALSE + ELSE NULL + END +ORDER BY a.key, a.value, a.ds, a.hr, b.key, b.value, b.ds, b.hr; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT * +FROM srcpart a JOIN srcpart b +ON a.key = b.key +WHERE a.ds = '2008-04-08' AND + b.ds = '2008-04-08' AND + CASE a.key + WHEN '27' THEN TRUE + WHEN '38' THEN FALSE + ELSE NULL + END +ORDER BY a.key, a.value, a.ds, a.hr, b.key, b.value, b.ds, b.hr; + +SELECT * +FROM srcpart a JOIN srcpart b +ON a.key = b.key +WHERE a.ds = '2008-04-08' AND + b.ds = '2008-04-08' AND + CASE a.key + WHEN '27' THEN TRUE + WHEN '38' THEN FALSE + ELSE NULL + END +ORDER BY a.key, a.value, a.ds, a.hr, b.key, b.value, b.ds, b.hr; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_col.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_col.q new file mode 100644 index 0000000000000..993da9752cb7f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_col.q @@ -0,0 +1,48 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +SELECT key, randum123 +FROM (SELECT *, cast(rand() as double) AS randum123 FROM src WHERE key = 100) a +WHERE randum123 <=0.1; + +EXPLAIN +SELECT * FROM +( +SELECT key, randum123 +FROM (SELECT *, cast(rand() as double) AS randum123 FROM src WHERE key = 100) a +WHERE randum123 <=0.1)s WHERE s.randum123>0.1 LIMIT 20; + +EXPLAIN +SELECT key,randum123, h4 +FROM (SELECT *, cast(rand() as double) AS randum123, hex(4) AS h4 FROM src WHERE key = 100) a +WHERE a.h4 <= 3; + +EXPLAIN +SELECT key,randum123, v10 +FROM (SELECT *, cast(rand() as double) AS randum123, value*10 AS v10 FROM src WHERE key = 100) a +WHERE a.v10 <= 200; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +SELECT key, randum123 +FROM (SELECT *, cast(rand() as double) AS randum123 FROM src WHERE key = 100) a +WHERE randum123 <=0.1; + +EXPLAIN +SELECT * FROM +( +SELECT key, randum123 +FROM (SELECT *, cast(rand() as double) AS randum123 FROM src WHERE key = 100) a +WHERE randum123 <=0.1)s WHERE s.randum123>0.1 LIMIT 20; + +EXPLAIN +SELECT key,randum123, h4 +FROM (SELECT *, cast(rand() as double) AS randum123, hex(4) AS h4 FROM src WHERE key = 100) a +WHERE a.h4 <= 3; + +EXPLAIN +SELECT key,randum123, v10 +FROM (SELECT *, cast(rand() as double) AS randum123, value*10 AS v10 FROM src WHERE key = 100) a +WHERE a.v10 <= 200; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_union.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_union.q new file mode 100644 index 0000000000000..2cd876e078e93 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_union.q @@ -0,0 +1,38 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=false; + +EXPLAIN +FROM ( + FROM src select src.key, src.value WHERE src.key < '100' + UNION ALL + FROM src SELECT src.* WHERE src.key > '150' +) unioned_query +SELECT unioned_query.* + WHERE key > '4' and value > 'val_4'; + +FROM ( + FROM src select src.key, src.value WHERE src.key < '100' + UNION ALL + FROM src SELECT src.* WHERE src.key > '150' +) unioned_query +SELECT unioned_query.* + WHERE key > '4' and value > 'val_4'; + +set hive.ppd.remove.duplicatefilters=true; + +EXPLAIN +FROM ( + FROM src select src.key, src.value WHERE src.key < '100' + UNION ALL + FROM src SELECT src.* WHERE src.key > '150' +) unioned_query +SELECT unioned_query.* + WHERE key > '4' and value > 'val_4'; + +FROM ( + FROM src select src.key, src.value WHERE src.key < '100' + UNION ALL + FROM src SELECT src.* WHERE src.key > '150' +) unioned_query +SELECT unioned_query.* + WHERE key > '4' and value > 'val_4'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_union_view.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_union_view.q new file mode 100644 index 0000000000000..d635e2d171469 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_union_view.q @@ -0,0 +1,51 @@ +-- test predicate pushdown on a view with a union + +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; + +insert overwrite table t1_new partition (ds = '2011-10-16') +select 'key2', 'value2' from src limit 1; + +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; + +insert overwrite table t1_old partition (ds = '2011-10-14') +select 'keymap4', 'value4' from src limit 1; + +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; + +insert overwrite table t1_mapping partition (ds = '2011-10-14') +select 'key4', 'keymap4' from src limit 1; + + +create view t1 partitioned on (ds) as +select * from +( +select key, value, ds from t1_new +union all +select key, value, t1_old.ds from t1_old join t1_mapping +on t1_old.keymap = t1_mapping.keymap and + t1_old.ds = t1_mapping.ds +) subq; + +explain extended +select * from t1 where ds = '2011-10-13'; + +select * from t1 where ds = '2011-10-13'; + +select * from t1 where ds = '2011-10-14'; + +explain extended +select * from t1 where ds = '2011-10-15'; + +select * from t1 where ds = '2011-10-15'; +select * from t1 where ds = '2011-10-16'; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_vc.q b/src/test/hive/ql/src/test/queries/clientpositive/ppd_vc.q new file mode 100644 index 0000000000000..b5efdb31bd78c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppd_vc.q @@ -0,0 +1,14 @@ +--HIVE-3926 PPD on virtual column of partitioned table is not working + +explain extended +select * from srcpart where BLOCK__OFFSET__INSIDE__FILE<100; +select * from srcpart where BLOCK__OFFSET__INSIDE__FILE<100; + +explain extended +select b.* from src a join + (select *,BLOCK__OFFSET__INSIDE__FILE from srcpart where BLOCK__OFFSET__INSIDE__FILE<100) b + on a.key=b.key AND b.BLOCK__OFFSET__INSIDE__FILE<50 order by ds,hr,BLOCK__OFFSET__INSIDE__FILE; + +select b.* from src a join + (select *,BLOCK__OFFSET__INSIDE__FILE from srcpart where BLOCK__OFFSET__INSIDE__FILE<100) b + on a.key=b.key AND b.BLOCK__OFFSET__INSIDE__FILE<50 order by ds,hr,BLOCK__OFFSET__INSIDE__FILE; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q b/src/test/hive/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q new file mode 100644 index 0000000000000..3cda01b8b343b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q @@ -0,0 +1,41 @@ +set hive.optimize.ppd=true; + +EXPLAIN EXTENDED +SELECT + CAST(key AS INT) AS user_id, value + FROM srcpart + WHERE ds='2008-04-08' and + ( CASE WHEN (value LIKE 'aaa%' OR value LIKE 'vvv%') + THEN 1 + ELSE 0 end ) > 0 +; + +SELECT + CAST(key AS INT) AS user_id, value + FROM srcpart + WHERE ds='2008-04-08' and + ( CASE WHEN (value LIKE 'aaa%' OR value LIKE 'vvv%') + THEN 1 + ELSE 0 end ) > 0 +; + +set hive.optimize.ppd=false; + +EXPLAIN EXTENDED +SELECT + CAST(key AS INT) AS user_id, value + FROM srcpart + WHERE ds='2008-04-08' and + ( CASE WHEN (value LIKE 'aaa%' OR value LIKE 'vvv%') + THEN 1 + ELSE 0 end ) > 0 +; + +SELECT + CAST(key AS INT) AS user_id, value + FROM srcpart + WHERE ds='2008-04-08' and + ( CASE WHEN (value LIKE 'aaa%' OR value LIKE 'vvv%') + THEN 1 + ELSE 0 end ) > 0 +; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown.q b/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown.q new file mode 100644 index 0000000000000..860dd631ce10a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown.q @@ -0,0 +1,41 @@ +create table ppr_test (key string) partitioned by (ds string); + +alter table ppr_test add partition (ds = '1234'); +alter table ppr_test add partition (ds = '1224'); +alter table ppr_test add partition (ds = '1214'); +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'); +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; + + +select * from ppr_test where ds = '1234' order by key; +select * from ppr_test where ds = '1224' order by key; +select * from ppr_test where ds = '1214' order by key; +select * from ppr_test where ds = '12.4' order by key; +select * from ppr_test where ds = '12+4' order by key; +select * from ppr_test where ds = '12:4' order by key; +select * from ppr_test where ds = '12%4' order by key; +select * from ppr_test where ds = '12*4' order by key; +select * from ppr_test where ds = '12.*4' order by key; + +select * from ppr_test where ds = '1234' and key = '1234'; +select * from ppr_test where ds = '1224' and key = '1224'; +select * from ppr_test where ds = '1214' and key = '1214'; +select * from ppr_test where ds = '12.4' and key = '12.4'; +select * from ppr_test where ds = '12+4' and key = '12+4'; +select * from ppr_test where ds = '12:4' and key = '12:4'; +select * from ppr_test where ds = '12%4' and key = '12%4'; +select * from ppr_test where ds = '12*4' and key = '12*4'; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown2.q b/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown2.q new file mode 100644 index 0000000000000..67c0da0dfc591 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown2.q @@ -0,0 +1,29 @@ +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; + +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; + +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; + +select * from ppr_test3 where l = '1'; +select * from ppr_test3 where l = '2'; +select * from ppr_test3 where ol = '1'; +select * from ppr_test3 where ol = '2'; +select * from ppr_test3 where col = '1'; +select * from ppr_test3 where ol = '2' and l = '1'; +select * from ppr_test3 where col='1' and ol = '2' and l = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown3.q b/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown3.q new file mode 100644 index 0000000000000..35c11be2d62e4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown3.q @@ -0,0 +1,10 @@ +set hive.mapred.mode=nonstrict; + +explain select * from srcpart where key < 10; +select * from srcpart where key < 10; + +explain select * from srcpart; +select * from srcpart; + +explain select key from srcpart; +select key from srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/print_header.q b/src/test/hive/ql/src/test/queries/clientpositive/print_header.q new file mode 100644 index 0000000000000..3ca0340e2d267 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/print_header.q @@ -0,0 +1,14 @@ + +set hive.cli.print.header=true; + +SELECT src1.key as k1, src1.value as v1, + src2.key as k2, src2.value as v2 FROM + (SELECT * FROM src WHERE src.key < 10) src1 + JOIN + (SELECT * FROM src WHERE src.key < 10) src2 + SORT BY k1, v1, k2, v2 + LIMIT 10; + +SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key LIMIT 10; + +use default; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/progress_1.q b/src/test/hive/ql/src/test/queries/clientpositive/progress_1.q new file mode 100644 index 0000000000000..ad908a02ad181 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/progress_1.q @@ -0,0 +1,9 @@ +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; + +select count(1) from PROGRESS_1 t1 join PROGRESS_1 t2 on t1.key=t2.key; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/protectmode.q b/src/test/hive/ql/src/test/queries/clientpositive/protectmode.q new file mode 100644 index 0000000000000..27055fbbb8235 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/protectmode.q @@ -0,0 +1,63 @@ +-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode + +drop table tbl1; +drop table tbl2; + +create table tbl1 (col string); +select * from tbl1; +select col from tbl1; +alter table tbl1 enable offline; +desc extended tbl1; +alter table tbl1 disable offline; +desc extended tbl1; +select * from tbl1; +select col from tbl1; + +create table tbl2 (col string) partitioned by (p string); +alter table tbl2 add partition (p='p1'); +alter table tbl2 add partition (p='p2'); +alter table tbl2 add partition (p='p3'); +alter table tbl2 drop partition (p='not_exist'); + +select * from tbl2 where p='p1'; +select * from tbl2 where p='p2'; + +alter table tbl2 partition (p='p1') enable offline; +desc extended tbl2 partition (p='p1'); + +alter table tbl2 enable offline; +desc extended tbl2; + +alter table tbl2 enable no_drop; +desc extended tbl2; +alter table tbl2 drop partition (p='p3'); + +alter table tbl2 disable offline; +desc extended tbl2; + +alter table tbl2 disable no_drop; +desc extended tbl2; + +select * from tbl2 where p='p2'; +select col from tbl2 where p='p2'; + +alter table tbl2 partition (p='p1') disable offline; +desc extended tbl2 partition (p='p1'); + +select * from tbl2 where p='p1'; +select col from tbl2 where p='p1'; + +insert overwrite table tbl1 select col from tbl2 where p='p1'; +insert overwrite table tbl1 select col from tbl1; + +alter table tbl2 partition (p='p1') enable no_drop; +alter table tbl2 partition (p='p1') disable no_drop; + +alter table tbl2 partition (p='p2') enable no_drop; + +alter table tbl2 drop partition (p='p1'); + +alter table tbl2 partition (p='p2') disable no_drop; + +drop table tbl1; +drop table tbl2; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/protectmode2.q b/src/test/hive/ql/src/test/queries/clientpositive/protectmode2.q new file mode 100644 index 0000000000000..978b03264bb6e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/protectmode2.q @@ -0,0 +1,23 @@ +drop table tbl1; + +create table tbl1 (col string); +alter table tbl1 enable no_drop cascade; +desc extended tbl1; +alter table tbl1 enable no_drop; +desc extended tbl1; +alter table tbl1 disable no_drop cascade; +desc extended tbl1; +alter table tbl1 disable no_drop; + +drop table tbl1; + +drop table tbl2; +create table tbl2 (col string) partitioned by (p string); +alter table tbl2 add partition (p='p1'); +alter table tbl2 add partition (p='p2'); +alter table tbl2 add partition (p='p3'); +alter table tbl2 enable no_drop cascade; +desc formatted tbl2; +alter table tbl2 disable no_drop cascade; +desc formatted tbl2; +drop table tbl2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf.q b/src/test/hive/ql/src/test/queries/clientpositive/ptf.q new file mode 100644 index 0000000000000..eea5415d682a1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ptf.q @@ -0,0 +1,306 @@ +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; + +--1. test1 +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part + partition by p_mfgr + order by p_name + ); + +-- 2. testJoinWithNoop +select p_mfgr, p_name, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from noop (on (select p1.* from part p1 join part p2 on p1.p_partkey = p2.p_partkey) j +distribute by j.p_mfgr +sort by j.p_name) +; + +-- 3. testOnlyPTF +select p_mfgr, p_name, p_size +from noop(on part +partition by p_mfgr +order by p_name); + +-- 4. testPTFAlias +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part + partition by p_mfgr + order by p_name + ) abc; + +-- 5. testPTFAndWhereWithWindowing +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +; + +-- 6. testSWQAndPTFAndGBy +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, p_size - lag(p_size,1,p_size) over (partition by p_mfgr order by p_name) as deltaSz +from noop(on part + partition by p_mfgr + order by p_name + ) +group by p_mfgr, p_name, p_size +; + +-- 7. testJoin +select abc.* +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey; + +-- 8. testJoinRight +select abc.* +from part p1 join noop(on part +partition by p_mfgr +order by p_name +) abc on abc.p_partkey = p1.p_partkey; + +-- 9. testNoopWithMap +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name, p_size desc) as r +from noopwithmap(on part +partition by p_mfgr +order by p_name, p_size desc); + +-- 10. testNoopWithMapWithWindowing +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noopwithmap(on part + partition by p_mfgr + order by p_name); + +-- 11. testHavingWithWindowingPTFNoGBY +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part +partition by p_mfgr +order by p_name) +; + +-- 12. testFunctionChain +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on noopwithmap(on noop(on part +partition by p_mfgr +order by p_mfgr, p_name +))); + +-- 13. testPTFAndWindowingInSubQ +select p_mfgr, p_name, +sub1.cd, sub1.s1 +from (select p_mfgr, p_name, +count(p_size) over (partition by p_mfgr order by p_name) as cd, +p_retailprice, +sum(p_retailprice) over w1 as s1 +from noop(on part +partition by p_mfgr +order by p_name) +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) +) sub1 ; + +-- 14. testPTFJoinWithWindowingWithCount +select abc.p_mfgr, abc.p_name, +rank() over (distribute by abc.p_mfgr sort by abc.p_name) as r, +dense_rank() over (distribute by abc.p_mfgr sort by abc.p_name) as dr, +count(abc.p_name) over (distribute by abc.p_mfgr sort by abc.p_name) as cd, +abc.p_retailprice, sum(abc.p_retailprice) over (distribute by abc.p_mfgr sort by abc.p_name rows between unbounded preceding and current row) as s1, +abc.p_size, abc.p_size - lag(abc.p_size,1,abc.p_size) over (distribute by abc.p_mfgr sort by abc.p_name) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +; + +-- 15. testDistinctInSelectWithPTF +select DISTINCT p_mfgr, p_name, p_size +from noop(on part +partition by p_mfgr +order by p_name); + + +-- 16. testViewAsTableInputToPTF +create view IF NOT EXISTS mfgr_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) as s +from part +group by p_mfgr, p_brand; + +select p_mfgr, p_brand, s, +sum(s) over w1 as s1 +from noop(on mfgr_price_view +partition by p_mfgr +order by p_mfgr) +window w1 as ( partition by p_mfgr order by p_brand rows between 2 preceding and current row); + +-- 17. testMultipleInserts2SWQsWithPTF +CREATE TABLE part_4( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +s DOUBLE); + +CREATE TABLE part_5( +p_mfgr STRING, +p_name STRING, +p_size INT, +s2 INT, +r INT, +dr INT, +cud DOUBLE, +fv1 INT); + +from noop(on part +partition by p_mfgr +order by p_name) +INSERT OVERWRITE TABLE part_4 select p_mfgr, p_name, p_size, +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, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s +INSERT OVERWRITE TABLE part_5 select p_mfgr,p_name, p_size, +round(sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row),1) as s2, +rank() over (distribute by p_mfgr sort by p_mfgr, p_name) as r, +dense_rank() over (distribute by p_mfgr sort by p_mfgr, p_name) as dr, +cume_dist() over (distribute by p_mfgr sort by p_mfgr, p_name) as cud, +first_value(p_size, true) over w1 as fv1 +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following); + +select * from part_4; + +select * from part_5; + +-- 18. testMulti2OperatorsFunctionChainWithMap +select p_mfgr, p_name, +rank() over (partition by p_mfgr,p_name) as r, +dense_rank() over (partition by p_mfgr,p_name) as dr, +p_size, sum(p_size) over (partition by p_mfgr,p_name rows between unbounded preceding and current row) as s1 +from noop(on + noopwithmap(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr,p_name + order by p_mfgr,p_name) ; + +-- 19. testMulti3OperatorsFunctionChain +select p_mfgr, p_name, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, sum(p_size) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr + order by p_mfgr) + ) + partition by p_mfgr,p_name + order by p_mfgr,p_name) + partition by p_mfgr + order by p_mfgr ) ; + +-- 20. testMultiOperatorChainWithNoWindowing +select p_mfgr, p_name, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, sum(p_size) over (partition by p_mfgr order by p_name) as s1 +from noop(on + noop(on + noop(on + noop(on part + partition by p_mfgr,p_name + order by p_mfgr,p_name) + ) + partition by p_mfgr + order by p_mfgr)); + + +-- 21. testMultiOperatorChainEndsWithNoopMap +select p_mfgr, p_name, +rank() over (partition by p_mfgr,p_name) as r, +dense_rank() over (partition by p_mfgr,p_name) as dr, +p_size, sum(p_size) over (partition by p_mfgr,p_name rows between unbounded preceding and current row) as s1 +from noopwithmap(on + noop(on + noop(on + noop(on part + partition by p_mfgr,p_name + order by p_mfgr,p_name) + ) + partition by p_mfgr + order by p_mfgr) + partition by p_mfgr,p_name + order by p_mfgr,p_name); + +-- 22. testMultiOperatorChainWithDiffPartitionForWindow1 +select p_mfgr, p_name, +rank() over (partition by p_mfgr,p_name order by p_mfgr,p_name) as r, +dense_rank() over (partition by p_mfgr,p_name order by p_mfgr,p_name) as dr, +p_size, +sum(p_size) over (partition by p_mfgr,p_name order by p_mfgr,p_name rows between unbounded preceding and current row) as s1, +sum(p_size) over (partition by p_mfgr,p_name order by p_mfgr,p_name rows between unbounded preceding and current row) as s2 +from noop(on + noopwithmap(on + noop(on part + partition by p_mfgr, p_name + order by p_mfgr, p_name) + partition by p_mfgr + order by p_mfgr + )); + +-- 23. testMultiOperatorChainWithDiffPartitionForWindow2 +select p_mfgr, p_name, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +p_size, +sum(p_size) over (partition by p_mfgr order by p_name range between unbounded preceding and current row) as s1, +sum(p_size) over (partition by p_mfgr order by p_name range between unbounded preceding and current row) as s2 +from noopwithmap(on + noop(on + noop(on part + partition by p_mfgr, p_name + order by p_mfgr, p_name) + )); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_decimal.q b/src/test/hive/ql/src/test/queries/clientpositive/ptf_decimal.q new file mode 100644 index 0000000000000..03f435e4539e2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ptf_decimal.q @@ -0,0 +1,49 @@ +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 DECIMAL, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; + +-- 1. aggregate functions with decimal type + +select p_mfgr, p_retailprice, +lead(p_retailprice) over (partition by p_mfgr ORDER BY p_name) as c1, +lag(p_retailprice) over (partition by p_mfgr ORDER BY p_name) as c2, +first_value(p_retailprice) over (partition by p_mfgr ORDER BY p_name) as c3, +last_value(p_retailprice) over (partition by p_mfgr ORDER BY p_name) as c4 +from part; + +-- 2. ranking functions with decimal type + +select p_mfgr, p_retailprice, +row_number() over (PARTITION BY p_mfgr ORDER BY p_retailprice) as c1, +rank() over (PARTITION BY p_mfgr ORDER BY p_retailprice) as c2, +dense_rank() over (PARTITION BY p_mfgr ORDER BY p_retailprice) as c3, +percent_rank() over (PARTITION BY p_mfgr ORDER BY p_retailprice) as c4, +cume_dist() over (PARTITION BY p_mfgr ORDER BY p_retailprice) as c5, +ntile(5) over (PARTITION BY p_mfgr ORDER BY p_retailprice) as c6 +from part; + +-- 3. order by decimal + +select p_mfgr, p_retailprice, +lag(p_retailprice) over (partition by p_mfgr ORDER BY p_retailprice desc) as c1 +from part; + +-- 4. partition by decimal + +select p_mfgr, p_retailprice, +lag(p_retailprice) over (partition by p_retailprice) as c1 +from part; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_general_queries.q b/src/test/hive/ql/src/test/queries/clientpositive/ptf_general_queries.q new file mode 100644 index 0000000000000..885c3b3d43e17 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ptf_general_queries.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; + +-- 1. testNoPTFNoWindowing +select p_mfgr, p_name, p_size +from part +distribute by p_mfgr +sort by p_name ; + +-- 2. testUDAFsNoWindowingNoPTFNoGBY +select p_mfgr,p_name, p_retailprice, +sum(p_retailprice) over(partition by p_mfgr order by p_name) as s, +min(p_retailprice) over(partition by p_mfgr order by p_name) as mi, +max(p_retailprice) over(partition by p_mfgr order by p_name) as ma, +avg(p_retailprice) over(partition by p_mfgr order by p_name) as av +from part +; + +-- 3. testConstExprInSelect +select 'tst1' as key, count(1) as value from part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_matchpath.q b/src/test/hive/ql/src/test/queries/clientpositive/ptf_matchpath.q new file mode 100644 index 0000000000000..72eeb104d5298 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ptf_matchpath.q @@ -0,0 +1,36 @@ +DROP TABLE flights_tiny; + +create table flights_tiny ( +ORIGIN_CITY_NAME string, +DEST_CITY_NAME string, +YEAR int, +MONTH int, +DAY_OF_MONTH int, +ARR_DELAY float, +FL_NUM string +); + +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 +from matchpath(on + flights_tiny + distribute by fl_num + sort by year, month, day_of_month + arg1('LATE.LATE+'), + arg2('LATE'), arg3(arr_delay > 15), + arg4('origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath') + ); + +-- 2. Matchpath on 1 partition +select origin_city_name, fl_num, year, month, day_of_month, sz, tpath +from matchpath(on + flights_tiny + sort by fl_num, year, month, day_of_month + arg1('LATE.LATE+'), + arg2('LATE'), arg3(arr_delay > 15), + arg4('origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath') + ) +where fl_num = 1142; + \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_rcfile.q b/src/test/hive/ql/src/test/queries/clientpositive/ptf_rcfile.q new file mode 100644 index 0000000000000..535a233a9e3c4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ptf_rcfile.q @@ -0,0 +1,24 @@ +DROP TABLE part_rc; + +CREATE TABLE part_rc( + 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 +) STORED AS RCFILE ; + +LOAD DATA LOCAL INPATH '../data/files/part.rc' overwrite into table part_rc; + +-- testWindowingPTFWithPartRC +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part_rc +partition by p_mfgr +order by p_name); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_register_tblfn.q b/src/test/hive/ql/src/test/queries/clientpositive/ptf_register_tblfn.q new file mode 100644 index 0000000000000..a2140cd049f2d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ptf_register_tblfn.q @@ -0,0 +1,29 @@ +DROP TABLE flights_tiny; + +create table flights_tiny ( +ORIGIN_CITY_NAME string, +DEST_CITY_NAME string, +YEAR int, +MONTH int, +DAY_OF_MONTH int, +ARR_DELAY float, +FL_NUM string +); + +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'; + + +-- 1. basic Matchpath test +select origin_city_name, fl_num, year, month, day_of_month, sz, tpath +from matchpathtest(on + flights_tiny + distribute by fl_num + sort by year, month, day_of_month + arg1('LATE.LATE+'), + arg2('LATE'), arg3(arr_delay > 15), + arg4('origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath') + ); + +drop temporary function matchpathtest; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_seqfile.q b/src/test/hive/ql/src/test/queries/clientpositive/ptf_seqfile.q new file mode 100644 index 0000000000000..4aa8ce11bec9c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ptf_seqfile.q @@ -0,0 +1,24 @@ +DROP TABLE part_seq; + +CREATE TABLE part_seq( + 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 +) STORED AS SEQUENCEFILE ; + +LOAD DATA LOCAL INPATH '../data/files/part.seq' overwrite into table part_seq; + +-- testWindowingPTFWithPartSeqFile +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name) as r, +dense_rank() over (partition by p_mfgr order by p_name) as dr, +sum(p_retailprice) over (partition by p_mfgr order by p_name rows between unbounded preceding and current row) as s1 +from noop(on part_seq +partition by p_mfgr +order by p_name); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/push_or.q b/src/test/hive/ql/src/test/queries/clientpositive/push_or.q new file mode 100644 index 0000000000000..c9b6cea23cd85 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/push_or.q @@ -0,0 +1,10 @@ +drop table push_or; + +create table push_or (key int, value string) partitioned by (ds string); + +insert overwrite table push_or partition (ds='2000-04-08') select * from src where key < 20 order by key; +insert overwrite table push_or partition (ds='2000-04-09') select * from src where key < 20 order by key; + +explain extended select key, value, ds from push_or where ds='2000-04-09' or key=5 order by key, ds; +select key, value, ds from push_or where ds='2000-04-09' or key=5 order by key, ds; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q b/src/test/hive/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q new file mode 100644 index 0000000000000..f198baa6e4c5b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q @@ -0,0 +1,170 @@ + +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; + +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; + +explain select l_shipdate, count(l_shipdate) +from lineitem +group by l_shipdate; + +select l_shipdate, count(l_shipdate) +from lineitem +group by l_shipdate +order by l_shipdate; + +set hive.optimize.index.groupby=true; + +explain select l_shipdate, count(l_shipdate) +from lineitem +group by l_shipdate; + +select l_shipdate, count(l_shipdate) +from lineitem +group by l_shipdate +order by l_shipdate; + +set hive.optimize.index.groupby=false; + + +explain select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem +group by year(l_shipdate), month(l_shipdate) +order by year, month; + +select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem +group by year(l_shipdate), month(l_shipdate) +order by year, month; + +set hive.optimize.index.groupby=true; + +explain select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem +group by year(l_shipdate), month(l_shipdate) +order by year, month; + +select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments +from lineitem +group by year(l_shipdate), month(l_shipdate) +order by year, month; + +explain select lastyear.month, + thisyear.month, + (thisyear.monthly_shipments - lastyear.monthly_shipments) / +lastyear.monthly_shipments as monthly_shipments_delta + from (select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments + from lineitem + where year(l_shipdate) = 1997 + group by year(l_shipdate), month(l_shipdate) + ) lastyear join + (select year(l_shipdate) as year, + month(l_shipdate) as month, + count(l_shipdate) as monthly_shipments + from lineitem + where year(l_shipdate) = 1998 + group by year(l_shipdate), month(l_shipdate) + ) thisyear + on lastyear.month = thisyear.month; + +explain select l_shipdate, cnt +from (select l_shipdate, count(l_shipdate) as cnt from lineitem group by l_shipdate +union all +select l_shipdate, l_orderkey as cnt +from lineitem) dummy; + +CREATE TABLE tbl(key int, value int); +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; + +EXPLAIN select key, count(key) from tbl where key = 1 group by key; +EXPLAIN select key, count(key) from tbl group by key; + +EXPLAIN select count(1) from tbl; +EXPLAIN select count(key) from tbl; + +EXPLAIN select key FROM tbl GROUP BY key; +EXPLAIN select key FROM tbl GROUP BY value, key; +EXPLAIN select key FROM tbl WHERE key = 3 GROUP BY key; +EXPLAIN select key FROM tbl WHERE value = 2 GROUP BY key; +EXPLAIN select key FROM tbl GROUP BY key, substr(key,2,3); + +EXPLAIN select key, value FROM tbl GROUP BY value, key; +EXPLAIN select key, value FROM tbl WHERE value = 1 GROUP BY key, value; + +EXPLAIN select DISTINCT key FROM tbl; +EXPLAIN select DISTINCT key FROM tbl; +EXPLAIN select DISTINCT key FROM tbl; +EXPLAIN select DISTINCT key, value FROM tbl; +EXPLAIN select DISTINCT key, value FROM tbl WHERE value = 2; +EXPLAIN select DISTINCT key, value FROM tbl WHERE value = 2 AND key = 3; +EXPLAIN select DISTINCT key, value FROM tbl WHERE value = key; +EXPLAIN select DISTINCT key, substr(value,2,3) FROM tbl WHERE value = key; +EXPLAIN select DISTINCT key, substr(value,2,3) FROM tbl; + +EXPLAIN select * FROM (select DISTINCT key, value FROM tbl) v1 WHERE v1.value = 2; + +DROP TABLE tbl; + +CREATE TABLE tblpart (key int, value string) PARTITIONED BY (ds string, hr int); +INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-08', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 11; +INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-08', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 12; +INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-09', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 11; +INSERT OVERWRITE TABLE tblpart PARTITION (ds='2008-04-09', hr=12) SELECT key, value FROM srcpart WHERE ds = '2008-04-09' AND hr = 12; + +CREATE INDEX tbl_part_index ON TABLE tblpart(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)"); + +ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-08', hr=11) REBUILD; +EXPLAIN SELECT key, count(key) FROM tblpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 GROUP BY key; + +ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-08', hr=12) REBUILD; +ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-09', hr=11) REBUILD; +ALTER INDEX tbl_part_index ON tblpart PARTITION (ds='2008-04-09', hr=12) REBUILD; +EXPLAIN SELECT key, count(key) FROM tblpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 GROUP BY key; + +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; + +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; + +set hive.optimize.index.groupby=false; +explain select key, count(key) from tbl group by key order by key; +select key, count(key) from tbl group by key order by key; +set hive.optimize.index.groupby=true; +explain select key, count(key) from tbl group by key order by key; +select key, count(key) from tbl group by key order by key; +DROP TABLE tbl; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/query_properties.q b/src/test/hive/ql/src/test/queries/clientpositive/query_properties.q new file mode 100644 index 0000000000000..027ba7f7a833c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/query_properties.q @@ -0,0 +1,15 @@ +set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.CheckQueryPropertiesHook; + +select * from src a join src b on a.key = b.key limit 0; +select * from src group by src.key, src.value limit 0; +select * from src order by src.key limit 0; +select * from src sort by src.key limit 0; +select a.key, sum(b.value) from src a join src b on a.key = b.key group by a.key limit 0; +select transform(*) using 'cat' from src limit 0; +select * from src distribute by src.key limit 0; +select * from src cluster by src.key limit 0; + +select key, sum(value) from (select a.key as key, b.value as value from src a join src b on a.key = b.key) c group by key limit 0; +select * from src a join src b on a.key = b.key order by a.key limit 0; +select * from src a join src b on a.key = b.key distribute by a.key sort by a.key, b.value limit 0; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/query_result_fileformat.q b/src/test/hive/ql/src/test/queries/clientpositive/query_result_fileformat.q new file mode 100644 index 0000000000000..3b87e4d170c1f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/query_result_fileformat.q @@ -0,0 +1,23 @@ +create table nzhang_test1 stored as sequencefile as select 'key1' as key, 'value +1 + +http://asdf' value from src limit 1; + +select * from nzhang_test1; +select count(*) from nzhang_test1; + +explain +select * from nzhang_test1 where key='key1'; + +select * from nzhang_test1 where key='key1'; + +set hive.query.result.fileformat=SequenceFile; + +select * from nzhang_test1; + +select count(*) from nzhang_test1; + +explain +select * from nzhang_test1 where key='key1'; + +select * from nzhang_test1 where key='key1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/query_with_semi.q b/src/test/hive/ql/src/test/queries/clientpositive/query_with_semi.q new file mode 100644 index 0000000000000..e0f90d5edb3e6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/query_with_semi.q @@ -0,0 +1,6 @@ +from src +select transform('aa\;') using 'cat' as a limit 1; + +from src +select transform('bb') using 'cat' as b limit 1; from src +select transform('cc') using 'cat' as c limit 1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/quote1.q b/src/test/hive/ql/src/test/queries/clientpositive/quote1.q new file mode 100644 index 0000000000000..f968132fa5913 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/quote1.q @@ -0,0 +1,13 @@ +CREATE TABLE dest1(`location` INT, `type` STRING) PARTITIONED BY(`table` STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 PARTITION(`table`='2008-04-08') SELECT src.key as `partition`, src.value as `from` WHERE src.key >= 200 and src.key < 300; + +EXPLAIN +SELECT `int`.`location`, `int`.`type`, `int`.`table` FROM dest1 `int` WHERE `int`.`table` = '2008-04-08'; + +FROM src +INSERT OVERWRITE TABLE dest1 PARTITION(`table`='2008-04-08') SELECT src.key as `partition`, src.value as `from` WHERE src.key >= 200 and src.key < 300; + +SELECT `int`.`location`, `int`.`type`, `int`.`table` FROM dest1 `int` WHERE `int`.`table` = '2008-04-08'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/quote2.q b/src/test/hive/ql/src/test/queries/clientpositive/quote2.q new file mode 100644 index 0000000000000..65b9f8776d192 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/quote2.q @@ -0,0 +1,28 @@ +EXPLAIN +SELECT + 'abc', "abc", + 'abc\'', "abc\"", + 'abc\\', "abc\\", + 'abc\\\'', "abc\\\"", + 'abc\\\\', "abc\\\\", + 'abc\\\\\'', "abc\\\\\"", + 'abc\\\\\\', "abc\\\\\\", + 'abc""""\\', "abc''''\\", + "awk '{print NR\"\\t\"$0}'", + 'tab\ttab', "tab\ttab" +FROM src +LIMIT 1; + +SELECT + 'abc', "abc", + 'abc\'', "abc\"", + 'abc\\', "abc\\", + 'abc\\\'', "abc\\\"", + 'abc\\\\', "abc\\\\", + 'abc\\\\\'', "abc\\\\\"", + 'abc\\\\\\', "abc\\\\\\", + 'abc""""\\', "abc''''\\", + "awk '{print NR\"\\t\"$0}'", + 'tab\ttab', "tab\ttab" +FROM src +LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner1.q b/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner1.q new file mode 100644 index 0000000000000..3eeb5e7b88f6e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner1.q @@ -0,0 +1,3 @@ +-- scanning un-partitioned data +explain extended select * from src where rand(1) < 0.1; +select * from src where rand(1) < 0.1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner2.q b/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner2.q new file mode 100644 index 0000000000000..e2f280efe754f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner2.q @@ -0,0 +1,14 @@ +-- scanning partitioned data + +create table tmptable(key string, value string, hr string, ds string); + +explain extended +insert overwrite table tmptable +select a.* from srcpart a where rand(1) < 0.1 and a.ds = '2008-04-08'; + + +insert overwrite table tmptable +select a.* from srcpart a where rand(1) < 0.1 and a.ds = '2008-04-08'; + +select * from tmptable x sort by x.key,x.value,x.ds,x.hr; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner3.q b/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner3.q new file mode 100644 index 0000000000000..9f4aaac267864 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner3.q @@ -0,0 +1,9 @@ +set hive.optimize.ppd=true; +-- complex predicates in the where clause + +explain extended select a.* from srcpart a where rand(1) < 0.1 and a.ds = '2008-04-08' and not(key > 50 or key < 10) and a.hr like '%2'; +select a.* from srcpart a where rand(1) < 0.1 and a.ds = '2008-04-08' and not(key > 50 or key < 10) and a.hr like '%2'; + +-- without rand for comparison +explain extended select a.* from srcpart a where a.ds = '2008-04-08' and not(key > 50 or key < 10) and a.hr like '%2'; +select a.* from srcpart a where a.ds = '2008-04-08' and not(key > 50 or key < 10) and a.hr like '%2'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_bigdata.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_bigdata.q new file mode 100644 index 0000000000000..3e83e6693b276 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_bigdata.q @@ -0,0 +1,19 @@ +set hive.map.aggr.hash.percentmemory = 0.3; +set hive.mapred.local.mem = 256; + +add file ../data/scripts/dumpdata_script.py; + +CREATE table columnTable_Bigdata (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 (FROM src MAP src.key,src.value USING 'python dumpdata_script.py' AS (key,value) WHERE src.key = 10) subq +INSERT OVERWRITE TABLE columnTable_Bigdata SELECT subq.key, subq.value; + +describe columnTable_Bigdata; +select count(columnTable_Bigdata.key) from columnTable_Bigdata; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_columnar.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_columnar.q new file mode 100644 index 0000000000000..0178e535cdb90 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_columnar.q @@ -0,0 +1,15 @@ + +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; + +SELECT columnTable.* FROM columnTable ORDER BY key ASC, value ASC; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_createas1.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_createas1.q new file mode 100644 index 0000000000000..f36203724c15f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_createas1.q @@ -0,0 +1,35 @@ +set hive.merge.rcfile.block.level=true; +set mapred.max.split.size=100; +set mapred.min.split.size=1; + +DROP TABLE rcfile_createas1a; +DROP TABLE rcfile_createas1b; + +CREATE TABLE rcfile_createas1a (key INT, value STRING) + PARTITIONED BY (ds string); +INSERT OVERWRITE TABLE rcfile_createas1a PARTITION (ds='1') + SELECT * FROM src; +INSERT OVERWRITE TABLE rcfile_createas1a PARTITION (ds='2') + SELECT * FROM src; + +EXPLAIN + CREATE TABLE rcfile_createas1b + STORED AS RCFILE AS + SELECT key, value, PMOD(HASH(key), 50) as part + FROM rcfile_createas1a; +CREATE TABLE rcfile_createas1b + STORED AS RCFILE AS + SELECT key, value, PMOD(HASH(key), 50) as part + FROM rcfile_createas1a; + +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(key, value) USING 'tr \t _' AS (c) + FROM rcfile_createas1a +) t; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(key, value) USING 'tr \t _' AS (c) + FROM rcfile_createas1b +) t; + +DROP TABLE rcfile_createas1a; +DROP TABLE rcfile_createas1b; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_default_format.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_default_format.q new file mode 100644 index 0000000000000..e61292e7f1edb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_default_format.q @@ -0,0 +1,31 @@ +SET hive.default.fileformat = RCFile; + +CREATE TABLE rcfile_default_format (key STRING); +DESCRIBE FORMATTED rcfile_default_format; + +CREATE TABLE rcfile_default_format_ctas AS SELECT key,value FROM src; +DESCRIBE FORMATTED rcfile_default_format_ctas; + +CREATE TABLE rcfile_default_format_txtfile (key STRING) STORED AS TEXTFILE; +INSERT OVERWRITE TABLE rcfile_default_format_txtfile SELECT key from src; +DESCRIBE FORMATTED rcfile_default_format_txtfile; + +SET hive.default.fileformat = TextFile; +CREATE TABLE textfile_default_format_ctas AS SELECT key,value FROM rcfile_default_format_ctas; +DESCRIBE FORMATTED textfile_default_format_ctas; + +SET hive.default.fileformat = RCFile; +SET hive.default.rcfile.serde = org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe; +CREATE TABLE rcfile_default_format_ctas_default_serde AS SELECT key,value FROM rcfile_default_format_ctas; +DESCRIBE FORMATTED rcfile_default_format_ctas_default_serde; + +CREATE TABLE rcfile_default_format_default_serde (key STRING); +DESCRIBE FORMATTED rcfile_default_format_default_serde; + +SET hive.default.fileformat = TextFile; +CREATE TABLE rcfile_ctas_default_serde STORED AS rcfile AS SELECT key,value FROM rcfile_default_format_ctas; +DESCRIBE FORMATTED rcfile_ctas_default_serde; + +CREATE TABLE rcfile_default_serde (key STRING) STORED AS rcfile; +DESCRIBE FORMATTED rcfile_default_serde; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q new file mode 100644 index 0000000000000..7f55d10bd6458 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q @@ -0,0 +1,27 @@ + +CREATE table rcfileTableLazyDecompress (key STRING, value STRING) STORED AS RCFile; + +FROM src +INSERT OVERWRITE TABLE rcfileTableLazyDecompress SELECT src.key, src.value LIMIT 10; + +SELECT key, value FROM rcfileTableLazyDecompress where key > 238 ORDER BY key ASC, value ASC; + +SELECT key, value FROM rcfileTableLazyDecompress where key > 238 and key < 400 ORDER BY key ASC, value ASC; + +SELECT key, count(1) FROM rcfileTableLazyDecompress where key > 238 group by key ORDER BY key ASC; + +set mapred.output.compress=true; +set hive.exec.compress.output=true; + +FROM src +INSERT OVERWRITE TABLE rcfileTableLazyDecompress SELECT src.key, src.value LIMIT 10; + +SELECT key, value FROM rcfileTableLazyDecompress where key > 238 ORDER BY key ASC, value ASC; + +SELECT key, value FROM rcfileTableLazyDecompress where key > 238 and key < 400 ORDER BY key ASC, value ASC; + +SELECT key, count(1) FROM rcfileTableLazyDecompress where key > 238 group by key ORDER BY key ASC; + +set mapred.output.compress=false; +set hive.exec.compress.output=false; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge1.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge1.q new file mode 100644 index 0000000000000..1f6f1bd251c25 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge1.q @@ -0,0 +1,43 @@ +set hive.merge.rcfile.block.level=false; +set hive.exec.dynamic.partition=true; +set mapred.max.split.size=100; +set mapref.min.split.size=1; + +DROP TABLE rcfile_merge1; +DROP TABLE rcfile_merge1b; + +CREATE TABLE rcfile_merge1 (key INT, value STRING) + PARTITIONED BY (ds STRING, part STRING) STORED AS RCFILE; +CREATE TABLE rcfile_merge1b (key INT, value STRING) + PARTITIONED BY (ds STRING, part STRING) STORED AS RCFILE; + +-- Use non block-level merge +EXPLAIN + INSERT OVERWRITE TABLE rcfile_merge1 PARTITION (ds='1', part) + SELECT key, value, PMOD(HASH(key), 100) as part + FROM src; +INSERT OVERWRITE TABLE rcfile_merge1 PARTITION (ds='1', part) + SELECT key, value, PMOD(HASH(key), 100) as part + FROM src; + +set hive.merge.rcfile.block.level=true; +EXPLAIN + INSERT OVERWRITE TABLE rcfile_merge1b PARTITION (ds='1', part) + SELECT key, value, PMOD(HASH(key), 100) as part + FROM src; +INSERT OVERWRITE TABLE rcfile_merge1b PARTITION (ds='1', part) + SELECT key, value, PMOD(HASH(key), 100) as part + FROM src; + +-- Verify +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) + FROM rcfile_merge1 WHERE ds='1' +) t; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) + FROM rcfile_merge1b WHERE ds='1' +) t; + +DROP TABLE rcfile_merge1; +DROP TABLE rcfile_merge1b; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge2.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge2.q new file mode 100644 index 0000000000000..215d5ebc4a25b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge2.q @@ -0,0 +1,32 @@ +set hive.merge.rcfile.block.level=true; +set hive.exec.dynamic.partition=true; +set mapred.max.split.size=100; +set mapred.min.split.size=1; + +DROP TABLE rcfile_merge2a; + +CREATE TABLE rcfile_merge2a (key INT, value STRING) + PARTITIONED BY (one string, two string, three string) + STORED AS RCFILE; + +EXPLAIN INSERT OVERWRITE TABLE rcfile_merge2a PARTITION (one='1', two, three) + SELECT key, value, PMOD(HASH(key), 10) as two, + PMOD(HASH(value), 10) as three + FROM src; +INSERT OVERWRITE TABLE rcfile_merge2a PARTITION (one='1', two, three) + SELECT key, value, PMOD(HASH(key), 10) as two, + PMOD(HASH(value), 10) as three + FROM src; + +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(*) USING 'tr \t _' AS (c) + FROM rcfile_merge2a +) t; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(key, value, '1', PMOD(HASH(key), 10), + PMOD(HASH(value), 10)) USING 'tr \t _' AS (c) + FROM src +) t; + +DROP TABLE rcfile_merge2a; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge3.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge3.q new file mode 100644 index 0000000000000..39fbd2564664b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge3.q @@ -0,0 +1,32 @@ +set hive.merge.rcfile.block.level=true; +set mapred.max.split.size=100; +set mapred.min.split.size=1; + +DROP TABLE rcfile_merge3a; +DROP TABLE rcfile_merge3b; + +CREATE TABLE rcfile_merge3a (key int, value string) + PARTITIONED BY (ds string) STORED AS TEXTFILE; +CREATE TABLE rcfile_merge3b (key int, value string) STORED AS RCFILE; + +INSERT OVERWRITE TABLE rcfile_merge3a PARTITION (ds='1') + SELECT * FROM src; +INSERT OVERWRITE TABLE rcfile_merge3a PARTITION (ds='2') + SELECT * FROM src; + +EXPLAIN INSERT OVERWRITE TABLE rcfile_merge3b + SELECT key, value FROM rcfile_merge3a; +INSERT OVERWRITE TABLE rcfile_merge3b + SELECT key, value FROM rcfile_merge3a; + +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(key, value) USING 'tr \t _' AS (c) + FROM rcfile_merge3a +) t; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(key, value) USING 'tr \t _' AS (c) + FROM rcfile_merge3b +) t; + +DROP TABLE rcfile_merge3a; +DROP TABLE rcfile_merge3b; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge4.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge4.q new file mode 100644 index 0000000000000..fe6df28566cf0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge4.q @@ -0,0 +1,32 @@ +set hive.merge.rcfile.block.level=true; +set mapred.max.split.size=100; +set mapred.min.split.size=1; + +DROP TABLE rcfile_merge3a; +DROP TABLE rcfile_merge3b; + +CREATE TABLE rcfile_merge3a (key int, value string) + PARTITIONED BY (ds string) STORED AS RCFILE; +CREATE TABLE rcfile_merge3b (key int, value string) STORED AS TEXTFILE; + +INSERT OVERWRITE TABLE rcfile_merge3a PARTITION (ds='1') + SELECT * FROM src; +INSERT OVERWRITE TABLE rcfile_merge3a PARTITION (ds='2') + SELECT * FROM src; + +EXPLAIN INSERT OVERWRITE TABLE rcfile_merge3b + SELECT key, value FROM rcfile_merge3a; +INSERT OVERWRITE TABLE rcfile_merge3b + SELECT key, value FROM rcfile_merge3a; + +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(key, value) USING 'tr \t _' AS (c) + FROM rcfile_merge3a +) t; +SELECT SUM(HASH(c)) FROM ( + SELECT TRANSFORM(key, value) USING 'tr \t _' AS (c) + FROM rcfile_merge3b +) t; + +DROP TABLE rcfile_merge3a; +DROP TABLE rcfile_merge3b; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_null_value.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_null_value.q new file mode 100644 index 0000000000000..2e4c929fb3f52 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_null_value.q @@ -0,0 +1,40 @@ +CREATE TABLE src1_rc(key STRING, value STRING) STORED AS RCFILE; +INSERT OVERWRITE TABLE src1_rc SELECT * FROM src1; +SELECT * FROM src1_rc; + + +CREATE TABLE dest1_rc(c1 INT, c2 STRING, c3 INT, c4 STRING) STORED AS RCFILE; + +EXPLAIN +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + RIGHT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1_rc SELECT c.c1, c.c2, c.c3, c.c4; + +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + RIGHT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +INSERT OVERWRITE TABLE dest1_rc SELECT c.c1, c.c2, c.c3, c.c4; + +SELECT dest1_rc.* FROM dest1_rc; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q new file mode 100644 index 0000000000000..077a52d3934ee --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q @@ -0,0 +1,8 @@ +CREATE TABLE test_src(key int, value string) stored as RCFILE; +set hive.io.rcfile.record.interval=5; +set hive.io.rcfile.record.buffer.size=100; +set hive.exec.compress.output=true; +INSERT OVERWRITE table test_src SELECT * FROM src; + +set hive.io.rcfile.tolerate.corruptions=true; +SELECT key, value FROM test_src order by key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_union.q b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_union.q new file mode 100644 index 0000000000000..fc8c7500804f2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rcfile_union.q @@ -0,0 +1,14 @@ + +CREATE table rcfile_unionTable (b STRING, c STRING) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' +STORED AS RCFILE; + +FROM src +INSERT OVERWRITE TABLE rcfile_unionTable SELECT src.key, src.value LIMIT 10; + +SELECT * FROM ( +SELECT b AS cola FROM rcfile_unionTable +UNION ALL +SELECT c AS cola FROM rcfile_unionTable) s ORDER BY cola ASC; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/recursive_dir.q b/src/test/hive/ql/src/test/queries/clientpositive/recursive_dir.q new file mode 100644 index 0000000000000..709c6eb454ee6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/recursive_dir.q @@ -0,0 +1,20 @@ +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +CREATE TABLE fact_daily(x int) PARTITIONED BY (ds STRING); +CREATE TABLE fact_tz(x int) PARTITIONED BY (ds STRING, hr STRING) +LOCATION 'pfile:${system:test.tmp.dir}/fact_tz'; + +INSERT OVERWRITE TABLE fact_tz PARTITION (ds='1', hr='1') +SELECT key+11 FROM src WHERE key=484; + +ALTER TABLE fact_daily SET TBLPROPERTIES('EXTERNAL'='TRUE'); +ALTER TABLE fact_daily ADD PARTITION (ds='1') +LOCATION 'pfile:${system:test.tmp.dir}/fact_tz/ds=1'; + +set hive.mapred.supports.subdirectories=true; +set mapred.input.dir.recursive=true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +SELECT * FROM fact_daily WHERE ds='1'; + +SELECT count(1) FROM fact_daily WHERE ds='1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate.q b/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate.q new file mode 100644 index 0000000000000..2e26adc9ed957 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate.q @@ -0,0 +1,45 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.enforce.bucketing = true; +set hive.exec.reducers.max = 1; +set hive.exec.script.trust = true; +set hive.optimize.reducededuplication = true; +set hive.optimize.reducededuplication.min.reducer = 1; + + +CREATE TABLE bucket5_1(key string, value string) CLUSTERED BY (key) INTO 2 BUCKETS; +explain extended +insert overwrite table bucket5_1 +select * from src cluster by key; + +insert overwrite table bucket5_1 +select * from src cluster by key; + +select sum(hash(key)),sum(hash(value)) from bucket5_1; +select sum(hash(key)),sum(hash(value)) from src; + + +create table complex_tbl_1(aid string, bid string, t int, ctime string, etime bigint, l string, et string) partitioned by (ds string); + + +create table complex_tbl_2(aet string, aes string) partitioned by (ds string); + +explain extended +insert overwrite table complex_tbl_1 partition (ds='2010-03-29') +select s2.* from +( + select TRANSFORM (aid,bid,t,ctime,etime,l,et) + USING 'cat' + AS (aid string, bid string, t int, ctime string, etime bigint, l string, et string) + from + ( + select transform(aet,aes) + using 'cat' + as (aid string, bid string, t int, ctime string, etime bigint, l string, et string) + from complex_tbl_2 where ds ='2010-03-29' cluster by bid +)s +)s2; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q b/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q new file mode 100644 index 0000000000000..e28ed9fc536bf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q @@ -0,0 +1,8 @@ +create table t1( key_int1 int, key_int2 int, key_string1 string, key_string2 string); + +set hive.optimize.reducededuplication=false; + +set hive.map.aggr=false; +select Q1.key_int1, sum(Q1.key_int1) from (select * from t1 cluster by key_int1) Q1 group by Q1.key_int1; + +drop table t1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q b/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q new file mode 100644 index 0000000000000..ff09754dd118e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q @@ -0,0 +1,3 @@ +set hive.optimize.reducededuplication=true; +set hive.auto.convert.join=true; +explain select * from (select * from src cluster by key) a join src b on a.key = b.key limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q b/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q new file mode 100644 index 0000000000000..48af2c2affc4a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q @@ -0,0 +1,55 @@ +set hive.optimize.reducededuplication=true; +set hive.optimize.reducededuplication.min.reducer=1; +set hive.map.aggr=true; + +-- HIVE-2340 deduplicate RS followed by RS +-- hive.optimize.reducededuplication : wherther using this optimization +-- hive.optimize.reducededuplication.min.reducer : number of reducer of deduped RS should be this at least + +-- RS-mGBY-RS-rGBY +explain select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key; +explain select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value); +explain select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1); +-- mGBY-RS-rGBY-RS +explain select key, sum(key) as value from src group by key order by key, value; +-- RS-JOIN-mGBY-RS-rGBY +explain select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value; +-- RS-JOIN-RS +explain select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value; +-- mGBY-RS-rGBY-mGBY-RS-rGBY +explain from (select key, value from src group by key, value) s select s.key group by s.key; +explain select key, count(distinct value) from (select key, value from src group by key, value) t group by key; + +select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key; +select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value); +select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1); +select key, sum(key) as value from src group by key order by key, value; +select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value; +select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value; +from (select key, value from src group by key, value) s select s.key group by s.key; +select key, count(distinct value) from (select key, value from src group by key, value) t group by key; + +set hive.map.aggr=false; + +-- RS-RS-GBY +explain select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key; +explain select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value); +explain select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1); +-- RS-GBY-RS +explain select key, sum(key) as value from src group by key order by key, value; +-- RS-JOIN-RS-GBY +explain select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value; +-- RS-JOIN-RS +explain select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value; +-- RS-GBY-RS-GBY +explain from (select key, value from src group by key, value) s select s.key group by s.key; +explain select key, count(distinct value) from (select key, value from src group by key, value) t group by key; + +select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key; +select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value); +select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1); +select key, sum(key) as value from src group by key order by key, value; +select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value; +select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value; +from (select key, value from src group by key, value) s select s.key group by s.key; +select key, count(distinct value) from (select key, value from src group by key, value) t group by key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reducesink_dedup.q b/src/test/hive/ql/src/test/queries/clientpositive/reducesink_dedup.q new file mode 100644 index 0000000000000..d5aa2513c8b3b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/reducesink_dedup.q @@ -0,0 +1,20 @@ +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 +); + + +select p_name +from (select p_name from part distribute by 1 sort by 1) p +distribute by 1 sort by 1 +; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/regex_col.q b/src/test/hive/ql/src/test/queries/clientpositive/regex_col.q new file mode 100644 index 0000000000000..9cfcee5a599c6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/regex_col.q @@ -0,0 +1,35 @@ +EXPLAIN +SELECT * FROM srcpart; + +EXPLAIN +SELECT `..` FROM srcpart; + +EXPLAIN +SELECT srcpart.`..` FROM srcpart; + +EXPLAIN +SELECT `..` FROM srcpart a JOIN srcpart b +ON a.key = b.key AND a.value = b.value; + +EXPLAIN +SELECT b.`..` FROM srcpart a JOIN srcpart b +ON a.key = b.key AND a.hr = b.hr AND a.ds = b.ds AND a.key = 103 +ORDER BY ds, hr; + +SELECT b.`..` FROM srcpart a JOIN srcpart b +ON a.key = b.key AND a.hr = b.hr AND a.ds = b.ds AND a.key = 103 +ORDER BY ds, hr; + +EXPLAIN +SELECT `.e.` FROM srcpart; + +EXPLAIN +SELECT `d.*` FROM srcpart; + +EXPLAIN +SELECT `(ds)?+.+` FROM srcpart; + +EXPLAIN +SELECT `(ds|hr)?+.+` FROM srcpart ORDER BY key, value LIMIT 10; + +SELECT `(ds|hr)?+.+` FROM srcpart ORDER BY key, value LIMIT 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/regexp_extract.q b/src/test/hive/ql/src/test/queries/clientpositive/regexp_extract.q new file mode 100644 index 0000000000000..2074eea4396f3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/regexp_extract.q @@ -0,0 +1,33 @@ +EXPLAIN EXTENDED +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value, 1+2, 3+4) + USING 'cat' + CLUSTER BY key +) tmap +SELECT tmap.key, regexp_extract(tmap.value, 'val_(\\d+\\t\\d+)',1) WHERE tmap.key < 100; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value, 1+2, 3+4) + USING 'cat' + CLUSTER BY key +) tmap +SELECT tmap.key, regexp_extract(tmap.value, 'val_(\\d+\\t\\d+)',1) WHERE tmap.key < 100; + +EXPLAIN EXTENDED +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value, 1+2, 3+4) + USING 'cat' + CLUSTER BY key +) tmap +SELECT tmap.key, regexp_extract(tmap.value, 'val_(\\d+\\t\\d+)') WHERE tmap.key < 100; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value, 1+2, 3+4) + USING 'cat' + CLUSTER BY key +) tmap +SELECT tmap.key, regexp_extract(tmap.value, 'val_(\\d+\\t\\d+)') WHERE tmap.key < 100; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/remote_script.q b/src/test/hive/ql/src/test/queries/clientpositive/remote_script.q new file mode 100644 index 0000000000000..926601c647bc3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/remote_script.q @@ -0,0 +1,13 @@ +dfs -put ../data/scripts/newline.py /newline.py; +add file hdfs:///newline.py; +set hive.transform.escape.input=true; + +create table tmp_tmp(key string, value string) stored as rcfile; +insert overwrite table tmp_tmp +SELECT TRANSFORM(key, value) USING +'python newline.py' AS key, value FROM src limit 6; + +select * from tmp_tmp ORDER BY key ASC, value ASC; + +dfs -rmr /newline.py; +drop table tmp_tmp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rename_column.q b/src/test/hive/ql/src/test/queries/clientpositive/rename_column.q new file mode 100644 index 0000000000000..a3f3f30466ced --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rename_column.q @@ -0,0 +1,57 @@ +CREATE TABLE kv_rename_test(a int, b int, c int); +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a a STRING; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a a1 INT; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a1 a2 INT FIRST; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a2 a INT AFTER b; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a a1 INT COMMENT 'test comment1'; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a1 a2 INT COMMENT 'test comment2' FIRST; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE COLUMN a2 a INT AFTER b; +DESCRIBE kv_rename_test; + +DROP TABLE kv_rename_test; +SHOW TABLES; + +-- Using non-default Database +CREATE DATABASE kv_rename_test_db; +USE kv_rename_test_db; + +CREATE TABLE kv_rename_test(a int, b int, c int); +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a a STRING; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a a1 INT; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a1 a2 INT FIRST; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a2 a INT AFTER b; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a a1 INT COMMENT 'test comment1'; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE a1 a2 INT COMMENT 'test comment2' FIRST; +DESCRIBE kv_rename_test; + +ALTER TABLE kv_rename_test CHANGE COLUMN a2 a INT AFTER b; +DESCRIBE kv_rename_test; + +DROP TABLE kv_rename_test; +SHOW TABLES; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rename_external_partition_location.q b/src/test/hive/ql/src/test/queries/clientpositive/rename_external_partition_location.q new file mode 100644 index 0000000000000..faebd55b05f49 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rename_external_partition_location.q @@ -0,0 +1,24 @@ + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/ex_table; + +CREATE EXTERNAL TABLE ex_table ( key INT, value STRING) + PARTITIONED BY (part STRING) + STORED AS textfile + LOCATION 'file:${system:test.tmp.dir}/ex_table'; + +INSERT OVERWRITE TABLE ex_table PARTITION (part='part1') +SELECT key, value FROM src WHERE key < 10; + +SHOW PARTITIONS ex_table; +SELECT * from ex_table where part='part1' ORDER BY key; + +dfs -ls ${system:test.tmp.dir}/ex_table/part=part1; +dfs -cat ${system:test.tmp.dir}/ex_table/part=part1/000000_0; + +ALTER TABLE ex_table PARTITION (part='part1') RENAME TO PARTITION (part='part2'); + +SHOW PARTITIONS ex_table; +SELECT * from ex_table where part='part2' ORDER BY key; + +dfs -ls ${system:test.tmp.dir}/ex_table/part=part1; +dfs -cat ${system:test.tmp.dir}/ex_table/part=part1/000000_0; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rename_partition_location.q b/src/test/hive/ql/src/test/queries/clientpositive/rename_partition_location.q new file mode 100644 index 0000000000000..ee4ff8171f60b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rename_partition_location.q @@ -0,0 +1,20 @@ +-- This test verifies that if the tables location changes, renaming a partition will not change +-- the partition location accordingly + +CREATE TABLE rename_partition_table (key STRING, value STRING) PARTITIONED BY (part STRING) +STORED AS RCFILE +LOCATION 'pfile:${system:test.tmp.dir}/rename_partition_table'; + +INSERT OVERWRITE TABLE rename_partition_table PARTITION (part = '1') SELECT * FROM src; + +ALTER TABLE rename_partition_table SET LOCATION 'file:${system:test.tmp.dir}/rename_partition_table'; + +ALTER TABLE rename_partition_table PARTITION (part = '1') RENAME TO PARTITION (part = '2'); + +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyPartitionIsNotSubdirectoryOfTableHook; + +SELECT count(*) FROM rename_partition_table where part = '2'; + +SET hive.exec.post.hooks=; + +DROP TABLE rename_partition_table; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rename_table_location.q b/src/test/hive/ql/src/test/queries/clientpositive/rename_table_location.q new file mode 100644 index 0000000000000..ad0ce3a933450 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/rename_table_location.q @@ -0,0 +1,17 @@ +-- This test verifies that if the tables location changes, renaming a table will not change +-- the table location scheme + +CREATE TABLE rename_partition_table (key STRING, value STRING) PARTITIONED BY (part STRING) +STORED AS RCFILE +LOCATION 'pfile:${system:test.tmp.dir}/rename_partition_table'; + +INSERT OVERWRITE TABLE rename_partition_table PARTITION (part = '1') SELECT * FROM src; + +ALTER TABLE rename_partition_table SET LOCATION 'file:${system:test.tmp.dir}/rename_partition_table'; + +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyOutputTableLocationSchemeIsFileHook; + +-- If the metastore attempts to change the scheme of the table back to the default pfile, it will get +-- an exception related to the source and destination file systems not matching + +ALTER TABLE rename_partition_table RENAME TO rename_partition_table_renamed; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/repair.q b/src/test/hive/ql/src/test/queries/clientpositive/repair.q new file mode 100644 index 0000000000000..8d04d3e991c59 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/repair.q @@ -0,0 +1,15 @@ +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; + +MSCK TABLE repairtable; + +MSCK REPAIR TABLE repairtable; + +MSCK TABLE repairtable; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reset_conf.q b/src/test/hive/ql/src/test/queries/clientpositive/reset_conf.q new file mode 100644 index 0000000000000..8ddde2364536d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/reset_conf.q @@ -0,0 +1,11 @@ +set hive.skewjoin.key; +set hive.skewjoin.mapjoin.min.split; +set hive.skewjoin.key=300000; +set hive.skewjoin.mapjoin.min.split=256000000; +set hive.skewjoin.key; +set hive.skewjoin.mapjoin.min.split; + +reset; + +set hive.skewjoin.key; +set hive.skewjoin.mapjoin.min.split; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/router_join_ppr.q b/src/test/hive/ql/src/test/queries/clientpositive/router_join_ppr.q new file mode 100644 index 0000000000000..9ec5b7b8f5647 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/router_join_ppr.q @@ -0,0 +1,71 @@ +set hive.optimize.ppd=true; + +EXPLAIN EXTENDED + FROM + src a + RIGHT OUTER JOIN + srcpart b + ON (a.key = b.key AND b.ds = '2008-04-08') + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + + FROM + src a + RIGHT OUTER JOIN + srcpart b + ON (a.key = b.key AND b.ds = '2008-04-08') + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + +EXPLAIN EXTENDED + FROM + srcpart a + RIGHT OUTER JOIN + src b + ON (a.key = b.key AND a.ds = '2008-04-08') + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + + FROM + srcpart a + RIGHT OUTER JOIN + src b + ON (a.key = b.key AND a.ds = '2008-04-08') + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25; + + +EXPLAIN EXTENDED + FROM + src a + RIGHT OUTER JOIN + srcpart b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25 AND b.ds = '2008-04-08'; + + FROM + src a + RIGHT OUTER JOIN + srcpart b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25 AND b.ds = '2008-04-08'; + +EXPLAIN EXTENDED + FROM + srcpart a + RIGHT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25 AND a.ds = '2008-04-08'; + + FROM + srcpart a + RIGHT OUTER JOIN + src b + ON (a.key = b.key) + SELECT a.key, a.value, b.key, b.value + WHERE a.key > 10 AND a.key < 20 AND b.key > 15 AND b.key < 25 AND a.ds = '2008-04-08'; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample1.q b/src/test/hive/ql/src/test/queries/clientpositive/sample1.q new file mode 100644 index 0000000000000..6e86cef2675bd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample1.q @@ -0,0 +1,15 @@ +CREATE TABLE dest1(key INT, value STRING, dt STRING, hr STRING) STORED AS TEXTFILE; + +-- no input pruning, no sample filter +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 1 ON rand()) s +WHERE s.ds='2008-04-08' and s.hr='11'; + +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 1 ON rand()) s +WHERE s.ds='2008-04-08' and s.hr='11'; + +SELECT dest1.* FROM dest1; + +select count(1) from srcbucket; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample10.q b/src/test/hive/ql/src/test/queries/clientpositive/sample10.q new file mode 100644 index 0000000000000..1c6695c5410e8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample10.q @@ -0,0 +1,25 @@ + +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.enforce.bucketing=true; +set hive.exec.reducers.max=4; +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set hive.default.fileformat=RCFILE; +set hive.exec.pre.hooks = org.apache.hadoop.hive.ql.hooks.PreExecutePrinter,org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables,org.apache.hadoop.hive.ql.hooks.UpdateInputAccessTimeHook$PreExec; + +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) + +create table srcpartbucket (key string, value string) partitioned by (ds string, hr string) clustered by (key) into 4 buckets; + +insert overwrite table srcpartbucket partition(ds, hr) select * from srcpart where ds is not null and key < 10; + +explain extended +select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 4 on key) where ds is not null group by ds ORDER BY ds ASC; + +select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 4 on key) where ds is not null group by ds ORDER BY ds ASC; + +select ds, count(1) from srcpartbucket tablesample (bucket 1 out of 2 on key) where ds is not null group by ds ORDER BY ds ASC; + +select * from srcpartbucket where ds is not null ORDER BY key ASC, value ASC, ds ASC, hr ASC; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample2.q b/src/test/hive/ql/src/test/queries/clientpositive/sample2.q new file mode 100644 index 0000000000000..9ca5d26c00895 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample2.q @@ -0,0 +1,12 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +-- input pruning, no sample filter +-- default table sample columns +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2) s; + +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2) s; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample3.q b/src/test/hive/ql/src/test/queries/clientpositive/sample3.q new file mode 100644 index 0000000000000..9dec7172d96af --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample3.q @@ -0,0 +1,8 @@ +-- no input pruning, sample filter +EXPLAIN +SELECT s.key +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 on key) s; + +SELECT s.key +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 on key) s SORT BY key; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample4.q b/src/test/hive/ql/src/test/queries/clientpositive/sample4.q new file mode 100644 index 0000000000000..85a9d92a2d144 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample4.q @@ -0,0 +1,12 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +-- bucket column is the same as table sample +-- No need for sample filter +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s; + +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample5.q b/src/test/hive/ql/src/test/queries/clientpositive/sample5.q new file mode 100644 index 0000000000000..836ad2f15e43e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample5.q @@ -0,0 +1,11 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +-- no input pruning, sample filter +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest1 SELECT s.* -- here's another test +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 on key) s; + +INSERT OVERWRITE TABLE dest1 SELECT s.* -- here's another test +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 on key) s; + +SELECT dest1.* FROM dest1 SORT BY key, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample6.q b/src/test/hive/ql/src/test/queries/clientpositive/sample6.q new file mode 100644 index 0000000000000..ea3511f2435d6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample6.q @@ -0,0 +1,50 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +-- both input pruning and sample filter +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s; + +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s; + +SELECT dest1.* FROM dest1; + +EXPLAIN EXTENDED SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 4 OUT OF 4 on key) s +ORDER BY key, value; +SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 4 OUT OF 4 on key) s +ORDER BY key, value; + +EXPLAIN EXTENDED SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s +ORDER BY key, value; +SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s +ORDER BY key, value; + +EXPLAIN EXTENDED SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 3 on key) s +ORDER BY key, value; +SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 3 on key) s +ORDER BY key, value; + +EXPLAIN EXTENDED SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 2 OUT OF 3 on key) s +ORDER BY key, value; +SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 2 OUT OF 3 on key) s +ORDER BY key, value; + +EXPLAIN EXTENDED SELECT s.* FROM srcbucket2 TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s +ORDER BY key, value; +SELECT s.* FROM srcbucket2 TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s +ORDER BY key, value; + +EXPLAIN EXTENDED SELECT s.* FROM srcbucket2 TABLESAMPLE (BUCKET 2 OUT OF 4 on key) s +ORDER BY key, value; +SELECT s.* FROM srcbucket2 TABLESAMPLE (BUCKET 2 OUT OF 4 on key) s +ORDER BY key, value; + +CREATE TABLE empty_bucket (key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +EXPLAIN EXTENDED SELECT s.* FROM empty_bucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s +ORDER BY key, value; +SELECT s.* FROM empty_bucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s +ORDER BY key, value; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample7.q b/src/test/hive/ql/src/test/queries/clientpositive/sample7.q new file mode 100644 index 0000000000000..18344d79c6a09 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample7.q @@ -0,0 +1,13 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +-- both input pruning and sample filter +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s +WHERE s.key > 100; + +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s +WHERE s.key > 100; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample8.q b/src/test/hive/ql/src/test/queries/clientpositive/sample8.q new file mode 100644 index 0000000000000..90fb1fb1ff724 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample8.q @@ -0,0 +1,15 @@ +-- sampling with join and alias +EXPLAIN EXTENDED +SELECT s.* +FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 1 ON key) s +JOIN srcpart TABLESAMPLE (BUCKET 1 OUT OF 10 ON key) t +WHERE t.key = s.key and t.value = s.value and s.ds='2008-04-08' and s.hr='11' and s.ds='2008-04-08' and s.hr='11' +DISTRIBUTE BY key, value +SORT BY key, value; + +SELECT s.key, s.value +FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 1 ON key) s +JOIN srcpart TABLESAMPLE (BUCKET 1 OUT OF 10 ON key) t +WHERE s.ds='2008-04-08' and s.hr='11' and s.ds='2008-04-08' and s.hr='11' +DISTRIBUTE BY key, value +SORT BY key, value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample9.q b/src/test/hive/ql/src/test/queries/clientpositive/sample9.q new file mode 100644 index 0000000000000..03139ca1fd758 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample9.q @@ -0,0 +1,6 @@ +EXPLAIN EXTENDED +SELECT s.* +FROM (SELECT a.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) a) s; + +SELECT s.* +FROM (SELECT a.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) a) s; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q b/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q new file mode 100644 index 0000000000000..12f2bcd46ec8f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q @@ -0,0 +1,39 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set mapred.max.split.size=300; +set mapred.min.split.size=300; +set mapred.min.split.size.per.node=300; +set mapred.min.split.size.per.rack=300; +set hive.exec.mode.local.auto=true; +set hive.merge.smallfiles.avgsize=1; + +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) + +-- create file inputs +create table sih_i_part (key int, value string) partitioned by (p string); +insert overwrite table sih_i_part partition (p='1') select key, value from src; +insert overwrite table sih_i_part partition (p='2') select key+10000, value from src; +insert overwrite table sih_i_part partition (p='3') select key+20000, value from src; +create table sih_src as select key, value from sih_i_part order by key, value; +create table sih_src2 as select key, value from sih_src order by key, value; + +set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.VerifyIsLocalModeHook; +set mapreduce.framework.name=yarn; +set mapreduce.jobtracker.address=localhost:58; +set hive.sample.seednumber=7; + +-- Relaxing hive.exec.mode.local.auto.input.files.max=1. +-- Hadoop20 will not generate more splits than there are files (one). +-- Hadoop23 generate splits correctly (four), hence the max needs to be adjusted to ensure running in local mode. +-- Default value is hive.exec.mode.local.auto.input.files.max=4 which produces expected behavior on Hadoop23. +-- hive.sample.seednumber is required because Hadoop23 generates multiple splits and tablesample is non-repeatable without it. + +-- sample split, running locally limited by num tasks +select count(1) from sih_src tablesample(1 percent); + +-- sample two tables +select count(1) from sih_src tablesample(1 percent) a join sih_src2 tablesample(1 percent) b on a.key = b.key; + +set hive.exec.mode.local.auto.inputbytes.max=1000; + +-- sample split, running locally limited by max bytes +select count(1) from sih_src tablesample(1 percent); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q b/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q new file mode 100644 index 0000000000000..484e1fa617d8a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q @@ -0,0 +1,41 @@ +USE default; + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set mapred.max.split.size=300; +set mapred.min.split.size=300; +set mapred.min.split.size.per.node=300; +set mapred.min.split.size.per.rack=300; +set hive.exec.mode.local.auto=true; +set hive.merge.smallfiles.avgsize=1; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) +-- This test sets mapred.max.split.size=300 and hive.merge.smallfiles.avgsize=1 +-- in an attempt to force the generation of multiple splits and multiple output files. +-- However, Hadoop 0.20 is incapable of generating splits smaller than the block size +-- when using CombineFileInputFormat, so only one split is generated. This has a +-- significant impact on the results of the TABLESAMPLE(x PERCENT). This issue was +-- fixed in MAPREDUCE-2046 which is included in 0.22. + +-- create file inputs +create table sih_i_part (key int, value string) partitioned by (p string); +insert overwrite table sih_i_part partition (p='1') select key, value from src; +insert overwrite table sih_i_part partition (p='2') select key+10000, value from src; +insert overwrite table sih_i_part partition (p='3') select key+20000, value from src; +create table sih_src as select key, value from sih_i_part order by key, value; +create table sih_src2 as select key, value from sih_src order by key, value; + +set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.VerifyIsLocalModeHook ; +set mapred.job.tracker=localhost:58; +set hive.exec.mode.local.auto.input.files.max=1; + +-- Sample split, running locally limited by num tasks +select count(1) from sih_src tablesample(1 percent); + +-- sample two tables +select count(1) from sih_src tablesample(1 percent)a join sih_src2 tablesample(1 percent)b on a.key = b.key; + +set hive.exec.mode.local.auto.inputbytes.max=1000; +set hive.exec.mode.local.auto.input.files.max=4; + +-- sample split, running locally limited by max bytes +select count(1) from sih_src tablesample(1 percent); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority.q b/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority.q new file mode 100644 index 0000000000000..c0e18c5688b3c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority.q @@ -0,0 +1,15 @@ +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) 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; +insert overwrite directory "/tmp/test" select key from src where (key = 20) order by key; +alter table dynPart add partition (value='0') location 'file:///tmp/test'; +alter table dynPart add partition (value='1') location 'hdfs:///tmp/test'; +select count(*) from dynPart; +select key from dynPart; +select key from src where (key = 10) order by key; +select key from src where (key = 20) order by key; + +dfs -rmr file:///tmp/test; +dfs -rmr hdfs:///tmp/test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority2.q b/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority2.q new file mode 100644 index 0000000000000..ecd4d13d0e23b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority2.q @@ -0,0 +1,12 @@ +dfs -mkdir file:///tmp/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; +insert overwrite directory "/tmp/test" select key from src where (key = 20) order by key; +alter table dynPart add partition (value='0', value2='clusterA') location 'file:///tmp/test'; +alter table dynPart add partition (value='0', value2='clusterB') location 'hdfs:///tmp/test'; +select value2, key from dynPart where value='0'; + +dfs -rmr file:///tmp/test; +dfs -rmr hdfs:///tmp/test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/script_env_var1.q b/src/test/hive/ql/src/test/queries/clientpositive/script_env_var1.q new file mode 100644 index 0000000000000..82268f4dfb595 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/script_env_var1.q @@ -0,0 +1,5 @@ +-- Verifies that script operator ID environment variables have unique values +-- in each instance of the script operator. +SELECT count(1) FROM +( SELECT TRANSFORM('echo $HIVE_SCRIPT_OPERATOR_ID') USING 'bash' AS key FROM src LIMIT 1 UNION ALL + SELECT TRANSFORM('echo $HIVE_SCRIPT_OPERATOR_ID') USING 'bash' AS key FROM src LIMIT 1 ) a GROUP BY key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/script_env_var2.q b/src/test/hive/ql/src/test/queries/clientpositive/script_env_var2.q new file mode 100644 index 0000000000000..7957fe518d55d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/script_env_var2.q @@ -0,0 +1,5 @@ +set hive.script.operator.id.env.var = MY_ID; +-- Same test as script_env_var1, but test setting the variable name +SELECT count(1) FROM +( SELECT TRANSFORM('echo $MY_ID') USING 'bash' AS key FROM src LIMIT 1 UNION ALL + SELECT TRANSFORM('echo $MY_ID') USING 'bash' AS key FROM src LIMIT 1 ) a GROUP BY key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/script_pipe.q b/src/test/hive/ql/src/test/queries/clientpositive/script_pipe.q new file mode 100644 index 0000000000000..ae2fd2ed8f2dc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/script_pipe.q @@ -0,0 +1,8 @@ +set hive.exec.script.allow.partial.consumption = true; +-- Tests exception in ScriptOperator.close() by passing to the operator a small amount of data +EXPLAIN SELECT TRANSFORM(*) USING 'true' AS a, b, c FROM (SELECT * FROM src LIMIT 1) tmp; +-- Tests exception in ScriptOperator.processOp() by passing extra data needed to fill pipe buffer +EXPLAIN SELECT TRANSFORM(key, value, key, value, key, value, key, value, key, value, key, value) USING 'head -n 1' as a,b,c,d FROM src; + +SELECT TRANSFORM(*) USING 'true' AS a, b, c FROM (SELECT * FROM src LIMIT 1) tmp; +SELECT TRANSFORM(key, value, key, value, key, value, key, value, key, value, key, value) USING 'head -n 1' as a,b,c,d FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/scriptfile1.q b/src/test/hive/ql/src/test/queries/clientpositive/scriptfile1.q new file mode 100644 index 0000000000000..4f65016f3081c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/scriptfile1.q @@ -0,0 +1,14 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +CREATE TABLE dest1(key INT, value STRING); + +ADD FILE src/test/scripts/testgrep; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'testgrep' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/select_as_omitted.q b/src/test/hive/ql/src/test/queries/clientpositive/select_as_omitted.q new file mode 100644 index 0000000000000..f0fe03f180c08 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/select_as_omitted.q @@ -0,0 +1,12 @@ +EXPLAIn +SELECT a, b FROM ( + SELECT key a, value b + FROM src +) src1 +ORDER BY a LIMIT 1; + +SELECT a, b FROM ( + SELECT key a, value b + FROM src +) src1 +ORDER BY a LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/select_transform_hint.q b/src/test/hive/ql/src/test/queries/clientpositive/select_transform_hint.q new file mode 100644 index 0000000000000..33959ed5b46a6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/select_transform_hint.q @@ -0,0 +1,24 @@ +EXPLAIN +SELECT /*+MAPJOIN(a)*/ +TRANSFORM(a.key, a.value) USING 'cat' AS (tkey, tvalue) +FROM src a join src b +on a.key = b.key; + + +SELECT /*+MAPJOIN(a)*/ +TRANSFORM(a.key, a.value) USING 'cat' AS (tkey, tvalue) +FROM src a join src b +on a.key = b.key; + + +EXPLAIN +SELECT /*+STREAMTABLE(a)*/ +TRANSFORM(a.key, a.value) USING 'cat' AS (tkey, tvalue) +FROM src a join src b +on a.key = b.key; + + +SELECT /*+STREAMTABLE(a)*/ +TRANSFORM(a.key, a.value) USING 'cat' AS (tkey, tvalue) +FROM src a join src b +on a.key = b.key; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_and.q b/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_and.q new file mode 100644 index 0000000000000..275c774d1d4eb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_and.q @@ -0,0 +1,15 @@ +CREATE TABLE npe_test (key STRING, value STRING) PARTITIONED BY (ds STRING); + +INSERT OVERWRITE TABLE npe_test PARTITION(ds='2012-12-11') +SELECT src.key, src.value FROM src WHERE key < '200'; + +INSERT OVERWRITE TABLE npe_test PARTITION(ds='2012-12-12') +SELECT src.key, src.value FROM src WHERE key > '200'; + +SELECT count(*) FROM npe_test; + +EXPLAIN SELECT * FROM npe_test WHERE ds > 2012-11-31 AND ds < 2012-12-15; + +SELECT count(*) FROM npe_test WHERE ds > 2012-11-31 AND ds < 2012-12-15; + +DROP TABLE npe_test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_not.q b/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_not.q new file mode 100644 index 0000000000000..93d992e69844f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_not.q @@ -0,0 +1,15 @@ +CREATE TABLE npe_test (key STRING, value STRING) PARTITIONED BY (ds STRING); + +INSERT OVERWRITE TABLE npe_test PARTITION(ds='2012-12-11') +SELECT src.key, src.value FROM src WHERE key < '200'; + +INSERT OVERWRITE TABLE npe_test PARTITION(ds='2012-12-12') +SELECT src.key, src.value FROM src WHERE key > '200'; + +SELECT count(*) FROM npe_test; + +EXPLAIN SELECT * FROM npe_test WHERE NOT ds < 2012-11-31; + +SELECT count(*) FROM npe_test WHERE NOT ds < 2012-11-31; + +DROP TABLE npe_test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_or.q b/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_or.q new file mode 100644 index 0000000000000..5dd0776792f5b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_or.q @@ -0,0 +1,15 @@ +CREATE TABLE npe_test (key STRING, value STRING) PARTITIONED BY (ds STRING); + +INSERT OVERWRITE TABLE npe_test PARTITION(ds='2012-12-11') +SELECT src.key, src.value FROM src WHERE key < '200'; + +INSERT OVERWRITE TABLE npe_test PARTITION(ds='2012-12-12') +SELECT src.key, src.value FROM src WHERE key > '200'; + +SELECT count(*) FROM npe_test; + +EXPLAIN SELECT * FROM npe_test WHERE ds > 2012-11-31 OR ds < 2012-12-15; + +SELECT count(*) FROM npe_test WHERE ds > 2012-11-31 OR ds < 2012-12-15; + +DROP TABLE npe_test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/semicolon.q b/src/test/hive/ql/src/test/queries/clientpositive/semicolon.q new file mode 100644 index 0000000000000..fef2687308d02 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/semicolon.q @@ -0,0 +1,4 @@ +-- comment +-- comment; +-- comment +SELECT COUNT(1) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/semijoin.q b/src/test/hive/ql/src/test/queries/clientpositive/semijoin.q new file mode 100644 index 0000000000000..142a51ab52e8f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/semijoin.q @@ -0,0 +1,81 @@ + + + + + +create table t1 as select cast(key as int) key, value from src where key <= 10; + +select * from t1 sort by key; + +create table t2 as select cast(2*key as int) key, value from t1; + +select * from t2 sort by key; + +create table t3 as select * from (select * from t1 union all select * from t2) b; +select * from t3 sort by key, value; + +create table t4 (key int, value string); +select * from t4; + +explain select * from t1 a left semi join t2 b on a.key=b.key sort by a.key, a.value; +select * from t1 a left semi join t2 b on a.key=b.key sort by a.key, a.value; + +explain select * from t2 a left semi join t1 b on b.key=a.key sort by a.key, a.value; +select * from t2 a left semi join t1 b on b.key=a.key sort by a.key, a.value; + +explain select * from t1 a left semi join t4 b on b.key=a.key sort by a.key, a.value; +select * from t1 a left semi join t4 b on b.key=a.key sort by a.key, a.value; + +explain select a.value from t1 a left semi join t3 b on (b.key = a.key and b.key < '15') sort by a.value; +select a.value from t1 a left semi join t3 b on (b.key = a.key and b.key < '15') sort by a.value; + +explain select * from t1 a left semi join t2 b on a.key = b.key and b.value < "val_10" sort by a.key, a.value; +select * from t1 a left semi join t2 b on a.key = b.key and b.value < "val_10" sort by a.key, a.value; + +explain select a.value from t1 a left semi join (select key from t3 where key > 5) b on a.key = b.key sort by a.value; +select a.value from t1 a left semi join (select key from t3 where key > 5) b on a.key = b.key sort by a.value; + +explain select a.value from t1 a left semi join (select key , value from t2 where key > 5) b on a.key = b.key and b.value <= 'val_20' sort by a.value ; +select a.value from t1 a left semi join (select key , value from t2 where key > 5) b on a.key = b.key and b.value <= 'val_20' sort by a.value ; + +explain select * from t2 a left semi join (select key , value from t1 where key > 2) b on a.key = b.key sort by a.key, a.value; +select * from t2 a left semi join (select key , value from t1 where key > 2) b on a.key = b.key sort by a.key, a.value; + +explain select /*+ mapjoin(b) */ a.key from t3 a left semi join t1 b on a.key = b.key sort by a.key; +select /*+ mapjoin(b) */ a.key from t3 a left semi join t1 b on a.key = b.key sort by a.key; + +explain select * from t1 a left semi join t2 b on a.key = 2*b.key sort by a.key, a.value; +select * from t1 a left semi join t2 b on a.key = 2*b.key sort by a.key, a.value; + +explain select * from t1 a join t2 b on a.key = b.key left semi join t3 c on b.key = c.key sort by a.key, a.value; +select * from t1 a join t2 b on a.key = b.key left semi join t3 c on b.key = c.key sort by a.key, a.value; + +explain select * from t3 a left semi join t1 b on a.key = b.key and a.value=b.value sort by a.key, a.value; +select * from t3 a left semi join t1 b on a.key = b.key and a.value=b.value sort by a.key, a.value; + +explain select /*+ mapjoin(b, c) */ a.key from t3 a left semi join t1 b on a.key = b.key left semi join t2 c on a.key = c.key sort by a.key; +select /*+ mapjoin(b, c) */ a.key from t3 a left semi join t1 b on a.key = b.key left semi join t2 c on a.key = c.key sort by a.key; + +explain select a.key from t3 a left outer join t1 b on a.key = b.key left semi join t2 c on b.key = c.key sort by a.key; +select a.key from t3 a left outer join t1 b on a.key = b.key left semi join t2 c on b.key = c.key sort by a.key; + +explain select a.key from t1 a right outer join t3 b on a.key = b.key left semi join t2 c on b.key = c.key sort by a.key; +select a.key from t1 a right outer join t3 b on a.key = b.key left semi join t2 c on b.key = c.key sort by a.key; + +explain select a.key from t1 a full outer join t3 b on a.key = b.key left semi join t2 c on b.key = c.key sort by a.key; +select a.key from t1 a full outer join t3 b on a.key = b.key left semi join t2 c on b.key = c.key sort by a.key; + +explain select a.key from t3 a left semi join t2 b on a.key = b.key left outer join t1 c on a.key = c.key sort by a.key; +select a.key from t3 a left semi join t2 b on a.key = b.key left outer join t1 c on a.key = c.key sort by a.key; + +explain select a.key from t3 a left semi join t2 b on a.key = b.key right outer join t1 c on a.key = c.key sort by a.key; +select a.key from t3 a left semi join t2 b on a.key = b.key right outer join t1 c on a.key = c.key sort by a.key; + +explain select a.key from t3 a left semi join t1 b on a.key = b.key full outer join t2 c on a.key = c.key sort by a.key; +select a.key from t3 a left semi join t1 b on a.key = b.key full outer join t2 c on a.key = c.key sort by a.key; + +explain select a.key from t3 a left semi join t2 b on a.key = b.key left outer join t1 c on a.value = c.value sort by a.key; +select a.key from t3 a left semi join t2 b on a.key = b.key left outer join t1 c on a.value = c.value sort by a.key; + +explain select a.key from t3 a left semi join t2 b on a.value = b.value where a.key > 100; +select a.key from t3 a left semi join t2 b on a.value = b.value where a.key > 100; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/serde_regex.q b/src/test/hive/ql/src/test/queries/clientpositive/serde_regex.q new file mode 100644 index 0000000000000..2a287bd877759 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/serde_regex.q @@ -0,0 +1,66 @@ +EXPLAIN +CREATE TABLE serde_regex( + host STRING, + identity STRING, + user STRING, + time STRING, + request STRING, + status STRING, + size INT, + referer STRING, + agent STRING) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' +WITH SERDEPROPERTIES ( + "input.regex" = "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?" +) +STORED AS TEXTFILE; + +CREATE TABLE serde_regex( + host STRING, + identity STRING, + user STRING, + time STRING, + request STRING, + status STRING, + size INT, + referer STRING, + agent STRING) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' +WITH SERDEPROPERTIES ( + "input.regex" = "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?" +) +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; + +SELECT * FROM serde_regex ORDER BY time; + +SELECT host, size, status, time from serde_regex ORDER BY time; + +DROP TABLE serde_regex; + +EXPLAIN +CREATE TABLE serde_regex1( + key decimal, + value int) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' +WITH SERDEPROPERTIES ( + "input.regex" = "([^ ]*) ([^ ]*)" +) +STORED AS TEXTFILE; + +CREATE TABLE serde_regex1( + key decimal, + value int) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' +WITH SERDEPROPERTIES ( + "input.regex" = "([^ ]*) ([^ ]*)" +) +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH "../data/files/kv7.txt" INTO TABLE serde_regex1; + +SELECT key, value FROM serde_regex1 ORDER BY key, value; + +DROP TABLE serde_regex1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/serde_reported_schema.q b/src/test/hive/ql/src/test/queries/clientpositive/serde_reported_schema.q new file mode 100644 index 0000000000000..01ec864bc925f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/serde_reported_schema.q @@ -0,0 +1,9 @@ +create table int_string + partitioned by (b string) + row format serde "org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer" + with serdeproperties ( + "serialization.class"="org.apache.hadoop.hive.serde2.thrift.test.IntString", + "serialization.format"="org.apache.thrift.protocol.TBinaryProtocol"); +describe extended int_string; +alter table int_string add partition (b='part1'); +describe extended int_string partition (b='part1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/serde_user_properties.q b/src/test/hive/ql/src/test/queries/clientpositive/serde_user_properties.q new file mode 100644 index 0000000000000..4d523e1bf7167 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/serde_user_properties.q @@ -0,0 +1,9 @@ +-- HIVE-2906 Table properties in SQL + +explain extended select key from src; +explain extended select a.key from src a; +explain extended select a.key from src tablesample(1 percent) a; +explain extended select key from src ('user.defined.key'='some.value'); +explain extended select key from src ('user.defined.key'='some.value') tablesample(1 percent); +explain extended select a.key from src ('user.defined.key'='some.value') a; +explain extended select a.key from src ('user.defined.key'='some.value') tablesample(1 percent) a; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/set_processor_namespaces.q b/src/test/hive/ql/src/test/queries/clientpositive/set_processor_namespaces.q new file mode 100644 index 0000000000000..7e3d1f4d8aa41 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/set_processor_namespaces.q @@ -0,0 +1,31 @@ +set zzz=5; +set zzz; + +set system:xxx=5; +set system:xxx; + +set system:yyy=${system:xxx}; +set system:yyy; + +set go=${hiveconf:zzz}; +set go; + +set hive.variable.substitute=false; +set raw=${hiveconf:zzz}; +set raw; + +set hive.variable.substitute=true; + +EXPLAIN SELECT * FROM src where key=${hiveconf:zzz}; +SELECT * FROM src where key=${hiveconf:zzz}; + +set a=1; +set b=a; +set c=${hiveconf:${hiveconf:b}}; +set c; + +set jar=${system:build.ivy.lib.dir}/default/derby-${system:derby.version}.jar; + +add file ${hiveconf:jar}; +delete file ${hiveconf:jar}; +list file; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/set_variable_sub.q b/src/test/hive/ql/src/test/queries/clientpositive/set_variable_sub.q new file mode 100644 index 0000000000000..7bf1e60a5d3f9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/set_variable_sub.q @@ -0,0 +1,15 @@ +set hivevar:key1=value1; + +EXPLAIN SELECT * FROM src where key="${key1}"; +EXPLAIN SELECT * FROM src where key="${hivevar:key1}"; + +set hivevar:a=1; +set hivevar:b=a; +set hivevar:c=${hivevar:${hivevar:b}}; +EXPLAIN SELECT * FROM src where key="${hivevar:c}"; + +set hivevar:a; +set hivevar:b; +set hivevar:c; +set hivevar:key1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_columns.q b/src/test/hive/ql/src/test/queries/clientpositive/show_columns.q new file mode 100644 index 0000000000000..58a3a2c29e2cf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_columns.q @@ -0,0 +1,26 @@ +CREATE TABLE shcol_test(KEY STRING, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE; + +EXPLAIN +SHOW COLUMNS from shcol_test; + +SHOW COLUMNS from shcol_test; + +-- SHOW COLUMNS +CREATE DATABASE test_db; +USE test_db; +CREATE TABLE foo(a INT); + +-- SHOW COLUMNS basic syntax tests +USE test_db; +SHOW COLUMNS from foo; +SHOW COLUMNS in foo; + +-- SHOW COLUMNS from a database with a name that requires escaping +CREATE DATABASE `database`; +USE `database`; +CREATE TABLE foo(a INT); +SHOW COLUMNS from foo; + +use default; +SHOW COLUMNS from test_db.foo; +SHOW COLUMNS from foo from test_db; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_alter.q b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_alter.q new file mode 100644 index 0000000000000..1158a69ba338b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_alter.q @@ -0,0 +1,23 @@ +-- Test SHOW CREATE TABLE on an external, clustered and sorted table. Then test the query again after ALTERs. + +CREATE EXTERNAL TABLE tmp_showcrt1 (key smallint, value float) +CLUSTERED BY (key) SORTED BY (value DESC) INTO 5 BUCKETS; +SHOW CREATE TABLE tmp_showcrt1; + +-- Add a comment to the table, change the EXTERNAL property, and test SHOW CREATE TABLE on the change. +ALTER TABLE tmp_showcrt1 SET TBLPROPERTIES ('comment'='temporary table', 'EXTERNAL'='FALSE'); +SHOW CREATE TABLE tmp_showcrt1; + +-- Alter the table comment, change the EXTERNAL property back and test SHOW CREATE TABLE on the change. +ALTER TABLE tmp_showcrt1 SET TBLPROPERTIES ('comment'='changed comment', 'EXTERNAL'='TRUE'); +SHOW CREATE TABLE tmp_showcrt1; + +-- Change the 'SORTBUCKETCOLSPREFIX' property and test SHOW CREATE TABLE. The output should not change. +ALTER TABLE tmp_showcrt1 SET TBLPROPERTIES ('SORTBUCKETCOLSPREFIX'='FALSE'); +SHOW CREATE TABLE tmp_showcrt1; + +-- Alter the storage handler of the table, and test SHOW CREATE TABLE. +ALTER TABLE tmp_showcrt1 SET TBLPROPERTIES ('storage_handler'='org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'); +SHOW CREATE TABLE tmp_showcrt1; +DROP TABLE tmp_showcrt1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_db_table.q b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_db_table.q new file mode 100644 index 0000000000000..1937acb547cdc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_db_table.q @@ -0,0 +1,10 @@ +-- Test SHOW CREATE TABLE on a table name of format "db.table". + +CREATE DATABASE tmp_feng comment 'for show create table test'; +SHOW DATABASES; +CREATE TABLE tmp_feng.tmp_showcrt (key string, value int); +USE default; +SHOW CREATE TABLE tmp_feng.tmp_showcrt; +DROP TABLE tmp_feng.tmp_showcrt; +DROP DATABASE tmp_feng; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_delimited.q b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_delimited.q new file mode 100644 index 0000000000000..77229641f121e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_delimited.q @@ -0,0 +1,9 @@ +-- Test SHOW CREATE TABLE on a table with delimiters, stored format, and location. + +CREATE TABLE tmp_showcrt1 (key int, value string, newvalue bigint) +ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' COLLECTION ITEMS TERMINATED BY '|' MAP KEYS TERMINATED BY '\045' LINES TERMINATED BY '\n' +STORED AS textfile +LOCATION 'file:${system:test.tmp.dir}/tmp_showcrt1'; +SHOW CREATE TABLE tmp_showcrt1; +DROP TABLE tmp_showcrt1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_partitioned.q b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_partitioned.q new file mode 100644 index 0000000000000..3b691544e5313 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_partitioned.q @@ -0,0 +1,8 @@ +-- Test SHOW CREATE TABLE on a table with partitions and column comments. + +CREATE EXTERNAL TABLE tmp_showcrt1 (key string, newvalue boolean COMMENT 'a new value') +COMMENT 'temporary table' +PARTITIONED BY (value bigint COMMENT 'some value'); +SHOW CREATE TABLE tmp_showcrt1; +DROP TABLE tmp_showcrt1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_serde.q b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_serde.q new file mode 100644 index 0000000000000..a3eb5a8f7d246 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_serde.q @@ -0,0 +1,19 @@ +-- Test SHOW CREATE TABLE on a table with serde. + +-- without a storage handler +CREATE TABLE tmp_showcrt1 (key int, value string, newvalue bigint) +COMMENT 'temporary table' +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'; +SHOW CREATE TABLE tmp_showcrt1; +DROP TABLE tmp_showcrt1; + +-- with a storage handler and serde properties +CREATE EXTERNAL TABLE tmp_showcrt1 (key string, value boolean) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' +STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler' +WITH SERDEPROPERTIES ('field.delim'=',', 'serialization.format'='$'); +SHOW CREATE TABLE tmp_showcrt1; +DROP TABLE tmp_showcrt1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_view.q b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_view.q new file mode 100644 index 0000000000000..a378456494ab4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_view.q @@ -0,0 +1,6 @@ +-- Test SHOW CREATE TABLE on a view name. + +CREATE VIEW tmp_copy_src AS SELECT * FROM src; +SHOW CREATE TABLE tmp_copy_src; +DROP VIEW tmp_copy_src; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_describe_func_quotes.q b/src/test/hive/ql/src/test/queries/clientpositive/show_describe_func_quotes.q new file mode 100644 index 0000000000000..de1451d633834 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_describe_func_quotes.q @@ -0,0 +1,7 @@ +SHOW FUNCTIONS 'concat'; + +SHOW FUNCTIONS concat; + +DESCRIBE FUNCTION 'concat'; + +DESCRIBE FUNCTION concat; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_functions.q b/src/test/hive/ql/src/test/queries/clientpositive/show_functions.q new file mode 100644 index 0000000000000..c09f50a10c0d2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_functions.q @@ -0,0 +1,11 @@ +SHOW FUNCTIONS; + +SHOW FUNCTIONS '^c.*'; + +SHOW FUNCTIONS '.*e$'; + +SHOW FUNCTIONS 'log.*'; + +SHOW FUNCTIONS '.*date.*'; + +SHOW FUNCTIONS '***'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q b/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q new file mode 100644 index 0000000000000..5fcdf97e2db47 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q @@ -0,0 +1,27 @@ +DROP TABLE show_idx_empty; +DROP TABLE show_idx_full; + +CREATE TABLE show_idx_empty(KEY STRING, VALUE STRING); +CREATE TABLE show_idx_full(KEY STRING, VALUE1 STRING, VALUE2 STRING); + +CREATE INDEX idx_1 ON TABLE show_idx_full(KEY) AS "COMPACT" WITH DEFERRED REBUILD; +CREATE INDEX idx_2 ON TABLE show_idx_full(VALUE1) AS "COMPACT" WITH DEFERRED REBUILD; + +CREATE INDEX idx_comment ON TABLE show_idx_full(VALUE2) AS "COMPACT" WITH DEFERRED REBUILD COMMENT "index comment"; +CREATE INDEX idx_compound ON TABLE show_idx_full(KEY, VALUE1) AS "COMPACT" WITH DEFERRED REBUILD; + +ALTER INDEX idx_1 ON show_idx_full REBUILD; +ALTER INDEX idx_2 ON show_idx_full REBUILD; +ALTER INDEX idx_comment ON show_idx_full REBUILD; +ALTER INDEX idx_compound ON show_idx_full REBUILD; + +EXPLAIN SHOW INDEXES ON show_idx_full; +SHOW INDEXES ON show_idx_full; + +EXPLAIN SHOW INDEXES ON show_idx_empty; +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 diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_syntax.q b/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_syntax.q new file mode 100644 index 0000000000000..ab588937e179b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_syntax.q @@ -0,0 +1,23 @@ +DROP TABLE show_idx_t1; + +CREATE TABLE show_idx_t1(KEY STRING, VALUE STRING); + +CREATE INDEX idx_t1 ON TABLE show_idx_t1(KEY) AS "COMPACT" WITH DEFERRED REBUILD; +ALTER INDEX idx_t1 ON show_idx_t1 REBUILD; + +EXPLAIN +SHOW INDEX ON show_idx_t1; + +SHOW INDEX ON show_idx_t1; + +EXPLAIN +SHOW INDEXES ON show_idx_t1; + +SHOW INDEXES ON show_idx_t1; + +EXPLAIN +SHOW FORMATTED INDEXES ON show_idx_t1; + +SHOW FORMATTED INDEXES ON show_idx_t1; + +DROP TABLE show_idx_t1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_partitions.q b/src/test/hive/ql/src/test/queries/clientpositive/show_partitions.q new file mode 100644 index 0000000000000..7fa7b828bd72e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_partitions.q @@ -0,0 +1,4 @@ +SHOW PARTITIONS 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 diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_tables.q b/src/test/hive/ql/src/test/queries/clientpositive/show_tables.q new file mode 100644 index 0000000000000..f33325dbb9e41 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_tables.q @@ -0,0 +1,35 @@ +CREATE TABLE shtb_test1(KEY STRING, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE; +CREATE TABLE shtb_test2(KEY STRING, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE; + +EXPLAIN +SHOW TABLES 'shtb_*'; + +SHOW TABLES 'shtb_*'; + +EXPLAIN +SHOW TABLES LIKE 'shtb_test1|shtb_test2'; + +SHOW TABLES LIKE 'shtb_test1|shtb_test2'; + +-- SHOW TABLES FROM/IN database +CREATE DATABASE test_db; +USE test_db; +CREATE TABLE foo(a INT); +CREATE TABLE bar(a INT); +CREATE TABLE baz(a INT); + +-- SHOW TABLES basic syntax tests +USE default; +SHOW TABLES FROM test_db; +SHOW TABLES FROM default; +SHOW TABLES IN test_db; +SHOW TABLES IN default; +SHOW TABLES IN test_db "test*"; +SHOW TABLES IN test_db LIKE "nomatch"; + +-- SHOW TABLES from a database with a name that requires escaping +CREATE DATABASE `database`; +USE `database`; +CREATE TABLE foo(a INT); +USE default; +SHOW TABLES FROM `database` LIKE "foo"; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_tablestatus.q b/src/test/hive/ql/src/test/queries/clientpositive/show_tablestatus.q new file mode 100644 index 0000000000000..9184d6da897c0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_tablestatus.q @@ -0,0 +1,20 @@ +EXPLAIN +SHOW TABLE EXTENDED IN default LIKE `src`; + +SHOW TABLE EXTENDED IN default LIKE `src`; + +SHOW TABLE EXTENDED from default LIKE `src`; + +SHOW TABLE EXTENDED LIKE `src`; + +SHOW TABLE EXTENDED LIKE `src.?`; + +SHOW TABLE EXTENDED from default LIKE `src.?`; + +SHOW TABLE EXTENDED LIKE `^s.*`; + +SHOW TABLE EXTENDED from default LIKE `^s.*`; + +SHOW TABLE EXTENDED LIKE `srcpart` PARTITION(ds='2008-04-08', hr=11); + +SHOW TABLE EXTENDED from default LIKE src; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_tblproperties.q b/src/test/hive/ql/src/test/queries/clientpositive/show_tblproperties.q new file mode 100644 index 0000000000000..c755108de22f1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/show_tblproperties.q @@ -0,0 +1,11 @@ + +create table tmpfoo (a String); +show tblproperties tmpfoo("bar"); + +alter table tmpfoo set tblproperties ("bar" = "bar value"); +alter table tmpfoo set tblproperties ("tmp" = "true"); + +show tblproperties tmpfoo; +show tblproperties tmpfoo("bar"); + +drop table tmpfoo; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/showparts.q b/src/test/hive/ql/src/test/queries/clientpositive/showparts.q new file mode 100644 index 0000000000000..dbee3efc24f7f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/showparts.q @@ -0,0 +1,4 @@ +EXPLAIN +SHOW PARTITIONS srcpart; + +SHOW PARTITIONS srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoin.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoin.q new file mode 100644 index 0000000000000..ad917beeef9ce --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoin.q @@ -0,0 +1,137 @@ +set hive.optimize.skewjoin = true; +set hive.skewjoin.key = 2; + + + + + + + +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; +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; + + +EXPLAIN +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value; + +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value; + +SELECT sum(hash(key)), sum(hash(value)) FROM dest_j1; + + +EXPLAIN +SELECT /*+ STREAMTABLE(a) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key; + +SELECT /*+ STREAMTABLE(a) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key; + +EXPLAIN +SELECT /*+ STREAMTABLE(a,c) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key; + +SELECT /*+ STREAMTABLE(a,c) */ * +FROM T1 a JOIN T2 b ON a.key = b.key + JOIN T3 c ON b.key = c.key + JOIN T4 d ON c.key = d.key; + + +EXPLAIN FROM T1 a JOIN src c ON c.key+1=a.key SELECT /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)); +FROM T1 a JOIN src c ON c.key+1=a.key SELECT /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)); + +EXPLAIN FROM +(SELECT src.* FROM src) x +JOIN +(SELECT src.* FROM src) Y +ON (x.key = Y.key) +SELECT sum(hash(Y.key)), sum(hash(Y.value)); + +FROM +(SELECT src.* FROM src) x +JOIN +(SELECT src.* FROM src) Y +ON (x.key = Y.key) +SELECT sum(hash(Y.key)), sum(hash(Y.value)); + + +EXPLAIN FROM +(SELECT src.* FROM src) x +JOIN +(SELECT src.* FROM src) Y +ON (x.key = Y.key and substring(x.value, 5)=substring(y.value, 5)+1) +SELECT sum(hash(Y.key)), sum(hash(Y.value)); + +FROM +(SELECT src.* FROM src) x +JOIN +(SELECT src.* FROM src) Y +ON (x.key = Y.key and substring(x.value, 5)=substring(y.value, 5)+1) +SELECT sum(hash(Y.key)), sum(hash(Y.value)); + + +EXPLAIN +SELECT sum(hash(src1.c1)), sum(hash(src2.c4)) +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100 +JOIN +(SELECT src.key as c5, src.value as c6 from src) src3 +ON src1.c1 = src3.c5 AND src3.c5 < 80; + +SELECT sum(hash(src1.c1)), sum(hash(src2.c4)) +FROM +(SELECT src.key as c1, src.value as c2 from src) src1 +JOIN +(SELECT src.key as c3, src.value as c4 from src) src2 +ON src1.c1 = src2.c3 AND src1.c1 < 100 +JOIN +(SELECT src.key as c5, src.value as c6 from src) src3 +ON src1.c1 = src3.c5 AND src3.c5 < 80; + +EXPLAIN +SELECT /*+ mapjoin(v)*/ sum(hash(k.key)), sum(hash(v.val)) FROM T1 k LEFT OUTER JOIN T1 v ON k.key+1=v.key; +SELECT /*+ mapjoin(v)*/ sum(hash(k.key)), sum(hash(v.val)) FROM T1 k LEFT OUTER JOIN T1 v ON k.key+1=v.key; + +select /*+ mapjoin(k)*/ sum(hash(k.key)), sum(hash(v.val)) from T1 k join T1 v on k.key=v.val; + +select /*+ mapjoin(k)*/ sum(hash(k.key)), sum(hash(v.val)) from T1 k join T1 v on k.key=v.key; + +select sum(hash(k.key)), sum(hash(v.val)) from T1 k join T1 v on k.key=v.key; + +select count(1) from T1 a join T1 b on a.key = b.key; + +FROM T1 a LEFT OUTER JOIN T2 c ON c.key+1=a.key SELECT sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)); + +FROM T1 a RIGHT OUTER JOIN T2 c ON c.key+1=a.key SELECT /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)); + +FROM T1 a FULL OUTER JOIN T2 c ON c.key+1=a.key SELECT /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key)); + +SELECT sum(hash(src1.key)), sum(hash(src1.val)), sum(hash(src2.key)) FROM T1 src1 LEFT OUTER JOIN T2 src2 ON src1.key+1 = src2.key RIGHT OUTER JOIN T2 src3 ON src2.key = src3.key; + +SELECT sum(hash(src1.key)), sum(hash(src1.val)), sum(hash(src2.key)) FROM T1 src1 JOIN T2 src2 ON src1.key+1 = src2.key JOIN T2 src3 ON src2.key = src3.key; + +select /*+ mapjoin(v)*/ sum(hash(k.key)), sum(hash(v.val)) from T1 k left outer join T1 v on k.key+1=v.key; + + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q new file mode 100644 index 0000000000000..03eab4cd6d54e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q @@ -0,0 +1,68 @@ +set hive.internal.ddl.list.bucketing.enable=true; +set hive.optimize.skewjoin.compiletime = true; +set hive.mapred.supports.subdirectories=true; + +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink and skewjoin optimization +-- Union of 2 map-reduce subqueries is performed for the skew join +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output, it might be easier to run the test +-- only on hadoop 23 + +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; + +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; + +-- a simple join query with skew on both the tables on the join key + +EXPLAIN +SELECT * FROM T1 a JOIN T2 b ON a.key = b.key; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +SELECT * FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; + +-- test outer joins also + +EXPLAIN +SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; + +create table DEST1(key1 STRING, val1 STRING, key2 STRING, val2 STRING); + +EXPLAIN +INSERT OVERWRITE TABLE DEST1 +SELECT * FROM T1 a JOIN T2 b ON a.key = b.key; + +INSERT OVERWRITE TABLE DEST1 +SELECT * FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT * FROM DEST1 +ORDER BY key1, key2, val1, val2; + +EXPLAIN +INSERT OVERWRITE TABLE DEST1 +SELECT * FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key; + +INSERT OVERWRITE TABLE DEST1 +SELECT * FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key; + +SELECT * FROM DEST1 +ORDER BY key1, key2, val1, val2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q new file mode 100644 index 0000000000000..9cb919531f7c5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q @@ -0,0 +1,40 @@ +set hive.internal.ddl.list.bucketing.enable=true; +set hive.optimize.skewjoin.compiletime = true; +set hive.mapred.supports.subdirectories=true; + +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +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; + +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; + +CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; + +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 +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table, it might be easier +-- to run the test only on hadoop 23 + +EXPLAIN +SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key +ORDER BY a.key, b.key, c.key, a.val, b.val, c.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt1.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt1.q new file mode 100644 index 0000000000000..af446bb65c08e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt1.q @@ -0,0 +1,42 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +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; + +-- 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.key = b.key; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; + +-- test outer joins also + +EXPLAIN +SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; + +-- an aggregation at the end should not change anything + +EXPLAIN +SELECT count(1) FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT count(1) FROM T1 a JOIN T2 b ON a.key = b.key; + +EXPLAIN +SELECT count(1) FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key; + +SELECT count(1) FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt10.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt10.q new file mode 100644 index 0000000000000..199f3201afb77 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt10.q @@ -0,0 +1,20 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +drop table array_valued_T1; +create table array_valued_T1 (key string, value array) SKEWED BY (key) ON ((8)); +insert overwrite table array_valued_T1 select key, array(value) from T1; + +-- This test is to verify the skew join compile optimization when the join is followed by a lateral view +-- adding a order by at the end to make the results deterministic + +explain +select * from (select a.key as key, b.value as array_val from T1 a join array_valued_T1 b on a.key=b.key) i lateral view explode (array_val) c as val; + +select * from (select a.key as key, b.value as array_val from T1 a join array_valued_T1 b on a.key=b.key) i lateral view explode (array_val) c as val +ORDER BY key, val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt11.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt11.q new file mode 100644 index 0000000000000..ef61fb22f17a1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt11.q @@ -0,0 +1,33 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; + +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 +-- skew join compile time optimization. +-- adding an order by at the end to make the results deterministic + +EXPLAIN +select * from +( + select a.key, a.val as val1, b.val as val2 from T1 a join T2 b on a.key = b.key + union all + select a.key, a.val as val1, b.val as val2 from T1 a join T2 b on a.key = b.key +) subq1; + +select * from +( + select a.key, a.val as val1, b.val as val2 from T1 a join T2 b on a.key = b.key + union all + select a.key, a.val as val1, b.val as val2 from T1 a join T2 b on a.key = b.key +) subq1 +ORDER BY key, val1, val2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt12.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt12.q new file mode 100644 index 0000000000000..b5d9d9bc46874 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt12.q @@ -0,0 +1,23 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +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; + +-- 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. +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key and a.val = b.val; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key and a.val = b.val +ORDER BY a.key, b.key, a.val, b.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt13.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt13.q new file mode 100644 index 0000000000000..0634c4f4ff6db --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt13.q @@ -0,0 +1,36 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; + +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; + +-- 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 +-- tables 1 and 2. Table 3 is skewed, but since one of the join sources for table +-- 3 consist of a sub-query which contains a join, the compile time skew join +-- optimization is not performed +-- adding a order by at the end to make the results deterministic + +EXPLAIN +select * +from +T1 a join T2 b on a.key = b.key +join T3 c on a.val = c.val; + +select * +from +T1 a join T2 b on a.key = b.key +join T3 c on a.val = c.val +order by a.key, b.key, c.key, a.val, b.val, c.val; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt14.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt14.q new file mode 100644 index 0000000000000..0f031dd4fc0ae --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt14.q @@ -0,0 +1,38 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; + +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; + +-- 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 +-- tables 1 and 2. Tables 1 and 3 are skewed. Since one of the join sources for table +-- 3 consist of a sub-query which contains a join, the compile time skew join +-- optimization is not enabled for table 3, but it is used for the first join between +-- tables 1 and 2 +-- adding a order by at the end to make the results deterministic + +EXPLAIN +select * +from +T1 a join T2 b on a.key = b.key +join T3 c on a.val = c.val; + +select * +from +T1 a join T2 b on a.key = b.key +join T3 c on a.val = c.val +order by a.key, b.key, a.val, b.val; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt15.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt15.q new file mode 100644 index 0000000000000..d5474a455e9dd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt15.q @@ -0,0 +1,51 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +-- testing skew on other data types - int +CREATE TABLE T1(key INT, val STRING) SKEWED BY (key) ON ((2)); +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; + +CREATE TABLE T2(key INT, val STRING) SKEWED BY (key) ON ((3)); + +INSERT OVERWRITE TABLE T2 SELECT key, val FROM tmpT2; + +-- The skewed key is a integer column. +-- Otherwise this test is similar to skewjoinopt1.q +-- Both the joined tables are skewed, and the joined column +-- is an integer +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; + +-- test outer joins also + +EXPLAIN +SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; + +-- an aggregation at the end should not change anything + +EXPLAIN +SELECT count(1) FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT count(1) FROM T1 a JOIN T2 b ON a.key = b.key; + +EXPLAIN +SELECT count(1) FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key; + +SELECT count(1) FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt16.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt16.q new file mode 100644 index 0000000000000..46b4f6d6e600d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt16.q @@ -0,0 +1,23 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +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; + +-- 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 +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key and a.val = b.val; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key and a.val = b.val +ORDER BY a.key, b.key, a.val, b.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt17.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt17.q new file mode 100644 index 0000000000000..0592ca8c3e498 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt17.q @@ -0,0 +1,49 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +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; + +-- 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 +-- The skewed value for the jon key is common to both the tables. +-- In this case, the skewed join value is not repeated in the filter. +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; + +DROP TABLE T1; +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; + +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; + +-- 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 +-- In this case, the skewed join value is repeated in the filter. + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key and a.val = b.val; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key and a.val = b.val +ORDER BY a.key, b.key, a.val, b.val; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt18.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt18.q new file mode 100644 index 0000000000000..433fea336dfd4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt18.q @@ -0,0 +1,30 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +-- testing skew on other data types - int +CREATE TABLE T1(key INT, val STRING) SKEWED BY (key) ON ((2)); +INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1; + +-- Tke skewed column is same in both the tables, however it is +-- INT in one of the tables, and STRING in the other table + +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; + +-- 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 +-- regular join +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt19.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt19.q new file mode 100644 index 0000000000000..0b11ebe4cb696 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt19.q @@ -0,0 +1,24 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +set hive.optimize.skewjoin.compiletime = true; + +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; + +CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; + +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 +-- optimization is performed +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt2.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt2.q new file mode 100644 index 0000000000000..34fcdbfac4cb7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt2.q @@ -0,0 +1,45 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +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; + +-- a simple query with skew on both the tables on the join key +-- multiple skew values are present for the skewed keys +-- but the skewed values do not overlap. +-- The join values are a superset of the skewed keys. +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key and a.val = b.val; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key and a.val = b.val +ORDER BY a.key, b.key, a.val, b.val; + +-- test outer joins also + +EXPLAIN +SELECT a.*, b.* FROM T1 a LEFT OUTER JOIN T2 b ON a.key = b.key and a.val = b.val; + +SELECT a.*, b.* FROM T1 a LEFT OUTER JOIN T2 b ON a.key = b.key and a.val = b.val +ORDER BY a.key, b.key, a.val, b.val; + +-- a group by at the end should not change anything + +EXPLAIN +SELECT a.key, count(1) FROM T1 a JOIN T2 b ON a.key = b.key and a.val = b.val group by a.key; + +SELECT a.key, count(1) FROM T1 a JOIN T2 b ON a.key = b.key and a.val = b.val group by a.key; + +EXPLAIN +SELECT a.key, count(1) FROM T1 a LEFT OUTER JOIN T2 b ON a.key = b.key and a.val = b.val group by a.key; + +SELECT a.key, count(1) FROM T1 a LEFT OUTER JOIN T2 b ON a.key = b.key and a.val = b.val group by a.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt20.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt20.q new file mode 100644 index 0000000000000..f217052881e24 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt20.q @@ -0,0 +1,24 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +set hive.optimize.skewjoin.compiletime = true; + +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; + +CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; + +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 +-- optimization is performed +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt3.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt3.q new file mode 100644 index 0000000000000..f6002ad49802a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt3.q @@ -0,0 +1,32 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +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; + +-- 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 +-- repeated in the filter. +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; + +-- test outer joins also + +EXPLAIN +SELECT a.*, b.* FROM T1 a FULL OUTER JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a FULL OUTER JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt4.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt4.q new file mode 100644 index 0000000000000..ca83c446085fa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt4.q @@ -0,0 +1,29 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; + +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 +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; + +-- the order of the join should not matter, just confirming +EXPLAIN +SELECT a.*, b.* FROM T2 a JOIN T1 b ON a.key = b.key; + +SELECT a.*, b.* FROM T2 a JOIN T1 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt5.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt5.q new file mode 100644 index 0000000000000..3d7884c5e3dcb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt5.q @@ -0,0 +1,23 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +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; + +-- 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 +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt6.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt6.q new file mode 100644 index 0000000000000..36cf8ceeaebb4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt6.q @@ -0,0 +1,24 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +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; + +-- 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: +-- it only contains the first skewed key for both the tables +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key; + +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key +ORDER BY a.key, b.key, a.val, b.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt7.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt7.q new file mode 100644 index 0000000000000..cf84f67b6a0fc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt7.q @@ -0,0 +1,28 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +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; + +CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; + +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. +-- 2 of the 3 tables are skewed on the join key +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key; + +SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key +ORDER BY a.key, b.key, c.key, a.val, b.val, c.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt8.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt8.q new file mode 100644 index 0000000000000..d0ac845f86581 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt8.q @@ -0,0 +1,27 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +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; + +CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; + +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. +-- 1 of the 3 tables are skewed on the join key +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key; + +SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key +ORDER BY a.key, b.key, c.key, a.val, b.val, c.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt9.q b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt9.q new file mode 100644 index 0000000000000..04834033a11e6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt9.q @@ -0,0 +1,49 @@ +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +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; + +CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; + +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 +-- adding a order by at the end to make the results deterministic +EXPLAIN +select * from +( +select key, val from T1 + union all +select key, val from T1 +) subq1 +join T2 b on subq1.key = b.key; + +select * from +( +select key, val from T1 + union all +select key, val from T1 +) subq1 +join T2 b on subq1.key = b.key +ORDER BY subq1.key, b.key, subq1.val, b.val; + +-- no skew join compile time optimization would be performed if one of the +-- join sources is a sub-query consisting of a group by +EXPLAIN +select * from +( +select key, count(1) as cnt from T1 group by key +) subq1 +join T2 b on subq1.key = b.key; + +select * from +( +select key, count(1) as cnt from T1 group by key +) subq1 +join T2 b on subq1.key = b.key +ORDER BY subq1.key, b.key, subq1.cnt, b.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin9.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin9.q new file mode 100644 index 0000000000000..b9590227576c9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin9.q @@ -0,0 +1,57 @@ +create table hive_test_smb_bucket1 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 2 buckets; +create table hive_test_smb_bucket2 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 2 buckets; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +-- empty partitions (HIVE-3205) +explain extended +SELECT /* + MAPJOIN(b) */ b.key as k1, b.value, b.ds, a.key as k2 +FROM hive_test_smb_bucket1 a JOIN +hive_test_smb_bucket2 b +ON a.key = b.key WHERE a.ds = '2010-10-15' and b.ds='2010-10-15' and b.key IS NOT NULL; + +SELECT /* + MAPJOIN(b) */ b.key as k1, b.value, b.ds, a.key as k2 +FROM hive_test_smb_bucket1 a JOIN +hive_test_smb_bucket2 b +ON a.key = b.key WHERE a.ds = '2010-10-15' and b.ds='2010-10-15' and b.key IS NOT NULL; + +explain extended +SELECT /* + MAPJOIN(a) */ b.key as k1, b.value, b.ds, a.key as k2 +FROM hive_test_smb_bucket1 a JOIN +hive_test_smb_bucket2 b +ON a.key = b.key WHERE a.ds = '2010-10-15' and b.ds='2010-10-15' and b.key IS NOT NULL; + +SELECT /* + MAPJOIN(a) */ b.key as k1, b.value, b.ds, a.key as k2 +FROM hive_test_smb_bucket1 a JOIN +hive_test_smb_bucket2 b +ON a.key = b.key WHERE a.ds = '2010-10-15' and b.ds='2010-10-15' and b.key IS NOT NULL; + +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.exec.reducers.max = 1; + +insert overwrite table hive_test_smb_bucket1 partition (ds='2010-10-15') select key, value from src; +insert overwrite table hive_test_smb_bucket2 partition (ds='2010-10-15') select key, value from src; + +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +explain +create table smb_mapjoin9_results as +SELECT /* + MAPJOIN(b) */ b.key as k1, b.value, b.ds, a.key as k2 +FROM hive_test_smb_bucket1 a JOIN +hive_test_smb_bucket2 b +ON a.key = b.key WHERE a.ds = '2010-10-15' and b.ds='2010-10-15' and b.key IS NOT NULL; + + +create table smb_mapjoin9_results as +SELECT /* + MAPJOIN(b) */ b.key as k1, b.value, b.ds, a.key as k2 +FROM hive_test_smb_bucket1 a JOIN +hive_test_smb_bucket2 b +ON a.key = b.key WHERE a.ds = '2010-10-15' and b.ds='2010-10-15' and b.key IS NOT NULL; + +drop table smb_mapjoin9_results; +drop table hive_test_smb_bucket1; +drop table hive_test_smb_bucket2; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_1.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_1.q new file mode 100644 index 0000000000000..359513e424dba --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_1.q @@ -0,0 +1,53 @@ + + + + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +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; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key; + + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_10.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_10.q new file mode 100644 index 0000000000000..a79ebf62d0693 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_10.q @@ -0,0 +1,26 @@ + +create table tmp_smb_bucket_10(userid int, pageid int, postid int, type string) partitioned by (ds string) CLUSTERED BY (userid) SORTED BY (pageid, postid, type, userid) INTO 2 BUCKETS STORED AS RCFILE; + +alter table tmp_smb_bucket_10 add partition (ds = '1'); +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='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; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +explain +select /*+mapjoin(a)*/ * from tmp_smb_bucket_10 a join tmp_smb_bucket_10 b +on (a.ds = '1' and b.ds = '2' and + a.userid = b.userid and + a.pageid = b.pageid and + a.postid = b.postid and + a.type = b.type); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_11.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_11.q new file mode 100644 index 0000000000000..48ba8e80cccf6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_11.q @@ -0,0 +1,34 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- This test verifies that the output of a sort merge join on 2 partitions (one on each side of the join) is bucketed + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT *; + +set hive.enforce.bucketing=false; +set hive.enforce.sorting=false; + +-- Create a bucketed table +CREATE TABLE test_table3 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) INTO 16 BUCKETS; + +-- Insert data into the bucketed table by joining the two bucketed and sorted tables, bucketing is not enforced +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT /*+ MAPJOIN(b) */ a.key, b.value FROM test_table1 a JOIN test_table2 b ON a.key = b.key AND a.ds = '1' AND b.ds = '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT /*+ MAPJOIN(b) */ a.key, b.value FROM test_table1 a JOIN test_table2 b ON a.key = b.key AND a.ds = '1' AND b.ds = '1'; + +-- Join data from a sampled bucket to verify the data is bucketed +SELECT COUNT(*) FROM test_table3 TABLESAMPLE(BUCKET 2 OUT OF 16) a JOIN test_table1 TABLESAMPLE(BUCKET 2 OUT OF 16) b ON a.key = b.key AND a.ds = '1' AND b.ds='1'; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_12.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_12.q new file mode 100644 index 0000000000000..9f9748c41f3ab --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_12.q @@ -0,0 +1,49 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- This test verifies that the output of a sort merge join on 1 big partition with multiple small partitions is bucketed and sorted + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '2') SELECT * +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '3') SELECT *; + +set hive.enforce.bucketing=false; +set hive.enforce.sorting=false; + +-- Create a bucketed table +CREATE TABLE test_table3 (key INT, value STRING) PARTITIONED BY (ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; + +-- Insert data into the bucketed table by joining the two bucketed and sorted tables, bucketing is not enforced +EXPLAIN EXTENDED +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT /*+ MAPJOIN(b) */ a.key, b.value FROM test_table1 a JOIN test_table2 b ON a.key = b.key AND a.ds = '1' AND b.ds >= '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT /*+ MAPJOIN(b) */ a.key, b.value FROM test_table1 a JOIN test_table2 b ON a.key = b.key AND a.ds = '1' AND b.ds >= '1'; + +-- Join data from a sampled bucket to verify the data is bucketed +SELECT COUNT(*) FROM test_table3 TABLESAMPLE(BUCKET 2 OUT OF 16) a JOIN test_table1 TABLESAMPLE(BUCKET 2 OUT OF 16) b ON a.key = b.key AND a.ds = '1' AND b.ds='1'; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +-- Join data from the sampled buckets of 2 tables to verify the data is bucketed and sorted +explain extended +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '2') +SELECT /*+mapjoin(b)*/ a.key, concat(a.value, b.value) FROM test_table3 a JOIN test_table1 b ON a.key = b.key AND a.ds = '1' AND b.ds='1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '2') +SELECT /*+mapjoin(b)*/ a.key, concat(a.value, b.value) FROM test_table3 a JOIN test_table1 b ON a.key = b.key AND a.ds = '1' AND b.ds='1'; + +SELECT count(*) from test_table3 tablesample (bucket 2 out of 16) a where ds = '2'; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_13.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_13.q new file mode 100644 index 0000000000000..056bccd68999b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_13.q @@ -0,0 +1,36 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- This test verifies that the sort merge join optimizer works when the tables are joined on columns with different names + +-- Create bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key ASC) INTO 16 BUCKETS; +CREATE TABLE test_table2 (value INT, key STRING) CLUSTERED BY (value) SORTED BY (value ASC) INTO 16 BUCKETS; +CREATE TABLE test_table3 (key INT, value STRING) CLUSTERED BY (key, value) SORTED BY (key ASC, value ASC) INTO 16 BUCKETS; +CREATE TABLE test_table4 (key INT, value STRING) CLUSTERED BY (key, value) SORTED BY (value ASC, key ASC) INTO 16 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 SELECT * +INSERT OVERWRITE TABLE test_table2 SELECT * +INSERT OVERWRITE TABLE test_table3 SELECT * +INSERT OVERWRITE TABLE test_table4 SELECT *; + +-- Join data from 2 tables on their respective sorted columns (one each, with different names) and +-- verify sort merge join is used +EXPLAIN EXTENDED +SELECT /*+mapjoin(b)*/ * FROM test_table1 a JOIN test_table2 b ON a.key = b.value ORDER BY a.key LIMIT 10; + +SELECT /*+mapjoin(b)*/ * FROM test_table1 a JOIN test_table2 b ON a.key = b.value ORDER BY a.key LIMIT 10; + +-- Join data from 2 tables on their respective columns (two each, with the same names but sorted +-- with different priorities) and verify sort merge join is not used +EXPLAIN EXTENDED +SELECT /*+mapjoin(b)*/ * FROM test_table3 a JOIN test_table4 b ON a.key = b.value ORDER BY a.key LIMIT 10; + +SELECT /*+mapjoin(b)*/ * FROM test_table3 a JOIN test_table4 b ON a.key = b.value ORDER BY a.key LIMIT 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_14.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_14.q new file mode 100644 index 0000000000000..f03f92e469d53 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_14.q @@ -0,0 +1,243 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +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; + +insert overwrite table tbl1 +select * from src where key < 10; + +insert overwrite table tbl2 +select * from src where key < 10; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +-- The mapjoin is being performed as part of sub-query. It should be converted to a sort-merge join +explain +select count(*) from ( + select /*+mapjoin(a)*/ a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +) subq1; + +select count(*) from ( + select /*+mapjoin(a)*/ a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +) subq1; + +-- The mapjoin is being performed as part of sub-query. It should be converted to a sort-merge join +-- Add a order by at the end to make the results deterministic. +explain +select key, count(*) from +( + select /*+mapjoin(a)*/ 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 +order by key; + +select key, count(*) from +( + select /*+mapjoin(a)*/ 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 +order by key; + +-- The mapjoin is being performed as part of more than one sub-query. It should be converted to a sort-merge join +explain +select count(*) from +( + select key, count(*) from + ( + select /*+mapjoin(a)*/ 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 +) subq2; + +select count(*) from +( + select key, count(*) from + ( + select /*+mapjoin(a)*/ 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 +) subq2; + +-- The subquery itself is being map-joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join. +explain +select /*+mapjoin(subq1)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +select /*+mapjoin(subq1)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +-- The subquery itself is being map-joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join, although there is more than one level of sub-query +explain +select /*+mapjoin(subq2)*/ count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key; + +select /*+mapjoin(subq2)*/ count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key; + +-- Both the big table and the small table are nested sub-queries i.e more then 1 level of sub-query. +-- The join should be converted to a sort-merge join +explain +select /*+mapjoin(subq2)*/ count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key; + +select /*+mapjoin(subq2)*/ count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key; + +-- The subquery itself is being map-joined. Since the sub-query only contains selects and filters and the join key +-- is not getting modified, it should be converted to a sort-merge join. Note that the sub-query modifies one +-- item, but that is not part of the join key. +explain +select /*+mapjoin(subq1)*/ count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key; + +select /*+mapjoin(subq1)*/ count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key; + +-- Since the join key is modified by the sub-query, neither sort-merge join not bucketized map-side +-- join should be performed +explain +select /*+mapjoin(subq1)*/ count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key; + +select /*+mapjoin(subq1)*/ count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key; + +-- The small table is a sub-query and the big table is not. +-- It should be converted to a sort-merge join. +explain +select /*+mapjoin(subq1)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +select /*+mapjoin(subq1)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +-- The big table is a sub-query and the small table is not. +-- It should be converted to a sort-merge join. +explain +select /*+mapjoin(a)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +select /*+mapjoin(a)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +-- There are more than 2 inputs to the join, all of them being sub-queries. +-- It should be converted to to a sort-merge join +explain +select /*+mapjoin(subq1, subq2)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on (subq1.key = subq2.key) + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key); + +select /*+mapjoin(subq1, subq2)*/ count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key); + +-- The mapjoin is being performed on a nested sub-query, and an aggregation is performed after that. +-- The join should be converted to a sort-merge join +explain +select count(*) from ( + select /*+mapjoin(subq2)*/ subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a; + +select count(*) from ( + select /*+mapjoin(subq2)*/ subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_15.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_15.q new file mode 100644 index 0000000000000..1e77a60670b73 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_15.q @@ -0,0 +1,55 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- This test verifies that the sort merge join optimizer works when the tables are sorted on columns which is a superset +-- of join columns + +-- Create bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key ASC, value ASC) INTO 16 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key ASC, value ASC) INTO 16 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 SELECT * +INSERT OVERWRITE TABLE test_table2 SELECT *; + +-- it should be converted to a sort-merge join, since the first sort column (#join columns = 1) contains the join columns +EXPLAIN EXTENDED +SELECT /*+mapjoin(b)*/ * FROM test_table1 a JOIN test_table2 b ON a.key = b.key ORDER BY a.key LIMIT 10; +SELECT /*+mapjoin(b)*/ * FROM test_table1 a JOIN test_table2 b ON a.key = b.key ORDER BY a.key LIMIT 10; + +DROP TABLE test_table1; +DROP TABLE test_table2; + +-- Create bucketed and sorted tables +CREATE TABLE test_table1 (key INT, key2 INT, value STRING) CLUSTERED BY (key) SORTED BY (key ASC, key2 ASC, value ASC) INTO 16 BUCKETS; +CREATE TABLE test_table2 (key INT, key2 INT, value STRING) CLUSTERED BY (key) SORTED BY (key ASC, key2 ASC, value ASC) INTO 16 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 SELECT key, key, value +INSERT OVERWRITE TABLE test_table2 SELECT key, key, value; + +-- it should be converted to a sort-merge join, since the first 2 sort columns (#join columns = 2) contain the join columns +EXPLAIN EXTENDED +SELECT /*+mapjoin(b)*/ * FROM test_table1 a JOIN test_table2 b ON a.key = b.key and a.key2 = b.key2 ORDER BY a.key LIMIT 10; +SELECT /*+mapjoin(b)*/ * FROM test_table1 a JOIN test_table2 b ON a.key = b.key and a.key2 = b.key2 ORDER BY a.key LIMIT 10; + +-- it should be converted to a sort-merge join, since the first 2 sort columns (#join columns = 2) contain the join columns +-- even if the order is not the same +EXPLAIN EXTENDED +SELECT /*+mapjoin(b)*/ * FROM test_table1 a JOIN test_table2 b ON a.key2 = b.key2 and a.key = b.key ORDER BY a.key LIMIT 10; +SELECT /*+mapjoin(b)*/ * FROM test_table1 a JOIN test_table2 b ON a.key2 = b.key2 and a.key = b.key ORDER BY a.key LIMIT 10; + +-- it should not be converted to a sort-merge join, since the first 2 sort columns (#join columns = 2) do not contain all +-- the join columns +EXPLAIN EXTENDED +SELECT /*+mapjoin(b)*/ * FROM test_table1 a JOIN test_table2 b ON a.key = b.key and a.value = b.value ORDER BY a.key LIMIT 10; +SELECT /*+mapjoin(b)*/ * FROM test_table1 a JOIN test_table2 b ON a.key = b.key and a.value = b.value ORDER BY a.key LIMIT 10; + +DROP TABLE test_table1; +DROP TABLE test_table2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_16.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_16.q new file mode 100644 index 0000000000000..3a3a872427526 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_16.q @@ -0,0 +1,21 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- Create bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 SELECT * +INSERT OVERWRITE TABLE test_table2 SELECT *; + +-- Mapjoin followed by a aggregation should be performed in a single MR job +EXPLAIN +SELECT /*+mapjoin(b)*/ count(*) FROM test_table1 a JOIN test_table2 b ON a.key = b.key; +SELECT /*+mapjoin(b)*/ count(*) FROM test_table1 a JOIN test_table2 b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_17.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_17.q new file mode 100644 index 0000000000000..238f7e011b72c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_17.q @@ -0,0 +1,125 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- Create bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table3 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table4 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table5 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table6 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table7 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table8 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +INSERT OVERWRITE TABLE test_table1 +SELECT * FROM src WHERE key < 10; + +INSERT OVERWRITE TABLE test_table2 +SELECT * FROM src WHERE key < 10; + +INSERT OVERWRITE TABLE test_table3 +SELECT * FROM src WHERE key < 10; + +INSERT OVERWRITE TABLE test_table4 +SELECT * FROM src WHERE key < 10; + +INSERT OVERWRITE TABLE test_table5 +SELECT * FROM src WHERE key < 10; + +INSERT OVERWRITE TABLE test_table6 +SELECT * FROM src WHERE key < 10; + +INSERT OVERWRITE TABLE test_table7 +SELECT * FROM src WHERE key < 10; + +INSERT OVERWRITE TABLE test_table8 +SELECT * FROM src WHERE key < 10; + +-- Mapjoin followed by a aggregation should be performed in a single MR job upto 7 tables +EXPLAIN +SELECT /*+ mapjoin(b, c, d, e, f, g) */ count(*) +FROM test_table1 a JOIN test_table2 b ON a.key = b.key +JOIN test_table3 c ON a.key = c.key +JOIN test_table4 d ON a.key = d.key +JOIN test_table5 e ON a.key = e.key +JOIN test_table6 f ON a.key = f.key +JOIN test_table7 g ON a.key = g.key; + +SELECT /*+ mapjoin(b, c, d, e, f, g) */ count(*) +FROM test_table1 a JOIN test_table2 b ON a.key = b.key +JOIN test_table3 c ON a.key = c.key +JOIN test_table4 d ON a.key = d.key +JOIN test_table5 e ON a.key = e.key +JOIN test_table6 f ON a.key = f.key +JOIN test_table7 g ON a.key = g.key; + +set hive.auto.convert.join=true; +set hive.auto.convert.sortmerge.join=true; + +-- It should be automatically converted to a sort-merge join followed by a groupby in +-- a single MR job +EXPLAIN +SELECT count(*) +FROM test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key; + +SELECT count(*) +FROM test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key; + +EXPLAIN +SELECT count(*) +FROM test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +LEFT OUTER JOIN test_table8 h ON a.key = h.key; + +SELECT count(*) +FROM test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +LEFT OUTER JOIN test_table8 h ON a.key = h.key; + +-- outer join with max 16 aliases +EXPLAIN +SELECT a.* +FROM test_table1 a +LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +LEFT OUTER JOIN test_table8 h ON a.key = h.key +LEFT OUTER JOIN test_table4 i ON a.key = i.key +LEFT OUTER JOIN test_table5 j ON a.key = j.key +LEFT OUTER JOIN test_table6 k ON a.key = k.key +LEFT OUTER JOIN test_table7 l ON a.key = l.key +LEFT OUTER JOIN test_table8 m ON a.key = m.key +LEFT OUTER JOIN test_table7 n ON a.key = n.key +LEFT OUTER JOIN test_table8 o ON a.key = o.key +LEFT OUTER JOIN test_table4 p ON a.key = p.key +LEFT OUTER JOIN test_table5 q ON a.key = q.key +LEFT OUTER JOIN test_table6 r ON a.key = r.key +LEFT OUTER JOIN test_table7 s ON a.key = s.key +LEFT OUTER JOIN test_table8 t ON a.key = t.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_18.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_18.q new file mode 100644 index 0000000000000..6880be340792a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_18.q @@ -0,0 +1,65 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT *; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +select count(*) from test_table1 where ds = '1'; +select count(*) from test_table1 where ds = '1' and hash(key) % 2 = 0; +select count(*) from test_table1 where ds = '1' and hash(key) % 2 = 1; +select count(*) from test_table1 tablesample (bucket 1 out of 2) s where ds = '1'; +select count(*) from test_table1 tablesample (bucket 2 out of 2) s where ds = '1'; + +select count(*) from test_table2 where ds = '1'; +select count(*) from test_table2 where ds = '1' and hash(key) % 2 = 0; +select count(*) from test_table2 where ds = '1' and hash(key) % 2 = 1; +select count(*) from test_table2 tablesample (bucket 1 out of 2) s where ds = '1'; +select count(*) from test_table2 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation, one of the buckets should be empty +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '2') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' and a.key = 238; + +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '2') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' and a.key = 238; + +select count(*) from test_table2 where ds = '2'; +select count(*) from test_table2 where ds = '2' and hash(key) % 2 = 0; +select count(*) from test_table2 where ds = '2' and hash(key) % 2 = 1; +select count(*) from test_table2 tablesample (bucket 1 out of 2) s where ds = '2'; +select count(*) from test_table2 tablesample (bucket 2 out of 2) s where ds = '2'; + +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '3') +SELECT a.key, a.value FROM test_table2 a WHERE a.ds = '2'; + +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '2') +SELECT a.key, a.value FROM test_table2 a WHERE a.ds = '2'; + +select count(*) from test_table2 where ds = '3'; +select count(*) from test_table2 where ds = '3' and hash(key) % 2 = 0; +select count(*) from test_table2 where ds = '3' and hash(key) % 2 = 1; +select count(*) from test_table2 tablesample (bucket 1 out of 2) s where ds = '3'; +select count(*) from test_table2 tablesample (bucket 2 out of 2) s where ds = '3'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_19.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_19.q new file mode 100644 index 0000000000000..ca48f61f9aaf8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_19.q @@ -0,0 +1,41 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 16 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT *; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +select count(*) from test_table1 where ds = '1'; +select count(*) from test_table1 where ds = '1' and hash(key) % 16 = 0; +select count(*) from test_table1 where ds = '1' and hash(key) % 16 = 5; +select count(*) from test_table1 where ds = '1' and hash(key) % 16 = 12; +select count(*) from test_table1 tablesample (bucket 1 out of 16) s where ds = '1'; +select count(*) from test_table1 tablesample (bucket 6 out of 16) s where ds = '1'; +select count(*) from test_table1 tablesample (bucket 13 out of 16) s where ds = '1'; + +select count(*) from test_table2 where ds = '1'; +select count(*) from test_table2 where ds = '1' and hash(key) % 16 = 0; +select count(*) from test_table2 where ds = '1' and hash(key) % 16 = 5; +select count(*) from test_table2 where ds = '1' and hash(key) % 16 = 12; +select count(*) from test_table2 tablesample (bucket 1 out of 16) s where ds = '1'; +select count(*) from test_table2 tablesample (bucket 6 out of 16) s where ds = '1'; +select count(*) from test_table2 tablesample (bucket 13 out of 16) s where ds = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_2.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_2.q new file mode 100644 index 0000000000000..9d86314879d68 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_2.q @@ -0,0 +1,53 @@ + + + + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +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; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key; + + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_20.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_20.q new file mode 100644 index 0000000000000..f70e7d5c86237 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_20.q @@ -0,0 +1,53 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key int, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key STRING, value1 STRING, value2 string) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT *; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- with different datatypes. This should be a map-reduce operation +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value, a.value FROM test_table1 a WHERE a.ds = '1'; + +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value, a.value FROM test_table1 a WHERE a.ds = '1'; + +select count(*) from test_table2 where ds = '1'; +select count(*) from test_table2 where ds = '1' and hash(key) % 2 = 0; +select count(*) from test_table2 where ds = '1' and hash(key) % 2 = 1; + +CREATE TABLE test_table3 (key STRING, value1 int, value2 string) PARTITIONED BY (ds STRING) +CLUSTERED BY (value1) SORTED BY (value1) INTO 2 BUCKETS; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation, although the bucketing positions dont match +EXPLAIN +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.value, a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') +SELECT a.value, a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +select count(*) from test_table3 where ds = '1'; +select count(*) from test_table3 where ds = '1' and hash(value1) % 2 = 0; +select count(*) from test_table3 where ds = '1' and hash(value1) % 2 = 1; +select count(*) from test_table3 tablesample (bucket 1 out of 2) s where ds = '1'; +select count(*) from test_table3 tablesample (bucket 2 out of 2) s where ds = '1'; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- However, since an expression is being selected, it should involve a reducer +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '2') +SELECT a.key+a.key, a.value, a.value FROM test_table1 a WHERE a.ds = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_21.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_21.q new file mode 100644 index 0000000000000..09edfc10cd950 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_21.q @@ -0,0 +1,77 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT *; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +drop table test_table2; + +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key desc) INTO 2 BUCKETS; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-reduce operation since the sort orders does not match +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +drop table test_table2; + +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key, value) INTO 2 BUCKETS; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-reduce operation since the sort columns do not match +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +drop table test_table2; + +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (value) INTO 2 BUCKETS; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-reduce operation since the sort columns do not match +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +drop table test_table2; + +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-reduce operation since the number of buckets do not match +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1'; + +drop table test_table2; + +CREATE TABLE test_table2 (key INT, value STRING) PARTITIONED BY (ds STRING) +CLUSTERED BY (key) INTO 2 BUCKETS; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-reduce operation since sort columns do not match +EXPLAIN +INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '1') +SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_22.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_22.q new file mode 100644 index 0000000000000..676f46ae6d2aa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_22.q @@ -0,0 +1,55 @@ +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +-- Create two bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 SELECT *; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN INSERT OVERWRITE TABLE test_table2 +SELECT * FROM test_table1; + +INSERT OVERWRITE TABLE test_table2 +SELECT * FROM test_table1; + +select count(*) from test_table1; +select count(*) from test_table1 tablesample (bucket 2 out of 2) s; + +select count(*) from test_table2; +select count(*) from test_table2 tablesample (bucket 2 out of 2) s; + +drop table test_table1; +drop table test_table2; + +CREATE TABLE test_table1 (key INT, value STRING) +CLUSTERED BY (key) INTO 2 BUCKETS; +CREATE TABLE test_table2 (key INT, value STRING) +CLUSTERED BY (key) INTO 2 BUCKETS; + +FROM src +INSERT OVERWRITE TABLE test_table1 SELECT *; + +-- Insert data into the bucketed table by selecting from another bucketed table +-- This should be a map-only operation +EXPLAIN INSERT OVERWRITE TABLE test_table2 +SELECT * FROM test_table1; + +INSERT OVERWRITE TABLE test_table2 +SELECT * FROM test_table1; + +select count(*) from test_table1; +select count(*) from test_table1 tablesample (bucket 2 out of 2) s; + +select count(*) from test_table2; +select count(*) from test_table2 tablesample (bucket 2 out of 2) s; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_25.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_25.q new file mode 100644 index 0000000000000..8b534e85aee1d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_25.q @@ -0,0 +1,41 @@ +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.max.dynamic.partitions.pernode=1000000; +set hive.exec.max.dynamic.partitions=1000000; +set hive.exec.max.created.files=1000000; +set hive.map.aggr=true; + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +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; + +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; + +set hive.optimize.bucketmapjoin=true; +set hive.optimize.bucketmapjoin.sortedmerge=true; +set hive.mapred.reduce.tasks.speculative.execution=false; +set hive.auto.convert.join=true; +set hive.auto.convert.sortmerge.join=true; +set hive.auto.convert.sortmerge.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000000000; +set hive.optimize.reducededuplication.min.reducer=1; +set hive.optimize.mapjoin.mapreduce=true; +set hive.auto.convert.sortmerge.join.bigtable.selection.policy=org.apache.hadoop.hive.ql.optimizer.LeftmostBigTableSelectorForAutoSMJ; + +-- explain +-- select * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join src c on a.key=c.value + +-- select a.key from smb_bucket_1 a + +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; + +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/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_3.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_3.q new file mode 100644 index 0000000000000..73b21fae250e0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_3.q @@ -0,0 +1,53 @@ + + + + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +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; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key; + + +explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_4.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_4.q new file mode 100644 index 0000000000000..83143b170ed5c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_4.q @@ -0,0 +1,72 @@ + + + + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +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; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_5.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_5.q new file mode 100644 index 0000000000000..61ec084f64ffa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_5.q @@ -0,0 +1,72 @@ + + + + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; +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; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_6.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_6.q new file mode 100644 index 0000000000000..2884a11433d32 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_6.q @@ -0,0 +1,78 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + + +CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS RCFILE; + + +CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS RCFILE; + +create table smb_join_results(k1 int, v1 string, k2 int, v2 string); +create table normal_join_results(k1 int, v1 string, k2 int, v2 string); + +insert overwrite table smb_bucket4_1 +select * from src; + +insert overwrite table smb_bucket4_2 +select * from src; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +explain +insert overwrite table smb_join_results +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; + +insert overwrite table smb_join_results +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; + +select * from smb_join_results order by k1; + +insert overwrite table normal_join_results select * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; + +select sum(hash(k1)) as k1, sum(hash(k2)) as k2, sum(hash(v1)) as v1, sum(hash(v2)) as v2 from normal_join_results; +select sum(hash(k1)) as k1, sum(hash(k2)) as k2, sum(hash(v1)) as v1, sum(hash(v2)) as v2 from smb_join_results; + + +explain +insert overwrite table smb_join_results +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; +insert overwrite table smb_join_results +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; + + +insert overwrite table smb_join_results +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; + +select * from smb_join_results order by k1; + +insert overwrite table normal_join_results select * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; + +select sum(hash(k1)) as k1, sum(hash(k2)) as k2, sum(hash(v1)) as v1, sum(hash(v2)) as v2 from normal_join_results; +select sum(hash(k1)) as k1, sum(hash(k2)) as k2, sum(hash(v1)) as v1, sum(hash(v2)) as v2 from smb_join_results; + + +explain +insert overwrite table smb_join_results +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000; +insert overwrite table smb_join_results +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000; + + +explain +insert overwrite table smb_join_results +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000; +insert overwrite table smb_join_results +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000; + + +explain +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key join smb_bucket4_2 c on b.key = c.key where a.key>1000; +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key join smb_bucket4_2 c on b.key = c.key where a.key>1000; + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_7.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_7.q new file mode 100644 index 0000000000000..1488b1f949527 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_7.q @@ -0,0 +1,55 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + + +CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + + +CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + + + + +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; + +insert overwrite table smb_bucket4_2 +select * from src; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +insert overwrite table smb_join_results_empty_bigtable +select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; + +insert overwrite table smb_join_results_empty_bigtable +select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; + +select * from smb_join_results_empty_bigtable order by k1, v1, k2, v2; + +explain +insert overwrite table smb_join_results +select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; + +insert overwrite table smb_join_results +select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; + +select * from smb_join_results order by k1, v1, k2, v2; + +insert overwrite table normal_join_results select * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; + +select sum(hash(k1)) as k1, sum(hash(k2)) as k2, sum(hash(v1)) as v1, sum(hash(v2)) as v2 from normal_join_results; +select sum(hash(k1)) as k1, sum(hash(k2)) as k2, sum(hash(v1)) as v1, sum(hash(v2)) as v2 from smb_join_results; +select sum(hash(k1)) as k1, sum(hash(k2)) as k2, sum(hash(v1)) as v1, sum(hash(v2)) as v2 from smb_join_results_empty_bigtable; + + + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_8.q b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_8.q new file mode 100644 index 0000000000000..6f282ed441bfa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_8.q @@ -0,0 +1,90 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +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; + + +CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; + +CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; + +CREATE TABLE smb_bucket4_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; + +insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4 or key=2000 or key=4000; +insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=484 or key=3000 or key=5000; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; + + +insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=2000 or key=4000; +insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=3000 or key=5000; + +select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; + + +insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4000; +insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=5000; + +select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; + + +insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000; +insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000; + +select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key; + + +insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000; +insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000; +insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=1000 or key=5000; + +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key; + + +insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000; +insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000; +insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=1000 or key=4000; + +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key; + + +insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4000; +insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=5000; +insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=4000; + +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key; + + +insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=00000; +insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=4000; +insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=5000; + +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key; + + +insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=1000; +insert overwrite table smb_bucket4_2 select * from smb_bucket_input where key=4000; +insert overwrite table smb_bucket4_3 select * from smb_bucket_input where key=5000; + +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key +full outer join smb_bucket4_3 c on a.key=c.key; + + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort.q b/src/test/hive/ql/src/test/queries/clientpositive/sort.q new file mode 100644 index 0000000000000..a2a54ee393f10 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sort.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT x.* FROM SRC x SORT BY key; + +SELECT x.* FROM SRC x SORT BY key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q new file mode 100644 index 0000000000000..8002ec55fa7f2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q @@ -0,0 +1,23 @@ +drop table table_desc1; +drop table table_desc2; + +set hive.enforce.sorting = true; + +create table table_desc1(key string, value string) clustered by (key) sorted by (key DESC) into 1 BUCKETS; +create table table_desc2(key string, value string) clustered by (key) sorted by (key DESC) into 1 BUCKETS; + +insert overwrite table table_desc1 select key, value from src; +insert overwrite table table_desc2 select key, value from src; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +-- The columns of the tables above are sorted in same descending order. +-- So, sort merge join should be performed + +explain +select /*+ mapjoin(b) */ count(*) from table_desc1 a join table_desc2 b on a.key=b.key where a.key < 10; + +select /*+ mapjoin(b) */ count(*) from table_desc1 a join table_desc2 b on a.key=b.key where a.key < 10; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q new file mode 100644 index 0000000000000..bd0cdb2b952b4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q @@ -0,0 +1,28 @@ +drop table table_desc1; +drop table table_desc2; + +set hive.enforce.sorting = true; + +create table table_desc1(key string, value string) clustered by (key, value) +sorted by (key DESC, value DESC) into 1 BUCKETS; +create table table_desc2(key string, value string) clustered by (key, value) +sorted by (key DESC, value DESC) into 1 BUCKETS; + +insert overwrite table table_desc1 select key, value from src; +insert overwrite table table_desc2 select key, value from src; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +-- The columns of the tables above are sorted in same order. +-- descending followed by descending +-- So, sort merge join should be performed + +explain +select /*+ mapjoin(b) */ count(*) from table_desc1 a join table_desc2 b +on a.key=b.key and a.value=b.value where a.key < 10; + +select /*+ mapjoin(b) */ count(*) from table_desc1 a join table_desc2 b +on a.key=b.key and a.value=b.value where a.key < 10; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q new file mode 100644 index 0000000000000..a109878f88585 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q @@ -0,0 +1,28 @@ +drop table table_desc1; +drop table table_desc2; + +set hive.enforce.sorting = true; + +create table table_desc1(key string, value string) clustered by (key, value) +sorted by (key DESC, value ASC) into 1 BUCKETS; +create table table_desc2(key string, value string) clustered by (key, value) +sorted by (key DESC, value ASC) into 1 BUCKETS; + +insert overwrite table table_desc1 select key, value from src; +insert overwrite table table_desc2 select key, value from src; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +-- The columns of the tables above are sorted in same orders. +-- descending followed by ascending +-- So, sort merge join should be performed + +explain +select /*+ mapjoin(b) */ count(*) from table_desc1 a join table_desc2 b +on a.key=b.key and a.value=b.value where a.key < 10; + +select /*+ mapjoin(b) */ count(*) from table_desc1 a join table_desc2 b +on a.key=b.key and a.value=b.value where a.key < 10; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q new file mode 100644 index 0000000000000..0bc507130c242 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q @@ -0,0 +1,27 @@ +drop table table_desc1; +drop table table_desc2; + +set hive.enforce.sorting = true; + +create table table_desc1(key string, value string) clustered by (key, value) +sorted by (key DESC, value ASC) into 1 BUCKETS; +create table table_desc2(key string, value string) clustered by (key, value) +sorted by (key DESC, value DESC) into 1 BUCKETS; + +insert overwrite table table_desc1 select key, value from src; +insert overwrite table table_desc2 select key, value from src; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +-- The columns of the tables above are sorted in different orders. +-- So, sort merge join should not be performed + +explain +select /*+ mapjoin(b) */ count(*) from table_desc1 a join table_desc2 b +on a.key=b.key and a.value=b.value where a.key < 10; + +select /*+ mapjoin(b) */ count(*) from table_desc1 a join table_desc2 b +on a.key=b.key and a.value=b.value where a.key < 10; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q new file mode 100644 index 0000000000000..3505db09dd3ee --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q @@ -0,0 +1,26 @@ +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; + +CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (key) SORTED BY (key DESC) INTO 1 BUCKETS; +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part='1') SELECT * FROM src; + +CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (key) SORTED BY (key DESC) INTO 1 BUCKETS; +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_2 PARTITION (part='1') SELECT * FROM src; + +ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) SORTED BY (value DESC) INTO 1 BUCKETS; + +set hive.optimize.bucketmapjoin=true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +-- The partition sorting metadata matches but the table metadata does not, sorted merge join should still be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' AND b.part = '1'; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' AND b.part = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q new file mode 100644 index 0000000000000..35b05353ccbdd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q @@ -0,0 +1,27 @@ +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; + +CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (key) SORTED BY (key DESC) INTO 2 BUCKETS; +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part='1') SELECT * FROM src; + +CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (key) SORTED BY (value DESC) INTO 2 BUCKETS; +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_2 PARTITION (part='1') SELECT * FROM src; + +ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) SORTED BY (key DESC) INTO 2 BUCKETS; + +set hive.optimize.bucketmapjoin=true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +-- The table sorting metadata matches but the partition metadata does not, sorted merge join should not be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' AND b.part = '1'; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part = '1' AND b.part = '1'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q new file mode 100644 index 0000000000000..65dc7f1d8eda9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q @@ -0,0 +1,33 @@ +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; +set hive.exec.reducers.max = 1; + +CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (key, value) SORTED BY (key DESC) INTO 2 BUCKETS; +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part='1') SELECT * FROM src; + +ALTER TABLE srcbucket_mapjoin_part_1 CLUSTERED BY (key, value) SORTED BY (value DESC) INTO 2 BUCKETS; +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part='2') SELECT * FROM src; + +CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) +CLUSTERED BY (key, value) SORTED BY (value DESC) INTO 2 BUCKETS; +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_2 PARTITION (part='1') SELECT * FROM src; + +ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key, value) SORTED BY (key DESC) INTO 2 BUCKETS; +INSERT OVERWRITE TABLE srcbucket_mapjoin_part_2 PARTITION (part='2') SELECT * FROM src; + +ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key, value) SORTED BY (value DESC) INTO 2 BUCKETS; + +set hive.optimize.bucketmapjoin=true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +-- The table sorting metadata matches but the partition metadata does not, sorted merge join should not be used + +EXPLAIN EXTENDED +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part IS NOT NULL AND b.part IS NOT NULL; + +SELECT /*+ MAPJOIN(b) */ count(*) +FROM srcbucket_mapjoin_part_1 a JOIN srcbucket_mapjoin_part_2 b +ON a.key = b.key AND a.part IS NOT NULL AND b.part IS NOT NULL; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/source.q b/src/test/hive/ql/src/test/queries/clientpositive/source.q new file mode 100644 index 0000000000000..6fe3d211a00b0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/source.q @@ -0,0 +1 @@ +source ../data/files/source.txt; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/split_sample.q b/src/test/hive/ql/src/test/queries/clientpositive/split_sample.q new file mode 100644 index 0000000000000..952eaf72f10c1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/split_sample.q @@ -0,0 +1,115 @@ +USE default; + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +set mapred.max.split.size=300; +set mapred.min.split.size=300; +set mapred.min.split.size.per.node=300; +set mapred.min.split.size.per.rack=300; +set hive.merge.smallfiles.avgsize=1; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20) +-- This test sets mapred.max.split.size=300 and hive.merge.smallfiles.avgsize=1 +-- in an attempt to force the generation of multiple splits and multiple output files. +-- However, Hadoop 0.20 is incapable of generating splits smaller than the block size +-- when using CombineFileInputFormat, so only one split is generated. This has a +-- significant impact on the results of the TABLESAMPLE(x PERCENT). This issue was +-- fixed in MAPREDUCE-2046 which is included in 0.22. + +-- create multiple file inputs (two enable multiple splits) +create table ss_i_part (key int, value string) partitioned by (p string); +insert overwrite table ss_i_part partition (p='1') select key, value from src; +insert overwrite table ss_i_part partition (p='2') select key, value from src; +insert overwrite table ss_i_part partition (p='3') select key, value from src; +create table ss_src2 as select key, value from ss_i_part; + +select count(1) from ss_src2 tablesample(1 percent); + +-- sample first split +desc ss_src2; +set hive.sample.seednumber=0; +explain select key, value from ss_src2 tablesample(1 percent) limit 10; +select key, value from ss_src2 tablesample(1 percent) limit 10; + +-- verify seed number of sampling +insert overwrite table ss_i_part partition (p='1') select key+10000, value from src; +insert overwrite table ss_i_part partition (p='2') select key+20000, value from src; +insert overwrite table ss_i_part partition (p='3') select key+30000, value from src; +create table ss_src3 as select key, value from ss_i_part; +set hive.sample.seednumber=3; +create table ss_t3 as select sum(key) % 397 as s from ss_src3 tablesample(1 percent) limit 10; +set hive.sample.seednumber=4; +create table ss_t4 as select sum(key) % 397 as s from ss_src3 tablesample(1 percent) limit 10; +set hive.sample.seednumber=5; +create table ss_t5 as select sum(key) % 397 as s from ss_src3 tablesample(1 percent) limit 10; +select sum(s) from (select s from ss_t3 union all select s from ss_t4 union all select s from ss_t5) t; + +-- sample more than one split +explain select count(distinct key) from ss_src2 tablesample(70 percent) limit 10; +select count(distinct key) from ss_src2 tablesample(70 percent) limit 10; + +-- sample all splits +select count(1) from ss_src2 tablesample(100 percent); + +-- subquery +explain select key from (select key from ss_src2 tablesample(1 percent) limit 10) subq; +select key from (select key from ss_src2 tablesample(1 percent) limit 10) subq; + +-- groupby +select key, count(1) from ss_src2 tablesample(1 percent) group by key order by key; + +-- sample one of two tables: +create table ss_src1 as select * from ss_src2; +select t2.key as k from ss_src1 join ss_src2 tablesample(1 percent) t2 on ss_src1.key=t2.key order by k; + +-- sample two tables +explain select * from ( +select t1.key as k1, t2.key as k from ss_src1 tablesample(80 percent) t1 full outer join ss_src2 tablesample(2 percent) t2 on t1.key=t2.key +) subq where k in (199, 10199, 20199) or k1 in (199, 10199, 20199); + +select * from ( +select t1.key as k1, t2.key as k from ss_src1 tablesample(80 percent) t1 full outer join ss_src2 tablesample(2 percent) t2 on t1.key=t2.key +) subq where k in (199, 10199, 20199) or k1 in (199, 10199, 20199); + +-- shrink last split +explain select count(1) from ss_src2 tablesample(1 percent); +set mapred.max.split.size=300000; +set mapred.min.split.size=300000; +set mapred.min.split.size.per.node=300000; +set mapred.min.split.size.per.rack=300000; +select count(1) from ss_src2 tablesample(1 percent); +select count(1) from ss_src2 tablesample(50 percent); + +--HIVE-3401 more split samplings + +-- total length +explain +select count(1) from ss_src2 tablesample(100B); +select count(1) from ss_src2 tablesample(100B); + +explain +select count(1) from ss_src2 tablesample(1K); +select count(1) from ss_src2 tablesample(1K); + +-- row per split +explain +select key, value from ss_src2 tablesample(0 ROWS); +select key, value from ss_src2 tablesample(0 ROWS); + +explain +select count(1) from ss_src2 tablesample(10 ROWS); +select count(1) from ss_src2 tablesample(10 ROWS); + +explain +select count(1) from ss_src2 tablesample(100 ROWS); +select count(1) from ss_src2 tablesample(100 ROWS); + +set hive.fetch.task.conversion=more; +select key from ss_src2 tablesample(200B); +select key from ss_src2 tablesample(10 ROWS); + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +-- ROW type works with other input formats (others, don't) +select count(1) from ss_src2 tablesample(10 ROWS); + +--HIVE-5061 row sampling in sub-query +select * from (select * from src TABLESAMPLE (1 ROWS)) x; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats0.q b/src/test/hive/ql/src/test/queries/clientpositive/stats0.q new file mode 100644 index 0000000000000..48b5c52600ec4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats0.q @@ -0,0 +1,70 @@ +set hive.stats.autogather=true; +set datanucleus.cache.collections=false; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +CREATE TABLE stats_non_partitioned (key string, value string); + +explain extended +insert overwrite table stats_non_partitioned +select * from src; + +insert overwrite table stats_non_partitioned +select * from src; + +desc extended stats_non_partitioned; + +select * from stats_non_partitioned; + + +CREATE TABLE stats_partitioned(key string, value string) partitioned by (ds string); + +explain +insert overwrite table stats_partitioned partition (ds='1') +select * from src; + +insert overwrite table stats_partitioned partition (ds='1') +select * from src; + +show partitions stats_partitioned; +select * from stats_partitioned where ds is not null; + +describe extended stats_partitioned partition (ds='1'); +describe extended stats_partitioned; + + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; + +drop table stats_non_partitioned; +drop table stats_partitioned; + +CREATE TABLE stats_non_partitioned (key string, value string); + +explain extended +insert overwrite table stats_non_partitioned +select * from src; + +insert overwrite table stats_non_partitioned +select * from src; + +desc extended stats_non_partitioned; + +select * from stats_non_partitioned; + + +CREATE TABLE stats_partitioned(key string, value string) partitioned by (ds string); + +explain +insert overwrite table stats_partitioned partition (ds='1') +select * from src; + +insert overwrite table stats_partitioned partition (ds='1') +select * from src; + +show partitions stats_partitioned; +select * from stats_partitioned where ds is not null; + +describe extended stats_partitioned partition (ds='1'); +describe extended stats_partitioned; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats1.q b/src/test/hive/ql/src/test/queries/clientpositive/stats1.q new file mode 100644 index 0000000000000..0b783de153b29 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats1.q @@ -0,0 +1,30 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.map.aggr=true; + +create table tmptable(key string, value string); + +EXPLAIN +INSERT OVERWRITE TABLE tmptable +SELECT unionsrc.key, unionsrc.value +FROM (SELECT 'tst1' AS key, cast(count(1) AS string) AS value FROM src s1 + UNION ALL + SELECT s2.key AS key, s2.value AS value FROM src1 s2) unionsrc; + +INSERT OVERWRITE TABLE tmptable +SELECT unionsrc.key, unionsrc.value +FROM (SELECT 'tst1' AS key, cast(count(1) AS string) AS value FROM src s1 + UNION ALL + SELECT s2.key AS key, s2.value AS value FROM src1 s2) unionsrc; + +SELECT * FROM tmptable x SORT BY x.key, x.value; + +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; +DESCRIBE FORMATTED tmptable; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats10.q b/src/test/hive/ql/src/test/queries/clientpositive/stats10.q new file mode 100644 index 0000000000000..a3f375e5a460a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats10.q @@ -0,0 +1,28 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=true; +set hive.enforce.bucketing = true; +set hive.exec.reducers.max = 1; + +CREATE TABLE bucket3_1(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS; + +explain +insert overwrite table bucket3_1 partition (ds='1') +select * from src; + +insert overwrite table bucket3_1 partition (ds='1') +select * from src; + +insert overwrite table bucket3_1 partition (ds='1') +select * from src; + +insert overwrite table bucket3_1 partition (ds='2') +select * from src; + +select * from bucket3_1 tablesample (bucket 1 out of 2) s where ds = '1' order by key; + +explain analyze table bucket3_1 partition (ds) compute statistics; +analyze table bucket3_1 partition (ds) compute statistics; + +describe formatted bucket3_1 partition (ds='1'); +describe formatted bucket3_1 partition (ds='2'); +describe formatted bucket3_1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats11.q b/src/test/hive/ql/src/test/queries/clientpositive/stats11.q new file mode 100644 index 0000000000000..6618c913ea700 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats11.q @@ -0,0 +1,93 @@ +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; + +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'); + +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'); +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'); +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'); +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'); + +create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); +create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); + +set hive.optimize.bucketmapjoin = true; +create table bucketmapjoin_tmp_result (key string , value1 string, value2 string); + +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; + +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(b)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; + + +set hive.optimize.bucketmapjoin = true; +explain extended +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; + + +insert overwrite table bucketmapjoin_hash_result_1 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +set hive.optimize.bucketmapjoin = false; +insert overwrite table bucketmapjoin_tmp_result +select /*+mapjoin(a)*/ a.key, a.value, b.value +from srcbucket_mapjoin a join srcbucket_mapjoin_part b +on a.key=b.key where b.ds="2008-04-08"; + +select count(1) from bucketmapjoin_tmp_result; +insert overwrite table bucketmapjoin_hash_result_2 +select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result; + +select a.key-b.key, a.value1-b.value1, a.value2-b.value2 +from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b +on a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats12.q b/src/test/hive/ql/src/test/queries/clientpositive/stats12.q new file mode 100644 index 0000000000000..c1db34618e753 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats12.q @@ -0,0 +1,19 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +create table analyze_srcpart like srcpart; +insert overwrite table analyze_srcpart partition (ds, hr) select * from srcpart where ds is not null; + +explain extended +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr) compute statistics; + +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr) compute statistics; + +desc formatted analyze_srcpart; +desc formatted analyze_srcpart partition (ds='2008-04-08', hr=11); +desc formatted analyze_srcpart partition (ds='2008-04-08', hr=12); +desc formatted analyze_srcpart partition (ds='2008-04-09', hr=11); +desc formatted analyze_srcpart partition (ds='2008-04-09', hr=12); + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats13.q b/src/test/hive/ql/src/test/queries/clientpositive/stats13.q new file mode 100644 index 0000000000000..beb7bfa42e10f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats13.q @@ -0,0 +1,22 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +create table analyze_srcpart like srcpart; +insert overwrite table analyze_srcpart partition (ds, hr) select * from srcpart where ds is not null; + +explain extended +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=11) compute statistics; + +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=11) compute statistics; + +desc formatted analyze_srcpart; +desc formatted analyze_srcpart partition (ds='2008-04-08', hr=11); +desc formatted analyze_srcpart partition (ds='2008-04-08', hr=12); +desc formatted analyze_srcpart partition (ds='2008-04-09', hr=11); +desc formatted analyze_srcpart partition (ds='2008-04-09', hr=12); + +create table analyze_srcpart2 like analyze_srcpart; + +desc formatted analyze_srcpart2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats14.q b/src/test/hive/ql/src/test/queries/clientpositive/stats14.q new file mode 100644 index 0000000000000..8e74df535d086 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats14.q @@ -0,0 +1,26 @@ +set datanucleus.cache.collections=false; + +create table stats_src like src; +insert overwrite table stats_src select * from src; +analyze table stats_src compute statistics; +desc formatted stats_src; + +create table stats_part like srcpart; + +insert overwrite table stats_part partition (ds='2010-04-08', hr = '11') select key, value from src; +insert overwrite table stats_part partition (ds='2010-04-08', hr = '12') select key, value from src; + +analyze table stats_part partition(ds='2010-04-08', hr='11') compute statistics; +analyze table stats_part partition(ds='2010-04-08', hr='12') compute statistics; + +insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src; + +desc formatted stats_part; +desc formatted stats_part partition (ds='2010-04-08', hr = '11'); +desc formatted stats_part partition (ds='2010-04-08', hr = '12'); + +analyze table stats_part partition(ds, hr) compute statistics; +desc formatted stats_part; + +drop table stats_src; +drop table stats_part; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats15.q b/src/test/hive/ql/src/test/queries/clientpositive/stats15.q new file mode 100644 index 0000000000000..9a557c6708544 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats15.q @@ -0,0 +1,27 @@ +set datanucleus.cache.collections=false; +set hive.stats.collect.uncompressedsize=false; + +create table stats_src like src; +insert overwrite table stats_src select * from src; +analyze table stats_src compute statistics; +desc formatted stats_src; + +create table stats_part like srcpart; + +insert overwrite table stats_part partition (ds='2010-04-08', hr = '11') select key, value from src; +insert overwrite table stats_part partition (ds='2010-04-08', hr = '12') select key, value from src; + +analyze table stats_part partition(ds='2010-04-08', hr='11') compute statistics; +analyze table stats_part partition(ds='2010-04-08', hr='12') compute statistics; + +insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src; + +desc formatted stats_part; +desc formatted stats_part partition (ds='2010-04-08', hr = '11'); +desc formatted stats_part partition (ds='2010-04-08', hr = '12'); + +analyze table stats_part partition(ds, hr) compute statistics; +desc formatted stats_part; + +drop table stats_src; +drop table stats_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats16.q b/src/test/hive/ql/src/test/queries/clientpositive/stats16.q new file mode 100644 index 0000000000000..0557ce6e868c4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats16.q @@ -0,0 +1,12 @@ +set hive.stats.autogather=false; + +drop table stats16; + +create table stats16 (key int, value string); +desc formatted stats16; + +insert into table stats16 select * from src; +analyze table stats16 compute statistics; +desc formatted stats16; + +drop table stats16; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats18.q b/src/test/hive/ql/src/test/queries/clientpositive/stats18.q new file mode 100644 index 0000000000000..425de64c26e83 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats18.q @@ -0,0 +1,21 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.map.aggr=true; + +create table stats_part like srcpart; + +insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src; + +-- Load a file into a existing partition +-- Some stats (numFiles, totalSize) should be updated correctly +-- 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'); + +desc formatted stats_part partition (ds='2010-04-08', hr='13'); + +drop table stats_src; +drop table stats_part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats19.q b/src/test/hive/ql/src/test/queries/clientpositive/stats19.q new file mode 100644 index 0000000000000..da4af9655d16a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats19.q @@ -0,0 +1,105 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=true; +set hive.stats.reliable=true; +set hive.stats.dbclass=dummy; +set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; +set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.KeyVerifyingStatsAggregator; + +-- Note, its important that the partitions created below have a name greater than 16 characters in +-- length since KeyVerifyingStatsAggregator depends on checking that a keyPrefix is hashed by the +-- length of the keyPrefix, having a partition name greather than 16 characters guarantees no false +-- positives. + +create table stats_part like srcpart; + +set hive.stats.key.prefix.max.length=0; + +-- The stats key should be hashed since the max length is too small +insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src; + +set hive.stats.key.prefix.max.length=200; + +-- The stats key should not be hashed since the max length is large enough +insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src; + +set hive.stats.key.prefix.max.length=-1; + +-- The stats key should not be hashed since negative values should imply hashing is turned off +insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src; + +set hive.stats.dbclass=jdbc:derby; +set hive.stats.default.publisher=; +set hive.stats.default.aggregator=; + +set hive.stats.key.prefix.max.length=0; + +-- Run the tests again and verify the stats are correct, this should verify that the stats publisher +-- is hashing as well where appropriate + +-- The stats key should be hashed since the max length is too small +insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src; + +desc formatted stats_part partition (ds='2010-04-08', hr = '13'); + +set hive.stats.key.prefix.max.length=200; + +-- The stats key should not be hashed since the max length is large enough +insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src; + +desc formatted stats_part partition (ds='2010-04-08', hr = '13'); + +set hive.stats.key.prefix.max.length=-1; + +-- The stats key should not be hashed since negative values should imply hashing is turned off +insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src; + +desc formatted stats_part partition (ds='2010-04-08', hr = '13'); + + +set hive.stats.dbclass=dummy; +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; + +-- Do the same for dynamic partitions + +-- The stats key should be hashed since the max length is too small +insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src; + +set hive.stats.key.prefix.max.length=200; + +-- The stats key should not be hashed since the max length is large enough +insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src; + +set hive.stats.key.prefix.max.length=-1; + +-- The stats key should not be hashed since negative values should imply hashing is turned off +insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src; + +set hive.stats.dbclass=jdbc:derby; +set hive.stats.default.publisher=; +set hive.stats.default.aggregator=; + +set hive.stats.key.prefix.max.length=0; + +-- Run the tests again and verify the stats are correct, this should verify that the stats publisher +-- is hashing as well where appropriate + +-- The stats key should be hashed since the max length is too small +insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src; + +desc formatted stats_part partition (ds='2010-04-08', hr = '13'); + +set hive.stats.key.prefix.max.length=200; + +-- The stats key should not be hashed since the max length is large enough +insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src; + +desc formatted stats_part partition (ds='2010-04-08', hr = '13'); + +set hive.stats.key.prefix.max.length=-1; + +-- The stats key should not be hashed since negative values should imply hashing is turned off +insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src; + +desc formatted stats_part partition (ds='2010-04-08', hr = '13'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats2.q b/src/test/hive/ql/src/test/queries/clientpositive/stats2.q new file mode 100644 index 0000000000000..d6bb258307d8e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats2.q @@ -0,0 +1,21 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.merge.mapfiles=false; + +create table analyze_t1 like srcpart; + + +explain +insert overwrite table analyze_t1 partition (ds, hr) select * from srcpart where ds is not null; + +insert overwrite table analyze_t1 partition (ds, hr) select * from srcpart where ds is not null; + +desc formatted analyze_t1; + +explain analyze table analyze_t1 partition (ds, hr) compute statistics; + +analyze table analyze_t1 partition (ds, hr) compute statistics; + +describe formatted analyze_t1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats20.q b/src/test/hive/ql/src/test/queries/clientpositive/stats20.q new file mode 100644 index 0000000000000..59701bde771dc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats20.q @@ -0,0 +1,16 @@ +set hive.stats.autogather=true; +set datanucleus.cache.collections=false; + +set hive.stats.collect.rawdatasize=true; +CREATE TABLE stats_partitioned(key string, value string) partitioned by (ds string); +insert overwrite table stats_partitioned partition (ds='1') +select * from src; +-- rawDataSize is 5312 after config is turned on +describe formatted stats_partitioned; + +set hive.stats.collect.rawdatasize=false; +insert overwrite table stats_partitioned partition (ds='1') +select * from src; +-- rawDataSize is 0 after config is turned off +describe formatted stats_partitioned; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats3.q b/src/test/hive/ql/src/test/queries/clientpositive/stats3.q new file mode 100644 index 0000000000000..5962348d9c317 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats3.q @@ -0,0 +1,32 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=true; +drop table hive_test_src; +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 ; + +desc formatted 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 ; +select * from hive_test_dst where pcol1='test_part' and pcol2='test_Part'; + +select count(1) from hive_test_dst; + +insert overwrite table hive_test_dst partition ( pCol1='test_part', pcol2='test_Part') select col1 from hive_test_src ; +select * from hive_test_dst where pcol1='test_part' and pcol2='test_part'; + +select count(1) from hive_test_dst; + +select * from hive_test_dst where pcol1='test_part'; +select * from hive_test_dst where pcol1='test_part' and pcol2='test_part'; +select * from hive_test_dst where pcol1='test_Part'; + +describe formatted hive_test_dst; + +drop table hive_test_src; +drop table hive_test_dst; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats4.q b/src/test/hive/ql/src/test/queries/clientpositive/stats4.q new file mode 100644 index 0000000000000..62580042d4ded --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats4.q @@ -0,0 +1,40 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=true; + +show partitions srcpart; + +drop table nzhang_part1; +drop table nzhang_part2; + +create table if not exists nzhang_part1 like srcpart; +create table if not exists nzhang_part2 like srcpart; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +explain +from srcpart +insert overwrite table nzhang_part1 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08' +insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08'; + +from srcpart +insert overwrite table nzhang_part1 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08' +insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08'; + + +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; + +describe formatted nzhang_part1 partition(ds='2008-04-08',hr=11); +describe formatted nzhang_part1 partition(ds='2008-04-08',hr=12); +describe formatted nzhang_part2 partition(ds='2008-12-31',hr=11); +describe formatted nzhang_part2 partition(ds='2008-12-31',hr=12); + +describe formatted nzhang_part1; +describe formatted nzhang_part2; + +drop table nzhang_part1; +drop table nzhang_part2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats5.q b/src/test/hive/ql/src/test/queries/clientpositive/stats5.q new file mode 100644 index 0000000000000..6b5d1384e065a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats5.q @@ -0,0 +1,10 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; + +create table analyze_src as select * from src; + +explain analyze table analyze_src compute statistics; + +analyze table analyze_src compute statistics; + +describe formatted analyze_src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats6.q b/src/test/hive/ql/src/test/queries/clientpositive/stats6.q new file mode 100644 index 0000000000000..bef6e8809ef97 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats6.q @@ -0,0 +1,17 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +create table analyze_srcpart like srcpart; +insert overwrite table analyze_srcpart partition (ds, hr) select * from srcpart where ds is not null; + +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=11) compute statistics; +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=12) compute statistics; + +describe formatted analyze_srcpart PARTITION(ds='2008-04-08',hr=11); +describe formatted analyze_srcpart PARTITION(ds='2008-04-08',hr=12); +describe formatted analyze_srcpart PARTITION(ds='2008-04-09',hr=11); +describe formatted analyze_srcpart PARTITION(ds='2008-04-09',hr=12); + +describe formatted analyze_srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats7.q b/src/test/hive/ql/src/test/queries/clientpositive/stats7.q new file mode 100644 index 0000000000000..19d0e1426930e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats7.q @@ -0,0 +1,16 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +create table analyze_srcpart like srcpart; +insert overwrite table analyze_srcpart partition (ds, hr) select * from srcpart where ds is not null; + +explain analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr) compute statistics; + +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr) compute statistics; + +describe formatted analyze_srcpart PARTITION(ds='2008-04-08',hr=11); +describe formatted analyze_srcpart PARTITION(ds='2008-04-08',hr=12); + +describe formatted analyze_srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats8.q b/src/test/hive/ql/src/test/queries/clientpositive/stats8.q new file mode 100644 index 0000000000000..46002caf4a0ec --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats8.q @@ -0,0 +1,33 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +create table analyze_srcpart like srcpart; +insert overwrite table analyze_srcpart partition (ds, hr) select * from srcpart where ds is not null; + +explain analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=11) compute statistics; +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=11) compute statistics; +describe formatted analyze_srcpart PARTITION(ds='2008-04-08',hr=11); +describe formatted analyze_srcpart; + +explain analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=12) compute statistics; +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=12) compute statistics; +describe formatted analyze_srcpart PARTITION(ds='2008-04-08',hr=12); + +explain analyze table analyze_srcpart PARTITION(ds='2008-04-09',hr=11) compute statistics; +analyze table analyze_srcpart PARTITION(ds='2008-04-09',hr=11) compute statistics; +describe formatted analyze_srcpart PARTITION(ds='2008-04-09',hr=11); + +explain analyze table analyze_srcpart PARTITION(ds='2008-04-09',hr=12) compute statistics; +analyze table analyze_srcpart PARTITION(ds='2008-04-09',hr=12) compute statistics; +describe formatted analyze_srcpart PARTITION(ds='2008-04-09',hr=12); + +explain analyze table analyze_srcpart PARTITION(ds, hr) compute statistics; +analyze table analyze_srcpart PARTITION(ds, hr) compute statistics; + +describe formatted analyze_srcpart PARTITION(ds='2008-04-08',hr=11); +describe formatted analyze_srcpart PARTITION(ds='2008-04-08',hr=12); +describe formatted analyze_srcpart PARTITION(ds='2008-04-09',hr=11); +describe formatted analyze_srcpart PARTITION(ds='2008-04-09',hr=12); +describe formatted analyze_srcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats9.q b/src/test/hive/ql/src/test/queries/clientpositive/stats9.q new file mode 100644 index 0000000000000..48b20443a9f1b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats9.q @@ -0,0 +1,9 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; + +create table analyze_srcbucket like srcbucket; +insert overwrite table analyze_srcbucket select * from srcbucket; + +explain analyze table analyze_srcbucket compute statistics; +analyze table analyze_srcbucket compute statistics; +describe formatted analyze_srcbucket; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q b/src/test/hive/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q new file mode 100644 index 0000000000000..4e7d3dc547a30 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q @@ -0,0 +1,29 @@ +-- 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) +-- 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.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; + +set hive.test.dummystats.aggregator=connect; + +INSERT OVERWRITE TABLE tmptable select * from src; +select count(1) from tmptable; + +set hive.test.dummystats.aggregator=closeConnection; +INSERT OVERWRITE TABLE tmptable select * from src; +select count(1) from tmptable; + +set hive.test.dummystats.aggregator=cleanUp; +INSERT OVERWRITE TABLE tmptable select * from src; +select count(1) from tmptable; + +set hive.stats.default.aggregator=""; +INSERT OVERWRITE TABLE tmptable select * from src; +select count(1) from tmptable; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q b/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q new file mode 100644 index 0000000000000..38ee6f7f4f3fb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q @@ -0,0 +1,13 @@ +-- This test verifies writing a query using dynamic partitions +-- which results in no partitions actually being created with +-- hive.stats.reliable set to true + +create table tmptable(key string) partitioned by (part string); + +set hive.stats.autogather=true; +set hive.stats.reliable=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +explain insert overwrite table tmptable partition (part) select key, value from src where key = 'no_such_value'; + +insert overwrite table tmptable partition (part) select key, value from src where key = 'no_such_value'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_partition.q b/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_partition.q new file mode 100644 index 0000000000000..305a97c929230 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_partition.q @@ -0,0 +1,11 @@ +-- This test verifies that writing an empty partition succeeds when +-- hive.stats.reliable is set to true. + +create table tmptable(key string, value string) partitioned by (part string); + +set hive.stats.autogather=true; +set hive.stats.reliable=true; + +insert overwrite table tmptable partition (part = '1') select * from src where key = 'no_such_value'; + +describe formatted tmptable partition (part = '1'); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_1.q b/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_1.q new file mode 100644 index 0000000000000..02b4c2aabf039 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_1.q @@ -0,0 +1,37 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +-- test analyze table ... compute statistics noscan + +-- 1. test full spec +create table analyze_srcpart like srcpart; +insert overwrite table analyze_srcpart partition (ds, hr) select * from srcpart where ds is not null; +explain +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=11) compute statistics noscan; +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=11) compute statistics noscan; +analyze table analyze_srcpart PARTITION(ds='2008-04-08',hr=12) compute statistics noscan; +-- confirm result +describe formatted analyze_srcpart PARTITION(ds='2008-04-08',hr=11); +describe formatted analyze_srcpart PARTITION(ds='2008-04-08',hr=12); +describe formatted analyze_srcpart PARTITION(ds='2008-04-09',hr=11); +describe formatted analyze_srcpart PARTITION(ds='2008-04-09',hr=12); +describe formatted analyze_srcpart; +drop table analyze_srcpart; + +-- 2. test partial spec +create table analyze_srcpart_partial like srcpart; +insert overwrite table analyze_srcpart_partial partition (ds, hr) select * from srcpart where ds is not null; +explain +analyze table analyze_srcpart_partial PARTITION(ds='2008-04-08') compute statistics noscan; +analyze table analyze_srcpart_partial PARTITION(ds='2008-04-08') compute statistics noscan; +-- confirm result +describe formatted analyze_srcpart_partial PARTITION(ds='2008-04-08',hr=11); +describe formatted analyze_srcpart_partial PARTITION(ds='2008-04-08',hr=12); +describe formatted analyze_srcpart_partial PARTITION(ds='2008-04-09',hr=11); +describe formatted analyze_srcpart_partial PARTITION(ds='2008-04-09',hr=12); +drop table analyze_srcpart_partial; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_2.q b/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_2.q new file mode 100644 index 0000000000000..c934fb2051258 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_2.q @@ -0,0 +1,32 @@ + +-- 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'; +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; +drop table anaylyze_external; + +-- 2 test partition +-- prepare data +create table texternal(key string, val string) partitioned by (insertdate string); +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/texternal/2008-01-01; +alter table texternal add partition (insertdate='2008-01-01') location 'pfile://${system:test.tmp.dir}/texternal/2008-01-01'; +from src insert overwrite table texternal partition (insertdate='2008-01-01') select *; +select count(*) from texternal where insertdate='2008-01-01'; +-- create external table +CREATE EXTERNAL TABLE anaylyze_external (key string, val string) partitioned by (insertdate string) LOCATION "pfile://${system:test.tmp.dir}/texternal"; +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'); +dfs -rmr ${system:test.tmp.dir}/texternal; +drop table anaylyze_external; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1.q b/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1.q new file mode 100644 index 0000000000000..cdf92e44cf676 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1.q @@ -0,0 +1,37 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set mapred.min.split.size=256; +set mapred.min.split.size.per.node=256; +set mapred.min.split.size.per.rack=256; +set mapred.max.split.size=256; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S) +-- This test uses mapred.max.split.size/mapred.max.split.size for controlling +-- number of input splits, which is not effective in hive 0.20. +-- stats_partscan_1_23.q is the same test with this but has different result. + +-- test analyze table ... compute statistics partialscan + +-- 1. prepare data +CREATE table analyze_srcpart_partial_scan (key STRING, value STRING) +partitioned by (ds string, hr string) +stored as rcfile; +insert overwrite table analyze_srcpart_partial_scan partition (ds, hr) select * from srcpart where ds is not null; +describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11); + +set hive.stats.autogather=true; + +-- 2. partialscan +explain +analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; +analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; + +-- 3. confirm result +describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11); +describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-09',hr=11); +drop table analyze_srcpart_partial_scan; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1_23.q b/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1_23.q new file mode 100644 index 0000000000000..1e5f360b20cbb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1_23.q @@ -0,0 +1,37 @@ +set datanucleus.cache.collections=false; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set mapred.min.split.size=256; +set mapred.min.split.size.per.node=256; +set mapred.min.split.size.per.rack=256; +set mapred.max.split.size=256; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- This test uses mapred.max.split.size/mapred.max.split.size for controlling +-- number of input splits. +-- stats_partscan_1.q is the same test with this but has different result. + +-- test analyze table ... compute statistics partialscan + +-- 1. prepare data +CREATE table analyze_srcpart_partial_scan (key STRING, value STRING) +partitioned by (ds string, hr string) +stored as rcfile; +insert overwrite table analyze_srcpart_partial_scan partition (ds, hr) select * from srcpart where ds is not null; +describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11); + +set hive.stats.autogather=true; + +-- 2. partialscan +explain +analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; +analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; + +-- 3. confirm result +describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11); +describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-09',hr=11); +drop table analyze_srcpart_partial_scan; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_publisher_error_1.q b/src/test/hive/ql/src/test/queries/clientpositive/stats_publisher_error_1.q new file mode 100644 index 0000000000000..6d383f213d1be --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/stats_publisher_error_1.q @@ -0,0 +1,29 @@ +-- In this test, there is a dummy stats publisher which throws an error when various +-- methods are called (as indicated by the parameter hive.test.dummystats.publisher) +-- 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 publisher +-- 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.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; + +set hive.test.dummystats.publisher=connect; + +INSERT OVERWRITE TABLE tmptable select * from src; +select count(1) from tmptable; + +set hive.test.dummystats.publisher=publishStat; +INSERT OVERWRITE TABLE tmptable select * from src; +select count(1) from tmptable; + +set hive.test.dummystats.publisher=closeConnection; +INSERT OVERWRITE TABLE tmptable select * from src; +select count(1) from tmptable; + +set hive.stats.default.publisher=""; +INSERT OVERWRITE TABLE tmptable select * from src; +select count(1) from tmptable; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/str_to_map.q b/src/test/hive/ql/src/test/queries/clientpositive/str_to_map.q new file mode 100644 index 0000000000000..c3b206bba6325 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/str_to_map.q @@ -0,0 +1,26 @@ +desc function str_to_map; +desc function extended str_to_map; + +explain select str_to_map('a=1,b=2,c=3',',','=')['a'] from src limit 3; +select str_to_map('a=1,b=2,c=3',',','=')['a'] from src limit 3; + +explain select str_to_map('a:1,b:2,c:3') from src limit 3; +select str_to_map('a:1,b:2,c:3') from src limit 3; + +explain select str_to_map('a:1,b:2,c:3',',',':') from src limit 3; +select str_to_map('a:1,b:2,c:3',',',':') from src limit 3; + +explain select str_to_map(t.ss,',',':')['a'] +from (select transform('a:1,b:2,c:3') using 'cat' as (ss) from src) t +limit 3; +select str_to_map(t.ss,',',':')['a'] +from (select transform('a:1,b:2,c:3') using 'cat' as (ss) from src) t +limit 3; + + +drop table tbl_s2m; +create table tbl_s2m as select 'ABC=CC_333=444' as t from src limit 3; + +select str_to_map(t,'_','=')['333'] from tbl_s2m; + +drop table tbl_s2m; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/subq.q b/src/test/hive/ql/src/test/queries/clientpositive/subq.q new file mode 100644 index 0000000000000..3fb1558a90f0b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/subq.q @@ -0,0 +1,13 @@ +EXPLAIN +FROM ( + FROM src select src.* WHERE src.key < 100 +) unioninput +INSERT OVERWRITE DIRECTORY '../build/ql/test/data/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.*; + +dfs -cat ../build/ql/test/data/warehouse/union.out/*; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/subq2.q b/src/test/hive/ql/src/test/queries/clientpositive/subq2.q new file mode 100644 index 0000000000000..7e174da4586fd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/subq2.q @@ -0,0 +1,8 @@ +EXPLAIN +SELECT a.k, a.c +FROM (SELECT b.key as k, count(1) as c FROM src b GROUP BY b.key) a +WHERE a.k >= 90; + +SELECT a.k, a.c +FROM (SELECT b.key as k, count(1) as c FROM src b GROUP BY b.key) a +WHERE a.k >= 90; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/symlink_text_input_format.q b/src/test/hive/ql/src/test/queries/clientpositive/symlink_text_input_format.q new file mode 100644 index 0000000000000..bb9d6f34ed8ec --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/symlink_text_input_format.q @@ -0,0 +1,24 @@ + + +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; + +EXPLAIN SELECT * FROM symlink_text_input_format order by key, value; + +SELECT * FROM symlink_text_input_format order by key, value; + +EXPLAIN SELECT value FROM symlink_text_input_format order by value; + +SELECT value FROM symlink_text_input_format order by value; + +EXPLAIN SELECT count(1) FROM symlink_text_input_format; + +SELECT count(1) FROM symlink_text_input_format; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/table_access_keys_stats.q b/src/test/hive/ql/src/test/queries/clientpositive/table_access_keys_stats.q new file mode 100644 index 0000000000000..8b1a390149865 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/table_access_keys_stats.q @@ -0,0 +1,218 @@ +SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.CheckTableAccessHook; +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; + +CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; +CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; + +-- Simple group-by queries +SELECT key, count(1) FROM T1 GROUP BY key; +SELECT key, val, count(1) FROM T1 GROUP BY key, val; + +-- With subqueries and column aliases +SELECT key, count(1) FROM (SELECT key, val FROM T1) subq1 GROUP BY key; +SELECT k, count(1) FROM (SELECT key as k, val as v FROM T1) subq1 GROUP BY k; + +-- With constants +SELECT 1, key, count(1) FROM T1 GROUP BY 1, key; +SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val; +SELECT key, 1, val, 2, count(1) FROM T1 GROUP BY key, 1, val, 2; + +-- no mapping with functions +SELECT key, key + 1, count(1) FROM T1 GROUP BY key, key + 1; + +SELECT key + key, sum(cnt) from +(SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 +group by key + key; + +-- group by followed by union +SELECT * FROM ( +SELECT key, count(1) as c FROM T1 GROUP BY key + UNION ALL +SELECT key, count(1) as c FROM T1 GROUP BY key +) subq1; + +-- group by followed by a join +SELECT * FROM +(SELECT key, count(1) as c FROM T1 GROUP BY key) subq1 +JOIN +(SELECT key, count(1) as c FROM T1 GROUP BY key) subq2 +ON subq1.key = subq2.key; + +SELECT * FROM +(SELECT key, count(1) as c FROM T1 GROUP BY key) subq1 +JOIN +(SELECT key, val, count(1) as c FROM T1 GROUP BY key, val) subq2 +ON subq1.key = subq2.key +ORDER BY subq1.key ASC, subq1.c ASC, subq2.key ASC, subq2.val ASC, subq2.c ASC; + +-- constants from sub-queries should work fine +SELECT key, constant, val, count(1) from +(SELECT key, 1 as constant, val from T1) subq1 +group by key, constant, val; + +-- multiple levels of constants from sub-queries should work fine +SELECT key, constant3, val, count(1) FROM +( + SELECT key, constant AS constant2, val, 2 AS constant3 + FROM + ( + SELECT key, 1 AS constant, val + FROM T1 + ) subq +) subq2 +GROUP BY key, constant3, val; + +-- work with insert overwrite +FROM T1 +INSERT OVERWRITE TABLE T2 SELECT key, count(1) GROUP BY key, 1 +INSERT OVERWRITE TABLE T3 SELECT key, sum(val) GROUP BY key; + +-- simple joins +SELECT * +FROM T1 JOIN T2 +ON T1.key = t2.key +ORDER BY T1.key ASC, T1.val ASC; + +SELECT * +FROM T1 JOIN T2 +ON T1.key = T2.key AND T1.val = T2.val; + +-- map join +SELECT /*+ MAPJOIN(a) */ * +FROM T1 a JOIN T2 b +ON a.key = b.key; + +-- with constant in join condition +SELECT * +FROM T1 JOIN T2 +ON T1.key = T2.key AND T1.val = 3 and T2.val = 3; + +-- subqueries +SELECT * +FROM +( + SELECT val FROM T1 WHERE key = 5 +) subq1 +JOIN +( + SELECT val FROM T2 WHERE key = 6 +) subq2 +ON subq1.val = subq2.val; + +SELECT * +FROM +( + SELECT val FROM T1 WHERE key = 5 +) subq1 +JOIN +T2 +ON subq1.val = T2.val; + +-- with column aliases in subqueries +SELECT * +FROM +( + SELECT val as v FROM T1 WHERE key = 5 +) subq1 +JOIN +( + SELECT val FROM T2 WHERE key = 6 +) subq2 +ON subq1.v = subq2.val; + +-- with constants in subqueries +SELECT * +FROM +( + SELECT key, val FROM T1 +) subq1 +JOIN +( + SELECT key, 'teststring' as val FROM T2 +) subq2 +ON subq1.val = subq2.val AND subq1.key = subq2.key; + +-- multiple levels of constants in subqueries +SELECT * +FROM +( + SELECT key, val from + ( + SELECT key, 'teststring' as val from T1 + ) subq1 +) subq2 +JOIN +( + SELECT key, val FROM T2 +) subq3 +ON subq3.val = subq2.val AND subq3.key = subq2.key; + +-- no mapping on functions +SELECT * +FROM +( + SELECT key, val from T1 +) subq1 +JOIN +( + SELECT key, val FROM T2 +) subq2 +ON subq1.val = subq2.val AND subq1.key + 1 = subq2.key; + +-- join followed by group by +SELECT subq1.val, COUNT(*) +FROM +( + SELECT key, val FROM T1 +) subq1 +JOIN +( + SELECT key, 'teststring' as val FROM T2 +) subq2 +ON subq1.val = subq2.val AND subq1.key = subq2.key +GROUP BY subq1.val; + +-- join followed by union +SELECT * +FROM +( + SELECT subq1.val, COUNT(*) + FROM + ( + SELECT key, val FROM T1 + ) subq1 + JOIN + ( + SELECT key, 'teststring' as val FROM T2 + ) subq2 + ON subq1.val = subq2.val AND subq1.key = subq2.key + GROUP BY subq1.val + UNION ALL + SELECT val, COUNT(*) + FROM T3 + GROUP BY val +) subq4; + +-- join followed by join +SELECT * +FROM +( + SELECT subq1.val as val, COUNT(*) + FROM + ( + SELECT key, val FROM T1 + ) subq1 + JOIN + ( + SELECT key, 'teststring' as val FROM T2 + ) subq2 + ON subq1.val = subq2.val AND subq1.key = subq2.key + GROUP by subq1.val +) T4 +JOIN T3 +ON T3.val = T4.val; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/tablename_with_select.q b/src/test/hive/ql/src/test/queries/clientpositive/tablename_with_select.q new file mode 100644 index 0000000000000..c48231b66c7b8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/tablename_with_select.q @@ -0,0 +1,9 @@ + +CREATE TABLE tmp_select(a INT, b STRING); +DESCRIBE tmp_select; + +INSERT OVERWRITE TABLE tmp_select SELECT key, value FROM src; + +SELECT a, b FROM tmp_select ORDER BY a; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/test_boolean_whereclause.q b/src/test/hive/ql/src/test/queries/clientpositive/test_boolean_whereclause.q new file mode 100644 index 0000000000000..d2da5ac174a15 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/test_boolean_whereclause.q @@ -0,0 +1,9 @@ +create table if not exists test_boolean(dummy tinyint); +insert overwrite table test_boolean select 1 from src limit 1; + +SELECT 1 +FROM ( +SELECT TRUE AS flag +FROM test_boolean +) a +WHERE flag; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_1.q b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_1.q new file mode 100644 index 0000000000000..f2c3b596af44a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_1.q @@ -0,0 +1,72 @@ +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; +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; +select cast(t as int) from timestamp_1 limit 1; +select cast(t as bigint) from timestamp_1 limit 1; +select cast(t as float) from timestamp_1 limit 1; +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 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; +select cast(t as int) from timestamp_1 limit 1; +select cast(t as bigint) from timestamp_1 limit 1; +select cast(t as float) from timestamp_1 limit 1; +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 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; +select cast(t as int) from timestamp_1 limit 1; +select cast(t as bigint) from timestamp_1 limit 1; +select cast(t as float) from timestamp_1 limit 1; +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 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; +select cast(t as int) from timestamp_1 limit 1; +select cast(t as bigint) from timestamp_1 limit 1; +select cast(t as float) from timestamp_1 limit 1; +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 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; +select cast(t as int) from timestamp_1 limit 1; +select cast(t as bigint) from timestamp_1 limit 1; +select cast(t as float) from timestamp_1 limit 1; +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 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; +select cast(t as int) from timestamp_1 limit 1; +select cast(t as bigint) from timestamp_1 limit 1; +select cast(t as float) from timestamp_1 limit 1; +select cast(t as double) from timestamp_1 limit 1; +select cast(t as string) from timestamp_1 limit 1; + +drop table timestamp_1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_2.q b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_2.q new file mode 100644 index 0000000000000..b93208f48c453 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_2.q @@ -0,0 +1,72 @@ +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; +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; +select cast(t as int) from timestamp_2 limit 1; +select cast(t as bigint) from timestamp_2 limit 1; +select cast(t as float) from timestamp_2 limit 1; +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 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; +select cast(t as int) from timestamp_2 limit 1; +select cast(t as bigint) from timestamp_2 limit 1; +select cast(t as float) from timestamp_2 limit 1; +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 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; +select cast(t as int) from timestamp_2 limit 1; +select cast(t as bigint) from timestamp_2 limit 1; +select cast(t as float) from timestamp_2 limit 1; +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 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; +select cast(t as int) from timestamp_2 limit 1; +select cast(t as bigint) from timestamp_2 limit 1; +select cast(t as float) from timestamp_2 limit 1; +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 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; +select cast(t as int) from timestamp_2 limit 1; +select cast(t as bigint) from timestamp_2 limit 1; +select cast(t as float) from timestamp_2 limit 1; +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 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; +select cast(t as int) from timestamp_2 limit 1; +select cast(t as bigint) from timestamp_2 limit 1; +select cast(t as float) from timestamp_2 limit 1; +select cast(t as double) from timestamp_2 limit 1; +select cast(t as string) from timestamp_2 limit 1; + +drop table timestamp_2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_3.q b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_3.q new file mode 100644 index 0000000000000..cda724f9e8f43 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_3.q @@ -0,0 +1,17 @@ +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; +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; +select cast(t as int) from timestamp_3 limit 1; +select cast(t as bigint) from timestamp_3 limit 1; +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; + +drop table timestamp_3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_comparison.q b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_comparison.q new file mode 100644 index 0000000000000..f64ae48b85db7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_comparison.q @@ -0,0 +1,28 @@ + +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) < + cast('2011-05-06 07:08:09' as timestamp) from src limit 1; + +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) <> + cast('2011-05-06 07:08:09' as timestamp) from src limit 1; + +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) <= + cast('2011-05-06 07:08:09' as timestamp) from src limit 1; + +select cast('2011-05-06 07:08:09' as timestamp) >= + cast('2011-05-06 07:08:09.1' as timestamp) from src limit 1; + +select cast('2011-05-06 07:08:09' as timestamp) < + cast('2011-05-06 07:08:09.1' as timestamp) from src limit 1; + +select cast('2011-05-06 07:08:09.1000' as timestamp) = + cast('2011-05-06 07:08:09.1' as timestamp) from src limit 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_lazy.q b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_lazy.q new file mode 100644 index 0000000000000..7a1005295eb28 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_lazy.q @@ -0,0 +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; + +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/src/test/hive/ql/src/test/queries/clientpositive/timestamp_null.q b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_null.q new file mode 100644 index 0000000000000..efd5bc4b78bf4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_null.q @@ -0,0 +1,7 @@ +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; + +SELECT * FROM timestamp_null LIMIT 1; + +SELECT t1 FROM timestamp_null LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_udf.q b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_udf.q new file mode 100644 index 0000000000000..2620acefee7f7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/timestamp_udf.q @@ -0,0 +1,58 @@ +drop table timestamp_udf; +drop table timestamp_udf_string; + +create table timestamp_udf (t timestamp); +create table timestamp_udf_string (t string); +from src + insert overwrite table timestamp_udf + select '2011-05-06 07:08:09.1234567' limit 1 + insert overwrite table timestamp_udf_string + select '2011-05-06 07:08:09.1234567' limit 1; + +-- Test UDFs with Timestamp input +select unix_timestamp(t), year(t), month(t), day(t), dayofmonth(t), + weekofyear(t), hour(t), minute(t), second(t), to_date(t) + from timestamp_udf; + +select date_add(t, 5), date_sub(t, 10) + from timestamp_udf; + +select datediff(t, t), datediff(t, '2002-03-21'), datediff('2002-03-21', t) + from timestamp_udf; + +select from_utc_timestamp(t, 'America/Chicago') + from timestamp_udf; + +select to_utc_timestamp(t, 'America/Chicago') + from timestamp_udf; + +select t, from_utc_timestamp(t, 'America/Chicago') + from timestamp_udf; + +select t, from_utc_timestamp(t, 'America/Chicago'), t, from_utc_timestamp(t, 'America/Chicago') + from timestamp_udf; + +select t, to_utc_timestamp(t, 'America/Chicago') + from timestamp_udf; + +select t, to_utc_timestamp(t, 'America/Chicago'), t, to_utc_timestamp(t, 'America/Chicago') + from timestamp_udf; + +-- Test UDFs with string input +select unix_timestamp(t), year(t), month(t), day(t), dayofmonth(t), + weekofyear(t), hour(t), minute(t), second(t), to_date(t) + from timestamp_udf_string; + +select date_add(t, 5), date_sub(t, 10) from timestamp_udf_string; + +select datediff(t, t), datediff(t, '2002-03-21'), datediff('2002-03-21', t) + from timestamp_udf_string; + +select from_utc_timestamp(t, 'America/Chicago') + from timestamp_udf_string; + +select to_utc_timestamp(t, 'America/Chicago') + from timestamp_udf_string; + +drop table timestamp_udf; +drop table timestamp_udf_string; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/touch.q b/src/test/hive/ql/src/test/queries/clientpositive/touch.q new file mode 100644 index 0000000000000..8a661ef12104e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/touch.q @@ -0,0 +1,17 @@ +drop table tstsrc; +drop table tstsrcpart; + +create table tstsrc like src; +insert overwrite table tstsrc select key, value from src; + +create table tstsrcpart like srcpart; +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12') +select key, value from srcpart where ds='2008-04-08' and hr='12'; + + +ALTER TABLE tstsrc TOUCH; +ALTER TABLE tstsrcpart TOUCH; +ALTER TABLE tstsrcpart TOUCH PARTITION (ds='2008-04-08', hr='12'); + +drop table tstsrc; +drop table tstsrcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/transform1.q b/src/test/hive/ql/src/test/queries/clientpositive/transform1.q new file mode 100644 index 0000000000000..962077c2ca565 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/transform1.q @@ -0,0 +1,23 @@ + +create table transform1_t1(a string, b string); + +EXPLAIN +SELECT transform(*) USING 'cat' AS (col array) FROM transform1_t1; + +SELECT transform(*) USING 'cat' AS (col array) FROM transform1_t1; + + + + +create table transform1_t2(col array); + +insert overwrite table transform1_t2 +select array(1,2,3) from src limit 1; + +EXPLAIN +SELECT transform('0\0021\0022') USING 'cat' AS (col array) FROM transform1_t2; + +SELECT transform('0\0021\0022') USING 'cat' AS (col array) FROM transform1_t2; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/transform2.q b/src/test/hive/ql/src/test/queries/clientpositive/transform2.q new file mode 100644 index 0000000000000..ddbc567596410 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/transform2.q @@ -0,0 +1,2 @@ +-- Transform with a function that has many parameters +SELECT TRANSFORM(substr(key, 1, 2)) USING 'cat' FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr1.q b/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr1.q new file mode 100644 index 0000000000000..07bb4d1dd08e6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr1.q @@ -0,0 +1,19 @@ +set hive.optimize.ppd=true; + +EXPLAIN EXTENDED +FROM ( + FROM srcpart src + SELECT TRANSFORM(src.ds, src.key, src.value) + USING 'cat' AS (ds, tkey, tvalue) + CLUSTER BY tkey +) tmap +SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100 AND tmap.ds = '2008-04-08'; + +FROM ( + FROM srcpart src + SELECT TRANSFORM(src.ds, src.key, src.value) + USING 'cat' AS (ds, tkey, tvalue) + CLUSTER BY tkey +) tmap +SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100 AND tmap.ds = '2008-04-08'; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr2.q b/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr2.q new file mode 100644 index 0000000000000..85ef3acf824d4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr2.q @@ -0,0 +1,21 @@ +set hive.optimize.ppd=true; + +EXPLAIN EXTENDED +FROM ( + FROM srcpart src + SELECT TRANSFORM(src.ds, src.key, src.value) + USING 'cat' AS (ds, tkey, tvalue) + WHERE src.ds = '2008-04-08' + CLUSTER BY tkey +) tmap +SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + +FROM ( + FROM srcpart src + SELECT TRANSFORM(src.ds, src.key, src.value) + USING 'cat' AS (ds, tkey, tvalue) + WHERE src.ds = '2008-04-08' + CLUSTER BY tkey +) tmap +SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/truncate_column.q b/src/test/hive/ql/src/test/queries/clientpositive/truncate_column.q new file mode 100644 index 0000000000000..d756b47e464d0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/truncate_column.q @@ -0,0 +1,79 @@ +-- Tests truncating column(s) from a table, also tests that stats are updated + +CREATE TABLE test_tab (key STRING, value STRING) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' STORED AS RCFILE; + +set hive.stats.autogather=true; + +INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10; + +DESC FORMATTED test_tab; + +SELECT * FROM test_tab ORDER BY value; + +-- Truncate 1 column +TRUNCATE TABLE test_tab COLUMNS (key); + +DESC FORMATTED test_tab; + +-- First column should be null +SELECT * FROM test_tab ORDER BY value; + +-- Truncate multiple columns +INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10; + +TRUNCATE TABLE test_tab COLUMNS (key, value); + +DESC FORMATTED test_tab; + +-- Both columns should be null +SELECT * FROM test_tab ORDER BY value; + +-- Truncate columns again +TRUNCATE TABLE test_tab COLUMNS (key, value); + +DESC FORMATTED test_tab; + +-- Both columns should be null +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; + +DESC FORMATTED test_tab; + +SELECT * FROM test_tab ORDER BY value; + +-- Truncate 1 column +TRUNCATE TABLE test_tab COLUMNS (key); + +DESC FORMATTED test_tab; + +-- First column should be null +SELECT * FROM test_tab ORDER BY value; + +-- Truncate 2 columns +TRUNCATE TABLE test_tab COLUMNS (key, value); + +DESC FORMATTED test_tab; + +-- Both columns should be null +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; + +DESC FORMATTED test_tab_part PARTITION (part = '1'); + +SELECT * FROM test_tab_part WHERE part = '1' ORDER BY value; + +TRUNCATE TABLE test_tab_part PARTITION (part = '1') COLUMNS (key); + +DESC FORMATTED test_tab_part PARTITION (part = '1'); + +-- First column should be null +SELECT * FROM test_tab_part WHERE part = '1' ORDER BY value; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_buckets.q b/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_buckets.q new file mode 100644 index 0000000000000..a2ce21558cdd7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_buckets.q @@ -0,0 +1,23 @@ +-- Tests truncating columns from a bucketed table, table should remain bucketed + +CREATE TABLE test_tab (key STRING, value STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS RCFILE; + +set hive.enforce.bucketing=true; + +INSERT OVERWRITE TABLE test_tab SELECT * FROM src; + +-- Check how many rows there are in each bucket, there should be two rows +SELECT cnt FROM ( +SELECT INPUT__FILE__NAME file_name, count(*) cnt FROM +test_tab GROUP BY INPUT__FILE__NAME +ORDER BY file_name DESC)a; + +-- Truncate a column on which the table is not bucketed +TRUNCATE TABLE test_tab COLUMNS (value); + +-- Check how many rows there are in each bucket, this should produce the same rows as before +-- because truncate should not break bucketing +SELECT cnt FROM ( +SELECT INPUT__FILE__NAME file_name, count(*) cnt FROM +test_tab GROUP BY INPUT__FILE__NAME +ORDER BY file_name DESC)a; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q b/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q new file mode 100644 index 0000000000000..87289f0d3d619 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q @@ -0,0 +1,33 @@ +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; + +-- 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; + +set hive.optimize.listbucketing=true; +SELECT * FROM test_tab WHERE part = '1' AND key = '0'; + +TRUNCATE TABLE test_tab PARTITION (part ='1') COLUMNS (value); + +-- In the following select statements the list bucketing optimization should still be used +-- In both cases value should be null + +EXPLAIN EXTENDED SELECT * FROM test_tab WHERE part = '1' AND key = '484'; + +SELECT * FROM test_tab WHERE part = '1' AND key = '484'; + +EXPLAIN EXTENDED SELECT * FROM test_tab WHERE part = '1' AND key = '0'; + +SELECT * FROM test_tab WHERE part = '1' AND key = '0'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_merge.q b/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_merge.q new file mode 100644 index 0000000000000..a7aab357eaf2f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_merge.q @@ -0,0 +1,20 @@ +-- Tests truncating a column from a table with multiple files, then merging those files + +CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; + +INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 5; + +INSERT INTO TABLE test_tab SELECT * FROM src LIMIT 5; + +-- The value should be 2 indicating the table has 2 files +SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM test_tab; + +TRUNCATE TABLE test_tab COLUMNS (key); + +ALTER TABLE test_tab CONCATENATE; + +-- The first column (key) should be null for all 10 rows +SELECT * FROM test_tab ORDER BY value; + +-- The value should be 1 indicating the table has 1 file +SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM test_tab; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/truncate_table.q b/src/test/hive/ql/src/test/queries/clientpositive/truncate_table.q new file mode 100644 index 0000000000000..c0e81e9ac051f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/truncate_table.q @@ -0,0 +1,35 @@ +create table src_truncate (key string, value string); +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'); +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'); + +set hive.fetch.task.convertion=more; + +-- truncate non-partitioned table +explain TRUNCATE TABLE src_truncate; +TRUNCATE TABLE src_truncate; +select * from src_truncate; + +-- truncate a partition +explain TRUNCATE TABLE srcpart_truncate partition (ds='2008-04-08', hr='11'); +TRUNCATE TABLE srcpart_truncate partition (ds='2008-04-08', hr='11'); +select * from srcpart_truncate where ds='2008-04-08' and hr='11'; + +-- truncate partitions with partial spec +explain TRUNCATE TABLE srcpart_truncate partition (ds, hr='12'); +TRUNCATE TABLE srcpart_truncate partition (ds, hr='12'); +select * from srcpart_truncate where hr='12'; + +-- truncate partitioned table +explain TRUNCATE TABLE srcpart_truncate; +TRUNCATE TABLE srcpart_truncate; +select * from srcpart_truncate; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/type_cast_1.q b/src/test/hive/ql/src/test/queries/clientpositive/type_cast_1.q new file mode 100644 index 0000000000000..a1607320c7a2d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/type_cast_1.q @@ -0,0 +1,4 @@ +EXPLAIN +SELECT IF(false, 1, cast(2 as smallint)) + 3 FROM src LIMIT 1; + +SELECT IF(false, 1, cast(2 as smallint)) + 3 FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/type_conversions_1.q b/src/test/hive/ql/src/test/queries/clientpositive/type_conversions_1.q new file mode 100644 index 0000000000000..63dd66ebedd0c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/type_conversions_1.q @@ -0,0 +1,24 @@ + +-- casting from null should yield null +select + cast(null as tinyint), + cast(null as smallint), + cast(null as int), + cast(null as bigint), + cast(null as float), + cast(null as double), + cast(null as decimal), + cast(null as date), + cast(null as timestamp), + cast(null as string), + cast(null as varchar(10)), + cast(null as boolean), + cast(null as binary) +from src limit 1; + +-- Invalid conversions, should all be null +select + cast('abcd' as date), + cast('abcd' as timestamp) +from src limit 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/type_widening.q b/src/test/hive/ql/src/test/queries/clientpositive/type_widening.q new file mode 100644 index 0000000000000..0d36bc44fe08b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/type_widening.q @@ -0,0 +1,6 @@ +-- 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; + +EXPLAIN SELECT * FROM (SELECT 0 AS numcol FROM src UNION ALL SELECT 9223372036854775807 AS numcol FROM src) a ORDER BY numcol; +SELECT * FROM (SELECT 0 AS numcol FROM src UNION ALL SELECT 9223372036854775807 AS numcol FROM src) a ORDER BY numcol; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_collect_set.q b/src/test/hive/ql/src/test/queries/clientpositive/udaf_collect_set.q new file mode 100644 index 0000000000000..45aaa022dc8a4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udaf_collect_set.q @@ -0,0 +1,30 @@ +DESCRIBE FUNCTION collect_set; +DESCRIBE FUNCTION EXTENDED collect_set; + +set hive.map.aggr = false; +set hive.groupby.skewindata = false; + +SELECT key, collect_set(value) +FROM src +GROUP BY key ORDER BY key limit 20; + +set hive.map.aggr = true; +set hive.groupby.skewindata = false; + +SELECT key, collect_set(value) +FROM src +GROUP BY key ORDER BY key limit 20; + +set hive.map.aggr = false; +set hive.groupby.skewindata = true; + +SELECT key, collect_set(value) +FROM src +GROUP BY key ORDER BY key limit 20; + +set hive.map.aggr = true; +set hive.groupby.skewindata = true; + +SELECT key, collect_set(value) +FROM src +GROUP BY key ORDER BY key limit 20; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_context_ngrams.q b/src/test/hive/ql/src/test/queries/clientpositive/udaf_context_ngrams.q new file mode 100644 index 0000000000000..dda7aaa5f2041 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udaf_context_ngrams.q @@ -0,0 +1,12 @@ +CREATE TABLE kafka (contents STRING); +LOAD DATA LOCAL INPATH '../data/files/text-en.txt' INTO TABLE kafka; +set mapred.reduce.tasks=1; +set hive.exec.reducers.max=1; + +SELECT context_ngrams(sentences(lower(contents)), array(null), 100, 1000).estfrequency FROM kafka; +SELECT context_ngrams(sentences(lower(contents)), array("he",null), 100, 1000) FROM kafka; +SELECT context_ngrams(sentences(lower(contents)), array(null,"salesmen"), 100, 1000) FROM kafka; +SELECT context_ngrams(sentences(lower(contents)), array("what","i",null), 100, 1000) FROM kafka; +SELECT context_ngrams(sentences(lower(contents)), array(null,null), 100, 1000).estfrequency FROM kafka; + +DROP TABLE kafka; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_corr.q b/src/test/hive/ql/src/test/queries/clientpositive/udaf_corr.q new file mode 100644 index 0000000000000..6cc9ce2630dd1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udaf_corr.q @@ -0,0 +1,16 @@ +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 +INTO TABLE covar_tab; + +DESCRIBE FUNCTION corr; +DESCRIBE FUNCTION EXTENDED corr; +SELECT corr(b, c) FROM covar_tab WHERE a < 1; +SELECT corr(b, c) FROM covar_tab WHERE a < 3; +SELECT corr(b, c) FROM covar_tab WHERE a = 3; +SELECT a, corr(b, c) FROM covar_tab GROUP BY a ORDER BY a; +SELECT corr(b, c) FROM covar_tab; + +DROP TABLE covar_tab; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_pop.q b/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_pop.q new file mode 100644 index 0000000000000..0f5d5f35bf02a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_pop.q @@ -0,0 +1,16 @@ +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 +INTO TABLE covar_tab; + +DESCRIBE FUNCTION covar_pop; +DESCRIBE FUNCTION EXTENDED covar_pop; +SELECT covar_pop(b, c) FROM covar_tab WHERE a < 1; +SELECT covar_pop(b, c) FROM covar_tab WHERE a < 3; +SELECT covar_pop(b, c) FROM covar_tab WHERE a = 3; +SELECT a, covar_pop(b, c) FROM covar_tab GROUP BY a ORDER BY a; +SELECT covar_pop(b, c) FROM covar_tab; + +DROP TABLE covar_tab; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_samp.q b/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_samp.q new file mode 100644 index 0000000000000..72b9c4bd40049 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_samp.q @@ -0,0 +1,16 @@ +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 +INTO TABLE covar_tab; + +DESCRIBE FUNCTION covar_samp; +DESCRIBE FUNCTION EXTENDED covar_samp; +SELECT covar_samp(b, c) FROM covar_tab WHERE a < 1; +SELECT covar_samp(b, c) FROM covar_tab WHERE a < 3; +SELECT covar_samp(b, c) FROM covar_tab WHERE a = 3; +SELECT a, covar_samp(b, c) FROM covar_tab GROUP BY a ORDER BY a; +SELECT covar_samp(b, c) FROM covar_tab; + +DROP TABLE covar_tab; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q b/src/test/hive/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q new file mode 100644 index 0000000000000..18bcd1c070a44 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q @@ -0,0 +1,5 @@ + +SELECT histogram_numeric(cast(substr(src.value,5) AS double), 2) FROM src; +SELECT histogram_numeric(cast(substr(src.value,5) AS double), 3) FROM src; +SELECT histogram_numeric(cast(substr(src.value,5) AS double), 20) FROM src; +SELECT histogram_numeric(cast(substr(src.value,5) AS double), 200) FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_ngrams.q b/src/test/hive/ql/src/test/queries/clientpositive/udaf_ngrams.q new file mode 100644 index 0000000000000..31ffd29a88acd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udaf_ngrams.q @@ -0,0 +1,12 @@ +CREATE TABLE kafka (contents STRING); +LOAD DATA LOCAL INPATH '../data/files/text-en.txt' INTO TABLE kafka; +set mapred.reduce.tasks=1; +set hive.exec.reducers.max=1; + +SELECT ngrams(sentences(lower(contents)), 1, 100, 1000).estfrequency FROM kafka; +SELECT ngrams(sentences(lower(contents)), 2, 100, 1000).estfrequency FROM kafka; +SELECT ngrams(sentences(lower(contents)), 3, 100, 1000).estfrequency FROM kafka; +SELECT ngrams(sentences(lower(contents)), 4, 100, 1000).estfrequency FROM kafka; +SELECT ngrams(sentences(lower(contents)), 5, 100, 1000).estfrequency FROM kafka; + +DROP TABLE kafka; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_number_format.q b/src/test/hive/ql/src/test/queries/clientpositive/udaf_number_format.q new file mode 100644 index 0000000000000..4f2ab453dc976 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udaf_number_format.q @@ -0,0 +1,14 @@ + +EXPLAIN SELECT + sum('a'), + avg('a'), + variance('a'), + std('a') +FROM src; + +SELECT + sum('a'), + avg('a'), + variance('a'), + std('a') +FROM src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q b/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q new file mode 100644 index 0000000000000..66c408d71bc1a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q @@ -0,0 +1,85 @@ +-- 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; + +create table t1 (result double); +create table t2 (result double); +create table t3 (result double); +create table t4 (result double); +create table t5 (result double); +create table t6 (result double); +create table t7 (result array); +create table t8 (result array); +create table t9 (result array); +create table t10 (result array); +create table t11 (result array); +create table t12 (result array); + +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.map.aggr=false; +-- disable map-side aggregation +FROM bucket +insert overwrite table t1 SELECT percentile_approx(cast(key AS double), 0.5) +insert overwrite table t2 SELECT percentile_approx(cast(key AS double), 0.5, 100) +insert overwrite table t3 SELECT percentile_approx(cast(key AS double), 0.5, 1000) + +insert overwrite table t4 SELECT percentile_approx(cast(key AS int), 0.5) +insert overwrite table t5 SELECT percentile_approx(cast(key AS int), 0.5, 100) +insert overwrite table t6 SELECT percentile_approx(cast(key AS int), 0.5, 1000) + +insert overwrite table t7 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98)) +insert overwrite table t8 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98), 100) +insert overwrite table t9 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98), 1000) + +insert overwrite table t10 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98)) +insert overwrite table t11 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98), 100) +insert overwrite table t12 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98), 1000); + +select * from t1; +select * from t2; +select * from t3; +select * from t4; +select * from t5; +select * from t6; +select * from t7; +select * from t8; +select * from t9; +select * from t10; +select * from t11; +select * from t12; + +set hive.map.aggr=true; +-- enable map-side aggregation +FROM bucket +insert overwrite table t1 SELECT percentile_approx(cast(key AS double), 0.5) +insert overwrite table t2 SELECT percentile_approx(cast(key AS double), 0.5, 100) +insert overwrite table t3 SELECT percentile_approx(cast(key AS double), 0.5, 1000) + +insert overwrite table t4 SELECT percentile_approx(cast(key AS int), 0.5) +insert overwrite table t5 SELECT percentile_approx(cast(key AS int), 0.5, 100) +insert overwrite table t6 SELECT percentile_approx(cast(key AS int), 0.5, 1000) + +insert overwrite table t7 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98)) +insert overwrite table t8 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98), 100) +insert overwrite table t9 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98), 1000) + +insert overwrite table t10 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98)) +insert overwrite table t11 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98), 100) +insert overwrite table t12 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98), 1000); + +select * from t1; +select * from t2; +select * from t3; +select * from t4; +select * from t5; +select * from t6; +select * from t7; +select * from t8; +select * from t9; +select * from t10; +select * from t11; +select * from t12; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q b/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q new file mode 100644 index 0000000000000..07bfb6e1fb2ab --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q @@ -0,0 +1,86 @@ +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- 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; + +create table t1 (result double); +create table t2 (result double); +create table t3 (result double); +create table t4 (result double); +create table t5 (result double); +create table t6 (result double); +create table t7 (result array); +create table t8 (result array); +create table t9 (result array); +create table t10 (result array); +create table t11 (result array); +create table t12 (result array); + +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +set hive.map.aggr=false; +-- disable map-side aggregation +FROM bucket +insert overwrite table t1 SELECT percentile_approx(cast(key AS double), 0.5) +insert overwrite table t2 SELECT percentile_approx(cast(key AS double), 0.5, 100) +insert overwrite table t3 SELECT percentile_approx(cast(key AS double), 0.5, 1000) + +insert overwrite table t4 SELECT percentile_approx(cast(key AS int), 0.5) +insert overwrite table t5 SELECT percentile_approx(cast(key AS int), 0.5, 100) +insert overwrite table t6 SELECT percentile_approx(cast(key AS int), 0.5, 1000) + +insert overwrite table t7 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98)) +insert overwrite table t8 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98), 100) +insert overwrite table t9 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98), 1000) + +insert overwrite table t10 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98)) +insert overwrite table t11 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98), 100) +insert overwrite table t12 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98), 1000); + +select * from t1; +select * from t2; +select * from t3; +select * from t4; +select * from t5; +select * from t6; +select * from t7; +select * from t8; +select * from t9; +select * from t10; +select * from t11; +select * from t12; + +set hive.map.aggr=true; +-- enable map-side aggregation +FROM bucket +insert overwrite table t1 SELECT percentile_approx(cast(key AS double), 0.5) +insert overwrite table t2 SELECT percentile_approx(cast(key AS double), 0.5, 100) +insert overwrite table t3 SELECT percentile_approx(cast(key AS double), 0.5, 1000) + +insert overwrite table t4 SELECT percentile_approx(cast(key AS int), 0.5) +insert overwrite table t5 SELECT percentile_approx(cast(key AS int), 0.5, 100) +insert overwrite table t6 SELECT percentile_approx(cast(key AS int), 0.5, 1000) + +insert overwrite table t7 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98)) +insert overwrite table t8 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98), 100) +insert overwrite table t9 SELECT percentile_approx(cast(key AS double), array(0.05,0.5,0.95,0.98), 1000) + +insert overwrite table t10 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98)) +insert overwrite table t11 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98), 100) +insert overwrite table t12 SELECT percentile_approx(cast(key AS int), array(0.05,0.5,0.95,0.98), 1000); + +select * from t1; +select * from t2; +select * from t3; +select * from t4; +select * from t5; +select * from t6; +select * from t7; +select * from t8; +select * from t9; +select * from t10; +select * from t11; +select * from t12; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf1.q b/src/test/hive/ql/src/test/queries/clientpositive/udf1.q new file mode 100644 index 0000000000000..2292c3f5ec271 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf1.q @@ -0,0 +1,24 @@ +CREATE TABLE dest1(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) STORED AS TEXTFILE; + +EXPLAIN +FROM src INSERT OVERWRITE TABLE dest1 SELECT 'a' LIKE '%a%', 'b' LIKE '%a%', 'ab' LIKE '%a%', 'ab' LIKE '%a_', + '%_' LIKE '\%\_', 'ab' LIKE '\%\_', 'ab' LIKE '_a%', 'ab' LIKE 'a', + '' RLIKE '.*', 'a' RLIKE '[ab]', '' RLIKE '[ab]', 'hadoop' RLIKE '[a-z]*', 'hadoop' RLIKE 'o*', + REGEXP_REPLACE('abc', 'b', 'c'), REGEXP_REPLACE('abc', 'z', 'a'), REGEXP_REPLACE('abbbb', 'bb', 'b'), + REGEXP_REPLACE('hadoop', '(.)[a-z]*', '$1ive'), REGEXP_REPLACE('hadoopAAA','A.*',''), + REGEXP_REPLACE('abc', '', 'A'), 'abc' RLIKE '' + WHERE src.key = 86; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT 'a' LIKE '%a%', 'b' LIKE '%a%', 'ab' LIKE '%a%', 'ab' LIKE '%a_', + '%_' LIKE '\%\_', 'ab' LIKE '\%\_', 'ab' LIKE '_a%', 'ab' LIKE 'a', + '' RLIKE '.*', 'a' RLIKE '[ab]', '' RLIKE '[ab]', 'hadoop' RLIKE '[a-z]*', 'hadoop' RLIKE 'o*', + REGEXP_REPLACE('abc', 'b', 'c'), REGEXP_REPLACE('abc', 'z', 'a'), REGEXP_REPLACE('abbbb', 'bb', 'b'), + REGEXP_REPLACE('hadoop', '(.)[a-z]*', '$1ive'), REGEXP_REPLACE('hadoopAAA','A.*',''), + REGEXP_REPLACE('abc', '', 'A'), 'abc' RLIKE '' + WHERE src.key = 86; + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf2.q b/src/test/hive/ql/src/test/queries/clientpositive/udf2.q new file mode 100644 index 0000000000000..5aa99e85d170b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf2.q @@ -0,0 +1,8 @@ +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT ' abc ' WHERE src.key = 86; + +EXPLAIN +SELECT '|', trim(dest1.c1), '|', rtrim(dest1.c1), '|', ltrim(dest1.c1), '|' FROM dest1; + +SELECT '|', trim(dest1.c1), '|', rtrim(dest1.c1), '|', ltrim(dest1.c1), '|' FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf3.q b/src/test/hive/ql/src/test/queries/clientpositive/udf3.q new file mode 100644 index 0000000000000..dcffeea8dd4c8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf3.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(c1 STRING, c2 STRING, c3 STRING, c4 STRING, c5 STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src INSERT OVERWRITE TABLE dest1 SELECT count(CAST('' AS INT)), sum(CAST('' AS INT)), avg(CAST('' AS INT)), +min(CAST('' AS INT)), max(CAST('' AS INT)); + +FROM src INSERT OVERWRITE TABLE dest1 SELECT count(CAST('' AS INT)), sum(CAST('' AS INT)), avg(CAST('' AS INT)), +min(CAST('' AS INT)), max(CAST('' AS INT)); + +SELECT dest1.* FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf4.q b/src/test/hive/ql/src/test/queries/clientpositive/udf4.q new file mode 100644 index 0000000000000..16f5b2cce2b03 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf4.q @@ -0,0 +1,51 @@ +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT ' abc ' WHERE src.key = 86; + +EXPLAIN +SELECT round(1.0), round(1.5), round(-1.5), floor(1.0), floor(1.5), floor(-1.5), sqrt(1.0), sqrt(-1.0), sqrt(0.0), ceil(1.0), ceil(1.5), ceil(-1.5), ceiling(1.0), rand(3), +3, -3, 1++2, 1+-2, + +~1, +~CAST(1 AS TINYINT), +~CAST(1 AS SMALLINT), +~CAST(1 AS BIGINT), + +CAST(1 AS TINYINT) & CAST(2 AS TINYINT), +CAST(1 AS SMALLINT) & CAST(2 AS SMALLINT), +1 & 2, +CAST(1 AS BIGINT) & CAST(2 AS BIGINT), + +CAST(1 AS TINYINT) | CAST(2 AS TINYINT), +CAST(1 AS SMALLINT) | CAST(2 AS SMALLINT), +1 | 2, +CAST(1 AS BIGINT) | CAST(2 AS BIGINT), + +CAST(1 AS TINYINT) ^ CAST(3 AS TINYINT), +CAST(1 AS SMALLINT) ^ CAST(3 AS SMALLINT), +1 ^ 3, +CAST(1 AS BIGINT) ^ CAST(3 AS BIGINT) + +FROM dest1; + +SELECT round(1.0), round(1.5), round(-1.5), floor(1.0), floor(1.5), floor(-1.5), sqrt(1.0), sqrt(-1.0), sqrt(0.0), ceil(1.0), ceil(1.5), ceil(-1.5), ceiling(1.0), rand(3), +3, -3, 1++2, 1+-2, +~1, +~CAST(1 AS TINYINT), +~CAST(1 AS SMALLINT), +~CAST(1 AS BIGINT), + +CAST(1 AS TINYINT) & CAST(2 AS TINYINT), +CAST(1 AS SMALLINT) & CAST(2 AS SMALLINT), +1 & 2, +CAST(1 AS BIGINT) & CAST(2 AS BIGINT), + +CAST(1 AS TINYINT) | CAST(2 AS TINYINT), +CAST(1 AS SMALLINT) | CAST(2 AS SMALLINT), +1 | 2, +CAST(1 AS BIGINT) | CAST(2 AS BIGINT), + +CAST(1 AS TINYINT) ^ CAST(3 AS TINYINT), +CAST(1 AS SMALLINT) ^ CAST(3 AS SMALLINT), +1 ^ 3, +CAST(1 AS BIGINT) ^ CAST(3 AS BIGINT) + +FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf5.q b/src/test/hive/ql/src/test/queries/clientpositive/udf5.q new file mode 100644 index 0000000000000..782e23dbfaefc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf5.q @@ -0,0 +1,13 @@ +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT ' abc ' WHERE src.key = 86; + +EXPLAIN +SELECT from_unixtime(1226446340), to_date(from_unixtime(1226446340)), day('2008-11-01'), month('2008-11-01'), year('2008-11-01'), day('2008-11-01 15:32:20'), month('2008-11-01 15:32:20'), year('2008-11-01 15:32:20') FROM dest1; + +SELECT from_unixtime(1226446340), to_date(from_unixtime(1226446340)), day('2008-11-01'), month('2008-11-01'), year('2008-11-01'), day('2008-11-01 15:32:20'), month('2008-11-01 15:32:20'), year('2008-11-01 15:32:20') FROM dest1; + +EXPLAIN +SELECT from_unixtime(unix_timestamp('2010-01-13 11:57:40', 'yyyy-MM-dd HH:mm:ss'), 'MM/dd/yy HH:mm:ss'), from_unixtime(unix_timestamp('2010-01-13 11:57:40')) from dest1; + +SELECT from_unixtime(unix_timestamp('2010-01-13 11:57:40', 'yyyy-MM-dd HH:mm:ss'), 'MM/dd/yy HH:mm:ss'), from_unixtime(unix_timestamp('2010-01-13 11:57:40')) from dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf6.q b/src/test/hive/ql/src/test/queries/clientpositive/udf6.q new file mode 100644 index 0000000000000..fd540d610a97e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf6.q @@ -0,0 +1,23 @@ +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT ' abc ' WHERE src.key = 86; + +EXPLAIN +SELECT IF(TRUE, 1, 2) FROM dest1; + +SELECT IF(TRUE, 1, 2) FROM dest1; + +EXPLAIN +SELECT IF(TRUE, 1, 2), IF(FALSE, 1, 2), IF(NULL, 1, 2), IF(TRUE, "a", "b"), + IF(TRUE, 0.1, 0.2), IF(FALSE, CAST(1 AS BIGINT), CAST(2 AS BIGINT)), + IF(FALSE, CAST(127 AS TINYINT), CAST(126 AS TINYINT)), + IF(FALSE, CAST(127 AS SMALLINT), CAST(128 AS SMALLINT)), + CAST(128 AS INT), CAST(1.0 AS DOUBLE), + CAST('128' AS STRING) FROM dest1; + +SELECT IF(TRUE, 1, 2), IF(FALSE, 1, 2), IF(NULL, 1, 2), IF(TRUE, "a", "b"), + IF(TRUE, 0.1, 0.2), IF(FALSE, CAST(1 AS BIGINT), CAST(2 AS BIGINT)), + IF(FALSE, CAST(127 AS TINYINT), CAST(126 AS TINYINT)), + IF(FALSE, CAST(127 AS SMALLINT), CAST(128 AS SMALLINT)), + CAST(128 AS INT), CAST(1.0 AS DOUBLE), + CAST('128' AS STRING) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf7.q b/src/test/hive/ql/src/test/queries/clientpositive/udf7.q new file mode 100644 index 0000000000000..d12394e2e7265 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf7.q @@ -0,0 +1,22 @@ +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT ' abc ' WHERE src.key = 86; + +EXPLAIN +SELECT ROUND(LN(3.0),12), LN(0.0), LN(-1), ROUND(LOG(3.0),12), LOG(0.0), + LOG(-1), ROUND(LOG2(3.0),12), LOG2(0.0), LOG2(-1), + ROUND(LOG10(3.0),12), LOG10(0.0), LOG10(-1), ROUND(LOG(2, 3.0),12), + LOG(2, 0.0), LOG(2, -1), LOG(0.5, 2), LOG(2, 0.5), ROUND(EXP(2.0),12), + POW(2,3), POWER(2,3), POWER(2,-3), POWER(0.5, -3), POWER(4, 0.5), + POWER(-1, 0.5), POWER(-1, 2), POWER(CAST (1 AS DECIMAL), CAST (0 AS INT)), + POWER(CAST (2 AS DECIMAL), CAST (3 AS INT)), + POW(CAST (2 AS DECIMAL), CAST(3 AS INT)) FROM dest1; + +SELECT ROUND(LN(3.0),12), LN(0.0), LN(-1), ROUND(LOG(3.0),12), LOG(0.0), + LOG(-1), ROUND(LOG2(3.0),12), LOG2(0.0), LOG2(-1), + ROUND(LOG10(3.0),12), LOG10(0.0), LOG10(-1), ROUND(LOG(2, 3.0),12), + LOG(2, 0.0), LOG(2, -1), LOG(0.5, 2), LOG(2, 0.5), ROUND(EXP(2.0),12), + POW(2,3), POWER(2,3), POWER(2,-3), POWER(0.5, -3), POWER(4, 0.5), + POWER(-1, 0.5), POWER(-1, 2), POWER(CAST (1 AS DECIMAL), CAST (0 AS INT)), + POWER(CAST (2 AS DECIMAL), CAST (3 AS INT)), + POW(CAST (2 AS DECIMAL), CAST(3 AS INT)) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf8.q b/src/test/hive/ql/src/test/queries/clientpositive/udf8.q new file mode 100644 index 0000000000000..a9e696369a047 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf8.q @@ -0,0 +1,10 @@ +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT '' WHERE src.key = 86; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT '1' WHERE src.key = 86; + +EXPLAIN +SELECT avg(c1), sum(c1), count(c1) FROM dest1; + +SELECT avg(c1), sum(c1), count(c1) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf9.q b/src/test/hive/ql/src/test/queries/clientpositive/udf9.q new file mode 100644 index 0000000000000..3b179b338e991 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf9.q @@ -0,0 +1,20 @@ +EXPLAIN +SELECT DATEDIFF('2008-12-31', '2009-01-01'), DATEDIFF('2008-03-01', '2008-02-28'), + DATEDIFF('2007-03-01', '2007-01-28'), DATEDIFF('2008-03-01 23:59:59', '2008-03-02 00:00:00'), + DATE_ADD('2008-12-31', 1), DATE_ADD('2008-12-31', 365), + DATE_ADD('2008-02-28', 2), DATE_ADD('2009-02-28', 2), + DATE_ADD('2007-02-28', 365), DATE_ADD('2007-02-28 23:59:59', 730), + DATE_SUB('2009-01-01', 1), DATE_SUB('2009-01-01', 365), + DATE_SUB('2008-02-28', 2), DATE_SUB('2009-02-28', 2), + DATE_SUB('2007-02-28', 365), DATE_SUB('2007-02-28 01:12:34', 730) + FROM src WHERE src.key = 86; + +SELECT DATEDIFF('2008-12-31', '2009-01-01'), DATEDIFF('2008-03-01', '2008-02-28'), + DATEDIFF('2007-03-01', '2007-01-28'), DATEDIFF('2008-03-01 23:59:59', '2008-03-02 00:00:00'), + DATE_ADD('2008-12-31', 1), DATE_ADD('2008-12-31', 365), + DATE_ADD('2008-02-28', 2), DATE_ADD('2009-02-28', 2), + DATE_ADD('2007-02-28', 365), DATE_ADD('2007-02-28 23:59:59', 730), + DATE_SUB('2009-01-01', 1), DATE_SUB('2009-01-01', 365), + DATE_SUB('2008-03-01', 2), DATE_SUB('2009-03-01', 2), + DATE_SUB('2007-02-28', 365), DATE_SUB('2007-02-28 01:12:34', 730) + FROM src WHERE src.key = 86; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_10_trims.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_10_trims.q new file mode 100644 index 0000000000000..8a7e3a520751e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_10_trims.q @@ -0,0 +1,12 @@ +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +EXPLAIN +INSERT OVERWRITE TABLE dest1 +SELECT trim(trim(trim(trim(trim(trim(trim(trim(trim(trim( ' abc ')))))))))) +FROM src +WHERE src.key = 86; + +INSERT OVERWRITE TABLE dest1 +SELECT trim(trim(trim(trim(trim(trim(trim(trim(trim(trim( ' abc ')))))))))) +FROM src +WHERE src.key = 86; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_E.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_E.q new file mode 100644 index 0000000000000..113af61062b0d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_E.q @@ -0,0 +1,14 @@ +explain +select E() FROM src LIMIT 1; + +select E() FROM src LIMIT 1; + +DESCRIBE FUNCTION E; +DESCRIBE FUNCTION EXTENDED E; +explain +select E() FROM src LIMIT 1; + +select E() FROM src LIMIT 1; + +DESCRIBE FUNCTION E; +DESCRIBE FUNCTION EXTENDED E; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_PI.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_PI.q new file mode 100644 index 0000000000000..1fde7df5d251e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_PI.q @@ -0,0 +1,14 @@ +explain +select PI() FROM src LIMIT 1; + +select PI() FROM src LIMIT 1; + +DESCRIBE FUNCTION PI; +DESCRIBE FUNCTION EXTENDED PI; +explain +select PI() FROM src LIMIT 1; + +select PI() FROM src LIMIT 1; + +DESCRIBE FUNCTION PI; +DESCRIBE FUNCTION EXTENDED PI; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_abs.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_abs.q new file mode 100644 index 0000000000000..f4f227d0dc168 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_abs.q @@ -0,0 +1,30 @@ +DESCRIBE FUNCTION abs; +DESCRIBE FUNCTION EXTENDED abs; + +EXPLAIN SELECT + abs(0), + abs(-1), + abs(123), + abs(-9223372036854775807), + abs(9223372036854775807) +FROM src LIMIT 1; + +SELECT + abs(0), + abs(-1), + abs(123), + abs(-9223372036854775807), + abs(9223372036854775807) +FROM src LIMIT 1; + +EXPLAIN SELECT + abs(0.0), + abs(-3.14159265), + abs(3.14159265) +FROM src LIMIT 1; + +SELECT + abs(0.0), + abs(-3.14159265), + abs(3.14159265) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_acos.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_acos.q new file mode 100644 index 0000000000000..625a2aa5c6aa6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_acos.q @@ -0,0 +1,14 @@ +DESCRIBE FUNCTION acos; +DESCRIBE FUNCTION EXTENDED acos; + +SELECT acos(null) +FROM src LIMIT 1; + +SELECT acos(0) +FROM src LIMIT 1; + +SELECT acos(-0.5), asin(0.66) +FROM src LIMIT 1; + +SELECT acos(2) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_add.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_add.q new file mode 100644 index 0000000000000..dbc3940f450ea --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_add.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION +; +DESCRIBE FUNCTION EXTENDED +; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_array.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_array.q new file mode 100644 index 0000000000000..fca8fe8d1c00c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_array.q @@ -0,0 +1,8 @@ +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_array_contains.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_array_contains.q new file mode 100644 index 0000000000000..937bb0be03bd3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_array_contains.q @@ -0,0 +1,9 @@ +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; + +-- 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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_ascii.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_ascii.q new file mode 100644 index 0000000000000..53b389fd38fd8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_ascii.q @@ -0,0 +1,14 @@ +DESCRIBE FUNCTION ascii; +DESCRIBE FUNCTION EXTENDED ascii; + +EXPLAIN SELECT + ascii('Facebook'), + ascii(''), + ascii('!') +FROM src LIMIT 1; + +SELECT + ascii('Facebook'), + ascii(''), + ascii('!') +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_asin.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_asin.q new file mode 100644 index 0000000000000..f95a5f57df8f8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_asin.q @@ -0,0 +1,14 @@ +DESCRIBE FUNCTION asin; +DESCRIBE FUNCTION EXTENDED asin; + +SELECT asin(null) +FROM src LIMIT 1; + +SELECT asin(0) +FROM src LIMIT 1; + +SELECT asin(-0.5), asin(0.66) +FROM src LIMIT 1; + +SELECT asin(2) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_atan.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_atan.q new file mode 100644 index 0000000000000..d4ef03deb81b0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_atan.q @@ -0,0 +1,16 @@ +DESCRIBE FUNCTION atan; +DESCRIBE FUNCTION EXTENDED atan; + +SELECT atan(null) +FROM src LIMIT 1; + +SELECT atan(1), atan(6), atan(-1.0) +FROM src LIMIT 1; +DESCRIBE FUNCTION atan; +DESCRIBE FUNCTION EXTENDED atan; + +SELECT atan(null) +FROM src LIMIT 1; + +SELECT atan(1), atan(6), atan(-1.0) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_avg.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_avg.q new file mode 100644 index 0000000000000..5c95bc407b63c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_avg.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION avg; +DESCRIBE FUNCTION EXTENDED avg; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_between.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_between.q new file mode 100644 index 0000000000000..eb3ccea82e631 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_between.q @@ -0,0 +1,14 @@ +describe function between; +describe function extended between; + +explain SELECT * FROM src where key + 100 between (150 + -50) AND (150 + 50) LIMIT 20; +SELECT * FROM src where key + 100 between (150 + -50) AND (150 + 50) LIMIT 20; + +explain SELECT * FROM src where key + 100 not between (150 + -50) AND (150 + 50) LIMIT 20; +SELECT * FROM src where key + 100 not between (150 + -50) AND (150 + 50) LIMIT 20; + +explain SELECT * FROM src where 'b' between 'a' AND 'c' LIMIT 1; +SELECT * FROM src where 'b' between 'a' AND 'c' LIMIT 1; + +explain SELECT * FROM src where 2 between 2 AND '3' LIMIT 1; +SELECT * FROM src where 2 between 2 AND '3' LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bigint.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_bigint.q new file mode 100644 index 0000000000000..e430866bf0952 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_bigint.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION bigint; +DESCRIBE FUNCTION EXTENDED bigint; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bin.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_bin.q new file mode 100644 index 0000000000000..2b9ad62a39dbe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_bin.q @@ -0,0 +1,11 @@ +DESCRIBE FUNCTION bin; +DESCRIBE FUNCTION EXTENDED bin; + +SELECT + bin(1), + bin(0), + bin(99992421) +FROM src LIMIT 1; + +-- Negative numbers should be treated as two's complement (64 bit). +SELECT bin(-5) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_and.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_and.q new file mode 100644 index 0000000000000..7ea50dac31d07 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_and.q @@ -0,0 +1,12 @@ +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; + +drop table bitmap_test; +create table bitmap_test (a array, b array); + +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 ewah_bitmap_and(a,b) from bitmap_test; + +drop table bitmap_test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_empty.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_empty.q new file mode 100644 index 0000000000000..88e961683e289 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_empty.q @@ -0,0 +1,3 @@ +select ewah_bitmap_empty(array(13,2,4,8589934592,0,0)) from src limit 1; + +select ewah_bitmap_empty(array(13,2,4,8589934592,4096,0)) from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_or.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_or.q new file mode 100644 index 0000000000000..0b71e681a53d5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_or.q @@ -0,0 +1,12 @@ +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; + +drop table bitmap_test; +create table bitmap_test (a array, b array); + +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 ewah_bitmap_or(a,b) from bitmap_test; + +drop table bitmap_test; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_and.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_and.q new file mode 100644 index 0000000000000..8e40111c1ff84 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_and.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION &; +DESCRIBE FUNCTION EXTENDED &; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_not.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_not.q new file mode 100644 index 0000000000000..fdd5485446520 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_not.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION ~; +DESCRIBE FUNCTION EXTENDED ~; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_or.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_or.q new file mode 100644 index 0000000000000..55375db9be5d7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_or.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION |; +DESCRIBE FUNCTION EXTENDED |; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_xor.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_xor.q new file mode 100644 index 0000000000000..9c4ee7366232f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_xor.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION ^; +DESCRIBE FUNCTION EXTENDED ^; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_boolean.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_boolean.q new file mode 100644 index 0000000000000..1dc5017b340f7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_boolean.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION boolean; +DESCRIBE FUNCTION EXTENDED boolean; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_case.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_case.q new file mode 100644 index 0000000000000..4f71e70e1f3bf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_case.q @@ -0,0 +1,63 @@ +DESCRIBE FUNCTION case; +DESCRIBE FUNCTION EXTENDED case; + +EXPLAIN +SELECT CASE 1 + WHEN 1 THEN 2 + WHEN 3 THEN 4 + ELSE 5 + END, + CASE 2 + WHEN 1 THEN 2 + ELSE 5 + END, + CASE 14 + WHEN 12 THEN 13 + WHEN 14 THEN 15 + END, + CASE 16 + WHEN 12 THEN 13 + WHEN 14 THEN 15 + END, + CASE 17 + WHEN 18 THEN NULL + WHEN 17 THEN 20 + END, + CASE 21 + WHEN 22 THEN 23 + WHEN 21 THEN 24 + END +FROM src LIMIT 1; + +SELECT CASE 1 + WHEN 1 THEN 2 + WHEN 3 THEN 4 + ELSE 5 + END, + CASE 2 + WHEN 1 THEN 2 + ELSE 5 + END, + CASE 14 + WHEN 12 THEN 13 + WHEN 14 THEN 15 + END, + CASE 16 + WHEN 12 THEN 13 + WHEN 14 THEN 15 + END, + CASE 17 + WHEN 18 THEN NULL + WHEN 17 THEN 20 + END, + CASE 21 + WHEN 22 THEN 23 + WHEN 21 THEN 24 + END +FROM src LIMIT 1; + +-- 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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_case_column_pruning.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_case_column_pruning.q new file mode 100644 index 0000000000000..da925515b30da --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_case_column_pruning.q @@ -0,0 +1,18 @@ +EXPLAIN +SELECT CASE a.key + WHEN '1' THEN 2 + WHEN '3' THEN 4 + ELSE 5 + END as key +FROM src a JOIN src b +ON a.key = b.key +ORDER BY key LIMIT 10; + +SELECT CASE a.key + WHEN '1' THEN 2 + WHEN '3' THEN 4 + ELSE 5 + END as key +FROM src a JOIN src b +ON a.key = b.key +ORDER BY key LIMIT 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_case_thrift.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_case_thrift.q new file mode 100644 index 0000000000000..736bb053cddc7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_case_thrift.q @@ -0,0 +1,34 @@ +EXPLAIN +SELECT CASE src_thrift.lint[0] + WHEN 0 THEN src_thrift.lint[0] + 1 + WHEN 1 THEN src_thrift.lint[0] + 2 + WHEN 2 THEN 100 + ELSE 5 + END, + CASE src_thrift.lstring[0] + WHEN '0' THEN 'zero' + WHEN '10' THEN CONCAT(src_thrift.lstring[0], " is ten") + ELSE 'default' + END, + (CASE src_thrift.lstring[0] + WHEN '0' THEN src_thrift.lstring + ELSE NULL + END)[0] +FROM src_thrift LIMIT 3; + +SELECT CASE src_thrift.lint[0] + WHEN 0 THEN src_thrift.lint[0] + 1 + WHEN 1 THEN src_thrift.lint[0] + 2 + WHEN 2 THEN 100 + ELSE 5 + END, + CASE src_thrift.lstring[0] + WHEN '0' THEN 'zero' + WHEN '10' THEN CONCAT(src_thrift.lstring[0], " is ten") + ELSE 'default' + END, + (CASE src_thrift.lstring[0] + WHEN '0' THEN src_thrift.lstring + ELSE NULL + END)[0] +FROM src_thrift LIMIT 3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_ceil.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_ceil.q new file mode 100644 index 0000000000000..34a91e18500fb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_ceil.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION ceil; +DESCRIBE FUNCTION EXTENDED ceil; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_ceiling.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_ceiling.q new file mode 100644 index 0000000000000..d830b4db4f67f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_ceiling.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION ceiling; +DESCRIBE FUNCTION EXTENDED ceiling; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_coalesce.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_coalesce.q new file mode 100644 index 0000000000000..48ca29cbc3ba6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_coalesce.q @@ -0,0 +1,54 @@ +DESCRIBE FUNCTION coalesce; +DESCRIBE FUNCTION EXTENDED coalesce; + +EXPLAIN +SELECT COALESCE(1), + COALESCE(1, 2), + COALESCE(NULL, 2), + COALESCE(1, NULL), + COALESCE(NULL, NULL, 3), + COALESCE(4, NULL, NULL, NULL), + COALESCE('1'), + COALESCE('1', '2'), + COALESCE(NULL, '2'), + COALESCE('1', NULL), + COALESCE(NULL, NULL, '3'), + COALESCE('4', NULL, NULL, NULL), + COALESCE(1.0), + COALESCE(1.0, 2.0), + COALESCE(NULL, 2.0), + COALESCE(NULL, 2.0, 3.0), + COALESCE(2.0, NULL, 3.0), + COALESCE(IF(TRUE, NULL, 0), NULL) +FROM src LIMIT 1; + +SELECT COALESCE(1), + COALESCE(1, 2), + COALESCE(NULL, 2), + COALESCE(1, NULL), + COALESCE(NULL, NULL, 3), + COALESCE(4, NULL, NULL, NULL), + COALESCE('1'), + COALESCE('1', '2'), + COALESCE(NULL, '2'), + COALESCE('1', NULL), + COALESCE(NULL, NULL, '3'), + COALESCE('4', NULL, NULL, NULL), + COALESCE(1.0), + COALESCE(1.0, 2.0), + COALESCE(NULL, 2.0), + COALESCE(NULL, 2.0, 3.0), + COALESCE(2.0, NULL, 3.0), + COALESCE(IF(TRUE, NULL, 0), NULL) +FROM src LIMIT 1; + +EXPLAIN +SELECT COALESCE(src_thrift.lint[1], 999), + COALESCE(src_thrift.lintstring[0].mystring, '999'), + COALESCE(src_thrift.mstringstring['key_2'], '999') +FROM src_thrift; + +SELECT COALESCE(src_thrift.lint[1], 999), + COALESCE(src_thrift.lintstring[0].mystring, '999'), + COALESCE(src_thrift.mstringstring['key_2'], '999') +FROM src_thrift; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_compare_java_string.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_compare_java_string.q new file mode 100644 index 0000000000000..6c12f81304e46 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_compare_java_string.q @@ -0,0 +1,10 @@ +EXPLAIN +CREATE TEMPORARY FUNCTION test_udf_get_java_string AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaString'; + +CREATE TEMPORARY FUNCTION test_udf_get_java_string AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaString'; + +select * from src where value = test_udf_get_java_string("val_66"); +select * from (select * from src where value = 'val_66' or value = 'val_8') t where value <> test_udf_get_java_string("val_8"); + + +DROP TEMPORARY FUNCTION test_udf_get_java_boolean; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_concat.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_concat.q new file mode 100644 index 0000000000000..f642f6a2d00df --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_concat.q @@ -0,0 +1,15 @@ +DESCRIBE FUNCTION concat; +DESCRIBE FUNCTION EXTENDED concat; + +SELECT + concat('a', 'b'), + concat('a', 'b', 'c'), + concat('a', null, 'c'), + concat(null), + concat('a'), + concat(null, 1, 2), + concat(1, 2, 3, 'a'), + concat(1, 2), + concat(1), + concat('1234', 'abc', 'extra argument') +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert1.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert1.q new file mode 100644 index 0000000000000..496f40372d088 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert1.q @@ -0,0 +1,7 @@ +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', concat(src.key) WHERE src.key < 100 group by src.key; + +SELECT dest1.* FROM dest1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert2.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert2.q new file mode 100644 index 0000000000000..6d1353feacd2d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert2.q @@ -0,0 +1,8 @@ +CREATE TABLE dest1(key STRING, value STRING) STORED AS TEXTFILE; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT concat('1234', 'abc', 'extra argument'), src.value WHERE src.key < 100; + +SELECT dest1.* FROM dest1; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_ws.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_ws.q new file mode 100644 index 0000000000000..6a0ce20dc37b7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_ws.q @@ -0,0 +1,43 @@ +DESCRIBE FUNCTION concat_ws; +DESCRIBE FUNCTION EXTENDED concat_ws; + +CREATE TABLE dest1(c1 STRING, c2 STRING, c3 STRING); + +FROM src INSERT OVERWRITE TABLE dest1 SELECT 'abc', 'xyz', '8675309' WHERE src.key = 86; + +EXPLAIN +SELECT concat_ws(dest1.c1, dest1.c2, dest1.c3), + concat_ws(',', dest1.c1, dest1.c2, dest1.c3), + concat_ws(NULL, dest1.c1, dest1.c2, dest1.c3), + concat_ws('**', dest1.c1, NULL, dest1.c3) FROM dest1; + +SELECT concat_ws(dest1.c1, dest1.c2, dest1.c3), + concat_ws(',', dest1.c1, dest1.c2, dest1.c3), + concat_ws(NULL, dest1.c1, dest1.c2, dest1.c3), + concat_ws('**', dest1.c1, NULL, dest1.c3) FROM dest1; + +-- evalutes function for array of strings +EXPLAIN +SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'), + concat_ws('-', 'www', array('face', 'book', 'com'), '1234'), + concat_ws('F', 'www', array('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; + +SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'), + concat_ws('-', 'www', array('face', 'book', 'com'), '1234'), + concat_ws('F', 'www', array('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; + +SELECT concat_ws(NULL, array('www', 'face', 'book', 'com'), '1234'), + concat_ws(NULL, 'www', array('face', 'book', 'com'), '1234'), + concat_ws(NULL, 'www', array('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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_context_aware.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_context_aware.q new file mode 100644 index 0000000000000..04cb512f4a3eb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_context_aware.q @@ -0,0 +1,5 @@ +create temporary function counter as 'org.apache.hadoop.hive.ql.udf.generic.DummyContextUDF'; + +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +select *, counter(key) from src limit 20; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_conv.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_conv.q new file mode 100644 index 0000000000000..212bcfb57938f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_conv.q @@ -0,0 +1,75 @@ +DESCRIBE FUNCTION conv; +DESCRIBE FUNCTION EXTENDED conv; + +-- conv must work on both strings and integers up to 64-bit precision + +-- Some simple conversions to test different bases +SELECT + conv('4521', 10, 36), + conv('22', 10, 10), + conv('110011', 2, 16), + conv('facebook', 36, 16) +FROM src LIMIT 1; + +-- Test negative numbers. If to_base is positive, the number should be handled +-- as a two's complement (64-bit) +SELECT + conv('-641', 10, -10), + conv('1011', 2, -16), + conv('-1', 10, 16), + conv('-15', 10, 16) +FROM src LIMIT 1; + +-- Test overflow. If a number is two large, the result should be -1 (if signed) +-- or MAX_LONG (if unsigned) +SELECT + conv('9223372036854775807', 36, 16), + conv('9223372036854775807', 36, -16), + conv('-9223372036854775807', 36, 16), + conv('-9223372036854775807', 36, -16) +FROM src LIMIT 1; + +-- 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 +-- prefix should be converted. +SELECT + conv('123455', 3, 10), + conv('131', 1, 5), + conv('515', 5, 100), + conv('10', -2, 2) +FROM src LIMIT 1; + +-- Perform the same tests with number arguments. + +SELECT + conv(4521, 10, 36), + conv(22, 10, 10), + conv(110011, 2, 16) +FROM src LIMIT 1; + +SELECT + conv(-641, 10, -10), + conv(1011, 2, -16), + conv(-1, 10, 16), + conv(-15, 10, 16) +FROM src LIMIT 1; + +SELECT + conv(9223372036854775807, 36, 16), + conv(9223372036854775807, 36, -16), + conv(-9223372036854775807, 36, 16), + conv(-9223372036854775807, 36, -16) +FROM src LIMIT 1; + +SELECT + conv(123455, 3, 10), + conv(131, 1, 5), + conv(515, 5, 100), + conv('10', -2, 2) +FROM src LIMIT 1; + +-- Make sure that state is properly reset. + +SELECT conv(key, 10, 16), + conv(key, 16, 10) +FROM src LIMIT 3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_cos.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_cos.q new file mode 100644 index 0000000000000..7887c4c2603f1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_cos.q @@ -0,0 +1,8 @@ +DESCRIBE FUNCTION cos; +DESCRIBE FUNCTION EXTENDED cos; + +SELECT cos(null) +FROM src LIMIT 1; + +SELECT cos(0.98), cos(1.57), cos(-0.5) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_count.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_count.q new file mode 100644 index 0000000000000..022d7a9f8dd4d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_count.q @@ -0,0 +1,20 @@ +DESCRIBE FUNCTION count; +DESCRIBE FUNCTION EXTENDED count; + +EXPLAIN SELECT count(key) FROM src; +SELECT count(key) FROM src; + +EXPLAIN SELECT count(DISTINCT key) FROM src; +SELECT count(DISTINCT key) FROM src; + +EXPLAIN SELECT count(DISTINCT key, value) FROM src; +SELECT count(DISTINCT key, value) FROM src; + +EXPLAIN SELECT count(*) FROM src; +SELECT count(*) FROM src; + +EXPLAIN SELECT count(1) FROM src; +SELECT count(1) FROM src; + +select count(1) from src where false; +select count(*) from src where false; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_date_add.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_date_add.q new file mode 100644 index 0000000000000..52e19ed3f417d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_date_add.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION date_add; +DESCRIBE FUNCTION EXTENDED date_add; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_date_sub.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_date_sub.q new file mode 100644 index 0000000000000..9fdef94456d06 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_date_sub.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION date_sub; +DESCRIBE FUNCTION EXTENDED date_sub; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_datediff.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_datediff.q new file mode 100644 index 0000000000000..30718fe8e4d5c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_datediff.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION datediff; +DESCRIBE FUNCTION EXTENDED datediff; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_day.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_day.q new file mode 100644 index 0000000000000..fb9b57da015bb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_day.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION day; +DESCRIBE FUNCTION EXTENDED day; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_dayofmonth.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_dayofmonth.q new file mode 100644 index 0000000000000..ab30f1d69b4af --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_dayofmonth.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION dayofmonth; +DESCRIBE FUNCTION EXTENDED dayofmonth; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_degrees.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_degrees.q new file mode 100644 index 0000000000000..014ca1c6e6afb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_degrees.q @@ -0,0 +1,14 @@ +explain +select degrees(PI()) FROM src LIMIT 1; + +select degrees(PI()) FROM src LIMIT 1; + +DESCRIBE FUNCTION degrees; +DESCRIBE FUNCTION EXTENDED degrees; +explain +select degrees(PI()) FROM src LIMIT 1; + +select degrees(PI()) FROM src LIMIT 1; + +DESCRIBE FUNCTION degrees; +DESCRIBE FUNCTION EXTENDED degrees; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_div.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_div.q new file mode 100644 index 0000000000000..4229e625a042a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_div.q @@ -0,0 +1,4 @@ +DESCRIBE FUNCTION div; +DESCRIBE FUNCTION EXTENDED div; + +SELECT 3 DIV 2 FROM SRC LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_divide.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_divide.q new file mode 100644 index 0000000000000..dc4b2e7884d68 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_divide.q @@ -0,0 +1,4 @@ +DESCRIBE FUNCTION /; +DESCRIBE FUNCTION EXTENDED /; + +SELECT 3 / 2 FROM SRC LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_double.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_double.q new file mode 100644 index 0000000000000..974dbf1595513 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_double.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION double; +DESCRIBE FUNCTION EXTENDED double; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_elt.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_elt.q new file mode 100644 index 0000000000000..c32340ac89feb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_elt.q @@ -0,0 +1,29 @@ +DESCRIBE FUNCTION elt; +DESCRIBE FUNCTION EXTENDED elt; + +EXPLAIN +SELECT elt(2, 'abc', 'defg'), + elt(3, 'aa', 'bb', 'cc', 'dd', 'ee', 'ff', 'gg'), + elt('1', 'abc', 'defg'), + elt(2, 'aa', CAST('2' AS TINYINT)), + elt(2, 'aa', CAST('12345' AS SMALLINT)), + elt(2, 'aa', CAST('123456789012' AS BIGINT)), + elt(2, 'aa', CAST(1.25 AS FLOAT)), + elt(2, 'aa', CAST(16.0 AS DOUBLE)), + elt(null, 'abc', 'defg'), + elt(0, 'abc', 'defg'), + elt(3, 'abc', 'defg') +FROM src LIMIT 1; + +SELECT elt(2, 'abc', 'defg'), + elt(3, 'aa', 'bb', 'cc', 'dd', 'ee', 'ff', 'gg'), + elt('1', 'abc', 'defg'), + elt(2, 'aa', CAST('2' AS TINYINT)), + elt(2, 'aa', CAST('12345' AS SMALLINT)), + elt(2, 'aa', CAST('123456789012' AS BIGINT)), + elt(2, 'aa', CAST(1.25 AS FLOAT)), + elt(2, 'aa', CAST(16.0 AS DOUBLE)), + elt(null, 'abc', 'defg'), + elt(0, 'abc', 'defg'), + elt(3, 'abc', 'defg') +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_equal.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_equal.q new file mode 100644 index 0000000000000..f5e9a7df4be00 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_equal.q @@ -0,0 +1,12 @@ +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_exp.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_exp.q new file mode 100644 index 0000000000000..612c9a75760f9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_exp.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION exp; +DESCRIBE FUNCTION EXTENDED exp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_explode.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_explode.q new file mode 100644 index 0000000000000..19af288ff8404 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_explode.q @@ -0,0 +1,22 @@ +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; + +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; + +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; + +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; + +drop table lazy_array_map; +create table lazy_array_map (map_col map, array_col array); +INSERT OVERWRITE TABLE lazy_array_map select map(1,'one',2,'two',3,'three'), array('100','200','300') FROM src LIMIT 1; + +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/src/test/hive/ql/src/test/queries/clientpositive/udf_field.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_field.q new file mode 100644 index 0000000000000..e995f5cf3c084 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_field.q @@ -0,0 +1,55 @@ +DESCRIBE FUNCTION field; +DESCRIBE FUNCTION EXTENDED field; + +SELECT + field("x", "a", "b", "c", "d"), + field(NULL, "a", "b", "c", "d"), + field(0, 1, 2, 3, 4) +FROM src LIMIT 1; + +SELECT + field("a", "a", "b", "c", "d"), + field("b", "a", "b", "c", "d"), + field("c", "a", "b", "c", "d"), + field("d", "a", "b", "c", "d"), + field("d", "a", "b", NULL, "d") +FROM src LIMIT 1; + +SELECT + field(1, 1, 2, 3, 4), + field(2, 1, 2, 3, 4), + field(3, 1, 2, 3, 4), + field(4, 1, 2, 3, 4), + field(4, 1, 2, NULL, 4) +FROM src LIMIT 1; + + +CREATE TABLE test_table(col1 STRING, col2 STRING) STORED AS TEXTFILE; +LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE test_table; + +select col1,col2, + field("66",col1), + field("66",col1, col2), + field("val_86",col1, col2), + field(NULL, col1, col2), + field(col1, 66, 88), + field(col1, "66", "88"), + field(col1, "666", "888"), + field(col2, "66", "88"), + field(col1, col2, col1), + field(col1, col2, "66") +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; + +select col1,col2, + field(66,col1), + field(66,col1, col2), + field(86, col2, col1), + field(86, col1, col1), + field(86,col1,n,col2), + field(NULL,col1,n,col2), + field(col1, col2) +from (select col1, col2, NULL as n from test_table1 where col1=86 or col1=66) t; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_find_in_set.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_find_in_set.q new file mode 100644 index 0000000000000..eac2c6e91c3d1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_find_in_set.q @@ -0,0 +1,23 @@ +DESCRIBE FUNCTION find_in_set; +DESCRIBE FUNCTION EXTENDED find_in_set; + +EXPLAIN +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 * FROM src1 WHERE NOT find_in_set(key,'311,128,345,2,956')=0; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_float.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_float.q new file mode 100644 index 0000000000000..abad4ed136ad0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_float.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION float; +DESCRIBE FUNCTION EXTENDED float; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_floor.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_floor.q new file mode 100644 index 0000000000000..a9c304b14fc7a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_floor.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION floor; +DESCRIBE FUNCTION EXTENDED floor; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_format_number.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_format_number.q new file mode 100644 index 0000000000000..e2084cddf0ec7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_format_number.q @@ -0,0 +1,57 @@ +use default; +-- Test format_number() UDF + +DESCRIBE FUNCTION format_number; +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; + +SELECT format_number(12332.123456, 4), + format_number(12332.1,4), + format_number(12332.2,0) +FROM src limit 1; + +-- 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; + +-- 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; + +-- zeros +SELECT format_number(0.0, 4), + format_number(0.000000, 1), + format_number(000.0000, 1), + format_number(00000.0000, 1), + format_number(-00.0, 4) +FROM src limit 1; + +-- integers +SELECT format_number(0, 0), + format_number(1, 4), + format_number(12, 2), + format_number(123, 5), + format_number(1234, 7) +FROM src limit 1; + +-- long and double boundary +-- 9223372036854775807 is LONG_MAX +-- -9223372036854775807 is one more than LONG_MIN, +-- due to HIVE-2733, put it here to check LONG_MIN boundary +-- 4.9E-324 and 1.7976931348623157E308 are Double.MIN_VALUE and Double.MAX_VALUE +-- check them for Double boundary +SELECT format_number(-9223372036854775807, 10), + format_number(9223372036854775807, 20), + format_number(4.9E-324, 324), + format_number(1.7976931348623157E308, 308) +FROM src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_from_unixtime.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_from_unixtime.q new file mode 100644 index 0000000000000..e67803703fa92 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_from_unixtime.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION from_unixtime; +DESCRIBE FUNCTION EXTENDED from_unixtime; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_get_json_object.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_get_json_object.q new file mode 100644 index 0000000000000..464f2df3dcd7a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_get_json_object.q @@ -0,0 +1,40 @@ +DESCRIBE FUNCTION get_json_object; +DESCRIBE FUNCTION EXTENDED get_json_object; + +CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; + +FROM src INSERT OVERWRITE TABLE dest1 SELECT ' abc ' WHERE src.key = 86; + +EXPLAIN +SELECT get_json_object(src_json.json, '$.owner') FROM src_json; + +SELECT get_json_object(src_json.json, '$') FROM src_json; + +SELECT get_json_object(src_json.json, '$.owner'), get_json_object(src_json.json, '$.store') FROM src_json; + +SELECT get_json_object(src_json.json, '$.store.bicycle'), get_json_object(src_json.json, '$.store.book') FROM src_json; + +SELECT get_json_object(src_json.json, '$.store.book[0]'), get_json_object(src_json.json, '$.store.book[*]') FROM src_json; + +SELECT get_json_object(src_json.json, '$.store.book[0].category'), get_json_object(src_json.json, '$.store.book[*].category'), get_json_object(src_json.json, '$.store.book[*].isbn'), get_json_object(src_json.json, '$.store.book[*].reader') FROM src_json; + +SELECT get_json_object(src_json.json, '$.store.book[*].reader[0].age'), get_json_object(src_json.json, '$.store.book[*].reader[*].age') FROM src_json; + +SELECT get_json_object(src_json.json, '$.store.basket[0][1]'), get_json_object(src_json.json, '$.store.basket[*]'), get_json_object(src_json.json, '$.store.basket[*][0]'), get_json_object(src_json.json, '$.store.basket[0][*]'), get_json_object(src_json.json, '$.store.basket[*][*]'), get_json_object(src_json.json, '$.store.basket[0][2].b'), get_json_object(src_json.json, '$.store.basket[0][*].b') FROM src_json; + +SELECT get_json_object(src_json.json, '$.non_exist_key'), get_json_object(src_json.json, '$..no_recursive'), get_json_object(src_json.json, '$.store.book[10]'), get_json_object(src_json.json, '$.store.book[0].non_exist_key'), get_json_object(src_json.json, '$.store.basket[*].non_exist_key'), get_json_object(src_json.json, '$.store.basket[0][*].non_exist_key') FROM src_json; + +SELECT get_json_object(src_json.json, '$.zip code') FROM src_json; + +SELECT get_json_object(src_json.json, '$.fb:testid') FROM src_json; + + +-- Verify that get_json_object can handle new lines in JSON values + +CREATE TABLE dest2(c1 STRING) STORED AS RCFILE; + +INSERT OVERWRITE TABLE dest2 SELECT '{"a":"b\nc"}' FROM src LIMIT 1; + +SELECT * FROM dest2; + +SELECT get_json_object(c1, '$.a') FROM dest2; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthan.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthan.q new file mode 100644 index 0000000000000..aea110a9431cf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthan.q @@ -0,0 +1,4 @@ +DESCRIBE FUNCTION >; +DESCRIBE FUNCTION EXTENDED >; + +SELECT true>false, false>true, false>false, true>true FROM src LIMIT 1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q new file mode 100644 index 0000000000000..8de165b9ea9ed --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q @@ -0,0 +1,4 @@ +DESCRIBE FUNCTION >=; +DESCRIBE FUNCTION EXTENDED >=; + +SELECT true>=false, false>=true, false>=false, true>=true FROM src LIMIT 1; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_hash.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_hash.q new file mode 100644 index 0000000000000..faf372218a107 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_hash.q @@ -0,0 +1,17 @@ +DESCRIBE FUNCTION hash; +DESCRIBE FUNCTION EXTENDED hash; + +EXPLAIN +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_hex.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_hex.q new file mode 100644 index 0000000000000..37e035ad42b00 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_hex.q @@ -0,0 +1,20 @@ +DESCRIBE FUNCTION hex; +DESCRIBE FUNCTION EXTENDED hex; + +-- If the argument is a string, hex should return a string containing two hex +-- digits for every character in the input. +SELECT + hex('Facebook'), + hex('\0'), + hex('qwertyuiopasdfghjkl') +FROM src LIMIT 1; + +-- If the argument is a number, hex should convert it to hexadecimal. +SELECT + hex(1), + hex(0), + hex(4207849477) +FROM src LIMIT 1; + +-- Negative numbers should be treated as two's complement (64 bit). +SELECT hex(-5) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_hour.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_hour.q new file mode 100644 index 0000000000000..c5c366daa8c83 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_hour.q @@ -0,0 +1,13 @@ +DESCRIBE FUNCTION hour; +DESCRIBE FUNCTION EXTENDED hour; + +EXPLAIN +SELECT hour('2009-08-07 13:14:15'), hour('13:14:15'), hour('2009-08-07') +FROM src WHERE key = 86; + +SELECT hour('2009-08-07 13:14:15'), hour('13:14:15'), hour('2009-08-07') +FROM src WHERE key = 86; + + +SELECT hour(cast('2009-08-07 13:14:15' as timestamp)) +FROM src WHERE key=86; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_if.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_if.q new file mode 100644 index 0000000000000..4f7c8b4a36bad --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_if.q @@ -0,0 +1,34 @@ +DESCRIBE FUNCTION if; +DESCRIBE FUNCTION EXTENDED if; + +EXPLAIN +SELECT IF(TRUE, 1, 2) AS COL1, + IF(FALSE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL2, + IF(1=1, IF(2=2, 1, 2), IF(3=3, 3, 4)) AS COL3, + 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; + + +SELECT IF(TRUE, 1, 2) AS COL1, + IF(FALSE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL2, + IF(1=1, IF(2=2, 1, 2), IF(3=3, 3, 4)) AS COL3, + 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; + +-- Type conversions +EXPLAIN +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_in.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_in.q new file mode 100644 index 0000000000000..75778138742d5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_in.q @@ -0,0 +1,13 @@ +SELECT 1 IN (1, 2, 3), + 4 IN (1, 2, 3), + array(1,2,3) IN (array(1,2,3)), + "bee" IN("aee", "bee", "cee", 1), + "dee" IN("aee", "bee", "cee"), + 1 = 1 IN(true, false), + true IN (true, false) = true, + 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; + +SELECT key FROM src WHERE key IN ("238", 86); \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_in_file.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_in_file.q new file mode 100644 index 0000000000000..4da478908ddbd --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_in_file.q @@ -0,0 +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") +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") +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_index.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_index.q new file mode 100644 index 0000000000000..9079d0e5743f6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_index.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION `index`; +DESCRIBE FUNCTION EXTENDED `index`; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_inline.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_inline.q new file mode 100644 index 0000000000000..39006f4b9b117 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_inline.q @@ -0,0 +1,26 @@ +describe function inline; + +explain SELECT inline( + ARRAY( + STRUCT (1,'dude!'), + STRUCT (2,'Wheres'), + STRUCT (3,'my car?') + ) +) as (id, text) FROM SRC limit 2; + +SELECT inline( + ARRAY( + STRUCT (1,'dude!'), + STRUCT (2,'Wheres'), + STRUCT (3,'my car?') + ) +) as (id, text) FROM SRC limit 2; + +-- HIVE-3475 INLINE UDTF doesn't convert types properly +select * from (SELECT + ARRAY( + STRUCT (1,'dude!'), + STRUCT (2,'Wheres'), + STRUCT (3,'my car?') + ) as value FROM SRC limit 1) input + LATERAL VIEW inline(value) myTable AS id, text; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_instr.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_instr.q new file mode 100644 index 0000000000000..20ed8e4ea0176 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_instr.q @@ -0,0 +1,33 @@ +DESCRIBE FUNCTION instr; +DESCRIBE FUNCTION EXTENDED instr; + +EXPLAIN +SELECT instr('abcd', 'abc'), + instr('abcabc', 'ccc'), + instr(123, '23'), + instr(123, 23), + instr(TRUE, 1), + instr(FALSE, 1), + instr('12345', CAST('2' AS TINYINT)), + instr(CAST('12345' AS SMALLINT), '34'), + instr(CAST('123456789012' AS BIGINT), '456'), + instr(CAST(1.25 AS FLOAT), '.25'), + instr(CAST(16.0 AS DOUBLE), '.0'), + instr(null, 'abc'), + instr('abcd', null) +FROM src LIMIT 1; + +SELECT instr('abcd', 'abc'), + instr('abcabc', 'ccc'), + instr(123, '23'), + instr(123, 23), + instr(TRUE, 1), + instr(FALSE, 1), + instr('12345', CAST('2' AS TINYINT)), + instr(CAST('12345' AS SMALLINT), '34'), + instr(CAST('123456789012' AS BIGINT), '456'), + instr(CAST(1.25 AS FLOAT), '.25'), + instr(CAST(16.0 AS DOUBLE), '.0'), + instr(null, 'abc'), + instr('abcd', null) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_int.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_int.q new file mode 100644 index 0000000000000..64e94a1dc32e1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_int.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION int; +DESCRIBE FUNCTION EXTENDED int; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_isnotnull.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_isnotnull.q new file mode 100644 index 0000000000000..75228617ae4f8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_isnotnull.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION isnotnull; +DESCRIBE FUNCTION EXTENDED isnotnull; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull.q new file mode 100644 index 0000000000000..3b5df163f55b2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION isnull; +DESCRIBE FUNCTION EXTENDED isnull; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q new file mode 100644 index 0000000000000..d1569cc7f6dbb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q @@ -0,0 +1,37 @@ +DESCRIBE FUNCTION isnull; +DESCRIBE FUNCTION EXTENDED isnull; + +DESCRIBE FUNCTION isnotnull; +DESCRIBE FUNCTION EXTENDED isnotnull; + + +EXPLAIN +SELECT NULL IS NULL, + 1 IS NOT NULL, + 'my string' IS NOT NULL +FROM src +WHERE true IS NOT NULL LIMIT 1; + + +SELECT NULL IS NULL, + 1 IS NOT NULL, + 'my string' IS NOT NULL +FROM src +WHERE true IS NOT NULL LIMIT 1; + + +EXPLAIN +FROM src_thrift +SELECT src_thrift.lint IS NOT NULL, + src_thrift.lintstring IS NOT NULL, + src_thrift.mstringstring IS NOT NULL +WHERE src_thrift.lint IS NOT NULL + AND NOT (src_thrift.mstringstring IS NULL) LIMIT 1; + + +FROM src_thrift +SELECT src_thrift.lint IS NOT NULL, + src_thrift.lintstring IS NOT NULL, + src_thrift.mstringstring IS NOT NULL +WHERE src_thrift.lint IS NOT NULL + AND NOT (src_thrift.mstringstring IS NULL) LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_java_method.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_java_method.q new file mode 100644 index 0000000000000..2f28be1e9d0e1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_java_method.q @@ -0,0 +1,25 @@ +DESCRIBE FUNCTION java_method; +DESCRIBE FUNCTION EXTENDED java_method; + +-- java_method() is a synonym for reflect() + +EXPLAIN EXTENDED +SELECT java_method("java.lang.String", "valueOf", 1), + java_method("java.lang.String", "isEmpty"), + java_method("java.lang.Math", "max", 2, 3), + java_method("java.lang.Math", "min", 2, 3), + 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; + + +SELECT java_method("java.lang.String", "valueOf", 1), + java_method("java.lang.String", "isEmpty"), + java_method("java.lang.Math", "max", 2, 3), + java_method("java.lang.Math", "min", 2, 3), + 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; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_lcase.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_lcase.q new file mode 100644 index 0000000000000..c2a09b0f6570c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_lcase.q @@ -0,0 +1,3 @@ +DESCRIBE FUNCTION lcase; +DESCRIBE FUNCTION EXTENDED lcase; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_length.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_length.q new file mode 100644 index 0000000000000..b84307970d630 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_length.q @@ -0,0 +1,14 @@ +DESCRIBE FUNCTION length; +DESCRIBE FUNCTION EXTENDED length; + +CREATE TABLE dest1(len INT); +EXPLAIN FROM src1 INSERT OVERWRITE TABLE dest1 SELECT length(src1.value); +FROM src1 INSERT OVERWRITE TABLE dest1 SELECT length(src1.value); +SELECT dest1.* FROM dest1; +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; +EXPLAIN SELECT length(dest1.name) FROM dest1; +SELECT length(dest1.name) FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_lessthan.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_lessthan.q new file mode 100644 index 0000000000000..a2577006a885d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_lessthan.q @@ -0,0 +1,4 @@ +DESCRIBE FUNCTION <; +DESCRIBE FUNCTION EXTENDED <; + +SELECT true; +DESCRIBE FUNCTION EXTENDED <>; + +DESCRIBE FUNCTION !=; +DESCRIBE FUNCTION EXTENDED !=; + +EXPLAIN +SELECT key, value +FROM src +WHERE key <> '302'; + +SELECT key, value +FROM src +WHERE key <> '302'; + +EXPLAIN +SELECT key, value +FROM src +WHERE key != '302'; + +SELECT key, value +FROM src +WHERE key != '302'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_notop.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_notop.q new file mode 100644 index 0000000000000..88396545de933 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_notop.q @@ -0,0 +1,10 @@ +SELECT 1 NOT IN (1, 2, 3), + 4 NOT IN (1, 2, 3), + 1 = 2 NOT IN (true, false), + "abc" NOT LIKE "a%", + "abc" NOT LIKE "b%", + "abc" NOT RLIKE "^ab", + "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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_nvl.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_nvl.q new file mode 100644 index 0000000000000..0133b4b90bcbc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_nvl.q @@ -0,0 +1,13 @@ + +DESCRIBE FUNCTION nvl; +DESCRIBE FUNCTION EXTENDED nvl; + +EXPLAIN +SELECT NVL( 1 , 2 ) AS COL1, + NVL( NULL, 5 ) AS COL2 +FROM src LIMIT 1; + +SELECT NVL( 1 , 2 ) AS COL1, + NVL( NULL, 5 ) AS COL2 +FROM src LIMIT 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_or.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_or.q new file mode 100644 index 0000000000000..fbd4486d49d6d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_or.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION or; +DESCRIBE FUNCTION EXTENDED or; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_parse_url.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_parse_url.q new file mode 100644 index 0000000000000..62c67db20b1a2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_parse_url.q @@ -0,0 +1,29 @@ +DESCRIBE FUNCTION parse_url; +DESCRIBE FUNCTION EXTENDED parse_url; + +EXPLAIN +SELECT parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'HOST'), +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'PATH'), +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'QUERY') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'REF') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'QUERY', 'k2') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'QUERY', 'k1') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'QUERY', 'k3') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'FILE') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'PROTOCOL') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'USERINFO') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'AUTHORITY') + FROM src WHERE key = 86; + +SELECT parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'HOST'), +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'PATH'), +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'QUERY') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'REF') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'QUERY', 'k2') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'QUERY', 'k1') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'QUERY', 'k3') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'FILE') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'PROTOCOL') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'USERINFO') , +parse_url('http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1', 'AUTHORITY') + FROM src WHERE key = 86; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_percentile.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_percentile.q new file mode 100644 index 0000000000000..936a514b1e7bc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_percentile.q @@ -0,0 +1,75 @@ +DESCRIBE FUNCTION percentile; +DESCRIBE FUNCTION EXTENDED percentile; + + +set hive.map.aggr = false; +set hive.groupby.skewindata = false; + +SELECT CAST(key AS INT) DIV 10, + percentile(CAST(substr(value, 5) AS INT), 0.0), + percentile(CAST(substr(value, 5) AS INT), 0.5), + percentile(CAST(substr(value, 5) AS INT), 1.0), + percentile(CAST(substr(value, 5) AS INT), array(0.0, 0.5, 0.99, 1.0)) +FROM src +GROUP BY CAST(key AS INT) DIV 10; + + +set hive.map.aggr = true; +set hive.groupby.skewindata = false; + +SELECT CAST(key AS INT) DIV 10, + percentile(CAST(substr(value, 5) AS INT), 0.0), + percentile(CAST(substr(value, 5) AS INT), 0.5), + percentile(CAST(substr(value, 5) AS INT), 1.0), + percentile(CAST(substr(value, 5) AS INT), array(0.0, 0.5, 0.99, 1.0)) +FROM src +GROUP BY CAST(key AS INT) DIV 10; + + + +set hive.map.aggr = false; +set hive.groupby.skewindata = true; + +SELECT CAST(key AS INT) DIV 10, + percentile(CAST(substr(value, 5) AS INT), 0.0), + percentile(CAST(substr(value, 5) AS INT), 0.5), + percentile(CAST(substr(value, 5) AS INT), 1.0), + percentile(CAST(substr(value, 5) AS INT), array(0.0, 0.5, 0.99, 1.0)) +FROM src +GROUP BY CAST(key AS INT) DIV 10; + + +set hive.map.aggr = true; +set hive.groupby.skewindata = true; + +SELECT CAST(key AS INT) DIV 10, + percentile(CAST(substr(value, 5) AS INT), 0.0), + percentile(CAST(substr(value, 5) AS INT), 0.5), + percentile(CAST(substr(value, 5) AS INT), 1.0), + percentile(CAST(substr(value, 5) AS INT), array(0.0, 0.5, 0.99, 1.0)) +FROM src +GROUP BY CAST(key AS INT) DIV 10; + + +set hive.map.aggr = true; +set hive.groupby.skewindata = false; + +-- test null handling +SELECT CAST(key AS INT) DIV 10, + percentile(NULL, 0.0), + percentile(NULL, array(0.0, 0.5, 0.99, 1.0)) +FROM src +GROUP BY CAST(key AS INT) DIV 10; + + +-- test empty array handling +SELECT CAST(key AS INT) DIV 10, + percentile(IF(CAST(key AS INT) DIV 10 < 5, 1, NULL), 0.5), + percentile(IF(CAST(key AS INT) DIV 10 < 5, 1, NULL), array(0.0, 0.5, 0.99, 1.0)) +FROM src +GROUP BY CAST(key AS INT) DIV 10; + +select percentile(cast(key as bigint), 0.5) from src where false; + +-- test where percentile list is empty +select percentile(cast(key as bigint), array()) from src where false; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_pmod.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_pmod.q new file mode 100644 index 0000000000000..9ff73d42b41e6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_pmod.q @@ -0,0 +1,20 @@ +DESCRIBE FUNCTION pmod; +DESCRIBE FUNCTION EXTENDED pmod; + +SELECT pmod(null, null) +FROM src LIMIT 1; + +SELECT pmod(-100,9), pmod(-50,101), pmod(-1000,29) +FROM src LIMIT 1; + +SELECT pmod(100,19), pmod(50,125), pmod(300,15) +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 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.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; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_positive.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_positive.q new file mode 100644 index 0000000000000..e5fd17e2a0a68 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_positive.q @@ -0,0 +1,6 @@ +DESCRIBE FUNCTION positive; +DESCRIBE FUNCTION EXTENDED positive; + +-- synonym +DESCRIBE FUNCTION +; +DESCRIBE FUNCTION EXTENDED +; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_pow.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_pow.q new file mode 100644 index 0000000000000..dc9ad9ddea600 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_pow.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION pow; +DESCRIBE FUNCTION EXTENDED pow; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_power.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_power.q new file mode 100644 index 0000000000000..c5a3f40e5f3bb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_power.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION power; +DESCRIBE FUNCTION EXTENDED power; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_printf.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_printf.q new file mode 100644 index 0000000000000..99e89ccbf773a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_printf.q @@ -0,0 +1,35 @@ +use default; +-- Test printf() UDF + +DESCRIBE FUNCTION printf; +DESCRIBE FUNCTION EXTENDED printf; + +EXPLAIN +SELECT printf("Hello World %d %s", 100, "days") FROM src LIMIT 1; + +-- 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; + +-- 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; + +-- Test Timestamp +create table timestamp_udf (t timestamp); +from src + insert overwrite table timestamp_udf + select '2011-05-06 07:08:09.1234567' limit 1; +select printf("timestamp: %s", t) from timestamp_udf; +drop table timestamp_udf; + +-- Test Binary +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; +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; +drop table dest1; +drop table binary_udf; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_radians.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_radians.q new file mode 100644 index 0000000000000..001d1cf8b6464 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_radians.q @@ -0,0 +1,16 @@ +explain +select radians(57.2958) FROM src LIMIT 1; + +select radians(57.2958) FROM src LIMIT 1; +select radians(143.2394) FROM src LIMIT 1; + +DESCRIBE FUNCTION radians; +DESCRIBE FUNCTION EXTENDED radians; +explain +select radians(57.2958) FROM src LIMIT 1; + +select radians(57.2958) FROM src LIMIT 1; +select radians(143.2394) FROM src LIMIT 1; + +DESCRIBE FUNCTION radians; +DESCRIBE FUNCTION EXTENDED radians; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_rand.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_rand.q new file mode 100644 index 0000000000000..ff5380c545414 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_rand.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION rand; +DESCRIBE FUNCTION EXTENDED rand; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect.q new file mode 100644 index 0000000000000..f357ff5049564 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect.q @@ -0,0 +1,24 @@ +DESCRIBE FUNCTION reflect; +DESCRIBE FUNCTION EXTENDED reflect; + +EXPLAIN EXTENDED +SELECT reflect("java.lang.String", "valueOf", 1), + reflect("java.lang.String", "isEmpty"), + reflect("java.lang.Math", "max", 2, 3), + reflect("java.lang.Math", "min", 2, 3), + reflect("java.lang.Math", "round", 2.5), + 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; + + +SELECT reflect("java.lang.String", "valueOf", 1), + reflect("java.lang.String", "isEmpty"), + reflect("java.lang.Math", "max", 2, 3), + reflect("java.lang.Math", "min", 2, 3), + reflect("java.lang.Math", "round", 2.5), + 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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect2.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect2.q new file mode 100644 index 0000000000000..9ffd755292f66 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect2.q @@ -0,0 +1,70 @@ +DESCRIBE FUNCTION reflect2; +DESCRIBE FUNCTION EXTENDED reflect2; + +EXPLAIN EXTENDED +SELECT key, + reflect2(key, "byteValue"), + reflect2(key, "shortValue"), + reflect2(key, "intValue"), + reflect2(key, "longValue"), + reflect2(key, "floatValue"), + reflect2(key, "doubleValue"), + reflect2(key, "toString"), + value, + reflect2(value, "concat", "_concat"), + reflect2(value, "contains", "86"), + reflect2(value, "startsWith", "v"), + reflect2(value, "endsWith", "6"), + reflect2(value, "equals", "val_86"), + reflect2(value, "equalsIgnoreCase", "VAL_86"), + reflect2(value, "getBytes"), + reflect2(value, "indexOf", "1"), + reflect2(value, "lastIndexOf", "1"), + reflect2(value, "replace", "val", "VALUE"), + reflect2(value, "substring", 1), + reflect2(value, "substring", 1, 5), + reflect2(value, "toUpperCase"), + reflect2(value, "trim"), + ts, + reflect2(ts, "getYear"), + reflect2(ts, "getMonth"), + reflect2(ts, "getDay"), + reflect2(ts, "getHours"), + reflect2(ts, "getMinutes"), + reflect2(ts, "getSeconds"), + reflect2(ts, "getTime") +FROM (select cast(key as int) key, value, cast('2013-02-15 19:41:20' as timestamp) ts from src) a LIMIT 5; + + +SELECT key, + reflect2(key, "byteValue"), + reflect2(key, "shortValue"), + reflect2(key, "intValue"), + reflect2(key, "longValue"), + reflect2(key, "floatValue"), + reflect2(key, "doubleValue"), + reflect2(key, "toString"), + value, + reflect2(value, "concat", "_concat"), + reflect2(value, "contains", "86"), + reflect2(value, "startsWith", "v"), + reflect2(value, "endsWith", "6"), + reflect2(value, "equals", "val_86"), + reflect2(value, "equalsIgnoreCase", "VAL_86"), + reflect2(value, "getBytes"), + reflect2(value, "indexOf", "1"), + reflect2(value, "lastIndexOf", "1"), + reflect2(value, "replace", "val", "VALUE"), + reflect2(value, "substring", 1), + reflect2(value, "substring", 1, 5), + reflect2(value, "toUpperCase"), + reflect2(value, "trim"), + ts, + reflect2(ts, "getYear"), + reflect2(ts, "getMonth"), + reflect2(ts, "getDay"), + reflect2(ts, "getHours"), + reflect2(ts, "getMinutes"), + reflect2(ts, "getSeconds"), + reflect2(ts, "getTime") +FROM (select cast(key as int) key, value, cast('2013-02-15 19:41:20' as timestamp) ts from src) a LIMIT 5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp.q new file mode 100644 index 0000000000000..3aee10963d747 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp.q @@ -0,0 +1,6 @@ +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_extract.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_extract.q new file mode 100644 index 0000000000000..7d7560d44d1f7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_extract.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION regexp_extract; +DESCRIBE FUNCTION EXTENDED regexp_extract; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_replace.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_replace.q new file mode 100644 index 0000000000000..db536a1efa4b9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_replace.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION regexp_replace; +DESCRIBE FUNCTION EXTENDED regexp_replace; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_repeat.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_repeat.q new file mode 100644 index 0000000000000..162085f4c71c8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_repeat.q @@ -0,0 +1,16 @@ +DESCRIBE FUNCTION repeat; +DESCRIBE FUNCTION EXTENDED repeat; + +EXPLAIN SELECT + repeat("Facebook", 3), + repeat("", 4), + repeat("asd", 0), + repeat("asdf", -1) +FROM src LIMIT 1; + +SELECT + repeat("Facebook", 3), + repeat("", 4), + repeat("asd", 0), + repeat("asdf", -1) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_reverse.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_reverse.q new file mode 100644 index 0000000000000..81f765ec5937c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_reverse.q @@ -0,0 +1,15 @@ +DESCRIBE FUNCTION reverse; +DESCRIBE FUNCTION EXTENDED reverse; + +CREATE TABLE dest1(len STRING); +EXPLAIN FROM src1 INSERT OVERWRITE TABLE dest1 SELECT reverse(src1.value); +FROM src1 INSERT OVERWRITE TABLE dest1 SELECT reverse(src1.value); +SELECT dest1.* FROM dest1; +DROP TABLE dest1; + +-- Test with non-ascii characters +-- 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; +SELECT count(1) FROM dest1 WHERE reverse(dest1.name) = _UTF-8 0xE993AEE982B5; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_rlike.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_rlike.q new file mode 100644 index 0000000000000..27581046a8f10 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_rlike.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION rlike; +DESCRIBE FUNCTION EXTENDED rlike; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_round.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_round.q new file mode 100644 index 0000000000000..18ebba8708493 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_round.q @@ -0,0 +1,44 @@ +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; + +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; + +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), + 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; + +SELECT + round(3.141592653589793, -15), round(3.141592653589793, -16), + round(3.141592653589793, -13), round(3.141592653589793, -14), + round(3.141592653589793, -11), round(3.141592653589793, -12), + round(3.141592653589793, -9), round(3.141592653589793, -10), + round(3.141592653589793, -7), round(3.141592653589793, -8), + round(3.141592653589793, -5), round(3.141592653589793, -6), + round(3.141592653589793, -3), round(3.141592653589793, -4), + round(3.141592653589793, -1), round(3.141592653589793, -2), + round(3.141592653589793, 0), round(3.141592653589793, 1), + round(3.141592653589793, 2), round(3.141592653589793, 3), + round(3.141592653589793, 4), round(3.141592653589793, 5), + round(3.141592653589793, 6), round(3.141592653589793, 7), + round(3.141592653589793, 8), round(3.141592653589793, 9), + round(3.141592653589793, 10), round(3.141592653589793, 11), + 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; + +SELECT round(1809242.3151111344, 9), round(-1809242.3151111344, 9) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_round_2.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_round_2.q new file mode 100644 index 0000000000000..6be30855aea1b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_round_2.q @@ -0,0 +1,13 @@ +-- test for NaN (not-a-number) +create table tstTbl1(n double); + +insert overwrite table tstTbl1 +select 'NaN' from src limit 1; + +select * from tstTbl1; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_round_3.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_round_3.q new file mode 100644 index 0000000000000..50a1f44419047 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_round_3.q @@ -0,0 +1,14 @@ +-- test for TINYINT +select round(-128), round(127), round(0) from src limit 1; + +-- test for SMALLINT +select round(-32768), round(32767), round(-129), round(128) from src limit 1; + +-- 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; + +-- 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; + +-- test for DOUBLE +select round(126.1), round(126.7), round(32766.1), round(32766.7) from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_rpad.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_rpad.q new file mode 100644 index 0000000000000..01e5fbd429b17 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_rpad.q @@ -0,0 +1,14 @@ +DESCRIBE FUNCTION rpad; +DESCRIBE FUNCTION EXTENDED rpad; + +EXPLAIN SELECT + rpad('hi', 1, '?'), + rpad('hi', 5, '.'), + rpad('hi', 6, '123') +FROM src LIMIT 1; + +SELECT + rpad('hi', 1, '?'), + rpad('hi', 5, '.'), + rpad('hi', 6, '123') +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_rtrim.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_rtrim.q new file mode 100644 index 0000000000000..33d09df284624 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_rtrim.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION rtrim; +DESCRIBE FUNCTION EXTENDED rtrim; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_second.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_second.q new file mode 100644 index 0000000000000..1943188086ea5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_second.q @@ -0,0 +1,9 @@ +DESCRIBE FUNCTION second; +DESCRIBE FUNCTION EXTENDED second; + +EXPLAIN +SELECT second('2009-08-07 13:14:15'), second('13:14:15'), second('2009-08-07') +FROM src WHERE key = 86; + +SELECT second('2009-08-07 13:14:15'), second('13:14:15'), second('2009-08-07') +FROM src WHERE key = 86; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sentences.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_sentences.q new file mode 100644 index 0000000000000..009e4cb48a1d4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_sentences.q @@ -0,0 +1,22 @@ +CREATE TABLE sent_tmp (val array); +CREATE TABLE sent_tmp2 (val string); +INSERT OVERWRITE TABLE sent_tmp +SELECT explode(sentences(decode(unhex("486976652065737420756E20657863656C6C656E74206F7574696C20706F7572206C65732072657175C3AA74657320646520646F6E6EC3A965732C20657420706575742DC3AA74726520706C757320706F6C7976616C656E7420717565206C612074726164756374696F6E206175746F6D61746971756521206C6120706F6E6374756174696F6E206D756C7469706C65732C206465732070687261736573206D616C20666F726DC3A96573202E2E2E20636F6E667573696F6E202D20657420706F757274616E742063652055444620666F6E6374696F6E6E6520656E636F72652121"), "UTF-8"), "fr")) AS val FROM src LIMIT 3; +INSERT OVERWRITE TABLE sent_tmp2 +SELECT explode(val) AS val FROM sent_tmp; +SELECT hex(val) AS value FROM sent_tmp2 ORDER BY value ASC; + +DROP TABLE sent_tmp; +DROP TABLE sent_tmp2; + +CREATE TABLE sent_tmp (val array); +CREATE TABLE sent_tmp2 (val string); +INSERT OVERWRITE TABLE sent_tmp +SELECT explode(sentences(decode(unhex("48697665206973742065696E2061757367657A656963686E65746573205765726B7A6575672066C3BC7220646965204162667261676520766F6E20446174656E2C20756E64207669656C6C6569636874207669656C736569746967657220616C7320646965206D61736368696E656C6C6520C39C6265727365747A756E6721204D756C7469706C652C207363686C6563687420676562696C646574656E2053C3A4747A65202E2E2E205665727765636873656C756E6720496E74657270756E6B74696F6E202D20756E6420646F636820697374206469657365205544462066756E6B74696F6E6965727420696D6D6572206E6F63682121"), "UTF-8"), "de")) AS val FROM src LIMIT 3; +INSERT OVERWRITE TABLE sent_tmp2 +SELECT explode(val) AS val FROM sent_tmp; +SELECT hex(val) AS value FROM sent_tmp2 ORDER BY value ASC; + + + +SELECT sentences("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!!!!") AS value FROM src ORDER BY value ASC LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sign.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_sign.q new file mode 100644 index 0000000000000..abceb3439ec1e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_sign.q @@ -0,0 +1,20 @@ +explain +select sign(0) FROM src LIMIT 1; +select sign(0) FROM src LIMIT 1; + +select sign(-45) FROM src LIMIT 1; + +select sign(46) FROM src LIMIT 1; + +DESCRIBE FUNCTION sign; +DESCRIBE FUNCTION EXTENDED sign; +explain +select sign(0) FROM src LIMIT 1; +select sign(0) FROM src LIMIT 1; + +select sign(-45) FROM src LIMIT 1; + +select sign(46) FROM src LIMIT 1; + +DESCRIBE FUNCTION sign; +DESCRIBE FUNCTION EXTENDED sign; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sin.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_sin.q new file mode 100644 index 0000000000000..abb7cac8dac9d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_sin.q @@ -0,0 +1,8 @@ +DESCRIBE FUNCTION sin; +DESCRIBE FUNCTION EXTENDED sin; + +SELECT sin(null) +FROM src LIMIT 1; + +SELECT sin(0.98), sin(1.57), sin(-0.5) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_size.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_size.q new file mode 100644 index 0000000000000..8aaa68a6e318a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_size.q @@ -0,0 +1,20 @@ +DESCRIBE FUNCTION size; +DESCRIBE FUNCTION EXTENDED size; + +EXPLAIN +FROM src_thrift +SELECT size(src_thrift.lint), + size(src_thrift.lintstring), + size(src_thrift.mstringstring), + size(null) +WHERE src_thrift.lint IS NOT NULL + AND NOT (src_thrift.mstringstring IS NULL) LIMIT 1; + + +FROM src_thrift +SELECT size(src_thrift.lint), + size(src_thrift.lintstring), + size(src_thrift.mstringstring), + size(null) +WHERE src_thrift.lint IS NOT NULL + AND NOT (src_thrift.mstringstring IS NULL) LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_smallint.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_smallint.q new file mode 100644 index 0000000000000..202a401eb705d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_smallint.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION smallint; +DESCRIBE FUNCTION EXTENDED smallint; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sort_array.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_sort_array.q new file mode 100644 index 0000000000000..ef0973212a27c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_sort_array.q @@ -0,0 +1,38 @@ +use default; +-- Test sort_array() UDF + +DESCRIBE FUNCTION sort_array; +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("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; + +-- Evaluate function against INT valued keys +SELECT sort_array(array(2, 9, 7, 3, 5, 4, 1, 6, 8)) FROM src LIMIT 1; + +-- 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; + +-- Test it against data in a table. +CREATE TABLE dest1 ( + tinyints ARRAY, + smallints ARRAY, + ints ARRAY, + bigints ARRAY, + booleans ARRAY, + floats ARRAY, + doubles ARRAY, + strings ARRAY, + timestamps ARRAY +) STORED AS TEXTFILE; + +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), + sort_array(doubles), sort_array(strings), sort_array(timestamps) + FROM dest1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_space.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_space.q new file mode 100644 index 0000000000000..cf6466fb63bad --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_space.q @@ -0,0 +1,27 @@ +DESCRIBE FUNCTION space; +DESCRIBE FUNCTION EXTENDED space; + +EXPLAIN SELECT + space(10), + space(0), + space(1), + space(-1), + space(-100) +FROM src LIMIT 1; + +SELECT + length(space(10)), + length(space(0)), + length(space(1)), + length(space(-1)), + length(space(-100)) +FROM src LIMIT 1; + +SELECT + space(10), + space(0), + space(1), + space(-1), + space(-100) +FROM src LIMIT 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_split.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_split.q new file mode 100644 index 0000000000000..f79901736cf7c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_split.q @@ -0,0 +1,16 @@ +DESCRIBE FUNCTION split; +DESCRIBE FUNCTION EXTENDED split; + +EXPLAIN SELECT + split('a b c', ' '), + split('oneAtwoBthreeC', '[ABC]'), + split('', '.'), + split(50401020, 0) +FROM src LIMIT 1; + +SELECT + split('a b c', ' '), + split('oneAtwoBthreeC', '[ABC]'), + split('', '.'), + split(50401020, 0) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sqrt.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_sqrt.q new file mode 100644 index 0000000000000..aec061ddd7b9c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_sqrt.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION sqrt; +DESCRIBE FUNCTION EXTENDED sqrt; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_std.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_std.q new file mode 100644 index 0000000000000..9a20256e3e5fc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_std.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION std; +DESCRIBE FUNCTION EXTENDED std; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev.q new file mode 100644 index 0000000000000..b3b8794c9d585 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION stddev; +DESCRIBE FUNCTION EXTENDED stddev; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_pop.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_pop.q new file mode 100644 index 0000000000000..4abede7af8b55 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_pop.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION udf_stddev_pop; +DESCRIBE FUNCTION EXTENDED udf_stddev_pop; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_samp.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_samp.q new file mode 100644 index 0000000000000..30d2f24513864 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_samp.q @@ -0,0 +1,4 @@ +DESCRIBE FUNCTION stddev_samp; +DESCRIBE FUNCTION EXTENDED stddev_samp; +DESCRIBE FUNCTION stddev_samp; +DESCRIBE FUNCTION EXTENDED stddev_samp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_string.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_string.q new file mode 100644 index 0000000000000..96e433e1751c5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_string.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION string; +DESCRIBE FUNCTION EXTENDED string; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_struct.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_struct.q new file mode 100644 index 0000000000000..3ee19c83699bb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_struct.q @@ -0,0 +1,9 @@ +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; + +SELECT struct(1), struct(1, "a"), struct(1, "b", 1.5).col1, struct(1, struct("a", 1.5)).col2.col1 +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_substr.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_substr.q new file mode 100644 index 0000000000000..32757bef51c1c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_substr.q @@ -0,0 +1,75 @@ +DESCRIBE FUNCTION substr; +DESCRIBE FUNCTION EXTENDED substr; + +SELECT + substr(null, 1), substr(null, 1, 1), + substr('ABC', null), substr('ABC', null, 1), + substr('ABC', 1, null) +FROM src LIMIT 1; + +SELECT + substr('ABC', 1, 0), substr('ABC', 1, -1), substr('ABC', 2, -100), + substr('ABC', 4), substr('ABC', 4, 100), + substr('ABC', -4), substr('ABC', -4, 100), + 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; + +SELECT + substr('ABCDEFG', 3, 4), substr('ABCDEFG', -5, 4), + substr('ABCDEFG', 3), substr('ABCDEFG', -5), + 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; + +SELECT + substr('ABC', 0, 1), substr('ABC', 0, 2), substr('ABC', 0, 3), substr('ABC', 0, 4), + 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; + +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; + +-- substring() is a synonim of substr(), so just perform some basic tests +SELECT + substring('ABCDEFG', 3, 4), substring('ABCDEFG', -5, 4), + substring('ABCDEFG', 3), substring('ABCDEFG', -5), + 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; + +-- test for binary substr +SELECT + substr(null, 1), substr(null, 1, 1), + substr(ABC, null), substr(ABC, null, 1), + substr(ABC, 1, null), + substr(ABC, 0, 1), substr(ABC, 0, 2), substr(ABC, 0, 3), substr(ABC, 0, 4), + 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), + 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 ( + select CAST(concat(substr(value, 1, 0), 'ABC') as BINARY) as ABC from src LIMIT 1 +) X; + +-- test UTF-8 substr +SELECT + substr("玩", 1), + substr("abc 玩", 5), + substr("abc 玩玩玩 abc", 5), + substr("abc 玩玩玩 abc", 5, 3) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_substring.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_substring.q new file mode 100644 index 0000000000000..d56b828b8b5c6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_substring.q @@ -0,0 +1,3 @@ +-- Synonym. See udf_substr.q +DESCRIBE FUNCTION substring; +DESCRIBE FUNCTION EXTENDED substring; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_subtract.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_subtract.q new file mode 100644 index 0000000000000..3bc583493487a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_subtract.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION -; +DESCRIBE FUNCTION EXTENDED -; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sum.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_sum.q new file mode 100644 index 0000000000000..325bcabd6c9aa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_sum.q @@ -0,0 +1,6 @@ +DESCRIBE FUNCTION sum; +DESCRIBE FUNCTION EXTENDED sum; + +DESCRIBE FUNCTION sum; +DESCRIBE FUNCTION EXTENDED sum; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_tan.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_tan.q new file mode 100644 index 0000000000000..f103da9ecb2de --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_tan.q @@ -0,0 +1,16 @@ +DESCRIBE FUNCTION tan; +DESCRIBE FUNCTION EXTENDED tan; + +SELECT tan(null) +FROM src LIMIT 1; + +SELECT tan(1), tan(6), tan(-1.0) +FROM src LIMIT 1; +DESCRIBE FUNCTION tan; +DESCRIBE FUNCTION EXTENDED tan; + +SELECT tan(null) +FROM src LIMIT 1; + +SELECT tan(1), tan(6), tan(-1.0) +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength.q new file mode 100644 index 0000000000000..322a061d645d7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength.q @@ -0,0 +1,12 @@ +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; + +DROP TEMPORARY FUNCTION testlength; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength2.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength2.q new file mode 100644 index 0000000000000..6de270902f7fb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength2.q @@ -0,0 +1,12 @@ +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; + +DROP TEMPORARY FUNCTION testlength2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_tinyint.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_tinyint.q new file mode 100644 index 0000000000000..56ea53af7766d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_tinyint.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION tinyint; +DESCRIBE FUNCTION EXTENDED tinyint; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_boolean.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_boolean.q new file mode 100644 index 0000000000000..ca23f719f93e3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_boolean.q @@ -0,0 +1,44 @@ +-- 'true' cases: + +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; + +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('Foo' AS STRING) AS BOOLEAN) FROM src LIMIT 1; + +SELECT CAST(CAST('2011-05-06 07:08:09' as timestamp) AS BOOLEAN) FROM src LIMIT 1; + +-- '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.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('' AS STRING) AS BOOLEAN) FROM src LIMIT 1; + +SELECT CAST(CAST(0 as timestamp) AS BOOLEAN) FROM src LIMIT 1; + +-- 'NULL' cases: +SELECT CAST(NULL AS BOOLEAN) FROM src LIMIT 1; + +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 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 STRING) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(NULL as timestamp) AS BOOLEAN) FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_byte.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_byte.q new file mode 100644 index 0000000000000..ded930d1115a4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_byte.q @@ -0,0 +1,15 @@ +-- Conversion of main primitive types to Byte type: +SELECT CAST(NULL AS TINYINT) FROM src LIMIT 1; + +SELECT CAST(TRUE AS TINYINT) FROM src LIMIT 1; + +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(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('-38' AS TINYINT) FROM src LIMIT 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_date.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_date.q new file mode 100644 index 0000000000000..935609bbc3e55 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_date.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION to_date; +DESCRIBE FUNCTION EXTENDED to_date; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_double.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_double.q new file mode 100644 index 0000000000000..b0a248ad70bba --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_double.q @@ -0,0 +1,15 @@ +-- Conversion of main primitive types to Double type: +SELECT CAST(NULL AS DOUBLE) FROM src LIMIT 1; + +SELECT CAST(TRUE AS DOUBLE) FROM src LIMIT 1; + +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(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('-38.14' AS DOUBLE) FROM src LIMIT 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_float.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_float.q new file mode 100644 index 0000000000000..c91d18cc2f5c7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_float.q @@ -0,0 +1,15 @@ +-- Conversion of main primitive types to Float type: +SELECT CAST(NULL AS FLOAT) FROM src LIMIT 1; + +SELECT CAST(TRUE AS FLOAT) FROM src LIMIT 1; + +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(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('-38.14' AS FLOAT) FROM src LIMIT 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_long.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_long.q new file mode 100644 index 0000000000000..45dc6f8bd32fa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_long.q @@ -0,0 +1,15 @@ +-- Conversion of main primitive types to Long type: +SELECT CAST(NULL AS BIGINT) FROM src LIMIT 1; + +SELECT CAST(TRUE AS BIGINT) FROM src LIMIT 1; + +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(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('-38' AS BIGINT) FROM src LIMIT 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_short.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_short.q new file mode 100644 index 0000000000000..7d843c1ea12ea --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_short.q @@ -0,0 +1,15 @@ +-- Conversion of main primitive types to Short type: +SELECT CAST(NULL AS SMALLINT) FROM src LIMIT 1; + +SELECT CAST(TRUE AS SMALLINT) FROM src LIMIT 1; + +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(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('-38' AS SMALLINT) FROM src LIMIT 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_string.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_string.q new file mode 100644 index 0000000000000..3b585e7170c60 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_string.q @@ -0,0 +1,16 @@ +-- Conversion of main primitive types to String type: +SELECT CAST(NULL AS STRING) FROM src LIMIT 1; + +SELECT CAST(TRUE AS STRING) FROM src LIMIT 1; + +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(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('Foo' AS STRING) FROM src LIMIT 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q new file mode 100644 index 0000000000000..3024074bba4b3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q @@ -0,0 +1,29 @@ +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; + +SELECT + '2009-03-20 11:30:01', + to_unix_timestamp('2009-03-20 11:30:01') +FROM oneline; + +SELECT + '2009-03-20', + to_unix_timestamp('2009-03-20', 'yyyy-MM-dd') +FROM oneline; + +SELECT + '2009 Mar 20 11:30:01 am', + to_unix_timestamp('2009 Mar 20 11:30:01 am', 'yyyy MMM dd h:mm:ss a') +FROM oneline; + +SELECT + 'random_string', + to_unix_timestamp('random_string') +FROM oneline; + +-- PPD +explain select * from (select * from src) a where unix_timestamp(a.key) > 10; +explain select * from (select * from src) a where to_unix_timestamp(a.key) > 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_translate.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_translate.q new file mode 100644 index 0000000000000..cba6ff90f643f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_translate.q @@ -0,0 +1,37 @@ +DESCRIBE FUNCTION translate; +DESCRIBE FUNCTION EXTENDED translate; + +-- Create some tables to serve some input data +CREATE TABLE table_input(input STRING); +CREATE TABLE table_translate(input_string STRING, from_string STRING, to_string STRING); + +FROM src INSERT OVERWRITE TABLE table_input SELECT 'abcd' WHERE src.key = 86; +FROM src INSERT OVERWRITE TABLE table_translate SELECT 'abcd', 'ahd', '12' WHERE src.key = 86; + +-- Run some queries on constant input parameters +SELECT translate('abcd', 'ab', '12'), + translate('abcd', 'abc', '12') FROM src LIMIT 1; + +-- 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; + +-- Run some queries where all parameters are coming from table columns +SELECT translate(input_string, from_string, to_string) FROM table_translate LIMIT 1; + +-- 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; + +-- 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; + +-- 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; + +-- Test proper function over UTF-8 characters +SELECT translate('Àbcd', 'À', 'Ã') FROM src LIMIT 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_trim.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_trim.q new file mode 100644 index 0000000000000..1fa34e00c90cf --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_trim.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION trim; +DESCRIBE FUNCTION EXTENDED trim; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_ucase.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_ucase.q new file mode 100644 index 0000000000000..a61b9ba67a4d6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_ucase.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION ucase; +DESCRIBE FUNCTION EXTENDED ucase; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_unhex.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_unhex.q new file mode 100644 index 0000000000000..e80021c8b2c51 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_unhex.q @@ -0,0 +1,19 @@ +DESCRIBE FUNCTION unhex; +DESCRIBE FUNCTION EXTENDED unhex; + +-- Good inputs + +SELECT + unhex('4D7953514C'), + unhex('31323637'), + unhex('61'), + unhex('2D34'), + unhex('') +FROM src limit 1; + +-- Bad inputs +SELECT + unhex('MySQL'), + unhex('G123'), + unhex('\0') +FROM src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_union.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_union.q new file mode 100644 index 0000000000000..9140d22fb58c7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_union.q @@ -0,0 +1,11 @@ +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; + +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_unix_timestamp.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_unix_timestamp.q new file mode 100644 index 0000000000000..89288a1193c87 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_unix_timestamp.q @@ -0,0 +1,27 @@ +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; + +SELECT + '2009-03-20 11:30:01', + unix_timestamp('2009-03-20 11:30:01') +FROM oneline; + +SELECT + '2009-03-20', + unix_timestamp('2009-03-20', 'yyyy-MM-dd') +FROM oneline; + +SELECT + '2009 Mar 20 11:30:01 am', + unix_timestamp('2009 Mar 20 11:30:01 am', 'yyyy MMM dd h:mm:ss a') +FROM oneline; + +SELECT + 'random_string', + unix_timestamp('random_string') +FROM oneline; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_upper.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_upper.q new file mode 100644 index 0000000000000..2a9d5ed4d6c90 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_upper.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION upper; +DESCRIBE FUNCTION EXTENDED upper; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_var_pop.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_var_pop.q new file mode 100644 index 0000000000000..7761506f28cbb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_var_pop.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION var_pop; +DESCRIBE FUNCTION EXTENDED var_pop; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_var_samp.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_var_samp.q new file mode 100644 index 0000000000000..de68e499005ef --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_var_samp.q @@ -0,0 +1,4 @@ +DESCRIBE FUNCTION var_samp; +DESCRIBE FUNCTION EXTENDED var_samp; +DESCRIBE FUNCTION var_samp; +DESCRIBE FUNCTION EXTENDED var_samp; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_variance.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_variance.q new file mode 100644 index 0000000000000..579cc8e68f49c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_variance.q @@ -0,0 +1,10 @@ +DESCRIBE FUNCTION variance; +DESCRIBE FUNCTION EXTENDED variance; + +DESCRIBE FUNCTION var_pop; +DESCRIBE FUNCTION EXTENDED var_pop; +DESCRIBE FUNCTION variance; +DESCRIBE FUNCTION EXTENDED variance; + +DESCRIBE FUNCTION var_pop; +DESCRIBE FUNCTION EXTENDED var_pop; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_weekofyear.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_weekofyear.q new file mode 100644 index 0000000000000..4b7b4ea55aa81 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_weekofyear.q @@ -0,0 +1,6 @@ +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; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_when.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_when.q new file mode 100644 index 0000000000000..d94a594f026c5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_when.q @@ -0,0 +1,57 @@ +DESCRIBE FUNCTION when; +DESCRIBE FUNCTION EXTENDED when; + +EXPLAIN +SELECT CASE + WHEN 1=1 THEN 2 + WHEN 1=3 THEN 4 + ELSE 5 + END, + CASE + WHEN 6=7 THEN 8 + ELSE 9 + END, + CASE + WHEN 10=11 THEN 12 + WHEN 13=13 THEN 14 + END, + CASE + WHEN 15=16 THEN 17 + WHEN 18=19 THEN 20 + END, + CASE + WHEN 21=22 THEN NULL + WHEN 23=23 THEN 24 + END, + CASE + WHEN 25=26 THEN 27 + WHEN 28=28 THEN NULL + END +FROM src LIMIT 1; + +SELECT CASE + WHEN 1=1 THEN 2 + WHEN 1=3 THEN 4 + ELSE 5 + END, + CASE + WHEN 6=7 THEN 8 + ELSE 9 + END, + CASE + WHEN 10=11 THEN 12 + WHEN 13=13 THEN 14 + END, + CASE + WHEN 15=16 THEN 17 + WHEN 18=19 THEN 20 + END, + CASE + WHEN 21=22 THEN NULL + WHEN 23=23 THEN 24 + END, + CASE + WHEN 25=26 THEN 27 + WHEN 28=28 THEN NULL + END +FROM src LIMIT 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath.q new file mode 100644 index 0000000000000..fca1ba11d85b2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath.q @@ -0,0 +1,8 @@ +DESCRIBE FUNCTION xpath ; +DESCRIBE FUNCTION EXTENDED xpath ; + +SELECT xpath ('b1b2b3c1c2', 'a/text()') FROM src LIMIT 1 ; +SELECT xpath ('b1b2b3c1c2', 'a/*/text()') FROM src LIMIT 1 ; +SELECT xpath ('b1b2b3c1c2', 'a/b/text()') FROM src LIMIT 1 ; +SELECT xpath ('b1b2b3c1c2', 'a/c/text()') FROM src LIMIT 1 ; +SELECT xpath ('b1b2b3c1c2', 'a/*[@class="bb"]/text()') FROM src LIMIT 1 ; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_boolean.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_boolean.q new file mode 100644 index 0000000000000..3a6e613eb6a0e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_boolean.q @@ -0,0 +1,9 @@ +DESCRIBE FUNCTION xpath_boolean ; +DESCRIBE FUNCTION EXTENDED xpath_boolean ; + +SELECT xpath_boolean ('b', 'a/b') FROM src LIMIT 1 ; +SELECT xpath_boolean ('b', 'a/c') FROM src LIMIT 1 ; +SELECT xpath_boolean ('b', 'a/b = "b"') FROM src LIMIT 1 ; +SELECT xpath_boolean ('b', 'a/b = "c"') FROM src LIMIT 1 ; +SELECT xpath_boolean ('10', 'a/b < 10') FROM src LIMIT 1 ; +SELECT xpath_boolean ('10', 'a/b = 10') FROM src LIMIT 1 ; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_double.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_double.q new file mode 100644 index 0000000000000..4328747f44497 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_double.q @@ -0,0 +1,14 @@ +DESCRIBE FUNCTION xpath_number ; +DESCRIBE FUNCTION EXTENDED xpath_number ; + +DESCRIBE FUNCTION xpath_double ; +DESCRIBE FUNCTION EXTENDED xpath_double ; + +SELECT xpath_double ('this is not a number', 'a') FROM src LIMIT 1 ; +SELECT xpath_double ('this 2 is not a number', 'a') FROM src LIMIT 1 ; +SELECT xpath_double ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; +SELECT xpath_double ('try a boolean', 'a = 10') FROM src LIMIT 1 ; +SELECT xpath_double ('1248', 'a/b') FROM src LIMIT 1 ; +SELECT xpath_double ('1248', 'sum(a/*)') FROM src LIMIT 1 ; +SELECT xpath_double ('1248', 'sum(a/b)') FROM src LIMIT 1 ; +SELECT xpath_double ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_float.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_float.q new file mode 100644 index 0000000000000..1f1482833c858 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_float.q @@ -0,0 +1,11 @@ +DESCRIBE FUNCTION xpath_float ; +DESCRIBE FUNCTION EXTENDED xpath_float ; + +SELECT xpath_float ('this is not a number', 'a') FROM src LIMIT 1 ; +SELECT xpath_float ('this 2 is not a number', 'a') FROM src LIMIT 1 ; +SELECT xpath_float ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; +SELECT xpath_float ('try a boolean', 'a = 10') FROM src LIMIT 1 ; +SELECT xpath_float ('1248', 'a/b') FROM src LIMIT 1 ; +SELECT xpath_float ('1248', 'sum(a/*)') FROM src LIMIT 1 ; +SELECT xpath_float ('1248', 'sum(a/b)') FROM src LIMIT 1 ; +SELECT xpath_float ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_int.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_int.q new file mode 100644 index 0000000000000..9b50bd9eeb365 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_int.q @@ -0,0 +1,11 @@ +DESCRIBE FUNCTION xpath_int ; +DESCRIBE FUNCTION EXTENDED xpath_int ; + +SELECT xpath_int ('this is not a number', 'a') FROM src LIMIT 1 ; +SELECT xpath_int ('this 2 is not a number', 'a') FROM src LIMIT 1 ; +SELECT xpath_int ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; +SELECT xpath_int ('try a boolean', 'a = 10') FROM src LIMIT 1 ; +SELECT xpath_int ('1248', 'a/b') FROM src LIMIT 1 ; +SELECT xpath_int ('1248', 'sum(a/*)') FROM src LIMIT 1 ; +SELECT xpath_int ('1248', 'sum(a/b)') FROM src LIMIT 1 ; +SELECT xpath_int ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_long.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_long.q new file mode 100644 index 0000000000000..04ee61b1f0e9d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_long.q @@ -0,0 +1,11 @@ +DESCRIBE FUNCTION xpath_long ; +DESCRIBE FUNCTION EXTENDED xpath_long ; + +SELECT xpath_long ('this is not a number', 'a') FROM src LIMIT 1 ; +SELECT xpath_long ('this 2 is not a number', 'a') FROM src LIMIT 1 ; +SELECT xpath_long ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; +SELECT xpath_long ('try a boolean', 'a = 10') FROM src LIMIT 1 ; +SELECT xpath_long ('1248', 'a/b') FROM src LIMIT 1 ; +SELECT xpath_long ('1248', 'sum(a/*)') FROM src LIMIT 1 ; +SELECT xpath_long ('1248', 'sum(a/b)') FROM src LIMIT 1 ; +SELECT xpath_long ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_short.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_short.q new file mode 100644 index 0000000000000..6a1abdc0bdb21 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_short.q @@ -0,0 +1,11 @@ +DESCRIBE FUNCTION xpath_short ; +DESCRIBE FUNCTION EXTENDED xpath_short ; + +SELECT xpath_short ('this is not a number', 'a') FROM src LIMIT 1 ; +SELECT xpath_short ('this 2 is not a number', 'a') FROM src LIMIT 1 ; +SELECT xpath_short ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; +SELECT xpath_short ('try a boolean', 'a = 10') FROM src LIMIT 1 ; +SELECT xpath_short ('1248', 'a/b') FROM src LIMIT 1 ; +SELECT xpath_short ('1248', 'sum(a/*)') FROM src LIMIT 1 ; +SELECT xpath_short ('1248', 'sum(a/b)') FROM src LIMIT 1 ; +SELECT xpath_short ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_string.q b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_string.q new file mode 100644 index 0000000000000..ebbc913b71bfb --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_string.q @@ -0,0 +1,11 @@ +DESCRIBE FUNCTION xpath_string ; +DESCRIBE FUNCTION EXTENDED xpath_string ; + +SELECT xpath_string ('bbcc', 'a') FROM src LIMIT 1 ; +SELECT xpath_string ('bbcc', 'a/b') FROM src LIMIT 1 ; +SELECT xpath_string ('bbcc', 'a/c') FROM src LIMIT 1 ; +SELECT xpath_string ('bbcc', 'a/d') FROM src LIMIT 1 ; +SELECT xpath_string ('b1b2', '//b') FROM src LIMIT 1 ; +SELECT xpath_string ('b1b2', 'a/b[1]') FROM src LIMIT 1 ; +SELECT xpath_string ('b1b2', 'a/b[2]') FROM src LIMIT 1 ; +SELECT xpath_string ('b1b2', 'a/b[@id="b_2"]') FROM src LIMIT 1 ; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udtf_explode.q b/src/test/hive/ql/src/test/queries/clientpositive/udtf_explode.q new file mode 100644 index 0000000000000..638a4e9ca57e7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udtf_explode.q @@ -0,0 +1,25 @@ +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; + +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; + +EXPLAIN SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal) FROM src LIMIT 3; +EXPLAIN EXTENDED SELECT a.myKey, a.myVal, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal) FROM src LIMIT 3) a GROUP BY a.myKey, a.myVal; + +SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal) FROM src LIMIT 3; +SELECT a.myKey, a.myVal, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal) FROM src LIMIT 3) a GROUP BY a.myKey, a.myVal; + +SELECT src.key, myCol FROM src lateral view explode(array(1,2,3)) x AS myCol LIMIT 3; +SELECT src.key, myKey, myVal FROM src lateral view explode(map(1,'one',2,'two',3,'three')) x AS myKey,myVal LIMIT 3; + +-- HIVE-4295 +SELECT BLOCK__OFFSET__INSIDE__FILE, src.key, myKey, myVal FROM src lateral view explode(map(1,'one',2,'two',3,'three')) x AS myKey,myVal LIMIT 3; + +set hive.optimize.cp=false; +SELECT src.key, myKey, myVal FROM src lateral view explode(map(1,'one',2,'two',3,'three')) x AS myKey,myVal LIMIT 3; +SELECT BLOCK__OFFSET__INSIDE__FILE, src.key, myKey, myVal FROM src lateral view explode(map(1,'one',2,'two',3,'three')) x AS myKey,myVal LIMIT 3; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udtf_json_tuple.q b/src/test/hive/ql/src/test/queries/clientpositive/udtf_json_tuple.q new file mode 100644 index 0000000000000..712d9598c755d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udtf_json_tuple.q @@ -0,0 +1,47 @@ +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 + union all + select '2', '{"f1": "value12", "f3": "value3", "f2": 2, "f4": 4.01}' from src limit 1 + union all + select '3', '{"f1": "value13", "f4": "value44", "f3": "value33", "f2": 2, "f5": 5.01}' from src limit 1 + union all + select '4', cast(null as string) from src limit 1 + union all + select '5', '{"f1": "", "f5": null}' from src limit 1 + union all + select '6', '[invalid JSON string]' from src limit 1 +) s; + +explain +select a.key, b.* from json_t a lateral view json_tuple(a.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') b as f1, f2, f3, f4, f5 order by a.key; + +select a.key, b.* from json_t a lateral view json_tuple(a.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') b as f1, f2, f3, f4, f5 order by a.key; + +explain +select json_tuple(a.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') as (f1, f2, f3, f4, f5) from json_t a order by f1, f2, f3; + +select json_tuple(a.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') as (f1, f2, f3, f4, f5) from json_t a order by f1, f2, f3; + +explain +select a.key, b.f2, b.f5 from json_t a lateral view json_tuple(a.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') b as f1, f2, f3, f4, f5 order by a.key; + +select a.key, b.f2, b.f5 from json_t a lateral view json_tuple(a.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') b as f1, f2, f3, f4, f5 order by a.key; + +explain +select f2, count(*) from json_t a lateral view json_tuple(a.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') b as f1, f2, f3, f4, f5 where f1 is not null group by f2 order by f2; + +select f2, count(*) from json_t a lateral view json_tuple(a.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') b as f1, f2, f3, f4, f5 where f1 is not null group by f2 order by f2; + + +-- Verify that json_tuple can handle new lines in JSON values + +CREATE TABLE dest1(c1 STRING) STORED AS RCFILE; + +INSERT OVERWRITE TABLE dest1 SELECT '{"a":"b\nc"}' FROM src LIMIT 1; + +SELECT * FROM dest1; + +SELECT json FROM dest1 a LATERAL VIEW json_tuple(c1, 'a') b AS json; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q b/src/test/hive/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q new file mode 100644 index 0000000000000..055e39b2973a6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q @@ -0,0 +1,42 @@ +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 + union all + select '2', 'https://www.socs.uts.edu.au:80/MosaicDocs-old/url-primer.html?k1=tps#chapter1' from src limit 1 + union all + select '3', 'ftp://sites.google.com/a/example.com/site/page' from src limit 1 + union all + select '4', cast(null as string) from src limit 1 + union all + select '5', 'htttp://' from src limit 1 + union all + select '6', '[invalid url string]' from src limit 1 +) s; + +describe function parse_url_tuple; +describe function extended parse_url_tuple; + +explain +select a.key, b.* from url_t a lateral view parse_url_tuple(a.fullurl, 'HOST', 'PATH', 'QUERY', 'REF', 'PROTOCOL', 'FILE', 'AUTHORITY', 'USERINFO', 'QUERY:k1') b as ho, pa, qu, re, pr, fi, au, us, qk1 order by a.key; + +select a.key, b.* from url_t a lateral view parse_url_tuple(a.fullurl, 'HOST', 'PATH', 'QUERY', 'REF', 'PROTOCOL', 'FILE', 'AUTHORITY', 'USERINFO', 'QUERY:k1') b as ho, pa, qu, re, pr, fi, au, us, qk1 order by a.key; + +explain +select parse_url_tuple(a.fullurl, 'HOST', 'PATH', 'QUERY', 'REF', 'PROTOCOL', 'FILE', 'AUTHORITY', 'USERINFO', 'QUERY:k1') as (ho, pa, qu, re, pr, fi, au, us, qk1) from url_t a order by ho, pa, qu; + +select parse_url_tuple(a.fullurl, 'HOST', 'PATH', 'QUERY', 'REF', 'PROTOCOL', 'FILE', 'AUTHORITY', 'USERINFO', 'QUERY:k1') as (ho, pa, qu, re, pr, fi, au, us, qk1) from url_t a order by ho, pa, qu; + +-- should return null for 'host', 'query', 'QUERY:nonExistCol' +explain +select a.key, b.ho, b.qu, b.qk1, b.err1, b.err2, b.err3 from url_t a lateral view parse_url_tuple(a.fullurl, 'HOST', 'PATH', 'QUERY', 'REF', 'PROTOCOL', 'FILE', 'AUTHORITY', 'USERINFO', 'QUERY:k1', 'host', 'query', 'QUERY:nonExistCol') b as ho, pa, qu, re, pr, fi, au, us, qk1, err1, err2, err3 order by a.key; + +select a.key, b.ho, b.qu, b.qk1, b.err1, b.err2, b.err3 from url_t a lateral view parse_url_tuple(a.fullurl, 'HOST', 'PATH', 'QUERY', 'REF', 'PROTOCOL', 'FILE', 'AUTHORITY', 'USERINFO', 'QUERY:k1', 'host', 'query', 'QUERY:nonExistCol') b as ho, pa, qu, re, pr, fi, au, us, qk1, err1, err2, err3 order by a.key; + + +explain +select ho, count(*) from url_t a lateral view parse_url_tuple(a.fullurl, 'HOST', 'PATH', 'QUERY', 'REF', 'PROTOCOL', 'FILE', 'AUTHORITY', 'USERINFO', 'QUERY:k1') b as ho, pa, qu, re, pr, fi, au, us, qk1 where qk1 is not null group by ho; + +select ho, count(*) from url_t a lateral view parse_url_tuple(a.fullurl, 'HOST', 'PATH', 'QUERY', 'REF', 'PROTOCOL', 'FILE', 'AUTHORITY', 'USERINFO', 'QUERY:k1') b as ho, pa, qu, re, pr, fi, au, us, qk1 where qk1 is not null group by ho; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udtf_stack.q b/src/test/hive/ql/src/test/queries/clientpositive/udtf_stack.q new file mode 100644 index 0000000000000..e7cbae0605954 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/udtf_stack.q @@ -0,0 +1,7 @@ +DESCRIBE FUNCTION stack; + +EXPLAIN SELECT x, y FROM src LATERAL VIEW STACK(2, 'x', array(1), 'z') a AS x, y LIMIT 2; +EXPLAIN SELECT x, y FROM src LATERAL VIEW STACK(2, 'x', array(1), 'z', array(4)) a AS x, y LIMIT 2; + +SELECT x, y FROM src LATERAL VIEW STACK(2, 'x', array(1), 'z') a AS x, y LIMIT 2; +SELECT x, y FROM src LATERAL VIEW STACK(2, 'x', array(1), 'z', array(4)) a AS x, y LIMIT 2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/unicode_notation.q b/src/test/hive/ql/src/test/queries/clientpositive/unicode_notation.q new file mode 100644 index 0000000000000..b3397363e1943 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/unicode_notation.q @@ -0,0 +1,13 @@ +-- HIVE-4618 hive should accept unicode notation like \uxxxx + +CREATE TABLE k1( a string)ROW FORMAT DELIMITED FIELDS TERMINATED BY '\u0001'; +desc formatted k1; +drop table k1; + +CREATE TABLE k1( a string)ROW FORMAT DELIMITED FIELDS TERMINATED BY '\001'; +desc formatted k1; +drop table k1; + +CREATE TABLE k1( a string)ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'; +desc formatted k1; +drop table k1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union.q b/src/test/hive/ql/src/test/queries/clientpositive/union.q new file mode 100644 index 0000000000000..91bbd1b07f08b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union.q @@ -0,0 +1,18 @@ +-- union case: both subqueries are map jobs on same input, followed by filesink + +EXPLAIN +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.*; + +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.*; + +dfs -cat ../build/ql/test/data/warehouse/union.out/*; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union10.q b/src/test/hive/ql/src/test/queries/clientpositive/union10.q new file mode 100644 index 0000000000000..8a5c5f78c8469 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union10.q @@ -0,0 +1,25 @@ +set hive.map.aggr = true; + +-- union case: all subqueries are a map-reduce jobs, 3 way union, same input for all sub-queries, followed by filesink + +create table tmptable(key string, value int); + +explain +insert overwrite table tmptable + select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src s2 + UNION ALL + select 'tst3' as key, count(1) as value from src s3) unionsrc; + + +insert overwrite table tmptable + select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src s2 + UNION ALL + select 'tst3' as key, count(1) as value from src s3) unionsrc; + + +select * from tmptable x sort by x.key; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union11.q b/src/test/hive/ql/src/test/queries/clientpositive/union11.q new file mode 100644 index 0000000000000..77dc2efec5692 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union11.q @@ -0,0 +1,20 @@ +set hive.map.aggr = true; + +-- union case: all subqueries are a map-reduce jobs, 3 way union, same input for all sub-queries, followed by reducesink + +explain + select unionsrc.key, count(1) FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src s2 + UNION ALL + select 'tst3' as key, count(1) as value from src s3) unionsrc group by unionsrc.key; + + + select unionsrc.key, count(1) FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src s2 + UNION ALL + select 'tst3' as key, count(1) as value from src s3) unionsrc group by unionsrc.key; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union12.q b/src/test/hive/ql/src/test/queries/clientpositive/union12.q new file mode 100644 index 0000000000000..b0893df97be67 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union12.q @@ -0,0 +1,23 @@ +set hive.map.aggr = true; + +-- union case: all subqueries are a map-reduce jobs, 3 way union, different inputs for all sub-queries, followed by filesink + +create table tmptable(key string, value int); + +explain +insert overwrite table tmptable + select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src1 s2 + UNION ALL + select 'tst3' as key, count(1) as value from srcbucket s3) unionsrc; + + +insert overwrite table tmptable + select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src1 s2 + UNION ALL + select 'tst3' as key, count(1) as value from srcbucket s3) unionsrc; + +select * from tmptable x sort by x.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union13.q b/src/test/hive/ql/src/test/queries/clientpositive/union13.q new file mode 100644 index 0000000000000..8bee1d7993570 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union13.q @@ -0,0 +1,8 @@ +-- union case: both subqueries are a map-only jobs, same input, followed by filesink + +explain + select unionsrc.key, unionsrc.value FROM (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) unionsrc; + +select unionsrc.key, unionsrc.value FROM (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) unionsrc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union14.q b/src/test/hive/ql/src/test/queries/clientpositive/union14.q new file mode 100644 index 0000000000000..4437ad8d01e49 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union14.q @@ -0,0 +1,16 @@ +set hive.map.aggr = true; + +-- union case: 1 subquery is a map-reduce job, different inputs for sub-queries, followed by reducesink + +explain + select unionsrc.key, count(1) FROM (select s2.key as key, s2.value as value from src1 s2 + UNION ALL + select 'tst1' as key, cast(count(1) as string) as value from src s1) + unionsrc group by unionsrc.key; + + + + select unionsrc.key, count(1) FROM (select s2.key as key, s2.value as value from src1 s2 + UNION ALL + select 'tst1' as key, cast(count(1) as string) as value from src s1) + unionsrc group by unionsrc.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union15.q b/src/test/hive/ql/src/test/queries/clientpositive/union15.q new file mode 100644 index 0000000000000..3080b077e9d35 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union15.q @@ -0,0 +1,18 @@ +set hive.map.aggr = true; + +-- union case: 1 subquery is a map-reduce job, different inputs for sub-queries, followed by reducesink + +explain + select unionsrc.key, count(1) FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src1 s2 + UNION ALL + select s3.key as key, s3.value as value from src1 s3) unionsrc group by unionsrc.key; + + select unionsrc.key, count(1) FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src1 s2 + UNION ALL + select s3.key as key, s3.value as value from src1 s3) unionsrc group by unionsrc.key; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union16.q b/src/test/hive/ql/src/test/queries/clientpositive/union16.q new file mode 100644 index 0000000000000..1df68b037ebe5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union16.q @@ -0,0 +1,63 @@ +EXPLAIN +SELECT count(1) FROM ( + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src) src; + + +SELECT count(1) FROM ( + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src UNION ALL + SELECT key, value FROM src) src; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union17.q b/src/test/hive/ql/src/test/queries/clientpositive/union17.q new file mode 100644 index 0000000000000..34b0e8cc6335f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union17.q @@ -0,0 +1,20 @@ +CREATE TABLE DEST1(key STRING, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key STRING, val1 STRING, val2 STRING) STORED AS TEXTFILE; + +-- union case:map-reduce sub-queries followed by multi-table insert + +explain +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2) unionsrc +INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key +INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key, unionsrc.value; + +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2) unionsrc +INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key +INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, COUNT(DISTINCT SUBSTR(unionsrc.value,5)) GROUP BY unionsrc.key, unionsrc.value; + +SELECT DEST1.* FROM DEST1; +SELECT DEST2.* FROM DEST2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union18.q b/src/test/hive/ql/src/test/queries/clientpositive/union18.q new file mode 100644 index 0000000000000..6207730963173 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union18.q @@ -0,0 +1,20 @@ +CREATE TABLE DEST1(key STRING, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key STRING, val1 STRING, val2 STRING) STORED AS TEXTFILE; + +-- union case:map-reduce sub-queries followed by multi-table insert + +explain +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2) unionsrc +INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, unionsrc.value +INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, unionsrc.value; + +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2) unionsrc +INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, unionsrc.value +INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, unionsrc.value; + +SELECT DEST1.* FROM DEST1 SORT BY DEST1.key, DEST1.value; +SELECT DEST2.* FROM DEST2 SORT BY DEST2.key, DEST2.val1, DEST2.val2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union19.q b/src/test/hive/ql/src/test/queries/clientpositive/union19.q new file mode 100644 index 0000000000000..dc61b74d02f0a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union19.q @@ -0,0 +1,26 @@ + + + +CREATE TABLE DEST1(key STRING, value STRING) STORED AS TEXTFILE; +CREATE TABLE DEST2(key STRING, val1 STRING, val2 STRING) STORED AS TEXTFILE; + +-- union case:map-reduce sub-queries followed by multi-table insert + +explain +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2) unionsrc +INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, count(unionsrc.value) group by unionsrc.key +INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, unionsrc.value; + +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2) unionsrc +INSERT OVERWRITE TABLE DEST1 SELECT unionsrc.key, count(unionsrc.value) group by unionsrc.key +INSERT OVERWRITE TABLE DEST2 SELECT unionsrc.key, unionsrc.value, unionsrc.value; + +SELECT DEST1.* FROM DEST1 SORT BY DEST1.key, DEST1.value; +SELECT DEST2.* FROM DEST2 SORT BY DEST2.key, DEST2.val1, DEST2.val2; + + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union2.q b/src/test/hive/ql/src/test/queries/clientpositive/union2.q new file mode 100644 index 0000000000000..581cbebd861e9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union2.q @@ -0,0 +1,8 @@ +-- union case: both subqueries are map-reduce jobs on same input, followed by reduce sink + +explain + select count(1) FROM (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) unionsrc; + +select count(1) FROM (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) unionsrc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union20.q b/src/test/hive/ql/src/test/queries/clientpositive/union20.q new file mode 100644 index 0000000000000..267262e371e18 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union20.q @@ -0,0 +1,22 @@ +-- union :map-reduce sub-queries followed by join + +explain +SELECT unionsrc1.key, unionsrc1.value, unionsrc2.key, unionsrc2.value +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2 where s2.key < 10) unionsrc1 +JOIN + (select 'tst1' as key, cast(count(1) as string) as value from src s3 + UNION ALL + select s4.key as key, s4.value as value from src s4 where s4.key < 10) unionsrc2 +ON (unionsrc1.key = unionsrc2.key); + +SELECT unionsrc1.key, unionsrc1.value, unionsrc2.key, unionsrc2.value +FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src s2 where s2.key < 10) unionsrc1 +JOIN + (select 'tst1' as key, cast(count(1) as string) as value from src s3 + UNION ALL + select s4.key as key, s4.value as value from src s4 where s4.key < 10) unionsrc2 +ON (unionsrc1.key = unionsrc2.key); diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union21.q b/src/test/hive/ql/src/test/queries/clientpositive/union21.q new file mode 100644 index 0000000000000..8185994655228 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union21.q @@ -0,0 +1,30 @@ +-- union of constants, udf outputs, and columns from text table and thrift table + +explain +SELECT key, count(1) +FROM ( + SELECT '1' as key from src + UNION ALL + SELECT reverse(key) as key from src + UNION ALL + SELECT key as key from src + UNION ALL + SELECT astring as key from src_thrift + UNION ALL + SELECT lstring[0] as key from src_thrift +) union_output +GROUP BY key; + +SELECT key, count(1) +FROM ( + SELECT '1' as key from src + UNION ALL + SELECT reverse(key) as key from src + UNION ALL + SELECT key as key from src + UNION ALL + SELECT astring as key from src_thrift + UNION ALL + SELECT lstring[0] as key from src_thrift +) union_output +GROUP BY key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union22.q b/src/test/hive/ql/src/test/queries/clientpositive/union22.q new file mode 100644 index 0000000000000..d0b125682f84e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union22.q @@ -0,0 +1,43 @@ +create table dst_union22(k1 string, k2 string, k3 string, k4 string) partitioned by (ds string); +create table dst_union22_delta(k0 string, k1 string, k2 string, k3 string, k4 string, k5 string) partitioned by (ds string); + +insert overwrite table dst_union22 partition (ds='1') +select key, value, key , value from src; + +insert overwrite table dst_union22_delta partition (ds='1') +select key, key, value, key, value, value from src; + +set hive.merge.mapfiles=false; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +-- Since the inputs are small, it should be automatically converted to mapjoin + +explain extended +insert overwrite table dst_union22 partition (ds='2') +select * from +( +select k1 as k1, k2 as k2, k3 as k3, k4 as k4 from dst_union22_delta where ds = '1' and k0 <= 50 +union all +select a.k1 as k1, a.k2 as k2, b.k3 as k3, b.k4 as k4 +from dst_union22 a left outer join (select * from dst_union22_delta where ds = '1' and k0 > 50) b on +a.k1 = b.k1 and a.ds='1' +where a.k1 > 20 +) +subq; + +insert overwrite table dst_union22 partition (ds='2') +select * from +( +select k1 as k1, k2 as k2, k3 as k3, k4 as k4 from dst_union22_delta where ds = '1' and k0 <= 50 +union all +select a.k1 as k1, a.k2 as k2, b.k3 as k3, b.k4 as k4 +from dst_union22 a left outer join (select * from dst_union22_delta where ds = '1' and k0 > 50) b on +a.k1 = b.k1 and a.ds='1' +where a.k1 > 20 +) +subq; + +select * from dst_union22 where ds = '2' order by k1, k2, k3, k4; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union23.q b/src/test/hive/ql/src/test/queries/clientpositive/union23.q new file mode 100644 index 0000000000000..41dca46ad2369 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union23.q @@ -0,0 +1,17 @@ +explain +select s.key2, s.value2 +from ( + select transform(key, value) using 'cat' as (key2, value2) + from src + union all + select key as key2, value as value2 from src) s +order by s.key2, s.value2; + +select s.key2, s.value2 +from ( + select transform(key, value) using 'cat' as (key2, value2) + from src + union all + select key as key2, value as value2 from src) s +order by s.key2, s.value2; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union24.q b/src/test/hive/ql/src/test/queries/clientpositive/union24.q new file mode 100644 index 0000000000000..e8f0e14076080 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union24.q @@ -0,0 +1,70 @@ +create table src2 as select key, count(1) as count from src group by key; +create table src3 as select * from src2; +create table src4 as select * from src2; +create table src5 as select * from src2; + + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + + +explain extended +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select key, count from src4 where key < 10 + union all + select key, count(1) as count from src5 where key < 10 group by key +)s +order by s.key ASC, s.count ASC; + +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select key, count from src4 where key < 10 + union all + select key, count(1) as count from src5 where key < 10 group by key +)s +order by s.key ASC, s.count ASC; + +explain extended +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, b.count as count from src4 a join src5 b on a.key=b.key where a.key < 10 +)s +order by s.key ASC, s.count ASC; + +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, b.count as count from src4 a join src5 b on a.key=b.key where a.key < 10 +)s +order by s.key ASC, s.count ASC; + +explain extended +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, count(1) as count from src4 a join src5 b on a.key=b.key where a.key < 10 group by a.key +)s +order by s.key ASC, s.count ASC; + +select s.key, s.count from ( + select key, count from src2 where key < 10 + union all + select key, count from src3 where key < 10 + union all + select a.key as key, count(1) as count from src4 a join src5 b on a.key=b.key where a.key < 10 group by a.key +)s +order by s.key ASC, s.count ASC; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union25.q b/src/test/hive/ql/src/test/queries/clientpositive/union25.q new file mode 100644 index 0000000000000..04e782e2a5e34 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union25.q @@ -0,0 +1,23 @@ +create table tmp_srcpart like srcpart; + +insert overwrite table tmp_srcpart partition (ds='2008-04-08', hr='11') +select key, value from srcpart where ds='2008-04-08' and hr='11'; + +explain +create table tmp_unionall as +SELECT count(1) as counts, key, value +FROM +( + SELECT key, value FROM srcpart a WHERE a.ds='2008-04-08' and a.hr='11' + + UNION ALL + + SELECT key, key as value FROM ( + SELECT distinct key FROM ( + SELECT key, value FROM tmp_srcpart a WHERE a.ds='2008-04-08' and a.hr='11' + UNION ALL + SELECT key, value FROM tmp_srcpart b WHERE b.ds='2008-04-08' and b.hr='11' + )t + ) master_table +) a GROUP BY key, value +; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union26.q b/src/test/hive/ql/src/test/queries/clientpositive/union26.q new file mode 100644 index 0000000000000..dce7bc3425117 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union26.q @@ -0,0 +1,89 @@ +EXPLAIN +SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value +; + +SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value order by key, value +; + + +SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value +; + +SELECT +count(1) as counts, +key, +value +FROM +( + +SELECT +a.key, a.value +FROM srcpart a JOIN srcpart b +ON a.ds='2008-04-08' and a.hr='11' and b.ds='2008-04-08' and b.hr='12' +AND a.key = b.key + +UNION ALL + +select key, value +FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +WHERE ds='2008-04-08' and hr='11' +) a +group by key, value order by key, value +; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union27.q b/src/test/hive/ql/src/test/queries/clientpositive/union27.q new file mode 100644 index 0000000000000..e0fccfcd72820 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union27.q @@ -0,0 +1,4 @@ +create table jackson_sev_same as select * from src; +create table dim_pho as select * from src; +create table jackson_sev_add as select * from src; +select b.* from jackson_sev_same a join (select * from dim_pho union all select * from jackson_sev_add)b on a.key=b.key and b.key=97; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union28.q b/src/test/hive/ql/src/test/queries/clientpositive/union28.q new file mode 100644 index 0000000000000..6aebfeb41a3c7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union28.q @@ -0,0 +1,30 @@ +create table union_subq_union(key int, value string); + +explain +insert overwrite table union_subq_union +select * from ( + select key, value from src + union all + select key, value from + ( + select key, value, count(1) from src group by key, value + union all + select key, value, count(1) from src group by key, value + ) subq +) a +; + +insert overwrite table union_subq_union +select * from ( + select key, value from src + union all + select key, value from + ( + select key, value, count(1) from src group by key, value + union all + select key, value, count(1) from src group by key, value + ) subq +) a +; + +select * from union_subq_union order by key, value limit 20; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union29.q b/src/test/hive/ql/src/test/queries/clientpositive/union29.q new file mode 100644 index 0000000000000..ea99d4c9f29be --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union29.q @@ -0,0 +1,30 @@ +create table union_subq_union(key int, value string); + +explain +insert overwrite table union_subq_union +select * from ( + select key, value from src + union all + select key, value from + ( + select key, value from src + union all + select key, value from src + ) subq +) a +; + +insert overwrite table union_subq_union +select * from ( + select key, value from src + union all + select key, value from + ( + select key, value from src + union all + select key, value from src + ) subq +) a +; + +select * from union_subq_union order by key, value limit 20; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union3.q b/src/test/hive/ql/src/test/queries/clientpositive/union3.q new file mode 100644 index 0000000000000..b26a2e2799f7a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union3.q @@ -0,0 +1,42 @@ + +explain +SELECT * +FROM ( + SELECT 1 AS id + FROM (SELECT * FROM src LIMIT 1) s1 + CLUSTER BY id + UNION ALL + SELECT 2 AS id + FROM (SELECT * FROM src LIMIT 1) s1 + CLUSTER BY id + UNION ALL + SELECT 3 AS id + FROM (SELECT * FROM src LIMIT 1) s2 + UNION ALL + SELECT 4 AS id + FROM (SELECT * FROM src LIMIT 1) s2 +) a; + + + +CREATE TABLE union_out (id int); + +insert overwrite table union_out +SELECT * +FROM ( + SELECT 1 AS id + FROM (SELECT * FROM src LIMIT 1) s1 + CLUSTER BY id + UNION ALL + SELECT 2 AS id + FROM (SELECT * FROM src LIMIT 1) s1 + CLUSTER BY id + UNION ALL + SELECT 3 AS id + FROM (SELECT * FROM src LIMIT 1) s2 + UNION ALL + SELECT 4 AS id + FROM (SELECT * FROM src LIMIT 1) s2 +) a; + +select * from union_out cluster by id; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union30.q b/src/test/hive/ql/src/test/queries/clientpositive/union30.q new file mode 100644 index 0000000000000..9763bc3c9a9c3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union30.q @@ -0,0 +1,44 @@ +create table union_subq_union(key int, value string); + +explain +insert overwrite table union_subq_union +select * from ( + +select * from ( + select key, value from src + union all + select key, value from + ( + select key, value, count(1) from src group by key, value + union all + select key, value, count(1) from src group by key, value + ) subq +) a + +union all + +select key, value from src +) aa +; + +insert overwrite table union_subq_union +select * from ( + +select * from ( + select key, value from src + union all + select key, value from + ( + select key, value, count(1) from src group by key, value + union all + select key, value, count(1) from src group by key, value + ) subq +) a + +union all + +select key, value from src +) aa +; + +select * from union_subq_union order by key, value limit 20; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union31.q b/src/test/hive/ql/src/test/queries/clientpositive/union31.q new file mode 100644 index 0000000000000..1d430fe1de815 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union31.q @@ -0,0 +1,97 @@ +drop table t1; +drop table t2; + + +create table t1 as select * from src where key < 10; +create table t2 as select * from src where key < 10; + +create table t3(key string, cnt int); +create table t4(value string, cnt int); + +explain +from +(select * from t1 + union all + select * from t2 +) x +insert overwrite table t3 + select key, count(1) group by key +insert overwrite table t4 + select value, count(1) group by value; + +from +(select * from t1 + union all + select * from t2 +) x +insert overwrite table t3 + select key, count(1) group by key +insert overwrite table t4 + select value, count(1) group by value; + +select * from t3 order by key; +select * from t4 order by value; + +create table t5(c1 string, cnt int); +create table t6(c1 string, cnt int); + +explain +from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, count(1) as cnt from t2 group by key +) x +insert overwrite table t5 + select c1, sum(cnt) group by c1 +insert overwrite table t6 + select c1, sum(cnt) group by c1; + +from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, count(1) as cnt from t2 group by key +) x +insert overwrite table t5 + select c1, sum(cnt) group by c1 +insert overwrite table t6 + select c1, sum(cnt) group by c1; + +select * from t5 order by c1; +select * from t6 order by c1; + +drop table t1; +drop table t2; + +create table t1 as select * from src where key < 10; +create table t2 as select key, count(1) as cnt from src where key < 10 group by key; + +create table t7(c1 string, cnt int); +create table t8(c1 string, cnt int); + +explain +from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, cnt from t2 +) x +insert overwrite table t7 + select c1, count(1) group by c1 +insert overwrite table t8 + select c1, count(1) group by c1; + +from +( + select key as c1, count(1) as cnt from t1 group by key + union all + select key as c1, cnt from t2 +) x +insert overwrite table t7 + select c1, count(1) group by c1 +insert overwrite table t8 + select c1, count(1) group by c1; + +select * from t7 order by c1; +select * from t8 order by c1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union32.q b/src/test/hive/ql/src/test/queries/clientpositive/union32.q new file mode 100644 index 0000000000000..9abb09bf76d40 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union32.q @@ -0,0 +1,75 @@ +-- This tests various union queries which have columns on one side of the query +-- being of double type and those on the other side another + +CREATE TABLE t1 AS SELECT * FROM src WHERE key < 10; +CREATE TABLE t2 AS SELECT * FROM src WHERE key < 10; + +-- Test simple union with double +EXPLAIN +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t1 +UNION ALL +SELECT CAST(key AS BIGINT) AS key FROM t2) a +ORDER BY key; + +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t1 +UNION ALL +SELECT CAST(key AS BIGINT) AS key FROM t2) a +ORDER BY key; + +-- Test union with join on the left +EXPLAIN +SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key FROM t2) a +ORDER BY key; + +SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key FROM t2) a +ORDER BY key; + +-- Test union with join on the right +EXPLAIN +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key) a +ORDER BY key; + +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key FROM t1 a JOIN t2 b ON a.key = b.key) a +ORDER BY key; + +-- Test union with join on the left selecting multiple columns +EXPLAIN +SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2) a +ORDER BY key; + +SELECT * FROM +(SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key +UNION ALL +SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2) a +ORDER BY key; + +-- Test union with join on the right selecting multiple columns +EXPLAIN +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key) a +ORDER BY key; + +SELECT * FROM +(SELECT CAST(key AS DOUBLE) AS key, CAST(key AS STRING) AS value FROM t2 +UNION ALL +SELECT CAST(a.key AS BIGINT) AS key, CAST(b.key AS DOUBLE) AS value FROM t1 a JOIN t2 b ON a.key = b.key) a +ORDER BY key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union33.q b/src/test/hive/ql/src/test/queries/clientpositive/union33.q new file mode 100644 index 0000000000000..69e46f44d7806 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union33.q @@ -0,0 +1,47 @@ +set hive.groupby.skewindata=true; + +-- This tests that a union all with a map only subquery on one side and a +-- subquery involving two map reduce jobs on the other runs correctly. + +CREATE TABLE test_src (key STRING, value STRING); + +EXPLAIN INSERT OVERWRITE TABLE test_src +SELECT key, value FROM ( + SELECT key, value FROM src + WHERE key = 0 +UNION ALL + SELECT key, COUNT(*) AS value FROM src + GROUP BY key +)a; + +INSERT OVERWRITE TABLE test_src +SELECT key, value FROM ( + SELECT key, value FROM src + WHERE key = 0 +UNION ALL + SELECT key, COUNT(*) AS value FROM src + GROUP BY key +)a; + +SELECT COUNT(*) FROM test_src; + +EXPLAIN INSERT OVERWRITE TABLE test_src +SELECT key, value FROM ( + SELECT key, COUNT(*) AS value FROM src + GROUP BY key +UNION ALL + SELECT key, value FROM src + WHERE key = 0 +)a; + +INSERT OVERWRITE TABLE test_src +SELECT key, value FROM ( + SELECT key, COUNT(*) AS value FROM src + GROUP BY key +UNION ALL + SELECT key, value FROM src + WHERE key = 0 +)a; + +SELECT COUNT(*) FROM test_src; + \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union34.q b/src/test/hive/ql/src/test/queries/clientpositive/union34.q new file mode 100644 index 0000000000000..36bc865be81fe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union34.q @@ -0,0 +1,44 @@ +create table src10_1 (key string, value string); +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 +insert overwrite table src10_1 select * +insert overwrite table src10_2 select * +insert overwrite table src10_3 select * +insert overwrite table src10_4 select *; + +set hive.auto.convert.join=true; +-- When we convert the Join of sub1 and sub0 into a MapJoin, +-- we can use a single MR job to evaluate this entire query. +explain +SELECT * FROM ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key; + +SELECT * FROM ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key; + +set hive.auto.convert.join=false; +-- When we do not convert the Join of sub1 and sub0 into a MapJoin, +-- we need to use two MR jobs to evaluate this query. +-- The first job is for the Join of sub1 and sub2. The second job +-- is for the UNION ALL and ORDER BY. +explain +SELECT * FROM ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key; + +SELECT * FROM ( + SELECT sub1.key,sub1.value FROM (SELECT * FROM src10_1) sub1 JOIN (SELECT * FROM src10_2) sub0 ON (sub0.key = sub1.key) + UNION ALL + SELECT key,value FROM (SELECT * FROM (SELECT * FROM src10_3) sub2 UNION ALL SELECT * FROM src10_4 ) alias0 +) alias1 order by key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union4.q b/src/test/hive/ql/src/test/queries/clientpositive/union4.q new file mode 100644 index 0000000000000..ee13e7ad3dd45 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union4.q @@ -0,0 +1,21 @@ +set hive.map.aggr = true; + +-- union case: both subqueries are map-reduce jobs on same input, followed by filesink + + +create table tmptable(key string, value int); + +explain +insert overwrite table tmptable + select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src s2) unionsrc; + +insert overwrite table tmptable +select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src s2) unionsrc; + +select * from tmptable x sort by x.key; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union5.q b/src/test/hive/ql/src/test/queries/clientpositive/union5.q new file mode 100644 index 0000000000000..9844127e4a4bc --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union5.q @@ -0,0 +1,12 @@ +set hive.map.aggr = true; + +-- union case: both subqueries are map-reduce jobs on same input, followed by reduce sink + +explain + select unionsrc.key, count(1) FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src s2) unionsrc group by unionsrc.key; + +select unionsrc.key, count(1) FROM (select 'tst1' as key, count(1) as value from src s1 + UNION ALL + select 'tst2' as key, count(1) as value from src s2) unionsrc group by unionsrc.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union6.q b/src/test/hive/ql/src/test/queries/clientpositive/union6.q new file mode 100644 index 0000000000000..b793002821147 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union6.q @@ -0,0 +1,21 @@ +set hive.map.aggr = true; + +-- union case: 1 subquery is a map-reduce job, different inputs for sub-queries, followed by filesink + + +create table tmptable(key string, value string); + +explain +insert overwrite table tmptable + select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src1 s2) unionsrc; + +insert overwrite table tmptable +select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src1 s2) unionsrc; + +select * from tmptable x sort by x.key, x.value; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union7.q b/src/test/hive/ql/src/test/queries/clientpositive/union7.q new file mode 100644 index 0000000000000..d66d5960bd187 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union7.q @@ -0,0 +1,14 @@ +set hive.map.aggr = true; + +-- union case: 1 subquery is a map-reduce job, different inputs for sub-queries, followed by reducesink + +explain + select unionsrc.key, count(1) FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src1 s2) unionsrc group by unionsrc.key; + +select unionsrc.key, count(1) FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 + UNION ALL + select s2.key as key, s2.value as value from src1 s2) unionsrc group by unionsrc.key; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union8.q b/src/test/hive/ql/src/test/queries/clientpositive/union8.q new file mode 100644 index 0000000000000..6d5bf673e0881 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union8.q @@ -0,0 +1,10 @@ +-- union case: all subqueries are a map-only jobs, 3 way union, same input for all sub-queries, followed by filesink + +explain + select unionsrc.key, unionsrc.value FROM (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 UNION ALL + select s3.key as key, s3.value as value from src s3) unionsrc; + +select unionsrc.key, unionsrc.value FROM (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 UNION ALL + select s3.key as key, s3.value as value from src s3) unionsrc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union9.q b/src/test/hive/ql/src/test/queries/clientpositive/union9.q new file mode 100644 index 0000000000000..7d4c11b252ab5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union9.q @@ -0,0 +1,10 @@ +-- union case: all subqueries are a map-only jobs, 3 way union, same input for all sub-queries, followed by reducesink + +explain + select count(1) FROM (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 UNION ALL + select s3.key as key, s3.value as value from src s3) unionsrc; + + select count(1) FROM (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 UNION ALL + select s3.key as key, s3.value as value from src s3) unionsrc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_date.q b/src/test/hive/ql/src/test/queries/clientpositive/union_date.q new file mode 100644 index 0000000000000..e332a8af7bb1b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_date.q @@ -0,0 +1,32 @@ +drop table union_date_1; +drop table union_date_2; + +create table union_date_1 ( + ORIGIN_CITY_NAME string, + DEST_CITY_NAME string, + FL_DATE date, + ARR_DELAY float, + FL_NUM int +); + +create table union_date_2 ( + ORIGIN_CITY_NAME string, + DEST_CITY_NAME string, + FL_DATE date, + ARR_DELAY float, + 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; + +select * from ( + select fl_num, fl_date from union_date_1 + union all + select fl_num, fl_date from union_date_2 +) union_result order by fl_date, fl_num; + +drop table union_date_1; +drop table union_date_2; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_lateralview.q b/src/test/hive/ql/src/test/queries/clientpositive/union_lateralview.q new file mode 100644 index 0000000000000..012296d7c3a51 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_lateralview.q @@ -0,0 +1,43 @@ + +create table test_union_lateral_view(key int, arr_ele int, value string); + +EXPLAIN +INSERT OVERWRITE TABLE test_union_lateral_view +SELECT b.key, d.arr_ele, d.value +FROM ( + SELECT c.arr_ele as arr_ele, a.key as key, a.value as value + FROM ( + SELECT key, value, array(1,2,3) as arr + FROM src + + UNION ALL + + SELECT key, value, array(1,2,3) as arr + FROM srcpart + WHERE ds = '2008-04-08' and hr='12' + ) a LATERAL VIEW EXPLODE(arr) c AS arr_ele +) d +LEFT OUTER JOIN src b +ON d.key = b.key +; + +INSERT OVERWRITE TABLE test_union_lateral_view +SELECT b.key, d.arr_ele, d.value +FROM ( + SELECT c.arr_ele as arr_ele, a.key as key, a.value as value + FROM ( + SELECT key, value, array(1,2,3) as arr + FROM src + + UNION ALL + + SELECT key, value, array(1,2,3) as arr + FROM srcpart + WHERE ds = '2008-04-08' and hr='12' + ) a LATERAL VIEW EXPLODE(arr) c AS arr_ele +) d +LEFT OUTER JOIN src b +ON d.key = b.key +; + +select key, arr_ele, value from test_union_lateral_view order by key, arr_ele limit 20; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_null.q b/src/test/hive/ql/src/test/queries/clientpositive/union_null.q new file mode 100644 index 0000000000000..70147070feaca --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_null.q @@ -0,0 +1,2 @@ +-- HIVE-2901 +select x from (select value as x from src union all select NULL as x from src)a limit 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_ppr.q b/src/test/hive/ql/src/test/queries/clientpositive/union_ppr.q new file mode 100644 index 0000000000000..27d2c262def40 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_ppr.q @@ -0,0 +1,16 @@ +EXPLAIN EXTENDED +SELECT * FROM ( + SELECT X.* FROM SRCPART X WHERE X.key < 100 + UNION ALL + SELECT Y.* FROM SRCPART Y WHERE Y.key < 100 +) A +WHERE A.ds = '2008-04-08' +SORT BY A.key, A.value, A.ds, A.hr; + +SELECT * FROM ( + SELECT X.* FROM SRCPART X WHERE X.key < 100 + UNION ALL + SELECT Y.* FROM SRCPART Y WHERE Y.key < 100 +) A +WHERE A.ds = '2008-04-08' +SORT BY A.key, A.value, A.ds, A.hr; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_1.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_1.q new file mode 100644 index 0000000000000..c9f920c648a92 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_1.q @@ -0,0 +1,44 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_10.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_10.q new file mode 100644 index 0000000000000..a072fe3b56e43 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_10.q @@ -0,0 +1,57 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where one of the sub-queries requires a map-reduce +-- job), followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The outer union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b; + +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_11.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_11.q new file mode 100644 index 0000000000000..6250a20210f9f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_11.q @@ -0,0 +1,57 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where also contains map only sub-queries), +-- followed by select star and a file sink. +-- There is no need for the union optimization, since the whole query can be performed +-- in a single map-only job +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b; + +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_12.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_12.q new file mode 100644 index 0000000000000..168eac34a0301 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_12.q @@ -0,0 +1,51 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; +set hive.auto.convert.join=true; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one is a map-join query), followed by select star and a file sink. +-- The union optimization is applied, and the union is removed. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +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; + +explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select a.key as key, b.val as values +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c; + +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select a.key as key, b.val as values +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_13.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_13.q new file mode 100644 index 0000000000000..a88a83e152a5f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_13.q @@ -0,0 +1,51 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; +set hive.auto.convert.join=true; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a mapred query, and the +-- other one is a map-join query), followed by select star and a file sink. +-- The union selectstar optimization should be performed, and the union should be removed. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +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; + +explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select a.key as key, b.val as values +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c; + +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select a.key as key, b.val as values +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_14.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_14.q new file mode 100644 index 0000000000000..e588e8fd62ab9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_14.q @@ -0,0 +1,52 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.auto.convert.join=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a join, which should be performed as a map-join query at runtime), +-- followed by select star and a file sink. +-- The union selectstar optimization should be performed, and the union should be removed. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +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; + +explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select a.key as key, b.val as values +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c; + +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select a.key as key, b.val as values +FROM inputTbl1 a join inputTbl1 b on a.key=b.key +)c; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_15.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_15.q new file mode 100644 index 0000000000000..237f2e7629057 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_15.q @@ -0,0 +1,53 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This tests demonstrates that this optimization works in the presence of dynamic partitions. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +show partitions outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 where ds = '1' order by key, values; +select * from outputTbl1 where ds = '2' order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_16.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_16.q new file mode 100644 index 0000000000000..06d5043f9cf9d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_16.q @@ -0,0 +1,52 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- This test demonstrates that this optimization works in the presence of dynamic partitions. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a; + +desc formatted outputTbl1; +show partitions outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 where ds = '1' order by key, values; +select * from outputTbl1 where ds = '2' order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_17.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_17.q new file mode 100644 index 0000000000000..65b8255c3e954 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_17.q @@ -0,0 +1,49 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need for this optimization, since the query is a map-only query. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a; + +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a; + +desc formatted outputTbl1; +show partitions outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 where ds = '1' order by key, values; +select * from outputTbl1 where ds = '2' order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_18.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_18.q new file mode 100644 index 0000000000000..265acfd35758f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_18.q @@ -0,0 +1,53 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This test demonstrates that the optimization works with dynamic partitions irrespective of the +-- file format of the output file +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a; + +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a; + +desc formatted outputTbl1; + +show partitions outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 where ds = '11' order by key, values; +select * from outputTbl1 where ds = '18' order by key, values; +select * from outputTbl1 where ds is not null order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_19.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_19.q new file mode 100644 index 0000000000000..1450caa2d07a7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_19.q @@ -0,0 +1,90 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +select * from outputTbl1; + +-- filter should be fine +explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7; + +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7; + +select * from outputTbl1; + +-- filters and sub-queries should be fine +explain +insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7; + +insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_2.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_2.q new file mode 100644 index 0000000000000..015c146ecadf0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_2.q @@ -0,0 +1,50 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_20.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_20.q new file mode 100644 index 0000000000000..ac727889273d7 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_20.q @@ -0,0 +1,45 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, the order of the columns in the select list is different. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +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; + +explain +insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_21.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_21.q new file mode 100644 index 0000000000000..f89744624116f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_21.q @@ -0,0 +1,45 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, all the columns are not selected. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +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; + +explain +insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_22.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_22.q new file mode 100644 index 0000000000000..f01053d27d0c0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_22.q @@ -0,0 +1,63 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, some columns are repeated. So, union cannot be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which selects columns from +-- both the sub-qeuries of the union) is pushed above the union. + +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; + +explain +insert overwrite table outputTbl1 +SELECT a.key, a.values, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT a.key, a.values, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +select * from outputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT a.key, concat(a.values, a.values), concat(a.values, a.values) +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT a.key, concat(a.values, a.values), concat(a.values, a.values) +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_23.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_23.q new file mode 100644 index 0000000000000..805dd76274c42 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_23.q @@ -0,0 +1,47 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. One of the sub-queries +-- would have multiple map-reduce jobs. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from + (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) subq2; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from + (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) subq2; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_24.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_24.q new file mode 100644 index 0000000000000..36fd947d6035d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_24.q @@ -0,0 +1,43 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- One sub-query has a double and the other sub-query has a bigint. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_3.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_3.q new file mode 100644 index 0000000000000..da0f1c0473ffe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_3.q @@ -0,0 +1,50 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->remove->filesink optimization +-- Union of 3 subqueries is performed (all of which are map-only queries) +-- followed by select star and a file sink. +-- There is no need for any optimization, since the whole query can be processed in +-- a single map-only job +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_4.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_4.q new file mode 100644 index 0000000000000..18d4730a12024 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_4.q @@ -0,0 +1,45 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set mapred.input.dir.recursive=true; +set hive.merge.smallfiles.avgsize=1; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_5.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_5.q new file mode 100644 index 0000000000000..a6fbeb03ddcc3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_5.q @@ -0,0 +1,51 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_6.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_6.q new file mode 100644 index 0000000000000..7ae5af30c1bae --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_6.q @@ -0,0 +1,41 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (all of which are mapred queries) +-- followed by select star and a file sink in 2 output tables. +-- The optimiaztion does not take affect since it is a multi-table insert. +-- It does not matter, whether the output is merged or not. In this case, +-- merging is turned off + +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; + +explain +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select *; + +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select *; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; +select * from outputTbl2 order by key, values;; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_7.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_7.q new file mode 100644 index 0000000000000..5a639ca11700d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_7.q @@ -0,0 +1,46 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_8.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_8.q new file mode 100644 index 0000000000000..79b84e81181d3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_8.q @@ -0,0 +1,51 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_9.q b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_9.q new file mode 100644 index 0000000000000..f6038384f3fa8 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_remove_9.q @@ -0,0 +1,55 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which contains a union and is map-only), +-- and the other one is a map-reduce query followed by select star and a file sink. +-- There is no need for the outer union. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +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; + +explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b; + +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_script.q b/src/test/hive/ql/src/test/queries/clientpositive/union_script.q new file mode 100644 index 0000000000000..df35859f597c5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_script.q @@ -0,0 +1,7 @@ +select * from ( + select transform(key) using 'cat' as cola from src)s order by cola; + +select * from ( + select transform(key) using 'cat' as cola from src + union all + select transform(key) using 'cat' as cola from src) s order by cola; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_view.q b/src/test/hive/ql/src/test/queries/clientpositive/union_view.q new file mode 100644 index 0000000000000..4f8bafe37a809 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/union_view.q @@ -0,0 +1,84 @@ +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; + +CREATE TABLE src_union_2 (key int, value string) PARTITIONED BY (ds string, part_1 string); +CREATE INDEX src_union_2_key_idx ON TABLE src_union_2(key) AS 'COMPACT' WITH DEFERRED REBUILD; + +CREATE TABLE src_union_3(key int, value string) PARTITIONED BY (ds string, part_1 string, part_2 string); +CREATE INDEX src_union_3_key_idx ON TABLE src_union_3(key) AS 'COMPACT' WITH DEFERRED REBUILD; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +SET hive.optimize.index.filter=true; +SET hive.optimize.index.filter.compact.minsize=0; + +SET hive.exec.pre.hooks=; +SET hive.exec.post.hooks=; +SET hive.semantic.analyzer.hook=; +SET hive.merge.mapfiles=false; +SET hive.merge.mapredfiles=false; + +INSERT OVERWRITE TABLE src_union_1 PARTITION (ds='1') SELECT * FROM src; +ALTER INDEX src_union_1_key_idx ON src_union_1 PARTITION (ds='1') REBUILD; + +INSERT OVERWRITE TABLE src_union_2 PARTITION (ds='2', part_1='1') SELECT * FROM src; +INSERT OVERWRITE TABLE src_union_2 PARTITION (ds='2', part_1='2') SELECT * FROM src; +ALTER INDEX src_union_2_key_idx ON src_union_2 PARTITION (ds='2', part_1='1') REBUILD; +ALTER INDEX src_union_2_key_idx ON src_union_2 PARTITION (ds='2', part_1='2') REBUILD; + +INSERT OVERWRITE TABLE src_union_3 PARTITION (ds='3', part_1='1', part_2='2:3+4') SELECT * FROM src; +INSERT OVERWRITE TABLE src_union_3 PARTITION (ds='3', part_1='2', part_2='2:3+4') SELECT * FROM src; +ALTER INDEX src_union_3_key_idx ON src_union_3 PARTITION (ds='3', part_1='1', part_2='2:3+4') REBUILD; +ALTER INDEX src_union_3_key_idx ON src_union_3 PARTITION (ds='3', part_1='2', part_2='2:3+4') REBUILD; + +EXPLAIN SELECT key, value, ds FROM src_union_1 WHERE key=86 and ds='1'; +EXPLAIN SELECT key, value, ds FROM src_union_2 WHERE key=86 and ds='2'; +EXPLAIN SELECT key, value, ds FROM src_union_3 WHERE key=86 and ds='3'; + +SELECT key, value, ds FROM src_union_1 WHERE key=86 AND ds ='1'; +SELECT key, value, ds FROM src_union_2 WHERE key=86 AND ds ='2'; +SELECT key, value, ds FROM src_union_3 WHERE key=86 AND ds ='3'; + +EXPLAIN SELECT count(1) from src_union_1 WHERE ds ='1'; +EXPLAIN SELECT count(1) from src_union_2 WHERE ds ='2'; +EXPLAIN SELECT count(1) from src_union_3 WHERE ds ='3'; + +SELECT count(1) from src_union_1 WHERE ds ='1'; +SELECT count(1) from src_union_2 WHERE ds ='2'; +SELECT count(1) from src_union_3 WHERE ds ='3'; + +CREATE VIEW src_union_view PARTITIONED ON (ds) as +SELECT key, value, ds FROM ( +SELECT key, value, ds FROM src_union_1 +UNION ALL +SELECT key, value, ds FROM src_union_2 +UNION ALL +SELECT key, value, ds FROM src_union_3 +) subq; + +EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='1'; +EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='2'; +EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='3'; +EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds IS NOT NULL order by ds; + +SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='1'; +SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='2'; +SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='3'; +SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds IS NOT NULL order by ds; + +EXPLAIN SELECT count(1) from src_union_view WHERE ds ='1'; +EXPLAIN SELECT count(1) from src_union_view WHERE ds ='2'; +EXPLAIN SELECT count(1) from src_union_view WHERE ds ='3'; + +SELECT count(1) from src_union_view WHERE ds ='1'; +SELECT count(1) from src_union_view WHERE ds ='2'; +SELECT count(1) from src_union_view WHERE ds ='3'; + +INSERT OVERWRITE TABLE src_union_3 PARTITION (ds='4', part_1='1', part_2='2:3+4') SELECT * FROM src; +ALTER INDEX src_union_3_key_idx ON src_union_3 PARTITION (ds='4', part_1='1', part_2='2:3+4') REBUILD; + +EXPLAIN SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='4'; +SELECT key, value, ds FROM src_union_view WHERE key=86 AND ds ='4'; + +EXPLAIN SELECT count(1) from src_union_view WHERE ds ='4'; +SELECT count(1) from src_union_view WHERE ds ='4'; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/uniquejoin.q b/src/test/hive/ql/src/test/queries/clientpositive/uniquejoin.q new file mode 100644 index 0000000000000..51bcf22bfa0be --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/uniquejoin.q @@ -0,0 +1,25 @@ +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; + +FROM UNIQUEJOIN PRESERVE T1 a (a.key), PRESERVE T2 b (b.key), PRESERVE T3 c (c.key) +SELECT a.key, b.key, c.key; + +FROM UNIQUEJOIN T1 a (a.key), T2 b (b.key), T3 c (c.key) +SELECT a.key, b.key, c.key; + +FROM UNIQUEJOIN T1 a (a.key), T2 b (b.key-1), T3 c (c.key) +SELECT a.key, b.key, c.key; + +FROM UNIQUEJOIN PRESERVE T1 a (a.key, a.val), PRESERVE T2 b (b.key, b.val), PRESERVE T3 c (c.key, c.val) +SELECT a.key, a.val, b.key, b.val, c.key, c.val; + +FROM UNIQUEJOIN PRESERVE T1 a (a.key), T2 b (b.key), PRESERVE T3 c (c.key) +SELECT a.key, b.key, c.key; + +FROM UNIQUEJOIN PRESERVE T1 a (a.key), T2 b(b.key) +SELECT a.key, b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/unset_table_view_property.q b/src/test/hive/ql/src/test/queries/clientpositive/unset_table_view_property.q new file mode 100644 index 0000000000000..f838cd1feb106 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/unset_table_view_property.q @@ -0,0 +1,64 @@ +CREATE TABLE testTable(col1 INT, col2 INT); +SHOW TBLPROPERTIES testTable; + +-- UNSET TABLE PROPERTIES +ALTER TABLE testTable SET TBLPROPERTIES ('a'='1', 'c'='3'); +SHOW TBLPROPERTIES testTable; + +-- UNSET all the properties +ALTER TABLE testTable UNSET TBLPROPERTIES ('a', 'c'); +SHOW TBLPROPERTIES testTable; + +ALTER TABLE testTable SET TBLPROPERTIES ('a'='1', 'c'='3', 'd'='4'); +SHOW TBLPROPERTIES testTable; + +-- UNSET a subset of the properties +ALTER TABLE testTable UNSET TBLPROPERTIES ('a', 'd'); +SHOW TBLPROPERTIES testTable; + +-- the same property being UNSET multiple times +ALTER TABLE testTable UNSET TBLPROPERTIES ('c', 'c', 'c'); +SHOW TBLPROPERTIES testTable; + +ALTER TABLE testTable SET TBLPROPERTIES ('a'='1', 'b' = '2', 'c'='3', 'd'='4'); +SHOW TBLPROPERTIES testTable; + +-- UNSET a subset of the properties and some non-existed properties using IF EXISTS +ALTER TABLE testTable UNSET TBLPROPERTIES IF EXISTS ('b', 'd', 'b', 'f'); +SHOW TBLPROPERTIES testTable; + +-- UNSET a subset of the properties and some non-existed properties using IF EXISTS +ALTER TABLE testTable UNSET TBLPROPERTIES IF EXISTS ('b', 'd', 'c', 'f', 'x', 'y', 'z'); +SHOW TBLPROPERTIES testTable; + +-- UNSET VIEW PROPERTIES +CREATE VIEW testView AS SELECT value FROM src WHERE key=86; +ALTER VIEW testView SET TBLPROPERTIES ('propA'='100', 'propB'='200'); +SHOW TBLPROPERTIES testView; + +-- UNSET all the properties +ALTER VIEW testView UNSET TBLPROPERTIES ('propA', 'propB'); +SHOW TBLPROPERTIES testView; + +ALTER VIEW testView SET TBLPROPERTIES ('propA'='100', 'propC'='300', 'propD'='400'); +SHOW TBLPROPERTIES testView; + +-- UNSET a subset of the properties +ALTER VIEW testView UNSET TBLPROPERTIES ('propA', 'propC'); +SHOW TBLPROPERTIES testView; + +-- the same property being UNSET multiple times +ALTER VIEW testView UNSET TBLPROPERTIES ('propD', 'propD', 'propD'); +SHOW TBLPROPERTIES testView; + +ALTER VIEW testView SET TBLPROPERTIES ('propA'='100', 'propB' = '200', 'propC'='300', 'propD'='400'); +SHOW TBLPROPERTIES testView; + +-- UNSET a subset of the properties and some non-existed properties using IF EXISTS +ALTER VIEW testView UNSET TBLPROPERTIES IF EXISTS ('propC', 'propD', 'propD', 'propC', 'propZ'); +SHOW TBLPROPERTIES testView; + +-- UNSET a subset of the properties and some non-existed properties using IF EXISTS +ALTER VIEW testView UNSET TBLPROPERTIES IF EXISTS ('propB', 'propC', 'propD', 'propF'); +SHOW TBLPROPERTIES testView; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/updateAccessTime.q b/src/test/hive/ql/src/test/queries/clientpositive/updateAccessTime.q new file mode 100644 index 0000000000000..908b6ecfa0778 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/updateAccessTime.q @@ -0,0 +1,30 @@ +drop table tstsrc; + +set hive.exec.pre.hooks = org.apache.hadoop.hive.ql.hooks.PreExecutePrinter,org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables,org.apache.hadoop.hive.ql.hooks.UpdateInputAccessTimeHook$PreExec; + +create table tstsrc as select * from src; +desc extended tstsrc; +select count(1) from tstsrc; +desc extended tstsrc; +drop table tstsrc; + +drop table tstsrcpart; +create table tstsrcpart like srcpart; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + + +insert overwrite table tstsrcpart partition (ds, hr) select key, value, ds, hr from srcpart; + +desc extended tstsrcpart; +desc extended tstsrcpart partition (ds='2008-04-08', hr='11'); +desc extended tstsrcpart partition (ds='2008-04-08', hr='12'); + +select count(1) from tstsrcpart where ds = '2008-04-08' and hr = '11'; + +desc extended tstsrcpart; +desc extended tstsrcpart partition (ds='2008-04-08', hr='11'); +desc extended tstsrcpart partition (ds='2008-04-08', hr='12'); + +drop table tstsrcpart; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_1.q b/src/test/hive/ql/src/test/queries/clientpositive/varchar_1.q new file mode 100644 index 0000000000000..53273b3187a78 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/varchar_1.q @@ -0,0 +1,32 @@ +drop table varchar1; +drop table varchar1_1; + +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; +select * from varchar1 order by key, value limit 2; + +-- insert overwrite, from same/different length varchar +insert overwrite table varchar1 + select cast(key as varchar(10)), cast(value as varchar(15)) from src order by key, value limit 2; +select key, value from varchar1 order by key, value; + +-- insert overwrite, from string +insert overwrite table varchar1 + select key, value from src order by key, value limit 2; +select key, value from varchar1 order by key, value; + +-- insert string from varchar +insert overwrite table varchar1_1 + select key, value from varchar1 order by key, value limit 2; +select key, value from varchar1_1 order by key, value; + +-- respect string length +insert overwrite table varchar1 + select key, cast(value as varchar(3)) from src order by key, value limit 2; +select key, value from varchar1 order by key, value; + +drop table varchar1; +drop table varchar1_1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_2.q b/src/test/hive/ql/src/test/queries/clientpositive/varchar_2.q new file mode 100644 index 0000000000000..1eb3b85d6a3b3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/varchar_2.q @@ -0,0 +1,36 @@ +drop table varchar_2; + +create table varchar_2 ( + key varchar(10), + value varchar(20) +); + +insert overwrite table varchar_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 varchar_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 varchar_2 +group by value +order by value desc +limit 5; + +drop table varchar_2; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_cast.q b/src/test/hive/ql/src/test/queries/clientpositive/varchar_cast.q new file mode 100644 index 0000000000000..550f3dc728df2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/varchar_cast.q @@ -0,0 +1,84 @@ + +-- Cast from varchar 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 varchar(10)) as tinyint), + cast(cast('11' as varchar(10)) as smallint), + cast(cast('11' as varchar(10)) as int), + cast(cast('11' as varchar(10)) as bigint), + cast(cast('11.00' as varchar(10)) as float), + cast(cast('11.00' as varchar(10)) as double), + cast(cast('11.00' as varchar(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 varchar(10)) as date), + cast(cast('2011-01-01 01:02:03' as varchar(30)) as timestamp) +from src limit 1; + +-- no tests from string/varchar to boolean, that conversion doesn't look useful +select + cast(cast('abc123' as string) as string), + cast(cast('abc123' as string) as varchar(10)) +from src limit 1; + +select + cast(cast('abc123' as varchar(10)) as string), + cast(cast('abc123' as varchar(10)) as varchar(10)) +from src limit 1; + +-- cast from other types to varchar +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 varchar(10)), + cast(cast(11 as smallint) as varchar(10)), + cast(cast(11 as int) as varchar(10)), + cast(cast(11 as bigint) as varchar(10)), + cast(cast(11.00 as float) as varchar(10)), + cast(cast(11.00 as double) as varchar(10)), + cast(cast(11.00 as decimal) as varchar(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 varchar(10)), + cast(timestamp('2011-01-01 01:02:03') as varchar(30)) +from src limit 1; + +select + cast(true as string), + cast(false as string) +from src limit 1; + +select + cast(true as varchar(10)), + cast(false as varchar(10)) +from src limit 1; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_comparison.q b/src/test/hive/ql/src/test/queries/clientpositive/varchar_comparison.q new file mode 100644 index 0000000000000..b6c6f406412fe --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/varchar_comparison.q @@ -0,0 +1,40 @@ + +-- Should all be true +select + cast('abc' as varchar(10)) = cast('abc' as varchar(10)), + cast('abc' as varchar(10)) <= cast('abc' as varchar(10)), + cast('abc' as varchar(10)) >= cast('abc' as varchar(10)), + cast('abc' as varchar(10)) < cast('abd' as varchar(10)), + cast('abc' as varchar(10)) > cast('abb' as varchar(10)), + cast('abc' as varchar(10)) <> cast('abb' as varchar(10)) +from src limit 1; + +-- Different varchar lengths should still compare the same +select + cast('abc' as varchar(10)) = cast('abc' as varchar(3)), + cast('abc' as varchar(10)) <= cast('abc' as varchar(3)), + cast('abc' as varchar(10)) >= cast('abc' as varchar(3)), + cast('abc' as varchar(10)) < cast('abd' as varchar(3)), + cast('abc' as varchar(10)) > cast('abb' as varchar(3)), + cast('abc' as varchar(10)) <> cast('abb' as varchar(3)) +from src limit 1; + +-- Should work with string types as well +select + cast('abc' as varchar(10)) = 'abc', + cast('abc' as varchar(10)) <= 'abc', + cast('abc' as varchar(10)) >= 'abc', + cast('abc' as varchar(10)) < 'abd', + cast('abc' as varchar(10)) > 'abb', + cast('abc' as varchar(10)) <> 'abb' +from src limit 1; + +-- leading space is significant for varchar +select + cast(' abc' as varchar(10)) <> cast('abc' as varchar(10)) +from src limit 1; + +-- trailing space is significant for varchar +select + cast('abc ' as varchar(10)) <> cast('abc' as varchar(10)) +from src limit 1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_join1.q b/src/test/hive/ql/src/test/queries/clientpositive/varchar_join1.q new file mode 100644 index 0000000000000..6a19efaa3c5a1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/varchar_join1.q @@ -0,0 +1,35 @@ +drop table varchar_join1_vc1; +drop table varchar_join1_vc2; +drop table varchar_join1_str; + +create table varchar_join1_vc1 ( + c1 int, + c2 varchar(10) +); + +create table varchar_join1_vc2 ( + c1 int, + c2 varchar(20) +); + +create table varchar_join1_str ( + c1 int, + 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; + +-- 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; + +-- Join varchar with different length varchar +select * from varchar_join1_vc1 a join varchar_join1_vc2 b on (a.c2 = b.c2) order by a.c1; + +-- Join varchar with string +select * from varchar_join1_vc1 a join varchar_join1_str b on (a.c2 = b.c2) order by a.c1; + +drop table varchar_join1_vc1; +drop table varchar_join1_vc2; +drop table varchar_join1_str; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_nested_types.q b/src/test/hive/ql/src/test/queries/clientpositive/varchar_nested_types.q new file mode 100644 index 0000000000000..5c0989f7fc5ae --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/varchar_nested_types.q @@ -0,0 +1,53 @@ +drop table varchar_nested_1; +drop table varchar_nested_array; +drop table varchar_nested_map; +drop table varchar_nested_struct; +drop table varchar_nested_cta; +drop table varchar_nested_view; + +create table varchar_nested_1 (key int, value varchar(20)); +insert overwrite table varchar_nested_1 + select key, value from src order by key limit 1; + +-- arrays +create table varchar_nested_array (c1 array); +insert overwrite table varchar_nested_array + select array(value, value) from varchar_nested_1; +describe varchar_nested_array; +select * from varchar_nested_array; + +-- maps +create table varchar_nested_map (c1 map); +insert overwrite table varchar_nested_map + select map(key, value) from varchar_nested_1; +describe varchar_nested_map; +select * from varchar_nested_map; + +-- structs +create table varchar_nested_struct (c1 struct); +insert overwrite table varchar_nested_struct + select named_struct('a', key, + 'b', value, + 'c', cast(value as string)) + from varchar_nested_1; +describe varchar_nested_struct; +select * from varchar_nested_struct; + +-- nested type with create table as +create table varchar_nested_cta as + select * from varchar_nested_struct; +describe varchar_nested_cta; +select * from varchar_nested_cta; + +-- nested type with view +create table varchar_nested_view as + select * from varchar_nested_struct; +describe varchar_nested_view; +select * from varchar_nested_view; + +drop table varchar_nested_1; +drop table varchar_nested_array; +drop table varchar_nested_map; +drop table varchar_nested_struct; +drop table varchar_nested_cta; +drop table varchar_nested_view; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_serde.q b/src/test/hive/ql/src/test/queries/clientpositive/varchar_serde.q new file mode 100644 index 0000000000000..7351b688cf504 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/varchar_serde.q @@ -0,0 +1,102 @@ +drop table if exists varchar_serde_regex; +drop table if exists varchar_serde_lb; +drop table if exists varchar_serde_ls; +drop table if exists varchar_serde_c; +drop table if exists varchar_serde_lbc; +drop table if exists varchar_serde_orc; + +-- +-- RegexSerDe +-- +create table varchar_serde_regex ( + key varchar(10), + value varchar(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 varchar_serde_regex; + +select * from varchar_serde_regex limit 5; +select value, count(*) from varchar_serde_regex group by value limit 5; + +-- +-- LazyBinary +-- +create table varchar_serde_lb ( + key varchar(10), + value varchar(20) +); +alter table varchar_serde_lb set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe'; + +insert overwrite table varchar_serde_lb + select key, value from varchar_serde_regex; +select * from varchar_serde_lb limit 5; +select value, count(*) from varchar_serde_lb group by value limit 5; + +-- +-- LazySimple +-- +create table varchar_serde_ls ( + key varchar(10), + value varchar(20) +); +alter table varchar_serde_ls set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; + +insert overwrite table varchar_serde_ls + select key, value from varchar_serde_lb; +select * from varchar_serde_ls limit 5; +select value, count(*) from varchar_serde_ls group by value limit 5; + +-- +-- Columnar +-- +create table varchar_serde_c ( + key varchar(10), + value varchar(20) +) stored as rcfile; +alter table varchar_serde_c set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; + +insert overwrite table varchar_serde_c + select key, value from varchar_serde_ls; +select * from varchar_serde_c limit 5; +select value, count(*) from varchar_serde_c group by value limit 5; + +-- +-- LazyBinaryColumnar +-- +create table varchar_serde_lbc ( + key varchar(10), + value varchar(20) +) stored as rcfile; +alter table varchar_serde_lbc set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; + +insert overwrite table varchar_serde_lbc + select key, value from varchar_serde_c; +select * from varchar_serde_lbc limit 5; +select value, count(*) from varchar_serde_lbc group by value limit 5; + +-- +-- ORC +-- +create table varchar_serde_orc ( + key varchar(10), + value varchar(20) +) stored as orc; +alter table varchar_serde_orc set serde 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'; + + +insert overwrite table varchar_serde_orc + select key, value from varchar_serde_lbc; +select * from varchar_serde_orc limit 5; +select value, count(*) from varchar_serde_orc group by value limit 5; + +drop table if exists varchar_serde_regex; +drop table if exists varchar_serde_lb; +drop table if exists varchar_serde_ls; +drop table if exists varchar_serde_c; +drop table if exists varchar_serde_lbc; +drop table if exists varchar_serde_orc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_udf1.q b/src/test/hive/ql/src/test/queries/clientpositive/varchar_udf1.q new file mode 100644 index 0000000000000..0a3012b5cd6d3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/varchar_udf1.q @@ -0,0 +1,156 @@ +drop table varchar_udf_1; + +create table varchar_udf_1 (c1 string, c2 string, c3 varchar(10), c4 varchar(20)); +insert overwrite table varchar_udf_1 + select key, value, key, value from src where key = '238' limit 1; + +-- UDFs with varchar support +select + concat(c1, c2), + concat(c3, c4), + concat(c1, c2) = concat(c3, c4) +from varchar_udf_1 limit 1; + +select + upper(c2), + upper(c4), + upper(c2) = upper(c4) +from varchar_udf_1 limit 1; + +select + lower(c2), + lower(c4), + lower(c2) = lower(c4) +from varchar_udf_1 limit 1; + +-- Scalar UDFs +select + ascii(c2), + ascii(c4), + ascii(c2) = ascii(c4) +from varchar_udf_1 limit 1; + +select + concat_ws('|', c1, c2), + concat_ws('|', c3, c4), + concat_ws('|', c1, c2) = concat_ws('|', c3, c4) +from varchar_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 varchar_udf_1 limit 1; + +select + instr(c2, '_'), + instr(c4, '_'), + instr(c2, '_') = instr(c4, '_') +from varchar_udf_1 limit 1; + +select + length(c2), + length(c4), + length(c2) = length(c4) +from varchar_udf_1 limit 1; + +select + locate('a', 'abcdabcd', 3), + locate(cast('a' as varchar(1)), cast('abcdabcd' as varchar(10)), 3), + locate('a', 'abcdabcd', 3) = locate(cast('a' as varchar(1)), cast('abcdabcd' as varchar(10)), 3) +from varchar_udf_1 limit 1; + +select + lpad(c2, 15, ' '), + lpad(c4, 15, ' '), + lpad(c2, 15, ' ') = lpad(c4, 15, ' ') +from varchar_udf_1 limit 1; + +select + ltrim(c2), + ltrim(c4), + ltrim(c2) = ltrim(c4) +from varchar_udf_1 limit 1; + +select + regexp(c2, 'val'), + regexp(c4, 'val'), + regexp(c2, 'val') = regexp(c4, 'val') +from varchar_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 varchar_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 varchar_udf_1 limit 1; + +select + reverse(c2), + reverse(c4), + reverse(c2) = reverse(c4) +from varchar_udf_1 limit 1; + +select + rpad(c2, 15, ' '), + rpad(c4, 15, ' '), + rpad(c2, 15, ' ') = rpad(c4, 15, ' ') +from varchar_udf_1 limit 1; + +select + rtrim(c2), + rtrim(c4), + rtrim(c2) = rtrim(c4) +from varchar_udf_1 limit 1; + +select + sentences('See spot run. See jane run.'), + sentences(cast('See spot run. See jane run.' as varchar(50))) +from varchar_udf_1 limit 1; + +select + split(c2, '_'), + split(c4, '_') +from varchar_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 varchar(20)),',',':') +from varchar_udf_1 limit 1; + +select + substr(c2, 1, 3), + substr(c4, 1, 3), + substr(c2, 1, 3) = substr(c4, 1, 3) +from varchar_udf_1 limit 1; + +select + trim(c2), + trim(c4), + trim(c2) = trim(c4) +from varchar_udf_1 limit 1; + + +-- Aggregate Functions +select + compute_stats(c2, 16), + compute_stats(c4, 16) +from varchar_udf_1; + +select + min(c2), + min(c4) +from varchar_udf_1; + +select + max(c2), + max(c4) +from varchar_udf_1; + + +drop table varchar_udf_1; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_union1.q b/src/test/hive/ql/src/test/queries/clientpositive/varchar_union1.q new file mode 100644 index 0000000000000..cf90eab33e2d0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/varchar_union1.q @@ -0,0 +1,47 @@ +drop table varchar_union1_vc1; +drop table varchar_union1_vc2; +drop table varchar_union1_str; + +create table varchar_union1_vc1 ( + c1 int, + c2 varchar(10) +); + +create table varchar_union1_vc2 ( + c1 int, + c2 varchar(20) +); + +create table varchar_union1_str ( + c1 int, + 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; + +-- union varchar with same length varchar +select * from ( + select * from varchar_union1_vc1 + union all + select * from varchar_union1_vc1 limit 1 +) q1 sort by c1; + +-- union varchar with different length varchar +select * from ( + select * from varchar_union1_vc1 + union all + select * from varchar_union1_vc2 limit 1 +) q1 sort by c1; + +-- union varchar with string +select * from ( + select * from varchar_union1_vc1 + union all + select * from varchar_union1_str limit 1 +) q1 sort by c1; + +drop table varchar_union1_vc1; +drop table varchar_union1_vc2; +drop table varchar_union1_str; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/view.q b/src/test/hive/ql/src/test/queries/clientpositive/view.q new file mode 100644 index 0000000000000..4e3d0572921d6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/view.q @@ -0,0 +1,51 @@ +CREATE DATABASE db1; +USE db1; + +CREATE TABLE table1 (key STRING, value STRING) +STORED AS TEXTFILE; + +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' +OVERWRITE INTO TABLE table2; + +-- relative reference, no alias +CREATE VIEW v1 AS SELECT * FROM table1; + +-- relative reference, aliased +CREATE VIEW v2 AS SELECT t1.* FROM table1 t1; + +-- relative reference, multiple tables +CREATE VIEW v3 AS SELECT t1.*, t2.key k FROM table1 t1 JOIN table2 t2 ON t1.key = t2.key; + +-- absolute reference, no alias +CREATE VIEW v4 AS SELECT * FROM db1.table1; + +-- absolute reference, aliased +CREATE VIEW v5 AS SELECT t1.* FROM db1.table1 t1; + +-- absolute reference, multiple tables +CREATE VIEW v6 AS SELECT t1.*, t2.key k FROM db1.table1 t1 JOIN db1.table2 t2 ON t1.key = t2.key; + +-- relative reference, explicit column +CREATE VIEW v7 AS SELECT key from table1; + +-- absolute reference, explicit column +CREATE VIEW v8 AS SELECT key from db1.table1; + +CREATE DATABASE db2; +USE db2; + +SELECT * FROM db1.v1; +SELECT * FROM db1.v2; +SELECT * FROM db1.v3; +SELECT * FROM db1.v4; +SELECT * FROM db1.v5; +SELECT * FROM db1.v6; +SELECT * FROM db1.v7; +SELECT * FROM db1.v8; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/view_cast.q b/src/test/hive/ql/src/test/queries/clientpositive/view_cast.q new file mode 100644 index 0000000000000..b0b078ec628b3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/view_cast.q @@ -0,0 +1,16 @@ +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'); + +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'); + +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; + +DROP VIEW IF EXISTS joined_aa_view_tw; +CREATE VIEW joined_aa_view_tw AS SELECT aa.ks_uid, aa.sr_id, aa.act, at_sr_uid, aa.act_date, aa.at_ks_uid, aa.content_creation_date, coalesce( other.ksc, 10.0) AS at_ksc, coalesce( self.ksc , 10.0 ) AS self_ksc FROM aa_view_tw aa LEFT OUTER JOIN ( SELECT ks_uid, csc AS ksc FROM mstab WHERE dt='20130311' ) self ON ( CAST(aa.ks_uid AS BIGINT) = CAST(self.ks_uid AS BIGINT) ) LEFT OUTER JOIN ( SELECT ks_uid, csc AS ksc FROM mstab WHERE dt='20130311' ) other ON ( CAST(aa.at_ks_uid AS BIGINT) = CAST(other.ks_uid AS BIGINT) ); + +SELECT * FROM joined_aa_view_tw; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/view_inputs.q b/src/test/hive/ql/src/test/queries/clientpositive/view_inputs.q new file mode 100644 index 0000000000000..7acc321e6f7d3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/view_inputs.q @@ -0,0 +1,7 @@ +-- Tests that selecting from a view and another view that selects from that same view + +CREATE VIEW test_view1 AS SELECT * FROM src; + +CREATE VIEW test_view2 AS SELECT * FROM test_view1; + +SELECT COUNT(*) FROM test_view1 a JOIN test_view2 b ON a.key = b.key; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/virtual_column.q b/src/test/hive/ql/src/test/queries/clientpositive/virtual_column.q new file mode 100644 index 0000000000000..30fde395ebb04 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/virtual_column.q @@ -0,0 +1,19 @@ +select INPUT__FILE__NAME, key, BLOCK__OFFSET__INSIDE__FILE from src; + +select key, count(INPUT__FILE__NAME) from src group by key order by key; + +select INPUT__FILE__NAME, key, collect_set(BLOCK__OFFSET__INSIDE__FILE) from src group by INPUT__FILE__NAME, key order by key; + +select * from src where BLOCK__OFFSET__INSIDE__FILE > 12000 order by key; + +select * from src where BLOCK__OFFSET__INSIDE__FILE > 5800 order by key; + + +CREATE TABLE src_index_test_rc (key int, value string) STORED AS RCFILE; + +set hive.io.rcfile.record.buffer.size = 1024; +INSERT OVERWRITE TABLE src_index_test_rc SELECT * FROM src; +select INPUT__FILE__NAME, key, BLOCK__OFFSET__INSIDE__FILE from src_index_test_rc order by key; + +DROP TABLE src_index_test_rc; +DROP INDEX src_index on src_index_test_rc; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing.q b/src/test/hive/ql/src/test/queries/clientpositive/windowing.q new file mode 100644 index 0000000000000..a7297db6104c0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/windowing.q @@ -0,0 +1,434 @@ +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; + +-- 1. testWindowing +select p_mfgr, p_name, p_size, +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, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 +from part +; + +-- 2. testGroupByWithPartitioning +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 +; + +-- 3. testGroupByHavingWithSWQ +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 +having p_size > 0 +; + +-- 4. testCount +select p_mfgr, p_name, +count(p_size) over(distribute by p_mfgr sort by p_name) as cd +from part +; + +-- 5. testCountWithWindowingUDAF +select p_mfgr, p_name, +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, +count(p_size) over(distribute by p_mfgr sort by p_name) as cd, +p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1, +p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz +from part +; + +-- 6. testCountInSubQ +select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz +from (select p_mfgr, p_name, +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, +count(p_size) over(distribute by p_mfgr sort by p_name) as cd, +p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1, +p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz +from part +) sub1; + +-- 7. testJoinWithWindowingAndPTF +select abc.p_mfgr, abc.p_name, +rank() over(distribute by abc.p_mfgr sort by abc.p_name) as r, +dense_rank() over(distribute by abc.p_mfgr sort by abc.p_name) as dr, +abc.p_retailprice, sum(abc.p_retailprice) over (distribute by abc.p_mfgr sort by abc.p_name rows between unbounded preceding and current row) as s1, +abc.p_size, abc.p_size - lag(abc.p_size,1,abc.p_size) over(distribute by abc.p_mfgr sort by abc.p_name) as deltaSz +from noop(on part +partition by p_mfgr +order by p_name +) abc join part p1 on abc.p_partkey = p1.p_partkey +; + +-- 8. testMixedCaseAlias +select p_mfgr, p_name, p_size, +rank() over(distribute by p_mfgr sort by p_name, p_size desc) as R +from part +; + +-- 9. testHavingWithWindowingNoGBY +select p_mfgr, p_name, p_size, +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, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 +from part +; + +-- 10. testHavingWithWindowingCondRankNoGBY +select p_mfgr, p_name, p_size, +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, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 +from part +; + +-- 11. testFirstLast +select p_mfgr,p_name, p_size, +sum(p_size) over (distribute by p_mfgr sort by p_name rows between current row and current row) as s2, +first_value(p_size) over w1 as f, +last_value(p_size, false) over w1 as l +from part +window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following); + +-- 12. testFirstLastWithWhere +select p_mfgr,p_name, p_size, +rank() over(distribute by p_mfgr sort by p_name) as r, +sum(p_size) over (distribute by p_mfgr sort by p_name rows between current row and current row) as s2, +first_value(p_size) over w1 as f, +last_value(p_size, false) over w1 as l +from part +where p_mfgr = 'Manufacturer#3' +window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following); + +-- 13. testSumWindow +select p_mfgr,p_name, p_size, +sum(p_size) over w1 as s1, +sum(p_size) over (distribute by p_mfgr sort by p_name rows between current row and current row) as s2 +from part +window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following); + +-- 14. testNoSortClause +select p_mfgr,p_name, p_size, +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 +from part +window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following); + +-- 15. testExpressions +select p_mfgr,p_name, p_size, +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, +cume_dist() over(distribute by p_mfgr sort by p_name) as cud, +percent_rank() over(distribute by p_mfgr sort by p_name) as pr, +ntile(3) over(distribute by p_mfgr sort by p_name) as nt, +count(p_size) over(distribute by p_mfgr sort by p_name) as ca, +avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, +stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, +first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, +last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, +first_value(p_size) over w1 as fvW1 +from part +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following); + +-- 16. testMultipleWindows +select p_mfgr,p_name, p_size, + 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, +cume_dist() over(distribute by p_mfgr sort by p_name) as cud, +sum(p_size) over (distribute by p_mfgr sort by p_name range between unbounded preceding and current row) as s1, +sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row) as s2, +first_value(p_size) over w1 as fv1 +from part +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following); + +-- 17. testCountStar +select p_mfgr,p_name, p_size, +count(*) over(distribute by p_mfgr sort by p_name ) as c, +count(p_size) over(distribute by p_mfgr sort by p_name) as ca, +first_value(p_size) over w1 as fvW1 +from part +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following); + +-- 18. testUDAFs +select p_mfgr,p_name, p_size, +sum(p_retailprice) over w1 as s, +min(p_retailprice) over w1 as mi, +max(p_retailprice) over w1 as ma, +avg(p_retailprice) over w1 as ag +from part +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following); + +-- 19. testUDAFsWithGBY +select p_mfgr,p_name, p_size, p_retailprice, +sum(p_retailprice) over w1 as s, +min(p_retailprice) as mi , +max(p_retailprice) as ma , +avg(p_retailprice) over w1 as ag +from part +group by p_mfgr,p_name, p_size, p_retailprice +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following); + +-- 20. testSTATs +select p_mfgr,p_name, p_size, +stddev(p_retailprice) over w1 as sdev, +stddev_pop(p_retailprice) over w1 as sdev_pop, +collect_set(p_size) over w1 as uniq_size, +variance(p_retailprice) over w1 as var, +corr(p_size, p_retailprice) over w1 as cor, +covar_pop(p_size, p_retailprice) over w1 as covarp +from part +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following); + +-- 21. testDISTs +select p_mfgr,p_name, p_size, +histogram_numeric(p_retailprice, 5) over w1 as hist, +percentile(p_partkey, 0.5) over w1 as per, +row_number() over(distribute by p_mfgr sort by p_name) as rn +from part +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following); + +-- 22. testViewAsTableInputWithWindowing +create view IF NOT EXISTS mfgr_price_view as +select p_mfgr, p_brand, +round(sum(p_retailprice),2) as s +from part +group by p_mfgr, p_brand; + +select * +from ( +select p_mfgr, p_brand, s, +round(sum(s) over w1 , 2) as s1 +from mfgr_price_view +window w1 as (distribute by p_mfgr sort by p_mfgr ) +) sq +order by p_mfgr, p_brand; + +select p_mfgr, p_brand, s, +round(sum(s) over w1 ,2) as s1 +from mfgr_price_view +window w1 as (distribute by p_mfgr sort by p_brand rows between 2 preceding and current row); + +-- 23. testCreateViewWithWindowingQuery +create view IF NOT EXISTS mfgr_brand_price_view as +select p_mfgr, p_brand, +sum(p_retailprice) over w1 as s +from part +window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and current row); + +select * from mfgr_brand_price_view; + +-- 24. testLateralViews +select p_mfgr, p_name, +lv_col, p_size, sum(p_size) over w1 as s +from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p +lateral view explode(arr) part_lv as lv_col +window w1 as (distribute by p_mfgr sort by p_size, lv_col rows between 2 preceding and current row); + +-- 25. testMultipleInserts3SWQs +CREATE TABLE part_1( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +s DOUBLE); + +CREATE TABLE part_2( +p_mfgr STRING, +p_name STRING, +p_size INT, +r INT, +dr INT, +cud INT, +s2 DOUBLE, +fv1 INT); + +CREATE TABLE part_3( +p_mfgr STRING, +p_name STRING, +p_size INT, +c INT, +ca INT, +fv INT); + +from part +INSERT OVERWRITE TABLE part_1 +select p_mfgr, p_name, p_size, +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, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s +INSERT OVERWRITE TABLE part_2 +select p_mfgr,p_name, p_size, +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, +cume_dist() over(distribute by p_mfgr sort by p_name) as cud, +round(sum(p_size) over (distribute by p_mfgr sort by p_size range between 5 preceding and current row),1) as s2, +first_value(p_size) over w1 as fv1 +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following) +INSERT OVERWRITE TABLE part_3 +select p_mfgr,p_name, p_size, +count(*) over(distribute by p_mfgr sort by p_name) as c, +count(p_size) over(distribute by p_mfgr sort by p_name) as ca, +first_value(p_size) over w1 as fv +window w1 as (distribute by p_mfgr sort by p_mfgr, p_name rows between 2 preceding and 2 following); + +select * from part_1; + +select * from part_2; + +select * from part_3; + +-- 26. testGroupByHavingWithSWQAndAlias +select p_mfgr, p_name, p_size, min(p_retailprice) as mi, +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 +having p_size > 0 +; + +-- 27. testMultipleRangeWindows +select p_mfgr,p_name, p_size, +sum(p_size) over (distribute by p_mfgr sort by p_size range between 10 preceding and current row) as s2, +sum(p_size) over (distribute by p_mfgr sort by p_size range between current row and 10 following ) as s1 +from part +window w1 as (rows between 2 preceding and 2 following); + +-- 28. testPartOrderInUDAFInvoke +select p_mfgr, p_name, p_size, +sum(p_size) over (partition by p_mfgr order by p_name rows between 2 preceding and 2 following) as s +from part; + +-- 29. testPartOrderInWdwDef +select p_mfgr, p_name, p_size, +sum(p_size) over w1 as s +from part +window w1 as (partition by p_mfgr order by p_name rows between 2 preceding and 2 following); + +-- 30. testDefaultPartitioningSpecRules +select p_mfgr, p_name, p_size, +sum(p_size) over w1 as s, +sum(p_size) over w2 as s2 +from part +window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following), + w2 as (partition by p_mfgr order by p_name); + +-- 31. testWindowCrossReference +select p_mfgr, p_name, p_size, +sum(p_size) over w1 as s1, +sum(p_size) over w2 as s2 +from part +window w1 as (partition by p_mfgr order by p_name range between 2 preceding and 2 following), + w2 as w1; + + +-- 32. testWindowInheritance +select p_mfgr, p_name, p_size, +sum(p_size) over w1 as s1, +sum(p_size) over w2 as s2 +from part +window w1 as (partition by p_mfgr order by p_name range between 2 preceding and 2 following), + w2 as (w1 rows between unbounded preceding and current row); + + +-- 33. testWindowForwardReference +select p_mfgr, p_name, p_size, +sum(p_size) over w1 as s1, +sum(p_size) over w2 as s2, +sum(p_size) over w3 as s3 +from part +window w1 as (distribute by p_mfgr sort by p_name range between 2 preceding and 2 following), + w2 as w3, + w3 as (distribute by p_mfgr sort by p_name range between unbounded preceding and current row); + + +-- 34. testWindowDefinitionPropagation +select p_mfgr, p_name, p_size, +sum(p_size) over w1 as s1, +sum(p_size) over w2 as s2, +sum(p_size) over (w3 rows between 2 preceding and 2 following) as s3 +from part +window w1 as (distribute by p_mfgr sort by p_name range between 2 preceding and 2 following), + w2 as w3, + w3 as (distribute by p_mfgr sort by p_name range between unbounded preceding and current row); + +-- 35. testDistinctWithWindowing +select DISTINCT p_mfgr, p_name, p_size, +sum(p_size) over w1 as s +from part +window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following); + +-- 36. testRankWithPartitioning +select p_mfgr, p_name, p_size, +rank() over (partition by p_mfgr order by p_name ) as r +from part; + +-- 37. testPartitioningVariousForms +select p_mfgr, +round(sum(p_retailprice) over (partition by p_mfgr order by p_mfgr),2) as s1, +min(p_retailprice) over (partition by p_mfgr) as s2, +max(p_retailprice) over (distribute by p_mfgr sort by p_mfgr) as s3, +round(avg(p_retailprice) over (distribute by p_mfgr),2) as s4, +count(p_retailprice) over (cluster by p_mfgr ) as s5 +from part; + +-- 38. testPartitioningVariousForms2 +select p_mfgr, p_name, p_size, +sum(p_retailprice) over (partition by p_mfgr, p_name order by p_mfgr, p_name rows between unbounded preceding and current row) as s1, +min(p_retailprice) over (distribute by p_mfgr, p_name sort by p_mfgr, p_name rows between unbounded preceding and current row) as s2, +max(p_retailprice) over (partition by p_mfgr, p_name order by p_name) as s3 +from part; + +-- 39. testUDFOnOrderCols +select p_mfgr, p_type, substr(p_type, 2) as short_ptype, +rank() over (partition by p_mfgr order by substr(p_type, 2)) as r +from part; + +-- 40. testNoBetweenForRows +select p_mfgr, p_name, p_size, + sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows unbounded preceding) as s1 + from part ; + +-- 41. testNoBetweenForRange +select p_mfgr, p_name, p_size, + sum(p_retailprice) over (distribute by p_mfgr sort by p_size range unbounded preceding) as s1 + from part ; + +-- 42. testUnboundedFollowingForRows +select p_mfgr, p_name, p_size, + sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between current row and unbounded following) as s1 + from part ; + +-- 43. testUnboundedFollowingForRange +select p_mfgr, p_name, p_size, + sum(p_retailprice) over (distribute by p_mfgr sort by p_size range between current row and unbounded following) as s1 + from part ; + +-- 44. testOverNoPartitionSingleAggregate +select p_name, p_retailprice, +round(avg(p_retailprice) over(),2) +from part +order by p_name; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q b/src/test/hive/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q new file mode 100644 index 0000000000000..9c7625dcd786a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q @@ -0,0 +1,27 @@ +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; + +set hive.join.cache.size=1; + +select p_mfgr, p_name, p_size, +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, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 +from part +; + +set hive.join.cache.size=25000; \ No newline at end of file diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_columnPruning.q b/src/test/hive/ql/src/test/queries/clientpositive/windowing_columnPruning.q new file mode 100644 index 0000000000000..7c4ab386145d5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/windowing_columnPruning.q @@ -0,0 +1,32 @@ +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; + +-- 1. testQueryLevelPartitionColsNotInSelect +select p_size, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 +from part + ; + +-- 2. testWindowPartitionColsNotInSelect +select p_size, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 +from part; + +-- 3. testHavingColNotInSelect +select p_mfgr, +sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 +from part; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_expressions.q b/src/test/hive/ql/src/test/queries/clientpositive/windowing_expressions.q new file mode 100644 index 0000000000000..2c3339095f22b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/windowing_expressions.q @@ -0,0 +1,70 @@ +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 over10k; + +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; + +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), +max(p_retailprice) over w1 - min(p_retailprice) over w1 = last_value(p_retailprice) over w1 - first_value(p_retailprice) over w1 +from part +window w1 as (distribute by p_mfgr sort by p_retailprice) +; +select p_mfgr, p_retailprice, p_size, +rank() over (distribute by p_mfgr sort by p_retailprice) as r, +sum(p_retailprice) over (distribute by p_mfgr sort by p_retailprice rows between unbounded preceding and current row) as s2, +sum(p_retailprice) over (distribute by p_mfgr sort by p_retailprice rows between unbounded preceding and current row) -5 as s1 +from part +; + +select s, si, f, si - lead(f, 3) over (partition by t order by bo,s,si,f desc) from over10k limit 100; +select s, i, i - lead(i, 3, 0) over (partition by si order by i,s) from over10k limit 100; +select s, si, d, si - lag(d, 3) over (partition by b order by si,s,d) from over10k limit 100; +select s, lag(s, 3, 'fred') over (partition by f order by b) from over10k limit 100; + +select p_mfgr, avg(p_retailprice) over(partition by p_mfgr, p_type order by p_mfgr) from part; + +select p_mfgr, avg(p_retailprice) over(partition by p_mfgr order by p_type,p_mfgr rows between unbounded preceding and current row) from part; + +-- multi table insert test +create table t1 (a1 int, b1 string); +create table t2 (a1 int, b1 string); +from (select sum(i) over (partition by ts order by i), s from over10k) tt insert overwrite table t1 select * insert overwrite table t2 select * ; +select * from t1 limit 3; +select * from t2 limit 3; + +select p_mfgr, p_retailprice, p_size, +round(sum(p_retailprice) over w1 , 2) + 50.0 = round(sum(lag(p_retailprice,1,50.0)) over w1 + (last_value(p_retailprice) over w1),2) +from part +window w1 as (distribute by p_mfgr sort by p_retailprice) +limit 11; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_multipartitioning.q b/src/test/hive/ql/src/test/queries/clientpositive/windowing_multipartitioning.q new file mode 100644 index 0000000000000..bb371e900975b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/windowing_multipartitioning.q @@ -0,0 +1,42 @@ +drop table over10k; + +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; + +select s, rank() over (partition by s order by si), sum(b) over (partition by s order by si) from over10k limit 100; + +select s, +rank() over (partition by s order by dec desc), +sum(b) over (partition by s order by ts desc) +from over10k +where s = 'tom allen' or s = 'bob steinbeck'; + +select s, sum(i) over (partition by s), sum(f) over (partition by si) from over10k where s = 'tom allen' or s = 'bob steinbeck' ; + +select s, rank() over (partition by s order by bo), rank() over (partition by si order by bin desc) from over10k +where s = 'tom allen' or s = 'bob steinbeck'; + +select s, sum(f) over (partition by i), row_number() over (order by f) from over10k where s = 'tom allen' or s = 'bob steinbeck'; + +select s, rank() over w1, +rank() over w2 +from over10k +where s = 'tom allen' or s = 'bob steinbeck' +window +w1 as (partition by s order by dec), +w2 as (partition by si order by f) +; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_navfn.q b/src/test/hive/ql/src/test/queries/clientpositive/windowing_navfn.q new file mode 100644 index 0000000000000..8a9d0012593e6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/windowing_navfn.q @@ -0,0 +1,31 @@ +drop table over10k; + +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; + +select s, row_number() over (partition by d order by dec) from over10k limit 100; + +select i, lead(s) over (partition by bin order by d,i desc) from over10k limit 100; + +select i, lag(dec) over (partition by i order by s,i,dec) from over10k limit 100; + +select s, last_value(t) over (partition by d order by f) from over10k limit 100; + +select s, first_value(s) over (partition by bo order by s) from over10k limit 100; + +select t, s, i, last_value(i) over (partition by t order by s) +from over10k where (s = 'oscar allen' or s = 'oscar carson') and t = 10; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_ntile.q b/src/test/hive/ql/src/test/queries/clientpositive/windowing_ntile.q new file mode 100644 index 0000000000000..505c259f4b7b3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/windowing_ntile.q @@ -0,0 +1,28 @@ +drop table over10k; + +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; + +select i, ntile(10) over (partition by s order by i) from over10k limit 100; + +select s, ntile(100) over (partition by i order by s) from over10k limit 100; + +select f, ntile(4) over (partition by d order by f) from over10k limit 100; + +select d, ntile(1000) over (partition by dec order by d) from over10k limit 100; + + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_rank.q b/src/test/hive/ql/src/test/queries/clientpositive/windowing_rank.q new file mode 100644 index 0000000000000..bf76867813e4e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/windowing_rank.q @@ -0,0 +1,66 @@ +drop table over10k; + +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; + +select s, rank() over (partition by f order by t) from over10k limit 100; + +select s, dense_rank() over (partition by ts order by i,s desc) from over10k limit 100; + +select s, cume_dist() over (partition by bo order by b,s) from over10k limit 100; + +select s, percent_rank() over (partition by dec order by f) from over10k limit 100; + +-- If following tests fail, look for the comments in class PTFPPD::process() + +select ts, dec, rnk +from + (select ts, dec, + rank() over (partition by ts order by dec) as rnk + from + (select other.ts, other.dec + from over10k other + join over10k on (other.b = over10k.b) + ) joined + ) ranked +where rnk = 1 limit 10; + +select ts, dec, rnk +from + (select ts, dec, + rank() over (partition by ts) as rnk + from + (select other.ts, other.dec + from over10k other + join over10k on (other.b = over10k.b) + ) joined + ) ranked +where dec = 89.5 limit 10; + +select ts, dec, rnk +from + (select ts, dec, + rank() over (partition by ts order by dec) as rnk + from + (select other.ts, other.dec + from over10k other + join over10k on (other.b = over10k.b) + where other.t < 10 + ) joined + ) ranked +where rnk = 1 limit 10; + diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_udaf.q b/src/test/hive/ql/src/test/queries/clientpositive/windowing_udaf.q new file mode 100644 index 0000000000000..f22b992cd4386 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/windowing_udaf.q @@ -0,0 +1,28 @@ +drop table over10k; + +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; + +select s, min(i) over (partition by s) from over10k limit 100; + +select s, avg(f) over (partition by si order by s) from over10k limit 100; + +select s, avg(i) over (partition by t, b order by s) from over10k limit 100; + +select max(i) over w from over10k window w as (partition by f) limit 100; + +select s, avg(d) over (partition by t order by f) from over10k limit 100; diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_windowspec.q b/src/test/hive/ql/src/test/queries/clientpositive/windowing_windowspec.q new file mode 100644 index 0000000000000..7cc1367306956 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/clientpositive/windowing_windowspec.q @@ -0,0 +1,36 @@ +drop table over10k; + +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; + +select s, sum(b) over (partition by i order by s,b rows unbounded preceding) from over10k limit 100; + +select s, sum(f) over (partition by d order by s,f rows unbounded preceding) from over10k limit 100; + +select s, sum(f) over (partition by ts order by f range between current row and unbounded following) from over10k limit 100; + +select s, avg(f) over (partition by ts order by s,f rows between current row and 5 following) from over10k limit 100; + +select s, avg(d) over (partition by t order by s,d desc rows between 5 preceding and 5 following) from over10k limit 100; + +select s, sum(i) over(partition by ts order by s) from over10k limit 100; + +select f, sum(f) over (partition by ts order by f range between unbounded preceding and current row) from over10k limit 100; + +select s, i, round(avg(d) over (partition by s order by i) / 10.0 , 2) from over10k limit 7; + +select s, i, round((avg(d) over w1 + 10.0) - (avg(d) over w1 - 10.0),2) from over10k window w1 as (partition by s order by i) limit 7; diff --git a/src/test/hive/ql/src/test/queries/negative/ambiguous_join_col.q b/src/test/hive/ql/src/test/queries/negative/ambiguous_join_col.q new file mode 100644 index 0000000000000..e70aae46275b0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/ambiguous_join_col.q @@ -0,0 +1,2 @@ +FROM src src1 JOIN src src2 ON src1.key = src2.key +INSERT OVERWRITE TABLE dest1 SELECT key diff --git a/src/test/hive/ql/src/test/queries/negative/duplicate_alias.q b/src/test/hive/ql/src/test/queries/negative/duplicate_alias.q new file mode 100644 index 0000000000000..5fd22460c037a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/duplicate_alias.q @@ -0,0 +1,2 @@ +FROM src a JOIN src a ON (a.key = a.key) +INSERT OVERWRITE TABLE dest1 SELECT a.key, a.value diff --git a/src/test/hive/ql/src/test/queries/negative/garbage.q b/src/test/hive/ql/src/test/queries/negative/garbage.q new file mode 100644 index 0000000000000..6c8c751f21c31 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/garbage.q @@ -0,0 +1 @@ +this is totally garbage SELECT src.key WHERE a lot of garbage diff --git a/src/test/hive/ql/src/test/queries/negative/insert_wrong_number_columns.q b/src/test/hive/ql/src/test/queries/negative/insert_wrong_number_columns.q new file mode 100644 index 0000000000000..aadfbde33836a --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/insert_wrong_number_columns.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value, 1 WHERE src.key < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_create_table.q b/src/test/hive/ql/src/test/queries/negative/invalid_create_table.q new file mode 100644 index 0000000000000..899bbd368b185 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/invalid_create_table.q @@ -0,0 +1,4 @@ +CREATE TABLE mytable ( + a INT + b STRING +); diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_dot.q b/src/test/hive/ql/src/test/queries/negative/invalid_dot.q new file mode 100644 index 0000000000000..36b9bd2a3b980 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/invalid_dot.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.value.member WHERE src.key < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_function_param2.q b/src/test/hive/ql/src/test/queries/negative/invalid_function_param2.q new file mode 100644 index 0000000000000..3543449b8870b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/invalid_function_param2.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT substr('1234', 'abc'), src.value WHERE src.key < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_index.q b/src/test/hive/ql/src/test/queries/negative/invalid_index.q new file mode 100644 index 0000000000000..146bc5dc9f3b0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/invalid_index.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key[0], src.value diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_list_index.q b/src/test/hive/ql/src/test/queries/negative/invalid_list_index.q new file mode 100644 index 0000000000000..c40f079f60aaa --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/invalid_list_index.q @@ -0,0 +1,2 @@ +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.lstring['abc'] diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_list_index2.q b/src/test/hive/ql/src/test/queries/negative/invalid_list_index2.q new file mode 100644 index 0000000000000..99d0b3d4162ac --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/invalid_list_index2.q @@ -0,0 +1,2 @@ +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.lstring[1 + 2] diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_map_index.q b/src/test/hive/ql/src/test/queries/negative/invalid_map_index.q new file mode 100644 index 0000000000000..c2b9eab61b80e --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/invalid_map_index.q @@ -0,0 +1,2 @@ +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.mstringstring[0] diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_map_index2.q b/src/test/hive/ql/src/test/queries/negative/invalid_map_index2.q new file mode 100644 index 0000000000000..5828f0709f53b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/invalid_map_index2.q @@ -0,0 +1,2 @@ +FROM src_thrift +INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.mstringstring[concat('abc', 'abc')] diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_select.q b/src/test/hive/ql/src/test/queries/negative/invalid_select.q new file mode 100644 index 0000000000000..fd1298577be8c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/invalid_select.q @@ -0,0 +1,4 @@ +SELECT + trim(trim(a)) + trim(b) +FROM src; diff --git a/src/test/hive/ql/src/test/queries/negative/macro_reserved_word.q b/src/test/hive/ql/src/test/queries/negative/macro_reserved_word.q new file mode 100644 index 0000000000000..359eb9de93bac --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/macro_reserved_word.q @@ -0,0 +1 @@ +CREATE TEMPORARY MACRO DOUBLE (x DOUBLE) 1.0 / (1.0 + EXP(-x)); diff --git a/src/test/hive/ql/src/test/queries/negative/missing_overwrite.q b/src/test/hive/ql/src/test/queries/negative/missing_overwrite.q new file mode 100644 index 0000000000000..1bfeee382ea37 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/missing_overwrite.q @@ -0,0 +1,2 @@ +FROM src +INSERT TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/nonkey_groupby.q b/src/test/hive/ql/src/test/queries/negative/nonkey_groupby.q new file mode 100644 index 0000000000000..ad0f4415cbd8c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/nonkey_groupby.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 group by src.key diff --git a/src/test/hive/ql/src/test/queries/negative/quoted_string.q b/src/test/hive/ql/src/test/queries/negative/quoted_string.q new file mode 100644 index 0000000000000..0252a9e11cdf5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/quoted_string.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234", src.value WHERE src.key < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column1.q b/src/test/hive/ql/src/test/queries/negative/unknown_column1.q new file mode 100644 index 0000000000000..429cead63beb1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_column1.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.dummycol WHERE src.key < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column2.q b/src/test/hive/ql/src/test/queries/negative/unknown_column2.q new file mode 100644 index 0000000000000..3767dc4e65021 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_column2.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.dummykey < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column3.q b/src/test/hive/ql/src/test/queries/negative/unknown_column3.q new file mode 100644 index 0000000000000..2fc5f490f118d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_column3.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 group by src.dummycol diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column4.q b/src/test/hive/ql/src/test/queries/negative/unknown_column4.q new file mode 100644 index 0000000000000..8ad8dd12e46e2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_column4.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 group by dummysrc.key diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column5.q b/src/test/hive/ql/src/test/queries/negative/unknown_column5.q new file mode 100644 index 0000000000000..766b0e5255fea --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_column5.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE dummysrc.key < 100 group by src.key diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column6.q b/src/test/hive/ql/src/test/queries/negative/unknown_column6.q new file mode 100644 index 0000000000000..bb76c28623484 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_column6.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', dummysrc.value WHERE src.key < 100 group by src.key diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_function1.q b/src/test/hive/ql/src/test/queries/negative/unknown_function1.q new file mode 100644 index 0000000000000..d8ff6325b95f2 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_function1.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', dummyfn(src.value, 10) WHERE src.key < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_function2.q b/src/test/hive/ql/src/test/queries/negative/unknown_function2.q new file mode 100644 index 0000000000000..f7d255934db5c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_function2.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE anotherdummyfn('abc', src.key) + 10 < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_function3.q b/src/test/hive/ql/src/test/queries/negative/unknown_function3.q new file mode 100644 index 0000000000000..87d4edc987865 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_function3.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE anotherdummyfn('abc', src.key) + 10 < 100 group by src.key diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_function4.q b/src/test/hive/ql/src/test/queries/negative/unknown_function4.q new file mode 100644 index 0000000000000..cfe70e4f2fdc4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_function4.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', dummyfn(src.key) WHERE src.key < 100 group by src.key diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_table1.q b/src/test/hive/ql/src/test/queries/negative/unknown_table1.q new file mode 100644 index 0000000000000..585ef6d7f2db1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_table1.q @@ -0,0 +1,2 @@ +FROM dummySrc +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_table2.q b/src/test/hive/ql/src/test/queries/negative/unknown_table2.q new file mode 100644 index 0000000000000..2c69c16be5905 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/unknown_table2.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dummyDest SELECT '1234', src.value WHERE src.key < 100 diff --git a/src/test/hive/ql/src/test/queries/negative/wrong_distinct1.q b/src/test/hive/ql/src/test/queries/negative/wrong_distinct1.q new file mode 100755 index 0000000000000..d92c3bb8df4b0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/wrong_distinct1.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT DISTINCT src.key, substr(src.value,4,1) GROUP BY src.key diff --git a/src/test/hive/ql/src/test/queries/negative/wrong_distinct2.q b/src/test/hive/ql/src/test/queries/negative/wrong_distinct2.q new file mode 100755 index 0000000000000..53fb550b3d111 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/negative/wrong_distinct2.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, DISTINCT substr(src.value,4,1) GROUP BY src.key diff --git a/src/test/hive/ql/src/test/queries/positive/case_sensitivity.q b/src/test/hive/ql/src/test/queries/positive/case_sensitivity.q new file mode 100644 index 0000000000000..d7f7371507662 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/case_sensitivity.q @@ -0,0 +1,2 @@ +FROM SRC_THRIFT +INSERT OVERWRITE TABLE dest1 SELECT src_Thrift.LINT[1], src_thrift.lintstring[0].MYSTRING where src_thrift.liNT[0] > 0 diff --git a/src/test/hive/ql/src/test/queries/positive/cast1.q b/src/test/hive/ql/src/test/queries/positive/cast1.q new file mode 100644 index 0000000000000..6269c6a4e76f9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/cast1.q @@ -0,0 +1,2 @@ +FROM src +SELECT 3 + 2, 3.0 + 2, 3 + 2.0, 3.0 + 2.0, 3 + CAST(2.0 AS INT), CAST(1 AS BOOLEAN), CAST(TRUE AS INT) WHERE src.key = 86 diff --git a/src/test/hive/ql/src/test/queries/positive/groupby1.q b/src/test/hive/ql/src/test/queries/positive/groupby1.q new file mode 100755 index 0000000000000..96b29b05cc7ad --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/groupby1.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key diff --git a/src/test/hive/ql/src/test/queries/positive/groupby2.q b/src/test/hive/ql/src/test/queries/positive/groupby2.q new file mode 100755 index 0000000000000..d741eb60b6bb1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/groupby2.q @@ -0,0 +1,2 @@ +FROM src +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) diff --git a/src/test/hive/ql/src/test/queries/positive/groupby3.q b/src/test/hive/ql/src/test/queries/positive/groupby3.q new file mode 100755 index 0000000000000..03b1248a11cb6 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/groupby3.q @@ -0,0 +1,2 @@ +FROM src +SELECT sum(substr(src.value,5)), avg(substr(src.value,5)), avg(DISTINCT substr(src.value,5)), max(substr(src.value,5)), min(substr(src.value,5)) diff --git a/src/test/hive/ql/src/test/queries/positive/groupby4.q b/src/test/hive/ql/src/test/queries/positive/groupby4.q new file mode 100755 index 0000000000000..85271a9caf6e3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/groupby4.q @@ -0,0 +1,2 @@ +FROM src +SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1) diff --git a/src/test/hive/ql/src/test/queries/positive/groupby5.q b/src/test/hive/ql/src/test/queries/positive/groupby5.q new file mode 100755 index 0000000000000..ebd65b3069722 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/groupby5.q @@ -0,0 +1,4 @@ + +SELECT src.key, sum(substr(src.value,5)) +FROM src +GROUP BY src.key diff --git a/src/test/hive/ql/src/test/queries/positive/groupby6.q b/src/test/hive/ql/src/test/queries/positive/groupby6.q new file mode 100755 index 0000000000000..80654f2a9ce6b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/groupby6.q @@ -0,0 +1,2 @@ +FROM src +SELECT DISTINCT substr(src.value,5,1) diff --git a/src/test/hive/ql/src/test/queries/positive/input1.q b/src/test/hive/ql/src/test/queries/positive/input1.q new file mode 100644 index 0000000000000..fdd290d6b136b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input1.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100 diff --git a/src/test/hive/ql/src/test/queries/positive/input2.q b/src/test/hive/ql/src/test/queries/positive/input2.q new file mode 100644 index 0000000000000..4e1612ea972e9 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input2.q @@ -0,0 +1,4 @@ +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, 2 WHERE src.key >= 200 diff --git a/src/test/hive/ql/src/test/queries/positive/input20.q b/src/test/hive/ql/src/test/queries/positive/input20.q new file mode 100644 index 0000000000000..f30cf27017d96 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input20.q @@ -0,0 +1,9 @@ +FROM ( + FROM src + MAP src.key % 2, src.key % 5 + USING 'cat' + CLUSTER BY key +) tmap +REDUCE tmap.key, tmap.value +USING 'uniq -c | sed "s@^ *@@" | sed "s@\t@_@" | sed "s@ @\t@"' +AS key, value diff --git a/src/test/hive/ql/src/test/queries/positive/input3.q b/src/test/hive/ql/src/test/queries/positive/input3.q new file mode 100644 index 0000000000000..fc53e94d39f02 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input3.q @@ -0,0 +1,5 @@ +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, 2 WHERE src.key >= 200 and src.key < 300 +INSERT OVERWRITE DIRECTORY '../../../../build/contrib/hive/ql/test/data/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300 diff --git a/src/test/hive/ql/src/test/queries/positive/input4.q b/src/test/hive/ql/src/test/queries/positive/input4.q new file mode 100644 index 0000000000000..03e6de48faca3 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input4.q @@ -0,0 +1,7 @@ +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING '/bin/cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100 diff --git a/src/test/hive/ql/src/test/queries/positive/input5.q b/src/test/hive/ql/src/test/queries/positive/input5.q new file mode 100644 index 0000000000000..a46abc75833f0 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input5.q @@ -0,0 +1,7 @@ +FROM ( + FROM src_thrift + SELECT TRANSFORM(src_thrift.lint, src_thrift.lintstring) + USING '/bin/cat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue diff --git a/src/test/hive/ql/src/test/queries/positive/input6.q b/src/test/hive/ql/src/test/queries/positive/input6.q new file mode 100644 index 0000000000000..d6f25a935ae7d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input6.q @@ -0,0 +1,2 @@ +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src1.value WHERE src1.key is null diff --git a/src/test/hive/ql/src/test/queries/positive/input7.q b/src/test/hive/ql/src/test/queries/positive/input7.q new file mode 100644 index 0000000000000..33a82953c26ed --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input7.q @@ -0,0 +1,2 @@ +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key diff --git a/src/test/hive/ql/src/test/queries/positive/input8.q b/src/test/hive/ql/src/test/queries/positive/input8.q new file mode 100644 index 0000000000000..0843b9ba4e557 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input8.q @@ -0,0 +1,2 @@ +FROM src1 +SELECT 4 + NULL, src1.key - NULL, NULL + NULL diff --git a/src/test/hive/ql/src/test/queries/positive/input9.q b/src/test/hive/ql/src/test/queries/positive/input9.q new file mode 100644 index 0000000000000..2892f0b2dfc4f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input9.q @@ -0,0 +1,2 @@ +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key where NULL = NULL diff --git a/src/test/hive/ql/src/test/queries/positive/input_part1.q b/src/test/hive/ql/src/test/queries/positive/input_part1.q new file mode 100644 index 0000000000000..d45d1cd0b47ec --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input_part1.q @@ -0,0 +1,2 @@ +FROM srcpart +SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-08' and srcpart.hr = '12' diff --git a/src/test/hive/ql/src/test/queries/positive/input_testsequencefile.q b/src/test/hive/ql/src/test/queries/positive/input_testsequencefile.q new file mode 100755 index 0000000000000..cf9a092417e1d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input_testsequencefile.q @@ -0,0 +1,2 @@ +FROM src +INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value diff --git a/src/test/hive/ql/src/test/queries/positive/input_testxpath.q b/src/test/hive/ql/src/test/queries/positive/input_testxpath.q new file mode 100755 index 0000000000000..7699bff755520 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input_testxpath.q @@ -0,0 +1,2 @@ +FROM src_thrift +SELECT src_thrift.lint[1], src_thrift.lintstring[0].mystring, src_thrift.mstringstring['key_2'] diff --git a/src/test/hive/ql/src/test/queries/positive/input_testxpath2.q b/src/test/hive/ql/src/test/queries/positive/input_testxpath2.q new file mode 100644 index 0000000000000..08abaf4fad8da --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/input_testxpath2.q @@ -0,0 +1,2 @@ +FROM src_thrift +SELECT size(src_thrift.lint), size(src_thrift.lintstring), size(src_thrift.mstringstring) where src_thrift.lint IS NOT NULL AND NOT (src_thrift.mstringstring IS NULL) diff --git a/src/test/hive/ql/src/test/queries/positive/join1.q b/src/test/hive/ql/src/test/queries/positive/join1.q new file mode 100644 index 0000000000000..739c39dd8f718 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/join1.q @@ -0,0 +1,3 @@ +FROM src src1 JOIN src src2 ON (src1.key = src2.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value + diff --git a/src/test/hive/ql/src/test/queries/positive/join2.q b/src/test/hive/ql/src/test/queries/positive/join2.q new file mode 100644 index 0000000000000..a02d87f09f58c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/join2.q @@ -0,0 +1,3 @@ +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value + diff --git a/src/test/hive/ql/src/test/queries/positive/join3.q b/src/test/hive/ql/src/test/queries/positive/join3.q new file mode 100644 index 0000000000000..b57c9569d7286 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/join3.q @@ -0,0 +1,4 @@ +FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key) +INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value + + diff --git a/src/test/hive/ql/src/test/queries/positive/join4.q b/src/test/hive/ql/src/test/queries/positive/join4.q new file mode 100644 index 0000000000000..2e5967fb7d854 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/join4.q @@ -0,0 +1,14 @@ +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + LEFT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +SELECT c.c1, c.c2, c.c3, c.c4 + diff --git a/src/test/hive/ql/src/test/queries/positive/join5.q b/src/test/hive/ql/src/test/queries/positive/join5.q new file mode 100644 index 0000000000000..63a38f554a24f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/join5.q @@ -0,0 +1,15 @@ +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + RIGHT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +SELECT c.c1, c.c2, c.c3, c.c4 + + diff --git a/src/test/hive/ql/src/test/queries/positive/join6.q b/src/test/hive/ql/src/test/queries/positive/join6.q new file mode 100644 index 0000000000000..110451cf3039b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/join6.q @@ -0,0 +1,16 @@ +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + FULL OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +SELECT c.c1, c.c2, c.c3, c.c4 + + + diff --git a/src/test/hive/ql/src/test/queries/positive/join7.q b/src/test/hive/ql/src/test/queries/positive/join7.q new file mode 100644 index 0000000000000..65797b44a2cb1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/join7.q @@ -0,0 +1,21 @@ +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + FULL OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + LEFT OUTER JOIN + ( + FROM src src3 SELECT src3.key AS c5, src3.value AS c6 WHERE src3.key > 20 and src3.key < 25 + ) c + ON (a.c1 = c.c5) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4, c.c5 AS c5, c.c6 AS c6 +) c +SELECT c.c1, c.c2, c.c3, c.c4, c.c5, c.c6 + + + diff --git a/src/test/hive/ql/src/test/queries/positive/join8.q b/src/test/hive/ql/src/test/queries/positive/join8.q new file mode 100644 index 0000000000000..d215b07a6720f --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/join8.q @@ -0,0 +1,14 @@ +FROM ( + FROM + ( + FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 + ) a + LEFT OUTER JOIN + ( + FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 + ) b + ON (a.c1 = b.c3) + SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 +) c +SELECT c.c1, c.c2, c.c3, c.c4 where c.c3 IS NULL AND c.c1 IS NOT NULL + diff --git a/src/test/hive/ql/src/test/queries/positive/sample1.q b/src/test/hive/ql/src/test/queries/positive/sample1.q new file mode 100644 index 0000000000000..3a168b999d707 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/sample1.q @@ -0,0 +1,5 @@ +-- no input pruning, no sample filter +SELECT s.* +FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 1 ON rand()) s +WHERE s.ds='2008-04-08' and s.hr='11' + diff --git a/src/test/hive/ql/src/test/queries/positive/sample2.q b/src/test/hive/ql/src/test/queries/positive/sample2.q new file mode 100644 index 0000000000000..b505b896fa2c5 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/sample2.q @@ -0,0 +1,4 @@ +-- input pruning, no sample filter +-- default table sample columns +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2) s diff --git a/src/test/hive/ql/src/test/queries/positive/sample3.q b/src/test/hive/ql/src/test/queries/positive/sample3.q new file mode 100644 index 0000000000000..42d5a2bbec34b --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/sample3.q @@ -0,0 +1,4 @@ +-- sample columns not same as bucket columns +-- no input pruning, sample filter +INSERT OVERWRITE TABLE dest1 SELECT s.* -- here's another test +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key, value) s diff --git a/src/test/hive/ql/src/test/queries/positive/sample4.q b/src/test/hive/ql/src/test/queries/positive/sample4.q new file mode 100644 index 0000000000000..7b5ab03380ae1 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/sample4.q @@ -0,0 +1,4 @@ +-- bucket column is the same as table sample +-- No need for sample filter +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s diff --git a/src/test/hive/ql/src/test/queries/positive/sample5.q b/src/test/hive/ql/src/test/queries/positive/sample5.q new file mode 100644 index 0000000000000..b9b48fdc71884 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/sample5.q @@ -0,0 +1,3 @@ +-- no input pruning, sample filter +INSERT OVERWRITE TABLE dest1 SELECT s.* -- here's another test +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 on key) s diff --git a/src/test/hive/ql/src/test/queries/positive/sample6.q b/src/test/hive/ql/src/test/queries/positive/sample6.q new file mode 100644 index 0000000000000..0ee026f0f368c --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/sample6.q @@ -0,0 +1,3 @@ +-- both input pruning and sample filter +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s diff --git a/src/test/hive/ql/src/test/queries/positive/sample7.q b/src/test/hive/ql/src/test/queries/positive/sample7.q new file mode 100644 index 0000000000000..f17ce105c3572 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/sample7.q @@ -0,0 +1,4 @@ +-- both input pruning and sample filter +INSERT OVERWRITE TABLE dest1 SELECT s.* +FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s +WHERE s.key > 100 diff --git a/src/test/hive/ql/src/test/queries/positive/subq.q b/src/test/hive/ql/src/test/queries/positive/subq.q new file mode 100644 index 0000000000000..6392dbcc4380d --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/subq.q @@ -0,0 +1,4 @@ +FROM ( + FROM src select src.* WHERE src.key < 100 +) unioninput +INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.* diff --git a/src/test/hive/ql/src/test/queries/positive/udf1.q b/src/test/hive/ql/src/test/queries/positive/udf1.q new file mode 100644 index 0000000000000..2ecf46e742c30 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/udf1.q @@ -0,0 +1,5 @@ +FROM src SELECT 'a' LIKE '%a%', 'b' LIKE '%a%', 'ab' LIKE '%a%', 'ab' LIKE '%a_', + '%_' LIKE '\%\_', 'ab' LIKE '\%\_', 'ab' LIKE '_a%', 'ab' LIKE 'a', + '' RLIKE '.*', 'a' RLIKE '[ab]', '' RLIKE '[ab]', 'hadoop' RLIKE '[a-z]*', 'hadoop' RLIKE 'o*', + REGEXP_REPLACE('abc', 'b', 'c'), REGEXP_REPLACE('abc', 'z', 'a'), REGEXP_REPLACE('abbbb', 'bb', 'b'), REGEXP_REPLACE('hadoop', '(.)[a-z]*', '$1ive') + WHERE src.key = 86 diff --git a/src/test/hive/ql/src/test/queries/positive/udf4.q b/src/test/hive/ql/src/test/queries/positive/udf4.q new file mode 100644 index 0000000000000..f3a7598e17210 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/udf4.q @@ -0,0 +1 @@ +SELECT round(1.0), round(1.5), round(-1.5), floor(1.0), floor(1.5), floor(-1.5), sqrt(1.0), sqrt(-1.0), sqrt(0.0), ceil(1.0), ceil(1.5), ceil(-1.5), ceiling(1.0), rand(3), +3, -3, 1++2, 1+-2, ~1 FROM dest1 diff --git a/src/test/hive/ql/src/test/queries/positive/udf6.q b/src/test/hive/ql/src/test/queries/positive/udf6.q new file mode 100644 index 0000000000000..65791c41c1ff4 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/udf6.q @@ -0,0 +1 @@ +FROM src SELECT CONCAT('a', 'b'), IF(TRUE, 1 ,2) diff --git a/src/test/hive/ql/src/test/queries/positive/udf_case.q b/src/test/hive/ql/src/test/queries/positive/udf_case.q new file mode 100644 index 0000000000000..0c86da2198699 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/udf_case.q @@ -0,0 +1,10 @@ +SELECT CASE 1 + WHEN 1 THEN 2 + WHEN 3 THEN 4 + ELSE 5 + END, + CASE 11 + WHEN 12 THEN 13 + WHEN 14 THEN 15 + END +FROM src LIMIT 1 diff --git a/src/test/hive/ql/src/test/queries/positive/udf_when.q b/src/test/hive/ql/src/test/queries/positive/udf_when.q new file mode 100644 index 0000000000000..99ed09990b874 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/udf_when.q @@ -0,0 +1,10 @@ +SELECT CASE + WHEN 1=1 THEN 2 + WHEN 3=5 THEN 4 + ELSE 5 + END, + CASE + WHEN 12=11 THEN 13 + WHEN 14=10 THEN 15 + END +FROM src LIMIT 1 diff --git a/src/test/hive/ql/src/test/queries/positive/union.q b/src/test/hive/ql/src/test/queries/positive/union.q new file mode 100644 index 0000000000000..6a6b9882aee71 --- /dev/null +++ b/src/test/hive/ql/src/test/queries/positive/union.q @@ -0,0 +1,6 @@ +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.* diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index f70904551a01c..4d4a77269813b 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -10,7 +10,7 @@ import util._ */ class HiveCompatibility extends HiveQueryFileTest { // TODO: bundle in jar files... get from classpath - lazy val hiveQueryDir = new File(TestShark.hiveDevHome, "ql/src/test/queries/clientpositive") + lazy val hiveQueryDir = TestShark.getHiveFile("ql/src/test/queries/clientpositive") def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) /** A list of tests deemed out of scope currently and thus completely disregarded. */ From d20b565a36533245d0357b18332e8c8658821a2e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 15:10:04 -0800 Subject: [PATCH 577/778] fix if style --- src/main/scala/catalyst/execution/TestShark.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 0ff5716d70e37..cf73abe135113 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -86,10 +86,11 @@ object TestShark extends SharkInstance { * hive test cases assume the system is set up. */ private def rewritePaths(cmd: String): String = - if (cmd.toUpperCase contains "LOAD DATA") + if (cmd.toUpperCase contains "LOAD DATA") { cmd.replaceAll("\\.\\.", TestShark.inRepoTests.getCanonicalPath) - else + } else { cmd + } val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") hiveFilesTemp.delete() From 807b2d7ce15ef78f73acfe4950a8fd14b6784545 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 16:03:46 -0800 Subject: [PATCH 578/778] check style and publish docs with travis --- .travis.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index e0abe3e54726d..335a3aa8bf5fe 100644 --- a/.travis.yml +++ b/.travis.yml @@ -2,4 +2,6 @@ scala: - "2.10.3" jdk: - - oraclejdk7 \ No newline at end of file + - oraclejdk7 + script: + - "GIT_AUTHOR_NAME=\"Michael Armbrust\" GIT_AUTHOR_EMAIL=\"michael@databricks.com\" GIT_COMMITTER_NAME=\"Michael Armbrust\" GIT_COMMITTER_EMAIL=\"michael@databricks.com\" sbt ++$TRAVIS_SCALA_VERSION 'set scalacOptions += \"-Xfatal-warnings\"' test:compile scalastyle ghpages-push-site" \ No newline at end of file From d3a3d48d6ad2aa3562b0859f2af13dd8d8b75fd7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 16:12:33 -0800 Subject: [PATCH 579/778] add testing to travis --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 335a3aa8bf5fe..3381d2fbb1171 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,4 +4,4 @@ jdk: - oraclejdk7 script: - - "GIT_AUTHOR_NAME=\"Michael Armbrust\" GIT_AUTHOR_EMAIL=\"michael@databricks.com\" GIT_COMMITTER_NAME=\"Michael Armbrust\" GIT_COMMITTER_EMAIL=\"michael@databricks.com\" sbt ++$TRAVIS_SCALA_VERSION 'set scalacOptions += \"-Xfatal-warnings\"' test:compile scalastyle ghpages-push-site" \ No newline at end of file + - "GIT_AUTHOR_NAME=\"Michael Armbrust\" GIT_AUTHOR_EMAIL=\"michael@databricks.com\" GIT_COMMITTER_NAME=\"Michael Armbrust\" GIT_COMMITTER_EMAIL=\"michael@databricks.com\" sbt ++$TRAVIS_SCALA_VERSION 'set scalacOptions += \"-Xfatal-warnings\"' test:compile scalastyle test ghpages-push-site" \ No newline at end of file From 271e483d65dc41a4feb6f9f4018379094c4ff0bf Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 16:28:47 -0800 Subject: [PATCH 580/778] Update build status icon. [no ci] --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index a20cfd8e01274..4e3fb4113992a 100644 --- a/README.md +++ b/README.md @@ -2,7 +2,7 @@ catalyst ======== Catalyst is a functional framework for optimizing relational query plans. -[![Build Status](https://databricks.ci.cloudbees.com/buildStatus/icon?job=Catalyst-Master)](https://databricks.ci.cloudbees.com/job/Catalyst-Master/) +[![Build Status](https://magnum.travis-ci.com/databricks/catalyst.png?token=sNeje9KkkWMHYrVqko4t&branch=master)](https://magnum.travis-ci.com/databricks/catalyst) More documentation can be found in the project's [scaladoc](http://databricks.github.io/catalyst/latest/api/#catalyst.package) From fc67b5078c23c88b6387cf2b948d84a99cc87e08 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 1 Feb 2014 08:46:18 +0800 Subject: [PATCH 581/778] Check for a Sort operator with the global flag set instead of an Exchange operator with a RangePartitioning. --- src/test/scala/catalyst/execution/HiveComparisonTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index 1b4d01c3ce3a9..4a557c06c6daf 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -109,7 +109,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G case _ => // TODO: Really we only care about the final total ordering here... val isOrdered = sharkQuery.executedPlan.collect { - case s @ Exchange(r: RangePartitioning, _) => s + case s @ Sort(_, global, _) if global => s }.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. if (!isOrdered) answer.sorted else answer From 45b334b4d06d254c3b9a8f03b2e64f14b48a3c88 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 1 Feb 2014 09:11:07 +0800 Subject: [PATCH 582/778] fix comments --- src/main/scala/catalyst/execution/PlanningStrategies.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index fd9e5e291d98b..10f0f051bddc8 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -143,10 +143,11 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - // Set the requiredDistribution of this SortPartitions to OrderedDistribution. + // This sort is a global sort. Its requiredDistribution will be an OrderedDistribution. execution.Sort(sortExprs, true, planLater(child)):: Nil case logical.SortPartitions(sortExprs, child) => - // Set the requiredDistribution of this SortPartitions to UnspecifiedDistribution. + // This sort only sort tuples within a partition. Its requiredDistribution will be + // an UnspecifiedDistribution. execution.Sort(sortExprs, false, planLater(child)) :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil From e079f2b32d3391bdfe835ca66dde7eaedf5df5c0 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Wed, 15 Jan 2014 22:53:00 -0800 Subject: [PATCH 583/778] Add GenericUDAF wrapper and HiveUDAFFunction --- .../catalyst/execution/FunctionRegistry.scala | 105 ++++++++++++++---- .../scala/catalyst/execution/TestShark.scala | 8 +- .../scala/catalyst/execution/aggregates.scala | 31 ++++++ src/main/scala/catalyst/frontend/Hive.scala | 13 +++ 4 files changed, 135 insertions(+), 22 deletions(-) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index 5ebc30fc982df..8b49366fcf46b 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -4,17 +4,21 @@ package execution import scala.collection.JavaConversions._ import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} -import org.apache.hadoop.hive.ql.udf.generic.GenericUDF +import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFEvaluator, AbstractGenericUDAFResolver, GenericUDF} import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.hive.serde2.{io => hiveIo} -import org.apache.hadoop.hive.serde2.objectinspector.primitive.AbstractPrimitiveJavaObjectInspector -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.{io => hadoopIo} import expressions._ import types._ +import org.apache.hadoop.hive.serde2.objectinspector.{ListObjectInspector, StructObjectInspector, ObjectInspector} +import catalyst.types.StructField +import catalyst.types.StructType +import catalyst.types.ArrayType +import catalyst.expressions.Cast -object HiveFunctionRegistry extends analysis.FunctionRegistry { +object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionFactory { def lookupFunction(name: String, children: Seq[Expression]): Expression = { // We only look it up to see if it exists, but do not include it in the HiveUDF since it is // not always serializable. @@ -22,8 +26,7 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry { sys.error(s"Couldn't find function $name")) if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { - val functionInfo = FunctionRegistry.getFunctionInfo(name) - val function = functionInfo.getFunctionClass.newInstance.asInstanceOf[UDF] + val function = createFunction[UDF](name) val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) lazy val expectedDataTypes = method.getParameterTypes.map(javaClassToDataType) @@ -34,6 +37,8 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry { ) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdf(name, IntegerType, children) + } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveGenericUdaf(name, children) } else { sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") } @@ -67,20 +72,10 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry { } } -abstract class HiveUdf extends Expression with ImplementedUdf with Logging { - self: Product => - - type UDFType - val name: String - - def nullable = true - def references = children.flatMap(_.references).toSet - - // FunctionInfo is not serializable so we must look it up here again. - lazy val functionInfo = FunctionRegistry.getFunctionInfo(name) - lazy val function = functionInfo.getFunctionClass.newInstance.asInstanceOf[UDFType] - - override def toString = s"${nodeName}#${functionInfo.getDisplayName}(${children.mkString(",")})" +trait HiveFunctionFactory { + def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) + def getFunctionClass(name: String) = getFunctionInfo(name).getFunctionClass + def createFunction[UDFType](name: String) = getFunctionClass(name).newInstance.asInstanceOf[UDFType] def unwrap(a: Any): Any = a match { case null => null @@ -93,6 +88,7 @@ abstract class HiveUdf extends Expression with ImplementedUdf with Logging { case b: hadoopIo.BooleanWritable => b.get() case b: hiveIo.ByteWritable => b.get case list: java.util.List[_] => list.map(unwrap) + case array: Array[_] => array.map(unwrap) case p: java.lang.Short => p case p: java.lang.Long => p case p: java.lang.Float => p @@ -104,6 +100,24 @@ abstract class HiveUdf extends Expression with ImplementedUdf with Logging { } } +abstract class HiveUdf extends Expression with ImplementedUdf with Logging with HiveFunctionFactory { + self: Product => + + type UDFType + val name: String + + def nullable = true + def references = children.flatMap(_.references).toSet + + // FunctionInfo is not serializable so we must look it up here again. + lazy val functionInfo = getFunctionInfo(name) + lazy val function = createFunction[UDFType](name) + + override def toString = s"${nodeName}#${functionInfo.getDisplayName}(${children.mkString(",")})" + + +} + case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { import HiveFunctionRegistry._ type UDFType = UDF @@ -194,3 +208,54 @@ case class HiveGenericUdf( unwrap(instance.evaluate(args)) } } + +trait HiveInspectors { + def toInspectors(exprs: Seq[Expression]) = exprs.map(_.dataType).map { + case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector + case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector + case DoubleType => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector + case BooleanType => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector + case LongType => PrimitiveObjectInspectorFactory.javaLongObjectInspector + case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector + case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector + } + + def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { + case s: StructObjectInspector => + StructType(s.getAllStructFieldRefs.map(f => { + StructField(f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), true) + })) + case l: ListObjectInspector => ArrayType(inspectorToDataType(l.getListElementObjectInspector)) + case _: WritableStringObjectInspector => StringType + case _: WritableIntObjectInspector => IntegerType + case _: WritableDoubleObjectInspector => DoubleType + case _: WritableBooleanObjectInspector => BooleanType + case _: WritableLongObjectInspector => LongType + case _: WritableShortObjectInspector => ShortType + case _: WritableByteObjectInspector => ByteType + } +} + +case class HiveGenericUdaf( + name: String, + children: Seq[Expression]) extends AggregateExpression + with HiveInspectors + with HiveFunctionFactory { + + lazy val resolver = createFunction[AbstractGenericUDAFResolver](name) + + lazy val objectInspector: ObjectInspector = { + resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) + .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) + } + + type UDFType = AbstractGenericUDAFResolver + + lazy val inspectors: Seq[ObjectInspector] = toInspectors(children) + + def dataType: DataType = inspectorToDataType(objectInspector) + + def nullable: Boolean = true + + def references: Set[Attribute] = children.map(_.references).flatten.toSet +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index cf73abe135113..bf8ef9b4c3865 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -86,8 +86,12 @@ object TestShark extends SharkInstance { * hive test cases assume the system is set up. */ private def rewritePaths(cmd: String): String = - if (cmd.toUpperCase contains "LOAD DATA") { - cmd.replaceAll("\\.\\.", TestShark.inRepoTests.getCanonicalPath) + if (cmd.toUpperCase.contains("LOAD DATA") && cmd.contains("..")) { + "[\"\']([\\./\\w]+)[\"\'] ".r.findFirstMatchIn(cmd) + .map(r => { + val newPath = new File(TestShark.inRepoTests.getCanonicalPath, cmd.substring(r.start + 1, r.end - 2).replaceFirst("(\\.\\./)+", "")).getAbsolutePath + cmd.substring(0, r.start + 1) + newPath + cmd.substring(r.end - 2) + }).getOrElse(cmd) } else { cmd } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index c15414cf002f9..2b01c6eccb719 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,6 +3,7 @@ package execution import catalyst.errors._ import catalyst.expressions._ +import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFEvaluator, AbstractGenericUDAFResolver} /* Implicits */ import org.apache.spark.SparkContext._ @@ -87,6 +88,35 @@ case class Aggregate( } override def otherCopyArgs = sc :: Nil + + case class HiveUdafFunction( + exprs: Seq[Expression], + base: AggregateExpression, + functionName: String) + extends AggregateFunction + with HiveInspectors + with HiveFunctionFactory { + + def this() = this(null, null, null) + + val resolver = createFunction[AbstractGenericUDAFResolver](functionName) + + val function = { + val evaluator = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray) + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, toInspectors(exprs).toArray) + evaluator + } + + val buffer = function.getNewAggregationBuffer + + def result: Any = unwrap(function.evaluate(buffer)) + + def apply(input: Seq[Row]): Unit = { + val inputs = exprs.map(Evaluate(_, input).asInstanceOf[AnyRef]).toArray + function.iterate(buffer, inputs) + } + } + def output = aggregateExpressions.map(_.toAttribute) def createAggregateImplementations() = aggregateExpressions.map { agg => @@ -97,6 +127,7 @@ case class Aggregate( // TODO: Create custom query plan node that calculates distinct values efficiently. case base @ CountDistinct(expr) => new CountDistinctFunction(expr, base) case base @ First(expr) => new FirstFunction(expr, base) + case base @ HiveGenericUdaf(resolver, expr) => new HiveUdafFunction(expr, base, resolver) } val remainingAttributes = impl.collect { case a: Attribute => a } diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index b0af22caa8eb2..7d3f1c654c996 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -703,6 +703,7 @@ object HiveQl { case Token("TOK_FUNCTION", Token(SUM(), Nil) :: arg :: Nil) => Sum(nodeToExpr(arg)) /* Casts */ +<<<<<<< HEAD case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => @@ -721,6 +722,18 @@ object HiveQl { Cast(nodeToExpr(arg), BinaryType) case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BooleanType) +======= + case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_VARCHAR", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), IntegerType) + case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), LongType) + case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), FloatType) + case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), DoubleType) + case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ShortType) + case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ByteType) + case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BinaryType) + case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BooleanType) +>>>>>>> Add GenericUDAF wrapper and HiveUDAFFunction /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) From 8e0931f1ca55aff597132c6a27ed058866680db5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 28 Jan 2014 14:15:03 -0800 Subject: [PATCH 584/778] Cast to avoid using deprecated hive API. --- src/main/scala/catalyst/execution/aggregates.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 2b01c6eccb719..a7b5d5da489e2 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -107,7 +107,8 @@ case class Aggregate( evaluator } - val buffer = function.getNewAggregationBuffer + // Cast required to avoid type inference selecting a deprecated Hive API. + val buffer = function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] def result: Any = unwrap(function.evaluate(buffer)) From b1151a8a13b6a3cd1dfa53115b67610955112d66 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Wed, 29 Jan 2014 09:58:26 -0800 Subject: [PATCH 585/778] Fix load data regex --- src/main/scala/catalyst/execution/TestShark.scala | 2 +- .../scala/catalyst/execution/aggregates.scala | 2 +- src/main/scala/catalyst/frontend/Hive.scala | 15 ++------------- 3 files changed, 4 insertions(+), 15 deletions(-) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index bf8ef9b4c3865..51d138e019ae4 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -87,7 +87,7 @@ object TestShark extends SharkInstance { */ private def rewritePaths(cmd: String): String = if (cmd.toUpperCase.contains("LOAD DATA") && cmd.contains("..")) { - "[\"\']([\\./\\w]+)[\"\'] ".r.findFirstMatchIn(cmd) + "[\"\'](../.*)[\"\'] ".r.findFirstMatchIn(cmd) .map(r => { val newPath = new File(TestShark.inRepoTests.getCanonicalPath, cmd.substring(r.start + 1, r.end - 2).replaceFirst("(\\.\\./)+", "")).getAbsolutePath cmd.substring(0, r.start + 1) + newPath + cmd.substring(r.end - 2) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index a7b5d5da489e2..5f2d2db15fe7e 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -107,7 +107,7 @@ case class Aggregate( evaluator } - // Cast required to avoid type inference selecting a deprecated Hive API. + // Cast required to avoid type inference selecting a deprecated Hive API. val buffer = function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] def result: Any = unwrap(function.evaluate(buffer)) diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 7d3f1c654c996..595d1cd9250c6 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -703,9 +703,10 @@ object HiveQl { case Token("TOK_FUNCTION", Token(SUM(), Nil) :: arg :: Nil) => Sum(nodeToExpr(arg)) /* Casts */ -<<<<<<< HEAD case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_VARCHAR", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), StringType) case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), IntegerType) case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => @@ -722,18 +723,6 @@ object HiveQl { Cast(nodeToExpr(arg), BinaryType) case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BooleanType) -======= - case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_VARCHAR", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), IntegerType) - case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), LongType) - case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), FloatType) - case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), DoubleType) - case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ShortType) - case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), ByteType) - case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BinaryType) - case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BooleanType) ->>>>>>> Add GenericUDAF wrapper and HiveUDAFFunction /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) From 41b41f3c6ff0b06e6ac76a6a17c929c3bae8be8a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 01:39:11 -0800 Subject: [PATCH 586/778] Only cast unresolved inserts. --- src/main/scala/catalyst/analysis/Analyzer.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 0ba054d7b3041..892a5db0102b3 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -152,17 +152,18 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool // Wait until children are resolved case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table, _, child) => + case p @ InsertIntoTable(table, _, child) if !p.resolved => val childOutputDataTypes = child.output.map(_.dataType) val tableOutputDataTypes = table.output.map(_.dataType) - if (childOutputDataTypes sameElements tableOutputDataTypes) { + if (childOutputDataTypes == tableOutputDataTypes) { p } else { // Only do the casting when child output data types differ from table output data types. val castedChildOutput = child.output.zip(table.output).map { - case (l, r) if l.dataType != r.dataType => Alias(Cast(l, r.dataType), l.name)() - case (l, _) => l + case (input, table) if input.dataType != table.dataType => + Alias(Cast(input, table.dataType), input.name)() + case (input, _) => input } p.copy(child = Project(castedChildOutput, child)) From 63003e90fb70e13d22ad7e260e29897286a7776b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 12:37:58 -0800 Subject: [PATCH 587/778] Fix spacing. --- src/main/scala/catalyst/execution/aggregates.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 51b6d27bd75d2..ea8b7a1aeab63 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -26,9 +26,9 @@ case class Aggregate( override def otherCopyArgs = sc :: Nil case class HiveUdafFunction( - exprs: Seq[Expression], - base: AggregateExpression, - functionName: String) + exprs: Seq[Expression], + base: AggregateExpression, + functionName: String) extends AggregateFunction with HiveInspectors with HiveFunctionFactory { From cb775ac99241f26461a19646b9c6db660a6a2eeb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 12 Jan 2014 14:15:44 -0800 Subject: [PATCH 588/778] get rid of SharkContext singleton --- .../execution/PlanningStrategies.scala | 6 ++--- .../catalyst/execution/SharkContext.scala | 24 +++++-------------- .../catalyst/execution/SharkInstance.scala | 2 +- .../scala/catalyst/execution/TestShark.scala | 4 +++- .../catalyst/execution/hiveOperators.scala | 5 ++-- 5 files changed, 16 insertions(+), 25 deletions(-) diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index 10f0f051bddc8..6add5115d5c2c 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -24,9 +24,9 @@ trait PlanningStrategies { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { // Push attributes into table scan when possible. case p @ logical.Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => - execution.HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m, None) :: Nil + execution.HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m, None)(sc) :: Nil case m: MetastoreRelation => - execution.HiveTableScan(m.output, m, None) :: Nil + execution.HiveTableScan(m.output, m, None)(sc) :: Nil case _ => Nil } @@ -60,7 +60,7 @@ trait PlanningStrategies { } val scan = execution.HiveTableScan( - relation.output, relation, pruningPredicates.reduceLeftOption(And)) + relation.output, relation, pruningPredicates.reduceLeftOption(And))(sc) otherPredicates .reduceLeftOption(And) diff --git a/src/main/scala/catalyst/execution/SharkContext.scala b/src/main/scala/catalyst/execution/SharkContext.scala index cdd53bcb90d8a..1f277b8312f19 100644 --- a/src/main/scala/catalyst/execution/SharkContext.scala +++ b/src/main/scala/catalyst/execution/SharkContext.scala @@ -15,7 +15,6 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.{SparkContext, SparkEnv} - class SharkContext( master: String, jobName: String, @@ -26,8 +25,12 @@ class SharkContext( @transient val sparkEnv = SparkEnv.get - SharkContext.init() - import SharkContext._ + + @transient val hiveconf = new HiveConf(classOf[SessionState]) + + @transient val sessionState = new SessionState(hiveconf) + sessionState.out = new PrintStream(System.out, true, "UTF-8") + sessionState.err = new PrintStream(System.out, true, "UTF-8") /** * Execute the command using Hive and return the results as a sequence. Each element @@ -65,18 +68,3 @@ class SharkContext( } -object SharkContext { - // Since we can never properly shut down Hive, we put the Hive related initializations - // here in a global singleton. - - @transient val hiveconf = new HiveConf(classOf[SessionState]) - - @transient val sessionState = new SessionState(hiveconf) - sessionState.out = new PrintStream(System.out, true, "UTF-8") - sessionState.err = new PrintStream(System.out, true, "UTF-8") - - // A dummy init to make sure the object is properly initialized. - def init() {} -} - - diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 044bb8eb2eb3c..e6da06b507a3f 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -53,7 +53,7 @@ abstract class SharkInstance extends Logging { configure() // Must be called before initializing the catalog below. /* A catalyst metadata catalog that points to the Shark/Hive Metastore. */ - val catalog = new HiveMetastoreCatalog(SharkContext.hiveconf) + val catalog = new HiveMetastoreCatalog(sc.hiveconf) /* An analyzer that uses the Shark/Hive metastore. */ val analyze = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 51d138e019ae4..3f9291f79a089 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -21,6 +21,8 @@ import plans.logical.LogicalPlan import frontend.hive._ import util._ +object TestShark extends TestSharkInstance + /** * A locally running test instance of spark. The lifecycle for a given query is managed by the * inner class [[SharkQuery]]. A [[SharkQuery]] can either be instantiated directly or using the @@ -49,7 +51,7 @@ import util._ * metastore seems to lead to weird non-deterministic failures. Therefore, the execution of * testcases that rely on TestShark must be serialized. */ -object TestShark extends SharkInstance { +class TestSharkInstance extends SharkInstance { self => lazy val master = "local" diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 0ba73f2404fec..759b0c016bd76 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -34,7 +34,8 @@ import scala.collection.JavaConversions._ case class HiveTableScan( attributes: Seq[Attribute], relation: MetastoreRelation, - partitionPruningPred: Option[Expression]) + partitionPruningPred: Option[Expression])( + @transient val sc: SharkContext) extends LeafNode { require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, @@ -51,7 +52,7 @@ case class HiveTableScan( } @transient - val hadoopReader = new HadoopTableReader(relation.tableDesc, SharkContext.hiveconf) + val hadoopReader = new HadoopTableReader(relation.tableDesc, sc.hiveconf) /** * The hive object inspector for this table, which can be used to extract values from the From dfb67aa73ce15d9a9c355afaa1d690b3aad41843 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 12 Jan 2014 17:47:55 -0800 Subject: [PATCH 589/778] add test case --- .../catalyst/execution/ConcurrentHiveTests.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 src/test/scala/catalyst/execution/ConcurrentHiveTests.scala diff --git a/src/test/scala/catalyst/execution/ConcurrentHiveTests.scala b/src/test/scala/catalyst/execution/ConcurrentHiveTests.scala new file mode 100644 index 0000000000000..002f992a21fff --- /dev/null +++ b/src/test/scala/catalyst/execution/ConcurrentHiveTests.scala @@ -0,0 +1,16 @@ +package catalyst +package execution + +import org.scalatest.{FunSuite, BeforeAndAfterAll} + +class ConcurrentHiveTests extends FunSuite with BeforeAndAfterAll { + test("Multiple Hive Instances") { + (1 to 10).map { i => + val ts = new TestSharkInstance + ts.runSqlHive("SHOW TABLES") + val q = ts.stringToTestQuery("SELECT * FROM src").q + q.toRdd.collect() + ts.runSqlHive("SHOW TABLES") + } + } +} \ No newline at end of file From 19bfd74f9b7a3cc9dc7b7cc6477908abbd6826d9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 21 Jan 2014 23:08:31 -0800 Subject: [PATCH 590/778] store hive output in circular buffer --- .../catalyst/execution/SharkContext.scala | 102 +++++++++++++----- .../catalyst/execution/SharkInstance.scala | 2 +- 2 files changed, 74 insertions(+), 30 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkContext.scala b/src/main/scala/catalyst/execution/SharkContext.scala index 1f277b8312f19..dd33b661b97fc 100644 --- a/src/main/scala/catalyst/execution/SharkContext.scala +++ b/src/main/scala/catalyst/execution/SharkContext.scala @@ -1,6 +1,6 @@ package catalyst.execution -import java.io.PrintStream +import java.io._ import java.util.{ArrayList => JArrayList} import scala.collection.Map @@ -24,45 +24,89 @@ class SharkContext( extends SparkContext(master, jobName, sparkHome, jars, environment) { @transient val sparkEnv = SparkEnv.get - - @transient val hiveconf = new HiveConf(classOf[SessionState]) @transient val sessionState = new SessionState(hiveconf) - sessionState.out = new PrintStream(System.out, true, "UTF-8") - sessionState.err = new PrintStream(System.out, true, "UTF-8") + + // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. + val outputBuffer = new java.io.OutputStream { + var pos: Int = 0 + var buffer = new Array[Int](1024) + def write(i: Int): Unit = { + buffer(pos) = i + pos = (pos + 1) % buffer.size + } + + override def toString = { + val (end, start) = buffer.splitAt(pos) + val input = new java.io.InputStream { + val iterator = (start ++ end).iterator + + def read(): Int = if(iterator.hasNext) iterator.next else -1 + } + val reader = new BufferedReader(new InputStreamReader(input)) + val stringBuilder = new StringBuilder + var line = reader.readLine() + while(line != null) { + stringBuilder.append(line) + line = reader.readLine() + } + stringBuilder.toString() + } + } + + sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") + sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") /** * Execute the command using Hive and return the results as a sequence. Each element * in the sequence is one row. */ def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { - SparkEnv.set(sparkEnv) - val cmd_trimmed: String = cmd.trim() - val tokens: Array[String] = cmd_trimmed.split("\\s+") - val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) - - SessionState.start(sessionState) - - if (proc.isInstanceOf[Driver]) { - val driver: Driver = proc.asInstanceOf[Driver] - driver.init() - - val results = new JArrayList[String] - val response: CommandProcessorResponse = driver.run(cmd) - // Throw an exception if there is an error in query processing. - if (response.getResponseCode != 0) { + try { + SparkEnv.set(sparkEnv) + val cmd_trimmed: String = cmd.trim() + val tokens: Array[String] = cmd_trimmed.split("\\s+") + val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() + val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) + + SessionState.start(sessionState) + + if (proc.isInstanceOf[Driver]) { + val driver: Driver = proc.asInstanceOf[Driver] + driver.init() + + val results = new JArrayList[String] + val response: CommandProcessorResponse = driver.run(cmd) + // Throw an exception if there is an error in query processing. + if (response.getResponseCode != 0) { + driver.destroy() + throw new QueryExecutionException(response.getErrorMessage) + } + driver.setMaxRows(maxRows) + driver.getResults(results) driver.destroy() - throw new QueryExecutionException(response.getErrorMessage) + results + } else { + sessionState.out.println(tokens(0) + " " + cmd_1) + Seq(proc.run(cmd_1).getResponseCode.toString) } - driver.setMaxRows(maxRows) - driver.getResults(results) - driver.destroy() - results - } else { - sessionState.out.println(tokens(0) + " " + cmd_1) - Seq(proc.run(cmd_1).getResponseCode.toString) + }catch { + case e: Exception => + println( + """ + |====================== + |HIVE FAILURE OUTPUT + |====================== + """.stripMargin) + println(outputBuffer.toString) + println( + """ + |====================== + |END HIVE FAILURE OUTPUT + |====================== + """.stripMargin) + throw e } } } diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index e6da06b507a3f..626ac11781376 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -22,7 +22,7 @@ class LocalSharkInstance(val master: String) extends SharkInstance { /** * An instance of the shark execution engine. This class is responsible for taking queries - * expressed either in SQl or as raw catalyst logical plans and optimizing them for execution + * expressed either in SQL or as raw catalyst logical plans and optimizing them for execution * using Spark. Additionally this class maintains the connection with the hive metadata store. */ abstract class SharkInstance extends Logging { From 1590568ddbeee565bc483ccfe089b287433643a4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 17:57:48 -0800 Subject: [PATCH 591/778] add log4j.properties --- src/test/resources/log4j.properties | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 src/test/resources/log4j.properties diff --git a/src/test/resources/log4j.properties b/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..baa849b47602c --- /dev/null +++ b/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# 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. +# + +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootCategory=WARN, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=core/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN From b649c20a124ef2e7cd8c026ffb06be759d608cec Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 21:13:30 -0800 Subject: [PATCH 592/778] fix test logging / caching. --- .travis.yml | 5 ++++ src/test/resources/log4j.properties | 26 ++++++++++++++----- .../execution/HiveComparisonTest.scala | 11 ++------ .../execution/HiveQueryFileTest.scala | 2 +- 4 files changed, 28 insertions(+), 16 deletions(-) diff --git a/.travis.yml b/.travis.yml index 3381d2fbb1171..d4a555eac43bb 100644 --- a/.travis.yml +++ b/.travis.yml @@ -3,5 +3,10 @@ - "2.10.3" jdk: - oraclejdk7 + cache: + directories: + - $HOME/.m2 + - $HOME/.ivy + - $HOME/.sbt script: - "GIT_AUTHOR_NAME=\"Michael Armbrust\" GIT_AUTHOR_EMAIL=\"michael@databricks.com\" GIT_COMMITTER_NAME=\"Michael Armbrust\" GIT_COMMITTER_EMAIL=\"michael@databricks.com\" sbt ++$TRAVIS_SCALA_VERSION 'set scalacOptions += \"-Xfatal-warnings\"' test:compile scalastyle test ghpages-push-site" \ No newline at end of file diff --git a/src/test/resources/log4j.properties b/src/test/resources/log4j.properties index baa849b47602c..f4bbd6bcca5e4 100644 --- a/src/test/resources/log4j.properties +++ b/src/test/resources/log4j.properties @@ -16,13 +16,27 @@ # # Set everything to be logged to the file core/target/unit-tests.log -log4j.rootCategory=WARN, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false -log4j.appender.file.file=core/target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.rootLogger=DEBUG, CA, FA + +#Console Appender +log4j.appender.CA=org.apache.log4j.ConsoleAppender +log4j.appender.CA.layout=org.apache.log4j.PatternLayout +log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.CA.Threshold = WARN + + +#File Appender +log4j.appender.FA=org.apache.log4j.FileAppender +log4j.appender.FA.append=false +log4j.appender.FA.file=target/unit-tests.log +log4j.appender.FA.layout=org.apache.log4j.PatternLayout +log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n + +# Set the logger level of File Appender to WARN +log4j.appender.FA.Threshold = INFO # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=ERROR +org.apache.hadoop.hive.serde2.lazy.LazyStruct=ERROR diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index 4a557c06c6daf..46ea0c1d94d46 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -171,12 +171,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G } test(testCaseName) { - logger.error( - s""" - |============================= - |HIVE TEST: $testCaseName - |============================= - """.stripMargin) + logger.error(s"=== HIVE TEST: $testCaseName ===") // Clear old output for this testcase. outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) @@ -236,7 +231,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G val hiveResults: Seq[Seq[String]] = if (hiveCachedResults.size == queryList.size) { - logger.warn(s"Using answer cache for test: $testCaseName") + logger.info(s"Using answer cache for test: $testCaseName") hiveCachedResults } else { @@ -252,7 +247,6 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G sys.error("hive exec hooks not supported for tests.") logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") - info(s"HIVE: $queryString") // Analyze the query with catalyst to ensure test tables are loaded. val answer = sharkQuery.analyzed match { case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. @@ -287,7 +281,6 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => - info(queryString) val query = new TestShark.SharkSqlQuery(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { case e: Exception => diff --git a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala index 10bf469dadda6..97f97b5b34148 100644 --- a/src/test/scala/catalyst/execution/HiveQueryFileTest.scala +++ b/src/test/scala/catalyst/execution/HiveQueryFileTest.scala @@ -34,7 +34,7 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { Option(System.getProperty(whiteListProperty)).map(_.split(",").toSeq).getOrElse(whiteList) // Go through all the test cases and add them to scala test. - testCases.foreach { + testCases.sorted.foreach { case (testCaseName, testCaseFile) => if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { logger.warn(s"Blacklisted test skipped $testCaseName") From 784536466cc3fe69ea230f0e63f7c4cd670fdadc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 21:13:40 -0800 Subject: [PATCH 593/778] deactivate concurrent test. --- .../catalyst/execution/ConcurrentHiveTests.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/src/test/scala/catalyst/execution/ConcurrentHiveTests.scala b/src/test/scala/catalyst/execution/ConcurrentHiveTests.scala index 002f992a21fff..7f45bfe4da528 100644 --- a/src/test/scala/catalyst/execution/ConcurrentHiveTests.scala +++ b/src/test/scala/catalyst/execution/ConcurrentHiveTests.scala @@ -4,13 +4,15 @@ package execution import org.scalatest.{FunSuite, BeforeAndAfterAll} class ConcurrentHiveTests extends FunSuite with BeforeAndAfterAll { - test("Multiple Hive Instances") { - (1 to 10).map { i => - val ts = new TestSharkInstance - ts.runSqlHive("SHOW TABLES") - val q = ts.stringToTestQuery("SELECT * FROM src").q - q.toRdd.collect() - ts.runSqlHive("SHOW TABLES") + ignore("multiple instances not supported") { + test("Multiple Hive Instances") { + (1 to 10).map { i => + val ts = new TestSharkInstance + ts.runSqlHive("SHOW TABLES") + val q = ts.stringToTestQuery("SELECT * FROM src").q + q.toRdd.collect() + ts.runSqlHive("SHOW TABLES") + } } } } \ No newline at end of file From ea6f37f740a5dfef3ca0c2f82e4c26ed3171851c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 21:13:53 -0800 Subject: [PATCH 594/778] fix style. --- .../scala/catalyst/execution/Exchange.scala | 6 +++--- .../catalyst/execution/FunctionRegistry.scala | 17 +++++++++++------ .../scala/catalyst/execution/SharkPlan.scala | 3 ++- .../scala/catalyst/execution/TestShark.scala | 10 +++++++--- .../scala/catalyst/execution/aggregates.scala | 3 ++- .../scala/catalyst/expressions/predicates.scala | 3 ++- .../catalyst/plans/logical/partitioning.scala | 1 + 7 files changed, 28 insertions(+), 15 deletions(-) diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 076d4a8e69a9f..bed10d2d63fcb 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -69,7 +69,7 @@ object AddExchange extends Rule[SharkPlan] { s"${if (valid) "Valid" else "Invalid"} distribution," + s"required: $required current: ${child.outputPartitioning}") valid - }.exists(_ == false) + }.exists(!_) // Check if outputPartitionings of children are compatible with each other. // It is possible that every child satisfies its required data distribution @@ -86,7 +86,7 @@ object AddExchange extends Rule[SharkPlan] { .map { case Seq(a) => true case Seq(a,b) => a compatibleWith b - }.exists(_ == false) + }.exists(!_) // Check if the partitioning we want to ensure is the same as the child's output // partitioning. If so, we do not need to add the Exchange operator. @@ -112,4 +112,4 @@ object AddExchange extends Rule[SharkPlan] { operator.withNewChildren(repartitionedChildren) } } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index 9ab8e8270e8b6..8d1b14928313e 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -4,7 +4,8 @@ package execution import scala.collection.JavaConversions._ import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} -import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFEvaluator, AbstractGenericUDAFResolver, GenericUDF} +import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFEvaluator, AbstractGenericUDAFResolver} +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.hive.serde2.objectinspector.primitive._ @@ -12,7 +13,8 @@ import org.apache.hadoop.{io => hadoopIo} import expressions._ import types._ -import org.apache.hadoop.hive.serde2.objectinspector.{ListObjectInspector, StructObjectInspector, ObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.{ListObjectInspector, StructObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import catalyst.types.StructField import catalyst.types.StructType import catalyst.types.ArrayType @@ -37,7 +39,8 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionF ) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdf(name, IntegerType, children) - } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { + } else if ( + classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdaf(name, children) } else { sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") @@ -75,7 +78,8 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionF trait HiveFunctionFactory { def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) def getFunctionClass(name: String) = getFunctionInfo(name).getFunctionClass - def createFunction[UDFType](name: String) = getFunctionClass(name).newInstance.asInstanceOf[UDFType] + def createFunction[UDFType](name: String) = + getFunctionClass(name).newInstance.asInstanceOf[UDFType] def unwrap(a: Any): Any = a match { case null => null @@ -100,7 +104,8 @@ trait HiveFunctionFactory { } } -abstract class HiveUdf extends Expression with ImplementedUdf with Logging with HiveFunctionFactory { +abstract class HiveUdf + extends Expression with ImplementedUdf with Logging with HiveFunctionFactory { self: Product => type UDFType @@ -256,4 +261,4 @@ case class HiveGenericUdaf( def nullable: Boolean = true def references: Set[Attribute] = children.map(_.references).flatten.toSet -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index c22ec2831fe1b..1b72c13fd7084 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -13,7 +13,8 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { /** Specifies how data is partitioned across different nodes in the cluster. */ def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG WIDTH! /** Specifies any partition requirements on the input data for this operator. */ - def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnspecifiedDistribution) + def requiredChildDistribution: Seq[Distribution] = + Seq.fill(children.size)(UnspecifiedDistribution) /** * Runs this query returning the result as an RDD. diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 3f9291f79a089..9de1beb765387 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -91,9 +91,13 @@ class TestSharkInstance extends SharkInstance { if (cmd.toUpperCase.contains("LOAD DATA") && cmd.contains("..")) { "[\"\'](../.*)[\"\'] ".r.findFirstMatchIn(cmd) .map(r => { - val newPath = new File(TestShark.inRepoTests.getCanonicalPath, cmd.substring(r.start + 1, r.end - 2).replaceFirst("(\\.\\./)+", "")).getAbsolutePath - cmd.substring(0, r.start + 1) + newPath + cmd.substring(r.end - 2) - }).getOrElse(cmd) + val newPath = + new File( + TestShark.inRepoTests.getCanonicalPath, + cmd.substring(r.start + 1, r.end - 2).replaceFirst("(\\.\\./)+", "")) + + cmd.substring(0, r.start + 1) + newPath.getAbsolutePath + cmd.substring(r.end - 2) + }).getOrElse(cmd) } else { cmd } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index ea8b7a1aeab63..a4152d24adb23 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -44,7 +44,8 @@ case class Aggregate( } // Cast required to avoid type inference selecting a deprecated Hive API. - val buffer = function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] + val buffer = + function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] def result: Any = unwrap(function.evaluate(buffer)) diff --git a/src/main/scala/catalyst/expressions/predicates.scala b/src/main/scala/catalyst/expressions/predicates.scala index b8d578dc1c546..5f6d062adf202 100644 --- a/src/main/scala/catalyst/expressions/predicates.scala +++ b/src/main/scala/catalyst/expressions/predicates.scala @@ -93,7 +93,8 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi def dataType = { if (!resolved) { throw new UnresolvedException( - this, s"Can not resolve due to differing types ${trueValue.dataType}, ${falseValue.dataType}") + this, + s"Can not resolve due to differing types ${trueValue.dataType}, ${falseValue.dataType}") } trueValue.dataType } diff --git a/src/main/scala/catalyst/plans/logical/partitioning.scala b/src/main/scala/catalyst/plans/logical/partitioning.scala index e655f1fcb115b..87e625d576483 100644 --- a/src/main/scala/catalyst/plans/logical/partitioning.scala +++ b/src/main/scala/catalyst/plans/logical/partitioning.scala @@ -25,3 +25,4 @@ case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan def references = partitionExpressions.flatMap(_.references).toSet } + From 82163e3e3c21804898e576e3a224e3a644e75d27 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 22:26:58 -0800 Subject: [PATCH 595/778] special case handling of partitionKeys when casting insert into tables --- src/main/scala/catalyst/analysis/Analyzer.scala | 7 +++++-- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 7 +++++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 892a5db0102b3..269f50ed0216b 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -4,6 +4,7 @@ package analysis import expressions._ import plans.logical._ import rules._ +import catalyst.execution.MetastoreRelation /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing @@ -152,9 +153,11 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool // Wait until children are resolved case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table, _, child) if !p.resolved => + case p @ InsertIntoTable(table: MetastoreRelation, _, child) => val childOutputDataTypes = child.output.map(_.dataType) - val tableOutputDataTypes = table.output.map(_.dataType) + // Only check attributes, not partitionKeys since they are always strings. + // TODO: Fully support inserting into partitioned tables. + val tableOutputDataTypes = table.attributes.map(_.dataType) if (childOutputDataTypes == tableOutputDataTypes) { p diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index f5f7325885e71..a69ff45f735fa 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -126,8 +126,11 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt )(qualifiers = tableName +: alias.toSeq) } + // Must be a stable value since new attributes are born here. val partitionKeys = hiveQlTable.getPartitionKeys.map(_.toAttribute) - // Must be a stable value since new attributes are born here. - val output = table.getSd.getCols.map(_.toAttribute) ++ partitionKeys + /** Non-partitionKey attributes */ + val attributes = table.getSd.getCols.map(_.toAttribute) + + val output = attributes ++ partitionKeys } From 9c22b4ebdda3955a88800dcf0dec0d14748394e7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 18:13:44 -0800 Subject: [PATCH 596/778] Support for parsing nested types. --- .../catalyst/execution/MetastoreCatalog.scala | 57 ++++++++++++------- 1 file changed, 38 insertions(+), 19 deletions(-) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index f5f7325885e71..e36064ec5bddb 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -17,6 +17,7 @@ import rules._ import types._ import collection.JavaConversions._ +import scala.util.parsing.combinator.RegexParsers class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { val client = new HiveMetaStoreClient(hiveConf) @@ -78,25 +79,43 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { } } -object HiveMetatoreTypes { - val VARCHAR = "(?i)VARCHAR\\((\\d+)\\)".r - // TODO: this will not work for nested arrays or maps. - val ARRAY = "(?i)array<([^>]+)>".r - val MAP = "(?i)map<([^,]+),([^>]*)>".r - def toDataType(metastoreType: String): DataType = - metastoreType match { - case "string" => StringType - case "float" => FloatType - case "int" => IntegerType - case "double" => DoubleType - case "bigint" => LongType - case "binary" => BinaryType - case "boolean" => BooleanType - case VARCHAR(_) => StringType - case ARRAY(elemType) => ArrayType(toDataType(elemType)) - case MAP(keyType, valueType) => MapType(toDataType(keyType), toDataType(valueType)) - case _ => sys.error(s"Unsupported dataType: $metastoreType") +object HiveMetastoreTypes extends RegexParsers { + protected lazy val primitiveType: Parser[DataType] = + "string" ^^^ StringType | + "float" ^^^ FloatType | + "int" ^^^ IntegerType | + "double" ^^^ DoubleType | + "bigint" ^^^ LongType | + "binary" ^^^ BinaryType | + "boolean" ^^^ BooleanType | + "(?i)VARCHAR\\((\\d+)\\)".r ^^^ StringType + + protected lazy val arrayType: Parser[DataType] = + "array" ~> "<" ~> dataType <~ ">" ^^ ArrayType + + protected lazy val mapType: Parser[DataType] = + "map" ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ { + case t1 ~ _ ~ t2 => MapType(t1, t2) } + + protected lazy val structField: Parser[StructField] = + "[a-zA-Z]".r ~ ":" ~ dataType ^^ { + case name ~ _ ~ tpe => StructField(name, tpe, true) + } + + protected lazy val structType: Parser[DataType] = + "struct" ~> "<" ~> repsep(structField,",") <~ ">" ^^ StructType + + protected lazy val dataType: Parser[DataType] = + arrayType | + mapType | + structType | + primitiveType + + def toDataType(metastoreType: String): DataType = parseAll(dataType, metastoreType) match { + case Success(result, _) => result + case failure: NoSuccess => sys.error(s"Unsupported dataType: $metastoreType") + } } case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String]) @@ -120,7 +139,7 @@ case class MetastoreRelation(databaseName: String, tableName: String, alias: Opt implicit class SchemaAttribute(f: FieldSchema) { def toAttribute = AttributeReference( f.getName, - HiveMetatoreTypes.toDataType(f.getType), + HiveMetastoreTypes.toDataType(f.getType), // Since data can be dumped in randomly with no validation, everything is nullable. nullable = true )(qualifiers = tableName +: alias.toSeq) From efa72170ebe27d84cb5ae2efeaed4054ceca1f9c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 18:19:31 -0800 Subject: [PATCH 597/778] Support for reading structs in HiveTableScan. --- src/main/scala/catalyst/execution/hiveOperators.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 759b0c016bd76..6264484de22bc 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -82,13 +82,19 @@ case class HiveTableScan( .getOrElse(sys.error(s"Can't find attribute $a")) (row: Any, _: Array[String]) => { val data = objectInspector.getStructFieldData(row, ref) - val inspector = ref.getFieldObjectInspector.asInstanceOf[PrimitiveObjectInspector] - inspector.getPrimitiveJavaObject(data) + unwrapData(data, ref) } } } } + def unwrapData(data: Any, ref: StructField): Any = ref.getFieldObjectInspector match { + case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) + case si: StructObjectInspector => + val allRefs = si.getAllStructFieldRefs + new GenericRow(allRefs.map(r => unwrapData(si.getStructFieldData(data,r), r))) + } + private def castFromString(value: String, dataType: DataType) = { Evaluate(Cast(Literal(value), dataType), Nil) } From d670e41dfaf93bc322079d5e93b938c2f868932c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 18:19:47 -0800 Subject: [PATCH 598/778] Print nested fields like hive does. --- .../catalyst/execution/SharkInstance.scala | 32 +++++++++++++++---- 1 file changed, 26 insertions(+), 6 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 626ac11781376..ec92b09594883 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -10,6 +10,7 @@ import optimizer.Optimize import planning.QueryPlanner import plans.logical.LogicalPlan import rules.RuleExecutor +import types._ /** * Starts up an instance of shark where metadata is stored locally. An in-process metadata data is @@ -108,11 +109,28 @@ abstract class SharkInstance extends Logging { lazy val toRdd = executedPlan.execute() - def toHiveString(a: Any): String = a match { - case seq: Seq[_] => seq.map(toHiveString).map(s => "\"" + s + "\"").mkString("[", ",", "]") - case "null" => "NULL" - case null => "NULL" - case other => other.toString + protected def toHiveString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], typ)=> + seq.map(v => (v, typ)).map(toHiveString).map(s => "\"" + s + "\"").mkString("[", ",", "]") + case (null, _) => "NULL" + case (other, _) => other.toString + } + + /** Hive outputs fields of structs slightly differently than top level attributes. */ + protected def toHiveStructString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], typ)=> + seq.map(v => (v, typ)).map(toHiveString).map(s => "\"" + s + "\"").mkString("[", ",", "]") + case (null, _) => "null" + case (s: String, _) => "\"" + s + "\"" + case (other, _) => other.toString } /** @@ -125,8 +143,10 @@ abstract class SharkInstance extends Logging { case ExplainCommand(plan) => new SharkQuery { val parsed = plan }.toString.split("\n") case query => val result: Seq[Seq[Any]] = toRdd.collect().toSeq + // We need the types so we can output struct field names + val types = analyzed.output.map(_.dataType) // Reformat to match hive tab delimited output. - val asString = result.map(_.map(toHiveString)).map(_.mkString("\t")).toSeq + val asString = result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq asString } From dc6463acaccfbdf3bae41ca746b678cb3b70cf9a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 18:20:11 -0800 Subject: [PATCH 599/778] Support for resolving access to nested fields using "." notation. --- .../catalyst/plans/logical/LogicalPlan.scala | 25 ++++++++++++++----- 1 file changed, 19 insertions(+), 6 deletions(-) diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala index b6e6d637e95e9..245fc2a2277bc 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala @@ -2,8 +2,9 @@ package catalyst package plans package logical -import expressions.Attribute -import errors._ +import catalyst.expressions._ +import catalyst.errors._ +import catalyst.types.StructType abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => @@ -33,16 +34,28 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { */ def childrenResolved = !children.exists(!_.resolved) - def resolve(name: String): Option[Attribute] = { + /** + * Optionally resolves the given string to a + * [[catalyst.expressions.NamedExpression NamedExpression]]. The attribute is expressed as + * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. + */ + def resolve(name: String): Option[NamedExpression] = { val parts = name.split("\\.") - val options = children.flatMap(_.output).filter { option => + val options = children.flatMap(_.output).flatMap { option => // If the first part of the desired name matches a qualifier for this possible match, drop it. val remainingParts = if (option.qualifiers contains parts.head) parts.drop(1) else parts - option.name == remainingParts.head + if(option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil } options.distinct match { - case a :: Nil => Some(a) // One match, use it. + case (a, Nil) :: Nil => Some(a) // One match, no nested fields, use it. + // One match, but we also need to extract the requested nested field. + case (a, nestedFields) :: Nil => + a.dataType match { + case StructType(fields) => + Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) + case _ => None // Don't know how to resolve these field references + } case Nil => None // No matches. case ambiguousReferences => throw new TreeNodeException( From 67094413d86c0d03fbb717a99916b9c906552d67 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 18:20:26 -0800 Subject: [PATCH 600/778] Evaluation for accessing nested fields. --- src/main/scala/catalyst/expressions/Evaluate.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 12ff84a3a28d7..d7813c6f9f4fd 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -236,6 +236,17 @@ object Evaluate extends Logging { /* Functions */ case Rand => scala.util.Random.nextDouble() + /* Complex Type Access */ + case GetField(e, name) => + val baseValue = eval(e).asInstanceOf[Row] + if(baseValue == null) { + null + } else { + // TODO: We could bind this to avoid having to look it up each time. + val ordinal = e.dataType.asInstanceOf[StructType].fields.indexWhere(_.name == name) + baseValue(ordinal) + } + /* UDFs */ case implementedFunction: ImplementedUdf => implementedFunction.evaluate(implementedFunction.children.map(eval)) From da7ae9da830a5260478a5d9cd4959bb5f3565df2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 18:21:11 -0800 Subject: [PATCH 601/778] Add boolean writable that was breaking udf_regexp test. Not sure how this was passing before... --- src/main/scala/catalyst/execution/FunctionRegistry.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index 8d1b14928313e..ab87829c78803 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -57,6 +57,7 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionF case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType case c: Class[_] if c == classOf[hadoopIo.FloatWritable] => FloatType + case c: Class[_] if c == classOf[hadoopIo.BooleanWritable] => BooleanType case c: Class[_] if c == classOf[java.lang.String] => StringType case c: Class[_] if c == java.lang.Short.TYPE => ShortType case c: Class[_] if c == java.lang.Integer.TYPE => ShortType From 6420c7c23b1fcbae009ce97c5dd2dc9ece75f0a0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 31 Jan 2014 18:28:56 -0800 Subject: [PATCH 602/778] Memoize the ordinal in the GetField expression. --- .../scala/catalyst/expressions/Evaluate.scala | 6 ++---- .../scala/catalyst/expressions/complexTypes.scala | 15 +++++++++++---- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index d7813c6f9f4fd..a866a68b471d1 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -237,14 +237,12 @@ object Evaluate extends Logging { case Rand => scala.util.Random.nextDouble() /* Complex Type Access */ - case GetField(e, name) => + case g @ GetField(e, name) => val baseValue = eval(e).asInstanceOf[Row] if(baseValue == null) { null } else { - // TODO: We could bind this to avoid having to look it up each time. - val ordinal = e.dataType.asInstanceOf[StructType].fields.indexWhere(_.name == name) - baseValue(ordinal) + baseValue(g.ordinal) } /* UDFs */ diff --git a/src/main/scala/catalyst/expressions/complexTypes.scala b/src/main/scala/catalyst/expressions/complexTypes.scala index 05ae9327e9d1e..932133b6cc12a 100644 --- a/src/main/scala/catalyst/expressions/complexTypes.scala +++ b/src/main/scala/catalyst/expressions/complexTypes.scala @@ -24,12 +24,19 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { case class GetField(child: Expression, fieldName: String) extends UnaryExpression { def dataType = field.dataType def nullable = field.nullable - lazy val field = child.dataType match { - case s: StructType => - s.fields + + protected def structType = child.dataType match { + case s: StructType => s + case otherType => sys.error(s"GetField is not valid on fields of type $otherType") + } + + lazy val field = + structType.fields .find(_.name == fieldName) .getOrElse(sys.error(s"No such field $fieldName in ${child.dataType}")) - } + + lazy val ordinal = structType.fields.indexOf(field) + override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType] override def toString = s"$child.$fieldName" } From 1579eecca917152c542a68149eddd636131dbb2f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 01:39:11 -0800 Subject: [PATCH 603/778] Only cast unresolved inserts. --- src/main/scala/catalyst/analysis/Analyzer.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 0ba054d7b3041..892a5db0102b3 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -152,17 +152,18 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool // Wait until children are resolved case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table, _, child) => + case p @ InsertIntoTable(table, _, child) if !p.resolved => val childOutputDataTypes = child.output.map(_.dataType) val tableOutputDataTypes = table.output.map(_.dataType) - if (childOutputDataTypes sameElements tableOutputDataTypes) { + if (childOutputDataTypes == tableOutputDataTypes) { p } else { // Only do the casting when child output data types differ from table output data types. val castedChildOutput = child.output.zip(table.output).map { - case (l, r) if l.dataType != r.dataType => Alias(Cast(l, r.dataType), l.name)() - case (l, _) => l + case (input, table) if input.dataType != table.dataType => + Alias(Cast(input, table.dataType), input.name)() + case (input, _) => input } p.copy(child = Project(castedChildOutput, child)) From cf8d99257ad87063bca4bc3a2d5a09b54a2cf2b1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 12:00:51 -0800 Subject: [PATCH 604/778] Use built in functions for creating temp directory. --- .../catalyst/execution/hiveOperators.scala | 26 +++---------------- 1 file changed, 4 insertions(+), 22 deletions(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 6264484de22bc..82cb2b9cee423 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -190,29 +190,11 @@ case class InsertIntoHiveTable( val childRdd = child.execute() assert(childRdd != null) - /** Create a temporary directory inside the given parent directory */ - def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { - var attempts = 0 - val maxAttempts = 10 - var dir: File = null - while (dir == null) { - attempts += 1 - if (attempts > maxAttempts) { - throw new IOException("Failed to create a temp directory (under " + root + ") after " + - maxAttempts + " attempts!") - } - try { - dir = new File(root, "spark-" + UUID.randomUUID.toString) - if (dir.exists() || !dir.mkdirs()) { - dir = null - } - } catch { case e: IOException => ; } - } - dir - } - // TODO write directly to Hive - val tempDir = createTempDir() + val tempDir = File.createTempFile("catalysthiveout", "") + tempDir.delete() + tempDir.mkdir() + // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer // instances within the closure, since AbstractSerDe is not serializable while TableDesc is. From c654f19ef6fec54537a4e704234b63c65c7e0d1e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 12:01:51 -0800 Subject: [PATCH 605/778] Support for list and maps in hive table scan. --- .../catalyst/execution/hiveOperators.scala | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 82cb2b9cee423..1716e739bca30 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -82,17 +82,28 @@ case class HiveTableScan( .getOrElse(sys.error(s"Can't find attribute $a")) (row: Any, _: Array[String]) => { val data = objectInspector.getStructFieldData(row, ref) - unwrapData(data, ref) + unwrapData(data, ref.getFieldObjectInspector) } } } } - def unwrapData(data: Any, ref: StructField): Any = ref.getFieldObjectInspector match { + def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) + case li: ListObjectInspector => + Option( + li.getList(data)).map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq).orNull + case mi: MapObjectInspector => + Option(mi.getMap(data)).map( + _.map { + case (k,v) => + (unwrapData(k, mi.getMapKeyObjectInspector), + unwrapData(v, mi.getMapValueObjectInspector)) + }.toMap).orNull case si: StructObjectInspector => val allRefs = si.getAllStructFieldRefs - new GenericRow(allRefs.map(r => unwrapData(si.getStructFieldData(data,r), r))) + new GenericRow( + allRefs.map(r => unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector))) } private def castFromString(value: String, dataType: DataType) = { From c3feda75938565b85ff401aeb29bdcb44e7accdc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 12:02:06 -0800 Subject: [PATCH 606/778] use toArray. --- src/main/scala/catalyst/execution/hiveOperators.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 1716e739bca30..5dd4f1f1426e0 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -224,7 +224,7 @@ case class InsertIntoHiveTable( case (obj, _) => obj } - (null, serializer.serialize(Array(mappedRow: _*), standardOI)) + (null, serializer.serialize(mappedRow.toArray, standardOI)) } }.saveAsHadoopFile( tempDir.getCanonicalPath, From a9388fb7274fe40b9d10eb8d4a3c97c32d365187 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 12:02:29 -0800 Subject: [PATCH 607/778] printing for map types. --- src/main/scala/catalyst/execution/SharkInstance.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index ec92b09594883..c98fb5d63448f 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -116,6 +116,11 @@ abstract class SharkInstance extends Logging { }.mkString("{", ",", "}") case (seq: Seq[_], typ)=> seq.map(v => (v, typ)).map(toHiveString).map(s => "\"" + s + "\"").mkString("[", ",", "]") + case (map: Map[_,_], MapType(kType, vType)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.mkString("{", ",", "}") case (null, _) => "NULL" case (other, _) => other.toString } @@ -128,6 +133,11 @@ abstract class SharkInstance extends Logging { }.mkString("{", ",", "}") case (seq: Seq[_], typ)=> seq.map(v => (v, typ)).map(toHiveString).map(s => "\"" + s + "\"").mkString("[", ",", "]") + case (map: Map[_,_], MapType(kType, vType)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.mkString("{", ",", "}") case (null, _) => "null" case (s: String, _) => "\"" + s + "\"" case (other, _) => other.toString From 35a70fbfd93b83856f86ea52bc1b3a850076960f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 13:28:05 -0800 Subject: [PATCH 608/778] multi-letter field names. --- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index e36064ec5bddb..23538f6fbe25f 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -99,8 +99,8 @@ object HiveMetastoreTypes extends RegexParsers { } protected lazy val structField: Parser[StructField] = - "[a-zA-Z]".r ~ ":" ~ dataType ^^ { - case name ~ _ ~ tpe => StructField(name, tpe, true) + "[a-zA-Z0-9]*".r ~ ":" ~ dataType ^^ { + case name ~ _ ~ tpe => StructField(name, tpe, nullable = true) } protected lazy val structType: Parser[DataType] = From 2c6deb37b104b5272d99917b6933a749da99d06e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 13:28:23 -0800 Subject: [PATCH 609/778] improve printing compatibility. --- .../catalyst/execution/SharkInstance.scala | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index c98fb5d63448f..6940afd462837 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -109,38 +109,42 @@ abstract class SharkInstance extends Logging { lazy val toRdd = executedPlan.execute() + protected val primitiveTypes = + Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, + ShortType) + protected def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" + case (v, t) => s"${t.name}:${toHiveStructString(v, t.dataType)}" }.mkString("{", ",", "}") - case (seq: Seq[_], typ)=> - seq.map(v => (v, typ)).map(toHiveString).map(s => "\"" + s + "\"").mkString("[", ",", "]") + case (seq: Seq[_], ArrayType(typ))=> + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") case (map: Map[_,_], MapType(kType, vType)) => map.map { case (key, value) => toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.mkString("{", ",", "}") + }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" - case (other, _) => other.toString + case (other, tpe) if primitiveTypes contains tpe => other.toString } /** Hive outputs fields of structs slightly differently than top level attributes. */ protected def toHiveStructString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" + case (v, t) => s"${t.name}:${toHiveStructString(v, t.dataType)}" }.mkString("{", ",", "}") - case (seq: Seq[_], typ)=> - seq.map(v => (v, typ)).map(toHiveString).map(s => "\"" + s + "\"").mkString("[", ",", "]") + case (seq: Seq[_], ArrayType(typ))=> + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") case (map: Map[_,_], MapType(kType, vType)) => map.map { case (key, value) => toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.mkString("{", ",", "}") + }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "null" case (s: String, _) => "\"" + s + "\"" - case (other, _) => other.toString + case (other, tpe) if primitiveTypes contains tpe => other.toString } /** From 5b33216d197ad7c649e36f9f9a2a48143120aeae Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 16:21:23 -0800 Subject: [PATCH 610/778] work on decimal support. --- src/main/scala/catalyst/analysis/HiveTypeCoercion.scala | 2 +- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 1 + src/main/scala/catalyst/execution/SharkInstance.scala | 2 +- src/main/scala/catalyst/execution/hiveOperators.scala | 8 ++++++-- src/main/scala/catalyst/frontend/Hive.scala | 2 ++ 5 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala index 9544eec10dee4..faf7a84b969f3 100644 --- a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala +++ b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala @@ -94,7 +94,7 @@ trait HiveTypeCoercion { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. // The conversion for integral and floating point types have a linear widening hierarchy: val numericPrecedence = - Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) + Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) // Boolean is only wider than Void val booleanPrecedence = Seq(NullType, BooleanType) val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: booleanPrecedence :: Nil diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 23538f6fbe25f..e3af55bb25ffa 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -88,6 +88,7 @@ object HiveMetastoreTypes extends RegexParsers { "bigint" ^^^ LongType | "binary" ^^^ BinaryType | "boolean" ^^^ BooleanType | + "decimal" ^^^ DecimalType | "(?i)VARCHAR\\((\\d+)\\)".r ^^^ StringType protected lazy val arrayType: Parser[DataType] = diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 6940afd462837..1a40dce1be13a 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -111,7 +111,7 @@ abstract class SharkInstance extends Logging { protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType) + ShortType, DecimalType) protected def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 5dd4f1f1426e0..dcab7184bc02f 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -5,14 +5,14 @@ import java.io.{File, IOException} import java.util.UUID import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.common.`type`.HiveVarchar +import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{JavaHiveDecimalObjectInspector, JavaHiveVarcharObjectInspector} import org.apache.hadoop.io.NullWritable import org.apache.hadoop.mapred.JobConf import org.apache.spark.SparkContext._ @@ -151,6 +151,8 @@ case class HiveTableScan( buildRow(values.map { case n: String if n.toLowerCase == "null" => null case varchar: org.apache.hadoop.hive.common.`type`.HiveVarchar => varchar.getValue + case decimal: org.apache.hadoop.hive.common.`type`.HiveDecimal => + BigDecimal(decimal.bigDecimalValue) case other => other }) } @@ -221,6 +223,8 @@ case class InsertIntoHiveTable( val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector) val mappedRow = row.zip(fieldOIs).map { case (s: String, oi: JavaHiveVarcharObjectInspector) => new HiveVarchar(s, s.size) + case (bd: BigDecimal, oi: JavaHiveDecimalObjectInspector) => + new HiveDecimal(bd.underlying()) case (obj, _) => obj } diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 595d1cd9250c6..2282dd7612a4d 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -723,6 +723,8 @@ object HiveQl { Cast(nodeToExpr(arg), BinaryType) case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BooleanType) + case Token("TOK_FUNCTION", Token("TOK_DECIMAL", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DecimalType) /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) From 5b3d2c80546848a9c6bf830c22ec5f029dca790f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 16:21:40 -0800 Subject: [PATCH 611/778] implement distinct. --- src/main/scala/catalyst/execution/PlanningStrategies.scala | 4 +++- src/main/scala/catalyst/execution/aggregates.scala | 1 - src/test/scala/catalyst/execution/HiveQueryTests.scala | 3 +++ 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index 6add5115d5c2c..c47e2b4655417 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -142,9 +142,11 @@ trait PlanningStrategies { // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { + case logical.Distinct(child) => + execution.Aggregate(child.output, child.output, planLater(child))(sc) :: Nil case logical.Sort(sortExprs, child) => // This sort is a global sort. Its requiredDistribution will be an OrderedDistribution. - execution.Sort(sortExprs, true, planLater(child)):: Nil + execution.Sort(sortExprs, global = true, planLater(child)):: Nil case logical.SortPartitions(sortExprs, child) => // This sort only sort tuples within a partition. Its requiredDistribution will be // an UnspecifiedDistribution. diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index a4152d24adb23..5f7a22e43fe9d 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -101,7 +101,6 @@ case class Aggregate( // Pull out all the functions so we can feed each row into them. val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f }) - assert(aggFunctions.nonEmpty) rows.foreach { row => val input = Vector(row) diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala index 920b875c149d4..6a4469fdf4272 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -57,6 +57,9 @@ class HiveQueryTests extends HiveComparisonTest { createQueryTest("LIKE", "SELECT * FROM src WHERE value LIKE '%1%'") + createQueryTest("DISTINCT", + "SELECT DISTINCT key, value FROM src") + ignore("empty aggregate input") { createQueryTest("empty aggregate input", "SELECT SUM(key) FROM (SELECT * FROM src LIMIT 0) a") From 3f9e519a16f9dc9f3eabda3ad91d80c088e3f384 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 16:30:14 -0800 Subject: [PATCH 612/778] use names w/ boolean args --- src/main/scala/catalyst/execution/PlanningStrategies.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index c47e2b4655417..22ee2188de19c 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -150,7 +150,7 @@ trait PlanningStrategies { case logical.SortPartitions(sortExprs, child) => // This sort only sort tuples within a partition. Its requiredDistribution will be // an UnspecifiedDistribution. - execution.Sort(sortExprs, false, planLater(child)) :: Nil + execution.Sort(sortExprs, global = false, planLater(child)) :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => From 3734a9416c1156030a7c2af9e43d9209ca17aa59 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 16:31:03 -0800 Subject: [PATCH 613/778] only quote string types. --- src/main/scala/catalyst/execution/SharkInstance.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 1a40dce1be13a..26565c5fcb655 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -143,7 +143,7 @@ abstract class SharkInstance extends Logging { toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "null" - case (s: String, _) => "\"" + s + "\"" + case (s: String, StringType) => "\"" + s + "\"" case (other, tpe) if primitiveTypes contains tpe => other.toString } From bbec500c4fc9a12cbc18b607147aa751308f4288 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 12:02:52 -0800 Subject: [PATCH 614/778] update test coverage, new golden --- build.sbt | 2 +- .../catalyst/execution/HiveCompatibility.scala | 18 ++++++++++++++---- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/build.sbt b/build.sbt index a52936e8f1889..58b00e1438bdc 100644 --- a/build.sbt +++ b/build.sbt @@ -17,7 +17,7 @@ resolvers += "SparkStaging" at "https://repository.apache.org/content/repositori libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" -libraryDependencies += "catalyst" % "hive-golden" % "5" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden5.jar" +libraryDependencies += "catalyst" % "hive-golden" % "6" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden6.jar" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index ce92e788db825..4933e273b7035 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -156,10 +156,7 @@ class HiveCompatibility extends HiveQueryFileTest { "alter_partition_format_loc", "alter_partition_protect_mode", "alter_partition_with_whitelist", - "alter_skewed_table", - "alter_table_not_sorted", "alter_table_serde", - "alter_table_serde2", "alter_varchar2", "ambiguous_col", "auto_join0", @@ -222,13 +219,22 @@ class HiveCompatibility extends HiveQueryFileTest { "correlationoptimizer7", "correlationoptimizer8", "count", + "create_nested_type", + "create_skewed_table1", + "create_struct_table", "ct_case_insensitive", + "database_location", "database_properties", + "decimal_join", "default_partition_name", "delimiter", "desc_non_existent_tbl", + "describe_comment_indent", + "describe_comment_nonascii", "describe_database_json", "describe_pretty", + "describe_syntax", + "describe_table", "describe_table_json", "diff_part_input_formats", "disable_file_format_check", @@ -358,6 +364,7 @@ class HiveCompatibility extends HiveQueryFileTest { "join8", "join9", "join_1to1", + "join_array", "join_casesensitive", "join_empty", "join_filters", @@ -407,6 +414,7 @@ class HiveCompatibility extends HiveQueryFileTest { "outer_join_ppr", "part_inherit_tbl_props", "part_inherit_tbl_props_empty", + "part_inherit_tbl_props_with_star", "partition_schema1", "plan_json", "ppd1", @@ -425,13 +433,14 @@ class HiveCompatibility extends HiveQueryFileTest { "ppd_repeated_alias", "ppd_udf_col", "ppd_union", + "ppr_allchildsarenull", "ppr_pushdown", "ppr_pushdown2", "ppr_pushdown3", "progress_1", "protectmode", + "protectmode2", "push_or", - "ql_rewrite_gbtoidx", "query_with_semi", "quote1", "quote2", @@ -631,6 +640,7 @@ class HiveCompatibility extends HiveQueryFileTest { "union8", "union9", "union_ppr", + "union_remove_3", "union_remove_6", "union_script", "varchar_2", From 5e54aa6dab3e3ed0f2e702abc038eee5f17fcb38 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 17:38:52 -0800 Subject: [PATCH 615/778] quotes for struct field names. --- src/main/scala/catalyst/execution/SharkInstance.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 26565c5fcb655..c81ace5387d2b 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -116,7 +116,7 @@ abstract class SharkInstance extends Logging { protected def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { - case (v, t) => s"${t.name}:${toHiveStructString(v, t.dataType)}" + case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" }.mkString("{", ",", "}") case (seq: Seq[_], ArrayType(typ))=> seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") @@ -133,7 +133,7 @@ abstract class SharkInstance extends Logging { protected def toHiveStructString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { - case (v, t) => s"${t.name}:${toHiveStructString(v, t.dataType)}" + case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" }.mkString("{", ",", "}") case (seq: Seq[_], ArrayType(typ))=> seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") From e4def6b2c917ebf28b3a11fc1aad690c2fddd55f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 17:39:19 -0800 Subject: [PATCH 616/778] set dataType for HiveGenericUdfs. --- .../catalyst/execution/FunctionRegistry.scala | 33 ++++++++++--------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index ab87829c78803..e189569dec8b4 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -3,22 +3,18 @@ package execution import scala.collection.JavaConversions._ +import org.apache.hadoop.hive.serde2.{io => hiveIo} +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ +import org.apache.hadoop.hive.serde2.objectinspector.{ListObjectInspector, StructObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.{MapObjectInspector, ObjectInspector} import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFEvaluator, AbstractGenericUDAFResolver} import org.apache.hadoop.hive.ql.udf.generic.GenericUDF import org.apache.hadoop.hive.ql.exec.UDF -import org.apache.hadoop.hive.serde2.{io => hiveIo} -import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.{io => hadoopIo} -import expressions._ -import types._ -import org.apache.hadoop.hive.serde2.objectinspector.{ListObjectInspector, StructObjectInspector} -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector -import catalyst.types.StructField -import catalyst.types.StructType -import catalyst.types.ArrayType -import catalyst.expressions.Cast +import catalyst.expressions._ +import catalyst.types._ object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionFactory { def lookupFunction(name: String, children: Seq[Expression]): Expression = { @@ -38,7 +34,7 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionF children.zip(expectedDataTypes).map { case (e, t) => Cast(e, t) } ) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdf(name, IntegerType, children) + HiveGenericUdf(name, children) } else if ( classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdaf(name, children) @@ -172,8 +168,7 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd case class HiveGenericUdf( name: String, - dataType: DataType, - children: Seq[Expression]) extends HiveUdf { + children: Seq[Expression]) extends HiveUdf with HiveInspectors { import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ type UDFType = GenericUDF @@ -188,11 +183,13 @@ case class HiveGenericUdf( case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector } - lazy val instance = { - function.initialize(inspectors.toArray) - function + lazy val (objectInspector, instance) = { + val oi = function.initialize(inspectors.toArray) + (oi, function) } + def dataType: DataType = inspectorToDataType(objectInspector) + def wrap(a: Any): Any = a match { case s: String => new hadoopIo.Text(s) case i: Int => i: java.lang.Integer @@ -230,6 +227,10 @@ trait HiveInspectors { StructField(f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), true) })) case l: ListObjectInspector => ArrayType(inspectorToDataType(l.getListElementObjectInspector)) + case m: MapObjectInspector => + MapType( + inspectorToDataType(m.getMapKeyObjectInspector), + inspectorToDataType(m.getMapValueObjectInspector)) case _: WritableStringObjectInspector => StringType case _: WritableIntObjectInspector => IntegerType case _: WritableDoubleObjectInspector => DoubleType From aa430e7ba7fd748619bd4b1959ca165ec2b13a5c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 22:58:51 -0800 Subject: [PATCH 617/778] Update .travis.yml --- .travis.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index d4a555eac43bb..987c66e58497c 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,7 +6,7 @@ cache: directories: - $HOME/.m2 - - $HOME/.ivy + - $HOME/.ivy2 - $HOME/.sbt script: - - "GIT_AUTHOR_NAME=\"Michael Armbrust\" GIT_AUTHOR_EMAIL=\"michael@databricks.com\" GIT_COMMITTER_NAME=\"Michael Armbrust\" GIT_COMMITTER_EMAIL=\"michael@databricks.com\" sbt ++$TRAVIS_SCALA_VERSION 'set scalacOptions += \"-Xfatal-warnings\"' test:compile scalastyle test ghpages-push-site" \ No newline at end of file + - "GIT_AUTHOR_NAME=\"Michael Armbrust\" GIT_AUTHOR_EMAIL=\"michael@databricks.com\" GIT_COMMITTER_NAME=\"Michael Armbrust\" GIT_COMMITTER_EMAIL=\"michael@databricks.com\" sbt ++$TRAVIS_SCALA_VERSION 'set scalacOptions += \"-Xfatal-warnings\"' test:compile scalastyle test ghpages-push-site" From 7661b6ce6b8cb1cfc816e87d0644cfc063dce921 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 23:21:24 -0800 Subject: [PATCH 618/778] blacklist machines specific tests --- src/test/scala/catalyst/execution/HiveCompatibility.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index 4933e273b7035..dd14b1feaf9b6 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -46,6 +46,9 @@ class HiveCompatibility extends HiveQueryFileTest { "alter_partition_clusterby_sortby", "alter_merge", "alter_concatenate_indexed_table", + "protectmode2", + "describe_table", + "describe_comment_nonascii", // Weird DDL differences result in failures on jenkins. "create_like2", From 72a003dd3dce58331205465fb43bbb9a412156c4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Feb 2014 23:41:45 -0800 Subject: [PATCH 619/778] revert regex change --- src/main/scala/catalyst/execution/TestShark.scala | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 9de1beb765387..0e81c6b72c7f2 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -88,16 +88,8 @@ class TestSharkInstance extends SharkInstance { * hive test cases assume the system is set up. */ private def rewritePaths(cmd: String): String = - if (cmd.toUpperCase.contains("LOAD DATA") && cmd.contains("..")) { - "[\"\'](../.*)[\"\'] ".r.findFirstMatchIn(cmd) - .map(r => { - val newPath = - new File( - TestShark.inRepoTests.getCanonicalPath, - cmd.substring(r.start + 1, r.end - 2).replaceFirst("(\\.\\./)+", "")) - - cmd.substring(0, r.start + 1) + newPath.getAbsolutePath + cmd.substring(r.end - 2) - }).getOrElse(cmd) + if (cmd.toUpperCase contains "LOAD DATA") { + cmd.replaceAll("\\.\\.", TestShark.inRepoTests.getCanonicalPath) } else { cmd } From 9c0677866e24293525602a8e76860b4785950c39 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 00:11:21 -0800 Subject: [PATCH 620/778] fix serialization issues, add JavaStringObjectInspector. --- .../scala/catalyst/execution/FunctionRegistry.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index e189569dec8b4..f7fcb80ea46ba 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -172,6 +172,7 @@ case class HiveGenericUdf( import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ type UDFType = GenericUDF + @transient lazy val inspectors: Seq[AbstractPrimitiveJavaObjectInspector] = children.map(_.dataType).map { case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector @@ -183,12 +184,10 @@ case class HiveGenericUdf( case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector } - lazy val (objectInspector, instance) = { - val oi = function.initialize(inspectors.toArray) - (oi, function) - } + @transient + lazy val objectInspector = function.initialize(inspectors.toArray) - def dataType: DataType = inspectorToDataType(objectInspector) + val dataType: DataType = inspectorToDataType(objectInspector) def wrap(a: Any): Any = a match { case s: String => new hadoopIo.Text(s) @@ -203,10 +202,11 @@ case class HiveGenericUdf( } def evaluate(evaluatedChildren: Seq[Any]): Any = { + objectInspector // Make sure initialized. val args = evaluatedChildren.map(wrap).map { v => new DeferredJavaObject(v): DeferredObject }.toArray - unwrap(instance.evaluate(args)) + unwrap(function.evaluate(args)) } } @@ -232,6 +232,7 @@ trait HiveInspectors { inspectorToDataType(m.getMapKeyObjectInspector), inspectorToDataType(m.getMapValueObjectInspector)) case _: WritableStringObjectInspector => StringType + case _: JavaStringObjectInspector => StringType case _: WritableIntObjectInspector => IntegerType case _: WritableDoubleObjectInspector => DoubleType case _: WritableBooleanObjectInspector => BooleanType From 692a4779af0a269ae1f16006ab129c00af2a6c5c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 18:36:48 -0800 Subject: [PATCH 621/778] Support for wrapping arrays to be written into hive tables. --- .../catalyst/execution/hiveOperators.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index dcab7184bc02f..d5b8986931269 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -194,6 +194,19 @@ case class InsertIntoHiveTable( def output = child.output + /** + * Wraps with Hive types based on object inspector. + * TODO: Consolidate all hive OI/data interface code. + */ + protected def wrap(a: (Any, ObjectInspector)): Any = a match { + case (s: String, oi: JavaHiveVarcharObjectInspector) => new HiveVarchar(s, s.size) + case (bd: BigDecimal, oi: JavaHiveDecimalObjectInspector) => + new HiveDecimal(bd.underlying()) + case (s: Seq[_], oi: ListObjectInspector) => + seqAsJavaList(s.map(wrap(_, oi.getListElementObjectInspector))) + case (obj, _) => obj + } + /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the * `org.apache.hadoop.hive.serde2.SerDe` and the @@ -221,13 +234,7 @@ case class InsertIntoHiveTable( iter.map { row => // Casts Strings to HiveVarchars when necessary. val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector) - val mappedRow = row.zip(fieldOIs).map { - case (s: String, oi: JavaHiveVarcharObjectInspector) => new HiveVarchar(s, s.size) - case (bd: BigDecimal, oi: JavaHiveDecimalObjectInspector) => - new HiveDecimal(bd.underlying()) - case (obj, _) => obj - } - + val mappedRow = row.zip(fieldOIs).map(wrap) (null, serializer.serialize(mappedRow.toArray, standardOI)) } }.saveAsHadoopFile( From ac9d7de4f973d4809d435d098def4de12c1c0dbc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 18:37:06 -0800 Subject: [PATCH 622/778] Resolve *s in Transform clauses. --- src/main/scala/catalyst/analysis/Analyzer.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 892a5db0102b3..3c74e4a3c84f1 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -126,6 +126,13 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case o => o :: Nil }, child) + case t: Transform if containsStar(t.input) => + t.copy( + input = t.input.flatMap { + case s: Star => s.expand(t.child.output) + case o => o :: Nil + } + ) // If the aggregate function argument contains Stars, expand it. case a: Aggregate if containsStar(a.aggregateExpressions) => a.copy( @@ -139,7 +146,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool /** * Returns true if `exprs` contains a [[Star]]. */ - protected def containsStar(exprs: Seq[NamedExpression]): Boolean = + protected def containsStar(exprs: Seq[Expression]): Boolean = exprs.collect { case _: Star => true }.nonEmpty } From 7a0f543431b196f78da2f473fd2f0d3e3764d0c3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 18:37:21 -0800 Subject: [PATCH 623/778] Avoid propagating types from unresolved nodes. --- src/main/scala/catalyst/analysis/HiveTypeCoercion.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala index faf7a84b969f3..fced1e2df89fa 100644 --- a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala +++ b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala @@ -27,6 +27,9 @@ trait HiveTypeCoercion { // No propagation required for leaf nodes. case q: LogicalPlan if q.children.isEmpty => q + // Don't propagate types from unresolved children. + case q: LogicalPlan if !q.childrenResolved => q + case q: LogicalPlan => q transformExpressions { case a: AttributeReference => q.inputSet.find(_.exprId == a.exprId) match { From 010accb872f179b97b6cc6e971a7e9f17ec2de73 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 18:37:39 -0800 Subject: [PATCH 624/778] add tinyint to metastore type parser. --- src/main/scala/catalyst/execution/MetastoreCatalog.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index e3af55bb25ffa..7e2130a36ce3c 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -84,6 +84,7 @@ object HiveMetastoreTypes extends RegexParsers { "string" ^^^ StringType | "float" ^^^ FloatType | "int" ^^^ IntegerType | + "tinyint" ^^^ ShortType | "double" ^^^ DoubleType | "bigint" ^^^ LongType | "binary" ^^^ BinaryType | From e7933e912356e686ce36cc8a52dc813a7cc8c430 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 18:38:13 -0800 Subject: [PATCH 625/778] fix casting bug when working with fractional expressions. --- src/main/scala/catalyst/expressions/Evaluate.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index a866a68b471d1..56c72b43e58e9 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -65,9 +65,9 @@ object Evaluate extends Logging { null } else { e1.dataType match { - case f: FractionalType => - f.asInstanceOf[(Fractional[f.JvmType], f.JvmType, f.JvmType) => f.JvmType]( - f.fractional, evalE1.asInstanceOf[f.JvmType], evalE2.asInstanceOf[f.JvmType]) + case ft: FractionalType => + f.asInstanceOf[(Fractional[ft.JvmType], ft.JvmType, ft.JvmType) => ft.JvmType]( + ft.fractional, evalE1.asInstanceOf[ft.JvmType], evalE2.asInstanceOf[ft.JvmType]) case other => sys.error(s"Type $other does not support fractional operations") } } From 25288d055a0bcf251e64c8653442f1ee5b466e70 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 18:38:38 -0800 Subject: [PATCH 626/778] Implement [] for arrays and maps. --- .../scala/catalyst/expressions/Evaluate.scala | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 56c72b43e58e9..a78f9017bb071 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -245,6 +245,26 @@ object Evaluate extends Logging { baseValue(g.ordinal) } + case GetItem(e, o) if e.dataType.isInstanceOf[ArrayType] => + val baseValue = eval(e).asInstanceOf[Seq[_]] + val ordinal = eval(o).asInstanceOf[Int] + if(baseValue == null) { + null + } else if (ordinal >= baseValue.size) { + null + } else { + baseValue(ordinal) + } + + case GetItem(e, k) if e.dataType.isInstanceOf[MapType] => + val baseValue = eval(e).asInstanceOf[Map[Any, _]] + val key = eval(k) + if(baseValue == null) { + null + } else { + baseValue.get(key).orNull + } + /* UDFs */ case implementedFunction: ImplementedUdf => implementedFunction.evaluate(implementedFunction.children.map(eval)) From ab9a131818884dd2258174956fdca65bd14dfd42 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 18:38:58 -0800 Subject: [PATCH 627/778] when UDFs fail they should return null. --- src/main/scala/catalyst/expressions/Evaluate.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index a78f9017bb071..8d5a946f14cea 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -267,7 +267,11 @@ object Evaluate extends Logging { /* UDFs */ case implementedFunction: ImplementedUdf => - implementedFunction.evaluate(implementedFunction.children.map(eval)) + try implementedFunction.evaluate(implementedFunction.children.map(eval)) catch { + case e: Exception => + logger.error(s"UDF Evaluation failed: $e") + null + } case a: Attribute => throw new TreeNodeException(a, From 1679554ae68dfc91212ebaf8401efaf6088d61a9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 18:39:12 -0800 Subject: [PATCH 628/778] add toString for if and IS NOT NULL. --- src/main/scala/catalyst/expressions/predicates.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/scala/catalyst/expressions/predicates.scala b/src/main/scala/catalyst/expressions/predicates.scala index 5f6d062adf202..457368c518139 100644 --- a/src/main/scala/catalyst/expressions/predicates.scala +++ b/src/main/scala/catalyst/expressions/predicates.scala @@ -81,6 +81,7 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E def references = child.references override def foldable = child.foldable def nullable = false + override def toString = s"IS NOT NULL $child" } case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) @@ -98,4 +99,6 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } trueValue.dataType } + + override def toString = s"if ($predicate) $trueValue else $falseValue" } From ab5bff387f2ced791527b4c20b2c30dc7da6c190 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 18:39:28 -0800 Subject: [PATCH 629/778] Support for get item of map types. --- src/main/scala/catalyst/expressions/complexTypes.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/expressions/complexTypes.scala b/src/main/scala/catalyst/expressions/complexTypes.scala index 932133b6cc12a..2d7904f558d48 100644 --- a/src/main/scala/catalyst/expressions/complexTypes.scala +++ b/src/main/scala/catalyst/expressions/complexTypes.scala @@ -4,7 +4,7 @@ package expressions import types._ /** - * Returns the item at `ordinal` in the Array `child`. + * Returns the item at `ordinal` in the Array `child` or the Key `ordinal` in Map `child`. */ case class GetItem(child: Expression, ordinal: Expression) extends Expression { val children = child :: ordinal :: Nil @@ -13,8 +13,12 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { override def references = children.flatMap(_.references).toSet def dataType = child.dataType match { case ArrayType(dt) => dt + case MapType(_, vt) => vt } - override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[ArrayType] + override lazy val resolved = + childrenResolved && + (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) + override def toString = s"$child[$ordinal]" } From 42ec4af79020a5952bf59a5e44d6852eef5d4b41 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Feb 2014 19:07:17 -0800 Subject: [PATCH 630/778] improve complex type support in hive udfs/udafs. --- .../catalyst/execution/FunctionRegistry.scala | 46 ++++++++++++------- .../scala/catalyst/execution/aggregates.scala | 4 +- 2 files changed, 32 insertions(+), 18 deletions(-) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index f7fcb80ea46ba..a118e6032e32e 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -3,10 +3,10 @@ package execution import scala.collection.JavaConversions._ +import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.hive.serde2.objectinspector.primitive._ -import org.apache.hadoop.hive.serde2.objectinspector.{ListObjectInspector, StructObjectInspector} -import org.apache.hadoop.hive.serde2.objectinspector.{MapObjectInspector, ObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFEvaluator, AbstractGenericUDAFResolver} import org.apache.hadoop.hive.ql.udf.generic.GenericUDF @@ -69,6 +69,7 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionF case c: Class[_] if c == classOf[java.lang.Byte] => ByteType case c: Class[_] if c == classOf[java.lang.Float] => FloatType case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType + case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) } } @@ -89,7 +90,8 @@ trait HiveFunctionFactory { case b: hadoopIo.BooleanWritable => b.get() case b: hiveIo.ByteWritable => b.get case list: java.util.List[_] => list.map(unwrap) - case array: Array[_] => array.map(unwrap) + case map: java.util.Map[_,_] => map.map { case (k, v) => (unwrap(k), unwrap(v)) }.toMap + case array: Array[_] => array.map(unwrap).toSeq case p: java.lang.Short => p case p: java.lang.Long => p case p: java.lang.Float => p @@ -130,7 +132,8 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd lazy val wrappers: Array[(Any) => AnyRef] = method.getParameterTypes.map { argClass => val primitiveClasses = Seq( Integer.TYPE, classOf[java.lang.Integer], classOf[java.lang.String], java.lang.Double.TYPE, - classOf[java.lang.Double], java.lang.Long.TYPE, classOf[java.lang.Long] + classOf[java.lang.Double], java.lang.Long.TYPE, classOf[java.lang.Long], + classOf[HiveDecimal] ) val matchingConstructor = argClass.getConstructors.find { c => c.getParameterTypes.size == 1 && primitiveClasses.contains(c.getParameterTypes.head) @@ -148,6 +151,7 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } else { constructor.newInstance(a match { case i: Int => i: java.lang.Integer + case bd: BigDecimal => new HiveDecimal(bd.underlying) case other: AnyRef => other }).asInstanceOf[AnyRef] } @@ -173,16 +177,7 @@ case class HiveGenericUdf( type UDFType = GenericUDF @transient - lazy val inspectors: Seq[AbstractPrimitiveJavaObjectInspector] = children.map(_.dataType).map { - case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector - case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector - case DoubleType => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector - case BooleanType => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector - case LongType => PrimitiveObjectInspectorFactory.javaLongObjectInspector - case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector - case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector - case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector - } + lazy val inspectors: Seq[ObjectInspector] = children.map(_.dataType).map(toInspector) @transient lazy val objectInspector = function.initialize(inspectors.toArray) @@ -198,6 +193,8 @@ case class HiveGenericUdf( case l: Short => l: java.lang.Short case l: Byte => l: java.lang.Byte case s: Seq[_] => seqAsJavaList(s.map(wrap)) + case m: Map[_,_] => + mapAsJavaMap(m.map { case (k, v) => wrap(k) -> wrap(v) }) case null => null } @@ -211,20 +208,27 @@ case class HiveGenericUdf( } trait HiveInspectors { - def toInspectors(exprs: Seq[Expression]) = exprs.map(_.dataType).map { + def toInspector(dataType: DataType): ObjectInspector = dataType match { + case ArrayType(tpe) => ObjectInspectorFactory.getStandardListObjectInspector(toInspector(tpe)) + case MapType(keyType, valueType) => + ObjectInspectorFactory.getStandardMapObjectInspector( + toInspector(keyType), toInspector(valueType)) case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector case DoubleType => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector case BooleanType => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector case LongType => PrimitiveObjectInspectorFactory.javaLongObjectInspector + case FloatType => PrimitiveObjectInspectorFactory.javaFloatObjectInspector case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector + case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector } def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { case s: StructObjectInspector => StructType(s.getAllStructFieldRefs.map(f => { - StructField(f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), true) + types.StructField( + f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), nullable = true) })) case l: ListObjectInspector => ArrayType(inspectorToDataType(l.getListElementObjectInspector)) case m: MapObjectInspector => @@ -234,11 +238,17 @@ trait HiveInspectors { case _: WritableStringObjectInspector => StringType case _: JavaStringObjectInspector => StringType case _: WritableIntObjectInspector => IntegerType + case _: JavaIntObjectInspector => IntegerType case _: WritableDoubleObjectInspector => DoubleType + case _: JavaDoubleObjectInspector => DoubleType case _: WritableBooleanObjectInspector => BooleanType + case _: JavaBooleanObjectInspector => BooleanType case _: WritableLongObjectInspector => LongType + case _: JavaLongObjectInspector => LongType case _: WritableShortObjectInspector => ShortType + case _: JavaShortObjectInspector => ShortType case _: WritableByteObjectInspector => ByteType + case _: JavaByteObjectInspector => ByteType } } @@ -257,11 +267,13 @@ case class HiveGenericUdaf( .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) } - lazy val inspectors: Seq[ObjectInspector] = toInspectors(children) + lazy val inspectors: Seq[ObjectInspector] = children.map(_.dataType).map(toInspector) def dataType: DataType = inspectorToDataType(objectInspector) def nullable: Boolean = true def references: Set[Attribute] = children.map(_.references).flatten.toSet + + override def toString = s"$nodeName#$name(${children.mkString(",")})" } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 5f7a22e43fe9d..5e3cecbe4dd04 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -37,9 +37,11 @@ case class Aggregate( val resolver = createFunction[AbstractGenericUDAFResolver](functionName) + val inspectors = exprs.map(_.dataType).map(toInspector).toArray + val function = { val evaluator = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray) - evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, toInspectors(exprs).toArray) + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) evaluator } From e3c10bd5649658995c3a347ebe1ab434fad50cdc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 4 Feb 2014 00:57:55 -0800 Subject: [PATCH 631/778] update whitelist. --- .../execution/HiveCompatibility.scala | 51 ++++++++++++++----- 1 file changed, 37 insertions(+), 14 deletions(-) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index dd14b1feaf9b6..9c0fdd1307ae7 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -150,12 +150,8 @@ class HiveCompatibility extends HiveQueryFileTest { "alter2", "alter4", "alter5", - "alter_concatenate_indexed_table", "alter_index", - "alter_merge", "alter_merge_2", - "alter_merge_stats", - "alter_partition_clusterby_sortby", "alter_partition_format_loc", "alter_partition_protect_mode", "alter_partition_with_whitelist", @@ -182,6 +178,7 @@ class HiveCompatibility extends HiveQueryFileTest { "auto_join24", "auto_join25", "auto_join26", + "auto_join27", "auto_join28", "auto_join3", "auto_join30", @@ -233,11 +230,9 @@ class HiveCompatibility extends HiveQueryFileTest { "delimiter", "desc_non_existent_tbl", "describe_comment_indent", - "describe_comment_nonascii", "describe_database_json", "describe_pretty", "describe_syntax", - "describe_table", "describe_table_json", "diff_part_input_formats", "disable_file_format_check", @@ -253,6 +248,7 @@ class HiveCompatibility extends HiveQueryFileTest { "escape_distributeby1", "escape_orderby1", "escape_sortby1", + "fetch_aggregation", "filter_join_breaktask", "filter_join_breaktask2", "groupby1", @@ -261,12 +257,18 @@ class HiveCompatibility extends HiveQueryFileTest { "groupby1_map_nomap", "groupby1_map_skew", "groupby1_noskew", + "groupby4", "groupby4_map", "groupby4_map_skew", + "groupby4_noskew", "groupby5", "groupby5_map", "groupby5_map_skew", "groupby5_noskew", + "groupby6", + "groupby6_map", + "groupby6_map_skew", + "groupby6_noskew", "groupby7", "groupby7_map", "groupby7_map_multi_single_reducer", @@ -277,11 +279,13 @@ class HiveCompatibility extends HiveQueryFileTest { "groupby8_noskew", "groupby_multi_single_reducer2", "groupby_mutli_insert_common_distinct", + "groupby_neg_float", + "groupby_sort_10", "groupby_sort_6", "groupby_sort_8", "groupby_sort_test_1", "implicit_cast1", - "index_auto_unused", + "index_stale_partitioned", "innerjoin", "inoutdriver", "input", @@ -383,18 +387,20 @@ class HiveCompatibility extends HiveQueryFileTest { "literal_string", "load_dyn_part7", "load_file_with_space_in_the_name", - "load_overwrite", "louter_join_ppr", + "mapjoin_distinct", "mapjoin_mapjoin", "mapjoin_subquery", "mapjoin_subquery2", "mapjoin_test_outer", "mapreduce3", + "mapreduce7", "merge1", "merge2", "mergejoins", "mergejoins_mixed", "multiMapJoin1", + "multiMapJoin2", "multi_join_union", "multigroupby_singlemr", "noalias_subq1", @@ -419,14 +425,17 @@ class HiveCompatibility extends HiveQueryFileTest { "part_inherit_tbl_props_empty", "part_inherit_tbl_props_with_star", "partition_schema1", + "partition_varchar1", "plan_json", "ppd1", "ppd_constant_where", "ppd_gby", + "ppd_gby2", "ppd_gby_join", "ppd_join", "ppd_join2", "ppd_join3", + "ppd_join_filter", "ppd_outer_join1", "ppd_outer_join2", "ppd_outer_join3", @@ -442,7 +451,6 @@ class HiveCompatibility extends HiveQueryFileTest { "ppr_pushdown3", "progress_1", "protectmode", - "protectmode2", "push_or", "query_with_semi", "quote1", @@ -485,18 +493,18 @@ class HiveCompatibility extends HiveQueryFileTest { "sort_merge_join_desc_6", "sort_merge_join_desc_7", "stats0", - "stats1", - "stats14", - "stats15", - "stats16", - "stats18", "stats20", "stats_empty_partition", "subq2", "tablename_with_select", "touch", "type_widening", + "udaf_collect_set", + "udaf_corr", + "udaf_covar_pop", + "udaf_covar_samp", "udf2", + "udf5", "udf6", "udf9", "udf_10_trims", @@ -505,21 +513,28 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_abs", "udf_acos", "udf_add", + "udf_array", + "udf_array_contains", "udf_ascii", "udf_asin", "udf_atan", "udf_avg", "udf_bigint", "udf_bin", + "udf_bitmap_and", + "udf_bitmap_empty", + "udf_bitmap_or", "udf_bitwise_and", "udf_bitwise_not", "udf_bitwise_or", "udf_bitwise_xor", "udf_boolean", + "udf_case", "udf_ceil", "udf_ceiling", "udf_concat", "udf_concat_insert2", + "udf_concat_ws", "udf_conv", "udf_cos", "udf_count", @@ -541,6 +556,7 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_greaterthan", "udf_greaterthanorequal", "udf_hex", + "udf_if", "udf_index", "udf_int", "udf_isnotnull", @@ -558,6 +574,7 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_lower", "udf_lpad", "udf_ltrim", + "udf_map", "udf_minute", "udf_modulo", "udf_month", @@ -578,6 +595,7 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_regexp_replace", "udf_repeat", "udf_rlike", + "udf_round", "udf_round_3", "udf_rpad", "udf_rtrim", @@ -597,7 +615,12 @@ class HiveCompatibility extends HiveQueryFileTest { "udf_sum", "udf_tan", "udf_tinyint", + "udf_to_byte", "udf_to_date", + "udf_to_double", + "udf_to_float", + "udf_to_long", + "udf_to_short", "udf_translate", "udf_trim", "udf_ucase", From 389525dedbc7c6c83d6686a7661c98354f60425e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 4 Feb 2014 10:44:35 -0800 Subject: [PATCH 632/778] update golden, blacklist mr. --- build.sbt | 2 +- src/test/scala/catalyst/execution/HiveCompatibility.scala | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/build.sbt b/build.sbt index 58b00e1438bdc..b8f7010573bd1 100644 --- a/build.sbt +++ b/build.sbt @@ -17,7 +17,7 @@ resolvers += "SparkStaging" at "https://repository.apache.org/content/repositori libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" -libraryDependencies += "catalyst" % "hive-golden" % "6" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden6.jar" +libraryDependencies += "catalyst" % "hive-golden" % "7" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden7.jar" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index 9c0fdd1307ae7..9114c617f7528 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -62,7 +62,7 @@ class HiveCompatibility extends HiveQueryFileTest { // Cant run without local map/reduce. "index_auto_update", "index_auto_self_join", - "index_stale", + "index_stale.*", "type_cast_1", "index_compression", "index_bitmap_compression", @@ -285,7 +285,6 @@ class HiveCompatibility extends HiveQueryFileTest { "groupby_sort_8", "groupby_sort_test_1", "implicit_cast1", - "index_stale_partitioned", "innerjoin", "inoutdriver", "input", From 2f276049070ccd873368441e652c0d6a2d3e2551 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 4 Feb 2014 11:23:12 -0800 Subject: [PATCH 633/778] Address comments / style errors. --- .../catalyst/execution/FunctionRegistry.scala | 10 +-- .../catalyst/execution/MetastoreCatalog.scala | 2 +- .../scala/catalyst/execution/aggregates.scala | 64 +++++++++---------- .../catalyst/execution/hiveOperators.scala | 8 ++- .../scala/catalyst/expressions/Evaluate.scala | 10 +-- .../catalyst/plans/logical/LogicalPlan.scala | 6 +- .../plans/physical/partitioning.scala | 2 +- 7 files changed, 53 insertions(+), 49 deletions(-) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index a118e6032e32e..6bcb04da3991f 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -79,6 +79,7 @@ trait HiveFunctionFactory { def createFunction[UDFType](name: String) = getFunctionClass(name).newInstance.asInstanceOf[UDFType] + /** Converts hive types to native catalyst types. */ def unwrap(a: Any): Any = a match { case null => null case i: hadoopIo.IntWritable => i.get @@ -177,13 +178,14 @@ case class HiveGenericUdf( type UDFType = GenericUDF @transient - lazy val inspectors: Seq[ObjectInspector] = children.map(_.dataType).map(toInspector) + lazy val argumentInspectors: Seq[ObjectInspector] = children.map(_.dataType).map(toInspector) @transient - lazy val objectInspector = function.initialize(inspectors.toArray) + lazy val returnInspector = function.initialize(argumentInspectors.toArray) - val dataType: DataType = inspectorToDataType(objectInspector) + val dataType: DataType = inspectorToDataType(returnInspector) + /** Converts native catalyst types to the types expected by Hive */ def wrap(a: Any): Any = a match { case s: String => new hadoopIo.Text(s) case i: Int => i: java.lang.Integer @@ -199,7 +201,7 @@ case class HiveGenericUdf( } def evaluate(evaluatedChildren: Seq[Any]): Any = { - objectInspector // Make sure initialized. + returnInspector // Make sure initialized. val args = evaluatedChildren.map(wrap).map { v => new DeferredJavaObject(v): DeferredObject }.toArray diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index b6e4fab20d8dc..2a98891a40f2f 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -90,7 +90,7 @@ object HiveMetastoreTypes extends RegexParsers { "binary" ^^^ BinaryType | "boolean" ^^^ BooleanType | "decimal" ^^^ DecimalType | - "(?i)VARCHAR\\((\\d+)\\)".r ^^^ StringType + "varchar\\((\\d+)\\)".r ^^^ StringType protected lazy val arrayType: Parser[DataType] = "array" ~> "<" ~> dataType <~ ">" ^^ ArrayType diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 5e3cecbe4dd04..ccc0a649812a2 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -25,38 +25,6 @@ case class Aggregate( override def otherCopyArgs = sc :: Nil - case class HiveUdafFunction( - exprs: Seq[Expression], - base: AggregateExpression, - functionName: String) - extends AggregateFunction - with HiveInspectors - with HiveFunctionFactory { - - def this() = this(null, null, null) - - val resolver = createFunction[AbstractGenericUDAFResolver](functionName) - - val inspectors = exprs.map(_.dataType).map(toInspector).toArray - - val function = { - val evaluator = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray) - evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) - evaluator - } - - // Cast required to avoid type inference selecting a deprecated Hive API. - val buffer = - function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] - - def result: Any = unwrap(function.evaluate(buffer)) - - def apply(input: Seq[Row]): Unit = { - val inputs = exprs.map(Evaluate(_, input).asInstanceOf[AnyRef]).toArray - function.iterate(buffer, inputs) - } - } - def output = aggregateExpressions.map(_.toAttribute) /* Replace all aggregate expressions with spark functions that will compute the result. */ @@ -122,6 +90,38 @@ case class Aggregate( } } +case class HiveUdafFunction( + exprs: Seq[Expression], + base: AggregateExpression, + functionName: String) + extends AggregateFunction + with HiveInspectors + with HiveFunctionFactory { + + def this() = this(null, null, null) + + private val resolver = createFunction[AbstractGenericUDAFResolver](functionName) + + private val inspectors = exprs.map(_.dataType).map(toInspector).toArray + + private val function = { + val evaluator = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray) + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) + evaluator + } + + // Cast required to avoid type inference selecting a deprecated Hive API. + private val buffer = + function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] + + def result: Any = unwrap(function.evaluate(buffer)) + + def apply(input: Seq[Row]): Unit = { + val inputs = exprs.map(Evaluate(_, input).asInstanceOf[AnyRef]).toArray + function.iterate(buffer, inputs) + } +} + // TODO: Move these default functions back to expressions. Build framework for instantiating them. case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index d5b8986931269..84a9bd5d8418c 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -12,7 +12,8 @@ import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.hadoop.hive.serde2.objectinspector.primitive.{JavaHiveDecimalObjectInspector, JavaHiveVarcharObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector import org.apache.hadoop.io.NullWritable import org.apache.hadoop.mapred.JobConf import org.apache.spark.SparkContext._ @@ -91,8 +92,9 @@ case class HiveTableScan( def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) case li: ListObjectInspector => - Option( - li.getList(data)).map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq).orNull + Option(li.getList(data)) + .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) + .orNull case mi: MapObjectInspector => Option(mi.getMap(data)).map( _.map { diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 8d5a946f14cea..209f191381485 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -239,16 +239,12 @@ object Evaluate extends Logging { /* Complex Type Access */ case g @ GetField(e, name) => val baseValue = eval(e).asInstanceOf[Row] - if(baseValue == null) { - null - } else { - baseValue(g.ordinal) - } + if (baseValue == null) null else baseValue(g.ordinal) case GetItem(e, o) if e.dataType.isInstanceOf[ArrayType] => val baseValue = eval(e).asInstanceOf[Seq[_]] val ordinal = eval(o).asInstanceOf[Int] - if(baseValue == null) { + if (baseValue == null) { null } else if (ordinal >= baseValue.size) { null @@ -259,7 +255,7 @@ object Evaluate extends Logging { case GetItem(e, k) if e.dataType.isInstanceOf[MapType] => val baseValue = eval(e).asInstanceOf[Map[Any, _]] val key = eval(k) - if(baseValue == null) { + if (baseValue == null) { null } else { baseValue.get(key).orNull diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala index 245fc2a2277bc..a2350d14dc6cd 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala @@ -41,10 +41,14 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { */ def resolve(name: String): Option[NamedExpression] = { val parts = name.split("\\.") + // Collect all attributes that are output by this nodes children where either the first part + // matches the name or where the first part matches the scope and the second part matches the + // name. Return these matches along with any remaining parts, which represent dotted access to + // struct fields. val options = children.flatMap(_.output).flatMap { option => // If the first part of the desired name matches a qualifier for this possible match, drop it. val remainingParts = if (option.qualifiers contains parts.head) parts.drop(1) else parts - if(option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil + if (option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil } options.distinct match { diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index dd91b5beb4f95..25e2240372e11 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -180,4 +180,4 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) case r: RangePartitioning if r == this => true case _ => false } -} \ No newline at end of file +} From cb57459ce009bdf8e58e7eaf1c301279b5a07ce7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 4 Feb 2014 11:24:33 -0800 Subject: [PATCH 634/778] blacklist machine specific test. --- src/test/scala/catalyst/execution/HiveCompatibility.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index 9114c617f7528..53ab694c16b97 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -49,6 +49,7 @@ class HiveCompatibility extends HiveQueryFileTest { "protectmode2", "describe_table", "describe_comment_nonascii", + "udf5", // Weird DDL differences result in failures on jenkins. "create_like2", @@ -503,7 +504,6 @@ class HiveCompatibility extends HiveQueryFileTest { "udaf_covar_pop", "udaf_covar_samp", "udf2", - "udf5", "udf6", "udf9", "udf_10_trims", From b4be6a5411cd3d25919bc71563da44638660ecb6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 4 Feb 2014 16:53:46 -0800 Subject: [PATCH 635/778] better logging when applying rules. --- .../scala/catalyst/rules/RuleExecutor.scala | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/rules/RuleExecutor.scala b/src/main/scala/catalyst/rules/RuleExecutor.scala index 59c5f29c83f49..69cd1dae5b00a 100644 --- a/src/main/scala/catalyst/rules/RuleExecutor.scala +++ b/src/main/scala/catalyst/rules/RuleExecutor.scala @@ -2,8 +2,9 @@ package catalyst package rules import trees._ +import util._ -abstract class RuleExecutor[TreeType <: TreeNode[_]] { +abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { /** * An execution strategy for rules that indicates the maximum number of executions. If the @@ -38,7 +39,19 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] { // Run until fix point (or the max number of iterations as specified in the strategy. while (iteration < batch.strategy.maxIterations && !curPlan.fastEquals(lastPlan)) { lastPlan = curPlan - curPlan = batch.rules.foldLeft(curPlan) { case (curPlan, rule) => rule(curPlan) } + curPlan = batch.rules.foldLeft(curPlan) { + case (curPlan, rule) => + val result = rule(curPlan) + if(!result.fastEquals(curPlan)) { + logger.debug( + s""" + |=== Applying Rule ${rule.ruleName} === + |${sideBySide(curPlan.treeString, result.treeString).mkString("\n")} + """.stripMargin) + } + + result + } iteration += 1 } } From ccdb07a18c62c7c955400e3253d81adbd6e8f42e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 4 Feb 2014 16:54:23 -0800 Subject: [PATCH 636/778] Fix bug where averages of strings are turned into sums of strings. Remove a blank line. --- src/main/scala/catalyst/analysis/HiveTypeCoercion.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala index fced1e2df89fa..ce2f660a7c778 100644 --- a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala +++ b/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala @@ -190,7 +190,7 @@ trait HiveTypeCoercion { case Sum(e) if e.dataType == StringType => Sum(Cast(e, DoubleType)) case Average(e) if e.dataType == StringType => - Sum(Cast(e, DoubleType)) + Average(Cast(e, DoubleType)) } } @@ -252,7 +252,6 @@ trait HiveTypeCoercion { case s @ Sum(e @ DecimalType()) => s // Decimal is already the biggest. case Sum(e @ IntegralType()) if e.dataType != LongType => Sum(Cast(e, LongType)) case Sum(e @ FractionalType()) if e.dataType != DoubleType => Sum(Cast(e, DoubleType)) - } } } From d8cb805193f7d8ffe96efc423bb86f781ea3ef41 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 4 Feb 2014 17:50:48 -0800 Subject: [PATCH 637/778] Implement partial aggregation. --- .../execution/PlanningStrategies.scala | 59 +++++++++++++++++- .../catalyst/execution/SharkInstance.scala | 1 + .../scala/catalyst/execution/aggregates.scala | 23 +++++-- .../catalyst/expressions/aggregates.scala | 62 ++++++++++++++++++- .../catalyst/execution/PlannerSuite.scala | 36 +++++++++++ 5 files changed, 172 insertions(+), 9 deletions(-) create mode 100644 src/test/scala/catalyst/execution/PlannerSuite.scala diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index 22ee2188de19c..4d8723d90c13e 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -119,6 +119,60 @@ trait PlanningStrategies { expr.references subsetOf plan.outputSet } + object PartialAggregation extends Strategy { + def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { + case logical.Aggregate(groupingExpressions, aggregateExpressions, child) => + // Collect all aggregate expressions. + val allAggregates = + aggregateExpressions.flatMap(_ collect { case a: AggregateExpression => a}) + // Collect all aggregate expressions that can be computed partially. + val partialAggregates = + aggregateExpressions.flatMap(_ collect { case p: PartialAggregate => p}) + + // Only do partial aggregation if supported by all aggregate expressions. + if (allAggregates.size == partialAggregates.size) { + // Create a map of expressions to their partial evaluations for all aggregate expressions. + val partialEvaluations: Map[Long, SplitEvaluation] = + partialAggregates.map(a => (a.id, a.asPartial)).toMap + + // We need to pass all grouping expressions though so the grouping can happen a second + // time. However some of them might be unnamed so we alias them allowing them to be + // referenced in the second aggregation. + val namedGroupingExpressions: Map[Expression, NamedExpression] = groupingExpressions.map { + case n: NamedExpression => (n, n) + case other => (other, Alias(other, "PartialGroup")()) + }.toMap + + // Replace aggregations with a new expression that computes the result from the already + // computed partial evaluations and grouping values. + val rewrittenAggregateExpressions = aggregateExpressions.map(_.transformUp { + case e: Expression if partialEvaluations.contains(e.id) => + partialEvaluations(e.id).finalEvaluation + case e: Expression if namedGroupingExpressions.contains(e) => + namedGroupingExpressions(e).toAttribute + }).asInstanceOf[Seq[NamedExpression]] + + val partialComputation = + (namedGroupingExpressions.values ++ + partialEvaluations.values.flatMap(_.partialEvaluations)).toSeq + + // Construct two phased aggregation. + execution.Aggregate( + partial = false, + namedGroupingExpressions.values.map(_.toAttribute).toSeq, + rewrittenAggregateExpressions, + execution.Aggregate( + partial = true, + groupingExpressions, + partialComputation, + planLater(child))(sc))(sc) :: Nil + } else { + Nil + } + case _ => Nil + } + } + object BroadcastNestedLoopJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Join(left, right, joinType, condition) => @@ -143,7 +197,8 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Distinct(child) => - execution.Aggregate(child.output, child.output, planLater(child))(sc) :: Nil + execution.Aggregate( + partial = false, child.output, child.output, planLater(child))(sc) :: Nil case logical.Sort(sortExprs, child) => // This sort is a global sort. Its requiredDistribution will be an OrderedDistribution. execution.Sort(sortExprs, global = true, planLater(child)):: Nil @@ -156,7 +211,7 @@ trait PlanningStrategies { case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(group, agg, planLater(child))(sc) :: Nil + execution.Aggregate(partial = false, group, agg, planLater(child))(sc) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index c81ace5387d2b..c5319cbcbd87c 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -73,6 +73,7 @@ abstract class SharkInstance extends Logging { object TrivialPlanner extends QueryPlanner[SharkPlan] with PlanningStrategies { val sc = self.sc val strategies = + PartialAggregation :: SparkEquiInnerJoin :: PartitionPrunings :: HiveTableScans :: diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index ccc0a649812a2..735683524cea2 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -5,22 +5,37 @@ import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFEvaluator, AbstractGene import catalyst.errors._ import catalyst.expressions._ -import catalyst.plans.physical.{ClusteredDistribution, AllTuples} +import catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples} /* Implicits */ import org.apache.spark.rdd.SharkPairRDDFunctions._ +/** + * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each + * group. + * + * @param partial if true then aggregation is done partially on local data without shuffling to + * ensure all values where `groupingExpressions` are equal are present. + * @param groupingExpressions expressions that are evaluated to determine grouping. + * @param aggregateExpressions expressions that are computed for each group. + * @param child the input data source. + */ case class Aggregate( + partial: Boolean, groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { override def requiredChildDistribution = - if (groupingExpressions == Nil) { - AllTuples :: Nil + if (partial) { + UnspecifiedDistribution :: Nil } else { - ClusteredDistribution(groupingExpressions) :: Nil + if (groupingExpressions == Nil) { + AllTuples :: Nil + } else { + ClusteredDistribution(groupingExpressions) :: Nil + } } override def otherCopyArgs = sc :: Nil diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala index 0778f40d23b52..e97a43bb6fdc9 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/src/main/scala/catalyst/expressions/aggregates.scala @@ -8,9 +8,35 @@ abstract class AggregateExpression extends Expression { } + +/** + * Represents an aggregation that has been rewritten to be performed in two steps. + * + * @param finalEvaluation an aggregate expression that evaluates to same final result as the + * original aggregation. + * @param partialEvaluations A sequence of [[NamedExpressions]] that can be computed on partial + * data sets and are required to compute the `finalEvaluation`. + */ +case class SplitEvaluation( + finalEvaluation: Expression, + partialEvaluations: Seq[NamedExpression]) + +/** + * An [[AggregateExpression]] that can be partially computed without seeing all relevent tuples. + * These partial evaluations can then be combined to compute the actual answer. + */ +abstract class PartialAggregate extends AggregateExpression { + self: Product => + + /** + * Returns a [[SplitEvaluation]] that computes this aggregation using partial aggregation. + */ + def asPartial: SplitEvaluation +} + /** * A specific implementation of an aggregate function. Used to wrap a generic - * [[AggregateExpression]] with an algorithm that will be used to compute the result. + * [[AggregateExpression]] with an algorithm that will be used to compute one specific result. */ abstract class AggregateFunction extends AggregateExpression with Serializable with trees.LeafNode[Expression] { @@ -26,11 +52,16 @@ abstract class AggregateFunction def result: Any } -case class Count(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { +case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { def references = child.references def nullable = false def dataType = IntegerType override def toString = s"COUNT($child)" + + def asPartial: SplitEvaluation = { + val partialCount = Alias(Count(child), "PartialCount")() + SplitEvaluation(Sum(partialCount.toAttribute), partialCount :: Nil) + } } case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { @@ -41,11 +72,22 @@ case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpressi override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")}})" } -case class Average(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { +case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { def references = child.references def nullable = false def dataType = DoubleType override def toString = s"AVG($child)" + + def asPartial: SplitEvaluation = { + val partialSum = Alias(Sum(child), "PartialSum")() + val partialCount = Alias(Count(child), "PartialCount")() + val castedSum = Cast(Sum(partialSum.toAttribute), dataType) + val castedCount = Cast(Sum(partialCount.toAttribute), dataType) + + SplitEvaluation( + Divide(castedSum, castedCount), + partialCount :: partialSum :: Nil) + } } case class Sum(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { @@ -53,6 +95,13 @@ case class Sum(child: Expression) extends AggregateExpression with trees.UnaryNo def nullable = false def dataType = child.dataType override def toString = s"SUM($child)" + + def asPartial: SplitEvaluation = { + val partialSum = Alias(Sum(child), "PartialSum")() + SplitEvaluation( + Sum(partialSum.toAttribute), + partialSum :: Nil) + } } case class First(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { @@ -60,4 +109,11 @@ case class First(child: Expression) extends AggregateExpression with trees.Unary def nullable = child.nullable def dataType = child.dataType override def toString = s"FIRST($child)" + + def asPartial: SplitEvaluation = { + val partialFirst = Alias(Sum(child), "PartialFirst")() + SplitEvaluation( + First(partialFirst.toAttribute), + partialFirst :: Nil) + } } diff --git a/src/test/scala/catalyst/execution/PlannerSuite.scala b/src/test/scala/catalyst/execution/PlannerSuite.scala new file mode 100644 index 0000000000000..6149f546c6441 --- /dev/null +++ b/src/test/scala/catalyst/execution/PlannerSuite.scala @@ -0,0 +1,36 @@ +package catalyst +package execution + +import org.scalatest.FunSuite + +import catalyst.expressions._ +import catalyst.plans.logical +import catalyst.dsl._ + +class PlannerSuite extends FunSuite { + import TestData._ + + test("Unions are collapsed") { + val query = testData.unionAll(testData).unionAll(testData) + val planned = TestShark.TrivialPlanner.BasicOperators(query).head + val logicalUnions = query collect { case u: logical.Union => u} + val physicalUnions = planned collect { case u: execution.Union => u} + + assert(logicalUnions.size === 2) + assert(physicalUnions.size === 1) + } + + test("counts are partially aggregated") { + val query = testData.groupBy('value)(Count('key)).analyze + val planned = TestShark.TrivialPlanner.PartialAggregation(query).head + val aggregations = planned.collect { case a: Aggregate => a } + + assert(aggregations.size === 2) + } + + test("counts are not partially aggregated") { + val query = testData.groupBy('value)(CountDistinct('key :: Nil)).analyze + val planned = TestShark.TrivialPlanner.PartialAggregation(query) + assert(planned.isEmpty) + } +} \ No newline at end of file From f94345cb0ed64b8566da623e765a04cac6739733 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 4 Feb 2014 18:44:38 -0800 Subject: [PATCH 638/778] fix doc link --- src/main/scala/catalyst/expressions/aggregates.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala index e97a43bb6fdc9..88e414a04a338 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/src/main/scala/catalyst/expressions/aggregates.scala @@ -14,7 +14,7 @@ abstract class AggregateExpression extends Expression { * * @param finalEvaluation an aggregate expression that evaluates to same final result as the * original aggregation. - * @param partialEvaluations A sequence of [[NamedExpressions]] that can be computed on partial + * @param partialEvaluations A sequence of [[NamedExpression]]s that can be computed on partial * data sets and are required to compute the `finalEvaluation`. */ case class SplitEvaluation( From e1999f927a41eae4a9affe2728296a1a9ee06cb8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 5 Feb 2014 12:38:11 +0800 Subject: [PATCH 639/778] Use Deserializer and Serializer instead of AbstractSerDe. --- .../scala/catalyst/execution/TableReader.scala | 10 +++++----- .../scala/catalyst/execution/hiveOperators.scala | 16 ++++++++-------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/main/scala/catalyst/execution/TableReader.scala b/src/main/scala/catalyst/execution/TableReader.scala index 1f7221e84bfe1..67572627fbc22 100644 --- a/src/main/scala/catalyst/execution/TableReader.scala +++ b/src/main/scala/catalyst/execution/TableReader.scala @@ -5,7 +5,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.serde2.AbstractSerDe +import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.io.Writable import org.apache.hadoop.fs.{Path, PathFilter} @@ -55,7 +55,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = makeRDDForTable( hiveTable, - _tableDesc.getDeserializerClass.asInstanceOf[Class[AbstractSerDe]], + _tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], filterOpt = None) /** @@ -69,7 +69,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf */ def makeRDDForTable( hiveTable: HiveTable, - deserializerClass: Class[_ <: AbstractSerDe], + deserializerClass: Class[_ <: Deserializer], filterOpt: Option[PathFilter]): RDD[_] = { assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, @@ -105,7 +105,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { val partitionToDeserializer = partitions.map(part => - (part, part.getDeserializer.getClass.asInstanceOf[Class[AbstractSerDe]])).toMap + (part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None) } @@ -120,7 +120,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient _localHConf * subdirectory of each partition being read. If None, then all files are accepted. */ def makeRDDForPartitionedTable( - partitionToDeserializer: Map[HivePartition, Class[_ <: AbstractSerDe]], + partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[_] = { val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 84a9bd5d8418c..57049a354888e 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -1,15 +1,14 @@ package catalyst package execution -import java.io.{File, IOException} -import java.util.UUID +import java.io.File import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} -import org.apache.hadoop.hive.serde2.AbstractSerDe +import org.apache.hadoop.hive.serde2.Serializer import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector @@ -186,8 +185,8 @@ case class InsertIntoHiveTable( new Path((new org.apache.hadoop.fs.RawLocalFileSystem).getWorkingDirectory, "test.out"), null) - private def newSerializer(tableDesc: TableDesc) = { - val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[AbstractSerDe] + private def newSerializer(tableDesc: TableDesc): Serializer = { + val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] serializer.initialize(null, tableDesc.getProperties) serializer } @@ -223,14 +222,15 @@ case class InsertIntoHiveTable( tempDir.delete() tempDir.mkdir() - // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer - // instances within the closure, since AbstractSerDe is not serializable while TableDesc is. + // instances within the closure, since Serializer is not serializable while TableDesc is. val tableDesc = table.tableDesc childRdd.mapPartitions { iter => val serializer = newSerializer(tableDesc) val standardOI = ObjectInspectorUtils - .getStandardObjectInspector(serializer.getObjectInspector, ObjectInspectorCopyOption.JAVA) + .getStandardObjectInspector( + tableDesc.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) .asInstanceOf[StructObjectInspector] iter.map { row => From 32b615b52e7c202b29e1242952092d09f3332745 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Feb 2014 01:36:12 -0800 Subject: [PATCH 640/778] add override to asPartial. --- src/main/scala/catalyst/expressions/aggregates.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala index 88e414a04a338..191cc39bec912 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/src/main/scala/catalyst/expressions/aggregates.scala @@ -78,7 +78,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN def dataType = DoubleType override def toString = s"AVG($child)" - def asPartial: SplitEvaluation = { + override def asPartial: SplitEvaluation = { val partialSum = Alias(Sum(child), "PartialSum")() val partialCount = Alias(Count(child), "PartialCount")() val castedSum = Cast(Sum(partialSum.toAttribute), dataType) @@ -96,7 +96,7 @@ case class Sum(child: Expression) extends AggregateExpression with trees.UnaryNo def dataType = child.dataType override def toString = s"SUM($child)" - def asPartial: SplitEvaluation = { + override def asPartial: SplitEvaluation = { val partialSum = Alias(Sum(child), "PartialSum")() SplitEvaluation( Sum(partialSum.toAttribute), @@ -104,13 +104,13 @@ case class Sum(child: Expression) extends AggregateExpression with trees.UnaryNo } } -case class First(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { +case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { def references = child.references def nullable = child.nullable def dataType = child.dataType override def toString = s"FIRST($child)" - def asPartial: SplitEvaluation = { + override def asPartial: SplitEvaluation = { val partialFirst = Alias(Sum(child), "PartialFirst")() SplitEvaluation( First(partialFirst.toAttribute), From 883006dd16cbd1ddb61f164ad28a8237f4c6becc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Feb 2014 01:46:39 -0800 Subject: [PATCH 641/778] improve tests. --- src/test/scala/catalyst/execution/PlannerSuite.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/test/scala/catalyst/execution/PlannerSuite.scala b/src/test/scala/catalyst/execution/PlannerSuite.scala index 6149f546c6441..3736030d3fceb 100644 --- a/src/test/scala/catalyst/execution/PlannerSuite.scala +++ b/src/test/scala/catalyst/execution/PlannerSuite.scala @@ -10,7 +10,7 @@ import catalyst.dsl._ class PlannerSuite extends FunSuite { import TestData._ - test("Unions are collapsed") { + test("unions are collapsed") { val query = testData.unionAll(testData).unionAll(testData) val planned = TestShark.TrivialPlanner.BasicOperators(query).head val logicalUnions = query collect { case u: logical.Union => u} @@ -20,7 +20,7 @@ class PlannerSuite extends FunSuite { assert(physicalUnions.size === 1) } - test("counts are partially aggregated") { + test("count is partially aggregated") { val query = testData.groupBy('value)(Count('key)).analyze val planned = TestShark.TrivialPlanner.PartialAggregation(query).head val aggregations = planned.collect { case a: Aggregate => a } @@ -28,9 +28,15 @@ class PlannerSuite extends FunSuite { assert(aggregations.size === 2) } - test("counts are not partially aggregated") { + test("count distinct is not partially aggregated") { val query = testData.groupBy('value)(CountDistinct('key :: Nil)).analyze val planned = TestShark.TrivialPlanner.PartialAggregation(query) assert(planned.isEmpty) } + + test("mixed aggregates are not partially aggregated") { + val query = testData.groupBy('value)(Count('value), CountDistinct('key :: Nil)).analyze + val planned = TestShark.TrivialPlanner.PartialAggregation(query) + assert(planned.isEmpty) + } } \ No newline at end of file From cab1a84b4811064fe217b0cd56d3fe9c48210b6a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Feb 2014 02:01:08 -0800 Subject: [PATCH 642/778] Fix PartialAggregate inheritance. --- src/main/scala/catalyst/expressions/aggregates.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala index 191cc39bec912..dc7072e2f5719 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/src/main/scala/catalyst/expressions/aggregates.scala @@ -8,7 +8,6 @@ abstract class AggregateExpression extends Expression { } - /** * Represents an aggregation that has been rewritten to be performed in two steps. * @@ -90,7 +89,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN } } -case class Sum(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { +case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { def references = child.references def nullable = false def dataType = child.dataType From dc6353be64bfe9c6522403a5a4124423cd62e22b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Feb 2014 02:03:58 -0800 Subject: [PATCH 643/778] turn off deprecation --- build.sbt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index b8f7010573bd1..b962979ba0c85 100644 --- a/build.sbt +++ b/build.sbt @@ -8,7 +8,8 @@ version := "0.1-SNAPSHOT" scalaVersion := "2.10.3" -scalacOptions ++= Seq("-deprecation", "-feature", "-unchecked") +// TODO: Re-enable -deprecation +scalacOptions ++= Seq("-feature", "-unchecked") resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" From 8017afb101b214635dcd1b372afcd21379c340f5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Feb 2014 10:40:23 -0800 Subject: [PATCH 644/778] fix copy paste error. --- src/main/scala/catalyst/expressions/aggregates.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala index dc7072e2f5719..094233098fe49 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/src/main/scala/catalyst/expressions/aggregates.scala @@ -110,7 +110,7 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod override def toString = s"FIRST($child)" override def asPartial: SplitEvaluation = { - val partialFirst = Alias(Sum(child), "PartialFirst")() + val partialFirst = Alias(First(child), "PartialFirst")() SplitEvaluation( First(partialFirst.toAttribute), partialFirst :: Nil) From 02ff8e4462793d8f37365f44cb2f269f619d72da Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 7 Feb 2014 21:41:42 +0800 Subject: [PATCH 645/778] Correctly parse the db name and table name in a CTAS query. --- .../scala/catalyst/execution/MetastoreCatalog.scala | 8 ++++---- src/main/scala/catalyst/frontend/Hive.scala | 11 +++++++---- .../scala/catalyst/plans/logical/basicOperators.scala | 5 ++++- .../scala/catalyst/execution/HiveQueryTests.scala | 6 ++++++ 4 files changed, 21 insertions(+), 9 deletions(-) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 2a98891a40f2f..ae59f388609c8 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -47,10 +47,10 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { */ object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case InsertIntoCreatedTable(name, child) => - val (databaseName, tableName) = name.split("\\.") match { - case Array(tableOnly) => (SessionState.get.getCurrentDatabase(), tableOnly) - case Array(db, table) => (db, table) + case InsertIntoCreatedTable(db, tableName, child) => + val databaseName = db match { + case None => SessionState.get.getCurrentDatabase() + case Some(databaseName) => databaseName } val table = new Table() diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 2282dd7612a4d..704602fbb0728 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -353,12 +353,15 @@ object HiveQl { case Token("TOK_CREATETABLE", children) if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty => - val (Some(Token("TOK_TABNAME", Token(tableName, Nil) :: Nil)) :: + getClauses(Seq("TOK_TABNAME", "TOK_LIKETABLE", "TOK_QUERY"), children) match { + case (Some(Token("TOK_TABNAME", Token(tableName, Nil) :: Nil)) :: _ /* likeTable */ :: - Some(query) :: Nil) = { - getClauses(Seq("TOK_TABNAME", "TOK_LIKETABLE", "TOK_QUERY"), children) + Some(query) :: Nil) => InsertIntoCreatedTable(None, tableName, nodeToPlan(query)) + case (Some(Token("TOK_TABNAME", Token(databaseName, Nil) :: Token(tableName, Nil) :: + Nil)) :: _ /* likeTable */ :: + Some(query) :: Nil) => + InsertIntoCreatedTable(Some(databaseName), tableName, nodeToPlan(query)) } - InsertIntoCreatedTable(tableName, nodeToPlan(query)) // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. case Token("TOK_CREATETABLE", _) => NativePlaceholder diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index 34fa3b68b1e6a..5a2b7ad4a164a 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -50,7 +50,10 @@ case class InsertIntoTable( } } -case class InsertIntoCreatedTable(tableName: String, child: LogicalPlan) extends UnaryNode { +case class InsertIntoCreatedTable( + databaseName: Option[String], + tableName: String, + child: LogicalPlan) extends UnaryNode { def references = Set.empty def output = child.output } diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala index 6a4469fdf4272..22c93417c342c 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -51,6 +51,12 @@ class HiveQueryTests extends HiveComparisonTest { |SELECT * FROM createdtable """.stripMargin) + createQueryTest("create table as with db name", + """ + |CREATE TABLE default.createdtable AS SELECT * FROM src; + |SELECT * FROM default.createdtable + """.stripMargin) + createQueryTest("transform", "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src") From 8841eb888d16edbb1bd34175ee13b664468e78b7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 14:01:51 -0800 Subject: [PATCH 646/778] Rename Transform -> ScriptTransformation. --- .../scala/catalyst/analysis/Analyzer.scala | 2 +- .../execution/PlanningStrategies.scala | 2 + .../execution/ScriptTransformation.scala | 57 +++++++++++++++++++ src/main/scala/catalyst/frontend/Hive.scala | 3 +- ...sform.scala => ScriptTransformation.scala} | 2 +- 5 files changed, 63 insertions(+), 3 deletions(-) create mode 100644 src/main/scala/catalyst/execution/ScriptTransformation.scala rename src/main/scala/catalyst/plans/logical/{Transform.scala => ScriptTransformation.scala} (94%) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 05c2fbdaf7cf8..4a0bb95146d55 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -127,7 +127,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case o => o :: Nil }, child) - case t: Transform if containsStar(t.input) => + case t: ScriptTransformation if containsStar(t.input) => t.copy( input = t.input.flatMap { case s: Star => s.expand(t.child.output) diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index 4d8723d90c13e..5f663c0bc472d 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -222,6 +222,8 @@ trait PlanningStrategies { execution.Union(unionChildren.map(planLater))(sc) :: Nil case logical.Transform(input, script, output, child) => execution.Transform(input, script, output, planLater(child))(sc) :: Nil + case logical.ScriptTransformation(input, script, output, child) => + execution.ScriptTransformation(input, script, output, planLater(child))(sc) :: Nil case _ => Nil } } diff --git a/src/main/scala/catalyst/execution/ScriptTransformation.scala b/src/main/scala/catalyst/execution/ScriptTransformation.scala new file mode 100644 index 0000000000000..911e0aaeaba0f --- /dev/null +++ b/src/main/scala/catalyst/execution/ScriptTransformation.scala @@ -0,0 +1,57 @@ +package catalyst +package execution + +import java.io.{InputStreamReader, BufferedReader} + +import catalyst.expressions._ + +import scala.collection.JavaConversions._ + +/** + * Transforms the input by forking and running the specified script. + * + * @param input the set of expression that should be passed to the script. + * @param script the command that should be executed. + * @param output the attributes that are produced by the script. + */ +case class ScriptTransformation( + input: Seq[Expression], + script: String, + output: Seq[Attribute], + child: SharkPlan)(@transient sc: SharkContext) + extends UnaryNode { + + override def otherCopyArgs = sc :: Nil + + def execute() = { + child.execute().mapPartitions { iter => + val cmd = List("/bin/bash", "-c", script) + val builder = new ProcessBuilder(cmd) + val proc = builder.start() + val inputStream = proc.getInputStream + val outputStream = proc.getOutputStream + val reader = new BufferedReader(new InputStreamReader(inputStream)) + + // TODO: This should be exposed as an iterator instead of reading in all the data at once. + val outputLines = collection.mutable.ArrayBuffer[Row]() + val readerThread = new Thread("Transform OutputReader") { + override def run() { + var curLine = reader.readLine() + while (curLine != null) { + // TODO: Use SerDe + outputLines += new GenericRow(curLine.split("\t")) + curLine = reader.readLine() + } + } + } + readerThread.start() + iter + .map(row => input.map(Evaluate(_, Vector(row)))) + // TODO: Use SerDe + .map(_.mkString("", "\t", "\n").getBytes).foreach(outputStream.write) + outputStream.close() + readerThread.join() + outputLines.toIterator + } + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 2282dd7612a4d..ebabf4caf8b53 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -429,7 +429,8 @@ object HiveQl { } val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) - Some(Transform(inputExprs.map(nodeToExpr), unescapedScript, output, withWhere)) + Some( + ScriptTransformation(inputExprs.map(nodeToExpr), unescapedScript, output, withWhere)) case _ => None } diff --git a/src/main/scala/catalyst/plans/logical/Transform.scala b/src/main/scala/catalyst/plans/logical/ScriptTransformation.scala similarity index 94% rename from src/main/scala/catalyst/plans/logical/Transform.scala rename to src/main/scala/catalyst/plans/logical/ScriptTransformation.scala index 50eac7b3be664..f69d81d2991f2 100644 --- a/src/main/scala/catalyst/plans/logical/Transform.scala +++ b/src/main/scala/catalyst/plans/logical/ScriptTransformation.scala @@ -11,7 +11,7 @@ import expressions._ * @param script the command that should be executed. * @param output the attributes that are produced by the script. */ -case class Transform( +case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], From acb956646de2a05475ff5086b5967e0e657f8aa0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 14:03:56 -0800 Subject: [PATCH 647/778] Correctly type attributes of CTAS. --- .../catalyst/execution/MetastoreCatalog.scala | 29 ++++++++++++++++--- 1 file changed, 25 insertions(+), 4 deletions(-) diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index 2a98891a40f2f..eec287cd2dd00 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -1,6 +1,8 @@ package catalyst package execution +import scala.util.parsing.combinator.RegexParsers + import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.HiveMetaStoreClient import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition, Table} @@ -14,10 +16,9 @@ import analysis.Catalog import expressions._ import plans.logical._ import rules._ -import types._ +import catalyst.types._ -import collection.JavaConversions._ -import scala.util.parsing.combinator.RegexParsers +import scala.collection.JavaConversions._ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { val client = new HiveMetaStoreClient(hiveConf) @@ -46,6 +47,8 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { * For example, because of a CREATE TABLE X AS statement. */ object CreateTables extends Rule[LogicalPlan] { + import HiveMetastoreTypes._ + def apply(plan: LogicalPlan): LogicalPlan = plan transform { case InsertIntoCreatedTable(name, child) => val (databaseName, tableName) = name.split("\\.") match { @@ -54,7 +57,8 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { } val table = new Table() - val schema = child.output.map(attr => new FieldSchema(attr.name, "string", "")) + val schema = + child.output.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), "")) table.setDbName(databaseName) table.setTableName(tableName) @@ -118,6 +122,23 @@ object HiveMetastoreTypes extends RegexParsers { case Success(result, _) => result case failure: NoSuccess => sys.error(s"Unsupported dataType: $metastoreType") } + + def toMetastoreType(dt: DataType): String = dt match { + case ArrayType(elementType) => s"array<${toMetastoreType(elementType)}>" + case StructType(fields) => + s"struct<${fields.map(f => s"${f.name}:${toMetastoreType(f.dataType)}").mkString(",")}>" + case MapType(keyType, valueType) => + s"map<${toMetastoreType(keyType)},${toMetastoreType(valueType)}>" + case StringType => "string" + case FloatType => "float" + case IntegerType => "int" + case ShortType =>"tinyint" + case DoubleType => "double" + case LongType => "bigint" + case BinaryType => "binary" + case BooleanType => "boolean" + case DecimalType => "decimal" + } } case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String]) From 016b48990ef37b32d1bd4b1d4790afbe15e7db57 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 14:04:17 -0800 Subject: [PATCH 648/778] fix typo. --- src/main/scala/catalyst/execution/PlanningStrategies.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index 5f663c0bc472d..0a8d6d0178d07 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -203,7 +203,7 @@ trait PlanningStrategies { // This sort is a global sort. Its requiredDistribution will be an OrderedDistribution. execution.Sort(sortExprs, global = true, planLater(child)):: Nil case logical.SortPartitions(sortExprs, child) => - // This sort only sort tuples within a partition. Its requiredDistribution will be + // This sort only sorts tuples within a partition. Its requiredDistribution will be // an UnspecifiedDistribution. execution.Sort(sortExprs, global = false, planLater(child)) :: Nil case logical.Project(projectList, child) => From bea4b7f1c3b091386bb8cacad8f8c2e154c579b7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 14:04:40 -0800 Subject: [PATCH 649/778] Add SumDistinct. --- .../scala/catalyst/execution/aggregates.scala | 20 +++++++++++++++++++ .../catalyst/expressions/aggregates.scala | 8 ++++++++ src/main/scala/catalyst/frontend/Hive.scala | 1 + 3 files changed, 29 insertions(+) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 735683524cea2..e33e7fab2d5bc 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -6,6 +6,7 @@ import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFEvaluator, AbstractGene import catalyst.errors._ import catalyst.expressions._ import catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples} +import catalyst.types._ /* Implicits */ import org.apache.spark.rdd.SharkPairRDDFunctions._ @@ -47,6 +48,7 @@ case class Aggregate( val impl = agg transform { case base @ Average(expr) => new AverageFunction(expr, base) case base @ Sum(expr) => new SumFunction(expr, base) + case base @ SumDistinct(expr) => new SumDistinctFunction(expr, base) case base @ Count(expr) => new CountFunction(expr, base) // TODO: Create custom query plan node that calculates distinct values efficiently. case base @ CountDistinct(expr) => new CountDistinctFunction(expr, base) @@ -181,6 +183,24 @@ case class SumFunction(expr: Expression, base: AggregateExpression) extends Aggr def result: Any = sum } +case class SumDistinctFunction(expr: Expression, base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new scala.collection.mutable.HashSet[Any]() + + def apply(input: Seq[Row]): Unit = { + val evaluatedExpr = Evaluate(expr, input) + if (evaluatedExpr != null) { + seen += evaluatedExpr + } + } + + def result: Any = + seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus) +} + case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) extends AggregateFunction { diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala index 094233098fe49..bba5b6d451276 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/src/main/scala/catalyst/expressions/aggregates.scala @@ -103,6 +103,14 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ } } +case class SumDistinct(child: Expression) + extends AggregateExpression with trees.UnaryNode[Expression] { + def references = child.references + def nullable = false + def dataType = child.dataType + override def toString = s"SUM(DISTINCT $child)" +} + case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { def references = child.references def nullable = child.nullable diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index ebabf4caf8b53..e4c62d127ded6 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -702,6 +702,7 @@ object HiveQl { case Token("TOK_FUNCTIONSTAR", Token(COUNT(), Nil) :: Nil) => Count(Literal(1)) case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) => CountDistinct(args.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(SUM(), Nil) :: arg :: Nil) => Sum(nodeToExpr(arg)) + case Token("TOK_FUNCTIONDI", Token(SUM(), Nil) :: arg :: Nil) => SumDistinct(nodeToExpr(arg)) /* Casts */ case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => From ea76cf9bf5e07dfa5435fa99ae1e0623a7c89262 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 14:05:13 -0800 Subject: [PATCH 650/778] Add NoRelation to planner. --- src/main/scala/catalyst/execution/PlanningStrategies.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index 0a8d6d0178d07..22d981f1a979d 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -224,6 +224,7 @@ trait PlanningStrategies { execution.Transform(input, script, output, planLater(child))(sc) :: Nil case logical.ScriptTransformation(input, script, output, child) => execution.ScriptTransformation(input, script, output, planLater(child))(sc) :: Nil + case logical.NoRelation => execution.LocalRelation(Nil, Seq(IndexedSeq()))(sc) :: Nil case _ => Nil } } From dd00b7e8df7356be40379ec560f2f476f74e1a8e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 14:11:33 -0800 Subject: [PATCH 651/778] initial implementation of generators. --- .../scala/catalyst/analysis/Analyzer.scala | 12 ++ .../catalyst/execution/FunctionRegistry.scala | 97 +++++++++++++--- .../scala/catalyst/execution/Generate.scala | 47 ++++++++ .../execution/PlanningStrategies.scala | 4 +- .../scala/catalyst/execution/Transform.scala | 55 --------- .../catalyst/expressions/generators.scala | 105 ++++++++++++++++++ src/main/scala/catalyst/frontend/Hive.scala | 66 ++++++++++- .../plans/logical/basicOperators.scala | 27 +++++ .../execution/HiveCompatibility.scala | 7 +- .../catalyst/execution/HiveQueryTests.scala | 24 ++++ 10 files changed, 366 insertions(+), 78 deletions(-) create mode 100644 src/main/scala/catalyst/execution/Generate.scala delete mode 100644 src/main/scala/catalyst/execution/Transform.scala create mode 100644 src/main/scala/catalyst/expressions/generators.scala diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 4a0bb95146d55..2e5d455fe7ac4 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -32,6 +32,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool Batch("Resolution", fixedPoint, ResolveReferences :: ResolveRelations :: + ImplicitGenerate :: StarExpansion :: ResolveFunctions :: GlobalAggregates :: @@ -112,6 +113,17 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } } + /** + * When a SELECT clause has only a single expression and that expression is a [[Generator]] + * we convert the [[Project]] to a [[Generate]]. + */ + object ImplicitGenerate extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case Project(Seq(Alias(g: Generator, _)), child) => + Generate(g, join = false, outer = false, None, child) + } + } + /** * Expands any references to [[Star]] (*) in project operators. */ diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index 6bcb04da3991f..2452883dc0815 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -2,14 +2,14 @@ package catalyst package execution import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} -import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFEvaluator, AbstractGenericUDAFResolver} -import org.apache.hadoop.hive.ql.udf.generic.GenericUDF +import org.apache.hadoop.hive.ql.udf.generic._ import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.{io => hadoopIo} @@ -38,6 +38,9 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionF } else if ( classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdaf(name, children) + + } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveGenericUdtf(name, Nil, children) } else { sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") } @@ -152,7 +155,7 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } else { constructor.newInstance(a match { case i: Int => i: java.lang.Integer - case bd: BigDecimal => new HiveDecimal(bd.underlying) + case bd: BigDecimal => new HiveDecimal(bd.underlying()) case other: AnyRef => other }).asInstanceOf[AnyRef] } @@ -185,8 +188,19 @@ case class HiveGenericUdf( val dataType: DataType = inspectorToDataType(returnInspector) + def evaluate(evaluatedChildren: Seq[Any]): Any = { + returnInspector // Make sure initialized. + val args = evaluatedChildren.map(wrap).map { v => + new DeferredJavaObject(v): DeferredObject + }.toArray + unwrap(function.evaluate(args)) + } +} + +trait HiveInspectors { + /** Converts native catalyst types to the types expected by Hive */ - def wrap(a: Any): Any = a match { + def wrap(a: Any): AnyRef = a match { case s: String => new hadoopIo.Text(s) case i: Int => i: java.lang.Integer case b: Boolean => b: java.lang.Boolean @@ -200,16 +214,6 @@ case class HiveGenericUdf( case null => null } - def evaluate(evaluatedChildren: Seq[Any]): Any = { - returnInspector // Make sure initialized. - val args = evaluatedChildren.map(wrap).map { v => - new DeferredJavaObject(v): DeferredObject - }.toArray - unwrap(function.evaluate(args)) - } -} - -trait HiveInspectors { def toInspector(dataType: DataType): ObjectInspector = dataType match { case ArrayType(tpe) => ObjectInspectorFactory.getStandardListObjectInspector(toInspector(tpe)) case MapType(keyType, valueType) => @@ -279,3 +283,68 @@ case class HiveGenericUdaf( override def toString = s"$nodeName#$name(${children.mkString(",")})" } + +case class HiveGenericUdtf( + name: String, + aliasNames: Seq[String], + children: Seq[Expression]) + extends Generator with HiveInspectors with HiveFunctionFactory { + + @transient + lazy val function = createFunction[GenericUDTF](name) + + lazy val inputInspectors = children.map(_.dataType).map(toInspector) + + lazy val outputInspectors = { + val structInspector = function.initialize(inputInspectors.toArray) + structInspector.getAllStructFieldRefs.map(_.getFieldObjectInspector) + } + + lazy val outputDataTypes = outputInspectors.map(inspectorToDataType) + + override def references = children.flatMap(_.references).toSet + + protected def makeOutput = { + // Use column names when given, otherwise c_1, c_2, ... c_n. + if (aliasNames.size == outputDataTypes.size) { + aliasNames.zip(outputDataTypes).map { + case (attrName, attrDataType) => + AttributeReference(attrName, attrDataType, nullable = true)() + } + } else { + outputDataTypes.zipWithIndex.map { + case (attrDataType, i) => + AttributeReference(s"c_$i", attrDataType, nullable = true)() + } + } + } + + def apply(input: Row): TraversableOnce[Row] = { + outputInspectors // Make sure initialized. + val collector = new UDTFCollector + function.setCollector(collector) + + val udtInput = children.map(Evaluate(_, Vector(input))).map(wrap).toArray + function.process(udtInput) + collector.collectRows() + } + + protected class UDTFCollector extends Collector { + var collected = new ArrayBuffer[Row] + + override def collect(input: java.lang.Object) { + // We need to clone the input here because implementations of + // GenericUDTF reuse the same object. Luckily they are always an array, so + // it is easy to clone. + collected += new GenericRow(input.asInstanceOf[Array[_]].map(unwrap)) + } + + def collectRows() = { + val toCollect = collected + collected = new ArrayBuffer[Row] + toCollect + } + } + + override def toString() = s"$nodeName#$name(${children.mkString(",")})" +} diff --git a/src/main/scala/catalyst/execution/Generate.scala b/src/main/scala/catalyst/execution/Generate.scala new file mode 100644 index 0000000000000..578bee48919d0 --- /dev/null +++ b/src/main/scala/catalyst/execution/Generate.scala @@ -0,0 +1,47 @@ +package catalyst +package execution + +import catalyst.expressions._ +import catalyst.types._ + +/** + * Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the + * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional + * programming with one important additional feature. When `join` is true, each output row is + * implicitly joined with the input tuple that produced it. Additionally, when `outer` is true, + * each input row will be output at least once, even if the output of the given `generator` is + * empty. + */ +case class Generate( + generator: Generator, + join: Boolean, + outer: Boolean, + child: SharkPlan) + extends UnaryNode { + + def output = + if(join) child.output ++ generator.output else generator.output + + def execute() = + if (join) { + val outerNulls = Seq.fill(generator.output.size)(null) + child.execute().mapPartitions { iter => + iter.flatMap {row => + val outputRows = generator(row) + if (outer && outputRows.isEmpty) { + new GenericRow(row ++ outerNulls) :: Nil + } else { + outputRows.map(or => new GenericRow(row ++ or)) + } + } + } + } else { + child.execute().mapPartitions(iter => iter.flatMap(generator)) + } + +} + + + + + diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index 22d981f1a979d..926bde8a9f70c 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -220,8 +220,8 @@ trait PlanningStrategies { execution.StopAfter(Evaluate(limit, Nil).asInstanceOf[Int], planLater(child))(sc) :: Nil case Unions(unionChildren) => execution.Union(unionChildren.map(planLater))(sc) :: Nil - case logical.Transform(input, script, output, child) => - execution.Transform(input, script, output, planLater(child))(sc) :: Nil + case logical.Generate(generator, join, outer, _, child) => + execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil case logical.ScriptTransformation(input, script, output, child) => execution.ScriptTransformation(input, script, output, planLater(child))(sc) :: Nil case logical.NoRelation => execution.LocalRelation(Nil, Seq(IndexedSeq()))(sc) :: Nil diff --git a/src/main/scala/catalyst/execution/Transform.scala b/src/main/scala/catalyst/execution/Transform.scala deleted file mode 100644 index 3387716f27627..0000000000000 --- a/src/main/scala/catalyst/execution/Transform.scala +++ /dev/null @@ -1,55 +0,0 @@ -package catalyst -package execution - -import java.io.{InputStreamReader, BufferedReader} - -import scala.collection.JavaConversions._ - -import catalyst.expressions._ - - -/** - * Transforms the input by forking and running the specified script. - * - * @param input the set of expression that should be passed to the script. - * @param script the command that should be executed. - * @param output the attributes that are produced by the script. - */ -case class Transform( - input: Seq[Expression], - script: String, - output: Seq[Attribute], - child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { - override def otherCopyArgs = sc :: Nil - - def execute() = { - child.execute().mapPartitions { partition => - val cmd = List("/bin/bash", "-c", script) - val builder = new ProcessBuilder(cmd) - val proc = builder.start() - val inputStream = proc.getInputStream - val outputStream = proc.getOutputStream - val reader = new BufferedReader(new InputStreamReader(inputStream)) - - // TODO: This should be exposed as an iterator instead of reading in all the data - // at once for a partition. - val outputLines = collection.mutable.ArrayBuffer[Row]() - val readerThread = new Thread("Transform OutoutReader") { - override def run() { - var curLine = reader.readLine() - while (curLine != null) { - outputLines += buildRow(curLine.split("\t")) - curLine = reader.readLine() - } - } - } - readerThread.start() - partition - .map(row => input.map(Evaluate(_, Vector(row)))) - .map(_.mkString("", "\t", "\n").getBytes).foreach(outputStream.write) - outputStream.close() - readerThread.join() - outputLines.toIterator - } - } -} diff --git a/src/main/scala/catalyst/expressions/generators.scala b/src/main/scala/catalyst/expressions/generators.scala new file mode 100644 index 0000000000000..272e64d98b711 --- /dev/null +++ b/src/main/scala/catalyst/expressions/generators.scala @@ -0,0 +1,105 @@ +package catalyst +package expressions + +import catalyst.types._ + +import dsl._ + +/** + * An expression that produces zero or more rows given a single input row. + * + * Generators produce multiple output rows instead of a single value like other expressions, + * and thus they must have a schema to associate with the rows that are output. + * + * However, unlike row producing relational operators, which are either leaves or determine their + * output schema functionally from their input, generators can contain other expressions that + * might result in their modification by rules. This structure means that they might be copied + * multiple times after first determining their output schema. If a new output schema is created for + * each copy references up the tree might be rendered invalid. As a result generators must + * instead define a function `makeOutput` which is called only once when the schema is first + * requested. The attributes produced by this function will be automatically copied anytime rules + * result in changes to the Generator or its children. + */ +abstract class Generator extends Expression with (Row => TraversableOnce[Row]) { + self: Product => + + lazy val dataType = + ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable)))) + + def nullable = false + + def references = children.flatMap(_.references).toSet + + /** + * Should be overridden by specific generators. Called only once for each instance to ensure + * that rule application does not change the output schema of a generator. + */ + protected def makeOutput(): Seq[Attribute] + + private var _output: Seq[Attribute] = null + + def output: Seq[Attribute] = { + if (output == null) { + _output = makeOutput() + } + _output + } + + /** Should be implemented by child classes to perform specific Generators. */ + def apply(input: Row): TraversableOnce[Row] + + /** Overridden `makeCopy` also copies the attributes that are produced by this generator. */ + override def makeCopy(newArgs: Array[AnyRef]): this.type = { + val copy = super.makeCopy(newArgs) + copy._output = _output + copy + } +} + +/** + * Given an input array produces a sequence of rows for each value in the array. + */ +case class Explode(attributeName: String, child: Expression) + extends Generator with trees.UnaryNode[Expression] { + + override lazy val resolved = child.resolved && child.dataType.isInstanceOf[ArrayType] + + lazy val elementType = child.dataType match { + case ArrayType(et) => et + } + + protected def makeOutput() = + AttributeReference(attributeName, elementType, nullable = true)() :: Nil + + def apply(input: Row): TraversableOnce[Row] = { + val inputArray = Evaluate(child, Vector(input)).asInstanceOf[Seq[Any]] + if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Vector(v))) + } + + override def toString() = s"explode($child)" +} + +/** + * This is an example TGF that uses UnresolvedAttributes 'name and 'age to access specific columns + * from the input data. These will be replaced during analysis with specific input attributes and + * then bound to specific ordinals during query planning. While TGFs could also access specific + * columns using hand-coded ordinals, doing so violates data independence. + * + * Note: this is only a rough example of how TGFs can be expressed, the final version will likely + * involve a lot more sugar for cleaner use in Scala/Java/etc. + */ +case class ExampleTGF(input: Seq[Attribute] = Seq('name, 'age)) extends Generator { + def children = input + protected def makeOutput() = 'nameAndAge.string :: Nil + + val Seq(nameAttr, ageAttr) = input + + def apply(input: Row): TraversableOnce[Row] = { + val name = Evaluate(nameAttr, Vector(input)) + val age = Evaluate(ageAttr, Vector(input)).asInstanceOf[Int] + + Iterator( + new GenericRow(Vector(s"$name is $age years old")), + new GenericRow(Vector(s"Next year, $name will be ${age + 1} years old"))) + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index e4c62d127ded6..e50c0d61a097e 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -12,6 +12,7 @@ import expressions._ import plans._ import plans.logical._ import types._ +import catalyst.execution.HiveGenericUdtf /** * A logical node that represents a non-query command to be executed by the system. For example, @@ -380,7 +381,8 @@ object HiveQl { sortByClause :: clusterByClause :: distributeByClause :: - limitClause :: Nil) = { + limitClause :: + lateralViewClause :: Nil) = { getClauses( Seq( "TOK_INSERT_INTO", @@ -393,7 +395,8 @@ object HiveQl { "TOK_SORTBY", "TOK_CLUSTERBY", "TOK_DISTRIBUTEBY", - "TOK_LIMIT"), + "TOK_LIMIT", + "TOK_LATERAL_VIEW"), singleInsert) } @@ -434,6 +437,22 @@ object HiveQl { case _ => None } + val withLateralView = lateralViewClause.map { lv => + val Token("TOK_SELECT", + Token("TOK_SELEXPR", clauses) :: Nil) = lv.getChildren.head + + val alias = + getClause("TOK_TABALIAS", clauses).getChildren.head.asInstanceOf[ASTNode].getText + + Generate( + nodesToGenerator(clauses), + join = true, + outer = false, + Some(alias.toLowerCase), + withWhere) + }.getOrElse(withWhere) + + // The projection of the query can either be a normal projection, an aggregation // (if there is a group by) or a script transformation. val withProject = transformation.getOrElse { @@ -442,9 +461,9 @@ object HiveQl { groupByClause match { case Some(groupBy) => - Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withWhere) + Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withLateralView) case None => - Project(selectExpressions, withWhere) + Project(selectExpressions, withLateralView) } } @@ -494,11 +513,25 @@ object HiveQl { } val allJoinTokens = "(TOK_.*JOIN)".r + val laterViewToken = "TOK_LATERAL_VIEW(.*)".r def nodeToRelation(node: Node): LogicalPlan = node match { case Token("TOK_SUBQUERY", query :: Token(alias, Nil) :: Nil) => Subquery(alias, nodeToPlan(query)) + case Token(laterViewToken(isOuter), selectClause :: relationClause :: Nil) => + val Token("TOK_SELECT", + Token("TOK_SELEXPR", clauses) :: Nil) = selectClause + + val alias = getClause("TOK_TABALIAS", clauses).getChildren.head.asInstanceOf[ASTNode].getText + + Generate( + nodesToGenerator(clauses), + join = true, + outer = isOuter.nonEmpty, + Some(alias.toLowerCase), + nodeToRelation(relationClause)) + /* All relations, possibly with aliases or sampling clauses. */ case Token("TOK_TABREF", clauses) => // If the last clause is not a token then it's the alias of the table. @@ -826,6 +859,31 @@ object HiveQl { """.stripMargin) } + + val explode = "(?i)explode".r + def nodesToGenerator(nodes: Seq[Node]): Generator = { + val function = nodes.head + + val attributes = nodes.flatMap { + case Token(a, Nil) => a.toLowerCase :: Nil + case _ => Nil + } + + function match { + case Token("TOK_FUNCTION", Token(explode(), Nil) :: child :: Nil) => + Explode(attributes.headOption.getOrElse("c0"), nodeToExpr(child)) + + case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => + HiveGenericUdtf(functionName, attributes, children.map(nodeToExpr)) + + case a: ASTNode => + throw new NotImplementedError( + s"""No parse rules for ASTNode type: ${a.getType}, text: ${a.getText}, tree: + |${dumpTree(a).toString} + """.stripMargin) + } + } + def dumpTree(node: Node, builder: StringBuilder = new StringBuilder, indent: Int = 0) : StringBuilder = { node match { diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index 34fa3b68b1e6a..b31950250330f 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -9,6 +9,33 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend def references = projectList.flatMap(_.references).toSet } +/** + * Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the + * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional + * programming with one important additional feature. When `join` is true, each output row is + * implicitly joined with the input tuple that produced it. Additionally, when `outer` is true, + * each input row will be output at least once, even if the output of the given `generator` is + * empty. The schema of the transformation is optionally given a qualifier of `alias`. + */ +case class Generate( + generator: Generator, + join: Boolean, + outer: Boolean, + alias: Option[String], + child: LogicalPlan) extends UnaryNode { + + protected def generatorOutput = + alias + .map(a => generator.output.map(_.withQualifiers(a :: Nil))) + .getOrElse(generator.output) + + def output = + if(join) child.output ++ generatorOutput else generatorOutput + + def references = + if (join) child.outputSet else generator.references +} + case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { def output = child.output def references = condition.references diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index 53ab694c16b97..b3d591df67f29 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -125,9 +125,6 @@ class HiveCompatibility extends HiveQueryFileTest { // No window support yet ".*window.*", - // Views are not supported - ".*view.*", - // Fails in hive with authorization errors. "alter_rename_partition_authorization", "authorization.*", @@ -381,6 +378,9 @@ class HiveCompatibility extends HiveQueryFileTest { "join_reorder3", "join_reorder4", "join_star", + "lateral_view_cp", + "lateral_view_outer", + "lateral_view_ppd", "lineage1", "literal_double", "literal_ints", @@ -664,6 +664,7 @@ class HiveCompatibility extends HiveQueryFileTest { "union7", "union8", "union9", + "union_lateralview", "union_ppr", "union_remove_3", "union_remove_6", diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala index 6a4469fdf4272..22eb7ae3b0914 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -64,4 +64,28 @@ class HiveQueryTests extends HiveComparisonTest { createQueryTest("empty aggregate input", "SELECT SUM(key) FROM (SELECT * FROM src LIMIT 0) a") } + + createQueryTest("lateral view1", + "SELECT tbl.* FROM src LATERAL VIEW explode(array(1,2)) tbl as a") + + createQueryTest("lateral view2", + "SELECT * FROM src LATERAL VIEW explode(array(1,2)) tbl") + + + createQueryTest("lateral view3", + "FROM src SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX") + + createQueryTest("lateral view4", + """ + |create table src_lv1 (key string, value string); + |create table src_lv2 (key string, value string); + | + |FROM src + |insert overwrite table src_lv1 SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX + |insert overwrite table src_lv2 SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX + """.stripMargin) + + createQueryTest("lateral view5", + "FROM src SELECT explode(array(key+3, key+4))") + } \ No newline at end of file From 0ce61b0f3d110567693bb340df6f5bdd6ee41a2c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 14:44:19 -0800 Subject: [PATCH 652/778] Docs for GenericHiveUdtf. --- .../scala/catalyst/execution/FunctionRegistry.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index 2452883dc0815..62070ed15daa1 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -284,6 +284,17 @@ case class HiveGenericUdaf( override def toString = s"$nodeName#$name(${children.mkString(",")})" } +/** + * Converts a Hive Generic User Defined Table Generating Function (UDTF) to a + * [[catalyst.expressions.Generator Generator]]. Note that the semantics of Generators do not allow + * Generators to to maintain state in between input rows. Thus UDTFs that rely on partitioning + * dependent operations like calls to `close()` before producing output will not operate the same as + * in Hive. However, in practice this should not affect compatibility for most sane UDTFs + * (e.g. explode). + * + * Operators that require maintaining state in between input rows should instead be implemented as + * user defined aggregations, which have clean semantics even in a partitioned execution. + */ case class HiveGenericUdtf( name: String, aliasNames: Seq[String], From 740febb71c94e40f436cb3ea5ebc81b0cda4db26 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 14:44:33 -0800 Subject: [PATCH 653/778] Tests for tgfs. --- src/main/scala/catalyst/dsl/package.scala | 7 ++ .../catalyst/expressions/generators.scala | 29 +------- .../catalyst/execution/DslQueryTests.scala | 71 +++++++++---------- .../scala/catalyst/execution/TgfSuite.scala | 52 ++++++++++++++ 4 files changed, 94 insertions(+), 65 deletions(-) create mode 100644 src/test/scala/catalyst/execution/TgfSuite.scala diff --git a/src/main/scala/catalyst/dsl/package.scala b/src/main/scala/catalyst/dsl/package.scala index 5c073008fb73a..68bdeb16b65fc 100644 --- a/src/main/scala/catalyst/dsl/package.scala +++ b/src/main/scala/catalyst/dsl/package.scala @@ -130,6 +130,13 @@ package object dsl { seed: Int = (math.random * 1000).toInt) = Sample(fraction, withReplacement, seed, plan) + def generate( + generator: Generator, + join: Boolean = false, + outer: Boolean = false, + alias: Option[String] = None) = + Generate(generator, join, outer, None, plan) + def analyze = analysis.SimpleAnalyzer(plan) } } diff --git a/src/main/scala/catalyst/expressions/generators.scala b/src/main/scala/catalyst/expressions/generators.scala index 272e64d98b711..5ae4298e5e9df 100644 --- a/src/main/scala/catalyst/expressions/generators.scala +++ b/src/main/scala/catalyst/expressions/generators.scala @@ -3,8 +3,6 @@ package expressions import catalyst.types._ -import dsl._ - /** * An expression that produces zero or more rows given a single input row. * @@ -39,7 +37,7 @@ abstract class Generator extends Expression with (Row => TraversableOnce[Row]) { private var _output: Seq[Attribute] = null def output: Seq[Attribute] = { - if (output == null) { + if (_output == null) { _output = makeOutput() } _output @@ -77,29 +75,4 @@ case class Explode(attributeName: String, child: Expression) } override def toString() = s"explode($child)" -} - -/** - * This is an example TGF that uses UnresolvedAttributes 'name and 'age to access specific columns - * from the input data. These will be replaced during analysis with specific input attributes and - * then bound to specific ordinals during query planning. While TGFs could also access specific - * columns using hand-coded ordinals, doing so violates data independence. - * - * Note: this is only a rough example of how TGFs can be expressed, the final version will likely - * involve a lot more sugar for cleaner use in Scala/Java/etc. - */ -case class ExampleTGF(input: Seq[Attribute] = Seq('name, 'age)) extends Generator { - def children = input - protected def makeOutput() = 'nameAndAge.string :: Nil - - val Seq(nameAttr, ageAttr) = input - - def apply(input: Row): TraversableOnce[Row] = { - val name = Evaluate(nameAttr, Vector(input)) - val age = Evaluate(ageAttr, Vector(input)).asInstanceOf[Int] - - Iterator( - new GenericRow(Vector(s"$name is $age years old")), - new GenericRow(Vector(s"Next year, $name will be ${age + 1} years old"))) - } } \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/DslQueryTests.scala b/src/test/scala/catalyst/execution/DslQueryTests.scala index 834f4849f4947..c2e1492e5294f 100644 --- a/src/test/scala/catalyst/execution/DslQueryTests.scala +++ b/src/test/scala/catalyst/execution/DslQueryTests.scala @@ -54,14 +54,42 @@ object TestData { ) } -class DslQueryTests extends FunSuite with BeforeAndAfterAll { - override def beforeAll() { - // By clearing the port we force Spark to pick a new one. This allows us to rerun tests - // without restarting the JVM. - System.clearProperty("spark.driver.port") - System.clearProperty("spark.hostPort") +class DslQueryTest extends FunSuite { + import TestShark._ + + /** + * Runs the plan and makes sure the answer matches the expected result. + * @param plan the query to be executed + * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ]. + */ + protected def checkAnswer(plan: LogicalPlan, expectedAnswer: Any): Unit = { + val convertedAnswer = expectedAnswer match { + case s: Seq[_] if s.isEmpty => s + case s: Seq[_] if s.head.isInstanceOf[Product] && + !s.head.isInstanceOf[Seq[_]] => s.map(_.asInstanceOf[Product].productIterator.toIndexedSeq) + case s: Seq[_] => s + case singleItem => Seq(Seq(singleItem)) + } + + val isSorted = plan.collect { case s: logical.Sort => s}.nonEmpty + def prepareAnswer(answer: Seq[Any]) = if (!isSorted) answer.sortBy(_.toString) else answer + val sharkAnswer = try plan.toRdd.collect().toSeq catch { + case e: Exception => + fail( + s""" + |Exception thrown while executing query: + |$plan + |== Physical Plan == + |${plan.executedPlan} + |== Exception == + |$e + """.stripMargin) + } + assert(prepareAnswer(convertedAnswer) === prepareAnswer(sharkAnswer)) } +} +class BasicQueryTests extends DslQueryTest { import TestShark._ import TestData._ @@ -226,35 +254,4 @@ class DslQueryTests extends FunSuite with BeforeAndAfterAll { (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) } - - /** - * Runs the plan and makes sure the answer matches the expected result. - * @param plan the query to be executed - * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ]. - */ - protected def checkAnswer(plan: LogicalPlan, expectedAnswer: Any): Unit = { - val convertedAnswer = expectedAnswer match { - case s: Seq[_] if s.isEmpty => s - case s: Seq[_] if s.head.isInstanceOf[Product] && - !s.head.isInstanceOf[Seq[_]] => s.map(_.asInstanceOf[Product].productIterator.toIndexedSeq) - case s: Seq[_] => s - case singleItem => Seq(Seq(singleItem)) - } - - val isSorted = plan.collect { case s: logical.Sort => s}.nonEmpty - def prepareAnswer(answer: Seq[Any]) = if (!isSorted) answer.sortBy(_.toString) else answer - val sharkAnswer = try plan.toRdd.collect().toSeq catch { - case e: Exception => - fail( - s""" - |Exception thrown while executing query: - |$plan - |== Physical Plan == - |${plan.executedPlan} - |== Exception == - |$e - """.stripMargin) - } - assert(prepareAnswer(convertedAnswer) === prepareAnswer(sharkAnswer)) - } } \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/TgfSuite.scala b/src/test/scala/catalyst/execution/TgfSuite.scala new file mode 100644 index 0000000000000..12852cb5822c2 --- /dev/null +++ b/src/test/scala/catalyst/execution/TgfSuite.scala @@ -0,0 +1,52 @@ +package catalyst +package execution + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import analysis._ +import expressions._ +import plans._ +import plans.logical.LogicalPlan +import types._ + +import dsl._ + +/** + * This is an example TGF that uses UnresolvedAttributes 'name and 'age to access specific columns + * from the input data. These will be replaced during analysis with specific AttributeReferences + * and then bound to specific ordinals during query planning. While TGFs could also access specific + * columns using hand-coded ordinals, doing so violates data independence. + * + * Note: this is only a rough example of how TGFs can be expressed, the final version will likely + * involve a lot more sugar for cleaner use in Scala/Java/etc. + */ +case class ExampleTGF(input: Seq[Attribute] = Seq('name, 'age)) extends Generator { + def children = input + protected def makeOutput() = 'nameAndAge.string :: Nil + + val Seq(nameAttr, ageAttr) = input + + def apply(input: Row): TraversableOnce[Row] = { + val name = Evaluate(nameAttr, Vector(input)) + val age = Evaluate(ageAttr, Vector(input)).asInstanceOf[Int] + + Iterator( + new GenericRow(Vector(s"$name is $age years old")), + new GenericRow(Vector(s"Next year, $name will be ${age + 1} years old"))) + } +} + +class TgfSuite extends DslQueryTest { + val inputData = + logical.LocalRelation('name.string, 'age.int).loadData( + ("michael", 29) :: Nil + ) + + test("simple tgf example") { + checkAnswer( + inputData.generate(ExampleTGF()), + Seq( + "michael is 29 years old" :: Nil, + "Next year, michael will be 30 years old" :: Nil)) + } +} From db92adc5ff5a0712d5104aad00cad67b520070b4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 15:58:28 -0800 Subject: [PATCH 654/778] more tests passing. clean up logging. --- .../scala/catalyst/execution/SharkContext.scala | 3 ++- src/test/resources/log4j.properties | 17 +++++++++++------ .../catalyst/execution/HiveCompatibility.scala | 9 ++++++++- 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkContext.scala b/src/main/scala/catalyst/execution/SharkContext.scala index dd33b661b97fc..fd337d0895ff0 100644 --- a/src/main/scala/catalyst/execution/SharkContext.scala +++ b/src/main/scala/catalyst/execution/SharkContext.scala @@ -31,7 +31,7 @@ class SharkContext( // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. val outputBuffer = new java.io.OutputStream { var pos: Int = 0 - var buffer = new Array[Int](1024) + var buffer = new Array[Int](10240) def write(i: Int): Unit = { buffer(pos) = i pos = (pos + 1) % buffer.size @@ -49,6 +49,7 @@ class SharkContext( var line = reader.readLine() while(line != null) { stringBuilder.append(line) + stringBuilder.append("\n") line = reader.readLine() } stringBuilder.toString() diff --git a/src/test/resources/log4j.properties b/src/test/resources/log4j.properties index f4bbd6bcca5e4..5e17e3b596ba1 100644 --- a/src/test/resources/log4j.properties +++ b/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.rootLogger=DEBUG, CA, FA #Console Appender log4j.appender.CA=org.apache.log4j.ConsoleAppender log4j.appender.CA.layout=org.apache.log4j.PatternLayout -log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n log4j.appender.CA.Threshold = WARN @@ -35,8 +35,13 @@ log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n # Set the logger level of File Appender to WARN log4j.appender.FA.Threshold = INFO -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN -log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=ERROR -org.apache.hadoop.hive.serde2.lazy.LazyStruct=ERROR +# Some packages are noisy for no good reason. +log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false +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.ql.metadata.Hive=false +log4j.logger.hive.ql.metadata.Hive=OFF + diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index b3d591df67f29..98466c3156681 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -84,8 +84,9 @@ class HiveCompatibility extends HiveQueryFileTest { // http://stackoverflow.com/a/1573715 "ops_comparison", - // The skewjoin test seems to never complete on hive... + // Tests that seems to never complete on hive... "skewjoin", + "database", // These tests fail and and exit the JVM. "auto_join18_multi_distinct", @@ -155,6 +156,7 @@ class HiveCompatibility extends HiveQueryFileTest { "alter_partition_with_whitelist", "alter_table_serde", "alter_varchar2", + "alter_view_as_select", "ambiguous_col", "auto_join0", "auto_join1", @@ -217,6 +219,7 @@ class HiveCompatibility extends HiveQueryFileTest { "correlationoptimizer7", "correlationoptimizer8", "count", + "create_like_view", "create_nested_type", "create_skewed_table1", "create_struct_table", @@ -242,6 +245,7 @@ class HiveCompatibility extends HiveQueryFileTest { "drop_partitions_ignore_protection", "drop_table", "drop_table2", + "drop_view", "escape_clusterby1", "escape_distributeby1", "escape_orderby1", @@ -275,6 +279,7 @@ class HiveCompatibility extends HiveQueryFileTest { "groupby8_map", "groupby8_map_skew", "groupby8_noskew", + "groupby_distinct_samekey", "groupby_multi_single_reducer2", "groupby_mutli_insert_common_distinct", "groupby_neg_float", @@ -378,6 +383,7 @@ class HiveCompatibility extends HiveQueryFileTest { "join_reorder3", "join_reorder4", "join_star", + "join_view", "lateral_view_cp", "lateral_view_outer", "lateral_view_ppd", @@ -652,6 +658,7 @@ class HiveCompatibility extends HiveQueryFileTest { "union22", "union23", "union24", + "union26", "union27", "union28", "union29", From ff5ea3f209eed028365a2b680dd7093340e355c8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 15:59:41 -0800 Subject: [PATCH 655/778] new golden --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index b8f7010573bd1..99cd4a5a9291b 100644 --- a/build.sbt +++ b/build.sbt @@ -17,7 +17,7 @@ resolvers += "SparkStaging" at "https://repository.apache.org/content/repositori libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" -libraryDependencies += "catalyst" % "hive-golden" % "7" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden7.jar" +libraryDependencies += "catalyst" % "hive-golden" % "8" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden8.jar" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" From 5cc367cdb9946b092c53ff1473ac3f784c0112d3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 17:34:34 -0800 Subject: [PATCH 656/778] use berkeley instead of cloudbees --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 99cd4a5a9291b..f77d6d477b63a 100644 --- a/build.sbt +++ b/build.sbt @@ -17,7 +17,7 @@ resolvers += "SparkStaging" at "https://repository.apache.org/content/repositori libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" -libraryDependencies += "catalyst" % "hive-golden" % "8" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden8.jar" +libraryDependencies += "catalyst" % "hive-golden" % "8" % "test" from "http://cs.berkeley.edu/~marmbrus/tmp/catalystGolden8.jar" // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" From b376d15652bd0372d1713429468d874614a9dd7a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 17:42:32 -0800 Subject: [PATCH 657/778] fix newlines at EOF --- src/main/scala/catalyst/execution/ScriptTransformation.scala | 2 +- src/main/scala/catalyst/expressions/generators.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/ScriptTransformation.scala b/src/main/scala/catalyst/execution/ScriptTransformation.scala index 911e0aaeaba0f..946c5061a9e5c 100644 --- a/src/main/scala/catalyst/execution/ScriptTransformation.scala +++ b/src/main/scala/catalyst/execution/ScriptTransformation.scala @@ -54,4 +54,4 @@ case class ScriptTransformation( outputLines.toIterator } } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/expressions/generators.scala b/src/main/scala/catalyst/expressions/generators.scala index 5ae4298e5e9df..f81ae2c5a5d3b 100644 --- a/src/main/scala/catalyst/expressions/generators.scala +++ b/src/main/scala/catalyst/expressions/generators.scala @@ -75,4 +75,4 @@ case class Explode(attributeName: String, child: Expression) } override def toString() = s"explode($child)" -} \ No newline at end of file +} From 7123225ae5e96dc7be38b13c2f2bcc86a19249ad Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 8 Feb 2014 09:44:01 +0800 Subject: [PATCH 658/778] Correctly parse the db name and table name in INSERT queries. --- .../scala/catalyst/analysis/Analyzer.scala | 6 ++- .../scala/catalyst/analysis/Catalog.scala | 10 ++++- .../scala/catalyst/analysis/unresolved.scala | 5 ++- .../catalyst/examples/ViewsExample.scala | 6 +-- .../catalyst/execution/MetastoreCatalog.scala | 17 +++---- .../scala/catalyst/execution/TestShark.scala | 2 +- .../catalyst/execution/hiveOperators.scala | 3 +- src/main/scala/catalyst/frontend/Hive.scala | 44 ++++++++++++------- .../catalyst/execution/HiveQueryTests.scala | 15 ++++++- 9 files changed, 69 insertions(+), 39 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 05c2fbdaf7cf8..1f5a310e48842 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -44,7 +44,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool */ object ResolveRelations extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case UnresolvedRelation(name, alias) => catalog.lookupRelation(name, alias) + case UnresolvedRelation(databaseName, name, alias) => + catalog.lookupRelation(databaseName, name, alias) } } @@ -53,7 +54,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool */ object LowercaseAttributeReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case UnresolvedRelation(name, alias) => UnresolvedRelation(name, alias.map(_.toLowerCase)) + case UnresolvedRelation(databaseName, name, alias) => + UnresolvedRelation(databaseName, name, alias.map(_.toLowerCase)) case Subquery(alias, child) => Subquery(alias.toLowerCase, child) case q: LogicalPlan => q transformExpressions { case s: Star => s.copy(table = s.table.map(_.toLowerCase)) diff --git a/src/main/scala/catalyst/analysis/Catalog.scala b/src/main/scala/catalyst/analysis/Catalog.scala index 2d4e307c8921b..a1cb32feef8c3 100644 --- a/src/main/scala/catalyst/analysis/Catalog.scala +++ b/src/main/scala/catalyst/analysis/Catalog.scala @@ -7,7 +7,10 @@ import plans.logical.LogicalPlan * An interface for looking up relations by name. Used by an [[Analyzer]]. */ trait Catalog { - def lookupRelation(name: String, alias: Option[String] = None): LogicalPlan + def lookupRelation( + databaseName: Option[String], + tableName: String, + alias: Option[String] = None): LogicalPlan } /** @@ -15,7 +18,10 @@ trait Catalog { * relations are already filled in and the analyser needs only to resolve attribute references. */ object EmptyCatalog extends Catalog { - def lookupRelation(name: String, alias: Option[String] = None) = { + def lookupRelation( + databaseName: Option[String], + tableName: String, + alias: Option[String] = None) = { throw new UnsupportedOperationException } } diff --git a/src/main/scala/catalyst/analysis/unresolved.scala b/src/main/scala/catalyst/analysis/unresolved.scala index d198c6dd8ce1c..af66ac457ec16 100644 --- a/src/main/scala/catalyst/analysis/unresolved.scala +++ b/src/main/scala/catalyst/analysis/unresolved.scala @@ -15,7 +15,10 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str /** * Holds the name of a relation that has yet to be looked up in a [[Catalog]]. */ -case class UnresolvedRelation(name: String, alias: Option[String] = None) extends BaseRelation { +case class UnresolvedRelation( + databaseName: Option[String], + name: String, + alias: Option[String] = None) extends BaseRelation { def output = Nil override lazy val resolved = false } diff --git a/src/main/scala/catalyst/examples/ViewsExample.scala b/src/main/scala/catalyst/examples/ViewsExample.scala index 15156893833cd..cf50b7343a159 100644 --- a/src/main/scala/catalyst/examples/ViewsExample.scala +++ b/src/main/scala/catalyst/examples/ViewsExample.scala @@ -25,15 +25,15 @@ object ViewsExample { // Construct a plan that has UnresolvedRelations in it using the DSL. val unresolvedPlan = - UnresolvedRelation("view1") - .join(UnresolvedRelation("view2"), Inner, Some('a === 'c)) + UnresolvedRelation(None, "view1") + .join(UnresolvedRelation(None, "view2"), Inner, Some('a === 'c)) .where('c < 1) .select('a, 'c) println(s"Unresolved Plan:\n$unresolvedPlan") // Replace UnresolvedRelations with logical plans from the views map. val withRelations = unresolvedPlan transform { - case UnresolvedRelation(name, _) => views(name) + case UnresolvedRelation(_, name, _) => views(name) } println(s"With relations:\n$withRelations ") diff --git a/src/main/scala/catalyst/execution/MetastoreCatalog.scala b/src/main/scala/catalyst/execution/MetastoreCatalog.scala index ae59f388609c8..576b90e951598 100644 --- a/src/main/scala/catalyst/execution/MetastoreCatalog.scala +++ b/src/main/scala/catalyst/execution/MetastoreCatalog.scala @@ -22,11 +22,11 @@ import scala.util.parsing.combinator.RegexParsers class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { val client = new HiveMetaStoreClient(hiveConf) - def lookupRelation(name: String, alias: Option[String]): BaseRelation = { - val (databaseName, tableName) = name.split("\\.") match { - case Array(tableOnly) => (SessionState.get.getCurrentDatabase(), tableOnly) - case Array(db, table) => (db, table) - } + def lookupRelation( + db: Option[String], + tableName: String, + alias: Option[String]): BaseRelation = { + val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase()) val table = client.getTable(databaseName, tableName) val hiveQlTable = new org.apache.hadoop.hive.ql.metadata.Table(table) val partitions = @@ -48,10 +48,7 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case InsertIntoCreatedTable(db, tableName, child) => - val databaseName = db match { - case None => SessionState.get.getCurrentDatabase() - case Some(databaseName) => databaseName - } + val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase()) val table = new Table() val schema = child.output.map(attr => new FieldSchema(attr.name, "string", "")) @@ -74,7 +71,7 @@ class HiveMetastoreCatalog(hiveConf: HiveConf) extends Catalog { sd.setSerdeInfo(serDeInfo) client.createTable(table) - InsertIntoTable(lookupRelation(tableName, None), Map.empty, child) + InsertIntoTable(lookupRelation(Some(databaseName), tableName, None), Map.empty, child) } } } diff --git a/src/main/scala/catalyst/execution/TestShark.scala b/src/main/scala/catalyst/execution/TestShark.scala index 0e81c6b72c7f2..af6264cb7afb3 100644 --- a/src/main/scala/catalyst/execution/TestShark.scala +++ b/src/main/scala/catalyst/execution/TestShark.scala @@ -122,7 +122,7 @@ class TestSharkInstance extends SharkInstance { // Make sure any test tables referenced are loaded. val referencedTables = describedTables ++ - parsed.collect { case UnresolvedRelation(name, _) => name.split("\\.").last } + parsed.collect { case UnresolvedRelation(databaseName, name, _) => name } val referencedTestTables = referencedTables.filter(testTables.contains) logger.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) diff --git a/src/main/scala/catalyst/execution/hiveOperators.scala b/src/main/scala/catalyst/execution/hiveOperators.scala index 84a9bd5d8418c..492abf3fd0a76 100644 --- a/src/main/scala/catalyst/execution/hiveOperators.scala +++ b/src/main/scala/catalyst/execution/hiveOperators.scala @@ -256,7 +256,8 @@ case class InsertIntoHiveTable( } val inpath = tempDir.getCanonicalPath + "/*" - sc.runHive(s"LOAD DATA LOCAL INPATH '$inpath' INTO TABLE ${table.tableName}$partitionSpec") + sc.runHive(s"""LOAD DATA LOCAL INPATH '$inpath' INTO TABLE + ${table.databaseName}.${table.tableName}$partitionSpec""") // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 704602fbb0728..97333f00e4539 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -353,15 +353,15 @@ object HiveQl { case Token("TOK_CREATETABLE", children) if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty => - getClauses(Seq("TOK_TABNAME", "TOK_LIKETABLE", "TOK_QUERY"), children) match { - case (Some(Token("TOK_TABNAME", Token(tableName, Nil) :: Nil)) :: - _ /* likeTable */ :: - Some(query) :: Nil) => InsertIntoCreatedTable(None, tableName, nodeToPlan(query)) - case (Some(Token("TOK_TABNAME", Token(databaseName, Nil) :: Token(tableName, Nil) :: - Nil)) :: _ /* likeTable */ :: - Some(query) :: Nil) => - InsertIntoCreatedTable(Some(databaseName), tableName, nodeToPlan(query)) - } + val (Some(tableNameParts) :: _ /* likeTable */ :: Some(query) :: Nil) = + getClauses(Seq("TOK_TABNAME", "TOK_LIKETABLE", "TOK_QUERY"), children) + + val (db, tableName) = + tableNameParts.getChildren.map{ case Token(part, Nil) => cleanIdentifier(part)} match { + case Seq(tableOnly) => (None, tableOnly) + case Seq(databaseName, table) => (Some(databaseName), table) + } + InsertIntoCreatedTable(db, tableName, nodeToPlan(query)) // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. case Token("TOK_CREATETABLE", _) => NativePlaceholder @@ -518,11 +518,13 @@ object HiveQl { nonAliasClauses) } - val tableName = tableNameParts.getChildren.map { case Token(part, Nil) => - cleanIdentifier(part) - }.mkString(".") + val (db, tableName) = + tableNameParts.getChildren.map{ case Token(part, Nil) => cleanIdentifier(part)} match { + case Seq(tableOnly) => (None, tableOnly) + case Seq(databaseName, table) => (Some(databaseName), table) + } val alias = aliasClause.map { case Token(a, Nil) => cleanIdentifier(a) } - val relation = UnresolvedRelation(tableName, alias) + val relation = UnresolvedRelation(db, tableName, alias) // Apply sampling if requested. (bucketSampleClause orElse splitSampleClause).map { @@ -622,16 +624,22 @@ object HiveQl { case Token(destinationToken(), Token("TOK_TAB", tableArgs) :: Nil) => - val Some(nameClause) :: partitionClause :: Nil = + val Some(tableNameParts) :: partitionClause :: Nil = getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) - val Token("TOK_TABNAME", Token(tableName, Nil) :: Nil) = nameClause + + val (db, tableName) = + tableNameParts.getChildren.map{ case Token(part, Nil) => cleanIdentifier(part)} match { + case Seq(tableOnly) => (None, tableOnly) + case Seq(databaseName, table) => (Some(databaseName), table) + } val partitionKeys = partitionClause.map(_.getChildren.map { - case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => key -> Some(value) + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => + key -> Some(value) case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => key -> None }.toMap).getOrElse(Map.empty) - InsertIntoTable(UnresolvedRelation(tableName, None), partitionKeys, query) + InsertIntoTable(UnresolvedRelation(db, tableName, None), partitionKeys, query) case a: ASTNode => throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") @@ -695,6 +703,8 @@ object HiveQl { /* Stars (*) */ case Token("TOK_ALLCOLREF", Nil) => Star(None) + // The format of dbName.tableName.* cannot be parsed by HiveParser. TOK_TABNAME will only + // has a single child which is tableName. case Token("TOK_ALLCOLREF", Token("TOK_TABNAME", Token(name, Nil) :: Nil) :: Nil) => Star(Some(name)) diff --git a/src/test/scala/catalyst/execution/HiveQueryTests.scala b/src/test/scala/catalyst/execution/HiveQueryTests.scala index 22c93417c342c..77253d15ef76d 100644 --- a/src/test/scala/catalyst/execution/HiveQueryTests.scala +++ b/src/test/scala/catalyst/execution/HiveQueryTests.scala @@ -53,8 +53,19 @@ class HiveQueryTests extends HiveComparisonTest { createQueryTest("create table as with db name", """ - |CREATE TABLE default.createdtable AS SELECT * FROM src; - |SELECT * FROM default.createdtable + |CREATE DATABASE IF NOT EXISTS testdb; + |CREATE TABLE testdb.createdtable AS SELECT * FROM default.src; + |SELECT * FROM testdb.createdtable; + |DROP DATABASE IF EXISTS testdb CASCADE + """.stripMargin) + + createQueryTest("insert table with db name", + """ + |CREATE DATABASE IF NOT EXISTS testdb; + |CREATE TABLE testdb.createdtable like default.src; + |INSERT INTO TABLE testdb.createdtable SELECT * FROM default.src; + |SELECT * FROM testdb.createdtable; + |DROP DATABASE IF EXISTS testdb CASCADE """.stripMargin) createQueryTest("transform", From 2897deb146c498bfc7ebcb80e3835ecb9899cfeb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Feb 2014 18:31:20 -0800 Subject: [PATCH 659/778] fix scaladoc --- src/main/scala/catalyst/analysis/Analyzer.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/src/main/scala/catalyst/analysis/Analyzer.scala index 2e5d455fe7ac4..6380b7c4249a2 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/src/main/scala/catalyst/analysis/Analyzer.scala @@ -114,8 +114,9 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * When a SELECT clause has only a single expression and that expression is a [[Generator]] - * we convert the [[Project]] to a [[Generate]]. + * When a SELECT clause has only a single expression and that expression is a + * [[catalyst.expressions.Generator Generator]] we convert the + * [[catalyst.plans.logical.Project Project]] to a [[catalyst.plans.logical.Generate Generate]]. */ object ImplicitGenerate extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { From 341116cb450ff72af793a5bd84d73ca2203200cb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 8 Feb 2014 12:09:59 -0800 Subject: [PATCH 660/778] address comments. --- .../catalyst/execution/FunctionRegistry.scala | 4 +-- .../scala/catalyst/execution/Generate.scala | 23 ++++++------- .../execution/PlanningStrategies.scala | 3 +- .../catalyst/expressions/aggregates.scala | 3 +- .../catalyst/expressions/generators.scala | 32 +++++++++++++++---- src/main/scala/catalyst/frontend/Hive.scala | 2 +- .../plans/logical/basicOperators.scala | 17 ++++++---- .../execution/HiveCompatibility.scala | 1 - 8 files changed, 53 insertions(+), 32 deletions(-) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index 62070ed15daa1..eff316bd9772d 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -287,10 +287,10 @@ case class HiveGenericUdaf( /** * Converts a Hive Generic User Defined Table Generating Function (UDTF) to a * [[catalyst.expressions.Generator Generator]]. Note that the semantics of Generators do not allow - * Generators to to maintain state in between input rows. Thus UDTFs that rely on partitioning + * Generators to maintain state in between input rows. Thus UDTFs that rely on partitioning * dependent operations like calls to `close()` before producing output will not operate the same as * in Hive. However, in practice this should not affect compatibility for most sane UDTFs - * (e.g. explode). + * (e.g. explode or GenericUDTFParseUrlTuple). * * Operators that require maintaining state in between input rows should instead be implemented as * user defined aggregations, which have clean semantics even in a partitioned execution. diff --git a/src/main/scala/catalyst/execution/Generate.scala b/src/main/scala/catalyst/execution/Generate.scala index 578bee48919d0..93f6be5a66df2 100644 --- a/src/main/scala/catalyst/execution/Generate.scala +++ b/src/main/scala/catalyst/execution/Generate.scala @@ -7,10 +7,12 @@ import catalyst.types._ /** * Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional - * programming with one important additional feature. When `join` is true, each output row is - * implicitly joined with the input tuple that produced it. Additionally, when `outer` is true, - * each input row will be output at least once, even if the output of the given `generator` is - * empty. + * programming with one important additional feature, which allows the input rows to be joined with + * their output. + * @param join when true, each output row is implicitly joined with the input tuple that produced + * it. + * @param outer when true, each input row will be output at least once, even if the output of the + * given `generator` is empty. `outer` has no effect when `join` is false. */ case class Generate( generator: Generator, @@ -20,9 +22,9 @@ case class Generate( extends UnaryNode { def output = - if(join) child.output ++ generator.output else generator.output + if (join) child.output ++ generator.output else generator.output - def execute() = + def execute() = { if (join) { val outerNulls = Seq.fill(generator.output.size)(null) child.execute().mapPartitions { iter => @@ -38,10 +40,5 @@ case class Generate( } else { child.execute().mapPartitions(iter => iter.flatMap(generator)) } - -} - - - - - + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index 926bde8a9f70c..88476b1b8c438 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -224,7 +224,8 @@ trait PlanningStrategies { execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil case logical.ScriptTransformation(input, script, output, child) => execution.ScriptTransformation(input, script, output, planLater(child))(sc) :: Nil - case logical.NoRelation => execution.LocalRelation(Nil, Seq(IndexedSeq()))(sc) :: Nil + case logical.NoRelation => + execution.LocalRelation(Nil, Seq(IndexedSeq()))(sc) :: Nil case _ => Nil } } diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/src/main/scala/catalyst/expressions/aggregates.scala index bba5b6d451276..3837149095cb3 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/src/main/scala/catalyst/expressions/aggregates.scala @@ -104,7 +104,8 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ } case class SumDistinct(child: Expression) - extends AggregateExpression with trees.UnaryNode[Expression] { + extends AggregateExpression with trees.UnaryNode[Expression] { + def references = child.references def nullable = false def dataType = child.dataType diff --git a/src/main/scala/catalyst/expressions/generators.scala b/src/main/scala/catalyst/expressions/generators.scala index f81ae2c5a5d3b..c78ddf930d196 100644 --- a/src/main/scala/catalyst/expressions/generators.scala +++ b/src/main/scala/catalyst/expressions/generators.scala @@ -57,21 +57,39 @@ abstract class Generator extends Expression with (Row => TraversableOnce[Row]) { /** * Given an input array produces a sequence of rows for each value in the array. */ -case class Explode(attributeName: String, child: Expression) +case class Explode(attributeNames: Seq[String], child: Expression) extends Generator with trees.UnaryNode[Expression] { - override lazy val resolved = child.resolved && child.dataType.isInstanceOf[ArrayType] + override lazy val resolved = + child.resolved && + (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) - lazy val elementType = child.dataType match { - case ArrayType(et) => et + lazy val elementTypes = child.dataType match { + case ArrayType(et) => et :: Nil + case MapType(kt,vt) => kt :: vt :: Nil } + // TODO: Move this pattern into Generator. protected def makeOutput() = - AttributeReference(attributeName, elementType, nullable = true)() :: Nil + if(attributeNames.size == elementTypes.size) { + attributeNames.zip(elementTypes).map { + case (n, t) => AttributeReference(n, t, nullable = true)() + } + } else { + elementTypes.zipWithIndex.map { + case (t, i) => AttributeReference(s"c_$i", t, nullable = true)() + } + } def apply(input: Row): TraversableOnce[Row] = { - val inputArray = Evaluate(child, Vector(input)).asInstanceOf[Seq[Any]] - if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Vector(v))) + child.dataType match { + case ArrayType(_) => + val inputArray = Evaluate(child, Vector(input)).asInstanceOf[Seq[Any]] + if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Vector(v))) + case MapType(_, _) => + val inputMap = Evaluate(child, Vector(input)).asInstanceOf[Map[Any,Any]] + if (inputMap == null) Nil else inputMap.map { case (k,v) => new GenericRow(Vector(k,v)) } + } } override def toString() = s"explode($child)" diff --git a/src/main/scala/catalyst/frontend/Hive.scala b/src/main/scala/catalyst/frontend/Hive.scala index 0391a4b775bf7..413671c83fca7 100644 --- a/src/main/scala/catalyst/frontend/Hive.scala +++ b/src/main/scala/catalyst/frontend/Hive.scala @@ -874,7 +874,7 @@ object HiveQl { function match { case Token("TOK_FUNCTION", Token(explode(), Nil) :: child :: Nil) => - Explode(attributes.headOption.getOrElse("c0"), nodeToExpr(child)) + Explode(attributes, nodeToExpr(child)) case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => HiveGenericUdtf(functionName, attributes, children.map(nodeToExpr)) diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/src/main/scala/catalyst/plans/logical/basicOperators.scala index a47cf63c13894..bc9928f6ea385 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/src/main/scala/catalyst/plans/logical/basicOperators.scala @@ -12,17 +12,22 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend /** * Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional - * programming with one important additional feature. When `join` is true, each output row is - * implicitly joined with the input tuple that produced it. Additionally, when `outer` is true, - * each input row will be output at least once, even if the output of the given `generator` is - * empty. The schema of the transformation is optionally given a qualifier of `alias`. + * programming with one important additional feature, which allows the input rows to be joined with + * their output. + * @param join when true, each output row is implicitly joined with the input tuple that produced + * it. + * @param outer when true, each input row will be output at least once, even if the output of the + * given `generator` is empty. `outer` has no effect when `join` is false. + * @param alias when set, this string is applied to the schema of the output of the transformation + * as a qualifier. */ case class Generate( generator: Generator, join: Boolean, outer: Boolean, alias: Option[String], - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan) + extends UnaryNode { protected def generatorOutput = alias @@ -30,7 +35,7 @@ case class Generate( .getOrElse(generator.output) def output = - if(join) child.output ++ generatorOutput else generatorOutput + if (join) child.output ++ generatorOutput else generatorOutput def references = if (join) child.outputSet else generator.references diff --git a/src/test/scala/catalyst/execution/HiveCompatibility.scala b/src/test/scala/catalyst/execution/HiveCompatibility.scala index 98466c3156681..4cca463425649 100644 --- a/src/test/scala/catalyst/execution/HiveCompatibility.scala +++ b/src/test/scala/catalyst/execution/HiveCompatibility.scala @@ -385,7 +385,6 @@ class HiveCompatibility extends HiveQueryFileTest { "join_star", "join_view", "lateral_view_cp", - "lateral_view_outer", "lateral_view_ppd", "lineage1", "literal_double", From 7785ee62e47c93390213ff3f1a8a67a293d878a6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 10 Feb 2014 15:14:49 -0800 Subject: [PATCH 661/778] Tighten visibility based on comments. --- .../catalyst/execution/FunctionRegistry.scala | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index eff316bd9772d..57c8839fa371c 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -129,11 +129,13 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd type UDFType = UDF @transient - lazy val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) + protected lazy val method = + function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) + @transient lazy val dataType = javaClassToDataType(method.getReturnType) - lazy val wrappers: Array[(Any) => AnyRef] = method.getParameterTypes.map { argClass => + protected lazy val wrappers: Array[(Any) => AnyRef] = method.getParameterTypes.map { argClass => val primitiveClasses = Seq( Integer.TYPE, classOf[java.lang.Integer], classOf[java.lang.String], java.lang.Double.TYPE, classOf[java.lang.Double], java.lang.Long.TYPE, classOf[java.lang.Long], @@ -181,10 +183,10 @@ case class HiveGenericUdf( type UDFType = GenericUDF @transient - lazy val argumentInspectors: Seq[ObjectInspector] = children.map(_.dataType).map(toInspector) + protected lazy val argumentInspectors = children.map(_.dataType).map(toInspector) @transient - lazy val returnInspector = function.initialize(argumentInspectors.toArray) + protected lazy val returnInspector = function.initialize(argumentInspectors.toArray) val dataType: DataType = inspectorToDataType(returnInspector) @@ -266,14 +268,14 @@ case class HiveGenericUdaf( type UDFType = AbstractGenericUDAFResolver - lazy val resolver = createFunction[AbstractGenericUDAFResolver](name) + protected lazy val resolver: AbstractGenericUDAFResolver = createFunction(name) - lazy val objectInspector: ObjectInspector = { + protected lazy val objectInspector = { resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) } - lazy val inspectors: Seq[ObjectInspector] = children.map(_.dataType).map(toInspector) + protected lazy val inspectors = children.map(_.dataType).map(toInspector) def dataType: DataType = inspectorToDataType(objectInspector) @@ -301,21 +303,21 @@ case class HiveGenericUdtf( children: Seq[Expression]) extends Generator with HiveInspectors with HiveFunctionFactory { + override def references = children.flatMap(_.references).toSet + @transient - lazy val function = createFunction[GenericUDTF](name) + protected lazy val function: GenericUDTF = createFunction(name) - lazy val inputInspectors = children.map(_.dataType).map(toInspector) + protected lazy val inputInspectors = children.map(_.dataType).map(toInspector) - lazy val outputInspectors = { + protected lazy val outputInspectors = { val structInspector = function.initialize(inputInspectors.toArray) structInspector.getAllStructFieldRefs.map(_.getFieldObjectInspector) } - lazy val outputDataTypes = outputInspectors.map(inspectorToDataType) - - override def references = children.flatMap(_.references).toSet + protected lazy val outputDataTypes = outputInspectors.map(inspectorToDataType) - protected def makeOutput = { + override protected def makeOutput() = { // Use column names when given, otherwise c_1, c_2, ... c_n. if (aliasNames.size == outputDataTypes.size) { aliasNames.zip(outputDataTypes).map { From dce0593034a30b802d9be2cf98590e9955df1b47 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 10 Feb 2014 16:04:56 -0800 Subject: [PATCH 662/778] move golden answer to the source code directory. --- build.sbt | 2 -- .../scala/catalyst/execution/HiveComparisonTest.scala | 8 +------- 2 files changed, 1 insertion(+), 9 deletions(-) diff --git a/build.sbt b/build.sbt index f77d6d477b63a..913d7b80784f1 100644 --- a/build.sbt +++ b/build.sbt @@ -17,8 +17,6 @@ resolvers += "SparkStaging" at "https://repository.apache.org/content/repositori libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" -libraryDependencies += "catalyst" % "hive-golden" % "8" % "test" from "http://cs.berkeley.edu/~marmbrus/tmp/catalystGolden8.jar" - // Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index 46ea0c1d94d46..ae7e740b2efcb 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -59,7 +59,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G .map(name => new File(targetDir, s"$suiteName.$name")) /** The local directory with cached golden answer will be stored. */ - protected val answerCache = new File(targetDir, "comparison-test-cache") + protected val answerCache = new File("src/test/hive/golden") if (!answerCache.exists) answerCache.mkdir() @@ -213,12 +213,6 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G logger.debug(s"Looking for cached answer file $cachedAnswerFile.") if (cachedAnswerFile.exists) { Some(fileToString(cachedAnswerFile)) - } else if ( - getClass.getClassLoader.getResourceAsStream(cachedAnswerFile.toString) != null && - !recomputeCache) { - val answer = resourceToString(cachedAnswerFile.toString, classLoader = testClassLoader) - stringToFile(cachedAnswerFile, answer) - Some(answer) } else { logger.debug(s"File $cachedAnswerFile not found") None From 9329820a9a85697a9bfad11b6f7266c07eb59235 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 10 Feb 2014 16:28:23 -0800 Subject: [PATCH 663/778] add golden answer files to repository --- ...' + '1'-0-77504a9f3d712143beb52f3c25a904cb | 1 + .../'1' + 1-0-130514c6116c311d808590a075b187b | 1 + ...1' + 1.0-0-5db3b55120a19863d96460d399c2d0e | 1 + ...1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 | 1 + ...1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 | 1 + ...1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a | 1 + .../1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a | 1 + .../1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 | 1 + ...1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 | 1 + .../1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e | 1 + .../1 + 1S-0-2e99da48f67f588c9e632a57c713522e | 1 + .../1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f | 1 + ...0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 | 1 + ...1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 | 1 + ...0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d | 1 + ....0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 | 1 + ....0 + 1S-0-31fbe14d01fb532176c1689680398368 | 1 + ....0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 | 1 + ...L + '1'-0-6e39c7be301f3846efa9b4c939815b4a | 1 + .../1L + 1-0-1864a260554255a09e4f28b8551eef9d | 1 + ...L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 | 1 + ...1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f | 1 + ...1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 | 1 + ...1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 | 1 + ...S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 | 1 + .../1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 | 1 + ...S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e | 1 + ...1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 | 1 + ...1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e | 1 + ...1S + 1Y-0-e59bc8279cd364224476ffc504c7685b | 1 + ...Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb | 1 + .../1Y + 1-0-a4541db51882b19503649138fbb295f | 1 + ...Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d | 1 + ...1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 | 1 + ...1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 | 1 + ...1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 | 1 + ...ISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a | 309 ++ ...quences-0-2f25c33d97c43f3276171624d988a286 | 1 + ...Explain-0-85d398864d4aa1d0e10ffd668fdf1a59 | 0 .../LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 | 175 ++ ...arated)-0-779101eb00fd8bb9f08908ab29e90c03 | 500 ++++ ...roSerDe-0-805f15ffbb03db90ec5757b328666d04 | 8 + ...exSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa | 2 + ...h group-0-f52ca483a3e5eadc1b20ba8320d029a7 | 309 ++ ...age + 1-0-5e296b921c470f0f0b5d099f28bd5935 | 1 + ...Average-0-c197ea78c4d8f85f1a317805b6da07e5 | 1 + ...t_exist-0-e58d09864bc6898e44fae80abe328702 | 0 ...t_exist-1-d824f22606f48dfca48ce241a7505f5b | 0 ..._exist-10-ecb27eb754e731429659224b5b6ac583 | 0 ..._exist-11-9c36cac1372650b703400c60dd29042c | 2 + ..._exist-12-2900a5065dd3adbb0b56a712bf848750 | 0 ..._exist-13-d69279050a0f44c3f8b775698fd29be0 | 0 ..._exist-14-9c36cac1372650b703400c60dd29042c | 0 ..._exist-15-e58d09864bc6898e44fae80abe328702 | 0 ..._exist-16-d824f22606f48dfca48ce241a7505f5b | 0 ..._exist-17-3432935e802ae46c6b2151cc4ebf783b | 0 ..._exist-18-d824f22606f48dfca48ce241a7505f5b | 1 + ..._exist-19-83cecdbfddb070a417050f9a18fff752 | 0 ...t_exist-2-3432935e802ae46c6b2151cc4ebf783b | 0 ..._exist-20-d824f22606f48dfca48ce241a7505f5b | 1 + ..._exist-21-2df4b20f67e7f15d3f4150bcfea43b0f | 0 ..._exist-22-d824f22606f48dfca48ce241a7505f5b | 2 + ..._exist-23-a3859b63665ce3bbfd19683e6e694dcb | 0 ..._exist-24-d824f22606f48dfca48ce241a7505f5b | 3 + ...t_exist-3-d824f22606f48dfca48ce241a7505f5b | 1 + ...t_exist-4-83cecdbfddb070a417050f9a18fff752 | 0 ...t_exist-5-d824f22606f48dfca48ce241a7505f5b | 1 + ...t_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f | 0 ...t_exist-7-d824f22606f48dfca48ce241a7505f5b | 2 + ...t_exist-8-a3859b63665ce3bbfd19683e6e694dcb | 0 ...t_exist-9-d824f22606f48dfca48ce241a7505f5b | 3 + ...ultiple-0-9c55143a4c92f0cfe7669f7681d7aa98 | 0 ...ultiple-1-4d9d4efbabc9fffef8841cc049f479c1 | 29 + ...ultiple-2-187930e0daa44eed17e092e961ab7955 | 0 ...ultiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef | 0 ...ultiple-4-7950c676506564b085b41426ed41747c | 4 + ...itelist-0-3806584ff765bca682594008b90fc304 | 1 + ...itelist-1-22eb96fe7d338e488182b5755c90d5af | 0 ...itelist-2-923fa18234ae73103c43722f70e000c0 | 0 ...itelist-3-b7c0bb09609fabad407feb6fdf2c748f | 0 ...itelist-0-3c23ae800b2f6fb93620890da2dba196 | 1 + ...itelist-1-67a0dbca9fecb7d34cceeecf2184c484 | 0 ...itelist-2-e6a91be97431de63e372088d370c6d36 | 0 ...itelist-3-b9a6b4e8acbfea5e1938eda085c4b893 | 0 ...alias.*-0-7bdb861d11e895aaea545810cdac316d | 1 + ...as.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 | 1 + ..._column-0-f69b60c6e896fcd1a69d9525fd988c66 | 0 ..._column-1-4432aec015f9423ed991b08cfb2af0e1 | 0 .../alter2-0-85e494848d1525843a3ff9b2b77f92 | 0 .../alter2-1-aac9c2c7033fd7264c9a107a88ff591 | 10 + ...alter2-10-2d1f665a92fe72bd253ae57c46d7b9da | 0 .../alter2-11-aac9c2c7033fd7264c9a107a88ff591 | 10 + ...alter2-12-75a213649242c2410ea6846f08c91d75 | 0 ...alter2-13-ca51e03a3de391983429b6ad877e573c | 0 .../alter2-14-aac9c2c7033fd7264c9a107a88ff591 | 10 + ...alter2-15-75a213649242c2410ea6846f08c91d75 | 1 + ...alter2-16-77500c83ffcece95511a4b21d67382dd | 0 .../alter2-17-aac9c2c7033fd7264c9a107a88ff591 | 10 + ...alter2-18-75a213649242c2410ea6846f08c91d75 | 2 + .../alter2-19-3d80bb2d1c541460b5b17c4124fa647 | 0 .../alter2-2-75a213649242c2410ea6846f08c91d75 | 0 ...alter2-20-9c36cac1372650b703400c60dd29042c | 2 + ...alter2-21-a6ea9efed3b9c680fca93588019ac5e3 | 0 ...alter2-22-ea07b8f664208f93a1a8b97fd486d226 | 0 ...alter2-23-9c36cac1372650b703400c60dd29042c | 0 .../alter2-24-775d69742a1c07df8da87e8a017d955 | 0 ...alter2-25-4ef75e12575453225738ea167c4617e5 | 10 + ...alter2-26-63f13c364546ddce5d2176c6604a948f | 0 ...alter2-27-ba521286c12ba29329bfa71bb185c62f | 0 ...alter2-28-4ef75e12575453225738ea167c4617e5 | 10 + ...alter2-29-63f13c364546ddce5d2176c6604a948f | 1 + .../alter2-3-ca51e03a3de391983429b6ad877e573c | 0 ...alter2-30-a336adf1d0ff00633c53600fc75ca3ae | 0 ...alter2-31-4ef75e12575453225738ea167c4617e5 | 10 + ...alter2-32-63f13c364546ddce5d2176c6604a948f | 2 + ...alter2-33-1934026d0228967097280eed35551f74 | 0 .../alter2-34-23b00f9c0101348e87da8a339b9da8b | 0 ...alter2-35-4ef75e12575453225738ea167c4617e5 | 10 + ...alter2-36-63f13c364546ddce5d2176c6604a948f | 0 ...alter2-37-ba521286c12ba29329bfa71bb185c62f | 0 ...alter2-38-4ef75e12575453225738ea167c4617e5 | 10 + ...alter2-39-63f13c364546ddce5d2176c6604a948f | 1 + .../alter2-4-aac9c2c7033fd7264c9a107a88ff591 | 10 + ...alter2-40-a336adf1d0ff00633c53600fc75ca3ae | 0 ...alter2-41-4ef75e12575453225738ea167c4617e5 | 10 + ...alter2-42-63f13c364546ddce5d2176c6604a948f | 2 + ...alter2-43-1934026d0228967097280eed35551f74 | 0 ...alter2-44-7a9e67189d3d4151f23b12c22bde06b5 | 0 .../alter2-45-adbc01277c01cc5647e89c8a2430b8c | 0 .../alter2-5-75a213649242c2410ea6846f08c91d75 | 1 + .../alter2-6-77500c83ffcece95511a4b21d67382dd | 0 .../alter2-7-aac9c2c7033fd7264c9a107a88ff591 | 10 + .../alter2-8-75a213649242c2410ea6846f08c91d75 | 2 + .../alter2-9-1986a53bb1944fe5f43e3e65693e7b1e | 0 .../alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade | 0 .../alter3-1-75be487df30e301e156a22eee075633d | 0 ...alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 | 0 ...alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 | 0 ...alter3-12-2fcb7fc251f682a584ad513fddfac506 | 11 + ...alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b | 0 ...alter3-14-3fa4d8a690a45cbf7b44cecfd352864f | 0 ...alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 | 0 ...alter3-16-9c36cac1372650b703400c60dd29042c | 2 + ...alter3-17-a9908f67f97588cbf15c0e7caddcbb0b | 0 ...alter3-18-581b65f9f467d0d4a33a16dda144a31d | 0 ...alter3-19-9c36cac1372650b703400c60dd29042c | 0 .../alter3-2-5a67d369d700eb96f806f8320c04d61f | 0 ...alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 | 0 ...alter3-21-231db1adbff5fc90e57cca6a087f3df5 | 0 ...alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 | 0 ...alter3-23-7ad62b397f6c9341da6bf0e9361314e2 | 0 ...alter3-24-9c23b682abda3841f01b4d9b750c68d9 | 0 ...alter3-25-568a59760e5d3241b63d65cce595face | 6 + ...alter3-26-3c725018f74a69f4d859c66af2f5b11e | 0 ...alter3-27-54ad133b447f67c6d1ed7d4c43803a87 | 11 + ...alter3-28-5332228ea451105c897d0c8c3c8f2773 | 11 + ...alter3-29-b8fba19b9009131caffbb5fe7468b67c | 6 + .../alter3-3-f031aa27bf7b494cb8de20a305be7064 | 0 ...alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 | 0 ...alter3-31-5a41cf8c1a828ac2c372536ee4afd962 | 0 ...alter3-32-327744965ee8ed630f56fa3e4a3c5c65 | 11 + .../alter3-4-399fc26b344c98ababa104522601c0cc | 0 .../alter3-5-bf2a8fd1884bb584059c848332e30c97 | 6 + .../alter3-6-fe6db2a539df10e4bc4715e2ed755135 | 0 .../alter3-7-30be5698ca15c1fd836686e7ad48ad8 | 11 + .../alter3-8-8f0a466bd1d021e40690865b7ae52a43 | 11 + .../alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 | 6 + .../alter4-0-c261e1fa9f838dd034d37af38305e9c6 | 0 .../alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 | 4 + ...alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 | 4 + ...alter4-11-7db7af854e4e10fb6f0338b85d65549d | 0 ...alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 | 4 + ...alter4-13-e9879d6bebc109340bbeecc3ca77492f | 0 ...alter4-14-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...alter4-15-63a545ee0e751a2729c8758a14712da5 | 0 ...alter4-16-549981e00a3d95f03dd5a9ef6044aa20 | 1 + .../alter4-2-7db7af854e4e10fb6f0338b85d65549d | 0 .../alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 | 4 + .../alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 | 0 .../alter4-5-9c36cac1372650b703400c60dd29042c | 2 + .../alter4-6-acd58e84952d310aeddf78579c36286 | 0 .../alter4-7-8e6ec3396f25c124de5b212d8ce6c568 | 0 .../alter4-8-9c36cac1372650b703400c60dd29042c | 0 .../alter4-9-c261e1fa9f838dd034d37af38305e9c6 | 0 .../alter5-0-953553e14d835682fa47338dcfffe227 | 0 .../alter5-1-b8349afaf8e62dc6608a889c04ee3d4b | 0 ...alter5-10-9c36cac1372650b703400c60dd29042c | 2 + ...alter5-11-e63fa029ab22ac4f5c880f2848f1b956 | 0 ...alter5-12-6b160869b8a9c846bc55a14f85bc5b52 | 0 ...alter5-13-9c36cac1372650b703400c60dd29042c | 0 ...alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 | 0 ...alter5-15-b8349afaf8e62dc6608a889c04ee3d4b | 0 ...alter5-16-4b76b7ff0df6adeded64b2a2f305530d | 0 ...alter5-17-9176dc5fb5206209fa907a289db1263d | 0 ...alter5-18-2a9c8219c1468a1cf0534c665d1fcebf | 9 + ...alter5-19-2f6ab691e291c74ecc4305eeb30e3438 | 0 .../alter5-2-4b76b7ff0df6adeded64b2a2f305530d | 0 ...alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 | 6 + ...alter5-21-2a9c8219c1468a1cf0534c665d1fcebf | 9 + .../alter5-3-2fc59e32c07186869811705c89aafadc | 0 .../alter5-4-2a9c8219c1468a1cf0534c665d1fcebf | 9 + .../alter5-5-2f6ab691e291c74ecc4305eeb30e3438 | 0 .../alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 | 6 + .../alter5-7-2a9c8219c1468a1cf0534c665d1fcebf | 9 + .../alter5-8-89c414c65a129f2fc408b3124f292b29 | 0 .../alter5-9-1e085f9741197e659413828c42386733 | 0 ...r_index-0-21bcf37075b02097f16c8fc8130a83b8 | 0 ...r_index-1-4c8f6b48c437bf0be109fc0be1dc840e | 0 ...er_index-2-f36cb2eed39691ca949b25182e2dd31 | 5 + ...r_index-3-33474b65c86b949d266541e0385bc6bb | 0 ...er_index-4-f36cb2eed39691ca949b25182e2dd31 | 5 + ...r_index-5-21bcf37075b02097f16c8fc8130a83b8 | 0 ...r_index-6-489b4ceb2f4301a7132628303f99240d | 2 + ...merge_2-0-48044f1a60d3e15e4e17f8f95098d995 | 0 ...merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 | 0 ...erge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 | 1 + ...merge_2-11-efe4e50f2330d4f0a737183ea51836c | 0 ...merge_2-2-bf243aa10b608872b9e8286f89c5ff30 | 12 + ...merge_2-3-cfef140167765d259320ed1e8aba718d | 0 ...merge_2-4-47aa9e3236da627ab505a6bd01e563be | 0 ...merge_2-5-43bd090cda258e25037e7c32d500a85f | 0 ...merge_2-6-f2eeb518a957cece4250cc7558839e02 | 1 + ...merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 | 1 + ...merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc | 0 ...merge_2-9-f2eeb518a957cece4250cc7558839e02 | 1 + ...coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a | 0 ...coltype-1-b9bb847419afb2b70d0cb887894f0b90 | 0 ...oltype-10-aa739a81271c760437de572a6c951eb9 | 0 ...oltype-11-94b98dca970e36e6d4d65a795c9413d6 | 0 ...oltype-12-84807e0be3e91250d4b8f2dc7938a256 | 1 + ...oltype-13-60ab4c242e19b882e4b1571544311e7e | 0 ...oltype-14-fdad866a990591083719bd45e4113f58 | 1 + ...oltype-15-634775da8cebfb8ce45b3965a0ae2880 | 0 ...oltype-16-b0534980e325d1fee63c2636ad3f8a4e | 75 + ...oltype-17-13c0443789a696bde8d08c05f526643f | 0 ...oltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c | 1 + ...oltype-19-63799ee74ccc42d9bb817e6d00a6fae3 | 1 + ..._coltype-2-17e04afbb81a724fe8c47c07b642f9a | 10 + ...oltype-20-b4411edb9c52a474a971640f037c8a30 | 0 ...coltype-21-17e04afbb81a724fe8c47c07b642f9a | 10 + ...oltype-22-639cdccdea5f4b1863f9b17b04183c93 | 10 + ...oltype-23-325be9a5d7c0277498a139c0a9fdb26a | 10 + ...oltype-24-71de9e476503e83f82baf1ab17da87f6 | 10 + ...oltype-25-40b997fef00cf1a26f73ddb5013bbeb6 | 0 ...coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 | 1 + ...coltype-4-f6ba42faebdf8ec8781716fec6f7813d | 0 ...coltype-5-eeb71f1bc830750174b8b0d03de8c37d | 0 ...coltype-6-db84ad9022cdc1828c24a0340321c8fd | 1 + ...coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 | 0 ...coltype-8-42a70ae131fbb834c79074fdbd7beea0 | 1 + ...coltype-9-2f7e10db0fcc7939dea528e04f460d42 | 0 ...mat_loc-0-72ba9397f487a914380dc15afaef1058 | 0 ...mat_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 | 4 + ...at_loc-10-71631c1e516c81ffdceac80f2d57ce09 | 0 ...at_loc-11-fe39b84ddc86b6bf042dc30c1b612321 | 10 + ...at_loc-12-1553ad79b098b737ea8def91134eb0e9 | 0 ...at_loc-13-fe39b84ddc86b6bf042dc30c1b612321 | 10 + ...at_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 | 10 + ...at_loc-15-bc83e8a2f8edf84f603109d14440dc83 | 0 ...at_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 | 10 + ...at_loc-17-7e411fcfdd8f169c503ed89dc56ee335 | 0 ...at_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 | 10 + ...at_loc-19-56cadf0f555e355726dfed1929ad0508 | 0 ...mat_loc-2-bc83e8a2f8edf84f603109d14440dc83 | 0 ...mat_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 | 4 + ...mat_loc-4-7e411fcfdd8f169c503ed89dc56ee335 | 0 ...mat_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 | 4 + ...mat_loc-6-56cadf0f555e355726dfed1929ad0508 | 0 ...mat_loc-7-cee355b012efdc3bc7d584268a7025c2 | 0 ...mat_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 | 0 ...mat_loc-9-fe39b84ddc86b6bf042dc30c1b612321 | 10 + ...ct_mode-0-2a230c069b09232acdd0d556007be97f | 0 ...ct_mode-1-fbbdf7be607407661749730f1a0efd9c | 0 ...t_mode-10-d71b99098bdb7f13db278dfa299b820d | 0 ...t_mode-11-482182c9d90710fb16b6803d602a0d8b | 0 ...t_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 | 0 ...t_mode-13-19ceced1d8238509f2416029ddfbbc4a | 0 ...t_mode-14-90d009f94408102945d43860e4a6c68a | 0 ...t_mode-15-7ab0e8f289c6846f9872edee0c40a628 | 0 ...t_mode-16-577e1c164866c3955a9d8587ef7918a4 | 0 ...ct_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 | 0 ...ct_mode-3-dd92c46e933d94b35c225daeef0285d4 | 0 ...ect_mode-4-9114d1f6859382a125fc4221d2d3ab6 | 0 ...ct_mode-5-3cc094c5aa537b12f98895b95765329c | 0 ...ct_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 | 12 + ...ct_mode-7-5439426a18bb2d3918b91d589dbbd014 | 0 ...ct_mode-8-71e8c12c533654c30e044a8f062598ed | 0 ...ct_mode-9-d1b12be1d01eabaf244f41e74d902d9d | 6 + ...itelist-0-3c23ae800b2f6fb93620890da2dba196 | 1 + ...itelist-1-67a0dbca9fecb7d34cceeecf2184c484 | 0 ...itelist-2-e6a91be97431de63e372088d370c6d36 | 0 ...itelist-3-b9a6b4e8acbfea5e1938eda085c4b893 | 0 ...itelist-4-f42e9ca89ed2944213a5d994a587391c | 0 ...artition-0-1ed18256c5230de3439fe75d925ea73 | 0 ...rtition-1-2f79bceed6fc8ada34a670396ee6aada | 0 ...tition-10-e3d9a36d53d30de215b855095c58d0d7 | 0 ...tition-11-bc84e38ccad173f5b47474d91db244d7 | 6 + ...rtition-12-1ed18256c5230de3439fe75d925ea73 | 0 ...tition-13-2f79bceed6fc8ada34a670396ee6aada | 0 ...tition-14-9c36cac1372650b703400c60dd29042c | 2 + ...tition-15-f3b7bcb5d95a356fee54c0ce7d60c611 | 0 ...tition-16-611cf586cf3a1adc93c543d2da574c24 | 0 ...tition-17-9c36cac1372650b703400c60dd29042c | 0 ...tition-18-bf6f780173f7b523b7ebd7925789372b | 0 ...tition-19-b2c9ded072d49abe14831bf48290319c | 0 ...rtition-2-9c36cac1372650b703400c60dd29042c | 2 + ...tition-20-ee3ad861d109dd98db10bc86c5bf7105 | 0 ...tition-21-d92bfe92d250f66b3df45cb4ab50c0e6 | 0 ...tition-22-d50111b57d14f1ded1c47c773b0e0ac2 | 6 + ...tition-23-aedbaca33604c76b65137905fd42e98f | 0 ...tition-24-21dd05d56ebba285a8eb5bde5904d6a3 | 1 + ...tition-25-9595f5b6ab31162c107076c35657c9f3 | 0 ...tition-26-270655c514bb6f04acd4459df52dd77b | 6 + ...rtition-3-b465c6126edd94e8d45f61e2a19d005b | 0 ...rtition-4-3479a886936571d5028971aecade705f | 0 ...rtition-5-6cc4e3014e34a862602a47357f4fb9f2 | 0 ...rtition-6-3324664e6500e2d256d0b8b3b8a14c24 | 0 ...rtition-7-e3d9a36d53d30de215b855095c58d0d7 | 6 + ...rtition-8-d3ea111b1a37613bdda2c6eae13790c9 | 0 ...rtition-9-21dd05d56ebba285a8eb5bde5904d6a3 | 1 + ...e_serde-0-35d2014351106b918c8e337a1919470c | 0 ...e_serde-1-5bc931a540f0fec54e852ff10f52f879 | 5 + ..._serde-10-ed0059ecd1cf948e8f75153593c8a5aa | 0 ..._serde-11-6ee4b3a60659ec5496f06347eda232a8 | 11 + ..._serde-12-63a8168d2eae62132c3cd9b90b3cdbcc | 0 ..._serde-13-6ee4b3a60659ec5496f06347eda232a8 | 11 + ..._serde-14-ab6d7ed387e6e2f1f8f32272e3d31def | 0 ..._serde-15-6ee4b3a60659ec5496f06347eda232a8 | 11 + ..._serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 | 0 ..._serde-17-6ee4b3a60659ec5496f06347eda232a8 | 11 + ..._serde-18-1649ba756fd9238f0608e4cb3affa3c1 | 0 ...e_serde-2-ed0059ecd1cf948e8f75153593c8a5aa | 0 ...e_serde-3-5bc931a540f0fec54e852ff10f52f879 | 5 + ...e_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc | 0 ...e_serde-5-5bc931a540f0fec54e852ff10f52f879 | 5 + ...e_serde-6-1649ba756fd9238f0608e4cb3affa3c1 | 0 ...e_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b | 0 ...e_serde-8-78d739d2409b59c0e01cde962451d295 | 0 ...e_serde-9-6ee4b3a60659ec5496f06347eda232a8 | 11 + ...archar1-0-5fa6071842a0443346cf6db677a33412 | 0 ...archar1-1-be11cb1f18ab19550011417126264fea | 0 ...rchar1-10-c1a57b45952193d04b5411c5b6a31139 | 0 ...rchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 | 5 + ...rchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd | 0 ...rchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 | 5 + ...rchar1-14-5fa6071842a0443346cf6db677a33412 | 0 ...archar1-2-ba9453c6b6a627286691f3930c2b26d0 | 0 ...archar1-3-fa89c704636fa7bd937cf1a975bb2ae6 | 5 + ...archar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d | 0 ...archar1-5-2756ef8fbe2cfa4609808a3855f50969 | 5 + ...archar1-6-f7d529dc66c022b64e0b287c82f92778 | 0 ...archar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f | 5 + ...varchar1-8-bdde28ebc875c39f9630d95379eee68 | 0 ...archar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 | 5 + ...archar2-0-22c4186110b5770deaf7f03cf08326b7 | 0 ...archar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc | 0 ...archar2-2-325238d61f56d84c17e29033105d7b19 | 0 ...archar2-3-fb3191f771e2396d5fc80659a8c68797 | 1 + ...varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb | 0 ...archar2-5-84e700f9dc6033c1f237fcdb95e31a0c | 1 + ...archar2-6-eb0f1b170900bb995674b0bc1968c656 | 0 ...archar2-7-84e700f9dc6033c1f237fcdb95e31a0c | 1 + ...archar2-8-4c12c4c53d99338796be34e603dc612c | 1 + ..._select-0-9f40bf1c2b92465189583446a6b40910 | 0 ..._select-1-5ba1b5ca1199ad7281ff9b5b71105aad | 0 ..._select-2-1ac845048a8c714a36a719ea8e4f570b | 30 + ..._select-3-9280ae6c369a9f30d3d021d00e435f01 | 0 ..._select-4-1ac845048a8c714a36a719ea8e4f570b | 27 + ..._select-5-48b435d96e34065b03c6d7e4e891fbe2 | 0 ..._select-6-1ac845048a8c714a36a719ea8e4f570b | 34 + ..._rename-0-bb255b994b5207324fba6988caa937e6 | 0 ..._rename-1-2a83c96363ca8d12cd2e9181209c8d8d | 0 ...ew_rename-2-67e47ee2746463594d5c48b10ba1bb | 5 + ..._rename-3-95655e33f22fc8f66549a9708812589a | 0 ..._rename-4-19c1c00f0aa99d81b7466958c15d88e3 | 5 + ...ous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d | 0 ...ous_col-1-dadfa3854356dead14b93c5a71a5d8ab | 0 ...ous_col-2-70509ccd2765d90b98666b6dff8afe1b | 0 ...archive-0-89cd75b0565e8d96910d5528db9984e7 | 1 + ...archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...rchive-10-f845008104fd12eb0f13f4b113f95cf1 | 0 ...rchive-11-27895cbe0ee6d24d7fc866314491e1bb | 1 + ...rchive-12-f07653bca86e1ecb614ffd0296790d05 | 0 ...rchive-13-27895cbe0ee6d24d7fc866314491e1bb | 1 + ...archive-14-2cde1a2d3bfcec814985f498eba0fb8 | 1 + ...rchive-15-c9f39b388ae698e385e092d0ffeb3c73 | 9 + ...rchive-16-892147913578bcf60620b7dd73893dd0 | 0 ...rchive-17-27895cbe0ee6d24d7fc866314491e1bb | 1 + ...rchive-18-8f980275ab3a5bcfc1784f4acd46447a | 0 ...rchive-19-f8a52a8a40141409a667a9ba2cf9630f | 0 ...archive-2-713efc113418b01f76ffd589840193c8 | 0 ...rchive-20-530277b0fee8b05c37b26846bceef827 | 6 + ...rchive-21-f07653bca86e1ecb614ffd0296790d05 | 0 ...rchive-22-530277b0fee8b05c37b26846bceef827 | 6 + ...rchive-23-892147913578bcf60620b7dd73893dd0 | 0 ...rchive-24-530277b0fee8b05c37b26846bceef827 | 6 + ...rchive-25-56bea24ffa83d9b5932893a8ff1cb44f | 0 ...rchive-26-44fa61c2bd0fd9acfa0d889e59880d8a | 0 ...archive-27-9ae773ebe64a3d437a035e9d94f49e5 | 0 ...rchive-28-188eb7912265ed8dffa5200517bbe526 | 1 + ...rchive-29-a0c6922e3a1dca51861c8a872dc8af19 | 0 ...archive-3-27ad2962fed131f51ba802596ba37278 | 0 ...rchive-30-bea4ae5a0d219d544ea0b53bf29ecc7a | 1 + ...rchive-31-cd46bc635e3010cf1b990a652a584a09 | 0 ...rchive-32-27ad2962fed131f51ba802596ba37278 | 0 ...archive-4-3e95421993ab28d18245ec2340f580a3 | 0 ...archive-5-c0c18ac884677231a41eea8d980d0451 | 0 ...archive-6-528ab9750a558af7f1a43b3108e793dd | 0 ...archive-7-e8d1d10c308a73eef78dde414a5e40ca | 0 ...archive-8-af459a0264559a2aeaa1341ce779ab3c | 0 .../archive-9-48b10f27e1459bb8e62d6c71484e2cf | 0 ...adoop20-0-89cd75b0565e8d96910d5528db9984e7 | 1 + ...adoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...doop20-10-f845008104fd12eb0f13f4b113f95cf1 | 0 ...doop20-11-27895cbe0ee6d24d7fc866314491e1bb | 1 + ...doop20-12-f07653bca86e1ecb614ffd0296790d05 | 0 ...doop20-13-27895cbe0ee6d24d7fc866314491e1bb | 1 + ...adoop20-14-2cde1a2d3bfcec814985f498eba0fb8 | 1 + ...doop20-15-c9f39b388ae698e385e092d0ffeb3c73 | 9 + ...doop20-16-892147913578bcf60620b7dd73893dd0 | 0 ...doop20-17-27895cbe0ee6d24d7fc866314491e1bb | 1 + ...doop20-18-8f980275ab3a5bcfc1784f4acd46447a | 0 ...doop20-19-f8a52a8a40141409a667a9ba2cf9630f | 0 ...adoop20-2-433a1b642df2cebe37927d6d89e0b301 | 0 ...doop20-20-530277b0fee8b05c37b26846bceef827 | 6 + ...doop20-21-f07653bca86e1ecb614ffd0296790d05 | 0 ...doop20-22-530277b0fee8b05c37b26846bceef827 | 6 + ...doop20-23-892147913578bcf60620b7dd73893dd0 | 0 ...doop20-24-530277b0fee8b05c37b26846bceef827 | 6 + ...doop20-25-56bea24ffa83d9b5932893a8ff1cb44f | 0 ...doop20-26-44fa61c2bd0fd9acfa0d889e59880d8a | 0 ...adoop20-27-9ae773ebe64a3d437a035e9d94f49e5 | 0 ...doop20-28-188eb7912265ed8dffa5200517bbe526 | 1 + ...doop20-29-a0c6922e3a1dca51861c8a872dc8af19 | 0 ...adoop20-3-27ad2962fed131f51ba802596ba37278 | 0 ...doop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a | 1 + ...doop20-31-cd46bc635e3010cf1b990a652a584a09 | 0 ...doop20-32-27ad2962fed131f51ba802596ba37278 | 0 ...adoop20-4-3e95421993ab28d18245ec2340f580a3 | 0 ...adoop20-5-c0c18ac884677231a41eea8d980d0451 | 0 ...adoop20-6-528ab9750a558af7f1a43b3108e793dd | 0 ...adoop20-7-e8d1d10c308a73eef78dde414a5e40ca | 0 ...adoop20-8-af459a0264559a2aeaa1341ce779ab3c | 0 ...Hadoop20-9-48b10f27e1459bb8e62d6c71484e2cf | 0 ...e_multi-0-89cd75b0565e8d96910d5528db9984e7 | 1 + ...e_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._multi-10-f845008104fd12eb0f13f4b113f95cf1 | 0 ..._multi-11-cf5431cd843666b95ad2a82b334ac01e | 1 + ..._multi-12-8419ad4ed6683ebd15f993f703975b31 | 0 ..._multi-13-27895cbe0ee6d24d7fc866314491e1bb | 1 + ...e_multi-14-2cde1a2d3bfcec814985f498eba0fb8 | 1 + ..._multi-15-c9f39b388ae698e385e092d0ffeb3c73 | 9 + ..._multi-16-ad80f33c39be583ad7ebf0c8f350d11d | 0 ..._multi-17-27895cbe0ee6d24d7fc866314491e1bb | 1 + ...e_multi-2-cd46bc635e3010cf1b990a652a584a09 | 0 ...e_multi-3-27ad2962fed131f51ba802596ba37278 | 0 ...e_multi-4-3e95421993ab28d18245ec2340f580a3 | 0 ...e_multi-5-c0c18ac884677231a41eea8d980d0451 | 0 ...e_multi-6-c06da7f8c1e98dc22e3171018e357f6a | 0 ...e_multi-7-e8d1d10c308a73eef78dde414a5e40ca | 0 ...e_multi-8-af459a0264559a2aeaa1341ce779ab3c | 0 ...ve_multi-9-48b10f27e1459bb8e62d6c71484e2cf | 0 .../attr-0-24e06ffd262f2a5a6eec3314445d83ba | 1 + ...o_join0-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...o_join0-1-383f34dec3ac939b7af2c9093a557641 | 0 ...o_join0-2-7bd04899197b027d81c24e45a99ad15c | 1 + ...o_join1-0-443afb71720bad780b5dbfb6dbf4b51a | 1 + ...o_join1-1-f1293ebf768eb04f2f0bfe6297c4509e | 0 ...o_join1-2-8a9624554e208e3d8fbe42908c715b92 | 0 ...to_join1-3-f6046c5229e3b0aa21498a3872f43b2 | 0 ...to_join1-4-ae1247a065c41ce0329ca6078ab586e | 1 + ..._join10-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 | 0 ..._join10-2-eef4ee52e0783b15fb5fe17378806b13 | 1 + ..._join11-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join11-1-82ea193ec76c2c6acd5f7178ef5ec417 | 0 ..._join11-2-5496e81f60ba1d8a95d8375589c71e05 | 1 + ..._join12-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join12-1-c2efec9ea2ba761603b723afc0d5d145 | 0 ..._join12-2-4df549c5f0b6bff0c843008fa35b1320 | 1 + ..._join13-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join13-1-f5e043288a21ea691c74fef2e39a52b4 | 0 ..._join13-2-5ff417533a1243cd6fc556960fa170c9 | 1 + ..._join14-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join14-1-c85f3dcbab496811604ea0ab84d0e995 | 0 ..._join14-2-9b141c1e5917ca82c6bc36a9a2950a1e | 1 + ...o_join14-3-2b9ccaa793eae0e73bf76335d3d6880 | 1 + ..._join14-4-bab89dfffa77258e34a595e0e79986e3 | 1 + ..._join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 | 0 ...o_join14-6-de39302191b63d7aa8f92885b089fe2 | 0 ...o_join14-7-5b5ded1412301eae5f8f705a39e6832 | 1 + ...adoop20-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...adoop20-1-98b7542190092fafcc8b1ad5b0024a22 | 0 ...adoop20-2-db1cd54a4cb36de2087605f32e41824f | 1 + ...adoop20-3-bab89dfffa77258e34a595e0e79986e3 | 1 + ...adoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 | 0 ...hadoop20-5-de39302191b63d7aa8f92885b089fe2 | 0 ...hadoop20-6-5b5ded1412301eae5f8f705a39e6832 | 1 + ..._join15-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join15-1-e23b9aa655061fb5a70d1f8f28f170f5 | 0 ..._join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 | 1 + ..._join16-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join16-1-bba773956a3bad8d400fe4216a3fa8bf | 0 ..._join16-2-66e56dcda38eb09819ac49e47e40d125 | 1 + ..._join17-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join17-1-387dd86b1e13f788ec677a08dc162c97 | 0 ..._join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 | 0 ..._join17-3-478a9f270a5d70f6f82f81e6962fb251 | 0 ..._join17-4-11d706a64d44a8b0d41b290c4671c29c | 1 + ..._join18-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join18-1-3839d176ee45fb0fc6702f4a7794ca1b | 0 ..._join18-2-f633ade9577c8b0e89d89124194c8d0f | 1 + ..._join19-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join19-1-13ab74a58da514fe01dbeda0c3e79883 | 0 ..._join19-2-70f3756d8b44d637ac4596cbbd48dc77 | 0 ..._join19-3-a3751c195480244a5ed497fd053cd433 | 0 ..._join19-4-eaa70da463b92e85e1796277f016c18f | 1 + ...o_join2-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...o_join2-1-3d1692c4710db1ff716d35e921f2bcca | 0 ...o_join2-2-15d7a6cb2e2f21077de2447c656e7a34 | 0 ...o_join2-3-d4673c03d04084b838fcd8149f59ad9a | 0 ...o_join2-4-9d8144612cb3132ad9f7c8fa93586185 | 1 + ..._join20-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join20-1-2afb0510178c4b66876dd91c7ca441fd | 0 ..._join20-2-903ee25e327188edaaf2040fec5a8e52 | 1 + ..._join20-3-cf1c52393ea3a7e21782a1c52b83f0ee | 0 ..._join20-4-e48e08d5b94719d56a58284eaef757f2 | 1 + ..._join21-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join21-1-9dd59784ca1555b607df0137d2666fb8 | 0 ..._join21-2-3536b7d78713e86ee67f5f6c9b88958f | 500 ++++ ..._join22-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join22-1-4044be0e5116357bd88b4eda0f9ccaa8 | 0 ..._join22-2-a4abc288c20edee53ede45d248cf3abb | 1 + ..._join23-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join23-1-b31437533a2b890788938455cb32f679 | 0 ..._join23-2-6d2c5b58222f31658a0cf957e093a150 | 100 + ..._join24-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join24-1-721dfa03bfea05e55506c571b6c3585b | 0 ..._join24-2-36de83b0ed6c9fdc03661b2f65b23a3d | 0 ..._join24-3-fa8b2736440ff35687dadb1bcae32666 | 0 ..._join24-4-d79325ef6494aa87843fdfd78de7c812 | 1 + ..._join25-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 | 1 + ...join25-10-f1293ebf768eb04f2f0bfe6297c4509e | 0 ..._join25-11-f6046c5229e3b0aa21498a3872f43b2 | 0 ..._join25-12-ae1247a065c41ce0329ca6078ab586e | 1 + ..._join25-2-8180638a57b64557e02815c863031755 | 1 + ..._join25-3-9aa914a687f1f63faf48eb500627855e | 1 + ..._join25-4-d83e6df8cd60d8ebeebd2100c51002d9 | 0 ..._join25-5-a3751c195480244a5ed497fd053cd433 | 0 ..._join25-6-eaa70da463b92e85e1796277f016c18f | 1 + ..._join25-7-3d1692c4710db1ff716d35e921f2bcca | 0 ..._join25-8-d4673c03d04084b838fcd8149f59ad9a | 0 ..._join25-9-9d8144612cb3132ad9f7c8fa93586185 | 1 + ..._join26-0-54a7280ab9eed0d2e3b33df35a721b66 | 0 ..._join26-1-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join26-2-5ebef1af539734d0335dbe6aacae3e13 | 0 ..._join26-3-fed383a65bd118b43de6b00be10fecb6 | 0 ..._join26-4-c9859bf9c9942c59f3b19d82bd1a3afa | 15 + ..._join27-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join27-1-c83f56f364b1da3def90d48953665fe5 | 0 ..._join27-2-fceaa1ebd63334061d2d8daf961e935e | 1 + ..._join28-0-10a2c01dccc8980fe6aff1f9dd65042c | 1 + ..._join28-1-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join28-2-9dd59784ca1555b607df0137d2666fb8 | 0 ..._join28-3-30739ff22c62b3becf56694642b7ae81 | 0 ...o_join28-4-c178253e7ce91b5aa35c2cc424bfa27 | 0 ...o_join28-5-142850e84341feb3f7f40dd4553f72e | 0 ...o_join3-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...o_join3-1-13ab74a58da514fe01dbeda0c3e79883 | 0 ...o_join3-2-6bed7d8089695e23914b29edaab2537d | 0 ...o_join3-3-e9f6d17b15064f953a588fb40aee2f90 | 0 ...o_join3-4-eaa70da463b92e85e1796277f016c18f | 1 + ..._join30-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join30-1-8cef272a7680529de5c6bd227a83cbc0 | 0 ...join30-10-820f36ed1bdf14c1deb383f508a5ed7a | 1 + ...join30-11-53e43f2e48f605ec92c8a18c53e80620 | 0 ..._join30-12-de6e26f52f6f9ea5ef9a4868e57d36d | 1 + ...join30-13-9ee597656aa92e48475d6542339915ba | 0 ...join30-14-7a248488c218919ab50e072fdbdecb73 | 1 + ...join30-15-47b7efabbd6046e2befcbbea7da62553 | 0 ...join30-16-f4f5bc179d84baf57e14cd2f8bd39436 | 1 + ..._join30-2-bc472f95600f47d5ea60fdeddc59dbc7 | 1 + ..._join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 | 0 ...to_join30-4-f5083eca9c3df277988d8b345b8d43 | 1 + ..._join30-5-3916f4b640f3579035153f6940113ef2 | 0 ..._join30-6-4a9144326fc7d066c9aadb13d1b95031 | 1 + ..._join30-7-f07b674c31ca9fdf837406cb9a96108e | 0 ..._join30-8-8a27209399df7f9c4d15988b11753a61 | 1 + ..._join30-9-4ee48fa9bfeb818c81768b6de0517263 | 0 ..._join31-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._join31-1-5a2b4475d9a88e53a2b6ec29279253c0 | 0 ...o_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 | 1 + ..._join32-0-24ca942f094b14b92086305cc125e833 | 1 + ..._join32-1-2e533cf988f613d5fc3fbde67ffd9118 | 0 ...join32-10-2e8ccb343bce61564bae209a589cca85 | 0 ...join32-11-4d6fc319375b6962eca0aa63dfabfdc1 | 0 ...join32-12-a8b69002151b3df4383d2c354dbaa7d4 | 0 ...join32-13-63241e3791725baad8baa00fb833ef5e | 0 ...join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f | 0 ...join32-15-187fd938996ae7d96e60475fb69b8d35 | 0 ...join32-16-2e8ccb343bce61564bae209a589cca85 | 0 ...join32-17-4d6fc319375b6962eca0aa63dfabfdc1 | 0 ...join32-18-d1d78b19d484e55d9da8a320253ece0f | 0 ...join32-19-83b9df41bc46afbbafd0cd30cb982332 | 0 ..._join32-2-865207407ff1acbccb47473d87e87e8d | 0 ..._join32-20-e67740fb52998f1d3afcfa667505cf7 | 0 ...join32-21-da5b4647f5605dff66aa47e059f9ec8b | 0 ...join32-22-a1d339a0d904c3f35771192a026c7f9c | 0 ...join32-23-1948951cc3c06cdf962d59e932a84588 | 0 ...join32-24-cda0994eb851b57fdb80e16b033d1b73 | 0 ...join32-25-e46226186de575c81cfab296607e1b4b | 0 ...join32-26-97d265cd7defca44e488c38bac4c5b7a | 0 ..._join32-27-b034eeb850810b5004ddff1f2a530bc | 0 ..._join32-28-751550ac0550e6a7dd737cad01d6d82 | 0 ...join32-29-34ecfdabf9c769027706f53fa2d66ed3 | 0 ..._join32-3-9ccdfe4052062a1dfc72c711179d9e43 | 0 ...join32-30-d3903985844b06c4af11334b72f383d1 | 0 ...join32-31-2415fd7a0c2e37b09679bb4c64f321bd | 0 ..._join32-4-a28f563122d1f0debd04f74c534523cf | 0 ..._join32-5-c23ea191ee4d60c0a6252ce763b1beed | 1 + ...o_join32-6-442e69416faaea9309bb8c2a3eb73ef | 1 + ..._join32-7-d0ec6d66ff349db09fd455eec149efdb | 1 + ..._join32-8-999683fa0291bf439b03557edec7dcee | 0 ...o_join32-9-1e6d4ec86f29d74828891c17986e84a | 0 ...o_join4-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...o_join4-1-531e91e11b3891627c2675935fda14cd | 0 ...o_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a | 0 ...o_join4-3-dc967001beb776f3a859e9360823c361 | 0 ...o_join4-4-998c3a307b074a6505bb7fcef276be04 | 1 + ...o_join5-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...o_join5-1-531e91e11b3891627c2675935fda14cd | 0 ...o_join5-2-a13b6523395e55c551ad42d92f9dbcd6 | 0 ...o_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 | 0 ...o_join5-4-998c3a307b074a6505bb7fcef276be04 | 1 + ...o_join6-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...o_join6-1-531e91e11b3891627c2675935fda14cd | 0 ...o_join6-2-46718fdfa123cc86fe288bff4185dc90 | 0 ...o_join6-3-bc6b6640b266ebe9b73702d3baf09c20 | 0 ...o_join6-4-998c3a307b074a6505bb7fcef276be04 | 1 + ...o_join7-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...o_join7-1-8f82881057bec4abf5a4d770a6f35838 | 0 ...o_join7-2-8a7f50dc7e382a11884f4e116041aa1e | 0 ...o_join7-3-56a30a1aa948bcf5ee54481897fc2208 | 0 ...o_join7-4-30d0c1a49784347fedbac21a69c3a899 | 1 + ...o_join8-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...o_join8-1-531e91e11b3891627c2675935fda14cd | 0 ...o_join8-2-cdb9885fe05daa90c228cf5b6550eeab | 0 ...o_join8-3-f1196bca86a749375da35f134206a8ca | 0 ...o_join8-4-998c3a307b074a6505bb7fcef276be04 | 1 + ...o_join9-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...o_join9-1-13ab74a58da514fe01dbeda0c3e79883 | 0 ...o_join9-2-62638666bf7f60c0d298547ea5e93ea7 | 0 ...o_join9-3-971c44e81ce17eb0849850b72ebd20f1 | 0 ...o_join9-4-eaa70da463b92e85e1796277f016c18f | 1 + ...filters-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...filters-1-5644ab44e5ba9f2941216b8d5dc33a99 | 0 ...ilters-10-b420f24d33b26cdf6c35eb702789904e | 1 + ...ilters-11-c06b548171893bae8def6bb348b70dc8 | 1 + ...ilters-12-42a4901e05e9ee92abcfcef008efaa65 | 1 + ...ilters-13-222c404c6265ed682579342113221e29 | 1 + ...ilters-14-ecd2885156f56973960d064211ee42f0 | 1 + ...ilters-15-11a471880f5e8fbad81e3869fe56ca4b | 1 + ...ilters-16-f26ad8c3537dc391ab1ca6a95470f75e | 1 + ...ilters-17-8e085f9886e5ee97334512f84bd7ab54 | 1 + ...ilters-18-d9438071b3c731dc3f6e3b7248a1042a | 1 + ...ilters-19-58355bd5c4b12e15cf1d3e2d8b308c9d | 1 + ...filters-2-100faa7fd01bfb4390c782bb262a9403 | 0 ...ilters-20-486a302359aecff37a4567480264bd62 | 1 + ...filters-21-8018df5b9572e89304b449e618fdbbf | 1 + ...ilters-22-f9a7bff2b42a03d21d8d3190d2702451 | 1 + ...ilters-23-2b5149e29c224a47af98ca10bc5c23b3 | 1 + ...ilters-24-a2161a58f512b8bdd836b48cd8e16668 | 1 + ...ilters-25-1b3177a066ba352539fd5473fbeda1a9 | 1 + ...ilters-26-ecfb8fe4bf85a05f321754d8ea8cbabd | 1 + ...ilters-27-ca73726ffaa5826a3db039cda440e6d9 | 1 + ...filters-28-c21c2acf7f276c0a26f0c19e3234506 | 1 + ...ilters-29-6d8955591f62d9cfc6af17df63d3d88e | 0 ...filters-3-64615cc6839c697c8c028ef8bb1ac40e | 1 + ...ilters-30-23ab7ac8229a53d391195be7ca092429 | 0 ...ilters-31-aa161b0d9fe9d1aad10654fce0e3670b | 0 ...ilters-32-3c52df82c7d78501610f3f898103f753 | 0 ...ilters-33-1d85bb008e02ef4025171a4bc0866a6c | 0 ..._filters-34-e79c906b894fed049ddfab4496a4e3 | 0 ...ilters-35-3e6612a89e9124592e790594775054b1 | 1 + ...ilters-36-60a5f56f33fc8854a2b687005f0d96ac | 1 + ...ilters-37-64cabe5164130a94f387288f37b62d71 | 1 + ...ilters-38-65c867e66bc773470f1487487086a180 | 1 + ...ilters-39-cc0cf9ff9ccbc44536b3187b27aa1ada | 1 + ...filters-4-7df3e2e761c272ddb3654e4dd86bd131 | 1 + ...filters-40-4dfa64337f711dc394a289adeac8666 | 1 + ...ilters-41-a905633bccd782f14115643b2d707b13 | 1 + ...filters-42-ae4cc72ddbbbd748179e0abcc985726 | 1 + ...ilters-43-f64cec45b154c4ba5172afcdff623a2b | 1 + ...ilters-44-c552dcc9b931dff05cf8c0d712e22841 | 1 + ...ilters-45-b420f24d33b26cdf6c35eb702789904e | 1 + ...ilters-46-c06b548171893bae8def6bb348b70dc8 | 1 + ...ilters-47-42a4901e05e9ee92abcfcef008efaa65 | 1 + ...ilters-48-222c404c6265ed682579342113221e29 | 1 + ...ilters-49-ecd2885156f56973960d064211ee42f0 | 1 + ...filters-5-87cf8865e2c35b680bba159b88b074bc | 1 + ...ilters-50-11a471880f5e8fbad81e3869fe56ca4b | 1 + ...ilters-51-f26ad8c3537dc391ab1ca6a95470f75e | 1 + ...ilters-52-8e085f9886e5ee97334512f84bd7ab54 | 1 + ...ilters-53-d9438071b3c731dc3f6e3b7248a1042a | 1 + ...ilters-54-58355bd5c4b12e15cf1d3e2d8b308c9d | 1 + ...ilters-55-486a302359aecff37a4567480264bd62 | 1 + ...filters-56-8018df5b9572e89304b449e618fdbbf | 1 + ...ilters-57-f9a7bff2b42a03d21d8d3190d2702451 | 1 + ...ilters-58-2b5149e29c224a47af98ca10bc5c23b3 | 1 + ...ilters-59-a2161a58f512b8bdd836b48cd8e16668 | 1 + ...filters-6-9362466c777fff7e677dd8da072f8744 | 1 + ...ilters-60-1b3177a066ba352539fd5473fbeda1a9 | 1 + ...ilters-61-ecfb8fe4bf85a05f321754d8ea8cbabd | 1 + ...ilters-62-ca73726ffaa5826a3db039cda440e6d9 | 1 + ...filters-63-c21c2acf7f276c0a26f0c19e3234506 | 1 + ..._filters-7-ae4cc72ddbbbd748179e0abcc985726 | 1 + ...filters-8-f64cec45b154c4ba5172afcdff623a2b | 1 + ...filters-9-c552dcc9b931dff05cf8c0d712e22841 | 1 + ...n_nulls-0-ce1ef910fff98f174931cc641f7cef3a | 1 + ...n_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 | 0 ..._nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 | 1 + ..._nulls-11-141c550a314d15c4e200e5baeb246de2 | 1 + ..._nulls-12-8a65225a88da0169af26848c06cb981c | 1 + ..._nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 | 1 + ..._nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 | 1 + ..._nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 | 1 + ..._nulls-16-dbe244d2c21e477c3703c4ce1903e8af | 1 + ..._nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 | 1 + ..._nulls-18-439a409bc50dfd86dee78c151c3de5eb | 1 + ..._nulls-19-92641e46934ebbf3d44e6e60de1882f4 | 1 + ...n_nulls-2-97641998eb9ddf2dff56de6758486aa0 | 0 ..._nulls-20-e34b2b210059a5f93c0a873d91859b5d | 1 + ..._nulls-21-c7aaa831acbd959c6d1793056e3c288a | 1 + ..._nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 | 1 + ..._nulls-23-5805a41289a26139c06604a40bf5a6fa | 1 + ..._nulls-24-80991af26d5d37e0864ecc2c8ab0b984 | 1 + ...n_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 | 1 + ...in_nulls-4-fc1128c86cd430db8cd4ff834be4562 | 1 + ...n_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc | 1 + ...n_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 | 1 + ...n_nulls-7-8395fa78507105c2a018e88f717b95e2 | 1 + ...n_nulls-8-fd992f2127a139aeb554d797e748ed54 | 1 + ...n_nulls-9-e3a86622a437e910b7225d1e6108da9e | 1 + ..._values-0-1d8e3d660bb4b29287df5700bfe63b63 | 0 ..._values-1-1247d9a5ffabd61647697dc186c7a2a2 | 0 ...values-10-f6d5bb38137da35b91da901ba310c2b8 | 0 ..._values-2-3ce329282fc72110e9ed6c78fa914395 | 0 ..._values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 | 0 ..._values-4-7fb82039c95389f11b174d9f22aacb35 | 0 ..._values-5-b5da89b0cb325cf684406b620eb9d8ee | 0 ..._values-6-2c91dc4b7c00d5f09862119c12295532 | 0 ..._values-7-880ba1dba6057dd6cde89d1b17724a6b | 0 ..._values-8-950af86c321a67ab3ed0fa5b63ea6aed | 1 + ..._values-9-ae02756bd46266ec7fd9cc809bc4757b | 0 ...join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...join_14-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...oin_14-10-d0ec6d66ff349db09fd455eec149efdb | 1 + ...oin_14-11-906a582602602372e1d4776243abeab5 | 0 ...oin_14-12-94538bc7322522a5534cafc0551d2189 | 1 + ...oin_14-13-1d603e61c2cb888499504ddab98ccc65 | 0 ...oin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 | 1 + ...oin_14-15-758d5532083d6279e169b54fd69bb580 | 0 ...oin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 | 6 + ...oin_14-17-ca77b5fb54de526972c16ad6118e86d6 | 0 ...oin_14-18-21269869cd3aaf4ade2170d9017de018 | 1 + ...oin_14-19-853c4fa5a2385b92fdb39d0ac2926973 | 0 ...pjoin_14-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...oin_14-20-4e0e8cd0626a84b21ca7d2f633623578 | 1 + ...oin_14-21-e210ced77b32cd7ce87044bb3e3370d1 | 0 ...oin_14-22-2fe7b834b341bf18e36cd79dd00ec16a | 1 + ...oin_14-23-63d105e0acd3521bb29ba8cec9ac4583 | 0 ...oin_14-24-43ba2c72db9db1ec18d835ec978f8da1 | 1 + ...oin_14-25-7b2a1128afe35706f1540bfc251d0736 | 0 ...oin_14-26-b66c416fdd98d76981f19e9c14b6a562 | 1 + ...oin_14-27-5438118dc1d9fab501a8e60eddd625a2 | 0 ...oin_14-28-b889b147255231f7fe44bd57e1f8ba66 | 1 + ...oin_14-29-ff6eca271c60cb15a3ea2395ac737b0d | 0 ...join_14-3-4938d4d724990d16336ee31f0390c7da | 0 ...oin_14-30-b9d66e78b8898a97a42d1118300fa0ce | 1 + ...oin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce | 0 ...oin_14-32-b0ca9e20cd48457e6cf1c313d5505213 | 1 + ...oin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 | 0 ...join_14-34-76ded9e08c765bf2e1b670b4ffb938b | 0 ...oin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 | 0 ...oin_14-36-1ba279881865c861a793797ae84a3934 | 0 ...oin_14-37-a45927057c01fd54818b5dd50e77f60e | 22 + ...oin_14-38-a988727daa49fb5e190f81c027bb7005 | 22 + ...oin_14-39-20c26228d10872eec10dbb9322dd74da | 0 ...join_14-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...oin_14-40-4c57b1c6c081294cbd72626ff0fd940e | 0 ...oin_14-41-81b3db8d18d4b3843ed0be6eca5d793c | 0 ...oin_14-42-80db3a67d59c8710edf9f695e7eeb37c | 0 ...oin_14-43-a45927057c01fd54818b5dd50e77f60e | 22 + ...oin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 | 6 + ...join_14-5-9140b367b5680860f4c7c0238377583f | 0 ...join_14-6-70c1d66123d434d3a1e1801e0b19bc3f | 0 ...join_14-7-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_14-8-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_14-9-b89ea2173180c8ae423d856f943e061f | 1 + ..._join_1-0-b24f5a262c6693f31ed376a5da0787f3 | 0 ...e_join_1-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...join_1-10-9666fb18356436e2800550df9ca90c04 | 0 ...join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 | 0 ...join_1-12-24ca942f094b14b92086305cc125e833 | 1 + ...join_1-13-d0ec6d66ff349db09fd455eec149efdb | 1 + ...join_1-14-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_1-15-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_1-16-cda81d86d127fca0e2fbc2161e91400d | 1 + ...join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc | 1 + ...join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...join_1-19-325432a220aa3ebe8b816069916924d8 | 1 + ..._join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...join_1-21-7d0c37fc09323ce11aae0b58dc687660 | 1 + ...join_1-22-f135547e33c01d1f543c8b1349d60348 | 1 + ...join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...join_1-24-7d0c37fc09323ce11aae0b58dc687660 | 1 + ..._join_1-3-18bd222285d3a3bd71d3cfa217d9b1db | 0 ..._join_1-4-bb969d3ec0038215a2698afceeb02b3a | 0 ..._join_1-5-2c3617157639fcd296a8ea2f121c58ab | 0 ..._join_1-6-307339322d96b8f923d57c0dc9cdcb60 | 0 ..._join_1-7-94cc219f61413ab321916821e1288152 | 0 ..._join_1-8-310c8d652c6f549b7759baec6012b77d | 0 ..._join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...join_10-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...oin_10-10-b89ea2173180c8ae423d856f943e061f | 1 + ...oin_10-11-d0ec6d66ff349db09fd455eec149efdb | 1 + ...oin_10-12-cda81d86d127fca0e2fbc2161e91400d | 1 + ...oin_10-13-469a09efa93fa9aec154a5967eec09c5 | 0 ...oin_10-14-95e18bd00f2de246efca1756681c1e87 | 1 + ...join_10-15-6a7fdb423721e7aefa2efda26785e1a | 0 ...join_10-16-caa641c820fcc5f601758c5f0385b4e | 1 + ..._join_10-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_10-3-4938d4d724990d16336ee31f0390c7da | 0 ...join_10-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...join_10-5-9140b367b5680860f4c7c0238377583f | 0 ...join_10-6-70c1d66123d434d3a1e1801e0b19bc3f | 0 ...join_10-7-24ca942f094b14b92086305cc125e833 | 1 + ...join_10-8-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_10-9-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_11-0-4705fafa08c6d927aa01337e19605c8a | 0 ..._join_11-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...oin_11-10-9666fb18356436e2800550df9ca90c04 | 0 ...oin_11-11-a54cefeeb6d79c72f01c61035e9dcf15 | 0 ...oin_11-12-24ca942f094b14b92086305cc125e833 | 1 + ...oin_11-13-398b81a1928284f29e832838ec3764fd | 0 ...oin_11-14-325432a220aa3ebe8b816069916924d8 | 1 + ...oin_11-15-d0ec6d66ff349db09fd455eec149efdb | 1 + ...oin_11-16-c23ea191ee4d60c0a6252ce763b1beed | 1 + ...join_11-17-442e69416faaea9309bb8c2a3eb73ef | 1 + ...oin_11-18-d8260daa82c8439e0c80a63998bd5d2e | 0 ...oin_11-19-325432a220aa3ebe8b816069916924d8 | 1 + ...join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...oin_11-20-62fab16c00f510c001f146c929360c71 | 0 ...oin_11-21-c4d55c247c9326f474d89b29b81d60aa | 1 + ...join_11-3-b4a6a67ac771394140ed695810930ac6 | 0 ...join_11-4-bb969d3ec0038215a2698afceeb02b3a | 0 ...join_11-5-2c3617157639fcd296a8ea2f121c58ab | 0 ...join_11-6-307339322d96b8f923d57c0dc9cdcb60 | 0 ...join_11-7-94cc219f61413ab321916821e1288152 | 0 ...join_11-8-310c8d652c6f549b7759baec6012b77d | 0 ...join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec | 0 ..._join_12-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...oin_12-10-9666fb18356436e2800550df9ca90c04 | 0 ...oin_12-11-a54cefeeb6d79c72f01c61035e9dcf15 | 0 ...oin_12-12-24ca942f094b14b92086305cc125e833 | 1 + ...oin_12-13-d0ec6d66ff349db09fd455eec149efdb | 1 + ...oin_12-14-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...oin_12-15-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e | 0 ...oin_12-17-a8c60901367229310f86a8521a26478a | 0 ...oin_12-18-f50f21c997d775c369fd52f9bafb9b36 | 0 ...oin_12-19-fe49b6f198661d2e020a0c8bd26c9237 | 0 ...join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...oin_12-20-3404f0b912c898d6c81aa88bf0cd8c11 | 0 ...oin_12-21-4ecd65f0e26e981b66770b3e91e128fc | 1 + ...join_12-3-18bd222285d3a3bd71d3cfa217d9b1db | 0 ...join_12-4-bb969d3ec0038215a2698afceeb02b3a | 0 ...join_12-5-2c3617157639fcd296a8ea2f121c58ab | 0 ...join_12-6-307339322d96b8f923d57c0dc9cdcb60 | 0 ...join_12-7-94cc219f61413ab321916821e1288152 | 0 ...join_12-8-310c8d652c6f549b7759baec6012b77d | 0 ...join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...join_13-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...oin_13-10-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...oin_13-11-cda81d86d127fca0e2fbc2161e91400d | 1 + ...oin_13-12-d0ec6d66ff349db09fd455eec149efdb | 1 + ...oin_13-13-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...oin_13-14-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_13-15-24ca942f094b14b92086305cc125e833 | 1 + ...oin_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 | 0 ...oin_13-17-b366dcf84021a7dc4a17a52fe381b5f0 | 0 ...oin_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 | 22 + ...oin_13-19-4b2ac2865384fbca7f374191d8021d51 | 22 + ..._join_13-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...oin_13-20-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...oin_13-21-ea23403b9eb55e8b06d1c198e439569f | 1 + ...oin_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 | 0 ...oin_13-23-b366dcf84021a7dc4a17a52fe381b5f0 | 0 ...oin_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 | 22 + ...oin_13-25-4b2ac2865384fbca7f374191d8021d51 | 22 + ...oin_13-26-f135547e33c01d1f543c8b1349d60348 | 1 + ...oin_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 | 0 ...oin_13-28-b366dcf84021a7dc4a17a52fe381b5f0 | 0 ...oin_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 | 22 + ...join_13-3-4938d4d724990d16336ee31f0390c7da | 0 ...oin_13-30-4b2ac2865384fbca7f374191d8021d51 | 22 + ...join_13-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...join_13-5-d73046d4785e9c89acb10eea77d32ca8 | 0 ...join_13-6-3d02238197b076b6f77daacb81aa2cb4 | 0 ...join_13-7-6f949602369ac3af6ded9884bc525310 | 0 ..._join_13-8-b334c03af5acdcb136072bb427683bb | 0 ...join_13-9-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...join_14-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...oin_14-10-d0ec6d66ff349db09fd455eec149efdb | 1 + ...oin_14-11-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...oin_14-12-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_14-13-24ca942f094b14b92086305cc125e833 | 1 + ...oin_14-14-51e4c81f56c64f6aa25322055694f641 | 0 ...oin_14-15-43ad2152b18d711adbdd1aeb940b662a | 1 + ...oin_14-16-51ee88184cbc22b5bef4e96856e41e7c | 0 ...join_14-17-334529f1a720bfb408efee90bc8be61 | 0 ...oin_14-18-a16ff76d72ad428fb9d43ab910f259fd | 1 + ..._join_14-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_14-3-4938d4d724990d16336ee31f0390c7da | 0 ...join_14-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...join_14-5-945b37381c2719e18e2945bf8b4e56ac | 0 ...join_14-6-3d02238197b076b6f77daacb81aa2cb4 | 0 ...join_14-7-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_14-8-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_14-9-f135547e33c01d1f543c8b1349d60348 | 1 + ...join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...join_15-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...oin_15-10-d0ec6d66ff349db09fd455eec149efdb | 1 + ...oin_15-11-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...oin_15-12-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_15-13-24ca942f094b14b92086305cc125e833 | 1 + ...oin_15-14-62b7e43463386c11e031cf7e4f584a53 | 0 ...oin_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 | 0 ..._join_15-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_15-3-4938d4d724990d16336ee31f0390c7da | 0 ...join_15-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...join_15-5-945b37381c2719e18e2945bf8b4e56ac | 0 ...join_15-6-3d02238197b076b6f77daacb81aa2cb4 | 0 ...join_15-7-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_15-8-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_15-9-f135547e33c01d1f543c8b1349d60348 | 1 + ..._join_2-0-ac562e10c3d4dd7c7cce920d29cde65d | 0 ...e_join_2-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...join_2-10-24ca942f094b14b92086305cc125e833 | 1 + ...join_2-11-d0ec6d66ff349db09fd455eec149efdb | 1 + ...join_2-12-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_2-13-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_2-14-cda81d86d127fca0e2fbc2161e91400d | 1 + ...join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 | 1 + ...join_2-16-af6016f3db000e6e180e2f3b10f120ce | 0 ...join_2-17-7d0c37fc09323ce11aae0b58dc687660 | 1 + ...join_2-18-f135547e33c01d1f543c8b1349d60348 | 1 + ...join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 | 1 + ..._join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...join_2-20-8180638a57b64557e02815c863031755 | 1 + ...join_2-21-e6283ea14d493b0d7bf390249665f289 | 0 ...join_2-22-7d0c37fc09323ce11aae0b58dc687660 | 1 + ..._join_2-3-55c1e972192580d734fad7f57dd62e6a | 0 ..._join_2-4-210f08b7e8c20c9ff364c215af412d87 | 0 ..._join_2-5-bfc04dda7e11f06d01689f2b57959ed7 | 0 ..._join_2-6-bb969d3ec0038215a2698afceeb02b3a | 0 ..._join_2-7-2c3617157639fcd296a8ea2f121c58ab | 0 ..._join_2-8-310c8d652c6f549b7759baec6012b77d | 0 ..._join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ..._join_3-0-71378da1900d130fd68aaebc45f87313 | 0 ...e_join_3-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...join_3-10-24ca942f094b14b92086305cc125e833 | 1 + ...join_3-11-d0ec6d66ff349db09fd455eec149efdb | 1 + ...join_3-12-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_3-13-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_3-14-cda81d86d127fca0e2fbc2161e91400d | 1 + ...join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc | 1 + ...join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...join_3-17-325432a220aa3ebe8b816069916924d8 | 1 + ...join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...join_3-19-7d0c37fc09323ce11aae0b58dc687660 | 1 + ..._join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...join_3-20-f135547e33c01d1f543c8b1349d60348 | 1 + ...join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...join_3-22-7d0c37fc09323ce11aae0b58dc687660 | 1 + ..._join_3-3-6876517daaf54cadefb6bbbf54bd4a24 | 0 ..._join_3-4-cd25b8502c668759783aaba4d550a05f | 0 ..._join_3-5-18bd222285d3a3bd71d3cfa217d9b1db | 0 ..._join_3-6-bb969d3ec0038215a2698afceeb02b3a | 0 ..._join_3-7-2c3617157639fcd296a8ea2f121c58ab | 0 ..._join_3-8-307339322d96b8f923d57c0dc9cdcb60 | 0 ..._join_3-9-94cc219f61413ab321916821e1288152 | 0 ..._join_4-0-1528e7173b91cd90f101ca27f51d963c | 0 ...e_join_4-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...join_4-10-bb969d3ec0038215a2698afceeb02b3a | 0 ...join_4-11-2c3617157639fcd296a8ea2f121c58ab | 0 ...join_4-12-24ca942f094b14b92086305cc125e833 | 1 + ...join_4-13-d0ec6d66ff349db09fd455eec149efdb | 1 + ...join_4-14-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_4-15-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_4-16-cda81d86d127fca0e2fbc2161e91400d | 1 + ...join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 | 1 + ...join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...join_4-19-325432a220aa3ebe8b816069916924d8 | 1 + ..._join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...join_4-21-7d0c37fc09323ce11aae0b58dc687660 | 1 + ...join_4-22-f135547e33c01d1f543c8b1349d60348 | 1 + ...join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...join_4-24-7d0c37fc09323ce11aae0b58dc687660 | 1 + ..._join_4-3-55c1e972192580d734fad7f57dd62e6a | 0 ..._join_4-4-210f08b7e8c20c9ff364c215af412d87 | 0 ..._join_4-5-6876517daaf54cadefb6bbbf54bd4a24 | 0 ..._join_4-6-cd25b8502c668759783aaba4d550a05f | 0 ..._join_4-7-c20aa9939d703c529c4538994dc6f066 | 0 ..._join_4-8-baa1253610c081917208199feb52a768 | 0 ..._join_4-9-bfc04dda7e11f06d01689f2b57959ed7 | 0 ..._join_5-0-9f8764dddb7b106f879d1a7c4318310d | 0 ..._join_5-1-fac79d1e5c34142393fc328b2935a9b8 | 0 ...join_5-10-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_5-11-cda81d86d127fca0e2fbc2161e91400d | 1 + ...join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 | 1 + ...join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...join_5-14-325432a220aa3ebe8b816069916924d8 | 1 + ...join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...join_5-16-7d0c37fc09323ce11aae0b58dc687660 | 1 + ...join_5-17-f135547e33c01d1f543c8b1349d60348 | 1 + ...join_5-18-24ca942f094b14b92086305cc125e833 | 1 + ...join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ..._join_5-2-7282f71445d3b6acef073be9b7cbab98 | 0 ...join_5-20-7d0c37fc09323ce11aae0b58dc687660 | 1 + ..._join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c | 0 ..._join_5-4-be71f06ad593935a8e81d61b695b2052 | 0 ..._join_5-5-ce5ee903a36a074293fa509149d94447 | 0 ..._join_5-6-12e01dc9146f45ded0a6655cb04467b4 | 0 ..._join_5-7-ec1aaae06a8dbb2faf36b53246124d4a | 0 ..._join_5-8-d0ec6d66ff349db09fd455eec149efdb | 1 + ..._join_5-9-3b0f76816be2c1b18a2058027a19bc9f | 1 + ..._join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._join_6-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...join_6-10-34779e6a90b2e9968a9a98b048cdaab6 | 0 ...join_6-11-d0ec6d66ff349db09fd455eec149efdb | 1 + ...join_6-12-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_6-13-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_6-14-24ca942f094b14b92086305cc125e833 | 1 + ...join_6-15-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...join_6-16-ea23403b9eb55e8b06d1c198e439569f | 1 + ...join_6-17-cda81d86d127fca0e2fbc2161e91400d | 1 + ..._join_6-18-32efb3656e05e40f9f928bbcb11d010 | 0 ...join_6-19-cf41f7ce9478536e823107d1810ff1d7 | 1 + ...e_join_6-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_6-20-b23f9ec74e58e5c53417bfff6794e8fd | 0 ...join_6-21-b55506a213ec710004e6d7f3462834d0 | 1 + ...join_6-22-4b066e39be51ea19a1790c1287ad0d2c | 0 ...join_6-23-4281442c87dcf6007f8bd42504eba186 | 1 + ...join_6-24-12ec3636a4c62fd56b40da2979f53f5f | 0 ...join_6-25-52f0e65724d29e2b4054b59a50d2837b | 1 + ...join_6-26-bf7478a041a164ef219964cb865aa63b | 0 ...join_6-27-961f141836f2cc9521f681cadbc3d140 | 1 + ..._join_6-28-171974ff7145ffc85c8ba2724ef1f31 | 0 ...join_6-29-fd0cc412e0987569a4ed879454b53fb0 | 1 + ..._join_6-3-4938d4d724990d16336ee31f0390c7da | 0 ...join_6-30-f135547e33c01d1f543c8b1349d60348 | 1 + ...join_6-31-940f423a57afd2734f62d93bcd4d7caf | 0 ...join_6-32-cf41f7ce9478536e823107d1810ff1d7 | 1 + ...join_6-33-12ec3636a4c62fd56b40da2979f53f5f | 0 ...join_6-34-52f0e65724d29e2b4054b59a50d2837b | 1 + ...join_6-35-bf7478a041a164ef219964cb865aa63b | 0 ...join_6-36-961f141836f2cc9521f681cadbc3d140 | 1 + ..._join_6-37-171974ff7145ffc85c8ba2724ef1f31 | 0 ...join_6-38-fd0cc412e0987569a4ed879454b53fb0 | 1 + ..._join_6-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ..._join_6-5-612f97716b8efe4b659206938e5ea5f2 | 0 ..._join_6-6-64856231335fc5fec61c3fd3aceefcc4 | 0 ..._join_6-7-b34c9b18481df043912e910ed3a5f149 | 0 ..._join_6-8-57e68163453d4632ef740ce1223f44d1 | 0 ..._join_6-9-36d2a698f88e53ab2a66e8baa980299b | 0 ...e_join_7-0-fa10661c7e8791fb319ade49f3cca50 | 0 ...e_join_7-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...join_7-10-bb969d3ec0038215a2698afceeb02b3a | 0 ...join_7-11-2c3617157639fcd296a8ea2f121c58ab | 0 ...join_7-12-310c8d652c6f549b7759baec6012b77d | 0 ...join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...join_7-14-24ca942f094b14b92086305cc125e833 | 1 + ...join_7-15-d0ec6d66ff349db09fd455eec149efdb | 1 + ...join_7-16-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_7-17-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_7-18-cda81d86d127fca0e2fbc2161e91400d | 1 + ...join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 | 1 + ..._join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...join_7-21-325432a220aa3ebe8b816069916924d8 | 1 + ...join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...join_7-23-7d0c37fc09323ce11aae0b58dc687660 | 1 + ...join_7-24-f135547e33c01d1f543c8b1349d60348 | 1 + ...join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...join_7-26-7d0c37fc09323ce11aae0b58dc687660 | 1 + ..._join_7-3-55c1e972192580d734fad7f57dd62e6a | 0 ..._join_7-4-210f08b7e8c20c9ff364c215af412d87 | 0 ..._join_7-5-6876517daaf54cadefb6bbbf54bd4a24 | 0 ..._join_7-6-cd25b8502c668759783aaba4d550a05f | 0 ..._join_7-7-c20aa9939d703c529c4538994dc6f066 | 0 ..._join_7-8-baa1253610c081917208199feb52a768 | 0 ..._join_7-9-bfc04dda7e11f06d01689f2b57959ed7 | 0 ..._join_8-0-42977c556a54aaaee9d1e59a6dcc06aa | 0 ...e_join_8-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...join_8-10-310c8d652c6f549b7759baec6012b77d | 0 ...join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...join_8-12-9666fb18356436e2800550df9ca90c04 | 0 ...join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 | 0 ...join_8-14-24ca942f094b14b92086305cc125e833 | 1 + ...join_8-15-d0ec6d66ff349db09fd455eec149efdb | 1 + ...join_8-16-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_8-17-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_8-18-cda81d86d127fca0e2fbc2161e91400d | 1 + ...join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc | 1 + ..._join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...join_8-21-325432a220aa3ebe8b816069916924d8 | 1 + ...join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...join_8-23-7d0c37fc09323ce11aae0b58dc687660 | 1 + ...join_8-24-f135547e33c01d1f543c8b1349d60348 | 1 + ...join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 | 1 + ...join_8-26-8180638a57b64557e02815c863031755 | 1 + ...join_8-27-e6283ea14d493b0d7bf390249665f289 | 0 ...join_8-28-7d0c37fc09323ce11aae0b58dc687660 | 1 + ..._join_8-3-6876517daaf54cadefb6bbbf54bd4a24 | 0 ..._join_8-4-cd25b8502c668759783aaba4d550a05f | 0 ..._join_8-5-18bd222285d3a3bd71d3cfa217d9b1db | 0 ..._join_8-6-bb969d3ec0038215a2698afceeb02b3a | 0 ..._join_8-7-2c3617157639fcd296a8ea2f121c58ab | 0 ..._join_8-8-307339322d96b8f923d57c0dc9cdcb60 | 0 ..._join_8-9-94cc219f61413ab321916821e1288152 | 0 ..._join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._join_9-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...join_9-10-d0ec6d66ff349db09fd455eec149efdb | 1 + ...join_9-11-cda81d86d127fca0e2fbc2161e91400d | 1 + ...join_9-12-906a582602602372e1d4776243abeab5 | 0 ...join_9-13-94538bc7322522a5534cafc0551d2189 | 1 + ...join_9-14-133023474337f2cdc53ee82ffeb1c13e | 0 ...join_9-15-63261d35ddda973eeeb97b994ab7a476 | 6 + ...join_9-16-1d603e61c2cb888499504ddab98ccc65 | 0 ...join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 | 1 + ...join_9-18-758d5532083d6279e169b54fd69bb580 | 0 ...join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 | 6 + ...e_join_9-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_9-20-ca77b5fb54de526972c16ad6118e86d6 | 0 ...join_9-21-21269869cd3aaf4ade2170d9017de018 | 1 + ...join_9-22-853c4fa5a2385b92fdb39d0ac2926973 | 0 ...join_9-23-4e0e8cd0626a84b21ca7d2f633623578 | 1 + ...join_9-24-e210ced77b32cd7ce87044bb3e3370d1 | 0 ...join_9-25-2fe7b834b341bf18e36cd79dd00ec16a | 1 + ...join_9-26-63d105e0acd3521bb29ba8cec9ac4583 | 0 ...join_9-27-43ba2c72db9db1ec18d835ec978f8da1 | 1 + ..._join_9-28-5965c5c6ef08240eb27eb9620cc2338 | 0 ...join_9-29-b66c416fdd98d76981f19e9c14b6a562 | 1 + ..._join_9-3-4938d4d724990d16336ee31f0390c7da | 0 ...join_9-30-4376bdd8412f94fe184d46481fee345d | 0 ...join_9-31-b889b147255231f7fe44bd57e1f8ba66 | 1 + ...join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 | 0 ...join_9-33-aa8d713ad4e19b72b5bd7628d60c295e | 1 + ...join_9-34-ff6eca271c60cb15a3ea2395ac737b0d | 0 ...join_9-35-b9d66e78b8898a97a42d1118300fa0ce | 1 + ...join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce | 0 ...join_9-37-b0ca9e20cd48457e6cf1c313d5505213 | 1 + ...join_9-38-f135547e33c01d1f543c8b1349d60348 | 1 + ...join_9-39-906a582602602372e1d4776243abeab5 | 0 ..._join_9-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...join_9-40-94538bc7322522a5534cafc0551d2189 | 1 + ...join_9-41-133023474337f2cdc53ee82ffeb1c13e | 0 ...join_9-42-63261d35ddda973eeeb97b994ab7a476 | 6 + ...join_9-43-1d603e61c2cb888499504ddab98ccc65 | 0 ...join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 | 1 + ...join_9-45-758d5532083d6279e169b54fd69bb580 | 0 ...join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 | 6 + ...join_9-47-ca77b5fb54de526972c16ad6118e86d6 | 0 ...join_9-48-21269869cd3aaf4ade2170d9017de018 | 1 + ...join_9-49-853c4fa5a2385b92fdb39d0ac2926973 | 0 ..._join_9-5-9140b367b5680860f4c7c0238377583f | 0 ...join_9-50-4e0e8cd0626a84b21ca7d2f633623578 | 1 + ...join_9-51-e210ced77b32cd7ce87044bb3e3370d1 | 0 ...join_9-52-2fe7b834b341bf18e36cd79dd00ec16a | 1 + ...join_9-53-63d105e0acd3521bb29ba8cec9ac4583 | 0 ...join_9-54-43ba2c72db9db1ec18d835ec978f8da1 | 1 + ...join_9-55-4376bdd8412f94fe184d46481fee345d | 0 ...join_9-56-b889b147255231f7fe44bd57e1f8ba66 | 1 + ...join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 | 0 ...join_9-58-aa8d713ad4e19b72b5bd7628d60c295e | 1 + ...join_9-59-ff6eca271c60cb15a3ea2395ac737b0d | 0 ..._join_9-6-70c1d66123d434d3a1e1801e0b19bc3f | 0 ...join_9-60-b9d66e78b8898a97a42d1118300fa0ce | 1 + ...join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce | 0 ...join_9-62-b0ca9e20cd48457e6cf1c313d5505213 | 1 + ..._join_9-7-24ca942f094b14b92086305cc125e833 | 1 + ..._join_9-8-3b0f76816be2c1b18a2058027a19bc9f | 1 + ..._join_9-9-86473a0498e4361e4db0b4a22f2e8571 | 1 + ..._table1-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 ..._table1-1-7b43ffa8083fda74ab342029dce2e3d9 | 0 ..._table1-2-7e72160489bbb59dadf24e0cc239a5f4 | 4 + ..._table2-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 ..._table2-1-6f3e37cab4fdc60491dea1ff6fc9931a | 0 ..._table2-2-8491941c2baa0c7d96e17b8f47dfebe7 | 0 ..._table2-3-7e72160489bbb59dadf24e0cc239a5f4 | 4 + ..._table3-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 ..._table3-1-28b12606c5369c783e63c17826a18d0d | 0 ...le_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...le_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b | 0 ...e_union-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 ...e_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 | 0 ...e_union-2-7e72160489bbb59dadf24e0cc239a5f4 | 4 + ...onstant-0-8c922b2264278dd481ef0dff2088e2b8 | 1 + ..._format-0-483cdc9eade175b0c89b9f5b3eb505f1 | 0 ..._format-1-9e0909b6330578a25806527dd0ecf7ef | 0 ..._format-2-a42be5ce444ef1d2a1dbe654a57d6f55 | 0 ..._format-3-84db2ef4a7f2151e26457db559b862d9 | 500 ++++ ...olserde-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 ...olserde-1-3defb7199b65cfd10cb2f46275c581b2 | 0 ...olserde-2-706a062089583074f30fb13c661fc81e | 0 ...olserde-3-7e72160489bbb59dadf24e0cc239a5f4 | 4 + ...olserde-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 ...olserde-1-179ac81920d8dfa6e324cc881b5f1624 | 0 ...olserde-2-7e72160489bbb59dadf24e0cc239a5f4 | 4 + ...table_1-0-3562c2ed956a59cc98362d2f64e19ce1 | 0 ...table_1-1-4a0ed18480313e66b869ec4f49371cf5 | 0 ...table_1-2-faa8d95365e4116734a056c911350c05 | 0 ...table_1-3-d6518380547e7eef338886f3bdc7bdd2 | 10 + .../cast1-0-b0e8966b7c06be9e044ed30b487d0661 | 0 .../cast1-1-1ee16b8209701131017533cfa6eb4680 | 0 .../cast1-2-3fe73e7435e30b37266ef6a33537dc4c | 0 .../cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a | 1 + ..._to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 | 1 + ...cluster-0-16681f9c2bdd44278817d72c138b6ee1 | 0 ...cluster-1-707a2295731e0d631a6c5f71c745c8d5 | 1 + ...cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 | 0 ...luster-11-dcf78a6537ba2b4d4b828a9a27cf545e | 1 + ...luster-12-6ad920e2ae83c78fccb06ff65308a438 | 0 ...luster-13-12635b4b7f34eba5554d5b892b5b64e7 | 1 + ...luster-14-cd2e125bceb1611137f0750f5d69c475 | 0 ...luster-15-b4c15c85c18f310f1b5bc56a78ad94b0 | 1 + ...luster-16-a2d5e5ec2504041ea1a62856c7086451 | 0 ...luster-17-62979aa9e6b4e6ffb44ec452aabbef65 | 1 + ...luster-18-e01f450969ae7e1cd018e6ef0cc67141 | 0 ...luster-19-e5284c7a7c36ee55740bd127ef4bf8c7 | 1 + ...cluster-2-50197277eb03ad20696a135bd7d18de7 | 0 ...luster-20-294891c1d956245540a80aa800ba393d | 0 ...luster-21-4787b258a786cf195bcb59cd90f6013f | 1 + ...cluster-22-8801aa93cf6dba7e13e99a0260fde68 | 0 ...luster-23-b66ed6ead4deecd49f0f67de1f2bab2e | 1 + ...luster-24-f492a7f78faf180621e83e5a69aa1eae | 0 ...luster-25-f57ce48b6a6e671b58c96535ab482b6a | 498 ++++ ...cluster-3-530671e2a5b8983c60cfedaf486f0f0f | 1 + ...cluster-4-cb4af90f52f2626213f918fda3b81dfc | 0 ...cluster-5-e99040f6a24c53a4c89801ff3663ff72 | 1 + ...cluster-6-56f8e3e7abe504522a2bfd77b5be3270 | 0 ...cluster-7-a22600d60c81a25061b1e20b6726c691 | 1 + ...cluster-8-251b178e4fe39ea03a30d2b9bd40710d | 0 ...cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 | 1 + ...hortcut-0-d300f67f11082f3614a8e93e8808960d | 0 ...hortcut-1-3a1329c4145738961e1b8bdbd056497c | 0 ...shortcut-2-b4b94bc85ee3bdef2b458d974d36935 | 0 ...hortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f | 11 + ...partlvl-0-78efaffd8fd417371fb888d6d1ba995c | 0 ...partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 | 0 ...partlvl-2-eb06998a353abc3022a9e0a17d5dba59 | 0 ...partlvl-3-3ab5479f002e412965f259485075f6bd | 0 ..._partlvl-4-30d92b61681b9ae7786ed46c3c3e808 | 58 + ...partlvl-5-f37a302cb19b4fe1c8280c08153294a3 | 129 + ...partlvl-6-86ba38eff353a720bbabf726365b6712 | 0 ...partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 | 58 + ...partlvl-8-dc5682403f4154cef30860f2b4e37bce | 129 + ..._partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe | 0 ..._tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 | 0 ..._tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c | 0 ..._tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 | 0 ..._tbllvl-3-7c45bd1125420b85a0374fecbf947a95 | 73 + ..._tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 | 141 + ..._tbllvl-5-ebf2d35321a3af996c150c6072d16a8c | 0 ..._tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 | 0 ..._tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d | 89 + ..._tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f | 0 ...ombine1-0-84b74227c9f1563f530cd3ac3b333e54 | 1 + ...ombine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...ombine1-2-c95dc367df88c9e5cf77157f29ba2daf | 1 + ...ombine1-3-6e53a3ac93113f20db3a12f1dcf30e86 | 1 + ...ombine1-4-84967075baa3e56fff2a23f8ab9ba076 | 1 + ...ombine1-5-2ee5d706fe3a3bcc38b795f6e94970ea | 1 + ...ombine1-6-1d1f97cce07323812de3027920b04b75 | 1 + ...ombine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 | 0 ...ombine1-8-da1fda96db80592bf2bbda8f22b5687c | 0 ...ombine1-9-e5ce23369b0ad260512a0f61c6969b73 | 500 ++++ ...ombine2-0-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...ombine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...mbine2-10-54649f87d403c6fcb163c4d51e382d3a | 0 ...mbine2-11-2d2d73a929c7d995ea57b40529b74b56 | 0 ...mbine2-12-cd15ffd140539cf86090814729ec4748 | 8 + ...mbine2-13-5ac3e540fd24f94fee378e49597817b3 | 0 ...mbine2-14-4695309eb4e91ef29c9857aa8fd6130c | 12 + ...mbine2-15-dd652175dac4463fed3c56aded11e6c1 | 0 ...ombine2-16-557997716a68312e8cae75428e3ce31 | 1 + ...mbine2-17-8e4598e3f0701478ed12042438699ce5 | 0 ...mbine2-18-2af7419c1d84fe155e23f3972e049b97 | 2 + ...ombine2-2-c95dc367df88c9e5cf77157f29ba2daf | 1 + ...ombine2-3-6e53a3ac93113f20db3a12f1dcf30e86 | 1 + ...ombine2-4-84967075baa3e56fff2a23f8ab9ba076 | 1 + ...ombine2-5-2ee5d706fe3a3bcc38b795f6e94970ea | 1 + ...ombine2-6-a4fb8359a2179ec70777aad6366071b7 | 1 + ...ombine2-7-16367c381d4b189b3640c92511244bfe | 1 + ...ombine2-8-99d1f07b2ce904afd6a809fd1814efe9 | 1 + ...ombine2-9-30cb07965e4b5025545361b948fc83c2 | 1 + ...adoop20-0-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...adoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...doop20-10-a5c8b73241600b6e2af8b3a41f5f5055 | 0 ...doop20-11-e25fd9e055710371ec90e0730c92f272 | 0 ...doop20-12-cd15ffd140539cf86090814729ec4748 | 8 + ...doop20-13-5ac3e540fd24f94fee378e49597817b3 | 0 ...doop20-14-4695309eb4e91ef29c9857aa8fd6130c | 12 + ...doop20-15-dd652175dac4463fed3c56aded11e6c1 | 0 ...adoop20-16-557997716a68312e8cae75428e3ce31 | 1 + ...doop20-17-8e4598e3f0701478ed12042438699ce5 | 0 ...doop20-18-2af7419c1d84fe155e23f3972e049b97 | 2 + ...adoop20-2-c95dc367df88c9e5cf77157f29ba2daf | 1 + ...adoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 | 1 + ...adoop20-4-84967075baa3e56fff2a23f8ab9ba076 | 1 + ...adoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea | 1 + ...adoop20-6-a4fb8359a2179ec70777aad6366071b7 | 1 + ...adoop20-7-16367c381d4b189b3640c92511244bfe | 1 + ...adoop20-8-99d1f07b2ce904afd6a809fd1814efe9 | 1 + ...adoop20-9-30cb07965e4b5025545361b948fc83c2 | 1 + ...ne2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...ne2_win-1-c95dc367df88c9e5cf77157f29ba2daf | 1 + ...e2_win-10-66ee1339a5a8cc224e83d583acc709c4 | 0 ...e2_win-11-cd15ffd140539cf86090814729ec4748 | 8 + ...e2_win-12-5ac3e540fd24f94fee378e49597817b3 | 0 ...e2_win-13-4695309eb4e91ef29c9857aa8fd6130c | 12 + ...e2_win-14-dd652175dac4463fed3c56aded11e6c1 | 0 ...ne2_win-15-557997716a68312e8cae75428e3ce31 | 1 + ...e2_win-16-8e4598e3f0701478ed12042438699ce5 | 0 ...e2_win-17-2af7419c1d84fe155e23f3972e049b97 | 2 + ...ne2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 | 1 + ...ne2_win-3-84967075baa3e56fff2a23f8ab9ba076 | 1 + ...ne2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea | 1 + ...ne2_win-5-a4fb8359a2179ec70777aad6366071b7 | 1 + ...ne2_win-6-16367c381d4b189b3640c92511244bfe | 1 + ...ne2_win-7-99d1f07b2ce904afd6a809fd1814efe9 | 1 + ...ne2_win-8-30cb07965e4b5025545361b948fc83c2 | 1 + ...ne2_win-9-7fd472d5ba7a41eb391f723c6dcf42af | 0 ...ombine3-0-84b74227c9f1563f530cd3ac3b333e54 | 1 + ...ombine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...mbine3-10-fb173ed4483e732d367e51f88be793b1 | 0 ...ombine3-11-30b8b79049113252bec1cbeac4018a3 | 12 + ...mbine3-12-f4baee25e0ad813258d67d707a6fc43b | 12 + ...mbine3-13-1c359bedf474e8e26f3b3562f7af6edc | 30 + ...mbine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...ombine3-15-7cccbdffc32975f8935eeba14a28147 | 1 + ...mbine3-16-6635f7f5c55557b06ad3acc321eaa739 | 0 ...mbine3-17-8cb751103da7c909276db6bddb50ae6a | 0 ...mbine3-18-31fae7c6be75b97d475512bd75a58a0b | 0 ...mbine3-19-e30d6cd92dc5a7a86fb2b9154497b04f | 0 ...ombine3-2-c95dc367df88c9e5cf77157f29ba2daf | 1 + ...mbine3-20-5bd4bb8b05f31b14bbc59287dff01ffd | 0 ...mbine3-21-8ba8f8723c1530be062cefc2d9246e56 | 30 + ...mbine3-22-11025483569617a9f014b5defd71e933 | 0 ...mbine3-23-4725c48df09565618cbffd05953a5f62 | 0 ...ombine3-3-6e53a3ac93113f20db3a12f1dcf30e86 | 1 + ...ombine3-4-84967075baa3e56fff2a23f8ab9ba076 | 1 + ...ombine3-5-2ee5d706fe3a3bcc38b795f6e94970ea | 1 + ...ombine3-6-4725c48df09565618cbffd05953a5f62 | 0 ...ombine3-7-53a5c509ebc9ee8458f27cc9bac46d00 | 0 ...ombine3-8-68399bc39f71ddc99ed09ed9d2fd897b | 0 ...ombine3-9-b5703b76bbe99c41cbb63582a09a6e69 | 0 ..._binary-0-16dcd4810ff82419cf1ae914d1860f21 | 0 ..._binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 | 0 ..._binary-2-c5a68f035051eef3e1c8d44d8b90017c | 1 + ..._binary-3-bcac92c4a17678873b01779e3d0e84e3 | 1 + ...boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 | 0 ...boolean-1-8300537a2a508b3390c3172cd69c69b5 | 0 ...boolean-2-fbea367698de65e22d4d660a518ea95e | 1 + ...boolean-3-a14d8a5835c94829271f9f463d96d83d | 1 + ..._double-0-76e8d4ba13c67a0834987b6dcd1d05ce | 0 ..._double-1-a23a25a680139ed823c77f3f9f486065 | 0 ..._double-2-8f988b757fd62f318f35447a3fd65452 | 1 + ..._double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 | 1 + ...y_table-0-12161b12442ad9b664b51e443fabaf5d | 0 ...y_table-1-1f720211105d9a6d611c5d378ee45ec0 | 1 + ...y_table-2-372df408a15de1e6f05e807a3aff223f | 1 + ...y_table-3-73f6626835884f34a47e8e78396bc25b | 1 + ...y_table-4-7f356d57c8c6125e4083f51ed4bae5cf | 1 + ...y_table-5-294a33b7c457eb7846335a4b1775ddc4 | 1 + ...ty_table-6-137180ebd2a072f08b5d849bdd9a464 | 1 + ...ts_long-0-2ee7f9e4c307417d4da2660e303c07c3 | 0 ...ts_long-1-a7bc730f9862198709539e35c0208248 | 0 ...ts_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 | 1 + ...ts_long-3-cf4bb755ade3079409e2251c7cd0118d | 1 + ..._string-0-3b932d28809fb8f40d81b4f8dfe2693b | 0 ..._string-1-3bddaed6594ed44249e4a30c43e83d1f | 0 ..._string-2-b87a68ae5ffa689bada75425169d131a | 1 + ...s_string-3-cea908dd41c78490990ee6b681d19fc | 1 + ..._string-0-f22bc1aaadc6f36ba36420073ea04543 | 0 ..._string-1-db089ff46f9826c7883198adacdfad59 | 27 + ...mizer11-0-b1e2ade89ae898650f0be4f796d8947b | 1 + ...imizer11-1-3a7f180f26dd2aec3ceab769f0cd965 | 0 ...izer11-10-b9d963d24994c47c3776dda6f7d3881f | 1 + ...izer11-11-f7918ee4d4941d3272e0262a750de700 | 0 ...izer11-12-5cefedc27b914d45b1512c92ad36c6e4 | 15 + ...izer11-13-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...izer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d | 0 ...izer11-15-5cefedc27b914d45b1512c92ad36c6e4 | 15 + ...mizer11-2-e148026f8994e22ca756c68753a0cc26 | 0 ...imizer11-3-b04195464e014cb47fd20a76b5f9ac0 | 0 ...mizer11-4-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer11-5-88d8dfbff4269af34724bf30ff4fec34 | 0 ...mizer11-6-fce66bdc5987a642f8f93471e62c1748 | 2 + ...mizer11-7-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer11-8-45d4d690886288ef04addbb659397ad1 | 0 ...mizer11-9-fce66bdc5987a642f8f93471e62c1748 | 2 + ...mizer14-0-aa047b3a8b40b68b93c4ad11e173c767 | 1 + ...mizer14-1-d0a93f40892e3894460553b443c77428 | 1 + ...izer14-10-a837e66f8c37cc3b2f6d3596b03a6eda | 37 + ...izer14-11-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...izer14-12-99f81dd0f33197c724eb58398542ff22 | 0 ...izer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 | 0 ...izer14-14-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer14-15-78fed7defb6154e01abbd97a0741adf | 0 ...mizer14-16-7ebe26e8a3620830e824b4099519395 | 37 + ...izer14-17-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer14-18-aa18035e1d8fdcedb91b76f9a32b11 | 0 ...mizer14-19-7ebe26e8a3620830e824b4099519395 | 37 + ...mizer14-2-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...izer14-20-b9d963d24994c47c3776dda6f7d3881f | 1 + ...izer14-21-e85444100b2e0c71b3d792e4bf1486d1 | 0 ...izer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 | 37 + ...izer14-23-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...izer14-24-8e88fd43a2c216b3409bee768425772c | 0 ...izer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 | 37 + ...mizer14-3-88b3974a7639097ed915402827e8941f | 0 ...mizer14-4-f58c909a1cbcbca3ea64bada41b0a18f | 0 ...mizer14-5-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer14-6-dcc6819f5848ff3d68b1d28c8787d41c | 0 ...mizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda | 37 + ...mizer14-8-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer14-9-e5f8709d75fbe813609cbdc8ed707489 | 0 ...mizer15-0-b1e2ade89ae898650f0be4f796d8947b | 1 + ...mizer15-1-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 | 0 ...mizer15-3-e149747103059314a9984235702b24b6 | 37 + ...mizer15-4-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer15-5-9914f44ecb6ae7587b62e5349ff60d04 | 1 + ...mizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd | 0 ...mizer15-7-e149747103059314a9984235702b24b6 | 37 + ...imizer2-0-b1e2ade89ae898650f0be4f796d8947b | 1 + ...imizer2-1-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer2-10-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer2-11-165752d0d250e5c9cddca50cf0c9cab | 0 ...mizer2-12-e6b368bfaea4d2838f8038b3bd29db06 | 1 + ...mizer2-13-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer2-14-b8a58e660d7416485c3d9a97d610bdfd | 0 ...mizer2-15-18f10d12e8bfa473a916c2f528500538 | 1 + ...mizer2-16-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer2-17-2a3e19b28e5262eb8c467e237df34421 | 0 ...mizer2-18-18f10d12e8bfa473a916c2f528500538 | 1 + ...mizer2-19-b9d963d24994c47c3776dda6f7d3881f | 1 + ...imizer2-2-354213872b92046d7aec97c9ba7b4e97 | 0 ...mizer2-20-6907d13fadd74e0df33c7a99c1de0baf | 0 ...mizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e | 1 + ...mizer2-22-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer2-23-38b273c68d644e801695d5414c0e119b | 0 ...mizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e | 1 + ...mizer2-25-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 | 0 ...mizer2-27-9b22dad2843cdc379d90687745561104 | 1 + ...mizer2-28-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer2-29-3781a20b72437434ef8fa7174edf36ab | 0 ...imizer2-3-d915fbdd493869aec42f548bdb66598d | 1 + ...mizer2-30-9b22dad2843cdc379d90687745561104 | 1 + ...mizer2-31-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer2-32-5ac93f83acfd31ce036381993eda303f | 0 ...mizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b | 1 + ...mizer2-34-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer2-35-996af7ef09a07d38d1a238b00c80da03 | 0 ...mizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b | 1 + ...imizer2-4-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 | 0 ...imizer2-6-d915fbdd493869aec42f548bdb66598d | 1 + ...imizer2-7-b9d963d24994c47c3776dda6f7d3881f | 1 + ...imizer2-8-74078f19dfe424f3211e6ce26de52152 | 0 ...imizer2-9-e6b368bfaea4d2838f8038b3bd29db06 | 1 + ...imizer3-0-b1e2ade89ae898650f0be4f796d8947b | 1 + ...imizer3-1-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer3-10-e3d5ff08760b877d49c0f10f63776325 | 1 + ...mizer3-11-b1e2ade89ae898650f0be4f796d8947b | 1 + ...mizer3-12-b9d963d24994c47c3776dda6f7d3881f | 1 + ...imizer3-13-11a7c4a1256b5f56dbf8f2722e24831 | 0 ...mizer3-14-1f13e03988991067d13a9f3f1b36fcf5 | 1 + ...mizer3-15-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer3-16-11a7c4a1256b5f56dbf8f2722e24831 | 0 ...mizer3-17-1f13e03988991067d13a9f3f1b36fcf5 | 1 + ...mizer3-18-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer3-19-24ca942f094b14b92086305cc125e833 | 1 + ...imizer3-2-9a7769de1ce7fc0b0fc46f17da287ace | 0 ...mizer3-20-4025759f01fa2169a061070319ee8bfe | 0 ...mizer3-21-1f13e03988991067d13a9f3f1b36fcf5 | 1 + ...imizer3-3-e3d5ff08760b877d49c0f10f63776325 | 1 + ...imizer3-4-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 | 0 ...imizer3-6-e3d5ff08760b877d49c0f10f63776325 | 1 + ...imizer3-7-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer3-8-24ca942f094b14b92086305cc125e833 | 1 + ...imizer3-9-c575bf5ba408caadb836d307b9971bea | 0 ...imizer4-0-d157f058f9e8659c4367e01c4da13579 | 0 ...imizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...mizer4-10-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer4-11-fb7cadb1f06690537178b2a04b1ee91e | 0 ...mizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e | 1 + ...mizer4-13-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer4-14-24ca942f094b14b92086305cc125e833 | 1 + ...mizer4-15-8db0d44941d0ce086e95088ef579c136 | 0 ...mizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e | 1 + ...mizer4-17-b1e2ade89ae898650f0be4f796d8947b | 1 + ...mizer4-18-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer4-19-5e241577196c76217ed9615fcbc76dcb | 0 ...timizer4-2-46c5eef67c57677810028451dd2b4d9 | 0 ...mizer4-20-6ad79a473a876923a247f0cb57bb4208 | 1 + ...mizer4-21-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer4-22-392b7eecdc93567cd0621e0efafbbaa8 | 0 ...mizer4-23-6ad79a473a876923a247f0cb57bb4208 | 1 + ...mizer4-24-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer4-25-18ec265534266497e1da68480cfe51cf | 0 ...mizer4-26-62a0fd05be48759c39f3c284458dde9b | 1 + ...mizer4-27-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer4-28-f14be583b8c69df924ac3bc4cf1761fb | 0 ...mizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf | 1 + ...imizer4-3-4abc4f450a58ccdd0df2e345f1276979 | 0 ...mizer4-30-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer4-31-7fea74857587a30456b095e20bc2bde1 | 0 ...mizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf | 1 + ...mizer4-33-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer4-34-fa4c05d7d8707d20d89338d744f51dd0 | 0 ...mizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 | 1 + ...mizer4-36-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 | 0 ...mizer4-38-85fff71b240d0e26bab91d670c1349f2 | 1 + ...mizer4-39-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer4-4-ee940bcfd73a883d16245ef746798e15 | 0 ...mizer4-40-ef5268865a18f57e15dc650a11527b11 | 0 ...mizer4-41-85fff71b240d0e26bab91d670c1349f2 | 1 + ...imizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 | 0 ...imizer4-6-b1e2ade89ae898650f0be4f796d8947b | 1 + ...imizer4-7-b9d963d24994c47c3776dda6f7d3881f | 1 + ...imizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 | 0 ...imizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e | 1 + ...imizer6-0-b1e2ade89ae898650f0be4f796d8947b | 1 + ...imizer6-1-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer6-10-25b1fe48ef86952fc7766fb74b63bf21 | 15 + ...mizer6-11-b1e2ade89ae898650f0be4f796d8947b | 1 + ...mizer6-12-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer6-13-f16444a052505377b7747949ffc2c028 | 0 ...mizer6-14-84463190baec77d61b287a071c8886db | 15 + ...mizer6-15-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer6-16-1cce9d13fab4b27d36c0fe705d239f68 | 0 ...mizer6-17-84463190baec77d61b287a071c8886db | 15 + ...mizer6-18-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer6-19-e7b8cde1cfc381901066753ecfa87ed4 | 0 ...imizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 | 0 ...mizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de | 15 + ...mizer6-21-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer6-22-a5e1c26011ddfbe79a886044dc17ea67 | 0 ...mizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de | 15 + ...mizer6-24-b1e2ade89ae898650f0be4f796d8947b | 1 + ...mizer6-25-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 | 0 ...mizer6-27-e149747103059314a9984235702b24b6 | 37 + ...mizer6-28-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd | 0 ...imizer6-3-25b1fe48ef86952fc7766fb74b63bf21 | 15 + ...mizer6-30-e149747103059314a9984235702b24b6 | 37 + ...mizer6-31-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer6-32-17c94297caaed15ea3ea2ea72d3508b7 | 0 ...mizer6-33-15d991127dc684513e2fff1aea3f1560 | 37 + ...mizer6-34-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer6-35-833150221aa5d9fbb0cb626bd8ce0762 | 0 ...mizer6-36-15d991127dc684513e2fff1aea3f1560 | 37 + ...mizer6-37-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer6-38-d1a903645384c97759e835649ce898fb | 0 ...mizer6-39-813d72763b5e9e3bed0f06232f55c8b8 | 37 + ...imizer6-4-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer6-40-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 | 0 ...mizer6-42-813d72763b5e9e3bed0f06232f55c8b8 | 37 + ...mizer6-43-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer6-44-ed339f429b82397bffb7298a534d59fb | 0 ...mizer6-45-ff6c54b483726ef15c90a4c68dc659a0 | 37 + ...mizer6-46-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer6-47-53940f5dd35ccbe9086533076c362bd4 | 0 ...mizer6-48-ff6c54b483726ef15c90a4c68dc659a0 | 37 + ...mizer6-49-b9d963d24994c47c3776dda6f7d3881f | 1 + ...imizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 | 0 ...mizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 | 0 ...mizer6-51-4746d944f4193018017984ca2df3c60d | 15 + ...mizer6-52-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer6-53-b9552c6399ce578cde69e663dd43d870 | 0 ...mizer6-54-4746d944f4193018017984ca2df3c60d | 15 + ...mizer6-55-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer6-56-24ca942f094b14b92086305cc125e833 | 1 + ...mizer6-57-b9552c6399ce578cde69e663dd43d870 | 0 ...mizer6-58-4746d944f4193018017984ca2df3c60d | 15 + ...imizer6-6-25b1fe48ef86952fc7766fb74b63bf21 | 15 + ...imizer6-7-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer6-8-24ca942f094b14b92086305cc125e833 | 1 + ...imizer6-9-fd372b979ca4b12d64c65cb24e7d82ae | 0 ...imizer7-0-24ca942f094b14b92086305cc125e833 | 1 + ...imizer7-1-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer7-10-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer7-11-fc438bb849eff3496559a916c7dee058 | 0 ...mizer7-12-22d71fb589c53776dabb4696b38c4a42 | 15 + ...mizer7-13-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 | 0 ...mizer7-15-22d71fb589c53776dabb4696b38c4a42 | 15 + ...imizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 | 0 ...imizer7-3-22d71fb589c53776dabb4696b38c4a42 | 15 + ...imizer7-4-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 | 0 ...imizer7-6-22d71fb589c53776dabb4696b38c4a42 | 15 + ...imizer7-7-24ca942f094b14b92086305cc125e833 | 1 + ...imizer7-8-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...imizer7-9-4d3e60a0e9bd8c12fdba4e010493537d | 1 + ...imizer8-0-b1e2ade89ae898650f0be4f796d8947b | 1 + ...imizer8-1-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer8-10-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer8-11-45693930a19f7e1b86202a5f5f2d139d | 0 ...timizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f | 24 + ...mizer8-13-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer8-14-2b8991eb2f59bae3250c8d379b6d6aea | 0 ...mizer8-15-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer8-16-8c9264ae28e72f3724a78412f26ddbcc | 0 ...imizer8-2-34cad62f8ca6f6b53cf593960055c56a | 0 ...imizer8-3-932db4b9935e4bc3d21dd33a8d12c275 | 13 + ...imizer8-4-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 | 0 ...imizer8-6-932db4b9935e4bc3d21dd33a8d12c275 | 13 + ...imizer8-7-b9d963d24994c47c3776dda6f7d3881f | 1 + ...imizer8-8-39a62b29e0ac91861c58c56fc654ba67 | 0 ...ptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f | 24 + .../count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b | 0 .../count-1-3531872d964bc2b4f07d51b9d1629df2 | 0 .../count-10-d0f804c7d06375db74a0fcf5f17603c6 | 4 + .../count-11-29aa086fe75b55482a91316c38498565 | 0 .../count-12-944f53db544c07a7b38a0544a21d8e13 | 1 + .../count-2-461bad3feb7dbc25fb35d45c6876d698 | 7 + .../count-3-dbcec232623048c7748b708123e18bf0 | 1 + .../count-4-590bf60b8d4dfa135f73dbb52180136f | 0 .../count-5-d0f804c7d06375db74a0fcf5f17603c6 | 4 + .../count-6-29aa086fe75b55482a91316c38498565 | 0 .../count-7-944f53db544c07a7b38a0544a21d8e13 | 1 + .../count-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + .../count-9-590bf60b8d4dfa135f73dbb52180136f | 0 ...p_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 | 0 ...cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 | 0 ...p_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e | 0 ...p_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 | 0 ...p_mj_rc-4-15beca26e35c885dff3bfe2336c26673 | 0 ...p_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 | 0 ...db name-0-a253b1ed35dbf503d1b8902dacbe23ac | 0 ...db name-0-b7dfeb6a941b42f7def5fdceae99f425 | 0 ...db name-1-417609d2bb67ba26de38e92ad834008f | 500 ++++ ...db name-1-febc8588a2a54ed5b00e47cb83dd85c7 | 0 ...db name-2-ce780d068b8d24786e639e361101a0c7 | 500 ++++ ...db name-3-afd6e46b6a289c3c24a8eec75a94043c | 0 ...able as-0-fd42e3ffae73415391acb1012a3531bd | 0 ...able as-1-b9002c1d71895be765575b62656d1928 | 500 ++++ ...reate_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d | 1 + ...reate_1-1-c2351f011b8ea41ff7dfa8f195148da3 | 0 ...eate_1-10-b9c0b95624e601614ea2561b83aaf0ba | 4 + ...eate_1-11-7daaeabd9c286e511e0628a32dc714d5 | 0 ...eate_1-12-fe8680bc5ef6fe3e64c68d2638e10785 | 2 + ...reate_1-13-5715f85456733a71fb7c83f1b254b3a | 4 + ...eate_1-14-437bd1acbae61f48e851addb769d3379 | 0 ...eate_1-15-fd9415e340cf54a9473cc4bf86673816 | 2 + ...eate_1-16-c99c700ca511b68577aae8ae513a4a32 | 4 + ...reate_1-2-ecd02bc3563cd6b60b8394956cb69084 | 2 + ...reate_1-3-c27702ff131e0ecfd71f1e1779fbe365 | 4 + ...reate_1-4-610b82bf7b0080d293977927e5ef780c | 0 ...reate_1-5-c77b018276b1558c1d9462e0625e152e | 0 ...reate_1-6-52dc9f900d7f7a559698aff9565f061a | 2 + ...reate_1-7-8564aa6cc2b0ee85292b3e8976fe9001 | 4 + ...reate_1-8-19331fe6a2a35f1171babfe4e1c86f59 | 0 ...reate_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 | 2 + ...big_view-0-bdf3c2e20793ef833f336a40791091d | 0 ...ig_view-1-fbd4b50d4d80d2a927a0db5037c62bee | 0 ..._escape-0-3e860648a3f2f258d59a62fd0bbe1c3e | 0 ..._escape-1-ecd02bc3563cd6b60b8394956cb69084 | 2 + ..._escape-2-c27702ff131e0ecfd71f1e1779fbe365 | 4 + ...e_escape-3-9541399cde42210bd7ac1beb07ceb14 | 0 ...e_escape-4-7d84873a6ce03e0e408fa67ef5dd733 | 1 + ...ricudaf-0-499d36ed8229cbf74a07b59914bdf717 | 9 + ...ricudaf-1-c7f934e9c76350a0d3caa694463a673b | 0 ...ricudaf-2-84f898ffd668a17fe2ef081866d1dcd2 | 0 ...ricudaf-3-d3b1af3e7f037de800255d9411a690e8 | 1 + ...ericudaf-4-c64a3266b9a1c3383cc56bd883345c1 | 0 ...ericudf-0-dd23fb149bb6d6937b838334559ad8d1 | 10 + ...tformat-0-16167c581df48112004009fef228e29a | 0 ...tformat-1-1246ba69e870178971f5ae062641cf47 | 0 ...tformat-2-8c2e4081b9d69297c35faab0a2ff86dc | 2 + ...tformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 | 0 ...tformat-4-2453e9c2ed5c7c517dce66a20e402a57 | 0 ...utformat-5-8552731917a8260c25e6df79b83bf5c | 2 + ...tformat-6-2f52c0c383ad83fae1620a0db9f6c863 | 0 ...utformat-7-a755c7e39694261510421e262b5005e | 0 ...tformat-8-33a4312fd617c5c9f2a560dc6ce868a5 | 2 + ...te_like-0-c2351f011b8ea41ff7dfa8f195148da3 | 0 ...ate_like-1-30485a2507b60b96cad3d293527e6af | 27 + ...te_like-10-7d84873a6ce03e0e408fa67ef5dd733 | 1 + ...e_like-11-ba64f0122b21f605d8b2928753784130 | 2 + ...ate_like-2-a20451f152e68606cc5e373fe5fd86a | 0 ...te_like-3-eea111a209cf8895f31f64699669c705 | 27 + ...te_like-4-39ead53334938635b60a5ffdaa2c9f86 | 0 ...te_like-5-dc9de26002604e9e436135bd4b40636d | 0 ...te_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 | 0 ...te_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 | 28 + ...te_like-8-639a13da6855b974fc2e170fd49b33cb | 0 ...te_like-9-a0ce573e299b66b8ce31da2890b318cb | 0 ...ke_view-0-3b48eae3848493703396156bedb1e98b | 0 ...ke_view-1-3c805fc10db9af83327e04d518f3753a | 0 ...e_view-10-eea111a209cf8895f31f64699669c705 | 27 + ...e_view-11-62e0dd08ff9214aa999d9a2f30704fe2 | 0 ...e_view-12-cb343b6b463c2e9b5735fbdf82d24811 | 0 ...e_view-13-fd80fbff7a622abe6b25c9fff7c5d608 | 0 ...e_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 | 2 + ...e_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 | 28 + ...e_view-16-639a13da6855b974fc2e170fd49b33cb | 0 ...e_view-17-a0ce573e299b66b8ce31da2890b318cb | 0 ...ike_view-18-d77d78569d86802f7c097d3d02150c | 1 + ...ke_view-19-deabf2d92205da2cbce9bdff854a81f | 2 + ...ke_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b | 0 ...e_view-20-152e6a694276b2fc1bb4fe6b4543fef0 | 0 ...e_view-21-f982cac17966dcb9d76c1b6b9d4a7914 | 0 ...e_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 | 0 ...e_view-23-44c601e364e42a83babab4a342dfbd2b | 0 ...e_view-24-df2d18dbae578430bfc7b9d27d201505 | 0 ...e_view-25-87a663f8fd80110a9cee249535037c0d | 33 + ...e_view-26-152e6a694276b2fc1bb4fe6b4543fef0 | 0 ...e_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 | 0 ...ke_view-3-559f17e8f827532749948b3b9e6c0f3f | 0 ...ke_view-4-c2351f011b8ea41ff7dfa8f195148da3 | 0 ...ke_view-5-ecd02bc3563cd6b60b8394956cb69084 | 2 + ...ike_view-6-30485a2507b60b96cad3d293527e6af | 27 + ...ke_view-7-fcc954b645304d258611f21d3aed7b76 | 0 ...ke_view-8-304a79a8a321b84aee91f907f756a7e3 | 0 ...ke_view-9-52dc9f900d7f7a559698aff9565f061a | 2 + ...pressed-0-366a4de0343396b9df03277f1098722c | 0 ...pressed-1-276fbe6fb296b13904516888ffa95342 | 0 ...ressed-10-d6fee0f05fa9b04cb7c557862402c929 | 14 + ...ressed-11-614c34f9e88015f21bffc4b8930bc95d | 1 + ...ressed-12-4d89cbe49f710527b54e6262472f0320 | 1 + ...ressed-13-440c6f8daa221613fe796d99b494e61f | 0 ...ressed-14-32251c08304629a3153e0b471de060c5 | 0 ...pressed-2-84b74227c9f1563f530cd3ac3b333e54 | 1 + ...pressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 | 0 ...pressed-4-c2a7e48e37375fc59126d71b9965b6c3 | 0 ...pressed-5-c2a7e48e37375fc59126d71b9965b6c3 | 0 ...pressed-6-d6fee0f05fa9b04cb7c557862402c929 | 14 + ...pressed-7-614c34f9e88015f21bffc4b8930bc95d | 1 + ...pressed-8-4d89cbe49f710527b54e6262472f0320 | 1 + ...pressed-9-16a420c1def17d14881bd75d35a4c0e5 | 0 ...ed_type-0-17320fbe4af5d2c6bf2d52425f70f968 | 0 ...ed_type-1-ecd02bc3563cd6b60b8394956cb69084 | 4 + ...ed_type-2-c27702ff131e0ecfd71f1e1779fbe365 | 6 + ...ed_type-3-ac452c9279877935983c37113898e53c | 0 ...ed_type-4-c26dff65f60f7c0c70183ce22f4529ff | 4 + ...ce_view-0-a14cfe3eff322066e61023ec06c7735d | 0 ...ce_view-1-a27131eb04bd5e071d3267c92d3f8dea | 0 ...ce_view-2-30dc3e80e3873af5115e4f5e39078a13 | 30 + ...ce_view-3-5fd147edbe44a96782923a3ef6caa47d | 0 ...ce_view-4-b1880014afc9ad1f8db91ba3db3867de | 0 ...ce_view-5-b1d2deb8a13dde4bf18c9b34836e00fb | 0 ..._table1-0-cafed8ca348b243372b9114910be1557 | 1 + ...d_table1-1-cc66bd64f1cdc97b953e20860305370 | 0 ..._table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e | 0 ..._table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 | 0 ..._table1-4-f3f1642674545762a4bff5cb75634e20 | 29 + ..._table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e | 29 + ..._table1-6-d7a147c6b0a3609663628b43457b2cef | 30 + ..._table1-7-81fc0a09afbcd674874961c09aa947d6 | 0 ..._table1-8-2ae6d96ecee0081ccc474388d50675fd | 0 ..._table1-9-758987cfb7302bdb76898290de49a80e | 0 ...t_table-0-d519c3ac0717b41167f8a40b472ad6b1 | 0 ...t_table-1-2ca90a28a994405e6150c96f4a572294 | 0 ...t_table-2-d51e74fd10cc16607137b7f715557ecd | 10 + ...te_udaf-0-a69c2b11dc912ef5444af32dce6aa33e | 10 + ...te_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 | 0 ...te_view-1-c186ac1fe46117acb6fd452df15e0d92 | 0 ...e_view-10-9f71514dffc747ddd49fbb1fafb6d3dd | 0 ...e_view-11-2021c047203276dd2db51a56e672fcea | 0 ...e_view-12-420752b11848e29bce1c8ace7d3060fc | 0 ...e_view-13-bff53e5df8356ac16f7b9b78b157e60a | 0 ...e_view-14-69162f2f22978113fea529d7fc7b78d3 | 0 ...e_view-15-ceebf4cb0dc23f517a444266bc8d2447 | 0 ...e_view-16-cb434f5704ee3ed21d1f1521a2a654f4 | 0 ...e_view-17-b3c1023d56f3439833c246e8bfd4502a | 0 ...e_view-18-437d0a699b73c61044ebf8539feb14f6 | 0 ...e_view-19-df2da604b5a8f2b236519510b5e4d34b | 1 + ...te_view-2-d80dcd1271ab264292e9938f3162427c | 0 ...e_view-20-56d203e4d0eb776bb4fa38409222b5b8 | 0 ...e_view-21-3609711e61b5b8d241d0e839557bfd64 | 0 ...e_view-22-3bc364c0ee46900d2201d706d2d58d67 | 0 ...te_view-3-25ffe475d52d6c399acaf120dc02afe8 | 0 ...te_view-4-87ed262d455e99ad45c909a2265a61b0 | 0 ...te_view-5-391caf27ff1589ec68d5f3bc4a27e711 | 0 ...te_view-6-d8d0e830783c383e3c00e9de3919c409 | 0 ...te_view-7-50b35b8a1772becc96cff65bba1eaee7 | 0 ...te_view-8-2ae18fc75eda9c3fe7d4e87829180805 | 0 ...te_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 | 0 ...itioned-0-d98274f9b34c8968292ccd6c959491dc | 0 ...itioned-1-ff29c88fac9c21eaf36469a4ce0fce18 | 0 ...itioned-2-45e7b89caadba56ec67638c341209f96 | 0 ...itioned-3-cf44ff130f66de720a77888260ef8d16 | 0 ...itioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 | 13 + ...itioned-5-d7a7d8592fca266745725192d3f875fc | 36 + ...nsitive-0-a22bfdbaf9720a07e0b2c0923036ce93 | 0 ...nsitive-1-893c61ec6ea62362324c213f588d8030 | 0 ...nsitive-2-9db45f87a7b1e69d7bf38ac6d5009122 | 0 ...varchar-0-311fdd725609cd47ea1b859f706da41e | 0 ...varchar-1-939814768fe997b27f01758d60fcd8f5 | 0 ...varchar-2-3223504c97628a44b65736565c1dda32 | 0 ...varchar-3-5d14fdc559b9790d81a8020bdcf4159b | 0 ...varchar-4-ccead78e4ec4583da3b48864e78cfd44 | 0 ...varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f | 0 ...varchar-6-351fa7f09c2e809f9cc87d83e11c1539 | 0 ...varchar-7-8620d7f55849ab163b6b85f51abdaaec | 5 + ...varchar-8-c93df93c0e8688f9c7a6167589670d32 | 5 + ..._format-0-94f3da887aa34aed74715bd2051bf3c5 | 1 + ...le.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd | 1 + ...e.table-0-c657beb729b6a7882309a203fc6f298e | 1 + ...se_drop-0-49f18014566b3e020dc19b1e61d25a4f | 0 ...se_drop-1-549981e00a3d95f03dd5a9ef6044aa20 | 2 + ...e_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 | 0 ...e_drop-11-2ea883422b74b701711e14e61472ba06 | 1 + ...e_drop-12-e02a53f7e798d2741152526516f14941 | 1 + ...se_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 | 0 ...se_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 | 1 + ...se_drop-4-d419f4ff197d4291208c2028cd158909 | 1 + ...se_drop-5-b7cf74929eabe781b0db79ed1043dc24 | 1 + ...se_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 | 0 ...se_drop-7-693736836ccc99f6e2449b94efcfeeba | 0 ...se_drop-8-97101266791d2b2c662bcde549422318 | 0 ...se_drop-9-8db536f925bf0f5058f97897e145a661 | 0 ...ocation-0-5528e36b3b0f5b14313898cc45f9c23a | 0 ...location-1-2b66b4554cf8ecf2ab6c325d4d89491 | 1 + ...cation-10-c5cd9c57a13da7f345563fbd75da4e45 | 27 + ...cation-11-9c36cac1372650b703400c60dd29042c | 1 + ...ocation-2-7650b86c86dd6b1a99c86ddc5a31bd63 | 0 ...ocation-3-81383a2b9568f942cc8e758f9a6ee2f1 | 0 ...ocation-4-be65cf649363681d54e593c42a5ecffb | 27 + ...ocation-5-9c36cac1372650b703400c60dd29042c | 1 + ...ocation-6-6fa58170a7c2e78b06a250403f02091a | 0 ...ocation-7-5698ac10441da07dbe3a947143c999c2 | 1 + ...ocation-8-6f2797b6f81943d3b53b8d247ae8512b | 0 ...ocation-9-92f087a5934481942995fc2aaf0d87e8 | 0 ...perties-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + ...perties-1-10de6a198e2b3f61974519ddd8623e68 | 1 + ...erties-10-26c10ff2ec4a69b16589069ced427d23 | 1 + ...perties-2-a1074315e598ad16bce55860e6e43363 | 0 ...perties-3-751417d45b8e80ee5cba2034458b5bc9 | 2 + ...perties-4-ddf44597db4fa15e89bee313f2dad371 | 0 ...perties-5-51c0974df1125b233936f25ce709ba4a | 1 + ...perties-6-26c10ff2ec4a69b16589069ced427d23 | 1 + ...perties-7-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + ...perties-8-10de6a198e2b3f61974519ddd8623e68 | 1 + ...perties-9-5a2bc556d3c66c5b33ab86e6cd37b54a | 0 .../date_2-0-8acfa0b538112534f57a3e051f0216bd | 0 .../date_2-1-116032b973a2060b533e1cdc9dfda301 | 0 .../date_2-2-cab14d992c53c106ab257fae52001e04 | 0 .../date_3-0-c26de4559926ddb0127d2dc5ea154774 | 0 .../date_3-1-d9a07d08f5204ae8208fd88c9255d447 | 0 .../date_3-2-a937c6e5a2c655930e0d3f80883ecc16 | 0 ...te_join1-0-70b9b49c55699fe94cfde069f5d197c | 0 ...te_join1-1-3a68de2112a212a07a3068916c608fb | 0 ...e_join1-2-894b6541812ac8b0abe2a24c966817d8 | 0 ...e_serde-0-ca88593bb7ec47fa782145d732100c07 | 0 ...e_serde-1-36e6041f53433482631018410bb62a99 | 0 ...e_serde-2-3ddfd8ecb28991aeed588f1ea852c427 | 0 ...e_serde-3-e6167e27465514356c557a77d956ea46 | 0 ...e_serde-4-c1e17c93582656c12970c37bac153bf2 | 0 ...e_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c | 0 ...e_serde-6-e00facec2986bc55074868eff87ba22a | 0 ...e_serde-7-a34279d8ebbadb78e925e8ed9c78947d | 0 ...ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 | 0 ...ddltime-1-426da52526f6f48c0ddeb0026fd566f1 | 4 + ...cimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 | 0 ...cimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd | 0 ...cimal_1-10-be179d261b9c42ed843dbf736b12e75 | 1 + ...imal_1-11-4c1fefa61e10a78f6406b526a60e4937 | 1 + ...imal_1-12-75b9add5e32a963cc9913c6ef4f84989 | 1 + ...imal_1-13-31ecaab3afa056fcc656d6e54f845cf4 | 0 ...cimal_1-2-80fc87cab17ceffea334afbb230a6653 | 0 ...cimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 | 0 ...cimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 | 1 + ...cimal_1-5-cbe6b235663cf78e602673ed715a2f40 | 1 + ...cimal_1-6-91b7219bd8c67673e16cb970fcc7be1b | 1 + ...decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a | 1 + ...cimal_1-8-cdd0932288d3cc43636334439805769d | 1 + ...cimal_1-9-1504e1b00092e2c58bafcc9936ad178c | 1 + ...cimal_2-0-3c8ecb1468952afe028596c65d587bee | 0 ...cimal_2-1-868e124edc1581325bd0fd10235a126b | 0 ...imal_2-10-f97d72aeb605ee18d34361c073552e92 | 1 + ...imal_2-11-58a090c30c59446d1e2b2a6c85fabf50 | 1 + ...imal_2-12-d63b5ea25e27852413132db4d9bfb035 | 0 ...imal_2-13-20ffe5115367abea9ea0ed1bda7a9439 | 1 + ...imal_2-14-3105d1029ad981af9cf1039ad9410fc0 | 1 + ...imal_2-15-3266fde6f6ab80a8bc027de0d5574f02 | 1 + ...imal_2-16-dc565c6c5bc24fd8b32729ce91999580 | 1 + ...imal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d | 1 + ...imal_2-18-f7c34d67fd579c82c636415172ec675e | 1 + ...imal_2-19-f97d72aeb605ee18d34361c073552e92 | 1 + ...cimal_2-2-6cc742523b3574e59ca21dad30f2d506 | 0 ...imal_2-20-58a090c30c59446d1e2b2a6c85fabf50 | 1 + ...imal_2-21-d72d68be1217c7b7a958f58456d85821 | 1 + ...imal_2-22-648e694eea042c59e8db30d067cb5bc8 | 1 + ...imal_2-23-a87b701c93a922b9e33ba16ae6a477ff | 1 + ...imal_2-24-8c2a8f1f3b792d5017be42078b15b94e | 1 + ...cimal_2-25-14face5c7104382196e65741a199c36 | 1 + ...imal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 | 1 + ...imal_2-27-a5ea3949eb5ab338916e4316c676c7f6 | 1 + ...imal_2-28-4a5410f96c6ef0843f12b0f593c104b1 | 1 + ...imal_2-29-659f627f0ff5fe2296a8a0a7daed6298 | 1 + ...cimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e | 0 ...imal_2-30-26a71d79e41353830b4ada96de6e2b8a | 1 + ...imal_2-31-3c8ecb1468952afe028596c65d587bee | 0 ...cimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 | 1 + ...cimal_2-5-3105d1029ad981af9cf1039ad9410fc0 | 1 + ...cimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 | 1 + ...cimal_2-7-dc565c6c5bc24fd8b32729ce91999580 | 1 + ...cimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d | 1 + ...cimal_2-9-f7c34d67fd579c82c636415172ec675e | 1 + ...cimal_3-0-90cd495a00051a0631b2021dbb9a4aef | 0 ...cimal_3-1-76f900dfe9ce95e8262a53939d33fb01 | 0 ...imal_3-10-420614bb0789115e008c96a7ad822624 | 4 + ...imal_3-11-63913753553b16d6c24e063fb49fdd15 | 4 + ...imal_3-12-d495d7178707ba55dcc01b9bb3398792 | 0 ...ecimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 | 0 ...cimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b | 38 + ...cimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 | 38 + ...cimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b | 38 + ...cimal_3-6-127a3a8400cae591c282dd24f8951e55 | 30 + ...cimal_3-7-9d4f27d4a4819113c5083462baa72052 | 30 + ...cimal_3-8-f65f4df6e3d971d575654ade4b4e4800 | 17 + ...cimal_3-9-b54243d38214362f9a9b1831548faac4 | 56 + ...cimal_4-0-98a58225355eb73036bb7b1144fa5a5f | 0 ...cimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 | 0 ...cimal_4-2-945542ec888136afaece8d7a5e20d52d | 0 ...decimal_4-3-399140971a10a5a0cc6a8c97a4635e | 0 ...cimal_4-4-81b37675c09ed874497325ae13233e5c | 0 ...cimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c | 0 ...cimal_4-6-693c2e345731f9b2b547c3b75218458e | 38 + ...cimal_4-7-f1eb45492510cb76cf6b452121af8531 | 38 + ...cimal_4-8-79734272b75fb9076bdb64644bed6276 | 0 ...cimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 | 0 ...al_join-0-c7659c1efa06c9eab1db28e916b750e4 | 0 ...al_join-1-c1524f17ee815171055a67ddc2f9de4e | 0 ...al_join-2-e966f01e702d4cc8f970dcdbc6007285 | 52 + ...ecision-0-cae8ceb6b0ab342948041c511a867b8e | 0 ...ecision-1-3f4119830536c92e5ccd76be0259e110 | 0 ...cision-10-d8a597810b222e9e121a11a1f5658fb0 | 1 + ...cision-11-673b15434ba47f11c71c3e8b2a575d83 | 0 ...cision-12-18906f5c6413065621430e3fe33c7e9e | 1 + ...cision-13-2a65d450f57f8ba9f594063b96074f0e | 1 + ...cision-14-34916eb904b8113a401ce78e6941a204 | 1 + ...cision-15-5c49f041326bc5a9e936910094f190ce | 0 ...ecision-2-d5be00a0fa6e2e290b40458442bd036c | 0 ...ecision-3-42cb35d680b3caeeb22e1c4865b8264b | 75 + ...ecision-4-38aaeba3e587b4dac72e26c4b02029fc | 75 + ...ecision-5-bb27734245ecbd0511be91af21c3b9ef | 75 + ...recision-6-b2547e6ef33325b2da12ce91b57af21 | 75 + ...ecision-7-ee5b7767c7fbb8614bb4ef907e8737b7 | 75 + ...ecision-8-6e6bd4655de124dc1fc482ce0d11930e | 75 + ...ecision-9-e7b465fbeb49487b2a972a314e2c01ab | 1 + ...on_name-0-312a37c422883aa0d660018248157cf8 | 0 ...on_name-1-9de8e5f66c536d4ace89c61759db829c | 1 + ...on_name-2-8732fdce7fb224dd783da2b83a93c795 | 0 ...on_name-3-a7047012b4bce0158edaafe5cf0a57be | 1 + ...limiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba | 0 ...limiter-1-d9e405c4107da8af78fcacb83a667b41 | 0 ...limiter-2-d7137294d2e53ea6edc259943e4c6069 | 3 + ...limiter-3-176724f76343433a8f2e6131b12206d7 | 3 + ...limiter-4-f17c3a91cdc84fbc6d14011b04f12a64 | 0 ..._indent-0-5b66d27453f15517fb266a5e1a0e3cbb | 0 ..._indent-1-5536eb772d43014b971c6da3a0c44904 | 6 + ..._indent-2-91bffa09f4f2caaaa6864bf935c2ea94 | 32 + ...se_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b | 1 + ...se_json-1-12cc2812ab067d58718c29ea6aa3d8a3 | 0 ...e_json-10-1afddec8522bd69f496b15980600a6e1 | 0 ...e_json-11-b05391400dc31139998dc3abaaf86320 | 1 + ...ase_json-2-8e7cfe3e6069e796124ca940125385a | 1 + ...se_json-3-d097973152d91fa8072facb0f739e304 | 1 + ...se_json-4-549981e00a3d95f03dd5a9ef6044aa20 | 1 + ...se_json-5-a3ee372283f45479db3f2cd7ebeedc8c | 1 + ...se_json-6-1afddec8522bd69f496b15980600a6e1 | 0 ...se_json-7-7529ec337ca17cdf95d037f29e1cb793 | 0 ...ase_json-8-8e7cfe3e6069e796124ca940125385a | 1 + ...se_json-9-d097973152d91fa8072facb0f739e304 | 1 + ...itioned-0-889714213a760ae9ab3ebe199eb30b62 | 0 ...itioned-1-cbd03c487eba9e34d57a8decaa3a0dfa | 0 ...itioned-2-db8910ace81a5102495905a508ef5c28 | 0 ...itioned-3-b80c7ae3530bfdbc8e865d49742da826 | 19 + ...itioned-4-889714213a760ae9ab3ebe199eb30b62 | 0 ...ed_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b | 1 + ...ed_json-1-889714213a760ae9ab3ebe199eb30b62 | 0 ...ed_json-2-cbd03c487eba9e34d57a8decaa3a0dfa | 0 ...ed_json-3-db8910ace81a5102495905a508ef5c28 | 0 ...ed_json-4-b80c7ae3530bfdbc8e865d49742da826 | 1 + ...ed_json-5-889714213a760ae9ab3ebe199eb30b62 | 0 ...e_pretty-0-f34ca99310bf1d4793cf64423c024ad | 0 ..._pretty-1-dbfaa12f26f99277b8397379189172cf | 1 + ...pretty-10-dbfaa12f26f99277b8397379189172cf | 1 + ...pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 | 12 + ...pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 | 1 + ...pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 | 24 + ..._pretty-14-22db46f42dc0c1bf01a76ca360c20a7 | 0 ...pretty-15-db4abe06aba81e685c52db6b43ba7c03 | 1 + ...pretty-16-ada55b65b72e0d65563ad6161e005f22 | 37 + ..._pretty-2-713712e0f6f18144d1f3a522e9b98861 | 23 + ..._pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 | 1 + ..._pretty-4-1546db18568697fa68a7fc781802d255 | 14 + ..._pretty-5-ce1966d8626096821b404ab8745c4914 | 1 + ..._pretty-6-1546db18568697fa68a7fc781802d255 | 44 + ..._pretty-7-db4abe06aba81e685c52db6b43ba7c03 | 1 + ..._pretty-8-1546db18568697fa68a7fc781802d255 | 31 + ..._pretty-9-e382a994134aefcd2652b57af9195644 | 0 ..._syntax-0-5528e36b3b0f5b14313898cc45f9c23a | 0 ..._syntax-1-4f3d3497418242124113538edab45df7 | 0 ...syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 | 12 + ...syntax-11-ab161e38c5d66b6c344c8372160ac74f | 33 + ...syntax-12-90c7890e1aa28e94520f35f5679560a4 | 1 + ...syntax-13-7c1216f9852d454bf93256e5a2588758 | 1 + ...syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 | 3 + ...syntax-15-e420b255509153b3326f00dcd25d11e4 | 1 + ...syntax-16-5043ee273a313d28adeca74fd33739a7 | 1 + ...syntax-17-c97a9e691cc08199678ead7f79d58b58 | 3 + ...syntax-18-69cecdc1889b7feb40b46c95524aaad4 | 1 + ...syntax-19-c1c3359705e256d7641bbffab00c43fa | 1 + ..._syntax-2-b198700c0129910d6205ef063ee83d5a | 0 ...syntax-20-781dd449a9b26dc54f3bba8e5178cd8a | 3 + ...syntax-21-5bac87eeb7e71928d01275b006720de3 | 10 + ...syntax-22-719a15ffd0018bb2898f9045be886e0f | 12 + ...syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 | 32 + ...syntax-24-ee226b42db35b219702319858e925468 | 10 + ...syntax-25-b6e10514fb473803c44bc793b9f9713e | 12 + ...syntax-26-4851dc65e26ec33c605ab1ed65b59bec | 32 + ..._syntax-3-458d6aaffeee94997f67a43b88382106 | 0 ..._syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 | 0 ..._syntax-5-a305e3d663b2697acca5abb9e7d897c4 | 0 ..._syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 | 10 + ..._syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d | 12 + ...be_syntax-8-9e40f8077f34fa35fccfeae972e8c4 | 33 + ..._syntax-9-fb744775fb30d92297534d29b6eafd6b | 10 + ...le_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b | 1 + ...le_json-1-e48b5f50bb2ff56f9886004366cfd491 | 0 ...le_json-2-9c36cac1372650b703400c60dd29042c | 1 + ...le_json-3-576670af142149302decb9bf8662e68a | 1 + ...le_json-4-4a57591b392bb8fd18238d068d191721 | 1 + ...le_json-5-865aeeea2647a71f7f25b03da4203ffb | 0 ...le_json-6-ac49d26a0211b804fee89bbe0808f430 | 1 + ...le_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 | 1 + ...le_json-8-deaf826aef1c9a7f36d7824eafd20f11 | 0 ...le_json-9-b05391400dc31139998dc3abaaf86320 | 1 + ...formats-0-12652a5a33548c245772e8d0894af5ad | 0 ...formats-1-961f7cb386a6eacd391dcb189cbeddaa | 0 ...formats-2-28cd0f9b01baa8627a013339dc9508ce | 0 ...formats-3-c6eef43568e8ed96299720d30a6235e1 | 0 ..._formats-4-a4890f2b20715c75e05c674d9155a5b | 1 + ...t_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c | 1 + ...t_check-1-ec4d7e2ce5bd580b24119860e96f376e | 0 ...at_check-2-d3e20a1484eabcd50e2039e55b4f549 | 0 ...t_check-3-93063fb3476a400ecfec0bfd02cbc23f | 0 ...t_check-4-fe6f402a026c882c2bc8f5251794dbbb | 0 ...nge_off-0-a071dedef216e84d1cb2f0de6d34fd1a | 1 + ...nge_off-1-ce3797dc14a603cba2a5e58c8612de5b | 1 + ...nge_off-2-f5340880d2be7b0643eb995673e89d11 | 0 ...nge_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 | 0 ...nge_off-4-34064fd15c28dba55865cb8f3c5ba68c | 1 + ...nge_off-5-f40a07d7654573e1a8517770eb8529e7 | 0 ...on_dirs-0-b454ca2d55b61fd597540dbe38eb51ab | 0 ...on_dirs-1-ece80e0bd1236c547da7eceac114e602 | 0 ...unction-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 1 + ...unction-1-ea5871f0a80a41e19fd6a42bd29b693a | 0 ...p_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 1 + ...op_index-1-5875a80dd89498c8b61db5f6bf26898 | 0 ...titions-0-c4449feb8f8e2c40c294ccf50680b47b | 0 ...titions-1-d738aa3c169c10f5b1e5959453dffbd4 | 0 ...itions-10-9120e865aa132bac6e0a29c907f0b760 | 1 + ...titions-2-7554be9025c7683c67dce09177396357 | 0 ...rtitions-3-bc92a4d9670709904a8d49ebe9ba8e5 | 0 ...titions-4-9120e865aa132bac6e0a29c907f0b760 | 3 + ...titions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 | 12 + ...titions-6-70ad97221d2be48259ea556f9d8e5353 | 0 ...titions-7-9120e865aa132bac6e0a29c907f0b760 | 1 + ...titions-8-46a4f646bbc04f70e7ae5ed992f102f1 | 1 + ...titions-9-3d4108847515b6386cd28b8862dcab53 | 0 ..._filter-0-6863a128b9a05e5c251ec0092e6124d3 | 0 ..._filter-1-e3eda6672f6602a1b9bc93ef789662f6 | 10 + ...filter-10-119b0fe0597fa478e1533a412e2d444b | 0 ...filter-11-83e3e422cdf9403523fa60d75376d7d7 | 9 + ...filter-12-e0e995c7bcc6b6d801b68878b6166835 | 0 ...filter-13-83e3e422cdf9403523fa60d75376d7d7 | 8 + ...filter-14-f47427726dd7546c3c59a2ec53891bb3 | 0 ...filter-15-83e3e422cdf9403523fa60d75376d7d7 | 6 + ...filter-16-fae7d99452ab708daa96c30f0b25c03b | 0 ...filter-17-83e3e422cdf9403523fa60d75376d7d7 | 5 + ...filter-18-4553ba96e8c121b2e98966a67220a0fd | 0 ...filter-19-83e3e422cdf9403523fa60d75376d7d7 | 3 + ..._filter-2-16027a4bed34a7610bbea1e11e83c3f2 | 0 ...filter-20-a08a2d37122fd5770f4197ec1f0ebd1c | 0 ...filter-21-83e3e422cdf9403523fa60d75376d7d7 | 1 + ...filter-22-46a4f646bbc04f70e7ae5ed992f102f1 | 1 + ...filter-23-45bb3dea1b8d4bd353885cd68729698e | 0 ...filter-24-83e3e422cdf9403523fa60d75376d7d7 | 1 + ...filter-25-67d75c72ea2d3982c55f3a850d93f83c | 0 ..._filter-3-a499bb560b6e228b924387759214bc3c | 0 ..._filter-4-8fde1e5c12230f4b8081950dfd68b18d | 0 ..._filter-5-d3dc443408a20518c42b839fba218216 | 0 ..._filter-6-8cf34fc1e9004b8fdf43770f7893506e | 0 ..._filter-7-7f8f890e3104e36ff8f5747d9a287b39 | 0 ..._filter-8-12ca7d0f34ab5127416bdb66d9e4a698 | 0 ..._filter-9-972bcd28a9e24b4cac70ef74524f1696 | 0 ...filter2-0-322b270dab4032668de9002e9e8bc7c5 | 0 ...filter2-1-e3eda6672f6602a1b9bc93ef789662f6 | 10 + ...ilter2-10-83e3e422cdf9403523fa60d75376d7d7 | 5 + ...ilter2-11-8b9e0542bfbf07e9ceabb1ce599d856a | 0 ...ilter2-12-83e3e422cdf9403523fa60d75376d7d7 | 3 + ...ilter2-13-67d75c72ea2d3982c55f3a850d93f83c | 0 ...filter2-2-e83e7a8f276f890c4eb29e192d684730 | 0 ..._filter2-3-31bf5686028f845040ae39acf642701 | 0 ...filter2-4-a2c778112718207a10070596cb4595d8 | 0 ...filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a | 0 ...filter2-6-bbe938b3b20589283cc4541f3e417268 | 0 ...filter2-7-74ed9df854eae5a025077b7012ef7b97 | 0 ...filter2-8-83e3e422cdf9403523fa60d75376d7d7 | 6 + ...filter2-9-185122a935af4fbe8466d7e39fc7648a | 0 ...filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 | 0 ...filter3-1-e3eda6672f6602a1b9bc93ef789662f6 | 10 + ...ilter3-10-83e3e422cdf9403523fa60d75376d7d7 | 5 + ...ilter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 | 0 ...ilter3-12-83e3e422cdf9403523fa60d75376d7d7 | 3 + ...ilter3-13-67d75c72ea2d3982c55f3a850d93f83c | 0 ...filter3-2-3e8e821dd63112223649b5d06febf7d9 | 0 ...filter3-3-431228f63002f8b7d1364aa2a07f92ec | 0 ...filter3-4-ce613fad87e72c2519c0d01b64f4a99a | 0 ...filter3-5-9a22b61cd027d740791ba847abc7e072 | 0 ...filter3-6-6127604e4c55b13778cc56c0068ce6ae | 0 ...filter3-7-688620ee5d61cce432e6c2d590b31404 | 0 ...filter3-8-83e3e422cdf9403523fa60d75376d7d7 | 6 + ...filter3-9-d1a5c03e520fbfa2249e0a32b824a275 | 0 ...otection-0-11d8788cb803a84dd4ca3b4103a8bcb | 0 ...tection-1-864c8c804db30687f4265ba081ca6368 | 0 ...tection-2-446c2380269a60eb4710d7dbeb7c2ec6 | 0 ...otection-3-312aa26fdea6da7907e3a91f75e36f1 | 10 + ...tection-4-11f9ef9dd235f44b29c77abf7ca27881 | 0 ...tection-5-1283e970d6dc39e7a86e86af904bf116 | 0 ...p_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 1 + ...p_table-1-afec243db5bd3a1b65d961e2325c6a57 | 0 ..._table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 | 1 + ..._table2-1-35229351a48427cf25b42ac8a61200fa | 0 ..._table2-2-cb72d751d94b6d8518c8d69017b6a293 | 0 ..._table2-3-5f94efc9e658389a6d63553abd7a517f | 0 ..._table2-4-1c852531c1e75093c27911b45315ed62 | 0 ..._table2-5-1fdd850f6c301619f91eb58c890f2ad4 | 3 + ..._table2-6-120f319d6031395a86c726e43d4ef678 | 0 ..._table2-7-35229351a48427cf25b42ac8a61200fa | 0 ..._table2-8-1fdd850f6c301619f91eb58c890f2ad4 | 0 ..._table2-9-120f319d6031395a86c726e43d4ef678 | 0 ...op_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 1 + ...op_view-1-70a24b7e47d8527298241bcbec922cf5 | 0 ...default-0-2bba07855af8c11899cc6b89f8c0ee02 | 0 ...default-1-16367c381d4b189b3640c92511244bfe | 1 + ...default-2-d71f115b7d42f6c67de701bf69c617a9 | 0 ..._default-3-b7f2a424f616cfb015937e9ef980277 | 0 ...default-4-d8c93b46de2a09a4af12017c6ba196f0 | 0 ...default-5-e394cdeb88f69b4d4b08450680f779b9 | 0 ...default-6-725ba4225501c1279f593b9c72eaca28 | 0 ...default-7-e707e693aa61edf87768fb71f6e936e1 | 1 + ...default-8-725ba4225501c1279f593b9c72eaca28 | 0 ...default-9-3b57aa58995f862f2713624b50db5b65 | 0 ...e_order-0-9b9af6de0a12a47694e6f93264c2ebf9 | 0 ...e_order-1-633034e3d966737cecf2befc5df1e35d | 0 ...e_order-2-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...e_order-3-794fc9cf9fb97001efa85f24fde0cd4f | 0 ...ce_order-4-3136edd49e681ea21aa35d0836eab65 | 0 ...e_order-5-15d62540b57faa68f58c5c83c3a296c9 | 0 ...e_order-6-277e01aa70e41e8cce47236fcbbb36c2 | 0 ...e_order-7-5fb418612e7c2201790d6f318c715ccf | 10 + ...e_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 | 10 + ...escape1-0-a4fb8359a2179ec70777aad6366071b7 | 1 + ...escape1-1-683124e29877d2c5a96b95c8ddba97b7 | 1 + ...escape1-2-395d5a528c5e7235a48b4ac90938e2d6 | 0 .../escape1-3-4267651148da591da38737028fdbd80 | 0 ...escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 | 0 ...escape1-5-70729c3d79ded87e884c176138174645 | 0 ...escape1-6-134b596abc363f0bfa7f770732ebb960 | 1 + ...escape1-7-486585cbb4de5bc908dde4c601dd7c17 | Bin 0 -> 252 bytes ...escape1-8-910536a438eec89c78bd611b3c4bb7e0 | 0 ...escape2-0-a4fb8359a2179ec70777aad6366071b7 | 1 + ...escape2-1-683124e29877d2c5a96b95c8ddba97b7 | 1 + ...scape2-10-13884d58efe80bd24862b3c54cb57c6e | 0 ...escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + .../escape2-3-1774adb1085f4ee6782a8dac0735399 | 1 + ...escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe | 0 ...escape2-5-fd0c88ad6ad131a16d1b78adbea65800 | 0 ...escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 | 0 ...escape2-7-70729c3d79ded87e884c176138174645 | 0 ...escape2-8-134b596abc363f0bfa7f770732ebb960 | 1 + ...escape2-9-486585cbb4de5bc908dde4c601dd7c17 | Bin 0 -> 252 bytes ...sterby1-0-e34202f0d36c00a68722c802139d17cf | 0 ...sterby1-1-914091aa635a64c707c69e296eb097a5 | 0 ...buteby1-0-1f178ec5a7ea27b365012c751f3a9cdb | 0 ...ibuteby1-1-f17d649f307c8c827bf2d136c5d02e5 | 0 ...rderby1-0-4057beace528a415308f7ca332f29941 | 0 ...rderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 | 0 ...sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 | 0 ...sortby1-1-bb5ad94d261df75e195d3051a4634d99 | 0 ...rt_empty-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...t_empty-1-baeaf0da490037e7ada642d23013075a | 1 + ...t_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...t_empty-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...t_empty-4-75eed21390055f8e397c81ab9d253a32 | 1 + ...t_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ..._nonpart-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...nonpart-1-baeaf0da490037e7ada642d23013075a | 1 + ...nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...nonpart-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...nonpart-5-75eed21390055f8e397c81ab9d253a32 | 1 + ...nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ...rt_empty-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...t_empty-1-baeaf0da490037e7ada642d23013075a | 1 + ...t_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...t_empty-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...t_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...t_empty-5-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ..._02_part-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...02_part-1-baeaf0da490037e7ada642d23013075a | 1 + ...02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...02_part-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...02_part-4-88b581725ecdd603117a1706ab9c34dc | 0 ...02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...02_part-6-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...r_compat-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ..._compat-1-baeaf0da490037e7ada642d23013075a | 1 + ..._compat-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ..._compat-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ..._compat-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ..._compat-5-75eed21390055f8e397c81ab9d253a32 | 1 + ..._compat-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ...all_part-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...ll_part-1-baeaf0da490037e7ada642d23013075a | 1 + ...ll_part-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...ll_part-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...ll_part-4-88b581725ecdd603117a1706ab9c34dc | 0 ...ll_part-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...ll_part-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...ll_part-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...ll_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...ll_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...ed_parts-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...d_parts-1-baeaf0da490037e7ada642d23013075a | 1 + ..._parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ..._parts-11-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...d_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...d_parts-3-c148cf39c4f8f02d44964cfd6919fa29 | 0 ...d_parts-4-3c27502d4f6977b959e0928755b43be3 | 0 ...d_parts-5-2d68fb88796f07b4bbe2b6895464ee62 | 0 ...d_parts-6-10c086eb97e0cae9a2d4b79d90925e85 | 0 ...d_parts-7-bff74501ebaea5bd2227d029003dbe08 | 0 ...d_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 | 0 ...d_parts-9-18a8affd3b07841b7cbe64003470a9f7 | 0 ...ome_part-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...me_part-1-baeaf0da490037e7ada642d23013075a | 1 + ...me_part-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...me_part-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...me_part-4-88b581725ecdd603117a1706ab9c34dc | 0 ...me_part-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...me_part-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...me_part-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...me_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...me_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...one_part-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...ne_part-1-baeaf0da490037e7ada642d23013075a | 1 + ...ne_part-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...ne_part-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...ne_part-4-88b581725ecdd603117a1706ab9c34dc | 0 ...ne_part-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...ne_part-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...ne_part-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...ne_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...ne_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...noverlap-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...overlap-1-baeaf0da490037e7ada642d23013075a | 1 + ...overlap-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...overlap-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...overlap-4-88b581725ecdd603117a1706ab9c34dc | 0 ...overlap-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...overlap-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...overlap-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...overlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...overlap-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...t_rename-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ..._rename-1-baeaf0da490037e7ada642d23013075a | 1 + ..._rename-2-ad7877a96aba7cd6e29edc19f4f0b394 | 1 + ..._rename-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ..._rename-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ..._rename-5-75eed21390055f8e397c81ab9d253a32 | 1 + ..._rename-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ...noverlap-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...overlap-1-baeaf0da490037e7ada642d23013075a | 1 + ...overlap-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...overlap-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...overlap-4-88b581725ecdd603117a1706ab9c34dc | 0 ...overlap-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...overlap-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...overlap-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...overlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...overlap-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ..._managed-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...managed-1-baeaf0da490037e7ada642d23013075a | 1 + ...managed-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...managed-3-be31972099603addb71187f19f7cd25d | 1 + ...managed-4-46667daf88f9c8b9b758ced6a6b28ef1 | 1 + ...external-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...xternal-1-baeaf0da490037e7ada642d23013075a | 1 + ...xternal-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...xternal-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...xternal-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...xternal-5-75eed21390055f8e397c81ab9d253a32 | 1 + ...xternal-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ...location-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...ocation-1-baeaf0da490037e7ada642d23013075a | 1 + ...ocation-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...ocation-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...ocation-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...ocation-5-75eed21390055f8e397c81ab9d253a32 | 1 + ...ocation-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ...location-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...ocation-1-baeaf0da490037e7ada642d23013075a | 1 + ...ocation-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...ocation-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...ocation-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...ocation-5-75eed21390055f8e397c81ab9d253a32 | 1 + ...ocation-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ...existing-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...xisting-1-baeaf0da490037e7ada642d23013075a | 1 + ...xisting-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...xisting-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...xisting-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...xisting-5-75eed21390055f8e397c81ab9d253a32 | 1 + ...xisting-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ...nal_part-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...al_part-1-baeaf0da490037e7ada642d23013075a | 1 + ...al_part-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...al_part-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...al_part-4-88b581725ecdd603117a1706ab9c34dc | 0 ...al_part-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...al_part-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...al_part-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...al_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...al_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...external-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...xternal-1-baeaf0da490037e7ada642d23013075a | 1 + ...xternal-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...xternal-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...xternal-4-88b581725ecdd603117a1706ab9c34dc | 0 ...xternal-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...xternal-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...xternal-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...xternal-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...xternal-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ..._managed-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...managed-1-baeaf0da490037e7ada642d23013075a | 1 + ...managed-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...managed-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...managed-4-88b581725ecdd603117a1706ab9c34dc | 0 ...managed-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...managed-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...managed-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...managed-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...external-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...xternal-1-baeaf0da490037e7ada642d23013075a | 1 + ...xternal-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...xternal-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...xternal-4-88b581725ecdd603117a1706ab9c34dc | 0 ...xternal-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...xternal-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...xternal-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...xternal-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...xternal-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...location-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...ocation-1-baeaf0da490037e7ada642d23013075a | 1 + ...ocation-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...ocation-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...ocation-4-88b581725ecdd603117a1706ab9c34dc | 0 ...ocation-5-75f428bb2aa8624ac08095cdfd7a6993 | 0 ...ocation-6-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...ocation-7-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...location-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...ocation-1-baeaf0da490037e7ada642d23013075a | 1 + ...ocation-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...ocation-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...ocation-4-88b581725ecdd603117a1706ab9c34dc | 0 ...ocation-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...ocation-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...ocation-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...ocation-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...ocation-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...location-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...ocation-1-baeaf0da490037e7ada642d23013075a | 1 + ...ocation-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...ocation-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...ocation-4-88b581725ecdd603117a1706ab9c34dc | 0 ...ocation-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...ocation-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...ocation-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...ocation-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...ocation-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...hsuccess-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...success-1-baeaf0da490037e7ada642d23013075a | 1 + ...success-2-f5d86ed3cbc46bb0c7619703081d5873 | 0 ...success-3-9fb7c47b98513bf3355e077ee9732cdd | 0 ...success-4-7368973ec1870dd3b237c37eb3857b1e | 1 + ...success-5-760e902318ec521eed07cf23e0f256a2 | 0 ...success-6-75eed21390055f8e397c81ab9d253a32 | 1 + ...success-7-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ...hsuccess-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...success-1-baeaf0da490037e7ada642d23013075a | 1 + ...success-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...success-3-f5d86ed3cbc46bb0c7619703081d5873 | 0 ...success-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...success-5-75eed21390055f8e397c81ab9d253a32 | 1 + ...success-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ...hsuccess-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...success-1-baeaf0da490037e7ada642d23013075a | 1 + ...success-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...success-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...success-4-88b581725ecdd603117a1706ab9c34dc | 0 ...success-5-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 + ...success-6-a9f93b4185a714e4f6d14171d10a6c07 | 1 + ...hsuccess-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...success-1-baeaf0da490037e7ada642d23013075a | 1 + ...success-2-e6e650bf4c6291ee2d78e5af5b60e906 | 1 + ...success-3-f5d86ed3cbc46bb0c7619703081d5873 | 0 ...success-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...success-5-d2ec90909f243a767be1aa299720f45d | 1 + ...success-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 + ...egation-0-739356d010a919138911f295fac81607 | 1 + ...regation-1-8dc96d77daa18d053fab3c134028788 | 0 ...egation-2-f015c961b6c34ac56cb8fc52fb36d7c7 | 1 + ...mat_mix-0-c6dff7eb0a793f9cd555164d23eda699 | 0 ...mat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b | 0 ...mat_mix-2-701660c0ea117b11d12de54dc661bc3e | 0 ...mat_mix-3-2b2316f235737a3f9a30fb05a082e132 | 0 ...mat_mix-4-fcda187f1366ff93a113cbe670335198 | 0 ...mat_mix-5-c2d0da9a0f01736a2163c99fc667f279 | 1 + ...mat_mix-6-4b658b3222b7a09ef41d023215e5b818 | 500 ++++ ...eaktask-0-5090cca7feb54de047bc535e234bd4a8 | 0 ...eaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 | 0 ...eaktask-2-aa61f4a2dd974ae0df026de640ed5802 | 0 ...eaktask-3-550e9b759fb088a81feddeff2e4be64e | 12 + ...aktask2-0-db1a6e42b4f880b00b389ae21c7658e1 | 0 ...aktask2-1-4a7480781402d0ac0a856a46ca3883fd | 0 ...ktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd | 1 + ...ktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 | 1 + ...ktask2-12-c608f51882260caec7bb9c57a0ba3179 | 1 + ...aktask2-2-b8486987aee5bac5f5b7301952e67d0c | 0 ...aktask2-3-1139b5e7f76614bc03bf0db677ed7d73 | 0 ...aktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 | 0 ...aktask2-5-25f891cf5e9138c14ba8c385c83230ba | 0 ...aktask2-6-4aaf479be27e3f8b38465ea946b530e3 | 0 ...aktask2-7-571467e86d08833eecf5d3e67b41bba6 | 0 ...aktask2-8-604cf64d16b9d438ee187a10d9f6352e | 1 + ...aktask2-9-6d551990a8a745dde1cb5244947b81a1 | 1 + ...roupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...roupby1-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...roupby1-2-f90acd59ace31e16059bae52583188cc | 0 ...roupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d | 1 + ...roupby1-4-19094f229f8af852ef8dad406333ae08 | 0 ...roupby1-5-dd7bf298b8c921355edd8665c6b0c168 | 1 + ...roupby1-6-977359ea0d2f429a9a35bbd8599cddea | 0 ...roupby1-7-c2c54378ffce53ade73a9dda783903e7 | 309 ++ ...oupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...oupby11-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...oupby11-2-f06bf6a360c7f9425cffa237f00445d8 | 0 ...oupby11-3-f11e9a986df49215fb8aa17aaccc7087 | 0 ...oupby11-4-1ae3e153888f9ca44e92ef43aea19092 | 0 ...oupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f | 0 ...oupby11-6-686ee4a5b2e24e51ba1d41b42215accd | 309 ++ ...oupby11-7-149d359546ab38226ffeb023d7414b3d | 309 ++ ...1_limit-0-83c59d378571a6e487aa20217bd87817 | 1 + ...1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 | 0 ...1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 | 0 ...1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 | 0 ...1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 | 5 + ...by1_map-0-dbcec232623048c7748b708123e18bf0 | 1 + ...by1_map-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...by1_map-2-83c59d378571a6e487aa20217bd87817 | 1 + ...by1_map-3-647cd470ff311f1879243a6e7f1e7bf6 | 0 ...by1_map-4-330a40b8e19028bbb370adf219f469fe | 0 ...by1_map-5-40f5168bfd9d124165bd207543b68a28 | 0 ...by1_map-6-adc1ec67836b26b60d8547c4996bfd8f | 309 ++ ...p_nomap-0-dbcec232623048c7748b708123e18bf0 | 1 + ...p_nomap-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...ap_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 | 1 + ...p_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 | 0 ...p_nomap-4-330a40b8e19028bbb370adf219f469fe | 0 ...p_nomap-5-40f5168bfd9d124165bd207543b68a28 | 0 ...p_nomap-6-adc1ec67836b26b60d8547c4996bfd8f | 309 ++ ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 1 + ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 1 + ...ap_skew-3-647cd470ff311f1879243a6e7f1e7bf6 | 0 ...ap_skew-4-330a40b8e19028bbb370adf219f469fe | 0 ...ap_skew-5-40f5168bfd9d124165bd207543b68a28 | 0 ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 309 ++ ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 1 + ..._noskew-3-f90acd59ace31e16059bae52583188cc | 0 ..._noskew-4-19094f229f8af852ef8dad406333ae08 | 0 ..._noskew-5-977359ea0d2f429a9a35bbd8599cddea | 0 ..._noskew-6-c2c54378ffce53ade73a9dda783903e7 | 309 ++ ...roupby2-0-43d53504df013e6b35f81811138a167a | 1 + ...roupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...roupby2-2-c67a488530dc7e20a9e7acf02c14380f | 1 + ...roupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b | 0 ...roupby2-4-67a2618eb44e68e6d8cf8792ded802f7 | 0 ...roupby2-5-c0660310e5d882732d07cb76bc0a7939 | 0 ...roupby2-6-41dfb7b036cae4972b275a0262ea2e4c | 10 + ...2_limit-0-83c59d378571a6e487aa20217bd87817 | 1 + ...2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 | 0 ...2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 | 5 + ...by2_map-0-dbcec232623048c7748b708123e18bf0 | 1 + ...by2_map-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...by2_map-2-83c59d378571a6e487aa20217bd87817 | 1 + ...by2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 | 0 ...by2_map-4-e193b8c0a9e1731dd46b145d166c78a7 | 0 ...pby2_map-5-ce0965adac15c4da6526d433d17ebc0 | 0 ...by2_map-6-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...istinct-0-dbcec232623048c7748b708123e18bf0 | 1 + ...istinct-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...istinct-2-83c59d378571a6e487aa20217bd87817 | 1 + ...istinct-3-3aa4057488720c0f514696154f2070b5 | 0 ...istinct-4-485f8328cdc8639bcdea5fb9f58c8695 | 0 ...istinct-5-23478129b7b6e377dcfb7caaa17dfb48 | 0 ...istinct-6-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 1 + ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 1 + ...ap_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 | 0 ...ap_skew-4-e193b8c0a9e1731dd46b145d166c78a7 | 0 ...map_skew-5-ce0965adac15c4da6526d433d17ebc0 | 0 ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 10 + ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 1 + ..._noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b | 0 ..._noskew-4-67a2618eb44e68e6d8cf8792ded802f7 | 0 ..._noskew-5-c0660310e5d882732d07cb76bc0a7939 | 0 ..._noskew-6-41dfb7b036cae4972b275a0262ea2e4c | 10 + ...istinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...istinct-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...istinct-2-83c59d378571a6e487aa20217bd87817 | 1 + ...istinct-3-f64b982c4bf34931f03447e635ae33d2 | 0 ...istinct-4-2115f3e7d207621ce2b07b6e33563844 | 0 ...istinct-5-92891db0de9b8cd00892f0a790aff494 | 0 ...istinct-6-41dfb7b036cae4972b275a0262ea2e4c | 10 + ...roupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...roupby4-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...roupby4-2-2a18d9570d9b676e240cda76df818c42 | 0 ...roupby4-3-132eabb30b8d011c848c531a6ab54529 | 0 ...roupby4-4-ac19a9a7f4a16763bfe7998179257933 | 0 ...roupby4-5-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...by4_map-0-dbcec232623048c7748b708123e18bf0 | 1 + ...by4_map-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...by4_map-2-83c59d378571a6e487aa20217bd87817 | 1 + ...by4_map-3-840759680be77463c13e4b19b2c1eb04 | 0 ...by4_map-4-19182d5780c4632c5bf8effdd5c7f36b | 0 ...by4_map-5-c0117072e2d392e3f860456d0226b7b9 | 0 ...by4_map-6-adc1ec67836b26b60d8547c4996bfd8f | 1 + ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 1 + ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 1 + ...ap_skew-3-840759680be77463c13e4b19b2c1eb04 | 0 ...ap_skew-4-19182d5780c4632c5bf8effdd5c7f36b | 0 ...ap_skew-5-c0117072e2d392e3f860456d0226b7b9 | 0 ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 1 + ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 1 + ..._noskew-3-2a18d9570d9b676e240cda76df818c42 | 0 ..._noskew-4-132eabb30b8d011c848c531a6ab54529 | 0 ..._noskew-5-ac19a9a7f4a16763bfe7998179257933 | 0 ..._noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...roupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...roupby5-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...roupby5-2-13ab74a58da514fe01dbeda0c3e79883 | 0 ...roupby5-3-a773aeb40af0516f2527f8e9d6907420 | 0 ...roupby5-4-c4570c2676d599793e1e9ece32aa596e | 0 ...roupby5-5-adc1ec67836b26b60d8547c4996bfd8f | 309 ++ ...by5_map-0-dbcec232623048c7748b708123e18bf0 | 1 + ...by5_map-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...by5_map-2-83c59d378571a6e487aa20217bd87817 | 1 + ...by5_map-3-840759680be77463c13e4b19b2c1eb04 | 0 ...by5_map-4-7b0346bd80d9833f2eccf8365b53d80f | 0 ...by5_map-5-586dc4493f66ea612a1d5b3cda89d725 | 0 ...by5_map-6-adc1ec67836b26b60d8547c4996bfd8f | 1 + ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 1 + ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 1 + ...ap_skew-3-840759680be77463c13e4b19b2c1eb04 | 0 ...ap_skew-4-7b0346bd80d9833f2eccf8365b53d80f | 0 ...ap_skew-5-586dc4493f66ea612a1d5b3cda89d725 | 0 ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 1 + ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 1 + ..._noskew-3-13ab74a58da514fe01dbeda0c3e79883 | 0 ..._noskew-4-a773aeb40af0516f2527f8e9d6907420 | 0 ..._noskew-5-c4570c2676d599793e1e9ece32aa596e | 0 ..._noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 309 ++ ...roupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...roupby6-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...roupby6-2-2a18d9570d9b676e240cda76df818c42 | 0 ...roupby6-3-ae61517a9997b80d512a9089cdb71fac | 0 ...roupby6-4-3f468a119e7975110b4063adb42c7dd9 | 0 ...roupby6-5-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...by6_map-0-dbcec232623048c7748b708123e18bf0 | 1 + ...by6_map-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...by6_map-2-83c59d378571a6e487aa20217bd87817 | 1 + ...by6_map-3-2a18d9570d9b676e240cda76df818c42 | 0 ...by6_map-4-ae61517a9997b80d512a9089cdb71fac | 0 ...by6_map-5-3f468a119e7975110b4063adb42c7dd9 | 0 ...by6_map-6-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 1 + ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 1 + ...ap_skew-3-2a18d9570d9b676e240cda76df818c42 | 0 ...ap_skew-4-ae61517a9997b80d512a9089cdb71fac | 0 ...ap_skew-5-3f468a119e7975110b4063adb42c7dd9 | 0 ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 10 + ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 1 + ..._noskew-3-2a18d9570d9b676e240cda76df818c42 | 0 ..._noskew-4-ae61517a9997b80d512a9089cdb71fac | 0 ..._noskew-5-3f468a119e7975110b4063adb42c7dd9 | 0 ..._noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...roupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...roupby7-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...roupby7-2-3678fb67b5c739bd87d4907630da1208 | 0 ...roupby7-3-95474af63c0f92717ed49c3a0f37b10a | 0 ...roupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 + ...roupby7-5-ed76c0068780120a6f23feefee303403 | 1 + ...roupby7-6-b2af91348c5fa9605702be50983c3bd2 | 0 ...roupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...roupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...by7_map-0-dbcec232623048c7748b708123e18bf0 | 1 + ...by7_map-1-7f98b724df05f51b3ec1f087a8da414e | 1 + ...y7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...y7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...by7_map-2-85c4f90b754cd88147d6b74e17d22063 | 1 + ...by7_map-3-83c59d378571a6e487aa20217bd87817 | 1 + ...by7_map-4-3678fb67b5c739bd87d4907630da1208 | 0 ...by7_map-5-95474af63c0f92717ed49c3a0f37b10a | 0 ...by7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 + ...by7_map-7-ed76c0068780120a6f23feefee303403 | 1 + ...by7_map-8-409f355bf35dfaa8b9e643510c58fabd | 0 ...by7_map-9-b2af91348c5fa9605702be50983c3bd2 | 0 ...reducer-0-dbcec232623048c7748b708123e18bf0 | 1 + ...reducer-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...educer-10-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...reducer-2-83c59d378571a6e487aa20217bd87817 | 1 + ...reducer-3-3678fb67b5c739bd87d4907630da1208 | 0 ...reducer-4-95474af63c0f92717ed49c3a0f37b10a | 0 ...reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 + ...reducer-6-ed76c0068780120a6f23feefee303403 | 1 + ...reducer-7-409f355bf35dfaa8b9e643510c58fabd | 0 ...reducer-8-b2af91348c5fa9605702be50983c3bd2 | 0 ...reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 1 + ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...p_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 1 + ...ap_skew-3-3678fb67b5c739bd87d4907630da1208 | 0 ...ap_skew-4-95474af63c0f92717ed49c3a0f37b10a | 0 ...ap_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 + ...ap_skew-6-ed76c0068780120a6f23feefee303403 | 1 + ...ap_skew-7-409f355bf35dfaa8b9e643510c58fabd | 0 ...ap_skew-8-b2af91348c5fa9605702be50983c3bd2 | 0 ...ap_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ..._noskew-1-7f98b724df05f51b3ec1f087a8da414e | 1 + ...noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ..._noskew-2-85c4f90b754cd88147d6b74e17d22063 | 1 + ..._noskew-3-83c59d378571a6e487aa20217bd87817 | 1 + ..._noskew-4-3678fb67b5c739bd87d4907630da1208 | 0 ..._noskew-5-95474af63c0f92717ed49c3a0f37b10a | 0 ..._noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 + ..._noskew-7-ed76c0068780120a6f23feefee303403 | 1 + ..._noskew-8-409f355bf35dfaa8b9e643510c58fabd | 0 ..._noskew-9-b2af91348c5fa9605702be50983c3bd2 | 0 ...reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...reducer-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...educer-10-96d0598a2a4c17041a31b908d1f065e5 | 10 + ...reducer-2-83c59d378571a6e487aa20217bd87817 | 1 + ...reducer-3-3678fb67b5c739bd87d4907630da1208 | 0 ...reducer-4-95474af63c0f92717ed49c3a0f37b10a | 0 ...reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 + ...reducer-6-ed76c0068780120a6f23feefee303403 | 1 + ...reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 | 0 ...reducer-8-e404ba29e723df8bd8662d4f48129c7a | 0 ...reducer-9-652510b8dc20117c65511f06e6e73d73 | 10 + ...roupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...roupby8-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...oupby8-10-83296fd5b5fc22af0c51514c4e67c95f | 0 ...oupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...oupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...roupby8-2-3678fb67b5c739bd87d4907630da1208 | 0 ...roupby8-3-95474af63c0f92717ed49c3a0f37b10a | 0 ...roupby8-4-a0a22ec83d6632cba3f17d79dbb9494d | 0 ...roupby8-5-83296fd5b5fc22af0c51514c4e67c95f | 0 ...roupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...roupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...roupby8-8-7f98b724df05f51b3ec1f087a8da414e | 1 + ...roupby8-9-a0a22ec83d6632cba3f17d79dbb9494d | 0 ...by8_map-0-dbcec232623048c7748b708123e18bf0 | 1 + ...by8_map-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...by8_map-2-83c59d378571a6e487aa20217bd87817 | 1 + ...by8_map-3-3678fb67b5c739bd87d4907630da1208 | 0 ...by8_map-4-95474af63c0f92717ed49c3a0f37b10a | 0 ...by8_map-5-a0a22ec83d6632cba3f17d79dbb9494d | 0 ...by8_map-6-83296fd5b5fc22af0c51514c4e67c95f | 0 ...by8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...by8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 1 + ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 1 + ...ap_skew-3-3678fb67b5c739bd87d4907630da1208 | 0 ...ap_skew-4-95474af63c0f92717ed49c3a0f37b10a | 0 ...ap_skew-5-a0a22ec83d6632cba3f17d79dbb9494d | 0 ...ap_skew-6-83296fd5b5fc22af0c51514c4e67c95f | 0 ...ap_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...ap_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 1 + ..._noskew-3-3678fb67b5c739bd87d4907630da1208 | 0 ..._noskew-4-95474af63c0f92717ed49c3a0f37b10a | 0 ..._noskew-5-a0a22ec83d6632cba3f17d79dbb9494d | 0 ..._noskew-6-83296fd5b5fc22af0c51514c4e67c95f | 0 ..._noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ..._noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...roupby9-0-3678fb67b5c739bd87d4907630da1208 | 0 ...roupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 | 0 ...oupby9-10-7f98b724df05f51b3ec1f087a8da414e | 1 + ...oupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f | 0 ...oupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 | 0 ...oupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...oupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...oupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed | 0 ...oupby9-16-c05b1bc66a607e43633dc457ecf48f3b | 0 ...oupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...oupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...oupby9-19-55781df7ed3ff9e37001fbd7739b9c2e | 0 ...roupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f | 0 ...oupby9-20-ccc5914317422f569e8b7171a3b2b243 | 0 ...oupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...oupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...roupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 | 0 ...roupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...roupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...roupby9-6-55781df7ed3ff9e37001fbd7739b9c2e | 0 ...roupby9-7-ccc5914317422f569e8b7171a3b2b243 | 0 ...roupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 ++ ...roupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 | 309 ++ ...bigdata-0-e011be1172043c0c6d0fd2c0e89f361e | 1 + ...bigdata-1-1100bb0c115c024998d35888ae5bbd71 | 1 + ...bigdata-2-3688b45adbdb190d58799c0b6d601055 | 1 + ...x_types-0-de39f8b5f4305136d945da94d5222283 | 0 ...x_types-1-10fe6134247226ab2b309bb62460d080 | 0 ...x_types-2-2500cd8e85b71222253a05a979442a4a | 0 ...x_types-3-85850072f0f9604d4e55a08fb9c45ba6 | 0 ...reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 | 1 + ...reducer-1-de39f8b5f4305136d945da94d5222283 | 0 ...reducer-2-10fe6134247226ab2b309bb62460d080 | 0 ...reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 | 0 ...samekey-0-63603572a0f759ea81f4649ae2210ef8 | 0 ...samekey-1-a3f3e279ab0be5093f4a926e265c0211 | 1 + ...samekey-2-59ee4550803e419b3adb1e9dd4220113 | 0 ...samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 | 0 ...samekey-4-d88c962262d3524bfc881b7309688e38 | 0 ...samekey-5-78c0b7be08118a14e0337ff552fcb9ad | 4 + ...samekey-6-edcbea736edb6104a73f0dc670418ee5 | 4 + ...samekey-7-ca16024e6f5399b1d035f5b9fd665163 | 0 ...map_ppr-0-dbcec232623048c7748b708123e18bf0 | 1 + ...map_ppr-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...map_ppr-2-83c59d378571a6e487aa20217bd87817 | 1 + ...map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 | 0 ...map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 | 0 ...map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a | 0 ...map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...istinct-0-dbcec232623048c7748b708123e18bf0 | 1 + ...istinct-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...istinct-2-83c59d378571a6e487aa20217bd87817 | 1 + ...istinct-3-a04c523002e79c588e350486c815e785 | 0 ...istinct-4-d6c134387c0c9343ec6ea88b5086dbe2 | 0 ...istinct-5-2efbb90d3df011282b70623e0905c390 | 0 ...istinct-6-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...istinct-0-dbcec232623048c7748b708123e18bf0 | 1 + ...istinct-1-388618d4d475de38d5c280fd7038730b | 0 ...stinct-10-6aa66df624cd32601218eee200977ce6 | 6 + ...stinct-11-59a77127c166fef57504c011ccb427a6 | 6 + ...istinct-2-2e1779fc58da364612a1c84d563ea7d5 | 0 ...istinct-3-febd68f62dd71550dd3db8335d1f93f7 | 0 ...istinct-4-c2781ed9ce9a29f484f7648ce6e06a9e | 0 ...istinct-5-6aa66df624cd32601218eee200977ce6 | 6 + ...istinct-6-59a77127c166fef57504c011ccb427a6 | 6 + ...istinct-7-293182ac89effb268855f5ac53e1ec11 | 1 + ...istinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 | 0 ...istinct-9-c2781ed9ce9a29f484f7648ce6e06a9e | 0 ...reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 | 1 + ...reducer-1-f64b982c4bf34931f03447e635ae33d2 | 0 ...educer-10-3b3ef4d975716744c85af560aa240abd | 10 + ...educer-11-4e95946ec07f04479da42ba5cbfa531b | 0 ...educer-12-51fe5b5a17ddc56fb8712340b38773b2 | 0 ...educer-13-38f492067df78144c272bb212633cc5e | 5 + ...educer-14-d7b3676444a9e95811184637dd0b3231 | 5 + ...educer-15-3b3ef4d975716744c85af560aa240abd | 10 + ...educer-16-b21ae88a5fc7f9b5c25ca954889ee421 | 10 + ...educer-17-7f344c2f066d74ecd135c43d39658bae | 32 + ...educer-18-4938ddc6b516cf67779be0d7dc29e7ab | 0 ...educer-19-180b611e08d4080fa098ed69169c2478 | 0 ...reducer-2-392062177be62090adedf1ab6c0a0b78 | 0 ...educer-20-db2149b42cdbf998187034233fc846cc | 0 ...reducer-21-322f23866cf3ca62d4ba93cf904c520 | 0 ...educer-22-21fcbcafc3b5b22572ddd5c87df1c06f | 0 ...reducer-3-82b65775ae9c6dcb85ca87804dbb0288 | 0 ...reducer-4-7df7c16e4063683d0ca40360da460799 | 0 ...reducer-5-c19e7bf0732e7a103966ebc43eb0bcea | 0 ...reducer-6-3be7ae6a87b9091bc61d221de8bdd55b | 0 ...reducer-7-ad39c0f14b6e0752489479138516bd57 | 0 ...reducer-8-38f492067df78144c272bb212633cc5e | 5 + ...reducer-9-d7b3676444a9e95811184637dd0b3231 | 5 + ...educer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 | 1 + ...educer2-1-660d3ff0ca588c2da097b5f3ca753ada | 0 ...educer2-2-7372ea92a15e7beed3be5e2dd2cbac47 | 0 ...educer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e | 0 ...educer2-4-427627f4cf6d91e5314c85e0c5aa2f84 | 0 ...educer2-5-376542befbcab97d864e874251720c40 | 5 + ...educer2-6-8566e4c94d34b0fc4a71d34e89529c74 | 5 + ...educer2-7-4938ddc6b516cf67779be0d7dc29e7ab | 0 ...educer2-8-180b611e08d4080fa098ed69169c2478 | 0 ...educer3-0-36cc74ebca5adb026757c5cd8df5a0dd | 0 ...educer3-1-9e61989d717403353689cbbb2816210d | 0 ...ducer3-10-73819ea1a7c0653a61652b3766afb003 | 1 + ...ducer3-11-8603afa3bf3deeda532fc69b7df49e09 | 0 ...ducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 | 0 ...ducer3-13-521e0c1054cfa35116c02245874a4e69 | 2 + ...ducer3-14-a9a491ed99b4629ee15bce994d5c6c63 | 1 + ...ducer3-15-36e81eba0a6a42532d8ee147086d668a | 0 ...ducer3-16-59b2c947e136092e7ca5019c96a9994b | 0 ...ducer3-17-521e0c1054cfa35116c02245874a4e69 | 2 + ...ducer3-18-a9a491ed99b4629ee15bce994d5c6c63 | 1 + ...educer3-2-8603afa3bf3deeda532fc69b7df49e09 | 0 ...educer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 | 0 ...educer3-4-521e0c1054cfa35116c02245874a4e69 | 2 + ...educer3-5-a9a491ed99b4629ee15bce994d5c6c63 | 1 + ...educer3-6-36e81eba0a6a42532d8ee147086d668a | 0 ...educer3-7-59b2c947e136092e7ca5019c96a9994b | 0 ...educer3-8-521e0c1054cfa35116c02245874a4e69 | 2 + ...educer3-9-a9a491ed99b4629ee15bce994d5c6c63 | 1 + ...g_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb | 1 + ...g_float-1-4a435e268a327404f75725b82a32ee03 | 1 + ...osition-0-422c2068a838f59324c1d9861225c824 | 1 + ...osition-1-6b06902de5c0ca13cebe03018d86f447 | 0 ...sition-10-b4c6c0aedcc924e2af65549b87b3f3f7 | 13 + ...sition-11-75a28e558d2fb7a78f43f55b0479c646 | 0 ...sition-12-5583e5cfcf8083d45a3bd80434c1829f | 14 + ...sition-13-9ee2150594ad2eece6ee14424155d396 | 0 ...sition-14-6f54558123eb5d63eeeb3e261c1dc5fb | 3 + ...osition-2-627bb7be9c0edb5ba4c677912800d364 | 0 ...osition-3-c39bd6c3c24658ec082bef9876d6e2ac | 0 ...osition-4-19461cbd2be1f2f3f3b65478e3eeb53c | 0 ...osition-5-e2470670c5d709001fb17ecbc6e4f85d | 13 + ...osition-6-b4c6c0aedcc924e2af65549b87b3f3f7 | 13 + ...osition-7-5b32a45af11e04b46f8566bd27a28014 | 0 ...osition-8-3ff6c255dda69ec117c6a7ec93db2f1c | 0 ...osition-9-e2470670c5d709001fb17ecbc6e4f85d | 13 + ...pby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...pby_ppr-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...pby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 | 0 ...pby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 | 0 ...pby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a | 0 ...pby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...istinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...istinct-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...istinct-2-3aa4057488720c0f514696154f2070b5 | 0 ...istinct-3-ace7b2624b125764e9f1f6b5559f023d | 0 ...distinct-4-61ae2f9b3a9a2f60a307aa25e42425d | 0 ...istinct-5-adc1ec67836b26b60d8547c4996bfd8f | 10 + ..._sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._sort_1-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 | 0 ...sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c | 0 ...sort_1-13-46c4a3675c8de0510b648856a193f3e7 | 0 ...sort_1-14-bbdd53118f788d7bb679d094c41243c8 | 6 + ...sort_1-15-54f3a2a97939d3eca8a601b74ef30dea | 0 ...sort_1-16-236d51792f4913b5858f367e3cff2c60 | 0 ...sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 | 0 ...sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 | 0 ..._sort_1-2-fc30020d09151dc29be807795ad9475e | 1 + ...sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...sort_1-21-27e2e662d517f32952145cc2a51bf564 | 0 ...sort_1-22-6775cb6aee040e22508cf3cac0b55f06 | 0 ...sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 | 0 ...sort_1-24-d53196339980a00a619788bd799a32e7 | 5 + ...sort_1-25-138e93f004f7bd16e63bbf8d9090af21 | 0 ...sort_1-26-93153fd67c1d19bb9ad64f98294e4981 | 0 ...sort_1-27-ca369ab23c32070e2d42ba8df036175f | 0 ...sort_1-28-c4ec0433a832ef551d70254957e3afca | 6 + ...sort_1-29-4095695e88e23dd42acb1749a83bdbb7 | 0 ..._sort_1-3-fffea659b633b1f269b38556a7f54634 | 1 + ...sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 | 0 ...sort_1-31-d53196339980a00a619788bd799a32e7 | 5 + ...sort_1-32-c88ee457dee7bb141a198a2ae39d787f | 0 ...sort_1-33-c04167e880fe3e942ce77e75d660f1ef | 0 ...sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...sort_1-35-deb3f1793f51d1edf011a8405abf4968 | 0 ...sort_1-36-7871722f392f801a868e0e2fb372c610 | 0 ...sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd | 10 + ...sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d | 0 ...sort_1-39-89aa7bab4272546e06cd7e504380d96b | 0 ..._sort_1-4-e906be6d27c9dfcffd4af171541639ad | 0 ...sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd | 10 + ...sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 | 0 ...sort_1-42-83889e7dc73d796cc869160b6b35102c | 0 ...sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...sort_1-44-41462b2e60bf44571a7b1fb435374d6a | 0 ...sort_1-45-8aea6edf0481e2a10f14879acba62648 | 0 ...sort_1-46-4999db9eb86d9455c1d75e97b052f279 | 0 ...sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa | 0 ...sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 | 0 ...sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ..._sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...sort_1-50-461847b174096e7a255fb07cb35ab434 | 0 ...sort_1-51-8da967e7c2210ad044ba8b08d1685065 | 0 ...sort_1-52-c4ec0433a832ef551d70254957e3afca | 6 + ...sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 | 0 ...sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c | 0 ...sort_1-55-1013d1ad014aa203b1dce26085b09c01 | 0 ...sort_1-56-5373c5449884d95bc7db9dab55239a49 | 6 + ...sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 | 0 ...sort_1-58-e671e63f6b70094048563a9c33748c97 | 0 ...sort_1-59-c4ec0433a832ef551d70254957e3afca | 6 + ..._sort_1-6-b6c452a800ff333aacb863bb3243c15b | 0 ...sort_1-60-51824b04f2a008f63e1469695e60d9c8 | 0 ...sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 | 0 ...sort_1-62-c4ec0433a832ef551d70254957e3afca | 6 + ...sort_1-63-dbcec232623048c7748b708123e18bf0 | 1 + ...sort_1-64-7f98b724df05f51b3ec1f087a8da414e | 1 + ...sort_1-65-83c59d378571a6e487aa20217bd87817 | 1 + ...sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 | 0 ...sort_1-67-b4fec0996399be2239961594897d6715 | 0 ...sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 + ...sort_1-69-ed76c0068780120a6f23feefee303403 | 1 + ..._sort_1-7-c0ea81b686236d661166912040a16ea7 | 0 ...sort_1-70-2d5403943a3efdf9fd3eccb6927499cc | 0 ...sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e | 0 ...sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 | 5 + ...sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd | 6 + ...sort_1-74-6f7caef1c773268350c9cf94ad85be01 | 0 ...sort_1-75-4931c5a72a5231f67317d27ca025bb97 | 0 ...sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 | 1 + ...sort_1-77-6d6eac558569563cc47a657daffa8ba7 | 2 + ...y_sort_1-8-f0ee61903aeacb758e2eada242e5e14 | 0 ..._sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 | 0 ...sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...sort_10-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...ort_10-10-b4e225bc4787d7718bae6f00d8addfe2 | 2 + ...ort_10-11-4d9341036906853bb9a1641f5e0179b3 | 0 ...sort_10-2-fc30020d09151dc29be807795ad9475e | 1 + ...sort_10-3-fffea659b633b1f269b38556a7f54634 | 1 + ...sort_10-4-475d50465b23adfb70e67122425ede9e | 0 ...sort_10-5-91f5326fe696124d862dfcfb72de2bf4 | 0 ...sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 | 0 ...sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 | 2 + ...sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 | 0 ..._sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c | 0 ...sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...sort_11-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...ort_11-10-dc4ffd56a4be5e2c52df2f70c614720d | 0 ...ort_11-11-d9bf7e80b71121935ed4b008ae916cb1 | 6 + ...ort_11-12-56f9f2a13698e71c4f00c93c48dffc30 | 0 ...ort_11-13-c7d70702783eb90d4f53028a63c318f8 | 6 + ...ort_11-14-d403afd709251edba6ae13b8cc14e0b7 | 0 ...ort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 | 1 + ...ort_11-16-166ae99e823bc1bcbee39a466f9e3551 | 0 ...ort_11-17-5298cc011d107ad06c365d132cab6fc9 | 1 + ...ort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...ort_11-19-46ca52d697918f2327b2293d9fd57d15 | 0 ..._sort_11-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...ort_11-20-bd99462ed878bf4bec74b3cb9132908d | 1 + ...sort_11-3-fffea659b633b1f269b38556a7f54634 | 1 + ...sort_11-4-475d50465b23adfb70e67122425ede9e | 0 ...sort_11-5-7ca5ebad57578206b8830da6746fb952 | 0 ..._sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 | 0 ...sort_11-7-bd99462ed878bf4bec74b3cb9132908d | 1 + ...sort_11-8-4bf6c5e938afa4f937b69d2a095c675c | 0 ...sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 | 1 + ..._sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._sort_2-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 | 6 + ..._sort_2-2-fc30020d09151dc29be807795ad9475e | 1 + ..._sort_2-3-fffea659b633b1f269b38556a7f54634 | 1 + ..._sort_2-4-7dc6959cca820ea825e6567e1b152088 | 0 ..._sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ..._sort_2-6-b6c452a800ff333aacb863bb3243c15b | 0 ..._sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 | 0 ..._sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 | 0 ..._sort_2-9-30377eb1022ca1f4f5201f8897dff9ed | 0 ..._sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._sort_3-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...sort_3-10-4bf8dba6e66e733423a3408d61897926 | 6 + ...sort_3-11-2437764cdf86cd2d67430bd323346086 | 0 ...sort_3-12-bc5b02bd034ead563d39d2685087005e | 0 ..._sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd | 0 ...sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f | 5 + ..._sort_3-2-fc30020d09151dc29be807795ad9475e | 1 + ..._sort_3-3-fffea659b633b1f269b38556a7f54634 | 1 + ..._sort_3-4-74ba824287893d3aaa1cdd957d472729 | 0 ..._sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ..._sort_3-6-b6c452a800ff333aacb863bb3243c15b | 0 ..._sort_3-7-fa27a0ebfb1024248031f281b5e320d8 | 0 ...y_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 | 0 ..._sort_3-9-37a53e292752d1a4caff70e64c5cdfbd | 0 ..._sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._sort_4-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 | 0 ...sort_4-12-1f082ad7217ad620063b58887b9b922f | 0 ...sort_4-13-46c4a3675c8de0510b648856a193f3e7 | 0 ...sort_4-14-75d59344b6114c0bb20d5eac301c2170 | 6 + ..._sort_4-2-fc30020d09151dc29be807795ad9475e | 1 + ..._sort_4-3-fffea659b633b1f269b38556a7f54634 | 1 + ..._sort_4-4-40891a8b7f896b11de173835c51aca4f | 0 ..._sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ..._sort_4-6-b6c452a800ff333aacb863bb3243c15b | 0 ..._sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e | 0 ..._sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef | 0 ..._sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 | 0 ..._sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._sort_5-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...sort_5-10-4bf8dba6e66e733423a3408d61897926 | 6 + ...sort_5-11-4d9341036906853bb9a1641f5e0179b3 | 0 ...sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f | 0 ...sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...sort_5-14-b6c452a800ff333aacb863bb3243c15b | 0 ...sort_5-15-e9baee9e98db00b9277dae33097aab82 | 0 ...sort_5-16-37a53e292752d1a4caff70e64c5cdfbd | 0 ...sort_5-17-4bf8dba6e66e733423a3408d61897926 | 6 + ...sort_5-18-4d9341036906853bb9a1641f5e0179b3 | 0 ...sort_5-19-2e441f92547a956942f4451e943bf5cf | 0 ..._sort_5-2-fc30020d09151dc29be807795ad9475e | 1 + ...sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...sort_5-21-b6c452a800ff333aacb863bb3243c15b | 0 ...sort_5-22-70a6c959960c299db2b961dffaa2628d | 0 ...sort_5-23-8dce8ffad6523bbe076b968e243f094f | 0 ..._sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd | 0 ...sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f | 5 + ...sort_5-26-4d9341036906853bb9a1641f5e0179b3 | 0 ..._sort_5-3-fffea659b633b1f269b38556a7f54634 | 1 + ..._sort_5-4-f3697ac93bcda24a9d7593b703b312e7 | 0 ..._sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ..._sort_5-6-b6c452a800ff333aacb863bb3243c15b | 0 ..._sort_5-7-5219a87f995d294a0c68ae6499dba7d2 | 0 ..._sort_5-8-e9baee9e98db00b9277dae33097aab82 | 0 ..._sort_5-9-37a53e292752d1a4caff70e64c5cdfbd | 0 ..._sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._sort_6-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...sort_6-10-788b62269990c43aa3cb1847f99b3343 | 0 ...sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 | 0 ...sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...sort_6-13-73805f987b3b2384352c5dd5201f1b29 | 0 ...sort_6-14-5574207f68aac30a893785c50c735864 | 0 ...sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ..._sort_6-2-fc30020d09151dc29be807795ad9475e | 1 + ..._sort_6-3-fffea659b633b1f269b38556a7f54634 | 1 + ..._sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e | 0 ..._sort_6-5-c0ea81b686236d661166912040a16ea7 | 0 ..._sort_6-6-788b62269990c43aa3cb1847f99b3343 | 0 ..._sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 | 0 ..._sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd | 0 ..._sort_6-9-7b84dbb0895a623d460c70678bd74a64 | 0 ..._sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._sort_7-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...sort_7-10-4bf8dba6e66e733423a3408d61897926 | 6 + ...sort_7-11-4d9341036906853bb9a1641f5e0179b3 | 0 ..._sort_7-2-fc30020d09151dc29be807795ad9475e | 1 + ..._sort_7-3-fffea659b633b1f269b38556a7f54634 | 1 + ..._sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 | 0 ..._sort_7-5-8b5d511014e1776743cacaf77f68d5fb | 0 ..._sort_7-6-137e81fded2b36969bf71279d7ffee34 | 0 ..._sort_7-7-5219a87f995d294a0c68ae6499dba7d2 | 0 ..._sort_7-8-289632719165e6e8182ebd8f5f766b7b | 0 ..._sort_7-9-6e36549828003492627aa5e96a63d3a2 | 0 ..._sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._sort_8-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 | 0 ...sort_8-11-bd99462ed878bf4bec74b3cb9132908d | 1 + ...sort_8-12-4d9341036906853bb9a1641f5e0179b3 | 0 ..._sort_8-2-fc30020d09151dc29be807795ad9475e | 1 + ..._sort_8-3-fffea659b633b1f269b38556a7f54634 | 1 + ..._sort_8-4-14283ed438d96ad881025b969c8fb69f | 0 ..._sort_8-5-8b5d511014e1776743cacaf77f68d5fb | 0 ..._sort_8-6-137e81fded2b36969bf71279d7ffee34 | 0 ..._sort_8-7-8188c7bcb9ead25f8c27af7def87218d | 0 ..._sort_8-8-bd99462ed878bf4bec74b3cb9132908d | 1 + ..._sort_8-9-8e06b51e940e956f14a8c7679c3d423a | 1 + ..._sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._sort_9-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...sort_9-10-4d9341036906853bb9a1641f5e0179b3 | 0 ..._sort_9-2-fc30020d09151dc29be807795ad9475e | 1 + ..._sort_9-3-fffea659b633b1f269b38556a7f54634 | 1 + ..._sort_9-4-14283ed438d96ad881025b969c8fb69f | 0 ..._sort_9-5-8b5d511014e1776743cacaf77f68d5fb | 0 ..._sort_9-6-137e81fded2b36969bf71279d7ffee34 | 0 ..._sort_9-7-c762061ecb1eea7ac218809b9e49900c | 0 ..._sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 | 0 ...y_sort_9-9-feec69facdc973a0ff78455f766845c | 5 + ..._skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._skew_1-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 | 0 ...skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 | 0 ...skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c | 0 ...skew_1-14-46c4a3675c8de0510b648856a193f3e7 | 0 ...skew_1-15-bbdd53118f788d7bb679d094c41243c8 | 6 + ...skew_1-16-54f3a2a97939d3eca8a601b74ef30dea | 0 ...skew_1-17-236d51792f4913b5858f367e3cff2c60 | 0 ...skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 | 0 ..._skew_1-2-fc30020d09151dc29be807795ad9475e | 1 + ...skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 | 0 ...skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...skew_1-22-27e2e662d517f32952145cc2a51bf564 | 0 ...skew_1-23-6775cb6aee040e22508cf3cac0b55f06 | 0 ...skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 | 0 ...skew_1-25-d53196339980a00a619788bd799a32e7 | 5 + ...skew_1-26-138e93f004f7bd16e63bbf8d9090af21 | 0 ...skew_1-27-93153fd67c1d19bb9ad64f98294e4981 | 0 ...skew_1-28-ca369ab23c32070e2d42ba8df036175f | 0 ...skew_1-29-c4ec0433a832ef551d70254957e3afca | 6 + ..._skew_1-3-fffea659b633b1f269b38556a7f54634 | 1 + ...skew_1-30-4095695e88e23dd42acb1749a83bdbb7 | 0 ...skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 | 0 ...skew_1-32-d53196339980a00a619788bd799a32e7 | 5 + ...skew_1-33-c88ee457dee7bb141a198a2ae39d787f | 0 ...skew_1-34-c04167e880fe3e942ce77e75d660f1ef | 0 ...skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...skew_1-36-deb3f1793f51d1edf011a8405abf4968 | 0 ...skew_1-37-7871722f392f801a868e0e2fb372c610 | 0 ...skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd | 10 + ...skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d | 0 ..._skew_1-4-c67a488530dc7e20a9e7acf02c14380f | 1 + ...skew_1-40-89aa7bab4272546e06cd7e504380d96b | 0 ...skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd | 10 + ...skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 | 0 ...skew_1-43-83889e7dc73d796cc869160b6b35102c | 0 ...skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...skew_1-45-41462b2e60bf44571a7b1fb435374d6a | 0 ...skew_1-46-8aea6edf0481e2a10f14879acba62648 | 0 ...skew_1-47-4999db9eb86d9455c1d75e97b052f279 | 0 ...skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa | 0 ...skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 | 0 ..._skew_1-5-e906be6d27c9dfcffd4af171541639ad | 0 ...skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd | 5 + ...skew_1-51-461847b174096e7a255fb07cb35ab434 | 0 ...skew_1-52-8da967e7c2210ad044ba8b08d1685065 | 0 ...skew_1-53-c4ec0433a832ef551d70254957e3afca | 6 + ...skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 | 0 ...skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c | 0 ...skew_1-56-1013d1ad014aa203b1dce26085b09c01 | 0 ...skew_1-57-5373c5449884d95bc7db9dab55239a49 | 6 + ...skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 | 0 ...skew_1-59-e671e63f6b70094048563a9c33748c97 | 0 ..._skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...skew_1-60-c4ec0433a832ef551d70254957e3afca | 6 + ...skew_1-61-51824b04f2a008f63e1469695e60d9c8 | 0 ...skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 | 0 ...skew_1-63-c4ec0433a832ef551d70254957e3afca | 6 + ...skew_1-64-dbcec232623048c7748b708123e18bf0 | 1 + ...skew_1-65-7f98b724df05f51b3ec1f087a8da414e | 1 + ...skew_1-66-83c59d378571a6e487aa20217bd87817 | 1 + ...skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 | 0 ...skew_1-68-b4fec0996399be2239961594897d6715 | 0 ...skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 + ..._skew_1-7-b6c452a800ff333aacb863bb3243c15b | 0 ...skew_1-70-ed76c0068780120a6f23feefee303403 | 1 + ...skew_1-71-2d5403943a3efdf9fd3eccb6927499cc | 0 ...skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e | 0 ...skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 | 5 + ...skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd | 6 + ...skew_1-75-6f7caef1c773268350c9cf94ad85be01 | 0 ...skew_1-76-4931c5a72a5231f67317d27ca025bb97 | 0 ...skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 | 1 + ...skew_1-78-6d6eac558569563cc47a657daffa8ba7 | 2 + ..._skew_1-8-c0ea81b686236d661166912040a16ea7 | 0 ...t_skew_1-9-f0ee61903aeacb758e2eada242e5e14 | 0 ..._test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ..._test_1-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ..._test_1-2-fc30020d09151dc29be807795ad9475e | 1 + ..._test_1-3-fffea659b633b1f269b38556a7f54634 | 1 + ..._test_1-4-8e06b51e940e956f14a8c7679c3d423a | 1 + ..._test_1-5-e906be6d27c9dfcffd4af171541639ad | 0 ..._test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ..._test_1-7-b6c452a800ff333aacb863bb3243c15b | 0 ..._test_1-8-c0ea81b686236d661166912040a16ea7 | 0 ..._test_1-9-4d3e8128fb29c232b984f41ed4e78794 | 0 .../hash-0-a658b129316d666d4b01c1581eed1c1f | 1 + ...text_cs-0-e319c8574a6cd8739e5fd5984ceed3cf | 0 ...text_cs-1-de3aa1c4674fb166b825b1e2f58d1950 | 0 ...t_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 | 0 ...t_cast1-1-66f185b6fdccddba498c463641e7dc7a | 0 ...t_cast1-2-6871be041d36ea813e1b2f331e3747f4 | 0 ...reation-0-f880114c33c99a5f23c1465fd88f0db3 | 0 ...reation-1-a8bc76559014d9cdf07184208d582d25 | 0 ...eation-10-4bd6c86cd3f6a94964b3d13235e8e261 | 5 + ...eation-11-b5b7e4f7af5186033be12a4393dc3bb7 | 0 ...eation-12-9cc02e06c6051810c50e225bb2c66669 | 5 + ...eation-13-9c0ec67e79a0d50b46bd5c944c710fc4 | 0 ...eation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 | 5 + ...reation-15-7d7e1a479e7dcd8f9d4199775e05bc1 | 0 ...eation-16-ffa6d3fcef97b7322dd3759d4a70881d | 5 + ...eation-17-18ed0b70b0b6d076b96c9f73bfa721ad | 0 ...eation-18-bcdb19db031341c4a50264ccf49328e4 | 5 + ...eation-19-98dbf83283f9e073e88ba770ec5a707f | 0 ...creation-2-9d5d11cb38f2e097f16d2db5693f4f1 | 0 ...eation-20-68c5c98581c683b17ceaf1c0fd192871 | 5 + ...eation-21-4c8f6b48c437bf0be109fc0be1dc840e | 0 ...reation-22-f36cb2eed39691ca949b25182e2dd31 | 5 + ...eation-23-e7f21f556d3aa2bedb0717a167720741 | 0 ...eation-24-8cb102bd5addf7fece0e2691468bc3bf | 5 + ...eation-25-33c48966230b934ae8ddf74ff18bb9ca | 0 ...eation-26-f85db55b3f63ae186a1b6d5cec545939 | 0 ...eation-27-e4856f13692e63d61f72aaf75e50e5f1 | 0 ...eation-28-bd20d4b59e6489082a92fcbfcc5f8dbe | 0 ...eation-29-ee8d287111069805c41c9c0032adc46f | 0 ...reation-3-14b999fc6dfb10a3632afe14e08003e1 | 0 ...eation-30-f880114c33c99a5f23c1465fd88f0db3 | 0 ...eation-31-a8bc76559014d9cdf07184208d582d25 | 0 ...reation-32-9d5d11cb38f2e097f16d2db5693f4f1 | 0 ...eation-33-14b999fc6dfb10a3632afe14e08003e1 | 0 ...eation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 | 0 ...eation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 | 0 ...eation-36-21bcf37075b02097f16c8fc8130a83b8 | 0 ...eation-37-9334418431eca405f13206bd8db42a1b | 0 ...eation-38-f1f56119aede4f42221a68f6aaa42a26 | 0 ...eation-39-489b4ceb2f4301a7132628303f99240d | 1 + ...reation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 | 0 ...reation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 | 0 ...reation-6-21bcf37075b02097f16c8fc8130a83b8 | 0 ...reation-7-9334418431eca405f13206bd8db42a1b | 0 ...reation-8-f1f56119aede4f42221a68f6aaa42a26 | 0 ...reation-9-bf40d4d50d050f2f8342c07f5a9dcf0c | 0 ...x_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 | 0 ...x_serde-1-f92d6c66d21791c11d2a822df04c1b63 | 3 + ..._serde-10-123301a057d4a46072d0431e00e20c4b | 5 + ..._serde-11-309e916d683a1a12ab62565697cb0046 | 0 ...x_serde-12-d590fd7cb9d433143de490d75686dd4 | 0 ...ex_serde-2-fd1b220f4eafb0ba9b519a156e3c87c | 0 ...x_serde-3-afcf2a156ccd4f79a0489b4593908d79 | 0 ...x_serde-4-d7547751c37375a9238043bbe250e716 | 5 + ...x_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 | 0 ...x_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae | 1 + ...x_serde-7-3b03210f94ec40db9ab02620645014d1 | 1 + ...x_serde-8-35f48c7d6fa164bb84643657bc9280a8 | 1 + ...x_serde-9-c85e061ea9c5b90ca69b7450faad14b6 | 0 ...st_type-0-e231c5154b18cbc0baa082a7461dd13e | 0 ...st_type-1-c836a98522549d2a3fd43998afd8ae94 | 0 ...st_type-2-d8590c7336ae771b7a685bb544e8d2bd | 0 ...nerjoin-0-43d53504df013e6b35f81811138a167a | 1 + ...nerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e | 0 ...nerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 | 0 ...nerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 | 0 ...nerjoin-4-35c7611180562dcd9dab834f41654095 | 1028 +++++++ ...nerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a | 0 ...nerjoin-6-17c49c593f6160e096b30dfee7b643ce | 0 ...nerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 | 0 ...nerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 | 0 ...nerjoin-9-326eeac56676d78fe489c464cddb526b | 0 ...tdriver-0-47981488ab51ed3cc6c335f5cf703908 | 0 ...tdriver-1-b2f337566a5075f3e3e81335008d95d3 | 3 + .../input-0-42022446607cfbe8a64d8fd7b9898ce7 | 0 .../input-1-6558e385bb08991302a72076dd7b7ff5 | 500 ++++ .../input0-0-fb47f8a8a8e3213f3fe47825a556c593 | 0 .../input0-1-efefd4364cd2790447fb0f908e87501f | 500 ++++ .../input1-0-28c9f2913833d6911f22e2e2e8c60f68 | 0 .../input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 | 17 + .../input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 | 2 + ...input10-0-acdd72fda64c5463ce29f31020012d42 | 0 ...input10-1-6970b6d2d451612b59fccbfd7ec68f74 | 17 + ...input10-2-73f00da5cfc254745d1d80f913eb6449 | 10 + ...input11-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...input11-1-3e66f12ae780a85721fa21c42fb3c8fb | 0 ...input11-2-24fe55bae88ad4a8e240376d012bc491 | 0 ...input11-3-adc1ec67836b26b60d8547c4996bfd8f | 84 + ...1_limit-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...1_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 | 0 ...1_limit-2-47fdba9764a8851379e2ed7e16c54583 | 0 ...1_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 | 10 + ...input12-0-9b141c1e5917ca82c6bc36a9a2950a1e | 1 + .../input12-1-2b9ccaa793eae0e73bf76335d3d6880 | 1 + ...nput12-10-4d9eb316259a8e7ed6627bc27a639f7c | 311 ++ ...input12-2-bab89dfffa77258e34a595e0e79986e3 | 1 + ...input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 | 0 ...input12-4-f69ce1f5b3e0e77d1c487877580e6a23 | 0 ...input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf | 0 ...input12-6-f6ae011490e0c8c037927767ad966ce4 | 0 ...input12-7-16da1c1293626b943343443368679e9c | 0 ...input12-8-adc1ec67836b26b60d8547c4996bfd8f | 84 + ...input12-9-3d08dc27c1a133c2497fc554c0d169bd | 105 + ...adoop20-0-db1cd54a4cb36de2087605f32e41824f | 1 + ...adoop20-1-bab89dfffa77258e34a595e0e79986e3 | 1 + ...adoop20-2-743c24cd29161d31a16ec446bd709d92 | 0 ...adoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 | 0 ...adoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf | 0 ...adoop20-5-f6ae011490e0c8c037927767ad966ce4 | 0 ...adoop20-6-16da1c1293626b943343443368679e9c | 0 ...adoop20-7-adc1ec67836b26b60d8547c4996bfd8f | 84 + ...adoop20-8-3d08dc27c1a133c2497fc554c0d169bd | 105 + ...adoop20-9-4d9eb316259a8e7ed6627bc27a639f7c | 311 ++ ...input14-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...input14-1-d0a1caeeeb79580713b7ecc28543f74a | 0 ...input14-2-1d791beabaa5288ea2fcf6b3675eda26 | 0 ...input14-3-adc1ec67836b26b60d8547c4996bfd8f | 84 + ...4_limit-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...4_limit-1-be9934fc5e6ecb9854eb7531a5929dcf | 0 ...4_limit-2-780cdc89e0e736790124b6bdac827951 | 0 ...4_limit-3-adc1ec67836b26b60d8547c4996bfd8f | 5 + ...ut16_cc-0-43d53504df013e6b35f81811138a167a | 1 + ...ut16_cc-1-5180e975a6babd51752706f1799e7df5 | 0 ...ut16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 | 1 + ...input17-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...input17-1-be9cde5e769f171f60f61a7739de8f17 | 0 ...input17-2-21166e268096f6ec67f4f57ec333e901 | 0 ...input17-3-adc1ec67836b26b60d8547c4996bfd8f | 11 + ...input19-0-a8eaa79d3a2762cdb263bc822d62df3a | 0 .../input19-1-f2832e249ab28bb3fb8e472012c5ffc | 0 ...input19-2-5a804e02e4419e785d15e7f39d6c5730 | 1 + ...1_limit-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 | 0 ...1_limit-2-c1d996fc96189d323f6af180708abc12 | 0 ...t1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e | 0 ...1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 | 10 + ...1_limit-5-eaaf713833e28a803c798562c7d6cd23 | 5 + .../input2-0-eaf4ec8905774e4be6ea3fa4f32a456c | 0 .../input2-1-e0efeda558cd0194f4764a5735147b16 | 2 + .../input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd | 2 + .../input2-3-45b9f786f64afa5e039d1856fe926649 | 0 .../input2-4-235f92683416fab031e6e7490487b15b | 3 + .../input2-5-9c36cac1372650b703400c60dd29042c | 4 + .../input2-6-4a6b9674c126337f71834f11613f996d | 0 .../input2-7-9c36cac1372650b703400c60dd29042c | 3 + .../input2-8-28c3fc507cd2e02a3a39331fc9c95334 | 0 .../input2-9-48bf8c06ed0a264d0863613fe79795e1 | 17 + ...input21-0-7f40c525398953892ffd6283f54ba427 | 0 ...input21-1-70b803742328eacc69eb1ed044a5c6b8 | 0 ...input21-2-a4b3aeb45ae0cf38777e126faed0eff7 | 0 ...input21-3-9809b74435cbaedef0dc6e6b88b180fe | 10 + ...input22-0-b9cd4041ca510639838a97376747b498 | 0 ...input22-1-b7f46eccd104e6ed1b29e2de45089f66 | 0 ...input22-2-10e46b53620b6b05019da4e58f53e6c3 | 0 ...input22-3-8285c1934441b12f6c016e13cb410e79 | 10 + ...input23-0-c983ab8049996ad5e1e0296b51ae81cf | 0 ...input23-1-c8e32187d09172eb32b0480fcd28cccb | 0 ...input24-0-f8116598acadb7589ae02b13b65e1ad8 | 0 .../input24-1-f57485de80ad64f9931bf92514fa1 | 0 ...input24-2-3fc20939000a5324a928feeb673a1ee2 | 0 ...input24-3-3189f3b2990de94619b9cb583d9dd3c5 | 1 + ...input25-0-f8116598acadb7589ae02b13b65e1ad8 | 0 .../input25-1-f57485de80ad64f9931bf92514fa1 | 0 ...input25-2-74a4b9d59ee26b650f567c07e9103b66 | 0 ...input25-3-d4940dd537ae72d234ffea23f8f0b103 | 0 ...input25-4-72c74c55dd7d85d194448e9c58488938 | 0 ...input26-0-1b6e9e73dd7ee60644c0997b21f48430 | 0 ...input26-1-8272225744e83ba4cbe158a5c113fce3 | 5 + ...input28-0-b85d5edb4640c4b154e91e9a1beeef8a | 0 .../input28-1-f57485de80ad64f9931bf92514fa1 | 0 ...input28-2-e88e8d960eeb128fbc6642274780ce4c | 0 ...input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 | 0 ...2_limit-0-44591b07e007def036cea6c61a3f0a7f | 0 ...2_limit-1-fed7e0bb996623da7dd17793e835f785 | 5 + .../input3-0-2c80ec90d4d2c9c7446c05651bb76bff | 0 .../input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d | 2 + ...input3-10-10a1a8a97f6417c3da16829f7e519475 | 4 + ...input3-11-9c36cac1372650b703400c60dd29042c | 4 + ...input3-12-a22d09de72e5067a0a94113cdecdaa95 | 14 + ...input3-13-23bbec31affef0d758bc4a40490e0b9a | 0 ...input3-14-efee6816e20fe61595a4a2a991071219 | 4 + .../input3-2-fa2aceba8cdcb869262e8ad6d431f491 | 0 .../input3-3-1c5990b1aed2be48311810dae3019994 | 3 + .../input3-4-9c36cac1372650b703400c60dd29042c | 4 + .../input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 | 14 + .../input3-6-ba8c440158c2519353d02471bfb05694 | 0 .../input3-7-1c5990b1aed2be48311810dae3019994 | 4 + .../input3-8-4dc0fefca4d158fd2ab40551ae9e35be | 14 + .../input3-9-5076c1c35053b09173f6acdf1b5e9d6e | 0 ...input30-0-582c5fcbe2fe12cc8e7b21225583d96c | 0 ...input30-1-90c0d1a75de78c405413fd627caea4ab | 0 .../input30-2-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...input30-3-c21dba410fb07a098f93430a9d21df79 | 1 + ...input30-4-f0ebd08e7675b19ae831824ef4d9e223 | 0 ...input30-5-38734677c27b5f90a8df5da6e6351c76 | 0 .../input30-6-f120ac8c87db9eebb5da7ac99b48600 | 1 + ...input30-7-95d10d459c088d5fbefd00bdd8d44c3f | 1 + .../input31-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...input31-1-c21dba410fb07a098f93430a9d21df79 | 1 + .../input31-2-705764f8f7cab9378964af30b83f7fe | 0 ...input31-3-50c905261882f7fd8539fdd91e68151f | 0 ...input31-4-2f886fa357df9342733551fa1b53f913 | 0 .../input32-0-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...input32-1-c21dba410fb07a098f93430a9d21df79 | 1 + ...input32-2-1ba7748b3d2f8908c2e81771ab229316 | 1 + ...input32-3-b0070890240c15d647af59f41b77ba3d | 0 ...input32-4-d0e1378a30e21e0198e47d9b668ee1f6 | 0 ...input32-5-8789d32fc5b784fe2d171566732c573e | 0 ...input37-0-86e2e274650fb56651607ea10d356fc0 | 0 ...input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 | 0 ...input39-0-7bd12162381231be9d578797818957a7 | 0 ...input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 | 0 ...nput39-10-28bf1b34c04a048da339eddd4c1fd779 | 0 ...nput39-11-6d0814cbb72eb96bfc75c95d06f1e528 | 1 + ...input39-12-f120ac8c87db9eebb5da7ac99b48600 | 1 + ...nput39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 | 1 + ...nput39-14-bcc1d5fd287f81bac1092a913b09956d | 1 + ...input39-2-ee667c6ab003bb83f6bf1c72153eba39 | 0 ...input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e | 0 ...input39-4-53453776bf062f28d371fc7336b7eae2 | 0 .../input39-5-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...input39-6-763ab5853bff619e6525c01e46b2a923 | 1 + ...input39-7-9b141c1e5917ca82c6bc36a9a2950a1e | 1 + .../input39-8-2b9ccaa793eae0e73bf76335d3d6880 | 1 + ...input39-9-bab89dfffa77258e34a595e0e79986e3 | 1 + ...adoop20-0-4c33233bafd910d69c2b8845d35abffe | 0 ...adoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 | 0 ...doop20-10-6d0814cbb72eb96bfc75c95d06f1e528 | 1 + ...adoop20-11-f120ac8c87db9eebb5da7ac99b48600 | 1 + ...doop20-12-a6a77ae749a7e7f8022d71c03322fc21 | 1 + ...adoop20-2-ee667c6ab003bb83f6bf1c72153eba39 | 0 ...adoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e | 0 ...adoop20-4-53453776bf062f28d371fc7336b7eae2 | 0 ...hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee | 1 + ...adoop20-6-763ab5853bff619e6525c01e46b2a923 | 1 + ...adoop20-7-db1cd54a4cb36de2087605f32e41824f | 1 + ...adoop20-8-bab89dfffa77258e34a595e0e79986e3 | 1 + ...adoop20-9-28bf1b34c04a048da339eddd4c1fd779 | 0 ...3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 | 0 ...3_limit-1-7b46b8baf9c7628da9c190c96b917057 | 0 ...3_limit-2-c6583bdb759c8a050238a32a6ce8273d | 0 ...3_limit-3-2a87d8faa18a6311376812bd0453fece | 0 ...3_limit-4-70dad45d534146923fce88b2ffb99b0d | 0 ...3_limit-5-3664b564747487df13a5d109837219b5 | 0 ...3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 | 20 + .../input4-0-b9cd4041ca510639838a97376747b498 | 0 .../input4-1-c139adc70f9942e527142e3be7fd2b87 | 27 + .../input4-2-b7f46eccd104e6ed1b29e2de45089f66 | 0 .../input4-3-d9280be6c9dd1e2263f8d50f0f43a486 | 0 .../input4-4-271b04e0fbee2ee81bae21dcb46d55e4 | 500 ++++ .../input40-0-37e74908fd43254141ae3664f13a48d | 0 ...input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 | 0 ...input40-2-e7ab3c9244fcfda76061b4153d796e93 | 500 ++++ ...input40-3-12f634800ffae17942ddd789a01af227 | 0 ...input40-4-f241eafbb8d5da3f9c1737aed7b4f94e | 0 ...input40-5-ccdc9c87a8fdbeee0fa48927f9700361 | 500 ++++ ...input40-6-93a0c37189dfe2318ba6ad84616f0d64 | 0 ...input40-7-ccdc9c87a8fdbeee0fa48927f9700361 | 1000 +++++++ ...input41-0-763ab5853bff619e6525c01e46b2a923 | 1 + ...input41-1-8112b3a278e8337dbd6f017ae178658b | 0 ...input41-2-61812b3093413d09f0fd2372c61f7d53 | 0 ...input41-3-526399455dc5ecd4ea9f676b09fafeee | 2 + ...input43-0-2baba8070f3585debc14b6bb3c83607a | 0 ...input46-0-b0cdbecce0321ac452c8e13e1bfc6924 | 0 ...input46-1-1efdd5ebfa732abdedeb10467ca71f7f | 0 ...input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca | 0 ...input46-3-c185163787977498a4b84f39f983c431 | 0 ...input46-4-7f05cde078d90c25780a7d5491b20c27 | 0 ...input46-5-f5c502e88a3dc3edb37b04af7d7955ab | 0 ...input49-0-2bd546beeb607da5b925ffbea20a741c | 0 ...input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d | 0 .../input49-2-d56860e68d333bb6132ef5ed0327eb1 | 1 + ...input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 | 0 ...b_delim-0-f8e975c0fc126982e8e70cc30d2b6367 | 0 ...b_delim-1-5692d0e91dd0114729b8eb3aee388b72 | 0 ...b_delim-2-e52787bf798a941c854eb09d75efe617 | 500 ++++ ...4_limit-0-4f0124854141b8be1defa7a6d0877d8d | 0 ...t4_limit-1-c634fc723fb3aac3ce007069bdcb2af | 10 + .../input5-0-659e06570690cceeb3f37e10e855d2ea | 0 .../input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 | 0 .../input5-2-3d6eb15b4fe23d0a1aa303da818d97ad | 0 .../input5-3-adc1ec67836b26b60d8547c4996bfd8f | 11 + .../input6-0-659e06570690cceeb3f37e10e855d2ea | 0 .../input6-1-21149162906e31964a328b6cb2d5286e | 0 .../input6-2-9601ace6dce45dc8d6281cc248c9e28c | 0 .../input6-3-adc1ec67836b26b60d8547c4996bfd8f | 10 + .../input7-0-8daa7823607b82336736a9d4f9a2ce45 | 0 .../input7-1-bb25de1b9a7791ec5609303bab350da0 | 0 .../input7-2-c740098a289f9a4267d26fc10347a0dc | 0 .../input7-3-adc1ec67836b26b60d8547c4996bfd8f | 25 + .../input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa | 0 .../input8-1-c9f9239e73b04fc5c0e0219f438ceefa | 0 .../input8-2-4a4cb89ba45d26bb3dd73e44620578ba | 0 .../input8-3-adc1ec67836b26b60d8547c4996bfd8f | 25 + .../input9-0-8cc1539c21d92af8ea8c1431b0e3d85 | 0 .../input9-1-b30105391cb53915997a84ca3137bd0a | 0 .../input9-2-171ea277f05a642699a0f91bacffccd8 | 0 .../input9-3-adc1ec67836b26b60d8547c4996bfd8f | 0 ...arserde-0-df919fd41f281bf7b45a2340d0c9d43e | 0 ...narserde-1-2db9e6115d209fabcb0c06e5e666fa3 | 0 ...arserde-2-ac60752a883d3204c215fa01811701be | 0 ...arserde-3-3455e6f385b0f60be5d0c842ade0f1d7 | 11 + ...arserde-4-c471b057cdb4f3c3405b311ea2b92aa3 | 11 + ...icserde-0-92c95af00fd419aa106571f72fcad67d | 0 ...icserde-1-39bc8a2aea379563a62a465cc54aecbc | 0 ...icserde-2-7bee1cc403a04b53d0a0324819e1d768 | 0 ...icserde-3-adc1ec67836b26b60d8547c4996bfd8f | 11 + ...icserde-4-8bf42e02f82b0ac58e7d0b525a993b31 | 11 + ...zyserde-0-92c95af00fd419aa106571f72fcad67d | 0 ...zyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 | 0 ...yserde-10-d915efbc5233a49f48e81e74eef2a3c8 | 0 ...yserde-11-6aaa88142e86a9a39e980caed145e32c | 0 ...yserde-12-abde20a4a37ed330bc7128576dd18d7c | 11 + ...zyserde-2-7a165139976654640c86db8e4e5871cc | 0 ...zyserde-3-bdf93641b97ab6347ece67e2fb636e97 | 11 + ...zyserde-4-8cc058fb7986f59965976cad813267db | 11 + ...zyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 ...azyserde-6-42e2838ee69484bf5301475905cee12 | 0 ...zyserde-7-b44a44827ad0dce044badd6b258eabd5 | 0 ...zyserde-8-abde20a4a37ed330bc7128576dd18d7c | 11 + ...zyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 ...t_limit-0-b2b4b45519484c8bac49d07debf678b9 | 0 ...t_limit-1-77d7012bc901b0a9bcf9dae500ae2034 | 20 + ...t_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 | 0 ...t_part0-1-1aa1486a207bedc275035acc3b37cbdb | 1000 +++++++ ...ut_part1-0-55f89c73a00f0f23ee04239ad9fc909 | 0 ...t_part1-1-593e11f2ca748f3ae6b8fdf5da435229 | 0 ...t_part1-2-4153379704a43162abf9dbdbd79a92bc | 0 ...t_part1-3-adc1ec67836b26b60d8547c4996bfd8f | 84 + ..._part10-0-77d7f7c938d7827a2135d9391376a518 | 0 ..._part10-1-35f67c0112026170d015b5f80a254bde | 0 ..._part10-2-40069c199502c2724ac2a2733f964248 | 0 ..._part10-3-48b242bc305c9bf879e083fa11edc967 | 12 + ..._part10-4-d0ba28297a8b73569d93605aa890aa09 | 1 + ...t10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf | 0 ...t10_win-1-35f67c0112026170d015b5f80a254bde | 0 ...t10_win-2-40069c199502c2724ac2a2733f964248 | 0 ...t10_win-3-48b242bc305c9bf879e083fa11edc967 | 12 + ...t10_win-4-d0ba28297a8b73569d93605aa890aa09 | 1 + ...ut_part2-0-55f89c73a00f0f23ee04239ad9fc909 | 0 ...t_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e | 0 ...t_part2-2-a8684c5574f90c3db2fc8a269a7556cd | 0 ...t_part2-3-fcb06513e757d4bf929ff681cb9f02b1 | 0 ...t_part2-4-93c97e1760e0d41b3791d6f08010a665 | 84 + ...t_part2-5-9d0992a91951e6e4242b5b4c38d4e861 | 84 + ...t_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff | 0 ...t_part3-1-ba5256285fb22a43b491253a2d519730 | 500 ++++ ...t_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef | 0 ...t_part4-1-88be7d93e9a03ea76085111c18d437a5 | 0 ...t_part5-0-679bf51b7de8df94fd2928744e887e3b | 0 ...t_part5-1-705301f0bf4fe9758e9f919d129dbc7c | 0 ...t_part5-2-d678533566fba4b922f01284538ca484 | 0 ...t_part5-3-e4419c33287ca1f48a43f61cca5b5928 | 168 ++ ...t_part6-0-6ec3d5a634d8c011f32a0277e654ae6d | 0 ...t_part6-1-9ffee7a86f484eab399266efa6e086f6 | 0 ...t_part7-0-14448a12627b8bccf6828032f938f39e | 0 ...t_part7-1-affad4cedcd29bb136bc477fc07e6ea0 | 336 +++ ...t_part8-0-f60652796af548135f9e104b456840e3 | 0 ...t_part8-1-60b409a520999ba50e8b7c6e30de3474 | 10 + ...t_part9-0-726907a7cf62f5a79466caa0fa6eca01 | 0 ...t_part9-1-e60c60afc073367464898b8396e8f643 | 1000 +++++++ ...ncefile-0-68975193b30cb34102b380e647d8d5f4 | 1 + ...ncefile-1-1c0f3be2d837dee49312e0a80440447e | 1 + ...ncefile-2-186e4009966778e765b18d6476cf5abf | 0 ...ncefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 | 0 ...ncefile-4-8ebe479eca517e80bf0348a1b3a89f42 | 0 ...ncefile-5-3708198aac609695b22e19e89306034c | 1 + ...ncefile-6-6716fa5aec125f0f8e9520e4318f71b9 | 500 ++++ ...stxpath-0-3c8a098a179d578119f75e5d7b214bd5 | 0 ...stxpath-1-ac18d29e8bd8aeed00296992edc17c0f | 0 ...estxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 | 0 ...stxpath-3-adc1ec67836b26b60d8547c4996bfd8f | 11 + ...txpath2-0-a1baa22f15f745a2dfe27ce52d363704 | 0 ...txpath2-1-27b77465d23aebf66a50c8074a75b755 | 0 ...txpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd | 0 ...txpath2-3-adc1ec67836b26b60d8547c4996bfd8f | 10 + ...txpath3-0-f05134d54292acd1f2067027889a4fac | 0 ...txpath3-1-807b097ac2f785f774db03069ebbde11 | 11 + ...txpath4-0-73819ea1a7c0653a61652b3766afb003 | 1 + ...txpath4-1-f746888141a38ba707fad01d86d41960 | 0 ...txpath4-2-e4beab1294dcac60ff36e4f2561810b9 | 10 + ...txpath4-3-ae225e86c2ae20519ffdf23190454161 | 1 + ...txpath4-4-f746888141a38ba707fad01d86d41960 | 0 ...txpath4-5-e4beab1294dcac60ff36e4f2561810b9 | 10 + ...putddl4-0-2f5570984afde1a30ff2f794d63d6ab3 | 0 ...putddl4-1-dd94576788fa50ebcf950cdf837fbcf6 | 15 + ...putddl4-2-7fdf00ff5c22ae284728e0f035396865 | 17 + ...putddl6-0-baa412d895c4fc57d40b338944a0ecd8 | 0 ...putddl6-1-8b96b4fba4cf54c399a008d0f3be1edf | 0 ...putddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c | 0 ...putddl6-3-e81b962986706e1c16f059b407e3f05c | 10 + ...putddl6-4-5855e2998e26f63e927854afa86c1f03 | 10 + ...putddl6-5-47e6a8d33932cb014830812e1f7b1f94 | 2 + ...putddl6-6-6eac06a114fe33c1278d47ad8652fe68 | 0 ...putddl6-7-47e6a8d33932cb014830812e1f7b1f94 | 1 + ...putddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 | 19 + ...putddl6-9-2164df0b7eab73cd8c4a553a762028e1 | 0 ...putddl7-0-6a30e2d3fc23ec0c18513265bd8c987f | 0 ...putddl7-1-7b46b8baf9c7628da9c190c96b917057 | 0 ...utddl7-10-4eec8cef401b6bba00457dfbec58cc2d | 0 ...utddl7-11-6e1f1709d770ed76aee6ff5f76856e63 | 1 + ...putddl7-12-2c56d4a781242b0521f82bb0d2cd277 | 3 + ...utddl7-13-45059a21c202b4658285738ee62a018a | 3 + ...utddl7-14-30c87bc734c2afa4fea0facdf7279145 | 9 + ...putddl7-15-fed9badc255db68c3ed1cd1446d9fbe | 9 + ...putddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 | 1 + ...putddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 | 0 ...putddl7-4-7513658e8abe9d9b72c7219321c56fa9 | 0 ...putddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 | 1 + ...putddl7-6-a590512af0ed021d33e2d71b7932a20a | 0 ...putddl7-7-48640fff8428a0dc6e90a7243adaf730 | 0 ...putddl7-8-495dc87b0bde752c890f213ff9531508 | 1 + ...putddl7-9-c943e781fb448aea5467251ee208fbac | 0 ...putddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 | 0 ...putddl8-1-c70f2d2544633366b76b92bcff18e995 | 16 + ...db name-0-a253b1ed35dbf503d1b8902dacbe23ac | 0 ...db name-1-4f5be7056d8dc15cf2334a7cc146a636 | 0 ...db name-2-a81aea129f78bc05305b8e887ee88f86 | 0 ...db name-3-ce780d068b8d24786e639e361101a0c7 | 500 ++++ ...db name-4-afd6e46b6a289c3c24a8eec75a94043c | 0 ...pressed-0-ea607fbed28d20e5726f4501285d698d | 1 + ...pressed-1-d20899578068ff4adfee5355cddca819 | 0 ...pressed-2-8dec751fd4148a431af064b7317f1530 | 0 ...pressed-3-35f4fbde823a5664fe9928a685745b35 | 0 ...pressed-4-5133d2457097962811a2adf0ecd9e4ef | 1 + ...pressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 | 0 ...pressed-6-5133d2457097962811a2adf0ecd9e4ef | 1 + ...pressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 | 0 ...pressed-8-5133d2457097962811a2adf0ecd9e4ef | 1 + ...pressed-9-d20899578068ff4adfee5355cddca819 | 0 ...t_into1-0-ae5ea07929262bde22fbe7ebe80d4992 | 0 ...t_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b | 0 ..._into1-10-c260979323c1ebdf68c6fbe003d43792 | 0 ...t_into1-11-41015d6409c5ebf670eed4999157fdb | 1 + ..._into1-12-ae5ea07929262bde22fbe7ebe80d4992 | 0 ...t_into1-2-ff6a1b25c911def274921df1bae476b7 | 0 ...t_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 | 0 ...rt_into1-4-41015d6409c5ebf670eed4999157fdb | 1 + ...t_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 | 0 ...t_into1-6-4b2e32fe57b2297d22bccb4656bdee30 | 0 ...rt_into1-7-41015d6409c5ebf670eed4999157fdb | 1 + ...t_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d | 1 + ...t_into1-9-31eb4770dc60e4765065ac7f84811d1b | 0 ...t_into2-0-957c6402df0fd7d2fccbc688e49e9661 | 0 ...t_into2-1-9828bb831fd11667b48678e5952a0941 | 0 ..._into2-10-df53336f364fe09e9591e769c13b5519 | 0 ..._into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa | 0 ..._into2-12-4cf03cb3982a457f2f72220265ecc844 | 1 + ..._into2-13-957c6402df0fd7d2fccbc688e49e9661 | 0 ...t_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 | 0 ...t_into2-3-9d5556040de01fd02d5501d141effff2 | 0 ...t_into2-4-9d5556040de01fd02d5501d141effff2 | 0 ...t_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 | 1 + ...t_into2-6-4cf03cb3982a457f2f72220265ecc844 | 1 + ...t_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 | 0 ...t_into2-8-452111285dda40205ee587de8e972896 | 0 ...t_into2-9-4cf03cb3982a457f2f72220265ecc844 | 1 + ...t_into3-0-7e35c30348a53b0748bc1bb85f75c195 | 0 ...t_into3-1-f6dd7262d45a21781d85bc343ede8fb5 | 0 ..._into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 | 1 + ..._into3-11-9cfd2d054f84262eb74a870b6365db87 | 1 + ..._into3-12-7e35c30348a53b0748bc1bb85f75c195 | 0 ..._into3-13-f6dd7262d45a21781d85bc343ede8fb5 | 0 ...t_into3-2-e51c25bae2408422a56826a263479468 | 0 ...t_into3-3-c46699c465fefe6baab35499a32b452d | 0 ...rt_into3-4-e9f4f47686fe97482b0a769a15481dd | 0 ...t_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 | 0 ...t_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 | 1 + ...t_into3-7-9cfd2d054f84262eb74a870b6365db87 | 1 + ...t_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 | 0 ...t_into3-9-19d1be183f7985f7066f86572abc82c8 | 0 ...t_into4-0-b4831621b2a02fc4e8e655b03c289310 | 1 + ...t_into4-1-3d466d45197fcf1eff55d76ef0a29720 | 0 ..._into4-10-a8016ca816f7b362cf3a0384ca91b159 | 1 + ..._into4-11-bdbfaf06bc4e323863db65fd29882eaa | 0 ..._into4-12-49b12993ebc1647b9dca8b9af19deca9 | 0 ..._into4-13-a5bb41af609f38f01d8c71334cc8d71b | 1 + ..._into4-14-3d466d45197fcf1eff55d76ef0a29720 | 0 ..._into4-15-f6ad1a49459fb6cd232fccc4a6062b25 | 0 ...t_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 | 0 ...t_into4-3-43629aaa698fb6e2db4586124561e69b | 0 ...t_into4-4-8adba808fd505f4bf0ffcc61a618480a | 0 ...t_into4-5-6bc47894aa917051abb98d0b52f43881 | 0 ...t_into4-6-8c1683bee2927da76bb0dbf44a373738 | 0 ...t_into4-7-a8016ca816f7b362cf3a0384ca91b159 | 1 + ...t_into4-8-6bc47894aa917051abb98d0b52f43881 | 0 ...t_into4-9-8c1683bee2927da76bb0dbf44a373738 | 0 ...t_into5-0-9afa473f2111cf0d9ae62041bd97f840 | 0 ...t_into5-1-8fc8db6a5564324339192f23208ffc1c | 0 ...t_into5-10-375cae396c768c1efe5d17b9f5f45f8 | 0 ..._into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d | 0 ..._into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 | 1 + ..._into5-13-e06a0b7252278141d50466e08f15b391 | 0 ..._into5-14-a3a4c16b0b723f97f654b4b5e80575c2 | 0 ..._into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 | 1 + ..._into5-16-9afa473f2111cf0d9ae62041bd97f840 | 0 ...t_into5-2-548a3a10c15c333c63fc1c239ee8b62c | 0 ...t_into5-3-a4b25f172af356ec98035329b95ddbd3 | 0 ...t_into5-4-3d5343a79ee8b680f3b74b22db6658e6 | 0 ...t_into5-5-f382c5b31afe9e0251fa9457c46c12a5 | 0 ...t_into5-6-a058ba199b9777d48b6c6595f2388533 | 1 + ...t_into5-7-2004b4ecf6ceddb033727f8d5ebab42f | 0 ...t_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 | 0 ...t_into5-9-a058ba199b9777d48b6c6595f2388533 | 1 + ...t_into6-0-16367c381d4b189b3640c92511244bfe | 1 + ...t_into6-1-a4fb8359a2179ec70777aad6366071b7 | 1 + ..._into6-10-16500f4af2c8638a670e867e59f6d457 | 0 ..._into6-11-95297f4dcc4dd6a51de4785ccd25fbba | 0 ..._into6-12-29afaab2cf10eaa65afaa6c0fcca0902 | 1 + ..._into6-13-b357a845fb5f668b09dea94c6d2e7d66 | 2 + ..._into6-14-1f00d9cc34225cb358a54c6d4008cb47 | 0 ..._into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d | 0 ...t_into6-2-1f00d9cc34225cb358a54c6d4008cb47 | 0 ...t_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d | 0 ...t_into6-4-5f12794e99c74355a23d2fda9c7c170f | 0 ...t_into6-5-de641eb41a9100e755a9ae641c752b30 | 0 ...t_into6-6-f6e7141a435922193937aa10085b0656 | 0 ...t_into6-7-6826c610b8d04ab3464712bd9cddbcd3 | 0 ...t_into6-8-33ec9514947e0b737e957bdcbbd87573 | 0 ...t_into6-9-3ccb7bc735e406ad1723c758f01bcaab | 1 + ...ternal1-0-eb0745518b859c8497506a627bfd9860 | 0 ...ternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a | 1 + ...ternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b | 1 + ...ternal1-3-f64289fb03ab105e12659fc3972ca241 | 1 + .../join0-0-7fbd114e2fc7f55cf1421698eac39397 | 0 .../join0-1-83f5e512d6058a47f92b5218781f5070 | 0 .../join0-2-52055f2dce8eac4e5249d02c42c0da87 | 100 + .../join1-0-43d53504df013e6b35f81811138a167a | 1 + .../join1-1-f1293ebf768eb04f2f0bfe6297c4509e | 0 .../join1-2-7b07671864bbfdc627ee794932e32b1e | 0 .../join1-3-f6046c5229e3b0aa21498a3872f43b2 | 0 .../join1-4-35c7611180562dcd9dab834f41654095 | 1028 +++++++ .../join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 | 0 .../join10-1-73136f8e6e9ba82f75570afd15c2828d | 1028 +++++++ .../join11-0-e2cc67c543c1209ebcd3f9048c4cb29 | 0 .../join11-1-b580f87daf1763cd8c5e59ad5b271232 | 148 + .../join12-0-12fa53be2c0ddbcfc85212e573a46af1 | 0 .../join12-1-496c8aabaf3261e8fefd7b357f2ac7f | 232 ++ .../join13-0-c4ebf24269e27919f4ba9ce3993e48f7 | 0 .../join13-1-696b36d15c4358145f77c2b15b7507d5 | 164 + .../join14-0-c85f3dcbab496811604ea0ab84d0e995 | 0 .../join14-1-9b141c1e5917ca82c6bc36a9a2950a1e | 1 + .../join14-2-2b9ccaa793eae0e73bf76335d3d6880 | 1 + .../join14-3-bab89dfffa77258e34a595e0e79986e3 | 1 + .../join14-4-27f1a57fbb815d169af86ae2f8305cb6 | 1 + .../join14-5-4c3f537b4df1ef16788a53cf65574187 | 0 .../join14-6-de39302191b63d7aa8f92885b089fe2 | 0 .../join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 | 1752 +++++++++++ ...adoop20-0-98b7542190092fafcc8b1ad5b0024a22 | 0 ...adoop20-1-db1cd54a4cb36de2087605f32e41824f | 1 + ...adoop20-2-bab89dfffa77258e34a595e0e79986e3 | 1 + ...adoop20-3-4c3f537b4df1ef16788a53cf65574187 | 0 ...hadoop20-4-de39302191b63d7aa8f92885b089fe2 | 0 ...adoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 | 1752 +++++++++++ .../join15-0-930db2823512f7f8f34cac104f2162e0 | 0 .../join15-1-81d76d3bf59889b07b413b6f88772667 | 1028 +++++++ .../join16-0-1bec3b0892d5c4a174d1b39f6d1b610c | 0 .../join17-0-387dd86b1e13f788ec677a08dc162c97 | 0 .../join17-1-37cef87fe56f66692799ccda8cda2e8b | 0 .../join17-2-478a9f270a5d70f6f82f81e6962fb251 | 0 .../join17-3-adc1ec67836b26b60d8547c4996bfd8f | 1028 +++++++ .../join18-0-269843197b598569f4b50cbe911960a5 | 0 .../join18-1-49f8ba8d43a6bb1d892ba66a812304f5 | 310 ++ .../join19-0-26ba1cd9acfcd799239da60c38e68a39 | 0 .../join19-1-7e7d1f07c34dd4be5425264196201766 | 0 .../join2-0-3d1692c4710db1ff716d35e921f2bcca | 0 .../join2-1-87896c3f04a259b81b1cfde3eec7d64c | 0 .../join2-2-d4673c03d04084b838fcd8149f59ad9a | 0 .../join2-3-cac2c9e0f8601dd56822c990774e0696 | 472 +++ .../join20-0-16cda49c2b2cd0c0cdae40af4d5c900e | 0 .../join20-1-9685f2d31ffe922d3ea217de32ca3049 | 548 ++++ .../join20-2-222655137ca9242f063c7717acbe4c65 | 0 .../join20-3-3331a020843caccf2fa32a1225c5c3a9 | 548 ++++ .../join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 | 0 .../join21-1-3536b7d78713e86ee67f5f6c9b88958f | 500 ++++ .../join22-0-97081b5f73052a612fb391a57ad9b7b | 0 .../join23-0-1fb76d3404f09a5c2b10422157c3cf5d | 0 .../join23-1-91b8e7fe75a7e3ba8147c56734436681 | 100 + .../join24-0-721dfa03bfea05e55506c571b6c3585b | 0 .../join24-1-36de83b0ed6c9fdc03661b2f65b23a3d | 0 .../join24-2-d79325ef6494aa87843fdfd78de7c812 | 1 + .../join25-0-8934d9b821aa4b34b760f73eff56cd06 | 1 + .../join25-1-360b3676605ccb3d89cf555342db87af | 0 .../join25-2-ef930bed933341636c71622e6f89e9e9 | 0 .../join25-3-e39577008d28ddd5741d8518b92eaa94 | 0 .../join25-4-c9859bf9c9942c59f3b19d82bd1a3afa | 37 + .../join26-0-775b1af8eae9b2250052be1b72b4086 | 0 .../join26-1-10fbe6c99e93cfeafcd454261744728e | 0 .../join26-2-8aa0193a2a80d4efe4c66bca19218a4d | 0 .../join26-3-c9859bf9c9942c59f3b19d82bd1a3afa | 107 + .../join27-0-360b3676605ccb3d89cf555342db87af | 0 .../join27-1-d56d8bbececcf48417b7dde9292a9cc6 | 0 .../join27-2-6d139e1aa07ff8d02bdf52b409956879 | 0 .../join27-3-e86808fdbd54120d1e2356f8f61c02f9 | 41 + .../join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 | 0 .../join28-1-24ca942f094b14b92086305cc125e833 | 1 + .../join28-2-44d382ce6848d3f0b900b0808747d8e9 | 1 + .../join28-3-25fc734982956a164adde6bb1d4d8751 | 1 + .../join28-4-bd4e19a3608681b0270668fe68b7d124 | 0 .../join28-5-24cc9ff9485313ade08ee83ecc4c0621 | 0 .../join28-6-c9859bf9c9942c59f3b19d82bd1a3afa | 107 + .../join29-0-e78402f7585b17c76c32790571d749eb | 0 .../join29-1-24ca942f094b14b92086305cc125e833 | 1 + .../join29-2-44d382ce6848d3f0b900b0808747d8e9 | 1 + .../join29-3-25fc734982956a164adde6bb1d4d8751 | 1 + .../join29-4-b0524de7985cdb7b57fd995721654856 | 0 .../join29-5-9e35870a3bc8224cde6b1ab322568d2 | 0 .../join29-6-c9859bf9c9942c59f3b19d82bd1a3afa | 15 + .../join3-0-13ab74a58da514fe01dbeda0c3e79883 | 0 .../join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 | 0 .../join3-2-e9f6d17b15064f953a588fb40aee2f90 | 0 .../join3-3-adc1ec67836b26b60d8547c4996bfd8f | 2654 +++++++++++++++++ .../join30-0-54a7280ab9eed0d2e3b33df35a721b66 | 0 .../join30-1-5859eba43fb180c2142035cd4c77ea4d | 0 .../join30-2-194edec0592cf441617ca7caf9756baa | 0 .../join30-3-c9859bf9c9942c59f3b19d82bd1a3afa | 15 + .../join31-0-619db2f9dd69aa236aa804cced726c9a | 0 .../join31-1-24ca942f094b14b92086305cc125e833 | 1 + .../join31-2-44d382ce6848d3f0b900b0808747d8e9 | 1 + .../join31-3-25fc734982956a164adde6bb1d4d8751 | 1 + .../join31-4-34e30fa5d6f35a20561c7754197f0626 | 0 .../join31-5-a2082be28be44e2bfa9a58fb45c23f07 | 0 .../join31-6-c9859bf9c9942c59f3b19d82bd1a3afa | 15 + .../join32-0-775b1af8eae9b2250052be1b72b4086 | 0 .../join32-1-24ca942f094b14b92086305cc125e833 | 1 + .../join32-2-44d382ce6848d3f0b900b0808747d8e9 | 1 + .../join32-3-25fc734982956a164adde6bb1d4d8751 | 1 + .../join32-4-fc6880e4e0750a3daa7fc108b72d11bb | 0 .../join32-5-425c5c35b23c42254dabec49aa6613bf | 0 .../join32-6-c9859bf9c9942c59f3b19d82bd1a3afa | 85 + ...lessSize-0-775b1af8eae9b2250052be1b72b4086 | 0 ...essSize-1-7a874b25490d3e1e186803646eb6a789 | 0 ...ssSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a | 85 + ...essSize-11-79c0e9faf5798c465fedee25c72e7ca | 0 ...essSize-12-83544f47197cccad30dd4f0ede433b8 | 0 ...ssSize-13-ed70124968560328930327ecb108c4e9 | 85 + ...ssSize-14-ac014eb214f762f1a61125af1964788a | 0 ...ssSize-15-78edd67daa31711374b4c25faf89f77e | 0 ...ssSize-16-ed70124968560328930327ecb108c4e9 | 97 + ...ssSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 | 0 ...ssSize-18-f9b31c552044d8e5cc193121eb71024d | 0 ...ssSize-19-ed70124968560328930327ecb108c4e9 | 85 + ...essSize-2-24ca942f094b14b92086305cc125e833 | 1 + ...essSize-20-ecf04f68517b6104d9cc8995796b424 | 0 ...ssSize-21-f6822e1bb0b193eec397ebb9c79fbf47 | 0 ...ssSize-22-ed70124968560328930327ecb108c4e9 | 85 + ...essSize-3-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...essSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 | 1 + ...essSize-5-fc6880e4e0750a3daa7fc108b72d11bb | 0 ...essSize-6-425c5c35b23c42254dabec49aa6613bf | 0 ...essSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a | 85 + ...essSize-8-5e277b322f831494ecd6f51a0c727c7b | 0 ...lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 | 0 .../join33-0-775b1af8eae9b2250052be1b72b4086 | 0 .../join33-1-24ca942f094b14b92086305cc125e833 | 1 + .../join33-2-44d382ce6848d3f0b900b0808747d8e9 | 1 + .../join33-3-25fc734982956a164adde6bb1d4d8751 | 1 + .../join33-4-fc6880e4e0750a3daa7fc108b72d11bb | 0 .../join33-5-425c5c35b23c42254dabec49aa6613bf | 0 .../join33-6-c9859bf9c9942c59f3b19d82bd1a3afa | 85 + .../join34-0-775b1af8eae9b2250052be1b72b4086 | 0 .../join34-1-24ca942f094b14b92086305cc125e833 | 1 + .../join34-2-44d382ce6848d3f0b900b0808747d8e9 | 1 + .../join34-3-25fc734982956a164adde6bb1d4d8751 | 1 + .../join34-4-51cc5d253befd784a9d466c0b402f47a | 0 .../join34-5-abf5cf110d6a0eb85ae257768c805f39 | 0 .../join34-6-c9859bf9c9942c59f3b19d82bd1a3afa | 34 + .../join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 | 0 .../join35-1-24ca942f094b14b92086305cc125e833 | 1 + .../join35-2-44d382ce6848d3f0b900b0808747d8e9 | 1 + .../join35-3-25fc734982956a164adde6bb1d4d8751 | 1 + .../join35-4-99cb704ccc2813400908a62571891dc7 | 0 .../join35-5-60956f3307651237f52ffbe41c827c1c | 0 .../join35-6-c9859bf9c9942c59f3b19d82bd1a3afa | 13 + .../join36-0-8934d9b821aa4b34b760f73eff56cd06 | 1 + .../join36-1-3aba153d62042206c0f001ea74980da | 0 .../join36-2-5b1cac8ef7857d2f8c675ed6f0224471 | 0 .../join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 | 0 .../join36-4-fbe7d714e988877ebe6202a60e8ef776 | 0 .../join36-5-9353d3e0b11c781f782af4013e8569e3 | 0 .../join36-6-d76aff886201d223c7dbed2b0f0d5512 | 0 .../join36-7-171ede21302050a33468d95e329a38a3 | 0 .../join36-8-c9859bf9c9942c59f3b19d82bd1a3afa | 309 ++ .../join37-0-8934d9b821aa4b34b760f73eff56cd06 | 1 + .../join37-1-360b3676605ccb3d89cf555342db87af | 0 .../join37-2-2310f2b8726137811543e04db98dd1db | 0 .../join37-3-c4b3a85965108ad7013ac3931598af09 | 0 .../join37-4-c9859bf9c9942c59f3b19d82bd1a3afa | 37 + .../join38-0-42e6ad674a5142d05258a2d29578e046 | 0 .../join38-1-d8ba5ec8a5a0bb702958400a3b3e458f | 0 .../join38-2-eacdb3417c4a563982c488812d654c9 | 2 + .../join38-3-22ead86c36dcd9ac9403fb52ac752046 | 0 .../join38-4-53d219706847e890de1dcd369563ebef | 1 + .../join39-0-54e9f7924abb40d1c601de908de6f25b | 0 .../join39-1-60178dec03adcbccbd4e5cae936c38f2 | 0 .../join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 | 0 .../join39-3-c9859bf9c9942c59f3b19d82bd1a3afa | 566 ++++ .../join4-0-531e91e11b3891627c2675935fda14cd | 0 .../join4-1-8884a9b112b0c767aa0bd8e745bd7b21 | 0 .../join4-2-dc967001beb776f3a859e9360823c361 | 0 .../join4-3-adc1ec67836b26b60d8547c4996bfd8f | 11 + .../join40-0-d2b5e23edec42a62e61750b110ecbaac | 1 + .../join40-1-4b663d50fee0a8afd32cdf25c3b82de8 | 0 ...join40-10-1d1f79e8e773d630f355c1a89d84b5aa | 566 ++++ ...join40-11-c8c5b97a744acf897888ab2d51d33a80 | 0 ...join40-12-aaddbef9069aba3ebeb310be74671cda | 1 + .../join40-2-507b1d9f6abbdb756a589d7bc4826251 | 566 ++++ .../join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 | 0 .../join40-4-61fb097bda1751910de966d6a4a8f0b7 | 1028 +++++++ .../join40-5-16cda49c2b2cd0c0cdae40af4d5c900e | 0 .../join40-6-9685f2d31ffe922d3ea217de32ca3049 | 548 ++++ .../join40-7-222655137ca9242f063c7717acbe4c65 | 0 .../join40-8-3331a020843caccf2fa32a1225c5c3a9 | 548 ++++ .../join40-9-6b7404af014eeb4de196577f14ef18cc | 0 .../join41-0-b74d94933c6670ccc4a1ad8161185686 | 0 .../join41-1-25e434b6d05e08fdd5f4d9957438917 | 1 + .../join41-2-e02c931c8bb07ad765d0a6cd73abc2a | 0 .../join41-3-1dc98363e7da167dc45711a87ef3a988 | 3 + .../join41-4-44d382ce6848d3f0b900b0808747d8e9 | 1 + .../join41-5-f18ea17bcfafde311a4bdc589362e00e | 0 .../join41-6-1dc98363e7da167dc45711a87ef3a988 | 3 + .../join5-0-531e91e11b3891627c2675935fda14cd | 0 .../join5-1-76feff1282895f38b673f52acfd7429e | 0 .../join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 | 0 .../join5-3-adc1ec67836b26b60d8547c4996bfd8f | 9 + .../join6-0-531e91e11b3891627c2675935fda14cd | 0 .../join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 | 0 .../join6-2-bc6b6640b266ebe9b73702d3baf09c20 | 0 .../join6-3-adc1ec67836b26b60d8547c4996bfd8f | 14 + .../join7-0-8f82881057bec4abf5a4d770a6f35838 | 0 .../join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 | 0 .../join7-2-56a30a1aa948bcf5ee54481897fc2208 | 0 .../join7-3-adc1ec67836b26b60d8547c4996bfd8f | 14 + .../join8-0-531e91e11b3891627c2675935fda14cd | 0 .../join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 | 0 .../join8-2-f1196bca86a749375da35f134206a8ca | 0 .../join8-3-adc1ec67836b26b60d8547c4996bfd8f | 5 + .../join9-0-13ab74a58da514fe01dbeda0c3e79883 | 0 .../join9-1-fb3387ec28fc9d527e773c8b70bdf67a | 0 .../join9-2-971c44e81ce17eb0849850b72ebd20f1 | 0 .../join9-3-adc1ec67836b26b60d8547c4996bfd8f | 1028 +++++++ ...in_1to1-0-7ea7cf54372f262b952108113a97a294 | 0 ...in_1to1-1-5bab379018a4fbef12cc93658f26580a | 0 ...n_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e | 1 + ...n_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e | 30 + ...n_1to1-12-6081eb976b4aef2132418510756a385b | 36 + ...n_1to1-13-281b888188eac90c4bf670417f25cc0c | 34 + ...n_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 | 36 + ...n_1to1-15-5e48ba086f1376939535081b60f82727 | 34 + ...n_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 | 1 + ...n_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e | 30 + ...n_1to1-18-6081eb976b4aef2132418510756a385b | 36 + ...n_1to1-19-281b888188eac90c4bf670417f25cc0c | 34 + ...in_1to1-2-1c35d445eca04025196ea700d02f9987 | 0 ...n_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 | 36 + ...n_1to1-21-5e48ba086f1376939535081b60f82727 | 34 + ...n_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 | 1 + ...n_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e | 30 + ...n_1to1-24-6081eb976b4aef2132418510756a385b | 36 + ...n_1to1-25-281b888188eac90c4bf670417f25cc0c | 34 + ...n_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 | 36 + ...n_1to1-27-5e48ba086f1376939535081b60f82727 | 34 + ...n_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e | 1 + ...n_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e | 30 + ...in_1to1-3-ee6db4188755bf471a12316ec7301500 | 0 ...n_1to1-30-6081eb976b4aef2132418510756a385b | 36 + ...n_1to1-31-281b888188eac90c4bf670417f25cc0c | 34 + ...n_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 | 36 + ...n_1to1-33-5e48ba086f1376939535081b60f82727 | 34 + ...n_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 | 1 + ...n_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e | 30 + ...n_1to1-36-6081eb976b4aef2132418510756a385b | 36 + ...n_1to1-37-281b888188eac90c4bf670417f25cc0c | 34 + ...n_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 | 36 + ...n_1to1-39-5e48ba086f1376939535081b60f82727 | 34 + ...in_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 | 1 + ...in_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e | 30 + ...in_1to1-6-6081eb976b4aef2132418510756a385b | 36 + ...in_1to1-7-281b888188eac90c4bf670417f25cc0c | 34 + ...in_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 | 36 + ...in_1to1-9-5e48ba086f1376939535081b60f82727 | 34 + ...n_array-0-60904bf2bd7bbfa8739d8e047e83e038 | 0 ...n_array-1-2c086fcf118fd2538bfa00724209091e | 0 ...n_array-2-a4363f7c7e4b7d717ed90e77c37581de | 0 ...n_array-3-ddd65703cdad8959cd0cd831304b0ab9 | 0 ...n_array-4-b235265cd6bd58fd743c27b02e547d62 | 1 + ...n_array-5-a9b9419b94631f8fe1c2297ebf103a9a | 2 + ...n_array-6-6eded94bd39189ea6d67fe383f9b865c | 2 + ...nsitive-0-3c6c18f40c89c8fe9891869f2f6acb48 | 0 ...nsitive-1-404d691e85c7b74bad73576ee80de290 | 0 ...ensitive-2-d508d777b4e4156bff9774925a6ca4d | 0 ...nsitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc | 0 ...nsitive-4-c880b2256f97413b8fe68d19d99747fd | 85 + ...n_empty-0-3fef0a906d6618ab28406d1edf0dc9ad | 0 ...n_empty-1-9bcf09f835d785760c0d4da0680837f2 | 0 ...n_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e | 0 ...n_empty-3-3b2231f25886c152f222856256ac3473 | 0 ...n_empty-4-df2401785dfa257de49c3ad80b0f480a | 1 + ...n_empty-5-ce1ef910fff98f174931cc641f7cef3a | 1 + ...n_empty-6-e807e73f2636bf03a572c9e0b8430b5a | 0 ...n_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 | 0 ...filters-0-5644ab44e5ba9f2941216b8d5dc33a99 | 0 ...filters-1-100faa7fd01bfb4390c782bb262a9403 | 0 ...ilters-10-f0c5c51de4151a17a77c780be0c13e01 | 4 + ...lters-100-a4c7cd87175222bea19fd33018890efe | 1 + ...lters-101-f086409eb336282af5a00f4c7192ef2b | 1 + ...lters-102-634888c4fa78ce10181c9514c6393554 | 1 + ...lters-103-c020d3461658ae8e118281f40264ae5b | 4 + ...lters-104-c9b79f30e1f25672ec89014f966b41b0 | 4 + ...ilters-105-b3d9dcbb7e1caef97646eb89edf82eb | 4 + ...lters-106-870999978978b22f21997899f1e652b8 | 4 + ...lters-107-94824a62d882227f270a66ec7ef28cd4 | 4 + ...lters-108-d793c1514545499f58fb1b355cbd3f0e | 4 + ...lters-109-2709001b4aa57ed01ba975e83b556475 | 2 + ...ilters-11-6fb35e1a65211e68de5df4a1cb9c7117 | 4 + ...lters-110-86868ef0f28c5b42f04fb9ca64aaa3ba | 2 + ...lters-111-fd3188d025e4c84d45cbb265d08ca569 | 2 + ...lters-112-a0cd07949ff7dbc2287dc060bedb1942 | 2 + ...ilters-113-48c4978768872751832149d72cdf0ba | 2 + ...lters-114-58b8db46ea5080791c7416838d3e8f95 | 2 + ...lters-115-98e45a3167d19d09219076a2d93afa49 | 2 + ...ilters-116-bec2950fff2c1b3c4dc5be846653ea6 | 2 + ...lters-117-ee8471d93c3778f4b48c5c59f473dd35 | 6 + ...lters-118-9b395d7db8722a467d46588d0f27fc9e | 6 + ...lters-119-7688784396db55ff148292a9755c918a | 6 + ...ilters-12-4c29d6be8717481332cd1ee7ca17690e | 4 + ...lters-120-6578651ffbf95c0b02825e2125e32709 | 6 + ...lters-121-6dc4977da7f8dc7c636a03d7ad892ada | 6 + ...lters-122-64bb9f9a0eef32d2392b80b93c9b2c98 | 6 + ...filters-13-c6a291879bdb37f0c84f6074f257d52 | 4 + ...ilters-14-ef8255dcad808f9128d79e6ee9f368cf | 4 + ...ilters-15-a83678913c62249c8fdf2dac1f6e3046 | 4 + ...ilters-16-a6511579b5e4c2d845664f0c4a6ce622 | 4 + ...ilters-17-c22aab929016fa6f4b6d1e14cc3a4940 | 4 + ...ilters-18-8ef51af7455eaeeaefff1c98e16dcc65 | 7 + ...ilters-19-e164192e6538b428c8b53e008a978d3f | 7 + ...filters-2-67fff9e6931a7320444e857e01b3d496 | 1 + ...ilters-20-7a5da20822bf51ed69ccf640cbb816cf | 7 + ...ilters-21-13d6d5335625fc3386a8011dc557002e | 7 + ...ilters-22-f12ffd6cc9213d9c64cbea1b6db0632e | 4 + ...ilters-23-a800b885358695758afdb719cdefa94f | 4 + ...ilters-24-df3db5a91f3f4e88e18d2b1cc5b47113 | 4 + ...ilters-25-435ecfc7f9cb5f32019fe498c21ccad3 | 4 + ...ilters-26-da36fab3d1686aedd415a7a0f0253eca | 4 + ...ilters-27-5f4a5437696f2a60bec9ac1443244242 | 4 + ...ilters-28-2acf41a9f6efac0d800df557db716359 | 1 + ...ilters-29-cf5fef4ddc313209f5ab1e5ea3763e35 | 1 + ...filters-3-989b3dbd15ba601ae80fe454e03213d7 | 4 + ...ilters-30-97f94f35ebc30f776a19bed59c7fb4bf | 1 + ...ilters-31-4923f8ba429f0c71ad9b6b9499e73a7f | 1 + ...ilters-32-5978cd7936c296493a16a31b926043ab | 1 + ...ilters-33-607d64d50ef9aad424bd22b358efe027 | 1 + ...ilters-34-35c2c8d74bc0ebc4710c81333cb254a9 | 1 + ...ilters-35-a4c7cd87175222bea19fd33018890efe | 1 + ...ilters-36-f086409eb336282af5a00f4c7192ef2b | 1 + ...ilters-37-634888c4fa78ce10181c9514c6393554 | 1 + ...ilters-38-c020d3461658ae8e118281f40264ae5b | 4 + ...ilters-39-c9b79f30e1f25672ec89014f966b41b0 | 4 + ..._filters-4-33bfcd576019d7e32683556f66e3757 | 4 + ...filters-40-b3d9dcbb7e1caef97646eb89edf82eb | 4 + ...ilters-41-870999978978b22f21997899f1e652b8 | 4 + ...ilters-42-94824a62d882227f270a66ec7ef28cd4 | 4 + ...ilters-43-d793c1514545499f58fb1b355cbd3f0e | 4 + ...ilters-44-6d8955591f62d9cfc6af17df63d3d88e | 0 ...ilters-45-23ab7ac8229a53d391195be7ca092429 | 0 ...ilters-46-aa161b0d9fe9d1aad10654fce0e3670b | 0 ...ilters-47-3c52df82c7d78501610f3f898103f753 | 0 ...ilters-48-1d85bb008e02ef4025171a4bc0866a6c | 0 ..._filters-49-e79c906b894fed049ddfab4496a4e3 | 0 ...filters-5-f0c0d07019afb1bbe162e3183e18023e | 7 + ...ilters-50-3e6612a89e9124592e790594775054b1 | 1 + ...ilters-51-60a5f56f33fc8854a2b687005f0d96ac | 1 + ...ilters-52-64cabe5164130a94f387288f37b62d71 | 1 + ...ilters-53-2709001b4aa57ed01ba975e83b556475 | 2 + ...ilters-54-86868ef0f28c5b42f04fb9ca64aaa3ba | 2 + ...ilters-55-fd3188d025e4c84d45cbb265d08ca569 | 2 + ...ilters-56-a0cd07949ff7dbc2287dc060bedb1942 | 2 + ...filters-57-48c4978768872751832149d72cdf0ba | 2 + ...ilters-58-58b8db46ea5080791c7416838d3e8f95 | 2 + ...ilters-59-98e45a3167d19d09219076a2d93afa49 | 2 + ...filters-6-c0c40d001cac0bc91095dddda1513ad9 | 1 + ...filters-60-bec2950fff2c1b3c4dc5be846653ea6 | 2 + ...ilters-61-ee8471d93c3778f4b48c5c59f473dd35 | 6 + ...ilters-62-9b395d7db8722a467d46588d0f27fc9e | 6 + ...ilters-63-7688784396db55ff148292a9755c918a | 6 + ...ilters-64-6578651ffbf95c0b02825e2125e32709 | 6 + ...ilters-65-6dc4977da7f8dc7c636a03d7ad892ada | 6 + ...ilters-66-64bb9f9a0eef32d2392b80b93c9b2c98 | 6 + ...ilters-67-67fff9e6931a7320444e857e01b3d496 | 1 + ...ilters-68-989b3dbd15ba601ae80fe454e03213d7 | 4 + ...filters-69-33bfcd576019d7e32683556f66e3757 | 4 + ...filters-7-89963646509154a2fb1ddbbf1f55349d | 1 + ...ilters-70-f0c0d07019afb1bbe162e3183e18023e | 7 + ...ilters-71-c0c40d001cac0bc91095dddda1513ad9 | 1 + ...ilters-72-89963646509154a2fb1ddbbf1f55349d | 1 + ...ilters-73-69e0235472d7cee7d83037cd083544a5 | 1 + ...ilters-74-b6372cc006844e8488a3b7836c67daaa | 1 + ...ilters-75-f0c5c51de4151a17a77c780be0c13e01 | 4 + ...ilters-76-6fb35e1a65211e68de5df4a1cb9c7117 | 4 + ...ilters-77-4c29d6be8717481332cd1ee7ca17690e | 4 + ...filters-78-c6a291879bdb37f0c84f6074f257d52 | 4 + ...ilters-79-ef8255dcad808f9128d79e6ee9f368cf | 4 + ...filters-8-69e0235472d7cee7d83037cd083544a5 | 1 + ...ilters-80-a83678913c62249c8fdf2dac1f6e3046 | 4 + ...ilters-81-a6511579b5e4c2d845664f0c4a6ce622 | 4 + ...ilters-82-c22aab929016fa6f4b6d1e14cc3a4940 | 4 + ...ilters-83-8ef51af7455eaeeaefff1c98e16dcc65 | 7 + ...ilters-84-e164192e6538b428c8b53e008a978d3f | 7 + ...ilters-85-7a5da20822bf51ed69ccf640cbb816cf | 7 + ...ilters-86-13d6d5335625fc3386a8011dc557002e | 7 + ...ilters-87-f12ffd6cc9213d9c64cbea1b6db0632e | 4 + ...ilters-88-a800b885358695758afdb719cdefa94f | 4 + ...ilters-89-df3db5a91f3f4e88e18d2b1cc5b47113 | 4 + ...filters-9-b6372cc006844e8488a3b7836c67daaa | 1 + ...ilters-90-435ecfc7f9cb5f32019fe498c21ccad3 | 4 + ...ilters-91-da36fab3d1686aedd415a7a0f0253eca | 4 + ...ilters-92-5f4a5437696f2a60bec9ac1443244242 | 4 + ...ilters-93-2acf41a9f6efac0d800df557db716359 | 1 + ...ilters-94-cf5fef4ddc313209f5ab1e5ea3763e35 | 1 + ...ilters-95-97f94f35ebc30f776a19bed59c7fb4bf | 1 + ...ilters-96-4923f8ba429f0c71ad9b6b9499e73a7f | 1 + ...ilters-97-5978cd7936c296493a16a31b926043ab | 1 + ...ilters-98-607d64d50ef9aad424bd22b358efe027 | 1 + ...ilters-99-35c2c8d74bc0ebc4710c81333cb254a9 | 1 + ...overlap-0-990e447b6447ced0d9684eb7db9e63ce | 0 ...overlap-1-a7336cd2d643f92715b42f9c6eb2c108 | 0 ...ive_626-0-3491c764e44ee92af3a88d301a1bd498 | 0 ...ive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 | 0 ...ive_626-2-42997e626819de9cdad544d7fe132c6e | 0 ...ive_626-3-2609f2809e0c6ae1ca93806e37960990 | 0 ...ive_626-4-387e3bc517b49d4e1c9752c07b72b790 | 0 ...ive_626-5-a0eb25c15b6ca1a04da14781b1213e11 | 0 ...ive_626-6-392f6f482886b2e7be5903d44bfba3f1 | 0 ...ive_626-7-d9c817eebc533bedc3ef9172d325a2c2 | 1 + ..._map_ppr-0-775b1af8eae9b2250052be1b72b4086 | 0 ...map_ppr-1-7b9086c721214279a2f0c64d6d35df77 | 0 ...ap_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa | 107 + ...map_ppr-2-e3e5db334ed33b17026b43f776daee1e | 0 ...map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa | 107 + ...map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 | 0 ...map_ppr-5-b7e6358104eceede7389c7d2a212b058 | 0 ...map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a | 0 ..._map_ppr-7-45704df71b70e3f5aa31966ac96b9db | 0 ...map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed | 0 ...map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a | 0 ...n_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 | 0 ...n_nulls-1-97641998eb9ddf2dff56de6758486aa0 | 0 ..._nulls-10-39071d8e6b246cfd405714dbf0b5337b | 3 + ..._nulls-11-545dff878ea0d79cdfee517572c9e0c8 | 3 + ..._nulls-12-5e1ca8627aa685435142d8e339e77062 | 3 + ..._nulls-13-5f670a20cbec42a34685ee0398ad7e2d | 3 + ...n_nulls-14-e97ba69145da387a4a66635b8499077 | 3 + ..._nulls-15-e9f9be8be6e936fb832483dfa0be5126 | 3 + ..._nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 | 3 + ...in_nulls-17-2b0bfe5754456475ceb6af4694165f | 5 + ..._nulls-18-321cf9d31dac835c3def6ca3b3b860a2 | 4 + ..._nulls-19-739bf8e440e698540d18c29226c3564c | 4 + ...n_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 | 9 + ..._nulls-20-fff960f1cb832bc93e3d1de519e573d5 | 5 + ..._nulls-21-62a25fb819ae5c1ea757b6e759082a2e | 3 + ..._nulls-22-5b2df5518994ae86c041484561857da0 | 3 + ..._nulls-23-982c3e55235cafa3d89b5dee4366fdf8 | 3 + ..._nulls-24-86ad66543a735d396f7336cb5bdfa495 | 9 + ..._nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 | 1 + ..._nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 | 2 + ..._nulls-27-6ee7affed896b1c539628ab081842b83 | 2 + ..._nulls-28-455aace3472c5840a885b6fab6a046cb | 1 + ..._nulls-29-2c278a88713aef5cd30ff3720ef3eeeb | 2 + ...n_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 | 9 + ..._nulls-30-2c0c41da38714d1b16feffb00fa08bb1 | 2 + ..._nulls-31-a33c48d38817ee3a7aca511dc7793486 | 1 + ..._nulls-32-e6b104ae96622ff75d510efc6efc9352 | 3 + ..._nulls-33-1284a11bf6aeef8ff87b471d41985f26 | 3 + ..._nulls-34-aeb90811861431cadc5512637793afc1 | 3 + ..._nulls-35-2d1d18d4e9775ec69333947fbd62bc82 | 3 + ..._nulls-36-7c029c91141b36af79ba0dc1de73a257 | 3 + ..._nulls-37-fa84731f5a6beec20d64a7981815b9bc | 3 + ..._nulls-38-6d8955591f62d9cfc6af17df63d3d88e | 0 ..._nulls-39-23ab7ac8229a53d391195be7ca092429 | 0 ...n_nulls-4-a1f20b4863428627bae1316755cc2d94 | 9 + ..._nulls-40-aa161b0d9fe9d1aad10654fce0e3670b | 0 ..._nulls-41-3c52df82c7d78501610f3f898103f753 | 0 ..._nulls-42-1d85bb008e02ef4025171a4bc0866a6c | 0 ...in_nulls-43-e79c906b894fed049ddfab4496a4e3 | 0 ..._nulls-44-2db30531137611e06fdba478ca7a8412 | 1 + ..._nulls-45-e58b2754e8d9c56a473557a549d0d2b9 | 1 + ..._nulls-46-64cabe5164130a94f387288f37b62d71 | 1 + ..._nulls-47-ebf794e8b51be738e2d664f249869de1 | 4 + ...n_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 | 2 + ..._nulls-49-2d20d8f4221281a6b1cd579e761f81b7 | 6 + ...n_nulls-5-5ec6e3df7181e0738244623cc01bf22c | 1 + ...n_nulls-50-8b26343e74c161cf74ece5f0bd21470 | 4 + ..._nulls-51-75339d2eb2afabf5dd088074b2563d8f | 6 + ...n_nulls-52-caad1db99085760daaf8f96c0ce5564 | 2 + ..._nulls-53-ec965e66e7fad403cd0ea388c3e70363 | 2 + ...in_nulls-54-6f15c794b5587430ebb685cc61c502 | 6 + ..._nulls-55-a1c73326f8c8d9beccda3ba365352564 | 6 + ..._nulls-56-f7b9629093c818b6c384c79d1458d178 | 4 + ..._nulls-57-cf353446d7f358a508f17d0984b90158 | 6 + ..._nulls-58-5f9a59160a76f9b649102a97987ed33a | 4 + ..._nulls-59-8753a39412ac59c7a05951aeeea73b24 | 6 + ...n_nulls-6-7eea211c80e7f1146098e80ffb890d67 | 2 + ...n_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 | 2 + ...n_nulls-8-609f6bd812a44b20da0a39c827e4d870 | 1 + ...n_nulls-9-ef4b27a877efc377cd5750af6725194b | 3 + ...join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 | 0 ...join_rc-1-6a7685f30de00ebb4867a4002d641a5e | 0 ...join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 | 0 ...join_rc-3-4a737f3e518f5322ba41a189e79c1dee | 0 ...join_rc-4-f60f7fdd08e85fae90af59475192b725 | 0 ...join_rc-5-1aef75afe38d512addb44dbf9a650263 | 1028 +++++++ ...eorder2-0-caf1c5fd299fdbdb655234d01d44caf2 | 0 ...eorder2-1-744a018c78bae6e09853dd202981e850 | 0 ...order2-10-45349471e0e919bd2185f584e87b891d | 0 ...order2-11-f8460b061fa00f6afbfe8eeccf4d3564 | 1 + ...eorder2-2-fa53198d9eecb9d274b09d4351b9274e | 0 ...eorder2-3-38a5e7f36d579835f9c2b1c52efd9643 | 0 ...eorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...eorder2-5-4abc4f450a58ccdd0df2e345f1276979 | 0 ...eorder2-6-3fda17e4414d191f837631438a19e700 | 0 ...eorder2-7-512b75ccb9459a6334da1d9699f4a5ec | 0 ...eorder2-8-68b763a6f1f673daa35fee55aeae84a1 | 0 ...eorder2-9-26ffed826eceda953b7124ee39ace828 | 1 + ...eorder3-0-caf1c5fd299fdbdb655234d01d44caf2 | 0 ...eorder3-1-744a018c78bae6e09853dd202981e850 | 0 ...order3-10-ff036400019164ed743ecd9cfc222ce1 | 0 ...order3-11-d6392b851f7dd5e3705e8ff51e1c6583 | 1 + ...eorder3-2-fa53198d9eecb9d274b09d4351b9274e | 0 ...eorder3-3-38a5e7f36d579835f9c2b1c52efd9643 | 0 ...eorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...eorder3-5-4abc4f450a58ccdd0df2e345f1276979 | 0 ...eorder3-6-3fda17e4414d191f837631438a19e700 | 0 ...eorder3-7-512b75ccb9459a6334da1d9699f4a5ec | 0 ...eorder3-8-743fefaadfe31ce43cd2dead85b7d741 | 0 ...eorder3-9-9d45e66a3990b7c53fd780f07cd52e13 | 1 + ...eorder4-0-799a471861f35cb065a6b1105c7d9ccf | 0 ...eorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a | 0 ...order4-10-1d6b948747ac31296710a491a5652e3f | 1 + ...eorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 | 0 ...eorder4-12-203aed2c4298eba4a3c51816a21a1c1 | 1 + ...eorder4-2-8b4643dd260f09d32af5d79de7359c0c | 0 ...eorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...eorder4-4-4abc4f450a58ccdd0df2e345f1276979 | 0 ...eorder4-5-3fda17e4414d191f837631438a19e700 | 0 ...eorder4-6-24ca942f094b14b92086305cc125e833 | 1 + ...eorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 | 0 ...reorder4-8-4bbb8e937245e2fa7cafbb423814754 | 1 + ...eorder4-9-b401eef67c1c04f875c084c9e0fda9ba | 0 ...in_star-0-6001f83f7780835737e3262a34c2b832 | 0 ...in_star-1-9f29849fe78f15b7594378a10fe3ed9d | 0 ...n_star-10-57ce75f989b3b3bfd2f2eceb228e892e | 0 ...n_star-11-eba1397e66f25cba4fd264209cc92bae | 0 ...n_star-12-89b53ae954ec88171ef87e0459f6eb82 | 0 ...n_star-13-342b7249c9ce1484869169b1b33191cb | 0 ...n_star-14-75513308d30b781fd2e06d81963c4363 | 0 ...n_star-15-24a0b1d0257bad4f85b3a122acf6bef9 | 0 ...n_star-16-24ca942f094b14b92086305cc125e833 | 1 + ...n_star-17-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...n_star-18-1c22e451845667bd6b4eac3c49c36965 | 1 + ...n_star-19-a1209595ce68e24a111c2a0d27863bfa | 0 ...in_star-2-75d7e03808482c11361ce72f2dd38de0 | 0 ...n_star-20-76473cb68a69b9408207fd43ddae9339 | 10 + ...n_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 | 0 ...n_star-22-429cfd140488242d569aee6594aae76b | 10 + ...n_star-23-bada259628918dbfb4837bcb58258530 | 0 ...n_star-24-e11ab68d72d874c7c6c658c1018f5a49 | 10 + ...n_star-25-be5af28b6bccf529a31ef68619de699e | 0 ...n_star-26-a412a0620a359a62e2ab4e45fa8e0330 | 10 + ...n_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 | 0 ...n_star-28-98a3f4d8a2209e771c57462d4b723ff9 | 10 + ...in_star-3-4f4ca11033af3894cc2e720ebb69e5da | 0 ...in_star-4-aaba115af7543b214820a48a694f2463 | 0 ...in_star-5-c72a8aff3a51f9e59bb62fd99807d8da | 0 ...in_star-6-73d96ed451a4964e5c364dbdb56d487a | 0 ...in_star-7-b007c3d4b6ed10df9e875e23644b673a | 0 ...in_star-8-a957982d8981ff0a35397ca449297024 | 0 ...in_star-9-904e30d8615eb411fb24c2cc08df94f4 | 0 ..._thrift-0-811b62ecbacdb26c67fa260ff3747a41 | 6 + ..._thrift-1-4f47dd107d2be1212411bda214c0d1db | 0 ..._thrift-2-d4bfb7bab262dc2898431680711dec1b | 11 + ...join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 | 0 ...in_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 | 0 ...in_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 | 0 ...in_view-2-eda40dba9678df62dde73fc5dafb2b44 | 0 ...in_view-3-eccb00d8dada8ab56a48c373e381e02b | 0 ...in_view-4-763ab5853bff619e6525c01e46b2a923 | 1 + ...in_view-5-85baeea910adc4589bc3ec2ce0b1e856 | 0 ...in_view-6-c37b7962ab7371f94a9c54d61f7638ef | 0 ...in_view-7-a14cfe3eff322066e61023ec06c7735d | 0 ...in_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 | 0 ...in_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 | 0 ...l view1-0-85bfbfd635aee807ede359073fb26dee | 1000 +++++++ ...l view2-0-6d92d10040c715df89db0f61ae4250bb | 1000 +++++++ ...l view3-0-55b6de1dbad92682804e98524c0d6c12 | 1000 +++++++ ...l view4-0-3ed6bfa23add415990b443fc232b4ba0 | 0 ...l view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 | 0 ...l view4-2-ed9961425eee97a5f35c5b6e69dc368e | 0 ...l view5-0-f797cf9d04a98cfe477d14703aeb294b | 1000 +++++++ ...l view6-0-f92fe6429a6630d4991f8ad76ff1d79a | 500 ++++ ...al_view-0-bc2e73d1da334f7bf550244ce6f6e77b | 0 ...al_view-1-82287db1270c77c3af7a60d65c4bdc80 | 0 ...l_view-10-1f2df08433fe12a9ae086e3d9309039e | 0 ...l_view-11-d884acac077c5a02ef048069dd8c16d7 | 3 + ...al_view-12-f539cc98118a276d38c61fcc0ad0b09 | 0 ...l_view-13-f1ca785bf643a4a3cd1f2f823da158ba | 0 ...l_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 | 20 + ...l_view-15-e1fad51e2f77de35d5781d788d70057e | 20 + ...l_view-16-5eafb0317b6974eb588075d152719a79 | 20 + ...l_view-17-4c6ff06840694c274bf8a91bade677ab | 20 + ...l_view-18-a98a136672c920c0463bd49b4ec7b65e | 20 + ...l_view-19-9b417d5554cbeb8f536a1613879295f7 | 20 + ...al_view-2-bfd7354229ac8a934c8f978b43b28565 | 0 ...al_view-20-d75015d8e44499526fec784ec00b905 | 0 ...l_view-21-9eec1a93e07de6acbc36b1156424ca11 | 13 + ...al_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e | 0 ...al_view-4-f5855ce2bc7877c698e7f94504d12d62 | 0 ...al_view-5-857f93a5c8d5ba00d214384117147bae | 0 ...al_view-6-e78723941c3b42349f9eb804b4b82d15 | 1 + ...al_view-7-7c88418f83112c55d08a0727d6248825 | 3 + ...al_view-8-d2bff21465d3db7bbff8b9913c012452 | 9 + ...al_view-9-60290e9a3b75a39c21dd44817d90d4a7 | 3 + ...view_cp-0-6ae48c990343343aad4145203c364f79 | 0 ...view_cp-1-72a14433dd2f25dd8c38107235fe1756 | 0 ...view_cp-2-647d019098d676b2fdb8a5127b633287 | 0 ...view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 | 0 ...view_cp-4-507876b610812506343e9d251645170b | 1 + ...noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 | 0 ...noalias-1-6d5806dd1d2511911a5de1e205523f42 | 2 + ...noalias-2-155b3cc2f5054725a9c2acca3c38c00a | 0 ...noalias-3-3b7045ace234af8e5e86d8ac23ccee56 | 2 + ..._noalias-4-e1eca4e08216897d090259d4fd1e3fe | 0 ...noalias-5-16d227442dd775615c6ecfceedc6c612 | 0 ...w_outer-0-b66c363516d1f717765df9b91da3b5b4 | 0 ...w_outer-1-8d4332785ff69bb86607700c133d0baa | 10 + ...w_outer-2-b4474ec12d042fca7a21a2cd551c1068 | 0 ...w_outer-3-57b2ffd7b60708551238c491a2a8685d | 10 + ...w_outer-4-96fe3dc4f8116e535c322437a59b5e4e | 0 ...w_outer-5-2ec3aeb923621c64da042402709e64e9 | 0 ...w_outer-6-511e4df505342e04c20e50fda8962120 | 10 + ...iew_ppd-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...iew_ppd-1-cd326124717660a333f0915395f96768 | 0 ...iew_ppd-2-25c2f5138fc88cf7fc34339a501de05f | 9 + ...iew_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 | 0 ...iew_ppd-4-c746547306ed67a5e8a0fce57023c9c9 | 3 + ...iew_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f | 0 ...iew_ppd-6-63f8025bbf66831e84b98f9429e6c7fa | 12 + ...iew_ppd-7-b8eb877331710c06ebc7dbaab5a7155b | 0 ...iew_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 | 27 + ...gth.udf-0-e85ebb8ce5d939964fd87bd13b326c02 | 1 + ...ushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb | 1 + ...ushdown-1-d0a93f40892e3894460553b443c77428 | 1 + ...shdown-10-19842c7708a0787e59cc5e6b8c3a4250 | 0 ...shdown-11-b435672262fc3c02d1ed1a93ff8d856f | 20 + ...shdown-12-a3516c212d6c79986536edbd9c961098 | 0 ...shdown-13-93906f7e87e5e3dc4b9590ec13b6af78 | 20 + ...shdown-14-cfae77212d164efb18208f71332fd368 | 0 ...shdown-15-62a90d3ba54cc68055b3502c3864a3c1 | 0 ...ushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 | 0 ...shdown-17-11c349c0db0f869be88351256650fe60 | 20 + ...shdown-18-66ecdd019c936dec1ab5c9389c9071b3 | 0 ...shdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...ushdown-2-4d1e292b00635298240ff909be64dce4 | 0 ...shdown-20-d4de935f7a059cce44889c6ba69cfddb | 0 ...shdown-21-f04dee0f94443ca01320657897cbb914 | 20 + ...shdown-22-4d378725e22e7e48c861983ba935bf5e | 1 + ...shdown-23-53b4dd4899cb2ba681a801a3b5ca155a | 0 ...shdown-24-cb58113b9dfd3e93d4623cba6a090976 | 100 + ...shdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a | 0 ...shdown-26-e691b284f1b830d7b83a36750105737c | 100 + ...ushdown-3-cc674af3ce71c06670e366932df43462 | 20 + ...ushdown-4-81bbb7300da27bc63f7a139677faac3f | 0 ...ushdown-5-c0664bc5bff121823ac2f37df13d6bd7 | 20 + ...pushdown-6-b722630e977a5ca509234fc417d7b30 | 0 ...ushdown-7-e8c8087a3910ea53a725b01b9d05b921 | 20 + ...ushdown-8-6194a19c1d5065731ec26ea1d5a390e1 | 0 ...ushdown-9-1446f634128a1c3e7cd224ea82452d0a | 20 + ...ineage1-0-c021950f995f1d6a7b83ffe044daa750 | 0 ...ineage1-1-aeb863486b8fe899ee741fc8c8418fc9 | 0 ...ineage1-2-f92e96c6357273ea6cbb2195a418df9c | 0 ..._double-0-10ef1098e35d900983be3814de8f974f | 0 ..._double-1-3863c17e03c9c1cd68452106a8721d13 | 1 + ...al_ints-0-5ffd1b49cdda4149aef2c61c53a56890 | 0 ...al_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 | 1 + ...l_string-0-9b48d41e5c9e41ddc070e2fd31ace15 | 0 ..._string-1-2cf4b7268b47246afdf6c792acca379d | 1 + ...yn_part1-0-463330cf55370dbe92d6ed74ef91302 | 4 + ...n_part1-1-cd849c4fe1229428da98947e3e43b46d | 0 ..._part1-10-fca9513ea05bfb8b7e0e6f337d184d66 | 1000 +++++++ ..._part1-11-9c82167763a771c175c656786d545798 | 1000 +++++++ ...n_part1-2-9c0d4354b6a9be351fa32a66ff58a177 | 0 ...n_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 | 12 + ...n_part1-4-16367c381d4b189b3640c92511244bfe | 1 + ...n_part1-5-a4fb8359a2179ec70777aad6366071b7 | 1 + ...n_part1-6-255ad4511130fb8c9ee9d65b7c95743f | 0 ...n_part1-7-a33b2c9d962e4921c98e62387f3989f7 | 0 ...n_part1-8-ea921e0af59a4940a11c94143b1c4b32 | 2 + ...n_part1-9-30bc31441828a053d1a675b225a5d617 | 2 + ...n_part10-0-463330cf55370dbe92d6ed74ef91302 | 4 + ..._part10-1-a5b30075b6c79d1a02e240f46ea1d318 | 0 ..._part10-2-151ba0c3b8317902f1235ac07d58135e | 12 + ..._part10-3-dc129f70e75cd575ce8c0de288884523 | 1 + ..._part10-4-a4fb8359a2179ec70777aad6366071b7 | 1 + ..._part10-5-16367c381d4b189b3640c92511244bfe | 1 + ..._part10-6-48d24be440fbbd48f82efeeb05f663c9 | 0 ..._part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 | 0 ..._part10-8-245027204484e281e1cfaf74386d2967 | 2 + ..._part10-9-eb29a5b2392bf821b2ff51b70863d531 | 1000 +++++++ ...n_part11-0-463330cf55370dbe92d6ed74ef91302 | 4 + ..._part11-1-bcf9e49f6a630b18108da59b243455f7 | 0 ..._part11-2-4301f87a8dbf9576788637386e26f9a2 | 12 + ..._part11-3-dc129f70e75cd575ce8c0de288884523 | 1 + ..._part11-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ..._part11-5-ea607fbed28d20e5726f4501285d698d | 1 + ..._part11-6-a4fb8359a2179ec70777aad6366071b7 | 1 + ..._part11-7-b377ea0092e921e5c07e8f34d7c9f920 | 0 ..._part11-8-9a4433518ac9ff49cb4b71812705adbb | 1000 +++++++ ..._part11-9-3889a0cba1cf3c8f8c2e67957e69406a | 1000 +++++++ ...n_part12-0-463330cf55370dbe92d6ed74ef91302 | 4 + ..._part12-1-37e5641a1239a05bbd1fb9707f69cd59 | 0 ..._part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b | 12 + ..._part12-3-dc129f70e75cd575ce8c0de288884523 | 1 + ..._part12-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ..._part12-5-a4fb8359a2179ec70777aad6366071b7 | 1 + ..._part12-6-e5c79bdfc92b7b754b003d718d9717a1 | 0 ..._part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 | 2 + ...n_part12-8-fd656b581b8f8fbb8ac22f444dbc345 | 2000 +++++++++++++ ...n_part13-0-463330cf55370dbe92d6ed74ef91302 | 4 + ..._part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 | 0 ..._part13-2-d52536b8ac62f6e8152e394fe135a3e0 | 12 + ..._part13-3-dc129f70e75cd575ce8c0de288884523 | 1 + ..._part13-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ..._part13-5-a4fb8359a2179ec70777aad6366071b7 | 1 + ..._part13-6-67ee926bc6b325a6bfc952bb81752a3d | 0 ..._part13-7-99993811a25b02e7904a9403f51775d5 | 0 ...n_part13-8-930d8e7a94f0cdf922322dae4a02e16 | 2 + ..._part13-9-c8de411bc094b37b59a2eb0baf6de55d | 34 + ..._part14-0-ff0d0473e56406f7cb66e83b9af25a6a | 0 ..._part14-1-253e2a15bfaef9aa781dc29fa324b51e | 9 + ..._part14-2-a4fb8359a2179ec70777aad6366071b7 | 1 + ..._part14-3-16367c381d4b189b3640c92511244bfe | 1 + ..._part14-4-584fc7f440280f67bf5bfdb23370cafd | 0 ..._part14-5-ce75e50bda381af53c8549b0d8662d94 | 0 ..._part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 | 2 + ..._part14-7-7c931249465f330d51ef0610f214429e | 6 + ...rt14_win-0-a58efbee279cc96fb5738e6ab389927 | 0 ...t14_win-1-253e2a15bfaef9aa781dc29fa324b51e | 9 + ...t14_win-2-a4fb8359a2179ec70777aad6366071b7 | 1 + ...t14_win-3-16367c381d4b189b3640c92511244bfe | 1 + ...t14_win-4-584fc7f440280f67bf5bfdb23370cafd | 0 ...t14_win-5-ce75e50bda381af53c8549b0d8662d94 | 0 ...t14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 | 2 + ...t14_win-7-7c931249465f330d51ef0610f214429e | 6 + ..._part15-0-a4fb8359a2179ec70777aad6366071b7 | 1 + ..._part15-1-16367c381d4b189b3640c92511244bfe | 1 + ..._part15-2-1379abc2de057dc6d240a526f0dd8a3c | 0 ..._part15-3-9940fad8d8590e60bd726e9503ae4fa9 | 0 ..._part15-4-fa01199bab3e6aa47a82c9aec88aa76a | 0 ..._part15-5-9940fad8d8590e60bd726e9503ae4fa9 | 3 + ...n_part2-0-294e3beb0241f9f8eac7f54e1bfd775f | 0 ...n_part2-1-845923af04bb331c9f5995a3a3e84424 | 12 + ...n_part2-2-dc129f70e75cd575ce8c0de288884523 | 1 + ...n_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...n_part2-4-a4fb8359a2179ec70777aad6366071b7 | 1 + ...n_part2-5-fdb342199af53c5c18529c3a0472a38b | 0 ...n_part2-6-60864ea22e1173981ed651ddc2d944c4 | 0 ...n_part2-7-86ffa99b03fa88235b61bf1af7062c33 | 2 + ...n_part2-8-a1ff8a12d94378e7e1165bd78cf724cf | 1000 +++++++ ...n_part2-9-3f29de9877006f9448272ef2422d6132 | 1000 +++++++ ...yn_part3-0-463330cf55370dbe92d6ed74ef91302 | 4 + ...n_part3-1-ce25d1f46dd5e5d9147e39566581514f | 0 ...n_part3-2-dbbba335c008a61a13c1472b34470397 | 12 + ...n_part3-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...n_part3-4-16367c381d4b189b3640c92511244bfe | 1 + ...n_part3-5-a4fb8359a2179ec70777aad6366071b7 | 1 + ...n_part3-6-17dee8c004f1e7db4542fbf4241edce3 | 0 ...n_part3-7-867958e24e25ad098c5001bbc7102762 | 0 ...n_part3-8-c3b6f0cc860b84851dd60c53b36a6437 | 2000 +++++++++++++ ...yn_part4-0-463330cf55370dbe92d6ed74ef91302 | 4 + ...n_part4-1-50822afef6986cfea20523ea6092e455 | 0 ..._part4-10-7f0c1e54518d4d1d345577a100e670e8 | 1500 ++++++++++ ..._part4-11-1313644c9dad948bfcebd7386c309ab7 | 2500 ++++++++++++++++ ...n_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd | 12 + ...n_part4-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...n_part4-4-a4fb8359a2179ec70777aad6366071b7 | 1 + ...n_part4-5-16367c381d4b189b3640c92511244bfe | 1 + ...n_part4-6-2869679fad49c57ba08169ea32271379 | 0 ...n_part4-7-f31c7a56b50a5ba425e85480e13c39e1 | 0 ...n_part4-8-cdff1da8e9e1e9242c49d895751af0a9 | 0 ...n_part4-9-3c344e5840c1df354a2a71722c27f0a0 | 5 + ...n_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 | 0 ...n_part5-1-9a4d1f6a14227bb66bd01557a464da8b | 9 + ..._part5-10-be7953ca7bd26623d3897f5060e13737 | 3 + ..._part5-11-6da8fa1e639db104128ba7e2f88f764d | 1 + ...n_part5-2-dc129f70e75cd575ce8c0de288884523 | 1 + ...n_part5-3-a4fb8359a2179ec70777aad6366071b7 | 1 + ...n_part5-4-16367c381d4b189b3640c92511244bfe | 1 + ...n_part5-5-67f2c7448db01b6804c846f9f4f76928 | 1 + ...n_part5-6-f6dcb13dead8bb4c003eb19099908190 | 1 + ...yn_part5-7-3ca3cc56a27939d62db1b52d86309df | 0 ...n_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 | 0 ...n_part5-9-854026d1c2add692f2f90ad74668bbf6 | 309 ++ ...yn_part6-0-463330cf55370dbe92d6ed74ef91302 | 4 + ...n_part6-1-9657a48a4266c176f84c7aaf115fbc30 | 0 ...n_part6-2-779aa345cf2875089312ec26b046415d | 12 + ...n_part6-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...n_part6-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...n_part6-5-a4fb8359a2179ec70777aad6366071b7 | 1 + ...n_part6-6-559ff31031a6a0ff6959655c6acd07a3 | 0 ...n_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 | 1000 +++++++ ...yn_part6-8-1009bd2cbd88ddba97186fb76e96a4f | 1000 +++++++ ...yn_part7-0-463330cf55370dbe92d6ed74ef91302 | 4 + ...n_part7-1-e31d34a1b14d706d2b78d083ea858c81 | 0 ...n_part7-2-5d4c3c48f53d55e26ca142ee70d1706a | 12 + ...n_part7-3-ad94ac1192dbace6b5cf5915387e94e2 | 0 ...n_part7-4-e98039d0d4ef775fb5594bebffacf4f6 | 1 + ...n_part7-5-b3615ce72a55dba805303145030c8a93 | 500 ++++ ...yn_part8-0-463330cf55370dbe92d6ed74ef91302 | 4 + ...n_part8-1-ff5ce932ae5ba496badee7f2465f272c | 0 ...n_part8-2-9e1df41acabef83f62464f52c2396c8a | 12 + ...n_part8-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...n_part8-4-a4fb8359a2179ec70777aad6366071b7 | 1 + ...n_part8-5-16367c381d4b189b3640c92511244bfe | 1 + ...n_part8-6-86db06cb739ceef70c8885469d847495 | 0 ...n_part8-7-be6ace08b68bc4346456103640308cf7 | 0 ...n_part8-8-cc6aa0fc59ede89158d4f19752660b8b | 4 + ...n_part8-9-a7456fceb35f62a282db750384f480db | 2000 +++++++++++++ ...yn_part9-0-463330cf55370dbe92d6ed74ef91302 | 4 + ...n_part9-1-2c61920dcd46ece096fd12875871709f | 0 ...n_part9-2-748ac33315295f8e55f2480f7714c27a | 12 + ...n_part9-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...n_part9-4-a4fb8359a2179ec70777aad6366071b7 | 1 + ...n_part9-5-16367c381d4b189b3640c92511244bfe | 1 + ...n_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 | 0 ...n_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b | 0 ...yn_part9-8-504a475a583d33f7c71db57e6774919 | 2 + ...n_part9-9-72b1ad2231269b704130903b35ac04bb | 1000 +++++++ ...success-0-84028c4ca541d126baffc20d6d876810 | 0 ...success-1-c84cd1b5b491bded8ac3b0521de599c5 | 0 ...success-2-7368973ec1870dd3b237c37eb3857b1e | 1 + ...success-3-9ee887603dcba5200918ae5200afa5d5 | 0 ...he_name-0-8b3d200fd29aeafd07f16e7a732585a6 | 0 ...he_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f | 0 ...oad_fs2-0-517732da2c84ae17095b0e1d96f74d97 | 0 ...oad_fs2-1-5018c84e09be70bf663594a89f3ad731 | 0 ...oad_fs2-2-94d2317b453b3b49bb9f2b58040b4748 | 0 ...he_name-0-f99b4f29506d65c841fb1db18263fbcc | 1 + ...he_name-1-b64a19f7101a4fb3b5d08b2f6e296400 | 1 + ...he_name-2-2087e00fe000e00f64e819dca59be450 | 0 ...success-0-fa705a031ff5d97558f29c2b5b9de282 | 0 ...success-1-7368973ec1870dd3b237c37eb3857b1e | 1 + ...success-2-9ee887603dcba5200918ae5200afa5d5 | 0 ...success-0-84028c4ca541d126baffc20d6d876810 | 0 ...success-1-7368973ec1870dd3b237c37eb3857b1e | 1 + ...success-2-9ee887603dcba5200918ae5200afa5d5 | 0 ...adpart1-0-fa705a031ff5d97558f29c2b5b9de282 | 0 ...adpart1-1-4bf1504274319c44d370b58092fe016c | 0 ...adpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 | 0 ...adpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 | 0 ...adpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 | 6 + ...oadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc | 0 ...oadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 | 0 ...adpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 | 6 + ...oadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 | 0 ...adpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 | 0 ...art_err-0-8d7fa992960b4c8ec2cb874be479de37 | 1 + ...art_err-1-8e68053c7f96f268ae1e25082e9b7517 | 1 + ...art_err-2-21fe8ff9059167209647e7ea086f483e | 0 .../lock1-0-cd46bc635e3010cf1b990a652a584a09 | 0 .../lock1-1-3e95421993ab28d18245ec2340f580a3 | 0 .../lock1-2-c0c18ac884677231a41eea8d980d0451 | 0 .../lock2-0-cd46bc635e3010cf1b990a652a584a09 | 0 .../lock2-1-3e95421993ab28d18245ec2340f580a3 | 0 .../lock2-2-c0c18ac884677231a41eea8d980d0451 | 0 .../lock2-3-27ad2962fed131f51ba802596ba37278 | 0 .../lock2-4-c06da7f8c1e98dc22e3171018e357f6a | 0 .../lock2-5-e8d1d10c308a73eef78dde414a5e40ca | 0 .../lock3-0-27ad2962fed131f51ba802596ba37278 | 0 .../lock3-1-c06da7f8c1e98dc22e3171018e357f6a | 0 .../lock3-2-b1ca816784e88f105b2fce1175340c33 | 0 .../lock3-3-16367c381d4b189b3640c92511244bfe | 1 + .../lock3-4-a4fb8359a2179ec70777aad6366071b7 | 1 + .../lock3-5-8096935c5c1755f9b88583e8c72921ac | 0 .../lock3-6-598ba296ba1c6931f4161a9f50b00cbe | 0 .../lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 | 1 + .../lock4-1-27ad2962fed131f51ba802596ba37278 | 0 .../lock4-2-c06da7f8c1e98dc22e3171018e357f6a | 0 .../lock4-3-b1ca816784e88f105b2fce1175340c33 | 0 .../lock4-4-16367c381d4b189b3640c92511244bfe | 1 + .../lock4-5-a4fb8359a2179ec70777aad6366071b7 | 1 + .../lock4-6-8096935c5c1755f9b88583e8c72921ac | 0 .../lock4-7-598ba296ba1c6931f4161a9f50b00cbe | 0 ...oin_ppr-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...oin_ppr-1-498e526f13a05a053a338d766f7351cd | 0 ...oin_ppr-2-3d41e966f69a64babb783d1aad0f1b73 | 12 + ...oin_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 | 0 ...oin_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef | 12 + ...oin_ppr-5-1209db6544b421ea32a360d863becd94 | 0 ...join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 | 12 + ...oin_ppr-7-9548b9e389f361ac8eccb3de7255da42 | 0 ...join_ppr-8-6fca189c46645f124d5fcb82564b703 | 12 + ...apjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d | 1 + ...apjoin1-1-abd9364d276ec89352232da5e2237768 | 0 ...pjoin1-10-c08fefa00b89f50dd365208151593505 | 10 + ...pjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 | 0 ...pjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 | 10 + ...apjoin1-2-fe84593f006c85e68fbb797394cdccd0 | 10 + ...apjoin1-3-8439a0592619790b64d16d2506f2233d | 0 ...apjoin1-4-c08fefa00b89f50dd365208151593505 | 10 + ...apjoin1-5-72068bd4cdac40e4d18fd729f39855ba | 0 ...apjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 | 10 + ...apjoin1-7-757dfb540b8a49b3663f8caba7476ec5 | 0 ...apjoin1-8-fe84593f006c85e68fbb797394cdccd0 | 10 + ...apjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 | 0 ...istinct-0-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...istinct-1-a00d1791b7fa7ac5a0505d95c3d12257 | 1 + ...stinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 | 0 ...stinct-11-4489654b888efc588b13ee1cda1b6a9f | 10 + ...stinct-12-1d351f7e821fcaf66c6f7503e42fb291 | 1 + ...stinct-13-a7dc16cb82c595b18d4258a38a304b1e | 1 + ...stinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 | 0 ...stinct-15-4489654b888efc588b13ee1cda1b6a9f | 10 + ...istinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 | 0 ...istinct-3-4489654b888efc588b13ee1cda1b6a9f | 10 + ...istinct-4-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...istinct-5-a7dc16cb82c595b18d4258a38a304b1e | 1 + ...istinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 | 0 ...istinct-7-4489654b888efc588b13ee1cda1b6a9f | 10 + ...istinct-8-1d351f7e821fcaf66c6f7503e42fb291 | 1 + ...istinct-9-a00d1791b7fa7ac5a0505d95c3d12257 | 1 + ...terjoin-0-407016bf2679fb9e9d076a2d115e859d | 1 + ...mapjoin-0-24ca942f094b14b92086305cc125e833 | 1 + ...mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...mapjoin-2-25fc734982956a164adde6bb1d4d8751 | 1 + ...mapjoin-3-cb38700198e89779e4dc1b25026d92a1 | 0 ...mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 | 0 ...mapjoin-5-c47698bac140454637a999e583941ce7 | 2 + ...ubquery-0-24ca942f094b14b92086305cc125e833 | 1 + ...ubquery-1-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...ubquery-2-25fc734982956a164adde6bb1d4d8751 | 1 + ...ubquery-3-4422532005e909173a4799d8d5091f1b | 0 ...ubquery-4-7df121f9774cb23edc557b98ad1e1924 | 107 + ...ubquery-5-d9e59bfa950495629b7ba4bc6700405c | 0 ...ubquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b | 107 + ...bquery2-0-59fdb2842fbf4f530366f0237ff235e2 | 0 ...bquery2-1-c0a9763a930555f846a2576d003fb517 | 0 ...query2-10-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...query2-11-25fc734982956a164adde6bb1d4d8751 | 1 + ...query2-12-7ebf0bee394756c0e0c78bdd1034f183 | 0 ...query2-13-c876a518451059f17fc15e29f6f57951 | 2 + ...query2-14-59fdb2842fbf4f530366f0237ff235e2 | 0 ...query2-15-c0a9763a930555f846a2576d003fb517 | 0 ...query2-16-2f8ecc54049960ee4707f5e6f491fdf4 | 0 ...bquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 | 0 ...bquery2-3-681c6f08aac965e4156dbd5800064a68 | 0 ...bquery2-4-ab6020d67f5a99c0a87e630790507345 | 0 ...bquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 | 0 ...bquery2-6-5353ee601eb42d5842690d3941683be1 | 0 ...bquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 | 0 ...bquery2-8-d524906728fef9f559709fe0922ab24e | 0 ...bquery2-9-24ca942f094b14b92086305cc125e833 | 1 + ...t_outer-0-407016bf2679fb9e9d076a2d115e859d | 1 + ...t_outer-1-b1ac944eac23eb8af1f1f659659c7bcc | 0 ..._outer-10-ce1ef910fff98f174931cc641f7cef3a | 1 + ..._outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 | 4 + ..._outer-12-80993ab7f757001e1f058bf8609f8420 | 4 + ..._outer-13-a5d200c74f7000ad3e36fac90c980d34 | 0 ..._outer-14-7fe52008c4a98853d086d17fc3c21906 | 5 + ...t_outer-2-43561759b314d02b2dba5eb49a65c515 | 0 ...t_outer-3-6ee8075ac3ad15ad1ac103d815544e7f | 0 ...st_outer-4-9e1f369b391b4a050250e0a954ffbb8 | 0 ...t_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 | 0 ...t_outer-6-7fe52008c4a98853d086d17fc3c21906 | 5 + ...t_outer-7-ab7726921abfa43bb20ddfbc05f73c24 | 0 ...t_outer-8-dfb08d397d3fe163d75c3b758097b68a | 5 + ...t_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 | 4 + ...reduce1-0-904b34e86c266384dc261655162dde3c | 0 ...preduce1-1-e43492b1834d823a66b0f6499b7d2fe | 0 ...preduce1-2-c32bd8b1734c410b3686469a7a3500e | 0 ...reduce1-3-adc1ec67836b26b60d8547c4996bfd8f | 500 ++++ ...reduce2-0-904b34e86c266384dc261655162dde3c | 0 ...reduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 | 0 ...reduce2-2-3442d32acb17e007a8f844b38fda9a1b | 0 ...reduce2-3-adea843673e541da8a735a5a34e7c7dc | 500 ++++ ...reduce3-0-904b34e86c266384dc261655162dde3c | 0 ...reduce3-1-efaeccafc3db890b344dc6037266b71b | 0 ...reduce3-2-51905797c8299159dc1cf1ef69cd7a9a | 0 ...reduce3-3-adc1ec67836b26b60d8547c4996bfd8f | 500 ++++ ...reduce4-0-904b34e86c266384dc261655162dde3c | 0 ...reduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f | 0 ...reduce4-2-6906be683bdd3666075760de019ac5ab | 0 ...reduce4-3-adc1ec67836b26b60d8547c4996bfd8f | 500 ++++ ...reduce5-0-904b34e86c266384dc261655162dde3c | 0 ...reduce5-1-52bff54eba53868ef3fd026cc6301a13 | 0 ...reduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca | 0 ...reduce5-3-adc1ec67836b26b60d8547c4996bfd8f | 500 ++++ ...reduce6-0-904b34e86c266384dc261655162dde3c | 0 ...reduce6-1-d5ed2c0aafd39b76fa414194add93ffb | 0 ...reduce6-2-c55bb2d4c519d62331671a5d7685f2b8 | 0 ...reduce6-3-adc1ec67836b26b60d8547c4996bfd8f | 500 ++++ ...reduce7-0-78c8b5bf2d2a258066786ba03696ab82 | 0 ...reduce7-1-8b7f1f886b749e43460052c98acd082a | 0 ...reduce7-2-b57470174a24aa3861df022375754e90 | 0 ...reduce7-3-adc1ec67836b26b60d8547c4996bfd8f | 500 ++++ ...reduce8-0-78c8b5bf2d2a258066786ba03696ab82 | 0 ...reduce8-1-301e5bdcbbcdaea60348b3c24336629b | 0 ...reduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 | 0 ...reduce8-3-adc1ec67836b26b60d8547c4996bfd8f | 500 ++++ .../merge1-0-593999fae618b6b38322bc9ae4e0c027 | 1 + .../merge1-1-2c73c923962b91afdf0004a705432550 | 0 ...merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 | 0 ...merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 | 0 ...merge1-12-2e8e4adbfb21f25e7557dd86363c7138 | 1 + ...merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 | 0 ...merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 | 0 .../merge1-2-34854e1283de8e9ada3edd6bf897bc67 | 0 .../merge1-3-1f560722f18ef618a0343313a0cac462 | 0 .../merge1-4-3277fe538b66923cd879b45371838d2b | 309 ++ .../merge1-5-2f16345a20407b01e5cc5ae26ea902b0 | 0 .../merge1-6-1c02b26792354eae7fb53fa3cb752ac1 | 0 .../merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 | 0 .../merge1-8-37bd183ad3b7ad1e8550a138f7beb88a | 0 .../merge1-9-64678b0928c4fd054e1578458001c86 | 0 .../merge2-0-b12e5c70d6d29757471b900b6160fa8a | 1 + .../merge2-1-593999fae618b6b38322bc9ae4e0c027 | 1 + ...merge2-10-9f60e54bc4add2b1aff8473e2a756c79 | 0 ...merge2-11-1c02b26792354eae7fb53fa3cb752ac1 | 0 .../merge2-12-d2a36a13f8531cde3c66d4003048416 | 0 ...merge2-13-37bd183ad3b7ad1e8550a138f7beb88a | 0 .../merge2-14-64678b0928c4fd054e1578458001c86 | 0 ...merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 | 0 ...merge2-16-d75f4405b860e4187882a72418ed6c83 | 0 ...merge2-17-2e8e4adbfb21f25e7557dd86363c7138 | 1 + ...merge2-18-945682265ce2c1fe7fa69aeb57b4290f | 0 ...merge2-19-d75f4405b860e4187882a72418ed6c83 | 0 .../merge2-2-c95dc367df88c9e5cf77157f29ba2daf | 1 + .../merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 | 1 + .../merge2-4-84967075baa3e56fff2a23f8ab9ba076 | 1 + .../merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea | 1 + .../merge2-6-319e3c6b684d2abd4dfd7314a05d4307 | 0 .../merge2-7-fa2f100bc67426120cb97cfc75a5fb36 | 0 .../merge2-8-7435e0bf03e747705b0858d5dcccfcc1 | 0 .../merge2-9-b81efaa65e1263e48278ef9062cca1dd | 309 ++ .../merge4-0-b12e5c70d6d29757471b900b6160fa8a | 1 + .../merge4-1-593999fae618b6b38322bc9ae4e0c027 | 1 + ...merge4-10-692a197bd688b48f762e72978f54aa32 | 0 ...merge4-11-ca0336ac3f600cb8b4230d9904686868 | 1500 ++++++++++ ...merge4-12-62541540a18d68a3cb8497a741061d11 | 0 ...merge4-13-ed1103f06609365b40e78d13c654cc71 | 0 ...merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 | 3 + ...merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a | 1 + .../merge4-2-43d53504df013e6b35f81811138a167a | 1 + .../merge4-3-a4fb8359a2179ec70777aad6366071b7 | 1 + .../merge4-4-16367c381d4b189b3640c92511244bfe | 1 + .../merge4-5-3d24d877366c42030f6d9a596665720d | 0 .../merge4-6-b3a76420183795720ab3a384046e5af | 0 .../merge4-7-631a45828eae3f5f562d992efe4cd56d | 0 .../merge4-8-ca0336ac3f600cb8b4230d9904686868 | 1000 +++++++ .../merge4-9-ad3dc168c8b6f048717e39ab16b0a319 | 0 ...rtition-0-a4fb8359a2179ec70777aad6366071b7 | 1 + ...rtition-1-16367c381d4b189b3640c92511244bfe | 1 + ...tition-10-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...tition-11-a49c9ee01ce8858a5f00c05523329200 | 1 + ...tition-12-dbf4eae8430a97a6e70b1c6222218cd3 | 0 ...tition-13-16adcdb0e324ad233769e124b5b349da | 0 ...tition-14-79da9a1ce5c2d058b924387ac9fcde92 | 500 ++++ ...tition-15-d60297fed03b455c29daa4afb4d1e858 | 14 + ...tition-16-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...tition-17-b12e5c70d6d29757471b900b6160fa8a | 1 + ...tition-18-593999fae618b6b38322bc9ae4e0c027 | 1 + ...tition-19-a49c9ee01ce8858a5f00c05523329200 | 1 + ...rtition-2-190cefc93e46906e404039de0fd5f513 | 0 ...tition-20-d295db835d4fdeea34298702295ff7c5 | 0 ...tition-21-9b9493a68ef7663e95ad86d02c45ec88 | 0 ...tition-22-79da9a1ce5c2d058b924387ac9fcde92 | 500 ++++ ...tition-23-d60297fed03b455c29daa4afb4d1e858 | 14 + ...tition-24-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...tition-25-b12e5c70d6d29757471b900b6160fa8a | 1 + ...tition-26-593999fae618b6b38322bc9ae4e0c027 | 1 + ...tition-27-a49c9ee01ce8858a5f00c05523329200 | 1 + ...tition-28-ef7b35be7210f099d46448994d9dc605 | 0 ...tition-29-876c8fa15a32ac5bfcc6cb60993f6a4d | 0 ...rtition-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...tition-30-79da9a1ce5c2d058b924387ac9fcde92 | 500 ++++ ...tition-31-d60297fed03b455c29daa4afb4d1e858 | 14 + ...rtition-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...rtition-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...rtition-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...rtition-7-777de794b7f27ea63f29a9784663393b | 0 ...rtition-8-2f1578dbc029b62daa9d47d8fa473960 | 1 + ...rtition-9-dc129f70e75cd575ce8c0de288884523 | 1 + ...tition2-0-a4fb8359a2179ec70777aad6366071b7 | 1 + ...tition2-1-16367c381d4b189b3640c92511244bfe | 1 + ...ition2-10-43d53504df013e6b35f81811138a167a | 1 + ...ition2-11-b12e5c70d6d29757471b900b6160fa8a | 1 + ...ition2-12-593999fae618b6b38322bc9ae4e0c027 | 1 + ...ition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 | 1 + ...tition2-14-3a4c24fd561f459025264baa3fb6d87 | 1 + ...ition2-15-dbf4eae8430a97a6e70b1c6222218cd3 | 0 ...ition2-16-16adcdb0e324ad233769e124b5b349da | 0 ...ition2-17-d60297fed03b455c29daa4afb4d1e858 | 14 + ...tition2-2-190cefc93e46906e404039de0fd5f513 | 0 ...tition2-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...tition2-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...tition2-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...tition2-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...tition2-7-777de794b7f27ea63f29a9784663393b | 0 ...tition2-8-6f7f59de1fbd607e844a2dc9394a2df8 | 0 ...tition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba | 0 ...tition3-0-a4fb8359a2179ec70777aad6366071b7 | 1 + ...tition3-1-16367c381d4b189b3640c92511244bfe | 1 + ...ition3-10-d176a1b243ac7190fbc319d73a164e2d | 0 ...ition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 | 0 ...ition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a | 0 ...ition3-13-c512eee654e7313ff9c6efb35c5b0a88 | 0 ...ition3-14-ea3e89ffe987e20dffd8388a2dbcc260 | 0 ...ition3-15-e525a096de36a3d157db1b4947e1fbb0 | 0 ...ition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 | 4 + ...ition3-17-43d53504df013e6b35f81811138a167a | 1 + ...ition3-18-b12e5c70d6d29757471b900b6160fa8a | 1 + ...ition3-19-593999fae618b6b38322bc9ae4e0c027 | 1 + ...tition3-2-190cefc93e46906e404039de0fd5f513 | 0 ...ition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 | 1 + ...tition3-21-3a4c24fd561f459025264baa3fb6d87 | 1 + ...ition3-22-bf8877c86d4b4cd7da2939cdf517acc5 | 0 ...ition3-23-3ffba3098571099bc2b13614ae3defc5 | 0 ...ition3-24-c9f91a11d9be1e981e6ec284572bbb2a | 4 + ...ition3-25-d60297fed03b455c29daa4afb4d1e858 | 14 + ...tition3-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...tition3-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...tition3-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...tition3-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...tition3-7-777de794b7f27ea63f29a9784663393b | 0 ...tition3-8-6916eceaa04091d1453a7d0d5257213c | 0 ...tition3-9-8d0305d089aa5198601cc39073fff528 | 0 ...gejoins-0-90c36ed2dea064c1951856a1a2cd3d38 | 0 ...gejoins-1-63de7fdfd7513d63a4eadafc8534f69b | 0 ...gejoins-2-6b9f3810606db1e9036561f1173ac75c | 0 ...gejoins-3-c408f69470d652da283442a62b384e46 | 0 ...gejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 | 0 ...rgejoins-5-adae80fe415023783fca5499e3edf6e | 0 ...gejoins-6-6169410e9f077097d1a766724dfc51df | 0 ...s_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 | 0 ...s_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 | 0 ..._mixed-10-abb02d2cadc535ff51660d83e6609dc8 | 0 ..._mixed-11-725ed77dd110398f461926f190b4b5c8 | 0 ..._mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b | 0 ...s_mixed-2-de24edb80e51049d241fa0ce2e3165ff | 0 ...s_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 | 0 ...s_mixed-4-24a8048db8d561e28bcb4941498b9687 | 0 ...s_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 | 0 ...s_mixed-6-e108b1560a601946194cecaf4da12491 | 0 ...ns_mixed-7-d2068e6569b5d253932ce9d59be5221 | 0 ...s_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 | 0 ...s_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 | 0 ...taonly1-0-f0c7b56d5ad529ae6c98875501a9548d | 0 ...taonly1-1-4d93504b19d34fd360de6af7fb1f3715 | 0 ...aonly1-10-1c7bb3877b2e1e7bb9166537715d442d | 0 ...aonly1-11-30df79b460916f3f67ccf1d7b7a076f2 | 0 ...taonly1-2-9eadfd16be30c653625fce7b74048d9d | 1 + ...taonly1-3-7980a98d580a002b7ad7eef780039f67 | 0 ...taonly1-4-4d93504b19d34fd360de6af7fb1f3715 | 0 ...taonly1-5-9eadfd16be30c653625fce7b74048d9d | 1 + ...taonly1-6-537256f669bc9101d4834df67aae8cdf | 0 ...taonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc | 1 + ...taonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 | 0 ...taonly1-9-2c3b29a3df9b07e7add15e4e7c561644 | 1 + .../mi-0-a4fb8359a2179ec70777aad6366071b7 | 1 + .../mi-1-16367c381d4b189b3640c92511244bfe | 1 + .../mi-2-abf8847fb25b96e0f9477808d8378e5e | 0 .../mi-3-b66a495f7bdf106a7886b72267b8659d | 0 .../mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c | 0 .../mi-5-4aad9be12cbe4e7be5540924e6b5e7dc | 1 + .../mi-6-b331d7aa963adac3b2e1de803f90e199 | 1 + .../mi-7-fca4e06de103c3cbb675fa43e7077800 | 500 ++++ .../mi-8-e946bdb81b0a831908c1c8196fdff215 | 309 ++ ...apJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 | 0 ...apJoin1-1-10b9657a205f63e6127e29e8b477b30a | 0 ...pJoin1-10-24ca942f094b14b92086305cc125e833 | 1 + ...pJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa | 0 ...pJoin1-12-204073e1287b0582d50e652d466f1e66 | 1 + ...pJoin1-13-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...pJoin1-14-25fc734982956a164adde6bb1d4d8751 | 1 + ...pJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 | 0 ...pJoin1-16-c14b300770b329ecb71e0275c88532d3 | 1 + ...pJoin1-17-17e0bad549b265f8237e6c539b848dd5 | 0 ...pJoin1-18-49bc7f430b2591978067ca8f7d181cee | 6 + ...pJoin1-19-51acc9734833601aa37e8da9f3a06b91 | 0 ...apJoin1-2-feca2d1242cf09e54dc177881a708842 | 0 ...pJoin1-20-6fb5b848305f559c6377cb7d34cc3216 | 0 ...pJoin1-21-79973475ca07cb9932f752f6547779ac | 0 ...apJoin1-22-25e434b6d05e08fdd5f4d9957438917 | 1 + ...apJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a | 0 ...pJoin1-24-feed626e3216bcbda66b17f48305b5a1 | 1 + ...pJoin1-25-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...pJoin1-26-25fc734982956a164adde6bb1d4d8751 | 1 + ...pJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 | 0 ...pJoin1-28-feed626e3216bcbda66b17f48305b5a1 | 1 + ...pJoin1-29-ea23403b9eb55e8b06d1c198e439569f | 1 + ...apJoin1-3-9280a4b7f1087dda801c81f3a5827785 | 0 ...pJoin1-30-d8c886592f60bedef5d8cb967adcead3 | 0 ...pJoin1-31-feed626e3216bcbda66b17f48305b5a1 | 1 + ...pJoin1-32-e93301ee4ba157b466d7460775f3d350 | 1 + ...pJoin1-33-7be587cefa8323cbe42cbf469b998f7b | 0 ...pJoin1-34-feed626e3216bcbda66b17f48305b5a1 | 1 + ...apJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 | 0 ...apJoin1-5-1e0b02b515a588ea99f6027f0aca36fe | 0 ...apJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 | 0 ...apJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba | 0 ...apJoin1-8-273de5cc585b04ea2210d90c1755568c | 0 ...apJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 | 0 ...apJoin2-0-24ca942f094b14b92086305cc125e833 | 1 + ...apJoin2-1-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...pJoin2-10-1905c7759350b107679aef86226739f8 | 53 + ...pJoin2-11-b9d963d24994c47c3776dda6f7d3881f | 1 + ...pJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e | 0 ...pJoin2-13-6b984427a771fe650fa875be98722cbe | 15 + ...pJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...pJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b | 0 ...pJoin2-16-6b984427a771fe650fa875be98722cbe | 15 + ...pJoin2-17-b9d963d24994c47c3776dda6f7d3881f | 1 + ...pJoin2-18-cb1f964731ee7ac045db89266a919586 | 0 ...pJoin2-19-77324702b091d514ca16d029f65d3d56 | 15 + ...apJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 | 1 + ...pJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...pJoin2-21-3db633aa7d2c47682bb15024d6abdd33 | 0 ...pJoin2-22-77324702b091d514ca16d029f65d3d56 | 15 + ...apJoin2-23-18139096bb78fa5080054686f27e5e9 | 0 ...pJoin2-24-e148026f8994e22ca756c68753a0cc26 | 0 ...apJoin2-25-b04195464e014cb47fd20a76b5f9ac0 | 0 ...pJoin2-26-2136f3783a9764de762e49c1ca28637f | 0 ...pJoin2-27-d28d0f671f5d913a56d75812d24cca8e | 1 + ...apJoin2-3-622f276b2eb5d55649a7a8689aacea5d | 0 ...apJoin2-4-5ede8243cc4ba2fbd24a77578502a656 | 74 + ...apJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 | 1 + ...apJoin2-6-d7733ae25ad6fcb1bac1135271732502 | 0 ...apJoin2-7-5ede8243cc4ba2fbd24a77578502a656 | 74 + ...apJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 | 1 + ...apJoin2-9-e14b355524f37fe25ebbb59b52e12c74 | 0 ...ert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 | 0 ...ert_gby-1-9e61989d717403353689cbbb2816210d | 0 ...ert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 | 0 ...ert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 | 0 ...ert_gby-4-521e0c1054cfa35116c02245874a4e69 | 37 + ...ert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 | 0 ...ert_gby-6-b601e40a9c3bb8144a0447ec829ae49c | 0 ...ert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f | 0 ...ert_gby-8-521e0c1054cfa35116c02245874a4e69 | 37 + ...ert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 | 309 ++ ...rt_gby2-0-b3ee4be40513342084411c5333416d69 | 0 ...rt_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 | 0 ...rt_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 | 0 ...rt_gby2-3-538a02b95c066b307652c8d503470c8e | 0 ...rt_gby2-4-521e0c1054cfa35116c02245874a4e69 | 1 + ...rt_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 | 1 + ...rt_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 | 0 ...ert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a | 0 ...t_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 | 309 ++ ...t_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc | 0 ...t_gby3-12-33d7e716735d24b7493209810d0b865f | 0 ...rt_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 | 0 ...rt_gby3-3-5ae64ea7cfb7bef5a99d788016213fca | 0 ...rt_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 | 0 ...rt_gby3-5-a66621daa1b2479beee5681a22d9d712 | 0 ...rt_gby3-6-521e0c1054cfa35116c02245874a4e69 | 309 ++ ...rt_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 | 309 ++ ...rt_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b | 0 ...rt_gby3-9-521e0c1054cfa35116c02245874a4e69 | 309 ++ ...al_view-0-531b7044d2fdaba4fff0094c4efdaf54 | 0 ...al_view-1-3ed6bfa23add415990b443fc232b4ba0 | 0 ...l_view-10-f26c10c6495fc3e86319cd5815caab4f | 10 + ...l_view-11-6c6b1588ab62ad11aef3a1147a2c7874 | 10 + ...l_view-12-622f0dd66c8633307fe56ccf9015f430 | 0 ...l_view-13-2c71748cfcdb9cc773d9ee61ae508b91 | 0 ...l_view-14-f26c10c6495fc3e86319cd5815caab4f | 10 + ...l_view-15-6c6b1588ab62ad11aef3a1147a2c7874 | 6 + ...l_view-16-33963ba7aefb7ab9a25afd540ecbbe98 | 4 + ...l_view-17-3219cf4fab8bf343bd273bd3a681fa46 | 0 ...l_view-18-bfe7dcb00f8b27e00d406de603635c8a | 0 ...l_view-19-f26c10c6495fc3e86319cd5815caab4f | 20 + ...al_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 | 0 ...l_view-20-6c6b1588ab62ad11aef3a1147a2c7874 | 20 + ...l_view-21-33963ba7aefb7ab9a25afd540ecbbe98 | 10 + ...l_view-22-283e0f384d0a015c252b34f79a895286 | 0 ...l_view-23-2f3b8b5fd961ee336d893cd45dc2696d | 0 ...l_view-24-4ed7df348bd8bf3fd275e15b6689c5df | 0 ...l_view-25-f26c10c6495fc3e86319cd5815caab4f | 10 + ...l_view-26-6c6b1588ab62ad11aef3a1147a2c7874 | 10 + ...l_view-27-33963ba7aefb7ab9a25afd540ecbbe98 | 6 + ...l_view-28-24e041343e158735db6262136de0e8b8 | 4 + ...al_view-3-a60af91a18d481fe1244e21108133489 | 0 ...al_view-4-67a95497041a0e81b4d5756731d8b27d | 0 ...al_view-5-b847c5dfb2f0395cfdd21f93de611b91 | 0 ...al_view-6-f26c10c6495fc3e86319cd5815caab4f | 20 + ...al_view-7-6c6b1588ab62ad11aef3a1147a2c7874 | 20 + ...al_view-8-a3eabeb4435c69ec4e242b8857c84e31 | 0 ...al_view-9-59102a6d824074da6cc85ca760385975 | 0 ...n_union-0-24ca942f094b14b92086305cc125e833 | 1 + ...n_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b | 0 ...n_union-2-97ea2263579f8d63dfdb1a6992fe0284 | 0 ...n_union-3-a164ae7bcb7c95beb8045b7064c9ab14 | 0 ...n_union-4-8c17422bd7041c596677251a46fa4085 | 0 ...n_union-5-9e89b8619411f46f675a9fab73298627 | 0 ...singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae | 0 ...inglemr-1-e67c221c0c19df7b3dc48b99426cf667 | 0 ...inglemr-2-31e7358a9f1e474e67155396fe803967 | 0 ...inglemr-3-d676a455ef6e9ec68756b0c1487dc3b1 | 0 ...inglemr-4-13bade1e01aed5d7e1af33a28a3b3574 | 0 ...inglemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 | 0 ...inglemr-6-80cb19f7a801a82add47a501756c8af7 | 0 ...inglemr-7-f8a4f52cd86ee679923248e09dd312e3 | 0 ...inglemr-8-985e6ff98c02a64d47b25817e9fef310 | 0 ...inglemr-9-faa99964430e2aee836221de1abe1c5e | 0 ..._complex-0-6a7c4841dab05ebae84309c9571bec6 | 0 ...complex-1-abac744dee1a4f4152781b1565fe1364 | 5 + ...complex-2-f07e9814229ed440bd0ac7e4bd924ce9 | 8 + ...complex-3-fb014bc1e9bfd919a3703cf5afb77448 | 0 ...complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d | 2 + ...virtual-0-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...virtual-1-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...irtual-10-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...irtual-11-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...irtual-12-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...irtual-13-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...irtual-14-67b834deba21676e02c155b25195a019 | 0 ...irtual-15-52f90e6bf3f2c17b82ed34318d2c612f | 0 ...irtual-16-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...irtual-17-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...virtual-2-67b834deba21676e02c155b25195a019 | 0 ...virtual-3-52f90e6bf3f2c17b82ed34318d2c612f | 0 ...virtual-4-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...virtual-5-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...virtual-6-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...virtual-7-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...virtual-8-67b834deba21676e02c155b25195a019 | 0 ...virtual-9-52f90e6bf3f2c17b82ed34318d2c612f | 0 ...newline-0-43392a20a8d249a279d50d96578e6a1b | 1 + ...newline-1-a19a19272149c732977c37e043910505 | 1 + .../newline-2-4eb54a664e549614d56ca088c8867d | 0 ...s_subq1-0-da2a1169620860b3dc558d21c6c5ea50 | 0 ...s_subq1-1-f91e87acd51f7477d96620b5f9deece6 | 84 + ...ble_col-0-535a12e87c72793bfba96520a0ea251b | 0 ...ble_col-1-8535e6c322e40f46b9a6e02fe6033ee0 | 0 ...ble_col-2-c2b97a35777322fe4a08d8c2216cb223 | 0 ...ble_col-3-29a7783c88e234ba32eaf1401ca8cc22 | 0 ...plicate-0-61c5ece0d210cf6158094f0f6fa24532 | 0 ...plicate-1-b1e2ade89ae898650f0be4f796d8947b | 1 + ...plicate-2-11ee085c0190a4007f410ddf96803ec0 | 0 ...plicate-3-a873d2d6991308b21ecdc46ac777c716 | 25 + ...plicate-4-24ca942f094b14b92086305cc125e833 | 1 + ...plicate-5-250d196b4449c835ddc518db2d2ab726 | 0 ...plicate-6-a873d2d6991308b21ecdc46ac777c716 | 25 + ...input37-0-6ed1b2ff177492c003161ee91e982c10 | 0 ...input37-1-e3ab2563222fb8678c7c269e09e1e8d8 | 0 ...t_into1-0-d3afbe52e3a1eb246a79c1320f82c480 | 0 ...t_into1-1-5c7fc72601c9add7bc86df7e4c24af63 | 0 ..._into1-10-3c8f9dff0a12ca2b929d04b4873a4681 | 0 ..._into1-11-6f59e35684a552a855e4dc3aee667092 | 1 + ..._into1-12-d3afbe52e3a1eb246a79c1320f82c480 | 0 ...t_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 | 0 ...t_into1-3-2983d09b973ea94bc701970a17fc3687 | 0 ...t_into1-4-a2f6c6c77e94833197095dc48643f9c9 | 1 + ...t_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 | 0 ...t_into1-6-2983d09b973ea94bc701970a17fc3687 | 0 ...t_into1-7-fe194a16b48b763e6efdf6fcc6116296 | 1 + ...t_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad | 1 + ...t_into1-9-461110270cec5b6d012b2c685cf2cce9 | 0 ..._alias1-0-695de796f21dce21056f8ba27cbadb0d | 0 ..._alias1-1-c4d51044e6802df266aa0dc137b825cd | 0 ..._alias1-2-bea59f268594a034d06b826c7a8e516e | 0 ..._alias1-3-adc1ec67836b26b60d8547c4996bfd8f | 57 + ..._alias2-0-695de796f21dce21056f8ba27cbadb0d | 0 ..._alias2-1-20b7bf01acc8d88670d347e6759aa407 | 0 ..._alias2-2-19c4fba994e06b28e0d912a6aa13ab78 | 0 ..._alias2-3-adc1ec67836b26b60d8547c4996bfd8f | 57 + ...ll_cast-0-48a80d4fdc8009234af31ebcb6e03364 | 0 ...ll_cast-1-7257e6f8170e545962d27741353f672c | 1 + ...llgroup-0-dbcec232623048c7748b708123e18bf0 | 1 + ...llgroup-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...lgroup-10-3994c1896dace613fa9f837b2f1676c0 | 0 ...lgroup-11-54a5fd76cdeff565c8c7724695aca302 | 1 + ...lgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...lgroup-13-85c4f90b754cd88147d6b74e17d22063 | 1 + ...lgroup-14-3994c1896dace613fa9f837b2f1676c0 | 0 ...lgroup-15-54a5fd76cdeff565c8c7724695aca302 | 1 + ...llgroup-2-3994c1896dace613fa9f837b2f1676c0 | 0 ...llgroup-3-54a5fd76cdeff565c8c7724695aca302 | 1 + ...llgroup-4-dbcec232623048c7748b708123e18bf0 | 1 + ...llgroup-5-85c4f90b754cd88147d6b74e17d22063 | 1 + ...llgroup-6-3994c1896dace613fa9f837b2f1676c0 | 0 ...llgroup-7-54a5fd76cdeff565c8c7724695aca302 | 1 + ...llgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...llgroup-9-c67a488530dc7e20a9e7acf02c14380f | 1 + ...lgroup2-0-dbcec232623048c7748b708123e18bf0 | 1 + ...lgroup2-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...group2-10-751fa56a198ad840868c76ad7ce2a6a0 | 0 ...group2-11-644b68261df70bbb46a3045a8abde17a | 0 ...group2-12-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...group2-13-85c4f90b754cd88147d6b74e17d22063 | 1 + ...group2-14-751fa56a198ad840868c76ad7ce2a6a0 | 0 ...group2-15-644b68261df70bbb46a3045a8abde17a | 0 ...lgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 | 0 ...lgroup2-3-644b68261df70bbb46a3045a8abde17a | 0 ...lgroup2-4-dbcec232623048c7748b708123e18bf0 | 1 + ...lgroup2-5-85c4f90b754cd88147d6b74e17d22063 | 1 + ...lgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 | 0 ...lgroup2-7-644b68261df70bbb46a3045a8abde17a | 0 ...lgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...lgroup2-9-c67a488530dc7e20a9e7acf02c14380f | 1 + ...lgroup3-0-d7ed4ae23442da095677d751a2b86c99 | 0 ...lgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 | 0 ...group3-10-a6455ec10891deb352bca6ccab2a8bf8 | 0 ...group3-11-d7ed4ae23442da095677d751a2b86c99 | 0 ...group3-12-a4d6d87d12f0e353d7a3ae5c087dab44 | 0 ...group3-13-f529994bef750d8edd02ede8a4323afb | 0 ...group3-14-d4e815f44f6369c991ea4390c481f31e | 0 ...group3-15-cd90e16da918bac569e9f04faaedd280 | 1 + ...group3-16-af767d7cfb4601ace72a3ed718071931 | 0 ...group3-17-45699aee54227552bb3ec84d92dfe450 | 0 ...group3-18-a24e178327b39f57ca7dfa2e69296bb2 | 0 ...group3-19-b8f60d35c1c09c57efd4c339799e5404 | 0 ...lgroup3-2-f529994bef750d8edd02ede8a4323afb | 0 ...group3-20-29fb8e3c12b8a705bc67d55a10566141 | 0 ...group3-21-e09c6bf2f6112981793fbd4386d43ff6 | 1 + ...lgroup3-3-d4e815f44f6369c991ea4390c481f31e | 0 ...lgroup3-4-cd90e16da918bac569e9f04faaedd280 | 1 + ...lgroup3-5-45699aee54227552bb3ec84d92dfe450 | 0 ...lgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 | 0 ...lgroup3-7-b8f60d35c1c09c57efd4c339799e5404 | 0 ...lgroup3-8-29fb8e3c12b8a705bc67d55a10566141 | 0 ...lgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 | 1 + ...lgroup4-0-dbcec232623048c7748b708123e18bf0 | 1 + ...lgroup4-1-c67a488530dc7e20a9e7acf02c14380f | 1 + ...group4-10-3a3f180144fead81b9e8b232b4132762 | 0 ...lgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 | 1 + ...group4-12-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...group4-13-85c4f90b754cd88147d6b74e17d22063 | 1 + ...group4-14-3a3f180144fead81b9e8b232b4132762 | 0 ...lgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 | 1 + ...lgroup4-2-3a3f180144fead81b9e8b232b4132762 | 0 ...llgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 | 1 + ...lgroup4-4-dbcec232623048c7748b708123e18bf0 | 1 + ...lgroup4-5-85c4f90b754cd88147d6b74e17d22063 | 1 + ...lgroup4-6-3a3f180144fead81b9e8b232b4132762 | 0 ...llgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 | 1 + ...lgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...lgroup4-9-c67a488530dc7e20a9e7acf02c14380f | 1 + ...istinct-0-dbcec232623048c7748b708123e18bf0 | 1 + ...istinct-1-85c4f90b754cd88147d6b74e17d22063 | 1 + ...istinct-2-43dec71d76c386394196a0e9d69457e5 | 0 ...istinct-3-c39b8f64123f56a05cdb0022a69f29b3 | 1 + ...istinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...istinct-5-85c4f90b754cd88147d6b74e17d22063 | 1 + ...istinct-6-43dec71d76c386394196a0e9d69457e5 | 0 ...istinct-7-c39b8f64123f56a05cdb0022a69f29b3 | 1 + ...lgroup5-0-d7ed4ae23442da095677d751a2b86c99 | 0 ...lgroup5-1-c75bafea030f127bce9a4a5dbb14f40b | 0 ...lgroup5-2-45699aee54227552bb3ec84d92dfe450 | 0 ...lgroup5-3-4492a9ce0d8502584b872860d53c449c | 0 ...lgroup5-4-14e2854b364b68e7526a52e6e8f7c94e | 0 ...lgroup5-5-60d7180632a63c79eeba47e30b854f4c | 500 ++++ ...llinput-0-2be9511f8cb3c9edaf50353187f24c11 | 0 ...llinput-1-bf48040d6cc6213cba90dbd76a796a66 | 0 ...llinput-2-61efe55ed8fef84e4cb8f9d7c317fabf | 0 ...linput2-0-40b73200f853b473855e85eba391f008 | 0 ...linput2-1-507c5a854d930361d8db66a3b90a5388 | 0 ...linput2-2-21058230c4992a682c4adef9881fa9a2 | 1 + ...lscript-0-4477b1566208a6805bb20630755a9375 | 0 ...lscript-1-3729d183a27e89a87ca9b9c9946e13a5 | 0 ...lscript-2-17238164053203d56d30704e2c098e80 | 0 ...lscript-3-56d47cf9468111b12fcef408bc767271 | 0 ...lscript-4-472199a0c6b8f760a90863deb69e9710 | 500 ++++ ...pe_conv-0-c3f17f31d887479896bf454a2f6b15dc | 0 ...pe_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 | 1 + ...l_outer-0-c157cc7014eda416b6248cf56165b62a | 0 ...l_outer-1-25aa8f4a1194304b0afc57a6b2c80205 | 0 ...l_outer-2-7871ab752b7065f13fb808c8d7cdb35f | 0 ...l_outer-3-4815a6c9014b80a79f61ab983dcdd23a | 0 ...l_outer-4-9519008c6ef6e8ed651c7f4405589c34 | 0 ...l_outer-5-4a868da4f3f6d96124bff2cf50a2673e | 0 ..._create-0-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ..._create-1-b5209e6f850fc958dc9ebced21519684 | 0 ...create-10-3027edcda6a6030c6bff93681529c34d | 0 ...create-11-14ea7dcc6898979aaa61650e3fc46187 | 32 + ...create-12-dd590639ea8d5b27ccf3312be2562fc2 | 0 ...create-13-14ea7dcc6898979aaa61650e3fc46187 | 34 + ...create-14-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ...create-15-e54b12bec275bc53ffe4b7252a99bab8 | 1 + ...create-16-3027edcda6a6030c6bff93681529c34d | 0 ...create-17-f42f66edf8a9f6ffdbe83608cc353beb | 1 + ...create-18-14ea7dcc6898979aaa61650e3fc46187 | 32 + ...create-19-a85f67347e5a19fc879ae83aa24c0ce4 | 0 ..._create-2-f375f322df98df65e6b1bd0d4ce8e208 | 0 ...create-20-176d469a0edba57404416535c7d48023 | 29 + ...create-21-e3accaf8ebc11bac7091db28d01fd2a7 | 0 ...create-22-31944ad765bbf262b4ebafdb06df59a6 | 3 + ...create-23-be779533ea8967231e644209114c8350 | 0 ...create-24-fe59ff341395bc347dfb9cfdee397da2 | 3 + ...create-25-c55e620d82921c40ebcdb94454ac1ead | 3 + ...create-26-aa4e0c8a3de340c1133d9fdddda8a18c | 3 + ..._create-27-2eefc0bb4283fc255409a5c41203c89 | 3 + ...create-28-cf8aa1014707dfe576820041e47436e2 | 3 + ...create-29-cee6b57822d79ce80e14e58663bf7b86 | 0 ..._create-3-8480c37b4f3a6768f459361cf8470dae | 0 ...create-30-b5e831bf4eb811ac4f06e9ac11a5283d | 0 ...create-31-c934a1ca351b76c5c4ef87cb5655f1d3 | 0 ...create-32-16367c381d4b189b3640c92511244bfe | 1 + ...create-33-2151d07a07ab6b28782b7e8e4f848e36 | 0 ...create-34-3b03210f94ec40db9ab02620645014d1 | 1 + ...create-35-a7bdd6d4d98ead017c08349ee53e9ac2 | 1 + ...create-36-ca15a32658195ecaffe11d11f487fb0a | 10 + ...create-37-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ...create-38-b5209e6f850fc958dc9ebced21519684 | 0 ...create-39-f375f322df98df65e6b1bd0d4ce8e208 | 0 ..._create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 | 0 ...create-40-8480c37b4f3a6768f459361cf8470dae | 0 ...create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 | 0 ..._create-5-4f8e2d1f18399679a97c49631c4ef921 | 0 ..._create-6-2ae0fd655aa777b41015e4125f680324 | 32 + ..._create-7-27aa4a8093e80a7437266f349ea927c0 | 0 ..._create-8-14ea7dcc6898979aaa61650e3fc46187 | 32 + ..._create-9-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ...reshold-0-a399c0ef0f1578f9d2456727008dee16 | 1 + ...reshold-1-a2f8227aafaee48079235d466c5049a0 | 0 ...reshold-2-939fd158d52e04da7ff5c13e74f51a8c | 0 ...hreshold-3-1a7f96f49e2992b93d97edcbdb0075d | 10 + ...reshold-4-f3df937417e80909d4a4b220779173b0 | 0 ...reshold-5-f5387ae613d41813186d7f1399545b02 | 0 ...reshold-6-272f5d299289829dc22cc31f70115dd9 | 0 ...reshold-7-acf39b28d4b76382acf5c56c21932ff9 | 1 + ...reshold-8-fe11a21ffbfa65e71ab62263077192a8 | 0 ...reshold-9-f7b722063a6948d22aaaab0707cddde1 | 1 + ...rt_cols-0-bfdd54175515a0557f8bd427ec23c453 | 0 ...rt_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...rt_cols-2-ab12575b9b663420d9b3702f5a32a086 | 0 ...rt_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 | 0 ...rt_cols-4-1c1eab8fc34159875afe38eb2413434e | 0 ...rt_cols-5-c0dce07a38f3c540c2b17db0a495a657 | 10 + ...y_files-0-eedb40290338ab680a930542cc7ddf0c | 0 ...y_files-1-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...ty_files-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...y_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...y_files-4-9c1451024d868c99833bbe1173c703d4 | 0 ...y_files-5-8c5d1eb83ebab33e284d70b11c4bc722 | 1 + ..._strings-0-a338239d2fc4e22e23ab82fa5f62139 | 0 ...strings-1-a67f0c90e3f99257a7364cc96e5405cf | 0 ...strings-2-a34bd419a63852c1c75f195a495ff333 | 9 + ...strings-3-3339ace17de3201296847caf29c42e99 | 0 ...strings-4-eee6b7f3a881c8f56510bbd53aeebe1e | 10 + ...th_nulls-0-a338239d2fc4e22e23ab82fa5f62139 | 0 ...h_nulls-1-2c16215823a5e6904059a48a3077da4e | 0 ...h_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 | 0 ...h_nulls-3-bbe8d353c397b795e2732bd59648d291 | 0 ...h_nulls-4-a0231a52d63534679bfab3f6a0b8effc | 0 ...h_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f | 5 + .../order-0-5c2fda577771db3b316e0d2bd02d048a | 0 .../order-1-57d93bd7619dfc460dfa763c12142bb9 | 10 + .../order-2-fe90320d98850ea5e9b6100f6d259fbf | 0 .../order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 | 10 + .../order2-0-ae225e86c2ae20519ffdf23190454161 | 1 + .../order2-1-484c94d6b57ccc9fffda150bee0bccdf | 0 .../order2-2-4162aa366dc0836eed15cc819226907f | 10 + ...oin_ppr-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...oin_ppr-1-1a374dec627d7109276f008f31be517c | 0 ...oin_ppr-2-b60ab17f7372863504804717c4276595 | 12 + ...oin_ppr-3-5924d0d5ab868c05f488ed741a1955ba | 0 ...join_ppr-4-be64a45a645ea5363109323a378d335 | 12 + ...arallel-0-23a4feaede17467a8cc26e4d86ec30f9 | 1 + ...arallel-1-851e262128626126ae1ad87869db7c54 | 1 + ...rallel-10-d196279de0b7c4b42521aa23634ca506 | 0 ...rallel-11-6230286bc168af7b010968b543690a2a | 309 ++ ...rallel-12-73a915d42e62c0e895a82602a502ee43 | 309 ++ ...arallel-2-43d53504df013e6b35f81811138a167a | 1 + ...arallel-3-195007722f0c2921f9c6c1eb461b6d2a | 0 ...arallel-4-fcf032bbe1f1569d934da7090db60a83 | 0 ...arallel-5-e27b511ffc5828b09069121031d17a2f | 0 ...arallel-6-d196279de0b7c4b42521aa23634ca506 | 0 ...arallel-7-6230286bc168af7b010968b543690a2a | 309 ++ ...arallel-8-73a915d42e62c0e895a82602a502ee43 | 309 ++ ...arallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...star_by-0-57a4ea931689f9475b687292f34abfa4 | 500 ++++ ...star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 | 500 ++++ ...star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc | 500 ++++ ...star_by-3-498e2973594ccf45448ba19552bfb1cd | 500 ++++ ...star_by-4-dc9d9d8bbb6259c89a97065902a345ec | 500 ++++ ...star_by-5-6888c7f7894910538d82eefa23443189 | 500 ++++ ...star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 | 500 ++++ ...star_by-7-f36c71c612ab8da4f4191980c1b61fea | 500 ++++ ...artInit-0-aaa3124841a8140a41d3556a4ccaa6a5 | 0 ...artInit-1-c0ec92801bec7ece0a156d407b601f7b | 1 + ...artInit-2-5db6fb2267a648ac57af8f56f91cf9a2 | 1 + ...artInit-3-878a82ca7519e3eafd3f2661b2ac1b88 | 0 ...artInit-4-5db6fb2267a648ac57af8f56f91cf9a2 | 1 + ...l_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 | 1 + ...l_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 | 0 ...bl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb | 0 ...l_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 32 + ...l_props-4-c04c695a6ebed215889ce75edcb33eb4 | 1 + ...s_empty-0-c367ba7f534037ab96efc7f2273508c7 | 1 + ...s_empty-1-d697ec36ecf73b0ee789972e6980e460 | 0 ...ps_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb | 0 ...s_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 30 + ...th_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 | 1 + ...th_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 | 0 ...ith_star-2-7bf3b5a09ed6cd06f27a0616de64ceb | 0 ...th_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 33 + ...th_star-4-c04c695a6ebed215889ce75edcb33eb4 | 1 + ...rtcols1-0-e234d14f9b1beb190c3c13d22ff02a0a | 0 ...rtcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 | 0 ...rtcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 | 10 + ...de_name-0-6ae3e0805cbfa967f50ad222bc33b772 | 0 ...de_name-1-4de8e998198c8df484e9102f60ba05c1 | 0 ...e_name-10-f1c90c8f4afd0c2ee66019e017997eb5 | 3 + ...e_name-11-94d637f6e5cee2771b9844438008a618 | 1 + ...de_name-2-a4fb8359a2179ec70777aad6366071b7 | 1 + ...de_name-3-16367c381d4b189b3640c92511244bfe | 1 + ...de_name-4-e90740a9a52c37a964ee204379f01412 | 1 + ...de_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...de_name-6-f1c90c8f4afd0c2ee66019e017997eb5 | 3 + ...de_name-7-94d637f6e5cee2771b9844438008a618 | 1 + ...de_name-8-bff58433eb2b500bb1d2f6ea495a5f20 | 1 + ...de_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c | 0 ...schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 | 0 ...schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 | 8 + ...schema1-3-fdef2e7e9e40868305d21c1b0df019bb | 0 ...schema1-4-9b756f83973c37236e72f6927b1c02d7 | 9 + ...schema1-5-52a518a4f7132598998c4f6781fd7634 | 8 + ..._format-0-65b98f7ed3ca5907e7ca5206de94939d | 0 ..._format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 | 0 ..._format-2-47b559b01e389cc9e327a2fd29255acf | 0 ..._format-3-54d18742b4eab85edd1946ef139771b4 | 20 + ...al_char-0-6ae3e0805cbfa967f50ad222bc33b772 | 0 ...al_char-1-4de8e998198c8df484e9102f60ba05c1 | 0 ...al_char-2-a4fb8359a2179ec70777aad6366071b7 | 1 + ...al_char-3-16367c381d4b189b3640c92511244bfe | 1 + ...al_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...al_char-5-f1c90c8f4afd0c2ee66019e017997eb5 | 3 + ...al_char-6-94d637f6e5cee2771b9844438008a618 | 1 + ...al_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...al_char-8-f1c90c8f4afd0c2ee66019e017997eb5 | 3 + ...al_char-9-94d637f6e5cee2771b9844438008a618 | 1 + ...e_check-0-b500f9a7ff3ef4ea3046cbaee22f434c | 1 + ...e_check-1-e676cfebd53fcc4c86407028dff8ddaa | 0 ...e_check-10-d51791c8b809ac86dc5b0f493a938fc | 0 ..._check-11-a1164f1770d2f787b520fbc3d345911a | 0 ...e_check-2-5e857e1536264658caf0df9b7e28652f | 0 ...e_check-3-7880e06d5cff6ea961dba5ff533cf346 | 6 + ...e_check-4-45fb706ff448da1fe609c7ff76a80d4d | 0 ...e_check-5-4a0b90a674d255ce00dd417dfefc46d4 | 0 ...e_check-6-1f5f161f5e992a6aa7358dcbbffc5939 | 0 ...e_check-7-7880e06d5cff6ea961dba5ff533cf346 | 6 + ...e_check-8-45fb706ff448da1fe609c7ff76a80d4d | 0 ...e_check-9-57b300095c52fd652e1d414724523587 | 0 ...archar1-0-ec359154c4bad3bfce1b7ab4914d1554 | 0 ...archar1-1-5064ea12af064b5e040d7b9ebd08b940 | 0 ...rchar1-10-1839df813809f21d8e0a0dd0006c7eb1 | 1 + ...archar1-11-83bf857460d5f49bf4bf6e480d9ce36 | 1 + ...rchar1-12-299a7b630adb14d0f6a8ea8f2e528489 | 1 + ...rchar1-13-4b9a64b41647af09b2f420c3b23b811c | 1 + ...rchar1-14-c07f977014280e92c02f24bb3ea0ec68 | 1 + ...rchar1-15-6442628796f33b87ef1ca11945fa4b48 | 1 + ...rchar1-16-3300093c71a371aed6225ffa9e05fc3d | 1 + ...rchar1-17-367ed375dd11bf57d5ab4288289a11d2 | 1 + ...rchar1-18-3df8ffe4c220764e59efb874bab97b9a | 1 + ...rchar1-19-ec359154c4bad3bfce1b7ab4914d1554 | 0 ...archar1-2-fc9381cdfb786fc3b7b9b35df6003474 | 0 ...archar1-3-968759281b7d1bf0a60991ed04953b93 | 0 ...archar1-4-deb9b7715610152bda285a3a33b772ef | 0 ...archar1-5-f77df9235261b945ca0a55e1afe2e5ce | 0 ...archar1-6-ad21f89ac813692cf47343c66e302ea6 | 2 + ...archar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e | 5 + ...archar1-8-6b968247a2c615af03fc6a386d72db9b | 1 + ...archar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 | 1 + ...etadata-0-c25482149887659ec66d7534cad22f63 | 0 ...etadata-1-29f193b3e8def419733366c578e8a236 | 0 ...etadata-2-14033db9e60aa0bc4f64376c96195adf | 0 ...etadata-3-9bdf636332ca722093413523366efa86 | 0 ...etadata-4-e784348b7221bb26830cc1ebda69bdde | 1000 +++++++ ...eformat-0-c854b607353e810be297d3159be30da4 | 0 ...eformat-1-9fad934993b77eef15a5d10eb203a378 | 0 ...format-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 15 + ...format-11-606ad10de7caf7e65e09778f2673e712 | 25 + ...format-12-c6bf6ee8fdc50f2abb3a71305d1e6882 | 25 + ...format-13-c262e8c736b67119b9806f69eb492ef3 | 50 + ...format-14-da1b1887eb530c7e9d37667b99c9793f | 0 ...format-15-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format-16-7d619408a560b5c8d4a06dcd0ee106e5 | 15 + ...format-17-a488cb92e1388a7542d8787a22f4fb55 | 15 + ...format-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 15 + ...format-19-e3b55daf170a032dcc8ed12ee26ccf63 | 15 + ...eformat-2-7d619408a560b5c8d4a06dcd0ee106e5 | 15 + ...format-20-606ad10de7caf7e65e09778f2673e712 | 25 + ...format-21-c6bf6ee8fdc50f2abb3a71305d1e6882 | 25 + ...format-22-c56c391659f5701a9d2e8782a60f7f8a | 25 + ...format-23-c262e8c736b67119b9806f69eb492ef3 | 75 + ...format-24-fe564b4f68241ec5c5884e44a1306f4f | 75 + ...eformat-3-a488cb92e1388a7542d8787a22f4fb55 | 15 + ...eformat-4-606ad10de7caf7e65e09778f2673e712 | 25 + ...eformat-5-c262e8c736b67119b9806f69eb492ef3 | 25 + ...eformat-6-6c4f7b115f18953dcc7710fa97287459 | 0 ...eformat-7-f5f427b174dca478c14eddc371c0025a | 0 ...eformat-8-7d619408a560b5c8d4a06dcd0ee106e5 | 15 + ...eformat-9-a488cb92e1388a7542d8787a22f4fb55 | 15 + ...ormat10-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...ormat10-1-198cb7d650a506ec3420b94b82a01375 | 0 ...ormat10-2-f723dedd396bd468107755b4495c1031 | 0 ...ormat10-3-c278fd699aa25809bdef310fb92f510e | 2 + ...ormat10-4-620cb6a290ef5297ac1d3a9ea776e2bf | 2 + ...ormat10-5-dd4c3f1636af9a7371edf7142abee088 | 0 ...ormat10-6-620cb6a290ef5297ac1d3a9ea776e2bf | 2 + ...ormat10-7-c278fd699aa25809bdef310fb92f510e | 2 + ...ormat11-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...ormat11-1-b30db33650de0545cbcd53769ed7cf40 | 0 ...rmat11-10-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...rmat11-11-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...ormat11-2-92bfcf88ca528eb6c9259142bf6541e5 | 0 ...ormat11-3-61f1abcdc66a64c11df85dded920d167 | 0 ...ormat11-4-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...ormat11-5-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...ormat11-6-ee7394c912991b8cd4401fb94942351f | 0 ...ormat11-7-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...ormat11-8-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...ormat11-9-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...ormat12-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...ormat12-1-b30db33650de0545cbcd53769ed7cf40 | 0 ...rmat12-10-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...rmat12-11-75a0aa2df39cbdc6a470b4c88803b42b | 4 + ...rmat12-12-3fc394a7a3d43a70968282ef1ee21dbd | 4 + ...rmat12-13-26a9a9cafa15d0223b934eba4777aea7 | 0 ...rmat12-14-d35f445501407d6fae5c3ad161fb2236 | 6 + ...rmat12-15-3fc394a7a3d43a70968282ef1ee21dbd | 6 + ...ormat12-2-92bfcf88ca528eb6c9259142bf6541e5 | 0 ...ormat12-3-61f1abcdc66a64c11df85dded920d167 | 0 ...ormat12-4-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...ormat12-5-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...ormat12-6-ee7394c912991b8cd4401fb94942351f | 0 ...ormat12-7-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...ormat12-8-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...ormat12-9-b8aed3dd7d7c151f5d96050c4058e71d | 0 ...ormat13-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...format13-1-9a812f44c4c7a4c72b4be95139e6fd6 | 0 ...rmat13-10-5bc619cec1d519c1dc6423f841b1c0a4 | 1 + ...ormat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 | 0 ...ormat13-3-9b3e4a236550f1900a36566830b24024 | 0 ...ormat13-4-49cf189a09e11c2c635fbb574b89a2af | 0 ...ormat13-5-2ebe5fe98e830143b9571b13fe59dc0b | 0 ...ormat13-6-fa2f56078df18add8e5a77e538756488 | 0 ...ormat13-7-1d822cc037d9281ce172e2d5685b1495 | 0 ...ormat13-8-e4531456a7418952ec1d69e18bc8460b | 0 ...ormat13-9-8f6983fda271fba18d218c75b8fb2b3d | 1 + ...ormat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...ormat14-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...rmat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 | 0 ...rmat14-11-bbfb0a95274044dea4732e35c1d7ecbe | 0 ...rmat14-12-be84e8199b0a3b9f72e552018854ac15 | 1 + ...rmat14-13-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...rmat14-14-b89ea2173180c8ae423d856f943e061f | 1 + ...rmat14-15-dd6e9965d271bd35604059540c23d967 | 1 + ...rmat14-16-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...rmat14-17-f4a3e39d5df18861e9fe67ef37af57e1 | 1 + ...rmat14-18-58080916a9f5883121bcaad719be0309 | 1 + ...format14-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...ormat14-3-2683f9835169a568c1f03dae859d27d2 | 0 ...ormat14-4-2d2e1149d2f035017bb37bbfaad4def0 | 0 ...ormat14-5-61a8225e20f36e286b4d02501d2c80d9 | 0 ...ormat14-6-b5165befb75ebeed42f3e69d4d64375c | 0 ...ormat14-7-e438f7ec386b2ab19660e9da7260dd95 | 0 ...ormat14-8-f0e3ddaa1e6ea067444b1f540bfac293 | 0 ...ormat14-9-adeaa033260f16c5bc106e761e5fef8b | 0 ...ormat15-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...ormat15-1-e1cf6c355de3ae8db7564b1676199117 | 0 ...rmat15-10-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...rmat15-11-662bece7f71ef854ca6e25f0eef5b830 | 0 ...rmat15-12-8250422b5ed16120ee33bd4fa69d3f47 | 3 + ...rmat15-13-3fc394a7a3d43a70968282ef1ee21dbd | 3 + ...ormat15-2-107d7c681b43611df056238be242127b | 0 ...ormat15-3-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...ormat15-4-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...ormat15-5-ee7394c912991b8cd4401fb94942351f | 0 ...ormat15-6-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...ormat15-7-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...ormat15-8-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...ormat15-9-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...ormat16-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...ormat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 | 0 ...rmat16-10-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...rmat16-11-662bece7f71ef854ca6e25f0eef5b830 | 0 ...rmat16-12-8250422b5ed16120ee33bd4fa69d3f47 | 3 + ...rmat16-13-3fc394a7a3d43a70968282ef1ee21dbd | 3 + ...ormat16-2-107d7c681b43611df056238be242127b | 0 ...ormat16-3-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...ormat16-4-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...ormat16-5-ee7394c912991b8cd4401fb94942351f | 0 ...ormat16-6-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...ormat16-7-3fc394a7a3d43a70968282ef1ee21dbd | 2 + ...ormat16-8-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...ormat16-9-75a0aa2df39cbdc6a470b4c88803b42b | 2 + ...ormat17-0-7c49277a7217a147685d30e27822d273 | 0 ...ormat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 | 1 + ...format2-0-c854b607353e810be297d3159be30da4 | 0 ...format2-1-9fad934993b77eef15a5d10eb203a378 | 0 ...ormat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d | 0 ...ormat2-11-20a02894f5e9340e89b55a30bef252b7 | 75 + ...format2-2-6c4f7b115f18953dcc7710fa97287459 | 0 ...format2-3-f5f427b174dca478c14eddc371c0025a | 0 ...format2-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...format2-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format2-6-32b36a6c3344c5fcca0ad6c93ffcab62 | 1 + ...format2-7-561ef0dbccfcbfbb0c75f33ebfd4203d | 0 ...format2-8-20a02894f5e9340e89b55a30bef252b7 | 75 + ...format2-9-50131c0ba7b7a6b65c789a5a8497bada | 1 + ...format3-0-c854b607353e810be297d3159be30da4 | 0 ...format3-1-6c4f7b115f18953dcc7710fa97287459 | 0 ...ormat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 | 25 + ...format3-2-f5f427b174dca478c14eddc371c0025a | 0 ...format3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 14 + ...format3-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...format3-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format3-6-e3b55daf170a032dcc8ed12ee26ccf63 | 14 + ...format3-7-c56c391659f5701a9d2e8782a60f7f8a | 25 + ...format3-8-f5f427b174dca478c14eddc371c0025a | 0 ...format3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 14 + ...format4-0-c854b607353e810be297d3159be30da4 | 0 ...format4-1-c561806d8f9ad419dc9b17ae995aab68 | 0 ...format4-2-b9f8c3b822051854770f61e5ae5b48b0 | 0 ...format4-3-9837451512e92e982f1bd9a12b132e84 | 0 ...format4-4-58cfa555b061057f559fc6b9c2f6c631 | 0 ...format4-5-ac79def5434bb8a926237d0db8db2e84 | 0 ...format5-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...format5-1-c854b607353e810be297d3159be30da4 | 0 ...format5-2-6c4f7b115f18953dcc7710fa97287459 | 0 ...format5-3-f5f427b174dca478c14eddc371c0025a | 0 ...format5-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...format5-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format5-6-a0e23b26ee1777ccc8947fb5eb1e8745 | 2 + ...format5-7-a0eeded14b3d337a74189a5d02c7a5ad | 0 ...format5-8-a0e23b26ee1777ccc8947fb5eb1e8745 | 3 + ...format6-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...format6-1-c854b607353e810be297d3159be30da4 | 0 ...format6-2-6c4f7b115f18953dcc7710fa97287459 | 0 ...format6-3-f5f427b174dca478c14eddc371c0025a | 0 ...format6-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...format6-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format6-6-e95296c9f7056b0075007c61d4e5e92f | 1 + ...format6-7-4758d41d052eba37a9acd90c2dbc58f0 | 1 + ...format7-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...format7-1-c854b607353e810be297d3159be30da4 | 0 ...format7-2-6c4f7b115f18953dcc7710fa97287459 | 0 ...format7-3-f5f427b174dca478c14eddc371c0025a | 0 ...format7-4-a34505bd397bb2a66e46408d1dfb6bf2 | 1 + ...format7-5-f2c42f1f32eb3cb300420fb36cbf2362 | 1 + ...format8-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...format8-1-22e3d59a0423473051535684bca72b27 | 0 ...format8-2-55ae9fbf6daa36225dd386e34025dd38 | 0 ...format8-3-c561806d8f9ad419dc9b17ae995aab68 | 0 ...format8-4-ae71ce67b5d4a91bce1b34acde830268 | 0 ...format8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e | 0 ...format8-6-ae6a5016b6a6ace6b6f2576a4c666b4e | 0 ...format8-7-d1a5913edaaec9654edb333e8207f57b | 20 + ...format8-8-624b059dfaa86e2c78f065169de325cd | 20 + ...format9-0-66ee62178e3576fb38cb09800cb610bf | 1 + ...format9-1-22e3d59a0423473051535684bca72b27 | 0 ...format9-2-55ae9fbf6daa36225dd386e34025dd38 | 0 ...format9-3-92bfcf88ca528eb6c9259142bf6541e5 | 0 ...format9-4-ae71ce67b5d4a91bce1b34acde830268 | 0 ...format9-5-d1a5913edaaec9654edb333e8207f57b | 20 + ...format9-6-624b059dfaa86e2c78f065169de325cd | 20 + ...an_json-0-74146da55d57b22443140e7fbab3375c | 0 .../ppd1-0-ae225e86c2ae20519ffdf23190454161 | 1 + .../ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + .../ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 | 0 .../ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 | 496 +++ .../ppd1-4-145c2779dadb5bd921dc2baac608b803 | 1 + .../ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 | 0 .../ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 | 496 +++ .../ppd2-0-ae225e86c2ae20519ffdf23190454161 | 1 + .../ppd2-1-145c2779dadb5bd921dc2baac608b803 | 1 + .../ppd2-2-4d5021216c5bc600a8c5344945f55a4 | 0 .../ppd2-3-fccdc7a24b626308d9ec17608f36564b | 136 + .../ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 | 0 .../ppd2-5-a2d5e5ec2504041ea1a62856c7086451 | 0 .../ppd2-6-ae225e86c2ae20519ffdf23190454161 | 1 + .../ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + .../ppd2-8-4d5021216c5bc600a8c5344945f55a4 | 0 .../ppd2-9-fccdc7a24b626308d9ec17608f36564b | 136 + ...usterby-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...usterby-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...sterby-10-62979aa9e6b4e6ffb44ec452aabbef65 | 1 + ...usterby-2-16681f9c2bdd44278817d72c138b6ee1 | 0 ...usterby-3-707a2295731e0d631a6c5f71c745c8d5 | 1 + ...usterby-4-a2d5e5ec2504041ea1a62856c7086451 | 0 ...usterby-5-62979aa9e6b4e6ffb44ec452aabbef65 | 1 + ...usterby-6-145c2779dadb5bd921dc2baac608b803 | 1 + ...usterby-7-16681f9c2bdd44278817d72c138b6ee1 | 0 ...usterby-8-707a2295731e0d631a6c5f71c745c8d5 | 1 + ...usterby-9-a2d5e5ec2504041ea1a62856c7086451 | 0 ...nt_expr-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...nt_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...nt_expr-2-2ab005b2ee21deedbe7d10904a034468 | 0 ...nt_expr-3-5b20725caf905c6674759fee7873d627 | 0 ...ant_expr-4-aded77ef8dced4717d919a949d109b0 | 0 ...nt_expr-5-7da8c74586c99d96254f6f031bcaeb1c | 25 + ...nt_expr-6-145c2779dadb5bd921dc2baac608b803 | 1 + ...nt_expr-7-5b20725caf905c6674759fee7873d627 | 0 ...ant_expr-8-aded77ef8dced4717d919a949d109b0 | 0 ...nt_expr-9-7da8c74586c99d96254f6f031bcaeb1c | 25 + ...t_where-0-345fa30edd72d631fee21c3beeeef3d9 | 0 ...t_where-1-84c951641740895ca1c8ddc098805da5 | 1 + ...ppd_gby-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...ppd_gby-2-fea67192aba8704a64a85da47f7f321e | 0 ...ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d | 129 + ...ppd_gby-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...ppd_gby-5-fea67192aba8704a64a85da47f7f321e | 0 ...ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d | 129 + ...pd_gby2-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...pd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...pd_gby2-2-25541db999d8c1d56ba36b63949b6073 | 0 ...ppd_gby2-3-321628d4c52f6992f2680a3a162f19f | 5 + ...pd_gby2-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...pd_gby2-5-25541db999d8c1d56ba36b63949b6073 | 0 ...ppd_gby2-6-321628d4c52f6992f2680a3a162f19f | 5 + ...by_join-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...by_join-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...by_join-2-45e102aabf0e90ac455e2bab3988d8c0 | 0 ...by_join-3-145c2779dadb5bd921dc2baac608b803 | 1 + ...by_join-4-45e102aabf0e90ac455e2bab3988d8c0 | 0 ...pd_join-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...pd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...pd_join-2-4c4dcd0c288653e39cfe077c19c68570 | 0 ...pd_join-3-aab5d092ca17ed23ee71d3a6ef653998 | 712 +++++ ...pd_join-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...pd_join-5-4c4dcd0c288653e39cfe077c19c68570 | 0 ...pd_join-6-aab5d092ca17ed23ee71d3a6ef653998 | 712 +++++ ...d_join2-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...d_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...d_join2-2-307ac4f1b13e310dc1c61c05a113945d | 0 ...d_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 | 1710 +++++++++++ ...d_join2-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...d_join2-5-307ac4f1b13e310dc1c61c05a113945d | 0 ...d_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 | 1710 +++++++++++ ...d_join3-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...d_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...d_join3-2-5018d137c74aed08cc4da4cbd1904092 | 0 ...d_join3-3-42cd793c031af1f7961d7b5e237de76b | 1767 +++++++++++ ...d_join3-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...d_join3-5-5018d137c74aed08cc4da4cbd1904092 | 0 ...d_join3-6-42cd793c031af1f7961d7b5e237de76b | 1767 +++++++++++ ..._filter-0-ae225e86c2ae20519ffdf23190454161 | 1 + ..._filter-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...filter-10-2c1710aa3e08f618c1930305ebdccc17 | 0 ...filter-11-41debade389a200de226d8ad0fe47d24 | 4 + ..._filter-12-3bcc51a124f8cbd456620853d808354 | 1 + ...filter-13-145c2779dadb5bd921dc2baac608b803 | 1 + ...filter-14-2c1710aa3e08f618c1930305ebdccc17 | 0 ...filter-15-41debade389a200de226d8ad0fe47d24 | 4 + ..._filter-2-2c1710aa3e08f618c1930305ebdccc17 | 0 ..._filter-3-41debade389a200de226d8ad0fe47d24 | 4 + ..._filter-4-ae225e86c2ae20519ffdf23190454161 | 1 + ..._filter-5-145c2779dadb5bd921dc2baac608b803 | 1 + ..._filter-6-2c1710aa3e08f618c1930305ebdccc17 | 0 ..._filter-7-41debade389a200de226d8ad0fe47d24 | 4 + ..._filter-8-73819ea1a7c0653a61652b3766afb003 | 1 + ..._filter-9-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...r_join1-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...r_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...r_join1-2-a239ac7d9503d93e0859043019f3e02e | 0 ...r_join1-3-9dfd333c14f82fd71c213e1d39c83524 | 6 + ...r_join1-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...r_join1-5-a239ac7d9503d93e0859043019f3e02e | 0 ...r_join1-6-9dfd333c14f82fd71c213e1d39c83524 | 6 + ...r_join2-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...r_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...r_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 | 0 ...r_join2-3-5340cd2d88dcf41dc18119389a475e36 | 6 + ...r_join2-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...r_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 | 0 ...r_join2-6-5340cd2d88dcf41dc18119389a475e36 | 6 + ...r_join3-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...r_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...r_join3-2-8f14853ac6ce5f40c98982ace9fa6221 | 0 ...r_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 | 6 + ...r_join3-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...r_join3-5-8f14853ac6ce5f40c98982ace9fa6221 | 0 ...r_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 | 6 + ...r_join4-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...r_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...r_join4-2-7b0e117314e5784ba06b7ab69e689c84 | 0 ...r_join4-3-ac57dc2a7151f20029d6a97049d6eebe | 10 + ...r_join4-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...r_join4-5-7b0e117314e5784ba06b7ab69e689c84 | 0 ...r_join4-6-ac57dc2a7151f20029d6a97049d6eebe | 10 + ...r_join5-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...r_join5-1-145c2779dadb5bd921dc2baac608b803 | 1 + ..._join5-10-b12aa87aaf64b573ce0be7013117651b | 0 ..._join5-11-c8e68d5a524b965c244f01782bc42e97 | 0 ...r_join5-12-fd1ebd8150ed40b4b91774f6db42df2 | 0 ...r_join5-2-5b859f606230a70698edf52ca814beee | 0 ...r_join5-3-2823683e3169487b80e882aa9c4e3bd6 | 0 ...r_join5-4-ab457e9f757065dbf6bf66a997ab76a0 | 0 ...r_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 | 0 ...r_join5-6-c669e5c7ae22c4fb96995c700ebadd9a | 0 ...r_join5-7-54d9885a797deb82236499f6bb99ddfd | 0 ...r_join5-8-e6949a27ee3c56243c81660de7d97f9c | 0 ...r_join5-9-ca16024e6f5399b1d035f5b9fd665163 | 0 ..._random-0-ae225e86c2ae20519ffdf23190454161 | 1 + ..._random-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ..._random-2-a589a2f51ebb3962202698c79db4a33b | 0 ..._random-3-145c2779dadb5bd921dc2baac608b803 | 1 + ..._random-4-a589a2f51ebb3962202698c79db4a33b | 0 ...d_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...d_alias-1-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...d_alias-2-588c0c8007b156167714d1cb06d10384 | 0 ...d_alias-3-106d8664d84c73f86268cf69e9ba6834 | 0 ...d_alias-4-2cd71603d825cddd0d181bd240c0051f | 0 ...d_alias-5-941a7af885ec77d91a78e03a6a568220 | 0 ...d_alias-6-c3746d419ceaab36dee78f4b4b38eb99 | 0 ...d_alias-7-54b70fb31202186b8984ae2feea299bf | 0 ...d_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...d_alias-9-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...ansform-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...ansform-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...ansform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 | 0 ...ansform-3-5d0e4236af589d3e63a8dd84e663b745 | 84 + ...ansform-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...ansform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 | 0 ...ansform-6-5d0e4236af589d3e63a8dd84e663b745 | 84 + ...df_case-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...df_case-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...df_case-2-c44d82e56d50653148bb4f9f98a42137 | 0 ...udf_col-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...df_col-10-53bbac80bbacf419ea971ddbb0f48542 | 0 ...udf_col-2-1c26c857485a928462c79e1b38ed1d9e | 0 ...udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed | 0 ...udf_col-4-c601016163c99a115feaca5fe5b74aae | 0 ...udf_col-5-53bbac80bbacf419ea971ddbb0f48542 | 0 ...udf_col-6-145c2779dadb5bd921dc2baac608b803 | 1 + ...udf_col-7-1c26c857485a928462c79e1b38ed1d9e | 0 ...udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed | 0 ...udf_col-9-c601016163c99a115feaca5fe5b74aae | 0 ...d_union-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...d_union-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 + ...d_union-2-fbfb76b9e6f2af48dcd227af03dffa89 | 0 ...d_union-3-678c8197f458b459171c266f7431683e | 171 ++ ...d_union-4-145c2779dadb5bd921dc2baac608b803 | 1 + ...d_union-5-fbfb76b9e6f2af48dcd227af03dffa89 | 0 ...d_union-6-678c8197f458b459171c266f7431683e | 171 ++ ...on_view-0-d680c8ac7e5121c6971458a9022c00b9 | 0 ...on_view-1-bfb5ca0dd2d5f070ce359790f8b91480 | 0 ...n_view-10-a74a5ff32204d842175b3d880477178f | 0 ...n_view-11-745e750f4f4a36af27e87338a979240c | 0 ...on_view-2-ac54e78582e6bd5d5533a0f3bfb51369 | 0 ...on_view-3-f6fd9a5271e172e4a65ea57aaa41d512 | 0 ...on_view-4-19cd3ea4e274befd809e4aad21da8d50 | 0 ...on_view-5-443c0979b586f6a6dfb0dc5d28cd5124 | 0 ...on_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 | 0 ...on_view-7-aab9d79f65d6edfc1cae88a14b8e106c | 0 ...on_view-8-e3ee46daeac605b05c7ada97b3c43578 | 0 ...on_view-9-8a77c30b6b07717d1c0bee971fb0132c | 0 .../ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 | 0 .../ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 | 36 + .../ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 | 0 .../ppd_vc-3-be78760949abf728580442e9e37a3ce7 | 36 + ...arenull-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...arenull-1-965b997838e7cc654cac68baef1be41d | 0 ...arenull-2-22b7886d627e6266f5f1415ba239c621 | 0 ...arenull-3-73819ea1a7c0653a61652b3766afb003 | 1 + ...arenull-4-965b997838e7cc654cac68baef1be41d | 0 ...arenull-5-22b7886d627e6266f5f1415ba239c621 | 0 ...ushdown-0-855b235f9c760ba9d6d0200bfd1ad08b | 0 ...ushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 | 0 ...shdown-10-178be64f122542983ff4384df4bb1530 | 0 ...shdown-11-3828393aa33a55cf1aea707e1be0a452 | 0 ...shdown-12-5affd35b94b0de3fb2a43f8729526055 | 0 ...shdown-13-2c64f5abe8c23957d7f0602e9c257cd1 | 0 ...shdown-14-c3188230751166d9d90082cd357ecb0c | 0 ...shdown-15-e52a39743f82af47902341a7ecd38afb | 0 ...shdown-16-70bef3ba441873327e188ff2ec6d68ae | 0 ...shdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 | 2 + ...shdown-18-ff36e3978571ac05f11e8322c024e4b6 | 2 + ...shdown-19-481005cf034ef3d7b998da32eb82aa9a | 2 + ...ushdown-2-5eede4874e678021938683fc2f5dc900 | 0 ...shdown-20-9073825e8b9804331f780980d1f9fa92 | 2 + ...shdown-21-9cb28f0d3a434c9d1ab039192906ec9d | 2 + ...shdown-22-678f50025924fee7b59f66e2abdb472d | 2 + ...shdown-23-4d5bfa800ba434f464b07bf9b5d39f20 | 2 + ...shdown-24-22663f09ea1c1bc303756067e84df5a7 | 2 + ...shdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b | 0 ...shdown-26-8065c18e387fd8bffae19a80af8dc1d4 | 1 + ...shdown-27-b72de558c88ae91460989938000e0d27 | 1 + ...shdown-28-7217735d357770af4ffee730e4e9add4 | 1 + ...shdown-29-630e2f7918b7727fc4ca057fa21e2eea | 1 + ...ushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 | 0 ...shdown-30-2e062414293b643ea4d7f6df92f939e4 | 1 + ...shdown-31-5eba4bf10315099129eae319d73636cf | 1 + ...shdown-32-35af0585a4f98bc222c786688cb6de6b | 1 + ...shdown-33-a5594625510703427ab8bae5d0563f73 | 1 + ...ushdown-4-f54bebec398f0fdfdbc0393123dba234 | 0 ...pushdown-5-78af12432bcbf99d4a0d41c25f964de | 0 ...ushdown-6-58724fbe96a0b3deceef20c8cc5e318d | 0 ...ushdown-7-14570f946e75924d7926c809485951d1 | 0 ...ushdown-8-3ba325662296fc455f07f1c835495e4c | 0 ...pushdown-9-4375f17bed264d5554a67d119fa5dd1 | 0 ...shdown2-0-855b235f9c760ba9d6d0200bfd1ad08b | 0 ...shdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 | 0 ...hdown2-10-ab3e46183487096745d7d90e3020e94c | 0 ...hdown2-11-680316eba447eb4649530fdc1c37d95b | 0 ...hdown2-12-720582e599a974ee9ca46b653678a14a | 0 ...hdown2-13-bd9067aeee8884a32db921b6d478f806 | 0 ...hdown2-14-8d01597374157f2d3d066840983ba1f8 | 1 + ...shdown2-15-5614065e1b8e709f68be4fa67666f41 | 1 + ...hdown2-16-3a67618e47c977f58c9dd8f4b9a576eb | 1 + ...hdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe | 2 + ...hdown2-18-e2c7e9b01ec95dfcc685827e24d66775 | 3 + ...hdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 | 1 + ...shdown2-2-efd3e4c333d4efb81218df8921e58f9e | 0 ...hdown2-20-ece6fe0efc1e658b36ddc10f0653d229 | 1 + ...shdown2-3-1886675984606b6c972c4a26dca6fd2c | 1 + ...shdown2-4-33b137b28e7246ec3c2acb937c638910 | 1 + ...shdown2-5-b5a2518af801f95fe52a75dfc1d3e867 | 0 ...shdown2-6-96059d8042ccb2ce355541daf9893954 | 0 ...shdown2-7-93922fe8d88643e1b8aa7e3ef4195404 | 0 ...shdown2-8-c86940e70f705f50e9091c257ee2bb40 | 1 + ...shdown2-9-4fa4ba4c739b6f44975d41e4705d4389 | 1 + ...shdown3-0-df2401785dfa257de49c3ad80b0f480a | 1 + ...shdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 | 0 ...shdown3-2-e879051803d0b64139e703e40fb007d0 | 40 + ...shdown3-3-c6fac1670fbd1ee490867efb1913a12d | 0 ...shdown3-4-a1c18483e5f5d2fe351be09af658acbb | 2000 +++++++++++++ ...shdown3-5-9b129023196c29071a92e91471872593 | 0 ...shdown3-6-67118904e6cc8e9f5210fab88e87fb8f | 2000 +++++++++++++ ..._header-0-860e298a0b70e7a531431e9386ddc0e7 | 1 + ..._header-1-8540676fc16ac91f3629c40f393a890a | 10 + ..._header-2-5cff10d4b561206e7e0b2e81d862ff93 | 10 + ..._header-3-e86d559aeb84a4cc017a103182c22bfb | 0 ...gress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 | 1 + ...gress_1-1-e93fbb4cb074376a645dc22075c0eab6 | 0 ...gress_1-2-70ba81c09588aa586e62ebaee2af685f | 0 ...gress_1-3-43d286eebddaee26cf95f26e62a75fe4 | 1 + ...ectmode-0-44765a9e389cd7be27e038d257ad2c61 | 0 ...ectmode-1-377270bf028b6a5e8a5a1679421aece6 | 0 ...ctmode-10-b993dfdb8636411e4e9182a5a1931362 | 0 ...ctmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 | 0 ...ectmode-12-d35a2c6ec13dcba7f4856522d2c781a | 0 ...ctmode-13-70e6f4e73e37e6288fafdf6b8da5c62c | 0 ...ctmode-14-f817d21ee3a7646bf107c87bb264586b | 0 ...ctmode-15-c51a200a345ceca9838507176a4a6dd6 | 0 ...ctmode-16-151259dfc1cea42392a7511991c58a5b | 0 ...ctmode-17-6fa385b2d2c6fee674dfff38f7338080 | 0 ...ctmode-18-968e78b685c3b83bf5eb8a3a786657a7 | 0 ...ctmode-19-b6e156f104768706aa587b762a9d4d18 | 9 + ...ectmode-2-f450722c47e95d5825d0cb920c212aea | 0 ...ctmode-20-719ec12056901bb0330fec1f05d4a94b | 0 ...ctmode-21-a31442a7d6c6950c137856ab861d622d | 9 + ...ctmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 | 0 ...ctmode-23-a31442a7d6c6950c137856ab861d622d | 9 + ...ctmode-24-2281fb42e04f3f3b1ae71e76feeb7733 | 0 ...ctmode-25-9095794bb72e75a50f84c7cc79b14df6 | 0 ...ctmode-26-a31442a7d6c6950c137856ab861d622d | 9 + ...ctmode-27-af002c5db76382ae7619968eaff3cc91 | 0 ...ctmode-28-a31442a7d6c6950c137856ab861d622d | 9 + ...ctmode-29-6fa385b2d2c6fee674dfff38f7338080 | 0 ...tectmode-3-4038060ccc6df535736b4b3cdf722cc | 0 ...ctmode-30-2cf1d601e4e37fa223cc841c0bf443f1 | 0 ...ctmode-31-97f50f7a49ee261db1888e93cf696c27 | 0 ...ctmode-32-b6e156f104768706aa587b762a9d4d18 | 9 + ...ctmode-33-151259dfc1cea42392a7511991c58a5b | 0 ...ctmode-34-520cc99451f889b26e12308b956f41e2 | 0 ...ctmode-35-67854bfbea7afeca2987ced73b68e671 | 0 ...ctmode-36-5b68bd2ad93372f1926166bb1eea7ae1 | 0 ...ctmode-37-8c07fbbf528fd9180db5a52474f6683e | 0 ...ctmode-38-da4a5ad43dbf3df8e31d23979af46423 | 0 ...ctmode-39-771f55b9c0dbadc2b8799cb4756c63e9 | 0 ...ectmode-4-b993dfdb8636411e4e9182a5a1931362 | 0 ...ctmode-40-39024332d020339bd9396c9b15403a0c | 0 ...ctmode-41-7643ff9b92368f2cc44ab8b1f5738d24 | 0 ...ctmode-42-3a8863ad6fac90ba48555c1042c92ebb | 0 ...ctmode-43-377270bf028b6a5e8a5a1679421aece6 | 0 ...ectmode-5-2af90060e2a14790441e0716e3372e7c | 0 ...ectmode-6-e2d1fc9c94e0d3597f393aec99850510 | 3 + ...ectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 | 0 ...ectmode-8-e2d1fc9c94e0d3597f393aec99850510 | 3 + ...tectmode-9-4038060ccc6df535736b4b3cdf722cc | 0 ...push_or-0-82eac35d076df1ee8020d57832f0f895 | 0 ...push_or-1-4c03e0f7fa028897c0caee62167eb68e | 0 ...push_or-2-62abd65e444981a96f7cf3dab3e013f1 | 0 ...push_or-3-34a713b9523589f1eb3205037d56a5b4 | 0 ...push_or-4-22888cdef1b94bde983541c1f3d510d9 | 0 ...push_or-5-c94def4b18b9c8c00e7a93eb19ec694a | 23 + ...th_semi-0-3731ce715b60549c17b7993927d48436 | 1 + ...th_semi-1-3f53ec3b276b32cf81729433e47010cb | 1 + ...th_semi-2-bf8cb175f9b13fcc23ba46be674b5767 | 1 + .../quote1-0-54ea0f3c04b185472a38a744d8003c13 | 0 .../quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 | 0 .../quote1-2-14d4949bc9215a1004c945885ef10de4 | 0 .../quote1-3-f36bdb38d0c1b79c6113fa0795d14533 | 0 .../quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 | 103 + .../quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 | 0 .../quote2-1-34f3c423b2fb1f0b11457f45a60042b9 | 1 + ...as.attr-0-97b3c408090f758257e4bd20597a525e | 1 + ...pruner1-0-a7e4414330751eb8ad486bb11643f64d | 0 ...pruner1-1-11cdebc422d7a0e7b257279ac9524321 | 56 + ...pruner2-0-b9598847d77e2c425423f51d755380e8 | 0 ...pruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 | 0 ...pruner2-2-db276de57ad86e8880037336886cd557 | 0 ...pruner2-3-e4419c33287ca1f48a43f61cca5b5928 | 116 + ...bigdata-0-e011be1172043c0c6d0fd2c0e89f361e | 1 + ...bigdata-1-6d0d6f4de136f56ab91987e19df8e178 | 1 + ...bigdata-2-3688b45adbdb190d58799c0b6d601055 | 1 + ...bigdata-3-fc0c054cdfbf5c130532e139d497866a | 0 ...olumnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 | 0 ...olumnar-1-5d636fc4682aee89328b7c7be17e1013 | 0 ...olumnar-2-b2f56374f2ea8a967b38e77b57138d4b | 2 + ...columnar-3-4653c703effa4cc65743128fa3c65ff | 10 + ...ompress-0-82f12a09df66752b05e89d3d12ba1e32 | 0 ...ompress-1-de09b3ac5ba6a77653eb361496863085 | 0 ...mpress-10-fb011a8ee13a82a873039bd28353205b | 5 + ...mpress-11-3708198aac609695b22e19e89306034c | 1 + ...ompress-12-3a4c24fd561f459025264baa3fb6d87 | 1 + ...ompress-2-bc4929594297fa2ee82b8ceff9118e46 | 5 + ...ompress-3-18a2fa22b8cef720cb30d75353b3f276 | 3 + ...ompress-4-fb011a8ee13a82a873039bd28353205b | 5 + ...ompress-5-68975193b30cb34102b380e647d8d5f4 | 1 + ...ompress-6-ea607fbed28d20e5726f4501285d698d | 1 + ...ompress-7-de09b3ac5ba6a77653eb361496863085 | 0 ...ompress-8-bc4929594297fa2ee82b8ceff9118e46 | 5 + ...ompress-9-18a2fa22b8cef720cb30d75353b3f276 | 3 + ..._merge1-0-d877ca1eefa4344eae30ad3ef2039b00 | 1 + ..._merge1-1-a4fb8359a2179ec70777aad6366071b7 | 1 + ...merge1-10-f94fdb0b79dcf73989e6fbce87355753 | 1 + ...merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 | 0 ...merge1-12-150cb190dc2343a747ea89298eb6352c | 0 ...merge1-13-af7b1e79eb40854ea82d61debaafcf40 | 1 + ..._merge1-14-4547f75ed3cb94914c2d025c1e057b5 | 1 + ...merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 | 0 ...merge1-16-c198c437f48c3844d8d0ace881b3107e | 0 ..._merge1-2-690b2898f94ef32f01ded0ddb737a056 | 1 + ..._merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee | 1 + ..._merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 | 0 ..._merge1-5-c198c437f48c3844d8d0ace881b3107e | 0 ..._merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 | 0 ..._merge1-7-8c76c8d41133f29366359c308b0c9cc0 | 0 ..._merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f | 0 ..._merge1-9-780b166bd3fcc932cb217273b0636d63 | 0 ..._merge2-0-f94fdb0b79dcf73989e6fbce87355753 | 1 + ..._merge2-1-a4fb8359a2179ec70777aad6366071b7 | 1 + ...merge2-10-5b4fb8038f522877101a4e429f082f11 | 0 ..._merge2-2-690b2898f94ef32f01ded0ddb737a056 | 1 + ..._merge2-3-778b659dba30ece782a956d87b1a31eb | 1 + ..._merge2-4-5b4fb8038f522877101a4e429f082f11 | 0 ..._merge2-5-78b97f09b49452e054cba2ae461813c3 | 0 ..._merge2-6-d5a91ca1ff9039b2dc55af8735d0249a | 0 ..._merge2-7-805fc662f9b3b7a587d1043b659e2424 | 0 ..._merge2-8-afd119f412a66bdf5cd1c74ae8965685 | 1 + ..._merge2-9-351c25ee0ca991ca9e0dbafa873b7592 | 1 + ..._merge3-0-f94fdb0b79dcf73989e6fbce87355753 | 1 + ..._merge3-1-690b2898f94ef32f01ded0ddb737a056 | 1 + ...merge3-10-1486feb77f13bb9a0ed606fe795ef686 | 0 ...merge3-11-7674266b00c52a2b6755910ea0793b69 | 1 + ...merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f | 1 + ...merge3-13-d57efd24ca36f282a37be5d0bf6452e6 | 0 ...merge3-14-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge3-2-778b659dba30ece782a956d87b1a31eb | 1 + ..._merge3-3-d57efd24ca36f282a37be5d0bf6452e6 | 0 ..._merge3-4-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 | 0 ..._merge3-6-e4d1c54cf6a802eef3342bec2505f69b | 0 ..._merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 | 0 ...e_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 | 0 ..._merge3-9-7c38c42af102069e08a85a27a735eb61 | 0 ..._merge4-0-f94fdb0b79dcf73989e6fbce87355753 | 1 + ..._merge4-1-690b2898f94ef32f01ded0ddb737a056 | 1 + ...merge4-10-1486feb77f13bb9a0ed606fe795ef686 | 0 ...merge4-11-7674266b00c52a2b6755910ea0793b69 | 1 + ...merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f | 1 + ...merge4-13-d57efd24ca36f282a37be5d0bf6452e6 | 0 ...merge4-14-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge4-2-778b659dba30ece782a956d87b1a31eb | 1 + ..._merge4-3-d57efd24ca36f282a37be5d0bf6452e6 | 0 ..._merge4-4-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge4-5-18f216bf1ea33debc2576ac85ac1a449 | 0 ..._merge4-6-3e292c7212dd02d7d6000837e09a42f9 | 0 ..._merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 | 0 ...e_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 | 0 ..._merge4-9-7c38c42af102069e08a85a27a735eb61 | 0 ...l_value-0-69edd47d7cca001865102e3fad925613 | 0 ...ll_value-1-38ac0f806892e3b25b0038c5f227c8d | 0 ...l_value-2-e721b8138774fdefca7171d1051841ee | 25 + ...l_value-3-3c82d1a57cf295901c6b931278a3760a | 0 ...l_value-4-8e839adf91e01e3230e15e9a936c9126 | 0 ...ll_value-5-d57d9c8882edbb3c6116d89516e4aed | 0 ...ll_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 | 9 + ...uptions-0-644898731e5228e863236f388ba1aa07 | 0 ...uptions-1-a9f718201b71fa855fb81a29e1d7bd3b | 1 + ...uptions-2-4a9a9175758ef576804c8b7309b019e8 | 1 + ...uptions-3-ea607fbed28d20e5726f4501285d698d | 1 + ...uptions-4-4df4ea1d1d30dedff582dc5c249dce85 | 0 ...uptions-5-6a3af12e36cec853c876a2cbae61c23a | 1 + ...uptions-6-718032defb864225dd664b1719f3b590 | 500 ++++ ...e_union-0-2a3aaaffda2fb4486871bed84c68fc60 | 0 ...e_union-1-ea555c16ad460f9a8d17b4e448d7c91b | 0 ...e_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 | 20 + ...ive_dir-0-fb096f0f4ecc530357ad76ae0353d338 | 0 ...plicate-0-43d53504df013e6b35f81811138a167a | 1 + ...plicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...licate-10-44d72569675c48e482ff53f45c7526e3 | 1 + ...licate-11-a9591dfbb3d8c6baa66fd9d56389bc81 | 0 ...licate-12-d6f20dc500809305e7454db01e24ce26 | 0 ...licate-13-1583df07fff7a2af0a9cc6a681446036 | 0 ...uplicate-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...plicate-3-45ca2bb223236cce4dcc7d7d7c96edbe | 1 + ...plicate-4-ac9d2110b999fb41f5f0f183742a8b27 | 1 + ...plicate-5-e9d5f1694b887c50ef17ba48545faa4d | 1 + ...plicate-6-890c5edd947ca276254ef4bd2813b296 | 0 ...plicate-7-e6ae500ce884875bd5f8e4e5200764e7 | 0 ...plicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 | 0 ...plicate-9-3b77920b01edb72aa3a7fc4f813e5d72 | 1 + ...ude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 | 0 ...ude_gby-1-a3f3e279ab0be5093f4a926e265c0211 | 1 + ...ude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...ude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 | 0 ...ude_gby-4-ca16024e6f5399b1d035f5b9fd665163 | 0 ...de_join-0-aa047b3a8b40b68b93c4ad11e173c767 | 1 + ...de_join-1-24ca942f094b14b92086305cc125e833 | 1 + ...de_join-2-f420affa96a8f13a91f6082805cf72a0 | 0 ...xtended-0-aa047b3a8b40b68b93c4ad11e173c767 | 1 + ...xtended-1-d0a93f40892e3894460553b443c77428 | 1 + ...tended-10-5a0cbc5744352a117afb34d1754496fa | 0 ...tended-11-9a4a8559680a45aad0a54144fb200f5e | 309 ++ ...tended-12-4c2ae1bc1c838240e1f355714ed8e785 | 309 ++ ...tended-13-ca02f80b104ee3dff445518501d0ee3e | 309 ++ ...tended-14-fd197d657a19ef5cdcb9cc7b5824035c | 309 ++ ...tended-15-dbc2b906b6cb99a138175eb4764da6bd | 15 + ...tended-16-22d9cdca0e70becd2ef86db59fc01ebb | 37 + ...tended-17-ffeb52326db68aafe4bb801ef6e7db5f | 309 ++ ...tended-18-4435a485aa7e3a72b789418e063d4cb8 | 309 ++ ...tended-19-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...xtended-2-dbcec232623048c7748b708123e18bf0 | 1 + ...tended-20-29dc99f23850e9f19368926d621a3029 | 0 ...tended-21-722d6027daf02b2e40f121a4fec362d2 | 0 ...tended-22-f3a586e75df55df355982eac0d4d4a63 | 0 ...tended-23-241faeae86e3e43e3ed8e35fb24e027d | 0 ...tended-24-dd3ed01a7ac656855ce76b4e6fcd27ed | 0 ...tended-25-bce36b45ce0c67af052bcf0487a53052 | 0 ...tended-26-93434017959c1be56608adff9309f02a | 0 ...tended-27-5a0cbc5744352a117afb34d1754496fa | 0 ...tended-28-9a4a8559680a45aad0a54144fb200f5e | 309 ++ ...tended-29-4c2ae1bc1c838240e1f355714ed8e785 | 309 ++ ...xtended-3-290a9fab18a45f29dd84ac3da0a47fb2 | 0 ...tended-30-ca02f80b104ee3dff445518501d0ee3e | 309 ++ ...tended-31-fd197d657a19ef5cdcb9cc7b5824035c | 309 ++ ...tended-32-dbc2b906b6cb99a138175eb4764da6bd | 15 + ...tended-33-22d9cdca0e70becd2ef86db59fc01ebb | 37 + ...tended-34-ffeb52326db68aafe4bb801ef6e7db5f | 309 ++ ...tended-35-4435a485aa7e3a72b789418e063d4cb8 | 309 ++ ...xtended-4-722d6027daf02b2e40f121a4fec362d2 | 0 ...xtended-5-f3a586e75df55df355982eac0d4d4a63 | 0 ...xtended-6-8801137a557caa5a75cfacd7aaf56bad | 0 ...xtended-7-7e350e31bce459f89b322b3356806174 | 0 ...xtended-8-bce36b45ce0c67af052bcf0487a53052 | 0 ...xtended-9-a94cab7395270afa10d73b5e8c2f4a71 | 0 ...k_dedup-0-d3f50875bd5dff172cf813fdb7d738eb | 0 ...k_dedup-1-dda16565b98926fc3587de937b9401c7 | 0 ...k_dedup-2-971f12c90619b02484c8e7db1b32f050 | 0 ...gex_col-0-ac78bd83c9aa538c2827598dd6007a69 | 0 ...gex_col-1-42751bfc3f1e83e7a014db9272d597db | 0 ...egex_col-2-21564f64cdfd46098e1254380490701 | 0 ...gex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 | 0 ...gex_col-4-daf9d3ca22b243870a138ba90d1593c4 | 0 ..._script-0-4e2e94d649b1792c5bd3fd879349ef92 | 1 + ..._script-1-e168f471980470d93b790702a70238fa | 1 + ..._script-2-a19a19272149c732977c37e043910505 | 1 + ...te_script-3-4eb54a664e549614d56ca088c8867d | 0 ..._column-0-f7eb4bd6f226be0c13117294be250271 | 0 ..._column-1-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-10-7ef160935cece55338bd4d52277b0203 | 0 ...column-11-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-12-379d54e3aa66daacff23c75007dfa008 | 0 ...column-13-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-14-25bfcf66698b12f82903f72f13fea4e6 | 0 ...column-15-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-16-d032f4795c1186255acea241387adf93 | 0 ...column-17-9c36cac1372650b703400c60dd29042c | 2 + ...column-18-fe4463a19f61099983f50bb51cfcd335 | 0 ...column-19-70b42434913b9d2eb17cd216c4f8039f | 0 ..._column-2-b2b2dfa681d01296fdacb4f56fb6db3a | 0 ...column-20-f7eb4bd6f226be0c13117294be250271 | 0 ...column-21-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-22-b2b2dfa681d01296fdacb4f56fb6db3a | 0 ...column-23-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-24-e4bf0dd372b886b2afcca5b2dc089409 | 0 ...column-25-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-26-89761e1c7afe3a5b9858f287cb808ccd | 0 ...column-27-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 | 0 ...column-29-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ..._column-3-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-30-7ef160935cece55338bd4d52277b0203 | 0 ...column-31-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-32-379d54e3aa66daacff23c75007dfa008 | 0 ...column-33-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-34-25bfcf66698b12f82903f72f13fea4e6 | 0 ...column-35-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...column-36-d032f4795c1186255acea241387adf93 | 0 ...column-37-9c36cac1372650b703400c60dd29042c | 0 ..._column-4-e4bf0dd372b886b2afcca5b2dc089409 | 0 ..._column-5-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ..._column-6-89761e1c7afe3a5b9858f287cb808ccd | 0 ..._column-7-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ..._column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 | 0 ..._column-9-6a3bbeb3203ce4df35275dccc4c4e37b | 3 + ...ocation-0-5c73d46fb91e9d4b3dc916622df09290 | 1 + ...oin_ppr-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...oin_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 | 0 ...oin_ppr-2-49b8b038ed8f5513405071c532967c47 | 12 + ...oin_ppr-3-a0b1014e173f410923de0d0446dfb4cc | 0 ...oin_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 | 12 + ...oin_ppr-5-89b4fc284984f51c12f15017707ad312 | 0 ...oin_ppr-6-2edf8e4ca99543089fdacbf506caa94f | 12 + ...oin_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 | 0 ...oin_ppr-8-b07ad21f47a74162f438abf579675f8e | 12 + ...sample2-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...sample2-1-a1d8184eab25b242a961533cc016efd1 | 0 ...nv_var1-0-16015162957e1d8e0ef586b44b276f64 | 2 + ...nv_var2-0-e5c4893b2ff412f9df0632328d404cef | 1 + ...nv_var2-1-81cb3e18ab89c533a253defff35e90f0 | 2 + ...pt_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 | 1 + ...pt_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 | 0 ...pt_pipe-2-3bf368261963ee10883e97c7746796b5 | 0 ...pt_pipe-3-afe5db164ccf986c5badd0655e009ea1 | 0 ...pt_pipe-4-7fe60c2fcca928a497252d99436b513f | 1 + ...ptfile1-0-43d53504df013e6b35f81811138a167a | 1 + ...ptfile1-1-89cf592f66b7276add70eb2c7689aa34 | 0 ...ptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 | 1 + ...omitted-0-39bd00e50d63ee4ff10427838591730a | 0 ...omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 | 1 + ...ote_and-0-7299c6986c06fc4a50cb543701527785 | 0 ...ote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 | 0 ...ote_and-2-8b4aac4293f2092c0661c26cb54b5619 | 0 ...ote_and-3-683007138a712792041ef3c8b84e914e | 1 + ...ote_and-4-8417a7b93132e20d468b7948bf46915a | 0 ...ote_and-5-1a5a68a098bfb9c93b76a458a9faf232 | 1 + ...ote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 | 0 ...ote_not-0-7299c6986c06fc4a50cb543701527785 | 0 ...ote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 | 0 ...ote_not-2-8b4aac4293f2092c0661c26cb54b5619 | 0 ...ote_not-3-683007138a712792041ef3c8b84e914e | 1 + ...ote_not-4-a6cbbf81715845f61af8fce977c0bcd5 | 0 ...ote_not-5-ceada0d3fa65bb2ec65154e797332cde | 1 + ...ote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 | 0 ...uote_or-0-7299c6986c06fc4a50cb543701527785 | 0 ...uote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 | 0 ...uote_or-2-8b4aac4293f2092c0661c26cb54b5619 | 0 ...uote_or-3-683007138a712792041ef3c8b84e914e | 1 + ...uote_or-4-954f604d17e36bbc08c04a0571d33045 | 0 ...uote_or-5-55cd874f705673f9de6ec8e3643c760f | 1 + ...uote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 | 0 ..._schema-0-765ac345199b944a6136273becf29469 | 0 ..._schema-1-e8c6de8cd50be582d78c9a8244cd92a1 | 11 + ..._schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d | 0 ..._schema-3-738e1d72a19c3db37ded87ca2fb148fa | 11 + ...ble_sub-0-b0b2ec9de0599f8dc2eb160299a2699d | 1 + ...ble_sub-1-e504c8259e45911765541f605c9a46f1 | 0 ...le_sub-10-bf5d1e710ce0974a67b4084aaa868f67 | 1 + ...ble_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 | 0 ...ble_sub-3-266170978f88a14c20c3944bfb55f5c7 | 1 + ...ble_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 | 1 + ...ble_sub-5-ee3a31bb9bb742f90daea98b290c34cd | 1 + ...ble_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a | 0 ...ble_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b | 1 + ...ble_sub-8-f321516e93eb0206453465a9b85cf67d | 1 + ...ble_sub-9-fbb54d457caeaafce723856429bbc0b2 | 1 + ..._quotes-0-65fee14fcf58502241f0772b21096780 | 1 + ..._quotes-1-26b98b2901556449d5431d731aaa642d | 1 + ..._quotes-2-f795383fcecedf7266cd7aed8283cec3 | 1 + ...c_quotes-3-7bc53505a4e6587132870d8d0a704d2 | 1 + ...nctions-0-45a7762c39f1b0f26f076220e2764043 | 192 ++ ...nctions-1-4a6f611305f58bdbafb2fd89ec62d797 | 17 + ...nctions-2-97cbada21ad9efda7ce9de5891deca7c | 32 + ...nctions-3-86945c60aed23626c43b507ee4ee6049 | 3 + ...nctions-4-4deaa213aff83575bbaf859f79bfdd48 | 4 + ...nctions-5-d6dad716f80a2e99ef9fc2e9782ef138 | 0 ...e_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 | 0 ...e_cases-1-3f42728fb0083beb962f07c43dd9c9b7 | 0 ..._cases-10-d759a63f08c878456c3401626f253ff5 | 0 ..._cases-11-afe66851d1cdc5d8a8a0d21c5705a59e | 0 ..._cases-12-f2dd38f0a56cd2137c9e1b870271550b | 16 + ..._cases-13-a9224a7a0012e407da67041bf680d490 | 4 + ..._cases-14-9d121385c5ab32d659dc7d0374ae8d6e | 16 + ..._cases-15-b032f4869c43d1278a890328d0591d5d | 0 ..._cases-16-8b2dc53c795584e1e23a8d631c82b43f | 0 ..._cases-17-c93fd07893f47b712165725c78d95555 | 0 ..._cases-18-9acff7f5096cdafc92aa2ddb0f296f83 | 0 ..._cases-19-3f42728fb0083beb962f07c43dd9c9b7 | 0 ...e_cases-2-e90c14f9edaeab8cf4540e9a35b11546 | 0 ...e_cases-3-aa5935155586821fb35e17156c8d8460 | 0 ...e_cases-4-6eb587e2751942de625c9229872ca0dc | 0 ...e_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 | 0 ...e_cases-6-ae97a64481efe733a19007ed400925bc | 0 ...e_cases-7-34016fb262ce768de45ec1b7693fd6c8 | 0 ...e_cases-8-16d39297488db165145e1546c4cb222c | 0 ...e_cases-9-fba02256147a1a753d40f56825449471 | 0 ..._syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec | 0 ..._syntax-1-fe3da04846e702e0bbe22e05b136a3b3 | 0 ...syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec | 0 ..._syntax-2-8207d7ca034ed62b9fb2c727497c22b3 | 0 ..._syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 | 0 ..._syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 | 16 + ..._syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 | 1 + ..._syntax-6-de64752733e0dcae32e692d2ad38e3d4 | 16 + ..._syntax-7-991839c8e50c5b4e490ec7faf2228d58 | 1 + ..._syntax-8-8c10f994f51bce851ecb0acee65ced7e | 16 + ..._syntax-9-f8385127c6405a2c9e48b5988184b515 | 4 + ...titions-0-73d0fdcea0bd9b828cbc3c2e88acb51a | 4 + ...titions-1-e94d4100cb64c67f1127b4e255d28ae0 | 2 + ...rtitions-2-a1bde7c2c040b4d45ddceac9983c2ca | 2 + ...titions-3-9e3f80cb6ed9883c715ea8aa3f391d70 | 1 + ..._tables-0-679cc07f8475a20b49927a5bbbd3d702 | 0 ..._tables-1-ac1c8cca812353544d3f7dead5d033ce | 0 ...tables-10-643b24446d74450c0f83144b1d0ec433 | 0 ...tables-11-3f9a7f993510123059493826470f78f7 | 0 ...tables-12-c31d2f4105ec3714cfc55eef68d3f60c | 3 + ...tables-13-f72d7ab6f3f04161ce2b8f8332244657 | 4 + ...tables-14-26ca9b5509544ebac182d8aa4264ff1c | 3 + ...tables-15-72a95694f749cb3d5760a69083e9cafe | 4 + ...tables-16-dfd802554b6764b3105b6fd8dbb8e173 | 0 ...tables-17-49777c49d2627373ed5e459c4848c9ab | 0 ...tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 | 0 ...tables-19-695a68c82308540eba1d0a04e032cf39 | 0 ..._tables-2-c96604d76bcb3721d5e5a327cac0d5e5 | 18 + ...tables-20-691b4e6664e6d435233ea4e8c3b585d5 | 0 ...tables-21-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...tables-22-274454ebeb7f98690a3c152617a0e391 | 1 + ..._tables-3-a56f51be557c6f636f99fafdbbbbcd9c | 2 + ..._tables-4-743d585ec3da5fbb927581cd0683ae35 | 18 + ..._tables-5-c685b358b604bf3ef980a78d9178d87d | 2 + ..._tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 | 0 ..._tables-7-a62fc229d241303bffb29b34ad125f8c | 0 ..._tables-8-691b4e6664e6d435233ea4e8c3b585d5 | 0 ..._tables-9-64c9bf0618541518f2ba30ec24a94423 | 0 ...estatus-0-4fa957197c8678b0a3a64d8f4f6da1fa | 14 + ...estatus-1-4c31924711bdb64603a14ce57da86ab7 | 14 + ...estatus-2-ecddce523f2af516700677a051581330 | 14 + ...estatus-3-dccfbc8b5a223a9fe47120ca771ee61d | 14 + ...estatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 | 0 ...lestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 | 0 ...estatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 | 0 ...estatus-7-f7b9148c16045269344c5d74fb8a449c | 0 ...estatus-8-1cd5acb4091d916e5e18948a39979b51 | 14 + ...estatus-9-e3cc1823000abb51d2635e0c824e81a9 | 14 + ...owparts-0-593619bb962b318b82896658deaea1f1 | 17 + ...owparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a | 4 + ...emove_1-0-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...emove_1-1-114600d46ae86edcb66a500b4cac657d | 1 + ...move_1-10-fa00cf008a039908eec64ad5dd415c5b | 0 ...move_1-11-4abc4f450a58ccdd0df2e345f1276979 | 0 ...move_1-12-dd683e148baed6b27642eebacaa87a4f | 0 ...move_1-13-43d53504df013e6b35f81811138a167a | 1 + ...emove_1-2-cafed8ca348b243372b9114910be1557 | 1 + ...remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_1-4-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...emove_1-5-dc129f70e75cd575ce8c0de288884523 | 1 + ...emove_1-6-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...emove_1-7-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_1-8-daf10744f465e055b35809a528135370 | 0 ...emove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...emove_2-0-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...emove_2-1-114600d46ae86edcb66a500b4cac657d | 1 + ...move_2-10-bebf0a312f3110d0b518153543030f06 | 0 ...move_2-11-4abc4f450a58ccdd0df2e345f1276979 | 0 ...move_2-12-fa53198d9eecb9d274b09d4351b9274e | 0 ...move_2-13-3fda17e4414d191f837631438a19e700 | 0 ...move_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 | 0 ...move_2-15-43d53504df013e6b35f81811138a167a | 1 + ...emove_2-2-cafed8ca348b243372b9114910be1557 | 1 + ...remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_2-4-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...emove_2-5-dc129f70e75cd575ce8c0de288884523 | 1 + ...emove_2-6-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...emove_2-7-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_2-8-c64266431d312784ebc7b9ca07ab5188 | 0 ...emove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt1-0-cafed8ca348b243372b9114910be1557 | 1 + ...oinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...oinopt1-2-114600d46ae86edcb66a500b4cac657d | 1 + ...oinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...oinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt1-5-fa00cf008a039908eec64ad5dd415c5b | 0 ...oinopt1-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt1-7-f66c3935651d3cc5fef7d0284e123614 | 0 ...inopt10-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt10-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...inopt10-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 | 0 ...inopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt10-5-9abe9cb11e3336a689a4116f8804c02a | 0 ...inopt10-6-bc7008e74f5eccab48d820877d83e7e4 | 0 ...inopt10-7-f4015c993efd5fc155e2faec784718d0 | 0 ...inopt10-8-c9624d1650d395f18d9e510cab4fab79 | 0 ...inopt11-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt11-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...inopt11-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt11-3-a079ede4f4245e62b02f624efedfb597 | 0 ...inopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt11-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt11-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt11-7-9e7e6016590d33c617cb568cbd45ef68 | 0 ...inopt12-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt12-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...inopt12-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt12-3-da45999e75a09b27161af2c7414c1170 | 0 ...inopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt12-5-e6db5d1ec4694475ab0b8f43eba415cd | 0 ...inopt12-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt12-7-8bbc680be8a68053db008789f335c2f0 | 0 ...inopt13-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt13-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...nopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 | 1 + ...inopt13-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt13-3-caf1c5fd299fdbdb655234d01d44caf2 | 0 ...inopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt13-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt13-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt13-7-c329b937ad1d7cf1c838640ef5f4d135 | 0 ...inopt13-8-3fda17e4414d191f837631438a19e700 | 0 ...inopt13-9-ab816627b2137d1850cf05b6f2b59e2d | 0 ...inopt14-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt14-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...inopt14-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt14-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...inopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt14-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt14-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt14-7-c329b937ad1d7cf1c838640ef5f4d135 | 0 ...inopt14-8-3fda17e4414d191f837631438a19e700 | 0 ...inopt14-9-cdf19a17f3295447b66e6e6445742a74 | 0 ...inopt15-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt15-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...nopt15-10-7df9fe6075a96bf9849848c93e449469 | 0 ...nopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 | 0 ...inopt15-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt15-3-32fed3a53e7c15b549a71c0e71d93484 | 0 ...inopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 | 0 ...inopt15-5-dff9d122eb83760f08d1d77814c24c91 | 0 ...inopt15-6-717b85f496a5cf006cb352f9d884608d | 0 ...oinopt15-7-ba43a86694107dd4cb754d676935408 | 0 ...inopt15-8-7381c1f36c997afac91d8f8f29e865f3 | 0 ...inopt15-9-ccb54093d518eaca1e4644686e9e776e | 0 ...inopt16-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt16-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...inopt16-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt16-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...inopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt16-5-fa00cf008a039908eec64ad5dd415c5b | 0 ...inopt16-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt16-7-b3208400a48223a410b76a4bd1362da5 | 0 ...inopt17-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt17-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...inopt17-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt17-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...inopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 | 0 ...inopt17-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt17-7-373b8a6363139ca37395b1cc8c9808d9 | 0 ...inopt18-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt18-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...nopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa | 6 + ...inopt18-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt18-3-32fed3a53e7c15b549a71c0e71d93484 | 0 ...inopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 | 0 ...inopt18-5-dff9d122eb83760f08d1d77814c24c91 | 0 ...inopt18-6-717b85f496a5cf006cb352f9d884608d | 0 ...inopt18-7-41ca600110e24166325d9426d974fff7 | 0 ...inopt18-8-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt18-9-ae751060e656d20aa1afbe3fac16d304 | 0 ...inopt19-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt19-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...inopt19-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt19-3-6eda8acf464a18cfd9909255ddcef37e | 0 ...inopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt19-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt19-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt19-7-1e9c17669089eea559f8fa0b6977b249 | 0 ...oinopt2-0-cafed8ca348b243372b9114910be1557 | 1 + ...oinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...oinopt2-2-114600d46ae86edcb66a500b4cac657d | 1 + ...oinopt2-3-52247b4dd98092bf829254e17424657d | 0 ...oinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt2-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt2-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 | 0 ...inopt20-0-cafed8ca348b243372b9114910be1557 | 1 + ...inopt20-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...inopt20-2-114600d46ae86edcb66a500b4cac657d | 1 + ...inopt20-3-4420d88e35aa84327bc95153eed299e0 | 0 ...inopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt20-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt20-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt20-7-e209254ae404366e6adca673d666aecb | 0 ...oinopt3-0-cafed8ca348b243372b9114910be1557 | 1 + ...oinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...oinopt3-2-114600d46ae86edcb66a500b4cac657d | 1 + ...oinopt3-3-c64266431d312784ebc7b9ca07ab5188 | 0 ...oinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt3-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt3-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt3-7-e54bbab48fcc3c41806a101293577e9f | 0 ...oinopt4-0-cafed8ca348b243372b9114910be1557 | 1 + ...oinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...oinopt4-2-114600d46ae86edcb66a500b4cac657d | 1 + ...oinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...oinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt4-5-744a018c78bae6e09853dd202981e850 | 0 ...oinopt4-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt4-7-c7e2ccaba57a581f27cfdcca76891133 | 0 ...oinopt5-0-cafed8ca348b243372b9114910be1557 | 1 + ...oinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...oinopt5-2-114600d46ae86edcb66a500b4cac657d | 1 + ...oinopt5-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...oinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt5-5-fa00cf008a039908eec64ad5dd415c5b | 0 ...oinopt5-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt5-7-f38878761b2aeeee0c04387dff60894d | 0 ...oinopt6-0-cafed8ca348b243372b9114910be1557 | 1 + ...oinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...oinopt6-2-114600d46ae86edcb66a500b4cac657d | 1 + ...oinopt6-3-da45999e75a09b27161af2c7414c1170 | 0 ...oinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd | 0 ...oinopt6-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 | 0 ...oinopt7-0-cafed8ca348b243372b9114910be1557 | 1 + ...oinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...oinopt7-2-114600d46ae86edcb66a500b4cac657d | 1 + ...oinopt7-3-c64266431d312784ebc7b9ca07ab5188 | 0 ...oinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt7-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt7-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt7-7-fa53198d9eecb9d274b09d4351b9274e | 0 ...oinopt7-8-3fda17e4414d191f837631438a19e700 | 0 ...joinopt7-9-b54d2a1f5d3bea81680ab06dead952c | 0 ...oinopt8-0-cafed8ca348b243372b9114910be1557 | 1 + ...oinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...oinopt8-2-114600d46ae86edcb66a500b4cac657d | 1 + ...oinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 | 0 ...oinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt8-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt8-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt8-7-fa53198d9eecb9d274b09d4351b9274e | 0 ...oinopt8-8-3fda17e4414d191f837631438a19e700 | 0 ...oinopt8-9-9b26e8e33d3109e059e7494b53aee6fd | 0 ...oinopt9-0-cafed8ca348b243372b9114910be1557 | 1 + ...oinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 + ...inopt9-10-5c8be465ceef8151f172b82a13e81fa8 | 4 + ...oinopt9-2-114600d46ae86edcb66a500b4cac657d | 1 + ...oinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...oinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt9-5-744a018c78bae6e09853dd202981e850 | 0 ...oinopt9-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt9-7-350dd4f731e2c4157f83c148c8569203 | 0 ...oinopt9-8-446c5e33062b109341add64a9860207d | 12 + ...oinopt9-9-83df7254049bc14de8aa53756850ceb5 | 0 ...rtesian-0-e3c7f62795b0e7c14b41b0dc29c47619 | 3 + ...pjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...pjoin_1-1-ef8d866d66ac801be99efef73664cbec | 0 ...join_1-10-b1114520feaa15ad7621b6a0e571c244 | 1 + ...pjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed | 0 ...join_1-12-8fb6fea440e845ce23b06deed8f519fa | 5 + ...pjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 | 0 ...join_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 | 4 + ...join_1-15-4d7cd71c68715335030bd41244a9cb2c | 0 ...join_1-16-c120e505c143721a36287bf992dbc1a1 | 8 + ...join_1-17-16c336c72785228fa1e6fcda01b20550 | 0 ...join_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 | 1 + ...join_1-19-9597d17924805ff4d34398d3a3d0a097 | 0 ...pjoin_1-2-31ed4593e624995154d0aedf94aa0f0f | 0 ...join_1-20-d7979e9ab355d8856c2d12e66e7bb838 | 5 + ...join_1-21-bce103f1a686915c28581717387cfeec | 0 ...join_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b | 4 + ...join_1-23-aef0ab90470007f2ae589ca5dddd02fd | 0 ...join_1-24-558e9ab6324f8082231b2fdd0e12f7ae | 8 + ...pjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...pjoin_1-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...pjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...pjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...pjoin_1-8-b89ea2173180c8ae423d856f943e061f | 1 + ...pjoin_1-9-17098316902a9e99455fa01bd3dae3ea | 0 ...join_10-0-a2d52f742b171d642bc7b8ec244791c1 | 0 ...join_10-1-9385271b1309ae1609b41d8a30dbef24 | 0 ...oin_10-10-189aeb9104e89114be759634d51c1a98 | 0 ...join_10-2-de26fe3669e18e6a170da561687fb514 | 0 ...join_10-3-bfb76fa2eccda1c64a85ea3841202705 | 0 ...join_10-4-d4746bf376bce0bf561450c75b83fb74 | 0 ...join_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 | 0 ...join_10-6-1094dbf800159e1e2382ec238b2466d7 | 0 ...join_10-7-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_10-8-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_10-9-b89ea2173180c8ae423d856f943e061f | 1 + ...join_11-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_11-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_11-10-3d92573eecd22525a75464ad27b1dcaa | 0 ...oin_11-11-685ec4fbbf73330e026fba9b8cc53e92 | 1 + ...oin_11-12-c05c09243793df14546e8577ee369d58 | 1 + ...oin_11-13-1e0f21b664c1940d10531b3025be7f10 | 0 ...oin_11-14-f13be826d8f11be64c5414d591f70fd6 | 0 ...oin_11-15-b62714cb184523454314d19949dba9f4 | 0 ...oin_11-16-bee943a85cd82bd199b089fbdd7450af | 1 + ...join_11-2-b89ea2173180c8ae423d856f943e061f | 1 + ...join_11-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...join_11-4-365488a703b0640acda73a7d7e6efa06 | 1 + ...pjoin_11-5-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_11-6-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_11-7-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_11-8-beae7266b997c97798631b9dc558534f | 0 ...join_11-9-74936eafc274242beb49bc83d7a8af30 | 0 ...join_12-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_12-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_12-10-bc8140f238cfd13ea4fc4c4173a05454 | 0 ...oin_12-11-685ec4fbbf73330e026fba9b8cc53e92 | 1 + ...oin_12-12-c05c09243793df14546e8577ee369d58 | 1 + ...join_12-13-7e7645d5ee7d79991d8fdde072c8dbb | 0 ...oin_12-14-e9715c66355e9bc54155c79a4e82b34f | 0 ...oin_12-15-42b623410c408e09153a773db91c0334 | 0 ...oin_12-16-bee943a85cd82bd199b089fbdd7450af | 1 + ...oin_12-17-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...oin_12-18-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_12-19-b89ea2173180c8ae423d856f943e061f | 1 + ...join_12-2-b89ea2173180c8ae423d856f943e061f | 1 + ...oin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b | 0 ...oin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e | 0 ...oin_12-22-b1e1754efd667aa801b194985d41eb6e | 1 + ...join_12-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...join_12-4-365488a703b0640acda73a7d7e6efa06 | 1 + ...pjoin_12-5-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_12-6-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_12-7-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_12-8-3d016b9a5b0143b7a01b34472b569fb9 | 0 ...join_12-9-74936eafc274242beb49bc83d7a8af30 | 0 ...join_13-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_13-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_13-10-b07488dd0aa9072057d8264271e93495 | 0 ...oin_13-11-c142d1f47152f247d4709b37780c46c4 | 0 ...oin_13-12-fb171dfe3de091a91e347f53cf68c355 | 0 ...oin_13-13-7b9c311217809cbb45a63be3c3e934ee | 0 ...oin_13-14-b92cb43f66838319f5d607c431fe1eb3 | 10 + ...oin_13-15-b80660184510c5a283b3c10dcddb9e69 | 0 ...oin_13-16-a05f3678364039a4cec1ced0dbda9884 | 0 ...join_13-2-b89ea2173180c8ae423d856f943e061f | 1 + ...join_13-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...join_13-4-365488a703b0640acda73a7d7e6efa06 | 1 + ...pjoin_13-5-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_13-6-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_13-7-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_13-8-721a75e4942a8ea97de6d70e441ce8a8 | 0 ...join_13-9-71b57a524e9ff17058ab319b18c606f6 | 0 ...join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...join_14-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...oin_14-10-fd4343f466f65fe84350a89ea501a8cd | 0 ...oin_14-11-23d2ee09b01309b9cfcd0d625afc535d | 1 + ...oin_14-12-2037b594916a80436d571a87d7b5f2e3 | 0 ...oin_14-13-95a09a0af2a77ac6b772c41a0f6a885a | 6 + ...oin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d | 0 ...oin_14-15-e0f20915e07acf5ddfdbde40ef924e55 | 1 + ...oin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 | 0 ...oin_14-17-5983d1e12e5a2bdd0f41549110e066ee | 1 + ...oin_14-18-c58871e5f7179cd903e24ad22198dea9 | 0 ...oin_14-19-163af33279f8d08d747a00ffd1cdfac7 | 1 + ...pjoin_14-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_14-20-b1651139605264c3b15178ae0543f7a | 0 ...oin_14-21-b6c3a3f68f212a966450286e23f59713 | 1 + ...join_14-22-f7839f8bcbb99d5731c9d4174c76b4b | 0 ...oin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 | 1 + ...oin_14-24-46db62b188fef9c6d636e63d64ef0c20 | 0 ...join_14-25-23f011143d8db18dd0f3d684adf7b8e | 1 + ...oin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b | 0 ...oin_14-27-ba437062dd6661fc2fdcd41514711093 | 1 + ...oin_14-28-db27758cc8bdb4ff80f346ad7e158da3 | 0 ...oin_14-29-d191c9ace84072ef430d7ed36ea8181b | 1 + ...join_14-3-4938d4d724990d16336ee31f0390c7da | 0 ...oin_14-30-377c21706143c1b714eeb0b3634343fb | 0 ...oin_14-31-fde2ad19052435bd8c155f18fe579070 | 1 + ...oin_14-32-edfe76bb21077fec096eb60333cec7f7 | 0 ...oin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 | 1 + ...join_14-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...join_14-5-9140b367b5680860f4c7c0238377583f | 0 ...join_14-6-70c1d66123d434d3a1e1801e0b19bc3f | 0 ...join_14-7-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_14-8-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_14-9-b89ea2173180c8ae423d856f943e061f | 1 + ...join_15-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_15-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_15-10-fb63a2ba41589835c01ba5f7570f643b | 0 ...oin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 | 0 ...oin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a | 10 + ...oin_15-13-5983712d9bdc150d8b1ffe23a555b370 | 0 ...oin_15-14-5455eb67634b186d7d8e54cdd32a9909 | 0 ...oin_15-15-268af28b787b5403a164447cbdbb0720 | 0 ...oin_15-16-98537b81d6e139521541753c3c448404 | 0 ...oin_15-17-c2312024b8de8d40faaf65427fa3d499 | 0 ...oin_15-18-7b1293b3066a5076e91b3d3dfd74259c | 0 ...oin_15-19-f49ef85423bb1766362f05651c9bb37f | 10 + ...join_15-2-b89ea2173180c8ae423d856f943e061f | 1 + ...oin_15-20-7577942157bcb84ad24c51306b046a44 | 0 ...oin_15-21-af3880637379684acd440830c2361f6e | 10 + ...oin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 | 0 ...oin_15-23-e5e54dd17b69773812af376bfec33200 | 10 + ...oin_15-24-5983712d9bdc150d8b1ffe23a555b370 | 0 ...oin_15-25-5455eb67634b186d7d8e54cdd32a9909 | 0 ...join_15-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...join_15-4-365488a703b0640acda73a7d7e6efa06 | 1 + ...pjoin_15-5-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_15-6-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_15-7-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_15-8-c19c008a41a546848f30a27237b124eb | 0 ...join_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 | 0 ...join_16-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_16-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_16-10-fb63a2ba41589835c01ba5f7570f643b | 0 ...oin_16-11-3c64de88fd6329a4346706ea296f7a4c | 0 ...oin_16-12-32f3716e22149e3d0c1f3ac26d414baf | 1 + ...join_16-2-b89ea2173180c8ae423d856f943e061f | 1 + ...join_16-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...join_16-4-365488a703b0640acda73a7d7e6efa06 | 1 + ...pjoin_16-5-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_16-6-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_16-7-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_16-8-14c221da1d1be2208aec9a641351ed09 | 0 ...join_16-9-6223202c291504ede983d8b04eafe08e | 0 ...join_17-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_17-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_17-10-516ad5af165e0e93b06b7541d35c96b8 | 0 ...join_17-11-c4daf31f90b01cfac9fb03b8b9de042 | 0 ...oin_17-12-cc5a32c1984378efafe0f05202359b20 | 0 ...oin_17-13-98da74d8dc8c7b722de5d2e25b481581 | 0 ...oin_17-14-672606a44ad31bacc472e7e2825d4fd3 | 0 ...oin_17-15-39292eb609c10f05f5c3dfacf520f485 | 0 ...oin_17-16-da60c2ca6af8844ed44778f4067f4f27 | 0 ...oin_17-17-bfc16d69eac4464e094e416842550590 | 0 ...oin_17-18-cf2846a07aab193ac997eae616373367 | 0 ...oin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 | 0 ...join_17-2-b89ea2173180c8ae423d856f943e061f | 1 + ...oin_17-20-50d120ef58fd6b737a01fccb3f212e9c | 0 ...oin_17-21-b1069c2d3f38f78fa5c5635a74b4466f | 0 ...oin_17-22-12d1838d39f395dbb212f322e83c49b0 | 0 ...oin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e | 0 ...oin_17-24-ce87a96826f6025541ef71fd88c3d370 | 0 ...oin_17-25-f066907fca3448b27aab623d05258a9a | 1 + ...oin_17-26-24ca942f094b14b92086305cc125e833 | 1 + ...oin_17-27-d0ec6d66ff349db09fd455eec149efdb | 1 + ...oin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 | 0 ...oin_17-29-65d3ae14a785f319658812b51e4620a3 | 1 + ...join_17-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...oin_17-30-59009edb2442f58edebf83c8d047cb1e | 0 ...oin_17-31-783fc1b07f117cd027395cf1c09149bc | 1 + ...oin_17-32-168e182e2c3feed13de0346cdeade2ff | 0 ...join_17-4-365488a703b0640acda73a7d7e6efa06 | 1 + ...pjoin_17-5-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_17-6-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_17-7-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_17-8-14c221da1d1be2208aec9a641351ed09 | 0 ...join_17-9-6223202c291504ede983d8b04eafe08e | 0 ...join_18-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_18-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e | 0 ...oin_18-11-fb15bd6eceb333302535c0bcdd2d078f | 0 ...oin_18-12-27762aa3d846e8f50d97350eaa7563a1 | 1 + ...oin_18-13-91f2af0da94e5a83601d02045980f556 | 1 + ...oin_18-14-1d0b88d6e6f84e485a05c712dd185531 | 1 + ...oin_18-15-21a6e09c24697160bf70cb17254ff504 | 1 + ...oin_18-16-35dfd037075aac5a9891bf99ea01e156 | 1 + ...oin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 1 + ...oin_18-18-ba77d1d4a1754489e2a8d5006b0af54d | 1 + ...oin_18-19-da6eb8ded1f72f518617339f58202cc5 | 1 + ...join_18-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...oin_18-20-940ff79d8f3b401a05b19b9417824d7a | 1 + ...oin_18-21-7ce17310f9fd362e2cc8a80211063264 | 1 + ...oin_18-22-a92f50cba6740abb3596c885a8157861 | 0 ...oin_18-23-f0675c53cddf75b012f64e797f5824c8 | 0 ...oin_18-24-a919b505c34a237ead34eea40b7d136c | 1 + ...join_18-25-d014ae435859316a1ad43548b72ecb7 | 1 + ...oin_18-26-52d3bbbbef7c580a324d991f52f44e1f | 1 + ...oin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 | 1 + ...oin_18-28-c83e75a3e18e68ef10d9970b3b8857ab | 1 + ...oin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 | 0 ...join_18-3-365488a703b0640acda73a7d7e6efa06 | 1 + ...oin_18-30-76e467313085467a3aa929b3665f9863 | 0 ...oin_18-31-7f1e9114f4db63982985068c4bf36a29 | 1 + ...oin_18-32-b59c406dae57fa6ab656a27e310ac54c | 1 + ...oin_18-33-95b42e61bf5eed67123e30d482e0c7fe | 1 + ...oin_18-34-14ae716c8c0f47db61b89a2a17e89415 | 1 + ...oin_18-35-549b3df453bf939e1c0a38df861245e5 | 1 + ...pjoin_18-4-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_18-5-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_18-6-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_18-7-c248759cecf0e2c223579f5e37f6669c | 0 ...join_18-8-724d37bd4a841f1fa4062f4f3e3eb353 | 0 ...join_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...join_19-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_19-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e | 0 ...oin_19-11-fb15bd6eceb333302535c0bcdd2d078f | 0 ...oin_19-12-27762aa3d846e8f50d97350eaa7563a1 | 1 + ...oin_19-13-4876f6e3c0ffde24afd82ec462962f19 | 1 + ...oin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 | 1 + ...oin_19-15-44ad799a82d847054f02d37139cc6aab | 1 + ...oin_19-16-8ee972ce0d73f28e98f46361082c83dc | 1 + ...oin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec | 1 + ...oin_19-18-b2546caccb501fca356930e6cd00ea2e | 1 + ...oin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 1 + ...join_19-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...oin_19-20-bceffa38b796fbc2a85daac23357da7b | 1 + ...oin_19-21-a1f4332461d1810334e7ae6d7d43f829 | 1 + ...oin_19-22-ebd323090f771227039cd21b1b8c4f3f | 1 + ...oin_19-23-90f0bad0606e63e4405eac9ec8c11deb | 1 + ...oin_19-24-3ae0277bb7a74cd6bee704313dc102e6 | 1 + ...oin_19-25-e621350131d50867015e75677cca031f | 1 + ...join_19-3-365488a703b0640acda73a7d7e6efa06 | 1 + ...pjoin_19-4-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_19-5-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_19-6-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_19-7-64b3fd0c215f4b8187866fa7eb55d34d | 0 ...join_19-8-ee04de92100063f9b2063baddb204864 | 0 ...join_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...pjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...pjoin_2-1-ef8d866d66ac801be99efef73664cbec | 0 ...join_2-10-1530d7565a22ace89ed16e0e1f9988ac | 7 + ...pjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 | 0 ...join_2-12-48e7d8fcb2a4c31c5304045517a3bb89 | 9 + ...join_2-13-596594a24f753a66ffd1c87dcd76781d | 0 ...join_2-14-3e7d435c7a5560abe539918adc4fa922 | 9 + ...join_2-15-2674b0d2498ac9392db40b4b83183d45 | 0 ...join_2-16-daeabb554f29b77f3c5ff7acff8c58ca | 11 + ...join_2-17-404d18544be565d251a6b0db1601d663 | 0 ...join_2-18-411bc909c1c29811d184d1f4aceb25b3 | 7 + ...join_2-19-95e94fc54d417dc08c7617642ee9de90 | 0 ...pjoin_2-2-31ed4593e624995154d0aedf94aa0f0f | 0 ...join_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b | 9 + ...join_2-21-b9609684ffdae8768578849392032ad5 | 0 ...pjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 | 9 + ...join_2-23-4dec46d167dbc1ac0bc9212e778d090a | 0 ...join_2-24-709966d157a75ffb1c6ef70d7c72a498 | 11 + ...pjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...pjoin_2-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...pjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...pjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...pjoin_2-8-b89ea2173180c8ae423d856f943e061f | 1 + ...pjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 | 0 ...join_20-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_20-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 | 0 ...oin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 | 0 ...oin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 1 + ...oin_20-13-ba77d1d4a1754489e2a8d5006b0af54d | 1 + ...oin_20-14-da6eb8ded1f72f518617339f58202cc5 | 1 + ...oin_20-15-5acb0ec2e1abcc6d57de7529f414a75a | 0 ...oin_20-16-5b9583aecebb7480d778dc2a8605434a | 0 ...oin_20-17-e26f212ca34d303036299ba709f65522 | 0 ...oin_20-18-7fb8778a58cc27dc04d5947df15b250e | 1 + ...oin_20-19-6dd859e98c140df728f858a7a7598462 | 1 + ...join_20-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...oin_20-20-fca1f7361c63f0ba1e9d365cf0743845 | 1 + ...oin_20-21-28d04e54c0a986079ae9d52788846519 | 1 + ...oin_20-22-1baa93dd15ab33240255c5d6d5d57366 | 1 + ...oin_20-23-5c5eaf9922e7110c6d7260c738b17457 | 0 ...join_20-3-365488a703b0640acda73a7d7e6efa06 | 1 + ...pjoin_20-4-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_20-5-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_20-6-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...pjoin_20-7-c9981ec081cbf54c9323e0dee977934 | 0 ...join_20-8-5c9994d48509136efd1dcb4e3f3e7aff | 0 ...join_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...join_21-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_21-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e | 0 ...oin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...oin_21-12-c3217069b040b92693a575bc077b6f78 | 0 ...oin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf | 0 ...oin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...oin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd | 0 ...join_21-16-c9250f30125e1235cb7ba01d97bbd1a | 0 ...oin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...oin_21-18-471c4e00fbbba3b23158aa4bf7595e79 | 0 ...join_21-19-c9250f30125e1235cb7ba01d97bbd1a | 0 ...join_21-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...oin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...oin_21-21-ee61731107316feaef2efca38bb43fab | 0 ...oin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 | 0 ...oin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...oin_21-24-c600a123838779df6a5babea49732659 | 0 ...oin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 | 0 ...join_21-3-365488a703b0640acda73a7d7e6efa06 | 1 + ...pjoin_21-4-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_21-5-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_21-6-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_21-7-c248759cecf0e2c223579f5e37f6669c | 0 ...join_21-8-724d37bd4a841f1fa4062f4f3e3eb353 | 0 ...join_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...join_22-0-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_22-1-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...oin_22-10-f0def0d77c93f6faebeca9b7a5340dbc | 0 ...oin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc | 0 ...oin_22-12-187c201f61c23833d0d193031926445a | 1 + ...oin_22-13-5c5c521954e5ade621fd0cbff5794c80 | 1 + ...join_22-14-b2534937758d2ff0a08e729f7b3ace4 | 1 + ...oin_22-15-83d9e3d23d6612a926e53d57a5d07307 | 1 + ...oin_22-16-abc95b36345d3e37abb196088897c7fe | 0 ...oin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...oin_22-18-2c29fc18b24368938f880a1bf3d5eb54 | 0 ...oin_22-19-5dc91a74bf4c43e20a8a213ad08c352e | 0 ...join_22-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...oin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad | 0 ...oin_22-21-f0def0d77c93f6faebeca9b7a5340dbc | 0 ...oin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc | 0 ...oin_22-23-187c201f61c23833d0d193031926445a | 1 + ...oin_22-24-5c5c521954e5ade621fd0cbff5794c80 | 1 + ...join_22-25-b2534937758d2ff0a08e729f7b3ace4 | 1 + ...oin_22-26-83d9e3d23d6612a926e53d57a5d07307 | 1 + ...join_22-3-365488a703b0640acda73a7d7e6efa06 | 1 + ...pjoin_22-4-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_22-5-dc129f70e75cd575ce8c0de288884523 | 1 + ...join_22-6-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...join_22-7-8317b719ffcf85da8c618e8f2379a31b | 0 ...join_22-8-8cfa26d100b454c0b0f3443f62389abb | 0 ...join_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad | 0 ...join_25-0-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ...join_25-1-365488a703b0640acda73a7d7e6efa06 | 1 + ...oin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...oin_25-11-3af09654f8d38d21b5d26e6cc21210de | 0 ...join_25-12-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...oin_25-13-182e8b532775f6e514195adbea3948a1 | 0 ...oin_25-14-c23ea191ee4d60c0a6252ce763b1beed | 1 + ...join_25-15-442e69416faaea9309bb8c2a3eb73ef | 1 + ...oin_25-16-beaea10543cfd753458b43d8aeb7571f | 1 + ...oin_25-17-24ca942f094b14b92086305cc125e833 | 1 + ...oin_25-18-d0ec6d66ff349db09fd455eec149efdb | 1 + ...oin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 | 1 + ...join_25-2-16367c381d4b189b3640c92511244bfe | 1 + ...oin_25-20-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...oin_25-21-4d3e60a0e9bd8c12fdba4e010493537d | 1 + ...oin_25-22-d0a93f40892e3894460553b443c77428 | 1 + ...join_25-23-ae45f6382268c1035c11aa4fc8a23e2 | 1 + ...oin_25-24-d5ad76f9178cb787cee037f25b19b270 | 1 + ...oin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 | 0 ...oin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 | 0 ...join_25-3-bcde511b8d560ca83888265b96a781ff | 1 + ...join_25-4-8067b7ab260021bc70af9ab47309ee35 | 1 + ...join_25-5-545c0ea2ebd7a141526ee13059a857f6 | 1 + ...join_25-6-dbcec232623048c7748b708123e18bf0 | 1 + ...join_25-7-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...join_25-8-ef8d866d66ac801be99efef73664cbec | 0 ...join_25-9-31ed4593e624995154d0aedf94aa0f0f | 0 ...pjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...pjoin_3-1-ef8d866d66ac801be99efef73664cbec | 0 ...pjoin_3-10-efadb45f09b92d27233601394d53d79 | 6 + ...join_3-11-52555bc44b4359aacfc42669bc5e4d12 | 0 ...join_3-12-77988b41de4b5a165c93942fbb2220d6 | 7 + ...join_3-13-812a0834a0e6546d5d0c8123d35c260f | 0 ...join_3-14-5456a3a43bfabcfdbb57257b633e299e | 10 + ...join_3-15-adc9940d6a104bca6ec791e57bdb09c0 | 0 ...join_3-16-fff1acc77150f6ea92fe8eefc04b079a | 11 + ...join_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe | 0 ...join_3-18-96a658e934543dd77020ad264ec9b8c1 | 6 + ...join_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 | 0 ...pjoin_3-2-31ed4593e624995154d0aedf94aa0f0f | 0 ...join_3-20-867845ed2cb38b55928f5310b4ae91bd | 7 + ...join_3-21-da48411b3e139037e171654fc9861755 | 0 ...join_3-22-5826f60da3a4b0c731c53340d50b0a06 | 10 + ...join_3-23-3e229e8219fe44b4a01f64b0967d7b31 | 0 ...join_3-24-c73176ea199d4d20934cf2659d42ec25 | 11 + ...pjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...pjoin_3-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...pjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...pjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...pjoin_3-8-b89ea2173180c8ae423d856f943e061f | 1 + ...pjoin_3-9-8928717d58becd4fd358f3fec65d60af | 0 ...pjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...pjoin_4-1-ef8d866d66ac801be99efef73664cbec | 0 ...join_4-10-73afe8a2334d1dbb7d4af749875e2f24 | 0 ...join_4-11-59fd75b48de9cbd23accc4c4414c6162 | 0 ...join_4-12-233250782bea3c7815e4c9949993d4f9 | 0 ...join_4-13-1ae07a8abc39f6a4195865ac583c1fb8 | 0 ...join_4-14-a6efb153fe9be91c7055a5642e0f642d | 5 + ...join_4-15-18da9766adcfcd53820d747d4b573886 | 0 ...join_4-16-514034935af5348f67f1934cf5429d57 | 6 + ...join_4-17-2a4723deed66d7aca3c272784dd11901 | 0 ...apjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 | 11 + ...join_4-19-afa62ebf8d41de52dfe732bdc470a995 | 0 ...pjoin_4-2-31ed4593e624995154d0aedf94aa0f0f | 0 ...join_4-20-be9350a00a523e1758427a1c8e4bdf09 | 6 + ...join_4-21-b4e83e2474975db8ed614e671af132f2 | 0 ...join_4-22-da023084bbab6727dc3823cfce500308 | 7 + ...join_4-23-48c28e1909386c95f78cb448bb6272e8 | 0 ...join_4-24-7783ab33a26d29a2c0235f52f0e4ad86 | 10 + ...join_4-25-c12edf424bd7b75fc06a21d6498b89d2 | 0 ...join_4-26-de14aa74d0da48dff6271410cc5dd98e | 11 + ...join_4-27-2617dc7a064cbb545c3ea54fe82c45ef | 0 ...pjoin_4-28-f71280b4b556515d068d074378c3a54 | 6 + ...join_4-29-239bc4053a7791815b587afef03515fb | 0 ...pjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_4-30-a0d3abda611809460bcea4101967f21f | 11 + ...join_4-31-35fa9389f1249c6cd736786522e6d3c5 | 0 ...join_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 | 10 + ...join_4-33-bc66eac0bf67a5c88f75ca66f0722e07 | 0 ...pjoin_4-34-48869ba51ea36864e720f94b09bf6b3 | 15 + ...pjoin_4-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...pjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...pjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...pjoin_4-8-b89ea2173180c8ae423d856f943e061f | 1 + ...pjoin_4-9-137cd40f999272507f72f2faa2fbd7db | 0 ...pjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...pjoin_5-1-ef8d866d66ac801be99efef73664cbec | 0 ...join_5-10-df63d18b19a05a728723ad01d4e526fc | 0 ...join_5-11-656085efab25db5dc019824b6b9a6bf5 | 0 ...join_5-12-a7a6dd05f89679cf3d621ca969a468e4 | 0 ...join_5-13-a1bada65e6df26b6e66681c565b41f2a | 0 ...join_5-14-e20bcd28cfd26259bcde9cffec8d2280 | 5 + ...pjoin_5-15-cff7ff4d03450148a945711bad28b34 | 0 ...join_5-16-8a8dc418de3716f5112b9868f472e4b9 | 6 + ...join_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f | 0 ...join_5-18-1f0dd26e6088c4f359fb691a8ef650bc | 11 + ...join_5-19-dd3d36a83baa3ac6155ad9b2c987657a | 0 ...pjoin_5-2-31ed4593e624995154d0aedf94aa0f0f | 0 ...join_5-20-18fe4e120d3c0d663c360b2504b4f88d | 6 + ...join_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 | 0 ...join_5-22-6ed2620fe017e454459a83061f25951a | 7 + ...join_5-23-e68f9dc71bb8d5bcaf24d11e21da572a | 0 ...join_5-24-87f758a20c15fa3b97b4ba6703ae541b | 10 + ...pjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 | 0 ...join_5-26-1c1a9519e207edc3c3a927c986a37177 | 11 + ...join_5-27-d29148ce40aca2915f54788d3890b6e2 | 0 ...join_5-28-94440444fff7d2629a23a30fd778fcc7 | 6 + ...join_5-29-8862326985836c72a2b21cb96a151579 | 0 ...pjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_5-30-574d26a5179e5cebbbff5bbb425a9609 | 11 + ...join_5-31-aa33e936a0d1a5ed534caef75647f0c3 | 0 ...pjoin_5-32-17c99f827824994cfd21c40dbf4abfc | 10 + ...join_5-33-487e89a8efb4dea08105f2fa79de5087 | 0 ...join_5-34-4db871582cf4f3038d43d0a2d5ae6895 | 15 + ...pjoin_5-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...pjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...pjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...pjoin_5-8-b89ea2173180c8ae423d856f943e061f | 1 + ...pjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df | 0 ...pjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...pjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...join_6-10-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_6-11-b89ea2173180c8ae423d856f943e061f | 1 + ...join_6-12-39e8889c37ee53d9a0ba48165b080b14 | 0 ...join_6-13-f9de7f57a4c89ccf84d35f47abb40d77 | 0 ...join_6-14-43123d2821871aa5b4d4a9e22e03d1ca | 1028 +++++++ ...join_6-15-c673bbec2ee476eea9609be3d3555438 | 0 ...join_6-16-449cb60e6537ba0810ea6879a7351a1e | 1 + ...join_6-17-aa5f237005fb69b3f52808c80b4a276a | 1 + ...join_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a | 0 ...join_6-19-4fd621581c51ad55a7e8389a94c6a411 | 0 ...apjoin_6-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_6-20-f9de7f57a4c89ccf84d35f47abb40d77 | 0 ...join_6-21-43123d2821871aa5b4d4a9e22e03d1ca | 1028 +++++++ ...join_6-22-c673bbec2ee476eea9609be3d3555438 | 0 ...join_6-23-449cb60e6537ba0810ea6879a7351a1e | 1 + ...join_6-24-aa5f237005fb69b3f52808c80b4a276a | 1 + ...join_6-25-e3552fabb4c9920d9bb964da5b907b46 | 0 ...join_6-26-5796487563f66866051a86f9c8cee822 | 0 ...pjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 | 0 ...join_6-28-9715967e9a70084a4600ccac53dfac60 | 0 ...join_6-29-495784f95b09cffa052805c6b9af9216 | 0 ...pjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 | 0 ...join_6-30-2cd20a399ce104eb7fb5881e59238196 | 0 ...pjoin_6-4-6eb37be25535293ca377a47c3e08bffa | 0 ...pjoin_6-5-ab8df8b65b35c17c45cac15468a4042c | 0 ...pjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 | 0 ...pjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 | 0 ...pjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 | 0 ...pjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...pjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...pjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...join_7-10-2e0e607c192151d31a6e6513fba79ef3 | 0 ...join_7-11-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_7-12-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_7-13-b89ea2173180c8ae423d856f943e061f | 1 + ...join_7-14-5e1546e0e05871eb1a6b8eb957252542 | 0 ...join_7-15-5e1546e0e05871eb1a6b8eb957252542 | 0 ...join_7-16-18d2a689883588ae3c24b2810663ab80 | 500 ++++ ...join_7-17-2ff32fe07d35a4d44b640f3ded5767fd | 0 ...pjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a | 0 ...join_7-19-ab46164010b27950c293f32fb89f9f8a | 500 ++++ ...apjoin_7-2-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 | 0 ...join_7-21-449cb60e6537ba0810ea6879a7351a1e | 1 + ...join_7-22-aa5f237005fb69b3f52808c80b4a276a | 1 + ...pjoin_7-23-1b6140f49d49eb1195c8551270b97ff | 1 + ...pjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa | 0 ...pjoin_7-4-85d6ffbe24f95df3128dee0e21a032de | 0 ...pjoin_7-5-ab8df8b65b35c17c45cac15468a4042c | 0 ...pjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 | 0 ...pjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 | 0 ...pjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb | 0 ...pjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f | 0 ...pjoin_8-0-43d53504df013e6b35f81811138a167a | 1 + ...pjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 + ...join_8-10-401c6c31bc3ae2cbd493a3855b829893 | 0 ...join_8-11-3b0f76816be2c1b18a2058027a19bc9f | 1 + ...join_8-12-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...join_8-13-b89ea2173180c8ae423d856f943e061f | 1 + ...pjoin_8-14-17529a2129bd485b814dde1ef894f06 | 0 ...join_8-15-343dfc1301b9f601ae03cb15f790102b | 0 ...join_8-16-e8f0853543694175095a8642e8f4fa03 | 0 ...join_8-17-5d2bab62264e462c007eb1a7a2c68866 | 0 ...pjoin_8-18-17529a2129bd485b814dde1ef894f06 | 0 ...join_8-19-343dfc1301b9f601ae03cb15f790102b | 0 ...pjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 | 1 + ...join_8-20-3af6a47b927b966f80a8f4c973b5f1f7 | 0 ...join_8-21-77ac2e23e8a8bc22274952933cd8d252 | 0 ...pjoin_8-22-17529a2129bd485b814dde1ef894f06 | 0 ...join_8-23-343dfc1301b9f601ae03cb15f790102b | 0 ...join_8-24-4e7e6b8eb897ed86e28d435c4f522e7c | 0 ...join_8-25-722aa3855f98f940fb34368c2789bcc3 | 0 ...pjoin_8-26-17529a2129bd485b814dde1ef894f06 | 0 ...join_8-27-343dfc1301b9f601ae03cb15f790102b | 0 ...join_8-28-4e7e6b8eb897ed86e28d435c4f522e7c | 0 ...join_8-29-722aa3855f98f940fb34368c2789bcc3 | 0 ...apjoin_8-3-7cccbdffc32975f8935eeba14a28147 | 1 + ...join_8-30-19b2b8c39155340929c605494826d30e | 0 ...join_8-31-3b6920dfb64f41b1b3c3c265275b1e4a | 0 ...join_8-32-4e7e6b8eb897ed86e28d435c4f522e7c | 0 ...join_8-33-722aa3855f98f940fb34368c2789bcc3 | 0 ...join_8-34-c8132556830d31a8803b07273aa1558f | 0 ...join_8-35-3b6920dfb64f41b1b3c3c265275b1e4a | 0 ...join_8-36-3af6a47b927b966f80a8f4c973b5f1f7 | 0 ...join_8-37-77ac2e23e8a8bc22274952933cd8d252 | 0 ...join_8-38-5ca28e78b53d14893518057c142f68a9 | 0 ...join_8-39-3b6920dfb64f41b1b3c3c265275b1e4a | 0 ...pjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 | 0 ...join_8-40-7aa01ceccf1bb45418145bdfc061206b | 0 ...join_8-41-8cf49829aada9587f04c9a5f334c433c | 0 ...join_8-42-d656a335f0f2f311717e2d72d0a9b6c6 | 0 ...join_8-43-3b6920dfb64f41b1b3c3c265275b1e4a | 0 ...join_8-44-559ecfd50bcc8c1bf63024c160bcad35 | 0 ...join_8-45-8cf49829aada9587f04c9a5f334c433c | 0 ...join_8-46-d656a335f0f2f311717e2d72d0a9b6c6 | 0 ...join_8-47-3b6920dfb64f41b1b3c3c265275b1e4a | 0 ...pjoin_8-5-a1f5562186e9e22e18ebd42208943525 | 0 ...pjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a | 0 ...pjoin_8-7-35e90a6bc292cfd33b48c52460937858 | 0 ...pjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a | 0 ...pjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 | 0 .../sort-0-eee35bbc888a705ae527625447668032 | 0 .../sort-1-10c8b0a592ed15eff488a71ec5909f45 | 500 ++++ ..._desc_1-0-6a35b44d4170109f641f8b68efef3719 | 0 ..._desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c | 0 ...desc_1-10-e4475ba09980280452542a0d595b8217 | 0 ...desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 | 1 + ..._desc_1-2-ffe97dc8c1df3195982e38263fbe8717 | 1 + ..._desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 | 0 ..._desc_1-4-a2e980fa0fbe8737cf256e21c784193e | 0 ..._desc_1-5-452fb932f6353effc9c458ec24dbeff9 | 0 ..._desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 | 0 ..._desc_1-7-3b0f76816be2c1b18a2058027a19bc9f | 1 + ..._desc_1-8-86473a0498e4361e4db0b4a22f2e8571 | 1 + ..._desc_1-9-b89ea2173180c8ae423d856f943e061f | 1 + ..._desc_2-0-6a35b44d4170109f641f8b68efef3719 | 0 ..._desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c | 0 ...desc_2-10-c0cac985e4115bf4014274b21222cac9 | 0 ...desc_2-11-47bc9cb5f049aaca33b394ea78578bdd | 1 + ..._desc_2-2-ffe97dc8c1df3195982e38263fbe8717 | 1 + ..._desc_2-3-a38a9b74e94a1eb032a52d905d863d46 | 0 ..._desc_2-4-ac10c36f76f1dcf25783bb796d182f64 | 0 ..._desc_2-5-452fb932f6353effc9c458ec24dbeff9 | 0 ..._desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 | 0 ..._desc_2-7-3b0f76816be2c1b18a2058027a19bc9f | 1 + ..._desc_2-8-86473a0498e4361e4db0b4a22f2e8571 | 1 + ..._desc_2-9-b89ea2173180c8ae423d856f943e061f | 1 + ..._desc_3-0-6a35b44d4170109f641f8b68efef3719 | 0 ..._desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c | 0 ...desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 | 0 ...desc_3-11-47bc9cb5f049aaca33b394ea78578bdd | 1 + ..._desc_3-2-ffe97dc8c1df3195982e38263fbe8717 | 1 + ..._desc_3-3-ce75a01507b158094bcf4441f3bfea95 | 0 ..._desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 | 0 ..._desc_3-5-452fb932f6353effc9c458ec24dbeff9 | 0 ..._desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 | 0 ..._desc_3-7-3b0f76816be2c1b18a2058027a19bc9f | 1 + ..._desc_3-8-86473a0498e4361e4db0b4a22f2e8571 | 1 + ..._desc_3-9-b89ea2173180c8ae423d856f943e061f | 1 + ..._desc_4-0-6a35b44d4170109f641f8b68efef3719 | 0 ..._desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c | 0 ...desc_4-10-99adddf7dacf8042707a9b36062fe103 | 0 ...desc_4-11-47bc9cb5f049aaca33b394ea78578bdd | 1 + ..._desc_4-2-ffe97dc8c1df3195982e38263fbe8717 | 1 + ..._desc_4-3-ce75a01507b158094bcf4441f3bfea95 | 0 ..._desc_4-4-ac10c36f76f1dcf25783bb796d182f64 | 0 ..._desc_4-5-452fb932f6353effc9c458ec24dbeff9 | 0 ..._desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 | 0 ..._desc_4-7-3b0f76816be2c1b18a2058027a19bc9f | 1 + ..._desc_4-8-86473a0498e4361e4db0b4a22f2e8571 | 1 + ..._desc_4-9-b89ea2173180c8ae423d856f943e061f | 1 + ..._desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ..._desc_5-1-365488a703b0640acda73a7d7e6efa06 | 1 + ...desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a | 1 + ..._desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb | 0 ..._desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d | 0 ..._desc_5-4-f392824f42188aac282046394108ef48 | 0 ..._desc_5-5-85d19fc31c2031491560208803fe63bf | 0 ..._desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 | 0 ..._desc_5-7-c23ea191ee4d60c0a6252ce763b1beed | 1 + ..._desc_5-8-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...n_desc_5-9-8b5783beaca02bba77d82adefe02c51 | 0 ..._desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ..._desc_6-1-365488a703b0640acda73a7d7e6efa06 | 1 + ...desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 | 0 ...desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a | 1 + ...n_desc_6-2-7cccbdffc32975f8935eeba14a28147 | 1 + ..._desc_6-3-d651a32232651880ad90428ada99358b | 0 ..._desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d | 0 ..._desc_6-5-9d2278807f7eef56292afd44b347ff64 | 0 ..._desc_6-6-85d19fc31c2031491560208803fe63bf | 0 ..._desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 | 0 ..._desc_6-8-c23ea191ee4d60c0a6252ce763b1beed | 1 + ..._desc_6-9-86473a0498e4361e4db0b4a22f2e8571 | 1 + ..._desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e | 1 + ..._desc_7-1-365488a703b0640acda73a7d7e6efa06 | 1 + ...desc_7-10-5918e837d2ae72cfaa753516c8652f24 | 0 ...desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 | 0 ...desc_7-12-c23ea191ee4d60c0a6252ce763b1beed | 1 + ...desc_7-13-86473a0498e4361e4db0b4a22f2e8571 | 1 + ...desc_7-14-afdad72caa2e23071817fe124da07763 | 0 ...desc_7-15-22330850dd2a9d18dedb504ea5879a07 | 1 + ...n_desc_7-2-7cccbdffc32975f8935eeba14a28147 | 1 + ..._desc_7-3-546c076289ceaf1fbee969aeb30b402b | 0 ..._desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d | 0 ..._desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 | 0 ..._desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 | 0 ..._desc_7-7-fecc7e5eef32fd38735e82d9d78a324f | 0 ..._desc_7-8-85d19fc31c2031491560208803fe63bf | 0 ..._desc_7-9-d1cfb738e07e78009bac4881c05853ec | 0 .../stats0-0-418ec894d08c33fd712eb358f579b7a0 | 1 + .../stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + ...stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c | 0 ...stats0-11-82294461be4728b4b191414bf2fb3bd7 | 0 .../stats0-12-31ebc4dcaa13c6dc5937c04f603e699 | 1 + ...stats0-13-ca0569fd4200d549843deadf84044649 | 500 ++++ ...stats0-14-4f18f4b06db06844920b14e2d19471a9 | 10 + ...stats0-15-4a7ed5b7e3deb4106204f8f950357e1c | 10 + ...stats0-16-b12e5c70d6d29757471b900b6160fa8a | 1 + ...stats0-17-593999fae618b6b38322bc9ae4e0c027 | 1 + ...stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 | 0 ...stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed | 0 .../stats0-2-dc129f70e75cd575ce8c0de288884523 | 1 + ...stats0-20-5f7439f97275da70f633e135205f2095 | 0 ...stats0-21-6bcc1360b3d777a8b51ae416ff43898a | 0 ...stats0-22-85de3349a12a9fa0536f20ad0377fca1 | 0 ...stats0-23-247568f4b3ce6b97979ca9d28c2ba05c | 4 + ...stats0-24-7f38972bcd18c477a027d881182d83f9 | 500 ++++ ...stats0-25-98c925a2b2c4de06e270e1b52437a98b | 0 ...stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c | 0 ...stats0-27-82294461be4728b4b191414bf2fb3bd7 | 0 .../stats0-28-31ebc4dcaa13c6dc5937c04f603e699 | 1 + ...stats0-29-ca0569fd4200d549843deadf84044649 | 500 ++++ .../stats0-3-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...stats0-30-4f18f4b06db06844920b14e2d19471a9 | 10 + ...stats0-31-4a7ed5b7e3deb4106204f8f950357e1c | 10 + .../stats0-4-5f7439f97275da70f633e135205f2095 | 0 .../stats0-5-6bcc1360b3d777a8b51ae416ff43898a | 0 .../stats0-6-85de3349a12a9fa0536f20ad0377fca1 | 0 .../stats0-7-247568f4b3ce6b97979ca9d28c2ba05c | 4 + .../stats0-8-7f38972bcd18c477a027d881182d83f9 | 500 ++++ .../stats0-9-98c925a2b2c4de06e270e1b52437a98b | 0 .../stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + .../stats2-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e | 0 ...stats2-11-912c785dbcef3251dd1d6b7fc56eae5b | 39 + .../stats2-2-a4fb8359a2179ec70777aad6366071b7 | 1 + .../stats2-3-16367c381d4b189b3640c92511244bfe | 1 + .../stats2-4-dc129f70e75cd575ce8c0de288884523 | 1 + .../stats2-5-6717e1026e24a38af19b6bce1951e3d7 | 0 .../stats2-6-45d4fb785fc94d266096fc29a9e90d73 | 0 .../stats2-7-6436db7a7506b560d0d1759db94a76b9 | 0 .../stats2-8-72621dba638b15d244850018e9f64d7 | 34 + .../stats2-9-6d93732dc2ca622eb60c171389caee8e | 19 + ...stats20-0-418ec894d08c33fd712eb358f579b7a0 | 1 + ...stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + ...stats20-2-4711e55648c25c86bb526ed50b0c3d09 | 1 + ...stats20-3-98c925a2b2c4de06e270e1b52437a98b | 0 ...stats20-4-82294461be4728b4b191414bf2fb3bd7 | 0 ...stats20-5-76509775cfe11bb98ee088188a07668a | 37 + ...stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 | 1 + ...stats20-7-82294461be4728b4b191414bf2fb3bd7 | 0 ...stats20-8-300c971de74642118d36d36349bc81aa | 37 + .../stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + .../stats3-1-418ec894d08c33fd712eb358f579b7a0 | 1 + ...stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 | 6 + ...stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa | 1 + .../stats3-12-892cb7ecc26e84f1c033b95a3ee3edc | 0 .../stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 | 0 ...stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa | 1 + ...stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 | 6 + .../stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 | 0 ...stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 | 0 ...stats3-18-eefbb3ee8b538aec85c609351e52901b | 38 + ...stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 | 0 .../stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 | 0 ...stats3-20-ca048ad81b3df7159822073d206f0790 | 0 .../stats3-3-ca048ad81b3df7159822073d206f0790 | 0 .../stats3-4-fa705a031ff5d97558f29c2b5b9de282 | 0 .../stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 | 43 + .../stats3-6-4bf1504274319c44d370b58092fe016c | 0 .../stats3-7-73d7d55d6e5a57aacce8618902904d | 32 + .../stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 | 0 .../stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 | 0 .../stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + .../stats4-1-418ec894d08c33fd712eb358f579b7a0 | 1 + ...stats4-10-a33b2c9d962e4921c98e62387f3989f7 | 0 ...stats4-11-ea921e0af59a4940a11c94143b1c4b32 | 2 + ...stats4-12-30bc31441828a053d1a675b225a5d617 | 2 + ...stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 | 1000 +++++++ ...stats4-14-9c82167763a771c175c656786d545798 | 1000 +++++++ ...stats4-15-f02b95f20b526fcf2850b07ca6be4f8c | 36 + ...stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 | 36 + ...stats4-17-746b888d14c1b3fa28aa8549c174f6d9 | 36 + ...stats4-18-dbe13731de4ab2a3c23343b78525b2f7 | 36 + ...stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 | 39 + .../stats4-2-463330cf55370dbe92d6ed74ef91302 | 4 + ...stats4-20-f63000f2c395b935199c9829964f98c1 | 39 + ...stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd | 0 ...stats4-22-f709d5c1d4c9f13b7a521867674ac54c | 0 .../stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd | 0 .../stats4-4-f709d5c1d4c9f13b7a521867674ac54c | 0 .../stats4-5-cd849c4fe1229428da98947e3e43b46d | 0 .../stats4-6-9c0d4354b6a9be351fa32a66ff58a177 | 0 .../stats4-7-16367c381d4b189b3640c92511244bfe | 1 + .../stats4-8-a4fb8359a2179ec70777aad6366071b7 | 1 + .../stats4-9-255ad4511130fb8c9ee9d65b7c95743f | 0 .../stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + .../stats5-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 + .../stats5-2-6f5d06d6100de19ec398891cb2eae161 | 0 .../stats5-3-96d9aa9c32a081518604959dcfac42df | 19 + .../stats5-4-dbf81a12f6c19c14dce831e942870744 | 0 .../stats5-5-3980dfc2e4f882dd3bf478e56929361 | 33 + .../stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + .../stats6-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...stats6-10-653f748fa2d690502ba4fda407841a20 | 32 + ...stats6-11-1c9f833953403596ad50fd32e513642c | 32 + ...stats6-12-fdbe263d925f28d598a71b7a4c577492 | 39 + .../stats6-2-a4fb8359a2179ec70777aad6366071b7 | 1 + .../stats6-3-16367c381d4b189b3640c92511244bfe | 1 + .../stats6-4-a88c476a632cd92f748967fadb242405 | 0 .../stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats6-6-8926095434b70c83bf88c70559d38dce | 0 .../stats6-7-6615049191cfa4026a3a870c3c3749f4 | 0 .../stats6-8-e15e242124e61ede9196130cb3fb69e7 | 36 + .../stats6-9-e6b884de17a29eb476fd6cc502fc615d | 36 + .../stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + .../stats7-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...stats7-10-fdbe263d925f28d598a71b7a4c577492 | 38 + .../stats7-2-a4fb8359a2179ec70777aad6366071b7 | 1 + .../stats7-3-16367c381d4b189b3640c92511244bfe | 1 + .../stats7-4-a88c476a632cd92f748967fadb242405 | 0 .../stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats7-6-5d0c6aa78c9786d904728ff7adab85f2 | 18 + .../stats7-7-4912a8bbc138ab97ac0983bc90951de4 | 0 .../stats7-8-e15e242124e61ede9196130cb3fb69e7 | 36 + .../stats7-9-e6b884de17a29eb476fd6cc502fc615d | 36 + .../stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + .../stats8-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 + .../stats8-10-ce78d029b7764edce3a26336cfac6a8 | 19 + ...stats8-11-6615049191cfa4026a3a870c3c3749f4 | 0 ...stats8-12-e6b884de17a29eb476fd6cc502fc615d | 36 + ...stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d | 19 + ...stats8-14-ea9afc1343991ed4d410231803a174f7 | 0 ...stats8-15-653f748fa2d690502ba4fda407841a20 | 36 + ...stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 | 19 + ...stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 | 0 ...stats8-18-1c9f833953403596ad50fd32e513642c | 36 + ...stats8-19-dbf72430cff3df0e6e9405ad64531b16 | 19 + .../stats8-2-a4fb8359a2179ec70777aad6366071b7 | 1 + ...stats8-20-bff592750d1057448d2cff7694b6dad2 | 0 ...stats8-21-e15e242124e61ede9196130cb3fb69e7 | 36 + ...stats8-22-e6b884de17a29eb476fd6cc502fc615d | 36 + ...stats8-23-653f748fa2d690502ba4fda407841a20 | 36 + ...stats8-24-1c9f833953403596ad50fd32e513642c | 36 + ...stats8-25-fdbe263d925f28d598a71b7a4c577492 | 39 + .../stats8-3-16367c381d4b189b3640c92511244bfe | 1 + .../stats8-4-a88c476a632cd92f748967fadb242405 | 0 .../stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats8-6-821e1f061960081b9b379d9bfb47f267 | 19 + .../stats8-7-8926095434b70c83bf88c70559d38dce | 0 .../stats8-8-e15e242124e61ede9196130cb3fb69e7 | 36 + .../stats8-9-fdbe263d925f28d598a71b7a4c577492 | 39 + .../stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + .../stats9-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...r_error_1-0-6d1832d28f897d0049de053617bd36 | 0 ...error_1-1-887fe99770f53e7e0a0fbdc190118612 | 1 + ...rror_1-10-a31221a0c377c14e11b14484ddaa49a6 | 1 + ...error_1-11-d58626190cded8d09f0457739a980eb | 1 + ...rror_1-12-3b7c5bcb71c75e913caaea3acd48530c | 0 ...rror_1-13-a31221a0c377c14e11b14484ddaa49a6 | 1 + ...rror_1-14-60a231b64a4a0e414d0ddce1c813c614 | 1 + ...rror_1-15-3b7c5bcb71c75e913caaea3acd48530c | 0 ...rror_1-16-a31221a0c377c14e11b14484ddaa49a6 | 1 + ..._error_1-2-2ca079278e0de95eecb5df315ce05c6 | 1 + ...error_1-3-66e3e0c942759f679c270698b49bfcf1 | 1 + ...error_1-4-d389db66cc7fd9b144445e364dac30e3 | 1 + ...error_1-5-63abf47101c5097e66a9c3ee92b128e3 | 1 + ...error_1-6-3b7c5bcb71c75e913caaea3acd48530c | 0 ...error_1-7-a31221a0c377c14e11b14484ddaa49a6 | 1 + ...error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 | 1 + ...error_1-9-3b7c5bcb71c75e913caaea3acd48530c | 0 ...yn_part-0-9505721cd28fc4ec94ef4baf07029027 | 0 ...yn_part-1-418ec894d08c33fd712eb358f579b7a0 | 1 + ...yn_part-2-c14f09f88961dbad4d800317079a9105 | 1 + ...yn_part-3-16367c381d4b189b3640c92511244bfe | 1 + ...yn_part-4-cc664530711607c530a2cd384e67a600 | 0 ...yn_part-5-76d56e06108f0c6da75aa821452fa873 | 0 ...rtition-0-42ae9007e347f7dedfc8974d21d84b41 | 0 ...rtition-1-418ec894d08c33fd712eb358f579b7a0 | 1 + ...rtition-2-c14f09f88961dbad4d800317079a9105 | 1 + ...rtition-3-868ba7a812c705caa29db49cdcb69c45 | 0 ...rtition-4-aed016ae4b528521874a719a5b129a55 | 35 + ...oscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + ...noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...scan_1-10-e6b884de17a29eb476fd6cc502fc615d | 36 + ...scan_1-11-653f748fa2d690502ba4fda407841a20 | 32 + ...scan_1-12-1c9f833953403596ad50fd32e513642c | 32 + ...scan_1-13-fdbe263d925f28d598a71b7a4c577492 | 38 + ...scan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab | 0 ...scan_1-15-db563e338e4f658e5072cc60aef15480 | 0 ...scan_1-16-e446db2c3ddce173d0a51bf77a489382 | 0 ...scan_1-17-82369b182db851e06bfddb62965e03a3 | 10 + ...scan_1-18-eece38f413a0a4f6f5b63cea0751d225 | 0 ...scan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e | 36 + ...oscan_1-2-a4fb8359a2179ec70777aad6366071b7 | 1 + ...scan_1-20-9871c619bb5bf0a28f8d60e6332a614f | 36 + ...scan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 | 32 + ...scan_1-22-521b49d223a32056025fb8dbd371a72a | 32 + ...scan_1-23-fba14d8647e0c8ca214b724486af7763 | 0 ...oscan_1-3-16367c381d4b189b3640c92511244bfe | 1 + ...oscan_1-4-7938a68104e163566da69ccc70a18f2c | 0 ...oscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 ...oscan_1-6-a1fd405e5175757aaa80033162c87670 | 10 + ...oscan_1-7-497861ae04753ffbb63ee43d89eedd9e | 0 ...oscan_1-8-623f3701ead20fff786c203d23dd60ae | 0 ...oscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 | 36 + ...tscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + ...rtscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...scan_1-10-c06f6ce9878b7eededf8c2a085ffb380 | 32 + ...scan_1-11-418ec894d08c33fd712eb358f579b7a0 | 1 + ...scan_1-12-3e423642a5a00dc66cc709f474a3ecef | 14 + ...scan_1-13-2fe3131322b6c82e217f27e95581e681 | 0 ...scan_1-14-2c66f128acea649e8debc3c4b04fcb9c | 36 + ...scan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 | 32 + ...scan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 | 0 ...tscan_1-2-a4fb8359a2179ec70777aad6366071b7 | 1 + ...tscan_1-3-16367c381d4b189b3640c92511244bfe | 1 + ...tscan_1-4-c95dc367df88c9e5cf77157f29ba2daf | 1 + ...tscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 | 1 + ...tscan_1-6-84967075baa3e56fff2a23f8ab9ba076 | 1 + ...tscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea | 1 + ...tscan_1-8-29279aa84d6ba9dea9e56b493504dd30 | 0 ...tscan_1-9-90d41ae72606f9616cb7b1303f997348 | 0 ...an_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 + ...can_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...n_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 | 32 + ...n_1_23-11-418ec894d08c33fd712eb358f579b7a0 | 1 + ...n_1_23-12-3e423642a5a00dc66cc709f474a3ecef | 15 + ...n_1_23-13-2fe3131322b6c82e217f27e95581e681 | 0 ...n_1_23-14-2c66f128acea649e8debc3c4b04fcb9c | 36 + ...n_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 | 32 + ...n_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 | 0 ...an_1_23-2-a4fb8359a2179ec70777aad6366071b7 | 1 + ...an_1_23-3-16367c381d4b189b3640c92511244bfe | 1 + ...an_1_23-4-c95dc367df88c9e5cf77157f29ba2daf | 1 + ...an_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 | 1 + ...an_1_23-6-84967075baa3e56fff2a23f8ab9ba076 | 1 + ...an_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea | 1 + ...an_1_23-8-b158e24051ecb66b8af46743917771ca | 0 ...an_1_23-9-90d41ae72606f9616cb7b1303f997348 | 0 ..._error_1-0-836d3689af091fdebbdf31e6b29a423 | 0 ...error_1-1-887fe99770f53e7e0a0fbdc190118612 | 1 + ...rror_1-10-a31221a0c377c14e11b14484ddaa49a6 | 1 + ...rror_1-11-50519f196d320ee97be731907409fef9 | 1 + ...rror_1-12-3b7c5bcb71c75e913caaea3acd48530c | 0 ...rror_1-13-a31221a0c377c14e11b14484ddaa49a6 | 1 + ...error_1-14-f596f33c4a9271a406371baef799e52 | 1 + ...rror_1-15-3b7c5bcb71c75e913caaea3acd48530c | 0 ...rror_1-16-a31221a0c377c14e11b14484ddaa49a6 | 1 + ..._error_1-2-2ca079278e0de95eecb5df315ce05c6 | 1 + ...error_1-3-66e3e0c942759f679c270698b49bfcf1 | 1 + ...error_1-4-d389db66cc7fd9b144445e364dac30e3 | 1 + ..._error_1-5-f39234f16f5819a2f51d3f200293332 | 1 + ...error_1-6-3b7c5bcb71c75e913caaea3acd48530c | 0 ...error_1-7-a31221a0c377c14e11b14484ddaa49a6 | 1 + ...error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b | 1 + ...error_1-9-3b7c5bcb71c75e913caaea3acd48530c | 0 ..._to_map-0-aefd618b58ad6c37956755b6572cbc73 | 1 + ..._to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c | 2 + ...to_map-10-32997010bba305ec40812df254490730 | 0 ...to_map-11-d99f1b631bc42a6a57c581025230537a | 0 ...to_map-12-f793eb7b36a1d6379b90e241df62c72e | 3 + ...to_map-13-32997010bba305ec40812df254490730 | 0 ..._to_map-2-28d8e6677d025e1266ce95ae23dfd9ec | 0 ..._to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 | 3 + ..._to_map-4-f356516aec917fe0d34df4dc8d9d0f95 | 0 ..._to_map-5-269cfcefe2ea6c19ac6c824d3259bbda | 3 + ..._to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf | 0 ..._to_map-7-5641678c53ce6ef1dbce3994843cfcad | 3 + ..._to_map-8-84121d964faad3547f0e5cce9d268612 | 0 ..._to_map-9-903f3b3137bfa3f4594fec7b604c62bd | 3 + ...literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 | 500 ++++ .../subq2-0-279fb87476c70e9384874fa6bf77a380 | 0 .../subq2-1-235919a7ddb574662158503b8052e7ca | 258 ++ ...as.attr-0-fc8183d758151be72b3d75d9df124504 | 1 + ...le.attr-0-26c9d24eb6305ea2106c26bdca38012e | 1 + ..._select-0-554898fe9d96fee7b652b8a771d66a25 | 0 ..._select-1-736d6a05e1fe3249a84eb58f7cd806d2 | 2 + ..._select-2-58d48c815413287858e2cfa16e5c6a5d | 0 ..._select-3-35c08c648a66f09f2cf7cfa6019c2113 | 500 ++++ ...eclause-0-b38bf01368da26ec9c60e9433a9c59a1 | 0 ...eclause-1-3e38e42c5b72364c5461c626f312be8c | 0 ...eclause-2-183920d856ad75e6d1e15121d3cd7364 | 1 + ...stamp_1-0-d362501d0176855077e65f8faf067fa8 | 0 ...stamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 | 0 ...stamp_1-2-74f477a8b726f5193dd42ef378a793c4 | 0 ...stamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 | 0 ...stamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 | 0 ...stamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 | 0 ...stamp_3-0-165256158e3db1ce19c3c9db3c8011d2 | 0 ...stamp_3-1-81edf5107270547641586aa02b4e7d9b | 0 ...stamp_3-2-7a012a0d98729da25b5ac374855dcee4 | 0 ...mp_lazy-0-c0b1d928c9dea9b247ace0a056586594 | 0 ...mp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc | 0 ...mp_null-0-3bd56783b637a47f6447d247024941a0 | 0 ...mp_null-1-4b9c19c1f57557979d3483d548f762cb | 0 ...mp_null-2-b3071984dee42c9e698e947fcbc2a1e8 | 0 ...amp_udf-0-79914c5347620c6e62a8e0b9a95984af | 0 ...amp_udf-1-59fc1842a23369235d42ed040d45fb3d | 0 ...amp_udf-2-9039f474f9a96e9f15ace528faeed923 | 0 ...amp_udf-3-b0fd4ca3b22eb732a32772399331352f | 0 .../touch-0-cd46bc635e3010cf1b990a652a584a09 | 0 .../touch-1-27ad2962fed131f51ba802596ba37278 | 0 .../touch-10-27ad2962fed131f51ba802596ba37278 | 0 .../touch-2-3e95421993ab28d18245ec2340f580a3 | 0 .../touch-3-c0c18ac884677231a41eea8d980d0451 | 0 .../touch-4-c06da7f8c1e98dc22e3171018e357f6a | 0 .../touch-5-af459a0264559a2aeaa1341ce779ab3c | 0 .../touch-6-5c7b24abea435f2628fe618f3a82e115 | 0 .../touch-7-3752320b12abae0d138148d56a27c5b1 | 0 .../touch-8-3562ab40242756452595cd7eae79b0ce | 0 .../touch-9-cd46bc635e3010cf1b990a652a584a09 | 0 ...ansform-0-d81d055660f6ef3d9cc60dd673a8c0fe | 500 ++++ ...nsform1-0-b6919fc48901e388c869c84ae0211102 | 0 ...nsform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 | 0 ...nsform1-2-25d6cab86c36d65fabf5645db3126a19 | 0 ...nsform1-3-8324a70d533a06a5171c1016b1fea7c3 | 0 ...nsform1-4-65527bae8e73262255ef83082c6968f9 | 0 ...nsform1-5-e0037a3f97ce0127a40d163af4c20ad5 | 0 ...ansform1-6-3b862abd732c9e9f0db50ad0b9dae6f | 1 + ...rm_ppr1-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...rm_ppr1-1-65fe307c21b350846b3c496890b0b619 | 0 ...rm_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 | 168 ++ ...rm_ppr2-0-ae225e86c2ae20519ffdf23190454161 | 1 + ...rm_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 | 0 ...rm_ppr2-2-636c4938673a273299c8569295d27c99 | 168 ++ ... clause-0-3b6afcbd622aa111ee260bebc763613d | 1028 +++++++ ... clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 | 1028 +++++++ ...idening-0-630ac2c7e7dea4837384ccd572209229 | 0 ...idening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 | 1 + ...idening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 | 0 ...idening-3-65da8c67f6903286168acb39ac67fc04 | 1000 +++++++ ...ect_set-0-38512a3299e2390dd813e61a0f63f35e | 1 + ...ect_set-1-78aa199d061d2ff9ba426849ea1eb449 | 1 + ...ect_set-10-c8bc33095e1a195bb7b5e579d8d78db | 20 + ...ct_set-11-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...ct_set-12-a00d1791b7fa7ac5a0505d95c3d12257 | 1 + ...ect_set-13-c8bc33095e1a195bb7b5e579d8d78db | 20 + ...ect_set-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 + ...ect_set-3-a7dc16cb82c595b18d4258a38a304b1e | 1 + ...lect_set-4-c8bc33095e1a195bb7b5e579d8d78db | 20 + ...ect_set-5-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...ect_set-6-a7dc16cb82c595b18d4258a38a304b1e | 1 + ...lect_set-7-c8bc33095e1a195bb7b5e579d8d78db | 20 + ...ect_set-8-1d351f7e821fcaf66c6f7503e42fb291 | 1 + ...ect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 | 1 + ...af_corr-0-2f783a39471a7f1df9a824d741c443a9 | 0 ...af_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 | 0 ...f_corr-10-2f783a39471a7f1df9a824d741c443a9 | 0 ...daf_corr-2-c6f2dc536bf105650a461816ae5e330 | 0 ...daf_corr-3-ddf417dbc3b551cc8df47d950cec03e | 2 + ...af_corr-4-8771b2d83d14b3b641e8a77fcdc5081a | 9 + ...af_corr-5-8abbd73784728b599f8c2a90f53da9fb | 1 + ...af_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 | 1 + ...af_corr-7-70e701f50c3418ff91649b2bd8287da3 | 1 + ...af_corr-8-f2f0c7735f8b24266d5aaff96644e369 | 6 + ...af_corr-9-e2a0fa75c43279764ebca015f62bcf16 | 1 + ...var_pop-0-2f783a39471a7f1df9a824d741c443a9 | 0 ...var_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 | 0 ...ar_pop-10-2f783a39471a7f1df9a824d741c443a9 | 0 ...ovar_pop-2-c6f2dc536bf105650a461816ae5e330 | 0 ...var_pop-3-fe27ea6dae14250e8835baef3c2e20f6 | 1 + ...var_pop-4-7af9231ae293d0c4b84050176e1d73ad | 6 + ...var_pop-5-22004d36f6f3770db284644317770fcd | 1 + ...var_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b | 1 + ...var_pop-7-37e59e993e08216e6c69f88d6ac673ae | 1 + ...var_pop-8-1e51388408dad651127edf940c11d91f | 6 + ...var_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 | 1 + ...ar_samp-0-2f783a39471a7f1df9a824d741c443a9 | 0 ...ar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 | 0 ...r_samp-10-2f783a39471a7f1df9a824d741c443a9 | 0 ...var_samp-2-c6f2dc536bf105650a461816ae5e330 | 0 ...ar_samp-3-7aa25da7ccb88ba67b100888b6227aaf | 1 + ...ar_samp-4-7e705a637950911e0a18059d8bf1fd2c | 6 + ...var_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b | 1 + ...ar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 | 1 + ...ar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 | 1 + ...ar_samp-8-44861ae58cf0951eeda28a5f778f778a | 6 + ...ar_samp-9-234a5b02085d2228473d7ef15a6af683 | 1 + ...numeric-0-86b9fb8bef8a5c51077623f1db3a0251 | 1 + ...numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 | 1 + ...numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 | 1 + ...numeric-3-ff41f4450d6ae372633fde865ae187c6 | 1 + ..._format-0-eff4ef3c207d14d5121368f294697964 | 0 ..._format-1-4a03c4328565c60ca99689239f07fb16 | 1 + ...prox_20-0-9ce9365f7b3f35a10b5305251c3e81ac | 0 ...prox_20-1-c7d32089880679d178dea94f1fe118e6 | 0 ...rox_20-10-b7e588217a3cd184dbbb8d419d3e33ae | 0 ...rox_20-11-5034ec132cb8b0a6bd6357a7e1abd755 | 0 ...rox_20-12-914ba18b45a27894bd82302f07efc789 | 0 ...rox_20-13-4bd5703fa32f3283f38841acadc97adb | 0 ...rox_20-14-d861a06b90896a097901d64ab9fbec53 | 0 ...rox_20-15-ca796efecd0d064e9e688a17ce75d80f | 0 ...rox_20-16-c838e13d9aafe1212a76d2cf5fe085a0 | 0 ...rox_20-17-b89ea2173180c8ae423d856f943e061f | 1 + ...rox_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...prox_20-19-b931aec4add0a66c23e444cdd5c33c5 | 0 ...prox_20-2-ac53a7ba5e8a208255008d3a71fa321a | 0 ...rox_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 + ...rox_20-21-3cd4e1282d82d07785051a1cf0e9b4ff | 1 + ...rox_20-22-ed1aec1a908310db90c5f8667631a1df | 1 + ...rox_20-23-333d72e8bce6d11a35fc7a30418f225b | 1 + ...rox_20-24-61903781f5cd75e6f11d85e7e89c1cb3 | 1 + ...rox_20-25-15f40568b41c4505841f5ad13c526f51 | 1 + ...rox_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 + ...rox_20-27-ee4c065e8557632a9ee348dd9223c3a1 | 1 + ...rox_20-28-ba77d1a26f87385f046129b6eb7d2ec3 | 1 + ...rox_20-29-956d6b6bc69c8035f80de2e60eda65fb | 1 + ...prox_20-3-1dae5b2a11507c83b0f76e677a368712 | 0 ...rox_20-30-d196cc7f52bb6ae19a5e66eb2a99577c | 1 + ...rox_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 + ...rox_20-32-dbcec232623048c7748b708123e18bf0 | 1 + ...rox_20-33-f28c7b0408737da815493741c806ff80 | 0 ...rox_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 + ...rox_20-35-3cd4e1282d82d07785051a1cf0e9b4ff | 1 + ...rox_20-36-ed1aec1a908310db90c5f8667631a1df | 1 + ...rox_20-37-333d72e8bce6d11a35fc7a30418f225b | 1 + ...rox_20-38-61903781f5cd75e6f11d85e7e89c1cb3 | 1 + ...rox_20-39-15f40568b41c4505841f5ad13c526f51 | 1 + ...prox_20-4-b2e21ffa55342d4f3c243728dfe6b11f | 0 ...rox_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 + ...rox_20-41-ee4c065e8557632a9ee348dd9223c3a1 | 1 + ...rox_20-42-ba77d1a26f87385f046129b6eb7d2ec3 | 1 + ...rox_20-43-956d6b6bc69c8035f80de2e60eda65fb | 1 + ...rox_20-44-d196cc7f52bb6ae19a5e66eb2a99577c | 1 + ...rox_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 + ...prox_20-5-8ae1465266d28bc2e5da8d89617873c4 | 0 ...prox_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 | 0 ...prox_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 | 0 ...pprox_20-8-7e3cf228c457279965b7414bd05527f | 0 ...prox_20-9-5aea8aa95a85c46284f7c1f45978a228 | 0 ...prox_23-0-477a942be95c0616c72f02a0077f9ace | 0 ...prox_23-1-c7d32089880679d178dea94f1fe118e6 | 0 ...rox_23-10-b7e588217a3cd184dbbb8d419d3e33ae | 0 ...rox_23-11-5034ec132cb8b0a6bd6357a7e1abd755 | 0 ...rox_23-12-914ba18b45a27894bd82302f07efc789 | 0 ...rox_23-13-4bd5703fa32f3283f38841acadc97adb | 0 ...rox_23-14-d861a06b90896a097901d64ab9fbec53 | 0 ...rox_23-15-ca796efecd0d064e9e688a17ce75d80f | 0 ...rox_23-16-c838e13d9aafe1212a76d2cf5fe085a0 | 0 ...rox_23-17-b89ea2173180c8ae423d856f943e061f | 1 + ...rox_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 + ...prox_23-19-b931aec4add0a66c23e444cdd5c33c5 | 0 ...prox_23-2-ac53a7ba5e8a208255008d3a71fa321a | 0 ...rox_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 + ...rox_23-21-3cd4e1282d82d07785051a1cf0e9b4ff | 1 + ...rox_23-22-ed1aec1a908310db90c5f8667631a1df | 1 + ...rox_23-23-333d72e8bce6d11a35fc7a30418f225b | 1 + ...rox_23-24-61903781f5cd75e6f11d85e7e89c1cb3 | 1 + ...rox_23-25-15f40568b41c4505841f5ad13c526f51 | 1 + ...rox_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 + ...rox_23-27-ee4c065e8557632a9ee348dd9223c3a1 | 1 + ...rox_23-28-ba77d1a26f87385f046129b6eb7d2ec3 | 1 + ...rox_23-29-956d6b6bc69c8035f80de2e60eda65fb | 1 + ...prox_23-3-1dae5b2a11507c83b0f76e677a368712 | 0 ...rox_23-30-d196cc7f52bb6ae19a5e66eb2a99577c | 1 + ...rox_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 + ...rox_23-32-dbcec232623048c7748b708123e18bf0 | 1 + ...rox_23-33-f28c7b0408737da815493741c806ff80 | 0 ...rox_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 + ...rox_23-35-3cd4e1282d82d07785051a1cf0e9b4ff | 1 + ...rox_23-36-ed1aec1a908310db90c5f8667631a1df | 1 + ...rox_23-37-333d72e8bce6d11a35fc7a30418f225b | 1 + ...rox_23-38-61903781f5cd75e6f11d85e7e89c1cb3 | 1 + ...rox_23-39-15f40568b41c4505841f5ad13c526f51 | 1 + ...prox_23-4-b2e21ffa55342d4f3c243728dfe6b11f | 0 ...rox_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 + ...rox_23-41-ee4c065e8557632a9ee348dd9223c3a1 | 1 + ...rox_23-42-ba77d1a26f87385f046129b6eb7d2ec3 | 1 + ...rox_23-43-956d6b6bc69c8035f80de2e60eda65fb | 1 + ...rox_23-44-d196cc7f52bb6ae19a5e66eb2a99577c | 1 + ...rox_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 + ...prox_23-5-8ae1465266d28bc2e5da8d89617873c4 | 0 ...prox_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 | 0 ...prox_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 | 0 ...pprox_23-8-7e3cf228c457279965b7414bd05527f | 0 ...prox_23-9-5aea8aa95a85c46284f7c1f45978a228 | 0 .../udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 | 0 .../udf1-1-8281592c818ada269024ac669bec78da | 0 .../udf1-2-c7648c913ec336559fb67e3ab6938c8f | 0 .../udf1-3-adc1ec67836b26b60d8547c4996bfd8f | 1 + .../udf2-0-2a18d9570d9b676e240cda76df818c42 | 0 .../udf2-1-f60851dc36f579e83d6848d7d3c589e6 | 0 .../udf2-2-31c8af064bac42541558a95fad3bca97 | 0 .../udf2-3-c5938fcdd5675b58a4ed54269b5f5591 | 1 + .../udf3-0-66a2b926dd5439d4da9eb58743c18a8c | 0 .../udf3-1-1d04874d496d05cfe0b9d86de1111 | 0 .../udf3-2-25fe77d053e2bad8ae99757ce237052e | 0 .../udf3-3-adc1ec67836b26b60d8547c4996bfd8f | 1 + .../udf6-0-2a18d9570d9b676e240cda76df818c42 | 0 .../udf6-1-f60851dc36f579e83d6848d7d3c589e6 | 0 .../udf6-2-4d2a0815afe8c050cabf7a2efbce8521 | 0 .../udf6-3-e579646b969eef49b09656114da52a73 | 1 + .../udf6-4-29f45ffe530dd8c27dfb82268017dbb2 | 0 .../udf6-5-fe336cd9850d6357980bd19139f76e | 1 + .../udf7-0-2a18d9570d9b676e240cda76df818c42 | 0 .../udf7-1-f60851dc36f579e83d6848d7d3c589e6 | 0 .../udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 | 0 .../udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 | 1 + .../udf8-0-2a18d9570d9b676e240cda76df818c42 | 0 .../udf8-1-63c38297946a2060c0cff4a426b0520c | 0 .../udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 | 0 .../udf8-3-72c1a1dd1627491550f6e19581a654cb | 0 .../udf8-4-9f22d5a65353432826a526b1d76eb65b | 1 + .../udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 | 0 .../udf9-1-dd0981dc44ac24d445af5412e9f7fa8c | 1 + ...0_trims-0-2a18d9570d9b676e240cda76df818c42 | 0 ...10_trims-1-4534b1b2e9101058e8d71756bcb416e | 0 ...0_trims-2-7624a192247c9c4c7be6a40d46f13597 | 0 .../udf_E-0-33251f00f840de3672f19d353fcfa66f | 0 .../udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c | 1 + .../udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a | 1 + .../udf_E-3-e8924af3bf99d2e01546a965303ffd09 | 5 + .../udf_E-4-33251f00f840de3672f19d353fcfa66f | 0 .../udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c | 1 + .../udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a | 1 + .../udf_E-7-e8924af3bf99d2e01546a965303ffd09 | 5 + .../udf_PI-0-b28e761e5564b51f98f182f561c1369f | 0 .../udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 | 1 + .../udf_PI-2-9c1476a2eab7455594e97b338ee3c188 | 1 + .../udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 | 5 + .../udf_PI-4-b28e761e5564b51f98f182f561c1369f | 0 .../udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 | 1 + .../udf_PI-6-9c1476a2eab7455594e97b338ee3c188 | 1 + .../udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 | 5 + ...udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 | 1 + ...udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 | 6 + ...udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 | 0 ...udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 | 1 + .../udf_abs-4-30cd5a94c13e1619ee18b9551db879c | 0 ...udf_abs-5-343e899acb67c283391387f02aa7b5c4 | 1 + ...df_acos-0-4f49cb5a5c87efea534d63ed76435f06 | 1 + ...df_acos-1-d330511cf4f626cd844b18f57f99a85f | 6 + ...df_acos-2-86fca49baf270873b46709c9eaeab87b | 1 + ...df_acos-3-f7f199e5f3dde8056465d55aca29e884 | 1 + ...df_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a | 1 + ...df_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e | 1 + ...udf_add-0-5db764318a918a5f6d7c1d95d9e86045 | 1 + ...udf_add-1-400b238f4e6cdf7120be566b0ef079c5 | 1 + ...f_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee | 1 + ...f_array-1-570741914bb78300b0233e5f38d7f08a | 1 + ...f_array-2-db6d3c6c67faed3ceb019cb1993502f6 | 0 ...f_array-3-a5d12c41277fb158e09281169c905122 | 1 + ...ontains-0-d9a90108b052b111e8de4433e008b25a | 1 + ...ontains-1-eff16c7836252e01f3d8190cd833f79c | 4 + ...ontains-2-42d966b28e61a465d638bffc20ac7247 | 1 + ...ontains-3-ec0cd851fd8135dd9bc822d9a0432569 | 1 + ...f_ascii-0-72924c23459330cca6a54c70b12a542c | 1 + ...f_ascii-1-dab8656d7f001e85615442d60df4b6b3 | 6 + ...f_ascii-2-a9e207503f52d43903877fb998eabeaa | 0 ...f_ascii-3-28fc6497c9835c2ef331aba44576f1b1 | 1 + ...df_asin-0-99da197a53767060e3fa0250254d59cd | 1 + ...df_asin-1-3d0c3fa6121f8f5158d221074f1d4129 | 6 + ...df_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f | 1 + ...df_asin-3-b67069d37df3a7fb7a3e2138d8558343 | 1 + ...df_asin-4-929dabad86ef0e564802de8f663a9e66 | 1 + ...df_asin-5-1ee8715cce9831623d0af0031964d284 | 1 + ...df_atan-0-c79ed30c2444c8493d0db98c33c9132b | 1 + ...df_atan-1-77e7ac1b89a4eac9102176cd73f67a62 | 4 + ...df_atan-2-bf1c7875364d5518e0ac9c1ac6943764 | 1 + ...df_atan-3-9a6252f36fe5ec1387186bf47d74a139 | 1 + ...df_atan-4-c79ed30c2444c8493d0db98c33c9132b | 1 + ...df_atan-5-77e7ac1b89a4eac9102176cd73f67a62 | 4 + ...df_atan-6-bf1c7875364d5518e0ac9c1ac6943764 | 1 + ...df_atan-7-9a6252f36fe5ec1387186bf47d74a139 | 1 + ...udf_avg-0-2d715528b290951fb9874f60d7e9b537 | 1 + ...udf_avg-1-c707c56871a903e4e022b3df5c92fc3f | 1 + ..._bigint-0-6c5b1e4b9d725caeb786bb18448a7927 | 1 + ..._bigint-1-4636e4f0083ea54814995a03b7c81202 | 1 + ...udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b | 1 + ...udf_bin-1-843803a1b6ada107c11397af1a2f55d6 | 5 + ...udf_bin-2-5ee3932ab9cd164f1005a4413a68007b | 1 + .../udf_bin-3-b72fc578a7c677e15b8598248c81901 | 1 + ...map_and-0-abea2a2780fad42422774174cbfd603d | 1 + ...map_and-1-414291f11711df40fb8362e0a0156b25 | 1 + ...map_and-2-d550d017f9eb9176593719ea34eaae9b | 0 ...map_and-3-a486db1f5a06f9623a0e5abd7737b0c6 | 0 ...map_and-4-6320e1c4829024f819d3883df9a782c0 | 0 ...map_and-5-ff2860a163cbe78d5affac8047199296 | 10 + ...map_and-6-d550d017f9eb9176593719ea34eaae9b | 0 ...p_empty-0-6c80525a4eab6761596e6ad5aa75bc90 | 1 + ...p_empty-1-a03987655a167f5b18c37b643391a0df | 1 + ...tmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c | 1 + ...tmap_or-1-85cab84fba419b436b033e3ea07e02ef | 1 + ...tmap_or-2-d550d017f9eb9176593719ea34eaae9b | 0 ...tmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 | 0 ...tmap_or-4-6320e1c4829024f819d3883df9a782c0 | 0 ...tmap_or-5-ea92fff4d814552b57535ed342a5dde0 | 10 + ...tmap_or-6-d550d017f9eb9176593719ea34eaae9b | 0 ...ise_and-0-e2374700cd32add926992d5539bd463a | 1 + ...ise_and-1-2e63ac31262106160ab043027e356a4b | 4 + ...ise_not-0-34abab2f47f90f0f194ef44aed1cdd7f | 1 + ...ise_not-1-ccc0c59ea3e29b6d55e1edee7029155d | 4 + ...wise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 | 1 + ...wise_or-1-272722c23fece2807c08191d3969c3bb | 4 + ...ise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 | 1 + ...ise_xor-1-8fc9b9bf1aced42754419310784e0a9f | 4 + ...boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 | 1 + ...boolean-1-23178b473a9d25d6be0abe378d44fb0e | 1 + ...df_case-0-5bcbe4c0e0a75462160814a10b9449f4 | 1 + ...df_case-1-54acf006155d8822a50e47729be24004 | 1 + ...df_case-2-98ee676f92950375917f09d2e492253f | 0 ...df_case-3-ec7343402fd77807842a0eaf2497a47c | 1 + ...df_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 | 1 + ...pruning-0-dd2d7a075df235f17c26bac8713e939c | 0 ..._thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 | 0 ..._thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 | 3 + ...df_ceil-0-4b40e67b8ca75729ab07df966d814e06 | 1 + ...df_ceil-1-f410065d893a263f375fcf74072877bb | 7 + ...ceiling-0-d5685e38400e68341e36a59671dcbdfd | 1 + ...ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df | 7 + ...oalesce-0-8d1c97c292c51447e58606f2cefa87c1 | 1 + ...oalesce-1-e2c086f5148f10264c09ecbd7284c809 | 4 + ...coalesce-2-bd78a25868e5598ea275e0be5e4c716 | 0 ...oalesce-3-badb02b0dfa13d6a1519e6198bb511d2 | 1 + ...oalesce-4-83f323874d7941c463ced2aee6cc5157 | 0 ...oalesce-5-4bcad31a47f4bfc3baef75b65baa8082 | 11 + ..._string-0-32b16ab99287db115e8de5214ac24b77 | 10 + ...f_concat-0-7bc53505a4e6587132870d8d0a704d2 | 1 + ..._concat-1-765c520f239cdff1ea3f8d22ba83e031 | 5 + ..._concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 | 1 + ...insert1-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...insert1-1-ed0d453b9879faed8bf93913f7690545 | 0 ...insert1-2-adc1ec67836b26b60d8547c4996bfd8f | 57 + ...insert2-0-659e06570690cceeb3f37e10e855d2ea | 0 ...insert2-1-39829ebc03d9e047f3eaac80b4428768 | 0 ...insert2-2-adc1ec67836b26b60d8547c4996bfd8f | 84 + ...ncat_ws-0-4c8bd7794a527e544c838408313eeaa8 | 1 + ...ncat_ws-1-b8b80f7e9bf4348367444c73020b3cab | 4 + ...ncat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c | 0 ...ncat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 | 0 ...ncat_ws-4-a507af4996b13433b0ae100fcb32358f | 0 ...ncat_ws-5-ca4f051369183cae36fc9a7bec6a9341 | 1 + ...ncat_ws-6-98276439c0605401ff89c6a5ae54be09 | 0 ...ncat_ws-7-8f08128276e7e50eeb5a6932c763126c | 1 + ...ncat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 | 1 + ...df_conv-0-d552befca345f3396464529cfde9f75a | 1 + ...df_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab | 7 + ...f_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 | 3 + ...df_conv-2-6d61a2118b54843716aef87fe539b595 | 1 + ...df_conv-3-97161f7a60851d445b23c4ebe4095a1d | 1 + ...df_conv-4-568e843076f358c404a8634b18541c55 | 1 + ...df_conv-5-3f23d98799b825a2e9594066f973d183 | 1 + ...df_conv-6-4981b5b92d87fd1000fa0ac26fa6163a | 1 + ...df_conv-7-77bd25ad13e3697c80464e4a2682360e | 1 + ...df_conv-8-2fae52d392251be476e0c8f6071a4aeb | 1 + ...df_conv-9-2f0098c00c10044522cd23a4a2f54957 | 1 + ...udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 | 1 + ...udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 | 4 + ...udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 | 1 + ...udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 | 1 + ...f_count-0-534a9b25b87d09e418645b1216949560 | 3 + ...f_count-1-d566feb21bc894b97e6416b65fe5c02f | 3 + ..._count-10-455b30e7df33c5eebcaacabe0a578483 | 0 ..._count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e | 1 + ..._count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e | 1 + ...f_count-13-9286bc5f08bf4db183719b1b49dc5b7 | 1 + ...f_count-2-3344645abe535426307b9327b381fc85 | 0 ...f_count-3-e43165f41597d2a1c9e8cf780b99a4a8 | 1 + ...f_count-4-93223bab63eada6ece8bc176e89c87a8 | 0 ...f_count-5-bdee61c35a27bfab974e2ba199d5dfa4 | 1 + ...f_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 | 0 ...f_count-7-b975ad0d5f293508ce4832a7b19399b6 | 1 + ...f_count-8-911e83c85155e1f811ba2c75aabbc6b7 | 0 ...f_count-9-75b3d8a0dac332ea00af5ef8971ca643 | 1 + ...ate_add-0-74d34471bfa0880230d8e3351eb0ab45 | 1 + ...date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 | 5 + ...ate_sub-0-f8fbf85026da1b0778fd325d9b5dae33 | 1 + ...ate_sub-1-7efeb74367835ade71e5e42b22f8ced4 | 5 + ...atediff-0-3bd040a96a4568e7ea4922faa408ada5 | 1 + ...atediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 | 5 + ...udf_day-0-c4c503756384ff1220222d84fd25e756 | 1 + .../udf_day-1-87168babe1110fe4c38269843414ca4 | 6 + ...ofmonth-0-7b2caf942528656555cf19c261a18502 | 1 + ...ofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 | 6 + ...degrees-0-85f4957cd7cd6c517f6941af1289d8ae | 0 ...degrees-1-aabc6065a03b7da809376cc127af47d7 | 1 + ...degrees-2-2fd3a55901fe765f8f154531a7f5fd6b | 1 + ...degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 | 5 + ...degrees-4-85f4957cd7cd6c517f6941af1289d8ae | 0 ...degrees-5-aabc6065a03b7da809376cc127af47d7 | 1 + ...degrees-6-2fd3a55901fe765f8f154531a7f5fd6b | 1 + ...degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 | 5 + ...udf_div-0-31d31c7d5c544327dabfd874c88314db | 1 + ...udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 | 4 + .../udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b | 1 + ..._divide-0-1af8b249439ee5b7d4978c31669bc208 | 1 + ..._divide-1-fa932664bae88683a222b71ac45fb840 | 4 + ..._divide-2-ce54773b1babe6dde982e4e91ebaeb50 | 1 + ..._double-0-aa32d73a99587cae2f1efd9a2094d617 | 1 + ..._double-1-79380157cbd6624d760335f8291e6fb4 | 1 + ...udf_elt-0-b46b060da76d1772db998c26a62a608f | 1 + ...udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 | 4 + ...udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a | 0 ...udf_elt-3-f3be980cf4fa166f299c6ec79e981814 | 1 + ...udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a | 1 + ...udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 | 4 + ...f_field-0-277b4a4dcb38cabb6df50147c77e0a33 | 1 + ...f_field-1-379d8580693493f620a8f4084709324f | 2 + ..._field-10-7982ea72163dbc4cd45f53454edf66c8 | 2 + ...f_field-2-d2c6583a79d77aabe388a52ec164c38b | 1 + ...f_field-3-fea09e934696af40bb604b40225bbc98 | 1 + ...f_field-4-b0815d34893d6cba8c07d0a0721c1d29 | 1 + ...f_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 | 0 ...f_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 | 0 ...f_field-7-6aa3518e9f55299754521e959e9376ef | 2 + ...f_field-8-66dc6c81db0ac9b2075783b0d8976083 | 0 ...f_field-9-f053f2d16812aa60b6dd1cab61e90a95 | 0 ..._in_set-0-18d3e88b18c18a00598146a3307276f2 | 1 + ..._in_set-1-5fb7a7a1725749dc3853eb80fba19246 | 7 + ...in_set-10-16355c6b7e169b3c0ef506c149c6853c | 1 + ...in_set-11-5a8515684c458d3fffea539a3d170e3a | 1 + ..._in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 | 1 + ..._in_set-13-2c986a80620c9238e1f663fc591760a | 1 + ...in_set-14-189def133b1871ce8345a8123811a6b5 | 1 + ...in_set-15-671bff8f50feea55015a8412fc6e5ceb | 1 + ...in_set-16-d5d22082588c5fc30ef502237c5797f4 | 1 + ...in_set-17-5b556a29e1685605bcc47bce60cf66c8 | 2 + ..._in_set-2-b3823bdc04a7f98951b55c3e30d2a772 | 0 ..._in_set-3-132b7bc7812db7683eb3bff607275d0e | 25 + ..._in_set-4-a35471c87ba597a6d3c7c880704cac0b | 1 + ..._in_set-5-ddaa3551dffb1169b2fbb671f373b82f | 1 + ..._in_set-6-591e070365f19c65e453b98b88f5f823 | 1 + ..._in_set-7-72d05b5cf99388d539adec38c40978c3 | 1 + ..._in_set-8-780771cad9bec96a216aea8ab293c941 | 1 + ..._in_set-9-d59f5aabe1ea0963f9328065c699d175 | 1 + ...f_float-0-7987032f1c9dcad07001445f3ca1f7a7 | 1 + ...f_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b | 1 + ...f_floor-0-e35abe1d5534243e96d71bd0c28761d6 | 1 + ...f_floor-1-497a1ddbcf738aead319fde4f90f5248 | 6 + ..._number-0-e86d559aeb84a4cc017a103182c22bfb | 0 ..._number-1-525f133cfff86d44afdeeda667c365a5 | 1 + ..._number-2-591f302d5c1cd24e153a598fa0b352fd | 4 + ..._number-3-c89564db1ab953e28b050b9740f2650c | 0 ..._number-4-295d41a2146a27320c2be90499343260 | 1 + ..._number-5-881f33f6727a30629bde6e4b178cf7d9 | 1 + ..._number-6-a6720a128716e179e18933992ca899b3 | 1 + ..._number-7-84a460780828b0b9a2235314cfc24766 | 1 + ..._number-8-e7eedc849c74ce7d33c559067dd9ca0e | 1 + ..._number-9-407a0a7c277bb4c5c94ce16533ce1646 | 1 + ...nixtime-0-c3adaeede5c48d232473d78acf0eed7f | 1 + ...nixtime-1-d1a511d2084c7c621b5f638908c8db65 | 4 + ..._object-0-c08e7139c00878b98d396e65d958100f | 1 + ..._object-1-706bcfd51431ec7f2b80145837f94917 | 16 + ..._object-2-2a18d9570d9b676e240cda76df818c42 | 0 ..._object-3-f60851dc36f579e83d6848d7d3c589e6 | 0 ..._object-4-4f08101fd66fb25d7b322d47773e49f3 | 0 ...terthan-0-99d268829a124103cb3429c53fdc4de4 | 1 + ...terthan-1-8aab8e39726a986e10e1e572939fd63c | 1 + ...terthan-2-79ba62f35a9432647e31c6722b44fe6f | 1 + ...orequal-0-a7214027a91abf6501881e2af313347a | 1 + ...orequal-1-3669f2008e7f428f365efadbcb5ae451 | 1 + ...orequal-2-d2690cc7713e91201bb10ef291c95819 | 1 + ...df_hash-0-b9e3a3986320d275982797140edfccf4 | 1 + ...df_hash-1-a18646b51501d0b1beb967dc79afbd1a | 1 + ...df_hash-2-cc121f3c38a7a522abd824940fe04285 | 0 ...df_hash-3-23991312391d518aacf3d4469c816eae | 1 + ...udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 | 1 + ...udf_hex-1-d55348c0ccd133b7abb690f6949b520c | 8 + ...udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 | 1 + ...udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf | 1 + ...udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea | 1 + ...df_hour-0-ba1c46a403b807fe0a28b85e62d869ce | 1 + ...df_hour-1-3db41b9fe9966a45e663bc42cf182c04 | 7 + ...udf_hour-2-ace1054795b20abd5ae829814cfe15a | 0 ...df_hour-3-415b0842ab0818c82baf9fbf07180613 | 1 + .../udf_if-0-b7ffa85b5785cccef2af1b285348cc2c | 1 + .../udf_if-1-30cf7f51f92b5684e556deff3032d49a | 1 + .../udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 | 0 .../udf_if-3-20206f17367ff284d67044abd745ce9f | 1 + .../udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca | 0 .../udf_if-5-a7db13aec05c97792f9331d63709d8cc | 1 + .../udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 | 1 + .../udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 | 3 + ...in_file-0-1775b929e50cae8b3e957d99c5378f59 | 1 + ...in_file-1-2f23153970a569a4643574dde8d78a58 | 0 ...f_index-0-a277ac394cae40cb55d1ef3aa5add260 | 1 + ...f_index-1-1f5e109131b0c67ebea521fa4902a8f6 | 1 + ...f_instr-0-2e76f819563dbaba4beb51e3a130b922 | 1 + ...f_instr-1-32da357fc754badd6e3898dcc8989182 | 4 + ...f_instr-2-10147893f38fc08dad4fa7f2bc843fc2 | 0 ...f_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 | 1 + ...udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba | 1 + ...udf_int-1-3f0405ff93adfe8b3402b118567867d7 | 1 + ...notnull-0-44584503014c378bb916b38e1879bfb6 | 1 + ...snotnull-1-6ca2ea7938c7dac331c603ad343c1a7 | 1 + ..._isnull-0-ac8e7827d760108923509f9ea1691d53 | 1 + ..._isnull-1-55d9d04204f30cde4aa2667db88db262 | 1 + ...notnull-0-ac8e7827d760108923509f9ea1691d53 | 1 + ...notnull-1-55d9d04204f30cde4aa2667db88db262 | 1 + ...notnull-2-44584503014c378bb916b38e1879bfb6 | 1 + ...snotnull-3-6ca2ea7938c7dac331c603ad343c1a7 | 1 + ...notnull-4-3dd03048c0152f565b21b6d3b7b010f1 | 0 ...notnull-5-253ed8f6f8659120af927871f57d81a1 | 1 + ...notnull-6-9daf0ab0e3b8315018341d6f72bd3174 | 0 ...notnull-7-bb1030dea18d2a0c2c00a6e2de835d6b | 1 + ..._method-0-991b98a25032b21802bc2a1efde606c7 | 1 + ..._method-1-a3b94d9f2c2caf85a588b6686a64630a | 3 + ..._method-2-69e6b8725086a8fb8f55721705442112 | 0 ..._method-3-c526dfd4d9eac718ced9afb3cf9a62fd | 1 + ...f_lcase-0-649df2b742e6a03d0e0e364f5bee76ad | 1 + ...f_lcase-1-d947c47e03bedbfd4954853cc134c66e | 5 + ..._length-0-38364b60c3a2409f53c9aa2dae19903b | 1 + ..._length-1-f183e1f8ae516bb483132ed106289b67 | 4 + ...length-10-f3a9bd30540345db0f69b6847014b333 | 1 + ..._length-2-af46cb6887618240836eaf5be8afbba6 | 0 ..._length-3-dcd6404afce1103d5054527e6c216d6d | 0 ..._length-4-ba49ba4e6505c74bc33b5003f3930c43 | 0 ..._length-5-adc1ec67836b26b60d8547c4996bfd8f | 25 + ..._length-6-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 ..._length-7-8f28e6c488df47d89dca670f04a7563f | 0 ..._length-8-5e0fe761b7520651c3446ce7f9179caf | 0 ..._length-9-de456a5765db4a06110d9483985aa4a6 | 0 ...essthan-0-a0d9e8b51e3d13685b3889db38f22427 | 1 + ...essthan-1-952c655a1092a410e5346f1205cb8142 | 1 + ...essthan-2-92fa47f7af4a03ce1a965472eaad23a7 | 1 + ...orequal-0-b3b021456c85da76d1879191886a425b | 1 + ...orequal-1-869d983466744ad73b109211e1638869 | 1 + ...orequal-2-56775013e20ecf2287e07e83eccf2e0c | 1 + ...df_like-0-e0ba9a953e50554bdcbc55585cffde09 | 1 + ...df_like-1-9781f89d352c506e972ad2a1d58ec03a | 4 + ...df_like-2-dbc46cb33f0dd356af03006d9492f8b7 | 0 ...df_like-3-bef03784eab9d5e8404fd24960dea4fc | 1 + ...df_like-4-af5fe5d5d176f751747bf14055d00a12 | 1 + .../udf_ln-0-779eed5722a0efaa85efe24c559072b4 | 1 + .../udf_ln-1-60e3541b3c703d6413869d774df9b7e4 | 4 + ..._locate-0-6e41693c9c6dceea4d7fab4c02884e4e | 1 + ..._locate-1-d9b5934457931447874d6bb7c13de478 | 4 + ..._locate-2-849e7058dbbd7d422a9f3eb08d85b15c | 0 ..._locate-3-2a260e4b8e909eb5e848bf31a07f2531 | 1 + ...udf_log-0-ca773bc1afa66218f3c13dee676bd87a | 1 + ...udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 | 4 + ...f_log10-0-35028570b378a2c7ea25b6bf6a4fac1f | 1 + ...f_log10-1-abf1173290ef905d24d422faf7801fe3 | 4 + ...df_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 | 1 + ...df_log2-1-a79f0dce2cfc000b11a3b5299f02db56 | 4 + ...boolean-0-2e7b9484514a049bbf72a4a0af5ee127 | 10 + ...f_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 | 1 + ...df_lower-1-550f0a6da388596a775d921b9da995c | 5 + ...f_lower-2-467230d9c8442c726e5377b70257ff5e | 0 ...f_lower-3-61b2e3e72180c80d52cf9bed18125e08 | 1 + ...df_lpad-0-995646acf1e23cea7825412915921bef | 1 + ...df_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 | 6 + ...df_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 | 0 ...df_lpad-3-5b04264ae9ada1304acd0410ce31f2ae | 1 + ...f_ltrim-0-398a623504c47bcd64fe8d200c41402f | 1 + ...udf_ltrim-1-658d495908097792a0e33a77becac2 | 4 + ...udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e | 1 + ...udf_map-1-1f8cd98df9bf7b2528506551fef87dcf | 1 + ...udf_map-2-a3f90085abab46205e732b4c27b18340 | 0 ...udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 | 1 + ...ap_keys-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...ap_keys-1-9a5714f8790219e9a9708a2c45fc87aa | 1 + ...ap_keys-2-731b529a9a234473312b7d1db15be75f | 1 + ...ap_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e | 1 + ...ap_keys-4-10cb8da631c1c058dacbbe4834a5928a | 1 + ..._values-0-e86d559aeb84a4cc017a103182c22bfb | 0 ..._values-1-a1d9810d9793d4ca2d17f539ca72bd08 | 1 + ..._values-2-ed39a40cbe55bb33d2bc19f0941dae69 | 1 + ..._values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 | 1 + ..._values-4-a000d06dd3941756b4bb9ccc46f3620e | 1 + ...udf_max-0-ac7d002a46f773ab680ed8c1ac97821f | 1 + ...udf_max-1-14afa1f14687893233a662f0f32a40c9 | 1 + ...df_max-10-1e9209f5b2ba926df36b692a4dcd09f6 | 1 + ...df_max-11-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...df_max-12-a00d1791b7fa7ac5a0505d95c3d12257 | 1 + ...df_max-13-1e9209f5b2ba926df36b692a4dcd09f6 | 1 + ...udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 + ...udf_max-3-a7dc16cb82c595b18d4258a38a304b1e | 1 + ...udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 | 1 + ...udf_max-5-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...udf_max-6-a7dc16cb82c595b18d4258a38a304b1e | 1 + ...udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 | 1 + ...udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 | 1 + ...udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 | 1 + ...udf_min-0-1a67398a47b4232c3786485b451d1ff8 | 1 + ...udf_min-1-69d749d0bca0ebe56e930009e30f4f19 | 1 + ...df_min-10-191613d4d46d1884d0694fcd8c5fb802 | 1 + ...df_min-11-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...df_min-12-a00d1791b7fa7ac5a0505d95c3d12257 | 1 + ...df_min-13-191613d4d46d1884d0694fcd8c5fb802 | 1 + ...udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 + ...udf_min-3-a7dc16cb82c595b18d4258a38a304b1e | 1 + ...udf_min-4-191613d4d46d1884d0694fcd8c5fb802 | 1 + ...udf_min-5-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...udf_min-6-a7dc16cb82c595b18d4258a38a304b1e | 1 + ...udf_min-7-191613d4d46d1884d0694fcd8c5fb802 | 1 + ...udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 | 1 + ...udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 | 1 + ..._minute-0-9a38997c1f41f4afe00faa0abc471aee | 1 + ..._minute-1-16995573ac4f4a1b047ad6ee88699e48 | 7 + ..._minute-2-2bf0b45e6608d66c7d7a5b320f662d75 | 0 ..._minute-3-270055c684846e87444b037226cf554c | 1 + ..._modulo-0-4e06551d4aa9464492e0f53374a280d5 | 1 + ..._modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 | 1 + ...f_month-0-9a38997c1f41f4afe00faa0abc471aee | 1 + ...f_month-1-16995573ac4f4a1b047ad6ee88699e48 | 7 + ...negative-0-1b770ec6fb07bb771af2231a9723ec8 | 1 + ...egative-1-5f64266721b1ed31cfe84ee2f2377bdf | 1 + ...egative-2-a6863d2c5fc8c3131fe70080a011392c | 1 + ...egative-3-b90eec030fee9cbd177f9615b782d722 | 1 + ...egative-4-e27bf3f44ccb2e051877da8a4455f50c | 1 + ..._negative-5-93d7dd808d4af59bda601faf249a9e | 1 + ...egative-6-6758b00c5acc7aac320238accf299219 | 1 + ...negative-7-6d8783f0ed7a4b7058c95f90da3fb4b | 1 + ...egative-8-634af0478ed9ed44b851cd7ef834a489 | 1 + ...egative-9-80b4c1fe00f7997838bba64a2cb5f8aa | 1 + ...udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc | 1 + ...udf_not-1-efefc8302b02224d20f4bb0f159a6911 | 2 + ...udf_not-2-7e63750d3027ced0e3452ad4eb0df117 | 1 + ...udf_not-3-aa0c674f9ce0feba86448448a211bd2a | 2 + ...otequal-0-27c0d57f7c7c48ceb7bc671f7160254e | 1 + ...otequal-1-a7f0d1064f8f398ef504008015fddf9a | 2 + ...otequal-2-7d815b1218c85e4cf69d6780cab17520 | 1 + ...otequal-3-b72baeb22fad61bb31ce2d2e69375f57 | 2 + ...otequal-4-eb04e5ee00967799c913e8a5b424a332 | 0 ...otequal-5-e361b9cf294c4aa25018b081a2c05e07 | 499 ++++ ...otequal-6-46a6514f2d7e6a097035ec1559df0096 | 0 ...otequal-7-a71fea4e9514cda0da9542a7701613dd | 499 ++++ ...f_notop-0-825431072651228a5a9de7f85a0498d6 | 1 + ...udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 | 1 + ...udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 | 4 + ...udf_nvl-2-175ed7006e8907b65e0e5357f00a0def | 0 ...udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 | 1 + .../udf_or-0-c404aa929eb0dd87269121f8f99ada70 | 1 + .../udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 | 1 + ...rse_url-0-7571c0423df7bf158ea9ca98142b26b8 | 1 + ...rse_url-1-67adfb10d4a35c4d031f26adde9f61ab | 10 + ...rse_url-2-ed7adf45f58ad014cf9f3d653b380a4d | 0 ...rse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 | 1 + ...centile-0-8f99f54ff944f252e47d0af1f4ed1553 | 1 + ...centile-1-c0825a744cd14917d2c904d014449a4a | 1 + ...centile-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 + ...centile-3-a7dc16cb82c595b18d4258a38a304b1e | 1 + ...df_pmod-0-ed67184beaf84c0542117c26651938e1 | 1 + ...df_pmod-1-90f75e01dcee85253a501d53b8562dae | 1 + ...f_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 | 1 + ...df_pmod-2-8ac9813b27801704082c6e9ea4cdc312 | 1 + ...df_pmod-3-26d9546f030281a29a50a3e8e5858234 | 1 + ...df_pmod-4-7695df16d24a821224676e6bad3d66d1 | 1 + ...df_pmod-5-cf5311d51d44afb8d73f588e27d5e029 | 1 + ...udf_pmod-6-3c09a8da2f5645e732c22a45d055125 | 1 + ...df_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 | 1 + ...df_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd | 1 + ...df_pmod-9-e7280393102077442aa1d10eb69a6d57 | 1 + ...ositive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 | 1 + ...ositive-1-1b98434a841d2248ed985c5f6ba2cc3c | 1 + ...ositive-2-610d421e590f035c24e29694a68b0d23 | 1 + ...ositive-3-400b238f4e6cdf7120be566b0ef079c5 | 1 + ...udf_pow-0-c7f5178951dd45dc2a41c16729314d81 | 1 + ...udf_pow-1-3c22c000c35144135aedbc7052f10803 | 5 + ...df_power-0-57001d802c281743322d28bbc520cd4 | 1 + ...f_power-1-ebd0398b2cb03f382a16382ddac13426 | 5 + ...radians-0-f899daf93b02ca681e0230a792c65e86 | 0 ...radians-1-eaaa62dd3935ff3152323dfafd136e93 | 1 + ...radians-2-bcaca433f704f71cf9d44c238a33c7b3 | 1 + ...radians-3-65e16c7b13de48a5d36793d0c7d35e14 | 1 + ..._radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 4 + ...radians-5-f899daf93b02ca681e0230a792c65e86 | 0 ...radians-6-eaaa62dd3935ff3152323dfafd136e93 | 1 + ...radians-7-bcaca433f704f71cf9d44c238a33c7b3 | 1 + ...radians-8-65e16c7b13de48a5d36793d0c7d35e14 | 1 + ..._radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 4 + ...df_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 | 1 + ...df_rand-1-c6229b8f2ca3001663229cfb8ee4763e | 1 + ...reflect-0-904138e2a1f831c308b7f0aacc859ae1 | 1 + ...reflect-1-21ec7caa253c7f95b7cf60191140e2ee | 4 + ...reflect-2-b868357466bab2f04685c2dc73604cf0 | 0 ...eflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 | 1 + ...eflect2-1-c5a05379f482215a5a484bed0299bf19 | 3 + ...eflect2-2-effc057c78c00b0af26a4ac0f5f116ca | 0 ..._regexp-0-19917611f74aedc0922560f7f2595948 | 1 + ..._regexp-1-f7f0527cd47612d7f256edd5f8963800 | 5 + ..._regexp-2-5e1798db3ba058e7b202d8a98f228b11 | 1 + ...extract-0-e251e1a4b1e191814f26c54b14ab6cd9 | 1 + ...extract-1-8add879ab5904bd805412ef8723276fb | 4 + ...replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e | 1 + ..._replace-1-e79b45aa220d3c4c3b4523ac9c897bc | 4 + ..._repeat-0-fdeae3e88f27ef148281d69ac8c4b23b | 1 + ..._repeat-1-836be47190989d8975a09a545ecbfe0b | 4 + ..._repeat-2-e1dbea7182ec1653e1123b5b67a6d20a | 0 ..._repeat-3-ba9dd02f59c74d63d60d60b6231a0365 | 1 + ...f_rlike-0-6ec6ef55ac041208627454e16b501d38 | 1 + ...f_rlike-1-829611a596e0c87431006f7247d25eca | 5 + ...f_round-0-10b53ca1f15fd7879365926f86512d15 | 1 + ...f_round-1-2367bcc43510dedc80bdb6707e434da8 | 4 + ...f_round-2-9ffa2b573360cd879338de46d91ab374 | 1 + ...f_round-3-42a221909d3f7ed51bed01a65670461c | 1 + ...f_round-4-dc80ec5189a4c6ce07688df3debe7de4 | 1 + ...f_round-5-a2414e7845ffafc61f75465508a1850a | 1 + ...f_round-6-48439efa5c34e7589ab5003ed916f12b | 1 + ...round_2-0-91afaf77ef4061fa20005a3c87dfef32 | 0 ...round_2-1-5e44354af73865d03e9088c0232f16ce | 0 ...round_2-2-83f91f60dcb2036b61b8b21f18281298 | 1 + ...round_2-3-c62cf7b74a91f605cf669e34a9315f93 | 1 + ...round_2-4-797fa8ed05cb953327d0f6306b30d6c3 | 1 + ...round_2-5-e41b862db8cc76c1fe004cf006fad50b | 1 + ...round_3-0-8415af605db167315e4d9d3c69d89e6c | 1 + ...round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 | 1 + ...round_3-2-5e7ada966f726ceb748f98c20eab4f10 | 1 + ...round_3-3-e94ab3326df006c7203ead86752f16a9 | 1 + ...round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 | 1 + ...df_rpad-0-56de896c5fe8b40b22a9ed55ed79889c | 1 + ...df_rpad-1-48d17e5d5d6188300d048f987fab2ca0 | 6 + ...udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 | 0 ...df_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 | 1 + ...f_rtrim-0-7acca21c725995febdf2a9c1fdf0535a | 1 + ...f_rtrim-1-66d61255134c09d37cbfedd757ae47fd | 4 + ..._second-0-e004a6f20da3fa6db868ec847b217ff2 | 1 + ..._second-1-3525f55f4f13253c42b3abaa53d77888 | 7 + ..._second-2-d678372e3837a16be245d2e33482f17f | 0 ..._second-3-2496e4d3c64ca028184431c2930d82cf | 1 + ...ntences-0-ec0dad44fa033691a731f6e4c6b5cf7f | 0 ...ntences-1-2dc07f4f0f0cb20d08c424e067ed8f69 | 0 ...tences-10-ce188a53f69129c14cbf378d2c3f6630 | 0 ...tences-11-3c8672506e23434187caf4e0064a8a80 | 31 + ...tences-12-d55c04a079ca97402509868f24921685 | 1 + ...ntences-2-1f218343f90e698fb9ed81c4f51d3d14 | 0 ...ntences-3-ce188a53f69129c14cbf378d2c3f6630 | 0 ...ntences-4-3c8672506e23434187caf4e0064a8a80 | 50 + ...ntences-5-60823f4912be26bee1a0b52a0a9588a9 | 0 ...ntences-6-27b7eeae683a87507f35e61fd4ce67de | 0 ...ntences-7-ec0dad44fa033691a731f6e4c6b5cf7f | 0 ...ntences-8-2dc07f4f0f0cb20d08c424e067ed8f69 | 0 ...ntences-9-68c61b4882802e416d5adaa2de440b59 | 0 ...df_sign-0-14f3c3034ac8039fc81681003bbf5e0e | 0 ...df_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 | 1 + ...f_sign-10-fc7341f89c3cd3c43e436242d8aa61fa | 1 + ...f_sign-11-74237f5ecc497813cf9738b21647847a | 4 + ...df_sign-2-85b743f0eed93904789cde4d1b5eafef | 1 + ...df_sign-3-9eeb3985359429abba5d1dd702c66b0d | 1 + ...df_sign-4-fc7341f89c3cd3c43e436242d8aa61fa | 1 + ...df_sign-5-74237f5ecc497813cf9738b21647847a | 4 + ...df_sign-6-14f3c3034ac8039fc81681003bbf5e0e | 0 ...df_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 | 1 + ...df_sign-8-85b743f0eed93904789cde4d1b5eafef | 1 + ...df_sign-9-9eeb3985359429abba5d1dd702c66b0d | 1 + ...udf_sin-0-40b50393869eb0bcde66e36fe41078ee | 1 + ...udf_sin-1-2f867f432fb322e21dce353d7eb50c63 | 4 + ...udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab | 1 + ...udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 | 1 + ...df_size-0-9c49a52514b1b940a7e8bb93c35eda62 | 1 + ...df_size-1-3608160636eaa7e053171bdcefc0b1a8 | 1 + ...df_size-2-96d41c9f054662827d1b6b63f5dd8db7 | 0 ...df_size-3-63df892215b6ce189d6c3e20cfc4bdbd | 1 + ...mallint-0-f28e857ef74c967303855c21dc60c042 | 1 + ...mallint-1-37d8db74267ae370d6a076b3057c5ed6 | 1 + ...t_array-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...t_array-1-976cd8b6b50a2748bbc768aa5e11cf82 | 1 + ...t_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 | 4 + ...t_array-3-55c4cdaf8438b06675d60848d68f35de | 0 ...t_array-4-3edb0151fae0622cb79cd04156cb4c44 | 1 + ...t_array-5-5d7dfaa9e4137938559eb536e28f8d0e | 1 + ...t_array-6-f754ac1296d16e168abb3d0ebcc35bd3 | 1 + ...t_array-7-a9c52a8684cefc095470f5a93f63d2a8 | 1 + ...t_array-8-d79f0084177230a7a2845c4791c22d25 | 0 ...t_array-9-45ef2679e195a269878527d5f264488a | 0 ...f_space-0-91e879c9f01d90eab7bf12fcef256010 | 1 + ...f_space-1-e4eaf5e96807e122548cb43be9a26754 | 4 + ...f_space-2-a23a06eef37709e8587647a74bbfa7e8 | 0 ...f_space-3-59903e27d8188d6209e007ff643d5956 | 1 + ...f_space-4-7adb35ad867ba3e980d937a0038ac1a5 | 1 + ...f_split-0-7accac7fc71ba74d61c01a69d3978338 | 1 + ...f_split-1-ebe303dfc3db504deffe5c355efd0fd1 | 4 + ...f_split-2-7bba11f8fc359f2d4b863cda11c643f9 | 0 ...f_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 | 1 + ...df_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 | 1 + ...df_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 | 4 + ...udf_std-0-e3613484de2b3fa707995720ec3f8a5b | 1 + ...udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 | 2 + ..._stddev-0-ad7627185d89a60b83ce19966eddbc92 | 1 + ..._stddev-1-18e1d598820013453fad45852e1a303d | 2 + ...dev_pop-0-96788538f1f20eb879a1add4bb5f9d12 | 1 + ...dev_pop-1-6286ef94de26050617bf69c17a3b4a10 | 1 + ...dev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 | 1 + ...ev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 | 1 + ...dev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 | 1 + ...ev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 | 1 + ..._string-0-17412ad1c1a827411caa7b5e891b6ac3 | 1 + ..._string-1-53b00551846b7f8bb27874b3a466e68d | 1 + ..._substr-0-20fb50d79b45264548b953e37d837fcd | 1 + ..._substr-1-2af34de7fd584c5f1ead85b3891b0920 | 10 + ..._substr-2-d0268ad73ab5d94af15d9e1703b424d1 | 1 + ..._substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 | 1 + ..._substr-4-f7933c0bb28e9a815555edfa3764524a | 1 + ..._substr-5-4f29c1273b8010ce2d05882fc44793d8 | 1 + ..._substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 | 1 + ..._substr-7-1742c72a1a8a6c6fd3eb2449d4747496 | 1 + ...bstring-0-8297700b238f417dea2bd60ba72a6ece | 1 + ...bstring-1-e5df65419ecd9e837dadfcdd7f9074f8 | 10 + ...ubtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 | 1 + ...ubtract-1-b90eec030fee9cbd177f9615b782d722 | 1 + ...udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 | 1 + ...udf_sum-1-ddae1a511d3371122ab79918be5b495b | 1 + ...udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 | 1 + ...udf_sum-3-ddae1a511d3371122ab79918be5b495b | 1 + ...udf_tan-0-c21aa640b4edabf6613dd705d029c878 | 1 + ...udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 4 + ...udf_tan-2-27a29c68f846824990e9e443ac95da85 | 1 + ...udf_tan-3-77bedd6c76bdc33428d13a894f468a97 | 1 + ...udf_tan-4-c21aa640b4edabf6613dd705d029c878 | 1 + ...udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 4 + ...udf_tan-6-27a29c68f846824990e9e443ac95da85 | 1 + ...udf_tan-7-77bedd6c76bdc33428d13a894f468a97 | 1 + ...tlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad | 10 + ...length2-0-3da4fe901124f2bbf3d02822652c4e55 | 10 + ...tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d | 1 + ...tinyint-1-417de1aeb44510aa9746729f9ff3b426 | 1 + ...boolean-0-35e75ee310b66710e2e88cf0fecca670 | 1 + ...boolean-1-cbead694a25ec357d69fd008776e19c9 | 1 + ...boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 | 1 + ...boolean-3-266b9601a9154438ab95550fcd36494c | 1 + ...boolean-4-2602c9a6c910ec7fdd439212c648333d | 1 + ...boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad | 1 + ...boolean-6-1e29d35d9b02f6800c8939910a1e4c2f | 1 + ...boolean-7-93d8249c035f34bfca2fa041ae97f55a | 1 + ...to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 | 1 + ...to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 | 1 + ...to_byte-2-108a5c01e12e8823f6affc3d1664a1fc | 1 + ...f_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b | 1 + ...to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 | 1 + ..._to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 | 1 + ...to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 | 1 + ...to_byte-7-4b9de0160a65b212cbe42059d98aa718 | 1 + ...to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 | 1 + ...to_date-0-ab5c4edc1825010642bd24f4cfc26166 | 1 + ...to_date-1-da3c817bc5f4458078c6199390ac915e | 4 + ..._double-0-cbe030be095a93a9ae810ce7e66bdca7 | 1 + ..._double-1-98d2ce732277c3a7fb4827d8b221a43a | 1 + ..._double-2-69bf8a5a4cb378bbd54c20cb8aa97abe | 1 + ..._double-3-ab23099412d24154ff369d8bd6bde89f | 1 + ..._double-4-293a639a2b61a11da6ca798c04624f68 | 1 + ..._double-5-42d1e80bb3324030c62a23c6d1b786a8 | 1 + ..._double-6-5bac1a7db00d788fd7d82e3a78e60be6 | 1 + ..._double-7-97080ab9cd416f8acd8803291e9dc9e5 | 1 + ..._double-8-df51146f6ef960c77cd1722191e4b982 | 1 + ...o_float-0-7646eca02448547eedf84a81bf42be89 | 1 + ...o_float-1-eac0237ee0294d635c2b538f6e2f0a5c | 1 + ...o_float-2-39a67183b6d2a4da005baed849c5e971 | 1 + ...o_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce | 1 + ...o_float-4-f3e0ab8ed691a386e6be4ce6993be507 | 1 + ...o_float-5-75f364708c01b5e31f988f19e52b2201 | 1 + ...o_float-6-1d9b690354f7b04df660a9e3c448a002 | 1 + ...o_float-7-cdfefa5173854b647a76383300f8b9d1 | 1 + ...o_float-8-5b0a785185bcaa98b581c5b3dbb3e12c | 1 + ...to_long-0-ebe447e8fb03de8844951250fe50320a | 1 + ...to_long-1-61e6679e5a37889bc596590bde0228f0 | 1 + ...to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 | 1 + ...to_long-3-cb3318ba365833316645e1b9890f4613 | 1 + ...to_long-4-55b7b7d046541d63e30f76111734c0f6 | 1 + ...to_long-5-a0d382d243a226f8d4381970b6831c3d | 1 + ...to_long-6-47ab11eae68329cc80232fc4089479f0 | 1 + ...to_long-7-b87c0cabb166f33984cc1b191694918e | 1 + ..._to_long-8-52a24d7040db321a842f9201d245ee9 | 1 + ...o_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a | 1 + ...o_short-1-32c4a61f9166c2a9d7038553d589bd9b | 1 + ...o_short-2-a4ab375c99efbfe0d5a45941552509c8 | 1 + ...o_short-3-8d5108d9585d8ca6ca3eefac05afc3ce | 1 + ...o_short-4-8d76a5a410e5533f2ae625504478cda8 | 1 + ...o_short-5-d467dc39319796ff5cbe600ff6530ad5 | 1 + ...o_short-6-c3b304122e874868f37c6fbaeccf0397 | 1 + ...o_short-7-827f3181c216fd2e990637c9a091bf0d | 1 + ...o_short-8-d35f9502fc7b4205e18ecdf53189bdc4 | 1 + ...o_string-0-a032eb8f819689a374852c20336d5cc | 1 + ..._string-1-b461f0e6e98991aede40c7c68561dc44 | 1 + ..._string-2-bc6b37e1d5a8d28df8a957c88104c9a5 | 1 + ..._string-3-ddbb829eedc6aa1e27169303a7957d15 | 1 + ..._string-4-c20301e9bbf10143bb9bf67cd7367c21 | 1 + ...to_string-5-1040b37847d20ef29d545934316303 | 1 + ..._string-6-4181d264a7af3c6488da2f1db20dc384 | 1 + ..._string-7-567bc77f96e7dc8c89bae912e9a3af15 | 1 + ..._string-8-a70b03d79ebd989c8141f9d70dbca8ea | 1 + ..._string-9-51677fbf5d2fc5478853cec1df039e3b | 1 + ...anslate-0-7fe940a2b26fa19a3cfee39e56fb1241 | 1 + ...anslate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 | 13 + ...nslate-10-2f9daada8878081cb8954880ad5a65c4 | 1 + ...nslate-11-76b7a339d5c62808b9f4f78816d4c55b | 1 + ...nslate-12-a5b3e5fadeec1e03371160879f060b05 | 1 + ...anslate-2-42aba80bf1913dd7c64545831f476c58 | 0 ...anslate-3-20904c8be8fed5cbd2d66ead6248a60a | 0 ...anslate-4-5d4abaf86254bacaa545c769bd7e50ba | 0 ...anslate-5-f2637240d227f9732d3db76f2e9d3a59 | 0 ...anslate-6-55122cc5ea4f49e737fee58945f3f21b | 1 + ...anslate-7-856c9d0157c34ab85cc6c83d560bfd47 | 1 + ...anslate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 | 1 + ...anslate-9-ee69663d7662760973b72785595be2b1 | 1 + ...df_trim-0-18aa2b7ff8d263124ea47c9e27fc672f | 1 + ...df_trim-1-e23715e112959e6840b6feed2ecf38a7 | 4 + ...f_ucase-0-8f8c18102eb02df524106be5ea49f23d | 1 + ...f_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f | 5 + ...f_unhex-0-11eb3cc5216d5446f4165007203acc47 | 1 + ...f_unhex-1-a660886085b8651852b9b77934848ae4 | 14 + ...df_unhex-2-78ba44cd7dae6619772c7620cb39b68 | 1 + ...f_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 | 1 + ...f_union-0-e3498ef95bc9d8c8ce55e75962b4a92c | 1 + ...f_union-1-f6f01250718087029165e23badc02cd6 | 4 + ...f_union-2-6af20858e3209d7cf37f736805ec5182 | 0 ...f_union-3-705d165fec6761744dd19b142c566d61 | 2 + ...f_upper-0-47dc226b0435f668df20fe0e84293ead | 1 + ...f_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d | 5 + ...var_pop-0-3187e740690ccc1988a19fea4202a6de | 1 + ...var_pop-1-fd25e5226312bf54d918858511814766 | 2 + ...ar_samp-0-b918928871d1b7f944315558c230c229 | 1 + ...ar_samp-1-59032ed5856fd4aa17c3e8e6721eec2b | 1 + ...ar_samp-2-b918928871d1b7f944315558c230c229 | 1 + ...ar_samp-3-59032ed5856fd4aa17c3e8e6721eec2b | 1 + ...ariance-0-fd23be1baa8b5ffa0d4519560d3fca87 | 1 + ...ariance-1-c1856abae908b05bfd6183189b4fd06a | 2 + ...ariance-2-3187e740690ccc1988a19fea4202a6de | 1 + ...ariance-3-fd25e5226312bf54d918858511814766 | 2 + ...ariance-4-fd23be1baa8b5ffa0d4519560d3fca87 | 1 + ...ariance-5-c1856abae908b05bfd6183189b4fd06a | 2 + ...ariance-6-3187e740690ccc1988a19fea4202a6de | 1 + ...ariance-7-fd25e5226312bf54d918858511814766 | 2 + ...kofyear-0-d6b4490b549a358be375511e39627dc2 | 1 + ...kofyear-1-b7bbdfabe6054a66701250fd70065ddd | 6 + ...kofyear-2-57ec578b3e226b6971e0fc0694b513d6 | 1 + ...df_when-0-88b97c6722176393e9b3d089559d2d11 | 1 + ...df_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 | 1 + ...df_when-2-ff1118e7d06a4725e1e98a6d70a59295 | 0 ...df_when-3-e63043e8d6ecf300c1fcf8654176896f | 1 + ...df_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 | 1 + ...f_xpath-1-5a6f10392053a4eabe62f5cabb073a71 | 8 + ...f_xpath-2-6b0a9d3874868d88d54ae133c978753d | 1 + ...f_xpath-3-5700d81a9c2a22dcca287bf8439277ea | 1 + ...f_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a | 1 + ...f_xpath-5-7395e1cd3b543316a753978f556975e0 | 1 + ...f_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f | 1 + ...boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 | 1 + ...boolean-1-9e935539c9a3f3a118145096e7f978c4 | 6 + ...boolean-2-66148c16fde3008348a24cec5d9ba02b | 1 + ...boolean-3-c6cf7ede46f0412fe7a37ac52061b060 | 1 + ...boolean-4-45d2e8baee72a0d741eb0a976af3a965 | 1 + ...boolean-5-dbc57416174a6ba26f29dfc13f91f302 | 1 + ...boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f | 1 + ...boolean-7-824c07ed4ef40cd509fea55575e43303 | 1 + ..._double-0-39199612969071d58b24034a2d17ca67 | 1 + ..._double-1-2d5ceab0a14d6e837ed153e1a2235bb2 | 5 + ...double-10-fe9ae5013ee4f11b357128ee5ffb56c0 | 1 + ...double-11-a1960676c1b2dc76aa178ea288d801d2 | 1 + ..._double-2-a4d22cea9dffaf26b485b3eb08963789 | 1 + ..._double-3-ea050c5b9e74d4b87b037236ef9e4fc2 | 5 + ..._double-4-93f47057c68385cff3b6f5c42307590c | 1 + ..._double-5-c811a2353f5baf585da8654acd13b0e5 | 1 + ..._double-6-10fcb281fd8db12dd1ac41380b8030c6 | 1 + ..._double-7-4532934141d2adb1f4ff56e65e8cf14c | 1 + ..._double-8-547e750f5e401511db56283e66d1231d | 1 + ..._double-9-b45b8b60031ac43ed2ffcd883ba2f19e | 1 + ...h_float-0-7483dafee0dc7334eecabba31977e791 | 1 + ...h_float-1-f6ddff2daba006787aeb861ca6f9d37a | 4 + ...h_float-2-70b7180f7bcd1d09727ad73084788d16 | 1 + ...h_float-3-79b7e57efe31a0373c39f0ba79b5bd54 | 1 + ...h_float-4-6e1f61ebe10c5fde60148e3a31706352 | 1 + ...h_float-5-410760f9560157587fbba7a677e12b9f | 1 + ...h_float-6-d83a5002f52e3acb7dbb725bb434eebf | 1 + ...h_float-7-b57c58f9343a82846f54ef60b216dfaa | 1 + ...h_float-8-3a62c730d4e5d9be6fb01307577d6f48 | 1 + ...h_float-9-16793f703b552fcb24f7aea9bb8a2401 | 1 + ...ath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 | 1 + ...ath_int-1-26ddf7e46a66065909e9e547f0459910 | 4 + ...ath_int-2-f10e246ebfd5f82545a3df64f51f58ba | 1 + ...ath_int-3-eaa4a790c726cfa76a247d08a6ee7aba | 1 + ...ath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 | 1 + ...ath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 | 1 + ...ath_int-6-9bfa4fdc7d551742ff23efad8507ba0a | 1 + ...ath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 | 1 + ...ath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 | 1 + ...ath_int-9-e6c2548098879ee747a933c755c4c869 | 1 + ...th_long-0-d274b272251e95ec2e8139bfa471bd0b | 1 + ...th_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 | 4 + ...th_long-2-d697d943b1f7e7762d804064d11b905b | 1 + ...th_long-3-c0b20c651ae5a352322067f97bd6ae5d | 1 + ...ath_long-4-821e1cdea837bee7a8c2745bc3b85b9 | 1 + ...th_long-5-22a71b88c5bcb3db6e299a88ab791d4d | 1 + ...th_long-6-f2460325cf46c830631d8bc32565c787 | 1 + ...th_long-7-b3f1d4b505151180b82fddb18cf795d0 | 1 + ...th_long-8-c21bebec7d1a4aec99fba6b0a9a03083 | 1 + ...th_long-9-7bafedc7b884df49a9f6752360969bf1 | 1 + ...h_short-0-7d5231aed9cbbf68cd740791f9e5be17 | 1 + ...th_short-1-5d45932563b78e0b435b8cfebfe3cc2 | 4 + ...h_short-2-10e9d4899d2fd352b58010c778c1f7a8 | 1 + ...h_short-3-5ac84f46610107b1846f4a9b26a69576 | 1 + ...h_short-4-8a300079521fefbe0d2f943851c1c53c | 1 + ...h_short-5-51f5de44cf1d5289fa5892ffe16e473e | 1 + ...h_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 | 1 + ...h_short-7-e24ee123f331429c22de0a06054d0d5d | 1 + ...h_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 | 1 + ...h_short-9-22088ba0fb00eaa28e3460ca018b343e | 1 + ..._string-0-e315d11e9feb29177b5cb1e221c7cfa4 | 1 + ..._string-1-110b583cde6cd23c486d8223c444cbe9 | 10 + ..._string-2-a147b4eaa40b03355f666ea660cbff1f | 1 + ..._string-3-a62072b86a6044d5b97911d662899b5a | 1 + ..._string-4-152b0424ec68120f4ef15269eac0528a | 1 + ..._string-5-2ab527e587583138ca764f5518a88c14 | 1 + ..._string-6-58204a4df0fbf861028a1a94fb20ec61 | 1 + ..._string-7-b5b211e4abd14990c28007c6638cb44f | 1 + ..._string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc | 1 + ..._string-9-647aca756f43e791dd5497e1b4c6af89 | 1 + ...f_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a | 1 + ...f_stack-1-879ca1a8453ced55a8617b390670a4e1 | 0 ...f_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 | 0 ...otation-0-8ee564d593fc64e0ad8a52b5d659f286 | 0 ...otation-1-3de206f543c9e1525c54547f076b99c3 | 27 + ...otation-2-9a9c2395773e362f0db32d5603673291 | 0 ...otation-3-4ac0de021c370cc6b8e753e00f96682e | 0 ...otation-4-3de206f543c9e1525c54547f076b99c3 | 27 + ...otation-5-9a9c2395773e362f0db32d5603673291 | 0 ...otation-6-b52052b427ea699f7bc7bee7e32d1de9 | 0 ...otation-7-3de206f543c9e1525c54547f076b99c3 | 27 + ...otation-8-9a9c2395773e362f0db32d5603673291 | 0 ...union10-0-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...union10-1-c5b4e04f745a28463e64aeeec6d4b2af | 0 ...union10-2-60ee9eae687170317ff91dafe6b799bf | 0 ...union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 | 0 ...union10-4-7f83822f19aa9b973198fe4c42c66856 | 3 + ...union11-0-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...union11-1-a6f37db310de2edce6696c25a4fe96e3 | 0 ...union11-2-cd756f39e22e121bdbd51400662aa47f | 3 + ...union12-0-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...union12-1-9d2793d1cfd2645ac7f373a0a127e599 | 0 ...union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 | 0 ...union13-0-6970d1e2405f3769a28a8804887ac657 | 0 ...union13-1-534d0853c5fc094404f65ca4631c1c20 | 1000 +++++++ ...union14-0-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...union14-1-11603fb6225423979af6c062cfa9389b | 0 ...union14-2-8e01b2f4a18ad41a622e0aadbe680398 | 17 + ...union15-0-863233ccd616401efb4bf83c4b9e3a52 | 1 + ...union15-1-caf8a4bf47a5f5dfd27388fe00686c56 | 0 ...union15-2-a89acfb4bbc044c483b94e28152a41e0 | 17 + ...union16-0-7d7f87e388835405f8c7baba969e04f5 | 0 ...union16-1-9f76074598f9b55d8afbb5659737a382 | 1 + .../union17-0-383b0c63b3fe31509d23612cb6ae88a | 0 ...union17-1-9491c271b7e2c351ddcf126f73679892 | 0 ...union17-2-e6fb104913b9da8193167ee263993dd1 | 0 ...union17-3-eef8248caa65e21a7c2956aa488297a0 | 0 ...union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 310 ++ ...union17-5-404392d6faff5db5f36b4aa87ac8e8c9 | 310 ++ .../union18-0-383b0c63b3fe31509d23612cb6ae88a | 0 ...union18-1-9491c271b7e2c351ddcf126f73679892 | 0 ...union18-2-584e61e1599fe449cc998a3332dcb7e5 | 0 ...union18-3-b86b4936bb620a8f5d929943f4aa75f2 | 0 ...union18-4-1799ebb147238db6032fd6fe2fd36878 | 501 ++++ ...union18-5-b12dcddfa4f02a14318f6564947c98a0 | 501 ++++ .../union19-0-383b0c63b3fe31509d23612cb6ae88a | 0 ...union19-1-9491c271b7e2c351ddcf126f73679892 | 0 ...union19-2-b6161b64e57a2502c79662866782cbdf | 0 ...union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 | 0 ...union19-4-1799ebb147238db6032fd6fe2fd36878 | 310 ++ ...union19-5-b12dcddfa4f02a14318f6564947c98a0 | 501 ++++ .../union2-0-58813daf3d1af57b17518b606f8345b2 | 0 .../union2-1-90d739774cb96e7d0d96513c1c9968b4 | 1 + ...union20-0-a7f64187712936e741d5c33b2f5dfa6d | 0 ...union20-1-968e353589f1fddb914242beb25be94c | 23 + ...union21-0-ecfd22e2a24ed9f113229c80a2aaee9c | 0 ...union21-1-fb1497f4c21bf7d28162f27d50320d13 | 536 ++++ ...union22-0-4bd6583b3635f1db95765565518bda8d | 0 ...union22-1-9037e5e0cb2d64e4efe6280dc03396f7 | 0 ...nion22-10-90e9c4388340428a1f68868e9322c400 | 1016 +++++++ ...union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 | 0 ...union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 | 0 ...union22-4-dc129f70e75cd575ce8c0de288884523 | 1 + ...union22-5-24ca942f094b14b92086305cc125e833 | 1 + ...union22-6-44d382ce6848d3f0b900b0808747d8e9 | 1 + ...union22-7-25fc734982956a164adde6bb1d4d8751 | 1 + ...union22-8-c78e8af8e8e327c58a74f7cb34a1c912 | 0 ...union22-9-f4414e4636a16596d04fcc433d1119b6 | 0 ...union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 | 0 ...union23-1-7830963417e3535034962e2597970ddd | 1000 +++++++ ...union24-0-5659f2e36b79fa312aaf6e233dc575c9 | 0 ...union24-1-92aaed1a225be0e1b3d4556680a77b8c | 0 ...union24-10-ae53b6e1d433f000da3fbe130222b89 | 0 ...nion24-11-a41e75063b4cac0dbaf99b4aabc3201a | 18 + ...union24-2-4afc74338258d50ae6ecdb0589bd2a38 | 0 ...union24-3-d3370b90a3ace4b4d1cefd1ffc79906f | 0 ...union24-4-dc129f70e75cd575ce8c0de288884523 | 1 + ...union24-5-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...union24-6-83d9274a535ebebe0170ce8f16062ef4 | 0 ...union24-7-afb3649d2de5113691a37d0c77b28b31 | 24 + ...union24-8-b6d9a41e2b537869e977cfb969b51edb | 0 ...union24-9-db8ce0526e33b73f5589e3e34aca0198 | 18 + ...union26-0-a0d1a7f8eb3fba9455ab517aab14552d | 0 ...union26-1-4702087f160230463e6114d6534d47e4 | 309 ++ ...union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 | 309 ++ ...union26-3-4702087f160230463e6114d6534d47e4 | 309 ++ ...union27-0-7f319eff3c0237b6c06cb704dcde9195 | 0 ...union27-1-fb2ddef0c7d9b05d7fb26442599d354f | 0 ...union27-2-10d1593b6663231ff7de938024ce6bb6 | 0 ...union27-3-ab84df3813ff23be99f148449610e530 | 8 + ...union28-0-6bccc08f175712c17aa66cebbf5b7e36 | 0 ...union28-1-f1bd585c115dbabf655d8dc38cd4e026 | 0 ...union28-2-9985bbc70b8dd624fac261eafb10a78f | 0 ...union28-3-b1d75ba0d33a452619e41f70e69616e9 | 20 + ...union29-0-6bccc08f175712c17aa66cebbf5b7e36 | 0 ...union29-1-a40036f756a306f4226634b6ab67c72c | 0 ...union29-2-c7fab64fc276c13149e1e68c0deb6594 | 0 ...union29-3-b1d75ba0d33a452619e41f70e69616e9 | 20 + .../union3-0-6a8a35102de1b0b88c6721a704eb174d | 0 .../union3-1-ae80732ae5db530cc4e355d5d4724457 | 0 .../union3-2-2a1dcd937f117f1955a169592b96d5f9 | 0 .../union3-3-8fc63f8edb2969a63cd4485f1867ba97 | 4 + ...union30-0-6bccc08f175712c17aa66cebbf5b7e36 | 0 ...union30-1-1aff6fba829bae7d20266cf91383ef78 | 0 ...union30-2-252f12f7532ca67132bfc62063c58430 | 0 ...union30-3-b1d75ba0d33a452619e41f70e69616e9 | 20 + ...union31-0-ca16024e6f5399b1d035f5b9fd665163 | 0 ...union31-1-b12aa87aaf64b573ce0be7013117651b | 0 ...nion31-10-ca695e6973bc98b7f06c998b964ae339 | 0 ...nion31-11-70dda95c1b749e4104cc2c767cd7fec0 | 0 ...nion31-12-de2477a134ab37bb22d879028becf2e5 | 0 ...nion31-13-22b1078764cb1c9b2fb95358c968e987 | 0 ...nion31-14-c36a1d8de2713f722ec42bc4686d6125 | 6 + ...nion31-15-5df6435aed6e0a6a6853480a027b911e | 6 + ...nion31-16-ca16024e6f5399b1d035f5b9fd665163 | 0 ...nion31-17-b12aa87aaf64b573ce0be7013117651b | 0 ...nion31-18-1522ec86f2e6735cf809a52c1f6f7883 | 0 ...nion31-19-1e9320795b3e4acd7e6d89a8c9ba897e | 0 ...union31-2-1522ec86f2e6735cf809a52c1f6f7883 | 0 ...nion31-20-dceb6d9a03096d68d090557692ed6899 | 0 ...nion31-21-bda2e62d0ecd30200b2359df420da574 | 0 ...nion31-22-fa1c85506afbfa956251cd1253f5d79a | 0 ...nion31-23-c9d9d45a7d1a0c69efb8d8babc4a50df | 0 ...nion31-24-df38c8164af7cc164c728b8178da72c5 | 6 + ...nion31-25-1485e295a99908e1862eae397b814045 | 6 + ...union31-3-35d1dd67e8c3d76c77baaeae0760ddfc | 0 ...union31-4-77653483e5ce18437c9f85d3b297335d | 0 ...union31-5-4651987fca60c1c1a59b8b28713ea737 | 0 ...union31-6-69c2b75a25f884a72d89e43e0186f5a6 | 0 ...union31-7-7b021f95774705e4d7ff7049ee0e0b74 | 0 ...union31-8-ba92b89786ffaecd74a740705e0fa0cb | 6 + ...union31-9-56dfdb30edd8a687f9aa9cad29b42760 | 6 + ...union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e | 0 ...union32-1-e6c80e7d6171ae5fc428506e57dc8753 | 0 ...nion32-10-44a9b373ee7d43a4ef2bc4b8a708601b | 0 ...nion32-11-40d51bc9958b27c36ef647f0598fdee5 | 32 + ...union32-2-8e39fef33c859ef83912d0fcda319218 | 0 ...union32-3-d31e252450077ac54f4cb18a9ad95a84 | 20 + ...union32-4-79787e084ca15d479cee3a7e1ed2281e | 0 ...union32-5-51c997d0a1103b60764bbb8316a38746 | 32 + ...union32-6-96fa13d8790bbfa1d6109b7cbf890d1b | 0 ...union32-7-f936440d63f4e1027dda4de605660633 | 32 + ...union32-8-f42d83f502a7b8d3a36331e0f5621cfb | 0 ...union32-9-74fb695786df4c024288ae23ac8c00db | 32 + ...union33-0-c67a488530dc7e20a9e7acf02c14380f | 1 + ...union33-1-1df7c476abb48234c839878414720873 | 0 ...union33-2-3987150ad66482f5186826efcfb117f7 | 0 ...union33-3-bfe75625806b414a5098af487b91046a | 0 ...union33-4-1d22c1e85a4bfa338aff713d45440877 | 1 + ...union33-5-13c01085e0a2c46e745985fa11bedf34 | 0 ...union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 | 0 ...union33-7-1d22c1e85a4bfa338aff713d45440877 | 1 + ...union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 | 0 ...union34-1-320c2fe802fab386957039aaff6399f1 | 0 ...nion34-10-da2b79118c21ac45ce85001fa61b0043 | 30 + .../union34-2-5707c9fd2cfd049426383e5e5dc80d7 | 0 ...union34-3-c179e14d840249d6953cb552279822f1 | 0 ...union34-4-101829a66cab2efd31dcb0d86e302956 | 0 ...union34-5-24ca942f094b14b92086305cc125e833 | 1 + ...union34-6-ff0312eeb487fc393a06880ef5bb286f | 0 ...union34-7-da2b79118c21ac45ce85001fa61b0043 | 30 + ...union34-8-b1e2ade89ae898650f0be4f796d8947b | 1 + ...union34-9-d82ffe361a119651bbf41a149b6eba91 | 0 .../union4-0-863233ccd616401efb4bf83c4b9e3a52 | 1 + .../union4-1-88deb3987655dc41989f39c0827cec74 | 0 .../union4-2-41739c4bd273f0cfa94b347c36a71f38 | 0 .../union4-3-2fb47e6ec0a780da5bda97752e573c0b | 0 .../union4-4-7f83822f19aa9b973198fe4c42c66856 | 2 + .../union5-0-863233ccd616401efb4bf83c4b9e3a52 | 1 + .../union5-1-54c6369ded20ba12138e6c53a157dae8 | 0 .../union5-2-2c19c8d564b010eeb42deee63d66a292 | 2 + .../union6-0-863233ccd616401efb4bf83c4b9e3a52 | 1 + .../union6-1-7d631b7dbfe39ccaf995e16186b3e94d | 0 .../union6-2-31cbd867e4b88605d230c542af64b7c4 | 0 .../union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 | 0 .../union6-4-a03959cc5aaa8f6521a73e6dae04cd15 | 26 + .../union7-0-863233ccd616401efb4bf83c4b9e3a52 | 1 + .../union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 | 0 .../union7-2-55d6e503a281acf3289a7874c0fba3f5 | 17 + .../union8-0-d0a83a1cedcbd1d2158acf7eb938d18e | 0 .../union8-1-1b422e4c1c8b97775518f760b995c771 | 1500 ++++++++++ .../union9-0-fad6a053ab42c3cad89052e88d819e83 | 0 .../union9-1-a77ee9f723b3b17a3a02164c5d0000c1 | 1 + ...on_date-0-a0bade1c77338d4f72962389a1f5bea2 | 0 ...on_date-1-21306adbd8be8ad75174ad9d3e42b73c | 0 ...on_date-2-b386e4435da29e7e240893639c6dd9c4 | 0 ...on_date-3-f0f678c9032603ca0ae729a31c8c9d38 | 0 ...ion_date-4-d812f7feef3b6857aeca9007f0af44c | 0 ...on_date-5-b54839e0200bec94aa751fec8c5dbd3d | 0 ...ralview-0-fbcc72d0cd912977292df8a9e55a4e98 | 0 ...ralview-1-55dc0ec224133e4714291cc13a7a8ce0 | 0 ...ralview-2-2252a6cc9b8af3e9c6eb12d470072339 | 0 ...ralview-3-97ea99689397f6e0c8b07d598dd9c042 | 20 + ...on_null-0-27e98c4939abf1ad4445b4e715b0262a | 10 + ...ion_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b | 0 ...ion_ppr-1-4d073ff9d6978f3c3be447920ab8daed | 336 +++ ...remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_1-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...move_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...emove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...move_1-12-43d53504df013e6b35f81811138a167a | 1 + ...emove_1-2-cafed8ca348b243372b9114910be1557 | 1 + ...emove_1-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...emove_1-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...emove_1-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...remove_1-6-a6c043a89a9c3456af8ee065cb17239 | 0 ...emove_1-7-48f70528347f5201f387d28dae37a14a | 0 ...emove_1-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_1-9-19865a08066d80cb069ae6312c465ee6 | 0 ...emove_10-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_10-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_10-10-7eaf13bc61bd4b43f4da67c347768598 | 0 ...ove_10-11-b62595b91d2d9e03a010b49ab81725d5 | 0 ...move_10-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...ove_10-13-43d53504df013e6b35f81811138a167a | 1 + ...move_10-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_10-3-b12e5c70d6d29757471b900b6160fa8a | 1 + ...move_10-4-593999fae618b6b38322bc9ae4e0c027 | 1 + ...move_10-5-6f53d5613262d393d82d159ec5dc16dc | 1 + ...move_10-6-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_10-7-a7ba0436265932086d2a2e228356971 | 0 ...move_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...move_10-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_11-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_11-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_11-10-9788525d32af3dccdefb459669f886c1 | 0 ...ove_11-11-42ecec4117d372551310ad1d85a9c09a | 0 ...move_11-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...ove_11-13-43d53504df013e6b35f81811138a167a | 1 + ...ove_11-14-e409e7032445097ace016b1876d95b3e | 18 + ...move_11-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_11-3-b12e5c70d6d29757471b900b6160fa8a | 1 + ...move_11-4-593999fae618b6b38322bc9ae4e0c027 | 1 + ...move_11-5-6f53d5613262d393d82d159ec5dc16dc | 1 + ...move_11-6-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_11-7-ba03d14418197f8ad6496d30c0c29a59 | 0 ...move_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...move_11-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_12-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_12-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_12-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_12-11-a667f24e26435cd2a29fef0ee45bab3c | 0 ...ove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 | 0 ...move_12-13-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...ove_12-14-43d53504df013e6b35f81811138a167a | 1 + ...move_12-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_12-3-24ca942f094b14b92086305cc125e833 | 1 + ...move_12-4-b12e5c70d6d29757471b900b6160fa8a | 1 + ...move_12-5-593999fae618b6b38322bc9ae4e0c027 | 1 + ...move_12-6-6f53d5613262d393d82d159ec5dc16dc | 1 + ...move_12-7-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_12-8-9dd030d38eece4630dec5951fc8a0622 | 0 ...move_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_13-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_13-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_13-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 | 0 ...ove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af | 0 ...move_13-13-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...ove_13-14-43d53504df013e6b35f81811138a167a | 1 + ...move_13-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_13-3-24ca942f094b14b92086305cc125e833 | 1 + ...move_13-4-b12e5c70d6d29757471b900b6160fa8a | 1 + ...move_13-5-593999fae618b6b38322bc9ae4e0c027 | 1 + ...move_13-6-6f53d5613262d393d82d159ec5dc16dc | 1 + ...move_13-7-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_13-8-234ecbc3f8c0e4686d3586f81cf191eb | 0 ...move_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_14-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_14-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_14-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_14-11-a667f24e26435cd2a29fef0ee45bab3c | 0 ...ove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 | 0 ...move_14-13-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...ove_14-14-43d53504df013e6b35f81811138a167a | 1 + ...move_14-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_14-3-b12e5c70d6d29757471b900b6160fa8a | 1 + ...move_14-4-593999fae618b6b38322bc9ae4e0c027 | 1 + ...move_14-5-24ca942f094b14b92086305cc125e833 | 1 + ...move_14-6-6f53d5613262d393d82d159ec5dc16dc | 1 + ...move_14-7-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_14-8-ed33d620523b2634285698a83f433b6d | 0 ...move_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_15-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_15-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_15-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_15-11-93b7341b523213ab6e58169459bc6818 | 0 ...ove_15-12-616cc477ed00e691dbc2b310d1c6dd12 | 0 ...move_15-13-ea111d286c70e4a0c6a68a7420dc7b7 | 32 + ...ove_15-14-37f9f7bc2d7456046a9f967347337e47 | 2 + ...ove_15-15-43d53504df013e6b35f81811138a167a | 1 + ...move_15-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_15-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...move_15-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...move_15-5-16367c381d4b189b3640c92511244bfe | 1 + ...move_15-6-a4fb8359a2179ec70777aad6366071b7 | 1 + ...move_15-7-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_15-8-552c4eba867e7408fc8652ff0a19170d | 0 ...move_15-9-a63925fb2aa8c5df6854c248e674b0ef | 0 ...emove_16-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_16-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_16-10-a63925fb2aa8c5df6854c248e674b0ef | 0 ...ove_16-11-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_16-12-93b7341b523213ab6e58169459bc6818 | 0 ...ove_16-13-616cc477ed00e691dbc2b310d1c6dd12 | 0 ...move_16-14-ea111d286c70e4a0c6a68a7420dc7b7 | 32 + ...ove_16-15-37f9f7bc2d7456046a9f967347337e47 | 2 + ...ove_16-16-43d53504df013e6b35f81811138a167a | 1 + ...move_16-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_16-3-b12e5c70d6d29757471b900b6160fa8a | 1 + ...move_16-4-593999fae618b6b38322bc9ae4e0c027 | 1 + ...move_16-5-6f53d5613262d393d82d159ec5dc16dc | 1 + ...move_16-6-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_16-7-16367c381d4b189b3640c92511244bfe | 1 + ...move_16-8-a4fb8359a2179ec70777aad6366071b7 | 1 + ...move_16-9-ec47ba0fc527a4a04d452a009d59147a | 0 ...emove_17-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_17-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_17-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 | 0 ...ove_17-12-c91289e16ad403babfc91c093ac9b86d | 0 ...move_17-13-ea111d286c70e4a0c6a68a7420dc7b7 | 32 + ...ove_17-14-37f9f7bc2d7456046a9f967347337e47 | 2 + ...ove_17-15-43d53504df013e6b35f81811138a167a | 1 + ...ove_17-16-626a252f75285872c1d72706f7f972c6 | 6 + ...ove_17-17-1252985379f11ae4b98d2a0e2f121b8a | 6 + ...move_17-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_17-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...move_17-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...move_17-5-16367c381d4b189b3640c92511244bfe | 1 + ...move_17-6-a4fb8359a2179ec70777aad6366071b7 | 1 + ...move_17-7-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_17-8-67e57f56d1106a57329bf75c491e3c8b | 0 ...move_17-9-a63925fb2aa8c5df6854c248e674b0ef | 0 ...emove_18-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_18-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_18-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad | 0 ...ove_18-12-461a24ba6c00f60d75afba62da2ac4f1 | 0 ...move_18-13-ea111d286c70e4a0c6a68a7420dc7b7 | 32 + ...ove_18-14-37f9f7bc2d7456046a9f967347337e47 | 6 + ...ove_18-15-43d53504df013e6b35f81811138a167a | 1 + ...move_18-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_18-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...move_18-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...move_18-5-16367c381d4b189b3640c92511244bfe | 1 + ...move_18-6-a4fb8359a2179ec70777aad6366071b7 | 1 + ...move_18-7-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_18-8-b1b996b2e72ca14150af7c82dbc6e139 | 0 ...move_18-9-ea467d0fee062a23c720cf47eacfef08 | 0 ...emove_19-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_19-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_19-10-8d1e5af37e7992708bf15ab7d887405b | 0 ...move_19-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...ove_19-12-68e15b1729669c4cb2291dcabfea4387 | 10 + ...ove_19-13-b8b9df8f376df228e3c8ae65defe2801 | 0 ...ove_19-14-eed866a1ad3106a306322519f4bb52f2 | 0 ...ove_19-15-68e15b1729669c4cb2291dcabfea4387 | 2 + ...ove_19-16-471f8e794fd712dce2e40334b383e08e | 0 ...ove_19-17-ae0f7d5734ca67cbfebed70c4657e330 | 0 ...ove_19-18-43d53504df013e6b35f81811138a167a | 1 + ...move_19-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_19-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...move_19-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...move_19-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_19-6-a6c043a89a9c3456af8ee065cb17239 | 0 ...move_19-7-48f70528347f5201f387d28dae37a14a | 0 ...move_19-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_19-9-8a08edd1aa63fd3b051da82246793259 | 0 ...remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_2-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...move_2-10-2309570010c3e679b884c100de57d002 | 0 ...emove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...move_2-12-43d53504df013e6b35f81811138a167a | 1 + ...emove_2-2-cafed8ca348b243372b9114910be1557 | 1 + ...emove_2-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...emove_2-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...emove_2-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...remove_2-6-1acf02741a2ff987d3e00ae9722c26e | 0 ...emove_2-7-48f70528347f5201f387d28dae37a14a | 0 ...emove_2-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...emove_20-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_20-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_20-10-89c57c91facbf54299e08955e3783ea6 | 0 ...move_20-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...ove_20-12-43d53504df013e6b35f81811138a167a | 1 + ...move_20-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_20-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...move_20-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...move_20-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_20-6-a73143117ffec1225f6d492e5aa577e | 0 ...move_20-7-82f81adc097c247475fd29076e0cb85f | 0 ...move_20-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_20-9-683949ae07de12da0b7e7ba7f4450daa | 0 ...emove_21-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_21-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_21-10-e19469a1b70be25caaf670fe68f0a747 | 0 ...move_21-11-ea111d286c70e4a0c6a68a7420dc7b7 | 26 + ...ove_21-12-43d53504df013e6b35f81811138a167a | 1 + ...move_21-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_21-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...move_21-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...move_21-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_21-6-aeba356b56f8659963d8b2dc07a84a6f | 0 ...move_21-7-5716c408db679fb88352eaceb1703bd7 | 0 ...move_21-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_21-9-deadbce171926623b0586587fbbcd144 | 0 ...emove_22-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_22-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_22-10-403471d96c56b565272d2e4c4926d240 | 0 ...move_22-11-ea111d286c70e4a0c6a68a7420dc7b7 | 28 + ...ove_22-12-68e15b1729669c4cb2291dcabfea4387 | 10 + ...ove_22-13-a28b876b5df29140ef2bf62b4d0de3fd | 0 ...ove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af | 0 ...ove_22-15-43d53504df013e6b35f81811138a167a | 1 + ...move_22-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_22-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...move_22-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...move_22-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_22-6-722acd65700dd132dc6b4bc8c56f4ce0 | 0 ...move_22-7-46da090f5a2c73b175207cf63ff46653 | 0 ...move_22-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_22-9-b3edbbee0543ff268db4059afb9cb2cb | 0 ...emove_23-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_23-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_23-10-613ce50facecdc8d7bf8806a8ff17c13 | 0 ...move_23-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...ove_23-12-43d53504df013e6b35f81811138a167a | 1 + ...move_23-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_23-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...move_23-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...move_23-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 | 0 ...move_23-7-48f70528347f5201f387d28dae37a14a | 0 ...move_23-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_23-9-2dca12ca51c74540e7cdbbb05e336ed5 | 0 ...emove_24-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...move_24-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...ove_24-10-6d89089b1eead05510dbccad5fcc4805 | 0 ...move_24-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...ove_24-12-43d53504df013e6b35f81811138a167a | 1 + ...move_24-2-cafed8ca348b243372b9114910be1557 | 1 + ...move_24-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...move_24-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...move_24-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...move_24-6-72ae7a9640ae611f61ac954ac1a4b682 | 0 ...move_24-7-44a7b0f14b4f5151c37498367ad7fe1e | 0 ...move_24-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_24-9-4c0550cc9c28de25993c1f98de39168f | 0 ...remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_3-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...move_3-10-cfc49418c6621c665dfb671ed5656fc9 | 0 ...emove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...move_3-12-43d53504df013e6b35f81811138a167a | 1 + ...move_3-13-e409e7032445097ace016b1876d95b3e | 18 + ...emove_3-2-cafed8ca348b243372b9114910be1557 | 1 + ...emove_3-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...emove_3-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...emove_3-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_3-6-c36130e41df18093eee24b45fc0846f0 | 0 ...emove_3-7-48f70528347f5201f387d28dae37a14a | 0 ...emove_3-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_3-9-7da122d7b42dc7ef504ccc442d6545f1 | 0 ...remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_4-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...move_4-10-19865a08066d80cb069ae6312c465ee6 | 0 ...move_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...emove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...move_4-13-43d53504df013e6b35f81811138a167a | 1 + ...emove_4-2-cafed8ca348b243372b9114910be1557 | 1 + ...emove_4-3-b12e5c70d6d29757471b900b6160fa8a | 1 + ...emove_4-4-593999fae618b6b38322bc9ae4e0c027 | 1 + ...emove_4-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_4-6-6f53d5613262d393d82d159ec5dc16dc | 1 + ...emove_4-7-90fcd4eb330919ad92aecb8a5bf30ead | 0 ...emove_4-8-48f70528347f5201f387d28dae37a14a | 0 ...emove_4-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_5-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...move_5-10-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...move_5-11-2309570010c3e679b884c100de57d002 | 0 ...emove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...move_5-13-43d53504df013e6b35f81811138a167a | 1 + ...emove_5-2-cafed8ca348b243372b9114910be1557 | 1 + ...emove_5-3-b12e5c70d6d29757471b900b6160fa8a | 1 + ...emove_5-4-593999fae618b6b38322bc9ae4e0c027 | 1 + ...emove_5-5-6f53d5613262d393d82d159ec5dc16dc | 1 + ...emove_5-6-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_5-7-4da6ca94af4caf4426e5676a32b70375 | 0 ...emove_5-8-48f70528347f5201f387d28dae37a14a | 0 ...emove_5-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_6-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...move_6-10-52171e0094ea92aaea8a485aedc88e88 | 0 ...move_6-11-60361645974d38829674ec6bb5502a62 | 0 ...move_6-12-43d53504df013e6b35f81811138a167a | 1 + ...move_6-13-e409e7032445097ace016b1876d95b3e | 10 + ...move_6-14-f74b28904e86047150396bc42680ca38 | 10 + ...emove_6-2-cafed8ca348b243372b9114910be1557 | 1 + ...emove_6-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...emove_6-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...emove_6-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 | 0 ...emove_6-7-48f70528347f5201f387d28dae37a14a | 0 ...emove_6-8-19e054c1050e5dbf703e91a48253ecdf | 0 ...emove_6-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_7-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...move_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...emove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...move_7-12-43d53504df013e6b35f81811138a167a | 1 + ...emove_7-2-cafed8ca348b243372b9114910be1557 | 1 + ...emove_7-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...emove_7-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...emove_7-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_7-6-3744f0ebf5c002fdfcec67fbce03dfca | 0 ...emove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_7-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_7-9-19865a08066d80cb069ae6312c465ee6 | 0 ...remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_8-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...move_8-10-2309570010c3e679b884c100de57d002 | 0 ...emove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...move_8-12-43d53504df013e6b35f81811138a167a | 1 + ...emove_8-2-cafed8ca348b243372b9114910be1557 | 1 + ...emove_8-3-dc129f70e75cd575ce8c0de288884523 | 1 + ...emove_8-4-a572a07cd60fd4607ddd7613db8a64ab | 1 + ...emove_8-5-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_8-6-895f2432b4be6fcb11641c1d063570ee | 0 ...emove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_8-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 + ...emove_9-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 + ...move_9-10-12cf3335c756f8715a07c5a604f10f64 | 0 ...move_9-11-4e84cd589eceda668833f8f19ec28e7c | 0 ...emove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 + ...move_9-13-43d53504df013e6b35f81811138a167a | 1 + ...emove_9-2-cafed8ca348b243372b9114910be1557 | 1 + ...emove_9-3-b12e5c70d6d29757471b900b6160fa8a | 1 + ...emove_9-4-593999fae618b6b38322bc9ae4e0c027 | 1 + ...emove_9-5-6f53d5613262d393d82d159ec5dc16dc | 1 + ...emove_9-6-a2a411ad6620aa1ab24550ade336e785 | 1 + ...emove_9-7-f77ac921b27860fac94cac6b352f3eb5 | 0 ...emove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_9-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ..._script-0-ca195b08d98d7f21fe93208499bf0ff6 | 500 ++++ ..._script-1-982cc6d7b98f8fb1055a10ef021e2769 | 1000 +++++++ ...on_view-0-e56367a21517656c18a5bcfeecb4327d | 0 ...on_view-1-c790d4344144460224b0f02be7e137a8 | 0 ...on_view-2-7e33b0744f57bdc8ebcd9d45348aef14 | 0 ...on_view-3-4a746bd076e063017c4d6a2f9218a6e4 | 0 ...on_view-4-d3d75f376f83b694b1dc62c46fa53f4e | 0 ...on_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 | 0 ...on_view-6-c9d7dcde469d3b9a66965a64dd15e4ae | 1 + ...on_view-7-3b03210f94ec40db9ab02620645014d1 | 1 + ...on_view-8-35f48c7d6fa164bb84643657bc9280a8 | 1 + ...rchar_2-0-fbced67562c536a7e9b6d3928bd9d0af | 0 ...rchar_2-1-bee4e3a9ded7276960e619e6185bf329 | 0 ...rchar_2-2-779a555944479991520be53f7f247626 | 0 ...rchar_2-3-a8c072e5b13997e9c79484b4af9d78da | 5 + ...rchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f | 5 + ...rchar_2-5-96353c24b5f2b361f72e5c26b4135519 | 5 + ...rchar_2-6-2b62789d07b4044bc32190261bf3490f | 5 + ...rchar_2-7-fbced67562c536a7e9b6d3928bd9d0af | 0 ...r_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e | 0 ...r_join1-1-8318072ee410cae36186c9880989a1a5 | 0 ..._join1-10-1958143ee083437e87662cadb48c37ce | 3 + ...r_join1-11-a55f750032663f77066e4979dedea1c | 3 + ..._join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e | 0 ..._join1-13-8318072ee410cae36186c9880989a1a5 | 0 ..._join1-14-8bbaae82c89553165b975c5b8ca3aee4 | 0 ...r_join1-2-8bbaae82c89553165b975c5b8ca3aee4 | 0 ...r_join1-3-735f68c77eae74623c8bdbebc995a3c7 | 0 ...r_join1-4-d5105d36e4d077e0929597124526c532 | 0 ...r_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 | 0 ...har_join1-6-6bb08c5baa913d9dc506aef65425ef | 0 ...r_join1-7-341118dab140b17f0b9d2c7b101d1298 | 0 ...r_join1-8-afe7304d94450481c01ddbaf6cc3f596 | 0 ...r_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 | 3 + ...d_types-0-e41b0e9db7a9dbd0540e29df341933bc | 0 ...d_types-1-87a28b0e800f051525899324a064b878 | 0 ...d_types-10-92f25849990eb777ac2711f9dd2e628 | 1 + ..._types-11-9780781a92fdd992f3cee080a8717238 | 1 + ..._types-12-e1a4006971319a352280fc52eabf449f | 0 ..._types-13-d4574217a243a7d506398a819cd0eab4 | 0 ..._types-14-7c33a62195359bc89460ad65f6a5f763 | 1 + ...d_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 | 1 + ..._types-16-93811d146a429c44a2494b3aa4b2caa2 | 0 ..._types-17-5724af3985c67a0cb69919c9bbce15dc | 0 ..._types-18-d1be2ee3765a80469837ba11eb8685e1 | 1 + ..._types-19-a54f9a284228e7cdce8c34b8094f2377 | 1 + ...d_types-2-ae69b82461acc2aa366d8b1f8626d6fb | 0 ..._types-20-51a9279006f61097e68a52201daf6710 | 0 ..._types-21-1d304e09cc2a8b2824bfc04bdbc976ad | 1 + ..._types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 | 1 + ..._types-23-c7ea918777b725f2790da4fa00a3efa6 | 0 ..._types-24-5a69236334a3a1c4e771206cf547d730 | 1 + ..._types-25-61b030bb2220a533532d871ae0e08cdb | 1 + ..._types-26-e41b0e9db7a9dbd0540e29df341933bc | 0 ..._types-27-87a28b0e800f051525899324a064b878 | 0 ..._types-28-ae69b82461acc2aa366d8b1f8626d6fb | 0 ..._types-29-39d44d19c7963a9647fd3293eef670d4 | 0 ...d_types-3-39d44d19c7963a9647fd3293eef670d4 | 0 ..._types-30-162806477075d97de16dfa6f2576b751 | 0 ..._types-31-5b28e1fdb28b365ef419008a4752ed53 | 0 ...d_types-4-162806477075d97de16dfa6f2576b751 | 0 ...d_types-5-5b28e1fdb28b365ef419008a4752ed53 | 0 ...d_types-6-a47b99c355df4aad56dceb7f81a7fd5b | 0 ...d_types-7-8c483a7a0e148ca13a292a625f8702f1 | 0 ...d_types-8-94309e2b4e68ab7e25f7d9656f10b352 | 0 ...d_types-9-f248796769bc7f57cf56a75034a45520 | 0 ...r_serde-0-750a23ebdd77f32b555d4caba7ac5445 | 0 ...r_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d | 0 ..._serde-10-496280d9365ee601d9b68a91495d7160 | 0 ..._serde-11-2f4d5f574f659df2bd0fb97f2d50a36e | 0 ..._serde-12-80727f22f1343407ba9200c86ed84280 | 0 ..._serde-13-4829e6e5822c73fb33ba3d619b4bd31e | 5 + ..._serde-14-4794ee93811ce3a81f64c65aed0b8b13 | 5 + ..._serde-15-6ae634e1ae1aa0730a33396bce5a6604 | 0 ..._serde-16-516202183287d734d35d8c7788d22652 | 0 ..._serde-17-8d4419222a728e2bbc6a464b5a0b5f7a | 0 ..._serde-18-b2dfd28d6a5b1e6f4556ad19755b739d | 5 + ..._serde-19-f258df2db09e9cc0e049e85e6ad950ad | 5 + ...r_serde-2-87ba3f40293b9c79fcdb3064d964232e | 0 ..._serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 | 0 ..._serde-21-64536c77ae91bfb6cf7f93f178c6200b | 0 ...r_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 | 0 ..._serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 | 5 + ..._serde-24-770169b632b76cedcd6dfb87fdc46575 | 5 + ..._serde-25-3470a259b04e126c655531491787e2fc | 0 ..._serde-26-55808e190e0ab81dcdc1feb52543ad9f | 0 ..._serde-27-8fe526fdd347c25529a383f27ad20566 | 0 ..._serde-28-5e4de93349ba89a8344bb799ad60678e | 5 + ..._serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c | 5 + ...r_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 | 0 ..._serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a | 0 ..._serde-31-db1e88efcd55aaef567edaa89c1c1e12 | 0 ..._serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb | 0 ..._serde-33-620729dc7661c22f1acdc425a7cf0364 | 5 + ..._serde-34-807ee73e1cd66704dd585f7e0de954d9 | 5 + ..._serde-35-750a23ebdd77f32b555d4caba7ac5445 | 0 ..._serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d | 0 ..._serde-37-87ba3f40293b9c79fcdb3064d964232e | 0 ..._serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 | 0 ..._serde-39-8b5f81c27c41807c757522e257a8003f | 0 ...r_serde-4-8b5f81c27c41807c757522e257a8003f | 0 ..._serde-40-787193a1679a2153c037d3e4c8192bba | 0 ...r_serde-5-787193a1679a2153c037d3e4c8192bba | 0 ...r_serde-6-122f15d410249b554e12eccdfa46cc43 | 0 ...r_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 | 0 ...r_serde-8-3bfa13085b001c1a333cc72d5c9f4244 | 5 + ...r_serde-9-ec43be632e5f74057aba54c4f562c601 | 5 + ..._union1-0-433f0ff2f1c8dc92fba226759326067e | 0 ..._union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec | 0 ...union1-10-6ec48d5fea3e4a35275956b9b4467715 | 4 + ...union1-11-78f6e219b974e1fdf3663e46f57892a9 | 4 + ...union1-12-433f0ff2f1c8dc92fba226759326067e | 0 ...union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec | 0 ...union1-14-217ba1b02c693576b652d719970a0048 | 0 ..._union1-2-217ba1b02c693576b652d719970a0048 | 0 ..._union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca | 0 ..._union1-4-86e7275fc6c5ce64da5031aa7c79821b | 0 ..._union1-5-bf77db56df2b2a077d668e47211be708 | 0 ..._union1-6-67e66fa14dddc17757436539eca9ef64 | 0 ..._union1-7-48766d09c5ed1b6abe9ce0b8996adf36 | 0 ..._union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 | 0 ..._union1-9-b9455ffec62df97cfec63204ce02a110 | 4 + .../view-0-5528e36b3b0f5b14313898cc45f9c23a | 0 .../view-1-7650b86c86dd6b1a99c86ddc5a31bd63 | 0 .../view-10-7aae4448a05e8a8a3bace7522e952cd0 | 0 .../view-11-dc95343d3e57846485dd543476391376 | 0 .../view-12-371764e1cae31ea0518c03060528d239 | 0 .../view-13-2abce88008f8a19164758ee821aaa8a6 | 0 .../view-14-deb504f4f70fd7db975950c3c47959ee | 0 .../view-15-6f2797b6f81943d3b53b8d247ae8512b | 0 .../view-2-9c529f486fa81a032bfe1253808fca8 | 0 .../view-3-89c80c0e90409d5e304775c9f420915a | 0 .../view-4-4a64d1a623ca71e515796787dbd0f904 | 0 .../view-5-f6d1bce095ecbf1aa484891392fdb07b | 0 .../view-6-47b5043f03a84695b6784682b4402ac8 | 0 .../view-7-8b1bbdadfd1e11af1b56064196164e58 | 0 .../view-8-60d2f3ee552ae7021f9fa72f0dcf2867 | 0 .../view-9-66c68babac10ae0f645fe8334c5a42d4 | 0 ...ew_cast-0-89bd46ad04f967f1f5ee17c6f201aacf | 0 ...ew_cast-1-85685a26971fb51ab6e28f9c5e8421bb | 0 ...w_cast-10-a7c865e5180df8d73dba90ede8be0d45 | 0 ...ew_cast-2-af2050aa97f0cd930cb1b8ec791007de | 0 ...iew_cast-3-2a232c31d056e6abc76f8ebe53ccd97 | 0 ...ew_cast-4-d9edb83f4cf847e141d97012314917d4 | 0 ...ew_cast-5-6db508ccd85562a9ca7841fb0a08981a | 0 ...ew_cast-6-aa5be3380ddc7104258567b406d93cc5 | 0 ...ew_cast-7-78ac3800b22682b31708b6a09b402bfb | 0 ...ew_cast-8-2cc0c576f0a008abf5bdf3308d500869 | 0 ...ew_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 | 0 ..._inputs-0-9e67dfd1d595ab8b1935b789645f76c0 | 0 ..._inputs-1-5af97e73bc3841793440105aae766bbe | 0 ..._column-0-9bacd1908e56d621913a74fe9a583d9d | 500 ++++ ..._column-1-3ebad682d4ff6ca9c806db3471cf3945 | 309 ++ ...column-10-2915b222a58bc994246591e536d388b4 | 0 ..._column-2-1536b365fe0a94b30a62364996529966 | 309 ++ ..._column-3-c66776673c986b59b27e704664935988 | 0 ..._column-4-e47094c927b1091e31c185db0a4e69a6 | 1 + ..._column-5-d137fa7c27bc98d5f1a33f666a07f6b7 | 0 ..._column-6-68d6973677af5c9f1f1f49360c3175e7 | 1 + ..._column-7-20d7d672a4289fbd1a5de485a8353ac6 | 0 ..._column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 | 500 ++++ ...l_column-9-c152da33c1517ecfc336f72b9c133d5 | 0 9181 files changed, 157728 insertions(+) create mode 100644 src/test/hive/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb create mode 100644 src/test/hive/golden/'1' + 1-0-130514c6116c311d808590a075b187b create mode 100644 src/test/hive/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e create mode 100644 src/test/hive/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 create mode 100644 src/test/hive/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 create mode 100644 src/test/hive/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a create mode 100644 src/test/hive/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a create mode 100644 src/test/hive/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 create mode 100644 src/test/hive/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 create mode 100644 src/test/hive/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e create mode 100644 src/test/hive/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e create mode 100644 src/test/hive/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f create mode 100644 src/test/hive/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 create mode 100644 src/test/hive/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 create mode 100644 src/test/hive/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d create mode 100644 src/test/hive/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 create mode 100644 src/test/hive/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 create mode 100644 src/test/hive/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 create mode 100644 src/test/hive/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a create mode 100644 src/test/hive/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d create mode 100644 src/test/hive/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 create mode 100644 src/test/hive/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f create mode 100644 src/test/hive/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 create mode 100644 src/test/hive/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 create mode 100644 src/test/hive/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 create mode 100644 src/test/hive/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 create mode 100644 src/test/hive/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e create mode 100644 src/test/hive/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 create mode 100644 src/test/hive/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e create mode 100644 src/test/hive/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b create mode 100644 src/test/hive/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb create mode 100644 src/test/hive/golden/1Y + 1-0-a4541db51882b19503649138fbb295f create mode 100644 src/test/hive/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d create mode 100644 src/test/hive/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 create mode 100644 src/test/hive/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 create mode 100644 src/test/hive/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 create mode 100644 src/test/hive/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a create mode 100644 src/test/hive/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 create mode 100644 src/test/hive/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 create mode 100644 src/test/hive/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 create mode 100644 src/test/hive/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 create mode 100644 src/test/hive/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 create mode 100644 src/test/hive/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa create mode 100644 src/test/hive/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 create mode 100644 src/test/hive/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 create mode 100644 src/test/hive/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 create mode 100644 src/test/hive/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 create mode 100644 src/test/hive/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b create mode 100644 src/test/hive/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 create mode 100644 src/test/hive/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 create mode 100644 src/test/hive/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 create mode 100644 src/test/hive/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 create mode 100644 src/test/hive/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b create mode 100644 src/test/hive/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b create mode 100644 src/test/hive/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b create mode 100644 src/test/hive/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 create mode 100644 src/test/hive/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b create mode 100644 src/test/hive/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b create mode 100644 src/test/hive/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f create mode 100644 src/test/hive/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b create mode 100644 src/test/hive/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb create mode 100644 src/test/hive/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b create mode 100644 src/test/hive/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b create mode 100644 src/test/hive/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 create mode 100644 src/test/hive/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b create mode 100644 src/test/hive/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f create mode 100644 src/test/hive/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b create mode 100644 src/test/hive/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb create mode 100644 src/test/hive/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b create mode 100644 src/test/hive/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 create mode 100644 src/test/hive/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 create mode 100644 src/test/hive/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 create mode 100644 src/test/hive/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef create mode 100644 src/test/hive/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c create mode 100644 src/test/hive/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 create mode 100644 src/test/hive/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af create mode 100644 src/test/hive/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 create mode 100644 src/test/hive/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f create mode 100644 src/test/hive/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 create mode 100644 src/test/hive/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 create mode 100644 src/test/hive/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 create mode 100644 src/test/hive/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 create mode 100644 src/test/hive/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d create mode 100644 src/test/hive/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 create mode 100644 src/test/hive/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 create mode 100644 src/test/hive/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 create mode 100644 src/test/hive/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 create mode 100644 src/test/hive/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 src/test/hive/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da create mode 100644 src/test/hive/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 src/test/hive/golden/alter2-12-75a213649242c2410ea6846f08c91d75 create mode 100644 src/test/hive/golden/alter2-13-ca51e03a3de391983429b6ad877e573c create mode 100644 src/test/hive/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 src/test/hive/golden/alter2-15-75a213649242c2410ea6846f08c91d75 create mode 100644 src/test/hive/golden/alter2-16-77500c83ffcece95511a4b21d67382dd create mode 100644 src/test/hive/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 src/test/hive/golden/alter2-18-75a213649242c2410ea6846f08c91d75 create mode 100644 src/test/hive/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 create mode 100644 src/test/hive/golden/alter2-2-75a213649242c2410ea6846f08c91d75 create mode 100644 src/test/hive/golden/alter2-20-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 create mode 100644 src/test/hive/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 create mode 100644 src/test/hive/golden/alter2-23-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter2-24-775d69742a1c07df8da87e8a017d955 create mode 100644 src/test/hive/golden/alter2-25-4ef75e12575453225738ea167c4617e5 create mode 100644 src/test/hive/golden/alter2-26-63f13c364546ddce5d2176c6604a948f create mode 100644 src/test/hive/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f create mode 100644 src/test/hive/golden/alter2-28-4ef75e12575453225738ea167c4617e5 create mode 100644 src/test/hive/golden/alter2-29-63f13c364546ddce5d2176c6604a948f create mode 100644 src/test/hive/golden/alter2-3-ca51e03a3de391983429b6ad877e573c create mode 100644 src/test/hive/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae create mode 100644 src/test/hive/golden/alter2-31-4ef75e12575453225738ea167c4617e5 create mode 100644 src/test/hive/golden/alter2-32-63f13c364546ddce5d2176c6604a948f create mode 100644 src/test/hive/golden/alter2-33-1934026d0228967097280eed35551f74 create mode 100644 src/test/hive/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b create mode 100644 src/test/hive/golden/alter2-35-4ef75e12575453225738ea167c4617e5 create mode 100644 src/test/hive/golden/alter2-36-63f13c364546ddce5d2176c6604a948f create mode 100644 src/test/hive/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f create mode 100644 src/test/hive/golden/alter2-38-4ef75e12575453225738ea167c4617e5 create mode 100644 src/test/hive/golden/alter2-39-63f13c364546ddce5d2176c6604a948f create mode 100644 src/test/hive/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 src/test/hive/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae create mode 100644 src/test/hive/golden/alter2-41-4ef75e12575453225738ea167c4617e5 create mode 100644 src/test/hive/golden/alter2-42-63f13c364546ddce5d2176c6604a948f create mode 100644 src/test/hive/golden/alter2-43-1934026d0228967097280eed35551f74 create mode 100644 src/test/hive/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 src/test/hive/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c create mode 100644 src/test/hive/golden/alter2-5-75a213649242c2410ea6846f08c91d75 create mode 100644 src/test/hive/golden/alter2-6-77500c83ffcece95511a4b21d67382dd create mode 100644 src/test/hive/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 src/test/hive/golden/alter2-8-75a213649242c2410ea6846f08c91d75 create mode 100644 src/test/hive/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e create mode 100644 src/test/hive/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade create mode 100644 src/test/hive/golden/alter3-1-75be487df30e301e156a22eee075633d create mode 100644 src/test/hive/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 create mode 100644 src/test/hive/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 create mode 100644 src/test/hive/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 create mode 100644 src/test/hive/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b create mode 100644 src/test/hive/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f create mode 100644 src/test/hive/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 create mode 100644 src/test/hive/golden/alter3-16-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b create mode 100644 src/test/hive/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d create mode 100644 src/test/hive/golden/alter3-19-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f create mode 100644 src/test/hive/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 create mode 100644 src/test/hive/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 create mode 100644 src/test/hive/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 create mode 100644 src/test/hive/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 create mode 100644 src/test/hive/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 create mode 100644 src/test/hive/golden/alter3-25-568a59760e5d3241b63d65cce595face create mode 100644 src/test/hive/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e create mode 100644 src/test/hive/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 create mode 100644 src/test/hive/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 create mode 100644 src/test/hive/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c create mode 100644 src/test/hive/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 create mode 100644 src/test/hive/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 create mode 100644 src/test/hive/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 create mode 100644 src/test/hive/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 create mode 100644 src/test/hive/golden/alter3-4-399fc26b344c98ababa104522601c0cc create mode 100644 src/test/hive/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 create mode 100644 src/test/hive/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 create mode 100644 src/test/hive/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 create mode 100644 src/test/hive/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 create mode 100644 src/test/hive/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 create mode 100644 src/test/hive/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 create mode 100644 src/test/hive/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 create mode 100644 src/test/hive/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 create mode 100644 src/test/hive/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d create mode 100644 src/test/hive/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 create mode 100644 src/test/hive/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f create mode 100644 src/test/hive/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 src/test/hive/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 create mode 100644 src/test/hive/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 create mode 100644 src/test/hive/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d create mode 100644 src/test/hive/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 create mode 100644 src/test/hive/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 create mode 100644 src/test/hive/golden/alter4-5-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter4-6-acd58e84952d310aeddf78579c36286 create mode 100644 src/test/hive/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 create mode 100644 src/test/hive/golden/alter4-8-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 create mode 100644 src/test/hive/golden/alter5-0-953553e14d835682fa47338dcfffe227 create mode 100644 src/test/hive/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b create mode 100644 src/test/hive/golden/alter5-10-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 create mode 100644 src/test/hive/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 create mode 100644 src/test/hive/golden/alter5-13-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 create mode 100644 src/test/hive/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b create mode 100644 src/test/hive/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d create mode 100644 src/test/hive/golden/alter5-17-9176dc5fb5206209fa907a289db1263d create mode 100644 src/test/hive/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf create mode 100644 src/test/hive/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 create mode 100644 src/test/hive/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d create mode 100644 src/test/hive/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 create mode 100644 src/test/hive/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf create mode 100644 src/test/hive/golden/alter5-3-2fc59e32c07186869811705c89aafadc create mode 100644 src/test/hive/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf create mode 100644 src/test/hive/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 create mode 100644 src/test/hive/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 create mode 100644 src/test/hive/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf create mode 100644 src/test/hive/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 create mode 100644 src/test/hive/golden/alter5-9-1e085f9741197e659413828c42386733 create mode 100644 src/test/hive/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 create mode 100644 src/test/hive/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e create mode 100644 src/test/hive/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 create mode 100644 src/test/hive/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb create mode 100644 src/test/hive/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 create mode 100644 src/test/hive/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 create mode 100644 src/test/hive/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d create mode 100644 src/test/hive/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 create mode 100644 src/test/hive/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 create mode 100644 src/test/hive/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 create mode 100644 src/test/hive/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c create mode 100644 src/test/hive/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 create mode 100644 src/test/hive/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d create mode 100644 src/test/hive/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be create mode 100644 src/test/hive/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f create mode 100644 src/test/hive/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 create mode 100644 src/test/hive/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 create mode 100644 src/test/hive/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc create mode 100644 src/test/hive/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 create mode 100644 src/test/hive/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a create mode 100644 src/test/hive/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 create mode 100644 src/test/hive/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 create mode 100644 src/test/hive/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 create mode 100644 src/test/hive/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 create mode 100644 src/test/hive/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e create mode 100644 src/test/hive/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 create mode 100644 src/test/hive/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 create mode 100644 src/test/hive/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e create mode 100644 src/test/hive/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f create mode 100644 src/test/hive/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c create mode 100644 src/test/hive/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 create mode 100644 src/test/hive/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a create mode 100644 src/test/hive/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 create mode 100644 src/test/hive/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a create mode 100644 src/test/hive/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 create mode 100644 src/test/hive/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a create mode 100644 src/test/hive/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 create mode 100644 src/test/hive/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 create mode 100644 src/test/hive/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 create mode 100644 src/test/hive/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d create mode 100644 src/test/hive/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d create mode 100644 src/test/hive/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd create mode 100644 src/test/hive/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 create mode 100644 src/test/hive/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 create mode 100644 src/test/hive/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 create mode 100644 src/test/hive/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 create mode 100644 src/test/hive/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 src/test/hive/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 create mode 100644 src/test/hive/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 create mode 100644 src/test/hive/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 create mode 100644 src/test/hive/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 create mode 100644 src/test/hive/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 src/test/hive/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 create mode 100644 src/test/hive/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 src/test/hive/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 create mode 100644 src/test/hive/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 src/test/hive/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 create mode 100644 src/test/hive/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 create mode 100644 src/test/hive/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 src/test/hive/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 create mode 100644 src/test/hive/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 src/test/hive/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 create mode 100644 src/test/hive/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 create mode 100644 src/test/hive/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 create mode 100644 src/test/hive/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 create mode 100644 src/test/hive/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f create mode 100644 src/test/hive/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c create mode 100644 src/test/hive/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d create mode 100644 src/test/hive/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b create mode 100644 src/test/hive/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 create mode 100644 src/test/hive/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a create mode 100644 src/test/hive/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a create mode 100644 src/test/hive/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 create mode 100644 src/test/hive/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 create mode 100644 src/test/hive/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 create mode 100644 src/test/hive/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 create mode 100644 src/test/hive/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 create mode 100644 src/test/hive/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c create mode 100644 src/test/hive/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 create mode 100644 src/test/hive/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 create mode 100644 src/test/hive/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed create mode 100644 src/test/hive/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d create mode 100644 src/test/hive/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 create mode 100644 src/test/hive/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 create mode 100644 src/test/hive/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 create mode 100644 src/test/hive/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 create mode 100644 src/test/hive/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c create mode 100644 src/test/hive/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 create mode 100644 src/test/hive/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada create mode 100644 src/test/hive/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 create mode 100644 src/test/hive/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 create mode 100644 src/test/hive/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 create mode 100644 src/test/hive/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada create mode 100644 src/test/hive/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 create mode 100644 src/test/hive/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 create mode 100644 src/test/hive/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b create mode 100644 src/test/hive/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c create mode 100644 src/test/hive/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 create mode 100644 src/test/hive/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 create mode 100644 src/test/hive/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 create mode 100644 src/test/hive/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f create mode 100644 src/test/hive/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 create mode 100644 src/test/hive/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 create mode 100644 src/test/hive/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b create mode 100644 src/test/hive/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b create mode 100644 src/test/hive/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f create mode 100644 src/test/hive/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 create mode 100644 src/test/hive/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 create mode 100644 src/test/hive/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 create mode 100644 src/test/hive/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 create mode 100644 src/test/hive/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 create mode 100644 src/test/hive/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c create mode 100644 src/test/hive/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 create mode 100644 src/test/hive/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa create mode 100644 src/test/hive/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 create mode 100644 src/test/hive/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc create mode 100644 src/test/hive/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 create mode 100644 src/test/hive/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def create mode 100644 src/test/hive/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 create mode 100644 src/test/hive/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 create mode 100644 src/test/hive/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 create mode 100644 src/test/hive/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 create mode 100644 src/test/hive/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa create mode 100644 src/test/hive/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 create mode 100644 src/test/hive/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc create mode 100644 src/test/hive/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 create mode 100644 src/test/hive/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 create mode 100644 src/test/hive/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b create mode 100644 src/test/hive/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 create mode 100644 src/test/hive/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 create mode 100644 src/test/hive/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 create mode 100644 src/test/hive/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea create mode 100644 src/test/hive/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 create mode 100644 src/test/hive/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 create mode 100644 src/test/hive/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd create mode 100644 src/test/hive/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 create mode 100644 src/test/hive/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 create mode 100644 src/test/hive/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 create mode 100644 src/test/hive/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 create mode 100644 src/test/hive/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d create mode 100644 src/test/hive/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 create mode 100644 src/test/hive/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 create mode 100644 src/test/hive/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f create mode 100644 src/test/hive/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 create mode 100644 src/test/hive/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 create mode 100644 src/test/hive/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 create mode 100644 src/test/hive/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc create mode 100644 src/test/hive/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 create mode 100644 src/test/hive/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 create mode 100644 src/test/hive/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb create mode 100644 src/test/hive/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c create mode 100644 src/test/hive/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 create mode 100644 src/test/hive/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c create mode 100644 src/test/hive/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c create mode 100644 src/test/hive/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 create mode 100644 src/test/hive/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad create mode 100644 src/test/hive/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b create mode 100644 src/test/hive/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 create mode 100644 src/test/hive/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b create mode 100644 src/test/hive/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 create mode 100644 src/test/hive/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b create mode 100644 src/test/hive/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 create mode 100644 src/test/hive/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d create mode 100644 src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb create mode 100644 src/test/hive/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a create mode 100644 src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 create mode 100644 src/test/hive/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d create mode 100644 src/test/hive/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab create mode 100644 src/test/hive/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b create mode 100644 src/test/hive/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 create mode 100644 src/test/hive/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 create mode 100644 src/test/hive/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 src/test/hive/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 create mode 100644 src/test/hive/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 src/test/hive/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 create mode 100644 src/test/hive/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 create mode 100644 src/test/hive/golden/archive-16-892147913578bcf60620b7dd73893dd0 create mode 100644 src/test/hive/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 src/test/hive/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a create mode 100644 src/test/hive/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f create mode 100644 src/test/hive/golden/archive-2-713efc113418b01f76ffd589840193c8 create mode 100644 src/test/hive/golden/archive-20-530277b0fee8b05c37b26846bceef827 create mode 100644 src/test/hive/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 create mode 100644 src/test/hive/golden/archive-22-530277b0fee8b05c37b26846bceef827 create mode 100644 src/test/hive/golden/archive-23-892147913578bcf60620b7dd73893dd0 create mode 100644 src/test/hive/golden/archive-24-530277b0fee8b05c37b26846bceef827 create mode 100644 src/test/hive/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f create mode 100644 src/test/hive/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a create mode 100644 src/test/hive/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 create mode 100644 src/test/hive/golden/archive-28-188eb7912265ed8dffa5200517bbe526 create mode 100644 src/test/hive/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 create mode 100644 src/test/hive/golden/archive-3-27ad2962fed131f51ba802596ba37278 create mode 100644 src/test/hive/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a create mode 100644 src/test/hive/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 create mode 100644 src/test/hive/golden/archive-32-27ad2962fed131f51ba802596ba37278 create mode 100644 src/test/hive/golden/archive-4-3e95421993ab28d18245ec2340f580a3 create mode 100644 src/test/hive/golden/archive-5-c0c18ac884677231a41eea8d980d0451 create mode 100644 src/test/hive/golden/archive-6-528ab9750a558af7f1a43b3108e793dd create mode 100644 src/test/hive/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca create mode 100644 src/test/hive/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c create mode 100644 src/test/hive/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf create mode 100644 src/test/hive/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 src/test/hive/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 src/test/hive/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 src/test/hive/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a create mode 100644 src/test/hive/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f create mode 100644 src/test/hive/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f create mode 100644 src/test/hive/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a create mode 100644 src/test/hive/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a create mode 100644 src/test/hive/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 create mode 100644 src/test/hive/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd create mode 100644 src/test/hive/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca create mode 100644 src/test/hive/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c create mode 100644 src/test/hive/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf create mode 100644 src/test/hive/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 create mode 100644 src/test/hive/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 create mode 100644 src/test/hive/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e create mode 100644 src/test/hive/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 create mode 100644 src/test/hive/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 src/test/hive/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 create mode 100644 src/test/hive/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 create mode 100644 src/test/hive/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d create mode 100644 src/test/hive/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 src/test/hive/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 create mode 100644 src/test/hive/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 create mode 100644 src/test/hive/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 create mode 100644 src/test/hive/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 create mode 100644 src/test/hive/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a create mode 100644 src/test/hive/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca create mode 100644 src/test/hive/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c create mode 100644 src/test/hive/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf create mode 100644 src/test/hive/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba create mode 100644 src/test/hive/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 create mode 100644 src/test/hive/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c create mode 100644 src/test/hive/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a create mode 100644 src/test/hive/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e create mode 100644 src/test/hive/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 create mode 100644 src/test/hive/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 create mode 100644 src/test/hive/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e create mode 100644 src/test/hive/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 create mode 100644 src/test/hive/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 create mode 100644 src/test/hive/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 create mode 100644 src/test/hive/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 create mode 100644 src/test/hive/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 create mode 100644 src/test/hive/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 create mode 100644 src/test/hive/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 create mode 100644 src/test/hive/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 create mode 100644 src/test/hive/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 create mode 100644 src/test/hive/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e create mode 100644 src/test/hive/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 create mode 100644 src/test/hive/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 create mode 100644 src/test/hive/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 create mode 100644 src/test/hive/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 create mode 100644 src/test/hive/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 create mode 100644 src/test/hive/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 create mode 100644 src/test/hive/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f create mode 100644 src/test/hive/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 create mode 100644 src/test/hive/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 create mode 100644 src/test/hive/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 create mode 100644 src/test/hive/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 create mode 100644 src/test/hive/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 create mode 100644 src/test/hive/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 create mode 100644 src/test/hive/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf create mode 100644 src/test/hive/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 create mode 100644 src/test/hive/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 create mode 100644 src/test/hive/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 create mode 100644 src/test/hive/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 create mode 100644 src/test/hive/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c create mode 100644 src/test/hive/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b create mode 100644 src/test/hive/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f create mode 100644 src/test/hive/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 create mode 100644 src/test/hive/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 create mode 100644 src/test/hive/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f create mode 100644 src/test/hive/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca create mode 100644 src/test/hive/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 create mode 100644 src/test/hive/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a create mode 100644 src/test/hive/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 create mode 100644 src/test/hive/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd create mode 100644 src/test/hive/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 create mode 100644 src/test/hive/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee create mode 100644 src/test/hive/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 create mode 100644 src/test/hive/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 create mode 100644 src/test/hive/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f create mode 100644 src/test/hive/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 create mode 100644 src/test/hive/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb create mode 100644 src/test/hive/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join23-1-b31437533a2b890788938455cb32f679 create mode 100644 src/test/hive/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 create mode 100644 src/test/hive/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b create mode 100644 src/test/hive/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d create mode 100644 src/test/hive/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 create mode 100644 src/test/hive/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 create mode 100644 src/test/hive/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 create mode 100644 src/test/hive/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e create mode 100644 src/test/hive/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 create mode 100644 src/test/hive/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e create mode 100644 src/test/hive/golden/auto_join25-2-8180638a57b64557e02815c863031755 create mode 100644 src/test/hive/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e create mode 100644 src/test/hive/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 create mode 100644 src/test/hive/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 create mode 100644 src/test/hive/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f create mode 100644 src/test/hive/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca create mode 100644 src/test/hive/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a create mode 100644 src/test/hive/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 create mode 100644 src/test/hive/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 create mode 100644 src/test/hive/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 create mode 100644 src/test/hive/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 create mode 100644 src/test/hive/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 create mode 100644 src/test/hive/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e create mode 100644 src/test/hive/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c create mode 100644 src/test/hive/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 create mode 100644 src/test/hive/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 create mode 100644 src/test/hive/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 create mode 100644 src/test/hive/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e create mode 100644 src/test/hive/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d create mode 100644 src/test/hive/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 create mode 100644 src/test/hive/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f create mode 100644 src/test/hive/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 create mode 100644 src/test/hive/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a create mode 100644 src/test/hive/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 create mode 100644 src/test/hive/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d create mode 100644 src/test/hive/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba create mode 100644 src/test/hive/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 create mode 100644 src/test/hive/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 create mode 100644 src/test/hive/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 create mode 100644 src/test/hive/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 create mode 100644 src/test/hive/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 create mode 100644 src/test/hive/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 create mode 100644 src/test/hive/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 create mode 100644 src/test/hive/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 create mode 100644 src/test/hive/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e create mode 100644 src/test/hive/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 create mode 100644 src/test/hive/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 create mode 100644 src/test/hive/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 create mode 100644 src/test/hive/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 create mode 100644 src/test/hive/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 create mode 100644 src/test/hive/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 create mode 100644 src/test/hive/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 create mode 100644 src/test/hive/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 create mode 100644 src/test/hive/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e create mode 100644 src/test/hive/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f create mode 100644 src/test/hive/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 create mode 100644 src/test/hive/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 create mode 100644 src/test/hive/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 create mode 100644 src/test/hive/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f create mode 100644 src/test/hive/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 create mode 100644 src/test/hive/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d create mode 100644 src/test/hive/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 create mode 100644 src/test/hive/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b create mode 100644 src/test/hive/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c create mode 100644 src/test/hive/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 create mode 100644 src/test/hive/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 create mode 100644 src/test/hive/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b create mode 100644 src/test/hive/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a create mode 100644 src/test/hive/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc create mode 100644 src/test/hive/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 create mode 100644 src/test/hive/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 create mode 100644 src/test/hive/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 create mode 100644 src/test/hive/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 create mode 100644 src/test/hive/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd create mode 100644 src/test/hive/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf create mode 100644 src/test/hive/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 src/test/hive/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef create mode 100644 src/test/hive/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee create mode 100644 src/test/hive/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a create mode 100644 src/test/hive/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd create mode 100644 src/test/hive/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a create mode 100644 src/test/hive/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 create mode 100644 src/test/hive/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 create mode 100644 src/test/hive/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd create mode 100644 src/test/hive/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 create mode 100644 src/test/hive/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 create mode 100644 src/test/hive/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 create mode 100644 src/test/hive/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd create mode 100644 src/test/hive/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 create mode 100644 src/test/hive/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 create mode 100644 src/test/hive/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 create mode 100644 src/test/hive/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 create mode 100644 src/test/hive/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e create mode 100644 src/test/hive/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 create mode 100644 src/test/hive/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 create mode 100644 src/test/hive/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd create mode 100644 src/test/hive/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab create mode 100644 src/test/hive/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca create mode 100644 src/test/hive/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 create mode 100644 src/test/hive/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 create mode 100644 src/test/hive/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 create mode 100644 src/test/hive/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f create mode 100644 src/test/hive/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 create mode 100644 src/test/hive/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e create mode 100644 src/test/hive/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 create mode 100644 src/test/hive/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 create mode 100644 src/test/hive/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 create mode 100644 src/test/hive/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 create mode 100644 src/test/hive/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b create mode 100644 src/test/hive/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e create mode 100644 src/test/hive/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 create mode 100644 src/test/hive/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a create mode 100644 src/test/hive/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d create mode 100644 src/test/hive/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 create mode 100644 src/test/hive/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 create mode 100644 src/test/hive/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf create mode 100644 src/test/hive/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 create mode 100644 src/test/hive/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 create mode 100644 src/test/hive/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 create mode 100644 src/test/hive/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 create mode 100644 src/test/hive/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd create mode 100644 src/test/hive/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 create mode 100644 src/test/hive/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 create mode 100644 src/test/hive/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e create mode 100644 src/test/hive/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e create mode 100644 src/test/hive/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 create mode 100644 src/test/hive/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b create mode 100644 src/test/hive/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 create mode 100644 src/test/hive/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c create mode 100644 src/test/hive/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 create mode 100644 src/test/hive/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 create mode 100644 src/test/hive/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac create mode 100644 src/test/hive/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 create mode 100644 src/test/hive/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 create mode 100644 src/test/hive/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada create mode 100644 src/test/hive/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 create mode 100644 src/test/hive/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 create mode 100644 src/test/hive/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 create mode 100644 src/test/hive/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 create mode 100644 src/test/hive/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b create mode 100644 src/test/hive/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 create mode 100644 src/test/hive/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e create mode 100644 src/test/hive/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 create mode 100644 src/test/hive/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 create mode 100644 src/test/hive/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 create mode 100644 src/test/hive/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 create mode 100644 src/test/hive/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc create mode 100644 src/test/hive/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b create mode 100644 src/test/hive/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e create mode 100644 src/test/hive/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 create mode 100644 src/test/hive/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a create mode 100644 src/test/hive/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d create mode 100644 src/test/hive/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 create mode 100644 src/test/hive/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf create mode 100644 src/test/hive/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 create mode 100644 src/test/hive/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 create mode 100644 src/test/hive/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 create mode 100644 src/test/hive/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 create mode 100644 src/test/hive/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 create mode 100644 src/test/hive/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd create mode 100644 src/test/hive/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 create mode 100644 src/test/hive/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 create mode 100644 src/test/hive/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 create mode 100644 src/test/hive/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b create mode 100644 src/test/hive/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 create mode 100644 src/test/hive/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 create mode 100644 src/test/hive/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 create mode 100644 src/test/hive/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 create mode 100644 src/test/hive/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c create mode 100644 src/test/hive/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 create mode 100644 src/test/hive/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 create mode 100644 src/test/hive/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 create mode 100644 src/test/hive/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af create mode 100644 src/test/hive/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 create mode 100644 src/test/hive/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb create mode 100644 src/test/hive/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 create mode 100644 src/test/hive/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 create mode 100644 src/test/hive/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d create mode 100644 src/test/hive/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a create mode 100644 src/test/hive/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 create mode 100644 src/test/hive/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa create mode 100644 src/test/hive/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 create mode 100644 src/test/hive/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 create mode 100644 src/test/hive/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 create mode 100644 src/test/hive/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc create mode 100644 src/test/hive/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 create mode 100644 src/test/hive/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 create mode 100644 src/test/hive/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 create mode 100644 src/test/hive/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e create mode 100644 src/test/hive/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 create mode 100644 src/test/hive/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 create mode 100644 src/test/hive/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 create mode 100644 src/test/hive/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 create mode 100644 src/test/hive/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 create mode 100644 src/test/hive/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 create mode 100644 src/test/hive/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee create mode 100644 src/test/hive/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 create mode 100644 src/test/hive/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b create mode 100644 src/test/hive/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed create mode 100644 src/test/hive/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d create mode 100644 src/test/hive/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d create mode 100644 src/test/hive/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d create mode 100644 src/test/hive/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb create mode 100644 src/test/hive/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d create mode 100644 src/test/hive/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 create mode 100644 src/test/hive/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 create mode 100644 src/test/hive/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 create mode 100644 src/test/hive/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 create mode 100644 src/test/hive/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 create mode 100644 src/test/hive/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 src/test/hive/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 create mode 100644 src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 create mode 100644 src/test/hive/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 src/test/hive/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a create mode 100644 src/test/hive/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 create mode 100644 src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 create mode 100644 src/test/hive/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 src/test/hive/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d create mode 100644 src/test/hive/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 src/test/hive/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b create mode 100644 src/test/hive/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 src/test/hive/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 create mode 100644 src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 create mode 100644 src/test/hive/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 create mode 100644 src/test/hive/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 create mode 100644 src/test/hive/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef create mode 100644 src/test/hive/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 create mode 100644 src/test/hive/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 create mode 100644 src/test/hive/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 src/test/hive/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 create mode 100644 src/test/hive/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e create mode 100644 src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 create mode 100644 src/test/hive/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 src/test/hive/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 create mode 100644 src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 create mode 100644 src/test/hive/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 create mode 100644 src/test/hive/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 create mode 100644 src/test/hive/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 create mode 100644 src/test/hive/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 create mode 100644 src/test/hive/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 create mode 100644 src/test/hive/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 create mode 100644 src/test/hive/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c create mode 100644 src/test/hive/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a create mode 100644 src/test/hive/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 create mode 100644 src/test/hive/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 create mode 100644 src/test/hive/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 create mode 100644 src/test/hive/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 create mode 100644 src/test/hive/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e create mode 100644 src/test/hive/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 create mode 100644 src/test/hive/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 create mode 100644 src/test/hive/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 create mode 100644 src/test/hive/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 create mode 100644 src/test/hive/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 create mode 100644 src/test/hive/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 create mode 100644 src/test/hive/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 create mode 100644 src/test/hive/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 create mode 100644 src/test/hive/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 create mode 100644 src/test/hive/golden/cluster-20-294891c1d956245540a80aa800ba393d create mode 100644 src/test/hive/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f create mode 100644 src/test/hive/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 create mode 100644 src/test/hive/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e create mode 100644 src/test/hive/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae create mode 100644 src/test/hive/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a create mode 100644 src/test/hive/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f create mode 100644 src/test/hive/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc create mode 100644 src/test/hive/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 create mode 100644 src/test/hive/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 create mode 100644 src/test/hive/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 create mode 100644 src/test/hive/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d create mode 100644 src/test/hive/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 create mode 100644 src/test/hive/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d create mode 100644 src/test/hive/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c create mode 100644 src/test/hive/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 create mode 100644 src/test/hive/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f create mode 100644 src/test/hive/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c create mode 100644 src/test/hive/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 create mode 100644 src/test/hive/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 create mode 100644 src/test/hive/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd create mode 100644 src/test/hive/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 create mode 100644 src/test/hive/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 create mode 100644 src/test/hive/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 create mode 100644 src/test/hive/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 create mode 100644 src/test/hive/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce create mode 100644 src/test/hive/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe create mode 100644 src/test/hive/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 create mode 100644 src/test/hive/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c create mode 100644 src/test/hive/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 create mode 100644 src/test/hive/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 create mode 100644 src/test/hive/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 create mode 100644 src/test/hive/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c create mode 100644 src/test/hive/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 create mode 100644 src/test/hive/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d create mode 100644 src/test/hive/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f create mode 100644 src/test/hive/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 create mode 100644 src/test/hive/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 src/test/hive/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 src/test/hive/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 src/test/hive/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 src/test/hive/golden/combine1-6-1d1f97cce07323812de3027920b04b75 create mode 100644 src/test/hive/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 create mode 100644 src/test/hive/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c create mode 100644 src/test/hive/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 create mode 100644 src/test/hive/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 src/test/hive/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a create mode 100644 src/test/hive/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 create mode 100644 src/test/hive/golden/combine2-12-cd15ffd140539cf86090814729ec4748 create mode 100644 src/test/hive/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 create mode 100644 src/test/hive/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c create mode 100644 src/test/hive/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 create mode 100644 src/test/hive/golden/combine2-16-557997716a68312e8cae75428e3ce31 create mode 100644 src/test/hive/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 create mode 100644 src/test/hive/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 create mode 100644 src/test/hive/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 src/test/hive/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 src/test/hive/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 src/test/hive/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 src/test/hive/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/combine2-7-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 create mode 100644 src/test/hive/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 create mode 100644 src/test/hive/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 src/test/hive/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 create mode 100644 src/test/hive/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 create mode 100644 src/test/hive/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 create mode 100644 src/test/hive/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 create mode 100644 src/test/hive/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c create mode 100644 src/test/hive/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 create mode 100644 src/test/hive/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 create mode 100644 src/test/hive/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 create mode 100644 src/test/hive/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 create mode 100644 src/test/hive/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 src/test/hive/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 src/test/hive/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 src/test/hive/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 src/test/hive/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 create mode 100644 src/test/hive/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 create mode 100644 src/test/hive/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 src/test/hive/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 create mode 100644 src/test/hive/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 create mode 100644 src/test/hive/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 create mode 100644 src/test/hive/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c create mode 100644 src/test/hive/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 create mode 100644 src/test/hive/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 create mode 100644 src/test/hive/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 create mode 100644 src/test/hive/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 create mode 100644 src/test/hive/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 src/test/hive/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 src/test/hive/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 src/test/hive/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 create mode 100644 src/test/hive/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 create mode 100644 src/test/hive/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af create mode 100644 src/test/hive/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 create mode 100644 src/test/hive/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 create mode 100644 src/test/hive/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 create mode 100644 src/test/hive/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b create mode 100644 src/test/hive/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc create mode 100644 src/test/hive/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 create mode 100644 src/test/hive/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a create mode 100644 src/test/hive/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b create mode 100644 src/test/hive/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f create mode 100644 src/test/hive/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 src/test/hive/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd create mode 100644 src/test/hive/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 create mode 100644 src/test/hive/golden/combine3-22-11025483569617a9f014b5defd71e933 create mode 100644 src/test/hive/golden/combine3-23-4725c48df09565618cbffd05953a5f62 create mode 100644 src/test/hive/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 src/test/hive/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 src/test/hive/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 src/test/hive/golden/combine3-6-4725c48df09565618cbffd05953a5f62 create mode 100644 src/test/hive/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 create mode 100644 src/test/hive/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b create mode 100644 src/test/hive/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 create mode 100644 src/test/hive/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 create mode 100644 src/test/hive/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 create mode 100644 src/test/hive/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c create mode 100644 src/test/hive/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 create mode 100644 src/test/hive/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 create mode 100644 src/test/hive/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 create mode 100644 src/test/hive/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e create mode 100644 src/test/hive/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d create mode 100644 src/test/hive/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce create mode 100644 src/test/hive/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 create mode 100644 src/test/hive/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 create mode 100644 src/test/hive/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 create mode 100644 src/test/hive/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d create mode 100644 src/test/hive/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 create mode 100644 src/test/hive/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f create mode 100644 src/test/hive/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b create mode 100644 src/test/hive/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf create mode 100644 src/test/hive/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 create mode 100644 src/test/hive/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 create mode 100644 src/test/hive/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 create mode 100644 src/test/hive/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 create mode 100644 src/test/hive/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 create mode 100644 src/test/hive/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d create mode 100644 src/test/hive/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b create mode 100644 src/test/hive/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f create mode 100644 src/test/hive/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a create mode 100644 src/test/hive/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc create mode 100644 src/test/hive/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 create mode 100644 src/test/hive/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 create mode 100644 src/test/hive/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 create mode 100644 src/test/hive/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 create mode 100644 src/test/hive/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 create mode 100644 src/test/hive/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d create mode 100644 src/test/hive/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 create mode 100644 src/test/hive/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 create mode 100644 src/test/hive/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 create mode 100644 src/test/hive/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 create mode 100644 src/test/hive/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 create mode 100644 src/test/hive/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 create mode 100644 src/test/hive/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 create mode 100644 src/test/hive/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 create mode 100644 src/test/hive/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 create mode 100644 src/test/hive/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda create mode 100644 src/test/hive/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 create mode 100644 src/test/hive/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 create mode 100644 src/test/hive/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf create mode 100644 src/test/hive/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 create mode 100644 src/test/hive/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 create mode 100644 src/test/hive/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 create mode 100644 src/test/hive/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 create mode 100644 src/test/hive/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 create mode 100644 src/test/hive/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c create mode 100644 src/test/hive/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 create mode 100644 src/test/hive/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f create mode 100644 src/test/hive/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f create mode 100644 src/test/hive/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c create mode 100644 src/test/hive/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda create mode 100644 src/test/hive/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 create mode 100644 src/test/hive/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 create mode 100644 src/test/hive/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 create mode 100644 src/test/hive/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 create mode 100644 src/test/hive/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd create mode 100644 src/test/hive/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 create mode 100644 src/test/hive/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab create mode 100644 src/test/hive/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 create mode 100644 src/test/hive/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd create mode 100644 src/test/hive/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 create mode 100644 src/test/hive/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 create mode 100644 src/test/hive/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 create mode 100644 src/test/hive/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 create mode 100644 src/test/hive/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf create mode 100644 src/test/hive/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e create mode 100644 src/test/hive/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b create mode 100644 src/test/hive/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e create mode 100644 src/test/hive/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 create mode 100644 src/test/hive/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 create mode 100644 src/test/hive/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab create mode 100644 src/test/hive/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d create mode 100644 src/test/hive/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 create mode 100644 src/test/hive/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f create mode 100644 src/test/hive/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b create mode 100644 src/test/hive/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 create mode 100644 src/test/hive/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b create mode 100644 src/test/hive/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 create mode 100644 src/test/hive/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d create mode 100644 src/test/hive/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 create mode 100644 src/test/hive/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 create mode 100644 src/test/hive/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 create mode 100644 src/test/hive/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 create mode 100644 src/test/hive/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 create mode 100644 src/test/hive/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 create mode 100644 src/test/hive/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 create mode 100644 src/test/hive/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace create mode 100644 src/test/hive/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe create mode 100644 src/test/hive/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 create mode 100644 src/test/hive/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 create mode 100644 src/test/hive/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 create mode 100644 src/test/hive/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 create mode 100644 src/test/hive/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea create mode 100644 src/test/hive/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 create mode 100644 src/test/hive/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e create mode 100644 src/test/hive/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e create mode 100644 src/test/hive/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 create mode 100644 src/test/hive/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e create mode 100644 src/test/hive/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb create mode 100644 src/test/hive/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 create mode 100644 src/test/hive/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 create mode 100644 src/test/hive/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 create mode 100644 src/test/hive/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 create mode 100644 src/test/hive/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf create mode 100644 src/test/hive/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b create mode 100644 src/test/hive/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb create mode 100644 src/test/hive/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf create mode 100644 src/test/hive/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 create mode 100644 src/test/hive/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf create mode 100644 src/test/hive/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 create mode 100644 src/test/hive/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 create mode 100644 src/test/hive/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 create mode 100644 src/test/hive/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 create mode 100644 src/test/hive/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 create mode 100644 src/test/hive/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 create mode 100644 src/test/hive/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 create mode 100644 src/test/hive/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 create mode 100644 src/test/hive/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 create mode 100644 src/test/hive/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e create mode 100644 src/test/hive/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 create mode 100644 src/test/hive/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 create mode 100644 src/test/hive/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db create mode 100644 src/test/hive/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 create mode 100644 src/test/hive/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db create mode 100644 src/test/hive/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 create mode 100644 src/test/hive/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 create mode 100644 src/test/hive/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de create mode 100644 src/test/hive/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 create mode 100644 src/test/hive/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de create mode 100644 src/test/hive/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 create mode 100644 src/test/hive/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 create mode 100644 src/test/hive/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd create mode 100644 src/test/hive/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 create mode 100644 src/test/hive/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 create mode 100644 src/test/hive/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 create mode 100644 src/test/hive/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 create mode 100644 src/test/hive/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 create mode 100644 src/test/hive/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 create mode 100644 src/test/hive/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb create mode 100644 src/test/hive/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 create mode 100644 src/test/hive/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 create mode 100644 src/test/hive/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 create mode 100644 src/test/hive/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb create mode 100644 src/test/hive/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 create mode 100644 src/test/hive/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 create mode 100644 src/test/hive/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 create mode 100644 src/test/hive/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 create mode 100644 src/test/hive/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 create mode 100644 src/test/hive/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d create mode 100644 src/test/hive/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 create mode 100644 src/test/hive/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d create mode 100644 src/test/hive/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 create mode 100644 src/test/hive/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d create mode 100644 src/test/hive/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 create mode 100644 src/test/hive/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae create mode 100644 src/test/hive/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 create mode 100644 src/test/hive/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 create mode 100644 src/test/hive/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 create mode 100644 src/test/hive/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 create mode 100644 src/test/hive/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 create mode 100644 src/test/hive/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 create mode 100644 src/test/hive/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 create mode 100644 src/test/hive/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 create mode 100644 src/test/hive/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d create mode 100644 src/test/hive/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d create mode 100644 src/test/hive/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f create mode 100644 src/test/hive/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea create mode 100644 src/test/hive/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc create mode 100644 src/test/hive/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a create mode 100644 src/test/hive/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 create mode 100644 src/test/hive/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 create mode 100644 src/test/hive/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 create mode 100644 src/test/hive/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 create mode 100644 src/test/hive/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f create mode 100644 src/test/hive/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b create mode 100644 src/test/hive/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 create mode 100644 src/test/hive/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 create mode 100644 src/test/hive/golden/count-11-29aa086fe75b55482a91316c38498565 create mode 100644 src/test/hive/golden/count-12-944f53db544c07a7b38a0544a21d8e13 create mode 100644 src/test/hive/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 create mode 100644 src/test/hive/golden/count-3-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/count-4-590bf60b8d4dfa135f73dbb52180136f create mode 100644 src/test/hive/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 create mode 100644 src/test/hive/golden/count-6-29aa086fe75b55482a91316c38498565 create mode 100644 src/test/hive/golden/count-7-944f53db544c07a7b38a0544a21d8e13 create mode 100644 src/test/hive/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/count-9-590bf60b8d4dfa135f73dbb52180136f create mode 100644 src/test/hive/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 create mode 100644 src/test/hive/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 create mode 100644 src/test/hive/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e create mode 100644 src/test/hive/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 create mode 100644 src/test/hive/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 create mode 100644 src/test/hive/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 create mode 100644 src/test/hive/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac create mode 100644 src/test/hive/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 create mode 100644 src/test/hive/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f create mode 100644 src/test/hive/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 create mode 100644 src/test/hive/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 create mode 100644 src/test/hive/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c create mode 100644 src/test/hive/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd create mode 100644 src/test/hive/golden/create table as-1-b9002c1d71895be765575b62656d1928 create mode 100644 src/test/hive/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d create mode 100644 src/test/hive/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 create mode 100644 src/test/hive/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba create mode 100644 src/test/hive/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 create mode 100644 src/test/hive/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 create mode 100644 src/test/hive/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a create mode 100644 src/test/hive/golden/create_1-14-437bd1acbae61f48e851addb769d3379 create mode 100644 src/test/hive/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 create mode 100644 src/test/hive/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 create mode 100644 src/test/hive/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 create mode 100644 src/test/hive/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 create mode 100644 src/test/hive/golden/create_1-4-610b82bf7b0080d293977927e5ef780c create mode 100644 src/test/hive/golden/create_1-5-c77b018276b1558c1d9462e0625e152e create mode 100644 src/test/hive/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a create mode 100644 src/test/hive/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 create mode 100644 src/test/hive/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 create mode 100644 src/test/hive/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 create mode 100644 src/test/hive/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d create mode 100644 src/test/hive/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee create mode 100644 src/test/hive/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e create mode 100644 src/test/hive/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 create mode 100644 src/test/hive/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 create mode 100644 src/test/hive/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 create mode 100644 src/test/hive/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 create mode 100644 src/test/hive/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 create mode 100644 src/test/hive/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b create mode 100644 src/test/hive/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 create mode 100644 src/test/hive/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 create mode 100644 src/test/hive/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 create mode 100644 src/test/hive/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 create mode 100644 src/test/hive/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a create mode 100644 src/test/hive/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 create mode 100644 src/test/hive/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc create mode 100644 src/test/hive/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 create mode 100644 src/test/hive/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 create mode 100644 src/test/hive/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c create mode 100644 src/test/hive/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 create mode 100644 src/test/hive/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e create mode 100644 src/test/hive/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 create mode 100644 src/test/hive/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 create mode 100644 src/test/hive/golden/create_like-1-30485a2507b60b96cad3d293527e6af create mode 100644 src/test/hive/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 create mode 100644 src/test/hive/golden/create_like-11-ba64f0122b21f605d8b2928753784130 create mode 100644 src/test/hive/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a create mode 100644 src/test/hive/golden/create_like-3-eea111a209cf8895f31f64699669c705 create mode 100644 src/test/hive/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 create mode 100644 src/test/hive/golden/create_like-5-dc9de26002604e9e436135bd4b40636d create mode 100644 src/test/hive/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 create mode 100644 src/test/hive/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 create mode 100644 src/test/hive/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb create mode 100644 src/test/hive/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb create mode 100644 src/test/hive/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b create mode 100644 src/test/hive/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a create mode 100644 src/test/hive/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 create mode 100644 src/test/hive/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 create mode 100644 src/test/hive/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 create mode 100644 src/test/hive/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 create mode 100644 src/test/hive/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 create mode 100644 src/test/hive/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 create mode 100644 src/test/hive/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb create mode 100644 src/test/hive/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb create mode 100644 src/test/hive/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c create mode 100644 src/test/hive/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f create mode 100644 src/test/hive/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b create mode 100644 src/test/hive/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 create mode 100644 src/test/hive/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 create mode 100644 src/test/hive/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 create mode 100644 src/test/hive/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b create mode 100644 src/test/hive/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 create mode 100644 src/test/hive/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d create mode 100644 src/test/hive/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 create mode 100644 src/test/hive/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 create mode 100644 src/test/hive/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f create mode 100644 src/test/hive/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 create mode 100644 src/test/hive/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 create mode 100644 src/test/hive/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af create mode 100644 src/test/hive/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 create mode 100644 src/test/hive/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 create mode 100644 src/test/hive/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a create mode 100644 src/test/hive/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c create mode 100644 src/test/hive/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 create mode 100644 src/test/hive/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 create mode 100644 src/test/hive/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d create mode 100644 src/test/hive/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 create mode 100644 src/test/hive/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f create mode 100644 src/test/hive/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 create mode 100644 src/test/hive/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 create mode 100644 src/test/hive/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 create mode 100644 src/test/hive/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 create mode 100644 src/test/hive/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 create mode 100644 src/test/hive/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 create mode 100644 src/test/hive/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d create mode 100644 src/test/hive/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 create mode 100644 src/test/hive/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 create mode 100644 src/test/hive/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 create mode 100644 src/test/hive/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 create mode 100644 src/test/hive/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 create mode 100644 src/test/hive/golden/create_nested_type-3-ac452c9279877935983c37113898e53c create mode 100644 src/test/hive/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff create mode 100644 src/test/hive/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d create mode 100644 src/test/hive/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea create mode 100644 src/test/hive/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 create mode 100644 src/test/hive/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d create mode 100644 src/test/hive/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de create mode 100644 src/test/hive/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb create mode 100644 src/test/hive/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 create mode 100644 src/test/hive/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e create mode 100644 src/test/hive/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 create mode 100644 src/test/hive/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 create mode 100644 src/test/hive/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e create mode 100644 src/test/hive/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef create mode 100644 src/test/hive/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 create mode 100644 src/test/hive/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd create mode 100644 src/test/hive/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e create mode 100644 src/test/hive/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 create mode 100644 src/test/hive/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 create mode 100644 src/test/hive/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd create mode 100644 src/test/hive/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e create mode 100644 src/test/hive/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 create mode 100644 src/test/hive/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 create mode 100644 src/test/hive/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd create mode 100644 src/test/hive/golden/create_view-11-2021c047203276dd2db51a56e672fcea create mode 100644 src/test/hive/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc create mode 100644 src/test/hive/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a create mode 100644 src/test/hive/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 create mode 100644 src/test/hive/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 create mode 100644 src/test/hive/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 create mode 100644 src/test/hive/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a create mode 100644 src/test/hive/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 create mode 100644 src/test/hive/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b create mode 100644 src/test/hive/golden/create_view-2-d80dcd1271ab264292e9938f3162427c create mode 100644 src/test/hive/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 create mode 100644 src/test/hive/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 create mode 100644 src/test/hive/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 create mode 100644 src/test/hive/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 create mode 100644 src/test/hive/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 create mode 100644 src/test/hive/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 create mode 100644 src/test/hive/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 create mode 100644 src/test/hive/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 create mode 100644 src/test/hive/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 create mode 100644 src/test/hive/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 create mode 100644 src/test/hive/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc create mode 100644 src/test/hive/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 create mode 100644 src/test/hive/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 create mode 100644 src/test/hive/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 create mode 100644 src/test/hive/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 create mode 100644 src/test/hive/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc create mode 100644 src/test/hive/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 create mode 100644 src/test/hive/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 create mode 100644 src/test/hive/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 create mode 100644 src/test/hive/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e create mode 100644 src/test/hive/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 create mode 100644 src/test/hive/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 create mode 100644 src/test/hive/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b create mode 100644 src/test/hive/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 create mode 100644 src/test/hive/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f create mode 100644 src/test/hive/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 create mode 100644 src/test/hive/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec create mode 100644 src/test/hive/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 create mode 100644 src/test/hive/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 create mode 100644 src/test/hive/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd create mode 100644 src/test/hive/golden/database.table-0-c657beb729b6a7882309a203fc6f298e create mode 100644 src/test/hive/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f create mode 100644 src/test/hive/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 create mode 100644 src/test/hive/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 create mode 100644 src/test/hive/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 create mode 100644 src/test/hive/golden/database_drop-12-e02a53f7e798d2741152526516f14941 create mode 100644 src/test/hive/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 create mode 100644 src/test/hive/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 create mode 100644 src/test/hive/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 create mode 100644 src/test/hive/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 create mode 100644 src/test/hive/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 create mode 100644 src/test/hive/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba create mode 100644 src/test/hive/golden/database_drop-8-97101266791d2b2c662bcde549422318 create mode 100644 src/test/hive/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 create mode 100644 src/test/hive/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a create mode 100644 src/test/hive/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 create mode 100644 src/test/hive/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 create mode 100644 src/test/hive/golden/database_location-11-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 create mode 100644 src/test/hive/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 create mode 100644 src/test/hive/golden/database_location-4-be65cf649363681d54e593c42a5ecffb create mode 100644 src/test/hive/golden/database_location-5-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a create mode 100644 src/test/hive/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 create mode 100644 src/test/hive/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b create mode 100644 src/test/hive/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 create mode 100644 src/test/hive/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 create mode 100644 src/test/hive/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 create mode 100644 src/test/hive/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 create mode 100644 src/test/hive/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 create mode 100644 src/test/hive/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 create mode 100644 src/test/hive/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a create mode 100644 src/test/hive/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 create mode 100644 src/test/hive/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 create mode 100644 src/test/hive/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a create mode 100644 src/test/hive/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd create mode 100644 src/test/hive/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 create mode 100644 src/test/hive/golden/date_2-2-cab14d992c53c106ab257fae52001e04 create mode 100644 src/test/hive/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 create mode 100644 src/test/hive/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 create mode 100644 src/test/hive/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 create mode 100644 src/test/hive/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c create mode 100644 src/test/hive/golden/date_join1-1-3a68de2112a212a07a3068916c608fb create mode 100644 src/test/hive/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 create mode 100644 src/test/hive/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 create mode 100644 src/test/hive/golden/date_serde-1-36e6041f53433482631018410bb62a99 create mode 100644 src/test/hive/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 create mode 100644 src/test/hive/golden/date_serde-3-e6167e27465514356c557a77d956ea46 create mode 100644 src/test/hive/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 create mode 100644 src/test/hive/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c create mode 100644 src/test/hive/golden/date_serde-6-e00facec2986bc55074868eff87ba22a create mode 100644 src/test/hive/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d create mode 100644 src/test/hive/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 create mode 100644 src/test/hive/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 create mode 100644 src/test/hive/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 create mode 100644 src/test/hive/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd create mode 100644 src/test/hive/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 create mode 100644 src/test/hive/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 create mode 100644 src/test/hive/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 create mode 100644 src/test/hive/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 create mode 100644 src/test/hive/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 create mode 100644 src/test/hive/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 create mode 100644 src/test/hive/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 create mode 100644 src/test/hive/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 create mode 100644 src/test/hive/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b create mode 100644 src/test/hive/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a create mode 100644 src/test/hive/golden/decimal_1-8-cdd0932288d3cc43636334439805769d create mode 100644 src/test/hive/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c create mode 100644 src/test/hive/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee create mode 100644 src/test/hive/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b create mode 100644 src/test/hive/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 create mode 100644 src/test/hive/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 create mode 100644 src/test/hive/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 create mode 100644 src/test/hive/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 create mode 100644 src/test/hive/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 create mode 100644 src/test/hive/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 create mode 100644 src/test/hive/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 create mode 100644 src/test/hive/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d create mode 100644 src/test/hive/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e create mode 100644 src/test/hive/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 create mode 100644 src/test/hive/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 create mode 100644 src/test/hive/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 create mode 100644 src/test/hive/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 create mode 100644 src/test/hive/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 create mode 100644 src/test/hive/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff create mode 100644 src/test/hive/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e create mode 100644 src/test/hive/golden/decimal_2-25-14face5c7104382196e65741a199c36 create mode 100644 src/test/hive/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 create mode 100644 src/test/hive/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 create mode 100644 src/test/hive/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 create mode 100644 src/test/hive/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 create mode 100644 src/test/hive/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e create mode 100644 src/test/hive/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a create mode 100644 src/test/hive/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee create mode 100644 src/test/hive/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 create mode 100644 src/test/hive/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 create mode 100644 src/test/hive/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 create mode 100644 src/test/hive/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 create mode 100644 src/test/hive/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d create mode 100644 src/test/hive/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e create mode 100644 src/test/hive/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef create mode 100644 src/test/hive/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 create mode 100644 src/test/hive/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 create mode 100644 src/test/hive/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 create mode 100644 src/test/hive/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 create mode 100644 src/test/hive/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 create mode 100644 src/test/hive/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b create mode 100644 src/test/hive/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 create mode 100644 src/test/hive/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b create mode 100644 src/test/hive/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 create mode 100644 src/test/hive/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 create mode 100644 src/test/hive/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 create mode 100644 src/test/hive/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 create mode 100644 src/test/hive/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f create mode 100644 src/test/hive/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 create mode 100644 src/test/hive/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d create mode 100644 src/test/hive/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e create mode 100644 src/test/hive/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c create mode 100644 src/test/hive/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c create mode 100644 src/test/hive/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e create mode 100644 src/test/hive/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 create mode 100644 src/test/hive/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 create mode 100644 src/test/hive/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 create mode 100644 src/test/hive/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 create mode 100644 src/test/hive/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e create mode 100644 src/test/hive/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 create mode 100644 src/test/hive/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e create mode 100644 src/test/hive/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 create mode 100644 src/test/hive/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 create mode 100644 src/test/hive/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 create mode 100644 src/test/hive/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e create mode 100644 src/test/hive/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e create mode 100644 src/test/hive/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 create mode 100644 src/test/hive/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce create mode 100644 src/test/hive/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c create mode 100644 src/test/hive/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b create mode 100644 src/test/hive/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc create mode 100644 src/test/hive/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef create mode 100644 src/test/hive/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 create mode 100644 src/test/hive/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 create mode 100644 src/test/hive/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e create mode 100644 src/test/hive/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab create mode 100644 src/test/hive/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 create mode 100644 src/test/hive/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c create mode 100644 src/test/hive/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 create mode 100644 src/test/hive/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be create mode 100644 src/test/hive/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba create mode 100644 src/test/hive/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 create mode 100644 src/test/hive/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 create mode 100644 src/test/hive/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 create mode 100644 src/test/hive/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 create mode 100644 src/test/hive/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb create mode 100644 src/test/hive/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 create mode 100644 src/test/hive/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 create mode 100644 src/test/hive/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b create mode 100644 src/test/hive/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 create mode 100644 src/test/hive/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 create mode 100644 src/test/hive/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 create mode 100644 src/test/hive/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a create mode 100644 src/test/hive/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 create mode 100644 src/test/hive/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 create mode 100644 src/test/hive/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c create mode 100644 src/test/hive/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 create mode 100644 src/test/hive/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 create mode 100644 src/test/hive/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a create mode 100644 src/test/hive/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 create mode 100644 src/test/hive/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 create mode 100644 src/test/hive/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad create mode 100644 src/test/hive/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf create mode 100644 src/test/hive/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf create mode 100644 src/test/hive/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 create mode 100644 src/test/hive/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 create mode 100644 src/test/hive/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 create mode 100644 src/test/hive/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 create mode 100644 src/test/hive/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 create mode 100644 src/test/hive/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 create mode 100644 src/test/hive/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 create mode 100644 src/test/hive/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 create mode 100644 src/test/hive/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 create mode 100644 src/test/hive/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 create mode 100644 src/test/hive/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 create mode 100644 src/test/hive/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 create mode 100644 src/test/hive/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 create mode 100644 src/test/hive/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 create mode 100644 src/test/hive/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a create mode 100644 src/test/hive/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 create mode 100644 src/test/hive/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 create mode 100644 src/test/hive/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f create mode 100644 src/test/hive/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 create mode 100644 src/test/hive/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 create mode 100644 src/test/hive/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 create mode 100644 src/test/hive/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 create mode 100644 src/test/hive/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 create mode 100644 src/test/hive/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 create mode 100644 src/test/hive/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 create mode 100644 src/test/hive/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa create mode 100644 src/test/hive/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a create mode 100644 src/test/hive/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a create mode 100644 src/test/hive/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 create mode 100644 src/test/hive/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f create mode 100644 src/test/hive/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 create mode 100644 src/test/hive/golden/describe_syntax-24-ee226b42db35b219702319858e925468 create mode 100644 src/test/hive/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e create mode 100644 src/test/hive/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec create mode 100644 src/test/hive/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 create mode 100644 src/test/hive/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 create mode 100644 src/test/hive/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 create mode 100644 src/test/hive/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 create mode 100644 src/test/hive/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d create mode 100644 src/test/hive/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 create mode 100644 src/test/hive/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b create mode 100644 src/test/hive/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b create mode 100644 src/test/hive/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 create mode 100644 src/test/hive/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/describe_table_json-3-576670af142149302decb9bf8662e68a create mode 100644 src/test/hive/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 create mode 100644 src/test/hive/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb create mode 100644 src/test/hive/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 create mode 100644 src/test/hive/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 create mode 100644 src/test/hive/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 create mode 100644 src/test/hive/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 create mode 100644 src/test/hive/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad create mode 100644 src/test/hive/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa create mode 100644 src/test/hive/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce create mode 100644 src/test/hive/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 create mode 100644 src/test/hive/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b create mode 100644 src/test/hive/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c create mode 100644 src/test/hive/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e create mode 100644 src/test/hive/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 create mode 100644 src/test/hive/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f create mode 100644 src/test/hive/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb create mode 100644 src/test/hive/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a create mode 100644 src/test/hive/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b create mode 100644 src/test/hive/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 create mode 100644 src/test/hive/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 create mode 100644 src/test/hive/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c create mode 100644 src/test/hive/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 create mode 100644 src/test/hive/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab create mode 100644 src/test/hive/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 create mode 100644 src/test/hive/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 create mode 100644 src/test/hive/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a create mode 100644 src/test/hive/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 create mode 100644 src/test/hive/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 create mode 100644 src/test/hive/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b create mode 100644 src/test/hive/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 create mode 100644 src/test/hive/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 create mode 100644 src/test/hive/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 create mode 100644 src/test/hive/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 create mode 100644 src/test/hive/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 create mode 100644 src/test/hive/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 create mode 100644 src/test/hive/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 create mode 100644 src/test/hive/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 create mode 100644 src/test/hive/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 create mode 100644 src/test/hive/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 create mode 100644 src/test/hive/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 create mode 100644 src/test/hive/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 create mode 100644 src/test/hive/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b create mode 100644 src/test/hive/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 create mode 100644 src/test/hive/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 create mode 100644 src/test/hive/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b create mode 100644 src/test/hive/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd create mode 100644 src/test/hive/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 create mode 100644 src/test/hive/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c create mode 100644 src/test/hive/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 create mode 100644 src/test/hive/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e create mode 100644 src/test/hive/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c create mode 100644 src/test/hive/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c create mode 100644 src/test/hive/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d create mode 100644 src/test/hive/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 create mode 100644 src/test/hive/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e create mode 100644 src/test/hive/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 create mode 100644 src/test/hive/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 create mode 100644 src/test/hive/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 create mode 100644 src/test/hive/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 create mode 100644 src/test/hive/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 create mode 100644 src/test/hive/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a create mode 100644 src/test/hive/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c create mode 100644 src/test/hive/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 create mode 100644 src/test/hive/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 create mode 100644 src/test/hive/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 create mode 100644 src/test/hive/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a create mode 100644 src/test/hive/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 create mode 100644 src/test/hive/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 create mode 100644 src/test/hive/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a create mode 100644 src/test/hive/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 create mode 100644 src/test/hive/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 create mode 100644 src/test/hive/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 create mode 100644 src/test/hive/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c create mode 100644 src/test/hive/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 create mode 100644 src/test/hive/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec create mode 100644 src/test/hive/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a create mode 100644 src/test/hive/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 create mode 100644 src/test/hive/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae create mode 100644 src/test/hive/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 create mode 100644 src/test/hive/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 src/test/hive/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 create mode 100644 src/test/hive/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb create mode 100644 src/test/hive/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 create mode 100644 src/test/hive/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 create mode 100644 src/test/hive/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 create mode 100644 src/test/hive/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 create mode 100644 src/test/hive/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 create mode 100644 src/test/hive/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 create mode 100644 src/test/hive/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 create mode 100644 src/test/hive/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 create mode 100644 src/test/hive/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa create mode 100644 src/test/hive/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 create mode 100644 src/test/hive/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f create mode 100644 src/test/hive/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 create mode 100644 src/test/hive/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 create mode 100644 src/test/hive/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 create mode 100644 src/test/hive/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa create mode 100644 src/test/hive/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 create mode 100644 src/test/hive/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 create mode 100644 src/test/hive/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 create mode 100644 src/test/hive/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 create mode 100644 src/test/hive/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 create mode 100644 src/test/hive/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 create mode 100644 src/test/hive/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 create mode 100644 src/test/hive/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 create mode 100644 src/test/hive/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 create mode 100644 src/test/hive/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 create mode 100644 src/test/hive/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 create mode 100644 src/test/hive/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 create mode 100644 src/test/hive/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 create mode 100644 src/test/hive/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 create mode 100644 src/test/hive/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d create mode 100644 src/test/hive/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f create mode 100644 src/test/hive/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 create mode 100644 src/test/hive/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 create mode 100644 src/test/hive/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 create mode 100644 src/test/hive/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf create mode 100644 src/test/hive/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 create mode 100644 src/test/hive/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 create mode 100644 src/test/hive/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 create mode 100644 src/test/hive/golden/escape1-3-4267651148da591da38737028fdbd80 create mode 100644 src/test/hive/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 create mode 100644 src/test/hive/golden/escape1-5-70729c3d79ded87e884c176138174645 create mode 100644 src/test/hive/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 create mode 100644 src/test/hive/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 create mode 100644 src/test/hive/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 create mode 100644 src/test/hive/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 create mode 100644 src/test/hive/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e create mode 100644 src/test/hive/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 create mode 100644 src/test/hive/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe create mode 100644 src/test/hive/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 create mode 100644 src/test/hive/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 create mode 100644 src/test/hive/golden/escape2-7-70729c3d79ded87e884c176138174645 create mode 100644 src/test/hive/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 create mode 100644 src/test/hive/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 create mode 100644 src/test/hive/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf create mode 100644 src/test/hive/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 create mode 100644 src/test/hive/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb create mode 100644 src/test/hive/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 create mode 100644 src/test/hive/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 create mode 100644 src/test/hive/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 create mode 100644 src/test/hive/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 create mode 100644 src/test/hive/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 create mode 100644 src/test/hive/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 src/test/hive/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 create mode 100644 src/test/hive/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 src/test/hive/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 src/test/hive/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 src/test/hive/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 src/test/hive/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 src/test/hive/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 src/test/hive/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 create mode 100644 src/test/hive/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 create mode 100644 src/test/hive/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 create mode 100644 src/test/hive/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 create mode 100644 src/test/hive/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 create mode 100644 src/test/hive/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 create mode 100644 src/test/hive/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 create mode 100644 src/test/hive/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 create mode 100644 src/test/hive/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 src/test/hive/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 src/test/hive/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 src/test/hive/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d create mode 100644 src/test/hive/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 create mode 100644 src/test/hive/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 src/test/hive/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 src/test/hive/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 src/test/hive/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 src/test/hive/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 src/test/hive/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 src/test/hive/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 src/test/hive/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 src/test/hive/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 src/test/hive/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 src/test/hive/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 src/test/hive/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 src/test/hive/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 create mode 100644 src/test/hive/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 src/test/hive/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 src/test/hive/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 src/test/hive/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 create mode 100644 src/test/hive/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 src/test/hive/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e create mode 100644 src/test/hive/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 create mode 100644 src/test/hive/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 create mode 100644 src/test/hive/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 create mode 100644 src/test/hive/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 src/test/hive/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 src/test/hive/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 src/test/hive/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 src/test/hive/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 src/test/hive/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 src/test/hive/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a create mode 100644 src/test/hive/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 src/test/hive/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 create mode 100644 src/test/hive/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 src/test/hive/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d create mode 100644 src/test/hive/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 src/test/hive/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 create mode 100644 src/test/hive/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 create mode 100644 src/test/hive/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 create mode 100644 src/test/hive/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 create mode 100644 src/test/hive/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b create mode 100644 src/test/hive/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e create mode 100644 src/test/hive/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 create mode 100644 src/test/hive/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 create mode 100644 src/test/hive/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 create mode 100644 src/test/hive/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 create mode 100644 src/test/hive/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 create mode 100644 src/test/hive/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 create mode 100644 src/test/hive/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 create mode 100644 src/test/hive/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e create mode 100644 src/test/hive/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 create mode 100644 src/test/hive/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd create mode 100644 src/test/hive/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd create mode 100644 src/test/hive/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 create mode 100644 src/test/hive/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 create mode 100644 src/test/hive/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c create mode 100644 src/test/hive/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 create mode 100644 src/test/hive/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 create mode 100644 src/test/hive/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba create mode 100644 src/test/hive/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 create mode 100644 src/test/hive/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 create mode 100644 src/test/hive/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e create mode 100644 src/test/hive/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 create mode 100644 src/test/hive/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby1-2-f90acd59ace31e16059bae52583188cc create mode 100644 src/test/hive/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d create mode 100644 src/test/hive/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 create mode 100644 src/test/hive/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 create mode 100644 src/test/hive/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea create mode 100644 src/test/hive/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 create mode 100644 src/test/hive/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 create mode 100644 src/test/hive/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 create mode 100644 src/test/hive/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 create mode 100644 src/test/hive/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f create mode 100644 src/test/hive/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd create mode 100644 src/test/hive/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d create mode 100644 src/test/hive/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 create mode 100644 src/test/hive/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 create mode 100644 src/test/hive/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 create mode 100644 src/test/hive/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 create mode 100644 src/test/hive/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 create mode 100644 src/test/hive/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe create mode 100644 src/test/hive/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 create mode 100644 src/test/hive/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 create mode 100644 src/test/hive/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 create mode 100644 src/test/hive/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe create mode 100644 src/test/hive/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 create mode 100644 src/test/hive/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 create mode 100644 src/test/hive/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe create mode 100644 src/test/hive/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 create mode 100644 src/test/hive/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc create mode 100644 src/test/hive/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 create mode 100644 src/test/hive/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea create mode 100644 src/test/hive/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 create mode 100644 src/test/hive/golden/groupby2-0-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b create mode 100644 src/test/hive/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 create mode 100644 src/test/hive/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 create mode 100644 src/test/hive/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c create mode 100644 src/test/hive/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 create mode 100644 src/test/hive/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 create mode 100644 src/test/hive/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 create mode 100644 src/test/hive/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 create mode 100644 src/test/hive/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 create mode 100644 src/test/hive/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 create mode 100644 src/test/hive/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 create mode 100644 src/test/hive/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 create mode 100644 src/test/hive/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 create mode 100644 src/test/hive/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 create mode 100644 src/test/hive/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 create mode 100644 src/test/hive/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b create mode 100644 src/test/hive/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 create mode 100644 src/test/hive/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 create mode 100644 src/test/hive/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c create mode 100644 src/test/hive/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 create mode 100644 src/test/hive/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 create mode 100644 src/test/hive/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 create mode 100644 src/test/hive/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c create mode 100644 src/test/hive/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 create mode 100644 src/test/hive/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 create mode 100644 src/test/hive/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 create mode 100644 src/test/hive/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b create mode 100644 src/test/hive/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 create mode 100644 src/test/hive/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 create mode 100644 src/test/hive/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b create mode 100644 src/test/hive/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 create mode 100644 src/test/hive/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 create mode 100644 src/test/hive/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 create mode 100644 src/test/hive/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 create mode 100644 src/test/hive/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e create mode 100644 src/test/hive/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 create mode 100644 src/test/hive/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f create mode 100644 src/test/hive/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 create mode 100644 src/test/hive/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 create mode 100644 src/test/hive/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f create mode 100644 src/test/hive/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 create mode 100644 src/test/hive/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 create mode 100644 src/test/hive/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e create mode 100644 src/test/hive/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac create mode 100644 src/test/hive/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 create mode 100644 src/test/hive/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac create mode 100644 src/test/hive/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 create mode 100644 src/test/hive/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac create mode 100644 src/test/hive/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 create mode 100644 src/test/hive/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac create mode 100644 src/test/hive/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 create mode 100644 src/test/hive/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a create mode 100644 src/test/hive/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 src/test/hive/golden/groupby7-5-ed76c0068780120a6f23feefee303403 create mode 100644 src/test/hive/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 create mode 100644 src/test/hive/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e create mode 100644 src/test/hive/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a create mode 100644 src/test/hive/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 src/test/hive/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 create mode 100644 src/test/hive/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd create mode 100644 src/test/hive/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 create mode 100644 src/test/hive/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 src/test/hive/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 src/test/hive/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 create mode 100644 src/test/hive/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd create mode 100644 src/test/hive/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 create mode 100644 src/test/hive/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e create mode 100644 src/test/hive/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a create mode 100644 src/test/hive/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 src/test/hive/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 create mode 100644 src/test/hive/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd create mode 100644 src/test/hive/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a create mode 100644 src/test/hive/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 create mode 100644 src/test/hive/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f create mode 100644 src/test/hive/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a create mode 100644 src/test/hive/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d create mode 100644 src/test/hive/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f create mode 100644 src/test/hive/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e create mode 100644 src/test/hive/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d create mode 100644 src/test/hive/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 src/test/hive/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d create mode 100644 src/test/hive/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f create mode 100644 src/test/hive/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 src/test/hive/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d create mode 100644 src/test/hive/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f create mode 100644 src/test/hive/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 src/test/hive/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d create mode 100644 src/test/hive/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f create mode 100644 src/test/hive/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 create mode 100644 src/test/hive/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 create mode 100644 src/test/hive/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e create mode 100644 src/test/hive/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f create mode 100644 src/test/hive/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 create mode 100644 src/test/hive/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed create mode 100644 src/test/hive/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b create mode 100644 src/test/hive/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e create mode 100644 src/test/hive/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f create mode 100644 src/test/hive/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 create mode 100644 src/test/hive/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 create mode 100644 src/test/hive/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e create mode 100644 src/test/hive/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 create mode 100644 src/test/hive/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e create mode 100644 src/test/hive/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 create mode 100644 src/test/hive/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 create mode 100644 src/test/hive/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 create mode 100644 src/test/hive/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 create mode 100644 src/test/hive/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a create mode 100644 src/test/hive/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 create mode 100644 src/test/hive/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 create mode 100644 src/test/hive/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 create mode 100644 src/test/hive/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 create mode 100644 src/test/hive/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 create mode 100644 src/test/hive/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 create mode 100644 src/test/hive/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 create mode 100644 src/test/hive/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 create mode 100644 src/test/hive/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 create mode 100644 src/test/hive/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 create mode 100644 src/test/hive/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad create mode 100644 src/test/hive/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 create mode 100644 src/test/hive/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 create mode 100644 src/test/hive/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 create mode 100644 src/test/hive/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 create mode 100644 src/test/hive/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a create mode 100644 src/test/hive/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 create mode 100644 src/test/hive/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 create mode 100644 src/test/hive/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 create mode 100644 src/test/hive/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 create mode 100644 src/test/hive/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e create mode 100644 src/test/hive/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada create mode 100644 src/test/hive/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e create mode 100644 src/test/hive/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab create mode 100644 src/test/hive/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 create mode 100644 src/test/hive/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 src/test/hive/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb create mode 100644 src/test/hive/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 create mode 100644 src/test/hive/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 create mode 100644 src/test/hive/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 create mode 100644 src/test/hive/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 create mode 100644 src/test/hive/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 create mode 100644 src/test/hive/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f create mode 100644 src/test/hive/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 create mode 100644 src/test/hive/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb create mode 100644 src/test/hive/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 create mode 100644 src/test/hive/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac create mode 100644 src/test/hive/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c create mode 100644 src/test/hive/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d create mode 100644 src/test/hive/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 create mode 100644 src/test/hive/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 create mode 100644 src/test/hive/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c create mode 100644 src/test/hive/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d create mode 100644 src/test/hive/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 create mode 100644 src/test/hive/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 create mode 100644 src/test/hive/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a create mode 100644 src/test/hive/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 create mode 100644 src/test/hive/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d create mode 100644 src/test/hive/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d create mode 100644 src/test/hive/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 create mode 100644 src/test/hive/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c create mode 100644 src/test/hive/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 create mode 100644 src/test/hive/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 create mode 100644 src/test/hive/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea create mode 100644 src/test/hive/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 create mode 100644 src/test/hive/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 create mode 100644 src/test/hive/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 create mode 100644 src/test/hive/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 create mode 100644 src/test/hive/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 create mode 100644 src/test/hive/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 create mode 100644 src/test/hive/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 create mode 100644 src/test/hive/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 create mode 100644 src/test/hive/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 create mode 100644 src/test/hive/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f create mode 100644 src/test/hive/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca create mode 100644 src/test/hive/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 create mode 100644 src/test/hive/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 create mode 100644 src/test/hive/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 create mode 100644 src/test/hive/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f create mode 100644 src/test/hive/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef create mode 100644 src/test/hive/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 create mode 100644 src/test/hive/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 create mode 100644 src/test/hive/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d create mode 100644 src/test/hive/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b create mode 100644 src/test/hive/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad create mode 100644 src/test/hive/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 create mode 100644 src/test/hive/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c create mode 100644 src/test/hive/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a create mode 100644 src/test/hive/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 create mode 100644 src/test/hive/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 create mode 100644 src/test/hive/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa create mode 100644 src/test/hive/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 create mode 100644 src/test/hive/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 create mode 100644 src/test/hive/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 create mode 100644 src/test/hive/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca create mode 100644 src/test/hive/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 create mode 100644 src/test/hive/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c create mode 100644 src/test/hive/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 create mode 100644 src/test/hive/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 create mode 100644 src/test/hive/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 create mode 100644 src/test/hive/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 create mode 100644 src/test/hive/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca create mode 100644 src/test/hive/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b create mode 100644 src/test/hive/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 create mode 100644 src/test/hive/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 create mode 100644 src/test/hive/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca create mode 100644 src/test/hive/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e create mode 100644 src/test/hive/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 create mode 100644 src/test/hive/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 create mode 100644 src/test/hive/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 src/test/hive/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 create mode 100644 src/test/hive/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 create mode 100644 src/test/hive/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc create mode 100644 src/test/hive/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e create mode 100644 src/test/hive/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 create mode 100644 src/test/hive/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd create mode 100644 src/test/hive/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 create mode 100644 src/test/hive/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 create mode 100644 src/test/hive/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 create mode 100644 src/test/hive/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 create mode 100644 src/test/hive/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 create mode 100644 src/test/hive/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 create mode 100644 src/test/hive/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 create mode 100644 src/test/hive/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 create mode 100644 src/test/hive/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e create mode 100644 src/test/hive/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 create mode 100644 src/test/hive/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 create mode 100644 src/test/hive/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 create mode 100644 src/test/hive/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 create mode 100644 src/test/hive/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c create mode 100644 src/test/hive/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d create mode 100644 src/test/hive/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 create mode 100644 src/test/hive/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 create mode 100644 src/test/hive/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 create mode 100644 src/test/hive/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 create mode 100644 src/test/hive/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 create mode 100644 src/test/hive/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 create mode 100644 src/test/hive/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 create mode 100644 src/test/hive/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 create mode 100644 src/test/hive/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d create mode 100644 src/test/hive/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e create mode 100644 src/test/hive/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 create mode 100644 src/test/hive/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 create mode 100644 src/test/hive/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d create mode 100644 src/test/hive/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c create mode 100644 src/test/hive/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 create mode 100644 src/test/hive/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 create mode 100644 src/test/hive/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 create mode 100644 src/test/hive/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b create mode 100644 src/test/hive/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 create mode 100644 src/test/hive/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 create mode 100644 src/test/hive/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed create mode 100644 src/test/hive/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 create mode 100644 src/test/hive/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 create mode 100644 src/test/hive/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e create mode 100644 src/test/hive/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd create mode 100644 src/test/hive/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f create mode 100644 src/test/hive/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 create mode 100644 src/test/hive/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b create mode 100644 src/test/hive/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 create mode 100644 src/test/hive/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 create mode 100644 src/test/hive/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd create mode 100644 src/test/hive/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 create mode 100644 src/test/hive/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f create mode 100644 src/test/hive/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 create mode 100644 src/test/hive/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 create mode 100644 src/test/hive/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f create mode 100644 src/test/hive/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b create mode 100644 src/test/hive/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e create mode 100644 src/test/hive/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef create mode 100644 src/test/hive/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 create mode 100644 src/test/hive/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 create mode 100644 src/test/hive/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 create mode 100644 src/test/hive/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f create mode 100644 src/test/hive/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b create mode 100644 src/test/hive/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 create mode 100644 src/test/hive/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd create mode 100644 src/test/hive/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 create mode 100644 src/test/hive/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 create mode 100644 src/test/hive/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf create mode 100644 src/test/hive/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b create mode 100644 src/test/hive/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d create mode 100644 src/test/hive/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f create mode 100644 src/test/hive/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd create mode 100644 src/test/hive/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f create mode 100644 src/test/hive/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 create mode 100644 src/test/hive/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 create mode 100644 src/test/hive/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b create mode 100644 src/test/hive/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 create mode 100644 src/test/hive/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 create mode 100644 src/test/hive/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd create mode 100644 src/test/hive/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 create mode 100644 src/test/hive/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 create mode 100644 src/test/hive/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 create mode 100644 src/test/hive/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 create mode 100644 src/test/hive/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e create mode 100644 src/test/hive/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 create mode 100644 src/test/hive/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 create mode 100644 src/test/hive/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 create mode 100644 src/test/hive/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 create mode 100644 src/test/hive/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 create mode 100644 src/test/hive/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 create mode 100644 src/test/hive/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 create mode 100644 src/test/hive/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb create mode 100644 src/test/hive/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 create mode 100644 src/test/hive/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 create mode 100644 src/test/hive/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b create mode 100644 src/test/hive/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 create mode 100644 src/test/hive/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 create mode 100644 src/test/hive/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d create mode 100644 src/test/hive/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 create mode 100644 src/test/hive/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f create mode 100644 src/test/hive/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb create mode 100644 src/test/hive/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 create mode 100644 src/test/hive/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d create mode 100644 src/test/hive/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d create mode 100644 src/test/hive/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a create mode 100644 src/test/hive/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 create mode 100644 src/test/hive/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f create mode 100644 src/test/hive/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb create mode 100644 src/test/hive/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 create mode 100644 src/test/hive/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c create mode 100644 src/test/hive/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 create mode 100644 src/test/hive/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c create mode 100644 src/test/hive/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c create mode 100644 src/test/hive/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea create mode 100644 src/test/hive/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f create mode 100644 src/test/hive/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca create mode 100644 src/test/hive/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f create mode 100644 src/test/hive/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef create mode 100644 src/test/hive/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d create mode 100644 src/test/hive/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b create mode 100644 src/test/hive/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c create mode 100644 src/test/hive/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a create mode 100644 src/test/hive/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa create mode 100644 src/test/hive/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad create mode 100644 src/test/hive/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 src/test/hive/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca create mode 100644 src/test/hive/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c create mode 100644 src/test/hive/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca create mode 100644 src/test/hive/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca create mode 100644 src/test/hive/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e create mode 100644 src/test/hive/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 src/test/hive/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b create mode 100644 src/test/hive/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc create mode 100644 src/test/hive/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e create mode 100644 src/test/hive/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd create mode 100644 src/test/hive/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 create mode 100644 src/test/hive/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 create mode 100644 src/test/hive/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e create mode 100644 src/test/hive/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 create mode 100644 src/test/hive/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a create mode 100644 src/test/hive/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad create mode 100644 src/test/hive/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b create mode 100644 src/test/hive/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 create mode 100644 src/test/hive/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 create mode 100644 src/test/hive/golden/hash-0-a658b129316d666d4b01c1581eed1c1f create mode 100644 src/test/hive/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf create mode 100644 src/test/hive/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 create mode 100644 src/test/hive/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 create mode 100644 src/test/hive/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a create mode 100644 src/test/hive/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 create mode 100644 src/test/hive/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 create mode 100644 src/test/hive/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 create mode 100644 src/test/hive/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 create mode 100644 src/test/hive/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 create mode 100644 src/test/hive/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 create mode 100644 src/test/hive/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 create mode 100644 src/test/hive/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 create mode 100644 src/test/hive/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 create mode 100644 src/test/hive/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d create mode 100644 src/test/hive/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad create mode 100644 src/test/hive/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 create mode 100644 src/test/hive/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f create mode 100644 src/test/hive/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 create mode 100644 src/test/hive/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 create mode 100644 src/test/hive/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e create mode 100644 src/test/hive/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 create mode 100644 src/test/hive/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 create mode 100644 src/test/hive/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf create mode 100644 src/test/hive/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca create mode 100644 src/test/hive/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 create mode 100644 src/test/hive/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 create mode 100644 src/test/hive/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe create mode 100644 src/test/hive/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f create mode 100644 src/test/hive/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 create mode 100644 src/test/hive/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 create mode 100644 src/test/hive/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 create mode 100644 src/test/hive/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 create mode 100644 src/test/hive/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 create mode 100644 src/test/hive/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 create mode 100644 src/test/hive/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 create mode 100644 src/test/hive/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 create mode 100644 src/test/hive/golden/index_creation-37-9334418431eca405f13206bd8db42a1b create mode 100644 src/test/hive/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 create mode 100644 src/test/hive/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d create mode 100644 src/test/hive/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 create mode 100644 src/test/hive/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 create mode 100644 src/test/hive/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 create mode 100644 src/test/hive/golden/index_creation-7-9334418431eca405f13206bd8db42a1b create mode 100644 src/test/hive/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 create mode 100644 src/test/hive/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c create mode 100644 src/test/hive/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 create mode 100644 src/test/hive/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 create mode 100644 src/test/hive/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b create mode 100644 src/test/hive/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 create mode 100644 src/test/hive/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 create mode 100644 src/test/hive/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c create mode 100644 src/test/hive/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 create mode 100644 src/test/hive/golden/index_serde-4-d7547751c37375a9238043bbe250e716 create mode 100644 src/test/hive/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 create mode 100644 src/test/hive/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae create mode 100644 src/test/hive/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 create mode 100644 src/test/hive/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 create mode 100644 src/test/hive/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 create mode 100644 src/test/hive/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e create mode 100644 src/test/hive/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 create mode 100644 src/test/hive/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd create mode 100644 src/test/hive/golden/innerjoin-0-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e create mode 100644 src/test/hive/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 create mode 100644 src/test/hive/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 create mode 100644 src/test/hive/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 create mode 100644 src/test/hive/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a create mode 100644 src/test/hive/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce create mode 100644 src/test/hive/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 create mode 100644 src/test/hive/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 create mode 100644 src/test/hive/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b create mode 100644 src/test/hive/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 create mode 100644 src/test/hive/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 create mode 100644 src/test/hive/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 create mode 100644 src/test/hive/golden/input-1-6558e385bb08991302a72076dd7b7ff5 create mode 100644 src/test/hive/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 create mode 100644 src/test/hive/golden/input0-1-efefd4364cd2790447fb0f908e87501f create mode 100644 src/test/hive/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 create mode 100644 src/test/hive/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 create mode 100644 src/test/hive/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 create mode 100644 src/test/hive/golden/input10-0-acdd72fda64c5463ce29f31020012d42 create mode 100644 src/test/hive/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 create mode 100644 src/test/hive/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 create mode 100644 src/test/hive/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb create mode 100644 src/test/hive/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 create mode 100644 src/test/hive/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 create mode 100644 src/test/hive/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 create mode 100644 src/test/hive/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 create mode 100644 src/test/hive/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e create mode 100644 src/test/hive/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 create mode 100644 src/test/hive/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c create mode 100644 src/test/hive/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 create mode 100644 src/test/hive/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 create mode 100644 src/test/hive/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 create mode 100644 src/test/hive/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf create mode 100644 src/test/hive/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 create mode 100644 src/test/hive/golden/input12-7-16da1c1293626b943343443368679e9c create mode 100644 src/test/hive/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd create mode 100644 src/test/hive/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f create mode 100644 src/test/hive/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 create mode 100644 src/test/hive/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 create mode 100644 src/test/hive/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 create mode 100644 src/test/hive/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf create mode 100644 src/test/hive/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 create mode 100644 src/test/hive/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c create mode 100644 src/test/hive/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd create mode 100644 src/test/hive/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c create mode 100644 src/test/hive/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a create mode 100644 src/test/hive/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 create mode 100644 src/test/hive/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf create mode 100644 src/test/hive/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 create mode 100644 src/test/hive/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input16_cc-0-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 create mode 100644 src/test/hive/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 create mode 100644 src/test/hive/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 create mode 100644 src/test/hive/golden/input17-2-21166e268096f6ec67f4f57ec333e901 create mode 100644 src/test/hive/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a create mode 100644 src/test/hive/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc create mode 100644 src/test/hive/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 create mode 100644 src/test/hive/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 create mode 100644 src/test/hive/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 create mode 100644 src/test/hive/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e create mode 100644 src/test/hive/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 create mode 100644 src/test/hive/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 create mode 100644 src/test/hive/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c create mode 100644 src/test/hive/golden/input2-1-e0efeda558cd0194f4764a5735147b16 create mode 100644 src/test/hive/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd create mode 100644 src/test/hive/golden/input2-3-45b9f786f64afa5e039d1856fe926649 create mode 100644 src/test/hive/golden/input2-4-235f92683416fab031e6e7490487b15b create mode 100644 src/test/hive/golden/input2-5-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/input2-6-4a6b9674c126337f71834f11613f996d create mode 100644 src/test/hive/golden/input2-7-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 create mode 100644 src/test/hive/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 create mode 100644 src/test/hive/golden/input21-0-7f40c525398953892ffd6283f54ba427 create mode 100644 src/test/hive/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 create mode 100644 src/test/hive/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 create mode 100644 src/test/hive/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe create mode 100644 src/test/hive/golden/input22-0-b9cd4041ca510639838a97376747b498 create mode 100644 src/test/hive/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 create mode 100644 src/test/hive/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 create mode 100644 src/test/hive/golden/input22-3-8285c1934441b12f6c016e13cb410e79 create mode 100644 src/test/hive/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf create mode 100644 src/test/hive/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb create mode 100644 src/test/hive/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 create mode 100644 src/test/hive/golden/input24-1-f57485de80ad64f9931bf92514fa1 create mode 100644 src/test/hive/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 create mode 100644 src/test/hive/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 create mode 100644 src/test/hive/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 create mode 100644 src/test/hive/golden/input25-1-f57485de80ad64f9931bf92514fa1 create mode 100644 src/test/hive/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 create mode 100644 src/test/hive/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 create mode 100644 src/test/hive/golden/input25-4-72c74c55dd7d85d194448e9c58488938 create mode 100644 src/test/hive/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 create mode 100644 src/test/hive/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 create mode 100644 src/test/hive/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a create mode 100644 src/test/hive/golden/input28-1-f57485de80ad64f9931bf92514fa1 create mode 100644 src/test/hive/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c create mode 100644 src/test/hive/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 create mode 100644 src/test/hive/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f create mode 100644 src/test/hive/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 create mode 100644 src/test/hive/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff create mode 100644 src/test/hive/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d create mode 100644 src/test/hive/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 create mode 100644 src/test/hive/golden/input3-11-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 create mode 100644 src/test/hive/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a create mode 100644 src/test/hive/golden/input3-14-efee6816e20fe61595a4a2a991071219 create mode 100644 src/test/hive/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 create mode 100644 src/test/hive/golden/input3-3-1c5990b1aed2be48311810dae3019994 create mode 100644 src/test/hive/golden/input3-4-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 create mode 100644 src/test/hive/golden/input3-6-ba8c440158c2519353d02471bfb05694 create mode 100644 src/test/hive/golden/input3-7-1c5990b1aed2be48311810dae3019994 create mode 100644 src/test/hive/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be create mode 100644 src/test/hive/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e create mode 100644 src/test/hive/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c create mode 100644 src/test/hive/golden/input30-1-90c0d1a75de78c405413fd627caea4ab create mode 100644 src/test/hive/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/input30-3-c21dba410fb07a098f93430a9d21df79 create mode 100644 src/test/hive/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 create mode 100644 src/test/hive/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 create mode 100644 src/test/hive/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 create mode 100644 src/test/hive/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f create mode 100644 src/test/hive/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/input31-1-c21dba410fb07a098f93430a9d21df79 create mode 100644 src/test/hive/golden/input31-2-705764f8f7cab9378964af30b83f7fe create mode 100644 src/test/hive/golden/input31-3-50c905261882f7fd8539fdd91e68151f create mode 100644 src/test/hive/golden/input31-4-2f886fa357df9342733551fa1b53f913 create mode 100644 src/test/hive/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/input32-1-c21dba410fb07a098f93430a9d21df79 create mode 100644 src/test/hive/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 create mode 100644 src/test/hive/golden/input32-3-b0070890240c15d647af59f41b77ba3d create mode 100644 src/test/hive/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 create mode 100644 src/test/hive/golden/input32-5-8789d32fc5b784fe2d171566732c573e create mode 100644 src/test/hive/golden/input37-0-86e2e274650fb56651607ea10d356fc0 create mode 100644 src/test/hive/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 create mode 100644 src/test/hive/golden/input39-0-7bd12162381231be9d578797818957a7 create mode 100644 src/test/hive/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 create mode 100644 src/test/hive/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 create mode 100644 src/test/hive/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 create mode 100644 src/test/hive/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 create mode 100644 src/test/hive/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 create mode 100644 src/test/hive/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d create mode 100644 src/test/hive/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 create mode 100644 src/test/hive/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e create mode 100644 src/test/hive/golden/input39-4-53453776bf062f28d371fc7336b7eae2 create mode 100644 src/test/hive/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/input39-6-763ab5853bff619e6525c01e46b2a923 create mode 100644 src/test/hive/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e create mode 100644 src/test/hive/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 create mode 100644 src/test/hive/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 create mode 100644 src/test/hive/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe create mode 100644 src/test/hive/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 create mode 100644 src/test/hive/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 create mode 100644 src/test/hive/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 create mode 100644 src/test/hive/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 create mode 100644 src/test/hive/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 create mode 100644 src/test/hive/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e create mode 100644 src/test/hive/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 create mode 100644 src/test/hive/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 src/test/hive/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 create mode 100644 src/test/hive/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f create mode 100644 src/test/hive/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 create mode 100644 src/test/hive/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 create mode 100644 src/test/hive/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 create mode 100644 src/test/hive/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 create mode 100644 src/test/hive/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d create mode 100644 src/test/hive/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece create mode 100644 src/test/hive/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d create mode 100644 src/test/hive/golden/input3_limit-5-3664b564747487df13a5d109837219b5 create mode 100644 src/test/hive/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 create mode 100644 src/test/hive/golden/input4-0-b9cd4041ca510639838a97376747b498 create mode 100644 src/test/hive/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 create mode 100644 src/test/hive/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 create mode 100644 src/test/hive/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 create mode 100644 src/test/hive/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 create mode 100644 src/test/hive/golden/input40-0-37e74908fd43254141ae3664f13a48d create mode 100644 src/test/hive/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 create mode 100644 src/test/hive/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 create mode 100644 src/test/hive/golden/input40-3-12f634800ffae17942ddd789a01af227 create mode 100644 src/test/hive/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e create mode 100644 src/test/hive/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 create mode 100644 src/test/hive/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 create mode 100644 src/test/hive/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 create mode 100644 src/test/hive/golden/input41-0-763ab5853bff619e6525c01e46b2a923 create mode 100644 src/test/hive/golden/input41-1-8112b3a278e8337dbd6f017ae178658b create mode 100644 src/test/hive/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 create mode 100644 src/test/hive/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee create mode 100644 src/test/hive/golden/input43-0-2baba8070f3585debc14b6bb3c83607a create mode 100644 src/test/hive/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 create mode 100644 src/test/hive/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f create mode 100644 src/test/hive/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca create mode 100644 src/test/hive/golden/input46-3-c185163787977498a4b84f39f983c431 create mode 100644 src/test/hive/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 create mode 100644 src/test/hive/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab create mode 100644 src/test/hive/golden/input49-0-2bd546beeb607da5b925ffbea20a741c create mode 100644 src/test/hive/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d create mode 100644 src/test/hive/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 create mode 100644 src/test/hive/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 create mode 100644 src/test/hive/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 create mode 100644 src/test/hive/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 create mode 100644 src/test/hive/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 create mode 100644 src/test/hive/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d create mode 100644 src/test/hive/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af create mode 100644 src/test/hive/golden/input5-0-659e06570690cceeb3f37e10e855d2ea create mode 100644 src/test/hive/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 create mode 100644 src/test/hive/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad create mode 100644 src/test/hive/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input6-0-659e06570690cceeb3f37e10e855d2ea create mode 100644 src/test/hive/golden/input6-1-21149162906e31964a328b6cb2d5286e create mode 100644 src/test/hive/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c create mode 100644 src/test/hive/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 create mode 100644 src/test/hive/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 create mode 100644 src/test/hive/golden/input7-2-c740098a289f9a4267d26fc10347a0dc create mode 100644 src/test/hive/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa create mode 100644 src/test/hive/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa create mode 100644 src/test/hive/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba create mode 100644 src/test/hive/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 create mode 100644 src/test/hive/golden/input9-1-b30105391cb53915997a84ca3137bd0a create mode 100644 src/test/hive/golden/input9-2-171ea277f05a642699a0f91bacffccd8 create mode 100644 src/test/hive/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e create mode 100644 src/test/hive/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 create mode 100644 src/test/hive/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be create mode 100644 src/test/hive/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 create mode 100644 src/test/hive/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 create mode 100644 src/test/hive/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d create mode 100644 src/test/hive/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc create mode 100644 src/test/hive/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 create mode 100644 src/test/hive/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 create mode 100644 src/test/hive/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d create mode 100644 src/test/hive/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 create mode 100644 src/test/hive/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 create mode 100644 src/test/hive/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c create mode 100644 src/test/hive/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c create mode 100644 src/test/hive/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc create mode 100644 src/test/hive/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 create mode 100644 src/test/hive/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db create mode 100644 src/test/hive/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a create mode 100644 src/test/hive/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 create mode 100644 src/test/hive/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 create mode 100644 src/test/hive/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c create mode 100644 src/test/hive/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a create mode 100644 src/test/hive/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 create mode 100644 src/test/hive/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 create mode 100644 src/test/hive/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 create mode 100644 src/test/hive/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb create mode 100644 src/test/hive/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 create mode 100644 src/test/hive/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 create mode 100644 src/test/hive/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc create mode 100644 src/test/hive/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 create mode 100644 src/test/hive/golden/input_part10-1-35f67c0112026170d015b5f80a254bde create mode 100644 src/test/hive/golden/input_part10-2-40069c199502c2724ac2a2733f964248 create mode 100644 src/test/hive/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 create mode 100644 src/test/hive/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 create mode 100644 src/test/hive/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf create mode 100644 src/test/hive/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde create mode 100644 src/test/hive/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 create mode 100644 src/test/hive/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 create mode 100644 src/test/hive/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 create mode 100644 src/test/hive/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 create mode 100644 src/test/hive/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e create mode 100644 src/test/hive/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd create mode 100644 src/test/hive/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 create mode 100644 src/test/hive/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 create mode 100644 src/test/hive/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 create mode 100644 src/test/hive/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff create mode 100644 src/test/hive/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 create mode 100644 src/test/hive/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef create mode 100644 src/test/hive/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 create mode 100644 src/test/hive/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b create mode 100644 src/test/hive/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c create mode 100644 src/test/hive/golden/input_part5-2-d678533566fba4b922f01284538ca484 create mode 100644 src/test/hive/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 create mode 100644 src/test/hive/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d create mode 100644 src/test/hive/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 create mode 100644 src/test/hive/golden/input_part7-0-14448a12627b8bccf6828032f938f39e create mode 100644 src/test/hive/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 create mode 100644 src/test/hive/golden/input_part8-0-f60652796af548135f9e104b456840e3 create mode 100644 src/test/hive/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 create mode 100644 src/test/hive/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 create mode 100644 src/test/hive/golden/input_part9-1-e60c60afc073367464898b8396e8f643 create mode 100644 src/test/hive/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 create mode 100644 src/test/hive/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e create mode 100644 src/test/hive/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf create mode 100644 src/test/hive/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 create mode 100644 src/test/hive/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 create mode 100644 src/test/hive/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c create mode 100644 src/test/hive/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 create mode 100644 src/test/hive/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 create mode 100644 src/test/hive/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f create mode 100644 src/test/hive/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 create mode 100644 src/test/hive/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 create mode 100644 src/test/hive/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 create mode 100644 src/test/hive/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd create mode 100644 src/test/hive/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac create mode 100644 src/test/hive/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 create mode 100644 src/test/hive/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 create mode 100644 src/test/hive/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 create mode 100644 src/test/hive/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 create mode 100644 src/test/hive/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 create mode 100644 src/test/hive/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 create mode 100644 src/test/hive/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 create mode 100644 src/test/hive/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 create mode 100644 src/test/hive/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 create mode 100644 src/test/hive/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 create mode 100644 src/test/hive/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf create mode 100644 src/test/hive/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c create mode 100644 src/test/hive/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c create mode 100644 src/test/hive/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 create mode 100644 src/test/hive/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 create mode 100644 src/test/hive/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 create mode 100644 src/test/hive/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 create mode 100644 src/test/hive/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 create mode 100644 src/test/hive/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 create mode 100644 src/test/hive/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f create mode 100644 src/test/hive/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 create mode 100644 src/test/hive/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d create mode 100644 src/test/hive/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 create mode 100644 src/test/hive/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 create mode 100644 src/test/hive/golden/inputddl7-13-45059a21c202b4658285738ee62a018a create mode 100644 src/test/hive/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 create mode 100644 src/test/hive/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe create mode 100644 src/test/hive/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 create mode 100644 src/test/hive/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 create mode 100644 src/test/hive/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 create mode 100644 src/test/hive/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 create mode 100644 src/test/hive/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a create mode 100644 src/test/hive/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 create mode 100644 src/test/hive/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 create mode 100644 src/test/hive/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac create mode 100644 src/test/hive/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 create mode 100644 src/test/hive/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 create mode 100644 src/test/hive/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac create mode 100644 src/test/hive/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 create mode 100644 src/test/hive/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 create mode 100644 src/test/hive/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 create mode 100644 src/test/hive/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c create mode 100644 src/test/hive/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d create mode 100644 src/test/hive/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 create mode 100644 src/test/hive/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 create mode 100644 src/test/hive/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 create mode 100644 src/test/hive/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef create mode 100644 src/test/hive/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 create mode 100644 src/test/hive/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef create mode 100644 src/test/hive/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 create mode 100644 src/test/hive/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef create mode 100644 src/test/hive/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 create mode 100644 src/test/hive/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 create mode 100644 src/test/hive/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b create mode 100644 src/test/hive/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 create mode 100644 src/test/hive/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb create mode 100644 src/test/hive/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 create mode 100644 src/test/hive/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 create mode 100644 src/test/hive/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 create mode 100644 src/test/hive/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb create mode 100644 src/test/hive/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 create mode 100644 src/test/hive/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 create mode 100644 src/test/hive/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb create mode 100644 src/test/hive/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d create mode 100644 src/test/hive/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b create mode 100644 src/test/hive/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 create mode 100644 src/test/hive/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 create mode 100644 src/test/hive/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 create mode 100644 src/test/hive/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa create mode 100644 src/test/hive/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 create mode 100644 src/test/hive/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 create mode 100644 src/test/hive/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 create mode 100644 src/test/hive/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 create mode 100644 src/test/hive/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 create mode 100644 src/test/hive/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 create mode 100644 src/test/hive/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 create mode 100644 src/test/hive/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 create mode 100644 src/test/hive/golden/insert_into2-8-452111285dda40205ee587de8e972896 create mode 100644 src/test/hive/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 create mode 100644 src/test/hive/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 create mode 100644 src/test/hive/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 create mode 100644 src/test/hive/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 create mode 100644 src/test/hive/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 create mode 100644 src/test/hive/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 create mode 100644 src/test/hive/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 create mode 100644 src/test/hive/golden/insert_into3-2-e51c25bae2408422a56826a263479468 create mode 100644 src/test/hive/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d create mode 100644 src/test/hive/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd create mode 100644 src/test/hive/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 create mode 100644 src/test/hive/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 create mode 100644 src/test/hive/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 create mode 100644 src/test/hive/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 create mode 100644 src/test/hive/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 create mode 100644 src/test/hive/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 create mode 100644 src/test/hive/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 create mode 100644 src/test/hive/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 create mode 100644 src/test/hive/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa create mode 100644 src/test/hive/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 create mode 100644 src/test/hive/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b create mode 100644 src/test/hive/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 create mode 100644 src/test/hive/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 create mode 100644 src/test/hive/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 create mode 100644 src/test/hive/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b create mode 100644 src/test/hive/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a create mode 100644 src/test/hive/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 create mode 100644 src/test/hive/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 create mode 100644 src/test/hive/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 create mode 100644 src/test/hive/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 create mode 100644 src/test/hive/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 create mode 100644 src/test/hive/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 create mode 100644 src/test/hive/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c create mode 100644 src/test/hive/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 create mode 100644 src/test/hive/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d create mode 100644 src/test/hive/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 create mode 100644 src/test/hive/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 create mode 100644 src/test/hive/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 create mode 100644 src/test/hive/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 create mode 100644 src/test/hive/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 create mode 100644 src/test/hive/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c create mode 100644 src/test/hive/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 create mode 100644 src/test/hive/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 create mode 100644 src/test/hive/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 create mode 100644 src/test/hive/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 create mode 100644 src/test/hive/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f create mode 100644 src/test/hive/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 create mode 100644 src/test/hive/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 create mode 100644 src/test/hive/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 create mode 100644 src/test/hive/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba create mode 100644 src/test/hive/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 create mode 100644 src/test/hive/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 create mode 100644 src/test/hive/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 create mode 100644 src/test/hive/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d create mode 100644 src/test/hive/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 create mode 100644 src/test/hive/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d create mode 100644 src/test/hive/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f create mode 100644 src/test/hive/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 create mode 100644 src/test/hive/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 create mode 100644 src/test/hive/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 create mode 100644 src/test/hive/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 create mode 100644 src/test/hive/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab create mode 100644 src/test/hive/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 create mode 100644 src/test/hive/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a create mode 100644 src/test/hive/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b create mode 100644 src/test/hive/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 create mode 100644 src/test/hive/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 create mode 100644 src/test/hive/golden/join0-1-83f5e512d6058a47f92b5218781f5070 create mode 100644 src/test/hive/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 create mode 100644 src/test/hive/golden/join1-0-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e create mode 100644 src/test/hive/golden/join1-2-7b07671864bbfdc627ee794932e32b1e create mode 100644 src/test/hive/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 create mode 100644 src/test/hive/golden/join1-4-35c7611180562dcd9dab834f41654095 create mode 100644 src/test/hive/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 create mode 100644 src/test/hive/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d create mode 100644 src/test/hive/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 create mode 100644 src/test/hive/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 create mode 100644 src/test/hive/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 create mode 100644 src/test/hive/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f create mode 100644 src/test/hive/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 create mode 100644 src/test/hive/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 create mode 100644 src/test/hive/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 create mode 100644 src/test/hive/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e create mode 100644 src/test/hive/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 create mode 100644 src/test/hive/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 create mode 100644 src/test/hive/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 create mode 100644 src/test/hive/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 create mode 100644 src/test/hive/golden/join14-6-de39302191b63d7aa8f92885b089fe2 create mode 100644 src/test/hive/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 create mode 100644 src/test/hive/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 create mode 100644 src/test/hive/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f create mode 100644 src/test/hive/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 create mode 100644 src/test/hive/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 create mode 100644 src/test/hive/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 create mode 100644 src/test/hive/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 create mode 100644 src/test/hive/golden/join15-0-930db2823512f7f8f34cac104f2162e0 create mode 100644 src/test/hive/golden/join15-1-81d76d3bf59889b07b413b6f88772667 create mode 100644 src/test/hive/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c create mode 100644 src/test/hive/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 create mode 100644 src/test/hive/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b create mode 100644 src/test/hive/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 create mode 100644 src/test/hive/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/join18-0-269843197b598569f4b50cbe911960a5 create mode 100644 src/test/hive/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 create mode 100644 src/test/hive/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 create mode 100644 src/test/hive/golden/join19-1-7e7d1f07c34dd4be5425264196201766 create mode 100644 src/test/hive/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca create mode 100644 src/test/hive/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c create mode 100644 src/test/hive/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a create mode 100644 src/test/hive/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 create mode 100644 src/test/hive/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e create mode 100644 src/test/hive/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 create mode 100644 src/test/hive/golden/join20-2-222655137ca9242f063c7717acbe4c65 create mode 100644 src/test/hive/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 create mode 100644 src/test/hive/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 create mode 100644 src/test/hive/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f create mode 100644 src/test/hive/golden/join22-0-97081b5f73052a612fb391a57ad9b7b create mode 100644 src/test/hive/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d create mode 100644 src/test/hive/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 create mode 100644 src/test/hive/golden/join24-0-721dfa03bfea05e55506c571b6c3585b create mode 100644 src/test/hive/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d create mode 100644 src/test/hive/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 create mode 100644 src/test/hive/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 create mode 100644 src/test/hive/golden/join25-1-360b3676605ccb3d89cf555342db87af create mode 100644 src/test/hive/golden/join25-2-ef930bed933341636c71622e6f89e9e9 create mode 100644 src/test/hive/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 create mode 100644 src/test/hive/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join26-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 src/test/hive/golden/join26-1-10fbe6c99e93cfeafcd454261744728e create mode 100644 src/test/hive/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d create mode 100644 src/test/hive/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join27-0-360b3676605ccb3d89cf555342db87af create mode 100644 src/test/hive/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 create mode 100644 src/test/hive/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 create mode 100644 src/test/hive/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 create mode 100644 src/test/hive/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 create mode 100644 src/test/hive/golden/join28-1-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/join28-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 create mode 100644 src/test/hive/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 create mode 100644 src/test/hive/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join29-0-e78402f7585b17c76c32790571d749eb create mode 100644 src/test/hive/golden/join29-1-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/join29-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/join29-4-b0524de7985cdb7b57fd995721654856 create mode 100644 src/test/hive/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 create mode 100644 src/test/hive/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 create mode 100644 src/test/hive/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 create mode 100644 src/test/hive/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 create mode 100644 src/test/hive/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d create mode 100644 src/test/hive/golden/join30-2-194edec0592cf441617ca7caf9756baa create mode 100644 src/test/hive/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join31-0-619db2f9dd69aa236aa804cced726c9a create mode 100644 src/test/hive/golden/join31-1-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/join31-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 create mode 100644 src/test/hive/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 create mode 100644 src/test/hive/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join32-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 src/test/hive/golden/join32-1-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/join32-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb create mode 100644 src/test/hive/golden/join32-5-425c5c35b23c42254dabec49aa6613bf create mode 100644 src/test/hive/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 src/test/hive/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 create mode 100644 src/test/hive/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a create mode 100644 src/test/hive/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca create mode 100644 src/test/hive/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 create mode 100644 src/test/hive/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 create mode 100644 src/test/hive/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a create mode 100644 src/test/hive/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e create mode 100644 src/test/hive/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 create mode 100644 src/test/hive/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 create mode 100644 src/test/hive/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d create mode 100644 src/test/hive/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 create mode 100644 src/test/hive/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 create mode 100644 src/test/hive/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 create mode 100644 src/test/hive/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 create mode 100644 src/test/hive/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 create mode 100644 src/test/hive/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb create mode 100644 src/test/hive/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf create mode 100644 src/test/hive/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a create mode 100644 src/test/hive/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b create mode 100644 src/test/hive/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 create mode 100644 src/test/hive/golden/join33-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 src/test/hive/golden/join33-1-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/join33-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb create mode 100644 src/test/hive/golden/join33-5-425c5c35b23c42254dabec49aa6613bf create mode 100644 src/test/hive/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join34-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 src/test/hive/golden/join34-1-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/join34-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/join34-4-51cc5d253befd784a9d466c0b402f47a create mode 100644 src/test/hive/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 create mode 100644 src/test/hive/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 create mode 100644 src/test/hive/golden/join35-1-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/join35-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/join35-4-99cb704ccc2813400908a62571891dc7 create mode 100644 src/test/hive/golden/join35-5-60956f3307651237f52ffbe41c827c1c create mode 100644 src/test/hive/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 create mode 100644 src/test/hive/golden/join36-1-3aba153d62042206c0f001ea74980da create mode 100644 src/test/hive/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 create mode 100644 src/test/hive/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 create mode 100644 src/test/hive/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 create mode 100644 src/test/hive/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 create mode 100644 src/test/hive/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 create mode 100644 src/test/hive/golden/join36-7-171ede21302050a33468d95e329a38a3 create mode 100644 src/test/hive/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 create mode 100644 src/test/hive/golden/join37-1-360b3676605ccb3d89cf555342db87af create mode 100644 src/test/hive/golden/join37-2-2310f2b8726137811543e04db98dd1db create mode 100644 src/test/hive/golden/join37-3-c4b3a85965108ad7013ac3931598af09 create mode 100644 src/test/hive/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join38-0-42e6ad674a5142d05258a2d29578e046 create mode 100644 src/test/hive/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f create mode 100644 src/test/hive/golden/join38-2-eacdb3417c4a563982c488812d654c9 create mode 100644 src/test/hive/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 create mode 100644 src/test/hive/golden/join38-4-53d219706847e890de1dcd369563ebef create mode 100644 src/test/hive/golden/join39-0-54e9f7924abb40d1c601de908de6f25b create mode 100644 src/test/hive/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 create mode 100644 src/test/hive/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 create mode 100644 src/test/hive/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join4-0-531e91e11b3891627c2675935fda14cd create mode 100644 src/test/hive/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 create mode 100644 src/test/hive/golden/join4-2-dc967001beb776f3a859e9360823c361 create mode 100644 src/test/hive/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac create mode 100644 src/test/hive/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 create mode 100644 src/test/hive/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa create mode 100644 src/test/hive/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 create mode 100644 src/test/hive/golden/join40-12-aaddbef9069aba3ebeb310be74671cda create mode 100644 src/test/hive/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 create mode 100644 src/test/hive/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 create mode 100644 src/test/hive/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 create mode 100644 src/test/hive/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e create mode 100644 src/test/hive/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 create mode 100644 src/test/hive/golden/join40-7-222655137ca9242f063c7717acbe4c65 create mode 100644 src/test/hive/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 create mode 100644 src/test/hive/golden/join40-9-6b7404af014eeb4de196577f14ef18cc create mode 100644 src/test/hive/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 create mode 100644 src/test/hive/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 create mode 100644 src/test/hive/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a create mode 100644 src/test/hive/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 create mode 100644 src/test/hive/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e create mode 100644 src/test/hive/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 create mode 100644 src/test/hive/golden/join5-0-531e91e11b3891627c2675935fda14cd create mode 100644 src/test/hive/golden/join5-1-76feff1282895f38b673f52acfd7429e create mode 100644 src/test/hive/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 create mode 100644 src/test/hive/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/join6-0-531e91e11b3891627c2675935fda14cd create mode 100644 src/test/hive/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 create mode 100644 src/test/hive/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 create mode 100644 src/test/hive/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 create mode 100644 src/test/hive/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 create mode 100644 src/test/hive/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 create mode 100644 src/test/hive/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/join8-0-531e91e11b3891627c2675935fda14cd create mode 100644 src/test/hive/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 create mode 100644 src/test/hive/golden/join8-2-f1196bca86a749375da35f134206a8ca create mode 100644 src/test/hive/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a create mode 100644 src/test/hive/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 create mode 100644 src/test/hive/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 create mode 100644 src/test/hive/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a create mode 100644 src/test/hive/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e create mode 100644 src/test/hive/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 src/test/hive/golden/join_1to1-12-6081eb976b4aef2132418510756a385b create mode 100644 src/test/hive/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c create mode 100644 src/test/hive/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 src/test/hive/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 create mode 100644 src/test/hive/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 create mode 100644 src/test/hive/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 src/test/hive/golden/join_1to1-18-6081eb976b4aef2132418510756a385b create mode 100644 src/test/hive/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c create mode 100644 src/test/hive/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 create mode 100644 src/test/hive/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 src/test/hive/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 create mode 100644 src/test/hive/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 create mode 100644 src/test/hive/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 src/test/hive/golden/join_1to1-24-6081eb976b4aef2132418510756a385b create mode 100644 src/test/hive/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c create mode 100644 src/test/hive/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 src/test/hive/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 create mode 100644 src/test/hive/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e create mode 100644 src/test/hive/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 src/test/hive/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 create mode 100644 src/test/hive/golden/join_1to1-30-6081eb976b4aef2132418510756a385b create mode 100644 src/test/hive/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c create mode 100644 src/test/hive/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 src/test/hive/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 create mode 100644 src/test/hive/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 create mode 100644 src/test/hive/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 src/test/hive/golden/join_1to1-36-6081eb976b4aef2132418510756a385b create mode 100644 src/test/hive/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c create mode 100644 src/test/hive/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 src/test/hive/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 create mode 100644 src/test/hive/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 create mode 100644 src/test/hive/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 src/test/hive/golden/join_1to1-6-6081eb976b4aef2132418510756a385b create mode 100644 src/test/hive/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c create mode 100644 src/test/hive/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 src/test/hive/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 create mode 100644 src/test/hive/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 create mode 100644 src/test/hive/golden/join_array-1-2c086fcf118fd2538bfa00724209091e create mode 100644 src/test/hive/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de create mode 100644 src/test/hive/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 create mode 100644 src/test/hive/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 create mode 100644 src/test/hive/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a create mode 100644 src/test/hive/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c create mode 100644 src/test/hive/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 create mode 100644 src/test/hive/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 create mode 100644 src/test/hive/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d create mode 100644 src/test/hive/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc create mode 100644 src/test/hive/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd create mode 100644 src/test/hive/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad create mode 100644 src/test/hive/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 create mode 100644 src/test/hive/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e create mode 100644 src/test/hive/golden/join_empty-3-3b2231f25886c152f222856256ac3473 create mode 100644 src/test/hive/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a create mode 100644 src/test/hive/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a create mode 100644 src/test/hive/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 create mode 100644 src/test/hive/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 create mode 100644 src/test/hive/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 create mode 100644 src/test/hive/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 create mode 100644 src/test/hive/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe create mode 100644 src/test/hive/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b create mode 100644 src/test/hive/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 create mode 100644 src/test/hive/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b create mode 100644 src/test/hive/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 create mode 100644 src/test/hive/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb create mode 100644 src/test/hive/golden/join_filters-106-870999978978b22f21997899f1e652b8 create mode 100644 src/test/hive/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 create mode 100644 src/test/hive/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e create mode 100644 src/test/hive/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 create mode 100644 src/test/hive/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 create mode 100644 src/test/hive/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba create mode 100644 src/test/hive/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 create mode 100644 src/test/hive/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 create mode 100644 src/test/hive/golden/join_filters-113-48c4978768872751832149d72cdf0ba create mode 100644 src/test/hive/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 create mode 100644 src/test/hive/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 create mode 100644 src/test/hive/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 create mode 100644 src/test/hive/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 create mode 100644 src/test/hive/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e create mode 100644 src/test/hive/golden/join_filters-119-7688784396db55ff148292a9755c918a create mode 100644 src/test/hive/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e create mode 100644 src/test/hive/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 create mode 100644 src/test/hive/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada create mode 100644 src/test/hive/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 create mode 100644 src/test/hive/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 create mode 100644 src/test/hive/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf create mode 100644 src/test/hive/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 create mode 100644 src/test/hive/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 create mode 100644 src/test/hive/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 create mode 100644 src/test/hive/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 create mode 100644 src/test/hive/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f create mode 100644 src/test/hive/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 create mode 100644 src/test/hive/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf create mode 100644 src/test/hive/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e create mode 100644 src/test/hive/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e create mode 100644 src/test/hive/golden/join_filters-23-a800b885358695758afdb719cdefa94f create mode 100644 src/test/hive/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 create mode 100644 src/test/hive/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 create mode 100644 src/test/hive/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca create mode 100644 src/test/hive/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 create mode 100644 src/test/hive/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 create mode 100644 src/test/hive/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 create mode 100644 src/test/hive/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 create mode 100644 src/test/hive/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf create mode 100644 src/test/hive/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f create mode 100644 src/test/hive/golden/join_filters-32-5978cd7936c296493a16a31b926043ab create mode 100644 src/test/hive/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 create mode 100644 src/test/hive/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 create mode 100644 src/test/hive/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe create mode 100644 src/test/hive/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b create mode 100644 src/test/hive/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 create mode 100644 src/test/hive/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b create mode 100644 src/test/hive/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 create mode 100644 src/test/hive/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 create mode 100644 src/test/hive/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb create mode 100644 src/test/hive/golden/join_filters-41-870999978978b22f21997899f1e652b8 create mode 100644 src/test/hive/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 create mode 100644 src/test/hive/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e create mode 100644 src/test/hive/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e create mode 100644 src/test/hive/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 create mode 100644 src/test/hive/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b create mode 100644 src/test/hive/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 create mode 100644 src/test/hive/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c create mode 100644 src/test/hive/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 create mode 100644 src/test/hive/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e create mode 100644 src/test/hive/golden/join_filters-50-3e6612a89e9124592e790594775054b1 create mode 100644 src/test/hive/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac create mode 100644 src/test/hive/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 create mode 100644 src/test/hive/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 create mode 100644 src/test/hive/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba create mode 100644 src/test/hive/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 create mode 100644 src/test/hive/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 create mode 100644 src/test/hive/golden/join_filters-57-48c4978768872751832149d72cdf0ba create mode 100644 src/test/hive/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 create mode 100644 src/test/hive/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 create mode 100644 src/test/hive/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 create mode 100644 src/test/hive/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 create mode 100644 src/test/hive/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 create mode 100644 src/test/hive/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e create mode 100644 src/test/hive/golden/join_filters-63-7688784396db55ff148292a9755c918a create mode 100644 src/test/hive/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 create mode 100644 src/test/hive/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada create mode 100644 src/test/hive/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 create mode 100644 src/test/hive/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 create mode 100644 src/test/hive/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 create mode 100644 src/test/hive/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 create mode 100644 src/test/hive/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d create mode 100644 src/test/hive/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e create mode 100644 src/test/hive/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 create mode 100644 src/test/hive/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d create mode 100644 src/test/hive/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 create mode 100644 src/test/hive/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa create mode 100644 src/test/hive/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 create mode 100644 src/test/hive/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 create mode 100644 src/test/hive/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e create mode 100644 src/test/hive/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 create mode 100644 src/test/hive/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf create mode 100644 src/test/hive/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 create mode 100644 src/test/hive/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 create mode 100644 src/test/hive/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 create mode 100644 src/test/hive/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 create mode 100644 src/test/hive/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 create mode 100644 src/test/hive/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f create mode 100644 src/test/hive/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf create mode 100644 src/test/hive/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e create mode 100644 src/test/hive/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e create mode 100644 src/test/hive/golden/join_filters-88-a800b885358695758afdb719cdefa94f create mode 100644 src/test/hive/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 create mode 100644 src/test/hive/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa create mode 100644 src/test/hive/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 create mode 100644 src/test/hive/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca create mode 100644 src/test/hive/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 create mode 100644 src/test/hive/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 create mode 100644 src/test/hive/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 create mode 100644 src/test/hive/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf create mode 100644 src/test/hive/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f create mode 100644 src/test/hive/golden/join_filters-97-5978cd7936c296493a16a31b926043ab create mode 100644 src/test/hive/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 create mode 100644 src/test/hive/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 create mode 100644 src/test/hive/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce create mode 100644 src/test/hive/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 create mode 100644 src/test/hive/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 create mode 100644 src/test/hive/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 create mode 100644 src/test/hive/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e create mode 100644 src/test/hive/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 create mode 100644 src/test/hive/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 create mode 100644 src/test/hive/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 create mode 100644 src/test/hive/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 create mode 100644 src/test/hive/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 create mode 100644 src/test/hive/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 src/test/hive/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 create mode 100644 src/test/hive/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e create mode 100644 src/test/hive/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 src/test/hive/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 create mode 100644 src/test/hive/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 create mode 100644 src/test/hive/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a create mode 100644 src/test/hive/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db create mode 100644 src/test/hive/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed create mode 100644 src/test/hive/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a create mode 100644 src/test/hive/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 create mode 100644 src/test/hive/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 create mode 100644 src/test/hive/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b create mode 100644 src/test/hive/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 create mode 100644 src/test/hive/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 create mode 100644 src/test/hive/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d create mode 100644 src/test/hive/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 create mode 100644 src/test/hive/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 create mode 100644 src/test/hive/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 create mode 100644 src/test/hive/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f create mode 100644 src/test/hive/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 create mode 100644 src/test/hive/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c create mode 100644 src/test/hive/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 create mode 100644 src/test/hive/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 create mode 100644 src/test/hive/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e create mode 100644 src/test/hive/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 create mode 100644 src/test/hive/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 create mode 100644 src/test/hive/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 create mode 100644 src/test/hive/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 create mode 100644 src/test/hive/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 create mode 100644 src/test/hive/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 create mode 100644 src/test/hive/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb create mode 100644 src/test/hive/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb create mode 100644 src/test/hive/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 create mode 100644 src/test/hive/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 create mode 100644 src/test/hive/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 create mode 100644 src/test/hive/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 create mode 100644 src/test/hive/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 create mode 100644 src/test/hive/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 create mode 100644 src/test/hive/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 create mode 100644 src/test/hive/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 create mode 100644 src/test/hive/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc create mode 100644 src/test/hive/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e create mode 100644 src/test/hive/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 create mode 100644 src/test/hive/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 create mode 100644 src/test/hive/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b create mode 100644 src/test/hive/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 create mode 100644 src/test/hive/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c create mode 100644 src/test/hive/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 create mode 100644 src/test/hive/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 create mode 100644 src/test/hive/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 create mode 100644 src/test/hive/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 create mode 100644 src/test/hive/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 create mode 100644 src/test/hive/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 create mode 100644 src/test/hive/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 create mode 100644 src/test/hive/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c create mode 100644 src/test/hive/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 create mode 100644 src/test/hive/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f create mode 100644 src/test/hive/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 create mode 100644 src/test/hive/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 create mode 100644 src/test/hive/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 create mode 100644 src/test/hive/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 create mode 100644 src/test/hive/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 create mode 100644 src/test/hive/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 create mode 100644 src/test/hive/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a create mode 100644 src/test/hive/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 create mode 100644 src/test/hive/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 create mode 100644 src/test/hive/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 create mode 100644 src/test/hive/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 create mode 100644 src/test/hive/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b create mode 100644 src/test/hive/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 create mode 100644 src/test/hive/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e create mode 100644 src/test/hive/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 create mode 100644 src/test/hive/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee create mode 100644 src/test/hive/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 create mode 100644 src/test/hive/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 create mode 100644 src/test/hive/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 create mode 100644 src/test/hive/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 create mode 100644 src/test/hive/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d create mode 100644 src/test/hive/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 create mode 100644 src/test/hive/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e create mode 100644 src/test/hive/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 create mode 100644 src/test/hive/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 create mode 100644 src/test/hive/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec create mode 100644 src/test/hive/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 create mode 100644 src/test/hive/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 create mode 100644 src/test/hive/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 create mode 100644 src/test/hive/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 create mode 100644 src/test/hive/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 create mode 100644 src/test/hive/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 create mode 100644 src/test/hive/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e create mode 100644 src/test/hive/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 create mode 100644 src/test/hive/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 create mode 100644 src/test/hive/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec create mode 100644 src/test/hive/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 create mode 100644 src/test/hive/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 create mode 100644 src/test/hive/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf create mode 100644 src/test/hive/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a create mode 100644 src/test/hive/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f create mode 100644 src/test/hive/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 create mode 100644 src/test/hive/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 create mode 100644 src/test/hive/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c create mode 100644 src/test/hive/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 create mode 100644 src/test/hive/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 create mode 100644 src/test/hive/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 create mode 100644 src/test/hive/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba create mode 100644 src/test/hive/golden/join_star-0-6001f83f7780835737e3262a34c2b832 create mode 100644 src/test/hive/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d create mode 100644 src/test/hive/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e create mode 100644 src/test/hive/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae create mode 100644 src/test/hive/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 create mode 100644 src/test/hive/golden/join_star-13-342b7249c9ce1484869169b1b33191cb create mode 100644 src/test/hive/golden/join_star-14-75513308d30b781fd2e06d81963c4363 create mode 100644 src/test/hive/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 create mode 100644 src/test/hive/golden/join_star-16-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 create mode 100644 src/test/hive/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa create mode 100644 src/test/hive/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 create mode 100644 src/test/hive/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 create mode 100644 src/test/hive/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 create mode 100644 src/test/hive/golden/join_star-22-429cfd140488242d569aee6594aae76b create mode 100644 src/test/hive/golden/join_star-23-bada259628918dbfb4837bcb58258530 create mode 100644 src/test/hive/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 create mode 100644 src/test/hive/golden/join_star-25-be5af28b6bccf529a31ef68619de699e create mode 100644 src/test/hive/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 create mode 100644 src/test/hive/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 create mode 100644 src/test/hive/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 create mode 100644 src/test/hive/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da create mode 100644 src/test/hive/golden/join_star-4-aaba115af7543b214820a48a694f2463 create mode 100644 src/test/hive/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da create mode 100644 src/test/hive/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a create mode 100644 src/test/hive/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a create mode 100644 src/test/hive/golden/join_star-8-a957982d8981ff0a35397ca449297024 create mode 100644 src/test/hive/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 create mode 100644 src/test/hive/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 create mode 100644 src/test/hive/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db create mode 100644 src/test/hive/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b create mode 100644 src/test/hive/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 create mode 100644 src/test/hive/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 create mode 100644 src/test/hive/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 create mode 100644 src/test/hive/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 create mode 100644 src/test/hive/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b create mode 100644 src/test/hive/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 create mode 100644 src/test/hive/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 create mode 100644 src/test/hive/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef create mode 100644 src/test/hive/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d create mode 100644 src/test/hive/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 create mode 100644 src/test/hive/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 create mode 100644 src/test/hive/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee create mode 100644 src/test/hive/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb create mode 100644 src/test/hive/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 create mode 100644 src/test/hive/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 create mode 100644 src/test/hive/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 create mode 100644 src/test/hive/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e create mode 100644 src/test/hive/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b create mode 100644 src/test/hive/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a create mode 100644 src/test/hive/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b create mode 100644 src/test/hive/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 create mode 100644 src/test/hive/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e create mode 100644 src/test/hive/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 create mode 100644 src/test/hive/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 create mode 100644 src/test/hive/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba create mode 100644 src/test/hive/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 create mode 100644 src/test/hive/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e create mode 100644 src/test/hive/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 create mode 100644 src/test/hive/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab create mode 100644 src/test/hive/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e create mode 100644 src/test/hive/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 create mode 100644 src/test/hive/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 create mode 100644 src/test/hive/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 create mode 100644 src/test/hive/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 create mode 100644 src/test/hive/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e create mode 100644 src/test/hive/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 create mode 100644 src/test/hive/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae create mode 100644 src/test/hive/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 create mode 100644 src/test/hive/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 create mode 100644 src/test/hive/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 create mode 100644 src/test/hive/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 create mode 100644 src/test/hive/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 create mode 100644 src/test/hive/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 create mode 100644 src/test/hive/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 create mode 100644 src/test/hive/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 create mode 100644 src/test/hive/golden/lateral_view_cp-4-507876b610812506343e9d251645170b create mode 100644 src/test/hive/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 create mode 100644 src/test/hive/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 create mode 100644 src/test/hive/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a create mode 100644 src/test/hive/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 create mode 100644 src/test/hive/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe create mode 100644 src/test/hive/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 create mode 100644 src/test/hive/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 create mode 100644 src/test/hive/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa create mode 100644 src/test/hive/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 create mode 100644 src/test/hive/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d create mode 100644 src/test/hive/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e create mode 100644 src/test/hive/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 create mode 100644 src/test/hive/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 create mode 100644 src/test/hive/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 create mode 100644 src/test/hive/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f create mode 100644 src/test/hive/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 create mode 100644 src/test/hive/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 create mode 100644 src/test/hive/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f create mode 100644 src/test/hive/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa create mode 100644 src/test/hive/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b create mode 100644 src/test/hive/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 create mode 100644 src/test/hive/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 create mode 100644 src/test/hive/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb create mode 100644 src/test/hive/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 create mode 100644 src/test/hive/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 create mode 100644 src/test/hive/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f create mode 100644 src/test/hive/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 create mode 100644 src/test/hive/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 create mode 100644 src/test/hive/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 create mode 100644 src/test/hive/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 create mode 100644 src/test/hive/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 create mode 100644 src/test/hive/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 create mode 100644 src/test/hive/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 create mode 100644 src/test/hive/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 create mode 100644 src/test/hive/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb create mode 100644 src/test/hive/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 create mode 100644 src/test/hive/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e create mode 100644 src/test/hive/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a create mode 100644 src/test/hive/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 create mode 100644 src/test/hive/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a create mode 100644 src/test/hive/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c create mode 100644 src/test/hive/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 create mode 100644 src/test/hive/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f create mode 100644 src/test/hive/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 create mode 100644 src/test/hive/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 create mode 100644 src/test/hive/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 create mode 100644 src/test/hive/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 create mode 100644 src/test/hive/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a create mode 100644 src/test/hive/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 create mode 100644 src/test/hive/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 create mode 100644 src/test/hive/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c create mode 100644 src/test/hive/golden/literal_double-0-10ef1098e35d900983be3814de8f974f create mode 100644 src/test/hive/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 create mode 100644 src/test/hive/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 create mode 100644 src/test/hive/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 create mode 100644 src/test/hive/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 create mode 100644 src/test/hive/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d create mode 100644 src/test/hive/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d create mode 100644 src/test/hive/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 create mode 100644 src/test/hive/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 create mode 100644 src/test/hive/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 create mode 100644 src/test/hive/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 create mode 100644 src/test/hive/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f create mode 100644 src/test/hive/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 create mode 100644 src/test/hive/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 create mode 100644 src/test/hive/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 create mode 100644 src/test/hive/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 create mode 100644 src/test/hive/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e create mode 100644 src/test/hive/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 create mode 100644 src/test/hive/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 create mode 100644 src/test/hive/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 create mode 100644 src/test/hive/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 create mode 100644 src/test/hive/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 create mode 100644 src/test/hive/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 create mode 100644 src/test/hive/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d create mode 100644 src/test/hive/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 create mode 100644 src/test/hive/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb create mode 100644 src/test/hive/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a create mode 100644 src/test/hive/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 create mode 100644 src/test/hive/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b create mode 100644 src/test/hive/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 create mode 100644 src/test/hive/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 create mode 100644 src/test/hive/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 create mode 100644 src/test/hive/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 create mode 100644 src/test/hive/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 create mode 100644 src/test/hive/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d create mode 100644 src/test/hive/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 create mode 100644 src/test/hive/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 create mode 100644 src/test/hive/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d create mode 100644 src/test/hive/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a create mode 100644 src/test/hive/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e create mode 100644 src/test/hive/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd create mode 100644 src/test/hive/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 create mode 100644 src/test/hive/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 create mode 100644 src/test/hive/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e create mode 100644 src/test/hive/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 create mode 100644 src/test/hive/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e create mode 100644 src/test/hive/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd create mode 100644 src/test/hive/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 create mode 100644 src/test/hive/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 create mode 100644 src/test/hive/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e create mode 100644 src/test/hive/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c create mode 100644 src/test/hive/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 create mode 100644 src/test/hive/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a create mode 100644 src/test/hive/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 create mode 100644 src/test/hive/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f create mode 100644 src/test/hive/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 create mode 100644 src/test/hive/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b create mode 100644 src/test/hive/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 create mode 100644 src/test/hive/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 create mode 100644 src/test/hive/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf create mode 100644 src/test/hive/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 create mode 100644 src/test/hive/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f create mode 100644 src/test/hive/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 create mode 100644 src/test/hive/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 create mode 100644 src/test/hive/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 create mode 100644 src/test/hive/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 create mode 100644 src/test/hive/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 create mode 100644 src/test/hive/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 create mode 100644 src/test/hive/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 create mode 100644 src/test/hive/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd create mode 100644 src/test/hive/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 create mode 100644 src/test/hive/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 create mode 100644 src/test/hive/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 create mode 100644 src/test/hive/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 create mode 100644 src/test/hive/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 create mode 100644 src/test/hive/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b create mode 100644 src/test/hive/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 create mode 100644 src/test/hive/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d create mode 100644 src/test/hive/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 create mode 100644 src/test/hive/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 create mode 100644 src/test/hive/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df create mode 100644 src/test/hive/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 create mode 100644 src/test/hive/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 create mode 100644 src/test/hive/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 create mode 100644 src/test/hive/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d create mode 100644 src/test/hive/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 create mode 100644 src/test/hive/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 create mode 100644 src/test/hive/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f create mode 100644 src/test/hive/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 create mode 100644 src/test/hive/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a create mode 100644 src/test/hive/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 create mode 100644 src/test/hive/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 create mode 100644 src/test/hive/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 create mode 100644 src/test/hive/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c create mode 100644 src/test/hive/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a create mode 100644 src/test/hive/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 create mode 100644 src/test/hive/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 create mode 100644 src/test/hive/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b create mode 100644 src/test/hive/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db create mode 100644 src/test/hive/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f create mode 100644 src/test/hive/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a create mode 100644 src/test/hive/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 create mode 100644 src/test/hive/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b create mode 100644 src/test/hive/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 create mode 100644 src/test/hive/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb create mode 100644 src/test/hive/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 create mode 100644 src/test/hive/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 create mode 100644 src/test/hive/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e create mode 100644 src/test/hive/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 create mode 100644 src/test/hive/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 create mode 100644 src/test/hive/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f create mode 100644 src/test/hive/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 create mode 100644 src/test/hive/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 create mode 100644 src/test/hive/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 create mode 100644 src/test/hive/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc create mode 100644 src/test/hive/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 create mode 100644 src/test/hive/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 create mode 100644 src/test/hive/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 create mode 100644 src/test/hive/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e create mode 100644 src/test/hive/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 create mode 100644 src/test/hive/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 create mode 100644 src/test/hive/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e create mode 100644 src/test/hive/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 create mode 100644 src/test/hive/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 create mode 100644 src/test/hive/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c create mode 100644 src/test/hive/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 create mode 100644 src/test/hive/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 create mode 100644 src/test/hive/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 create mode 100644 src/test/hive/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc create mode 100644 src/test/hive/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 create mode 100644 src/test/hive/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 create mode 100644 src/test/hive/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 create mode 100644 src/test/hive/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 create mode 100644 src/test/hive/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 create mode 100644 src/test/hive/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 create mode 100644 src/test/hive/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e create mode 100644 src/test/hive/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 create mode 100644 src/test/hive/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 create mode 100644 src/test/hive/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 create mode 100644 src/test/hive/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 create mode 100644 src/test/hive/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 create mode 100644 src/test/hive/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 create mode 100644 src/test/hive/golden/lock2-3-27ad2962fed131f51ba802596ba37278 create mode 100644 src/test/hive/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a create mode 100644 src/test/hive/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca create mode 100644 src/test/hive/golden/lock3-0-27ad2962fed131f51ba802596ba37278 create mode 100644 src/test/hive/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a create mode 100644 src/test/hive/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 create mode 100644 src/test/hive/golden/lock3-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac create mode 100644 src/test/hive/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe create mode 100644 src/test/hive/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 create mode 100644 src/test/hive/golden/lock4-1-27ad2962fed131f51ba802596ba37278 create mode 100644 src/test/hive/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a create mode 100644 src/test/hive/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 create mode 100644 src/test/hive/golden/lock4-4-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac create mode 100644 src/test/hive/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe create mode 100644 src/test/hive/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd create mode 100644 src/test/hive/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 create mode 100644 src/test/hive/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 create mode 100644 src/test/hive/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef create mode 100644 src/test/hive/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 create mode 100644 src/test/hive/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 create mode 100644 src/test/hive/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 create mode 100644 src/test/hive/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 create mode 100644 src/test/hive/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d create mode 100644 src/test/hive/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 create mode 100644 src/test/hive/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 create mode 100644 src/test/hive/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 create mode 100644 src/test/hive/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 create mode 100644 src/test/hive/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 create mode 100644 src/test/hive/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d create mode 100644 src/test/hive/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 create mode 100644 src/test/hive/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba create mode 100644 src/test/hive/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 create mode 100644 src/test/hive/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 create mode 100644 src/test/hive/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 create mode 100644 src/test/hive/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 create mode 100644 src/test/hive/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 src/test/hive/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 create mode 100644 src/test/hive/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f create mode 100644 src/test/hive/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 src/test/hive/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 src/test/hive/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 create mode 100644 src/test/hive/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f create mode 100644 src/test/hive/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 create mode 100644 src/test/hive/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f create mode 100644 src/test/hive/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 src/test/hive/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 create mode 100644 src/test/hive/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f create mode 100644 src/test/hive/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 src/test/hive/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 src/test/hive/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d create mode 100644 src/test/hive/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 create mode 100644 src/test/hive/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 create mode 100644 src/test/hive/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 create mode 100644 src/test/hive/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b create mode 100644 src/test/hive/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 create mode 100644 src/test/hive/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c create mode 100644 src/test/hive/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b create mode 100644 src/test/hive/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 create mode 100644 src/test/hive/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 create mode 100644 src/test/hive/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 create mode 100644 src/test/hive/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 create mode 100644 src/test/hive/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 create mode 100644 src/test/hive/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 create mode 100644 src/test/hive/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 create mode 100644 src/test/hive/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 create mode 100644 src/test/hive/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 create mode 100644 src/test/hive/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 create mode 100644 src/test/hive/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 create mode 100644 src/test/hive/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 create mode 100644 src/test/hive/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 create mode 100644 src/test/hive/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e create mode 100644 src/test/hive/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d create mode 100644 src/test/hive/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc create mode 100644 src/test/hive/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a create mode 100644 src/test/hive/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 create mode 100644 src/test/hive/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 create mode 100644 src/test/hive/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 create mode 100644 src/test/hive/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 create mode 100644 src/test/hive/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 create mode 100644 src/test/hive/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f create mode 100644 src/test/hive/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 create mode 100644 src/test/hive/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 create mode 100644 src/test/hive/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 create mode 100644 src/test/hive/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 create mode 100644 src/test/hive/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a create mode 100644 src/test/hive/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 create mode 100644 src/test/hive/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c create mode 100644 src/test/hive/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe create mode 100644 src/test/hive/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e create mode 100644 src/test/hive/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c create mode 100644 src/test/hive/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 create mode 100644 src/test/hive/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b create mode 100644 src/test/hive/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc create mode 100644 src/test/hive/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c create mode 100644 src/test/hive/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b create mode 100644 src/test/hive/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a create mode 100644 src/test/hive/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c create mode 100644 src/test/hive/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f create mode 100644 src/test/hive/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab create mode 100644 src/test/hive/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c create mode 100644 src/test/hive/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 create mode 100644 src/test/hive/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca create mode 100644 src/test/hive/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c create mode 100644 src/test/hive/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb create mode 100644 src/test/hive/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 create mode 100644 src/test/hive/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 create mode 100644 src/test/hive/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a create mode 100644 src/test/hive/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 create mode 100644 src/test/hive/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 create mode 100644 src/test/hive/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b create mode 100644 src/test/hive/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 create mode 100644 src/test/hive/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/merge1-1-2c73c923962b91afdf0004a705432550 create mode 100644 src/test/hive/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 create mode 100644 src/test/hive/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 create mode 100644 src/test/hive/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 create mode 100644 src/test/hive/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 create mode 100644 src/test/hive/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 create mode 100644 src/test/hive/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 create mode 100644 src/test/hive/golden/merge1-3-1f560722f18ef618a0343313a0cac462 create mode 100644 src/test/hive/golden/merge1-4-3277fe538b66923cd879b45371838d2b create mode 100644 src/test/hive/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 create mode 100644 src/test/hive/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 create mode 100644 src/test/hive/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 create mode 100644 src/test/hive/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a create mode 100644 src/test/hive/golden/merge1-9-64678b0928c4fd054e1578458001c86 create mode 100644 src/test/hive/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 create mode 100644 src/test/hive/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 create mode 100644 src/test/hive/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 create mode 100644 src/test/hive/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a create mode 100644 src/test/hive/golden/merge2-14-64678b0928c4fd054e1578458001c86 create mode 100644 src/test/hive/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 create mode 100644 src/test/hive/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 create mode 100644 src/test/hive/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 create mode 100644 src/test/hive/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f create mode 100644 src/test/hive/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 create mode 100644 src/test/hive/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 src/test/hive/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 src/test/hive/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 src/test/hive/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 src/test/hive/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 create mode 100644 src/test/hive/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 create mode 100644 src/test/hive/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 create mode 100644 src/test/hive/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd create mode 100644 src/test/hive/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/merge4-10-692a197bd688b48f762e72978f54aa32 create mode 100644 src/test/hive/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 create mode 100644 src/test/hive/golden/merge4-12-62541540a18d68a3cb8497a741061d11 create mode 100644 src/test/hive/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 create mode 100644 src/test/hive/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 create mode 100644 src/test/hive/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a create mode 100644 src/test/hive/golden/merge4-2-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/merge4-4-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/merge4-5-3d24d877366c42030f6d9a596665720d create mode 100644 src/test/hive/golden/merge4-6-b3a76420183795720ab3a384046e5af create mode 100644 src/test/hive/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d create mode 100644 src/test/hive/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 create mode 100644 src/test/hive/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 create mode 100644 src/test/hive/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 create mode 100644 src/test/hive/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 create mode 100644 src/test/hive/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da create mode 100644 src/test/hive/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 create mode 100644 src/test/hive/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 create mode 100644 src/test/hive/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 create mode 100644 src/test/hive/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 create mode 100644 src/test/hive/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 create mode 100644 src/test/hive/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 create mode 100644 src/test/hive/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 create mode 100644 src/test/hive/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 create mode 100644 src/test/hive/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 create mode 100644 src/test/hive/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 create mode 100644 src/test/hive/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d create mode 100644 src/test/hive/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f create mode 100644 src/test/hive/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 create mode 100644 src/test/hive/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 create mode 100644 src/test/hive/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf create mode 100644 src/test/hive/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 create mode 100644 src/test/hive/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 create mode 100644 src/test/hive/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b create mode 100644 src/test/hive/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 create mode 100644 src/test/hive/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da create mode 100644 src/test/hive/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f create mode 100644 src/test/hive/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf create mode 100644 src/test/hive/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b create mode 100644 src/test/hive/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 create mode 100644 src/test/hive/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba create mode 100644 src/test/hive/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d create mode 100644 src/test/hive/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a create mode 100644 src/test/hive/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a create mode 100644 src/test/hive/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f create mode 100644 src/test/hive/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf create mode 100644 src/test/hive/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 create mode 100644 src/test/hive/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b create mode 100644 src/test/hive/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c create mode 100644 src/test/hive/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 create mode 100644 src/test/hive/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 create mode 100644 src/test/hive/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b create mode 100644 src/test/hive/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c create mode 100644 src/test/hive/golden/mergejoins-3-c408f69470d652da283442a62b384e46 create mode 100644 src/test/hive/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 create mode 100644 src/test/hive/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e create mode 100644 src/test/hive/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df create mode 100644 src/test/hive/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 create mode 100644 src/test/hive/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 create mode 100644 src/test/hive/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 create mode 100644 src/test/hive/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 create mode 100644 src/test/hive/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b create mode 100644 src/test/hive/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff create mode 100644 src/test/hive/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 create mode 100644 src/test/hive/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 create mode 100644 src/test/hive/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 create mode 100644 src/test/hive/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 create mode 100644 src/test/hive/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 create mode 100644 src/test/hive/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 create mode 100644 src/test/hive/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 create mode 100644 src/test/hive/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d create mode 100644 src/test/hive/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 create mode 100644 src/test/hive/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d create mode 100644 src/test/hive/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 create mode 100644 src/test/hive/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d create mode 100644 src/test/hive/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 create mode 100644 src/test/hive/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 create mode 100644 src/test/hive/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d create mode 100644 src/test/hive/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf create mode 100644 src/test/hive/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc create mode 100644 src/test/hive/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 create mode 100644 src/test/hive/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 create mode 100644 src/test/hive/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/mi-1-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e create mode 100644 src/test/hive/golden/mi-3-b66a495f7bdf106a7886b72267b8659d create mode 100644 src/test/hive/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c create mode 100644 src/test/hive/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc create mode 100644 src/test/hive/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 create mode 100644 src/test/hive/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 create mode 100644 src/test/hive/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 create mode 100644 src/test/hive/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 create mode 100644 src/test/hive/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a create mode 100644 src/test/hive/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa create mode 100644 src/test/hive/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 create mode 100644 src/test/hive/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 create mode 100644 src/test/hive/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 create mode 100644 src/test/hive/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 create mode 100644 src/test/hive/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee create mode 100644 src/test/hive/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 create mode 100644 src/test/hive/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 create mode 100644 src/test/hive/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 create mode 100644 src/test/hive/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac create mode 100644 src/test/hive/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 create mode 100644 src/test/hive/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a create mode 100644 src/test/hive/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 create mode 100644 src/test/hive/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 create mode 100644 src/test/hive/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 create mode 100644 src/test/hive/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f create mode 100644 src/test/hive/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 create mode 100644 src/test/hive/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 create mode 100644 src/test/hive/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 create mode 100644 src/test/hive/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 create mode 100644 src/test/hive/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b create mode 100644 src/test/hive/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 create mode 100644 src/test/hive/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 create mode 100644 src/test/hive/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe create mode 100644 src/test/hive/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 create mode 100644 src/test/hive/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba create mode 100644 src/test/hive/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c create mode 100644 src/test/hive/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 create mode 100644 src/test/hive/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 create mode 100644 src/test/hive/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e create mode 100644 src/test/hive/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe create mode 100644 src/test/hive/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b create mode 100644 src/test/hive/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe create mode 100644 src/test/hive/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f create mode 100644 src/test/hive/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 create mode 100644 src/test/hive/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 create mode 100644 src/test/hive/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 create mode 100644 src/test/hive/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 src/test/hive/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 create mode 100644 src/test/hive/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 create mode 100644 src/test/hive/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 create mode 100644 src/test/hive/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 create mode 100644 src/test/hive/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 create mode 100644 src/test/hive/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f create mode 100644 src/test/hive/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e create mode 100644 src/test/hive/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d create mode 100644 src/test/hive/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 create mode 100644 src/test/hive/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 create mode 100644 src/test/hive/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 create mode 100644 src/test/hive/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 create mode 100644 src/test/hive/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 create mode 100644 src/test/hive/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 create mode 100644 src/test/hive/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 create mode 100644 src/test/hive/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d create mode 100644 src/test/hive/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 create mode 100644 src/test/hive/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 create mode 100644 src/test/hive/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 create mode 100644 src/test/hive/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 src/test/hive/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c create mode 100644 src/test/hive/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f create mode 100644 src/test/hive/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 create mode 100644 src/test/hive/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 src/test/hive/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 create mode 100644 src/test/hive/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 create mode 100644 src/test/hive/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 create mode 100644 src/test/hive/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e create mode 100644 src/test/hive/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 create mode 100644 src/test/hive/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 src/test/hive/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 create mode 100644 src/test/hive/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a create mode 100644 src/test/hive/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 src/test/hive/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc create mode 100644 src/test/hive/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f create mode 100644 src/test/hive/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 create mode 100644 src/test/hive/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca create mode 100644 src/test/hive/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 create mode 100644 src/test/hive/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 create mode 100644 src/test/hive/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 create mode 100644 src/test/hive/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 src/test/hive/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b create mode 100644 src/test/hive/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f create mode 100644 src/test/hive/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f create mode 100644 src/test/hive/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a create mode 100644 src/test/hive/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f create mode 100644 src/test/hive/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d create mode 100644 src/test/hive/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df create mode 100644 src/test/hive/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f create mode 100644 src/test/hive/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d create mode 100644 src/test/hive/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f create mode 100644 src/test/hive/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 create mode 100644 src/test/hive/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 create mode 100644 src/test/hive/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b create mode 100644 src/test/hive/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 create mode 100644 src/test/hive/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 create mode 100644 src/test/hive/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 create mode 100644 src/test/hive/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 create mode 100644 src/test/hive/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae create mode 100644 src/test/hive/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 create mode 100644 src/test/hive/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 create mode 100644 src/test/hive/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 create mode 100644 src/test/hive/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 create mode 100644 src/test/hive/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 create mode 100644 src/test/hive/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 create mode 100644 src/test/hive/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 create mode 100644 src/test/hive/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 create mode 100644 src/test/hive/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e create mode 100644 src/test/hive/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 create mode 100644 src/test/hive/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 create mode 100644 src/test/hive/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 create mode 100644 src/test/hive/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 create mode 100644 src/test/hive/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d create mode 100644 src/test/hive/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 create mode 100644 src/test/hive/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f create mode 100644 src/test/hive/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 create mode 100644 src/test/hive/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 create mode 100644 src/test/hive/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 create mode 100644 src/test/hive/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f create mode 100644 src/test/hive/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 create mode 100644 src/test/hive/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f create mode 100644 src/test/hive/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 create mode 100644 src/test/hive/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 create mode 100644 src/test/hive/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 create mode 100644 src/test/hive/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f create mode 100644 src/test/hive/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 create mode 100644 src/test/hive/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 create mode 100644 src/test/hive/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 create mode 100644 src/test/hive/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f create mode 100644 src/test/hive/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 create mode 100644 src/test/hive/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f create mode 100644 src/test/hive/golden/newline-0-43392a20a8d249a279d50d96578e6a1b create mode 100644 src/test/hive/golden/newline-1-a19a19272149c732977c37e043910505 create mode 100644 src/test/hive/golden/newline-2-4eb54a664e549614d56ca088c8867d create mode 100644 src/test/hive/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 create mode 100644 src/test/hive/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 create mode 100644 src/test/hive/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b create mode 100644 src/test/hive/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 create mode 100644 src/test/hive/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 create mode 100644 src/test/hive/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 create mode 100644 src/test/hive/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 create mode 100644 src/test/hive/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 create mode 100644 src/test/hive/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 create mode 100644 src/test/hive/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 create mode 100644 src/test/hive/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 create mode 100644 src/test/hive/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 create mode 100644 src/test/hive/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad create mode 100644 src/test/hive/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 create mode 100644 src/test/hive/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d create mode 100644 src/test/hive/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd create mode 100644 src/test/hive/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e create mode 100644 src/test/hive/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d create mode 100644 src/test/hive/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 create mode 100644 src/test/hive/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 create mode 100644 src/test/hive/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 create mode 100644 src/test/hive/golden/null_cast-1-7257e6f8170e545962d27741353f672c create mode 100644 src/test/hive/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 create mode 100644 src/test/hive/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 create mode 100644 src/test/hive/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 create mode 100644 src/test/hive/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 create mode 100644 src/test/hive/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 create mode 100644 src/test/hive/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 create mode 100644 src/test/hive/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 create mode 100644 src/test/hive/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 create mode 100644 src/test/hive/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 create mode 100644 src/test/hive/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a create mode 100644 src/test/hive/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 create mode 100644 src/test/hive/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a create mode 100644 src/test/hive/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 create mode 100644 src/test/hive/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a create mode 100644 src/test/hive/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 create mode 100644 src/test/hive/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a create mode 100644 src/test/hive/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 create mode 100644 src/test/hive/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 create mode 100644 src/test/hive/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 create mode 100644 src/test/hive/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 create mode 100644 src/test/hive/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 create mode 100644 src/test/hive/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb create mode 100644 src/test/hive/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e create mode 100644 src/test/hive/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 create mode 100644 src/test/hive/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 create mode 100644 src/test/hive/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 create mode 100644 src/test/hive/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 create mode 100644 src/test/hive/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 create mode 100644 src/test/hive/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb create mode 100644 src/test/hive/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 create mode 100644 src/test/hive/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 create mode 100644 src/test/hive/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e create mode 100644 src/test/hive/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 create mode 100644 src/test/hive/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 create mode 100644 src/test/hive/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 create mode 100644 src/test/hive/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 create mode 100644 src/test/hive/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 create mode 100644 src/test/hive/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 create mode 100644 src/test/hive/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 create mode 100644 src/test/hive/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 create mode 100644 src/test/hive/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 create mode 100644 src/test/hive/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 create mode 100644 src/test/hive/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 create mode 100644 src/test/hive/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 create mode 100644 src/test/hive/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 create mode 100644 src/test/hive/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 create mode 100644 src/test/hive/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 create mode 100644 src/test/hive/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 create mode 100644 src/test/hive/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 create mode 100644 src/test/hive/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 create mode 100644 src/test/hive/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 create mode 100644 src/test/hive/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 create mode 100644 src/test/hive/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b create mode 100644 src/test/hive/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 create mode 100644 src/test/hive/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c create mode 100644 src/test/hive/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e create mode 100644 src/test/hive/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c create mode 100644 src/test/hive/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 create mode 100644 src/test/hive/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 create mode 100644 src/test/hive/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf create mode 100644 src/test/hive/golden/nullinput2-0-40b73200f853b473855e85eba391f008 create mode 100644 src/test/hive/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 create mode 100644 src/test/hive/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 create mode 100644 src/test/hive/golden/nullscript-0-4477b1566208a6805bb20630755a9375 create mode 100644 src/test/hive/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 create mode 100644 src/test/hive/golden/nullscript-2-17238164053203d56d30704e2c098e80 create mode 100644 src/test/hive/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 create mode 100644 src/test/hive/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 create mode 100644 src/test/hive/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc create mode 100644 src/test/hive/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 create mode 100644 src/test/hive/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a create mode 100644 src/test/hive/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 create mode 100644 src/test/hive/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f create mode 100644 src/test/hive/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a create mode 100644 src/test/hive/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 create mode 100644 src/test/hive/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e create mode 100644 src/test/hive/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 create mode 100644 src/test/hive/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 create mode 100644 src/test/hive/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d create mode 100644 src/test/hive/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 create mode 100644 src/test/hive/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 create mode 100644 src/test/hive/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 create mode 100644 src/test/hive/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 create mode 100644 src/test/hive/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 create mode 100644 src/test/hive/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d create mode 100644 src/test/hive/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb create mode 100644 src/test/hive/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 create mode 100644 src/test/hive/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 create mode 100644 src/test/hive/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 create mode 100644 src/test/hive/golden/orc_create-20-176d469a0edba57404416535c7d48023 create mode 100644 src/test/hive/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 create mode 100644 src/test/hive/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 create mode 100644 src/test/hive/golden/orc_create-23-be779533ea8967231e644209114c8350 create mode 100644 src/test/hive/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 create mode 100644 src/test/hive/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead create mode 100644 src/test/hive/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c create mode 100644 src/test/hive/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 create mode 100644 src/test/hive/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 create mode 100644 src/test/hive/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 create mode 100644 src/test/hive/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae create mode 100644 src/test/hive/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d create mode 100644 src/test/hive/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 create mode 100644 src/test/hive/golden/orc_create-32-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 create mode 100644 src/test/hive/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 create mode 100644 src/test/hive/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 create mode 100644 src/test/hive/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a create mode 100644 src/test/hive/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 create mode 100644 src/test/hive/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 create mode 100644 src/test/hive/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 create mode 100644 src/test/hive/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 create mode 100644 src/test/hive/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae create mode 100644 src/test/hive/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 create mode 100644 src/test/hive/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 create mode 100644 src/test/hive/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 create mode 100644 src/test/hive/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 create mode 100644 src/test/hive/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 create mode 100644 src/test/hive/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 create mode 100644 src/test/hive/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 create mode 100644 src/test/hive/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 create mode 100644 src/test/hive/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c create mode 100644 src/test/hive/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d create mode 100644 src/test/hive/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 create mode 100644 src/test/hive/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 create mode 100644 src/test/hive/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 create mode 100644 src/test/hive/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 create mode 100644 src/test/hive/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 create mode 100644 src/test/hive/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 create mode 100644 src/test/hive/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 create mode 100644 src/test/hive/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 create mode 100644 src/test/hive/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 create mode 100644 src/test/hive/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e create mode 100644 src/test/hive/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 create mode 100644 src/test/hive/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c create mode 100644 src/test/hive/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 create mode 100644 src/test/hive/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 create mode 100644 src/test/hive/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 create mode 100644 src/test/hive/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf create mode 100644 src/test/hive/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 create mode 100644 src/test/hive/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 create mode 100644 src/test/hive/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e create mode 100644 src/test/hive/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 create mode 100644 src/test/hive/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e create mode 100644 src/test/hive/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 create mode 100644 src/test/hive/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 create mode 100644 src/test/hive/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc create mode 100644 src/test/hive/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f create mode 100644 src/test/hive/golden/order-0-5c2fda577771db3b316e0d2bd02d048a create mode 100644 src/test/hive/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 create mode 100644 src/test/hive/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf create mode 100644 src/test/hive/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 create mode 100644 src/test/hive/golden/order2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf create mode 100644 src/test/hive/golden/order2-2-4162aa366dc0836eed15cc819226907f create mode 100644 src/test/hive/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c create mode 100644 src/test/hive/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 create mode 100644 src/test/hive/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba create mode 100644 src/test/hive/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 create mode 100644 src/test/hive/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 create mode 100644 src/test/hive/golden/parallel-1-851e262128626126ae1ad87869db7c54 create mode 100644 src/test/hive/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 create mode 100644 src/test/hive/golden/parallel-11-6230286bc168af7b010968b543690a2a create mode 100644 src/test/hive/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 create mode 100644 src/test/hive/golden/parallel-2-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a create mode 100644 src/test/hive/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 create mode 100644 src/test/hive/golden/parallel-5-e27b511ffc5828b09069121031d17a2f create mode 100644 src/test/hive/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 create mode 100644 src/test/hive/golden/parallel-7-6230286bc168af7b010968b543690a2a create mode 100644 src/test/hive/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 create mode 100644 src/test/hive/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 src/test/hive/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 create mode 100644 src/test/hive/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 create mode 100644 src/test/hive/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc create mode 100644 src/test/hive/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd create mode 100644 src/test/hive/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec create mode 100644 src/test/hive/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 create mode 100644 src/test/hive/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 create mode 100644 src/test/hive/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea create mode 100644 src/test/hive/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 create mode 100644 src/test/hive/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b create mode 100644 src/test/hive/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 create mode 100644 src/test/hive/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 create mode 100644 src/test/hive/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 create mode 100644 src/test/hive/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 create mode 100644 src/test/hive/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 create mode 100644 src/test/hive/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb create mode 100644 src/test/hive/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 create mode 100644 src/test/hive/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 create mode 100644 src/test/hive/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 create mode 100644 src/test/hive/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 create mode 100644 src/test/hive/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb create mode 100644 src/test/hive/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 create mode 100644 src/test/hive/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 create mode 100644 src/test/hive/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 create mode 100644 src/test/hive/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb create mode 100644 src/test/hive/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 create mode 100644 src/test/hive/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 create mode 100644 src/test/hive/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a create mode 100644 src/test/hive/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 create mode 100644 src/test/hive/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 create mode 100644 src/test/hive/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 create mode 100644 src/test/hive/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 create mode 100644 src/test/hive/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 create mode 100644 src/test/hive/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 create mode 100644 src/test/hive/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 create mode 100644 src/test/hive/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 create mode 100644 src/test/hive/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 create mode 100644 src/test/hive/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 create mode 100644 src/test/hive/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 create mode 100644 src/test/hive/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 create mode 100644 src/test/hive/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c create mode 100644 src/test/hive/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 create mode 100644 src/test/hive/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 create mode 100644 src/test/hive/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb create mode 100644 src/test/hive/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 create mode 100644 src/test/hive/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 create mode 100644 src/test/hive/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d create mode 100644 src/test/hive/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 create mode 100644 src/test/hive/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf create mode 100644 src/test/hive/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 create mode 100644 src/test/hive/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 create mode 100644 src/test/hive/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 create mode 100644 src/test/hive/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 create mode 100644 src/test/hive/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 create mode 100644 src/test/hive/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 create mode 100644 src/test/hive/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 create mode 100644 src/test/hive/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 create mode 100644 src/test/hive/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 create mode 100644 src/test/hive/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c create mode 100644 src/test/hive/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa create mode 100644 src/test/hive/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc create mode 100644 src/test/hive/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a create mode 100644 src/test/hive/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f create mode 100644 src/test/hive/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 create mode 100644 src/test/hive/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d create mode 100644 src/test/hive/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 create mode 100644 src/test/hive/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 create mode 100644 src/test/hive/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 create mode 100644 src/test/hive/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d create mode 100644 src/test/hive/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 create mode 100644 src/test/hive/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 create mode 100644 src/test/hive/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 create mode 100644 src/test/hive/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 create mode 100644 src/test/hive/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 create mode 100644 src/test/hive/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 create mode 100644 src/test/hive/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c create mode 100644 src/test/hive/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 create mode 100644 src/test/hive/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 create mode 100644 src/test/hive/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d create mode 100644 src/test/hive/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 create mode 100644 src/test/hive/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a create mode 100644 src/test/hive/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 create mode 100644 src/test/hive/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 create mode 100644 src/test/hive/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 create mode 100644 src/test/hive/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef create mode 100644 src/test/hive/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce create mode 100644 src/test/hive/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 create mode 100644 src/test/hive/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e create mode 100644 src/test/hive/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b create mode 100644 src/test/hive/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 create mode 100644 src/test/hive/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 create mode 100644 src/test/hive/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 create mode 100644 src/test/hive/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf create mode 100644 src/test/hive/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 create mode 100644 src/test/hive/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde create mode 100644 src/test/hive/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 create mode 100644 src/test/hive/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 create mode 100644 src/test/hive/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a create mode 100644 src/test/hive/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 create mode 100644 src/test/hive/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 create mode 100644 src/test/hive/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 create mode 100644 src/test/hive/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f create mode 100644 src/test/hive/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff create mode 100644 src/test/hive/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 create mode 100644 src/test/hive/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 create mode 100644 src/test/hive/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a create mode 100644 src/test/hive/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 create mode 100644 src/test/hive/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 create mode 100644 src/test/hive/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 create mode 100644 src/test/hive/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 create mode 100644 src/test/hive/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a create mode 100644 src/test/hive/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 create mode 100644 src/test/hive/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f create mode 100644 src/test/hive/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 create mode 100644 src/test/hive/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 create mode 100644 src/test/hive/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 create mode 100644 src/test/hive/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 create mode 100644 src/test/hive/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a create mode 100644 src/test/hive/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 create mode 100644 src/test/hive/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 create mode 100644 src/test/hive/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 create mode 100644 src/test/hive/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 create mode 100644 src/test/hive/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e create mode 100644 src/test/hive/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf create mode 100644 src/test/hive/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 create mode 100644 src/test/hive/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf create mode 100644 src/test/hive/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e create mode 100644 src/test/hive/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 create mode 100644 src/test/hive/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 create mode 100644 src/test/hive/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 create mode 100644 src/test/hive/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f create mode 100644 src/test/hive/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba create mode 100644 src/test/hive/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 create mode 100644 src/test/hive/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba create mode 100644 src/test/hive/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 create mode 100644 src/test/hive/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 create mode 100644 src/test/hive/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 create mode 100644 src/test/hive/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 create mode 100644 src/test/hive/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f create mode 100644 src/test/hive/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d create mode 100644 src/test/hive/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 create mode 100644 src/test/hive/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 create mode 100644 src/test/hive/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 create mode 100644 src/test/hive/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 create mode 100644 src/test/hive/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af create mode 100644 src/test/hive/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b create mode 100644 src/test/hive/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 create mode 100644 src/test/hive/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 create mode 100644 src/test/hive/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b create mode 100644 src/test/hive/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d create mode 100644 src/test/hive/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe create mode 100644 src/test/hive/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c create mode 100644 src/test/hive/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 create mode 100644 src/test/hive/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b create mode 100644 src/test/hive/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 create mode 100644 src/test/hive/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 create mode 100644 src/test/hive/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 create mode 100644 src/test/hive/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b create mode 100644 src/test/hive/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f create mode 100644 src/test/hive/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba create mode 100644 src/test/hive/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 create mode 100644 src/test/hive/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 create mode 100644 src/test/hive/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 create mode 100644 src/test/hive/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b create mode 100644 src/test/hive/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f create mode 100644 src/test/hive/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 src/test/hive/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba create mode 100644 src/test/hive/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 src/test/hive/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 create mode 100644 src/test/hive/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 create mode 100644 src/test/hive/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 create mode 100644 src/test/hive/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 create mode 100644 src/test/hive/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d create mode 100644 src/test/hive/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 create mode 100644 src/test/hive/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 create mode 100644 src/test/hive/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a create mode 100644 src/test/hive/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f create mode 100644 src/test/hive/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff create mode 100644 src/test/hive/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 create mode 100644 src/test/hive/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d create mode 100644 src/test/hive/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 create mode 100644 src/test/hive/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada create mode 100644 src/test/hive/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 create mode 100644 src/test/hive/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 create mode 100644 src/test/hive/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 create mode 100644 src/test/hive/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a create mode 100644 src/test/hive/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a create mode 100644 src/test/hive/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f create mode 100644 src/test/hive/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff create mode 100644 src/test/hive/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 create mode 100644 src/test/hive/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a create mode 100644 src/test/hive/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a create mode 100644 src/test/hive/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a create mode 100644 src/test/hive/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 create mode 100644 src/test/hive/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 create mode 100644 src/test/hive/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 create mode 100644 src/test/hive/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 create mode 100644 src/test/hive/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 create mode 100644 src/test/hive/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 create mode 100644 src/test/hive/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 create mode 100644 src/test/hive/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 create mode 100644 src/test/hive/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a create mode 100644 src/test/hive/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f create mode 100644 src/test/hive/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff create mode 100644 src/test/hive/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 create mode 100644 src/test/hive/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad create mode 100644 src/test/hive/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 create mode 100644 src/test/hive/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 create mode 100644 src/test/hive/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 create mode 100644 src/test/hive/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a create mode 100644 src/test/hive/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f create mode 100644 src/test/hive/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff create mode 100644 src/test/hive/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f create mode 100644 src/test/hive/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 create mode 100644 src/test/hive/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 create mode 100644 src/test/hive/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 create mode 100644 src/test/hive/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a create mode 100644 src/test/hive/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 create mode 100644 src/test/hive/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 create mode 100644 src/test/hive/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 create mode 100644 src/test/hive/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 create mode 100644 src/test/hive/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 create mode 100644 src/test/hive/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 create mode 100644 src/test/hive/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e create mode 100644 src/test/hive/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e create mode 100644 src/test/hive/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b create mode 100644 src/test/hive/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd create mode 100644 src/test/hive/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 src/test/hive/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 create mode 100644 src/test/hive/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 create mode 100644 src/test/hive/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 create mode 100644 src/test/hive/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 create mode 100644 src/test/hive/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b create mode 100644 src/test/hive/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd create mode 100644 src/test/hive/golden/plan_json-0-74146da55d57b22443140e7fbab3375c create mode 100644 src/test/hive/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 create mode 100644 src/test/hive/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 create mode 100644 src/test/hive/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 create mode 100644 src/test/hive/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 create mode 100644 src/test/hive/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 create mode 100644 src/test/hive/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b create mode 100644 src/test/hive/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 create mode 100644 src/test/hive/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 create mode 100644 src/test/hive/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 create mode 100644 src/test/hive/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b create mode 100644 src/test/hive/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 create mode 100644 src/test/hive/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 create mode 100644 src/test/hive/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 create mode 100644 src/test/hive/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 create mode 100644 src/test/hive/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 create mode 100644 src/test/hive/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 create mode 100644 src/test/hive/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 create mode 100644 src/test/hive/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 create mode 100644 src/test/hive/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 create mode 100644 src/test/hive/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 create mode 100644 src/test/hive/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 create mode 100644 src/test/hive/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c create mode 100644 src/test/hive/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 create mode 100644 src/test/hive/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 create mode 100644 src/test/hive/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c create mode 100644 src/test/hive/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 create mode 100644 src/test/hive/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 create mode 100644 src/test/hive/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e create mode 100644 src/test/hive/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d create mode 100644 src/test/hive/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e create mode 100644 src/test/hive/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d create mode 100644 src/test/hive/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 create mode 100644 src/test/hive/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f create mode 100644 src/test/hive/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 create mode 100644 src/test/hive/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f create mode 100644 src/test/hive/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 create mode 100644 src/test/hive/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 create mode 100644 src/test/hive/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 create mode 100644 src/test/hive/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 create mode 100644 src/test/hive/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 create mode 100644 src/test/hive/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 create mode 100644 src/test/hive/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d create mode 100644 src/test/hive/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 create mode 100644 src/test/hive/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d create mode 100644 src/test/hive/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 create mode 100644 src/test/hive/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 create mode 100644 src/test/hive/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b create mode 100644 src/test/hive/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 create mode 100644 src/test/hive/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b create mode 100644 src/test/hive/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 create mode 100644 src/test/hive/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 create mode 100644 src/test/hive/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 create mode 100644 src/test/hive/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 create mode 100644 src/test/hive/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 create mode 100644 src/test/hive/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 create mode 100644 src/test/hive/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 create mode 100644 src/test/hive/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 create mode 100644 src/test/hive/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 create mode 100644 src/test/hive/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 create mode 100644 src/test/hive/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e create mode 100644 src/test/hive/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 create mode 100644 src/test/hive/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e create mode 100644 src/test/hive/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 create mode 100644 src/test/hive/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 create mode 100644 src/test/hive/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 create mode 100644 src/test/hive/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 create mode 100644 src/test/hive/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 create mode 100644 src/test/hive/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 create mode 100644 src/test/hive/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 create mode 100644 src/test/hive/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 create mode 100644 src/test/hive/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 create mode 100644 src/test/hive/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 create mode 100644 src/test/hive/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe create mode 100644 src/test/hive/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 create mode 100644 src/test/hive/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe create mode 100644 src/test/hive/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b create mode 100644 src/test/hive/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 create mode 100644 src/test/hive/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 create mode 100644 src/test/hive/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee create mode 100644 src/test/hive/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 create mode 100644 src/test/hive/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 create mode 100644 src/test/hive/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 create mode 100644 src/test/hive/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a create mode 100644 src/test/hive/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd create mode 100644 src/test/hive/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c create mode 100644 src/test/hive/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 create mode 100644 src/test/hive/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b create mode 100644 src/test/hive/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b create mode 100644 src/test/hive/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 create mode 100644 src/test/hive/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 create mode 100644 src/test/hive/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 create mode 100644 src/test/hive/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 create mode 100644 src/test/hive/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f create mode 100644 src/test/hive/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 create mode 100644 src/test/hive/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 create mode 100644 src/test/hive/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf create mode 100644 src/test/hive/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 create mode 100644 src/test/hive/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 create mode 100644 src/test/hive/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 create mode 100644 src/test/hive/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 create mode 100644 src/test/hive/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 create mode 100644 src/test/hive/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 create mode 100644 src/test/hive/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 create mode 100644 src/test/hive/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 create mode 100644 src/test/hive/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e create mode 100644 src/test/hive/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed create mode 100644 src/test/hive/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae create mode 100644 src/test/hive/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 create mode 100644 src/test/hive/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e create mode 100644 src/test/hive/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed create mode 100644 src/test/hive/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae create mode 100644 src/test/hive/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 src/test/hive/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 create mode 100644 src/test/hive/golden/ppd_union-3-678c8197f458b459171c266f7431683e create mode 100644 src/test/hive/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 src/test/hive/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 create mode 100644 src/test/hive/golden/ppd_union-6-678c8197f458b459171c266f7431683e create mode 100644 src/test/hive/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 create mode 100644 src/test/hive/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 create mode 100644 src/test/hive/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f create mode 100644 src/test/hive/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c create mode 100644 src/test/hive/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 create mode 100644 src/test/hive/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 create mode 100644 src/test/hive/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 create mode 100644 src/test/hive/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 create mode 100644 src/test/hive/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 create mode 100644 src/test/hive/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c create mode 100644 src/test/hive/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 create mode 100644 src/test/hive/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c create mode 100644 src/test/hive/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 create mode 100644 src/test/hive/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 create mode 100644 src/test/hive/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 create mode 100644 src/test/hive/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 create mode 100644 src/test/hive/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d create mode 100644 src/test/hive/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 create mode 100644 src/test/hive/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 create mode 100644 src/test/hive/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d create mode 100644 src/test/hive/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 create mode 100644 src/test/hive/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b create mode 100644 src/test/hive/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 create mode 100644 src/test/hive/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 create mode 100644 src/test/hive/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 create mode 100644 src/test/hive/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 create mode 100644 src/test/hive/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 create mode 100644 src/test/hive/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c create mode 100644 src/test/hive/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb create mode 100644 src/test/hive/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae create mode 100644 src/test/hive/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 create mode 100644 src/test/hive/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 create mode 100644 src/test/hive/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a create mode 100644 src/test/hive/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 create mode 100644 src/test/hive/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 create mode 100644 src/test/hive/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d create mode 100644 src/test/hive/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d create mode 100644 src/test/hive/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 create mode 100644 src/test/hive/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 create mode 100644 src/test/hive/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b create mode 100644 src/test/hive/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 create mode 100644 src/test/hive/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 create mode 100644 src/test/hive/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 create mode 100644 src/test/hive/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea create mode 100644 src/test/hive/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 create mode 100644 src/test/hive/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 create mode 100644 src/test/hive/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf create mode 100644 src/test/hive/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b create mode 100644 src/test/hive/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 create mode 100644 src/test/hive/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 create mode 100644 src/test/hive/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de create mode 100644 src/test/hive/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d create mode 100644 src/test/hive/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 create mode 100644 src/test/hive/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c create mode 100644 src/test/hive/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 create mode 100644 src/test/hive/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b create mode 100644 src/test/hive/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 create mode 100644 src/test/hive/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c create mode 100644 src/test/hive/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b create mode 100644 src/test/hive/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a create mode 100644 src/test/hive/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 create mode 100644 src/test/hive/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 create mode 100644 src/test/hive/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 create mode 100644 src/test/hive/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb create mode 100644 src/test/hive/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe create mode 100644 src/test/hive/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 create mode 100644 src/test/hive/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 create mode 100644 src/test/hive/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e create mode 100644 src/test/hive/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 create mode 100644 src/test/hive/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c create mode 100644 src/test/hive/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 create mode 100644 src/test/hive/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 create mode 100644 src/test/hive/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 create mode 100644 src/test/hive/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 create mode 100644 src/test/hive/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 create mode 100644 src/test/hive/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 create mode 100644 src/test/hive/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a create mode 100644 src/test/hive/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 create mode 100644 src/test/hive/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 create mode 100644 src/test/hive/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d create mode 100644 src/test/hive/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb create mode 100644 src/test/hive/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 create mode 100644 src/test/hive/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f create mode 100644 src/test/hive/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 create mode 100644 src/test/hive/golden/print_header-1-8540676fc16ac91f3629c40f393a890a create mode 100644 src/test/hive/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 create mode 100644 src/test/hive/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb create mode 100644 src/test/hive/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 create mode 100644 src/test/hive/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 create mode 100644 src/test/hive/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f create mode 100644 src/test/hive/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 create mode 100644 src/test/hive/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 create mode 100644 src/test/hive/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 create mode 100644 src/test/hive/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 create mode 100644 src/test/hive/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 create mode 100644 src/test/hive/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a create mode 100644 src/test/hive/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c create mode 100644 src/test/hive/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b create mode 100644 src/test/hive/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 create mode 100644 src/test/hive/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b create mode 100644 src/test/hive/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 create mode 100644 src/test/hive/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 create mode 100644 src/test/hive/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 create mode 100644 src/test/hive/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea create mode 100644 src/test/hive/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b create mode 100644 src/test/hive/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d create mode 100644 src/test/hive/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 create mode 100644 src/test/hive/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d create mode 100644 src/test/hive/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 create mode 100644 src/test/hive/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 create mode 100644 src/test/hive/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d create mode 100644 src/test/hive/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 create mode 100644 src/test/hive/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d create mode 100644 src/test/hive/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 create mode 100644 src/test/hive/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc create mode 100644 src/test/hive/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 create mode 100644 src/test/hive/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 create mode 100644 src/test/hive/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 create mode 100644 src/test/hive/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b create mode 100644 src/test/hive/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 create mode 100644 src/test/hive/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 create mode 100644 src/test/hive/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 create mode 100644 src/test/hive/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e create mode 100644 src/test/hive/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 create mode 100644 src/test/hive/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 create mode 100644 src/test/hive/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 create mode 100644 src/test/hive/golden/protectmode-40-39024332d020339bd9396c9b15403a0c create mode 100644 src/test/hive/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 create mode 100644 src/test/hive/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb create mode 100644 src/test/hive/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 create mode 100644 src/test/hive/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c create mode 100644 src/test/hive/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 create mode 100644 src/test/hive/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 create mode 100644 src/test/hive/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 create mode 100644 src/test/hive/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc create mode 100644 src/test/hive/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 create mode 100644 src/test/hive/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e create mode 100644 src/test/hive/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 create mode 100644 src/test/hive/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 create mode 100644 src/test/hive/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 create mode 100644 src/test/hive/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a create mode 100644 src/test/hive/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 create mode 100644 src/test/hive/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb create mode 100644 src/test/hive/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 create mode 100644 src/test/hive/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 create mode 100644 src/test/hive/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 create mode 100644 src/test/hive/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 create mode 100644 src/test/hive/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 create mode 100644 src/test/hive/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 create mode 100644 src/test/hive/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 create mode 100644 src/test/hive/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 create mode 100644 src/test/hive/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e create mode 100644 src/test/hive/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d create mode 100644 src/test/hive/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 create mode 100644 src/test/hive/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 create mode 100644 src/test/hive/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 create mode 100644 src/test/hive/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 create mode 100644 src/test/hive/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 create mode 100644 src/test/hive/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e create mode 100644 src/test/hive/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 create mode 100644 src/test/hive/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 create mode 100644 src/test/hive/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a create mode 100644 src/test/hive/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 create mode 100644 src/test/hive/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 create mode 100644 src/test/hive/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b create mode 100644 src/test/hive/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff create mode 100644 src/test/hive/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 create mode 100644 src/test/hive/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 create mode 100644 src/test/hive/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b create mode 100644 src/test/hive/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c create mode 100644 src/test/hive/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 create mode 100644 src/test/hive/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 create mode 100644 src/test/hive/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 create mode 100644 src/test/hive/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b create mode 100644 src/test/hive/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 create mode 100644 src/test/hive/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d create mode 100644 src/test/hive/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 create mode 100644 src/test/hive/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 create mode 100644 src/test/hive/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 create mode 100644 src/test/hive/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 create mode 100644 src/test/hive/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 create mode 100644 src/test/hive/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 create mode 100644 src/test/hive/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c create mode 100644 src/test/hive/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 create mode 100644 src/test/hive/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 create mode 100644 src/test/hive/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 create mode 100644 src/test/hive/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e create mode 100644 src/test/hive/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 create mode 100644 src/test/hive/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee create mode 100644 src/test/hive/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 create mode 100644 src/test/hive/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e create mode 100644 src/test/hive/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 create mode 100644 src/test/hive/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 create mode 100644 src/test/hive/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f create mode 100644 src/test/hive/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 create mode 100644 src/test/hive/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 create mode 100644 src/test/hive/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 create mode 100644 src/test/hive/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 create mode 100644 src/test/hive/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb create mode 100644 src/test/hive/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 create mode 100644 src/test/hive/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 create mode 100644 src/test/hive/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a create mode 100644 src/test/hive/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 create mode 100644 src/test/hive/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 create mode 100644 src/test/hive/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 create mode 100644 src/test/hive/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 create mode 100644 src/test/hive/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 create mode 100644 src/test/hive/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 create mode 100644 src/test/hive/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 create mode 100644 src/test/hive/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f create mode 100644 src/test/hive/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 create mode 100644 src/test/hive/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b create mode 100644 src/test/hive/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb create mode 100644 src/test/hive/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 create mode 100644 src/test/hive/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b create mode 100644 src/test/hive/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 create mode 100644 src/test/hive/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b create mode 100644 src/test/hive/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 create mode 100644 src/test/hive/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 create mode 100644 src/test/hive/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 create mode 100644 src/test/hive/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 create mode 100644 src/test/hive/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 create mode 100644 src/test/hive/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 create mode 100644 src/test/hive/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 create mode 100644 src/test/hive/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f create mode 100644 src/test/hive/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 create mode 100644 src/test/hive/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b create mode 100644 src/test/hive/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb create mode 100644 src/test/hive/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 create mode 100644 src/test/hive/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b create mode 100644 src/test/hive/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 create mode 100644 src/test/hive/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 create mode 100644 src/test/hive/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 create mode 100644 src/test/hive/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 create mode 100644 src/test/hive/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 create mode 100644 src/test/hive/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 create mode 100644 src/test/hive/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d create mode 100644 src/test/hive/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee create mode 100644 src/test/hive/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a create mode 100644 src/test/hive/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 create mode 100644 src/test/hive/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed create mode 100644 src/test/hive/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 create mode 100644 src/test/hive/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 create mode 100644 src/test/hive/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b create mode 100644 src/test/hive/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 create mode 100644 src/test/hive/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d create mode 100644 src/test/hive/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 create mode 100644 src/test/hive/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a create mode 100644 src/test/hive/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 create mode 100644 src/test/hive/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 create mode 100644 src/test/hive/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b create mode 100644 src/test/hive/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 create mode 100644 src/test/hive/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 create mode 100644 src/test/hive/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 create mode 100644 src/test/hive/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 create mode 100644 src/test/hive/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 create mode 100644 src/test/hive/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 create mode 100644 src/test/hive/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe create mode 100644 src/test/hive/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 create mode 100644 src/test/hive/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d create mode 100644 src/test/hive/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 create mode 100644 src/test/hive/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 create mode 100644 src/test/hive/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 create mode 100644 src/test/hive/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 create mode 100644 src/test/hive/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 create mode 100644 src/test/hive/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 create mode 100644 src/test/hive/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 create mode 100644 src/test/hive/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 create mode 100644 src/test/hive/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 create mode 100644 src/test/hive/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 create mode 100644 src/test/hive/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 create mode 100644 src/test/hive/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb create mode 100644 src/test/hive/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 create mode 100644 src/test/hive/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 create mode 100644 src/test/hive/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 create mode 100644 src/test/hive/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db create mode 100644 src/test/hive/golden/regex_col-2-21564f64cdfd46098e1254380490701 create mode 100644 src/test/hive/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 create mode 100644 src/test/hive/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 create mode 100644 src/test/hive/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 create mode 100644 src/test/hive/golden/remote_script-1-e168f471980470d93b790702a70238fa create mode 100644 src/test/hive/golden/remote_script-2-a19a19272149c732977c37e043910505 create mode 100644 src/test/hive/golden/remote_script-3-4eb54a664e549614d56ca088c8867d create mode 100644 src/test/hive/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 create mode 100644 src/test/hive/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 create mode 100644 src/test/hive/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 create mode 100644 src/test/hive/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 create mode 100644 src/test/hive/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-16-d032f4795c1186255acea241387adf93 create mode 100644 src/test/hive/golden/rename_column-17-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 create mode 100644 src/test/hive/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f create mode 100644 src/test/hive/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a create mode 100644 src/test/hive/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 create mode 100644 src/test/hive/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a create mode 100644 src/test/hive/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 create mode 100644 src/test/hive/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd create mode 100644 src/test/hive/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 create mode 100644 src/test/hive/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 create mode 100644 src/test/hive/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 create mode 100644 src/test/hive/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 create mode 100644 src/test/hive/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-36-d032f4795c1186255acea241387adf93 create mode 100644 src/test/hive/golden/rename_column-37-9c36cac1372650b703400c60dd29042c create mode 100644 src/test/hive/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 create mode 100644 src/test/hive/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd create mode 100644 src/test/hive/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 create mode 100644 src/test/hive/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 src/test/hive/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 create mode 100644 src/test/hive/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 create mode 100644 src/test/hive/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 create mode 100644 src/test/hive/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc create mode 100644 src/test/hive/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 create mode 100644 src/test/hive/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 create mode 100644 src/test/hive/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f create mode 100644 src/test/hive/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 create mode 100644 src/test/hive/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e create mode 100644 src/test/hive/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 create mode 100644 src/test/hive/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 create mode 100644 src/test/hive/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef create mode 100644 src/test/hive/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 create mode 100644 src/test/hive/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 create mode 100644 src/test/hive/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 create mode 100644 src/test/hive/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 create mode 100644 src/test/hive/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 create mode 100644 src/test/hive/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f create mode 100644 src/test/hive/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 create mode 100644 src/test/hive/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 create mode 100644 src/test/hive/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a create mode 100644 src/test/hive/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 create mode 100644 src/test/hive/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 create mode 100644 src/test/hive/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 create mode 100644 src/test/hive/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 create mode 100644 src/test/hive/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e create mode 100644 src/test/hive/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a create mode 100644 src/test/hive/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 create mode 100644 src/test/hive/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 create mode 100644 src/test/hive/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 create mode 100644 src/test/hive/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 create mode 100644 src/test/hive/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 create mode 100644 src/test/hive/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e create mode 100644 src/test/hive/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 create mode 100644 src/test/hive/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde create mode 100644 src/test/hive/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 create mode 100644 src/test/hive/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 create mode 100644 src/test/hive/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 create mode 100644 src/test/hive/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 create mode 100644 src/test/hive/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e create mode 100644 src/test/hive/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 create mode 100644 src/test/hive/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f create mode 100644 src/test/hive/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 create mode 100644 src/test/hive/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 create mode 100644 src/test/hive/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 create mode 100644 src/test/hive/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d create mode 100644 src/test/hive/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa create mode 100644 src/test/hive/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d create mode 100644 src/test/hive/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 create mode 100644 src/test/hive/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 create mode 100644 src/test/hive/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 create mode 100644 src/test/hive/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 create mode 100644 src/test/hive/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 create mode 100644 src/test/hive/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd create mode 100644 src/test/hive/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a create mode 100644 src/test/hive/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b create mode 100644 src/test/hive/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d create mode 100644 src/test/hive/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 create mode 100644 src/test/hive/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 create mode 100644 src/test/hive/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d create mode 100644 src/test/hive/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 create mode 100644 src/test/hive/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 create mode 100644 src/test/hive/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 create mode 100644 src/test/hive/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 create mode 100644 src/test/hive/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c create mode 100644 src/test/hive/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 create mode 100644 src/test/hive/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 create mode 100644 src/test/hive/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e create mode 100644 src/test/hive/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b create mode 100644 src/test/hive/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e create mode 100644 src/test/hive/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d create mode 100644 src/test/hive/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f create mode 100644 src/test/hive/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc create mode 100644 src/test/hive/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc create mode 100644 src/test/hive/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 create mode 100644 src/test/hive/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c create mode 100644 src/test/hive/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 create mode 100644 src/test/hive/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec create mode 100644 src/test/hive/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 create mode 100644 src/test/hive/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec create mode 100644 src/test/hive/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 create mode 100644 src/test/hive/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 create mode 100644 src/test/hive/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 create mode 100644 src/test/hive/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 create mode 100644 src/test/hive/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 create mode 100644 src/test/hive/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 create mode 100644 src/test/hive/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e create mode 100644 src/test/hive/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 create mode 100644 src/test/hive/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a create mode 100644 src/test/hive/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 create mode 100644 src/test/hive/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca create mode 100644 src/test/hive/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 create mode 100644 src/test/hive/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 create mode 100644 src/test/hive/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce create mode 100644 src/test/hive/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 create mode 100644 src/test/hive/golden/show_tables-11-3f9a7f993510123059493826470f78f7 create mode 100644 src/test/hive/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c create mode 100644 src/test/hive/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 create mode 100644 src/test/hive/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c create mode 100644 src/test/hive/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe create mode 100644 src/test/hive/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 create mode 100644 src/test/hive/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab create mode 100644 src/test/hive/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 create mode 100644 src/test/hive/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 create mode 100644 src/test/hive/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 create mode 100644 src/test/hive/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 create mode 100644 src/test/hive/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 src/test/hive/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 create mode 100644 src/test/hive/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c create mode 100644 src/test/hive/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 create mode 100644 src/test/hive/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d create mode 100644 src/test/hive/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 create mode 100644 src/test/hive/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c create mode 100644 src/test/hive/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 create mode 100644 src/test/hive/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 create mode 100644 src/test/hive/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa create mode 100644 src/test/hive/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 create mode 100644 src/test/hive/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 create mode 100644 src/test/hive/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d create mode 100644 src/test/hive/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 create mode 100644 src/test/hive/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 create mode 100644 src/test/hive/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 create mode 100644 src/test/hive/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c create mode 100644 src/test/hive/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 create mode 100644 src/test/hive/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 create mode 100644 src/test/hive/golden/showparts-0-593619bb962b318b82896658deaea1f1 create mode 100644 src/test/hive/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 create mode 100644 src/test/hive/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 create mode 100644 src/test/hive/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 create mode 100644 src/test/hive/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b create mode 100644 src/test/hive/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 create mode 100644 src/test/hive/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 create mode 100644 src/test/hive/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a create mode 100644 src/test/hive/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 create mode 100644 src/test/hive/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 create mode 100644 src/test/hive/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 create mode 100644 src/test/hive/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 create mode 100644 src/test/hive/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 create mode 100644 src/test/hive/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 create mode 100644 src/test/hive/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 create mode 100644 src/test/hive/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd create mode 100644 src/test/hive/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 create mode 100644 src/test/hive/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 create mode 100644 src/test/hive/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 create mode 100644 src/test/hive/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 create mode 100644 src/test/hive/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 create mode 100644 src/test/hive/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 create mode 100644 src/test/hive/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d create mode 100644 src/test/hive/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 create mode 100644 src/test/hive/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 create mode 100644 src/test/hive/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 create mode 100644 src/test/hive/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 create mode 100644 src/test/hive/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 create mode 100644 src/test/hive/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 create mode 100644 src/test/hive/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 create mode 100644 src/test/hive/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 create mode 100644 src/test/hive/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 create mode 100644 src/test/hive/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 create mode 100644 src/test/hive/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d create mode 100644 src/test/hive/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 create mode 100644 src/test/hive/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 create mode 100644 src/test/hive/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e create mode 100644 src/test/hive/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b create mode 100644 src/test/hive/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b create mode 100644 src/test/hive/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 create mode 100644 src/test/hive/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b create mode 100644 src/test/hive/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 create mode 100644 src/test/hive/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 create mode 100644 src/test/hive/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa create mode 100644 src/test/hive/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 create mode 100644 src/test/hive/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 create mode 100644 src/test/hive/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 create mode 100644 src/test/hive/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d create mode 100644 src/test/hive/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 create mode 100644 src/test/hive/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 create mode 100644 src/test/hive/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e create mode 100644 src/test/hive/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 create mode 100644 src/test/hive/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 create mode 100644 src/test/hive/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d create mode 100644 src/test/hive/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 create mode 100644 src/test/hive/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 create mode 100644 src/test/hive/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 create mode 100644 src/test/hive/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 create mode 100644 src/test/hive/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb create mode 100644 src/test/hive/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 create mode 100644 src/test/hive/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 create mode 100644 src/test/hive/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f create mode 100644 src/test/hive/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 create mode 100644 src/test/hive/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 create mode 100644 src/test/hive/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 create mode 100644 src/test/hive/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b create mode 100644 src/test/hive/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b create mode 100644 src/test/hive/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d create mode 100644 src/test/hive/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 create mode 100644 src/test/hive/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd create mode 100644 src/test/hive/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 create mode 100644 src/test/hive/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 create mode 100644 src/test/hive/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 create mode 100644 src/test/hive/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e create mode 100644 src/test/hive/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 create mode 100644 src/test/hive/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c create mode 100644 src/test/hive/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 create mode 100644 src/test/hive/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 create mode 100644 src/test/hive/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e create mode 100644 src/test/hive/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 create mode 100644 src/test/hive/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd create mode 100644 src/test/hive/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 src/test/hive/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 create mode 100644 src/test/hive/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d create mode 100644 src/test/hive/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 create mode 100644 src/test/hive/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 src/test/hive/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 create mode 100644 src/test/hive/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 src/test/hive/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 create mode 100644 src/test/hive/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d create mode 100644 src/test/hive/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 create mode 100644 src/test/hive/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 create mode 100644 src/test/hive/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 src/test/hive/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec create mode 100644 src/test/hive/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 create mode 100644 src/test/hive/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed create mode 100644 src/test/hive/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa create mode 100644 src/test/hive/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 create mode 100644 src/test/hive/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 create mode 100644 src/test/hive/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c create mode 100644 src/test/hive/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 create mode 100644 src/test/hive/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 create mode 100644 src/test/hive/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 create mode 100644 src/test/hive/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 create mode 100644 src/test/hive/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f create mode 100644 src/test/hive/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 create mode 100644 src/test/hive/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec create mode 100644 src/test/hive/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b create mode 100644 src/test/hive/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd create mode 100644 src/test/hive/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae create mode 100644 src/test/hive/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 src/test/hive/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de create mode 100644 src/test/hive/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 src/test/hive/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea create mode 100644 src/test/hive/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 create mode 100644 src/test/hive/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 create mode 100644 src/test/hive/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 create mode 100644 src/test/hive/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 create mode 100644 src/test/hive/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 create mode 100644 src/test/hive/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 create mode 100644 src/test/hive/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 create mode 100644 src/test/hive/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 create mode 100644 src/test/hive/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa create mode 100644 src/test/hive/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 create mode 100644 src/test/hive/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 create mode 100644 src/test/hive/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 create mode 100644 src/test/hive/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 create mode 100644 src/test/hive/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 create mode 100644 src/test/hive/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af create mode 100644 src/test/hive/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f create mode 100644 src/test/hive/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 create mode 100644 src/test/hive/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 create mode 100644 src/test/hive/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 create mode 100644 src/test/hive/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 create mode 100644 src/test/hive/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb create mode 100644 src/test/hive/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f create mode 100644 src/test/hive/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 create mode 100644 src/test/hive/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af create mode 100644 src/test/hive/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b create mode 100644 src/test/hive/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e create mode 100644 src/test/hive/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e create mode 100644 src/test/hive/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 create mode 100644 src/test/hive/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 create mode 100644 src/test/hive/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 create mode 100644 src/test/hive/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 create mode 100644 src/test/hive/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 create mode 100644 src/test/hive/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee create mode 100644 src/test/hive/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 create mode 100644 src/test/hive/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 create mode 100644 src/test/hive/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 create mode 100644 src/test/hive/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 create mode 100644 src/test/hive/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 create mode 100644 src/test/hive/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd create mode 100644 src/test/hive/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d create mode 100644 src/test/hive/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 create mode 100644 src/test/hive/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a create mode 100644 src/test/hive/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d create mode 100644 src/test/hive/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 create mode 100644 src/test/hive/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 create mode 100644 src/test/hive/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee create mode 100644 src/test/hive/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 create mode 100644 src/test/hive/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 create mode 100644 src/test/hive/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a create mode 100644 src/test/hive/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 create mode 100644 src/test/hive/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b create mode 100644 src/test/hive/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 create mode 100644 src/test/hive/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 create mode 100644 src/test/hive/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e create mode 100644 src/test/hive/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b create mode 100644 src/test/hive/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 create mode 100644 src/test/hive/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 create mode 100644 src/test/hive/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b create mode 100644 src/test/hive/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da create mode 100644 src/test/hive/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb create mode 100644 src/test/hive/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 create mode 100644 src/test/hive/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 create mode 100644 src/test/hive/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 create mode 100644 src/test/hive/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 src/test/hive/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f create mode 100644 src/test/hive/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f create mode 100644 src/test/hive/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b create mode 100644 src/test/hive/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 create mode 100644 src/test/hive/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a create mode 100644 src/test/hive/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 create mode 100644 src/test/hive/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 create mode 100644 src/test/hive/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 create mode 100644 src/test/hive/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 create mode 100644 src/test/hive/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 create mode 100644 src/test/hive/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c create mode 100644 src/test/hive/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f create mode 100644 src/test/hive/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 create mode 100644 src/test/hive/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e create mode 100644 src/test/hive/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 create mode 100644 src/test/hive/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 create mode 100644 src/test/hive/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 create mode 100644 src/test/hive/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 create mode 100644 src/test/hive/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb create mode 100644 src/test/hive/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 create mode 100644 src/test/hive/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b create mode 100644 src/test/hive/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c create mode 100644 src/test/hive/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf create mode 100644 src/test/hive/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 create mode 100644 src/test/hive/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e create mode 100644 src/test/hive/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 create mode 100644 src/test/hive/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 create mode 100644 src/test/hive/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 create mode 100644 src/test/hive/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 create mode 100644 src/test/hive/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 create mode 100644 src/test/hive/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 create mode 100644 src/test/hive/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 create mode 100644 src/test/hive/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 create mode 100644 src/test/hive/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 create mode 100644 src/test/hive/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 create mode 100644 src/test/hive/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c create mode 100644 src/test/hive/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f create mode 100644 src/test/hive/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 create mode 100644 src/test/hive/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e create mode 100644 src/test/hive/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 create mode 100644 src/test/hive/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a create mode 100644 src/test/hive/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 create mode 100644 src/test/hive/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 create mode 100644 src/test/hive/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e create mode 100644 src/test/hive/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc create mode 100644 src/test/hive/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff create mode 100644 src/test/hive/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 create mode 100644 src/test/hive/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e create mode 100644 src/test/hive/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e create mode 100644 src/test/hive/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f create mode 100644 src/test/hive/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 create mode 100644 src/test/hive/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 create mode 100644 src/test/hive/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 create mode 100644 src/test/hive/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 create mode 100644 src/test/hive/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 create mode 100644 src/test/hive/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 create mode 100644 src/test/hive/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d create mode 100644 src/test/hive/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 create mode 100644 src/test/hive/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a create mode 100644 src/test/hive/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 create mode 100644 src/test/hive/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 create mode 100644 src/test/hive/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 create mode 100644 src/test/hive/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c create mode 100644 src/test/hive/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 create mode 100644 src/test/hive/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f create mode 100644 src/test/hive/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 create mode 100644 src/test/hive/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab create mode 100644 src/test/hive/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 create mode 100644 src/test/hive/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 create mode 100644 src/test/hive/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 create mode 100644 src/test/hive/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c create mode 100644 src/test/hive/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe create mode 100644 src/test/hive/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 create mode 100644 src/test/hive/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 create mode 100644 src/test/hive/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c create mode 100644 src/test/hive/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 create mode 100644 src/test/hive/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec create mode 100644 src/test/hive/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e create mode 100644 src/test/hive/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f create mode 100644 src/test/hive/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 create mode 100644 src/test/hive/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 create mode 100644 src/test/hive/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 create mode 100644 src/test/hive/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab create mode 100644 src/test/hive/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc create mode 100644 src/test/hive/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec create mode 100644 src/test/hive/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e create mode 100644 src/test/hive/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 create mode 100644 src/test/hive/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b create mode 100644 src/test/hive/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 create mode 100644 src/test/hive/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f create mode 100644 src/test/hive/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb create mode 100644 src/test/hive/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 create mode 100644 src/test/hive/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f create mode 100644 src/test/hive/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d create mode 100644 src/test/hive/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 create mode 100644 src/test/hive/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec create mode 100644 src/test/hive/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 src/test/hive/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec create mode 100644 src/test/hive/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac create mode 100644 src/test/hive/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 create mode 100644 src/test/hive/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 create mode 100644 src/test/hive/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d create mode 100644 src/test/hive/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 create mode 100644 src/test/hive/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 create mode 100644 src/test/hive/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca create mode 100644 src/test/hive/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 create mode 100644 src/test/hive/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 create mode 100644 src/test/hive/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 create mode 100644 src/test/hive/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f create mode 100644 src/test/hive/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b create mode 100644 src/test/hive/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 create mode 100644 src/test/hive/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 create mode 100644 src/test/hive/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a create mode 100644 src/test/hive/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 create mode 100644 src/test/hive/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 src/test/hive/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de create mode 100644 src/test/hive/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 src/test/hive/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 create mode 100644 src/test/hive/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 create mode 100644 src/test/hive/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 create mode 100644 src/test/hive/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 create mode 100644 src/test/hive/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d create mode 100644 src/test/hive/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 create mode 100644 src/test/hive/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a create mode 100644 src/test/hive/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a create mode 100644 src/test/hive/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 create mode 100644 src/test/hive/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e create mode 100644 src/test/hive/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 create mode 100644 src/test/hive/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 create mode 100644 src/test/hive/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 create mode 100644 src/test/hive/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 create mode 100644 src/test/hive/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 create mode 100644 src/test/hive/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 create mode 100644 src/test/hive/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff create mode 100644 src/test/hive/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec create mode 100644 src/test/hive/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e create mode 100644 src/test/hive/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 src/test/hive/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 create mode 100644 src/test/hive/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf create mode 100644 src/test/hive/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 src/test/hive/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd create mode 100644 src/test/hive/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a create mode 100644 src/test/hive/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 src/test/hive/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 create mode 100644 src/test/hive/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a create mode 100644 src/test/hive/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 src/test/hive/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab create mode 100644 src/test/hive/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 create mode 100644 src/test/hive/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 src/test/hive/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 create mode 100644 src/test/hive/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 create mode 100644 src/test/hive/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c create mode 100644 src/test/hive/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 create mode 100644 src/test/hive/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec create mode 100644 src/test/hive/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc create mode 100644 src/test/hive/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc create mode 100644 src/test/hive/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a create mode 100644 src/test/hive/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 create mode 100644 src/test/hive/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 create mode 100644 src/test/hive/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 create mode 100644 src/test/hive/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe create mode 100644 src/test/hive/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 src/test/hive/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 create mode 100644 src/test/hive/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e create mode 100644 src/test/hive/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad create mode 100644 src/test/hive/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc create mode 100644 src/test/hive/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc create mode 100644 src/test/hive/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a create mode 100644 src/test/hive/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 create mode 100644 src/test/hive/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 create mode 100644 src/test/hive/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 create mode 100644 src/test/hive/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b create mode 100644 src/test/hive/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb create mode 100644 src/test/hive/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad create mode 100644 src/test/hive/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 src/test/hive/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de create mode 100644 src/test/hive/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 src/test/hive/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 create mode 100644 src/test/hive/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 src/test/hive/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef create mode 100644 src/test/hive/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f create mode 100644 src/test/hive/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb create mode 100644 src/test/hive/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 create mode 100644 src/test/hive/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d create mode 100644 src/test/hive/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 create mode 100644 src/test/hive/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 create mode 100644 src/test/hive/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 create mode 100644 src/test/hive/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 create mode 100644 src/test/hive/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 create mode 100644 src/test/hive/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff create mode 100644 src/test/hive/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 create mode 100644 src/test/hive/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 create mode 100644 src/test/hive/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 src/test/hive/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec create mode 100644 src/test/hive/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f create mode 100644 src/test/hive/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 src/test/hive/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec create mode 100644 src/test/hive/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 create mode 100644 src/test/hive/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 create mode 100644 src/test/hive/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 create mode 100644 src/test/hive/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f create mode 100644 src/test/hive/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e create mode 100644 src/test/hive/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 create mode 100644 src/test/hive/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a create mode 100644 src/test/hive/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe create mode 100644 src/test/hive/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 create mode 100644 src/test/hive/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 create mode 100644 src/test/hive/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f create mode 100644 src/test/hive/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd create mode 100644 src/test/hive/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 create mode 100644 src/test/hive/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 create mode 100644 src/test/hive/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 create mode 100644 src/test/hive/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 create mode 100644 src/test/hive/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 src/test/hive/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de create mode 100644 src/test/hive/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 src/test/hive/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af create mode 100644 src/test/hive/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 src/test/hive/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec create mode 100644 src/test/hive/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 create mode 100644 src/test/hive/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 create mode 100644 src/test/hive/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 create mode 100644 src/test/hive/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 create mode 100644 src/test/hive/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d create mode 100644 src/test/hive/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 create mode 100644 src/test/hive/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 create mode 100644 src/test/hive/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 create mode 100644 src/test/hive/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 create mode 100644 src/test/hive/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 create mode 100644 src/test/hive/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f create mode 100644 src/test/hive/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 create mode 100644 src/test/hive/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 create mode 100644 src/test/hive/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 create mode 100644 src/test/hive/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 create mode 100644 src/test/hive/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 create mode 100644 src/test/hive/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 create mode 100644 src/test/hive/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e create mode 100644 src/test/hive/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef create mode 100644 src/test/hive/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 create mode 100644 src/test/hive/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb create mode 100644 src/test/hive/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 src/test/hive/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f create mode 100644 src/test/hive/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 create mode 100644 src/test/hive/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 create mode 100644 src/test/hive/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 create mode 100644 src/test/hive/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 create mode 100644 src/test/hive/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de create mode 100644 src/test/hive/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 src/test/hive/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db create mode 100644 src/test/hive/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 src/test/hive/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec create mode 100644 src/test/hive/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc create mode 100644 src/test/hive/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 create mode 100644 src/test/hive/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 create mode 100644 src/test/hive/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a create mode 100644 src/test/hive/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 create mode 100644 src/test/hive/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 create mode 100644 src/test/hive/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 create mode 100644 src/test/hive/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f create mode 100644 src/test/hive/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc create mode 100644 src/test/hive/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a create mode 100644 src/test/hive/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f create mode 100644 src/test/hive/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d create mode 100644 src/test/hive/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 create mode 100644 src/test/hive/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a create mode 100644 src/test/hive/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a create mode 100644 src/test/hive/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b create mode 100644 src/test/hive/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 create mode 100644 src/test/hive/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 create mode 100644 src/test/hive/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 create mode 100644 src/test/hive/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 create mode 100644 src/test/hive/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 create mode 100644 src/test/hive/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 src/test/hive/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 create mode 100644 src/test/hive/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 create mode 100644 src/test/hive/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc create mode 100644 src/test/hive/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 create mode 100644 src/test/hive/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 create mode 100644 src/test/hive/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de create mode 100644 src/test/hive/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 src/test/hive/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df create mode 100644 src/test/hive/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 create mode 100644 src/test/hive/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 create mode 100644 src/test/hive/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca create mode 100644 src/test/hive/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 create mode 100644 src/test/hive/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e create mode 100644 src/test/hive/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a create mode 100644 src/test/hive/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a create mode 100644 src/test/hive/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 create mode 100644 src/test/hive/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 create mode 100644 src/test/hive/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca create mode 100644 src/test/hive/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 create mode 100644 src/test/hive/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e create mode 100644 src/test/hive/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a create mode 100644 src/test/hive/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 create mode 100644 src/test/hive/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 create mode 100644 src/test/hive/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 create mode 100644 src/test/hive/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 create mode 100644 src/test/hive/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 create mode 100644 src/test/hive/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 create mode 100644 src/test/hive/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 create mode 100644 src/test/hive/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa create mode 100644 src/test/hive/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c create mode 100644 src/test/hive/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 create mode 100644 src/test/hive/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 create mode 100644 src/test/hive/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 create mode 100644 src/test/hive/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 create mode 100644 src/test/hive/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 create mode 100644 src/test/hive/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 create mode 100644 src/test/hive/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 create mode 100644 src/test/hive/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd create mode 100644 src/test/hive/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a create mode 100644 src/test/hive/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a create mode 100644 src/test/hive/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 create mode 100644 src/test/hive/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e create mode 100644 src/test/hive/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a create mode 100644 src/test/hive/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff create mode 100644 src/test/hive/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa create mode 100644 src/test/hive/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de create mode 100644 src/test/hive/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c create mode 100644 src/test/hive/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 create mode 100644 src/test/hive/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 create mode 100644 src/test/hive/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb create mode 100644 src/test/hive/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f create mode 100644 src/test/hive/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 src/test/hive/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 create mode 100644 src/test/hive/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 create mode 100644 src/test/hive/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b create mode 100644 src/test/hive/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 create mode 100644 src/test/hive/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 create mode 100644 src/test/hive/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 create mode 100644 src/test/hive/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b create mode 100644 src/test/hive/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 create mode 100644 src/test/hive/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 create mode 100644 src/test/hive/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 create mode 100644 src/test/hive/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b create mode 100644 src/test/hive/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c create mode 100644 src/test/hive/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 create mode 100644 src/test/hive/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 create mode 100644 src/test/hive/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b create mode 100644 src/test/hive/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c create mode 100644 src/test/hive/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 create mode 100644 src/test/hive/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e create mode 100644 src/test/hive/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a create mode 100644 src/test/hive/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c create mode 100644 src/test/hive/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 create mode 100644 src/test/hive/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f create mode 100644 src/test/hive/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a create mode 100644 src/test/hive/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 create mode 100644 src/test/hive/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 create mode 100644 src/test/hive/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 create mode 100644 src/test/hive/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a create mode 100644 src/test/hive/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 create mode 100644 src/test/hive/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b create mode 100644 src/test/hive/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c create mode 100644 src/test/hive/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 create mode 100644 src/test/hive/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a create mode 100644 src/test/hive/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 create mode 100644 src/test/hive/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c create mode 100644 src/test/hive/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 create mode 100644 src/test/hive/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a create mode 100644 src/test/hive/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 create mode 100644 src/test/hive/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a create mode 100644 src/test/hive/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 create mode 100644 src/test/hive/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a create mode 100644 src/test/hive/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 create mode 100644 src/test/hive/golden/sort-0-eee35bbc888a705ae527625447668032 create mode 100644 src/test/hive/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 src/test/hive/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf create mode 100644 src/test/hive/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec create mode 100644 src/test/hive/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 create mode 100644 src/test/hive/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c create mode 100644 src/test/hive/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 create mode 100644 src/test/hive/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 create mode 100644 src/test/hive/golden/stats0-13-ca0569fd4200d549843deadf84044649 create mode 100644 src/test/hive/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 create mode 100644 src/test/hive/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c create mode 100644 src/test/hive/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 create mode 100644 src/test/hive/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed create mode 100644 src/test/hive/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/stats0-20-5f7439f97275da70f633e135205f2095 create mode 100644 src/test/hive/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a create mode 100644 src/test/hive/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 create mode 100644 src/test/hive/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c create mode 100644 src/test/hive/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 create mode 100644 src/test/hive/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b create mode 100644 src/test/hive/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c create mode 100644 src/test/hive/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 create mode 100644 src/test/hive/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 create mode 100644 src/test/hive/golden/stats0-29-ca0569fd4200d549843deadf84044649 create mode 100644 src/test/hive/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 create mode 100644 src/test/hive/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c create mode 100644 src/test/hive/golden/stats0-4-5f7439f97275da70f633e135205f2095 create mode 100644 src/test/hive/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a create mode 100644 src/test/hive/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 create mode 100644 src/test/hive/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c create mode 100644 src/test/hive/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 create mode 100644 src/test/hive/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b create mode 100644 src/test/hive/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e create mode 100644 src/test/hive/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b create mode 100644 src/test/hive/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/stats2-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 create mode 100644 src/test/hive/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 create mode 100644 src/test/hive/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 create mode 100644 src/test/hive/golden/stats2-8-72621dba638b15d244850018e9f64d7 create mode 100644 src/test/hive/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e create mode 100644 src/test/hive/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 create mode 100644 src/test/hive/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 create mode 100644 src/test/hive/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b create mode 100644 src/test/hive/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 create mode 100644 src/test/hive/golden/stats20-5-76509775cfe11bb98ee088188a07668a create mode 100644 src/test/hive/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 create mode 100644 src/test/hive/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 create mode 100644 src/test/hive/golden/stats20-8-300c971de74642118d36d36349bc81aa create mode 100644 src/test/hive/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 create mode 100644 src/test/hive/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 create mode 100644 src/test/hive/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa create mode 100644 src/test/hive/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc create mode 100644 src/test/hive/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 create mode 100644 src/test/hive/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa create mode 100644 src/test/hive/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 create mode 100644 src/test/hive/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 create mode 100644 src/test/hive/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 create mode 100644 src/test/hive/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b create mode 100644 src/test/hive/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 create mode 100644 src/test/hive/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 create mode 100644 src/test/hive/golden/stats3-20-ca048ad81b3df7159822073d206f0790 create mode 100644 src/test/hive/golden/stats3-3-ca048ad81b3df7159822073d206f0790 create mode 100644 src/test/hive/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 create mode 100644 src/test/hive/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 create mode 100644 src/test/hive/golden/stats3-6-4bf1504274319c44d370b58092fe016c create mode 100644 src/test/hive/golden/stats3-7-73d7d55d6e5a57aacce8618902904d create mode 100644 src/test/hive/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 create mode 100644 src/test/hive/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 create mode 100644 src/test/hive/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 create mode 100644 src/test/hive/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 create mode 100644 src/test/hive/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 create mode 100644 src/test/hive/golden/stats4-12-30bc31441828a053d1a675b225a5d617 create mode 100644 src/test/hive/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 create mode 100644 src/test/hive/golden/stats4-14-9c82167763a771c175c656786d545798 create mode 100644 src/test/hive/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c create mode 100644 src/test/hive/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 create mode 100644 src/test/hive/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 create mode 100644 src/test/hive/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 create mode 100644 src/test/hive/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 create mode 100644 src/test/hive/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 create mode 100644 src/test/hive/golden/stats4-20-f63000f2c395b935199c9829964f98c1 create mode 100644 src/test/hive/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd create mode 100644 src/test/hive/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c create mode 100644 src/test/hive/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd create mode 100644 src/test/hive/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c create mode 100644 src/test/hive/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d create mode 100644 src/test/hive/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 create mode 100644 src/test/hive/golden/stats4-7-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f create mode 100644 src/test/hive/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 create mode 100644 src/test/hive/golden/stats5-3-96d9aa9c32a081518604959dcfac42df create mode 100644 src/test/hive/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 create mode 100644 src/test/hive/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 create mode 100644 src/test/hive/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/stats6-10-653f748fa2d690502ba4fda407841a20 create mode 100644 src/test/hive/golden/stats6-11-1c9f833953403596ad50fd32e513642c create mode 100644 src/test/hive/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 create mode 100644 src/test/hive/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/stats6-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/stats6-4-a88c476a632cd92f748967fadb242405 create mode 100644 src/test/hive/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc create mode 100644 src/test/hive/golden/stats6-6-8926095434b70c83bf88c70559d38dce create mode 100644 src/test/hive/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 create mode 100644 src/test/hive/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 create mode 100644 src/test/hive/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d create mode 100644 src/test/hive/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 create mode 100644 src/test/hive/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/stats7-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/stats7-4-a88c476a632cd92f748967fadb242405 create mode 100644 src/test/hive/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc create mode 100644 src/test/hive/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 create mode 100644 src/test/hive/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 create mode 100644 src/test/hive/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 create mode 100644 src/test/hive/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d create mode 100644 src/test/hive/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 create mode 100644 src/test/hive/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 create mode 100644 src/test/hive/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d create mode 100644 src/test/hive/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d create mode 100644 src/test/hive/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 create mode 100644 src/test/hive/golden/stats8-15-653f748fa2d690502ba4fda407841a20 create mode 100644 src/test/hive/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 create mode 100644 src/test/hive/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 create mode 100644 src/test/hive/golden/stats8-18-1c9f833953403596ad50fd32e513642c create mode 100644 src/test/hive/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 create mode 100644 src/test/hive/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 create mode 100644 src/test/hive/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 create mode 100644 src/test/hive/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d create mode 100644 src/test/hive/golden/stats8-23-653f748fa2d690502ba4fda407841a20 create mode 100644 src/test/hive/golden/stats8-24-1c9f833953403596ad50fd32e513642c create mode 100644 src/test/hive/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 create mode 100644 src/test/hive/golden/stats8-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/stats8-4-a88c476a632cd92f748967fadb242405 create mode 100644 src/test/hive/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc create mode 100644 src/test/hive/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 create mode 100644 src/test/hive/golden/stats8-7-8926095434b70c83bf88c70559d38dce create mode 100644 src/test/hive/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 create mode 100644 src/test/hive/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 create mode 100644 src/test/hive/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb create mode 100644 src/test/hive/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 src/test/hive/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 src/test/hive/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 src/test/hive/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 create mode 100644 src/test/hive/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 src/test/hive/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 create mode 100644 src/test/hive/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 create mode 100644 src/test/hive/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 create mode 100644 src/test/hive/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 create mode 100644 src/test/hive/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 create mode 100644 src/test/hive/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 create mode 100644 src/test/hive/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 create mode 100644 src/test/hive/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 create mode 100644 src/test/hive/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 create mode 100644 src/test/hive/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 create mode 100644 src/test/hive/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d create mode 100644 src/test/hive/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 create mode 100644 src/test/hive/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c create mode 100644 src/test/hive/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 create mode 100644 src/test/hive/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab create mode 100644 src/test/hive/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 create mode 100644 src/test/hive/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 create mode 100644 src/test/hive/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 create mode 100644 src/test/hive/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 create mode 100644 src/test/hive/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e create mode 100644 src/test/hive/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f create mode 100644 src/test/hive/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 create mode 100644 src/test/hive/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a create mode 100644 src/test/hive/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 create mode 100644 src/test/hive/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c create mode 100644 src/test/hive/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc create mode 100644 src/test/hive/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 create mode 100644 src/test/hive/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e create mode 100644 src/test/hive/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae create mode 100644 src/test/hive/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 create mode 100644 src/test/hive/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 create mode 100644 src/test/hive/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 create mode 100644 src/test/hive/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef create mode 100644 src/test/hive/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 create mode 100644 src/test/hive/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c create mode 100644 src/test/hive/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 create mode 100644 src/test/hive/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 create mode 100644 src/test/hive/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 src/test/hive/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 src/test/hive/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 src/test/hive/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 src/test/hive/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 create mode 100644 src/test/hive/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 create mode 100644 src/test/hive/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 src/test/hive/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 create mode 100644 src/test/hive/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 create mode 100644 src/test/hive/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef create mode 100644 src/test/hive/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 create mode 100644 src/test/hive/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c create mode 100644 src/test/hive/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 create mode 100644 src/test/hive/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 create mode 100644 src/test/hive/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 src/test/hive/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 src/test/hive/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 src/test/hive/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 src/test/hive/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca create mode 100644 src/test/hive/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 create mode 100644 src/test/hive/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 create mode 100644 src/test/hive/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 create mode 100644 src/test/hive/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 src/test/hive/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 create mode 100644 src/test/hive/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 src/test/hive/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 src/test/hive/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 create mode 100644 src/test/hive/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 src/test/hive/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 src/test/hive/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 create mode 100644 src/test/hive/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 create mode 100644 src/test/hive/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 create mode 100644 src/test/hive/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 create mode 100644 src/test/hive/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 src/test/hive/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 src/test/hive/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b create mode 100644 src/test/hive/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 src/test/hive/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 create mode 100644 src/test/hive/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c create mode 100644 src/test/hive/golden/str_to_map-10-32997010bba305ec40812df254490730 create mode 100644 src/test/hive/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a create mode 100644 src/test/hive/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e create mode 100644 src/test/hive/golden/str_to_map-13-32997010bba305ec40812df254490730 create mode 100644 src/test/hive/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec create mode 100644 src/test/hive/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 create mode 100644 src/test/hive/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 create mode 100644 src/test/hive/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda create mode 100644 src/test/hive/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf create mode 100644 src/test/hive/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad create mode 100644 src/test/hive/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 create mode 100644 src/test/hive/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd create mode 100644 src/test/hive/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 create mode 100644 src/test/hive/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 create mode 100644 src/test/hive/golden/subq2-1-235919a7ddb574662158503b8052e7ca create mode 100644 src/test/hive/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 create mode 100644 src/test/hive/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e create mode 100644 src/test/hive/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 create mode 100644 src/test/hive/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 create mode 100644 src/test/hive/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d create mode 100644 src/test/hive/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 create mode 100644 src/test/hive/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 create mode 100644 src/test/hive/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c create mode 100644 src/test/hive/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 create mode 100644 src/test/hive/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 create mode 100644 src/test/hive/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 create mode 100644 src/test/hive/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 create mode 100644 src/test/hive/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 create mode 100644 src/test/hive/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 create mode 100644 src/test/hive/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 create mode 100644 src/test/hive/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 create mode 100644 src/test/hive/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b create mode 100644 src/test/hive/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 create mode 100644 src/test/hive/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 create mode 100644 src/test/hive/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc create mode 100644 src/test/hive/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 create mode 100644 src/test/hive/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb create mode 100644 src/test/hive/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 create mode 100644 src/test/hive/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af create mode 100644 src/test/hive/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d create mode 100644 src/test/hive/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 create mode 100644 src/test/hive/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f create mode 100644 src/test/hive/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 create mode 100644 src/test/hive/golden/touch-1-27ad2962fed131f51ba802596ba37278 create mode 100644 src/test/hive/golden/touch-10-27ad2962fed131f51ba802596ba37278 create mode 100644 src/test/hive/golden/touch-2-3e95421993ab28d18245ec2340f580a3 create mode 100644 src/test/hive/golden/touch-3-c0c18ac884677231a41eea8d980d0451 create mode 100644 src/test/hive/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a create mode 100644 src/test/hive/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c create mode 100644 src/test/hive/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 create mode 100644 src/test/hive/golden/touch-7-3752320b12abae0d138148d56a27c5b1 create mode 100644 src/test/hive/golden/touch-8-3562ab40242756452595cd7eae79b0ce create mode 100644 src/test/hive/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 create mode 100644 src/test/hive/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe create mode 100644 src/test/hive/golden/transform1-0-b6919fc48901e388c869c84ae0211102 create mode 100644 src/test/hive/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 create mode 100644 src/test/hive/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 create mode 100644 src/test/hive/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 create mode 100644 src/test/hive/golden/transform1-4-65527bae8e73262255ef83082c6968f9 create mode 100644 src/test/hive/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 create mode 100644 src/test/hive/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f create mode 100644 src/test/hive/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 create mode 100644 src/test/hive/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 create mode 100644 src/test/hive/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 src/test/hive/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 create mode 100644 src/test/hive/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 create mode 100644 src/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d create mode 100644 src/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 create mode 100644 src/test/hive/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 create mode 100644 src/test/hive/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 create mode 100644 src/test/hive/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 create mode 100644 src/test/hive/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 create mode 100644 src/test/hive/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e create mode 100644 src/test/hive/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 create mode 100644 src/test/hive/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 src/test/hive/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 src/test/hive/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 src/test/hive/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 src/test/hive/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 src/test/hive/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 src/test/hive/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 src/test/hive/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 src/test/hive/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 src/test/hive/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 src/test/hive/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 create mode 100644 src/test/hive/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 create mode 100644 src/test/hive/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 create mode 100644 src/test/hive/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 create mode 100644 src/test/hive/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e create mode 100644 src/test/hive/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a create mode 100644 src/test/hive/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb create mode 100644 src/test/hive/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 create mode 100644 src/test/hive/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 create mode 100644 src/test/hive/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 create mode 100644 src/test/hive/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 create mode 100644 src/test/hive/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 create mode 100644 src/test/hive/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 create mode 100644 src/test/hive/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 create mode 100644 src/test/hive/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 create mode 100644 src/test/hive/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 create mode 100644 src/test/hive/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad create mode 100644 src/test/hive/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd create mode 100644 src/test/hive/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b create mode 100644 src/test/hive/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae create mode 100644 src/test/hive/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f create mode 100644 src/test/hive/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 create mode 100644 src/test/hive/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 create mode 100644 src/test/hive/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 create mode 100644 src/test/hive/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 create mode 100644 src/test/hive/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 create mode 100644 src/test/hive/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf create mode 100644 src/test/hive/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c create mode 100644 src/test/hive/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b create mode 100644 src/test/hive/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 create mode 100644 src/test/hive/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 create mode 100644 src/test/hive/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a create mode 100644 src/test/hive/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 create mode 100644 src/test/hive/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 create mode 100644 src/test/hive/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 create mode 100644 src/test/hive/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 create mode 100644 src/test/hive/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 create mode 100644 src/test/hive/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 create mode 100644 src/test/hive/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f create mode 100644 src/test/hive/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f create mode 100644 src/test/hive/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 create mode 100644 src/test/hive/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 create mode 100644 src/test/hive/golden/udf1-1-8281592c818ada269024ac669bec78da create mode 100644 src/test/hive/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f create mode 100644 src/test/hive/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 create mode 100644 src/test/hive/golden/udf2-2-31c8af064bac42541558a95fad3bca97 create mode 100644 src/test/hive/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 create mode 100644 src/test/hive/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c create mode 100644 src/test/hive/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 create mode 100644 src/test/hive/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e create mode 100644 src/test/hive/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 create mode 100644 src/test/hive/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 create mode 100644 src/test/hive/golden/udf6-3-e579646b969eef49b09656114da52a73 create mode 100644 src/test/hive/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 create mode 100644 src/test/hive/golden/udf6-5-fe336cd9850d6357980bd19139f76e create mode 100644 src/test/hive/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 create mode 100644 src/test/hive/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 create mode 100644 src/test/hive/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 create mode 100644 src/test/hive/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/udf8-1-63c38297946a2060c0cff4a426b0520c create mode 100644 src/test/hive/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 create mode 100644 src/test/hive/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb create mode 100644 src/test/hive/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b create mode 100644 src/test/hive/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 create mode 100644 src/test/hive/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c create mode 100644 src/test/hive/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e create mode 100644 src/test/hive/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 create mode 100644 src/test/hive/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f create mode 100644 src/test/hive/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c create mode 100644 src/test/hive/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a create mode 100644 src/test/hive/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 create mode 100644 src/test/hive/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f create mode 100644 src/test/hive/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c create mode 100644 src/test/hive/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a create mode 100644 src/test/hive/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 create mode 100644 src/test/hive/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f create mode 100644 src/test/hive/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 create mode 100644 src/test/hive/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 create mode 100644 src/test/hive/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 create mode 100644 src/test/hive/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f create mode 100644 src/test/hive/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 create mode 100644 src/test/hive/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 create mode 100644 src/test/hive/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 create mode 100644 src/test/hive/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 create mode 100644 src/test/hive/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 create mode 100644 src/test/hive/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 create mode 100644 src/test/hive/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 create mode 100644 src/test/hive/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c create mode 100644 src/test/hive/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 create mode 100644 src/test/hive/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 create mode 100644 src/test/hive/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f create mode 100644 src/test/hive/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b create mode 100644 src/test/hive/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 create mode 100644 src/test/hive/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a create mode 100644 src/test/hive/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e create mode 100644 src/test/hive/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 create mode 100644 src/test/hive/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 create mode 100644 src/test/hive/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee create mode 100644 src/test/hive/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a create mode 100644 src/test/hive/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 create mode 100644 src/test/hive/golden/udf_array-3-a5d12c41277fb158e09281169c905122 create mode 100644 src/test/hive/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a create mode 100644 src/test/hive/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c create mode 100644 src/test/hive/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 create mode 100644 src/test/hive/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 create mode 100644 src/test/hive/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c create mode 100644 src/test/hive/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 create mode 100644 src/test/hive/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa create mode 100644 src/test/hive/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 create mode 100644 src/test/hive/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd create mode 100644 src/test/hive/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 create mode 100644 src/test/hive/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f create mode 100644 src/test/hive/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 create mode 100644 src/test/hive/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 create mode 100644 src/test/hive/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 create mode 100644 src/test/hive/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b create mode 100644 src/test/hive/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 create mode 100644 src/test/hive/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 create mode 100644 src/test/hive/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 create mode 100644 src/test/hive/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b create mode 100644 src/test/hive/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 create mode 100644 src/test/hive/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 create mode 100644 src/test/hive/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 create mode 100644 src/test/hive/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 create mode 100644 src/test/hive/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f create mode 100644 src/test/hive/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 create mode 100644 src/test/hive/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 create mode 100644 src/test/hive/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b create mode 100644 src/test/hive/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 create mode 100644 src/test/hive/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b create mode 100644 src/test/hive/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 create mode 100644 src/test/hive/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d create mode 100644 src/test/hive/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 create mode 100644 src/test/hive/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b create mode 100644 src/test/hive/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 create mode 100644 src/test/hive/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 create mode 100644 src/test/hive/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 create mode 100644 src/test/hive/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b create mode 100644 src/test/hive/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 create mode 100644 src/test/hive/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df create mode 100644 src/test/hive/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c create mode 100644 src/test/hive/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef create mode 100644 src/test/hive/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b create mode 100644 src/test/hive/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 create mode 100644 src/test/hive/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 create mode 100644 src/test/hive/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 create mode 100644 src/test/hive/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b create mode 100644 src/test/hive/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a create mode 100644 src/test/hive/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b create mode 100644 src/test/hive/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f create mode 100644 src/test/hive/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d create mode 100644 src/test/hive/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 create mode 100644 src/test/hive/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb create mode 100644 src/test/hive/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 create mode 100644 src/test/hive/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f create mode 100644 src/test/hive/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 create mode 100644 src/test/hive/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e create mode 100644 src/test/hive/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 create mode 100644 src/test/hive/golden/udf_case-1-54acf006155d8822a50e47729be24004 create mode 100644 src/test/hive/golden/udf_case-2-98ee676f92950375917f09d2e492253f create mode 100644 src/test/hive/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c create mode 100644 src/test/hive/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 create mode 100644 src/test/hive/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c create mode 100644 src/test/hive/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 create mode 100644 src/test/hive/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 create mode 100644 src/test/hive/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 create mode 100644 src/test/hive/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb create mode 100644 src/test/hive/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd create mode 100644 src/test/hive/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df create mode 100644 src/test/hive/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 create mode 100644 src/test/hive/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 create mode 100644 src/test/hive/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 create mode 100644 src/test/hive/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 create mode 100644 src/test/hive/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 create mode 100644 src/test/hive/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 create mode 100644 src/test/hive/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 create mode 100644 src/test/hive/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 create mode 100644 src/test/hive/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 create mode 100644 src/test/hive/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 create mode 100644 src/test/hive/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 src/test/hive/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 create mode 100644 src/test/hive/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea create mode 100644 src/test/hive/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 create mode 100644 src/test/hive/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 create mode 100644 src/test/hive/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab create mode 100644 src/test/hive/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c create mode 100644 src/test/hive/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 create mode 100644 src/test/hive/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f create mode 100644 src/test/hive/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 create mode 100644 src/test/hive/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 create mode 100644 src/test/hive/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c create mode 100644 src/test/hive/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 create mode 100644 src/test/hive/golden/udf_conv-0-d552befca345f3396464529cfde9f75a create mode 100644 src/test/hive/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab create mode 100644 src/test/hive/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 create mode 100644 src/test/hive/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 create mode 100644 src/test/hive/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d create mode 100644 src/test/hive/golden/udf_conv-4-568e843076f358c404a8634b18541c55 create mode 100644 src/test/hive/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 create mode 100644 src/test/hive/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a create mode 100644 src/test/hive/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e create mode 100644 src/test/hive/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb create mode 100644 src/test/hive/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 create mode 100644 src/test/hive/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 create mode 100644 src/test/hive/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 create mode 100644 src/test/hive/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 create mode 100644 src/test/hive/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 create mode 100644 src/test/hive/golden/udf_count-0-534a9b25b87d09e418645b1216949560 create mode 100644 src/test/hive/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f create mode 100644 src/test/hive/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 create mode 100644 src/test/hive/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e create mode 100644 src/test/hive/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e create mode 100644 src/test/hive/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 create mode 100644 src/test/hive/golden/udf_count-2-3344645abe535426307b9327b381fc85 create mode 100644 src/test/hive/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 create mode 100644 src/test/hive/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 create mode 100644 src/test/hive/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 create mode 100644 src/test/hive/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 create mode 100644 src/test/hive/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 create mode 100644 src/test/hive/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 create mode 100644 src/test/hive/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 create mode 100644 src/test/hive/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 create mode 100644 src/test/hive/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 create mode 100644 src/test/hive/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 create mode 100644 src/test/hive/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 create mode 100644 src/test/hive/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 create mode 100644 src/test/hive/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 create mode 100644 src/test/hive/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 create mode 100644 src/test/hive/golden/udf_day-1-87168babe1110fe4c38269843414ca4 create mode 100644 src/test/hive/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 create mode 100644 src/test/hive/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 create mode 100644 src/test/hive/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae create mode 100644 src/test/hive/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 create mode 100644 src/test/hive/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b create mode 100644 src/test/hive/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 create mode 100644 src/test/hive/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae create mode 100644 src/test/hive/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 create mode 100644 src/test/hive/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b create mode 100644 src/test/hive/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 create mode 100644 src/test/hive/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db create mode 100644 src/test/hive/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 create mode 100644 src/test/hive/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b create mode 100644 src/test/hive/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 create mode 100644 src/test/hive/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 create mode 100644 src/test/hive/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 create mode 100644 src/test/hive/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 create mode 100644 src/test/hive/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 create mode 100644 src/test/hive/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f create mode 100644 src/test/hive/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 create mode 100644 src/test/hive/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a create mode 100644 src/test/hive/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 create mode 100644 src/test/hive/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a create mode 100644 src/test/hive/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 create mode 100644 src/test/hive/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 create mode 100644 src/test/hive/golden/udf_field-1-379d8580693493f620a8f4084709324f create mode 100644 src/test/hive/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 create mode 100644 src/test/hive/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b create mode 100644 src/test/hive/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 create mode 100644 src/test/hive/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 create mode 100644 src/test/hive/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 create mode 100644 src/test/hive/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 create mode 100644 src/test/hive/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef create mode 100644 src/test/hive/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 create mode 100644 src/test/hive/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 create mode 100644 src/test/hive/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 create mode 100644 src/test/hive/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 create mode 100644 src/test/hive/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c create mode 100644 src/test/hive/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a create mode 100644 src/test/hive/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 create mode 100644 src/test/hive/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a create mode 100644 src/test/hive/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 create mode 100644 src/test/hive/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb create mode 100644 src/test/hive/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 create mode 100644 src/test/hive/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 create mode 100644 src/test/hive/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 create mode 100644 src/test/hive/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e create mode 100644 src/test/hive/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b create mode 100644 src/test/hive/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f create mode 100644 src/test/hive/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 create mode 100644 src/test/hive/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 create mode 100644 src/test/hive/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 create mode 100644 src/test/hive/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 create mode 100644 src/test/hive/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 create mode 100644 src/test/hive/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b create mode 100644 src/test/hive/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 create mode 100644 src/test/hive/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 create mode 100644 src/test/hive/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 src/test/hive/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 create mode 100644 src/test/hive/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd create mode 100644 src/test/hive/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c create mode 100644 src/test/hive/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 create mode 100644 src/test/hive/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 create mode 100644 src/test/hive/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 create mode 100644 src/test/hive/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 create mode 100644 src/test/hive/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e create mode 100644 src/test/hive/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 create mode 100644 src/test/hive/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f create mode 100644 src/test/hive/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 create mode 100644 src/test/hive/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f create mode 100644 src/test/hive/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 create mode 100644 src/test/hive/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 create mode 100644 src/test/hive/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 create mode 100644 src/test/hive/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 create mode 100644 src/test/hive/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 create mode 100644 src/test/hive/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c create mode 100644 src/test/hive/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f create mode 100644 src/test/hive/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a create mode 100644 src/test/hive/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 create mode 100644 src/test/hive/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 create mode 100644 src/test/hive/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 create mode 100644 src/test/hive/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a create mode 100644 src/test/hive/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 create mode 100644 src/test/hive/golden/udf_hash-3-23991312391d518aacf3d4469c816eae create mode 100644 src/test/hive/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 create mode 100644 src/test/hive/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c create mode 100644 src/test/hive/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 create mode 100644 src/test/hive/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf create mode 100644 src/test/hive/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea create mode 100644 src/test/hive/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce create mode 100644 src/test/hive/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 create mode 100644 src/test/hive/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a create mode 100644 src/test/hive/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 create mode 100644 src/test/hive/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c create mode 100644 src/test/hive/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a create mode 100644 src/test/hive/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 create mode 100644 src/test/hive/golden/udf_if-3-20206f17367ff284d67044abd745ce9f create mode 100644 src/test/hive/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca create mode 100644 src/test/hive/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc create mode 100644 src/test/hive/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 create mode 100644 src/test/hive/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 create mode 100644 src/test/hive/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 create mode 100644 src/test/hive/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 create mode 100644 src/test/hive/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 create mode 100644 src/test/hive/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 create mode 100644 src/test/hive/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 create mode 100644 src/test/hive/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 create mode 100644 src/test/hive/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 create mode 100644 src/test/hive/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 create mode 100644 src/test/hive/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba create mode 100644 src/test/hive/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 create mode 100644 src/test/hive/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 create mode 100644 src/test/hive/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 create mode 100644 src/test/hive/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 create mode 100644 src/test/hive/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 create mode 100644 src/test/hive/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 create mode 100644 src/test/hive/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 create mode 100644 src/test/hive/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 create mode 100644 src/test/hive/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 create mode 100644 src/test/hive/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 create mode 100644 src/test/hive/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 create mode 100644 src/test/hive/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 create mode 100644 src/test/hive/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b create mode 100644 src/test/hive/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 create mode 100644 src/test/hive/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a create mode 100644 src/test/hive/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 create mode 100644 src/test/hive/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd create mode 100644 src/test/hive/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad create mode 100644 src/test/hive/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e create mode 100644 src/test/hive/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b create mode 100644 src/test/hive/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 create mode 100644 src/test/hive/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 create mode 100644 src/test/hive/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 create mode 100644 src/test/hive/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d create mode 100644 src/test/hive/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 create mode 100644 src/test/hive/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 src/test/hive/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a create mode 100644 src/test/hive/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f create mode 100644 src/test/hive/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf create mode 100644 src/test/hive/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 create mode 100644 src/test/hive/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 create mode 100644 src/test/hive/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 create mode 100644 src/test/hive/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 create mode 100644 src/test/hive/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b create mode 100644 src/test/hive/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 create mode 100644 src/test/hive/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c create mode 100644 src/test/hive/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 create mode 100644 src/test/hive/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a create mode 100644 src/test/hive/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 create mode 100644 src/test/hive/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc create mode 100644 src/test/hive/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 create mode 100644 src/test/hive/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 create mode 100644 src/test/hive/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 create mode 100644 src/test/hive/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e create mode 100644 src/test/hive/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 create mode 100644 src/test/hive/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c create mode 100644 src/test/hive/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 create mode 100644 src/test/hive/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a create mode 100644 src/test/hive/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 create mode 100644 src/test/hive/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f create mode 100644 src/test/hive/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 create mode 100644 src/test/hive/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 create mode 100644 src/test/hive/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 create mode 100644 src/test/hive/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 create mode 100644 src/test/hive/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 create mode 100644 src/test/hive/golden/udf_lower-1-550f0a6da388596a775d921b9da995c create mode 100644 src/test/hive/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e create mode 100644 src/test/hive/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 create mode 100644 src/test/hive/golden/udf_lpad-0-995646acf1e23cea7825412915921bef create mode 100644 src/test/hive/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 create mode 100644 src/test/hive/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 create mode 100644 src/test/hive/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae create mode 100644 src/test/hive/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f create mode 100644 src/test/hive/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 create mode 100644 src/test/hive/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e create mode 100644 src/test/hive/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf create mode 100644 src/test/hive/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 create mode 100644 src/test/hive/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 create mode 100644 src/test/hive/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 src/test/hive/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa create mode 100644 src/test/hive/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f create mode 100644 src/test/hive/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e create mode 100644 src/test/hive/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a create mode 100644 src/test/hive/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 src/test/hive/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 create mode 100644 src/test/hive/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 create mode 100644 src/test/hive/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 create mode 100644 src/test/hive/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e create mode 100644 src/test/hive/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f create mode 100644 src/test/hive/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 create mode 100644 src/test/hive/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 create mode 100644 src/test/hive/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 src/test/hive/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 create mode 100644 src/test/hive/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 src/test/hive/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 src/test/hive/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 create mode 100644 src/test/hive/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 src/test/hive/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 create mode 100644 src/test/hive/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 src/test/hive/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 src/test/hive/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 create mode 100644 src/test/hive/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 create mode 100644 src/test/hive/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 create mode 100644 src/test/hive/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 src/test/hive/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 create mode 100644 src/test/hive/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 src/test/hive/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 src/test/hive/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 create mode 100644 src/test/hive/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 src/test/hive/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 create mode 100644 src/test/hive/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 src/test/hive/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 src/test/hive/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee create mode 100644 src/test/hive/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 create mode 100644 src/test/hive/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 create mode 100644 src/test/hive/golden/udf_minute-3-270055c684846e87444b037226cf554c create mode 100644 src/test/hive/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 create mode 100644 src/test/hive/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 create mode 100644 src/test/hive/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee create mode 100644 src/test/hive/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 create mode 100644 src/test/hive/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 create mode 100644 src/test/hive/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf create mode 100644 src/test/hive/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c create mode 100644 src/test/hive/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 create mode 100644 src/test/hive/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c create mode 100644 src/test/hive/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e create mode 100644 src/test/hive/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 create mode 100644 src/test/hive/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b create mode 100644 src/test/hive/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 create mode 100644 src/test/hive/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa create mode 100644 src/test/hive/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc create mode 100644 src/test/hive/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 create mode 100644 src/test/hive/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 create mode 100644 src/test/hive/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a create mode 100644 src/test/hive/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e create mode 100644 src/test/hive/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a create mode 100644 src/test/hive/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 create mode 100644 src/test/hive/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 create mode 100644 src/test/hive/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 create mode 100644 src/test/hive/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 create mode 100644 src/test/hive/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 create mode 100644 src/test/hive/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd create mode 100644 src/test/hive/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 create mode 100644 src/test/hive/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 create mode 100644 src/test/hive/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 create mode 100644 src/test/hive/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def create mode 100644 src/test/hive/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 create mode 100644 src/test/hive/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 create mode 100644 src/test/hive/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 create mode 100644 src/test/hive/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 create mode 100644 src/test/hive/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab create mode 100644 src/test/hive/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d create mode 100644 src/test/hive/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 create mode 100644 src/test/hive/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 create mode 100644 src/test/hive/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a create mode 100644 src/test/hive/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 src/test/hive/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 src/test/hive/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 create mode 100644 src/test/hive/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae create mode 100644 src/test/hive/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 create mode 100644 src/test/hive/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 create mode 100644 src/test/hive/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 create mode 100644 src/test/hive/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 create mode 100644 src/test/hive/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 create mode 100644 src/test/hive/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 create mode 100644 src/test/hive/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 create mode 100644 src/test/hive/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd create mode 100644 src/test/hive/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 create mode 100644 src/test/hive/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 create mode 100644 src/test/hive/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c create mode 100644 src/test/hive/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 create mode 100644 src/test/hive/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 create mode 100644 src/test/hive/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 create mode 100644 src/test/hive/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 create mode 100644 src/test/hive/golden/udf_power-0-57001d802c281743322d28bbc520cd4 create mode 100644 src/test/hive/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 create mode 100644 src/test/hive/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 create mode 100644 src/test/hive/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 create mode 100644 src/test/hive/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 create mode 100644 src/test/hive/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 create mode 100644 src/test/hive/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 create mode 100644 src/test/hive/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 create mode 100644 src/test/hive/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 create mode 100644 src/test/hive/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 create mode 100644 src/test/hive/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 create mode 100644 src/test/hive/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 create mode 100644 src/test/hive/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 create mode 100644 src/test/hive/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e create mode 100644 src/test/hive/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 create mode 100644 src/test/hive/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee create mode 100644 src/test/hive/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 create mode 100644 src/test/hive/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 create mode 100644 src/test/hive/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 create mode 100644 src/test/hive/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca create mode 100644 src/test/hive/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 create mode 100644 src/test/hive/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 create mode 100644 src/test/hive/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 create mode 100644 src/test/hive/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 create mode 100644 src/test/hive/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb create mode 100644 src/test/hive/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e create mode 100644 src/test/hive/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc create mode 100644 src/test/hive/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b create mode 100644 src/test/hive/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b create mode 100644 src/test/hive/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a create mode 100644 src/test/hive/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 create mode 100644 src/test/hive/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 create mode 100644 src/test/hive/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca create mode 100644 src/test/hive/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 create mode 100644 src/test/hive/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 create mode 100644 src/test/hive/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 create mode 100644 src/test/hive/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c create mode 100644 src/test/hive/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 create mode 100644 src/test/hive/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a create mode 100644 src/test/hive/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b create mode 100644 src/test/hive/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 create mode 100644 src/test/hive/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce create mode 100644 src/test/hive/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 create mode 100644 src/test/hive/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 create mode 100644 src/test/hive/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 create mode 100644 src/test/hive/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b create mode 100644 src/test/hive/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c create mode 100644 src/test/hive/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 create mode 100644 src/test/hive/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 create mode 100644 src/test/hive/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 create mode 100644 src/test/hive/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 create mode 100644 src/test/hive/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c create mode 100644 src/test/hive/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 create mode 100644 src/test/hive/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 create mode 100644 src/test/hive/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 create mode 100644 src/test/hive/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a create mode 100644 src/test/hive/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd create mode 100644 src/test/hive/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 create mode 100644 src/test/hive/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 create mode 100644 src/test/hive/golden/udf_second-2-d678372e3837a16be245d2e33482f17f create mode 100644 src/test/hive/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf create mode 100644 src/test/hive/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f create mode 100644 src/test/hive/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 create mode 100644 src/test/hive/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 create mode 100644 src/test/hive/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 create mode 100644 src/test/hive/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 create mode 100644 src/test/hive/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 create mode 100644 src/test/hive/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 create mode 100644 src/test/hive/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 create mode 100644 src/test/hive/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 create mode 100644 src/test/hive/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de create mode 100644 src/test/hive/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f create mode 100644 src/test/hive/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 create mode 100644 src/test/hive/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 create mode 100644 src/test/hive/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e create mode 100644 src/test/hive/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 create mode 100644 src/test/hive/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa create mode 100644 src/test/hive/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a create mode 100644 src/test/hive/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef create mode 100644 src/test/hive/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d create mode 100644 src/test/hive/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa create mode 100644 src/test/hive/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a create mode 100644 src/test/hive/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e create mode 100644 src/test/hive/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 create mode 100644 src/test/hive/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef create mode 100644 src/test/hive/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d create mode 100644 src/test/hive/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee create mode 100644 src/test/hive/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 create mode 100644 src/test/hive/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab create mode 100644 src/test/hive/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 create mode 100644 src/test/hive/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 create mode 100644 src/test/hive/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 create mode 100644 src/test/hive/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 create mode 100644 src/test/hive/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd create mode 100644 src/test/hive/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 create mode 100644 src/test/hive/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 create mode 100644 src/test/hive/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 src/test/hive/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 create mode 100644 src/test/hive/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 create mode 100644 src/test/hive/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de create mode 100644 src/test/hive/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 create mode 100644 src/test/hive/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e create mode 100644 src/test/hive/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 create mode 100644 src/test/hive/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 create mode 100644 src/test/hive/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 create mode 100644 src/test/hive/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a create mode 100644 src/test/hive/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 create mode 100644 src/test/hive/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 create mode 100644 src/test/hive/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 create mode 100644 src/test/hive/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 create mode 100644 src/test/hive/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 create mode 100644 src/test/hive/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 create mode 100644 src/test/hive/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 create mode 100644 src/test/hive/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 create mode 100644 src/test/hive/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 create mode 100644 src/test/hive/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 create mode 100644 src/test/hive/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 create mode 100644 src/test/hive/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b create mode 100644 src/test/hive/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 create mode 100644 src/test/hive/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 create mode 100644 src/test/hive/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d create mode 100644 src/test/hive/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 create mode 100644 src/test/hive/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 create mode 100644 src/test/hive/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 create mode 100644 src/test/hive/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 create mode 100644 src/test/hive/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 create mode 100644 src/test/hive/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 create mode 100644 src/test/hive/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 create mode 100644 src/test/hive/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d create mode 100644 src/test/hive/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd create mode 100644 src/test/hive/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 create mode 100644 src/test/hive/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 create mode 100644 src/test/hive/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 create mode 100644 src/test/hive/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a create mode 100644 src/test/hive/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 create mode 100644 src/test/hive/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 create mode 100644 src/test/hive/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 create mode 100644 src/test/hive/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece create mode 100644 src/test/hive/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 create mode 100644 src/test/hive/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 create mode 100644 src/test/hive/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 create mode 100644 src/test/hive/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 create mode 100644 src/test/hive/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b create mode 100644 src/test/hive/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 create mode 100644 src/test/hive/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b create mode 100644 src/test/hive/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 create mode 100644 src/test/hive/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 create mode 100644 src/test/hive/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 create mode 100644 src/test/hive/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 create mode 100644 src/test/hive/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 create mode 100644 src/test/hive/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 create mode 100644 src/test/hive/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 create mode 100644 src/test/hive/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 create mode 100644 src/test/hive/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad create mode 100644 src/test/hive/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 create mode 100644 src/test/hive/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d create mode 100644 src/test/hive/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 create mode 100644 src/test/hive/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 create mode 100644 src/test/hive/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 create mode 100644 src/test/hive/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 create mode 100644 src/test/hive/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c create mode 100644 src/test/hive/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d create mode 100644 src/test/hive/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad create mode 100644 src/test/hive/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f create mode 100644 src/test/hive/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a create mode 100644 src/test/hive/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 create mode 100644 src/test/hive/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 create mode 100644 src/test/hive/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc create mode 100644 src/test/hive/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b create mode 100644 src/test/hive/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 create mode 100644 src/test/hive/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 create mode 100644 src/test/hive/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 create mode 100644 src/test/hive/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 create mode 100644 src/test/hive/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 create mode 100644 src/test/hive/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 create mode 100644 src/test/hive/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e create mode 100644 src/test/hive/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 create mode 100644 src/test/hive/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a create mode 100644 src/test/hive/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe create mode 100644 src/test/hive/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f create mode 100644 src/test/hive/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 create mode 100644 src/test/hive/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 create mode 100644 src/test/hive/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 create mode 100644 src/test/hive/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 create mode 100644 src/test/hive/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 create mode 100644 src/test/hive/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 create mode 100644 src/test/hive/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c create mode 100644 src/test/hive/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 create mode 100644 src/test/hive/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce create mode 100644 src/test/hive/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 create mode 100644 src/test/hive/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 create mode 100644 src/test/hive/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 create mode 100644 src/test/hive/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 create mode 100644 src/test/hive/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c create mode 100644 src/test/hive/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a create mode 100644 src/test/hive/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 create mode 100644 src/test/hive/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 create mode 100644 src/test/hive/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 create mode 100644 src/test/hive/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 create mode 100644 src/test/hive/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d create mode 100644 src/test/hive/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 create mode 100644 src/test/hive/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e create mode 100644 src/test/hive/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 create mode 100644 src/test/hive/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a create mode 100644 src/test/hive/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b create mode 100644 src/test/hive/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 create mode 100644 src/test/hive/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce create mode 100644 src/test/hive/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 create mode 100644 src/test/hive/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 create mode 100644 src/test/hive/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 create mode 100644 src/test/hive/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d create mode 100644 src/test/hive/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 create mode 100644 src/test/hive/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc create mode 100644 src/test/hive/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 create mode 100644 src/test/hive/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 create mode 100644 src/test/hive/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 create mode 100644 src/test/hive/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 create mode 100644 src/test/hive/golden/udf_to_string-5-1040b37847d20ef29d545934316303 create mode 100644 src/test/hive/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 create mode 100644 src/test/hive/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 create mode 100644 src/test/hive/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea create mode 100644 src/test/hive/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b create mode 100644 src/test/hive/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 create mode 100644 src/test/hive/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 create mode 100644 src/test/hive/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 create mode 100644 src/test/hive/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b create mode 100644 src/test/hive/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 create mode 100644 src/test/hive/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 create mode 100644 src/test/hive/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a create mode 100644 src/test/hive/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba create mode 100644 src/test/hive/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 create mode 100644 src/test/hive/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b create mode 100644 src/test/hive/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 create mode 100644 src/test/hive/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 create mode 100644 src/test/hive/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 create mode 100644 src/test/hive/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f create mode 100644 src/test/hive/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 create mode 100644 src/test/hive/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d create mode 100644 src/test/hive/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f create mode 100644 src/test/hive/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 create mode 100644 src/test/hive/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 create mode 100644 src/test/hive/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 create mode 100644 src/test/hive/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 create mode 100644 src/test/hive/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c create mode 100644 src/test/hive/golden/udf_union-1-f6f01250718087029165e23badc02cd6 create mode 100644 src/test/hive/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 create mode 100644 src/test/hive/golden/udf_union-3-705d165fec6761744dd19b142c566d61 create mode 100644 src/test/hive/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead create mode 100644 src/test/hive/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d create mode 100644 src/test/hive/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de create mode 100644 src/test/hive/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 create mode 100644 src/test/hive/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 create mode 100644 src/test/hive/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b create mode 100644 src/test/hive/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 create mode 100644 src/test/hive/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b create mode 100644 src/test/hive/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 create mode 100644 src/test/hive/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a create mode 100644 src/test/hive/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de create mode 100644 src/test/hive/golden/udf_variance-3-fd25e5226312bf54d918858511814766 create mode 100644 src/test/hive/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 create mode 100644 src/test/hive/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a create mode 100644 src/test/hive/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de create mode 100644 src/test/hive/golden/udf_variance-7-fd25e5226312bf54d918858511814766 create mode 100644 src/test/hive/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 create mode 100644 src/test/hive/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd create mode 100644 src/test/hive/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 create mode 100644 src/test/hive/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 create mode 100644 src/test/hive/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 create mode 100644 src/test/hive/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 create mode 100644 src/test/hive/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f create mode 100644 src/test/hive/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 create mode 100644 src/test/hive/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 create mode 100644 src/test/hive/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d create mode 100644 src/test/hive/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea create mode 100644 src/test/hive/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a create mode 100644 src/test/hive/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 create mode 100644 src/test/hive/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f create mode 100644 src/test/hive/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 create mode 100644 src/test/hive/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 create mode 100644 src/test/hive/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b create mode 100644 src/test/hive/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 create mode 100644 src/test/hive/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 create mode 100644 src/test/hive/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 create mode 100644 src/test/hive/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f create mode 100644 src/test/hive/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 create mode 100644 src/test/hive/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 create mode 100644 src/test/hive/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 create mode 100644 src/test/hive/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 create mode 100644 src/test/hive/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 create mode 100644 src/test/hive/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 create mode 100644 src/test/hive/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 create mode 100644 src/test/hive/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c create mode 100644 src/test/hive/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 create mode 100644 src/test/hive/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 create mode 100644 src/test/hive/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c create mode 100644 src/test/hive/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d create mode 100644 src/test/hive/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e create mode 100644 src/test/hive/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 create mode 100644 src/test/hive/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a create mode 100644 src/test/hive/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 create mode 100644 src/test/hive/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 create mode 100644 src/test/hive/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 create mode 100644 src/test/hive/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f create mode 100644 src/test/hive/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf create mode 100644 src/test/hive/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa create mode 100644 src/test/hive/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 create mode 100644 src/test/hive/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 create mode 100644 src/test/hive/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 create mode 100644 src/test/hive/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 create mode 100644 src/test/hive/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba create mode 100644 src/test/hive/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba create mode 100644 src/test/hive/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 create mode 100644 src/test/hive/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 create mode 100644 src/test/hive/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a create mode 100644 src/test/hive/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 create mode 100644 src/test/hive/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 create mode 100644 src/test/hive/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 create mode 100644 src/test/hive/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b create mode 100644 src/test/hive/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 create mode 100644 src/test/hive/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b create mode 100644 src/test/hive/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d create mode 100644 src/test/hive/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 create mode 100644 src/test/hive/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d create mode 100644 src/test/hive/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 create mode 100644 src/test/hive/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 create mode 100644 src/test/hive/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 create mode 100644 src/test/hive/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 create mode 100644 src/test/hive/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 create mode 100644 src/test/hive/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 create mode 100644 src/test/hive/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 create mode 100644 src/test/hive/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 create mode 100644 src/test/hive/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c create mode 100644 src/test/hive/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e create mode 100644 src/test/hive/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 create mode 100644 src/test/hive/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d create mode 100644 src/test/hive/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 create mode 100644 src/test/hive/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e create mode 100644 src/test/hive/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 create mode 100644 src/test/hive/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 create mode 100644 src/test/hive/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f create mode 100644 src/test/hive/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a create mode 100644 src/test/hive/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a create mode 100644 src/test/hive/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 create mode 100644 src/test/hive/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 create mode 100644 src/test/hive/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f create mode 100644 src/test/hive/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc create mode 100644 src/test/hive/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 create mode 100644 src/test/hive/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a create mode 100644 src/test/hive/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 create mode 100644 src/test/hive/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 create mode 100644 src/test/hive/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 create mode 100644 src/test/hive/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 create mode 100644 src/test/hive/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 create mode 100644 src/test/hive/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e create mode 100644 src/test/hive/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 create mode 100644 src/test/hive/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 create mode 100644 src/test/hive/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 create mode 100644 src/test/hive/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 create mode 100644 src/test/hive/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 create mode 100644 src/test/hive/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af create mode 100644 src/test/hive/golden/union10-2-60ee9eae687170317ff91dafe6b799bf create mode 100644 src/test/hive/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 create mode 100644 src/test/hive/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 create mode 100644 src/test/hive/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 create mode 100644 src/test/hive/golden/union11-2-cd756f39e22e121bdbd51400662aa47f create mode 100644 src/test/hive/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 create mode 100644 src/test/hive/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 create mode 100644 src/test/hive/golden/union13-0-6970d1e2405f3769a28a8804887ac657 create mode 100644 src/test/hive/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 create mode 100644 src/test/hive/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/union14-1-11603fb6225423979af6c062cfa9389b create mode 100644 src/test/hive/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 create mode 100644 src/test/hive/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 create mode 100644 src/test/hive/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 create mode 100644 src/test/hive/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 create mode 100644 src/test/hive/golden/union16-1-9f76074598f9b55d8afbb5659737a382 create mode 100644 src/test/hive/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a create mode 100644 src/test/hive/golden/union17-1-9491c271b7e2c351ddcf126f73679892 create mode 100644 src/test/hive/golden/union17-2-e6fb104913b9da8193167ee263993dd1 create mode 100644 src/test/hive/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 create mode 100644 src/test/hive/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 src/test/hive/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 src/test/hive/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a create mode 100644 src/test/hive/golden/union18-1-9491c271b7e2c351ddcf126f73679892 create mode 100644 src/test/hive/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 create mode 100644 src/test/hive/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 create mode 100644 src/test/hive/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 create mode 100644 src/test/hive/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 create mode 100644 src/test/hive/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a create mode 100644 src/test/hive/golden/union19-1-9491c271b7e2c351ddcf126f73679892 create mode 100644 src/test/hive/golden/union19-2-b6161b64e57a2502c79662866782cbdf create mode 100644 src/test/hive/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 create mode 100644 src/test/hive/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 create mode 100644 src/test/hive/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 create mode 100644 src/test/hive/golden/union2-0-58813daf3d1af57b17518b606f8345b2 create mode 100644 src/test/hive/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 create mode 100644 src/test/hive/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d create mode 100644 src/test/hive/golden/union20-1-968e353589f1fddb914242beb25be94c create mode 100644 src/test/hive/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c create mode 100644 src/test/hive/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 create mode 100644 src/test/hive/golden/union22-0-4bd6583b3635f1db95765565518bda8d create mode 100644 src/test/hive/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 create mode 100644 src/test/hive/golden/union22-10-90e9c4388340428a1f68868e9322c400 create mode 100644 src/test/hive/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 create mode 100644 src/test/hive/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 create mode 100644 src/test/hive/golden/union22-4-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union22-5-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 src/test/hive/golden/union22-7-25fc734982956a164adde6bb1d4d8751 create mode 100644 src/test/hive/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 create mode 100644 src/test/hive/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 create mode 100644 src/test/hive/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 create mode 100644 src/test/hive/golden/union23-1-7830963417e3535034962e2597970ddd create mode 100644 src/test/hive/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 create mode 100644 src/test/hive/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c create mode 100644 src/test/hive/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 create mode 100644 src/test/hive/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a create mode 100644 src/test/hive/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 create mode 100644 src/test/hive/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f create mode 100644 src/test/hive/golden/union24-4-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 create mode 100644 src/test/hive/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 create mode 100644 src/test/hive/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb create mode 100644 src/test/hive/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 create mode 100644 src/test/hive/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d create mode 100644 src/test/hive/golden/union26-1-4702087f160230463e6114d6534d47e4 create mode 100644 src/test/hive/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 create mode 100644 src/test/hive/golden/union26-3-4702087f160230463e6114d6534d47e4 create mode 100644 src/test/hive/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 create mode 100644 src/test/hive/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f create mode 100644 src/test/hive/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 create mode 100644 src/test/hive/golden/union27-3-ab84df3813ff23be99f148449610e530 create mode 100644 src/test/hive/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 create mode 100644 src/test/hive/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 create mode 100644 src/test/hive/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f create mode 100644 src/test/hive/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 create mode 100644 src/test/hive/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 create mode 100644 src/test/hive/golden/union29-1-a40036f756a306f4226634b6ab67c72c create mode 100644 src/test/hive/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 create mode 100644 src/test/hive/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 create mode 100644 src/test/hive/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d create mode 100644 src/test/hive/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 create mode 100644 src/test/hive/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 create mode 100644 src/test/hive/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 create mode 100644 src/test/hive/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 create mode 100644 src/test/hive/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 create mode 100644 src/test/hive/golden/union30-2-252f12f7532ca67132bfc62063c58430 create mode 100644 src/test/hive/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 create mode 100644 src/test/hive/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 create mode 100644 src/test/hive/golden/union31-1-b12aa87aaf64b573ce0be7013117651b create mode 100644 src/test/hive/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 create mode 100644 src/test/hive/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 create mode 100644 src/test/hive/golden/union31-12-de2477a134ab37bb22d879028becf2e5 create mode 100644 src/test/hive/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 create mode 100644 src/test/hive/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 create mode 100644 src/test/hive/golden/union31-15-5df6435aed6e0a6a6853480a027b911e create mode 100644 src/test/hive/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 create mode 100644 src/test/hive/golden/union31-17-b12aa87aaf64b573ce0be7013117651b create mode 100644 src/test/hive/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 create mode 100644 src/test/hive/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e create mode 100644 src/test/hive/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 create mode 100644 src/test/hive/golden/union31-20-dceb6d9a03096d68d090557692ed6899 create mode 100644 src/test/hive/golden/union31-21-bda2e62d0ecd30200b2359df420da574 create mode 100644 src/test/hive/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a create mode 100644 src/test/hive/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df create mode 100644 src/test/hive/golden/union31-24-df38c8164af7cc164c728b8178da72c5 create mode 100644 src/test/hive/golden/union31-25-1485e295a99908e1862eae397b814045 create mode 100644 src/test/hive/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc create mode 100644 src/test/hive/golden/union31-4-77653483e5ce18437c9f85d3b297335d create mode 100644 src/test/hive/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 create mode 100644 src/test/hive/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 create mode 100644 src/test/hive/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 create mode 100644 src/test/hive/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb create mode 100644 src/test/hive/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 create mode 100644 src/test/hive/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e create mode 100644 src/test/hive/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 create mode 100644 src/test/hive/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b create mode 100644 src/test/hive/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 create mode 100644 src/test/hive/golden/union32-2-8e39fef33c859ef83912d0fcda319218 create mode 100644 src/test/hive/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 create mode 100644 src/test/hive/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e create mode 100644 src/test/hive/golden/union32-5-51c997d0a1103b60764bbb8316a38746 create mode 100644 src/test/hive/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b create mode 100644 src/test/hive/golden/union32-7-f936440d63f4e1027dda4de605660633 create mode 100644 src/test/hive/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb create mode 100644 src/test/hive/golden/union32-9-74fb695786df4c024288ae23ac8c00db create mode 100644 src/test/hive/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f create mode 100644 src/test/hive/golden/union33-1-1df7c476abb48234c839878414720873 create mode 100644 src/test/hive/golden/union33-2-3987150ad66482f5186826efcfb117f7 create mode 100644 src/test/hive/golden/union33-3-bfe75625806b414a5098af487b91046a create mode 100644 src/test/hive/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 create mode 100644 src/test/hive/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 create mode 100644 src/test/hive/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 create mode 100644 src/test/hive/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 create mode 100644 src/test/hive/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 create mode 100644 src/test/hive/golden/union34-1-320c2fe802fab386957039aaff6399f1 create mode 100644 src/test/hive/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 create mode 100644 src/test/hive/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 create mode 100644 src/test/hive/golden/union34-3-c179e14d840249d6953cb552279822f1 create mode 100644 src/test/hive/golden/union34-4-101829a66cab2efd31dcb0d86e302956 create mode 100644 src/test/hive/golden/union34-5-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f create mode 100644 src/test/hive/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 create mode 100644 src/test/hive/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b create mode 100644 src/test/hive/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 create mode 100644 src/test/hive/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/union4-1-88deb3987655dc41989f39c0827cec74 create mode 100644 src/test/hive/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 create mode 100644 src/test/hive/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b create mode 100644 src/test/hive/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 create mode 100644 src/test/hive/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 create mode 100644 src/test/hive/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 create mode 100644 src/test/hive/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d create mode 100644 src/test/hive/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 create mode 100644 src/test/hive/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 create mode 100644 src/test/hive/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 create mode 100644 src/test/hive/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 src/test/hive/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 create mode 100644 src/test/hive/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 create mode 100644 src/test/hive/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e create mode 100644 src/test/hive/golden/union8-1-1b422e4c1c8b97775518f760b995c771 create mode 100644 src/test/hive/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 create mode 100644 src/test/hive/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 create mode 100644 src/test/hive/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 create mode 100644 src/test/hive/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c create mode 100644 src/test/hive/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 create mode 100644 src/test/hive/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 create mode 100644 src/test/hive/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c create mode 100644 src/test/hive/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d create mode 100644 src/test/hive/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 create mode 100644 src/test/hive/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 create mode 100644 src/test/hive/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 create mode 100644 src/test/hive/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 create mode 100644 src/test/hive/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a create mode 100644 src/test/hive/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b create mode 100644 src/test/hive/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed create mode 100644 src/test/hive/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 create mode 100644 src/test/hive/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 create mode 100644 src/test/hive/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a create mode 100644 src/test/hive/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 create mode 100644 src/test/hive/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 create mode 100644 src/test/hive/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 create mode 100644 src/test/hive/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc create mode 100644 src/test/hive/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 create mode 100644 src/test/hive/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 src/test/hive/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 create mode 100644 src/test/hive/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a create mode 100644 src/test/hive/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e create mode 100644 src/test/hive/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc create mode 100644 src/test/hive/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 create mode 100644 src/test/hive/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 src/test/hive/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c create mode 100644 src/test/hive/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 create mode 100644 src/test/hive/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc create mode 100644 src/test/hive/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 create mode 100644 src/test/hive/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 src/test/hive/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 create mode 100644 src/test/hive/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af create mode 100644 src/test/hive/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc create mode 100644 src/test/hive/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb create mode 100644 src/test/hive/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 src/test/hive/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c create mode 100644 src/test/hive/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 create mode 100644 src/test/hive/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 create mode 100644 src/test/hive/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc create mode 100644 src/test/hive/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d create mode 100644 src/test/hive/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 src/test/hive/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 create mode 100644 src/test/hive/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 create mode 100644 src/test/hive/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 create mode 100644 src/test/hive/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d create mode 100644 src/test/hive/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef create mode 100644 src/test/hive/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef create mode 100644 src/test/hive/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 create mode 100644 src/test/hive/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 create mode 100644 src/test/hive/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 create mode 100644 src/test/hive/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc create mode 100644 src/test/hive/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a create mode 100644 src/test/hive/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 create mode 100644 src/test/hive/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d create mode 100644 src/test/hive/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 create mode 100644 src/test/hive/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 create mode 100644 src/test/hive/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a create mode 100644 src/test/hive/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b create mode 100644 src/test/hive/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef create mode 100644 src/test/hive/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad create mode 100644 src/test/hive/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 create mode 100644 src/test/hive/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 create mode 100644 src/test/hive/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe create mode 100644 src/test/hive/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 src/test/hive/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 create mode 100644 src/test/hive/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 create mode 100644 src/test/hive/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b create mode 100644 src/test/hive/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 create mode 100644 src/test/hive/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 create mode 100644 src/test/hive/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 create mode 100644 src/test/hive/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 create mode 100644 src/test/hive/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e create mode 100644 src/test/hive/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 create mode 100644 src/test/hive/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 create mode 100644 src/test/hive/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a create mode 100644 src/test/hive/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 create mode 100644 src/test/hive/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 create mode 100644 src/test/hive/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e create mode 100644 src/test/hive/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a create mode 100644 src/test/hive/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f create mode 100644 src/test/hive/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 create mode 100644 src/test/hive/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e create mode 100644 src/test/hive/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f create mode 100644 src/test/hive/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa create mode 100644 src/test/hive/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 create mode 100644 src/test/hive/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f create mode 100644 src/test/hive/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 create mode 100644 src/test/hive/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 create mode 100644 src/test/hive/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 create mode 100644 src/test/hive/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 create mode 100644 src/test/hive/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd create mode 100644 src/test/hive/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af create mode 100644 src/test/hive/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 create mode 100644 src/test/hive/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 create mode 100644 src/test/hive/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb create mode 100644 src/test/hive/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 create mode 100644 src/test/hive/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 create mode 100644 src/test/hive/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a create mode 100644 src/test/hive/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 create mode 100644 src/test/hive/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 create mode 100644 src/test/hive/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 create mode 100644 src/test/hive/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e create mode 100644 src/test/hive/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f create mode 100644 src/test/hive/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 create mode 100644 src/test/hive/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e create mode 100644 src/test/hive/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 create mode 100644 src/test/hive/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a create mode 100644 src/test/hive/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 create mode 100644 src/test/hive/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 create mode 100644 src/test/hive/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 create mode 100644 src/test/hive/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc create mode 100644 src/test/hive/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead create mode 100644 src/test/hive/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a create mode 100644 src/test/hive/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f create mode 100644 src/test/hive/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 create mode 100644 src/test/hive/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc create mode 100644 src/test/hive/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 create mode 100644 src/test/hive/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a create mode 100644 src/test/hive/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 create mode 100644 src/test/hive/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 create mode 100644 src/test/hive/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e create mode 100644 src/test/hive/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 create mode 100644 src/test/hive/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 create mode 100644 src/test/hive/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a create mode 100644 src/test/hive/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf create mode 100644 src/test/hive/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 create mode 100644 src/test/hive/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca create mode 100644 src/test/hive/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 src/test/hive/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 create mode 100644 src/test/hive/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 create mode 100644 src/test/hive/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 src/test/hive/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 src/test/hive/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee create mode 100644 src/test/hive/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 src/test/hive/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f create mode 100644 src/test/hive/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 src/test/hive/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 src/test/hive/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 create mode 100644 src/test/hive/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c create mode 100644 src/test/hive/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 src/test/hive/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a create mode 100644 src/test/hive/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 create mode 100644 src/test/hive/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 src/test/hive/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 src/test/hive/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc create mode 100644 src/test/hive/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 create mode 100644 src/test/hive/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 create mode 100644 src/test/hive/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 src/test/hive/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 src/test/hive/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 create mode 100644 src/test/hive/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 create mode 100644 src/test/hive/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d create mode 100644 src/test/hive/golden/union_view-1-c790d4344144460224b0f02be7e137a8 create mode 100644 src/test/hive/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 create mode 100644 src/test/hive/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 create mode 100644 src/test/hive/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e create mode 100644 src/test/hive/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 create mode 100644 src/test/hive/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae create mode 100644 src/test/hive/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 create mode 100644 src/test/hive/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 create mode 100644 src/test/hive/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af create mode 100644 src/test/hive/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 create mode 100644 src/test/hive/golden/varchar_2-2-779a555944479991520be53f7f247626 create mode 100644 src/test/hive/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da create mode 100644 src/test/hive/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f create mode 100644 src/test/hive/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 create mode 100644 src/test/hive/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f create mode 100644 src/test/hive/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af create mode 100644 src/test/hive/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e create mode 100644 src/test/hive/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 create mode 100644 src/test/hive/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce create mode 100644 src/test/hive/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c create mode 100644 src/test/hive/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e create mode 100644 src/test/hive/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 create mode 100644 src/test/hive/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 create mode 100644 src/test/hive/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 create mode 100644 src/test/hive/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 create mode 100644 src/test/hive/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 create mode 100644 src/test/hive/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 create mode 100644 src/test/hive/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef create mode 100644 src/test/hive/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 create mode 100644 src/test/hive/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 create mode 100644 src/test/hive/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 create mode 100644 src/test/hive/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc create mode 100644 src/test/hive/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 create mode 100644 src/test/hive/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 create mode 100644 src/test/hive/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 create mode 100644 src/test/hive/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f create mode 100644 src/test/hive/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 create mode 100644 src/test/hive/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 create mode 100644 src/test/hive/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 create mode 100644 src/test/hive/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 create mode 100644 src/test/hive/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc create mode 100644 src/test/hive/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 create mode 100644 src/test/hive/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 create mode 100644 src/test/hive/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb create mode 100644 src/test/hive/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 create mode 100644 src/test/hive/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad create mode 100644 src/test/hive/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 create mode 100644 src/test/hive/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 create mode 100644 src/test/hive/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 create mode 100644 src/test/hive/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb create mode 100644 src/test/hive/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc create mode 100644 src/test/hive/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 create mode 100644 src/test/hive/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb create mode 100644 src/test/hive/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 create mode 100644 src/test/hive/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 create mode 100644 src/test/hive/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 create mode 100644 src/test/hive/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 create mode 100644 src/test/hive/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 create mode 100644 src/test/hive/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 create mode 100644 src/test/hive/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b create mode 100644 src/test/hive/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 create mode 100644 src/test/hive/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 create mode 100644 src/test/hive/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 create mode 100644 src/test/hive/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 create mode 100644 src/test/hive/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d create mode 100644 src/test/hive/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 create mode 100644 src/test/hive/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e create mode 100644 src/test/hive/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 create mode 100644 src/test/hive/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e create mode 100644 src/test/hive/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 create mode 100644 src/test/hive/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 create mode 100644 src/test/hive/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 create mode 100644 src/test/hive/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a create mode 100644 src/test/hive/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d create mode 100644 src/test/hive/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad create mode 100644 src/test/hive/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e create mode 100644 src/test/hive/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 create mode 100644 src/test/hive/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b create mode 100644 src/test/hive/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 create mode 100644 src/test/hive/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 create mode 100644 src/test/hive/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 create mode 100644 src/test/hive/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc create mode 100644 src/test/hive/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f create mode 100644 src/test/hive/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 create mode 100644 src/test/hive/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e create mode 100644 src/test/hive/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c create mode 100644 src/test/hive/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 create mode 100644 src/test/hive/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a create mode 100644 src/test/hive/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 create mode 100644 src/test/hive/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb create mode 100644 src/test/hive/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 create mode 100644 src/test/hive/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 create mode 100644 src/test/hive/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 create mode 100644 src/test/hive/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d create mode 100644 src/test/hive/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e create mode 100644 src/test/hive/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 create mode 100644 src/test/hive/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f create mode 100644 src/test/hive/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f create mode 100644 src/test/hive/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba create mode 100644 src/test/hive/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba create mode 100644 src/test/hive/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 create mode 100644 src/test/hive/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 create mode 100644 src/test/hive/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 create mode 100644 src/test/hive/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 create mode 100644 src/test/hive/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e create mode 100644 src/test/hive/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec create mode 100644 src/test/hive/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 create mode 100644 src/test/hive/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 create mode 100644 src/test/hive/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e create mode 100644 src/test/hive/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec create mode 100644 src/test/hive/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 create mode 100644 src/test/hive/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 create mode 100644 src/test/hive/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca create mode 100644 src/test/hive/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b create mode 100644 src/test/hive/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 create mode 100644 src/test/hive/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 create mode 100644 src/test/hive/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 create mode 100644 src/test/hive/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 create mode 100644 src/test/hive/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 create mode 100644 src/test/hive/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a create mode 100644 src/test/hive/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 create mode 100644 src/test/hive/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 create mode 100644 src/test/hive/golden/view-11-dc95343d3e57846485dd543476391376 create mode 100644 src/test/hive/golden/view-12-371764e1cae31ea0518c03060528d239 create mode 100644 src/test/hive/golden/view-13-2abce88008f8a19164758ee821aaa8a6 create mode 100644 src/test/hive/golden/view-14-deb504f4f70fd7db975950c3c47959ee create mode 100644 src/test/hive/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b create mode 100644 src/test/hive/golden/view-2-9c529f486fa81a032bfe1253808fca8 create mode 100644 src/test/hive/golden/view-3-89c80c0e90409d5e304775c9f420915a create mode 100644 src/test/hive/golden/view-4-4a64d1a623ca71e515796787dbd0f904 create mode 100644 src/test/hive/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b create mode 100644 src/test/hive/golden/view-6-47b5043f03a84695b6784682b4402ac8 create mode 100644 src/test/hive/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 create mode 100644 src/test/hive/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 create mode 100644 src/test/hive/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 create mode 100644 src/test/hive/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf create mode 100644 src/test/hive/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb create mode 100644 src/test/hive/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 create mode 100644 src/test/hive/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de create mode 100644 src/test/hive/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 create mode 100644 src/test/hive/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 create mode 100644 src/test/hive/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a create mode 100644 src/test/hive/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 create mode 100644 src/test/hive/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb create mode 100644 src/test/hive/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 create mode 100644 src/test/hive/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 create mode 100644 src/test/hive/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 create mode 100644 src/test/hive/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe create mode 100644 src/test/hive/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d create mode 100644 src/test/hive/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 create mode 100644 src/test/hive/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 create mode 100644 src/test/hive/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 create mode 100644 src/test/hive/golden/virtual_column-3-c66776673c986b59b27e704664935988 create mode 100644 src/test/hive/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 create mode 100644 src/test/hive/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 create mode 100644 src/test/hive/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 create mode 100644 src/test/hive/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 create mode 100644 src/test/hive/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 create mode 100644 src/test/hive/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 diff --git a/src/test/hive/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb b/src/test/hive/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/'1' + 1-0-130514c6116c311d808590a075b187b b/src/test/hive/golden/'1' + 1-0-130514c6116c311d808590a075b187b new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/'1' + 1-0-130514c6116c311d808590a075b187b @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e b/src/test/hive/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 b/src/test/hive/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 b/src/test/hive/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a b/src/test/hive/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a b/src/test/hive/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 b/src/test/hive/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 b/src/test/hive/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e b/src/test/hive/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e b/src/test/hive/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f b/src/test/hive/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 b/src/test/hive/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 b/src/test/hive/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d b/src/test/hive/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 b/src/test/hive/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 b/src/test/hive/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 b/src/test/hive/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a b/src/test/hive/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d b/src/test/hive/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 b/src/test/hive/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f b/src/test/hive/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 b/src/test/hive/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 b/src/test/hive/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 b/src/test/hive/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 b/src/test/hive/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e b/src/test/hive/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 b/src/test/hive/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e b/src/test/hive/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b b/src/test/hive/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb b/src/test/hive/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1Y + 1-0-a4541db51882b19503649138fbb295f b/src/test/hive/golden/1Y + 1-0-a4541db51882b19503649138fbb295f new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1Y + 1-0-a4541db51882b19503649138fbb295f @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d b/src/test/hive/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/src/test/hive/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/src/test/hive/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 b/src/test/hive/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 b/src/test/hive/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 b/src/test/hive/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a b/src/test/hive/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a new file mode 100644 index 0000000000000..5710fb29e7e02 --- /dev/null +++ b/src/test/hive/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a @@ -0,0 +1,309 @@ +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 +34 val_34 +35 val_35 +37 val_37 +41 val_41 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +69 val_69 +70 val_70 +72 val_72 +74 val_74 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 +120 val_120 +125 val_125 +126 val_126 +128 val_128 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +136 val_136 +137 val_137 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +149 val_149 +150 val_150 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +165 val_165 +166 val_166 +167 val_167 +168 val_168 +169 val_169 +170 val_170 +172 val_172 +174 val_174 +175 val_175 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +192 val_192 +193 val_193 +194 val_194 +195 val_195 +196 val_196 +197 val_197 +199 val_199 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +205 val_205 +207 val_207 +208 val_208 +209 val_209 +213 val_213 +214 val_214 +216 val_216 +217 val_217 +218 val_218 +219 val_219 +221 val_221 +222 val_222 +223 val_223 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +230 val_230 +233 val_233 +235 val_235 +237 val_237 +238 val_238 +239 val_239 +241 val_241 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +266 val_266 +272 val_272 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +278 val_278 +280 val_280 +281 val_281 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +308 val_308 +309 val_309 +310 val_310 +311 val_311 +315 val_315 +316 val_316 +317 val_317 +318 val_318 +321 val_321 +322 val_322 +323 val_323 +325 val_325 +327 val_327 +331 val_331 +332 val_332 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +344 val_344 +345 val_345 +348 val_348 +351 val_351 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +368 val_368 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +396 val_396 +397 val_397 +399 val_399 +400 val_400 +401 val_401 +402 val_402 +403 val_403 +404 val_404 +406 val_406 +407 val_407 +409 val_409 +411 val_411 +413 val_413 +414 val_414 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +427 val_427 +429 val_429 +430 val_430 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +459 val_459 +460 val_460 +462 val_462 +463 val_463 +466 val_466 +467 val_467 +468 val_468 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +479 val_479 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 diff --git a/src/test/hive/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 b/src/test/hive/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 new file mode 100644 index 0000000000000..bbe37f8e2a790 --- /dev/null +++ b/src/test/hive/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 @@ -0,0 +1 @@ +86 \ \ \ No newline at end of file diff --git a/src/test/hive/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 b/src/test/hive/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 b/src/test/hive/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 new file mode 100644 index 0000000000000..8007988316af0 --- /dev/null +++ b/src/test/hive/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 @@ -0,0 +1,175 @@ +311 val_311 +165 val_165 +193 val_193 +401 val_401 +150 val_150 +128 val_128 +213 val_213 +146 val_146 +152 val_152 +145 val_145 +281 val_281 +15 val_15 +166 val_166 +417 val_417 +219 val_219 +153 val_153 +193 val_193 +174 val_174 +413 val_413 +199 val_199 +174 val_174 +417 val_417 +162 val_162 +167 val_167 +195 val_195 +17 val_17 +113 val_113 +155 val_155 +128 val_128 +311 val_311 +316 val_316 +149 val_149 +129 val_129 +170 val_170 +157 val_157 +221 val_221 +111 val_111 +169 val_169 +125 val_125 +192 val_192 +187 val_187 +176 val_176 +51 val_51 +138 val_138 +103 val_103 +213 val_213 +216 val_216 +176 val_176 +221 val_221 +318 val_318 +311 val_311 +137 val_137 +241 val_241 +180 val_180 +12 val_12 +181 val_181 +138 val_138 +217 val_217 +411 val_411 +431 val_431 +179 val_179 +172 val_172 +129 val_129 +158 val_158 +119 val_119 +197 val_197 +100 val_100 +199 val_199 +191 val_191 +418 val_418 +165 val_165 +120 val_120 +131 val_131 +51 val_51 +156 val_156 +196 val_196 +481 val_481 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +137 val_137 +316 val_316 +169 val_169 +413 val_413 +179 val_179 +118 val_118 +134 val_134 +138 val_138 +419 val_419 +15 val_15 +118 val_118 +19 val_19 +10 val_10 +331 val_331 +401 val_401 +177 val_177 +317 val_317 +11 val_11 +168 val_168 +143 val_143 +160 val_160 +195 val_195 +321 val_321 +119 val_119 +41 val_41 +149 val_149 +218 val_218 +138 val_138 +103 val_103 +113 val_113 +216 val_216 +167 val_167 +219 val_219 +116 val_116 +401 val_401 +191 val_191 +128 val_128 +316 val_316 +331 val_331 +193 val_193 +321 val_321 +104 val_104 +175 val_175 +105 val_105 +190 val_190 +401 val_401 +114 val_114 +12 val_12 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +125 val_125 +431 val_431 +164 val_164 +187 val_187 +291 val_291 +351 val_351 +104 val_104 +163 val_163 +119 val_119 +414 val_414 +491 val_491 +417 val_417 +199 val_199 +120 val_120 +169 val_169 +178 val_178 +310 val_310 +317 val_317 +136 val_136 +172 val_172 +214 val_214 +133 val_133 +175 val_175 +189 val_189 +401 val_401 +421 val_421 +134 val_134 +18 val_18 +100 val_100 +341 val_341 +146 val_146 +186 val_186 +167 val_167 +18 val_18 +183 val_183 +281 val_281 +315 val_315 +152 val_152 +194 val_194 +414 val_414 +126 val_126 +169 val_169 \ No newline at end of file diff --git a/src/test/hive/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 b/src/test/hive/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/src/test/hive/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/src/test/hive/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 b/src/test/hive/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 new file mode 100644 index 0000000000000..ee2a405fb6ab8 --- /dev/null +++ b/src/test/hive/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 @@ -0,0 +1,8 @@ +The Eleventh Hour 3 April 2010 11 +The Doctor's Wife 14 May 2011 11 +Horror of Fang Rock 3 September 1977 4 +An Unearthly Child 23 November 1963 1 +The Mysterious Planet 6 September 1986 6 +Rose 26 March 2005 9 +The Power of the Daleks 5 November 1966 2 +Castrolava 4 January 1982 5 diff --git a/src/test/hive/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa b/src/test/hive/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa new file mode 100644 index 0000000000000..6407b72051b51 --- /dev/null +++ b/src/test/hive/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa @@ -0,0 +1,2 @@ +Joe 2 +Hank 2 diff --git a/src/test/hive/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 b/src/test/hive/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 new file mode 100644 index 0000000000000..595d7af04a69a --- /dev/null +++ b/src/test/hive/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 @@ -0,0 +1,309 @@ +1.0 val_0 +11.0 val_10 +101.0 val_100 +104.0 val_103 +105.0 val_104 +106.0 val_105 +12.0 val_11 +112.0 val_111 +114.0 val_113 +115.0 val_114 +117.0 val_116 +119.0 val_118 +120.0 val_119 +13.0 val_12 +121.0 val_120 +126.0 val_125 +127.0 val_126 +129.0 val_128 +130.0 val_129 +132.0 val_131 +134.0 val_133 +135.0 val_134 +137.0 val_136 +138.0 val_137 +139.0 val_138 +144.0 val_143 +146.0 val_145 +147.0 val_146 +150.0 val_149 +16.0 val_15 +151.0 val_150 +153.0 val_152 +154.0 val_153 +156.0 val_155 +157.0 val_156 +158.0 val_157 +159.0 val_158 +161.0 val_160 +163.0 val_162 +164.0 val_163 +165.0 val_164 +166.0 val_165 +167.0 val_166 +168.0 val_167 +169.0 val_168 +170.0 val_169 +18.0 val_17 +171.0 val_170 +173.0 val_172 +175.0 val_174 +176.0 val_175 +177.0 val_176 +178.0 val_177 +179.0 val_178 +180.0 val_179 +19.0 val_18 +181.0 val_180 +182.0 val_181 +184.0 val_183 +187.0 val_186 +188.0 val_187 +190.0 val_189 +20.0 val_19 +191.0 val_190 +192.0 val_191 +193.0 val_192 +194.0 val_193 +195.0 val_194 +196.0 val_195 +197.0 val_196 +198.0 val_197 +200.0 val_199 +3.0 val_2 +21.0 val_20 +201.0 val_200 +202.0 val_201 +203.0 val_202 +204.0 val_203 +206.0 val_205 +208.0 val_207 +209.0 val_208 +210.0 val_209 +214.0 val_213 +215.0 val_214 +217.0 val_216 +218.0 val_217 +219.0 val_218 +220.0 val_219 +222.0 val_221 +223.0 val_222 +224.0 val_223 +225.0 val_224 +227.0 val_226 +229.0 val_228 +230.0 val_229 +231.0 val_230 +234.0 val_233 +236.0 val_235 +238.0 val_237 +239.0 val_238 +240.0 val_239 +25.0 val_24 +242.0 val_241 +243.0 val_242 +245.0 val_244 +248.0 val_247 +249.0 val_248 +250.0 val_249 +253.0 val_252 +256.0 val_255 +257.0 val_256 +258.0 val_257 +259.0 val_258 +27.0 val_26 +261.0 val_260 +263.0 val_262 +264.0 val_263 +266.0 val_265 +267.0 val_266 +28.0 val_27 +273.0 val_272 +274.0 val_273 +275.0 val_274 +276.0 val_275 +278.0 val_277 +279.0 val_278 +29.0 val_28 +281.0 val_280 +282.0 val_281 +283.0 val_282 +284.0 val_283 +285.0 val_284 +286.0 val_285 +287.0 val_286 +288.0 val_287 +289.0 val_288 +290.0 val_289 +292.0 val_291 +293.0 val_292 +297.0 val_296 +299.0 val_298 +31.0 val_30 +303.0 val_302 +306.0 val_305 +307.0 val_306 +308.0 val_307 +309.0 val_308 +310.0 val_309 +311.0 val_310 +312.0 val_311 +316.0 val_315 +317.0 val_316 +318.0 val_317 +319.0 val_318 +322.0 val_321 +323.0 val_322 +324.0 val_323 +326.0 val_325 +328.0 val_327 +34.0 val_33 +332.0 val_331 +333.0 val_332 +334.0 val_333 +336.0 val_335 +337.0 val_336 +339.0 val_338 +340.0 val_339 +35.0 val_34 +342.0 val_341 +343.0 val_342 +345.0 val_344 +346.0 val_345 +349.0 val_348 +36.0 val_35 +352.0 val_351 +354.0 val_353 +357.0 val_356 +361.0 val_360 +363.0 val_362 +365.0 val_364 +366.0 val_365 +367.0 val_366 +368.0 val_367 +369.0 val_368 +370.0 val_369 +38.0 val_37 +374.0 val_373 +375.0 val_374 +376.0 val_375 +378.0 val_377 +379.0 val_378 +380.0 val_379 +383.0 val_382 +385.0 val_384 +387.0 val_386 +390.0 val_389 +393.0 val_392 +394.0 val_393 +395.0 val_394 +396.0 val_395 +397.0 val_396 +398.0 val_397 +400.0 val_399 +5.0 val_4 +401.0 val_400 +402.0 val_401 +403.0 val_402 +404.0 val_403 +405.0 val_404 +407.0 val_406 +408.0 val_407 +410.0 val_409 +42.0 val_41 +412.0 val_411 +414.0 val_413 +415.0 val_414 +418.0 val_417 +419.0 val_418 +420.0 val_419 +43.0 val_42 +422.0 val_421 +425.0 val_424 +428.0 val_427 +430.0 val_429 +44.0 val_43 +431.0 val_430 +432.0 val_431 +433.0 val_432 +436.0 val_435 +437.0 val_436 +438.0 val_437 +439.0 val_438 +440.0 val_439 +45.0 val_44 +444.0 val_443 +445.0 val_444 +447.0 val_446 +449.0 val_448 +450.0 val_449 +453.0 val_452 +454.0 val_453 +455.0 val_454 +456.0 val_455 +458.0 val_457 +459.0 val_458 +460.0 val_459 +461.0 val_460 +463.0 val_462 +464.0 val_463 +467.0 val_466 +468.0 val_467 +469.0 val_468 +470.0 val_469 +48.0 val_47 +471.0 val_470 +473.0 val_472 +476.0 val_475 +478.0 val_477 +479.0 val_478 +480.0 val_479 +481.0 val_480 +482.0 val_481 +483.0 val_482 +484.0 val_483 +485.0 val_484 +486.0 val_485 +488.0 val_487 +490.0 val_489 +491.0 val_490 +492.0 val_491 +493.0 val_492 +494.0 val_493 +495.0 val_494 +496.0 val_495 +497.0 val_496 +498.0 val_497 +499.0 val_498 +6.0 val_5 +52.0 val_51 +54.0 val_53 +55.0 val_54 +58.0 val_57 +59.0 val_58 +65.0 val_64 +66.0 val_65 +67.0 val_66 +68.0 val_67 +70.0 val_69 +71.0 val_70 +73.0 val_72 +75.0 val_74 +77.0 val_76 +78.0 val_77 +79.0 val_78 +9.0 val_8 +81.0 val_80 +83.0 val_82 +84.0 val_83 +85.0 val_84 +86.0 val_85 +87.0 val_86 +88.0 val_87 +10.0 val_9 +91.0 val_90 +93.0 val_92 +96.0 val_95 +97.0 val_96 +98.0 val_97 +99.0 val_98 \ No newline at end of file diff --git a/src/test/hive/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 b/src/test/hive/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 new file mode 100644 index 0000000000000..3d2dbbd731543 --- /dev/null +++ b/src/test/hive/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 @@ -0,0 +1 @@ +261.182 \ No newline at end of file diff --git a/src/test/hive/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 b/src/test/hive/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 new file mode 100644 index 0000000000000..cbc70c89f8859 --- /dev/null +++ b/src/test/hive/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 @@ -0,0 +1 @@ +260.182 \ No newline at end of file diff --git a/src/test/hive/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 b/src/test/hive/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b b/src/test/hive/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 b/src/test/hive/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/src/test/hive/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/src/test/hive/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 b/src/test/hive/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 b/src/test/hive/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 b/src/test/hive/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b b/src/test/hive/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b b/src/test/hive/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b b/src/test/hive/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..f1ea8b0260d2c --- /dev/null +++ b/src/test/hive/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1 @@ +ds=2010-01-01 \ No newline at end of file diff --git a/src/test/hive/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 b/src/test/hive/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b b/src/test/hive/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b b/src/test/hive/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..f1ea8b0260d2c --- /dev/null +++ b/src/test/hive/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1 @@ +ds=2010-01-01 \ No newline at end of file diff --git a/src/test/hive/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f b/src/test/hive/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b b/src/test/hive/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..121aebdfab9af --- /dev/null +++ b/src/test/hive/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1,2 @@ +ds=2010-01-01 +ds=2010-01-02 \ No newline at end of file diff --git a/src/test/hive/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb b/src/test/hive/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b b/src/test/hive/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..64467c12563c8 --- /dev/null +++ b/src/test/hive/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1,3 @@ +ds=2010-01-01 +ds=2010-01-02 +ds=2010-01-03 \ No newline at end of file diff --git a/src/test/hive/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b b/src/test/hive/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..f1ea8b0260d2c --- /dev/null +++ b/src/test/hive/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1 @@ +ds=2010-01-01 \ No newline at end of file diff --git a/src/test/hive/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 b/src/test/hive/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b b/src/test/hive/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..f1ea8b0260d2c --- /dev/null +++ b/src/test/hive/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1 @@ +ds=2010-01-01 \ No newline at end of file diff --git a/src/test/hive/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f b/src/test/hive/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b b/src/test/hive/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..121aebdfab9af --- /dev/null +++ b/src/test/hive/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1,2 @@ +ds=2010-01-01 +ds=2010-01-02 \ No newline at end of file diff --git a/src/test/hive/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb b/src/test/hive/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b b/src/test/hive/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..64467c12563c8 --- /dev/null +++ b/src/test/hive/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1,3 @@ +ds=2010-01-01 +ds=2010-01-02 +ds=2010-01-03 \ No newline at end of file diff --git a/src/test/hive/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 b/src/test/hive/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 b/src/test/hive/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 new file mode 100644 index 0000000000000..da21345385b2a --- /dev/null +++ b/src/test/hive/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 @@ -0,0 +1,29 @@ +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/src/test/hive/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 b/src/test/hive/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef b/src/test/hive/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c b/src/test/hive/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c new file mode 100644 index 0000000000000..fc4021c39737f --- /dev/null +++ b/src/test/hive/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c @@ -0,0 +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 diff --git a/src/test/hive/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 b/src/test/hive/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af b/src/test/hive/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 b/src/test/hive/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f b/src/test/hive/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 b/src/test/hive/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 b/src/test/hive/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 b/src/test/hive/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 b/src/test/hive/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d b/src/test/hive/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d new file mode 100644 index 0000000000000..5f4de85940513 --- /dev/null +++ b/src/test/hive/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d @@ -0,0 +1 @@ +0 val_0 \ No newline at end of file diff --git a/src/test/hive/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 b/src/test/hive/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 b/src/test/hive/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 b/src/test/hive/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 b/src/test/hive/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 b/src/test/hive/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..62a8ea1d115fb --- /dev/null +++ b/src/test/hive/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da b/src/test/hive/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 b/src/test/hive/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..72621add45bb3 --- /dev/null +++ b/src/test/hive/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-12-75a213649242c2410ea6846f08c91d75 b/src/test/hive/golden/alter2-12-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-13-ca51e03a3de391983429b6ad877e573c b/src/test/hive/golden/alter2-13-ca51e03a3de391983429b6ad877e573c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 b/src/test/hive/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..72621add45bb3 --- /dev/null +++ b/src/test/hive/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-15-75a213649242c2410ea6846f08c91d75 b/src/test/hive/golden/alter2-15-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..15e6bc8823150 --- /dev/null +++ b/src/test/hive/golden/alter2-15-75a213649242c2410ea6846f08c91d75 @@ -0,0 +1 @@ +insertdate=2008-01-01 \ No newline at end of file diff --git a/src/test/hive/golden/alter2-16-77500c83ffcece95511a4b21d67382dd b/src/test/hive/golden/alter2-16-77500c83ffcece95511a4b21d67382dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 b/src/test/hive/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..72621add45bb3 --- /dev/null +++ b/src/test/hive/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-18-75a213649242c2410ea6846f08c91d75 b/src/test/hive/golden/alter2-18-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..d31318a383fc1 --- /dev/null +++ b/src/test/hive/golden/alter2-18-75a213649242c2410ea6846f08c91d75 @@ -0,0 +1,2 @@ +insertdate=2008-01-01 +insertdate=2008-01-02 \ No newline at end of file diff --git a/src/test/hive/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 b/src/test/hive/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-2-75a213649242c2410ea6846f08c91d75 b/src/test/hive/golden/alter2-2-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-20-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter2-20-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/src/test/hive/golden/alter2-20-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/src/test/hive/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 b/src/test/hive/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 b/src/test/hive/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-23-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter2-23-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-24-775d69742a1c07df8da87e8a017d955 b/src/test/hive/golden/alter2-24-775d69742a1c07df8da87e8a017d955 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-25-4ef75e12575453225738ea167c4617e5 b/src/test/hive/golden/alter2-25-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..83184977e9da8 --- /dev/null +++ b/src/test/hive/golden/alter2-25-4ef75e12575453225738ea167c4617e5 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-26-63f13c364546ddce5d2176c6604a948f b/src/test/hive/golden/alter2-26-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f b/src/test/hive/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-28-4ef75e12575453225738ea167c4617e5 b/src/test/hive/golden/alter2-28-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..83184977e9da8 --- /dev/null +++ b/src/test/hive/golden/alter2-28-4ef75e12575453225738ea167c4617e5 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-29-63f13c364546ddce5d2176c6604a948f b/src/test/hive/golden/alter2-29-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..15e6bc8823150 --- /dev/null +++ b/src/test/hive/golden/alter2-29-63f13c364546ddce5d2176c6604a948f @@ -0,0 +1 @@ +insertdate=2008-01-01 \ No newline at end of file diff --git a/src/test/hive/golden/alter2-3-ca51e03a3de391983429b6ad877e573c b/src/test/hive/golden/alter2-3-ca51e03a3de391983429b6ad877e573c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae b/src/test/hive/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-31-4ef75e12575453225738ea167c4617e5 b/src/test/hive/golden/alter2-31-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..83184977e9da8 --- /dev/null +++ b/src/test/hive/golden/alter2-31-4ef75e12575453225738ea167c4617e5 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-32-63f13c364546ddce5d2176c6604a948f b/src/test/hive/golden/alter2-32-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..d31318a383fc1 --- /dev/null +++ b/src/test/hive/golden/alter2-32-63f13c364546ddce5d2176c6604a948f @@ -0,0 +1,2 @@ +insertdate=2008-01-01 +insertdate=2008-01-02 \ No newline at end of file diff --git a/src/test/hive/golden/alter2-33-1934026d0228967097280eed35551f74 b/src/test/hive/golden/alter2-33-1934026d0228967097280eed35551f74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b b/src/test/hive/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-35-4ef75e12575453225738ea167c4617e5 b/src/test/hive/golden/alter2-35-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..6e30936a881bd --- /dev/null +++ b/src/test/hive/golden/alter2-35-4ef75e12575453225738ea167c4617e5 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-36-63f13c364546ddce5d2176c6604a948f b/src/test/hive/golden/alter2-36-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f b/src/test/hive/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-38-4ef75e12575453225738ea167c4617e5 b/src/test/hive/golden/alter2-38-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..6e30936a881bd --- /dev/null +++ b/src/test/hive/golden/alter2-38-4ef75e12575453225738ea167c4617e5 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-39-63f13c364546ddce5d2176c6604a948f b/src/test/hive/golden/alter2-39-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..15e6bc8823150 --- /dev/null +++ b/src/test/hive/golden/alter2-39-63f13c364546ddce5d2176c6604a948f @@ -0,0 +1 @@ +insertdate=2008-01-01 \ No newline at end of file diff --git a/src/test/hive/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 b/src/test/hive/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..62a8ea1d115fb --- /dev/null +++ b/src/test/hive/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae b/src/test/hive/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-41-4ef75e12575453225738ea167c4617e5 b/src/test/hive/golden/alter2-41-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..6e30936a881bd --- /dev/null +++ b/src/test/hive/golden/alter2-41-4ef75e12575453225738ea167c4617e5 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-42-63f13c364546ddce5d2176c6604a948f b/src/test/hive/golden/alter2-42-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..d31318a383fc1 --- /dev/null +++ b/src/test/hive/golden/alter2-42-63f13c364546ddce5d2176c6604a948f @@ -0,0 +1,2 @@ +insertdate=2008-01-01 +insertdate=2008-01-02 \ No newline at end of file diff --git a/src/test/hive/golden/alter2-43-1934026d0228967097280eed35551f74 b/src/test/hive/golden/alter2-43-1934026d0228967097280eed35551f74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 b/src/test/hive/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c b/src/test/hive/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-5-75a213649242c2410ea6846f08c91d75 b/src/test/hive/golden/alter2-5-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..15e6bc8823150 --- /dev/null +++ b/src/test/hive/golden/alter2-5-75a213649242c2410ea6846f08c91d75 @@ -0,0 +1 @@ +insertdate=2008-01-01 \ No newline at end of file diff --git a/src/test/hive/golden/alter2-6-77500c83ffcece95511a4b21d67382dd b/src/test/hive/golden/alter2-6-77500c83ffcece95511a4b21d67382dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 b/src/test/hive/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..62a8ea1d115fb --- /dev/null +++ b/src/test/hive/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 @@ -0,0 +1,10 @@ +a int None +b int None +insertdate string None + +# Partition Information +# col_name data_type comment + +insertdate string None + +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 diff --git a/src/test/hive/golden/alter2-8-75a213649242c2410ea6846f08c91d75 b/src/test/hive/golden/alter2-8-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..d31318a383fc1 --- /dev/null +++ b/src/test/hive/golden/alter2-8-75a213649242c2410ea6846f08c91d75 @@ -0,0 +1,2 @@ +insertdate=2008-01-01 +insertdate=2008-01-02 \ No newline at end of file diff --git a/src/test/hive/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e b/src/test/hive/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade b/src/test/hive/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-1-75be487df30e301e156a22eee075633d b/src/test/hive/golden/alter3-1-75be487df30e301e156a22eee075633d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 b/src/test/hive/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 b/src/test/hive/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 b/src/test/hive/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 new file mode 100644 index 0000000000000..ba1746da5ce69 --- /dev/null +++ b/src/test/hive/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 @@ -0,0 +1,11 @@ +col1 string None +pcol1 string None +pcol2 string None + +# Partition Information +# col_name data_type comment + +pcol1 string None +pcol2 string None + +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 diff --git a/src/test/hive/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b b/src/test/hive/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f b/src/test/hive/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 b/src/test/hive/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-16-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter3-16-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/src/test/hive/golden/alter3-16-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/src/test/hive/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b b/src/test/hive/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d b/src/test/hive/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-19-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter3-19-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f b/src/test/hive/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 b/src/test/hive/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 b/src/test/hive/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 b/src/test/hive/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 b/src/test/hive/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 b/src/test/hive/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-25-568a59760e5d3241b63d65cce595face b/src/test/hive/golden/alter3-25-568a59760e5d3241b63d65cce595face new file mode 100644 index 0000000000000..8e609740f34a7 --- /dev/null +++ b/src/test/hive/golden/alter3-25-568a59760e5d3241b63d65cce595face @@ -0,0 +1,6 @@ +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: \ No newline at end of file diff --git a/src/test/hive/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e b/src/test/hive/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 b/src/test/hive/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 new file mode 100644 index 0000000000000..0e12e0e166336 --- /dev/null +++ b/src/test/hive/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 @@ -0,0 +1,11 @@ +col1 string None +pcol1 string None +pcol2 string None + +# Partition Information +# col_name data_type comment + +pcol1 string None +pcol2 string None + +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 diff --git a/src/test/hive/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 b/src/test/hive/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 new file mode 100644 index 0000000000000..4acb920388ceb --- /dev/null +++ b/src/test/hive/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 @@ -0,0 +1,11 @@ +col1 string None +pcol1 string None +pcol2 string None + +# Partition Information +# col_name data_type comment + +pcol1 string None +pcol2 string None + +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 diff --git a/src/test/hive/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c b/src/test/hive/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c new file mode 100644 index 0000000000000..8e609740f34a7 --- /dev/null +++ b/src/test/hive/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c @@ -0,0 +1,6 @@ +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: \ No newline at end of file diff --git a/src/test/hive/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 b/src/test/hive/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 b/src/test/hive/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 b/src/test/hive/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 b/src/test/hive/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 new file mode 100644 index 0000000000000..12b038204bef4 --- /dev/null +++ b/src/test/hive/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 @@ -0,0 +1,11 @@ +col1 string None +pcol1 string None +pcol2 string None + +# Partition Information +# col_name data_type comment + +pcol1 string None +pcol2 string None + +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 diff --git a/src/test/hive/golden/alter3-4-399fc26b344c98ababa104522601c0cc b/src/test/hive/golden/alter3-4-399fc26b344c98ababa104522601c0cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 b/src/test/hive/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 new file mode 100644 index 0000000000000..8e609740f34a7 --- /dev/null +++ b/src/test/hive/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 @@ -0,0 +1,6 @@ +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: \ No newline at end of file diff --git a/src/test/hive/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 b/src/test/hive/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 b/src/test/hive/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 new file mode 100644 index 0000000000000..0c47fb0ac64aa --- /dev/null +++ b/src/test/hive/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 @@ -0,0 +1,11 @@ +col1 string None +pcol1 string None +pcol2 string None + +# Partition Information +# col_name data_type comment + +pcol1 string None +pcol2 string None + +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 diff --git a/src/test/hive/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 b/src/test/hive/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 new file mode 100644 index 0000000000000..1165de3a92fd0 --- /dev/null +++ b/src/test/hive/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 @@ -0,0 +1,11 @@ +col1 string None +pcol1 string None +pcol2 string None + +# Partition Information +# col_name data_type comment + +pcol1 string None +pcol2 string None + +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 diff --git a/src/test/hive/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 b/src/test/hive/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 new file mode 100644 index 0000000000000..8e609740f34a7 --- /dev/null +++ b/src/test/hive/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 @@ -0,0 +1,6 @@ +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: \ No newline at end of file diff --git a/src/test/hive/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 b/src/test/hive/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 b/src/test/hive/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 new file mode 100644 index 0000000000000..8d2aebeb4d29d --- /dev/null +++ b/src/test/hive/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 @@ -0,0 +1,4 @@ +key int None +value string None + +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 diff --git a/src/test/hive/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 b/src/test/hive/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 new file mode 100644 index 0000000000000..cf67a009ff291 --- /dev/null +++ b/src/test/hive/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 @@ -0,0 +1,4 @@ +key int None +value string None + +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 diff --git a/src/test/hive/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d b/src/test/hive/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 b/src/test/hive/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 new file mode 100644 index 0000000000000..a6375189a7a23 --- /dev/null +++ b/src/test/hive/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 @@ -0,0 +1,4 @@ +key int None +value string None + +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 diff --git a/src/test/hive/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f b/src/test/hive/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 b/src/test/hive/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 b/src/test/hive/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 b/src/test/hive/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 new file mode 100644 index 0000000000000..331d858ce9b12 --- /dev/null +++ b/src/test/hive/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 @@ -0,0 +1 @@ +default \ No newline at end of file diff --git a/src/test/hive/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d b/src/test/hive/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 b/src/test/hive/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 new file mode 100644 index 0000000000000..6b79a37a85f0c --- /dev/null +++ b/src/test/hive/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 @@ -0,0 +1,4 @@ +key int None +value string None + +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 diff --git a/src/test/hive/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 b/src/test/hive/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter4-5-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter4-5-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/src/test/hive/golden/alter4-5-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/src/test/hive/golden/alter4-6-acd58e84952d310aeddf78579c36286 b/src/test/hive/golden/alter4-6-acd58e84952d310aeddf78579c36286 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 b/src/test/hive/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter4-8-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter4-8-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 b/src/test/hive/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-0-953553e14d835682fa47338dcfffe227 b/src/test/hive/golden/alter5-0-953553e14d835682fa47338dcfffe227 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b b/src/test/hive/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-10-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter5-10-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..85c1918f46567 --- /dev/null +++ b/src/test/hive/golden/alter5-10-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart diff --git a/src/test/hive/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 b/src/test/hive/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 b/src/test/hive/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-13-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter5-13-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 b/src/test/hive/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b b/src/test/hive/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d b/src/test/hive/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-17-9176dc5fb5206209fa907a289db1263d b/src/test/hive/golden/alter5-17-9176dc5fb5206209fa907a289db1263d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf b/src/test/hive/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf new file mode 100644 index 0000000000000..e180d4c53ae73 --- /dev/null +++ b/src/test/hive/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf @@ -0,0 +1,9 @@ +col1 string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None + +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}) diff --git a/src/test/hive/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 b/src/test/hive/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d b/src/test/hive/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 b/src/test/hive/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 new file mode 100644 index 0000000000000..de522457dcada --- /dev/null +++ b/src/test/hive/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 @@ -0,0 +1,6 @@ +1 a +2 a +3 a +4 a +5 a +6 a diff --git a/src/test/hive/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf b/src/test/hive/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf new file mode 100644 index 0000000000000..3d4ba2636ab66 --- /dev/null +++ b/src/test/hive/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf @@ -0,0 +1,9 @@ +col1 string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None + +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}) diff --git a/src/test/hive/golden/alter5-3-2fc59e32c07186869811705c89aafadc b/src/test/hive/golden/alter5-3-2fc59e32c07186869811705c89aafadc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf b/src/test/hive/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf new file mode 100644 index 0000000000000..6669b628fc224 --- /dev/null +++ b/src/test/hive/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf @@ -0,0 +1,9 @@ +col1 string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None + +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}) diff --git a/src/test/hive/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 b/src/test/hive/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 b/src/test/hive/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 new file mode 100644 index 0000000000000..de522457dcada --- /dev/null +++ b/src/test/hive/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 @@ -0,0 +1,6 @@ +1 a +2 a +3 a +4 a +5 a +6 a diff --git a/src/test/hive/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf b/src/test/hive/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf new file mode 100644 index 0000000000000..e866ae0fa7654 --- /dev/null +++ b/src/test/hive/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf @@ -0,0 +1,9 @@ +col1 string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None + +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}) diff --git a/src/test/hive/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 b/src/test/hive/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter5-9-1e085f9741197e659413828c42386733 b/src/test/hive/golden/alter5-9-1e085f9741197e659413828c42386733 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 b/src/test/hive/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e b/src/test/hive/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 b/src/test/hive/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 new file mode 100644 index 0000000000000..2860d5c6b4cfb --- /dev/null +++ b/src/test/hive/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 @@ -0,0 +1,5 @@ +key int None +_bucketname string +_offsets array + +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, 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 diff --git a/src/test/hive/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb b/src/test/hive/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 b/src/test/hive/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 new file mode 100644 index 0000000000000..2860d5c6b4cfb --- /dev/null +++ b/src/test/hive/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 @@ -0,0 +1,5 @@ +key int None +_bucketname string +_offsets array + +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, 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 diff --git a/src/test/hive/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 b/src/test/hive/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d b/src/test/hive/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/src/test/hive/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/src/test/hive/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 b/src/test/hive/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 b/src/test/hive/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 b/src/test/hive/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 new file mode 100644 index 0000000000000..c80ef36c96ad4 --- /dev/null +++ b/src/test/hive/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 @@ -0,0 +1 @@ +754 -7678496319 \ No newline at end of file diff --git a/src/test/hive/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c b/src/test/hive/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 b/src/test/hive/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 new file mode 100644 index 0000000000000..618c1d01b726d --- /dev/null +++ b/src/test/hive/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 @@ -0,0 +1,12 @@ +key int from deserializer +value string from deserializer +ds string None +ts string None + +# Partition Information +# col_name data_type comment + +ds string None +ts string None + +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 diff --git a/src/test/hive/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d b/src/test/hive/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be b/src/test/hive/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f b/src/test/hive/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 b/src/test/hive/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/src/test/hive/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/src/test/hive/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 b/src/test/hive/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 new file mode 100644 index 0000000000000..c80ef36c96ad4 --- /dev/null +++ b/src/test/hive/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 @@ -0,0 +1 @@ +754 -7678496319 \ No newline at end of file diff --git a/src/test/hive/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc b/src/test/hive/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 b/src/test/hive/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/src/test/hive/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a b/src/test/hive/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 b/src/test/hive/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 b/src/test/hive/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 b/src/test/hive/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 b/src/test/hive/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 new file mode 100644 index 0000000000000..410b14d2ce6f9 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 @@ -0,0 +1 @@ +25 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e b/src/test/hive/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 b/src/test/hive/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 new file mode 100644 index 0000000000000..410b14d2ce6f9 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 @@ -0,0 +1 @@ +25 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 b/src/test/hive/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e b/src/test/hive/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e new file mode 100644 index 0000000000000..b28ab5ccf8a1b --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e @@ -0,0 +1,75 @@ +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/src/test/hive/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f b/src/test/hive/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c b/src/test/hive/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c new file mode 100644 index 0000000000000..c5b431b6cba29 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c @@ -0,0 +1 @@ +50 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 b/src/test/hive/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 new file mode 100644 index 0000000000000..a76c74dcec6ab --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 @@ -0,0 +1 @@ +75 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a b/src/test/hive/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a new file mode 100644 index 0000000000000..316ca7f65ba20 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a @@ -0,0 +1,10 @@ +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/src/test/hive/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 b/src/test/hive/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a b/src/test/hive/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a new file mode 100644 index 0000000000000..a7382fabfcb49 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a @@ -0,0 +1,10 @@ +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/src/test/hive/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 b/src/test/hive/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 new file mode 100644 index 0000000000000..a7382fabfcb49 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a b/src/test/hive/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a new file mode 100644 index 0000000000000..a7382fabfcb49 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a @@ -0,0 +1,10 @@ +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/src/test/hive/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 b/src/test/hive/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 new file mode 100644 index 0000000000000..a7382fabfcb49 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 b/src/test/hive/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 b/src/test/hive/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 new file mode 100644 index 0000000000000..410b14d2ce6f9 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 @@ -0,0 +1 @@ +25 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d b/src/test/hive/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d b/src/test/hive/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd b/src/test/hive/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd new file mode 100644 index 0000000000000..410b14d2ce6f9 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd @@ -0,0 +1 @@ +25 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 b/src/test/hive/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 b/src/test/hive/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 b/src/test/hive/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 b/src/test/hive/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 b/src/test/hive/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..db182e444d31d --- /dev/null +++ b/src/test/hive/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 @@ -0,0 +1,4 @@ +key int None +value string None + +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 diff --git a/src/test/hive/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 b/src/test/hive/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 b/src/test/hive/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 new file mode 100644 index 0000000000000..81e23f0bc1951 --- /dev/null +++ b/src/test/hive/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 @@ -0,0 +1,10 @@ +key int from deserializer +value string from deserializer +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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 diff --git a/src/test/hive/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 b/src/test/hive/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 b/src/test/hive/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 new file mode 100644 index 0000000000000..6dc1f3ca2c187 --- /dev/null +++ b/src/test/hive/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 @@ -0,0 +1,10 @@ +key int from deserializer +value string from deserializer +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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 diff --git a/src/test/hive/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 b/src/test/hive/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..4b754043d63ab --- /dev/null +++ b/src/test/hive/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 @@ -0,0 +1,10 @@ +key int None +value string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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 diff --git a/src/test/hive/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 b/src/test/hive/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 b/src/test/hive/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..f44c28ee36760 --- /dev/null +++ b/src/test/hive/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 @@ -0,0 +1,10 @@ +key int from deserializer +value string from deserializer +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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 diff --git a/src/test/hive/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 b/src/test/hive/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 b/src/test/hive/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..e739ad4992ec9 --- /dev/null +++ b/src/test/hive/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 @@ -0,0 +1,10 @@ +key int from deserializer +value string from deserializer +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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 diff --git a/src/test/hive/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 b/src/test/hive/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 b/src/test/hive/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 b/src/test/hive/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..092e5ed6e8a46 --- /dev/null +++ b/src/test/hive/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 @@ -0,0 +1,4 @@ +key int from deserializer +value string from deserializer + +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 diff --git a/src/test/hive/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 b/src/test/hive/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 b/src/test/hive/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..197e67d09bf49 --- /dev/null +++ b/src/test/hive/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 @@ -0,0 +1,4 @@ +key int from deserializer +value string from deserializer + +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 diff --git a/src/test/hive/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 b/src/test/hive/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 b/src/test/hive/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 b/src/test/hive/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 b/src/test/hive/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 new file mode 100644 index 0000000000000..29b5b693b8589 --- /dev/null +++ b/src/test/hive/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 @@ -0,0 +1,10 @@ +key int None +value string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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 diff --git a/src/test/hive/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f b/src/test/hive/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c b/src/test/hive/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d b/src/test/hive/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b b/src/test/hive/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 b/src/test/hive/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a b/src/test/hive/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a b/src/test/hive/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 b/src/test/hive/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 b/src/test/hive/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 b/src/test/hive/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 b/src/test/hive/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 b/src/test/hive/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c b/src/test/hive/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 b/src/test/hive/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 new file mode 100644 index 0000000000000..02d72f4292749 --- /dev/null +++ b/src/test/hive/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 @@ -0,0 +1,12 @@ +1 11 1996 10 +2 12 1996 10 +3 13 1996 10 +7 17 1996 10 +8 18 1996 10 +8 28 1996 10 +1 11 1996 12 +2 12 1996 12 +3 13 1996 12 +7 17 1996 12 +8 18 1996 12 +8 28 1996 12 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 b/src/test/hive/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed b/src/test/hive/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d b/src/test/hive/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d new file mode 100644 index 0000000000000..bca47334cedaa --- /dev/null +++ b/src/test/hive/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d @@ -0,0 +1,6 @@ +1 11 1995 09 +2 12 1995 09 +3 13 1995 09 +7 17 1995 09 +8 18 1995 09 +8 28 1995 09 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 b/src/test/hive/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 b/src/test/hive/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 b/src/test/hive/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 b/src/test/hive/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c b/src/test/hive/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 b/src/test/hive/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada b/src/test/hive/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 b/src/test/hive/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 b/src/test/hive/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 new file mode 100644 index 0000000000000..e881b2a4b74eb --- /dev/null +++ b/src/test/hive/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 @@ -0,0 +1,6 @@ +1 new_part1: new_part2: +2 new_part1: new_part2: +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 diff --git a/src/test/hive/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 b/src/test/hive/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada b/src/test/hive/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/src/test/hive/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/src/test/hive/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 b/src/test/hive/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 b/src/test/hive/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b b/src/test/hive/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c b/src/test/hive/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/src/test/hive/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/src/test/hive/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 b/src/test/hive/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 b/src/test/hive/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 b/src/test/hive/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 new file mode 100644 index 0000000000000..684f1da0fd0a6 --- /dev/null +++ b/src/test/hive/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 @@ -0,0 +1,6 @@ +1 old_part1: old_part2: +2 old_part1: old_part2: +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 diff --git a/src/test/hive/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f b/src/test/hive/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 b/src/test/hive/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 new file mode 100644 index 0000000000000..fc31ec62a1280 --- /dev/null +++ b/src/test/hive/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 @@ -0,0 +1 @@ +pcol1=new_part1%3A/pcol2=new_part2%3A \ No newline at end of file diff --git a/src/test/hive/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 b/src/test/hive/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b b/src/test/hive/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b new file mode 100644 index 0000000000000..e881b2a4b74eb --- /dev/null +++ b/src/test/hive/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b @@ -0,0 +1,6 @@ +1 new_part1: new_part2: +2 new_part1: new_part2: +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 diff --git a/src/test/hive/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b b/src/test/hive/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f b/src/test/hive/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 b/src/test/hive/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 b/src/test/hive/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 b/src/test/hive/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 new file mode 100644 index 0000000000000..684f1da0fd0a6 --- /dev/null +++ b/src/test/hive/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 @@ -0,0 +1,6 @@ +1 old_part1: old_part2: +2 old_part1: old_part2: +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 diff --git a/src/test/hive/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 b/src/test/hive/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 b/src/test/hive/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 new file mode 100644 index 0000000000000..fc31ec62a1280 --- /dev/null +++ b/src/test/hive/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 @@ -0,0 +1 @@ +pcol1=new_part1%3A/pcol2=new_part2%3A \ No newline at end of file diff --git a/src/test/hive/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c b/src/test/hive/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 b/src/test/hive/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 new file mode 100644 index 0000000000000..ccd6518a50f7c --- /dev/null +++ b/src/test/hive/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 @@ -0,0 +1,5 @@ +id int None +query string None +name string None + +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 diff --git a/src/test/hive/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa b/src/test/hive/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 b/src/test/hive/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 new file mode 100644 index 0000000000000..d135e450e6e8e --- /dev/null +++ b/src/test/hive/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 @@ -0,0 +1,11 @@ +id int None +query string None +name string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None + +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 diff --git a/src/test/hive/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc b/src/test/hive/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 b/src/test/hive/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 new file mode 100644 index 0000000000000..d135e450e6e8e --- /dev/null +++ b/src/test/hive/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 @@ -0,0 +1,11 @@ +id int None +query string None +name string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None + +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 diff --git a/src/test/hive/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def b/src/test/hive/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 b/src/test/hive/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 new file mode 100644 index 0000000000000..c3fac88f91a36 --- /dev/null +++ b/src/test/hive/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 @@ -0,0 +1,11 @@ +id int None +query string None +name string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None + +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 diff --git a/src/test/hive/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 b/src/test/hive/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 b/src/test/hive/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 new file mode 100644 index 0000000000000..6c8f91de7cded --- /dev/null +++ b/src/test/hive/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 @@ -0,0 +1,11 @@ +id int None +query string None +name string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None + +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 diff --git a/src/test/hive/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 b/src/test/hive/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa b/src/test/hive/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 b/src/test/hive/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 new file mode 100644 index 0000000000000..37a5b2cc47bad --- /dev/null +++ b/src/test/hive/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 @@ -0,0 +1,5 @@ +id int from deserializer +query string from deserializer +name string from deserializer + +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 diff --git a/src/test/hive/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc b/src/test/hive/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 b/src/test/hive/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 new file mode 100644 index 0000000000000..0348dd15fd4f5 --- /dev/null +++ b/src/test/hive/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 @@ -0,0 +1,5 @@ +id int from deserializer +query string from deserializer +name string from deserializer + +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 diff --git a/src/test/hive/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 b/src/test/hive/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b b/src/test/hive/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 b/src/test/hive/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 b/src/test/hive/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 new file mode 100644 index 0000000000000..d135e450e6e8e --- /dev/null +++ b/src/test/hive/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 @@ -0,0 +1,11 @@ +id int None +query string None +name string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None + +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 diff --git a/src/test/hive/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 b/src/test/hive/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea b/src/test/hive/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 b/src/test/hive/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 b/src/test/hive/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 new file mode 100644 index 0000000000000..dd347f3e8f58e --- /dev/null +++ b/src/test/hive/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 @@ -0,0 +1,5 @@ +0 val_0 NULL NULL +0 val_0 NULL NULL +0 val_0 NULL NULL +2 val_2 NULL NULL +4 val_4 NULL NULL diff --git a/src/test/hive/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd b/src/test/hive/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 b/src/test/hive/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 new file mode 100644 index 0000000000000..12087837cebf1 --- /dev/null +++ b/src/test/hive/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 @@ -0,0 +1,5 @@ +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +2 val_2 2 val_2 +4 val_4 4 val_4 diff --git a/src/test/hive/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 b/src/test/hive/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 b/src/test/hive/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 b/src/test/hive/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/src/test/hive/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/src/test/hive/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d b/src/test/hive/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 b/src/test/hive/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/src/test/hive/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/src/test/hive/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 b/src/test/hive/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f b/src/test/hive/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f new file mode 100644 index 0000000000000..879a6e7bcbd18 --- /dev/null +++ b/src/test/hive/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f @@ -0,0 +1,5 @@ +0 val +0 val +0 val +2 val +4 val diff --git a/src/test/hive/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 b/src/test/hive/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 b/src/test/hive/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/src/test/hive/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/src/test/hive/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 b/src/test/hive/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc b/src/test/hive/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 b/src/test/hive/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 b/src/test/hive/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 new file mode 100644 index 0000000000000..40818a7de46d0 --- /dev/null +++ b/src/test/hive/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 @@ -0,0 +1 @@ +val_238 7 \ No newline at end of file diff --git a/src/test/hive/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb b/src/test/hive/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c b/src/test/hive/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c new file mode 100644 index 0000000000000..827220bd4996f --- /dev/null +++ b/src/test/hive/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c @@ -0,0 +1 @@ +1 val_238 7 \ No newline at end of file diff --git a/src/test/hive/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 b/src/test/hive/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c b/src/test/hive/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c new file mode 100644 index 0000000000000..827220bd4996f --- /dev/null +++ b/src/test/hive/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c @@ -0,0 +1 @@ +1 val_238 7 \ No newline at end of file diff --git a/src/test/hive/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c b/src/test/hive/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c new file mode 100644 index 0000000000000..8a8234a35f6bb --- /dev/null +++ b/src/test/hive/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c @@ -0,0 +1 @@ +2 238 3 \ No newline at end of file diff --git a/src/test/hive/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 b/src/test/hive/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad b/src/test/hive/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b b/src/test/hive/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b new file mode 100644 index 0000000000000..a99747531cef4 --- /dev/null +++ b/src/test/hive/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b @@ -0,0 +1,30 @@ +# 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: Fri Feb 07 14:47:52 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Table Type: VIRTUAL_VIEW +Table Parameters: + transient_lastDdlTime 1391813272 + +# 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/src/test/hive/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 b/src/test/hive/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b b/src/test/hive/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b new file mode 100644 index 0000000000000..499c73127d890 --- /dev/null +++ b/src/test/hive/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b @@ -0,0 +1,27 @@ +# col_name data_type comment + +value string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Fri Feb 07 14:47:52 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Table Type: VIRTUAL_VIEW +Table Parameters: + transient_lastDdlTime 1391813272 + +# 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 value FROM src WHERE key=86 +View Expanded Text: SELECT `src`.`value` FROM `default`.`src` WHERE `src`.`key`=86 diff --git a/src/test/hive/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 b/src/test/hive/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b b/src/test/hive/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b new file mode 100644 index 0000000000000..a5fba77abdf07 --- /dev/null +++ b/src/test/hive/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b @@ -0,0 +1,34 @@ +# col_name data_type comment + +key int None +value string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Fri Feb 07 14:47:52 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Table Type: VIRTUAL_VIEW +Table Parameters: + transient_lastDdlTime 1391813272 + +# 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 src +WHERE key > 80 AND key < 100 +ORDER BY key, value +LIMIT 10 +View Expanded Text: SELECT `src`.`key`, `src`.`value` FROM `default`.`src` +WHERE `src`.`key` > 80 AND `src`.`key` < 100 +ORDER BY `src`.`key`, `src`.`value` +LIMIT 10 diff --git a/src/test/hive/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 b/src/test/hive/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d b/src/test/hive/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb b/src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb new file mode 100644 index 0000000000000..43b9042d0bffe --- /dev/null +++ b/src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb @@ -0,0 +1,5 @@ +foo int None +bar string None +ds string None + +Detailed Table Information Table(tableName:view1, dbName:default, owner:marmbrus, createTime:1392059376, 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=1392059376}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW) diff --git a/src/test/hive/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a b/src/test/hive/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 b/src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 new file mode 100644 index 0000000000000..0419cd9376ce4 --- /dev/null +++ b/src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 @@ -0,0 +1,5 @@ +foo int None +bar string None +ds string None + +Detailed Table Information Table(tableName:view2, dbName:default, owner:marmbrus, createTime:1392059376, 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=marmbrus, last_modified_time=1392059376, transient_lastDdlTime=1392059376}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW) diff --git a/src/test/hive/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d b/src/test/hive/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab b/src/test/hive/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b b/src/test/hive/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 b/src/test/hive/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 b/src/test/hive/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb b/src/test/hive/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/src/test/hive/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/src/test/hive/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 b/src/test/hive/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb b/src/test/hive/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/src/test/hive/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/src/test/hive/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 b/src/test/hive/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 new file mode 100644 index 0000000000000..21b3b13a81191 --- /dev/null +++ b/src/test/hive/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 @@ -0,0 +1 @@ +0 3 \ No newline at end of file diff --git a/src/test/hive/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 b/src/test/hive/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 new file mode 100644 index 0000000000000..5e5f6ff96623f --- /dev/null +++ b/src/test/hive/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 @@ -0,0 +1,9 @@ +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/src/test/hive/golden/archive-16-892147913578bcf60620b7dd73893dd0 b/src/test/hive/golden/archive-16-892147913578bcf60620b7dd73893dd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb b/src/test/hive/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/src/test/hive/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/src/test/hive/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a b/src/test/hive/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f b/src/test/hive/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-2-713efc113418b01f76ffd589840193c8 b/src/test/hive/golden/archive-2-713efc113418b01f76ffd589840193c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-20-530277b0fee8b05c37b26846bceef827 b/src/test/hive/golden/archive-20-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/src/test/hive/golden/archive-20-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/src/test/hive/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 b/src/test/hive/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-22-530277b0fee8b05c37b26846bceef827 b/src/test/hive/golden/archive-22-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/src/test/hive/golden/archive-22-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/src/test/hive/golden/archive-23-892147913578bcf60620b7dd73893dd0 b/src/test/hive/golden/archive-23-892147913578bcf60620b7dd73893dd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-24-530277b0fee8b05c37b26846bceef827 b/src/test/hive/golden/archive-24-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/src/test/hive/golden/archive-24-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/src/test/hive/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f b/src/test/hive/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a b/src/test/hive/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 b/src/test/hive/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-28-188eb7912265ed8dffa5200517bbe526 b/src/test/hive/golden/archive-28-188eb7912265ed8dffa5200517bbe526 new file mode 100644 index 0000000000000..18a1a7925ff29 --- /dev/null +++ b/src/test/hive/golden/archive-28-188eb7912265ed8dffa5200517bbe526 @@ -0,0 +1 @@ +48656137 \ No newline at end of file diff --git a/src/test/hive/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 b/src/test/hive/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-3-27ad2962fed131f51ba802596ba37278 b/src/test/hive/golden/archive-3-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/src/test/hive/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a new file mode 100644 index 0000000000000..18a1a7925ff29 --- /dev/null +++ b/src/test/hive/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a @@ -0,0 +1 @@ +48656137 \ No newline at end of file diff --git a/src/test/hive/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 b/src/test/hive/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-32-27ad2962fed131f51ba802596ba37278 b/src/test/hive/golden/archive-32-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-4-3e95421993ab28d18245ec2340f580a3 b/src/test/hive/golden/archive-4-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-5-c0c18ac884677231a41eea8d980d0451 b/src/test/hive/golden/archive-5-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-6-528ab9750a558af7f1a43b3108e793dd b/src/test/hive/golden/archive-6-528ab9750a558af7f1a43b3108e793dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca b/src/test/hive/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c b/src/test/hive/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf b/src/test/hive/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 b/src/test/hive/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 b/src/test/hive/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb b/src/test/hive/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 b/src/test/hive/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb b/src/test/hive/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 b/src/test/hive/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 new file mode 100644 index 0000000000000..21b3b13a81191 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 @@ -0,0 +1 @@ +0 3 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 b/src/test/hive/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 new file mode 100644 index 0000000000000..5e5f6ff96623f --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 @@ -0,0 +1,9 @@ +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/src/test/hive/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 b/src/test/hive/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb b/src/test/hive/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a b/src/test/hive/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f b/src/test/hive/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 b/src/test/hive/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 b/src/test/hive/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 b/src/test/hive/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 b/src/test/hive/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 b/src/test/hive/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 b/src/test/hive/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f b/src/test/hive/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a b/src/test/hive/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 b/src/test/hive/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 b/src/test/hive/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 new file mode 100644 index 0000000000000..18a1a7925ff29 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 @@ -0,0 +1 @@ +48656137 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 b/src/test/hive/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 b/src/test/hive/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/src/test/hive/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a new file mode 100644 index 0000000000000..18a1a7925ff29 --- /dev/null +++ b/src/test/hive/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a @@ -0,0 +1 @@ +48656137 \ No newline at end of file diff --git a/src/test/hive/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 b/src/test/hive/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 b/src/test/hive/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 b/src/test/hive/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 b/src/test/hive/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd b/src/test/hive/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca b/src/test/hive/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c b/src/test/hive/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf b/src/test/hive/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 b/src/test/hive/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 b/src/test/hive/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e b/src/test/hive/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/src/test/hive/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/src/test/hive/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 b/src/test/hive/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb b/src/test/hive/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/src/test/hive/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/src/test/hive/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 b/src/test/hive/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 new file mode 100644 index 0000000000000..21b3b13a81191 --- /dev/null +++ b/src/test/hive/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 @@ -0,0 +1 @@ +0 3 \ No newline at end of file diff --git a/src/test/hive/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 b/src/test/hive/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 new file mode 100644 index 0000000000000..5e5f6ff96623f --- /dev/null +++ b/src/test/hive/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 @@ -0,0 +1,9 @@ +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/src/test/hive/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d b/src/test/hive/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb b/src/test/hive/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/src/test/hive/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/src/test/hive/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 b/src/test/hive/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 b/src/test/hive/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 b/src/test/hive/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 b/src/test/hive/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a b/src/test/hive/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca b/src/test/hive/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c b/src/test/hive/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf b/src/test/hive/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba b/src/test/hive/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 b/src/test/hive/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c b/src/test/hive/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c new file mode 100644 index 0000000000000..308fc0924e670 --- /dev/null +++ b/src/test/hive/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c @@ -0,0 +1 @@ +34298511120 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a b/src/test/hive/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e b/src/test/hive/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 b/src/test/hive/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 b/src/test/hive/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e b/src/test/hive/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/src/test/hive/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 b/src/test/hive/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 b/src/test/hive/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/src/test/hive/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 b/src/test/hive/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 b/src/test/hive/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 new file mode 100644 index 0000000000000..69dbf8c3143e9 --- /dev/null +++ b/src/test/hive/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 @@ -0,0 +1 @@ +-101339664144 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 b/src/test/hive/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 b/src/test/hive/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 new file mode 100644 index 0000000000000..eff107c7ce6bc --- /dev/null +++ b/src/test/hive/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 @@ -0,0 +1 @@ +-136852761207 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 b/src/test/hive/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 b/src/test/hive/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 new file mode 100644 index 0000000000000..de6c015da2059 --- /dev/null +++ b/src/test/hive/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 @@ -0,0 +1 @@ +-97676500536 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 b/src/test/hive/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e b/src/test/hive/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 b/src/test/hive/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 b/src/test/hive/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 b/src/test/hive/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 b/src/test/hive/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 b/src/test/hive/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 new file mode 100644 index 0000000000000..f1871a4957ddb --- /dev/null +++ b/src/test/hive/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 @@ -0,0 +1 @@ +404554174174 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 b/src/test/hive/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f b/src/test/hive/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 b/src/test/hive/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 b/src/test/hive/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 b/src/test/hive/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 b/src/test/hive/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 new file mode 100644 index 0000000000000..0f27a9bde401c --- /dev/null +++ b/src/test/hive/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 @@ -0,0 +1 @@ +404554174174 diff --git a/src/test/hive/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 b/src/test/hive/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 b/src/test/hive/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 new file mode 100644 index 0000000000000..006e1f82c0a47 --- /dev/null +++ b/src/test/hive/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 @@ -0,0 +1 @@ +-793937029770 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf b/src/test/hive/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 b/src/test/hive/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 b/src/test/hive/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 b/src/test/hive/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 b/src/test/hive/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c b/src/test/hive/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c new file mode 100644 index 0000000000000..006e1f82c0a47 --- /dev/null +++ b/src/test/hive/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c @@ -0,0 +1 @@ +-793937029770 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b b/src/test/hive/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f b/src/test/hive/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f new file mode 100644 index 0000000000000..0c9b518e65ece --- /dev/null +++ b/src/test/hive/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f @@ -0,0 +1 @@ +2358131334 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 b/src/test/hive/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 b/src/test/hive/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f b/src/test/hive/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f new file mode 100644 index 0000000000000..795166629df40 --- /dev/null +++ b/src/test/hive/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f @@ -0,0 +1 @@ +407444119660 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca b/src/test/hive/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 b/src/test/hive/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a b/src/test/hive/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 b/src/test/hive/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 new file mode 100644 index 0000000000000..1c958900f5013 --- /dev/null +++ b/src/test/hive/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 @@ -0,0 +1 @@ +33815990627 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd b/src/test/hive/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 b/src/test/hive/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 new file mode 100644 index 0000000000000..b1a6075f768c8 --- /dev/null +++ b/src/test/hive/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 @@ -0,0 +1 @@ +-24276731469 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee b/src/test/hive/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 b/src/test/hive/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 new file mode 100644 index 0000000000000..b1a6075f768c8 --- /dev/null +++ b/src/test/hive/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 @@ -0,0 +1 @@ +-24276731469 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 b/src/test/hive/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f b/src/test/hive/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f new file mode 100644 index 0000000000000..9672e21fa0323 --- /dev/null +++ b/src/test/hive/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f @@ -0,0 +1,500 @@ +NULL NULL NULL NULL 0 val_0 +NULL NULL NULL NULL 0 val_0 +NULL NULL NULL NULL 0 val_0 +NULL NULL NULL NULL 2 val_2 +NULL NULL NULL NULL 4 val_4 +NULL NULL NULL NULL 5 val_5 +NULL NULL NULL NULL 5 val_5 +NULL NULL NULL NULL 5 val_5 +NULL NULL NULL NULL 8 val_8 +NULL NULL NULL NULL 9 val_9 +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 11 val_11 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 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 NULL NULL 26 val_26 +NULL NULL NULL NULL 26 val_26 +NULL NULL NULL NULL 27 val_27 +NULL NULL NULL NULL 28 val_28 +NULL NULL NULL NULL 30 val_30 +NULL NULL NULL NULL 33 val_33 +NULL NULL NULL NULL 34 val_34 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 41 val_41 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 43 val_43 +NULL NULL NULL NULL 44 val_44 +NULL NULL NULL NULL 47 val_47 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 53 val_53 +NULL NULL NULL NULL 54 val_54 +NULL NULL NULL NULL 57 val_57 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 64 val_64 +NULL NULL NULL NULL 65 val_65 +NULL NULL NULL NULL 66 val_66 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 69 val_69 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 74 val_74 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 77 val_77 +NULL NULL NULL NULL 78 val_78 +NULL NULL NULL NULL 80 val_80 +NULL NULL NULL NULL 82 val_82 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 85 val_85 +NULL NULL NULL NULL 86 val_86 +NULL NULL NULL NULL 87 val_87 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 92 val_92 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 96 val_96 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 105 val_105 +NULL NULL NULL NULL 111 val_111 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 114 val_114 +NULL NULL NULL NULL 116 val_116 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 126 val_126 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 131 val_131 +NULL NULL NULL NULL 133 val_133 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 136 val_136 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 143 val_143 +NULL NULL NULL NULL 145 val_145 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 150 val_150 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 153 val_153 +NULL NULL NULL NULL 155 val_155 +NULL NULL NULL NULL 156 val_156 +NULL NULL NULL NULL 157 val_157 +NULL NULL NULL NULL 158 val_158 +NULL NULL NULL NULL 160 val_160 +NULL NULL NULL NULL 162 val_162 +NULL NULL NULL NULL 163 val_163 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 166 val_166 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 168 val_168 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 170 val_170 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 177 val_177 +NULL NULL NULL NULL 178 val_178 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 180 val_180 +NULL NULL NULL NULL 181 val_181 +NULL NULL NULL NULL 183 val_183 +NULL NULL NULL NULL 186 val_186 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 189 val_189 +NULL NULL NULL NULL 190 val_190 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 192 val_192 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 194 val_194 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 196 val_196 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 201 val_201 +NULL NULL NULL NULL 202 val_202 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 214 val_214 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 218 val_218 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 222 val_222 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 226 val_226 +NULL NULL NULL NULL 228 val_228 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 235 val_235 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 241 val_241 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 244 val_244 +NULL NULL NULL NULL 247 val_247 +NULL NULL NULL NULL 248 val_248 +NULL NULL NULL NULL 249 val_249 +NULL NULL NULL NULL 252 val_252 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 257 val_257 +NULL NULL NULL NULL 258 val_258 +NULL NULL NULL NULL 260 val_260 +NULL NULL NULL NULL 262 val_262 +NULL NULL NULL NULL 263 val_263 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 266 val_266 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 274 val_274 +NULL NULL NULL NULL 275 val_275 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 283 val_283 +NULL NULL NULL NULL 284 val_284 +NULL NULL NULL NULL 285 val_285 +NULL NULL NULL NULL 286 val_286 +NULL NULL NULL NULL 287 val_287 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 289 val_289 +NULL NULL NULL NULL 291 val_291 +NULL NULL NULL NULL 292 val_292 +NULL NULL NULL NULL 296 val_296 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 302 val_302 +NULL NULL NULL NULL 305 val_305 +NULL NULL NULL NULL 306 val_306 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 308 val_308 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 310 val_310 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 315 val_315 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 323 val_323 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 332 val_332 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 335 val_335 +NULL NULL NULL NULL 336 val_336 +NULL NULL NULL NULL 338 val_338 +NULL NULL NULL NULL 339 val_339 +NULL NULL NULL NULL 341 val_341 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 345 val_345 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 351 val_351 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 356 val_356 +NULL NULL NULL NULL 360 val_360 +NULL NULL NULL NULL 362 val_362 +NULL NULL NULL NULL 364 val_364 +NULL NULL NULL NULL 365 val_365 +NULL NULL NULL NULL 366 val_366 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 368 val_368 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 373 val_373 +NULL NULL NULL NULL 374 val_374 +NULL NULL NULL NULL 375 val_375 +NULL NULL NULL NULL 377 val_377 +NULL NULL NULL NULL 378 val_378 +NULL NULL NULL NULL 379 val_379 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 386 val_386 +NULL NULL NULL NULL 389 val_389 +NULL NULL NULL NULL 392 val_392 +NULL NULL NULL NULL 393 val_393 +NULL NULL NULL NULL 394 val_394 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 400 val_400 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 402 val_402 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 407 val_407 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 411 val_411 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 418 val_418 +NULL NULL NULL NULL 419 val_419 +NULL NULL NULL NULL 421 val_421 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 427 val_427 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 432 val_432 +NULL NULL NULL NULL 435 val_435 +NULL NULL NULL NULL 436 val_436 +NULL NULL NULL NULL 437 val_437 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 443 val_443 +NULL NULL NULL NULL 444 val_444 +NULL NULL NULL NULL 446 val_446 +NULL NULL NULL NULL 448 val_448 +NULL NULL NULL NULL 449 val_449 +NULL NULL NULL NULL 452 val_452 +NULL NULL NULL NULL 453 val_453 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 455 val_455 +NULL NULL NULL NULL 457 val_457 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 460 val_460 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 467 val_467 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 470 val_470 +NULL NULL NULL NULL 472 val_472 +NULL NULL NULL NULL 475 val_475 +NULL NULL NULL NULL 477 val_477 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 479 val_479 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 481 val_481 +NULL NULL NULL NULL 482 val_482 +NULL NULL NULL NULL 483 val_483 +NULL NULL NULL NULL 484 val_484 +NULL NULL NULL NULL 485 val_485 +NULL NULL NULL NULL 487 val_487 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 490 val_490 +NULL NULL NULL NULL 491 val_491 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 493 val_493 +NULL NULL NULL NULL 494 val_494 +NULL NULL NULL NULL 495 val_495 +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 diff --git a/src/test/hive/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 b/src/test/hive/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb b/src/test/hive/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb new file mode 100644 index 0000000000000..dba3bca53f72d --- /dev/null +++ b/src/test/hive/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb @@ -0,0 +1 @@ +344337359100 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join23-1-b31437533a2b890788938455cb32f679 b/src/test/hive/golden/auto_join23-1-b31437533a2b890788938455cb32f679 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 b/src/test/hive/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 new file mode 100644 index 0000000000000..5707ed08e7e54 --- /dev/null +++ b/src/test/hive/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 @@ -0,0 +1,100 @@ +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 +0 val_0 2 val_2 +0 val_0 2 val_2 +0 val_0 4 val_4 +0 val_0 4 val_4 +0 val_0 4 val_4 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 8 val_8 +0 val_0 8 val_8 +0 val_0 8 val_8 +0 val_0 9 val_9 +0 val_0 9 val_9 +0 val_0 9 val_9 +2 val_2 0 val_0 +2 val_2 0 val_0 +2 val_2 0 val_0 +2 val_2 2 val_2 +2 val_2 4 val_4 +2 val_2 5 val_5 +2 val_2 5 val_5 +2 val_2 5 val_5 +2 val_2 8 val_8 +2 val_2 9 val_9 +4 val_4 0 val_0 +4 val_4 0 val_0 +4 val_4 0 val_0 +4 val_4 2 val_2 +4 val_4 4 val_4 +4 val_4 5 val_5 +4 val_4 5 val_5 +4 val_4 5 val_5 +4 val_4 8 val_8 +4 val_4 9 val_9 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 2 val_2 +5 val_5 2 val_2 +5 val_5 2 val_2 +5 val_5 4 val_4 +5 val_5 4 val_4 +5 val_5 4 val_4 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 8 val_8 +5 val_5 8 val_8 +5 val_5 8 val_8 +5 val_5 9 val_9 +5 val_5 9 val_9 +5 val_5 9 val_9 +8 val_8 0 val_0 +8 val_8 0 val_0 +8 val_8 0 val_0 +8 val_8 2 val_2 +8 val_8 4 val_4 +8 val_8 5 val_5 +8 val_8 5 val_5 +8 val_8 5 val_5 +8 val_8 8 val_8 +8 val_8 9 val_9 +9 val_9 0 val_0 +9 val_9 0 val_0 +9 val_9 0 val_0 +9 val_9 2 val_2 +9 val_9 4 val_4 +9 val_9 5 val_5 +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 diff --git a/src/test/hive/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b b/src/test/hive/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d b/src/test/hive/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 b/src/test/hive/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 b/src/test/hive/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/src/test/hive/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e b/src/test/hive/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 b/src/test/hive/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e b/src/test/hive/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e new file mode 100644 index 0000000000000..d14fbdc94256c --- /dev/null +++ b/src/test/hive/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e @@ -0,0 +1 @@ +101861029915 diff --git a/src/test/hive/golden/auto_join25-2-8180638a57b64557e02815c863031755 b/src/test/hive/golden/auto_join25-2-8180638a57b64557e02815c863031755 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join25-2-8180638a57b64557e02815c863031755 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e b/src/test/hive/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 b/src/test/hive/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 b/src/test/hive/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f b/src/test/hive/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f new file mode 100644 index 0000000000000..069b64b649977 --- /dev/null +++ b/src/test/hive/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f @@ -0,0 +1 @@ +407444119660 diff --git a/src/test/hive/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca b/src/test/hive/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a b/src/test/hive/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 b/src/test/hive/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 new file mode 100644 index 0000000000000..b8f473bf53aa3 --- /dev/null +++ b/src/test/hive/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 @@ -0,0 +1 @@ +33815990627 diff --git a/src/test/hive/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 b/src/test/hive/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 b/src/test/hive/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 b/src/test/hive/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..71094ee7360db --- /dev/null +++ b/src/test/hive/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,15 @@ +66 1 +98 2 +128 3 +146 2 +150 1 +213 2 +224 2 +238 2 +255 2 +273 3 +278 2 +311 3 +369 3 +401 5 +406 4 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 b/src/test/hive/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e b/src/test/hive/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e new file mode 100644 index 0000000000000..dd35c6b71fc80 --- /dev/null +++ b/src/test/hive/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e @@ -0,0 +1 @@ +548 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c b/src/test/hive/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 b/src/test/hive/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 b/src/test/hive/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 b/src/test/hive/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e b/src/test/hive/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d b/src/test/hive/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 b/src/test/hive/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f b/src/test/hive/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/src/test/hive/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 b/src/test/hive/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a b/src/test/hive/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/src/test/hive/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 b/src/test/hive/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d b/src/test/hive/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/src/test/hive/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba b/src/test/hive/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 b/src/test/hive/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/src/test/hive/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 b/src/test/hive/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 b/src/test/hive/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/src/test/hive/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 b/src/test/hive/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/src/test/hive/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 b/src/test/hive/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 b/src/test/hive/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/src/test/hive/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 b/src/test/hive/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 b/src/test/hive/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/src/test/hive/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e b/src/test/hive/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 b/src/test/hive/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/src/test/hive/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 b/src/test/hive/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 b/src/test/hive/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 b/src/test/hive/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/src/test/hive/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 b/src/test/hive/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 b/src/test/hive/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 b/src/test/hive/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 b/src/test/hive/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e b/src/test/hive/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f b/src/test/hive/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 b/src/test/hive/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 b/src/test/hive/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 b/src/test/hive/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f b/src/test/hive/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 b/src/test/hive/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d b/src/test/hive/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 b/src/test/hive/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b b/src/test/hive/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c b/src/test/hive/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 b/src/test/hive/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 b/src/test/hive/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b b/src/test/hive/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a b/src/test/hive/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc b/src/test/hive/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 b/src/test/hive/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 b/src/test/hive/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 b/src/test/hive/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 b/src/test/hive/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd b/src/test/hive/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf b/src/test/hive/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed b/src/test/hive/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef b/src/test/hive/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee b/src/test/hive/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a b/src/test/hive/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd b/src/test/hive/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a b/src/test/hive/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 b/src/test/hive/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 b/src/test/hive/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 new file mode 100644 index 0000000000000..f7d1e92d77207 --- /dev/null +++ b/src/test/hive/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 @@ -0,0 +1 @@ +5079148035 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd b/src/test/hive/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 b/src/test/hive/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 b/src/test/hive/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 b/src/test/hive/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 new file mode 100644 index 0000000000000..e859b7c4ada7b --- /dev/null +++ b/src/test/hive/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 @@ -0,0 +1 @@ +9766083196 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd b/src/test/hive/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 b/src/test/hive/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 b/src/test/hive/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 b/src/test/hive/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 new file mode 100644 index 0000000000000..f15ab2fb14eaa --- /dev/null +++ b/src/test/hive/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 @@ -0,0 +1 @@ +2607643291 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 b/src/test/hive/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e b/src/test/hive/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 b/src/test/hive/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 b/src/test/hive/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 new file mode 100644 index 0000000000000..5f7f06c079d24 --- /dev/null +++ b/src/test/hive/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 @@ -0,0 +1 @@ +-2315698213 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd b/src/test/hive/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab b/src/test/hive/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca b/src/test/hive/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 b/src/test/hive/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 new file mode 100644 index 0000000000000..d73e8745ecc00 --- /dev/null +++ b/src/test/hive/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 @@ -0,0 +1 @@ +-7158439905 diff --git a/src/test/hive/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 b/src/test/hive/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 b/src/test/hive/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f b/src/test/hive/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/src/test/hive/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 b/src/test/hive/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e b/src/test/hive/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 b/src/test/hive/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 @@ -0,0 +1 @@ +4937935 diff --git a/src/test/hive/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 b/src/test/hive/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 @@ -0,0 +1 @@ +4937935 diff --git a/src/test/hive/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 b/src/test/hive/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 @@ -0,0 +1 @@ +4937935 diff --git a/src/test/hive/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 b/src/test/hive/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 @@ -0,0 +1 @@ +4937935 diff --git a/src/test/hive/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b b/src/test/hive/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e b/src/test/hive/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 b/src/test/hive/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a b/src/test/hive/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d b/src/test/hive/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d @@ -0,0 +1 @@ +4939870 diff --git a/src/test/hive/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 b/src/test/hive/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 b/src/test/hive/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 @@ -0,0 +1 @@ +4939870 diff --git a/src/test/hive/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf b/src/test/hive/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf @@ -0,0 +1 @@ +4939870 diff --git a/src/test/hive/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 b/src/test/hive/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 @@ -0,0 +1 @@ +4939870 diff --git a/src/test/hive/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 b/src/test/hive/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 b/src/test/hive/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 b/src/test/hive/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd b/src/test/hive/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 b/src/test/hive/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 b/src/test/hive/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e b/src/test/hive/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e b/src/test/hive/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 b/src/test/hive/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b b/src/test/hive/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 b/src/test/hive/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c b/src/test/hive/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 b/src/test/hive/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 b/src/test/hive/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac b/src/test/hive/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 b/src/test/hive/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 b/src/test/hive/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada b/src/test/hive/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada @@ -0,0 +1 @@ +4937935 diff --git a/src/test/hive/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 b/src/test/hive/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 @@ -0,0 +1 @@ +4937935 diff --git a/src/test/hive/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 b/src/test/hive/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 b/src/test/hive/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 @@ -0,0 +1 @@ +4939870 diff --git a/src/test/hive/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 b/src/test/hive/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b b/src/test/hive/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 b/src/test/hive/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e b/src/test/hive/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 b/src/test/hive/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 @@ -0,0 +1 @@ +4937935 diff --git a/src/test/hive/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 b/src/test/hive/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 @@ -0,0 +1 @@ +4937935 diff --git a/src/test/hive/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 b/src/test/hive/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 @@ -0,0 +1 @@ +4937935 diff --git a/src/test/hive/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 b/src/test/hive/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 @@ -0,0 +1 @@ +4937935 diff --git a/src/test/hive/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc b/src/test/hive/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b b/src/test/hive/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e b/src/test/hive/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 b/src/test/hive/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a b/src/test/hive/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d b/src/test/hive/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d @@ -0,0 +1 @@ +4939870 diff --git a/src/test/hive/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 b/src/test/hive/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 @@ -0,0 +1 @@ +4939870 diff --git a/src/test/hive/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf b/src/test/hive/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf @@ -0,0 +1 @@ +4939870 diff --git a/src/test/hive/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 b/src/test/hive/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 @@ -0,0 +1 @@ +4939870 diff --git a/src/test/hive/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 b/src/test/hive/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 b/src/test/hive/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 b/src/test/hive/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 @@ -0,0 +1 @@ +4939870 diff --git a/src/test/hive/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 b/src/test/hive/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd b/src/test/hive/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 b/src/test/hive/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 @@ -0,0 +1 @@ +3080335 diff --git a/src/test/hive/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 b/src/test/hive/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 b/src/test/hive/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b b/src/test/hive/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 b/src/test/hive/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/src/test/hive/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 @@ -0,0 +1 @@ +3078400 diff --git a/src/test/hive/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 b/src/test/hive/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 b/src/test/hive/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 new file mode 100644 index 0000000000000..b201b9c4d9348 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 @@ -0,0 +1 @@ +4542003 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 b/src/test/hive/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 new file mode 100644 index 0000000000000..d365cdf04366c --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 @@ -0,0 +1 @@ +4542038 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c b/src/test/hive/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c new file mode 100644 index 0000000000000..bc7bcdca25bfb --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c @@ -0,0 +1 @@ +4543491 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 b/src/test/hive/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 new file mode 100644 index 0000000000000..b201b9c4d9348 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 @@ -0,0 +1 @@ +4542003 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 b/src/test/hive/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 new file mode 100644 index 0000000000000..feea6ee0a8e0d --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 @@ -0,0 +1 @@ +3079923 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 b/src/test/hive/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 new file mode 100644 index 0000000000000..f713b04028bbd --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 @@ -0,0 +1 @@ +4509891 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af b/src/test/hive/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af new file mode 100644 index 0000000000000..a94eda6b2c374 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af @@ -0,0 +1 @@ +3113558 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 b/src/test/hive/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 new file mode 100644 index 0000000000000..feea6ee0a8e0d --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 @@ -0,0 +1 @@ +3079923 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb b/src/test/hive/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb new file mode 100644 index 0000000000000..88c5f95e0d838 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb @@ -0,0 +1 @@ +4543526 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 b/src/test/hive/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 new file mode 100644 index 0000000000000..88c5f95e0d838 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 @@ -0,0 +1 @@ +4543526 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 b/src/test/hive/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d b/src/test/hive/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d new file mode 100644 index 0000000000000..88c5f95e0d838 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d @@ -0,0 +1 @@ +4543526 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a b/src/test/hive/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a new file mode 100644 index 0000000000000..88c5f95e0d838 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a @@ -0,0 +1 @@ +4543526 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 b/src/test/hive/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 new file mode 100644 index 0000000000000..f2ec932ce57f4 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 @@ -0,0 +1 @@ +3112070 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa b/src/test/hive/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa new file mode 100644 index 0000000000000..a94eda6b2c374 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa @@ -0,0 +1 @@ +3113558 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 b/src/test/hive/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 new file mode 100644 index 0000000000000..f2ec932ce57f4 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 @@ -0,0 +1 @@ +3112070 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 b/src/test/hive/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 new file mode 100644 index 0000000000000..4125efd2dd065 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 @@ -0,0 +1 @@ +13630578 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 b/src/test/hive/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 new file mode 100644 index 0000000000000..4125efd2dd065 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 @@ -0,0 +1 @@ +13630578 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc b/src/test/hive/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc new file mode 100644 index 0000000000000..4125efd2dd065 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc @@ -0,0 +1 @@ +13630578 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 b/src/test/hive/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 new file mode 100644 index 0000000000000..e877d44372ecb --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 @@ -0,0 +1 @@ +3078400 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 b/src/test/hive/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 new file mode 100644 index 0000000000000..18be36a9bdb54 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 @@ -0,0 +1 @@ +4509856 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 b/src/test/hive/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 new file mode 100644 index 0000000000000..f2ec932ce57f4 --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 @@ -0,0 +1 @@ +3112070 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e b/src/test/hive/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e new file mode 100644 index 0000000000000..e877d44372ecb --- /dev/null +++ b/src/test/hive/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e @@ -0,0 +1 @@ +3078400 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 b/src/test/hive/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 b/src/test/hive/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 b/src/test/hive/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 b/src/test/hive/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 b/src/test/hive/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 b/src/test/hive/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee b/src/test/hive/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 b/src/test/hive/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b b/src/test/hive/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed b/src/test/hive/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b b/src/test/hive/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 b/src/test/hive/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 b/src/test/hive/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 b/src/test/hive/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 b/src/test/hive/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 new file mode 100644 index 0000000000000..62f9457511f87 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -0,0 +1 @@ +6 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 b/src/test/hive/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 b/src/test/hive/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 new file mode 100644 index 0000000000000..2eafac63a9a98 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 @@ -0,0 +1,6 @@ +0 9 9 +2 1 1 +4 1 1 +5 9 9 +8 1 1 +9 1 1 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 b/src/test/hive/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 b/src/test/hive/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 b/src/test/hive/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 b/src/test/hive/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 b/src/test/hive/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a b/src/test/hive/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 b/src/test/hive/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 b/src/test/hive/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 b/src/test/hive/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 b/src/test/hive/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 b/src/test/hive/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 b/src/test/hive/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d b/src/test/hive/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da b/src/test/hive/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce b/src/test/hive/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce new file mode 100644 index 0000000000000..2ebc6516c7df1 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce @@ -0,0 +1 @@ +56 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce b/src/test/hive/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 b/src/test/hive/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 b/src/test/hive/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b b/src/test/hive/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 b/src/test/hive/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 b/src/test/hive/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e b/src/test/hive/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e new file mode 100644 index 0000000000000..3d2e6576f591f --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e @@ -0,0 +1,22 @@ +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 +4 val_4 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +8 val_8 +9 val_9 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 b/src/test/hive/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 new file mode 100644 index 0000000000000..34d56da297220 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 @@ -0,0 +1,22 @@ +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +2 val_2 val_2 +4 val_4 val_4 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +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 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da b/src/test/hive/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 b/src/test/hive/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e b/src/test/hive/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c b/src/test/hive/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c b/src/test/hive/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e b/src/test/hive/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e new file mode 100644 index 0000000000000..3d2e6576f591f --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e @@ -0,0 +1,22 @@ +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 +4 val_4 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +8 val_8 +9 val_9 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 b/src/test/hive/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 new file mode 100644 index 0000000000000..4a9735f855f96 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 @@ -0,0 +1,6 @@ +0 9 +2 1 +4 1 +5 9 +8 1 +9 1 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f b/src/test/hive/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f b/src/test/hive/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 b/src/test/hive/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 b/src/test/hive/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 b/src/test/hive/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 b/src/test/hive/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc b/src/test/hive/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 b/src/test/hive/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 b/src/test/hive/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/src/test/hive/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db b/src/test/hive/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a b/src/test/hive/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab b/src/test/hive/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 b/src/test/hive/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 b/src/test/hive/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d b/src/test/hive/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/src/test/hive/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 b/src/test/hive/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 b/src/test/hive/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 new file mode 100644 index 0000000000000..86ee83a4a2686 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 @@ -0,0 +1 @@ +40 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a b/src/test/hive/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e b/src/test/hive/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e new file mode 100644 index 0000000000000..301160a93062d --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e @@ -0,0 +1 @@ +8 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da b/src/test/hive/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 b/src/test/hive/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f b/src/test/hive/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f b/src/test/hive/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a b/src/test/hive/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 b/src/test/hive/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 b/src/test/hive/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 b/src/test/hive/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd b/src/test/hive/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 b/src/test/hive/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed b/src/test/hive/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef b/src/test/hive/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e b/src/test/hive/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 b/src/test/hive/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/src/test/hive/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 b/src/test/hive/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa b/src/test/hive/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 b/src/test/hive/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a b/src/test/hive/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab b/src/test/hive/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 b/src/test/hive/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 b/src/test/hive/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d b/src/test/hive/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/src/test/hive/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec b/src/test/hive/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 b/src/test/hive/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 b/src/test/hive/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 b/src/test/hive/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e b/src/test/hive/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a b/src/test/hive/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 b/src/test/hive/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 b/src/test/hive/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/src/test/hive/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 b/src/test/hive/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc b/src/test/hive/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc new file mode 100644 index 0000000000000..83be903e06482 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc @@ -0,0 +1 @@ +570 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db b/src/test/hive/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a b/src/test/hive/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab b/src/test/hive/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 b/src/test/hive/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 b/src/test/hive/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d b/src/test/hive/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/src/test/hive/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 b/src/test/hive/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 b/src/test/hive/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 b/src/test/hive/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 new file mode 100644 index 0000000000000..251ff85eda52d --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 @@ -0,0 +1,22 @@ +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +2 2 +4 4 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +8 8 +9 9 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 b/src/test/hive/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 new file mode 100644 index 0000000000000..af8f457e93476 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 @@ -0,0 +1,22 @@ +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_2 val_2 +val_4 val_4 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +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 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f b/src/test/hive/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 b/src/test/hive/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 b/src/test/hive/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 b/src/test/hive/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 new file mode 100644 index 0000000000000..251ff85eda52d --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 @@ -0,0 +1,22 @@ +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +2 2 +4 4 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +8 8 +9 9 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 b/src/test/hive/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 new file mode 100644 index 0000000000000..af8f457e93476 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 @@ -0,0 +1,22 @@ +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_2 val_2 +val_4 val_4 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +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 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 b/src/test/hive/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 b/src/test/hive/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 b/src/test/hive/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 new file mode 100644 index 0000000000000..251ff85eda52d --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 @@ -0,0 +1,22 @@ +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +2 2 +4 4 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +8 8 +9 9 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da b/src/test/hive/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 b/src/test/hive/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 new file mode 100644 index 0000000000000..af8f457e93476 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 @@ -0,0 +1,22 @@ +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_2 val_2 +val_4 val_4 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +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 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 b/src/test/hive/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 b/src/test/hive/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 b/src/test/hive/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 b/src/test/hive/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb b/src/test/hive/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 b/src/test/hive/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a b/src/test/hive/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a new file mode 100644 index 0000000000000..1758dddccea2b --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a @@ -0,0 +1 @@ +32 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c b/src/test/hive/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 b/src/test/hive/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd b/src/test/hive/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd new file mode 100644 index 0000000000000..bea0d09c49935 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd @@ -0,0 +1 @@ +207 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da b/src/test/hive/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 b/src/test/hive/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac b/src/test/hive/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 b/src/test/hive/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 b/src/test/hive/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 b/src/test/hive/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da b/src/test/hive/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 b/src/test/hive/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac b/src/test/hive/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 b/src/test/hive/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d b/src/test/hive/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 b/src/test/hive/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 b/src/test/hive/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce b/src/test/hive/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/src/test/hive/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/src/test/hive/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 b/src/test/hive/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 b/src/test/hive/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a b/src/test/hive/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 b/src/test/hive/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 b/src/test/hive/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a b/src/test/hive/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab b/src/test/hive/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d b/src/test/hive/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/src/test/hive/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 b/src/test/hive/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 b/src/test/hive/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc b/src/test/hive/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 b/src/test/hive/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 b/src/test/hive/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/src/test/hive/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 b/src/test/hive/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f b/src/test/hive/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db b/src/test/hive/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a b/src/test/hive/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab b/src/test/hive/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 b/src/test/hive/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 b/src/test/hive/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c b/src/test/hive/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 b/src/test/hive/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a b/src/test/hive/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab b/src/test/hive/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 b/src/test/hive/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 b/src/test/hive/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 b/src/test/hive/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/src/test/hive/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a b/src/test/hive/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 b/src/test/hive/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 b/src/test/hive/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f b/src/test/hive/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 b/src/test/hive/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 b/src/test/hive/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 b/src/test/hive/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d b/src/test/hive/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 b/src/test/hive/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 b/src/test/hive/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 b/src/test/hive/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 b/src/test/hive/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..dec2bf5d6199c --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +19 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..dec2bf5d6199c --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +19 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 b/src/test/hive/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..dec2bf5d6199c --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +19 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c b/src/test/hive/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 b/src/test/hive/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 b/src/test/hive/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 b/src/test/hive/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a b/src/test/hive/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 b/src/test/hive/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f b/src/test/hive/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 b/src/test/hive/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 b/src/test/hive/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd b/src/test/hive/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 b/src/test/hive/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c b/src/test/hive/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 b/src/test/hive/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f b/src/test/hive/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b b/src/test/hive/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b b/src/test/hive/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 b/src/test/hive/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 b/src/test/hive/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 b/src/test/hive/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da b/src/test/hive/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf b/src/test/hive/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 b/src/test/hive/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f b/src/test/hive/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b b/src/test/hive/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b b/src/test/hive/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 b/src/test/hive/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 b/src/test/hive/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 b/src/test/hive/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 b/src/test/hive/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 b/src/test/hive/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 b/src/test/hive/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 b/src/test/hive/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 b/src/test/hive/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b b/src/test/hive/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 b/src/test/hive/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 b/src/test/hive/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a b/src/test/hive/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab b/src/test/hive/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d b/src/test/hive/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 b/src/test/hive/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 b/src/test/hive/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/src/test/hive/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 b/src/test/hive/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 b/src/test/hive/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a b/src/test/hive/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 b/src/test/hive/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 b/src/test/hive/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f b/src/test/hive/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 b/src/test/hive/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 b/src/test/hive/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 b/src/test/hive/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa b/src/test/hive/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 b/src/test/hive/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d b/src/test/hive/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 b/src/test/hive/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 b/src/test/hive/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 b/src/test/hive/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc b/src/test/hive/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/src/test/hive/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 b/src/test/hive/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 b/src/test/hive/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 b/src/test/hive/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/src/test/hive/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 b/src/test/hive/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 b/src/test/hive/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 b/src/test/hive/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 b/src/test/hive/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f b/src/test/hive/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db b/src/test/hive/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a b/src/test/hive/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab b/src/test/hive/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 b/src/test/hive/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 b/src/test/hive/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d b/src/test/hive/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 b/src/test/hive/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 b/src/test/hive/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e b/src/test/hive/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 b/src/test/hive/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 new file mode 100644 index 0000000000000..4a9735f855f96 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 @@ -0,0 +1,6 @@ +0 9 +2 1 +4 1 +5 9 +8 1 +9 1 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 b/src/test/hive/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 b/src/test/hive/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 new file mode 100644 index 0000000000000..62f9457511f87 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -0,0 +1 @@ +6 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 b/src/test/hive/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 b/src/test/hive/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 new file mode 100644 index 0000000000000..2eafac63a9a98 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 @@ -0,0 +1,6 @@ +0 9 9 +2 1 1 +4 1 1 +5 9 9 +8 1 1 +9 1 1 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 b/src/test/hive/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 b/src/test/hive/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 b/src/test/hive/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 b/src/test/hive/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 b/src/test/hive/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a b/src/test/hive/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 b/src/test/hive/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 b/src/test/hive/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 b/src/test/hive/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 b/src/test/hive/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da b/src/test/hive/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d b/src/test/hive/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 b/src/test/hive/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 b/src/test/hive/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e b/src/test/hive/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d b/src/test/hive/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce b/src/test/hive/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce new file mode 100644 index 0000000000000..2ebc6516c7df1 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce @@ -0,0 +1 @@ +56 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce b/src/test/hive/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 b/src/test/hive/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 b/src/test/hive/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 b/src/test/hive/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 b/src/test/hive/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 b/src/test/hive/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e b/src/test/hive/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 b/src/test/hive/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 new file mode 100644 index 0000000000000..4a9735f855f96 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 @@ -0,0 +1,6 @@ +0 9 +2 1 +4 1 +5 9 +8 1 +9 1 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 b/src/test/hive/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 b/src/test/hive/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 new file mode 100644 index 0000000000000..62f9457511f87 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -0,0 +1 @@ +6 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 b/src/test/hive/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 b/src/test/hive/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 new file mode 100644 index 0000000000000..2eafac63a9a98 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 @@ -0,0 +1,6 @@ +0 9 9 +2 1 1 +4 1 1 +5 9 9 +8 1 1 +9 1 1 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 b/src/test/hive/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 b/src/test/hive/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 b/src/test/hive/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f b/src/test/hive/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 b/src/test/hive/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 b/src/test/hive/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a b/src/test/hive/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 b/src/test/hive/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 b/src/test/hive/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d b/src/test/hive/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 b/src/test/hive/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 b/src/test/hive/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e b/src/test/hive/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d b/src/test/hive/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f b/src/test/hive/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce b/src/test/hive/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce new file mode 100644 index 0000000000000..2ebc6516c7df1 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce @@ -0,0 +1 @@ +56 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce b/src/test/hive/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 b/src/test/hive/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 b/src/test/hive/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 b/src/test/hive/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 b/src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 new file mode 100644 index 0000000000000..03866397e028e --- /dev/null +++ b/src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 @@ -0,0 +1,4 @@ +ba_key binary None +ba_val binary None + +Detailed Table Information Table(tableName:ba_test, dbName:default, owner:marmbrus, createTime:1392059487, 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:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/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=1392059487}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/src/test/hive/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 b/src/test/hive/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a b/src/test/hive/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 b/src/test/hive/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 b/src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 new file mode 100644 index 0000000000000..f23b62931be46 --- /dev/null +++ b/src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 @@ -0,0 +1,4 @@ +ba_key binary from deserializer +ba_val binary from deserializer + +Detailed Table Information Table(tableName:ba_test, dbName:default, owner:marmbrus, createTime:1392059487, 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:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/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=marmbrus, last_modified_time=1392059487, transient_lastDdlTime=1392059487}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/src/test/hive/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 b/src/test/hive/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d b/src/test/hive/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 b/src/test/hive/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b b/src/test/hive/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 b/src/test/hive/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 b/src/test/hive/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 b/src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 new file mode 100644 index 0000000000000..af60a5daa50a4 --- /dev/null +++ b/src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 @@ -0,0 +1,4 @@ +ba_key binary None +ba_val binary None + +Detailed Table Information Table(tableName:ba_test, dbName:default, owner:marmbrus, createTime:1392059489, 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:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/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=1392059489}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/src/test/hive/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 b/src/test/hive/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 new file mode 100644 index 0000000000000..2e65efe2a145d --- /dev/null +++ b/src/test/hive/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 @@ -0,0 +1 @@ +a \ No newline at end of file diff --git a/src/test/hive/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 b/src/test/hive/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef b/src/test/hive/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 b/src/test/hive/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 b/src/test/hive/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/src/test/hive/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/src/test/hive/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 b/src/test/hive/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e b/src/test/hive/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 b/src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 new file mode 100644 index 0000000000000..9bf06320a9e6b --- /dev/null +++ b/src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 @@ -0,0 +1,4 @@ +ba_key binary from deserializer +ba_val binary from deserializer + +Detailed Table Information Table(tableName:ba_test, dbName:default, owner:marmbrus, createTime:1392059490, 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:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/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=marmbrus, last_modified_time=1392059490, transient_lastDdlTime=1392059490}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/src/test/hive/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/src/test/hive/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 b/src/test/hive/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 b/src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 new file mode 100644 index 0000000000000..f5117886c8b1b --- /dev/null +++ b/src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 @@ -0,0 +1,4 @@ +ba_key binary from deserializer +ba_val binary from deserializer + +Detailed Table Information Table(tableName:ba_test, dbName:default, owner:marmbrus, createTime:1392059491, 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:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/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=1392059491}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/src/test/hive/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 b/src/test/hive/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 b/src/test/hive/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 b/src/test/hive/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 b/src/test/hive/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 new file mode 100644 index 0000000000000..27687b47813a9 --- /dev/null +++ b/src/test/hive/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 @@ -0,0 +1,10 @@ +^@^@^@ 7.0 +^@^A^@ 9.0 +^@test^@ 2.0 +^A^@^A 10.0 +^A^A^A 8.0 +^Atest^A 3.0 +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 diff --git a/src/test/hive/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 b/src/test/hive/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 b/src/test/hive/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c b/src/test/hive/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a b/src/test/hive/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a new file mode 100644 index 0000000000000..9bdd310949be8 --- /dev/null +++ b/src/test/hive/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a @@ -0,0 +1 @@ +5 5.0 5.0 5.0 5 true 1 \ No newline at end of file diff --git a/src/test/hive/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 b/src/test/hive/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 new file mode 100644 index 0000000000000..bbe268ea91ddf --- /dev/null +++ b/src/test/hive/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 @@ -0,0 +1 @@ +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/src/test/hive/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 b/src/test/hive/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 b/src/test/hive/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 new file mode 100644 index 0000000000000..87d0f8dd52b68 --- /dev/null +++ b/src/test/hive/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 @@ -0,0 +1 @@ +10 val_10 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 b/src/test/hive/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e b/src/test/hive/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/src/test/hive/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 b/src/test/hive/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 b/src/test/hive/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/src/test/hive/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 b/src/test/hive/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 b/src/test/hive/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/src/test/hive/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 b/src/test/hive/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 b/src/test/hive/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 new file mode 100644 index 0000000000000..1a018b640eb6f --- /dev/null +++ b/src/test/hive/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 @@ -0,0 +1 @@ +20 val_20 20 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 b/src/test/hive/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 b/src/test/hive/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 new file mode 100644 index 0000000000000..333a4cf9af123 --- /dev/null +++ b/src/test/hive/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 @@ -0,0 +1 @@ +20 val_20 20 val_20 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 b/src/test/hive/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-20-294891c1d956245540a80aa800ba393d b/src/test/hive/golden/cluster-20-294891c1d956245540a80aa800ba393d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f b/src/test/hive/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f new file mode 100644 index 0000000000000..333a4cf9af123 --- /dev/null +++ b/src/test/hive/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f @@ -0,0 +1 @@ +20 val_20 20 val_20 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 b/src/test/hive/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e b/src/test/hive/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e new file mode 100644 index 0000000000000..1a018b640eb6f --- /dev/null +++ b/src/test/hive/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e @@ -0,0 +1 @@ +20 val_20 20 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae b/src/test/hive/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a b/src/test/hive/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a new file mode 100644 index 0000000000000..a79654385b09f --- /dev/null +++ b/src/test/hive/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a @@ -0,0 +1,498 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f b/src/test/hive/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/src/test/hive/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc b/src/test/hive/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 b/src/test/hive/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/src/test/hive/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 b/src/test/hive/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 b/src/test/hive/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/src/test/hive/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/src/test/hive/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d b/src/test/hive/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 b/src/test/hive/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/src/test/hive/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/src/test/hive/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d b/src/test/hive/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c b/src/test/hive/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 b/src/test/hive/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f b/src/test/hive/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f new file mode 100644 index 0000000000000..15e92afeeca27 --- /dev/null +++ b/src/test/hive/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f @@ -0,0 +1,11 @@ +[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/src/test/hive/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c b/src/test/hive/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 b/src/test/hive/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 b/src/test/hive/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd b/src/test/hive/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 b/src/test/hive/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 new file mode 100644 index 0000000000000..6ad4db1788424 --- /dev/null +++ b/src/test/hive/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 @@ -0,0 +1,58 @@ +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 + 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 + 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/src/test/hive/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 b/src/test/hive/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 new file mode 100644 index 0000000000000..91ce2a521cde1 --- /dev/null +++ b/src/test/hive/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 @@ -0,0 +1,129 @@ +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 + 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 + 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 + 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/src/test/hive/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 b/src/test/hive/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 b/src/test/hive/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 new file mode 100644 index 0000000000000..777024f6946e3 --- /dev/null +++ b/src/test/hive/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 @@ -0,0 +1,58 @@ +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 + 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 + 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/src/test/hive/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce b/src/test/hive/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce new file mode 100644 index 0000000000000..cd72c7efbf56f --- /dev/null +++ b/src/test/hive/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce @@ -0,0 +1,129 @@ +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 + 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 + 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 + 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/src/test/hive/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe b/src/test/hive/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 b/src/test/hive/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c b/src/test/hive/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 b/src/test/hive/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 b/src/test/hive/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 new file mode 100644 index 0000000000000..d1e5e7375467d --- /dev/null +++ b/src/test/hive/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 @@ -0,0 +1,73 @@ +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 + expr: _col1 + type: struct + expr: _col2 + type: struct + 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 + expr: _col1 + type: struct + expr: _col2 + type: struct + 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/src/test/hive/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 b/src/test/hive/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 new file mode 100644 index 0000000000000..3f3aa581b43f9 --- /dev/null +++ b/src/test/hive/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 @@ -0,0 +1,141 @@ +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 + expr: _col1 + type: struct + expr: _col2 + type: struct + 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 + expr: _col1 + type: struct + expr: _col2 + type: struct + 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:struct:struct + 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/src/test/hive/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c b/src/test/hive/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 b/src/test/hive/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d b/src/test/hive/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d new file mode 100644 index 0000000000000..4ff444febde63 --- /dev/null +++ b/src/test/hive/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d @@ -0,0 +1,89 @@ +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 + expr: _col1 + type: struct + expr: _col2 + type: struct + expr: _col3 + type: struct + expr: _col4 + type: struct + 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 + expr: _col1 + type: struct + expr: _col2 + type: struct + expr: _col3 + type: struct + expr: _col4 + type: struct + 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/src/test/hive/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f b/src/test/hive/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 b/src/test/hive/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf b/src/test/hive/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 b/src/test/hive/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 b/src/test/hive/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea b/src/test/hive/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine1-6-1d1f97cce07323812de3027920b04b75 b/src/test/hive/golden/combine1-6-1d1f97cce07323812de3027920b04b75 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine1-6-1d1f97cce07323812de3027920b04b75 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 b/src/test/hive/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c b/src/test/hive/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 b/src/test/hive/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 new file mode 100644 index 0000000000000..8f8e1f4b21fe3 --- /dev/null +++ b/src/test/hive/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 @@ -0,0 +1,500 @@ +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 +105 val_105 +11 val_11 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +12 val_12 +12 val_12 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +15 val_15 +15 val_15 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +17 val_17 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +18 val_18 +18 val_18 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +19 val_19 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +2 val_2 +20 val_20 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +24 val_24 +24 val_24 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +26 val_26 +26 val_26 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +27 val_27 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +28 val_28 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +30 val_30 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +33 val_33 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +34 val_34 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +35 val_35 +35 val_35 +35 val_35 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +37 val_37 +37 val_37 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +4 val_4 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +41 val_41 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +42 val_42 +42 val_42 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +43 val_43 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +44 val_44 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +47 val_47 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 +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 \ No newline at end of file diff --git a/src/test/hive/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 b/src/test/hive/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a b/src/test/hive/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 b/src/test/hive/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2-12-cd15ffd140539cf86090814729ec4748 b/src/test/hive/golden/combine2-12-cd15ffd140539cf86090814729ec4748 new file mode 100644 index 0000000000000..4c538dc5ccda1 --- /dev/null +++ b/src/test/hive/golden/combine2-12-cd15ffd140539cf86090814729ec4748 @@ -0,0 +1,8 @@ +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=| diff --git a/src/test/hive/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 b/src/test/hive/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c b/src/test/hive/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c new file mode 100644 index 0000000000000..d492cb3452d87 --- /dev/null +++ b/src/test/hive/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c @@ -0,0 +1,12 @@ +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 diff --git a/src/test/hive/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 b/src/test/hive/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2-16-557997716a68312e8cae75428e3ce31 b/src/test/hive/golden/combine2-16-557997716a68312e8cae75428e3ce31 new file mode 100644 index 0000000000000..48082f72f087c --- /dev/null +++ b/src/test/hive/golden/combine2-16-557997716a68312e8cae75428e3ce31 @@ -0,0 +1 @@ +12 diff --git a/src/test/hive/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 b/src/test/hive/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 b/src/test/hive/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 new file mode 100644 index 0000000000000..41be2c5b41f01 --- /dev/null +++ b/src/test/hive/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 @@ -0,0 +1,2 @@ +2008-04-08 1000 +2008-04-09 1000 diff --git a/src/test/hive/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf b/src/test/hive/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/src/test/hive/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 b/src/test/hive/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/src/test/hive/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine2-7-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/combine2-7-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine2-7-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 b/src/test/hive/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 b/src/test/hive/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/src/test/hive/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 b/src/test/hive/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 b/src/test/hive/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 b/src/test/hive/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 new file mode 100644 index 0000000000000..80fa68b84c17e --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 @@ -0,0 +1,8 @@ +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/src/test/hive/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 b/src/test/hive/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c b/src/test/hive/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c new file mode 100644 index 0000000000000..ff6141674e603 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c @@ -0,0 +1,12 @@ +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/src/test/hive/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 b/src/test/hive/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 b/src/test/hive/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 new file mode 100644 index 0000000000000..3cacc0b93c9c9 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 @@ -0,0 +1 @@ +12 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 b/src/test/hive/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 b/src/test/hive/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 new file mode 100644 index 0000000000000..1a0aa74952afa --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 @@ -0,0 +1,2 @@ +2008-04-08 1000 +2008-04-09 1000 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf b/src/test/hive/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 b/src/test/hive/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 b/src/test/hive/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea b/src/test/hive/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 b/src/test/hive/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 b/src/test/hive/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf b/src/test/hive/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 b/src/test/hive/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 b/src/test/hive/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 new file mode 100644 index 0000000000000..80fa68b84c17e --- /dev/null +++ b/src/test/hive/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 @@ -0,0 +1,8 @@ +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/src/test/hive/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 b/src/test/hive/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c b/src/test/hive/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c new file mode 100644 index 0000000000000..ff6141674e603 --- /dev/null +++ b/src/test/hive/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c @@ -0,0 +1,12 @@ +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/src/test/hive/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 b/src/test/hive/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 b/src/test/hive/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 new file mode 100644 index 0000000000000..3cacc0b93c9c9 --- /dev/null +++ b/src/test/hive/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 @@ -0,0 +1 @@ +12 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 b/src/test/hive/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 b/src/test/hive/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 new file mode 100644 index 0000000000000..1a0aa74952afa --- /dev/null +++ b/src/test/hive/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 @@ -0,0 +1,2 @@ +2008-04-08 1000 +2008-04-09 1000 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 b/src/test/hive/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 b/src/test/hive/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea b/src/test/hive/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 b/src/test/hive/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 b/src/test/hive/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af b/src/test/hive/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 b/src/test/hive/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 b/src/test/hive/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 b/src/test/hive/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 new file mode 100644 index 0000000000000..80665a4d4c983 --- /dev/null +++ b/src/test/hive/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 @@ -0,0 +1,12 @@ +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/src/test/hive/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b b/src/test/hive/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b new file mode 100644 index 0000000000000..5a87a3aec7cf5 --- /dev/null +++ b/src/test/hive/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b @@ -0,0 +1,12 @@ +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/src/test/hive/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc b/src/test/hive/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc new file mode 100644 index 0000000000000..7b6455db7834b --- /dev/null +++ b/src/test/hive/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc @@ -0,0 +1,30 @@ +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/src/test/hive/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 b/src/test/hive/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a b/src/test/hive/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b b/src/test/hive/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f b/src/test/hive/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf b/src/test/hive/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd b/src/test/hive/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 b/src/test/hive/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 new file mode 100644 index 0000000000000..d57cb5369e219 --- /dev/null +++ b/src/test/hive/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 @@ -0,0 +1,30 @@ +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/src/test/hive/golden/combine3-22-11025483569617a9f014b5defd71e933 b/src/test/hive/golden/combine3-22-11025483569617a9f014b5defd71e933 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-23-4725c48df09565618cbffd05953a5f62 b/src/test/hive/golden/combine3-23-4725c48df09565618cbffd05953a5f62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 b/src/test/hive/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 b/src/test/hive/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea b/src/test/hive/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/combine3-6-4725c48df09565618cbffd05953a5f62 b/src/test/hive/golden/combine3-6-4725c48df09565618cbffd05953a5f62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 b/src/test/hive/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b b/src/test/hive/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 b/src/test/hive/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 b/src/test/hive/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 b/src/test/hive/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c b/src/test/hive/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/src/test/hive/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c @@ -0,0 +1 @@ +10 diff --git a/src/test/hive/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 b/src/test/hive/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 new file mode 100644 index 0000000000000..fe2e2d7663026 --- /dev/null +++ b/src/test/hive/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 @@ -0,0 +1 @@ +{"columntype":"Binary","maxlength":36,"avglength":20.0,"countnulls":0} diff --git a/src/test/hive/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 b/src/test/hive/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 b/src/test/hive/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e b/src/test/hive/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e new file mode 100644 index 0000000000000..dc7b54ad01435 --- /dev/null +++ b/src/test/hive/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e @@ -0,0 +1 @@ +33 \ No newline at end of file diff --git a/src/test/hive/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d b/src/test/hive/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d new file mode 100644 index 0000000000000..dd487e6fea3ff --- /dev/null +++ b/src/test/hive/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d @@ -0,0 +1 @@ +{"columntype":"Boolean","counttrues":13,"countfalses":19,"countnulls":1} \ No newline at end of file diff --git a/src/test/hive/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce b/src/test/hive/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 b/src/test/hive/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 b/src/test/hive/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 new file mode 100644 index 0000000000000..b6a7d89c68e0c --- /dev/null +++ b/src/test/hive/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 @@ -0,0 +1 @@ +16 diff --git a/src/test/hive/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 b/src/test/hive/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 new file mode 100644 index 0000000000000..31a1d0792beeb --- /dev/null +++ b/src/test/hive/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 @@ -0,0 +1 @@ +{"columntype":"Double","min":-87.2,"max":435.33,"countnulls":2,"numdistinctvalues":11} diff --git a/src/test/hive/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d b/src/test/hive/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 b/src/test/hive/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f b/src/test/hive/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f new file mode 100644 index 0000000000000..8d4251bcc08c8 --- /dev/null +++ b/src/test/hive/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f @@ -0,0 +1 @@ +{"columntype":"Boolean","counttrues":0,"countfalses":0,"countnulls":0} diff --git a/src/test/hive/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b b/src/test/hive/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b new file mode 100644 index 0000000000000..ce543362c3ffc --- /dev/null +++ b/src/test/hive/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b @@ -0,0 +1 @@ +{"columntype":"Long","min":0,"max":0,"countnulls":0,"numdistinctvalues":0} diff --git a/src/test/hive/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf b/src/test/hive/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf new file mode 100644 index 0000000000000..51062737dd0de --- /dev/null +++ b/src/test/hive/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf @@ -0,0 +1 @@ +{"columntype":"Double","min":0.0,"max":0.0,"countnulls":0,"numdistinctvalues":0} diff --git a/src/test/hive/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 b/src/test/hive/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 new file mode 100644 index 0000000000000..9a834007cec70 --- /dev/null +++ b/src/test/hive/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 @@ -0,0 +1 @@ +{"columntype":"String","maxlength":0,"avglength":0.0,"countnulls":0,"numdistinctvalues":0} diff --git a/src/test/hive/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 b/src/test/hive/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 new file mode 100644 index 0000000000000..f6c0cb997b2a6 --- /dev/null +++ b/src/test/hive/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 @@ -0,0 +1 @@ +{"columntype":"Binary","maxlength":0,"avglength":0.0,"countnulls":0} diff --git a/src/test/hive/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 b/src/test/hive/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 b/src/test/hive/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 b/src/test/hive/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 new file mode 100644 index 0000000000000..48082f72f087c --- /dev/null +++ b/src/test/hive/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 @@ -0,0 +1 @@ +12 diff --git a/src/test/hive/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d b/src/test/hive/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d new file mode 100644 index 0000000000000..cb4c03d1da303 --- /dev/null +++ b/src/test/hive/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d @@ -0,0 +1 @@ +{"columntype":"Long","min":0,"max":344,"countnulls":1,"numdistinctvalues":11} diff --git a/src/test/hive/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b b/src/test/hive/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f b/src/test/hive/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a b/src/test/hive/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/src/test/hive/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a @@ -0,0 +1 @@ +10 diff --git a/src/test/hive/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc b/src/test/hive/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc new file mode 100644 index 0000000000000..a8fc88cbd539d --- /dev/null +++ b/src/test/hive/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc @@ -0,0 +1 @@ +{"columntype":"String","maxlength":11,"avglength":3.9,"countnulls":0,"numdistinctvalues":7} diff --git a/src/test/hive/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 b/src/test/hive/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 b/src/test/hive/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 new file mode 100644 index 0000000000000..c615f0148ccd1 --- /dev/null +++ b/src/test/hive/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 @@ -0,0 +1,27 @@ +my_bool boolean from deserializer +my_byte tinyint from deserializer +my_16bit_int smallint from deserializer +my_32bit_int int from deserializer +my_64bit_int bigint from deserializer +my_double double from deserializer +my_string string from deserializer +my_binary struct from deserializer +my_string_string_map map from deserializer +my_string_enum_map map from deserializer +my_enum_string_map map from deserializer +my_enum_struct_map map>> from deserializer +my_enum_stringlist_map map> from deserializer +my_enum_structlist_map map>>> from deserializer +my_stringlist array from deserializer +my_structlist array>> from deserializer +my_enumlist array from deserializer +my_stringset struct<> from deserializer +my_enumset struct<> from deserializer +my_structset struct<> from deserializer +optionals struct<> from deserializer +b string None + +# Partition Information +# col_name data_type comment + +b string None \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 b/src/test/hive/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 b/src/test/hive/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 b/src/test/hive/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 new file mode 100644 index 0000000000000..7c33b34887d6b --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 @@ -0,0 +1,15 @@ +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 +66 1 +98 1 diff --git a/src/test/hive/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d b/src/test/hive/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 b/src/test/hive/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 new file mode 100644 index 0000000000000..7c33b34887d6b --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 @@ -0,0 +1,15 @@ +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 +66 1 +98 1 diff --git a/src/test/hive/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 b/src/test/hive/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 b/src/test/hive/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 b/src/test/hive/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 b/src/test/hive/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 new file mode 100644 index 0000000000000..b97104d25bb51 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 @@ -0,0 +1,2 @@ +66 1 +98 2 diff --git a/src/test/hive/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 b/src/test/hive/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 b/src/test/hive/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 new file mode 100644 index 0000000000000..b97104d25bb51 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 @@ -0,0 +1,2 @@ +66 1 +98 2 diff --git a/src/test/hive/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 b/src/test/hive/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 b/src/test/hive/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda b/src/test/hive/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda new file mode 100644 index 0000000000000..e8eb644933e82 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda @@ -0,0 +1,37 @@ +66 val_66 66 val_66 +98 val_98 98 val_98 +98 val_98 98 val_98 +128 val_128 128 +128 val_128 128 +128 val_128 128 +146 val_146 146 val_146 +146 val_146 146 val_146 +150 val_150 150 val_150 +213 val_213 213 val_213 +213 val_213 213 val_213 +224 val_224 224 +224 val_224 224 +238 val_238 238 val_238 +238 val_238 238 val_238 +255 val_255 255 val_255 +255 val_255 255 val_255 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +278 val_278 278 val_278 +278 val_278 278 val_278 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +369 val_369 369 +369 val_369 369 +369 val_369 369 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 diff --git a/src/test/hive/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 b/src/test/hive/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 b/src/test/hive/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf b/src/test/hive/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 b/src/test/hive/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 new file mode 100644 index 0000000000000..e8eb644933e82 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 @@ -0,0 +1,37 @@ +66 val_66 66 val_66 +98 val_98 98 val_98 +98 val_98 98 val_98 +128 val_128 128 +128 val_128 128 +128 val_128 128 +146 val_146 146 val_146 +146 val_146 146 val_146 +150 val_150 150 val_150 +213 val_213 213 val_213 +213 val_213 213 val_213 +224 val_224 224 +224 val_224 224 +238 val_238 238 val_238 +238 val_238 238 val_238 +255 val_255 255 val_255 +255 val_255 255 val_255 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +278 val_278 278 val_278 +278 val_278 278 val_278 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +369 val_369 369 +369 val_369 369 +369 val_369 369 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 diff --git a/src/test/hive/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 b/src/test/hive/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 b/src/test/hive/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 new file mode 100644 index 0000000000000..e8eb644933e82 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 @@ -0,0 +1,37 @@ +66 val_66 66 val_66 +98 val_98 98 val_98 +98 val_98 98 val_98 +128 val_128 128 +128 val_128 128 +128 val_128 128 +146 val_146 146 val_146 +146 val_146 146 val_146 +150 val_150 150 val_150 +213 val_213 213 val_213 +213 val_213 213 val_213 +224 val_224 224 +224 val_224 224 +238 val_238 238 val_238 +238 val_238 238 val_238 +255 val_255 255 val_255 +255 val_255 255 val_255 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +278 val_278 278 val_278 +278 val_278 278 val_278 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +369 val_369 369 +369 val_369 369 +369 val_369 369 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 diff --git a/src/test/hive/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 b/src/test/hive/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 b/src/test/hive/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 new file mode 100644 index 0000000000000..0a6a1f70d34ef --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 @@ -0,0 +1,37 @@ +66 val_66 66 1 +98 val_98 98 1 +98 val_98 98 1 +128 val_128 128 1 +128 val_128 128 1 +128 val_128 128 1 +146 val_146 146 1 +146 val_146 146 1 +150 val_150 150 1 +213 val_213 213 1 +213 val_213 213 1 +224 val_224 224 1 +224 val_224 224 1 +238 val_238 238 1 +238 val_238 238 1 +255 val_255 255 1 +255 val_255 255 1 +273 val_273 273 1 +273 val_273 273 1 +273 val_273 273 1 +278 val_278 278 1 +278 val_278 278 1 +311 val_311 311 1 +311 val_311 311 1 +311 val_311 311 1 +369 val_369 369 1 +369 val_369 369 1 +369 val_369 369 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +406 val_406 406 1 +406 val_406 406 1 +406 val_406 406 1 +406 val_406 406 1 diff --git a/src/test/hive/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c b/src/test/hive/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 b/src/test/hive/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 new file mode 100644 index 0000000000000..0a6a1f70d34ef --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 @@ -0,0 +1,37 @@ +66 val_66 66 1 +98 val_98 98 1 +98 val_98 98 1 +128 val_128 128 1 +128 val_128 128 1 +128 val_128 128 1 +146 val_146 146 1 +146 val_146 146 1 +150 val_150 150 1 +213 val_213 213 1 +213 val_213 213 1 +224 val_224 224 1 +224 val_224 224 1 +238 val_238 238 1 +238 val_238 238 1 +255 val_255 255 1 +255 val_255 255 1 +273 val_273 273 1 +273 val_273 273 1 +273 val_273 273 1 +278 val_278 278 1 +278 val_278 278 1 +311 val_311 311 1 +311 val_311 311 1 +311 val_311 311 1 +369 val_369 369 1 +369 val_369 369 1 +369 val_369 369 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +406 val_406 406 1 +406 val_406 406 1 +406 val_406 406 1 +406 val_406 406 1 diff --git a/src/test/hive/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f b/src/test/hive/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f b/src/test/hive/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c b/src/test/hive/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda b/src/test/hive/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda new file mode 100644 index 0000000000000..e8eb644933e82 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda @@ -0,0 +1,37 @@ +66 val_66 66 val_66 +98 val_98 98 val_98 +98 val_98 98 val_98 +128 val_128 128 +128 val_128 128 +128 val_128 128 +146 val_146 146 val_146 +146 val_146 146 val_146 +150 val_150 150 val_150 +213 val_213 213 val_213 +213 val_213 213 val_213 +224 val_224 224 +224 val_224 224 +238 val_238 238 val_238 +238 val_238 238 val_238 +255 val_255 255 val_255 +255 val_255 255 val_255 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +278 val_278 278 val_278 +278 val_278 278 val_278 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +369 val_369 369 +369 val_369 369 +369 val_369 369 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 diff --git a/src/test/hive/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 b/src/test/hive/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 b/src/test/hive/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 b/src/test/hive/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 new file mode 100644 index 0000000000000..0b1ee37d7831c --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 @@ -0,0 +1,37 @@ +66 1 66 +98 1 98 +98 1 98 +128 1 128 +128 1 128 +128 1 128 +146 1 146 +146 1 146 +150 1 150 +213 1 213 +213 1 213 +224 1 224 +224 1 224 +238 1 238 +238 1 238 +255 1 255 +255 1 255 +273 1 273 +273 1 273 +273 1 273 +278 1 278 +278 1 278 +311 1 311 +311 1 311 +311 1 311 +369 1 369 +369 1 369 +369 1 369 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +406 1 406 +406 1 406 +406 1 406 +406 1 406 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 b/src/test/hive/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd b/src/test/hive/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 b/src/test/hive/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 new file mode 100644 index 0000000000000..0b1ee37d7831c --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 @@ -0,0 +1,37 @@ +66 1 66 +98 1 98 +98 1 98 +128 1 128 +128 1 128 +128 1 128 +146 1 146 +146 1 146 +150 1 150 +213 1 213 +213 1 213 +224 1 224 +224 1 224 +238 1 238 +238 1 238 +255 1 255 +255 1 255 +273 1 273 +273 1 273 +273 1 273 +278 1 278 +278 1 278 +311 1 311 +311 1 311 +311 1 311 +369 1 369 +369 1 369 +369 1 369 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +406 1 406 +406 1 406 +406 1 406 +406 1 406 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab b/src/test/hive/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 b/src/test/hive/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 new file mode 100644 index 0000000000000..6c6dc691bbff2 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 @@ -0,0 +1 @@ +79136 500 3556 15 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd b/src/test/hive/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 b/src/test/hive/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 new file mode 100644 index 0000000000000..f4bb720dfd7f8 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 @@ -0,0 +1 @@ +3556 37 3556 25 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 b/src/test/hive/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 b/src/test/hive/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 new file mode 100644 index 0000000000000..f4bb720dfd7f8 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 @@ -0,0 +1 @@ +3556 37 3556 25 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 b/src/test/hive/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf b/src/test/hive/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e b/src/test/hive/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e new file mode 100644 index 0000000000000..4acbb60e81661 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e @@ -0,0 +1 @@ +79136 500 3556 25 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b b/src/test/hive/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e b/src/test/hive/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e new file mode 100644 index 0000000000000..4acbb60e81661 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e @@ -0,0 +1 @@ +79136 500 3556 25 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 b/src/test/hive/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 b/src/test/hive/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 new file mode 100644 index 0000000000000..a95fce80fd7b4 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 @@ -0,0 +1 @@ +79136 310 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab b/src/test/hive/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d b/src/test/hive/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d new file mode 100644 index 0000000000000..c6243d7056353 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d @@ -0,0 +1 @@ +3556 37 3556 15 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 b/src/test/hive/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 new file mode 100644 index 0000000000000..a95fce80fd7b4 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 @@ -0,0 +1 @@ +79136 310 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f b/src/test/hive/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b b/src/test/hive/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b new file mode 100644 index 0000000000000..556b77ecfc9eb --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b @@ -0,0 +1 @@ +9992 3531902962 9992 37 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 b/src/test/hive/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b b/src/test/hive/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b new file mode 100644 index 0000000000000..556b77ecfc9eb --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b @@ -0,0 +1 @@ +9992 3531902962 9992 37 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 b/src/test/hive/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d b/src/test/hive/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d new file mode 100644 index 0000000000000..c6243d7056353 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d @@ -0,0 +1 @@ +3556 37 3556 15 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 b/src/test/hive/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 b/src/test/hive/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 new file mode 100644 index 0000000000000..6c6dc691bbff2 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 @@ -0,0 +1 @@ +79136 500 3556 15 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 b/src/test/hive/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 b/src/test/hive/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 b/src/test/hive/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 b/src/test/hive/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 b/src/test/hive/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace b/src/test/hive/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe b/src/test/hive/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 b/src/test/hive/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 b/src/test/hive/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 b/src/test/hive/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 b/src/test/hive/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea b/src/test/hive/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 b/src/test/hive/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e b/src/test/hive/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e b/src/test/hive/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e new file mode 100644 index 0000000000000..14e309fdcad89 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -0,0 +1 @@ +13 10 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 b/src/test/hive/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e b/src/test/hive/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e new file mode 100644 index 0000000000000..14e309fdcad89 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -0,0 +1 @@ +13 10 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb b/src/test/hive/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 b/src/test/hive/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 b/src/test/hive/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 new file mode 100644 index 0000000000000..1b8c137073e37 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 @@ -0,0 +1 @@ +22 12 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 b/src/test/hive/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 b/src/test/hive/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 new file mode 100644 index 0000000000000..1b8c137073e37 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 @@ -0,0 +1 @@ +22 12 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf b/src/test/hive/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b b/src/test/hive/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b new file mode 100644 index 0000000000000..e6c95e600a2c0 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b @@ -0,0 +1 @@ +13 12 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb b/src/test/hive/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf b/src/test/hive/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf new file mode 100644 index 0000000000000..0248cc90cb2c6 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf @@ -0,0 +1 @@ +21 12 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 b/src/test/hive/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf b/src/test/hive/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf new file mode 100644 index 0000000000000..0248cc90cb2c6 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf @@ -0,0 +1 @@ +21 12 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 b/src/test/hive/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 b/src/test/hive/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 new file mode 100644 index 0000000000000..0248cc90cb2c6 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 @@ -0,0 +1 @@ +21 12 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 b/src/test/hive/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 b/src/test/hive/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 new file mode 100644 index 0000000000000..fd3a42ebe67e7 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 @@ -0,0 +1 @@ +21 14 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 b/src/test/hive/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 b/src/test/hive/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 b/src/test/hive/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 new file mode 100644 index 0000000000000..fd3a42ebe67e7 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 @@ -0,0 +1 @@ +21 14 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 b/src/test/hive/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 b/src/test/hive/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e b/src/test/hive/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e new file mode 100644 index 0000000000000..14e309fdcad89 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -0,0 +1 @@ +13 10 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 b/src/test/hive/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 new file mode 100644 index 0000000000000..19b8a2aea8f64 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 @@ -0,0 +1,15 @@ +66 1 66 1 +98 1 98 4 +128 1 128 9 +146 1 146 4 +150 1 150 1 +213 1 213 4 +224 1 224 4 +238 1 238 4 +255 1 255 4 +273 1 273 9 +278 1 278 4 +311 1 311 9 +369 1 369 9 +401 1 401 25 +406 1 406 16 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 b/src/test/hive/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db b/src/test/hive/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db new file mode 100644 index 0000000000000..c4a418f59625b --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db @@ -0,0 +1,15 @@ +66 66 1 +98 98 2 +128 128 3 +146 146 2 +150 150 1 +213 213 2 +224 224 2 +238 238 2 +255 255 2 +273 273 3 +278 278 2 +311 311 3 +369 369 3 +401 401 5 +406 406 4 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 b/src/test/hive/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db b/src/test/hive/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db new file mode 100644 index 0000000000000..c4a418f59625b --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db @@ -0,0 +1,15 @@ +66 66 1 +98 98 2 +128 128 3 +146 146 2 +150 150 1 +213 213 2 +224 224 2 +238 238 2 +255 255 2 +273 273 3 +278 278 2 +311 311 3 +369 369 3 +401 401 5 +406 406 4 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 b/src/test/hive/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 b/src/test/hive/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/src/test/hive/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de new file mode 100644 index 0000000000000..9c8189500649e --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de @@ -0,0 +1,15 @@ +66 66 1 +98 98 4 +128 128 9 +146 146 4 +150 150 1 +213 213 4 +224 224 4 +238 238 4 +255 255 4 +273 273 9 +278 278 4 +311 311 9 +369 369 9 +401 401 25 +406 406 16 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 b/src/test/hive/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/src/test/hive/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de new file mode 100644 index 0000000000000..9c8189500649e --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de @@ -0,0 +1,15 @@ +66 66 1 +98 98 4 +128 128 9 +146 146 4 +150 150 1 +213 213 4 +224 224 4 +238 238 4 +255 255 4 +273 273 9 +278 278 4 +311 311 9 +369 369 9 +401 401 25 +406 406 16 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 b/src/test/hive/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 b/src/test/hive/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 new file mode 100644 index 0000000000000..0b1ee37d7831c --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 @@ -0,0 +1,37 @@ +66 1 66 +98 1 98 +98 1 98 +128 1 128 +128 1 128 +128 1 128 +146 1 146 +146 1 146 +150 1 150 +213 1 213 +213 1 213 +224 1 224 +224 1 224 +238 1 238 +238 1 238 +255 1 255 +255 1 255 +273 1 273 +273 1 273 +273 1 273 +278 1 278 +278 1 278 +311 1 311 +311 1 311 +311 1 311 +369 1 369 +369 1 369 +369 1 369 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +406 1 406 +406 1 406 +406 1 406 +406 1 406 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd b/src/test/hive/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 b/src/test/hive/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 new file mode 100644 index 0000000000000..19b8a2aea8f64 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 @@ -0,0 +1,15 @@ +66 1 66 1 +98 1 98 4 +128 1 128 9 +146 1 146 4 +150 1 150 1 +213 1 213 4 +224 1 224 4 +238 1 238 4 +255 1 255 4 +273 1 273 9 +278 1 278 4 +311 1 311 9 +369 1 369 9 +401 1 401 25 +406 1 406 16 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 b/src/test/hive/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 new file mode 100644 index 0000000000000..0b1ee37d7831c --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 @@ -0,0 +1,37 @@ +66 1 66 +98 1 98 +98 1 98 +128 1 128 +128 1 128 +128 1 128 +146 1 146 +146 1 146 +150 1 150 +213 1 213 +213 1 213 +224 1 224 +224 1 224 +238 1 238 +238 1 238 +255 1 255 +255 1 255 +273 1 273 +273 1 273 +273 1 273 +278 1 278 +278 1 278 +311 1 311 +311 1 311 +311 1 311 +369 1 369 +369 1 369 +369 1 369 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +406 1 406 +406 1 406 +406 1 406 +406 1 406 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 b/src/test/hive/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 b/src/test/hive/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 new file mode 100644 index 0000000000000..7c591d6146fd6 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 @@ -0,0 +1,37 @@ +66 66 1 +98 98 4 +98 98 4 +128 128 9 +128 128 9 +128 128 9 +146 146 4 +146 146 4 +150 150 1 +213 213 4 +213 213 4 +224 224 4 +224 224 4 +238 238 4 +238 238 4 +255 255 4 +255 255 4 +273 273 9 +273 273 9 +273 273 9 +278 278 4 +278 278 4 +311 311 9 +311 311 9 +311 311 9 +369 369 9 +369 369 9 +369 369 9 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +406 406 16 +406 406 16 +406 406 16 +406 406 16 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 b/src/test/hive/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 b/src/test/hive/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 new file mode 100644 index 0000000000000..7c591d6146fd6 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 @@ -0,0 +1,37 @@ +66 66 1 +98 98 4 +98 98 4 +128 128 9 +128 128 9 +128 128 9 +146 146 4 +146 146 4 +150 150 1 +213 213 4 +213 213 4 +224 224 4 +224 224 4 +238 238 4 +238 238 4 +255 255 4 +255 255 4 +273 273 9 +273 273 9 +273 273 9 +278 278 4 +278 278 4 +311 311 9 +311 311 9 +311 311 9 +369 369 9 +369 369 9 +369 369 9 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +406 406 16 +406 406 16 +406 406 16 +406 406 16 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb b/src/test/hive/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 b/src/test/hive/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 new file mode 100644 index 0000000000000..7c591d6146fd6 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 @@ -0,0 +1,37 @@ +66 66 1 +98 98 4 +98 98 4 +128 128 9 +128 128 9 +128 128 9 +146 146 4 +146 146 4 +150 150 1 +213 213 4 +213 213 4 +224 224 4 +224 224 4 +238 238 4 +238 238 4 +255 255 4 +255 255 4 +273 273 9 +273 273 9 +273 273 9 +278 278 4 +278 278 4 +311 311 9 +311 311 9 +311 311 9 +369 369 9 +369 369 9 +369 369 9 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +406 406 16 +406 406 16 +406 406 16 +406 406 16 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 b/src/test/hive/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 b/src/test/hive/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 new file mode 100644 index 0000000000000..7c591d6146fd6 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 @@ -0,0 +1,37 @@ +66 66 1 +98 98 4 +98 98 4 +128 128 9 +128 128 9 +128 128 9 +146 146 4 +146 146 4 +150 150 1 +213 213 4 +213 213 4 +224 224 4 +224 224 4 +238 238 4 +238 238 4 +255 255 4 +255 255 4 +273 273 9 +273 273 9 +273 273 9 +278 278 4 +278 278 4 +311 311 9 +311 311 9 +311 311 9 +369 369 9 +369 369 9 +369 369 9 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +406 406 16 +406 406 16 +406 406 16 +406 406 16 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb b/src/test/hive/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 b/src/test/hive/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 new file mode 100644 index 0000000000000..8338433cd5a27 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 @@ -0,0 +1,37 @@ +66 1 1 66 val_66 +98 2 1 98 val_98 +98 2 1 98 val_98 +128 3 1 128 val_128 +128 3 1 128 val_128 +128 3 1 128 val_128 +146 2 1 146 val_146 +146 2 1 146 val_146 +150 1 1 150 val_150 +213 2 1 213 val_213 +213 2 1 213 val_213 +224 2 1 224 val_224 +224 2 1 224 val_224 +238 2 1 238 val_238 +238 2 1 238 val_238 +255 2 1 255 val_255 +255 2 1 255 val_255 +273 3 1 273 val_273 +273 3 1 273 val_273 +273 3 1 273 val_273 +278 2 1 278 val_278 +278 2 1 278 val_278 +311 3 1 311 val_311 +311 3 1 311 val_311 +311 3 1 311 val_311 +369 3 1 369 val_369 +369 3 1 369 val_369 +369 3 1 369 val_369 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +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 diff --git a/src/test/hive/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 b/src/test/hive/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 b/src/test/hive/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 new file mode 100644 index 0000000000000..8338433cd5a27 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 @@ -0,0 +1,37 @@ +66 1 1 66 val_66 +98 2 1 98 val_98 +98 2 1 98 val_98 +128 3 1 128 val_128 +128 3 1 128 val_128 +128 3 1 128 val_128 +146 2 1 146 val_146 +146 2 1 146 val_146 +150 1 1 150 val_150 +213 2 1 213 val_213 +213 2 1 213 val_213 +224 2 1 224 val_224 +224 2 1 224 val_224 +238 2 1 238 val_238 +238 2 1 238 val_238 +255 2 1 255 val_255 +255 2 1 255 val_255 +273 3 1 273 val_273 +273 3 1 273 val_273 +273 3 1 273 val_273 +278 2 1 278 val_278 +278 2 1 278 val_278 +311 3 1 311 val_311 +311 3 1 311 val_311 +311 3 1 311 val_311 +369 3 1 369 val_369 +369 3 1 369 val_369 +369 3 1 369 val_369 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +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 diff --git a/src/test/hive/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 b/src/test/hive/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 b/src/test/hive/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d b/src/test/hive/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d new file mode 100644 index 0000000000000..8b1d3cd388fa2 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d @@ -0,0 +1,15 @@ +66 1 66 val_66 1 +98 1 98 val_98 4 +128 1 128 val_128 9 +146 1 146 val_146 4 +150 1 150 val_150 1 +213 1 213 val_213 4 +224 1 224 val_224 4 +238 1 238 val_238 4 +255 1 255 val_255 4 +273 1 273 val_273 9 +278 1 278 val_278 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 diff --git a/src/test/hive/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 b/src/test/hive/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d b/src/test/hive/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d new file mode 100644 index 0000000000000..8b1d3cd388fa2 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d @@ -0,0 +1,15 @@ +66 1 66 val_66 1 +98 1 98 val_98 4 +128 1 128 val_128 9 +146 1 146 val_146 4 +150 1 150 val_150 1 +213 1 213 val_213 4 +224 1 224 val_224 4 +238 1 238 val_238 4 +255 1 255 val_255 4 +273 1 273 val_273 9 +278 1 278 val_278 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 diff --git a/src/test/hive/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 b/src/test/hive/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d b/src/test/hive/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d new file mode 100644 index 0000000000000..8b1d3cd388fa2 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d @@ -0,0 +1,15 @@ +66 1 66 val_66 1 +98 1 98 val_98 4 +128 1 128 val_128 9 +146 1 146 val_146 4 +150 1 150 val_150 1 +213 1 213 val_213 4 +224 1 224 val_224 4 +238 1 238 val_238 4 +255 1 255 val_255 4 +273 1 273 val_273 9 +278 1 278 val_278 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 diff --git a/src/test/hive/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 b/src/test/hive/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 new file mode 100644 index 0000000000000..19b8a2aea8f64 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 @@ -0,0 +1,15 @@ +66 1 66 1 +98 1 98 4 +128 1 128 9 +146 1 146 4 +150 1 150 1 +213 1 213 4 +224 1 224 4 +238 1 238 4 +255 1 255 4 +273 1 273 9 +278 1 278 4 +311 1 311 9 +369 1 369 9 +401 1 401 25 +406 1 406 16 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae b/src/test/hive/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 b/src/test/hive/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 b/src/test/hive/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 new file mode 100644 index 0000000000000..747b650237b53 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 @@ -0,0 +1,15 @@ +66 1 66 val_66 +98 2 98 val_98 +128 3 128 +146 2 146 val_146 +150 1 150 val_150 +213 2 213 val_213 +224 2 224 +238 2 238 val_238 +255 2 255 val_255 +273 3 273 val_273 +278 2 278 val_278 +311 3 311 val_311 +369 3 369 +401 5 401 val_401 +406 4 406 val_406 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 b/src/test/hive/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 b/src/test/hive/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 new file mode 100644 index 0000000000000..747b650237b53 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 @@ -0,0 +1,15 @@ +66 1 66 val_66 +98 2 98 val_98 +128 3 128 +146 2 146 val_146 +150 1 150 val_150 +213 2 213 val_213 +224 2 224 +238 2 238 val_238 +255 2 255 val_255 +273 3 273 val_273 +278 2 278 val_278 +311 3 311 val_311 +369 3 369 +401 5 401 val_401 +406 4 406 val_406 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 b/src/test/hive/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 b/src/test/hive/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 new file mode 100644 index 0000000000000..747b650237b53 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 @@ -0,0 +1,15 @@ +66 1 66 val_66 +98 2 98 val_98 +128 3 128 +146 2 146 val_146 +150 1 150 val_150 +213 2 213 val_213 +224 2 224 +238 2 238 val_238 +255 2 255 val_255 +273 3 273 val_273 +278 2 278 val_278 +311 3 311 val_311 +369 3 369 +401 5 401 val_401 +406 4 406 val_406 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 b/src/test/hive/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 b/src/test/hive/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 new file mode 100644 index 0000000000000..747b650237b53 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 @@ -0,0 +1,15 @@ +66 1 66 val_66 +98 2 98 val_98 +128 3 128 +146 2 146 val_146 +150 1 150 val_150 +213 2 213 val_213 +224 2 224 +238 2 238 val_238 +255 2 255 val_255 +273 3 273 val_273 +278 2 278 val_278 +311 3 311 val_311 +369 3 369 +401 5 401 val_401 +406 4 406 val_406 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d b/src/test/hive/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d b/src/test/hive/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f b/src/test/hive/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f new file mode 100644 index 0000000000000..7c0af7229d62a --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f @@ -0,0 +1,24 @@ + 3 NULL NULL +0 3 NULL NULL +10 1 NULL NULL +11 1 NULL NULL +12 2 NULL NULL +15 2 NULL NULL +17 1 NULL NULL +18 2 NULL NULL +19 1 NULL NULL +2 1 NULL NULL +4 1 NULL NULL +5 3 NULL NULL +8 1 NULL NULL +9 1 NULL NULL +val_146 1 NULL NULL +val_150 1 NULL NULL +val_213 1 NULL NULL +val_238 1 NULL NULL +val_255 1 NULL NULL +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 diff --git a/src/test/hive/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea b/src/test/hive/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc b/src/test/hive/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a b/src/test/hive/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 b/src/test/hive/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 new file mode 100644 index 0000000000000..efdd80c9f8b89 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 @@ -0,0 +1,13 @@ +128 3 +146 val_146 2 +150 val_150 1 +213 val_213 2 +224 2 +238 val_238 2 +255 val_255 2 +273 val_273 3 +278 val_278 2 +311 val_311 3 +369 3 +401 val_401 5 +406 val_406 4 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 b/src/test/hive/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 b/src/test/hive/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 new file mode 100644 index 0000000000000..efdd80c9f8b89 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 @@ -0,0 +1,13 @@ +128 3 +146 val_146 2 +150 val_150 1 +213 val_213 2 +224 2 +238 val_238 2 +255 val_255 2 +273 val_273 3 +278 val_278 2 +311 val_311 3 +369 3 +401 val_401 5 +406 val_406 4 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 b/src/test/hive/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f b/src/test/hive/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f new file mode 100644 index 0000000000000..7c0af7229d62a --- /dev/null +++ b/src/test/hive/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f @@ -0,0 +1,24 @@ + 3 NULL NULL +0 3 NULL NULL +10 1 NULL NULL +11 1 NULL NULL +12 2 NULL NULL +15 2 NULL NULL +17 1 NULL NULL +18 2 NULL NULL +19 1 NULL NULL +2 1 NULL NULL +4 1 NULL NULL +5 3 NULL NULL +8 1 NULL NULL +9 1 NULL NULL +val_146 1 NULL NULL +val_150 1 NULL NULL +val_213 1 NULL NULL +val_238 1 NULL NULL +val_255 1 NULL NULL +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 diff --git a/src/test/hive/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b b/src/test/hive/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 b/src/test/hive/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 b/src/test/hive/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 new file mode 100644 index 0000000000000..4b020e0595d2c --- /dev/null +++ b/src/test/hive/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 @@ -0,0 +1,4 @@ +NULL 1 1 6 +10 2 2 10 +12 1 2 9 +100 1 1 3 \ No newline at end of file diff --git a/src/test/hive/golden/count-11-29aa086fe75b55482a91316c38498565 b/src/test/hive/golden/count-11-29aa086fe75b55482a91316c38498565 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/count-12-944f53db544c07a7b38a0544a21d8e13 b/src/test/hive/golden/count-12-944f53db544c07a7b38a0544a21d8e13 new file mode 100644 index 0000000000000..54199fdb8166d --- /dev/null +++ b/src/test/hive/golden/count-12-944f53db544c07a7b38a0544a21d8e13 @@ -0,0 +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 diff --git a/src/test/hive/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 b/src/test/hive/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 new file mode 100644 index 0000000000000..2e9278da88858 --- /dev/null +++ b/src/test/hive/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 @@ -0,0 +1,7 @@ +NULL 35 23 6 +10 1000 50 1 +100 100 10 3 +12 NULL 80 2 +10 100 NULL 5 +10 100 45 4 +12 100 75 7 \ No newline at end of file diff --git a/src/test/hive/golden/count-3-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/count-3-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/count-3-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/count-4-590bf60b8d4dfa135f73dbb52180136f b/src/test/hive/golden/count-4-590bf60b8d4dfa135f73dbb52180136f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 b/src/test/hive/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 new file mode 100644 index 0000000000000..4b020e0595d2c --- /dev/null +++ b/src/test/hive/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 @@ -0,0 +1,4 @@ +NULL 1 1 6 +10 2 2 10 +12 1 2 9 +100 1 1 3 \ No newline at end of file diff --git a/src/test/hive/golden/count-6-29aa086fe75b55482a91316c38498565 b/src/test/hive/golden/count-6-29aa086fe75b55482a91316c38498565 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/count-7-944f53db544c07a7b38a0544a21d8e13 b/src/test/hive/golden/count-7-944f53db544c07a7b38a0544a21d8e13 new file mode 100644 index 0000000000000..54199fdb8166d --- /dev/null +++ b/src/test/hive/golden/count-7-944f53db544c07a7b38a0544a21d8e13 @@ -0,0 +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 diff --git a/src/test/hive/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/count-9-590bf60b8d4dfa135f73dbb52180136f b/src/test/hive/golden/count-9-590bf60b8d4dfa135f73dbb52180136f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 b/src/test/hive/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 b/src/test/hive/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e b/src/test/hive/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 b/src/test/hive/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 b/src/test/hive/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 b/src/test/hive/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac b/src/test/hive/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 b/src/test/hive/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f b/src/test/hive/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/src/test/hive/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/src/test/hive/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 b/src/test/hive/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 b/src/test/hive/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/src/test/hive/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/src/test/hive/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c b/src/test/hive/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd b/src/test/hive/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create table as-1-b9002c1d71895be765575b62656d1928 b/src/test/hive/golden/create table as-1-b9002c1d71895be765575b62656d1928 new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/src/test/hive/golden/create table as-1-b9002c1d71895be765575b62656d1928 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d b/src/test/hive/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 b/src/test/hive/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba b/src/test/hive/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba new file mode 100644 index 0000000000000..66d40e52a4539 --- /dev/null +++ b/src/test/hive/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba @@ -0,0 +1,4 @@ +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/src/test/hive/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 b/src/test/hive/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 b/src/test/hive/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 new file mode 100644 index 0000000000000..8e606f61a1c30 --- /dev/null +++ b/src/test/hive/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 @@ -0,0 +1,2 @@ +a string None +b string None \ No newline at end of file diff --git a/src/test/hive/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a b/src/test/hive/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a new file mode 100644 index 0000000000000..05b460270525d --- /dev/null +++ b/src/test/hive/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a @@ -0,0 +1,4 @@ +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/src/test/hive/golden/create_1-14-437bd1acbae61f48e851addb769d3379 b/src/test/hive/golden/create_1-14-437bd1acbae61f48e851addb769d3379 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 b/src/test/hive/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 new file mode 100644 index 0000000000000..eda3946588e3f --- /dev/null +++ b/src/test/hive/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 @@ -0,0 +1,2 @@ +a string from deserializer +b string from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 b/src/test/hive/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 new file mode 100644 index 0000000000000..8fc60adf10167 --- /dev/null +++ b/src/test/hive/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 b/src/test/hive/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 new file mode 100644 index 0000000000000..8e606f61a1c30 --- /dev/null +++ b/src/test/hive/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 @@ -0,0 +1,2 @@ +a string None +b string None \ No newline at end of file diff --git a/src/test/hive/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 b/src/test/hive/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 new file mode 100644 index 0000000000000..5e9bc70c3a1a3 --- /dev/null +++ b/src/test/hive/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/create_1-4-610b82bf7b0080d293977927e5ef780c b/src/test/hive/golden/create_1-4-610b82bf7b0080d293977927e5ef780c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_1-5-c77b018276b1558c1d9462e0625e152e b/src/test/hive/golden/create_1-5-c77b018276b1558c1d9462e0625e152e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a b/src/test/hive/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a new file mode 100644 index 0000000000000..6ed3515ba6876 --- /dev/null +++ b/src/test/hive/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a @@ -0,0 +1,2 @@ +a string None +b int None \ No newline at end of file diff --git a/src/test/hive/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 b/src/test/hive/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 new file mode 100644 index 0000000000000..bec4c72e49857 --- /dev/null +++ b/src/test/hive/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 b/src/test/hive/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 b/src/test/hive/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 new file mode 100644 index 0000000000000..8e606f61a1c30 --- /dev/null +++ b/src/test/hive/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 @@ -0,0 +1,2 @@ +a string None +b string None \ No newline at end of file diff --git a/src/test/hive/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d b/src/test/hive/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee b/src/test/hive/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e b/src/test/hive/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 b/src/test/hive/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 new file mode 100644 index 0000000000000..8e606f61a1c30 --- /dev/null +++ b/src/test/hive/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 @@ -0,0 +1,2 @@ +a string None +b string None \ No newline at end of file diff --git a/src/test/hive/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 b/src/test/hive/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 new file mode 100644 index 0000000000000..8ab6b24deab60 --- /dev/null +++ b/src/test/hive/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 b/src/test/hive/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 b/src/test/hive/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 new file mode 100644 index 0000000000000..bbe37f8e2a790 --- /dev/null +++ b/src/test/hive/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 @@ -0,0 +1 @@ +86 \ \ \ No newline at end of file diff --git a/src/test/hive/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 b/src/test/hive/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 new file mode 100644 index 0000000000000..cbcdfbe72e8c6 --- /dev/null +++ b/src/test/hive/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 @@ -0,0 +1,9 @@ +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/src/test/hive/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b b/src/test/hive/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 b/src/test/hive/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 b/src/test/hive/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 new file mode 100644 index 0000000000000..2f958c483a9df --- /dev/null +++ b/src/test/hive/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 @@ -0,0 +1 @@ +1.0 260.182 \ No newline at end of file diff --git a/src/test/hive/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 b/src/test/hive/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 b/src/test/hive/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 new file mode 100644 index 0000000000000..0e9c06c882602 --- /dev/null +++ b/src/test/hive/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a b/src/test/hive/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 b/src/test/hive/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc b/src/test/hive/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc new file mode 100644 index 0000000000000..83f572c424926 --- /dev/null +++ b/src/test/hive/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc @@ -0,0 +1,2 @@ +key int None +value string None \ No newline at end of file diff --git a/src/test/hive/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 b/src/test/hive/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 b/src/test/hive/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c b/src/test/hive/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c new file mode 100644 index 0000000000000..83f572c424926 --- /dev/null +++ b/src/test/hive/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c @@ -0,0 +1,2 @@ +key int None +value string None \ No newline at end of file diff --git a/src/test/hive/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 b/src/test/hive/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e b/src/test/hive/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 b/src/test/hive/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 new file mode 100644 index 0000000000000..83f572c424926 --- /dev/null +++ b/src/test/hive/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 @@ -0,0 +1,2 @@ +key int None +value string None \ No newline at end of file diff --git a/src/test/hive/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 b/src/test/hive/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like-1-30485a2507b60b96cad3d293527e6af b/src/test/hive/golden/create_like-1-30485a2507b60b96cad3d293527e6af new file mode 100644 index 0000000000000..b8ddba7f50b97 --- /dev/null +++ b/src/test/hive/golden/create_like-1-30485a2507b60b96cad3d293527e6af @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 b/src/test/hive/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 new file mode 100644 index 0000000000000..e87fb81a6f043 --- /dev/null +++ b/src/test/hive/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 @@ -0,0 +1 @@ +86 val_86 diff --git a/src/test/hive/golden/create_like-11-ba64f0122b21f605d8b2928753784130 b/src/test/hive/golden/create_like-11-ba64f0122b21f605d8b2928753784130 new file mode 100644 index 0000000000000..375dcacb8db71 --- /dev/null +++ b/src/test/hive/golden/create_like-11-ba64f0122b21f605d8b2928753784130 @@ -0,0 +1,2 @@ +100 val_100 +100 val_100 diff --git a/src/test/hive/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a b/src/test/hive/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like-3-eea111a209cf8895f31f64699669c705 b/src/test/hive/golden/create_like-3-eea111a209cf8895f31f64699669c705 new file mode 100644 index 0000000000000..52b637c27f98d --- /dev/null +++ b/src/test/hive/golden/create_like-3-eea111a209cf8895f31f64699669c705 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 b/src/test/hive/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like-5-dc9de26002604e9e436135bd4b40636d b/src/test/hive/golden/create_like-5-dc9de26002604e9e436135bd4b40636d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 b/src/test/hive/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 b/src/test/hive/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 new file mode 100644 index 0000000000000..d9308798dcb1c --- /dev/null +++ b/src/test/hive/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 @@ -0,0 +1,28 @@ +# 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/src/test/hive/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb b/src/test/hive/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb b/src/test/hive/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b b/src/test/hive/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a b/src/test/hive/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 b/src/test/hive/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 new file mode 100644 index 0000000000000..3e2a912824ab3 --- /dev/null +++ b/src/test/hive/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 @@ -0,0 +1,27 @@ +# col_name data_type comment + +a string None +b string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Fri Feb 07 14:52:37 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table2 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1391813557 + +# 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/src/test/hive/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 b/src/test/hive/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 b/src/test/hive/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 b/src/test/hive/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 b/src/test/hive/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 new file mode 100644 index 0000000000000..115d12fb70c81 --- /dev/null +++ b/src/test/hive/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 @@ -0,0 +1,2 @@ +a string None +b string None diff --git a/src/test/hive/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 b/src/test/hive/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 new file mode 100644 index 0000000000000..61d34badb1a2d --- /dev/null +++ b/src/test/hive/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 @@ -0,0 +1,28 @@ +# col_name data_type comment + +a string None +b string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Fri Feb 07 14:52:37 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table3 +Table Type: EXTERNAL_TABLE +Table Parameters: + EXTERNAL TRUE + transient_lastDdlTime 1391813557 + +# 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/src/test/hive/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb b/src/test/hive/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb b/src/test/hive/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c b/src/test/hive/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c new file mode 100644 index 0000000000000..e87fb81a6f043 --- /dev/null +++ b/src/test/hive/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c @@ -0,0 +1 @@ +86 val_86 diff --git a/src/test/hive/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f b/src/test/hive/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f new file mode 100644 index 0000000000000..375dcacb8db71 --- /dev/null +++ b/src/test/hive/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f @@ -0,0 +1,2 @@ +100 val_100 +100 val_100 diff --git a/src/test/hive/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b b/src/test/hive/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 b/src/test/hive/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 b/src/test/hive/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/src/test/hive/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b b/src/test/hive/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 b/src/test/hive/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d b/src/test/hive/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d new file mode 100644 index 0000000000000..4c2967215fe66 --- /dev/null +++ b/src/test/hive/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d @@ -0,0 +1,33 @@ +# 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: Fri Feb 07 14:53:16 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table1 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1391813596 + +# 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/src/test/hive/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 b/src/test/hive/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/src/test/hive/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f b/src/test/hive/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 b/src/test/hive/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 b/src/test/hive/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 new file mode 100644 index 0000000000000..115d12fb70c81 --- /dev/null +++ b/src/test/hive/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 @@ -0,0 +1,2 @@ +a string None +b string None diff --git a/src/test/hive/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af b/src/test/hive/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af new file mode 100644 index 0000000000000..02dee147bca42 --- /dev/null +++ b/src/test/hive/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af @@ -0,0 +1,27 @@ +# col_name data_type comment + +a string None +b string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Fri Feb 07 14:52:37 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table1 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1391813557 + +# 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/src/test/hive/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 b/src/test/hive/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 b/src/test/hive/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a b/src/test/hive/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a new file mode 100644 index 0000000000000..115d12fb70c81 --- /dev/null +++ b/src/test/hive/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a @@ -0,0 +1,2 @@ +a string None +b string None diff --git a/src/test/hive/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c b/src/test/hive/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 b/src/test/hive/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 b/src/test/hive/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 new file mode 100644 index 0000000000000..dbdf4585360ae --- /dev/null +++ b/src/test/hive/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d b/src/test/hive/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d new file mode 100644 index 0000000000000..9a037142aa3c1 --- /dev/null +++ b/src/test/hive/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d @@ -0,0 +1 @@ +10 \ No newline at end of file diff --git a/src/test/hive/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 b/src/test/hive/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 new file mode 100644 index 0000000000000..eb6c3f6aef813 --- /dev/null +++ b/src/test/hive/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 @@ -0,0 +1 @@ +508 -751895388 \ No newline at end of file diff --git a/src/test/hive/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f b/src/test/hive/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 b/src/test/hive/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 b/src/test/hive/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 b/src/test/hive/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 b/src/test/hive/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 b/src/test/hive/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 b/src/test/hive/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 new file mode 100644 index 0000000000000..8a9d9d56a66d6 --- /dev/null +++ b/src/test/hive/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d b/src/test/hive/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d new file mode 100644 index 0000000000000..9a037142aa3c1 --- /dev/null +++ b/src/test/hive/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d @@ -0,0 +1 @@ +10 \ No newline at end of file diff --git a/src/test/hive/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 b/src/test/hive/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 new file mode 100644 index 0000000000000..eb6c3f6aef813 --- /dev/null +++ b/src/test/hive/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 @@ -0,0 +1 @@ +508 -751895388 \ No newline at end of file diff --git a/src/test/hive/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 b/src/test/hive/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 b/src/test/hive/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 b/src/test/hive/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 new file mode 100644 index 0000000000000..c49ad26c04d67 --- /dev/null +++ b/src/test/hive/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 @@ -0,0 +1,4 @@ +a string None +b array None +c array> None +d map> None diff --git a/src/test/hive/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 b/src/test/hive/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 new file mode 100644 index 0000000000000..41c7202c8b886 --- /dev/null +++ b/src/test/hive/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 @@ -0,0 +1,6 @@ +a string None +b array None +c array> None +d map> None + +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, comment:null), FieldSchema(name:c, type:array>, comment:null), FieldSchema(name:d, type:map>, 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) diff --git a/src/test/hive/golden/create_nested_type-3-ac452c9279877935983c37113898e53c b/src/test/hive/golden/create_nested_type-3-ac452c9279877935983c37113898e53c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff b/src/test/hive/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff new file mode 100644 index 0000000000000..45bb8c610c677 --- /dev/null +++ b/src/test/hive/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff @@ -0,0 +1,4 @@ +a0 ["b00","b01"] [{"c001":"C001","c002":"C002"},{"c011":null,"c012":"C012"}] {"d01":["d011","d012"],"d02":["d021","d022"]} +a1 ["b10"] [{"c001":"C001","c002":"C002"}] {"d01":["d011","d012"],"d02":null} +a2 [] [{"c001":null,"c002":"C002"},{"c011":"C011","c012":"C012"}] {"d01":[null,"d012"],"d02":["d021","d022"]} +a3 NULL NULL NULL diff --git a/src/test/hive/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d b/src/test/hive/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea b/src/test/hive/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 b/src/test/hive/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 new file mode 100644 index 0000000000000..46869fc9e88eb --- /dev/null +++ b/src/test/hive/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 @@ -0,0 +1,30 @@ +# 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/src/test/hive/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d b/src/test/hive/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de b/src/test/hive/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb b/src/test/hive/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 b/src/test/hive/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e b/src/test/hive/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 b/src/test/hive/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 b/src/test/hive/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 new file mode 100644 index 0000000000000..d7c386e17c096 --- /dev/null +++ b/src/test/hive/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 @@ -0,0 +1,29 @@ +# col_name data_type comment + +key string None +value string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Tue Jan 14 22:51:44 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_single_2 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1389768704 + +# 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: [] +Skewed Columns: [key] +Skewed Values: [[1], [5], [6]] +Storage Desc Params: + serialization.format 1 \ No newline at end of file diff --git a/src/test/hive/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e b/src/test/hive/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e new file mode 100644 index 0000000000000..0817efac83d79 --- /dev/null +++ b/src/test/hive/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e @@ -0,0 +1,29 @@ +# col_name data_type comment + +key string None +value string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Tue Jan 14 22:51:44 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_single +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1389768704 + +# 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: [] +Skewed Columns: [key] +Skewed Values: [[1], [5], [6]] +Storage Desc Params: + serialization.format 1 \ No newline at end of file diff --git a/src/test/hive/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef b/src/test/hive/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef new file mode 100644 index 0000000000000..2986dd43b0fd6 --- /dev/null +++ b/src/test/hive/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef @@ -0,0 +1,30 @@ +# col_name data_type comment + +col1 string None +col2 int None +col3 string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Tue Jan 14 22:51:44 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_multiple +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1389768704 + +# 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: [] +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 diff --git a/src/test/hive/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 b/src/test/hive/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd b/src/test/hive/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e b/src/test/hive/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 b/src/test/hive/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 b/src/test/hive/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd b/src/test/hive/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd new file mode 100644 index 0000000000000..6a45cf1600994 --- /dev/null +++ b/src/test/hive/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd @@ -0,0 +1,10 @@ +{"a":238,"b":"val_238","c":null} 238 val_238 +{"a":86,"b":"val_86","c":null} 86 val_86 +{"a":311,"b":"val_311","c":null} 311 val_311 +{"a":27,"b":"val_27","c":null} 27 val_27 +{"a":165,"b":"val_165","c":null} 165 val_165 +{"a":409,"b":"val_409","c":null} 409 val_409 +{"a":255,"b":"val_255","c":null} 255 val_255 +{"a":278,"b":"val_278","c":null} 278 val_278 +{"a":98,"b":"val_98","c":null} 98 val_98 +{"a":484,"b":"val_484","c":null} 484 val_484 diff --git a/src/test/hive/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e b/src/test/hive/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e new file mode 100644 index 0000000000000..8af6a0338d65a --- /dev/null +++ b/src/test/hive/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e @@ -0,0 +1,10 @@ +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/src/test/hive/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/src/test/hive/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 b/src/test/hive/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd b/src/test/hive/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-11-2021c047203276dd2db51a56e672fcea b/src/test/hive/golden/create_view-11-2021c047203276dd2db51a56e672fcea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc b/src/test/hive/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a b/src/test/hive/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 b/src/test/hive/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 b/src/test/hive/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 b/src/test/hive/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a b/src/test/hive/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 b/src/test/hive/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b b/src/test/hive/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b new file mode 100644 index 0000000000000..e87fb81a6f043 --- /dev/null +++ b/src/test/hive/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b @@ -0,0 +1 @@ +86 val_86 diff --git a/src/test/hive/golden/create_view-2-d80dcd1271ab264292e9938f3162427c b/src/test/hive/golden/create_view-2-d80dcd1271ab264292e9938f3162427c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 b/src/test/hive/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 b/src/test/hive/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 b/src/test/hive/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 b/src/test/hive/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 b/src/test/hive/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 b/src/test/hive/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 b/src/test/hive/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 b/src/test/hive/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 b/src/test/hive/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 b/src/test/hive/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc b/src/test/hive/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 b/src/test/hive/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 b/src/test/hive/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 b/src/test/hive/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 b/src/test/hive/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 new file mode 100644 index 0000000000000..70d2ecdbc8f78 --- /dev/null +++ b/src/test/hive/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 @@ -0,0 +1,13 @@ +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/src/test/hive/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc b/src/test/hive/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc new file mode 100644 index 0000000000000..43bc3de44f4df --- /dev/null +++ b/src/test/hive/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 b/src/test/hive/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 b/src/test/hive/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 b/src/test/hive/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e b/src/test/hive/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 b/src/test/hive/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 b/src/test/hive/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b b/src/test/hive/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 b/src/test/hive/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f b/src/test/hive/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 b/src/test/hive/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec b/src/test/hive/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/src/test/hive/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/src/test/hive/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 b/src/test/hive/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/src/test/hive/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 diff --git a/src/test/hive/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 b/src/test/hive/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd b/src/test/hive/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/database.table-0-c657beb729b6a7882309a203fc6f298e b/src/test/hive/golden/database.table-0-c657beb729b6a7882309a203fc6f298e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/database.table-0-c657beb729b6a7882309a203fc6f298e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f b/src/test/hive/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 b/src/test/hive/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 new file mode 100644 index 0000000000000..585786f6a8a41 --- /dev/null +++ b/src/test/hive/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 @@ -0,0 +1,2 @@ +db5 +default diff --git a/src/test/hive/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 b/src/test/hive/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 b/src/test/hive/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/database_drop-12-e02a53f7e798d2741152526516f14941 b/src/test/hive/golden/database_drop-12-e02a53f7e798d2741152526516f14941 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/database_drop-12-e02a53f7e798d2741152526516f14941 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 b/src/test/hive/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 b/src/test/hive/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 b/src/test/hive/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 b/src/test/hive/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 b/src/test/hive/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba b/src/test/hive/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_drop-8-97101266791d2b2c662bcde549422318 b/src/test/hive/golden/database_drop-8-97101266791d2b2c662bcde549422318 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 b/src/test/hive/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a b/src/test/hive/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 b/src/test/hive/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 new file mode 100644 index 0000000000000..f05305df0c3d3 --- /dev/null +++ b/src/test/hive/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 @@ -0,0 +1 @@ +db1 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db \ No newline at end of file diff --git a/src/test/hive/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 b/src/test/hive/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 new file mode 100644 index 0000000000000..e92c241d50092 --- /dev/null +++ b/src/test/hive/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 @@ -0,0 +1,27 @@ +# col_name data_type comment + +name string None +value int None + +# Detailed Table Information +Database: db2 +Owner: marmbrus +CreateTime: Tue Jan 14 11:40:51 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2/table_db2 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1389728451 + +# 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/src/test/hive/golden/database_location-11-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/database_location-11-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..7c77c06cda82b --- /dev/null +++ b/src/test/hive/golden/database_location-11-9c36cac1372650b703400c60dd29042c @@ -0,0 +1 @@ +table_db2 \ No newline at end of file diff --git a/src/test/hive/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 b/src/test/hive/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 b/src/test/hive/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_location-4-be65cf649363681d54e593c42a5ecffb b/src/test/hive/golden/database_location-4-be65cf649363681d54e593c42a5ecffb new file mode 100644 index 0000000000000..cda1736e9ca6b --- /dev/null +++ b/src/test/hive/golden/database_location-4-be65cf649363681d54e593c42a5ecffb @@ -0,0 +1,27 @@ +# col_name data_type comment + +name string None +value int None + +# Detailed Table Information +Database: db1 +Owner: marmbrus +CreateTime: Tue Jan 14 11:40:50 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/table_db1 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1389728450 + +# 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/src/test/hive/golden/database_location-5-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/database_location-5-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..4d8fc6a6d6fe0 --- /dev/null +++ b/src/test/hive/golden/database_location-5-9c36cac1372650b703400c60dd29042c @@ -0,0 +1 @@ +table_db1 \ No newline at end of file diff --git a/src/test/hive/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a b/src/test/hive/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 b/src/test/hive/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 new file mode 100644 index 0000000000000..3e23970adddcf --- /dev/null +++ b/src/test/hive/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 @@ -0,0 +1 @@ +db2 database 2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2 \ No newline at end of file diff --git a/src/test/hive/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b b/src/test/hive/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 b/src/test/hive/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 b/src/test/hive/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 b/src/test/hive/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 new file mode 100644 index 0000000000000..2f5fbe26f4945 --- /dev/null +++ b/src/test/hive/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 @@ -0,0 +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 diff --git a/src/test/hive/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 b/src/test/hive/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 b/src/test/hive/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 new file mode 100644 index 0000000000000..86362ead004d9 --- /dev/null +++ b/src/test/hive/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 @@ -0,0 +1,2 @@ +db1 +default \ No newline at end of file diff --git a/src/test/hive/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 b/src/test/hive/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a b/src/test/hive/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a new file mode 100644 index 0000000000000..ff89c3fe899d2 --- /dev/null +++ b/src/test/hive/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a @@ -0,0 +1 @@ +db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2.db \ No newline at end of file diff --git a/src/test/hive/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 b/src/test/hive/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 new file mode 100644 index 0000000000000..5827d2726d084 --- /dev/null +++ b/src/test/hive/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 @@ -0,0 +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 diff --git a/src/test/hive/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 b/src/test/hive/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a b/src/test/hive/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd b/src/test/hive/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 b/src/test/hive/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_2-2-cab14d992c53c106ab257fae52001e04 b/src/test/hive/golden/date_2-2-cab14d992c53c106ab257fae52001e04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 b/src/test/hive/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 b/src/test/hive/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 b/src/test/hive/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c b/src/test/hive/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_join1-1-3a68de2112a212a07a3068916c608fb b/src/test/hive/golden/date_join1-1-3a68de2112a212a07a3068916c608fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 b/src/test/hive/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 b/src/test/hive/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_serde-1-36e6041f53433482631018410bb62a99 b/src/test/hive/golden/date_serde-1-36e6041f53433482631018410bb62a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 b/src/test/hive/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_serde-3-e6167e27465514356c557a77d956ea46 b/src/test/hive/golden/date_serde-3-e6167e27465514356c557a77d956ea46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 b/src/test/hive/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c b/src/test/hive/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_serde-6-e00facec2986bc55074868eff87ba22a b/src/test/hive/golden/date_serde-6-e00facec2986bc55074868eff87ba22a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d b/src/test/hive/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 b/src/test/hive/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 b/src/test/hive/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 new file mode 100644 index 0000000000000..c270c7cbdfa1f --- /dev/null +++ b/src/test/hive/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 b/src/test/hive/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd b/src/test/hive/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 b/src/test/hive/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/src/test/hive/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 @@ -0,0 +1 @@ +17.29 diff --git a/src/test/hive/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 b/src/test/hive/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/src/test/hive/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 @@ -0,0 +1 @@ +17.29 diff --git a/src/test/hive/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 b/src/test/hive/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 new file mode 100644 index 0000000000000..c4a17c1b14c88 --- /dev/null +++ b/src/test/hive/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 @@ -0,0 +1 @@ +1969-12-31 16:00:17.29 diff --git a/src/test/hive/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 b/src/test/hive/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 b/src/test/hive/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 b/src/test/hive/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 b/src/test/hive/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 b/src/test/hive/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/src/test/hive/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 @@ -0,0 +1 @@ +17 diff --git a/src/test/hive/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b b/src/test/hive/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/src/test/hive/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b @@ -0,0 +1 @@ +17 diff --git a/src/test/hive/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a b/src/test/hive/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/src/test/hive/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a @@ -0,0 +1 @@ +17 diff --git a/src/test/hive/golden/decimal_1-8-cdd0932288d3cc43636334439805769d b/src/test/hive/golden/decimal_1-8-cdd0932288d3cc43636334439805769d new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/src/test/hive/golden/decimal_1-8-cdd0932288d3cc43636334439805769d @@ -0,0 +1 @@ +17 diff --git a/src/test/hive/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c b/src/test/hive/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/src/test/hive/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c @@ -0,0 +1 @@ +17.29 diff --git a/src/test/hive/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee b/src/test/hive/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b b/src/test/hive/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 b/src/test/hive/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/src/test/hive/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 @@ -0,0 +1 @@ +17.29 diff --git a/src/test/hive/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 b/src/test/hive/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/src/test/hive/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 @@ -0,0 +1 @@ +17.29 diff --git a/src/test/hive/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 b/src/test/hive/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 b/src/test/hive/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 b/src/test/hive/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 new file mode 100644 index 0000000000000..b1bd38b62a080 --- /dev/null +++ b/src/test/hive/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 @@ -0,0 +1 @@ +13 diff --git a/src/test/hive/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 b/src/test/hive/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 new file mode 100644 index 0000000000000..de7771ac23570 --- /dev/null +++ b/src/test/hive/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 @@ -0,0 +1 @@ +-3827 diff --git a/src/test/hive/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 b/src/test/hive/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 new file mode 100644 index 0000000000000..272791f402250 --- /dev/null +++ b/src/test/hive/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 @@ -0,0 +1 @@ +3404045 diff --git a/src/test/hive/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d b/src/test/hive/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d new file mode 100644 index 0000000000000..272791f402250 --- /dev/null +++ b/src/test/hive/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d @@ -0,0 +1 @@ +3404045 diff --git a/src/test/hive/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e b/src/test/hive/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e new file mode 100644 index 0000000000000..deb8427800ee4 --- /dev/null +++ b/src/test/hive/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e @@ -0,0 +1 @@ +3404045.5 diff --git a/src/test/hive/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 b/src/test/hive/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 new file mode 100644 index 0000000000000..6f31e8fe55034 --- /dev/null +++ b/src/test/hive/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 @@ -0,0 +1 @@ +3404045.5044003 diff --git a/src/test/hive/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 b/src/test/hive/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 b/src/test/hive/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 new file mode 100644 index 0000000000000..6f31e8fe55034 --- /dev/null +++ b/src/test/hive/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 @@ -0,0 +1 @@ +3404045.5044003 diff --git a/src/test/hive/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 b/src/test/hive/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 new file mode 100644 index 0000000000000..6324d401a069f --- /dev/null +++ b/src/test/hive/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 @@ -0,0 +1 @@ +3.14 diff --git a/src/test/hive/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 b/src/test/hive/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 new file mode 100644 index 0000000000000..6324d401a069f --- /dev/null +++ b/src/test/hive/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 @@ -0,0 +1 @@ +3.14 diff --git a/src/test/hive/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff b/src/test/hive/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff new file mode 100644 index 0000000000000..603f18cc37bc4 --- /dev/null +++ b/src/test/hive/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff @@ -0,0 +1 @@ +1355944339.1234567 diff --git a/src/test/hive/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e b/src/test/hive/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/decimal_2-25-14face5c7104382196e65741a199c36 b/src/test/hive/golden/decimal_2-25-14face5c7104382196e65741a199c36 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/src/test/hive/golden/decimal_2-25-14face5c7104382196e65741a199c36 @@ -0,0 +1 @@ +3 diff --git a/src/test/hive/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 b/src/test/hive/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/src/test/hive/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 @@ -0,0 +1 @@ +3 diff --git a/src/test/hive/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 b/src/test/hive/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/src/test/hive/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 @@ -0,0 +1 @@ +3 diff --git a/src/test/hive/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 b/src/test/hive/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/src/test/hive/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 @@ -0,0 +1 @@ +3 diff --git a/src/test/hive/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 b/src/test/hive/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e b/src/test/hive/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a b/src/test/hive/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a new file mode 100644 index 0000000000000..474c8b180aea9 --- /dev/null +++ b/src/test/hive/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a @@ -0,0 +1 @@ +0.99999999999999999999 diff --git a/src/test/hive/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee b/src/test/hive/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 b/src/test/hive/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 b/src/test/hive/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/src/test/hive/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 @@ -0,0 +1 @@ +17 diff --git a/src/test/hive/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 b/src/test/hive/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/src/test/hive/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 @@ -0,0 +1 @@ +17 diff --git a/src/test/hive/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 b/src/test/hive/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/src/test/hive/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 @@ -0,0 +1 @@ +17 diff --git a/src/test/hive/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d b/src/test/hive/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/src/test/hive/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d @@ -0,0 +1 @@ +17 diff --git a/src/test/hive/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e b/src/test/hive/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/src/test/hive/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e @@ -0,0 +1 @@ +17.29 diff --git a/src/test/hive/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef b/src/test/hive/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 b/src/test/hive/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 b/src/test/hive/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 new file mode 100644 index 0000000000000..8d8753f153d7c --- /dev/null +++ b/src/test/hive/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 @@ -0,0 +1,4 @@ +3.14 3 +3.14 3 +3.14 3 +3.14 4 diff --git a/src/test/hive/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 b/src/test/hive/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 new file mode 100644 index 0000000000000..8d8753f153d7c --- /dev/null +++ b/src/test/hive/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 @@ -0,0 +1,4 @@ +3.14 3 +3.14 3 +3.14 3 +3.14 4 diff --git a/src/test/hive/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 b/src/test/hive/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 b/src/test/hive/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b b/src/test/hive/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b new file mode 100644 index 0000000000000..3e290231c27e2 --- /dev/null +++ b/src/test/hive/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b @@ -0,0 +1,38 @@ +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/src/test/hive/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 b/src/test/hive/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 new file mode 100644 index 0000000000000..64fa7bca9a81b --- /dev/null +++ b/src/test/hive/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 @@ -0,0 +1,38 @@ +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/src/test/hive/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b b/src/test/hive/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b new file mode 100644 index 0000000000000..3e290231c27e2 --- /dev/null +++ b/src/test/hive/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b @@ -0,0 +1,38 @@ +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/src/test/hive/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 b/src/test/hive/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 new file mode 100644 index 0000000000000..24d34ee5d8c1b --- /dev/null +++ b/src/test/hive/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 @@ -0,0 +1,30 @@ +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/src/test/hive/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 b/src/test/hive/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 new file mode 100644 index 0000000000000..e08f588c89461 --- /dev/null +++ b/src/test/hive/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 @@ -0,0 +1,30 @@ +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/src/test/hive/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 b/src/test/hive/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 new file mode 100644 index 0000000000000..796707d06b0dd --- /dev/null +++ b/src/test/hive/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 @@ -0,0 +1,17 @@ +-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/src/test/hive/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 b/src/test/hive/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 new file mode 100644 index 0000000000000..4217ad848170e --- /dev/null +++ b/src/test/hive/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 @@ -0,0 +1,56 @@ +-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/src/test/hive/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f b/src/test/hive/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 b/src/test/hive/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d b/src/test/hive/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e b/src/test/hive/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c b/src/test/hive/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c b/src/test/hive/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e b/src/test/hive/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e new file mode 100644 index 0000000000000..3e290231c27e2 --- /dev/null +++ b/src/test/hive/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e @@ -0,0 +1,38 @@ +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/src/test/hive/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 b/src/test/hive/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 new file mode 100644 index 0000000000000..795a4b567ab7f --- /dev/null +++ b/src/test/hive/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 @@ -0,0 +1,38 @@ +NULL NULL +NULL NULL +-1234567890.123456789 -3703703670.370370367 +-4400 -13200 +-1255.49 -3766.47 +-1.122 -3.366 +-1.12 -3.36 +-1.12 -3.36 +-0.333 -0.999 +-0.33 -0.99 +-0.3 -0.9 +0 0 +0 0 +0.01 0.03 +0.02 0.06 +0.1 0.3 +0.2 0.6 +0.3 0.9 +0.33 0.99 +0.333 0.999 +0.9999999999999999999999999 2.9999999999999999999999997 +1 3 +1 3 +1.12 3.36 +1.122 3.366 +2 6 +2 6 +3.14 9.42 +3.14 9.42 +3.14 9.42 +3.14 9.42 +10 30 +20 60 +100 300 +124 372 +125.2 375.6 +200 600 +1234567890.12345678 3703703670.37037034 diff --git a/src/test/hive/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 b/src/test/hive/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 b/src/test/hive/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 b/src/test/hive/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e b/src/test/hive/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 b/src/test/hive/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 new file mode 100644 index 0000000000000..23ec633b6a337 --- /dev/null +++ b/src/test/hive/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 @@ -0,0 +1,52 @@ +477 val_477 27 val_27 +487 val_487 37 val_37 +467 val_467 17 val_17 +470 val_470 20 val_20 +497 val_497 47 val_47 +454 val_454 4 val_4 +454 val_454 4 val_4 +454 val_454 4 val_4 +485 val_485 35 val_35 +462 val_462 12 val_12 +462 val_462 12 val_12 +458 val_458 8 val_8 +458 val_458 8 val_8 +492 val_492 42 val_42 +492 val_492 42 val_42 +493 val_493 43 val_43 +469 val_469 19 val_19 +469 val_469 19 val_19 +469 val_469 19 val_19 +469 val_469 19 val_19 +469 val_469 19 val_19 +460 val_460 10 val_10 +455 val_455 5 val_5 +485 val_485 35 val_35 +484 val_484 34 val_34 +492 val_492 42 val_42 +492 val_492 42 val_42 +491 val_491 41 val_41 +480 val_480 30 val_30 +480 val_480 30 val_30 +480 val_480 30 val_30 +483 val_483 33 val_33 +455 val_455 5 val_5 +452 val_452 2 val_2 +485 val_485 35 val_35 +494 val_494 44 val_44 +462 val_462 12 val_12 +462 val_462 12 val_12 +455 val_455 5 val_5 +468 val_468 18 val_18 +468 val_468 18 val_18 +468 val_468 18 val_18 +468 val_468 18 val_18 +459 val_459 9 val_9 +459 val_459 9 val_9 +468 val_468 18 val_18 +468 val_468 18 val_18 +468 val_468 18 val_18 +468 val_468 18 val_18 +478 val_478 28 val_28 +478 val_478 28 val_28 +487 val_487 37 val_37 diff --git a/src/test/hive/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e b/src/test/hive/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 b/src/test/hive/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 b/src/test/hive/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 b/src/test/hive/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e b/src/test/hive/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e b/src/test/hive/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 b/src/test/hive/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce b/src/test/hive/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c b/src/test/hive/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b b/src/test/hive/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b new file mode 100644 index 0000000000000..3d9e792183f3c --- /dev/null +++ b/src/test/hive/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b @@ -0,0 +1,75 @@ +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/src/test/hive/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc b/src/test/hive/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc new file mode 100644 index 0000000000000..9853ce72ed8c3 --- /dev/null +++ b/src/test/hive/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc @@ -0,0 +1,75 @@ +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/src/test/hive/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef b/src/test/hive/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef new file mode 100644 index 0000000000000..7cc75c789dee1 --- /dev/null +++ b/src/test/hive/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef @@ -0,0 +1,75 @@ +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/src/test/hive/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 b/src/test/hive/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 new file mode 100644 index 0000000000000..c40875630d1b2 --- /dev/null +++ b/src/test/hive/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 @@ -0,0 +1,75 @@ +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/src/test/hive/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 b/src/test/hive/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 new file mode 100644 index 0000000000000..bd23d17293f79 --- /dev/null +++ b/src/test/hive/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 @@ -0,0 +1,75 @@ +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/src/test/hive/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e b/src/test/hive/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e new file mode 100644 index 0000000000000..c1e0db0174c63 --- /dev/null +++ b/src/test/hive/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e @@ -0,0 +1,75 @@ +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/src/test/hive/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab b/src/test/hive/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab new file mode 100644 index 0000000000000..81af0e4cd3ab8 --- /dev/null +++ b/src/test/hive/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab @@ -0,0 +1 @@ +NULL NULL diff --git a/src/test/hive/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 b/src/test/hive/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c b/src/test/hive/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 b/src/test/hive/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be b/src/test/hive/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be new file mode 100644 index 0000000000000..ded23df148827 --- /dev/null +++ b/src/test/hive/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be @@ -0,0 +1 @@ +ds=__HIVE_DEFAULT_PARTITION__ \ No newline at end of file diff --git a/src/test/hive/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba b/src/test/hive/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 b/src/test/hive/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 b/src/test/hive/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 new file mode 100644 index 0000000000000..f438072c76b5f --- /dev/null +++ b/src/test/hive/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 @@ -0,0 +1,3 @@ +35 40 +48 32 +100100 40 \ No newline at end of file diff --git a/src/test/hive/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 b/src/test/hive/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 new file mode 100644 index 0000000000000..f438072c76b5f --- /dev/null +++ b/src/test/hive/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 @@ -0,0 +1,3 @@ +35 40 +48 32 +100100 40 \ No newline at end of file diff --git a/src/test/hive/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 b/src/test/hive/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb b/src/test/hive/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 b/src/test/hive/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 new file mode 100644 index 0000000000000..d980efc81b947 --- /dev/null +++ b/src/test/hive/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 b/src/test/hive/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 new file mode 100644 index 0000000000000..01b9151074b22 --- /dev/null +++ b/src/test/hive/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/src/test/hive/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 b/src/test/hive/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 b/src/test/hive/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 b/src/test/hive/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a b/src/test/hive/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a new file mode 100644 index 0000000000000..c56a79e4f322e --- /dev/null +++ b/src/test/hive/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a @@ -0,0 +1 @@ +{"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/src/test/hive/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 b/src/test/hive/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 new file mode 100644 index 0000000000000..aa08c38c68d1d --- /dev/null +++ b/src/test/hive/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 @@ -0,0 +1 @@ +{"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/src/test/hive/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 b/src/test/hive/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 new file mode 100644 index 0000000000000..513aeaab1dc66 --- /dev/null +++ b/src/test/hive/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 @@ -0,0 +1 @@ +{"databases":["default","jsondb1"]} \ No newline at end of file diff --git a/src/test/hive/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c b/src/test/hive/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c new file mode 100644 index 0000000000000..606069d6291b4 --- /dev/null +++ b/src/test/hive/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c @@ -0,0 +1 @@ +{"databases":["jsondb1"]} \ No newline at end of file diff --git a/src/test/hive/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 b/src/test/hive/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 b/src/test/hive/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a b/src/test/hive/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a new file mode 100644 index 0000000000000..bea7c01440c46 --- /dev/null +++ b/src/test/hive/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a @@ -0,0 +1 @@ +{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1.db","database":"jsondb1"} \ No newline at end of file diff --git a/src/test/hive/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 b/src/test/hive/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 new file mode 100644 index 0000000000000..bea7c01440c46 --- /dev/null +++ b/src/test/hive/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 @@ -0,0 +1 @@ +{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1.db","database":"jsondb1"} \ No newline at end of file diff --git a/src/test/hive/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 b/src/test/hive/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa b/src/test/hive/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 b/src/test/hive/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 b/src/test/hive/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 new file mode 100644 index 0000000000000..381821184d693 --- /dev/null +++ b/src/test/hive/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 @@ -0,0 +1,19 @@ +# col_name data_type comment + +key int None + +# Partition Information +# col_name data_type comment + +value string None + +# Detailed Partition Information +Partition Value: [val_86] +Database: default +Table: view_partitioned +CreateTime: Fri Feb 07 15:09:16 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Location: null +Partition Parameters: + transient_lastDdlTime 1391814556 diff --git a/src/test/hive/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 b/src/test/hive/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/src/test/hive/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 b/src/test/hive/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa b/src/test/hive/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 b/src/test/hive/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 b/src/test/hive/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 new file mode 100644 index 0000000000000..7b51873776ad8 --- /dev/null +++ b/src/test/hive/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 @@ -0,0 +1 @@ +{"columns":[{"name":"key","type":"int"}]} diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 b/src/test/hive/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad b/src/test/hive/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf b/src/test/hive/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf b/src/test/hive/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 b/src/test/hive/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 new file mode 100644 index 0000000000000..35fac1b6f2579 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 @@ -0,0 +1,12 @@ +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/src/test/hive/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 b/src/test/hive/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 b/src/test/hive/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 new file mode 100644 index 0000000000000..b57f8955ca397 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 @@ -0,0 +1,24 @@ +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/src/test/hive/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 b/src/test/hive/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 b/src/test/hive/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 b/src/test/hive/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 new file mode 100644 index 0000000000000..3b7fe3c133089 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 @@ -0,0 +1,37 @@ +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/src/test/hive/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 b/src/test/hive/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 new file mode 100644 index 0000000000000..49175da27357f --- /dev/null +++ b/src/test/hive/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 @@ -0,0 +1,23 @@ +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/src/test/hive/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 b/src/test/hive/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 b/src/test/hive/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 new file mode 100644 index 0000000000000..c3e77e079a9d4 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 b/src/test/hive/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 b/src/test/hive/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 new file mode 100644 index 0000000000000..452f75a11fdd0 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 @@ -0,0 +1,44 @@ +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/src/test/hive/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 b/src/test/hive/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 b/src/test/hive/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 new file mode 100644 index 0000000000000..ee5a10c85057a --- /dev/null +++ b/src/test/hive/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 @@ -0,0 +1,31 @@ +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/src/test/hive/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 b/src/test/hive/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a b/src/test/hive/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 b/src/test/hive/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 b/src/test/hive/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 new file mode 100644 index 0000000000000..4184ce21dc079 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 @@ -0,0 +1,12 @@ +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/src/test/hive/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f b/src/test/hive/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f new file mode 100644 index 0000000000000..c94d6dcb90042 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f @@ -0,0 +1,33 @@ +# 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/src/test/hive/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 b/src/test/hive/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 b/src/test/hive/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 b/src/test/hive/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 new file mode 100644 index 0000000000000..f3d242157dd98 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 @@ -0,0 +1,3 @@ +# col_name data_type comment + +key1 int from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 b/src/test/hive/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 b/src/test/hive/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 b/src/test/hive/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 new file mode 100644 index 0000000000000..f3d242157dd98 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 @@ -0,0 +1,3 @@ +# col_name data_type comment + +key1 int from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 b/src/test/hive/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa b/src/test/hive/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a b/src/test/hive/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a b/src/test/hive/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a new file mode 100644 index 0000000000000..f3d242157dd98 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a @@ -0,0 +1,3 @@ +# col_name data_type comment + +key1 int from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 b/src/test/hive/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 new file mode 100644 index 0000000000000..4f76eaca6cd8b --- /dev/null +++ b/src/test/hive/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f b/src/test/hive/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f new file mode 100644 index 0000000000000..aa25ca5a29dd3 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f @@ -0,0 +1,12 @@ +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/src/test/hive/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 b/src/test/hive/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 new file mode 100644 index 0000000000000..311870f6ad6b0 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/describe_syntax-24-ee226b42db35b219702319858e925468 b/src/test/hive/golden/describe_syntax-24-ee226b42db35b219702319858e925468 new file mode 100644 index 0000000000000..4f76eaca6cd8b --- /dev/null +++ b/src/test/hive/golden/describe_syntax-24-ee226b42db35b219702319858e925468 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e b/src/test/hive/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e new file mode 100644 index 0000000000000..aa25ca5a29dd3 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e @@ -0,0 +1,12 @@ +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/src/test/hive/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec b/src/test/hive/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec new file mode 100644 index 0000000000000..311870f6ad6b0 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 b/src/test/hive/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 b/src/test/hive/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 b/src/test/hive/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 b/src/test/hive/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 new file mode 100644 index 0000000000000..4f76eaca6cd8b --- /dev/null +++ b/src/test/hive/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d b/src/test/hive/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d new file mode 100644 index 0000000000000..4184ce21dc079 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d @@ -0,0 +1,12 @@ +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/src/test/hive/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 b/src/test/hive/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 new file mode 100644 index 0000000000000..c94d6dcb90042 --- /dev/null +++ b/src/test/hive/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 @@ -0,0 +1,33 @@ +# 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/src/test/hive/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b b/src/test/hive/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b new file mode 100644 index 0000000000000..4f76eaca6cd8b --- /dev/null +++ b/src/test/hive/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b @@ -0,0 +1,10 @@ +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/src/test/hive/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/src/test/hive/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 b/src/test/hive/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..f8bc404bf7308 --- /dev/null +++ b/src/test/hive/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c @@ -0,0 +1 @@ +{"tables":["jsontable","src","srcpart"]} \ No newline at end of file diff --git a/src/test/hive/golden/describe_table_json-3-576670af142149302decb9bf8662e68a b/src/test/hive/golden/describe_table_json-3-576670af142149302decb9bf8662e68a new file mode 100644 index 0000000000000..5895645dbbb50 --- /dev/null +++ b/src/test/hive/golden/describe_table_json-3-576670af142149302decb9bf8662e68a @@ -0,0 +1 @@ +{"tables":["jsontable"]} \ No newline at end of file diff --git a/src/test/hive/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 b/src/test/hive/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 new file mode 100644 index 0000000000000..353bf2df92f18 --- /dev/null +++ b/src/test/hive/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 @@ -0,0 +1 @@ +{"tables":[]} \ No newline at end of file diff --git a/src/test/hive/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb b/src/test/hive/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 b/src/test/hive/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 new file mode 100644 index 0000000000000..96c1178ae6eab --- /dev/null +++ b/src/test/hive/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 @@ -0,0 +1 @@ +{"columns":[{"name":"key","type":"int"},{"name":"value","type":"string"}]} \ No newline at end of file diff --git a/src/test/hive/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 b/src/test/hive/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 new file mode 100644 index 0000000000000..4cf10d1d762b0 --- /dev/null +++ b/src/test/hive/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 @@ -0,0 +1 @@ +{"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/src/test/hive/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 b/src/test/hive/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 b/src/test/hive/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad b/src/test/hive/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa b/src/test/hive/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce b/src/test/hive/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 b/src/test/hive/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b b/src/test/hive/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c b/src/test/hive/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e b/src/test/hive/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 b/src/test/hive/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f b/src/test/hive/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb b/src/test/hive/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a b/src/test/hive/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b b/src/test/hive/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b new file mode 100644 index 0000000000000..7ae602958428e --- /dev/null +++ b/src/test/hive/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b @@ -0,0 +1 @@ +238 val_238 \ No newline at end of file diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 b/src/test/hive/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 b/src/test/hive/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c b/src/test/hive/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c new file mode 100644 index 0000000000000..ca21e093aa698 --- /dev/null +++ b/src/test/hive/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c @@ -0,0 +1 @@ +1 {"a1":"b1"} foo1 \ No newline at end of file diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 b/src/test/hive/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab b/src/test/hive/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 b/src/test/hive/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/src/test/hive/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a b/src/test/hive/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/src/test/hive/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 b/src/test/hive/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b b/src/test/hive/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 b/src/test/hive/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 b/src/test/hive/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 new file mode 100644 index 0000000000000..c9d701778f9ab --- /dev/null +++ b/src/test/hive/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 @@ -0,0 +1 @@ +b=2/c=2 \ No newline at end of file diff --git a/src/test/hive/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 b/src/test/hive/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 b/src/test/hive/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 b/src/test/hive/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 new file mode 100644 index 0000000000000..53e09b6e34202 --- /dev/null +++ b/src/test/hive/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 @@ -0,0 +1,3 @@ +b=1/c=1 +b=1/c=2 +b=2/c=2 \ No newline at end of file diff --git a/src/test/hive/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 b/src/test/hive/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 new file mode 100644 index 0000000000000..31b543e8b4122 --- /dev/null +++ b/src/test/hive/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 @@ -0,0 +1,12 @@ +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/src/test/hive/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 b/src/test/hive/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 b/src/test/hive/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 new file mode 100644 index 0000000000000..c9d701778f9ab --- /dev/null +++ b/src/test/hive/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 @@ -0,0 +1 @@ +b=2/c=2 \ No newline at end of file diff --git a/src/test/hive/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 b/src/test/hive/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 b/src/test/hive/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 b/src/test/hive/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 b/src/test/hive/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 new file mode 100644 index 0000000000000..d1e9fd1cd0a21 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 @@ -0,0 +1,10 @@ +a string None +b int None +c string None +d string None + +# Partition Information +# col_name data_type comment + +c string None +d string None \ No newline at end of file diff --git a/src/test/hive/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b b/src/test/hive/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..3377ef5cf498f --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,9 @@ +c=Canada/d=3 +c=France/d=4 +c=Germany/d=2 +c=Greece/d=2 +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 diff --git a/src/test/hive/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 b/src/test/hive/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..40e71fb79ad0f --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,8 @@ +c=Canada/d=3 +c=France/d=4 +c=Germany/d=2 +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 diff --git a/src/test/hive/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 b/src/test/hive/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..c03d86a551c29 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,6 @@ +c=Canada/d=3 +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 diff --git a/src/test/hive/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b b/src/test/hive/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..133c0256f898f --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd b/src/test/hive/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..98e52eedc2b03 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,3 @@ +c=Canada/d=3 +c=France/d=4 +c=Germany/d=2 \ No newline at end of file diff --git a/src/test/hive/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 b/src/test/hive/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c b/src/test/hive/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..687f7dd7e13a0 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1 @@ +c=France/d=4 \ No newline at end of file diff --git a/src/test/hive/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 b/src/test/hive/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e b/src/test/hive/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..687f7dd7e13a0 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1 @@ +c=France/d=4 \ No newline at end of file diff --git a/src/test/hive/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c b/src/test/hive/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c b/src/test/hive/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d b/src/test/hive/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 b/src/test/hive/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e b/src/test/hive/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 b/src/test/hive/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 b/src/test/hive/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 b/src/test/hive/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 b/src/test/hive/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 b/src/test/hive/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 new file mode 100644 index 0000000000000..1329d173d6a21 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 @@ -0,0 +1,10 @@ +a string None +b int None +c int None +d int None + +# Partition Information +# col_name data_type comment + +c int None +d int None \ No newline at end of file diff --git a/src/test/hive/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..08051a26d24cc --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a b/src/test/hive/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..7f6e4ae8abf83 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,3 @@ +c=1/d=2 +c=3/d=1 +c=3/d=2 \ No newline at end of file diff --git a/src/test/hive/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c b/src/test/hive/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 b/src/test/hive/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 b/src/test/hive/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 b/src/test/hive/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a b/src/test/hive/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 b/src/test/hive/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 b/src/test/hive/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..226ef460b53a6 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,6 @@ +c=1/d=1 +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 diff --git a/src/test/hive/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a b/src/test/hive/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 b/src/test/hive/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 b/src/test/hive/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 new file mode 100644 index 0000000000000..01562f65d807f --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 @@ -0,0 +1,10 @@ +a string None +b int None +c string None +d int None + +# Partition Information +# col_name data_type comment + +c string None +d int None \ No newline at end of file diff --git a/src/test/hive/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..08051a26d24cc --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 b/src/test/hive/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..7f6e4ae8abf83 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,3 @@ +c=1/d=2 +c=3/d=1 +c=3/d=2 \ No newline at end of file diff --git a/src/test/hive/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c b/src/test/hive/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 b/src/test/hive/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec b/src/test/hive/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a b/src/test/hive/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 b/src/test/hive/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae b/src/test/hive/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 b/src/test/hive/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 b/src/test/hive/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..226ef460b53a6 --- /dev/null +++ b/src/test/hive/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1,6 @@ +c=1/d=1 +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 diff --git a/src/test/hive/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 b/src/test/hive/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb b/src/test/hive/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 b/src/test/hive/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 b/src/test/hive/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 b/src/test/hive/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 new file mode 100644 index 0000000000000..8caab1c99b27d --- /dev/null +++ b/src/test/hive/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 @@ -0,0 +1,10 @@ +c1 string None +c2 string None +p string None + +# Partition Information +# col_name data_type comment + +p string None + +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 diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 b/src/test/hive/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 b/src/test/hive/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/src/test/hive/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 b/src/test/hive/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 b/src/test/hive/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa b/src/test/hive/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 b/src/test/hive/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f b/src/test/hive/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 b/src/test/hive/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 b/src/test/hive/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 new file mode 100644 index 0000000000000..fbe12dca4efc8 --- /dev/null +++ b/src/test/hive/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 @@ -0,0 +1,3 @@ +p=p1 +p=p2 +p=p3 \ No newline at end of file diff --git a/src/test/hive/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 b/src/test/hive/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa b/src/test/hive/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 b/src/test/hive/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 b/src/test/hive/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/src/test/hive/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 b/src/test/hive/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 b/src/test/hive/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 b/src/test/hive/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 b/src/test/hive/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 b/src/test/hive/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 b/src/test/hive/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 b/src/test/hive/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 b/src/test/hive/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 b/src/test/hive/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 b/src/test/hive/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 b/src/test/hive/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d b/src/test/hive/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f b/src/test/hive/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 b/src/test/hive/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 b/src/test/hive/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 b/src/test/hive/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf b/src/test/hive/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf new file mode 100644 index 0000000000000..8273b7ed19da6 --- /dev/null +++ b/src/test/hive/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf @@ -0,0 +1,10 @@ +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/src/test/hive/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 b/src/test/hive/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 new file mode 100644 index 0000000000000..045906a29a1da --- /dev/null +++ b/src/test/hive/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 b/src/test/hive/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 b/src/test/hive/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape1-3-4267651148da591da38737028fdbd80 b/src/test/hive/golden/escape1-3-4267651148da591da38737028fdbd80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 b/src/test/hive/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape1-5-70729c3d79ded87e884c176138174645 b/src/test/hive/golden/escape1-5-70729c3d79ded87e884c176138174645 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 b/src/test/hive/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 new file mode 100644 index 0000000000000..a949a93dfcca6 --- /dev/null +++ b/src/test/hive/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 @@ -0,0 +1 @@ +128 diff --git a/src/test/hive/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 b/src/test/hive/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 new file mode 100644 index 0000000000000000000000000000000000000000..e2799aa7e14b5885aad48117308f489740683449 GIT binary patch literal 252 zcmV~$15yM4006)yw>8yeIstVOFf?K*Vo(yd3YKK%v^3K=qN#Hcaj zCQO<#ZN{uQ^A;>xvTVhwHS0EP+OloOu08t>96ECB#HlmqE?l~D?Z&M;_Z~cY^6bT{ OH}5`t`tt3^uRs6QQ6wq= literal 0 HcmV?d00001 diff --git a/src/test/hive/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 b/src/test/hive/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 b/src/test/hive/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e b/src/test/hive/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 b/src/test/hive/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe b/src/test/hive/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 b/src/test/hive/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 b/src/test/hive/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape2-7-70729c3d79ded87e884c176138174645 b/src/test/hive/golden/escape2-7-70729c3d79ded87e884c176138174645 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 b/src/test/hive/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 new file mode 100644 index 0000000000000..a949a93dfcca6 --- /dev/null +++ b/src/test/hive/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 @@ -0,0 +1 @@ +128 diff --git a/src/test/hive/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 b/src/test/hive/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 new file mode 100644 index 0000000000000000000000000000000000000000..e2799aa7e14b5885aad48117308f489740683449 GIT binary patch literal 252 zcmV~$15yM4006)yw>8yeIstVOFf?K*Vo(yd3YKK%v^3K=qN#Hcaj zCQO<#ZN{uQ^A;>xvTVhwHS0EP+OloOu08t>96ECB#HlmqE?l~D?Z&M;_Z~cY^6bT{ OH}5`t`tt3^uRs6QQ6wq= literal 0 HcmV?d00001 diff --git a/src/test/hive/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf b/src/test/hive/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 b/src/test/hive/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb b/src/test/hive/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 b/src/test/hive/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 b/src/test/hive/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 b/src/test/hive/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 b/src/test/hive/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 b/src/test/hive/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 b/src/test/hive/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 b/src/test/hive/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 b/src/test/hive/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd b/src/test/hive/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 b/src/test/hive/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 b/src/test/hive/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd b/src/test/hive/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 b/src/test/hive/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 b/src/test/hive/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 b/src/test/hive/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 b/src/test/hive/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 b/src/test/hive/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 b/src/test/hive/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 b/src/test/hive/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 b/src/test/hive/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 b/src/test/hive/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 b/src/test/hive/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd b/src/test/hive/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 b/src/test/hive/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d b/src/test/hive/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 b/src/test/hive/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 b/src/test/hive/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd b/src/test/hive/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 b/src/test/hive/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/src/test/hive/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd b/src/test/hive/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 b/src/test/hive/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/src/test/hive/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd b/src/test/hive/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 b/src/test/hive/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 b/src/test/hive/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd b/src/test/hive/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 b/src/test/hive/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 b/src/test/hive/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/src/test/hive/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/src/test/hive/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/src/test/hive/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 b/src/test/hive/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd b/src/test/hive/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e b/src/test/hive/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 b/src/test/hive/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 b/src/test/hive/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/src/test/hive/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/src/test/hive/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 b/src/test/hive/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 b/src/test/hive/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc b/src/test/hive/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/src/test/hive/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 b/src/test/hive/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 b/src/test/hive/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 b/src/test/hive/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 b/src/test/hive/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 new file mode 100644 index 0000000000000..fb12634ea81c1 --- /dev/null +++ b/src/test/hive/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 @@ -0,0 +1 @@ +500 130091 260.182 0 498 142.92680950752384 20428.072876000006 \ No newline at end of file diff --git a/src/test/hive/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 b/src/test/hive/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b b/src/test/hive/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e b/src/test/hive/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 b/src/test/hive/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 b/src/test/hive/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 b/src/test/hive/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 b/src/test/hive/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 new file mode 100644 index 0000000000000..d23e05acf7ba5 --- /dev/null +++ b/src/test/hive/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 @@ -0,0 +1,500 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 \ No newline at end of file diff --git a/src/test/hive/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 b/src/test/hive/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 b/src/test/hive/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 b/src/test/hive/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e b/src/test/hive/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e new file mode 100644 index 0000000000000..cc545367b951b --- /dev/null +++ b/src/test/hive/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e @@ -0,0 +1,12 @@ +238 val_238 +311 val_311 +255 val_255 +278 val_278 +98 val_98 +401 val_401 +150 val_150 +273 val_273 +66 val_66 +213 val_213 +146 val_146 +406 val_406 \ No newline at end of file diff --git a/src/test/hive/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 b/src/test/hive/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd b/src/test/hive/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd b/src/test/hive/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd new file mode 100644 index 0000000000000..140c6590a27fc --- /dev/null +++ b/src/test/hive/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd @@ -0,0 +1 @@ +4 5 0 2010-04-17 \ No newline at end of file diff --git a/src/test/hive/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 b/src/test/hive/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 new file mode 100644 index 0000000000000..51f5701ceae46 --- /dev/null +++ b/src/test/hive/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 @@ -0,0 +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 diff --git a/src/test/hive/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 b/src/test/hive/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 new file mode 100644 index 0000000000000..69dcc336a73f2 --- /dev/null +++ b/src/test/hive/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 @@ -0,0 +1 @@ +5 5 4 \ No newline at end of file diff --git a/src/test/hive/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c b/src/test/hive/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 b/src/test/hive/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 b/src/test/hive/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba b/src/test/hive/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 b/src/test/hive/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 b/src/test/hive/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e b/src/test/hive/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e new file mode 100644 index 0000000000000..c0f577c3cd2fd --- /dev/null +++ b/src/test/hive/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e @@ -0,0 +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 diff --git a/src/test/hive/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 b/src/test/hive/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 new file mode 100644 index 0000000000000..e426b4879bcb6 --- /dev/null +++ b/src/test/hive/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 @@ -0,0 +1 @@ +5 1 1 1 0 0 4 2010-04-17 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1-2-f90acd59ace31e16059bae52583188cc b/src/test/hive/golden/groupby1-2-f90acd59ace31e16059bae52583188cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d b/src/test/hive/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 b/src/test/hive/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 b/src/test/hive/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea b/src/test/hive/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 b/src/test/hive/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 b/src/test/hive/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 b/src/test/hive/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 b/src/test/hive/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f b/src/test/hive/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd b/src/test/hive/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd new file mode 100644 index 0000000000000..8939d9f8af186 --- /dev/null +++ b/src/test/hive/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd @@ -0,0 +1,309 @@ +val_0 3 1 111 +val_10 1 1 111 +val_100 2 1 111 +val_103 2 1 111 +val_104 2 1 111 +val_105 1 1 111 +val_11 1 1 111 +val_111 1 1 111 +val_113 2 1 111 +val_114 1 1 111 +val_116 1 1 111 +val_118 2 1 111 +val_119 3 1 111 +val_12 2 1 111 +val_120 2 1 111 +val_125 2 1 111 +val_126 1 1 111 +val_128 3 1 111 +val_129 2 1 111 +val_131 1 1 111 +val_133 1 1 111 +val_134 2 1 111 +val_136 1 1 111 +val_137 2 1 111 +val_138 4 1 111 +val_143 1 1 111 +val_145 1 1 111 +val_146 2 1 111 +val_149 2 1 111 +val_15 2 1 111 +val_150 1 1 111 +val_152 2 1 111 +val_153 1 1 111 +val_155 1 1 111 +val_156 1 1 111 +val_157 1 1 111 +val_158 1 1 111 +val_160 1 1 111 +val_162 1 1 111 +val_163 1 1 111 +val_164 2 1 111 +val_165 2 1 111 +val_166 1 1 111 +val_167 3 1 111 +val_168 1 1 111 +val_169 4 1 111 +val_17 1 1 111 +val_170 1 1 111 +val_172 2 1 111 +val_174 2 1 111 +val_175 2 1 111 +val_176 2 1 111 +val_177 1 1 111 +val_178 1 1 111 +val_179 2 1 111 +val_18 2 1 111 +val_180 1 1 111 +val_181 1 1 111 +val_183 1 1 111 +val_186 1 1 111 +val_187 3 1 111 +val_189 1 1 111 +val_19 1 1 111 +val_190 1 1 111 +val_191 2 1 111 +val_192 1 1 111 +val_193 3 1 111 +val_194 1 1 111 +val_195 2 1 111 +val_196 1 1 111 +val_197 2 1 111 +val_199 3 1 111 +val_2 1 1 111 +val_20 1 1 111 +val_200 2 1 111 +val_201 1 1 111 +val_202 1 1 111 +val_203 2 1 111 +val_205 2 1 111 +val_207 2 1 111 +val_208 3 1 111 +val_209 2 1 111 +val_213 2 1 111 +val_214 1 1 111 +val_216 2 1 111 +val_217 2 1 111 +val_218 1 1 111 +val_219 2 1 111 +val_221 2 1 111 +val_222 1 1 111 +val_223 2 1 111 +val_224 2 1 111 +val_226 1 1 111 +val_228 1 1 111 +val_229 2 1 111 +val_230 5 1 111 +val_233 2 1 111 +val_235 1 1 111 +val_237 2 1 111 +val_238 2 1 111 +val_239 2 1 111 +val_24 2 1 111 +val_241 1 1 111 +val_242 2 1 111 +val_244 1 1 111 +val_247 1 1 111 +val_248 1 1 111 +val_249 1 1 111 +val_252 1 1 111 +val_255 2 1 111 +val_256 2 1 111 +val_257 1 1 111 +val_258 1 1 111 +val_26 2 1 111 +val_260 1 1 111 +val_262 1 1 111 +val_263 1 1 111 +val_265 2 1 111 +val_266 1 1 111 +val_27 1 1 111 +val_272 2 1 111 +val_273 3 1 111 +val_274 1 1 111 +val_275 1 1 111 +val_277 4 1 111 +val_278 2 1 111 +val_28 1 1 111 +val_280 2 1 111 +val_281 2 1 111 +val_282 2 1 111 +val_283 1 1 111 +val_284 1 1 111 +val_285 1 1 111 +val_286 1 1 111 +val_287 1 1 111 +val_288 2 1 111 +val_289 1 1 111 +val_291 1 1 111 +val_292 1 1 111 +val_296 1 1 111 +val_298 3 1 111 +val_30 1 1 111 +val_302 1 1 111 +val_305 1 1 111 +val_306 1 1 111 +val_307 2 1 111 +val_308 1 1 111 +val_309 2 1 111 +val_310 1 1 111 +val_311 3 1 111 +val_315 1 1 111 +val_316 3 1 111 +val_317 2 1 111 +val_318 3 1 111 +val_321 2 1 111 +val_322 2 1 111 +val_323 1 1 111 +val_325 2 1 111 +val_327 3 1 111 +val_33 1 1 111 +val_331 2 1 111 +val_332 1 1 111 +val_333 2 1 111 +val_335 1 1 111 +val_336 1 1 111 +val_338 1 1 111 +val_339 1 1 111 +val_34 1 1 111 +val_341 1 1 111 +val_342 2 1 111 +val_344 2 1 111 +val_345 1 1 111 +val_348 5 1 111 +val_35 3 1 111 +val_351 1 1 111 +val_353 2 1 111 +val_356 1 1 111 +val_360 1 1 111 +val_362 1 1 111 +val_364 1 1 111 +val_365 1 1 111 +val_366 1 1 111 +val_367 2 1 111 +val_368 1 1 111 +val_369 3 1 111 +val_37 2 1 111 +val_373 1 1 111 +val_374 1 1 111 +val_375 1 1 111 +val_377 1 1 111 +val_378 1 1 111 +val_379 1 1 111 +val_382 2 1 111 +val_384 3 1 111 +val_386 1 1 111 +val_389 1 1 111 +val_392 1 1 111 +val_393 1 1 111 +val_394 1 1 111 +val_395 2 1 111 +val_396 3 1 111 +val_397 2 1 111 +val_399 2 1 111 +val_4 1 1 111 +val_400 1 1 111 +val_401 5 1 111 +val_402 1 1 111 +val_403 3 1 111 +val_404 2 1 111 +val_406 4 1 111 +val_407 1 1 111 +val_409 3 1 111 +val_41 1 1 111 +val_411 1 1 111 +val_413 2 1 111 +val_414 2 1 111 +val_417 3 1 111 +val_418 1 1 111 +val_419 1 1 111 +val_42 2 1 111 +val_421 1 1 111 +val_424 2 1 111 +val_427 1 1 111 +val_429 2 1 111 +val_43 1 1 111 +val_430 3 1 111 +val_431 3 1 111 +val_432 1 1 111 +val_435 1 1 111 +val_436 1 1 111 +val_437 1 1 111 +val_438 3 1 111 +val_439 2 1 111 +val_44 1 1 111 +val_443 1 1 111 +val_444 1 1 111 +val_446 1 1 111 +val_448 1 1 111 +val_449 1 1 111 +val_452 1 1 111 +val_453 1 1 111 +val_454 3 1 111 +val_455 1 1 111 +val_457 1 1 111 +val_458 2 1 111 +val_459 2 1 111 +val_460 1 1 111 +val_462 2 1 111 +val_463 2 1 111 +val_466 3 1 111 +val_467 1 1 111 +val_468 4 1 111 +val_469 5 1 111 +val_47 1 1 111 +val_470 1 1 111 +val_472 1 1 111 +val_475 1 1 111 +val_477 1 1 111 +val_478 2 1 111 +val_479 1 1 111 +val_480 3 1 111 +val_481 1 1 111 +val_482 1 1 111 +val_483 1 1 111 +val_484 1 1 111 +val_485 1 1 111 +val_487 1 1 111 +val_489 4 1 111 +val_490 1 1 111 +val_491 1 1 111 +val_492 2 1 111 +val_493 1 1 111 +val_494 1 1 111 +val_495 1 1 111 +val_496 1 1 111 +val_497 1 1 111 +val_498 3 1 111 +val_5 3 1 111 +val_51 2 1 111 +val_53 1 1 111 +val_54 1 1 111 +val_57 1 1 111 +val_58 2 1 111 +val_64 1 1 111 +val_65 1 1 111 +val_66 1 1 111 +val_67 2 1 111 +val_69 1 1 111 +val_70 3 1 111 +val_72 2 1 111 +val_74 1 1 111 +val_76 2 1 111 +val_77 1 1 111 +val_78 1 1 111 +val_8 1 1 111 +val_80 1 1 111 +val_82 1 1 111 +val_83 2 1 111 +val_84 2 1 111 +val_85 1 1 111 +val_86 1 1 111 +val_87 1 1 111 +val_9 1 1 111 +val_90 3 1 111 +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 diff --git a/src/test/hive/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d b/src/test/hive/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d new file mode 100644 index 0000000000000..82a0329b4d459 --- /dev/null +++ b/src/test/hive/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d @@ -0,0 +1,309 @@ +0 3 1 111 +10 1 1 111 +100 2 1 111 +103 2 1 111 +104 2 1 111 +105 1 1 111 +11 1 1 111 +111 1 1 111 +113 2 1 111 +114 1 1 111 +116 1 1 111 +118 2 1 111 +119 3 1 111 +12 2 1 111 +120 2 1 111 +125 2 1 111 +126 1 1 111 +128 3 1 111 +129 2 1 111 +131 1 1 111 +133 1 1 111 +134 2 1 111 +136 1 1 111 +137 2 1 111 +138 4 1 111 +143 1 1 111 +145 1 1 111 +146 2 1 111 +149 2 1 111 +15 2 1 111 +150 1 1 111 +152 2 1 111 +153 1 1 111 +155 1 1 111 +156 1 1 111 +157 1 1 111 +158 1 1 111 +160 1 1 111 +162 1 1 111 +163 1 1 111 +164 2 1 111 +165 2 1 111 +166 1 1 111 +167 3 1 111 +168 1 1 111 +169 4 1 111 +17 1 1 111 +170 1 1 111 +172 2 1 111 +174 2 1 111 +175 2 1 111 +176 2 1 111 +177 1 1 111 +178 1 1 111 +179 2 1 111 +18 2 1 111 +180 1 1 111 +181 1 1 111 +183 1 1 111 +186 1 1 111 +187 3 1 111 +189 1 1 111 +19 1 1 111 +190 1 1 111 +191 2 1 111 +192 1 1 111 +193 3 1 111 +194 1 1 111 +195 2 1 111 +196 1 1 111 +197 2 1 111 +199 3 1 111 +2 1 1 111 +20 1 1 111 +200 2 1 111 +201 1 1 111 +202 1 1 111 +203 2 1 111 +205 2 1 111 +207 2 1 111 +208 3 1 111 +209 2 1 111 +213 2 1 111 +214 1 1 111 +216 2 1 111 +217 2 1 111 +218 1 1 111 +219 2 1 111 +221 2 1 111 +222 1 1 111 +223 2 1 111 +224 2 1 111 +226 1 1 111 +228 1 1 111 +229 2 1 111 +230 5 1 111 +233 2 1 111 +235 1 1 111 +237 2 1 111 +238 2 1 111 +239 2 1 111 +24 2 1 111 +241 1 1 111 +242 2 1 111 +244 1 1 111 +247 1 1 111 +248 1 1 111 +249 1 1 111 +252 1 1 111 +255 2 1 111 +256 2 1 111 +257 1 1 111 +258 1 1 111 +26 2 1 111 +260 1 1 111 +262 1 1 111 +263 1 1 111 +265 2 1 111 +266 1 1 111 +27 1 1 111 +272 2 1 111 +273 3 1 111 +274 1 1 111 +275 1 1 111 +277 4 1 111 +278 2 1 111 +28 1 1 111 +280 2 1 111 +281 2 1 111 +282 2 1 111 +283 1 1 111 +284 1 1 111 +285 1 1 111 +286 1 1 111 +287 1 1 111 +288 2 1 111 +289 1 1 111 +291 1 1 111 +292 1 1 111 +296 1 1 111 +298 3 1 111 +30 1 1 111 +302 1 1 111 +305 1 1 111 +306 1 1 111 +307 2 1 111 +308 1 1 111 +309 2 1 111 +310 1 1 111 +311 3 1 111 +315 1 1 111 +316 3 1 111 +317 2 1 111 +318 3 1 111 +321 2 1 111 +322 2 1 111 +323 1 1 111 +325 2 1 111 +327 3 1 111 +33 1 1 111 +331 2 1 111 +332 1 1 111 +333 2 1 111 +335 1 1 111 +336 1 1 111 +338 1 1 111 +339 1 1 111 +34 1 1 111 +341 1 1 111 +342 2 1 111 +344 2 1 111 +345 1 1 111 +348 5 1 111 +35 3 1 111 +351 1 1 111 +353 2 1 111 +356 1 1 111 +360 1 1 111 +362 1 1 111 +364 1 1 111 +365 1 1 111 +366 1 1 111 +367 2 1 111 +368 1 1 111 +369 3 1 111 +37 2 1 111 +373 1 1 111 +374 1 1 111 +375 1 1 111 +377 1 1 111 +378 1 1 111 +379 1 1 111 +382 2 1 111 +384 3 1 111 +386 1 1 111 +389 1 1 111 +392 1 1 111 +393 1 1 111 +394 1 1 111 +395 2 1 111 +396 3 1 111 +397 2 1 111 +399 2 1 111 +4 1 1 111 +400 1 1 111 +401 5 1 111 +402 1 1 111 +403 3 1 111 +404 2 1 111 +406 4 1 111 +407 1 1 111 +409 3 1 111 +41 1 1 111 +411 1 1 111 +413 2 1 111 +414 2 1 111 +417 3 1 111 +418 1 1 111 +419 1 1 111 +42 2 1 111 +421 1 1 111 +424 2 1 111 +427 1 1 111 +429 2 1 111 +43 1 1 111 +430 3 1 111 +431 3 1 111 +432 1 1 111 +435 1 1 111 +436 1 1 111 +437 1 1 111 +438 3 1 111 +439 2 1 111 +44 1 1 111 +443 1 1 111 +444 1 1 111 +446 1 1 111 +448 1 1 111 +449 1 1 111 +452 1 1 111 +453 1 1 111 +454 3 1 111 +455 1 1 111 +457 1 1 111 +458 2 1 111 +459 2 1 111 +460 1 1 111 +462 2 1 111 +463 2 1 111 +466 3 1 111 +467 1 1 111 +468 4 1 111 +469 5 1 111 +47 1 1 111 +470 1 1 111 +472 1 1 111 +475 1 1 111 +477 1 1 111 +478 2 1 111 +479 1 1 111 +480 3 1 111 +481 1 1 111 +482 1 1 111 +483 1 1 111 +484 1 1 111 +485 1 1 111 +487 1 1 111 +489 4 1 111 +490 1 1 111 +491 1 1 111 +492 2 1 111 +493 1 1 111 +494 1 1 111 +495 1 1 111 +496 1 1 111 +497 1 1 111 +498 3 1 111 +5 3 1 111 +51 2 1 111 +53 1 1 111 +54 1 1 111 +57 1 1 111 +58 2 1 111 +64 1 1 111 +65 1 1 111 +66 1 1 111 +67 2 1 111 +69 1 1 111 +70 3 1 111 +72 2 1 111 +74 1 1 111 +76 2 1 111 +77 1 1 111 +78 1 1 111 +8 1 1 111 +80 1 1 111 +82 1 1 111 +83 2 1 111 +84 2 1 111 +85 1 1 111 +86 1 1 111 +87 1 1 111 +9 1 1 111 +90 3 1 111 +92 1 1 111 +95 2 1 111 +96 1 1 111 +97 2 1 111 +98 2 1 111 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 b/src/test/hive/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 b/src/test/hive/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 b/src/test/hive/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 b/src/test/hive/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 new file mode 100644 index 0000000000000..9cf9606d751ac --- /dev/null +++ b/src/test/hive/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 @@ -0,0 +1,5 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 b/src/test/hive/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe b/src/test/hive/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 b/src/test/hive/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 b/src/test/hive/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 b/src/test/hive/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe b/src/test/hive/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 b/src/test/hive/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 b/src/test/hive/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe b/src/test/hive/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 b/src/test/hive/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc b/src/test/hive/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 b/src/test/hive/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea b/src/test/hive/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 b/src/test/hive/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2-0-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/groupby2-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b b/src/test/hive/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 b/src/test/hive/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 b/src/test/hive/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c b/src/test/hive/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c new file mode 100644 index 0000000000000..3210a3048ff48 --- /dev/null +++ b/src/test/hive/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 b/src/test/hive/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 b/src/test/hive/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 new file mode 100644 index 0000000000000..9cf9606d751ac --- /dev/null +++ b/src/test/hive/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 @@ -0,0 +1,5 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 b/src/test/hive/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 b/src/test/hive/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 b/src/test/hive/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..3210a3048ff48 --- /dev/null +++ b/src/test/hive/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 b/src/test/hive/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 b/src/test/hive/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 b/src/test/hive/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..922f4bfc83e44 --- /dev/null +++ b/src/test/hive/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 b/src/test/hive/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 b/src/test/hive/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 b/src/test/hive/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..3210a3048ff48 --- /dev/null +++ b/src/test/hive/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b b/src/test/hive/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 b/src/test/hive/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 b/src/test/hive/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c b/src/test/hive/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c new file mode 100644 index 0000000000000..3210a3048ff48 --- /dev/null +++ b/src/test/hive/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 b/src/test/hive/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 b/src/test/hive/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 b/src/test/hive/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c b/src/test/hive/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c new file mode 100644 index 0000000000000..922f4bfc83e44 --- /dev/null +++ b/src/test/hive/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 b/src/test/hive/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 b/src/test/hive/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/src/test/hive/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 b/src/test/hive/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b b/src/test/hive/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 b/src/test/hive/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 b/src/test/hive/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b b/src/test/hive/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 b/src/test/hive/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 b/src/test/hive/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 b/src/test/hive/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/src/test/hive/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 b/src/test/hive/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e b/src/test/hive/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 b/src/test/hive/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f b/src/test/hive/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 b/src/test/hive/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..349d8b75d942b --- /dev/null +++ b/src/test/hive/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +130091 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 b/src/test/hive/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f b/src/test/hive/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 b/src/test/hive/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..349d8b75d942b --- /dev/null +++ b/src/test/hive/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +130091 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 b/src/test/hive/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e b/src/test/hive/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac b/src/test/hive/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 b/src/test/hive/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/src/test/hive/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac b/src/test/hive/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 b/src/test/hive/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/src/test/hive/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac b/src/test/hive/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 b/src/test/hive/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/src/test/hive/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac b/src/test/hive/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 b/src/test/hive/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/src/test/hive/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a b/src/test/hive/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/src/test/hive/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7-5-ed76c0068780120a6f23feefee303403 b/src/test/hive/golden/groupby7-5-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7-5-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 b/src/test/hive/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e b/src/test/hive/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a b/src/test/hive/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/src/test/hive/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 b/src/test/hive/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd b/src/test/hive/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 b/src/test/hive/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a b/src/test/hive/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/src/test/hive/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/src/test/hive/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd b/src/test/hive/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 b/src/test/hive/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a b/src/test/hive/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/src/test/hive/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 b/src/test/hive/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd b/src/test/hive/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 b/src/test/hive/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e b/src/test/hive/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a b/src/test/hive/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/src/test/hive/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 b/src/test/hive/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd b/src/test/hive/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 b/src/test/hive/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 new file mode 100644 index 0000000000000..dfca4e169cbe8 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 @@ -0,0 +1,10 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 new file mode 100644 index 0000000000000..dfca4e169cbe8 --- /dev/null +++ b/src/test/hive/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 @@ -0,0 +1,10 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f b/src/test/hive/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a b/src/test/hive/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d b/src/test/hive/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f b/src/test/hive/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e b/src/test/hive/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d b/src/test/hive/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a b/src/test/hive/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d b/src/test/hive/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f b/src/test/hive/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a b/src/test/hive/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d b/src/test/hive/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f b/src/test/hive/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a b/src/test/hive/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d b/src/test/hive/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f b/src/test/hive/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 b/src/test/hive/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 b/src/test/hive/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e b/src/test/hive/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f b/src/test/hive/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 b/src/test/hive/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..0e40f3f1a49a9 --- /dev/null +++ b/src/test/hive/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 @@ -0,0 +1,309 @@ +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 +20 val_20 1 +24 val_24 1 +26 val_26 1 +27 val_27 1 +28 val_28 1 +30 val_30 1 +33 val_33 1 +34 val_34 1 +35 val_35 1 +37 val_37 1 +41 val_41 1 +42 val_42 1 +43 val_43 1 +44 val_44 1 +47 val_47 1 +51 val_51 1 +53 val_53 1 +54 val_54 1 +57 val_57 1 +58 val_58 1 +64 val_64 1 +65 val_65 1 +66 val_66 1 +67 val_67 1 +69 val_69 1 +70 val_70 1 +72 val_72 1 +74 val_74 1 +76 val_76 1 +77 val_77 1 +78 val_78 1 +80 val_80 1 +82 val_82 1 +83 val_83 1 +84 val_84 1 +85 val_85 1 +86 val_86 1 +87 val_87 1 +90 val_90 1 +92 val_92 1 +95 val_95 1 +96 val_96 1 +97 val_97 1 +98 val_98 1 +100 val_100 1 +103 val_103 1 +104 val_104 1 +105 val_105 1 +111 val_111 1 +113 val_113 1 +114 val_114 1 +116 val_116 1 +118 val_118 1 +119 val_119 1 +120 val_120 1 +125 val_125 1 +126 val_126 1 +128 val_128 1 +129 val_129 1 +131 val_131 1 +133 val_133 1 +134 val_134 1 +136 val_136 1 +137 val_137 1 +138 val_138 1 +143 val_143 1 +145 val_145 1 +146 val_146 1 +149 val_149 1 +150 val_150 1 +152 val_152 1 +153 val_153 1 +155 val_155 1 +156 val_156 1 +157 val_157 1 +158 val_158 1 +160 val_160 1 +162 val_162 1 +163 val_163 1 +164 val_164 1 +165 val_165 1 +166 val_166 1 +167 val_167 1 +168 val_168 1 +169 val_169 1 +170 val_170 1 +172 val_172 1 +174 val_174 1 +175 val_175 1 +176 val_176 1 +177 val_177 1 +178 val_178 1 +179 val_179 1 +180 val_180 1 +181 val_181 1 +183 val_183 1 +186 val_186 1 +187 val_187 1 +189 val_189 1 +190 val_190 1 +191 val_191 1 +192 val_192 1 +193 val_193 1 +194 val_194 1 +195 val_195 1 +196 val_196 1 +197 val_197 1 +199 val_199 1 +200 val_200 1 +201 val_201 1 +202 val_202 1 +203 val_203 1 +205 val_205 1 +207 val_207 1 +208 val_208 1 +209 val_209 1 +213 val_213 1 +214 val_214 1 +216 val_216 1 +217 val_217 1 +218 val_218 1 +219 val_219 1 +221 val_221 1 +222 val_222 1 +223 val_223 1 +224 val_224 1 +226 val_226 1 +228 val_228 1 +229 val_229 1 +230 val_230 1 +233 val_233 1 +235 val_235 1 +237 val_237 1 +238 val_238 1 +239 val_239 1 +241 val_241 1 +242 val_242 1 +244 val_244 1 +247 val_247 1 +248 val_248 1 +249 val_249 1 +252 val_252 1 +255 val_255 1 +256 val_256 1 +257 val_257 1 +258 val_258 1 +260 val_260 1 +262 val_262 1 +263 val_263 1 +265 val_265 1 +266 val_266 1 +272 val_272 1 +273 val_273 1 +274 val_274 1 +275 val_275 1 +277 val_277 1 +278 val_278 1 +280 val_280 1 +281 val_281 1 +282 val_282 1 +283 val_283 1 +284 val_284 1 +285 val_285 1 +286 val_286 1 +287 val_287 1 +288 val_288 1 +289 val_289 1 +291 val_291 1 +292 val_292 1 +296 val_296 1 +298 val_298 1 +302 val_302 1 +305 val_305 1 +306 val_306 1 +307 val_307 1 +308 val_308 1 +309 val_309 1 +310 val_310 1 +311 val_311 1 +315 val_315 1 +316 val_316 1 +317 val_317 1 +318 val_318 1 +321 val_321 1 +322 val_322 1 +323 val_323 1 +325 val_325 1 +327 val_327 1 +331 val_331 1 +332 val_332 1 +333 val_333 1 +335 val_335 1 +336 val_336 1 +338 val_338 1 +339 val_339 1 +341 val_341 1 +342 val_342 1 +344 val_344 1 +345 val_345 1 +348 val_348 1 +351 val_351 1 +353 val_353 1 +356 val_356 1 +360 val_360 1 +362 val_362 1 +364 val_364 1 +365 val_365 1 +366 val_366 1 +367 val_367 1 +368 val_368 1 +369 val_369 1 +373 val_373 1 +374 val_374 1 +375 val_375 1 +377 val_377 1 +378 val_378 1 +379 val_379 1 +382 val_382 1 +384 val_384 1 +386 val_386 1 +389 val_389 1 +392 val_392 1 +393 val_393 1 +394 val_394 1 +395 val_395 1 +396 val_396 1 +397 val_397 1 +399 val_399 1 +400 val_400 1 +401 val_401 1 +402 val_402 1 +403 val_403 1 +404 val_404 1 +406 val_406 1 +407 val_407 1 +409 val_409 1 +411 val_411 1 +413 val_413 1 +414 val_414 1 +417 val_417 1 +418 val_418 1 +419 val_419 1 +421 val_421 1 +424 val_424 1 +427 val_427 1 +429 val_429 1 +430 val_430 1 +431 val_431 1 +432 val_432 1 +435 val_435 1 +436 val_436 1 +437 val_437 1 +438 val_438 1 +439 val_439 1 +443 val_443 1 +444 val_444 1 +446 val_446 1 +448 val_448 1 +449 val_449 1 +452 val_452 1 +453 val_453 1 +454 val_454 1 +455 val_455 1 +457 val_457 1 +458 val_458 1 +459 val_459 1 +460 val_460 1 +462 val_462 1 +463 val_463 1 +466 val_466 1 +467 val_467 1 +468 val_468 1 +469 val_469 1 +470 val_470 1 +472 val_472 1 +475 val_475 1 +477 val_477 1 +478 val_478 1 +479 val_479 1 +480 val_480 1 +481 val_481 1 +482 val_482 1 +483 val_483 1 +484 val_484 1 +485 val_485 1 +487 val_487 1 +489 val_489 1 +490 val_490 1 +491 val_491 1 +492 val_492 1 +493 val_493 1 +494 val_494 1 +495 val_495 1 +496 val_496 1 +497 val_497 1 +498 val_498 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed b/src/test/hive/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b b/src/test/hive/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..df07a9da29f01 --- /dev/null +++ b/src/test/hive/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -0,0 +1,309 @@ +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/src/test/hive/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..cf0c065dc6532 --- /dev/null +++ b/src/test/hive/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 @@ -0,0 +1,309 @@ +0 val_0 3 +2 val_2 1 +4 val_4 1 +5 val_5 3 +8 val_8 1 +9 val_9 1 +10 val_10 1 +11 val_11 1 +12 val_12 2 +15 val_15 2 +17 val_17 1 +18 val_18 2 +19 val_19 1 +20 val_20 1 +24 val_24 2 +26 val_26 2 +27 val_27 1 +28 val_28 1 +30 val_30 1 +33 val_33 1 +34 val_34 1 +35 val_35 3 +37 val_37 2 +41 val_41 1 +42 val_42 2 +43 val_43 1 +44 val_44 1 +47 val_47 1 +51 val_51 2 +53 val_53 1 +54 val_54 1 +57 val_57 1 +58 val_58 2 +64 val_64 1 +65 val_65 1 +66 val_66 1 +67 val_67 2 +69 val_69 1 +70 val_70 3 +72 val_72 2 +74 val_74 1 +76 val_76 2 +77 val_77 1 +78 val_78 1 +80 val_80 1 +82 val_82 1 +83 val_83 2 +84 val_84 2 +85 val_85 1 +86 val_86 1 +87 val_87 1 +90 val_90 3 +92 val_92 1 +95 val_95 2 +96 val_96 1 +97 val_97 2 +98 val_98 2 +100 val_100 2 +103 val_103 2 +104 val_104 2 +105 val_105 1 +111 val_111 1 +113 val_113 2 +114 val_114 1 +116 val_116 1 +118 val_118 2 +119 val_119 3 +120 val_120 2 +125 val_125 2 +126 val_126 1 +128 val_128 3 +129 val_129 2 +131 val_131 1 +133 val_133 1 +134 val_134 2 +136 val_136 1 +137 val_137 2 +138 val_138 4 +143 val_143 1 +145 val_145 1 +146 val_146 2 +149 val_149 2 +150 val_150 1 +152 val_152 2 +153 val_153 1 +155 val_155 1 +156 val_156 1 +157 val_157 1 +158 val_158 1 +160 val_160 1 +162 val_162 1 +163 val_163 1 +164 val_164 2 +165 val_165 2 +166 val_166 1 +167 val_167 3 +168 val_168 1 +169 val_169 4 +170 val_170 1 +172 val_172 2 +174 val_174 2 +175 val_175 2 +176 val_176 2 +177 val_177 1 +178 val_178 1 +179 val_179 2 +180 val_180 1 +181 val_181 1 +183 val_183 1 +186 val_186 1 +187 val_187 3 +189 val_189 1 +190 val_190 1 +191 val_191 2 +192 val_192 1 +193 val_193 3 +194 val_194 1 +195 val_195 2 +196 val_196 1 +197 val_197 2 +199 val_199 3 +200 val_200 2 +201 val_201 1 +202 val_202 1 +203 val_203 2 +205 val_205 2 +207 val_207 2 +208 val_208 3 +209 val_209 2 +213 val_213 2 +214 val_214 1 +216 val_216 2 +217 val_217 2 +218 val_218 1 +219 val_219 2 +221 val_221 2 +222 val_222 1 +223 val_223 2 +224 val_224 2 +226 val_226 1 +228 val_228 1 +229 val_229 2 +230 val_230 5 +233 val_233 2 +235 val_235 1 +237 val_237 2 +238 val_238 2 +239 val_239 2 +241 val_241 1 +242 val_242 2 +244 val_244 1 +247 val_247 1 +248 val_248 1 +249 val_249 1 +252 val_252 1 +255 val_255 2 +256 val_256 2 +257 val_257 1 +258 val_258 1 +260 val_260 1 +262 val_262 1 +263 val_263 1 +265 val_265 2 +266 val_266 1 +272 val_272 2 +273 val_273 3 +274 val_274 1 +275 val_275 1 +277 val_277 4 +278 val_278 2 +280 val_280 2 +281 val_281 2 +282 val_282 2 +283 val_283 1 +284 val_284 1 +285 val_285 1 +286 val_286 1 +287 val_287 1 +288 val_288 2 +289 val_289 1 +291 val_291 1 +292 val_292 1 +296 val_296 1 +298 val_298 3 +302 val_302 1 +305 val_305 1 +306 val_306 1 +307 val_307 2 +308 val_308 1 +309 val_309 2 +310 val_310 1 +311 val_311 3 +315 val_315 1 +316 val_316 3 +317 val_317 2 +318 val_318 3 +321 val_321 2 +322 val_322 2 +323 val_323 1 +325 val_325 2 +327 val_327 3 +331 val_331 2 +332 val_332 1 +333 val_333 2 +335 val_335 1 +336 val_336 1 +338 val_338 1 +339 val_339 1 +341 val_341 1 +342 val_342 2 +344 val_344 2 +345 val_345 1 +348 val_348 5 +351 val_351 1 +353 val_353 2 +356 val_356 1 +360 val_360 1 +362 val_362 1 +364 val_364 1 +365 val_365 1 +366 val_366 1 +367 val_367 2 +368 val_368 1 +369 val_369 3 +373 val_373 1 +374 val_374 1 +375 val_375 1 +377 val_377 1 +378 val_378 1 +379 val_379 1 +382 val_382 2 +384 val_384 3 +386 val_386 1 +389 val_389 1 +392 val_392 1 +393 val_393 1 +394 val_394 1 +395 val_395 2 +396 val_396 3 +397 val_397 2 +399 val_399 2 +400 val_400 1 +401 val_401 5 +402 val_402 1 +403 val_403 3 +404 val_404 2 +406 val_406 4 +407 val_407 1 +409 val_409 3 +411 val_411 1 +413 val_413 2 +414 val_414 2 +417 val_417 3 +418 val_418 1 +419 val_419 1 +421 val_421 1 +424 val_424 2 +427 val_427 1 +429 val_429 2 +430 val_430 3 +431 val_431 3 +432 val_432 1 +435 val_435 1 +436 val_436 1 +437 val_437 1 +438 val_438 3 +439 val_439 2 +443 val_443 1 +444 val_444 1 +446 val_446 1 +448 val_448 1 +449 val_449 1 +452 val_452 1 +453 val_453 1 +454 val_454 3 +455 val_455 1 +457 val_457 1 +458 val_458 2 +459 val_459 2 +460 val_460 1 +462 val_462 2 +463 val_463 2 +466 val_466 3 +467 val_467 1 +468 val_468 4 +469 val_469 5 +470 val_470 1 +472 val_472 1 +475 val_475 1 +477 val_477 1 +478 val_478 2 +479 val_479 1 +480 val_480 3 +481 val_481 1 +482 val_482 1 +483 val_483 1 +484 val_484 1 +485 val_485 1 +487 val_487 1 +489 val_489 4 +490 val_490 1 +491 val_491 1 +492 val_492 2 +493 val_493 1 +494 val_494 1 +495 val_495 1 +496 val_496 1 +497 val_497 1 +498 val_498 3 \ No newline at end of file diff --git a/src/test/hive/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e b/src/test/hive/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f b/src/test/hive/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 b/src/test/hive/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..4ef88d57aa72b --- /dev/null +++ b/src/test/hive/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 @@ -0,0 +1,309 @@ +0 val_0 1 +10 val_10 1 +100 val_100 1 +103 val_103 1 +104 val_104 1 +105 val_105 1 +11 val_11 1 +111 val_111 1 +113 val_113 1 +114 val_114 1 +116 val_116 1 +118 val_118 1 +119 val_119 1 +12 val_12 1 +120 val_120 1 +125 val_125 1 +126 val_126 1 +128 val_128 1 +129 val_129 1 +131 val_131 1 +133 val_133 1 +134 val_134 1 +136 val_136 1 +137 val_137 1 +138 val_138 1 +143 val_143 1 +145 val_145 1 +146 val_146 1 +149 val_149 1 +15 val_15 1 +150 val_150 1 +152 val_152 1 +153 val_153 1 +155 val_155 1 +156 val_156 1 +157 val_157 1 +158 val_158 1 +160 val_160 1 +162 val_162 1 +163 val_163 1 +164 val_164 1 +165 val_165 1 +166 val_166 1 +167 val_167 1 +168 val_168 1 +169 val_169 1 +17 val_17 1 +170 val_170 1 +172 val_172 1 +174 val_174 1 +175 val_175 1 +176 val_176 1 +177 val_177 1 +178 val_178 1 +179 val_179 1 +18 val_18 1 +180 val_180 1 +181 val_181 1 +183 val_183 1 +186 val_186 1 +187 val_187 1 +189 val_189 1 +19 val_19 1 +190 val_190 1 +191 val_191 1 +192 val_192 1 +193 val_193 1 +194 val_194 1 +195 val_195 1 +196 val_196 1 +197 val_197 1 +199 val_199 1 +2 val_2 1 +20 val_20 1 +200 val_200 1 +201 val_201 1 +202 val_202 1 +203 val_203 1 +205 val_205 1 +207 val_207 1 +208 val_208 1 +209 val_209 1 +213 val_213 1 +214 val_214 1 +216 val_216 1 +217 val_217 1 +218 val_218 1 +219 val_219 1 +221 val_221 1 +222 val_222 1 +223 val_223 1 +224 val_224 1 +226 val_226 1 +228 val_228 1 +229 val_229 1 +230 val_230 1 +233 val_233 1 +235 val_235 1 +237 val_237 1 +238 val_238 1 +239 val_239 1 +24 val_24 1 +241 val_241 1 +242 val_242 1 +244 val_244 1 +247 val_247 1 +248 val_248 1 +249 val_249 1 +252 val_252 1 +255 val_255 1 +256 val_256 1 +257 val_257 1 +258 val_258 1 +26 val_26 1 +260 val_260 1 +262 val_262 1 +263 val_263 1 +265 val_265 1 +266 val_266 1 +27 val_27 1 +272 val_272 1 +273 val_273 1 +274 val_274 1 +275 val_275 1 +277 val_277 1 +278 val_278 1 +28 val_28 1 +280 val_280 1 +281 val_281 1 +282 val_282 1 +283 val_283 1 +284 val_284 1 +285 val_285 1 +286 val_286 1 +287 val_287 1 +288 val_288 1 +289 val_289 1 +291 val_291 1 +292 val_292 1 +296 val_296 1 +298 val_298 1 +30 val_30 1 +302 val_302 1 +305 val_305 1 +306 val_306 1 +307 val_307 1 +308 val_308 1 +309 val_309 1 +310 val_310 1 +311 val_311 1 +315 val_315 1 +316 val_316 1 +317 val_317 1 +318 val_318 1 +321 val_321 1 +322 val_322 1 +323 val_323 1 +325 val_325 1 +327 val_327 1 +33 val_33 1 +331 val_331 1 +332 val_332 1 +333 val_333 1 +335 val_335 1 +336 val_336 1 +338 val_338 1 +339 val_339 1 +34 val_34 1 +341 val_341 1 +342 val_342 1 +344 val_344 1 +345 val_345 1 +348 val_348 1 +35 val_35 1 +351 val_351 1 +353 val_353 1 +356 val_356 1 +360 val_360 1 +362 val_362 1 +364 val_364 1 +365 val_365 1 +366 val_366 1 +367 val_367 1 +368 val_368 1 +369 val_369 1 +37 val_37 1 +373 val_373 1 +374 val_374 1 +375 val_375 1 +377 val_377 1 +378 val_378 1 +379 val_379 1 +382 val_382 1 +384 val_384 1 +386 val_386 1 +389 val_389 1 +392 val_392 1 +393 val_393 1 +394 val_394 1 +395 val_395 1 +396 val_396 1 +397 val_397 1 +399 val_399 1 +4 val_4 1 +400 val_400 1 +401 val_401 1 +402 val_402 1 +403 val_403 1 +404 val_404 1 +406 val_406 1 +407 val_407 1 +409 val_409 1 +41 val_41 1 +411 val_411 1 +413 val_413 1 +414 val_414 1 +417 val_417 1 +418 val_418 1 +419 val_419 1 +42 val_42 1 +421 val_421 1 +424 val_424 1 +427 val_427 1 +429 val_429 1 +43 val_43 1 +430 val_430 1 +431 val_431 1 +432 val_432 1 +435 val_435 1 +436 val_436 1 +437 val_437 1 +438 val_438 1 +439 val_439 1 +44 val_44 1 +443 val_443 1 +444 val_444 1 +446 val_446 1 +448 val_448 1 +449 val_449 1 +452 val_452 1 +453 val_453 1 +454 val_454 1 +455 val_455 1 +457 val_457 1 +458 val_458 1 +459 val_459 1 +460 val_460 1 +462 val_462 1 +463 val_463 1 +466 val_466 1 +467 val_467 1 +468 val_468 1 +469 val_469 1 +47 val_47 1 +470 val_470 1 +472 val_472 1 +475 val_475 1 +477 val_477 1 +478 val_478 1 +479 val_479 1 +480 val_480 1 +481 val_481 1 +482 val_482 1 +483 val_483 1 +484 val_484 1 +485 val_485 1 +487 val_487 1 +489 val_489 1 +490 val_490 1 +491 val_491 1 +492 val_492 1 +493 val_493 1 +494 val_494 1 +495 val_495 1 +496 val_496 1 +497 val_497 1 +498 val_498 1 +5 val_5 1 +51 val_51 1 +53 val_53 1 +54 val_54 1 +57 val_57 1 +58 val_58 1 +64 val_64 1 +65 val_65 1 +66 val_66 1 +67 val_67 1 +69 val_69 1 +70 val_70 1 +72 val_72 1 +74 val_74 1 +76 val_76 1 +77 val_77 1 +78 val_78 1 +8 val_8 1 +80 val_80 1 +82 val_82 1 +83 val_83 1 +84 val_84 1 +85 val_85 1 +86 val_86 1 +87 val_87 1 +9 val_9 1 +90 val_90 1 +92 val_92 1 +95 val_95 1 +96 val_96 1 +97 val_97 1 +98 val_98 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 b/src/test/hive/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..0e40f3f1a49a9 --- /dev/null +++ b/src/test/hive/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 @@ -0,0 +1,309 @@ +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 +20 val_20 1 +24 val_24 1 +26 val_26 1 +27 val_27 1 +28 val_28 1 +30 val_30 1 +33 val_33 1 +34 val_34 1 +35 val_35 1 +37 val_37 1 +41 val_41 1 +42 val_42 1 +43 val_43 1 +44 val_44 1 +47 val_47 1 +51 val_51 1 +53 val_53 1 +54 val_54 1 +57 val_57 1 +58 val_58 1 +64 val_64 1 +65 val_65 1 +66 val_66 1 +67 val_67 1 +69 val_69 1 +70 val_70 1 +72 val_72 1 +74 val_74 1 +76 val_76 1 +77 val_77 1 +78 val_78 1 +80 val_80 1 +82 val_82 1 +83 val_83 1 +84 val_84 1 +85 val_85 1 +86 val_86 1 +87 val_87 1 +90 val_90 1 +92 val_92 1 +95 val_95 1 +96 val_96 1 +97 val_97 1 +98 val_98 1 +100 val_100 1 +103 val_103 1 +104 val_104 1 +105 val_105 1 +111 val_111 1 +113 val_113 1 +114 val_114 1 +116 val_116 1 +118 val_118 1 +119 val_119 1 +120 val_120 1 +125 val_125 1 +126 val_126 1 +128 val_128 1 +129 val_129 1 +131 val_131 1 +133 val_133 1 +134 val_134 1 +136 val_136 1 +137 val_137 1 +138 val_138 1 +143 val_143 1 +145 val_145 1 +146 val_146 1 +149 val_149 1 +150 val_150 1 +152 val_152 1 +153 val_153 1 +155 val_155 1 +156 val_156 1 +157 val_157 1 +158 val_158 1 +160 val_160 1 +162 val_162 1 +163 val_163 1 +164 val_164 1 +165 val_165 1 +166 val_166 1 +167 val_167 1 +168 val_168 1 +169 val_169 1 +170 val_170 1 +172 val_172 1 +174 val_174 1 +175 val_175 1 +176 val_176 1 +177 val_177 1 +178 val_178 1 +179 val_179 1 +180 val_180 1 +181 val_181 1 +183 val_183 1 +186 val_186 1 +187 val_187 1 +189 val_189 1 +190 val_190 1 +191 val_191 1 +192 val_192 1 +193 val_193 1 +194 val_194 1 +195 val_195 1 +196 val_196 1 +197 val_197 1 +199 val_199 1 +200 val_200 1 +201 val_201 1 +202 val_202 1 +203 val_203 1 +205 val_205 1 +207 val_207 1 +208 val_208 1 +209 val_209 1 +213 val_213 1 +214 val_214 1 +216 val_216 1 +217 val_217 1 +218 val_218 1 +219 val_219 1 +221 val_221 1 +222 val_222 1 +223 val_223 1 +224 val_224 1 +226 val_226 1 +228 val_228 1 +229 val_229 1 +230 val_230 1 +233 val_233 1 +235 val_235 1 +237 val_237 1 +238 val_238 1 +239 val_239 1 +241 val_241 1 +242 val_242 1 +244 val_244 1 +247 val_247 1 +248 val_248 1 +249 val_249 1 +252 val_252 1 +255 val_255 1 +256 val_256 1 +257 val_257 1 +258 val_258 1 +260 val_260 1 +262 val_262 1 +263 val_263 1 +265 val_265 1 +266 val_266 1 +272 val_272 1 +273 val_273 1 +274 val_274 1 +275 val_275 1 +277 val_277 1 +278 val_278 1 +280 val_280 1 +281 val_281 1 +282 val_282 1 +283 val_283 1 +284 val_284 1 +285 val_285 1 +286 val_286 1 +287 val_287 1 +288 val_288 1 +289 val_289 1 +291 val_291 1 +292 val_292 1 +296 val_296 1 +298 val_298 1 +302 val_302 1 +305 val_305 1 +306 val_306 1 +307 val_307 1 +308 val_308 1 +309 val_309 1 +310 val_310 1 +311 val_311 1 +315 val_315 1 +316 val_316 1 +317 val_317 1 +318 val_318 1 +321 val_321 1 +322 val_322 1 +323 val_323 1 +325 val_325 1 +327 val_327 1 +331 val_331 1 +332 val_332 1 +333 val_333 1 +335 val_335 1 +336 val_336 1 +338 val_338 1 +339 val_339 1 +341 val_341 1 +342 val_342 1 +344 val_344 1 +345 val_345 1 +348 val_348 1 +351 val_351 1 +353 val_353 1 +356 val_356 1 +360 val_360 1 +362 val_362 1 +364 val_364 1 +365 val_365 1 +366 val_366 1 +367 val_367 1 +368 val_368 1 +369 val_369 1 +373 val_373 1 +374 val_374 1 +375 val_375 1 +377 val_377 1 +378 val_378 1 +379 val_379 1 +382 val_382 1 +384 val_384 1 +386 val_386 1 +389 val_389 1 +392 val_392 1 +393 val_393 1 +394 val_394 1 +395 val_395 1 +396 val_396 1 +397 val_397 1 +399 val_399 1 +400 val_400 1 +401 val_401 1 +402 val_402 1 +403 val_403 1 +404 val_404 1 +406 val_406 1 +407 val_407 1 +409 val_409 1 +411 val_411 1 +413 val_413 1 +414 val_414 1 +417 val_417 1 +418 val_418 1 +419 val_419 1 +421 val_421 1 +424 val_424 1 +427 val_427 1 +429 val_429 1 +430 val_430 1 +431 val_431 1 +432 val_432 1 +435 val_435 1 +436 val_436 1 +437 val_437 1 +438 val_438 1 +439 val_439 1 +443 val_443 1 +444 val_444 1 +446 val_446 1 +448 val_448 1 +449 val_449 1 +452 val_452 1 +453 val_453 1 +454 val_454 1 +455 val_455 1 +457 val_457 1 +458 val_458 1 +459 val_459 1 +460 val_460 1 +462 val_462 1 +463 val_463 1 +466 val_466 1 +467 val_467 1 +468 val_468 1 +469 val_469 1 +470 val_470 1 +472 val_472 1 +475 val_475 1 +477 val_477 1 +478 val_478 1 +479 val_479 1 +480 val_480 1 +481 val_481 1 +482 val_482 1 +483 val_483 1 +484 val_484 1 +485 val_485 1 +487 val_487 1 +489 val_489 1 +490 val_490 1 +491 val_491 1 +492 val_492 1 +493 val_493 1 +494 val_494 1 +495 val_495 1 +496 val_496 1 +497 val_497 1 +498 val_498 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e b/src/test/hive/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 b/src/test/hive/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/src/test/hive/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..4ef88d57aa72b --- /dev/null +++ b/src/test/hive/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 @@ -0,0 +1,309 @@ +0 val_0 1 +10 val_10 1 +100 val_100 1 +103 val_103 1 +104 val_104 1 +105 val_105 1 +11 val_11 1 +111 val_111 1 +113 val_113 1 +114 val_114 1 +116 val_116 1 +118 val_118 1 +119 val_119 1 +12 val_12 1 +120 val_120 1 +125 val_125 1 +126 val_126 1 +128 val_128 1 +129 val_129 1 +131 val_131 1 +133 val_133 1 +134 val_134 1 +136 val_136 1 +137 val_137 1 +138 val_138 1 +143 val_143 1 +145 val_145 1 +146 val_146 1 +149 val_149 1 +15 val_15 1 +150 val_150 1 +152 val_152 1 +153 val_153 1 +155 val_155 1 +156 val_156 1 +157 val_157 1 +158 val_158 1 +160 val_160 1 +162 val_162 1 +163 val_163 1 +164 val_164 1 +165 val_165 1 +166 val_166 1 +167 val_167 1 +168 val_168 1 +169 val_169 1 +17 val_17 1 +170 val_170 1 +172 val_172 1 +174 val_174 1 +175 val_175 1 +176 val_176 1 +177 val_177 1 +178 val_178 1 +179 val_179 1 +18 val_18 1 +180 val_180 1 +181 val_181 1 +183 val_183 1 +186 val_186 1 +187 val_187 1 +189 val_189 1 +19 val_19 1 +190 val_190 1 +191 val_191 1 +192 val_192 1 +193 val_193 1 +194 val_194 1 +195 val_195 1 +196 val_196 1 +197 val_197 1 +199 val_199 1 +2 val_2 1 +20 val_20 1 +200 val_200 1 +201 val_201 1 +202 val_202 1 +203 val_203 1 +205 val_205 1 +207 val_207 1 +208 val_208 1 +209 val_209 1 +213 val_213 1 +214 val_214 1 +216 val_216 1 +217 val_217 1 +218 val_218 1 +219 val_219 1 +221 val_221 1 +222 val_222 1 +223 val_223 1 +224 val_224 1 +226 val_226 1 +228 val_228 1 +229 val_229 1 +230 val_230 1 +233 val_233 1 +235 val_235 1 +237 val_237 1 +238 val_238 1 +239 val_239 1 +24 val_24 1 +241 val_241 1 +242 val_242 1 +244 val_244 1 +247 val_247 1 +248 val_248 1 +249 val_249 1 +252 val_252 1 +255 val_255 1 +256 val_256 1 +257 val_257 1 +258 val_258 1 +26 val_26 1 +260 val_260 1 +262 val_262 1 +263 val_263 1 +265 val_265 1 +266 val_266 1 +27 val_27 1 +272 val_272 1 +273 val_273 1 +274 val_274 1 +275 val_275 1 +277 val_277 1 +278 val_278 1 +28 val_28 1 +280 val_280 1 +281 val_281 1 +282 val_282 1 +283 val_283 1 +284 val_284 1 +285 val_285 1 +286 val_286 1 +287 val_287 1 +288 val_288 1 +289 val_289 1 +291 val_291 1 +292 val_292 1 +296 val_296 1 +298 val_298 1 +30 val_30 1 +302 val_302 1 +305 val_305 1 +306 val_306 1 +307 val_307 1 +308 val_308 1 +309 val_309 1 +310 val_310 1 +311 val_311 1 +315 val_315 1 +316 val_316 1 +317 val_317 1 +318 val_318 1 +321 val_321 1 +322 val_322 1 +323 val_323 1 +325 val_325 1 +327 val_327 1 +33 val_33 1 +331 val_331 1 +332 val_332 1 +333 val_333 1 +335 val_335 1 +336 val_336 1 +338 val_338 1 +339 val_339 1 +34 val_34 1 +341 val_341 1 +342 val_342 1 +344 val_344 1 +345 val_345 1 +348 val_348 1 +35 val_35 1 +351 val_351 1 +353 val_353 1 +356 val_356 1 +360 val_360 1 +362 val_362 1 +364 val_364 1 +365 val_365 1 +366 val_366 1 +367 val_367 1 +368 val_368 1 +369 val_369 1 +37 val_37 1 +373 val_373 1 +374 val_374 1 +375 val_375 1 +377 val_377 1 +378 val_378 1 +379 val_379 1 +382 val_382 1 +384 val_384 1 +386 val_386 1 +389 val_389 1 +392 val_392 1 +393 val_393 1 +394 val_394 1 +395 val_395 1 +396 val_396 1 +397 val_397 1 +399 val_399 1 +4 val_4 1 +400 val_400 1 +401 val_401 1 +402 val_402 1 +403 val_403 1 +404 val_404 1 +406 val_406 1 +407 val_407 1 +409 val_409 1 +41 val_41 1 +411 val_411 1 +413 val_413 1 +414 val_414 1 +417 val_417 1 +418 val_418 1 +419 val_419 1 +42 val_42 1 +421 val_421 1 +424 val_424 1 +427 val_427 1 +429 val_429 1 +43 val_43 1 +430 val_430 1 +431 val_431 1 +432 val_432 1 +435 val_435 1 +436 val_436 1 +437 val_437 1 +438 val_438 1 +439 val_439 1 +44 val_44 1 +443 val_443 1 +444 val_444 1 +446 val_446 1 +448 val_448 1 +449 val_449 1 +452 val_452 1 +453 val_453 1 +454 val_454 1 +455 val_455 1 +457 val_457 1 +458 val_458 1 +459 val_459 1 +460 val_460 1 +462 val_462 1 +463 val_463 1 +466 val_466 1 +467 val_467 1 +468 val_468 1 +469 val_469 1 +47 val_47 1 +470 val_470 1 +472 val_472 1 +475 val_475 1 +477 val_477 1 +478 val_478 1 +479 val_479 1 +480 val_480 1 +481 val_481 1 +482 val_482 1 +483 val_483 1 +484 val_484 1 +485 val_485 1 +487 val_487 1 +489 val_489 1 +490 val_490 1 +491 val_491 1 +492 val_492 1 +493 val_493 1 +494 val_494 1 +495 val_495 1 +496 val_496 1 +497 val_497 1 +498 val_498 1 +5 val_5 1 +51 val_51 1 +53 val_53 1 +54 val_54 1 +57 val_57 1 +58 val_58 1 +64 val_64 1 +65 val_65 1 +66 val_66 1 +67 val_67 1 +69 val_69 1 +70 val_70 1 +72 val_72 1 +74 val_74 1 +76 val_76 1 +77 val_77 1 +78 val_78 1 +8 val_8 1 +80 val_80 1 +82 val_82 1 +83 val_83 1 +84 val_84 1 +85 val_85 1 +86 val_86 1 +87 val_87 1 +9 val_9 1 +90 val_90 1 +92 val_92 1 +95 val_95 1 +96 val_96 1 +97 val_97 1 +98 val_98 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/src/test/hive/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 b/src/test/hive/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/src/test/hive/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 b/src/test/hive/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 b/src/test/hive/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a b/src/test/hive/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 b/src/test/hive/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/src/test/hive/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 b/src/test/hive/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 b/src/test/hive/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 b/src/test/hive/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 b/src/test/hive/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 b/src/test/hive/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 b/src/test/hive/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 b/src/test/hive/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 b/src/test/hive/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad b/src/test/hive/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad new file mode 100644 index 0000000000000..917d19a8f8cd8 --- /dev/null +++ b/src/test/hive/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad @@ -0,0 +1,4 @@ +0 0 +2 2 +4 4 +5 5 diff --git a/src/test/hive/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 b/src/test/hive/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 new file mode 100644 index 0000000000000..917d19a8f8cd8 --- /dev/null +++ b/src/test/hive/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 @@ -0,0 +1,4 @@ +0 0 +2 2 +4 4 +5 5 diff --git a/src/test/hive/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 b/src/test/hive/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 b/src/test/hive/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 b/src/test/hive/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a b/src/test/hive/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..ae9bbc3e2c2f7 --- /dev/null +++ b/src/test/hive/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 b/src/test/hive/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 b/src/test/hive/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 b/src/test/hive/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..64bb7c62c1885 --- /dev/null +++ b/src/test/hive/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b b/src/test/hive/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 b/src/test/hive/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 new file mode 100644 index 0000000000000..6b6a788e382db --- /dev/null +++ b/src/test/hive/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 @@ -0,0 +1,6 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 b/src/test/hive/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 new file mode 100644 index 0000000000000..c33eed60c8d5b --- /dev/null +++ b/src/test/hive/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 @@ -0,0 +1,6 @@ +0 1 +4 1 +8 1 +10 1 +16 1 +18 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 b/src/test/hive/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 b/src/test/hive/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e b/src/test/hive/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 b/src/test/hive/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 new file mode 100644 index 0000000000000..6b6a788e382db --- /dev/null +++ b/src/test/hive/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 @@ -0,0 +1,6 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 b/src/test/hive/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 new file mode 100644 index 0000000000000..c33eed60c8d5b --- /dev/null +++ b/src/test/hive/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 @@ -0,0 +1,6 @@ +0 1 +4 1 +8 1 +10 1 +16 1 +18 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 b/src/test/hive/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 b/src/test/hive/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e b/src/test/hive/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/src/test/hive/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 b/src/test/hive/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd b/src/test/hive/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd new file mode 100644 index 0000000000000..922f4bfc83e44 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b b/src/test/hive/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 b/src/test/hive/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e b/src/test/hive/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e new file mode 100644 index 0000000000000..106132fc993d3 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e @@ -0,0 +1,5 @@ +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/src/test/hive/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 b/src/test/hive/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 new file mode 100644 index 0000000000000..65235356ea425 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 @@ -0,0 +1,5 @@ +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/src/test/hive/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd b/src/test/hive/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd new file mode 100644 index 0000000000000..922f4bfc83e44 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 b/src/test/hive/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 new file mode 100644 index 0000000000000..017878bc9bee5 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae b/src/test/hive/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae new file mode 100644 index 0000000000000..f21a658e3c68f --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae @@ -0,0 +1,32 @@ +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/src/test/hive/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab b/src/test/hive/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 b/src/test/hive/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 b/src/test/hive/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc b/src/test/hive/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 b/src/test/hive/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f b/src/test/hive/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 b/src/test/hive/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 b/src/test/hive/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea b/src/test/hive/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b b/src/test/hive/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 b/src/test/hive/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e b/src/test/hive/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e new file mode 100644 index 0000000000000..106132fc993d3 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e @@ -0,0 +1,5 @@ +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/src/test/hive/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 b/src/test/hive/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 new file mode 100644 index 0000000000000..65235356ea425 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 @@ -0,0 +1,5 @@ +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/src/test/hive/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/src/test/hive/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada b/src/test/hive/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 b/src/test/hive/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e b/src/test/hive/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 b/src/test/hive/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 b/src/test/hive/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 new file mode 100644 index 0000000000000..dbc7aebdbbae5 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 @@ -0,0 +1,5 @@ +5 6 +6 5 +7 6 +8 8 +9 7 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 b/src/test/hive/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 new file mode 100644 index 0000000000000..ebf9d6978dec6 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 @@ -0,0 +1,5 @@ +0 1 3 +1 71 115 +2 69 111 +3 62 99 +4 74 124 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab b/src/test/hive/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 b/src/test/hive/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd b/src/test/hive/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d b/src/test/hive/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 b/src/test/hive/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 b/src/test/hive/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 b/src/test/hive/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 b/src/test/hive/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..a5ae9e2a62227 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1,2 @@ +100 2 +200 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 b/src/test/hive/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..16c7a647a2344 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1 @@ +400 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a b/src/test/hive/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b b/src/test/hive/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 b/src/test/hive/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..67f077e51a647 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1,2 @@ +val_100 2 +val_200 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 b/src/test/hive/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..b06ad20135fbe --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1 @@ +val_200 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 b/src/test/hive/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 b/src/test/hive/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 b/src/test/hive/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..a5ae9e2a62227 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1,2 @@ +100 2 +200 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 b/src/test/hive/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..16c7a647a2344 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1 @@ +400 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a b/src/test/hive/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b b/src/test/hive/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 b/src/test/hive/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..67f077e51a647 --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1,2 @@ +val_100 2 +val_200 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 b/src/test/hive/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..b06ad20135fbe --- /dev/null +++ b/src/test/hive/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1 @@ +val_200 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb b/src/test/hive/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb new file mode 100644 index 0000000000000..efe5cc7795c65 --- /dev/null +++ b/src/test/hive/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb @@ -0,0 +1 @@ +-30.33 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 b/src/test/hive/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 new file mode 100644 index 0000000000000..efe5cc7795c65 --- /dev/null +++ b/src/test/hive/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 @@ -0,0 +1 @@ +-30.33 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 b/src/test/hive/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 b/src/test/hive/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 b/src/test/hive/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 new file mode 100644 index 0000000000000..f2a91fe3bfab0 --- /dev/null +++ b/src/test/hive/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 @@ -0,0 +1,13 @@ +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/src/test/hive/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 b/src/test/hive/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f b/src/test/hive/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f new file mode 100644 index 0000000000000..c8e666cb01e8e --- /dev/null +++ b/src/test/hive/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f @@ -0,0 +1,14 @@ +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/src/test/hive/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 b/src/test/hive/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb b/src/test/hive/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb new file mode 100644 index 0000000000000..96a824a81c589 --- /dev/null +++ b/src/test/hive/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb @@ -0,0 +1,3 @@ +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/src/test/hive/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 b/src/test/hive/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac b/src/test/hive/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c b/src/test/hive/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d b/src/test/hive/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d new file mode 100644 index 0000000000000..a79396dac079a --- /dev/null +++ b/src/test/hive/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d @@ -0,0 +1,13 @@ +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/src/test/hive/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 b/src/test/hive/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 new file mode 100644 index 0000000000000..f2a91fe3bfab0 --- /dev/null +++ b/src/test/hive/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 @@ -0,0 +1,13 @@ +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/src/test/hive/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 b/src/test/hive/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c b/src/test/hive/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d b/src/test/hive/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d new file mode 100644 index 0000000000000..a79396dac079a --- /dev/null +++ b/src/test/hive/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d @@ -0,0 +1,13 @@ +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/src/test/hive/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 b/src/test/hive/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 b/src/test/hive/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a b/src/test/hive/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..ae9bbc3e2c2f7 --- /dev/null +++ b/src/test/hive/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 b/src/test/hive/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d b/src/test/hive/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d b/src/test/hive/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..64bb7c62c1885 --- /dev/null +++ b/src/test/hive/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 b/src/test/hive/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c b/src/test/hive/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 b/src/test/hive/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 b/src/test/hive/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea b/src/test/hive/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 b/src/test/hive/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 b/src/test/hive/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 b/src/test/hive/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 b/src/test/hive/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 b/src/test/hive/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 b/src/test/hive/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 b/src/test/hive/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 new file mode 100644 index 0000000000000..c1cc4ee204773 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 @@ -0,0 +1,5 @@ +1 1 1 +1 2 1 +1 3 1 +1 7 1 +1 8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 b/src/test/hive/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 b/src/test/hive/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f b/src/test/hive/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca b/src/test/hive/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 b/src/test/hive/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 b/src/test/hive/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 b/src/test/hive/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 new file mode 100644 index 0000000000000..f0192040e147b --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 @@ -0,0 +1,5 @@ +1 2 1 +2 3 1 +3 4 1 +7 8 1 +8 9 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f b/src/test/hive/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef b/src/test/hive/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..b6c2eb98e5e49 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +2 1 +4 1 +6 1 +14 1 +16 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 b/src/test/hive/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 b/src/test/hive/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..8e7ee8a2b47bb --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d b/src/test/hive/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b b/src/test/hive/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad b/src/test/hive/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..0b2b54cd94e4f --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 b/src/test/hive/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c b/src/test/hive/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..3d4708b7c9d64 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 2 +2 2 +3 2 +7 2 +8 4 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a b/src/test/hive/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 b/src/test/hive/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 b/src/test/hive/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa b/src/test/hive/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 b/src/test/hive/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 b/src/test/hive/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 b/src/test/hive/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca b/src/test/hive/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 b/src/test/hive/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c b/src/test/hive/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 b/src/test/hive/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 b/src/test/hive/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 new file mode 100644 index 0000000000000..7ca6b0b28a960 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 b/src/test/hive/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 b/src/test/hive/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca b/src/test/hive/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b b/src/test/hive/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 b/src/test/hive/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 b/src/test/hive/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca b/src/test/hive/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..58e16ef3c0ef3 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e b/src/test/hive/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 b/src/test/hive/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 b/src/test/hive/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/src/test/hive/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 b/src/test/hive/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 b/src/test/hive/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc b/src/test/hive/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e b/src/test/hive/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 b/src/test/hive/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd b/src/test/hive/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 b/src/test/hive/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 b/src/test/hive/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 b/src/test/hive/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 new file mode 100644 index 0000000000000..d15db8c5d079f --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 @@ -0,0 +1 @@ +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 b/src/test/hive/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 new file mode 100644 index 0000000000000..3d1609d961673 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 @@ -0,0 +1,2 @@ +8 18 1 +8 28 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 b/src/test/hive/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 b/src/test/hive/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 b/src/test/hive/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 new file mode 100644 index 0000000000000..c6bb9dbfd6497 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 @@ -0,0 +1,2 @@ +0 +11 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 b/src/test/hive/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e b/src/test/hive/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 b/src/test/hive/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 b/src/test/hive/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 b/src/test/hive/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 new file mode 100644 index 0000000000000..c6bb9dbfd6497 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 @@ -0,0 +1,2 @@ +0 +11 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 b/src/test/hive/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c b/src/test/hive/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d b/src/test/hive/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 b/src/test/hive/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 new file mode 100644 index 0000000000000..ded2854cdf564 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 b/src/test/hive/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 b/src/test/hive/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 new file mode 100644 index 0000000000000..487b4c4a5cc6f --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 b/src/test/hive/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 b/src/test/hive/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 new file mode 100644 index 0000000000000..1e8b314962144 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 @@ -0,0 +1 @@ +6 diff --git a/src/test/hive/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 b/src/test/hive/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 b/src/test/hive/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 b/src/test/hive/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d b/src/test/hive/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d new file mode 100644 index 0000000000000..1e8b314962144 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d @@ -0,0 +1 @@ +6 diff --git a/src/test/hive/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e b/src/test/hive/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 b/src/test/hive/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 b/src/test/hive/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d b/src/test/hive/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d new file mode 100644 index 0000000000000..1e8b314962144 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d @@ -0,0 +1 @@ +6 diff --git a/src/test/hive/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c b/src/test/hive/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 b/src/test/hive/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 new file mode 100644 index 0000000000000..6a5fe2835fc56 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 @@ -0,0 +1 @@ +6 10 10 28.0 diff --git a/src/test/hive/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 b/src/test/hive/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 new file mode 100644 index 0000000000000..c5b99ed941efc --- /dev/null +++ b/src/test/hive/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 @@ -0,0 +1,6 @@ +11 1 +12 1 +13 1 +17 1 +18 1 +28 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 b/src/test/hive/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b b/src/test/hive/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 b/src/test/hive/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 b/src/test/hive/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed b/src/test/hive/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 b/src/test/hive/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 b/src/test/hive/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e b/src/test/hive/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd b/src/test/hive/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f b/src/test/hive/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 b/src/test/hive/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b b/src/test/hive/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 b/src/test/hive/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 b/src/test/hive/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd b/src/test/hive/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 b/src/test/hive/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f b/src/test/hive/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 b/src/test/hive/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 b/src/test/hive/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f b/src/test/hive/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b b/src/test/hive/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e b/src/test/hive/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef b/src/test/hive/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 b/src/test/hive/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 b/src/test/hive/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 b/src/test/hive/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f b/src/test/hive/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b b/src/test/hive/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 b/src/test/hive/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd b/src/test/hive/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 b/src/test/hive/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 b/src/test/hive/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf b/src/test/hive/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b b/src/test/hive/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d b/src/test/hive/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f b/src/test/hive/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd b/src/test/hive/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f b/src/test/hive/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 b/src/test/hive/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 b/src/test/hive/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b b/src/test/hive/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 b/src/test/hive/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 b/src/test/hive/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd b/src/test/hive/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 b/src/test/hive/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 b/src/test/hive/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 b/src/test/hive/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 b/src/test/hive/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e b/src/test/hive/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 b/src/test/hive/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 b/src/test/hive/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 b/src/test/hive/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 b/src/test/hive/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 b/src/test/hive/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 b/src/test/hive/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 b/src/test/hive/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb b/src/test/hive/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 b/src/test/hive/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 b/src/test/hive/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b b/src/test/hive/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 b/src/test/hive/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 b/src/test/hive/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d b/src/test/hive/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 b/src/test/hive/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f b/src/test/hive/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb b/src/test/hive/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 b/src/test/hive/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d b/src/test/hive/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d b/src/test/hive/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a b/src/test/hive/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 b/src/test/hive/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f b/src/test/hive/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb b/src/test/hive/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 b/src/test/hive/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c b/src/test/hive/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 b/src/test/hive/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c b/src/test/hive/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c new file mode 100644 index 0000000000000..3d4708b7c9d64 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c @@ -0,0 +1,5 @@ +1 2 +2 2 +3 2 +7 2 +8 4 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 b/src/test/hive/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 b/src/test/hive/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c b/src/test/hive/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 b/src/test/hive/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 b/src/test/hive/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea b/src/test/hive/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 b/src/test/hive/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 b/src/test/hive/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 b/src/test/hive/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 b/src/test/hive/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 b/src/test/hive/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 b/src/test/hive/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 b/src/test/hive/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 new file mode 100644 index 0000000000000..c1cc4ee204773 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 @@ -0,0 +1,5 @@ +1 1 1 +1 2 1 +1 3 1 +1 7 1 +1 8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 b/src/test/hive/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 b/src/test/hive/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f b/src/test/hive/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca b/src/test/hive/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 b/src/test/hive/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 b/src/test/hive/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 b/src/test/hive/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 new file mode 100644 index 0000000000000..f0192040e147b --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 @@ -0,0 +1,5 @@ +1 2 1 +2 3 1 +3 4 1 +7 8 1 +8 9 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f b/src/test/hive/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef b/src/test/hive/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..b6c2eb98e5e49 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +2 1 +4 1 +6 1 +14 1 +16 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 b/src/test/hive/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 b/src/test/hive/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..8e7ee8a2b47bb --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d b/src/test/hive/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b b/src/test/hive/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..0b2b54cd94e4f --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,10 @@ +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/src/test/hive/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 b/src/test/hive/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c b/src/test/hive/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..3d4708b7c9d64 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 2 +2 2 +3 2 +7 2 +8 4 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a b/src/test/hive/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 b/src/test/hive/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 b/src/test/hive/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa b/src/test/hive/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 b/src/test/hive/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad b/src/test/hive/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd b/src/test/hive/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 b/src/test/hive/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 b/src/test/hive/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca b/src/test/hive/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 b/src/test/hive/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c b/src/test/hive/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 b/src/test/hive/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 b/src/test/hive/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 new file mode 100644 index 0000000000000..7ca6b0b28a960 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 b/src/test/hive/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 b/src/test/hive/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca b/src/test/hive/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 b/src/test/hive/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 b/src/test/hive/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca b/src/test/hive/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..58e16ef3c0ef3 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e b/src/test/hive/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 b/src/test/hive/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 b/src/test/hive/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 b/src/test/hive/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/src/test/hive/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b b/src/test/hive/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 b/src/test/hive/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc b/src/test/hive/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e b/src/test/hive/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 b/src/test/hive/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 @@ -0,0 +1,5 @@ +1 1 +2 1 +3 1 +7 1 +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd b/src/test/hive/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd @@ -0,0 +1,6 @@ +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/src/test/hive/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 b/src/test/hive/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 b/src/test/hive/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 b/src/test/hive/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 new file mode 100644 index 0000000000000..d15db8c5d079f --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 @@ -0,0 +1 @@ +8 2 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 b/src/test/hive/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 new file mode 100644 index 0000000000000..3d1609d961673 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 @@ -0,0 +1,2 @@ +8 18 1 +8 28 1 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 b/src/test/hive/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 b/src/test/hive/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e b/src/test/hive/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 b/src/test/hive/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a b/src/test/hive/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad b/src/test/hive/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b b/src/test/hive/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 b/src/test/hive/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 b/src/test/hive/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/hash-0-a658b129316d666d4b01c1581eed1c1f b/src/test/hive/golden/hash-0-a658b129316d666d4b01c1581eed1c1f new file mode 100644 index 0000000000000..a99cee758fe57 --- /dev/null +++ b/src/test/hive/golden/hash-0-a658b129316d666d4b01c1581eed1c1f @@ -0,0 +1 @@ +3556498 \ No newline at end of file diff --git a/src/test/hive/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf b/src/test/hive/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 b/src/test/hive/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 b/src/test/hive/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a b/src/test/hive/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 b/src/test/hive/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 b/src/test/hive/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 b/src/test/hive/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 b/src/test/hive/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 new file mode 100644 index 0000000000000..d7c6f236687d8 --- /dev/null +++ b/src/test/hive/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 @@ -0,0 +1,5 @@ +key int None +_bucketname string +_offsets array + +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, 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/src/test/hive/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 b/src/test/hive/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 b/src/test/hive/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 new file mode 100644 index 0000000000000..4c6ec0ba34bb8 --- /dev/null +++ b/src/test/hive/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 @@ -0,0 +1,5 @@ +key int None +_bucketname string +_offsets array + +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, 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/src/test/hive/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 b/src/test/hive/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 b/src/test/hive/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 new file mode 100644 index 0000000000000..0c6af94247b85 --- /dev/null +++ b/src/test/hive/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 @@ -0,0 +1,5 @@ +key int None +_bucketname string +_offsets array + +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, 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/src/test/hive/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 b/src/test/hive/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d b/src/test/hive/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d new file mode 100644 index 0000000000000..e4a5816e6f1cc --- /dev/null +++ b/src/test/hive/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d @@ -0,0 +1,5 @@ +key int None +_bucketname string +_offsets array + +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, 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/src/test/hive/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad b/src/test/hive/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 b/src/test/hive/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 new file mode 100644 index 0000000000000..3d8751ca47049 --- /dev/null +++ b/src/test/hive/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 @@ -0,0 +1,5 @@ +key int from deserializer +_bucketname string from deserializer +_offsets array 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, 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/src/test/hive/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f b/src/test/hive/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 b/src/test/hive/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 b/src/test/hive/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 new file mode 100644 index 0000000000000..0e4852b319052 --- /dev/null +++ b/src/test/hive/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 @@ -0,0 +1,5 @@ +key int from deserializer +_bucketname string from deserializer +_offsets array 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, 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/src/test/hive/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e b/src/test/hive/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 b/src/test/hive/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 new file mode 100644 index 0000000000000..41a5492fc5331 --- /dev/null +++ b/src/test/hive/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 @@ -0,0 +1,5 @@ +key int None +_bucketname string +_offsets array + +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, 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/src/test/hive/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 b/src/test/hive/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf b/src/test/hive/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf new file mode 100644 index 0000000000000..8212bf8b7d2fb --- /dev/null +++ b/src/test/hive/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf @@ -0,0 +1,5 @@ +key int None +_bucketname string +_offsets array + +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, 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/src/test/hive/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca b/src/test/hive/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 b/src/test/hive/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 b/src/test/hive/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe b/src/test/hive/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f b/src/test/hive/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 b/src/test/hive/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 b/src/test/hive/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 b/src/test/hive/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 b/src/test/hive/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 b/src/test/hive/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/src/test/hive/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/src/test/hive/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 b/src/test/hive/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-37-9334418431eca405f13206bd8db42a1b b/src/test/hive/golden/index_creation-37-9334418431eca405f13206bd8db42a1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 b/src/test/hive/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d b/src/test/hive/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d new file mode 100644 index 0000000000000..e8310385c56dc --- /dev/null +++ b/src/test/hive/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d @@ -0,0 +1 @@ +src \ No newline at end of file diff --git a/src/test/hive/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/src/test/hive/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/src/test/hive/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 b/src/test/hive/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-7-9334418431eca405f13206bd8db42a1b b/src/test/hive/golden/index_creation-7-9334418431eca405f13206bd8db42a1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 b/src/test/hive/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c b/src/test/hive/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 b/src/test/hive/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 b/src/test/hive/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 new file mode 100644 index 0000000000000..48522980f81a8 --- /dev/null +++ b/src/test/hive/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 @@ -0,0 +1,3 @@ +number int from deserializer +first_name string from deserializer +last_name string from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b b/src/test/hive/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b new file mode 100644 index 0000000000000..4ed570f9070eb --- /dev/null +++ b/src/test/hive/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b @@ -0,0 +1,5 @@ +7 Sylvester McCoy +8 Paul McGann +9 Christopher Eccleston +10 David Tennant +11 Matt Smith \ No newline at end of file diff --git a/src/test/hive/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 b/src/test/hive/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 b/src/test/hive/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c b/src/test/hive/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 b/src/test/hive/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_serde-4-d7547751c37375a9238043bbe250e716 b/src/test/hive/golden/index_serde-4-d7547751c37375a9238043bbe250e716 new file mode 100644 index 0000000000000..d2ca633d0ae01 --- /dev/null +++ b/src/test/hive/golden/index_serde-4-d7547751c37375a9238043bbe250e716 @@ -0,0 +1,5 @@ +number int from deserializer +_bucketname string +_offsets array + +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, 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/src/test/hive/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 b/src/test/hive/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae b/src/test/hive/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 b/src/test/hive/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 b/src/test/hive/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 b/src/test/hive/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e b/src/test/hive/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 b/src/test/hive/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd b/src/test/hive/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/innerjoin-0-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/innerjoin-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/innerjoin-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e b/src/test/hive/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 b/src/test/hive/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 b/src/test/hive/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 b/src/test/hive/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 new file mode 100644 index 0000000000000..3b7cf42f96358 --- /dev/null +++ b/src/test/hive/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 @@ -0,0 +1,1028 @@ +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 +4 val_4 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +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 +12 val_12 +12 val_12 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +17 val_17 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +19 val_19 +20 val_20 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +26 val_26 +26 val_26 +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 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +41 val_41 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +74 val_74 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +96 val_96 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +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 diff --git a/src/test/hive/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a b/src/test/hive/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce b/src/test/hive/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 b/src/test/hive/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 b/src/test/hive/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b b/src/test/hive/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 b/src/test/hive/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 b/src/test/hive/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 new file mode 100644 index 0000000000000..010e999c36749 --- /dev/null +++ b/src/test/hive/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 @@ -0,0 +1,3 @@ +a int None + +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 diff --git a/src/test/hive/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 b/src/test/hive/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input-1-6558e385bb08991302a72076dd7b7ff5 b/src/test/hive/golden/input-1-6558e385bb08991302a72076dd7b7ff5 new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/src/test/hive/golden/input-1-6558e385bb08991302a72076dd7b7ff5 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 b/src/test/hive/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input0-1-efefd4364cd2790447fb0f908e87501f b/src/test/hive/golden/input0-1-efefd4364cd2790447fb0f908e87501f new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/src/test/hive/golden/input0-1-efefd4364cd2790447fb0f908e87501f @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 b/src/test/hive/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 b/src/test/hive/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 new file mode 100644 index 0000000000000..e00bf4ec2c7e5 --- /dev/null +++ b/src/test/hive/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 @@ -0,0 +1,17 @@ +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/src/test/hive/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 b/src/test/hive/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 new file mode 100644 index 0000000000000..743be67e8d1c3 --- /dev/null +++ b/src/test/hive/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 @@ -0,0 +1,2 @@ +a int None +b double None \ No newline at end of file diff --git a/src/test/hive/golden/input10-0-acdd72fda64c5463ce29f31020012d42 b/src/test/hive/golden/input10-0-acdd72fda64c5463ce29f31020012d42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 b/src/test/hive/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 new file mode 100644 index 0000000000000..175d371fd09c9 --- /dev/null +++ b/src/test/hive/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 @@ -0,0 +1,17 @@ +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/src/test/hive/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 b/src/test/hive/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 new file mode 100644 index 0000000000000..8dcdf43e31be3 --- /dev/null +++ b/src/test/hive/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 @@ -0,0 +1,10 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb b/src/test/hive/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 b/src/test/hive/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..37dd922fd0787 --- /dev/null +++ b/src/test/hive/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,84 @@ +86 val_86 +27 val_27 +98 val_98 +66 val_66 +37 val_37 +15 val_15 +82 val_82 +17 val_17 +0 val_0 +57 val_57 +20 val_20 +92 val_92 +47 val_47 +72 val_72 +4 val_4 +35 val_35 +54 val_54 +51 val_51 +65 val_65 +83 val_83 +12 val_12 +67 val_67 +84 val_84 +58 val_58 +8 val_8 +24 val_24 +42 val_42 +0 val_0 +96 val_96 +26 val_26 +51 val_51 +43 val_43 +95 val_95 +98 val_98 +85 val_85 +77 val_77 +0 val_0 +87 val_87 +15 val_15 +72 val_72 +90 val_90 +19 val_19 +10 val_10 +5 val_5 +58 val_58 +35 val_35 +95 val_95 +11 val_11 +34 val_34 +42 val_42 +78 val_78 +76 val_76 +41 val_41 +30 val_30 +64 val_64 +76 val_76 +74 val_74 +69 val_69 +33 val_33 +70 val_70 +5 val_5 +2 val_2 +35 val_35 +80 val_80 +44 val_44 +53 val_53 +90 val_90 +12 val_12 +5 val_5 +70 val_70 +24 val_24 +70 val_70 +83 val_83 +26 val_26 +67 val_67 +18 val_18 +9 val_9 +18 val_18 +97 val_97 +84 val_84 +28 val_28 +37 val_37 +90 val_90 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 b/src/test/hive/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 b/src/test/hive/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 b/src/test/hive/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 new file mode 100644 index 0000000000000..fdf93911ee847 --- /dev/null +++ b/src/test/hive/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 @@ -0,0 +1,10 @@ +0 val_0 +15 val_15 +17 val_17 +27 val_27 +37 val_37 +57 val_57 +66 val_66 +82 val_82 +86 val_86 +98 val_98 \ No newline at end of file diff --git a/src/test/hive/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e b/src/test/hive/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 b/src/test/hive/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c b/src/test/hive/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c new file mode 100644 index 0000000000000..4220cf5f30392 --- /dev/null +++ b/src/test/hive/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c @@ -0,0 +1,311 @@ +238 2008-04-08 12 +311 2008-04-08 12 +409 2008-04-08 12 +255 2008-04-08 12 +278 2008-04-08 12 +484 2008-04-08 12 +265 2008-04-08 12 +401 2008-04-08 12 +273 2008-04-08 12 +224 2008-04-08 12 +369 2008-04-08 12 +213 2008-04-08 12 +406 2008-04-08 12 +429 2008-04-08 12 +374 2008-04-08 12 +469 2008-04-08 12 +495 2008-04-08 12 +327 2008-04-08 12 +281 2008-04-08 12 +277 2008-04-08 12 +209 2008-04-08 12 +403 2008-04-08 12 +417 2008-04-08 12 +430 2008-04-08 12 +252 2008-04-08 12 +292 2008-04-08 12 +219 2008-04-08 12 +287 2008-04-08 12 +338 2008-04-08 12 +446 2008-04-08 12 +459 2008-04-08 12 +394 2008-04-08 12 +237 2008-04-08 12 +482 2008-04-08 12 +413 2008-04-08 12 +494 2008-04-08 12 +207 2008-04-08 12 +466 2008-04-08 12 +208 2008-04-08 12 +399 2008-04-08 12 +396 2008-04-08 12 +247 2008-04-08 12 +417 2008-04-08 12 +489 2008-04-08 12 +377 2008-04-08 12 +397 2008-04-08 12 +309 2008-04-08 12 +365 2008-04-08 12 +266 2008-04-08 12 +439 2008-04-08 12 +342 2008-04-08 12 +367 2008-04-08 12 +325 2008-04-08 12 +475 2008-04-08 12 +203 2008-04-08 12 +339 2008-04-08 12 +455 2008-04-08 12 +311 2008-04-08 12 +316 2008-04-08 12 +302 2008-04-08 12 +205 2008-04-08 12 +438 2008-04-08 12 +345 2008-04-08 12 +489 2008-04-08 12 +378 2008-04-08 12 +221 2008-04-08 12 +280 2008-04-08 12 +427 2008-04-08 12 +277 2008-04-08 12 +208 2008-04-08 12 +356 2008-04-08 12 +399 2008-04-08 12 +382 2008-04-08 12 +498 2008-04-08 12 +386 2008-04-08 12 +437 2008-04-08 12 +469 2008-04-08 12 +286 2008-04-08 12 +459 2008-04-08 12 +239 2008-04-08 12 +213 2008-04-08 12 +216 2008-04-08 12 +430 2008-04-08 12 +278 2008-04-08 12 +289 2008-04-08 12 +221 2008-04-08 12 +318 2008-04-08 12 +332 2008-04-08 12 +311 2008-04-08 12 +275 2008-04-08 12 +241 2008-04-08 12 +333 2008-04-08 12 +284 2008-04-08 12 +230 2008-04-08 12 +260 2008-04-08 12 +404 2008-04-08 12 +384 2008-04-08 12 +489 2008-04-08 12 +353 2008-04-08 12 +373 2008-04-08 12 +272 2008-04-08 12 +217 2008-04-08 12 +348 2008-04-08 12 +466 2008-04-08 12 +411 2008-04-08 12 +230 2008-04-08 12 +208 2008-04-08 12 +348 2008-04-08 12 +463 2008-04-08 12 +431 2008-04-08 12 +496 2008-04-08 12 +322 2008-04-08 12 +468 2008-04-08 12 +393 2008-04-08 12 +454 2008-04-08 12 +298 2008-04-08 12 +418 2008-04-08 12 +327 2008-04-08 12 +230 2008-04-08 12 +205 2008-04-08 12 +404 2008-04-08 12 +436 2008-04-08 12 +469 2008-04-08 12 +468 2008-04-08 12 +308 2008-04-08 12 +288 2008-04-08 12 +481 2008-04-08 12 +457 2008-04-08 12 +282 2008-04-08 12 +318 2008-04-08 12 +318 2008-04-08 12 +409 2008-04-08 12 +470 2008-04-08 12 +369 2008-04-08 12 +316 2008-04-08 12 +413 2008-04-08 12 +490 2008-04-08 12 +364 2008-04-08 12 +395 2008-04-08 12 +282 2008-04-08 12 +238 2008-04-08 12 +419 2008-04-08 12 +307 2008-04-08 12 +435 2008-04-08 12 +277 2008-04-08 12 +273 2008-04-08 12 +306 2008-04-08 12 +224 2008-04-08 12 +309 2008-04-08 12 +389 2008-04-08 12 +327 2008-04-08 12 +242 2008-04-08 12 +369 2008-04-08 12 +392 2008-04-08 12 +272 2008-04-08 12 +331 2008-04-08 12 +401 2008-04-08 12 +242 2008-04-08 12 +452 2008-04-08 12 +226 2008-04-08 12 +497 2008-04-08 12 +402 2008-04-08 12 +396 2008-04-08 12 +317 2008-04-08 12 +395 2008-04-08 12 +336 2008-04-08 12 +229 2008-04-08 12 +233 2008-04-08 12 +472 2008-04-08 12 +322 2008-04-08 12 +498 2008-04-08 12 +321 2008-04-08 12 +430 2008-04-08 12 +489 2008-04-08 12 +458 2008-04-08 12 +223 2008-04-08 12 +492 2008-04-08 12 +449 2008-04-08 12 +218 2008-04-08 12 +228 2008-04-08 12 +453 2008-04-08 12 +209 2008-04-08 12 +468 2008-04-08 12 +342 2008-04-08 12 +230 2008-04-08 12 +368 2008-04-08 12 +296 2008-04-08 12 +216 2008-04-08 12 +367 2008-04-08 12 +344 2008-04-08 12 +274 2008-04-08 12 +219 2008-04-08 12 +239 2008-04-08 12 +485 2008-04-08 12 +223 2008-04-08 12 +256 2008-04-08 12 +263 2008-04-08 12 +487 2008-04-08 12 +480 2008-04-08 12 +401 2008-04-08 12 +288 2008-04-08 12 +244 2008-04-08 12 +438 2008-04-08 12 +467 2008-04-08 12 +432 2008-04-08 12 +202 2008-04-08 12 +316 2008-04-08 12 +229 2008-04-08 12 +469 2008-04-08 12 +463 2008-04-08 12 +280 2008-04-08 12 +283 2008-04-08 12 +331 2008-04-08 12 +235 2008-04-08 12 +321 2008-04-08 12 +335 2008-04-08 12 +466 2008-04-08 12 +366 2008-04-08 12 +403 2008-04-08 12 +483 2008-04-08 12 +257 2008-04-08 12 +406 2008-04-08 12 +409 2008-04-08 12 +406 2008-04-08 12 +401 2008-04-08 12 +258 2008-04-08 12 +203 2008-04-08 12 +262 2008-04-08 12 +348 2008-04-08 12 +424 2008-04-08 12 +396 2008-04-08 12 +201 2008-04-08 12 +217 2008-04-08 12 +431 2008-04-08 12 +454 2008-04-08 12 +478 2008-04-08 12 +298 2008-04-08 12 +431 2008-04-08 12 +424 2008-04-08 12 +382 2008-04-08 12 +397 2008-04-08 12 +480 2008-04-08 12 +291 2008-04-08 12 +351 2008-04-08 12 +255 2008-04-08 12 +438 2008-04-08 12 +414 2008-04-08 12 +200 2008-04-08 12 +491 2008-04-08 12 +237 2008-04-08 12 +439 2008-04-08 12 +360 2008-04-08 12 +248 2008-04-08 12 +479 2008-04-08 12 +305 2008-04-08 12 +417 2008-04-08 12 +444 2008-04-08 12 +429 2008-04-08 12 +443 2008-04-08 12 +323 2008-04-08 12 +325 2008-04-08 12 +277 2008-04-08 12 +230 2008-04-08 12 +478 2008-04-08 12 +468 2008-04-08 12 +310 2008-04-08 12 +317 2008-04-08 12 +333 2008-04-08 12 +493 2008-04-08 12 +460 2008-04-08 12 +207 2008-04-08 12 +249 2008-04-08 12 +265 2008-04-08 12 +480 2008-04-08 12 +353 2008-04-08 12 +214 2008-04-08 12 +462 2008-04-08 12 +233 2008-04-08 12 +406 2008-04-08 12 +454 2008-04-08 12 +375 2008-04-08 12 +401 2008-04-08 12 +421 2008-04-08 12 +407 2008-04-08 12 +384 2008-04-08 12 +256 2008-04-08 12 +384 2008-04-08 12 +379 2008-04-08 12 +462 2008-04-08 12 +492 2008-04-08 12 +298 2008-04-08 12 +341 2008-04-08 12 +498 2008-04-08 12 +458 2008-04-08 12 +362 2008-04-08 12 +285 2008-04-08 12 +348 2008-04-08 12 +273 2008-04-08 12 +281 2008-04-08 12 +344 2008-04-08 12 +469 2008-04-08 12 +315 2008-04-08 12 +448 2008-04-08 12 +348 2008-04-08 12 +307 2008-04-08 12 +414 2008-04-08 12 +477 2008-04-08 12 +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 diff --git a/src/test/hive/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 b/src/test/hive/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 b/src/test/hive/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 b/src/test/hive/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf b/src/test/hive/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 b/src/test/hive/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input12-7-16da1c1293626b943343443368679e9c b/src/test/hive/golden/input12-7-16da1c1293626b943343443368679e9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..37dd922fd0787 --- /dev/null +++ b/src/test/hive/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,84 @@ +86 val_86 +27 val_27 +98 val_98 +66 val_66 +37 val_37 +15 val_15 +82 val_82 +17 val_17 +0 val_0 +57 val_57 +20 val_20 +92 val_92 +47 val_47 +72 val_72 +4 val_4 +35 val_35 +54 val_54 +51 val_51 +65 val_65 +83 val_83 +12 val_12 +67 val_67 +84 val_84 +58 val_58 +8 val_8 +24 val_24 +42 val_42 +0 val_0 +96 val_96 +26 val_26 +51 val_51 +43 val_43 +95 val_95 +98 val_98 +85 val_85 +77 val_77 +0 val_0 +87 val_87 +15 val_15 +72 val_72 +90 val_90 +19 val_19 +10 val_10 +5 val_5 +58 val_58 +35 val_35 +95 val_95 +11 val_11 +34 val_34 +42 val_42 +78 val_78 +76 val_76 +41 val_41 +30 val_30 +64 val_64 +76 val_76 +74 val_74 +69 val_69 +33 val_33 +70 val_70 +5 val_5 +2 val_2 +35 val_35 +80 val_80 +44 val_44 +53 val_53 +90 val_90 +12 val_12 +5 val_5 +70 val_70 +24 val_24 +70 val_70 +83 val_83 +26 val_26 +67 val_67 +18 val_18 +9 val_9 +18 val_18 +97 val_97 +84 val_84 +28 val_28 +37 val_37 +90 val_90 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd b/src/test/hive/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd new file mode 100644 index 0000000000000..d6e0c29932b9b --- /dev/null +++ b/src/test/hive/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd @@ -0,0 +1,105 @@ +165 val_165 +193 val_193 +150 val_150 +128 val_128 +146 val_146 +152 val_152 +145 val_145 +166 val_166 +153 val_153 +193 val_193 +174 val_174 +199 val_199 +174 val_174 +162 val_162 +167 val_167 +195 val_195 +113 val_113 +155 val_155 +128 val_128 +149 val_149 +129 val_129 +170 val_170 +157 val_157 +111 val_111 +169 val_169 +125 val_125 +192 val_192 +187 val_187 +176 val_176 +138 val_138 +103 val_103 +176 val_176 +137 val_137 +180 val_180 +181 val_181 +138 val_138 +179 val_179 +172 val_172 +129 val_129 +158 val_158 +119 val_119 +197 val_197 +100 val_100 +199 val_199 +191 val_191 +165 val_165 +120 val_120 +131 val_131 +156 val_156 +196 val_196 +197 val_197 +187 val_187 +137 val_137 +169 val_169 +179 val_179 +118 val_118 +134 val_134 +138 val_138 +118 val_118 +177 val_177 +168 val_168 +143 val_143 +160 val_160 +195 val_195 +119 val_119 +149 val_149 +138 val_138 +103 val_103 +113 val_113 +167 val_167 +116 val_116 +191 val_191 +128 val_128 +193 val_193 +104 val_104 +175 val_175 +105 val_105 +190 val_190 +114 val_114 +164 val_164 +125 val_125 +164 val_164 +187 val_187 +104 val_104 +163 val_163 +119 val_119 +199 val_199 +120 val_120 +169 val_169 +178 val_178 +136 val_136 +172 val_172 +133 val_133 +175 val_175 +189 val_189 +134 val_134 +100 val_100 +146 val_146 +186 val_186 +167 val_167 +183 val_183 +152 val_152 +194 val_194 +126 val_126 +169 val_169 \ No newline at end of file diff --git a/src/test/hive/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f b/src/test/hive/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 b/src/test/hive/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 b/src/test/hive/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 b/src/test/hive/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf b/src/test/hive/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 b/src/test/hive/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c b/src/test/hive/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..46057aa0a8fca --- /dev/null +++ b/src/test/hive/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,84 @@ +86 val_86 +27 val_27 +98 val_98 +66 val_66 +37 val_37 +15 val_15 +82 val_82 +17 val_17 +0 val_0 +57 val_57 +20 val_20 +92 val_92 +47 val_47 +72 val_72 +4 val_4 +35 val_35 +54 val_54 +51 val_51 +65 val_65 +83 val_83 +12 val_12 +67 val_67 +84 val_84 +58 val_58 +8 val_8 +24 val_24 +42 val_42 +0 val_0 +96 val_96 +26 val_26 +51 val_51 +43 val_43 +95 val_95 +98 val_98 +85 val_85 +77 val_77 +0 val_0 +87 val_87 +15 val_15 +72 val_72 +90 val_90 +19 val_19 +10 val_10 +5 val_5 +58 val_58 +35 val_35 +95 val_95 +11 val_11 +34 val_34 +42 val_42 +78 val_78 +76 val_76 +41 val_41 +30 val_30 +64 val_64 +76 val_76 +74 val_74 +69 val_69 +33 val_33 +70 val_70 +5 val_5 +2 val_2 +35 val_35 +80 val_80 +44 val_44 +53 val_53 +90 val_90 +12 val_12 +5 val_5 +70 val_70 +24 val_24 +70 val_70 +83 val_83 +26 val_26 +67 val_67 +18 val_18 +9 val_9 +18 val_18 +97 val_97 +84 val_84 +28 val_28 +37 val_37 +90 val_90 +97 val_97 diff --git a/src/test/hive/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd b/src/test/hive/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd new file mode 100644 index 0000000000000..9ee31317478d5 --- /dev/null +++ b/src/test/hive/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd @@ -0,0 +1,105 @@ +165 val_165 +193 val_193 +150 val_150 +128 val_128 +146 val_146 +152 val_152 +145 val_145 +166 val_166 +153 val_153 +193 val_193 +174 val_174 +199 val_199 +174 val_174 +162 val_162 +167 val_167 +195 val_195 +113 val_113 +155 val_155 +128 val_128 +149 val_149 +129 val_129 +170 val_170 +157 val_157 +111 val_111 +169 val_169 +125 val_125 +192 val_192 +187 val_187 +176 val_176 +138 val_138 +103 val_103 +176 val_176 +137 val_137 +180 val_180 +181 val_181 +138 val_138 +179 val_179 +172 val_172 +129 val_129 +158 val_158 +119 val_119 +197 val_197 +100 val_100 +199 val_199 +191 val_191 +165 val_165 +120 val_120 +131 val_131 +156 val_156 +196 val_196 +197 val_197 +187 val_187 +137 val_137 +169 val_169 +179 val_179 +118 val_118 +134 val_134 +138 val_138 +118 val_118 +177 val_177 +168 val_168 +143 val_143 +160 val_160 +195 val_195 +119 val_119 +149 val_149 +138 val_138 +103 val_103 +113 val_113 +167 val_167 +116 val_116 +191 val_191 +128 val_128 +193 val_193 +104 val_104 +175 val_175 +105 val_105 +190 val_190 +114 val_114 +164 val_164 +125 val_125 +164 val_164 +187 val_187 +104 val_104 +163 val_163 +119 val_119 +199 val_199 +120 val_120 +169 val_169 +178 val_178 +136 val_136 +172 val_172 +133 val_133 +175 val_175 +189 val_189 +134 val_134 +100 val_100 +146 val_146 +186 val_186 +167 val_167 +183 val_183 +152 val_152 +194 val_194 +126 val_126 +169 val_169 diff --git a/src/test/hive/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c b/src/test/hive/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c new file mode 100644 index 0000000000000..a66a07386eef8 --- /dev/null +++ b/src/test/hive/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c @@ -0,0 +1,311 @@ +238 2008-04-08 12 +311 2008-04-08 12 +409 2008-04-08 12 +255 2008-04-08 12 +278 2008-04-08 12 +484 2008-04-08 12 +265 2008-04-08 12 +401 2008-04-08 12 +273 2008-04-08 12 +224 2008-04-08 12 +369 2008-04-08 12 +213 2008-04-08 12 +406 2008-04-08 12 +429 2008-04-08 12 +374 2008-04-08 12 +469 2008-04-08 12 +495 2008-04-08 12 +327 2008-04-08 12 +281 2008-04-08 12 +277 2008-04-08 12 +209 2008-04-08 12 +403 2008-04-08 12 +417 2008-04-08 12 +430 2008-04-08 12 +252 2008-04-08 12 +292 2008-04-08 12 +219 2008-04-08 12 +287 2008-04-08 12 +338 2008-04-08 12 +446 2008-04-08 12 +459 2008-04-08 12 +394 2008-04-08 12 +237 2008-04-08 12 +482 2008-04-08 12 +413 2008-04-08 12 +494 2008-04-08 12 +207 2008-04-08 12 +466 2008-04-08 12 +208 2008-04-08 12 +399 2008-04-08 12 +396 2008-04-08 12 +247 2008-04-08 12 +417 2008-04-08 12 +489 2008-04-08 12 +377 2008-04-08 12 +397 2008-04-08 12 +309 2008-04-08 12 +365 2008-04-08 12 +266 2008-04-08 12 +439 2008-04-08 12 +342 2008-04-08 12 +367 2008-04-08 12 +325 2008-04-08 12 +475 2008-04-08 12 +203 2008-04-08 12 +339 2008-04-08 12 +455 2008-04-08 12 +311 2008-04-08 12 +316 2008-04-08 12 +302 2008-04-08 12 +205 2008-04-08 12 +438 2008-04-08 12 +345 2008-04-08 12 +489 2008-04-08 12 +378 2008-04-08 12 +221 2008-04-08 12 +280 2008-04-08 12 +427 2008-04-08 12 +277 2008-04-08 12 +208 2008-04-08 12 +356 2008-04-08 12 +399 2008-04-08 12 +382 2008-04-08 12 +498 2008-04-08 12 +386 2008-04-08 12 +437 2008-04-08 12 +469 2008-04-08 12 +286 2008-04-08 12 +459 2008-04-08 12 +239 2008-04-08 12 +213 2008-04-08 12 +216 2008-04-08 12 +430 2008-04-08 12 +278 2008-04-08 12 +289 2008-04-08 12 +221 2008-04-08 12 +318 2008-04-08 12 +332 2008-04-08 12 +311 2008-04-08 12 +275 2008-04-08 12 +241 2008-04-08 12 +333 2008-04-08 12 +284 2008-04-08 12 +230 2008-04-08 12 +260 2008-04-08 12 +404 2008-04-08 12 +384 2008-04-08 12 +489 2008-04-08 12 +353 2008-04-08 12 +373 2008-04-08 12 +272 2008-04-08 12 +217 2008-04-08 12 +348 2008-04-08 12 +466 2008-04-08 12 +411 2008-04-08 12 +230 2008-04-08 12 +208 2008-04-08 12 +348 2008-04-08 12 +463 2008-04-08 12 +431 2008-04-08 12 +496 2008-04-08 12 +322 2008-04-08 12 +468 2008-04-08 12 +393 2008-04-08 12 +454 2008-04-08 12 +298 2008-04-08 12 +418 2008-04-08 12 +327 2008-04-08 12 +230 2008-04-08 12 +205 2008-04-08 12 +404 2008-04-08 12 +436 2008-04-08 12 +469 2008-04-08 12 +468 2008-04-08 12 +308 2008-04-08 12 +288 2008-04-08 12 +481 2008-04-08 12 +457 2008-04-08 12 +282 2008-04-08 12 +318 2008-04-08 12 +318 2008-04-08 12 +409 2008-04-08 12 +470 2008-04-08 12 +369 2008-04-08 12 +316 2008-04-08 12 +413 2008-04-08 12 +490 2008-04-08 12 +364 2008-04-08 12 +395 2008-04-08 12 +282 2008-04-08 12 +238 2008-04-08 12 +419 2008-04-08 12 +307 2008-04-08 12 +435 2008-04-08 12 +277 2008-04-08 12 +273 2008-04-08 12 +306 2008-04-08 12 +224 2008-04-08 12 +309 2008-04-08 12 +389 2008-04-08 12 +327 2008-04-08 12 +242 2008-04-08 12 +369 2008-04-08 12 +392 2008-04-08 12 +272 2008-04-08 12 +331 2008-04-08 12 +401 2008-04-08 12 +242 2008-04-08 12 +452 2008-04-08 12 +226 2008-04-08 12 +497 2008-04-08 12 +402 2008-04-08 12 +396 2008-04-08 12 +317 2008-04-08 12 +395 2008-04-08 12 +336 2008-04-08 12 +229 2008-04-08 12 +233 2008-04-08 12 +472 2008-04-08 12 +322 2008-04-08 12 +498 2008-04-08 12 +321 2008-04-08 12 +430 2008-04-08 12 +489 2008-04-08 12 +458 2008-04-08 12 +223 2008-04-08 12 +492 2008-04-08 12 +449 2008-04-08 12 +218 2008-04-08 12 +228 2008-04-08 12 +453 2008-04-08 12 +209 2008-04-08 12 +468 2008-04-08 12 +342 2008-04-08 12 +230 2008-04-08 12 +368 2008-04-08 12 +296 2008-04-08 12 +216 2008-04-08 12 +367 2008-04-08 12 +344 2008-04-08 12 +274 2008-04-08 12 +219 2008-04-08 12 +239 2008-04-08 12 +485 2008-04-08 12 +223 2008-04-08 12 +256 2008-04-08 12 +263 2008-04-08 12 +487 2008-04-08 12 +480 2008-04-08 12 +401 2008-04-08 12 +288 2008-04-08 12 +244 2008-04-08 12 +438 2008-04-08 12 +467 2008-04-08 12 +432 2008-04-08 12 +202 2008-04-08 12 +316 2008-04-08 12 +229 2008-04-08 12 +469 2008-04-08 12 +463 2008-04-08 12 +280 2008-04-08 12 +283 2008-04-08 12 +331 2008-04-08 12 +235 2008-04-08 12 +321 2008-04-08 12 +335 2008-04-08 12 +466 2008-04-08 12 +366 2008-04-08 12 +403 2008-04-08 12 +483 2008-04-08 12 +257 2008-04-08 12 +406 2008-04-08 12 +409 2008-04-08 12 +406 2008-04-08 12 +401 2008-04-08 12 +258 2008-04-08 12 +203 2008-04-08 12 +262 2008-04-08 12 +348 2008-04-08 12 +424 2008-04-08 12 +396 2008-04-08 12 +201 2008-04-08 12 +217 2008-04-08 12 +431 2008-04-08 12 +454 2008-04-08 12 +478 2008-04-08 12 +298 2008-04-08 12 +431 2008-04-08 12 +424 2008-04-08 12 +382 2008-04-08 12 +397 2008-04-08 12 +480 2008-04-08 12 +291 2008-04-08 12 +351 2008-04-08 12 +255 2008-04-08 12 +438 2008-04-08 12 +414 2008-04-08 12 +200 2008-04-08 12 +491 2008-04-08 12 +237 2008-04-08 12 +439 2008-04-08 12 +360 2008-04-08 12 +248 2008-04-08 12 +479 2008-04-08 12 +305 2008-04-08 12 +417 2008-04-08 12 +444 2008-04-08 12 +429 2008-04-08 12 +443 2008-04-08 12 +323 2008-04-08 12 +325 2008-04-08 12 +277 2008-04-08 12 +230 2008-04-08 12 +478 2008-04-08 12 +468 2008-04-08 12 +310 2008-04-08 12 +317 2008-04-08 12 +333 2008-04-08 12 +493 2008-04-08 12 +460 2008-04-08 12 +207 2008-04-08 12 +249 2008-04-08 12 +265 2008-04-08 12 +480 2008-04-08 12 +353 2008-04-08 12 +214 2008-04-08 12 +462 2008-04-08 12 +233 2008-04-08 12 +406 2008-04-08 12 +454 2008-04-08 12 +375 2008-04-08 12 +401 2008-04-08 12 +421 2008-04-08 12 +407 2008-04-08 12 +384 2008-04-08 12 +256 2008-04-08 12 +384 2008-04-08 12 +379 2008-04-08 12 +462 2008-04-08 12 +492 2008-04-08 12 +298 2008-04-08 12 +341 2008-04-08 12 +498 2008-04-08 12 +458 2008-04-08 12 +362 2008-04-08 12 +285 2008-04-08 12 +348 2008-04-08 12 +273 2008-04-08 12 +281 2008-04-08 12 +344 2008-04-08 12 +469 2008-04-08 12 +315 2008-04-08 12 +448 2008-04-08 12 +348 2008-04-08 12 +307 2008-04-08 12 +414 2008-04-08 12 +477 2008-04-08 12 +222 2008-04-08 12 +403 2008-04-08 12 +400 2008-04-08 12 +200 2008-04-08 12 diff --git a/src/test/hive/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a b/src/test/hive/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 b/src/test/hive/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..703a4eef24f3f --- /dev/null +++ b/src/test/hive/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,84 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf b/src/test/hive/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 b/src/test/hive/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..4335dce6a9929 --- /dev/null +++ b/src/test/hive/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,5 @@ +0 val_0 +0 val_0 +0 val_0 +10 val_10 +11 val_11 \ No newline at end of file diff --git a/src/test/hive/golden/input16_cc-0-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/input16_cc-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/input16_cc-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 b/src/test/hive/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 b/src/test/hive/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 b/src/test/hive/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input17-2-21166e268096f6ec67f4f57ec333e901 b/src/test/hive/golden/input17-2-21166e268096f6ec67f4f57ec333e901 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..1c48b3680a3ac --- /dev/null +++ b/src/test/hive/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,11 @@ +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/src/test/hive/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a b/src/test/hive/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc b/src/test/hive/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 b/src/test/hive/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 new file mode 100644 index 0000000000000..795dbe3a976eb --- /dev/null +++ b/src/test/hive/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 @@ -0,0 +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 diff --git a/src/test/hive/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 b/src/test/hive/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 b/src/test/hive/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e b/src/test/hive/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 b/src/test/hive/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 new file mode 100644 index 0000000000000..fdf93911ee847 --- /dev/null +++ b/src/test/hive/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 @@ -0,0 +1,10 @@ +0 val_0 +15 val_15 +17 val_17 +27 val_27 +37 val_37 +57 val_57 +66 val_66 +82 val_82 +86 val_86 +98 val_98 \ No newline at end of file diff --git a/src/test/hive/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 b/src/test/hive/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 new file mode 100644 index 0000000000000..8a8f1a1b8bbe1 --- /dev/null +++ b/src/test/hive/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 @@ -0,0 +1,5 @@ +27 val_27 +37 val_37 +66 val_66 +86 val_86 +98 val_98 \ No newline at end of file diff --git a/src/test/hive/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c b/src/test/hive/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input2-1-e0efeda558cd0194f4764a5735147b16 b/src/test/hive/golden/input2-1-e0efeda558cd0194f4764a5735147b16 new file mode 100644 index 0000000000000..743be67e8d1c3 --- /dev/null +++ b/src/test/hive/golden/input2-1-e0efeda558cd0194f4764a5735147b16 @@ -0,0 +1,2 @@ +a int None +b double None \ No newline at end of file diff --git a/src/test/hive/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd b/src/test/hive/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd new file mode 100644 index 0000000000000..743be67e8d1c3 --- /dev/null +++ b/src/test/hive/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd @@ -0,0 +1,2 @@ +a int None +b double None \ No newline at end of file diff --git a/src/test/hive/golden/input2-3-45b9f786f64afa5e039d1856fe926649 b/src/test/hive/golden/input2-3-45b9f786f64afa5e039d1856fe926649 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input2-4-235f92683416fab031e6e7490487b15b b/src/test/hive/golden/input2-4-235f92683416fab031e6e7490487b15b new file mode 100644 index 0000000000000..ca0726f517eeb --- /dev/null +++ b/src/test/hive/golden/input2-4-235f92683416fab031e6e7490487b15b @@ -0,0 +1,3 @@ +a array None +b double None +c map None \ No newline at end of file diff --git a/src/test/hive/golden/input2-5-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/input2-5-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..2c861553f9aa2 --- /dev/null +++ b/src/test/hive/golden/input2-5-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,4 @@ +src +srcpart +test2a +test2b \ No newline at end of file diff --git a/src/test/hive/golden/input2-6-4a6b9674c126337f71834f11613f996d b/src/test/hive/golden/input2-6-4a6b9674c126337f71834f11613f996d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input2-7-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/input2-7-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..3e40a0c866d4d --- /dev/null +++ b/src/test/hive/golden/input2-7-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,3 @@ +src +srcpart +test2b \ No newline at end of file diff --git a/src/test/hive/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 b/src/test/hive/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 b/src/test/hive/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 new file mode 100644 index 0000000000000..5337f342fedd8 --- /dev/null +++ b/src/test/hive/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 @@ -0,0 +1,17 @@ +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/src/test/hive/golden/input21-0-7f40c525398953892ffd6283f54ba427 b/src/test/hive/golden/input21-0-7f40c525398953892ffd6283f54ba427 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 b/src/test/hive/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 b/src/test/hive/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe b/src/test/hive/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe new file mode 100644 index 0000000000000..c6c298df200ca --- /dev/null +++ b/src/test/hive/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe @@ -0,0 +1,10 @@ +1.0 1 same 0 +1.0 1 same 1 +1.0 1 same 2 +1.0 1 same 3 +1.0 1 same 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 diff --git a/src/test/hive/golden/input22-0-b9cd4041ca510639838a97376747b498 b/src/test/hive/golden/input22-0-b9cd4041ca510639838a97376747b498 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 b/src/test/hive/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 b/src/test/hive/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input22-3-8285c1934441b12f6c016e13cb410e79 b/src/test/hive/golden/input22-3-8285c1934441b12f6c016e13cb410e79 new file mode 100644 index 0000000000000..336ba0545635d --- /dev/null +++ b/src/test/hive/golden/input22-3-8285c1934441b12f6c016e13cb410e79 @@ -0,0 +1,10 @@ +0 +0 +0 +10 +100 +100 +103 +103 +104 +104 \ No newline at end of file diff --git a/src/test/hive/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf b/src/test/hive/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb b/src/test/hive/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 b/src/test/hive/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input24-1-f57485de80ad64f9931bf92514fa1 b/src/test/hive/golden/input24-1-f57485de80ad64f9931bf92514fa1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 b/src/test/hive/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 b/src/test/hive/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 b/src/test/hive/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input25-1-f57485de80ad64f9931bf92514fa1 b/src/test/hive/golden/input25-1-f57485de80ad64f9931bf92514fa1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 b/src/test/hive/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 b/src/test/hive/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input25-4-72c74c55dd7d85d194448e9c58488938 b/src/test/hive/golden/input25-4-72c74c55dd7d85d194448e9c58488938 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 b/src/test/hive/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 b/src/test/hive/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 new file mode 100644 index 0000000000000..b8fd0ab6545d6 --- /dev/null +++ b/src/test/hive/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 @@ -0,0 +1,5 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a b/src/test/hive/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input28-1-f57485de80ad64f9931bf92514fa1 b/src/test/hive/golden/input28-1-f57485de80ad64f9931bf92514fa1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c b/src/test/hive/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 b/src/test/hive/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f b/src/test/hive/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 b/src/test/hive/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 new file mode 100644 index 0000000000000..899417ee77ad8 --- /dev/null +++ b/src/test/hive/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 @@ -0,0 +1,5 @@ +238 val_238 +86 val_86 +27 val_27 +165 val_165 +255 val_255 \ No newline at end of file diff --git a/src/test/hive/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff b/src/test/hive/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d b/src/test/hive/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d new file mode 100644 index 0000000000000..743be67e8d1c3 --- /dev/null +++ b/src/test/hive/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d @@ -0,0 +1,2 @@ +a int None +b double None \ No newline at end of file diff --git a/src/test/hive/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 b/src/test/hive/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 new file mode 100644 index 0000000000000..594b29ca1410f --- /dev/null +++ b/src/test/hive/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 @@ -0,0 +1,4 @@ +a array None +b double None +c map None +x double None \ No newline at end of file diff --git a/src/test/hive/golden/input3-11-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/input3-11-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ac382c7369264 --- /dev/null +++ b/src/test/hive/golden/input3-11-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,4 @@ +src +srcpart +test3a +test3c \ No newline at end of file diff --git a/src/test/hive/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 b/src/test/hive/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 new file mode 100644 index 0000000000000..cb17be511e875 --- /dev/null +++ b/src/test/hive/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a b/src/test/hive/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3-14-efee6816e20fe61595a4a2a991071219 b/src/test/hive/golden/input3-14-efee6816e20fe61595a4a2a991071219 new file mode 100644 index 0000000000000..b906fd3c2e775 --- /dev/null +++ b/src/test/hive/golden/input3-14-efee6816e20fe61595a4a2a991071219 @@ -0,0 +1,4 @@ +r1 int None +r2 double None + +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 diff --git a/src/test/hive/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 b/src/test/hive/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3-3-1c5990b1aed2be48311810dae3019994 b/src/test/hive/golden/input3-3-1c5990b1aed2be48311810dae3019994 new file mode 100644 index 0000000000000..ca0726f517eeb --- /dev/null +++ b/src/test/hive/golden/input3-3-1c5990b1aed2be48311810dae3019994 @@ -0,0 +1,3 @@ +a array None +b double None +c map None \ No newline at end of file diff --git a/src/test/hive/golden/input3-4-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/input3-4-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..6b1ce270d97e9 --- /dev/null +++ b/src/test/hive/golden/input3-4-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,4 @@ +src +srcpart +test3a +test3b \ No newline at end of file diff --git a/src/test/hive/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 b/src/test/hive/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 new file mode 100644 index 0000000000000..92c0ed68f8a7b --- /dev/null +++ b/src/test/hive/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/input3-6-ba8c440158c2519353d02471bfb05694 b/src/test/hive/golden/input3-6-ba8c440158c2519353d02471bfb05694 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3-7-1c5990b1aed2be48311810dae3019994 b/src/test/hive/golden/input3-7-1c5990b1aed2be48311810dae3019994 new file mode 100644 index 0000000000000..594b29ca1410f --- /dev/null +++ b/src/test/hive/golden/input3-7-1c5990b1aed2be48311810dae3019994 @@ -0,0 +1,4 @@ +a array None +b double None +c map None +x double None \ No newline at end of file diff --git a/src/test/hive/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be b/src/test/hive/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be new file mode 100644 index 0000000000000..09bbc29377720 --- /dev/null +++ b/src/test/hive/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be @@ -0,0 +1,14 @@ +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/src/test/hive/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e b/src/test/hive/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c b/src/test/hive/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input30-1-90c0d1a75de78c405413fd627caea4ab b/src/test/hive/golden/input30-1-90c0d1a75de78c405413fd627caea4ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input30-3-c21dba410fb07a098f93430a9d21df79 b/src/test/hive/golden/input30-3-c21dba410fb07a098f93430a9d21df79 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input30-3-c21dba410fb07a098f93430a9d21df79 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 b/src/test/hive/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 b/src/test/hive/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 b/src/test/hive/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f b/src/test/hive/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f new file mode 100644 index 0000000000000..25bf17fc5aaab --- /dev/null +++ b/src/test/hive/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f @@ -0,0 +1 @@ +18 \ No newline at end of file diff --git a/src/test/hive/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/input31-1-c21dba410fb07a098f93430a9d21df79 b/src/test/hive/golden/input31-1-c21dba410fb07a098f93430a9d21df79 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/input31-1-c21dba410fb07a098f93430a9d21df79 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/input31-2-705764f8f7cab9378964af30b83f7fe b/src/test/hive/golden/input31-2-705764f8f7cab9378964af30b83f7fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input31-3-50c905261882f7fd8539fdd91e68151f b/src/test/hive/golden/input31-3-50c905261882f7fd8539fdd91e68151f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input31-4-2f886fa357df9342733551fa1b53f913 b/src/test/hive/golden/input31-4-2f886fa357df9342733551fa1b53f913 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/input32-1-c21dba410fb07a098f93430a9d21df79 b/src/test/hive/golden/input32-1-c21dba410fb07a098f93430a9d21df79 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/input32-1-c21dba410fb07a098f93430a9d21df79 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 b/src/test/hive/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/input32-3-b0070890240c15d647af59f41b77ba3d b/src/test/hive/golden/input32-3-b0070890240c15d647af59f41b77ba3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 b/src/test/hive/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input32-5-8789d32fc5b784fe2d171566732c573e b/src/test/hive/golden/input32-5-8789d32fc5b784fe2d171566732c573e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input37-0-86e2e274650fb56651607ea10d356fc0 b/src/test/hive/golden/input37-0-86e2e274650fb56651607ea10d356fc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 b/src/test/hive/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39-0-7bd12162381231be9d578797818957a7 b/src/test/hive/golden/input39-0-7bd12162381231be9d578797818957a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/src/test/hive/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 b/src/test/hive/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 b/src/test/hive/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 new file mode 100644 index 0000000000000..25bf17fc5aaab --- /dev/null +++ b/src/test/hive/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 @@ -0,0 +1 @@ +18 \ No newline at end of file diff --git a/src/test/hive/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 b/src/test/hive/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 b/src/test/hive/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d b/src/test/hive/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 b/src/test/hive/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/src/test/hive/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39-4-53453776bf062f28d371fc7336b7eae2 b/src/test/hive/golden/input39-4-53453776bf062f28d371fc7336b7eae2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39-6-763ab5853bff619e6525c01e46b2a923 b/src/test/hive/golden/input39-6-763ab5853bff619e6525c01e46b2a923 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39-6-763ab5853bff619e6525c01e46b2a923 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e b/src/test/hive/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 b/src/test/hive/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 b/src/test/hive/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe b/src/test/hive/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/src/test/hive/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 b/src/test/hive/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 new file mode 100644 index 0000000000000..25bf17fc5aaab --- /dev/null +++ b/src/test/hive/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 @@ -0,0 +1 @@ +18 \ No newline at end of file diff --git a/src/test/hive/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 b/src/test/hive/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 b/src/test/hive/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 b/src/test/hive/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/src/test/hive/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 b/src/test/hive/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee b/src/test/hive/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 b/src/test/hive/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f b/src/test/hive/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 b/src/test/hive/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 b/src/test/hive/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 b/src/test/hive/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 b/src/test/hive/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d b/src/test/hive/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece b/src/test/hive/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d b/src/test/hive/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3_limit-5-3664b564747487df13a5d109837219b5 b/src/test/hive/golden/input3_limit-5-3664b564747487df13a5d109837219b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 b/src/test/hive/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 new file mode 100644 index 0000000000000..6731b0cb0baaa --- /dev/null +++ b/src/test/hive/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 @@ -0,0 +1,20 @@ +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/src/test/hive/golden/input4-0-b9cd4041ca510639838a97376747b498 b/src/test/hive/golden/input4-0-b9cd4041ca510639838a97376747b498 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 b/src/test/hive/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 new file mode 100644 index 0000000000000..65a457b52b0a6 --- /dev/null +++ b/src/test/hive/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 @@ -0,0 +1,27 @@ +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/src/test/hive/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 b/src/test/hive/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 b/src/test/hive/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 b/src/test/hive/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 new file mode 100644 index 0000000000000..218c223b050b7 --- /dev/null +++ b/src/test/hive/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 @@ -0,0 +1,500 @@ +val_238 238 +val_86 86 +val_311 311 +val_27 27 +val_165 165 +val_409 409 +val_255 255 +val_278 278 +val_98 98 +val_484 484 +val_265 265 +val_193 193 +val_401 401 +val_150 150 +val_273 273 +val_224 224 +val_369 369 +val_66 66 +val_128 128 +val_213 213 +val_146 146 +val_406 406 +val_429 429 +val_374 374 +val_152 152 +val_469 469 +val_145 145 +val_495 495 +val_37 37 +val_327 327 +val_281 281 +val_277 277 +val_209 209 +val_15 15 +val_82 82 +val_403 403 +val_166 166 +val_417 417 +val_430 430 +val_252 252 +val_292 292 +val_219 219 +val_287 287 +val_153 153 +val_193 193 +val_338 338 +val_446 446 +val_459 459 +val_394 394 +val_237 237 +val_482 482 +val_174 174 +val_413 413 +val_494 494 +val_207 207 +val_199 199 +val_466 466 +val_208 208 +val_174 174 +val_399 399 +val_396 396 +val_247 247 +val_417 417 +val_489 489 +val_162 162 +val_377 377 +val_397 397 +val_309 309 +val_365 365 +val_266 266 +val_439 439 +val_342 342 +val_367 367 +val_325 325 +val_167 167 +val_195 195 +val_475 475 +val_17 17 +val_113 113 +val_155 155 +val_203 203 +val_339 339 +val_0 0 +val_455 455 +val_128 128 +val_311 311 +val_316 316 +val_57 57 +val_302 302 +val_205 205 +val_149 149 +val_438 438 +val_345 345 +val_129 129 +val_170 170 +val_20 20 +val_489 489 +val_157 157 +val_378 378 +val_221 221 +val_92 92 +val_111 111 +val_47 47 +val_72 72 +val_4 4 +val_280 280 +val_35 35 +val_427 427 +val_277 277 +val_208 208 +val_356 356 +val_399 399 +val_169 169 +val_382 382 +val_498 498 +val_125 125 +val_386 386 +val_437 437 +val_469 469 +val_192 192 +val_286 286 +val_187 187 +val_176 176 +val_54 54 +val_459 459 +val_51 51 +val_138 138 +val_103 103 +val_239 239 +val_213 213 +val_216 216 +val_430 430 +val_278 278 +val_176 176 +val_289 289 +val_221 221 +val_65 65 +val_318 318 +val_332 332 +val_311 311 +val_275 275 +val_137 137 +val_241 241 +val_83 83 +val_333 333 +val_180 180 +val_284 284 +val_12 12 +val_230 230 +val_181 181 +val_67 67 +val_260 260 +val_404 404 +val_384 384 +val_489 489 +val_353 353 +val_373 373 +val_272 272 +val_138 138 +val_217 217 +val_84 84 +val_348 348 +val_466 466 +val_58 58 +val_8 8 +val_411 411 +val_230 230 +val_208 208 +val_348 348 +val_24 24 +val_463 463 +val_431 431 +val_179 179 +val_172 172 +val_42 42 +val_129 129 +val_158 158 +val_119 119 +val_496 496 +val_0 0 +val_322 322 +val_197 197 +val_468 468 +val_393 393 +val_454 454 +val_100 100 +val_298 298 +val_199 199 +val_191 191 +val_418 418 +val_96 96 +val_26 26 +val_165 165 +val_327 327 +val_230 230 +val_205 205 +val_120 120 +val_131 131 +val_51 51 +val_404 404 +val_43 43 +val_436 436 +val_156 156 +val_469 469 +val_468 468 +val_308 308 +val_95 95 +val_196 196 +val_288 288 +val_481 481 +val_457 457 +val_98 98 +val_282 282 +val_197 197 +val_187 187 +val_318 318 +val_318 318 +val_409 409 +val_470 470 +val_137 137 +val_369 369 +val_316 316 +val_169 169 +val_413 413 +val_85 85 +val_77 77 +val_0 0 +val_490 490 +val_87 87 +val_364 364 +val_179 179 +val_118 118 +val_134 134 +val_395 395 +val_282 282 +val_138 138 +val_238 238 +val_419 419 +val_15 15 +val_118 118 +val_72 72 +val_90 90 +val_307 307 +val_19 19 +val_435 435 +val_10 10 +val_277 277 +val_273 273 +val_306 306 +val_224 224 +val_309 309 +val_389 389 +val_327 327 +val_242 242 +val_369 369 +val_392 392 +val_272 272 +val_331 331 +val_401 401 +val_242 242 +val_452 452 +val_177 177 +val_226 226 +val_5 5 +val_497 497 +val_402 402 +val_396 396 +val_317 317 +val_395 395 +val_58 58 +val_35 35 +val_336 336 +val_95 95 +val_11 11 +val_168 168 +val_34 34 +val_229 229 +val_233 233 +val_143 143 +val_472 472 +val_322 322 +val_498 498 +val_160 160 +val_195 195 +val_42 42 +val_321 321 +val_430 430 +val_119 119 +val_489 489 +val_458 458 +val_78 78 +val_76 76 +val_41 41 +val_223 223 +val_492 492 +val_149 149 +val_449 449 +val_218 218 +val_228 228 +val_138 138 +val_453 453 +val_30 30 +val_209 209 +val_64 64 +val_468 468 +val_76 76 +val_74 74 +val_342 342 +val_69 69 +val_230 230 +val_33 33 +val_368 368 +val_103 103 +val_296 296 +val_113 113 +val_216 216 +val_367 367 +val_344 344 +val_167 167 +val_274 274 +val_219 219 +val_239 239 +val_485 485 +val_116 116 +val_223 223 +val_256 256 +val_263 263 +val_70 70 +val_487 487 +val_480 480 +val_401 401 +val_288 288 +val_191 191 +val_5 5 +val_244 244 +val_438 438 +val_128 128 +val_467 467 +val_432 432 +val_202 202 +val_316 316 +val_229 229 +val_469 469 +val_463 463 +val_280 280 +val_2 2 +val_35 35 +val_283 283 +val_331 331 +val_235 235 +val_80 80 +val_44 44 +val_193 193 +val_321 321 +val_335 335 +val_104 104 +val_466 466 +val_366 366 +val_175 175 +val_403 403 +val_483 483 +val_53 53 +val_105 105 +val_257 257 +val_406 406 +val_409 409 +val_190 190 +val_406 406 +val_401 401 +val_114 114 +val_258 258 +val_90 90 +val_203 203 +val_262 262 +val_348 348 +val_424 424 +val_12 12 +val_396 396 +val_201 201 +val_217 217 +val_164 164 +val_431 431 +val_454 454 +val_478 478 +val_298 298 +val_125 125 +val_431 431 +val_164 164 +val_424 424 +val_187 187 +val_382 382 +val_5 5 +val_70 70 +val_397 397 +val_480 480 +val_291 291 +val_24 24 +val_351 351 +val_255 255 +val_104 104 +val_70 70 +val_163 163 +val_438 438 +val_119 119 +val_414 414 +val_200 200 +val_491 491 +val_237 237 +val_439 439 +val_360 360 +val_248 248 +val_479 479 +val_305 305 +val_417 417 +val_199 199 +val_444 444 +val_120 120 +val_429 429 +val_169 169 +val_443 443 +val_323 323 +val_325 325 +val_277 277 +val_230 230 +val_478 478 +val_178 178 +val_468 468 +val_310 310 +val_317 317 +val_333 333 +val_493 493 +val_460 460 +val_207 207 +val_249 249 +val_265 265 +val_480 480 +val_83 83 +val_136 136 +val_353 353 +val_172 172 +val_214 214 +val_462 462 +val_233 233 +val_406 406 +val_133 133 +val_175 175 +val_189 189 +val_454 454 +val_375 375 +val_401 401 +val_421 421 +val_407 407 +val_384 384 +val_256 256 +val_26 26 +val_134 134 +val_67 67 +val_384 384 +val_379 379 +val_18 18 +val_462 462 +val_492 492 +val_100 100 +val_298 298 +val_9 9 +val_341 341 +val_498 498 +val_146 146 +val_458 458 +val_362 362 +val_186 186 +val_285 285 +val_348 348 +val_167 167 +val_18 18 +val_273 273 +val_183 183 +val_281 281 +val_344 344 +val_97 97 +val_469 469 +val_315 315 +val_84 84 +val_28 28 +val_37 37 +val_448 448 +val_152 152 +val_348 348 +val_307 307 +val_194 194 +val_414 414 +val_477 477 +val_222 222 +val_126 126 +val_90 90 +val_169 169 +val_403 403 +val_400 400 +val_200 200 +val_97 97 \ No newline at end of file diff --git a/src/test/hive/golden/input40-0-37e74908fd43254141ae3664f13a48d b/src/test/hive/golden/input40-0-37e74908fd43254141ae3664f13a48d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 b/src/test/hive/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 b/src/test/hive/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/src/test/hive/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/input40-3-12f634800ffae17942ddd789a01af227 b/src/test/hive/golden/input40-3-12f634800ffae17942ddd789a01af227 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e b/src/test/hive/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 b/src/test/hive/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 new file mode 100644 index 0000000000000..6bc66cd8fe19b --- /dev/null +++ b/src/test/hive/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 @@ -0,0 +1,500 @@ +0 val_0 2009-08-01 +0 val_0 2009-08-01 +0 val_0 2009-08-01 +10 val_10 2009-08-01 +100 val_100 2009-08-01 +100 val_100 2009-08-01 +103 val_103 2009-08-01 +103 val_103 2009-08-01 +104 val_104 2009-08-01 +104 val_104 2009-08-01 +105 val_105 2009-08-01 +11 val_11 2009-08-01 +111 val_111 2009-08-01 +113 val_113 2009-08-01 +113 val_113 2009-08-01 +114 val_114 2009-08-01 +116 val_116 2009-08-01 +118 val_118 2009-08-01 +118 val_118 2009-08-01 +119 val_119 2009-08-01 +119 val_119 2009-08-01 +119 val_119 2009-08-01 +12 val_12 2009-08-01 +12 val_12 2009-08-01 +120 val_120 2009-08-01 +120 val_120 2009-08-01 +125 val_125 2009-08-01 +125 val_125 2009-08-01 +126 val_126 2009-08-01 +128 val_128 2009-08-01 +128 val_128 2009-08-01 +128 val_128 2009-08-01 +129 val_129 2009-08-01 +129 val_129 2009-08-01 +131 val_131 2009-08-01 +133 val_133 2009-08-01 +134 val_134 2009-08-01 +134 val_134 2009-08-01 +136 val_136 2009-08-01 +137 val_137 2009-08-01 +137 val_137 2009-08-01 +138 val_138 2009-08-01 +138 val_138 2009-08-01 +138 val_138 2009-08-01 +138 val_138 2009-08-01 +143 val_143 2009-08-01 +145 val_145 2009-08-01 +146 val_146 2009-08-01 +146 val_146 2009-08-01 +149 val_149 2009-08-01 +149 val_149 2009-08-01 +15 val_15 2009-08-01 +15 val_15 2009-08-01 +150 val_150 2009-08-01 +152 val_152 2009-08-01 +152 val_152 2009-08-01 +153 val_153 2009-08-01 +155 val_155 2009-08-01 +156 val_156 2009-08-01 +157 val_157 2009-08-01 +158 val_158 2009-08-01 +160 val_160 2009-08-01 +162 val_162 2009-08-01 +163 val_163 2009-08-01 +164 val_164 2009-08-01 +164 val_164 2009-08-01 +165 val_165 2009-08-01 +165 val_165 2009-08-01 +166 val_166 2009-08-01 +167 val_167 2009-08-01 +167 val_167 2009-08-01 +167 val_167 2009-08-01 +168 val_168 2009-08-01 +169 val_169 2009-08-01 +169 val_169 2009-08-01 +169 val_169 2009-08-01 +169 val_169 2009-08-01 +17 val_17 2009-08-01 +170 val_170 2009-08-01 +172 val_172 2009-08-01 +172 val_172 2009-08-01 +174 val_174 2009-08-01 +174 val_174 2009-08-01 +175 val_175 2009-08-01 +175 val_175 2009-08-01 +176 val_176 2009-08-01 +176 val_176 2009-08-01 +177 val_177 2009-08-01 +178 val_178 2009-08-01 +179 val_179 2009-08-01 +179 val_179 2009-08-01 +18 val_18 2009-08-01 +18 val_18 2009-08-01 +180 val_180 2009-08-01 +181 val_181 2009-08-01 +183 val_183 2009-08-01 +186 val_186 2009-08-01 +187 val_187 2009-08-01 +187 val_187 2009-08-01 +187 val_187 2009-08-01 +189 val_189 2009-08-01 +19 val_19 2009-08-01 +190 val_190 2009-08-01 +191 val_191 2009-08-01 +191 val_191 2009-08-01 +192 val_192 2009-08-01 +193 val_193 2009-08-01 +193 val_193 2009-08-01 +193 val_193 2009-08-01 +194 val_194 2009-08-01 +195 val_195 2009-08-01 +195 val_195 2009-08-01 +196 val_196 2009-08-01 +197 val_197 2009-08-01 +197 val_197 2009-08-01 +199 val_199 2009-08-01 +199 val_199 2009-08-01 +199 val_199 2009-08-01 +2 val_2 2009-08-01 +20 val_20 2009-08-01 +200 val_200 2009-08-01 +200 val_200 2009-08-01 +201 val_201 2009-08-01 +202 val_202 2009-08-01 +203 val_203 2009-08-01 +203 val_203 2009-08-01 +205 val_205 2009-08-01 +205 val_205 2009-08-01 +207 val_207 2009-08-01 +207 val_207 2009-08-01 +208 val_208 2009-08-01 +208 val_208 2009-08-01 +208 val_208 2009-08-01 +209 val_209 2009-08-01 +209 val_209 2009-08-01 +213 val_213 2009-08-01 +213 val_213 2009-08-01 +214 val_214 2009-08-01 +216 val_216 2009-08-01 +216 val_216 2009-08-01 +217 val_217 2009-08-01 +217 val_217 2009-08-01 +218 val_218 2009-08-01 +219 val_219 2009-08-01 +219 val_219 2009-08-01 +221 val_221 2009-08-01 +221 val_221 2009-08-01 +222 val_222 2009-08-01 +223 val_223 2009-08-01 +223 val_223 2009-08-01 +224 val_224 2009-08-01 +224 val_224 2009-08-01 +226 val_226 2009-08-01 +228 val_228 2009-08-01 +229 val_229 2009-08-01 +229 val_229 2009-08-01 +230 val_230 2009-08-01 +230 val_230 2009-08-01 +230 val_230 2009-08-01 +230 val_230 2009-08-01 +230 val_230 2009-08-01 +233 val_233 2009-08-01 +233 val_233 2009-08-01 +235 val_235 2009-08-01 +237 val_237 2009-08-01 +237 val_237 2009-08-01 +238 val_238 2009-08-01 +238 val_238 2009-08-01 +239 val_239 2009-08-01 +239 val_239 2009-08-01 +24 val_24 2009-08-01 +24 val_24 2009-08-01 +241 val_241 2009-08-01 +242 val_242 2009-08-01 +242 val_242 2009-08-01 +244 val_244 2009-08-01 +247 val_247 2009-08-01 +248 val_248 2009-08-01 +249 val_249 2009-08-01 +252 val_252 2009-08-01 +255 val_255 2009-08-01 +255 val_255 2009-08-01 +256 val_256 2009-08-01 +256 val_256 2009-08-01 +257 val_257 2009-08-01 +258 val_258 2009-08-01 +26 val_26 2009-08-01 +26 val_26 2009-08-01 +260 val_260 2009-08-01 +262 val_262 2009-08-01 +263 val_263 2009-08-01 +265 val_265 2009-08-01 +265 val_265 2009-08-01 +266 val_266 2009-08-01 +27 val_27 2009-08-01 +272 val_272 2009-08-01 +272 val_272 2009-08-01 +273 val_273 2009-08-01 +273 val_273 2009-08-01 +273 val_273 2009-08-01 +274 val_274 2009-08-01 +275 val_275 2009-08-01 +277 val_277 2009-08-01 +277 val_277 2009-08-01 +277 val_277 2009-08-01 +277 val_277 2009-08-01 +278 val_278 2009-08-01 +278 val_278 2009-08-01 +28 val_28 2009-08-01 +280 val_280 2009-08-01 +280 val_280 2009-08-01 +281 val_281 2009-08-01 +281 val_281 2009-08-01 +282 val_282 2009-08-01 +282 val_282 2009-08-01 +283 val_283 2009-08-01 +284 val_284 2009-08-01 +285 val_285 2009-08-01 +286 val_286 2009-08-01 +287 val_287 2009-08-01 +288 val_288 2009-08-01 +288 val_288 2009-08-01 +289 val_289 2009-08-01 +291 val_291 2009-08-01 +292 val_292 2009-08-01 +296 val_296 2009-08-01 +298 val_298 2009-08-01 +298 val_298 2009-08-01 +298 val_298 2009-08-01 +30 val_30 2009-08-01 +302 val_302 2009-08-01 +305 val_305 2009-08-01 +306 val_306 2009-08-01 +307 val_307 2009-08-01 +307 val_307 2009-08-01 +308 val_308 2009-08-01 +309 val_309 2009-08-01 +309 val_309 2009-08-01 +310 val_310 2009-08-01 +311 val_311 2009-08-01 +311 val_311 2009-08-01 +311 val_311 2009-08-01 +315 val_315 2009-08-01 +316 val_316 2009-08-01 +316 val_316 2009-08-01 +316 val_316 2009-08-01 +317 val_317 2009-08-01 +317 val_317 2009-08-01 +318 val_318 2009-08-01 +318 val_318 2009-08-01 +318 val_318 2009-08-01 +321 val_321 2009-08-01 +321 val_321 2009-08-01 +322 val_322 2009-08-01 +322 val_322 2009-08-01 +323 val_323 2009-08-01 +325 val_325 2009-08-01 +325 val_325 2009-08-01 +327 val_327 2009-08-01 +327 val_327 2009-08-01 +327 val_327 2009-08-01 +33 val_33 2009-08-01 +331 val_331 2009-08-01 +331 val_331 2009-08-01 +332 val_332 2009-08-01 +333 val_333 2009-08-01 +333 val_333 2009-08-01 +335 val_335 2009-08-01 +336 val_336 2009-08-01 +338 val_338 2009-08-01 +339 val_339 2009-08-01 +34 val_34 2009-08-01 +341 val_341 2009-08-01 +342 val_342 2009-08-01 +342 val_342 2009-08-01 +344 val_344 2009-08-01 +344 val_344 2009-08-01 +345 val_345 2009-08-01 +348 val_348 2009-08-01 +348 val_348 2009-08-01 +348 val_348 2009-08-01 +348 val_348 2009-08-01 +348 val_348 2009-08-01 +35 val_35 2009-08-01 +35 val_35 2009-08-01 +35 val_35 2009-08-01 +351 val_351 2009-08-01 +353 val_353 2009-08-01 +353 val_353 2009-08-01 +356 val_356 2009-08-01 +360 val_360 2009-08-01 +362 val_362 2009-08-01 +364 val_364 2009-08-01 +365 val_365 2009-08-01 +366 val_366 2009-08-01 +367 val_367 2009-08-01 +367 val_367 2009-08-01 +368 val_368 2009-08-01 +369 val_369 2009-08-01 +369 val_369 2009-08-01 +369 val_369 2009-08-01 +37 val_37 2009-08-01 +37 val_37 2009-08-01 +373 val_373 2009-08-01 +374 val_374 2009-08-01 +375 val_375 2009-08-01 +377 val_377 2009-08-01 +378 val_378 2009-08-01 +379 val_379 2009-08-01 +382 val_382 2009-08-01 +382 val_382 2009-08-01 +384 val_384 2009-08-01 +384 val_384 2009-08-01 +384 val_384 2009-08-01 +386 val_386 2009-08-01 +389 val_389 2009-08-01 +392 val_392 2009-08-01 +393 val_393 2009-08-01 +394 val_394 2009-08-01 +395 val_395 2009-08-01 +395 val_395 2009-08-01 +396 val_396 2009-08-01 +396 val_396 2009-08-01 +396 val_396 2009-08-01 +397 val_397 2009-08-01 +397 val_397 2009-08-01 +399 val_399 2009-08-01 +399 val_399 2009-08-01 +4 val_4 2009-08-01 +400 val_400 2009-08-01 +401 val_401 2009-08-01 +401 val_401 2009-08-01 +401 val_401 2009-08-01 +401 val_401 2009-08-01 +401 val_401 2009-08-01 +402 val_402 2009-08-01 +403 val_403 2009-08-01 +403 val_403 2009-08-01 +403 val_403 2009-08-01 +404 val_404 2009-08-01 +404 val_404 2009-08-01 +406 val_406 2009-08-01 +406 val_406 2009-08-01 +406 val_406 2009-08-01 +406 val_406 2009-08-01 +407 val_407 2009-08-01 +409 val_409 2009-08-01 +409 val_409 2009-08-01 +409 val_409 2009-08-01 +41 val_41 2009-08-01 +411 val_411 2009-08-01 +413 val_413 2009-08-01 +413 val_413 2009-08-01 +414 val_414 2009-08-01 +414 val_414 2009-08-01 +417 val_417 2009-08-01 +417 val_417 2009-08-01 +417 val_417 2009-08-01 +418 val_418 2009-08-01 +419 val_419 2009-08-01 +42 val_42 2009-08-01 +42 val_42 2009-08-01 +421 val_421 2009-08-01 +424 val_424 2009-08-01 +424 val_424 2009-08-01 +427 val_427 2009-08-01 +429 val_429 2009-08-01 +429 val_429 2009-08-01 +43 val_43 2009-08-01 +430 val_430 2009-08-01 +430 val_430 2009-08-01 +430 val_430 2009-08-01 +431 val_431 2009-08-01 +431 val_431 2009-08-01 +431 val_431 2009-08-01 +432 val_432 2009-08-01 +435 val_435 2009-08-01 +436 val_436 2009-08-01 +437 val_437 2009-08-01 +438 val_438 2009-08-01 +438 val_438 2009-08-01 +438 val_438 2009-08-01 +439 val_439 2009-08-01 +439 val_439 2009-08-01 +44 val_44 2009-08-01 +443 val_443 2009-08-01 +444 val_444 2009-08-01 +446 val_446 2009-08-01 +448 val_448 2009-08-01 +449 val_449 2009-08-01 +452 val_452 2009-08-01 +453 val_453 2009-08-01 +454 val_454 2009-08-01 +454 val_454 2009-08-01 +454 val_454 2009-08-01 +455 val_455 2009-08-01 +457 val_457 2009-08-01 +458 val_458 2009-08-01 +458 val_458 2009-08-01 +459 val_459 2009-08-01 +459 val_459 2009-08-01 +460 val_460 2009-08-01 +462 val_462 2009-08-01 +462 val_462 2009-08-01 +463 val_463 2009-08-01 +463 val_463 2009-08-01 +466 val_466 2009-08-01 +466 val_466 2009-08-01 +466 val_466 2009-08-01 +467 val_467 2009-08-01 +468 val_468 2009-08-01 +468 val_468 2009-08-01 +468 val_468 2009-08-01 +468 val_468 2009-08-01 +469 val_469 2009-08-01 +469 val_469 2009-08-01 +469 val_469 2009-08-01 +469 val_469 2009-08-01 +469 val_469 2009-08-01 +47 val_47 2009-08-01 +470 val_470 2009-08-01 +472 val_472 2009-08-01 +475 val_475 2009-08-01 +477 val_477 2009-08-01 +478 val_478 2009-08-01 +478 val_478 2009-08-01 +479 val_479 2009-08-01 +480 val_480 2009-08-01 +480 val_480 2009-08-01 +480 val_480 2009-08-01 +481 val_481 2009-08-01 +482 val_482 2009-08-01 +483 val_483 2009-08-01 +484 val_484 2009-08-01 +485 val_485 2009-08-01 +487 val_487 2009-08-01 +489 val_489 2009-08-01 +489 val_489 2009-08-01 +489 val_489 2009-08-01 +489 val_489 2009-08-01 +490 val_490 2009-08-01 +491 val_491 2009-08-01 +492 val_492 2009-08-01 +492 val_492 2009-08-01 +493 val_493 2009-08-01 +494 val_494 2009-08-01 +495 val_495 2009-08-01 +496 val_496 2009-08-01 +497 val_497 2009-08-01 +498 val_498 2009-08-01 +498 val_498 2009-08-01 +498 val_498 2009-08-01 +5 val_5 2009-08-01 +5 val_5 2009-08-01 +5 val_5 2009-08-01 +51 val_51 2009-08-01 +51 val_51 2009-08-01 +53 val_53 2009-08-01 +54 val_54 2009-08-01 +57 val_57 2009-08-01 +58 val_58 2009-08-01 +58 val_58 2009-08-01 +64 val_64 2009-08-01 +65 val_65 2009-08-01 +66 val_66 2009-08-01 +67 val_67 2009-08-01 +67 val_67 2009-08-01 +69 val_69 2009-08-01 +70 val_70 2009-08-01 +70 val_70 2009-08-01 +70 val_70 2009-08-01 +72 val_72 2009-08-01 +72 val_72 2009-08-01 +74 val_74 2009-08-01 +76 val_76 2009-08-01 +76 val_76 2009-08-01 +77 val_77 2009-08-01 +78 val_78 2009-08-01 +8 val_8 2009-08-01 +80 val_80 2009-08-01 +82 val_82 2009-08-01 +83 val_83 2009-08-01 +83 val_83 2009-08-01 +84 val_84 2009-08-01 +84 val_84 2009-08-01 +85 val_85 2009-08-01 +86 val_86 2009-08-01 +87 val_87 2009-08-01 +9 val_9 2009-08-01 +90 val_90 2009-08-01 +90 val_90 2009-08-01 +90 val_90 2009-08-01 +92 val_92 2009-08-01 +95 val_95 2009-08-01 +95 val_95 2009-08-01 +96 val_96 2009-08-01 +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 diff --git a/src/test/hive/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 b/src/test/hive/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 b/src/test/hive/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 new file mode 100644 index 0000000000000..ee3dddd8cff71 --- /dev/null +++ b/src/test/hive/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 @@ -0,0 +1,1000 @@ +0 val_0 2009-08-01 +0 val_0 2009-08-01 +0 val_0 2009-08-01 +0 val_1 2009-08-01 +0 val_1 2009-08-01 +1 val_2 2009-08-01 +10 val_10 2009-08-01 +10 val_11 2009-08-01 +100 val_100 2009-08-01 +100 val_100 2009-08-01 +100 val_101 2009-08-01 +100 val_101 2009-08-01 +101 val_102 2009-08-01 +102 val_103 2009-08-01 +103 val_103 2009-08-01 +103 val_103 2009-08-01 +104 val_104 2009-08-01 +104 val_104 2009-08-01 +104 val_105 2009-08-01 +104 val_105 2009-08-01 +104 val_105 2009-08-01 +105 val_105 2009-08-01 +105 val_106 2009-08-01 +105 val_106 2009-08-01 +106 val_107 2009-08-01 +11 val_11 2009-08-01 +11 val_12 2009-08-01 +11 val_12 2009-08-01 +11 val_12 2009-08-01 +110 val_111 2009-08-01 +111 val_111 2009-08-01 +113 val_113 2009-08-01 +113 val_113 2009-08-01 +114 val_114 2009-08-01 +114 val_115 2009-08-01 +114 val_115 2009-08-01 +114 val_115 2009-08-01 +116 val_116 2009-08-01 +116 val_117 2009-08-01 +117 val_118 2009-08-01 +117 val_118 2009-08-01 +118 val_118 2009-08-01 +118 val_118 2009-08-01 +118 val_119 2009-08-01 +118 val_119 2009-08-01 +118 val_119 2009-08-01 +119 val_119 2009-08-01 +119 val_119 2009-08-01 +119 val_119 2009-08-01 +119 val_120 2009-08-01 +119 val_120 2009-08-01 +119 val_120 2009-08-01 +12 val_12 2009-08-01 +12 val_12 2009-08-01 +12 val_13 2009-08-01 +120 val_120 2009-08-01 +120 val_120 2009-08-01 +120 val_121 2009-08-01 +121 val_122 2009-08-01 +121 val_122 2009-08-01 +122 val_123 2009-08-01 +122 val_123 2009-08-01 +122 val_123 2009-08-01 +123 val_124 2009-08-01 +123 val_124 2009-08-01 +125 val_125 2009-08-01 +125 val_125 2009-08-01 +125 val_126 2009-08-01 +126 val_126 2009-08-01 +126 val_127 2009-08-01 +126 val_127 2009-08-01 +128 val_128 2009-08-01 +128 val_128 2009-08-01 +128 val_128 2009-08-01 +128 val_129 2009-08-01 +128 val_129 2009-08-01 +129 val_129 2009-08-01 +129 val_129 2009-08-01 +129 val_130 2009-08-01 +129 val_130 2009-08-01 +131 val_131 2009-08-01 +132 val_133 2009-08-01 +132 val_133 2009-08-01 +133 val_133 2009-08-01 +133 val_134 2009-08-01 +134 val_134 2009-08-01 +134 val_134 2009-08-01 +134 val_135 2009-08-01 +135 val_136 2009-08-01 +135 val_136 2009-08-01 +135 val_136 2009-08-01 +136 val_136 2009-08-01 +136 val_137 2009-08-01 +137 val_137 2009-08-01 +137 val_137 2009-08-01 +137 val_138 2009-08-01 +138 val_138 2009-08-01 +138 val_138 2009-08-01 +138 val_138 2009-08-01 +138 val_138 2009-08-01 +138 val_139 2009-08-01 +138 val_139 2009-08-01 +140 val_141 2009-08-01 +143 val_143 2009-08-01 +143 val_144 2009-08-01 +144 val_145 2009-08-01 +145 val_145 2009-08-01 +146 val_146 2009-08-01 +146 val_146 2009-08-01 +147 val_148 2009-08-01 +147 val_148 2009-08-01 +149 val_149 2009-08-01 +149 val_149 2009-08-01 +149 val_150 2009-08-01 +15 val_15 2009-08-01 +15 val_15 2009-08-01 +15 val_16 2009-08-01 +15 val_16 2009-08-01 +150 val_150 2009-08-01 +151 val_152 2009-08-01 +151 val_152 2009-08-01 +152 val_152 2009-08-01 +152 val_152 2009-08-01 +152 val_153 2009-08-01 +152 val_153 2009-08-01 +152 val_153 2009-08-01 +153 val_153 2009-08-01 +153 val_154 2009-08-01 +153 val_154 2009-08-01 +155 val_155 2009-08-01 +156 val_156 2009-08-01 +156 val_157 2009-08-01 +156 val_157 2009-08-01 +157 val_157 2009-08-01 +157 val_158 2009-08-01 +157 val_158 2009-08-01 +158 val_158 2009-08-01 +16 val_17 2009-08-01 +16 val_17 2009-08-01 +160 val_160 2009-08-01 +160 val_161 2009-08-01 +161 val_162 2009-08-01 +161 val_162 2009-08-01 +161 val_162 2009-08-01 +161 val_162 2009-08-01 +162 val_162 2009-08-01 +162 val_163 2009-08-01 +163 val_163 2009-08-01 +164 val_164 2009-08-01 +164 val_164 2009-08-01 +164 val_165 2009-08-01 +164 val_165 2009-08-01 +165 val_165 2009-08-01 +165 val_165 2009-08-01 +165 val_166 2009-08-01 +166 val_166 2009-08-01 +167 val_167 2009-08-01 +167 val_167 2009-08-01 +167 val_167 2009-08-01 +167 val_168 2009-08-01 +168 val_168 2009-08-01 +168 val_169 2009-08-01 +169 val_169 2009-08-01 +169 val_169 2009-08-01 +169 val_169 2009-08-01 +169 val_169 2009-08-01 +17 val_17 2009-08-01 +170 val_170 2009-08-01 +170 val_171 2009-08-01 +172 val_172 2009-08-01 +172 val_172 2009-08-01 +172 val_173 2009-08-01 +174 val_174 2009-08-01 +174 val_174 2009-08-01 +174 val_175 2009-08-01 +174 val_175 2009-08-01 +175 val_175 2009-08-01 +175 val_175 2009-08-01 +175 val_176 2009-08-01 +175 val_176 2009-08-01 +176 val_176 2009-08-01 +176 val_176 2009-08-01 +177 val_177 2009-08-01 +177 val_178 2009-08-01 +177 val_178 2009-08-01 +178 val_178 2009-08-01 +178 val_179 2009-08-01 +178 val_179 2009-08-01 +179 val_179 2009-08-01 +179 val_179 2009-08-01 +179 val_180 2009-08-01 +18 val_18 2009-08-01 +18 val_18 2009-08-01 +180 val_180 2009-08-01 +181 val_181 2009-08-01 +182 val_183 2009-08-01 +183 val_183 2009-08-01 +183 val_184 2009-08-01 +184 val_185 2009-08-01 +185 val_186 2009-08-01 +186 val_186 2009-08-01 +187 val_187 2009-08-01 +187 val_187 2009-08-01 +187 val_187 2009-08-01 +189 val_189 2009-08-01 +189 val_190 2009-08-01 +19 val_19 2009-08-01 +19 val_20 2009-08-01 +190 val_190 2009-08-01 +191 val_191 2009-08-01 +191 val_191 2009-08-01 +191 val_192 2009-08-01 +192 val_192 2009-08-01 +192 val_193 2009-08-01 +193 val_193 2009-08-01 +193 val_193 2009-08-01 +193 val_193 2009-08-01 +194 val_194 2009-08-01 +195 val_195 2009-08-01 +195 val_195 2009-08-01 +196 val_196 2009-08-01 +196 val_197 2009-08-01 +196 val_197 2009-08-01 +196 val_197 2009-08-01 +197 val_197 2009-08-01 +197 val_197 2009-08-01 +197 val_198 2009-08-01 +199 val_199 2009-08-01 +199 val_199 2009-08-01 +199 val_199 2009-08-01 +199 val_200 2009-08-01 +2 val_2 2009-08-01 +2 val_3 2009-08-01 +20 val_20 2009-08-01 +20 val_21 2009-08-01 +20 val_21 2009-08-01 +200 val_200 2009-08-01 +200 val_200 2009-08-01 +201 val_201 2009-08-01 +202 val_202 2009-08-01 +203 val_203 2009-08-01 +203 val_203 2009-08-01 +204 val_205 2009-08-01 +205 val_205 2009-08-01 +205 val_205 2009-08-01 +205 val_206 2009-08-01 +206 val_207 2009-08-01 +206 val_207 2009-08-01 +206 val_207 2009-08-01 +207 val_207 2009-08-01 +207 val_207 2009-08-01 +208 val_208 2009-08-01 +208 val_208 2009-08-01 +208 val_208 2009-08-01 +209 val_209 2009-08-01 +209 val_209 2009-08-01 +209 val_210 2009-08-01 +209 val_210 2009-08-01 +21 val_22 2009-08-01 +21 val_22 2009-08-01 +21 val_22 2009-08-01 +21 val_22 2009-08-01 +212 val_213 2009-08-01 +213 val_213 2009-08-01 +213 val_213 2009-08-01 +213 val_214 2009-08-01 +214 val_214 2009-08-01 +215 val_216 2009-08-01 +216 val_216 2009-08-01 +216 val_216 2009-08-01 +216 val_217 2009-08-01 +217 val_217 2009-08-01 +217 val_217 2009-08-01 +217 val_218 2009-08-01 +217 val_218 2009-08-01 +218 val_218 2009-08-01 +219 val_219 2009-08-01 +219 val_219 2009-08-01 +22 val_23 2009-08-01 +221 val_221 2009-08-01 +221 val_221 2009-08-01 +222 val_222 2009-08-01 +222 val_223 2009-08-01 +223 val_223 2009-08-01 +223 val_223 2009-08-01 +224 val_224 2009-08-01 +224 val_224 2009-08-01 +224 val_225 2009-08-01 +226 val_226 2009-08-01 +226 val_227 2009-08-01 +226 val_227 2009-08-01 +226 val_227 2009-08-01 +226 val_227 2009-08-01 +227 val_228 2009-08-01 +228 val_228 2009-08-01 +228 val_229 2009-08-01 +229 val_229 2009-08-01 +229 val_229 2009-08-01 +23 val_24 2009-08-01 +230 val_230 2009-08-01 +230 val_230 2009-08-01 +230 val_230 2009-08-01 +230 val_230 2009-08-01 +230 val_230 2009-08-01 +231 val_232 2009-08-01 +233 val_233 2009-08-01 +233 val_233 2009-08-01 +235 val_235 2009-08-01 +235 val_236 2009-08-01 +237 val_237 2009-08-01 +237 val_237 2009-08-01 +238 val_238 2009-08-01 +238 val_238 2009-08-01 +238 val_239 2009-08-01 +239 val_239 2009-08-01 +239 val_239 2009-08-01 +239 val_240 2009-08-01 +239 val_240 2009-08-01 +24 val_24 2009-08-01 +24 val_24 2009-08-01 +240 val_241 2009-08-01 +241 val_241 2009-08-01 +241 val_242 2009-08-01 +241 val_242 2009-08-01 +241 val_242 2009-08-01 +241 val_242 2009-08-01 +242 val_242 2009-08-01 +242 val_242 2009-08-01 +242 val_243 2009-08-01 +243 val_244 2009-08-01 +243 val_244 2009-08-01 +244 val_244 2009-08-01 +244 val_245 2009-08-01 +244 val_245 2009-08-01 +244 val_245 2009-08-01 +245 val_246 2009-08-01 +245 val_246 2009-08-01 +246 val_247 2009-08-01 +246 val_247 2009-08-01 +247 val_247 2009-08-01 +248 val_248 2009-08-01 +248 val_249 2009-08-01 +249 val_249 2009-08-01 +249 val_250 2009-08-01 +249 val_250 2009-08-01 +252 val_252 2009-08-01 +252 val_253 2009-08-01 +254 val_255 2009-08-01 +255 val_255 2009-08-01 +255 val_255 2009-08-01 +256 val_256 2009-08-01 +256 val_256 2009-08-01 +256 val_257 2009-08-01 +257 val_257 2009-08-01 +257 val_258 2009-08-01 +257 val_258 2009-08-01 +258 val_258 2009-08-01 +258 val_259 2009-08-01 +259 val_260 2009-08-01 +259 val_260 2009-08-01 +26 val_26 2009-08-01 +26 val_26 2009-08-01 +260 val_260 2009-08-01 +260 val_261 2009-08-01 +260 val_261 2009-08-01 +261 val_262 2009-08-01 +262 val_262 2009-08-01 +262 val_263 2009-08-01 +262 val_263 2009-08-01 +263 val_263 2009-08-01 +264 val_265 2009-08-01 +264 val_265 2009-08-01 +265 val_265 2009-08-01 +265 val_265 2009-08-01 +265 val_266 2009-08-01 +266 val_266 2009-08-01 +267 val_268 2009-08-01 +268 val_269 2009-08-01 +27 val_27 2009-08-01 +271 val_272 2009-08-01 +272 val_272 2009-08-01 +272 val_272 2009-08-01 +272 val_273 2009-08-01 +273 val_273 2009-08-01 +273 val_273 2009-08-01 +273 val_273 2009-08-01 +273 val_274 2009-08-01 +274 val_274 2009-08-01 +274 val_275 2009-08-01 +275 val_275 2009-08-01 +275 val_276 2009-08-01 +275 val_276 2009-08-01 +276 val_277 2009-08-01 +277 val_277 2009-08-01 +277 val_277 2009-08-01 +277 val_277 2009-08-01 +277 val_277 2009-08-01 +277 val_278 2009-08-01 +277 val_278 2009-08-01 +278 val_278 2009-08-01 +278 val_278 2009-08-01 +278 val_279 2009-08-01 +28 val_28 2009-08-01 +280 val_280 2009-08-01 +280 val_280 2009-08-01 +281 val_281 2009-08-01 +281 val_281 2009-08-01 +281 val_282 2009-08-01 +281 val_282 2009-08-01 +281 val_282 2009-08-01 +282 val_282 2009-08-01 +282 val_282 2009-08-01 +283 val_283 2009-08-01 +284 val_284 2009-08-01 +284 val_285 2009-08-01 +284 val_285 2009-08-01 +285 val_285 2009-08-01 +285 val_286 2009-08-01 +286 val_286 2009-08-01 +286 val_287 2009-08-01 +287 val_287 2009-08-01 +287 val_288 2009-08-01 +287 val_288 2009-08-01 +288 val_288 2009-08-01 +288 val_288 2009-08-01 +289 val_289 2009-08-01 +289 val_290 2009-08-01 +29 val_30 2009-08-01 +29 val_30 2009-08-01 +291 val_291 2009-08-01 +291 val_292 2009-08-01 +291 val_292 2009-08-01 +292 val_292 2009-08-01 +292 val_293 2009-08-01 +292 val_293 2009-08-01 +293 val_294 2009-08-01 +293 val_294 2009-08-01 +295 val_296 2009-08-01 +295 val_296 2009-08-01 +296 val_296 2009-08-01 +296 val_297 2009-08-01 +298 val_298 2009-08-01 +298 val_298 2009-08-01 +298 val_298 2009-08-01 +3 val_4 2009-08-01 +30 val_30 2009-08-01 +30 val_31 2009-08-01 +300 val_301 2009-08-01 +300 val_301 2009-08-01 +302 val_302 2009-08-01 +302 val_303 2009-08-01 +303 val_304 2009-08-01 +303 val_304 2009-08-01 +304 val_305 2009-08-01 +305 val_305 2009-08-01 +305 val_306 2009-08-01 +306 val_306 2009-08-01 +306 val_307 2009-08-01 +307 val_307 2009-08-01 +307 val_307 2009-08-01 +308 val_308 2009-08-01 +308 val_309 2009-08-01 +308 val_309 2009-08-01 +309 val_309 2009-08-01 +309 val_309 2009-08-01 +309 val_310 2009-08-01 +31 val_32 2009-08-01 +310 val_310 2009-08-01 +310 val_311 2009-08-01 +310 val_311 2009-08-01 +310 val_311 2009-08-01 +311 val_311 2009-08-01 +311 val_311 2009-08-01 +311 val_311 2009-08-01 +313 val_314 2009-08-01 +314 val_315 2009-08-01 +315 val_315 2009-08-01 +316 val_316 2009-08-01 +316 val_316 2009-08-01 +316 val_316 2009-08-01 +317 val_317 2009-08-01 +317 val_317 2009-08-01 +317 val_318 2009-08-01 +318 val_318 2009-08-01 +318 val_318 2009-08-01 +318 val_318 2009-08-01 +318 val_319 2009-08-01 +32 val_33 2009-08-01 +321 val_321 2009-08-01 +321 val_321 2009-08-01 +322 val_322 2009-08-01 +322 val_322 2009-08-01 +322 val_323 2009-08-01 +323 val_323 2009-08-01 +323 val_324 2009-08-01 +324 val_325 2009-08-01 +325 val_325 2009-08-01 +325 val_325 2009-08-01 +326 val_327 2009-08-01 +327 val_327 2009-08-01 +327 val_327 2009-08-01 +327 val_327 2009-08-01 +328 val_329 2009-08-01 +328 val_329 2009-08-01 +33 val_33 2009-08-01 +33 val_34 2009-08-01 +330 val_331 2009-08-01 +331 val_331 2009-08-01 +331 val_331 2009-08-01 +331 val_332 2009-08-01 +331 val_332 2009-08-01 +332 val_332 2009-08-01 +333 val_333 2009-08-01 +333 val_333 2009-08-01 +333 val_334 2009-08-01 +334 val_335 2009-08-01 +335 val_335 2009-08-01 +335 val_336 2009-08-01 +335 val_336 2009-08-01 +336 val_336 2009-08-01 +336 val_337 2009-08-01 +337 val_338 2009-08-01 +338 val_338 2009-08-01 +338 val_339 2009-08-01 +339 val_339 2009-08-01 +34 val_34 2009-08-01 +340 val_341 2009-08-01 +341 val_341 2009-08-01 +341 val_342 2009-08-01 +341 val_342 2009-08-01 +341 val_342 2009-08-01 +342 val_342 2009-08-01 +342 val_342 2009-08-01 +342 val_343 2009-08-01 +343 val_344 2009-08-01 +344 val_344 2009-08-01 +344 val_344 2009-08-01 +344 val_345 2009-08-01 +345 val_345 2009-08-01 +347 val_348 2009-08-01 +347 val_348 2009-08-01 +348 val_348 2009-08-01 +348 val_348 2009-08-01 +348 val_348 2009-08-01 +348 val_348 2009-08-01 +348 val_348 2009-08-01 +348 val_349 2009-08-01 +349 val_350 2009-08-01 +349 val_350 2009-08-01 +349 val_350 2009-08-01 +349 val_350 2009-08-01 +35 val_35 2009-08-01 +35 val_35 2009-08-01 +35 val_35 2009-08-01 +35 val_36 2009-08-01 +35 val_36 2009-08-01 +35 val_36 2009-08-01 +351 val_351 2009-08-01 +351 val_352 2009-08-01 +351 val_352 2009-08-01 +352 val_353 2009-08-01 +352 val_353 2009-08-01 +353 val_353 2009-08-01 +353 val_353 2009-08-01 +353 val_354 2009-08-01 +355 val_356 2009-08-01 +355 val_356 2009-08-01 +356 val_356 2009-08-01 +356 val_357 2009-08-01 +356 val_357 2009-08-01 +358 val_359 2009-08-01 +360 val_360 2009-08-01 +360 val_361 2009-08-01 +362 val_362 2009-08-01 +363 val_364 2009-08-01 +363 val_364 2009-08-01 +363 val_364 2009-08-01 +364 val_364 2009-08-01 +364 val_365 2009-08-01 +365 val_365 2009-08-01 +366 val_366 2009-08-01 +367 val_367 2009-08-01 +367 val_367 2009-08-01 +367 val_368 2009-08-01 +367 val_368 2009-08-01 +368 val_368 2009-08-01 +369 val_369 2009-08-01 +369 val_369 2009-08-01 +369 val_369 2009-08-01 +369 val_370 2009-08-01 +37 val_37 2009-08-01 +37 val_37 2009-08-01 +371 val_372 2009-08-01 +371 val_372 2009-08-01 +371 val_372 2009-08-01 +371 val_372 2009-08-01 +373 val_373 2009-08-01 +373 val_374 2009-08-01 +374 val_374 2009-08-01 +374 val_375 2009-08-01 +375 val_375 2009-08-01 +375 val_376 2009-08-01 +375 val_376 2009-08-01 +375 val_376 2009-08-01 +375 val_376 2009-08-01 +375 val_376 2009-08-01 +376 val_377 2009-08-01 +377 val_377 2009-08-01 +378 val_378 2009-08-01 +378 val_379 2009-08-01 +379 val_379 2009-08-01 +379 val_380 2009-08-01 +381 val_382 2009-08-01 +382 val_382 2009-08-01 +382 val_382 2009-08-01 +382 val_383 2009-08-01 +382 val_383 2009-08-01 +384 val_384 2009-08-01 +384 val_384 2009-08-01 +384 val_384 2009-08-01 +384 val_385 2009-08-01 +384 val_385 2009-08-01 +384 val_385 2009-08-01 +385 val_386 2009-08-01 +385 val_386 2009-08-01 +386 val_386 2009-08-01 +386 val_387 2009-08-01 +386 val_387 2009-08-01 +388 val_389 2009-08-01 +389 val_389 2009-08-01 +389 val_390 2009-08-01 +389 val_390 2009-08-01 +390 val_391 2009-08-01 +390 val_391 2009-08-01 +390 val_391 2009-08-01 +391 val_392 2009-08-01 +391 val_392 2009-08-01 +392 val_392 2009-08-01 +392 val_393 2009-08-01 +392 val_393 2009-08-01 +393 val_393 2009-08-01 +393 val_394 2009-08-01 +393 val_394 2009-08-01 +394 val_394 2009-08-01 +395 val_395 2009-08-01 +395 val_395 2009-08-01 +395 val_396 2009-08-01 +395 val_396 2009-08-01 +396 val_396 2009-08-01 +396 val_396 2009-08-01 +396 val_396 2009-08-01 +397 val_397 2009-08-01 +397 val_397 2009-08-01 +398 val_399 2009-08-01 +399 val_399 2009-08-01 +399 val_399 2009-08-01 +399 val_400 2009-08-01 +399 val_400 2009-08-01 +4 val_4 2009-08-01 +4 val_5 2009-08-01 +40 val_41 2009-08-01 +40 val_41 2009-08-01 +400 val_400 2009-08-01 +401 val_401 2009-08-01 +401 val_401 2009-08-01 +401 val_401 2009-08-01 +401 val_401 2009-08-01 +401 val_401 2009-08-01 +401 val_402 2009-08-01 +402 val_402 2009-08-01 +402 val_403 2009-08-01 +402 val_403 2009-08-01 +402 val_403 2009-08-01 +403 val_403 2009-08-01 +403 val_403 2009-08-01 +403 val_403 2009-08-01 +404 val_404 2009-08-01 +404 val_404 2009-08-01 +404 val_405 2009-08-01 +404 val_405 2009-08-01 +404 val_405 2009-08-01 +405 val_406 2009-08-01 +406 val_406 2009-08-01 +406 val_406 2009-08-01 +406 val_406 2009-08-01 +406 val_406 2009-08-01 +406 val_407 2009-08-01 +407 val_407 2009-08-01 +407 val_408 2009-08-01 +407 val_408 2009-08-01 +407 val_408 2009-08-01 +408 val_409 2009-08-01 +408 val_409 2009-08-01 +409 val_409 2009-08-01 +409 val_409 2009-08-01 +409 val_409 2009-08-01 +409 val_410 2009-08-01 +409 val_410 2009-08-01 +41 val_41 2009-08-01 +410 val_411 2009-08-01 +411 val_411 2009-08-01 +411 val_412 2009-08-01 +412 val_413 2009-08-01 +412 val_413 2009-08-01 +413 val_413 2009-08-01 +413 val_413 2009-08-01 +413 val_414 2009-08-01 +414 val_414 2009-08-01 +414 val_414 2009-08-01 +414 val_415 2009-08-01 +415 val_416 2009-08-01 +416 val_417 2009-08-01 +417 val_417 2009-08-01 +417 val_417 2009-08-01 +417 val_417 2009-08-01 +418 val_418 2009-08-01 +419 val_419 2009-08-01 +42 val_42 2009-08-01 +42 val_42 2009-08-01 +42 val_43 2009-08-01 +42 val_43 2009-08-01 +42 val_43 2009-08-01 +421 val_421 2009-08-01 +421 val_422 2009-08-01 +421 val_422 2009-08-01 +423 val_424 2009-08-01 +424 val_424 2009-08-01 +424 val_424 2009-08-01 +424 val_425 2009-08-01 +425 val_426 2009-08-01 +426 val_427 2009-08-01 +427 val_427 2009-08-01 +427 val_428 2009-08-01 +427 val_428 2009-08-01 +428 val_429 2009-08-01 +429 val_429 2009-08-01 +429 val_429 2009-08-01 +429 val_430 2009-08-01 +429 val_430 2009-08-01 +43 val_43 2009-08-01 +430 val_430 2009-08-01 +430 val_430 2009-08-01 +430 val_430 2009-08-01 +430 val_431 2009-08-01 +431 val_431 2009-08-01 +431 val_431 2009-08-01 +431 val_431 2009-08-01 +431 val_432 2009-08-01 +432 val_432 2009-08-01 +432 val_433 2009-08-01 +435 val_435 2009-08-01 +435 val_436 2009-08-01 +436 val_436 2009-08-01 +436 val_437 2009-08-01 +437 val_437 2009-08-01 +437 val_438 2009-08-01 +438 val_438 2009-08-01 +438 val_438 2009-08-01 +438 val_438 2009-08-01 +438 val_439 2009-08-01 +438 val_439 2009-08-01 +439 val_439 2009-08-01 +439 val_439 2009-08-01 +439 val_440 2009-08-01 +439 val_440 2009-08-01 +44 val_44 2009-08-01 +440 val_441 2009-08-01 +440 val_441 2009-08-01 +441 val_442 2009-08-01 +442 val_443 2009-08-01 +443 val_443 2009-08-01 +443 val_444 2009-08-01 +443 val_444 2009-08-01 +443 val_444 2009-08-01 +444 val_444 2009-08-01 +446 val_446 2009-08-01 +446 val_447 2009-08-01 +446 val_447 2009-08-01 +447 val_448 2009-08-01 +448 val_448 2009-08-01 +448 val_449 2009-08-01 +449 val_449 2009-08-01 +450 val_451 2009-08-01 +450 val_451 2009-08-01 +451 val_452 2009-08-01 +452 val_452 2009-08-01 +453 val_453 2009-08-01 +453 val_454 2009-08-01 +454 val_454 2009-08-01 +454 val_454 2009-08-01 +454 val_454 2009-08-01 +454 val_455 2009-08-01 +454 val_455 2009-08-01 +455 val_455 2009-08-01 +455 val_456 2009-08-01 +455 val_456 2009-08-01 +457 val_457 2009-08-01 +457 val_458 2009-08-01 +457 val_458 2009-08-01 +458 val_458 2009-08-01 +458 val_458 2009-08-01 +459 val_459 2009-08-01 +459 val_459 2009-08-01 +459 val_460 2009-08-01 +46 val_47 2009-08-01 +460 val_460 2009-08-01 +461 val_462 2009-08-01 +462 val_462 2009-08-01 +462 val_462 2009-08-01 +462 val_463 2009-08-01 +463 val_463 2009-08-01 +463 val_463 2009-08-01 +463 val_464 2009-08-01 +466 val_466 2009-08-01 +466 val_466 2009-08-01 +466 val_466 2009-08-01 +467 val_467 2009-08-01 +467 val_468 2009-08-01 +468 val_468 2009-08-01 +468 val_468 2009-08-01 +468 val_468 2009-08-01 +468 val_468 2009-08-01 +468 val_469 2009-08-01 +468 val_469 2009-08-01 +468 val_469 2009-08-01 +469 val_469 2009-08-01 +469 val_469 2009-08-01 +469 val_469 2009-08-01 +469 val_469 2009-08-01 +469 val_469 2009-08-01 +469 val_470 2009-08-01 +47 val_47 2009-08-01 +47 val_48 2009-08-01 +470 val_470 2009-08-01 +470 val_471 2009-08-01 +472 val_472 2009-08-01 +473 val_474 2009-08-01 +474 val_475 2009-08-01 +474 val_475 2009-08-01 +475 val_475 2009-08-01 +475 val_476 2009-08-01 +476 val_477 2009-08-01 +476 val_477 2009-08-01 +477 val_477 2009-08-01 +477 val_478 2009-08-01 +478 val_478 2009-08-01 +478 val_478 2009-08-01 +478 val_479 2009-08-01 +478 val_479 2009-08-01 +479 val_479 2009-08-01 +48 val_49 2009-08-01 +48 val_49 2009-08-01 +480 val_480 2009-08-01 +480 val_480 2009-08-01 +480 val_480 2009-08-01 +480 val_481 2009-08-01 +480 val_481 2009-08-01 +481 val_481 2009-08-01 +481 val_482 2009-08-01 +482 val_482 2009-08-01 +482 val_483 2009-08-01 +483 val_483 2009-08-01 +484 val_484 2009-08-01 +484 val_485 2009-08-01 +485 val_485 2009-08-01 +485 val_486 2009-08-01 +485 val_486 2009-08-01 +486 val_487 2009-08-01 +487 val_487 2009-08-01 +487 val_488 2009-08-01 +488 val_489 2009-08-01 +489 val_489 2009-08-01 +489 val_489 2009-08-01 +489 val_489 2009-08-01 +489 val_489 2009-08-01 +489 val_490 2009-08-01 +49 val_50 2009-08-01 +49 val_50 2009-08-01 +490 val_490 2009-08-01 +490 val_491 2009-08-01 +491 val_491 2009-08-01 +491 val_492 2009-08-01 +491 val_492 2009-08-01 +492 val_492 2009-08-01 +492 val_492 2009-08-01 +492 val_493 2009-08-01 +492 val_493 2009-08-01 +493 val_493 2009-08-01 +494 val_494 2009-08-01 +494 val_495 2009-08-01 +494 val_495 2009-08-01 +495 val_495 2009-08-01 +495 val_496 2009-08-01 +496 val_496 2009-08-01 +496 val_497 2009-08-01 +497 val_497 2009-08-01 +497 val_498 2009-08-01 +497 val_498 2009-08-01 +498 val_498 2009-08-01 +498 val_498 2009-08-01 +498 val_498 2009-08-01 +5 val_5 2009-08-01 +5 val_5 2009-08-01 +5 val_5 2009-08-01 +5 val_6 2009-08-01 +50 val_51 2009-08-01 +51 val_51 2009-08-01 +51 val_51 2009-08-01 +51 val_52 2009-08-01 +52 val_53 2009-08-01 +52 val_53 2009-08-01 +52 val_53 2009-08-01 +52 val_53 2009-08-01 +53 val_53 2009-08-01 +53 val_54 2009-08-01 +54 val_54 2009-08-01 +56 val_57 2009-08-01 +57 val_57 2009-08-01 +58 val_58 2009-08-01 +58 val_58 2009-08-01 +58 val_59 2009-08-01 +58 val_59 2009-08-01 +59 val_60 2009-08-01 +6 val_7 2009-08-01 +6 val_7 2009-08-01 +60 val_61 2009-08-01 +61 val_62 2009-08-01 +62 val_63 2009-08-01 +62 val_63 2009-08-01 +63 val_64 2009-08-01 +64 val_64 2009-08-01 +65 val_65 2009-08-01 +65 val_66 2009-08-01 +65 val_66 2009-08-01 +66 val_66 2009-08-01 +67 val_67 2009-08-01 +67 val_67 2009-08-01 +68 val_69 2009-08-01 +69 val_69 2009-08-01 +69 val_70 2009-08-01 +70 val_70 2009-08-01 +70 val_70 2009-08-01 +70 val_70 2009-08-01 +70 val_71 2009-08-01 +71 val_72 2009-08-01 +72 val_72 2009-08-01 +72 val_72 2009-08-01 +74 val_74 2009-08-01 +75 val_76 2009-08-01 +76 val_76 2009-08-01 +76 val_76 2009-08-01 +76 val_77 2009-08-01 +76 val_77 2009-08-01 +76 val_77 2009-08-01 +77 val_77 2009-08-01 +77 val_78 2009-08-01 +77 val_78 2009-08-01 +78 val_78 2009-08-01 +78 val_79 2009-08-01 +8 val_8 2009-08-01 +8 val_9 2009-08-01 +80 val_80 2009-08-01 +80 val_81 2009-08-01 +82 val_82 2009-08-01 +82 val_83 2009-08-01 +82 val_83 2009-08-01 +83 val_83 2009-08-01 +83 val_83 2009-08-01 +84 val_84 2009-08-01 +84 val_84 2009-08-01 +85 val_85 2009-08-01 +85 val_86 2009-08-01 +86 val_86 2009-08-01 +86 val_87 2009-08-01 +87 val_87 2009-08-01 +87 val_88 2009-08-01 +87 val_88 2009-08-01 +89 val_90 2009-08-01 +89 val_90 2009-08-01 +89 val_90 2009-08-01 +9 val_9 2009-08-01 +90 val_90 2009-08-01 +90 val_90 2009-08-01 +90 val_90 2009-08-01 +91 val_92 2009-08-01 +92 val_92 2009-08-01 +93 val_94 2009-08-01 +93 val_94 2009-08-01 +93 val_94 2009-08-01 +94 val_95 2009-08-01 +95 val_95 2009-08-01 +95 val_95 2009-08-01 +96 val_96 2009-08-01 +97 val_97 2009-08-01 +97 val_97 2009-08-01 +97 val_98 2009-08-01 +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 diff --git a/src/test/hive/golden/input41-0-763ab5853bff619e6525c01e46b2a923 b/src/test/hive/golden/input41-0-763ab5853bff619e6525c01e46b2a923 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input41-0-763ab5853bff619e6525c01e46b2a923 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input41-1-8112b3a278e8337dbd6f017ae178658b b/src/test/hive/golden/input41-1-8112b3a278e8337dbd6f017ae178658b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 b/src/test/hive/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee b/src/test/hive/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee new file mode 100644 index 0000000000000..579784a58a66c --- /dev/null +++ b/src/test/hive/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee @@ -0,0 +1,2 @@ +0 +500 \ No newline at end of file diff --git a/src/test/hive/golden/input43-0-2baba8070f3585debc14b6bb3c83607a b/src/test/hive/golden/input43-0-2baba8070f3585debc14b6bb3c83607a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 b/src/test/hive/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f b/src/test/hive/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca b/src/test/hive/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input46-3-c185163787977498a4b84f39f983c431 b/src/test/hive/golden/input46-3-c185163787977498a4b84f39f983c431 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 b/src/test/hive/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab b/src/test/hive/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input49-0-2bd546beeb607da5b925ffbea20a741c b/src/test/hive/golden/input49-0-2bd546beeb607da5b925ffbea20a741c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d b/src/test/hive/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 b/src/test/hive/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 new file mode 100644 index 0000000000000..0d3a5cb1af472 --- /dev/null +++ b/src/test/hive/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 @@ -0,0 +1 @@ +false 2.9098519E7 1410.0 996 40408519555 test_string 12 diff --git a/src/test/hive/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 b/src/test/hive/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 b/src/test/hive/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 b/src/test/hive/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 b/src/test/hive/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 new file mode 100644 index 0000000000000..218c223b050b7 --- /dev/null +++ b/src/test/hive/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 @@ -0,0 +1,500 @@ +val_238 238 +val_86 86 +val_311 311 +val_27 27 +val_165 165 +val_409 409 +val_255 255 +val_278 278 +val_98 98 +val_484 484 +val_265 265 +val_193 193 +val_401 401 +val_150 150 +val_273 273 +val_224 224 +val_369 369 +val_66 66 +val_128 128 +val_213 213 +val_146 146 +val_406 406 +val_429 429 +val_374 374 +val_152 152 +val_469 469 +val_145 145 +val_495 495 +val_37 37 +val_327 327 +val_281 281 +val_277 277 +val_209 209 +val_15 15 +val_82 82 +val_403 403 +val_166 166 +val_417 417 +val_430 430 +val_252 252 +val_292 292 +val_219 219 +val_287 287 +val_153 153 +val_193 193 +val_338 338 +val_446 446 +val_459 459 +val_394 394 +val_237 237 +val_482 482 +val_174 174 +val_413 413 +val_494 494 +val_207 207 +val_199 199 +val_466 466 +val_208 208 +val_174 174 +val_399 399 +val_396 396 +val_247 247 +val_417 417 +val_489 489 +val_162 162 +val_377 377 +val_397 397 +val_309 309 +val_365 365 +val_266 266 +val_439 439 +val_342 342 +val_367 367 +val_325 325 +val_167 167 +val_195 195 +val_475 475 +val_17 17 +val_113 113 +val_155 155 +val_203 203 +val_339 339 +val_0 0 +val_455 455 +val_128 128 +val_311 311 +val_316 316 +val_57 57 +val_302 302 +val_205 205 +val_149 149 +val_438 438 +val_345 345 +val_129 129 +val_170 170 +val_20 20 +val_489 489 +val_157 157 +val_378 378 +val_221 221 +val_92 92 +val_111 111 +val_47 47 +val_72 72 +val_4 4 +val_280 280 +val_35 35 +val_427 427 +val_277 277 +val_208 208 +val_356 356 +val_399 399 +val_169 169 +val_382 382 +val_498 498 +val_125 125 +val_386 386 +val_437 437 +val_469 469 +val_192 192 +val_286 286 +val_187 187 +val_176 176 +val_54 54 +val_459 459 +val_51 51 +val_138 138 +val_103 103 +val_239 239 +val_213 213 +val_216 216 +val_430 430 +val_278 278 +val_176 176 +val_289 289 +val_221 221 +val_65 65 +val_318 318 +val_332 332 +val_311 311 +val_275 275 +val_137 137 +val_241 241 +val_83 83 +val_333 333 +val_180 180 +val_284 284 +val_12 12 +val_230 230 +val_181 181 +val_67 67 +val_260 260 +val_404 404 +val_384 384 +val_489 489 +val_353 353 +val_373 373 +val_272 272 +val_138 138 +val_217 217 +val_84 84 +val_348 348 +val_466 466 +val_58 58 +val_8 8 +val_411 411 +val_230 230 +val_208 208 +val_348 348 +val_24 24 +val_463 463 +val_431 431 +val_179 179 +val_172 172 +val_42 42 +val_129 129 +val_158 158 +val_119 119 +val_496 496 +val_0 0 +val_322 322 +val_197 197 +val_468 468 +val_393 393 +val_454 454 +val_100 100 +val_298 298 +val_199 199 +val_191 191 +val_418 418 +val_96 96 +val_26 26 +val_165 165 +val_327 327 +val_230 230 +val_205 205 +val_120 120 +val_131 131 +val_51 51 +val_404 404 +val_43 43 +val_436 436 +val_156 156 +val_469 469 +val_468 468 +val_308 308 +val_95 95 +val_196 196 +val_288 288 +val_481 481 +val_457 457 +val_98 98 +val_282 282 +val_197 197 +val_187 187 +val_318 318 +val_318 318 +val_409 409 +val_470 470 +val_137 137 +val_369 369 +val_316 316 +val_169 169 +val_413 413 +val_85 85 +val_77 77 +val_0 0 +val_490 490 +val_87 87 +val_364 364 +val_179 179 +val_118 118 +val_134 134 +val_395 395 +val_282 282 +val_138 138 +val_238 238 +val_419 419 +val_15 15 +val_118 118 +val_72 72 +val_90 90 +val_307 307 +val_19 19 +val_435 435 +val_10 10 +val_277 277 +val_273 273 +val_306 306 +val_224 224 +val_309 309 +val_389 389 +val_327 327 +val_242 242 +val_369 369 +val_392 392 +val_272 272 +val_331 331 +val_401 401 +val_242 242 +val_452 452 +val_177 177 +val_226 226 +val_5 5 +val_497 497 +val_402 402 +val_396 396 +val_317 317 +val_395 395 +val_58 58 +val_35 35 +val_336 336 +val_95 95 +val_11 11 +val_168 168 +val_34 34 +val_229 229 +val_233 233 +val_143 143 +val_472 472 +val_322 322 +val_498 498 +val_160 160 +val_195 195 +val_42 42 +val_321 321 +val_430 430 +val_119 119 +val_489 489 +val_458 458 +val_78 78 +val_76 76 +val_41 41 +val_223 223 +val_492 492 +val_149 149 +val_449 449 +val_218 218 +val_228 228 +val_138 138 +val_453 453 +val_30 30 +val_209 209 +val_64 64 +val_468 468 +val_76 76 +val_74 74 +val_342 342 +val_69 69 +val_230 230 +val_33 33 +val_368 368 +val_103 103 +val_296 296 +val_113 113 +val_216 216 +val_367 367 +val_344 344 +val_167 167 +val_274 274 +val_219 219 +val_239 239 +val_485 485 +val_116 116 +val_223 223 +val_256 256 +val_263 263 +val_70 70 +val_487 487 +val_480 480 +val_401 401 +val_288 288 +val_191 191 +val_5 5 +val_244 244 +val_438 438 +val_128 128 +val_467 467 +val_432 432 +val_202 202 +val_316 316 +val_229 229 +val_469 469 +val_463 463 +val_280 280 +val_2 2 +val_35 35 +val_283 283 +val_331 331 +val_235 235 +val_80 80 +val_44 44 +val_193 193 +val_321 321 +val_335 335 +val_104 104 +val_466 466 +val_366 366 +val_175 175 +val_403 403 +val_483 483 +val_53 53 +val_105 105 +val_257 257 +val_406 406 +val_409 409 +val_190 190 +val_406 406 +val_401 401 +val_114 114 +val_258 258 +val_90 90 +val_203 203 +val_262 262 +val_348 348 +val_424 424 +val_12 12 +val_396 396 +val_201 201 +val_217 217 +val_164 164 +val_431 431 +val_454 454 +val_478 478 +val_298 298 +val_125 125 +val_431 431 +val_164 164 +val_424 424 +val_187 187 +val_382 382 +val_5 5 +val_70 70 +val_397 397 +val_480 480 +val_291 291 +val_24 24 +val_351 351 +val_255 255 +val_104 104 +val_70 70 +val_163 163 +val_438 438 +val_119 119 +val_414 414 +val_200 200 +val_491 491 +val_237 237 +val_439 439 +val_360 360 +val_248 248 +val_479 479 +val_305 305 +val_417 417 +val_199 199 +val_444 444 +val_120 120 +val_429 429 +val_169 169 +val_443 443 +val_323 323 +val_325 325 +val_277 277 +val_230 230 +val_478 478 +val_178 178 +val_468 468 +val_310 310 +val_317 317 +val_333 333 +val_493 493 +val_460 460 +val_207 207 +val_249 249 +val_265 265 +val_480 480 +val_83 83 +val_136 136 +val_353 353 +val_172 172 +val_214 214 +val_462 462 +val_233 233 +val_406 406 +val_133 133 +val_175 175 +val_189 189 +val_454 454 +val_375 375 +val_401 401 +val_421 421 +val_407 407 +val_384 384 +val_256 256 +val_26 26 +val_134 134 +val_67 67 +val_384 384 +val_379 379 +val_18 18 +val_462 462 +val_492 492 +val_100 100 +val_298 298 +val_9 9 +val_341 341 +val_498 498 +val_146 146 +val_458 458 +val_362 362 +val_186 186 +val_285 285 +val_348 348 +val_167 167 +val_18 18 +val_273 273 +val_183 183 +val_281 281 +val_344 344 +val_97 97 +val_469 469 +val_315 315 +val_84 84 +val_28 28 +val_37 37 +val_448 448 +val_152 152 +val_348 348 +val_307 307 +val_194 194 +val_414 414 +val_477 477 +val_222 222 +val_126 126 +val_90 90 +val_169 169 +val_403 403 +val_400 400 +val_200 200 +val_97 97 \ No newline at end of file diff --git a/src/test/hive/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d b/src/test/hive/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af b/src/test/hive/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af new file mode 100644 index 0000000000000..217a1915f8826 --- /dev/null +++ b/src/test/hive/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af @@ -0,0 +1,10 @@ +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/src/test/hive/golden/input5-0-659e06570690cceeb3f37e10e855d2ea b/src/test/hive/golden/input5-0-659e06570690cceeb3f37e10e855d2ea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 b/src/test/hive/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad b/src/test/hive/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f8aa003a65bb4 --- /dev/null +++ b/src/test/hive/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,11 @@ +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/src/test/hive/golden/input6-0-659e06570690cceeb3f37e10e855d2ea b/src/test/hive/golden/input6-0-659e06570690cceeb3f37e10e855d2ea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input6-1-21149162906e31964a328b6cb2d5286e b/src/test/hive/golden/input6-1-21149162906e31964a328b6cb2d5286e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c b/src/test/hive/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..b4dc488518f24 --- /dev/null +++ b/src/test/hive/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +NULL +NULL val_27 +NULL val_165 +NULL val_409 +NULL val_484 +NULL val_265 +NULL val_193 +NULL +NULL +NULL \ No newline at end of file diff --git a/src/test/hive/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 b/src/test/hive/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 b/src/test/hive/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input7-2-c740098a289f9a4267d26fc10347a0dc b/src/test/hive/golden/input7-2-c740098a289f9a4267d26fc10347a0dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..0b8a8960a992b --- /dev/null +++ b/src/test/hive/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,25 @@ +NULL 238 +NULL NULL +NULL 311 +NULL NULL +NULL NULL +NULL NULL +NULL 255 +NULL 278 +NULL 98 +NULL NULL +NULL NULL +NULL NULL +NULL 401 +NULL 150 +NULL 273 +NULL 224 +NULL 369 +NULL 66 +NULL 128 +NULL 213 +NULL 146 +NULL 406 +NULL NULL +NULL NULL +NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa b/src/test/hive/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa b/src/test/hive/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba b/src/test/hive/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..c87107a2f1168 --- /dev/null +++ b/src/test/hive/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,25 @@ +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 b/src/test/hive/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input9-1-b30105391cb53915997a84ca3137bd0a b/src/test/hive/golden/input9-1-b30105391cb53915997a84ca3137bd0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input9-2-171ea277f05a642699a0f91bacffccd8 b/src/test/hive/golden/input9-2-171ea277f05a642699a0f91bacffccd8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e b/src/test/hive/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 b/src/test/hive/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be b/src/test/hive/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 b/src/test/hive/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 new file mode 100644 index 0000000000000..e782acd4d1e7f --- /dev/null +++ b/src/test/hive/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 @@ -0,0 +1,11 @@ +[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/src/test/hive/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 b/src/test/hive/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 new file mode 100644 index 0000000000000..6038b8aa32884 --- /dev/null +++ b/src/test/hive/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 @@ -0,0 +1,11 @@ +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/src/test/hive/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d b/src/test/hive/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc b/src/test/hive/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 b/src/test/hive/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..e782acd4d1e7f --- /dev/null +++ b/src/test/hive/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,11 @@ +[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/src/test/hive/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 b/src/test/hive/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 new file mode 100644 index 0000000000000..6038b8aa32884 --- /dev/null +++ b/src/test/hive/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 @@ -0,0 +1,11 @@ +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/src/test/hive/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d b/src/test/hive/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 b/src/test/hive/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 b/src/test/hive/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c b/src/test/hive/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c b/src/test/hive/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c new file mode 100644 index 0000000000000..6af528eab23a8 --- /dev/null +++ b/src/test/hive/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c @@ -0,0 +1,11 @@ +{"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/src/test/hive/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc b/src/test/hive/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 b/src/test/hive/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 new file mode 100644 index 0000000000000..e782acd4d1e7f --- /dev/null +++ b/src/test/hive/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 @@ -0,0 +1,11 @@ +[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/src/test/hive/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db b/src/test/hive/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db new file mode 100644 index 0000000000000..6038b8aa32884 --- /dev/null +++ b/src/test/hive/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db @@ -0,0 +1,11 @@ +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/src/test/hive/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a b/src/test/hive/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 b/src/test/hive/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 b/src/test/hive/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c b/src/test/hive/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c new file mode 100644 index 0000000000000..1bb008b44d6ee --- /dev/null +++ b/src/test/hive/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c @@ -0,0 +1,11 @@ +[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/src/test/hive/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a b/src/test/hive/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 b/src/test/hive/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 b/src/test/hive/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 new file mode 100644 index 0000000000000..17f24d2991d14 --- /dev/null +++ b/src/test/hive/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 @@ -0,0 +1,20 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 b/src/test/hive/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb b/src/test/hive/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb new file mode 100644 index 0000000000000..499e6b8ab6b7e --- /dev/null +++ b/src/test/hive/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb @@ -0,0 +1,1000 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 b/src/test/hive/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 b/src/test/hive/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc b/src/test/hive/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..3e5ae10e4670a --- /dev/null +++ b/src/test/hive/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,84 @@ +86 val_86 12 2008-04-08 +27 val_27 12 2008-04-08 +98 val_98 12 2008-04-08 +66 val_66 12 2008-04-08 +37 val_37 12 2008-04-08 +15 val_15 12 2008-04-08 +82 val_82 12 2008-04-08 +17 val_17 12 2008-04-08 +0 val_0 12 2008-04-08 +57 val_57 12 2008-04-08 +20 val_20 12 2008-04-08 +92 val_92 12 2008-04-08 +47 val_47 12 2008-04-08 +72 val_72 12 2008-04-08 +4 val_4 12 2008-04-08 +35 val_35 12 2008-04-08 +54 val_54 12 2008-04-08 +51 val_51 12 2008-04-08 +65 val_65 12 2008-04-08 +83 val_83 12 2008-04-08 +12 val_12 12 2008-04-08 +67 val_67 12 2008-04-08 +84 val_84 12 2008-04-08 +58 val_58 12 2008-04-08 +8 val_8 12 2008-04-08 +24 val_24 12 2008-04-08 +42 val_42 12 2008-04-08 +0 val_0 12 2008-04-08 +96 val_96 12 2008-04-08 +26 val_26 12 2008-04-08 +51 val_51 12 2008-04-08 +43 val_43 12 2008-04-08 +95 val_95 12 2008-04-08 +98 val_98 12 2008-04-08 +85 val_85 12 2008-04-08 +77 val_77 12 2008-04-08 +0 val_0 12 2008-04-08 +87 val_87 12 2008-04-08 +15 val_15 12 2008-04-08 +72 val_72 12 2008-04-08 +90 val_90 12 2008-04-08 +19 val_19 12 2008-04-08 +10 val_10 12 2008-04-08 +5 val_5 12 2008-04-08 +58 val_58 12 2008-04-08 +35 val_35 12 2008-04-08 +95 val_95 12 2008-04-08 +11 val_11 12 2008-04-08 +34 val_34 12 2008-04-08 +42 val_42 12 2008-04-08 +78 val_78 12 2008-04-08 +76 val_76 12 2008-04-08 +41 val_41 12 2008-04-08 +30 val_30 12 2008-04-08 +64 val_64 12 2008-04-08 +76 val_76 12 2008-04-08 +74 val_74 12 2008-04-08 +69 val_69 12 2008-04-08 +33 val_33 12 2008-04-08 +70 val_70 12 2008-04-08 +5 val_5 12 2008-04-08 +2 val_2 12 2008-04-08 +35 val_35 12 2008-04-08 +80 val_80 12 2008-04-08 +44 val_44 12 2008-04-08 +53 val_53 12 2008-04-08 +90 val_90 12 2008-04-08 +12 val_12 12 2008-04-08 +5 val_5 12 2008-04-08 +70 val_70 12 2008-04-08 +24 val_24 12 2008-04-08 +70 val_70 12 2008-04-08 +83 val_83 12 2008-04-08 +26 val_26 12 2008-04-08 +67 val_67 12 2008-04-08 +18 val_18 12 2008-04-08 +9 val_9 12 2008-04-08 +18 val_18 12 2008-04-08 +97 val_97 12 2008-04-08 +84 val_84 12 2008-04-08 +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 diff --git a/src/test/hive/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 b/src/test/hive/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part10-1-35f67c0112026170d015b5f80a254bde b/src/test/hive/golden/input_part10-1-35f67c0112026170d015b5f80a254bde new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part10-2-40069c199502c2724ac2a2733f964248 b/src/test/hive/golden/input_part10-2-40069c199502c2724ac2a2733f964248 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 b/src/test/hive/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 new file mode 100644 index 0000000000000..82116102c1f54 --- /dev/null +++ b/src/test/hive/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 @@ -0,0 +1,12 @@ +a string None +b string None +ds string None +ts string None + +# Partition Information +# col_name data_type comment + +ds string None +ts string None + +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 diff --git a/src/test/hive/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 b/src/test/hive/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 new file mode 100644 index 0000000000000..c8d0d55930069 --- /dev/null +++ b/src/test/hive/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 @@ -0,0 +1 @@ +1 2 2008 04 08 10:11:12=455 \ No newline at end of file diff --git a/src/test/hive/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf b/src/test/hive/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde b/src/test/hive/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 b/src/test/hive/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 b/src/test/hive/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 new file mode 100644 index 0000000000000..869eb58e70d82 --- /dev/null +++ b/src/test/hive/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 @@ -0,0 +1,12 @@ +a string None +b string None +ds string None +ts string None + +# Partition Information +# col_name data_type comment + +ds string None +ts string None + +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 diff --git a/src/test/hive/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 b/src/test/hive/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 new file mode 100644 index 0000000000000..c8d0d55930069 --- /dev/null +++ b/src/test/hive/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 @@ -0,0 +1 @@ +1 2 2008 04 08 10:11:12=455 \ No newline at end of file diff --git a/src/test/hive/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 b/src/test/hive/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e b/src/test/hive/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd b/src/test/hive/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 b/src/test/hive/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 b/src/test/hive/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 new file mode 100644 index 0000000000000..fd945cc15d9ca --- /dev/null +++ b/src/test/hive/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 @@ -0,0 +1,84 @@ +0 val_0 12 2008-04-08 +0 val_0 12 2008-04-08 +0 val_0 12 2008-04-08 +2 val_2 12 2008-04-08 +4 val_4 12 2008-04-08 +5 val_5 12 2008-04-08 +5 val_5 12 2008-04-08 +5 val_5 12 2008-04-08 +8 val_8 12 2008-04-08 +9 val_9 12 2008-04-08 +10 val_10 12 2008-04-08 +11 val_11 12 2008-04-08 +12 val_12 12 2008-04-08 +12 val_12 12 2008-04-08 +15 val_15 12 2008-04-08 +15 val_15 12 2008-04-08 +17 val_17 12 2008-04-08 +18 val_18 12 2008-04-08 +18 val_18 12 2008-04-08 +19 val_19 12 2008-04-08 +20 val_20 12 2008-04-08 +24 val_24 12 2008-04-08 +24 val_24 12 2008-04-08 +26 val_26 12 2008-04-08 +26 val_26 12 2008-04-08 +27 val_27 12 2008-04-08 +28 val_28 12 2008-04-08 +30 val_30 12 2008-04-08 +33 val_33 12 2008-04-08 +34 val_34 12 2008-04-08 +35 val_35 12 2008-04-08 +35 val_35 12 2008-04-08 +35 val_35 12 2008-04-08 +37 val_37 12 2008-04-08 +37 val_37 12 2008-04-08 +41 val_41 12 2008-04-08 +42 val_42 12 2008-04-08 +42 val_42 12 2008-04-08 +43 val_43 12 2008-04-08 +44 val_44 12 2008-04-08 +47 val_47 12 2008-04-08 +51 val_51 12 2008-04-08 +51 val_51 12 2008-04-08 +53 val_53 12 2008-04-08 +54 val_54 12 2008-04-08 +57 val_57 12 2008-04-08 +58 val_58 12 2008-04-08 +58 val_58 12 2008-04-08 +64 val_64 12 2008-04-08 +65 val_65 12 2008-04-08 +66 val_66 12 2008-04-08 +67 val_67 12 2008-04-08 +67 val_67 12 2008-04-08 +69 val_69 12 2008-04-08 +70 val_70 12 2008-04-08 +70 val_70 12 2008-04-08 +70 val_70 12 2008-04-08 +72 val_72 12 2008-04-08 +72 val_72 12 2008-04-08 +74 val_74 12 2008-04-08 +76 val_76 12 2008-04-08 +76 val_76 12 2008-04-08 +77 val_77 12 2008-04-08 +78 val_78 12 2008-04-08 +80 val_80 12 2008-04-08 +82 val_82 12 2008-04-08 +83 val_83 12 2008-04-08 +83 val_83 12 2008-04-08 +84 val_84 12 2008-04-08 +84 val_84 12 2008-04-08 +85 val_85 12 2008-04-08 +86 val_86 12 2008-04-08 +87 val_87 12 2008-04-08 +90 val_90 12 2008-04-08 +90 val_90 12 2008-04-08 +90 val_90 12 2008-04-08 +92 val_92 12 2008-04-08 +95 val_95 12 2008-04-08 +95 val_95 12 2008-04-08 +96 val_96 12 2008-04-08 +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 diff --git a/src/test/hive/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 b/src/test/hive/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 new file mode 100644 index 0000000000000..f23877361f33b --- /dev/null +++ b/src/test/hive/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 @@ -0,0 +1,84 @@ +0 val_0 12 2008-04-09 +0 val_0 12 2008-04-09 +0 val_0 12 2008-04-09 +2 val_2 12 2008-04-09 +4 val_4 12 2008-04-09 +5 val_5 12 2008-04-09 +5 val_5 12 2008-04-09 +5 val_5 12 2008-04-09 +8 val_8 12 2008-04-09 +9 val_9 12 2008-04-09 +10 val_10 12 2008-04-09 +11 val_11 12 2008-04-09 +12 val_12 12 2008-04-09 +12 val_12 12 2008-04-09 +15 val_15 12 2008-04-09 +15 val_15 12 2008-04-09 +17 val_17 12 2008-04-09 +18 val_18 12 2008-04-09 +18 val_18 12 2008-04-09 +19 val_19 12 2008-04-09 +20 val_20 12 2008-04-09 +24 val_24 12 2008-04-09 +24 val_24 12 2008-04-09 +26 val_26 12 2008-04-09 +26 val_26 12 2008-04-09 +27 val_27 12 2008-04-09 +28 val_28 12 2008-04-09 +30 val_30 12 2008-04-09 +33 val_33 12 2008-04-09 +34 val_34 12 2008-04-09 +35 val_35 12 2008-04-09 +35 val_35 12 2008-04-09 +35 val_35 12 2008-04-09 +37 val_37 12 2008-04-09 +37 val_37 12 2008-04-09 +41 val_41 12 2008-04-09 +42 val_42 12 2008-04-09 +42 val_42 12 2008-04-09 +43 val_43 12 2008-04-09 +44 val_44 12 2008-04-09 +47 val_47 12 2008-04-09 +51 val_51 12 2008-04-09 +51 val_51 12 2008-04-09 +53 val_53 12 2008-04-09 +54 val_54 12 2008-04-09 +57 val_57 12 2008-04-09 +58 val_58 12 2008-04-09 +58 val_58 12 2008-04-09 +64 val_64 12 2008-04-09 +65 val_65 12 2008-04-09 +66 val_66 12 2008-04-09 +67 val_67 12 2008-04-09 +67 val_67 12 2008-04-09 +69 val_69 12 2008-04-09 +70 val_70 12 2008-04-09 +70 val_70 12 2008-04-09 +70 val_70 12 2008-04-09 +72 val_72 12 2008-04-09 +72 val_72 12 2008-04-09 +74 val_74 12 2008-04-09 +76 val_76 12 2008-04-09 +76 val_76 12 2008-04-09 +77 val_77 12 2008-04-09 +78 val_78 12 2008-04-09 +80 val_80 12 2008-04-09 +82 val_82 12 2008-04-09 +83 val_83 12 2008-04-09 +83 val_83 12 2008-04-09 +84 val_84 12 2008-04-09 +84 val_84 12 2008-04-09 +85 val_85 12 2008-04-09 +86 val_86 12 2008-04-09 +87 val_87 12 2008-04-09 +90 val_90 12 2008-04-09 +90 val_90 12 2008-04-09 +90 val_90 12 2008-04-09 +92 val_92 12 2008-04-09 +95 val_95 12 2008-04-09 +95 val_95 12 2008-04-09 +96 val_96 12 2008-04-09 +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 diff --git a/src/test/hive/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff b/src/test/hive/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 b/src/test/hive/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 new file mode 100644 index 0000000000000..f4026a591a958 --- /dev/null +++ b/src/test/hive/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 @@ -0,0 +1,500 @@ +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/src/test/hive/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef b/src/test/hive/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 b/src/test/hive/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b b/src/test/hive/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c b/src/test/hive/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part5-2-d678533566fba4b922f01284538ca484 b/src/test/hive/golden/input_part5-2-d678533566fba4b922f01284538ca484 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 b/src/test/hive/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 new file mode 100644 index 0000000000000..853c3bc8df7f0 --- /dev/null +++ b/src/test/hive/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 @@ -0,0 +1,168 @@ +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 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 +10 val_10 2008-04-08 11 +10 val_10 2008-04-08 12 +11 val_11 2008-04-08 11 +11 val_11 2008-04-08 12 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 12 +12 val_12 2008-04-08 12 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 12 +15 val_15 2008-04-08 12 +17 val_17 2008-04-08 11 +17 val_17 2008-04-08 12 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 12 +18 val_18 2008-04-08 12 +19 val_19 2008-04-08 11 +19 val_19 2008-04-08 12 +2 val_2 2008-04-08 11 +2 val_2 2008-04-08 12 +20 val_20 2008-04-08 11 +20 val_20 2008-04-08 12 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 12 +24 val_24 2008-04-08 12 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 12 +26 val_26 2008-04-08 12 +27 val_27 2008-04-08 11 +27 val_27 2008-04-08 12 +28 val_28 2008-04-08 11 +28 val_28 2008-04-08 12 +30 val_30 2008-04-08 11 +30 val_30 2008-04-08 12 +33 val_33 2008-04-08 11 +33 val_33 2008-04-08 12 +34 val_34 2008-04-08 11 +34 val_34 2008-04-08 12 +35 val_35 2008-04-08 11 +35 val_35 2008-04-08 11 +35 val_35 2008-04-08 11 +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 11 +37 val_37 2008-04-08 11 +37 val_37 2008-04-08 12 +37 val_37 2008-04-08 12 +4 val_4 2008-04-08 11 +4 val_4 2008-04-08 12 +41 val_41 2008-04-08 11 +41 val_41 2008-04-08 12 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 12 +42 val_42 2008-04-08 12 +43 val_43 2008-04-08 11 +43 val_43 2008-04-08 12 +44 val_44 2008-04-08 11 +44 val_44 2008-04-08 12 +47 val_47 2008-04-08 11 +47 val_47 2008-04-08 12 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 12 +5 val_5 2008-04-08 12 +5 val_5 2008-04-08 12 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 12 +51 val_51 2008-04-08 12 +53 val_53 2008-04-08 11 +53 val_53 2008-04-08 12 +54 val_54 2008-04-08 11 +54 val_54 2008-04-08 12 +57 val_57 2008-04-08 11 +57 val_57 2008-04-08 12 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 12 +58 val_58 2008-04-08 12 +64 val_64 2008-04-08 11 +64 val_64 2008-04-08 12 +65 val_65 2008-04-08 11 +65 val_65 2008-04-08 12 +66 val_66 2008-04-08 11 +66 val_66 2008-04-08 12 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 12 +67 val_67 2008-04-08 12 +69 val_69 2008-04-08 11 +69 val_69 2008-04-08 12 +70 val_70 2008-04-08 11 +70 val_70 2008-04-08 11 +70 val_70 2008-04-08 11 +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 11 +72 val_72 2008-04-08 11 +72 val_72 2008-04-08 12 +72 val_72 2008-04-08 12 +74 val_74 2008-04-08 11 +74 val_74 2008-04-08 12 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 12 +76 val_76 2008-04-08 12 +77 val_77 2008-04-08 11 +77 val_77 2008-04-08 12 +78 val_78 2008-04-08 11 +78 val_78 2008-04-08 12 +8 val_8 2008-04-08 11 +8 val_8 2008-04-08 12 +80 val_80 2008-04-08 11 +80 val_80 2008-04-08 12 +82 val_82 2008-04-08 11 +82 val_82 2008-04-08 12 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 12 +83 val_83 2008-04-08 12 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 12 +84 val_84 2008-04-08 12 +85 val_85 2008-04-08 11 +85 val_85 2008-04-08 12 +86 val_86 2008-04-08 11 +86 val_86 2008-04-08 12 +87 val_87 2008-04-08 11 +87 val_87 2008-04-08 12 +9 val_9 2008-04-08 11 +9 val_9 2008-04-08 12 +90 val_90 2008-04-08 11 +90 val_90 2008-04-08 11 +90 val_90 2008-04-08 11 +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 11 +92 val_92 2008-04-08 12 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 12 +95 val_95 2008-04-08 12 +96 val_96 2008-04-08 11 +96 val_96 2008-04-08 12 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 12 +97 val_97 2008-04-08 12 +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 diff --git a/src/test/hive/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d b/src/test/hive/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 b/src/test/hive/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part7-0-14448a12627b8bccf6828032f938f39e b/src/test/hive/golden/input_part7-0-14448a12627b8bccf6828032f938f39e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 b/src/test/hive/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 new file mode 100644 index 0000000000000..540ca86636f3c --- /dev/null +++ b/src/test/hive/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 @@ -0,0 +1,336 @@ +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 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 +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 11 +2 val_2 2008-04-08 11 +2 val_2 2008-04-08 12 +2 val_2 2008-04-08 12 +4 val_4 2008-04-08 11 +4 val_4 2008-04-08 11 +4 val_4 2008-04-08 12 +4 val_4 2008-04-08 12 +5 val_5 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 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 11 +5 val_5 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 +5 val_5 2008-04-08 12 +5 val_5 2008-04-08 12 +8 val_8 2008-04-08 11 +8 val_8 2008-04-08 11 +8 val_8 2008-04-08 12 +8 val_8 2008-04-08 12 +9 val_9 2008-04-08 11 +9 val_9 2008-04-08 11 +9 val_9 2008-04-08 12 +9 val_9 2008-04-08 12 +10 val_10 2008-04-08 11 +10 val_10 2008-04-08 11 +10 val_10 2008-04-08 12 +10 val_10 2008-04-08 12 +11 val_11 2008-04-08 11 +11 val_11 2008-04-08 11 +11 val_11 2008-04-08 12 +11 val_11 2008-04-08 12 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 12 +12 val_12 2008-04-08 12 +12 val_12 2008-04-08 12 +12 val_12 2008-04-08 12 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 12 +15 val_15 2008-04-08 12 +15 val_15 2008-04-08 12 +15 val_15 2008-04-08 12 +17 val_17 2008-04-08 11 +17 val_17 2008-04-08 11 +17 val_17 2008-04-08 12 +17 val_17 2008-04-08 12 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 12 +18 val_18 2008-04-08 12 +18 val_18 2008-04-08 12 +18 val_18 2008-04-08 12 +19 val_19 2008-04-08 11 +19 val_19 2008-04-08 11 +19 val_19 2008-04-08 12 +19 val_19 2008-04-08 12 +20 val_20 2008-04-08 11 +20 val_20 2008-04-08 11 +20 val_20 2008-04-08 12 +20 val_20 2008-04-08 12 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 12 +24 val_24 2008-04-08 12 +24 val_24 2008-04-08 12 +24 val_24 2008-04-08 12 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 12 +26 val_26 2008-04-08 12 +26 val_26 2008-04-08 12 +26 val_26 2008-04-08 12 +27 val_27 2008-04-08 11 +27 val_27 2008-04-08 11 +27 val_27 2008-04-08 12 +27 val_27 2008-04-08 12 +28 val_28 2008-04-08 11 +28 val_28 2008-04-08 11 +28 val_28 2008-04-08 12 +28 val_28 2008-04-08 12 +30 val_30 2008-04-08 11 +30 val_30 2008-04-08 11 +30 val_30 2008-04-08 12 +30 val_30 2008-04-08 12 +33 val_33 2008-04-08 11 +33 val_33 2008-04-08 11 +33 val_33 2008-04-08 12 +33 val_33 2008-04-08 12 +34 val_34 2008-04-08 11 +34 val_34 2008-04-08 11 +34 val_34 2008-04-08 12 +34 val_34 2008-04-08 12 +35 val_35 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 +35 val_35 2008-04-08 11 +35 val_35 2008-04-08 11 +35 val_35 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 +35 val_35 2008-04-08 12 +35 val_35 2008-04-08 12 +37 val_37 2008-04-08 11 +37 val_37 2008-04-08 11 +37 val_37 2008-04-08 11 +37 val_37 2008-04-08 11 +37 val_37 2008-04-08 12 +37 val_37 2008-04-08 12 +37 val_37 2008-04-08 12 +37 val_37 2008-04-08 12 +41 val_41 2008-04-08 11 +41 val_41 2008-04-08 11 +41 val_41 2008-04-08 12 +41 val_41 2008-04-08 12 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 12 +42 val_42 2008-04-08 12 +42 val_42 2008-04-08 12 +42 val_42 2008-04-08 12 +43 val_43 2008-04-08 11 +43 val_43 2008-04-08 11 +43 val_43 2008-04-08 12 +43 val_43 2008-04-08 12 +44 val_44 2008-04-08 11 +44 val_44 2008-04-08 11 +44 val_44 2008-04-08 12 +44 val_44 2008-04-08 12 +47 val_47 2008-04-08 11 +47 val_47 2008-04-08 11 +47 val_47 2008-04-08 12 +47 val_47 2008-04-08 12 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 12 +51 val_51 2008-04-08 12 +51 val_51 2008-04-08 12 +51 val_51 2008-04-08 12 +53 val_53 2008-04-08 11 +53 val_53 2008-04-08 11 +53 val_53 2008-04-08 12 +53 val_53 2008-04-08 12 +54 val_54 2008-04-08 11 +54 val_54 2008-04-08 11 +54 val_54 2008-04-08 12 +54 val_54 2008-04-08 12 +57 val_57 2008-04-08 11 +57 val_57 2008-04-08 11 +57 val_57 2008-04-08 12 +57 val_57 2008-04-08 12 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 12 +58 val_58 2008-04-08 12 +58 val_58 2008-04-08 12 +58 val_58 2008-04-08 12 +64 val_64 2008-04-08 11 +64 val_64 2008-04-08 11 +64 val_64 2008-04-08 12 +64 val_64 2008-04-08 12 +65 val_65 2008-04-08 11 +65 val_65 2008-04-08 11 +65 val_65 2008-04-08 12 +65 val_65 2008-04-08 12 +66 val_66 2008-04-08 11 +66 val_66 2008-04-08 11 +66 val_66 2008-04-08 12 +66 val_66 2008-04-08 12 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 12 +67 val_67 2008-04-08 12 +67 val_67 2008-04-08 12 +67 val_67 2008-04-08 12 +69 val_69 2008-04-08 11 +69 val_69 2008-04-08 11 +69 val_69 2008-04-08 12 +69 val_69 2008-04-08 12 +70 val_70 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 +70 val_70 2008-04-08 11 +70 val_70 2008-04-08 11 +70 val_70 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 +70 val_70 2008-04-08 12 +70 val_70 2008-04-08 12 +72 val_72 2008-04-08 11 +72 val_72 2008-04-08 11 +72 val_72 2008-04-08 11 +72 val_72 2008-04-08 11 +72 val_72 2008-04-08 12 +72 val_72 2008-04-08 12 +72 val_72 2008-04-08 12 +72 val_72 2008-04-08 12 +74 val_74 2008-04-08 11 +74 val_74 2008-04-08 11 +74 val_74 2008-04-08 12 +74 val_74 2008-04-08 12 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 12 +76 val_76 2008-04-08 12 +76 val_76 2008-04-08 12 +76 val_76 2008-04-08 12 +77 val_77 2008-04-08 11 +77 val_77 2008-04-08 11 +77 val_77 2008-04-08 12 +77 val_77 2008-04-08 12 +78 val_78 2008-04-08 11 +78 val_78 2008-04-08 11 +78 val_78 2008-04-08 12 +78 val_78 2008-04-08 12 +80 val_80 2008-04-08 11 +80 val_80 2008-04-08 11 +80 val_80 2008-04-08 12 +80 val_80 2008-04-08 12 +82 val_82 2008-04-08 11 +82 val_82 2008-04-08 11 +82 val_82 2008-04-08 12 +82 val_82 2008-04-08 12 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 12 +83 val_83 2008-04-08 12 +83 val_83 2008-04-08 12 +83 val_83 2008-04-08 12 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 12 +84 val_84 2008-04-08 12 +84 val_84 2008-04-08 12 +84 val_84 2008-04-08 12 +85 val_85 2008-04-08 11 +85 val_85 2008-04-08 11 +85 val_85 2008-04-08 12 +85 val_85 2008-04-08 12 +86 val_86 2008-04-08 11 +86 val_86 2008-04-08 11 +86 val_86 2008-04-08 12 +86 val_86 2008-04-08 12 +87 val_87 2008-04-08 11 +87 val_87 2008-04-08 11 +87 val_87 2008-04-08 12 +87 val_87 2008-04-08 12 +90 val_90 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 +90 val_90 2008-04-08 11 +90 val_90 2008-04-08 11 +90 val_90 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 +90 val_90 2008-04-08 12 +90 val_90 2008-04-08 12 +92 val_92 2008-04-08 11 +92 val_92 2008-04-08 11 +92 val_92 2008-04-08 12 +92 val_92 2008-04-08 12 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 12 +95 val_95 2008-04-08 12 +95 val_95 2008-04-08 12 +95 val_95 2008-04-08 12 +96 val_96 2008-04-08 11 +96 val_96 2008-04-08 11 +96 val_96 2008-04-08 12 +96 val_96 2008-04-08 12 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 12 +97 val_97 2008-04-08 12 +97 val_97 2008-04-08 12 +97 val_97 2008-04-08 12 +98 val_98 2008-04-08 11 +98 val_98 2008-04-08 11 +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 +98 val_98 2008-04-08 12 +98 val_98 2008-04-08 12 \ No newline at end of file diff --git a/src/test/hive/golden/input_part8-0-f60652796af548135f9e104b456840e3 b/src/test/hive/golden/input_part8-0-f60652796af548135f9e104b456840e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 b/src/test/hive/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 new file mode 100644 index 0000000000000..a1728c82f0b35 --- /dev/null +++ b/src/test/hive/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 @@ -0,0 +1,10 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 b/src/test/hive/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_part9-1-e60c60afc073367464898b8396e8f643 b/src/test/hive/golden/input_part9-1-e60c60afc073367464898b8396e8f643 new file mode 100644 index 0000000000000..438355d7b06f5 --- /dev/null +++ b/src/test/hive/golden/input_part9-1-e60c60afc073367464898b8396e8f643 @@ -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 +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 11 +2 val_2 2008-04-08 12 +4 val_4 2008-04-08 11 +4 val_4 2008-04-08 12 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 11 +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 11 +8 val_8 2008-04-08 12 +9 val_9 2008-04-08 11 +9 val_9 2008-04-08 12 +10 val_10 2008-04-08 11 +10 val_10 2008-04-08 12 +11 val_11 2008-04-08 11 +11 val_11 2008-04-08 12 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 12 +12 val_12 2008-04-08 12 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 12 +15 val_15 2008-04-08 12 +17 val_17 2008-04-08 11 +17 val_17 2008-04-08 12 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 12 +18 val_18 2008-04-08 12 +19 val_19 2008-04-08 11 +19 val_19 2008-04-08 12 +20 val_20 2008-04-08 11 +20 val_20 2008-04-08 12 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 12 +24 val_24 2008-04-08 12 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 12 +26 val_26 2008-04-08 12 +27 val_27 2008-04-08 11 +27 val_27 2008-04-08 12 +28 val_28 2008-04-08 11 +28 val_28 2008-04-08 12 +30 val_30 2008-04-08 11 +30 val_30 2008-04-08 12 +33 val_33 2008-04-08 11 +33 val_33 2008-04-08 12 +34 val_34 2008-04-08 11 +34 val_34 2008-04-08 12 +35 val_35 2008-04-08 11 +35 val_35 2008-04-08 11 +35 val_35 2008-04-08 11 +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 11 +37 val_37 2008-04-08 11 +37 val_37 2008-04-08 12 +37 val_37 2008-04-08 12 +41 val_41 2008-04-08 11 +41 val_41 2008-04-08 12 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 12 +42 val_42 2008-04-08 12 +43 val_43 2008-04-08 11 +43 val_43 2008-04-08 12 +44 val_44 2008-04-08 11 +44 val_44 2008-04-08 12 +47 val_47 2008-04-08 11 +47 val_47 2008-04-08 12 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 12 +51 val_51 2008-04-08 12 +53 val_53 2008-04-08 11 +53 val_53 2008-04-08 12 +54 val_54 2008-04-08 11 +54 val_54 2008-04-08 12 +57 val_57 2008-04-08 11 +57 val_57 2008-04-08 12 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 12 +58 val_58 2008-04-08 12 +64 val_64 2008-04-08 11 +64 val_64 2008-04-08 12 +65 val_65 2008-04-08 11 +65 val_65 2008-04-08 12 +66 val_66 2008-04-08 11 +66 val_66 2008-04-08 12 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 12 +67 val_67 2008-04-08 12 +69 val_69 2008-04-08 11 +69 val_69 2008-04-08 12 +70 val_70 2008-04-08 11 +70 val_70 2008-04-08 11 +70 val_70 2008-04-08 11 +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 11 +72 val_72 2008-04-08 11 +72 val_72 2008-04-08 12 +72 val_72 2008-04-08 12 +74 val_74 2008-04-08 11 +74 val_74 2008-04-08 12 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 12 +76 val_76 2008-04-08 12 +77 val_77 2008-04-08 11 +77 val_77 2008-04-08 12 +78 val_78 2008-04-08 11 +78 val_78 2008-04-08 12 +80 val_80 2008-04-08 11 +80 val_80 2008-04-08 12 +82 val_82 2008-04-08 11 +82 val_82 2008-04-08 12 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 12 +83 val_83 2008-04-08 12 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 12 +84 val_84 2008-04-08 12 +85 val_85 2008-04-08 11 +85 val_85 2008-04-08 12 +86 val_86 2008-04-08 11 +86 val_86 2008-04-08 12 +87 val_87 2008-04-08 11 +87 val_87 2008-04-08 12 +90 val_90 2008-04-08 11 +90 val_90 2008-04-08 11 +90 val_90 2008-04-08 11 +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 11 +92 val_92 2008-04-08 12 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 12 +95 val_95 2008-04-08 12 +96 val_96 2008-04-08 11 +96 val_96 2008-04-08 12 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 12 +97 val_97 2008-04-08 12 +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 +100 val_100 2008-04-08 11 +100 val_100 2008-04-08 11 +100 val_100 2008-04-08 12 +100 val_100 2008-04-08 12 +103 val_103 2008-04-08 11 +103 val_103 2008-04-08 11 +103 val_103 2008-04-08 12 +103 val_103 2008-04-08 12 +104 val_104 2008-04-08 11 +104 val_104 2008-04-08 11 +104 val_104 2008-04-08 12 +104 val_104 2008-04-08 12 +105 val_105 2008-04-08 11 +105 val_105 2008-04-08 12 +111 val_111 2008-04-08 11 +111 val_111 2008-04-08 12 +113 val_113 2008-04-08 11 +113 val_113 2008-04-08 11 +113 val_113 2008-04-08 12 +113 val_113 2008-04-08 12 +114 val_114 2008-04-08 11 +114 val_114 2008-04-08 12 +116 val_116 2008-04-08 11 +116 val_116 2008-04-08 12 +118 val_118 2008-04-08 11 +118 val_118 2008-04-08 11 +118 val_118 2008-04-08 12 +118 val_118 2008-04-08 12 +119 val_119 2008-04-08 11 +119 val_119 2008-04-08 11 +119 val_119 2008-04-08 11 +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 11 +120 val_120 2008-04-08 11 +120 val_120 2008-04-08 12 +120 val_120 2008-04-08 12 +125 val_125 2008-04-08 11 +125 val_125 2008-04-08 11 +125 val_125 2008-04-08 12 +125 val_125 2008-04-08 12 +126 val_126 2008-04-08 11 +126 val_126 2008-04-08 12 +128 val_128 2008-04-08 11 +128 val_128 2008-04-08 11 +128 val_128 2008-04-08 11 +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 11 +129 val_129 2008-04-08 11 +129 val_129 2008-04-08 12 +129 val_129 2008-04-08 12 +131 val_131 2008-04-08 11 +131 val_131 2008-04-08 12 +133 val_133 2008-04-08 11 +133 val_133 2008-04-08 12 +134 val_134 2008-04-08 11 +134 val_134 2008-04-08 11 +134 val_134 2008-04-08 12 +134 val_134 2008-04-08 12 +136 val_136 2008-04-08 11 +136 val_136 2008-04-08 12 +137 val_137 2008-04-08 11 +137 val_137 2008-04-08 11 +137 val_137 2008-04-08 12 +137 val_137 2008-04-08 12 +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 +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 11 +143 val_143 2008-04-08 12 +145 val_145 2008-04-08 11 +145 val_145 2008-04-08 12 +146 val_146 2008-04-08 11 +146 val_146 2008-04-08 11 +146 val_146 2008-04-08 12 +146 val_146 2008-04-08 12 +149 val_149 2008-04-08 11 +149 val_149 2008-04-08 11 +149 val_149 2008-04-08 12 +149 val_149 2008-04-08 12 +150 val_150 2008-04-08 11 +150 val_150 2008-04-08 12 +152 val_152 2008-04-08 11 +152 val_152 2008-04-08 11 +152 val_152 2008-04-08 12 +152 val_152 2008-04-08 12 +153 val_153 2008-04-08 11 +153 val_153 2008-04-08 12 +155 val_155 2008-04-08 11 +155 val_155 2008-04-08 12 +156 val_156 2008-04-08 11 +156 val_156 2008-04-08 12 +157 val_157 2008-04-08 11 +157 val_157 2008-04-08 12 +158 val_158 2008-04-08 11 +158 val_158 2008-04-08 12 +160 val_160 2008-04-08 11 +160 val_160 2008-04-08 12 +162 val_162 2008-04-08 11 +162 val_162 2008-04-08 12 +163 val_163 2008-04-08 11 +163 val_163 2008-04-08 12 +164 val_164 2008-04-08 11 +164 val_164 2008-04-08 11 +164 val_164 2008-04-08 12 +164 val_164 2008-04-08 12 +165 val_165 2008-04-08 11 +165 val_165 2008-04-08 11 +165 val_165 2008-04-08 12 +165 val_165 2008-04-08 12 +166 val_166 2008-04-08 11 +166 val_166 2008-04-08 12 +167 val_167 2008-04-08 11 +167 val_167 2008-04-08 11 +167 val_167 2008-04-08 11 +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 11 +168 val_168 2008-04-08 12 +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 +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 11 +170 val_170 2008-04-08 12 +172 val_172 2008-04-08 11 +172 val_172 2008-04-08 11 +172 val_172 2008-04-08 12 +172 val_172 2008-04-08 12 +174 val_174 2008-04-08 11 +174 val_174 2008-04-08 11 +174 val_174 2008-04-08 12 +174 val_174 2008-04-08 12 +175 val_175 2008-04-08 11 +175 val_175 2008-04-08 11 +175 val_175 2008-04-08 12 +175 val_175 2008-04-08 12 +176 val_176 2008-04-08 11 +176 val_176 2008-04-08 11 +176 val_176 2008-04-08 12 +176 val_176 2008-04-08 12 +177 val_177 2008-04-08 11 +177 val_177 2008-04-08 12 +178 val_178 2008-04-08 11 +178 val_178 2008-04-08 12 +179 val_179 2008-04-08 11 +179 val_179 2008-04-08 11 +179 val_179 2008-04-08 12 +179 val_179 2008-04-08 12 +180 val_180 2008-04-08 11 +180 val_180 2008-04-08 12 +181 val_181 2008-04-08 11 +181 val_181 2008-04-08 12 +183 val_183 2008-04-08 11 +183 val_183 2008-04-08 12 +186 val_186 2008-04-08 11 +186 val_186 2008-04-08 12 +187 val_187 2008-04-08 11 +187 val_187 2008-04-08 11 +187 val_187 2008-04-08 11 +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 11 +189 val_189 2008-04-08 12 +190 val_190 2008-04-08 11 +190 val_190 2008-04-08 12 +191 val_191 2008-04-08 11 +191 val_191 2008-04-08 11 +191 val_191 2008-04-08 12 +191 val_191 2008-04-08 12 +192 val_192 2008-04-08 11 +192 val_192 2008-04-08 12 +193 val_193 2008-04-08 11 +193 val_193 2008-04-08 11 +193 val_193 2008-04-08 11 +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 11 +194 val_194 2008-04-08 12 +195 val_195 2008-04-08 11 +195 val_195 2008-04-08 11 +195 val_195 2008-04-08 12 +195 val_195 2008-04-08 12 +196 val_196 2008-04-08 11 +196 val_196 2008-04-08 12 +197 val_197 2008-04-08 11 +197 val_197 2008-04-08 11 +197 val_197 2008-04-08 12 +197 val_197 2008-04-08 12 +199 val_199 2008-04-08 11 +199 val_199 2008-04-08 11 +199 val_199 2008-04-08 11 +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 11 +200 val_200 2008-04-08 11 +200 val_200 2008-04-08 12 +200 val_200 2008-04-08 12 +201 val_201 2008-04-08 11 +201 val_201 2008-04-08 12 +202 val_202 2008-04-08 11 +202 val_202 2008-04-08 12 +203 val_203 2008-04-08 11 +203 val_203 2008-04-08 11 +203 val_203 2008-04-08 12 +203 val_203 2008-04-08 12 +205 val_205 2008-04-08 11 +205 val_205 2008-04-08 11 +205 val_205 2008-04-08 12 +205 val_205 2008-04-08 12 +207 val_207 2008-04-08 11 +207 val_207 2008-04-08 11 +207 val_207 2008-04-08 12 +207 val_207 2008-04-08 12 +208 val_208 2008-04-08 11 +208 val_208 2008-04-08 11 +208 val_208 2008-04-08 11 +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 11 +209 val_209 2008-04-08 11 +209 val_209 2008-04-08 12 +209 val_209 2008-04-08 12 +213 val_213 2008-04-08 11 +213 val_213 2008-04-08 11 +213 val_213 2008-04-08 12 +213 val_213 2008-04-08 12 +214 val_214 2008-04-08 11 +214 val_214 2008-04-08 12 +216 val_216 2008-04-08 11 +216 val_216 2008-04-08 11 +216 val_216 2008-04-08 12 +216 val_216 2008-04-08 12 +217 val_217 2008-04-08 11 +217 val_217 2008-04-08 11 +217 val_217 2008-04-08 12 +217 val_217 2008-04-08 12 +218 val_218 2008-04-08 11 +218 val_218 2008-04-08 12 +219 val_219 2008-04-08 11 +219 val_219 2008-04-08 11 +219 val_219 2008-04-08 12 +219 val_219 2008-04-08 12 +221 val_221 2008-04-08 11 +221 val_221 2008-04-08 11 +221 val_221 2008-04-08 12 +221 val_221 2008-04-08 12 +222 val_222 2008-04-08 11 +222 val_222 2008-04-08 12 +223 val_223 2008-04-08 11 +223 val_223 2008-04-08 11 +223 val_223 2008-04-08 12 +223 val_223 2008-04-08 12 +224 val_224 2008-04-08 11 +224 val_224 2008-04-08 11 +224 val_224 2008-04-08 12 +224 val_224 2008-04-08 12 +226 val_226 2008-04-08 11 +226 val_226 2008-04-08 12 +228 val_228 2008-04-08 11 +228 val_228 2008-04-08 12 +229 val_229 2008-04-08 11 +229 val_229 2008-04-08 11 +229 val_229 2008-04-08 12 +229 val_229 2008-04-08 12 +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 +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 11 +233 val_233 2008-04-08 11 +233 val_233 2008-04-08 12 +233 val_233 2008-04-08 12 +235 val_235 2008-04-08 11 +235 val_235 2008-04-08 12 +237 val_237 2008-04-08 11 +237 val_237 2008-04-08 11 +237 val_237 2008-04-08 12 +237 val_237 2008-04-08 12 +238 val_238 2008-04-08 11 +238 val_238 2008-04-08 11 +238 val_238 2008-04-08 12 +238 val_238 2008-04-08 12 +239 val_239 2008-04-08 11 +239 val_239 2008-04-08 11 +239 val_239 2008-04-08 12 +239 val_239 2008-04-08 12 +241 val_241 2008-04-08 11 +241 val_241 2008-04-08 12 +242 val_242 2008-04-08 11 +242 val_242 2008-04-08 11 +242 val_242 2008-04-08 12 +242 val_242 2008-04-08 12 +244 val_244 2008-04-08 11 +244 val_244 2008-04-08 12 +247 val_247 2008-04-08 11 +247 val_247 2008-04-08 12 +248 val_248 2008-04-08 11 +248 val_248 2008-04-08 12 +249 val_249 2008-04-08 11 +249 val_249 2008-04-08 12 +252 val_252 2008-04-08 11 +252 val_252 2008-04-08 12 +255 val_255 2008-04-08 11 +255 val_255 2008-04-08 11 +255 val_255 2008-04-08 12 +255 val_255 2008-04-08 12 +256 val_256 2008-04-08 11 +256 val_256 2008-04-08 11 +256 val_256 2008-04-08 12 +256 val_256 2008-04-08 12 +257 val_257 2008-04-08 11 +257 val_257 2008-04-08 12 +258 val_258 2008-04-08 11 +258 val_258 2008-04-08 12 +260 val_260 2008-04-08 11 +260 val_260 2008-04-08 12 +262 val_262 2008-04-08 11 +262 val_262 2008-04-08 12 +263 val_263 2008-04-08 11 +263 val_263 2008-04-08 12 +265 val_265 2008-04-08 11 +265 val_265 2008-04-08 11 +265 val_265 2008-04-08 12 +265 val_265 2008-04-08 12 +266 val_266 2008-04-08 11 +266 val_266 2008-04-08 12 +272 val_272 2008-04-08 11 +272 val_272 2008-04-08 11 +272 val_272 2008-04-08 12 +272 val_272 2008-04-08 12 +273 val_273 2008-04-08 11 +273 val_273 2008-04-08 11 +273 val_273 2008-04-08 11 +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 11 +274 val_274 2008-04-08 12 +275 val_275 2008-04-08 11 +275 val_275 2008-04-08 12 +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 +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 11 +278 val_278 2008-04-08 11 +278 val_278 2008-04-08 12 +278 val_278 2008-04-08 12 +280 val_280 2008-04-08 11 +280 val_280 2008-04-08 11 +280 val_280 2008-04-08 12 +280 val_280 2008-04-08 12 +281 val_281 2008-04-08 11 +281 val_281 2008-04-08 11 +281 val_281 2008-04-08 12 +281 val_281 2008-04-08 12 +282 val_282 2008-04-08 11 +282 val_282 2008-04-08 11 +282 val_282 2008-04-08 12 +282 val_282 2008-04-08 12 +283 val_283 2008-04-08 11 +283 val_283 2008-04-08 12 +284 val_284 2008-04-08 11 +284 val_284 2008-04-08 12 +285 val_285 2008-04-08 11 +285 val_285 2008-04-08 12 +286 val_286 2008-04-08 11 +286 val_286 2008-04-08 12 +287 val_287 2008-04-08 11 +287 val_287 2008-04-08 12 +288 val_288 2008-04-08 11 +288 val_288 2008-04-08 11 +288 val_288 2008-04-08 12 +288 val_288 2008-04-08 12 +289 val_289 2008-04-08 11 +289 val_289 2008-04-08 12 +291 val_291 2008-04-08 11 +291 val_291 2008-04-08 12 +292 val_292 2008-04-08 11 +292 val_292 2008-04-08 12 +296 val_296 2008-04-08 11 +296 val_296 2008-04-08 12 +298 val_298 2008-04-08 11 +298 val_298 2008-04-08 11 +298 val_298 2008-04-08 11 +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 11 +302 val_302 2008-04-08 12 +305 val_305 2008-04-08 11 +305 val_305 2008-04-08 12 +306 val_306 2008-04-08 11 +306 val_306 2008-04-08 12 +307 val_307 2008-04-08 11 +307 val_307 2008-04-08 11 +307 val_307 2008-04-08 12 +307 val_307 2008-04-08 12 +308 val_308 2008-04-08 11 +308 val_308 2008-04-08 12 +309 val_309 2008-04-08 11 +309 val_309 2008-04-08 11 +309 val_309 2008-04-08 12 +309 val_309 2008-04-08 12 +310 val_310 2008-04-08 11 +310 val_310 2008-04-08 12 +311 val_311 2008-04-08 11 +311 val_311 2008-04-08 11 +311 val_311 2008-04-08 11 +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 11 +315 val_315 2008-04-08 12 +316 val_316 2008-04-08 11 +316 val_316 2008-04-08 11 +316 val_316 2008-04-08 11 +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 11 +317 val_317 2008-04-08 11 +317 val_317 2008-04-08 12 +317 val_317 2008-04-08 12 +318 val_318 2008-04-08 11 +318 val_318 2008-04-08 11 +318 val_318 2008-04-08 11 +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 11 +321 val_321 2008-04-08 11 +321 val_321 2008-04-08 12 +321 val_321 2008-04-08 12 +322 val_322 2008-04-08 11 +322 val_322 2008-04-08 11 +322 val_322 2008-04-08 12 +322 val_322 2008-04-08 12 +323 val_323 2008-04-08 11 +323 val_323 2008-04-08 12 +325 val_325 2008-04-08 11 +325 val_325 2008-04-08 11 +325 val_325 2008-04-08 12 +325 val_325 2008-04-08 12 +327 val_327 2008-04-08 11 +327 val_327 2008-04-08 11 +327 val_327 2008-04-08 11 +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 11 +331 val_331 2008-04-08 11 +331 val_331 2008-04-08 12 +331 val_331 2008-04-08 12 +332 val_332 2008-04-08 11 +332 val_332 2008-04-08 12 +333 val_333 2008-04-08 11 +333 val_333 2008-04-08 11 +333 val_333 2008-04-08 12 +333 val_333 2008-04-08 12 +335 val_335 2008-04-08 11 +335 val_335 2008-04-08 12 +336 val_336 2008-04-08 11 +336 val_336 2008-04-08 12 +338 val_338 2008-04-08 11 +338 val_338 2008-04-08 12 +339 val_339 2008-04-08 11 +339 val_339 2008-04-08 12 +341 val_341 2008-04-08 11 +341 val_341 2008-04-08 12 +342 val_342 2008-04-08 11 +342 val_342 2008-04-08 11 +342 val_342 2008-04-08 12 +342 val_342 2008-04-08 12 +344 val_344 2008-04-08 11 +344 val_344 2008-04-08 11 +344 val_344 2008-04-08 12 +344 val_344 2008-04-08 12 +345 val_345 2008-04-08 11 +345 val_345 2008-04-08 12 +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 +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 11 +351 val_351 2008-04-08 12 +353 val_353 2008-04-08 11 +353 val_353 2008-04-08 11 +353 val_353 2008-04-08 12 +353 val_353 2008-04-08 12 +356 val_356 2008-04-08 11 +356 val_356 2008-04-08 12 +360 val_360 2008-04-08 11 +360 val_360 2008-04-08 12 +362 val_362 2008-04-08 11 +362 val_362 2008-04-08 12 +364 val_364 2008-04-08 11 +364 val_364 2008-04-08 12 +365 val_365 2008-04-08 11 +365 val_365 2008-04-08 12 +366 val_366 2008-04-08 11 +366 val_366 2008-04-08 12 +367 val_367 2008-04-08 11 +367 val_367 2008-04-08 11 +367 val_367 2008-04-08 12 +367 val_367 2008-04-08 12 +368 val_368 2008-04-08 11 +368 val_368 2008-04-08 12 +369 val_369 2008-04-08 11 +369 val_369 2008-04-08 11 +369 val_369 2008-04-08 11 +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 11 +373 val_373 2008-04-08 12 +374 val_374 2008-04-08 11 +374 val_374 2008-04-08 12 +375 val_375 2008-04-08 11 +375 val_375 2008-04-08 12 +377 val_377 2008-04-08 11 +377 val_377 2008-04-08 12 +378 val_378 2008-04-08 11 +378 val_378 2008-04-08 12 +379 val_379 2008-04-08 11 +379 val_379 2008-04-08 12 +382 val_382 2008-04-08 11 +382 val_382 2008-04-08 11 +382 val_382 2008-04-08 12 +382 val_382 2008-04-08 12 +384 val_384 2008-04-08 11 +384 val_384 2008-04-08 11 +384 val_384 2008-04-08 11 +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 11 +386 val_386 2008-04-08 12 +389 val_389 2008-04-08 11 +389 val_389 2008-04-08 12 +392 val_392 2008-04-08 11 +392 val_392 2008-04-08 12 +393 val_393 2008-04-08 11 +393 val_393 2008-04-08 12 +394 val_394 2008-04-08 11 +394 val_394 2008-04-08 12 +395 val_395 2008-04-08 11 +395 val_395 2008-04-08 11 +395 val_395 2008-04-08 12 +395 val_395 2008-04-08 12 +396 val_396 2008-04-08 11 +396 val_396 2008-04-08 11 +396 val_396 2008-04-08 11 +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 11 +397 val_397 2008-04-08 11 +397 val_397 2008-04-08 12 +397 val_397 2008-04-08 12 +399 val_399 2008-04-08 11 +399 val_399 2008-04-08 11 +399 val_399 2008-04-08 12 +399 val_399 2008-04-08 12 +400 val_400 2008-04-08 11 +400 val_400 2008-04-08 12 +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 +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 11 +402 val_402 2008-04-08 12 +403 val_403 2008-04-08 11 +403 val_403 2008-04-08 11 +403 val_403 2008-04-08 11 +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 11 +404 val_404 2008-04-08 11 +404 val_404 2008-04-08 12 +404 val_404 2008-04-08 12 +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 +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 11 +407 val_407 2008-04-08 12 +409 val_409 2008-04-08 11 +409 val_409 2008-04-08 11 +409 val_409 2008-04-08 11 +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 11 +411 val_411 2008-04-08 12 +413 val_413 2008-04-08 11 +413 val_413 2008-04-08 11 +413 val_413 2008-04-08 12 +413 val_413 2008-04-08 12 +414 val_414 2008-04-08 11 +414 val_414 2008-04-08 11 +414 val_414 2008-04-08 12 +414 val_414 2008-04-08 12 +417 val_417 2008-04-08 11 +417 val_417 2008-04-08 11 +417 val_417 2008-04-08 11 +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 11 +418 val_418 2008-04-08 12 +419 val_419 2008-04-08 11 +419 val_419 2008-04-08 12 +421 val_421 2008-04-08 11 +421 val_421 2008-04-08 12 +424 val_424 2008-04-08 11 +424 val_424 2008-04-08 11 +424 val_424 2008-04-08 12 +424 val_424 2008-04-08 12 +427 val_427 2008-04-08 11 +427 val_427 2008-04-08 12 +429 val_429 2008-04-08 11 +429 val_429 2008-04-08 11 +429 val_429 2008-04-08 12 +429 val_429 2008-04-08 12 +430 val_430 2008-04-08 11 +430 val_430 2008-04-08 11 +430 val_430 2008-04-08 11 +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 11 +431 val_431 2008-04-08 11 +431 val_431 2008-04-08 11 +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 11 +432 val_432 2008-04-08 12 +435 val_435 2008-04-08 11 +435 val_435 2008-04-08 12 +436 val_436 2008-04-08 11 +436 val_436 2008-04-08 12 +437 val_437 2008-04-08 11 +437 val_437 2008-04-08 12 +438 val_438 2008-04-08 11 +438 val_438 2008-04-08 11 +438 val_438 2008-04-08 11 +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 11 +439 val_439 2008-04-08 11 +439 val_439 2008-04-08 12 +439 val_439 2008-04-08 12 +443 val_443 2008-04-08 11 +443 val_443 2008-04-08 12 +444 val_444 2008-04-08 11 +444 val_444 2008-04-08 12 +446 val_446 2008-04-08 11 +446 val_446 2008-04-08 12 +448 val_448 2008-04-08 11 +448 val_448 2008-04-08 12 +449 val_449 2008-04-08 11 +449 val_449 2008-04-08 12 +452 val_452 2008-04-08 11 +452 val_452 2008-04-08 12 +453 val_453 2008-04-08 11 +453 val_453 2008-04-08 12 +454 val_454 2008-04-08 11 +454 val_454 2008-04-08 11 +454 val_454 2008-04-08 11 +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 11 +455 val_455 2008-04-08 12 +457 val_457 2008-04-08 11 +457 val_457 2008-04-08 12 +458 val_458 2008-04-08 11 +458 val_458 2008-04-08 11 +458 val_458 2008-04-08 12 +458 val_458 2008-04-08 12 +459 val_459 2008-04-08 11 +459 val_459 2008-04-08 11 +459 val_459 2008-04-08 12 +459 val_459 2008-04-08 12 +460 val_460 2008-04-08 11 +460 val_460 2008-04-08 12 +462 val_462 2008-04-08 11 +462 val_462 2008-04-08 11 +462 val_462 2008-04-08 12 +462 val_462 2008-04-08 12 +463 val_463 2008-04-08 11 +463 val_463 2008-04-08 11 +463 val_463 2008-04-08 12 +463 val_463 2008-04-08 12 +466 val_466 2008-04-08 11 +466 val_466 2008-04-08 11 +466 val_466 2008-04-08 11 +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 11 +467 val_467 2008-04-08 12 +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 +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 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 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 11 +470 val_470 2008-04-08 12 +472 val_472 2008-04-08 11 +472 val_472 2008-04-08 12 +475 val_475 2008-04-08 11 +475 val_475 2008-04-08 12 +477 val_477 2008-04-08 11 +477 val_477 2008-04-08 12 +478 val_478 2008-04-08 11 +478 val_478 2008-04-08 11 +478 val_478 2008-04-08 12 +478 val_478 2008-04-08 12 +479 val_479 2008-04-08 11 +479 val_479 2008-04-08 12 +480 val_480 2008-04-08 11 +480 val_480 2008-04-08 11 +480 val_480 2008-04-08 11 +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 11 +481 val_481 2008-04-08 12 +482 val_482 2008-04-08 11 +482 val_482 2008-04-08 12 +483 val_483 2008-04-08 11 +483 val_483 2008-04-08 12 +484 val_484 2008-04-08 11 +484 val_484 2008-04-08 12 +485 val_485 2008-04-08 11 +485 val_485 2008-04-08 12 +487 val_487 2008-04-08 11 +487 val_487 2008-04-08 12 +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 +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 11 +490 val_490 2008-04-08 12 +491 val_491 2008-04-08 11 +491 val_491 2008-04-08 12 +492 val_492 2008-04-08 11 +492 val_492 2008-04-08 11 +492 val_492 2008-04-08 12 +492 val_492 2008-04-08 12 +493 val_493 2008-04-08 11 +493 val_493 2008-04-08 12 +494 val_494 2008-04-08 11 +494 val_494 2008-04-08 12 +495 val_495 2008-04-08 11 +495 val_495 2008-04-08 12 +496 val_496 2008-04-08 11 +496 val_496 2008-04-08 12 +497 val_497 2008-04-08 11 +497 val_497 2008-04-08 12 +498 val_498 2008-04-08 11 +498 val_498 2008-04-08 11 +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 diff --git a/src/test/hive/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 b/src/test/hive/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e b/src/test/hive/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf b/src/test/hive/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 b/src/test/hive/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 b/src/test/hive/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c b/src/test/hive/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 b/src/test/hive/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/src/test/hive/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 b/src/test/hive/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f b/src/test/hive/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 b/src/test/hive/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..c1a6abba038e7 --- /dev/null +++ b/src/test/hive/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,11 @@ +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/src/test/hive/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 b/src/test/hive/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 b/src/test/hive/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd b/src/test/hive/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..51645b2a07a39 --- /dev/null +++ b/src/test/hive/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +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/src/test/hive/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac b/src/test/hive/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 b/src/test/hive/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 new file mode 100644 index 0000000000000..373a573714f4e --- /dev/null +++ b/src/test/hive/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 @@ -0,0 +1,11 @@ +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/src/test/hive/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 b/src/test/hive/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 b/src/test/hive/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 b/src/test/hive/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 new file mode 100644 index 0000000000000..7490d2d44d71a --- /dev/null +++ b/src/test/hive/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 b/src/test/hive/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 b/src/test/hive/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 new file mode 100644 index 0000000000000..7490d2d44d71a --- /dev/null +++ b/src/test/hive/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 b/src/test/hive/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 b/src/test/hive/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 new file mode 100644 index 0000000000000..d5a489a9a5ed5 --- /dev/null +++ b/src/test/hive/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 @@ -0,0 +1,15 @@ +viewtime string None +userid int None +page_url string None +referrer_url string None +friends array None +properties map None +ip string IP Address of the User +ds string None +country string None + +# Partition Information +# col_name data_type comment + +ds string None +country string None \ No newline at end of file diff --git a/src/test/hive/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 b/src/test/hive/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 new file mode 100644 index 0000000000000..e0bedb0512cfc --- /dev/null +++ b/src/test/hive/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 @@ -0,0 +1,17 @@ +viewtime string None +userid int None +page_url string None +referrer_url string None +friends array None +properties map None +ip string IP Address of the User +ds string None +country string None + +# Partition Information +# col_name data_type comment + +ds string None +country string None + +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, comment:null), FieldSchema(name:properties, type:map, 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 diff --git a/src/test/hive/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 b/src/test/hive/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf b/src/test/hive/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c b/src/test/hive/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c b/src/test/hive/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c new file mode 100644 index 0000000000000..f1427896e8330 --- /dev/null +++ b/src/test/hive/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c @@ -0,0 +1,10 @@ +key string None +value string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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 diff --git a/src/test/hive/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 b/src/test/hive/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 new file mode 100644 index 0000000000000..822897217e867 --- /dev/null +++ b/src/test/hive/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 @@ -0,0 +1,10 @@ +key string None +value string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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 diff --git a/src/test/hive/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 b/src/test/hive/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 new file mode 100644 index 0000000000000..94bcaaee2408c --- /dev/null +++ b/src/test/hive/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 @@ -0,0 +1,2 @@ +ds=2008-04-08 +ds=2008-04-09 \ No newline at end of file diff --git a/src/test/hive/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 b/src/test/hive/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 b/src/test/hive/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 new file mode 100644 index 0000000000000..b12a9f82cd90a --- /dev/null +++ b/src/test/hive/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 @@ -0,0 +1 @@ +ds=2008-04-09 \ No newline at end of file diff --git a/src/test/hive/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 b/src/test/hive/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 new file mode 100644 index 0000000000000..a6c282ab6f573 --- /dev/null +++ b/src/test/hive/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 @@ -0,0 +1,19 @@ +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/src/test/hive/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 b/src/test/hive/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f b/src/test/hive/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 b/src/test/hive/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d b/src/test/hive/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 b/src/test/hive/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 b/src/test/hive/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 new file mode 100644 index 0000000000000..ef633a4aa09e0 --- /dev/null +++ b/src/test/hive/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 @@ -0,0 +1,3 @@ +name string None + +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 diff --git a/src/test/hive/golden/inputddl7-13-45059a21c202b4658285738ee62a018a b/src/test/hive/golden/inputddl7-13-45059a21c202b4658285738ee62a018a new file mode 100644 index 0000000000000..86d5d8a125fbe --- /dev/null +++ b/src/test/hive/golden/inputddl7-13-45059a21c202b4658285738ee62a018a @@ -0,0 +1,3 @@ +name string None + +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 diff --git a/src/test/hive/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 b/src/test/hive/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 new file mode 100644 index 0000000000000..b14d78536150c --- /dev/null +++ b/src/test/hive/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 @@ -0,0 +1,9 @@ +name string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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 diff --git a/src/test/hive/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe b/src/test/hive/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe new file mode 100644 index 0000000000000..e476a7fa3959b --- /dev/null +++ b/src/test/hive/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe @@ -0,0 +1,9 @@ +name string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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 diff --git a/src/test/hive/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 b/src/test/hive/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 b/src/test/hive/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 b/src/test/hive/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 b/src/test/hive/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a b/src/test/hive/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 b/src/test/hive/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 b/src/test/hive/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac b/src/test/hive/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 b/src/test/hive/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 b/src/test/hive/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 new file mode 100644 index 0000000000000..66e6efceed3dc --- /dev/null +++ b/src/test/hive/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 @@ -0,0 +1,16 @@ +aint int from deserializer +astring string from deserializer +lint array from deserializer +lstring array from deserializer +lintstring array> from deserializer +mstringstring map from deserializer +ds string None +country string None + +# Partition Information +# col_name data_type comment + +ds string None +country string None + +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 diff --git a/src/test/hive/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac b/src/test/hive/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 b/src/test/hive/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 b/src/test/hive/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 b/src/test/hive/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/src/test/hive/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/src/test/hive/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c b/src/test/hive/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d b/src/test/hive/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 b/src/test/hive/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 b/src/test/hive/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 b/src/test/hive/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef b/src/test/hive/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 b/src/test/hive/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef b/src/test/hive/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef new file mode 100644 index 0000000000000..e37d32abba426 --- /dev/null +++ b/src/test/hive/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef @@ -0,0 +1 @@ +1000 \ No newline at end of file diff --git a/src/test/hive/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 b/src/test/hive/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef b/src/test/hive/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef new file mode 100644 index 0000000000000..37021f4a27201 --- /dev/null +++ b/src/test/hive/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef @@ -0,0 +1 @@ +1500 \ No newline at end of file diff --git a/src/test/hive/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 b/src/test/hive/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 b/src/test/hive/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b b/src/test/hive/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 b/src/test/hive/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb b/src/test/hive/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb new file mode 100644 index 0000000000000..5e96d815b6b78 --- /dev/null +++ b/src/test/hive/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb @@ -0,0 +1 @@ +-826625916 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 b/src/test/hive/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 b/src/test/hive/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 b/src/test/hive/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb b/src/test/hive/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb new file mode 100644 index 0000000000000..eb9dc5833c2f9 --- /dev/null +++ b/src/test/hive/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb @@ -0,0 +1 @@ +10226524244 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 b/src/test/hive/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 b/src/test/hive/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb b/src/test/hive/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb new file mode 100644 index 0000000000000..28ced898ab537 --- /dev/null +++ b/src/test/hive/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb @@ -0,0 +1 @@ +20453048488 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d b/src/test/hive/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d new file mode 100644 index 0000000000000..ae4ee13c08e76 --- /dev/null +++ b/src/test/hive/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d @@ -0,0 +1 @@ +200 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b b/src/test/hive/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 b/src/test/hive/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 b/src/test/hive/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 b/src/test/hive/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa b/src/test/hive/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 b/src/test/hive/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 new file mode 100644 index 0000000000000..84d7f3929d86d --- /dev/null +++ b/src/test/hive/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 @@ -0,0 +1 @@ +-27100860056 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 b/src/test/hive/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 b/src/test/hive/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 b/src/test/hive/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 b/src/test/hive/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 b/src/test/hive/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 new file mode 100644 index 0000000000000..ae4ee13c08e76 --- /dev/null +++ b/src/test/hive/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 @@ -0,0 +1 @@ +200 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 b/src/test/hive/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 new file mode 100644 index 0000000000000..3395f3bcc7b51 --- /dev/null +++ b/src/test/hive/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 @@ -0,0 +1 @@ +-24159954504 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 b/src/test/hive/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-8-452111285dda40205ee587de8e972896 b/src/test/hive/golden/insert_into2-8-452111285dda40205ee587de8e972896 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 b/src/test/hive/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 new file mode 100644 index 0000000000000..ee0a47c9f6e00 --- /dev/null +++ b/src/test/hive/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 @@ -0,0 +1 @@ +-36239931656 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 b/src/test/hive/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 b/src/test/hive/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 b/src/test/hive/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 new file mode 100644 index 0000000000000..5e96d815b6b78 --- /dev/null +++ b/src/test/hive/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 @@ -0,0 +1 @@ +-826625916 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 b/src/test/hive/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 new file mode 100644 index 0000000000000..2ed5a7da11dcd --- /dev/null +++ b/src/test/hive/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 @@ -0,0 +1 @@ +9399898328 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 b/src/test/hive/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 b/src/test/hive/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into3-2-e51c25bae2408422a56826a263479468 b/src/test/hive/golden/insert_into3-2-e51c25bae2408422a56826a263479468 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d b/src/test/hive/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd b/src/test/hive/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 b/src/test/hive/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 b/src/test/hive/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 new file mode 100644 index 0000000000000..00ffdd24b0cff --- /dev/null +++ b/src/test/hive/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 @@ -0,0 +1 @@ +7813690682 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 b/src/test/hive/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 new file mode 100644 index 0000000000000..eb9dc5833c2f9 --- /dev/null +++ b/src/test/hive/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 @@ -0,0 +1 @@ +10226524244 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 b/src/test/hive/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 b/src/test/hive/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 b/src/test/hive/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 b/src/test/hive/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 b/src/test/hive/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 new file mode 100644 index 0000000000000..e2954bd63682e --- /dev/null +++ b/src/test/hive/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 @@ -0,0 +1 @@ +-1653251832 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa b/src/test/hive/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 b/src/test/hive/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b b/src/test/hive/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b new file mode 100644 index 0000000000000..e2954bd63682e --- /dev/null +++ b/src/test/hive/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b @@ -0,0 +1 @@ +-1653251832 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 b/src/test/hive/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 b/src/test/hive/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 b/src/test/hive/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b b/src/test/hive/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a b/src/test/hive/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 b/src/test/hive/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 b/src/test/hive/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 b/src/test/hive/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 new file mode 100644 index 0000000000000..5e96d815b6b78 --- /dev/null +++ b/src/test/hive/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 @@ -0,0 +1 @@ +-826625916 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 b/src/test/hive/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 b/src/test/hive/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 b/src/test/hive/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c b/src/test/hive/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 b/src/test/hive/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d b/src/test/hive/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 b/src/test/hive/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 new file mode 100644 index 0000000000000..3b6ef434b37e7 --- /dev/null +++ b/src/test/hive/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 @@ -0,0 +1 @@ +-18626052920 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 b/src/test/hive/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 b/src/test/hive/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 b/src/test/hive/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 new file mode 100644 index 0000000000000..0744f3dae0e0a --- /dev/null +++ b/src/test/hive/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 @@ -0,0 +1 @@ +-37252105840 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 b/src/test/hive/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c b/src/test/hive/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 b/src/test/hive/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 b/src/test/hive/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 b/src/test/hive/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 b/src/test/hive/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 new file mode 100644 index 0000000000000..c0066b75af40e --- /dev/null +++ b/src/test/hive/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 @@ -0,0 +1 @@ +481928560 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f b/src/test/hive/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 b/src/test/hive/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 b/src/test/hive/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 new file mode 100644 index 0000000000000..0b4a44e064f85 --- /dev/null +++ b/src/test/hive/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 @@ -0,0 +1 @@ +963857120 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 b/src/test/hive/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba b/src/test/hive/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 b/src/test/hive/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 new file mode 100644 index 0000000000000..d541b5d207233 --- /dev/null +++ b/src/test/hive/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 @@ -0,0 +1 @@ +-35226404960 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 b/src/test/hive/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 new file mode 100644 index 0000000000000..c21f4017362c1 --- /dev/null +++ b/src/test/hive/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 @@ -0,0 +1,2 @@ +ds=1 +ds=2 \ No newline at end of file diff --git a/src/test/hive/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 b/src/test/hive/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/src/test/hive/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 b/src/test/hive/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/src/test/hive/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f b/src/test/hive/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 b/src/test/hive/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 b/src/test/hive/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 b/src/test/hive/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 b/src/test/hive/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab b/src/test/hive/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab new file mode 100644 index 0000000000000..d541b5d207233 --- /dev/null +++ b/src/test/hive/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab @@ -0,0 +1 @@ +-35226404960 \ No newline at end of file diff --git a/src/test/hive/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 b/src/test/hive/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a b/src/test/hive/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b b/src/test/hive/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 b/src/test/hive/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 b/src/test/hive/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join0-1-83f5e512d6058a47f92b5218781f5070 b/src/test/hive/golden/join0-1-83f5e512d6058a47f92b5218781f5070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 b/src/test/hive/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 new file mode 100644 index 0000000000000..5707ed08e7e54 --- /dev/null +++ b/src/test/hive/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 @@ -0,0 +1,100 @@ +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 +0 val_0 2 val_2 +0 val_0 2 val_2 +0 val_0 4 val_4 +0 val_0 4 val_4 +0 val_0 4 val_4 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 8 val_8 +0 val_0 8 val_8 +0 val_0 8 val_8 +0 val_0 9 val_9 +0 val_0 9 val_9 +0 val_0 9 val_9 +2 val_2 0 val_0 +2 val_2 0 val_0 +2 val_2 0 val_0 +2 val_2 2 val_2 +2 val_2 4 val_4 +2 val_2 5 val_5 +2 val_2 5 val_5 +2 val_2 5 val_5 +2 val_2 8 val_8 +2 val_2 9 val_9 +4 val_4 0 val_0 +4 val_4 0 val_0 +4 val_4 0 val_0 +4 val_4 2 val_2 +4 val_4 4 val_4 +4 val_4 5 val_5 +4 val_4 5 val_5 +4 val_4 5 val_5 +4 val_4 8 val_8 +4 val_4 9 val_9 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 2 val_2 +5 val_5 2 val_2 +5 val_5 2 val_2 +5 val_5 4 val_4 +5 val_5 4 val_4 +5 val_5 4 val_4 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 8 val_8 +5 val_5 8 val_8 +5 val_5 8 val_8 +5 val_5 9 val_9 +5 val_5 9 val_9 +5 val_5 9 val_9 +8 val_8 0 val_0 +8 val_8 0 val_0 +8 val_8 0 val_0 +8 val_8 2 val_2 +8 val_8 4 val_4 +8 val_8 5 val_5 +8 val_8 5 val_5 +8 val_8 5 val_5 +8 val_8 8 val_8 +8 val_8 9 val_9 +9 val_9 0 val_0 +9 val_9 0 val_0 +9 val_9 0 val_0 +9 val_9 2 val_2 +9 val_9 4 val_4 +9 val_9 5 val_5 +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 diff --git a/src/test/hive/golden/join1-0-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/join1-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join1-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e b/src/test/hive/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join1-2-7b07671864bbfdc627ee794932e32b1e b/src/test/hive/golden/join1-2-7b07671864bbfdc627ee794932e32b1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 b/src/test/hive/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join1-4-35c7611180562dcd9dab834f41654095 b/src/test/hive/golden/join1-4-35c7611180562dcd9dab834f41654095 new file mode 100644 index 0000000000000..59821aeea3008 --- /dev/null +++ b/src/test/hive/golden/join1-4-35c7611180562dcd9dab834f41654095 @@ -0,0 +1,1028 @@ +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +374 val_374 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +82 val_82 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +459 val_459 +394 val_394 +237 val_237 +237 val_237 +482 val_482 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +494 val_494 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +0 val_0 +0 val_0 +0 val_0 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +4 val_4 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +54 val_54 +459 val_459 +459 val_459 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +12 val_12 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +260 val_260 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +58 val_58 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +0 val_0 +0 val_0 +0 val_0 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +404 val_404 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +0 val_0 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +419 val_419 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +78 val_78 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +209 val_209 +64 val_64 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +12 val_12 +12 val_12 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +491 val_491 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +18 val_18 +18 val_18 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +448 val_448 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +414 val_414 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +200 val_200 +200 val_200 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 b/src/test/hive/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d b/src/test/hive/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d new file mode 100644 index 0000000000000..59821aeea3008 --- /dev/null +++ b/src/test/hive/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d @@ -0,0 +1,1028 @@ +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +374 val_374 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +82 val_82 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +459 val_459 +394 val_394 +237 val_237 +237 val_237 +482 val_482 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +494 val_494 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +0 val_0 +0 val_0 +0 val_0 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +4 val_4 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +54 val_54 +459 val_459 +459 val_459 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +12 val_12 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +260 val_260 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +58 val_58 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +0 val_0 +0 val_0 +0 val_0 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +404 val_404 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +0 val_0 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +419 val_419 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +78 val_78 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +209 val_209 +64 val_64 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +12 val_12 +12 val_12 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +491 val_491 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +18 val_18 +18 val_18 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +448 val_448 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +414 val_414 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +200 val_200 +200 val_200 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 b/src/test/hive/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 b/src/test/hive/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 new file mode 100644 index 0000000000000..c888cfd6a0479 --- /dev/null +++ b/src/test/hive/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 @@ -0,0 +1,148 @@ +86 val_86 +27 val_27 +98 val_98 +98 val_98 +66 val_66 +37 val_37 +37 val_37 +15 val_15 +15 val_15 +82 val_82 +17 val_17 +0 val_0 +0 val_0 +0 val_0 +57 val_57 +20 val_20 +92 val_92 +47 val_47 +72 val_72 +72 val_72 +4 val_4 +35 val_35 +35 val_35 +35 val_35 +54 val_54 +51 val_51 +51 val_51 +65 val_65 +83 val_83 +83 val_83 +12 val_12 +12 val_12 +67 val_67 +67 val_67 +84 val_84 +84 val_84 +58 val_58 +58 val_58 +8 val_8 +24 val_24 +24 val_24 +42 val_42 +42 val_42 +0 val_0 +0 val_0 +0 val_0 +96 val_96 +26 val_26 +26 val_26 +51 val_51 +51 val_51 +43 val_43 +95 val_95 +95 val_95 +98 val_98 +98 val_98 +85 val_85 +77 val_77 +0 val_0 +0 val_0 +0 val_0 +87 val_87 +15 val_15 +15 val_15 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +19 val_19 +10 val_10 +5 val_5 +5 val_5 +5 val_5 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +95 val_95 +95 val_95 +11 val_11 +34 val_34 +42 val_42 +42 val_42 +78 val_78 +76 val_76 +76 val_76 +41 val_41 +30 val_30 +64 val_64 +76 val_76 +76 val_76 +74 val_74 +69 val_69 +33 val_33 +70 val_70 +70 val_70 +70 val_70 +5 val_5 +5 val_5 +5 val_5 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +80 val_80 +44 val_44 +53 val_53 +90 val_90 +90 val_90 +90 val_90 +12 val_12 +12 val_12 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +24 val_24 +24 val_24 +70 val_70 +70 val_70 +70 val_70 +83 val_83 +83 val_83 +26 val_26 +26 val_26 +67 val_67 +67 val_67 +18 val_18 +18 val_18 +9 val_9 +18 val_18 +18 val_18 +97 val_97 +97 val_97 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +90 val_90 +90 val_90 +90 val_90 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 b/src/test/hive/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f b/src/test/hive/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f new file mode 100644 index 0000000000000..1780076ae513f --- /dev/null +++ b/src/test/hive/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f @@ -0,0 +1,232 @@ +27 val_27 +66 val_66 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +17 val_17 +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 +57 val_57 +20 val_20 +47 val_47 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +4 val_4 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +54 val_54 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +65 val_65 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +8 val_8 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +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 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +43 val_43 +77 val_77 +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 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +19 val_19 +10 val_10 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +11 val_11 +34 val_34 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +78 val_78 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +41 val_41 +30 val_30 +64 val_64 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +74 val_74 +69 val_69 +33 val_33 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +44 val_44 +53 val_53 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +9 val_9 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +28 val_28 +37 val_37 +37 val_37 +37 val_37 +37 val_37 \ No newline at end of file diff --git a/src/test/hive/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 b/src/test/hive/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 b/src/test/hive/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 new file mode 100644 index 0000000000000..a82a9a22c6b87 --- /dev/null +++ b/src/test/hive/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 @@ -0,0 +1,164 @@ +86 val_86 +86 val_86 +27 val_27 +98 val_98 +98 val_98 +37 val_37 +37 val_37 +15 val_15 +15 val_15 +82 val_82 +82 val_82 +17 val_17 +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 +57 val_57 +4 val_4 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +83 val_83 +83 val_83 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +84 val_84 +84 val_84 +58 val_58 +58 val_58 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +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 +96 val_96 +43 val_43 +95 val_95 +95 val_95 +98 val_98 +98 val_98 +85 val_85 +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 +87 val_87 +87 val_87 +15 val_15 +15 val_15 +90 val_90 +90 val_90 +90 val_90 +10 val_10 +5 val_5 +5 val_5 +5 val_5 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +95 val_95 +95 val_95 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +78 val_78 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +41 val_41 +64 val_64 +64 val_64 +64 val_64 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +69 val_69 +69 val_69 +69 val_69 +69 val_69 +33 val_33 +5 val_5 +5 val_5 +5 val_5 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +80 val_80 +90 val_90 +90 val_90 +90 val_90 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +5 val_5 +5 val_5 +5 val_5 +83 val_83 +83 val_83 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +9 val_9 +9 val_9 +97 val_97 +97 val_97 +84 val_84 +84 val_84 +37 val_37 +37 val_37 +90 val_90 +90 val_90 +90 val_90 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 b/src/test/hive/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e b/src/test/hive/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 b/src/test/hive/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 b/src/test/hive/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 b/src/test/hive/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 b/src/test/hive/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join14-6-de39302191b63d7aa8f92885b089fe2 b/src/test/hive/golden/join14-6-de39302191b63d7aa8f92885b089fe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/src/test/hive/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 new file mode 100644 index 0000000000000..941e6bb7c887e --- /dev/null +++ b/src/test/hive/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 @@ -0,0 +1,1752 @@ +238 val_238 +238 val_238 +311 val_311 +311 val_311 +311 val_311 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +374 val_374 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +495 val_495 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +459 val_459 +394 val_394 +237 val_237 +237 val_237 +482 val_482 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +494 val_494 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +475 val_475 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +111 val_111 +280 val_280 +280 val_280 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +459 val_459 +459 val_459 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +260 val_260 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +418 val_418 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +404 val_404 +404 val_404 +436 val_436 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +196 val_196 +288 val_288 +288 val_288 +481 val_481 +457 val_457 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +490 val_490 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +419 val_419 +118 val_118 +118 val_118 +307 val_307 +307 val_307 +435 val_435 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +497 val_497 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +336 val_336 +168 val_168 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +195 val_195 +195 val_195 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +209 val_209 +209 val_209 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +342 val_342 +342 val_342 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +105 val_105 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +258 val_258 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +491 val_491 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +134 val_134 +134 val_134 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +298 val_298 +298 val_298 +298 val_298 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +448 val_448 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +414 val_414 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +200 val_200 +200 val_200 +238 val_238 +238 val_238 +311 val_311 +311 val_311 +311 val_311 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +374 val_374 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +495 val_495 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +459 val_459 +394 val_394 +237 val_237 +237 val_237 +482 val_482 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +494 val_494 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +475 val_475 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +111 val_111 +280 val_280 +280 val_280 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +459 val_459 +459 val_459 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +260 val_260 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +418 val_418 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +404 val_404 +404 val_404 +436 val_436 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +196 val_196 +288 val_288 +288 val_288 +481 val_481 +457 val_457 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +490 val_490 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +419 val_419 +118 val_118 +118 val_118 +307 val_307 +307 val_307 +435 val_435 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +497 val_497 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +336 val_336 +168 val_168 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +195 val_195 +195 val_195 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +209 val_209 +209 val_209 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +342 val_342 +342 val_342 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +105 val_105 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +258 val_258 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +491 val_491 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +134 val_134 +134 val_134 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +298 val_298 +298 val_298 +298 val_298 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +448 val_448 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +414 val_414 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +200 val_200 +200 val_200 \ No newline at end of file diff --git a/src/test/hive/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 b/src/test/hive/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f b/src/test/hive/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 b/src/test/hive/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 b/src/test/hive/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 b/src/test/hive/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/src/test/hive/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 new file mode 100644 index 0000000000000..941e6bb7c887e --- /dev/null +++ b/src/test/hive/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 @@ -0,0 +1,1752 @@ +238 val_238 +238 val_238 +311 val_311 +311 val_311 +311 val_311 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +374 val_374 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +495 val_495 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +459 val_459 +394 val_394 +237 val_237 +237 val_237 +482 val_482 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +494 val_494 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +475 val_475 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +111 val_111 +280 val_280 +280 val_280 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +459 val_459 +459 val_459 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +260 val_260 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +418 val_418 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +404 val_404 +404 val_404 +436 val_436 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +196 val_196 +288 val_288 +288 val_288 +481 val_481 +457 val_457 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +490 val_490 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +419 val_419 +118 val_118 +118 val_118 +307 val_307 +307 val_307 +435 val_435 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +497 val_497 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +336 val_336 +168 val_168 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +195 val_195 +195 val_195 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +209 val_209 +209 val_209 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +342 val_342 +342 val_342 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +105 val_105 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +258 val_258 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +491 val_491 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +134 val_134 +134 val_134 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +298 val_298 +298 val_298 +298 val_298 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +448 val_448 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +414 val_414 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +200 val_200 +200 val_200 +238 val_238 +238 val_238 +311 val_311 +311 val_311 +311 val_311 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +374 val_374 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +495 val_495 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +459 val_459 +394 val_394 +237 val_237 +237 val_237 +482 val_482 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +494 val_494 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +475 val_475 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +111 val_111 +280 val_280 +280 val_280 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +459 val_459 +459 val_459 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +260 val_260 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +418 val_418 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +404 val_404 +404 val_404 +436 val_436 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +196 val_196 +288 val_288 +288 val_288 +481 val_481 +457 val_457 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +490 val_490 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +419 val_419 +118 val_118 +118 val_118 +307 val_307 +307 val_307 +435 val_435 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +497 val_497 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +336 val_336 +168 val_168 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +195 val_195 +195 val_195 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +209 val_209 +209 val_209 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +342 val_342 +342 val_342 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +105 val_105 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +258 val_258 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +491 val_491 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +134 val_134 +134 val_134 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +298 val_298 +298 val_298 +298 val_298 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +448 val_448 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +414 val_414 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +200 val_200 +200 val_200 \ No newline at end of file diff --git a/src/test/hive/golden/join15-0-930db2823512f7f8f34cac104f2162e0 b/src/test/hive/golden/join15-0-930db2823512f7f8f34cac104f2162e0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join15-1-81d76d3bf59889b07b413b6f88772667 b/src/test/hive/golden/join15-1-81d76d3bf59889b07b413b6f88772667 new file mode 100644 index 0000000000000..c2e3ea8b0c8e2 --- /dev/null +++ b/src/test/hive/golden/join15-1-81d76d3bf59889b07b413b6f88772667 @@ -0,0 +1,1028 @@ +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 2 val_2 +4 val_4 4 val_4 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +8 val_8 8 val_8 +9 val_9 9 val_9 +10 val_10 10 val_10 +11 val_11 11 val_11 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +17 val_17 17 val_17 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +19 val_19 19 val_19 +20 val_20 20 val_20 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +27 val_27 27 val_27 +28 val_28 28 val_28 +30 val_30 30 val_30 +33 val_33 33 val_33 +34 val_34 34 val_34 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +41 val_41 41 val_41 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +43 val_43 43 val_43 +44 val_44 44 val_44 +47 val_47 47 val_47 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +53 val_53 53 val_53 +54 val_54 54 val_54 +57 val_57 57 val_57 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +64 val_64 64 val_64 +65 val_65 65 val_65 +66 val_66 66 val_66 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +69 val_69 69 val_69 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +74 val_74 74 val_74 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +77 val_77 77 val_77 +78 val_78 78 val_78 +80 val_80 80 val_80 +82 val_82 82 val_82 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +85 val_85 85 val_85 +86 val_86 86 val_86 +87 val_87 87 val_87 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +92 val_92 92 val_92 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +96 val_96 96 val_96 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +98 val_98 98 val_98 +98 val_98 98 val_98 +98 val_98 98 val_98 +98 val_98 98 val_98 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +105 val_105 105 val_105 +111 val_111 111 val_111 +113 val_113 113 val_113 +113 val_113 113 val_113 +113 val_113 113 val_113 +113 val_113 113 val_113 +114 val_114 114 val_114 +116 val_116 116 val_116 +118 val_118 118 val_118 +118 val_118 118 val_118 +118 val_118 118 val_118 +118 val_118 118 val_118 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +120 val_120 120 val_120 +120 val_120 120 val_120 +120 val_120 120 val_120 +120 val_120 120 val_120 +125 val_125 125 val_125 +125 val_125 125 val_125 +125 val_125 125 val_125 +125 val_125 125 val_125 +126 val_126 126 val_126 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +129 val_129 129 val_129 +129 val_129 129 val_129 +129 val_129 129 val_129 +129 val_129 129 val_129 +131 val_131 131 val_131 +133 val_133 133 val_133 +134 val_134 134 val_134 +134 val_134 134 val_134 +134 val_134 134 val_134 +134 val_134 134 val_134 +136 val_136 136 val_136 +137 val_137 137 val_137 +137 val_137 137 val_137 +137 val_137 137 val_137 +137 val_137 137 val_137 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +143 val_143 143 val_143 +145 val_145 145 val_145 +146 val_146 146 val_146 +146 val_146 146 val_146 +146 val_146 146 val_146 +146 val_146 146 val_146 +149 val_149 149 val_149 +149 val_149 149 val_149 +149 val_149 149 val_149 +149 val_149 149 val_149 +150 val_150 150 val_150 +152 val_152 152 val_152 +152 val_152 152 val_152 +152 val_152 152 val_152 +152 val_152 152 val_152 +153 val_153 153 val_153 +155 val_155 155 val_155 +156 val_156 156 val_156 +157 val_157 157 val_157 +158 val_158 158 val_158 +160 val_160 160 val_160 +162 val_162 162 val_162 +163 val_163 163 val_163 +164 val_164 164 val_164 +164 val_164 164 val_164 +164 val_164 164 val_164 +164 val_164 164 val_164 +165 val_165 165 val_165 +165 val_165 165 val_165 +165 val_165 165 val_165 +165 val_165 165 val_165 +166 val_166 166 val_166 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +168 val_168 168 val_168 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +170 val_170 170 val_170 +172 val_172 172 val_172 +172 val_172 172 val_172 +172 val_172 172 val_172 +172 val_172 172 val_172 +174 val_174 174 val_174 +174 val_174 174 val_174 +174 val_174 174 val_174 +174 val_174 174 val_174 +175 val_175 175 val_175 +175 val_175 175 val_175 +175 val_175 175 val_175 +175 val_175 175 val_175 +176 val_176 176 val_176 +176 val_176 176 val_176 +176 val_176 176 val_176 +176 val_176 176 val_176 +177 val_177 177 val_177 +178 val_178 178 val_178 +179 val_179 179 val_179 +179 val_179 179 val_179 +179 val_179 179 val_179 +179 val_179 179 val_179 +180 val_180 180 val_180 +181 val_181 181 val_181 +183 val_183 183 val_183 +186 val_186 186 val_186 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +189 val_189 189 val_189 +190 val_190 190 val_190 +191 val_191 191 val_191 +191 val_191 191 val_191 +191 val_191 191 val_191 +191 val_191 191 val_191 +192 val_192 192 val_192 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +194 val_194 194 val_194 +195 val_195 195 val_195 +195 val_195 195 val_195 +195 val_195 195 val_195 +195 val_195 195 val_195 +196 val_196 196 val_196 +197 val_197 197 val_197 +197 val_197 197 val_197 +197 val_197 197 val_197 +197 val_197 197 val_197 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +200 val_200 200 val_200 +200 val_200 200 val_200 +200 val_200 200 val_200 +200 val_200 200 val_200 +201 val_201 201 val_201 +202 val_202 202 val_202 +203 val_203 203 val_203 +203 val_203 203 val_203 +203 val_203 203 val_203 +203 val_203 203 val_203 +205 val_205 205 val_205 +205 val_205 205 val_205 +205 val_205 205 val_205 +205 val_205 205 val_205 +207 val_207 207 val_207 +207 val_207 207 val_207 +207 val_207 207 val_207 +207 val_207 207 val_207 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +209 val_209 209 val_209 +209 val_209 209 val_209 +209 val_209 209 val_209 +209 val_209 209 val_209 +213 val_213 213 val_213 +213 val_213 213 val_213 +213 val_213 213 val_213 +213 val_213 213 val_213 +214 val_214 214 val_214 +216 val_216 216 val_216 +216 val_216 216 val_216 +216 val_216 216 val_216 +216 val_216 216 val_216 +217 val_217 217 val_217 +217 val_217 217 val_217 +217 val_217 217 val_217 +217 val_217 217 val_217 +218 val_218 218 val_218 +219 val_219 219 val_219 +219 val_219 219 val_219 +219 val_219 219 val_219 +219 val_219 219 val_219 +221 val_221 221 val_221 +221 val_221 221 val_221 +221 val_221 221 val_221 +221 val_221 221 val_221 +222 val_222 222 val_222 +223 val_223 223 val_223 +223 val_223 223 val_223 +223 val_223 223 val_223 +223 val_223 223 val_223 +224 val_224 224 val_224 +224 val_224 224 val_224 +224 val_224 224 val_224 +224 val_224 224 val_224 +226 val_226 226 val_226 +228 val_228 228 val_228 +229 val_229 229 val_229 +229 val_229 229 val_229 +229 val_229 229 val_229 +229 val_229 229 val_229 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +233 val_233 233 val_233 +233 val_233 233 val_233 +233 val_233 233 val_233 +233 val_233 233 val_233 +235 val_235 235 val_235 +237 val_237 237 val_237 +237 val_237 237 val_237 +237 val_237 237 val_237 +237 val_237 237 val_237 +238 val_238 238 val_238 +238 val_238 238 val_238 +238 val_238 238 val_238 +238 val_238 238 val_238 +239 val_239 239 val_239 +239 val_239 239 val_239 +239 val_239 239 val_239 +239 val_239 239 val_239 +241 val_241 241 val_241 +242 val_242 242 val_242 +242 val_242 242 val_242 +242 val_242 242 val_242 +242 val_242 242 val_242 +244 val_244 244 val_244 +247 val_247 247 val_247 +248 val_248 248 val_248 +249 val_249 249 val_249 +252 val_252 252 val_252 +255 val_255 255 val_255 +255 val_255 255 val_255 +255 val_255 255 val_255 +255 val_255 255 val_255 +256 val_256 256 val_256 +256 val_256 256 val_256 +256 val_256 256 val_256 +256 val_256 256 val_256 +257 val_257 257 val_257 +258 val_258 258 val_258 +260 val_260 260 val_260 +262 val_262 262 val_262 +263 val_263 263 val_263 +265 val_265 265 val_265 +265 val_265 265 val_265 +265 val_265 265 val_265 +265 val_265 265 val_265 +266 val_266 266 val_266 +272 val_272 272 val_272 +272 val_272 272 val_272 +272 val_272 272 val_272 +272 val_272 272 val_272 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +274 val_274 274 val_274 +275 val_275 275 val_275 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +278 val_278 278 val_278 +278 val_278 278 val_278 +278 val_278 278 val_278 +278 val_278 278 val_278 +280 val_280 280 val_280 +280 val_280 280 val_280 +280 val_280 280 val_280 +280 val_280 280 val_280 +281 val_281 281 val_281 +281 val_281 281 val_281 +281 val_281 281 val_281 +281 val_281 281 val_281 +282 val_282 282 val_282 +282 val_282 282 val_282 +282 val_282 282 val_282 +282 val_282 282 val_282 +283 val_283 283 val_283 +284 val_284 284 val_284 +285 val_285 285 val_285 +286 val_286 286 val_286 +287 val_287 287 val_287 +288 val_288 288 val_288 +288 val_288 288 val_288 +288 val_288 288 val_288 +288 val_288 288 val_288 +289 val_289 289 val_289 +291 val_291 291 val_291 +292 val_292 292 val_292 +296 val_296 296 val_296 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +302 val_302 302 val_302 +305 val_305 305 val_305 +306 val_306 306 val_306 +307 val_307 307 val_307 +307 val_307 307 val_307 +307 val_307 307 val_307 +307 val_307 307 val_307 +308 val_308 308 val_308 +309 val_309 309 val_309 +309 val_309 309 val_309 +309 val_309 309 val_309 +309 val_309 309 val_309 +310 val_310 310 val_310 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +315 val_315 315 val_315 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +317 val_317 317 val_317 +317 val_317 317 val_317 +317 val_317 317 val_317 +317 val_317 317 val_317 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +321 val_321 321 val_321 +321 val_321 321 val_321 +321 val_321 321 val_321 +321 val_321 321 val_321 +322 val_322 322 val_322 +322 val_322 322 val_322 +322 val_322 322 val_322 +322 val_322 322 val_322 +323 val_323 323 val_323 +325 val_325 325 val_325 +325 val_325 325 val_325 +325 val_325 325 val_325 +325 val_325 325 val_325 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +331 val_331 331 val_331 +331 val_331 331 val_331 +331 val_331 331 val_331 +331 val_331 331 val_331 +332 val_332 332 val_332 +333 val_333 333 val_333 +333 val_333 333 val_333 +333 val_333 333 val_333 +333 val_333 333 val_333 +335 val_335 335 val_335 +336 val_336 336 val_336 +338 val_338 338 val_338 +339 val_339 339 val_339 +341 val_341 341 val_341 +342 val_342 342 val_342 +342 val_342 342 val_342 +342 val_342 342 val_342 +342 val_342 342 val_342 +344 val_344 344 val_344 +344 val_344 344 val_344 +344 val_344 344 val_344 +344 val_344 344 val_344 +345 val_345 345 val_345 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +351 val_351 351 val_351 +353 val_353 353 val_353 +353 val_353 353 val_353 +353 val_353 353 val_353 +353 val_353 353 val_353 +356 val_356 356 val_356 +360 val_360 360 val_360 +362 val_362 362 val_362 +364 val_364 364 val_364 +365 val_365 365 val_365 +366 val_366 366 val_366 +367 val_367 367 val_367 +367 val_367 367 val_367 +367 val_367 367 val_367 +367 val_367 367 val_367 +368 val_368 368 val_368 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +373 val_373 373 val_373 +374 val_374 374 val_374 +375 val_375 375 val_375 +377 val_377 377 val_377 +378 val_378 378 val_378 +379 val_379 379 val_379 +382 val_382 382 val_382 +382 val_382 382 val_382 +382 val_382 382 val_382 +382 val_382 382 val_382 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +386 val_386 386 val_386 +389 val_389 389 val_389 +392 val_392 392 val_392 +393 val_393 393 val_393 +394 val_394 394 val_394 +395 val_395 395 val_395 +395 val_395 395 val_395 +395 val_395 395 val_395 +395 val_395 395 val_395 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +397 val_397 397 val_397 +397 val_397 397 val_397 +397 val_397 397 val_397 +397 val_397 397 val_397 +399 val_399 399 val_399 +399 val_399 399 val_399 +399 val_399 399 val_399 +399 val_399 399 val_399 +400 val_400 400 val_400 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +402 val_402 402 val_402 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +404 val_404 404 val_404 +404 val_404 404 val_404 +404 val_404 404 val_404 +404 val_404 404 val_404 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +407 val_407 407 val_407 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +411 val_411 411 val_411 +413 val_413 413 val_413 +413 val_413 413 val_413 +413 val_413 413 val_413 +413 val_413 413 val_413 +414 val_414 414 val_414 +414 val_414 414 val_414 +414 val_414 414 val_414 +414 val_414 414 val_414 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +418 val_418 418 val_418 +419 val_419 419 val_419 +421 val_421 421 val_421 +424 val_424 424 val_424 +424 val_424 424 val_424 +424 val_424 424 val_424 +424 val_424 424 val_424 +427 val_427 427 val_427 +429 val_429 429 val_429 +429 val_429 429 val_429 +429 val_429 429 val_429 +429 val_429 429 val_429 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +432 val_432 432 val_432 +435 val_435 435 val_435 +436 val_436 436 val_436 +437 val_437 437 val_437 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +439 val_439 439 val_439 +439 val_439 439 val_439 +439 val_439 439 val_439 +439 val_439 439 val_439 +443 val_443 443 val_443 +444 val_444 444 val_444 +446 val_446 446 val_446 +448 val_448 448 val_448 +449 val_449 449 val_449 +452 val_452 452 val_452 +453 val_453 453 val_453 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +455 val_455 455 val_455 +457 val_457 457 val_457 +458 val_458 458 val_458 +458 val_458 458 val_458 +458 val_458 458 val_458 +458 val_458 458 val_458 +459 val_459 459 val_459 +459 val_459 459 val_459 +459 val_459 459 val_459 +459 val_459 459 val_459 +460 val_460 460 val_460 +462 val_462 462 val_462 +462 val_462 462 val_462 +462 val_462 462 val_462 +462 val_462 462 val_462 +463 val_463 463 val_463 +463 val_463 463 val_463 +463 val_463 463 val_463 +463 val_463 463 val_463 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +467 val_467 467 val_467 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +470 val_470 470 val_470 +472 val_472 472 val_472 +475 val_475 475 val_475 +477 val_477 477 val_477 +478 val_478 478 val_478 +478 val_478 478 val_478 +478 val_478 478 val_478 +478 val_478 478 val_478 +479 val_479 479 val_479 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +481 val_481 481 val_481 +482 val_482 482 val_482 +483 val_483 483 val_483 +484 val_484 484 val_484 +485 val_485 485 val_485 +487 val_487 487 val_487 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +490 val_490 490 val_490 +491 val_491 491 val_491 +492 val_492 492 val_492 +492 val_492 492 val_492 +492 val_492 492 val_492 +492 val_492 492 val_492 +493 val_493 493 val_493 +494 val_494 494 val_494 +495 val_495 495 val_495 +496 val_496 496 val_496 +497 val_497 497 val_497 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +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 diff --git a/src/test/hive/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c b/src/test/hive/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 b/src/test/hive/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b b/src/test/hive/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 b/src/test/hive/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..66fafbdf72b63 --- /dev/null +++ b/src/test/hive/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,1028 @@ +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 +409 val_409 409 val_409 +409 val_409 409 val_409 +255 val_255 255 val_255 +255 val_255 255 val_255 +278 val_278 278 val_278 +278 val_278 278 val_278 +98 val_98 98 val_98 +98 val_98 98 val_98 +484 val_484 484 val_484 +265 val_265 265 val_265 +265 val_265 265 val_265 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +150 val_150 150 val_150 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +224 val_224 224 val_224 +224 val_224 224 val_224 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +66 val_66 66 val_66 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +213 val_213 213 val_213 +213 val_213 213 val_213 +146 val_146 146 val_146 +146 val_146 146 val_146 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +429 val_429 429 val_429 +429 val_429 429 val_429 +374 val_374 374 val_374 +152 val_152 152 val_152 +152 val_152 152 val_152 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +145 val_145 145 val_145 +495 val_495 495 val_495 +37 val_37 37 val_37 +37 val_37 37 val_37 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +281 val_281 281 val_281 +281 val_281 281 val_281 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +209 val_209 209 val_209 +209 val_209 209 val_209 +15 val_15 15 val_15 +15 val_15 15 val_15 +82 val_82 82 val_82 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +166 val_166 166 val_166 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +252 val_252 252 val_252 +292 val_292 292 val_292 +219 val_219 219 val_219 +219 val_219 219 val_219 +287 val_287 287 val_287 +153 val_153 153 val_153 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +338 val_338 338 val_338 +446 val_446 446 val_446 +459 val_459 459 val_459 +459 val_459 459 val_459 +394 val_394 394 val_394 +237 val_237 237 val_237 +237 val_237 237 val_237 +482 val_482 482 val_482 +174 val_174 174 val_174 +174 val_174 174 val_174 +413 val_413 413 val_413 +413 val_413 413 val_413 +494 val_494 494 val_494 +207 val_207 207 val_207 +207 val_207 207 val_207 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +174 val_174 174 val_174 +174 val_174 174 val_174 +399 val_399 399 val_399 +399 val_399 399 val_399 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +247 val_247 247 val_247 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +162 val_162 162 val_162 +377 val_377 377 val_377 +397 val_397 397 val_397 +397 val_397 397 val_397 +309 val_309 309 val_309 +309 val_309 309 val_309 +365 val_365 365 val_365 +266 val_266 266 val_266 +439 val_439 439 val_439 +439 val_439 439 val_439 +342 val_342 342 val_342 +342 val_342 342 val_342 +367 val_367 367 val_367 +367 val_367 367 val_367 +325 val_325 325 val_325 +325 val_325 325 val_325 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +195 val_195 195 val_195 +195 val_195 195 val_195 +475 val_475 475 val_475 +17 val_17 17 val_17 +113 val_113 113 val_113 +113 val_113 113 val_113 +155 val_155 155 val_155 +203 val_203 203 val_203 +203 val_203 203 val_203 +339 val_339 339 val_339 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +455 val_455 455 val_455 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +57 val_57 57 val_57 +302 val_302 302 val_302 +205 val_205 205 val_205 +205 val_205 205 val_205 +149 val_149 149 val_149 +149 val_149 149 val_149 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +345 val_345 345 val_345 +129 val_129 129 val_129 +129 val_129 129 val_129 +170 val_170 170 val_170 +20 val_20 20 val_20 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +157 val_157 157 val_157 +378 val_378 378 val_378 +221 val_221 221 val_221 +221 val_221 221 val_221 +92 val_92 92 val_92 +111 val_111 111 val_111 +47 val_47 47 val_47 +72 val_72 72 val_72 +72 val_72 72 val_72 +4 val_4 4 val_4 +280 val_280 280 val_280 +280 val_280 280 val_280 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +427 val_427 427 val_427 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +356 val_356 356 val_356 +399 val_399 399 val_399 +399 val_399 399 val_399 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +382 val_382 382 val_382 +382 val_382 382 val_382 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +125 val_125 125 val_125 +125 val_125 125 val_125 +386 val_386 386 val_386 +437 val_437 437 val_437 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +192 val_192 192 val_192 +286 val_286 286 val_286 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +176 val_176 176 val_176 +176 val_176 176 val_176 +54 val_54 54 val_54 +459 val_459 459 val_459 +459 val_459 459 val_459 +51 val_51 51 val_51 +51 val_51 51 val_51 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +103 val_103 103 val_103 +103 val_103 103 val_103 +239 val_239 239 val_239 +239 val_239 239 val_239 +213 val_213 213 val_213 +213 val_213 213 val_213 +216 val_216 216 val_216 +216 val_216 216 val_216 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +278 val_278 278 val_278 +278 val_278 278 val_278 +176 val_176 176 val_176 +176 val_176 176 val_176 +289 val_289 289 val_289 +221 val_221 221 val_221 +221 val_221 221 val_221 +65 val_65 65 val_65 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +332 val_332 332 val_332 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +275 val_275 275 val_275 +137 val_137 137 val_137 +137 val_137 137 val_137 +241 val_241 241 val_241 +83 val_83 83 val_83 +83 val_83 83 val_83 +333 val_333 333 val_333 +333 val_333 333 val_333 +180 val_180 180 val_180 +284 val_284 284 val_284 +12 val_12 12 val_12 +12 val_12 12 val_12 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +181 val_181 181 val_181 +67 val_67 67 val_67 +67 val_67 67 val_67 +260 val_260 260 val_260 +404 val_404 404 val_404 +404 val_404 404 val_404 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +353 val_353 353 val_353 +353 val_353 353 val_353 +373 val_373 373 val_373 +272 val_272 272 val_272 +272 val_272 272 val_272 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +217 val_217 217 val_217 +217 val_217 217 val_217 +84 val_84 84 val_84 +84 val_84 84 val_84 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +58 val_58 58 val_58 +58 val_58 58 val_58 +8 val_8 8 val_8 +411 val_411 411 val_411 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +24 val_24 24 val_24 +24 val_24 24 val_24 +463 val_463 463 val_463 +463 val_463 463 val_463 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +179 val_179 179 val_179 +179 val_179 179 val_179 +172 val_172 172 val_172 +172 val_172 172 val_172 +42 val_42 42 val_42 +42 val_42 42 val_42 +129 val_129 129 val_129 +129 val_129 129 val_129 +158 val_158 158 val_158 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +496 val_496 496 val_496 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +322 val_322 322 val_322 +322 val_322 322 val_322 +197 val_197 197 val_197 +197 val_197 197 val_197 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +393 val_393 393 val_393 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +100 val_100 100 val_100 +100 val_100 100 val_100 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +191 val_191 191 val_191 +191 val_191 191 val_191 +418 val_418 418 val_418 +96 val_96 96 val_96 +26 val_26 26 val_26 +26 val_26 26 val_26 +165 val_165 165 val_165 +165 val_165 165 val_165 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +205 val_205 205 val_205 +205 val_205 205 val_205 +120 val_120 120 val_120 +120 val_120 120 val_120 +131 val_131 131 val_131 +51 val_51 51 val_51 +51 val_51 51 val_51 +404 val_404 404 val_404 +404 val_404 404 val_404 +43 val_43 43 val_43 +436 val_436 436 val_436 +156 val_156 156 val_156 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +308 val_308 308 val_308 +95 val_95 95 val_95 +95 val_95 95 val_95 +196 val_196 196 val_196 +288 val_288 288 val_288 +288 val_288 288 val_288 +481 val_481 481 val_481 +457 val_457 457 val_457 +98 val_98 98 val_98 +98 val_98 98 val_98 +282 val_282 282 val_282 +282 val_282 282 val_282 +197 val_197 197 val_197 +197 val_197 197 val_197 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +470 val_470 470 val_470 +137 val_137 137 val_137 +137 val_137 137 val_137 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +413 val_413 413 val_413 +413 val_413 413 val_413 +85 val_85 85 val_85 +77 val_77 77 val_77 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +490 val_490 490 val_490 +87 val_87 87 val_87 +364 val_364 364 val_364 +179 val_179 179 val_179 +179 val_179 179 val_179 +118 val_118 118 val_118 +118 val_118 118 val_118 +134 val_134 134 val_134 +134 val_134 134 val_134 +395 val_395 395 val_395 +395 val_395 395 val_395 +282 val_282 282 val_282 +282 val_282 282 val_282 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +238 val_238 238 val_238 +238 val_238 238 val_238 +419 val_419 419 val_419 +15 val_15 15 val_15 +15 val_15 15 val_15 +118 val_118 118 val_118 +118 val_118 118 val_118 +72 val_72 72 val_72 +72 val_72 72 val_72 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +307 val_307 307 val_307 +307 val_307 307 val_307 +19 val_19 19 val_19 +435 val_435 435 val_435 +10 val_10 10 val_10 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +306 val_306 306 val_306 +224 val_224 224 val_224 +224 val_224 224 val_224 +309 val_309 309 val_309 +309 val_309 309 val_309 +389 val_389 389 val_389 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +242 val_242 242 val_242 +242 val_242 242 val_242 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +392 val_392 392 val_392 +272 val_272 272 val_272 +272 val_272 272 val_272 +331 val_331 331 val_331 +331 val_331 331 val_331 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +242 val_242 242 val_242 +242 val_242 242 val_242 +452 val_452 452 val_452 +177 val_177 177 val_177 +226 val_226 226 val_226 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +497 val_497 497 val_497 +402 val_402 402 val_402 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +317 val_317 317 val_317 +317 val_317 317 val_317 +395 val_395 395 val_395 +395 val_395 395 val_395 +58 val_58 58 val_58 +58 val_58 58 val_58 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +336 val_336 336 val_336 +95 val_95 95 val_95 +95 val_95 95 val_95 +11 val_11 11 val_11 +168 val_168 168 val_168 +34 val_34 34 val_34 +229 val_229 229 val_229 +229 val_229 229 val_229 +233 val_233 233 val_233 +233 val_233 233 val_233 +143 val_143 143 val_143 +472 val_472 472 val_472 +322 val_322 322 val_322 +322 val_322 322 val_322 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +160 val_160 160 val_160 +195 val_195 195 val_195 +195 val_195 195 val_195 +42 val_42 42 val_42 +42 val_42 42 val_42 +321 val_321 321 val_321 +321 val_321 321 val_321 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +458 val_458 458 val_458 +458 val_458 458 val_458 +78 val_78 78 val_78 +76 val_76 76 val_76 +76 val_76 76 val_76 +41 val_41 41 val_41 +223 val_223 223 val_223 +223 val_223 223 val_223 +492 val_492 492 val_492 +492 val_492 492 val_492 +149 val_149 149 val_149 +149 val_149 149 val_149 +449 val_449 449 val_449 +218 val_218 218 val_218 +228 val_228 228 val_228 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +453 val_453 453 val_453 +30 val_30 30 val_30 +209 val_209 209 val_209 +209 val_209 209 val_209 +64 val_64 64 val_64 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +76 val_76 76 val_76 +76 val_76 76 val_76 +74 val_74 74 val_74 +342 val_342 342 val_342 +342 val_342 342 val_342 +69 val_69 69 val_69 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +33 val_33 33 val_33 +368 val_368 368 val_368 +103 val_103 103 val_103 +103 val_103 103 val_103 +296 val_296 296 val_296 +113 val_113 113 val_113 +113 val_113 113 val_113 +216 val_216 216 val_216 +216 val_216 216 val_216 +367 val_367 367 val_367 +367 val_367 367 val_367 +344 val_344 344 val_344 +344 val_344 344 val_344 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +274 val_274 274 val_274 +219 val_219 219 val_219 +219 val_219 219 val_219 +239 val_239 239 val_239 +239 val_239 239 val_239 +485 val_485 485 val_485 +116 val_116 116 val_116 +223 val_223 223 val_223 +223 val_223 223 val_223 +256 val_256 256 val_256 +256 val_256 256 val_256 +263 val_263 263 val_263 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +487 val_487 487 val_487 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +288 val_288 288 val_288 +288 val_288 288 val_288 +191 val_191 191 val_191 +191 val_191 191 val_191 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +244 val_244 244 val_244 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +467 val_467 467 val_467 +432 val_432 432 val_432 +202 val_202 202 val_202 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +229 val_229 229 val_229 +229 val_229 229 val_229 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +463 val_463 463 val_463 +463 val_463 463 val_463 +280 val_280 280 val_280 +280 val_280 280 val_280 +2 val_2 2 val_2 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +283 val_283 283 val_283 +331 val_331 331 val_331 +331 val_331 331 val_331 +235 val_235 235 val_235 +80 val_80 80 val_80 +44 val_44 44 val_44 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +321 val_321 321 val_321 +321 val_321 321 val_321 +335 val_335 335 val_335 +104 val_104 104 val_104 +104 val_104 104 val_104 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +366 val_366 366 val_366 +175 val_175 175 val_175 +175 val_175 175 val_175 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +483 val_483 483 val_483 +53 val_53 53 val_53 +105 val_105 105 val_105 +257 val_257 257 val_257 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +190 val_190 190 val_190 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +114 val_114 114 val_114 +258 val_258 258 val_258 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +203 val_203 203 val_203 +203 val_203 203 val_203 +262 val_262 262 val_262 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +424 val_424 424 val_424 +424 val_424 424 val_424 +12 val_12 12 val_12 +12 val_12 12 val_12 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +201 val_201 201 val_201 +217 val_217 217 val_217 +217 val_217 217 val_217 +164 val_164 164 val_164 +164 val_164 164 val_164 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +478 val_478 478 val_478 +478 val_478 478 val_478 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +125 val_125 125 val_125 +125 val_125 125 val_125 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +164 val_164 164 val_164 +164 val_164 164 val_164 +424 val_424 424 val_424 +424 val_424 424 val_424 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +382 val_382 382 val_382 +382 val_382 382 val_382 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +397 val_397 397 val_397 +397 val_397 397 val_397 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +291 val_291 291 val_291 +24 val_24 24 val_24 +24 val_24 24 val_24 +351 val_351 351 val_351 +255 val_255 255 val_255 +255 val_255 255 val_255 +104 val_104 104 val_104 +104 val_104 104 val_104 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +163 val_163 163 val_163 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +414 val_414 414 val_414 +414 val_414 414 val_414 +200 val_200 200 val_200 +200 val_200 200 val_200 +491 val_491 491 val_491 +237 val_237 237 val_237 +237 val_237 237 val_237 +439 val_439 439 val_439 +439 val_439 439 val_439 +360 val_360 360 val_360 +248 val_248 248 val_248 +479 val_479 479 val_479 +305 val_305 305 val_305 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +444 val_444 444 val_444 +120 val_120 120 val_120 +120 val_120 120 val_120 +429 val_429 429 val_429 +429 val_429 429 val_429 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +443 val_443 443 val_443 +323 val_323 323 val_323 +325 val_325 325 val_325 +325 val_325 325 val_325 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +478 val_478 478 val_478 +478 val_478 478 val_478 +178 val_178 178 val_178 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +310 val_310 310 val_310 +317 val_317 317 val_317 +317 val_317 317 val_317 +333 val_333 333 val_333 +333 val_333 333 val_333 +493 val_493 493 val_493 +460 val_460 460 val_460 +207 val_207 207 val_207 +207 val_207 207 val_207 +249 val_249 249 val_249 +265 val_265 265 val_265 +265 val_265 265 val_265 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +83 val_83 83 val_83 +83 val_83 83 val_83 +136 val_136 136 val_136 +353 val_353 353 val_353 +353 val_353 353 val_353 +172 val_172 172 val_172 +172 val_172 172 val_172 +214 val_214 214 val_214 +462 val_462 462 val_462 +462 val_462 462 val_462 +233 val_233 233 val_233 +233 val_233 233 val_233 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +133 val_133 133 val_133 +175 val_175 175 val_175 +175 val_175 175 val_175 +189 val_189 189 val_189 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +375 val_375 375 val_375 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +421 val_421 421 val_421 +407 val_407 407 val_407 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +256 val_256 256 val_256 +256 val_256 256 val_256 +26 val_26 26 val_26 +26 val_26 26 val_26 +134 val_134 134 val_134 +134 val_134 134 val_134 +67 val_67 67 val_67 +67 val_67 67 val_67 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +379 val_379 379 val_379 +18 val_18 18 val_18 +18 val_18 18 val_18 +462 val_462 462 val_462 +462 val_462 462 val_462 +492 val_492 492 val_492 +492 val_492 492 val_492 +100 val_100 100 val_100 +100 val_100 100 val_100 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +9 val_9 9 val_9 +341 val_341 341 val_341 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +146 val_146 146 val_146 +146 val_146 146 val_146 +458 val_458 458 val_458 +458 val_458 458 val_458 +362 val_362 362 val_362 +186 val_186 186 val_186 +285 val_285 285 val_285 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +18 val_18 18 val_18 +18 val_18 18 val_18 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +183 val_183 183 val_183 +281 val_281 281 val_281 +281 val_281 281 val_281 +344 val_344 344 val_344 +344 val_344 344 val_344 +97 val_97 97 val_97 +97 val_97 97 val_97 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +315 val_315 315 val_315 +84 val_84 84 val_84 +84 val_84 84 val_84 +28 val_28 28 val_28 +37 val_37 37 val_37 +37 val_37 37 val_37 +448 val_448 448 val_448 +152 val_152 152 val_152 +152 val_152 152 val_152 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +307 val_307 307 val_307 +307 val_307 307 val_307 +194 val_194 194 val_194 +414 val_414 414 val_414 +414 val_414 414 val_414 +477 val_477 477 val_477 +222 val_222 222 val_222 +126 val_126 126 val_126 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +400 val_400 400 val_400 +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 diff --git a/src/test/hive/golden/join18-0-269843197b598569f4b50cbe911960a5 b/src/test/hive/golden/join18-0-269843197b598569f4b50cbe911960a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 b/src/test/hive/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 new file mode 100644 index 0000000000000..10628f6c64bd4 --- /dev/null +++ b/src/test/hive/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 @@ -0,0 +1,310 @@ +NULL NULL NULL 7 +0 3 NULL NULL +2 1 NULL NULL +4 1 NULL NULL +5 3 NULL NULL +8 1 NULL NULL +9 1 NULL NULL +10 1 NULL NULL +11 1 NULL NULL +12 2 NULL NULL +15 2 NULL NULL +17 1 NULL NULL +18 2 NULL NULL +19 1 NULL NULL +20 1 NULL NULL +24 2 NULL NULL +26 2 NULL NULL +27 1 NULL NULL +28 1 NULL NULL +30 1 NULL NULL +33 1 NULL NULL +34 1 NULL NULL +35 3 NULL NULL +37 2 NULL NULL +41 1 NULL NULL +42 2 NULL NULL +43 1 NULL NULL +44 1 NULL NULL +47 1 NULL NULL +51 2 NULL NULL +53 1 NULL NULL +54 1 NULL NULL +57 1 NULL NULL +58 2 NULL NULL +64 1 NULL NULL +65 1 NULL NULL +66 1 66 1 +67 2 NULL NULL +69 1 NULL NULL +70 3 NULL NULL +72 2 NULL NULL +74 1 NULL NULL +76 2 NULL NULL +77 1 NULL NULL +78 1 NULL NULL +80 1 NULL NULL +82 1 NULL NULL +83 2 NULL NULL +84 2 NULL NULL +85 1 NULL NULL +86 1 NULL NULL +87 1 NULL NULL +90 3 NULL NULL +92 1 NULL NULL +95 2 NULL NULL +96 1 NULL NULL +97 2 NULL NULL +98 2 98 1 +100 2 NULL NULL +103 2 NULL NULL +104 2 NULL NULL +105 1 NULL NULL +111 1 NULL NULL +113 2 NULL NULL +114 1 NULL NULL +116 1 NULL NULL +118 2 NULL NULL +119 3 NULL NULL +120 2 NULL NULL +125 2 NULL NULL +126 1 NULL NULL +128 3 128 1 +129 2 NULL NULL +131 1 NULL NULL +133 1 NULL NULL +134 2 NULL NULL +136 1 NULL NULL +137 2 NULL NULL +138 4 NULL NULL +143 1 NULL NULL +145 1 NULL NULL +146 2 146 1 +149 2 NULL NULL +150 1 150 1 +152 2 NULL NULL +153 1 NULL NULL +155 1 NULL NULL +156 1 NULL NULL +157 1 NULL NULL +158 1 NULL NULL +160 1 NULL NULL +162 1 NULL NULL +163 1 NULL NULL +164 2 NULL NULL +165 2 NULL NULL +166 1 NULL NULL +167 3 NULL NULL +168 1 NULL NULL +169 4 NULL NULL +170 1 NULL NULL +172 2 NULL NULL +174 2 NULL NULL +175 2 NULL NULL +176 2 NULL NULL +177 1 NULL NULL +178 1 NULL NULL +179 2 NULL NULL +180 1 NULL NULL +181 1 NULL NULL +183 1 NULL NULL +186 1 NULL NULL +187 3 NULL NULL +189 1 NULL NULL +190 1 NULL NULL +191 2 NULL NULL +192 1 NULL NULL +193 3 NULL NULL +194 1 NULL NULL +195 2 NULL NULL +196 1 NULL NULL +197 2 NULL NULL +199 3 NULL NULL +200 2 NULL NULL +201 1 NULL NULL +202 1 NULL NULL +203 2 NULL NULL +205 2 NULL NULL +207 2 NULL NULL +208 3 NULL NULL +209 2 NULL NULL +213 2 213 1 +214 1 NULL NULL +216 2 NULL NULL +217 2 NULL NULL +218 1 NULL NULL +219 2 NULL NULL +221 2 NULL NULL +222 1 NULL NULL +223 2 NULL NULL +224 2 224 1 +226 1 NULL NULL +228 1 NULL NULL +229 2 NULL NULL +230 5 NULL NULL +233 2 NULL NULL +235 1 NULL NULL +237 2 NULL NULL +238 2 238 1 +239 2 NULL NULL +241 1 NULL NULL +242 2 NULL NULL +244 1 NULL NULL +247 1 NULL NULL +248 1 NULL NULL +249 1 NULL NULL +252 1 NULL NULL +255 2 255 1 +256 2 NULL NULL +257 1 NULL NULL +258 1 NULL NULL +260 1 NULL NULL +262 1 NULL NULL +263 1 NULL NULL +265 2 NULL NULL +266 1 NULL NULL +272 2 NULL NULL +273 3 273 1 +274 1 NULL NULL +275 1 NULL NULL +277 4 NULL NULL +278 2 278 1 +280 2 NULL NULL +281 2 NULL NULL +282 2 NULL NULL +283 1 NULL NULL +284 1 NULL NULL +285 1 NULL NULL +286 1 NULL NULL +287 1 NULL NULL +288 2 NULL NULL +289 1 NULL NULL +291 1 NULL NULL +292 1 NULL NULL +296 1 NULL NULL +298 3 NULL NULL +302 1 NULL NULL +305 1 NULL NULL +306 1 NULL NULL +307 2 NULL NULL +308 1 NULL NULL +309 2 NULL NULL +310 1 NULL NULL +311 3 311 1 +315 1 NULL NULL +316 3 NULL NULL +317 2 NULL NULL +318 3 NULL NULL +321 2 NULL NULL +322 2 NULL NULL +323 1 NULL NULL +325 2 NULL NULL +327 3 NULL NULL +331 2 NULL NULL +332 1 NULL NULL +333 2 NULL NULL +335 1 NULL NULL +336 1 NULL NULL +338 1 NULL NULL +339 1 NULL NULL +341 1 NULL NULL +342 2 NULL NULL +344 2 NULL NULL +345 1 NULL NULL +348 5 NULL NULL +351 1 NULL NULL +353 2 NULL NULL +356 1 NULL NULL +360 1 NULL NULL +362 1 NULL NULL +364 1 NULL NULL +365 1 NULL NULL +366 1 NULL NULL +367 2 NULL NULL +368 1 NULL NULL +369 3 369 1 +373 1 NULL NULL +374 1 NULL NULL +375 1 NULL NULL +377 1 NULL NULL +378 1 NULL NULL +379 1 NULL NULL +382 2 NULL NULL +384 3 NULL NULL +386 1 NULL NULL +389 1 NULL NULL +392 1 NULL NULL +393 1 NULL NULL +394 1 NULL NULL +395 2 NULL NULL +396 3 NULL NULL +397 2 NULL NULL +399 2 NULL NULL +400 1 NULL NULL +401 5 401 1 +402 1 NULL NULL +403 3 NULL NULL +404 2 NULL NULL +406 4 406 1 +407 1 NULL NULL +409 3 NULL NULL +411 1 NULL NULL +413 2 NULL NULL +414 2 NULL NULL +417 3 NULL NULL +418 1 NULL NULL +419 1 NULL NULL +421 1 NULL NULL +424 2 NULL NULL +427 1 NULL NULL +429 2 NULL NULL +430 3 NULL NULL +431 3 NULL NULL +432 1 NULL NULL +435 1 NULL NULL +436 1 NULL NULL +437 1 NULL NULL +438 3 NULL NULL +439 2 NULL NULL +443 1 NULL NULL +444 1 NULL NULL +446 1 NULL NULL +448 1 NULL NULL +449 1 NULL NULL +452 1 NULL NULL +453 1 NULL NULL +454 3 NULL NULL +455 1 NULL NULL +457 1 NULL NULL +458 2 NULL NULL +459 2 NULL NULL +460 1 NULL NULL +462 2 NULL NULL +463 2 NULL NULL +466 3 NULL NULL +467 1 NULL NULL +468 4 NULL NULL +469 5 NULL NULL +470 1 NULL NULL +472 1 NULL NULL +475 1 NULL NULL +477 1 NULL NULL +478 2 NULL NULL +479 1 NULL NULL +480 3 NULL NULL +481 1 NULL NULL +482 1 NULL NULL +483 1 NULL NULL +484 1 NULL NULL +485 1 NULL NULL +487 1 NULL NULL +489 4 NULL NULL +490 1 NULL NULL +491 1 NULL NULL +492 2 NULL NULL +493 1 NULL NULL +494 1 NULL NULL +495 1 NULL NULL +496 1 NULL NULL +497 1 NULL NULL +498 3 NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 b/src/test/hive/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join19-1-7e7d1f07c34dd4be5425264196201766 b/src/test/hive/golden/join19-1-7e7d1f07c34dd4be5425264196201766 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca b/src/test/hive/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c b/src/test/hive/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a b/src/test/hive/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 b/src/test/hive/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 new file mode 100644 index 0000000000000..2945c31ed4f00 --- /dev/null +++ b/src/test/hive/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 @@ -0,0 +1,472 @@ +86 val_172 +86 val_172 +27 val_54 +98 val_196 +98 val_196 +193 val_386 +193 val_386 +193 val_386 +224 val_448 +224 val_448 +128 val_256 +128 val_256 +128 val_256 +128 val_256 +128 val_256 +128 val_256 +146 val_292 +146 val_292 +37 val_74 +37 val_74 +209 val_418 +209 val_418 +15 val_30 +15 val_30 +82 val_164 +82 val_164 +166 val_332 +219 val_438 +219 val_438 +219 val_438 +219 val_438 +219 val_438 +219 val_438 +153 val_306 +193 val_386 +193 val_386 +193 val_386 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +207 val_414 +207 val_414 +207 val_414 +207 val_414 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +174 val_348 +247 val_494 +17 val_34 +113 val_226 +113 val_226 +155 val_310 +203 val_406 +203 val_406 +203 val_406 +203 val_406 +203 val_406 +203 val_406 +203 val_406 +203 val_406 +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 +128 val_256 +128 val_256 +128 val_256 +128 val_256 +128 val_256 +128 val_256 +57 val_114 +149 val_298 +149 val_298 +149 val_298 +149 val_298 +149 val_298 +149 val_298 +129 val_258 +129 val_258 +111 val_222 +4 val_8 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +169 val_338 +169 val_338 +169 val_338 +169 val_338 +192 val_384 +192 val_384 +192 val_384 +187 val_374 +187 val_374 +187 val_374 +239 val_478 +239 val_478 +239 val_478 +239 val_478 +216 val_432 +216 val_432 +137 val_274 +137 val_274 +241 val_482 +83 val_166 +83 val_166 +180 val_360 +12 val_24 +12 val_24 +12 val_24 +12 val_24 +230 val_460 +230 val_460 +230 val_460 +230 val_460 +230 val_460 +181 val_362 +67 val_134 +67 val_134 +67 val_134 +67 val_134 +84 val_168 +84 val_168 +58 val_116 +58 val_116 +230 val_460 +230 val_460 +230 val_460 +230 val_460 +230 val_460 +172 val_344 +172 val_344 +172 val_344 +172 val_344 +42 val_84 +42 val_84 +42 val_84 +42 val_84 +129 val_258 +129 val_258 +158 val_316 +158 val_316 +158 val_316 +119 val_238 +119 val_238 +119 val_238 +119 val_238 +119 val_238 +119 val_238 +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 +197 val_394 +197 val_394 +100 val_200 +100 val_200 +100 val_200 +100 val_200 +191 val_382 +191 val_382 +191 val_382 +191 val_382 +96 val_192 +230 val_460 +230 val_460 +230 val_460 +230 val_460 +230 val_460 +131 val_262 +43 val_86 +95 val_190 +95 val_190 +196 val_392 +98 val_196 +98 val_196 +197 val_394 +197 val_394 +187 val_374 +187 val_374 +187 val_374 +137 val_274 +137 val_274 +169 val_338 +169 val_338 +169 val_338 +169 val_338 +85 val_170 +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 +87 val_174 +87 val_174 +15 val_30 +15 val_30 +90 val_180 +90 val_180 +90 val_180 +10 val_20 +224 val_448 +224 val_448 +242 val_484 +242 val_484 +242 val_484 +242 val_484 +226 val_452 +5 val_10 +5 val_10 +5 val_10 +58 val_116 +58 val_116 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +95 val_190 +95 val_190 +168 val_336 +229 val_458 +229 val_458 +229 val_458 +229 val_458 +233 val_466 +233 val_466 +233 val_466 +233 val_466 +233 val_466 +233 val_466 +143 val_286 +42 val_84 +42 val_84 +42 val_84 +42 val_84 +119 val_238 +119 val_238 +119 val_238 +119 val_238 +119 val_238 +119 val_238 +78 val_156 +76 val_152 +76 val_152 +76 val_152 +76 val_152 +41 val_82 +223 val_446 +223 val_446 +149 val_298 +149 val_298 +149 val_298 +149 val_298 +149 val_298 +149 val_298 +218 val_436 +209 val_418 +209 val_418 +64 val_128 +64 val_128 +64 val_128 +76 val_152 +76 val_152 +76 val_152 +76 val_152 +69 val_138 +69 val_138 +69 val_138 +69 val_138 +230 val_460 +230 val_460 +230 val_460 +230 val_460 +230 val_460 +33 val_66 +113 val_226 +113 val_226 +216 val_432 +216 val_432 +219 val_438 +219 val_438 +219 val_438 +219 val_438 +219 val_438 +219 val_438 +239 val_478 +239 val_478 +239 val_478 +239 val_478 +223 val_446 +223 val_446 +191 val_382 +191 val_382 +191 val_382 +191 val_382 +5 val_10 +5 val_10 +5 val_10 +128 val_256 +128 val_256 +128 val_256 +128 val_256 +128 val_256 +128 val_256 +202 val_404 +202 val_404 +229 val_458 +229 val_458 +229 val_458 +229 val_458 +2 val_4 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +35 val_70 +235 val_470 +80 val_160 +193 val_386 +193 val_386 +193 val_386 +104 val_208 +104 val_208 +104 val_208 +104 val_208 +104 val_208 +104 val_208 +114 val_228 +90 val_180 +90 val_180 +90 val_180 +203 val_406 +203 val_406 +203 val_406 +203 val_406 +203 val_406 +203 val_406 +203 val_406 +203 val_406 +12 val_24 +12 val_24 +12 val_24 +12 val_24 +201 val_402 +187 val_374 +187 val_374 +187 val_374 +5 val_10 +5 val_10 +5 val_10 +104 val_208 +104 val_208 +104 val_208 +104 val_208 +104 val_208 +104 val_208 +119 val_238 +119 val_238 +119 val_238 +119 val_238 +119 val_238 +119 val_238 +200 val_400 +200 val_400 +248 val_496 +169 val_338 +169 val_338 +169 val_338 +169 val_338 +230 val_460 +230 val_460 +230 val_460 +230 val_460 +230 val_460 +178 val_356 +207 val_414 +207 val_414 +207 val_414 +207 val_414 +249 val_498 +249 val_498 +249 val_498 +83 val_166 +83 val_166 +136 val_272 +136 val_272 +172 val_344 +172 val_344 +172 val_344 +172 val_344 +233 val_466 +233 val_466 +233 val_466 +233 val_466 +233 val_466 +233 val_466 +133 val_266 +189 val_378 +67 val_134 +67 val_134 +67 val_134 +67 val_134 +100 val_200 +100 val_200 +100 val_200 +100 val_200 +9 val_18 +9 val_18 +146 val_292 +146 val_292 +183 val_366 +97 val_194 +97 val_194 +84 val_168 +84 val_168 +37 val_74 +37 val_74 +222 val_444 +126 val_252 +90 val_180 +90 val_180 +90 val_180 +169 val_338 +169 val_338 +169 val_338 +169 val_338 +200 val_400 +200 val_400 +97 val_194 +97 val_194 \ No newline at end of file diff --git a/src/test/hive/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e b/src/test/hive/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 b/src/test/hive/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 new file mode 100644 index 0000000000000..bd6e20b8de1e4 --- /dev/null +++ b/src/test/hive/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 @@ -0,0 +1,548 @@ +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 11 val_11 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 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 NULL NULL 26 val_26 +NULL NULL NULL NULL 26 val_26 +NULL NULL NULL NULL 27 val_27 +NULL NULL NULL NULL 28 val_28 +NULL NULL NULL NULL 30 val_30 +NULL NULL NULL NULL 33 val_33 +NULL NULL NULL NULL 34 val_34 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 41 val_41 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 43 val_43 +NULL NULL NULL NULL 44 val_44 +NULL NULL NULL NULL 47 val_47 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 53 val_53 +NULL NULL NULL NULL 54 val_54 +NULL NULL NULL NULL 57 val_57 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 64 val_64 +NULL NULL NULL NULL 65 val_65 +NULL NULL NULL NULL 66 val_66 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 69 val_69 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 74 val_74 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 77 val_77 +NULL NULL NULL NULL 78 val_78 +NULL NULL NULL NULL 80 val_80 +NULL NULL NULL NULL 82 val_82 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 85 val_85 +NULL NULL NULL NULL 86 val_86 +NULL NULL NULL NULL 87 val_87 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 92 val_92 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 96 val_96 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 105 val_105 +NULL NULL NULL NULL 111 val_111 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 114 val_114 +NULL NULL NULL NULL 116 val_116 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 126 val_126 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 131 val_131 +NULL NULL NULL NULL 133 val_133 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 136 val_136 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 143 val_143 +NULL NULL NULL NULL 145 val_145 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 150 val_150 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 153 val_153 +NULL NULL NULL NULL 155 val_155 +NULL NULL NULL NULL 156 val_156 +NULL NULL NULL NULL 157 val_157 +NULL NULL NULL NULL 158 val_158 +NULL NULL NULL NULL 160 val_160 +NULL NULL NULL NULL 162 val_162 +NULL NULL NULL NULL 163 val_163 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 166 val_166 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 168 val_168 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 170 val_170 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 177 val_177 +NULL NULL NULL NULL 178 val_178 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 180 val_180 +NULL NULL NULL NULL 181 val_181 +NULL NULL NULL NULL 183 val_183 +NULL NULL NULL NULL 186 val_186 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 189 val_189 +NULL NULL NULL NULL 190 val_190 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 192 val_192 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 194 val_194 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 196 val_196 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 201 val_201 +NULL NULL NULL NULL 202 val_202 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 214 val_214 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 218 val_218 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 222 val_222 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 226 val_226 +NULL NULL NULL NULL 228 val_228 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 235 val_235 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 241 val_241 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 244 val_244 +NULL NULL NULL NULL 247 val_247 +NULL NULL NULL NULL 248 val_248 +NULL NULL NULL NULL 249 val_249 +NULL NULL NULL NULL 252 val_252 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 257 val_257 +NULL NULL NULL NULL 258 val_258 +NULL NULL NULL NULL 260 val_260 +NULL NULL NULL NULL 262 val_262 +NULL NULL NULL NULL 263 val_263 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 266 val_266 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 274 val_274 +NULL NULL NULL NULL 275 val_275 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 283 val_283 +NULL NULL NULL NULL 284 val_284 +NULL NULL NULL NULL 285 val_285 +NULL NULL NULL NULL 286 val_286 +NULL NULL NULL NULL 287 val_287 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 289 val_289 +NULL NULL NULL NULL 291 val_291 +NULL NULL NULL NULL 292 val_292 +NULL NULL NULL NULL 296 val_296 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 302 val_302 +NULL NULL NULL NULL 305 val_305 +NULL NULL NULL NULL 306 val_306 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 308 val_308 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 310 val_310 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 315 val_315 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 323 val_323 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 332 val_332 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 335 val_335 +NULL NULL NULL NULL 336 val_336 +NULL NULL NULL NULL 338 val_338 +NULL NULL NULL NULL 339 val_339 +NULL NULL NULL NULL 341 val_341 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 345 val_345 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 351 val_351 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 356 val_356 +NULL NULL NULL NULL 360 val_360 +NULL NULL NULL NULL 362 val_362 +NULL NULL NULL NULL 364 val_364 +NULL NULL NULL NULL 365 val_365 +NULL NULL NULL NULL 366 val_366 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 368 val_368 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 373 val_373 +NULL NULL NULL NULL 374 val_374 +NULL NULL NULL NULL 375 val_375 +NULL NULL NULL NULL 377 val_377 +NULL NULL NULL NULL 378 val_378 +NULL NULL NULL NULL 379 val_379 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 386 val_386 +NULL NULL NULL NULL 389 val_389 +NULL NULL NULL NULL 392 val_392 +NULL NULL NULL NULL 393 val_393 +NULL NULL NULL NULL 394 val_394 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 400 val_400 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 402 val_402 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 407 val_407 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 411 val_411 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 418 val_418 +NULL NULL NULL NULL 419 val_419 +NULL NULL NULL NULL 421 val_421 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 427 val_427 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 432 val_432 +NULL NULL NULL NULL 435 val_435 +NULL NULL NULL NULL 436 val_436 +NULL NULL NULL NULL 437 val_437 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 443 val_443 +NULL NULL NULL NULL 444 val_444 +NULL NULL NULL NULL 446 val_446 +NULL NULL NULL NULL 448 val_448 +NULL NULL NULL NULL 449 val_449 +NULL NULL NULL NULL 452 val_452 +NULL NULL NULL NULL 453 val_453 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 455 val_455 +NULL NULL NULL NULL 457 val_457 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 460 val_460 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 467 val_467 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 470 val_470 +NULL NULL NULL NULL 472 val_472 +NULL NULL NULL NULL 475 val_475 +NULL NULL NULL NULL 477 val_477 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 479 val_479 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 481 val_481 +NULL NULL NULL NULL 482 val_482 +NULL NULL NULL NULL 483 val_483 +NULL NULL NULL NULL 484 val_484 +NULL NULL NULL NULL 485 val_485 +NULL NULL NULL NULL 487 val_487 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 490 val_490 +NULL NULL NULL NULL 491 val_491 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 493 val_493 +NULL NULL NULL NULL 494 val_494 +NULL NULL NULL NULL 495 val_495 +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 +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 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 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 +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 0 val_0 0 val_0 +0 val_0 0 val_0 0 val_0 +2 val_2 2 val_2 2 val_2 +4 val_4 4 val_4 4 val_4 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +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 diff --git a/src/test/hive/golden/join20-2-222655137ca9242f063c7717acbe4c65 b/src/test/hive/golden/join20-2-222655137ca9242f063c7717acbe4c65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 b/src/test/hive/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 new file mode 100644 index 0000000000000..bd6e20b8de1e4 --- /dev/null +++ b/src/test/hive/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 @@ -0,0 +1,548 @@ +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 11 val_11 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 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 NULL NULL 26 val_26 +NULL NULL NULL NULL 26 val_26 +NULL NULL NULL NULL 27 val_27 +NULL NULL NULL NULL 28 val_28 +NULL NULL NULL NULL 30 val_30 +NULL NULL NULL NULL 33 val_33 +NULL NULL NULL NULL 34 val_34 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 41 val_41 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 43 val_43 +NULL NULL NULL NULL 44 val_44 +NULL NULL NULL NULL 47 val_47 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 53 val_53 +NULL NULL NULL NULL 54 val_54 +NULL NULL NULL NULL 57 val_57 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 64 val_64 +NULL NULL NULL NULL 65 val_65 +NULL NULL NULL NULL 66 val_66 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 69 val_69 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 74 val_74 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 77 val_77 +NULL NULL NULL NULL 78 val_78 +NULL NULL NULL NULL 80 val_80 +NULL NULL NULL NULL 82 val_82 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 85 val_85 +NULL NULL NULL NULL 86 val_86 +NULL NULL NULL NULL 87 val_87 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 92 val_92 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 96 val_96 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 105 val_105 +NULL NULL NULL NULL 111 val_111 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 114 val_114 +NULL NULL NULL NULL 116 val_116 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 126 val_126 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 131 val_131 +NULL NULL NULL NULL 133 val_133 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 136 val_136 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 143 val_143 +NULL NULL NULL NULL 145 val_145 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 150 val_150 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 153 val_153 +NULL NULL NULL NULL 155 val_155 +NULL NULL NULL NULL 156 val_156 +NULL NULL NULL NULL 157 val_157 +NULL NULL NULL NULL 158 val_158 +NULL NULL NULL NULL 160 val_160 +NULL NULL NULL NULL 162 val_162 +NULL NULL NULL NULL 163 val_163 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 166 val_166 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 168 val_168 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 170 val_170 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 177 val_177 +NULL NULL NULL NULL 178 val_178 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 180 val_180 +NULL NULL NULL NULL 181 val_181 +NULL NULL NULL NULL 183 val_183 +NULL NULL NULL NULL 186 val_186 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 189 val_189 +NULL NULL NULL NULL 190 val_190 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 192 val_192 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 194 val_194 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 196 val_196 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 201 val_201 +NULL NULL NULL NULL 202 val_202 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 214 val_214 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 218 val_218 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 222 val_222 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 226 val_226 +NULL NULL NULL NULL 228 val_228 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 235 val_235 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 241 val_241 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 244 val_244 +NULL NULL NULL NULL 247 val_247 +NULL NULL NULL NULL 248 val_248 +NULL NULL NULL NULL 249 val_249 +NULL NULL NULL NULL 252 val_252 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 257 val_257 +NULL NULL NULL NULL 258 val_258 +NULL NULL NULL NULL 260 val_260 +NULL NULL NULL NULL 262 val_262 +NULL NULL NULL NULL 263 val_263 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 266 val_266 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 274 val_274 +NULL NULL NULL NULL 275 val_275 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 283 val_283 +NULL NULL NULL NULL 284 val_284 +NULL NULL NULL NULL 285 val_285 +NULL NULL NULL NULL 286 val_286 +NULL NULL NULL NULL 287 val_287 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 289 val_289 +NULL NULL NULL NULL 291 val_291 +NULL NULL NULL NULL 292 val_292 +NULL NULL NULL NULL 296 val_296 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 302 val_302 +NULL NULL NULL NULL 305 val_305 +NULL NULL NULL NULL 306 val_306 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 308 val_308 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 310 val_310 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 315 val_315 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 323 val_323 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 332 val_332 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 335 val_335 +NULL NULL NULL NULL 336 val_336 +NULL NULL NULL NULL 338 val_338 +NULL NULL NULL NULL 339 val_339 +NULL NULL NULL NULL 341 val_341 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 345 val_345 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 351 val_351 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 356 val_356 +NULL NULL NULL NULL 360 val_360 +NULL NULL NULL NULL 362 val_362 +NULL NULL NULL NULL 364 val_364 +NULL NULL NULL NULL 365 val_365 +NULL NULL NULL NULL 366 val_366 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 368 val_368 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 373 val_373 +NULL NULL NULL NULL 374 val_374 +NULL NULL NULL NULL 375 val_375 +NULL NULL NULL NULL 377 val_377 +NULL NULL NULL NULL 378 val_378 +NULL NULL NULL NULL 379 val_379 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 386 val_386 +NULL NULL NULL NULL 389 val_389 +NULL NULL NULL NULL 392 val_392 +NULL NULL NULL NULL 393 val_393 +NULL NULL NULL NULL 394 val_394 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 400 val_400 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 402 val_402 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 407 val_407 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 411 val_411 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 418 val_418 +NULL NULL NULL NULL 419 val_419 +NULL NULL NULL NULL 421 val_421 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 427 val_427 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 432 val_432 +NULL NULL NULL NULL 435 val_435 +NULL NULL NULL NULL 436 val_436 +NULL NULL NULL NULL 437 val_437 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 443 val_443 +NULL NULL NULL NULL 444 val_444 +NULL NULL NULL NULL 446 val_446 +NULL NULL NULL NULL 448 val_448 +NULL NULL NULL NULL 449 val_449 +NULL NULL NULL NULL 452 val_452 +NULL NULL NULL NULL 453 val_453 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 455 val_455 +NULL NULL NULL NULL 457 val_457 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 460 val_460 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 467 val_467 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 470 val_470 +NULL NULL NULL NULL 472 val_472 +NULL NULL NULL NULL 475 val_475 +NULL NULL NULL NULL 477 val_477 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 479 val_479 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 481 val_481 +NULL NULL NULL NULL 482 val_482 +NULL NULL NULL NULL 483 val_483 +NULL NULL NULL NULL 484 val_484 +NULL NULL NULL NULL 485 val_485 +NULL NULL NULL NULL 487 val_487 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 490 val_490 +NULL NULL NULL NULL 491 val_491 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 493 val_493 +NULL NULL NULL NULL 494 val_494 +NULL NULL NULL NULL 495 val_495 +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 +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 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 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 +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 0 val_0 0 val_0 +0 val_0 0 val_0 0 val_0 +2 val_2 2 val_2 2 val_2 +4 val_4 4 val_4 4 val_4 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +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 diff --git a/src/test/hive/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 b/src/test/hive/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f b/src/test/hive/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f new file mode 100644 index 0000000000000..9672e21fa0323 --- /dev/null +++ b/src/test/hive/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f @@ -0,0 +1,500 @@ +NULL NULL NULL NULL 0 val_0 +NULL NULL NULL NULL 0 val_0 +NULL NULL NULL NULL 0 val_0 +NULL NULL NULL NULL 2 val_2 +NULL NULL NULL NULL 4 val_4 +NULL NULL NULL NULL 5 val_5 +NULL NULL NULL NULL 5 val_5 +NULL NULL NULL NULL 5 val_5 +NULL NULL NULL NULL 8 val_8 +NULL NULL NULL NULL 9 val_9 +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 11 val_11 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 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 NULL NULL 26 val_26 +NULL NULL NULL NULL 26 val_26 +NULL NULL NULL NULL 27 val_27 +NULL NULL NULL NULL 28 val_28 +NULL NULL NULL NULL 30 val_30 +NULL NULL NULL NULL 33 val_33 +NULL NULL NULL NULL 34 val_34 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 41 val_41 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 43 val_43 +NULL NULL NULL NULL 44 val_44 +NULL NULL NULL NULL 47 val_47 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 53 val_53 +NULL NULL NULL NULL 54 val_54 +NULL NULL NULL NULL 57 val_57 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 64 val_64 +NULL NULL NULL NULL 65 val_65 +NULL NULL NULL NULL 66 val_66 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 69 val_69 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 74 val_74 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 77 val_77 +NULL NULL NULL NULL 78 val_78 +NULL NULL NULL NULL 80 val_80 +NULL NULL NULL NULL 82 val_82 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 85 val_85 +NULL NULL NULL NULL 86 val_86 +NULL NULL NULL NULL 87 val_87 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 92 val_92 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 96 val_96 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 105 val_105 +NULL NULL NULL NULL 111 val_111 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 114 val_114 +NULL NULL NULL NULL 116 val_116 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 126 val_126 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 131 val_131 +NULL NULL NULL NULL 133 val_133 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 136 val_136 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 143 val_143 +NULL NULL NULL NULL 145 val_145 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 150 val_150 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 153 val_153 +NULL NULL NULL NULL 155 val_155 +NULL NULL NULL NULL 156 val_156 +NULL NULL NULL NULL 157 val_157 +NULL NULL NULL NULL 158 val_158 +NULL NULL NULL NULL 160 val_160 +NULL NULL NULL NULL 162 val_162 +NULL NULL NULL NULL 163 val_163 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 166 val_166 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 168 val_168 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 170 val_170 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 177 val_177 +NULL NULL NULL NULL 178 val_178 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 180 val_180 +NULL NULL NULL NULL 181 val_181 +NULL NULL NULL NULL 183 val_183 +NULL NULL NULL NULL 186 val_186 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 189 val_189 +NULL NULL NULL NULL 190 val_190 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 192 val_192 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 194 val_194 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 196 val_196 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 201 val_201 +NULL NULL NULL NULL 202 val_202 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 214 val_214 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 218 val_218 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 222 val_222 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 226 val_226 +NULL NULL NULL NULL 228 val_228 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 235 val_235 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 241 val_241 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 244 val_244 +NULL NULL NULL NULL 247 val_247 +NULL NULL NULL NULL 248 val_248 +NULL NULL NULL NULL 249 val_249 +NULL NULL NULL NULL 252 val_252 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 257 val_257 +NULL NULL NULL NULL 258 val_258 +NULL NULL NULL NULL 260 val_260 +NULL NULL NULL NULL 262 val_262 +NULL NULL NULL NULL 263 val_263 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 266 val_266 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 274 val_274 +NULL NULL NULL NULL 275 val_275 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 283 val_283 +NULL NULL NULL NULL 284 val_284 +NULL NULL NULL NULL 285 val_285 +NULL NULL NULL NULL 286 val_286 +NULL NULL NULL NULL 287 val_287 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 289 val_289 +NULL NULL NULL NULL 291 val_291 +NULL NULL NULL NULL 292 val_292 +NULL NULL NULL NULL 296 val_296 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 302 val_302 +NULL NULL NULL NULL 305 val_305 +NULL NULL NULL NULL 306 val_306 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 308 val_308 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 310 val_310 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 315 val_315 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 323 val_323 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 332 val_332 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 335 val_335 +NULL NULL NULL NULL 336 val_336 +NULL NULL NULL NULL 338 val_338 +NULL NULL NULL NULL 339 val_339 +NULL NULL NULL NULL 341 val_341 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 345 val_345 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 351 val_351 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 356 val_356 +NULL NULL NULL NULL 360 val_360 +NULL NULL NULL NULL 362 val_362 +NULL NULL NULL NULL 364 val_364 +NULL NULL NULL NULL 365 val_365 +NULL NULL NULL NULL 366 val_366 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 368 val_368 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 373 val_373 +NULL NULL NULL NULL 374 val_374 +NULL NULL NULL NULL 375 val_375 +NULL NULL NULL NULL 377 val_377 +NULL NULL NULL NULL 378 val_378 +NULL NULL NULL NULL 379 val_379 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 386 val_386 +NULL NULL NULL NULL 389 val_389 +NULL NULL NULL NULL 392 val_392 +NULL NULL NULL NULL 393 val_393 +NULL NULL NULL NULL 394 val_394 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 400 val_400 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 402 val_402 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 407 val_407 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 411 val_411 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 418 val_418 +NULL NULL NULL NULL 419 val_419 +NULL NULL NULL NULL 421 val_421 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 427 val_427 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 432 val_432 +NULL NULL NULL NULL 435 val_435 +NULL NULL NULL NULL 436 val_436 +NULL NULL NULL NULL 437 val_437 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 443 val_443 +NULL NULL NULL NULL 444 val_444 +NULL NULL NULL NULL 446 val_446 +NULL NULL NULL NULL 448 val_448 +NULL NULL NULL NULL 449 val_449 +NULL NULL NULL NULL 452 val_452 +NULL NULL NULL NULL 453 val_453 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 455 val_455 +NULL NULL NULL NULL 457 val_457 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 460 val_460 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 467 val_467 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 470 val_470 +NULL NULL NULL NULL 472 val_472 +NULL NULL NULL NULL 475 val_475 +NULL NULL NULL NULL 477 val_477 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 479 val_479 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 481 val_481 +NULL NULL NULL NULL 482 val_482 +NULL NULL NULL NULL 483 val_483 +NULL NULL NULL NULL 484 val_484 +NULL NULL NULL NULL 485 val_485 +NULL NULL NULL NULL 487 val_487 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 490 val_490 +NULL NULL NULL NULL 491 val_491 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 493 val_493 +NULL NULL NULL NULL 494 val_494 +NULL NULL NULL NULL 495 val_495 +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 diff --git a/src/test/hive/golden/join22-0-97081b5f73052a612fb391a57ad9b7b b/src/test/hive/golden/join22-0-97081b5f73052a612fb391a57ad9b7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d b/src/test/hive/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 b/src/test/hive/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 new file mode 100644 index 0000000000000..5707ed08e7e54 --- /dev/null +++ b/src/test/hive/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 @@ -0,0 +1,100 @@ +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 +0 val_0 2 val_2 +0 val_0 2 val_2 +0 val_0 4 val_4 +0 val_0 4 val_4 +0 val_0 4 val_4 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 5 val_5 +0 val_0 8 val_8 +0 val_0 8 val_8 +0 val_0 8 val_8 +0 val_0 9 val_9 +0 val_0 9 val_9 +0 val_0 9 val_9 +2 val_2 0 val_0 +2 val_2 0 val_0 +2 val_2 0 val_0 +2 val_2 2 val_2 +2 val_2 4 val_4 +2 val_2 5 val_5 +2 val_2 5 val_5 +2 val_2 5 val_5 +2 val_2 8 val_8 +2 val_2 9 val_9 +4 val_4 0 val_0 +4 val_4 0 val_0 +4 val_4 0 val_0 +4 val_4 2 val_2 +4 val_4 4 val_4 +4 val_4 5 val_5 +4 val_4 5 val_5 +4 val_4 5 val_5 +4 val_4 8 val_8 +4 val_4 9 val_9 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 0 val_0 +5 val_5 2 val_2 +5 val_5 2 val_2 +5 val_5 2 val_2 +5 val_5 4 val_4 +5 val_5 4 val_4 +5 val_5 4 val_4 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 8 val_8 +5 val_5 8 val_8 +5 val_5 8 val_8 +5 val_5 9 val_9 +5 val_5 9 val_9 +5 val_5 9 val_9 +8 val_8 0 val_0 +8 val_8 0 val_0 +8 val_8 0 val_0 +8 val_8 2 val_2 +8 val_8 4 val_4 +8 val_8 5 val_5 +8 val_8 5 val_5 +8 val_8 5 val_5 +8 val_8 8 val_8 +8 val_8 9 val_9 +9 val_9 0 val_0 +9 val_9 0 val_0 +9 val_9 0 val_0 +9 val_9 2 val_2 +9 val_9 4 val_4 +9 val_9 5 val_5 +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 diff --git a/src/test/hive/golden/join24-0-721dfa03bfea05e55506c571b6c3585b b/src/test/hive/golden/join24-0-721dfa03bfea05e55506c571b6c3585b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d b/src/test/hive/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 b/src/test/hive/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 b/src/test/hive/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join25-1-360b3676605ccb3d89cf555342db87af b/src/test/hive/golden/join25-1-360b3676605ccb3d89cf555342db87af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join25-2-ef930bed933341636c71622e6f89e9e9 b/src/test/hive/golden/join25-2-ef930bed933341636c71622e6f89e9e9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 b/src/test/hive/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..183353e5c705b --- /dev/null +++ b/src/test/hive/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,37 @@ +66 val_66 val_66 +98 val_98 val_98 +98 val_98 val_98 +128 val_128 +128 val_128 +128 val_128 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +224 val_224 +224 val_224 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +369 val_369 +369 val_369 +369 val_369 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +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 diff --git a/src/test/hive/golden/join26-0-775b1af8eae9b2250052be1b72b4086 b/src/test/hive/golden/join26-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join26-1-10fbe6c99e93cfeafcd454261744728e b/src/test/hive/golden/join26-1-10fbe6c99e93cfeafcd454261744728e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d b/src/test/hive/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..4c88927e56579 --- /dev/null +++ b/src/test/hive/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,107 @@ +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +224 val_224 val_224 +224 val_224 val_224 +224 val_224 val_224 +224 val_224 val_224 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join27-0-360b3676605ccb3d89cf555342db87af b/src/test/hive/golden/join27-0-360b3676605ccb3d89cf555342db87af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 b/src/test/hive/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 b/src/test/hive/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 b/src/test/hive/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 new file mode 100644 index 0000000000000..2b75023ea4a9f --- /dev/null +++ b/src/test/hive/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 @@ -0,0 +1,41 @@ +NULL val_165 val_165 +NULL val_165 val_165 +NULL val_193 val_193 +NULL val_193 val_193 +NULL val_193 val_193 +NULL val_265 val_265 +NULL val_265 val_265 +NULL val_27 val_27 +NULL val_409 val_409 +NULL val_409 val_409 +NULL val_409 val_409 +NULL val_484 val_484 +66 val_66 val_66 +98 val_98 val_98 +98 val_98 val_98 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +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 diff --git a/src/test/hive/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 b/src/test/hive/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join28-1-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/join28-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join28-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join28-3-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/join28-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join28-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 b/src/test/hive/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 b/src/test/hive/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..da3c427cab73f --- /dev/null +++ b/src/test/hive/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,107 @@ +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +150 val_150 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +66 val_66 +98 val_98 +98 val_98 +98 val_98 +98 val_98 \ No newline at end of file diff --git a/src/test/hive/golden/join29-0-e78402f7585b17c76c32790571d749eb b/src/test/hive/golden/join29-0-e78402f7585b17c76c32790571d749eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join29-1-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/join29-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join29-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join29-3-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/join29-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join29-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join29-4-b0524de7985cdb7b57fd995721654856 b/src/test/hive/golden/join29-4-b0524de7985cdb7b57fd995721654856 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 b/src/test/hive/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..f553ce0ca41f3 --- /dev/null +++ b/src/test/hive/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,15 @@ +128 1 3 +146 1 2 +150 1 1 +213 1 2 +224 1 2 +238 1 2 +255 1 2 +273 1 3 +278 1 2 +311 1 3 +369 1 3 +401 1 5 +406 1 4 +66 1 1 +98 1 2 \ No newline at end of file diff --git a/src/test/hive/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 b/src/test/hive/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 b/src/test/hive/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..8886dc7e8f229 --- /dev/null +++ b/src/test/hive/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,2654 @@ +238 val_238 +238 val_238 +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +484 val_484 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +429 val_429 +429 val_429 +374 val_374 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +82 val_82 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +394 val_394 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +482 val_482 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +494 val_494 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +339 val_339 +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 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +4 val_4 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +54 val_54 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +260 val_260 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +463 val_463 +463 val_463 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +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 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +85 val_85 +77 val_77 +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 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +419 val_419 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +458 val_458 +458 val_458 +78 val_78 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +64 val_64 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +491 val_491 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +478 val_478 +478 val_478 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +214 val_214 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +492 val_492 +492 val_492 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +458 val_458 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +448 val_448 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +194 val_194 +414 val_414 +414 val_414 +414 val_414 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +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 diff --git a/src/test/hive/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 b/src/test/hive/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d b/src/test/hive/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join30-2-194edec0592cf441617ca7caf9756baa b/src/test/hive/golden/join30-2-194edec0592cf441617ca7caf9756baa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..71094ee7360db --- /dev/null +++ b/src/test/hive/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,15 @@ +66 1 +98 2 +128 3 +146 2 +150 1 +213 2 +224 2 +238 2 +255 2 +273 3 +278 2 +311 3 +369 3 +401 5 +406 4 \ No newline at end of file diff --git a/src/test/hive/golden/join31-0-619db2f9dd69aa236aa804cced726c9a b/src/test/hive/golden/join31-0-619db2f9dd69aa236aa804cced726c9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join31-1-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/join31-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join31-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join31-3-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/join31-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join31-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 b/src/test/hive/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 b/src/test/hive/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..5d28208ab255c --- /dev/null +++ b/src/test/hive/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,15 @@ +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 +66 1 +98 1 \ No newline at end of file diff --git a/src/test/hive/golden/join32-0-775b1af8eae9b2250052be1b72b4086 b/src/test/hive/golden/join32-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32-1-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/join32-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join32-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join32-3-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/join32-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join32-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb b/src/test/hive/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32-5-425c5c35b23c42254dabec49aa6613bf b/src/test/hive/golden/join32-5-425c5c35b23c42254dabec49aa6613bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/src/test/hive/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,85 @@ +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 b/src/test/hive/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 b/src/test/hive/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/src/test/hive/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/src/test/hive/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a @@ -0,0 +1,85 @@ +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca b/src/test/hive/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 b/src/test/hive/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 b/src/test/hive/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/src/test/hive/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 @@ -0,0 +1,85 @@ +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a b/src/test/hive/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e b/src/test/hive/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 b/src/test/hive/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 new file mode 100644 index 0000000000000..b1251794645ce --- /dev/null +++ b/src/test/hive/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 @@ -0,0 +1,97 @@ +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +NULL val_165 val_165 +NULL val_165 val_165 +NULL val_193 val_193 +NULL val_193 val_193 +NULL val_193 val_193 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +NULL val_265 val_265 +NULL val_265 val_265 +NULL val_27 val_27 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +NULL val_409 val_409 +NULL val_409 val_409 +NULL val_409 val_409 +NULL val_484 val_484 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 b/src/test/hive/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d b/src/test/hive/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 b/src/test/hive/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/src/test/hive/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 @@ -0,0 +1,85 @@ +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 b/src/test/hive/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 b/src/test/hive/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 b/src/test/hive/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/src/test/hive/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 @@ -0,0 +1,85 @@ +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 b/src/test/hive/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb b/src/test/hive/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf b/src/test/hive/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/src/test/hive/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/src/test/hive/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a @@ -0,0 +1,85 @@ +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b b/src/test/hive/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 b/src/test/hive/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join33-0-775b1af8eae9b2250052be1b72b4086 b/src/test/hive/golden/join33-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join33-1-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/join33-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join33-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join33-3-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/join33-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join33-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb b/src/test/hive/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join33-5-425c5c35b23c42254dabec49aa6613bf b/src/test/hive/golden/join33-5-425c5c35b23c42254dabec49aa6613bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/src/test/hive/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,85 @@ +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join34-0-775b1af8eae9b2250052be1b72b4086 b/src/test/hive/golden/join34-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join34-1-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/join34-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join34-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join34-3-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/join34-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join34-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join34-4-51cc5d253befd784a9d466c0b402f47a b/src/test/hive/golden/join34-4-51cc5d253befd784a9d466c0b402f47a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 b/src/test/hive/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..e18d1ff802c93 --- /dev/null +++ b/src/test/hive/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,34 @@ +128 val_128 +128 val_128 +128 val_128 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +224 val_224 +224 val_224 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +369 val_369 +369 val_369 +369 val_369 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +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 diff --git a/src/test/hive/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 b/src/test/hive/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join35-1-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/join35-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join35-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join35-3-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/join35-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join35-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join35-4-99cb704ccc2813400908a62571891dc7 b/src/test/hive/golden/join35-4-99cb704ccc2813400908a62571891dc7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join35-5-60956f3307651237f52ffbe41c827c1c b/src/test/hive/golden/join35-5-60956f3307651237f52ffbe41c827c1c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..efdd80c9f8b89 --- /dev/null +++ b/src/test/hive/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,13 @@ +128 3 +146 val_146 2 +150 val_150 1 +213 val_213 2 +224 2 +238 val_238 2 +255 val_255 2 +273 val_273 3 +278 val_278 2 +311 val_311 3 +369 3 +401 val_401 5 +406 val_406 4 \ No newline at end of file diff --git a/src/test/hive/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 b/src/test/hive/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join36-1-3aba153d62042206c0f001ea74980da b/src/test/hive/golden/join36-1-3aba153d62042206c0f001ea74980da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 b/src/test/hive/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 b/src/test/hive/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 b/src/test/hive/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 b/src/test/hive/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 b/src/test/hive/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join36-7-171ede21302050a33468d95e329a38a3 b/src/test/hive/golden/join36-7-171ede21302050a33468d95e329a38a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..a250f202c4df0 --- /dev/null +++ b/src/test/hive/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,309 @@ +0 3 3 +2 1 1 +4 1 1 +5 3 3 +8 1 1 +9 1 1 +10 1 1 +11 1 1 +12 2 2 +15 2 2 +17 1 1 +18 2 2 +19 1 1 +20 1 1 +24 2 2 +26 2 2 +27 1 1 +28 1 1 +30 1 1 +33 1 1 +34 1 1 +35 3 3 +37 2 2 +41 1 1 +42 2 2 +43 1 1 +44 1 1 +47 1 1 +51 2 2 +53 1 1 +54 1 1 +57 1 1 +58 2 2 +64 1 1 +65 1 1 +66 1 1 +67 2 2 +69 1 1 +70 3 3 +72 2 2 +74 1 1 +76 2 2 +77 1 1 +78 1 1 +80 1 1 +82 1 1 +83 2 2 +84 2 2 +85 1 1 +86 1 1 +87 1 1 +90 3 3 +92 1 1 +95 2 2 +96 1 1 +97 2 2 +98 2 2 +100 2 2 +103 2 2 +104 2 2 +105 1 1 +111 1 1 +113 2 2 +114 1 1 +116 1 1 +118 2 2 +119 3 3 +120 2 2 +125 2 2 +126 1 1 +128 3 3 +129 2 2 +131 1 1 +133 1 1 +134 2 2 +136 1 1 +137 2 2 +138 4 4 +143 1 1 +145 1 1 +146 2 2 +149 2 2 +150 1 1 +152 2 2 +153 1 1 +155 1 1 +156 1 1 +157 1 1 +158 1 1 +160 1 1 +162 1 1 +163 1 1 +164 2 2 +165 2 2 +166 1 1 +167 3 3 +168 1 1 +169 4 4 +170 1 1 +172 2 2 +174 2 2 +175 2 2 +176 2 2 +177 1 1 +178 1 1 +179 2 2 +180 1 1 +181 1 1 +183 1 1 +186 1 1 +187 3 3 +189 1 1 +190 1 1 +191 2 2 +192 1 1 +193 3 3 +194 1 1 +195 2 2 +196 1 1 +197 2 2 +199 3 3 +200 2 2 +201 1 1 +202 1 1 +203 2 2 +205 2 2 +207 2 2 +208 3 3 +209 2 2 +213 2 2 +214 1 1 +216 2 2 +217 2 2 +218 1 1 +219 2 2 +221 2 2 +222 1 1 +223 2 2 +224 2 2 +226 1 1 +228 1 1 +229 2 2 +230 5 5 +233 2 2 +235 1 1 +237 2 2 +238 2 2 +239 2 2 +241 1 1 +242 2 2 +244 1 1 +247 1 1 +248 1 1 +249 1 1 +252 1 1 +255 2 2 +256 2 2 +257 1 1 +258 1 1 +260 1 1 +262 1 1 +263 1 1 +265 2 2 +266 1 1 +272 2 2 +273 3 3 +274 1 1 +275 1 1 +277 4 4 +278 2 2 +280 2 2 +281 2 2 +282 2 2 +283 1 1 +284 1 1 +285 1 1 +286 1 1 +287 1 1 +288 2 2 +289 1 1 +291 1 1 +292 1 1 +296 1 1 +298 3 3 +302 1 1 +305 1 1 +306 1 1 +307 2 2 +308 1 1 +309 2 2 +310 1 1 +311 3 3 +315 1 1 +316 3 3 +317 2 2 +318 3 3 +321 2 2 +322 2 2 +323 1 1 +325 2 2 +327 3 3 +331 2 2 +332 1 1 +333 2 2 +335 1 1 +336 1 1 +338 1 1 +339 1 1 +341 1 1 +342 2 2 +344 2 2 +345 1 1 +348 5 5 +351 1 1 +353 2 2 +356 1 1 +360 1 1 +362 1 1 +364 1 1 +365 1 1 +366 1 1 +367 2 2 +368 1 1 +369 3 3 +373 1 1 +374 1 1 +375 1 1 +377 1 1 +378 1 1 +379 1 1 +382 2 2 +384 3 3 +386 1 1 +389 1 1 +392 1 1 +393 1 1 +394 1 1 +395 2 2 +396 3 3 +397 2 2 +399 2 2 +400 1 1 +401 5 5 +402 1 1 +403 3 3 +404 2 2 +406 4 4 +407 1 1 +409 3 3 +411 1 1 +413 2 2 +414 2 2 +417 3 3 +418 1 1 +419 1 1 +421 1 1 +424 2 2 +427 1 1 +429 2 2 +430 3 3 +431 3 3 +432 1 1 +435 1 1 +436 1 1 +437 1 1 +438 3 3 +439 2 2 +443 1 1 +444 1 1 +446 1 1 +448 1 1 +449 1 1 +452 1 1 +453 1 1 +454 3 3 +455 1 1 +457 1 1 +458 2 2 +459 2 2 +460 1 1 +462 2 2 +463 2 2 +466 3 3 +467 1 1 +468 4 4 +469 5 5 +470 1 1 +472 1 1 +475 1 1 +477 1 1 +478 2 2 +479 1 1 +480 3 3 +481 1 1 +482 1 1 +483 1 1 +484 1 1 +485 1 1 +487 1 1 +489 4 4 +490 1 1 +491 1 1 +492 2 2 +493 1 1 +494 1 1 +495 1 1 +496 1 1 +497 1 1 +498 3 3 \ No newline at end of file diff --git a/src/test/hive/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 b/src/test/hive/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join37-1-360b3676605ccb3d89cf555342db87af b/src/test/hive/golden/join37-1-360b3676605ccb3d89cf555342db87af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join37-2-2310f2b8726137811543e04db98dd1db b/src/test/hive/golden/join37-2-2310f2b8726137811543e04db98dd1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join37-3-c4b3a85965108ad7013ac3931598af09 b/src/test/hive/golden/join37-3-c4b3a85965108ad7013ac3931598af09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..183353e5c705b --- /dev/null +++ b/src/test/hive/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,37 @@ +66 val_66 val_66 +98 val_98 val_98 +98 val_98 val_98 +128 val_128 +128 val_128 +128 val_128 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +224 val_224 +224 val_224 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +369 val_369 +369 val_369 +369 val_369 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +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 diff --git a/src/test/hive/golden/join38-0-42e6ad674a5142d05258a2d29578e046 b/src/test/hive/golden/join38-0-42e6ad674a5142d05258a2d29578e046 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f b/src/test/hive/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join38-2-eacdb3417c4a563982c488812d654c9 b/src/test/hive/golden/join38-2-eacdb3417c4a563982c488812d654c9 new file mode 100644 index 0000000000000..5e020e3aff653 --- /dev/null +++ b/src/test/hive/golden/join38-2-eacdb3417c4a563982c488812d654c9 @@ -0,0 +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 diff --git a/src/test/hive/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 b/src/test/hive/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join38-4-53d219706847e890de1dcd369563ebef b/src/test/hive/golden/join38-4-53d219706847e890de1dcd369563ebef new file mode 100644 index 0000000000000..fce97c9211916 --- /dev/null +++ b/src/test/hive/golden/join38-4-53d219706847e890de1dcd369563ebef @@ -0,0 +1 @@ +val_111 105 2 \ No newline at end of file diff --git a/src/test/hive/golden/join39-0-54e9f7924abb40d1c601de908de6f25b b/src/test/hive/golden/join39-0-54e9f7924abb40d1c601de908de6f25b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 b/src/test/hive/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 b/src/test/hive/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..5b45955ceb274 --- /dev/null +++ b/src/test/hive/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,566 @@ +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 +10 val_10 10 val_10 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +103 val_103 NULL NULL +103 val_103 NULL NULL +104 val_104 NULL NULL +104 val_104 NULL NULL +105 val_105 NULL NULL +11 val_11 11 val_11 +111 val_111 NULL NULL +113 val_113 NULL NULL +113 val_113 NULL NULL +114 val_114 NULL NULL +116 val_116 NULL NULL +118 val_118 NULL NULL +118 val_118 NULL NULL +119 val_119 NULL NULL +119 val_119 NULL NULL +119 val_119 NULL NULL +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +120 val_120 NULL NULL +120 val_120 NULL NULL +125 val_125 NULL NULL +125 val_125 NULL NULL +126 val_126 NULL NULL +128 val_128 NULL NULL +128 val_128 NULL NULL +128 val_128 NULL NULL +129 val_129 NULL NULL +129 val_129 NULL NULL +131 val_131 NULL NULL +133 val_133 NULL NULL +134 val_134 NULL NULL +134 val_134 NULL NULL +136 val_136 NULL NULL +137 val_137 NULL NULL +137 val_137 NULL NULL +138 val_138 NULL NULL +138 val_138 NULL NULL +138 val_138 NULL NULL +138 val_138 NULL NULL +143 val_143 NULL NULL +145 val_145 NULL NULL +146 val_146 NULL NULL +146 val_146 NULL NULL +149 val_149 NULL NULL +149 val_149 NULL NULL +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +150 val_150 NULL NULL +152 val_152 NULL NULL +152 val_152 NULL NULL +153 val_153 NULL NULL +155 val_155 NULL NULL +156 val_156 NULL NULL +157 val_157 NULL NULL +158 val_158 NULL NULL +160 val_160 NULL NULL +162 val_162 NULL NULL +163 val_163 NULL NULL +164 val_164 NULL NULL +164 val_164 NULL NULL +165 val_165 NULL NULL +165 val_165 NULL NULL +166 val_166 NULL NULL +167 val_167 NULL NULL +167 val_167 NULL NULL +167 val_167 NULL NULL +168 val_168 NULL NULL +169 val_169 NULL NULL +169 val_169 NULL NULL +169 val_169 NULL NULL +169 val_169 NULL NULL +17 val_17 17 val_17 +170 val_170 NULL NULL +172 val_172 NULL NULL +172 val_172 NULL NULL +174 val_174 NULL NULL +174 val_174 NULL NULL +175 val_175 NULL NULL +175 val_175 NULL NULL +176 val_176 NULL NULL +176 val_176 NULL NULL +177 val_177 NULL NULL +178 val_178 NULL NULL +179 val_179 NULL NULL +179 val_179 NULL NULL +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +180 val_180 NULL NULL +181 val_181 NULL NULL +183 val_183 NULL NULL +186 val_186 NULL NULL +187 val_187 NULL NULL +187 val_187 NULL NULL +187 val_187 NULL NULL +189 val_189 NULL NULL +19 val_19 19 val_19 +190 val_190 NULL NULL +191 val_191 NULL NULL +191 val_191 NULL NULL +192 val_192 NULL NULL +193 val_193 NULL NULL +193 val_193 NULL NULL +193 val_193 NULL NULL +194 val_194 NULL NULL +195 val_195 NULL NULL +195 val_195 NULL NULL +196 val_196 NULL NULL +197 val_197 NULL NULL +197 val_197 NULL NULL +199 val_199 NULL NULL +199 val_199 NULL NULL +199 val_199 NULL NULL +2 val_2 2 val_2 +20 val_20 20 val_20 +200 val_200 NULL NULL +200 val_200 NULL NULL +201 val_201 NULL NULL +202 val_202 NULL NULL +203 val_203 NULL NULL +203 val_203 NULL NULL +205 val_205 NULL NULL +205 val_205 NULL NULL +207 val_207 NULL NULL +207 val_207 NULL NULL +208 val_208 NULL NULL +208 val_208 NULL NULL +208 val_208 NULL NULL +209 val_209 NULL NULL +209 val_209 NULL NULL +213 val_213 NULL NULL +213 val_213 NULL NULL +214 val_214 NULL NULL +216 val_216 NULL NULL +216 val_216 NULL NULL +217 val_217 NULL NULL +217 val_217 NULL NULL +218 val_218 NULL NULL +219 val_219 NULL NULL +219 val_219 NULL NULL +221 val_221 NULL NULL +221 val_221 NULL NULL +222 val_222 NULL NULL +223 val_223 NULL NULL +223 val_223 NULL NULL +224 val_224 NULL NULL +224 val_224 NULL NULL +226 val_226 NULL NULL +228 val_228 NULL NULL +229 val_229 NULL NULL +229 val_229 NULL NULL +230 val_230 NULL NULL +230 val_230 NULL NULL +230 val_230 NULL NULL +230 val_230 NULL NULL +230 val_230 NULL NULL +233 val_233 NULL NULL +233 val_233 NULL NULL +235 val_235 NULL NULL +237 val_237 NULL NULL +237 val_237 NULL NULL +238 val_238 NULL NULL +238 val_238 NULL NULL +239 val_239 NULL NULL +239 val_239 NULL NULL +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +241 val_241 NULL NULL +242 val_242 NULL NULL +242 val_242 NULL NULL +244 val_244 NULL NULL +247 val_247 NULL NULL +248 val_248 NULL NULL +249 val_249 NULL NULL +252 val_252 NULL NULL +255 val_255 NULL NULL +255 val_255 NULL NULL +256 val_256 NULL NULL +256 val_256 NULL NULL +257 val_257 NULL NULL +258 val_258 NULL NULL +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +260 val_260 NULL NULL +262 val_262 NULL NULL +263 val_263 NULL NULL +265 val_265 NULL NULL +265 val_265 NULL NULL +266 val_266 NULL NULL +27 val_27 27 val_27 +272 val_272 NULL NULL +272 val_272 NULL NULL +273 val_273 NULL NULL +273 val_273 NULL NULL +273 val_273 NULL NULL +274 val_274 NULL NULL +275 val_275 NULL NULL +277 val_277 NULL NULL +277 val_277 NULL NULL +277 val_277 NULL NULL +277 val_277 NULL NULL +278 val_278 NULL NULL +278 val_278 NULL NULL +28 val_28 28 val_28 +280 val_280 NULL NULL +280 val_280 NULL NULL +281 val_281 NULL NULL +281 val_281 NULL NULL +282 val_282 NULL NULL +282 val_282 NULL NULL +283 val_283 NULL NULL +284 val_284 NULL NULL +285 val_285 NULL NULL +286 val_286 NULL NULL +287 val_287 NULL NULL +288 val_288 NULL NULL +288 val_288 NULL NULL +289 val_289 NULL NULL +291 val_291 NULL NULL +292 val_292 NULL NULL +296 val_296 NULL NULL +298 val_298 NULL NULL +298 val_298 NULL NULL +298 val_298 NULL NULL +30 val_30 30 val_30 +302 val_302 NULL NULL +305 val_305 NULL NULL +306 val_306 NULL NULL +307 val_307 NULL NULL +307 val_307 NULL NULL +308 val_308 NULL NULL +309 val_309 NULL NULL +309 val_309 NULL NULL +310 val_310 NULL NULL +311 val_311 NULL NULL +311 val_311 NULL NULL +311 val_311 NULL NULL +315 val_315 NULL NULL +316 val_316 NULL NULL +316 val_316 NULL NULL +316 val_316 NULL NULL +317 val_317 NULL NULL +317 val_317 NULL NULL +318 val_318 NULL NULL +318 val_318 NULL NULL +318 val_318 NULL NULL +321 val_321 NULL NULL +321 val_321 NULL NULL +322 val_322 NULL NULL +322 val_322 NULL NULL +323 val_323 NULL NULL +325 val_325 NULL NULL +325 val_325 NULL NULL +327 val_327 NULL NULL +327 val_327 NULL NULL +327 val_327 NULL NULL +33 val_33 33 val_33 +331 val_331 NULL NULL +331 val_331 NULL NULL +332 val_332 NULL NULL +333 val_333 NULL NULL +333 val_333 NULL NULL +335 val_335 NULL NULL +336 val_336 NULL NULL +338 val_338 NULL NULL +339 val_339 NULL NULL +34 val_34 34 val_34 +341 val_341 NULL NULL +342 val_342 NULL NULL +342 val_342 NULL NULL +344 val_344 NULL NULL +344 val_344 NULL NULL +345 val_345 NULL NULL +348 val_348 NULL NULL +348 val_348 NULL NULL +348 val_348 NULL NULL +348 val_348 NULL NULL +348 val_348 NULL NULL +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +351 val_351 NULL NULL +353 val_353 NULL NULL +353 val_353 NULL NULL +356 val_356 NULL NULL +360 val_360 NULL NULL +362 val_362 NULL NULL +364 val_364 NULL NULL +365 val_365 NULL NULL +366 val_366 NULL NULL +367 val_367 NULL NULL +367 val_367 NULL NULL +368 val_368 NULL NULL +369 val_369 NULL NULL +369 val_369 NULL NULL +369 val_369 NULL NULL +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +373 val_373 NULL NULL +374 val_374 NULL NULL +375 val_375 NULL NULL +377 val_377 NULL NULL +378 val_378 NULL NULL +379 val_379 NULL NULL +382 val_382 NULL NULL +382 val_382 NULL NULL +384 val_384 NULL NULL +384 val_384 NULL NULL +384 val_384 NULL NULL +386 val_386 NULL NULL +389 val_389 NULL NULL +392 val_392 NULL NULL +393 val_393 NULL NULL +394 val_394 NULL NULL +395 val_395 NULL NULL +395 val_395 NULL NULL +396 val_396 NULL NULL +396 val_396 NULL NULL +396 val_396 NULL NULL +397 val_397 NULL NULL +397 val_397 NULL NULL +399 val_399 NULL NULL +399 val_399 NULL NULL +4 val_4 4 val_4 +400 val_400 NULL NULL +401 val_401 NULL NULL +401 val_401 NULL NULL +401 val_401 NULL NULL +401 val_401 NULL NULL +401 val_401 NULL NULL +402 val_402 NULL NULL +403 val_403 NULL NULL +403 val_403 NULL NULL +403 val_403 NULL NULL +404 val_404 NULL NULL +404 val_404 NULL NULL +406 val_406 NULL NULL +406 val_406 NULL NULL +406 val_406 NULL NULL +406 val_406 NULL NULL +407 val_407 NULL NULL +409 val_409 NULL NULL +409 val_409 NULL NULL +409 val_409 NULL NULL +41 val_41 41 val_41 +411 val_411 NULL NULL +413 val_413 NULL NULL +413 val_413 NULL NULL +414 val_414 NULL NULL +414 val_414 NULL NULL +417 val_417 NULL NULL +417 val_417 NULL NULL +417 val_417 NULL NULL +418 val_418 NULL NULL +419 val_419 NULL NULL +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +421 val_421 NULL NULL +424 val_424 NULL NULL +424 val_424 NULL NULL +427 val_427 NULL NULL +429 val_429 NULL NULL +429 val_429 NULL NULL +43 val_43 43 val_43 +430 val_430 NULL NULL +430 val_430 NULL NULL +430 val_430 NULL NULL +431 val_431 NULL NULL +431 val_431 NULL NULL +431 val_431 NULL NULL +432 val_432 NULL NULL +435 val_435 NULL NULL +436 val_436 NULL NULL +437 val_437 NULL NULL +438 val_438 NULL NULL +438 val_438 NULL NULL +438 val_438 NULL NULL +439 val_439 NULL NULL +439 val_439 NULL NULL +44 val_44 44 val_44 +443 val_443 NULL NULL +444 val_444 NULL NULL +446 val_446 NULL NULL +448 val_448 NULL NULL +449 val_449 NULL NULL +452 val_452 NULL NULL +453 val_453 NULL NULL +454 val_454 NULL NULL +454 val_454 NULL NULL +454 val_454 NULL NULL +455 val_455 NULL NULL +457 val_457 NULL NULL +458 val_458 NULL NULL +458 val_458 NULL NULL +459 val_459 NULL NULL +459 val_459 NULL NULL +460 val_460 NULL NULL +462 val_462 NULL NULL +462 val_462 NULL NULL +463 val_463 NULL NULL +463 val_463 NULL NULL +466 val_466 NULL NULL +466 val_466 NULL NULL +466 val_466 NULL NULL +467 val_467 NULL NULL +468 val_468 NULL NULL +468 val_468 NULL NULL +468 val_468 NULL NULL +468 val_468 NULL NULL +469 val_469 NULL NULL +469 val_469 NULL NULL +469 val_469 NULL NULL +469 val_469 NULL NULL +469 val_469 NULL NULL +47 val_47 47 val_47 +470 val_470 NULL NULL +472 val_472 NULL NULL +475 val_475 NULL NULL +477 val_477 NULL NULL +478 val_478 NULL NULL +478 val_478 NULL NULL +479 val_479 NULL NULL +480 val_480 NULL NULL +480 val_480 NULL NULL +480 val_480 NULL NULL +481 val_481 NULL NULL +482 val_482 NULL NULL +483 val_483 NULL NULL +484 val_484 NULL NULL +485 val_485 NULL NULL +487 val_487 NULL NULL +489 val_489 NULL NULL +489 val_489 NULL NULL +489 val_489 NULL NULL +489 val_489 NULL NULL +490 val_490 NULL NULL +491 val_491 NULL NULL +492 val_492 NULL NULL +492 val_492 NULL NULL +493 val_493 NULL NULL +494 val_494 NULL NULL +495 val_495 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 NULL NULL +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +53 val_53 53 val_53 +54 val_54 54 val_54 +57 val_57 57 val_57 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +64 val_64 64 val_64 +65 val_65 65 val_65 +66 val_66 66 val_66 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +69 val_69 69 val_69 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +74 val_74 74 val_74 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +77 val_77 77 val_77 +78 val_78 78 val_78 +8 val_8 8 val_8 +80 val_80 80 val_80 +82 val_82 82 val_82 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +85 val_85 85 val_85 +86 val_86 86 val_86 +87 val_87 87 val_87 +9 val_9 9 val_9 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +92 val_92 92 val_92 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +96 val_96 96 val_96 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +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 diff --git a/src/test/hive/golden/join4-0-531e91e11b3891627c2675935fda14cd b/src/test/hive/golden/join4-0-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 b/src/test/hive/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join4-2-dc967001beb776f3a859e9360823c361 b/src/test/hive/golden/join4-2-dc967001beb776f3a859e9360823c361 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..e45f3234d5a88 --- /dev/null +++ b/src/test/hive/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,11 @@ +15 val_15 NULL NULL +17 val_17 17 val_17 +12 val_12 NULL NULL +15 val_15 NULL NULL +19 val_19 19 val_19 +11 val_11 NULL NULL +12 val_12 NULL NULL +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 diff --git a/src/test/hive/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac b/src/test/hive/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 b/src/test/hive/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa b/src/test/hive/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa new file mode 100644 index 0000000000000..8543fe173f87f --- /dev/null +++ b/src/test/hive/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa @@ -0,0 +1,566 @@ +238 val_238 NULL NULL +86 val_86 86 val_86 +311 val_311 NULL NULL +27 val_27 27 val_27 +165 val_165 NULL NULL +409 val_409 NULL NULL +255 val_255 NULL NULL +278 val_278 NULL NULL +98 val_98 98 val_98 +98 val_98 98 val_98 +484 val_484 NULL NULL +265 val_265 NULL NULL +193 val_193 NULL NULL +401 val_401 NULL NULL +150 val_150 NULL NULL +273 val_273 NULL NULL +224 val_224 NULL NULL +369 val_369 NULL NULL +66 val_66 66 val_66 +128 val_128 NULL NULL +213 val_213 NULL NULL +146 val_146 NULL NULL +406 val_406 NULL NULL +429 val_429 NULL NULL +374 val_374 NULL NULL +152 val_152 NULL NULL +469 val_469 NULL NULL +145 val_145 NULL NULL +495 val_495 NULL NULL +37 val_37 37 val_37 +37 val_37 37 val_37 +327 val_327 NULL NULL +281 val_281 NULL NULL +277 val_277 NULL NULL +209 val_209 NULL NULL +15 val_15 15 val_15 +15 val_15 15 val_15 +82 val_82 82 val_82 +403 val_403 NULL NULL +166 val_166 NULL NULL +417 val_417 NULL NULL +430 val_430 NULL NULL +252 val_252 NULL NULL +292 val_292 NULL NULL +219 val_219 NULL NULL +287 val_287 NULL NULL +153 val_153 NULL NULL +193 val_193 NULL NULL +338 val_338 NULL NULL +446 val_446 NULL NULL +459 val_459 NULL NULL +394 val_394 NULL NULL +237 val_237 NULL NULL +482 val_482 NULL NULL +174 val_174 NULL NULL +413 val_413 NULL NULL +494 val_494 NULL NULL +207 val_207 NULL NULL +199 val_199 NULL NULL +466 val_466 NULL NULL +208 val_208 NULL NULL +174 val_174 NULL NULL +399 val_399 NULL NULL +396 val_396 NULL NULL +247 val_247 NULL NULL +417 val_417 NULL NULL +489 val_489 NULL NULL +162 val_162 NULL NULL +377 val_377 NULL NULL +397 val_397 NULL NULL +309 val_309 NULL NULL +365 val_365 NULL NULL +266 val_266 NULL NULL +439 val_439 NULL NULL +342 val_342 NULL NULL +367 val_367 NULL NULL +325 val_325 NULL NULL +167 val_167 NULL NULL +195 val_195 NULL NULL +475 val_475 NULL NULL +17 val_17 17 val_17 +113 val_113 NULL NULL +155 val_155 NULL NULL +203 val_203 NULL NULL +339 val_339 NULL NULL +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +455 val_455 NULL NULL +128 val_128 NULL NULL +311 val_311 NULL NULL +316 val_316 NULL NULL +57 val_57 57 val_57 +302 val_302 NULL NULL +205 val_205 NULL NULL +149 val_149 NULL NULL +438 val_438 NULL NULL +345 val_345 NULL NULL +129 val_129 NULL NULL +170 val_170 NULL NULL +20 val_20 20 val_20 +489 val_489 NULL NULL +157 val_157 NULL NULL +378 val_378 NULL NULL +221 val_221 NULL NULL +92 val_92 92 val_92 +111 val_111 NULL NULL +47 val_47 47 val_47 +72 val_72 72 val_72 +72 val_72 72 val_72 +4 val_4 4 val_4 +280 val_280 NULL NULL +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +427 val_427 NULL NULL +277 val_277 NULL NULL +208 val_208 NULL NULL +356 val_356 NULL NULL +399 val_399 NULL NULL +169 val_169 NULL NULL +382 val_382 NULL NULL +498 val_498 NULL NULL +125 val_125 NULL NULL +386 val_386 NULL NULL +437 val_437 NULL NULL +469 val_469 NULL NULL +192 val_192 NULL NULL +286 val_286 NULL NULL +187 val_187 NULL NULL +176 val_176 NULL NULL +54 val_54 54 val_54 +459 val_459 NULL NULL +51 val_51 51 val_51 +51 val_51 51 val_51 +138 val_138 NULL NULL +103 val_103 NULL NULL +239 val_239 NULL NULL +213 val_213 NULL NULL +216 val_216 NULL NULL +430 val_430 NULL NULL +278 val_278 NULL NULL +176 val_176 NULL NULL +289 val_289 NULL NULL +221 val_221 NULL NULL +65 val_65 65 val_65 +318 val_318 NULL NULL +332 val_332 NULL NULL +311 val_311 NULL NULL +275 val_275 NULL NULL +137 val_137 NULL NULL +241 val_241 NULL NULL +83 val_83 83 val_83 +83 val_83 83 val_83 +333 val_333 NULL NULL +180 val_180 NULL NULL +284 val_284 NULL NULL +12 val_12 12 val_12 +12 val_12 12 val_12 +230 val_230 NULL NULL +181 val_181 NULL NULL +67 val_67 67 val_67 +67 val_67 67 val_67 +260 val_260 NULL NULL +404 val_404 NULL NULL +384 val_384 NULL NULL +489 val_489 NULL NULL +353 val_353 NULL NULL +373 val_373 NULL NULL +272 val_272 NULL NULL +138 val_138 NULL NULL +217 val_217 NULL NULL +84 val_84 84 val_84 +84 val_84 84 val_84 +348 val_348 NULL NULL +466 val_466 NULL NULL +58 val_58 58 val_58 +58 val_58 58 val_58 +8 val_8 8 val_8 +411 val_411 NULL NULL +230 val_230 NULL NULL +208 val_208 NULL NULL +348 val_348 NULL NULL +24 val_24 24 val_24 +24 val_24 24 val_24 +463 val_463 NULL NULL +431 val_431 NULL NULL +179 val_179 NULL NULL +172 val_172 NULL NULL +42 val_42 42 val_42 +42 val_42 42 val_42 +129 val_129 NULL NULL +158 val_158 NULL NULL +119 val_119 NULL NULL +496 val_496 NULL NULL +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +322 val_322 NULL NULL +197 val_197 NULL NULL +468 val_468 NULL NULL +393 val_393 NULL NULL +454 val_454 NULL NULL +100 val_100 100 val_100 +100 val_100 100 val_100 +298 val_298 NULL NULL +199 val_199 NULL NULL +191 val_191 NULL NULL +418 val_418 NULL NULL +96 val_96 96 val_96 +26 val_26 26 val_26 +26 val_26 26 val_26 +165 val_165 NULL NULL +327 val_327 NULL NULL +230 val_230 NULL NULL +205 val_205 NULL NULL +120 val_120 NULL NULL +131 val_131 NULL NULL +51 val_51 51 val_51 +51 val_51 51 val_51 +404 val_404 NULL NULL +43 val_43 43 val_43 +436 val_436 NULL NULL +156 val_156 NULL NULL +469 val_469 NULL NULL +468 val_468 NULL NULL +308 val_308 NULL NULL +95 val_95 95 val_95 +95 val_95 95 val_95 +196 val_196 NULL NULL +288 val_288 NULL NULL +481 val_481 NULL NULL +457 val_457 NULL NULL +98 val_98 98 val_98 +98 val_98 98 val_98 +282 val_282 NULL NULL +197 val_197 NULL NULL +187 val_187 NULL NULL +318 val_318 NULL NULL +318 val_318 NULL NULL +409 val_409 NULL NULL +470 val_470 NULL NULL +137 val_137 NULL NULL +369 val_369 NULL NULL +316 val_316 NULL NULL +169 val_169 NULL NULL +413 val_413 NULL NULL +85 val_85 85 val_85 +77 val_77 77 val_77 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +490 val_490 NULL NULL +87 val_87 87 val_87 +364 val_364 NULL NULL +179 val_179 NULL NULL +118 val_118 NULL NULL +134 val_134 NULL NULL +395 val_395 NULL NULL +282 val_282 NULL NULL +138 val_138 NULL NULL +238 val_238 NULL NULL +419 val_419 NULL NULL +15 val_15 15 val_15 +15 val_15 15 val_15 +118 val_118 NULL NULL +72 val_72 72 val_72 +72 val_72 72 val_72 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +307 val_307 NULL NULL +19 val_19 19 val_19 +435 val_435 NULL NULL +10 val_10 10 val_10 +277 val_277 NULL NULL +273 val_273 NULL NULL +306 val_306 NULL NULL +224 val_224 NULL NULL +309 val_309 NULL NULL +389 val_389 NULL NULL +327 val_327 NULL NULL +242 val_242 NULL NULL +369 val_369 NULL NULL +392 val_392 NULL NULL +272 val_272 NULL NULL +331 val_331 NULL NULL +401 val_401 NULL NULL +242 val_242 NULL NULL +452 val_452 NULL NULL +177 val_177 NULL NULL +226 val_226 NULL NULL +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +497 val_497 NULL NULL +402 val_402 NULL NULL +396 val_396 NULL NULL +317 val_317 NULL NULL +395 val_395 NULL NULL +58 val_58 58 val_58 +58 val_58 58 val_58 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +336 val_336 NULL NULL +95 val_95 95 val_95 +95 val_95 95 val_95 +11 val_11 11 val_11 +168 val_168 NULL NULL +34 val_34 34 val_34 +229 val_229 NULL NULL +233 val_233 NULL NULL +143 val_143 NULL NULL +472 val_472 NULL NULL +322 val_322 NULL NULL +498 val_498 NULL NULL +160 val_160 NULL NULL +195 val_195 NULL NULL +42 val_42 42 val_42 +42 val_42 42 val_42 +321 val_321 NULL NULL +430 val_430 NULL NULL +119 val_119 NULL NULL +489 val_489 NULL NULL +458 val_458 NULL NULL +78 val_78 78 val_78 +76 val_76 76 val_76 +76 val_76 76 val_76 +41 val_41 41 val_41 +223 val_223 NULL NULL +492 val_492 NULL NULL +149 val_149 NULL NULL +449 val_449 NULL NULL +218 val_218 NULL NULL +228 val_228 NULL NULL +138 val_138 NULL NULL +453 val_453 NULL NULL +30 val_30 30 val_30 +209 val_209 NULL NULL +64 val_64 64 val_64 +468 val_468 NULL NULL +76 val_76 76 val_76 +76 val_76 76 val_76 +74 val_74 74 val_74 +342 val_342 NULL NULL +69 val_69 69 val_69 +230 val_230 NULL NULL +33 val_33 33 val_33 +368 val_368 NULL NULL +103 val_103 NULL NULL +296 val_296 NULL NULL +113 val_113 NULL NULL +216 val_216 NULL NULL +367 val_367 NULL NULL +344 val_344 NULL NULL +167 val_167 NULL NULL +274 val_274 NULL NULL +219 val_219 NULL NULL +239 val_239 NULL NULL +485 val_485 NULL NULL +116 val_116 NULL NULL +223 val_223 NULL NULL +256 val_256 NULL NULL +263 val_263 NULL NULL +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +487 val_487 NULL NULL +480 val_480 NULL NULL +401 val_401 NULL NULL +288 val_288 NULL NULL +191 val_191 NULL NULL +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +244 val_244 NULL NULL +438 val_438 NULL NULL +128 val_128 NULL NULL +467 val_467 NULL NULL +432 val_432 NULL NULL +202 val_202 NULL NULL +316 val_316 NULL NULL +229 val_229 NULL NULL +469 val_469 NULL NULL +463 val_463 NULL NULL +280 val_280 NULL NULL +2 val_2 2 val_2 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +283 val_283 NULL NULL +331 val_331 NULL NULL +235 val_235 NULL NULL +80 val_80 80 val_80 +44 val_44 44 val_44 +193 val_193 NULL NULL +321 val_321 NULL NULL +335 val_335 NULL NULL +104 val_104 NULL NULL +466 val_466 NULL NULL +366 val_366 NULL NULL +175 val_175 NULL NULL +403 val_403 NULL NULL +483 val_483 NULL NULL +53 val_53 53 val_53 +105 val_105 NULL NULL +257 val_257 NULL NULL +406 val_406 NULL NULL +409 val_409 NULL NULL +190 val_190 NULL NULL +406 val_406 NULL NULL +401 val_401 NULL NULL +114 val_114 NULL NULL +258 val_258 NULL NULL +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +203 val_203 NULL NULL +262 val_262 NULL NULL +348 val_348 NULL NULL +424 val_424 NULL NULL +12 val_12 12 val_12 +12 val_12 12 val_12 +396 val_396 NULL NULL +201 val_201 NULL NULL +217 val_217 NULL NULL +164 val_164 NULL NULL +431 val_431 NULL NULL +454 val_454 NULL NULL +478 val_478 NULL NULL +298 val_298 NULL NULL +125 val_125 NULL NULL +431 val_431 NULL NULL +164 val_164 NULL NULL +424 val_424 NULL NULL +187 val_187 NULL NULL +382 val_382 NULL NULL +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +397 val_397 NULL NULL +480 val_480 NULL NULL +291 val_291 NULL NULL +24 val_24 24 val_24 +24 val_24 24 val_24 +351 val_351 NULL NULL +255 val_255 NULL NULL +104 val_104 NULL NULL +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +163 val_163 NULL NULL +438 val_438 NULL NULL +119 val_119 NULL NULL +414 val_414 NULL NULL +200 val_200 NULL NULL +491 val_491 NULL NULL +237 val_237 NULL NULL +439 val_439 NULL NULL +360 val_360 NULL NULL +248 val_248 NULL NULL +479 val_479 NULL NULL +305 val_305 NULL NULL +417 val_417 NULL NULL +199 val_199 NULL NULL +444 val_444 NULL NULL +120 val_120 NULL NULL +429 val_429 NULL NULL +169 val_169 NULL NULL +443 val_443 NULL NULL +323 val_323 NULL NULL +325 val_325 NULL NULL +277 val_277 NULL NULL +230 val_230 NULL NULL +478 val_478 NULL NULL +178 val_178 NULL NULL +468 val_468 NULL NULL +310 val_310 NULL NULL +317 val_317 NULL NULL +333 val_333 NULL NULL +493 val_493 NULL NULL +460 val_460 NULL NULL +207 val_207 NULL NULL +249 val_249 NULL NULL +265 val_265 NULL NULL +480 val_480 NULL NULL +83 val_83 83 val_83 +83 val_83 83 val_83 +136 val_136 NULL NULL +353 val_353 NULL NULL +172 val_172 NULL NULL +214 val_214 NULL NULL +462 val_462 NULL NULL +233 val_233 NULL NULL +406 val_406 NULL NULL +133 val_133 NULL NULL +175 val_175 NULL NULL +189 val_189 NULL NULL +454 val_454 NULL NULL +375 val_375 NULL NULL +401 val_401 NULL NULL +421 val_421 NULL NULL +407 val_407 NULL NULL +384 val_384 NULL NULL +256 val_256 NULL NULL +26 val_26 26 val_26 +26 val_26 26 val_26 +134 val_134 NULL NULL +67 val_67 67 val_67 +67 val_67 67 val_67 +384 val_384 NULL NULL +379 val_379 NULL NULL +18 val_18 18 val_18 +18 val_18 18 val_18 +462 val_462 NULL NULL +492 val_492 NULL NULL +100 val_100 100 val_100 +100 val_100 100 val_100 +298 val_298 NULL NULL +9 val_9 9 val_9 +341 val_341 NULL NULL +498 val_498 NULL NULL +146 val_146 NULL NULL +458 val_458 NULL NULL +362 val_362 NULL NULL +186 val_186 NULL NULL +285 val_285 NULL NULL +348 val_348 NULL NULL +167 val_167 NULL NULL +18 val_18 18 val_18 +18 val_18 18 val_18 +273 val_273 NULL NULL +183 val_183 NULL NULL +281 val_281 NULL NULL +344 val_344 NULL NULL +97 val_97 97 val_97 +97 val_97 97 val_97 +469 val_469 NULL NULL +315 val_315 NULL NULL +84 val_84 84 val_84 +84 val_84 84 val_84 +28 val_28 28 val_28 +37 val_37 37 val_37 +37 val_37 37 val_37 +448 val_448 NULL NULL +152 val_152 NULL NULL +348 val_348 NULL NULL +307 val_307 NULL NULL +194 val_194 NULL NULL +414 val_414 NULL NULL +477 val_477 NULL NULL +222 val_222 NULL NULL +126 val_126 NULL NULL +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +169 val_169 NULL NULL +403 val_403 NULL NULL +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 diff --git a/src/test/hive/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 b/src/test/hive/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join40-12-aaddbef9069aba3ebeb310be74671cda b/src/test/hive/golden/join40-12-aaddbef9069aba3ebeb310be74671cda new file mode 100644 index 0000000000000..67d892c80f493 --- /dev/null +++ b/src/test/hive/golden/join40-12-aaddbef9069aba3ebeb310be74671cda @@ -0,0 +1 @@ +1028 \ No newline at end of file diff --git a/src/test/hive/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 b/src/test/hive/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 new file mode 100644 index 0000000000000..8543fe173f87f --- /dev/null +++ b/src/test/hive/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 @@ -0,0 +1,566 @@ +238 val_238 NULL NULL +86 val_86 86 val_86 +311 val_311 NULL NULL +27 val_27 27 val_27 +165 val_165 NULL NULL +409 val_409 NULL NULL +255 val_255 NULL NULL +278 val_278 NULL NULL +98 val_98 98 val_98 +98 val_98 98 val_98 +484 val_484 NULL NULL +265 val_265 NULL NULL +193 val_193 NULL NULL +401 val_401 NULL NULL +150 val_150 NULL NULL +273 val_273 NULL NULL +224 val_224 NULL NULL +369 val_369 NULL NULL +66 val_66 66 val_66 +128 val_128 NULL NULL +213 val_213 NULL NULL +146 val_146 NULL NULL +406 val_406 NULL NULL +429 val_429 NULL NULL +374 val_374 NULL NULL +152 val_152 NULL NULL +469 val_469 NULL NULL +145 val_145 NULL NULL +495 val_495 NULL NULL +37 val_37 37 val_37 +37 val_37 37 val_37 +327 val_327 NULL NULL +281 val_281 NULL NULL +277 val_277 NULL NULL +209 val_209 NULL NULL +15 val_15 15 val_15 +15 val_15 15 val_15 +82 val_82 82 val_82 +403 val_403 NULL NULL +166 val_166 NULL NULL +417 val_417 NULL NULL +430 val_430 NULL NULL +252 val_252 NULL NULL +292 val_292 NULL NULL +219 val_219 NULL NULL +287 val_287 NULL NULL +153 val_153 NULL NULL +193 val_193 NULL NULL +338 val_338 NULL NULL +446 val_446 NULL NULL +459 val_459 NULL NULL +394 val_394 NULL NULL +237 val_237 NULL NULL +482 val_482 NULL NULL +174 val_174 NULL NULL +413 val_413 NULL NULL +494 val_494 NULL NULL +207 val_207 NULL NULL +199 val_199 NULL NULL +466 val_466 NULL NULL +208 val_208 NULL NULL +174 val_174 NULL NULL +399 val_399 NULL NULL +396 val_396 NULL NULL +247 val_247 NULL NULL +417 val_417 NULL NULL +489 val_489 NULL NULL +162 val_162 NULL NULL +377 val_377 NULL NULL +397 val_397 NULL NULL +309 val_309 NULL NULL +365 val_365 NULL NULL +266 val_266 NULL NULL +439 val_439 NULL NULL +342 val_342 NULL NULL +367 val_367 NULL NULL +325 val_325 NULL NULL +167 val_167 NULL NULL +195 val_195 NULL NULL +475 val_475 NULL NULL +17 val_17 17 val_17 +113 val_113 NULL NULL +155 val_155 NULL NULL +203 val_203 NULL NULL +339 val_339 NULL NULL +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +455 val_455 NULL NULL +128 val_128 NULL NULL +311 val_311 NULL NULL +316 val_316 NULL NULL +57 val_57 57 val_57 +302 val_302 NULL NULL +205 val_205 NULL NULL +149 val_149 NULL NULL +438 val_438 NULL NULL +345 val_345 NULL NULL +129 val_129 NULL NULL +170 val_170 NULL NULL +20 val_20 20 val_20 +489 val_489 NULL NULL +157 val_157 NULL NULL +378 val_378 NULL NULL +221 val_221 NULL NULL +92 val_92 92 val_92 +111 val_111 NULL NULL +47 val_47 47 val_47 +72 val_72 72 val_72 +72 val_72 72 val_72 +4 val_4 4 val_4 +280 val_280 NULL NULL +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +427 val_427 NULL NULL +277 val_277 NULL NULL +208 val_208 NULL NULL +356 val_356 NULL NULL +399 val_399 NULL NULL +169 val_169 NULL NULL +382 val_382 NULL NULL +498 val_498 NULL NULL +125 val_125 NULL NULL +386 val_386 NULL NULL +437 val_437 NULL NULL +469 val_469 NULL NULL +192 val_192 NULL NULL +286 val_286 NULL NULL +187 val_187 NULL NULL +176 val_176 NULL NULL +54 val_54 54 val_54 +459 val_459 NULL NULL +51 val_51 51 val_51 +51 val_51 51 val_51 +138 val_138 NULL NULL +103 val_103 NULL NULL +239 val_239 NULL NULL +213 val_213 NULL NULL +216 val_216 NULL NULL +430 val_430 NULL NULL +278 val_278 NULL NULL +176 val_176 NULL NULL +289 val_289 NULL NULL +221 val_221 NULL NULL +65 val_65 65 val_65 +318 val_318 NULL NULL +332 val_332 NULL NULL +311 val_311 NULL NULL +275 val_275 NULL NULL +137 val_137 NULL NULL +241 val_241 NULL NULL +83 val_83 83 val_83 +83 val_83 83 val_83 +333 val_333 NULL NULL +180 val_180 NULL NULL +284 val_284 NULL NULL +12 val_12 12 val_12 +12 val_12 12 val_12 +230 val_230 NULL NULL +181 val_181 NULL NULL +67 val_67 67 val_67 +67 val_67 67 val_67 +260 val_260 NULL NULL +404 val_404 NULL NULL +384 val_384 NULL NULL +489 val_489 NULL NULL +353 val_353 NULL NULL +373 val_373 NULL NULL +272 val_272 NULL NULL +138 val_138 NULL NULL +217 val_217 NULL NULL +84 val_84 84 val_84 +84 val_84 84 val_84 +348 val_348 NULL NULL +466 val_466 NULL NULL +58 val_58 58 val_58 +58 val_58 58 val_58 +8 val_8 8 val_8 +411 val_411 NULL NULL +230 val_230 NULL NULL +208 val_208 NULL NULL +348 val_348 NULL NULL +24 val_24 24 val_24 +24 val_24 24 val_24 +463 val_463 NULL NULL +431 val_431 NULL NULL +179 val_179 NULL NULL +172 val_172 NULL NULL +42 val_42 42 val_42 +42 val_42 42 val_42 +129 val_129 NULL NULL +158 val_158 NULL NULL +119 val_119 NULL NULL +496 val_496 NULL NULL +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +322 val_322 NULL NULL +197 val_197 NULL NULL +468 val_468 NULL NULL +393 val_393 NULL NULL +454 val_454 NULL NULL +100 val_100 100 val_100 +100 val_100 100 val_100 +298 val_298 NULL NULL +199 val_199 NULL NULL +191 val_191 NULL NULL +418 val_418 NULL NULL +96 val_96 96 val_96 +26 val_26 26 val_26 +26 val_26 26 val_26 +165 val_165 NULL NULL +327 val_327 NULL NULL +230 val_230 NULL NULL +205 val_205 NULL NULL +120 val_120 NULL NULL +131 val_131 NULL NULL +51 val_51 51 val_51 +51 val_51 51 val_51 +404 val_404 NULL NULL +43 val_43 43 val_43 +436 val_436 NULL NULL +156 val_156 NULL NULL +469 val_469 NULL NULL +468 val_468 NULL NULL +308 val_308 NULL NULL +95 val_95 95 val_95 +95 val_95 95 val_95 +196 val_196 NULL NULL +288 val_288 NULL NULL +481 val_481 NULL NULL +457 val_457 NULL NULL +98 val_98 98 val_98 +98 val_98 98 val_98 +282 val_282 NULL NULL +197 val_197 NULL NULL +187 val_187 NULL NULL +318 val_318 NULL NULL +318 val_318 NULL NULL +409 val_409 NULL NULL +470 val_470 NULL NULL +137 val_137 NULL NULL +369 val_369 NULL NULL +316 val_316 NULL NULL +169 val_169 NULL NULL +413 val_413 NULL NULL +85 val_85 85 val_85 +77 val_77 77 val_77 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +490 val_490 NULL NULL +87 val_87 87 val_87 +364 val_364 NULL NULL +179 val_179 NULL NULL +118 val_118 NULL NULL +134 val_134 NULL NULL +395 val_395 NULL NULL +282 val_282 NULL NULL +138 val_138 NULL NULL +238 val_238 NULL NULL +419 val_419 NULL NULL +15 val_15 15 val_15 +15 val_15 15 val_15 +118 val_118 NULL NULL +72 val_72 72 val_72 +72 val_72 72 val_72 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +307 val_307 NULL NULL +19 val_19 19 val_19 +435 val_435 NULL NULL +10 val_10 10 val_10 +277 val_277 NULL NULL +273 val_273 NULL NULL +306 val_306 NULL NULL +224 val_224 NULL NULL +309 val_309 NULL NULL +389 val_389 NULL NULL +327 val_327 NULL NULL +242 val_242 NULL NULL +369 val_369 NULL NULL +392 val_392 NULL NULL +272 val_272 NULL NULL +331 val_331 NULL NULL +401 val_401 NULL NULL +242 val_242 NULL NULL +452 val_452 NULL NULL +177 val_177 NULL NULL +226 val_226 NULL NULL +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +497 val_497 NULL NULL +402 val_402 NULL NULL +396 val_396 NULL NULL +317 val_317 NULL NULL +395 val_395 NULL NULL +58 val_58 58 val_58 +58 val_58 58 val_58 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +336 val_336 NULL NULL +95 val_95 95 val_95 +95 val_95 95 val_95 +11 val_11 11 val_11 +168 val_168 NULL NULL +34 val_34 34 val_34 +229 val_229 NULL NULL +233 val_233 NULL NULL +143 val_143 NULL NULL +472 val_472 NULL NULL +322 val_322 NULL NULL +498 val_498 NULL NULL +160 val_160 NULL NULL +195 val_195 NULL NULL +42 val_42 42 val_42 +42 val_42 42 val_42 +321 val_321 NULL NULL +430 val_430 NULL NULL +119 val_119 NULL NULL +489 val_489 NULL NULL +458 val_458 NULL NULL +78 val_78 78 val_78 +76 val_76 76 val_76 +76 val_76 76 val_76 +41 val_41 41 val_41 +223 val_223 NULL NULL +492 val_492 NULL NULL +149 val_149 NULL NULL +449 val_449 NULL NULL +218 val_218 NULL NULL +228 val_228 NULL NULL +138 val_138 NULL NULL +453 val_453 NULL NULL +30 val_30 30 val_30 +209 val_209 NULL NULL +64 val_64 64 val_64 +468 val_468 NULL NULL +76 val_76 76 val_76 +76 val_76 76 val_76 +74 val_74 74 val_74 +342 val_342 NULL NULL +69 val_69 69 val_69 +230 val_230 NULL NULL +33 val_33 33 val_33 +368 val_368 NULL NULL +103 val_103 NULL NULL +296 val_296 NULL NULL +113 val_113 NULL NULL +216 val_216 NULL NULL +367 val_367 NULL NULL +344 val_344 NULL NULL +167 val_167 NULL NULL +274 val_274 NULL NULL +219 val_219 NULL NULL +239 val_239 NULL NULL +485 val_485 NULL NULL +116 val_116 NULL NULL +223 val_223 NULL NULL +256 val_256 NULL NULL +263 val_263 NULL NULL +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +487 val_487 NULL NULL +480 val_480 NULL NULL +401 val_401 NULL NULL +288 val_288 NULL NULL +191 val_191 NULL NULL +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +244 val_244 NULL NULL +438 val_438 NULL NULL +128 val_128 NULL NULL +467 val_467 NULL NULL +432 val_432 NULL NULL +202 val_202 NULL NULL +316 val_316 NULL NULL +229 val_229 NULL NULL +469 val_469 NULL NULL +463 val_463 NULL NULL +280 val_280 NULL NULL +2 val_2 2 val_2 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +283 val_283 NULL NULL +331 val_331 NULL NULL +235 val_235 NULL NULL +80 val_80 80 val_80 +44 val_44 44 val_44 +193 val_193 NULL NULL +321 val_321 NULL NULL +335 val_335 NULL NULL +104 val_104 NULL NULL +466 val_466 NULL NULL +366 val_366 NULL NULL +175 val_175 NULL NULL +403 val_403 NULL NULL +483 val_483 NULL NULL +53 val_53 53 val_53 +105 val_105 NULL NULL +257 val_257 NULL NULL +406 val_406 NULL NULL +409 val_409 NULL NULL +190 val_190 NULL NULL +406 val_406 NULL NULL +401 val_401 NULL NULL +114 val_114 NULL NULL +258 val_258 NULL NULL +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +203 val_203 NULL NULL +262 val_262 NULL NULL +348 val_348 NULL NULL +424 val_424 NULL NULL +12 val_12 12 val_12 +12 val_12 12 val_12 +396 val_396 NULL NULL +201 val_201 NULL NULL +217 val_217 NULL NULL +164 val_164 NULL NULL +431 val_431 NULL NULL +454 val_454 NULL NULL +478 val_478 NULL NULL +298 val_298 NULL NULL +125 val_125 NULL NULL +431 val_431 NULL NULL +164 val_164 NULL NULL +424 val_424 NULL NULL +187 val_187 NULL NULL +382 val_382 NULL NULL +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +397 val_397 NULL NULL +480 val_480 NULL NULL +291 val_291 NULL NULL +24 val_24 24 val_24 +24 val_24 24 val_24 +351 val_351 NULL NULL +255 val_255 NULL NULL +104 val_104 NULL NULL +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +163 val_163 NULL NULL +438 val_438 NULL NULL +119 val_119 NULL NULL +414 val_414 NULL NULL +200 val_200 NULL NULL +491 val_491 NULL NULL +237 val_237 NULL NULL +439 val_439 NULL NULL +360 val_360 NULL NULL +248 val_248 NULL NULL +479 val_479 NULL NULL +305 val_305 NULL NULL +417 val_417 NULL NULL +199 val_199 NULL NULL +444 val_444 NULL NULL +120 val_120 NULL NULL +429 val_429 NULL NULL +169 val_169 NULL NULL +443 val_443 NULL NULL +323 val_323 NULL NULL +325 val_325 NULL NULL +277 val_277 NULL NULL +230 val_230 NULL NULL +478 val_478 NULL NULL +178 val_178 NULL NULL +468 val_468 NULL NULL +310 val_310 NULL NULL +317 val_317 NULL NULL +333 val_333 NULL NULL +493 val_493 NULL NULL +460 val_460 NULL NULL +207 val_207 NULL NULL +249 val_249 NULL NULL +265 val_265 NULL NULL +480 val_480 NULL NULL +83 val_83 83 val_83 +83 val_83 83 val_83 +136 val_136 NULL NULL +353 val_353 NULL NULL +172 val_172 NULL NULL +214 val_214 NULL NULL +462 val_462 NULL NULL +233 val_233 NULL NULL +406 val_406 NULL NULL +133 val_133 NULL NULL +175 val_175 NULL NULL +189 val_189 NULL NULL +454 val_454 NULL NULL +375 val_375 NULL NULL +401 val_401 NULL NULL +421 val_421 NULL NULL +407 val_407 NULL NULL +384 val_384 NULL NULL +256 val_256 NULL NULL +26 val_26 26 val_26 +26 val_26 26 val_26 +134 val_134 NULL NULL +67 val_67 67 val_67 +67 val_67 67 val_67 +384 val_384 NULL NULL +379 val_379 NULL NULL +18 val_18 18 val_18 +18 val_18 18 val_18 +462 val_462 NULL NULL +492 val_492 NULL NULL +100 val_100 100 val_100 +100 val_100 100 val_100 +298 val_298 NULL NULL +9 val_9 9 val_9 +341 val_341 NULL NULL +498 val_498 NULL NULL +146 val_146 NULL NULL +458 val_458 NULL NULL +362 val_362 NULL NULL +186 val_186 NULL NULL +285 val_285 NULL NULL +348 val_348 NULL NULL +167 val_167 NULL NULL +18 val_18 18 val_18 +18 val_18 18 val_18 +273 val_273 NULL NULL +183 val_183 NULL NULL +281 val_281 NULL NULL +344 val_344 NULL NULL +97 val_97 97 val_97 +97 val_97 97 val_97 +469 val_469 NULL NULL +315 val_315 NULL NULL +84 val_84 84 val_84 +84 val_84 84 val_84 +28 val_28 28 val_28 +37 val_37 37 val_37 +37 val_37 37 val_37 +448 val_448 NULL NULL +152 val_152 NULL NULL +348 val_348 NULL NULL +307 val_307 NULL NULL +194 val_194 NULL NULL +414 val_414 NULL NULL +477 val_477 NULL NULL +222 val_222 NULL NULL +126 val_126 NULL NULL +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +169 val_169 NULL NULL +403 val_403 NULL NULL +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 diff --git a/src/test/hive/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 b/src/test/hive/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 b/src/test/hive/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 new file mode 100644 index 0000000000000..59821aeea3008 --- /dev/null +++ b/src/test/hive/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 @@ -0,0 +1,1028 @@ +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +374 val_374 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +82 val_82 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +459 val_459 +394 val_394 +237 val_237 +237 val_237 +482 val_482 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +494 val_494 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +0 val_0 +0 val_0 +0 val_0 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +4 val_4 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +54 val_54 +459 val_459 +459 val_459 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +12 val_12 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +260 val_260 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +58 val_58 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +0 val_0 +0 val_0 +0 val_0 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +404 val_404 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +0 val_0 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +419 val_419 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +78 val_78 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +209 val_209 +64 val_64 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +12 val_12 +12 val_12 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +491 val_491 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +18 val_18 +18 val_18 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +448 val_448 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +414 val_414 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +200 val_200 +200 val_200 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e b/src/test/hive/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 b/src/test/hive/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 new file mode 100644 index 0000000000000..bd6e20b8de1e4 --- /dev/null +++ b/src/test/hive/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 @@ -0,0 +1,548 @@ +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 11 val_11 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 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 NULL NULL 26 val_26 +NULL NULL NULL NULL 26 val_26 +NULL NULL NULL NULL 27 val_27 +NULL NULL NULL NULL 28 val_28 +NULL NULL NULL NULL 30 val_30 +NULL NULL NULL NULL 33 val_33 +NULL NULL NULL NULL 34 val_34 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 41 val_41 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 43 val_43 +NULL NULL NULL NULL 44 val_44 +NULL NULL NULL NULL 47 val_47 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 53 val_53 +NULL NULL NULL NULL 54 val_54 +NULL NULL NULL NULL 57 val_57 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 64 val_64 +NULL NULL NULL NULL 65 val_65 +NULL NULL NULL NULL 66 val_66 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 69 val_69 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 74 val_74 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 77 val_77 +NULL NULL NULL NULL 78 val_78 +NULL NULL NULL NULL 80 val_80 +NULL NULL NULL NULL 82 val_82 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 85 val_85 +NULL NULL NULL NULL 86 val_86 +NULL NULL NULL NULL 87 val_87 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 92 val_92 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 96 val_96 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 105 val_105 +NULL NULL NULL NULL 111 val_111 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 114 val_114 +NULL NULL NULL NULL 116 val_116 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 126 val_126 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 131 val_131 +NULL NULL NULL NULL 133 val_133 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 136 val_136 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 143 val_143 +NULL NULL NULL NULL 145 val_145 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 150 val_150 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 153 val_153 +NULL NULL NULL NULL 155 val_155 +NULL NULL NULL NULL 156 val_156 +NULL NULL NULL NULL 157 val_157 +NULL NULL NULL NULL 158 val_158 +NULL NULL NULL NULL 160 val_160 +NULL NULL NULL NULL 162 val_162 +NULL NULL NULL NULL 163 val_163 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 166 val_166 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 168 val_168 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 170 val_170 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 177 val_177 +NULL NULL NULL NULL 178 val_178 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 180 val_180 +NULL NULL NULL NULL 181 val_181 +NULL NULL NULL NULL 183 val_183 +NULL NULL NULL NULL 186 val_186 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 189 val_189 +NULL NULL NULL NULL 190 val_190 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 192 val_192 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 194 val_194 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 196 val_196 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 201 val_201 +NULL NULL NULL NULL 202 val_202 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 214 val_214 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 218 val_218 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 222 val_222 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 226 val_226 +NULL NULL NULL NULL 228 val_228 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 235 val_235 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 241 val_241 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 244 val_244 +NULL NULL NULL NULL 247 val_247 +NULL NULL NULL NULL 248 val_248 +NULL NULL NULL NULL 249 val_249 +NULL NULL NULL NULL 252 val_252 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 257 val_257 +NULL NULL NULL NULL 258 val_258 +NULL NULL NULL NULL 260 val_260 +NULL NULL NULL NULL 262 val_262 +NULL NULL NULL NULL 263 val_263 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 266 val_266 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 274 val_274 +NULL NULL NULL NULL 275 val_275 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 283 val_283 +NULL NULL NULL NULL 284 val_284 +NULL NULL NULL NULL 285 val_285 +NULL NULL NULL NULL 286 val_286 +NULL NULL NULL NULL 287 val_287 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 289 val_289 +NULL NULL NULL NULL 291 val_291 +NULL NULL NULL NULL 292 val_292 +NULL NULL NULL NULL 296 val_296 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 302 val_302 +NULL NULL NULL NULL 305 val_305 +NULL NULL NULL NULL 306 val_306 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 308 val_308 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 310 val_310 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 315 val_315 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 323 val_323 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 332 val_332 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 335 val_335 +NULL NULL NULL NULL 336 val_336 +NULL NULL NULL NULL 338 val_338 +NULL NULL NULL NULL 339 val_339 +NULL NULL NULL NULL 341 val_341 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 345 val_345 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 351 val_351 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 356 val_356 +NULL NULL NULL NULL 360 val_360 +NULL NULL NULL NULL 362 val_362 +NULL NULL NULL NULL 364 val_364 +NULL NULL NULL NULL 365 val_365 +NULL NULL NULL NULL 366 val_366 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 368 val_368 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 373 val_373 +NULL NULL NULL NULL 374 val_374 +NULL NULL NULL NULL 375 val_375 +NULL NULL NULL NULL 377 val_377 +NULL NULL NULL NULL 378 val_378 +NULL NULL NULL NULL 379 val_379 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 386 val_386 +NULL NULL NULL NULL 389 val_389 +NULL NULL NULL NULL 392 val_392 +NULL NULL NULL NULL 393 val_393 +NULL NULL NULL NULL 394 val_394 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 400 val_400 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 402 val_402 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 407 val_407 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 411 val_411 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 418 val_418 +NULL NULL NULL NULL 419 val_419 +NULL NULL NULL NULL 421 val_421 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 427 val_427 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 432 val_432 +NULL NULL NULL NULL 435 val_435 +NULL NULL NULL NULL 436 val_436 +NULL NULL NULL NULL 437 val_437 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 443 val_443 +NULL NULL NULL NULL 444 val_444 +NULL NULL NULL NULL 446 val_446 +NULL NULL NULL NULL 448 val_448 +NULL NULL NULL NULL 449 val_449 +NULL NULL NULL NULL 452 val_452 +NULL NULL NULL NULL 453 val_453 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 455 val_455 +NULL NULL NULL NULL 457 val_457 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 460 val_460 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 467 val_467 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 470 val_470 +NULL NULL NULL NULL 472 val_472 +NULL NULL NULL NULL 475 val_475 +NULL NULL NULL NULL 477 val_477 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 479 val_479 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 481 val_481 +NULL NULL NULL NULL 482 val_482 +NULL NULL NULL NULL 483 val_483 +NULL NULL NULL NULL 484 val_484 +NULL NULL NULL NULL 485 val_485 +NULL NULL NULL NULL 487 val_487 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 490 val_490 +NULL NULL NULL NULL 491 val_491 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 493 val_493 +NULL NULL NULL NULL 494 val_494 +NULL NULL NULL NULL 495 val_495 +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 +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 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 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 +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 0 val_0 0 val_0 +0 val_0 0 val_0 0 val_0 +2 val_2 2 val_2 2 val_2 +4 val_4 4 val_4 4 val_4 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +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 diff --git a/src/test/hive/golden/join40-7-222655137ca9242f063c7717acbe4c65 b/src/test/hive/golden/join40-7-222655137ca9242f063c7717acbe4c65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 b/src/test/hive/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 new file mode 100644 index 0000000000000..bd6e20b8de1e4 --- /dev/null +++ b/src/test/hive/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 @@ -0,0 +1,548 @@ +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 11 val_11 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 12 val_12 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 15 val_15 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 18 val_18 +NULL NULL NULL NULL 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 NULL NULL 26 val_26 +NULL NULL NULL NULL 26 val_26 +NULL NULL NULL NULL 27 val_27 +NULL NULL NULL NULL 28 val_28 +NULL NULL NULL NULL 30 val_30 +NULL NULL NULL NULL 33 val_33 +NULL NULL NULL NULL 34 val_34 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 35 val_35 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 37 val_37 +NULL NULL NULL NULL 41 val_41 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 42 val_42 +NULL NULL NULL NULL 43 val_43 +NULL NULL NULL NULL 44 val_44 +NULL NULL NULL NULL 47 val_47 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 51 val_51 +NULL NULL NULL NULL 53 val_53 +NULL NULL NULL NULL 54 val_54 +NULL NULL NULL NULL 57 val_57 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 58 val_58 +NULL NULL NULL NULL 64 val_64 +NULL NULL NULL NULL 65 val_65 +NULL NULL NULL NULL 66 val_66 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 67 val_67 +NULL NULL NULL NULL 69 val_69 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 70 val_70 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 72 val_72 +NULL NULL NULL NULL 74 val_74 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 76 val_76 +NULL NULL NULL NULL 77 val_77 +NULL NULL NULL NULL 78 val_78 +NULL NULL NULL NULL 80 val_80 +NULL NULL NULL NULL 82 val_82 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 83 val_83 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 84 val_84 +NULL NULL NULL NULL 85 val_85 +NULL NULL NULL NULL 86 val_86 +NULL NULL NULL NULL 87 val_87 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 90 val_90 +NULL NULL NULL NULL 92 val_92 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 95 val_95 +NULL NULL NULL NULL 96 val_96 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 97 val_97 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 98 val_98 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 100 val_100 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 103 val_103 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 104 val_104 +NULL NULL NULL NULL 105 val_105 +NULL NULL NULL NULL 111 val_111 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 113 val_113 +NULL NULL NULL NULL 114 val_114 +NULL NULL NULL NULL 116 val_116 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 118 val_118 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 119 val_119 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 120 val_120 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 125 val_125 +NULL NULL NULL NULL 126 val_126 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 128 val_128 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 129 val_129 +NULL NULL NULL NULL 131 val_131 +NULL NULL NULL NULL 133 val_133 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 134 val_134 +NULL NULL NULL NULL 136 val_136 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 137 val_137 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 138 val_138 +NULL NULL NULL NULL 143 val_143 +NULL NULL NULL NULL 145 val_145 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 146 val_146 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 149 val_149 +NULL NULL NULL NULL 150 val_150 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 152 val_152 +NULL NULL NULL NULL 153 val_153 +NULL NULL NULL NULL 155 val_155 +NULL NULL NULL NULL 156 val_156 +NULL NULL NULL NULL 157 val_157 +NULL NULL NULL NULL 158 val_158 +NULL NULL NULL NULL 160 val_160 +NULL NULL NULL NULL 162 val_162 +NULL NULL NULL NULL 163 val_163 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 164 val_164 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 165 val_165 +NULL NULL NULL NULL 166 val_166 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 167 val_167 +NULL NULL NULL NULL 168 val_168 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 169 val_169 +NULL NULL NULL NULL 170 val_170 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 172 val_172 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 174 val_174 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 175 val_175 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 176 val_176 +NULL NULL NULL NULL 177 val_177 +NULL NULL NULL NULL 178 val_178 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 179 val_179 +NULL NULL NULL NULL 180 val_180 +NULL NULL NULL NULL 181 val_181 +NULL NULL NULL NULL 183 val_183 +NULL NULL NULL NULL 186 val_186 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 187 val_187 +NULL NULL NULL NULL 189 val_189 +NULL NULL NULL NULL 190 val_190 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 191 val_191 +NULL NULL NULL NULL 192 val_192 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 193 val_193 +NULL NULL NULL NULL 194 val_194 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 195 val_195 +NULL NULL NULL NULL 196 val_196 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 197 val_197 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 199 val_199 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 200 val_200 +NULL NULL NULL NULL 201 val_201 +NULL NULL NULL NULL 202 val_202 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 203 val_203 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 205 val_205 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 207 val_207 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 208 val_208 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 209 val_209 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 213 val_213 +NULL NULL NULL NULL 214 val_214 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 216 val_216 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 217 val_217 +NULL NULL NULL NULL 218 val_218 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 219 val_219 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 221 val_221 +NULL NULL NULL NULL 222 val_222 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 223 val_223 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 224 val_224 +NULL NULL NULL NULL 226 val_226 +NULL NULL NULL NULL 228 val_228 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 229 val_229 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 230 val_230 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 233 val_233 +NULL NULL NULL NULL 235 val_235 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 237 val_237 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 238 val_238 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 239 val_239 +NULL NULL NULL NULL 241 val_241 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 242 val_242 +NULL NULL NULL NULL 244 val_244 +NULL NULL NULL NULL 247 val_247 +NULL NULL NULL NULL 248 val_248 +NULL NULL NULL NULL 249 val_249 +NULL NULL NULL NULL 252 val_252 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 255 val_255 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 256 val_256 +NULL NULL NULL NULL 257 val_257 +NULL NULL NULL NULL 258 val_258 +NULL NULL NULL NULL 260 val_260 +NULL NULL NULL NULL 262 val_262 +NULL NULL NULL NULL 263 val_263 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 265 val_265 +NULL NULL NULL NULL 266 val_266 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 272 val_272 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 273 val_273 +NULL NULL NULL NULL 274 val_274 +NULL NULL NULL NULL 275 val_275 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 277 val_277 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 278 val_278 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 280 val_280 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 281 val_281 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 282 val_282 +NULL NULL NULL NULL 283 val_283 +NULL NULL NULL NULL 284 val_284 +NULL NULL NULL NULL 285 val_285 +NULL NULL NULL NULL 286 val_286 +NULL NULL NULL NULL 287 val_287 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 288 val_288 +NULL NULL NULL NULL 289 val_289 +NULL NULL NULL NULL 291 val_291 +NULL NULL NULL NULL 292 val_292 +NULL NULL NULL NULL 296 val_296 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 298 val_298 +NULL NULL NULL NULL 302 val_302 +NULL NULL NULL NULL 305 val_305 +NULL NULL NULL NULL 306 val_306 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 307 val_307 +NULL NULL NULL NULL 308 val_308 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 309 val_309 +NULL NULL NULL NULL 310 val_310 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 311 val_311 +NULL NULL NULL NULL 315 val_315 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 316 val_316 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 317 val_317 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 318 val_318 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 321 val_321 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 322 val_322 +NULL NULL NULL NULL 323 val_323 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 325 val_325 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 327 val_327 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 331 val_331 +NULL NULL NULL NULL 332 val_332 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 333 val_333 +NULL NULL NULL NULL 335 val_335 +NULL NULL NULL NULL 336 val_336 +NULL NULL NULL NULL 338 val_338 +NULL NULL NULL NULL 339 val_339 +NULL NULL NULL NULL 341 val_341 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 342 val_342 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 344 val_344 +NULL NULL NULL NULL 345 val_345 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 348 val_348 +NULL NULL NULL NULL 351 val_351 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 353 val_353 +NULL NULL NULL NULL 356 val_356 +NULL NULL NULL NULL 360 val_360 +NULL NULL NULL NULL 362 val_362 +NULL NULL NULL NULL 364 val_364 +NULL NULL NULL NULL 365 val_365 +NULL NULL NULL NULL 366 val_366 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 367 val_367 +NULL NULL NULL NULL 368 val_368 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 369 val_369 +NULL NULL NULL NULL 373 val_373 +NULL NULL NULL NULL 374 val_374 +NULL NULL NULL NULL 375 val_375 +NULL NULL NULL NULL 377 val_377 +NULL NULL NULL NULL 378 val_378 +NULL NULL NULL NULL 379 val_379 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 382 val_382 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 384 val_384 +NULL NULL NULL NULL 386 val_386 +NULL NULL NULL NULL 389 val_389 +NULL NULL NULL NULL 392 val_392 +NULL NULL NULL NULL 393 val_393 +NULL NULL NULL NULL 394 val_394 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 395 val_395 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 396 val_396 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 397 val_397 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 399 val_399 +NULL NULL NULL NULL 400 val_400 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 401 val_401 +NULL NULL NULL NULL 402 val_402 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 403 val_403 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 404 val_404 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 406 val_406 +NULL NULL NULL NULL 407 val_407 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 409 val_409 +NULL NULL NULL NULL 411 val_411 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 413 val_413 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 414 val_414 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 417 val_417 +NULL NULL NULL NULL 418 val_418 +NULL NULL NULL NULL 419 val_419 +NULL NULL NULL NULL 421 val_421 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 424 val_424 +NULL NULL NULL NULL 427 val_427 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 429 val_429 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 430 val_430 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 431 val_431 +NULL NULL NULL NULL 432 val_432 +NULL NULL NULL NULL 435 val_435 +NULL NULL NULL NULL 436 val_436 +NULL NULL NULL NULL 437 val_437 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 438 val_438 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 439 val_439 +NULL NULL NULL NULL 443 val_443 +NULL NULL NULL NULL 444 val_444 +NULL NULL NULL NULL 446 val_446 +NULL NULL NULL NULL 448 val_448 +NULL NULL NULL NULL 449 val_449 +NULL NULL NULL NULL 452 val_452 +NULL NULL NULL NULL 453 val_453 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 454 val_454 +NULL NULL NULL NULL 455 val_455 +NULL NULL NULL NULL 457 val_457 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 458 val_458 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 459 val_459 +NULL NULL NULL NULL 460 val_460 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 462 val_462 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 463 val_463 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 466 val_466 +NULL NULL NULL NULL 467 val_467 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 468 val_468 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 469 val_469 +NULL NULL NULL NULL 470 val_470 +NULL NULL NULL NULL 472 val_472 +NULL NULL NULL NULL 475 val_475 +NULL NULL NULL NULL 477 val_477 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 478 val_478 +NULL NULL NULL NULL 479 val_479 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 480 val_480 +NULL NULL NULL NULL 481 val_481 +NULL NULL NULL NULL 482 val_482 +NULL NULL NULL NULL 483 val_483 +NULL NULL NULL NULL 484 val_484 +NULL NULL NULL NULL 485 val_485 +NULL NULL NULL NULL 487 val_487 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 489 val_489 +NULL NULL NULL NULL 490 val_490 +NULL NULL NULL NULL 491 val_491 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 492 val_492 +NULL NULL NULL NULL 493 val_493 +NULL NULL NULL NULL 494 val_494 +NULL NULL NULL NULL 495 val_495 +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 +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 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 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 +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 0 val_0 0 val_0 +0 val_0 0 val_0 0 val_0 +2 val_2 2 val_2 2 val_2 +4 val_4 4 val_4 4 val_4 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +5 val_5 5 val_5 5 val_5 +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 diff --git a/src/test/hive/golden/join40-9-6b7404af014eeb4de196577f14ef18cc b/src/test/hive/golden/join40-9-6b7404af014eeb4de196577f14ef18cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 b/src/test/hive/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 b/src/test/hive/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a b/src/test/hive/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 b/src/test/hive/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 new file mode 100644 index 0000000000000..7265626e5dbde --- /dev/null +++ b/src/test/hive/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 @@ -0,0 +1,3 @@ +0 val_0 NULL NULL +0 val_0 NULL NULL +0 val_0 NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e b/src/test/hive/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 b/src/test/hive/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 new file mode 100644 index 0000000000000..7265626e5dbde --- /dev/null +++ b/src/test/hive/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 @@ -0,0 +1,3 @@ +0 val_0 NULL NULL +0 val_0 NULL NULL +0 val_0 NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/join5-0-531e91e11b3891627c2675935fda14cd b/src/test/hive/golden/join5-0-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join5-1-76feff1282895f38b673f52acfd7429e b/src/test/hive/golden/join5-1-76feff1282895f38b673f52acfd7429e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 b/src/test/hive/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..06b2b4d7e6d47 --- /dev/null +++ b/src/test/hive/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,9 @@ +17 val_17 17 val_17 +NULL NULL 20 val_20 +NULL NULL 24 val_24 +19 val_19 19 val_19 +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 diff --git a/src/test/hive/golden/join6-0-531e91e11b3891627c2675935fda14cd b/src/test/hive/golden/join6-0-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 b/src/test/hive/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 b/src/test/hive/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..36ca4a0805f4b --- /dev/null +++ b/src/test/hive/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,14 @@ +11 val_11 NULL NULL +12 val_12 NULL NULL +12 val_12 NULL NULL +15 val_15 NULL NULL +15 val_15 NULL NULL +17 val_17 17 val_17 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +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 diff --git a/src/test/hive/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 b/src/test/hive/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 b/src/test/hive/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 b/src/test/hive/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..eb0a6246c1988 --- /dev/null +++ b/src/test/hive/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,14 @@ +11 val_11 NULL NULL NULL NULL +12 val_12 NULL NULL NULL NULL +12 val_12 NULL NULL NULL NULL +15 val_15 NULL NULL NULL NULL +15 val_15 NULL NULL NULL NULL +17 val_17 17 val_17 NULL NULL +18 val_18 18 val_18 NULL NULL +18 val_18 18 val_18 NULL NULL +18 val_18 18 val_18 NULL NULL +18 val_18 18 val_18 NULL NULL +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 diff --git a/src/test/hive/golden/join8-0-531e91e11b3891627c2675935fda14cd b/src/test/hive/golden/join8-0-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 b/src/test/hive/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join8-2-f1196bca86a749375da35f134206a8ca b/src/test/hive/golden/join8-2-f1196bca86a749375da35f134206a8ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f2e6e07d8fa1f --- /dev/null +++ b/src/test/hive/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,5 @@ +15 val_15 NULL NULL +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 diff --git a/src/test/hive/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a b/src/test/hive/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 b/src/test/hive/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..59821aeea3008 --- /dev/null +++ b/src/test/hive/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,1028 @@ +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +374 val_374 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +82 val_82 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +459 val_459 +394 val_394 +237 val_237 +237 val_237 +482 val_482 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +494 val_494 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +0 val_0 +0 val_0 +0 val_0 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +4 val_4 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +54 val_54 +459 val_459 +459 val_459 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +12 val_12 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +260 val_260 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +58 val_58 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +0 val_0 +0 val_0 +0 val_0 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +404 val_404 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +0 val_0 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +419 val_419 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +78 val_78 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +209 val_209 +64 val_64 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +12 val_12 +12 val_12 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +491 val_491 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +18 val_18 +18 val_18 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +448 val_448 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +414 val_414 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +200 val_200 +200 val_200 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 b/src/test/hive/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a b/src/test/hive/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e b/src/test/hive/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e b/src/test/hive/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/src/test/hive/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -0,0 +1,30 @@ +5 10005 66 5 10005 66 +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-12-6081eb976b4aef2132418510756a385b b/src/test/hive/golden/join_1to1-12-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-12-6081eb976b4aef2132418510756a385b @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c b/src/test/hive/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 b/src/test/hive/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 b/src/test/hive/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 b/src/test/hive/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e b/src/test/hive/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/src/test/hive/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -0,0 +1,30 @@ +5 10005 66 5 10005 66 +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-18-6081eb976b4aef2132418510756a385b b/src/test/hive/golden/join_1to1-18-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-18-6081eb976b4aef2132418510756a385b @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c b/src/test/hive/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 b/src/test/hive/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 b/src/test/hive/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 b/src/test/hive/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 b/src/test/hive/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e b/src/test/hive/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/src/test/hive/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -0,0 +1,30 @@ +5 10005 66 5 10005 66 +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-24-6081eb976b4aef2132418510756a385b b/src/test/hive/golden/join_1to1-24-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-24-6081eb976b4aef2132418510756a385b @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c b/src/test/hive/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 b/src/test/hive/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 b/src/test/hive/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e b/src/test/hive/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e b/src/test/hive/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/src/test/hive/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -0,0 +1,30 @@ +5 10005 66 5 10005 66 +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 b/src/test/hive/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_1to1-30-6081eb976b4aef2132418510756a385b b/src/test/hive/golden/join_1to1-30-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-30-6081eb976b4aef2132418510756a385b @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c b/src/test/hive/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 b/src/test/hive/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 b/src/test/hive/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 b/src/test/hive/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e b/src/test/hive/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/src/test/hive/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -0,0 +1,30 @@ +5 10005 66 5 10005 66 +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-36-6081eb976b4aef2132418510756a385b b/src/test/hive/golden/join_1to1-36-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-36-6081eb976b4aef2132418510756a385b @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c b/src/test/hive/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 b/src/test/hive/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 b/src/test/hive/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 b/src/test/hive/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e b/src/test/hive/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/src/test/hive/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -0,0 +1,30 @@ +5 10005 66 5 10005 66 +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-6-6081eb976b4aef2132418510756a385b b/src/test/hive/golden/join_1to1-6-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-6-6081eb976b4aef2132418510756a385b @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c b/src/test/hive/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 b/src/test/hive/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/src/test/hive/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 @@ -0,0 +1,36 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 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 diff --git a/src/test/hive/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 b/src/test/hive/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/src/test/hive/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 @@ -0,0 +1,34 @@ +NULL NULL NULL NULL NULL 66 +NULL NULL NULL NULL 10050 66 +NULL NULL NULL 10 10010 66 +NULL NULL NULL 25 10025 66 +NULL NULL NULL 30 10030 88 +NULL NULL NULL 35 10035 88 +NULL NULL NULL 40 10040 88 +NULL NULL NULL 50 10050 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 70 10040 88 +NULL NULL NULL 80 10040 66 +NULL NULL NULL 80 10040 66 +NULL NULL 66 NULL NULL NULL +NULL 10050 66 NULL NULL NULL +5 10005 66 5 10005 66 +15 10015 66 NULL NULL NULL +20 10020 66 20 10020 66 +25 10025 88 NULL NULL NULL +30 10030 66 NULL NULL NULL +35 10035 88 NULL NULL NULL +40 10040 66 40 10040 66 +40 10040 88 NULL NULL NULL +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 88 NULL NULL NULL +50 10050 88 NULL NULL NULL +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +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 diff --git a/src/test/hive/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 b/src/test/hive/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_array-1-2c086fcf118fd2538bfa00724209091e b/src/test/hive/golden/join_array-1-2c086fcf118fd2538bfa00724209091e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de b/src/test/hive/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 b/src/test/hive/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 b/src/test/hive/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 new file mode 100644 index 0000000000000..ce9688a065b34 --- /dev/null +++ b/src/test/hive/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 @@ -0,0 +1 @@ +10320092026892491 3312 \ No newline at end of file diff --git a/src/test/hive/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a b/src/test/hive/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a new file mode 100644 index 0000000000000..f937af554adef --- /dev/null +++ b/src/test/hive/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a @@ -0,0 +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 diff --git a/src/test/hive/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c b/src/test/hive/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c new file mode 100644 index 0000000000000..f937af554adef --- /dev/null +++ b/src/test/hive/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c @@ -0,0 +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 diff --git a/src/test/hive/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 b/src/test/hive/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 b/src/test/hive/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d b/src/test/hive/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc b/src/test/hive/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd b/src/test/hive/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd new file mode 100644 index 0000000000000..810f01f682c37 --- /dev/null +++ b/src/test/hive/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd @@ -0,0 +1,85 @@ +NULL 10050 66 NULL 10050 66 +NULL 10050 66 50 10050 66 +NULL 10050 66 50 10050 66 +NULL 10050 66 50 10050 88 +5 10005 66 5 10005 66 +20 10020 66 20 10020 66 +25 10025 88 25 10025 66 +30 10030 66 30 10030 88 +35 10035 88 35 10035 88 +40 10040 66 40 10040 66 +40 10040 66 40 10040 88 +40 10040 66 60 10040 66 +40 10040 66 60 10040 66 +40 10040 66 70 10040 88 +40 10040 66 70 10040 88 +40 10040 66 80 10040 66 +40 10040 66 80 10040 66 +40 10040 88 40 10040 66 +40 10040 88 40 10040 88 +40 10040 88 60 10040 66 +40 10040 88 60 10040 66 +40 10040 88 70 10040 88 +40 10040 88 70 10040 88 +40 10040 88 80 10040 66 +40 10040 88 80 10040 66 +50 10050 66 NULL 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 66 +50 10050 66 50 10050 88 +50 10050 88 NULL 10050 66 +50 10050 88 NULL 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 66 +50 10050 88 50 10050 88 +50 10050 88 50 10050 88 +60 10040 66 40 10040 66 +60 10040 66 40 10040 66 +60 10040 66 40 10040 88 +60 10040 66 40 10040 88 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 60 10040 66 +60 10040 66 70 10040 88 +60 10040 66 70 10040 88 +60 10040 66 70 10040 88 +60 10040 66 70 10040 88 +60 10040 66 80 10040 66 +60 10040 66 80 10040 66 +60 10040 66 80 10040 66 +60 10040 66 80 10040 66 +70 10040 66 40 10040 66 +70 10040 66 40 10040 66 +70 10040 66 40 10040 88 +70 10040 66 40 10040 88 +70 10040 66 60 10040 66 +70 10040 66 60 10040 66 +70 10040 66 60 10040 66 +70 10040 66 60 10040 66 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 70 10040 88 +70 10040 66 80 10040 66 +70 10040 66 80 10040 66 +70 10040 66 80 10040 66 +70 10040 66 80 10040 66 +80 10040 88 40 10040 66 +80 10040 88 40 10040 66 +80 10040 88 40 10040 88 +80 10040 88 40 10040 88 +80 10040 88 60 10040 66 +80 10040 88 60 10040 66 +80 10040 88 60 10040 66 +80 10040 88 60 10040 66 +80 10040 88 70 10040 88 +80 10040 88 70 10040 88 +80 10040 88 70 10040 88 +80 10040 88 70 10040 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 diff --git a/src/test/hive/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad b/src/test/hive/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 b/src/test/hive/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e b/src/test/hive/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_empty-3-3b2231f25886c152f222856256ac3473 b/src/test/hive/golden/join_empty-3-3b2231f25886c152f222856256ac3473 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a b/src/test/hive/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a b/src/test/hive/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 b/src/test/hive/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 b/src/test/hive/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 b/src/test/hive/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 b/src/test/hive/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe b/src/test/hive/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b b/src/test/hive/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 b/src/test/hive/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b b/src/test/hive/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 b/src/test/hive/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb b/src/test/hive/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-106-870999978978b22f21997899f1e652b8 b/src/test/hive/golden/join_filters-106-870999978978b22f21997899f1e652b8 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-106-870999978978b22f21997899f1e652b8 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 b/src/test/hive/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e b/src/test/hive/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 b/src/test/hive/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 b/src/test/hive/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba b/src/test/hive/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 b/src/test/hive/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 b/src/test/hive/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-113-48c4978768872751832149d72cdf0ba b/src/test/hive/golden/join_filters-113-48c4978768872751832149d72cdf0ba new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-113-48c4978768872751832149d72cdf0ba @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 b/src/test/hive/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 b/src/test/hive/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 b/src/test/hive/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 b/src/test/hive/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/src/test/hive/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 @@ -0,0 +1,6 @@ +NULL 35 NULL NULL +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 diff --git a/src/test/hive/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e b/src/test/hive/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/src/test/hive/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e @@ -0,0 +1,6 @@ +NULL 35 NULL NULL +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 diff --git a/src/test/hive/golden/join_filters-119-7688784396db55ff148292a9755c918a b/src/test/hive/golden/join_filters-119-7688784396db55ff148292a9755c918a new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/src/test/hive/golden/join_filters-119-7688784396db55ff148292a9755c918a @@ -0,0 +1,6 @@ +NULL 35 NULL NULL +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 diff --git a/src/test/hive/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e b/src/test/hive/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 b/src/test/hive/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/src/test/hive/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 @@ -0,0 +1,6 @@ +NULL NULL NULL 35 +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 diff --git a/src/test/hive/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada b/src/test/hive/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/src/test/hive/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada @@ -0,0 +1,6 @@ +NULL NULL NULL 35 +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 diff --git a/src/test/hive/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 b/src/test/hive/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/src/test/hive/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 @@ -0,0 +1,6 @@ +NULL NULL NULL 35 +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 diff --git a/src/test/hive/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 b/src/test/hive/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf b/src/test/hive/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 b/src/test/hive/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 b/src/test/hive/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 b/src/test/hive/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 b/src/test/hive/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/src/test/hive/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 @@ -0,0 +1,7 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +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 diff --git a/src/test/hive/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f b/src/test/hive/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/src/test/hive/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f @@ -0,0 +1,7 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +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 diff --git a/src/test/hive/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 b/src/test/hive/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf b/src/test/hive/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/src/test/hive/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf @@ -0,0 +1,7 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +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 diff --git a/src/test/hive/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e b/src/test/hive/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/src/test/hive/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e @@ -0,0 +1,7 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +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 diff --git a/src/test/hive/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e b/src/test/hive/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/src/test/hive/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-23-a800b885358695758afdb719cdefa94f b/src/test/hive/golden/join_filters-23-a800b885358695758afdb719cdefa94f new file mode 100644 index 0000000000000..ea646edf6d143 --- /dev/null +++ b/src/test/hive/golden/join_filters-23-a800b885358695758afdb719cdefa94f @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 b/src/test/hive/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/src/test/hive/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 b/src/test/hive/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/src/test/hive/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca b/src/test/hive/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca new file mode 100644 index 0000000000000..ea646edf6d143 --- /dev/null +++ b/src/test/hive/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 b/src/test/hive/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/src/test/hive/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 b/src/test/hive/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 b/src/test/hive/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 b/src/test/hive/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf b/src/test/hive/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f b/src/test/hive/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-32-5978cd7936c296493a16a31b926043ab b/src/test/hive/golden/join_filters-32-5978cd7936c296493a16a31b926043ab new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-32-5978cd7936c296493a16a31b926043ab @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 b/src/test/hive/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 b/src/test/hive/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe b/src/test/hive/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b b/src/test/hive/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 b/src/test/hive/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b b/src/test/hive/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 b/src/test/hive/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 b/src/test/hive/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb b/src/test/hive/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-41-870999978978b22f21997899f1e652b8 b/src/test/hive/golden/join_filters-41-870999978978b22f21997899f1e652b8 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-41-870999978978b22f21997899f1e652b8 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 b/src/test/hive/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e b/src/test/hive/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e b/src/test/hive/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 b/src/test/hive/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b b/src/test/hive/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 b/src/test/hive/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c b/src/test/hive/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 b/src/test/hive/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e b/src/test/hive/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/src/test/hive/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e @@ -0,0 +1,7 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +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 diff --git a/src/test/hive/golden/join_filters-50-3e6612a89e9124592e790594775054b1 b/src/test/hive/golden/join_filters-50-3e6612a89e9124592e790594775054b1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_filters-50-3e6612a89e9124592e790594775054b1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac b/src/test/hive/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 b/src/test/hive/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 b/src/test/hive/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba b/src/test/hive/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 b/src/test/hive/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 b/src/test/hive/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-57-48c4978768872751832149d72cdf0ba b/src/test/hive/golden/join_filters-57-48c4978768872751832149d72cdf0ba new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-57-48c4978768872751832149d72cdf0ba @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 b/src/test/hive/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 b/src/test/hive/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 b/src/test/hive/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 b/src/test/hive/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 b/src/test/hive/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/src/test/hive/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 @@ -0,0 +1,6 @@ +NULL 35 NULL NULL +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 diff --git a/src/test/hive/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e b/src/test/hive/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/src/test/hive/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e @@ -0,0 +1,6 @@ +NULL 35 NULL NULL +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 diff --git a/src/test/hive/golden/join_filters-63-7688784396db55ff148292a9755c918a b/src/test/hive/golden/join_filters-63-7688784396db55ff148292a9755c918a new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/src/test/hive/golden/join_filters-63-7688784396db55ff148292a9755c918a @@ -0,0 +1,6 @@ +NULL 35 NULL NULL +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 diff --git a/src/test/hive/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 b/src/test/hive/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/src/test/hive/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 @@ -0,0 +1,6 @@ +NULL NULL NULL 35 +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 diff --git a/src/test/hive/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada b/src/test/hive/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/src/test/hive/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada @@ -0,0 +1,6 @@ +NULL NULL NULL 35 +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 diff --git a/src/test/hive/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 b/src/test/hive/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/src/test/hive/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 @@ -0,0 +1,6 @@ +NULL NULL NULL 35 +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 diff --git a/src/test/hive/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 b/src/test/hive/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 b/src/test/hive/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 b/src/test/hive/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d b/src/test/hive/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e b/src/test/hive/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/src/test/hive/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e @@ -0,0 +1,7 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +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 diff --git a/src/test/hive/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 b/src/test/hive/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d b/src/test/hive/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 b/src/test/hive/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa b/src/test/hive/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 b/src/test/hive/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 b/src/test/hive/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e b/src/test/hive/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 b/src/test/hive/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/src/test/hive/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 @@ -0,0 +1,4 @@ +NULL 40 NULL NULL +12 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf b/src/test/hive/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 b/src/test/hive/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 b/src/test/hive/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 b/src/test/hive/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 b/src/test/hive/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/src/test/hive/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 @@ -0,0 +1,4 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 b/src/test/hive/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/src/test/hive/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 @@ -0,0 +1,7 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +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 diff --git a/src/test/hive/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f b/src/test/hive/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/src/test/hive/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f @@ -0,0 +1,7 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +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 diff --git a/src/test/hive/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf b/src/test/hive/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/src/test/hive/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf @@ -0,0 +1,7 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +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 diff --git a/src/test/hive/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e b/src/test/hive/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/src/test/hive/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e @@ -0,0 +1,7 @@ +NULL NULL NULL 40 +NULL NULL 12 35 +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 diff --git a/src/test/hive/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e b/src/test/hive/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/src/test/hive/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-88-a800b885358695758afdb719cdefa94f b/src/test/hive/golden/join_filters-88-a800b885358695758afdb719cdefa94f new file mode 100644 index 0000000000000..ea646edf6d143 --- /dev/null +++ b/src/test/hive/golden/join_filters-88-a800b885358695758afdb719cdefa94f @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 b/src/test/hive/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/src/test/hive/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa b/src/test/hive/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 b/src/test/hive/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/src/test/hive/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca b/src/test/hive/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca new file mode 100644 index 0000000000000..ea646edf6d143 --- /dev/null +++ b/src/test/hive/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 b/src/test/hive/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/src/test/hive/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 @@ -0,0 +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 diff --git a/src/test/hive/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 b/src/test/hive/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 b/src/test/hive/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf b/src/test/hive/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f b/src/test/hive/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-97-5978cd7936c296493a16a31b926043ab b/src/test/hive/golden/join_filters-97-5978cd7936c296493a16a31b926043ab new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-97-5978cd7936c296493a16a31b926043ab @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 b/src/test/hive/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 b/src/test/hive/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce b/src/test/hive/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 b/src/test/hive/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 b/src/test/hive/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 b/src/test/hive/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e b/src/test/hive/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 b/src/test/hive/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 b/src/test/hive/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 b/src/test/hive/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 b/src/test/hive/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 b/src/test/hive/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 new file mode 100644 index 0000000000000..e3143e49c2bad --- /dev/null +++ b/src/test/hive/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 @@ -0,0 +1 @@ +foo1 bar10 2 \ No newline at end of file diff --git a/src/test/hive/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 b/src/test/hive/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 b/src/test/hive/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..4c88927e56579 --- /dev/null +++ b/src/test/hive/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,107 @@ +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +224 val_224 val_224 +224 val_224 val_224 +224 val_224 val_224 +224 val_224 val_224 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e b/src/test/hive/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa b/src/test/hive/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..4c88927e56579 --- /dev/null +++ b/src/test/hive/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -0,0 +1,107 @@ +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +146 val_146 val_146 +150 val_150 val_150 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +224 val_224 val_224 +224 val_224 val_224 +224 val_224 val_224 +224 val_224 val_224 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +238 val_238 val_238 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +278 val_278 val_278 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +66 val_66 val_66 +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 diff --git a/src/test/hive/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 b/src/test/hive/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 b/src/test/hive/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a b/src/test/hive/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db b/src/test/hive/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed b/src/test/hive/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a b/src/test/hive/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 b/src/test/hive/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 b/src/test/hive/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b b/src/test/hive/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b new file mode 100644 index 0000000000000..4bc2d3969f17f --- /dev/null +++ b/src/test/hive/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b @@ -0,0 +1,3 @@ +NULL 35 NULL 35 +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 b/src/test/hive/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 new file mode 100644 index 0000000000000..8414cfbede40a --- /dev/null +++ b/src/test/hive/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 @@ -0,0 +1,3 @@ +NULL 35 NULL NULL +48 NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 b/src/test/hive/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 new file mode 100644 index 0000000000000..c11ae6cce7abb --- /dev/null +++ b/src/test/hive/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 @@ -0,0 +1,3 @@ +NULL 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d b/src/test/hive/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d new file mode 100644 index 0000000000000..e28e1b1e5e8b1 --- /dev/null +++ b/src/test/hive/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d @@ -0,0 +1,3 @@ +NULL NULL NULL 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 b/src/test/hive/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 new file mode 100644 index 0000000000000..85192b86801e2 --- /dev/null +++ b/src/test/hive/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 @@ -0,0 +1,3 @@ +NULL NULL NULL 35 +48 NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 b/src/test/hive/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 new file mode 100644 index 0000000000000..c19237d5d4309 --- /dev/null +++ b/src/test/hive/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 @@ -0,0 +1,3 @@ +NULL NULL 48 NULL +NULL 35 NULL 35 +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 b/src/test/hive/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 new file mode 100644 index 0000000000000..e28e1b1e5e8b1 --- /dev/null +++ b/src/test/hive/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 @@ -0,0 +1,3 @@ +NULL NULL NULL 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f b/src/test/hive/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f new file mode 100644 index 0000000000000..56ca29ffe263c --- /dev/null +++ b/src/test/hive/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 b/src/test/hive/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 new file mode 100644 index 0000000000000..92e228d381628 --- /dev/null +++ b/src/test/hive/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 @@ -0,0 +1,4 @@ +NULL NULL NULL 35 +NULL 35 NULL NULL +48 NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c b/src/test/hive/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c new file mode 100644 index 0000000000000..f57bbd5152852 --- /dev/null +++ b/src/test/hive/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c @@ -0,0 +1,4 @@ +NULL NULL 48 NULL +NULL 35 NULL 35 +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 b/src/test/hive/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 new file mode 100644 index 0000000000000..0d14bf4e38509 --- /dev/null +++ b/src/test/hive/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 @@ -0,0 +1,9 @@ +NULL 35 NULL 35 +NULL 35 48 NULL +NULL 35 100 100 +48 NULL NULL 35 +48 NULL 48 NULL +48 NULL 100 100 +100 100 NULL 35 +100 100 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 b/src/test/hive/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 new file mode 100644 index 0000000000000..56ca29ffe263c --- /dev/null +++ b/src/test/hive/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e b/src/test/hive/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e new file mode 100644 index 0000000000000..0db9ffd61a3ad --- /dev/null +++ b/src/test/hive/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e @@ -0,0 +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 diff --git a/src/test/hive/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 b/src/test/hive/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 new file mode 100644 index 0000000000000..ed1b35fbe80ae --- /dev/null +++ b/src/test/hive/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 @@ -0,0 +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 diff --git a/src/test/hive/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 b/src/test/hive/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 new file mode 100644 index 0000000000000..0db9ffd61a3ad --- /dev/null +++ b/src/test/hive/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 @@ -0,0 +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 diff --git a/src/test/hive/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 b/src/test/hive/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 new file mode 100644 index 0000000000000..0d14bf4e38509 --- /dev/null +++ b/src/test/hive/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 @@ -0,0 +1,9 @@ +NULL 35 NULL 35 +NULL 35 48 NULL +NULL 35 100 100 +48 NULL NULL 35 +48 NULL 48 NULL +48 NULL 100 100 +100 100 NULL 35 +100 100 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 b/src/test/hive/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 b/src/test/hive/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 new file mode 100644 index 0000000000000..bdd09da460bbe --- /dev/null +++ b/src/test/hive/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 @@ -0,0 +1,2 @@ +48 NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 b/src/test/hive/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 new file mode 100644 index 0000000000000..dfe4f4318fc66 --- /dev/null +++ b/src/test/hive/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 @@ -0,0 +1,2 @@ +NULL 35 NULL 35 +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb b/src/test/hive/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb b/src/test/hive/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb new file mode 100644 index 0000000000000..bdd09da460bbe --- /dev/null +++ b/src/test/hive/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb @@ -0,0 +1,2 @@ +48 NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 b/src/test/hive/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 new file mode 100644 index 0000000000000..0d14bf4e38509 --- /dev/null +++ b/src/test/hive/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 @@ -0,0 +1,9 @@ +NULL 35 NULL 35 +NULL 35 48 NULL +NULL 35 100 100 +48 NULL NULL 35 +48 NULL 48 NULL +48 NULL 100 100 +100 100 NULL 35 +100 100 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 b/src/test/hive/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 new file mode 100644 index 0000000000000..dfe4f4318fc66 --- /dev/null +++ b/src/test/hive/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 @@ -0,0 +1,2 @@ +NULL 35 NULL 35 +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 b/src/test/hive/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 b/src/test/hive/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 new file mode 100644 index 0000000000000..c11ae6cce7abb --- /dev/null +++ b/src/test/hive/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 @@ -0,0 +1,3 @@ +NULL 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 b/src/test/hive/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 new file mode 100644 index 0000000000000..8414cfbede40a --- /dev/null +++ b/src/test/hive/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 @@ -0,0 +1,3 @@ +NULL 35 NULL NULL +48 NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 b/src/test/hive/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 new file mode 100644 index 0000000000000..4bc2d3969f17f --- /dev/null +++ b/src/test/hive/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 @@ -0,0 +1,3 @@ +NULL 35 NULL 35 +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 b/src/test/hive/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 new file mode 100644 index 0000000000000..e28e1b1e5e8b1 --- /dev/null +++ b/src/test/hive/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 @@ -0,0 +1,3 @@ +NULL NULL NULL 35 +NULL NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 b/src/test/hive/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 new file mode 100644 index 0000000000000..85192b86801e2 --- /dev/null +++ b/src/test/hive/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 @@ -0,0 +1,3 @@ +NULL NULL NULL 35 +48 NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc b/src/test/hive/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc new file mode 100644 index 0000000000000..c19237d5d4309 --- /dev/null +++ b/src/test/hive/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc @@ -0,0 +1,3 @@ +NULL NULL 48 NULL +NULL 35 NULL 35 +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e b/src/test/hive/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 b/src/test/hive/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 b/src/test/hive/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 new file mode 100644 index 0000000000000..0d14bf4e38509 --- /dev/null +++ b/src/test/hive/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 @@ -0,0 +1,9 @@ +NULL 35 NULL 35 +NULL 35 48 NULL +NULL 35 100 100 +48 NULL NULL 35 +48 NULL 48 NULL +48 NULL 100 100 +100 100 NULL 35 +100 100 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b b/src/test/hive/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 b/src/test/hive/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c b/src/test/hive/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 b/src/test/hive/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 b/src/test/hive/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 b/src/test/hive/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 b/src/test/hive/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 b/src/test/hive/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 new file mode 100644 index 0000000000000..7d9efd15fa287 --- /dev/null +++ b/src/test/hive/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 @@ -0,0 +1,4 @@ +48 NULL 48 NULL +100 100 100 100 +148 NULL 148 NULL +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 b/src/test/hive/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 b/src/test/hive/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 new file mode 100644 index 0000000000000..4d2ff6a237416 --- /dev/null +++ b/src/test/hive/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 @@ -0,0 +1,6 @@ +NULL NULL NULL 35 +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 diff --git a/src/test/hive/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c b/src/test/hive/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 b/src/test/hive/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 new file mode 100644 index 0000000000000..7d9efd15fa287 --- /dev/null +++ b/src/test/hive/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 @@ -0,0 +1,4 @@ +48 NULL 48 NULL +100 100 100 100 +148 NULL 148 NULL +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f b/src/test/hive/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f new file mode 100644 index 0000000000000..695bb1544ec89 --- /dev/null +++ b/src/test/hive/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f @@ -0,0 +1,6 @@ +NULL 35 NULL NULL +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 diff --git a/src/test/hive/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 b/src/test/hive/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 b/src/test/hive/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/src/test/hive/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 @@ -0,0 +1,2 @@ +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 b/src/test/hive/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/src/test/hive/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 @@ -0,0 +1,6 @@ +NULL 35 NULL NULL +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 diff --git a/src/test/hive/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 b/src/test/hive/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/src/test/hive/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 @@ -0,0 +1,6 @@ +NULL NULL NULL 35 +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 diff --git a/src/test/hive/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 b/src/test/hive/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 new file mode 100644 index 0000000000000..86b3dab6bffc8 --- /dev/null +++ b/src/test/hive/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 @@ -0,0 +1,4 @@ +NULL 35 NULL 35 +NULL 135 NULL 135 +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 b/src/test/hive/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 new file mode 100644 index 0000000000000..169df31887b96 --- /dev/null +++ b/src/test/hive/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 @@ -0,0 +1,6 @@ +NULL NULL 48 NULL +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 diff --git a/src/test/hive/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a b/src/test/hive/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a new file mode 100644 index 0000000000000..86b3dab6bffc8 --- /dev/null +++ b/src/test/hive/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a @@ -0,0 +1,4 @@ +NULL 35 NULL 35 +NULL 135 NULL 135 +100 100 100 100 +200 200 200 200 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 b/src/test/hive/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 new file mode 100644 index 0000000000000..bdd5dcf63fd00 --- /dev/null +++ b/src/test/hive/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 @@ -0,0 +1,6 @@ +NULL 35 NULL 35 +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 diff --git a/src/test/hive/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 b/src/test/hive/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 new file mode 100644 index 0000000000000..bdd09da460bbe --- /dev/null +++ b/src/test/hive/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 @@ -0,0 +1,2 @@ +48 NULL 48 NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 b/src/test/hive/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 new file mode 100644 index 0000000000000..dfe4f4318fc66 --- /dev/null +++ b/src/test/hive/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 @@ -0,0 +1,2 @@ +NULL 35 NULL 35 +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 b/src/test/hive/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b b/src/test/hive/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b new file mode 100644 index 0000000000000..c11ae6cce7abb --- /dev/null +++ b/src/test/hive/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b @@ -0,0 +1,3 @@ +NULL 35 NULL NULL +48 NULL NULL NULL +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 b/src/test/hive/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e b/src/test/hive/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 b/src/test/hive/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee b/src/test/hive/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 b/src/test/hive/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 b/src/test/hive/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 new file mode 100644 index 0000000000000..59821aeea3008 --- /dev/null +++ b/src/test/hive/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 @@ -0,0 +1,1028 @@ +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +374 val_374 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +82 val_82 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +459 val_459 +394 val_394 +237 val_237 +237 val_237 +482 val_482 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +494 val_494 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +0 val_0 +0 val_0 +0 val_0 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +4 val_4 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +54 val_54 +459 val_459 +459 val_459 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +12 val_12 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +260 val_260 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +58 val_58 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +0 val_0 +0 val_0 +0 val_0 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +404 val_404 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +0 val_0 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +419 val_419 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +78 val_78 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +209 val_209 +64 val_64 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +12 val_12 +12 val_12 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +491 val_491 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +18 val_18 +18 val_18 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +448 val_448 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +414 val_414 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +200 val_200 +200 val_200 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 b/src/test/hive/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 b/src/test/hive/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d b/src/test/hive/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 b/src/test/hive/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 new file mode 100644 index 0000000000000..06a83e95d4f79 --- /dev/null +++ b/src/test/hive/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 @@ -0,0 +1 @@ +2 12 2 22 2 12 2 12 \ No newline at end of file diff --git a/src/test/hive/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e b/src/test/hive/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 b/src/test/hive/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 b/src/test/hive/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec b/src/test/hive/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 b/src/test/hive/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 b/src/test/hive/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 new file mode 100644 index 0000000000000..06a83e95d4f79 --- /dev/null +++ b/src/test/hive/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 @@ -0,0 +1 @@ +2 12 2 22 2 12 2 12 \ No newline at end of file diff --git a/src/test/hive/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 b/src/test/hive/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 b/src/test/hive/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 b/src/test/hive/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 b/src/test/hive/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 new file mode 100644 index 0000000000000..06a83e95d4f79 --- /dev/null +++ b/src/test/hive/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 @@ -0,0 +1 @@ +2 12 2 22 2 12 2 12 \ No newline at end of file diff --git a/src/test/hive/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e b/src/test/hive/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 b/src/test/hive/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 b/src/test/hive/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec b/src/test/hive/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 b/src/test/hive/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 b/src/test/hive/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 new file mode 100644 index 0000000000000..06a83e95d4f79 --- /dev/null +++ b/src/test/hive/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 @@ -0,0 +1 @@ +2 12 2 22 2 12 2 12 \ No newline at end of file diff --git a/src/test/hive/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf b/src/test/hive/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a b/src/test/hive/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f b/src/test/hive/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f new file mode 100644 index 0000000000000..872146532307a --- /dev/null +++ b/src/test/hive/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f @@ -0,0 +1 @@ +2 12 2 22 2 12 \ No newline at end of file diff --git a/src/test/hive/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 b/src/test/hive/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 b/src/test/hive/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 new file mode 100644 index 0000000000000..872146532307a --- /dev/null +++ b/src/test/hive/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 @@ -0,0 +1 @@ +2 12 2 22 2 12 \ No newline at end of file diff --git a/src/test/hive/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c b/src/test/hive/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 b/src/test/hive/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 b/src/test/hive/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 b/src/test/hive/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 new file mode 100644 index 0000000000000..872146532307a --- /dev/null +++ b/src/test/hive/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 @@ -0,0 +1 @@ +2 12 2 22 2 12 \ No newline at end of file diff --git a/src/test/hive/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba b/src/test/hive/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-0-6001f83f7780835737e3262a34c2b832 b/src/test/hive/golden/join_star-0-6001f83f7780835737e3262a34c2b832 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d b/src/test/hive/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e b/src/test/hive/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae b/src/test/hive/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 b/src/test/hive/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-13-342b7249c9ce1484869169b1b33191cb b/src/test/hive/golden/join_star-13-342b7249c9ce1484869169b1b33191cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-14-75513308d30b781fd2e06d81963c4363 b/src/test/hive/golden/join_star-14-75513308d30b781fd2e06d81963c4363 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 b/src/test/hive/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-16-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/join_star-16-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_star-16-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 b/src/test/hive/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa b/src/test/hive/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 b/src/test/hive/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 b/src/test/hive/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 new file mode 100644 index 0000000000000..e60bca6269264 --- /dev/null +++ b/src/test/hive/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 @@ -0,0 +1,10 @@ +1 2 1 +11 12 1 +21 22 1 +31 32 1 +41 42 1 +51 52 3 +61 62 3 +71 72 3 +81 82 3 +91 92 3 \ No newline at end of file diff --git a/src/test/hive/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 b/src/test/hive/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-22-429cfd140488242d569aee6594aae76b b/src/test/hive/golden/join_star-22-429cfd140488242d569aee6594aae76b new file mode 100644 index 0000000000000..e08d04997c56b --- /dev/null +++ b/src/test/hive/golden/join_star-22-429cfd140488242d569aee6594aae76b @@ -0,0 +1,10 @@ +1 2 1 2 +11 12 1 2 +21 22 1 2 +31 32 1 2 +41 42 1 2 +51 52 3 4 +61 62 3 4 +71 72 3 4 +81 82 3 4 +91 92 3 4 \ No newline at end of file diff --git a/src/test/hive/golden/join_star-23-bada259628918dbfb4837bcb58258530 b/src/test/hive/golden/join_star-23-bada259628918dbfb4837bcb58258530 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 b/src/test/hive/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 new file mode 100644 index 0000000000000..b89911131bd13 --- /dev/null +++ b/src/test/hive/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 @@ -0,0 +1,10 @@ +1 2 1 1 +11 12 1 1 +21 22 1 1 +31 32 1 1 +41 42 1 1 +51 52 3 3 +61 62 3 3 +71 72 3 3 +81 82 3 3 +91 92 3 3 \ No newline at end of file diff --git a/src/test/hive/golden/join_star-25-be5af28b6bccf529a31ef68619de699e b/src/test/hive/golden/join_star-25-be5af28b6bccf529a31ef68619de699e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 b/src/test/hive/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 new file mode 100644 index 0000000000000..b89911131bd13 --- /dev/null +++ b/src/test/hive/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 @@ -0,0 +1,10 @@ +1 2 1 1 +11 12 1 1 +21 22 1 1 +31 32 1 1 +41 42 1 1 +51 52 3 3 +61 62 3 3 +71 72 3 3 +81 82 3 3 +91 92 3 3 \ No newline at end of file diff --git a/src/test/hive/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 b/src/test/hive/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 b/src/test/hive/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 new file mode 100644 index 0000000000000..19611e75e33c3 --- /dev/null +++ b/src/test/hive/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 @@ -0,0 +1,10 @@ +1 2 1 1 2 2 2 2 2 +11 12 1 1 2 2 2 2 2 +21 22 1 1 2 2 2 2 2 +31 32 1 1 2 2 2 2 2 +41 42 1 1 2 2 2 2 2 +51 52 3 3 4 4 4 4 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 diff --git a/src/test/hive/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da b/src/test/hive/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-4-aaba115af7543b214820a48a694f2463 b/src/test/hive/golden/join_star-4-aaba115af7543b214820a48a694f2463 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da b/src/test/hive/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a b/src/test/hive/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a b/src/test/hive/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-8-a957982d8981ff0a35397ca449297024 b/src/test/hive/golden/join_star-8-a957982d8981ff0a35397ca449297024 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 b/src/test/hive/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 b/src/test/hive/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 new file mode 100644 index 0000000000000..07433297e6de1 --- /dev/null +++ b/src/test/hive/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 @@ -0,0 +1,6 @@ +aint int from deserializer +astring string from deserializer +lint array from deserializer +lstring array from deserializer +lintstring array> from deserializer +mstringstring map from deserializer \ No newline at end of file diff --git a/src/test/hive/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db b/src/test/hive/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b b/src/test/hive/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b new file mode 100644 index 0000000000000..4aeaf4ff1f1b0 --- /dev/null +++ b/src/test/hive/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b @@ -0,0 +1,11 @@ +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/src/test/hive/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 b/src/test/hive/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 b/src/test/hive/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 b/src/test/hive/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 b/src/test/hive/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b b/src/test/hive/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 b/src/test/hive/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 b/src/test/hive/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef b/src/test/hive/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d b/src/test/hive/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 b/src/test/hive/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 b/src/test/hive/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee b/src/test/hive/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee new file mode 100644 index 0000000000000..1b8942763520b --- /dev/null +++ b/src/test/hive/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee @@ -0,0 +1,1000 @@ +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 +1 +2 diff --git a/src/test/hive/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb b/src/test/hive/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb new file mode 100644 index 0000000000000..86296b92d98ad --- /dev/null +++ b/src/test/hive/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb @@ -0,0 +1,1000 @@ +238 val_238 1 +238 val_238 2 +86 val_86 1 +86 val_86 2 +311 val_311 1 +311 val_311 2 +27 val_27 1 +27 val_27 2 +165 val_165 1 +165 val_165 2 +409 val_409 1 +409 val_409 2 +255 val_255 1 +255 val_255 2 +278 val_278 1 +278 val_278 2 +98 val_98 1 +98 val_98 2 +484 val_484 1 +484 val_484 2 +265 val_265 1 +265 val_265 2 +193 val_193 1 +193 val_193 2 +401 val_401 1 +401 val_401 2 +150 val_150 1 +150 val_150 2 +273 val_273 1 +273 val_273 2 +224 val_224 1 +224 val_224 2 +369 val_369 1 +369 val_369 2 +66 val_66 1 +66 val_66 2 +128 val_128 1 +128 val_128 2 +213 val_213 1 +213 val_213 2 +146 val_146 1 +146 val_146 2 +406 val_406 1 +406 val_406 2 +429 val_429 1 +429 val_429 2 +374 val_374 1 +374 val_374 2 +152 val_152 1 +152 val_152 2 +469 val_469 1 +469 val_469 2 +145 val_145 1 +145 val_145 2 +495 val_495 1 +495 val_495 2 +37 val_37 1 +37 val_37 2 +327 val_327 1 +327 val_327 2 +281 val_281 1 +281 val_281 2 +277 val_277 1 +277 val_277 2 +209 val_209 1 +209 val_209 2 +15 val_15 1 +15 val_15 2 +82 val_82 1 +82 val_82 2 +403 val_403 1 +403 val_403 2 +166 val_166 1 +166 val_166 2 +417 val_417 1 +417 val_417 2 +430 val_430 1 +430 val_430 2 +252 val_252 1 +252 val_252 2 +292 val_292 1 +292 val_292 2 +219 val_219 1 +219 val_219 2 +287 val_287 1 +287 val_287 2 +153 val_153 1 +153 val_153 2 +193 val_193 1 +193 val_193 2 +338 val_338 1 +338 val_338 2 +446 val_446 1 +446 val_446 2 +459 val_459 1 +459 val_459 2 +394 val_394 1 +394 val_394 2 +237 val_237 1 +237 val_237 2 +482 val_482 1 +482 val_482 2 +174 val_174 1 +174 val_174 2 +413 val_413 1 +413 val_413 2 +494 val_494 1 +494 val_494 2 +207 val_207 1 +207 val_207 2 +199 val_199 1 +199 val_199 2 +466 val_466 1 +466 val_466 2 +208 val_208 1 +208 val_208 2 +174 val_174 1 +174 val_174 2 +399 val_399 1 +399 val_399 2 +396 val_396 1 +396 val_396 2 +247 val_247 1 +247 val_247 2 +417 val_417 1 +417 val_417 2 +489 val_489 1 +489 val_489 2 +162 val_162 1 +162 val_162 2 +377 val_377 1 +377 val_377 2 +397 val_397 1 +397 val_397 2 +309 val_309 1 +309 val_309 2 +365 val_365 1 +365 val_365 2 +266 val_266 1 +266 val_266 2 +439 val_439 1 +439 val_439 2 +342 val_342 1 +342 val_342 2 +367 val_367 1 +367 val_367 2 +325 val_325 1 +325 val_325 2 +167 val_167 1 +167 val_167 2 +195 val_195 1 +195 val_195 2 +475 val_475 1 +475 val_475 2 +17 val_17 1 +17 val_17 2 +113 val_113 1 +113 val_113 2 +155 val_155 1 +155 val_155 2 +203 val_203 1 +203 val_203 2 +339 val_339 1 +339 val_339 2 +0 val_0 1 +0 val_0 2 +455 val_455 1 +455 val_455 2 +128 val_128 1 +128 val_128 2 +311 val_311 1 +311 val_311 2 +316 val_316 1 +316 val_316 2 +57 val_57 1 +57 val_57 2 +302 val_302 1 +302 val_302 2 +205 val_205 1 +205 val_205 2 +149 val_149 1 +149 val_149 2 +438 val_438 1 +438 val_438 2 +345 val_345 1 +345 val_345 2 +129 val_129 1 +129 val_129 2 +170 val_170 1 +170 val_170 2 +20 val_20 1 +20 val_20 2 +489 val_489 1 +489 val_489 2 +157 val_157 1 +157 val_157 2 +378 val_378 1 +378 val_378 2 +221 val_221 1 +221 val_221 2 +92 val_92 1 +92 val_92 2 +111 val_111 1 +111 val_111 2 +47 val_47 1 +47 val_47 2 +72 val_72 1 +72 val_72 2 +4 val_4 1 +4 val_4 2 +280 val_280 1 +280 val_280 2 +35 val_35 1 +35 val_35 2 +427 val_427 1 +427 val_427 2 +277 val_277 1 +277 val_277 2 +208 val_208 1 +208 val_208 2 +356 val_356 1 +356 val_356 2 +399 val_399 1 +399 val_399 2 +169 val_169 1 +169 val_169 2 +382 val_382 1 +382 val_382 2 +498 val_498 1 +498 val_498 2 +125 val_125 1 +125 val_125 2 +386 val_386 1 +386 val_386 2 +437 val_437 1 +437 val_437 2 +469 val_469 1 +469 val_469 2 +192 val_192 1 +192 val_192 2 +286 val_286 1 +286 val_286 2 +187 val_187 1 +187 val_187 2 +176 val_176 1 +176 val_176 2 +54 val_54 1 +54 val_54 2 +459 val_459 1 +459 val_459 2 +51 val_51 1 +51 val_51 2 +138 val_138 1 +138 val_138 2 +103 val_103 1 +103 val_103 2 +239 val_239 1 +239 val_239 2 +213 val_213 1 +213 val_213 2 +216 val_216 1 +216 val_216 2 +430 val_430 1 +430 val_430 2 +278 val_278 1 +278 val_278 2 +176 val_176 1 +176 val_176 2 +289 val_289 1 +289 val_289 2 +221 val_221 1 +221 val_221 2 +65 val_65 1 +65 val_65 2 +318 val_318 1 +318 val_318 2 +332 val_332 1 +332 val_332 2 +311 val_311 1 +311 val_311 2 +275 val_275 1 +275 val_275 2 +137 val_137 1 +137 val_137 2 +241 val_241 1 +241 val_241 2 +83 val_83 1 +83 val_83 2 +333 val_333 1 +333 val_333 2 +180 val_180 1 +180 val_180 2 +284 val_284 1 +284 val_284 2 +12 val_12 1 +12 val_12 2 +230 val_230 1 +230 val_230 2 +181 val_181 1 +181 val_181 2 +67 val_67 1 +67 val_67 2 +260 val_260 1 +260 val_260 2 +404 val_404 1 +404 val_404 2 +384 val_384 1 +384 val_384 2 +489 val_489 1 +489 val_489 2 +353 val_353 1 +353 val_353 2 +373 val_373 1 +373 val_373 2 +272 val_272 1 +272 val_272 2 +138 val_138 1 +138 val_138 2 +217 val_217 1 +217 val_217 2 +84 val_84 1 +84 val_84 2 +348 val_348 1 +348 val_348 2 +466 val_466 1 +466 val_466 2 +58 val_58 1 +58 val_58 2 +8 val_8 1 +8 val_8 2 +411 val_411 1 +411 val_411 2 +230 val_230 1 +230 val_230 2 +208 val_208 1 +208 val_208 2 +348 val_348 1 +348 val_348 2 +24 val_24 1 +24 val_24 2 +463 val_463 1 +463 val_463 2 +431 val_431 1 +431 val_431 2 +179 val_179 1 +179 val_179 2 +172 val_172 1 +172 val_172 2 +42 val_42 1 +42 val_42 2 +129 val_129 1 +129 val_129 2 +158 val_158 1 +158 val_158 2 +119 val_119 1 +119 val_119 2 +496 val_496 1 +496 val_496 2 +0 val_0 1 +0 val_0 2 +322 val_322 1 +322 val_322 2 +197 val_197 1 +197 val_197 2 +468 val_468 1 +468 val_468 2 +393 val_393 1 +393 val_393 2 +454 val_454 1 +454 val_454 2 +100 val_100 1 +100 val_100 2 +298 val_298 1 +298 val_298 2 +199 val_199 1 +199 val_199 2 +191 val_191 1 +191 val_191 2 +418 val_418 1 +418 val_418 2 +96 val_96 1 +96 val_96 2 +26 val_26 1 +26 val_26 2 +165 val_165 1 +165 val_165 2 +327 val_327 1 +327 val_327 2 +230 val_230 1 +230 val_230 2 +205 val_205 1 +205 val_205 2 +120 val_120 1 +120 val_120 2 +131 val_131 1 +131 val_131 2 +51 val_51 1 +51 val_51 2 +404 val_404 1 +404 val_404 2 +43 val_43 1 +43 val_43 2 +436 val_436 1 +436 val_436 2 +156 val_156 1 +156 val_156 2 +469 val_469 1 +469 val_469 2 +468 val_468 1 +468 val_468 2 +308 val_308 1 +308 val_308 2 +95 val_95 1 +95 val_95 2 +196 val_196 1 +196 val_196 2 +288 val_288 1 +288 val_288 2 +481 val_481 1 +481 val_481 2 +457 val_457 1 +457 val_457 2 +98 val_98 1 +98 val_98 2 +282 val_282 1 +282 val_282 2 +197 val_197 1 +197 val_197 2 +187 val_187 1 +187 val_187 2 +318 val_318 1 +318 val_318 2 +318 val_318 1 +318 val_318 2 +409 val_409 1 +409 val_409 2 +470 val_470 1 +470 val_470 2 +137 val_137 1 +137 val_137 2 +369 val_369 1 +369 val_369 2 +316 val_316 1 +316 val_316 2 +169 val_169 1 +169 val_169 2 +413 val_413 1 +413 val_413 2 +85 val_85 1 +85 val_85 2 +77 val_77 1 +77 val_77 2 +0 val_0 1 +0 val_0 2 +490 val_490 1 +490 val_490 2 +87 val_87 1 +87 val_87 2 +364 val_364 1 +364 val_364 2 +179 val_179 1 +179 val_179 2 +118 val_118 1 +118 val_118 2 +134 val_134 1 +134 val_134 2 +395 val_395 1 +395 val_395 2 +282 val_282 1 +282 val_282 2 +138 val_138 1 +138 val_138 2 +238 val_238 1 +238 val_238 2 +419 val_419 1 +419 val_419 2 +15 val_15 1 +15 val_15 2 +118 val_118 1 +118 val_118 2 +72 val_72 1 +72 val_72 2 +90 val_90 1 +90 val_90 2 +307 val_307 1 +307 val_307 2 +19 val_19 1 +19 val_19 2 +435 val_435 1 +435 val_435 2 +10 val_10 1 +10 val_10 2 +277 val_277 1 +277 val_277 2 +273 val_273 1 +273 val_273 2 +306 val_306 1 +306 val_306 2 +224 val_224 1 +224 val_224 2 +309 val_309 1 +309 val_309 2 +389 val_389 1 +389 val_389 2 +327 val_327 1 +327 val_327 2 +242 val_242 1 +242 val_242 2 +369 val_369 1 +369 val_369 2 +392 val_392 1 +392 val_392 2 +272 val_272 1 +272 val_272 2 +331 val_331 1 +331 val_331 2 +401 val_401 1 +401 val_401 2 +242 val_242 1 +242 val_242 2 +452 val_452 1 +452 val_452 2 +177 val_177 1 +177 val_177 2 +226 val_226 1 +226 val_226 2 +5 val_5 1 +5 val_5 2 +497 val_497 1 +497 val_497 2 +402 val_402 1 +402 val_402 2 +396 val_396 1 +396 val_396 2 +317 val_317 1 +317 val_317 2 +395 val_395 1 +395 val_395 2 +58 val_58 1 +58 val_58 2 +35 val_35 1 +35 val_35 2 +336 val_336 1 +336 val_336 2 +95 val_95 1 +95 val_95 2 +11 val_11 1 +11 val_11 2 +168 val_168 1 +168 val_168 2 +34 val_34 1 +34 val_34 2 +229 val_229 1 +229 val_229 2 +233 val_233 1 +233 val_233 2 +143 val_143 1 +143 val_143 2 +472 val_472 1 +472 val_472 2 +322 val_322 1 +322 val_322 2 +498 val_498 1 +498 val_498 2 +160 val_160 1 +160 val_160 2 +195 val_195 1 +195 val_195 2 +42 val_42 1 +42 val_42 2 +321 val_321 1 +321 val_321 2 +430 val_430 1 +430 val_430 2 +119 val_119 1 +119 val_119 2 +489 val_489 1 +489 val_489 2 +458 val_458 1 +458 val_458 2 +78 val_78 1 +78 val_78 2 +76 val_76 1 +76 val_76 2 +41 val_41 1 +41 val_41 2 +223 val_223 1 +223 val_223 2 +492 val_492 1 +492 val_492 2 +149 val_149 1 +149 val_149 2 +449 val_449 1 +449 val_449 2 +218 val_218 1 +218 val_218 2 +228 val_228 1 +228 val_228 2 +138 val_138 1 +138 val_138 2 +453 val_453 1 +453 val_453 2 +30 val_30 1 +30 val_30 2 +209 val_209 1 +209 val_209 2 +64 val_64 1 +64 val_64 2 +468 val_468 1 +468 val_468 2 +76 val_76 1 +76 val_76 2 +74 val_74 1 +74 val_74 2 +342 val_342 1 +342 val_342 2 +69 val_69 1 +69 val_69 2 +230 val_230 1 +230 val_230 2 +33 val_33 1 +33 val_33 2 +368 val_368 1 +368 val_368 2 +103 val_103 1 +103 val_103 2 +296 val_296 1 +296 val_296 2 +113 val_113 1 +113 val_113 2 +216 val_216 1 +216 val_216 2 +367 val_367 1 +367 val_367 2 +344 val_344 1 +344 val_344 2 +167 val_167 1 +167 val_167 2 +274 val_274 1 +274 val_274 2 +219 val_219 1 +219 val_219 2 +239 val_239 1 +239 val_239 2 +485 val_485 1 +485 val_485 2 +116 val_116 1 +116 val_116 2 +223 val_223 1 +223 val_223 2 +256 val_256 1 +256 val_256 2 +263 val_263 1 +263 val_263 2 +70 val_70 1 +70 val_70 2 +487 val_487 1 +487 val_487 2 +480 val_480 1 +480 val_480 2 +401 val_401 1 +401 val_401 2 +288 val_288 1 +288 val_288 2 +191 val_191 1 +191 val_191 2 +5 val_5 1 +5 val_5 2 +244 val_244 1 +244 val_244 2 +438 val_438 1 +438 val_438 2 +128 val_128 1 +128 val_128 2 +467 val_467 1 +467 val_467 2 +432 val_432 1 +432 val_432 2 +202 val_202 1 +202 val_202 2 +316 val_316 1 +316 val_316 2 +229 val_229 1 +229 val_229 2 +469 val_469 1 +469 val_469 2 +463 val_463 1 +463 val_463 2 +280 val_280 1 +280 val_280 2 +2 val_2 1 +2 val_2 2 +35 val_35 1 +35 val_35 2 +283 val_283 1 +283 val_283 2 +331 val_331 1 +331 val_331 2 +235 val_235 1 +235 val_235 2 +80 val_80 1 +80 val_80 2 +44 val_44 1 +44 val_44 2 +193 val_193 1 +193 val_193 2 +321 val_321 1 +321 val_321 2 +335 val_335 1 +335 val_335 2 +104 val_104 1 +104 val_104 2 +466 val_466 1 +466 val_466 2 +366 val_366 1 +366 val_366 2 +175 val_175 1 +175 val_175 2 +403 val_403 1 +403 val_403 2 +483 val_483 1 +483 val_483 2 +53 val_53 1 +53 val_53 2 +105 val_105 1 +105 val_105 2 +257 val_257 1 +257 val_257 2 +406 val_406 1 +406 val_406 2 +409 val_409 1 +409 val_409 2 +190 val_190 1 +190 val_190 2 +406 val_406 1 +406 val_406 2 +401 val_401 1 +401 val_401 2 +114 val_114 1 +114 val_114 2 +258 val_258 1 +258 val_258 2 +90 val_90 1 +90 val_90 2 +203 val_203 1 +203 val_203 2 +262 val_262 1 +262 val_262 2 +348 val_348 1 +348 val_348 2 +424 val_424 1 +424 val_424 2 +12 val_12 1 +12 val_12 2 +396 val_396 1 +396 val_396 2 +201 val_201 1 +201 val_201 2 +217 val_217 1 +217 val_217 2 +164 val_164 1 +164 val_164 2 +431 val_431 1 +431 val_431 2 +454 val_454 1 +454 val_454 2 +478 val_478 1 +478 val_478 2 +298 val_298 1 +298 val_298 2 +125 val_125 1 +125 val_125 2 +431 val_431 1 +431 val_431 2 +164 val_164 1 +164 val_164 2 +424 val_424 1 +424 val_424 2 +187 val_187 1 +187 val_187 2 +382 val_382 1 +382 val_382 2 +5 val_5 1 +5 val_5 2 +70 val_70 1 +70 val_70 2 +397 val_397 1 +397 val_397 2 +480 val_480 1 +480 val_480 2 +291 val_291 1 +291 val_291 2 +24 val_24 1 +24 val_24 2 +351 val_351 1 +351 val_351 2 +255 val_255 1 +255 val_255 2 +104 val_104 1 +104 val_104 2 +70 val_70 1 +70 val_70 2 +163 val_163 1 +163 val_163 2 +438 val_438 1 +438 val_438 2 +119 val_119 1 +119 val_119 2 +414 val_414 1 +414 val_414 2 +200 val_200 1 +200 val_200 2 +491 val_491 1 +491 val_491 2 +237 val_237 1 +237 val_237 2 +439 val_439 1 +439 val_439 2 +360 val_360 1 +360 val_360 2 +248 val_248 1 +248 val_248 2 +479 val_479 1 +479 val_479 2 +305 val_305 1 +305 val_305 2 +417 val_417 1 +417 val_417 2 +199 val_199 1 +199 val_199 2 +444 val_444 1 +444 val_444 2 +120 val_120 1 +120 val_120 2 +429 val_429 1 +429 val_429 2 +169 val_169 1 +169 val_169 2 +443 val_443 1 +443 val_443 2 +323 val_323 1 +323 val_323 2 +325 val_325 1 +325 val_325 2 +277 val_277 1 +277 val_277 2 +230 val_230 1 +230 val_230 2 +478 val_478 1 +478 val_478 2 +178 val_178 1 +178 val_178 2 +468 val_468 1 +468 val_468 2 +310 val_310 1 +310 val_310 2 +317 val_317 1 +317 val_317 2 +333 val_333 1 +333 val_333 2 +493 val_493 1 +493 val_493 2 +460 val_460 1 +460 val_460 2 +207 val_207 1 +207 val_207 2 +249 val_249 1 +249 val_249 2 +265 val_265 1 +265 val_265 2 +480 val_480 1 +480 val_480 2 +83 val_83 1 +83 val_83 2 +136 val_136 1 +136 val_136 2 +353 val_353 1 +353 val_353 2 +172 val_172 1 +172 val_172 2 +214 val_214 1 +214 val_214 2 +462 val_462 1 +462 val_462 2 +233 val_233 1 +233 val_233 2 +406 val_406 1 +406 val_406 2 +133 val_133 1 +133 val_133 2 +175 val_175 1 +175 val_175 2 +189 val_189 1 +189 val_189 2 +454 val_454 1 +454 val_454 2 +375 val_375 1 +375 val_375 2 +401 val_401 1 +401 val_401 2 +421 val_421 1 +421 val_421 2 +407 val_407 1 +407 val_407 2 +384 val_384 1 +384 val_384 2 +256 val_256 1 +256 val_256 2 +26 val_26 1 +26 val_26 2 +134 val_134 1 +134 val_134 2 +67 val_67 1 +67 val_67 2 +384 val_384 1 +384 val_384 2 +379 val_379 1 +379 val_379 2 +18 val_18 1 +18 val_18 2 +462 val_462 1 +462 val_462 2 +492 val_492 1 +492 val_492 2 +100 val_100 1 +100 val_100 2 +298 val_298 1 +298 val_298 2 +9 val_9 1 +9 val_9 2 +341 val_341 1 +341 val_341 2 +498 val_498 1 +498 val_498 2 +146 val_146 1 +146 val_146 2 +458 val_458 1 +458 val_458 2 +362 val_362 1 +362 val_362 2 +186 val_186 1 +186 val_186 2 +285 val_285 1 +285 val_285 2 +348 val_348 1 +348 val_348 2 +167 val_167 1 +167 val_167 2 +18 val_18 1 +18 val_18 2 +273 val_273 1 +273 val_273 2 +183 val_183 1 +183 val_183 2 +281 val_281 1 +281 val_281 2 +344 val_344 1 +344 val_344 2 +97 val_97 1 +97 val_97 2 +469 val_469 1 +469 val_469 2 +315 val_315 1 +315 val_315 2 +84 val_84 1 +84 val_84 2 +28 val_28 1 +28 val_28 2 +37 val_37 1 +37 val_37 2 +448 val_448 1 +448 val_448 2 +152 val_152 1 +152 val_152 2 +348 val_348 1 +348 val_348 2 +307 val_307 1 +307 val_307 2 +194 val_194 1 +194 val_194 2 +414 val_414 1 +414 val_414 2 +477 val_477 1 +477 val_477 2 +222 val_222 1 +222 val_222 2 +126 val_126 1 +126 val_126 2 +90 val_90 1 +90 val_90 2 +169 val_169 1 +169 val_169 2 +403 val_403 1 +403 val_403 2 +400 val_400 1 +400 val_400 2 +200 val_200 1 +200 val_200 2 +97 val_97 1 +97 val_97 2 diff --git a/src/test/hive/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 b/src/test/hive/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 new file mode 100644 index 0000000000000..8216956a5c636 --- /dev/null +++ b/src/test/hive/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 @@ -0,0 +1,1000 @@ +238 241 +238 242 +86 89 +86 90 +311 314 +311 315 +27 30 +27 31 +165 168 +165 169 +409 412 +409 413 +255 258 +255 259 +278 281 +278 282 +98 101 +98 102 +484 487 +484 488 +265 268 +265 269 +193 196 +193 197 +401 404 +401 405 +150 153 +150 154 +273 276 +273 277 +224 227 +224 228 +369 372 +369 373 +66 69 +66 70 +128 131 +128 132 +213 216 +213 217 +146 149 +146 150 +406 409 +406 410 +429 432 +429 433 +374 377 +374 378 +152 155 +152 156 +469 472 +469 473 +145 148 +145 149 +495 498 +495 499 +37 40 +37 41 +327 330 +327 331 +281 284 +281 285 +277 280 +277 281 +209 212 +209 213 +15 18 +15 19 +82 85 +82 86 +403 406 +403 407 +166 169 +166 170 +417 420 +417 421 +430 433 +430 434 +252 255 +252 256 +292 295 +292 296 +219 222 +219 223 +287 290 +287 291 +153 156 +153 157 +193 196 +193 197 +338 341 +338 342 +446 449 +446 450 +459 462 +459 463 +394 397 +394 398 +237 240 +237 241 +482 485 +482 486 +174 177 +174 178 +413 416 +413 417 +494 497 +494 498 +207 210 +207 211 +199 202 +199 203 +466 469 +466 470 +208 211 +208 212 +174 177 +174 178 +399 402 +399 403 +396 399 +396 400 +247 250 +247 251 +417 420 +417 421 +489 492 +489 493 +162 165 +162 166 +377 380 +377 381 +397 400 +397 401 +309 312 +309 313 +365 368 +365 369 +266 269 +266 270 +439 442 +439 443 +342 345 +342 346 +367 370 +367 371 +325 328 +325 329 +167 170 +167 171 +195 198 +195 199 +475 478 +475 479 +17 20 +17 21 +113 116 +113 117 +155 158 +155 159 +203 206 +203 207 +339 342 +339 343 +0 3 +0 4 +455 458 +455 459 +128 131 +128 132 +311 314 +311 315 +316 319 +316 320 +57 60 +57 61 +302 305 +302 306 +205 208 +205 209 +149 152 +149 153 +438 441 +438 442 +345 348 +345 349 +129 132 +129 133 +170 173 +170 174 +20 23 +20 24 +489 492 +489 493 +157 160 +157 161 +378 381 +378 382 +221 224 +221 225 +92 95 +92 96 +111 114 +111 115 +47 50 +47 51 +72 75 +72 76 +4 7 +4 8 +280 283 +280 284 +35 38 +35 39 +427 430 +427 431 +277 280 +277 281 +208 211 +208 212 +356 359 +356 360 +399 402 +399 403 +169 172 +169 173 +382 385 +382 386 +498 501 +498 502 +125 128 +125 129 +386 389 +386 390 +437 440 +437 441 +469 472 +469 473 +192 195 +192 196 +286 289 +286 290 +187 190 +187 191 +176 179 +176 180 +54 57 +54 58 +459 462 +459 463 +51 54 +51 55 +138 141 +138 142 +103 106 +103 107 +239 242 +239 243 +213 216 +213 217 +216 219 +216 220 +430 433 +430 434 +278 281 +278 282 +176 179 +176 180 +289 292 +289 293 +221 224 +221 225 +65 68 +65 69 +318 321 +318 322 +332 335 +332 336 +311 314 +311 315 +275 278 +275 279 +137 140 +137 141 +241 244 +241 245 +83 86 +83 87 +333 336 +333 337 +180 183 +180 184 +284 287 +284 288 +12 15 +12 16 +230 233 +230 234 +181 184 +181 185 +67 70 +67 71 +260 263 +260 264 +404 407 +404 408 +384 387 +384 388 +489 492 +489 493 +353 356 +353 357 +373 376 +373 377 +272 275 +272 276 +138 141 +138 142 +217 220 +217 221 +84 87 +84 88 +348 351 +348 352 +466 469 +466 470 +58 61 +58 62 +8 11 +8 12 +411 414 +411 415 +230 233 +230 234 +208 211 +208 212 +348 351 +348 352 +24 27 +24 28 +463 466 +463 467 +431 434 +431 435 +179 182 +179 183 +172 175 +172 176 +42 45 +42 46 +129 132 +129 133 +158 161 +158 162 +119 122 +119 123 +496 499 +496 500 +0 3 +0 4 +322 325 +322 326 +197 200 +197 201 +468 471 +468 472 +393 396 +393 397 +454 457 +454 458 +100 103 +100 104 +298 301 +298 302 +199 202 +199 203 +191 194 +191 195 +418 421 +418 422 +96 99 +96 100 +26 29 +26 30 +165 168 +165 169 +327 330 +327 331 +230 233 +230 234 +205 208 +205 209 +120 123 +120 124 +131 134 +131 135 +51 54 +51 55 +404 407 +404 408 +43 46 +43 47 +436 439 +436 440 +156 159 +156 160 +469 472 +469 473 +468 471 +468 472 +308 311 +308 312 +95 98 +95 99 +196 199 +196 200 +288 291 +288 292 +481 484 +481 485 +457 460 +457 461 +98 101 +98 102 +282 285 +282 286 +197 200 +197 201 +187 190 +187 191 +318 321 +318 322 +318 321 +318 322 +409 412 +409 413 +470 473 +470 474 +137 140 +137 141 +369 372 +369 373 +316 319 +316 320 +169 172 +169 173 +413 416 +413 417 +85 88 +85 89 +77 80 +77 81 +0 3 +0 4 +490 493 +490 494 +87 90 +87 91 +364 367 +364 368 +179 182 +179 183 +118 121 +118 122 +134 137 +134 138 +395 398 +395 399 +282 285 +282 286 +138 141 +138 142 +238 241 +238 242 +419 422 +419 423 +15 18 +15 19 +118 121 +118 122 +72 75 +72 76 +90 93 +90 94 +307 310 +307 311 +19 22 +19 23 +435 438 +435 439 +10 13 +10 14 +277 280 +277 281 +273 276 +273 277 +306 309 +306 310 +224 227 +224 228 +309 312 +309 313 +389 392 +389 393 +327 330 +327 331 +242 245 +242 246 +369 372 +369 373 +392 395 +392 396 +272 275 +272 276 +331 334 +331 335 +401 404 +401 405 +242 245 +242 246 +452 455 +452 456 +177 180 +177 181 +226 229 +226 230 +5 8 +5 9 +497 500 +497 501 +402 405 +402 406 +396 399 +396 400 +317 320 +317 321 +395 398 +395 399 +58 61 +58 62 +35 38 +35 39 +336 339 +336 340 +95 98 +95 99 +11 14 +11 15 +168 171 +168 172 +34 37 +34 38 +229 232 +229 233 +233 236 +233 237 +143 146 +143 147 +472 475 +472 476 +322 325 +322 326 +498 501 +498 502 +160 163 +160 164 +195 198 +195 199 +42 45 +42 46 +321 324 +321 325 +430 433 +430 434 +119 122 +119 123 +489 492 +489 493 +458 461 +458 462 +78 81 +78 82 +76 79 +76 80 +41 44 +41 45 +223 226 +223 227 +492 495 +492 496 +149 152 +149 153 +449 452 +449 453 +218 221 +218 222 +228 231 +228 232 +138 141 +138 142 +453 456 +453 457 +30 33 +30 34 +209 212 +209 213 +64 67 +64 68 +468 471 +468 472 +76 79 +76 80 +74 77 +74 78 +342 345 +342 346 +69 72 +69 73 +230 233 +230 234 +33 36 +33 37 +368 371 +368 372 +103 106 +103 107 +296 299 +296 300 +113 116 +113 117 +216 219 +216 220 +367 370 +367 371 +344 347 +344 348 +167 170 +167 171 +274 277 +274 278 +219 222 +219 223 +239 242 +239 243 +485 488 +485 489 +116 119 +116 120 +223 226 +223 227 +256 259 +256 260 +263 266 +263 267 +70 73 +70 74 +487 490 +487 491 +480 483 +480 484 +401 404 +401 405 +288 291 +288 292 +191 194 +191 195 +5 8 +5 9 +244 247 +244 248 +438 441 +438 442 +128 131 +128 132 +467 470 +467 471 +432 435 +432 436 +202 205 +202 206 +316 319 +316 320 +229 232 +229 233 +469 472 +469 473 +463 466 +463 467 +280 283 +280 284 +2 5 +2 6 +35 38 +35 39 +283 286 +283 287 +331 334 +331 335 +235 238 +235 239 +80 83 +80 84 +44 47 +44 48 +193 196 +193 197 +321 324 +321 325 +335 338 +335 339 +104 107 +104 108 +466 469 +466 470 +366 369 +366 370 +175 178 +175 179 +403 406 +403 407 +483 486 +483 487 +53 56 +53 57 +105 108 +105 109 +257 260 +257 261 +406 409 +406 410 +409 412 +409 413 +190 193 +190 194 +406 409 +406 410 +401 404 +401 405 +114 117 +114 118 +258 261 +258 262 +90 93 +90 94 +203 206 +203 207 +262 265 +262 266 +348 351 +348 352 +424 427 +424 428 +12 15 +12 16 +396 399 +396 400 +201 204 +201 205 +217 220 +217 221 +164 167 +164 168 +431 434 +431 435 +454 457 +454 458 +478 481 +478 482 +298 301 +298 302 +125 128 +125 129 +431 434 +431 435 +164 167 +164 168 +424 427 +424 428 +187 190 +187 191 +382 385 +382 386 +5 8 +5 9 +70 73 +70 74 +397 400 +397 401 +480 483 +480 484 +291 294 +291 295 +24 27 +24 28 +351 354 +351 355 +255 258 +255 259 +104 107 +104 108 +70 73 +70 74 +163 166 +163 167 +438 441 +438 442 +119 122 +119 123 +414 417 +414 418 +200 203 +200 204 +491 494 +491 495 +237 240 +237 241 +439 442 +439 443 +360 363 +360 364 +248 251 +248 252 +479 482 +479 483 +305 308 +305 309 +417 420 +417 421 +199 202 +199 203 +444 447 +444 448 +120 123 +120 124 +429 432 +429 433 +169 172 +169 173 +443 446 +443 447 +323 326 +323 327 +325 328 +325 329 +277 280 +277 281 +230 233 +230 234 +478 481 +478 482 +178 181 +178 182 +468 471 +468 472 +310 313 +310 314 +317 320 +317 321 +333 336 +333 337 +493 496 +493 497 +460 463 +460 464 +207 210 +207 211 +249 252 +249 253 +265 268 +265 269 +480 483 +480 484 +83 86 +83 87 +136 139 +136 140 +353 356 +353 357 +172 175 +172 176 +214 217 +214 218 +462 465 +462 466 +233 236 +233 237 +406 409 +406 410 +133 136 +133 137 +175 178 +175 179 +189 192 +189 193 +454 457 +454 458 +375 378 +375 379 +401 404 +401 405 +421 424 +421 425 +407 410 +407 411 +384 387 +384 388 +256 259 +256 260 +26 29 +26 30 +134 137 +134 138 +67 70 +67 71 +384 387 +384 388 +379 382 +379 383 +18 21 +18 22 +462 465 +462 466 +492 495 +492 496 +100 103 +100 104 +298 301 +298 302 +9 12 +9 13 +341 344 +341 345 +498 501 +498 502 +146 149 +146 150 +458 461 +458 462 +362 365 +362 366 +186 189 +186 190 +285 288 +285 289 +348 351 +348 352 +167 170 +167 171 +18 21 +18 22 +273 276 +273 277 +183 186 +183 187 +281 284 +281 285 +344 347 +344 348 +97 100 +97 101 +469 472 +469 473 +315 318 +315 319 +84 87 +84 88 +28 31 +28 32 +37 40 +37 41 +448 451 +448 452 +152 155 +152 156 +348 351 +348 352 +307 310 +307 311 +194 197 +194 198 +414 417 +414 418 +477 480 +477 481 +222 225 +222 226 +126 129 +126 130 +90 93 +90 94 +169 172 +169 173 +403 406 +403 407 +400 403 +400 404 +200 203 +200 204 +97 100 +97 101 diff --git a/src/test/hive/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 b/src/test/hive/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 b/src/test/hive/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e b/src/test/hive/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b b/src/test/hive/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b new file mode 100644 index 0000000000000..4895558ca6142 --- /dev/null +++ b/src/test/hive/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b @@ -0,0 +1,1000 @@ +241 +242 +89 +90 +314 +315 +30 +31 +168 +169 +412 +413 +258 +259 +281 +282 +101 +102 +487 +488 +268 +269 +196 +197 +404 +405 +153 +154 +276 +277 +227 +228 +372 +373 +69 +70 +131 +132 +216 +217 +149 +150 +409 +410 +432 +433 +377 +378 +155 +156 +472 +473 +148 +149 +498 +499 +40 +41 +330 +331 +284 +285 +280 +281 +212 +213 +18 +19 +85 +86 +406 +407 +169 +170 +420 +421 +433 +434 +255 +256 +295 +296 +222 +223 +290 +291 +156 +157 +196 +197 +341 +342 +449 +450 +462 +463 +397 +398 +240 +241 +485 +486 +177 +178 +416 +417 +497 +498 +210 +211 +202 +203 +469 +470 +211 +212 +177 +178 +402 +403 +399 +400 +250 +251 +420 +421 +492 +493 +165 +166 +380 +381 +400 +401 +312 +313 +368 +369 +269 +270 +442 +443 +345 +346 +370 +371 +328 +329 +170 +171 +198 +199 +478 +479 +20 +21 +116 +117 +158 +159 +206 +207 +342 +343 +3 +4 +458 +459 +131 +132 +314 +315 +319 +320 +60 +61 +305 +306 +208 +209 +152 +153 +441 +442 +348 +349 +132 +133 +173 +174 +23 +24 +492 +493 +160 +161 +381 +382 +224 +225 +95 +96 +114 +115 +50 +51 +75 +76 +7 +8 +283 +284 +38 +39 +430 +431 +280 +281 +211 +212 +359 +360 +402 +403 +172 +173 +385 +386 +501 +502 +128 +129 +389 +390 +440 +441 +472 +473 +195 +196 +289 +290 +190 +191 +179 +180 +57 +58 +462 +463 +54 +55 +141 +142 +106 +107 +242 +243 +216 +217 +219 +220 +433 +434 +281 +282 +179 +180 +292 +293 +224 +225 +68 +69 +321 +322 +335 +336 +314 +315 +278 +279 +140 +141 +244 +245 +86 +87 +336 +337 +183 +184 +287 +288 +15 +16 +233 +234 +184 +185 +70 +71 +263 +264 +407 +408 +387 +388 +492 +493 +356 +357 +376 +377 +275 +276 +141 +142 +220 +221 +87 +88 +351 +352 +469 +470 +61 +62 +11 +12 +414 +415 +233 +234 +211 +212 +351 +352 +27 +28 +466 +467 +434 +435 +182 +183 +175 +176 +45 +46 +132 +133 +161 +162 +122 +123 +499 +500 +3 +4 +325 +326 +200 +201 +471 +472 +396 +397 +457 +458 +103 +104 +301 +302 +202 +203 +194 +195 +421 +422 +99 +100 +29 +30 +168 +169 +330 +331 +233 +234 +208 +209 +123 +124 +134 +135 +54 +55 +407 +408 +46 +47 +439 +440 +159 +160 +472 +473 +471 +472 +311 +312 +98 +99 +199 +200 +291 +292 +484 +485 +460 +461 +101 +102 +285 +286 +200 +201 +190 +191 +321 +322 +321 +322 +412 +413 +473 +474 +140 +141 +372 +373 +319 +320 +172 +173 +416 +417 +88 +89 +80 +81 +3 +4 +493 +494 +90 +91 +367 +368 +182 +183 +121 +122 +137 +138 +398 +399 +285 +286 +141 +142 +241 +242 +422 +423 +18 +19 +121 +122 +75 +76 +93 +94 +310 +311 +22 +23 +438 +439 +13 +14 +280 +281 +276 +277 +309 +310 +227 +228 +312 +313 +392 +393 +330 +331 +245 +246 +372 +373 +395 +396 +275 +276 +334 +335 +404 +405 +245 +246 +455 +456 +180 +181 +229 +230 +8 +9 +500 +501 +405 +406 +399 +400 +320 +321 +398 +399 +61 +62 +38 +39 +339 +340 +98 +99 +14 +15 +171 +172 +37 +38 +232 +233 +236 +237 +146 +147 +475 +476 +325 +326 +501 +502 +163 +164 +198 +199 +45 +46 +324 +325 +433 +434 +122 +123 +492 +493 +461 +462 +81 +82 +79 +80 +44 +45 +226 +227 +495 +496 +152 +153 +452 +453 +221 +222 +231 +232 +141 +142 +456 +457 +33 +34 +212 +213 +67 +68 +471 +472 +79 +80 +77 +78 +345 +346 +72 +73 +233 +234 +36 +37 +371 +372 +106 +107 +299 +300 +116 +117 +219 +220 +370 +371 +347 +348 +170 +171 +277 +278 +222 +223 +242 +243 +488 +489 +119 +120 +226 +227 +259 +260 +266 +267 +73 +74 +490 +491 +483 +484 +404 +405 +291 +292 +194 +195 +8 +9 +247 +248 +441 +442 +131 +132 +470 +471 +435 +436 +205 +206 +319 +320 +232 +233 +472 +473 +466 +467 +283 +284 +5 +6 +38 +39 +286 +287 +334 +335 +238 +239 +83 +84 +47 +48 +196 +197 +324 +325 +338 +339 +107 +108 +469 +470 +369 +370 +178 +179 +406 +407 +486 +487 +56 +57 +108 +109 +260 +261 +409 +410 +412 +413 +193 +194 +409 +410 +404 +405 +117 +118 +261 +262 +93 +94 +206 +207 +265 +266 +351 +352 +427 +428 +15 +16 +399 +400 +204 +205 +220 +221 +167 +168 +434 +435 +457 +458 +481 +482 +301 +302 +128 +129 +434 +435 +167 +168 +427 +428 +190 +191 +385 +386 +8 +9 +73 +74 +400 +401 +483 +484 +294 +295 +27 +28 +354 +355 +258 +259 +107 +108 +73 +74 +166 +167 +441 +442 +122 +123 +417 +418 +203 +204 +494 +495 +240 +241 +442 +443 +363 +364 +251 +252 +482 +483 +308 +309 +420 +421 +202 +203 +447 +448 +123 +124 +432 +433 +172 +173 +446 +447 +326 +327 +328 +329 +280 +281 +233 +234 +481 +482 +181 +182 +471 +472 +313 +314 +320 +321 +336 +337 +496 +497 +463 +464 +210 +211 +252 +253 +268 +269 +483 +484 +86 +87 +139 +140 +356 +357 +175 +176 +217 +218 +465 +466 +236 +237 +409 +410 +136 +137 +178 +179 +192 +193 +457 +458 +378 +379 +404 +405 +424 +425 +410 +411 +387 +388 +259 +260 +29 +30 +137 +138 +70 +71 +387 +388 +382 +383 +21 +22 +465 +466 +495 +496 +103 +104 +301 +302 +12 +13 +344 +345 +501 +502 +149 +150 +461 +462 +365 +366 +189 +190 +288 +289 +351 +352 +170 +171 +21 +22 +276 +277 +186 +187 +284 +285 +347 +348 +100 +101 +472 +473 +318 +319 +87 +88 +31 +32 +40 +41 +451 +452 +155 +156 +351 +352 +310 +311 +197 +198 +417 +418 +480 +481 +225 +226 +129 +130 +93 +94 +172 +173 +406 +407 +403 +404 +203 +204 +100 +101 diff --git a/src/test/hive/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a b/src/test/hive/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a new file mode 100644 index 0000000000000..269df08795dc9 --- /dev/null +++ b/src/test/hive/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a @@ -0,0 +1,500 @@ +238 val_238 241 242 +86 val_86 89 90 +311 val_311 314 315 +27 val_27 30 31 +165 val_165 168 169 +409 val_409 412 413 +255 val_255 258 259 +278 val_278 281 282 +98 val_98 101 102 +484 val_484 487 488 +265 val_265 268 269 +193 val_193 196 197 +401 val_401 404 405 +150 val_150 153 154 +273 val_273 276 277 +224 val_224 227 228 +369 val_369 372 373 +66 val_66 69 70 +128 val_128 131 132 +213 val_213 216 217 +146 val_146 149 150 +406 val_406 409 410 +429 val_429 432 433 +374 val_374 377 378 +152 val_152 155 156 +469 val_469 472 473 +145 val_145 148 149 +495 val_495 498 499 +37 val_37 40 41 +327 val_327 330 331 +281 val_281 284 285 +277 val_277 280 281 +209 val_209 212 213 +15 val_15 18 19 +82 val_82 85 86 +403 val_403 406 407 +166 val_166 169 170 +417 val_417 420 421 +430 val_430 433 434 +252 val_252 255 256 +292 val_292 295 296 +219 val_219 222 223 +287 val_287 290 291 +153 val_153 156 157 +193 val_193 196 197 +338 val_338 341 342 +446 val_446 449 450 +459 val_459 462 463 +394 val_394 397 398 +237 val_237 240 241 +482 val_482 485 486 +174 val_174 177 178 +413 val_413 416 417 +494 val_494 497 498 +207 val_207 210 211 +199 val_199 202 203 +466 val_466 469 470 +208 val_208 211 212 +174 val_174 177 178 +399 val_399 402 403 +396 val_396 399 400 +247 val_247 250 251 +417 val_417 420 421 +489 val_489 492 493 +162 val_162 165 166 +377 val_377 380 381 +397 val_397 400 401 +309 val_309 312 313 +365 val_365 368 369 +266 val_266 269 270 +439 val_439 442 443 +342 val_342 345 346 +367 val_367 370 371 +325 val_325 328 329 +167 val_167 170 171 +195 val_195 198 199 +475 val_475 478 479 +17 val_17 20 21 +113 val_113 116 117 +155 val_155 158 159 +203 val_203 206 207 +339 val_339 342 343 +0 val_0 3 4 +455 val_455 458 459 +128 val_128 131 132 +311 val_311 314 315 +316 val_316 319 320 +57 val_57 60 61 +302 val_302 305 306 +205 val_205 208 209 +149 val_149 152 153 +438 val_438 441 442 +345 val_345 348 349 +129 val_129 132 133 +170 val_170 173 174 +20 val_20 23 24 +489 val_489 492 493 +157 val_157 160 161 +378 val_378 381 382 +221 val_221 224 225 +92 val_92 95 96 +111 val_111 114 115 +47 val_47 50 51 +72 val_72 75 76 +4 val_4 7 8 +280 val_280 283 284 +35 val_35 38 39 +427 val_427 430 431 +277 val_277 280 281 +208 val_208 211 212 +356 val_356 359 360 +399 val_399 402 403 +169 val_169 172 173 +382 val_382 385 386 +498 val_498 501 502 +125 val_125 128 129 +386 val_386 389 390 +437 val_437 440 441 +469 val_469 472 473 +192 val_192 195 196 +286 val_286 289 290 +187 val_187 190 191 +176 val_176 179 180 +54 val_54 57 58 +459 val_459 462 463 +51 val_51 54 55 +138 val_138 141 142 +103 val_103 106 107 +239 val_239 242 243 +213 val_213 216 217 +216 val_216 219 220 +430 val_430 433 434 +278 val_278 281 282 +176 val_176 179 180 +289 val_289 292 293 +221 val_221 224 225 +65 val_65 68 69 +318 val_318 321 322 +332 val_332 335 336 +311 val_311 314 315 +275 val_275 278 279 +137 val_137 140 141 +241 val_241 244 245 +83 val_83 86 87 +333 val_333 336 337 +180 val_180 183 184 +284 val_284 287 288 +12 val_12 15 16 +230 val_230 233 234 +181 val_181 184 185 +67 val_67 70 71 +260 val_260 263 264 +404 val_404 407 408 +384 val_384 387 388 +489 val_489 492 493 +353 val_353 356 357 +373 val_373 376 377 +272 val_272 275 276 +138 val_138 141 142 +217 val_217 220 221 +84 val_84 87 88 +348 val_348 351 352 +466 val_466 469 470 +58 val_58 61 62 +8 val_8 11 12 +411 val_411 414 415 +230 val_230 233 234 +208 val_208 211 212 +348 val_348 351 352 +24 val_24 27 28 +463 val_463 466 467 +431 val_431 434 435 +179 val_179 182 183 +172 val_172 175 176 +42 val_42 45 46 +129 val_129 132 133 +158 val_158 161 162 +119 val_119 122 123 +496 val_496 499 500 +0 val_0 3 4 +322 val_322 325 326 +197 val_197 200 201 +468 val_468 471 472 +393 val_393 396 397 +454 val_454 457 458 +100 val_100 103 104 +298 val_298 301 302 +199 val_199 202 203 +191 val_191 194 195 +418 val_418 421 422 +96 val_96 99 100 +26 val_26 29 30 +165 val_165 168 169 +327 val_327 330 331 +230 val_230 233 234 +205 val_205 208 209 +120 val_120 123 124 +131 val_131 134 135 +51 val_51 54 55 +404 val_404 407 408 +43 val_43 46 47 +436 val_436 439 440 +156 val_156 159 160 +469 val_469 472 473 +468 val_468 471 472 +308 val_308 311 312 +95 val_95 98 99 +196 val_196 199 200 +288 val_288 291 292 +481 val_481 484 485 +457 val_457 460 461 +98 val_98 101 102 +282 val_282 285 286 +197 val_197 200 201 +187 val_187 190 191 +318 val_318 321 322 +318 val_318 321 322 +409 val_409 412 413 +470 val_470 473 474 +137 val_137 140 141 +369 val_369 372 373 +316 val_316 319 320 +169 val_169 172 173 +413 val_413 416 417 +85 val_85 88 89 +77 val_77 80 81 +0 val_0 3 4 +490 val_490 493 494 +87 val_87 90 91 +364 val_364 367 368 +179 val_179 182 183 +118 val_118 121 122 +134 val_134 137 138 +395 val_395 398 399 +282 val_282 285 286 +138 val_138 141 142 +238 val_238 241 242 +419 val_419 422 423 +15 val_15 18 19 +118 val_118 121 122 +72 val_72 75 76 +90 val_90 93 94 +307 val_307 310 311 +19 val_19 22 23 +435 val_435 438 439 +10 val_10 13 14 +277 val_277 280 281 +273 val_273 276 277 +306 val_306 309 310 +224 val_224 227 228 +309 val_309 312 313 +389 val_389 392 393 +327 val_327 330 331 +242 val_242 245 246 +369 val_369 372 373 +392 val_392 395 396 +272 val_272 275 276 +331 val_331 334 335 +401 val_401 404 405 +242 val_242 245 246 +452 val_452 455 456 +177 val_177 180 181 +226 val_226 229 230 +5 val_5 8 9 +497 val_497 500 501 +402 val_402 405 406 +396 val_396 399 400 +317 val_317 320 321 +395 val_395 398 399 +58 val_58 61 62 +35 val_35 38 39 +336 val_336 339 340 +95 val_95 98 99 +11 val_11 14 15 +168 val_168 171 172 +34 val_34 37 38 +229 val_229 232 233 +233 val_233 236 237 +143 val_143 146 147 +472 val_472 475 476 +322 val_322 325 326 +498 val_498 501 502 +160 val_160 163 164 +195 val_195 198 199 +42 val_42 45 46 +321 val_321 324 325 +430 val_430 433 434 +119 val_119 122 123 +489 val_489 492 493 +458 val_458 461 462 +78 val_78 81 82 +76 val_76 79 80 +41 val_41 44 45 +223 val_223 226 227 +492 val_492 495 496 +149 val_149 152 153 +449 val_449 452 453 +218 val_218 221 222 +228 val_228 231 232 +138 val_138 141 142 +453 val_453 456 457 +30 val_30 33 34 +209 val_209 212 213 +64 val_64 67 68 +468 val_468 471 472 +76 val_76 79 80 +74 val_74 77 78 +342 val_342 345 346 +69 val_69 72 73 +230 val_230 233 234 +33 val_33 36 37 +368 val_368 371 372 +103 val_103 106 107 +296 val_296 299 300 +113 val_113 116 117 +216 val_216 219 220 +367 val_367 370 371 +344 val_344 347 348 +167 val_167 170 171 +274 val_274 277 278 +219 val_219 222 223 +239 val_239 242 243 +485 val_485 488 489 +116 val_116 119 120 +223 val_223 226 227 +256 val_256 259 260 +263 val_263 266 267 +70 val_70 73 74 +487 val_487 490 491 +480 val_480 483 484 +401 val_401 404 405 +288 val_288 291 292 +191 val_191 194 195 +5 val_5 8 9 +244 val_244 247 248 +438 val_438 441 442 +128 val_128 131 132 +467 val_467 470 471 +432 val_432 435 436 +202 val_202 205 206 +316 val_316 319 320 +229 val_229 232 233 +469 val_469 472 473 +463 val_463 466 467 +280 val_280 283 284 +2 val_2 5 6 +35 val_35 38 39 +283 val_283 286 287 +331 val_331 334 335 +235 val_235 238 239 +80 val_80 83 84 +44 val_44 47 48 +193 val_193 196 197 +321 val_321 324 325 +335 val_335 338 339 +104 val_104 107 108 +466 val_466 469 470 +366 val_366 369 370 +175 val_175 178 179 +403 val_403 406 407 +483 val_483 486 487 +53 val_53 56 57 +105 val_105 108 109 +257 val_257 260 261 +406 val_406 409 410 +409 val_409 412 413 +190 val_190 193 194 +406 val_406 409 410 +401 val_401 404 405 +114 val_114 117 118 +258 val_258 261 262 +90 val_90 93 94 +203 val_203 206 207 +262 val_262 265 266 +348 val_348 351 352 +424 val_424 427 428 +12 val_12 15 16 +396 val_396 399 400 +201 val_201 204 205 +217 val_217 220 221 +164 val_164 167 168 +431 val_431 434 435 +454 val_454 457 458 +478 val_478 481 482 +298 val_298 301 302 +125 val_125 128 129 +431 val_431 434 435 +164 val_164 167 168 +424 val_424 427 428 +187 val_187 190 191 +382 val_382 385 386 +5 val_5 8 9 +70 val_70 73 74 +397 val_397 400 401 +480 val_480 483 484 +291 val_291 294 295 +24 val_24 27 28 +351 val_351 354 355 +255 val_255 258 259 +104 val_104 107 108 +70 val_70 73 74 +163 val_163 166 167 +438 val_438 441 442 +119 val_119 122 123 +414 val_414 417 418 +200 val_200 203 204 +491 val_491 494 495 +237 val_237 240 241 +439 val_439 442 443 +360 val_360 363 364 +248 val_248 251 252 +479 val_479 482 483 +305 val_305 308 309 +417 val_417 420 421 +199 val_199 202 203 +444 val_444 447 448 +120 val_120 123 124 +429 val_429 432 433 +169 val_169 172 173 +443 val_443 446 447 +323 val_323 326 327 +325 val_325 328 329 +277 val_277 280 281 +230 val_230 233 234 +478 val_478 481 482 +178 val_178 181 182 +468 val_468 471 472 +310 val_310 313 314 +317 val_317 320 321 +333 val_333 336 337 +493 val_493 496 497 +460 val_460 463 464 +207 val_207 210 211 +249 val_249 252 253 +265 val_265 268 269 +480 val_480 483 484 +83 val_83 86 87 +136 val_136 139 140 +353 val_353 356 357 +172 val_172 175 176 +214 val_214 217 218 +462 val_462 465 466 +233 val_233 236 237 +406 val_406 409 410 +133 val_133 136 137 +175 val_175 178 179 +189 val_189 192 193 +454 val_454 457 458 +375 val_375 378 379 +401 val_401 404 405 +421 val_421 424 425 +407 val_407 410 411 +384 val_384 387 388 +256 val_256 259 260 +26 val_26 29 30 +134 val_134 137 138 +67 val_67 70 71 +384 val_384 387 388 +379 val_379 382 383 +18 val_18 21 22 +462 val_462 465 466 +492 val_492 495 496 +100 val_100 103 104 +298 val_298 301 302 +9 val_9 12 13 +341 val_341 344 345 +498 val_498 501 502 +146 val_146 149 150 +458 val_458 461 462 +362 val_362 365 366 +186 val_186 189 190 +285 val_285 288 289 +348 val_348 351 352 +167 val_167 170 171 +18 val_18 21 22 +273 val_273 276 277 +183 val_183 186 187 +281 val_281 284 285 +344 val_344 347 348 +97 val_97 100 101 +469 val_469 472 473 +315 val_315 318 319 +84 val_84 87 88 +28 val_28 31 32 +37 val_37 40 41 +448 val_448 451 452 +152 val_152 155 156 +348 val_348 351 352 +307 val_307 310 311 +194 val_194 197 198 +414 val_414 417 418 +477 val_477 480 481 +222 val_222 225 226 +126 val_126 129 130 +90 val_90 93 94 +169 val_169 172 173 +403 val_403 406 407 +400 val_400 403 404 +200 val_200 203 204 +97 val_97 100 101 diff --git a/src/test/hive/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b b/src/test/hive/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 b/src/test/hive/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e b/src/test/hive/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 b/src/test/hive/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 new file mode 100644 index 0000000000000..01e79c32a8c99 --- /dev/null +++ b/src/test/hive/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/src/test/hive/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 b/src/test/hive/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba b/src/test/hive/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 b/src/test/hive/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 new file mode 100644 index 0000000000000..fd859feff393b --- /dev/null +++ b/src/test/hive/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 @@ -0,0 +1,20 @@ +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"] diff --git a/src/test/hive/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e b/src/test/hive/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e new file mode 100644 index 0000000000000..512cb498a2f6a --- /dev/null +++ b/src/test/hive/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e @@ -0,0 +1,20 @@ +val_0 +val_0 +val_0 +val_2 +val_4 +val_5 +val_5 +val_5 +val_8 +val_9 +val_10 +val_11 +val_12 +val_12 +val_15 +val_15 +val_17 +val_18 +val_18 +val_19 diff --git a/src/test/hive/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 b/src/test/hive/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 new file mode 100644 index 0000000000000..1f752ac7f4a39 --- /dev/null +++ b/src/test/hive/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 @@ -0,0 +1,20 @@ +0 ["val_0"] val_0 +0 ["val_0"] val_0 +0 ["val_0"] val_0 +2 ["val_2"] val_2 +4 ["val_4"] val_4 +5 ["val_5"] val_5 +5 ["val_5"] val_5 +5 ["val_5"] val_5 +8 ["val_8"] val_8 +9 ["val_9"] val_9 +10 ["val_10"] val_10 +11 ["val_11"] val_11 +12 ["val_12"] val_12 +12 ["val_12"] val_12 +15 ["val_15"] val_15 +15 ["val_15"] val_15 +17 ["val_17"] val_17 +18 ["val_18"] val_18 +18 ["val_18"] val_18 +19 ["val_19"] val_19 diff --git a/src/test/hive/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab b/src/test/hive/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab new file mode 100644 index 0000000000000..fd859feff393b --- /dev/null +++ b/src/test/hive/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab @@ -0,0 +1,20 @@ +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"] diff --git a/src/test/hive/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e b/src/test/hive/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e new file mode 100644 index 0000000000000..512cb498a2f6a --- /dev/null +++ b/src/test/hive/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e @@ -0,0 +1,20 @@ +val_0 +val_0 +val_0 +val_2 +val_4 +val_5 +val_5 +val_5 +val_8 +val_9 +val_10 +val_11 +val_12 +val_12 +val_15 +val_15 +val_17 +val_18 +val_18 +val_19 diff --git a/src/test/hive/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 b/src/test/hive/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 new file mode 100644 index 0000000000000..f372b042130e7 --- /dev/null +++ b/src/test/hive/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 @@ -0,0 +1,20 @@ +0 +0 +0 +2 +4 +5 +5 +5 +8 +9 +10 +11 +12 +12 +15 +15 +17 +18 +18 +19 diff --git a/src/test/hive/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 b/src/test/hive/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 b/src/test/hive/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 b/src/test/hive/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 new file mode 100644 index 0000000000000..30e4fa100f778 --- /dev/null +++ b/src/test/hive/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 @@ -0,0 +1,13 @@ +["val_0"] val_0 +["val_10"] val_10 +["val_11"] val_11 +["val_12"] val_12 +["val_15"] val_15 +["val_17"] val_17 +["val_18"] val_18 +["val_19"] val_19 +["val_2"] val_2 +["val_4"] val_4 +["val_5"] val_5 +["val_8"] val_8 +["val_9"] val_9 diff --git a/src/test/hive/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e b/src/test/hive/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 b/src/test/hive/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae b/src/test/hive/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 b/src/test/hive/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 new file mode 100644 index 0000000000000..d6426b33c4d5d --- /dev/null +++ b/src/test/hive/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 @@ -0,0 +1 @@ +0 val_0 1 diff --git a/src/test/hive/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 b/src/test/hive/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 new file mode 100644 index 0000000000000..01e79c32a8c99 --- /dev/null +++ b/src/test/hive/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/src/test/hive/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 b/src/test/hive/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 new file mode 100644 index 0000000000000..423bb3e88a24e --- /dev/null +++ b/src/test/hive/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 @@ -0,0 +1,9 @@ +1 a +1 b +1 c +2 a +2 b +2 c +3 a +3 b +3 c diff --git a/src/test/hive/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 b/src/test/hive/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 new file mode 100644 index 0000000000000..01e79c32a8c99 --- /dev/null +++ b/src/test/hive/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/src/test/hive/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 b/src/test/hive/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 b/src/test/hive/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 b/src/test/hive/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 b/src/test/hive/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_cp-4-507876b610812506343e9d251645170b b/src/test/hive/golden/lateral_view_cp-4-507876b610812506343e9d251645170b new file mode 100644 index 0000000000000..1f3d8a7a1fc08 --- /dev/null +++ b/src/test/hive/golden/lateral_view_cp-4-507876b610812506343e9d251645170b @@ -0,0 +1 @@ +1028 diff --git a/src/test/hive/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 b/src/test/hive/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 b/src/test/hive/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 new file mode 100644 index 0000000000000..0da0d93886e01 --- /dev/null +++ b/src/test/hive/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 @@ -0,0 +1,2 @@ +key1 100 +key2 200 diff --git a/src/test/hive/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a b/src/test/hive/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 b/src/test/hive/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 new file mode 100644 index 0000000000000..0da0d93886e01 --- /dev/null +++ b/src/test/hive/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 @@ -0,0 +1,2 @@ +key1 100 +key2 200 diff --git a/src/test/hive/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe b/src/test/hive/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 b/src/test/hive/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 b/src/test/hive/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa b/src/test/hive/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa new file mode 100644 index 0000000000000..a2a7fdd233a2a --- /dev/null +++ b/src/test/hive/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa @@ -0,0 +1,10 @@ +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/src/test/hive/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 b/src/test/hive/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d b/src/test/hive/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d new file mode 100644 index 0000000000000..37d918a386d7d --- /dev/null +++ b/src/test/hive/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d @@ -0,0 +1,10 @@ +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/src/test/hive/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e b/src/test/hive/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 b/src/test/hive/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 b/src/test/hive/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 new file mode 100644 index 0000000000000..3ad56f646ed85 --- /dev/null +++ b/src/test/hive/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 b/src/test/hive/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f b/src/test/hive/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f new file mode 100644 index 0000000000000..f0a72370f697b --- /dev/null +++ b/src/test/hive/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f @@ -0,0 +1,9 @@ +val_0 1 +val_0 2 +val_0 3 +val_0 1 +val_0 2 +val_0 3 +val_0 1 +val_0 2 +val_0 3 diff --git a/src/test/hive/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 b/src/test/hive/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 b/src/test/hive/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 new file mode 100644 index 0000000000000..1c7ec3df89594 --- /dev/null +++ b/src/test/hive/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 @@ -0,0 +1,3 @@ +val_0 1 +val_0 1 +val_0 1 diff --git a/src/test/hive/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f b/src/test/hive/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa b/src/test/hive/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa new file mode 100644 index 0000000000000..3c52da67708d8 --- /dev/null +++ b/src/test/hive/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa @@ -0,0 +1,12 @@ +val_238 1 +val_238 2 +val_238 3 +val_86 1 +val_86 2 +val_86 3 +val_311 1 +val_311 2 +val_311 3 +val_27 1 +val_27 2 +val_27 3 diff --git a/src/test/hive/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b b/src/test/hive/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 b/src/test/hive/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 new file mode 100644 index 0000000000000..78b21ab365aac --- /dev/null +++ b/src/test/hive/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 @@ -0,0 +1,27 @@ +val_0 1 +val_0 1 +val_0 1 +val_0 2 +val_0 2 +val_0 2 +val_0 3 +val_0 3 +val_0 3 +val_0 1 +val_0 1 +val_0 1 +val_0 2 +val_0 2 +val_0 2 +val_0 3 +val_0 3 +val_0 3 +val_0 1 +val_0 1 +val_0 1 +val_0 2 +val_0 2 +val_0 2 +val_0 3 +val_0 3 +val_0 3 diff --git a/src/test/hive/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 b/src/test/hive/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 new file mode 100644 index 0000000000000..bf0d87ab1b2b0 --- /dev/null +++ b/src/test/hive/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 @@ -0,0 +1 @@ +4 \ No newline at end of file diff --git a/src/test/hive/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb b/src/test/hive/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 b/src/test/hive/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 b/src/test/hive/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f b/src/test/hive/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f new file mode 100644 index 0000000000000..9e3b31ad52c13 --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f @@ -0,0 +1,20 @@ +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/src/test/hive/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 b/src/test/hive/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 b/src/test/hive/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 new file mode 100644 index 0000000000000..ebf6c0424c26d --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 @@ -0,0 +1,20 @@ +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/src/test/hive/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 b/src/test/hive/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 b/src/test/hive/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 b/src/test/hive/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 b/src/test/hive/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 new file mode 100644 index 0000000000000..153dcec21bc5b --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 @@ -0,0 +1,20 @@ +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/src/test/hive/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 b/src/test/hive/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 b/src/test/hive/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb b/src/test/hive/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 b/src/test/hive/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 new file mode 100644 index 0000000000000..ae8f0265b71ca --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 @@ -0,0 +1,20 @@ +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/src/test/hive/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e b/src/test/hive/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a b/src/test/hive/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 b/src/test/hive/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 new file mode 100644 index 0000000000000..e604892422d59 --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 @@ -0,0 +1,100 @@ +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/src/test/hive/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a b/src/test/hive/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c b/src/test/hive/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c new file mode 100644 index 0000000000000..92dc6ce9dbf9e --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c @@ -0,0 +1,100 @@ +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/src/test/hive/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 b/src/test/hive/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 new file mode 100644 index 0000000000000..95f5492558a9b --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 @@ -0,0 +1,20 @@ +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/src/test/hive/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f b/src/test/hive/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 b/src/test/hive/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 new file mode 100644 index 0000000000000..0bac402cfa497 --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 @@ -0,0 +1,20 @@ +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/src/test/hive/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 b/src/test/hive/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 b/src/test/hive/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 new file mode 100644 index 0000000000000..b57d0cc951566 --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 @@ -0,0 +1,20 @@ +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/src/test/hive/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 b/src/test/hive/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a b/src/test/hive/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a new file mode 100644 index 0000000000000..0e95c446f3b0d --- /dev/null +++ b/src/test/hive/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a @@ -0,0 +1,20 @@ +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/src/test/hive/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 b/src/test/hive/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 b/src/test/hive/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c b/src/test/hive/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/literal_double-0-10ef1098e35d900983be3814de8f974f b/src/test/hive/golden/literal_double-0-10ef1098e35d900983be3814de8f974f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 b/src/test/hive/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 new file mode 100644 index 0000000000000..69505e73feb1d --- /dev/null +++ b/src/test/hive/golden/literal_double-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 b/src/test/hive/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 b/src/test/hive/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/src/test/hive/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/src/test/hive/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 b/src/test/hive/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d b/src/test/hive/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d new file mode 100644 index 0000000000000..6323f4efa99c7 --- /dev/null +++ b/src/test/hive/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d @@ -0,0 +1 @@ +facebook facebook facebook facebook facebook facebook facebook facebook facebook facebook \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d b/src/test/hive/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 b/src/test/hive/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 new file mode 100644 index 0000000000000..d7a8f25b41301 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 @@ -0,0 +1,1000 @@ +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/src/test/hive/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 b/src/test/hive/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 new file mode 100644 index 0000000000000..653516475da22 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 @@ -0,0 +1,1000 @@ +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/src/test/hive/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 b/src/test/hive/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 b/src/test/hive/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 new file mode 100644 index 0000000000000..e0dcec0869734 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 @@ -0,0 +1,12 @@ +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 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) diff --git a/src/test/hive/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f b/src/test/hive/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 b/src/test/hive/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 b/src/test/hive/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 new file mode 100644 index 0000000000000..0a88e449f625a --- /dev/null +++ b/src/test/hive/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 @@ -0,0 +1,2 @@ +ds=2008-04-08/hr=11 +ds=2008-04-08/hr=12 diff --git a/src/test/hive/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 b/src/test/hive/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 new file mode 100644 index 0000000000000..1634adfc4d70d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 @@ -0,0 +1,2 @@ +ds=2008-12-31/hr=11 +ds=2008-12-31/hr=12 diff --git a/src/test/hive/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/src/test/hive/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 b/src/test/hive/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e b/src/test/hive/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e new file mode 100644 index 0000000000000..9d3a522ff81dc --- /dev/null +++ b/src/test/hive/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e @@ -0,0 +1,12 @@ +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 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 diff --git a/src/test/hive/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 b/src/test/hive/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 b/src/test/hive/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 b/src/test/hive/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 new file mode 100644 index 0000000000000..574727266b183 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 @@ -0,0 +1,2 @@ +ds=2008-12-31/hr=11 +ds=2008-12-31/hr=12 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 b/src/test/hive/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 new file mode 100644 index 0000000000000..3994499f92576 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 @@ -0,0 +1,1000 @@ +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/src/test/hive/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/src/test/hive/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 b/src/test/hive/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 b/src/test/hive/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 new file mode 100644 index 0000000000000..6aa67737fa92d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 @@ -0,0 +1,12 @@ +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 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 diff --git a/src/test/hive/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d b/src/test/hive/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 b/src/test/hive/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb b/src/test/hive/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb new file mode 100644 index 0000000000000..29ade2aff0542 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb @@ -0,0 +1,1000 @@ +238 val_238 2010-03-03 11 +86 val_86 2010-03-03 11 +311 val_311 2010-03-03 11 +27 val_27 2010-03-03 11 +165 val_165 2010-03-03 11 +409 val_409 2010-03-03 11 +255 val_255 2010-03-03 11 +278 val_278 2010-03-03 11 +98 val_98 2010-03-03 11 +484 val_484 2010-03-03 11 +265 val_265 2010-03-03 11 +193 val_193 2010-03-03 11 +401 val_401 2010-03-03 11 +150 val_150 2010-03-03 11 +273 val_273 2010-03-03 11 +224 val_224 2010-03-03 11 +369 val_369 2010-03-03 11 +66 val_66 2010-03-03 11 +128 val_128 2010-03-03 11 +213 val_213 2010-03-03 11 +146 val_146 2010-03-03 11 +406 val_406 2010-03-03 11 +429 val_429 2010-03-03 11 +374 val_374 2010-03-03 11 +152 val_152 2010-03-03 11 +469 val_469 2010-03-03 11 +145 val_145 2010-03-03 11 +495 val_495 2010-03-03 11 +37 val_37 2010-03-03 11 +327 val_327 2010-03-03 11 +281 val_281 2010-03-03 11 +277 val_277 2010-03-03 11 +209 val_209 2010-03-03 11 +15 val_15 2010-03-03 11 +82 val_82 2010-03-03 11 +403 val_403 2010-03-03 11 +166 val_166 2010-03-03 11 +417 val_417 2010-03-03 11 +430 val_430 2010-03-03 11 +252 val_252 2010-03-03 11 +292 val_292 2010-03-03 11 +219 val_219 2010-03-03 11 +287 val_287 2010-03-03 11 +153 val_153 2010-03-03 11 +193 val_193 2010-03-03 11 +338 val_338 2010-03-03 11 +446 val_446 2010-03-03 11 +459 val_459 2010-03-03 11 +394 val_394 2010-03-03 11 +237 val_237 2010-03-03 11 +482 val_482 2010-03-03 11 +174 val_174 2010-03-03 11 +413 val_413 2010-03-03 11 +494 val_494 2010-03-03 11 +207 val_207 2010-03-03 11 +199 val_199 2010-03-03 11 +466 val_466 2010-03-03 11 +208 val_208 2010-03-03 11 +174 val_174 2010-03-03 11 +399 val_399 2010-03-03 11 +396 val_396 2010-03-03 11 +247 val_247 2010-03-03 11 +417 val_417 2010-03-03 11 +489 val_489 2010-03-03 11 +162 val_162 2010-03-03 11 +377 val_377 2010-03-03 11 +397 val_397 2010-03-03 11 +309 val_309 2010-03-03 11 +365 val_365 2010-03-03 11 +266 val_266 2010-03-03 11 +439 val_439 2010-03-03 11 +342 val_342 2010-03-03 11 +367 val_367 2010-03-03 11 +325 val_325 2010-03-03 11 +167 val_167 2010-03-03 11 +195 val_195 2010-03-03 11 +475 val_475 2010-03-03 11 +17 val_17 2010-03-03 11 +113 val_113 2010-03-03 11 +155 val_155 2010-03-03 11 +203 val_203 2010-03-03 11 +339 val_339 2010-03-03 11 +0 val_0 2010-03-03 11 +455 val_455 2010-03-03 11 +128 val_128 2010-03-03 11 +311 val_311 2010-03-03 11 +316 val_316 2010-03-03 11 +57 val_57 2010-03-03 11 +302 val_302 2010-03-03 11 +205 val_205 2010-03-03 11 +149 val_149 2010-03-03 11 +438 val_438 2010-03-03 11 +345 val_345 2010-03-03 11 +129 val_129 2010-03-03 11 +170 val_170 2010-03-03 11 +20 val_20 2010-03-03 11 +489 val_489 2010-03-03 11 +157 val_157 2010-03-03 11 +378 val_378 2010-03-03 11 +221 val_221 2010-03-03 11 +92 val_92 2010-03-03 11 +111 val_111 2010-03-03 11 +47 val_47 2010-03-03 11 +72 val_72 2010-03-03 11 +4 val_4 2010-03-03 11 +280 val_280 2010-03-03 11 +35 val_35 2010-03-03 11 +427 val_427 2010-03-03 11 +277 val_277 2010-03-03 11 +208 val_208 2010-03-03 11 +356 val_356 2010-03-03 11 +399 val_399 2010-03-03 11 +169 val_169 2010-03-03 11 +382 val_382 2010-03-03 11 +498 val_498 2010-03-03 11 +125 val_125 2010-03-03 11 +386 val_386 2010-03-03 11 +437 val_437 2010-03-03 11 +469 val_469 2010-03-03 11 +192 val_192 2010-03-03 11 +286 val_286 2010-03-03 11 +187 val_187 2010-03-03 11 +176 val_176 2010-03-03 11 +54 val_54 2010-03-03 11 +459 val_459 2010-03-03 11 +51 val_51 2010-03-03 11 +138 val_138 2010-03-03 11 +103 val_103 2010-03-03 11 +239 val_239 2010-03-03 11 +213 val_213 2010-03-03 11 +216 val_216 2010-03-03 11 +430 val_430 2010-03-03 11 +278 val_278 2010-03-03 11 +176 val_176 2010-03-03 11 +289 val_289 2010-03-03 11 +221 val_221 2010-03-03 11 +65 val_65 2010-03-03 11 +318 val_318 2010-03-03 11 +332 val_332 2010-03-03 11 +311 val_311 2010-03-03 11 +275 val_275 2010-03-03 11 +137 val_137 2010-03-03 11 +241 val_241 2010-03-03 11 +83 val_83 2010-03-03 11 +333 val_333 2010-03-03 11 +180 val_180 2010-03-03 11 +284 val_284 2010-03-03 11 +12 val_12 2010-03-03 11 +230 val_230 2010-03-03 11 +181 val_181 2010-03-03 11 +67 val_67 2010-03-03 11 +260 val_260 2010-03-03 11 +404 val_404 2010-03-03 11 +384 val_384 2010-03-03 11 +489 val_489 2010-03-03 11 +353 val_353 2010-03-03 11 +373 val_373 2010-03-03 11 +272 val_272 2010-03-03 11 +138 val_138 2010-03-03 11 +217 val_217 2010-03-03 11 +84 val_84 2010-03-03 11 +348 val_348 2010-03-03 11 +466 val_466 2010-03-03 11 +58 val_58 2010-03-03 11 +8 val_8 2010-03-03 11 +411 val_411 2010-03-03 11 +230 val_230 2010-03-03 11 +208 val_208 2010-03-03 11 +348 val_348 2010-03-03 11 +24 val_24 2010-03-03 11 +463 val_463 2010-03-03 11 +431 val_431 2010-03-03 11 +179 val_179 2010-03-03 11 +172 val_172 2010-03-03 11 +42 val_42 2010-03-03 11 +129 val_129 2010-03-03 11 +158 val_158 2010-03-03 11 +119 val_119 2010-03-03 11 +496 val_496 2010-03-03 11 +0 val_0 2010-03-03 11 +322 val_322 2010-03-03 11 +197 val_197 2010-03-03 11 +468 val_468 2010-03-03 11 +393 val_393 2010-03-03 11 +454 val_454 2010-03-03 11 +100 val_100 2010-03-03 11 +298 val_298 2010-03-03 11 +199 val_199 2010-03-03 11 +191 val_191 2010-03-03 11 +418 val_418 2010-03-03 11 +96 val_96 2010-03-03 11 +26 val_26 2010-03-03 11 +165 val_165 2010-03-03 11 +327 val_327 2010-03-03 11 +230 val_230 2010-03-03 11 +205 val_205 2010-03-03 11 +120 val_120 2010-03-03 11 +131 val_131 2010-03-03 11 +51 val_51 2010-03-03 11 +404 val_404 2010-03-03 11 +43 val_43 2010-03-03 11 +436 val_436 2010-03-03 11 +156 val_156 2010-03-03 11 +469 val_469 2010-03-03 11 +468 val_468 2010-03-03 11 +308 val_308 2010-03-03 11 +95 val_95 2010-03-03 11 +196 val_196 2010-03-03 11 +288 val_288 2010-03-03 11 +481 val_481 2010-03-03 11 +457 val_457 2010-03-03 11 +98 val_98 2010-03-03 11 +282 val_282 2010-03-03 11 +197 val_197 2010-03-03 11 +187 val_187 2010-03-03 11 +318 val_318 2010-03-03 11 +318 val_318 2010-03-03 11 +409 val_409 2010-03-03 11 +470 val_470 2010-03-03 11 +137 val_137 2010-03-03 11 +369 val_369 2010-03-03 11 +316 val_316 2010-03-03 11 +169 val_169 2010-03-03 11 +413 val_413 2010-03-03 11 +85 val_85 2010-03-03 11 +77 val_77 2010-03-03 11 +0 val_0 2010-03-03 11 +490 val_490 2010-03-03 11 +87 val_87 2010-03-03 11 +364 val_364 2010-03-03 11 +179 val_179 2010-03-03 11 +118 val_118 2010-03-03 11 +134 val_134 2010-03-03 11 +395 val_395 2010-03-03 11 +282 val_282 2010-03-03 11 +138 val_138 2010-03-03 11 +238 val_238 2010-03-03 11 +419 val_419 2010-03-03 11 +15 val_15 2010-03-03 11 +118 val_118 2010-03-03 11 +72 val_72 2010-03-03 11 +90 val_90 2010-03-03 11 +307 val_307 2010-03-03 11 +19 val_19 2010-03-03 11 +435 val_435 2010-03-03 11 +10 val_10 2010-03-03 11 +277 val_277 2010-03-03 11 +273 val_273 2010-03-03 11 +306 val_306 2010-03-03 11 +224 val_224 2010-03-03 11 +309 val_309 2010-03-03 11 +389 val_389 2010-03-03 11 +327 val_327 2010-03-03 11 +242 val_242 2010-03-03 11 +369 val_369 2010-03-03 11 +392 val_392 2010-03-03 11 +272 val_272 2010-03-03 11 +331 val_331 2010-03-03 11 +401 val_401 2010-03-03 11 +242 val_242 2010-03-03 11 +452 val_452 2010-03-03 11 +177 val_177 2010-03-03 11 +226 val_226 2010-03-03 11 +5 val_5 2010-03-03 11 +497 val_497 2010-03-03 11 +402 val_402 2010-03-03 11 +396 val_396 2010-03-03 11 +317 val_317 2010-03-03 11 +395 val_395 2010-03-03 11 +58 val_58 2010-03-03 11 +35 val_35 2010-03-03 11 +336 val_336 2010-03-03 11 +95 val_95 2010-03-03 11 +11 val_11 2010-03-03 11 +168 val_168 2010-03-03 11 +34 val_34 2010-03-03 11 +229 val_229 2010-03-03 11 +233 val_233 2010-03-03 11 +143 val_143 2010-03-03 11 +472 val_472 2010-03-03 11 +322 val_322 2010-03-03 11 +498 val_498 2010-03-03 11 +160 val_160 2010-03-03 11 +195 val_195 2010-03-03 11 +42 val_42 2010-03-03 11 +321 val_321 2010-03-03 11 +430 val_430 2010-03-03 11 +119 val_119 2010-03-03 11 +489 val_489 2010-03-03 11 +458 val_458 2010-03-03 11 +78 val_78 2010-03-03 11 +76 val_76 2010-03-03 11 +41 val_41 2010-03-03 11 +223 val_223 2010-03-03 11 +492 val_492 2010-03-03 11 +149 val_149 2010-03-03 11 +449 val_449 2010-03-03 11 +218 val_218 2010-03-03 11 +228 val_228 2010-03-03 11 +138 val_138 2010-03-03 11 +453 val_453 2010-03-03 11 +30 val_30 2010-03-03 11 +209 val_209 2010-03-03 11 +64 val_64 2010-03-03 11 +468 val_468 2010-03-03 11 +76 val_76 2010-03-03 11 +74 val_74 2010-03-03 11 +342 val_342 2010-03-03 11 +69 val_69 2010-03-03 11 +230 val_230 2010-03-03 11 +33 val_33 2010-03-03 11 +368 val_368 2010-03-03 11 +103 val_103 2010-03-03 11 +296 val_296 2010-03-03 11 +113 val_113 2010-03-03 11 +216 val_216 2010-03-03 11 +367 val_367 2010-03-03 11 +344 val_344 2010-03-03 11 +167 val_167 2010-03-03 11 +274 val_274 2010-03-03 11 +219 val_219 2010-03-03 11 +239 val_239 2010-03-03 11 +485 val_485 2010-03-03 11 +116 val_116 2010-03-03 11 +223 val_223 2010-03-03 11 +256 val_256 2010-03-03 11 +263 val_263 2010-03-03 11 +70 val_70 2010-03-03 11 +487 val_487 2010-03-03 11 +480 val_480 2010-03-03 11 +401 val_401 2010-03-03 11 +288 val_288 2010-03-03 11 +191 val_191 2010-03-03 11 +5 val_5 2010-03-03 11 +244 val_244 2010-03-03 11 +438 val_438 2010-03-03 11 +128 val_128 2010-03-03 11 +467 val_467 2010-03-03 11 +432 val_432 2010-03-03 11 +202 val_202 2010-03-03 11 +316 val_316 2010-03-03 11 +229 val_229 2010-03-03 11 +469 val_469 2010-03-03 11 +463 val_463 2010-03-03 11 +280 val_280 2010-03-03 11 +2 val_2 2010-03-03 11 +35 val_35 2010-03-03 11 +283 val_283 2010-03-03 11 +331 val_331 2010-03-03 11 +235 val_235 2010-03-03 11 +80 val_80 2010-03-03 11 +44 val_44 2010-03-03 11 +193 val_193 2010-03-03 11 +321 val_321 2010-03-03 11 +335 val_335 2010-03-03 11 +104 val_104 2010-03-03 11 +466 val_466 2010-03-03 11 +366 val_366 2010-03-03 11 +175 val_175 2010-03-03 11 +403 val_403 2010-03-03 11 +483 val_483 2010-03-03 11 +53 val_53 2010-03-03 11 +105 val_105 2010-03-03 11 +257 val_257 2010-03-03 11 +406 val_406 2010-03-03 11 +409 val_409 2010-03-03 11 +190 val_190 2010-03-03 11 +406 val_406 2010-03-03 11 +401 val_401 2010-03-03 11 +114 val_114 2010-03-03 11 +258 val_258 2010-03-03 11 +90 val_90 2010-03-03 11 +203 val_203 2010-03-03 11 +262 val_262 2010-03-03 11 +348 val_348 2010-03-03 11 +424 val_424 2010-03-03 11 +12 val_12 2010-03-03 11 +396 val_396 2010-03-03 11 +201 val_201 2010-03-03 11 +217 val_217 2010-03-03 11 +164 val_164 2010-03-03 11 +431 val_431 2010-03-03 11 +454 val_454 2010-03-03 11 +478 val_478 2010-03-03 11 +298 val_298 2010-03-03 11 +125 val_125 2010-03-03 11 +431 val_431 2010-03-03 11 +164 val_164 2010-03-03 11 +424 val_424 2010-03-03 11 +187 val_187 2010-03-03 11 +382 val_382 2010-03-03 11 +5 val_5 2010-03-03 11 +70 val_70 2010-03-03 11 +397 val_397 2010-03-03 11 +480 val_480 2010-03-03 11 +291 val_291 2010-03-03 11 +24 val_24 2010-03-03 11 +351 val_351 2010-03-03 11 +255 val_255 2010-03-03 11 +104 val_104 2010-03-03 11 +70 val_70 2010-03-03 11 +163 val_163 2010-03-03 11 +438 val_438 2010-03-03 11 +119 val_119 2010-03-03 11 +414 val_414 2010-03-03 11 +200 val_200 2010-03-03 11 +491 val_491 2010-03-03 11 +237 val_237 2010-03-03 11 +439 val_439 2010-03-03 11 +360 val_360 2010-03-03 11 +248 val_248 2010-03-03 11 +479 val_479 2010-03-03 11 +305 val_305 2010-03-03 11 +417 val_417 2010-03-03 11 +199 val_199 2010-03-03 11 +444 val_444 2010-03-03 11 +120 val_120 2010-03-03 11 +429 val_429 2010-03-03 11 +169 val_169 2010-03-03 11 +443 val_443 2010-03-03 11 +323 val_323 2010-03-03 11 +325 val_325 2010-03-03 11 +277 val_277 2010-03-03 11 +230 val_230 2010-03-03 11 +478 val_478 2010-03-03 11 +178 val_178 2010-03-03 11 +468 val_468 2010-03-03 11 +310 val_310 2010-03-03 11 +317 val_317 2010-03-03 11 +333 val_333 2010-03-03 11 +493 val_493 2010-03-03 11 +460 val_460 2010-03-03 11 +207 val_207 2010-03-03 11 +249 val_249 2010-03-03 11 +265 val_265 2010-03-03 11 +480 val_480 2010-03-03 11 +83 val_83 2010-03-03 11 +136 val_136 2010-03-03 11 +353 val_353 2010-03-03 11 +172 val_172 2010-03-03 11 +214 val_214 2010-03-03 11 +462 val_462 2010-03-03 11 +233 val_233 2010-03-03 11 +406 val_406 2010-03-03 11 +133 val_133 2010-03-03 11 +175 val_175 2010-03-03 11 +189 val_189 2010-03-03 11 +454 val_454 2010-03-03 11 +375 val_375 2010-03-03 11 +401 val_401 2010-03-03 11 +421 val_421 2010-03-03 11 +407 val_407 2010-03-03 11 +384 val_384 2010-03-03 11 +256 val_256 2010-03-03 11 +26 val_26 2010-03-03 11 +134 val_134 2010-03-03 11 +67 val_67 2010-03-03 11 +384 val_384 2010-03-03 11 +379 val_379 2010-03-03 11 +18 val_18 2010-03-03 11 +462 val_462 2010-03-03 11 +492 val_492 2010-03-03 11 +100 val_100 2010-03-03 11 +298 val_298 2010-03-03 11 +9 val_9 2010-03-03 11 +341 val_341 2010-03-03 11 +498 val_498 2010-03-03 11 +146 val_146 2010-03-03 11 +458 val_458 2010-03-03 11 +362 val_362 2010-03-03 11 +186 val_186 2010-03-03 11 +285 val_285 2010-03-03 11 +348 val_348 2010-03-03 11 +167 val_167 2010-03-03 11 +18 val_18 2010-03-03 11 +273 val_273 2010-03-03 11 +183 val_183 2010-03-03 11 +281 val_281 2010-03-03 11 +344 val_344 2010-03-03 11 +97 val_97 2010-03-03 11 +469 val_469 2010-03-03 11 +315 val_315 2010-03-03 11 +84 val_84 2010-03-03 11 +28 val_28 2010-03-03 11 +37 val_37 2010-03-03 11 +448 val_448 2010-03-03 11 +152 val_152 2010-03-03 11 +348 val_348 2010-03-03 11 +307 val_307 2010-03-03 11 +194 val_194 2010-03-03 11 +414 val_414 2010-03-03 11 +477 val_477 2010-03-03 11 +222 val_222 2010-03-03 11 +126 val_126 2010-03-03 11 +90 val_90 2010-03-03 11 +169 val_169 2010-03-03 11 +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 +238 val_238 2010-03-03 11 +86 val_86 2010-03-03 11 +311 val_311 2010-03-03 11 +27 val_27 2010-03-03 11 +165 val_165 2010-03-03 11 +409 val_409 2010-03-03 11 +255 val_255 2010-03-03 11 +278 val_278 2010-03-03 11 +98 val_98 2010-03-03 11 +484 val_484 2010-03-03 11 +265 val_265 2010-03-03 11 +193 val_193 2010-03-03 11 +401 val_401 2010-03-03 11 +150 val_150 2010-03-03 11 +273 val_273 2010-03-03 11 +224 val_224 2010-03-03 11 +369 val_369 2010-03-03 11 +66 val_66 2010-03-03 11 +128 val_128 2010-03-03 11 +213 val_213 2010-03-03 11 +146 val_146 2010-03-03 11 +406 val_406 2010-03-03 11 +429 val_429 2010-03-03 11 +374 val_374 2010-03-03 11 +152 val_152 2010-03-03 11 +469 val_469 2010-03-03 11 +145 val_145 2010-03-03 11 +495 val_495 2010-03-03 11 +37 val_37 2010-03-03 11 +327 val_327 2010-03-03 11 +281 val_281 2010-03-03 11 +277 val_277 2010-03-03 11 +209 val_209 2010-03-03 11 +15 val_15 2010-03-03 11 +82 val_82 2010-03-03 11 +403 val_403 2010-03-03 11 +166 val_166 2010-03-03 11 +417 val_417 2010-03-03 11 +430 val_430 2010-03-03 11 +252 val_252 2010-03-03 11 +292 val_292 2010-03-03 11 +219 val_219 2010-03-03 11 +287 val_287 2010-03-03 11 +153 val_153 2010-03-03 11 +193 val_193 2010-03-03 11 +338 val_338 2010-03-03 11 +446 val_446 2010-03-03 11 +459 val_459 2010-03-03 11 +394 val_394 2010-03-03 11 +237 val_237 2010-03-03 11 +482 val_482 2010-03-03 11 +174 val_174 2010-03-03 11 +413 val_413 2010-03-03 11 +494 val_494 2010-03-03 11 +207 val_207 2010-03-03 11 +199 val_199 2010-03-03 11 +466 val_466 2010-03-03 11 +208 val_208 2010-03-03 11 +174 val_174 2010-03-03 11 +399 val_399 2010-03-03 11 +396 val_396 2010-03-03 11 +247 val_247 2010-03-03 11 +417 val_417 2010-03-03 11 +489 val_489 2010-03-03 11 +162 val_162 2010-03-03 11 +377 val_377 2010-03-03 11 +397 val_397 2010-03-03 11 +309 val_309 2010-03-03 11 +365 val_365 2010-03-03 11 +266 val_266 2010-03-03 11 +439 val_439 2010-03-03 11 +342 val_342 2010-03-03 11 +367 val_367 2010-03-03 11 +325 val_325 2010-03-03 11 +167 val_167 2010-03-03 11 +195 val_195 2010-03-03 11 +475 val_475 2010-03-03 11 +17 val_17 2010-03-03 11 +113 val_113 2010-03-03 11 +155 val_155 2010-03-03 11 +203 val_203 2010-03-03 11 +339 val_339 2010-03-03 11 +0 val_0 2010-03-03 11 +455 val_455 2010-03-03 11 +128 val_128 2010-03-03 11 +311 val_311 2010-03-03 11 +316 val_316 2010-03-03 11 +57 val_57 2010-03-03 11 +302 val_302 2010-03-03 11 +205 val_205 2010-03-03 11 +149 val_149 2010-03-03 11 +438 val_438 2010-03-03 11 +345 val_345 2010-03-03 11 +129 val_129 2010-03-03 11 +170 val_170 2010-03-03 11 +20 val_20 2010-03-03 11 +489 val_489 2010-03-03 11 +157 val_157 2010-03-03 11 +378 val_378 2010-03-03 11 +221 val_221 2010-03-03 11 +92 val_92 2010-03-03 11 +111 val_111 2010-03-03 11 +47 val_47 2010-03-03 11 +72 val_72 2010-03-03 11 +4 val_4 2010-03-03 11 +280 val_280 2010-03-03 11 +35 val_35 2010-03-03 11 +427 val_427 2010-03-03 11 +277 val_277 2010-03-03 11 +208 val_208 2010-03-03 11 +356 val_356 2010-03-03 11 +399 val_399 2010-03-03 11 +169 val_169 2010-03-03 11 +382 val_382 2010-03-03 11 +498 val_498 2010-03-03 11 +125 val_125 2010-03-03 11 +386 val_386 2010-03-03 11 +437 val_437 2010-03-03 11 +469 val_469 2010-03-03 11 +192 val_192 2010-03-03 11 +286 val_286 2010-03-03 11 +187 val_187 2010-03-03 11 +176 val_176 2010-03-03 11 +54 val_54 2010-03-03 11 +459 val_459 2010-03-03 11 +51 val_51 2010-03-03 11 +138 val_138 2010-03-03 11 +103 val_103 2010-03-03 11 +239 val_239 2010-03-03 11 +213 val_213 2010-03-03 11 +216 val_216 2010-03-03 11 +430 val_430 2010-03-03 11 +278 val_278 2010-03-03 11 +176 val_176 2010-03-03 11 +289 val_289 2010-03-03 11 +221 val_221 2010-03-03 11 +65 val_65 2010-03-03 11 +318 val_318 2010-03-03 11 +332 val_332 2010-03-03 11 +311 val_311 2010-03-03 11 +275 val_275 2010-03-03 11 +137 val_137 2010-03-03 11 +241 val_241 2010-03-03 11 +83 val_83 2010-03-03 11 +333 val_333 2010-03-03 11 +180 val_180 2010-03-03 11 +284 val_284 2010-03-03 11 +12 val_12 2010-03-03 11 +230 val_230 2010-03-03 11 +181 val_181 2010-03-03 11 +67 val_67 2010-03-03 11 +260 val_260 2010-03-03 11 +404 val_404 2010-03-03 11 +384 val_384 2010-03-03 11 +489 val_489 2010-03-03 11 +353 val_353 2010-03-03 11 +373 val_373 2010-03-03 11 +272 val_272 2010-03-03 11 +138 val_138 2010-03-03 11 +217 val_217 2010-03-03 11 +84 val_84 2010-03-03 11 +348 val_348 2010-03-03 11 +466 val_466 2010-03-03 11 +58 val_58 2010-03-03 11 +8 val_8 2010-03-03 11 +411 val_411 2010-03-03 11 +230 val_230 2010-03-03 11 +208 val_208 2010-03-03 11 +348 val_348 2010-03-03 11 +24 val_24 2010-03-03 11 +463 val_463 2010-03-03 11 +431 val_431 2010-03-03 11 +179 val_179 2010-03-03 11 +172 val_172 2010-03-03 11 +42 val_42 2010-03-03 11 +129 val_129 2010-03-03 11 +158 val_158 2010-03-03 11 +119 val_119 2010-03-03 11 +496 val_496 2010-03-03 11 +0 val_0 2010-03-03 11 +322 val_322 2010-03-03 11 +197 val_197 2010-03-03 11 +468 val_468 2010-03-03 11 +393 val_393 2010-03-03 11 +454 val_454 2010-03-03 11 +100 val_100 2010-03-03 11 +298 val_298 2010-03-03 11 +199 val_199 2010-03-03 11 +191 val_191 2010-03-03 11 +418 val_418 2010-03-03 11 +96 val_96 2010-03-03 11 +26 val_26 2010-03-03 11 +165 val_165 2010-03-03 11 +327 val_327 2010-03-03 11 +230 val_230 2010-03-03 11 +205 val_205 2010-03-03 11 +120 val_120 2010-03-03 11 +131 val_131 2010-03-03 11 +51 val_51 2010-03-03 11 +404 val_404 2010-03-03 11 +43 val_43 2010-03-03 11 +436 val_436 2010-03-03 11 +156 val_156 2010-03-03 11 +469 val_469 2010-03-03 11 +468 val_468 2010-03-03 11 +308 val_308 2010-03-03 11 +95 val_95 2010-03-03 11 +196 val_196 2010-03-03 11 +288 val_288 2010-03-03 11 +481 val_481 2010-03-03 11 +457 val_457 2010-03-03 11 +98 val_98 2010-03-03 11 +282 val_282 2010-03-03 11 +197 val_197 2010-03-03 11 +187 val_187 2010-03-03 11 +318 val_318 2010-03-03 11 +318 val_318 2010-03-03 11 +409 val_409 2010-03-03 11 +470 val_470 2010-03-03 11 +137 val_137 2010-03-03 11 +369 val_369 2010-03-03 11 +316 val_316 2010-03-03 11 +169 val_169 2010-03-03 11 +413 val_413 2010-03-03 11 +85 val_85 2010-03-03 11 +77 val_77 2010-03-03 11 +0 val_0 2010-03-03 11 +490 val_490 2010-03-03 11 +87 val_87 2010-03-03 11 +364 val_364 2010-03-03 11 +179 val_179 2010-03-03 11 +118 val_118 2010-03-03 11 +134 val_134 2010-03-03 11 +395 val_395 2010-03-03 11 +282 val_282 2010-03-03 11 +138 val_138 2010-03-03 11 +238 val_238 2010-03-03 11 +419 val_419 2010-03-03 11 +15 val_15 2010-03-03 11 +118 val_118 2010-03-03 11 +72 val_72 2010-03-03 11 +90 val_90 2010-03-03 11 +307 val_307 2010-03-03 11 +19 val_19 2010-03-03 11 +435 val_435 2010-03-03 11 +10 val_10 2010-03-03 11 +277 val_277 2010-03-03 11 +273 val_273 2010-03-03 11 +306 val_306 2010-03-03 11 +224 val_224 2010-03-03 11 +309 val_309 2010-03-03 11 +389 val_389 2010-03-03 11 +327 val_327 2010-03-03 11 +242 val_242 2010-03-03 11 +369 val_369 2010-03-03 11 +392 val_392 2010-03-03 11 +272 val_272 2010-03-03 11 +331 val_331 2010-03-03 11 +401 val_401 2010-03-03 11 +242 val_242 2010-03-03 11 +452 val_452 2010-03-03 11 +177 val_177 2010-03-03 11 +226 val_226 2010-03-03 11 +5 val_5 2010-03-03 11 +497 val_497 2010-03-03 11 +402 val_402 2010-03-03 11 +396 val_396 2010-03-03 11 +317 val_317 2010-03-03 11 +395 val_395 2010-03-03 11 +58 val_58 2010-03-03 11 +35 val_35 2010-03-03 11 +336 val_336 2010-03-03 11 +95 val_95 2010-03-03 11 +11 val_11 2010-03-03 11 +168 val_168 2010-03-03 11 +34 val_34 2010-03-03 11 +229 val_229 2010-03-03 11 +233 val_233 2010-03-03 11 +143 val_143 2010-03-03 11 +472 val_472 2010-03-03 11 +322 val_322 2010-03-03 11 +498 val_498 2010-03-03 11 +160 val_160 2010-03-03 11 +195 val_195 2010-03-03 11 +42 val_42 2010-03-03 11 +321 val_321 2010-03-03 11 +430 val_430 2010-03-03 11 +119 val_119 2010-03-03 11 +489 val_489 2010-03-03 11 +458 val_458 2010-03-03 11 +78 val_78 2010-03-03 11 +76 val_76 2010-03-03 11 +41 val_41 2010-03-03 11 +223 val_223 2010-03-03 11 +492 val_492 2010-03-03 11 +149 val_149 2010-03-03 11 +449 val_449 2010-03-03 11 +218 val_218 2010-03-03 11 +228 val_228 2010-03-03 11 +138 val_138 2010-03-03 11 +453 val_453 2010-03-03 11 +30 val_30 2010-03-03 11 +209 val_209 2010-03-03 11 +64 val_64 2010-03-03 11 +468 val_468 2010-03-03 11 +76 val_76 2010-03-03 11 +74 val_74 2010-03-03 11 +342 val_342 2010-03-03 11 +69 val_69 2010-03-03 11 +230 val_230 2010-03-03 11 +33 val_33 2010-03-03 11 +368 val_368 2010-03-03 11 +103 val_103 2010-03-03 11 +296 val_296 2010-03-03 11 +113 val_113 2010-03-03 11 +216 val_216 2010-03-03 11 +367 val_367 2010-03-03 11 +344 val_344 2010-03-03 11 +167 val_167 2010-03-03 11 +274 val_274 2010-03-03 11 +219 val_219 2010-03-03 11 +239 val_239 2010-03-03 11 +485 val_485 2010-03-03 11 +116 val_116 2010-03-03 11 +223 val_223 2010-03-03 11 +256 val_256 2010-03-03 11 +263 val_263 2010-03-03 11 +70 val_70 2010-03-03 11 +487 val_487 2010-03-03 11 +480 val_480 2010-03-03 11 +401 val_401 2010-03-03 11 +288 val_288 2010-03-03 11 +191 val_191 2010-03-03 11 +5 val_5 2010-03-03 11 +244 val_244 2010-03-03 11 +438 val_438 2010-03-03 11 +128 val_128 2010-03-03 11 +467 val_467 2010-03-03 11 +432 val_432 2010-03-03 11 +202 val_202 2010-03-03 11 +316 val_316 2010-03-03 11 +229 val_229 2010-03-03 11 +469 val_469 2010-03-03 11 +463 val_463 2010-03-03 11 +280 val_280 2010-03-03 11 +2 val_2 2010-03-03 11 +35 val_35 2010-03-03 11 +283 val_283 2010-03-03 11 +331 val_331 2010-03-03 11 +235 val_235 2010-03-03 11 +80 val_80 2010-03-03 11 +44 val_44 2010-03-03 11 +193 val_193 2010-03-03 11 +321 val_321 2010-03-03 11 +335 val_335 2010-03-03 11 +104 val_104 2010-03-03 11 +466 val_466 2010-03-03 11 +366 val_366 2010-03-03 11 +175 val_175 2010-03-03 11 +403 val_403 2010-03-03 11 +483 val_483 2010-03-03 11 +53 val_53 2010-03-03 11 +105 val_105 2010-03-03 11 +257 val_257 2010-03-03 11 +406 val_406 2010-03-03 11 +409 val_409 2010-03-03 11 +190 val_190 2010-03-03 11 +406 val_406 2010-03-03 11 +401 val_401 2010-03-03 11 +114 val_114 2010-03-03 11 +258 val_258 2010-03-03 11 +90 val_90 2010-03-03 11 +203 val_203 2010-03-03 11 +262 val_262 2010-03-03 11 +348 val_348 2010-03-03 11 +424 val_424 2010-03-03 11 +12 val_12 2010-03-03 11 +396 val_396 2010-03-03 11 +201 val_201 2010-03-03 11 +217 val_217 2010-03-03 11 +164 val_164 2010-03-03 11 +431 val_431 2010-03-03 11 +454 val_454 2010-03-03 11 +478 val_478 2010-03-03 11 +298 val_298 2010-03-03 11 +125 val_125 2010-03-03 11 +431 val_431 2010-03-03 11 +164 val_164 2010-03-03 11 +424 val_424 2010-03-03 11 +187 val_187 2010-03-03 11 +382 val_382 2010-03-03 11 +5 val_5 2010-03-03 11 +70 val_70 2010-03-03 11 +397 val_397 2010-03-03 11 +480 val_480 2010-03-03 11 +291 val_291 2010-03-03 11 +24 val_24 2010-03-03 11 +351 val_351 2010-03-03 11 +255 val_255 2010-03-03 11 +104 val_104 2010-03-03 11 +70 val_70 2010-03-03 11 +163 val_163 2010-03-03 11 +438 val_438 2010-03-03 11 +119 val_119 2010-03-03 11 +414 val_414 2010-03-03 11 +200 val_200 2010-03-03 11 +491 val_491 2010-03-03 11 +237 val_237 2010-03-03 11 +439 val_439 2010-03-03 11 +360 val_360 2010-03-03 11 +248 val_248 2010-03-03 11 +479 val_479 2010-03-03 11 +305 val_305 2010-03-03 11 +417 val_417 2010-03-03 11 +199 val_199 2010-03-03 11 +444 val_444 2010-03-03 11 +120 val_120 2010-03-03 11 +429 val_429 2010-03-03 11 +169 val_169 2010-03-03 11 +443 val_443 2010-03-03 11 +323 val_323 2010-03-03 11 +325 val_325 2010-03-03 11 +277 val_277 2010-03-03 11 +230 val_230 2010-03-03 11 +478 val_478 2010-03-03 11 +178 val_178 2010-03-03 11 +468 val_468 2010-03-03 11 +310 val_310 2010-03-03 11 +317 val_317 2010-03-03 11 +333 val_333 2010-03-03 11 +493 val_493 2010-03-03 11 +460 val_460 2010-03-03 11 +207 val_207 2010-03-03 11 +249 val_249 2010-03-03 11 +265 val_265 2010-03-03 11 +480 val_480 2010-03-03 11 +83 val_83 2010-03-03 11 +136 val_136 2010-03-03 11 +353 val_353 2010-03-03 11 +172 val_172 2010-03-03 11 +214 val_214 2010-03-03 11 +462 val_462 2010-03-03 11 +233 val_233 2010-03-03 11 +406 val_406 2010-03-03 11 +133 val_133 2010-03-03 11 +175 val_175 2010-03-03 11 +189 val_189 2010-03-03 11 +454 val_454 2010-03-03 11 +375 val_375 2010-03-03 11 +401 val_401 2010-03-03 11 +421 val_421 2010-03-03 11 +407 val_407 2010-03-03 11 +384 val_384 2010-03-03 11 +256 val_256 2010-03-03 11 +26 val_26 2010-03-03 11 +134 val_134 2010-03-03 11 +67 val_67 2010-03-03 11 +384 val_384 2010-03-03 11 +379 val_379 2010-03-03 11 +18 val_18 2010-03-03 11 +462 val_462 2010-03-03 11 +492 val_492 2010-03-03 11 +100 val_100 2010-03-03 11 +298 val_298 2010-03-03 11 +9 val_9 2010-03-03 11 +341 val_341 2010-03-03 11 +498 val_498 2010-03-03 11 +146 val_146 2010-03-03 11 +458 val_458 2010-03-03 11 +362 val_362 2010-03-03 11 +186 val_186 2010-03-03 11 +285 val_285 2010-03-03 11 +348 val_348 2010-03-03 11 +167 val_167 2010-03-03 11 +18 val_18 2010-03-03 11 +273 val_273 2010-03-03 11 +183 val_183 2010-03-03 11 +281 val_281 2010-03-03 11 +344 val_344 2010-03-03 11 +97 val_97 2010-03-03 11 +469 val_469 2010-03-03 11 +315 val_315 2010-03-03 11 +84 val_84 2010-03-03 11 +28 val_28 2010-03-03 11 +37 val_37 2010-03-03 11 +448 val_448 2010-03-03 11 +152 val_152 2010-03-03 11 +348 val_348 2010-03-03 11 +307 val_307 2010-03-03 11 +194 val_194 2010-03-03 11 +414 val_414 2010-03-03 11 +477 val_477 2010-03-03 11 +222 val_222 2010-03-03 11 +126 val_126 2010-03-03 11 +90 val_90 2010-03-03 11 +169 val_169 2010-03-03 11 +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 diff --git a/src/test/hive/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a b/src/test/hive/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a new file mode 100644 index 0000000000000..b7b1b914db857 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a @@ -0,0 +1,1000 @@ +238 val_238 2010-03-03 12 +86 val_86 2010-03-03 12 +311 val_311 2010-03-03 12 +27 val_27 2010-03-03 12 +165 val_165 2010-03-03 12 +409 val_409 2010-03-03 12 +255 val_255 2010-03-03 12 +278 val_278 2010-03-03 12 +98 val_98 2010-03-03 12 +484 val_484 2010-03-03 12 +265 val_265 2010-03-03 12 +193 val_193 2010-03-03 12 +401 val_401 2010-03-03 12 +150 val_150 2010-03-03 12 +273 val_273 2010-03-03 12 +224 val_224 2010-03-03 12 +369 val_369 2010-03-03 12 +66 val_66 2010-03-03 12 +128 val_128 2010-03-03 12 +213 val_213 2010-03-03 12 +146 val_146 2010-03-03 12 +406 val_406 2010-03-03 12 +429 val_429 2010-03-03 12 +374 val_374 2010-03-03 12 +152 val_152 2010-03-03 12 +469 val_469 2010-03-03 12 +145 val_145 2010-03-03 12 +495 val_495 2010-03-03 12 +37 val_37 2010-03-03 12 +327 val_327 2010-03-03 12 +281 val_281 2010-03-03 12 +277 val_277 2010-03-03 12 +209 val_209 2010-03-03 12 +15 val_15 2010-03-03 12 +82 val_82 2010-03-03 12 +403 val_403 2010-03-03 12 +166 val_166 2010-03-03 12 +417 val_417 2010-03-03 12 +430 val_430 2010-03-03 12 +252 val_252 2010-03-03 12 +292 val_292 2010-03-03 12 +219 val_219 2010-03-03 12 +287 val_287 2010-03-03 12 +153 val_153 2010-03-03 12 +193 val_193 2010-03-03 12 +338 val_338 2010-03-03 12 +446 val_446 2010-03-03 12 +459 val_459 2010-03-03 12 +394 val_394 2010-03-03 12 +237 val_237 2010-03-03 12 +482 val_482 2010-03-03 12 +174 val_174 2010-03-03 12 +413 val_413 2010-03-03 12 +494 val_494 2010-03-03 12 +207 val_207 2010-03-03 12 +199 val_199 2010-03-03 12 +466 val_466 2010-03-03 12 +208 val_208 2010-03-03 12 +174 val_174 2010-03-03 12 +399 val_399 2010-03-03 12 +396 val_396 2010-03-03 12 +247 val_247 2010-03-03 12 +417 val_417 2010-03-03 12 +489 val_489 2010-03-03 12 +162 val_162 2010-03-03 12 +377 val_377 2010-03-03 12 +397 val_397 2010-03-03 12 +309 val_309 2010-03-03 12 +365 val_365 2010-03-03 12 +266 val_266 2010-03-03 12 +439 val_439 2010-03-03 12 +342 val_342 2010-03-03 12 +367 val_367 2010-03-03 12 +325 val_325 2010-03-03 12 +167 val_167 2010-03-03 12 +195 val_195 2010-03-03 12 +475 val_475 2010-03-03 12 +17 val_17 2010-03-03 12 +113 val_113 2010-03-03 12 +155 val_155 2010-03-03 12 +203 val_203 2010-03-03 12 +339 val_339 2010-03-03 12 +0 val_0 2010-03-03 12 +455 val_455 2010-03-03 12 +128 val_128 2010-03-03 12 +311 val_311 2010-03-03 12 +316 val_316 2010-03-03 12 +57 val_57 2010-03-03 12 +302 val_302 2010-03-03 12 +205 val_205 2010-03-03 12 +149 val_149 2010-03-03 12 +438 val_438 2010-03-03 12 +345 val_345 2010-03-03 12 +129 val_129 2010-03-03 12 +170 val_170 2010-03-03 12 +20 val_20 2010-03-03 12 +489 val_489 2010-03-03 12 +157 val_157 2010-03-03 12 +378 val_378 2010-03-03 12 +221 val_221 2010-03-03 12 +92 val_92 2010-03-03 12 +111 val_111 2010-03-03 12 +47 val_47 2010-03-03 12 +72 val_72 2010-03-03 12 +4 val_4 2010-03-03 12 +280 val_280 2010-03-03 12 +35 val_35 2010-03-03 12 +427 val_427 2010-03-03 12 +277 val_277 2010-03-03 12 +208 val_208 2010-03-03 12 +356 val_356 2010-03-03 12 +399 val_399 2010-03-03 12 +169 val_169 2010-03-03 12 +382 val_382 2010-03-03 12 +498 val_498 2010-03-03 12 +125 val_125 2010-03-03 12 +386 val_386 2010-03-03 12 +437 val_437 2010-03-03 12 +469 val_469 2010-03-03 12 +192 val_192 2010-03-03 12 +286 val_286 2010-03-03 12 +187 val_187 2010-03-03 12 +176 val_176 2010-03-03 12 +54 val_54 2010-03-03 12 +459 val_459 2010-03-03 12 +51 val_51 2010-03-03 12 +138 val_138 2010-03-03 12 +103 val_103 2010-03-03 12 +239 val_239 2010-03-03 12 +213 val_213 2010-03-03 12 +216 val_216 2010-03-03 12 +430 val_430 2010-03-03 12 +278 val_278 2010-03-03 12 +176 val_176 2010-03-03 12 +289 val_289 2010-03-03 12 +221 val_221 2010-03-03 12 +65 val_65 2010-03-03 12 +318 val_318 2010-03-03 12 +332 val_332 2010-03-03 12 +311 val_311 2010-03-03 12 +275 val_275 2010-03-03 12 +137 val_137 2010-03-03 12 +241 val_241 2010-03-03 12 +83 val_83 2010-03-03 12 +333 val_333 2010-03-03 12 +180 val_180 2010-03-03 12 +284 val_284 2010-03-03 12 +12 val_12 2010-03-03 12 +230 val_230 2010-03-03 12 +181 val_181 2010-03-03 12 +67 val_67 2010-03-03 12 +260 val_260 2010-03-03 12 +404 val_404 2010-03-03 12 +384 val_384 2010-03-03 12 +489 val_489 2010-03-03 12 +353 val_353 2010-03-03 12 +373 val_373 2010-03-03 12 +272 val_272 2010-03-03 12 +138 val_138 2010-03-03 12 +217 val_217 2010-03-03 12 +84 val_84 2010-03-03 12 +348 val_348 2010-03-03 12 +466 val_466 2010-03-03 12 +58 val_58 2010-03-03 12 +8 val_8 2010-03-03 12 +411 val_411 2010-03-03 12 +230 val_230 2010-03-03 12 +208 val_208 2010-03-03 12 +348 val_348 2010-03-03 12 +24 val_24 2010-03-03 12 +463 val_463 2010-03-03 12 +431 val_431 2010-03-03 12 +179 val_179 2010-03-03 12 +172 val_172 2010-03-03 12 +42 val_42 2010-03-03 12 +129 val_129 2010-03-03 12 +158 val_158 2010-03-03 12 +119 val_119 2010-03-03 12 +496 val_496 2010-03-03 12 +0 val_0 2010-03-03 12 +322 val_322 2010-03-03 12 +197 val_197 2010-03-03 12 +468 val_468 2010-03-03 12 +393 val_393 2010-03-03 12 +454 val_454 2010-03-03 12 +100 val_100 2010-03-03 12 +298 val_298 2010-03-03 12 +199 val_199 2010-03-03 12 +191 val_191 2010-03-03 12 +418 val_418 2010-03-03 12 +96 val_96 2010-03-03 12 +26 val_26 2010-03-03 12 +165 val_165 2010-03-03 12 +327 val_327 2010-03-03 12 +230 val_230 2010-03-03 12 +205 val_205 2010-03-03 12 +120 val_120 2010-03-03 12 +131 val_131 2010-03-03 12 +51 val_51 2010-03-03 12 +404 val_404 2010-03-03 12 +43 val_43 2010-03-03 12 +436 val_436 2010-03-03 12 +156 val_156 2010-03-03 12 +469 val_469 2010-03-03 12 +468 val_468 2010-03-03 12 +308 val_308 2010-03-03 12 +95 val_95 2010-03-03 12 +196 val_196 2010-03-03 12 +288 val_288 2010-03-03 12 +481 val_481 2010-03-03 12 +457 val_457 2010-03-03 12 +98 val_98 2010-03-03 12 +282 val_282 2010-03-03 12 +197 val_197 2010-03-03 12 +187 val_187 2010-03-03 12 +318 val_318 2010-03-03 12 +318 val_318 2010-03-03 12 +409 val_409 2010-03-03 12 +470 val_470 2010-03-03 12 +137 val_137 2010-03-03 12 +369 val_369 2010-03-03 12 +316 val_316 2010-03-03 12 +169 val_169 2010-03-03 12 +413 val_413 2010-03-03 12 +85 val_85 2010-03-03 12 +77 val_77 2010-03-03 12 +0 val_0 2010-03-03 12 +490 val_490 2010-03-03 12 +87 val_87 2010-03-03 12 +364 val_364 2010-03-03 12 +179 val_179 2010-03-03 12 +118 val_118 2010-03-03 12 +134 val_134 2010-03-03 12 +395 val_395 2010-03-03 12 +282 val_282 2010-03-03 12 +138 val_138 2010-03-03 12 +238 val_238 2010-03-03 12 +419 val_419 2010-03-03 12 +15 val_15 2010-03-03 12 +118 val_118 2010-03-03 12 +72 val_72 2010-03-03 12 +90 val_90 2010-03-03 12 +307 val_307 2010-03-03 12 +19 val_19 2010-03-03 12 +435 val_435 2010-03-03 12 +10 val_10 2010-03-03 12 +277 val_277 2010-03-03 12 +273 val_273 2010-03-03 12 +306 val_306 2010-03-03 12 +224 val_224 2010-03-03 12 +309 val_309 2010-03-03 12 +389 val_389 2010-03-03 12 +327 val_327 2010-03-03 12 +242 val_242 2010-03-03 12 +369 val_369 2010-03-03 12 +392 val_392 2010-03-03 12 +272 val_272 2010-03-03 12 +331 val_331 2010-03-03 12 +401 val_401 2010-03-03 12 +242 val_242 2010-03-03 12 +452 val_452 2010-03-03 12 +177 val_177 2010-03-03 12 +226 val_226 2010-03-03 12 +5 val_5 2010-03-03 12 +497 val_497 2010-03-03 12 +402 val_402 2010-03-03 12 +396 val_396 2010-03-03 12 +317 val_317 2010-03-03 12 +395 val_395 2010-03-03 12 +58 val_58 2010-03-03 12 +35 val_35 2010-03-03 12 +336 val_336 2010-03-03 12 +95 val_95 2010-03-03 12 +11 val_11 2010-03-03 12 +168 val_168 2010-03-03 12 +34 val_34 2010-03-03 12 +229 val_229 2010-03-03 12 +233 val_233 2010-03-03 12 +143 val_143 2010-03-03 12 +472 val_472 2010-03-03 12 +322 val_322 2010-03-03 12 +498 val_498 2010-03-03 12 +160 val_160 2010-03-03 12 +195 val_195 2010-03-03 12 +42 val_42 2010-03-03 12 +321 val_321 2010-03-03 12 +430 val_430 2010-03-03 12 +119 val_119 2010-03-03 12 +489 val_489 2010-03-03 12 +458 val_458 2010-03-03 12 +78 val_78 2010-03-03 12 +76 val_76 2010-03-03 12 +41 val_41 2010-03-03 12 +223 val_223 2010-03-03 12 +492 val_492 2010-03-03 12 +149 val_149 2010-03-03 12 +449 val_449 2010-03-03 12 +218 val_218 2010-03-03 12 +228 val_228 2010-03-03 12 +138 val_138 2010-03-03 12 +453 val_453 2010-03-03 12 +30 val_30 2010-03-03 12 +209 val_209 2010-03-03 12 +64 val_64 2010-03-03 12 +468 val_468 2010-03-03 12 +76 val_76 2010-03-03 12 +74 val_74 2010-03-03 12 +342 val_342 2010-03-03 12 +69 val_69 2010-03-03 12 +230 val_230 2010-03-03 12 +33 val_33 2010-03-03 12 +368 val_368 2010-03-03 12 +103 val_103 2010-03-03 12 +296 val_296 2010-03-03 12 +113 val_113 2010-03-03 12 +216 val_216 2010-03-03 12 +367 val_367 2010-03-03 12 +344 val_344 2010-03-03 12 +167 val_167 2010-03-03 12 +274 val_274 2010-03-03 12 +219 val_219 2010-03-03 12 +239 val_239 2010-03-03 12 +485 val_485 2010-03-03 12 +116 val_116 2010-03-03 12 +223 val_223 2010-03-03 12 +256 val_256 2010-03-03 12 +263 val_263 2010-03-03 12 +70 val_70 2010-03-03 12 +487 val_487 2010-03-03 12 +480 val_480 2010-03-03 12 +401 val_401 2010-03-03 12 +288 val_288 2010-03-03 12 +191 val_191 2010-03-03 12 +5 val_5 2010-03-03 12 +244 val_244 2010-03-03 12 +438 val_438 2010-03-03 12 +128 val_128 2010-03-03 12 +467 val_467 2010-03-03 12 +432 val_432 2010-03-03 12 +202 val_202 2010-03-03 12 +316 val_316 2010-03-03 12 +229 val_229 2010-03-03 12 +469 val_469 2010-03-03 12 +463 val_463 2010-03-03 12 +280 val_280 2010-03-03 12 +2 val_2 2010-03-03 12 +35 val_35 2010-03-03 12 +283 val_283 2010-03-03 12 +331 val_331 2010-03-03 12 +235 val_235 2010-03-03 12 +80 val_80 2010-03-03 12 +44 val_44 2010-03-03 12 +193 val_193 2010-03-03 12 +321 val_321 2010-03-03 12 +335 val_335 2010-03-03 12 +104 val_104 2010-03-03 12 +466 val_466 2010-03-03 12 +366 val_366 2010-03-03 12 +175 val_175 2010-03-03 12 +403 val_403 2010-03-03 12 +483 val_483 2010-03-03 12 +53 val_53 2010-03-03 12 +105 val_105 2010-03-03 12 +257 val_257 2010-03-03 12 +406 val_406 2010-03-03 12 +409 val_409 2010-03-03 12 +190 val_190 2010-03-03 12 +406 val_406 2010-03-03 12 +401 val_401 2010-03-03 12 +114 val_114 2010-03-03 12 +258 val_258 2010-03-03 12 +90 val_90 2010-03-03 12 +203 val_203 2010-03-03 12 +262 val_262 2010-03-03 12 +348 val_348 2010-03-03 12 +424 val_424 2010-03-03 12 +12 val_12 2010-03-03 12 +396 val_396 2010-03-03 12 +201 val_201 2010-03-03 12 +217 val_217 2010-03-03 12 +164 val_164 2010-03-03 12 +431 val_431 2010-03-03 12 +454 val_454 2010-03-03 12 +478 val_478 2010-03-03 12 +298 val_298 2010-03-03 12 +125 val_125 2010-03-03 12 +431 val_431 2010-03-03 12 +164 val_164 2010-03-03 12 +424 val_424 2010-03-03 12 +187 val_187 2010-03-03 12 +382 val_382 2010-03-03 12 +5 val_5 2010-03-03 12 +70 val_70 2010-03-03 12 +397 val_397 2010-03-03 12 +480 val_480 2010-03-03 12 +291 val_291 2010-03-03 12 +24 val_24 2010-03-03 12 +351 val_351 2010-03-03 12 +255 val_255 2010-03-03 12 +104 val_104 2010-03-03 12 +70 val_70 2010-03-03 12 +163 val_163 2010-03-03 12 +438 val_438 2010-03-03 12 +119 val_119 2010-03-03 12 +414 val_414 2010-03-03 12 +200 val_200 2010-03-03 12 +491 val_491 2010-03-03 12 +237 val_237 2010-03-03 12 +439 val_439 2010-03-03 12 +360 val_360 2010-03-03 12 +248 val_248 2010-03-03 12 +479 val_479 2010-03-03 12 +305 val_305 2010-03-03 12 +417 val_417 2010-03-03 12 +199 val_199 2010-03-03 12 +444 val_444 2010-03-03 12 +120 val_120 2010-03-03 12 +429 val_429 2010-03-03 12 +169 val_169 2010-03-03 12 +443 val_443 2010-03-03 12 +323 val_323 2010-03-03 12 +325 val_325 2010-03-03 12 +277 val_277 2010-03-03 12 +230 val_230 2010-03-03 12 +478 val_478 2010-03-03 12 +178 val_178 2010-03-03 12 +468 val_468 2010-03-03 12 +310 val_310 2010-03-03 12 +317 val_317 2010-03-03 12 +333 val_333 2010-03-03 12 +493 val_493 2010-03-03 12 +460 val_460 2010-03-03 12 +207 val_207 2010-03-03 12 +249 val_249 2010-03-03 12 +265 val_265 2010-03-03 12 +480 val_480 2010-03-03 12 +83 val_83 2010-03-03 12 +136 val_136 2010-03-03 12 +353 val_353 2010-03-03 12 +172 val_172 2010-03-03 12 +214 val_214 2010-03-03 12 +462 val_462 2010-03-03 12 +233 val_233 2010-03-03 12 +406 val_406 2010-03-03 12 +133 val_133 2010-03-03 12 +175 val_175 2010-03-03 12 +189 val_189 2010-03-03 12 +454 val_454 2010-03-03 12 +375 val_375 2010-03-03 12 +401 val_401 2010-03-03 12 +421 val_421 2010-03-03 12 +407 val_407 2010-03-03 12 +384 val_384 2010-03-03 12 +256 val_256 2010-03-03 12 +26 val_26 2010-03-03 12 +134 val_134 2010-03-03 12 +67 val_67 2010-03-03 12 +384 val_384 2010-03-03 12 +379 val_379 2010-03-03 12 +18 val_18 2010-03-03 12 +462 val_462 2010-03-03 12 +492 val_492 2010-03-03 12 +100 val_100 2010-03-03 12 +298 val_298 2010-03-03 12 +9 val_9 2010-03-03 12 +341 val_341 2010-03-03 12 +498 val_498 2010-03-03 12 +146 val_146 2010-03-03 12 +458 val_458 2010-03-03 12 +362 val_362 2010-03-03 12 +186 val_186 2010-03-03 12 +285 val_285 2010-03-03 12 +348 val_348 2010-03-03 12 +167 val_167 2010-03-03 12 +18 val_18 2010-03-03 12 +273 val_273 2010-03-03 12 +183 val_183 2010-03-03 12 +281 val_281 2010-03-03 12 +344 val_344 2010-03-03 12 +97 val_97 2010-03-03 12 +469 val_469 2010-03-03 12 +315 val_315 2010-03-03 12 +84 val_84 2010-03-03 12 +28 val_28 2010-03-03 12 +37 val_37 2010-03-03 12 +448 val_448 2010-03-03 12 +152 val_152 2010-03-03 12 +348 val_348 2010-03-03 12 +307 val_307 2010-03-03 12 +194 val_194 2010-03-03 12 +414 val_414 2010-03-03 12 +477 val_477 2010-03-03 12 +222 val_222 2010-03-03 12 +126 val_126 2010-03-03 12 +90 val_90 2010-03-03 12 +169 val_169 2010-03-03 12 +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 +238 val_238 2010-03-03 12 +86 val_86 2010-03-03 12 +311 val_311 2010-03-03 12 +27 val_27 2010-03-03 12 +165 val_165 2010-03-03 12 +409 val_409 2010-03-03 12 +255 val_255 2010-03-03 12 +278 val_278 2010-03-03 12 +98 val_98 2010-03-03 12 +484 val_484 2010-03-03 12 +265 val_265 2010-03-03 12 +193 val_193 2010-03-03 12 +401 val_401 2010-03-03 12 +150 val_150 2010-03-03 12 +273 val_273 2010-03-03 12 +224 val_224 2010-03-03 12 +369 val_369 2010-03-03 12 +66 val_66 2010-03-03 12 +128 val_128 2010-03-03 12 +213 val_213 2010-03-03 12 +146 val_146 2010-03-03 12 +406 val_406 2010-03-03 12 +429 val_429 2010-03-03 12 +374 val_374 2010-03-03 12 +152 val_152 2010-03-03 12 +469 val_469 2010-03-03 12 +145 val_145 2010-03-03 12 +495 val_495 2010-03-03 12 +37 val_37 2010-03-03 12 +327 val_327 2010-03-03 12 +281 val_281 2010-03-03 12 +277 val_277 2010-03-03 12 +209 val_209 2010-03-03 12 +15 val_15 2010-03-03 12 +82 val_82 2010-03-03 12 +403 val_403 2010-03-03 12 +166 val_166 2010-03-03 12 +417 val_417 2010-03-03 12 +430 val_430 2010-03-03 12 +252 val_252 2010-03-03 12 +292 val_292 2010-03-03 12 +219 val_219 2010-03-03 12 +287 val_287 2010-03-03 12 +153 val_153 2010-03-03 12 +193 val_193 2010-03-03 12 +338 val_338 2010-03-03 12 +446 val_446 2010-03-03 12 +459 val_459 2010-03-03 12 +394 val_394 2010-03-03 12 +237 val_237 2010-03-03 12 +482 val_482 2010-03-03 12 +174 val_174 2010-03-03 12 +413 val_413 2010-03-03 12 +494 val_494 2010-03-03 12 +207 val_207 2010-03-03 12 +199 val_199 2010-03-03 12 +466 val_466 2010-03-03 12 +208 val_208 2010-03-03 12 +174 val_174 2010-03-03 12 +399 val_399 2010-03-03 12 +396 val_396 2010-03-03 12 +247 val_247 2010-03-03 12 +417 val_417 2010-03-03 12 +489 val_489 2010-03-03 12 +162 val_162 2010-03-03 12 +377 val_377 2010-03-03 12 +397 val_397 2010-03-03 12 +309 val_309 2010-03-03 12 +365 val_365 2010-03-03 12 +266 val_266 2010-03-03 12 +439 val_439 2010-03-03 12 +342 val_342 2010-03-03 12 +367 val_367 2010-03-03 12 +325 val_325 2010-03-03 12 +167 val_167 2010-03-03 12 +195 val_195 2010-03-03 12 +475 val_475 2010-03-03 12 +17 val_17 2010-03-03 12 +113 val_113 2010-03-03 12 +155 val_155 2010-03-03 12 +203 val_203 2010-03-03 12 +339 val_339 2010-03-03 12 +0 val_0 2010-03-03 12 +455 val_455 2010-03-03 12 +128 val_128 2010-03-03 12 +311 val_311 2010-03-03 12 +316 val_316 2010-03-03 12 +57 val_57 2010-03-03 12 +302 val_302 2010-03-03 12 +205 val_205 2010-03-03 12 +149 val_149 2010-03-03 12 +438 val_438 2010-03-03 12 +345 val_345 2010-03-03 12 +129 val_129 2010-03-03 12 +170 val_170 2010-03-03 12 +20 val_20 2010-03-03 12 +489 val_489 2010-03-03 12 +157 val_157 2010-03-03 12 +378 val_378 2010-03-03 12 +221 val_221 2010-03-03 12 +92 val_92 2010-03-03 12 +111 val_111 2010-03-03 12 +47 val_47 2010-03-03 12 +72 val_72 2010-03-03 12 +4 val_4 2010-03-03 12 +280 val_280 2010-03-03 12 +35 val_35 2010-03-03 12 +427 val_427 2010-03-03 12 +277 val_277 2010-03-03 12 +208 val_208 2010-03-03 12 +356 val_356 2010-03-03 12 +399 val_399 2010-03-03 12 +169 val_169 2010-03-03 12 +382 val_382 2010-03-03 12 +498 val_498 2010-03-03 12 +125 val_125 2010-03-03 12 +386 val_386 2010-03-03 12 +437 val_437 2010-03-03 12 +469 val_469 2010-03-03 12 +192 val_192 2010-03-03 12 +286 val_286 2010-03-03 12 +187 val_187 2010-03-03 12 +176 val_176 2010-03-03 12 +54 val_54 2010-03-03 12 +459 val_459 2010-03-03 12 +51 val_51 2010-03-03 12 +138 val_138 2010-03-03 12 +103 val_103 2010-03-03 12 +239 val_239 2010-03-03 12 +213 val_213 2010-03-03 12 +216 val_216 2010-03-03 12 +430 val_430 2010-03-03 12 +278 val_278 2010-03-03 12 +176 val_176 2010-03-03 12 +289 val_289 2010-03-03 12 +221 val_221 2010-03-03 12 +65 val_65 2010-03-03 12 +318 val_318 2010-03-03 12 +332 val_332 2010-03-03 12 +311 val_311 2010-03-03 12 +275 val_275 2010-03-03 12 +137 val_137 2010-03-03 12 +241 val_241 2010-03-03 12 +83 val_83 2010-03-03 12 +333 val_333 2010-03-03 12 +180 val_180 2010-03-03 12 +284 val_284 2010-03-03 12 +12 val_12 2010-03-03 12 +230 val_230 2010-03-03 12 +181 val_181 2010-03-03 12 +67 val_67 2010-03-03 12 +260 val_260 2010-03-03 12 +404 val_404 2010-03-03 12 +384 val_384 2010-03-03 12 +489 val_489 2010-03-03 12 +353 val_353 2010-03-03 12 +373 val_373 2010-03-03 12 +272 val_272 2010-03-03 12 +138 val_138 2010-03-03 12 +217 val_217 2010-03-03 12 +84 val_84 2010-03-03 12 +348 val_348 2010-03-03 12 +466 val_466 2010-03-03 12 +58 val_58 2010-03-03 12 +8 val_8 2010-03-03 12 +411 val_411 2010-03-03 12 +230 val_230 2010-03-03 12 +208 val_208 2010-03-03 12 +348 val_348 2010-03-03 12 +24 val_24 2010-03-03 12 +463 val_463 2010-03-03 12 +431 val_431 2010-03-03 12 +179 val_179 2010-03-03 12 +172 val_172 2010-03-03 12 +42 val_42 2010-03-03 12 +129 val_129 2010-03-03 12 +158 val_158 2010-03-03 12 +119 val_119 2010-03-03 12 +496 val_496 2010-03-03 12 +0 val_0 2010-03-03 12 +322 val_322 2010-03-03 12 +197 val_197 2010-03-03 12 +468 val_468 2010-03-03 12 +393 val_393 2010-03-03 12 +454 val_454 2010-03-03 12 +100 val_100 2010-03-03 12 +298 val_298 2010-03-03 12 +199 val_199 2010-03-03 12 +191 val_191 2010-03-03 12 +418 val_418 2010-03-03 12 +96 val_96 2010-03-03 12 +26 val_26 2010-03-03 12 +165 val_165 2010-03-03 12 +327 val_327 2010-03-03 12 +230 val_230 2010-03-03 12 +205 val_205 2010-03-03 12 +120 val_120 2010-03-03 12 +131 val_131 2010-03-03 12 +51 val_51 2010-03-03 12 +404 val_404 2010-03-03 12 +43 val_43 2010-03-03 12 +436 val_436 2010-03-03 12 +156 val_156 2010-03-03 12 +469 val_469 2010-03-03 12 +468 val_468 2010-03-03 12 +308 val_308 2010-03-03 12 +95 val_95 2010-03-03 12 +196 val_196 2010-03-03 12 +288 val_288 2010-03-03 12 +481 val_481 2010-03-03 12 +457 val_457 2010-03-03 12 +98 val_98 2010-03-03 12 +282 val_282 2010-03-03 12 +197 val_197 2010-03-03 12 +187 val_187 2010-03-03 12 +318 val_318 2010-03-03 12 +318 val_318 2010-03-03 12 +409 val_409 2010-03-03 12 +470 val_470 2010-03-03 12 +137 val_137 2010-03-03 12 +369 val_369 2010-03-03 12 +316 val_316 2010-03-03 12 +169 val_169 2010-03-03 12 +413 val_413 2010-03-03 12 +85 val_85 2010-03-03 12 +77 val_77 2010-03-03 12 +0 val_0 2010-03-03 12 +490 val_490 2010-03-03 12 +87 val_87 2010-03-03 12 +364 val_364 2010-03-03 12 +179 val_179 2010-03-03 12 +118 val_118 2010-03-03 12 +134 val_134 2010-03-03 12 +395 val_395 2010-03-03 12 +282 val_282 2010-03-03 12 +138 val_138 2010-03-03 12 +238 val_238 2010-03-03 12 +419 val_419 2010-03-03 12 +15 val_15 2010-03-03 12 +118 val_118 2010-03-03 12 +72 val_72 2010-03-03 12 +90 val_90 2010-03-03 12 +307 val_307 2010-03-03 12 +19 val_19 2010-03-03 12 +435 val_435 2010-03-03 12 +10 val_10 2010-03-03 12 +277 val_277 2010-03-03 12 +273 val_273 2010-03-03 12 +306 val_306 2010-03-03 12 +224 val_224 2010-03-03 12 +309 val_309 2010-03-03 12 +389 val_389 2010-03-03 12 +327 val_327 2010-03-03 12 +242 val_242 2010-03-03 12 +369 val_369 2010-03-03 12 +392 val_392 2010-03-03 12 +272 val_272 2010-03-03 12 +331 val_331 2010-03-03 12 +401 val_401 2010-03-03 12 +242 val_242 2010-03-03 12 +452 val_452 2010-03-03 12 +177 val_177 2010-03-03 12 +226 val_226 2010-03-03 12 +5 val_5 2010-03-03 12 +497 val_497 2010-03-03 12 +402 val_402 2010-03-03 12 +396 val_396 2010-03-03 12 +317 val_317 2010-03-03 12 +395 val_395 2010-03-03 12 +58 val_58 2010-03-03 12 +35 val_35 2010-03-03 12 +336 val_336 2010-03-03 12 +95 val_95 2010-03-03 12 +11 val_11 2010-03-03 12 +168 val_168 2010-03-03 12 +34 val_34 2010-03-03 12 +229 val_229 2010-03-03 12 +233 val_233 2010-03-03 12 +143 val_143 2010-03-03 12 +472 val_472 2010-03-03 12 +322 val_322 2010-03-03 12 +498 val_498 2010-03-03 12 +160 val_160 2010-03-03 12 +195 val_195 2010-03-03 12 +42 val_42 2010-03-03 12 +321 val_321 2010-03-03 12 +430 val_430 2010-03-03 12 +119 val_119 2010-03-03 12 +489 val_489 2010-03-03 12 +458 val_458 2010-03-03 12 +78 val_78 2010-03-03 12 +76 val_76 2010-03-03 12 +41 val_41 2010-03-03 12 +223 val_223 2010-03-03 12 +492 val_492 2010-03-03 12 +149 val_149 2010-03-03 12 +449 val_449 2010-03-03 12 +218 val_218 2010-03-03 12 +228 val_228 2010-03-03 12 +138 val_138 2010-03-03 12 +453 val_453 2010-03-03 12 +30 val_30 2010-03-03 12 +209 val_209 2010-03-03 12 +64 val_64 2010-03-03 12 +468 val_468 2010-03-03 12 +76 val_76 2010-03-03 12 +74 val_74 2010-03-03 12 +342 val_342 2010-03-03 12 +69 val_69 2010-03-03 12 +230 val_230 2010-03-03 12 +33 val_33 2010-03-03 12 +368 val_368 2010-03-03 12 +103 val_103 2010-03-03 12 +296 val_296 2010-03-03 12 +113 val_113 2010-03-03 12 +216 val_216 2010-03-03 12 +367 val_367 2010-03-03 12 +344 val_344 2010-03-03 12 +167 val_167 2010-03-03 12 +274 val_274 2010-03-03 12 +219 val_219 2010-03-03 12 +239 val_239 2010-03-03 12 +485 val_485 2010-03-03 12 +116 val_116 2010-03-03 12 +223 val_223 2010-03-03 12 +256 val_256 2010-03-03 12 +263 val_263 2010-03-03 12 +70 val_70 2010-03-03 12 +487 val_487 2010-03-03 12 +480 val_480 2010-03-03 12 +401 val_401 2010-03-03 12 +288 val_288 2010-03-03 12 +191 val_191 2010-03-03 12 +5 val_5 2010-03-03 12 +244 val_244 2010-03-03 12 +438 val_438 2010-03-03 12 +128 val_128 2010-03-03 12 +467 val_467 2010-03-03 12 +432 val_432 2010-03-03 12 +202 val_202 2010-03-03 12 +316 val_316 2010-03-03 12 +229 val_229 2010-03-03 12 +469 val_469 2010-03-03 12 +463 val_463 2010-03-03 12 +280 val_280 2010-03-03 12 +2 val_2 2010-03-03 12 +35 val_35 2010-03-03 12 +283 val_283 2010-03-03 12 +331 val_331 2010-03-03 12 +235 val_235 2010-03-03 12 +80 val_80 2010-03-03 12 +44 val_44 2010-03-03 12 +193 val_193 2010-03-03 12 +321 val_321 2010-03-03 12 +335 val_335 2010-03-03 12 +104 val_104 2010-03-03 12 +466 val_466 2010-03-03 12 +366 val_366 2010-03-03 12 +175 val_175 2010-03-03 12 +403 val_403 2010-03-03 12 +483 val_483 2010-03-03 12 +53 val_53 2010-03-03 12 +105 val_105 2010-03-03 12 +257 val_257 2010-03-03 12 +406 val_406 2010-03-03 12 +409 val_409 2010-03-03 12 +190 val_190 2010-03-03 12 +406 val_406 2010-03-03 12 +401 val_401 2010-03-03 12 +114 val_114 2010-03-03 12 +258 val_258 2010-03-03 12 +90 val_90 2010-03-03 12 +203 val_203 2010-03-03 12 +262 val_262 2010-03-03 12 +348 val_348 2010-03-03 12 +424 val_424 2010-03-03 12 +12 val_12 2010-03-03 12 +396 val_396 2010-03-03 12 +201 val_201 2010-03-03 12 +217 val_217 2010-03-03 12 +164 val_164 2010-03-03 12 +431 val_431 2010-03-03 12 +454 val_454 2010-03-03 12 +478 val_478 2010-03-03 12 +298 val_298 2010-03-03 12 +125 val_125 2010-03-03 12 +431 val_431 2010-03-03 12 +164 val_164 2010-03-03 12 +424 val_424 2010-03-03 12 +187 val_187 2010-03-03 12 +382 val_382 2010-03-03 12 +5 val_5 2010-03-03 12 +70 val_70 2010-03-03 12 +397 val_397 2010-03-03 12 +480 val_480 2010-03-03 12 +291 val_291 2010-03-03 12 +24 val_24 2010-03-03 12 +351 val_351 2010-03-03 12 +255 val_255 2010-03-03 12 +104 val_104 2010-03-03 12 +70 val_70 2010-03-03 12 +163 val_163 2010-03-03 12 +438 val_438 2010-03-03 12 +119 val_119 2010-03-03 12 +414 val_414 2010-03-03 12 +200 val_200 2010-03-03 12 +491 val_491 2010-03-03 12 +237 val_237 2010-03-03 12 +439 val_439 2010-03-03 12 +360 val_360 2010-03-03 12 +248 val_248 2010-03-03 12 +479 val_479 2010-03-03 12 +305 val_305 2010-03-03 12 +417 val_417 2010-03-03 12 +199 val_199 2010-03-03 12 +444 val_444 2010-03-03 12 +120 val_120 2010-03-03 12 +429 val_429 2010-03-03 12 +169 val_169 2010-03-03 12 +443 val_443 2010-03-03 12 +323 val_323 2010-03-03 12 +325 val_325 2010-03-03 12 +277 val_277 2010-03-03 12 +230 val_230 2010-03-03 12 +478 val_478 2010-03-03 12 +178 val_178 2010-03-03 12 +468 val_468 2010-03-03 12 +310 val_310 2010-03-03 12 +317 val_317 2010-03-03 12 +333 val_333 2010-03-03 12 +493 val_493 2010-03-03 12 +460 val_460 2010-03-03 12 +207 val_207 2010-03-03 12 +249 val_249 2010-03-03 12 +265 val_265 2010-03-03 12 +480 val_480 2010-03-03 12 +83 val_83 2010-03-03 12 +136 val_136 2010-03-03 12 +353 val_353 2010-03-03 12 +172 val_172 2010-03-03 12 +214 val_214 2010-03-03 12 +462 val_462 2010-03-03 12 +233 val_233 2010-03-03 12 +406 val_406 2010-03-03 12 +133 val_133 2010-03-03 12 +175 val_175 2010-03-03 12 +189 val_189 2010-03-03 12 +454 val_454 2010-03-03 12 +375 val_375 2010-03-03 12 +401 val_401 2010-03-03 12 +421 val_421 2010-03-03 12 +407 val_407 2010-03-03 12 +384 val_384 2010-03-03 12 +256 val_256 2010-03-03 12 +26 val_26 2010-03-03 12 +134 val_134 2010-03-03 12 +67 val_67 2010-03-03 12 +384 val_384 2010-03-03 12 +379 val_379 2010-03-03 12 +18 val_18 2010-03-03 12 +462 val_462 2010-03-03 12 +492 val_492 2010-03-03 12 +100 val_100 2010-03-03 12 +298 val_298 2010-03-03 12 +9 val_9 2010-03-03 12 +341 val_341 2010-03-03 12 +498 val_498 2010-03-03 12 +146 val_146 2010-03-03 12 +458 val_458 2010-03-03 12 +362 val_362 2010-03-03 12 +186 val_186 2010-03-03 12 +285 val_285 2010-03-03 12 +348 val_348 2010-03-03 12 +167 val_167 2010-03-03 12 +18 val_18 2010-03-03 12 +273 val_273 2010-03-03 12 +183 val_183 2010-03-03 12 +281 val_281 2010-03-03 12 +344 val_344 2010-03-03 12 +97 val_97 2010-03-03 12 +469 val_469 2010-03-03 12 +315 val_315 2010-03-03 12 +84 val_84 2010-03-03 12 +28 val_28 2010-03-03 12 +37 val_37 2010-03-03 12 +448 val_448 2010-03-03 12 +152 val_152 2010-03-03 12 +348 val_348 2010-03-03 12 +307 val_307 2010-03-03 12 +194 val_194 2010-03-03 12 +414 val_414 2010-03-03 12 +477 val_477 2010-03-03 12 +222 val_222 2010-03-03 12 +126 val_126 2010-03-03 12 +90 val_90 2010-03-03 12 +169 val_169 2010-03-03 12 +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 diff --git a/src/test/hive/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/src/test/hive/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 b/src/test/hive/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b b/src/test/hive/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b new file mode 100644 index 0000000000000..0c283c5378a6f --- /dev/null +++ b/src/test/hive/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b @@ -0,0 +1,12 @@ +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 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 diff --git a/src/test/hive/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 b/src/test/hive/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 b/src/test/hive/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 new file mode 100644 index 0000000000000..8eb67466a7a89 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 @@ -0,0 +1,2 @@ +ds=2010-03-03/hr=22 +ds=2010-03-03/hr=24 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 b/src/test/hive/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 new file mode 100644 index 0000000000000..9f50245c18177 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 @@ -0,0 +1,2000 @@ +238 val_238 2010-03-03 22 +86 val_86 2010-03-03 22 +311 val_311 2010-03-03 22 +27 val_27 2010-03-03 22 +165 val_165 2010-03-03 22 +409 val_409 2010-03-03 22 +255 val_255 2010-03-03 22 +278 val_278 2010-03-03 22 +98 val_98 2010-03-03 22 +484 val_484 2010-03-03 22 +265 val_265 2010-03-03 22 +193 val_193 2010-03-03 22 +401 val_401 2010-03-03 22 +150 val_150 2010-03-03 22 +273 val_273 2010-03-03 22 +224 val_224 2010-03-03 22 +369 val_369 2010-03-03 22 +66 val_66 2010-03-03 22 +128 val_128 2010-03-03 22 +213 val_213 2010-03-03 22 +146 val_146 2010-03-03 22 +406 val_406 2010-03-03 22 +429 val_429 2010-03-03 22 +374 val_374 2010-03-03 22 +152 val_152 2010-03-03 22 +469 val_469 2010-03-03 22 +145 val_145 2010-03-03 22 +495 val_495 2010-03-03 22 +37 val_37 2010-03-03 22 +327 val_327 2010-03-03 22 +281 val_281 2010-03-03 22 +277 val_277 2010-03-03 22 +209 val_209 2010-03-03 22 +15 val_15 2010-03-03 22 +82 val_82 2010-03-03 22 +403 val_403 2010-03-03 22 +166 val_166 2010-03-03 22 +417 val_417 2010-03-03 22 +430 val_430 2010-03-03 22 +252 val_252 2010-03-03 22 +292 val_292 2010-03-03 22 +219 val_219 2010-03-03 22 +287 val_287 2010-03-03 22 +153 val_153 2010-03-03 22 +193 val_193 2010-03-03 22 +338 val_338 2010-03-03 22 +446 val_446 2010-03-03 22 +459 val_459 2010-03-03 22 +394 val_394 2010-03-03 22 +237 val_237 2010-03-03 22 +482 val_482 2010-03-03 22 +174 val_174 2010-03-03 22 +413 val_413 2010-03-03 22 +494 val_494 2010-03-03 22 +207 val_207 2010-03-03 22 +199 val_199 2010-03-03 22 +466 val_466 2010-03-03 22 +208 val_208 2010-03-03 22 +174 val_174 2010-03-03 22 +399 val_399 2010-03-03 22 +396 val_396 2010-03-03 22 +247 val_247 2010-03-03 22 +417 val_417 2010-03-03 22 +489 val_489 2010-03-03 22 +162 val_162 2010-03-03 22 +377 val_377 2010-03-03 22 +397 val_397 2010-03-03 22 +309 val_309 2010-03-03 22 +365 val_365 2010-03-03 22 +266 val_266 2010-03-03 22 +439 val_439 2010-03-03 22 +342 val_342 2010-03-03 22 +367 val_367 2010-03-03 22 +325 val_325 2010-03-03 22 +167 val_167 2010-03-03 22 +195 val_195 2010-03-03 22 +475 val_475 2010-03-03 22 +17 val_17 2010-03-03 22 +113 val_113 2010-03-03 22 +155 val_155 2010-03-03 22 +203 val_203 2010-03-03 22 +339 val_339 2010-03-03 22 +0 val_0 2010-03-03 22 +455 val_455 2010-03-03 22 +128 val_128 2010-03-03 22 +311 val_311 2010-03-03 22 +316 val_316 2010-03-03 22 +57 val_57 2010-03-03 22 +302 val_302 2010-03-03 22 +205 val_205 2010-03-03 22 +149 val_149 2010-03-03 22 +438 val_438 2010-03-03 22 +345 val_345 2010-03-03 22 +129 val_129 2010-03-03 22 +170 val_170 2010-03-03 22 +20 val_20 2010-03-03 22 +489 val_489 2010-03-03 22 +157 val_157 2010-03-03 22 +378 val_378 2010-03-03 22 +221 val_221 2010-03-03 22 +92 val_92 2010-03-03 22 +111 val_111 2010-03-03 22 +47 val_47 2010-03-03 22 +72 val_72 2010-03-03 22 +4 val_4 2010-03-03 22 +280 val_280 2010-03-03 22 +35 val_35 2010-03-03 22 +427 val_427 2010-03-03 22 +277 val_277 2010-03-03 22 +208 val_208 2010-03-03 22 +356 val_356 2010-03-03 22 +399 val_399 2010-03-03 22 +169 val_169 2010-03-03 22 +382 val_382 2010-03-03 22 +498 val_498 2010-03-03 22 +125 val_125 2010-03-03 22 +386 val_386 2010-03-03 22 +437 val_437 2010-03-03 22 +469 val_469 2010-03-03 22 +192 val_192 2010-03-03 22 +286 val_286 2010-03-03 22 +187 val_187 2010-03-03 22 +176 val_176 2010-03-03 22 +54 val_54 2010-03-03 22 +459 val_459 2010-03-03 22 +51 val_51 2010-03-03 22 +138 val_138 2010-03-03 22 +103 val_103 2010-03-03 22 +239 val_239 2010-03-03 22 +213 val_213 2010-03-03 22 +216 val_216 2010-03-03 22 +430 val_430 2010-03-03 22 +278 val_278 2010-03-03 22 +176 val_176 2010-03-03 22 +289 val_289 2010-03-03 22 +221 val_221 2010-03-03 22 +65 val_65 2010-03-03 22 +318 val_318 2010-03-03 22 +332 val_332 2010-03-03 22 +311 val_311 2010-03-03 22 +275 val_275 2010-03-03 22 +137 val_137 2010-03-03 22 +241 val_241 2010-03-03 22 +83 val_83 2010-03-03 22 +333 val_333 2010-03-03 22 +180 val_180 2010-03-03 22 +284 val_284 2010-03-03 22 +12 val_12 2010-03-03 22 +230 val_230 2010-03-03 22 +181 val_181 2010-03-03 22 +67 val_67 2010-03-03 22 +260 val_260 2010-03-03 22 +404 val_404 2010-03-03 22 +384 val_384 2010-03-03 22 +489 val_489 2010-03-03 22 +353 val_353 2010-03-03 22 +373 val_373 2010-03-03 22 +272 val_272 2010-03-03 22 +138 val_138 2010-03-03 22 +217 val_217 2010-03-03 22 +84 val_84 2010-03-03 22 +348 val_348 2010-03-03 22 +466 val_466 2010-03-03 22 +58 val_58 2010-03-03 22 +8 val_8 2010-03-03 22 +411 val_411 2010-03-03 22 +230 val_230 2010-03-03 22 +208 val_208 2010-03-03 22 +348 val_348 2010-03-03 22 +24 val_24 2010-03-03 22 +463 val_463 2010-03-03 22 +431 val_431 2010-03-03 22 +179 val_179 2010-03-03 22 +172 val_172 2010-03-03 22 +42 val_42 2010-03-03 22 +129 val_129 2010-03-03 22 +158 val_158 2010-03-03 22 +119 val_119 2010-03-03 22 +496 val_496 2010-03-03 22 +0 val_0 2010-03-03 22 +322 val_322 2010-03-03 22 +197 val_197 2010-03-03 22 +468 val_468 2010-03-03 22 +393 val_393 2010-03-03 22 +454 val_454 2010-03-03 22 +100 val_100 2010-03-03 22 +298 val_298 2010-03-03 22 +199 val_199 2010-03-03 22 +191 val_191 2010-03-03 22 +418 val_418 2010-03-03 22 +96 val_96 2010-03-03 22 +26 val_26 2010-03-03 22 +165 val_165 2010-03-03 22 +327 val_327 2010-03-03 22 +230 val_230 2010-03-03 22 +205 val_205 2010-03-03 22 +120 val_120 2010-03-03 22 +131 val_131 2010-03-03 22 +51 val_51 2010-03-03 22 +404 val_404 2010-03-03 22 +43 val_43 2010-03-03 22 +436 val_436 2010-03-03 22 +156 val_156 2010-03-03 22 +469 val_469 2010-03-03 22 +468 val_468 2010-03-03 22 +308 val_308 2010-03-03 22 +95 val_95 2010-03-03 22 +196 val_196 2010-03-03 22 +288 val_288 2010-03-03 22 +481 val_481 2010-03-03 22 +457 val_457 2010-03-03 22 +98 val_98 2010-03-03 22 +282 val_282 2010-03-03 22 +197 val_197 2010-03-03 22 +187 val_187 2010-03-03 22 +318 val_318 2010-03-03 22 +318 val_318 2010-03-03 22 +409 val_409 2010-03-03 22 +470 val_470 2010-03-03 22 +137 val_137 2010-03-03 22 +369 val_369 2010-03-03 22 +316 val_316 2010-03-03 22 +169 val_169 2010-03-03 22 +413 val_413 2010-03-03 22 +85 val_85 2010-03-03 22 +77 val_77 2010-03-03 22 +0 val_0 2010-03-03 22 +490 val_490 2010-03-03 22 +87 val_87 2010-03-03 22 +364 val_364 2010-03-03 22 +179 val_179 2010-03-03 22 +118 val_118 2010-03-03 22 +134 val_134 2010-03-03 22 +395 val_395 2010-03-03 22 +282 val_282 2010-03-03 22 +138 val_138 2010-03-03 22 +238 val_238 2010-03-03 22 +419 val_419 2010-03-03 22 +15 val_15 2010-03-03 22 +118 val_118 2010-03-03 22 +72 val_72 2010-03-03 22 +90 val_90 2010-03-03 22 +307 val_307 2010-03-03 22 +19 val_19 2010-03-03 22 +435 val_435 2010-03-03 22 +10 val_10 2010-03-03 22 +277 val_277 2010-03-03 22 +273 val_273 2010-03-03 22 +306 val_306 2010-03-03 22 +224 val_224 2010-03-03 22 +309 val_309 2010-03-03 22 +389 val_389 2010-03-03 22 +327 val_327 2010-03-03 22 +242 val_242 2010-03-03 22 +369 val_369 2010-03-03 22 +392 val_392 2010-03-03 22 +272 val_272 2010-03-03 22 +331 val_331 2010-03-03 22 +401 val_401 2010-03-03 22 +242 val_242 2010-03-03 22 +452 val_452 2010-03-03 22 +177 val_177 2010-03-03 22 +226 val_226 2010-03-03 22 +5 val_5 2010-03-03 22 +497 val_497 2010-03-03 22 +402 val_402 2010-03-03 22 +396 val_396 2010-03-03 22 +317 val_317 2010-03-03 22 +395 val_395 2010-03-03 22 +58 val_58 2010-03-03 22 +35 val_35 2010-03-03 22 +336 val_336 2010-03-03 22 +95 val_95 2010-03-03 22 +11 val_11 2010-03-03 22 +168 val_168 2010-03-03 22 +34 val_34 2010-03-03 22 +229 val_229 2010-03-03 22 +233 val_233 2010-03-03 22 +143 val_143 2010-03-03 22 +472 val_472 2010-03-03 22 +322 val_322 2010-03-03 22 +498 val_498 2010-03-03 22 +160 val_160 2010-03-03 22 +195 val_195 2010-03-03 22 +42 val_42 2010-03-03 22 +321 val_321 2010-03-03 22 +430 val_430 2010-03-03 22 +119 val_119 2010-03-03 22 +489 val_489 2010-03-03 22 +458 val_458 2010-03-03 22 +78 val_78 2010-03-03 22 +76 val_76 2010-03-03 22 +41 val_41 2010-03-03 22 +223 val_223 2010-03-03 22 +492 val_492 2010-03-03 22 +149 val_149 2010-03-03 22 +449 val_449 2010-03-03 22 +218 val_218 2010-03-03 22 +228 val_228 2010-03-03 22 +138 val_138 2010-03-03 22 +453 val_453 2010-03-03 22 +30 val_30 2010-03-03 22 +209 val_209 2010-03-03 22 +64 val_64 2010-03-03 22 +468 val_468 2010-03-03 22 +76 val_76 2010-03-03 22 +74 val_74 2010-03-03 22 +342 val_342 2010-03-03 22 +69 val_69 2010-03-03 22 +230 val_230 2010-03-03 22 +33 val_33 2010-03-03 22 +368 val_368 2010-03-03 22 +103 val_103 2010-03-03 22 +296 val_296 2010-03-03 22 +113 val_113 2010-03-03 22 +216 val_216 2010-03-03 22 +367 val_367 2010-03-03 22 +344 val_344 2010-03-03 22 +167 val_167 2010-03-03 22 +274 val_274 2010-03-03 22 +219 val_219 2010-03-03 22 +239 val_239 2010-03-03 22 +485 val_485 2010-03-03 22 +116 val_116 2010-03-03 22 +223 val_223 2010-03-03 22 +256 val_256 2010-03-03 22 +263 val_263 2010-03-03 22 +70 val_70 2010-03-03 22 +487 val_487 2010-03-03 22 +480 val_480 2010-03-03 22 +401 val_401 2010-03-03 22 +288 val_288 2010-03-03 22 +191 val_191 2010-03-03 22 +5 val_5 2010-03-03 22 +244 val_244 2010-03-03 22 +438 val_438 2010-03-03 22 +128 val_128 2010-03-03 22 +467 val_467 2010-03-03 22 +432 val_432 2010-03-03 22 +202 val_202 2010-03-03 22 +316 val_316 2010-03-03 22 +229 val_229 2010-03-03 22 +469 val_469 2010-03-03 22 +463 val_463 2010-03-03 22 +280 val_280 2010-03-03 22 +2 val_2 2010-03-03 22 +35 val_35 2010-03-03 22 +283 val_283 2010-03-03 22 +331 val_331 2010-03-03 22 +235 val_235 2010-03-03 22 +80 val_80 2010-03-03 22 +44 val_44 2010-03-03 22 +193 val_193 2010-03-03 22 +321 val_321 2010-03-03 22 +335 val_335 2010-03-03 22 +104 val_104 2010-03-03 22 +466 val_466 2010-03-03 22 +366 val_366 2010-03-03 22 +175 val_175 2010-03-03 22 +403 val_403 2010-03-03 22 +483 val_483 2010-03-03 22 +53 val_53 2010-03-03 22 +105 val_105 2010-03-03 22 +257 val_257 2010-03-03 22 +406 val_406 2010-03-03 22 +409 val_409 2010-03-03 22 +190 val_190 2010-03-03 22 +406 val_406 2010-03-03 22 +401 val_401 2010-03-03 22 +114 val_114 2010-03-03 22 +258 val_258 2010-03-03 22 +90 val_90 2010-03-03 22 +203 val_203 2010-03-03 22 +262 val_262 2010-03-03 22 +348 val_348 2010-03-03 22 +424 val_424 2010-03-03 22 +12 val_12 2010-03-03 22 +396 val_396 2010-03-03 22 +201 val_201 2010-03-03 22 +217 val_217 2010-03-03 22 +164 val_164 2010-03-03 22 +431 val_431 2010-03-03 22 +454 val_454 2010-03-03 22 +478 val_478 2010-03-03 22 +298 val_298 2010-03-03 22 +125 val_125 2010-03-03 22 +431 val_431 2010-03-03 22 +164 val_164 2010-03-03 22 +424 val_424 2010-03-03 22 +187 val_187 2010-03-03 22 +382 val_382 2010-03-03 22 +5 val_5 2010-03-03 22 +70 val_70 2010-03-03 22 +397 val_397 2010-03-03 22 +480 val_480 2010-03-03 22 +291 val_291 2010-03-03 22 +24 val_24 2010-03-03 22 +351 val_351 2010-03-03 22 +255 val_255 2010-03-03 22 +104 val_104 2010-03-03 22 +70 val_70 2010-03-03 22 +163 val_163 2010-03-03 22 +438 val_438 2010-03-03 22 +119 val_119 2010-03-03 22 +414 val_414 2010-03-03 22 +200 val_200 2010-03-03 22 +491 val_491 2010-03-03 22 +237 val_237 2010-03-03 22 +439 val_439 2010-03-03 22 +360 val_360 2010-03-03 22 +248 val_248 2010-03-03 22 +479 val_479 2010-03-03 22 +305 val_305 2010-03-03 22 +417 val_417 2010-03-03 22 +199 val_199 2010-03-03 22 +444 val_444 2010-03-03 22 +120 val_120 2010-03-03 22 +429 val_429 2010-03-03 22 +169 val_169 2010-03-03 22 +443 val_443 2010-03-03 22 +323 val_323 2010-03-03 22 +325 val_325 2010-03-03 22 +277 val_277 2010-03-03 22 +230 val_230 2010-03-03 22 +478 val_478 2010-03-03 22 +178 val_178 2010-03-03 22 +468 val_468 2010-03-03 22 +310 val_310 2010-03-03 22 +317 val_317 2010-03-03 22 +333 val_333 2010-03-03 22 +493 val_493 2010-03-03 22 +460 val_460 2010-03-03 22 +207 val_207 2010-03-03 22 +249 val_249 2010-03-03 22 +265 val_265 2010-03-03 22 +480 val_480 2010-03-03 22 +83 val_83 2010-03-03 22 +136 val_136 2010-03-03 22 +353 val_353 2010-03-03 22 +172 val_172 2010-03-03 22 +214 val_214 2010-03-03 22 +462 val_462 2010-03-03 22 +233 val_233 2010-03-03 22 +406 val_406 2010-03-03 22 +133 val_133 2010-03-03 22 +175 val_175 2010-03-03 22 +189 val_189 2010-03-03 22 +454 val_454 2010-03-03 22 +375 val_375 2010-03-03 22 +401 val_401 2010-03-03 22 +421 val_421 2010-03-03 22 +407 val_407 2010-03-03 22 +384 val_384 2010-03-03 22 +256 val_256 2010-03-03 22 +26 val_26 2010-03-03 22 +134 val_134 2010-03-03 22 +67 val_67 2010-03-03 22 +384 val_384 2010-03-03 22 +379 val_379 2010-03-03 22 +18 val_18 2010-03-03 22 +462 val_462 2010-03-03 22 +492 val_492 2010-03-03 22 +100 val_100 2010-03-03 22 +298 val_298 2010-03-03 22 +9 val_9 2010-03-03 22 +341 val_341 2010-03-03 22 +498 val_498 2010-03-03 22 +146 val_146 2010-03-03 22 +458 val_458 2010-03-03 22 +362 val_362 2010-03-03 22 +186 val_186 2010-03-03 22 +285 val_285 2010-03-03 22 +348 val_348 2010-03-03 22 +167 val_167 2010-03-03 22 +18 val_18 2010-03-03 22 +273 val_273 2010-03-03 22 +183 val_183 2010-03-03 22 +281 val_281 2010-03-03 22 +344 val_344 2010-03-03 22 +97 val_97 2010-03-03 22 +469 val_469 2010-03-03 22 +315 val_315 2010-03-03 22 +84 val_84 2010-03-03 22 +28 val_28 2010-03-03 22 +37 val_37 2010-03-03 22 +448 val_448 2010-03-03 22 +152 val_152 2010-03-03 22 +348 val_348 2010-03-03 22 +307 val_307 2010-03-03 22 +194 val_194 2010-03-03 22 +414 val_414 2010-03-03 22 +477 val_477 2010-03-03 22 +222 val_222 2010-03-03 22 +126 val_126 2010-03-03 22 +90 val_90 2010-03-03 22 +169 val_169 2010-03-03 22 +403 val_403 2010-03-03 22 +400 val_400 2010-03-03 22 +200 val_200 2010-03-03 22 +97 val_97 2010-03-03 22 +238 val_238 2010-03-03 22 +86 val_86 2010-03-03 22 +311 val_311 2010-03-03 22 +27 val_27 2010-03-03 22 +165 val_165 2010-03-03 22 +409 val_409 2010-03-03 22 +255 val_255 2010-03-03 22 +278 val_278 2010-03-03 22 +98 val_98 2010-03-03 22 +484 val_484 2010-03-03 22 +265 val_265 2010-03-03 22 +193 val_193 2010-03-03 22 +401 val_401 2010-03-03 22 +150 val_150 2010-03-03 22 +273 val_273 2010-03-03 22 +224 val_224 2010-03-03 22 +369 val_369 2010-03-03 22 +66 val_66 2010-03-03 22 +128 val_128 2010-03-03 22 +213 val_213 2010-03-03 22 +146 val_146 2010-03-03 22 +406 val_406 2010-03-03 22 +429 val_429 2010-03-03 22 +374 val_374 2010-03-03 22 +152 val_152 2010-03-03 22 +469 val_469 2010-03-03 22 +145 val_145 2010-03-03 22 +495 val_495 2010-03-03 22 +37 val_37 2010-03-03 22 +327 val_327 2010-03-03 22 +281 val_281 2010-03-03 22 +277 val_277 2010-03-03 22 +209 val_209 2010-03-03 22 +15 val_15 2010-03-03 22 +82 val_82 2010-03-03 22 +403 val_403 2010-03-03 22 +166 val_166 2010-03-03 22 +417 val_417 2010-03-03 22 +430 val_430 2010-03-03 22 +252 val_252 2010-03-03 22 +292 val_292 2010-03-03 22 +219 val_219 2010-03-03 22 +287 val_287 2010-03-03 22 +153 val_153 2010-03-03 22 +193 val_193 2010-03-03 22 +338 val_338 2010-03-03 22 +446 val_446 2010-03-03 22 +459 val_459 2010-03-03 22 +394 val_394 2010-03-03 22 +237 val_237 2010-03-03 22 +482 val_482 2010-03-03 22 +174 val_174 2010-03-03 22 +413 val_413 2010-03-03 22 +494 val_494 2010-03-03 22 +207 val_207 2010-03-03 22 +199 val_199 2010-03-03 22 +466 val_466 2010-03-03 22 +208 val_208 2010-03-03 22 +174 val_174 2010-03-03 22 +399 val_399 2010-03-03 22 +396 val_396 2010-03-03 22 +247 val_247 2010-03-03 22 +417 val_417 2010-03-03 22 +489 val_489 2010-03-03 22 +162 val_162 2010-03-03 22 +377 val_377 2010-03-03 22 +397 val_397 2010-03-03 22 +309 val_309 2010-03-03 22 +365 val_365 2010-03-03 22 +266 val_266 2010-03-03 22 +439 val_439 2010-03-03 22 +342 val_342 2010-03-03 22 +367 val_367 2010-03-03 22 +325 val_325 2010-03-03 22 +167 val_167 2010-03-03 22 +195 val_195 2010-03-03 22 +475 val_475 2010-03-03 22 +17 val_17 2010-03-03 22 +113 val_113 2010-03-03 22 +155 val_155 2010-03-03 22 +203 val_203 2010-03-03 22 +339 val_339 2010-03-03 22 +0 val_0 2010-03-03 22 +455 val_455 2010-03-03 22 +128 val_128 2010-03-03 22 +311 val_311 2010-03-03 22 +316 val_316 2010-03-03 22 +57 val_57 2010-03-03 22 +302 val_302 2010-03-03 22 +205 val_205 2010-03-03 22 +149 val_149 2010-03-03 22 +438 val_438 2010-03-03 22 +345 val_345 2010-03-03 22 +129 val_129 2010-03-03 22 +170 val_170 2010-03-03 22 +20 val_20 2010-03-03 22 +489 val_489 2010-03-03 22 +157 val_157 2010-03-03 22 +378 val_378 2010-03-03 22 +221 val_221 2010-03-03 22 +92 val_92 2010-03-03 22 +111 val_111 2010-03-03 22 +47 val_47 2010-03-03 22 +72 val_72 2010-03-03 22 +4 val_4 2010-03-03 22 +280 val_280 2010-03-03 22 +35 val_35 2010-03-03 22 +427 val_427 2010-03-03 22 +277 val_277 2010-03-03 22 +208 val_208 2010-03-03 22 +356 val_356 2010-03-03 22 +399 val_399 2010-03-03 22 +169 val_169 2010-03-03 22 +382 val_382 2010-03-03 22 +498 val_498 2010-03-03 22 +125 val_125 2010-03-03 22 +386 val_386 2010-03-03 22 +437 val_437 2010-03-03 22 +469 val_469 2010-03-03 22 +192 val_192 2010-03-03 22 +286 val_286 2010-03-03 22 +187 val_187 2010-03-03 22 +176 val_176 2010-03-03 22 +54 val_54 2010-03-03 22 +459 val_459 2010-03-03 22 +51 val_51 2010-03-03 22 +138 val_138 2010-03-03 22 +103 val_103 2010-03-03 22 +239 val_239 2010-03-03 22 +213 val_213 2010-03-03 22 +216 val_216 2010-03-03 22 +430 val_430 2010-03-03 22 +278 val_278 2010-03-03 22 +176 val_176 2010-03-03 22 +289 val_289 2010-03-03 22 +221 val_221 2010-03-03 22 +65 val_65 2010-03-03 22 +318 val_318 2010-03-03 22 +332 val_332 2010-03-03 22 +311 val_311 2010-03-03 22 +275 val_275 2010-03-03 22 +137 val_137 2010-03-03 22 +241 val_241 2010-03-03 22 +83 val_83 2010-03-03 22 +333 val_333 2010-03-03 22 +180 val_180 2010-03-03 22 +284 val_284 2010-03-03 22 +12 val_12 2010-03-03 22 +230 val_230 2010-03-03 22 +181 val_181 2010-03-03 22 +67 val_67 2010-03-03 22 +260 val_260 2010-03-03 22 +404 val_404 2010-03-03 22 +384 val_384 2010-03-03 22 +489 val_489 2010-03-03 22 +353 val_353 2010-03-03 22 +373 val_373 2010-03-03 22 +272 val_272 2010-03-03 22 +138 val_138 2010-03-03 22 +217 val_217 2010-03-03 22 +84 val_84 2010-03-03 22 +348 val_348 2010-03-03 22 +466 val_466 2010-03-03 22 +58 val_58 2010-03-03 22 +8 val_8 2010-03-03 22 +411 val_411 2010-03-03 22 +230 val_230 2010-03-03 22 +208 val_208 2010-03-03 22 +348 val_348 2010-03-03 22 +24 val_24 2010-03-03 22 +463 val_463 2010-03-03 22 +431 val_431 2010-03-03 22 +179 val_179 2010-03-03 22 +172 val_172 2010-03-03 22 +42 val_42 2010-03-03 22 +129 val_129 2010-03-03 22 +158 val_158 2010-03-03 22 +119 val_119 2010-03-03 22 +496 val_496 2010-03-03 22 +0 val_0 2010-03-03 22 +322 val_322 2010-03-03 22 +197 val_197 2010-03-03 22 +468 val_468 2010-03-03 22 +393 val_393 2010-03-03 22 +454 val_454 2010-03-03 22 +100 val_100 2010-03-03 22 +298 val_298 2010-03-03 22 +199 val_199 2010-03-03 22 +191 val_191 2010-03-03 22 +418 val_418 2010-03-03 22 +96 val_96 2010-03-03 22 +26 val_26 2010-03-03 22 +165 val_165 2010-03-03 22 +327 val_327 2010-03-03 22 +230 val_230 2010-03-03 22 +205 val_205 2010-03-03 22 +120 val_120 2010-03-03 22 +131 val_131 2010-03-03 22 +51 val_51 2010-03-03 22 +404 val_404 2010-03-03 22 +43 val_43 2010-03-03 22 +436 val_436 2010-03-03 22 +156 val_156 2010-03-03 22 +469 val_469 2010-03-03 22 +468 val_468 2010-03-03 22 +308 val_308 2010-03-03 22 +95 val_95 2010-03-03 22 +196 val_196 2010-03-03 22 +288 val_288 2010-03-03 22 +481 val_481 2010-03-03 22 +457 val_457 2010-03-03 22 +98 val_98 2010-03-03 22 +282 val_282 2010-03-03 22 +197 val_197 2010-03-03 22 +187 val_187 2010-03-03 22 +318 val_318 2010-03-03 22 +318 val_318 2010-03-03 22 +409 val_409 2010-03-03 22 +470 val_470 2010-03-03 22 +137 val_137 2010-03-03 22 +369 val_369 2010-03-03 22 +316 val_316 2010-03-03 22 +169 val_169 2010-03-03 22 +413 val_413 2010-03-03 22 +85 val_85 2010-03-03 22 +77 val_77 2010-03-03 22 +0 val_0 2010-03-03 22 +490 val_490 2010-03-03 22 +87 val_87 2010-03-03 22 +364 val_364 2010-03-03 22 +179 val_179 2010-03-03 22 +118 val_118 2010-03-03 22 +134 val_134 2010-03-03 22 +395 val_395 2010-03-03 22 +282 val_282 2010-03-03 22 +138 val_138 2010-03-03 22 +238 val_238 2010-03-03 22 +419 val_419 2010-03-03 22 +15 val_15 2010-03-03 22 +118 val_118 2010-03-03 22 +72 val_72 2010-03-03 22 +90 val_90 2010-03-03 22 +307 val_307 2010-03-03 22 +19 val_19 2010-03-03 22 +435 val_435 2010-03-03 22 +10 val_10 2010-03-03 22 +277 val_277 2010-03-03 22 +273 val_273 2010-03-03 22 +306 val_306 2010-03-03 22 +224 val_224 2010-03-03 22 +309 val_309 2010-03-03 22 +389 val_389 2010-03-03 22 +327 val_327 2010-03-03 22 +242 val_242 2010-03-03 22 +369 val_369 2010-03-03 22 +392 val_392 2010-03-03 22 +272 val_272 2010-03-03 22 +331 val_331 2010-03-03 22 +401 val_401 2010-03-03 22 +242 val_242 2010-03-03 22 +452 val_452 2010-03-03 22 +177 val_177 2010-03-03 22 +226 val_226 2010-03-03 22 +5 val_5 2010-03-03 22 +497 val_497 2010-03-03 22 +402 val_402 2010-03-03 22 +396 val_396 2010-03-03 22 +317 val_317 2010-03-03 22 +395 val_395 2010-03-03 22 +58 val_58 2010-03-03 22 +35 val_35 2010-03-03 22 +336 val_336 2010-03-03 22 +95 val_95 2010-03-03 22 +11 val_11 2010-03-03 22 +168 val_168 2010-03-03 22 +34 val_34 2010-03-03 22 +229 val_229 2010-03-03 22 +233 val_233 2010-03-03 22 +143 val_143 2010-03-03 22 +472 val_472 2010-03-03 22 +322 val_322 2010-03-03 22 +498 val_498 2010-03-03 22 +160 val_160 2010-03-03 22 +195 val_195 2010-03-03 22 +42 val_42 2010-03-03 22 +321 val_321 2010-03-03 22 +430 val_430 2010-03-03 22 +119 val_119 2010-03-03 22 +489 val_489 2010-03-03 22 +458 val_458 2010-03-03 22 +78 val_78 2010-03-03 22 +76 val_76 2010-03-03 22 +41 val_41 2010-03-03 22 +223 val_223 2010-03-03 22 +492 val_492 2010-03-03 22 +149 val_149 2010-03-03 22 +449 val_449 2010-03-03 22 +218 val_218 2010-03-03 22 +228 val_228 2010-03-03 22 +138 val_138 2010-03-03 22 +453 val_453 2010-03-03 22 +30 val_30 2010-03-03 22 +209 val_209 2010-03-03 22 +64 val_64 2010-03-03 22 +468 val_468 2010-03-03 22 +76 val_76 2010-03-03 22 +74 val_74 2010-03-03 22 +342 val_342 2010-03-03 22 +69 val_69 2010-03-03 22 +230 val_230 2010-03-03 22 +33 val_33 2010-03-03 22 +368 val_368 2010-03-03 22 +103 val_103 2010-03-03 22 +296 val_296 2010-03-03 22 +113 val_113 2010-03-03 22 +216 val_216 2010-03-03 22 +367 val_367 2010-03-03 22 +344 val_344 2010-03-03 22 +167 val_167 2010-03-03 22 +274 val_274 2010-03-03 22 +219 val_219 2010-03-03 22 +239 val_239 2010-03-03 22 +485 val_485 2010-03-03 22 +116 val_116 2010-03-03 22 +223 val_223 2010-03-03 22 +256 val_256 2010-03-03 22 +263 val_263 2010-03-03 22 +70 val_70 2010-03-03 22 +487 val_487 2010-03-03 22 +480 val_480 2010-03-03 22 +401 val_401 2010-03-03 22 +288 val_288 2010-03-03 22 +191 val_191 2010-03-03 22 +5 val_5 2010-03-03 22 +244 val_244 2010-03-03 22 +438 val_438 2010-03-03 22 +128 val_128 2010-03-03 22 +467 val_467 2010-03-03 22 +432 val_432 2010-03-03 22 +202 val_202 2010-03-03 22 +316 val_316 2010-03-03 22 +229 val_229 2010-03-03 22 +469 val_469 2010-03-03 22 +463 val_463 2010-03-03 22 +280 val_280 2010-03-03 22 +2 val_2 2010-03-03 22 +35 val_35 2010-03-03 22 +283 val_283 2010-03-03 22 +331 val_331 2010-03-03 22 +235 val_235 2010-03-03 22 +80 val_80 2010-03-03 22 +44 val_44 2010-03-03 22 +193 val_193 2010-03-03 22 +321 val_321 2010-03-03 22 +335 val_335 2010-03-03 22 +104 val_104 2010-03-03 22 +466 val_466 2010-03-03 22 +366 val_366 2010-03-03 22 +175 val_175 2010-03-03 22 +403 val_403 2010-03-03 22 +483 val_483 2010-03-03 22 +53 val_53 2010-03-03 22 +105 val_105 2010-03-03 22 +257 val_257 2010-03-03 22 +406 val_406 2010-03-03 22 +409 val_409 2010-03-03 22 +190 val_190 2010-03-03 22 +406 val_406 2010-03-03 22 +401 val_401 2010-03-03 22 +114 val_114 2010-03-03 22 +258 val_258 2010-03-03 22 +90 val_90 2010-03-03 22 +203 val_203 2010-03-03 22 +262 val_262 2010-03-03 22 +348 val_348 2010-03-03 22 +424 val_424 2010-03-03 22 +12 val_12 2010-03-03 22 +396 val_396 2010-03-03 22 +201 val_201 2010-03-03 22 +217 val_217 2010-03-03 22 +164 val_164 2010-03-03 22 +431 val_431 2010-03-03 22 +454 val_454 2010-03-03 22 +478 val_478 2010-03-03 22 +298 val_298 2010-03-03 22 +125 val_125 2010-03-03 22 +431 val_431 2010-03-03 22 +164 val_164 2010-03-03 22 +424 val_424 2010-03-03 22 +187 val_187 2010-03-03 22 +382 val_382 2010-03-03 22 +5 val_5 2010-03-03 22 +70 val_70 2010-03-03 22 +397 val_397 2010-03-03 22 +480 val_480 2010-03-03 22 +291 val_291 2010-03-03 22 +24 val_24 2010-03-03 22 +351 val_351 2010-03-03 22 +255 val_255 2010-03-03 22 +104 val_104 2010-03-03 22 +70 val_70 2010-03-03 22 +163 val_163 2010-03-03 22 +438 val_438 2010-03-03 22 +119 val_119 2010-03-03 22 +414 val_414 2010-03-03 22 +200 val_200 2010-03-03 22 +491 val_491 2010-03-03 22 +237 val_237 2010-03-03 22 +439 val_439 2010-03-03 22 +360 val_360 2010-03-03 22 +248 val_248 2010-03-03 22 +479 val_479 2010-03-03 22 +305 val_305 2010-03-03 22 +417 val_417 2010-03-03 22 +199 val_199 2010-03-03 22 +444 val_444 2010-03-03 22 +120 val_120 2010-03-03 22 +429 val_429 2010-03-03 22 +169 val_169 2010-03-03 22 +443 val_443 2010-03-03 22 +323 val_323 2010-03-03 22 +325 val_325 2010-03-03 22 +277 val_277 2010-03-03 22 +230 val_230 2010-03-03 22 +478 val_478 2010-03-03 22 +178 val_178 2010-03-03 22 +468 val_468 2010-03-03 22 +310 val_310 2010-03-03 22 +317 val_317 2010-03-03 22 +333 val_333 2010-03-03 22 +493 val_493 2010-03-03 22 +460 val_460 2010-03-03 22 +207 val_207 2010-03-03 22 +249 val_249 2010-03-03 22 +265 val_265 2010-03-03 22 +480 val_480 2010-03-03 22 +83 val_83 2010-03-03 22 +136 val_136 2010-03-03 22 +353 val_353 2010-03-03 22 +172 val_172 2010-03-03 22 +214 val_214 2010-03-03 22 +462 val_462 2010-03-03 22 +233 val_233 2010-03-03 22 +406 val_406 2010-03-03 22 +133 val_133 2010-03-03 22 +175 val_175 2010-03-03 22 +189 val_189 2010-03-03 22 +454 val_454 2010-03-03 22 +375 val_375 2010-03-03 22 +401 val_401 2010-03-03 22 +421 val_421 2010-03-03 22 +407 val_407 2010-03-03 22 +384 val_384 2010-03-03 22 +256 val_256 2010-03-03 22 +26 val_26 2010-03-03 22 +134 val_134 2010-03-03 22 +67 val_67 2010-03-03 22 +384 val_384 2010-03-03 22 +379 val_379 2010-03-03 22 +18 val_18 2010-03-03 22 +462 val_462 2010-03-03 22 +492 val_492 2010-03-03 22 +100 val_100 2010-03-03 22 +298 val_298 2010-03-03 22 +9 val_9 2010-03-03 22 +341 val_341 2010-03-03 22 +498 val_498 2010-03-03 22 +146 val_146 2010-03-03 22 +458 val_458 2010-03-03 22 +362 val_362 2010-03-03 22 +186 val_186 2010-03-03 22 +285 val_285 2010-03-03 22 +348 val_348 2010-03-03 22 +167 val_167 2010-03-03 22 +18 val_18 2010-03-03 22 +273 val_273 2010-03-03 22 +183 val_183 2010-03-03 22 +281 val_281 2010-03-03 22 +344 val_344 2010-03-03 22 +97 val_97 2010-03-03 22 +469 val_469 2010-03-03 22 +315 val_315 2010-03-03 22 +84 val_84 2010-03-03 22 +28 val_28 2010-03-03 22 +37 val_37 2010-03-03 22 +448 val_448 2010-03-03 22 +152 val_152 2010-03-03 22 +348 val_348 2010-03-03 22 +307 val_307 2010-03-03 22 +194 val_194 2010-03-03 22 +414 val_414 2010-03-03 22 +477 val_477 2010-03-03 22 +222 val_222 2010-03-03 22 +126 val_126 2010-03-03 22 +90 val_90 2010-03-03 22 +169 val_169 2010-03-03 22 +403 val_403 2010-03-03 22 +400 val_400 2010-03-03 22 +200 val_200 2010-03-03 22 +97 val_97 2010-03-03 22 +238 val_238 2010-03-03 24 +86 val_86 2010-03-03 24 +311 val_311 2010-03-03 24 +27 val_27 2010-03-03 24 +165 val_165 2010-03-03 24 +409 val_409 2010-03-03 24 +255 val_255 2010-03-03 24 +278 val_278 2010-03-03 24 +98 val_98 2010-03-03 24 +484 val_484 2010-03-03 24 +265 val_265 2010-03-03 24 +193 val_193 2010-03-03 24 +401 val_401 2010-03-03 24 +150 val_150 2010-03-03 24 +273 val_273 2010-03-03 24 +224 val_224 2010-03-03 24 +369 val_369 2010-03-03 24 +66 val_66 2010-03-03 24 +128 val_128 2010-03-03 24 +213 val_213 2010-03-03 24 +146 val_146 2010-03-03 24 +406 val_406 2010-03-03 24 +429 val_429 2010-03-03 24 +374 val_374 2010-03-03 24 +152 val_152 2010-03-03 24 +469 val_469 2010-03-03 24 +145 val_145 2010-03-03 24 +495 val_495 2010-03-03 24 +37 val_37 2010-03-03 24 +327 val_327 2010-03-03 24 +281 val_281 2010-03-03 24 +277 val_277 2010-03-03 24 +209 val_209 2010-03-03 24 +15 val_15 2010-03-03 24 +82 val_82 2010-03-03 24 +403 val_403 2010-03-03 24 +166 val_166 2010-03-03 24 +417 val_417 2010-03-03 24 +430 val_430 2010-03-03 24 +252 val_252 2010-03-03 24 +292 val_292 2010-03-03 24 +219 val_219 2010-03-03 24 +287 val_287 2010-03-03 24 +153 val_153 2010-03-03 24 +193 val_193 2010-03-03 24 +338 val_338 2010-03-03 24 +446 val_446 2010-03-03 24 +459 val_459 2010-03-03 24 +394 val_394 2010-03-03 24 +237 val_237 2010-03-03 24 +482 val_482 2010-03-03 24 +174 val_174 2010-03-03 24 +413 val_413 2010-03-03 24 +494 val_494 2010-03-03 24 +207 val_207 2010-03-03 24 +199 val_199 2010-03-03 24 +466 val_466 2010-03-03 24 +208 val_208 2010-03-03 24 +174 val_174 2010-03-03 24 +399 val_399 2010-03-03 24 +396 val_396 2010-03-03 24 +247 val_247 2010-03-03 24 +417 val_417 2010-03-03 24 +489 val_489 2010-03-03 24 +162 val_162 2010-03-03 24 +377 val_377 2010-03-03 24 +397 val_397 2010-03-03 24 +309 val_309 2010-03-03 24 +365 val_365 2010-03-03 24 +266 val_266 2010-03-03 24 +439 val_439 2010-03-03 24 +342 val_342 2010-03-03 24 +367 val_367 2010-03-03 24 +325 val_325 2010-03-03 24 +167 val_167 2010-03-03 24 +195 val_195 2010-03-03 24 +475 val_475 2010-03-03 24 +17 val_17 2010-03-03 24 +113 val_113 2010-03-03 24 +155 val_155 2010-03-03 24 +203 val_203 2010-03-03 24 +339 val_339 2010-03-03 24 +0 val_0 2010-03-03 24 +455 val_455 2010-03-03 24 +128 val_128 2010-03-03 24 +311 val_311 2010-03-03 24 +316 val_316 2010-03-03 24 +57 val_57 2010-03-03 24 +302 val_302 2010-03-03 24 +205 val_205 2010-03-03 24 +149 val_149 2010-03-03 24 +438 val_438 2010-03-03 24 +345 val_345 2010-03-03 24 +129 val_129 2010-03-03 24 +170 val_170 2010-03-03 24 +20 val_20 2010-03-03 24 +489 val_489 2010-03-03 24 +157 val_157 2010-03-03 24 +378 val_378 2010-03-03 24 +221 val_221 2010-03-03 24 +92 val_92 2010-03-03 24 +111 val_111 2010-03-03 24 +47 val_47 2010-03-03 24 +72 val_72 2010-03-03 24 +4 val_4 2010-03-03 24 +280 val_280 2010-03-03 24 +35 val_35 2010-03-03 24 +427 val_427 2010-03-03 24 +277 val_277 2010-03-03 24 +208 val_208 2010-03-03 24 +356 val_356 2010-03-03 24 +399 val_399 2010-03-03 24 +169 val_169 2010-03-03 24 +382 val_382 2010-03-03 24 +498 val_498 2010-03-03 24 +125 val_125 2010-03-03 24 +386 val_386 2010-03-03 24 +437 val_437 2010-03-03 24 +469 val_469 2010-03-03 24 +192 val_192 2010-03-03 24 +286 val_286 2010-03-03 24 +187 val_187 2010-03-03 24 +176 val_176 2010-03-03 24 +54 val_54 2010-03-03 24 +459 val_459 2010-03-03 24 +51 val_51 2010-03-03 24 +138 val_138 2010-03-03 24 +103 val_103 2010-03-03 24 +239 val_239 2010-03-03 24 +213 val_213 2010-03-03 24 +216 val_216 2010-03-03 24 +430 val_430 2010-03-03 24 +278 val_278 2010-03-03 24 +176 val_176 2010-03-03 24 +289 val_289 2010-03-03 24 +221 val_221 2010-03-03 24 +65 val_65 2010-03-03 24 +318 val_318 2010-03-03 24 +332 val_332 2010-03-03 24 +311 val_311 2010-03-03 24 +275 val_275 2010-03-03 24 +137 val_137 2010-03-03 24 +241 val_241 2010-03-03 24 +83 val_83 2010-03-03 24 +333 val_333 2010-03-03 24 +180 val_180 2010-03-03 24 +284 val_284 2010-03-03 24 +12 val_12 2010-03-03 24 +230 val_230 2010-03-03 24 +181 val_181 2010-03-03 24 +67 val_67 2010-03-03 24 +260 val_260 2010-03-03 24 +404 val_404 2010-03-03 24 +384 val_384 2010-03-03 24 +489 val_489 2010-03-03 24 +353 val_353 2010-03-03 24 +373 val_373 2010-03-03 24 +272 val_272 2010-03-03 24 +138 val_138 2010-03-03 24 +217 val_217 2010-03-03 24 +84 val_84 2010-03-03 24 +348 val_348 2010-03-03 24 +466 val_466 2010-03-03 24 +58 val_58 2010-03-03 24 +8 val_8 2010-03-03 24 +411 val_411 2010-03-03 24 +230 val_230 2010-03-03 24 +208 val_208 2010-03-03 24 +348 val_348 2010-03-03 24 +24 val_24 2010-03-03 24 +463 val_463 2010-03-03 24 +431 val_431 2010-03-03 24 +179 val_179 2010-03-03 24 +172 val_172 2010-03-03 24 +42 val_42 2010-03-03 24 +129 val_129 2010-03-03 24 +158 val_158 2010-03-03 24 +119 val_119 2010-03-03 24 +496 val_496 2010-03-03 24 +0 val_0 2010-03-03 24 +322 val_322 2010-03-03 24 +197 val_197 2010-03-03 24 +468 val_468 2010-03-03 24 +393 val_393 2010-03-03 24 +454 val_454 2010-03-03 24 +100 val_100 2010-03-03 24 +298 val_298 2010-03-03 24 +199 val_199 2010-03-03 24 +191 val_191 2010-03-03 24 +418 val_418 2010-03-03 24 +96 val_96 2010-03-03 24 +26 val_26 2010-03-03 24 +165 val_165 2010-03-03 24 +327 val_327 2010-03-03 24 +230 val_230 2010-03-03 24 +205 val_205 2010-03-03 24 +120 val_120 2010-03-03 24 +131 val_131 2010-03-03 24 +51 val_51 2010-03-03 24 +404 val_404 2010-03-03 24 +43 val_43 2010-03-03 24 +436 val_436 2010-03-03 24 +156 val_156 2010-03-03 24 +469 val_469 2010-03-03 24 +468 val_468 2010-03-03 24 +308 val_308 2010-03-03 24 +95 val_95 2010-03-03 24 +196 val_196 2010-03-03 24 +288 val_288 2010-03-03 24 +481 val_481 2010-03-03 24 +457 val_457 2010-03-03 24 +98 val_98 2010-03-03 24 +282 val_282 2010-03-03 24 +197 val_197 2010-03-03 24 +187 val_187 2010-03-03 24 +318 val_318 2010-03-03 24 +318 val_318 2010-03-03 24 +409 val_409 2010-03-03 24 +470 val_470 2010-03-03 24 +137 val_137 2010-03-03 24 +369 val_369 2010-03-03 24 +316 val_316 2010-03-03 24 +169 val_169 2010-03-03 24 +413 val_413 2010-03-03 24 +85 val_85 2010-03-03 24 +77 val_77 2010-03-03 24 +0 val_0 2010-03-03 24 +490 val_490 2010-03-03 24 +87 val_87 2010-03-03 24 +364 val_364 2010-03-03 24 +179 val_179 2010-03-03 24 +118 val_118 2010-03-03 24 +134 val_134 2010-03-03 24 +395 val_395 2010-03-03 24 +282 val_282 2010-03-03 24 +138 val_138 2010-03-03 24 +238 val_238 2010-03-03 24 +419 val_419 2010-03-03 24 +15 val_15 2010-03-03 24 +118 val_118 2010-03-03 24 +72 val_72 2010-03-03 24 +90 val_90 2010-03-03 24 +307 val_307 2010-03-03 24 +19 val_19 2010-03-03 24 +435 val_435 2010-03-03 24 +10 val_10 2010-03-03 24 +277 val_277 2010-03-03 24 +273 val_273 2010-03-03 24 +306 val_306 2010-03-03 24 +224 val_224 2010-03-03 24 +309 val_309 2010-03-03 24 +389 val_389 2010-03-03 24 +327 val_327 2010-03-03 24 +242 val_242 2010-03-03 24 +369 val_369 2010-03-03 24 +392 val_392 2010-03-03 24 +272 val_272 2010-03-03 24 +331 val_331 2010-03-03 24 +401 val_401 2010-03-03 24 +242 val_242 2010-03-03 24 +452 val_452 2010-03-03 24 +177 val_177 2010-03-03 24 +226 val_226 2010-03-03 24 +5 val_5 2010-03-03 24 +497 val_497 2010-03-03 24 +402 val_402 2010-03-03 24 +396 val_396 2010-03-03 24 +317 val_317 2010-03-03 24 +395 val_395 2010-03-03 24 +58 val_58 2010-03-03 24 +35 val_35 2010-03-03 24 +336 val_336 2010-03-03 24 +95 val_95 2010-03-03 24 +11 val_11 2010-03-03 24 +168 val_168 2010-03-03 24 +34 val_34 2010-03-03 24 +229 val_229 2010-03-03 24 +233 val_233 2010-03-03 24 +143 val_143 2010-03-03 24 +472 val_472 2010-03-03 24 +322 val_322 2010-03-03 24 +498 val_498 2010-03-03 24 +160 val_160 2010-03-03 24 +195 val_195 2010-03-03 24 +42 val_42 2010-03-03 24 +321 val_321 2010-03-03 24 +430 val_430 2010-03-03 24 +119 val_119 2010-03-03 24 +489 val_489 2010-03-03 24 +458 val_458 2010-03-03 24 +78 val_78 2010-03-03 24 +76 val_76 2010-03-03 24 +41 val_41 2010-03-03 24 +223 val_223 2010-03-03 24 +492 val_492 2010-03-03 24 +149 val_149 2010-03-03 24 +449 val_449 2010-03-03 24 +218 val_218 2010-03-03 24 +228 val_228 2010-03-03 24 +138 val_138 2010-03-03 24 +453 val_453 2010-03-03 24 +30 val_30 2010-03-03 24 +209 val_209 2010-03-03 24 +64 val_64 2010-03-03 24 +468 val_468 2010-03-03 24 +76 val_76 2010-03-03 24 +74 val_74 2010-03-03 24 +342 val_342 2010-03-03 24 +69 val_69 2010-03-03 24 +230 val_230 2010-03-03 24 +33 val_33 2010-03-03 24 +368 val_368 2010-03-03 24 +103 val_103 2010-03-03 24 +296 val_296 2010-03-03 24 +113 val_113 2010-03-03 24 +216 val_216 2010-03-03 24 +367 val_367 2010-03-03 24 +344 val_344 2010-03-03 24 +167 val_167 2010-03-03 24 +274 val_274 2010-03-03 24 +219 val_219 2010-03-03 24 +239 val_239 2010-03-03 24 +485 val_485 2010-03-03 24 +116 val_116 2010-03-03 24 +223 val_223 2010-03-03 24 +256 val_256 2010-03-03 24 +263 val_263 2010-03-03 24 +70 val_70 2010-03-03 24 +487 val_487 2010-03-03 24 +480 val_480 2010-03-03 24 +401 val_401 2010-03-03 24 +288 val_288 2010-03-03 24 +191 val_191 2010-03-03 24 +5 val_5 2010-03-03 24 +244 val_244 2010-03-03 24 +438 val_438 2010-03-03 24 +128 val_128 2010-03-03 24 +467 val_467 2010-03-03 24 +432 val_432 2010-03-03 24 +202 val_202 2010-03-03 24 +316 val_316 2010-03-03 24 +229 val_229 2010-03-03 24 +469 val_469 2010-03-03 24 +463 val_463 2010-03-03 24 +280 val_280 2010-03-03 24 +2 val_2 2010-03-03 24 +35 val_35 2010-03-03 24 +283 val_283 2010-03-03 24 +331 val_331 2010-03-03 24 +235 val_235 2010-03-03 24 +80 val_80 2010-03-03 24 +44 val_44 2010-03-03 24 +193 val_193 2010-03-03 24 +321 val_321 2010-03-03 24 +335 val_335 2010-03-03 24 +104 val_104 2010-03-03 24 +466 val_466 2010-03-03 24 +366 val_366 2010-03-03 24 +175 val_175 2010-03-03 24 +403 val_403 2010-03-03 24 +483 val_483 2010-03-03 24 +53 val_53 2010-03-03 24 +105 val_105 2010-03-03 24 +257 val_257 2010-03-03 24 +406 val_406 2010-03-03 24 +409 val_409 2010-03-03 24 +190 val_190 2010-03-03 24 +406 val_406 2010-03-03 24 +401 val_401 2010-03-03 24 +114 val_114 2010-03-03 24 +258 val_258 2010-03-03 24 +90 val_90 2010-03-03 24 +203 val_203 2010-03-03 24 +262 val_262 2010-03-03 24 +348 val_348 2010-03-03 24 +424 val_424 2010-03-03 24 +12 val_12 2010-03-03 24 +396 val_396 2010-03-03 24 +201 val_201 2010-03-03 24 +217 val_217 2010-03-03 24 +164 val_164 2010-03-03 24 +431 val_431 2010-03-03 24 +454 val_454 2010-03-03 24 +478 val_478 2010-03-03 24 +298 val_298 2010-03-03 24 +125 val_125 2010-03-03 24 +431 val_431 2010-03-03 24 +164 val_164 2010-03-03 24 +424 val_424 2010-03-03 24 +187 val_187 2010-03-03 24 +382 val_382 2010-03-03 24 +5 val_5 2010-03-03 24 +70 val_70 2010-03-03 24 +397 val_397 2010-03-03 24 +480 val_480 2010-03-03 24 +291 val_291 2010-03-03 24 +24 val_24 2010-03-03 24 +351 val_351 2010-03-03 24 +255 val_255 2010-03-03 24 +104 val_104 2010-03-03 24 +70 val_70 2010-03-03 24 +163 val_163 2010-03-03 24 +438 val_438 2010-03-03 24 +119 val_119 2010-03-03 24 +414 val_414 2010-03-03 24 +200 val_200 2010-03-03 24 +491 val_491 2010-03-03 24 +237 val_237 2010-03-03 24 +439 val_439 2010-03-03 24 +360 val_360 2010-03-03 24 +248 val_248 2010-03-03 24 +479 val_479 2010-03-03 24 +305 val_305 2010-03-03 24 +417 val_417 2010-03-03 24 +199 val_199 2010-03-03 24 +444 val_444 2010-03-03 24 +120 val_120 2010-03-03 24 +429 val_429 2010-03-03 24 +169 val_169 2010-03-03 24 +443 val_443 2010-03-03 24 +323 val_323 2010-03-03 24 +325 val_325 2010-03-03 24 +277 val_277 2010-03-03 24 +230 val_230 2010-03-03 24 +478 val_478 2010-03-03 24 +178 val_178 2010-03-03 24 +468 val_468 2010-03-03 24 +310 val_310 2010-03-03 24 +317 val_317 2010-03-03 24 +333 val_333 2010-03-03 24 +493 val_493 2010-03-03 24 +460 val_460 2010-03-03 24 +207 val_207 2010-03-03 24 +249 val_249 2010-03-03 24 +265 val_265 2010-03-03 24 +480 val_480 2010-03-03 24 +83 val_83 2010-03-03 24 +136 val_136 2010-03-03 24 +353 val_353 2010-03-03 24 +172 val_172 2010-03-03 24 +214 val_214 2010-03-03 24 +462 val_462 2010-03-03 24 +233 val_233 2010-03-03 24 +406 val_406 2010-03-03 24 +133 val_133 2010-03-03 24 +175 val_175 2010-03-03 24 +189 val_189 2010-03-03 24 +454 val_454 2010-03-03 24 +375 val_375 2010-03-03 24 +401 val_401 2010-03-03 24 +421 val_421 2010-03-03 24 +407 val_407 2010-03-03 24 +384 val_384 2010-03-03 24 +256 val_256 2010-03-03 24 +26 val_26 2010-03-03 24 +134 val_134 2010-03-03 24 +67 val_67 2010-03-03 24 +384 val_384 2010-03-03 24 +379 val_379 2010-03-03 24 +18 val_18 2010-03-03 24 +462 val_462 2010-03-03 24 +492 val_492 2010-03-03 24 +100 val_100 2010-03-03 24 +298 val_298 2010-03-03 24 +9 val_9 2010-03-03 24 +341 val_341 2010-03-03 24 +498 val_498 2010-03-03 24 +146 val_146 2010-03-03 24 +458 val_458 2010-03-03 24 +362 val_362 2010-03-03 24 +186 val_186 2010-03-03 24 +285 val_285 2010-03-03 24 +348 val_348 2010-03-03 24 +167 val_167 2010-03-03 24 +18 val_18 2010-03-03 24 +273 val_273 2010-03-03 24 +183 val_183 2010-03-03 24 +281 val_281 2010-03-03 24 +344 val_344 2010-03-03 24 +97 val_97 2010-03-03 24 +469 val_469 2010-03-03 24 +315 val_315 2010-03-03 24 +84 val_84 2010-03-03 24 +28 val_28 2010-03-03 24 +37 val_37 2010-03-03 24 +448 val_448 2010-03-03 24 +152 val_152 2010-03-03 24 +348 val_348 2010-03-03 24 +307 val_307 2010-03-03 24 +194 val_194 2010-03-03 24 +414 val_414 2010-03-03 24 +477 val_477 2010-03-03 24 +222 val_222 2010-03-03 24 +126 val_126 2010-03-03 24 +90 val_90 2010-03-03 24 +169 val_169 2010-03-03 24 +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 +238 val_238 2010-03-03 24 +86 val_86 2010-03-03 24 +311 val_311 2010-03-03 24 +27 val_27 2010-03-03 24 +165 val_165 2010-03-03 24 +409 val_409 2010-03-03 24 +255 val_255 2010-03-03 24 +278 val_278 2010-03-03 24 +98 val_98 2010-03-03 24 +484 val_484 2010-03-03 24 +265 val_265 2010-03-03 24 +193 val_193 2010-03-03 24 +401 val_401 2010-03-03 24 +150 val_150 2010-03-03 24 +273 val_273 2010-03-03 24 +224 val_224 2010-03-03 24 +369 val_369 2010-03-03 24 +66 val_66 2010-03-03 24 +128 val_128 2010-03-03 24 +213 val_213 2010-03-03 24 +146 val_146 2010-03-03 24 +406 val_406 2010-03-03 24 +429 val_429 2010-03-03 24 +374 val_374 2010-03-03 24 +152 val_152 2010-03-03 24 +469 val_469 2010-03-03 24 +145 val_145 2010-03-03 24 +495 val_495 2010-03-03 24 +37 val_37 2010-03-03 24 +327 val_327 2010-03-03 24 +281 val_281 2010-03-03 24 +277 val_277 2010-03-03 24 +209 val_209 2010-03-03 24 +15 val_15 2010-03-03 24 +82 val_82 2010-03-03 24 +403 val_403 2010-03-03 24 +166 val_166 2010-03-03 24 +417 val_417 2010-03-03 24 +430 val_430 2010-03-03 24 +252 val_252 2010-03-03 24 +292 val_292 2010-03-03 24 +219 val_219 2010-03-03 24 +287 val_287 2010-03-03 24 +153 val_153 2010-03-03 24 +193 val_193 2010-03-03 24 +338 val_338 2010-03-03 24 +446 val_446 2010-03-03 24 +459 val_459 2010-03-03 24 +394 val_394 2010-03-03 24 +237 val_237 2010-03-03 24 +482 val_482 2010-03-03 24 +174 val_174 2010-03-03 24 +413 val_413 2010-03-03 24 +494 val_494 2010-03-03 24 +207 val_207 2010-03-03 24 +199 val_199 2010-03-03 24 +466 val_466 2010-03-03 24 +208 val_208 2010-03-03 24 +174 val_174 2010-03-03 24 +399 val_399 2010-03-03 24 +396 val_396 2010-03-03 24 +247 val_247 2010-03-03 24 +417 val_417 2010-03-03 24 +489 val_489 2010-03-03 24 +162 val_162 2010-03-03 24 +377 val_377 2010-03-03 24 +397 val_397 2010-03-03 24 +309 val_309 2010-03-03 24 +365 val_365 2010-03-03 24 +266 val_266 2010-03-03 24 +439 val_439 2010-03-03 24 +342 val_342 2010-03-03 24 +367 val_367 2010-03-03 24 +325 val_325 2010-03-03 24 +167 val_167 2010-03-03 24 +195 val_195 2010-03-03 24 +475 val_475 2010-03-03 24 +17 val_17 2010-03-03 24 +113 val_113 2010-03-03 24 +155 val_155 2010-03-03 24 +203 val_203 2010-03-03 24 +339 val_339 2010-03-03 24 +0 val_0 2010-03-03 24 +455 val_455 2010-03-03 24 +128 val_128 2010-03-03 24 +311 val_311 2010-03-03 24 +316 val_316 2010-03-03 24 +57 val_57 2010-03-03 24 +302 val_302 2010-03-03 24 +205 val_205 2010-03-03 24 +149 val_149 2010-03-03 24 +438 val_438 2010-03-03 24 +345 val_345 2010-03-03 24 +129 val_129 2010-03-03 24 +170 val_170 2010-03-03 24 +20 val_20 2010-03-03 24 +489 val_489 2010-03-03 24 +157 val_157 2010-03-03 24 +378 val_378 2010-03-03 24 +221 val_221 2010-03-03 24 +92 val_92 2010-03-03 24 +111 val_111 2010-03-03 24 +47 val_47 2010-03-03 24 +72 val_72 2010-03-03 24 +4 val_4 2010-03-03 24 +280 val_280 2010-03-03 24 +35 val_35 2010-03-03 24 +427 val_427 2010-03-03 24 +277 val_277 2010-03-03 24 +208 val_208 2010-03-03 24 +356 val_356 2010-03-03 24 +399 val_399 2010-03-03 24 +169 val_169 2010-03-03 24 +382 val_382 2010-03-03 24 +498 val_498 2010-03-03 24 +125 val_125 2010-03-03 24 +386 val_386 2010-03-03 24 +437 val_437 2010-03-03 24 +469 val_469 2010-03-03 24 +192 val_192 2010-03-03 24 +286 val_286 2010-03-03 24 +187 val_187 2010-03-03 24 +176 val_176 2010-03-03 24 +54 val_54 2010-03-03 24 +459 val_459 2010-03-03 24 +51 val_51 2010-03-03 24 +138 val_138 2010-03-03 24 +103 val_103 2010-03-03 24 +239 val_239 2010-03-03 24 +213 val_213 2010-03-03 24 +216 val_216 2010-03-03 24 +430 val_430 2010-03-03 24 +278 val_278 2010-03-03 24 +176 val_176 2010-03-03 24 +289 val_289 2010-03-03 24 +221 val_221 2010-03-03 24 +65 val_65 2010-03-03 24 +318 val_318 2010-03-03 24 +332 val_332 2010-03-03 24 +311 val_311 2010-03-03 24 +275 val_275 2010-03-03 24 +137 val_137 2010-03-03 24 +241 val_241 2010-03-03 24 +83 val_83 2010-03-03 24 +333 val_333 2010-03-03 24 +180 val_180 2010-03-03 24 +284 val_284 2010-03-03 24 +12 val_12 2010-03-03 24 +230 val_230 2010-03-03 24 +181 val_181 2010-03-03 24 +67 val_67 2010-03-03 24 +260 val_260 2010-03-03 24 +404 val_404 2010-03-03 24 +384 val_384 2010-03-03 24 +489 val_489 2010-03-03 24 +353 val_353 2010-03-03 24 +373 val_373 2010-03-03 24 +272 val_272 2010-03-03 24 +138 val_138 2010-03-03 24 +217 val_217 2010-03-03 24 +84 val_84 2010-03-03 24 +348 val_348 2010-03-03 24 +466 val_466 2010-03-03 24 +58 val_58 2010-03-03 24 +8 val_8 2010-03-03 24 +411 val_411 2010-03-03 24 +230 val_230 2010-03-03 24 +208 val_208 2010-03-03 24 +348 val_348 2010-03-03 24 +24 val_24 2010-03-03 24 +463 val_463 2010-03-03 24 +431 val_431 2010-03-03 24 +179 val_179 2010-03-03 24 +172 val_172 2010-03-03 24 +42 val_42 2010-03-03 24 +129 val_129 2010-03-03 24 +158 val_158 2010-03-03 24 +119 val_119 2010-03-03 24 +496 val_496 2010-03-03 24 +0 val_0 2010-03-03 24 +322 val_322 2010-03-03 24 +197 val_197 2010-03-03 24 +468 val_468 2010-03-03 24 +393 val_393 2010-03-03 24 +454 val_454 2010-03-03 24 +100 val_100 2010-03-03 24 +298 val_298 2010-03-03 24 +199 val_199 2010-03-03 24 +191 val_191 2010-03-03 24 +418 val_418 2010-03-03 24 +96 val_96 2010-03-03 24 +26 val_26 2010-03-03 24 +165 val_165 2010-03-03 24 +327 val_327 2010-03-03 24 +230 val_230 2010-03-03 24 +205 val_205 2010-03-03 24 +120 val_120 2010-03-03 24 +131 val_131 2010-03-03 24 +51 val_51 2010-03-03 24 +404 val_404 2010-03-03 24 +43 val_43 2010-03-03 24 +436 val_436 2010-03-03 24 +156 val_156 2010-03-03 24 +469 val_469 2010-03-03 24 +468 val_468 2010-03-03 24 +308 val_308 2010-03-03 24 +95 val_95 2010-03-03 24 +196 val_196 2010-03-03 24 +288 val_288 2010-03-03 24 +481 val_481 2010-03-03 24 +457 val_457 2010-03-03 24 +98 val_98 2010-03-03 24 +282 val_282 2010-03-03 24 +197 val_197 2010-03-03 24 +187 val_187 2010-03-03 24 +318 val_318 2010-03-03 24 +318 val_318 2010-03-03 24 +409 val_409 2010-03-03 24 +470 val_470 2010-03-03 24 +137 val_137 2010-03-03 24 +369 val_369 2010-03-03 24 +316 val_316 2010-03-03 24 +169 val_169 2010-03-03 24 +413 val_413 2010-03-03 24 +85 val_85 2010-03-03 24 +77 val_77 2010-03-03 24 +0 val_0 2010-03-03 24 +490 val_490 2010-03-03 24 +87 val_87 2010-03-03 24 +364 val_364 2010-03-03 24 +179 val_179 2010-03-03 24 +118 val_118 2010-03-03 24 +134 val_134 2010-03-03 24 +395 val_395 2010-03-03 24 +282 val_282 2010-03-03 24 +138 val_138 2010-03-03 24 +238 val_238 2010-03-03 24 +419 val_419 2010-03-03 24 +15 val_15 2010-03-03 24 +118 val_118 2010-03-03 24 +72 val_72 2010-03-03 24 +90 val_90 2010-03-03 24 +307 val_307 2010-03-03 24 +19 val_19 2010-03-03 24 +435 val_435 2010-03-03 24 +10 val_10 2010-03-03 24 +277 val_277 2010-03-03 24 +273 val_273 2010-03-03 24 +306 val_306 2010-03-03 24 +224 val_224 2010-03-03 24 +309 val_309 2010-03-03 24 +389 val_389 2010-03-03 24 +327 val_327 2010-03-03 24 +242 val_242 2010-03-03 24 +369 val_369 2010-03-03 24 +392 val_392 2010-03-03 24 +272 val_272 2010-03-03 24 +331 val_331 2010-03-03 24 +401 val_401 2010-03-03 24 +242 val_242 2010-03-03 24 +452 val_452 2010-03-03 24 +177 val_177 2010-03-03 24 +226 val_226 2010-03-03 24 +5 val_5 2010-03-03 24 +497 val_497 2010-03-03 24 +402 val_402 2010-03-03 24 +396 val_396 2010-03-03 24 +317 val_317 2010-03-03 24 +395 val_395 2010-03-03 24 +58 val_58 2010-03-03 24 +35 val_35 2010-03-03 24 +336 val_336 2010-03-03 24 +95 val_95 2010-03-03 24 +11 val_11 2010-03-03 24 +168 val_168 2010-03-03 24 +34 val_34 2010-03-03 24 +229 val_229 2010-03-03 24 +233 val_233 2010-03-03 24 +143 val_143 2010-03-03 24 +472 val_472 2010-03-03 24 +322 val_322 2010-03-03 24 +498 val_498 2010-03-03 24 +160 val_160 2010-03-03 24 +195 val_195 2010-03-03 24 +42 val_42 2010-03-03 24 +321 val_321 2010-03-03 24 +430 val_430 2010-03-03 24 +119 val_119 2010-03-03 24 +489 val_489 2010-03-03 24 +458 val_458 2010-03-03 24 +78 val_78 2010-03-03 24 +76 val_76 2010-03-03 24 +41 val_41 2010-03-03 24 +223 val_223 2010-03-03 24 +492 val_492 2010-03-03 24 +149 val_149 2010-03-03 24 +449 val_449 2010-03-03 24 +218 val_218 2010-03-03 24 +228 val_228 2010-03-03 24 +138 val_138 2010-03-03 24 +453 val_453 2010-03-03 24 +30 val_30 2010-03-03 24 +209 val_209 2010-03-03 24 +64 val_64 2010-03-03 24 +468 val_468 2010-03-03 24 +76 val_76 2010-03-03 24 +74 val_74 2010-03-03 24 +342 val_342 2010-03-03 24 +69 val_69 2010-03-03 24 +230 val_230 2010-03-03 24 +33 val_33 2010-03-03 24 +368 val_368 2010-03-03 24 +103 val_103 2010-03-03 24 +296 val_296 2010-03-03 24 +113 val_113 2010-03-03 24 +216 val_216 2010-03-03 24 +367 val_367 2010-03-03 24 +344 val_344 2010-03-03 24 +167 val_167 2010-03-03 24 +274 val_274 2010-03-03 24 +219 val_219 2010-03-03 24 +239 val_239 2010-03-03 24 +485 val_485 2010-03-03 24 +116 val_116 2010-03-03 24 +223 val_223 2010-03-03 24 +256 val_256 2010-03-03 24 +263 val_263 2010-03-03 24 +70 val_70 2010-03-03 24 +487 val_487 2010-03-03 24 +480 val_480 2010-03-03 24 +401 val_401 2010-03-03 24 +288 val_288 2010-03-03 24 +191 val_191 2010-03-03 24 +5 val_5 2010-03-03 24 +244 val_244 2010-03-03 24 +438 val_438 2010-03-03 24 +128 val_128 2010-03-03 24 +467 val_467 2010-03-03 24 +432 val_432 2010-03-03 24 +202 val_202 2010-03-03 24 +316 val_316 2010-03-03 24 +229 val_229 2010-03-03 24 +469 val_469 2010-03-03 24 +463 val_463 2010-03-03 24 +280 val_280 2010-03-03 24 +2 val_2 2010-03-03 24 +35 val_35 2010-03-03 24 +283 val_283 2010-03-03 24 +331 val_331 2010-03-03 24 +235 val_235 2010-03-03 24 +80 val_80 2010-03-03 24 +44 val_44 2010-03-03 24 +193 val_193 2010-03-03 24 +321 val_321 2010-03-03 24 +335 val_335 2010-03-03 24 +104 val_104 2010-03-03 24 +466 val_466 2010-03-03 24 +366 val_366 2010-03-03 24 +175 val_175 2010-03-03 24 +403 val_403 2010-03-03 24 +483 val_483 2010-03-03 24 +53 val_53 2010-03-03 24 +105 val_105 2010-03-03 24 +257 val_257 2010-03-03 24 +406 val_406 2010-03-03 24 +409 val_409 2010-03-03 24 +190 val_190 2010-03-03 24 +406 val_406 2010-03-03 24 +401 val_401 2010-03-03 24 +114 val_114 2010-03-03 24 +258 val_258 2010-03-03 24 +90 val_90 2010-03-03 24 +203 val_203 2010-03-03 24 +262 val_262 2010-03-03 24 +348 val_348 2010-03-03 24 +424 val_424 2010-03-03 24 +12 val_12 2010-03-03 24 +396 val_396 2010-03-03 24 +201 val_201 2010-03-03 24 +217 val_217 2010-03-03 24 +164 val_164 2010-03-03 24 +431 val_431 2010-03-03 24 +454 val_454 2010-03-03 24 +478 val_478 2010-03-03 24 +298 val_298 2010-03-03 24 +125 val_125 2010-03-03 24 +431 val_431 2010-03-03 24 +164 val_164 2010-03-03 24 +424 val_424 2010-03-03 24 +187 val_187 2010-03-03 24 +382 val_382 2010-03-03 24 +5 val_5 2010-03-03 24 +70 val_70 2010-03-03 24 +397 val_397 2010-03-03 24 +480 val_480 2010-03-03 24 +291 val_291 2010-03-03 24 +24 val_24 2010-03-03 24 +351 val_351 2010-03-03 24 +255 val_255 2010-03-03 24 +104 val_104 2010-03-03 24 +70 val_70 2010-03-03 24 +163 val_163 2010-03-03 24 +438 val_438 2010-03-03 24 +119 val_119 2010-03-03 24 +414 val_414 2010-03-03 24 +200 val_200 2010-03-03 24 +491 val_491 2010-03-03 24 +237 val_237 2010-03-03 24 +439 val_439 2010-03-03 24 +360 val_360 2010-03-03 24 +248 val_248 2010-03-03 24 +479 val_479 2010-03-03 24 +305 val_305 2010-03-03 24 +417 val_417 2010-03-03 24 +199 val_199 2010-03-03 24 +444 val_444 2010-03-03 24 +120 val_120 2010-03-03 24 +429 val_429 2010-03-03 24 +169 val_169 2010-03-03 24 +443 val_443 2010-03-03 24 +323 val_323 2010-03-03 24 +325 val_325 2010-03-03 24 +277 val_277 2010-03-03 24 +230 val_230 2010-03-03 24 +478 val_478 2010-03-03 24 +178 val_178 2010-03-03 24 +468 val_468 2010-03-03 24 +310 val_310 2010-03-03 24 +317 val_317 2010-03-03 24 +333 val_333 2010-03-03 24 +493 val_493 2010-03-03 24 +460 val_460 2010-03-03 24 +207 val_207 2010-03-03 24 +249 val_249 2010-03-03 24 +265 val_265 2010-03-03 24 +480 val_480 2010-03-03 24 +83 val_83 2010-03-03 24 +136 val_136 2010-03-03 24 +353 val_353 2010-03-03 24 +172 val_172 2010-03-03 24 +214 val_214 2010-03-03 24 +462 val_462 2010-03-03 24 +233 val_233 2010-03-03 24 +406 val_406 2010-03-03 24 +133 val_133 2010-03-03 24 +175 val_175 2010-03-03 24 +189 val_189 2010-03-03 24 +454 val_454 2010-03-03 24 +375 val_375 2010-03-03 24 +401 val_401 2010-03-03 24 +421 val_421 2010-03-03 24 +407 val_407 2010-03-03 24 +384 val_384 2010-03-03 24 +256 val_256 2010-03-03 24 +26 val_26 2010-03-03 24 +134 val_134 2010-03-03 24 +67 val_67 2010-03-03 24 +384 val_384 2010-03-03 24 +379 val_379 2010-03-03 24 +18 val_18 2010-03-03 24 +462 val_462 2010-03-03 24 +492 val_492 2010-03-03 24 +100 val_100 2010-03-03 24 +298 val_298 2010-03-03 24 +9 val_9 2010-03-03 24 +341 val_341 2010-03-03 24 +498 val_498 2010-03-03 24 +146 val_146 2010-03-03 24 +458 val_458 2010-03-03 24 +362 val_362 2010-03-03 24 +186 val_186 2010-03-03 24 +285 val_285 2010-03-03 24 +348 val_348 2010-03-03 24 +167 val_167 2010-03-03 24 +18 val_18 2010-03-03 24 +273 val_273 2010-03-03 24 +183 val_183 2010-03-03 24 +281 val_281 2010-03-03 24 +344 val_344 2010-03-03 24 +97 val_97 2010-03-03 24 +469 val_469 2010-03-03 24 +315 val_315 2010-03-03 24 +84 val_84 2010-03-03 24 +28 val_28 2010-03-03 24 +37 val_37 2010-03-03 24 +448 val_448 2010-03-03 24 +152 val_152 2010-03-03 24 +348 val_348 2010-03-03 24 +307 val_307 2010-03-03 24 +194 val_194 2010-03-03 24 +414 val_414 2010-03-03 24 +477 val_477 2010-03-03 24 +222 val_222 2010-03-03 24 +126 val_126 2010-03-03 24 +90 val_90 2010-03-03 24 +169 val_169 2010-03-03 24 +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 diff --git a/src/test/hive/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/src/test/hive/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 b/src/test/hive/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 b/src/test/hive/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 new file mode 100644 index 0000000000000..08699f286e384 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 @@ -0,0 +1,12 @@ +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 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 diff --git a/src/test/hive/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d b/src/test/hive/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 b/src/test/hive/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 b/src/test/hive/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 new file mode 100644 index 0000000000000..dfe9bcc93bcd3 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 @@ -0,0 +1,2 @@ +ds=2010-03-03/hr=22 +ds=2010-03-03/hr=33 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d b/src/test/hive/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d new file mode 100644 index 0000000000000..88fe75804e584 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d @@ -0,0 +1,34 @@ +15 val_15 2010-03-03 22 +17 val_17 2010-03-03 22 +0 val_0 2010-03-03 22 +4 val_4 2010-03-03 22 +12 val_12 2010-03-03 22 +8 val_8 2010-03-03 22 +0 val_0 2010-03-03 22 +0 val_0 2010-03-03 22 +15 val_15 2010-03-03 22 +19 val_19 2010-03-03 22 +10 val_10 2010-03-03 22 +5 val_5 2010-03-03 22 +11 val_11 2010-03-03 22 +5 val_5 2010-03-03 22 +2 val_2 2010-03-03 22 +12 val_12 2010-03-03 22 +5 val_5 2010-03-03 22 +18 val_18 2010-03-03 22 +9 val_9 2010-03-03 22 +18 val_18 2010-03-03 22 +27 val_27 2010-03-03 33 +37 val_37 2010-03-03 33 +35 val_35 2010-03-03 33 +24 val_24 2010-03-03 33 +26 val_26 2010-03-03 33 +35 val_35 2010-03-03 33 +34 val_34 2010-03-03 33 +30 val_30 2010-03-03 33 +33 val_33 2010-03-03 33 +35 val_35 2010-03-03 33 +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 diff --git a/src/test/hive/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a b/src/test/hive/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e b/src/test/hive/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e new file mode 100644 index 0000000000000..170e3b095c5a9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e @@ -0,0 +1,9 @@ +key string None +value string None + +# Partition Information +# col_name data_type comment + +value string None + +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 diff --git a/src/test/hive/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd b/src/test/hive/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 b/src/test/hive/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 b/src/test/hive/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 new file mode 100644 index 0000000000000..538ed22d29976 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 @@ -0,0 +1,2 @@ +value= +value=__HIVE_DEFAULT_PARTITION__ \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e b/src/test/hive/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e new file mode 100644 index 0000000000000..9e1bc82fe15f9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e @@ -0,0 +1,6 @@ +k1 __HIVE_DEFAULT_PARTITION__ +k1 __HIVE_DEFAULT_PARTITION__ +k2 __HIVE_DEFAULT_PARTITION__ +k2 __HIVE_DEFAULT_PARTITION__ +k3 +k3 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 b/src/test/hive/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e b/src/test/hive/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e new file mode 100644 index 0000000000000..df0090edb6b9e --- /dev/null +++ b/src/test/hive/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e @@ -0,0 +1,9 @@ +key string None +value string None + +# Partition Information +# col_name data_type comment + +value string None + +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 diff --git a/src/test/hive/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd b/src/test/hive/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 b/src/test/hive/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 b/src/test/hive/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 new file mode 100644 index 0000000000000..538ed22d29976 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 @@ -0,0 +1,2 @@ +value= +value=__HIVE_DEFAULT_PARTITION__ \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e b/src/test/hive/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e new file mode 100644 index 0000000000000..9e1bc82fe15f9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e @@ -0,0 +1,6 @@ +k1 __HIVE_DEFAULT_PARTITION__ +k1 __HIVE_DEFAULT_PARTITION__ +k2 __HIVE_DEFAULT_PARTITION__ +k2 __HIVE_DEFAULT_PARTITION__ +k3 +k3 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c b/src/test/hive/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 b/src/test/hive/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a b/src/test/hive/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 b/src/test/hive/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 new file mode 100644 index 0000000000000..99a66d603300d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 @@ -0,0 +1,3 @@ +part_key=%7B2 +part_key=1 +part_key=3%5D diff --git a/src/test/hive/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f b/src/test/hive/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 b/src/test/hive/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 new file mode 100644 index 0000000000000..def850839a47b --- /dev/null +++ b/src/test/hive/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 @@ -0,0 +1,12 @@ +key string None +value string None +ds string None +hr string None + +# Partition Information +# col_name data_type comment + +ds string None +hr string None + +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 diff --git a/src/test/hive/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b b/src/test/hive/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 b/src/test/hive/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 b/src/test/hive/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 new file mode 100644 index 0000000000000..1f345dad614ad --- /dev/null +++ b/src/test/hive/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 @@ -0,0 +1,2 @@ +ds=2010-03-23/hr=11 +ds=2010-03-23/hr=12 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf b/src/test/hive/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf new file mode 100644 index 0000000000000..5f936fa91d2f5 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf @@ -0,0 +1,1000 @@ +0 val_0 2010-03-23 11 +0 val_0 2010-03-23 11 +0 val_0 2010-03-23 11 +0 val_0 2010-03-23 11 +0 val_0 2010-03-23 11 +0 val_0 2010-03-23 11 +10 val_10 2010-03-23 11 +10 val_10 2010-03-23 11 +100 val_100 2010-03-23 11 +100 val_100 2010-03-23 11 +100 val_100 2010-03-23 11 +100 val_100 2010-03-23 11 +103 val_103 2010-03-23 11 +103 val_103 2010-03-23 11 +103 val_103 2010-03-23 11 +103 val_103 2010-03-23 11 +104 val_104 2010-03-23 11 +104 val_104 2010-03-23 11 +104 val_104 2010-03-23 11 +104 val_104 2010-03-23 11 +105 val_105 2010-03-23 11 +105 val_105 2010-03-23 11 +11 val_11 2010-03-23 11 +11 val_11 2010-03-23 11 +111 val_111 2010-03-23 11 +111 val_111 2010-03-23 11 +113 val_113 2010-03-23 11 +113 val_113 2010-03-23 11 +113 val_113 2010-03-23 11 +113 val_113 2010-03-23 11 +114 val_114 2010-03-23 11 +114 val_114 2010-03-23 11 +116 val_116 2010-03-23 11 +116 val_116 2010-03-23 11 +118 val_118 2010-03-23 11 +118 val_118 2010-03-23 11 +118 val_118 2010-03-23 11 +118 val_118 2010-03-23 11 +119 val_119 2010-03-23 11 +119 val_119 2010-03-23 11 +119 val_119 2010-03-23 11 +119 val_119 2010-03-23 11 +119 val_119 2010-03-23 11 +119 val_119 2010-03-23 11 +12 val_12 2010-03-23 11 +12 val_12 2010-03-23 11 +12 val_12 2010-03-23 11 +12 val_12 2010-03-23 11 +120 val_120 2010-03-23 11 +120 val_120 2010-03-23 11 +120 val_120 2010-03-23 11 +120 val_120 2010-03-23 11 +125 val_125 2010-03-23 11 +125 val_125 2010-03-23 11 +125 val_125 2010-03-23 11 +125 val_125 2010-03-23 11 +126 val_126 2010-03-23 11 +126 val_126 2010-03-23 11 +128 val_128 2010-03-23 11 +128 val_128 2010-03-23 11 +128 val_128 2010-03-23 11 +128 val_128 2010-03-23 11 +128 val_128 2010-03-23 11 +128 val_128 2010-03-23 11 +129 val_129 2010-03-23 11 +129 val_129 2010-03-23 11 +129 val_129 2010-03-23 11 +129 val_129 2010-03-23 11 +131 val_131 2010-03-23 11 +131 val_131 2010-03-23 11 +133 val_133 2010-03-23 11 +133 val_133 2010-03-23 11 +134 val_134 2010-03-23 11 +134 val_134 2010-03-23 11 +134 val_134 2010-03-23 11 +134 val_134 2010-03-23 11 +136 val_136 2010-03-23 11 +136 val_136 2010-03-23 11 +137 val_137 2010-03-23 11 +137 val_137 2010-03-23 11 +137 val_137 2010-03-23 11 +137 val_137 2010-03-23 11 +138 val_138 2010-03-23 11 +138 val_138 2010-03-23 11 +138 val_138 2010-03-23 11 +138 val_138 2010-03-23 11 +138 val_138 2010-03-23 11 +138 val_138 2010-03-23 11 +138 val_138 2010-03-23 11 +138 val_138 2010-03-23 11 +143 val_143 2010-03-23 11 +143 val_143 2010-03-23 11 +145 val_145 2010-03-23 11 +145 val_145 2010-03-23 11 +146 val_146 2010-03-23 11 +146 val_146 2010-03-23 11 +146 val_146 2010-03-23 11 +146 val_146 2010-03-23 11 +149 val_149 2010-03-23 11 +149 val_149 2010-03-23 11 +149 val_149 2010-03-23 11 +149 val_149 2010-03-23 11 +15 val_15 2010-03-23 11 +15 val_15 2010-03-23 11 +15 val_15 2010-03-23 11 +15 val_15 2010-03-23 11 +150 val_150 2010-03-23 11 +150 val_150 2010-03-23 11 +152 val_152 2010-03-23 11 +152 val_152 2010-03-23 11 +152 val_152 2010-03-23 11 +152 val_152 2010-03-23 11 +153 val_153 2010-03-23 11 +153 val_153 2010-03-23 11 +155 val_155 2010-03-23 11 +155 val_155 2010-03-23 11 +156 val_156 2010-03-23 11 +156 val_156 2010-03-23 11 +157 val_157 2010-03-23 11 +157 val_157 2010-03-23 11 +158 val_158 2010-03-23 11 +158 val_158 2010-03-23 11 +160 val_160 2010-03-23 11 +160 val_160 2010-03-23 11 +162 val_162 2010-03-23 11 +162 val_162 2010-03-23 11 +163 val_163 2010-03-23 11 +163 val_163 2010-03-23 11 +164 val_164 2010-03-23 11 +164 val_164 2010-03-23 11 +164 val_164 2010-03-23 11 +164 val_164 2010-03-23 11 +165 val_165 2010-03-23 11 +165 val_165 2010-03-23 11 +165 val_165 2010-03-23 11 +165 val_165 2010-03-23 11 +166 val_166 2010-03-23 11 +166 val_166 2010-03-23 11 +167 val_167 2010-03-23 11 +167 val_167 2010-03-23 11 +167 val_167 2010-03-23 11 +167 val_167 2010-03-23 11 +167 val_167 2010-03-23 11 +167 val_167 2010-03-23 11 +168 val_168 2010-03-23 11 +168 val_168 2010-03-23 11 +169 val_169 2010-03-23 11 +169 val_169 2010-03-23 11 +169 val_169 2010-03-23 11 +169 val_169 2010-03-23 11 +169 val_169 2010-03-23 11 +169 val_169 2010-03-23 11 +169 val_169 2010-03-23 11 +169 val_169 2010-03-23 11 +17 val_17 2010-03-23 11 +17 val_17 2010-03-23 11 +170 val_170 2010-03-23 11 +170 val_170 2010-03-23 11 +172 val_172 2010-03-23 11 +172 val_172 2010-03-23 11 +172 val_172 2010-03-23 11 +172 val_172 2010-03-23 11 +174 val_174 2010-03-23 11 +174 val_174 2010-03-23 11 +174 val_174 2010-03-23 11 +174 val_174 2010-03-23 11 +175 val_175 2010-03-23 11 +175 val_175 2010-03-23 11 +175 val_175 2010-03-23 11 +175 val_175 2010-03-23 11 +176 val_176 2010-03-23 11 +176 val_176 2010-03-23 11 +176 val_176 2010-03-23 11 +176 val_176 2010-03-23 11 +177 val_177 2010-03-23 11 +177 val_177 2010-03-23 11 +178 val_178 2010-03-23 11 +178 val_178 2010-03-23 11 +179 val_179 2010-03-23 11 +179 val_179 2010-03-23 11 +179 val_179 2010-03-23 11 +179 val_179 2010-03-23 11 +18 val_18 2010-03-23 11 +18 val_18 2010-03-23 11 +18 val_18 2010-03-23 11 +18 val_18 2010-03-23 11 +180 val_180 2010-03-23 11 +180 val_180 2010-03-23 11 +181 val_181 2010-03-23 11 +181 val_181 2010-03-23 11 +183 val_183 2010-03-23 11 +183 val_183 2010-03-23 11 +186 val_186 2010-03-23 11 +186 val_186 2010-03-23 11 +187 val_187 2010-03-23 11 +187 val_187 2010-03-23 11 +187 val_187 2010-03-23 11 +187 val_187 2010-03-23 11 +187 val_187 2010-03-23 11 +187 val_187 2010-03-23 11 +189 val_189 2010-03-23 11 +189 val_189 2010-03-23 11 +19 val_19 2010-03-23 11 +19 val_19 2010-03-23 11 +190 val_190 2010-03-23 11 +190 val_190 2010-03-23 11 +191 val_191 2010-03-23 11 +191 val_191 2010-03-23 11 +191 val_191 2010-03-23 11 +191 val_191 2010-03-23 11 +192 val_192 2010-03-23 11 +192 val_192 2010-03-23 11 +193 val_193 2010-03-23 11 +193 val_193 2010-03-23 11 +193 val_193 2010-03-23 11 +193 val_193 2010-03-23 11 +193 val_193 2010-03-23 11 +193 val_193 2010-03-23 11 +194 val_194 2010-03-23 11 +194 val_194 2010-03-23 11 +195 val_195 2010-03-23 11 +195 val_195 2010-03-23 11 +195 val_195 2010-03-23 11 +195 val_195 2010-03-23 11 +196 val_196 2010-03-23 11 +196 val_196 2010-03-23 11 +197 val_197 2010-03-23 11 +197 val_197 2010-03-23 11 +197 val_197 2010-03-23 11 +197 val_197 2010-03-23 11 +199 val_199 2010-03-23 11 +199 val_199 2010-03-23 11 +199 val_199 2010-03-23 11 +199 val_199 2010-03-23 11 +199 val_199 2010-03-23 11 +199 val_199 2010-03-23 11 +2 val_2 2010-03-23 11 +2 val_2 2010-03-23 11 +20 val_20 2010-03-23 11 +20 val_20 2010-03-23 11 +200 val_200 2010-03-23 11 +200 val_200 2010-03-23 11 +200 val_200 2010-03-23 11 +200 val_200 2010-03-23 11 +201 val_201 2010-03-23 11 +201 val_201 2010-03-23 11 +202 val_202 2010-03-23 11 +202 val_202 2010-03-23 11 +203 val_203 2010-03-23 11 +203 val_203 2010-03-23 11 +203 val_203 2010-03-23 11 +203 val_203 2010-03-23 11 +205 val_205 2010-03-23 11 +205 val_205 2010-03-23 11 +205 val_205 2010-03-23 11 +205 val_205 2010-03-23 11 +207 val_207 2010-03-23 11 +207 val_207 2010-03-23 11 +207 val_207 2010-03-23 11 +207 val_207 2010-03-23 11 +208 val_208 2010-03-23 11 +208 val_208 2010-03-23 11 +208 val_208 2010-03-23 11 +208 val_208 2010-03-23 11 +208 val_208 2010-03-23 11 +208 val_208 2010-03-23 11 +209 val_209 2010-03-23 11 +209 val_209 2010-03-23 11 +209 val_209 2010-03-23 11 +209 val_209 2010-03-23 11 +213 val_213 2010-03-23 11 +213 val_213 2010-03-23 11 +213 val_213 2010-03-23 11 +213 val_213 2010-03-23 11 +214 val_214 2010-03-23 11 +214 val_214 2010-03-23 11 +216 val_216 2010-03-23 11 +216 val_216 2010-03-23 11 +216 val_216 2010-03-23 11 +216 val_216 2010-03-23 11 +217 val_217 2010-03-23 11 +217 val_217 2010-03-23 11 +217 val_217 2010-03-23 11 +217 val_217 2010-03-23 11 +218 val_218 2010-03-23 11 +218 val_218 2010-03-23 11 +219 val_219 2010-03-23 11 +219 val_219 2010-03-23 11 +219 val_219 2010-03-23 11 +219 val_219 2010-03-23 11 +221 val_221 2010-03-23 11 +221 val_221 2010-03-23 11 +221 val_221 2010-03-23 11 +221 val_221 2010-03-23 11 +222 val_222 2010-03-23 11 +222 val_222 2010-03-23 11 +223 val_223 2010-03-23 11 +223 val_223 2010-03-23 11 +223 val_223 2010-03-23 11 +223 val_223 2010-03-23 11 +224 val_224 2010-03-23 11 +224 val_224 2010-03-23 11 +224 val_224 2010-03-23 11 +224 val_224 2010-03-23 11 +226 val_226 2010-03-23 11 +226 val_226 2010-03-23 11 +228 val_228 2010-03-23 11 +228 val_228 2010-03-23 11 +229 val_229 2010-03-23 11 +229 val_229 2010-03-23 11 +229 val_229 2010-03-23 11 +229 val_229 2010-03-23 11 +230 val_230 2010-03-23 11 +230 val_230 2010-03-23 11 +230 val_230 2010-03-23 11 +230 val_230 2010-03-23 11 +230 val_230 2010-03-23 11 +230 val_230 2010-03-23 11 +230 val_230 2010-03-23 11 +230 val_230 2010-03-23 11 +230 val_230 2010-03-23 11 +230 val_230 2010-03-23 11 +233 val_233 2010-03-23 11 +233 val_233 2010-03-23 11 +233 val_233 2010-03-23 11 +233 val_233 2010-03-23 11 +235 val_235 2010-03-23 11 +235 val_235 2010-03-23 11 +237 val_237 2010-03-23 11 +237 val_237 2010-03-23 11 +237 val_237 2010-03-23 11 +237 val_237 2010-03-23 11 +238 val_238 2010-03-23 11 +238 val_238 2010-03-23 11 +238 val_238 2010-03-23 11 +238 val_238 2010-03-23 11 +239 val_239 2010-03-23 11 +239 val_239 2010-03-23 11 +239 val_239 2010-03-23 11 +239 val_239 2010-03-23 11 +24 val_24 2010-03-23 11 +24 val_24 2010-03-23 11 +24 val_24 2010-03-23 11 +24 val_24 2010-03-23 11 +241 val_241 2010-03-23 11 +241 val_241 2010-03-23 11 +242 val_242 2010-03-23 11 +242 val_242 2010-03-23 11 +242 val_242 2010-03-23 11 +242 val_242 2010-03-23 11 +244 val_244 2010-03-23 11 +244 val_244 2010-03-23 11 +247 val_247 2010-03-23 11 +247 val_247 2010-03-23 11 +248 val_248 2010-03-23 11 +248 val_248 2010-03-23 11 +249 val_249 2010-03-23 11 +249 val_249 2010-03-23 11 +252 val_252 2010-03-23 11 +252 val_252 2010-03-23 11 +255 val_255 2010-03-23 11 +255 val_255 2010-03-23 11 +255 val_255 2010-03-23 11 +255 val_255 2010-03-23 11 +256 val_256 2010-03-23 11 +256 val_256 2010-03-23 11 +256 val_256 2010-03-23 11 +256 val_256 2010-03-23 11 +257 val_257 2010-03-23 11 +257 val_257 2010-03-23 11 +258 val_258 2010-03-23 11 +258 val_258 2010-03-23 11 +26 val_26 2010-03-23 11 +26 val_26 2010-03-23 11 +26 val_26 2010-03-23 11 +26 val_26 2010-03-23 11 +260 val_260 2010-03-23 11 +260 val_260 2010-03-23 11 +262 val_262 2010-03-23 11 +262 val_262 2010-03-23 11 +263 val_263 2010-03-23 11 +263 val_263 2010-03-23 11 +265 val_265 2010-03-23 11 +265 val_265 2010-03-23 11 +265 val_265 2010-03-23 11 +265 val_265 2010-03-23 11 +266 val_266 2010-03-23 11 +266 val_266 2010-03-23 11 +27 val_27 2010-03-23 11 +27 val_27 2010-03-23 11 +272 val_272 2010-03-23 11 +272 val_272 2010-03-23 11 +272 val_272 2010-03-23 11 +272 val_272 2010-03-23 11 +273 val_273 2010-03-23 11 +273 val_273 2010-03-23 11 +273 val_273 2010-03-23 11 +273 val_273 2010-03-23 11 +273 val_273 2010-03-23 11 +273 val_273 2010-03-23 11 +274 val_274 2010-03-23 11 +274 val_274 2010-03-23 11 +275 val_275 2010-03-23 11 +275 val_275 2010-03-23 11 +277 val_277 2010-03-23 11 +277 val_277 2010-03-23 11 +277 val_277 2010-03-23 11 +277 val_277 2010-03-23 11 +277 val_277 2010-03-23 11 +277 val_277 2010-03-23 11 +277 val_277 2010-03-23 11 +277 val_277 2010-03-23 11 +278 val_278 2010-03-23 11 +278 val_278 2010-03-23 11 +278 val_278 2010-03-23 11 +278 val_278 2010-03-23 11 +28 val_28 2010-03-23 11 +28 val_28 2010-03-23 11 +280 val_280 2010-03-23 11 +280 val_280 2010-03-23 11 +280 val_280 2010-03-23 11 +280 val_280 2010-03-23 11 +281 val_281 2010-03-23 11 +281 val_281 2010-03-23 11 +281 val_281 2010-03-23 11 +281 val_281 2010-03-23 11 +282 val_282 2010-03-23 11 +282 val_282 2010-03-23 11 +282 val_282 2010-03-23 11 +282 val_282 2010-03-23 11 +283 val_283 2010-03-23 11 +283 val_283 2010-03-23 11 +284 val_284 2010-03-23 11 +284 val_284 2010-03-23 11 +285 val_285 2010-03-23 11 +285 val_285 2010-03-23 11 +286 val_286 2010-03-23 11 +286 val_286 2010-03-23 11 +287 val_287 2010-03-23 11 +287 val_287 2010-03-23 11 +288 val_288 2010-03-23 11 +288 val_288 2010-03-23 11 +288 val_288 2010-03-23 11 +288 val_288 2010-03-23 11 +289 val_289 2010-03-23 11 +289 val_289 2010-03-23 11 +291 val_291 2010-03-23 11 +291 val_291 2010-03-23 11 +292 val_292 2010-03-23 11 +292 val_292 2010-03-23 11 +296 val_296 2010-03-23 11 +296 val_296 2010-03-23 11 +298 val_298 2010-03-23 11 +298 val_298 2010-03-23 11 +298 val_298 2010-03-23 11 +298 val_298 2010-03-23 11 +298 val_298 2010-03-23 11 +298 val_298 2010-03-23 11 +30 val_30 2010-03-23 11 +30 val_30 2010-03-23 11 +302 val_302 2010-03-23 11 +302 val_302 2010-03-23 11 +305 val_305 2010-03-23 11 +305 val_305 2010-03-23 11 +306 val_306 2010-03-23 11 +306 val_306 2010-03-23 11 +307 val_307 2010-03-23 11 +307 val_307 2010-03-23 11 +307 val_307 2010-03-23 11 +307 val_307 2010-03-23 11 +308 val_308 2010-03-23 11 +308 val_308 2010-03-23 11 +309 val_309 2010-03-23 11 +309 val_309 2010-03-23 11 +309 val_309 2010-03-23 11 +309 val_309 2010-03-23 11 +310 val_310 2010-03-23 11 +310 val_310 2010-03-23 11 +311 val_311 2010-03-23 11 +311 val_311 2010-03-23 11 +311 val_311 2010-03-23 11 +311 val_311 2010-03-23 11 +311 val_311 2010-03-23 11 +311 val_311 2010-03-23 11 +315 val_315 2010-03-23 11 +315 val_315 2010-03-23 11 +316 val_316 2010-03-23 11 +316 val_316 2010-03-23 11 +316 val_316 2010-03-23 11 +316 val_316 2010-03-23 11 +316 val_316 2010-03-23 11 +316 val_316 2010-03-23 11 +317 val_317 2010-03-23 11 +317 val_317 2010-03-23 11 +317 val_317 2010-03-23 11 +317 val_317 2010-03-23 11 +318 val_318 2010-03-23 11 +318 val_318 2010-03-23 11 +318 val_318 2010-03-23 11 +318 val_318 2010-03-23 11 +318 val_318 2010-03-23 11 +318 val_318 2010-03-23 11 +321 val_321 2010-03-23 11 +321 val_321 2010-03-23 11 +321 val_321 2010-03-23 11 +321 val_321 2010-03-23 11 +322 val_322 2010-03-23 11 +322 val_322 2010-03-23 11 +322 val_322 2010-03-23 11 +322 val_322 2010-03-23 11 +323 val_323 2010-03-23 11 +323 val_323 2010-03-23 11 +325 val_325 2010-03-23 11 +325 val_325 2010-03-23 11 +325 val_325 2010-03-23 11 +325 val_325 2010-03-23 11 +327 val_327 2010-03-23 11 +327 val_327 2010-03-23 11 +327 val_327 2010-03-23 11 +327 val_327 2010-03-23 11 +327 val_327 2010-03-23 11 +327 val_327 2010-03-23 11 +33 val_33 2010-03-23 11 +33 val_33 2010-03-23 11 +331 val_331 2010-03-23 11 +331 val_331 2010-03-23 11 +331 val_331 2010-03-23 11 +331 val_331 2010-03-23 11 +332 val_332 2010-03-23 11 +332 val_332 2010-03-23 11 +333 val_333 2010-03-23 11 +333 val_333 2010-03-23 11 +333 val_333 2010-03-23 11 +333 val_333 2010-03-23 11 +335 val_335 2010-03-23 11 +335 val_335 2010-03-23 11 +336 val_336 2010-03-23 11 +336 val_336 2010-03-23 11 +338 val_338 2010-03-23 11 +338 val_338 2010-03-23 11 +339 val_339 2010-03-23 11 +339 val_339 2010-03-23 11 +34 val_34 2010-03-23 11 +34 val_34 2010-03-23 11 +341 val_341 2010-03-23 11 +341 val_341 2010-03-23 11 +342 val_342 2010-03-23 11 +342 val_342 2010-03-23 11 +342 val_342 2010-03-23 11 +342 val_342 2010-03-23 11 +344 val_344 2010-03-23 11 +344 val_344 2010-03-23 11 +344 val_344 2010-03-23 11 +344 val_344 2010-03-23 11 +345 val_345 2010-03-23 11 +345 val_345 2010-03-23 11 +348 val_348 2010-03-23 11 +348 val_348 2010-03-23 11 +348 val_348 2010-03-23 11 +348 val_348 2010-03-23 11 +348 val_348 2010-03-23 11 +348 val_348 2010-03-23 11 +348 val_348 2010-03-23 11 +348 val_348 2010-03-23 11 +348 val_348 2010-03-23 11 +348 val_348 2010-03-23 11 +35 val_35 2010-03-23 11 +35 val_35 2010-03-23 11 +35 val_35 2010-03-23 11 +35 val_35 2010-03-23 11 +35 val_35 2010-03-23 11 +35 val_35 2010-03-23 11 +351 val_351 2010-03-23 11 +351 val_351 2010-03-23 11 +353 val_353 2010-03-23 11 +353 val_353 2010-03-23 11 +353 val_353 2010-03-23 11 +353 val_353 2010-03-23 11 +356 val_356 2010-03-23 11 +356 val_356 2010-03-23 11 +360 val_360 2010-03-23 11 +360 val_360 2010-03-23 11 +362 val_362 2010-03-23 11 +362 val_362 2010-03-23 11 +364 val_364 2010-03-23 11 +364 val_364 2010-03-23 11 +365 val_365 2010-03-23 11 +365 val_365 2010-03-23 11 +366 val_366 2010-03-23 11 +366 val_366 2010-03-23 11 +367 val_367 2010-03-23 11 +367 val_367 2010-03-23 11 +367 val_367 2010-03-23 11 +367 val_367 2010-03-23 11 +368 val_368 2010-03-23 11 +368 val_368 2010-03-23 11 +369 val_369 2010-03-23 11 +369 val_369 2010-03-23 11 +369 val_369 2010-03-23 11 +369 val_369 2010-03-23 11 +369 val_369 2010-03-23 11 +369 val_369 2010-03-23 11 +37 val_37 2010-03-23 11 +37 val_37 2010-03-23 11 +37 val_37 2010-03-23 11 +37 val_37 2010-03-23 11 +373 val_373 2010-03-23 11 +373 val_373 2010-03-23 11 +374 val_374 2010-03-23 11 +374 val_374 2010-03-23 11 +375 val_375 2010-03-23 11 +375 val_375 2010-03-23 11 +377 val_377 2010-03-23 11 +377 val_377 2010-03-23 11 +378 val_378 2010-03-23 11 +378 val_378 2010-03-23 11 +379 val_379 2010-03-23 11 +379 val_379 2010-03-23 11 +382 val_382 2010-03-23 11 +382 val_382 2010-03-23 11 +382 val_382 2010-03-23 11 +382 val_382 2010-03-23 11 +384 val_384 2010-03-23 11 +384 val_384 2010-03-23 11 +384 val_384 2010-03-23 11 +384 val_384 2010-03-23 11 +384 val_384 2010-03-23 11 +384 val_384 2010-03-23 11 +386 val_386 2010-03-23 11 +386 val_386 2010-03-23 11 +389 val_389 2010-03-23 11 +389 val_389 2010-03-23 11 +392 val_392 2010-03-23 11 +392 val_392 2010-03-23 11 +393 val_393 2010-03-23 11 +393 val_393 2010-03-23 11 +394 val_394 2010-03-23 11 +394 val_394 2010-03-23 11 +395 val_395 2010-03-23 11 +395 val_395 2010-03-23 11 +395 val_395 2010-03-23 11 +395 val_395 2010-03-23 11 +396 val_396 2010-03-23 11 +396 val_396 2010-03-23 11 +396 val_396 2010-03-23 11 +396 val_396 2010-03-23 11 +396 val_396 2010-03-23 11 +396 val_396 2010-03-23 11 +397 val_397 2010-03-23 11 +397 val_397 2010-03-23 11 +397 val_397 2010-03-23 11 +397 val_397 2010-03-23 11 +399 val_399 2010-03-23 11 +399 val_399 2010-03-23 11 +399 val_399 2010-03-23 11 +399 val_399 2010-03-23 11 +4 val_4 2010-03-23 11 +4 val_4 2010-03-23 11 +400 val_400 2010-03-23 11 +400 val_400 2010-03-23 11 +401 val_401 2010-03-23 11 +401 val_401 2010-03-23 11 +401 val_401 2010-03-23 11 +401 val_401 2010-03-23 11 +401 val_401 2010-03-23 11 +401 val_401 2010-03-23 11 +401 val_401 2010-03-23 11 +401 val_401 2010-03-23 11 +401 val_401 2010-03-23 11 +401 val_401 2010-03-23 11 +402 val_402 2010-03-23 11 +402 val_402 2010-03-23 11 +403 val_403 2010-03-23 11 +403 val_403 2010-03-23 11 +403 val_403 2010-03-23 11 +403 val_403 2010-03-23 11 +403 val_403 2010-03-23 11 +403 val_403 2010-03-23 11 +404 val_404 2010-03-23 11 +404 val_404 2010-03-23 11 +404 val_404 2010-03-23 11 +404 val_404 2010-03-23 11 +406 val_406 2010-03-23 11 +406 val_406 2010-03-23 11 +406 val_406 2010-03-23 11 +406 val_406 2010-03-23 11 +406 val_406 2010-03-23 11 +406 val_406 2010-03-23 11 +406 val_406 2010-03-23 11 +406 val_406 2010-03-23 11 +407 val_407 2010-03-23 11 +407 val_407 2010-03-23 11 +409 val_409 2010-03-23 11 +409 val_409 2010-03-23 11 +409 val_409 2010-03-23 11 +409 val_409 2010-03-23 11 +409 val_409 2010-03-23 11 +409 val_409 2010-03-23 11 +41 val_41 2010-03-23 11 +41 val_41 2010-03-23 11 +411 val_411 2010-03-23 11 +411 val_411 2010-03-23 11 +413 val_413 2010-03-23 11 +413 val_413 2010-03-23 11 +413 val_413 2010-03-23 11 +413 val_413 2010-03-23 11 +414 val_414 2010-03-23 11 +414 val_414 2010-03-23 11 +414 val_414 2010-03-23 11 +414 val_414 2010-03-23 11 +417 val_417 2010-03-23 11 +417 val_417 2010-03-23 11 +417 val_417 2010-03-23 11 +417 val_417 2010-03-23 11 +417 val_417 2010-03-23 11 +417 val_417 2010-03-23 11 +418 val_418 2010-03-23 11 +418 val_418 2010-03-23 11 +419 val_419 2010-03-23 11 +419 val_419 2010-03-23 11 +42 val_42 2010-03-23 11 +42 val_42 2010-03-23 11 +42 val_42 2010-03-23 11 +42 val_42 2010-03-23 11 +421 val_421 2010-03-23 11 +421 val_421 2010-03-23 11 +424 val_424 2010-03-23 11 +424 val_424 2010-03-23 11 +424 val_424 2010-03-23 11 +424 val_424 2010-03-23 11 +427 val_427 2010-03-23 11 +427 val_427 2010-03-23 11 +429 val_429 2010-03-23 11 +429 val_429 2010-03-23 11 +429 val_429 2010-03-23 11 +429 val_429 2010-03-23 11 +43 val_43 2010-03-23 11 +43 val_43 2010-03-23 11 +430 val_430 2010-03-23 11 +430 val_430 2010-03-23 11 +430 val_430 2010-03-23 11 +430 val_430 2010-03-23 11 +430 val_430 2010-03-23 11 +430 val_430 2010-03-23 11 +431 val_431 2010-03-23 11 +431 val_431 2010-03-23 11 +431 val_431 2010-03-23 11 +431 val_431 2010-03-23 11 +431 val_431 2010-03-23 11 +431 val_431 2010-03-23 11 +432 val_432 2010-03-23 11 +432 val_432 2010-03-23 11 +435 val_435 2010-03-23 11 +435 val_435 2010-03-23 11 +436 val_436 2010-03-23 11 +436 val_436 2010-03-23 11 +437 val_437 2010-03-23 11 +437 val_437 2010-03-23 11 +438 val_438 2010-03-23 11 +438 val_438 2010-03-23 11 +438 val_438 2010-03-23 11 +438 val_438 2010-03-23 11 +438 val_438 2010-03-23 11 +438 val_438 2010-03-23 11 +439 val_439 2010-03-23 11 +439 val_439 2010-03-23 11 +439 val_439 2010-03-23 11 +439 val_439 2010-03-23 11 +44 val_44 2010-03-23 11 +44 val_44 2010-03-23 11 +443 val_443 2010-03-23 11 +443 val_443 2010-03-23 11 +444 val_444 2010-03-23 11 +444 val_444 2010-03-23 11 +446 val_446 2010-03-23 11 +446 val_446 2010-03-23 11 +448 val_448 2010-03-23 11 +448 val_448 2010-03-23 11 +449 val_449 2010-03-23 11 +449 val_449 2010-03-23 11 +452 val_452 2010-03-23 11 +452 val_452 2010-03-23 11 +453 val_453 2010-03-23 11 +453 val_453 2010-03-23 11 +454 val_454 2010-03-23 11 +454 val_454 2010-03-23 11 +454 val_454 2010-03-23 11 +454 val_454 2010-03-23 11 +454 val_454 2010-03-23 11 +454 val_454 2010-03-23 11 +455 val_455 2010-03-23 11 +455 val_455 2010-03-23 11 +457 val_457 2010-03-23 11 +457 val_457 2010-03-23 11 +458 val_458 2010-03-23 11 +458 val_458 2010-03-23 11 +458 val_458 2010-03-23 11 +458 val_458 2010-03-23 11 +459 val_459 2010-03-23 11 +459 val_459 2010-03-23 11 +459 val_459 2010-03-23 11 +459 val_459 2010-03-23 11 +460 val_460 2010-03-23 11 +460 val_460 2010-03-23 11 +462 val_462 2010-03-23 11 +462 val_462 2010-03-23 11 +462 val_462 2010-03-23 11 +462 val_462 2010-03-23 11 +463 val_463 2010-03-23 11 +463 val_463 2010-03-23 11 +463 val_463 2010-03-23 11 +463 val_463 2010-03-23 11 +466 val_466 2010-03-23 11 +466 val_466 2010-03-23 11 +466 val_466 2010-03-23 11 +466 val_466 2010-03-23 11 +466 val_466 2010-03-23 11 +466 val_466 2010-03-23 11 +467 val_467 2010-03-23 11 +467 val_467 2010-03-23 11 +468 val_468 2010-03-23 11 +468 val_468 2010-03-23 11 +468 val_468 2010-03-23 11 +468 val_468 2010-03-23 11 +468 val_468 2010-03-23 11 +468 val_468 2010-03-23 11 +468 val_468 2010-03-23 11 +468 val_468 2010-03-23 11 +469 val_469 2010-03-23 11 +469 val_469 2010-03-23 11 +469 val_469 2010-03-23 11 +469 val_469 2010-03-23 11 +469 val_469 2010-03-23 11 +469 val_469 2010-03-23 11 +469 val_469 2010-03-23 11 +469 val_469 2010-03-23 11 +469 val_469 2010-03-23 11 +469 val_469 2010-03-23 11 +47 val_47 2010-03-23 11 +47 val_47 2010-03-23 11 +470 val_470 2010-03-23 11 +470 val_470 2010-03-23 11 +472 val_472 2010-03-23 11 +472 val_472 2010-03-23 11 +475 val_475 2010-03-23 11 +475 val_475 2010-03-23 11 +477 val_477 2010-03-23 11 +477 val_477 2010-03-23 11 +478 val_478 2010-03-23 11 +478 val_478 2010-03-23 11 +478 val_478 2010-03-23 11 +478 val_478 2010-03-23 11 +479 val_479 2010-03-23 11 +479 val_479 2010-03-23 11 +480 val_480 2010-03-23 11 +480 val_480 2010-03-23 11 +480 val_480 2010-03-23 11 +480 val_480 2010-03-23 11 +480 val_480 2010-03-23 11 +480 val_480 2010-03-23 11 +481 val_481 2010-03-23 11 +481 val_481 2010-03-23 11 +482 val_482 2010-03-23 11 +482 val_482 2010-03-23 11 +483 val_483 2010-03-23 11 +483 val_483 2010-03-23 11 +484 val_484 2010-03-23 11 +484 val_484 2010-03-23 11 +485 val_485 2010-03-23 11 +485 val_485 2010-03-23 11 +487 val_487 2010-03-23 11 +487 val_487 2010-03-23 11 +489 val_489 2010-03-23 11 +489 val_489 2010-03-23 11 +489 val_489 2010-03-23 11 +489 val_489 2010-03-23 11 +489 val_489 2010-03-23 11 +489 val_489 2010-03-23 11 +489 val_489 2010-03-23 11 +489 val_489 2010-03-23 11 +490 val_490 2010-03-23 11 +490 val_490 2010-03-23 11 +491 val_491 2010-03-23 11 +491 val_491 2010-03-23 11 +492 val_492 2010-03-23 11 +492 val_492 2010-03-23 11 +492 val_492 2010-03-23 11 +492 val_492 2010-03-23 11 +493 val_493 2010-03-23 11 +493 val_493 2010-03-23 11 +494 val_494 2010-03-23 11 +494 val_494 2010-03-23 11 +495 val_495 2010-03-23 11 +495 val_495 2010-03-23 11 +496 val_496 2010-03-23 11 +496 val_496 2010-03-23 11 +497 val_497 2010-03-23 11 +497 val_497 2010-03-23 11 +498 val_498 2010-03-23 11 +498 val_498 2010-03-23 11 +498 val_498 2010-03-23 11 +498 val_498 2010-03-23 11 +498 val_498 2010-03-23 11 +498 val_498 2010-03-23 11 +5 val_5 2010-03-23 11 +5 val_5 2010-03-23 11 +5 val_5 2010-03-23 11 +5 val_5 2010-03-23 11 +5 val_5 2010-03-23 11 +5 val_5 2010-03-23 11 +51 val_51 2010-03-23 11 +51 val_51 2010-03-23 11 +51 val_51 2010-03-23 11 +51 val_51 2010-03-23 11 +53 val_53 2010-03-23 11 +53 val_53 2010-03-23 11 +54 val_54 2010-03-23 11 +54 val_54 2010-03-23 11 +57 val_57 2010-03-23 11 +57 val_57 2010-03-23 11 +58 val_58 2010-03-23 11 +58 val_58 2010-03-23 11 +58 val_58 2010-03-23 11 +58 val_58 2010-03-23 11 +64 val_64 2010-03-23 11 +64 val_64 2010-03-23 11 +65 val_65 2010-03-23 11 +65 val_65 2010-03-23 11 +66 val_66 2010-03-23 11 +66 val_66 2010-03-23 11 +67 val_67 2010-03-23 11 +67 val_67 2010-03-23 11 +67 val_67 2010-03-23 11 +67 val_67 2010-03-23 11 +69 val_69 2010-03-23 11 +69 val_69 2010-03-23 11 +70 val_70 2010-03-23 11 +70 val_70 2010-03-23 11 +70 val_70 2010-03-23 11 +70 val_70 2010-03-23 11 +70 val_70 2010-03-23 11 +70 val_70 2010-03-23 11 +72 val_72 2010-03-23 11 +72 val_72 2010-03-23 11 +72 val_72 2010-03-23 11 +72 val_72 2010-03-23 11 +74 val_74 2010-03-23 11 +74 val_74 2010-03-23 11 +76 val_76 2010-03-23 11 +76 val_76 2010-03-23 11 +76 val_76 2010-03-23 11 +76 val_76 2010-03-23 11 +77 val_77 2010-03-23 11 +77 val_77 2010-03-23 11 +78 val_78 2010-03-23 11 +78 val_78 2010-03-23 11 +8 val_8 2010-03-23 11 +8 val_8 2010-03-23 11 +80 val_80 2010-03-23 11 +80 val_80 2010-03-23 11 +82 val_82 2010-03-23 11 +82 val_82 2010-03-23 11 +83 val_83 2010-03-23 11 +83 val_83 2010-03-23 11 +83 val_83 2010-03-23 11 +83 val_83 2010-03-23 11 +84 val_84 2010-03-23 11 +84 val_84 2010-03-23 11 +84 val_84 2010-03-23 11 +84 val_84 2010-03-23 11 +85 val_85 2010-03-23 11 +85 val_85 2010-03-23 11 +86 val_86 2010-03-23 11 +86 val_86 2010-03-23 11 +87 val_87 2010-03-23 11 +87 val_87 2010-03-23 11 +9 val_9 2010-03-23 11 +9 val_9 2010-03-23 11 +90 val_90 2010-03-23 11 +90 val_90 2010-03-23 11 +90 val_90 2010-03-23 11 +90 val_90 2010-03-23 11 +90 val_90 2010-03-23 11 +90 val_90 2010-03-23 11 +92 val_92 2010-03-23 11 +92 val_92 2010-03-23 11 +95 val_95 2010-03-23 11 +95 val_95 2010-03-23 11 +95 val_95 2010-03-23 11 +95 val_95 2010-03-23 11 +96 val_96 2010-03-23 11 +96 val_96 2010-03-23 11 +97 val_97 2010-03-23 11 +97 val_97 2010-03-23 11 +97 val_97 2010-03-23 11 +97 val_97 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 +98 val_98 2010-03-23 11 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 b/src/test/hive/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 new file mode 100644 index 0000000000000..45c45d0082ee3 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 @@ -0,0 +1,1000 @@ +0 val_0 2010-03-23 12 +0 val_0 2010-03-23 12 +0 val_0 2010-03-23 12 +0 val_0 2010-03-23 12 +0 val_0 2010-03-23 12 +0 val_0 2010-03-23 12 +10 val_10 2010-03-23 12 +10 val_10 2010-03-23 12 +100 val_100 2010-03-23 12 +100 val_100 2010-03-23 12 +100 val_100 2010-03-23 12 +100 val_100 2010-03-23 12 +103 val_103 2010-03-23 12 +103 val_103 2010-03-23 12 +103 val_103 2010-03-23 12 +103 val_103 2010-03-23 12 +104 val_104 2010-03-23 12 +104 val_104 2010-03-23 12 +104 val_104 2010-03-23 12 +104 val_104 2010-03-23 12 +105 val_105 2010-03-23 12 +105 val_105 2010-03-23 12 +11 val_11 2010-03-23 12 +11 val_11 2010-03-23 12 +111 val_111 2010-03-23 12 +111 val_111 2010-03-23 12 +113 val_113 2010-03-23 12 +113 val_113 2010-03-23 12 +113 val_113 2010-03-23 12 +113 val_113 2010-03-23 12 +114 val_114 2010-03-23 12 +114 val_114 2010-03-23 12 +116 val_116 2010-03-23 12 +116 val_116 2010-03-23 12 +118 val_118 2010-03-23 12 +118 val_118 2010-03-23 12 +118 val_118 2010-03-23 12 +118 val_118 2010-03-23 12 +119 val_119 2010-03-23 12 +119 val_119 2010-03-23 12 +119 val_119 2010-03-23 12 +119 val_119 2010-03-23 12 +119 val_119 2010-03-23 12 +119 val_119 2010-03-23 12 +12 val_12 2010-03-23 12 +12 val_12 2010-03-23 12 +12 val_12 2010-03-23 12 +12 val_12 2010-03-23 12 +120 val_120 2010-03-23 12 +120 val_120 2010-03-23 12 +120 val_120 2010-03-23 12 +120 val_120 2010-03-23 12 +125 val_125 2010-03-23 12 +125 val_125 2010-03-23 12 +125 val_125 2010-03-23 12 +125 val_125 2010-03-23 12 +126 val_126 2010-03-23 12 +126 val_126 2010-03-23 12 +128 val_128 2010-03-23 12 +128 val_128 2010-03-23 12 +128 val_128 2010-03-23 12 +128 val_128 2010-03-23 12 +128 val_128 2010-03-23 12 +128 val_128 2010-03-23 12 +129 val_129 2010-03-23 12 +129 val_129 2010-03-23 12 +129 val_129 2010-03-23 12 +129 val_129 2010-03-23 12 +131 val_131 2010-03-23 12 +131 val_131 2010-03-23 12 +133 val_133 2010-03-23 12 +133 val_133 2010-03-23 12 +134 val_134 2010-03-23 12 +134 val_134 2010-03-23 12 +134 val_134 2010-03-23 12 +134 val_134 2010-03-23 12 +136 val_136 2010-03-23 12 +136 val_136 2010-03-23 12 +137 val_137 2010-03-23 12 +137 val_137 2010-03-23 12 +137 val_137 2010-03-23 12 +137 val_137 2010-03-23 12 +138 val_138 2010-03-23 12 +138 val_138 2010-03-23 12 +138 val_138 2010-03-23 12 +138 val_138 2010-03-23 12 +138 val_138 2010-03-23 12 +138 val_138 2010-03-23 12 +138 val_138 2010-03-23 12 +138 val_138 2010-03-23 12 +143 val_143 2010-03-23 12 +143 val_143 2010-03-23 12 +145 val_145 2010-03-23 12 +145 val_145 2010-03-23 12 +146 val_146 2010-03-23 12 +146 val_146 2010-03-23 12 +146 val_146 2010-03-23 12 +146 val_146 2010-03-23 12 +149 val_149 2010-03-23 12 +149 val_149 2010-03-23 12 +149 val_149 2010-03-23 12 +149 val_149 2010-03-23 12 +15 val_15 2010-03-23 12 +15 val_15 2010-03-23 12 +15 val_15 2010-03-23 12 +15 val_15 2010-03-23 12 +150 val_150 2010-03-23 12 +150 val_150 2010-03-23 12 +152 val_152 2010-03-23 12 +152 val_152 2010-03-23 12 +152 val_152 2010-03-23 12 +152 val_152 2010-03-23 12 +153 val_153 2010-03-23 12 +153 val_153 2010-03-23 12 +155 val_155 2010-03-23 12 +155 val_155 2010-03-23 12 +156 val_156 2010-03-23 12 +156 val_156 2010-03-23 12 +157 val_157 2010-03-23 12 +157 val_157 2010-03-23 12 +158 val_158 2010-03-23 12 +158 val_158 2010-03-23 12 +160 val_160 2010-03-23 12 +160 val_160 2010-03-23 12 +162 val_162 2010-03-23 12 +162 val_162 2010-03-23 12 +163 val_163 2010-03-23 12 +163 val_163 2010-03-23 12 +164 val_164 2010-03-23 12 +164 val_164 2010-03-23 12 +164 val_164 2010-03-23 12 +164 val_164 2010-03-23 12 +165 val_165 2010-03-23 12 +165 val_165 2010-03-23 12 +165 val_165 2010-03-23 12 +165 val_165 2010-03-23 12 +166 val_166 2010-03-23 12 +166 val_166 2010-03-23 12 +167 val_167 2010-03-23 12 +167 val_167 2010-03-23 12 +167 val_167 2010-03-23 12 +167 val_167 2010-03-23 12 +167 val_167 2010-03-23 12 +167 val_167 2010-03-23 12 +168 val_168 2010-03-23 12 +168 val_168 2010-03-23 12 +169 val_169 2010-03-23 12 +169 val_169 2010-03-23 12 +169 val_169 2010-03-23 12 +169 val_169 2010-03-23 12 +169 val_169 2010-03-23 12 +169 val_169 2010-03-23 12 +169 val_169 2010-03-23 12 +169 val_169 2010-03-23 12 +17 val_17 2010-03-23 12 +17 val_17 2010-03-23 12 +170 val_170 2010-03-23 12 +170 val_170 2010-03-23 12 +172 val_172 2010-03-23 12 +172 val_172 2010-03-23 12 +172 val_172 2010-03-23 12 +172 val_172 2010-03-23 12 +174 val_174 2010-03-23 12 +174 val_174 2010-03-23 12 +174 val_174 2010-03-23 12 +174 val_174 2010-03-23 12 +175 val_175 2010-03-23 12 +175 val_175 2010-03-23 12 +175 val_175 2010-03-23 12 +175 val_175 2010-03-23 12 +176 val_176 2010-03-23 12 +176 val_176 2010-03-23 12 +176 val_176 2010-03-23 12 +176 val_176 2010-03-23 12 +177 val_177 2010-03-23 12 +177 val_177 2010-03-23 12 +178 val_178 2010-03-23 12 +178 val_178 2010-03-23 12 +179 val_179 2010-03-23 12 +179 val_179 2010-03-23 12 +179 val_179 2010-03-23 12 +179 val_179 2010-03-23 12 +18 val_18 2010-03-23 12 +18 val_18 2010-03-23 12 +18 val_18 2010-03-23 12 +18 val_18 2010-03-23 12 +180 val_180 2010-03-23 12 +180 val_180 2010-03-23 12 +181 val_181 2010-03-23 12 +181 val_181 2010-03-23 12 +183 val_183 2010-03-23 12 +183 val_183 2010-03-23 12 +186 val_186 2010-03-23 12 +186 val_186 2010-03-23 12 +187 val_187 2010-03-23 12 +187 val_187 2010-03-23 12 +187 val_187 2010-03-23 12 +187 val_187 2010-03-23 12 +187 val_187 2010-03-23 12 +187 val_187 2010-03-23 12 +189 val_189 2010-03-23 12 +189 val_189 2010-03-23 12 +19 val_19 2010-03-23 12 +19 val_19 2010-03-23 12 +190 val_190 2010-03-23 12 +190 val_190 2010-03-23 12 +191 val_191 2010-03-23 12 +191 val_191 2010-03-23 12 +191 val_191 2010-03-23 12 +191 val_191 2010-03-23 12 +192 val_192 2010-03-23 12 +192 val_192 2010-03-23 12 +193 val_193 2010-03-23 12 +193 val_193 2010-03-23 12 +193 val_193 2010-03-23 12 +193 val_193 2010-03-23 12 +193 val_193 2010-03-23 12 +193 val_193 2010-03-23 12 +194 val_194 2010-03-23 12 +194 val_194 2010-03-23 12 +195 val_195 2010-03-23 12 +195 val_195 2010-03-23 12 +195 val_195 2010-03-23 12 +195 val_195 2010-03-23 12 +196 val_196 2010-03-23 12 +196 val_196 2010-03-23 12 +197 val_197 2010-03-23 12 +197 val_197 2010-03-23 12 +197 val_197 2010-03-23 12 +197 val_197 2010-03-23 12 +199 val_199 2010-03-23 12 +199 val_199 2010-03-23 12 +199 val_199 2010-03-23 12 +199 val_199 2010-03-23 12 +199 val_199 2010-03-23 12 +199 val_199 2010-03-23 12 +2 val_2 2010-03-23 12 +2 val_2 2010-03-23 12 +20 val_20 2010-03-23 12 +20 val_20 2010-03-23 12 +200 val_200 2010-03-23 12 +200 val_200 2010-03-23 12 +200 val_200 2010-03-23 12 +200 val_200 2010-03-23 12 +201 val_201 2010-03-23 12 +201 val_201 2010-03-23 12 +202 val_202 2010-03-23 12 +202 val_202 2010-03-23 12 +203 val_203 2010-03-23 12 +203 val_203 2010-03-23 12 +203 val_203 2010-03-23 12 +203 val_203 2010-03-23 12 +205 val_205 2010-03-23 12 +205 val_205 2010-03-23 12 +205 val_205 2010-03-23 12 +205 val_205 2010-03-23 12 +207 val_207 2010-03-23 12 +207 val_207 2010-03-23 12 +207 val_207 2010-03-23 12 +207 val_207 2010-03-23 12 +208 val_208 2010-03-23 12 +208 val_208 2010-03-23 12 +208 val_208 2010-03-23 12 +208 val_208 2010-03-23 12 +208 val_208 2010-03-23 12 +208 val_208 2010-03-23 12 +209 val_209 2010-03-23 12 +209 val_209 2010-03-23 12 +209 val_209 2010-03-23 12 +209 val_209 2010-03-23 12 +213 val_213 2010-03-23 12 +213 val_213 2010-03-23 12 +213 val_213 2010-03-23 12 +213 val_213 2010-03-23 12 +214 val_214 2010-03-23 12 +214 val_214 2010-03-23 12 +216 val_216 2010-03-23 12 +216 val_216 2010-03-23 12 +216 val_216 2010-03-23 12 +216 val_216 2010-03-23 12 +217 val_217 2010-03-23 12 +217 val_217 2010-03-23 12 +217 val_217 2010-03-23 12 +217 val_217 2010-03-23 12 +218 val_218 2010-03-23 12 +218 val_218 2010-03-23 12 +219 val_219 2010-03-23 12 +219 val_219 2010-03-23 12 +219 val_219 2010-03-23 12 +219 val_219 2010-03-23 12 +221 val_221 2010-03-23 12 +221 val_221 2010-03-23 12 +221 val_221 2010-03-23 12 +221 val_221 2010-03-23 12 +222 val_222 2010-03-23 12 +222 val_222 2010-03-23 12 +223 val_223 2010-03-23 12 +223 val_223 2010-03-23 12 +223 val_223 2010-03-23 12 +223 val_223 2010-03-23 12 +224 val_224 2010-03-23 12 +224 val_224 2010-03-23 12 +224 val_224 2010-03-23 12 +224 val_224 2010-03-23 12 +226 val_226 2010-03-23 12 +226 val_226 2010-03-23 12 +228 val_228 2010-03-23 12 +228 val_228 2010-03-23 12 +229 val_229 2010-03-23 12 +229 val_229 2010-03-23 12 +229 val_229 2010-03-23 12 +229 val_229 2010-03-23 12 +230 val_230 2010-03-23 12 +230 val_230 2010-03-23 12 +230 val_230 2010-03-23 12 +230 val_230 2010-03-23 12 +230 val_230 2010-03-23 12 +230 val_230 2010-03-23 12 +230 val_230 2010-03-23 12 +230 val_230 2010-03-23 12 +230 val_230 2010-03-23 12 +230 val_230 2010-03-23 12 +233 val_233 2010-03-23 12 +233 val_233 2010-03-23 12 +233 val_233 2010-03-23 12 +233 val_233 2010-03-23 12 +235 val_235 2010-03-23 12 +235 val_235 2010-03-23 12 +237 val_237 2010-03-23 12 +237 val_237 2010-03-23 12 +237 val_237 2010-03-23 12 +237 val_237 2010-03-23 12 +238 val_238 2010-03-23 12 +238 val_238 2010-03-23 12 +238 val_238 2010-03-23 12 +238 val_238 2010-03-23 12 +239 val_239 2010-03-23 12 +239 val_239 2010-03-23 12 +239 val_239 2010-03-23 12 +239 val_239 2010-03-23 12 +24 val_24 2010-03-23 12 +24 val_24 2010-03-23 12 +24 val_24 2010-03-23 12 +24 val_24 2010-03-23 12 +241 val_241 2010-03-23 12 +241 val_241 2010-03-23 12 +242 val_242 2010-03-23 12 +242 val_242 2010-03-23 12 +242 val_242 2010-03-23 12 +242 val_242 2010-03-23 12 +244 val_244 2010-03-23 12 +244 val_244 2010-03-23 12 +247 val_247 2010-03-23 12 +247 val_247 2010-03-23 12 +248 val_248 2010-03-23 12 +248 val_248 2010-03-23 12 +249 val_249 2010-03-23 12 +249 val_249 2010-03-23 12 +252 val_252 2010-03-23 12 +252 val_252 2010-03-23 12 +255 val_255 2010-03-23 12 +255 val_255 2010-03-23 12 +255 val_255 2010-03-23 12 +255 val_255 2010-03-23 12 +256 val_256 2010-03-23 12 +256 val_256 2010-03-23 12 +256 val_256 2010-03-23 12 +256 val_256 2010-03-23 12 +257 val_257 2010-03-23 12 +257 val_257 2010-03-23 12 +258 val_258 2010-03-23 12 +258 val_258 2010-03-23 12 +26 val_26 2010-03-23 12 +26 val_26 2010-03-23 12 +26 val_26 2010-03-23 12 +26 val_26 2010-03-23 12 +260 val_260 2010-03-23 12 +260 val_260 2010-03-23 12 +262 val_262 2010-03-23 12 +262 val_262 2010-03-23 12 +263 val_263 2010-03-23 12 +263 val_263 2010-03-23 12 +265 val_265 2010-03-23 12 +265 val_265 2010-03-23 12 +265 val_265 2010-03-23 12 +265 val_265 2010-03-23 12 +266 val_266 2010-03-23 12 +266 val_266 2010-03-23 12 +27 val_27 2010-03-23 12 +27 val_27 2010-03-23 12 +272 val_272 2010-03-23 12 +272 val_272 2010-03-23 12 +272 val_272 2010-03-23 12 +272 val_272 2010-03-23 12 +273 val_273 2010-03-23 12 +273 val_273 2010-03-23 12 +273 val_273 2010-03-23 12 +273 val_273 2010-03-23 12 +273 val_273 2010-03-23 12 +273 val_273 2010-03-23 12 +274 val_274 2010-03-23 12 +274 val_274 2010-03-23 12 +275 val_275 2010-03-23 12 +275 val_275 2010-03-23 12 +277 val_277 2010-03-23 12 +277 val_277 2010-03-23 12 +277 val_277 2010-03-23 12 +277 val_277 2010-03-23 12 +277 val_277 2010-03-23 12 +277 val_277 2010-03-23 12 +277 val_277 2010-03-23 12 +277 val_277 2010-03-23 12 +278 val_278 2010-03-23 12 +278 val_278 2010-03-23 12 +278 val_278 2010-03-23 12 +278 val_278 2010-03-23 12 +28 val_28 2010-03-23 12 +28 val_28 2010-03-23 12 +280 val_280 2010-03-23 12 +280 val_280 2010-03-23 12 +280 val_280 2010-03-23 12 +280 val_280 2010-03-23 12 +281 val_281 2010-03-23 12 +281 val_281 2010-03-23 12 +281 val_281 2010-03-23 12 +281 val_281 2010-03-23 12 +282 val_282 2010-03-23 12 +282 val_282 2010-03-23 12 +282 val_282 2010-03-23 12 +282 val_282 2010-03-23 12 +283 val_283 2010-03-23 12 +283 val_283 2010-03-23 12 +284 val_284 2010-03-23 12 +284 val_284 2010-03-23 12 +285 val_285 2010-03-23 12 +285 val_285 2010-03-23 12 +286 val_286 2010-03-23 12 +286 val_286 2010-03-23 12 +287 val_287 2010-03-23 12 +287 val_287 2010-03-23 12 +288 val_288 2010-03-23 12 +288 val_288 2010-03-23 12 +288 val_288 2010-03-23 12 +288 val_288 2010-03-23 12 +289 val_289 2010-03-23 12 +289 val_289 2010-03-23 12 +291 val_291 2010-03-23 12 +291 val_291 2010-03-23 12 +292 val_292 2010-03-23 12 +292 val_292 2010-03-23 12 +296 val_296 2010-03-23 12 +296 val_296 2010-03-23 12 +298 val_298 2010-03-23 12 +298 val_298 2010-03-23 12 +298 val_298 2010-03-23 12 +298 val_298 2010-03-23 12 +298 val_298 2010-03-23 12 +298 val_298 2010-03-23 12 +30 val_30 2010-03-23 12 +30 val_30 2010-03-23 12 +302 val_302 2010-03-23 12 +302 val_302 2010-03-23 12 +305 val_305 2010-03-23 12 +305 val_305 2010-03-23 12 +306 val_306 2010-03-23 12 +306 val_306 2010-03-23 12 +307 val_307 2010-03-23 12 +307 val_307 2010-03-23 12 +307 val_307 2010-03-23 12 +307 val_307 2010-03-23 12 +308 val_308 2010-03-23 12 +308 val_308 2010-03-23 12 +309 val_309 2010-03-23 12 +309 val_309 2010-03-23 12 +309 val_309 2010-03-23 12 +309 val_309 2010-03-23 12 +310 val_310 2010-03-23 12 +310 val_310 2010-03-23 12 +311 val_311 2010-03-23 12 +311 val_311 2010-03-23 12 +311 val_311 2010-03-23 12 +311 val_311 2010-03-23 12 +311 val_311 2010-03-23 12 +311 val_311 2010-03-23 12 +315 val_315 2010-03-23 12 +315 val_315 2010-03-23 12 +316 val_316 2010-03-23 12 +316 val_316 2010-03-23 12 +316 val_316 2010-03-23 12 +316 val_316 2010-03-23 12 +316 val_316 2010-03-23 12 +316 val_316 2010-03-23 12 +317 val_317 2010-03-23 12 +317 val_317 2010-03-23 12 +317 val_317 2010-03-23 12 +317 val_317 2010-03-23 12 +318 val_318 2010-03-23 12 +318 val_318 2010-03-23 12 +318 val_318 2010-03-23 12 +318 val_318 2010-03-23 12 +318 val_318 2010-03-23 12 +318 val_318 2010-03-23 12 +321 val_321 2010-03-23 12 +321 val_321 2010-03-23 12 +321 val_321 2010-03-23 12 +321 val_321 2010-03-23 12 +322 val_322 2010-03-23 12 +322 val_322 2010-03-23 12 +322 val_322 2010-03-23 12 +322 val_322 2010-03-23 12 +323 val_323 2010-03-23 12 +323 val_323 2010-03-23 12 +325 val_325 2010-03-23 12 +325 val_325 2010-03-23 12 +325 val_325 2010-03-23 12 +325 val_325 2010-03-23 12 +327 val_327 2010-03-23 12 +327 val_327 2010-03-23 12 +327 val_327 2010-03-23 12 +327 val_327 2010-03-23 12 +327 val_327 2010-03-23 12 +327 val_327 2010-03-23 12 +33 val_33 2010-03-23 12 +33 val_33 2010-03-23 12 +331 val_331 2010-03-23 12 +331 val_331 2010-03-23 12 +331 val_331 2010-03-23 12 +331 val_331 2010-03-23 12 +332 val_332 2010-03-23 12 +332 val_332 2010-03-23 12 +333 val_333 2010-03-23 12 +333 val_333 2010-03-23 12 +333 val_333 2010-03-23 12 +333 val_333 2010-03-23 12 +335 val_335 2010-03-23 12 +335 val_335 2010-03-23 12 +336 val_336 2010-03-23 12 +336 val_336 2010-03-23 12 +338 val_338 2010-03-23 12 +338 val_338 2010-03-23 12 +339 val_339 2010-03-23 12 +339 val_339 2010-03-23 12 +34 val_34 2010-03-23 12 +34 val_34 2010-03-23 12 +341 val_341 2010-03-23 12 +341 val_341 2010-03-23 12 +342 val_342 2010-03-23 12 +342 val_342 2010-03-23 12 +342 val_342 2010-03-23 12 +342 val_342 2010-03-23 12 +344 val_344 2010-03-23 12 +344 val_344 2010-03-23 12 +344 val_344 2010-03-23 12 +344 val_344 2010-03-23 12 +345 val_345 2010-03-23 12 +345 val_345 2010-03-23 12 +348 val_348 2010-03-23 12 +348 val_348 2010-03-23 12 +348 val_348 2010-03-23 12 +348 val_348 2010-03-23 12 +348 val_348 2010-03-23 12 +348 val_348 2010-03-23 12 +348 val_348 2010-03-23 12 +348 val_348 2010-03-23 12 +348 val_348 2010-03-23 12 +348 val_348 2010-03-23 12 +35 val_35 2010-03-23 12 +35 val_35 2010-03-23 12 +35 val_35 2010-03-23 12 +35 val_35 2010-03-23 12 +35 val_35 2010-03-23 12 +35 val_35 2010-03-23 12 +351 val_351 2010-03-23 12 +351 val_351 2010-03-23 12 +353 val_353 2010-03-23 12 +353 val_353 2010-03-23 12 +353 val_353 2010-03-23 12 +353 val_353 2010-03-23 12 +356 val_356 2010-03-23 12 +356 val_356 2010-03-23 12 +360 val_360 2010-03-23 12 +360 val_360 2010-03-23 12 +362 val_362 2010-03-23 12 +362 val_362 2010-03-23 12 +364 val_364 2010-03-23 12 +364 val_364 2010-03-23 12 +365 val_365 2010-03-23 12 +365 val_365 2010-03-23 12 +366 val_366 2010-03-23 12 +366 val_366 2010-03-23 12 +367 val_367 2010-03-23 12 +367 val_367 2010-03-23 12 +367 val_367 2010-03-23 12 +367 val_367 2010-03-23 12 +368 val_368 2010-03-23 12 +368 val_368 2010-03-23 12 +369 val_369 2010-03-23 12 +369 val_369 2010-03-23 12 +369 val_369 2010-03-23 12 +369 val_369 2010-03-23 12 +369 val_369 2010-03-23 12 +369 val_369 2010-03-23 12 +37 val_37 2010-03-23 12 +37 val_37 2010-03-23 12 +37 val_37 2010-03-23 12 +37 val_37 2010-03-23 12 +373 val_373 2010-03-23 12 +373 val_373 2010-03-23 12 +374 val_374 2010-03-23 12 +374 val_374 2010-03-23 12 +375 val_375 2010-03-23 12 +375 val_375 2010-03-23 12 +377 val_377 2010-03-23 12 +377 val_377 2010-03-23 12 +378 val_378 2010-03-23 12 +378 val_378 2010-03-23 12 +379 val_379 2010-03-23 12 +379 val_379 2010-03-23 12 +382 val_382 2010-03-23 12 +382 val_382 2010-03-23 12 +382 val_382 2010-03-23 12 +382 val_382 2010-03-23 12 +384 val_384 2010-03-23 12 +384 val_384 2010-03-23 12 +384 val_384 2010-03-23 12 +384 val_384 2010-03-23 12 +384 val_384 2010-03-23 12 +384 val_384 2010-03-23 12 +386 val_386 2010-03-23 12 +386 val_386 2010-03-23 12 +389 val_389 2010-03-23 12 +389 val_389 2010-03-23 12 +392 val_392 2010-03-23 12 +392 val_392 2010-03-23 12 +393 val_393 2010-03-23 12 +393 val_393 2010-03-23 12 +394 val_394 2010-03-23 12 +394 val_394 2010-03-23 12 +395 val_395 2010-03-23 12 +395 val_395 2010-03-23 12 +395 val_395 2010-03-23 12 +395 val_395 2010-03-23 12 +396 val_396 2010-03-23 12 +396 val_396 2010-03-23 12 +396 val_396 2010-03-23 12 +396 val_396 2010-03-23 12 +396 val_396 2010-03-23 12 +396 val_396 2010-03-23 12 +397 val_397 2010-03-23 12 +397 val_397 2010-03-23 12 +397 val_397 2010-03-23 12 +397 val_397 2010-03-23 12 +399 val_399 2010-03-23 12 +399 val_399 2010-03-23 12 +399 val_399 2010-03-23 12 +399 val_399 2010-03-23 12 +4 val_4 2010-03-23 12 +4 val_4 2010-03-23 12 +400 val_400 2010-03-23 12 +400 val_400 2010-03-23 12 +401 val_401 2010-03-23 12 +401 val_401 2010-03-23 12 +401 val_401 2010-03-23 12 +401 val_401 2010-03-23 12 +401 val_401 2010-03-23 12 +401 val_401 2010-03-23 12 +401 val_401 2010-03-23 12 +401 val_401 2010-03-23 12 +401 val_401 2010-03-23 12 +401 val_401 2010-03-23 12 +402 val_402 2010-03-23 12 +402 val_402 2010-03-23 12 +403 val_403 2010-03-23 12 +403 val_403 2010-03-23 12 +403 val_403 2010-03-23 12 +403 val_403 2010-03-23 12 +403 val_403 2010-03-23 12 +403 val_403 2010-03-23 12 +404 val_404 2010-03-23 12 +404 val_404 2010-03-23 12 +404 val_404 2010-03-23 12 +404 val_404 2010-03-23 12 +406 val_406 2010-03-23 12 +406 val_406 2010-03-23 12 +406 val_406 2010-03-23 12 +406 val_406 2010-03-23 12 +406 val_406 2010-03-23 12 +406 val_406 2010-03-23 12 +406 val_406 2010-03-23 12 +406 val_406 2010-03-23 12 +407 val_407 2010-03-23 12 +407 val_407 2010-03-23 12 +409 val_409 2010-03-23 12 +409 val_409 2010-03-23 12 +409 val_409 2010-03-23 12 +409 val_409 2010-03-23 12 +409 val_409 2010-03-23 12 +409 val_409 2010-03-23 12 +41 val_41 2010-03-23 12 +41 val_41 2010-03-23 12 +411 val_411 2010-03-23 12 +411 val_411 2010-03-23 12 +413 val_413 2010-03-23 12 +413 val_413 2010-03-23 12 +413 val_413 2010-03-23 12 +413 val_413 2010-03-23 12 +414 val_414 2010-03-23 12 +414 val_414 2010-03-23 12 +414 val_414 2010-03-23 12 +414 val_414 2010-03-23 12 +417 val_417 2010-03-23 12 +417 val_417 2010-03-23 12 +417 val_417 2010-03-23 12 +417 val_417 2010-03-23 12 +417 val_417 2010-03-23 12 +417 val_417 2010-03-23 12 +418 val_418 2010-03-23 12 +418 val_418 2010-03-23 12 +419 val_419 2010-03-23 12 +419 val_419 2010-03-23 12 +42 val_42 2010-03-23 12 +42 val_42 2010-03-23 12 +42 val_42 2010-03-23 12 +42 val_42 2010-03-23 12 +421 val_421 2010-03-23 12 +421 val_421 2010-03-23 12 +424 val_424 2010-03-23 12 +424 val_424 2010-03-23 12 +424 val_424 2010-03-23 12 +424 val_424 2010-03-23 12 +427 val_427 2010-03-23 12 +427 val_427 2010-03-23 12 +429 val_429 2010-03-23 12 +429 val_429 2010-03-23 12 +429 val_429 2010-03-23 12 +429 val_429 2010-03-23 12 +43 val_43 2010-03-23 12 +43 val_43 2010-03-23 12 +430 val_430 2010-03-23 12 +430 val_430 2010-03-23 12 +430 val_430 2010-03-23 12 +430 val_430 2010-03-23 12 +430 val_430 2010-03-23 12 +430 val_430 2010-03-23 12 +431 val_431 2010-03-23 12 +431 val_431 2010-03-23 12 +431 val_431 2010-03-23 12 +431 val_431 2010-03-23 12 +431 val_431 2010-03-23 12 +431 val_431 2010-03-23 12 +432 val_432 2010-03-23 12 +432 val_432 2010-03-23 12 +435 val_435 2010-03-23 12 +435 val_435 2010-03-23 12 +436 val_436 2010-03-23 12 +436 val_436 2010-03-23 12 +437 val_437 2010-03-23 12 +437 val_437 2010-03-23 12 +438 val_438 2010-03-23 12 +438 val_438 2010-03-23 12 +438 val_438 2010-03-23 12 +438 val_438 2010-03-23 12 +438 val_438 2010-03-23 12 +438 val_438 2010-03-23 12 +439 val_439 2010-03-23 12 +439 val_439 2010-03-23 12 +439 val_439 2010-03-23 12 +439 val_439 2010-03-23 12 +44 val_44 2010-03-23 12 +44 val_44 2010-03-23 12 +443 val_443 2010-03-23 12 +443 val_443 2010-03-23 12 +444 val_444 2010-03-23 12 +444 val_444 2010-03-23 12 +446 val_446 2010-03-23 12 +446 val_446 2010-03-23 12 +448 val_448 2010-03-23 12 +448 val_448 2010-03-23 12 +449 val_449 2010-03-23 12 +449 val_449 2010-03-23 12 +452 val_452 2010-03-23 12 +452 val_452 2010-03-23 12 +453 val_453 2010-03-23 12 +453 val_453 2010-03-23 12 +454 val_454 2010-03-23 12 +454 val_454 2010-03-23 12 +454 val_454 2010-03-23 12 +454 val_454 2010-03-23 12 +454 val_454 2010-03-23 12 +454 val_454 2010-03-23 12 +455 val_455 2010-03-23 12 +455 val_455 2010-03-23 12 +457 val_457 2010-03-23 12 +457 val_457 2010-03-23 12 +458 val_458 2010-03-23 12 +458 val_458 2010-03-23 12 +458 val_458 2010-03-23 12 +458 val_458 2010-03-23 12 +459 val_459 2010-03-23 12 +459 val_459 2010-03-23 12 +459 val_459 2010-03-23 12 +459 val_459 2010-03-23 12 +460 val_460 2010-03-23 12 +460 val_460 2010-03-23 12 +462 val_462 2010-03-23 12 +462 val_462 2010-03-23 12 +462 val_462 2010-03-23 12 +462 val_462 2010-03-23 12 +463 val_463 2010-03-23 12 +463 val_463 2010-03-23 12 +463 val_463 2010-03-23 12 +463 val_463 2010-03-23 12 +466 val_466 2010-03-23 12 +466 val_466 2010-03-23 12 +466 val_466 2010-03-23 12 +466 val_466 2010-03-23 12 +466 val_466 2010-03-23 12 +466 val_466 2010-03-23 12 +467 val_467 2010-03-23 12 +467 val_467 2010-03-23 12 +468 val_468 2010-03-23 12 +468 val_468 2010-03-23 12 +468 val_468 2010-03-23 12 +468 val_468 2010-03-23 12 +468 val_468 2010-03-23 12 +468 val_468 2010-03-23 12 +468 val_468 2010-03-23 12 +468 val_468 2010-03-23 12 +469 val_469 2010-03-23 12 +469 val_469 2010-03-23 12 +469 val_469 2010-03-23 12 +469 val_469 2010-03-23 12 +469 val_469 2010-03-23 12 +469 val_469 2010-03-23 12 +469 val_469 2010-03-23 12 +469 val_469 2010-03-23 12 +469 val_469 2010-03-23 12 +469 val_469 2010-03-23 12 +47 val_47 2010-03-23 12 +47 val_47 2010-03-23 12 +470 val_470 2010-03-23 12 +470 val_470 2010-03-23 12 +472 val_472 2010-03-23 12 +472 val_472 2010-03-23 12 +475 val_475 2010-03-23 12 +475 val_475 2010-03-23 12 +477 val_477 2010-03-23 12 +477 val_477 2010-03-23 12 +478 val_478 2010-03-23 12 +478 val_478 2010-03-23 12 +478 val_478 2010-03-23 12 +478 val_478 2010-03-23 12 +479 val_479 2010-03-23 12 +479 val_479 2010-03-23 12 +480 val_480 2010-03-23 12 +480 val_480 2010-03-23 12 +480 val_480 2010-03-23 12 +480 val_480 2010-03-23 12 +480 val_480 2010-03-23 12 +480 val_480 2010-03-23 12 +481 val_481 2010-03-23 12 +481 val_481 2010-03-23 12 +482 val_482 2010-03-23 12 +482 val_482 2010-03-23 12 +483 val_483 2010-03-23 12 +483 val_483 2010-03-23 12 +484 val_484 2010-03-23 12 +484 val_484 2010-03-23 12 +485 val_485 2010-03-23 12 +485 val_485 2010-03-23 12 +487 val_487 2010-03-23 12 +487 val_487 2010-03-23 12 +489 val_489 2010-03-23 12 +489 val_489 2010-03-23 12 +489 val_489 2010-03-23 12 +489 val_489 2010-03-23 12 +489 val_489 2010-03-23 12 +489 val_489 2010-03-23 12 +489 val_489 2010-03-23 12 +489 val_489 2010-03-23 12 +490 val_490 2010-03-23 12 +490 val_490 2010-03-23 12 +491 val_491 2010-03-23 12 +491 val_491 2010-03-23 12 +492 val_492 2010-03-23 12 +492 val_492 2010-03-23 12 +492 val_492 2010-03-23 12 +492 val_492 2010-03-23 12 +493 val_493 2010-03-23 12 +493 val_493 2010-03-23 12 +494 val_494 2010-03-23 12 +494 val_494 2010-03-23 12 +495 val_495 2010-03-23 12 +495 val_495 2010-03-23 12 +496 val_496 2010-03-23 12 +496 val_496 2010-03-23 12 +497 val_497 2010-03-23 12 +497 val_497 2010-03-23 12 +498 val_498 2010-03-23 12 +498 val_498 2010-03-23 12 +498 val_498 2010-03-23 12 +498 val_498 2010-03-23 12 +498 val_498 2010-03-23 12 +498 val_498 2010-03-23 12 +5 val_5 2010-03-23 12 +5 val_5 2010-03-23 12 +5 val_5 2010-03-23 12 +5 val_5 2010-03-23 12 +5 val_5 2010-03-23 12 +5 val_5 2010-03-23 12 +51 val_51 2010-03-23 12 +51 val_51 2010-03-23 12 +51 val_51 2010-03-23 12 +51 val_51 2010-03-23 12 +53 val_53 2010-03-23 12 +53 val_53 2010-03-23 12 +54 val_54 2010-03-23 12 +54 val_54 2010-03-23 12 +57 val_57 2010-03-23 12 +57 val_57 2010-03-23 12 +58 val_58 2010-03-23 12 +58 val_58 2010-03-23 12 +58 val_58 2010-03-23 12 +58 val_58 2010-03-23 12 +64 val_64 2010-03-23 12 +64 val_64 2010-03-23 12 +65 val_65 2010-03-23 12 +65 val_65 2010-03-23 12 +66 val_66 2010-03-23 12 +66 val_66 2010-03-23 12 +67 val_67 2010-03-23 12 +67 val_67 2010-03-23 12 +67 val_67 2010-03-23 12 +67 val_67 2010-03-23 12 +69 val_69 2010-03-23 12 +69 val_69 2010-03-23 12 +70 val_70 2010-03-23 12 +70 val_70 2010-03-23 12 +70 val_70 2010-03-23 12 +70 val_70 2010-03-23 12 +70 val_70 2010-03-23 12 +70 val_70 2010-03-23 12 +72 val_72 2010-03-23 12 +72 val_72 2010-03-23 12 +72 val_72 2010-03-23 12 +72 val_72 2010-03-23 12 +74 val_74 2010-03-23 12 +74 val_74 2010-03-23 12 +76 val_76 2010-03-23 12 +76 val_76 2010-03-23 12 +76 val_76 2010-03-23 12 +76 val_76 2010-03-23 12 +77 val_77 2010-03-23 12 +77 val_77 2010-03-23 12 +78 val_78 2010-03-23 12 +78 val_78 2010-03-23 12 +8 val_8 2010-03-23 12 +8 val_8 2010-03-23 12 +80 val_80 2010-03-23 12 +80 val_80 2010-03-23 12 +82 val_82 2010-03-23 12 +82 val_82 2010-03-23 12 +83 val_83 2010-03-23 12 +83 val_83 2010-03-23 12 +83 val_83 2010-03-23 12 +83 val_83 2010-03-23 12 +84 val_84 2010-03-23 12 +84 val_84 2010-03-23 12 +84 val_84 2010-03-23 12 +84 val_84 2010-03-23 12 +85 val_85 2010-03-23 12 +85 val_85 2010-03-23 12 +86 val_86 2010-03-23 12 +86 val_86 2010-03-23 12 +87 val_87 2010-03-23 12 +87 val_87 2010-03-23 12 +9 val_9 2010-03-23 12 +9 val_9 2010-03-23 12 +90 val_90 2010-03-23 12 +90 val_90 2010-03-23 12 +90 val_90 2010-03-23 12 +90 val_90 2010-03-23 12 +90 val_90 2010-03-23 12 +90 val_90 2010-03-23 12 +92 val_92 2010-03-23 12 +92 val_92 2010-03-23 12 +95 val_95 2010-03-23 12 +95 val_95 2010-03-23 12 +95 val_95 2010-03-23 12 +95 val_95 2010-03-23 12 +96 val_96 2010-03-23 12 +96 val_96 2010-03-23 12 +97 val_97 2010-03-23 12 +97 val_97 2010-03-23 12 +97 val_97 2010-03-23 12 +97 val_97 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 +98 val_98 2010-03-23 12 \ No newline at end of file diff --git a/src/test/hive/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f b/src/test/hive/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 b/src/test/hive/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 new file mode 100644 index 0000000000000..d35fbec80c19e --- /dev/null +++ b/src/test/hive/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 @@ -0,0 +1,12 @@ +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 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) diff --git a/src/test/hive/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 b/src/test/hive/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 b/src/test/hive/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 b/src/test/hive/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 new file mode 100644 index 0000000000000..2857cdf0aba86 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 @@ -0,0 +1,2000 @@ +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/src/test/hive/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 b/src/test/hive/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 b/src/test/hive/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 new file mode 100644 index 0000000000000..3bbd322e374ff --- /dev/null +++ b/src/test/hive/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 @@ -0,0 +1,1500 @@ +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/src/test/hive/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 b/src/test/hive/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 new file mode 100644 index 0000000000000..f1801743dd4e1 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 @@ -0,0 +1,2500 @@ +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/src/test/hive/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd b/src/test/hive/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd new file mode 100644 index 0000000000000..8017948fc5f2f --- /dev/null +++ b/src/test/hive/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd @@ -0,0 +1,12 @@ +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 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) diff --git a/src/test/hive/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 b/src/test/hive/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 b/src/test/hive/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 b/src/test/hive/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 b/src/test/hive/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 new file mode 100644 index 0000000000000..4cf846d3d3559 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 @@ -0,0 +1,5 @@ +ds=2008-04-08/hr=11 +ds=2008-04-08/hr=12 +ds=2008-04-08/hr=existing_value +ds=2008-04-09/hr=11 +ds=2008-04-09/hr=12 diff --git a/src/test/hive/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 b/src/test/hive/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b b/src/test/hive/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b new file mode 100644 index 0000000000000..3f5f6b6f83715 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b @@ -0,0 +1,9 @@ +key string None +value string None + +# Partition Information +# col_name data_type comment + +value string None + +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) diff --git a/src/test/hive/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 b/src/test/hive/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 new file mode 100644 index 0000000000000..2dcdfd1217ced --- /dev/null +++ b/src/test/hive/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 @@ -0,0 +1,3 @@ +0 val_0 +0 val_0 +0 val_0 diff --git a/src/test/hive/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d b/src/test/hive/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d new file mode 100644 index 0000000000000..dcd1d8643e3cb --- /dev/null +++ b/src/test/hive/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d @@ -0,0 +1 @@ +2 val_2 diff --git a/src/test/hive/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 b/src/test/hive/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 b/src/test/hive/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df b/src/test/hive/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 b/src/test/hive/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 b/src/test/hive/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 new file mode 100644 index 0000000000000..414ce6d5a4941 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 @@ -0,0 +1,309 @@ +value=val_0 +value=val_10 +value=val_100 +value=val_103 +value=val_104 +value=val_105 +value=val_11 +value=val_111 +value=val_113 +value=val_114 +value=val_116 +value=val_118 +value=val_119 +value=val_12 +value=val_120 +value=val_125 +value=val_126 +value=val_128 +value=val_129 +value=val_131 +value=val_133 +value=val_134 +value=val_136 +value=val_137 +value=val_138 +value=val_143 +value=val_145 +value=val_146 +value=val_149 +value=val_15 +value=val_150 +value=val_152 +value=val_153 +value=val_155 +value=val_156 +value=val_157 +value=val_158 +value=val_160 +value=val_162 +value=val_163 +value=val_164 +value=val_165 +value=val_166 +value=val_167 +value=val_168 +value=val_169 +value=val_17 +value=val_170 +value=val_172 +value=val_174 +value=val_175 +value=val_176 +value=val_177 +value=val_178 +value=val_179 +value=val_18 +value=val_180 +value=val_181 +value=val_183 +value=val_186 +value=val_187 +value=val_189 +value=val_19 +value=val_190 +value=val_191 +value=val_192 +value=val_193 +value=val_194 +value=val_195 +value=val_196 +value=val_197 +value=val_199 +value=val_2 +value=val_20 +value=val_200 +value=val_201 +value=val_202 +value=val_203 +value=val_205 +value=val_207 +value=val_208 +value=val_209 +value=val_213 +value=val_214 +value=val_216 +value=val_217 +value=val_218 +value=val_219 +value=val_221 +value=val_222 +value=val_223 +value=val_224 +value=val_226 +value=val_228 +value=val_229 +value=val_230 +value=val_233 +value=val_235 +value=val_237 +value=val_238 +value=val_239 +value=val_24 +value=val_241 +value=val_242 +value=val_244 +value=val_247 +value=val_248 +value=val_249 +value=val_252 +value=val_255 +value=val_256 +value=val_257 +value=val_258 +value=val_26 +value=val_260 +value=val_262 +value=val_263 +value=val_265 +value=val_266 +value=val_27 +value=val_272 +value=val_273 +value=val_274 +value=val_275 +value=val_277 +value=val_278 +value=val_28 +value=val_280 +value=val_281 +value=val_282 +value=val_283 +value=val_284 +value=val_285 +value=val_286 +value=val_287 +value=val_288 +value=val_289 +value=val_291 +value=val_292 +value=val_296 +value=val_298 +value=val_30 +value=val_302 +value=val_305 +value=val_306 +value=val_307 +value=val_308 +value=val_309 +value=val_310 +value=val_311 +value=val_315 +value=val_316 +value=val_317 +value=val_318 +value=val_321 +value=val_322 +value=val_323 +value=val_325 +value=val_327 +value=val_33 +value=val_331 +value=val_332 +value=val_333 +value=val_335 +value=val_336 +value=val_338 +value=val_339 +value=val_34 +value=val_341 +value=val_342 +value=val_344 +value=val_345 +value=val_348 +value=val_35 +value=val_351 +value=val_353 +value=val_356 +value=val_360 +value=val_362 +value=val_364 +value=val_365 +value=val_366 +value=val_367 +value=val_368 +value=val_369 +value=val_37 +value=val_373 +value=val_374 +value=val_375 +value=val_377 +value=val_378 +value=val_379 +value=val_382 +value=val_384 +value=val_386 +value=val_389 +value=val_392 +value=val_393 +value=val_394 +value=val_395 +value=val_396 +value=val_397 +value=val_399 +value=val_4 +value=val_400 +value=val_401 +value=val_402 +value=val_403 +value=val_404 +value=val_406 +value=val_407 +value=val_409 +value=val_41 +value=val_411 +value=val_413 +value=val_414 +value=val_417 +value=val_418 +value=val_419 +value=val_42 +value=val_421 +value=val_424 +value=val_427 +value=val_429 +value=val_43 +value=val_430 +value=val_431 +value=val_432 +value=val_435 +value=val_436 +value=val_437 +value=val_438 +value=val_439 +value=val_44 +value=val_443 +value=val_444 +value=val_446 +value=val_448 +value=val_449 +value=val_452 +value=val_453 +value=val_454 +value=val_455 +value=val_457 +value=val_458 +value=val_459 +value=val_460 +value=val_462 +value=val_463 +value=val_466 +value=val_467 +value=val_468 +value=val_469 +value=val_47 +value=val_470 +value=val_472 +value=val_475 +value=val_477 +value=val_478 +value=val_479 +value=val_480 +value=val_481 +value=val_482 +value=val_483 +value=val_484 +value=val_485 +value=val_487 +value=val_489 +value=val_490 +value=val_491 +value=val_492 +value=val_493 +value=val_494 +value=val_495 +value=val_496 +value=val_497 +value=val_498 +value=val_5 +value=val_51 +value=val_53 +value=val_54 +value=val_57 +value=val_58 +value=val_64 +value=val_65 +value=val_66 +value=val_67 +value=val_69 +value=val_70 +value=val_72 +value=val_74 +value=val_76 +value=val_77 +value=val_78 +value=val_8 +value=val_80 +value=val_82 +value=val_83 +value=val_84 +value=val_85 +value=val_86 +value=val_87 +value=val_9 +value=val_90 +value=val_92 +value=val_95 +value=val_96 +value=val_97 +value=val_98 diff --git a/src/test/hive/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 b/src/test/hive/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d b/src/test/hive/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d new file mode 100644 index 0000000000000..4b9e5b31f4401 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d @@ -0,0 +1,12 @@ +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 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) diff --git a/src/test/hive/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 b/src/test/hive/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 b/src/test/hive/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 new file mode 100644 index 0000000000000..7df5f90186db3 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 @@ -0,0 +1,1000 @@ +238 val_238 2010-03-03 11 +86 val_86 2010-03-03 11 +311 val_311 2010-03-03 11 +27 val_27 2010-03-03 11 +165 val_165 2010-03-03 11 +409 val_409 2010-03-03 11 +255 val_255 2010-03-03 11 +278 val_278 2010-03-03 11 +98 val_98 2010-03-03 11 +484 val_484 2010-03-03 11 +265 val_265 2010-03-03 11 +193 val_193 2010-03-03 11 +401 val_401 2010-03-03 11 +150 val_150 2010-03-03 11 +273 val_273 2010-03-03 11 +224 val_224 2010-03-03 11 +369 val_369 2010-03-03 11 +66 val_66 2010-03-03 11 +128 val_128 2010-03-03 11 +213 val_213 2010-03-03 11 +146 val_146 2010-03-03 11 +406 val_406 2010-03-03 11 +429 val_429 2010-03-03 11 +374 val_374 2010-03-03 11 +152 val_152 2010-03-03 11 +469 val_469 2010-03-03 11 +145 val_145 2010-03-03 11 +495 val_495 2010-03-03 11 +37 val_37 2010-03-03 11 +327 val_327 2010-03-03 11 +281 val_281 2010-03-03 11 +277 val_277 2010-03-03 11 +209 val_209 2010-03-03 11 +15 val_15 2010-03-03 11 +82 val_82 2010-03-03 11 +403 val_403 2010-03-03 11 +166 val_166 2010-03-03 11 +417 val_417 2010-03-03 11 +430 val_430 2010-03-03 11 +252 val_252 2010-03-03 11 +292 val_292 2010-03-03 11 +219 val_219 2010-03-03 11 +287 val_287 2010-03-03 11 +153 val_153 2010-03-03 11 +193 val_193 2010-03-03 11 +338 val_338 2010-03-03 11 +446 val_446 2010-03-03 11 +459 val_459 2010-03-03 11 +394 val_394 2010-03-03 11 +237 val_237 2010-03-03 11 +482 val_482 2010-03-03 11 +174 val_174 2010-03-03 11 +413 val_413 2010-03-03 11 +494 val_494 2010-03-03 11 +207 val_207 2010-03-03 11 +199 val_199 2010-03-03 11 +466 val_466 2010-03-03 11 +208 val_208 2010-03-03 11 +174 val_174 2010-03-03 11 +399 val_399 2010-03-03 11 +396 val_396 2010-03-03 11 +247 val_247 2010-03-03 11 +417 val_417 2010-03-03 11 +489 val_489 2010-03-03 11 +162 val_162 2010-03-03 11 +377 val_377 2010-03-03 11 +397 val_397 2010-03-03 11 +309 val_309 2010-03-03 11 +365 val_365 2010-03-03 11 +266 val_266 2010-03-03 11 +439 val_439 2010-03-03 11 +342 val_342 2010-03-03 11 +367 val_367 2010-03-03 11 +325 val_325 2010-03-03 11 +167 val_167 2010-03-03 11 +195 val_195 2010-03-03 11 +475 val_475 2010-03-03 11 +17 val_17 2010-03-03 11 +113 val_113 2010-03-03 11 +155 val_155 2010-03-03 11 +203 val_203 2010-03-03 11 +339 val_339 2010-03-03 11 +0 val_0 2010-03-03 11 +455 val_455 2010-03-03 11 +128 val_128 2010-03-03 11 +311 val_311 2010-03-03 11 +316 val_316 2010-03-03 11 +57 val_57 2010-03-03 11 +302 val_302 2010-03-03 11 +205 val_205 2010-03-03 11 +149 val_149 2010-03-03 11 +438 val_438 2010-03-03 11 +345 val_345 2010-03-03 11 +129 val_129 2010-03-03 11 +170 val_170 2010-03-03 11 +20 val_20 2010-03-03 11 +489 val_489 2010-03-03 11 +157 val_157 2010-03-03 11 +378 val_378 2010-03-03 11 +221 val_221 2010-03-03 11 +92 val_92 2010-03-03 11 +111 val_111 2010-03-03 11 +47 val_47 2010-03-03 11 +72 val_72 2010-03-03 11 +4 val_4 2010-03-03 11 +280 val_280 2010-03-03 11 +35 val_35 2010-03-03 11 +427 val_427 2010-03-03 11 +277 val_277 2010-03-03 11 +208 val_208 2010-03-03 11 +356 val_356 2010-03-03 11 +399 val_399 2010-03-03 11 +169 val_169 2010-03-03 11 +382 val_382 2010-03-03 11 +498 val_498 2010-03-03 11 +125 val_125 2010-03-03 11 +386 val_386 2010-03-03 11 +437 val_437 2010-03-03 11 +469 val_469 2010-03-03 11 +192 val_192 2010-03-03 11 +286 val_286 2010-03-03 11 +187 val_187 2010-03-03 11 +176 val_176 2010-03-03 11 +54 val_54 2010-03-03 11 +459 val_459 2010-03-03 11 +51 val_51 2010-03-03 11 +138 val_138 2010-03-03 11 +103 val_103 2010-03-03 11 +239 val_239 2010-03-03 11 +213 val_213 2010-03-03 11 +216 val_216 2010-03-03 11 +430 val_430 2010-03-03 11 +278 val_278 2010-03-03 11 +176 val_176 2010-03-03 11 +289 val_289 2010-03-03 11 +221 val_221 2010-03-03 11 +65 val_65 2010-03-03 11 +318 val_318 2010-03-03 11 +332 val_332 2010-03-03 11 +311 val_311 2010-03-03 11 +275 val_275 2010-03-03 11 +137 val_137 2010-03-03 11 +241 val_241 2010-03-03 11 +83 val_83 2010-03-03 11 +333 val_333 2010-03-03 11 +180 val_180 2010-03-03 11 +284 val_284 2010-03-03 11 +12 val_12 2010-03-03 11 +230 val_230 2010-03-03 11 +181 val_181 2010-03-03 11 +67 val_67 2010-03-03 11 +260 val_260 2010-03-03 11 +404 val_404 2010-03-03 11 +384 val_384 2010-03-03 11 +489 val_489 2010-03-03 11 +353 val_353 2010-03-03 11 +373 val_373 2010-03-03 11 +272 val_272 2010-03-03 11 +138 val_138 2010-03-03 11 +217 val_217 2010-03-03 11 +84 val_84 2010-03-03 11 +348 val_348 2010-03-03 11 +466 val_466 2010-03-03 11 +58 val_58 2010-03-03 11 +8 val_8 2010-03-03 11 +411 val_411 2010-03-03 11 +230 val_230 2010-03-03 11 +208 val_208 2010-03-03 11 +348 val_348 2010-03-03 11 +24 val_24 2010-03-03 11 +463 val_463 2010-03-03 11 +431 val_431 2010-03-03 11 +179 val_179 2010-03-03 11 +172 val_172 2010-03-03 11 +42 val_42 2010-03-03 11 +129 val_129 2010-03-03 11 +158 val_158 2010-03-03 11 +119 val_119 2010-03-03 11 +496 val_496 2010-03-03 11 +0 val_0 2010-03-03 11 +322 val_322 2010-03-03 11 +197 val_197 2010-03-03 11 +468 val_468 2010-03-03 11 +393 val_393 2010-03-03 11 +454 val_454 2010-03-03 11 +100 val_100 2010-03-03 11 +298 val_298 2010-03-03 11 +199 val_199 2010-03-03 11 +191 val_191 2010-03-03 11 +418 val_418 2010-03-03 11 +96 val_96 2010-03-03 11 +26 val_26 2010-03-03 11 +165 val_165 2010-03-03 11 +327 val_327 2010-03-03 11 +230 val_230 2010-03-03 11 +205 val_205 2010-03-03 11 +120 val_120 2010-03-03 11 +131 val_131 2010-03-03 11 +51 val_51 2010-03-03 11 +404 val_404 2010-03-03 11 +43 val_43 2010-03-03 11 +436 val_436 2010-03-03 11 +156 val_156 2010-03-03 11 +469 val_469 2010-03-03 11 +468 val_468 2010-03-03 11 +308 val_308 2010-03-03 11 +95 val_95 2010-03-03 11 +196 val_196 2010-03-03 11 +288 val_288 2010-03-03 11 +481 val_481 2010-03-03 11 +457 val_457 2010-03-03 11 +98 val_98 2010-03-03 11 +282 val_282 2010-03-03 11 +197 val_197 2010-03-03 11 +187 val_187 2010-03-03 11 +318 val_318 2010-03-03 11 +318 val_318 2010-03-03 11 +409 val_409 2010-03-03 11 +470 val_470 2010-03-03 11 +137 val_137 2010-03-03 11 +369 val_369 2010-03-03 11 +316 val_316 2010-03-03 11 +169 val_169 2010-03-03 11 +413 val_413 2010-03-03 11 +85 val_85 2010-03-03 11 +77 val_77 2010-03-03 11 +0 val_0 2010-03-03 11 +490 val_490 2010-03-03 11 +87 val_87 2010-03-03 11 +364 val_364 2010-03-03 11 +179 val_179 2010-03-03 11 +118 val_118 2010-03-03 11 +134 val_134 2010-03-03 11 +395 val_395 2010-03-03 11 +282 val_282 2010-03-03 11 +138 val_138 2010-03-03 11 +238 val_238 2010-03-03 11 +419 val_419 2010-03-03 11 +15 val_15 2010-03-03 11 +118 val_118 2010-03-03 11 +72 val_72 2010-03-03 11 +90 val_90 2010-03-03 11 +307 val_307 2010-03-03 11 +19 val_19 2010-03-03 11 +435 val_435 2010-03-03 11 +10 val_10 2010-03-03 11 +277 val_277 2010-03-03 11 +273 val_273 2010-03-03 11 +306 val_306 2010-03-03 11 +224 val_224 2010-03-03 11 +309 val_309 2010-03-03 11 +389 val_389 2010-03-03 11 +327 val_327 2010-03-03 11 +242 val_242 2010-03-03 11 +369 val_369 2010-03-03 11 +392 val_392 2010-03-03 11 +272 val_272 2010-03-03 11 +331 val_331 2010-03-03 11 +401 val_401 2010-03-03 11 +242 val_242 2010-03-03 11 +452 val_452 2010-03-03 11 +177 val_177 2010-03-03 11 +226 val_226 2010-03-03 11 +5 val_5 2010-03-03 11 +497 val_497 2010-03-03 11 +402 val_402 2010-03-03 11 +396 val_396 2010-03-03 11 +317 val_317 2010-03-03 11 +395 val_395 2010-03-03 11 +58 val_58 2010-03-03 11 +35 val_35 2010-03-03 11 +336 val_336 2010-03-03 11 +95 val_95 2010-03-03 11 +11 val_11 2010-03-03 11 +168 val_168 2010-03-03 11 +34 val_34 2010-03-03 11 +229 val_229 2010-03-03 11 +233 val_233 2010-03-03 11 +143 val_143 2010-03-03 11 +472 val_472 2010-03-03 11 +322 val_322 2010-03-03 11 +498 val_498 2010-03-03 11 +160 val_160 2010-03-03 11 +195 val_195 2010-03-03 11 +42 val_42 2010-03-03 11 +321 val_321 2010-03-03 11 +430 val_430 2010-03-03 11 +119 val_119 2010-03-03 11 +489 val_489 2010-03-03 11 +458 val_458 2010-03-03 11 +78 val_78 2010-03-03 11 +76 val_76 2010-03-03 11 +41 val_41 2010-03-03 11 +223 val_223 2010-03-03 11 +492 val_492 2010-03-03 11 +149 val_149 2010-03-03 11 +449 val_449 2010-03-03 11 +218 val_218 2010-03-03 11 +228 val_228 2010-03-03 11 +138 val_138 2010-03-03 11 +453 val_453 2010-03-03 11 +30 val_30 2010-03-03 11 +209 val_209 2010-03-03 11 +64 val_64 2010-03-03 11 +468 val_468 2010-03-03 11 +76 val_76 2010-03-03 11 +74 val_74 2010-03-03 11 +342 val_342 2010-03-03 11 +69 val_69 2010-03-03 11 +230 val_230 2010-03-03 11 +33 val_33 2010-03-03 11 +368 val_368 2010-03-03 11 +103 val_103 2010-03-03 11 +296 val_296 2010-03-03 11 +113 val_113 2010-03-03 11 +216 val_216 2010-03-03 11 +367 val_367 2010-03-03 11 +344 val_344 2010-03-03 11 +167 val_167 2010-03-03 11 +274 val_274 2010-03-03 11 +219 val_219 2010-03-03 11 +239 val_239 2010-03-03 11 +485 val_485 2010-03-03 11 +116 val_116 2010-03-03 11 +223 val_223 2010-03-03 11 +256 val_256 2010-03-03 11 +263 val_263 2010-03-03 11 +70 val_70 2010-03-03 11 +487 val_487 2010-03-03 11 +480 val_480 2010-03-03 11 +401 val_401 2010-03-03 11 +288 val_288 2010-03-03 11 +191 val_191 2010-03-03 11 +5 val_5 2010-03-03 11 +244 val_244 2010-03-03 11 +438 val_438 2010-03-03 11 +128 val_128 2010-03-03 11 +467 val_467 2010-03-03 11 +432 val_432 2010-03-03 11 +202 val_202 2010-03-03 11 +316 val_316 2010-03-03 11 +229 val_229 2010-03-03 11 +469 val_469 2010-03-03 11 +463 val_463 2010-03-03 11 +280 val_280 2010-03-03 11 +2 val_2 2010-03-03 11 +35 val_35 2010-03-03 11 +283 val_283 2010-03-03 11 +331 val_331 2010-03-03 11 +235 val_235 2010-03-03 11 +80 val_80 2010-03-03 11 +44 val_44 2010-03-03 11 +193 val_193 2010-03-03 11 +321 val_321 2010-03-03 11 +335 val_335 2010-03-03 11 +104 val_104 2010-03-03 11 +466 val_466 2010-03-03 11 +366 val_366 2010-03-03 11 +175 val_175 2010-03-03 11 +403 val_403 2010-03-03 11 +483 val_483 2010-03-03 11 +53 val_53 2010-03-03 11 +105 val_105 2010-03-03 11 +257 val_257 2010-03-03 11 +406 val_406 2010-03-03 11 +409 val_409 2010-03-03 11 +190 val_190 2010-03-03 11 +406 val_406 2010-03-03 11 +401 val_401 2010-03-03 11 +114 val_114 2010-03-03 11 +258 val_258 2010-03-03 11 +90 val_90 2010-03-03 11 +203 val_203 2010-03-03 11 +262 val_262 2010-03-03 11 +348 val_348 2010-03-03 11 +424 val_424 2010-03-03 11 +12 val_12 2010-03-03 11 +396 val_396 2010-03-03 11 +201 val_201 2010-03-03 11 +217 val_217 2010-03-03 11 +164 val_164 2010-03-03 11 +431 val_431 2010-03-03 11 +454 val_454 2010-03-03 11 +478 val_478 2010-03-03 11 +298 val_298 2010-03-03 11 +125 val_125 2010-03-03 11 +431 val_431 2010-03-03 11 +164 val_164 2010-03-03 11 +424 val_424 2010-03-03 11 +187 val_187 2010-03-03 11 +382 val_382 2010-03-03 11 +5 val_5 2010-03-03 11 +70 val_70 2010-03-03 11 +397 val_397 2010-03-03 11 +480 val_480 2010-03-03 11 +291 val_291 2010-03-03 11 +24 val_24 2010-03-03 11 +351 val_351 2010-03-03 11 +255 val_255 2010-03-03 11 +104 val_104 2010-03-03 11 +70 val_70 2010-03-03 11 +163 val_163 2010-03-03 11 +438 val_438 2010-03-03 11 +119 val_119 2010-03-03 11 +414 val_414 2010-03-03 11 +200 val_200 2010-03-03 11 +491 val_491 2010-03-03 11 +237 val_237 2010-03-03 11 +439 val_439 2010-03-03 11 +360 val_360 2010-03-03 11 +248 val_248 2010-03-03 11 +479 val_479 2010-03-03 11 +305 val_305 2010-03-03 11 +417 val_417 2010-03-03 11 +199 val_199 2010-03-03 11 +444 val_444 2010-03-03 11 +120 val_120 2010-03-03 11 +429 val_429 2010-03-03 11 +169 val_169 2010-03-03 11 +443 val_443 2010-03-03 11 +323 val_323 2010-03-03 11 +325 val_325 2010-03-03 11 +277 val_277 2010-03-03 11 +230 val_230 2010-03-03 11 +478 val_478 2010-03-03 11 +178 val_178 2010-03-03 11 +468 val_468 2010-03-03 11 +310 val_310 2010-03-03 11 +317 val_317 2010-03-03 11 +333 val_333 2010-03-03 11 +493 val_493 2010-03-03 11 +460 val_460 2010-03-03 11 +207 val_207 2010-03-03 11 +249 val_249 2010-03-03 11 +265 val_265 2010-03-03 11 +480 val_480 2010-03-03 11 +83 val_83 2010-03-03 11 +136 val_136 2010-03-03 11 +353 val_353 2010-03-03 11 +172 val_172 2010-03-03 11 +214 val_214 2010-03-03 11 +462 val_462 2010-03-03 11 +233 val_233 2010-03-03 11 +406 val_406 2010-03-03 11 +133 val_133 2010-03-03 11 +175 val_175 2010-03-03 11 +189 val_189 2010-03-03 11 +454 val_454 2010-03-03 11 +375 val_375 2010-03-03 11 +401 val_401 2010-03-03 11 +421 val_421 2010-03-03 11 +407 val_407 2010-03-03 11 +384 val_384 2010-03-03 11 +256 val_256 2010-03-03 11 +26 val_26 2010-03-03 11 +134 val_134 2010-03-03 11 +67 val_67 2010-03-03 11 +384 val_384 2010-03-03 11 +379 val_379 2010-03-03 11 +18 val_18 2010-03-03 11 +462 val_462 2010-03-03 11 +492 val_492 2010-03-03 11 +100 val_100 2010-03-03 11 +298 val_298 2010-03-03 11 +9 val_9 2010-03-03 11 +341 val_341 2010-03-03 11 +498 val_498 2010-03-03 11 +146 val_146 2010-03-03 11 +458 val_458 2010-03-03 11 +362 val_362 2010-03-03 11 +186 val_186 2010-03-03 11 +285 val_285 2010-03-03 11 +348 val_348 2010-03-03 11 +167 val_167 2010-03-03 11 +18 val_18 2010-03-03 11 +273 val_273 2010-03-03 11 +183 val_183 2010-03-03 11 +281 val_281 2010-03-03 11 +344 val_344 2010-03-03 11 +97 val_97 2010-03-03 11 +469 val_469 2010-03-03 11 +315 val_315 2010-03-03 11 +84 val_84 2010-03-03 11 +28 val_28 2010-03-03 11 +37 val_37 2010-03-03 11 +448 val_448 2010-03-03 11 +152 val_152 2010-03-03 11 +348 val_348 2010-03-03 11 +307 val_307 2010-03-03 11 +194 val_194 2010-03-03 11 +414 val_414 2010-03-03 11 +477 val_477 2010-03-03 11 +222 val_222 2010-03-03 11 +126 val_126 2010-03-03 11 +90 val_90 2010-03-03 11 +169 val_169 2010-03-03 11 +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 +238 val_238 2010-03-03 11 +86 val_86 2010-03-03 11 +311 val_311 2010-03-03 11 +27 val_27 2010-03-03 11 +165 val_165 2010-03-03 11 +409 val_409 2010-03-03 11 +255 val_255 2010-03-03 11 +278 val_278 2010-03-03 11 +98 val_98 2010-03-03 11 +484 val_484 2010-03-03 11 +265 val_265 2010-03-03 11 +193 val_193 2010-03-03 11 +401 val_401 2010-03-03 11 +150 val_150 2010-03-03 11 +273 val_273 2010-03-03 11 +224 val_224 2010-03-03 11 +369 val_369 2010-03-03 11 +66 val_66 2010-03-03 11 +128 val_128 2010-03-03 11 +213 val_213 2010-03-03 11 +146 val_146 2010-03-03 11 +406 val_406 2010-03-03 11 +429 val_429 2010-03-03 11 +374 val_374 2010-03-03 11 +152 val_152 2010-03-03 11 +469 val_469 2010-03-03 11 +145 val_145 2010-03-03 11 +495 val_495 2010-03-03 11 +37 val_37 2010-03-03 11 +327 val_327 2010-03-03 11 +281 val_281 2010-03-03 11 +277 val_277 2010-03-03 11 +209 val_209 2010-03-03 11 +15 val_15 2010-03-03 11 +82 val_82 2010-03-03 11 +403 val_403 2010-03-03 11 +166 val_166 2010-03-03 11 +417 val_417 2010-03-03 11 +430 val_430 2010-03-03 11 +252 val_252 2010-03-03 11 +292 val_292 2010-03-03 11 +219 val_219 2010-03-03 11 +287 val_287 2010-03-03 11 +153 val_153 2010-03-03 11 +193 val_193 2010-03-03 11 +338 val_338 2010-03-03 11 +446 val_446 2010-03-03 11 +459 val_459 2010-03-03 11 +394 val_394 2010-03-03 11 +237 val_237 2010-03-03 11 +482 val_482 2010-03-03 11 +174 val_174 2010-03-03 11 +413 val_413 2010-03-03 11 +494 val_494 2010-03-03 11 +207 val_207 2010-03-03 11 +199 val_199 2010-03-03 11 +466 val_466 2010-03-03 11 +208 val_208 2010-03-03 11 +174 val_174 2010-03-03 11 +399 val_399 2010-03-03 11 +396 val_396 2010-03-03 11 +247 val_247 2010-03-03 11 +417 val_417 2010-03-03 11 +489 val_489 2010-03-03 11 +162 val_162 2010-03-03 11 +377 val_377 2010-03-03 11 +397 val_397 2010-03-03 11 +309 val_309 2010-03-03 11 +365 val_365 2010-03-03 11 +266 val_266 2010-03-03 11 +439 val_439 2010-03-03 11 +342 val_342 2010-03-03 11 +367 val_367 2010-03-03 11 +325 val_325 2010-03-03 11 +167 val_167 2010-03-03 11 +195 val_195 2010-03-03 11 +475 val_475 2010-03-03 11 +17 val_17 2010-03-03 11 +113 val_113 2010-03-03 11 +155 val_155 2010-03-03 11 +203 val_203 2010-03-03 11 +339 val_339 2010-03-03 11 +0 val_0 2010-03-03 11 +455 val_455 2010-03-03 11 +128 val_128 2010-03-03 11 +311 val_311 2010-03-03 11 +316 val_316 2010-03-03 11 +57 val_57 2010-03-03 11 +302 val_302 2010-03-03 11 +205 val_205 2010-03-03 11 +149 val_149 2010-03-03 11 +438 val_438 2010-03-03 11 +345 val_345 2010-03-03 11 +129 val_129 2010-03-03 11 +170 val_170 2010-03-03 11 +20 val_20 2010-03-03 11 +489 val_489 2010-03-03 11 +157 val_157 2010-03-03 11 +378 val_378 2010-03-03 11 +221 val_221 2010-03-03 11 +92 val_92 2010-03-03 11 +111 val_111 2010-03-03 11 +47 val_47 2010-03-03 11 +72 val_72 2010-03-03 11 +4 val_4 2010-03-03 11 +280 val_280 2010-03-03 11 +35 val_35 2010-03-03 11 +427 val_427 2010-03-03 11 +277 val_277 2010-03-03 11 +208 val_208 2010-03-03 11 +356 val_356 2010-03-03 11 +399 val_399 2010-03-03 11 +169 val_169 2010-03-03 11 +382 val_382 2010-03-03 11 +498 val_498 2010-03-03 11 +125 val_125 2010-03-03 11 +386 val_386 2010-03-03 11 +437 val_437 2010-03-03 11 +469 val_469 2010-03-03 11 +192 val_192 2010-03-03 11 +286 val_286 2010-03-03 11 +187 val_187 2010-03-03 11 +176 val_176 2010-03-03 11 +54 val_54 2010-03-03 11 +459 val_459 2010-03-03 11 +51 val_51 2010-03-03 11 +138 val_138 2010-03-03 11 +103 val_103 2010-03-03 11 +239 val_239 2010-03-03 11 +213 val_213 2010-03-03 11 +216 val_216 2010-03-03 11 +430 val_430 2010-03-03 11 +278 val_278 2010-03-03 11 +176 val_176 2010-03-03 11 +289 val_289 2010-03-03 11 +221 val_221 2010-03-03 11 +65 val_65 2010-03-03 11 +318 val_318 2010-03-03 11 +332 val_332 2010-03-03 11 +311 val_311 2010-03-03 11 +275 val_275 2010-03-03 11 +137 val_137 2010-03-03 11 +241 val_241 2010-03-03 11 +83 val_83 2010-03-03 11 +333 val_333 2010-03-03 11 +180 val_180 2010-03-03 11 +284 val_284 2010-03-03 11 +12 val_12 2010-03-03 11 +230 val_230 2010-03-03 11 +181 val_181 2010-03-03 11 +67 val_67 2010-03-03 11 +260 val_260 2010-03-03 11 +404 val_404 2010-03-03 11 +384 val_384 2010-03-03 11 +489 val_489 2010-03-03 11 +353 val_353 2010-03-03 11 +373 val_373 2010-03-03 11 +272 val_272 2010-03-03 11 +138 val_138 2010-03-03 11 +217 val_217 2010-03-03 11 +84 val_84 2010-03-03 11 +348 val_348 2010-03-03 11 +466 val_466 2010-03-03 11 +58 val_58 2010-03-03 11 +8 val_8 2010-03-03 11 +411 val_411 2010-03-03 11 +230 val_230 2010-03-03 11 +208 val_208 2010-03-03 11 +348 val_348 2010-03-03 11 +24 val_24 2010-03-03 11 +463 val_463 2010-03-03 11 +431 val_431 2010-03-03 11 +179 val_179 2010-03-03 11 +172 val_172 2010-03-03 11 +42 val_42 2010-03-03 11 +129 val_129 2010-03-03 11 +158 val_158 2010-03-03 11 +119 val_119 2010-03-03 11 +496 val_496 2010-03-03 11 +0 val_0 2010-03-03 11 +322 val_322 2010-03-03 11 +197 val_197 2010-03-03 11 +468 val_468 2010-03-03 11 +393 val_393 2010-03-03 11 +454 val_454 2010-03-03 11 +100 val_100 2010-03-03 11 +298 val_298 2010-03-03 11 +199 val_199 2010-03-03 11 +191 val_191 2010-03-03 11 +418 val_418 2010-03-03 11 +96 val_96 2010-03-03 11 +26 val_26 2010-03-03 11 +165 val_165 2010-03-03 11 +327 val_327 2010-03-03 11 +230 val_230 2010-03-03 11 +205 val_205 2010-03-03 11 +120 val_120 2010-03-03 11 +131 val_131 2010-03-03 11 +51 val_51 2010-03-03 11 +404 val_404 2010-03-03 11 +43 val_43 2010-03-03 11 +436 val_436 2010-03-03 11 +156 val_156 2010-03-03 11 +469 val_469 2010-03-03 11 +468 val_468 2010-03-03 11 +308 val_308 2010-03-03 11 +95 val_95 2010-03-03 11 +196 val_196 2010-03-03 11 +288 val_288 2010-03-03 11 +481 val_481 2010-03-03 11 +457 val_457 2010-03-03 11 +98 val_98 2010-03-03 11 +282 val_282 2010-03-03 11 +197 val_197 2010-03-03 11 +187 val_187 2010-03-03 11 +318 val_318 2010-03-03 11 +318 val_318 2010-03-03 11 +409 val_409 2010-03-03 11 +470 val_470 2010-03-03 11 +137 val_137 2010-03-03 11 +369 val_369 2010-03-03 11 +316 val_316 2010-03-03 11 +169 val_169 2010-03-03 11 +413 val_413 2010-03-03 11 +85 val_85 2010-03-03 11 +77 val_77 2010-03-03 11 +0 val_0 2010-03-03 11 +490 val_490 2010-03-03 11 +87 val_87 2010-03-03 11 +364 val_364 2010-03-03 11 +179 val_179 2010-03-03 11 +118 val_118 2010-03-03 11 +134 val_134 2010-03-03 11 +395 val_395 2010-03-03 11 +282 val_282 2010-03-03 11 +138 val_138 2010-03-03 11 +238 val_238 2010-03-03 11 +419 val_419 2010-03-03 11 +15 val_15 2010-03-03 11 +118 val_118 2010-03-03 11 +72 val_72 2010-03-03 11 +90 val_90 2010-03-03 11 +307 val_307 2010-03-03 11 +19 val_19 2010-03-03 11 +435 val_435 2010-03-03 11 +10 val_10 2010-03-03 11 +277 val_277 2010-03-03 11 +273 val_273 2010-03-03 11 +306 val_306 2010-03-03 11 +224 val_224 2010-03-03 11 +309 val_309 2010-03-03 11 +389 val_389 2010-03-03 11 +327 val_327 2010-03-03 11 +242 val_242 2010-03-03 11 +369 val_369 2010-03-03 11 +392 val_392 2010-03-03 11 +272 val_272 2010-03-03 11 +331 val_331 2010-03-03 11 +401 val_401 2010-03-03 11 +242 val_242 2010-03-03 11 +452 val_452 2010-03-03 11 +177 val_177 2010-03-03 11 +226 val_226 2010-03-03 11 +5 val_5 2010-03-03 11 +497 val_497 2010-03-03 11 +402 val_402 2010-03-03 11 +396 val_396 2010-03-03 11 +317 val_317 2010-03-03 11 +395 val_395 2010-03-03 11 +58 val_58 2010-03-03 11 +35 val_35 2010-03-03 11 +336 val_336 2010-03-03 11 +95 val_95 2010-03-03 11 +11 val_11 2010-03-03 11 +168 val_168 2010-03-03 11 +34 val_34 2010-03-03 11 +229 val_229 2010-03-03 11 +233 val_233 2010-03-03 11 +143 val_143 2010-03-03 11 +472 val_472 2010-03-03 11 +322 val_322 2010-03-03 11 +498 val_498 2010-03-03 11 +160 val_160 2010-03-03 11 +195 val_195 2010-03-03 11 +42 val_42 2010-03-03 11 +321 val_321 2010-03-03 11 +430 val_430 2010-03-03 11 +119 val_119 2010-03-03 11 +489 val_489 2010-03-03 11 +458 val_458 2010-03-03 11 +78 val_78 2010-03-03 11 +76 val_76 2010-03-03 11 +41 val_41 2010-03-03 11 +223 val_223 2010-03-03 11 +492 val_492 2010-03-03 11 +149 val_149 2010-03-03 11 +449 val_449 2010-03-03 11 +218 val_218 2010-03-03 11 +228 val_228 2010-03-03 11 +138 val_138 2010-03-03 11 +453 val_453 2010-03-03 11 +30 val_30 2010-03-03 11 +209 val_209 2010-03-03 11 +64 val_64 2010-03-03 11 +468 val_468 2010-03-03 11 +76 val_76 2010-03-03 11 +74 val_74 2010-03-03 11 +342 val_342 2010-03-03 11 +69 val_69 2010-03-03 11 +230 val_230 2010-03-03 11 +33 val_33 2010-03-03 11 +368 val_368 2010-03-03 11 +103 val_103 2010-03-03 11 +296 val_296 2010-03-03 11 +113 val_113 2010-03-03 11 +216 val_216 2010-03-03 11 +367 val_367 2010-03-03 11 +344 val_344 2010-03-03 11 +167 val_167 2010-03-03 11 +274 val_274 2010-03-03 11 +219 val_219 2010-03-03 11 +239 val_239 2010-03-03 11 +485 val_485 2010-03-03 11 +116 val_116 2010-03-03 11 +223 val_223 2010-03-03 11 +256 val_256 2010-03-03 11 +263 val_263 2010-03-03 11 +70 val_70 2010-03-03 11 +487 val_487 2010-03-03 11 +480 val_480 2010-03-03 11 +401 val_401 2010-03-03 11 +288 val_288 2010-03-03 11 +191 val_191 2010-03-03 11 +5 val_5 2010-03-03 11 +244 val_244 2010-03-03 11 +438 val_438 2010-03-03 11 +128 val_128 2010-03-03 11 +467 val_467 2010-03-03 11 +432 val_432 2010-03-03 11 +202 val_202 2010-03-03 11 +316 val_316 2010-03-03 11 +229 val_229 2010-03-03 11 +469 val_469 2010-03-03 11 +463 val_463 2010-03-03 11 +280 val_280 2010-03-03 11 +2 val_2 2010-03-03 11 +35 val_35 2010-03-03 11 +283 val_283 2010-03-03 11 +331 val_331 2010-03-03 11 +235 val_235 2010-03-03 11 +80 val_80 2010-03-03 11 +44 val_44 2010-03-03 11 +193 val_193 2010-03-03 11 +321 val_321 2010-03-03 11 +335 val_335 2010-03-03 11 +104 val_104 2010-03-03 11 +466 val_466 2010-03-03 11 +366 val_366 2010-03-03 11 +175 val_175 2010-03-03 11 +403 val_403 2010-03-03 11 +483 val_483 2010-03-03 11 +53 val_53 2010-03-03 11 +105 val_105 2010-03-03 11 +257 val_257 2010-03-03 11 +406 val_406 2010-03-03 11 +409 val_409 2010-03-03 11 +190 val_190 2010-03-03 11 +406 val_406 2010-03-03 11 +401 val_401 2010-03-03 11 +114 val_114 2010-03-03 11 +258 val_258 2010-03-03 11 +90 val_90 2010-03-03 11 +203 val_203 2010-03-03 11 +262 val_262 2010-03-03 11 +348 val_348 2010-03-03 11 +424 val_424 2010-03-03 11 +12 val_12 2010-03-03 11 +396 val_396 2010-03-03 11 +201 val_201 2010-03-03 11 +217 val_217 2010-03-03 11 +164 val_164 2010-03-03 11 +431 val_431 2010-03-03 11 +454 val_454 2010-03-03 11 +478 val_478 2010-03-03 11 +298 val_298 2010-03-03 11 +125 val_125 2010-03-03 11 +431 val_431 2010-03-03 11 +164 val_164 2010-03-03 11 +424 val_424 2010-03-03 11 +187 val_187 2010-03-03 11 +382 val_382 2010-03-03 11 +5 val_5 2010-03-03 11 +70 val_70 2010-03-03 11 +397 val_397 2010-03-03 11 +480 val_480 2010-03-03 11 +291 val_291 2010-03-03 11 +24 val_24 2010-03-03 11 +351 val_351 2010-03-03 11 +255 val_255 2010-03-03 11 +104 val_104 2010-03-03 11 +70 val_70 2010-03-03 11 +163 val_163 2010-03-03 11 +438 val_438 2010-03-03 11 +119 val_119 2010-03-03 11 +414 val_414 2010-03-03 11 +200 val_200 2010-03-03 11 +491 val_491 2010-03-03 11 +237 val_237 2010-03-03 11 +439 val_439 2010-03-03 11 +360 val_360 2010-03-03 11 +248 val_248 2010-03-03 11 +479 val_479 2010-03-03 11 +305 val_305 2010-03-03 11 +417 val_417 2010-03-03 11 +199 val_199 2010-03-03 11 +444 val_444 2010-03-03 11 +120 val_120 2010-03-03 11 +429 val_429 2010-03-03 11 +169 val_169 2010-03-03 11 +443 val_443 2010-03-03 11 +323 val_323 2010-03-03 11 +325 val_325 2010-03-03 11 +277 val_277 2010-03-03 11 +230 val_230 2010-03-03 11 +478 val_478 2010-03-03 11 +178 val_178 2010-03-03 11 +468 val_468 2010-03-03 11 +310 val_310 2010-03-03 11 +317 val_317 2010-03-03 11 +333 val_333 2010-03-03 11 +493 val_493 2010-03-03 11 +460 val_460 2010-03-03 11 +207 val_207 2010-03-03 11 +249 val_249 2010-03-03 11 +265 val_265 2010-03-03 11 +480 val_480 2010-03-03 11 +83 val_83 2010-03-03 11 +136 val_136 2010-03-03 11 +353 val_353 2010-03-03 11 +172 val_172 2010-03-03 11 +214 val_214 2010-03-03 11 +462 val_462 2010-03-03 11 +233 val_233 2010-03-03 11 +406 val_406 2010-03-03 11 +133 val_133 2010-03-03 11 +175 val_175 2010-03-03 11 +189 val_189 2010-03-03 11 +454 val_454 2010-03-03 11 +375 val_375 2010-03-03 11 +401 val_401 2010-03-03 11 +421 val_421 2010-03-03 11 +407 val_407 2010-03-03 11 +384 val_384 2010-03-03 11 +256 val_256 2010-03-03 11 +26 val_26 2010-03-03 11 +134 val_134 2010-03-03 11 +67 val_67 2010-03-03 11 +384 val_384 2010-03-03 11 +379 val_379 2010-03-03 11 +18 val_18 2010-03-03 11 +462 val_462 2010-03-03 11 +492 val_492 2010-03-03 11 +100 val_100 2010-03-03 11 +298 val_298 2010-03-03 11 +9 val_9 2010-03-03 11 +341 val_341 2010-03-03 11 +498 val_498 2010-03-03 11 +146 val_146 2010-03-03 11 +458 val_458 2010-03-03 11 +362 val_362 2010-03-03 11 +186 val_186 2010-03-03 11 +285 val_285 2010-03-03 11 +348 val_348 2010-03-03 11 +167 val_167 2010-03-03 11 +18 val_18 2010-03-03 11 +273 val_273 2010-03-03 11 +183 val_183 2010-03-03 11 +281 val_281 2010-03-03 11 +344 val_344 2010-03-03 11 +97 val_97 2010-03-03 11 +469 val_469 2010-03-03 11 +315 val_315 2010-03-03 11 +84 val_84 2010-03-03 11 +28 val_28 2010-03-03 11 +37 val_37 2010-03-03 11 +448 val_448 2010-03-03 11 +152 val_152 2010-03-03 11 +348 val_348 2010-03-03 11 +307 val_307 2010-03-03 11 +194 val_194 2010-03-03 11 +414 val_414 2010-03-03 11 +477 val_477 2010-03-03 11 +222 val_222 2010-03-03 11 +126 val_126 2010-03-03 11 +90 val_90 2010-03-03 11 +169 val_169 2010-03-03 11 +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 diff --git a/src/test/hive/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f b/src/test/hive/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f new file mode 100644 index 0000000000000..c869646753baa --- /dev/null +++ b/src/test/hive/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f @@ -0,0 +1,1000 @@ +238 val_238 2010-03-03 12 +86 val_86 2010-03-03 12 +311 val_311 2010-03-03 12 +27 val_27 2010-03-03 12 +165 val_165 2010-03-03 12 +409 val_409 2010-03-03 12 +255 val_255 2010-03-03 12 +278 val_278 2010-03-03 12 +98 val_98 2010-03-03 12 +484 val_484 2010-03-03 12 +265 val_265 2010-03-03 12 +193 val_193 2010-03-03 12 +401 val_401 2010-03-03 12 +150 val_150 2010-03-03 12 +273 val_273 2010-03-03 12 +224 val_224 2010-03-03 12 +369 val_369 2010-03-03 12 +66 val_66 2010-03-03 12 +128 val_128 2010-03-03 12 +213 val_213 2010-03-03 12 +146 val_146 2010-03-03 12 +406 val_406 2010-03-03 12 +429 val_429 2010-03-03 12 +374 val_374 2010-03-03 12 +152 val_152 2010-03-03 12 +469 val_469 2010-03-03 12 +145 val_145 2010-03-03 12 +495 val_495 2010-03-03 12 +37 val_37 2010-03-03 12 +327 val_327 2010-03-03 12 +281 val_281 2010-03-03 12 +277 val_277 2010-03-03 12 +209 val_209 2010-03-03 12 +15 val_15 2010-03-03 12 +82 val_82 2010-03-03 12 +403 val_403 2010-03-03 12 +166 val_166 2010-03-03 12 +417 val_417 2010-03-03 12 +430 val_430 2010-03-03 12 +252 val_252 2010-03-03 12 +292 val_292 2010-03-03 12 +219 val_219 2010-03-03 12 +287 val_287 2010-03-03 12 +153 val_153 2010-03-03 12 +193 val_193 2010-03-03 12 +338 val_338 2010-03-03 12 +446 val_446 2010-03-03 12 +459 val_459 2010-03-03 12 +394 val_394 2010-03-03 12 +237 val_237 2010-03-03 12 +482 val_482 2010-03-03 12 +174 val_174 2010-03-03 12 +413 val_413 2010-03-03 12 +494 val_494 2010-03-03 12 +207 val_207 2010-03-03 12 +199 val_199 2010-03-03 12 +466 val_466 2010-03-03 12 +208 val_208 2010-03-03 12 +174 val_174 2010-03-03 12 +399 val_399 2010-03-03 12 +396 val_396 2010-03-03 12 +247 val_247 2010-03-03 12 +417 val_417 2010-03-03 12 +489 val_489 2010-03-03 12 +162 val_162 2010-03-03 12 +377 val_377 2010-03-03 12 +397 val_397 2010-03-03 12 +309 val_309 2010-03-03 12 +365 val_365 2010-03-03 12 +266 val_266 2010-03-03 12 +439 val_439 2010-03-03 12 +342 val_342 2010-03-03 12 +367 val_367 2010-03-03 12 +325 val_325 2010-03-03 12 +167 val_167 2010-03-03 12 +195 val_195 2010-03-03 12 +475 val_475 2010-03-03 12 +17 val_17 2010-03-03 12 +113 val_113 2010-03-03 12 +155 val_155 2010-03-03 12 +203 val_203 2010-03-03 12 +339 val_339 2010-03-03 12 +0 val_0 2010-03-03 12 +455 val_455 2010-03-03 12 +128 val_128 2010-03-03 12 +311 val_311 2010-03-03 12 +316 val_316 2010-03-03 12 +57 val_57 2010-03-03 12 +302 val_302 2010-03-03 12 +205 val_205 2010-03-03 12 +149 val_149 2010-03-03 12 +438 val_438 2010-03-03 12 +345 val_345 2010-03-03 12 +129 val_129 2010-03-03 12 +170 val_170 2010-03-03 12 +20 val_20 2010-03-03 12 +489 val_489 2010-03-03 12 +157 val_157 2010-03-03 12 +378 val_378 2010-03-03 12 +221 val_221 2010-03-03 12 +92 val_92 2010-03-03 12 +111 val_111 2010-03-03 12 +47 val_47 2010-03-03 12 +72 val_72 2010-03-03 12 +4 val_4 2010-03-03 12 +280 val_280 2010-03-03 12 +35 val_35 2010-03-03 12 +427 val_427 2010-03-03 12 +277 val_277 2010-03-03 12 +208 val_208 2010-03-03 12 +356 val_356 2010-03-03 12 +399 val_399 2010-03-03 12 +169 val_169 2010-03-03 12 +382 val_382 2010-03-03 12 +498 val_498 2010-03-03 12 +125 val_125 2010-03-03 12 +386 val_386 2010-03-03 12 +437 val_437 2010-03-03 12 +469 val_469 2010-03-03 12 +192 val_192 2010-03-03 12 +286 val_286 2010-03-03 12 +187 val_187 2010-03-03 12 +176 val_176 2010-03-03 12 +54 val_54 2010-03-03 12 +459 val_459 2010-03-03 12 +51 val_51 2010-03-03 12 +138 val_138 2010-03-03 12 +103 val_103 2010-03-03 12 +239 val_239 2010-03-03 12 +213 val_213 2010-03-03 12 +216 val_216 2010-03-03 12 +430 val_430 2010-03-03 12 +278 val_278 2010-03-03 12 +176 val_176 2010-03-03 12 +289 val_289 2010-03-03 12 +221 val_221 2010-03-03 12 +65 val_65 2010-03-03 12 +318 val_318 2010-03-03 12 +332 val_332 2010-03-03 12 +311 val_311 2010-03-03 12 +275 val_275 2010-03-03 12 +137 val_137 2010-03-03 12 +241 val_241 2010-03-03 12 +83 val_83 2010-03-03 12 +333 val_333 2010-03-03 12 +180 val_180 2010-03-03 12 +284 val_284 2010-03-03 12 +12 val_12 2010-03-03 12 +230 val_230 2010-03-03 12 +181 val_181 2010-03-03 12 +67 val_67 2010-03-03 12 +260 val_260 2010-03-03 12 +404 val_404 2010-03-03 12 +384 val_384 2010-03-03 12 +489 val_489 2010-03-03 12 +353 val_353 2010-03-03 12 +373 val_373 2010-03-03 12 +272 val_272 2010-03-03 12 +138 val_138 2010-03-03 12 +217 val_217 2010-03-03 12 +84 val_84 2010-03-03 12 +348 val_348 2010-03-03 12 +466 val_466 2010-03-03 12 +58 val_58 2010-03-03 12 +8 val_8 2010-03-03 12 +411 val_411 2010-03-03 12 +230 val_230 2010-03-03 12 +208 val_208 2010-03-03 12 +348 val_348 2010-03-03 12 +24 val_24 2010-03-03 12 +463 val_463 2010-03-03 12 +431 val_431 2010-03-03 12 +179 val_179 2010-03-03 12 +172 val_172 2010-03-03 12 +42 val_42 2010-03-03 12 +129 val_129 2010-03-03 12 +158 val_158 2010-03-03 12 +119 val_119 2010-03-03 12 +496 val_496 2010-03-03 12 +0 val_0 2010-03-03 12 +322 val_322 2010-03-03 12 +197 val_197 2010-03-03 12 +468 val_468 2010-03-03 12 +393 val_393 2010-03-03 12 +454 val_454 2010-03-03 12 +100 val_100 2010-03-03 12 +298 val_298 2010-03-03 12 +199 val_199 2010-03-03 12 +191 val_191 2010-03-03 12 +418 val_418 2010-03-03 12 +96 val_96 2010-03-03 12 +26 val_26 2010-03-03 12 +165 val_165 2010-03-03 12 +327 val_327 2010-03-03 12 +230 val_230 2010-03-03 12 +205 val_205 2010-03-03 12 +120 val_120 2010-03-03 12 +131 val_131 2010-03-03 12 +51 val_51 2010-03-03 12 +404 val_404 2010-03-03 12 +43 val_43 2010-03-03 12 +436 val_436 2010-03-03 12 +156 val_156 2010-03-03 12 +469 val_469 2010-03-03 12 +468 val_468 2010-03-03 12 +308 val_308 2010-03-03 12 +95 val_95 2010-03-03 12 +196 val_196 2010-03-03 12 +288 val_288 2010-03-03 12 +481 val_481 2010-03-03 12 +457 val_457 2010-03-03 12 +98 val_98 2010-03-03 12 +282 val_282 2010-03-03 12 +197 val_197 2010-03-03 12 +187 val_187 2010-03-03 12 +318 val_318 2010-03-03 12 +318 val_318 2010-03-03 12 +409 val_409 2010-03-03 12 +470 val_470 2010-03-03 12 +137 val_137 2010-03-03 12 +369 val_369 2010-03-03 12 +316 val_316 2010-03-03 12 +169 val_169 2010-03-03 12 +413 val_413 2010-03-03 12 +85 val_85 2010-03-03 12 +77 val_77 2010-03-03 12 +0 val_0 2010-03-03 12 +490 val_490 2010-03-03 12 +87 val_87 2010-03-03 12 +364 val_364 2010-03-03 12 +179 val_179 2010-03-03 12 +118 val_118 2010-03-03 12 +134 val_134 2010-03-03 12 +395 val_395 2010-03-03 12 +282 val_282 2010-03-03 12 +138 val_138 2010-03-03 12 +238 val_238 2010-03-03 12 +419 val_419 2010-03-03 12 +15 val_15 2010-03-03 12 +118 val_118 2010-03-03 12 +72 val_72 2010-03-03 12 +90 val_90 2010-03-03 12 +307 val_307 2010-03-03 12 +19 val_19 2010-03-03 12 +435 val_435 2010-03-03 12 +10 val_10 2010-03-03 12 +277 val_277 2010-03-03 12 +273 val_273 2010-03-03 12 +306 val_306 2010-03-03 12 +224 val_224 2010-03-03 12 +309 val_309 2010-03-03 12 +389 val_389 2010-03-03 12 +327 val_327 2010-03-03 12 +242 val_242 2010-03-03 12 +369 val_369 2010-03-03 12 +392 val_392 2010-03-03 12 +272 val_272 2010-03-03 12 +331 val_331 2010-03-03 12 +401 val_401 2010-03-03 12 +242 val_242 2010-03-03 12 +452 val_452 2010-03-03 12 +177 val_177 2010-03-03 12 +226 val_226 2010-03-03 12 +5 val_5 2010-03-03 12 +497 val_497 2010-03-03 12 +402 val_402 2010-03-03 12 +396 val_396 2010-03-03 12 +317 val_317 2010-03-03 12 +395 val_395 2010-03-03 12 +58 val_58 2010-03-03 12 +35 val_35 2010-03-03 12 +336 val_336 2010-03-03 12 +95 val_95 2010-03-03 12 +11 val_11 2010-03-03 12 +168 val_168 2010-03-03 12 +34 val_34 2010-03-03 12 +229 val_229 2010-03-03 12 +233 val_233 2010-03-03 12 +143 val_143 2010-03-03 12 +472 val_472 2010-03-03 12 +322 val_322 2010-03-03 12 +498 val_498 2010-03-03 12 +160 val_160 2010-03-03 12 +195 val_195 2010-03-03 12 +42 val_42 2010-03-03 12 +321 val_321 2010-03-03 12 +430 val_430 2010-03-03 12 +119 val_119 2010-03-03 12 +489 val_489 2010-03-03 12 +458 val_458 2010-03-03 12 +78 val_78 2010-03-03 12 +76 val_76 2010-03-03 12 +41 val_41 2010-03-03 12 +223 val_223 2010-03-03 12 +492 val_492 2010-03-03 12 +149 val_149 2010-03-03 12 +449 val_449 2010-03-03 12 +218 val_218 2010-03-03 12 +228 val_228 2010-03-03 12 +138 val_138 2010-03-03 12 +453 val_453 2010-03-03 12 +30 val_30 2010-03-03 12 +209 val_209 2010-03-03 12 +64 val_64 2010-03-03 12 +468 val_468 2010-03-03 12 +76 val_76 2010-03-03 12 +74 val_74 2010-03-03 12 +342 val_342 2010-03-03 12 +69 val_69 2010-03-03 12 +230 val_230 2010-03-03 12 +33 val_33 2010-03-03 12 +368 val_368 2010-03-03 12 +103 val_103 2010-03-03 12 +296 val_296 2010-03-03 12 +113 val_113 2010-03-03 12 +216 val_216 2010-03-03 12 +367 val_367 2010-03-03 12 +344 val_344 2010-03-03 12 +167 val_167 2010-03-03 12 +274 val_274 2010-03-03 12 +219 val_219 2010-03-03 12 +239 val_239 2010-03-03 12 +485 val_485 2010-03-03 12 +116 val_116 2010-03-03 12 +223 val_223 2010-03-03 12 +256 val_256 2010-03-03 12 +263 val_263 2010-03-03 12 +70 val_70 2010-03-03 12 +487 val_487 2010-03-03 12 +480 val_480 2010-03-03 12 +401 val_401 2010-03-03 12 +288 val_288 2010-03-03 12 +191 val_191 2010-03-03 12 +5 val_5 2010-03-03 12 +244 val_244 2010-03-03 12 +438 val_438 2010-03-03 12 +128 val_128 2010-03-03 12 +467 val_467 2010-03-03 12 +432 val_432 2010-03-03 12 +202 val_202 2010-03-03 12 +316 val_316 2010-03-03 12 +229 val_229 2010-03-03 12 +469 val_469 2010-03-03 12 +463 val_463 2010-03-03 12 +280 val_280 2010-03-03 12 +2 val_2 2010-03-03 12 +35 val_35 2010-03-03 12 +283 val_283 2010-03-03 12 +331 val_331 2010-03-03 12 +235 val_235 2010-03-03 12 +80 val_80 2010-03-03 12 +44 val_44 2010-03-03 12 +193 val_193 2010-03-03 12 +321 val_321 2010-03-03 12 +335 val_335 2010-03-03 12 +104 val_104 2010-03-03 12 +466 val_466 2010-03-03 12 +366 val_366 2010-03-03 12 +175 val_175 2010-03-03 12 +403 val_403 2010-03-03 12 +483 val_483 2010-03-03 12 +53 val_53 2010-03-03 12 +105 val_105 2010-03-03 12 +257 val_257 2010-03-03 12 +406 val_406 2010-03-03 12 +409 val_409 2010-03-03 12 +190 val_190 2010-03-03 12 +406 val_406 2010-03-03 12 +401 val_401 2010-03-03 12 +114 val_114 2010-03-03 12 +258 val_258 2010-03-03 12 +90 val_90 2010-03-03 12 +203 val_203 2010-03-03 12 +262 val_262 2010-03-03 12 +348 val_348 2010-03-03 12 +424 val_424 2010-03-03 12 +12 val_12 2010-03-03 12 +396 val_396 2010-03-03 12 +201 val_201 2010-03-03 12 +217 val_217 2010-03-03 12 +164 val_164 2010-03-03 12 +431 val_431 2010-03-03 12 +454 val_454 2010-03-03 12 +478 val_478 2010-03-03 12 +298 val_298 2010-03-03 12 +125 val_125 2010-03-03 12 +431 val_431 2010-03-03 12 +164 val_164 2010-03-03 12 +424 val_424 2010-03-03 12 +187 val_187 2010-03-03 12 +382 val_382 2010-03-03 12 +5 val_5 2010-03-03 12 +70 val_70 2010-03-03 12 +397 val_397 2010-03-03 12 +480 val_480 2010-03-03 12 +291 val_291 2010-03-03 12 +24 val_24 2010-03-03 12 +351 val_351 2010-03-03 12 +255 val_255 2010-03-03 12 +104 val_104 2010-03-03 12 +70 val_70 2010-03-03 12 +163 val_163 2010-03-03 12 +438 val_438 2010-03-03 12 +119 val_119 2010-03-03 12 +414 val_414 2010-03-03 12 +200 val_200 2010-03-03 12 +491 val_491 2010-03-03 12 +237 val_237 2010-03-03 12 +439 val_439 2010-03-03 12 +360 val_360 2010-03-03 12 +248 val_248 2010-03-03 12 +479 val_479 2010-03-03 12 +305 val_305 2010-03-03 12 +417 val_417 2010-03-03 12 +199 val_199 2010-03-03 12 +444 val_444 2010-03-03 12 +120 val_120 2010-03-03 12 +429 val_429 2010-03-03 12 +169 val_169 2010-03-03 12 +443 val_443 2010-03-03 12 +323 val_323 2010-03-03 12 +325 val_325 2010-03-03 12 +277 val_277 2010-03-03 12 +230 val_230 2010-03-03 12 +478 val_478 2010-03-03 12 +178 val_178 2010-03-03 12 +468 val_468 2010-03-03 12 +310 val_310 2010-03-03 12 +317 val_317 2010-03-03 12 +333 val_333 2010-03-03 12 +493 val_493 2010-03-03 12 +460 val_460 2010-03-03 12 +207 val_207 2010-03-03 12 +249 val_249 2010-03-03 12 +265 val_265 2010-03-03 12 +480 val_480 2010-03-03 12 +83 val_83 2010-03-03 12 +136 val_136 2010-03-03 12 +353 val_353 2010-03-03 12 +172 val_172 2010-03-03 12 +214 val_214 2010-03-03 12 +462 val_462 2010-03-03 12 +233 val_233 2010-03-03 12 +406 val_406 2010-03-03 12 +133 val_133 2010-03-03 12 +175 val_175 2010-03-03 12 +189 val_189 2010-03-03 12 +454 val_454 2010-03-03 12 +375 val_375 2010-03-03 12 +401 val_401 2010-03-03 12 +421 val_421 2010-03-03 12 +407 val_407 2010-03-03 12 +384 val_384 2010-03-03 12 +256 val_256 2010-03-03 12 +26 val_26 2010-03-03 12 +134 val_134 2010-03-03 12 +67 val_67 2010-03-03 12 +384 val_384 2010-03-03 12 +379 val_379 2010-03-03 12 +18 val_18 2010-03-03 12 +462 val_462 2010-03-03 12 +492 val_492 2010-03-03 12 +100 val_100 2010-03-03 12 +298 val_298 2010-03-03 12 +9 val_9 2010-03-03 12 +341 val_341 2010-03-03 12 +498 val_498 2010-03-03 12 +146 val_146 2010-03-03 12 +458 val_458 2010-03-03 12 +362 val_362 2010-03-03 12 +186 val_186 2010-03-03 12 +285 val_285 2010-03-03 12 +348 val_348 2010-03-03 12 +167 val_167 2010-03-03 12 +18 val_18 2010-03-03 12 +273 val_273 2010-03-03 12 +183 val_183 2010-03-03 12 +281 val_281 2010-03-03 12 +344 val_344 2010-03-03 12 +97 val_97 2010-03-03 12 +469 val_469 2010-03-03 12 +315 val_315 2010-03-03 12 +84 val_84 2010-03-03 12 +28 val_28 2010-03-03 12 +37 val_37 2010-03-03 12 +448 val_448 2010-03-03 12 +152 val_152 2010-03-03 12 +348 val_348 2010-03-03 12 +307 val_307 2010-03-03 12 +194 val_194 2010-03-03 12 +414 val_414 2010-03-03 12 +477 val_477 2010-03-03 12 +222 val_222 2010-03-03 12 +126 val_126 2010-03-03 12 +90 val_90 2010-03-03 12 +169 val_169 2010-03-03 12 +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 +238 val_238 2010-03-03 12 +86 val_86 2010-03-03 12 +311 val_311 2010-03-03 12 +27 val_27 2010-03-03 12 +165 val_165 2010-03-03 12 +409 val_409 2010-03-03 12 +255 val_255 2010-03-03 12 +278 val_278 2010-03-03 12 +98 val_98 2010-03-03 12 +484 val_484 2010-03-03 12 +265 val_265 2010-03-03 12 +193 val_193 2010-03-03 12 +401 val_401 2010-03-03 12 +150 val_150 2010-03-03 12 +273 val_273 2010-03-03 12 +224 val_224 2010-03-03 12 +369 val_369 2010-03-03 12 +66 val_66 2010-03-03 12 +128 val_128 2010-03-03 12 +213 val_213 2010-03-03 12 +146 val_146 2010-03-03 12 +406 val_406 2010-03-03 12 +429 val_429 2010-03-03 12 +374 val_374 2010-03-03 12 +152 val_152 2010-03-03 12 +469 val_469 2010-03-03 12 +145 val_145 2010-03-03 12 +495 val_495 2010-03-03 12 +37 val_37 2010-03-03 12 +327 val_327 2010-03-03 12 +281 val_281 2010-03-03 12 +277 val_277 2010-03-03 12 +209 val_209 2010-03-03 12 +15 val_15 2010-03-03 12 +82 val_82 2010-03-03 12 +403 val_403 2010-03-03 12 +166 val_166 2010-03-03 12 +417 val_417 2010-03-03 12 +430 val_430 2010-03-03 12 +252 val_252 2010-03-03 12 +292 val_292 2010-03-03 12 +219 val_219 2010-03-03 12 +287 val_287 2010-03-03 12 +153 val_153 2010-03-03 12 +193 val_193 2010-03-03 12 +338 val_338 2010-03-03 12 +446 val_446 2010-03-03 12 +459 val_459 2010-03-03 12 +394 val_394 2010-03-03 12 +237 val_237 2010-03-03 12 +482 val_482 2010-03-03 12 +174 val_174 2010-03-03 12 +413 val_413 2010-03-03 12 +494 val_494 2010-03-03 12 +207 val_207 2010-03-03 12 +199 val_199 2010-03-03 12 +466 val_466 2010-03-03 12 +208 val_208 2010-03-03 12 +174 val_174 2010-03-03 12 +399 val_399 2010-03-03 12 +396 val_396 2010-03-03 12 +247 val_247 2010-03-03 12 +417 val_417 2010-03-03 12 +489 val_489 2010-03-03 12 +162 val_162 2010-03-03 12 +377 val_377 2010-03-03 12 +397 val_397 2010-03-03 12 +309 val_309 2010-03-03 12 +365 val_365 2010-03-03 12 +266 val_266 2010-03-03 12 +439 val_439 2010-03-03 12 +342 val_342 2010-03-03 12 +367 val_367 2010-03-03 12 +325 val_325 2010-03-03 12 +167 val_167 2010-03-03 12 +195 val_195 2010-03-03 12 +475 val_475 2010-03-03 12 +17 val_17 2010-03-03 12 +113 val_113 2010-03-03 12 +155 val_155 2010-03-03 12 +203 val_203 2010-03-03 12 +339 val_339 2010-03-03 12 +0 val_0 2010-03-03 12 +455 val_455 2010-03-03 12 +128 val_128 2010-03-03 12 +311 val_311 2010-03-03 12 +316 val_316 2010-03-03 12 +57 val_57 2010-03-03 12 +302 val_302 2010-03-03 12 +205 val_205 2010-03-03 12 +149 val_149 2010-03-03 12 +438 val_438 2010-03-03 12 +345 val_345 2010-03-03 12 +129 val_129 2010-03-03 12 +170 val_170 2010-03-03 12 +20 val_20 2010-03-03 12 +489 val_489 2010-03-03 12 +157 val_157 2010-03-03 12 +378 val_378 2010-03-03 12 +221 val_221 2010-03-03 12 +92 val_92 2010-03-03 12 +111 val_111 2010-03-03 12 +47 val_47 2010-03-03 12 +72 val_72 2010-03-03 12 +4 val_4 2010-03-03 12 +280 val_280 2010-03-03 12 +35 val_35 2010-03-03 12 +427 val_427 2010-03-03 12 +277 val_277 2010-03-03 12 +208 val_208 2010-03-03 12 +356 val_356 2010-03-03 12 +399 val_399 2010-03-03 12 +169 val_169 2010-03-03 12 +382 val_382 2010-03-03 12 +498 val_498 2010-03-03 12 +125 val_125 2010-03-03 12 +386 val_386 2010-03-03 12 +437 val_437 2010-03-03 12 +469 val_469 2010-03-03 12 +192 val_192 2010-03-03 12 +286 val_286 2010-03-03 12 +187 val_187 2010-03-03 12 +176 val_176 2010-03-03 12 +54 val_54 2010-03-03 12 +459 val_459 2010-03-03 12 +51 val_51 2010-03-03 12 +138 val_138 2010-03-03 12 +103 val_103 2010-03-03 12 +239 val_239 2010-03-03 12 +213 val_213 2010-03-03 12 +216 val_216 2010-03-03 12 +430 val_430 2010-03-03 12 +278 val_278 2010-03-03 12 +176 val_176 2010-03-03 12 +289 val_289 2010-03-03 12 +221 val_221 2010-03-03 12 +65 val_65 2010-03-03 12 +318 val_318 2010-03-03 12 +332 val_332 2010-03-03 12 +311 val_311 2010-03-03 12 +275 val_275 2010-03-03 12 +137 val_137 2010-03-03 12 +241 val_241 2010-03-03 12 +83 val_83 2010-03-03 12 +333 val_333 2010-03-03 12 +180 val_180 2010-03-03 12 +284 val_284 2010-03-03 12 +12 val_12 2010-03-03 12 +230 val_230 2010-03-03 12 +181 val_181 2010-03-03 12 +67 val_67 2010-03-03 12 +260 val_260 2010-03-03 12 +404 val_404 2010-03-03 12 +384 val_384 2010-03-03 12 +489 val_489 2010-03-03 12 +353 val_353 2010-03-03 12 +373 val_373 2010-03-03 12 +272 val_272 2010-03-03 12 +138 val_138 2010-03-03 12 +217 val_217 2010-03-03 12 +84 val_84 2010-03-03 12 +348 val_348 2010-03-03 12 +466 val_466 2010-03-03 12 +58 val_58 2010-03-03 12 +8 val_8 2010-03-03 12 +411 val_411 2010-03-03 12 +230 val_230 2010-03-03 12 +208 val_208 2010-03-03 12 +348 val_348 2010-03-03 12 +24 val_24 2010-03-03 12 +463 val_463 2010-03-03 12 +431 val_431 2010-03-03 12 +179 val_179 2010-03-03 12 +172 val_172 2010-03-03 12 +42 val_42 2010-03-03 12 +129 val_129 2010-03-03 12 +158 val_158 2010-03-03 12 +119 val_119 2010-03-03 12 +496 val_496 2010-03-03 12 +0 val_0 2010-03-03 12 +322 val_322 2010-03-03 12 +197 val_197 2010-03-03 12 +468 val_468 2010-03-03 12 +393 val_393 2010-03-03 12 +454 val_454 2010-03-03 12 +100 val_100 2010-03-03 12 +298 val_298 2010-03-03 12 +199 val_199 2010-03-03 12 +191 val_191 2010-03-03 12 +418 val_418 2010-03-03 12 +96 val_96 2010-03-03 12 +26 val_26 2010-03-03 12 +165 val_165 2010-03-03 12 +327 val_327 2010-03-03 12 +230 val_230 2010-03-03 12 +205 val_205 2010-03-03 12 +120 val_120 2010-03-03 12 +131 val_131 2010-03-03 12 +51 val_51 2010-03-03 12 +404 val_404 2010-03-03 12 +43 val_43 2010-03-03 12 +436 val_436 2010-03-03 12 +156 val_156 2010-03-03 12 +469 val_469 2010-03-03 12 +468 val_468 2010-03-03 12 +308 val_308 2010-03-03 12 +95 val_95 2010-03-03 12 +196 val_196 2010-03-03 12 +288 val_288 2010-03-03 12 +481 val_481 2010-03-03 12 +457 val_457 2010-03-03 12 +98 val_98 2010-03-03 12 +282 val_282 2010-03-03 12 +197 val_197 2010-03-03 12 +187 val_187 2010-03-03 12 +318 val_318 2010-03-03 12 +318 val_318 2010-03-03 12 +409 val_409 2010-03-03 12 +470 val_470 2010-03-03 12 +137 val_137 2010-03-03 12 +369 val_369 2010-03-03 12 +316 val_316 2010-03-03 12 +169 val_169 2010-03-03 12 +413 val_413 2010-03-03 12 +85 val_85 2010-03-03 12 +77 val_77 2010-03-03 12 +0 val_0 2010-03-03 12 +490 val_490 2010-03-03 12 +87 val_87 2010-03-03 12 +364 val_364 2010-03-03 12 +179 val_179 2010-03-03 12 +118 val_118 2010-03-03 12 +134 val_134 2010-03-03 12 +395 val_395 2010-03-03 12 +282 val_282 2010-03-03 12 +138 val_138 2010-03-03 12 +238 val_238 2010-03-03 12 +419 val_419 2010-03-03 12 +15 val_15 2010-03-03 12 +118 val_118 2010-03-03 12 +72 val_72 2010-03-03 12 +90 val_90 2010-03-03 12 +307 val_307 2010-03-03 12 +19 val_19 2010-03-03 12 +435 val_435 2010-03-03 12 +10 val_10 2010-03-03 12 +277 val_277 2010-03-03 12 +273 val_273 2010-03-03 12 +306 val_306 2010-03-03 12 +224 val_224 2010-03-03 12 +309 val_309 2010-03-03 12 +389 val_389 2010-03-03 12 +327 val_327 2010-03-03 12 +242 val_242 2010-03-03 12 +369 val_369 2010-03-03 12 +392 val_392 2010-03-03 12 +272 val_272 2010-03-03 12 +331 val_331 2010-03-03 12 +401 val_401 2010-03-03 12 +242 val_242 2010-03-03 12 +452 val_452 2010-03-03 12 +177 val_177 2010-03-03 12 +226 val_226 2010-03-03 12 +5 val_5 2010-03-03 12 +497 val_497 2010-03-03 12 +402 val_402 2010-03-03 12 +396 val_396 2010-03-03 12 +317 val_317 2010-03-03 12 +395 val_395 2010-03-03 12 +58 val_58 2010-03-03 12 +35 val_35 2010-03-03 12 +336 val_336 2010-03-03 12 +95 val_95 2010-03-03 12 +11 val_11 2010-03-03 12 +168 val_168 2010-03-03 12 +34 val_34 2010-03-03 12 +229 val_229 2010-03-03 12 +233 val_233 2010-03-03 12 +143 val_143 2010-03-03 12 +472 val_472 2010-03-03 12 +322 val_322 2010-03-03 12 +498 val_498 2010-03-03 12 +160 val_160 2010-03-03 12 +195 val_195 2010-03-03 12 +42 val_42 2010-03-03 12 +321 val_321 2010-03-03 12 +430 val_430 2010-03-03 12 +119 val_119 2010-03-03 12 +489 val_489 2010-03-03 12 +458 val_458 2010-03-03 12 +78 val_78 2010-03-03 12 +76 val_76 2010-03-03 12 +41 val_41 2010-03-03 12 +223 val_223 2010-03-03 12 +492 val_492 2010-03-03 12 +149 val_149 2010-03-03 12 +449 val_449 2010-03-03 12 +218 val_218 2010-03-03 12 +228 val_228 2010-03-03 12 +138 val_138 2010-03-03 12 +453 val_453 2010-03-03 12 +30 val_30 2010-03-03 12 +209 val_209 2010-03-03 12 +64 val_64 2010-03-03 12 +468 val_468 2010-03-03 12 +76 val_76 2010-03-03 12 +74 val_74 2010-03-03 12 +342 val_342 2010-03-03 12 +69 val_69 2010-03-03 12 +230 val_230 2010-03-03 12 +33 val_33 2010-03-03 12 +368 val_368 2010-03-03 12 +103 val_103 2010-03-03 12 +296 val_296 2010-03-03 12 +113 val_113 2010-03-03 12 +216 val_216 2010-03-03 12 +367 val_367 2010-03-03 12 +344 val_344 2010-03-03 12 +167 val_167 2010-03-03 12 +274 val_274 2010-03-03 12 +219 val_219 2010-03-03 12 +239 val_239 2010-03-03 12 +485 val_485 2010-03-03 12 +116 val_116 2010-03-03 12 +223 val_223 2010-03-03 12 +256 val_256 2010-03-03 12 +263 val_263 2010-03-03 12 +70 val_70 2010-03-03 12 +487 val_487 2010-03-03 12 +480 val_480 2010-03-03 12 +401 val_401 2010-03-03 12 +288 val_288 2010-03-03 12 +191 val_191 2010-03-03 12 +5 val_5 2010-03-03 12 +244 val_244 2010-03-03 12 +438 val_438 2010-03-03 12 +128 val_128 2010-03-03 12 +467 val_467 2010-03-03 12 +432 val_432 2010-03-03 12 +202 val_202 2010-03-03 12 +316 val_316 2010-03-03 12 +229 val_229 2010-03-03 12 +469 val_469 2010-03-03 12 +463 val_463 2010-03-03 12 +280 val_280 2010-03-03 12 +2 val_2 2010-03-03 12 +35 val_35 2010-03-03 12 +283 val_283 2010-03-03 12 +331 val_331 2010-03-03 12 +235 val_235 2010-03-03 12 +80 val_80 2010-03-03 12 +44 val_44 2010-03-03 12 +193 val_193 2010-03-03 12 +321 val_321 2010-03-03 12 +335 val_335 2010-03-03 12 +104 val_104 2010-03-03 12 +466 val_466 2010-03-03 12 +366 val_366 2010-03-03 12 +175 val_175 2010-03-03 12 +403 val_403 2010-03-03 12 +483 val_483 2010-03-03 12 +53 val_53 2010-03-03 12 +105 val_105 2010-03-03 12 +257 val_257 2010-03-03 12 +406 val_406 2010-03-03 12 +409 val_409 2010-03-03 12 +190 val_190 2010-03-03 12 +406 val_406 2010-03-03 12 +401 val_401 2010-03-03 12 +114 val_114 2010-03-03 12 +258 val_258 2010-03-03 12 +90 val_90 2010-03-03 12 +203 val_203 2010-03-03 12 +262 val_262 2010-03-03 12 +348 val_348 2010-03-03 12 +424 val_424 2010-03-03 12 +12 val_12 2010-03-03 12 +396 val_396 2010-03-03 12 +201 val_201 2010-03-03 12 +217 val_217 2010-03-03 12 +164 val_164 2010-03-03 12 +431 val_431 2010-03-03 12 +454 val_454 2010-03-03 12 +478 val_478 2010-03-03 12 +298 val_298 2010-03-03 12 +125 val_125 2010-03-03 12 +431 val_431 2010-03-03 12 +164 val_164 2010-03-03 12 +424 val_424 2010-03-03 12 +187 val_187 2010-03-03 12 +382 val_382 2010-03-03 12 +5 val_5 2010-03-03 12 +70 val_70 2010-03-03 12 +397 val_397 2010-03-03 12 +480 val_480 2010-03-03 12 +291 val_291 2010-03-03 12 +24 val_24 2010-03-03 12 +351 val_351 2010-03-03 12 +255 val_255 2010-03-03 12 +104 val_104 2010-03-03 12 +70 val_70 2010-03-03 12 +163 val_163 2010-03-03 12 +438 val_438 2010-03-03 12 +119 val_119 2010-03-03 12 +414 val_414 2010-03-03 12 +200 val_200 2010-03-03 12 +491 val_491 2010-03-03 12 +237 val_237 2010-03-03 12 +439 val_439 2010-03-03 12 +360 val_360 2010-03-03 12 +248 val_248 2010-03-03 12 +479 val_479 2010-03-03 12 +305 val_305 2010-03-03 12 +417 val_417 2010-03-03 12 +199 val_199 2010-03-03 12 +444 val_444 2010-03-03 12 +120 val_120 2010-03-03 12 +429 val_429 2010-03-03 12 +169 val_169 2010-03-03 12 +443 val_443 2010-03-03 12 +323 val_323 2010-03-03 12 +325 val_325 2010-03-03 12 +277 val_277 2010-03-03 12 +230 val_230 2010-03-03 12 +478 val_478 2010-03-03 12 +178 val_178 2010-03-03 12 +468 val_468 2010-03-03 12 +310 val_310 2010-03-03 12 +317 val_317 2010-03-03 12 +333 val_333 2010-03-03 12 +493 val_493 2010-03-03 12 +460 val_460 2010-03-03 12 +207 val_207 2010-03-03 12 +249 val_249 2010-03-03 12 +265 val_265 2010-03-03 12 +480 val_480 2010-03-03 12 +83 val_83 2010-03-03 12 +136 val_136 2010-03-03 12 +353 val_353 2010-03-03 12 +172 val_172 2010-03-03 12 +214 val_214 2010-03-03 12 +462 val_462 2010-03-03 12 +233 val_233 2010-03-03 12 +406 val_406 2010-03-03 12 +133 val_133 2010-03-03 12 +175 val_175 2010-03-03 12 +189 val_189 2010-03-03 12 +454 val_454 2010-03-03 12 +375 val_375 2010-03-03 12 +401 val_401 2010-03-03 12 +421 val_421 2010-03-03 12 +407 val_407 2010-03-03 12 +384 val_384 2010-03-03 12 +256 val_256 2010-03-03 12 +26 val_26 2010-03-03 12 +134 val_134 2010-03-03 12 +67 val_67 2010-03-03 12 +384 val_384 2010-03-03 12 +379 val_379 2010-03-03 12 +18 val_18 2010-03-03 12 +462 val_462 2010-03-03 12 +492 val_492 2010-03-03 12 +100 val_100 2010-03-03 12 +298 val_298 2010-03-03 12 +9 val_9 2010-03-03 12 +341 val_341 2010-03-03 12 +498 val_498 2010-03-03 12 +146 val_146 2010-03-03 12 +458 val_458 2010-03-03 12 +362 val_362 2010-03-03 12 +186 val_186 2010-03-03 12 +285 val_285 2010-03-03 12 +348 val_348 2010-03-03 12 +167 val_167 2010-03-03 12 +18 val_18 2010-03-03 12 +273 val_273 2010-03-03 12 +183 val_183 2010-03-03 12 +281 val_281 2010-03-03 12 +344 val_344 2010-03-03 12 +97 val_97 2010-03-03 12 +469 val_469 2010-03-03 12 +315 val_315 2010-03-03 12 +84 val_84 2010-03-03 12 +28 val_28 2010-03-03 12 +37 val_37 2010-03-03 12 +448 val_448 2010-03-03 12 +152 val_152 2010-03-03 12 +348 val_348 2010-03-03 12 +307 val_307 2010-03-03 12 +194 val_194 2010-03-03 12 +414 val_414 2010-03-03 12 +477 val_477 2010-03-03 12 +222 val_222 2010-03-03 12 +126 val_126 2010-03-03 12 +90 val_90 2010-03-03 12 +169 val_169 2010-03-03 12 +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 diff --git a/src/test/hive/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 b/src/test/hive/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a b/src/test/hive/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a new file mode 100644 index 0000000000000..d62156d392d5e --- /dev/null +++ b/src/test/hive/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a @@ -0,0 +1,12 @@ +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 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) diff --git a/src/test/hive/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 b/src/test/hive/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 b/src/test/hive/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 new file mode 100644 index 0000000000000..ed0d81e7a9d44 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 @@ -0,0 +1 @@ +ds=2010-03-03/hr=12 diff --git a/src/test/hive/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 b/src/test/hive/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 new file mode 100644 index 0000000000000..542a689ea8311 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 @@ -0,0 +1,500 @@ +238 val_238 2010-03-03 12 +86 val_86 2010-03-03 12 +311 val_311 2010-03-03 12 +27 val_27 2010-03-03 12 +165 val_165 2010-03-03 12 +409 val_409 2010-03-03 12 +255 val_255 2010-03-03 12 +278 val_278 2010-03-03 12 +98 val_98 2010-03-03 12 +484 val_484 2010-03-03 12 +265 val_265 2010-03-03 12 +193 val_193 2010-03-03 12 +401 val_401 2010-03-03 12 +150 val_150 2010-03-03 12 +273 val_273 2010-03-03 12 +224 val_224 2010-03-03 12 +369 val_369 2010-03-03 12 +66 val_66 2010-03-03 12 +128 val_128 2010-03-03 12 +213 val_213 2010-03-03 12 +146 val_146 2010-03-03 12 +406 val_406 2010-03-03 12 +429 val_429 2010-03-03 12 +374 val_374 2010-03-03 12 +152 val_152 2010-03-03 12 +469 val_469 2010-03-03 12 +145 val_145 2010-03-03 12 +495 val_495 2010-03-03 12 +37 val_37 2010-03-03 12 +327 val_327 2010-03-03 12 +281 val_281 2010-03-03 12 +277 val_277 2010-03-03 12 +209 val_209 2010-03-03 12 +15 val_15 2010-03-03 12 +82 val_82 2010-03-03 12 +403 val_403 2010-03-03 12 +166 val_166 2010-03-03 12 +417 val_417 2010-03-03 12 +430 val_430 2010-03-03 12 +252 val_252 2010-03-03 12 +292 val_292 2010-03-03 12 +219 val_219 2010-03-03 12 +287 val_287 2010-03-03 12 +153 val_153 2010-03-03 12 +193 val_193 2010-03-03 12 +338 val_338 2010-03-03 12 +446 val_446 2010-03-03 12 +459 val_459 2010-03-03 12 +394 val_394 2010-03-03 12 +237 val_237 2010-03-03 12 +482 val_482 2010-03-03 12 +174 val_174 2010-03-03 12 +413 val_413 2010-03-03 12 +494 val_494 2010-03-03 12 +207 val_207 2010-03-03 12 +199 val_199 2010-03-03 12 +466 val_466 2010-03-03 12 +208 val_208 2010-03-03 12 +174 val_174 2010-03-03 12 +399 val_399 2010-03-03 12 +396 val_396 2010-03-03 12 +247 val_247 2010-03-03 12 +417 val_417 2010-03-03 12 +489 val_489 2010-03-03 12 +162 val_162 2010-03-03 12 +377 val_377 2010-03-03 12 +397 val_397 2010-03-03 12 +309 val_309 2010-03-03 12 +365 val_365 2010-03-03 12 +266 val_266 2010-03-03 12 +439 val_439 2010-03-03 12 +342 val_342 2010-03-03 12 +367 val_367 2010-03-03 12 +325 val_325 2010-03-03 12 +167 val_167 2010-03-03 12 +195 val_195 2010-03-03 12 +475 val_475 2010-03-03 12 +17 val_17 2010-03-03 12 +113 val_113 2010-03-03 12 +155 val_155 2010-03-03 12 +203 val_203 2010-03-03 12 +339 val_339 2010-03-03 12 +0 val_0 2010-03-03 12 +455 val_455 2010-03-03 12 +128 val_128 2010-03-03 12 +311 val_311 2010-03-03 12 +316 val_316 2010-03-03 12 +57 val_57 2010-03-03 12 +302 val_302 2010-03-03 12 +205 val_205 2010-03-03 12 +149 val_149 2010-03-03 12 +438 val_438 2010-03-03 12 +345 val_345 2010-03-03 12 +129 val_129 2010-03-03 12 +170 val_170 2010-03-03 12 +20 val_20 2010-03-03 12 +489 val_489 2010-03-03 12 +157 val_157 2010-03-03 12 +378 val_378 2010-03-03 12 +221 val_221 2010-03-03 12 +92 val_92 2010-03-03 12 +111 val_111 2010-03-03 12 +47 val_47 2010-03-03 12 +72 val_72 2010-03-03 12 +4 val_4 2010-03-03 12 +280 val_280 2010-03-03 12 +35 val_35 2010-03-03 12 +427 val_427 2010-03-03 12 +277 val_277 2010-03-03 12 +208 val_208 2010-03-03 12 +356 val_356 2010-03-03 12 +399 val_399 2010-03-03 12 +169 val_169 2010-03-03 12 +382 val_382 2010-03-03 12 +498 val_498 2010-03-03 12 +125 val_125 2010-03-03 12 +386 val_386 2010-03-03 12 +437 val_437 2010-03-03 12 +469 val_469 2010-03-03 12 +192 val_192 2010-03-03 12 +286 val_286 2010-03-03 12 +187 val_187 2010-03-03 12 +176 val_176 2010-03-03 12 +54 val_54 2010-03-03 12 +459 val_459 2010-03-03 12 +51 val_51 2010-03-03 12 +138 val_138 2010-03-03 12 +103 val_103 2010-03-03 12 +239 val_239 2010-03-03 12 +213 val_213 2010-03-03 12 +216 val_216 2010-03-03 12 +430 val_430 2010-03-03 12 +278 val_278 2010-03-03 12 +176 val_176 2010-03-03 12 +289 val_289 2010-03-03 12 +221 val_221 2010-03-03 12 +65 val_65 2010-03-03 12 +318 val_318 2010-03-03 12 +332 val_332 2010-03-03 12 +311 val_311 2010-03-03 12 +275 val_275 2010-03-03 12 +137 val_137 2010-03-03 12 +241 val_241 2010-03-03 12 +83 val_83 2010-03-03 12 +333 val_333 2010-03-03 12 +180 val_180 2010-03-03 12 +284 val_284 2010-03-03 12 +12 val_12 2010-03-03 12 +230 val_230 2010-03-03 12 +181 val_181 2010-03-03 12 +67 val_67 2010-03-03 12 +260 val_260 2010-03-03 12 +404 val_404 2010-03-03 12 +384 val_384 2010-03-03 12 +489 val_489 2010-03-03 12 +353 val_353 2010-03-03 12 +373 val_373 2010-03-03 12 +272 val_272 2010-03-03 12 +138 val_138 2010-03-03 12 +217 val_217 2010-03-03 12 +84 val_84 2010-03-03 12 +348 val_348 2010-03-03 12 +466 val_466 2010-03-03 12 +58 val_58 2010-03-03 12 +8 val_8 2010-03-03 12 +411 val_411 2010-03-03 12 +230 val_230 2010-03-03 12 +208 val_208 2010-03-03 12 +348 val_348 2010-03-03 12 +24 val_24 2010-03-03 12 +463 val_463 2010-03-03 12 +431 val_431 2010-03-03 12 +179 val_179 2010-03-03 12 +172 val_172 2010-03-03 12 +42 val_42 2010-03-03 12 +129 val_129 2010-03-03 12 +158 val_158 2010-03-03 12 +119 val_119 2010-03-03 12 +496 val_496 2010-03-03 12 +0 val_0 2010-03-03 12 +322 val_322 2010-03-03 12 +197 val_197 2010-03-03 12 +468 val_468 2010-03-03 12 +393 val_393 2010-03-03 12 +454 val_454 2010-03-03 12 +100 val_100 2010-03-03 12 +298 val_298 2010-03-03 12 +199 val_199 2010-03-03 12 +191 val_191 2010-03-03 12 +418 val_418 2010-03-03 12 +96 val_96 2010-03-03 12 +26 val_26 2010-03-03 12 +165 val_165 2010-03-03 12 +327 val_327 2010-03-03 12 +230 val_230 2010-03-03 12 +205 val_205 2010-03-03 12 +120 val_120 2010-03-03 12 +131 val_131 2010-03-03 12 +51 val_51 2010-03-03 12 +404 val_404 2010-03-03 12 +43 val_43 2010-03-03 12 +436 val_436 2010-03-03 12 +156 val_156 2010-03-03 12 +469 val_469 2010-03-03 12 +468 val_468 2010-03-03 12 +308 val_308 2010-03-03 12 +95 val_95 2010-03-03 12 +196 val_196 2010-03-03 12 +288 val_288 2010-03-03 12 +481 val_481 2010-03-03 12 +457 val_457 2010-03-03 12 +98 val_98 2010-03-03 12 +282 val_282 2010-03-03 12 +197 val_197 2010-03-03 12 +187 val_187 2010-03-03 12 +318 val_318 2010-03-03 12 +318 val_318 2010-03-03 12 +409 val_409 2010-03-03 12 +470 val_470 2010-03-03 12 +137 val_137 2010-03-03 12 +369 val_369 2010-03-03 12 +316 val_316 2010-03-03 12 +169 val_169 2010-03-03 12 +413 val_413 2010-03-03 12 +85 val_85 2010-03-03 12 +77 val_77 2010-03-03 12 +0 val_0 2010-03-03 12 +490 val_490 2010-03-03 12 +87 val_87 2010-03-03 12 +364 val_364 2010-03-03 12 +179 val_179 2010-03-03 12 +118 val_118 2010-03-03 12 +134 val_134 2010-03-03 12 +395 val_395 2010-03-03 12 +282 val_282 2010-03-03 12 +138 val_138 2010-03-03 12 +238 val_238 2010-03-03 12 +419 val_419 2010-03-03 12 +15 val_15 2010-03-03 12 +118 val_118 2010-03-03 12 +72 val_72 2010-03-03 12 +90 val_90 2010-03-03 12 +307 val_307 2010-03-03 12 +19 val_19 2010-03-03 12 +435 val_435 2010-03-03 12 +10 val_10 2010-03-03 12 +277 val_277 2010-03-03 12 +273 val_273 2010-03-03 12 +306 val_306 2010-03-03 12 +224 val_224 2010-03-03 12 +309 val_309 2010-03-03 12 +389 val_389 2010-03-03 12 +327 val_327 2010-03-03 12 +242 val_242 2010-03-03 12 +369 val_369 2010-03-03 12 +392 val_392 2010-03-03 12 +272 val_272 2010-03-03 12 +331 val_331 2010-03-03 12 +401 val_401 2010-03-03 12 +242 val_242 2010-03-03 12 +452 val_452 2010-03-03 12 +177 val_177 2010-03-03 12 +226 val_226 2010-03-03 12 +5 val_5 2010-03-03 12 +497 val_497 2010-03-03 12 +402 val_402 2010-03-03 12 +396 val_396 2010-03-03 12 +317 val_317 2010-03-03 12 +395 val_395 2010-03-03 12 +58 val_58 2010-03-03 12 +35 val_35 2010-03-03 12 +336 val_336 2010-03-03 12 +95 val_95 2010-03-03 12 +11 val_11 2010-03-03 12 +168 val_168 2010-03-03 12 +34 val_34 2010-03-03 12 +229 val_229 2010-03-03 12 +233 val_233 2010-03-03 12 +143 val_143 2010-03-03 12 +472 val_472 2010-03-03 12 +322 val_322 2010-03-03 12 +498 val_498 2010-03-03 12 +160 val_160 2010-03-03 12 +195 val_195 2010-03-03 12 +42 val_42 2010-03-03 12 +321 val_321 2010-03-03 12 +430 val_430 2010-03-03 12 +119 val_119 2010-03-03 12 +489 val_489 2010-03-03 12 +458 val_458 2010-03-03 12 +78 val_78 2010-03-03 12 +76 val_76 2010-03-03 12 +41 val_41 2010-03-03 12 +223 val_223 2010-03-03 12 +492 val_492 2010-03-03 12 +149 val_149 2010-03-03 12 +449 val_449 2010-03-03 12 +218 val_218 2010-03-03 12 +228 val_228 2010-03-03 12 +138 val_138 2010-03-03 12 +453 val_453 2010-03-03 12 +30 val_30 2010-03-03 12 +209 val_209 2010-03-03 12 +64 val_64 2010-03-03 12 +468 val_468 2010-03-03 12 +76 val_76 2010-03-03 12 +74 val_74 2010-03-03 12 +342 val_342 2010-03-03 12 +69 val_69 2010-03-03 12 +230 val_230 2010-03-03 12 +33 val_33 2010-03-03 12 +368 val_368 2010-03-03 12 +103 val_103 2010-03-03 12 +296 val_296 2010-03-03 12 +113 val_113 2010-03-03 12 +216 val_216 2010-03-03 12 +367 val_367 2010-03-03 12 +344 val_344 2010-03-03 12 +167 val_167 2010-03-03 12 +274 val_274 2010-03-03 12 +219 val_219 2010-03-03 12 +239 val_239 2010-03-03 12 +485 val_485 2010-03-03 12 +116 val_116 2010-03-03 12 +223 val_223 2010-03-03 12 +256 val_256 2010-03-03 12 +263 val_263 2010-03-03 12 +70 val_70 2010-03-03 12 +487 val_487 2010-03-03 12 +480 val_480 2010-03-03 12 +401 val_401 2010-03-03 12 +288 val_288 2010-03-03 12 +191 val_191 2010-03-03 12 +5 val_5 2010-03-03 12 +244 val_244 2010-03-03 12 +438 val_438 2010-03-03 12 +128 val_128 2010-03-03 12 +467 val_467 2010-03-03 12 +432 val_432 2010-03-03 12 +202 val_202 2010-03-03 12 +316 val_316 2010-03-03 12 +229 val_229 2010-03-03 12 +469 val_469 2010-03-03 12 +463 val_463 2010-03-03 12 +280 val_280 2010-03-03 12 +2 val_2 2010-03-03 12 +35 val_35 2010-03-03 12 +283 val_283 2010-03-03 12 +331 val_331 2010-03-03 12 +235 val_235 2010-03-03 12 +80 val_80 2010-03-03 12 +44 val_44 2010-03-03 12 +193 val_193 2010-03-03 12 +321 val_321 2010-03-03 12 +335 val_335 2010-03-03 12 +104 val_104 2010-03-03 12 +466 val_466 2010-03-03 12 +366 val_366 2010-03-03 12 +175 val_175 2010-03-03 12 +403 val_403 2010-03-03 12 +483 val_483 2010-03-03 12 +53 val_53 2010-03-03 12 +105 val_105 2010-03-03 12 +257 val_257 2010-03-03 12 +406 val_406 2010-03-03 12 +409 val_409 2010-03-03 12 +190 val_190 2010-03-03 12 +406 val_406 2010-03-03 12 +401 val_401 2010-03-03 12 +114 val_114 2010-03-03 12 +258 val_258 2010-03-03 12 +90 val_90 2010-03-03 12 +203 val_203 2010-03-03 12 +262 val_262 2010-03-03 12 +348 val_348 2010-03-03 12 +424 val_424 2010-03-03 12 +12 val_12 2010-03-03 12 +396 val_396 2010-03-03 12 +201 val_201 2010-03-03 12 +217 val_217 2010-03-03 12 +164 val_164 2010-03-03 12 +431 val_431 2010-03-03 12 +454 val_454 2010-03-03 12 +478 val_478 2010-03-03 12 +298 val_298 2010-03-03 12 +125 val_125 2010-03-03 12 +431 val_431 2010-03-03 12 +164 val_164 2010-03-03 12 +424 val_424 2010-03-03 12 +187 val_187 2010-03-03 12 +382 val_382 2010-03-03 12 +5 val_5 2010-03-03 12 +70 val_70 2010-03-03 12 +397 val_397 2010-03-03 12 +480 val_480 2010-03-03 12 +291 val_291 2010-03-03 12 +24 val_24 2010-03-03 12 +351 val_351 2010-03-03 12 +255 val_255 2010-03-03 12 +104 val_104 2010-03-03 12 +70 val_70 2010-03-03 12 +163 val_163 2010-03-03 12 +438 val_438 2010-03-03 12 +119 val_119 2010-03-03 12 +414 val_414 2010-03-03 12 +200 val_200 2010-03-03 12 +491 val_491 2010-03-03 12 +237 val_237 2010-03-03 12 +439 val_439 2010-03-03 12 +360 val_360 2010-03-03 12 +248 val_248 2010-03-03 12 +479 val_479 2010-03-03 12 +305 val_305 2010-03-03 12 +417 val_417 2010-03-03 12 +199 val_199 2010-03-03 12 +444 val_444 2010-03-03 12 +120 val_120 2010-03-03 12 +429 val_429 2010-03-03 12 +169 val_169 2010-03-03 12 +443 val_443 2010-03-03 12 +323 val_323 2010-03-03 12 +325 val_325 2010-03-03 12 +277 val_277 2010-03-03 12 +230 val_230 2010-03-03 12 +478 val_478 2010-03-03 12 +178 val_178 2010-03-03 12 +468 val_468 2010-03-03 12 +310 val_310 2010-03-03 12 +317 val_317 2010-03-03 12 +333 val_333 2010-03-03 12 +493 val_493 2010-03-03 12 +460 val_460 2010-03-03 12 +207 val_207 2010-03-03 12 +249 val_249 2010-03-03 12 +265 val_265 2010-03-03 12 +480 val_480 2010-03-03 12 +83 val_83 2010-03-03 12 +136 val_136 2010-03-03 12 +353 val_353 2010-03-03 12 +172 val_172 2010-03-03 12 +214 val_214 2010-03-03 12 +462 val_462 2010-03-03 12 +233 val_233 2010-03-03 12 +406 val_406 2010-03-03 12 +133 val_133 2010-03-03 12 +175 val_175 2010-03-03 12 +189 val_189 2010-03-03 12 +454 val_454 2010-03-03 12 +375 val_375 2010-03-03 12 +401 val_401 2010-03-03 12 +421 val_421 2010-03-03 12 +407 val_407 2010-03-03 12 +384 val_384 2010-03-03 12 +256 val_256 2010-03-03 12 +26 val_26 2010-03-03 12 +134 val_134 2010-03-03 12 +67 val_67 2010-03-03 12 +384 val_384 2010-03-03 12 +379 val_379 2010-03-03 12 +18 val_18 2010-03-03 12 +462 val_462 2010-03-03 12 +492 val_492 2010-03-03 12 +100 val_100 2010-03-03 12 +298 val_298 2010-03-03 12 +9 val_9 2010-03-03 12 +341 val_341 2010-03-03 12 +498 val_498 2010-03-03 12 +146 val_146 2010-03-03 12 +458 val_458 2010-03-03 12 +362 val_362 2010-03-03 12 +186 val_186 2010-03-03 12 +285 val_285 2010-03-03 12 +348 val_348 2010-03-03 12 +167 val_167 2010-03-03 12 +18 val_18 2010-03-03 12 +273 val_273 2010-03-03 12 +183 val_183 2010-03-03 12 +281 val_281 2010-03-03 12 +344 val_344 2010-03-03 12 +97 val_97 2010-03-03 12 +469 val_469 2010-03-03 12 +315 val_315 2010-03-03 12 +84 val_84 2010-03-03 12 +28 val_28 2010-03-03 12 +37 val_37 2010-03-03 12 +448 val_448 2010-03-03 12 +152 val_152 2010-03-03 12 +348 val_348 2010-03-03 12 +307 val_307 2010-03-03 12 +194 val_194 2010-03-03 12 +414 val_414 2010-03-03 12 +477 val_477 2010-03-03 12 +222 val_222 2010-03-03 12 +126 val_126 2010-03-03 12 +90 val_90 2010-03-03 12 +169 val_169 2010-03-03 12 +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 diff --git a/src/test/hive/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c b/src/test/hive/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a b/src/test/hive/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a new file mode 100644 index 0000000000000..91fa51871a4b8 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a @@ -0,0 +1,12 @@ +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 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) diff --git a/src/test/hive/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 b/src/test/hive/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 b/src/test/hive/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b b/src/test/hive/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b new file mode 100644 index 0000000000000..a3c07bb68d96d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b @@ -0,0 +1,4 @@ +ds=2008-04-08/hr=11 +ds=2008-04-08/hr=12 +ds=2008-12-31/hr=11 +ds=2008-12-31/hr=12 diff --git a/src/test/hive/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db b/src/test/hive/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db new file mode 100644 index 0000000000000..4c0ed5d6fc5f1 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db @@ -0,0 +1,2000 @@ +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/src/test/hive/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f b/src/test/hive/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a b/src/test/hive/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a new file mode 100644 index 0000000000000..267cb634d6bd3 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a @@ -0,0 +1,12 @@ +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 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) diff --git a/src/test/hive/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 b/src/test/hive/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b b/src/test/hive/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 b/src/test/hive/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 new file mode 100644 index 0000000000000..0a88e449f625a --- /dev/null +++ b/src/test/hive/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 @@ -0,0 +1,2 @@ +ds=2008-04-08/hr=11 +ds=2008-04-08/hr=12 diff --git a/src/test/hive/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb b/src/test/hive/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb new file mode 100644 index 0000000000000..d7a8f25b41301 --- /dev/null +++ b/src/test/hive/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb @@ -0,0 +1,1000 @@ +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/src/test/hive/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 b/src/test/hive/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 b/src/test/hive/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e b/src/test/hive/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 b/src/test/hive/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 b/src/test/hive/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f b/src/test/hive/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 b/src/test/hive/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 b/src/test/hive/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 b/src/test/hive/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc b/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 b/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 b/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 b/src/test/hive/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/src/test/hive/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 b/src/test/hive/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 b/src/test/hive/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/src/test/hive/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 b/src/test/hive/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 b/src/test/hive/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c b/src/test/hive/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 b/src/test/hive/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 b/src/test/hive/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 b/src/test/hive/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 new file mode 100644 index 0000000000000..cf367dacbdf0e --- /dev/null +++ b/src/test/hive/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 @@ -0,0 +1,6 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc b/src/test/hive/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 b/src/test/hive/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 b/src/test/hive/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 new file mode 100644 index 0000000000000..cf367dacbdf0e --- /dev/null +++ b/src/test/hive/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 @@ -0,0 +1,6 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 b/src/test/hive/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 b/src/test/hive/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 b/src/test/hive/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 b/src/test/hive/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e b/src/test/hive/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 b/src/test/hive/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 b/src/test/hive/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 b/src/test/hive/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 b/src/test/hive/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 b/src/test/hive/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 b/src/test/hive/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock2-3-27ad2962fed131f51ba802596ba37278 b/src/test/hive/golden/lock2-3-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a b/src/test/hive/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca b/src/test/hive/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock3-0-27ad2962fed131f51ba802596ba37278 b/src/test/hive/golden/lock3-0-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a b/src/test/hive/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 b/src/test/hive/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock3-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/lock3-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/lock3-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac b/src/test/hive/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe b/src/test/hive/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 b/src/test/hive/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/lock4-1-27ad2962fed131f51ba802596ba37278 b/src/test/hive/golden/lock4-1-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a b/src/test/hive/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 b/src/test/hive/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock4-4-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/lock4-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/lock4-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac b/src/test/hive/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe b/src/test/hive/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd b/src/test/hive/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 b/src/test/hive/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 new file mode 100644 index 0000000000000..19492fd335bcb --- /dev/null +++ b/src/test/hive/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 @@ -0,0 +1,12 @@ +17 val_17 17 val_17 +17 val_17 17 val_17 +19 val_19 19 val_19 +19 val_19 19 val_19 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +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 diff --git a/src/test/hive/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 b/src/test/hive/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef b/src/test/hive/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef new file mode 100644 index 0000000000000..3a57720041fb3 --- /dev/null +++ b/src/test/hive/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef @@ -0,0 +1,12 @@ +17 val_17 17 val_17 +19 val_19 19 val_19 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +17 val_17 17 val_17 +19 val_19 19 val_19 +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 diff --git a/src/test/hive/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 b/src/test/hive/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 b/src/test/hive/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 new file mode 100644 index 0000000000000..19492fd335bcb --- /dev/null +++ b/src/test/hive/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 @@ -0,0 +1,12 @@ +17 val_17 17 val_17 +17 val_17 17 val_17 +19 val_19 19 val_19 +19 val_19 19 val_19 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +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 diff --git a/src/test/hive/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 b/src/test/hive/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 b/src/test/hive/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 new file mode 100644 index 0000000000000..3a57720041fb3 --- /dev/null +++ b/src/test/hive/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 @@ -0,0 +1,12 @@ +17 val_17 17 val_17 +19 val_19 19 val_19 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +17 val_17 17 val_17 +19 val_19 19 val_19 +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 diff --git a/src/test/hive/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d b/src/test/hive/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d new file mode 100644 index 0000000000000..657eea30d475f --- /dev/null +++ b/src/test/hive/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d @@ -0,0 +1 @@ +1114788 diff --git a/src/test/hive/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 b/src/test/hive/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 b/src/test/hive/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 new file mode 100644 index 0000000000000..cfae441c4a10a --- /dev/null +++ b/src/test/hive/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 b/src/test/hive/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 b/src/test/hive/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 new file mode 100644 index 0000000000000..6d8155efd76cd --- /dev/null +++ b/src/test/hive/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 b/src/test/hive/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 new file mode 100644 index 0000000000000..44f1acd59de68 --- /dev/null +++ b/src/test/hive/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d b/src/test/hive/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 b/src/test/hive/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 new file mode 100644 index 0000000000000..cfae441c4a10a --- /dev/null +++ b/src/test/hive/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba b/src/test/hive/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 b/src/test/hive/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 new file mode 100644 index 0000000000000..6d8155efd76cd --- /dev/null +++ b/src/test/hive/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 b/src/test/hive/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 b/src/test/hive/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 new file mode 100644 index 0000000000000..44f1acd59de68 --- /dev/null +++ b/src/test/hive/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 b/src/test/hive/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 b/src/test/hive/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 b/src/test/hive/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f b/src/test/hive/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f new file mode 100644 index 0000000000000..eab7fd7a51ea4 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f @@ -0,0 +1,10 @@ +val_0 +val_10 +val_100 +val_103 +val_104 +val_105 +val_11 +val_111 +val_113 +val_114 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 b/src/test/hive/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e b/src/test/hive/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 b/src/test/hive/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f b/src/test/hive/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f new file mode 100644 index 0000000000000..eab7fd7a51ea4 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f @@ -0,0 +1,10 @@ +val_0 +val_10 +val_100 +val_103 +val_104 +val_105 +val_11 +val_111 +val_113 +val_114 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 b/src/test/hive/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f b/src/test/hive/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f new file mode 100644 index 0000000000000..eab7fd7a51ea4 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f @@ -0,0 +1,10 @@ +val_0 +val_10 +val_100 +val_103 +val_104 +val_105 +val_11 +val_111 +val_113 +val_114 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e b/src/test/hive/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 b/src/test/hive/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f b/src/test/hive/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f new file mode 100644 index 0000000000000..eab7fd7a51ea4 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f @@ -0,0 +1,10 @@ +val_0 +val_10 +val_100 +val_103 +val_104 +val_105 +val_11 +val_111 +val_113 +val_114 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 b/src/test/hive/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 b/src/test/hive/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d b/src/test/hive/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 b/src/test/hive/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 b/src/test/hive/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 b/src/test/hive/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 new file mode 100644 index 0000000000000..8e7fe1e4cae08 --- /dev/null +++ b/src/test/hive/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 @@ -0,0 +1,2 @@ +5308 +5308 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b b/src/test/hive/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 b/src/test/hive/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 new file mode 100644 index 0000000000000..482848c1ef8aa --- /dev/null +++ b/src/test/hive/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 @@ -0,0 +1,107 @@ +66 val_66 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +150 val_150 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c b/src/test/hive/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b b/src/test/hive/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b new file mode 100644 index 0000000000000..482848c1ef8aa --- /dev/null +++ b/src/test/hive/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b @@ -0,0 +1,107 @@ +66 val_66 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +150 val_150 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 b/src/test/hive/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 b/src/test/hive/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 b/src/test/hive/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 b/src/test/hive/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 new file mode 100644 index 0000000000000..5c62121a6600c --- /dev/null +++ b/src/test/hive/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 @@ -0,0 +1,2 @@ +2 Joe 2 Tie 2 Tie +2 Hank 2 Tie 2 Tie \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 b/src/test/hive/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 b/src/test/hive/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 b/src/test/hive/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 b/src/test/hive/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 b/src/test/hive/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 b/src/test/hive/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 b/src/test/hive/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 b/src/test/hive/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 b/src/test/hive/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e b/src/test/hive/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d b/src/test/hive/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc b/src/test/hive/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a b/src/test/hive/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 b/src/test/hive/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 new file mode 100644 index 0000000000000..271f3e168fd23 --- /dev/null +++ b/src/test/hive/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 @@ -0,0 +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 diff --git a/src/test/hive/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 b/src/test/hive/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 new file mode 100644 index 0000000000000..271f3e168fd23 --- /dev/null +++ b/src/test/hive/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 @@ -0,0 +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 diff --git a/src/test/hive/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 b/src/test/hive/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 b/src/test/hive/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 new file mode 100644 index 0000000000000..f00666f6113d2 --- /dev/null +++ b/src/test/hive/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 b/src/test/hive/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f b/src/test/hive/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 b/src/test/hive/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 b/src/test/hive/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 b/src/test/hive/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 new file mode 100644 index 0000000000000..f00666f6113d2 --- /dev/null +++ b/src/test/hive/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 b/src/test/hive/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a b/src/test/hive/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a new file mode 100644 index 0000000000000..f00666f6113d2 --- /dev/null +++ b/src/test/hive/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 b/src/test/hive/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 new file mode 100644 index 0000000000000..271f3e168fd23 --- /dev/null +++ b/src/test/hive/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 @@ -0,0 +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 diff --git a/src/test/hive/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c b/src/test/hive/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe b/src/test/hive/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e b/src/test/hive/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..3af52be3cd1fc --- /dev/null +++ b/src/test/hive/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,500 @@ +0 0 0 val_0 +0 0 0 val_0 +0 0 0 val_0 +2 0 2 val_2 +4 0 4 val_4 +5 0 5 val_5 +5 0 5 val_5 +5 0 5 val_5 +8 0 8 val_8 +9 0 9 val_9 +10 1 0 val_10 +11 1 1 val_11 +12 1 2 val_12 +12 1 2 val_12 +15 1 5 val_15 +15 1 5 val_15 +17 1 7 val_17 +18 1 8 val_18 +18 1 8 val_18 +19 1 9 val_19 +100 10 0 val_100 +100 10 0 val_100 +103 10 3 val_103 +103 10 3 val_103 +104 10 4 val_104 +104 10 4 val_104 +105 10 5 val_105 +111 11 1 val_111 +113 11 3 val_113 +113 11 3 val_113 +114 11 4 val_114 +116 11 6 val_116 +118 11 8 val_118 +118 11 8 val_118 +119 11 9 val_119 +119 11 9 val_119 +119 11 9 val_119 +120 12 0 val_120 +120 12 0 val_120 +125 12 5 val_125 +125 12 5 val_125 +126 12 6 val_126 +128 12 8 val_128 +128 12 8 val_128 +128 12 8 val_128 +129 12 9 val_129 +129 12 9 val_129 +131 13 1 val_131 +133 13 3 val_133 +134 13 4 val_134 +134 13 4 val_134 +136 13 6 val_136 +137 13 7 val_137 +137 13 7 val_137 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +143 14 3 val_143 +145 14 5 val_145 +146 14 6 val_146 +146 14 6 val_146 +149 14 9 val_149 +149 14 9 val_149 +150 15 0 val_150 +152 15 2 val_152 +152 15 2 val_152 +153 15 3 val_153 +155 15 5 val_155 +156 15 6 val_156 +157 15 7 val_157 +158 15 8 val_158 +160 16 0 val_160 +162 16 2 val_162 +163 16 3 val_163 +164 16 4 val_164 +164 16 4 val_164 +165 16 5 val_165 +165 16 5 val_165 +166 16 6 val_166 +167 16 7 val_167 +167 16 7 val_167 +167 16 7 val_167 +168 16 8 val_168 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +170 17 0 val_170 +172 17 2 val_172 +172 17 2 val_172 +174 17 4 val_174 +174 17 4 val_174 +175 17 5 val_175 +175 17 5 val_175 +176 17 6 val_176 +176 17 6 val_176 +177 17 7 val_177 +178 17 8 val_178 +179 17 9 val_179 +179 17 9 val_179 +180 18 0 val_180 +181 18 1 val_181 +183 18 3 val_183 +186 18 6 val_186 +187 18 7 val_187 +187 18 7 val_187 +187 18 7 val_187 +189 18 9 val_189 +190 19 0 val_190 +191 19 1 val_191 +191 19 1 val_191 +192 19 2 val_192 +193 19 3 val_193 +193 19 3 val_193 +193 19 3 val_193 +194 19 4 val_194 +195 19 5 val_195 +195 19 5 val_195 +196 19 6 val_196 +197 19 7 val_197 +197 19 7 val_197 +199 19 9 val_199 +199 19 9 val_199 +199 19 9 val_199 +20 2 0 val_20 +24 2 4 val_24 +24 2 4 val_24 +26 2 6 val_26 +26 2 6 val_26 +27 2 7 val_27 +28 2 8 val_28 +200 20 0 val_200 +200 20 0 val_200 +201 20 1 val_201 +202 20 2 val_202 +203 20 3 val_203 +203 20 3 val_203 +205 20 5 val_205 +205 20 5 val_205 +207 20 7 val_207 +207 20 7 val_207 +208 20 8 val_208 +208 20 8 val_208 +208 20 8 val_208 +209 20 9 val_209 +209 20 9 val_209 +213 21 3 val_213 +213 21 3 val_213 +214 21 4 val_214 +216 21 6 val_216 +216 21 6 val_216 +217 21 7 val_217 +217 21 7 val_217 +218 21 8 val_218 +219 21 9 val_219 +219 21 9 val_219 +221 22 1 val_221 +221 22 1 val_221 +222 22 2 val_222 +223 22 3 val_223 +223 22 3 val_223 +224 22 4 val_224 +224 22 4 val_224 +226 22 6 val_226 +228 22 8 val_228 +229 22 9 val_229 +229 22 9 val_229 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +233 23 3 val_233 +233 23 3 val_233 +235 23 5 val_235 +237 23 7 val_237 +237 23 7 val_237 +238 23 8 val_238 +238 23 8 val_238 +239 23 9 val_239 +239 23 9 val_239 +241 24 1 val_241 +242 24 2 val_242 +242 24 2 val_242 +244 24 4 val_244 +247 24 7 val_247 +248 24 8 val_248 +249 24 9 val_249 +252 25 2 val_252 +255 25 5 val_255 +255 25 5 val_255 +256 25 6 val_256 +256 25 6 val_256 +257 25 7 val_257 +258 25 8 val_258 +260 26 0 val_260 +262 26 2 val_262 +263 26 3 val_263 +265 26 5 val_265 +265 26 5 val_265 +266 26 6 val_266 +272 27 2 val_272 +272 27 2 val_272 +273 27 3 val_273 +273 27 3 val_273 +273 27 3 val_273 +274 27 4 val_274 +275 27 5 val_275 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +278 27 8 val_278 +278 27 8 val_278 +280 28 0 val_280 +280 28 0 val_280 +281 28 1 val_281 +281 28 1 val_281 +282 28 2 val_282 +282 28 2 val_282 +283 28 3 val_283 +284 28 4 val_284 +285 28 5 val_285 +286 28 6 val_286 +287 28 7 val_287 +288 28 8 val_288 +288 28 8 val_288 +289 28 9 val_289 +291 29 1 val_291 +292 29 2 val_292 +296 29 6 val_296 +298 29 8 val_298 +298 29 8 val_298 +298 29 8 val_298 +30 3 0 val_30 +33 3 3 val_33 +34 3 4 val_34 +35 3 5 val_35 +35 3 5 val_35 +35 3 5 val_35 +37 3 7 val_37 +37 3 7 val_37 +302 30 2 val_302 +305 30 5 val_305 +306 30 6 val_306 +307 30 7 val_307 +307 30 7 val_307 +308 30 8 val_308 +309 30 9 val_309 +309 30 9 val_309 +310 31 0 val_310 +311 31 1 val_311 +311 31 1 val_311 +311 31 1 val_311 +315 31 5 val_315 +316 31 6 val_316 +316 31 6 val_316 +316 31 6 val_316 +317 31 7 val_317 +317 31 7 val_317 +318 31 8 val_318 +318 31 8 val_318 +318 31 8 val_318 +321 32 1 val_321 +321 32 1 val_321 +322 32 2 val_322 +322 32 2 val_322 +323 32 3 val_323 +325 32 5 val_325 +325 32 5 val_325 +327 32 7 val_327 +327 32 7 val_327 +327 32 7 val_327 +331 33 1 val_331 +331 33 1 val_331 +332 33 2 val_332 +333 33 3 val_333 +333 33 3 val_333 +335 33 5 val_335 +336 33 6 val_336 +338 33 8 val_338 +339 33 9 val_339 +341 34 1 val_341 +342 34 2 val_342 +342 34 2 val_342 +344 34 4 val_344 +344 34 4 val_344 +345 34 5 val_345 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +351 35 1 val_351 +353 35 3 val_353 +353 35 3 val_353 +356 35 6 val_356 +360 36 0 val_360 +362 36 2 val_362 +364 36 4 val_364 +365 36 5 val_365 +366 36 6 val_366 +367 36 7 val_367 +367 36 7 val_367 +368 36 8 val_368 +369 36 9 val_369 +369 36 9 val_369 +369 36 9 val_369 +373 37 3 val_373 +374 37 4 val_374 +375 37 5 val_375 +377 37 7 val_377 +378 37 8 val_378 +379 37 9 val_379 +382 38 2 val_382 +382 38 2 val_382 +384 38 4 val_384 +384 38 4 val_384 +384 38 4 val_384 +386 38 6 val_386 +389 38 9 val_389 +392 39 2 val_392 +393 39 3 val_393 +394 39 4 val_394 +395 39 5 val_395 +395 39 5 val_395 +396 39 6 val_396 +396 39 6 val_396 +396 39 6 val_396 +397 39 7 val_397 +397 39 7 val_397 +399 39 9 val_399 +399 39 9 val_399 +41 4 1 val_41 +42 4 2 val_42 +42 4 2 val_42 +43 4 3 val_43 +44 4 4 val_44 +47 4 7 val_47 +400 40 0 val_400 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +402 40 2 val_402 +403 40 3 val_403 +403 40 3 val_403 +403 40 3 val_403 +404 40 4 val_404 +404 40 4 val_404 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +407 40 7 val_407 +409 40 9 val_409 +409 40 9 val_409 +409 40 9 val_409 +411 41 1 val_411 +413 41 3 val_413 +413 41 3 val_413 +414 41 4 val_414 +414 41 4 val_414 +417 41 7 val_417 +417 41 7 val_417 +417 41 7 val_417 +418 41 8 val_418 +419 41 9 val_419 +421 42 1 val_421 +424 42 4 val_424 +424 42 4 val_424 +427 42 7 val_427 +429 42 9 val_429 +429 42 9 val_429 +430 43 0 val_430 +430 43 0 val_430 +430 43 0 val_430 +431 43 1 val_431 +431 43 1 val_431 +431 43 1 val_431 +432 43 2 val_432 +435 43 5 val_435 +436 43 6 val_436 +437 43 7 val_437 +438 43 8 val_438 +438 43 8 val_438 +438 43 8 val_438 +439 43 9 val_439 +439 43 9 val_439 +443 44 3 val_443 +444 44 4 val_444 +446 44 6 val_446 +448 44 8 val_448 +449 44 9 val_449 +452 45 2 val_452 +453 45 3 val_453 +454 45 4 val_454 +454 45 4 val_454 +454 45 4 val_454 +455 45 5 val_455 +457 45 7 val_457 +458 45 8 val_458 +458 45 8 val_458 +459 45 9 val_459 +459 45 9 val_459 +460 46 0 val_460 +462 46 2 val_462 +462 46 2 val_462 +463 46 3 val_463 +463 46 3 val_463 +466 46 6 val_466 +466 46 6 val_466 +466 46 6 val_466 +467 46 7 val_467 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +470 47 0 val_470 +472 47 2 val_472 +475 47 5 val_475 +477 47 7 val_477 +478 47 8 val_478 +478 47 8 val_478 +479 47 9 val_479 +480 48 0 val_480 +480 48 0 val_480 +480 48 0 val_480 +481 48 1 val_481 +482 48 2 val_482 +483 48 3 val_483 +484 48 4 val_484 +485 48 5 val_485 +487 48 7 val_487 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +490 49 0 val_490 +491 49 1 val_491 +492 49 2 val_492 +492 49 2 val_492 +493 49 3 val_493 +494 49 4 val_494 +495 49 5 val_495 +496 49 6 val_496 +497 49 7 val_497 +498 49 8 val_498 +498 49 8 val_498 +498 49 8 val_498 +51 5 1 val_51 +51 5 1 val_51 +53 5 3 val_53 +54 5 4 val_54 +57 5 7 val_57 +58 5 8 val_58 +58 5 8 val_58 +64 6 4 val_64 +65 6 5 val_65 +66 6 6 val_66 +67 6 7 val_67 +67 6 7 val_67 +69 6 9 val_69 +70 7 0 val_70 +70 7 0 val_70 +70 7 0 val_70 +72 7 2 val_72 +72 7 2 val_72 +74 7 4 val_74 +76 7 6 val_76 +76 7 6 val_76 +77 7 7 val_77 +78 7 8 val_78 +80 8 0 val_80 +82 8 2 val_82 +83 8 3 val_83 +83 8 3 val_83 +84 8 4 val_84 +84 8 4 val_84 +85 8 5 val_85 +86 8 6 val_86 +87 8 7 val_87 +90 9 0 val_90 +90 9 0 val_90 +90 9 0 val_90 +92 9 2 val_92 +95 9 5 val_95 +95 9 5 val_95 +96 9 6 val_96 +97 9 7 val_97 +97 9 7 val_97 +98 9 8 val_98 +98 9 8 val_98 diff --git a/src/test/hive/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c b/src/test/hive/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 b/src/test/hive/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b b/src/test/hive/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc b/src/test/hive/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc new file mode 100644 index 0000000000000..4ab18ace38f46 --- /dev/null +++ b/src/test/hive/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc @@ -0,0 +1,500 @@ +0 0 0 val_0 +0 0 0 val_0 +0 0 0 val_0 +2 0 2 val_2 +4 0 4 val_4 +5 0 5 val_5 +5 0 5 val_5 +5 0 5 val_5 +8 0 8 val_8 +9 0 9 val_9 +10 1 0 val_10 +11 1 1 val_11 +12 1 2 val_12 +12 1 2 val_12 +15 1 5 val_15 +15 1 5 val_15 +17 1 7 val_17 +18 1 8 val_18 +18 1 8 val_18 +19 1 9 val_19 +20 2 0 val_20 +24 2 4 val_24 +24 2 4 val_24 +26 2 6 val_26 +26 2 6 val_26 +27 2 7 val_27 +28 2 8 val_28 +30 3 0 val_30 +33 3 3 val_33 +34 3 4 val_34 +35 3 5 val_35 +35 3 5 val_35 +35 3 5 val_35 +37 3 7 val_37 +37 3 7 val_37 +41 4 1 val_41 +42 4 2 val_42 +42 4 2 val_42 +43 4 3 val_43 +44 4 4 val_44 +47 4 7 val_47 +51 5 1 val_51 +51 5 1 val_51 +53 5 3 val_53 +54 5 4 val_54 +57 5 7 val_57 +58 5 8 val_58 +58 5 8 val_58 +64 6 4 val_64 +65 6 5 val_65 +66 6 6 val_66 +67 6 7 val_67 +67 6 7 val_67 +69 6 9 val_69 +70 7 0 val_70 +70 7 0 val_70 +70 7 0 val_70 +72 7 2 val_72 +72 7 2 val_72 +74 7 4 val_74 +76 7 6 val_76 +76 7 6 val_76 +77 7 7 val_77 +78 7 8 val_78 +80 8 0 val_80 +82 8 2 val_82 +83 8 3 val_83 +83 8 3 val_83 +84 8 4 val_84 +84 8 4 val_84 +85 8 5 val_85 +86 8 6 val_86 +87 8 7 val_87 +90 9 0 val_90 +90 9 0 val_90 +90 9 0 val_90 +92 9 2 val_92 +95 9 5 val_95 +95 9 5 val_95 +96 9 6 val_96 +97 9 7 val_97 +97 9 7 val_97 +98 9 8 val_98 +98 9 8 val_98 +100 10 0 val_100 +100 10 0 val_100 +103 10 3 val_103 +103 10 3 val_103 +104 10 4 val_104 +104 10 4 val_104 +105 10 5 val_105 +111 11 1 val_111 +113 11 3 val_113 +113 11 3 val_113 +114 11 4 val_114 +116 11 6 val_116 +118 11 8 val_118 +118 11 8 val_118 +119 11 9 val_119 +119 11 9 val_119 +119 11 9 val_119 +120 12 0 val_120 +120 12 0 val_120 +125 12 5 val_125 +125 12 5 val_125 +126 12 6 val_126 +128 12 8 val_128 +128 12 8 val_128 +128 12 8 val_128 +129 12 9 val_129 +129 12 9 val_129 +131 13 1 val_131 +133 13 3 val_133 +134 13 4 val_134 +134 13 4 val_134 +136 13 6 val_136 +137 13 7 val_137 +137 13 7 val_137 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +143 14 3 val_143 +145 14 5 val_145 +146 14 6 val_146 +146 14 6 val_146 +149 14 9 val_149 +149 14 9 val_149 +150 15 0 val_150 +152 15 2 val_152 +152 15 2 val_152 +153 15 3 val_153 +155 15 5 val_155 +156 15 6 val_156 +157 15 7 val_157 +158 15 8 val_158 +160 16 0 val_160 +162 16 2 val_162 +163 16 3 val_163 +164 16 4 val_164 +164 16 4 val_164 +165 16 5 val_165 +165 16 5 val_165 +166 16 6 val_166 +167 16 7 val_167 +167 16 7 val_167 +167 16 7 val_167 +168 16 8 val_168 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +170 17 0 val_170 +172 17 2 val_172 +172 17 2 val_172 +174 17 4 val_174 +174 17 4 val_174 +175 17 5 val_175 +175 17 5 val_175 +176 17 6 val_176 +176 17 6 val_176 +177 17 7 val_177 +178 17 8 val_178 +179 17 9 val_179 +179 17 9 val_179 +180 18 0 val_180 +181 18 1 val_181 +183 18 3 val_183 +186 18 6 val_186 +187 18 7 val_187 +187 18 7 val_187 +187 18 7 val_187 +189 18 9 val_189 +190 19 0 val_190 +191 19 1 val_191 +191 19 1 val_191 +192 19 2 val_192 +193 19 3 val_193 +193 19 3 val_193 +193 19 3 val_193 +194 19 4 val_194 +195 19 5 val_195 +195 19 5 val_195 +196 19 6 val_196 +197 19 7 val_197 +197 19 7 val_197 +199 19 9 val_199 +199 19 9 val_199 +199 19 9 val_199 +200 20 0 val_200 +200 20 0 val_200 +201 20 1 val_201 +202 20 2 val_202 +203 20 3 val_203 +203 20 3 val_203 +205 20 5 val_205 +205 20 5 val_205 +207 20 7 val_207 +207 20 7 val_207 +208 20 8 val_208 +208 20 8 val_208 +208 20 8 val_208 +209 20 9 val_209 +209 20 9 val_209 +213 21 3 val_213 +213 21 3 val_213 +214 21 4 val_214 +216 21 6 val_216 +216 21 6 val_216 +217 21 7 val_217 +217 21 7 val_217 +218 21 8 val_218 +219 21 9 val_219 +219 21 9 val_219 +221 22 1 val_221 +221 22 1 val_221 +222 22 2 val_222 +223 22 3 val_223 +223 22 3 val_223 +224 22 4 val_224 +224 22 4 val_224 +226 22 6 val_226 +228 22 8 val_228 +229 22 9 val_229 +229 22 9 val_229 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +233 23 3 val_233 +233 23 3 val_233 +235 23 5 val_235 +237 23 7 val_237 +237 23 7 val_237 +238 23 8 val_238 +238 23 8 val_238 +239 23 9 val_239 +239 23 9 val_239 +241 24 1 val_241 +242 24 2 val_242 +242 24 2 val_242 +244 24 4 val_244 +247 24 7 val_247 +248 24 8 val_248 +249 24 9 val_249 +252 25 2 val_252 +255 25 5 val_255 +255 25 5 val_255 +256 25 6 val_256 +256 25 6 val_256 +257 25 7 val_257 +258 25 8 val_258 +260 26 0 val_260 +262 26 2 val_262 +263 26 3 val_263 +265 26 5 val_265 +265 26 5 val_265 +266 26 6 val_266 +272 27 2 val_272 +272 27 2 val_272 +273 27 3 val_273 +273 27 3 val_273 +273 27 3 val_273 +274 27 4 val_274 +275 27 5 val_275 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +278 27 8 val_278 +278 27 8 val_278 +280 28 0 val_280 +280 28 0 val_280 +281 28 1 val_281 +281 28 1 val_281 +282 28 2 val_282 +282 28 2 val_282 +283 28 3 val_283 +284 28 4 val_284 +285 28 5 val_285 +286 28 6 val_286 +287 28 7 val_287 +288 28 8 val_288 +288 28 8 val_288 +289 28 9 val_289 +291 29 1 val_291 +292 29 2 val_292 +296 29 6 val_296 +298 29 8 val_298 +298 29 8 val_298 +298 29 8 val_298 +302 30 2 val_302 +305 30 5 val_305 +306 30 6 val_306 +307 30 7 val_307 +307 30 7 val_307 +308 30 8 val_308 +309 30 9 val_309 +309 30 9 val_309 +310 31 0 val_310 +311 31 1 val_311 +311 31 1 val_311 +311 31 1 val_311 +315 31 5 val_315 +316 31 6 val_316 +316 31 6 val_316 +316 31 6 val_316 +317 31 7 val_317 +317 31 7 val_317 +318 31 8 val_318 +318 31 8 val_318 +318 31 8 val_318 +321 32 1 val_321 +321 32 1 val_321 +322 32 2 val_322 +322 32 2 val_322 +323 32 3 val_323 +325 32 5 val_325 +325 32 5 val_325 +327 32 7 val_327 +327 32 7 val_327 +327 32 7 val_327 +331 33 1 val_331 +331 33 1 val_331 +332 33 2 val_332 +333 33 3 val_333 +333 33 3 val_333 +335 33 5 val_335 +336 33 6 val_336 +338 33 8 val_338 +339 33 9 val_339 +341 34 1 val_341 +342 34 2 val_342 +342 34 2 val_342 +344 34 4 val_344 +344 34 4 val_344 +345 34 5 val_345 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +351 35 1 val_351 +353 35 3 val_353 +353 35 3 val_353 +356 35 6 val_356 +360 36 0 val_360 +362 36 2 val_362 +364 36 4 val_364 +365 36 5 val_365 +366 36 6 val_366 +367 36 7 val_367 +367 36 7 val_367 +368 36 8 val_368 +369 36 9 val_369 +369 36 9 val_369 +369 36 9 val_369 +373 37 3 val_373 +374 37 4 val_374 +375 37 5 val_375 +377 37 7 val_377 +378 37 8 val_378 +379 37 9 val_379 +382 38 2 val_382 +382 38 2 val_382 +384 38 4 val_384 +384 38 4 val_384 +384 38 4 val_384 +386 38 6 val_386 +389 38 9 val_389 +392 39 2 val_392 +393 39 3 val_393 +394 39 4 val_394 +395 39 5 val_395 +395 39 5 val_395 +396 39 6 val_396 +396 39 6 val_396 +396 39 6 val_396 +397 39 7 val_397 +397 39 7 val_397 +399 39 9 val_399 +399 39 9 val_399 +400 40 0 val_400 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +402 40 2 val_402 +403 40 3 val_403 +403 40 3 val_403 +403 40 3 val_403 +404 40 4 val_404 +404 40 4 val_404 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +407 40 7 val_407 +409 40 9 val_409 +409 40 9 val_409 +409 40 9 val_409 +411 41 1 val_411 +413 41 3 val_413 +413 41 3 val_413 +414 41 4 val_414 +414 41 4 val_414 +417 41 7 val_417 +417 41 7 val_417 +417 41 7 val_417 +418 41 8 val_418 +419 41 9 val_419 +421 42 1 val_421 +424 42 4 val_424 +424 42 4 val_424 +427 42 7 val_427 +429 42 9 val_429 +429 42 9 val_429 +430 43 0 val_430 +430 43 0 val_430 +430 43 0 val_430 +431 43 1 val_431 +431 43 1 val_431 +431 43 1 val_431 +432 43 2 val_432 +435 43 5 val_435 +436 43 6 val_436 +437 43 7 val_437 +438 43 8 val_438 +438 43 8 val_438 +438 43 8 val_438 +439 43 9 val_439 +439 43 9 val_439 +443 44 3 val_443 +444 44 4 val_444 +446 44 6 val_446 +448 44 8 val_448 +449 44 9 val_449 +452 45 2 val_452 +453 45 3 val_453 +454 45 4 val_454 +454 45 4 val_454 +454 45 4 val_454 +455 45 5 val_455 +457 45 7 val_457 +458 45 8 val_458 +458 45 8 val_458 +459 45 9 val_459 +459 45 9 val_459 +460 46 0 val_460 +462 46 2 val_462 +462 46 2 val_462 +463 46 3 val_463 +463 46 3 val_463 +466 46 6 val_466 +466 46 6 val_466 +466 46 6 val_466 +467 46 7 val_467 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +470 47 0 val_470 +472 47 2 val_472 +475 47 5 val_475 +477 47 7 val_477 +478 47 8 val_478 +478 47 8 val_478 +479 47 9 val_479 +480 48 0 val_480 +480 48 0 val_480 +480 48 0 val_480 +481 48 1 val_481 +482 48 2 val_482 +483 48 3 val_483 +484 48 4 val_484 +485 48 5 val_485 +487 48 7 val_487 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +490 49 0 val_490 +491 49 1 val_491 +492 49 2 val_492 +492 49 2 val_492 +493 49 3 val_493 +494 49 4 val_494 +495 49 5 val_495 +496 49 6 val_496 +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 diff --git a/src/test/hive/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c b/src/test/hive/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b b/src/test/hive/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a b/src/test/hive/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..1504c19f78eb2 --- /dev/null +++ b/src/test/hive/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,500 @@ +0 0 0 val_0 +0 0 0 val_0 +0 0 0 val_0 +10 1 0 val_10 +100 10 0 val_100 +100 10 0 val_100 +103 10 3 val_103 +103 10 3 val_103 +104 10 4 val_104 +104 10 4 val_104 +105 10 5 val_105 +11 1 1 val_11 +111 11 1 val_111 +113 11 3 val_113 +113 11 3 val_113 +114 11 4 val_114 +116 11 6 val_116 +118 11 8 val_118 +118 11 8 val_118 +119 11 9 val_119 +119 11 9 val_119 +119 11 9 val_119 +12 1 2 val_12 +12 1 2 val_12 +120 12 0 val_120 +120 12 0 val_120 +125 12 5 val_125 +125 12 5 val_125 +126 12 6 val_126 +128 12 8 val_128 +128 12 8 val_128 +128 12 8 val_128 +129 12 9 val_129 +129 12 9 val_129 +131 13 1 val_131 +133 13 3 val_133 +134 13 4 val_134 +134 13 4 val_134 +136 13 6 val_136 +137 13 7 val_137 +137 13 7 val_137 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +143 14 3 val_143 +145 14 5 val_145 +146 14 6 val_146 +146 14 6 val_146 +149 14 9 val_149 +149 14 9 val_149 +15 1 5 val_15 +15 1 5 val_15 +150 15 0 val_150 +152 15 2 val_152 +152 15 2 val_152 +153 15 3 val_153 +155 15 5 val_155 +156 15 6 val_156 +157 15 7 val_157 +158 15 8 val_158 +160 16 0 val_160 +162 16 2 val_162 +163 16 3 val_163 +164 16 4 val_164 +164 16 4 val_164 +165 16 5 val_165 +165 16 5 val_165 +166 16 6 val_166 +167 16 7 val_167 +167 16 7 val_167 +167 16 7 val_167 +168 16 8 val_168 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +17 1 7 val_17 +170 17 0 val_170 +172 17 2 val_172 +172 17 2 val_172 +174 17 4 val_174 +174 17 4 val_174 +175 17 5 val_175 +175 17 5 val_175 +176 17 6 val_176 +176 17 6 val_176 +177 17 7 val_177 +178 17 8 val_178 +179 17 9 val_179 +179 17 9 val_179 +18 1 8 val_18 +18 1 8 val_18 +180 18 0 val_180 +181 18 1 val_181 +183 18 3 val_183 +186 18 6 val_186 +187 18 7 val_187 +187 18 7 val_187 +187 18 7 val_187 +189 18 9 val_189 +19 1 9 val_19 +190 19 0 val_190 +191 19 1 val_191 +191 19 1 val_191 +192 19 2 val_192 +193 19 3 val_193 +193 19 3 val_193 +193 19 3 val_193 +194 19 4 val_194 +195 19 5 val_195 +195 19 5 val_195 +196 19 6 val_196 +197 19 7 val_197 +197 19 7 val_197 +199 19 9 val_199 +199 19 9 val_199 +199 19 9 val_199 +2 0 2 val_2 +20 2 0 val_20 +200 20 0 val_200 +200 20 0 val_200 +201 20 1 val_201 +202 20 2 val_202 +203 20 3 val_203 +203 20 3 val_203 +205 20 5 val_205 +205 20 5 val_205 +207 20 7 val_207 +207 20 7 val_207 +208 20 8 val_208 +208 20 8 val_208 +208 20 8 val_208 +209 20 9 val_209 +209 20 9 val_209 +213 21 3 val_213 +213 21 3 val_213 +214 21 4 val_214 +216 21 6 val_216 +216 21 6 val_216 +217 21 7 val_217 +217 21 7 val_217 +218 21 8 val_218 +219 21 9 val_219 +219 21 9 val_219 +221 22 1 val_221 +221 22 1 val_221 +222 22 2 val_222 +223 22 3 val_223 +223 22 3 val_223 +224 22 4 val_224 +224 22 4 val_224 +226 22 6 val_226 +228 22 8 val_228 +229 22 9 val_229 +229 22 9 val_229 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +233 23 3 val_233 +233 23 3 val_233 +235 23 5 val_235 +237 23 7 val_237 +237 23 7 val_237 +238 23 8 val_238 +238 23 8 val_238 +239 23 9 val_239 +239 23 9 val_239 +24 2 4 val_24 +24 2 4 val_24 +241 24 1 val_241 +242 24 2 val_242 +242 24 2 val_242 +244 24 4 val_244 +247 24 7 val_247 +248 24 8 val_248 +249 24 9 val_249 +252 25 2 val_252 +255 25 5 val_255 +255 25 5 val_255 +256 25 6 val_256 +256 25 6 val_256 +257 25 7 val_257 +258 25 8 val_258 +26 2 6 val_26 +26 2 6 val_26 +260 26 0 val_260 +262 26 2 val_262 +263 26 3 val_263 +265 26 5 val_265 +265 26 5 val_265 +266 26 6 val_266 +27 2 7 val_27 +272 27 2 val_272 +272 27 2 val_272 +273 27 3 val_273 +273 27 3 val_273 +273 27 3 val_273 +274 27 4 val_274 +275 27 5 val_275 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +278 27 8 val_278 +278 27 8 val_278 +28 2 8 val_28 +280 28 0 val_280 +280 28 0 val_280 +281 28 1 val_281 +281 28 1 val_281 +282 28 2 val_282 +282 28 2 val_282 +283 28 3 val_283 +284 28 4 val_284 +285 28 5 val_285 +286 28 6 val_286 +287 28 7 val_287 +288 28 8 val_288 +288 28 8 val_288 +289 28 9 val_289 +291 29 1 val_291 +292 29 2 val_292 +296 29 6 val_296 +298 29 8 val_298 +298 29 8 val_298 +298 29 8 val_298 +30 3 0 val_30 +302 30 2 val_302 +305 30 5 val_305 +306 30 6 val_306 +307 30 7 val_307 +307 30 7 val_307 +308 30 8 val_308 +309 30 9 val_309 +309 30 9 val_309 +310 31 0 val_310 +311 31 1 val_311 +311 31 1 val_311 +311 31 1 val_311 +315 31 5 val_315 +316 31 6 val_316 +316 31 6 val_316 +316 31 6 val_316 +317 31 7 val_317 +317 31 7 val_317 +318 31 8 val_318 +318 31 8 val_318 +318 31 8 val_318 +321 32 1 val_321 +321 32 1 val_321 +322 32 2 val_322 +322 32 2 val_322 +323 32 3 val_323 +325 32 5 val_325 +325 32 5 val_325 +327 32 7 val_327 +327 32 7 val_327 +327 32 7 val_327 +33 3 3 val_33 +331 33 1 val_331 +331 33 1 val_331 +332 33 2 val_332 +333 33 3 val_333 +333 33 3 val_333 +335 33 5 val_335 +336 33 6 val_336 +338 33 8 val_338 +339 33 9 val_339 +34 3 4 val_34 +341 34 1 val_341 +342 34 2 val_342 +342 34 2 val_342 +344 34 4 val_344 +344 34 4 val_344 +345 34 5 val_345 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +35 3 5 val_35 +35 3 5 val_35 +35 3 5 val_35 +351 35 1 val_351 +353 35 3 val_353 +353 35 3 val_353 +356 35 6 val_356 +360 36 0 val_360 +362 36 2 val_362 +364 36 4 val_364 +365 36 5 val_365 +366 36 6 val_366 +367 36 7 val_367 +367 36 7 val_367 +368 36 8 val_368 +369 36 9 val_369 +369 36 9 val_369 +369 36 9 val_369 +37 3 7 val_37 +37 3 7 val_37 +373 37 3 val_373 +374 37 4 val_374 +375 37 5 val_375 +377 37 7 val_377 +378 37 8 val_378 +379 37 9 val_379 +382 38 2 val_382 +382 38 2 val_382 +384 38 4 val_384 +384 38 4 val_384 +384 38 4 val_384 +386 38 6 val_386 +389 38 9 val_389 +392 39 2 val_392 +393 39 3 val_393 +394 39 4 val_394 +395 39 5 val_395 +395 39 5 val_395 +396 39 6 val_396 +396 39 6 val_396 +396 39 6 val_396 +397 39 7 val_397 +397 39 7 val_397 +399 39 9 val_399 +399 39 9 val_399 +4 0 4 val_4 +400 40 0 val_400 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +402 40 2 val_402 +403 40 3 val_403 +403 40 3 val_403 +403 40 3 val_403 +404 40 4 val_404 +404 40 4 val_404 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +407 40 7 val_407 +409 40 9 val_409 +409 40 9 val_409 +409 40 9 val_409 +41 4 1 val_41 +411 41 1 val_411 +413 41 3 val_413 +413 41 3 val_413 +414 41 4 val_414 +414 41 4 val_414 +417 41 7 val_417 +417 41 7 val_417 +417 41 7 val_417 +418 41 8 val_418 +419 41 9 val_419 +42 4 2 val_42 +42 4 2 val_42 +421 42 1 val_421 +424 42 4 val_424 +424 42 4 val_424 +427 42 7 val_427 +429 42 9 val_429 +429 42 9 val_429 +43 4 3 val_43 +430 43 0 val_430 +430 43 0 val_430 +430 43 0 val_430 +431 43 1 val_431 +431 43 1 val_431 +431 43 1 val_431 +432 43 2 val_432 +435 43 5 val_435 +436 43 6 val_436 +437 43 7 val_437 +438 43 8 val_438 +438 43 8 val_438 +438 43 8 val_438 +439 43 9 val_439 +439 43 9 val_439 +44 4 4 val_44 +443 44 3 val_443 +444 44 4 val_444 +446 44 6 val_446 +448 44 8 val_448 +449 44 9 val_449 +452 45 2 val_452 +453 45 3 val_453 +454 45 4 val_454 +454 45 4 val_454 +454 45 4 val_454 +455 45 5 val_455 +457 45 7 val_457 +458 45 8 val_458 +458 45 8 val_458 +459 45 9 val_459 +459 45 9 val_459 +460 46 0 val_460 +462 46 2 val_462 +462 46 2 val_462 +463 46 3 val_463 +463 46 3 val_463 +466 46 6 val_466 +466 46 6 val_466 +466 46 6 val_466 +467 46 7 val_467 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +47 4 7 val_47 +470 47 0 val_470 +472 47 2 val_472 +475 47 5 val_475 +477 47 7 val_477 +478 47 8 val_478 +478 47 8 val_478 +479 47 9 val_479 +480 48 0 val_480 +480 48 0 val_480 +480 48 0 val_480 +481 48 1 val_481 +482 48 2 val_482 +483 48 3 val_483 +484 48 4 val_484 +485 48 5 val_485 +487 48 7 val_487 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +490 49 0 val_490 +491 49 1 val_491 +492 49 2 val_492 +492 49 2 val_492 +493 49 3 val_493 +494 49 4 val_494 +495 49 5 val_495 +496 49 6 val_496 +497 49 7 val_497 +498 49 8 val_498 +498 49 8 val_498 +498 49 8 val_498 +5 0 5 val_5 +5 0 5 val_5 +5 0 5 val_5 +51 5 1 val_51 +51 5 1 val_51 +53 5 3 val_53 +54 5 4 val_54 +57 5 7 val_57 +58 5 8 val_58 +58 5 8 val_58 +64 6 4 val_64 +65 6 5 val_65 +66 6 6 val_66 +67 6 7 val_67 +67 6 7 val_67 +69 6 9 val_69 +70 7 0 val_70 +70 7 0 val_70 +70 7 0 val_70 +72 7 2 val_72 +72 7 2 val_72 +74 7 4 val_74 +76 7 6 val_76 +76 7 6 val_76 +77 7 7 val_77 +78 7 8 val_78 +8 0 8 val_8 +80 8 0 val_80 +82 8 2 val_82 +83 8 3 val_83 +83 8 3 val_83 +84 8 4 val_84 +84 8 4 val_84 +85 8 5 val_85 +86 8 6 val_86 +87 8 7 val_87 +9 0 9 val_9 +90 9 0 val_90 +90 9 0 val_90 +90 9 0 val_90 +92 9 2 val_92 +95 9 5 val_95 +95 9 5 val_95 +96 9 6 val_96 +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 diff --git a/src/test/hive/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c b/src/test/hive/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f b/src/test/hive/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab b/src/test/hive/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..bf6c49af3ba4e --- /dev/null +++ b/src/test/hive/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,500 @@ +90 9 0 val_90 +90 9 0 val_90 +90 9 0 val_90 +92 9 2 val_92 +95 9 5 val_95 +95 9 5 val_95 +96 9 6 val_96 +97 9 7 val_97 +97 9 7 val_97 +98 9 8 val_98 +98 9 8 val_98 +80 8 0 val_80 +82 8 2 val_82 +83 8 3 val_83 +83 8 3 val_83 +84 8 4 val_84 +84 8 4 val_84 +85 8 5 val_85 +86 8 6 val_86 +87 8 7 val_87 +70 7 0 val_70 +70 7 0 val_70 +70 7 0 val_70 +72 7 2 val_72 +72 7 2 val_72 +74 7 4 val_74 +76 7 6 val_76 +76 7 6 val_76 +77 7 7 val_77 +78 7 8 val_78 +64 6 4 val_64 +65 6 5 val_65 +66 6 6 val_66 +67 6 7 val_67 +67 6 7 val_67 +69 6 9 val_69 +51 5 1 val_51 +51 5 1 val_51 +53 5 3 val_53 +54 5 4 val_54 +57 5 7 val_57 +58 5 8 val_58 +58 5 8 val_58 +490 49 0 val_490 +491 49 1 val_491 +492 49 2 val_492 +492 49 2 val_492 +493 49 3 val_493 +494 49 4 val_494 +495 49 5 val_495 +496 49 6 val_496 +497 49 7 val_497 +498 49 8 val_498 +498 49 8 val_498 +498 49 8 val_498 +480 48 0 val_480 +480 48 0 val_480 +480 48 0 val_480 +481 48 1 val_481 +482 48 2 val_482 +483 48 3 val_483 +484 48 4 val_484 +485 48 5 val_485 +487 48 7 val_487 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +470 47 0 val_470 +472 47 2 val_472 +475 47 5 val_475 +477 47 7 val_477 +478 47 8 val_478 +478 47 8 val_478 +479 47 9 val_479 +460 46 0 val_460 +462 46 2 val_462 +462 46 2 val_462 +463 46 3 val_463 +463 46 3 val_463 +466 46 6 val_466 +466 46 6 val_466 +466 46 6 val_466 +467 46 7 val_467 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +452 45 2 val_452 +453 45 3 val_453 +454 45 4 val_454 +454 45 4 val_454 +454 45 4 val_454 +455 45 5 val_455 +457 45 7 val_457 +458 45 8 val_458 +458 45 8 val_458 +459 45 9 val_459 +459 45 9 val_459 +443 44 3 val_443 +444 44 4 val_444 +446 44 6 val_446 +448 44 8 val_448 +449 44 9 val_449 +430 43 0 val_430 +430 43 0 val_430 +430 43 0 val_430 +431 43 1 val_431 +431 43 1 val_431 +431 43 1 val_431 +432 43 2 val_432 +435 43 5 val_435 +436 43 6 val_436 +437 43 7 val_437 +438 43 8 val_438 +438 43 8 val_438 +438 43 8 val_438 +439 43 9 val_439 +439 43 9 val_439 +421 42 1 val_421 +424 42 4 val_424 +424 42 4 val_424 +427 42 7 val_427 +429 42 9 val_429 +429 42 9 val_429 +411 41 1 val_411 +413 41 3 val_413 +413 41 3 val_413 +414 41 4 val_414 +414 41 4 val_414 +417 41 7 val_417 +417 41 7 val_417 +417 41 7 val_417 +418 41 8 val_418 +419 41 9 val_419 +400 40 0 val_400 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +402 40 2 val_402 +403 40 3 val_403 +403 40 3 val_403 +403 40 3 val_403 +404 40 4 val_404 +404 40 4 val_404 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +407 40 7 val_407 +409 40 9 val_409 +409 40 9 val_409 +409 40 9 val_409 +41 4 1 val_41 +42 4 2 val_42 +42 4 2 val_42 +43 4 3 val_43 +44 4 4 val_44 +47 4 7 val_47 +392 39 2 val_392 +393 39 3 val_393 +394 39 4 val_394 +395 39 5 val_395 +395 39 5 val_395 +396 39 6 val_396 +396 39 6 val_396 +396 39 6 val_396 +397 39 7 val_397 +397 39 7 val_397 +399 39 9 val_399 +399 39 9 val_399 +382 38 2 val_382 +382 38 2 val_382 +384 38 4 val_384 +384 38 4 val_384 +384 38 4 val_384 +386 38 6 val_386 +389 38 9 val_389 +373 37 3 val_373 +374 37 4 val_374 +375 37 5 val_375 +377 37 7 val_377 +378 37 8 val_378 +379 37 9 val_379 +360 36 0 val_360 +362 36 2 val_362 +364 36 4 val_364 +365 36 5 val_365 +366 36 6 val_366 +367 36 7 val_367 +367 36 7 val_367 +368 36 8 val_368 +369 36 9 val_369 +369 36 9 val_369 +369 36 9 val_369 +351 35 1 val_351 +353 35 3 val_353 +353 35 3 val_353 +356 35 6 val_356 +341 34 1 val_341 +342 34 2 val_342 +342 34 2 val_342 +344 34 4 val_344 +344 34 4 val_344 +345 34 5 val_345 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +331 33 1 val_331 +331 33 1 val_331 +332 33 2 val_332 +333 33 3 val_333 +333 33 3 val_333 +335 33 5 val_335 +336 33 6 val_336 +338 33 8 val_338 +339 33 9 val_339 +321 32 1 val_321 +321 32 1 val_321 +322 32 2 val_322 +322 32 2 val_322 +323 32 3 val_323 +325 32 5 val_325 +325 32 5 val_325 +327 32 7 val_327 +327 32 7 val_327 +327 32 7 val_327 +310 31 0 val_310 +311 31 1 val_311 +311 31 1 val_311 +311 31 1 val_311 +315 31 5 val_315 +316 31 6 val_316 +316 31 6 val_316 +316 31 6 val_316 +317 31 7 val_317 +317 31 7 val_317 +318 31 8 val_318 +318 31 8 val_318 +318 31 8 val_318 +302 30 2 val_302 +305 30 5 val_305 +306 30 6 val_306 +307 30 7 val_307 +307 30 7 val_307 +308 30 8 val_308 +309 30 9 val_309 +309 30 9 val_309 +30 3 0 val_30 +33 3 3 val_33 +34 3 4 val_34 +35 3 5 val_35 +35 3 5 val_35 +35 3 5 val_35 +37 3 7 val_37 +37 3 7 val_37 +291 29 1 val_291 +292 29 2 val_292 +296 29 6 val_296 +298 29 8 val_298 +298 29 8 val_298 +298 29 8 val_298 +280 28 0 val_280 +280 28 0 val_280 +281 28 1 val_281 +281 28 1 val_281 +282 28 2 val_282 +282 28 2 val_282 +283 28 3 val_283 +284 28 4 val_284 +285 28 5 val_285 +286 28 6 val_286 +287 28 7 val_287 +288 28 8 val_288 +288 28 8 val_288 +289 28 9 val_289 +272 27 2 val_272 +272 27 2 val_272 +273 27 3 val_273 +273 27 3 val_273 +273 27 3 val_273 +274 27 4 val_274 +275 27 5 val_275 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +278 27 8 val_278 +278 27 8 val_278 +260 26 0 val_260 +262 26 2 val_262 +263 26 3 val_263 +265 26 5 val_265 +265 26 5 val_265 +266 26 6 val_266 +252 25 2 val_252 +255 25 5 val_255 +255 25 5 val_255 +256 25 6 val_256 +256 25 6 val_256 +257 25 7 val_257 +258 25 8 val_258 +241 24 1 val_241 +242 24 2 val_242 +242 24 2 val_242 +244 24 4 val_244 +247 24 7 val_247 +248 24 8 val_248 +249 24 9 val_249 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +233 23 3 val_233 +233 23 3 val_233 +235 23 5 val_235 +237 23 7 val_237 +237 23 7 val_237 +238 23 8 val_238 +238 23 8 val_238 +239 23 9 val_239 +239 23 9 val_239 +221 22 1 val_221 +221 22 1 val_221 +222 22 2 val_222 +223 22 3 val_223 +223 22 3 val_223 +224 22 4 val_224 +224 22 4 val_224 +226 22 6 val_226 +228 22 8 val_228 +229 22 9 val_229 +229 22 9 val_229 +213 21 3 val_213 +213 21 3 val_213 +214 21 4 val_214 +216 21 6 val_216 +216 21 6 val_216 +217 21 7 val_217 +217 21 7 val_217 +218 21 8 val_218 +219 21 9 val_219 +219 21 9 val_219 +200 20 0 val_200 +200 20 0 val_200 +201 20 1 val_201 +202 20 2 val_202 +203 20 3 val_203 +203 20 3 val_203 +205 20 5 val_205 +205 20 5 val_205 +207 20 7 val_207 +207 20 7 val_207 +208 20 8 val_208 +208 20 8 val_208 +208 20 8 val_208 +209 20 9 val_209 +209 20 9 val_209 +20 2 0 val_20 +24 2 4 val_24 +24 2 4 val_24 +26 2 6 val_26 +26 2 6 val_26 +27 2 7 val_27 +28 2 8 val_28 +190 19 0 val_190 +191 19 1 val_191 +191 19 1 val_191 +192 19 2 val_192 +193 19 3 val_193 +193 19 3 val_193 +193 19 3 val_193 +194 19 4 val_194 +195 19 5 val_195 +195 19 5 val_195 +196 19 6 val_196 +197 19 7 val_197 +197 19 7 val_197 +199 19 9 val_199 +199 19 9 val_199 +199 19 9 val_199 +180 18 0 val_180 +181 18 1 val_181 +183 18 3 val_183 +186 18 6 val_186 +187 18 7 val_187 +187 18 7 val_187 +187 18 7 val_187 +189 18 9 val_189 +170 17 0 val_170 +172 17 2 val_172 +172 17 2 val_172 +174 17 4 val_174 +174 17 4 val_174 +175 17 5 val_175 +175 17 5 val_175 +176 17 6 val_176 +176 17 6 val_176 +177 17 7 val_177 +178 17 8 val_178 +179 17 9 val_179 +179 17 9 val_179 +160 16 0 val_160 +162 16 2 val_162 +163 16 3 val_163 +164 16 4 val_164 +164 16 4 val_164 +165 16 5 val_165 +165 16 5 val_165 +166 16 6 val_166 +167 16 7 val_167 +167 16 7 val_167 +167 16 7 val_167 +168 16 8 val_168 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +150 15 0 val_150 +152 15 2 val_152 +152 15 2 val_152 +153 15 3 val_153 +155 15 5 val_155 +156 15 6 val_156 +157 15 7 val_157 +158 15 8 val_158 +143 14 3 val_143 +145 14 5 val_145 +146 14 6 val_146 +146 14 6 val_146 +149 14 9 val_149 +149 14 9 val_149 +131 13 1 val_131 +133 13 3 val_133 +134 13 4 val_134 +134 13 4 val_134 +136 13 6 val_136 +137 13 7 val_137 +137 13 7 val_137 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +120 12 0 val_120 +120 12 0 val_120 +125 12 5 val_125 +125 12 5 val_125 +126 12 6 val_126 +128 12 8 val_128 +128 12 8 val_128 +128 12 8 val_128 +129 12 9 val_129 +129 12 9 val_129 +111 11 1 val_111 +113 11 3 val_113 +113 11 3 val_113 +114 11 4 val_114 +116 11 6 val_116 +118 11 8 val_118 +118 11 8 val_118 +119 11 9 val_119 +119 11 9 val_119 +119 11 9 val_119 +100 10 0 val_100 +100 10 0 val_100 +103 10 3 val_103 +103 10 3 val_103 +104 10 4 val_104 +104 10 4 val_104 +105 10 5 val_105 +10 1 0 val_10 +11 1 1 val_11 +12 1 2 val_12 +12 1 2 val_12 +15 1 5 val_15 +15 1 5 val_15 +17 1 7 val_17 +18 1 8 val_18 +18 1 8 val_18 +19 1 9 val_19 +0 0 0 val_0 +0 0 0 val_0 +0 0 0 val_0 +2 0 2 val_2 +4 0 4 val_4 +5 0 5 val_5 +5 0 5 val_5 +5 0 5 val_5 +8 0 8 val_8 +9 0 9 val_9 diff --git a/src/test/hive/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c b/src/test/hive/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 b/src/test/hive/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca b/src/test/hive/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..1ecd5ad908e74 --- /dev/null +++ b/src/test/hive/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,500 @@ +490 49 0 val_490 +491 49 1 val_491 +492 49 2 val_492 +492 49 2 val_492 +493 49 3 val_493 +494 49 4 val_494 +495 49 5 val_495 +496 49 6 val_496 +497 49 7 val_497 +498 49 8 val_498 +498 49 8 val_498 +498 49 8 val_498 +480 48 0 val_480 +480 48 0 val_480 +480 48 0 val_480 +481 48 1 val_481 +482 48 2 val_482 +483 48 3 val_483 +484 48 4 val_484 +485 48 5 val_485 +487 48 7 val_487 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +470 47 0 val_470 +472 47 2 val_472 +475 47 5 val_475 +477 47 7 val_477 +478 47 8 val_478 +478 47 8 val_478 +479 47 9 val_479 +460 46 0 val_460 +462 46 2 val_462 +462 46 2 val_462 +463 46 3 val_463 +463 46 3 val_463 +466 46 6 val_466 +466 46 6 val_466 +466 46 6 val_466 +467 46 7 val_467 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +452 45 2 val_452 +453 45 3 val_453 +454 45 4 val_454 +454 45 4 val_454 +454 45 4 val_454 +455 45 5 val_455 +457 45 7 val_457 +458 45 8 val_458 +458 45 8 val_458 +459 45 9 val_459 +459 45 9 val_459 +443 44 3 val_443 +444 44 4 val_444 +446 44 6 val_446 +448 44 8 val_448 +449 44 9 val_449 +430 43 0 val_430 +430 43 0 val_430 +430 43 0 val_430 +431 43 1 val_431 +431 43 1 val_431 +431 43 1 val_431 +432 43 2 val_432 +435 43 5 val_435 +436 43 6 val_436 +437 43 7 val_437 +438 43 8 val_438 +438 43 8 val_438 +438 43 8 val_438 +439 43 9 val_439 +439 43 9 val_439 +421 42 1 val_421 +424 42 4 val_424 +424 42 4 val_424 +427 42 7 val_427 +429 42 9 val_429 +429 42 9 val_429 +411 41 1 val_411 +413 41 3 val_413 +413 41 3 val_413 +414 41 4 val_414 +414 41 4 val_414 +417 41 7 val_417 +417 41 7 val_417 +417 41 7 val_417 +418 41 8 val_418 +419 41 9 val_419 +400 40 0 val_400 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +402 40 2 val_402 +403 40 3 val_403 +403 40 3 val_403 +403 40 3 val_403 +404 40 4 val_404 +404 40 4 val_404 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +407 40 7 val_407 +409 40 9 val_409 +409 40 9 val_409 +409 40 9 val_409 +392 39 2 val_392 +393 39 3 val_393 +394 39 4 val_394 +395 39 5 val_395 +395 39 5 val_395 +396 39 6 val_396 +396 39 6 val_396 +396 39 6 val_396 +397 39 7 val_397 +397 39 7 val_397 +399 39 9 val_399 +399 39 9 val_399 +382 38 2 val_382 +382 38 2 val_382 +384 38 4 val_384 +384 38 4 val_384 +384 38 4 val_384 +386 38 6 val_386 +389 38 9 val_389 +373 37 3 val_373 +374 37 4 val_374 +375 37 5 val_375 +377 37 7 val_377 +378 37 8 val_378 +379 37 9 val_379 +360 36 0 val_360 +362 36 2 val_362 +364 36 4 val_364 +365 36 5 val_365 +366 36 6 val_366 +367 36 7 val_367 +367 36 7 val_367 +368 36 8 val_368 +369 36 9 val_369 +369 36 9 val_369 +369 36 9 val_369 +351 35 1 val_351 +353 35 3 val_353 +353 35 3 val_353 +356 35 6 val_356 +341 34 1 val_341 +342 34 2 val_342 +342 34 2 val_342 +344 34 4 val_344 +344 34 4 val_344 +345 34 5 val_345 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +331 33 1 val_331 +331 33 1 val_331 +332 33 2 val_332 +333 33 3 val_333 +333 33 3 val_333 +335 33 5 val_335 +336 33 6 val_336 +338 33 8 val_338 +339 33 9 val_339 +321 32 1 val_321 +321 32 1 val_321 +322 32 2 val_322 +322 32 2 val_322 +323 32 3 val_323 +325 32 5 val_325 +325 32 5 val_325 +327 32 7 val_327 +327 32 7 val_327 +327 32 7 val_327 +310 31 0 val_310 +311 31 1 val_311 +311 31 1 val_311 +311 31 1 val_311 +315 31 5 val_315 +316 31 6 val_316 +316 31 6 val_316 +316 31 6 val_316 +317 31 7 val_317 +317 31 7 val_317 +318 31 8 val_318 +318 31 8 val_318 +318 31 8 val_318 +302 30 2 val_302 +305 30 5 val_305 +306 30 6 val_306 +307 30 7 val_307 +307 30 7 val_307 +308 30 8 val_308 +309 30 9 val_309 +309 30 9 val_309 +291 29 1 val_291 +292 29 2 val_292 +296 29 6 val_296 +298 29 8 val_298 +298 29 8 val_298 +298 29 8 val_298 +280 28 0 val_280 +280 28 0 val_280 +281 28 1 val_281 +281 28 1 val_281 +282 28 2 val_282 +282 28 2 val_282 +283 28 3 val_283 +284 28 4 val_284 +285 28 5 val_285 +286 28 6 val_286 +287 28 7 val_287 +288 28 8 val_288 +288 28 8 val_288 +289 28 9 val_289 +272 27 2 val_272 +272 27 2 val_272 +273 27 3 val_273 +273 27 3 val_273 +273 27 3 val_273 +274 27 4 val_274 +275 27 5 val_275 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +278 27 8 val_278 +278 27 8 val_278 +260 26 0 val_260 +262 26 2 val_262 +263 26 3 val_263 +265 26 5 val_265 +265 26 5 val_265 +266 26 6 val_266 +252 25 2 val_252 +255 25 5 val_255 +255 25 5 val_255 +256 25 6 val_256 +256 25 6 val_256 +257 25 7 val_257 +258 25 8 val_258 +241 24 1 val_241 +242 24 2 val_242 +242 24 2 val_242 +244 24 4 val_244 +247 24 7 val_247 +248 24 8 val_248 +249 24 9 val_249 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +233 23 3 val_233 +233 23 3 val_233 +235 23 5 val_235 +237 23 7 val_237 +237 23 7 val_237 +238 23 8 val_238 +238 23 8 val_238 +239 23 9 val_239 +239 23 9 val_239 +221 22 1 val_221 +221 22 1 val_221 +222 22 2 val_222 +223 22 3 val_223 +223 22 3 val_223 +224 22 4 val_224 +224 22 4 val_224 +226 22 6 val_226 +228 22 8 val_228 +229 22 9 val_229 +229 22 9 val_229 +213 21 3 val_213 +213 21 3 val_213 +214 21 4 val_214 +216 21 6 val_216 +216 21 6 val_216 +217 21 7 val_217 +217 21 7 val_217 +218 21 8 val_218 +219 21 9 val_219 +219 21 9 val_219 +200 20 0 val_200 +200 20 0 val_200 +201 20 1 val_201 +202 20 2 val_202 +203 20 3 val_203 +203 20 3 val_203 +205 20 5 val_205 +205 20 5 val_205 +207 20 7 val_207 +207 20 7 val_207 +208 20 8 val_208 +208 20 8 val_208 +208 20 8 val_208 +209 20 9 val_209 +209 20 9 val_209 +190 19 0 val_190 +191 19 1 val_191 +191 19 1 val_191 +192 19 2 val_192 +193 19 3 val_193 +193 19 3 val_193 +193 19 3 val_193 +194 19 4 val_194 +195 19 5 val_195 +195 19 5 val_195 +196 19 6 val_196 +197 19 7 val_197 +197 19 7 val_197 +199 19 9 val_199 +199 19 9 val_199 +199 19 9 val_199 +180 18 0 val_180 +181 18 1 val_181 +183 18 3 val_183 +186 18 6 val_186 +187 18 7 val_187 +187 18 7 val_187 +187 18 7 val_187 +189 18 9 val_189 +170 17 0 val_170 +172 17 2 val_172 +172 17 2 val_172 +174 17 4 val_174 +174 17 4 val_174 +175 17 5 val_175 +175 17 5 val_175 +176 17 6 val_176 +176 17 6 val_176 +177 17 7 val_177 +178 17 8 val_178 +179 17 9 val_179 +179 17 9 val_179 +160 16 0 val_160 +162 16 2 val_162 +163 16 3 val_163 +164 16 4 val_164 +164 16 4 val_164 +165 16 5 val_165 +165 16 5 val_165 +166 16 6 val_166 +167 16 7 val_167 +167 16 7 val_167 +167 16 7 val_167 +168 16 8 val_168 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +150 15 0 val_150 +152 15 2 val_152 +152 15 2 val_152 +153 15 3 val_153 +155 15 5 val_155 +156 15 6 val_156 +157 15 7 val_157 +158 15 8 val_158 +143 14 3 val_143 +145 14 5 val_145 +146 14 6 val_146 +146 14 6 val_146 +149 14 9 val_149 +149 14 9 val_149 +131 13 1 val_131 +133 13 3 val_133 +134 13 4 val_134 +134 13 4 val_134 +136 13 6 val_136 +137 13 7 val_137 +137 13 7 val_137 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +120 12 0 val_120 +120 12 0 val_120 +125 12 5 val_125 +125 12 5 val_125 +126 12 6 val_126 +128 12 8 val_128 +128 12 8 val_128 +128 12 8 val_128 +129 12 9 val_129 +129 12 9 val_129 +111 11 1 val_111 +113 11 3 val_113 +113 11 3 val_113 +114 11 4 val_114 +116 11 6 val_116 +118 11 8 val_118 +118 11 8 val_118 +119 11 9 val_119 +119 11 9 val_119 +119 11 9 val_119 +100 10 0 val_100 +100 10 0 val_100 +103 10 3 val_103 +103 10 3 val_103 +104 10 4 val_104 +104 10 4 val_104 +105 10 5 val_105 +90 9 0 val_90 +90 9 0 val_90 +90 9 0 val_90 +92 9 2 val_92 +95 9 5 val_95 +95 9 5 val_95 +96 9 6 val_96 +97 9 7 val_97 +97 9 7 val_97 +98 9 8 val_98 +98 9 8 val_98 +80 8 0 val_80 +82 8 2 val_82 +83 8 3 val_83 +83 8 3 val_83 +84 8 4 val_84 +84 8 4 val_84 +85 8 5 val_85 +86 8 6 val_86 +87 8 7 val_87 +70 7 0 val_70 +70 7 0 val_70 +70 7 0 val_70 +72 7 2 val_72 +72 7 2 val_72 +74 7 4 val_74 +76 7 6 val_76 +76 7 6 val_76 +77 7 7 val_77 +78 7 8 val_78 +64 6 4 val_64 +65 6 5 val_65 +66 6 6 val_66 +67 6 7 val_67 +67 6 7 val_67 +69 6 9 val_69 +51 5 1 val_51 +51 5 1 val_51 +53 5 3 val_53 +54 5 4 val_54 +57 5 7 val_57 +58 5 8 val_58 +58 5 8 val_58 +41 4 1 val_41 +42 4 2 val_42 +42 4 2 val_42 +43 4 3 val_43 +44 4 4 val_44 +47 4 7 val_47 +30 3 0 val_30 +33 3 3 val_33 +34 3 4 val_34 +35 3 5 val_35 +35 3 5 val_35 +35 3 5 val_35 +37 3 7 val_37 +37 3 7 val_37 +20 2 0 val_20 +24 2 4 val_24 +24 2 4 val_24 +26 2 6 val_26 +26 2 6 val_26 +27 2 7 val_27 +28 2 8 val_28 +10 1 0 val_10 +11 1 1 val_11 +12 1 2 val_12 +12 1 2 val_12 +15 1 5 val_15 +15 1 5 val_15 +17 1 7 val_17 +18 1 8 val_18 +18 1 8 val_18 +19 1 9 val_19 +0 0 0 val_0 +0 0 0 val_0 +0 0 0 val_0 +2 0 2 val_2 +4 0 4 val_4 +5 0 5 val_5 +5 0 5 val_5 +5 0 5 val_5 +8 0 8 val_8 +9 0 9 val_9 diff --git a/src/test/hive/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c b/src/test/hive/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb b/src/test/hive/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 b/src/test/hive/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..1ecd5ad908e74 --- /dev/null +++ b/src/test/hive/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,500 @@ +490 49 0 val_490 +491 49 1 val_491 +492 49 2 val_492 +492 49 2 val_492 +493 49 3 val_493 +494 49 4 val_494 +495 49 5 val_495 +496 49 6 val_496 +497 49 7 val_497 +498 49 8 val_498 +498 49 8 val_498 +498 49 8 val_498 +480 48 0 val_480 +480 48 0 val_480 +480 48 0 val_480 +481 48 1 val_481 +482 48 2 val_482 +483 48 3 val_483 +484 48 4 val_484 +485 48 5 val_485 +487 48 7 val_487 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +489 48 9 val_489 +470 47 0 val_470 +472 47 2 val_472 +475 47 5 val_475 +477 47 7 val_477 +478 47 8 val_478 +478 47 8 val_478 +479 47 9 val_479 +460 46 0 val_460 +462 46 2 val_462 +462 46 2 val_462 +463 46 3 val_463 +463 46 3 val_463 +466 46 6 val_466 +466 46 6 val_466 +466 46 6 val_466 +467 46 7 val_467 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +468 46 8 val_468 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +469 46 9 val_469 +452 45 2 val_452 +453 45 3 val_453 +454 45 4 val_454 +454 45 4 val_454 +454 45 4 val_454 +455 45 5 val_455 +457 45 7 val_457 +458 45 8 val_458 +458 45 8 val_458 +459 45 9 val_459 +459 45 9 val_459 +443 44 3 val_443 +444 44 4 val_444 +446 44 6 val_446 +448 44 8 val_448 +449 44 9 val_449 +430 43 0 val_430 +430 43 0 val_430 +430 43 0 val_430 +431 43 1 val_431 +431 43 1 val_431 +431 43 1 val_431 +432 43 2 val_432 +435 43 5 val_435 +436 43 6 val_436 +437 43 7 val_437 +438 43 8 val_438 +438 43 8 val_438 +438 43 8 val_438 +439 43 9 val_439 +439 43 9 val_439 +421 42 1 val_421 +424 42 4 val_424 +424 42 4 val_424 +427 42 7 val_427 +429 42 9 val_429 +429 42 9 val_429 +411 41 1 val_411 +413 41 3 val_413 +413 41 3 val_413 +414 41 4 val_414 +414 41 4 val_414 +417 41 7 val_417 +417 41 7 val_417 +417 41 7 val_417 +418 41 8 val_418 +419 41 9 val_419 +400 40 0 val_400 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +401 40 1 val_401 +402 40 2 val_402 +403 40 3 val_403 +403 40 3 val_403 +403 40 3 val_403 +404 40 4 val_404 +404 40 4 val_404 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +406 40 6 val_406 +407 40 7 val_407 +409 40 9 val_409 +409 40 9 val_409 +409 40 9 val_409 +392 39 2 val_392 +393 39 3 val_393 +394 39 4 val_394 +395 39 5 val_395 +395 39 5 val_395 +396 39 6 val_396 +396 39 6 val_396 +396 39 6 val_396 +397 39 7 val_397 +397 39 7 val_397 +399 39 9 val_399 +399 39 9 val_399 +382 38 2 val_382 +382 38 2 val_382 +384 38 4 val_384 +384 38 4 val_384 +384 38 4 val_384 +386 38 6 val_386 +389 38 9 val_389 +373 37 3 val_373 +374 37 4 val_374 +375 37 5 val_375 +377 37 7 val_377 +378 37 8 val_378 +379 37 9 val_379 +360 36 0 val_360 +362 36 2 val_362 +364 36 4 val_364 +365 36 5 val_365 +366 36 6 val_366 +367 36 7 val_367 +367 36 7 val_367 +368 36 8 val_368 +369 36 9 val_369 +369 36 9 val_369 +369 36 9 val_369 +351 35 1 val_351 +353 35 3 val_353 +353 35 3 val_353 +356 35 6 val_356 +341 34 1 val_341 +342 34 2 val_342 +342 34 2 val_342 +344 34 4 val_344 +344 34 4 val_344 +345 34 5 val_345 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +348 34 8 val_348 +331 33 1 val_331 +331 33 1 val_331 +332 33 2 val_332 +333 33 3 val_333 +333 33 3 val_333 +335 33 5 val_335 +336 33 6 val_336 +338 33 8 val_338 +339 33 9 val_339 +321 32 1 val_321 +321 32 1 val_321 +322 32 2 val_322 +322 32 2 val_322 +323 32 3 val_323 +325 32 5 val_325 +325 32 5 val_325 +327 32 7 val_327 +327 32 7 val_327 +327 32 7 val_327 +310 31 0 val_310 +311 31 1 val_311 +311 31 1 val_311 +311 31 1 val_311 +315 31 5 val_315 +316 31 6 val_316 +316 31 6 val_316 +316 31 6 val_316 +317 31 7 val_317 +317 31 7 val_317 +318 31 8 val_318 +318 31 8 val_318 +318 31 8 val_318 +302 30 2 val_302 +305 30 5 val_305 +306 30 6 val_306 +307 30 7 val_307 +307 30 7 val_307 +308 30 8 val_308 +309 30 9 val_309 +309 30 9 val_309 +291 29 1 val_291 +292 29 2 val_292 +296 29 6 val_296 +298 29 8 val_298 +298 29 8 val_298 +298 29 8 val_298 +280 28 0 val_280 +280 28 0 val_280 +281 28 1 val_281 +281 28 1 val_281 +282 28 2 val_282 +282 28 2 val_282 +283 28 3 val_283 +284 28 4 val_284 +285 28 5 val_285 +286 28 6 val_286 +287 28 7 val_287 +288 28 8 val_288 +288 28 8 val_288 +289 28 9 val_289 +272 27 2 val_272 +272 27 2 val_272 +273 27 3 val_273 +273 27 3 val_273 +273 27 3 val_273 +274 27 4 val_274 +275 27 5 val_275 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +277 27 7 val_277 +278 27 8 val_278 +278 27 8 val_278 +260 26 0 val_260 +262 26 2 val_262 +263 26 3 val_263 +265 26 5 val_265 +265 26 5 val_265 +266 26 6 val_266 +252 25 2 val_252 +255 25 5 val_255 +255 25 5 val_255 +256 25 6 val_256 +256 25 6 val_256 +257 25 7 val_257 +258 25 8 val_258 +241 24 1 val_241 +242 24 2 val_242 +242 24 2 val_242 +244 24 4 val_244 +247 24 7 val_247 +248 24 8 val_248 +249 24 9 val_249 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +230 23 0 val_230 +233 23 3 val_233 +233 23 3 val_233 +235 23 5 val_235 +237 23 7 val_237 +237 23 7 val_237 +238 23 8 val_238 +238 23 8 val_238 +239 23 9 val_239 +239 23 9 val_239 +221 22 1 val_221 +221 22 1 val_221 +222 22 2 val_222 +223 22 3 val_223 +223 22 3 val_223 +224 22 4 val_224 +224 22 4 val_224 +226 22 6 val_226 +228 22 8 val_228 +229 22 9 val_229 +229 22 9 val_229 +213 21 3 val_213 +213 21 3 val_213 +214 21 4 val_214 +216 21 6 val_216 +216 21 6 val_216 +217 21 7 val_217 +217 21 7 val_217 +218 21 8 val_218 +219 21 9 val_219 +219 21 9 val_219 +200 20 0 val_200 +200 20 0 val_200 +201 20 1 val_201 +202 20 2 val_202 +203 20 3 val_203 +203 20 3 val_203 +205 20 5 val_205 +205 20 5 val_205 +207 20 7 val_207 +207 20 7 val_207 +208 20 8 val_208 +208 20 8 val_208 +208 20 8 val_208 +209 20 9 val_209 +209 20 9 val_209 +190 19 0 val_190 +191 19 1 val_191 +191 19 1 val_191 +192 19 2 val_192 +193 19 3 val_193 +193 19 3 val_193 +193 19 3 val_193 +194 19 4 val_194 +195 19 5 val_195 +195 19 5 val_195 +196 19 6 val_196 +197 19 7 val_197 +197 19 7 val_197 +199 19 9 val_199 +199 19 9 val_199 +199 19 9 val_199 +180 18 0 val_180 +181 18 1 val_181 +183 18 3 val_183 +186 18 6 val_186 +187 18 7 val_187 +187 18 7 val_187 +187 18 7 val_187 +189 18 9 val_189 +170 17 0 val_170 +172 17 2 val_172 +172 17 2 val_172 +174 17 4 val_174 +174 17 4 val_174 +175 17 5 val_175 +175 17 5 val_175 +176 17 6 val_176 +176 17 6 val_176 +177 17 7 val_177 +178 17 8 val_178 +179 17 9 val_179 +179 17 9 val_179 +160 16 0 val_160 +162 16 2 val_162 +163 16 3 val_163 +164 16 4 val_164 +164 16 4 val_164 +165 16 5 val_165 +165 16 5 val_165 +166 16 6 val_166 +167 16 7 val_167 +167 16 7 val_167 +167 16 7 val_167 +168 16 8 val_168 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +169 16 9 val_169 +150 15 0 val_150 +152 15 2 val_152 +152 15 2 val_152 +153 15 3 val_153 +155 15 5 val_155 +156 15 6 val_156 +157 15 7 val_157 +158 15 8 val_158 +143 14 3 val_143 +145 14 5 val_145 +146 14 6 val_146 +146 14 6 val_146 +149 14 9 val_149 +149 14 9 val_149 +131 13 1 val_131 +133 13 3 val_133 +134 13 4 val_134 +134 13 4 val_134 +136 13 6 val_136 +137 13 7 val_137 +137 13 7 val_137 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +138 13 8 val_138 +120 12 0 val_120 +120 12 0 val_120 +125 12 5 val_125 +125 12 5 val_125 +126 12 6 val_126 +128 12 8 val_128 +128 12 8 val_128 +128 12 8 val_128 +129 12 9 val_129 +129 12 9 val_129 +111 11 1 val_111 +113 11 3 val_113 +113 11 3 val_113 +114 11 4 val_114 +116 11 6 val_116 +118 11 8 val_118 +118 11 8 val_118 +119 11 9 val_119 +119 11 9 val_119 +119 11 9 val_119 +100 10 0 val_100 +100 10 0 val_100 +103 10 3 val_103 +103 10 3 val_103 +104 10 4 val_104 +104 10 4 val_104 +105 10 5 val_105 +90 9 0 val_90 +90 9 0 val_90 +90 9 0 val_90 +92 9 2 val_92 +95 9 5 val_95 +95 9 5 val_95 +96 9 6 val_96 +97 9 7 val_97 +97 9 7 val_97 +98 9 8 val_98 +98 9 8 val_98 +80 8 0 val_80 +82 8 2 val_82 +83 8 3 val_83 +83 8 3 val_83 +84 8 4 val_84 +84 8 4 val_84 +85 8 5 val_85 +86 8 6 val_86 +87 8 7 val_87 +70 7 0 val_70 +70 7 0 val_70 +70 7 0 val_70 +72 7 2 val_72 +72 7 2 val_72 +74 7 4 val_74 +76 7 6 val_76 +76 7 6 val_76 +77 7 7 val_77 +78 7 8 val_78 +64 6 4 val_64 +65 6 5 val_65 +66 6 6 val_66 +67 6 7 val_67 +67 6 7 val_67 +69 6 9 val_69 +51 5 1 val_51 +51 5 1 val_51 +53 5 3 val_53 +54 5 4 val_54 +57 5 7 val_57 +58 5 8 val_58 +58 5 8 val_58 +41 4 1 val_41 +42 4 2 val_42 +42 4 2 val_42 +43 4 3 val_43 +44 4 4 val_44 +47 4 7 val_47 +30 3 0 val_30 +33 3 3 val_33 +34 3 4 val_34 +35 3 5 val_35 +35 3 5 val_35 +35 3 5 val_35 +37 3 7 val_37 +37 3 7 val_37 +20 2 0 val_20 +24 2 4 val_24 +24 2 4 val_24 +26 2 6 val_26 +26 2 6 val_26 +27 2 7 val_27 +28 2 8 val_28 +10 1 0 val_10 +11 1 1 val_11 +12 1 2 val_12 +12 1 2 val_12 +15 1 5 val_15 +15 1 5 val_15 +17 1 7 val_17 +18 1 8 val_18 +18 1 8 val_18 +19 1 9 val_19 +0 0 0 val_0 +0 0 0 val_0 +0 0 0 val_0 +2 0 2 val_2 +4 0 4 val_4 +5 0 5 val_5 +5 0 5 val_5 +5 0 5 val_5 +8 0 8 val_8 +9 0 9 val_9 diff --git a/src/test/hive/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 b/src/test/hive/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a b/src/test/hive/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 b/src/test/hive/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..7fcf1a078a3da --- /dev/null +++ b/src/test/hive/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,500 @@ +0 val_0 0 0 0 val_0 +0 val_0 0 0 0 val_0 +0 val_0 0 0 0 val_0 +10 val_10 10 1 0 val_10 +100 val_100 100 10 0 val_100 +100 val_100 100 10 0 val_100 +103 val_103 103 10 3 val_103 +103 val_103 103 10 3 val_103 +104 val_104 104 10 4 val_104 +104 val_104 104 10 4 val_104 +105 val_105 105 10 5 val_105 +11 val_11 11 1 1 val_11 +111 val_111 111 11 1 val_111 +113 val_113 113 11 3 val_113 +113 val_113 113 11 3 val_113 +114 val_114 114 11 4 val_114 +116 val_116 116 11 6 val_116 +118 val_118 118 11 8 val_118 +118 val_118 118 11 8 val_118 +119 val_119 119 11 9 val_119 +119 val_119 119 11 9 val_119 +119 val_119 119 11 9 val_119 +12 val_12 12 1 2 val_12 +12 val_12 12 1 2 val_12 +120 val_120 120 12 0 val_120 +120 val_120 120 12 0 val_120 +125 val_125 125 12 5 val_125 +125 val_125 125 12 5 val_125 +126 val_126 126 12 6 val_126 +128 val_128 128 12 8 val_128 +128 val_128 128 12 8 val_128 +128 val_128 128 12 8 val_128 +129 val_129 129 12 9 val_129 +129 val_129 129 12 9 val_129 +131 val_131 131 13 1 val_131 +133 val_133 133 13 3 val_133 +134 val_134 134 13 4 val_134 +134 val_134 134 13 4 val_134 +136 val_136 136 13 6 val_136 +137 val_137 137 13 7 val_137 +137 val_137 137 13 7 val_137 +138 val_138 138 13 8 val_138 +138 val_138 138 13 8 val_138 +138 val_138 138 13 8 val_138 +138 val_138 138 13 8 val_138 +143 val_143 143 14 3 val_143 +145 val_145 145 14 5 val_145 +146 val_146 146 14 6 val_146 +146 val_146 146 14 6 val_146 +149 val_149 149 14 9 val_149 +149 val_149 149 14 9 val_149 +15 val_15 15 1 5 val_15 +15 val_15 15 1 5 val_15 +150 val_150 150 15 0 val_150 +152 val_152 152 15 2 val_152 +152 val_152 152 15 2 val_152 +153 val_153 153 15 3 val_153 +155 val_155 155 15 5 val_155 +156 val_156 156 15 6 val_156 +157 val_157 157 15 7 val_157 +158 val_158 158 15 8 val_158 +160 val_160 160 16 0 val_160 +162 val_162 162 16 2 val_162 +163 val_163 163 16 3 val_163 +164 val_164 164 16 4 val_164 +164 val_164 164 16 4 val_164 +165 val_165 165 16 5 val_165 +165 val_165 165 16 5 val_165 +166 val_166 166 16 6 val_166 +167 val_167 167 16 7 val_167 +167 val_167 167 16 7 val_167 +167 val_167 167 16 7 val_167 +168 val_168 168 16 8 val_168 +169 val_169 169 16 9 val_169 +169 val_169 169 16 9 val_169 +169 val_169 169 16 9 val_169 +169 val_169 169 16 9 val_169 +17 val_17 17 1 7 val_17 +170 val_170 170 17 0 val_170 +172 val_172 172 17 2 val_172 +172 val_172 172 17 2 val_172 +174 val_174 174 17 4 val_174 +174 val_174 174 17 4 val_174 +175 val_175 175 17 5 val_175 +175 val_175 175 17 5 val_175 +176 val_176 176 17 6 val_176 +176 val_176 176 17 6 val_176 +177 val_177 177 17 7 val_177 +178 val_178 178 17 8 val_178 +179 val_179 179 17 9 val_179 +179 val_179 179 17 9 val_179 +18 val_18 18 1 8 val_18 +18 val_18 18 1 8 val_18 +180 val_180 180 18 0 val_180 +181 val_181 181 18 1 val_181 +183 val_183 183 18 3 val_183 +186 val_186 186 18 6 val_186 +187 val_187 187 18 7 val_187 +187 val_187 187 18 7 val_187 +187 val_187 187 18 7 val_187 +189 val_189 189 18 9 val_189 +19 val_19 19 1 9 val_19 +190 val_190 190 19 0 val_190 +191 val_191 191 19 1 val_191 +191 val_191 191 19 1 val_191 +192 val_192 192 19 2 val_192 +193 val_193 193 19 3 val_193 +193 val_193 193 19 3 val_193 +193 val_193 193 19 3 val_193 +194 val_194 194 19 4 val_194 +195 val_195 195 19 5 val_195 +195 val_195 195 19 5 val_195 +196 val_196 196 19 6 val_196 +197 val_197 197 19 7 val_197 +197 val_197 197 19 7 val_197 +199 val_199 199 19 9 val_199 +199 val_199 199 19 9 val_199 +199 val_199 199 19 9 val_199 +2 val_2 2 0 2 val_2 +20 val_20 20 2 0 val_20 +200 val_200 200 20 0 val_200 +200 val_200 200 20 0 val_200 +201 val_201 201 20 1 val_201 +202 val_202 202 20 2 val_202 +203 val_203 203 20 3 val_203 +203 val_203 203 20 3 val_203 +205 val_205 205 20 5 val_205 +205 val_205 205 20 5 val_205 +207 val_207 207 20 7 val_207 +207 val_207 207 20 7 val_207 +208 val_208 208 20 8 val_208 +208 val_208 208 20 8 val_208 +208 val_208 208 20 8 val_208 +209 val_209 209 20 9 val_209 +209 val_209 209 20 9 val_209 +213 val_213 213 21 3 val_213 +213 val_213 213 21 3 val_213 +214 val_214 214 21 4 val_214 +216 val_216 216 21 6 val_216 +216 val_216 216 21 6 val_216 +217 val_217 217 21 7 val_217 +217 val_217 217 21 7 val_217 +218 val_218 218 21 8 val_218 +219 val_219 219 21 9 val_219 +219 val_219 219 21 9 val_219 +221 val_221 221 22 1 val_221 +221 val_221 221 22 1 val_221 +222 val_222 222 22 2 val_222 +223 val_223 223 22 3 val_223 +223 val_223 223 22 3 val_223 +224 val_224 224 22 4 val_224 +224 val_224 224 22 4 val_224 +226 val_226 226 22 6 val_226 +228 val_228 228 22 8 val_228 +229 val_229 229 22 9 val_229 +229 val_229 229 22 9 val_229 +230 val_230 230 23 0 val_230 +230 val_230 230 23 0 val_230 +230 val_230 230 23 0 val_230 +230 val_230 230 23 0 val_230 +230 val_230 230 23 0 val_230 +233 val_233 233 23 3 val_233 +233 val_233 233 23 3 val_233 +235 val_235 235 23 5 val_235 +237 val_237 237 23 7 val_237 +237 val_237 237 23 7 val_237 +238 val_238 238 23 8 val_238 +238 val_238 238 23 8 val_238 +239 val_239 239 23 9 val_239 +239 val_239 239 23 9 val_239 +24 val_24 24 2 4 val_24 +24 val_24 24 2 4 val_24 +241 val_241 241 24 1 val_241 +242 val_242 242 24 2 val_242 +242 val_242 242 24 2 val_242 +244 val_244 244 24 4 val_244 +247 val_247 247 24 7 val_247 +248 val_248 248 24 8 val_248 +249 val_249 249 24 9 val_249 +252 val_252 252 25 2 val_252 +255 val_255 255 25 5 val_255 +255 val_255 255 25 5 val_255 +256 val_256 256 25 6 val_256 +256 val_256 256 25 6 val_256 +257 val_257 257 25 7 val_257 +258 val_258 258 25 8 val_258 +26 val_26 26 2 6 val_26 +26 val_26 26 2 6 val_26 +260 val_260 260 26 0 val_260 +262 val_262 262 26 2 val_262 +263 val_263 263 26 3 val_263 +265 val_265 265 26 5 val_265 +265 val_265 265 26 5 val_265 +266 val_266 266 26 6 val_266 +27 val_27 27 2 7 val_27 +272 val_272 272 27 2 val_272 +272 val_272 272 27 2 val_272 +273 val_273 273 27 3 val_273 +273 val_273 273 27 3 val_273 +273 val_273 273 27 3 val_273 +274 val_274 274 27 4 val_274 +275 val_275 275 27 5 val_275 +277 val_277 277 27 7 val_277 +277 val_277 277 27 7 val_277 +277 val_277 277 27 7 val_277 +277 val_277 277 27 7 val_277 +278 val_278 278 27 8 val_278 +278 val_278 278 27 8 val_278 +28 val_28 28 2 8 val_28 +280 val_280 280 28 0 val_280 +280 val_280 280 28 0 val_280 +281 val_281 281 28 1 val_281 +281 val_281 281 28 1 val_281 +282 val_282 282 28 2 val_282 +282 val_282 282 28 2 val_282 +283 val_283 283 28 3 val_283 +284 val_284 284 28 4 val_284 +285 val_285 285 28 5 val_285 +286 val_286 286 28 6 val_286 +287 val_287 287 28 7 val_287 +288 val_288 288 28 8 val_288 +288 val_288 288 28 8 val_288 +289 val_289 289 28 9 val_289 +291 val_291 291 29 1 val_291 +292 val_292 292 29 2 val_292 +296 val_296 296 29 6 val_296 +298 val_298 298 29 8 val_298 +298 val_298 298 29 8 val_298 +298 val_298 298 29 8 val_298 +30 val_30 30 3 0 val_30 +302 val_302 302 30 2 val_302 +305 val_305 305 30 5 val_305 +306 val_306 306 30 6 val_306 +307 val_307 307 30 7 val_307 +307 val_307 307 30 7 val_307 +308 val_308 308 30 8 val_308 +309 val_309 309 30 9 val_309 +309 val_309 309 30 9 val_309 +310 val_310 310 31 0 val_310 +311 val_311 311 31 1 val_311 +311 val_311 311 31 1 val_311 +311 val_311 311 31 1 val_311 +315 val_315 315 31 5 val_315 +316 val_316 316 31 6 val_316 +316 val_316 316 31 6 val_316 +316 val_316 316 31 6 val_316 +317 val_317 317 31 7 val_317 +317 val_317 317 31 7 val_317 +318 val_318 318 31 8 val_318 +318 val_318 318 31 8 val_318 +318 val_318 318 31 8 val_318 +321 val_321 321 32 1 val_321 +321 val_321 321 32 1 val_321 +322 val_322 322 32 2 val_322 +322 val_322 322 32 2 val_322 +323 val_323 323 32 3 val_323 +325 val_325 325 32 5 val_325 +325 val_325 325 32 5 val_325 +327 val_327 327 32 7 val_327 +327 val_327 327 32 7 val_327 +327 val_327 327 32 7 val_327 +33 val_33 33 3 3 val_33 +331 val_331 331 33 1 val_331 +331 val_331 331 33 1 val_331 +332 val_332 332 33 2 val_332 +333 val_333 333 33 3 val_333 +333 val_333 333 33 3 val_333 +335 val_335 335 33 5 val_335 +336 val_336 336 33 6 val_336 +338 val_338 338 33 8 val_338 +339 val_339 339 33 9 val_339 +34 val_34 34 3 4 val_34 +341 val_341 341 34 1 val_341 +342 val_342 342 34 2 val_342 +342 val_342 342 34 2 val_342 +344 val_344 344 34 4 val_344 +344 val_344 344 34 4 val_344 +345 val_345 345 34 5 val_345 +348 val_348 348 34 8 val_348 +348 val_348 348 34 8 val_348 +348 val_348 348 34 8 val_348 +348 val_348 348 34 8 val_348 +348 val_348 348 34 8 val_348 +35 val_35 35 3 5 val_35 +35 val_35 35 3 5 val_35 +35 val_35 35 3 5 val_35 +351 val_351 351 35 1 val_351 +353 val_353 353 35 3 val_353 +353 val_353 353 35 3 val_353 +356 val_356 356 35 6 val_356 +360 val_360 360 36 0 val_360 +362 val_362 362 36 2 val_362 +364 val_364 364 36 4 val_364 +365 val_365 365 36 5 val_365 +366 val_366 366 36 6 val_366 +367 val_367 367 36 7 val_367 +367 val_367 367 36 7 val_367 +368 val_368 368 36 8 val_368 +369 val_369 369 36 9 val_369 +369 val_369 369 36 9 val_369 +369 val_369 369 36 9 val_369 +37 val_37 37 3 7 val_37 +37 val_37 37 3 7 val_37 +373 val_373 373 37 3 val_373 +374 val_374 374 37 4 val_374 +375 val_375 375 37 5 val_375 +377 val_377 377 37 7 val_377 +378 val_378 378 37 8 val_378 +379 val_379 379 37 9 val_379 +382 val_382 382 38 2 val_382 +382 val_382 382 38 2 val_382 +384 val_384 384 38 4 val_384 +384 val_384 384 38 4 val_384 +384 val_384 384 38 4 val_384 +386 val_386 386 38 6 val_386 +389 val_389 389 38 9 val_389 +392 val_392 392 39 2 val_392 +393 val_393 393 39 3 val_393 +394 val_394 394 39 4 val_394 +395 val_395 395 39 5 val_395 +395 val_395 395 39 5 val_395 +396 val_396 396 39 6 val_396 +396 val_396 396 39 6 val_396 +396 val_396 396 39 6 val_396 +397 val_397 397 39 7 val_397 +397 val_397 397 39 7 val_397 +399 val_399 399 39 9 val_399 +399 val_399 399 39 9 val_399 +4 val_4 4 0 4 val_4 +400 val_400 400 40 0 val_400 +401 val_401 401 40 1 val_401 +401 val_401 401 40 1 val_401 +401 val_401 401 40 1 val_401 +401 val_401 401 40 1 val_401 +401 val_401 401 40 1 val_401 +402 val_402 402 40 2 val_402 +403 val_403 403 40 3 val_403 +403 val_403 403 40 3 val_403 +403 val_403 403 40 3 val_403 +404 val_404 404 40 4 val_404 +404 val_404 404 40 4 val_404 +406 val_406 406 40 6 val_406 +406 val_406 406 40 6 val_406 +406 val_406 406 40 6 val_406 +406 val_406 406 40 6 val_406 +407 val_407 407 40 7 val_407 +409 val_409 409 40 9 val_409 +409 val_409 409 40 9 val_409 +409 val_409 409 40 9 val_409 +41 val_41 41 4 1 val_41 +411 val_411 411 41 1 val_411 +413 val_413 413 41 3 val_413 +413 val_413 413 41 3 val_413 +414 val_414 414 41 4 val_414 +414 val_414 414 41 4 val_414 +417 val_417 417 41 7 val_417 +417 val_417 417 41 7 val_417 +417 val_417 417 41 7 val_417 +418 val_418 418 41 8 val_418 +419 val_419 419 41 9 val_419 +42 val_42 42 4 2 val_42 +42 val_42 42 4 2 val_42 +421 val_421 421 42 1 val_421 +424 val_424 424 42 4 val_424 +424 val_424 424 42 4 val_424 +427 val_427 427 42 7 val_427 +429 val_429 429 42 9 val_429 +429 val_429 429 42 9 val_429 +43 val_43 43 4 3 val_43 +430 val_430 430 43 0 val_430 +430 val_430 430 43 0 val_430 +430 val_430 430 43 0 val_430 +431 val_431 431 43 1 val_431 +431 val_431 431 43 1 val_431 +431 val_431 431 43 1 val_431 +432 val_432 432 43 2 val_432 +435 val_435 435 43 5 val_435 +436 val_436 436 43 6 val_436 +437 val_437 437 43 7 val_437 +438 val_438 438 43 8 val_438 +438 val_438 438 43 8 val_438 +438 val_438 438 43 8 val_438 +439 val_439 439 43 9 val_439 +439 val_439 439 43 9 val_439 +44 val_44 44 4 4 val_44 +443 val_443 443 44 3 val_443 +444 val_444 444 44 4 val_444 +446 val_446 446 44 6 val_446 +448 val_448 448 44 8 val_448 +449 val_449 449 44 9 val_449 +452 val_452 452 45 2 val_452 +453 val_453 453 45 3 val_453 +454 val_454 454 45 4 val_454 +454 val_454 454 45 4 val_454 +454 val_454 454 45 4 val_454 +455 val_455 455 45 5 val_455 +457 val_457 457 45 7 val_457 +458 val_458 458 45 8 val_458 +458 val_458 458 45 8 val_458 +459 val_459 459 45 9 val_459 +459 val_459 459 45 9 val_459 +460 val_460 460 46 0 val_460 +462 val_462 462 46 2 val_462 +462 val_462 462 46 2 val_462 +463 val_463 463 46 3 val_463 +463 val_463 463 46 3 val_463 +466 val_466 466 46 6 val_466 +466 val_466 466 46 6 val_466 +466 val_466 466 46 6 val_466 +467 val_467 467 46 7 val_467 +468 val_468 468 46 8 val_468 +468 val_468 468 46 8 val_468 +468 val_468 468 46 8 val_468 +468 val_468 468 46 8 val_468 +469 val_469 469 46 9 val_469 +469 val_469 469 46 9 val_469 +469 val_469 469 46 9 val_469 +469 val_469 469 46 9 val_469 +469 val_469 469 46 9 val_469 +47 val_47 47 4 7 val_47 +470 val_470 470 47 0 val_470 +472 val_472 472 47 2 val_472 +475 val_475 475 47 5 val_475 +477 val_477 477 47 7 val_477 +478 val_478 478 47 8 val_478 +478 val_478 478 47 8 val_478 +479 val_479 479 47 9 val_479 +480 val_480 480 48 0 val_480 +480 val_480 480 48 0 val_480 +480 val_480 480 48 0 val_480 +481 val_481 481 48 1 val_481 +482 val_482 482 48 2 val_482 +483 val_483 483 48 3 val_483 +484 val_484 484 48 4 val_484 +485 val_485 485 48 5 val_485 +487 val_487 487 48 7 val_487 +489 val_489 489 48 9 val_489 +489 val_489 489 48 9 val_489 +489 val_489 489 48 9 val_489 +489 val_489 489 48 9 val_489 +490 val_490 490 49 0 val_490 +491 val_491 491 49 1 val_491 +492 val_492 492 49 2 val_492 +492 val_492 492 49 2 val_492 +493 val_493 493 49 3 val_493 +494 val_494 494 49 4 val_494 +495 val_495 495 49 5 val_495 +496 val_496 496 49 6 val_496 +497 val_497 497 49 7 val_497 +498 val_498 498 49 8 val_498 +498 val_498 498 49 8 val_498 +498 val_498 498 49 8 val_498 +5 val_5 5 0 5 val_5 +5 val_5 5 0 5 val_5 +5 val_5 5 0 5 val_5 +51 val_51 51 5 1 val_51 +51 val_51 51 5 1 val_51 +53 val_53 53 5 3 val_53 +54 val_54 54 5 4 val_54 +57 val_57 57 5 7 val_57 +58 val_58 58 5 8 val_58 +58 val_58 58 5 8 val_58 +64 val_64 64 6 4 val_64 +65 val_65 65 6 5 val_65 +66 val_66 66 6 6 val_66 +67 val_67 67 6 7 val_67 +67 val_67 67 6 7 val_67 +69 val_69 69 6 9 val_69 +70 val_70 70 7 0 val_70 +70 val_70 70 7 0 val_70 +70 val_70 70 7 0 val_70 +72 val_72 72 7 2 val_72 +72 val_72 72 7 2 val_72 +74 val_74 74 7 4 val_74 +76 val_76 76 7 6 val_76 +76 val_76 76 7 6 val_76 +77 val_77 77 7 7 val_77 +78 val_78 78 7 8 val_78 +8 val_8 8 0 8 val_8 +80 val_80 80 8 0 val_80 +82 val_82 82 8 2 val_82 +83 val_83 83 8 3 val_83 +83 val_83 83 8 3 val_83 +84 val_84 84 8 4 val_84 +84 val_84 84 8 4 val_84 +85 val_85 85 8 5 val_85 +86 val_86 86 8 6 val_86 +87 val_87 87 8 7 val_87 +9 val_9 9 0 9 val_9 +90 val_90 90 9 0 val_90 +90 val_90 90 9 0 val_90 +90 val_90 90 9 0 val_90 +92 val_92 92 9 2 val_92 +95 val_95 95 9 5 val_95 +95 val_95 95 9 5 val_95 +96 val_96 96 9 6 val_96 +97 val_97 97 9 7 val_97 +97 val_97 97 9 7 val_97 +98 val_98 98 9 8 val_98 +98 val_98 98 9 8 val_98 diff --git a/src/test/hive/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 b/src/test/hive/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b b/src/test/hive/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 b/src/test/hive/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..7fcf1a078a3da --- /dev/null +++ b/src/test/hive/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,500 @@ +0 val_0 0 0 0 val_0 +0 val_0 0 0 0 val_0 +0 val_0 0 0 0 val_0 +10 val_10 10 1 0 val_10 +100 val_100 100 10 0 val_100 +100 val_100 100 10 0 val_100 +103 val_103 103 10 3 val_103 +103 val_103 103 10 3 val_103 +104 val_104 104 10 4 val_104 +104 val_104 104 10 4 val_104 +105 val_105 105 10 5 val_105 +11 val_11 11 1 1 val_11 +111 val_111 111 11 1 val_111 +113 val_113 113 11 3 val_113 +113 val_113 113 11 3 val_113 +114 val_114 114 11 4 val_114 +116 val_116 116 11 6 val_116 +118 val_118 118 11 8 val_118 +118 val_118 118 11 8 val_118 +119 val_119 119 11 9 val_119 +119 val_119 119 11 9 val_119 +119 val_119 119 11 9 val_119 +12 val_12 12 1 2 val_12 +12 val_12 12 1 2 val_12 +120 val_120 120 12 0 val_120 +120 val_120 120 12 0 val_120 +125 val_125 125 12 5 val_125 +125 val_125 125 12 5 val_125 +126 val_126 126 12 6 val_126 +128 val_128 128 12 8 val_128 +128 val_128 128 12 8 val_128 +128 val_128 128 12 8 val_128 +129 val_129 129 12 9 val_129 +129 val_129 129 12 9 val_129 +131 val_131 131 13 1 val_131 +133 val_133 133 13 3 val_133 +134 val_134 134 13 4 val_134 +134 val_134 134 13 4 val_134 +136 val_136 136 13 6 val_136 +137 val_137 137 13 7 val_137 +137 val_137 137 13 7 val_137 +138 val_138 138 13 8 val_138 +138 val_138 138 13 8 val_138 +138 val_138 138 13 8 val_138 +138 val_138 138 13 8 val_138 +143 val_143 143 14 3 val_143 +145 val_145 145 14 5 val_145 +146 val_146 146 14 6 val_146 +146 val_146 146 14 6 val_146 +149 val_149 149 14 9 val_149 +149 val_149 149 14 9 val_149 +15 val_15 15 1 5 val_15 +15 val_15 15 1 5 val_15 +150 val_150 150 15 0 val_150 +152 val_152 152 15 2 val_152 +152 val_152 152 15 2 val_152 +153 val_153 153 15 3 val_153 +155 val_155 155 15 5 val_155 +156 val_156 156 15 6 val_156 +157 val_157 157 15 7 val_157 +158 val_158 158 15 8 val_158 +160 val_160 160 16 0 val_160 +162 val_162 162 16 2 val_162 +163 val_163 163 16 3 val_163 +164 val_164 164 16 4 val_164 +164 val_164 164 16 4 val_164 +165 val_165 165 16 5 val_165 +165 val_165 165 16 5 val_165 +166 val_166 166 16 6 val_166 +167 val_167 167 16 7 val_167 +167 val_167 167 16 7 val_167 +167 val_167 167 16 7 val_167 +168 val_168 168 16 8 val_168 +169 val_169 169 16 9 val_169 +169 val_169 169 16 9 val_169 +169 val_169 169 16 9 val_169 +169 val_169 169 16 9 val_169 +17 val_17 17 1 7 val_17 +170 val_170 170 17 0 val_170 +172 val_172 172 17 2 val_172 +172 val_172 172 17 2 val_172 +174 val_174 174 17 4 val_174 +174 val_174 174 17 4 val_174 +175 val_175 175 17 5 val_175 +175 val_175 175 17 5 val_175 +176 val_176 176 17 6 val_176 +176 val_176 176 17 6 val_176 +177 val_177 177 17 7 val_177 +178 val_178 178 17 8 val_178 +179 val_179 179 17 9 val_179 +179 val_179 179 17 9 val_179 +18 val_18 18 1 8 val_18 +18 val_18 18 1 8 val_18 +180 val_180 180 18 0 val_180 +181 val_181 181 18 1 val_181 +183 val_183 183 18 3 val_183 +186 val_186 186 18 6 val_186 +187 val_187 187 18 7 val_187 +187 val_187 187 18 7 val_187 +187 val_187 187 18 7 val_187 +189 val_189 189 18 9 val_189 +19 val_19 19 1 9 val_19 +190 val_190 190 19 0 val_190 +191 val_191 191 19 1 val_191 +191 val_191 191 19 1 val_191 +192 val_192 192 19 2 val_192 +193 val_193 193 19 3 val_193 +193 val_193 193 19 3 val_193 +193 val_193 193 19 3 val_193 +194 val_194 194 19 4 val_194 +195 val_195 195 19 5 val_195 +195 val_195 195 19 5 val_195 +196 val_196 196 19 6 val_196 +197 val_197 197 19 7 val_197 +197 val_197 197 19 7 val_197 +199 val_199 199 19 9 val_199 +199 val_199 199 19 9 val_199 +199 val_199 199 19 9 val_199 +2 val_2 2 0 2 val_2 +20 val_20 20 2 0 val_20 +200 val_200 200 20 0 val_200 +200 val_200 200 20 0 val_200 +201 val_201 201 20 1 val_201 +202 val_202 202 20 2 val_202 +203 val_203 203 20 3 val_203 +203 val_203 203 20 3 val_203 +205 val_205 205 20 5 val_205 +205 val_205 205 20 5 val_205 +207 val_207 207 20 7 val_207 +207 val_207 207 20 7 val_207 +208 val_208 208 20 8 val_208 +208 val_208 208 20 8 val_208 +208 val_208 208 20 8 val_208 +209 val_209 209 20 9 val_209 +209 val_209 209 20 9 val_209 +213 val_213 213 21 3 val_213 +213 val_213 213 21 3 val_213 +214 val_214 214 21 4 val_214 +216 val_216 216 21 6 val_216 +216 val_216 216 21 6 val_216 +217 val_217 217 21 7 val_217 +217 val_217 217 21 7 val_217 +218 val_218 218 21 8 val_218 +219 val_219 219 21 9 val_219 +219 val_219 219 21 9 val_219 +221 val_221 221 22 1 val_221 +221 val_221 221 22 1 val_221 +222 val_222 222 22 2 val_222 +223 val_223 223 22 3 val_223 +223 val_223 223 22 3 val_223 +224 val_224 224 22 4 val_224 +224 val_224 224 22 4 val_224 +226 val_226 226 22 6 val_226 +228 val_228 228 22 8 val_228 +229 val_229 229 22 9 val_229 +229 val_229 229 22 9 val_229 +230 val_230 230 23 0 val_230 +230 val_230 230 23 0 val_230 +230 val_230 230 23 0 val_230 +230 val_230 230 23 0 val_230 +230 val_230 230 23 0 val_230 +233 val_233 233 23 3 val_233 +233 val_233 233 23 3 val_233 +235 val_235 235 23 5 val_235 +237 val_237 237 23 7 val_237 +237 val_237 237 23 7 val_237 +238 val_238 238 23 8 val_238 +238 val_238 238 23 8 val_238 +239 val_239 239 23 9 val_239 +239 val_239 239 23 9 val_239 +24 val_24 24 2 4 val_24 +24 val_24 24 2 4 val_24 +241 val_241 241 24 1 val_241 +242 val_242 242 24 2 val_242 +242 val_242 242 24 2 val_242 +244 val_244 244 24 4 val_244 +247 val_247 247 24 7 val_247 +248 val_248 248 24 8 val_248 +249 val_249 249 24 9 val_249 +252 val_252 252 25 2 val_252 +255 val_255 255 25 5 val_255 +255 val_255 255 25 5 val_255 +256 val_256 256 25 6 val_256 +256 val_256 256 25 6 val_256 +257 val_257 257 25 7 val_257 +258 val_258 258 25 8 val_258 +26 val_26 26 2 6 val_26 +26 val_26 26 2 6 val_26 +260 val_260 260 26 0 val_260 +262 val_262 262 26 2 val_262 +263 val_263 263 26 3 val_263 +265 val_265 265 26 5 val_265 +265 val_265 265 26 5 val_265 +266 val_266 266 26 6 val_266 +27 val_27 27 2 7 val_27 +272 val_272 272 27 2 val_272 +272 val_272 272 27 2 val_272 +273 val_273 273 27 3 val_273 +273 val_273 273 27 3 val_273 +273 val_273 273 27 3 val_273 +274 val_274 274 27 4 val_274 +275 val_275 275 27 5 val_275 +277 val_277 277 27 7 val_277 +277 val_277 277 27 7 val_277 +277 val_277 277 27 7 val_277 +277 val_277 277 27 7 val_277 +278 val_278 278 27 8 val_278 +278 val_278 278 27 8 val_278 +28 val_28 28 2 8 val_28 +280 val_280 280 28 0 val_280 +280 val_280 280 28 0 val_280 +281 val_281 281 28 1 val_281 +281 val_281 281 28 1 val_281 +282 val_282 282 28 2 val_282 +282 val_282 282 28 2 val_282 +283 val_283 283 28 3 val_283 +284 val_284 284 28 4 val_284 +285 val_285 285 28 5 val_285 +286 val_286 286 28 6 val_286 +287 val_287 287 28 7 val_287 +288 val_288 288 28 8 val_288 +288 val_288 288 28 8 val_288 +289 val_289 289 28 9 val_289 +291 val_291 291 29 1 val_291 +292 val_292 292 29 2 val_292 +296 val_296 296 29 6 val_296 +298 val_298 298 29 8 val_298 +298 val_298 298 29 8 val_298 +298 val_298 298 29 8 val_298 +30 val_30 30 3 0 val_30 +302 val_302 302 30 2 val_302 +305 val_305 305 30 5 val_305 +306 val_306 306 30 6 val_306 +307 val_307 307 30 7 val_307 +307 val_307 307 30 7 val_307 +308 val_308 308 30 8 val_308 +309 val_309 309 30 9 val_309 +309 val_309 309 30 9 val_309 +310 val_310 310 31 0 val_310 +311 val_311 311 31 1 val_311 +311 val_311 311 31 1 val_311 +311 val_311 311 31 1 val_311 +315 val_315 315 31 5 val_315 +316 val_316 316 31 6 val_316 +316 val_316 316 31 6 val_316 +316 val_316 316 31 6 val_316 +317 val_317 317 31 7 val_317 +317 val_317 317 31 7 val_317 +318 val_318 318 31 8 val_318 +318 val_318 318 31 8 val_318 +318 val_318 318 31 8 val_318 +321 val_321 321 32 1 val_321 +321 val_321 321 32 1 val_321 +322 val_322 322 32 2 val_322 +322 val_322 322 32 2 val_322 +323 val_323 323 32 3 val_323 +325 val_325 325 32 5 val_325 +325 val_325 325 32 5 val_325 +327 val_327 327 32 7 val_327 +327 val_327 327 32 7 val_327 +327 val_327 327 32 7 val_327 +33 val_33 33 3 3 val_33 +331 val_331 331 33 1 val_331 +331 val_331 331 33 1 val_331 +332 val_332 332 33 2 val_332 +333 val_333 333 33 3 val_333 +333 val_333 333 33 3 val_333 +335 val_335 335 33 5 val_335 +336 val_336 336 33 6 val_336 +338 val_338 338 33 8 val_338 +339 val_339 339 33 9 val_339 +34 val_34 34 3 4 val_34 +341 val_341 341 34 1 val_341 +342 val_342 342 34 2 val_342 +342 val_342 342 34 2 val_342 +344 val_344 344 34 4 val_344 +344 val_344 344 34 4 val_344 +345 val_345 345 34 5 val_345 +348 val_348 348 34 8 val_348 +348 val_348 348 34 8 val_348 +348 val_348 348 34 8 val_348 +348 val_348 348 34 8 val_348 +348 val_348 348 34 8 val_348 +35 val_35 35 3 5 val_35 +35 val_35 35 3 5 val_35 +35 val_35 35 3 5 val_35 +351 val_351 351 35 1 val_351 +353 val_353 353 35 3 val_353 +353 val_353 353 35 3 val_353 +356 val_356 356 35 6 val_356 +360 val_360 360 36 0 val_360 +362 val_362 362 36 2 val_362 +364 val_364 364 36 4 val_364 +365 val_365 365 36 5 val_365 +366 val_366 366 36 6 val_366 +367 val_367 367 36 7 val_367 +367 val_367 367 36 7 val_367 +368 val_368 368 36 8 val_368 +369 val_369 369 36 9 val_369 +369 val_369 369 36 9 val_369 +369 val_369 369 36 9 val_369 +37 val_37 37 3 7 val_37 +37 val_37 37 3 7 val_37 +373 val_373 373 37 3 val_373 +374 val_374 374 37 4 val_374 +375 val_375 375 37 5 val_375 +377 val_377 377 37 7 val_377 +378 val_378 378 37 8 val_378 +379 val_379 379 37 9 val_379 +382 val_382 382 38 2 val_382 +382 val_382 382 38 2 val_382 +384 val_384 384 38 4 val_384 +384 val_384 384 38 4 val_384 +384 val_384 384 38 4 val_384 +386 val_386 386 38 6 val_386 +389 val_389 389 38 9 val_389 +392 val_392 392 39 2 val_392 +393 val_393 393 39 3 val_393 +394 val_394 394 39 4 val_394 +395 val_395 395 39 5 val_395 +395 val_395 395 39 5 val_395 +396 val_396 396 39 6 val_396 +396 val_396 396 39 6 val_396 +396 val_396 396 39 6 val_396 +397 val_397 397 39 7 val_397 +397 val_397 397 39 7 val_397 +399 val_399 399 39 9 val_399 +399 val_399 399 39 9 val_399 +4 val_4 4 0 4 val_4 +400 val_400 400 40 0 val_400 +401 val_401 401 40 1 val_401 +401 val_401 401 40 1 val_401 +401 val_401 401 40 1 val_401 +401 val_401 401 40 1 val_401 +401 val_401 401 40 1 val_401 +402 val_402 402 40 2 val_402 +403 val_403 403 40 3 val_403 +403 val_403 403 40 3 val_403 +403 val_403 403 40 3 val_403 +404 val_404 404 40 4 val_404 +404 val_404 404 40 4 val_404 +406 val_406 406 40 6 val_406 +406 val_406 406 40 6 val_406 +406 val_406 406 40 6 val_406 +406 val_406 406 40 6 val_406 +407 val_407 407 40 7 val_407 +409 val_409 409 40 9 val_409 +409 val_409 409 40 9 val_409 +409 val_409 409 40 9 val_409 +41 val_41 41 4 1 val_41 +411 val_411 411 41 1 val_411 +413 val_413 413 41 3 val_413 +413 val_413 413 41 3 val_413 +414 val_414 414 41 4 val_414 +414 val_414 414 41 4 val_414 +417 val_417 417 41 7 val_417 +417 val_417 417 41 7 val_417 +417 val_417 417 41 7 val_417 +418 val_418 418 41 8 val_418 +419 val_419 419 41 9 val_419 +42 val_42 42 4 2 val_42 +42 val_42 42 4 2 val_42 +421 val_421 421 42 1 val_421 +424 val_424 424 42 4 val_424 +424 val_424 424 42 4 val_424 +427 val_427 427 42 7 val_427 +429 val_429 429 42 9 val_429 +429 val_429 429 42 9 val_429 +43 val_43 43 4 3 val_43 +430 val_430 430 43 0 val_430 +430 val_430 430 43 0 val_430 +430 val_430 430 43 0 val_430 +431 val_431 431 43 1 val_431 +431 val_431 431 43 1 val_431 +431 val_431 431 43 1 val_431 +432 val_432 432 43 2 val_432 +435 val_435 435 43 5 val_435 +436 val_436 436 43 6 val_436 +437 val_437 437 43 7 val_437 +438 val_438 438 43 8 val_438 +438 val_438 438 43 8 val_438 +438 val_438 438 43 8 val_438 +439 val_439 439 43 9 val_439 +439 val_439 439 43 9 val_439 +44 val_44 44 4 4 val_44 +443 val_443 443 44 3 val_443 +444 val_444 444 44 4 val_444 +446 val_446 446 44 6 val_446 +448 val_448 448 44 8 val_448 +449 val_449 449 44 9 val_449 +452 val_452 452 45 2 val_452 +453 val_453 453 45 3 val_453 +454 val_454 454 45 4 val_454 +454 val_454 454 45 4 val_454 +454 val_454 454 45 4 val_454 +455 val_455 455 45 5 val_455 +457 val_457 457 45 7 val_457 +458 val_458 458 45 8 val_458 +458 val_458 458 45 8 val_458 +459 val_459 459 45 9 val_459 +459 val_459 459 45 9 val_459 +460 val_460 460 46 0 val_460 +462 val_462 462 46 2 val_462 +462 val_462 462 46 2 val_462 +463 val_463 463 46 3 val_463 +463 val_463 463 46 3 val_463 +466 val_466 466 46 6 val_466 +466 val_466 466 46 6 val_466 +466 val_466 466 46 6 val_466 +467 val_467 467 46 7 val_467 +468 val_468 468 46 8 val_468 +468 val_468 468 46 8 val_468 +468 val_468 468 46 8 val_468 +468 val_468 468 46 8 val_468 +469 val_469 469 46 9 val_469 +469 val_469 469 46 9 val_469 +469 val_469 469 46 9 val_469 +469 val_469 469 46 9 val_469 +469 val_469 469 46 9 val_469 +47 val_47 47 4 7 val_47 +470 val_470 470 47 0 val_470 +472 val_472 472 47 2 val_472 +475 val_475 475 47 5 val_475 +477 val_477 477 47 7 val_477 +478 val_478 478 47 8 val_478 +478 val_478 478 47 8 val_478 +479 val_479 479 47 9 val_479 +480 val_480 480 48 0 val_480 +480 val_480 480 48 0 val_480 +480 val_480 480 48 0 val_480 +481 val_481 481 48 1 val_481 +482 val_482 482 48 2 val_482 +483 val_483 483 48 3 val_483 +484 val_484 484 48 4 val_484 +485 val_485 485 48 5 val_485 +487 val_487 487 48 7 val_487 +489 val_489 489 48 9 val_489 +489 val_489 489 48 9 val_489 +489 val_489 489 48 9 val_489 +489 val_489 489 48 9 val_489 +490 val_490 490 49 0 val_490 +491 val_491 491 49 1 val_491 +492 val_492 492 49 2 val_492 +492 val_492 492 49 2 val_492 +493 val_493 493 49 3 val_493 +494 val_494 494 49 4 val_494 +495 val_495 495 49 5 val_495 +496 val_496 496 49 6 val_496 +497 val_497 497 49 7 val_497 +498 val_498 498 49 8 val_498 +498 val_498 498 49 8 val_498 +498 val_498 498 49 8 val_498 +5 val_5 5 0 5 val_5 +5 val_5 5 0 5 val_5 +5 val_5 5 0 5 val_5 +51 val_51 51 5 1 val_51 +51 val_51 51 5 1 val_51 +53 val_53 53 5 3 val_53 +54 val_54 54 5 4 val_54 +57 val_57 57 5 7 val_57 +58 val_58 58 5 8 val_58 +58 val_58 58 5 8 val_58 +64 val_64 64 6 4 val_64 +65 val_65 65 6 5 val_65 +66 val_66 66 6 6 val_66 +67 val_67 67 6 7 val_67 +67 val_67 67 6 7 val_67 +69 val_69 69 6 9 val_69 +70 val_70 70 7 0 val_70 +70 val_70 70 7 0 val_70 +70 val_70 70 7 0 val_70 +72 val_72 72 7 2 val_72 +72 val_72 72 7 2 val_72 +74 val_74 74 7 4 val_74 +76 val_76 76 7 6 val_76 +76 val_76 76 7 6 val_76 +77 val_77 77 7 7 val_77 +78 val_78 78 7 8 val_78 +8 val_8 8 0 8 val_8 +80 val_80 80 8 0 val_80 +82 val_82 82 8 2 val_82 +83 val_83 83 8 3 val_83 +83 val_83 83 8 3 val_83 +84 val_84 84 8 4 val_84 +84 val_84 84 8 4 val_84 +85 val_85 85 8 5 val_85 +86 val_86 86 8 6 val_86 +87 val_87 87 8 7 val_87 +9 val_9 9 0 9 val_9 +90 val_90 90 9 0 val_90 +90 val_90 90 9 0 val_90 +90 val_90 90 9 0 val_90 +92 val_92 92 9 2 val_92 +95 val_95 95 9 5 val_95 +95 val_95 95 9 5 val_95 +96 val_96 96 9 6 val_96 +97 val_97 97 9 7 val_97 +97 val_97 97 9 7 val_97 +98 val_98 98 9 8 val_98 +98 val_98 98 9 8 val_98 diff --git a/src/test/hive/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge1-1-2c73c923962b91afdf0004a705432550 b/src/test/hive/golden/merge1-1-2c73c923962b91afdf0004a705432550 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 b/src/test/hive/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 b/src/test/hive/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 b/src/test/hive/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 b/src/test/hive/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 b/src/test/hive/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 b/src/test/hive/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-3-1f560722f18ef618a0343313a0cac462 b/src/test/hive/golden/merge1-3-1f560722f18ef618a0343313a0cac462 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-4-3277fe538b66923cd879b45371838d2b b/src/test/hive/golden/merge1-4-3277fe538b66923cd879b45371838d2b new file mode 100644 index 0000000000000..df07a9da29f01 --- /dev/null +++ b/src/test/hive/golden/merge1-4-3277fe538b66923cd879b45371838d2b @@ -0,0 +1,309 @@ +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/src/test/hive/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 b/src/test/hive/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 b/src/test/hive/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 b/src/test/hive/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a b/src/test/hive/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge1-9-64678b0928c4fd054e1578458001c86 b/src/test/hive/golden/merge1-9-64678b0928c4fd054e1578458001c86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 b/src/test/hive/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 b/src/test/hive/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 b/src/test/hive/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a b/src/test/hive/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-14-64678b0928c4fd054e1578458001c86 b/src/test/hive/golden/merge2-14-64678b0928c4fd054e1578458001c86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 b/src/test/hive/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 b/src/test/hive/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 b/src/test/hive/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f b/src/test/hive/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 b/src/test/hive/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf b/src/test/hive/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/src/test/hive/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 b/src/test/hive/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/src/test/hive/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 b/src/test/hive/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 b/src/test/hive/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 b/src/test/hive/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd b/src/test/hive/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd new file mode 100644 index 0000000000000..df07a9da29f01 --- /dev/null +++ b/src/test/hive/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd @@ -0,0 +1,309 @@ +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/src/test/hive/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/merge4-10-692a197bd688b48f762e72978f54aa32 b/src/test/hive/golden/merge4-10-692a197bd688b48f762e72978f54aa32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 b/src/test/hive/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 new file mode 100644 index 0000000000000..67c6db8591549 --- /dev/null +++ b/src/test/hive/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 @@ -0,0 +1,1500 @@ +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/src/test/hive/golden/merge4-12-62541540a18d68a3cb8497a741061d11 b/src/test/hive/golden/merge4-12-62541540a18d68a3cb8497a741061d11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 b/src/test/hive/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 b/src/test/hive/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 new file mode 100644 index 0000000000000..30becc42d7b5a --- /dev/null +++ b/src/test/hive/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 @@ -0,0 +1,3 @@ +ds=2010-08-15/hr=11 +ds=2010-08-15/hr=12 +ds=2010-08-15/hr=file, diff --git a/src/test/hive/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a b/src/test/hive/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a new file mode 100644 index 0000000000000..4c867a5deff08 --- /dev/null +++ b/src/test/hive/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a @@ -0,0 +1 @@ +1 1 2010-08-15 file, diff --git a/src/test/hive/golden/merge4-2-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/merge4-2-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/merge4-2-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/merge4-4-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/merge4-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/merge4-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/merge4-5-3d24d877366c42030f6d9a596665720d b/src/test/hive/golden/merge4-5-3d24d877366c42030f6d9a596665720d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge4-6-b3a76420183795720ab3a384046e5af b/src/test/hive/golden/merge4-6-b3a76420183795720ab3a384046e5af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d b/src/test/hive/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 b/src/test/hive/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 new file mode 100644 index 0000000000000..9feba1dea5fd8 --- /dev/null +++ b/src/test/hive/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 @@ -0,0 +1,1000 @@ +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/src/test/hive/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 b/src/test/hive/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 b/src/test/hive/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 b/src/test/hive/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da b/src/test/hive/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 b/src/test/hive/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 new file mode 100644 index 0000000000000..a4c81ff9f99cd --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 @@ -0,0 +1,500 @@ +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/src/test/hive/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 b/src/test/hive/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 new file mode 100644 index 0000000000000..d66ce5d097ce6 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 b/src/test/hive/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 b/src/test/hive/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 b/src/test/hive/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 b/src/test/hive/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 b/src/test/hive/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 new file mode 100644 index 0000000000000..a4c81ff9f99cd --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 @@ -0,0 +1,500 @@ +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/src/test/hive/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 b/src/test/hive/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 new file mode 100644 index 0000000000000..06444f372bd60 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 b/src/test/hive/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 b/src/test/hive/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d b/src/test/hive/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f b/src/test/hive/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 b/src/test/hive/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 new file mode 100644 index 0000000000000..a4c81ff9f99cd --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 @@ -0,0 +1,500 @@ +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/src/test/hive/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 b/src/test/hive/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 new file mode 100644 index 0000000000000..352ab5a6b96a4 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf b/src/test/hive/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 b/src/test/hive/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 b/src/test/hive/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b b/src/test/hive/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 b/src/test/hive/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 b/src/test/hive/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 b/src/test/hive/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 b/src/test/hive/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da b/src/test/hive/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 b/src/test/hive/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 new file mode 100644 index 0000000000000..f3812861b3d6d --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 b/src/test/hive/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f b/src/test/hive/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf b/src/test/hive/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 b/src/test/hive/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 b/src/test/hive/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b b/src/test/hive/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 b/src/test/hive/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba b/src/test/hive/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d b/src/test/hive/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 b/src/test/hive/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a b/src/test/hive/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 b/src/test/hive/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 b/src/test/hive/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 b/src/test/hive/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 b/src/test/hive/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 @@ -0,0 +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 diff --git a/src/test/hive/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 b/src/test/hive/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 b/src/test/hive/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 b/src/test/hive/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 b/src/test/hive/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 b/src/test/hive/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a b/src/test/hive/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a new file mode 100644 index 0000000000000..5e44ab6b5cef9 --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a @@ -0,0 +1,4 @@ +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/src/test/hive/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 b/src/test/hive/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 new file mode 100644 index 0000000000000..8cd9e4d2c201a --- /dev/null +++ b/src/test/hive/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f b/src/test/hive/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf b/src/test/hive/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 b/src/test/hive/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 b/src/test/hive/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b b/src/test/hive/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c b/src/test/hive/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 b/src/test/hive/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 b/src/test/hive/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b b/src/test/hive/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c b/src/test/hive/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins-3-c408f69470d652da283442a62b384e46 b/src/test/hive/golden/mergejoins-3-c408f69470d652da283442a62b384e46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 b/src/test/hive/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e b/src/test/hive/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df b/src/test/hive/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 b/src/test/hive/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 b/src/test/hive/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 b/src/test/hive/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 b/src/test/hive/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b b/src/test/hive/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff b/src/test/hive/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 b/src/test/hive/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 b/src/test/hive/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 b/src/test/hive/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 b/src/test/hive/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 b/src/test/hive/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 b/src/test/hive/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 b/src/test/hive/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d b/src/test/hive/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 b/src/test/hive/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d b/src/test/hive/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 b/src/test/hive/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d b/src/test/hive/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 b/src/test/hive/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 b/src/test/hive/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d b/src/test/hive/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf b/src/test/hive/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc b/src/test/hive/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 b/src/test/hive/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 b/src/test/hive/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mi-1-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/mi-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/mi-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e b/src/test/hive/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mi-3-b66a495f7bdf106a7886b72267b8659d b/src/test/hive/golden/mi-3-b66a495f7bdf106a7886b72267b8659d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c b/src/test/hive/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc b/src/test/hive/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc new file mode 100644 index 0000000000000..3b011a048ba42 --- /dev/null +++ b/src/test/hive/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc @@ -0,0 +1 @@ +ds=2008-04-08/hr=11 \ No newline at end of file diff --git a/src/test/hive/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 b/src/test/hive/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 new file mode 100644 index 0000000000000..0cdd3e8594c59 --- /dev/null +++ b/src/test/hive/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 @@ -0,0 +1 @@ +ds=2008-04-08/hr=12 \ No newline at end of file diff --git a/src/test/hive/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 b/src/test/hive/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 new file mode 100644 index 0000000000000..f4026a591a958 --- /dev/null +++ b/src/test/hive/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 @@ -0,0 +1,500 @@ +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/src/test/hive/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 b/src/test/hive/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 new file mode 100644 index 0000000000000..f132aba9f3ee1 --- /dev/null +++ b/src/test/hive/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 @@ -0,0 +1,309 @@ +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/src/test/hive/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 b/src/test/hive/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a b/src/test/hive/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa b/src/test/hive/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 b/src/test/hive/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 new file mode 100644 index 0000000000000..55b22b1aa9f24 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 @@ -0,0 +1 @@ +580 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 b/src/test/hive/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 b/src/test/hive/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 new file mode 100644 index 0000000000000..55b22b1aa9f24 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 @@ -0,0 +1 @@ +580 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 b/src/test/hive/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee b/src/test/hive/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee new file mode 100644 index 0000000000000..fa8ef14301a13 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee @@ -0,0 +1,6 @@ +270 +10 +10 +270 +10 +10 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 b/src/test/hive/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 b/src/test/hive/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 b/src/test/hive/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac b/src/test/hive/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 b/src/test/hive/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a b/src/test/hive/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 b/src/test/hive/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 new file mode 100644 index 0000000000000..a3a38a80b7910 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 @@ -0,0 +1 @@ +247580 247580 247580 247580 247580 247580 548662743780 548662743780 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 b/src/test/hive/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 b/src/test/hive/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 new file mode 100644 index 0000000000000..a3a38a80b7910 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 @@ -0,0 +1 @@ +247580 247580 247580 247580 247580 247580 548662743780 548662743780 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f b/src/test/hive/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 b/src/test/hive/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 b/src/test/hive/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 b/src/test/hive/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 new file mode 100644 index 0000000000000..a3a38a80b7910 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 @@ -0,0 +1 @@ +247580 247580 247580 247580 247580 247580 548662743780 548662743780 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 b/src/test/hive/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b b/src/test/hive/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 b/src/test/hive/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 new file mode 100644 index 0000000000000..a3a38a80b7910 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 @@ -0,0 +1 @@ +247580 247580 247580 247580 247580 247580 548662743780 548662743780 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 b/src/test/hive/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe b/src/test/hive/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 b/src/test/hive/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba b/src/test/hive/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c b/src/test/hive/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 b/src/test/hive/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 b/src/test/hive/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 new file mode 100644 index 0000000000000..3c505d9663010 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 @@ -0,0 +1,53 @@ +NULL +66 +66 +98 +98 +98 +128 +128 +128 +128 +146 +146 +146 +150 +150 +213 +213 +213 +224 +224 +224 +238 +238 +238 +255 +255 +255 +273 +273 +273 +273 +278 +278 +278 +311 +311 +311 +311 +369 +369 +369 +369 +401 +401 +401 +401 +401 +401 +406 +406 +406 +406 +406 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e b/src/test/hive/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe b/src/test/hive/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe new file mode 100644 index 0000000000000..a877195ef3ab1 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe @@ -0,0 +1,15 @@ +66 1 +98 1 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b b/src/test/hive/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe b/src/test/hive/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe new file mode 100644 index 0000000000000..a877195ef3ab1 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe @@ -0,0 +1,15 @@ +66 1 +98 1 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f b/src/test/hive/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 b/src/test/hive/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 b/src/test/hive/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 new file mode 100644 index 0000000000000..a877195ef3ab1 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 @@ -0,0 +1,15 @@ +66 1 +98 1 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 b/src/test/hive/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 b/src/test/hive/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 b/src/test/hive/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 b/src/test/hive/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 new file mode 100644 index 0000000000000..a877195ef3ab1 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 @@ -0,0 +1,15 @@ +66 1 +98 1 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 b/src/test/hive/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 b/src/test/hive/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 b/src/test/hive/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f b/src/test/hive/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e b/src/test/hive/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e new file mode 100644 index 0000000000000..25bf17fc5aaab --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e @@ -0,0 +1 @@ +18 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d b/src/test/hive/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 b/src/test/hive/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 new file mode 100644 index 0000000000000..8984af9eb83dd --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 @@ -0,0 +1,74 @@ +66 +66 +98 +98 +98 +98 +128 +128 +128 +128 +128 +128 +146 +146 +146 +146 +150 +150 +213 +213 +213 +213 +224 +224 +224 +224 +238 +238 +238 +238 +255 +255 +255 +255 +273 +273 +273 +273 +273 +273 +278 +278 +278 +278 +311 +311 +311 +311 +311 +311 +369 +369 +369 +369 +369 +369 +401 +401 +401 +401 +401 +401 +401 +401 +401 +401 +406 +406 +406 +406 +406 +406 +406 +406 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 b/src/test/hive/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 b/src/test/hive/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 b/src/test/hive/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 new file mode 100644 index 0000000000000..8984af9eb83dd --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 @@ -0,0 +1,74 @@ +66 +66 +98 +98 +98 +98 +128 +128 +128 +128 +128 +128 +146 +146 +146 +146 +150 +150 +213 +213 +213 +213 +224 +224 +224 +224 +238 +238 +238 +238 +255 +255 +255 +255 +273 +273 +273 +273 +273 +273 +278 +278 +278 +278 +311 +311 +311 +311 +311 +311 +369 +369 +369 +369 +369 +369 +401 +401 +401 +401 +401 +401 +401 +401 +401 +401 +406 +406 +406 +406 +406 +406 +406 +406 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 b/src/test/hive/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 b/src/test/hive/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 b/src/test/hive/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d b/src/test/hive/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 b/src/test/hive/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 b/src/test/hive/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 b/src/test/hive/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..0e3cc2ad6a4e6 --- /dev/null +++ b/src/test/hive/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1,37 @@ +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/src/test/hive/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 b/src/test/hive/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c b/src/test/hive/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f b/src/test/hive/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 b/src/test/hive/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..0e3cc2ad6a4e6 --- /dev/null +++ b/src/test/hive/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1,37 @@ +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/src/test/hive/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 b/src/test/hive/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..df07a9da29f01 --- /dev/null +++ b/src/test/hive/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1,309 @@ +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/src/test/hive/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 b/src/test/hive/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 b/src/test/hive/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 b/src/test/hive/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e b/src/test/hive/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 b/src/test/hive/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 b/src/test/hive/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 b/src/test/hive/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a b/src/test/hive/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 b/src/test/hive/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..e8804e62d0aaf --- /dev/null +++ b/src/test/hive/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1,309 @@ +0 0.0 val_0 +2 2.0 val_2 +4 4.0 val_4 +5 15.0 val_5 +8 8.0 val_8 +9 9.0 val_9 +10 10.0 val_10 +11 11.0 val_11 +12 24.0 val_12 +15 30.0 val_15 +17 17.0 val_17 +18 36.0 val_18 +19 19.0 val_19 +20 20.0 val_20 +24 48.0 val_24 +26 52.0 val_26 +27 27.0 val_27 +28 28.0 val_28 +30 30.0 val_30 +33 33.0 val_33 +34 34.0 val_34 +35 105.0 val_35 +37 74.0 val_37 +41 41.0 val_41 +42 84.0 val_42 +43 43.0 val_43 +44 44.0 val_44 +47 47.0 val_47 +51 102.0 val_51 +53 53.0 val_53 +54 54.0 val_54 +57 57.0 val_57 +58 116.0 val_58 +64 64.0 val_64 +65 65.0 val_65 +66 66.0 val_66 +67 134.0 val_67 +69 69.0 val_69 +70 210.0 val_70 +72 144.0 val_72 +74 74.0 val_74 +76 152.0 val_76 +77 77.0 val_77 +78 78.0 val_78 +80 80.0 val_80 +82 82.0 val_82 +83 166.0 val_83 +84 168.0 val_84 +85 85.0 val_85 +86 86.0 val_86 +87 87.0 val_87 +90 270.0 val_90 +92 92.0 val_92 +95 190.0 val_95 +96 96.0 val_96 +97 194.0 val_97 +98 196.0 val_98 +100 200.0 val_100 +103 206.0 val_103 +104 208.0 val_104 +105 105.0 val_105 +111 111.0 val_111 +113 226.0 val_113 +114 114.0 val_114 +116 116.0 val_116 +118 236.0 val_118 +119 357.0 val_119 +120 240.0 val_120 +125 250.0 val_125 +126 126.0 val_126 +128 384.0 val_128 +129 258.0 val_129 +131 131.0 val_131 +133 133.0 val_133 +134 268.0 val_134 +136 136.0 val_136 +137 274.0 val_137 +138 552.0 val_138 +143 143.0 val_143 +145 145.0 val_145 +146 292.0 val_146 +149 298.0 val_149 +150 150.0 val_150 +152 304.0 val_152 +153 153.0 val_153 +155 155.0 val_155 +156 156.0 val_156 +157 157.0 val_157 +158 158.0 val_158 +160 160.0 val_160 +162 162.0 val_162 +163 163.0 val_163 +164 328.0 val_164 +165 330.0 val_165 +166 166.0 val_166 +167 501.0 val_167 +168 168.0 val_168 +169 676.0 val_169 +170 170.0 val_170 +172 344.0 val_172 +174 348.0 val_174 +175 350.0 val_175 +176 352.0 val_176 +177 177.0 val_177 +178 178.0 val_178 +179 358.0 val_179 +180 180.0 val_180 +181 181.0 val_181 +183 183.0 val_183 +186 186.0 val_186 +187 561.0 val_187 +189 189.0 val_189 +190 190.0 val_190 +191 382.0 val_191 +192 192.0 val_192 +193 579.0 val_193 +194 194.0 val_194 +195 390.0 val_195 +196 196.0 val_196 +197 394.0 val_197 +199 597.0 val_199 +200 400.0 val_200 +201 201.0 val_201 +202 202.0 val_202 +203 406.0 val_203 +205 410.0 val_205 +207 414.0 val_207 +208 624.0 val_208 +209 418.0 val_209 +213 426.0 val_213 +214 214.0 val_214 +216 432.0 val_216 +217 434.0 val_217 +218 218.0 val_218 +219 438.0 val_219 +221 442.0 val_221 +222 222.0 val_222 +223 446.0 val_223 +224 448.0 val_224 +226 226.0 val_226 +228 228.0 val_228 +229 458.0 val_229 +230 1150.0 val_230 +233 466.0 val_233 +235 235.0 val_235 +237 474.0 val_237 +238 476.0 val_238 +239 478.0 val_239 +241 241.0 val_241 +242 484.0 val_242 +244 244.0 val_244 +247 247.0 val_247 +248 248.0 val_248 +249 249.0 val_249 +252 252.0 val_252 +255 510.0 val_255 +256 512.0 val_256 +257 257.0 val_257 +258 258.0 val_258 +260 260.0 val_260 +262 262.0 val_262 +263 263.0 val_263 +265 530.0 val_265 +266 266.0 val_266 +272 544.0 val_272 +273 819.0 val_273 +274 274.0 val_274 +275 275.0 val_275 +277 1108.0 val_277 +278 556.0 val_278 +280 560.0 val_280 +281 562.0 val_281 +282 564.0 val_282 +283 283.0 val_283 +284 284.0 val_284 +285 285.0 val_285 +286 286.0 val_286 +287 287.0 val_287 +288 576.0 val_288 +289 289.0 val_289 +291 291.0 val_291 +292 292.0 val_292 +296 296.0 val_296 +298 894.0 val_298 +302 302.0 val_302 +305 305.0 val_305 +306 306.0 val_306 +307 614.0 val_307 +308 308.0 val_308 +309 618.0 val_309 +310 310.0 val_310 +311 933.0 val_311 +315 315.0 val_315 +316 948.0 val_316 +317 634.0 val_317 +318 954.0 val_318 +321 642.0 val_321 +322 644.0 val_322 +323 323.0 val_323 +325 650.0 val_325 +327 981.0 val_327 +331 662.0 val_331 +332 332.0 val_332 +333 666.0 val_333 +335 335.0 val_335 +336 336.0 val_336 +338 338.0 val_338 +339 339.0 val_339 +341 341.0 val_341 +342 684.0 val_342 +344 688.0 val_344 +345 345.0 val_345 +348 1740.0 val_348 +351 351.0 val_351 +353 706.0 val_353 +356 356.0 val_356 +360 360.0 val_360 +362 362.0 val_362 +364 364.0 val_364 +365 365.0 val_365 +366 366.0 val_366 +367 734.0 val_367 +368 368.0 val_368 +369 1107.0 val_369 +373 373.0 val_373 +374 374.0 val_374 +375 375.0 val_375 +377 377.0 val_377 +378 378.0 val_378 +379 379.0 val_379 +382 764.0 val_382 +384 1152.0 val_384 +386 386.0 val_386 +389 389.0 val_389 +392 392.0 val_392 +393 393.0 val_393 +394 394.0 val_394 +395 790.0 val_395 +396 1188.0 val_396 +397 794.0 val_397 +399 798.0 val_399 +400 400.0 val_400 +401 2005.0 val_401 +402 402.0 val_402 +403 1209.0 val_403 +404 808.0 val_404 +406 1624.0 val_406 +407 407.0 val_407 +409 1227.0 val_409 +411 411.0 val_411 +413 826.0 val_413 +414 828.0 val_414 +417 1251.0 val_417 +418 418.0 val_418 +419 419.0 val_419 +421 421.0 val_421 +424 848.0 val_424 +427 427.0 val_427 +429 858.0 val_429 +430 1290.0 val_430 +431 1293.0 val_431 +432 432.0 val_432 +435 435.0 val_435 +436 436.0 val_436 +437 437.0 val_437 +438 1314.0 val_438 +439 878.0 val_439 +443 443.0 val_443 +444 444.0 val_444 +446 446.0 val_446 +448 448.0 val_448 +449 449.0 val_449 +452 452.0 val_452 +453 453.0 val_453 +454 1362.0 val_454 +455 455.0 val_455 +457 457.0 val_457 +458 916.0 val_458 +459 918.0 val_459 +460 460.0 val_460 +462 924.0 val_462 +463 926.0 val_463 +466 1398.0 val_466 +467 467.0 val_467 +468 1872.0 val_468 +469 2345.0 val_469 +470 470.0 val_470 +472 472.0 val_472 +475 475.0 val_475 +477 477.0 val_477 +478 956.0 val_478 +479 479.0 val_479 +480 1440.0 val_480 +481 481.0 val_481 +482 482.0 val_482 +483 483.0 val_483 +484 484.0 val_484 +485 485.0 val_485 +487 487.0 val_487 +489 1956.0 val_489 +490 490.0 val_490 +491 491.0 val_491 +492 984.0 val_492 +493 493.0 val_493 +494 494.0 val_494 +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 diff --git a/src/test/hive/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc b/src/test/hive/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f b/src/test/hive/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 b/src/test/hive/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca b/src/test/hive/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 b/src/test/hive/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 b/src/test/hive/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 b/src/test/hive/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..fc0b77c7a3628 --- /dev/null +++ b/src/test/hive/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1,309 @@ +0 1.0 +2 1.0 +4 1.0 +5 1.0 +8 1.0 +9 1.0 +10 1.0 +11 1.0 +12 1.0 +15 1.0 +17 1.0 +18 1.0 +19 1.0 +20 1.0 +24 1.0 +26 1.0 +27 1.0 +28 1.0 +30 1.0 +33 1.0 +34 1.0 +35 1.0 +37 1.0 +41 1.0 +42 1.0 +43 1.0 +44 1.0 +47 1.0 +51 1.0 +53 1.0 +54 1.0 +57 1.0 +58 1.0 +64 1.0 +65 1.0 +66 1.0 +67 1.0 +69 1.0 +70 1.0 +72 1.0 +74 1.0 +76 1.0 +77 1.0 +78 1.0 +80 1.0 +82 1.0 +83 1.0 +84 1.0 +85 1.0 +86 1.0 +87 1.0 +90 1.0 +92 1.0 +95 1.0 +96 1.0 +97 1.0 +98 1.0 +100 1.0 +103 1.0 +104 1.0 +105 1.0 +111 1.0 +113 1.0 +114 1.0 +116 1.0 +118 1.0 +119 1.0 +120 1.0 +125 1.0 +126 1.0 +128 1.0 +129 1.0 +131 1.0 +133 1.0 +134 1.0 +136 1.0 +137 1.0 +138 1.0 +143 1.0 +145 1.0 +146 1.0 +149 1.0 +150 1.0 +152 1.0 +153 1.0 +155 1.0 +156 1.0 +157 1.0 +158 1.0 +160 1.0 +162 1.0 +163 1.0 +164 1.0 +165 1.0 +166 1.0 +167 1.0 +168 1.0 +169 1.0 +170 1.0 +172 1.0 +174 1.0 +175 1.0 +176 1.0 +177 1.0 +178 1.0 +179 1.0 +180 1.0 +181 1.0 +183 1.0 +186 1.0 +187 1.0 +189 1.0 +190 1.0 +191 1.0 +192 1.0 +193 1.0 +194 1.0 +195 1.0 +196 1.0 +197 1.0 +199 1.0 +200 1.0 +201 1.0 +202 1.0 +203 1.0 +205 1.0 +207 1.0 +208 1.0 +209 1.0 +213 1.0 +214 1.0 +216 1.0 +217 1.0 +218 1.0 +219 1.0 +221 1.0 +222 1.0 +223 1.0 +224 1.0 +226 1.0 +228 1.0 +229 1.0 +230 1.0 +233 1.0 +235 1.0 +237 1.0 +238 1.0 +239 1.0 +241 1.0 +242 1.0 +244 1.0 +247 1.0 +248 1.0 +249 1.0 +252 1.0 +255 1.0 +256 1.0 +257 1.0 +258 1.0 +260 1.0 +262 1.0 +263 1.0 +265 1.0 +266 1.0 +272 1.0 +273 1.0 +274 1.0 +275 1.0 +277 1.0 +278 1.0 +280 1.0 +281 1.0 +282 1.0 +283 1.0 +284 1.0 +285 1.0 +286 1.0 +287 1.0 +288 1.0 +289 1.0 +291 1.0 +292 1.0 +296 1.0 +298 1.0 +302 1.0 +305 1.0 +306 1.0 +307 1.0 +308 1.0 +309 1.0 +310 1.0 +311 1.0 +315 1.0 +316 1.0 +317 1.0 +318 1.0 +321 1.0 +322 1.0 +323 1.0 +325 1.0 +327 1.0 +331 1.0 +332 1.0 +333 1.0 +335 1.0 +336 1.0 +338 1.0 +339 1.0 +341 1.0 +342 1.0 +344 1.0 +345 1.0 +348 1.0 +351 1.0 +353 1.0 +356 1.0 +360 1.0 +362 1.0 +364 1.0 +365 1.0 +366 1.0 +367 1.0 +368 1.0 +369 1.0 +373 1.0 +374 1.0 +375 1.0 +377 1.0 +378 1.0 +379 1.0 +382 1.0 +384 1.0 +386 1.0 +389 1.0 +392 1.0 +393 1.0 +394 1.0 +395 1.0 +396 1.0 +397 1.0 +399 1.0 +400 1.0 +401 1.0 +402 1.0 +403 1.0 +404 1.0 +406 1.0 +407 1.0 +409 1.0 +411 1.0 +413 1.0 +414 1.0 +417 1.0 +418 1.0 +419 1.0 +421 1.0 +424 1.0 +427 1.0 +429 1.0 +430 1.0 +431 1.0 +432 1.0 +435 1.0 +436 1.0 +437 1.0 +438 1.0 +439 1.0 +443 1.0 +444 1.0 +446 1.0 +448 1.0 +449 1.0 +452 1.0 +453 1.0 +454 1.0 +455 1.0 +457 1.0 +458 1.0 +459 1.0 +460 1.0 +462 1.0 +463 1.0 +466 1.0 +467 1.0 +468 1.0 +469 1.0 +470 1.0 +472 1.0 +475 1.0 +477 1.0 +478 1.0 +479 1.0 +480 1.0 +481 1.0 +482 1.0 +483 1.0 +484 1.0 +485 1.0 +487 1.0 +489 1.0 +490 1.0 +491 1.0 +492 1.0 +493 1.0 +494 1.0 +495 1.0 +496 1.0 +497 1.0 +498 1.0 \ No newline at end of file diff --git a/src/test/hive/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 b/src/test/hive/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..e8804e62d0aaf --- /dev/null +++ b/src/test/hive/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1,309 @@ +0 0.0 val_0 +2 2.0 val_2 +4 4.0 val_4 +5 15.0 val_5 +8 8.0 val_8 +9 9.0 val_9 +10 10.0 val_10 +11 11.0 val_11 +12 24.0 val_12 +15 30.0 val_15 +17 17.0 val_17 +18 36.0 val_18 +19 19.0 val_19 +20 20.0 val_20 +24 48.0 val_24 +26 52.0 val_26 +27 27.0 val_27 +28 28.0 val_28 +30 30.0 val_30 +33 33.0 val_33 +34 34.0 val_34 +35 105.0 val_35 +37 74.0 val_37 +41 41.0 val_41 +42 84.0 val_42 +43 43.0 val_43 +44 44.0 val_44 +47 47.0 val_47 +51 102.0 val_51 +53 53.0 val_53 +54 54.0 val_54 +57 57.0 val_57 +58 116.0 val_58 +64 64.0 val_64 +65 65.0 val_65 +66 66.0 val_66 +67 134.0 val_67 +69 69.0 val_69 +70 210.0 val_70 +72 144.0 val_72 +74 74.0 val_74 +76 152.0 val_76 +77 77.0 val_77 +78 78.0 val_78 +80 80.0 val_80 +82 82.0 val_82 +83 166.0 val_83 +84 168.0 val_84 +85 85.0 val_85 +86 86.0 val_86 +87 87.0 val_87 +90 270.0 val_90 +92 92.0 val_92 +95 190.0 val_95 +96 96.0 val_96 +97 194.0 val_97 +98 196.0 val_98 +100 200.0 val_100 +103 206.0 val_103 +104 208.0 val_104 +105 105.0 val_105 +111 111.0 val_111 +113 226.0 val_113 +114 114.0 val_114 +116 116.0 val_116 +118 236.0 val_118 +119 357.0 val_119 +120 240.0 val_120 +125 250.0 val_125 +126 126.0 val_126 +128 384.0 val_128 +129 258.0 val_129 +131 131.0 val_131 +133 133.0 val_133 +134 268.0 val_134 +136 136.0 val_136 +137 274.0 val_137 +138 552.0 val_138 +143 143.0 val_143 +145 145.0 val_145 +146 292.0 val_146 +149 298.0 val_149 +150 150.0 val_150 +152 304.0 val_152 +153 153.0 val_153 +155 155.0 val_155 +156 156.0 val_156 +157 157.0 val_157 +158 158.0 val_158 +160 160.0 val_160 +162 162.0 val_162 +163 163.0 val_163 +164 328.0 val_164 +165 330.0 val_165 +166 166.0 val_166 +167 501.0 val_167 +168 168.0 val_168 +169 676.0 val_169 +170 170.0 val_170 +172 344.0 val_172 +174 348.0 val_174 +175 350.0 val_175 +176 352.0 val_176 +177 177.0 val_177 +178 178.0 val_178 +179 358.0 val_179 +180 180.0 val_180 +181 181.0 val_181 +183 183.0 val_183 +186 186.0 val_186 +187 561.0 val_187 +189 189.0 val_189 +190 190.0 val_190 +191 382.0 val_191 +192 192.0 val_192 +193 579.0 val_193 +194 194.0 val_194 +195 390.0 val_195 +196 196.0 val_196 +197 394.0 val_197 +199 597.0 val_199 +200 400.0 val_200 +201 201.0 val_201 +202 202.0 val_202 +203 406.0 val_203 +205 410.0 val_205 +207 414.0 val_207 +208 624.0 val_208 +209 418.0 val_209 +213 426.0 val_213 +214 214.0 val_214 +216 432.0 val_216 +217 434.0 val_217 +218 218.0 val_218 +219 438.0 val_219 +221 442.0 val_221 +222 222.0 val_222 +223 446.0 val_223 +224 448.0 val_224 +226 226.0 val_226 +228 228.0 val_228 +229 458.0 val_229 +230 1150.0 val_230 +233 466.0 val_233 +235 235.0 val_235 +237 474.0 val_237 +238 476.0 val_238 +239 478.0 val_239 +241 241.0 val_241 +242 484.0 val_242 +244 244.0 val_244 +247 247.0 val_247 +248 248.0 val_248 +249 249.0 val_249 +252 252.0 val_252 +255 510.0 val_255 +256 512.0 val_256 +257 257.0 val_257 +258 258.0 val_258 +260 260.0 val_260 +262 262.0 val_262 +263 263.0 val_263 +265 530.0 val_265 +266 266.0 val_266 +272 544.0 val_272 +273 819.0 val_273 +274 274.0 val_274 +275 275.0 val_275 +277 1108.0 val_277 +278 556.0 val_278 +280 560.0 val_280 +281 562.0 val_281 +282 564.0 val_282 +283 283.0 val_283 +284 284.0 val_284 +285 285.0 val_285 +286 286.0 val_286 +287 287.0 val_287 +288 576.0 val_288 +289 289.0 val_289 +291 291.0 val_291 +292 292.0 val_292 +296 296.0 val_296 +298 894.0 val_298 +302 302.0 val_302 +305 305.0 val_305 +306 306.0 val_306 +307 614.0 val_307 +308 308.0 val_308 +309 618.0 val_309 +310 310.0 val_310 +311 933.0 val_311 +315 315.0 val_315 +316 948.0 val_316 +317 634.0 val_317 +318 954.0 val_318 +321 642.0 val_321 +322 644.0 val_322 +323 323.0 val_323 +325 650.0 val_325 +327 981.0 val_327 +331 662.0 val_331 +332 332.0 val_332 +333 666.0 val_333 +335 335.0 val_335 +336 336.0 val_336 +338 338.0 val_338 +339 339.0 val_339 +341 341.0 val_341 +342 684.0 val_342 +344 688.0 val_344 +345 345.0 val_345 +348 1740.0 val_348 +351 351.0 val_351 +353 706.0 val_353 +356 356.0 val_356 +360 360.0 val_360 +362 362.0 val_362 +364 364.0 val_364 +365 365.0 val_365 +366 366.0 val_366 +367 734.0 val_367 +368 368.0 val_368 +369 1107.0 val_369 +373 373.0 val_373 +374 374.0 val_374 +375 375.0 val_375 +377 377.0 val_377 +378 378.0 val_378 +379 379.0 val_379 +382 764.0 val_382 +384 1152.0 val_384 +386 386.0 val_386 +389 389.0 val_389 +392 392.0 val_392 +393 393.0 val_393 +394 394.0 val_394 +395 790.0 val_395 +396 1188.0 val_396 +397 794.0 val_397 +399 798.0 val_399 +400 400.0 val_400 +401 2005.0 val_401 +402 402.0 val_402 +403 1209.0 val_403 +404 808.0 val_404 +406 1624.0 val_406 +407 407.0 val_407 +409 1227.0 val_409 +411 411.0 val_411 +413 826.0 val_413 +414 828.0 val_414 +417 1251.0 val_417 +418 418.0 val_418 +419 419.0 val_419 +421 421.0 val_421 +424 848.0 val_424 +427 427.0 val_427 +429 858.0 val_429 +430 1290.0 val_430 +431 1293.0 val_431 +432 432.0 val_432 +435 435.0 val_435 +436 436.0 val_436 +437 437.0 val_437 +438 1314.0 val_438 +439 878.0 val_439 +443 443.0 val_443 +444 444.0 val_444 +446 446.0 val_446 +448 448.0 val_448 +449 449.0 val_449 +452 452.0 val_452 +453 453.0 val_453 +454 1362.0 val_454 +455 455.0 val_455 +457 457.0 val_457 +458 916.0 val_458 +459 918.0 val_459 +460 460.0 val_460 +462 924.0 val_462 +463 926.0 val_463 +466 1398.0 val_466 +467 467.0 val_467 +468 1872.0 val_468 +469 2345.0 val_469 +470 470.0 val_470 +472 472.0 val_472 +475 475.0 val_475 +477 477.0 val_477 +478 956.0 val_478 +479 479.0 val_479 +480 1440.0 val_480 +481 481.0 val_481 +482 482.0 val_482 +483 483.0 val_483 +484 484.0 val_484 +485 485.0 val_485 +487 487.0 val_487 +489 1956.0 val_489 +490 490.0 val_490 +491 491.0 val_491 +492 984.0 val_492 +493 493.0 val_493 +494 494.0 val_494 +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 diff --git a/src/test/hive/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b b/src/test/hive/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 b/src/test/hive/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..fc0b77c7a3628 --- /dev/null +++ b/src/test/hive/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1,309 @@ +0 1.0 +2 1.0 +4 1.0 +5 1.0 +8 1.0 +9 1.0 +10 1.0 +11 1.0 +12 1.0 +15 1.0 +17 1.0 +18 1.0 +19 1.0 +20 1.0 +24 1.0 +26 1.0 +27 1.0 +28 1.0 +30 1.0 +33 1.0 +34 1.0 +35 1.0 +37 1.0 +41 1.0 +42 1.0 +43 1.0 +44 1.0 +47 1.0 +51 1.0 +53 1.0 +54 1.0 +57 1.0 +58 1.0 +64 1.0 +65 1.0 +66 1.0 +67 1.0 +69 1.0 +70 1.0 +72 1.0 +74 1.0 +76 1.0 +77 1.0 +78 1.0 +80 1.0 +82 1.0 +83 1.0 +84 1.0 +85 1.0 +86 1.0 +87 1.0 +90 1.0 +92 1.0 +95 1.0 +96 1.0 +97 1.0 +98 1.0 +100 1.0 +103 1.0 +104 1.0 +105 1.0 +111 1.0 +113 1.0 +114 1.0 +116 1.0 +118 1.0 +119 1.0 +120 1.0 +125 1.0 +126 1.0 +128 1.0 +129 1.0 +131 1.0 +133 1.0 +134 1.0 +136 1.0 +137 1.0 +138 1.0 +143 1.0 +145 1.0 +146 1.0 +149 1.0 +150 1.0 +152 1.0 +153 1.0 +155 1.0 +156 1.0 +157 1.0 +158 1.0 +160 1.0 +162 1.0 +163 1.0 +164 1.0 +165 1.0 +166 1.0 +167 1.0 +168 1.0 +169 1.0 +170 1.0 +172 1.0 +174 1.0 +175 1.0 +176 1.0 +177 1.0 +178 1.0 +179 1.0 +180 1.0 +181 1.0 +183 1.0 +186 1.0 +187 1.0 +189 1.0 +190 1.0 +191 1.0 +192 1.0 +193 1.0 +194 1.0 +195 1.0 +196 1.0 +197 1.0 +199 1.0 +200 1.0 +201 1.0 +202 1.0 +203 1.0 +205 1.0 +207 1.0 +208 1.0 +209 1.0 +213 1.0 +214 1.0 +216 1.0 +217 1.0 +218 1.0 +219 1.0 +221 1.0 +222 1.0 +223 1.0 +224 1.0 +226 1.0 +228 1.0 +229 1.0 +230 1.0 +233 1.0 +235 1.0 +237 1.0 +238 1.0 +239 1.0 +241 1.0 +242 1.0 +244 1.0 +247 1.0 +248 1.0 +249 1.0 +252 1.0 +255 1.0 +256 1.0 +257 1.0 +258 1.0 +260 1.0 +262 1.0 +263 1.0 +265 1.0 +266 1.0 +272 1.0 +273 1.0 +274 1.0 +275 1.0 +277 1.0 +278 1.0 +280 1.0 +281 1.0 +282 1.0 +283 1.0 +284 1.0 +285 1.0 +286 1.0 +287 1.0 +288 1.0 +289 1.0 +291 1.0 +292 1.0 +296 1.0 +298 1.0 +302 1.0 +305 1.0 +306 1.0 +307 1.0 +308 1.0 +309 1.0 +310 1.0 +311 1.0 +315 1.0 +316 1.0 +317 1.0 +318 1.0 +321 1.0 +322 1.0 +323 1.0 +325 1.0 +327 1.0 +331 1.0 +332 1.0 +333 1.0 +335 1.0 +336 1.0 +338 1.0 +339 1.0 +341 1.0 +342 1.0 +344 1.0 +345 1.0 +348 1.0 +351 1.0 +353 1.0 +356 1.0 +360 1.0 +362 1.0 +364 1.0 +365 1.0 +366 1.0 +367 1.0 +368 1.0 +369 1.0 +373 1.0 +374 1.0 +375 1.0 +377 1.0 +378 1.0 +379 1.0 +382 1.0 +384 1.0 +386 1.0 +389 1.0 +392 1.0 +393 1.0 +394 1.0 +395 1.0 +396 1.0 +397 1.0 +399 1.0 +400 1.0 +401 1.0 +402 1.0 +403 1.0 +404 1.0 +406 1.0 +407 1.0 +409 1.0 +411 1.0 +413 1.0 +414 1.0 +417 1.0 +418 1.0 +419 1.0 +421 1.0 +424 1.0 +427 1.0 +429 1.0 +430 1.0 +431 1.0 +432 1.0 +435 1.0 +436 1.0 +437 1.0 +438 1.0 +439 1.0 +443 1.0 +444 1.0 +446 1.0 +448 1.0 +449 1.0 +452 1.0 +453 1.0 +454 1.0 +455 1.0 +457 1.0 +458 1.0 +459 1.0 +460 1.0 +462 1.0 +463 1.0 +466 1.0 +467 1.0 +468 1.0 +469 1.0 +470 1.0 +472 1.0 +475 1.0 +477 1.0 +478 1.0 +479 1.0 +480 1.0 +481 1.0 +482 1.0 +483 1.0 +484 1.0 +485 1.0 +487 1.0 +489 1.0 +490 1.0 +491 1.0 +492 1.0 +493 1.0 +494 1.0 +495 1.0 +496 1.0 +497 1.0 +498 1.0 \ No newline at end of file diff --git a/src/test/hive/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 b/src/test/hive/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 b/src/test/hive/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f b/src/test/hive/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f new file mode 100644 index 0000000000000..310ad7306b634 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f @@ -0,0 +1,10 @@ +165 333 +238 479 +255 513 +27 57 +278 559 +311 625 +409 821 +484 971 +86 175 +98 199 diff --git a/src/test/hive/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 b/src/test/hive/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 new file mode 100644 index 0000000000000..afb670d7efab7 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 @@ -0,0 +1,10 @@ +165 337 +238 483 +255 517 +27 61 +278 563 +311 629 +409 825 +484 975 +86 179 +98 203 diff --git a/src/test/hive/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 b/src/test/hive/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 b/src/test/hive/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f b/src/test/hive/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f new file mode 100644 index 0000000000000..310ad7306b634 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f @@ -0,0 +1,10 @@ +165 333 +238 479 +255 513 +27 57 +278 559 +311 625 +409 821 +484 971 +86 175 +98 199 diff --git a/src/test/hive/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 b/src/test/hive/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 new file mode 100644 index 0000000000000..f1441c3445d94 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 @@ -0,0 +1,6 @@ +238 1 +255 1 +278 1 +311 1 +409 1 +484 1 diff --git a/src/test/hive/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 b/src/test/hive/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 new file mode 100644 index 0000000000000..5421ac20e8621 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 @@ -0,0 +1,4 @@ +165 1 +27 1 +86 1 +98 1 diff --git a/src/test/hive/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 b/src/test/hive/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a b/src/test/hive/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f b/src/test/hive/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f new file mode 100644 index 0000000000000..7dc40b688b128 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f @@ -0,0 +1,20 @@ +100 98 +166 165 +167 165 +239 238 +240 238 +256 255 +257 255 +279 278 +28 27 +280 278 +29 27 +312 311 +313 311 +410 409 +411 409 +485 484 +486 484 +87 86 +88 86 +99 98 diff --git a/src/test/hive/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 b/src/test/hive/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 b/src/test/hive/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 new file mode 100644 index 0000000000000..77d4df18b3e16 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 @@ -0,0 +1,20 @@ +101 98 +102 98 +168 165 +169 165 +241 238 +242 238 +258 255 +259 255 +281 278 +282 278 +30 27 +31 27 +314 311 +315 311 +412 409 +413 409 +487 484 +488 484 +89 86 +90 86 diff --git a/src/test/hive/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 b/src/test/hive/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 new file mode 100644 index 0000000000000..2889b9a801049 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 @@ -0,0 +1,10 @@ +val_165 165 +val_238 238 +val_255 255 +val_27 27 +val_278 278 +val_311 311 +val_409 409 +val_484 484 +val_86 86 +val_98 98 diff --git a/src/test/hive/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 b/src/test/hive/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d b/src/test/hive/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df b/src/test/hive/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f b/src/test/hive/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f new file mode 100644 index 0000000000000..310ad7306b634 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f @@ -0,0 +1,10 @@ +165 333 +238 479 +255 513 +27 57 +278 559 +311 625 +409 821 +484 971 +86 175 +98 199 diff --git a/src/test/hive/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 b/src/test/hive/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 new file mode 100644 index 0000000000000..afb670d7efab7 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 @@ -0,0 +1,10 @@ +165 337 +238 483 +255 517 +27 61 +278 563 +311 629 +409 825 +484 975 +86 179 +98 203 diff --git a/src/test/hive/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 b/src/test/hive/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 new file mode 100644 index 0000000000000..983e095ac80d6 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 @@ -0,0 +1,6 @@ +val_238 238 +val_255 255 +val_278 278 +val_311 311 +val_409 409 +val_484 484 diff --git a/src/test/hive/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 b/src/test/hive/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 new file mode 100644 index 0000000000000..1bc1e319ea180 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 @@ -0,0 +1,4 @@ +val_165 165 +val_27 27 +val_86 86 +val_98 98 diff --git a/src/test/hive/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 b/src/test/hive/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d b/src/test/hive/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 b/src/test/hive/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f b/src/test/hive/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f new file mode 100644 index 0000000000000..acbb7cd281fb8 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f @@ -0,0 +1,20 @@ +165 166 +165 167 +238 239 +238 240 +255 256 +255 257 +27 28 +27 29 +278 279 +278 280 +311 312 +311 313 +409 410 +409 411 +484 485 +484 486 +86 87 +86 88 +98 100 +98 99 diff --git a/src/test/hive/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 b/src/test/hive/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 new file mode 100644 index 0000000000000..4edc92defe127 --- /dev/null +++ b/src/test/hive/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 @@ -0,0 +1,20 @@ +165 168 +165 169 +238 241 +238 242 +255 258 +255 259 +27 30 +27 31 +278 281 +278 282 +311 314 +311 315 +409 412 +409 413 +484 487 +484 488 +86 89 +86 90 +98 101 +98 102 diff --git a/src/test/hive/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 b/src/test/hive/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 b/src/test/hive/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b b/src/test/hive/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 b/src/test/hive/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 b/src/test/hive/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 b/src/test/hive/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 b/src/test/hive/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae b/src/test/hive/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 b/src/test/hive/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 b/src/test/hive/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 b/src/test/hive/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 b/src/test/hive/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 b/src/test/hive/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 b/src/test/hive/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 b/src/test/hive/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 b/src/test/hive/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e b/src/test/hive/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 b/src/test/hive/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 b/src/test/hive/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 new file mode 100644 index 0000000000000..c10c17978a0d8 --- /dev/null +++ b/src/test/hive/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 @@ -0,0 +1,5 @@ +simple_int int None +max_nested_array array>>>>>>>>>>>>>>>>>>>>>> None +max_nested_map array>>>>>>>>>>>>>>>>>>>>> None +max_nested_struct array>>>>>>>>>>>>>>>>>>>>>> None +simple_string string None diff --git a/src/test/hive/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 b/src/test/hive/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 new file mode 100644 index 0000000000000..8ce70d8c71782 --- /dev/null +++ b/src/test/hive/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 @@ -0,0 +1,8 @@ +simple_int int None +max_nested_array array>>>>>>>>>>>>>>>>>>>>>> None +max_nested_map array>>>>>>>>>>>>>>>>>>>>> None +max_nested_struct array>>>>>>>>>>>>>>>>>>>>>> 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>>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:max_nested_map, type:array>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:max_nested_struct, type:array>>>>>>>>>>>>>>>>>>>>>>, 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/src/test/hive/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 b/src/test/hive/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d b/src/test/hive/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d new file mode 100644 index 0000000000000..228853cffe527 --- /dev/null +++ b/src/test/hive/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d @@ -0,0 +1,2 @@ +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/src/test/hive/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 b/src/test/hive/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f b/src/test/hive/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 b/src/test/hive/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 b/src/test/hive/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 b/src/test/hive/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f b/src/test/hive/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 b/src/test/hive/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f b/src/test/hive/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 b/src/test/hive/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 b/src/test/hive/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 b/src/test/hive/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f b/src/test/hive/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 b/src/test/hive/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 b/src/test/hive/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 b/src/test/hive/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f b/src/test/hive/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 b/src/test/hive/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f b/src/test/hive/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/newline-0-43392a20a8d249a279d50d96578e6a1b b/src/test/hive/golden/newline-0-43392a20a8d249a279d50d96578e6a1b new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/newline-0-43392a20a8d249a279d50d96578e6a1b @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/newline-1-a19a19272149c732977c37e043910505 b/src/test/hive/golden/newline-1-a19a19272149c732977c37e043910505 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/newline-1-a19a19272149c732977c37e043910505 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/newline-2-4eb54a664e549614d56ca088c8867d b/src/test/hive/golden/newline-2-4eb54a664e549614d56ca088c8867d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 b/src/test/hive/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 b/src/test/hive/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 new file mode 100644 index 0000000000000..cdcd12dca3ca2 --- /dev/null +++ b/src/test/hive/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 @@ -0,0 +1,84 @@ +val_86 +val_27 +val_98 +val_66 +val_37 +val_15 +val_82 +val_17 +val_0 +val_57 +val_20 +val_92 +val_47 +val_72 +val_4 +val_35 +val_54 +val_51 +val_65 +val_83 +val_12 +val_67 +val_84 +val_58 +val_8 +val_24 +val_42 +val_0 +val_96 +val_26 +val_51 +val_43 +val_95 +val_98 +val_85 +val_77 +val_0 +val_87 +val_15 +val_72 +val_90 +val_19 +val_10 +val_5 +val_58 +val_35 +val_95 +val_11 +val_34 +val_42 +val_78 +val_76 +val_41 +val_30 +val_64 +val_76 +val_74 +val_69 +val_33 +val_70 +val_5 +val_2 +val_35 +val_80 +val_44 +val_53 +val_90 +val_12 +val_5 +val_70 +val_24 +val_70 +val_83 +val_26 +val_67 +val_18 +val_9 +val_18 +val_97 +val_84 +val_28 +val_37 +val_90 +val_97 \ No newline at end of file diff --git a/src/test/hive/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b b/src/test/hive/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 b/src/test/hive/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 b/src/test/hive/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 b/src/test/hive/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 b/src/test/hive/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 b/src/test/hive/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 b/src/test/hive/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 new file mode 100644 index 0000000000000..a0cfcf2621f2e --- /dev/null +++ b/src/test/hive/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 @@ -0,0 +1,25 @@ +NULL 25 +NULL 25 +NULL 25 +NULL 25 +NULL val_165 25 +NULL val_193 25 +NULL val_265 25 +NULL val_27 25 +NULL val_409 25 +NULL val_484 25 +66 val_66 25 +98 val_98 25 +128 25 +146 val_146 25 +150 val_150 25 +213 val_213 25 +224 25 +238 val_238 25 +255 val_255 25 +273 val_273 25 +278 val_278 25 +311 val_311 25 +369 25 +401 val_401 25 +406 val_406 25 \ No newline at end of file diff --git a/src/test/hive/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 b/src/test/hive/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 b/src/test/hive/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 new file mode 100644 index 0000000000000..a0cfcf2621f2e --- /dev/null +++ b/src/test/hive/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 @@ -0,0 +1,25 @@ +NULL 25 +NULL 25 +NULL 25 +NULL 25 +NULL val_165 25 +NULL val_193 25 +NULL val_265 25 +NULL val_27 25 +NULL val_409 25 +NULL val_484 25 +66 val_66 25 +98 val_98 25 +128 25 +146 val_146 25 +150 val_150 25 +213 val_213 25 +224 25 +238 val_238 25 +255 val_255 25 +273 val_273 25 +278 val_278 25 +311 val_311 25 +369 25 +401 val_401 25 +406 val_406 25 \ No newline at end of file diff --git a/src/test/hive/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 b/src/test/hive/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 b/src/test/hive/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 b/src/test/hive/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 b/src/test/hive/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 b/src/test/hive/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 b/src/test/hive/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 new file mode 100644 index 0000000000000..24e19ec6caa8f --- /dev/null +++ b/src/test/hive/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 @@ -0,0 +1 @@ +-826625916 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 b/src/test/hive/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 b/src/test/hive/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 b/src/test/hive/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 b/src/test/hive/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 new file mode 100644 index 0000000000000..05b0c43f926bd --- /dev/null +++ b/src/test/hive/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 @@ -0,0 +1 @@ +10226524244 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 b/src/test/hive/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 b/src/test/hive/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 b/src/test/hive/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 new file mode 100644 index 0000000000000..bc15b9ca54985 --- /dev/null +++ b/src/test/hive/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 @@ -0,0 +1 @@ +20453048488 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad b/src/test/hive/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad new file mode 100644 index 0000000000000..08839f6bb296e --- /dev/null +++ b/src/test/hive/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad @@ -0,0 +1 @@ +200 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 b/src/test/hive/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d b/src/test/hive/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd b/src/test/hive/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e b/src/test/hive/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..fce122031e88f --- /dev/null +++ b/src/test/hive/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,57 @@ +1234 0 3.0 +1234 2 1.0 +1234 4 1.0 +1234 5 3.0 +1234 8 1.0 +1234 9 1.0 +1234 10 1.0 +1234 11 1.0 +1234 12 2.0 +1234 15 2.0 +1234 17 1.0 +1234 18 2.0 +1234 19 1.0 +1234 20 1.0 +1234 24 2.0 +1234 26 2.0 +1234 27 1.0 +1234 28 1.0 +1234 30 1.0 +1234 33 1.0 +1234 34 1.0 +1234 35 3.0 +1234 37 2.0 +1234 41 1.0 +1234 42 2.0 +1234 43 1.0 +1234 44 1.0 +1234 47 1.0 +1234 51 2.0 +1234 53 1.0 +1234 54 1.0 +1234 57 1.0 +1234 58 2.0 +1234 64 1.0 +1234 65 1.0 +1234 66 1.0 +1234 67 2.0 +1234 69 1.0 +1234 70 3.0 +1234 72 2.0 +1234 74 1.0 +1234 76 2.0 +1234 77 1.0 +1234 78 1.0 +1234 80 1.0 +1234 82 1.0 +1234 83 2.0 +1234 84 2.0 +1234 85 1.0 +1234 86 1.0 +1234 87 1.0 +1234 90 3.0 +1234 92 1.0 +1234 95 2.0 +1234 96 1.0 +1234 97 2.0 +1234 98 2.0 \ No newline at end of file diff --git a/src/test/hive/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d b/src/test/hive/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 b/src/test/hive/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 b/src/test/hive/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..fce122031e88f --- /dev/null +++ b/src/test/hive/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,57 @@ +1234 0 3.0 +1234 2 1.0 +1234 4 1.0 +1234 5 3.0 +1234 8 1.0 +1234 9 1.0 +1234 10 1.0 +1234 11 1.0 +1234 12 2.0 +1234 15 2.0 +1234 17 1.0 +1234 18 2.0 +1234 19 1.0 +1234 20 1.0 +1234 24 2.0 +1234 26 2.0 +1234 27 1.0 +1234 28 1.0 +1234 30 1.0 +1234 33 1.0 +1234 34 1.0 +1234 35 3.0 +1234 37 2.0 +1234 41 1.0 +1234 42 2.0 +1234 43 1.0 +1234 44 1.0 +1234 47 1.0 +1234 51 2.0 +1234 53 1.0 +1234 54 1.0 +1234 57 1.0 +1234 58 2.0 +1234 64 1.0 +1234 65 1.0 +1234 66 1.0 +1234 67 2.0 +1234 69 1.0 +1234 70 3.0 +1234 72 2.0 +1234 74 1.0 +1234 76 2.0 +1234 77 1.0 +1234 78 1.0 +1234 80 1.0 +1234 82 1.0 +1234 83 2.0 +1234 84 2.0 +1234 85 1.0 +1234 86 1.0 +1234 87 1.0 +1234 90 3.0 +1234 92 1.0 +1234 95 2.0 +1234 96 1.0 +1234 97 2.0 +1234 98 2.0 \ No newline at end of file diff --git a/src/test/hive/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 b/src/test/hive/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/null_cast-1-7257e6f8170e545962d27741353f672c b/src/test/hive/golden/null_cast-1-7257e6f8170e545962d27741353f672c new file mode 100644 index 0000000000000..d111428eaabb0 --- /dev/null +++ b/src/test/hive/golden/null_cast-1-7257e6f8170e545962d27741353f672c @@ -0,0 +1 @@ +[null,0] [null,[]] [null,{}] [null,{"col1":0}] diff --git a/src/test/hive/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 b/src/test/hive/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 b/src/test/hive/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 b/src/test/hive/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 b/src/test/hive/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 b/src/test/hive/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 b/src/test/hive/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 b/src/test/hive/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 b/src/test/hive/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 b/src/test/hive/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a b/src/test/hive/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 b/src/test/hive/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a b/src/test/hive/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 b/src/test/hive/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a b/src/test/hive/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 b/src/test/hive/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a b/src/test/hive/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 b/src/test/hive/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 b/src/test/hive/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 b/src/test/hive/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 b/src/test/hive/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 b/src/test/hive/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb b/src/test/hive/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e b/src/test/hive/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 b/src/test/hive/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 b/src/test/hive/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 b/src/test/hive/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 b/src/test/hive/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 b/src/test/hive/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb b/src/test/hive/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 b/src/test/hive/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 b/src/test/hive/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e b/src/test/hive/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 b/src/test/hive/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 b/src/test/hive/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 b/src/test/hive/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 b/src/test/hive/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 b/src/test/hive/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 b/src/test/hive/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 b/src/test/hive/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 b/src/test/hive/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 new file mode 100644 index 0000000000000..d05b1f927f4be --- /dev/null +++ b/src/test/hive/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 @@ -0,0 +1 @@ +0 0 diff --git a/src/test/hive/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 b/src/test/hive/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 b/src/test/hive/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 new file mode 100644 index 0000000000000..d05b1f927f4be --- /dev/null +++ b/src/test/hive/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 @@ -0,0 +1 @@ +0 0 diff --git a/src/test/hive/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 b/src/test/hive/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 b/src/test/hive/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 new file mode 100644 index 0000000000000..d05b1f927f4be --- /dev/null +++ b/src/test/hive/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 @@ -0,0 +1 @@ +0 0 diff --git a/src/test/hive/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 b/src/test/hive/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 b/src/test/hive/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 new file mode 100644 index 0000000000000..d05b1f927f4be --- /dev/null +++ b/src/test/hive/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 @@ -0,0 +1 @@ +0 0 diff --git a/src/test/hive/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 b/src/test/hive/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 b/src/test/hive/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 new file mode 100644 index 0000000000000..06b63ea6c2f01 --- /dev/null +++ b/src/test/hive/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 @@ -0,0 +1 @@ +0 0 0 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 b/src/test/hive/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 b/src/test/hive/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 b/src/test/hive/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 new file mode 100644 index 0000000000000..06b63ea6c2f01 --- /dev/null +++ b/src/test/hive/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 @@ -0,0 +1 @@ +0 0 0 diff --git a/src/test/hive/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 b/src/test/hive/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b b/src/test/hive/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 b/src/test/hive/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c b/src/test/hive/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e b/src/test/hive/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c b/src/test/hive/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/src/test/hive/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 b/src/test/hive/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 b/src/test/hive/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf b/src/test/hive/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullinput2-0-40b73200f853b473855e85eba391f008 b/src/test/hive/golden/nullinput2-0-40b73200f853b473855e85eba391f008 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 b/src/test/hive/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 b/src/test/hive/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/nullscript-0-4477b1566208a6805bb20630755a9375 b/src/test/hive/golden/nullscript-0-4477b1566208a6805bb20630755a9375 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 b/src/test/hive/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullscript-2-17238164053203d56d30704e2c098e80 b/src/test/hive/golden/nullscript-2-17238164053203d56d30704e2c098e80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 b/src/test/hive/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 b/src/test/hive/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 new file mode 100644 index 0000000000000..d23e05acf7ba5 --- /dev/null +++ b/src/test/hive/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 @@ -0,0 +1,500 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 \ No newline at end of file diff --git a/src/test/hive/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc b/src/test/hive/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 b/src/test/hive/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 new file mode 100644 index 0000000000000..da0ddb96957e1 --- /dev/null +++ b/src/test/hive/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 @@ -0,0 +1 @@ +NULL NULL NULL 1 0 0.0 diff --git a/src/test/hive/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a b/src/test/hive/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 b/src/test/hive/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f b/src/test/hive/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a b/src/test/hive/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 b/src/test/hive/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e b/src/test/hive/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 b/src/test/hive/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 b/src/test/hive/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d b/src/test/hive/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 b/src/test/hive/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 new file mode 100644 index 0000000000000..7541739b48608 --- /dev/null +++ b/src/test/hive/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 b/src/test/hive/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 b/src/test/hive/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 new file mode 100644 index 0000000000000..78c0010bebbc6 --- /dev/null +++ b/src/test/hive/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 @@ -0,0 +1,34 @@ +# 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/src/test/hive/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 b/src/test/hive/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 b/src/test/hive/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d b/src/test/hive/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb b/src/test/hive/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 b/src/test/hive/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 new file mode 100644 index 0000000000000..6881640dd3cd4 --- /dev/null +++ b/src/test/hive/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 b/src/test/hive/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 b/src/test/hive/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-20-176d469a0edba57404416535c7d48023 b/src/test/hive/golden/orc_create-20-176d469a0edba57404416535c7d48023 new file mode 100644 index 0000000000000..dd4fa77f326ad --- /dev/null +++ b/src/test/hive/golden/orc_create-20-176d469a0edba57404416535c7d48023 @@ -0,0 +1,29 @@ +# col_name data_type comment + +str string from deserializer +mp map from deserializer +lst array from deserializer +strct struct 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/src/test/hive/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 b/src/test/hive/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 b/src/test/hive/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 new file mode 100644 index 0000000000000..88c8812029d82 --- /dev/null +++ b/src/test/hive/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 @@ -0,0 +1,3 @@ +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/src/test/hive/golden/orc_create-23-be779533ea8967231e644209114c8350 b/src/test/hive/golden/orc_create-23-be779533ea8967231e644209114c8350 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 b/src/test/hive/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 new file mode 100644 index 0000000000000..67946888f1baf --- /dev/null +++ b/src/test/hive/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 @@ -0,0 +1,3 @@ +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/src/test/hive/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead b/src/test/hive/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead new file mode 100644 index 0000000000000..83db48f84ec87 --- /dev/null +++ b/src/test/hive/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead @@ -0,0 +1,3 @@ +line1 +line2 +line3 diff --git a/src/test/hive/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c b/src/test/hive/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c new file mode 100644 index 0000000000000..db4876dd3c809 --- /dev/null +++ b/src/test/hive/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c @@ -0,0 +1,3 @@ +{"key12":"value12","key11":"value11","key13":"value13"} +{"key21":"value21","key23":"value23","key22":"value22"} +{"key33":"value33","key31":"value31","key32":"value32"} diff --git a/src/test/hive/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 b/src/test/hive/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 new file mode 100644 index 0000000000000..3b2e1cf7e0098 --- /dev/null +++ b/src/test/hive/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 @@ -0,0 +1,3 @@ +["a","b","c"] +["d","e","f"] +["g","h","i"] diff --git a/src/test/hive/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 b/src/test/hive/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 new file mode 100644 index 0000000000000..0f890dfd226d6 --- /dev/null +++ b/src/test/hive/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 @@ -0,0 +1,3 @@ +{"a":"one","b":"two"} +{"a":"three","b":"four"} +{"a":"five","b":"six"} diff --git a/src/test/hive/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 b/src/test/hive/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae b/src/test/hive/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d b/src/test/hive/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 b/src/test/hive/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-32-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/orc_create-32-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/orc_create-32-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 b/src/test/hive/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 b/src/test/hive/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 b/src/test/hive/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/src/test/hive/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 @@ -0,0 +1 @@ +5 diff --git a/src/test/hive/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a b/src/test/hive/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a new file mode 100644 index 0000000000000..0c6f532488607 --- /dev/null +++ b/src/test/hive/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a @@ -0,0 +1,10 @@ +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/src/test/hive/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 b/src/test/hive/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 b/src/test/hive/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 b/src/test/hive/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/src/test/hive/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae b/src/test/hive/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/src/test/hive/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 b/src/test/hive/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 b/src/test/hive/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 new file mode 100644 index 0000000000000..ecd0704ce3386 --- /dev/null +++ b/src/test/hive/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 @@ -0,0 +1,32 @@ +# col_name data_type comment + +str string None +mp map None +lst array None +strct struct 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/src/test/hive/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 b/src/test/hive/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 b/src/test/hive/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 new file mode 100644 index 0000000000000..6881640dd3cd4 --- /dev/null +++ b/src/test/hive/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 b/src/test/hive/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 b/src/test/hive/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 b/src/test/hive/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c b/src/test/hive/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d b/src/test/hive/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d new file mode 100644 index 0000000000000..f799fca27c353 --- /dev/null +++ b/src/test/hive/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d @@ -0,0 +1,10 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 \ No newline at end of file diff --git a/src/test/hive/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 b/src/test/hive/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 b/src/test/hive/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 b/src/test/hive/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 b/src/test/hive/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 b/src/test/hive/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 b/src/test/hive/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 new file mode 100644 index 0000000000000..f953881c894c1 --- /dev/null +++ b/src/test/hive/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 @@ -0,0 +1 @@ +1082202951192 \ No newline at end of file diff --git a/src/test/hive/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 b/src/test/hive/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 b/src/test/hive/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 b/src/test/hive/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e b/src/test/hive/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 b/src/test/hive/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 new file mode 100644 index 0000000000000..f172be3e72712 --- /dev/null +++ b/src/test/hive/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c b/src/test/hive/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 b/src/test/hive/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 b/src/test/hive/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 b/src/test/hive/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf b/src/test/hive/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 b/src/test/hive/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 new file mode 100644 index 0000000000000..e3c0074c9d4f6 --- /dev/null +++ b/src/test/hive/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 @@ -0,0 +1,9 @@ + + + + + + + + + diff --git a/src/test/hive/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 b/src/test/hive/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e b/src/test/hive/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e new file mode 100644 index 0000000000000..4cd1242d9fe61 --- /dev/null +++ b/src/test/hive/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e @@ -0,0 +1,10 @@ + + + + + + + +165 +255 +27 \ No newline at end of file diff --git a/src/test/hive/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 b/src/test/hive/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e b/src/test/hive/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 b/src/test/hive/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 b/src/test/hive/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc b/src/test/hive/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f b/src/test/hive/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f new file mode 100644 index 0000000000000..e7700b8aee5c6 --- /dev/null +++ b/src/test/hive/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f @@ -0,0 +1,5 @@ +1 +NULL +NULL +NULL +NULL diff --git a/src/test/hive/golden/order-0-5c2fda577771db3b316e0d2bd02d048a b/src/test/hive/golden/order-0-5c2fda577771db3b316e0d2bd02d048a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 b/src/test/hive/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 new file mode 100644 index 0000000000000..217a1915f8826 --- /dev/null +++ b/src/test/hive/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf b/src/test/hive/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 b/src/test/hive/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 new file mode 100644 index 0000000000000..ab34c87e42364 --- /dev/null +++ b/src/test/hive/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 @@ -0,0 +1,10 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/order2-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/order2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/order2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf b/src/test/hive/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/order2-2-4162aa366dc0836eed15cc819226907f b/src/test/hive/golden/order2-2-4162aa366dc0836eed15cc819226907f new file mode 100644 index 0000000000000..217a1915f8826 --- /dev/null +++ b/src/test/hive/golden/order2-2-4162aa366dc0836eed15cc819226907f @@ -0,0 +1,10 @@ +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/src/test/hive/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c b/src/test/hive/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 b/src/test/hive/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 new file mode 100644 index 0000000000000..549da558b18da --- /dev/null +++ b/src/test/hive/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 @@ -0,0 +1,12 @@ +17 val_17 17 val_17 +17 val_17 17 val_17 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +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 diff --git a/src/test/hive/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba b/src/test/hive/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 b/src/test/hive/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 new file mode 100644 index 0000000000000..549da558b18da --- /dev/null +++ b/src/test/hive/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 @@ -0,0 +1,12 @@ +17 val_17 17 val_17 +17 val_17 17 val_17 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +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 diff --git a/src/test/hive/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 b/src/test/hive/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/parallel-1-851e262128626126ae1ad87869db7c54 b/src/test/hive/golden/parallel-1-851e262128626126ae1ad87869db7c54 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/parallel-1-851e262128626126ae1ad87869db7c54 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 b/src/test/hive/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/parallel-11-6230286bc168af7b010968b543690a2a b/src/test/hive/golden/parallel-11-6230286bc168af7b010968b543690a2a new file mode 100644 index 0000000000000..c912af4a5c676 --- /dev/null +++ b/src/test/hive/golden/parallel-11-6230286bc168af7b010968b543690a2a @@ -0,0 +1,309 @@ +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 +34 val_34 +35 val_35 +37 val_37 +41 val_41 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +69 val_69 +70 val_70 +72 val_72 +74 val_74 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 +120 val_120 +125 val_125 +126 val_126 +128 val_128 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +136 val_136 +137 val_137 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +149 val_149 +150 val_150 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +165 val_165 +166 val_166 +167 val_167 +168 val_168 +169 val_169 +170 val_170 +172 val_172 +174 val_174 +175 val_175 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +192 val_192 +193 val_193 +194 val_194 +195 val_195 +196 val_196 +197 val_197 +199 val_199 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +205 val_205 +207 val_207 +208 val_208 +209 val_209 +213 val_213 +214 val_214 +216 val_216 +217 val_217 +218 val_218 +219 val_219 +221 val_221 +222 val_222 +223 val_223 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +230 val_230 +233 val_233 +235 val_235 +237 val_237 +238 val_238 +239 val_239 +241 val_241 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +266 val_266 +272 val_272 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +278 val_278 +280 val_280 +281 val_281 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +308 val_308 +309 val_309 +310 val_310 +311 val_311 +315 val_315 +316 val_316 +317 val_317 +318 val_318 +321 val_321 +322 val_322 +323 val_323 +325 val_325 +327 val_327 +331 val_331 +332 val_332 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +344 val_344 +345 val_345 +348 val_348 +351 val_351 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +368 val_368 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +396 val_396 +397 val_397 +399 val_399 +400 val_400 +401 val_401 +402 val_402 +403 val_403 +404 val_404 +406 val_406 +407 val_407 +409 val_409 +411 val_411 +413 val_413 +414 val_414 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +427 val_427 +429 val_429 +430 val_430 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +459 val_459 +460 val_460 +462 val_462 +463 val_463 +466 val_466 +467 val_467 +468 val_468 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +479 val_479 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 b/src/test/hive/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 new file mode 100644 index 0000000000000..c912af4a5c676 --- /dev/null +++ b/src/test/hive/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 @@ -0,0 +1,309 @@ +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 +34 val_34 +35 val_35 +37 val_37 +41 val_41 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +69 val_69 +70 val_70 +72 val_72 +74 val_74 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 +120 val_120 +125 val_125 +126 val_126 +128 val_128 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +136 val_136 +137 val_137 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +149 val_149 +150 val_150 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +165 val_165 +166 val_166 +167 val_167 +168 val_168 +169 val_169 +170 val_170 +172 val_172 +174 val_174 +175 val_175 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +192 val_192 +193 val_193 +194 val_194 +195 val_195 +196 val_196 +197 val_197 +199 val_199 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +205 val_205 +207 val_207 +208 val_208 +209 val_209 +213 val_213 +214 val_214 +216 val_216 +217 val_217 +218 val_218 +219 val_219 +221 val_221 +222 val_222 +223 val_223 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +230 val_230 +233 val_233 +235 val_235 +237 val_237 +238 val_238 +239 val_239 +241 val_241 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +266 val_266 +272 val_272 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +278 val_278 +280 val_280 +281 val_281 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +308 val_308 +309 val_309 +310 val_310 +311 val_311 +315 val_315 +316 val_316 +317 val_317 +318 val_318 +321 val_321 +322 val_322 +323 val_323 +325 val_325 +327 val_327 +331 val_331 +332 val_332 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +344 val_344 +345 val_345 +348 val_348 +351 val_351 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +368 val_368 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +396 val_396 +397 val_397 +399 val_399 +400 val_400 +401 val_401 +402 val_402 +403 val_403 +404 val_404 +406 val_406 +407 val_407 +409 val_409 +411 val_411 +413 val_413 +414 val_414 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +427 val_427 +429 val_429 +430 val_430 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +459 val_459 +460 val_460 +462 val_462 +463 val_463 +466 val_466 +467 val_467 +468 val_468 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +479 val_479 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parallel-2-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/parallel-2-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/parallel-2-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a b/src/test/hive/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 b/src/test/hive/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/parallel-5-e27b511ffc5828b09069121031d17a2f b/src/test/hive/golden/parallel-5-e27b511ffc5828b09069121031d17a2f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 b/src/test/hive/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/parallel-7-6230286bc168af7b010968b543690a2a b/src/test/hive/golden/parallel-7-6230286bc168af7b010968b543690a2a new file mode 100644 index 0000000000000..c912af4a5c676 --- /dev/null +++ b/src/test/hive/golden/parallel-7-6230286bc168af7b010968b543690a2a @@ -0,0 +1,309 @@ +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 +34 val_34 +35 val_35 +37 val_37 +41 val_41 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +69 val_69 +70 val_70 +72 val_72 +74 val_74 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 +120 val_120 +125 val_125 +126 val_126 +128 val_128 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +136 val_136 +137 val_137 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +149 val_149 +150 val_150 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +165 val_165 +166 val_166 +167 val_167 +168 val_168 +169 val_169 +170 val_170 +172 val_172 +174 val_174 +175 val_175 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +192 val_192 +193 val_193 +194 val_194 +195 val_195 +196 val_196 +197 val_197 +199 val_199 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +205 val_205 +207 val_207 +208 val_208 +209 val_209 +213 val_213 +214 val_214 +216 val_216 +217 val_217 +218 val_218 +219 val_219 +221 val_221 +222 val_222 +223 val_223 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +230 val_230 +233 val_233 +235 val_235 +237 val_237 +238 val_238 +239 val_239 +241 val_241 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +266 val_266 +272 val_272 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +278 val_278 +280 val_280 +281 val_281 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +308 val_308 +309 val_309 +310 val_310 +311 val_311 +315 val_315 +316 val_316 +317 val_317 +318 val_318 +321 val_321 +322 val_322 +323 val_323 +325 val_325 +327 val_327 +331 val_331 +332 val_332 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +344 val_344 +345 val_345 +348 val_348 +351 val_351 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +368 val_368 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +396 val_396 +397 val_397 +399 val_399 +400 val_400 +401 val_401 +402 val_402 +403 val_403 +404 val_404 +406 val_406 +407 val_407 +409 val_409 +411 val_411 +413 val_413 +414 val_414 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +427 val_427 +429 val_429 +430 val_430 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +459 val_459 +460 val_460 +462 val_462 +463 val_463 +466 val_466 +467 val_467 +468 val_468 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +479 val_479 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 b/src/test/hive/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 new file mode 100644 index 0000000000000..c912af4a5c676 --- /dev/null +++ b/src/test/hive/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 @@ -0,0 +1,309 @@ +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 +34 val_34 +35 val_35 +37 val_37 +41 val_41 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +69 val_69 +70 val_70 +72 val_72 +74 val_74 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 +120 val_120 +125 val_125 +126 val_126 +128 val_128 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +136 val_136 +137 val_137 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +149 val_149 +150 val_150 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +165 val_165 +166 val_166 +167 val_167 +168 val_168 +169 val_169 +170 val_170 +172 val_172 +174 val_174 +175 val_175 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +192 val_192 +193 val_193 +194 val_194 +195 val_195 +196 val_196 +197 val_197 +199 val_199 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +205 val_205 +207 val_207 +208 val_208 +209 val_209 +213 val_213 +214 val_214 +216 val_216 +217 val_217 +218 val_218 +219 val_219 +221 val_221 +222 val_222 +223 val_223 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +230 val_230 +233 val_233 +235 val_235 +237 val_237 +238 val_238 +239 val_239 +241 val_241 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +266 val_266 +272 val_272 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +278 val_278 +280 val_280 +281 val_281 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +308 val_308 +309 val_309 +310 val_310 +311 val_311 +315 val_315 +316 val_316 +317 val_317 +318 val_318 +321 val_321 +322 val_322 +323 val_323 +325 val_325 +327 val_327 +331 val_331 +332 val_332 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +344 val_344 +345 val_345 +348 val_348 +351 val_351 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +368 val_368 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +396 val_396 +397 val_397 +399 val_399 +400 val_400 +401 val_401 +402 val_402 +403 val_403 +404 val_404 +406 val_406 +407 val_407 +409 val_409 +411 val_411 +413 val_413 +414 val_414 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +427 val_427 +429 val_429 +430 val_430 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +459 val_459 +460 val_460 +462 val_462 +463 val_463 +466 val_466 +467 val_467 +468 val_468 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +479 val_479 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c b/src/test/hive/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 b/src/test/hive/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 b/src/test/hive/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc b/src/test/hive/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd b/src/test/hive/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec b/src/test/hive/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 b/src/test/hive/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 b/src/test/hive/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea b/src/test/hive/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 b/src/test/hive/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b b/src/test/hive/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 b/src/test/hive/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 b/src/test/hive/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 b/src/test/hive/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/src/test/hive/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/src/test/hive/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 b/src/test/hive/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 b/src/test/hive/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb b/src/test/hive/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/src/test/hive/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 new file mode 100644 index 0000000000000..ffa0de8a0c322 --- /dev/null +++ b/src/test/hive/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 @@ -0,0 +1,32 @@ +# col_name data_type comment + +c1 tinyint None + +# Partition Information +# col_name data_type comment + +c2 string None + +# Detailed Partition Information +Partition Value: [v1] +Database: default +Table: mytbl +CreateTime: Fri Jan 03 17:18:52 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1 +Partition Parameters: + a myval + b yourval + transient_lastDdlTime 1388798332 + +# 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/src/test/hive/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 b/src/test/hive/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 b/src/test/hive/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 b/src/test/hive/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb b/src/test/hive/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/src/test/hive/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 new file mode 100644 index 0000000000000..0909bab11a6ec --- /dev/null +++ b/src/test/hive/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 @@ -0,0 +1,30 @@ +# col_name data_type comment + +c1 tinyint None + +# Partition Information +# col_name data_type comment + +c2 string None + +# Detailed Partition Information +Partition Value: [v1] +Database: default +Table: mytbl +CreateTime: Fri Jan 03 17:18:59 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1 +Partition Parameters: + transient_lastDdlTime 1388798339 + +# 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/src/test/hive/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 b/src/test/hive/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 b/src/test/hive/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb b/src/test/hive/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/src/test/hive/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 new file mode 100644 index 0000000000000..55fb6df62f0a7 --- /dev/null +++ b/src/test/hive/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 @@ -0,0 +1,33 @@ +# col_name data_type comment + +c1 tinyint None + +# Partition Information +# col_name data_type comment + +c2 string None + +# Detailed Partition Information +Partition Value: [v1] +Database: default +Table: mytbl +CreateTime: Fri Jan 03 17:19:02 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1 +Partition Parameters: + a myval + b yourval + c noval + transient_lastDdlTime 1388798342 + +# 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/src/test/hive/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 b/src/test/hive/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a b/src/test/hive/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 b/src/test/hive/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 b/src/test/hive/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 new file mode 100644 index 0000000000000..8fb73f609efce --- /dev/null +++ b/src/test/hive/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 @@ -0,0 +1,10 @@ +1 111 222 333 +1 111 222 333 +1 111 222 333 +1 111 222 333 +1 111 222 333 +1 111 222 333 +1 111 222 333 +1 111 222 333 +1 111 222 333 +1 111 222 333 diff --git a/src/test/hive/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 b/src/test/hive/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 b/src/test/hive/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 b/src/test/hive/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 new file mode 100644 index 0000000000000..81df179618406 --- /dev/null +++ b/src/test/hive/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 @@ -0,0 +1,3 @@ +ts=2011-01-11+14:18:26 +ts=2011-01-11+15:18:26 +ts=2011-01-11+16:18:26 diff --git a/src/test/hive/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 b/src/test/hive/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/src/test/hive/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 @@ -0,0 +1 @@ +3 diff --git a/src/test/hive/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 b/src/test/hive/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/src/test/hive/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 b/src/test/hive/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 new file mode 100644 index 0000000000000..e16053e3110ae --- /dev/null +++ b/src/test/hive/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 @@ -0,0 +1,3 @@ +ts=2011-01-11+14%3A18%3A26 +ts=2011-01-11+15%3A18%3A26 +ts=2011-01-11+16%3A18%3A26 diff --git a/src/test/hive/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 b/src/test/hive/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/src/test/hive/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 @@ -0,0 +1 @@ +3 diff --git a/src/test/hive/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 b/src/test/hive/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/src/test/hive/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c b/src/test/hive/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 b/src/test/hive/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 b/src/test/hive/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 new file mode 100644 index 0000000000000..8af58e2b29f4a --- /dev/null +++ b/src/test/hive/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 @@ -0,0 +1,8 @@ +key string None +value string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None \ No newline at end of file diff --git a/src/test/hive/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb b/src/test/hive/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 b/src/test/hive/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 new file mode 100644 index 0000000000000..82a07522b0af0 --- /dev/null +++ b/src/test/hive/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 @@ -0,0 +1,9 @@ +key string None +value string None +x string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None \ No newline at end of file diff --git a/src/test/hive/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 b/src/test/hive/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 new file mode 100644 index 0000000000000..8af58e2b29f4a --- /dev/null +++ b/src/test/hive/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 @@ -0,0 +1,8 @@ +key string None +value string None +dt string None + +# Partition Information +# col_name data_type comment + +dt string None \ No newline at end of file diff --git a/src/test/hive/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d b/src/test/hive/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 b/src/test/hive/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf b/src/test/hive/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 b/src/test/hive/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 new file mode 100644 index 0000000000000..95f5492558a9b --- /dev/null +++ b/src/test/hive/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 @@ -0,0 +1,20 @@ +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/src/test/hive/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 b/src/test/hive/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 b/src/test/hive/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/src/test/hive/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 b/src/test/hive/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 new file mode 100644 index 0000000000000..e16053e3110ae --- /dev/null +++ b/src/test/hive/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 @@ -0,0 +1,3 @@ +ts=2011-01-11+14%3A18%3A26 +ts=2011-01-11+15%3A18%3A26 +ts=2011-01-11+16%3A18%3A26 diff --git a/src/test/hive/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 b/src/test/hive/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/src/test/hive/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 @@ -0,0 +1 @@ +3 diff --git a/src/test/hive/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/src/test/hive/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 b/src/test/hive/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 new file mode 100644 index 0000000000000..e16053e3110ae --- /dev/null +++ b/src/test/hive/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 @@ -0,0 +1,3 @@ +ts=2011-01-11+14%3A18%3A26 +ts=2011-01-11+15%3A18%3A26 +ts=2011-01-11+16%3A18%3A26 diff --git a/src/test/hive/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 b/src/test/hive/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/src/test/hive/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 @@ -0,0 +1 @@ +3 diff --git a/src/test/hive/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c b/src/test/hive/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa b/src/test/hive/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc b/src/test/hive/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a b/src/test/hive/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f b/src/test/hive/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 b/src/test/hive/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 new file mode 100644 index 0000000000000..c4f5a8a640fd8 --- /dev/null +++ b/src/test/hive/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 @@ -0,0 +1,6 @@ +1 11 June 2 +2 12 June 2 +3 13 June 2 +7 17 June 2 +8 18 June 2 +8 28 June 2 diff --git a/src/test/hive/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d b/src/test/hive/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 b/src/test/hive/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 b/src/test/hive/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 b/src/test/hive/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 new file mode 100644 index 0000000000000..c4f5a8a640fd8 --- /dev/null +++ b/src/test/hive/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 @@ -0,0 +1,6 @@ +1 11 June 2 +2 12 June 2 +3 13 June 2 +7 17 June 2 +8 18 June 2 +8 28 June 2 diff --git a/src/test/hive/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d b/src/test/hive/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 b/src/test/hive/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 b/src/test/hive/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 b/src/test/hive/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 b/src/test/hive/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 new file mode 100644 index 0000000000000..b4de394767536 --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 @@ -0,0 +1 @@ +11 diff --git a/src/test/hive/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 b/src/test/hive/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 new file mode 100644 index 0000000000000..64bb6b746dcea --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 @@ -0,0 +1 @@ +30 diff --git a/src/test/hive/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 b/src/test/hive/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c b/src/test/hive/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 b/src/test/hive/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 @@ -0,0 +1 @@ +20 diff --git a/src/test/hive/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 b/src/test/hive/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 @@ -0,0 +1 @@ +10 diff --git a/src/test/hive/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d b/src/test/hive/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d @@ -0,0 +1 @@ +20 diff --git a/src/test/hive/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 b/src/test/hive/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 @@ -0,0 +1 @@ +10 diff --git a/src/test/hive/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a b/src/test/hive/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a @@ -0,0 +1 @@ +20 diff --git a/src/test/hive/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 b/src/test/hive/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 b/src/test/hive/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 b/src/test/hive/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef b/src/test/hive/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce b/src/test/hive/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 b/src/test/hive/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 new file mode 100644 index 0000000000000..051ca3d3c28e7 --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 @@ -0,0 +1,2 @@ +2000-01-01 +2013-08-08 diff --git a/src/test/hive/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e b/src/test/hive/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e new file mode 100644 index 0000000000000..24192eefd2caf --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e @@ -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/src/test/hive/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b b/src/test/hive/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b @@ -0,0 +1 @@ +15 diff --git a/src/test/hive/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 b/src/test/hive/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/src/test/hive/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 @@ -0,0 +1 @@ +5 diff --git a/src/test/hive/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 b/src/test/hive/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 b/src/test/hive/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf b/src/test/hive/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 b/src/test/hive/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde b/src/test/hive/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde new file mode 100644 index 0000000000000..39c80f1b77eab --- /dev/null +++ b/src/test/hive/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde @@ -0,0 +1,1000 @@ +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/src/test/hive/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 b/src/test/hive/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 b/src/test/hive/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/src/test/hive/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a new file mode 100644 index 0000000000000..dcf40c875b9ad --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a @@ -0,0 +1,15 @@ +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/src/test/hive/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 b/src/test/hive/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 @@ -0,0 +1,25 @@ +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/src/test/hive/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 b/src/test/hive/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 @@ -0,0 +1,25 @@ +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/src/test/hive/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 b/src/test/hive/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 new file mode 100644 index 0000000000000..9639a1e84c615 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 @@ -0,0 +1,50 @@ +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/src/test/hive/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f b/src/test/hive/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff b/src/test/hive/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 b/src/test/hive/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 new file mode 100644 index 0000000000000..707036ebbc76c --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 @@ -0,0 +1,15 @@ +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/src/test/hive/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 b/src/test/hive/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 new file mode 100644 index 0000000000000..06316a924c38b --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 @@ -0,0 +1,15 @@ +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/src/test/hive/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/src/test/hive/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a new file mode 100644 index 0000000000000..735d41e54e958 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a @@ -0,0 +1,15 @@ +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/src/test/hive/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 b/src/test/hive/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 new file mode 100644 index 0000000000000..27dfc622c9bb6 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 @@ -0,0 +1,15 @@ +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/src/test/hive/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 b/src/test/hive/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 new file mode 100644 index 0000000000000..1812e0d53e443 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 @@ -0,0 +1,15 @@ +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/src/test/hive/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 b/src/test/hive/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 @@ -0,0 +1,25 @@ +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/src/test/hive/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 b/src/test/hive/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 @@ -0,0 +1,25 @@ +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/src/test/hive/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a b/src/test/hive/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a @@ -0,0 +1,25 @@ +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/src/test/hive/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 b/src/test/hive/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 new file mode 100644 index 0000000000000..fca99e91bbd8f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 @@ -0,0 +1,75 @@ +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/src/test/hive/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f b/src/test/hive/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f new file mode 100644 index 0000000000000..fca99e91bbd8f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f @@ -0,0 +1,75 @@ +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/src/test/hive/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 b/src/test/hive/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 new file mode 100644 index 0000000000000..3532257511613 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 @@ -0,0 +1,15 @@ +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/src/test/hive/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 b/src/test/hive/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 @@ -0,0 +1,25 @@ +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/src/test/hive/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 b/src/test/hive/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 @@ -0,0 +1,25 @@ +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/src/test/hive/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 b/src/test/hive/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a b/src/test/hive/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 b/src/test/hive/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 new file mode 100644 index 0000000000000..a02458b88bf52 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 @@ -0,0 +1,15 @@ +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/src/test/hive/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 b/src/test/hive/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 new file mode 100644 index 0000000000000..301bff44316ff --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 @@ -0,0 +1,15 @@ +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/src/test/hive/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 b/src/test/hive/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 b/src/test/hive/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e b/src/test/hive/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e new file mode 100644 index 0000000000000..001841c8a1cd4 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf b/src/test/hive/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf new file mode 100644 index 0000000000000..63f35fd827de3 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf @@ -0,0 +1,2 @@ +476.0 val_238 +476.0 val_238 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 b/src/test/hive/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf b/src/test/hive/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf new file mode 100644 index 0000000000000..63f35fd827de3 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf @@ -0,0 +1,2 @@ +476.0 val_238 +476.0 val_238 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e b/src/test/hive/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e new file mode 100644 index 0000000000000..f75126345f351 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e @@ -0,0 +1,2 @@ +238 val_238 NULL 1 +238 val_238 NULL 1 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 b/src/test/hive/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..11542e84a3768 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 NULL 1 +238 val_238 NULL 1 diff --git a/src/test/hive/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 b/src/test/hive/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 b/src/test/hive/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/src/test/hive/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..53aab40f88b50 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476.0 val_238 +476.0 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f b/src/test/hive/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/src/test/hive/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba b/src/test/hive/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 b/src/test/hive/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba b/src/test/hive/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..1b97612ce3f5a --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,4 @@ +476 val_238 +476 val_238 +194 val_97 +194 val_97 diff --git a/src/test/hive/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..f8e13d5235028 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,4 @@ +238 val_238 NULL 1 +238 val_238 NULL 1 +97 val_97 NULL 2 +97 val_97 NULL 2 diff --git a/src/test/hive/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 b/src/test/hive/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 b/src/test/hive/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 new file mode 100644 index 0000000000000..919a66a94c5cb --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..45c169f0d8330 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,6 @@ +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/src/test/hive/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 b/src/test/hive/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 b/src/test/hive/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/src/test/hive/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..53aab40f88b50 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476.0 val_238 +476.0 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f b/src/test/hive/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/src/test/hive/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d b/src/test/hive/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 b/src/test/hive/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 b/src/test/hive/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 new file mode 100644 index 0000000000000..b6a7d89c68e0c --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 @@ -0,0 +1 @@ +16 diff --git a/src/test/hive/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 b/src/test/hive/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 b/src/test/hive/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af b/src/test/hive/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b b/src/test/hive/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 b/src/test/hive/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 b/src/test/hive/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b b/src/test/hive/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d b/src/test/hive/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d new file mode 100644 index 0000000000000..b6a7d89c68e0c --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d @@ -0,0 +1 @@ +16 diff --git a/src/test/hive/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 b/src/test/hive/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe b/src/test/hive/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 b/src/test/hive/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 new file mode 100644 index 0000000000000..425151f3a411f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 @@ -0,0 +1 @@ +40 diff --git a/src/test/hive/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 b/src/test/hive/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 new file mode 100644 index 0000000000000..425151f3a411f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 @@ -0,0 +1 @@ +40 diff --git a/src/test/hive/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 b/src/test/hive/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 new file mode 100644 index 0000000000000..425151f3a411f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 @@ -0,0 +1 @@ +40 diff --git a/src/test/hive/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 b/src/test/hive/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 new file mode 100644 index 0000000000000..c739b42c4d2ce --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 @@ -0,0 +1 @@ +44 diff --git a/src/test/hive/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 b/src/test/hive/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 b/src/test/hive/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 b/src/test/hive/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c b/src/test/hive/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 b/src/test/hive/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 b/src/test/hive/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b b/src/test/hive/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 b/src/test/hive/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..11542e84a3768 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 NULL 1 +238 val_238 NULL 1 diff --git a/src/test/hive/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 b/src/test/hive/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 b/src/test/hive/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 new file mode 100644 index 0000000000000..025abe4ec330b --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 @@ -0,0 +1,3 @@ +476 val_238 NULL 1 +476 val_238 NULL 1 +172 val_86 val_86 2 diff --git a/src/test/hive/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..f067c1ed602dc --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,3 @@ +238 val_238 NULL 1 +238 val_238 NULL 1 +86 val_86 val_86 2 diff --git a/src/test/hive/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b b/src/test/hive/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/src/test/hive/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..53aab40f88b50 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476.0 val_238 +476.0 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f b/src/test/hive/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/src/test/hive/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba b/src/test/hive/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 b/src/test/hive/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..11542e84a3768 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 NULL 1 +238 val_238 NULL 1 diff --git a/src/test/hive/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 b/src/test/hive/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 b/src/test/hive/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 new file mode 100644 index 0000000000000..025abe4ec330b --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 @@ -0,0 +1,3 @@ +476 val_238 NULL 1 +476 val_238 NULL 1 +172 val_86 val_86 2 diff --git a/src/test/hive/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..f067c1ed602dc --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,3 @@ +238 val_238 NULL 1 +238 val_238 NULL 1 +86 val_86 val_86 2 diff --git a/src/test/hive/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b b/src/test/hive/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/src/test/hive/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..53aab40f88b50 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476.0 val_238 +476.0 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f b/src/test/hive/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd b/src/test/hive/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/src/test/hive/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba b/src/test/hive/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b b/src/test/hive/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/src/test/hive/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 b/src/test/hive/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 b/src/test/hive/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 b/src/test/hive/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 b/src/test/hive/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d b/src/test/hive/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 b/src/test/hive/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 new file mode 100644 index 0000000000000..325e26b3d9737 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 @@ -0,0 +1,75 @@ +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/src/test/hive/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 b/src/test/hive/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a b/src/test/hive/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f b/src/test/hive/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff b/src/test/hive/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 b/src/test/hive/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d b/src/test/hive/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 b/src/test/hive/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 new file mode 100644 index 0000000000000..325e26b3d9737 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 @@ -0,0 +1,75 @@ +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/src/test/hive/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada b/src/test/hive/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 b/src/test/hive/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 b/src/test/hive/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 b/src/test/hive/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 new file mode 100644 index 0000000000000..79c8f8e614a1f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 @@ -0,0 +1,25 @@ +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/src/test/hive/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a b/src/test/hive/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/src/test/hive/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a new file mode 100644 index 0000000000000..f487b340cd55b --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a @@ -0,0 +1,14 @@ +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/src/test/hive/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f b/src/test/hive/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff b/src/test/hive/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 b/src/test/hive/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 new file mode 100644 index 0000000000000..0c7739c2b9fbb --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a b/src/test/hive/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a new file mode 100644 index 0000000000000..79c8f8e614a1f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a @@ -0,0 +1,25 @@ +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/src/test/hive/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a b/src/test/hive/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/src/test/hive/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a new file mode 100644 index 0000000000000..0c6fbc997de7a --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a @@ -0,0 +1,14 @@ +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/src/test/hive/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 b/src/test/hive/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 b/src/test/hive/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 b/src/test/hive/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 b/src/test/hive/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 b/src/test/hive/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 b/src/test/hive/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 b/src/test/hive/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 b/src/test/hive/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a b/src/test/hive/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f b/src/test/hive/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff b/src/test/hive/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 b/src/test/hive/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 new file mode 100644 index 0000000000000..f0d140e18c053 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 @@ -0,0 +1,2 @@ +101 25 +102 25 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad b/src/test/hive/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 b/src/test/hive/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 new file mode 100644 index 0000000000000..6425bae2a6a8f --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 @@ -0,0 +1,3 @@ +101 25 +102 25 +103 25 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 b/src/test/hive/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 b/src/test/hive/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a b/src/test/hive/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f b/src/test/hive/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff b/src/test/hive/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f b/src/test/hive/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 b/src/test/hive/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 b/src/test/hive/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 b/src/test/hive/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a b/src/test/hive/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 b/src/test/hive/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 b/src/test/hive/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 b/src/test/hive/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 b/src/test/hive/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 b/src/test/hive/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 b/src/test/hive/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e b/src/test/hive/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e b/src/test/hive/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b b/src/test/hive/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b new file mode 100644 index 0000000000000..5e06930239fea --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b @@ -0,0 +1,20 @@ +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/src/test/hive/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd b/src/test/hive/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd new file mode 100644 index 0000000000000..1bd9063a9c63c --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd @@ -0,0 +1,20 @@ +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/src/test/hive/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf b/src/test/hive/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 b/src/test/hive/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 b/src/test/hive/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 b/src/test/hive/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 b/src/test/hive/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b b/src/test/hive/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b new file mode 100644 index 0000000000000..f259b4eefc608 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b @@ -0,0 +1,20 @@ +0 val_0 1 +0 val_0 1 +0 val_0 1 +0 val_0 2 +0 val_0 2 +0 val_0 2 +10 val_10 1 +10 val_10 2 +100 val_100 1 +100 val_100 1 +100 val_100 2 +100 val_100 2 +103 val_103 1 +103 val_103 1 +103 val_103 2 +103 val_103 2 +104 val_104 1 +104 val_104 1 +104 val_104 2 +104 val_104 2 \ No newline at end of file diff --git a/src/test/hive/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd b/src/test/hive/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd new file mode 100644 index 0000000000000..89afce150e7a9 --- /dev/null +++ b/src/test/hive/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd @@ -0,0 +1,20 @@ +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 +4.0 val_2 1 +4.0 val_2 2 +8.0 val_4 1 +8.0 val_4 2 +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 +10.0 val_5 2 +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 diff --git a/src/test/hive/golden/plan_json-0-74146da55d57b22443140e7fbab3375c b/src/test/hive/golden/plan_json-0-74146da55d57b22443140e7fbab3375c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 b/src/test/hive/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 b/src/test/hive/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 new file mode 100644 index 0000000000000..55b2a1b47afa2 --- /dev/null +++ b/src/test/hive/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 @@ -0,0 +1,496 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 \ No newline at end of file diff --git a/src/test/hive/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 b/src/test/hive/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 b/src/test/hive/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 new file mode 100644 index 0000000000000..55b2a1b47afa2 --- /dev/null +++ b/src/test/hive/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 @@ -0,0 +1,496 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 \ No newline at end of file diff --git a/src/test/hive/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 b/src/test/hive/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b b/src/test/hive/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b new file mode 100644 index 0000000000000..878c86cb46b8a --- /dev/null +++ b/src/test/hive/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b @@ -0,0 +1,136 @@ +0 3 +5 3 +12 2 +15 2 +18 2 +24 2 +26 2 +35 3 +37 2 +42 2 +51 2 +58 2 +67 2 +70 3 +72 2 +76 2 +83 2 +84 2 +90 3 +95 2 +97 2 +98 2 +100 2 +103 2 +104 2 +113 2 +118 2 +119 3 +120 2 +125 2 +128 3 +129 2 +134 2 +137 2 +138 4 +146 2 +149 2 +152 2 +164 2 +165 2 +167 3 +169 4 +172 2 +174 2 +175 2 +176 2 +179 2 +187 3 +191 2 +193 3 +195 2 +197 2 +199 3 +200 2 +203 2 +205 2 +207 2 +208 3 +209 2 +213 2 +216 2 +217 2 +219 2 +221 2 +223 2 +224 2 +229 2 +230 5 +233 2 +237 2 +238 2 +239 2 +242 2 +255 2 +256 2 +265 2 +272 2 +273 3 +277 4 +278 2 +280 2 +281 2 +282 2 +288 2 +298 3 +307 2 +309 2 +311 3 +316 3 +317 2 +318 3 +321 2 +322 2 +325 2 +327 3 +331 2 +333 2 +342 2 +344 2 +348 5 +353 2 +367 2 +369 3 +382 2 +384 3 +395 2 +396 3 +397 2 +399 2 +401 5 +403 3 +404 2 +406 4 +409 3 +413 2 +414 2 +417 3 +424 2 +429 2 +430 3 +431 3 +438 3 +439 2 +454 3 +458 2 +459 2 +462 2 +463 2 +466 3 +468 4 +469 5 +478 2 +480 3 +489 4 +492 2 +498 3 \ No newline at end of file diff --git a/src/test/hive/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 b/src/test/hive/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 b/src/test/hive/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 b/src/test/hive/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b b/src/test/hive/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b new file mode 100644 index 0000000000000..878c86cb46b8a --- /dev/null +++ b/src/test/hive/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b @@ -0,0 +1,136 @@ +0 3 +5 3 +12 2 +15 2 +18 2 +24 2 +26 2 +35 3 +37 2 +42 2 +51 2 +58 2 +67 2 +70 3 +72 2 +76 2 +83 2 +84 2 +90 3 +95 2 +97 2 +98 2 +100 2 +103 2 +104 2 +113 2 +118 2 +119 3 +120 2 +125 2 +128 3 +129 2 +134 2 +137 2 +138 4 +146 2 +149 2 +152 2 +164 2 +165 2 +167 3 +169 4 +172 2 +174 2 +175 2 +176 2 +179 2 +187 3 +191 2 +193 3 +195 2 +197 2 +199 3 +200 2 +203 2 +205 2 +207 2 +208 3 +209 2 +213 2 +216 2 +217 2 +219 2 +221 2 +223 2 +224 2 +229 2 +230 5 +233 2 +237 2 +238 2 +239 2 +242 2 +255 2 +256 2 +265 2 +272 2 +273 3 +277 4 +278 2 +280 2 +281 2 +282 2 +288 2 +298 3 +307 2 +309 2 +311 3 +316 3 +317 2 +318 3 +321 2 +322 2 +325 2 +327 3 +331 2 +333 2 +342 2 +344 2 +348 5 +353 2 +367 2 +369 3 +382 2 +384 3 +395 2 +396 3 +397 2 +399 2 +401 5 +403 3 +404 2 +406 4 +409 3 +413 2 +414 2 +417 3 +424 2 +429 2 +430 3 +431 3 +438 3 +439 2 +454 3 +458 2 +459 2 +462 2 +463 2 +466 3 +468 4 +469 5 +478 2 +480 3 +489 4 +492 2 +498 3 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 b/src/test/hive/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 new file mode 100644 index 0000000000000..e2ef71e141c46 --- /dev/null +++ b/src/test/hive/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 @@ -0,0 +1 @@ +20 val_20 20 diff --git a/src/test/hive/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 b/src/test/hive/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 b/src/test/hive/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 new file mode 100644 index 0000000000000..ae8b7551d9630 --- /dev/null +++ b/src/test/hive/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 @@ -0,0 +1 @@ +10 val_10 diff --git a/src/test/hive/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 b/src/test/hive/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 b/src/test/hive/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 new file mode 100644 index 0000000000000..e2ef71e141c46 --- /dev/null +++ b/src/test/hive/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 @@ -0,0 +1 @@ +20 val_20 20 diff --git a/src/test/hive/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 b/src/test/hive/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 b/src/test/hive/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 new file mode 100644 index 0000000000000..ae8b7551d9630 --- /dev/null +++ b/src/test/hive/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 @@ -0,0 +1 @@ +10 val_10 diff --git a/src/test/hive/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 b/src/test/hive/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 b/src/test/hive/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 b/src/test/hive/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 b/src/test/hive/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c b/src/test/hive/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c new file mode 100644 index 0000000000000..c87107a2f1168 --- /dev/null +++ b/src/test/hive/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c @@ -0,0 +1,25 @@ +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 b/src/test/hive/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 b/src/test/hive/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c b/src/test/hive/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c new file mode 100644 index 0000000000000..c87107a2f1168 --- /dev/null +++ b/src/test/hive/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c @@ -0,0 +1,25 @@ +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 b/src/test/hive/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 b/src/test/hive/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 new file mode 100644 index 0000000000000..e37d32abba426 --- /dev/null +++ b/src/test/hive/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 @@ -0,0 +1 @@ +1000 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e b/src/test/hive/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d b/src/test/hive/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d new file mode 100644 index 0000000000000..e1659e6072577 --- /dev/null +++ b/src/test/hive/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d @@ -0,0 +1,129 @@ +val_201 +val_202 +val_203 +val_205 +val_207 +val_208 +val_209 +val_213 +val_214 +val_216 +val_217 +val_218 +val_219 +val_221 +val_222 +val_223 +val_224 +val_226 +val_228 +val_229 +val_230 +val_233 +val_235 +val_237 +val_238 +val_239 +val_24 +val_241 +val_242 +val_244 +val_247 +val_248 +val_249 +val_252 +val_255 +val_256 +val_257 +val_258 +val_26 +val_260 +val_262 +val_263 +val_265 +val_266 +val_27 +val_272 +val_273 +val_274 +val_275 +val_277 +val_278 +val_28 +val_280 +val_281 +val_282 +val_283 +val_284 +val_285 +val_286 +val_287 +val_288 +val_289 +val_291 +val_292 +val_296 +val_298 +val_30 +val_302 +val_305 +val_306 +val_307 +val_308 +val_309 +val_310 +val_311 +val_315 +val_316 +val_317 +val_318 +val_321 +val_322 +val_323 +val_325 +val_327 +val_33 +val_331 +val_332 +val_333 +val_335 +val_336 +val_338 +val_339 +val_34 +val_341 +val_342 +val_344 +val_345 +val_348 +val_35 +val_351 +val_353 +val_356 +val_360 +val_362 +val_364 +val_365 +val_366 +val_367 +val_368 +val_369 +val_37 +val_373 +val_374 +val_375 +val_377 +val_378 +val_379 +val_382 +val_384 +val_386 +val_389 +val_392 +val_393 +val_394 +val_395 +val_396 +val_397 +val_399 +val_4 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e b/src/test/hive/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d b/src/test/hive/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d new file mode 100644 index 0000000000000..e1659e6072577 --- /dev/null +++ b/src/test/hive/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d @@ -0,0 +1,129 @@ +val_201 +val_202 +val_203 +val_205 +val_207 +val_208 +val_209 +val_213 +val_214 +val_216 +val_217 +val_218 +val_219 +val_221 +val_222 +val_223 +val_224 +val_226 +val_228 +val_229 +val_230 +val_233 +val_235 +val_237 +val_238 +val_239 +val_24 +val_241 +val_242 +val_244 +val_247 +val_248 +val_249 +val_252 +val_255 +val_256 +val_257 +val_258 +val_26 +val_260 +val_262 +val_263 +val_265 +val_266 +val_27 +val_272 +val_273 +val_274 +val_275 +val_277 +val_278 +val_28 +val_280 +val_281 +val_282 +val_283 +val_284 +val_285 +val_286 +val_287 +val_288 +val_289 +val_291 +val_292 +val_296 +val_298 +val_30 +val_302 +val_305 +val_306 +val_307 +val_308 +val_309 +val_310 +val_311 +val_315 +val_316 +val_317 +val_318 +val_321 +val_322 +val_323 +val_325 +val_327 +val_33 +val_331 +val_332 +val_333 +val_335 +val_336 +val_338 +val_339 +val_34 +val_341 +val_342 +val_344 +val_345 +val_348 +val_35 +val_351 +val_353 +val_356 +val_360 +val_362 +val_364 +val_365 +val_366 +val_367 +val_368 +val_369 +val_37 +val_373 +val_374 +val_375 +val_377 +val_378 +val_379 +val_382 +val_384 +val_386 +val_389 +val_392 +val_393 +val_394 +val_395 +val_396 +val_397 +val_399 +val_4 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 b/src/test/hive/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f b/src/test/hive/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f new file mode 100644 index 0000000000000..7e66866e2dd60 --- /dev/null +++ b/src/test/hive/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f @@ -0,0 +1,5 @@ +val_4 1 +val_399 2 +val_396 3 +val_277 4 +val_348 5 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 b/src/test/hive/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f b/src/test/hive/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f new file mode 100644 index 0000000000000..7e66866e2dd60 --- /dev/null +++ b/src/test/hive/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f @@ -0,0 +1,5 @@ +val_4 1 +val_399 2 +val_396 3 +val_277 4 +val_348 5 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 b/src/test/hive/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 b/src/test/hive/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 b/src/test/hive/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 b/src/test/hive/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 new file mode 100644 index 0000000000000..fbce9efa766d7 --- /dev/null +++ b/src/test/hive/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 @@ -0,0 +1,712 @@ +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +374 val_374 +152 val_152 +152 val_152 +145 val_145 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +82 val_82 +166 val_166 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +394 val_394 +237 val_237 +237 val_237 +174 val_174 +174 val_174 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +125 val_125 +125 val_125 +386 val_386 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +54 val_54 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +260 val_260 +384 val_384 +384 val_384 +384 val_384 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +58 val_58 +58 val_58 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +393 val_393 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +96 val_96 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +43 val_43 +156 val_156 +308 val_308 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +85 val_85 +77 val_77 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +242 val_242 +242 val_242 +177 val_177 +226 val_226 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +322 val_322 +322 val_322 +160 val_160 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +119 val_119 +119 val_119 +119 val_119 +78 val_78 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +149 val_149 +149 val_149 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +30 val_30 +209 val_209 +209 val_209 +64 val_64 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +244 val_244 +128 val_128 +128 val_128 +128 val_128 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +366 val_366 +175 val_175 +175 val_175 +53 val_53 +105 val_105 +257 val_257 +190 val_190 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +164 val_164 +164 val_164 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +291 val_291 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +119 val_119 +119 val_119 +119 val_119 +200 val_200 +200 val_200 +237 val_237 +237 val_237 +360 val_360 +248 val_248 +305 val_305 +199 val_199 +199 val_199 +199 val_199 +120 val_120 +120 val_120 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +178 val_178 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +233 val_233 +233 val_233 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +375 val_375 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +341 val_341 +146 val_146 +146 val_146 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +315 val_315 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +200 val_200 +200 val_200 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 b/src/test/hive/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 b/src/test/hive/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 new file mode 100644 index 0000000000000..fbce9efa766d7 --- /dev/null +++ b/src/test/hive/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 @@ -0,0 +1,712 @@ +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +374 val_374 +152 val_152 +152 val_152 +145 val_145 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +82 val_82 +166 val_166 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +394 val_394 +237 val_237 +237 val_237 +174 val_174 +174 val_174 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +125 val_125 +125 val_125 +386 val_386 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +54 val_54 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +260 val_260 +384 val_384 +384 val_384 +384 val_384 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +58 val_58 +58 val_58 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +393 val_393 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +96 val_96 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +43 val_43 +156 val_156 +308 val_308 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +85 val_85 +77 val_77 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +242 val_242 +242 val_242 +177 val_177 +226 val_226 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +322 val_322 +322 val_322 +160 val_160 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +119 val_119 +119 val_119 +119 val_119 +78 val_78 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +149 val_149 +149 val_149 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +30 val_30 +209 val_209 +209 val_209 +64 val_64 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +244 val_244 +128 val_128 +128 val_128 +128 val_128 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +366 val_366 +175 val_175 +175 val_175 +53 val_53 +105 val_105 +257 val_257 +190 val_190 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +164 val_164 +164 val_164 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +291 val_291 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +119 val_119 +119 val_119 +119 val_119 +200 val_200 +200 val_200 +237 val_237 +237 val_237 +360 val_360 +248 val_248 +305 val_305 +199 val_199 +199 val_199 +199 val_199 +120 val_120 +120 val_120 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +178 val_178 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +233 val_233 +233 val_233 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +375 val_375 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +341 val_341 +146 val_146 +146 val_146 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +315 val_315 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +200 val_200 +200 val_200 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d b/src/test/hive/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 b/src/test/hive/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 new file mode 100644 index 0000000000000..9e6c4359e78a8 --- /dev/null +++ b/src/test/hive/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 @@ -0,0 +1,1710 @@ +238 val_238 +238 val_238 +238 val_238 +238 val_238 +86 val_86 +27 val_27 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +374 val_374 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +145 val_145 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +82 val_82 +166 val_166 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +394 val_394 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +17 val_17 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +339 val_339 +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 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +345 val_345 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +170 val_170 +20 val_20 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +4 val_4 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +386 val_386 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +54 val_54 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +275 val_275 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +260 val_260 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +8 val_8 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +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 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +393 val_393 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +96 val_96 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +43 val_43 +156 val_156 +308 val_308 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +85 val_85 +77 val_77 +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 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +19 val_19 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +177 val_177 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +143 val_143 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +160 val_160 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +78 val_78 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +30 val_30 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +64 val_64 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +116 val_116 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +366 val_366 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +53 val_53 +105 val_105 +257 val_257 +190 val_190 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +291 val_291 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +360 val_360 +248 val_248 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +323 val_323 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +178 val_178 +310 val_310 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +214 val_214 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +133 val_133 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +189 val_189 +375 val_375 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +341 val_341 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +315 val_315 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +194 val_194 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +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 diff --git a/src/test/hive/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d b/src/test/hive/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 b/src/test/hive/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 new file mode 100644 index 0000000000000..9e6c4359e78a8 --- /dev/null +++ b/src/test/hive/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 @@ -0,0 +1,1710 @@ +238 val_238 +238 val_238 +238 val_238 +238 val_238 +86 val_86 +27 val_27 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +374 val_374 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +145 val_145 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +82 val_82 +166 val_166 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +394 val_394 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +17 val_17 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +339 val_339 +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 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +345 val_345 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +170 val_170 +20 val_20 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +4 val_4 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +386 val_386 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +54 val_54 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +275 val_275 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +260 val_260 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +8 val_8 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +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 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +393 val_393 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +96 val_96 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +43 val_43 +156 val_156 +308 val_308 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +85 val_85 +77 val_77 +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 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +19 val_19 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +177 val_177 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +143 val_143 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +160 val_160 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +78 val_78 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +30 val_30 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +64 val_64 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +116 val_116 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +366 val_366 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +53 val_53 +105 val_105 +257 val_257 +190 val_190 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +291 val_291 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +360 val_360 +248 val_248 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +323 val_323 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +178 val_178 +310 val_310 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +214 val_214 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +133 val_133 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +189 val_189 +375 val_375 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +341 val_341 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +315 val_315 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +194 val_194 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +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 diff --git a/src/test/hive/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 b/src/test/hive/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b b/src/test/hive/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b new file mode 100644 index 0000000000000..0e11aea14d61d --- /dev/null +++ b/src/test/hive/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b @@ -0,0 +1,1767 @@ +238 val_238 +238 val_238 +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +374 val_374 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +145 val_145 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +82 val_82 +166 val_166 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +394 val_394 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +17 val_17 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +339 val_339 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +345 val_345 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +170 val_170 +20 val_20 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +386 val_386 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +54 val_54 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +260 val_260 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +8 val_8 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +393 val_393 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +96 val_96 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +43 val_43 +156 val_156 +308 val_308 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +85 val_85 +77 val_77 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +19 val_19 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +177 val_177 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +143 val_143 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +160 val_160 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +78 val_78 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +30 val_30 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +64 val_64 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +116 val_116 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +366 val_366 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +53 val_53 +105 val_105 +257 val_257 +190 val_190 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +291 val_291 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +360 val_360 +248 val_248 +305 val_305 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +323 val_323 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +178 val_178 +310 val_310 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +214 val_214 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +133 val_133 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +189 val_189 +375 val_375 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +341 val_341 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +315 val_315 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +194 val_194 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +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 diff --git a/src/test/hive/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 b/src/test/hive/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b b/src/test/hive/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b new file mode 100644 index 0000000000000..0e11aea14d61d --- /dev/null +++ b/src/test/hive/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b @@ -0,0 +1,1767 @@ +238 val_238 +238 val_238 +238 val_238 +238 val_238 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +374 val_374 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +145 val_145 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +82 val_82 +166 val_166 +252 val_252 +292 val_292 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +394 val_394 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +162 val_162 +377 val_377 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +365 val_365 +266 val_266 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +17 val_17 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +155 val_155 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +339 val_339 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +345 val_345 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +170 val_170 +20 val_20 +157 val_157 +378 val_378 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +386 val_386 +192 val_192 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +54 val_54 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +289 val_289 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +241 val_241 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +180 val_180 +284 val_284 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +260 val_260 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +373 val_373 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +8 val_8 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +393 val_393 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +96 val_96 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +131 val_131 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +43 val_43 +156 val_156 +308 val_308 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +196 val_196 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +85 val_85 +77 val_77 +87 val_87 +364 val_364 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +19 val_19 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +177 val_177 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +168 val_168 +34 val_34 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +143 val_143 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +160 val_160 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +78 val_78 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +41 val_41 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +218 val_218 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +30 val_30 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +64 val_64 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +74 val_74 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +296 val_296 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +116 val_116 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +335 val_335 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +366 val_366 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +53 val_53 +105 val_105 +257 val_257 +190 val_190 +114 val_114 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +291 val_291 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +351 val_351 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +360 val_360 +248 val_248 +305 val_305 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +323 val_323 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +178 val_178 +310 val_310 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +249 val_249 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +136 val_136 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +214 val_214 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +133 val_133 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +189 val_189 +375 val_375 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +341 val_341 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +315 val_315 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +28 val_28 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +194 val_194 +222 val_222 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +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 diff --git a/src/test/hive/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 b/src/test/hive/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 b/src/test/hive/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 new file mode 100644 index 0000000000000..62127cc213b1d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 @@ -0,0 +1,4 @@ +0 2 3 +0 2 3 +0 2 3 +2 4 5 diff --git a/src/test/hive/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 b/src/test/hive/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 b/src/test/hive/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 b/src/test/hive/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 new file mode 100644 index 0000000000000..62127cc213b1d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 @@ -0,0 +1,4 @@ +0 2 3 +0 2 3 +0 2 3 +2 4 5 diff --git a/src/test/hive/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 b/src/test/hive/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 b/src/test/hive/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 new file mode 100644 index 0000000000000..62127cc213b1d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 @@ -0,0 +1,4 @@ +0 2 3 +0 2 3 +0 2 3 +2 4 5 diff --git a/src/test/hive/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 b/src/test/hive/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 b/src/test/hive/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 new file mode 100644 index 0000000000000..62127cc213b1d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 @@ -0,0 +1,4 @@ +0 2 3 +0 2 3 +0 2 3 +2 4 5 diff --git a/src/test/hive/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 b/src/test/hive/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e b/src/test/hive/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 b/src/test/hive/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 new file mode 100644 index 0000000000000..997f37b76bedb --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 @@ -0,0 +1,6 @@ +17 val_17 17 val_17 +19 val_19 19 val_19 +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 diff --git a/src/test/hive/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e b/src/test/hive/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 b/src/test/hive/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 new file mode 100644 index 0000000000000..997f37b76bedb --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 @@ -0,0 +1,6 @@ +17 val_17 17 val_17 +19 val_19 19 val_19 +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 diff --git a/src/test/hive/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 b/src/test/hive/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 b/src/test/hive/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 new file mode 100644 index 0000000000000..997f37b76bedb --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 @@ -0,0 +1,6 @@ +17 val_17 17 val_17 +19 val_19 19 val_19 +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 diff --git a/src/test/hive/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 b/src/test/hive/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 b/src/test/hive/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 new file mode 100644 index 0000000000000..997f37b76bedb --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 @@ -0,0 +1,6 @@ +17 val_17 17 val_17 +19 val_19 19 val_19 +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 diff --git a/src/test/hive/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 b/src/test/hive/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 b/src/test/hive/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 new file mode 100644 index 0000000000000..b3bf95dd32bac --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 @@ -0,0 +1,6 @@ +17 val_17 17 val_17 +18 val_18 18 val_18 +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 diff --git a/src/test/hive/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 b/src/test/hive/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 b/src/test/hive/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 new file mode 100644 index 0000000000000..b3bf95dd32bac --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 @@ -0,0 +1,6 @@ +17 val_17 17 val_17 +18 val_18 18 val_18 +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 diff --git a/src/test/hive/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 b/src/test/hive/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe b/src/test/hive/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe new file mode 100644 index 0000000000000..d4a363c49aeaf --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe @@ -0,0 +1,10 @@ +17 val_17 17 val_17 17 +19 val_19 19 val_19 19 +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 +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 diff --git a/src/test/hive/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 b/src/test/hive/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe b/src/test/hive/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe new file mode 100644 index 0000000000000..d4a363c49aeaf --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe @@ -0,0 +1,10 @@ +17 val_17 17 val_17 17 +19 val_19 19 val_19 19 +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 +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 diff --git a/src/test/hive/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b b/src/test/hive/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 b/src/test/hive/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 b/src/test/hive/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee b/src/test/hive/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 b/src/test/hive/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 b/src/test/hive/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 b/src/test/hive/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a b/src/test/hive/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd b/src/test/hive/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c b/src/test/hive/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 b/src/test/hive/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b b/src/test/hive/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b b/src/test/hive/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 b/src/test/hive/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 b/src/test/hive/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 b/src/test/hive/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 b/src/test/hive/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f b/src/test/hive/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 b/src/test/hive/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 b/src/test/hive/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf b/src/test/hive/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 b/src/test/hive/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 b/src/test/hive/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/src/test/hive/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 b/src/test/hive/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 new file mode 100644 index 0000000000000..0190981db84ed --- /dev/null +++ b/src/test/hive/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 @@ -0,0 +1,84 @@ +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/src/test/hive/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/src/test/hive/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 b/src/test/hive/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 new file mode 100644 index 0000000000000..0190981db84ed --- /dev/null +++ b/src/test/hive/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 @@ -0,0 +1,84 @@ +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/src/test/hive/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 b/src/test/hive/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 b/src/test/hive/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e b/src/test/hive/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed b/src/test/hive/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae b/src/test/hive/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 b/src/test/hive/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e b/src/test/hive/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed b/src/test/hive/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae b/src/test/hive/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 b/src/test/hive/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 b/src/test/hive/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union-3-678c8197f458b459171c266f7431683e b/src/test/hive/golden/ppd_union-3-678c8197f458b459171c266f7431683e new file mode 100644 index 0000000000000..4c3919232c73e --- /dev/null +++ b/src/test/hive/golden/ppd_union-3-678c8197f458b459171c266f7431683e @@ -0,0 +1,171 @@ +86 val_86 +409 val_409 +98 val_98 +484 val_484 +401 val_401 +66 val_66 +406 val_406 +429 val_429 +469 val_469 +495 val_495 +82 val_82 +403 val_403 +417 val_417 +430 val_430 +446 val_446 +459 val_459 +482 val_482 +413 val_413 +494 val_494 +466 val_466 +417 val_417 +489 val_489 +439 val_439 +475 val_475 +455 val_455 +57 val_57 +438 val_438 +489 val_489 +92 val_92 +47 val_47 +72 val_72 +427 val_427 +498 val_498 +437 val_437 +469 val_469 +54 val_54 +459 val_459 +51 val_51 +430 val_430 +65 val_65 +83 val_83 +67 val_67 +404 val_404 +489 val_489 +84 val_84 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +463 val_463 +431 val_431 +42 val_42 +496 val_496 +468 val_468 +454 val_454 +418 val_418 +96 val_96 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +469 val_469 +468 val_468 +95 val_95 +481 val_481 +457 val_457 +98 val_98 +409 val_409 +470 val_470 +413 val_413 +85 val_85 +77 val_77 +490 val_490 +87 val_87 +419 val_419 +72 val_72 +90 val_90 +435 val_435 +401 val_401 +452 val_452 +5 val_5 +497 val_497 +402 val_402 +58 val_58 +95 val_95 +472 val_472 +498 val_498 +42 val_42 +430 val_430 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +492 val_492 +449 val_449 +453 val_453 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +69 val_69 +485 val_485 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +5 val_5 +438 val_438 +467 val_467 +432 val_432 +469 val_469 +463 val_463 +80 val_80 +44 val_44 +466 val_466 +403 val_403 +483 val_483 +53 val_53 +406 val_406 +409 val_409 +406 val_406 +401 val_401 +90 val_90 +424 val_424 +431 val_431 +454 val_454 +478 val_478 +431 val_431 +424 val_424 +5 val_5 +70 val_70 +480 val_480 +70 val_70 +438 val_438 +414 val_414 +491 val_491 +439 val_439 +479 val_479 +417 val_417 +444 val_444 +429 val_429 +443 val_443 +478 val_478 +468 val_468 +493 val_493 +460 val_460 +480 val_480 +83 val_83 +462 val_462 +406 val_406 +454 val_454 +401 val_401 +421 val_421 +407 val_407 +67 val_67 +462 val_462 +492 val_492 +9 val_9 +498 val_498 +458 val_458 +97 val_97 +469 val_469 +84 val_84 +448 val_448 +414 val_414 +477 val_477 +90 val_90 +403 val_403 +400 val_400 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 b/src/test/hive/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 b/src/test/hive/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union-6-678c8197f458b459171c266f7431683e b/src/test/hive/golden/ppd_union-6-678c8197f458b459171c266f7431683e new file mode 100644 index 0000000000000..4c3919232c73e --- /dev/null +++ b/src/test/hive/golden/ppd_union-6-678c8197f458b459171c266f7431683e @@ -0,0 +1,171 @@ +86 val_86 +409 val_409 +98 val_98 +484 val_484 +401 val_401 +66 val_66 +406 val_406 +429 val_429 +469 val_469 +495 val_495 +82 val_82 +403 val_403 +417 val_417 +430 val_430 +446 val_446 +459 val_459 +482 val_482 +413 val_413 +494 val_494 +466 val_466 +417 val_417 +489 val_489 +439 val_439 +475 val_475 +455 val_455 +57 val_57 +438 val_438 +489 val_489 +92 val_92 +47 val_47 +72 val_72 +427 val_427 +498 val_498 +437 val_437 +469 val_469 +54 val_54 +459 val_459 +51 val_51 +430 val_430 +65 val_65 +83 val_83 +67 val_67 +404 val_404 +489 val_489 +84 val_84 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +463 val_463 +431 val_431 +42 val_42 +496 val_496 +468 val_468 +454 val_454 +418 val_418 +96 val_96 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +469 val_469 +468 val_468 +95 val_95 +481 val_481 +457 val_457 +98 val_98 +409 val_409 +470 val_470 +413 val_413 +85 val_85 +77 val_77 +490 val_490 +87 val_87 +419 val_419 +72 val_72 +90 val_90 +435 val_435 +401 val_401 +452 val_452 +5 val_5 +497 val_497 +402 val_402 +58 val_58 +95 val_95 +472 val_472 +498 val_498 +42 val_42 +430 val_430 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +492 val_492 +449 val_449 +453 val_453 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +69 val_69 +485 val_485 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +5 val_5 +438 val_438 +467 val_467 +432 val_432 +469 val_469 +463 val_463 +80 val_80 +44 val_44 +466 val_466 +403 val_403 +483 val_483 +53 val_53 +406 val_406 +409 val_409 +406 val_406 +401 val_401 +90 val_90 +424 val_424 +431 val_431 +454 val_454 +478 val_478 +431 val_431 +424 val_424 +5 val_5 +70 val_70 +480 val_480 +70 val_70 +438 val_438 +414 val_414 +491 val_491 +439 val_439 +479 val_479 +417 val_417 +444 val_444 +429 val_429 +443 val_443 +478 val_478 +468 val_468 +493 val_493 +460 val_460 +480 val_480 +83 val_83 +462 val_462 +406 val_406 +454 val_454 +401 val_401 +421 val_421 +407 val_407 +67 val_67 +462 val_462 +492 val_492 +9 val_9 +498 val_498 +458 val_458 +97 val_97 +469 val_469 +84 val_84 +448 val_448 +414 val_414 +477 val_477 +90 val_90 +403 val_403 +400 val_400 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 b/src/test/hive/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 b/src/test/hive/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f b/src/test/hive/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c b/src/test/hive/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 b/src/test/hive/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 b/src/test/hive/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 b/src/test/hive/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 b/src/test/hive/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 b/src/test/hive/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c b/src/test/hive/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 b/src/test/hive/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c b/src/test/hive/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 b/src/test/hive/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 b/src/test/hive/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 new file mode 100644 index 0000000000000..d183a30ddf3b6 --- /dev/null +++ b/src/test/hive/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 @@ -0,0 +1,36 @@ +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/src/test/hive/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 b/src/test/hive/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 b/src/test/hive/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 new file mode 100644 index 0000000000000..5b2461e35b5f4 --- /dev/null +++ b/src/test/hive/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 @@ -0,0 +1,36 @@ +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/src/test/hive/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d b/src/test/hive/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 b/src/test/hive/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 b/src/test/hive/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d b/src/test/hive/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 b/src/test/hive/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b b/src/test/hive/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 b/src/test/hive/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 b/src/test/hive/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 b/src/test/hive/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 b/src/test/hive/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 b/src/test/hive/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c b/src/test/hive/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb b/src/test/hive/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae b/src/test/hive/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 b/src/test/hive/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 new file mode 100644 index 0000000000000..eb3e1bc1a37f1 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 @@ -0,0 +1,2 @@ +1234 1234 +abcd 1234 diff --git a/src/test/hive/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 b/src/test/hive/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 new file mode 100644 index 0000000000000..7ba18baf99d20 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 @@ -0,0 +1,2 @@ +1224 1224 +abcd 1224 diff --git a/src/test/hive/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a b/src/test/hive/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a new file mode 100644 index 0000000000000..e1d91a4660f36 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a @@ -0,0 +1,2 @@ +1214 1214 +abcd 1214 diff --git a/src/test/hive/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 b/src/test/hive/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 b/src/test/hive/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 new file mode 100644 index 0000000000000..1872f733e0ee7 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 @@ -0,0 +1,2 @@ +12.4 12.4 +abcd 12.4 diff --git a/src/test/hive/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d b/src/test/hive/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d new file mode 100644 index 0000000000000..9fb4a9236e0fe --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d @@ -0,0 +1,2 @@ +12+4 12+4 +abcd 12+4 diff --git a/src/test/hive/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d b/src/test/hive/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d new file mode 100644 index 0000000000000..1c8af4d81713a --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d @@ -0,0 +1,2 @@ +12:4 12:4 +abcd 12:4 diff --git a/src/test/hive/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 b/src/test/hive/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 new file mode 100644 index 0000000000000..422b06c2f2841 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 @@ -0,0 +1,2 @@ +12%4 12%4 +abcd 12%4 diff --git a/src/test/hive/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 b/src/test/hive/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 new file mode 100644 index 0000000000000..51b2c2fdb2887 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 @@ -0,0 +1,2 @@ +12*4 12*4 +abcd 12*4 diff --git a/src/test/hive/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b b/src/test/hive/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 b/src/test/hive/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 new file mode 100644 index 0000000000000..e338e5eb13d50 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 @@ -0,0 +1 @@ +1234 1234 diff --git a/src/test/hive/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 b/src/test/hive/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 new file mode 100644 index 0000000000000..ffc2dc5792f54 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 @@ -0,0 +1 @@ +1224 1224 diff --git a/src/test/hive/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 b/src/test/hive/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 new file mode 100644 index 0000000000000..fcdf68d826db1 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 @@ -0,0 +1 @@ +1214 1214 diff --git a/src/test/hive/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea b/src/test/hive/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea new file mode 100644 index 0000000000000..18abbb411527e --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea @@ -0,0 +1 @@ +12.4 12.4 diff --git a/src/test/hive/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 b/src/test/hive/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 b/src/test/hive/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 new file mode 100644 index 0000000000000..b8286f78807d2 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 @@ -0,0 +1 @@ +12+4 12+4 diff --git a/src/test/hive/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf b/src/test/hive/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf new file mode 100644 index 0000000000000..6ba2e7177ed23 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf @@ -0,0 +1 @@ +12:4 12:4 diff --git a/src/test/hive/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b b/src/test/hive/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b new file mode 100644 index 0000000000000..7152ea7cecfb3 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b @@ -0,0 +1 @@ +12%4 12%4 diff --git a/src/test/hive/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 b/src/test/hive/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 new file mode 100644 index 0000000000000..44880cb42d749 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 @@ -0,0 +1 @@ +12*4 12*4 diff --git a/src/test/hive/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 b/src/test/hive/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de b/src/test/hive/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d b/src/test/hive/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 b/src/test/hive/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c b/src/test/hive/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 b/src/test/hive/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b b/src/test/hive/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 b/src/test/hive/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c b/src/test/hive/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b b/src/test/hive/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a b/src/test/hive/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 b/src/test/hive/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 b/src/test/hive/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 new file mode 100644 index 0000000000000..e3e04ee48543d --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 @@ -0,0 +1 @@ +3 1 2 1 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 b/src/test/hive/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 new file mode 100644 index 0000000000000..c458b0f57aba9 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 @@ -0,0 +1 @@ +2 1 1 2 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb b/src/test/hive/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb new file mode 100644 index 0000000000000..c458b0f57aba9 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb @@ -0,0 +1 @@ +2 1 1 2 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe b/src/test/hive/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe new file mode 100644 index 0000000000000..63511415ddf55 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe @@ -0,0 +1,2 @@ +3 1 2 1 +1 1 2 3 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 b/src/test/hive/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 new file mode 100644 index 0000000000000..c592b5d505b0e --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 @@ -0,0 +1,3 @@ +2 1 1 2 +3 1 2 1 +1 1 2 3 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 b/src/test/hive/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 new file mode 100644 index 0000000000000..e3e04ee48543d --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 @@ -0,0 +1 @@ +3 1 2 1 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e b/src/test/hive/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 b/src/test/hive/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 new file mode 100644 index 0000000000000..e3e04ee48543d --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 @@ -0,0 +1 @@ +3 1 2 1 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c b/src/test/hive/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c new file mode 100644 index 0000000000000..679926f7d738c --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c @@ -0,0 +1 @@ +2 2 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 b/src/test/hive/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 new file mode 100644 index 0000000000000..fcc7be2cb12aa --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 @@ -0,0 +1 @@ +22 22 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 b/src/test/hive/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 b/src/test/hive/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 b/src/test/hive/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 b/src/test/hive/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 new file mode 100644 index 0000000000000..3ebc6d7fa2375 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 @@ -0,0 +1 @@ +2 2 1 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 b/src/test/hive/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 new file mode 100644 index 0000000000000..a4544ab84afa3 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 @@ -0,0 +1 @@ +1 1 2 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a b/src/test/hive/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 b/src/test/hive/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 b/src/test/hive/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 new file mode 100644 index 0000000000000..e22f840876d2e --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 @@ -0,0 +1,40 @@ +0 val_0 2008-04-08 11 +4 val_4 2008-04-08 11 +8 val_8 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 11 +2 val_2 2008-04-08 11 +5 val_5 2008-04-08 11 +9 val_9 2008-04-08 11 +0 val_0 2008-04-08 12 +4 val_4 2008-04-08 12 +8 val_8 2008-04-08 12 +0 val_0 2008-04-08 12 +0 val_0 2008-04-08 12 +5 val_5 2008-04-08 12 +5 val_5 2008-04-08 12 +2 val_2 2008-04-08 12 +5 val_5 2008-04-08 12 +9 val_9 2008-04-08 12 +0 val_0 2008-04-09 11 +4 val_4 2008-04-09 11 +8 val_8 2008-04-09 11 +0 val_0 2008-04-09 11 +0 val_0 2008-04-09 11 +5 val_5 2008-04-09 11 +5 val_5 2008-04-09 11 +2 val_2 2008-04-09 11 +5 val_5 2008-04-09 11 +9 val_9 2008-04-09 11 +0 val_0 2008-04-09 12 +4 val_4 2008-04-09 12 +8 val_8 2008-04-09 12 +0 val_0 2008-04-09 12 +0 val_0 2008-04-09 12 +5 val_5 2008-04-09 12 +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 diff --git a/src/test/hive/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d b/src/test/hive/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb b/src/test/hive/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb new file mode 100644 index 0000000000000..355ed1617e200 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb @@ -0,0 +1,2000 @@ +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 \ No newline at end of file diff --git a/src/test/hive/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 b/src/test/hive/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f b/src/test/hive/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f new file mode 100644 index 0000000000000..57005044dde38 --- /dev/null +++ b/src/test/hive/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f @@ -0,0 +1,2000 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 \ No newline at end of file diff --git a/src/test/hive/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 b/src/test/hive/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/print_header-1-8540676fc16ac91f3629c40f393a890a b/src/test/hive/golden/print_header-1-8540676fc16ac91f3629c40f393a890a new file mode 100644 index 0000000000000..943e1be13b615 --- /dev/null +++ b/src/test/hive/golden/print_header-1-8540676fc16ac91f3629c40f393a890a @@ -0,0 +1,10 @@ +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/src/test/hive/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 b/src/test/hive/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 new file mode 100644 index 0000000000000..951e74db0fe23 --- /dev/null +++ b/src/test/hive/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb b/src/test/hive/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 b/src/test/hive/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 b/src/test/hive/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f b/src/test/hive/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 b/src/test/hive/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 new file mode 100644 index 0000000000000..0b3e0a69a8c79 --- /dev/null +++ b/src/test/hive/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 @@ -0,0 +1 @@ +5000 \ No newline at end of file diff --git a/src/test/hive/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 b/src/test/hive/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 b/src/test/hive/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 b/src/test/hive/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 b/src/test/hive/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a b/src/test/hive/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c b/src/test/hive/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b b/src/test/hive/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 b/src/test/hive/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b b/src/test/hive/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 b/src/test/hive/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 b/src/test/hive/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 b/src/test/hive/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 new file mode 100644 index 0000000000000..eb63636d0bfd4 --- /dev/null +++ b/src/test/hive/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 @@ -0,0 +1,9 @@ +col string None +p string None + +# Partition Information +# col_name data_type comment + +p string None + +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 diff --git a/src/test/hive/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea b/src/test/hive/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b b/src/test/hive/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d b/src/test/hive/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d new file mode 100644 index 0000000000000..964ede006ad21 --- /dev/null +++ b/src/test/hive/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d @@ -0,0 +1,9 @@ +col string None +p string None + +# Partition Information +# col_name data_type comment + +p string None + +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 diff --git a/src/test/hive/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 b/src/test/hive/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d b/src/test/hive/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d new file mode 100644 index 0000000000000..68b03670f5f80 --- /dev/null +++ b/src/test/hive/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d @@ -0,0 +1,9 @@ +col string None +p string None + +# Partition Information +# col_name data_type comment + +p string None + +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 diff --git a/src/test/hive/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 b/src/test/hive/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 b/src/test/hive/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d b/src/test/hive/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d new file mode 100644 index 0000000000000..ec18ce7411535 --- /dev/null +++ b/src/test/hive/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d @@ -0,0 +1,9 @@ +col string None +p string None + +# Partition Information +# col_name data_type comment + +p string None + +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 diff --git a/src/test/hive/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 b/src/test/hive/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d b/src/test/hive/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d new file mode 100644 index 0000000000000..a6202ac5fc0eb --- /dev/null +++ b/src/test/hive/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d @@ -0,0 +1,9 @@ +col string None +p string None + +# Partition Information +# col_name data_type comment + +p string None + +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 diff --git a/src/test/hive/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 b/src/test/hive/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc b/src/test/hive/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 b/src/test/hive/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 b/src/test/hive/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 b/src/test/hive/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 new file mode 100644 index 0000000000000..0a6cebbbd0b82 --- /dev/null +++ b/src/test/hive/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 @@ -0,0 +1,9 @@ +col string None +p string None + +# Partition Information +# col_name data_type comment + +p string None + +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 diff --git a/src/test/hive/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b b/src/test/hive/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 b/src/test/hive/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 b/src/test/hive/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 b/src/test/hive/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e b/src/test/hive/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 b/src/test/hive/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 b/src/test/hive/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 b/src/test/hive/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-40-39024332d020339bd9396c9b15403a0c b/src/test/hive/golden/protectmode-40-39024332d020339bd9396c9b15403a0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 b/src/test/hive/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb b/src/test/hive/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 b/src/test/hive/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c b/src/test/hive/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 b/src/test/hive/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 new file mode 100644 index 0000000000000..a1ff1f8341f30 --- /dev/null +++ b/src/test/hive/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 @@ -0,0 +1,3 @@ +col string None + +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 diff --git a/src/test/hive/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 b/src/test/hive/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 b/src/test/hive/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 new file mode 100644 index 0000000000000..9098c9419556b --- /dev/null +++ b/src/test/hive/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 @@ -0,0 +1,3 @@ +col string None + +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 diff --git a/src/test/hive/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc b/src/test/hive/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 b/src/test/hive/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e b/src/test/hive/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 b/src/test/hive/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 b/src/test/hive/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 b/src/test/hive/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a b/src/test/hive/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a new file mode 100644 index 0000000000000..3a2f20c637883 --- /dev/null +++ b/src/test/hive/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a @@ -0,0 +1,23 @@ +0 val_0 2000-04-09 +0 val_0 2000-04-09 +0 val_0 2000-04-09 +2 val_2 2000-04-09 +4 val_4 2000-04-09 +5 val_5 2000-04-08 +5 val_5 2000-04-08 +5 val_5 2000-04-08 +5 val_5 2000-04-09 +5 val_5 2000-04-09 +5 val_5 2000-04-09 +8 val_8 2000-04-09 +9 val_9 2000-04-09 +10 val_10 2000-04-09 +11 val_11 2000-04-09 +12 val_12 2000-04-09 +12 val_12 2000-04-09 +15 val_15 2000-04-09 +15 val_15 2000-04-09 +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 diff --git a/src/test/hive/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 b/src/test/hive/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 new file mode 100644 index 0000000000000..2f3fe0189d2c6 --- /dev/null +++ b/src/test/hive/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 @@ -0,0 +1 @@ +aa; \ No newline at end of file diff --git a/src/test/hive/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb b/src/test/hive/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb new file mode 100644 index 0000000000000..b5b5773c405b4 --- /dev/null +++ b/src/test/hive/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb @@ -0,0 +1 @@ +bb \ No newline at end of file diff --git a/src/test/hive/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 b/src/test/hive/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 new file mode 100644 index 0000000000000..2652f5f42c003 --- /dev/null +++ b/src/test/hive/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 @@ -0,0 +1 @@ +cc \ No newline at end of file diff --git a/src/test/hive/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 b/src/test/hive/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 b/src/test/hive/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 b/src/test/hive/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 b/src/test/hive/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 b/src/test/hive/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 new file mode 100644 index 0000000000000..9a0990555b81f --- /dev/null +++ b/src/test/hive/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 @@ -0,0 +1,103 @@ +238 val_238 2008-04-08 +255 val_255 2008-04-08 +278 val_278 2008-04-08 +265 val_265 2008-04-08 +273 val_273 2008-04-08 +224 val_224 2008-04-08 +213 val_213 2008-04-08 +281 val_281 2008-04-08 +277 val_277 2008-04-08 +209 val_209 2008-04-08 +252 val_252 2008-04-08 +292 val_292 2008-04-08 +219 val_219 2008-04-08 +287 val_287 2008-04-08 +237 val_237 2008-04-08 +207 val_207 2008-04-08 +208 val_208 2008-04-08 +247 val_247 2008-04-08 +266 val_266 2008-04-08 +203 val_203 2008-04-08 +205 val_205 2008-04-08 +221 val_221 2008-04-08 +280 val_280 2008-04-08 +277 val_277 2008-04-08 +208 val_208 2008-04-08 +286 val_286 2008-04-08 +239 val_239 2008-04-08 +213 val_213 2008-04-08 +216 val_216 2008-04-08 +278 val_278 2008-04-08 +289 val_289 2008-04-08 +221 val_221 2008-04-08 +275 val_275 2008-04-08 +241 val_241 2008-04-08 +284 val_284 2008-04-08 +230 val_230 2008-04-08 +260 val_260 2008-04-08 +272 val_272 2008-04-08 +217 val_217 2008-04-08 +230 val_230 2008-04-08 +208 val_208 2008-04-08 +298 val_298 2008-04-08 +230 val_230 2008-04-08 +205 val_205 2008-04-08 +288 val_288 2008-04-08 +282 val_282 2008-04-08 +282 val_282 2008-04-08 +238 val_238 2008-04-08 +277 val_277 2008-04-08 +273 val_273 2008-04-08 +224 val_224 2008-04-08 +242 val_242 2008-04-08 +272 val_272 2008-04-08 +242 val_242 2008-04-08 +226 val_226 2008-04-08 +229 val_229 2008-04-08 +233 val_233 2008-04-08 +223 val_223 2008-04-08 +218 val_218 2008-04-08 +228 val_228 2008-04-08 +209 val_209 2008-04-08 +230 val_230 2008-04-08 +296 val_296 2008-04-08 +216 val_216 2008-04-08 +274 val_274 2008-04-08 +219 val_219 2008-04-08 +239 val_239 2008-04-08 +223 val_223 2008-04-08 +256 val_256 2008-04-08 +263 val_263 2008-04-08 +288 val_288 2008-04-08 +244 val_244 2008-04-08 +202 val_202 2008-04-08 +229 val_229 2008-04-08 +280 val_280 2008-04-08 +283 val_283 2008-04-08 +235 val_235 2008-04-08 +257 val_257 2008-04-08 +258 val_258 2008-04-08 +203 val_203 2008-04-08 +262 val_262 2008-04-08 +201 val_201 2008-04-08 +217 val_217 2008-04-08 +298 val_298 2008-04-08 +291 val_291 2008-04-08 +255 val_255 2008-04-08 +200 val_200 2008-04-08 +237 val_237 2008-04-08 +248 val_248 2008-04-08 +277 val_277 2008-04-08 +230 val_230 2008-04-08 +207 val_207 2008-04-08 +249 val_249 2008-04-08 +265 val_265 2008-04-08 +214 val_214 2008-04-08 +233 val_233 2008-04-08 +256 val_256 2008-04-08 +298 val_298 2008-04-08 +285 val_285 2008-04-08 +273 val_273 2008-04-08 +281 val_281 2008-04-08 +222 val_222 2008-04-08 +200 val_200 2008-04-08 diff --git a/src/test/hive/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 b/src/test/hive/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 b/src/test/hive/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 new file mode 100644 index 0000000000000..ed1f53a6588d0 --- /dev/null +++ b/src/test/hive/golden/quote2-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e b/src/test/hive/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d b/src/test/hive/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 b/src/test/hive/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 new file mode 100644 index 0000000000000..73b8edab36833 --- /dev/null +++ b/src/test/hive/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 @@ -0,0 +1,56 @@ +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/src/test/hive/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 b/src/test/hive/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 b/src/test/hive/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 b/src/test/hive/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 b/src/test/hive/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 new file mode 100644 index 0000000000000..e8738e9c837ee --- /dev/null +++ b/src/test/hive/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 @@ -0,0 +1,116 @@ +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/src/test/hive/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/src/test/hive/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 b/src/test/hive/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/src/test/hive/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a b/src/test/hive/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 b/src/test/hive/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 b/src/test/hive/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b b/src/test/hive/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b new file mode 100644 index 0000000000000..d1b3011bffd91 --- /dev/null +++ b/src/test/hive/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b @@ -0,0 +1,2 @@ +key string None +value string None diff --git a/src/test/hive/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff b/src/test/hive/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff new file mode 100644 index 0000000000000..b7bef323d7df0 --- /dev/null +++ b/src/test/hive/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff @@ -0,0 +1,10 @@ +165 val_165 +238 val_238 +255 val_255 +27 val_27 +278 val_278 +311 val_311 +409 val_409 +484 val_484 +86 val_86 +98 val_98 diff --git a/src/test/hive/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 b/src/test/hive/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 b/src/test/hive/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b b/src/test/hive/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b new file mode 100644 index 0000000000000..96855269dbfa8 --- /dev/null +++ b/src/test/hive/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b @@ -0,0 +1,5 @@ +255 1 +278 1 +311 1 +409 1 +484 1 diff --git a/src/test/hive/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c b/src/test/hive/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 b/src/test/hive/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 b/src/test/hive/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 new file mode 100644 index 0000000000000..2ba7d9f4f32ad --- /dev/null +++ b/src/test/hive/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 @@ -0,0 +1,5 @@ +255 val_255 +278 val_278 +311 val_311 +409 val_409 +484 val_484 diff --git a/src/test/hive/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 b/src/test/hive/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 new file mode 100644 index 0000000000000..1b977ce91e68d --- /dev/null +++ b/src/test/hive/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 @@ -0,0 +1,3 @@ +255 val_255 +278 val_278 +311 val_311 diff --git a/src/test/hive/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b b/src/test/hive/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b new file mode 100644 index 0000000000000..96855269dbfa8 --- /dev/null +++ b/src/test/hive/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b @@ -0,0 +1,5 @@ +255 1 +278 1 +311 1 +409 1 +484 1 diff --git a/src/test/hive/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 b/src/test/hive/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d b/src/test/hive/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 b/src/test/hive/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 b/src/test/hive/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 new file mode 100644 index 0000000000000..2ba7d9f4f32ad --- /dev/null +++ b/src/test/hive/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 @@ -0,0 +1,5 @@ +255 val_255 +278 val_278 +311 val_311 +409 val_409 +484 val_484 diff --git a/src/test/hive/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 b/src/test/hive/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 new file mode 100644 index 0000000000000..1b977ce91e68d --- /dev/null +++ b/src/test/hive/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 @@ -0,0 +1,3 @@ +255 val_255 +278 val_278 +311 val_311 diff --git a/src/test/hive/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 b/src/test/hive/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 b/src/test/hive/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 b/src/test/hive/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c b/src/test/hive/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 b/src/test/hive/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 new file mode 100644 index 0000000000000..b141fed81f206 --- /dev/null +++ b/src/test/hive/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 @@ -0,0 +1 @@ +53278638794 diff --git a/src/test/hive/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 b/src/test/hive/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 new file mode 100644 index 0000000000000..b141fed81f206 --- /dev/null +++ b/src/test/hive/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 @@ -0,0 +1 @@ +53278638794 diff --git a/src/test/hive/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 b/src/test/hive/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e b/src/test/hive/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 b/src/test/hive/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee b/src/test/hive/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 b/src/test/hive/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e b/src/test/hive/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 b/src/test/hive/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 b/src/test/hive/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f b/src/test/hive/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 b/src/test/hive/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 b/src/test/hive/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 b/src/test/hive/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 b/src/test/hive/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb b/src/test/hive/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 b/src/test/hive/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 b/src/test/hive/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a b/src/test/hive/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 b/src/test/hive/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 b/src/test/hive/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 new file mode 100644 index 0000000000000..8d25d618795b9 --- /dev/null +++ b/src/test/hive/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 @@ -0,0 +1 @@ +-4208881187 diff --git a/src/test/hive/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 b/src/test/hive/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 new file mode 100644 index 0000000000000..8d25d618795b9 --- /dev/null +++ b/src/test/hive/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 @@ -0,0 +1 @@ +-4208881187 diff --git a/src/test/hive/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 b/src/test/hive/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 b/src/test/hive/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 b/src/test/hive/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 b/src/test/hive/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 new file mode 100644 index 0000000000000..11ff946b46f0f --- /dev/null +++ b/src/test/hive/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 @@ -0,0 +1 @@ +14412220296 diff --git a/src/test/hive/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f b/src/test/hive/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f new file mode 100644 index 0000000000000..11ff946b46f0f --- /dev/null +++ b/src/test/hive/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f @@ -0,0 +1 @@ +14412220296 diff --git a/src/test/hive/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 b/src/test/hive/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b b/src/test/hive/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb b/src/test/hive/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 b/src/test/hive/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b b/src/test/hive/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 b/src/test/hive/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b b/src/test/hive/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/src/test/hive/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 b/src/test/hive/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 b/src/test/hive/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 b/src/test/hive/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 b/src/test/hive/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 b/src/test/hive/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 b/src/test/hive/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 new file mode 100644 index 0000000000000..11ff946b46f0f --- /dev/null +++ b/src/test/hive/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 @@ -0,0 +1 @@ +14412220296 diff --git a/src/test/hive/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f b/src/test/hive/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f new file mode 100644 index 0000000000000..11ff946b46f0f --- /dev/null +++ b/src/test/hive/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f @@ -0,0 +1 @@ +14412220296 diff --git a/src/test/hive/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 b/src/test/hive/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b b/src/test/hive/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb b/src/test/hive/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 b/src/test/hive/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b b/src/test/hive/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 b/src/test/hive/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 b/src/test/hive/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/src/test/hive/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 b/src/test/hive/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 b/src/test/hive/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 b/src/test/hive/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d b/src/test/hive/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee b/src/test/hive/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee new file mode 100644 index 0000000000000..b0135533064c9 --- /dev/null +++ b/src/test/hive/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee @@ -0,0 +1,25 @@ +238 val_238 +NULL +311 val_311 +NULL val_27 +NULL val_165 +NULL val_409 +255 val_255 +278 val_278 +98 val_98 +NULL val_484 +NULL val_265 +NULL val_193 +401 val_401 +150 val_150 +273 val_273 +224 +369 +66 val_66 +128 +213 val_213 +146 val_146 +406 val_406 +NULL +NULL +NULL \ No newline at end of file diff --git a/src/test/hive/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a b/src/test/hive/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 b/src/test/hive/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed b/src/test/hive/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 b/src/test/hive/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 new file mode 100644 index 0000000000000..06b2b4d7e6d47 --- /dev/null +++ b/src/test/hive/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 @@ -0,0 +1,9 @@ +17 val_17 17 val_17 +NULL NULL 20 val_20 +NULL NULL 24 val_24 +19 val_19 19 val_19 +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 diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 b/src/test/hive/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b b/src/test/hive/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 b/src/test/hive/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d b/src/test/hive/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 b/src/test/hive/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a b/src/test/hive/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 b/src/test/hive/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 b/src/test/hive/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b b/src/test/hive/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 b/src/test/hive/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 new file mode 100644 index 0000000000000..c0e3764b993b4 --- /dev/null +++ b/src/test/hive/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 @@ -0,0 +1,20 @@ +165 +238 +255 +27 +278 +311 +409 +484 +86 +98 +val_165 +val_238 +val_255 +val_27 +val_278 +val_311 +val_409 +val_484 +val_86 +val_98 diff --git a/src/test/hive/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 b/src/test/hive/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 b/src/test/hive/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 new file mode 100644 index 0000000000000..1ab53eed2f15a --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 @@ -0,0 +1 @@ +130091 36210398070 diff --git a/src/test/hive/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 b/src/test/hive/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 b/src/test/hive/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 b/src/test/hive/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe b/src/test/hive/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 b/src/test/hive/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d b/src/test/hive/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 b/src/test/hive/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 b/src/test/hive/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 b/src/test/hive/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 b/src/test/hive/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 new file mode 100644 index 0000000000000..da664fda89fd7 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 @@ -0,0 +1 @@ +21025334 36210398070 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 b/src/test/hive/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 b/src/test/hive/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 b/src/test/hive/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 b/src/test/hive/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 b/src/test/hive/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 b/src/test/hive/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 b/src/test/hive/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 b/src/test/hive/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa b/src/test/hive/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e b/src/test/hive/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e new file mode 100644 index 0000000000000..4a827d502b934 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e @@ -0,0 +1,309 @@ +0 0 +2 2 +4 4 +5 15 +8 8 +9 9 +10 10 +11 11 +12 24 +15 30 +17 17 +18 36 +19 19 +20 20 +24 48 +26 52 +27 27 +28 28 +30 30 +33 33 +34 34 +35 105 +37 74 +41 41 +42 84 +43 43 +44 44 +47 47 +51 102 +53 53 +54 54 +57 57 +58 116 +64 64 +65 65 +66 66 +67 134 +69 69 +70 210 +72 144 +74 74 +76 152 +77 77 +78 78 +80 80 +82 82 +83 166 +84 168 +85 85 +86 86 +87 87 +90 270 +92 92 +95 190 +96 96 +97 194 +98 196 +100 200 +103 206 +104 208 +105 105 +111 111 +113 226 +114 114 +116 116 +118 236 +119 357 +120 240 +125 250 +126 126 +128 384 +129 258 +131 131 +133 133 +134 268 +136 136 +137 274 +138 552 +143 143 +145 145 +146 292 +149 298 +150 150 +152 304 +153 153 +155 155 +156 156 +157 157 +158 158 +160 160 +162 162 +163 163 +164 328 +165 330 +166 166 +167 501 +168 168 +169 676 +170 170 +172 344 +174 348 +175 350 +176 352 +177 177 +178 178 +179 358 +180 180 +181 181 +183 183 +186 186 +187 561 +189 189 +190 190 +191 382 +192 192 +193 579 +194 194 +195 390 +196 196 +197 394 +199 597 +200 400 +201 201 +202 202 +203 406 +205 410 +207 414 +208 624 +209 418 +213 426 +214 214 +216 432 +217 434 +218 218 +219 438 +221 442 +222 222 +223 446 +224 448 +226 226 +228 228 +229 458 +230 1150 +233 466 +235 235 +237 474 +238 476 +239 478 +241 241 +242 484 +244 244 +247 247 +248 248 +249 249 +252 252 +255 510 +256 512 +257 257 +258 258 +260 260 +262 262 +263 263 +265 530 +266 266 +272 544 +273 819 +274 274 +275 275 +277 1108 +278 556 +280 560 +281 562 +282 564 +283 283 +284 284 +285 285 +286 286 +287 287 +288 576 +289 289 +291 291 +292 292 +296 296 +298 894 +302 302 +305 305 +306 306 +307 614 +308 308 +309 618 +310 310 +311 933 +315 315 +316 948 +317 634 +318 954 +321 642 +322 644 +323 323 +325 650 +327 981 +331 662 +332 332 +333 666 +335 335 +336 336 +338 338 +339 339 +341 341 +342 684 +344 688 +345 345 +348 1740 +351 351 +353 706 +356 356 +360 360 +362 362 +364 364 +365 365 +366 366 +367 734 +368 368 +369 1107 +373 373 +374 374 +375 375 +377 377 +378 378 +379 379 +382 764 +384 1152 +386 386 +389 389 +392 392 +393 393 +394 394 +395 790 +396 1188 +397 794 +399 798 +400 400 +401 2005 +402 402 +403 1209 +404 808 +406 1624 +407 407 +409 1227 +411 411 +413 826 +414 828 +417 1251 +418 418 +419 419 +421 421 +424 848 +427 427 +429 858 +430 1290 +431 1293 +432 432 +435 435 +436 436 +437 437 +438 1314 +439 878 +443 443 +444 444 +446 446 +448 448 +449 449 +452 452 +453 453 +454 1362 +455 455 +457 457 +458 916 +459 918 +460 460 +462 924 +463 926 +466 1398 +467 467 +468 1872 +469 2345 +470 470 +472 472 +475 475 +477 477 +478 956 +479 479 +480 1440 +481 481 +482 482 +483 483 +484 484 +485 485 +487 487 +489 1956 +490 490 +491 491 +492 984 +493 493 +494 494 +495 495 +496 496 +497 497 +498 1494 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 b/src/test/hive/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 new file mode 100644 index 0000000000000..af5653b0de758 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 @@ -0,0 +1,309 @@ +0 0 val_0 +2 2 val_2 +4 4 val_4 +5 15 val_5 +8 8 val_8 +9 9 val_9 +10 10 val_10 +11 11 val_11 +12 24 val_12 +15 30 val_15 +17 17 val_17 +18 36 val_18 +19 19 val_19 +20 20 val_20 +24 48 val_24 +26 52 val_26 +27 27 val_27 +28 28 val_28 +30 30 val_30 +33 33 val_33 +34 34 val_34 +35 105 val_35 +37 74 val_37 +41 41 val_41 +42 84 val_42 +43 43 val_43 +44 44 val_44 +47 47 val_47 +51 102 val_51 +53 53 val_53 +54 54 val_54 +57 57 val_57 +58 116 val_58 +64 64 val_64 +65 65 val_65 +66 66 val_66 +67 134 val_67 +69 69 val_69 +70 210 val_70 +72 144 val_72 +74 74 val_74 +76 152 val_76 +77 77 val_77 +78 78 val_78 +80 80 val_80 +82 82 val_82 +83 166 val_83 +84 168 val_84 +85 85 val_85 +86 86 val_86 +87 87 val_87 +90 270 val_90 +92 92 val_92 +95 190 val_95 +96 96 val_96 +97 194 val_97 +98 196 val_98 +100 200 val_100 +103 206 val_103 +104 208 val_104 +105 105 val_105 +111 111 val_111 +113 226 val_113 +114 114 val_114 +116 116 val_116 +118 236 val_118 +119 357 val_119 +120 240 val_120 +125 250 val_125 +126 126 val_126 +128 384 val_128 +129 258 val_129 +131 131 val_131 +133 133 val_133 +134 268 val_134 +136 136 val_136 +137 274 val_137 +138 552 val_138 +143 143 val_143 +145 145 val_145 +146 292 val_146 +149 298 val_149 +150 150 val_150 +152 304 val_152 +153 153 val_153 +155 155 val_155 +156 156 val_156 +157 157 val_157 +158 158 val_158 +160 160 val_160 +162 162 val_162 +163 163 val_163 +164 328 val_164 +165 330 val_165 +166 166 val_166 +167 501 val_167 +168 168 val_168 +169 676 val_169 +170 170 val_170 +172 344 val_172 +174 348 val_174 +175 350 val_175 +176 352 val_176 +177 177 val_177 +178 178 val_178 +179 358 val_179 +180 180 val_180 +181 181 val_181 +183 183 val_183 +186 186 val_186 +187 561 val_187 +189 189 val_189 +190 190 val_190 +191 382 val_191 +192 192 val_192 +193 579 val_193 +194 194 val_194 +195 390 val_195 +196 196 val_196 +197 394 val_197 +199 597 val_199 +200 400 val_200 +201 201 val_201 +202 202 val_202 +203 406 val_203 +205 410 val_205 +207 414 val_207 +208 624 val_208 +209 418 val_209 +213 426 val_213 +214 214 val_214 +216 432 val_216 +217 434 val_217 +218 218 val_218 +219 438 val_219 +221 442 val_221 +222 222 val_222 +223 446 val_223 +224 448 val_224 +226 226 val_226 +228 228 val_228 +229 458 val_229 +230 1150 val_230 +233 466 val_233 +235 235 val_235 +237 474 val_237 +238 476 val_238 +239 478 val_239 +241 241 val_241 +242 484 val_242 +244 244 val_244 +247 247 val_247 +248 248 val_248 +249 249 val_249 +252 252 val_252 +255 510 val_255 +256 512 val_256 +257 257 val_257 +258 258 val_258 +260 260 val_260 +262 262 val_262 +263 263 val_263 +265 530 val_265 +266 266 val_266 +272 544 val_272 +273 819 val_273 +274 274 val_274 +275 275 val_275 +277 1108 val_277 +278 556 val_278 +280 560 val_280 +281 562 val_281 +282 564 val_282 +283 283 val_283 +284 284 val_284 +285 285 val_285 +286 286 val_286 +287 287 val_287 +288 576 val_288 +289 289 val_289 +291 291 val_291 +292 292 val_292 +296 296 val_296 +298 894 val_298 +302 302 val_302 +305 305 val_305 +306 306 val_306 +307 614 val_307 +308 308 val_308 +309 618 val_309 +310 310 val_310 +311 933 val_311 +315 315 val_315 +316 948 val_316 +317 634 val_317 +318 954 val_318 +321 642 val_321 +322 644 val_322 +323 323 val_323 +325 650 val_325 +327 981 val_327 +331 662 val_331 +332 332 val_332 +333 666 val_333 +335 335 val_335 +336 336 val_336 +338 338 val_338 +339 339 val_339 +341 341 val_341 +342 684 val_342 +344 688 val_344 +345 345 val_345 +348 1740 val_348 +351 351 val_351 +353 706 val_353 +356 356 val_356 +360 360 val_360 +362 362 val_362 +364 364 val_364 +365 365 val_365 +366 366 val_366 +367 734 val_367 +368 368 val_368 +369 1107 val_369 +373 373 val_373 +374 374 val_374 +375 375 val_375 +377 377 val_377 +378 378 val_378 +379 379 val_379 +382 764 val_382 +384 1152 val_384 +386 386 val_386 +389 389 val_389 +392 392 val_392 +393 393 val_393 +394 394 val_394 +395 790 val_395 +396 1188 val_396 +397 794 val_397 +399 798 val_399 +400 400 val_400 +401 2005 val_401 +402 402 val_402 +403 1209 val_403 +404 808 val_404 +406 1624 val_406 +407 407 val_407 +409 1227 val_409 +411 411 val_411 +413 826 val_413 +414 828 val_414 +417 1251 val_417 +418 418 val_418 +419 419 val_419 +421 421 val_421 +424 848 val_424 +427 427 val_427 +429 858 val_429 +430 1290 val_430 +431 1293 val_431 +432 432 val_432 +435 435 val_435 +436 436 val_436 +437 437 val_437 +438 1314 val_438 +439 878 val_439 +443 443 val_443 +444 444 val_444 +446 446 val_446 +448 448 val_448 +449 449 val_449 +452 452 val_452 +453 453 val_453 +454 1362 val_454 +455 455 val_455 +457 457 val_457 +458 916 val_458 +459 918 val_459 +460 460 val_460 +462 924 val_462 +463 926 val_463 +466 1398 val_466 +467 467 val_467 +468 1872 val_468 +469 2345 val_469 +470 470 val_470 +472 472 val_472 +475 475 val_475 +477 477 val_477 +478 956 val_478 +479 479 val_479 +480 1440 val_480 +481 481 val_481 +482 482 val_482 +483 483 val_483 +484 484 val_484 +485 485 val_485 +487 487 val_487 +489 1956 val_489 +490 490 val_490 +491 491 val_491 +492 984 val_492 +493 493 val_493 +494 494 val_494 +495 495 val_495 +496 496 val_496 +497 497 val_497 +498 1494 val_498 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e b/src/test/hive/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e new file mode 100644 index 0000000000000..5d9e6bbf9cedb --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e @@ -0,0 +1,309 @@ +0 0 NULL +2 2 NULL +4 4 NULL +5 15 NULL +8 8 NULL +9 9 NULL +10 10 NULL +11 11 NULL +12 24 NULL +15 30 NULL +17 17 NULL +18 36 NULL +19 19 NULL +20 20 NULL +24 48 NULL +26 52 NULL +27 27 NULL +28 28 NULL +30 30 NULL +33 33 NULL +34 34 NULL +35 105 NULL +37 74 NULL +41 41 NULL +42 84 NULL +43 43 NULL +44 44 NULL +47 47 NULL +51 102 NULL +53 53 NULL +54 54 NULL +57 57 NULL +58 116 NULL +64 64 NULL +65 65 NULL +66 66 NULL +67 134 NULL +69 69 NULL +70 210 NULL +72 144 NULL +74 74 NULL +76 152 NULL +77 77 NULL +78 78 NULL +80 80 NULL +82 82 NULL +83 166 NULL +84 168 NULL +85 85 NULL +86 86 NULL +87 87 NULL +90 270 NULL +92 92 NULL +95 190 NULL +96 96 NULL +97 194 NULL +98 196 NULL +100 200 NULL +103 206 NULL +104 208 NULL +105 105 NULL +111 111 NULL +113 226 NULL +114 114 NULL +116 116 NULL +118 236 NULL +119 357 NULL +120 240 NULL +125 250 NULL +126 126 NULL +128 384 NULL +129 258 NULL +131 131 NULL +133 133 NULL +134 268 NULL +136 136 NULL +137 274 NULL +138 552 NULL +143 143 NULL +145 145 NULL +146 292 NULL +149 298 NULL +150 150 NULL +152 304 NULL +153 153 NULL +155 155 NULL +156 156 NULL +157 157 NULL +158 158 NULL +160 160 NULL +162 162 NULL +163 163 NULL +164 328 NULL +165 330 NULL +166 166 NULL +167 501 NULL +168 168 NULL +169 676 NULL +170 170 NULL +172 344 NULL +174 348 NULL +175 350 NULL +176 352 NULL +177 177 NULL +178 178 NULL +179 358 NULL +180 180 NULL +181 181 NULL +183 183 NULL +186 186 NULL +187 561 NULL +189 189 NULL +190 190 NULL +191 382 NULL +192 192 NULL +193 579 NULL +194 194 NULL +195 390 NULL +196 196 NULL +197 394 NULL +199 597 NULL +200 400 NULL +201 201 NULL +202 202 NULL +203 406 NULL +205 410 NULL +207 414 NULL +208 624 NULL +209 418 NULL +213 426 NULL +214 214 NULL +216 432 NULL +217 434 NULL +218 218 NULL +219 438 NULL +221 442 NULL +222 222 NULL +223 446 NULL +224 448 NULL +226 226 NULL +228 228 NULL +229 458 NULL +230 1150 NULL +233 466 NULL +235 235 NULL +237 474 NULL +238 476 NULL +239 478 NULL +241 241 NULL +242 484 NULL +244 244 NULL +247 247 NULL +248 248 NULL +249 249 NULL +252 252 NULL +255 510 NULL +256 512 NULL +257 257 NULL +258 258 NULL +260 260 NULL +262 262 NULL +263 263 NULL +265 530 NULL +266 266 NULL +272 544 NULL +273 819 NULL +274 274 NULL +275 275 NULL +277 1108 NULL +278 556 NULL +280 560 NULL +281 562 NULL +282 564 NULL +283 283 NULL +284 284 NULL +285 285 NULL +286 286 NULL +287 287 NULL +288 576 NULL +289 289 NULL +291 291 NULL +292 292 NULL +296 296 NULL +298 894 NULL +302 302 NULL +305 305 NULL +306 306 NULL +307 614 NULL +308 308 NULL +309 618 NULL +310 310 NULL +311 933 NULL +315 315 NULL +316 948 NULL +317 634 NULL +318 954 NULL +321 642 NULL +322 644 NULL +323 323 NULL +325 650 NULL +327 981 NULL +331 662 NULL +332 332 NULL +333 666 NULL +335 335 NULL +336 336 NULL +338 338 NULL +339 339 NULL +341 341 NULL +342 684 NULL +344 688 NULL +345 345 NULL +348 1740 NULL +351 351 NULL +353 706 NULL +356 356 NULL +360 360 NULL +362 362 NULL +364 364 NULL +365 365 NULL +366 366 NULL +367 734 NULL +368 368 NULL +369 1107 NULL +373 373 NULL +374 374 NULL +375 375 NULL +377 377 NULL +378 378 NULL +379 379 NULL +382 764 NULL +384 1152 NULL +386 386 NULL +389 389 NULL +392 392 NULL +393 393 NULL +394 394 NULL +395 790 NULL +396 1188 NULL +397 794 NULL +399 798 NULL +400 400 NULL +401 2005 NULL +402 402 NULL +403 1209 NULL +404 808 NULL +406 1624 NULL +407 407 NULL +409 1227 NULL +411 411 NULL +413 826 NULL +414 828 NULL +417 1251 NULL +418 418 NULL +419 419 NULL +421 421 NULL +424 848 NULL +427 427 NULL +429 858 NULL +430 1290 NULL +431 1293 NULL +432 432 NULL +435 435 NULL +436 436 NULL +437 437 NULL +438 1314 NULL +439 878 NULL +443 443 NULL +444 444 NULL +446 446 NULL +448 448 NULL +449 449 NULL +452 452 NULL +453 453 NULL +454 1362 NULL +455 455 NULL +457 457 NULL +458 916 NULL +459 918 NULL +460 460 NULL +462 924 NULL +463 926 NULL +466 1398 NULL +467 467 NULL +468 1872 NULL +469 2345 NULL +470 470 NULL +472 472 NULL +475 475 NULL +477 477 NULL +478 956 NULL +479 479 NULL +480 1440 NULL +481 481 NULL +482 482 NULL +483 483 NULL +484 484 NULL +485 485 NULL +487 487 NULL +489 1956 NULL +490 490 NULL +491 491 NULL +492 984 NULL +493 493 NULL +494 494 NULL +495 495 NULL +496 496 NULL +497 497 NULL +498 1494 NULL diff --git a/src/test/hive/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c b/src/test/hive/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c new file mode 100644 index 0000000000000..4a827d502b934 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c @@ -0,0 +1,309 @@ +0 0 +2 2 +4 4 +5 15 +8 8 +9 9 +10 10 +11 11 +12 24 +15 30 +17 17 +18 36 +19 19 +20 20 +24 48 +26 52 +27 27 +28 28 +30 30 +33 33 +34 34 +35 105 +37 74 +41 41 +42 84 +43 43 +44 44 +47 47 +51 102 +53 53 +54 54 +57 57 +58 116 +64 64 +65 65 +66 66 +67 134 +69 69 +70 210 +72 144 +74 74 +76 152 +77 77 +78 78 +80 80 +82 82 +83 166 +84 168 +85 85 +86 86 +87 87 +90 270 +92 92 +95 190 +96 96 +97 194 +98 196 +100 200 +103 206 +104 208 +105 105 +111 111 +113 226 +114 114 +116 116 +118 236 +119 357 +120 240 +125 250 +126 126 +128 384 +129 258 +131 131 +133 133 +134 268 +136 136 +137 274 +138 552 +143 143 +145 145 +146 292 +149 298 +150 150 +152 304 +153 153 +155 155 +156 156 +157 157 +158 158 +160 160 +162 162 +163 163 +164 328 +165 330 +166 166 +167 501 +168 168 +169 676 +170 170 +172 344 +174 348 +175 350 +176 352 +177 177 +178 178 +179 358 +180 180 +181 181 +183 183 +186 186 +187 561 +189 189 +190 190 +191 382 +192 192 +193 579 +194 194 +195 390 +196 196 +197 394 +199 597 +200 400 +201 201 +202 202 +203 406 +205 410 +207 414 +208 624 +209 418 +213 426 +214 214 +216 432 +217 434 +218 218 +219 438 +221 442 +222 222 +223 446 +224 448 +226 226 +228 228 +229 458 +230 1150 +233 466 +235 235 +237 474 +238 476 +239 478 +241 241 +242 484 +244 244 +247 247 +248 248 +249 249 +252 252 +255 510 +256 512 +257 257 +258 258 +260 260 +262 262 +263 263 +265 530 +266 266 +272 544 +273 819 +274 274 +275 275 +277 1108 +278 556 +280 560 +281 562 +282 564 +283 283 +284 284 +285 285 +286 286 +287 287 +288 576 +289 289 +291 291 +292 292 +296 296 +298 894 +302 302 +305 305 +306 306 +307 614 +308 308 +309 618 +310 310 +311 933 +315 315 +316 948 +317 634 +318 954 +321 642 +322 644 +323 323 +325 650 +327 981 +331 662 +332 332 +333 666 +335 335 +336 336 +338 338 +339 339 +341 341 +342 684 +344 688 +345 345 +348 1740 +351 351 +353 706 +356 356 +360 360 +362 362 +364 364 +365 365 +366 366 +367 734 +368 368 +369 1107 +373 373 +374 374 +375 375 +377 377 +378 378 +379 379 +382 764 +384 1152 +386 386 +389 389 +392 392 +393 393 +394 394 +395 790 +396 1188 +397 794 +399 798 +400 400 +401 2005 +402 402 +403 1209 +404 808 +406 1624 +407 407 +409 1227 +411 411 +413 826 +414 828 +417 1251 +418 418 +419 419 +421 421 +424 848 +427 427 +429 858 +430 1290 +431 1293 +432 432 +435 435 +436 436 +437 437 +438 1314 +439 878 +443 443 +444 444 +446 446 +448 448 +449 449 +452 452 +453 453 +454 1362 +455 455 +457 457 +458 916 +459 918 +460 460 +462 924 +463 926 +466 1398 +467 467 +468 1872 +469 2345 +470 470 +472 472 +475 475 +477 477 +478 956 +479 479 +480 1440 +481 481 +482 482 +483 483 +484 484 +485 485 +487 487 +489 1956 +490 490 +491 491 +492 984 +493 493 +494 494 +495 495 +496 496 +497 497 +498 1494 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd b/src/test/hive/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd new file mode 100644 index 0000000000000..32740bfc546a9 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd @@ -0,0 +1,15 @@ +66 66 +98 196 +128 384 +146 292 +150 150 +213 426 +224 448 +238 476 +255 510 +273 819 +278 556 +311 933 +369 1107 +401 2005 +406 1624 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb b/src/test/hive/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb new file mode 100644 index 0000000000000..cc33012c6e2eb --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb @@ -0,0 +1,37 @@ +66 val_66 +98 val_98 +98 val_98 +128 val_128 +128 val_128 +128 val_128 +146 val_146 +146 val_146 +150 val_150 +213 val_213 +213 val_213 +224 val_224 +224 val_224 +238 val_238 +238 val_238 +255 val_255 +255 val_255 +273 val_273 +273 val_273 +273 val_273 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +311 val_311 +369 val_369 +369 val_369 +369 val_369 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +406 val_406 +406 val_406 +406 val_406 +406 val_406 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f b/src/test/hive/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f new file mode 100644 index 0000000000000..851d5ed155549 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f @@ -0,0 +1,309 @@ +0 +2 +4 +5 +8 +9 +10 +11 +12 +15 +17 +18 +19 +20 +24 +26 +27 +28 +30 +33 +34 +35 +37 +41 +42 +43 +44 +47 +51 +53 +54 +57 +58 +64 +65 +66 +67 +69 +70 +72 +74 +76 +77 +78 +80 +82 +83 +84 +85 +86 +87 +90 +92 +95 +96 +97 +98 +100 +103 +104 +105 +111 +113 +114 +116 +118 +119 +120 +125 +126 +128 +129 +131 +133 +134 +136 +137 +138 +143 +145 +146 +149 +150 +152 +153 +155 +156 +157 +158 +160 +162 +163 +164 +165 +166 +167 +168 +169 +170 +172 +174 +175 +176 +177 +178 +179 +180 +181 +183 +186 +187 +189 +190 +191 +192 +193 +194 +195 +196 +197 +199 +200 +201 +202 +203 +205 +207 +208 +209 +213 +214 +216 +217 +218 +219 +221 +222 +223 +224 +226 +228 +229 +230 +233 +235 +237 +238 +239 +241 +242 +244 +247 +248 +249 +252 +255 +256 +257 +258 +260 +262 +263 +265 +266 +272 +273 +274 +275 +277 +278 +280 +281 +282 +283 +284 +285 +286 +287 +288 +289 +291 +292 +296 +298 +302 +305 +306 +307 +308 +309 +310 +311 +315 +316 +317 +318 +321 +322 +323 +325 +327 +331 +332 +333 +335 +336 +338 +339 +341 +342 +344 +345 +348 +351 +353 +356 +360 +362 +364 +365 +366 +367 +368 +369 +373 +374 +375 +377 +378 +379 +382 +384 +386 +389 +392 +393 +394 +395 +396 +397 +399 +400 +401 +402 +403 +404 +406 +407 +409 +411 +413 +414 +417 +418 +419 +421 +424 +427 +429 +430 +431 +432 +435 +436 +437 +438 +439 +443 +444 +446 +448 +449 +452 +453 +454 +455 +457 +458 +459 +460 +462 +463 +466 +467 +468 +469 +470 +472 +475 +477 +478 +479 +480 +481 +482 +483 +484 +485 +487 +489 +490 +491 +492 +493 +494 +495 +496 +497 +498 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 b/src/test/hive/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 @@ -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/src/test/hive/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 b/src/test/hive/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 b/src/test/hive/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 b/src/test/hive/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d b/src/test/hive/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed b/src/test/hive/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 b/src/test/hive/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a b/src/test/hive/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa b/src/test/hive/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e b/src/test/hive/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e new file mode 100644 index 0000000000000..4a827d502b934 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e @@ -0,0 +1,309 @@ +0 0 +2 2 +4 4 +5 15 +8 8 +9 9 +10 10 +11 11 +12 24 +15 30 +17 17 +18 36 +19 19 +20 20 +24 48 +26 52 +27 27 +28 28 +30 30 +33 33 +34 34 +35 105 +37 74 +41 41 +42 84 +43 43 +44 44 +47 47 +51 102 +53 53 +54 54 +57 57 +58 116 +64 64 +65 65 +66 66 +67 134 +69 69 +70 210 +72 144 +74 74 +76 152 +77 77 +78 78 +80 80 +82 82 +83 166 +84 168 +85 85 +86 86 +87 87 +90 270 +92 92 +95 190 +96 96 +97 194 +98 196 +100 200 +103 206 +104 208 +105 105 +111 111 +113 226 +114 114 +116 116 +118 236 +119 357 +120 240 +125 250 +126 126 +128 384 +129 258 +131 131 +133 133 +134 268 +136 136 +137 274 +138 552 +143 143 +145 145 +146 292 +149 298 +150 150 +152 304 +153 153 +155 155 +156 156 +157 157 +158 158 +160 160 +162 162 +163 163 +164 328 +165 330 +166 166 +167 501 +168 168 +169 676 +170 170 +172 344 +174 348 +175 350 +176 352 +177 177 +178 178 +179 358 +180 180 +181 181 +183 183 +186 186 +187 561 +189 189 +190 190 +191 382 +192 192 +193 579 +194 194 +195 390 +196 196 +197 394 +199 597 +200 400 +201 201 +202 202 +203 406 +205 410 +207 414 +208 624 +209 418 +213 426 +214 214 +216 432 +217 434 +218 218 +219 438 +221 442 +222 222 +223 446 +224 448 +226 226 +228 228 +229 458 +230 1150 +233 466 +235 235 +237 474 +238 476 +239 478 +241 241 +242 484 +244 244 +247 247 +248 248 +249 249 +252 252 +255 510 +256 512 +257 257 +258 258 +260 260 +262 262 +263 263 +265 530 +266 266 +272 544 +273 819 +274 274 +275 275 +277 1108 +278 556 +280 560 +281 562 +282 564 +283 283 +284 284 +285 285 +286 286 +287 287 +288 576 +289 289 +291 291 +292 292 +296 296 +298 894 +302 302 +305 305 +306 306 +307 614 +308 308 +309 618 +310 310 +311 933 +315 315 +316 948 +317 634 +318 954 +321 642 +322 644 +323 323 +325 650 +327 981 +331 662 +332 332 +333 666 +335 335 +336 336 +338 338 +339 339 +341 341 +342 684 +344 688 +345 345 +348 1740 +351 351 +353 706 +356 356 +360 360 +362 362 +364 364 +365 365 +366 366 +367 734 +368 368 +369 1107 +373 373 +374 374 +375 375 +377 377 +378 378 +379 379 +382 764 +384 1152 +386 386 +389 389 +392 392 +393 393 +394 394 +395 790 +396 1188 +397 794 +399 798 +400 400 +401 2005 +402 402 +403 1209 +404 808 +406 1624 +407 407 +409 1227 +411 411 +413 826 +414 828 +417 1251 +418 418 +419 419 +421 421 +424 848 +427 427 +429 858 +430 1290 +431 1293 +432 432 +435 435 +436 436 +437 437 +438 1314 +439 878 +443 443 +444 444 +446 446 +448 448 +449 449 +452 452 +453 453 +454 1362 +455 455 +457 457 +458 916 +459 918 +460 460 +462 924 +463 926 +466 1398 +467 467 +468 1872 +469 2345 +470 470 +472 472 +475 475 +477 477 +478 956 +479 479 +480 1440 +481 481 +482 482 +483 483 +484 484 +485 485 +487 487 +489 1956 +490 490 +491 491 +492 984 +493 493 +494 494 +495 495 +496 496 +497 497 +498 1494 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 b/src/test/hive/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 new file mode 100644 index 0000000000000..af5653b0de758 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 @@ -0,0 +1,309 @@ +0 0 val_0 +2 2 val_2 +4 4 val_4 +5 15 val_5 +8 8 val_8 +9 9 val_9 +10 10 val_10 +11 11 val_11 +12 24 val_12 +15 30 val_15 +17 17 val_17 +18 36 val_18 +19 19 val_19 +20 20 val_20 +24 48 val_24 +26 52 val_26 +27 27 val_27 +28 28 val_28 +30 30 val_30 +33 33 val_33 +34 34 val_34 +35 105 val_35 +37 74 val_37 +41 41 val_41 +42 84 val_42 +43 43 val_43 +44 44 val_44 +47 47 val_47 +51 102 val_51 +53 53 val_53 +54 54 val_54 +57 57 val_57 +58 116 val_58 +64 64 val_64 +65 65 val_65 +66 66 val_66 +67 134 val_67 +69 69 val_69 +70 210 val_70 +72 144 val_72 +74 74 val_74 +76 152 val_76 +77 77 val_77 +78 78 val_78 +80 80 val_80 +82 82 val_82 +83 166 val_83 +84 168 val_84 +85 85 val_85 +86 86 val_86 +87 87 val_87 +90 270 val_90 +92 92 val_92 +95 190 val_95 +96 96 val_96 +97 194 val_97 +98 196 val_98 +100 200 val_100 +103 206 val_103 +104 208 val_104 +105 105 val_105 +111 111 val_111 +113 226 val_113 +114 114 val_114 +116 116 val_116 +118 236 val_118 +119 357 val_119 +120 240 val_120 +125 250 val_125 +126 126 val_126 +128 384 val_128 +129 258 val_129 +131 131 val_131 +133 133 val_133 +134 268 val_134 +136 136 val_136 +137 274 val_137 +138 552 val_138 +143 143 val_143 +145 145 val_145 +146 292 val_146 +149 298 val_149 +150 150 val_150 +152 304 val_152 +153 153 val_153 +155 155 val_155 +156 156 val_156 +157 157 val_157 +158 158 val_158 +160 160 val_160 +162 162 val_162 +163 163 val_163 +164 328 val_164 +165 330 val_165 +166 166 val_166 +167 501 val_167 +168 168 val_168 +169 676 val_169 +170 170 val_170 +172 344 val_172 +174 348 val_174 +175 350 val_175 +176 352 val_176 +177 177 val_177 +178 178 val_178 +179 358 val_179 +180 180 val_180 +181 181 val_181 +183 183 val_183 +186 186 val_186 +187 561 val_187 +189 189 val_189 +190 190 val_190 +191 382 val_191 +192 192 val_192 +193 579 val_193 +194 194 val_194 +195 390 val_195 +196 196 val_196 +197 394 val_197 +199 597 val_199 +200 400 val_200 +201 201 val_201 +202 202 val_202 +203 406 val_203 +205 410 val_205 +207 414 val_207 +208 624 val_208 +209 418 val_209 +213 426 val_213 +214 214 val_214 +216 432 val_216 +217 434 val_217 +218 218 val_218 +219 438 val_219 +221 442 val_221 +222 222 val_222 +223 446 val_223 +224 448 val_224 +226 226 val_226 +228 228 val_228 +229 458 val_229 +230 1150 val_230 +233 466 val_233 +235 235 val_235 +237 474 val_237 +238 476 val_238 +239 478 val_239 +241 241 val_241 +242 484 val_242 +244 244 val_244 +247 247 val_247 +248 248 val_248 +249 249 val_249 +252 252 val_252 +255 510 val_255 +256 512 val_256 +257 257 val_257 +258 258 val_258 +260 260 val_260 +262 262 val_262 +263 263 val_263 +265 530 val_265 +266 266 val_266 +272 544 val_272 +273 819 val_273 +274 274 val_274 +275 275 val_275 +277 1108 val_277 +278 556 val_278 +280 560 val_280 +281 562 val_281 +282 564 val_282 +283 283 val_283 +284 284 val_284 +285 285 val_285 +286 286 val_286 +287 287 val_287 +288 576 val_288 +289 289 val_289 +291 291 val_291 +292 292 val_292 +296 296 val_296 +298 894 val_298 +302 302 val_302 +305 305 val_305 +306 306 val_306 +307 614 val_307 +308 308 val_308 +309 618 val_309 +310 310 val_310 +311 933 val_311 +315 315 val_315 +316 948 val_316 +317 634 val_317 +318 954 val_318 +321 642 val_321 +322 644 val_322 +323 323 val_323 +325 650 val_325 +327 981 val_327 +331 662 val_331 +332 332 val_332 +333 666 val_333 +335 335 val_335 +336 336 val_336 +338 338 val_338 +339 339 val_339 +341 341 val_341 +342 684 val_342 +344 688 val_344 +345 345 val_345 +348 1740 val_348 +351 351 val_351 +353 706 val_353 +356 356 val_356 +360 360 val_360 +362 362 val_362 +364 364 val_364 +365 365 val_365 +366 366 val_366 +367 734 val_367 +368 368 val_368 +369 1107 val_369 +373 373 val_373 +374 374 val_374 +375 375 val_375 +377 377 val_377 +378 378 val_378 +379 379 val_379 +382 764 val_382 +384 1152 val_384 +386 386 val_386 +389 389 val_389 +392 392 val_392 +393 393 val_393 +394 394 val_394 +395 790 val_395 +396 1188 val_396 +397 794 val_397 +399 798 val_399 +400 400 val_400 +401 2005 val_401 +402 402 val_402 +403 1209 val_403 +404 808 val_404 +406 1624 val_406 +407 407 val_407 +409 1227 val_409 +411 411 val_411 +413 826 val_413 +414 828 val_414 +417 1251 val_417 +418 418 val_418 +419 419 val_419 +421 421 val_421 +424 848 val_424 +427 427 val_427 +429 858 val_429 +430 1290 val_430 +431 1293 val_431 +432 432 val_432 +435 435 val_435 +436 436 val_436 +437 437 val_437 +438 1314 val_438 +439 878 val_439 +443 443 val_443 +444 444 val_444 +446 446 val_446 +448 448 val_448 +449 449 val_449 +452 452 val_452 +453 453 val_453 +454 1362 val_454 +455 455 val_455 +457 457 val_457 +458 916 val_458 +459 918 val_459 +460 460 val_460 +462 924 val_462 +463 926 val_463 +466 1398 val_466 +467 467 val_467 +468 1872 val_468 +469 2345 val_469 +470 470 val_470 +472 472 val_472 +475 475 val_475 +477 477 val_477 +478 956 val_478 +479 479 val_479 +480 1440 val_480 +481 481 val_481 +482 482 val_482 +483 483 val_483 +484 484 val_484 +485 485 val_485 +487 487 val_487 +489 1956 val_489 +490 490 val_490 +491 491 val_491 +492 984 val_492 +493 493 val_493 +494 494 val_494 +495 495 val_495 +496 496 val_496 +497 497 val_497 +498 1494 val_498 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 b/src/test/hive/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e b/src/test/hive/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e new file mode 100644 index 0000000000000..5d9e6bbf9cedb --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e @@ -0,0 +1,309 @@ +0 0 NULL +2 2 NULL +4 4 NULL +5 15 NULL +8 8 NULL +9 9 NULL +10 10 NULL +11 11 NULL +12 24 NULL +15 30 NULL +17 17 NULL +18 36 NULL +19 19 NULL +20 20 NULL +24 48 NULL +26 52 NULL +27 27 NULL +28 28 NULL +30 30 NULL +33 33 NULL +34 34 NULL +35 105 NULL +37 74 NULL +41 41 NULL +42 84 NULL +43 43 NULL +44 44 NULL +47 47 NULL +51 102 NULL +53 53 NULL +54 54 NULL +57 57 NULL +58 116 NULL +64 64 NULL +65 65 NULL +66 66 NULL +67 134 NULL +69 69 NULL +70 210 NULL +72 144 NULL +74 74 NULL +76 152 NULL +77 77 NULL +78 78 NULL +80 80 NULL +82 82 NULL +83 166 NULL +84 168 NULL +85 85 NULL +86 86 NULL +87 87 NULL +90 270 NULL +92 92 NULL +95 190 NULL +96 96 NULL +97 194 NULL +98 196 NULL +100 200 NULL +103 206 NULL +104 208 NULL +105 105 NULL +111 111 NULL +113 226 NULL +114 114 NULL +116 116 NULL +118 236 NULL +119 357 NULL +120 240 NULL +125 250 NULL +126 126 NULL +128 384 NULL +129 258 NULL +131 131 NULL +133 133 NULL +134 268 NULL +136 136 NULL +137 274 NULL +138 552 NULL +143 143 NULL +145 145 NULL +146 292 NULL +149 298 NULL +150 150 NULL +152 304 NULL +153 153 NULL +155 155 NULL +156 156 NULL +157 157 NULL +158 158 NULL +160 160 NULL +162 162 NULL +163 163 NULL +164 328 NULL +165 330 NULL +166 166 NULL +167 501 NULL +168 168 NULL +169 676 NULL +170 170 NULL +172 344 NULL +174 348 NULL +175 350 NULL +176 352 NULL +177 177 NULL +178 178 NULL +179 358 NULL +180 180 NULL +181 181 NULL +183 183 NULL +186 186 NULL +187 561 NULL +189 189 NULL +190 190 NULL +191 382 NULL +192 192 NULL +193 579 NULL +194 194 NULL +195 390 NULL +196 196 NULL +197 394 NULL +199 597 NULL +200 400 NULL +201 201 NULL +202 202 NULL +203 406 NULL +205 410 NULL +207 414 NULL +208 624 NULL +209 418 NULL +213 426 NULL +214 214 NULL +216 432 NULL +217 434 NULL +218 218 NULL +219 438 NULL +221 442 NULL +222 222 NULL +223 446 NULL +224 448 NULL +226 226 NULL +228 228 NULL +229 458 NULL +230 1150 NULL +233 466 NULL +235 235 NULL +237 474 NULL +238 476 NULL +239 478 NULL +241 241 NULL +242 484 NULL +244 244 NULL +247 247 NULL +248 248 NULL +249 249 NULL +252 252 NULL +255 510 NULL +256 512 NULL +257 257 NULL +258 258 NULL +260 260 NULL +262 262 NULL +263 263 NULL +265 530 NULL +266 266 NULL +272 544 NULL +273 819 NULL +274 274 NULL +275 275 NULL +277 1108 NULL +278 556 NULL +280 560 NULL +281 562 NULL +282 564 NULL +283 283 NULL +284 284 NULL +285 285 NULL +286 286 NULL +287 287 NULL +288 576 NULL +289 289 NULL +291 291 NULL +292 292 NULL +296 296 NULL +298 894 NULL +302 302 NULL +305 305 NULL +306 306 NULL +307 614 NULL +308 308 NULL +309 618 NULL +310 310 NULL +311 933 NULL +315 315 NULL +316 948 NULL +317 634 NULL +318 954 NULL +321 642 NULL +322 644 NULL +323 323 NULL +325 650 NULL +327 981 NULL +331 662 NULL +332 332 NULL +333 666 NULL +335 335 NULL +336 336 NULL +338 338 NULL +339 339 NULL +341 341 NULL +342 684 NULL +344 688 NULL +345 345 NULL +348 1740 NULL +351 351 NULL +353 706 NULL +356 356 NULL +360 360 NULL +362 362 NULL +364 364 NULL +365 365 NULL +366 366 NULL +367 734 NULL +368 368 NULL +369 1107 NULL +373 373 NULL +374 374 NULL +375 375 NULL +377 377 NULL +378 378 NULL +379 379 NULL +382 764 NULL +384 1152 NULL +386 386 NULL +389 389 NULL +392 392 NULL +393 393 NULL +394 394 NULL +395 790 NULL +396 1188 NULL +397 794 NULL +399 798 NULL +400 400 NULL +401 2005 NULL +402 402 NULL +403 1209 NULL +404 808 NULL +406 1624 NULL +407 407 NULL +409 1227 NULL +411 411 NULL +413 826 NULL +414 828 NULL +417 1251 NULL +418 418 NULL +419 419 NULL +421 421 NULL +424 848 NULL +427 427 NULL +429 858 NULL +430 1290 NULL +431 1293 NULL +432 432 NULL +435 435 NULL +436 436 NULL +437 437 NULL +438 1314 NULL +439 878 NULL +443 443 NULL +444 444 NULL +446 446 NULL +448 448 NULL +449 449 NULL +452 452 NULL +453 453 NULL +454 1362 NULL +455 455 NULL +457 457 NULL +458 916 NULL +459 918 NULL +460 460 NULL +462 924 NULL +463 926 NULL +466 1398 NULL +467 467 NULL +468 1872 NULL +469 2345 NULL +470 470 NULL +472 472 NULL +475 475 NULL +477 477 NULL +478 956 NULL +479 479 NULL +480 1440 NULL +481 481 NULL +482 482 NULL +483 483 NULL +484 484 NULL +485 485 NULL +487 487 NULL +489 1956 NULL +490 490 NULL +491 491 NULL +492 984 NULL +493 493 NULL +494 494 NULL +495 495 NULL +496 496 NULL +497 497 NULL +498 1494 NULL diff --git a/src/test/hive/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c b/src/test/hive/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c new file mode 100644 index 0000000000000..4a827d502b934 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c @@ -0,0 +1,309 @@ +0 0 +2 2 +4 4 +5 15 +8 8 +9 9 +10 10 +11 11 +12 24 +15 30 +17 17 +18 36 +19 19 +20 20 +24 48 +26 52 +27 27 +28 28 +30 30 +33 33 +34 34 +35 105 +37 74 +41 41 +42 84 +43 43 +44 44 +47 47 +51 102 +53 53 +54 54 +57 57 +58 116 +64 64 +65 65 +66 66 +67 134 +69 69 +70 210 +72 144 +74 74 +76 152 +77 77 +78 78 +80 80 +82 82 +83 166 +84 168 +85 85 +86 86 +87 87 +90 270 +92 92 +95 190 +96 96 +97 194 +98 196 +100 200 +103 206 +104 208 +105 105 +111 111 +113 226 +114 114 +116 116 +118 236 +119 357 +120 240 +125 250 +126 126 +128 384 +129 258 +131 131 +133 133 +134 268 +136 136 +137 274 +138 552 +143 143 +145 145 +146 292 +149 298 +150 150 +152 304 +153 153 +155 155 +156 156 +157 157 +158 158 +160 160 +162 162 +163 163 +164 328 +165 330 +166 166 +167 501 +168 168 +169 676 +170 170 +172 344 +174 348 +175 350 +176 352 +177 177 +178 178 +179 358 +180 180 +181 181 +183 183 +186 186 +187 561 +189 189 +190 190 +191 382 +192 192 +193 579 +194 194 +195 390 +196 196 +197 394 +199 597 +200 400 +201 201 +202 202 +203 406 +205 410 +207 414 +208 624 +209 418 +213 426 +214 214 +216 432 +217 434 +218 218 +219 438 +221 442 +222 222 +223 446 +224 448 +226 226 +228 228 +229 458 +230 1150 +233 466 +235 235 +237 474 +238 476 +239 478 +241 241 +242 484 +244 244 +247 247 +248 248 +249 249 +252 252 +255 510 +256 512 +257 257 +258 258 +260 260 +262 262 +263 263 +265 530 +266 266 +272 544 +273 819 +274 274 +275 275 +277 1108 +278 556 +280 560 +281 562 +282 564 +283 283 +284 284 +285 285 +286 286 +287 287 +288 576 +289 289 +291 291 +292 292 +296 296 +298 894 +302 302 +305 305 +306 306 +307 614 +308 308 +309 618 +310 310 +311 933 +315 315 +316 948 +317 634 +318 954 +321 642 +322 644 +323 323 +325 650 +327 981 +331 662 +332 332 +333 666 +335 335 +336 336 +338 338 +339 339 +341 341 +342 684 +344 688 +345 345 +348 1740 +351 351 +353 706 +356 356 +360 360 +362 362 +364 364 +365 365 +366 366 +367 734 +368 368 +369 1107 +373 373 +374 374 +375 375 +377 377 +378 378 +379 379 +382 764 +384 1152 +386 386 +389 389 +392 392 +393 393 +394 394 +395 790 +396 1188 +397 794 +399 798 +400 400 +401 2005 +402 402 +403 1209 +404 808 +406 1624 +407 407 +409 1227 +411 411 +413 826 +414 828 +417 1251 +418 418 +419 419 +421 421 +424 848 +427 427 +429 858 +430 1290 +431 1293 +432 432 +435 435 +436 436 +437 437 +438 1314 +439 878 +443 443 +444 444 +446 446 +448 448 +449 449 +452 452 +453 453 +454 1362 +455 455 +457 457 +458 916 +459 918 +460 460 +462 924 +463 926 +466 1398 +467 467 +468 1872 +469 2345 +470 470 +472 472 +475 475 +477 477 +478 956 +479 479 +480 1440 +481 481 +482 482 +483 483 +484 484 +485 485 +487 487 +489 1956 +490 490 +491 491 +492 984 +493 493 +494 494 +495 495 +496 496 +497 497 +498 1494 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd b/src/test/hive/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd new file mode 100644 index 0000000000000..32740bfc546a9 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd @@ -0,0 +1,15 @@ +66 66 +98 196 +128 384 +146 292 +150 150 +213 426 +224 448 +238 476 +255 510 +273 819 +278 556 +311 933 +369 1107 +401 2005 +406 1624 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb b/src/test/hive/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb new file mode 100644 index 0000000000000..cc33012c6e2eb --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb @@ -0,0 +1,37 @@ +66 val_66 +98 val_98 +98 val_98 +128 val_128 +128 val_128 +128 val_128 +146 val_146 +146 val_146 +150 val_150 +213 val_213 +213 val_213 +224 val_224 +224 val_224 +238 val_238 +238 val_238 +255 val_255 +255 val_255 +273 val_273 +273 val_273 +273 val_273 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +311 val_311 +369 val_369 +369 val_369 +369 val_369 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +406 val_406 +406 val_406 +406 val_406 +406 val_406 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f b/src/test/hive/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f new file mode 100644 index 0000000000000..851d5ed155549 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f @@ -0,0 +1,309 @@ +0 +2 +4 +5 +8 +9 +10 +11 +12 +15 +17 +18 +19 +20 +24 +26 +27 +28 +30 +33 +34 +35 +37 +41 +42 +43 +44 +47 +51 +53 +54 +57 +58 +64 +65 +66 +67 +69 +70 +72 +74 +76 +77 +78 +80 +82 +83 +84 +85 +86 +87 +90 +92 +95 +96 +97 +98 +100 +103 +104 +105 +111 +113 +114 +116 +118 +119 +120 +125 +126 +128 +129 +131 +133 +134 +136 +137 +138 +143 +145 +146 +149 +150 +152 +153 +155 +156 +157 +158 +160 +162 +163 +164 +165 +166 +167 +168 +169 +170 +172 +174 +175 +176 +177 +178 +179 +180 +181 +183 +186 +187 +189 +190 +191 +192 +193 +194 +195 +196 +197 +199 +200 +201 +202 +203 +205 +207 +208 +209 +213 +214 +216 +217 +218 +219 +221 +222 +223 +224 +226 +228 +229 +230 +233 +235 +237 +238 +239 +241 +242 +244 +247 +248 +249 +252 +255 +256 +257 +258 +260 +262 +263 +265 +266 +272 +273 +274 +275 +277 +278 +280 +281 +282 +283 +284 +285 +286 +287 +288 +289 +291 +292 +296 +298 +302 +305 +306 +307 +308 +309 +310 +311 +315 +316 +317 +318 +321 +322 +323 +325 +327 +331 +332 +333 +335 +336 +338 +339 +341 +342 +344 +345 +348 +351 +353 +356 +360 +362 +364 +365 +366 +367 +368 +369 +373 +374 +375 +377 +378 +379 +382 +384 +386 +389 +392 +393 +394 +395 +396 +397 +399 +400 +401 +402 +403 +404 +406 +407 +409 +411 +413 +414 +417 +418 +419 +421 +424 +427 +429 +430 +431 +432 +435 +436 +437 +438 +439 +443 +444 +446 +448 +449 +452 +453 +454 +455 +457 +458 +459 +460 +462 +463 +466 +467 +468 +469 +470 +472 +475 +477 +478 +479 +480 +481 +482 +483 +484 +485 +487 +489 +490 +491 +492 +493 +494 +495 +496 +497 +498 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 b/src/test/hive/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/src/test/hive/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 @@ -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/src/test/hive/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 b/src/test/hive/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 b/src/test/hive/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad b/src/test/hive/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 b/src/test/hive/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 b/src/test/hive/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 b/src/test/hive/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb b/src/test/hive/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 b/src/test/hive/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 b/src/test/hive/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 b/src/test/hive/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db b/src/test/hive/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/regex_col-2-21564f64cdfd46098e1254380490701 b/src/test/hive/golden/regex_col-2-21564f64cdfd46098e1254380490701 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 b/src/test/hive/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 b/src/test/hive/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 b/src/test/hive/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/remote_script-1-e168f471980470d93b790702a70238fa b/src/test/hive/golden/remote_script-1-e168f471980470d93b790702a70238fa new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/remote_script-1-e168f471980470d93b790702a70238fa @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/remote_script-2-a19a19272149c732977c37e043910505 b/src/test/hive/golden/remote_script-2-a19a19272149c732977c37e043910505 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/remote_script-2-a19a19272149c732977c37e043910505 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/remote_script-3-4eb54a664e549614d56ca088c8867d b/src/test/hive/golden/remote_script-3-4eb54a664e549614d56ca088c8867d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 b/src/test/hive/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..726e0947d8302 --- /dev/null +++ b/src/test/hive/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +a int None +b int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 b/src/test/hive/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..500201be8d922 --- /dev/null +++ b/src/test/hive/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +b int None +a1 int test comment1 +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 b/src/test/hive/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..71af16a5fa4d7 --- /dev/null +++ b/src/test/hive/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +a2 int test comment2 +b int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 b/src/test/hive/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..f5b2e72aeccab --- /dev/null +++ b/src/test/hive/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +b int None +a int test comment2 +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-16-d032f4795c1186255acea241387adf93 b/src/test/hive/golden/rename_column-16-d032f4795c1186255acea241387adf93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-17-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/rename_column-17-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/src/test/hive/golden/rename_column-17-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 b/src/test/hive/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f b/src/test/hive/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a b/src/test/hive/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 b/src/test/hive/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..726e0947d8302 --- /dev/null +++ b/src/test/hive/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +a int None +b int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a b/src/test/hive/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..17127eaec9755 --- /dev/null +++ b/src/test/hive/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +a string None +b int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 b/src/test/hive/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..bfcefe4d18046 --- /dev/null +++ b/src/test/hive/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +a1 int None +b int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd b/src/test/hive/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..c436c39a16b8a --- /dev/null +++ b/src/test/hive/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +a2 int None +b int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 b/src/test/hive/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..bb1507e7488f5 --- /dev/null +++ b/src/test/hive/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +b int None +a int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..17127eaec9755 --- /dev/null +++ b/src/test/hive/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +a string None +b int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 b/src/test/hive/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..500201be8d922 --- /dev/null +++ b/src/test/hive/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +b int None +a1 int test comment1 +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 b/src/test/hive/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..71af16a5fa4d7 --- /dev/null +++ b/src/test/hive/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +a2 int test comment2 +b int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 b/src/test/hive/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..f5b2e72aeccab --- /dev/null +++ b/src/test/hive/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +b int None +a int test comment2 +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-36-d032f4795c1186255acea241387adf93 b/src/test/hive/golden/rename_column-36-d032f4795c1186255acea241387adf93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-37-9c36cac1372650b703400c60dd29042c b/src/test/hive/golden/rename_column-37-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 b/src/test/hive/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..bfcefe4d18046 --- /dev/null +++ b/src/test/hive/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +a1 int None +b int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd b/src/test/hive/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..c436c39a16b8a --- /dev/null +++ b/src/test/hive/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +a2 int None +b int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 b/src/test/hive/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b b/src/test/hive/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..bb1507e7488f5 --- /dev/null +++ b/src/test/hive/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b @@ -0,0 +1,3 @@ +b int None +a int None +c int None \ No newline at end of file diff --git a/src/test/hive/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 b/src/test/hive/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 b/src/test/hive/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 b/src/test/hive/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 new file mode 100644 index 0000000000000..3a57720041fb3 --- /dev/null +++ b/src/test/hive/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 @@ -0,0 +1,12 @@ +17 val_17 17 val_17 +19 val_19 19 val_19 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +17 val_17 17 val_17 +19 val_19 19 val_19 +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 diff --git a/src/test/hive/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc b/src/test/hive/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 b/src/test/hive/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 new file mode 100644 index 0000000000000..19492fd335bcb --- /dev/null +++ b/src/test/hive/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 @@ -0,0 +1,12 @@ +17 val_17 17 val_17 +17 val_17 17 val_17 +19 val_19 19 val_19 +19 val_19 19 val_19 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +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 diff --git a/src/test/hive/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 b/src/test/hive/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f b/src/test/hive/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f new file mode 100644 index 0000000000000..3a57720041fb3 --- /dev/null +++ b/src/test/hive/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f @@ -0,0 +1,12 @@ +17 val_17 17 val_17 +19 val_19 19 val_19 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +17 val_17 17 val_17 +19 val_19 19 val_19 +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 diff --git a/src/test/hive/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 b/src/test/hive/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e b/src/test/hive/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e new file mode 100644 index 0000000000000..19492fd335bcb --- /dev/null +++ b/src/test/hive/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e @@ -0,0 +1,12 @@ +17 val_17 17 val_17 +17 val_17 17 val_17 +19 val_19 19 val_19 +19 val_19 19 val_19 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +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 diff --git a/src/test/hive/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 b/src/test/hive/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 b/src/test/hive/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 new file mode 100644 index 0000000000000..0bc999a3e6aa2 --- /dev/null +++ b/src/test/hive/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 @@ -0,0 +1,2 @@ +1 +1 \ No newline at end of file diff --git a/src/test/hive/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef b/src/test/hive/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 b/src/test/hive/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 new file mode 100644 index 0000000000000..0bc999a3e6aa2 --- /dev/null +++ b/src/test/hive/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 @@ -0,0 +1,2 @@ +1 +1 \ No newline at end of file diff --git a/src/test/hive/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 b/src/test/hive/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 b/src/test/hive/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 b/src/test/hive/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 b/src/test/hive/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f b/src/test/hive/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f new file mode 100644 index 0000000000000..9cf5170e82d7d --- /dev/null +++ b/src/test/hive/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f @@ -0,0 +1 @@ +238 val_238 238 val_238 \ No newline at end of file diff --git a/src/test/hive/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 b/src/test/hive/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 b/src/test/hive/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a b/src/test/hive/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 b/src/test/hive/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 new file mode 100644 index 0000000000000..5f4de85940513 --- /dev/null +++ b/src/test/hive/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 @@ -0,0 +1 @@ +0 val_0 \ No newline at end of file diff --git a/src/test/hive/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 b/src/test/hive/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 b/src/test/hive/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 b/src/test/hive/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e b/src/test/hive/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e new file mode 100644 index 0000000000000..f35d9602e1a7b --- /dev/null +++ b/src/test/hive/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e @@ -0,0 +1 @@ +498 \ No newline at end of file diff --git a/src/test/hive/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a b/src/test/hive/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 b/src/test/hive/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 b/src/test/hive/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 b/src/test/hive/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 b/src/test/hive/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 b/src/test/hive/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e b/src/test/hive/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e new file mode 100644 index 0000000000000..f35d9602e1a7b --- /dev/null +++ b/src/test/hive/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e @@ -0,0 +1 @@ +498 \ No newline at end of file diff --git a/src/test/hive/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 b/src/test/hive/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde b/src/test/hive/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 b/src/test/hive/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 b/src/test/hive/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 b/src/test/hive/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 b/src/test/hive/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e b/src/test/hive/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e new file mode 100644 index 0000000000000..f35d9602e1a7b --- /dev/null +++ b/src/test/hive/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e @@ -0,0 +1 @@ +498 \ No newline at end of file diff --git a/src/test/hive/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 b/src/test/hive/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f b/src/test/hive/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 b/src/test/hive/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 b/src/test/hive/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 b/src/test/hive/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 new file mode 100644 index 0000000000000..d1168556e09d4 --- /dev/null +++ b/src/test/hive/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 @@ -0,0 +1,11 @@ +myint int from deserializer +mystring string from deserializer +underscore_int int from deserializer +b string None + +# Partition Information +# col_name data_type comment + +b string None + +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 diff --git a/src/test/hive/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d b/src/test/hive/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa b/src/test/hive/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa new file mode 100644 index 0000000000000..0b8f428b24193 --- /dev/null +++ b/src/test/hive/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa @@ -0,0 +1,11 @@ +myint int from deserializer +mystring string from deserializer +underscore_int int from deserializer +b string None + +# Partition Information +# col_name data_type comment + +b string None + +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 diff --git a/src/test/hive/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d b/src/test/hive/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 b/src/test/hive/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 b/src/test/hive/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 b/src/test/hive/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 b/src/test/hive/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 b/src/test/hive/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd b/src/test/hive/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a b/src/test/hive/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b b/src/test/hive/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d b/src/test/hive/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 b/src/test/hive/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 b/src/test/hive/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 new file mode 100644 index 0000000000000..fd33cfcc9ab20 --- /dev/null +++ b/src/test/hive/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 @@ -0,0 +1 @@ +concat \ No newline at end of file diff --git a/src/test/hive/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d b/src/test/hive/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d new file mode 100644 index 0000000000000..fd33cfcc9ab20 --- /dev/null +++ b/src/test/hive/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d @@ -0,0 +1 @@ +concat \ No newline at end of file diff --git a/src/test/hive/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 b/src/test/hive/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 new file mode 100644 index 0000000000000..a3998eeeee623 --- /dev/null +++ b/src/test/hive/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 b/src/test/hive/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 new file mode 100644 index 0000000000000..a3998eeeee623 --- /dev/null +++ b/src/test/hive/golden/show_describe_func_quotes-3-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 \ No newline at end of file diff --git a/src/test/hive/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 b/src/test/hive/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 new file mode 100644 index 0000000000000..9d40ffaef5862 --- /dev/null +++ b/src/test/hive/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 @@ -0,0 +1,192 @@ +! +!= +% +& +* ++ +- +/ +< +<= +<=> +<> += +== +> +>= +^ +abs +acos +and +array +array_contains +ascii +asin +assert_true +atan +avg +base64 +between +bin +case +ceil +ceiling +coalesce +collect_set +compute_stats +concat +concat_ws +context_ngrams +conv +corr +cos +count +covar_pop +covar_samp +create_union +cume_dist +date_add +date_sub +datediff +day +dayofmonth +decode +degrees +dense_rank +div +e +elt +encode +ewah_bitmap +ewah_bitmap_and +ewah_bitmap_empty +ewah_bitmap_or +exp +explode +field +find_in_set +first_value +floor +format_number +from_unixtime +from_utc_timestamp +get_json_object +hash +hex +histogram_numeric +hour +if +in +in_file +index +inline +instr +isnotnull +isnull +java_method +json_tuple +lag +last_value +lcase +lead +length +like +ln +locate +log +log10 +log2 +lower +lpad +ltrim +map +map_keys +map_values +matchpath +max +min +minute +month +named_struct +negative +ngrams +noop +noopwithmap +not +ntile +nvl +or +parse_url +parse_url_tuple +percent_rank +percentile +percentile_approx +pi +pmod +positive +pow +power +printf +radians +rand +rank +reflect +reflect2 +regexp +regexp_extract +regexp_replace +repeat +reverse +rlike +round +row_number +rpad +rtrim +second +sentences +sign +sin +size +sort_array +space +split +sqrt +stack +std +stddev +stddev_pop +stddev_samp +str_to_map +struct +substr +substring +sum +tan +to_date +to_unix_timestamp +to_utc_timestamp +translate +trim +ucase +unbase64 +unhex +unix_timestamp +upper +var_pop +var_samp +variance +weekofyear +when +windowingtablefunction +xpath +xpath_boolean +xpath_double +xpath_float +xpath_int +xpath_long +xpath_number +xpath_short +xpath_string +year +| +~ \ No newline at end of file diff --git a/src/test/hive/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 b/src/test/hive/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 new file mode 100644 index 0000000000000..c62b965cb1559 --- /dev/null +++ b/src/test/hive/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 @@ -0,0 +1,17 @@ +case +ceil +ceiling +coalesce +collect_set +compute_stats +concat +concat_ws +context_ngrams +conv +corr +cos +count +covar_pop +covar_samp +create_union +cume_dist \ No newline at end of file diff --git a/src/test/hive/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c b/src/test/hive/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c new file mode 100644 index 0000000000000..a56b5a3766c5c --- /dev/null +++ b/src/test/hive/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c @@ -0,0 +1,32 @@ +assert_true +case +coalesce +decode +e +encode +explode +first_value +from_unixtime +in_file +inline +json_tuple +last_value +lcase +like +locate +minute +negative +ntile +parse_url_tuple +percentile +positive +regexp_replace +reverse +rlike +size +space +to_date +translate +ucase +variance +xpath_double \ No newline at end of file diff --git a/src/test/hive/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 b/src/test/hive/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 new file mode 100644 index 0000000000000..86605075c3d25 --- /dev/null +++ b/src/test/hive/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 @@ -0,0 +1,3 @@ +log +log10 +log2 \ No newline at end of file diff --git a/src/test/hive/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 b/src/test/hive/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 new file mode 100644 index 0000000000000..312f6cdbf68c7 --- /dev/null +++ b/src/test/hive/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 @@ -0,0 +1,4 @@ +date_add +date_sub +datediff +to_date \ No newline at end of file diff --git a/src/test/hive/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 b/src/test/hive/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 b/src/test/hive/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 b/src/test/hive/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 b/src/test/hive/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e b/src/test/hive/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b b/src/test/hive/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b new file mode 100644 index 0000000000000..e772f4a83fecd --- /dev/null +++ b/src/test/hive/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b @@ -0,0 +1,16 @@ +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/src/test/hive/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 b/src/test/hive/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 new file mode 100644 index 0000000000000..d68fbfc9c1e0f --- /dev/null +++ b/src/test/hive/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e b/src/test/hive/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e new file mode 100644 index 0000000000000..94ee57be9dcaf --- /dev/null +++ b/src/test/hive/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e @@ -0,0 +1,16 @@ +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/src/test/hive/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d b/src/test/hive/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f b/src/test/hive/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 b/src/test/hive/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 b/src/test/hive/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 b/src/test/hive/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 b/src/test/hive/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 b/src/test/hive/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc b/src/test/hive/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 b/src/test/hive/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc b/src/test/hive/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 b/src/test/hive/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c b/src/test/hive/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 b/src/test/hive/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec b/src/test/hive/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 b/src/test/hive/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec b/src/test/hive/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 b/src/test/hive/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 b/src/test/hive/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 b/src/test/hive/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 new file mode 100644 index 0000000000000..7e68a8acb1a87 --- /dev/null +++ b/src/test/hive/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 @@ -0,0 +1,16 @@ +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/src/test/hive/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 b/src/test/hive/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 new file mode 100644 index 0000000000000..36d22451eba3e --- /dev/null +++ b/src/test/hive/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 @@ -0,0 +1 @@ +idx_t1 show_idx_t1 key default__show_idx_t1_idx_t1__ compact \ No newline at end of file diff --git a/src/test/hive/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 b/src/test/hive/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 new file mode 100644 index 0000000000000..7e68a8acb1a87 --- /dev/null +++ b/src/test/hive/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 @@ -0,0 +1,16 @@ +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/src/test/hive/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 b/src/test/hive/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 new file mode 100644 index 0000000000000..36d22451eba3e --- /dev/null +++ b/src/test/hive/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 @@ -0,0 +1 @@ +idx_t1 show_idx_t1 key default__show_idx_t1_idx_t1__ compact \ No newline at end of file diff --git a/src/test/hive/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e b/src/test/hive/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e new file mode 100644 index 0000000000000..4dddeee9a233f --- /dev/null +++ b/src/test/hive/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e @@ -0,0 +1,16 @@ +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/src/test/hive/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 b/src/test/hive/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 new file mode 100644 index 0000000000000..76e0434294b4f --- /dev/null +++ b/src/test/hive/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a b/src/test/hive/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/src/test/hive/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a @@ -0,0 +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 diff --git a/src/test/hive/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 b/src/test/hive/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 new file mode 100644 index 0000000000000..19b4a62499762 --- /dev/null +++ b/src/test/hive/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 @@ -0,0 +1,2 @@ +ds=2008-04-08/hr=11 +ds=2008-04-09/hr=11 \ No newline at end of file diff --git a/src/test/hive/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca b/src/test/hive/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca new file mode 100644 index 0000000000000..f3614273fa8fa --- /dev/null +++ b/src/test/hive/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca @@ -0,0 +1,2 @@ +ds=2008-04-08/hr=11 +ds=2008-04-08/hr=12 \ No newline at end of file diff --git a/src/test/hive/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 b/src/test/hive/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 new file mode 100644 index 0000000000000..0cdd3e8594c59 --- /dev/null +++ b/src/test/hive/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 @@ -0,0 +1 @@ +ds=2008-04-08/hr=12 \ No newline at end of file diff --git a/src/test/hive/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 b/src/test/hive/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce b/src/test/hive/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 b/src/test/hive/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-11-3f9a7f993510123059493826470f78f7 b/src/test/hive/golden/show_tables-11-3f9a7f993510123059493826470f78f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c b/src/test/hive/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c new file mode 100644 index 0000000000000..60f7943eda4a9 --- /dev/null +++ b/src/test/hive/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c @@ -0,0 +1,3 @@ +bar +baz +foo \ No newline at end of file diff --git a/src/test/hive/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 b/src/test/hive/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 new file mode 100644 index 0000000000000..4ffc580e2b8f3 --- /dev/null +++ b/src/test/hive/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 @@ -0,0 +1,4 @@ +shtb_test1 +shtb_test2 +src +srcpart \ No newline at end of file diff --git a/src/test/hive/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c b/src/test/hive/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c new file mode 100644 index 0000000000000..60f7943eda4a9 --- /dev/null +++ b/src/test/hive/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c @@ -0,0 +1,3 @@ +bar +baz +foo \ No newline at end of file diff --git a/src/test/hive/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe b/src/test/hive/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe new file mode 100644 index 0000000000000..4ffc580e2b8f3 --- /dev/null +++ b/src/test/hive/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe @@ -0,0 +1,4 @@ +shtb_test1 +shtb_test2 +src +srcpart \ No newline at end of file diff --git a/src/test/hive/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 b/src/test/hive/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab b/src/test/hive/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 b/src/test/hive/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 b/src/test/hive/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 b/src/test/hive/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 new file mode 100644 index 0000000000000..916ac1482c061 --- /dev/null +++ b/src/test/hive/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 @@ -0,0 +1,18 @@ +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/src/test/hive/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 b/src/test/hive/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 b/src/test/hive/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 b/src/test/hive/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 new file mode 100644 index 0000000000000..19102815663d2 --- /dev/null +++ b/src/test/hive/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 @@ -0,0 +1 @@ +foo \ No newline at end of file diff --git a/src/test/hive/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c b/src/test/hive/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c new file mode 100644 index 0000000000000..b67b816ee4b45 --- /dev/null +++ b/src/test/hive/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c @@ -0,0 +1,2 @@ +shtb_test1 +shtb_test2 \ No newline at end of file diff --git a/src/test/hive/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 b/src/test/hive/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 new file mode 100644 index 0000000000000..8f06e234b2a6e --- /dev/null +++ b/src/test/hive/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 @@ -0,0 +1,18 @@ +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/src/test/hive/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d b/src/test/hive/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d new file mode 100644 index 0000000000000..b67b816ee4b45 --- /dev/null +++ b/src/test/hive/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d @@ -0,0 +1,2 @@ +shtb_test1 +shtb_test2 \ No newline at end of file diff --git a/src/test/hive/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 b/src/test/hive/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c b/src/test/hive/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 b/src/test/hive/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 b/src/test/hive/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa b/src/test/hive/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa new file mode 100644 index 0000000000000..bec424bb026e9 --- /dev/null +++ b/src/test/hive/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa @@ -0,0 +1,14 @@ +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/src/test/hive/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 b/src/test/hive/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 new file mode 100644 index 0000000000000..9392b7dc686f6 --- /dev/null +++ b/src/test/hive/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 b/src/test/hive/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 new file mode 100644 index 0000000000000..9392b7dc686f6 --- /dev/null +++ b/src/test/hive/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d b/src/test/hive/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d new file mode 100644 index 0000000000000..9392b7dc686f6 --- /dev/null +++ b/src/test/hive/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d @@ -0,0 +1,14 @@ +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/src/test/hive/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 b/src/test/hive/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 b/src/test/hive/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 b/src/test/hive/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c b/src/test/hive/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 b/src/test/hive/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 new file mode 100644 index 0000000000000..f8b64f6056ea7 --- /dev/null +++ b/src/test/hive/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 b/src/test/hive/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 new file mode 100644 index 0000000000000..9392b7dc686f6 --- /dev/null +++ b/src/test/hive/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/showparts-0-593619bb962b318b82896658deaea1f1 b/src/test/hive/golden/showparts-0-593619bb962b318b82896658deaea1f1 new file mode 100644 index 0000000000000..b590724bca78d --- /dev/null +++ b/src/test/hive/golden/showparts-0-593619bb962b318b82896658deaea1f1 @@ -0,0 +1,17 @@ +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/src/test/hive/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a b/src/test/hive/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/src/test/hive/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a @@ -0,0 +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 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b b/src/test/hive/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f b/src/test/hive/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 b/src/test/hive/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 b/src/test/hive/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e b/src/test/hive/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 b/src/test/hive/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 b/src/test/hive/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 b/src/test/hive/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 b/src/test/hive/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b b/src/test/hive/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 b/src/test/hive/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 b/src/test/hive/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a b/src/test/hive/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 b/src/test/hive/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 b/src/test/hive/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 b/src/test/hive/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 b/src/test/hive/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 b/src/test/hive/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 b/src/test/hive/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 b/src/test/hive/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd b/src/test/hive/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 b/src/test/hive/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 b/src/test/hive/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 new file mode 100644 index 0000000000000..872146532307a --- /dev/null +++ b/src/test/hive/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 @@ -0,0 +1 @@ +2 12 2 22 2 12 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 b/src/test/hive/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 b/src/test/hive/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 b/src/test/hive/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 b/src/test/hive/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d b/src/test/hive/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 b/src/test/hive/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 b/src/test/hive/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 b/src/test/hive/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 b/src/test/hive/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 b/src/test/hive/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 b/src/test/hive/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 b/src/test/hive/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 b/src/test/hive/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/src/test/hive/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 b/src/test/hive/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d b/src/test/hive/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 b/src/test/hive/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 b/src/test/hive/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e b/src/test/hive/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b b/src/test/hive/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b b/src/test/hive/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 b/src/test/hive/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b b/src/test/hive/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 b/src/test/hive/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 b/src/test/hive/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa b/src/test/hive/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa new file mode 100644 index 0000000000000..ee1bb6b112381 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa @@ -0,0 +1,6 @@ +2 12 2 22 +3 13 3 13 +8 18 8 18 +8 18 8 18 +8 28 8 18 +8 28 8 18 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 b/src/test/hive/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/src/test/hive/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 b/src/test/hive/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d b/src/test/hive/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 b/src/test/hive/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 b/src/test/hive/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e b/src/test/hive/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 b/src/test/hive/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 b/src/test/hive/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d b/src/test/hive/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 b/src/test/hive/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 b/src/test/hive/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 b/src/test/hive/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 b/src/test/hive/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb b/src/test/hive/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 b/src/test/hive/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 b/src/test/hive/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f b/src/test/hive/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 b/src/test/hive/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 b/src/test/hive/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 b/src/test/hive/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b b/src/test/hive/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b b/src/test/hive/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d b/src/test/hive/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 b/src/test/hive/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd b/src/test/hive/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 b/src/test/hive/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 b/src/test/hive/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 b/src/test/hive/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e b/src/test/hive/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 b/src/test/hive/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c b/src/test/hive/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 b/src/test/hive/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 b/src/test/hive/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e b/src/test/hive/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 b/src/test/hive/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd b/src/test/hive/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 b/src/test/hive/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 b/src/test/hive/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 new file mode 100644 index 0000000000000..acd4039d35669 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 @@ -0,0 +1,4 @@ +2 1 2 22 +3 1 3 13 +8 2 8 18 +8 2 8 18 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d b/src/test/hive/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 b/src/test/hive/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/src/test/hive/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 b/src/test/hive/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 b/src/test/hive/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 b/src/test/hive/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d b/src/test/hive/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d new file mode 100644 index 0000000000000..f608d53f51ebe --- /dev/null +++ b/src/test/hive/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d @@ -0,0 +1,12 @@ +2 12 2 22 +2 12 2 22 +3 13 3 13 +3 13 3 13 +8 18 8 18 +8 18 8 18 +8 18 8 18 +8 18 8 18 +8 28 8 18 +8 28 8 18 +8 28 8 18 +8 28 8 18 \ No newline at end of file diff --git a/src/test/hive/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 b/src/test/hive/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 b/src/test/hive/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 new file mode 100644 index 0000000000000..7a442f02e8d7a --- /dev/null +++ b/src/test/hive/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 @@ -0,0 +1,3 @@ +0 2 +0 2 +0 2 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a b/src/test/hive/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec b/src/test/hive/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 b/src/test/hive/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 new file mode 100644 index 0000000000000..836ee718649ad --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 @@ -0,0 +1 @@ +51 val_3 51 val_30 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed b/src/test/hive/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa b/src/test/hive/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa new file mode 100644 index 0000000000000..8f358bffec51d --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa @@ -0,0 +1,5 @@ +49 val_1 NULL NULL +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 diff --git a/src/test/hive/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 b/src/test/hive/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 b/src/test/hive/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 new file mode 100644 index 0000000000000..6197cf72c3454 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 @@ -0,0 +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 diff --git a/src/test/hive/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c b/src/test/hive/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 b/src/test/hive/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 new file mode 100644 index 0000000000000..d650d44f42404 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 @@ -0,0 +1,8 @@ +49 val_1 NULL NULL +49 val_10 NULL NULL +NULL NULL 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 b/src/test/hive/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 b/src/test/hive/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 new file mode 100644 index 0000000000000..836ee718649ad --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 @@ -0,0 +1 @@ +51 val_3 51 val_30 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 b/src/test/hive/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f b/src/test/hive/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 b/src/test/hive/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 new file mode 100644 index 0000000000000..8f358bffec51d --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 @@ -0,0 +1,5 @@ +49 val_1 NULL NULL +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 diff --git a/src/test/hive/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec b/src/test/hive/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b b/src/test/hive/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b new file mode 100644 index 0000000000000..6197cf72c3454 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b @@ -0,0 +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 diff --git a/src/test/hive/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd b/src/test/hive/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae b/src/test/hive/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae new file mode 100644 index 0000000000000..d650d44f42404 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae @@ -0,0 +1,8 @@ +49 val_1 NULL NULL +49 val_10 NULL NULL +NULL NULL 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 b/src/test/hive/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de b/src/test/hive/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 b/src/test/hive/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea b/src/test/hive/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 b/src/test/hive/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 b/src/test/hive/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 b/src/test/hive/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 b/src/test/hive/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 b/src/test/hive/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 b/src/test/hive/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 b/src/test/hive/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 b/src/test/hive/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa b/src/test/hive/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 b/src/test/hive/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 b/src/test/hive/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 b/src/test/hive/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 b/src/test/hive/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 b/src/test/hive/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af b/src/test/hive/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f b/src/test/hive/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 b/src/test/hive/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 b/src/test/hive/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 b/src/test/hive/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 b/src/test/hive/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb b/src/test/hive/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f b/src/test/hive/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 b/src/test/hive/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af b/src/test/hive/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af new file mode 100644 index 0000000000000..8975db9a05036 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af @@ -0,0 +1 @@ +293 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b b/src/test/hive/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e b/src/test/hive/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e b/src/test/hive/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e new file mode 100644 index 0000000000000..8975db9a05036 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e @@ -0,0 +1 @@ +293 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 b/src/test/hive/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 b/src/test/hive/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 b/src/test/hive/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 b/src/test/hive/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 b/src/test/hive/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee b/src/test/hive/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 b/src/test/hive/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 new file mode 100644 index 0000000000000..b828077157966 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 @@ -0,0 +1,10 @@ +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 2 val_2 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 b/src/test/hive/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 b/src/test/hive/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 b/src/test/hive/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 b/src/test/hive/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd b/src/test/hive/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d b/src/test/hive/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 b/src/test/hive/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a b/src/test/hive/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a new file mode 100644 index 0000000000000..4a9735f855f96 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a @@ -0,0 +1,6 @@ +0 9 +2 1 +4 1 +5 9 +8 1 +9 1 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d b/src/test/hive/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 b/src/test/hive/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 new file mode 100644 index 0000000000000..62f9457511f87 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 @@ -0,0 +1 @@ +6 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 b/src/test/hive/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee b/src/test/hive/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 b/src/test/hive/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 b/src/test/hive/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a b/src/test/hive/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 b/src/test/hive/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b b/src/test/hive/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 b/src/test/hive/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 b/src/test/hive/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e b/src/test/hive/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b b/src/test/hive/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 b/src/test/hive/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 b/src/test/hive/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b b/src/test/hive/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da b/src/test/hive/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb b/src/test/hive/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 b/src/test/hive/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 new file mode 100644 index 0000000000000..2ebc6516c7df1 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 @@ -0,0 +1 @@ +56 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 b/src/test/hive/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 b/src/test/hive/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 b/src/test/hive/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f b/src/test/hive/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f b/src/test/hive/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b b/src/test/hive/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 b/src/test/hive/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a b/src/test/hive/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a new file mode 100644 index 0000000000000..b828077157966 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a @@ -0,0 +1,10 @@ +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 2 val_2 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 b/src/test/hive/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 b/src/test/hive/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 b/src/test/hive/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 b/src/test/hive/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 b/src/test/hive/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c b/src/test/hive/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f b/src/test/hive/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f new file mode 100644 index 0000000000000..471d725e7bfa3 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f @@ -0,0 +1,10 @@ +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +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 diff --git a/src/test/hive/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 b/src/test/hive/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e b/src/test/hive/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e new file mode 100644 index 0000000000000..471d725e7bfa3 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e @@ -0,0 +1,10 @@ +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +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 diff --git a/src/test/hive/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 b/src/test/hive/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 b/src/test/hive/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 new file mode 100644 index 0000000000000..471d725e7bfa3 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 @@ -0,0 +1,10 @@ +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +0 0 val_0 0 0 val_0 +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 diff --git a/src/test/hive/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 b/src/test/hive/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 b/src/test/hive/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb b/src/test/hive/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 b/src/test/hive/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b b/src/test/hive/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c b/src/test/hive/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf b/src/test/hive/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf new file mode 100644 index 0000000000000..67d892c80f493 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf @@ -0,0 +1 @@ +1028 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 b/src/test/hive/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e b/src/test/hive/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 b/src/test/hive/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 b/src/test/hive/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 b/src/test/hive/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 b/src/test/hive/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 b/src/test/hive/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 b/src/test/hive/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 b/src/test/hive/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 b/src/test/hive/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 b/src/test/hive/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 b/src/test/hive/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c b/src/test/hive/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f b/src/test/hive/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 b/src/test/hive/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e b/src/test/hive/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 b/src/test/hive/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a b/src/test/hive/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a new file mode 100644 index 0000000000000..25398d9017c7f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a @@ -0,0 +1 @@ +4378 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 b/src/test/hive/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 b/src/test/hive/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 new file mode 100644 index 0000000000000..25398d9017c7f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 @@ -0,0 +1 @@ +4378 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e b/src/test/hive/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc b/src/test/hive/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc new file mode 100644 index 0000000000000..09b5b315bcf45 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc @@ -0,0 +1 @@ +13126 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff b/src/test/hive/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 b/src/test/hive/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e b/src/test/hive/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/src/test/hive/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f b/src/test/hive/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 b/src/test/hive/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 b/src/test/hive/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 b/src/test/hive/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 b/src/test/hive/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 b/src/test/hive/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/src/test/hive/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d b/src/test/hive/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 b/src/test/hive/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a b/src/test/hive/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 b/src/test/hive/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 b/src/test/hive/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 b/src/test/hive/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c b/src/test/hive/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 b/src/test/hive/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f b/src/test/hive/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 b/src/test/hive/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab b/src/test/hive/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 b/src/test/hive/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 b/src/test/hive/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 b/src/test/hive/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c b/src/test/hive/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe b/src/test/hive/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 b/src/test/hive/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 b/src/test/hive/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c b/src/test/hive/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 b/src/test/hive/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/src/test/hive/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/src/test/hive/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f b/src/test/hive/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 b/src/test/hive/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 b/src/test/hive/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 new file mode 100644 index 0000000000000..dce6588ca1420 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 @@ -0,0 +1 @@ +36 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 b/src/test/hive/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 new file mode 100644 index 0000000000000..86ee83a4a2686 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 @@ -0,0 +1 @@ +40 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab b/src/test/hive/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab new file mode 100644 index 0000000000000..d99e90eb9675f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab @@ -0,0 +1 @@ +29 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc b/src/test/hive/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc new file mode 100644 index 0000000000000..dce6588ca1420 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc @@ -0,0 +1 @@ +36 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec b/src/test/hive/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec new file mode 100644 index 0000000000000..86ee83a4a2686 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec @@ -0,0 +1 @@ +40 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e b/src/test/hive/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e new file mode 100644 index 0000000000000..d99e90eb9675f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e @@ -0,0 +1 @@ +29 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/src/test/hive/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b b/src/test/hive/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b new file mode 100644 index 0000000000000..dce6588ca1420 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b @@ -0,0 +1 @@ +36 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 b/src/test/hive/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 new file mode 100644 index 0000000000000..86ee83a4a2686 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 @@ -0,0 +1 @@ +40 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f b/src/test/hive/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f new file mode 100644 index 0000000000000..d99e90eb9675f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f @@ -0,0 +1 @@ +29 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb b/src/test/hive/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb new file mode 100644 index 0000000000000..dce6588ca1420 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb @@ -0,0 +1 @@ +36 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 b/src/test/hive/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 new file mode 100644 index 0000000000000..86ee83a4a2686 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 @@ -0,0 +1 @@ +40 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f b/src/test/hive/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f new file mode 100644 index 0000000000000..d99e90eb9675f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f @@ -0,0 +1 @@ +29 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d b/src/test/hive/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 b/src/test/hive/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/src/test/hive/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a b/src/test/hive/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec b/src/test/hive/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac b/src/test/hive/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac new file mode 100644 index 0000000000000..7b5974818c085 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac @@ -0,0 +1,7 @@ +52 val_4 52 val_4 +49 val_1 49 val_10 +49 val_10 49 val_10 +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 diff --git a/src/test/hive/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 b/src/test/hive/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 b/src/test/hive/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 new file mode 100644 index 0000000000000..0e1d132524064 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 @@ -0,0 +1,9 @@ +49 val_1 49 val_10 +49 val_1 49 val_17 +49 val_1 49 val_19 +51 val_3 NULL NULL +52 val_4 52 val_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 diff --git a/src/test/hive/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d b/src/test/hive/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 b/src/test/hive/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 new file mode 100644 index 0000000000000..32be455fba8c4 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 @@ -0,0 +1,9 @@ +52 val_4 52 val_4 +49 val_1 49 val_10 +49 val_10 49 val_10 +49 val_1 49 val_17 +49 val_10 49 val_17 +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 diff --git a/src/test/hive/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 b/src/test/hive/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca b/src/test/hive/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca new file mode 100644 index 0000000000000..2496c4400b0b9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca @@ -0,0 +1,11 @@ +49 val_1 49 val_10 +49 val_1 49 val_17 +49 val_1 49 val_19 +49 val_10 49 val_10 +49 val_10 49 val_17 +49 val_10 49 val_19 +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 diff --git a/src/test/hive/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 b/src/test/hive/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 b/src/test/hive/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 new file mode 100644 index 0000000000000..7b5974818c085 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 @@ -0,0 +1,7 @@ +52 val_4 52 val_4 +49 val_1 49 val_10 +49 val_10 49 val_10 +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 diff --git a/src/test/hive/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 b/src/test/hive/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f b/src/test/hive/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b b/src/test/hive/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b new file mode 100644 index 0000000000000..0e1d132524064 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b @@ -0,0 +1,9 @@ +49 val_1 49 val_10 +49 val_1 49 val_17 +49 val_1 49 val_19 +51 val_3 NULL NULL +52 val_4 52 val_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 diff --git a/src/test/hive/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 b/src/test/hive/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 b/src/test/hive/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 new file mode 100644 index 0000000000000..32be455fba8c4 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 @@ -0,0 +1,9 @@ +52 val_4 52 val_4 +49 val_1 49 val_10 +49 val_10 49 val_10 +49 val_1 49 val_17 +49 val_10 49 val_17 +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 diff --git a/src/test/hive/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a b/src/test/hive/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 b/src/test/hive/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 new file mode 100644 index 0000000000000..2496c4400b0b9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 @@ -0,0 +1,11 @@ +49 val_1 49 val_10 +49 val_1 49 val_17 +49 val_1 49 val_19 +49 val_10 49 val_10 +49 val_10 49 val_17 +49 val_10 49 val_19 +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 diff --git a/src/test/hive/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 b/src/test/hive/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de b/src/test/hive/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 b/src/test/hive/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 b/src/test/hive/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 b/src/test/hive/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 b/src/test/hive/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/src/test/hive/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d b/src/test/hive/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d new file mode 100644 index 0000000000000..b6e27607fb529 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d @@ -0,0 +1 @@ +242 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 b/src/test/hive/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 new file mode 100644 index 0000000000000..ce83bd94b3310 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 @@ -0,0 +1 @@ +258 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a b/src/test/hive/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a b/src/test/hive/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 b/src/test/hive/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e b/src/test/hive/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 b/src/test/hive/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 b/src/test/hive/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 b/src/test/hive/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 b/src/test/hive/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 b/src/test/hive/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 b/src/test/hive/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff b/src/test/hive/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/src/test/hive/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/src/test/hive/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b b/src/test/hive/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 b/src/test/hive/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf b/src/test/hive/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b b/src/test/hive/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd b/src/test/hive/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a b/src/test/hive/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b b/src/test/hive/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 b/src/test/hive/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a b/src/test/hive/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b b/src/test/hive/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab b/src/test/hive/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 b/src/test/hive/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b b/src/test/hive/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 b/src/test/hive/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 b/src/test/hive/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c b/src/test/hive/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 b/src/test/hive/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/src/test/hive/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc b/src/test/hive/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc b/src/test/hive/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a b/src/test/hive/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 b/src/test/hive/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 b/src/test/hive/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 b/src/test/hive/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe b/src/test/hive/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b b/src/test/hive/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 b/src/test/hive/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e b/src/test/hive/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad b/src/test/hive/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc b/src/test/hive/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc b/src/test/hive/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a b/src/test/hive/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 b/src/test/hive/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 b/src/test/hive/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 b/src/test/hive/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b b/src/test/hive/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb b/src/test/hive/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad b/src/test/hive/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 b/src/test/hive/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de b/src/test/hive/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 b/src/test/hive/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 b/src/test/hive/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed b/src/test/hive/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef b/src/test/hive/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f b/src/test/hive/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb b/src/test/hive/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 b/src/test/hive/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d b/src/test/hive/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 b/src/test/hive/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 b/src/test/hive/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 b/src/test/hive/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 b/src/test/hive/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 b/src/test/hive/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff b/src/test/hive/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 b/src/test/hive/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 b/src/test/hive/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a b/src/test/hive/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec b/src/test/hive/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f b/src/test/hive/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a b/src/test/hive/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec b/src/test/hive/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 b/src/test/hive/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 new file mode 100644 index 0000000000000..2c05a72679b22 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 @@ -0,0 +1,6 @@ +50 val_20 50 val_20 +50 val_23 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 b/src/test/hive/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 b/src/test/hive/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 new file mode 100644 index 0000000000000..0d9bce83ea87c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 @@ -0,0 +1,7 @@ +50 val_20 50 val_20 +50 val_20 50 val_23 +50 val_23 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f b/src/test/hive/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e b/src/test/hive/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e new file mode 100644 index 0000000000000..d4c1adc92802d --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e @@ -0,0 +1,10 @@ +NULL NULL 52 val_4 +NULL NULL 49 val_10 +NULL NULL 49 val_17 +NULL NULL 49 val_19 +50 val_20 50 val_20 +50 val_23 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 b/src/test/hive/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a b/src/test/hive/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a new file mode 100644 index 0000000000000..b62eec8583c92 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a @@ -0,0 +1,11 @@ +NULL NULL 49 val_10 +NULL NULL 49 val_17 +NULL NULL 49 val_19 +50 val_20 50 val_20 +50 val_20 50 val_23 +50 val_23 50 val_20 +50 val_23 50 val_23 +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 diff --git a/src/test/hive/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe b/src/test/hive/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 b/src/test/hive/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 new file mode 100644 index 0000000000000..2c05a72679b22 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 @@ -0,0 +1,6 @@ +50 val_20 50 val_20 +50 val_23 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 b/src/test/hive/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f b/src/test/hive/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd b/src/test/hive/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd new file mode 100644 index 0000000000000..0d9bce83ea87c --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd @@ -0,0 +1,7 @@ +50 val_20 50 val_20 +50 val_20 50 val_23 +50 val_23 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 b/src/test/hive/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 b/src/test/hive/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 new file mode 100644 index 0000000000000..d4c1adc92802d --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 @@ -0,0 +1,10 @@ +NULL NULL 52 val_4 +NULL NULL 49 val_10 +NULL NULL 49 val_17 +NULL NULL 49 val_19 +50 val_20 50 val_20 +50 val_23 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 b/src/test/hive/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 b/src/test/hive/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 new file mode 100644 index 0000000000000..b62eec8583c92 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 @@ -0,0 +1,11 @@ +NULL NULL 49 val_10 +NULL NULL 49 val_17 +NULL NULL 49 val_19 +50 val_20 50 val_20 +50 val_20 50 val_23 +50 val_23 50 val_20 +50 val_23 50 val_23 +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 diff --git a/src/test/hive/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 b/src/test/hive/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de b/src/test/hive/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 b/src/test/hive/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af b/src/test/hive/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a b/src/test/hive/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec b/src/test/hive/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 b/src/test/hive/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 b/src/test/hive/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 b/src/test/hive/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 b/src/test/hive/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d b/src/test/hive/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d new file mode 100644 index 0000000000000..46af2ffd2b620 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d @@ -0,0 +1,5 @@ +49 val_1 NULL NULL NULL NULL +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 diff --git a/src/test/hive/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 b/src/test/hive/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 b/src/test/hive/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 new file mode 100644 index 0000000000000..9047baeb236e5 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 @@ -0,0 +1,6 @@ +NULL NULL NULL NULL 52 val_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 diff --git a/src/test/hive/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 b/src/test/hive/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 b/src/test/hive/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 new file mode 100644 index 0000000000000..47626758b4182 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 @@ -0,0 +1,11 @@ +49 val_1 NULL NULL NULL NULL +NULL NULL NULL NULL 49 val_10 +NULL NULL NULL NULL 49 val_17 +NULL NULL NULL NULL 49 val_19 +49 val_10 NULL NULL NULL NULL +NULL NULL NULL NULL 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 b/src/test/hive/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f b/src/test/hive/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 b/src/test/hive/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 new file mode 100644 index 0000000000000..29616d6368661 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 @@ -0,0 +1,6 @@ +NULL NULL 50 val_20 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 b/src/test/hive/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 b/src/test/hive/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 new file mode 100644 index 0000000000000..6f90cbf839656 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 @@ -0,0 +1,7 @@ +NULL NULL 50 val_20 50 val_20 +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 +51 val_3 51 val_30 NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 b/src/test/hive/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 b/src/test/hive/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 new file mode 100644 index 0000000000000..3b195f9dff935 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 @@ -0,0 +1,10 @@ +NULL NULL NULL NULL 52 val_4 +NULL NULL NULL NULL 49 val_10 +NULL NULL NULL NULL 49 val_17 +NULL NULL NULL NULL 49 val_19 +NULL NULL 50 val_20 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 b/src/test/hive/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e b/src/test/hive/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e new file mode 100644 index 0000000000000..e18ae75d14a37 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e @@ -0,0 +1,11 @@ +NULL NULL NULL NULL 49 val_10 +NULL NULL NULL NULL 49 val_17 +NULL NULL NULL NULL 49 val_19 +NULL NULL 50 val_20 50 val_20 +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 +51 val_3 51 val_30 NULL NULL +NULL NULL NULL NULL 52 val_4 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef b/src/test/hive/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 b/src/test/hive/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 new file mode 100644 index 0000000000000..8e51f0864314a --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 @@ -0,0 +1,6 @@ +NULL NULL 50 val_20 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb b/src/test/hive/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 b/src/test/hive/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f b/src/test/hive/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f new file mode 100644 index 0000000000000..51676ca2c94e4 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f @@ -0,0 +1,11 @@ +49 val_1 NULL NULL NULL NULL +49 val_10 NULL NULL NULL NULL +NULL NULL 50 val_20 50 val_20 +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 +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 diff --git a/src/test/hive/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 b/src/test/hive/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 b/src/test/hive/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 new file mode 100644 index 0000000000000..4a125291de271 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 @@ -0,0 +1,10 @@ +NULL NULL NULL NULL 49 val_10 +NULL NULL NULL NULL 49 val_17 +NULL NULL NULL NULL 49 val_19 +NULL NULL 50 val_20 50 val_20 +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 +NULL NULL NULL NULL 52 val_4 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 b/src/test/hive/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 b/src/test/hive/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 new file mode 100644 index 0000000000000..660e9044e889a --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 @@ -0,0 +1,15 @@ +49 val_1 NULL NULL NULL NULL +NULL NULL NULL NULL 49 val_10 +NULL NULL NULL NULL 49 val_17 +NULL NULL NULL NULL 49 val_19 +49 val_10 NULL NULL NULL NULL +NULL NULL 50 val_20 50 val_20 +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 +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 diff --git a/src/test/hive/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de b/src/test/hive/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 b/src/test/hive/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db b/src/test/hive/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a b/src/test/hive/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec b/src/test/hive/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc b/src/test/hive/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 b/src/test/hive/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 b/src/test/hive/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a b/src/test/hive/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 b/src/test/hive/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 new file mode 100644 index 0000000000000..46af2ffd2b620 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 @@ -0,0 +1,5 @@ +49 val_1 NULL NULL NULL NULL +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 diff --git a/src/test/hive/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 b/src/test/hive/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 b/src/test/hive/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 new file mode 100644 index 0000000000000..9047baeb236e5 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 @@ -0,0 +1,6 @@ +NULL NULL NULL NULL 52 val_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 diff --git a/src/test/hive/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f b/src/test/hive/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc b/src/test/hive/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc new file mode 100644 index 0000000000000..47626758b4182 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc @@ -0,0 +1,11 @@ +49 val_1 NULL NULL NULL NULL +NULL NULL NULL NULL 49 val_10 +NULL NULL NULL NULL 49 val_17 +NULL NULL NULL NULL 49 val_19 +49 val_10 NULL NULL NULL NULL +NULL NULL NULL NULL 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a b/src/test/hive/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f b/src/test/hive/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d b/src/test/hive/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d new file mode 100644 index 0000000000000..29616d6368661 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d @@ -0,0 +1,6 @@ +NULL NULL 50 val_20 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 b/src/test/hive/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a b/src/test/hive/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a new file mode 100644 index 0000000000000..6f90cbf839656 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a @@ -0,0 +1,7 @@ +NULL NULL 50 val_20 50 val_20 +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 +51 val_3 51 val_30 NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a b/src/test/hive/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b b/src/test/hive/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b new file mode 100644 index 0000000000000..3b195f9dff935 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b @@ -0,0 +1,10 @@ +NULL NULL NULL NULL 52 val_4 +NULL NULL NULL NULL 49 val_10 +NULL NULL NULL NULL 49 val_17 +NULL NULL NULL NULL 49 val_19 +NULL NULL 50 val_20 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 b/src/test/hive/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 b/src/test/hive/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 new file mode 100644 index 0000000000000..e18ae75d14a37 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 @@ -0,0 +1,11 @@ +NULL NULL NULL NULL 49 val_10 +NULL NULL NULL NULL 49 val_17 +NULL NULL NULL NULL 49 val_19 +NULL NULL 50 val_20 50 val_20 +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 +51 val_3 51 val_30 NULL NULL +NULL NULL NULL NULL 52 val_4 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 b/src/test/hive/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 b/src/test/hive/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 new file mode 100644 index 0000000000000..8e51f0864314a --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 @@ -0,0 +1,6 @@ +NULL NULL 50 val_20 50 val_20 +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 diff --git a/src/test/hive/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 b/src/test/hive/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 b/src/test/hive/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 b/src/test/hive/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 new file mode 100644 index 0000000000000..51676ca2c94e4 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 @@ -0,0 +1,11 @@ +49 val_1 NULL NULL NULL NULL +49 val_10 NULL NULL NULL NULL +NULL NULL 50 val_20 50 val_20 +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 +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 diff --git a/src/test/hive/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 b/src/test/hive/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc b/src/test/hive/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc new file mode 100644 index 0000000000000..4a125291de271 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc @@ -0,0 +1,10 @@ +NULL NULL NULL NULL 49 val_10 +NULL NULL NULL NULL 49 val_17 +NULL NULL NULL NULL 49 val_19 +NULL NULL 50 val_20 50 val_20 +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 +NULL NULL NULL NULL 52 val_4 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 b/src/test/hive/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 b/src/test/hive/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 new file mode 100644 index 0000000000000..660e9044e889a --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 @@ -0,0 +1,15 @@ +49 val_1 NULL NULL NULL NULL +NULL NULL NULL NULL 49 val_10 +NULL NULL NULL NULL 49 val_17 +NULL NULL NULL NULL 49 val_19 +49 val_10 NULL NULL NULL NULL +NULL NULL 50 val_20 50 val_20 +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 +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 diff --git a/src/test/hive/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de b/src/test/hive/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 b/src/test/hive/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df b/src/test/hive/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 b/src/test/hive/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 b/src/test/hive/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca b/src/test/hive/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca new file mode 100644 index 0000000000000..c2e3ea8b0c8e2 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca @@ -0,0 +1,1028 @@ +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 2 val_2 +4 val_4 4 val_4 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +8 val_8 8 val_8 +9 val_9 9 val_9 +10 val_10 10 val_10 +11 val_11 11 val_11 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +17 val_17 17 val_17 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +19 val_19 19 val_19 +20 val_20 20 val_20 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +27 val_27 27 val_27 +28 val_28 28 val_28 +30 val_30 30 val_30 +33 val_33 33 val_33 +34 val_34 34 val_34 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +41 val_41 41 val_41 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +43 val_43 43 val_43 +44 val_44 44 val_44 +47 val_47 47 val_47 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +53 val_53 53 val_53 +54 val_54 54 val_54 +57 val_57 57 val_57 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +64 val_64 64 val_64 +65 val_65 65 val_65 +66 val_66 66 val_66 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +69 val_69 69 val_69 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +74 val_74 74 val_74 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +77 val_77 77 val_77 +78 val_78 78 val_78 +80 val_80 80 val_80 +82 val_82 82 val_82 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +85 val_85 85 val_85 +86 val_86 86 val_86 +87 val_87 87 val_87 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +92 val_92 92 val_92 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +96 val_96 96 val_96 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +98 val_98 98 val_98 +98 val_98 98 val_98 +98 val_98 98 val_98 +98 val_98 98 val_98 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +105 val_105 105 val_105 +111 val_111 111 val_111 +113 val_113 113 val_113 +113 val_113 113 val_113 +113 val_113 113 val_113 +113 val_113 113 val_113 +114 val_114 114 val_114 +116 val_116 116 val_116 +118 val_118 118 val_118 +118 val_118 118 val_118 +118 val_118 118 val_118 +118 val_118 118 val_118 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +120 val_120 120 val_120 +120 val_120 120 val_120 +120 val_120 120 val_120 +120 val_120 120 val_120 +125 val_125 125 val_125 +125 val_125 125 val_125 +125 val_125 125 val_125 +125 val_125 125 val_125 +126 val_126 126 val_126 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +129 val_129 129 val_129 +129 val_129 129 val_129 +129 val_129 129 val_129 +129 val_129 129 val_129 +131 val_131 131 val_131 +133 val_133 133 val_133 +134 val_134 134 val_134 +134 val_134 134 val_134 +134 val_134 134 val_134 +134 val_134 134 val_134 +136 val_136 136 val_136 +137 val_137 137 val_137 +137 val_137 137 val_137 +137 val_137 137 val_137 +137 val_137 137 val_137 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +143 val_143 143 val_143 +145 val_145 145 val_145 +146 val_146 146 val_146 +146 val_146 146 val_146 +146 val_146 146 val_146 +146 val_146 146 val_146 +149 val_149 149 val_149 +149 val_149 149 val_149 +149 val_149 149 val_149 +149 val_149 149 val_149 +150 val_150 150 val_150 +152 val_152 152 val_152 +152 val_152 152 val_152 +152 val_152 152 val_152 +152 val_152 152 val_152 +153 val_153 153 val_153 +155 val_155 155 val_155 +156 val_156 156 val_156 +157 val_157 157 val_157 +158 val_158 158 val_158 +160 val_160 160 val_160 +162 val_162 162 val_162 +163 val_163 163 val_163 +164 val_164 164 val_164 +164 val_164 164 val_164 +164 val_164 164 val_164 +164 val_164 164 val_164 +165 val_165 165 val_165 +165 val_165 165 val_165 +165 val_165 165 val_165 +165 val_165 165 val_165 +166 val_166 166 val_166 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +168 val_168 168 val_168 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +170 val_170 170 val_170 +172 val_172 172 val_172 +172 val_172 172 val_172 +172 val_172 172 val_172 +172 val_172 172 val_172 +174 val_174 174 val_174 +174 val_174 174 val_174 +174 val_174 174 val_174 +174 val_174 174 val_174 +175 val_175 175 val_175 +175 val_175 175 val_175 +175 val_175 175 val_175 +175 val_175 175 val_175 +176 val_176 176 val_176 +176 val_176 176 val_176 +176 val_176 176 val_176 +176 val_176 176 val_176 +177 val_177 177 val_177 +178 val_178 178 val_178 +179 val_179 179 val_179 +179 val_179 179 val_179 +179 val_179 179 val_179 +179 val_179 179 val_179 +180 val_180 180 val_180 +181 val_181 181 val_181 +183 val_183 183 val_183 +186 val_186 186 val_186 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +189 val_189 189 val_189 +190 val_190 190 val_190 +191 val_191 191 val_191 +191 val_191 191 val_191 +191 val_191 191 val_191 +191 val_191 191 val_191 +192 val_192 192 val_192 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +194 val_194 194 val_194 +195 val_195 195 val_195 +195 val_195 195 val_195 +195 val_195 195 val_195 +195 val_195 195 val_195 +196 val_196 196 val_196 +197 val_197 197 val_197 +197 val_197 197 val_197 +197 val_197 197 val_197 +197 val_197 197 val_197 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +200 val_200 200 val_200 +200 val_200 200 val_200 +200 val_200 200 val_200 +200 val_200 200 val_200 +201 val_201 201 val_201 +202 val_202 202 val_202 +203 val_203 203 val_203 +203 val_203 203 val_203 +203 val_203 203 val_203 +203 val_203 203 val_203 +205 val_205 205 val_205 +205 val_205 205 val_205 +205 val_205 205 val_205 +205 val_205 205 val_205 +207 val_207 207 val_207 +207 val_207 207 val_207 +207 val_207 207 val_207 +207 val_207 207 val_207 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +209 val_209 209 val_209 +209 val_209 209 val_209 +209 val_209 209 val_209 +209 val_209 209 val_209 +213 val_213 213 val_213 +213 val_213 213 val_213 +213 val_213 213 val_213 +213 val_213 213 val_213 +214 val_214 214 val_214 +216 val_216 216 val_216 +216 val_216 216 val_216 +216 val_216 216 val_216 +216 val_216 216 val_216 +217 val_217 217 val_217 +217 val_217 217 val_217 +217 val_217 217 val_217 +217 val_217 217 val_217 +218 val_218 218 val_218 +219 val_219 219 val_219 +219 val_219 219 val_219 +219 val_219 219 val_219 +219 val_219 219 val_219 +221 val_221 221 val_221 +221 val_221 221 val_221 +221 val_221 221 val_221 +221 val_221 221 val_221 +222 val_222 222 val_222 +223 val_223 223 val_223 +223 val_223 223 val_223 +223 val_223 223 val_223 +223 val_223 223 val_223 +224 val_224 224 val_224 +224 val_224 224 val_224 +224 val_224 224 val_224 +224 val_224 224 val_224 +226 val_226 226 val_226 +228 val_228 228 val_228 +229 val_229 229 val_229 +229 val_229 229 val_229 +229 val_229 229 val_229 +229 val_229 229 val_229 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +233 val_233 233 val_233 +233 val_233 233 val_233 +233 val_233 233 val_233 +233 val_233 233 val_233 +235 val_235 235 val_235 +237 val_237 237 val_237 +237 val_237 237 val_237 +237 val_237 237 val_237 +237 val_237 237 val_237 +238 val_238 238 val_238 +238 val_238 238 val_238 +238 val_238 238 val_238 +238 val_238 238 val_238 +239 val_239 239 val_239 +239 val_239 239 val_239 +239 val_239 239 val_239 +239 val_239 239 val_239 +241 val_241 241 val_241 +242 val_242 242 val_242 +242 val_242 242 val_242 +242 val_242 242 val_242 +242 val_242 242 val_242 +244 val_244 244 val_244 +247 val_247 247 val_247 +248 val_248 248 val_248 +249 val_249 249 val_249 +252 val_252 252 val_252 +255 val_255 255 val_255 +255 val_255 255 val_255 +255 val_255 255 val_255 +255 val_255 255 val_255 +256 val_256 256 val_256 +256 val_256 256 val_256 +256 val_256 256 val_256 +256 val_256 256 val_256 +257 val_257 257 val_257 +258 val_258 258 val_258 +260 val_260 260 val_260 +262 val_262 262 val_262 +263 val_263 263 val_263 +265 val_265 265 val_265 +265 val_265 265 val_265 +265 val_265 265 val_265 +265 val_265 265 val_265 +266 val_266 266 val_266 +272 val_272 272 val_272 +272 val_272 272 val_272 +272 val_272 272 val_272 +272 val_272 272 val_272 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +274 val_274 274 val_274 +275 val_275 275 val_275 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +278 val_278 278 val_278 +278 val_278 278 val_278 +278 val_278 278 val_278 +278 val_278 278 val_278 +280 val_280 280 val_280 +280 val_280 280 val_280 +280 val_280 280 val_280 +280 val_280 280 val_280 +281 val_281 281 val_281 +281 val_281 281 val_281 +281 val_281 281 val_281 +281 val_281 281 val_281 +282 val_282 282 val_282 +282 val_282 282 val_282 +282 val_282 282 val_282 +282 val_282 282 val_282 +283 val_283 283 val_283 +284 val_284 284 val_284 +285 val_285 285 val_285 +286 val_286 286 val_286 +287 val_287 287 val_287 +288 val_288 288 val_288 +288 val_288 288 val_288 +288 val_288 288 val_288 +288 val_288 288 val_288 +289 val_289 289 val_289 +291 val_291 291 val_291 +292 val_292 292 val_292 +296 val_296 296 val_296 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +302 val_302 302 val_302 +305 val_305 305 val_305 +306 val_306 306 val_306 +307 val_307 307 val_307 +307 val_307 307 val_307 +307 val_307 307 val_307 +307 val_307 307 val_307 +308 val_308 308 val_308 +309 val_309 309 val_309 +309 val_309 309 val_309 +309 val_309 309 val_309 +309 val_309 309 val_309 +310 val_310 310 val_310 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +315 val_315 315 val_315 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +317 val_317 317 val_317 +317 val_317 317 val_317 +317 val_317 317 val_317 +317 val_317 317 val_317 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +321 val_321 321 val_321 +321 val_321 321 val_321 +321 val_321 321 val_321 +321 val_321 321 val_321 +322 val_322 322 val_322 +322 val_322 322 val_322 +322 val_322 322 val_322 +322 val_322 322 val_322 +323 val_323 323 val_323 +325 val_325 325 val_325 +325 val_325 325 val_325 +325 val_325 325 val_325 +325 val_325 325 val_325 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +331 val_331 331 val_331 +331 val_331 331 val_331 +331 val_331 331 val_331 +331 val_331 331 val_331 +332 val_332 332 val_332 +333 val_333 333 val_333 +333 val_333 333 val_333 +333 val_333 333 val_333 +333 val_333 333 val_333 +335 val_335 335 val_335 +336 val_336 336 val_336 +338 val_338 338 val_338 +339 val_339 339 val_339 +341 val_341 341 val_341 +342 val_342 342 val_342 +342 val_342 342 val_342 +342 val_342 342 val_342 +342 val_342 342 val_342 +344 val_344 344 val_344 +344 val_344 344 val_344 +344 val_344 344 val_344 +344 val_344 344 val_344 +345 val_345 345 val_345 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +351 val_351 351 val_351 +353 val_353 353 val_353 +353 val_353 353 val_353 +353 val_353 353 val_353 +353 val_353 353 val_353 +356 val_356 356 val_356 +360 val_360 360 val_360 +362 val_362 362 val_362 +364 val_364 364 val_364 +365 val_365 365 val_365 +366 val_366 366 val_366 +367 val_367 367 val_367 +367 val_367 367 val_367 +367 val_367 367 val_367 +367 val_367 367 val_367 +368 val_368 368 val_368 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +373 val_373 373 val_373 +374 val_374 374 val_374 +375 val_375 375 val_375 +377 val_377 377 val_377 +378 val_378 378 val_378 +379 val_379 379 val_379 +382 val_382 382 val_382 +382 val_382 382 val_382 +382 val_382 382 val_382 +382 val_382 382 val_382 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +386 val_386 386 val_386 +389 val_389 389 val_389 +392 val_392 392 val_392 +393 val_393 393 val_393 +394 val_394 394 val_394 +395 val_395 395 val_395 +395 val_395 395 val_395 +395 val_395 395 val_395 +395 val_395 395 val_395 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +397 val_397 397 val_397 +397 val_397 397 val_397 +397 val_397 397 val_397 +397 val_397 397 val_397 +399 val_399 399 val_399 +399 val_399 399 val_399 +399 val_399 399 val_399 +399 val_399 399 val_399 +400 val_400 400 val_400 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +402 val_402 402 val_402 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +404 val_404 404 val_404 +404 val_404 404 val_404 +404 val_404 404 val_404 +404 val_404 404 val_404 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +407 val_407 407 val_407 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +411 val_411 411 val_411 +413 val_413 413 val_413 +413 val_413 413 val_413 +413 val_413 413 val_413 +413 val_413 413 val_413 +414 val_414 414 val_414 +414 val_414 414 val_414 +414 val_414 414 val_414 +414 val_414 414 val_414 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +418 val_418 418 val_418 +419 val_419 419 val_419 +421 val_421 421 val_421 +424 val_424 424 val_424 +424 val_424 424 val_424 +424 val_424 424 val_424 +424 val_424 424 val_424 +427 val_427 427 val_427 +429 val_429 429 val_429 +429 val_429 429 val_429 +429 val_429 429 val_429 +429 val_429 429 val_429 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +432 val_432 432 val_432 +435 val_435 435 val_435 +436 val_436 436 val_436 +437 val_437 437 val_437 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +439 val_439 439 val_439 +439 val_439 439 val_439 +439 val_439 439 val_439 +439 val_439 439 val_439 +443 val_443 443 val_443 +444 val_444 444 val_444 +446 val_446 446 val_446 +448 val_448 448 val_448 +449 val_449 449 val_449 +452 val_452 452 val_452 +453 val_453 453 val_453 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +455 val_455 455 val_455 +457 val_457 457 val_457 +458 val_458 458 val_458 +458 val_458 458 val_458 +458 val_458 458 val_458 +458 val_458 458 val_458 +459 val_459 459 val_459 +459 val_459 459 val_459 +459 val_459 459 val_459 +459 val_459 459 val_459 +460 val_460 460 val_460 +462 val_462 462 val_462 +462 val_462 462 val_462 +462 val_462 462 val_462 +462 val_462 462 val_462 +463 val_463 463 val_463 +463 val_463 463 val_463 +463 val_463 463 val_463 +463 val_463 463 val_463 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +467 val_467 467 val_467 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +470 val_470 470 val_470 +472 val_472 472 val_472 +475 val_475 475 val_475 +477 val_477 477 val_477 +478 val_478 478 val_478 +478 val_478 478 val_478 +478 val_478 478 val_478 +478 val_478 478 val_478 +479 val_479 479 val_479 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +481 val_481 481 val_481 +482 val_482 482 val_482 +483 val_483 483 val_483 +484 val_484 484 val_484 +485 val_485 485 val_485 +487 val_487 487 val_487 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +490 val_490 490 val_490 +491 val_491 491 val_491 +492 val_492 492 val_492 +492 val_492 492 val_492 +492 val_492 492 val_492 +492 val_492 492 val_492 +493 val_493 493 val_493 +494 val_494 494 val_494 +495 val_495 495 val_495 +496 val_496 496 val_496 +497 val_497 497 val_497 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +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 diff --git a/src/test/hive/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 b/src/test/hive/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e b/src/test/hive/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e new file mode 100644 index 0000000000000..0e17d179f4167 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e @@ -0,0 +1 @@ +278697 278697 101852390308 101852390308 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a b/src/test/hive/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a new file mode 100644 index 0000000000000..0e17d179f4167 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a @@ -0,0 +1 @@ +278697 278697 101852390308 101852390308 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a b/src/test/hive/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 b/src/test/hive/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 b/src/test/hive/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca b/src/test/hive/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca new file mode 100644 index 0000000000000..c2e3ea8b0c8e2 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca @@ -0,0 +1,1028 @@ +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 2 val_2 +4 val_4 4 val_4 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +8 val_8 8 val_8 +9 val_9 9 val_9 +10 val_10 10 val_10 +11 val_11 11 val_11 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +17 val_17 17 val_17 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +19 val_19 19 val_19 +20 val_20 20 val_20 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +27 val_27 27 val_27 +28 val_28 28 val_28 +30 val_30 30 val_30 +33 val_33 33 val_33 +34 val_34 34 val_34 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +41 val_41 41 val_41 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +43 val_43 43 val_43 +44 val_44 44 val_44 +47 val_47 47 val_47 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +53 val_53 53 val_53 +54 val_54 54 val_54 +57 val_57 57 val_57 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +64 val_64 64 val_64 +65 val_65 65 val_65 +66 val_66 66 val_66 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +69 val_69 69 val_69 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +74 val_74 74 val_74 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +77 val_77 77 val_77 +78 val_78 78 val_78 +80 val_80 80 val_80 +82 val_82 82 val_82 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +85 val_85 85 val_85 +86 val_86 86 val_86 +87 val_87 87 val_87 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +92 val_92 92 val_92 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +96 val_96 96 val_96 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +98 val_98 98 val_98 +98 val_98 98 val_98 +98 val_98 98 val_98 +98 val_98 98 val_98 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +105 val_105 105 val_105 +111 val_111 111 val_111 +113 val_113 113 val_113 +113 val_113 113 val_113 +113 val_113 113 val_113 +113 val_113 113 val_113 +114 val_114 114 val_114 +116 val_116 116 val_116 +118 val_118 118 val_118 +118 val_118 118 val_118 +118 val_118 118 val_118 +118 val_118 118 val_118 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +120 val_120 120 val_120 +120 val_120 120 val_120 +120 val_120 120 val_120 +120 val_120 120 val_120 +125 val_125 125 val_125 +125 val_125 125 val_125 +125 val_125 125 val_125 +125 val_125 125 val_125 +126 val_126 126 val_126 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +129 val_129 129 val_129 +129 val_129 129 val_129 +129 val_129 129 val_129 +129 val_129 129 val_129 +131 val_131 131 val_131 +133 val_133 133 val_133 +134 val_134 134 val_134 +134 val_134 134 val_134 +134 val_134 134 val_134 +134 val_134 134 val_134 +136 val_136 136 val_136 +137 val_137 137 val_137 +137 val_137 137 val_137 +137 val_137 137 val_137 +137 val_137 137 val_137 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +143 val_143 143 val_143 +145 val_145 145 val_145 +146 val_146 146 val_146 +146 val_146 146 val_146 +146 val_146 146 val_146 +146 val_146 146 val_146 +149 val_149 149 val_149 +149 val_149 149 val_149 +149 val_149 149 val_149 +149 val_149 149 val_149 +150 val_150 150 val_150 +152 val_152 152 val_152 +152 val_152 152 val_152 +152 val_152 152 val_152 +152 val_152 152 val_152 +153 val_153 153 val_153 +155 val_155 155 val_155 +156 val_156 156 val_156 +157 val_157 157 val_157 +158 val_158 158 val_158 +160 val_160 160 val_160 +162 val_162 162 val_162 +163 val_163 163 val_163 +164 val_164 164 val_164 +164 val_164 164 val_164 +164 val_164 164 val_164 +164 val_164 164 val_164 +165 val_165 165 val_165 +165 val_165 165 val_165 +165 val_165 165 val_165 +165 val_165 165 val_165 +166 val_166 166 val_166 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +168 val_168 168 val_168 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +170 val_170 170 val_170 +172 val_172 172 val_172 +172 val_172 172 val_172 +172 val_172 172 val_172 +172 val_172 172 val_172 +174 val_174 174 val_174 +174 val_174 174 val_174 +174 val_174 174 val_174 +174 val_174 174 val_174 +175 val_175 175 val_175 +175 val_175 175 val_175 +175 val_175 175 val_175 +175 val_175 175 val_175 +176 val_176 176 val_176 +176 val_176 176 val_176 +176 val_176 176 val_176 +176 val_176 176 val_176 +177 val_177 177 val_177 +178 val_178 178 val_178 +179 val_179 179 val_179 +179 val_179 179 val_179 +179 val_179 179 val_179 +179 val_179 179 val_179 +180 val_180 180 val_180 +181 val_181 181 val_181 +183 val_183 183 val_183 +186 val_186 186 val_186 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +189 val_189 189 val_189 +190 val_190 190 val_190 +191 val_191 191 val_191 +191 val_191 191 val_191 +191 val_191 191 val_191 +191 val_191 191 val_191 +192 val_192 192 val_192 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +194 val_194 194 val_194 +195 val_195 195 val_195 +195 val_195 195 val_195 +195 val_195 195 val_195 +195 val_195 195 val_195 +196 val_196 196 val_196 +197 val_197 197 val_197 +197 val_197 197 val_197 +197 val_197 197 val_197 +197 val_197 197 val_197 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +200 val_200 200 val_200 +200 val_200 200 val_200 +200 val_200 200 val_200 +200 val_200 200 val_200 +201 val_201 201 val_201 +202 val_202 202 val_202 +203 val_203 203 val_203 +203 val_203 203 val_203 +203 val_203 203 val_203 +203 val_203 203 val_203 +205 val_205 205 val_205 +205 val_205 205 val_205 +205 val_205 205 val_205 +205 val_205 205 val_205 +207 val_207 207 val_207 +207 val_207 207 val_207 +207 val_207 207 val_207 +207 val_207 207 val_207 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +209 val_209 209 val_209 +209 val_209 209 val_209 +209 val_209 209 val_209 +209 val_209 209 val_209 +213 val_213 213 val_213 +213 val_213 213 val_213 +213 val_213 213 val_213 +213 val_213 213 val_213 +214 val_214 214 val_214 +216 val_216 216 val_216 +216 val_216 216 val_216 +216 val_216 216 val_216 +216 val_216 216 val_216 +217 val_217 217 val_217 +217 val_217 217 val_217 +217 val_217 217 val_217 +217 val_217 217 val_217 +218 val_218 218 val_218 +219 val_219 219 val_219 +219 val_219 219 val_219 +219 val_219 219 val_219 +219 val_219 219 val_219 +221 val_221 221 val_221 +221 val_221 221 val_221 +221 val_221 221 val_221 +221 val_221 221 val_221 +222 val_222 222 val_222 +223 val_223 223 val_223 +223 val_223 223 val_223 +223 val_223 223 val_223 +223 val_223 223 val_223 +224 val_224 224 val_224 +224 val_224 224 val_224 +224 val_224 224 val_224 +224 val_224 224 val_224 +226 val_226 226 val_226 +228 val_228 228 val_228 +229 val_229 229 val_229 +229 val_229 229 val_229 +229 val_229 229 val_229 +229 val_229 229 val_229 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +233 val_233 233 val_233 +233 val_233 233 val_233 +233 val_233 233 val_233 +233 val_233 233 val_233 +235 val_235 235 val_235 +237 val_237 237 val_237 +237 val_237 237 val_237 +237 val_237 237 val_237 +237 val_237 237 val_237 +238 val_238 238 val_238 +238 val_238 238 val_238 +238 val_238 238 val_238 +238 val_238 238 val_238 +239 val_239 239 val_239 +239 val_239 239 val_239 +239 val_239 239 val_239 +239 val_239 239 val_239 +241 val_241 241 val_241 +242 val_242 242 val_242 +242 val_242 242 val_242 +242 val_242 242 val_242 +242 val_242 242 val_242 +244 val_244 244 val_244 +247 val_247 247 val_247 +248 val_248 248 val_248 +249 val_249 249 val_249 +252 val_252 252 val_252 +255 val_255 255 val_255 +255 val_255 255 val_255 +255 val_255 255 val_255 +255 val_255 255 val_255 +256 val_256 256 val_256 +256 val_256 256 val_256 +256 val_256 256 val_256 +256 val_256 256 val_256 +257 val_257 257 val_257 +258 val_258 258 val_258 +260 val_260 260 val_260 +262 val_262 262 val_262 +263 val_263 263 val_263 +265 val_265 265 val_265 +265 val_265 265 val_265 +265 val_265 265 val_265 +265 val_265 265 val_265 +266 val_266 266 val_266 +272 val_272 272 val_272 +272 val_272 272 val_272 +272 val_272 272 val_272 +272 val_272 272 val_272 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +274 val_274 274 val_274 +275 val_275 275 val_275 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +278 val_278 278 val_278 +278 val_278 278 val_278 +278 val_278 278 val_278 +278 val_278 278 val_278 +280 val_280 280 val_280 +280 val_280 280 val_280 +280 val_280 280 val_280 +280 val_280 280 val_280 +281 val_281 281 val_281 +281 val_281 281 val_281 +281 val_281 281 val_281 +281 val_281 281 val_281 +282 val_282 282 val_282 +282 val_282 282 val_282 +282 val_282 282 val_282 +282 val_282 282 val_282 +283 val_283 283 val_283 +284 val_284 284 val_284 +285 val_285 285 val_285 +286 val_286 286 val_286 +287 val_287 287 val_287 +288 val_288 288 val_288 +288 val_288 288 val_288 +288 val_288 288 val_288 +288 val_288 288 val_288 +289 val_289 289 val_289 +291 val_291 291 val_291 +292 val_292 292 val_292 +296 val_296 296 val_296 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +302 val_302 302 val_302 +305 val_305 305 val_305 +306 val_306 306 val_306 +307 val_307 307 val_307 +307 val_307 307 val_307 +307 val_307 307 val_307 +307 val_307 307 val_307 +308 val_308 308 val_308 +309 val_309 309 val_309 +309 val_309 309 val_309 +309 val_309 309 val_309 +309 val_309 309 val_309 +310 val_310 310 val_310 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +315 val_315 315 val_315 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +317 val_317 317 val_317 +317 val_317 317 val_317 +317 val_317 317 val_317 +317 val_317 317 val_317 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +321 val_321 321 val_321 +321 val_321 321 val_321 +321 val_321 321 val_321 +321 val_321 321 val_321 +322 val_322 322 val_322 +322 val_322 322 val_322 +322 val_322 322 val_322 +322 val_322 322 val_322 +323 val_323 323 val_323 +325 val_325 325 val_325 +325 val_325 325 val_325 +325 val_325 325 val_325 +325 val_325 325 val_325 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +331 val_331 331 val_331 +331 val_331 331 val_331 +331 val_331 331 val_331 +331 val_331 331 val_331 +332 val_332 332 val_332 +333 val_333 333 val_333 +333 val_333 333 val_333 +333 val_333 333 val_333 +333 val_333 333 val_333 +335 val_335 335 val_335 +336 val_336 336 val_336 +338 val_338 338 val_338 +339 val_339 339 val_339 +341 val_341 341 val_341 +342 val_342 342 val_342 +342 val_342 342 val_342 +342 val_342 342 val_342 +342 val_342 342 val_342 +344 val_344 344 val_344 +344 val_344 344 val_344 +344 val_344 344 val_344 +344 val_344 344 val_344 +345 val_345 345 val_345 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +351 val_351 351 val_351 +353 val_353 353 val_353 +353 val_353 353 val_353 +353 val_353 353 val_353 +353 val_353 353 val_353 +356 val_356 356 val_356 +360 val_360 360 val_360 +362 val_362 362 val_362 +364 val_364 364 val_364 +365 val_365 365 val_365 +366 val_366 366 val_366 +367 val_367 367 val_367 +367 val_367 367 val_367 +367 val_367 367 val_367 +367 val_367 367 val_367 +368 val_368 368 val_368 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +373 val_373 373 val_373 +374 val_374 374 val_374 +375 val_375 375 val_375 +377 val_377 377 val_377 +378 val_378 378 val_378 +379 val_379 379 val_379 +382 val_382 382 val_382 +382 val_382 382 val_382 +382 val_382 382 val_382 +382 val_382 382 val_382 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +386 val_386 386 val_386 +389 val_389 389 val_389 +392 val_392 392 val_392 +393 val_393 393 val_393 +394 val_394 394 val_394 +395 val_395 395 val_395 +395 val_395 395 val_395 +395 val_395 395 val_395 +395 val_395 395 val_395 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +397 val_397 397 val_397 +397 val_397 397 val_397 +397 val_397 397 val_397 +397 val_397 397 val_397 +399 val_399 399 val_399 +399 val_399 399 val_399 +399 val_399 399 val_399 +399 val_399 399 val_399 +400 val_400 400 val_400 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +402 val_402 402 val_402 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +404 val_404 404 val_404 +404 val_404 404 val_404 +404 val_404 404 val_404 +404 val_404 404 val_404 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +407 val_407 407 val_407 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +411 val_411 411 val_411 +413 val_413 413 val_413 +413 val_413 413 val_413 +413 val_413 413 val_413 +413 val_413 413 val_413 +414 val_414 414 val_414 +414 val_414 414 val_414 +414 val_414 414 val_414 +414 val_414 414 val_414 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +418 val_418 418 val_418 +419 val_419 419 val_419 +421 val_421 421 val_421 +424 val_424 424 val_424 +424 val_424 424 val_424 +424 val_424 424 val_424 +424 val_424 424 val_424 +427 val_427 427 val_427 +429 val_429 429 val_429 +429 val_429 429 val_429 +429 val_429 429 val_429 +429 val_429 429 val_429 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +432 val_432 432 val_432 +435 val_435 435 val_435 +436 val_436 436 val_436 +437 val_437 437 val_437 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +439 val_439 439 val_439 +439 val_439 439 val_439 +439 val_439 439 val_439 +439 val_439 439 val_439 +443 val_443 443 val_443 +444 val_444 444 val_444 +446 val_446 446 val_446 +448 val_448 448 val_448 +449 val_449 449 val_449 +452 val_452 452 val_452 +453 val_453 453 val_453 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +455 val_455 455 val_455 +457 val_457 457 val_457 +458 val_458 458 val_458 +458 val_458 458 val_458 +458 val_458 458 val_458 +458 val_458 458 val_458 +459 val_459 459 val_459 +459 val_459 459 val_459 +459 val_459 459 val_459 +459 val_459 459 val_459 +460 val_460 460 val_460 +462 val_462 462 val_462 +462 val_462 462 val_462 +462 val_462 462 val_462 +462 val_462 462 val_462 +463 val_463 463 val_463 +463 val_463 463 val_463 +463 val_463 463 val_463 +463 val_463 463 val_463 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +467 val_467 467 val_467 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +470 val_470 470 val_470 +472 val_472 472 val_472 +475 val_475 475 val_475 +477 val_477 477 val_477 +478 val_478 478 val_478 +478 val_478 478 val_478 +478 val_478 478 val_478 +478 val_478 478 val_478 +479 val_479 479 val_479 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +481 val_481 481 val_481 +482 val_482 482 val_482 +483 val_483 483 val_483 +484 val_484 484 val_484 +485 val_485 485 val_485 +487 val_487 487 val_487 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +490 val_490 490 val_490 +491 val_491 491 val_491 +492 val_492 492 val_492 +492 val_492 492 val_492 +492 val_492 492 val_492 +492 val_492 492 val_492 +493 val_493 493 val_493 +494 val_494 494 val_494 +495 val_495 495 val_495 +496 val_496 496 val_496 +497 val_497 497 val_497 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +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 diff --git a/src/test/hive/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 b/src/test/hive/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e b/src/test/hive/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e new file mode 100644 index 0000000000000..0e17d179f4167 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e @@ -0,0 +1 @@ +278697 278697 101852390308 101852390308 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a b/src/test/hive/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a new file mode 100644 index 0000000000000..0e17d179f4167 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a @@ -0,0 +1 @@ +278697 278697 101852390308 101852390308 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 b/src/test/hive/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 b/src/test/hive/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 b/src/test/hive/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 b/src/test/hive/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 b/src/test/hive/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 b/src/test/hive/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 b/src/test/hive/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa b/src/test/hive/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c b/src/test/hive/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 b/src/test/hive/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 b/src/test/hive/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 b/src/test/hive/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 b/src/test/hive/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 b/src/test/hive/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 b/src/test/hive/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 b/src/test/hive/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 new file mode 100644 index 0000000000000..4cd5eefea2a45 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 @@ -0,0 +1,500 @@ +NULL NULL 0 val_0 +NULL NULL 0 val_0 +NULL NULL 0 val_0 +NULL NULL 2 val_2 +NULL NULL 4 val_4 +NULL NULL 5 val_5 +NULL NULL 5 val_5 +NULL NULL 5 val_5 +NULL NULL 8 val_8 +NULL NULL 9 val_9 +NULL NULL 10 val_10 +NULL NULL 11 val_11 +NULL NULL 12 val_12 +NULL NULL 12 val_12 +NULL NULL 15 val_15 +NULL NULL 15 val_15 +NULL NULL 17 val_17 +NULL NULL 18 val_18 +NULL NULL 18 val_18 +NULL NULL 19 val_19 +NULL NULL 20 val_20 +NULL NULL 24 val_24 +NULL NULL 24 val_24 +NULL NULL 26 val_26 +NULL NULL 26 val_26 +NULL NULL 27 val_27 +NULL NULL 28 val_28 +NULL NULL 30 val_30 +NULL NULL 33 val_33 +NULL NULL 34 val_34 +NULL NULL 35 val_35 +NULL NULL 35 val_35 +NULL NULL 35 val_35 +NULL NULL 37 val_37 +NULL NULL 37 val_37 +NULL NULL 41 val_41 +NULL NULL 42 val_42 +NULL NULL 42 val_42 +NULL NULL 43 val_43 +NULL NULL 44 val_44 +NULL NULL 47 val_47 +NULL NULL 51 val_51 +NULL NULL 51 val_51 +NULL NULL 53 val_53 +NULL NULL 54 val_54 +NULL NULL 57 val_57 +NULL NULL 58 val_58 +NULL NULL 58 val_58 +NULL NULL 64 val_64 +NULL NULL 65 val_65 +NULL NULL 66 val_66 +NULL NULL 67 val_67 +NULL NULL 67 val_67 +NULL NULL 69 val_69 +NULL NULL 70 val_70 +NULL NULL 70 val_70 +NULL NULL 70 val_70 +NULL NULL 72 val_72 +NULL NULL 72 val_72 +NULL NULL 74 val_74 +NULL NULL 76 val_76 +NULL NULL 76 val_76 +NULL NULL 77 val_77 +NULL NULL 78 val_78 +NULL NULL 80 val_80 +NULL NULL 82 val_82 +NULL NULL 83 val_83 +NULL NULL 83 val_83 +NULL NULL 84 val_84 +NULL NULL 84 val_84 +NULL NULL 85 val_85 +NULL NULL 86 val_86 +NULL NULL 87 val_87 +NULL NULL 90 val_90 +NULL NULL 90 val_90 +NULL NULL 90 val_90 +NULL NULL 92 val_92 +NULL NULL 95 val_95 +NULL NULL 95 val_95 +NULL NULL 96 val_96 +NULL NULL 97 val_97 +NULL NULL 97 val_97 +NULL NULL 98 val_98 +NULL NULL 98 val_98 +NULL NULL 100 val_100 +NULL NULL 100 val_100 +NULL NULL 103 val_103 +NULL NULL 103 val_103 +NULL NULL 104 val_104 +NULL NULL 104 val_104 +NULL NULL 105 val_105 +NULL NULL 111 val_111 +NULL NULL 113 val_113 +NULL NULL 113 val_113 +NULL NULL 114 val_114 +NULL NULL 116 val_116 +NULL NULL 118 val_118 +NULL NULL 118 val_118 +NULL NULL 119 val_119 +NULL NULL 119 val_119 +NULL NULL 119 val_119 +NULL NULL 120 val_120 +NULL NULL 120 val_120 +NULL NULL 125 val_125 +NULL NULL 125 val_125 +NULL NULL 126 val_126 +NULL NULL 128 val_128 +NULL NULL 128 val_128 +NULL NULL 128 val_128 +NULL NULL 129 val_129 +NULL NULL 129 val_129 +NULL NULL 131 val_131 +NULL NULL 133 val_133 +NULL NULL 134 val_134 +NULL NULL 134 val_134 +NULL NULL 136 val_136 +NULL NULL 137 val_137 +NULL NULL 137 val_137 +NULL NULL 138 val_138 +NULL NULL 138 val_138 +NULL NULL 138 val_138 +NULL NULL 138 val_138 +NULL NULL 143 val_143 +NULL NULL 145 val_145 +NULL NULL 146 val_146 +NULL NULL 146 val_146 +NULL NULL 149 val_149 +NULL NULL 149 val_149 +NULL NULL 150 val_150 +NULL NULL 152 val_152 +NULL NULL 152 val_152 +NULL NULL 153 val_153 +NULL NULL 155 val_155 +NULL NULL 156 val_156 +NULL NULL 157 val_157 +NULL NULL 158 val_158 +NULL NULL 160 val_160 +NULL NULL 162 val_162 +NULL NULL 163 val_163 +NULL NULL 164 val_164 +NULL NULL 164 val_164 +NULL NULL 165 val_165 +NULL NULL 165 val_165 +NULL NULL 166 val_166 +NULL NULL 167 val_167 +NULL NULL 167 val_167 +NULL NULL 167 val_167 +NULL NULL 168 val_168 +NULL NULL 169 val_169 +NULL NULL 169 val_169 +NULL NULL 169 val_169 +NULL NULL 169 val_169 +NULL NULL 170 val_170 +NULL NULL 172 val_172 +NULL NULL 172 val_172 +NULL NULL 174 val_174 +NULL NULL 174 val_174 +NULL NULL 175 val_175 +NULL NULL 175 val_175 +NULL NULL 176 val_176 +NULL NULL 176 val_176 +NULL NULL 177 val_177 +NULL NULL 178 val_178 +NULL NULL 179 val_179 +NULL NULL 179 val_179 +NULL NULL 180 val_180 +NULL NULL 181 val_181 +NULL NULL 183 val_183 +NULL NULL 186 val_186 +NULL NULL 187 val_187 +NULL NULL 187 val_187 +NULL NULL 187 val_187 +NULL NULL 189 val_189 +NULL NULL 190 val_190 +NULL NULL 191 val_191 +NULL NULL 191 val_191 +NULL NULL 192 val_192 +NULL NULL 193 val_193 +NULL NULL 193 val_193 +NULL NULL 193 val_193 +NULL NULL 194 val_194 +NULL NULL 195 val_195 +NULL NULL 195 val_195 +NULL NULL 196 val_196 +NULL NULL 197 val_197 +NULL NULL 197 val_197 +NULL NULL 199 val_199 +NULL NULL 199 val_199 +NULL NULL 199 val_199 +NULL NULL 200 val_200 +NULL NULL 200 val_200 +NULL NULL 201 val_201 +NULL NULL 202 val_202 +NULL NULL 203 val_203 +NULL NULL 203 val_203 +NULL NULL 205 val_205 +NULL NULL 205 val_205 +NULL NULL 207 val_207 +NULL NULL 207 val_207 +NULL NULL 208 val_208 +NULL NULL 208 val_208 +NULL NULL 208 val_208 +NULL NULL 209 val_209 +NULL NULL 209 val_209 +NULL NULL 213 val_213 +NULL NULL 213 val_213 +NULL NULL 214 val_214 +NULL NULL 216 val_216 +NULL NULL 216 val_216 +NULL NULL 217 val_217 +NULL NULL 217 val_217 +NULL NULL 218 val_218 +NULL NULL 219 val_219 +NULL NULL 219 val_219 +NULL NULL 221 val_221 +NULL NULL 221 val_221 +NULL NULL 222 val_222 +NULL NULL 223 val_223 +NULL NULL 223 val_223 +NULL NULL 224 val_224 +NULL NULL 224 val_224 +NULL NULL 226 val_226 +NULL NULL 228 val_228 +NULL NULL 229 val_229 +NULL NULL 229 val_229 +NULL NULL 230 val_230 +NULL NULL 230 val_230 +NULL NULL 230 val_230 +NULL NULL 230 val_230 +NULL NULL 230 val_230 +NULL NULL 233 val_233 +NULL NULL 233 val_233 +NULL NULL 235 val_235 +NULL NULL 237 val_237 +NULL NULL 237 val_237 +NULL NULL 238 val_238 +NULL NULL 238 val_238 +NULL NULL 239 val_239 +NULL NULL 239 val_239 +NULL NULL 241 val_241 +NULL NULL 242 val_242 +NULL NULL 242 val_242 +NULL NULL 244 val_244 +NULL NULL 247 val_247 +NULL NULL 248 val_248 +NULL NULL 249 val_249 +NULL NULL 252 val_252 +NULL NULL 255 val_255 +NULL NULL 255 val_255 +NULL NULL 256 val_256 +NULL NULL 256 val_256 +NULL NULL 257 val_257 +NULL NULL 258 val_258 +NULL NULL 260 val_260 +NULL NULL 262 val_262 +NULL NULL 263 val_263 +NULL NULL 265 val_265 +NULL NULL 265 val_265 +NULL NULL 266 val_266 +NULL NULL 272 val_272 +NULL NULL 272 val_272 +NULL NULL 273 val_273 +NULL NULL 273 val_273 +NULL NULL 273 val_273 +NULL NULL 274 val_274 +NULL NULL 275 val_275 +NULL NULL 277 val_277 +NULL NULL 277 val_277 +NULL NULL 277 val_277 +NULL NULL 277 val_277 +NULL NULL 278 val_278 +NULL NULL 278 val_278 +NULL NULL 280 val_280 +NULL NULL 280 val_280 +NULL NULL 281 val_281 +NULL NULL 281 val_281 +NULL NULL 282 val_282 +NULL NULL 282 val_282 +NULL NULL 283 val_283 +NULL NULL 284 val_284 +NULL NULL 285 val_285 +NULL NULL 286 val_286 +NULL NULL 287 val_287 +NULL NULL 288 val_288 +NULL NULL 288 val_288 +NULL NULL 289 val_289 +NULL NULL 291 val_291 +NULL NULL 292 val_292 +NULL NULL 296 val_296 +NULL NULL 298 val_298 +NULL NULL 298 val_298 +NULL NULL 298 val_298 +NULL NULL 302 val_302 +NULL NULL 305 val_305 +NULL NULL 306 val_306 +NULL NULL 307 val_307 +NULL NULL 307 val_307 +NULL NULL 308 val_308 +NULL NULL 309 val_309 +NULL NULL 309 val_309 +NULL NULL 310 val_310 +NULL NULL 311 val_311 +NULL NULL 311 val_311 +NULL NULL 311 val_311 +NULL NULL 315 val_315 +NULL NULL 316 val_316 +NULL NULL 316 val_316 +NULL NULL 316 val_316 +NULL NULL 317 val_317 +NULL NULL 317 val_317 +NULL NULL 318 val_318 +NULL NULL 318 val_318 +NULL NULL 318 val_318 +NULL NULL 321 val_321 +NULL NULL 321 val_321 +NULL NULL 322 val_322 +NULL NULL 322 val_322 +NULL NULL 323 val_323 +NULL NULL 325 val_325 +NULL NULL 325 val_325 +NULL NULL 327 val_327 +NULL NULL 327 val_327 +NULL NULL 327 val_327 +NULL NULL 331 val_331 +NULL NULL 331 val_331 +NULL NULL 332 val_332 +NULL NULL 333 val_333 +NULL NULL 333 val_333 +NULL NULL 335 val_335 +NULL NULL 336 val_336 +NULL NULL 338 val_338 +NULL NULL 339 val_339 +NULL NULL 341 val_341 +NULL NULL 342 val_342 +NULL NULL 342 val_342 +NULL NULL 344 val_344 +NULL NULL 344 val_344 +NULL NULL 345 val_345 +NULL NULL 348 val_348 +NULL NULL 348 val_348 +NULL NULL 348 val_348 +NULL NULL 348 val_348 +NULL NULL 348 val_348 +NULL NULL 351 val_351 +NULL NULL 353 val_353 +NULL NULL 353 val_353 +NULL NULL 356 val_356 +NULL NULL 360 val_360 +NULL NULL 362 val_362 +NULL NULL 364 val_364 +NULL NULL 365 val_365 +NULL NULL 366 val_366 +NULL NULL 367 val_367 +NULL NULL 367 val_367 +NULL NULL 368 val_368 +NULL NULL 369 val_369 +NULL NULL 369 val_369 +NULL NULL 369 val_369 +NULL NULL 373 val_373 +NULL NULL 374 val_374 +NULL NULL 375 val_375 +NULL NULL 377 val_377 +NULL NULL 378 val_378 +NULL NULL 379 val_379 +NULL NULL 382 val_382 +NULL NULL 382 val_382 +NULL NULL 384 val_384 +NULL NULL 384 val_384 +NULL NULL 384 val_384 +NULL NULL 386 val_386 +NULL NULL 389 val_389 +NULL NULL 392 val_392 +NULL NULL 393 val_393 +NULL NULL 394 val_394 +NULL NULL 395 val_395 +NULL NULL 395 val_395 +NULL NULL 396 val_396 +NULL NULL 396 val_396 +NULL NULL 396 val_396 +NULL NULL 397 val_397 +NULL NULL 397 val_397 +NULL NULL 399 val_399 +NULL NULL 399 val_399 +NULL NULL 400 val_400 +NULL NULL 401 val_401 +NULL NULL 401 val_401 +NULL NULL 401 val_401 +NULL NULL 401 val_401 +NULL NULL 401 val_401 +NULL NULL 402 val_402 +NULL NULL 403 val_403 +NULL NULL 403 val_403 +NULL NULL 403 val_403 +NULL NULL 404 val_404 +NULL NULL 404 val_404 +NULL NULL 406 val_406 +NULL NULL 406 val_406 +NULL NULL 406 val_406 +NULL NULL 406 val_406 +NULL NULL 407 val_407 +NULL NULL 409 val_409 +NULL NULL 409 val_409 +NULL NULL 409 val_409 +NULL NULL 411 val_411 +NULL NULL 413 val_413 +NULL NULL 413 val_413 +NULL NULL 414 val_414 +NULL NULL 414 val_414 +NULL NULL 417 val_417 +NULL NULL 417 val_417 +NULL NULL 417 val_417 +NULL NULL 418 val_418 +NULL NULL 419 val_419 +NULL NULL 421 val_421 +NULL NULL 424 val_424 +NULL NULL 424 val_424 +NULL NULL 427 val_427 +NULL NULL 429 val_429 +NULL NULL 429 val_429 +NULL NULL 430 val_430 +NULL NULL 430 val_430 +NULL NULL 430 val_430 +NULL NULL 431 val_431 +NULL NULL 431 val_431 +NULL NULL 431 val_431 +NULL NULL 432 val_432 +NULL NULL 435 val_435 +NULL NULL 436 val_436 +NULL NULL 437 val_437 +NULL NULL 438 val_438 +NULL NULL 438 val_438 +NULL NULL 438 val_438 +NULL NULL 439 val_439 +NULL NULL 439 val_439 +NULL NULL 443 val_443 +NULL NULL 444 val_444 +NULL NULL 446 val_446 +NULL NULL 448 val_448 +NULL NULL 449 val_449 +NULL NULL 452 val_452 +NULL NULL 453 val_453 +NULL NULL 454 val_454 +NULL NULL 454 val_454 +NULL NULL 454 val_454 +NULL NULL 455 val_455 +NULL NULL 457 val_457 +NULL NULL 458 val_458 +NULL NULL 458 val_458 +NULL NULL 459 val_459 +NULL NULL 459 val_459 +NULL NULL 460 val_460 +NULL NULL 462 val_462 +NULL NULL 462 val_462 +NULL NULL 463 val_463 +NULL NULL 463 val_463 +NULL NULL 466 val_466 +NULL NULL 466 val_466 +NULL NULL 466 val_466 +NULL NULL 467 val_467 +NULL NULL 468 val_468 +NULL NULL 468 val_468 +NULL NULL 468 val_468 +NULL NULL 468 val_468 +NULL NULL 469 val_469 +NULL NULL 469 val_469 +NULL NULL 469 val_469 +NULL NULL 469 val_469 +NULL NULL 469 val_469 +NULL NULL 470 val_470 +NULL NULL 472 val_472 +NULL NULL 475 val_475 +NULL NULL 477 val_477 +NULL NULL 478 val_478 +NULL NULL 478 val_478 +NULL NULL 479 val_479 +NULL NULL 480 val_480 +NULL NULL 480 val_480 +NULL NULL 480 val_480 +NULL NULL 481 val_481 +NULL NULL 482 val_482 +NULL NULL 483 val_483 +NULL NULL 484 val_484 +NULL NULL 485 val_485 +NULL NULL 487 val_487 +NULL NULL 489 val_489 +NULL NULL 489 val_489 +NULL NULL 489 val_489 +NULL NULL 489 val_489 +NULL NULL 490 val_490 +NULL NULL 491 val_491 +NULL NULL 492 val_492 +NULL NULL 492 val_492 +NULL NULL 493 val_493 +NULL NULL 494 val_494 +NULL NULL 495 val_495 +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 diff --git a/src/test/hive/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd b/src/test/hive/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a b/src/test/hive/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a b/src/test/hive/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a new file mode 100644 index 0000000000000..4cd5eefea2a45 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a @@ -0,0 +1,500 @@ +NULL NULL 0 val_0 +NULL NULL 0 val_0 +NULL NULL 0 val_0 +NULL NULL 2 val_2 +NULL NULL 4 val_4 +NULL NULL 5 val_5 +NULL NULL 5 val_5 +NULL NULL 5 val_5 +NULL NULL 8 val_8 +NULL NULL 9 val_9 +NULL NULL 10 val_10 +NULL NULL 11 val_11 +NULL NULL 12 val_12 +NULL NULL 12 val_12 +NULL NULL 15 val_15 +NULL NULL 15 val_15 +NULL NULL 17 val_17 +NULL NULL 18 val_18 +NULL NULL 18 val_18 +NULL NULL 19 val_19 +NULL NULL 20 val_20 +NULL NULL 24 val_24 +NULL NULL 24 val_24 +NULL NULL 26 val_26 +NULL NULL 26 val_26 +NULL NULL 27 val_27 +NULL NULL 28 val_28 +NULL NULL 30 val_30 +NULL NULL 33 val_33 +NULL NULL 34 val_34 +NULL NULL 35 val_35 +NULL NULL 35 val_35 +NULL NULL 35 val_35 +NULL NULL 37 val_37 +NULL NULL 37 val_37 +NULL NULL 41 val_41 +NULL NULL 42 val_42 +NULL NULL 42 val_42 +NULL NULL 43 val_43 +NULL NULL 44 val_44 +NULL NULL 47 val_47 +NULL NULL 51 val_51 +NULL NULL 51 val_51 +NULL NULL 53 val_53 +NULL NULL 54 val_54 +NULL NULL 57 val_57 +NULL NULL 58 val_58 +NULL NULL 58 val_58 +NULL NULL 64 val_64 +NULL NULL 65 val_65 +NULL NULL 66 val_66 +NULL NULL 67 val_67 +NULL NULL 67 val_67 +NULL NULL 69 val_69 +NULL NULL 70 val_70 +NULL NULL 70 val_70 +NULL NULL 70 val_70 +NULL NULL 72 val_72 +NULL NULL 72 val_72 +NULL NULL 74 val_74 +NULL NULL 76 val_76 +NULL NULL 76 val_76 +NULL NULL 77 val_77 +NULL NULL 78 val_78 +NULL NULL 80 val_80 +NULL NULL 82 val_82 +NULL NULL 83 val_83 +NULL NULL 83 val_83 +NULL NULL 84 val_84 +NULL NULL 84 val_84 +NULL NULL 85 val_85 +NULL NULL 86 val_86 +NULL NULL 87 val_87 +NULL NULL 90 val_90 +NULL NULL 90 val_90 +NULL NULL 90 val_90 +NULL NULL 92 val_92 +NULL NULL 95 val_95 +NULL NULL 95 val_95 +NULL NULL 96 val_96 +NULL NULL 97 val_97 +NULL NULL 97 val_97 +NULL NULL 98 val_98 +NULL NULL 98 val_98 +NULL NULL 100 val_100 +NULL NULL 100 val_100 +NULL NULL 103 val_103 +NULL NULL 103 val_103 +NULL NULL 104 val_104 +NULL NULL 104 val_104 +NULL NULL 105 val_105 +NULL NULL 111 val_111 +NULL NULL 113 val_113 +NULL NULL 113 val_113 +NULL NULL 114 val_114 +NULL NULL 116 val_116 +NULL NULL 118 val_118 +NULL NULL 118 val_118 +NULL NULL 119 val_119 +NULL NULL 119 val_119 +NULL NULL 119 val_119 +NULL NULL 120 val_120 +NULL NULL 120 val_120 +NULL NULL 125 val_125 +NULL NULL 125 val_125 +NULL NULL 126 val_126 +NULL NULL 128 val_128 +NULL NULL 128 val_128 +NULL NULL 128 val_128 +NULL NULL 129 val_129 +NULL NULL 129 val_129 +NULL NULL 131 val_131 +NULL NULL 133 val_133 +NULL NULL 134 val_134 +NULL NULL 134 val_134 +NULL NULL 136 val_136 +NULL NULL 137 val_137 +NULL NULL 137 val_137 +NULL NULL 138 val_138 +NULL NULL 138 val_138 +NULL NULL 138 val_138 +NULL NULL 138 val_138 +NULL NULL 143 val_143 +NULL NULL 145 val_145 +NULL NULL 146 val_146 +NULL NULL 146 val_146 +NULL NULL 149 val_149 +NULL NULL 149 val_149 +NULL NULL 150 val_150 +NULL NULL 152 val_152 +NULL NULL 152 val_152 +NULL NULL 153 val_153 +NULL NULL 155 val_155 +NULL NULL 156 val_156 +NULL NULL 157 val_157 +NULL NULL 158 val_158 +NULL NULL 160 val_160 +NULL NULL 162 val_162 +NULL NULL 163 val_163 +NULL NULL 164 val_164 +NULL NULL 164 val_164 +NULL NULL 165 val_165 +NULL NULL 165 val_165 +NULL NULL 166 val_166 +NULL NULL 167 val_167 +NULL NULL 167 val_167 +NULL NULL 167 val_167 +NULL NULL 168 val_168 +NULL NULL 169 val_169 +NULL NULL 169 val_169 +NULL NULL 169 val_169 +NULL NULL 169 val_169 +NULL NULL 170 val_170 +NULL NULL 172 val_172 +NULL NULL 172 val_172 +NULL NULL 174 val_174 +NULL NULL 174 val_174 +NULL NULL 175 val_175 +NULL NULL 175 val_175 +NULL NULL 176 val_176 +NULL NULL 176 val_176 +NULL NULL 177 val_177 +NULL NULL 178 val_178 +NULL NULL 179 val_179 +NULL NULL 179 val_179 +NULL NULL 180 val_180 +NULL NULL 181 val_181 +NULL NULL 183 val_183 +NULL NULL 186 val_186 +NULL NULL 187 val_187 +NULL NULL 187 val_187 +NULL NULL 187 val_187 +NULL NULL 189 val_189 +NULL NULL 190 val_190 +NULL NULL 191 val_191 +NULL NULL 191 val_191 +NULL NULL 192 val_192 +NULL NULL 193 val_193 +NULL NULL 193 val_193 +NULL NULL 193 val_193 +NULL NULL 194 val_194 +NULL NULL 195 val_195 +NULL NULL 195 val_195 +NULL NULL 196 val_196 +NULL NULL 197 val_197 +NULL NULL 197 val_197 +NULL NULL 199 val_199 +NULL NULL 199 val_199 +NULL NULL 199 val_199 +NULL NULL 200 val_200 +NULL NULL 200 val_200 +NULL NULL 201 val_201 +NULL NULL 202 val_202 +NULL NULL 203 val_203 +NULL NULL 203 val_203 +NULL NULL 205 val_205 +NULL NULL 205 val_205 +NULL NULL 207 val_207 +NULL NULL 207 val_207 +NULL NULL 208 val_208 +NULL NULL 208 val_208 +NULL NULL 208 val_208 +NULL NULL 209 val_209 +NULL NULL 209 val_209 +NULL NULL 213 val_213 +NULL NULL 213 val_213 +NULL NULL 214 val_214 +NULL NULL 216 val_216 +NULL NULL 216 val_216 +NULL NULL 217 val_217 +NULL NULL 217 val_217 +NULL NULL 218 val_218 +NULL NULL 219 val_219 +NULL NULL 219 val_219 +NULL NULL 221 val_221 +NULL NULL 221 val_221 +NULL NULL 222 val_222 +NULL NULL 223 val_223 +NULL NULL 223 val_223 +NULL NULL 224 val_224 +NULL NULL 224 val_224 +NULL NULL 226 val_226 +NULL NULL 228 val_228 +NULL NULL 229 val_229 +NULL NULL 229 val_229 +NULL NULL 230 val_230 +NULL NULL 230 val_230 +NULL NULL 230 val_230 +NULL NULL 230 val_230 +NULL NULL 230 val_230 +NULL NULL 233 val_233 +NULL NULL 233 val_233 +NULL NULL 235 val_235 +NULL NULL 237 val_237 +NULL NULL 237 val_237 +NULL NULL 238 val_238 +NULL NULL 238 val_238 +NULL NULL 239 val_239 +NULL NULL 239 val_239 +NULL NULL 241 val_241 +NULL NULL 242 val_242 +NULL NULL 242 val_242 +NULL NULL 244 val_244 +NULL NULL 247 val_247 +NULL NULL 248 val_248 +NULL NULL 249 val_249 +NULL NULL 252 val_252 +NULL NULL 255 val_255 +NULL NULL 255 val_255 +NULL NULL 256 val_256 +NULL NULL 256 val_256 +NULL NULL 257 val_257 +NULL NULL 258 val_258 +NULL NULL 260 val_260 +NULL NULL 262 val_262 +NULL NULL 263 val_263 +NULL NULL 265 val_265 +NULL NULL 265 val_265 +NULL NULL 266 val_266 +NULL NULL 272 val_272 +NULL NULL 272 val_272 +NULL NULL 273 val_273 +NULL NULL 273 val_273 +NULL NULL 273 val_273 +NULL NULL 274 val_274 +NULL NULL 275 val_275 +NULL NULL 277 val_277 +NULL NULL 277 val_277 +NULL NULL 277 val_277 +NULL NULL 277 val_277 +NULL NULL 278 val_278 +NULL NULL 278 val_278 +NULL NULL 280 val_280 +NULL NULL 280 val_280 +NULL NULL 281 val_281 +NULL NULL 281 val_281 +NULL NULL 282 val_282 +NULL NULL 282 val_282 +NULL NULL 283 val_283 +NULL NULL 284 val_284 +NULL NULL 285 val_285 +NULL NULL 286 val_286 +NULL NULL 287 val_287 +NULL NULL 288 val_288 +NULL NULL 288 val_288 +NULL NULL 289 val_289 +NULL NULL 291 val_291 +NULL NULL 292 val_292 +NULL NULL 296 val_296 +NULL NULL 298 val_298 +NULL NULL 298 val_298 +NULL NULL 298 val_298 +NULL NULL 302 val_302 +NULL NULL 305 val_305 +NULL NULL 306 val_306 +NULL NULL 307 val_307 +NULL NULL 307 val_307 +NULL NULL 308 val_308 +NULL NULL 309 val_309 +NULL NULL 309 val_309 +NULL NULL 310 val_310 +NULL NULL 311 val_311 +NULL NULL 311 val_311 +NULL NULL 311 val_311 +NULL NULL 315 val_315 +NULL NULL 316 val_316 +NULL NULL 316 val_316 +NULL NULL 316 val_316 +NULL NULL 317 val_317 +NULL NULL 317 val_317 +NULL NULL 318 val_318 +NULL NULL 318 val_318 +NULL NULL 318 val_318 +NULL NULL 321 val_321 +NULL NULL 321 val_321 +NULL NULL 322 val_322 +NULL NULL 322 val_322 +NULL NULL 323 val_323 +NULL NULL 325 val_325 +NULL NULL 325 val_325 +NULL NULL 327 val_327 +NULL NULL 327 val_327 +NULL NULL 327 val_327 +NULL NULL 331 val_331 +NULL NULL 331 val_331 +NULL NULL 332 val_332 +NULL NULL 333 val_333 +NULL NULL 333 val_333 +NULL NULL 335 val_335 +NULL NULL 336 val_336 +NULL NULL 338 val_338 +NULL NULL 339 val_339 +NULL NULL 341 val_341 +NULL NULL 342 val_342 +NULL NULL 342 val_342 +NULL NULL 344 val_344 +NULL NULL 344 val_344 +NULL NULL 345 val_345 +NULL NULL 348 val_348 +NULL NULL 348 val_348 +NULL NULL 348 val_348 +NULL NULL 348 val_348 +NULL NULL 348 val_348 +NULL NULL 351 val_351 +NULL NULL 353 val_353 +NULL NULL 353 val_353 +NULL NULL 356 val_356 +NULL NULL 360 val_360 +NULL NULL 362 val_362 +NULL NULL 364 val_364 +NULL NULL 365 val_365 +NULL NULL 366 val_366 +NULL NULL 367 val_367 +NULL NULL 367 val_367 +NULL NULL 368 val_368 +NULL NULL 369 val_369 +NULL NULL 369 val_369 +NULL NULL 369 val_369 +NULL NULL 373 val_373 +NULL NULL 374 val_374 +NULL NULL 375 val_375 +NULL NULL 377 val_377 +NULL NULL 378 val_378 +NULL NULL 379 val_379 +NULL NULL 382 val_382 +NULL NULL 382 val_382 +NULL NULL 384 val_384 +NULL NULL 384 val_384 +NULL NULL 384 val_384 +NULL NULL 386 val_386 +NULL NULL 389 val_389 +NULL NULL 392 val_392 +NULL NULL 393 val_393 +NULL NULL 394 val_394 +NULL NULL 395 val_395 +NULL NULL 395 val_395 +NULL NULL 396 val_396 +NULL NULL 396 val_396 +NULL NULL 396 val_396 +NULL NULL 397 val_397 +NULL NULL 397 val_397 +NULL NULL 399 val_399 +NULL NULL 399 val_399 +NULL NULL 400 val_400 +NULL NULL 401 val_401 +NULL NULL 401 val_401 +NULL NULL 401 val_401 +NULL NULL 401 val_401 +NULL NULL 401 val_401 +NULL NULL 402 val_402 +NULL NULL 403 val_403 +NULL NULL 403 val_403 +NULL NULL 403 val_403 +NULL NULL 404 val_404 +NULL NULL 404 val_404 +NULL NULL 406 val_406 +NULL NULL 406 val_406 +NULL NULL 406 val_406 +NULL NULL 406 val_406 +NULL NULL 407 val_407 +NULL NULL 409 val_409 +NULL NULL 409 val_409 +NULL NULL 409 val_409 +NULL NULL 411 val_411 +NULL NULL 413 val_413 +NULL NULL 413 val_413 +NULL NULL 414 val_414 +NULL NULL 414 val_414 +NULL NULL 417 val_417 +NULL NULL 417 val_417 +NULL NULL 417 val_417 +NULL NULL 418 val_418 +NULL NULL 419 val_419 +NULL NULL 421 val_421 +NULL NULL 424 val_424 +NULL NULL 424 val_424 +NULL NULL 427 val_427 +NULL NULL 429 val_429 +NULL NULL 429 val_429 +NULL NULL 430 val_430 +NULL NULL 430 val_430 +NULL NULL 430 val_430 +NULL NULL 431 val_431 +NULL NULL 431 val_431 +NULL NULL 431 val_431 +NULL NULL 432 val_432 +NULL NULL 435 val_435 +NULL NULL 436 val_436 +NULL NULL 437 val_437 +NULL NULL 438 val_438 +NULL NULL 438 val_438 +NULL NULL 438 val_438 +NULL NULL 439 val_439 +NULL NULL 439 val_439 +NULL NULL 443 val_443 +NULL NULL 444 val_444 +NULL NULL 446 val_446 +NULL NULL 448 val_448 +NULL NULL 449 val_449 +NULL NULL 452 val_452 +NULL NULL 453 val_453 +NULL NULL 454 val_454 +NULL NULL 454 val_454 +NULL NULL 454 val_454 +NULL NULL 455 val_455 +NULL NULL 457 val_457 +NULL NULL 458 val_458 +NULL NULL 458 val_458 +NULL NULL 459 val_459 +NULL NULL 459 val_459 +NULL NULL 460 val_460 +NULL NULL 462 val_462 +NULL NULL 462 val_462 +NULL NULL 463 val_463 +NULL NULL 463 val_463 +NULL NULL 466 val_466 +NULL NULL 466 val_466 +NULL NULL 466 val_466 +NULL NULL 467 val_467 +NULL NULL 468 val_468 +NULL NULL 468 val_468 +NULL NULL 468 val_468 +NULL NULL 468 val_468 +NULL NULL 469 val_469 +NULL NULL 469 val_469 +NULL NULL 469 val_469 +NULL NULL 469 val_469 +NULL NULL 469 val_469 +NULL NULL 470 val_470 +NULL NULL 472 val_472 +NULL NULL 475 val_475 +NULL NULL 477 val_477 +NULL NULL 478 val_478 +NULL NULL 478 val_478 +NULL NULL 479 val_479 +NULL NULL 480 val_480 +NULL NULL 480 val_480 +NULL NULL 480 val_480 +NULL NULL 481 val_481 +NULL NULL 482 val_482 +NULL NULL 483 val_483 +NULL NULL 484 val_484 +NULL NULL 485 val_485 +NULL NULL 487 val_487 +NULL NULL 489 val_489 +NULL NULL 489 val_489 +NULL NULL 489 val_489 +NULL NULL 489 val_489 +NULL NULL 490 val_490 +NULL NULL 491 val_491 +NULL NULL 492 val_492 +NULL NULL 492 val_492 +NULL NULL 493 val_493 +NULL NULL 494 val_494 +NULL NULL 495 val_495 +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 diff --git a/src/test/hive/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 b/src/test/hive/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e b/src/test/hive/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e new file mode 100644 index 0000000000000..ca97f45c90026 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e @@ -0,0 +1 @@ +0 130091 0 36210398070 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a b/src/test/hive/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a new file mode 100644 index 0000000000000..ca97f45c90026 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a @@ -0,0 +1 @@ +0 130091 0 36210398070 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff b/src/test/hive/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff new file mode 100644 index 0000000000000..ca97f45c90026 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff @@ -0,0 +1 @@ +0 130091 0 36210398070 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa b/src/test/hive/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de b/src/test/hive/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c b/src/test/hive/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 b/src/test/hive/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 b/src/test/hive/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb b/src/test/hive/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f b/src/test/hive/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/src/test/hive/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 b/src/test/hive/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 b/src/test/hive/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b b/src/test/hive/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 b/src/test/hive/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 b/src/test/hive/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 b/src/test/hive/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b b/src/test/hive/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 b/src/test/hive/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 b/src/test/hive/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 b/src/test/hive/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b b/src/test/hive/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c b/src/test/hive/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 b/src/test/hive/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 b/src/test/hive/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b b/src/test/hive/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c b/src/test/hive/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 b/src/test/hive/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e b/src/test/hive/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a b/src/test/hive/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c b/src/test/hive/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 b/src/test/hive/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f b/src/test/hive/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a b/src/test/hive/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 b/src/test/hive/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 b/src/test/hive/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 b/src/test/hive/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a b/src/test/hive/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 b/src/test/hive/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b b/src/test/hive/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c b/src/test/hive/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 b/src/test/hive/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a b/src/test/hive/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 b/src/test/hive/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c b/src/test/hive/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 b/src/test/hive/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a b/src/test/hive/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 b/src/test/hive/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a b/src/test/hive/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 b/src/test/hive/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a b/src/test/hive/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 b/src/test/hive/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort-0-eee35bbc888a705ae527625447668032 b/src/test/hive/golden/sort-0-eee35bbc888a705ae527625447668032 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 b/src/test/hive/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 b/src/test/hive/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c b/src/test/hive/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 b/src/test/hive/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 b/src/test/hive/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 b/src/test/hive/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e b/src/test/hive/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 b/src/test/hive/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 b/src/test/hive/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 b/src/test/hive/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c b/src/test/hive/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 b/src/test/hive/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd b/src/test/hive/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 b/src/test/hive/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 b/src/test/hive/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 b/src/test/hive/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 b/src/test/hive/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 b/src/test/hive/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c b/src/test/hive/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 b/src/test/hive/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd b/src/test/hive/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 b/src/test/hive/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 b/src/test/hive/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 b/src/test/hive/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 b/src/test/hive/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 b/src/test/hive/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c b/src/test/hive/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 b/src/test/hive/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd b/src/test/hive/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 b/src/test/hive/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 b/src/test/hive/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 b/src/test/hive/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 b/src/test/hive/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 b/src/test/hive/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f b/src/test/hive/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a b/src/test/hive/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a new file mode 100644 index 0000000000000..67d892c80f493 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a @@ -0,0 +1 @@ +1028 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb b/src/test/hive/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d b/src/test/hive/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 b/src/test/hive/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf b/src/test/hive/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 b/src/test/hive/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed b/src/test/hive/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 b/src/test/hive/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 b/src/test/hive/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a b/src/test/hive/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a new file mode 100644 index 0000000000000..67d892c80f493 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a @@ -0,0 +1 @@ +1028 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b b/src/test/hive/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d b/src/test/hive/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 b/src/test/hive/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf b/src/test/hive/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 b/src/test/hive/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed b/src/test/hive/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e b/src/test/hive/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 b/src/test/hive/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 b/src/test/hive/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 b/src/test/hive/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed b/src/test/hive/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 b/src/test/hive/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 b/src/test/hive/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 b/src/test/hive/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 new file mode 100644 index 0000000000000..8983bf8a289a9 --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 @@ -0,0 +1 @@ +4112 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 b/src/test/hive/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b b/src/test/hive/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d b/src/test/hive/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 b/src/test/hive/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 b/src/test/hive/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f b/src/test/hive/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf b/src/test/hive/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec b/src/test/hive/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 b/src/test/hive/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c b/src/test/hive/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 b/src/test/hive/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 b/src/test/hive/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 new file mode 100644 index 0000000000000..284da67c9c7c1 --- /dev/null +++ b/src/test/hive/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 @@ -0,0 +1 @@ +ds=1 diff --git a/src/test/hive/golden/stats0-13-ca0569fd4200d549843deadf84044649 b/src/test/hive/golden/stats0-13-ca0569fd4200d549843deadf84044649 new file mode 100644 index 0000000000000..93381fd6177d3 --- /dev/null +++ b/src/test/hive/golden/stats0-13-ca0569fd4200d549843deadf84044649 @@ -0,0 +1,500 @@ +238 val_238 1 +86 val_86 1 +311 val_311 1 +27 val_27 1 +165 val_165 1 +409 val_409 1 +255 val_255 1 +278 val_278 1 +98 val_98 1 +484 val_484 1 +265 val_265 1 +193 val_193 1 +401 val_401 1 +150 val_150 1 +273 val_273 1 +224 val_224 1 +369 val_369 1 +66 val_66 1 +128 val_128 1 +213 val_213 1 +146 val_146 1 +406 val_406 1 +429 val_429 1 +374 val_374 1 +152 val_152 1 +469 val_469 1 +145 val_145 1 +495 val_495 1 +37 val_37 1 +327 val_327 1 +281 val_281 1 +277 val_277 1 +209 val_209 1 +15 val_15 1 +82 val_82 1 +403 val_403 1 +166 val_166 1 +417 val_417 1 +430 val_430 1 +252 val_252 1 +292 val_292 1 +219 val_219 1 +287 val_287 1 +153 val_153 1 +193 val_193 1 +338 val_338 1 +446 val_446 1 +459 val_459 1 +394 val_394 1 +237 val_237 1 +482 val_482 1 +174 val_174 1 +413 val_413 1 +494 val_494 1 +207 val_207 1 +199 val_199 1 +466 val_466 1 +208 val_208 1 +174 val_174 1 +399 val_399 1 +396 val_396 1 +247 val_247 1 +417 val_417 1 +489 val_489 1 +162 val_162 1 +377 val_377 1 +397 val_397 1 +309 val_309 1 +365 val_365 1 +266 val_266 1 +439 val_439 1 +342 val_342 1 +367 val_367 1 +325 val_325 1 +167 val_167 1 +195 val_195 1 +475 val_475 1 +17 val_17 1 +113 val_113 1 +155 val_155 1 +203 val_203 1 +339 val_339 1 +0 val_0 1 +455 val_455 1 +128 val_128 1 +311 val_311 1 +316 val_316 1 +57 val_57 1 +302 val_302 1 +205 val_205 1 +149 val_149 1 +438 val_438 1 +345 val_345 1 +129 val_129 1 +170 val_170 1 +20 val_20 1 +489 val_489 1 +157 val_157 1 +378 val_378 1 +221 val_221 1 +92 val_92 1 +111 val_111 1 +47 val_47 1 +72 val_72 1 +4 val_4 1 +280 val_280 1 +35 val_35 1 +427 val_427 1 +277 val_277 1 +208 val_208 1 +356 val_356 1 +399 val_399 1 +169 val_169 1 +382 val_382 1 +498 val_498 1 +125 val_125 1 +386 val_386 1 +437 val_437 1 +469 val_469 1 +192 val_192 1 +286 val_286 1 +187 val_187 1 +176 val_176 1 +54 val_54 1 +459 val_459 1 +51 val_51 1 +138 val_138 1 +103 val_103 1 +239 val_239 1 +213 val_213 1 +216 val_216 1 +430 val_430 1 +278 val_278 1 +176 val_176 1 +289 val_289 1 +221 val_221 1 +65 val_65 1 +318 val_318 1 +332 val_332 1 +311 val_311 1 +275 val_275 1 +137 val_137 1 +241 val_241 1 +83 val_83 1 +333 val_333 1 +180 val_180 1 +284 val_284 1 +12 val_12 1 +230 val_230 1 +181 val_181 1 +67 val_67 1 +260 val_260 1 +404 val_404 1 +384 val_384 1 +489 val_489 1 +353 val_353 1 +373 val_373 1 +272 val_272 1 +138 val_138 1 +217 val_217 1 +84 val_84 1 +348 val_348 1 +466 val_466 1 +58 val_58 1 +8 val_8 1 +411 val_411 1 +230 val_230 1 +208 val_208 1 +348 val_348 1 +24 val_24 1 +463 val_463 1 +431 val_431 1 +179 val_179 1 +172 val_172 1 +42 val_42 1 +129 val_129 1 +158 val_158 1 +119 val_119 1 +496 val_496 1 +0 val_0 1 +322 val_322 1 +197 val_197 1 +468 val_468 1 +393 val_393 1 +454 val_454 1 +100 val_100 1 +298 val_298 1 +199 val_199 1 +191 val_191 1 +418 val_418 1 +96 val_96 1 +26 val_26 1 +165 val_165 1 +327 val_327 1 +230 val_230 1 +205 val_205 1 +120 val_120 1 +131 val_131 1 +51 val_51 1 +404 val_404 1 +43 val_43 1 +436 val_436 1 +156 val_156 1 +469 val_469 1 +468 val_468 1 +308 val_308 1 +95 val_95 1 +196 val_196 1 +288 val_288 1 +481 val_481 1 +457 val_457 1 +98 val_98 1 +282 val_282 1 +197 val_197 1 +187 val_187 1 +318 val_318 1 +318 val_318 1 +409 val_409 1 +470 val_470 1 +137 val_137 1 +369 val_369 1 +316 val_316 1 +169 val_169 1 +413 val_413 1 +85 val_85 1 +77 val_77 1 +0 val_0 1 +490 val_490 1 +87 val_87 1 +364 val_364 1 +179 val_179 1 +118 val_118 1 +134 val_134 1 +395 val_395 1 +282 val_282 1 +138 val_138 1 +238 val_238 1 +419 val_419 1 +15 val_15 1 +118 val_118 1 +72 val_72 1 +90 val_90 1 +307 val_307 1 +19 val_19 1 +435 val_435 1 +10 val_10 1 +277 val_277 1 +273 val_273 1 +306 val_306 1 +224 val_224 1 +309 val_309 1 +389 val_389 1 +327 val_327 1 +242 val_242 1 +369 val_369 1 +392 val_392 1 +272 val_272 1 +331 val_331 1 +401 val_401 1 +242 val_242 1 +452 val_452 1 +177 val_177 1 +226 val_226 1 +5 val_5 1 +497 val_497 1 +402 val_402 1 +396 val_396 1 +317 val_317 1 +395 val_395 1 +58 val_58 1 +35 val_35 1 +336 val_336 1 +95 val_95 1 +11 val_11 1 +168 val_168 1 +34 val_34 1 +229 val_229 1 +233 val_233 1 +143 val_143 1 +472 val_472 1 +322 val_322 1 +498 val_498 1 +160 val_160 1 +195 val_195 1 +42 val_42 1 +321 val_321 1 +430 val_430 1 +119 val_119 1 +489 val_489 1 +458 val_458 1 +78 val_78 1 +76 val_76 1 +41 val_41 1 +223 val_223 1 +492 val_492 1 +149 val_149 1 +449 val_449 1 +218 val_218 1 +228 val_228 1 +138 val_138 1 +453 val_453 1 +30 val_30 1 +209 val_209 1 +64 val_64 1 +468 val_468 1 +76 val_76 1 +74 val_74 1 +342 val_342 1 +69 val_69 1 +230 val_230 1 +33 val_33 1 +368 val_368 1 +103 val_103 1 +296 val_296 1 +113 val_113 1 +216 val_216 1 +367 val_367 1 +344 val_344 1 +167 val_167 1 +274 val_274 1 +219 val_219 1 +239 val_239 1 +485 val_485 1 +116 val_116 1 +223 val_223 1 +256 val_256 1 +263 val_263 1 +70 val_70 1 +487 val_487 1 +480 val_480 1 +401 val_401 1 +288 val_288 1 +191 val_191 1 +5 val_5 1 +244 val_244 1 +438 val_438 1 +128 val_128 1 +467 val_467 1 +432 val_432 1 +202 val_202 1 +316 val_316 1 +229 val_229 1 +469 val_469 1 +463 val_463 1 +280 val_280 1 +2 val_2 1 +35 val_35 1 +283 val_283 1 +331 val_331 1 +235 val_235 1 +80 val_80 1 +44 val_44 1 +193 val_193 1 +321 val_321 1 +335 val_335 1 +104 val_104 1 +466 val_466 1 +366 val_366 1 +175 val_175 1 +403 val_403 1 +483 val_483 1 +53 val_53 1 +105 val_105 1 +257 val_257 1 +406 val_406 1 +409 val_409 1 +190 val_190 1 +406 val_406 1 +401 val_401 1 +114 val_114 1 +258 val_258 1 +90 val_90 1 +203 val_203 1 +262 val_262 1 +348 val_348 1 +424 val_424 1 +12 val_12 1 +396 val_396 1 +201 val_201 1 +217 val_217 1 +164 val_164 1 +431 val_431 1 +454 val_454 1 +478 val_478 1 +298 val_298 1 +125 val_125 1 +431 val_431 1 +164 val_164 1 +424 val_424 1 +187 val_187 1 +382 val_382 1 +5 val_5 1 +70 val_70 1 +397 val_397 1 +480 val_480 1 +291 val_291 1 +24 val_24 1 +351 val_351 1 +255 val_255 1 +104 val_104 1 +70 val_70 1 +163 val_163 1 +438 val_438 1 +119 val_119 1 +414 val_414 1 +200 val_200 1 +491 val_491 1 +237 val_237 1 +439 val_439 1 +360 val_360 1 +248 val_248 1 +479 val_479 1 +305 val_305 1 +417 val_417 1 +199 val_199 1 +444 val_444 1 +120 val_120 1 +429 val_429 1 +169 val_169 1 +443 val_443 1 +323 val_323 1 +325 val_325 1 +277 val_277 1 +230 val_230 1 +478 val_478 1 +178 val_178 1 +468 val_468 1 +310 val_310 1 +317 val_317 1 +333 val_333 1 +493 val_493 1 +460 val_460 1 +207 val_207 1 +249 val_249 1 +265 val_265 1 +480 val_480 1 +83 val_83 1 +136 val_136 1 +353 val_353 1 +172 val_172 1 +214 val_214 1 +462 val_462 1 +233 val_233 1 +406 val_406 1 +133 val_133 1 +175 val_175 1 +189 val_189 1 +454 val_454 1 +375 val_375 1 +401 val_401 1 +421 val_421 1 +407 val_407 1 +384 val_384 1 +256 val_256 1 +26 val_26 1 +134 val_134 1 +67 val_67 1 +384 val_384 1 +379 val_379 1 +18 val_18 1 +462 val_462 1 +492 val_492 1 +100 val_100 1 +298 val_298 1 +9 val_9 1 +341 val_341 1 +498 val_498 1 +146 val_146 1 +458 val_458 1 +362 val_362 1 +186 val_186 1 +285 val_285 1 +348 val_348 1 +167 val_167 1 +18 val_18 1 +273 val_273 1 +183 val_183 1 +281 val_281 1 +344 val_344 1 +97 val_97 1 +469 val_469 1 +315 val_315 1 +84 val_84 1 +28 val_28 1 +37 val_37 1 +448 val_448 1 +152 val_152 1 +348 val_348 1 +307 val_307 1 +194 val_194 1 +414 val_414 1 +477 val_477 1 +222 val_222 1 +126 val_126 1 +90 val_90 1 +169 val_169 1 +403 val_403 1 +400 val_400 1 +200 val_200 1 +97 val_97 1 diff --git a/src/test/hive/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 b/src/test/hive/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 new file mode 100644 index 0000000000000..688eea009d292 --- /dev/null +++ b/src/test/hive/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 @@ -0,0 +1,10 @@ +key string None +value string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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}) diff --git a/src/test/hive/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c b/src/test/hive/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c new file mode 100644 index 0000000000000..4d3a3d8f237b3 --- /dev/null +++ b/src/test/hive/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c @@ -0,0 +1,10 @@ +key string None +value string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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) diff --git a/src/test/hive/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 b/src/test/hive/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed b/src/test/hive/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats0-20-5f7439f97275da70f633e135205f2095 b/src/test/hive/golden/stats0-20-5f7439f97275da70f633e135205f2095 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a b/src/test/hive/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 b/src/test/hive/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c b/src/test/hive/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c new file mode 100644 index 0000000000000..a18ab93992cc5 --- /dev/null +++ b/src/test/hive/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c @@ -0,0 +1,4 @@ +key string None +value string None + +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) diff --git a/src/test/hive/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 b/src/test/hive/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/src/test/hive/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/src/test/hive/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b b/src/test/hive/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c b/src/test/hive/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 b/src/test/hive/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 b/src/test/hive/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 new file mode 100644 index 0000000000000..284da67c9c7c1 --- /dev/null +++ b/src/test/hive/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 @@ -0,0 +1 @@ +ds=1 diff --git a/src/test/hive/golden/stats0-29-ca0569fd4200d549843deadf84044649 b/src/test/hive/golden/stats0-29-ca0569fd4200d549843deadf84044649 new file mode 100644 index 0000000000000..93381fd6177d3 --- /dev/null +++ b/src/test/hive/golden/stats0-29-ca0569fd4200d549843deadf84044649 @@ -0,0 +1,500 @@ +238 val_238 1 +86 val_86 1 +311 val_311 1 +27 val_27 1 +165 val_165 1 +409 val_409 1 +255 val_255 1 +278 val_278 1 +98 val_98 1 +484 val_484 1 +265 val_265 1 +193 val_193 1 +401 val_401 1 +150 val_150 1 +273 val_273 1 +224 val_224 1 +369 val_369 1 +66 val_66 1 +128 val_128 1 +213 val_213 1 +146 val_146 1 +406 val_406 1 +429 val_429 1 +374 val_374 1 +152 val_152 1 +469 val_469 1 +145 val_145 1 +495 val_495 1 +37 val_37 1 +327 val_327 1 +281 val_281 1 +277 val_277 1 +209 val_209 1 +15 val_15 1 +82 val_82 1 +403 val_403 1 +166 val_166 1 +417 val_417 1 +430 val_430 1 +252 val_252 1 +292 val_292 1 +219 val_219 1 +287 val_287 1 +153 val_153 1 +193 val_193 1 +338 val_338 1 +446 val_446 1 +459 val_459 1 +394 val_394 1 +237 val_237 1 +482 val_482 1 +174 val_174 1 +413 val_413 1 +494 val_494 1 +207 val_207 1 +199 val_199 1 +466 val_466 1 +208 val_208 1 +174 val_174 1 +399 val_399 1 +396 val_396 1 +247 val_247 1 +417 val_417 1 +489 val_489 1 +162 val_162 1 +377 val_377 1 +397 val_397 1 +309 val_309 1 +365 val_365 1 +266 val_266 1 +439 val_439 1 +342 val_342 1 +367 val_367 1 +325 val_325 1 +167 val_167 1 +195 val_195 1 +475 val_475 1 +17 val_17 1 +113 val_113 1 +155 val_155 1 +203 val_203 1 +339 val_339 1 +0 val_0 1 +455 val_455 1 +128 val_128 1 +311 val_311 1 +316 val_316 1 +57 val_57 1 +302 val_302 1 +205 val_205 1 +149 val_149 1 +438 val_438 1 +345 val_345 1 +129 val_129 1 +170 val_170 1 +20 val_20 1 +489 val_489 1 +157 val_157 1 +378 val_378 1 +221 val_221 1 +92 val_92 1 +111 val_111 1 +47 val_47 1 +72 val_72 1 +4 val_4 1 +280 val_280 1 +35 val_35 1 +427 val_427 1 +277 val_277 1 +208 val_208 1 +356 val_356 1 +399 val_399 1 +169 val_169 1 +382 val_382 1 +498 val_498 1 +125 val_125 1 +386 val_386 1 +437 val_437 1 +469 val_469 1 +192 val_192 1 +286 val_286 1 +187 val_187 1 +176 val_176 1 +54 val_54 1 +459 val_459 1 +51 val_51 1 +138 val_138 1 +103 val_103 1 +239 val_239 1 +213 val_213 1 +216 val_216 1 +430 val_430 1 +278 val_278 1 +176 val_176 1 +289 val_289 1 +221 val_221 1 +65 val_65 1 +318 val_318 1 +332 val_332 1 +311 val_311 1 +275 val_275 1 +137 val_137 1 +241 val_241 1 +83 val_83 1 +333 val_333 1 +180 val_180 1 +284 val_284 1 +12 val_12 1 +230 val_230 1 +181 val_181 1 +67 val_67 1 +260 val_260 1 +404 val_404 1 +384 val_384 1 +489 val_489 1 +353 val_353 1 +373 val_373 1 +272 val_272 1 +138 val_138 1 +217 val_217 1 +84 val_84 1 +348 val_348 1 +466 val_466 1 +58 val_58 1 +8 val_8 1 +411 val_411 1 +230 val_230 1 +208 val_208 1 +348 val_348 1 +24 val_24 1 +463 val_463 1 +431 val_431 1 +179 val_179 1 +172 val_172 1 +42 val_42 1 +129 val_129 1 +158 val_158 1 +119 val_119 1 +496 val_496 1 +0 val_0 1 +322 val_322 1 +197 val_197 1 +468 val_468 1 +393 val_393 1 +454 val_454 1 +100 val_100 1 +298 val_298 1 +199 val_199 1 +191 val_191 1 +418 val_418 1 +96 val_96 1 +26 val_26 1 +165 val_165 1 +327 val_327 1 +230 val_230 1 +205 val_205 1 +120 val_120 1 +131 val_131 1 +51 val_51 1 +404 val_404 1 +43 val_43 1 +436 val_436 1 +156 val_156 1 +469 val_469 1 +468 val_468 1 +308 val_308 1 +95 val_95 1 +196 val_196 1 +288 val_288 1 +481 val_481 1 +457 val_457 1 +98 val_98 1 +282 val_282 1 +197 val_197 1 +187 val_187 1 +318 val_318 1 +318 val_318 1 +409 val_409 1 +470 val_470 1 +137 val_137 1 +369 val_369 1 +316 val_316 1 +169 val_169 1 +413 val_413 1 +85 val_85 1 +77 val_77 1 +0 val_0 1 +490 val_490 1 +87 val_87 1 +364 val_364 1 +179 val_179 1 +118 val_118 1 +134 val_134 1 +395 val_395 1 +282 val_282 1 +138 val_138 1 +238 val_238 1 +419 val_419 1 +15 val_15 1 +118 val_118 1 +72 val_72 1 +90 val_90 1 +307 val_307 1 +19 val_19 1 +435 val_435 1 +10 val_10 1 +277 val_277 1 +273 val_273 1 +306 val_306 1 +224 val_224 1 +309 val_309 1 +389 val_389 1 +327 val_327 1 +242 val_242 1 +369 val_369 1 +392 val_392 1 +272 val_272 1 +331 val_331 1 +401 val_401 1 +242 val_242 1 +452 val_452 1 +177 val_177 1 +226 val_226 1 +5 val_5 1 +497 val_497 1 +402 val_402 1 +396 val_396 1 +317 val_317 1 +395 val_395 1 +58 val_58 1 +35 val_35 1 +336 val_336 1 +95 val_95 1 +11 val_11 1 +168 val_168 1 +34 val_34 1 +229 val_229 1 +233 val_233 1 +143 val_143 1 +472 val_472 1 +322 val_322 1 +498 val_498 1 +160 val_160 1 +195 val_195 1 +42 val_42 1 +321 val_321 1 +430 val_430 1 +119 val_119 1 +489 val_489 1 +458 val_458 1 +78 val_78 1 +76 val_76 1 +41 val_41 1 +223 val_223 1 +492 val_492 1 +149 val_149 1 +449 val_449 1 +218 val_218 1 +228 val_228 1 +138 val_138 1 +453 val_453 1 +30 val_30 1 +209 val_209 1 +64 val_64 1 +468 val_468 1 +76 val_76 1 +74 val_74 1 +342 val_342 1 +69 val_69 1 +230 val_230 1 +33 val_33 1 +368 val_368 1 +103 val_103 1 +296 val_296 1 +113 val_113 1 +216 val_216 1 +367 val_367 1 +344 val_344 1 +167 val_167 1 +274 val_274 1 +219 val_219 1 +239 val_239 1 +485 val_485 1 +116 val_116 1 +223 val_223 1 +256 val_256 1 +263 val_263 1 +70 val_70 1 +487 val_487 1 +480 val_480 1 +401 val_401 1 +288 val_288 1 +191 val_191 1 +5 val_5 1 +244 val_244 1 +438 val_438 1 +128 val_128 1 +467 val_467 1 +432 val_432 1 +202 val_202 1 +316 val_316 1 +229 val_229 1 +469 val_469 1 +463 val_463 1 +280 val_280 1 +2 val_2 1 +35 val_35 1 +283 val_283 1 +331 val_331 1 +235 val_235 1 +80 val_80 1 +44 val_44 1 +193 val_193 1 +321 val_321 1 +335 val_335 1 +104 val_104 1 +466 val_466 1 +366 val_366 1 +175 val_175 1 +403 val_403 1 +483 val_483 1 +53 val_53 1 +105 val_105 1 +257 val_257 1 +406 val_406 1 +409 val_409 1 +190 val_190 1 +406 val_406 1 +401 val_401 1 +114 val_114 1 +258 val_258 1 +90 val_90 1 +203 val_203 1 +262 val_262 1 +348 val_348 1 +424 val_424 1 +12 val_12 1 +396 val_396 1 +201 val_201 1 +217 val_217 1 +164 val_164 1 +431 val_431 1 +454 val_454 1 +478 val_478 1 +298 val_298 1 +125 val_125 1 +431 val_431 1 +164 val_164 1 +424 val_424 1 +187 val_187 1 +382 val_382 1 +5 val_5 1 +70 val_70 1 +397 val_397 1 +480 val_480 1 +291 val_291 1 +24 val_24 1 +351 val_351 1 +255 val_255 1 +104 val_104 1 +70 val_70 1 +163 val_163 1 +438 val_438 1 +119 val_119 1 +414 val_414 1 +200 val_200 1 +491 val_491 1 +237 val_237 1 +439 val_439 1 +360 val_360 1 +248 val_248 1 +479 val_479 1 +305 val_305 1 +417 val_417 1 +199 val_199 1 +444 val_444 1 +120 val_120 1 +429 val_429 1 +169 val_169 1 +443 val_443 1 +323 val_323 1 +325 val_325 1 +277 val_277 1 +230 val_230 1 +478 val_478 1 +178 val_178 1 +468 val_468 1 +310 val_310 1 +317 val_317 1 +333 val_333 1 +493 val_493 1 +460 val_460 1 +207 val_207 1 +249 val_249 1 +265 val_265 1 +480 val_480 1 +83 val_83 1 +136 val_136 1 +353 val_353 1 +172 val_172 1 +214 val_214 1 +462 val_462 1 +233 val_233 1 +406 val_406 1 +133 val_133 1 +175 val_175 1 +189 val_189 1 +454 val_454 1 +375 val_375 1 +401 val_401 1 +421 val_421 1 +407 val_407 1 +384 val_384 1 +256 val_256 1 +26 val_26 1 +134 val_134 1 +67 val_67 1 +384 val_384 1 +379 val_379 1 +18 val_18 1 +462 val_462 1 +492 val_492 1 +100 val_100 1 +298 val_298 1 +9 val_9 1 +341 val_341 1 +498 val_498 1 +146 val_146 1 +458 val_458 1 +362 val_362 1 +186 val_186 1 +285 val_285 1 +348 val_348 1 +167 val_167 1 +18 val_18 1 +273 val_273 1 +183 val_183 1 +281 val_281 1 +344 val_344 1 +97 val_97 1 +469 val_469 1 +315 val_315 1 +84 val_84 1 +28 val_28 1 +37 val_37 1 +448 val_448 1 +152 val_152 1 +348 val_348 1 +307 val_307 1 +194 val_194 1 +414 val_414 1 +477 val_477 1 +222 val_222 1 +126 val_126 1 +90 val_90 1 +169 val_169 1 +403 val_403 1 +400 val_400 1 +200 val_200 1 +97 val_97 1 diff --git a/src/test/hive/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 b/src/test/hive/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 new file mode 100644 index 0000000000000..d861060f13cae --- /dev/null +++ b/src/test/hive/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 @@ -0,0 +1,10 @@ +key string None +value string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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}) diff --git a/src/test/hive/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c b/src/test/hive/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c new file mode 100644 index 0000000000000..aa4dfe1e5ba39 --- /dev/null +++ b/src/test/hive/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c @@ -0,0 +1,10 @@ +key string None +value string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None + +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) diff --git a/src/test/hive/golden/stats0-4-5f7439f97275da70f633e135205f2095 b/src/test/hive/golden/stats0-4-5f7439f97275da70f633e135205f2095 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a b/src/test/hive/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 b/src/test/hive/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c b/src/test/hive/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c new file mode 100644 index 0000000000000..8a5ae7cda1c57 --- /dev/null +++ b/src/test/hive/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c @@ -0,0 +1,4 @@ +key string None +value string None + +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) diff --git a/src/test/hive/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 b/src/test/hive/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/src/test/hive/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/src/test/hive/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b b/src/test/hive/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e b/src/test/hive/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b b/src/test/hive/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b new file mode 100644 index 0000000000000..f87b67cb2006f --- /dev/null +++ b/src/test/hive/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b @@ -0,0 +1,39 @@ +# 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/src/test/hive/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats2-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/stats2-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats2-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 b/src/test/hive/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 b/src/test/hive/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 b/src/test/hive/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats2-8-72621dba638b15d244850018e9f64d7 b/src/test/hive/golden/stats2-8-72621dba638b15d244850018e9f64d7 new file mode 100644 index 0000000000000..19a5e1e042ae7 --- /dev/null +++ b/src/test/hive/golden/stats2-8-72621dba638b15d244850018e9f64d7 @@ -0,0 +1,34 @@ +# 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/src/test/hive/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e b/src/test/hive/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e new file mode 100644 index 0000000000000..7ddea146475d1 --- /dev/null +++ b/src/test/hive/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e @@ -0,0 +1,19 @@ +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/src/test/hive/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 b/src/test/hive/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 b/src/test/hive/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b b/src/test/hive/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 b/src/test/hive/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats20-5-76509775cfe11bb98ee088188a07668a b/src/test/hive/golden/stats20-5-76509775cfe11bb98ee088188a07668a new file mode 100644 index 0000000000000..5c8878cba5e71 --- /dev/null +++ b/src/test/hive/golden/stats20-5-76509775cfe11bb98ee088188a07668a @@ -0,0 +1,37 @@ +# 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/src/test/hive/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 b/src/test/hive/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 b/src/test/hive/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats20-8-300c971de74642118d36d36349bc81aa b/src/test/hive/golden/stats20-8-300c971de74642118d36d36349bc81aa new file mode 100644 index 0000000000000..d9ba3814de367 --- /dev/null +++ b/src/test/hive/golden/stats20-8-300c971de74642118d36d36349bc81aa @@ -0,0 +1,37 @@ +# 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/src/test/hive/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 b/src/test/hive/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 b/src/test/hive/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 new file mode 100644 index 0000000000000..bf2a7f452edc3 --- /dev/null +++ b/src/test/hive/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/src/test/hive/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa new file mode 100644 index 0000000000000..1e8b314962144 --- /dev/null +++ b/src/test/hive/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa @@ -0,0 +1 @@ +6 diff --git a/src/test/hive/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc b/src/test/hive/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 b/src/test/hive/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/src/test/hive/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa new file mode 100644 index 0000000000000..1e8b314962144 --- /dev/null +++ b/src/test/hive/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa @@ -0,0 +1 @@ +6 diff --git a/src/test/hive/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 b/src/test/hive/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 new file mode 100644 index 0000000000000..bf2a7f452edc3 --- /dev/null +++ b/src/test/hive/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 b/src/test/hive/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 b/src/test/hive/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b b/src/test/hive/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b new file mode 100644 index 0000000000000..cb1c9def371df --- /dev/null +++ b/src/test/hive/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b @@ -0,0 +1,38 @@ +# 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/src/test/hive/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 b/src/test/hive/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 b/src/test/hive/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-20-ca048ad81b3df7159822073d206f0790 b/src/test/hive/golden/stats3-20-ca048ad81b3df7159822073d206f0790 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-3-ca048ad81b3df7159822073d206f0790 b/src/test/hive/golden/stats3-3-ca048ad81b3df7159822073d206f0790 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 b/src/test/hive/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 b/src/test/hive/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 new file mode 100644 index 0000000000000..8bc7c14574497 --- /dev/null +++ b/src/test/hive/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 @@ -0,0 +1,43 @@ +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/src/test/hive/golden/stats3-6-4bf1504274319c44d370b58092fe016c b/src/test/hive/golden/stats3-6-4bf1504274319c44d370b58092fe016c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-7-73d7d55d6e5a57aacce8618902904d b/src/test/hive/golden/stats3-7-73d7d55d6e5a57aacce8618902904d new file mode 100644 index 0000000000000..4916ef91b0866 --- /dev/null +++ b/src/test/hive/golden/stats3-7-73d7d55d6e5a57aacce8618902904d @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 b/src/test/hive/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 b/src/test/hive/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 b/src/test/hive/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 b/src/test/hive/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 b/src/test/hive/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 new file mode 100644 index 0000000000000..0a88e449f625a --- /dev/null +++ b/src/test/hive/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 @@ -0,0 +1,2 @@ +ds=2008-04-08/hr=11 +ds=2008-04-08/hr=12 diff --git a/src/test/hive/golden/stats4-12-30bc31441828a053d1a675b225a5d617 b/src/test/hive/golden/stats4-12-30bc31441828a053d1a675b225a5d617 new file mode 100644 index 0000000000000..1634adfc4d70d --- /dev/null +++ b/src/test/hive/golden/stats4-12-30bc31441828a053d1a675b225a5d617 @@ -0,0 +1,2 @@ +ds=2008-12-31/hr=11 +ds=2008-12-31/hr=12 diff --git a/src/test/hive/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 b/src/test/hive/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 new file mode 100644 index 0000000000000..d7a8f25b41301 --- /dev/null +++ b/src/test/hive/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 @@ -0,0 +1,1000 @@ +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/src/test/hive/golden/stats4-14-9c82167763a771c175c656786d545798 b/src/test/hive/golden/stats4-14-9c82167763a771c175c656786d545798 new file mode 100644 index 0000000000000..653516475da22 --- /dev/null +++ b/src/test/hive/golden/stats4-14-9c82167763a771c175c656786d545798 @@ -0,0 +1,1000 @@ +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/src/test/hive/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c b/src/test/hive/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c new file mode 100644 index 0000000000000..2ac44ee23f45c --- /dev/null +++ b/src/test/hive/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 b/src/test/hive/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 new file mode 100644 index 0000000000000..f0b07ca89fcb3 --- /dev/null +++ b/src/test/hive/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 b/src/test/hive/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 new file mode 100644 index 0000000000000..f94614124bcf6 --- /dev/null +++ b/src/test/hive/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 b/src/test/hive/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 new file mode 100644 index 0000000000000..f00fb5b3bd491 --- /dev/null +++ b/src/test/hive/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 b/src/test/hive/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 new file mode 100644 index 0000000000000..7335c8d32fa68 --- /dev/null +++ b/src/test/hive/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 @@ -0,0 +1,39 @@ +# 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/src/test/hive/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 b/src/test/hive/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/src/test/hive/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 @@ -0,0 +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 diff --git a/src/test/hive/golden/stats4-20-f63000f2c395b935199c9829964f98c1 b/src/test/hive/golden/stats4-20-f63000f2c395b935199c9829964f98c1 new file mode 100644 index 0000000000000..674cd9b0b64e5 --- /dev/null +++ b/src/test/hive/golden/stats4-20-f63000f2c395b935199c9829964f98c1 @@ -0,0 +1,39 @@ +# 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/src/test/hive/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd b/src/test/hive/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c b/src/test/hive/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd b/src/test/hive/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c b/src/test/hive/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d b/src/test/hive/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 b/src/test/hive/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats4-7-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/stats4-7-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats4-7-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f b/src/test/hive/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 b/src/test/hive/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats5-3-96d9aa9c32a081518604959dcfac42df b/src/test/hive/golden/stats5-3-96d9aa9c32a081518604959dcfac42df new file mode 100644 index 0000000000000..c9a75eb6639d4 --- /dev/null +++ b/src/test/hive/golden/stats5-3-96d9aa9c32a081518604959dcfac42df @@ -0,0 +1,19 @@ +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/src/test/hive/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 b/src/test/hive/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 b/src/test/hive/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 new file mode 100644 index 0000000000000..e622a0d3704a1 --- /dev/null +++ b/src/test/hive/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 @@ -0,0 +1,33 @@ +# 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/src/test/hive/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats6-10-653f748fa2d690502ba4fda407841a20 b/src/test/hive/golden/stats6-10-653f748fa2d690502ba4fda407841a20 new file mode 100644 index 0000000000000..bf589ab894312 --- /dev/null +++ b/src/test/hive/golden/stats6-10-653f748fa2d690502ba4fda407841a20 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats6-11-1c9f833953403596ad50fd32e513642c b/src/test/hive/golden/stats6-11-1c9f833953403596ad50fd32e513642c new file mode 100644 index 0000000000000..e6170a33d7de7 --- /dev/null +++ b/src/test/hive/golden/stats6-11-1c9f833953403596ad50fd32e513642c @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 b/src/test/hive/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 new file mode 100644 index 0000000000000..45f6b344b04ca --- /dev/null +++ b/src/test/hive/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 @@ -0,0 +1,39 @@ +# 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/src/test/hive/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats6-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/stats6-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats6-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats6-4-a88c476a632cd92f748967fadb242405 b/src/test/hive/golden/stats6-4-a88c476a632cd92f748967fadb242405 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc b/src/test/hive/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats6-6-8926095434b70c83bf88c70559d38dce b/src/test/hive/golden/stats6-6-8926095434b70c83bf88c70559d38dce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 b/src/test/hive/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 b/src/test/hive/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 new file mode 100644 index 0000000000000..9ff40db416cb9 --- /dev/null +++ b/src/test/hive/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d b/src/test/hive/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d new file mode 100644 index 0000000000000..5c13e8793cc76 --- /dev/null +++ b/src/test/hive/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 b/src/test/hive/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 new file mode 100644 index 0000000000000..bafa551db56e9 --- /dev/null +++ b/src/test/hive/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 @@ -0,0 +1,38 @@ +# 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/src/test/hive/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats7-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/stats7-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats7-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats7-4-a88c476a632cd92f748967fadb242405 b/src/test/hive/golden/stats7-4-a88c476a632cd92f748967fadb242405 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc b/src/test/hive/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 b/src/test/hive/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 new file mode 100644 index 0000000000000..d5341825cff70 --- /dev/null +++ b/src/test/hive/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 @@ -0,0 +1,18 @@ +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/src/test/hive/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 b/src/test/hive/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 b/src/test/hive/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 new file mode 100644 index 0000000000000..7085e9397e59b --- /dev/null +++ b/src/test/hive/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d b/src/test/hive/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d new file mode 100644 index 0000000000000..a3eb8c6466032 --- /dev/null +++ b/src/test/hive/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 b/src/test/hive/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 new file mode 100644 index 0000000000000..d1e0c6243244d --- /dev/null +++ b/src/test/hive/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 @@ -0,0 +1,19 @@ +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/src/test/hive/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 b/src/test/hive/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d b/src/test/hive/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d new file mode 100644 index 0000000000000..27ff9a687f5c6 --- /dev/null +++ b/src/test/hive/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d b/src/test/hive/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d new file mode 100644 index 0000000000000..ce340cced8844 --- /dev/null +++ b/src/test/hive/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d @@ -0,0 +1,19 @@ +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/src/test/hive/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 b/src/test/hive/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats8-15-653f748fa2d690502ba4fda407841a20 b/src/test/hive/golden/stats8-15-653f748fa2d690502ba4fda407841a20 new file mode 100644 index 0000000000000..649828a63174b --- /dev/null +++ b/src/test/hive/golden/stats8-15-653f748fa2d690502ba4fda407841a20 @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 b/src/test/hive/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 new file mode 100644 index 0000000000000..e4c8bf3e0005f --- /dev/null +++ b/src/test/hive/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 @@ -0,0 +1,19 @@ +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/src/test/hive/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 b/src/test/hive/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats8-18-1c9f833953403596ad50fd32e513642c b/src/test/hive/golden/stats8-18-1c9f833953403596ad50fd32e513642c new file mode 100644 index 0000000000000..74aeceec37398 --- /dev/null +++ b/src/test/hive/golden/stats8-18-1c9f833953403596ad50fd32e513642c @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 b/src/test/hive/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 new file mode 100644 index 0000000000000..25d9c86f74634 --- /dev/null +++ b/src/test/hive/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 @@ -0,0 +1,19 @@ +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/src/test/hive/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 b/src/test/hive/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 b/src/test/hive/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 new file mode 100644 index 0000000000000..5939c257f2f67 --- /dev/null +++ b/src/test/hive/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d b/src/test/hive/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d new file mode 100644 index 0000000000000..8dc22de8e0ee6 --- /dev/null +++ b/src/test/hive/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats8-23-653f748fa2d690502ba4fda407841a20 b/src/test/hive/golden/stats8-23-653f748fa2d690502ba4fda407841a20 new file mode 100644 index 0000000000000..a19f38cc6a364 --- /dev/null +++ b/src/test/hive/golden/stats8-23-653f748fa2d690502ba4fda407841a20 @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats8-24-1c9f833953403596ad50fd32e513642c b/src/test/hive/golden/stats8-24-1c9f833953403596ad50fd32e513642c new file mode 100644 index 0000000000000..aead24acd518c --- /dev/null +++ b/src/test/hive/golden/stats8-24-1c9f833953403596ad50fd32e513642c @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 b/src/test/hive/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 new file mode 100644 index 0000000000000..5f8bf1774f144 --- /dev/null +++ b/src/test/hive/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 @@ -0,0 +1,39 @@ +# 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/src/test/hive/golden/stats8-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/stats8-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats8-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats8-4-a88c476a632cd92f748967fadb242405 b/src/test/hive/golden/stats8-4-a88c476a632cd92f748967fadb242405 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc b/src/test/hive/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 b/src/test/hive/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 new file mode 100644 index 0000000000000..d30acbf86a295 --- /dev/null +++ b/src/test/hive/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 @@ -0,0 +1,19 @@ +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/src/test/hive/golden/stats8-7-8926095434b70c83bf88c70559d38dce b/src/test/hive/golden/stats8-7-8926095434b70c83bf88c70559d38dce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 b/src/test/hive/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 new file mode 100644 index 0000000000000..07a61fc1a8bd1 --- /dev/null +++ b/src/test/hive/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 b/src/test/hive/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 new file mode 100644 index 0000000000000..bd16e76d28d83 --- /dev/null +++ b/src/test/hive/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 @@ -0,0 +1,39 @@ +# 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/src/test/hive/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 b/src/test/hive/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/src/test/hive/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 b/src/test/hive/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb b/src/test/hive/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c b/src/test/hive/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 b/src/test/hive/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 b/src/test/hive/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c b/src/test/hive/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 b/src/test/hive/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 b/src/test/hive/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 b/src/test/hive/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 b/src/test/hive/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 b/src/test/hive/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c b/src/test/hive/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 b/src/test/hive/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 b/src/test/hive/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c b/src/test/hive/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 b/src/test/hive/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 b/src/test/hive/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 b/src/test/hive/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 b/src/test/hive/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 b/src/test/hive/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 b/src/test/hive/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 b/src/test/hive/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 b/src/test/hive/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 b/src/test/hive/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 b/src/test/hive/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 new file mode 100644 index 0000000000000..4475cb837eaa4 --- /dev/null +++ b/src/test/hive/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 @@ -0,0 +1,35 @@ +# col_name data_type comment + +key string None +value string None + +# Partition Information +# col_name data_type comment + +part string None + +# Detailed Partition Information +Partition Value: [1] +Database: default +Table: tmptable +CreateTime: Sun Jan 05 00:32:00 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2020775722466758355/tmptable/part=1 +Partition Parameters: + numFiles 1 + numRows 0 + rawDataSize 0 + totalSize 0 + transient_lastDdlTime 1388910720 + +# 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/src/test/hive/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d b/src/test/hive/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d new file mode 100644 index 0000000000000..5949a76d84a11 --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 b/src/test/hive/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 new file mode 100644 index 0000000000000..97f5929fd816a --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c b/src/test/hive/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c new file mode 100644 index 0000000000000..6d08ff47abc2c --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 b/src/test/hive/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 new file mode 100644 index 0000000000000..f441c8b483868 --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 @@ -0,0 +1,38 @@ +# 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/src/test/hive/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab b/src/test/hive/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 b/src/test/hive/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 b/src/test/hive/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 b/src/test/hive/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 new file mode 100644 index 0000000000000..f263eed117b10 --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 b/src/test/hive/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e b/src/test/hive/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e new file mode 100644 index 0000000000000..6a115f7e6e61a --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f b/src/test/hive/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f new file mode 100644 index 0000000000000..1d96413b915da --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 b/src/test/hive/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 new file mode 100644 index 0000000000000..4c4380e63b7f9 --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a b/src/test/hive/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a new file mode 100644 index 0000000000000..ba90d8d3d2477 --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 b/src/test/hive/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c b/src/test/hive/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc b/src/test/hive/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 b/src/test/hive/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 new file mode 100644 index 0000000000000..b671e68cc4c3b --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e b/src/test/hive/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae b/src/test/hive/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 b/src/test/hive/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 new file mode 100644 index 0000000000000..96499c0fb0f7e --- /dev/null +++ b/src/test/hive/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 b/src/test/hive/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 new file mode 100644 index 0000000000000..b5fc469438c83 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 b/src/test/hive/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef b/src/test/hive/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef new file mode 100644 index 0000000000000..cc58ef9026786 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef @@ -0,0 +1,14 @@ +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/src/test/hive/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 b/src/test/hive/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c b/src/test/hive/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c new file mode 100644 index 0000000000000..3243fe9fb497d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 b/src/test/hive/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 new file mode 100644 index 0000000000000..6accd64d06d69 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 b/src/test/hive/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf b/src/test/hive/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 b/src/test/hive/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 b/src/test/hive/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea b/src/test/hive/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 b/src/test/hive/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 b/src/test/hive/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f b/src/test/hive/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 b/src/test/hive/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 new file mode 100644 index 0000000000000..942b7cfe07235 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 b/src/test/hive/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef b/src/test/hive/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef new file mode 100644 index 0000000000000..9c704a6ef4126 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef @@ -0,0 +1,15 @@ +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/src/test/hive/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 b/src/test/hive/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c b/src/test/hive/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c new file mode 100644 index 0000000000000..feee75f095d0d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c @@ -0,0 +1,36 @@ +# 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/src/test/hive/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 b/src/test/hive/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 new file mode 100644 index 0000000000000..f39d366764c95 --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 b/src/test/hive/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf b/src/test/hive/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 b/src/test/hive/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 b/src/test/hive/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea b/src/test/hive/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca b/src/test/hive/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 b/src/test/hive/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 b/src/test/hive/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/src/test/hive/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 b/src/test/hive/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..1b79f38e25b24 --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 diff --git a/src/test/hive/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 b/src/test/hive/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c b/src/test/hive/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 b/src/test/hive/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..1b79f38e25b24 --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 diff --git a/src/test/hive/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 b/src/test/hive/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c b/src/test/hive/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 b/src/test/hive/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..1b79f38e25b24 --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 diff --git a/src/test/hive/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 b/src/test/hive/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 b/src/test/hive/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 b/src/test/hive/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 b/src/test/hive/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c b/src/test/hive/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 b/src/test/hive/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..1b79f38e25b24 --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 diff --git a/src/test/hive/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b b/src/test/hive/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c b/src/test/hive/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 b/src/test/hive/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 new file mode 100644 index 0000000000000..9bed96d855476 --- /dev/null +++ b/src/test/hive/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 @@ -0,0 +1 @@ +str_to_map(text, delimiter1, delimiter2) - Creates a map by parsing text diff --git a/src/test/hive/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c b/src/test/hive/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c new file mode 100644 index 0000000000000..1da11fbfa31d5 --- /dev/null +++ b/src/test/hive/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c @@ -0,0 +1,2 @@ +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/src/test/hive/golden/str_to_map-10-32997010bba305ec40812df254490730 b/src/test/hive/golden/str_to_map-10-32997010bba305ec40812df254490730 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a b/src/test/hive/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e b/src/test/hive/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e new file mode 100644 index 0000000000000..cd4d4a384163b --- /dev/null +++ b/src/test/hive/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e @@ -0,0 +1,3 @@ +444 +444 +444 diff --git a/src/test/hive/golden/str_to_map-13-32997010bba305ec40812df254490730 b/src/test/hive/golden/str_to_map-13-32997010bba305ec40812df254490730 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec b/src/test/hive/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 b/src/test/hive/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 new file mode 100644 index 0000000000000..e8183f05f5db6 --- /dev/null +++ b/src/test/hive/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/src/test/hive/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 b/src/test/hive/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda b/src/test/hive/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda new file mode 100644 index 0000000000000..62813f9d10491 --- /dev/null +++ b/src/test/hive/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda @@ -0,0 +1,3 @@ +{"b":"2","a":"1","c":"3"} +{"b":"2","a":"1","c":"3"} +{"b":"2","a":"1","c":"3"} diff --git a/src/test/hive/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf b/src/test/hive/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad b/src/test/hive/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad new file mode 100644 index 0000000000000..62813f9d10491 --- /dev/null +++ b/src/test/hive/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad @@ -0,0 +1,3 @@ +{"b":"2","a":"1","c":"3"} +{"b":"2","a":"1","c":"3"} +{"b":"2","a":"1","c":"3"} diff --git a/src/test/hive/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 b/src/test/hive/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd b/src/test/hive/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd new file mode 100644 index 0000000000000..e8183f05f5db6 --- /dev/null +++ b/src/test/hive/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/src/test/hive/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 b/src/test/hive/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 new file mode 100644 index 0000000000000..8938b37682023 --- /dev/null +++ b/src/test/hive/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 @@ -0,0 +1,500 @@ +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test +test \ No newline at end of file diff --git a/src/test/hive/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 b/src/test/hive/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/subq2-1-235919a7ddb574662158503b8052e7ca b/src/test/hive/golden/subq2-1-235919a7ddb574662158503b8052e7ca new file mode 100644 index 0000000000000..5b3f76a5f7797 --- /dev/null +++ b/src/test/hive/golden/subq2-1-235919a7ddb574662158503b8052e7ca @@ -0,0 +1,258 @@ +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/src/test/hive/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 b/src/test/hive/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e b/src/test/hive/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 b/src/test/hive/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 b/src/test/hive/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 new file mode 100644 index 0000000000000..92cca3b6f1e24 --- /dev/null +++ b/src/test/hive/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 @@ -0,0 +1,2 @@ +a int None +b string None \ No newline at end of file diff --git a/src/test/hive/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d b/src/test/hive/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 b/src/test/hive/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/src/test/hive/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 @@ -0,0 +1,500 @@ +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 +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 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +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 +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 +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 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 \ No newline at end of file diff --git a/src/test/hive/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 b/src/test/hive/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c b/src/test/hive/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 b/src/test/hive/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 b/src/test/hive/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 b/src/test/hive/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 b/src/test/hive/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 b/src/test/hive/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 b/src/test/hive/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 b/src/test/hive/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 b/src/test/hive/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b b/src/test/hive/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 b/src/test/hive/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 b/src/test/hive/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc b/src/test/hive/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 b/src/test/hive/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb b/src/test/hive/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 b/src/test/hive/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af b/src/test/hive/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d b/src/test/hive/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 b/src/test/hive/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f b/src/test/hive/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 b/src/test/hive/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-1-27ad2962fed131f51ba802596ba37278 b/src/test/hive/golden/touch-1-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-10-27ad2962fed131f51ba802596ba37278 b/src/test/hive/golden/touch-10-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-2-3e95421993ab28d18245ec2340f580a3 b/src/test/hive/golden/touch-2-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-3-c0c18ac884677231a41eea8d980d0451 b/src/test/hive/golden/touch-3-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a b/src/test/hive/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c b/src/test/hive/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 b/src/test/hive/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-7-3752320b12abae0d138148d56a27c5b1 b/src/test/hive/golden/touch-7-3752320b12abae0d138148d56a27c5b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-8-3562ab40242756452595cd7eae79b0ce b/src/test/hive/golden/touch-8-3562ab40242756452595cd7eae79b0ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 b/src/test/hive/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe b/src/test/hive/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe new file mode 100644 index 0000000000000..d23e05acf7ba5 --- /dev/null +++ b/src/test/hive/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe @@ -0,0 +1,500 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 \ No newline at end of file diff --git a/src/test/hive/golden/transform1-0-b6919fc48901e388c869c84ae0211102 b/src/test/hive/golden/transform1-0-b6919fc48901e388c869c84ae0211102 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 b/src/test/hive/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 b/src/test/hive/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 b/src/test/hive/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/transform1-4-65527bae8e73262255ef83082c6968f9 b/src/test/hive/golden/transform1-4-65527bae8e73262255ef83082c6968f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 b/src/test/hive/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f b/src/test/hive/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f new file mode 100644 index 0000000000000..c6f628b1a3eef --- /dev/null +++ b/src/test/hive/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f @@ -0,0 +1 @@ +[0,1,2] \ No newline at end of file diff --git a/src/test/hive/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 b/src/test/hive/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 b/src/test/hive/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 new file mode 100644 index 0000000000000..839efe2d57956 --- /dev/null +++ b/src/test/hive/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 @@ -0,0 +1,168 @@ +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +10 val_10 +10 val_10 +11 val_11 +11 val_11 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +17 val_17 +17 val_17 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +19 val_19 +19 val_19 +2 val_2 +2 val_2 +20 val_20 +20 val_20 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +27 val_27 +27 val_27 +28 val_28 +28 val_28 +30 val_30 +30 val_30 +33 val_33 +33 val_33 +34 val_34 +34 val_34 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +4 val_4 +4 val_4 +41 val_41 +41 val_41 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +43 val_43 +43 val_43 +44 val_44 +44 val_44 +47 val_47 +47 val_47 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +53 val_53 +53 val_53 +54 val_54 +54 val_54 +57 val_57 +57 val_57 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +64 val_64 +64 val_64 +65 val_65 +65 val_65 +66 val_66 +66 val_66 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +69 val_69 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +74 val_74 +74 val_74 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +77 val_77 +77 val_77 +78 val_78 +78 val_78 +8 val_8 +8 val_8 +80 val_80 +80 val_80 +82 val_82 +82 val_82 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +85 val_85 +85 val_85 +86 val_86 +86 val_86 +87 val_87 +87 val_87 +9 val_9 +9 val_9 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +92 val_92 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +96 val_96 +96 val_96 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +98 val_98 +98 val_98 \ No newline at end of file diff --git a/src/test/hive/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 b/src/test/hive/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 b/src/test/hive/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 b/src/test/hive/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 new file mode 100644 index 0000000000000..839efe2d57956 --- /dev/null +++ b/src/test/hive/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 @@ -0,0 +1,168 @@ +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +10 val_10 +10 val_10 +11 val_11 +11 val_11 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +17 val_17 +17 val_17 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +19 val_19 +19 val_19 +2 val_2 +2 val_2 +20 val_20 +20 val_20 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +27 val_27 +27 val_27 +28 val_28 +28 val_28 +30 val_30 +30 val_30 +33 val_33 +33 val_33 +34 val_34 +34 val_34 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +4 val_4 +4 val_4 +41 val_41 +41 val_41 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +43 val_43 +43 val_43 +44 val_44 +44 val_44 +47 val_47 +47 val_47 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +53 val_53 +53 val_53 +54 val_54 +54 val_54 +57 val_57 +57 val_57 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +64 val_64 +64 val_64 +65 val_65 +65 val_65 +66 val_66 +66 val_66 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +69 val_69 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +74 val_74 +74 val_74 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +77 val_77 +77 val_77 +78 val_78 +78 val_78 +8 val_8 +8 val_8 +80 val_80 +80 val_80 +82 val_82 +82 val_82 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +85 val_85 +85 val_85 +86 val_86 +86 val_86 +87 val_87 +87 val_87 +9 val_9 +9 val_9 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +92 val_92 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +96 val_96 +96 val_96 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +98 val_98 +98 val_98 \ No newline at end of file diff --git a/src/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d b/src/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d new file mode 100644 index 0000000000000..66fafbdf72b63 --- /dev/null +++ b/src/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d @@ -0,0 +1,1028 @@ +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 +409 val_409 409 val_409 +409 val_409 409 val_409 +255 val_255 255 val_255 +255 val_255 255 val_255 +278 val_278 278 val_278 +278 val_278 278 val_278 +98 val_98 98 val_98 +98 val_98 98 val_98 +484 val_484 484 val_484 +265 val_265 265 val_265 +265 val_265 265 val_265 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +150 val_150 150 val_150 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +224 val_224 224 val_224 +224 val_224 224 val_224 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +66 val_66 66 val_66 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +213 val_213 213 val_213 +213 val_213 213 val_213 +146 val_146 146 val_146 +146 val_146 146 val_146 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +429 val_429 429 val_429 +429 val_429 429 val_429 +374 val_374 374 val_374 +152 val_152 152 val_152 +152 val_152 152 val_152 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +145 val_145 145 val_145 +495 val_495 495 val_495 +37 val_37 37 val_37 +37 val_37 37 val_37 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +281 val_281 281 val_281 +281 val_281 281 val_281 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +209 val_209 209 val_209 +209 val_209 209 val_209 +15 val_15 15 val_15 +15 val_15 15 val_15 +82 val_82 82 val_82 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +166 val_166 166 val_166 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +252 val_252 252 val_252 +292 val_292 292 val_292 +219 val_219 219 val_219 +219 val_219 219 val_219 +287 val_287 287 val_287 +153 val_153 153 val_153 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +338 val_338 338 val_338 +446 val_446 446 val_446 +459 val_459 459 val_459 +459 val_459 459 val_459 +394 val_394 394 val_394 +237 val_237 237 val_237 +237 val_237 237 val_237 +482 val_482 482 val_482 +174 val_174 174 val_174 +174 val_174 174 val_174 +413 val_413 413 val_413 +413 val_413 413 val_413 +494 val_494 494 val_494 +207 val_207 207 val_207 +207 val_207 207 val_207 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +174 val_174 174 val_174 +174 val_174 174 val_174 +399 val_399 399 val_399 +399 val_399 399 val_399 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +247 val_247 247 val_247 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +162 val_162 162 val_162 +377 val_377 377 val_377 +397 val_397 397 val_397 +397 val_397 397 val_397 +309 val_309 309 val_309 +309 val_309 309 val_309 +365 val_365 365 val_365 +266 val_266 266 val_266 +439 val_439 439 val_439 +439 val_439 439 val_439 +342 val_342 342 val_342 +342 val_342 342 val_342 +367 val_367 367 val_367 +367 val_367 367 val_367 +325 val_325 325 val_325 +325 val_325 325 val_325 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +195 val_195 195 val_195 +195 val_195 195 val_195 +475 val_475 475 val_475 +17 val_17 17 val_17 +113 val_113 113 val_113 +113 val_113 113 val_113 +155 val_155 155 val_155 +203 val_203 203 val_203 +203 val_203 203 val_203 +339 val_339 339 val_339 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +455 val_455 455 val_455 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +57 val_57 57 val_57 +302 val_302 302 val_302 +205 val_205 205 val_205 +205 val_205 205 val_205 +149 val_149 149 val_149 +149 val_149 149 val_149 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +345 val_345 345 val_345 +129 val_129 129 val_129 +129 val_129 129 val_129 +170 val_170 170 val_170 +20 val_20 20 val_20 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +157 val_157 157 val_157 +378 val_378 378 val_378 +221 val_221 221 val_221 +221 val_221 221 val_221 +92 val_92 92 val_92 +111 val_111 111 val_111 +47 val_47 47 val_47 +72 val_72 72 val_72 +72 val_72 72 val_72 +4 val_4 4 val_4 +280 val_280 280 val_280 +280 val_280 280 val_280 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +427 val_427 427 val_427 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +356 val_356 356 val_356 +399 val_399 399 val_399 +399 val_399 399 val_399 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +382 val_382 382 val_382 +382 val_382 382 val_382 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +125 val_125 125 val_125 +125 val_125 125 val_125 +386 val_386 386 val_386 +437 val_437 437 val_437 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +192 val_192 192 val_192 +286 val_286 286 val_286 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +176 val_176 176 val_176 +176 val_176 176 val_176 +54 val_54 54 val_54 +459 val_459 459 val_459 +459 val_459 459 val_459 +51 val_51 51 val_51 +51 val_51 51 val_51 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +103 val_103 103 val_103 +103 val_103 103 val_103 +239 val_239 239 val_239 +239 val_239 239 val_239 +213 val_213 213 val_213 +213 val_213 213 val_213 +216 val_216 216 val_216 +216 val_216 216 val_216 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +278 val_278 278 val_278 +278 val_278 278 val_278 +176 val_176 176 val_176 +176 val_176 176 val_176 +289 val_289 289 val_289 +221 val_221 221 val_221 +221 val_221 221 val_221 +65 val_65 65 val_65 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +332 val_332 332 val_332 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +275 val_275 275 val_275 +137 val_137 137 val_137 +137 val_137 137 val_137 +241 val_241 241 val_241 +83 val_83 83 val_83 +83 val_83 83 val_83 +333 val_333 333 val_333 +333 val_333 333 val_333 +180 val_180 180 val_180 +284 val_284 284 val_284 +12 val_12 12 val_12 +12 val_12 12 val_12 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +181 val_181 181 val_181 +67 val_67 67 val_67 +67 val_67 67 val_67 +260 val_260 260 val_260 +404 val_404 404 val_404 +404 val_404 404 val_404 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +353 val_353 353 val_353 +353 val_353 353 val_353 +373 val_373 373 val_373 +272 val_272 272 val_272 +272 val_272 272 val_272 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +217 val_217 217 val_217 +217 val_217 217 val_217 +84 val_84 84 val_84 +84 val_84 84 val_84 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +58 val_58 58 val_58 +58 val_58 58 val_58 +8 val_8 8 val_8 +411 val_411 411 val_411 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +24 val_24 24 val_24 +24 val_24 24 val_24 +463 val_463 463 val_463 +463 val_463 463 val_463 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +179 val_179 179 val_179 +179 val_179 179 val_179 +172 val_172 172 val_172 +172 val_172 172 val_172 +42 val_42 42 val_42 +42 val_42 42 val_42 +129 val_129 129 val_129 +129 val_129 129 val_129 +158 val_158 158 val_158 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +496 val_496 496 val_496 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +322 val_322 322 val_322 +322 val_322 322 val_322 +197 val_197 197 val_197 +197 val_197 197 val_197 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +393 val_393 393 val_393 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +100 val_100 100 val_100 +100 val_100 100 val_100 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +191 val_191 191 val_191 +191 val_191 191 val_191 +418 val_418 418 val_418 +96 val_96 96 val_96 +26 val_26 26 val_26 +26 val_26 26 val_26 +165 val_165 165 val_165 +165 val_165 165 val_165 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +205 val_205 205 val_205 +205 val_205 205 val_205 +120 val_120 120 val_120 +120 val_120 120 val_120 +131 val_131 131 val_131 +51 val_51 51 val_51 +51 val_51 51 val_51 +404 val_404 404 val_404 +404 val_404 404 val_404 +43 val_43 43 val_43 +436 val_436 436 val_436 +156 val_156 156 val_156 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +308 val_308 308 val_308 +95 val_95 95 val_95 +95 val_95 95 val_95 +196 val_196 196 val_196 +288 val_288 288 val_288 +288 val_288 288 val_288 +481 val_481 481 val_481 +457 val_457 457 val_457 +98 val_98 98 val_98 +98 val_98 98 val_98 +282 val_282 282 val_282 +282 val_282 282 val_282 +197 val_197 197 val_197 +197 val_197 197 val_197 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +470 val_470 470 val_470 +137 val_137 137 val_137 +137 val_137 137 val_137 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +413 val_413 413 val_413 +413 val_413 413 val_413 +85 val_85 85 val_85 +77 val_77 77 val_77 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +490 val_490 490 val_490 +87 val_87 87 val_87 +364 val_364 364 val_364 +179 val_179 179 val_179 +179 val_179 179 val_179 +118 val_118 118 val_118 +118 val_118 118 val_118 +134 val_134 134 val_134 +134 val_134 134 val_134 +395 val_395 395 val_395 +395 val_395 395 val_395 +282 val_282 282 val_282 +282 val_282 282 val_282 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +238 val_238 238 val_238 +238 val_238 238 val_238 +419 val_419 419 val_419 +15 val_15 15 val_15 +15 val_15 15 val_15 +118 val_118 118 val_118 +118 val_118 118 val_118 +72 val_72 72 val_72 +72 val_72 72 val_72 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +307 val_307 307 val_307 +307 val_307 307 val_307 +19 val_19 19 val_19 +435 val_435 435 val_435 +10 val_10 10 val_10 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +306 val_306 306 val_306 +224 val_224 224 val_224 +224 val_224 224 val_224 +309 val_309 309 val_309 +309 val_309 309 val_309 +389 val_389 389 val_389 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +242 val_242 242 val_242 +242 val_242 242 val_242 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +392 val_392 392 val_392 +272 val_272 272 val_272 +272 val_272 272 val_272 +331 val_331 331 val_331 +331 val_331 331 val_331 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +242 val_242 242 val_242 +242 val_242 242 val_242 +452 val_452 452 val_452 +177 val_177 177 val_177 +226 val_226 226 val_226 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +497 val_497 497 val_497 +402 val_402 402 val_402 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +317 val_317 317 val_317 +317 val_317 317 val_317 +395 val_395 395 val_395 +395 val_395 395 val_395 +58 val_58 58 val_58 +58 val_58 58 val_58 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +336 val_336 336 val_336 +95 val_95 95 val_95 +95 val_95 95 val_95 +11 val_11 11 val_11 +168 val_168 168 val_168 +34 val_34 34 val_34 +229 val_229 229 val_229 +229 val_229 229 val_229 +233 val_233 233 val_233 +233 val_233 233 val_233 +143 val_143 143 val_143 +472 val_472 472 val_472 +322 val_322 322 val_322 +322 val_322 322 val_322 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +160 val_160 160 val_160 +195 val_195 195 val_195 +195 val_195 195 val_195 +42 val_42 42 val_42 +42 val_42 42 val_42 +321 val_321 321 val_321 +321 val_321 321 val_321 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +458 val_458 458 val_458 +458 val_458 458 val_458 +78 val_78 78 val_78 +76 val_76 76 val_76 +76 val_76 76 val_76 +41 val_41 41 val_41 +223 val_223 223 val_223 +223 val_223 223 val_223 +492 val_492 492 val_492 +492 val_492 492 val_492 +149 val_149 149 val_149 +149 val_149 149 val_149 +449 val_449 449 val_449 +218 val_218 218 val_218 +228 val_228 228 val_228 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +453 val_453 453 val_453 +30 val_30 30 val_30 +209 val_209 209 val_209 +209 val_209 209 val_209 +64 val_64 64 val_64 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +76 val_76 76 val_76 +76 val_76 76 val_76 +74 val_74 74 val_74 +342 val_342 342 val_342 +342 val_342 342 val_342 +69 val_69 69 val_69 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +33 val_33 33 val_33 +368 val_368 368 val_368 +103 val_103 103 val_103 +103 val_103 103 val_103 +296 val_296 296 val_296 +113 val_113 113 val_113 +113 val_113 113 val_113 +216 val_216 216 val_216 +216 val_216 216 val_216 +367 val_367 367 val_367 +367 val_367 367 val_367 +344 val_344 344 val_344 +344 val_344 344 val_344 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +274 val_274 274 val_274 +219 val_219 219 val_219 +219 val_219 219 val_219 +239 val_239 239 val_239 +239 val_239 239 val_239 +485 val_485 485 val_485 +116 val_116 116 val_116 +223 val_223 223 val_223 +223 val_223 223 val_223 +256 val_256 256 val_256 +256 val_256 256 val_256 +263 val_263 263 val_263 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +487 val_487 487 val_487 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +288 val_288 288 val_288 +288 val_288 288 val_288 +191 val_191 191 val_191 +191 val_191 191 val_191 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +244 val_244 244 val_244 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +467 val_467 467 val_467 +432 val_432 432 val_432 +202 val_202 202 val_202 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +229 val_229 229 val_229 +229 val_229 229 val_229 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +463 val_463 463 val_463 +463 val_463 463 val_463 +280 val_280 280 val_280 +280 val_280 280 val_280 +2 val_2 2 val_2 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +283 val_283 283 val_283 +331 val_331 331 val_331 +331 val_331 331 val_331 +235 val_235 235 val_235 +80 val_80 80 val_80 +44 val_44 44 val_44 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +321 val_321 321 val_321 +321 val_321 321 val_321 +335 val_335 335 val_335 +104 val_104 104 val_104 +104 val_104 104 val_104 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +366 val_366 366 val_366 +175 val_175 175 val_175 +175 val_175 175 val_175 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +483 val_483 483 val_483 +53 val_53 53 val_53 +105 val_105 105 val_105 +257 val_257 257 val_257 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +190 val_190 190 val_190 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +114 val_114 114 val_114 +258 val_258 258 val_258 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +203 val_203 203 val_203 +203 val_203 203 val_203 +262 val_262 262 val_262 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +424 val_424 424 val_424 +424 val_424 424 val_424 +12 val_12 12 val_12 +12 val_12 12 val_12 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +201 val_201 201 val_201 +217 val_217 217 val_217 +217 val_217 217 val_217 +164 val_164 164 val_164 +164 val_164 164 val_164 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +478 val_478 478 val_478 +478 val_478 478 val_478 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +125 val_125 125 val_125 +125 val_125 125 val_125 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +164 val_164 164 val_164 +164 val_164 164 val_164 +424 val_424 424 val_424 +424 val_424 424 val_424 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +382 val_382 382 val_382 +382 val_382 382 val_382 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +397 val_397 397 val_397 +397 val_397 397 val_397 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +291 val_291 291 val_291 +24 val_24 24 val_24 +24 val_24 24 val_24 +351 val_351 351 val_351 +255 val_255 255 val_255 +255 val_255 255 val_255 +104 val_104 104 val_104 +104 val_104 104 val_104 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +163 val_163 163 val_163 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +414 val_414 414 val_414 +414 val_414 414 val_414 +200 val_200 200 val_200 +200 val_200 200 val_200 +491 val_491 491 val_491 +237 val_237 237 val_237 +237 val_237 237 val_237 +439 val_439 439 val_439 +439 val_439 439 val_439 +360 val_360 360 val_360 +248 val_248 248 val_248 +479 val_479 479 val_479 +305 val_305 305 val_305 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +444 val_444 444 val_444 +120 val_120 120 val_120 +120 val_120 120 val_120 +429 val_429 429 val_429 +429 val_429 429 val_429 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +443 val_443 443 val_443 +323 val_323 323 val_323 +325 val_325 325 val_325 +325 val_325 325 val_325 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +478 val_478 478 val_478 +478 val_478 478 val_478 +178 val_178 178 val_178 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +310 val_310 310 val_310 +317 val_317 317 val_317 +317 val_317 317 val_317 +333 val_333 333 val_333 +333 val_333 333 val_333 +493 val_493 493 val_493 +460 val_460 460 val_460 +207 val_207 207 val_207 +207 val_207 207 val_207 +249 val_249 249 val_249 +265 val_265 265 val_265 +265 val_265 265 val_265 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +83 val_83 83 val_83 +83 val_83 83 val_83 +136 val_136 136 val_136 +353 val_353 353 val_353 +353 val_353 353 val_353 +172 val_172 172 val_172 +172 val_172 172 val_172 +214 val_214 214 val_214 +462 val_462 462 val_462 +462 val_462 462 val_462 +233 val_233 233 val_233 +233 val_233 233 val_233 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +133 val_133 133 val_133 +175 val_175 175 val_175 +175 val_175 175 val_175 +189 val_189 189 val_189 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +375 val_375 375 val_375 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +421 val_421 421 val_421 +407 val_407 407 val_407 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +256 val_256 256 val_256 +256 val_256 256 val_256 +26 val_26 26 val_26 +26 val_26 26 val_26 +134 val_134 134 val_134 +134 val_134 134 val_134 +67 val_67 67 val_67 +67 val_67 67 val_67 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +379 val_379 379 val_379 +18 val_18 18 val_18 +18 val_18 18 val_18 +462 val_462 462 val_462 +462 val_462 462 val_462 +492 val_492 492 val_492 +492 val_492 492 val_492 +100 val_100 100 val_100 +100 val_100 100 val_100 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +9 val_9 9 val_9 +341 val_341 341 val_341 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +146 val_146 146 val_146 +146 val_146 146 val_146 +458 val_458 458 val_458 +458 val_458 458 val_458 +362 val_362 362 val_362 +186 val_186 186 val_186 +285 val_285 285 val_285 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +18 val_18 18 val_18 +18 val_18 18 val_18 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +183 val_183 183 val_183 +281 val_281 281 val_281 +281 val_281 281 val_281 +344 val_344 344 val_344 +344 val_344 344 val_344 +97 val_97 97 val_97 +97 val_97 97 val_97 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +315 val_315 315 val_315 +84 val_84 84 val_84 +84 val_84 84 val_84 +28 val_28 28 val_28 +37 val_37 37 val_37 +37 val_37 37 val_37 +448 val_448 448 val_448 +152 val_152 152 val_152 +152 val_152 152 val_152 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +307 val_307 307 val_307 +307 val_307 307 val_307 +194 val_194 194 val_194 +414 val_414 414 val_414 +414 val_414 414 val_414 +477 val_477 477 val_477 +222 val_222 222 val_222 +126 val_126 126 val_126 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +400 val_400 400 val_400 +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 diff --git a/src/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 b/src/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 new file mode 100644 index 0000000000000..66fafbdf72b63 --- /dev/null +++ b/src/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 @@ -0,0 +1,1028 @@ +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 +409 val_409 409 val_409 +409 val_409 409 val_409 +255 val_255 255 val_255 +255 val_255 255 val_255 +278 val_278 278 val_278 +278 val_278 278 val_278 +98 val_98 98 val_98 +98 val_98 98 val_98 +484 val_484 484 val_484 +265 val_265 265 val_265 +265 val_265 265 val_265 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +150 val_150 150 val_150 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +224 val_224 224 val_224 +224 val_224 224 val_224 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +66 val_66 66 val_66 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +213 val_213 213 val_213 +213 val_213 213 val_213 +146 val_146 146 val_146 +146 val_146 146 val_146 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +429 val_429 429 val_429 +429 val_429 429 val_429 +374 val_374 374 val_374 +152 val_152 152 val_152 +152 val_152 152 val_152 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +145 val_145 145 val_145 +495 val_495 495 val_495 +37 val_37 37 val_37 +37 val_37 37 val_37 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +281 val_281 281 val_281 +281 val_281 281 val_281 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +209 val_209 209 val_209 +209 val_209 209 val_209 +15 val_15 15 val_15 +15 val_15 15 val_15 +82 val_82 82 val_82 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +166 val_166 166 val_166 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +252 val_252 252 val_252 +292 val_292 292 val_292 +219 val_219 219 val_219 +219 val_219 219 val_219 +287 val_287 287 val_287 +153 val_153 153 val_153 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +338 val_338 338 val_338 +446 val_446 446 val_446 +459 val_459 459 val_459 +459 val_459 459 val_459 +394 val_394 394 val_394 +237 val_237 237 val_237 +237 val_237 237 val_237 +482 val_482 482 val_482 +174 val_174 174 val_174 +174 val_174 174 val_174 +413 val_413 413 val_413 +413 val_413 413 val_413 +494 val_494 494 val_494 +207 val_207 207 val_207 +207 val_207 207 val_207 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +174 val_174 174 val_174 +174 val_174 174 val_174 +399 val_399 399 val_399 +399 val_399 399 val_399 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +247 val_247 247 val_247 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +162 val_162 162 val_162 +377 val_377 377 val_377 +397 val_397 397 val_397 +397 val_397 397 val_397 +309 val_309 309 val_309 +309 val_309 309 val_309 +365 val_365 365 val_365 +266 val_266 266 val_266 +439 val_439 439 val_439 +439 val_439 439 val_439 +342 val_342 342 val_342 +342 val_342 342 val_342 +367 val_367 367 val_367 +367 val_367 367 val_367 +325 val_325 325 val_325 +325 val_325 325 val_325 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +195 val_195 195 val_195 +195 val_195 195 val_195 +475 val_475 475 val_475 +17 val_17 17 val_17 +113 val_113 113 val_113 +113 val_113 113 val_113 +155 val_155 155 val_155 +203 val_203 203 val_203 +203 val_203 203 val_203 +339 val_339 339 val_339 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +455 val_455 455 val_455 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +57 val_57 57 val_57 +302 val_302 302 val_302 +205 val_205 205 val_205 +205 val_205 205 val_205 +149 val_149 149 val_149 +149 val_149 149 val_149 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +345 val_345 345 val_345 +129 val_129 129 val_129 +129 val_129 129 val_129 +170 val_170 170 val_170 +20 val_20 20 val_20 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +157 val_157 157 val_157 +378 val_378 378 val_378 +221 val_221 221 val_221 +221 val_221 221 val_221 +92 val_92 92 val_92 +111 val_111 111 val_111 +47 val_47 47 val_47 +72 val_72 72 val_72 +72 val_72 72 val_72 +4 val_4 4 val_4 +280 val_280 280 val_280 +280 val_280 280 val_280 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +427 val_427 427 val_427 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +356 val_356 356 val_356 +399 val_399 399 val_399 +399 val_399 399 val_399 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +382 val_382 382 val_382 +382 val_382 382 val_382 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +125 val_125 125 val_125 +125 val_125 125 val_125 +386 val_386 386 val_386 +437 val_437 437 val_437 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +192 val_192 192 val_192 +286 val_286 286 val_286 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +176 val_176 176 val_176 +176 val_176 176 val_176 +54 val_54 54 val_54 +459 val_459 459 val_459 +459 val_459 459 val_459 +51 val_51 51 val_51 +51 val_51 51 val_51 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +103 val_103 103 val_103 +103 val_103 103 val_103 +239 val_239 239 val_239 +239 val_239 239 val_239 +213 val_213 213 val_213 +213 val_213 213 val_213 +216 val_216 216 val_216 +216 val_216 216 val_216 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +278 val_278 278 val_278 +278 val_278 278 val_278 +176 val_176 176 val_176 +176 val_176 176 val_176 +289 val_289 289 val_289 +221 val_221 221 val_221 +221 val_221 221 val_221 +65 val_65 65 val_65 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +332 val_332 332 val_332 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +275 val_275 275 val_275 +137 val_137 137 val_137 +137 val_137 137 val_137 +241 val_241 241 val_241 +83 val_83 83 val_83 +83 val_83 83 val_83 +333 val_333 333 val_333 +333 val_333 333 val_333 +180 val_180 180 val_180 +284 val_284 284 val_284 +12 val_12 12 val_12 +12 val_12 12 val_12 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +181 val_181 181 val_181 +67 val_67 67 val_67 +67 val_67 67 val_67 +260 val_260 260 val_260 +404 val_404 404 val_404 +404 val_404 404 val_404 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +353 val_353 353 val_353 +353 val_353 353 val_353 +373 val_373 373 val_373 +272 val_272 272 val_272 +272 val_272 272 val_272 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +217 val_217 217 val_217 +217 val_217 217 val_217 +84 val_84 84 val_84 +84 val_84 84 val_84 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +58 val_58 58 val_58 +58 val_58 58 val_58 +8 val_8 8 val_8 +411 val_411 411 val_411 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +24 val_24 24 val_24 +24 val_24 24 val_24 +463 val_463 463 val_463 +463 val_463 463 val_463 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +179 val_179 179 val_179 +179 val_179 179 val_179 +172 val_172 172 val_172 +172 val_172 172 val_172 +42 val_42 42 val_42 +42 val_42 42 val_42 +129 val_129 129 val_129 +129 val_129 129 val_129 +158 val_158 158 val_158 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +496 val_496 496 val_496 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +322 val_322 322 val_322 +322 val_322 322 val_322 +197 val_197 197 val_197 +197 val_197 197 val_197 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +393 val_393 393 val_393 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +100 val_100 100 val_100 +100 val_100 100 val_100 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +191 val_191 191 val_191 +191 val_191 191 val_191 +418 val_418 418 val_418 +96 val_96 96 val_96 +26 val_26 26 val_26 +26 val_26 26 val_26 +165 val_165 165 val_165 +165 val_165 165 val_165 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +205 val_205 205 val_205 +205 val_205 205 val_205 +120 val_120 120 val_120 +120 val_120 120 val_120 +131 val_131 131 val_131 +51 val_51 51 val_51 +51 val_51 51 val_51 +404 val_404 404 val_404 +404 val_404 404 val_404 +43 val_43 43 val_43 +436 val_436 436 val_436 +156 val_156 156 val_156 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +308 val_308 308 val_308 +95 val_95 95 val_95 +95 val_95 95 val_95 +196 val_196 196 val_196 +288 val_288 288 val_288 +288 val_288 288 val_288 +481 val_481 481 val_481 +457 val_457 457 val_457 +98 val_98 98 val_98 +98 val_98 98 val_98 +282 val_282 282 val_282 +282 val_282 282 val_282 +197 val_197 197 val_197 +197 val_197 197 val_197 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +470 val_470 470 val_470 +137 val_137 137 val_137 +137 val_137 137 val_137 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +413 val_413 413 val_413 +413 val_413 413 val_413 +85 val_85 85 val_85 +77 val_77 77 val_77 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +490 val_490 490 val_490 +87 val_87 87 val_87 +364 val_364 364 val_364 +179 val_179 179 val_179 +179 val_179 179 val_179 +118 val_118 118 val_118 +118 val_118 118 val_118 +134 val_134 134 val_134 +134 val_134 134 val_134 +395 val_395 395 val_395 +395 val_395 395 val_395 +282 val_282 282 val_282 +282 val_282 282 val_282 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +238 val_238 238 val_238 +238 val_238 238 val_238 +419 val_419 419 val_419 +15 val_15 15 val_15 +15 val_15 15 val_15 +118 val_118 118 val_118 +118 val_118 118 val_118 +72 val_72 72 val_72 +72 val_72 72 val_72 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +307 val_307 307 val_307 +307 val_307 307 val_307 +19 val_19 19 val_19 +435 val_435 435 val_435 +10 val_10 10 val_10 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +306 val_306 306 val_306 +224 val_224 224 val_224 +224 val_224 224 val_224 +309 val_309 309 val_309 +309 val_309 309 val_309 +389 val_389 389 val_389 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +242 val_242 242 val_242 +242 val_242 242 val_242 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +392 val_392 392 val_392 +272 val_272 272 val_272 +272 val_272 272 val_272 +331 val_331 331 val_331 +331 val_331 331 val_331 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +242 val_242 242 val_242 +242 val_242 242 val_242 +452 val_452 452 val_452 +177 val_177 177 val_177 +226 val_226 226 val_226 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +497 val_497 497 val_497 +402 val_402 402 val_402 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +317 val_317 317 val_317 +317 val_317 317 val_317 +395 val_395 395 val_395 +395 val_395 395 val_395 +58 val_58 58 val_58 +58 val_58 58 val_58 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +336 val_336 336 val_336 +95 val_95 95 val_95 +95 val_95 95 val_95 +11 val_11 11 val_11 +168 val_168 168 val_168 +34 val_34 34 val_34 +229 val_229 229 val_229 +229 val_229 229 val_229 +233 val_233 233 val_233 +233 val_233 233 val_233 +143 val_143 143 val_143 +472 val_472 472 val_472 +322 val_322 322 val_322 +322 val_322 322 val_322 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +160 val_160 160 val_160 +195 val_195 195 val_195 +195 val_195 195 val_195 +42 val_42 42 val_42 +42 val_42 42 val_42 +321 val_321 321 val_321 +321 val_321 321 val_321 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +458 val_458 458 val_458 +458 val_458 458 val_458 +78 val_78 78 val_78 +76 val_76 76 val_76 +76 val_76 76 val_76 +41 val_41 41 val_41 +223 val_223 223 val_223 +223 val_223 223 val_223 +492 val_492 492 val_492 +492 val_492 492 val_492 +149 val_149 149 val_149 +149 val_149 149 val_149 +449 val_449 449 val_449 +218 val_218 218 val_218 +228 val_228 228 val_228 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +453 val_453 453 val_453 +30 val_30 30 val_30 +209 val_209 209 val_209 +209 val_209 209 val_209 +64 val_64 64 val_64 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +76 val_76 76 val_76 +76 val_76 76 val_76 +74 val_74 74 val_74 +342 val_342 342 val_342 +342 val_342 342 val_342 +69 val_69 69 val_69 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +33 val_33 33 val_33 +368 val_368 368 val_368 +103 val_103 103 val_103 +103 val_103 103 val_103 +296 val_296 296 val_296 +113 val_113 113 val_113 +113 val_113 113 val_113 +216 val_216 216 val_216 +216 val_216 216 val_216 +367 val_367 367 val_367 +367 val_367 367 val_367 +344 val_344 344 val_344 +344 val_344 344 val_344 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +274 val_274 274 val_274 +219 val_219 219 val_219 +219 val_219 219 val_219 +239 val_239 239 val_239 +239 val_239 239 val_239 +485 val_485 485 val_485 +116 val_116 116 val_116 +223 val_223 223 val_223 +223 val_223 223 val_223 +256 val_256 256 val_256 +256 val_256 256 val_256 +263 val_263 263 val_263 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +487 val_487 487 val_487 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +288 val_288 288 val_288 +288 val_288 288 val_288 +191 val_191 191 val_191 +191 val_191 191 val_191 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +244 val_244 244 val_244 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +467 val_467 467 val_467 +432 val_432 432 val_432 +202 val_202 202 val_202 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +229 val_229 229 val_229 +229 val_229 229 val_229 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +463 val_463 463 val_463 +463 val_463 463 val_463 +280 val_280 280 val_280 +280 val_280 280 val_280 +2 val_2 2 val_2 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +283 val_283 283 val_283 +331 val_331 331 val_331 +331 val_331 331 val_331 +235 val_235 235 val_235 +80 val_80 80 val_80 +44 val_44 44 val_44 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +321 val_321 321 val_321 +321 val_321 321 val_321 +335 val_335 335 val_335 +104 val_104 104 val_104 +104 val_104 104 val_104 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +366 val_366 366 val_366 +175 val_175 175 val_175 +175 val_175 175 val_175 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +483 val_483 483 val_483 +53 val_53 53 val_53 +105 val_105 105 val_105 +257 val_257 257 val_257 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +190 val_190 190 val_190 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +114 val_114 114 val_114 +258 val_258 258 val_258 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +203 val_203 203 val_203 +203 val_203 203 val_203 +262 val_262 262 val_262 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +424 val_424 424 val_424 +424 val_424 424 val_424 +12 val_12 12 val_12 +12 val_12 12 val_12 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +201 val_201 201 val_201 +217 val_217 217 val_217 +217 val_217 217 val_217 +164 val_164 164 val_164 +164 val_164 164 val_164 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +478 val_478 478 val_478 +478 val_478 478 val_478 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +125 val_125 125 val_125 +125 val_125 125 val_125 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +164 val_164 164 val_164 +164 val_164 164 val_164 +424 val_424 424 val_424 +424 val_424 424 val_424 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +382 val_382 382 val_382 +382 val_382 382 val_382 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +397 val_397 397 val_397 +397 val_397 397 val_397 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +291 val_291 291 val_291 +24 val_24 24 val_24 +24 val_24 24 val_24 +351 val_351 351 val_351 +255 val_255 255 val_255 +255 val_255 255 val_255 +104 val_104 104 val_104 +104 val_104 104 val_104 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +163 val_163 163 val_163 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +414 val_414 414 val_414 +414 val_414 414 val_414 +200 val_200 200 val_200 +200 val_200 200 val_200 +491 val_491 491 val_491 +237 val_237 237 val_237 +237 val_237 237 val_237 +439 val_439 439 val_439 +439 val_439 439 val_439 +360 val_360 360 val_360 +248 val_248 248 val_248 +479 val_479 479 val_479 +305 val_305 305 val_305 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +444 val_444 444 val_444 +120 val_120 120 val_120 +120 val_120 120 val_120 +429 val_429 429 val_429 +429 val_429 429 val_429 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +443 val_443 443 val_443 +323 val_323 323 val_323 +325 val_325 325 val_325 +325 val_325 325 val_325 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +478 val_478 478 val_478 +478 val_478 478 val_478 +178 val_178 178 val_178 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +310 val_310 310 val_310 +317 val_317 317 val_317 +317 val_317 317 val_317 +333 val_333 333 val_333 +333 val_333 333 val_333 +493 val_493 493 val_493 +460 val_460 460 val_460 +207 val_207 207 val_207 +207 val_207 207 val_207 +249 val_249 249 val_249 +265 val_265 265 val_265 +265 val_265 265 val_265 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +83 val_83 83 val_83 +83 val_83 83 val_83 +136 val_136 136 val_136 +353 val_353 353 val_353 +353 val_353 353 val_353 +172 val_172 172 val_172 +172 val_172 172 val_172 +214 val_214 214 val_214 +462 val_462 462 val_462 +462 val_462 462 val_462 +233 val_233 233 val_233 +233 val_233 233 val_233 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +133 val_133 133 val_133 +175 val_175 175 val_175 +175 val_175 175 val_175 +189 val_189 189 val_189 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +375 val_375 375 val_375 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +421 val_421 421 val_421 +407 val_407 407 val_407 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +256 val_256 256 val_256 +256 val_256 256 val_256 +26 val_26 26 val_26 +26 val_26 26 val_26 +134 val_134 134 val_134 +134 val_134 134 val_134 +67 val_67 67 val_67 +67 val_67 67 val_67 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +379 val_379 379 val_379 +18 val_18 18 val_18 +18 val_18 18 val_18 +462 val_462 462 val_462 +462 val_462 462 val_462 +492 val_492 492 val_492 +492 val_492 492 val_492 +100 val_100 100 val_100 +100 val_100 100 val_100 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +9 val_9 9 val_9 +341 val_341 341 val_341 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +146 val_146 146 val_146 +146 val_146 146 val_146 +458 val_458 458 val_458 +458 val_458 458 val_458 +362 val_362 362 val_362 +186 val_186 186 val_186 +285 val_285 285 val_285 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +18 val_18 18 val_18 +18 val_18 18 val_18 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +183 val_183 183 val_183 +281 val_281 281 val_281 +281 val_281 281 val_281 +344 val_344 344 val_344 +344 val_344 344 val_344 +97 val_97 97 val_97 +97 val_97 97 val_97 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +315 val_315 315 val_315 +84 val_84 84 val_84 +84 val_84 84 val_84 +28 val_28 28 val_28 +37 val_37 37 val_37 +37 val_37 37 val_37 +448 val_448 448 val_448 +152 val_152 152 val_152 +152 val_152 152 val_152 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +307 val_307 307 val_307 +307 val_307 307 val_307 +194 val_194 194 val_194 +414 val_414 414 val_414 +414 val_414 414 val_414 +477 val_477 477 val_477 +222 val_222 222 val_222 +126 val_126 126 val_126 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +400 val_400 400 val_400 +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 diff --git a/src/test/hive/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 b/src/test/hive/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 b/src/test/hive/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 b/src/test/hive/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 b/src/test/hive/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 new file mode 100644 index 0000000000000..cf940f4c5faa8 --- /dev/null +++ b/src/test/hive/golden/type_widening-3-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 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e b/src/test/hive/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e new file mode 100644 index 0000000000000..fed3d6802023a --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e @@ -0,0 +1 @@ +collect_set(x) - Returns a set of objects with duplicate elements eliminated \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 b/src/test/hive/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 new file mode 100644 index 0000000000000..fed3d6802023a --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 @@ -0,0 +1 @@ +collect_set(x) - Returns a set of objects with duplicate elements eliminated \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db b/src/test/hive/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..cedc3068ee4a1 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-10-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"] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 b/src/test/hive/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db b/src/test/hive/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..cedc3068ee4a1 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-13-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"] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 b/src/test/hive/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e b/src/test/hive/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db b/src/test/hive/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..cedc3068ee4a1 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-4-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"] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e b/src/test/hive/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db b/src/test/hive/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..cedc3068ee4a1 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-7-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"] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 b/src/test/hive/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 b/src/test/hive/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 b/src/test/hive/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 b/src/test/hive/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 b/src/test/hive/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 b/src/test/hive/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e b/src/test/hive/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e new file mode 100644 index 0000000000000..6e5d422b3132e --- /dev/null +++ b/src/test/hive/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e @@ -0,0 +1,2 @@ +corr(x,y) - Returns the Pearson coefficient of correlation +between a set of number pairs \ No newline at end of file diff --git a/src/test/hive/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a b/src/test/hive/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a new file mode 100644 index 0000000000000..fcb49ae69f74f --- /dev/null +++ b/src/test/hive/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a @@ -0,0 +1,9 @@ +corr(x,y) - Returns the Pearson coefficient of correlation +between a set of number pairs +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 or +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 diff --git a/src/test/hive/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb b/src/test/hive/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 b/src/test/hive/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 b/src/test/hive/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 b/src/test/hive/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 new file mode 100644 index 0000000000000..3f730875aef8c --- /dev/null +++ b/src/test/hive/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 @@ -0,0 +1,6 @@ +1 NULL +2 NULL +3 NULL +4 NULL +5 NULL +6 NULL \ No newline at end of file diff --git a/src/test/hive/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 b/src/test/hive/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 new file mode 100644 index 0000000000000..5d97236e8b03f --- /dev/null +++ b/src/test/hive/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 @@ -0,0 +1 @@ +0.6633880657639323 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 b/src/test/hive/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 b/src/test/hive/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 b/src/test/hive/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 b/src/test/hive/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 b/src/test/hive/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 new file mode 100644 index 0000000000000..0fed030c22af3 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 @@ -0,0 +1 @@ +covar_pop(x,y) - Returns the population covariance of a set of number pairs \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad b/src/test/hive/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad new file mode 100644 index 0000000000000..4037062d2da93 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad @@ -0,0 +1,6 @@ +covar_pop(x,y) - Returns the population covariance of a set of number pairs +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 diff --git a/src/test/hive/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd b/src/test/hive/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b b/src/test/hive/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae b/src/test/hive/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/src/test/hive/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f b/src/test/hive/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f new file mode 100644 index 0000000000000..848e15bc61476 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f @@ -0,0 +1,6 @@ +1 NULL +2 NULL +3 0.0 +4 0.0 +5 0.0 +6 0.0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 b/src/test/hive/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 new file mode 100644 index 0000000000000..1a49bf590b346 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 @@ -0,0 +1 @@ +3.624999999999999 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 b/src/test/hive/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 b/src/test/hive/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 b/src/test/hive/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 b/src/test/hive/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf b/src/test/hive/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf new file mode 100644 index 0000000000000..97a97c8b5965a --- /dev/null +++ b/src/test/hive/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf @@ -0,0 +1 @@ +covar_samp(x,y) - Returns the sample covariance of a set of number pairs \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c b/src/test/hive/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c new file mode 100644 index 0000000000000..e666adba8df68 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c @@ -0,0 +1,6 @@ +covar_samp(x,y) - Returns the sample covariance of a set of number pairs +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 diff --git a/src/test/hive/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b b/src/test/hive/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 b/src/test/hive/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 b/src/test/hive/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/src/test/hive/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a b/src/test/hive/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a new file mode 100644 index 0000000000000..848e15bc61476 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a @@ -0,0 +1,6 @@ +1 NULL +2 NULL +3 0.0 +4 0.0 +5 0.0 +6 0.0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 b/src/test/hive/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 new file mode 100644 index 0000000000000..30d98234a23e9 --- /dev/null +++ b/src/test/hive/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 @@ -0,0 +1 @@ +4.833333333333332 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 b/src/test/hive/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 new file mode 100644 index 0000000000000..3e8bb17f24d6f --- /dev/null +++ b/src/test/hive/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 @@ -0,0 +1 @@ +[{"x":135.0284552845532,"y":246.0},{"x":381.39370078740143,"y":254.0}] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 b/src/test/hive/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 new file mode 100644 index 0000000000000..048770d4c8402 --- /dev/null +++ b/src/test/hive/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 @@ -0,0 +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 diff --git a/src/test/hive/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 b/src/test/hive/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 new file mode 100644 index 0000000000000..8b54db8da7167 --- /dev/null +++ b/src/test/hive/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 @@ -0,0 +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 diff --git a/src/test/hive/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 b/src/test/hive/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 new file mode 100644 index 0000000000000..aed3f1e704750 --- /dev/null +++ b/src/test/hive/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 @@ -0,0 +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 diff --git a/src/test/hive/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 b/src/test/hive/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 b/src/test/hive/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 new file mode 100644 index 0000000000000..2953abcf1e644 --- /dev/null +++ b/src/test/hive/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 @@ -0,0 +1 @@ +0.0 NULL NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac b/src/test/hive/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 b/src/test/hive/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae b/src/test/hive/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 b/src/test/hive/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 b/src/test/hive/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb b/src/test/hive/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 b/src/test/hive/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f b/src/test/hive/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 b/src/test/hive/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 b/src/test/hive/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a b/src/test/hive/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/src/test/hive/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff b/src/test/hive/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df b/src/test/hive/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b b/src/test/hive/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 b/src/test/hive/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 b/src/test/hive/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/src/test/hive/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 b/src/test/hive/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 @@ -0,0 +1 @@ +[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 b/src/test/hive/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb b/src/test/hive/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 b/src/test/hive/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c b/src/test/hive/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c @@ -0,0 +1 @@ +[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/src/test/hive/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 b/src/test/hive/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/src/test/hive/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff b/src/test/hive/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df b/src/test/hive/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b b/src/test/hive/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 b/src/test/hive/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 b/src/test/hive/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f b/src/test/hive/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/src/test/hive/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 b/src/test/hive/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 @@ -0,0 +1 @@ +[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 b/src/test/hive/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb b/src/test/hive/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c b/src/test/hive/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c @@ -0,0 +1 @@ +[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/src/test/hive/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 b/src/test/hive/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/src/test/hive/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/src/test/hive/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f b/src/test/hive/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 b/src/test/hive/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace b/src/test/hive/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 b/src/test/hive/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae b/src/test/hive/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 b/src/test/hive/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 b/src/test/hive/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb b/src/test/hive/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 b/src/test/hive/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f b/src/test/hive/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 b/src/test/hive/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f b/src/test/hive/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/src/test/hive/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 b/src/test/hive/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a b/src/test/hive/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/src/test/hive/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff b/src/test/hive/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df b/src/test/hive/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b b/src/test/hive/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 b/src/test/hive/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 b/src/test/hive/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/src/test/hive/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 b/src/test/hive/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 @@ -0,0 +1 @@ +[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 b/src/test/hive/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb b/src/test/hive/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 b/src/test/hive/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c b/src/test/hive/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c @@ -0,0 +1 @@ +[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/src/test/hive/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 b/src/test/hive/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 b/src/test/hive/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/src/test/hive/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff b/src/test/hive/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df b/src/test/hive/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b b/src/test/hive/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 b/src/test/hive/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 b/src/test/hive/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f b/src/test/hive/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/src/test/hive/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 b/src/test/hive/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 @@ -0,0 +1 @@ +[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 b/src/test/hive/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb b/src/test/hive/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c b/src/test/hive/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c @@ -0,0 +1 @@ +[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/src/test/hive/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/src/test/hive/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 @@ -0,0 +1 @@ +[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/src/test/hive/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 b/src/test/hive/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/src/test/hive/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/src/test/hive/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f b/src/test/hive/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 b/src/test/hive/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 b/src/test/hive/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf1-1-8281592c818ada269024ac669bec78da b/src/test/hive/golden/udf1-1-8281592c818ada269024ac669bec78da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f b/src/test/hive/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..493daf5d79c54 --- /dev/null +++ b/src/test/hive/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +true false true true true false false false true true false true true acc abc abb hive hadoop AaAbAcA false diff --git a/src/test/hive/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 b/src/test/hive/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf2-2-31c8af064bac42541558a95fad3bca97 b/src/test/hive/golden/udf2-2-31c8af064bac42541558a95fad3bca97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 b/src/test/hive/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 new file mode 100644 index 0000000000000..398f517b25b5f --- /dev/null +++ b/src/test/hive/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 @@ -0,0 +1 @@ +| abc | abc | abc | \ No newline at end of file diff --git a/src/test/hive/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c b/src/test/hive/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 b/src/test/hive/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e b/src/test/hive/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..b2890dc4cdb3a --- /dev/null +++ b/src/test/hive/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +0 NULL NULL NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 b/src/test/hive/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 b/src/test/hive/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf6-3-e579646b969eef49b09656114da52a73 b/src/test/hive/golden/udf6-3-e579646b969eef49b09656114da52a73 new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/src/test/hive/golden/udf6-3-e579646b969eef49b09656114da52a73 @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/src/test/hive/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 b/src/test/hive/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf6-5-fe336cd9850d6357980bd19139f76e b/src/test/hive/golden/udf6-5-fe336cd9850d6357980bd19139f76e new file mode 100644 index 0000000000000..132dbea91f8a0 --- /dev/null +++ b/src/test/hive/golden/udf6-5-fe336cd9850d6357980bd19139f76e @@ -0,0 +1 @@ +1 2 2 a 0.1 2 126 128 128 1.0 128 \ No newline at end of file diff --git a/src/test/hive/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 b/src/test/hive/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 b/src/test/hive/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 b/src/test/hive/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 new file mode 100644 index 0000000000000..51958ff45ab8d --- /dev/null +++ b/src/test/hive/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf8-1-63c38297946a2060c0cff4a426b0520c b/src/test/hive/golden/udf8-1-63c38297946a2060c0cff4a426b0520c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 b/src/test/hive/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb b/src/test/hive/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b b/src/test/hive/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b new file mode 100644 index 0000000000000..8a89b039b7151 --- /dev/null +++ b/src/test/hive/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b @@ -0,0 +1 @@ +1.0 1.0 1 \ No newline at end of file diff --git a/src/test/hive/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 b/src/test/hive/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c b/src/test/hive/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c new file mode 100644 index 0000000000000..83e2e121222a2 --- /dev/null +++ b/src/test/hive/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e b/src/test/hive/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 b/src/test/hive/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f b/src/test/hive/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c b/src/test/hive/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c new file mode 100644 index 0000000000000..83593acfe8f2d --- /dev/null +++ b/src/test/hive/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c @@ -0,0 +1 @@ +2.718281828459045 diff --git a/src/test/hive/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a b/src/test/hive/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a new file mode 100644 index 0000000000000..23a1652370236 --- /dev/null +++ b/src/test/hive/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a @@ -0,0 +1 @@ +E() - returns E diff --git a/src/test/hive/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 b/src/test/hive/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 new file mode 100644 index 0000000000000..8ad91668cd245 --- /dev/null +++ b/src/test/hive/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 @@ -0,0 +1,5 @@ +E() - returns E +Synonyms: e +Example: + > SELECT E() FROM src LIMIT 1; + 2.718281828459045 diff --git a/src/test/hive/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f b/src/test/hive/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c b/src/test/hive/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c new file mode 100644 index 0000000000000..83593acfe8f2d --- /dev/null +++ b/src/test/hive/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c @@ -0,0 +1 @@ +2.718281828459045 diff --git a/src/test/hive/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a b/src/test/hive/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a new file mode 100644 index 0000000000000..23a1652370236 --- /dev/null +++ b/src/test/hive/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a @@ -0,0 +1 @@ +E() - returns E diff --git a/src/test/hive/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 b/src/test/hive/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 new file mode 100644 index 0000000000000..8ad91668cd245 --- /dev/null +++ b/src/test/hive/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 @@ -0,0 +1,5 @@ +E() - returns E +Synonyms: e +Example: + > SELECT E() FROM src LIMIT 1; + 2.718281828459045 diff --git a/src/test/hive/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f b/src/test/hive/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 b/src/test/hive/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 new file mode 100644 index 0000000000000..bd1956ec6788c --- /dev/null +++ b/src/test/hive/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 @@ -0,0 +1 @@ +3.141592653589793 diff --git a/src/test/hive/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 b/src/test/hive/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 new file mode 100644 index 0000000000000..dd554e4ac9c28 --- /dev/null +++ b/src/test/hive/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 @@ -0,0 +1 @@ +PI() - returns pi diff --git a/src/test/hive/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 b/src/test/hive/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 new file mode 100644 index 0000000000000..b66e03b623c75 --- /dev/null +++ b/src/test/hive/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 @@ -0,0 +1,5 @@ +PI() - returns pi +Synonyms: pi +Example: + > SELECT PI() FROM src LIMIT 1; + 3.14159... diff --git a/src/test/hive/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f b/src/test/hive/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 b/src/test/hive/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 new file mode 100644 index 0000000000000..bd1956ec6788c --- /dev/null +++ b/src/test/hive/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 @@ -0,0 +1 @@ +3.141592653589793 diff --git a/src/test/hive/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 b/src/test/hive/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 new file mode 100644 index 0000000000000..dd554e4ac9c28 --- /dev/null +++ b/src/test/hive/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 @@ -0,0 +1 @@ +PI() - returns pi diff --git a/src/test/hive/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 b/src/test/hive/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 new file mode 100644 index 0000000000000..b66e03b623c75 --- /dev/null +++ b/src/test/hive/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 @@ -0,0 +1,5 @@ +PI() - returns pi +Synonyms: pi +Example: + > SELECT PI() FROM src LIMIT 1; + 3.14159... diff --git a/src/test/hive/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 b/src/test/hive/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 new file mode 100644 index 0000000000000..b613b3b9fe967 --- /dev/null +++ b/src/test/hive/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 @@ -0,0 +1 @@ +abs(x) - returns the absolute value of x \ No newline at end of file diff --git a/src/test/hive/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 b/src/test/hive/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 new file mode 100644 index 0000000000000..c2fb6dc49322d --- /dev/null +++ b/src/test/hive/golden/udf_abs-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 b/src/test/hive/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 b/src/test/hive/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 new file mode 100644 index 0000000000000..f5a78bc6e59b8 --- /dev/null +++ b/src/test/hive/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 @@ -0,0 +1 @@ +0 1 123 9223372036854775807 9223372036854775807 \ No newline at end of file diff --git a/src/test/hive/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c b/src/test/hive/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 b/src/test/hive/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 new file mode 100644 index 0000000000000..9e57fa65e8014 --- /dev/null +++ b/src/test/hive/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 @@ -0,0 +1 @@ +0.0 3.14159265 3.14159265 \ No newline at end of file diff --git a/src/test/hive/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 b/src/test/hive/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 new file mode 100644 index 0000000000000..204db415144a7 --- /dev/null +++ b/src/test/hive/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 @@ -0,0 +1 @@ +acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f b/src/test/hive/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f new file mode 100644 index 0000000000000..d49af1cbf4800 --- /dev/null +++ b/src/test/hive/golden/udf_acos-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b b/src/test/hive/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 b/src/test/hive/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 new file mode 100644 index 0000000000000..5548bdb7cf26a --- /dev/null +++ b/src/test/hive/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 @@ -0,0 +1 @@ +1.5707963267948966 \ No newline at end of file diff --git a/src/test/hive/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a b/src/test/hive/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a new file mode 100644 index 0000000000000..1f2d6faad9a2c --- /dev/null +++ b/src/test/hive/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a @@ -0,0 +1 @@ +2.0943951023931957 0.7208187608700897 \ No newline at end of file diff --git a/src/test/hive/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e b/src/test/hive/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/src/test/hive/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/src/test/hive/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 b/src/test/hive/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 new file mode 100644 index 0000000000000..e8f11c444a808 --- /dev/null +++ b/src/test/hive/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 @@ -0,0 +1 @@ +a + b - Returns a+b \ No newline at end of file diff --git a/src/test/hive/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 b/src/test/hive/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 new file mode 100644 index 0000000000000..e8f11c444a808 --- /dev/null +++ b/src/test/hive/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 @@ -0,0 +1 @@ +a + b - Returns a+b \ No newline at end of file diff --git a/src/test/hive/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee b/src/test/hive/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee new file mode 100644 index 0000000000000..8c5ef9c86c107 --- /dev/null +++ b/src/test/hive/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee @@ -0,0 +1 @@ +array(n0, n1...) - Creates an array with the given elements diff --git a/src/test/hive/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a b/src/test/hive/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a new file mode 100644 index 0000000000000..8c5ef9c86c107 --- /dev/null +++ b/src/test/hive/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a @@ -0,0 +1 @@ +array(n0, n1...) - Creates an array with the given elements diff --git a/src/test/hive/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 b/src/test/hive/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_array-3-a5d12c41277fb158e09281169c905122 b/src/test/hive/golden/udf_array-3-a5d12c41277fb158e09281169c905122 new file mode 100644 index 0000000000000..7b41de31897cc --- /dev/null +++ b/src/test/hive/golden/udf_array-3-a5d12c41277fb158e09281169c905122 @@ -0,0 +1 @@ +[] NULL [1,2,3] 3 ["1","a","2","3"] 2 2 diff --git a/src/test/hive/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a b/src/test/hive/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a new file mode 100644 index 0000000000000..783e7d086a5c6 --- /dev/null +++ b/src/test/hive/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a @@ -0,0 +1 @@ +array_contains(array, value) - Returns TRUE if the array contains value. \ No newline at end of file diff --git a/src/test/hive/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c b/src/test/hive/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c new file mode 100644 index 0000000000000..0ccae5649af4a --- /dev/null +++ b/src/test/hive/golden/udf_array_contains-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 b/src/test/hive/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/src/test/hive/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/src/test/hive/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 b/src/test/hive/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/src/test/hive/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/src/test/hive/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c b/src/test/hive/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c new file mode 100644 index 0000000000000..e121c27212c66 --- /dev/null +++ b/src/test/hive/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c @@ -0,0 +1 @@ +ascii(str) - returns the numeric value of the first character of str \ No newline at end of file diff --git a/src/test/hive/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 b/src/test/hive/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 new file mode 100644 index 0000000000000..7c9fb835dfe57 --- /dev/null +++ b/src/test/hive/golden/udf_ascii-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa b/src/test/hive/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 b/src/test/hive/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 new file mode 100644 index 0000000000000..726fda4c15ed3 --- /dev/null +++ b/src/test/hive/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 @@ -0,0 +1 @@ +70 0 33 \ No newline at end of file diff --git a/src/test/hive/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd b/src/test/hive/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd new file mode 100644 index 0000000000000..3a56bffc8da34 --- /dev/null +++ b/src/test/hive/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd @@ -0,0 +1 @@ +asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 b/src/test/hive/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 new file mode 100644 index 0000000000000..ef207d3f5da38 --- /dev/null +++ b/src/test/hive/golden/udf_asin-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f b/src/test/hive/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 b/src/test/hive/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/src/test/hive/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 b/src/test/hive/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 new file mode 100644 index 0000000000000..a9d9b65be4189 --- /dev/null +++ b/src/test/hive/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 @@ -0,0 +1 @@ +-0.5235987755982989 0.7208187608700897 \ No newline at end of file diff --git a/src/test/hive/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 b/src/test/hive/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/src/test/hive/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/src/test/hive/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b b/src/test/hive/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b new file mode 100644 index 0000000000000..019676ad50bc3 --- /dev/null +++ b/src/test/hive/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b @@ -0,0 +1 @@ +atan(x) - returns the atan (arctan) of x (x is in radians) \ No newline at end of file diff --git a/src/test/hive/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 b/src/test/hive/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 new file mode 100644 index 0000000000000..ce9e52716ff44 --- /dev/null +++ b/src/test/hive/golden/udf_atan-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 b/src/test/hive/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 b/src/test/hive/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 new file mode 100644 index 0000000000000..37b3e6018d62c --- /dev/null +++ b/src/test/hive/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 @@ -0,0 +1 @@ +0.7853981633974483 1.4056476493802699 -0.7853981633974483 \ No newline at end of file diff --git a/src/test/hive/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b b/src/test/hive/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b new file mode 100644 index 0000000000000..019676ad50bc3 --- /dev/null +++ b/src/test/hive/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b @@ -0,0 +1 @@ +atan(x) - returns the atan (arctan) of x (x is in radians) \ No newline at end of file diff --git a/src/test/hive/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 b/src/test/hive/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 new file mode 100644 index 0000000000000..ce9e52716ff44 --- /dev/null +++ b/src/test/hive/golden/udf_atan-5-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 b/src/test/hive/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 b/src/test/hive/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 new file mode 100644 index 0000000000000..37b3e6018d62c --- /dev/null +++ b/src/test/hive/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 @@ -0,0 +1 @@ +0.7853981633974483 1.4056476493802699 -0.7853981633974483 \ No newline at end of file diff --git a/src/test/hive/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 b/src/test/hive/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 new file mode 100644 index 0000000000000..bd171b0e9a29c --- /dev/null +++ b/src/test/hive/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 @@ -0,0 +1 @@ +avg(x) - Returns the mean of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f b/src/test/hive/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f new file mode 100644 index 0000000000000..bd171b0e9a29c --- /dev/null +++ b/src/test/hive/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f @@ -0,0 +1 @@ +avg(x) - Returns the mean of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 b/src/test/hive/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 new file mode 100644 index 0000000000000..27b6cd42b68cf --- /dev/null +++ b/src/test/hive/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 @@ -0,0 +1 @@ +There is no documentation for function 'bigint' \ No newline at end of file diff --git a/src/test/hive/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 b/src/test/hive/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 new file mode 100644 index 0000000000000..27b6cd42b68cf --- /dev/null +++ b/src/test/hive/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 @@ -0,0 +1 @@ +There is no documentation for function 'bigint' \ No newline at end of file diff --git a/src/test/hive/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b b/src/test/hive/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b new file mode 100644 index 0000000000000..4790a2c1f9292 --- /dev/null +++ b/src/test/hive/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b @@ -0,0 +1 @@ +bin(n) - returns n in binary \ No newline at end of file diff --git a/src/test/hive/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 b/src/test/hive/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 new file mode 100644 index 0000000000000..4d82712e858e4 --- /dev/null +++ b/src/test/hive/golden/udf_bin-1-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' \ No newline at end of file diff --git a/src/test/hive/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b b/src/test/hive/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b new file mode 100644 index 0000000000000..20fae133f0b4d --- /dev/null +++ b/src/test/hive/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b @@ -0,0 +1 @@ +1 0 101111101011100001101100101 \ No newline at end of file diff --git a/src/test/hive/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 b/src/test/hive/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 new file mode 100644 index 0000000000000..76ea1be7f9214 --- /dev/null +++ b/src/test/hive/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 @@ -0,0 +1 @@ +1111111111111111111111111111111111111111111111111111111111111011 \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d b/src/test/hive/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d new file mode 100644 index 0000000000000..652a71b48011c --- /dev/null +++ b/src/test/hive/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d @@ -0,0 +1 @@ +[13,2,4,8589934592,4096,0] \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 b/src/test/hive/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 new file mode 100644 index 0000000000000..6d061bf1198c2 --- /dev/null +++ b/src/test/hive/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 @@ -0,0 +1 @@ +[13,1,4,2,0] \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b b/src/test/hive/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 b/src/test/hive/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 b/src/test/hive/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 b/src/test/hive/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 new file mode 100644 index 0000000000000..43be09952b09c --- /dev/null +++ b/src/test/hive/golden/udf_bitmap_and-5-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] \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b b/src/test/hive/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 b/src/test/hive/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/src/test/hive/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df b/src/test/hive/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df new file mode 100644 index 0000000000000..02e4a84d62c4b --- /dev/null +++ b/src/test/hive/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df @@ -0,0 +1 @@ +false \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c b/src/test/hive/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c new file mode 100644 index 0000000000000..652a71b48011c --- /dev/null +++ b/src/test/hive/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c @@ -0,0 +1 @@ +[13,2,4,8589934592,4096,0] \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef b/src/test/hive/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef new file mode 100644 index 0000000000000..163f4734f4c21 --- /dev/null +++ b/src/test/hive/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef @@ -0,0 +1 @@ +[13,2,4,8589934592,4224,0] \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b b/src/test/hive/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 b/src/test/hive/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 b/src/test/hive/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 b/src/test/hive/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 new file mode 100644 index 0000000000000..c977a4ce25c3f --- /dev/null +++ b/src/test/hive/golden/udf_bitmap_or-5-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] \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b b/src/test/hive/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a b/src/test/hive/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a new file mode 100644 index 0000000000000..71cadba7649e8 --- /dev/null +++ b/src/test/hive/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a @@ -0,0 +1 @@ +a & b - Bitwise and \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b b/src/test/hive/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b new file mode 100644 index 0000000000000..ead02cba53aa4 --- /dev/null +++ b/src/test/hive/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b @@ -0,0 +1,4 @@ +a & b - Bitwise and +Example: + > SELECT 3 & 5 FROM src LIMIT 1; + 1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f b/src/test/hive/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f new file mode 100644 index 0000000000000..8b06e374c708d --- /dev/null +++ b/src/test/hive/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f @@ -0,0 +1 @@ +~ n - Bitwise not \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d b/src/test/hive/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d new file mode 100644 index 0000000000000..413ca8a653e1f --- /dev/null +++ b/src/test/hive/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d @@ -0,0 +1,4 @@ +~ n - Bitwise not +Example: + > SELECT ~ 0 FROM src LIMIT 1; + -1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 b/src/test/hive/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 new file mode 100644 index 0000000000000..bdbd28cad97c4 --- /dev/null +++ b/src/test/hive/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 @@ -0,0 +1 @@ +a | b - Bitwise or \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb b/src/test/hive/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb new file mode 100644 index 0000000000000..4c116be2acae3 --- /dev/null +++ b/src/test/hive/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb @@ -0,0 +1,4 @@ +a | b - Bitwise or +Example: + > SELECT 3 | 5 FROM src LIMIT 1; + 7 \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 b/src/test/hive/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 new file mode 100644 index 0000000000000..b483f75f1b956 --- /dev/null +++ b/src/test/hive/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 @@ -0,0 +1 @@ +a ^ b - Bitwise exclusive or \ No newline at end of file diff --git a/src/test/hive/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f b/src/test/hive/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f new file mode 100644 index 0000000000000..03bf261d7cf51 --- /dev/null +++ b/src/test/hive/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f @@ -0,0 +1,4 @@ +a ^ b - Bitwise exclusive or +Example: + > SELECT 3 ^ 5 FROM src LIMIT 1; + 2 \ No newline at end of file diff --git a/src/test/hive/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 b/src/test/hive/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 new file mode 100644 index 0000000000000..8134698dfbdcd --- /dev/null +++ b/src/test/hive/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 @@ -0,0 +1 @@ +There is no documentation for function 'boolean' \ No newline at end of file diff --git a/src/test/hive/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e b/src/test/hive/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e new file mode 100644 index 0000000000000..8134698dfbdcd --- /dev/null +++ b/src/test/hive/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e @@ -0,0 +1 @@ +There is no documentation for function 'boolean' \ No newline at end of file diff --git a/src/test/hive/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 b/src/test/hive/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 new file mode 100644 index 0000000000000..645bd7302810f --- /dev/null +++ b/src/test/hive/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 @@ -0,0 +1 @@ +There is no documentation for function 'case' \ No newline at end of file diff --git a/src/test/hive/golden/udf_case-1-54acf006155d8822a50e47729be24004 b/src/test/hive/golden/udf_case-1-54acf006155d8822a50e47729be24004 new file mode 100644 index 0000000000000..645bd7302810f --- /dev/null +++ b/src/test/hive/golden/udf_case-1-54acf006155d8822a50e47729be24004 @@ -0,0 +1 @@ +There is no documentation for function 'case' \ No newline at end of file diff --git a/src/test/hive/golden/udf_case-2-98ee676f92950375917f09d2e492253f b/src/test/hive/golden/udf_case-2-98ee676f92950375917f09d2e492253f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c b/src/test/hive/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c new file mode 100644 index 0000000000000..1ca086d1dca4d --- /dev/null +++ b/src/test/hive/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c @@ -0,0 +1 @@ +2 5 15 NULL 20 24 \ No newline at end of file diff --git a/src/test/hive/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 b/src/test/hive/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 new file mode 100644 index 0000000000000..b920295f69a53 --- /dev/null +++ b/src/test/hive/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 @@ -0,0 +1 @@ +yo \ No newline at end of file diff --git a/src/test/hive/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c b/src/test/hive/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 b/src/test/hive/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 b/src/test/hive/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 new file mode 100644 index 0000000000000..d97a0ef52b2e8 --- /dev/null +++ b/src/test/hive/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 @@ -0,0 +1,3 @@ +1 zero 0 +3 10 is ten NULL +100 default NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 b/src/test/hive/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 new file mode 100644 index 0000000000000..4854a3c7773ac --- /dev/null +++ b/src/test/hive/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 @@ -0,0 +1 @@ +ceil(x) - Find the smallest integer not smaller than x \ No newline at end of file diff --git a/src/test/hive/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb b/src/test/hive/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb new file mode 100644 index 0000000000000..23685df97892d --- /dev/null +++ b/src/test/hive/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb @@ -0,0 +1,7 @@ +ceil(x) - Find the smallest integer not smaller than x +Synonyms: ceiling +Example: + > SELECT ceil(-0.1) FROM src LIMIT 1; + 0 + > SELECT ceil(5) FROM src LIMIT 1; + 5 \ No newline at end of file diff --git a/src/test/hive/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd b/src/test/hive/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd new file mode 100644 index 0000000000000..bf61b7e89d8b3 --- /dev/null +++ b/src/test/hive/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd @@ -0,0 +1 @@ +ceiling(x) - Find the smallest integer not smaller than x \ No newline at end of file diff --git a/src/test/hive/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df b/src/test/hive/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df new file mode 100644 index 0000000000000..79a06f0ee2a5d --- /dev/null +++ b/src/test/hive/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df @@ -0,0 +1,7 @@ +ceiling(x) - Find the smallest integer not smaller than x +Synonyms: ceil +Example: + > SELECT ceiling(-0.1) FROM src LIMIT 1; + 0 + > SELECT ceiling(5) FROM src LIMIT 1; + 5 \ No newline at end of file diff --git a/src/test/hive/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 b/src/test/hive/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 new file mode 100644 index 0000000000000..66313c54cc684 --- /dev/null +++ b/src/test/hive/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 @@ -0,0 +1 @@ +coalesce(a1, a2, ...) - Returns the first non-null argument \ No newline at end of file diff --git a/src/test/hive/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 b/src/test/hive/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 new file mode 100644 index 0000000000000..04d69977c3df4 --- /dev/null +++ b/src/test/hive/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 b/src/test/hive/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 b/src/test/hive/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 new file mode 100644 index 0000000000000..cd5686d89eba4 --- /dev/null +++ b/src/test/hive/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 b/src/test/hive/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 b/src/test/hive/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 new file mode 100644 index 0000000000000..98339f97ef2f8 --- /dev/null +++ b/src/test/hive/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 @@ -0,0 +1,11 @@ +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/src/test/hive/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 b/src/test/hive/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 new file mode 100644 index 0000000000000..59d3f01051903 --- /dev/null +++ b/src/test/hive/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 b/src/test/hive/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 new file mode 100644 index 0000000000000..a3998eeeee623 --- /dev/null +++ b/src/test/hive/golden/udf_concat-0-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 b/src/test/hive/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 new file mode 100644 index 0000000000000..0afd542a05033 --- /dev/null +++ b/src/test/hive/golden/udf_concat-1-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' \ No newline at end of file diff --git a/src/test/hive/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 b/src/test/hive/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 new file mode 100644 index 0000000000000..69af4e63792dc --- /dev/null +++ b/src/test/hive/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 @@ -0,0 +1 @@ +ab abc NULL NULL a NULL 123a 12 1 1234abcextra argument \ No newline at end of file diff --git a/src/test/hive/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 b/src/test/hive/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 b/src/test/hive/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..aecd5d9f2a5c2 --- /dev/null +++ b/src/test/hive/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,57 @@ +1234 0 +1234 2 +1234 4 +1234 5 +1234 8 +1234 9 +1234 10 +1234 11 +1234 12 +1234 15 +1234 17 +1234 18 +1234 19 +1234 20 +1234 24 +1234 26 +1234 27 +1234 28 +1234 30 +1234 33 +1234 34 +1234 35 +1234 37 +1234 41 +1234 42 +1234 43 +1234 44 +1234 47 +1234 51 +1234 53 +1234 54 +1234 57 +1234 58 +1234 64 +1234 65 +1234 66 +1234 67 +1234 69 +1234 70 +1234 72 +1234 74 +1234 76 +1234 77 +1234 78 +1234 80 +1234 82 +1234 83 +1234 84 +1234 85 +1234 86 +1234 87 +1234 90 +1234 92 +1234 95 +1234 96 +1234 97 +1234 98 \ No newline at end of file diff --git a/src/test/hive/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea b/src/test/hive/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 b/src/test/hive/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..455cef3f57162 --- /dev/null +++ b/src/test/hive/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,84 @@ +1234abcextra argument val_86 +1234abcextra argument val_27 +1234abcextra argument val_98 +1234abcextra argument val_66 +1234abcextra argument val_37 +1234abcextra argument val_15 +1234abcextra argument val_82 +1234abcextra argument val_17 +1234abcextra argument val_0 +1234abcextra argument val_57 +1234abcextra argument val_20 +1234abcextra argument val_92 +1234abcextra argument val_47 +1234abcextra argument val_72 +1234abcextra argument val_4 +1234abcextra argument val_35 +1234abcextra argument val_54 +1234abcextra argument val_51 +1234abcextra argument val_65 +1234abcextra argument val_83 +1234abcextra argument val_12 +1234abcextra argument val_67 +1234abcextra argument val_84 +1234abcextra argument val_58 +1234abcextra argument val_8 +1234abcextra argument val_24 +1234abcextra argument val_42 +1234abcextra argument val_0 +1234abcextra argument val_96 +1234abcextra argument val_26 +1234abcextra argument val_51 +1234abcextra argument val_43 +1234abcextra argument val_95 +1234abcextra argument val_98 +1234abcextra argument val_85 +1234abcextra argument val_77 +1234abcextra argument val_0 +1234abcextra argument val_87 +1234abcextra argument val_15 +1234abcextra argument val_72 +1234abcextra argument val_90 +1234abcextra argument val_19 +1234abcextra argument val_10 +1234abcextra argument val_5 +1234abcextra argument val_58 +1234abcextra argument val_35 +1234abcextra argument val_95 +1234abcextra argument val_11 +1234abcextra argument val_34 +1234abcextra argument val_42 +1234abcextra argument val_78 +1234abcextra argument val_76 +1234abcextra argument val_41 +1234abcextra argument val_30 +1234abcextra argument val_64 +1234abcextra argument val_76 +1234abcextra argument val_74 +1234abcextra argument val_69 +1234abcextra argument val_33 +1234abcextra argument val_70 +1234abcextra argument val_5 +1234abcextra argument val_2 +1234abcextra argument val_35 +1234abcextra argument val_80 +1234abcextra argument val_44 +1234abcextra argument val_53 +1234abcextra argument val_90 +1234abcextra argument val_12 +1234abcextra argument val_5 +1234abcextra argument val_70 +1234abcextra argument val_24 +1234abcextra argument val_70 +1234abcextra argument val_83 +1234abcextra argument val_26 +1234abcextra argument val_67 +1234abcextra argument val_18 +1234abcextra argument val_9 +1234abcextra argument val_18 +1234abcextra argument val_97 +1234abcextra argument val_84 +1234abcextra argument val_28 +1234abcextra argument val_37 +1234abcextra argument val_90 +1234abcextra argument val_97 \ No newline at end of file diff --git a/src/test/hive/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 b/src/test/hive/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 new file mode 100644 index 0000000000000..c901f70500cba --- /dev/null +++ b/src/test/hive/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab b/src/test/hive/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab new file mode 100644 index 0000000000000..3e751463e055d --- /dev/null +++ b/src/test/hive/golden/udf_concat_ws-1-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' \ No newline at end of file diff --git a/src/test/hive/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c b/src/test/hive/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 b/src/test/hive/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f b/src/test/hive/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 b/src/test/hive/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 new file mode 100644 index 0000000000000..7896fd787f3a2 --- /dev/null +++ b/src/test/hive/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 @@ -0,0 +1 @@ +xyzabc8675309 abc,xyz,8675309 NULL abc**8675309 \ No newline at end of file diff --git a/src/test/hive/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 b/src/test/hive/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c b/src/test/hive/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c new file mode 100644 index 0000000000000..a0ec688a3b084 --- /dev/null +++ b/src/test/hive/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 b/src/test/hive/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 new file mode 100644 index 0000000000000..0c9f2d12ba117 --- /dev/null +++ b/src/test/hive/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 @@ -0,0 +1 @@ +NULL NULL NULL NULL NULL NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-0-d552befca345f3396464529cfde9f75a b/src/test/hive/golden/udf_conv-0-d552befca345f3396464529cfde9f75a new file mode 100644 index 0000000000000..b8fbe88a19971 --- /dev/null +++ b/src/test/hive/golden/udf_conv-0-d552befca345f3396464529cfde9f75a @@ -0,0 +1 @@ +conv(num, from_base, to_base) - convert num from from_base to to_base \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab b/src/test/hive/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab new file mode 100644 index 0000000000000..4f4b0c594c459 --- /dev/null +++ b/src/test/hive/golden/udf_conv-1-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' \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 b/src/test/hive/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 new file mode 100644 index 0000000000000..9512cc4241554 --- /dev/null +++ b/src/test/hive/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 @@ -0,0 +1,3 @@ +EE 568 +56 134 +137 785 \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 b/src/test/hive/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 new file mode 100644 index 0000000000000..91e56a11fa628 --- /dev/null +++ b/src/test/hive/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 @@ -0,0 +1 @@ +3HL 22 33 116ED2B2FB4 \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d b/src/test/hive/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d new file mode 100644 index 0000000000000..08f70d742b8e4 --- /dev/null +++ b/src/test/hive/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d @@ -0,0 +1 @@ +-641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-4-568e843076f358c404a8634b18541c55 b/src/test/hive/golden/udf_conv-4-568e843076f358c404a8634b18541c55 new file mode 100644 index 0000000000000..a2a44daa718cc --- /dev/null +++ b/src/test/hive/golden/udf_conv-4-568e843076f358c404a8634b18541c55 @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 b/src/test/hive/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 new file mode 100644 index 0000000000000..6948fe3783119 --- /dev/null +++ b/src/test/hive/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 @@ -0,0 +1 @@ +5 NULL NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a b/src/test/hive/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a new file mode 100644 index 0000000000000..e1021e50fdcbb --- /dev/null +++ b/src/test/hive/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a @@ -0,0 +1 @@ +3HL 22 33 \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e b/src/test/hive/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e new file mode 100644 index 0000000000000..08f70d742b8e4 --- /dev/null +++ b/src/test/hive/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e @@ -0,0 +1 @@ +-641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb b/src/test/hive/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb new file mode 100644 index 0000000000000..a2a44daa718cc --- /dev/null +++ b/src/test/hive/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 b/src/test/hive/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 new file mode 100644 index 0000000000000..6948fe3783119 --- /dev/null +++ b/src/test/hive/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 @@ -0,0 +1 @@ +5 NULL NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 b/src/test/hive/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 new file mode 100644 index 0000000000000..99a7d42bb5b57 --- /dev/null +++ b/src/test/hive/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 @@ -0,0 +1 @@ +cos(x) - returns the cosine of x (x is in radians) \ No newline at end of file diff --git a/src/test/hive/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 b/src/test/hive/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 new file mode 100644 index 0000000000000..196294de1f19f --- /dev/null +++ b/src/test/hive/golden/udf_cos-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 b/src/test/hive/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 b/src/test/hive/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 new file mode 100644 index 0000000000000..c0690acff887b --- /dev/null +++ b/src/test/hive/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 @@ -0,0 +1 @@ +0.5570225467662173 7.963267107332633E-4 0.8775825618903728 \ No newline at end of file diff --git a/src/test/hive/golden/udf_count-0-534a9b25b87d09e418645b1216949560 b/src/test/hive/golden/udf_count-0-534a9b25b87d09e418645b1216949560 new file mode 100644 index 0000000000000..e01715295adc8 --- /dev/null +++ b/src/test/hive/golden/udf_count-0-534a9b25b87d09e418645b1216949560 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f b/src/test/hive/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f new file mode 100644 index 0000000000000..e01715295adc8 --- /dev/null +++ b/src/test/hive/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 b/src/test/hive/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e b/src/test/hive/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e b/src/test/hive/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 b/src/test/hive/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_count-2-3344645abe535426307b9327b381fc85 b/src/test/hive/golden/udf_count-2-3344645abe535426307b9327b381fc85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 b/src/test/hive/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 b/src/test/hive/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 b/src/test/hive/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 new file mode 100644 index 0000000000000..e0da8ae09ae60 --- /dev/null +++ b/src/test/hive/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 @@ -0,0 +1 @@ +309 \ No newline at end of file diff --git a/src/test/hive/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 b/src/test/hive/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 b/src/test/hive/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 new file mode 100644 index 0000000000000..e0da8ae09ae60 --- /dev/null +++ b/src/test/hive/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 @@ -0,0 +1 @@ +309 \ No newline at end of file diff --git a/src/test/hive/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 b/src/test/hive/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 b/src/test/hive/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/src/test/hive/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/src/test/hive/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 b/src/test/hive/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 new file mode 100644 index 0000000000000..83e0fc4e63a25 --- /dev/null +++ b/src/test/hive/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 @@ -0,0 +1 @@ +date_add(start_date, num_days) - Returns the date that is num_days after start_date. \ No newline at end of file diff --git a/src/test/hive/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 b/src/test/hive/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 new file mode 100644 index 0000000000000..83b9851499f71 --- /dev/null +++ b/src/test/hive/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 @@ -0,0 +1,5 @@ +date_add(start_date, num_days) - Returns the date that is num_days after start_date. +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 diff --git a/src/test/hive/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 b/src/test/hive/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 new file mode 100644 index 0000000000000..7dec81f2b6d4e --- /dev/null +++ b/src/test/hive/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 @@ -0,0 +1 @@ +date_sub(start_date, num_days) - Returns the date that is num_days before start_date. \ No newline at end of file diff --git a/src/test/hive/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 b/src/test/hive/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 new file mode 100644 index 0000000000000..105b63424062b --- /dev/null +++ b/src/test/hive/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 @@ -0,0 +1,5 @@ +date_sub(start_date, num_days) - Returns the date that is num_days before start_date. +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 diff --git a/src/test/hive/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 b/src/test/hive/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 new file mode 100644 index 0000000000000..b59d456397d53 --- /dev/null +++ b/src/test/hive/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 @@ -0,0 +1 @@ +datediff(date1, date2) - Returns the number of days between date1 and date2 \ No newline at end of file diff --git a/src/test/hive/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 b/src/test/hive/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 new file mode 100644 index 0000000000000..c240df94594d4 --- /dev/null +++ b/src/test/hive/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 b/src/test/hive/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 new file mode 100644 index 0000000000000..11e32a4509b3f --- /dev/null +++ b/src/test/hive/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 @@ -0,0 +1 @@ +day(date) - Returns the date of the month of date \ No newline at end of file diff --git a/src/test/hive/golden/udf_day-1-87168babe1110fe4c38269843414ca4 b/src/test/hive/golden/udf_day-1-87168babe1110fe4c38269843414ca4 new file mode 100644 index 0000000000000..9da0858ba92a5 --- /dev/null +++ b/src/test/hive/golden/udf_day-1-87168babe1110fe4c38269843414ca4 @@ -0,0 +1,6 @@ +day(date) - Returns the date of the month of date +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 diff --git a/src/test/hive/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 b/src/test/hive/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 new file mode 100644 index 0000000000000..33e90a2af295f --- /dev/null +++ b/src/test/hive/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 @@ -0,0 +1 @@ +dayofmonth(date) - Returns the date of the month of date \ No newline at end of file diff --git a/src/test/hive/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 b/src/test/hive/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 new file mode 100644 index 0000000000000..ee9911af3248c --- /dev/null +++ b/src/test/hive/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 @@ -0,0 +1,6 @@ +dayofmonth(date) - Returns the date of the month of date +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 diff --git a/src/test/hive/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae b/src/test/hive/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 b/src/test/hive/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 new file mode 100644 index 0000000000000..36f165be67119 --- /dev/null +++ b/src/test/hive/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 @@ -0,0 +1 @@ +180.0 diff --git a/src/test/hive/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b b/src/test/hive/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b new file mode 100644 index 0000000000000..60a5fff4a87a8 --- /dev/null +++ b/src/test/hive/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b @@ -0,0 +1 @@ +degrees(x) - Converts radians to degrees diff --git a/src/test/hive/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/src/test/hive/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 new file mode 100644 index 0000000000000..4a33ce356e237 --- /dev/null +++ b/src/test/hive/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 @@ -0,0 +1,5 @@ +degrees(x) - Converts radians to degrees +Example: + > SELECT degrees(30) FROM src LIMIT 1; + -1 + diff --git a/src/test/hive/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae b/src/test/hive/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 b/src/test/hive/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 new file mode 100644 index 0000000000000..36f165be67119 --- /dev/null +++ b/src/test/hive/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 @@ -0,0 +1 @@ +180.0 diff --git a/src/test/hive/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b b/src/test/hive/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b new file mode 100644 index 0000000000000..60a5fff4a87a8 --- /dev/null +++ b/src/test/hive/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b @@ -0,0 +1 @@ +degrees(x) - Converts radians to degrees diff --git a/src/test/hive/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/src/test/hive/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 new file mode 100644 index 0000000000000..4a33ce356e237 --- /dev/null +++ b/src/test/hive/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 @@ -0,0 +1,5 @@ +degrees(x) - Converts radians to degrees +Example: + > SELECT degrees(30) FROM src LIMIT 1; + -1 + diff --git a/src/test/hive/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db b/src/test/hive/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db new file mode 100644 index 0000000000000..d98718752a36f --- /dev/null +++ b/src/test/hive/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db @@ -0,0 +1 @@ +a div b - Divide a by b rounded to the long integer \ No newline at end of file diff --git a/src/test/hive/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 b/src/test/hive/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 new file mode 100644 index 0000000000000..59265a74f9b3f --- /dev/null +++ b/src/test/hive/golden/udf_div-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b b/src/test/hive/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/src/test/hive/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 b/src/test/hive/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 new file mode 100644 index 0000000000000..8b623e47785f6 --- /dev/null +++ b/src/test/hive/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 @@ -0,0 +1 @@ +a / b - Divide a by b \ No newline at end of file diff --git a/src/test/hive/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 b/src/test/hive/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 new file mode 100644 index 0000000000000..2acf2b6a64648 --- /dev/null +++ b/src/test/hive/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 b/src/test/hive/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 new file mode 100644 index 0000000000000..400122e60f599 --- /dev/null +++ b/src/test/hive/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 @@ -0,0 +1 @@ +1.5 \ No newline at end of file diff --git a/src/test/hive/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 b/src/test/hive/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 new file mode 100644 index 0000000000000..54de9e9f8edec --- /dev/null +++ b/src/test/hive/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 @@ -0,0 +1 @@ +There is no documentation for function 'double' \ No newline at end of file diff --git a/src/test/hive/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 b/src/test/hive/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 new file mode 100644 index 0000000000000..54de9e9f8edec --- /dev/null +++ b/src/test/hive/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 @@ -0,0 +1 @@ +There is no documentation for function 'double' \ No newline at end of file diff --git a/src/test/hive/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f b/src/test/hive/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f new file mode 100644 index 0000000000000..7a22029a2156f --- /dev/null +++ b/src/test/hive/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f @@ -0,0 +1 @@ +elt(n, str1, str2, ...) - returns the n-th string diff --git a/src/test/hive/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 b/src/test/hive/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 new file mode 100644 index 0000000000000..61f204dacf9dc --- /dev/null +++ b/src/test/hive/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 @@ -0,0 +1,4 @@ +elt(n, str1, str2, ...) - returns the n-th string +Example: + > SELECT elt(1, 'face', 'book') FROM src LIMIT 1; + 'face' diff --git a/src/test/hive/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a b/src/test/hive/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 b/src/test/hive/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 new file mode 100644 index 0000000000000..d009faddeafa8 --- /dev/null +++ b/src/test/hive/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 @@ -0,0 +1 @@ +defg cc abc 2 12345 123456789012 1.25 16.0 NULL NULL NULL diff --git a/src/test/hive/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a b/src/test/hive/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a new file mode 100644 index 0000000000000..9b3dbf316d1fb --- /dev/null +++ b/src/test/hive/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a @@ -0,0 +1 @@ +exp(x) - Returns e to the power of x \ No newline at end of file diff --git a/src/test/hive/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 b/src/test/hive/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 new file mode 100644 index 0000000000000..a42c95bb8d213 --- /dev/null +++ b/src/test/hive/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 b/src/test/hive/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 new file mode 100644 index 0000000000000..a30bc26f5ba58 --- /dev/null +++ b/src/test/hive/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_field-1-379d8580693493f620a8f4084709324f b/src/test/hive/golden/udf_field-1-379d8580693493f620a8f4084709324f new file mode 100644 index 0000000000000..bb55c0a1db1f5 --- /dev/null +++ b/src/test/hive/golden/udf_field-1-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. \ No newline at end of file diff --git a/src/test/hive/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 b/src/test/hive/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 new file mode 100644 index 0000000000000..275f46482425d --- /dev/null +++ b/src/test/hive/golden/udf_field-10-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b b/src/test/hive/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b new file mode 100644 index 0000000000000..c42823854fb0f --- /dev/null +++ b/src/test/hive/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b @@ -0,0 +1 @@ +0 0 0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 b/src/test/hive/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 new file mode 100644 index 0000000000000..5869234249808 --- /dev/null +++ b/src/test/hive/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 @@ -0,0 +1 @@ +1 2 3 4 4 \ No newline at end of file diff --git a/src/test/hive/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 b/src/test/hive/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 new file mode 100644 index 0000000000000..5869234249808 --- /dev/null +++ b/src/test/hive/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 @@ -0,0 +1 @@ +1 2 3 4 4 \ No newline at end of file diff --git a/src/test/hive/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 b/src/test/hive/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 b/src/test/hive/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef b/src/test/hive/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef new file mode 100644 index 0000000000000..a76563207da24 --- /dev/null +++ b/src/test/hive/golden/udf_field-7-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 b/src/test/hive/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 b/src/test/hive/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 b/src/test/hive/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 new file mode 100644 index 0000000000000..f14679978b79e --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-0-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. \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 b/src/test/hive/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 new file mode 100644 index 0000000000000..1bebc68416bff --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c b/src/test/hive/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a b/src/test/hive/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 b/src/test/hive/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a b/src/test/hive/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a new file mode 100644 index 0000000000000..e440e5c842586 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a @@ -0,0 +1 @@ +3 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 b/src/test/hive/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb b/src/test/hive/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 b/src/test/hive/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 b/src/test/hive/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 new file mode 100644 index 0000000000000..99f516951ae7d --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 @@ -0,0 +1,2 @@ +311 val_311 +128 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 b/src/test/hive/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e b/src/test/hive/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e new file mode 100644 index 0000000000000..5817cbcff62ac --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-3-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b b/src/test/hive/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f b/src/test/hive/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 b/src/test/hive/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 new file mode 100644 index 0000000000000..e440e5c842586 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 @@ -0,0 +1 @@ +3 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 b/src/test/hive/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 b/src/test/hive/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 b/src/test/hive/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 b/src/test/hive/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 new file mode 100644 index 0000000000000..9d15b5f5956d7 --- /dev/null +++ b/src/test/hive/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 @@ -0,0 +1 @@ +There is no documentation for function 'float' \ No newline at end of file diff --git a/src/test/hive/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b b/src/test/hive/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b new file mode 100644 index 0000000000000..9d15b5f5956d7 --- /dev/null +++ b/src/test/hive/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b @@ -0,0 +1 @@ +There is no documentation for function 'float' \ No newline at end of file diff --git a/src/test/hive/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 b/src/test/hive/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 new file mode 100644 index 0000000000000..c76710bfcc1b5 --- /dev/null +++ b/src/test/hive/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 @@ -0,0 +1 @@ +floor(x) - Find the largest integer not greater than x \ No newline at end of file diff --git a/src/test/hive/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 b/src/test/hive/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 new file mode 100644 index 0000000000000..3f73eea16e183 --- /dev/null +++ b/src/test/hive/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 @@ -0,0 +1,6 @@ +floor(x) - Find the largest integer not greater than x +Example: + > SELECT floor(-0.1) FROM src LIMIT 1; + -1 + > SELECT floor(5) FROM src LIMIT 1; + 5 \ No newline at end of file diff --git a/src/test/hive/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb b/src/test/hive/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 b/src/test/hive/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 new file mode 100644 index 0000000000000..c981e1726c070 --- /dev/null +++ b/src/test/hive/golden/udf_format_number-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd b/src/test/hive/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd new file mode 100644 index 0000000000000..b2aa527ca0ddb --- /dev/null +++ b/src/test/hive/golden/udf_format_number-2-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' \ No newline at end of file diff --git a/src/test/hive/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c b/src/test/hive/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 b/src/test/hive/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 new file mode 100644 index 0000000000000..89e118cc62bf3 --- /dev/null +++ b/src/test/hive/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 @@ -0,0 +1 @@ +12,332.1235 12,332.1000 12,332 \ No newline at end of file diff --git a/src/test/hive/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 b/src/test/hive/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 new file mode 100644 index 0000000000000..33e21fa7dbfc4 --- /dev/null +++ b/src/test/hive/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 @@ -0,0 +1 @@ +0.123456789000 12,345,678.12346 1,234,567.1234568 123,456 \ No newline at end of file diff --git a/src/test/hive/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 b/src/test/hive/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 new file mode 100644 index 0000000000000..07b05cd16f5a3 --- /dev/null +++ b/src/test/hive/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 @@ -0,0 +1 @@ +-123,456 -1,234,567.12 -0.123456789000000 -12,345.1235 \ No newline at end of file diff --git a/src/test/hive/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 b/src/test/hive/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 new file mode 100644 index 0000000000000..2b600a6a83aa8 --- /dev/null +++ b/src/test/hive/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 @@ -0,0 +1 @@ +0.0000 0.0 0.0 0.0 -0.0000 \ No newline at end of file diff --git a/src/test/hive/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e b/src/test/hive/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e new file mode 100644 index 0000000000000..d027b2cb0b94a --- /dev/null +++ b/src/test/hive/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e @@ -0,0 +1 @@ +0 1.0000 12.00 123.00000 1,234.0000000 \ No newline at end of file diff --git a/src/test/hive/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 b/src/test/hive/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 new file mode 100644 index 0000000000000..afdec63c8dfca --- /dev/null +++ b/src/test/hive/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f b/src/test/hive/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f new file mode 100644 index 0000000000000..10ce106f121ba --- /dev/null +++ b/src/test/hive/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f @@ -0,0 +1 @@ +from_unixtime(unix_time, format) - returns unix_time in the specified format \ No newline at end of file diff --git a/src/test/hive/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 b/src/test/hive/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 new file mode 100644 index 0000000000000..20f4d2b35c49a --- /dev/null +++ b/src/test/hive/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f b/src/test/hive/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f new file mode 100644 index 0000000000000..4e4f3f7255fd3 --- /dev/null +++ b/src/test/hive/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f @@ -0,0 +1 @@ +get_json_object(json_txt, path) - Extract a json object from path diff --git a/src/test/hive/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 b/src/test/hive/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 new file mode 100644 index 0000000000000..0e1dcf934a023 --- /dev/null +++ b/src/test/hive/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 @@ -0,0 +1,16 @@ +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 + &#064; : Current object/element + () : Script expression + ?() : Filter (script) expression. + [,] : Union operator + [start:end:step] : array slice operator + diff --git a/src/test/hive/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 b/src/test/hive/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 b/src/test/hive/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 b/src/test/hive/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 b/src/test/hive/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 new file mode 100644 index 0000000000000..54d6096d07bbe --- /dev/null +++ b/src/test/hive/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 @@ -0,0 +1 @@ +a > b - Returns TRUE if a is greater than b \ No newline at end of file diff --git a/src/test/hive/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c b/src/test/hive/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c new file mode 100644 index 0000000000000..54d6096d07bbe --- /dev/null +++ b/src/test/hive/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c @@ -0,0 +1 @@ +a > b - Returns TRUE if a is greater than b \ No newline at end of file diff --git a/src/test/hive/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f b/src/test/hive/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f new file mode 100644 index 0000000000000..679b0376125f0 --- /dev/null +++ b/src/test/hive/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f @@ -0,0 +1 @@ +true false false false \ No newline at end of file diff --git a/src/test/hive/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a b/src/test/hive/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a new file mode 100644 index 0000000000000..abf7dfdab730c --- /dev/null +++ b/src/test/hive/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a @@ -0,0 +1 @@ +a >= b - Returns TRUE if a is not smaller than b \ No newline at end of file diff --git a/src/test/hive/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 b/src/test/hive/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 new file mode 100644 index 0000000000000..abf7dfdab730c --- /dev/null +++ b/src/test/hive/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 @@ -0,0 +1 @@ +a >= b - Returns TRUE if a is not smaller than b \ No newline at end of file diff --git a/src/test/hive/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 b/src/test/hive/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 new file mode 100644 index 0000000000000..1fb1894fc21f4 --- /dev/null +++ b/src/test/hive/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 @@ -0,0 +1 @@ +true false true true \ No newline at end of file diff --git a/src/test/hive/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 b/src/test/hive/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 new file mode 100644 index 0000000000000..d1284cf87da3c --- /dev/null +++ b/src/test/hive/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 @@ -0,0 +1 @@ +hash(a1, a2, ...) - Returns a hash value of the arguments diff --git a/src/test/hive/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a b/src/test/hive/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a new file mode 100644 index 0000000000000..d1284cf87da3c --- /dev/null +++ b/src/test/hive/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a @@ -0,0 +1 @@ +hash(a1, a2, ...) - Returns a hash value of the arguments diff --git a/src/test/hive/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 b/src/test/hive/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_hash-3-23991312391d518aacf3d4469c816eae b/src/test/hive/golden/udf_hash-3-23991312391d518aacf3d4469c816eae new file mode 100644 index 0000000000000..d50944d3d0c0f --- /dev/null +++ b/src/test/hive/golden/udf_hash-3-23991312391d518aacf3d4469c816eae @@ -0,0 +1 @@ +1 2 3 -1097262584 1067450368 1076887552 51508 96354 1 0 1026 diff --git a/src/test/hive/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 b/src/test/hive/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 new file mode 100644 index 0000000000000..f87a6117eacf7 --- /dev/null +++ b/src/test/hive/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 @@ -0,0 +1 @@ +hex(n, bin, or str) - Convert the argument to hexadecimal \ No newline at end of file diff --git a/src/test/hive/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c b/src/test/hive/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c new file mode 100644 index 0000000000000..e8cee4afc3a80 --- /dev/null +++ b/src/test/hive/golden/udf_hex-1-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' \ No newline at end of file diff --git a/src/test/hive/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 b/src/test/hive/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 new file mode 100644 index 0000000000000..34eb75a6c784a --- /dev/null +++ b/src/test/hive/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 @@ -0,0 +1 @@ +46616365626F6F6B 00 71776572747975696F706173646667686A6B6C \ No newline at end of file diff --git a/src/test/hive/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf b/src/test/hive/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf new file mode 100644 index 0000000000000..2a45250c67a63 --- /dev/null +++ b/src/test/hive/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf @@ -0,0 +1 @@ +1 0 FACEB005 \ No newline at end of file diff --git a/src/test/hive/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea b/src/test/hive/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea new file mode 100644 index 0000000000000..50d9557967ac4 --- /dev/null +++ b/src/test/hive/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFB \ No newline at end of file diff --git a/src/test/hive/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce b/src/test/hive/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce new file mode 100644 index 0000000000000..48911456dc339 --- /dev/null +++ b/src/test/hive/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce @@ -0,0 +1 @@ +hour(date) - Returns the hour of date diff --git a/src/test/hive/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 b/src/test/hive/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 new file mode 100644 index 0000000000000..be1a966635202 --- /dev/null +++ b/src/test/hive/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 @@ -0,0 +1,7 @@ +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/src/test/hive/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a b/src/test/hive/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 b/src/test/hive/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 new file mode 100644 index 0000000000000..1a35f9f158133 --- /dev/null +++ b/src/test/hive/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 @@ -0,0 +1 @@ +13 13 NULL diff --git a/src/test/hive/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c b/src/test/hive/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c new file mode 100644 index 0000000000000..2cf0d9d61882e --- /dev/null +++ b/src/test/hive/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c @@ -0,0 +1 @@ +There is no documentation for function 'if' diff --git a/src/test/hive/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a b/src/test/hive/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a new file mode 100644 index 0000000000000..2cf0d9d61882e --- /dev/null +++ b/src/test/hive/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a @@ -0,0 +1 @@ +There is no documentation for function 'if' diff --git a/src/test/hive/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 b/src/test/hive/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_if-3-20206f17367ff284d67044abd745ce9f b/src/test/hive/golden/udf_if-3-20206f17367ff284d67044abd745ce9f new file mode 100644 index 0000000000000..a29e96cbd1db7 --- /dev/null +++ b/src/test/hive/golden/udf_if-3-20206f17367ff284d67044abd745ce9f @@ -0,0 +1 @@ +1 1 1 1 NULL 2 diff --git a/src/test/hive/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca b/src/test/hive/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc b/src/test/hive/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc new file mode 100644 index 0000000000000..f0669b86989d0 --- /dev/null +++ b/src/test/hive/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc @@ -0,0 +1 @@ +128 1.1 ABC 12.3 diff --git a/src/test/hive/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 b/src/test/hive/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 new file mode 100644 index 0000000000000..ee6e42ce6a83d --- /dev/null +++ b/src/test/hive/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 @@ -0,0 +1 @@ +true false true true false true true true NULL NULL true diff --git a/src/test/hive/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 b/src/test/hive/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 new file mode 100644 index 0000000000000..993d93304f95e --- /dev/null +++ b/src/test/hive/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 @@ -0,0 +1,3 @@ +238 +86 +238 diff --git a/src/test/hive/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 b/src/test/hive/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 new file mode 100644 index 0000000000000..cd15a08a539ab --- /dev/null +++ b/src/test/hive/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 @@ -0,0 +1 @@ +in_file(str, filename) - Returns true if str appears in the file diff --git a/src/test/hive/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 b/src/test/hive/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 b/src/test/hive/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 new file mode 100644 index 0000000000000..df5a0561fb8f5 --- /dev/null +++ b/src/test/hive/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 @@ -0,0 +1 @@ +Function '`index`' does not exist. \ No newline at end of file diff --git a/src/test/hive/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 b/src/test/hive/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 new file mode 100644 index 0000000000000..df5a0561fb8f5 --- /dev/null +++ b/src/test/hive/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 @@ -0,0 +1 @@ +Function '`index`' does not exist. \ No newline at end of file diff --git a/src/test/hive/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 b/src/test/hive/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 new file mode 100644 index 0000000000000..ae27b5efea045 --- /dev/null +++ b/src/test/hive/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 @@ -0,0 +1 @@ +instr(str, substr) - Returns the index of the first occurance of substr in str \ No newline at end of file diff --git a/src/test/hive/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 b/src/test/hive/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 new file mode 100644 index 0000000000000..35de2f0d8653f --- /dev/null +++ b/src/test/hive/golden/udf_instr-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 b/src/test/hive/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 b/src/test/hive/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 new file mode 100644 index 0000000000000..4d34e6df7039b --- /dev/null +++ b/src/test/hive/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 @@ -0,0 +1 @@ +1 0 2 2 0 0 2 3 4 2 3 NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba b/src/test/hive/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba new file mode 100644 index 0000000000000..cda33a8e2e555 --- /dev/null +++ b/src/test/hive/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba @@ -0,0 +1 @@ +There is no documentation for function 'int' \ No newline at end of file diff --git a/src/test/hive/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 b/src/test/hive/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 new file mode 100644 index 0000000000000..cda33a8e2e555 --- /dev/null +++ b/src/test/hive/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 @@ -0,0 +1 @@ +There is no documentation for function 'int' \ No newline at end of file diff --git a/src/test/hive/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 b/src/test/hive/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 new file mode 100644 index 0000000000000..723b5aaf2af6b --- /dev/null +++ b/src/test/hive/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 @@ -0,0 +1 @@ +isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 b/src/test/hive/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 new file mode 100644 index 0000000000000..723b5aaf2af6b --- /dev/null +++ b/src/test/hive/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 @@ -0,0 +1 @@ +isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 b/src/test/hive/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 new file mode 100644 index 0000000000000..99510c6f1db15 --- /dev/null +++ b/src/test/hive/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 @@ -0,0 +1 @@ +isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 b/src/test/hive/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 new file mode 100644 index 0000000000000..99510c6f1db15 --- /dev/null +++ b/src/test/hive/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 @@ -0,0 +1 @@ +isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 b/src/test/hive/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 new file mode 100644 index 0000000000000..99510c6f1db15 --- /dev/null +++ b/src/test/hive/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 @@ -0,0 +1 @@ +isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 b/src/test/hive/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 new file mode 100644 index 0000000000000..99510c6f1db15 --- /dev/null +++ b/src/test/hive/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 @@ -0,0 +1 @@ +isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 b/src/test/hive/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 new file mode 100644 index 0000000000000..723b5aaf2af6b --- /dev/null +++ b/src/test/hive/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 @@ -0,0 +1 @@ +isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 b/src/test/hive/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 new file mode 100644 index 0000000000000..723b5aaf2af6b --- /dev/null +++ b/src/test/hive/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 @@ -0,0 +1 @@ +isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 b/src/test/hive/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 b/src/test/hive/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 new file mode 100644 index 0000000000000..eedfbc67d50c7 --- /dev/null +++ b/src/test/hive/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 @@ -0,0 +1 @@ +true true true \ No newline at end of file diff --git a/src/test/hive/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 b/src/test/hive/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b b/src/test/hive/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b new file mode 100644 index 0000000000000..eedfbc67d50c7 --- /dev/null +++ b/src/test/hive/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b @@ -0,0 +1 @@ +true true true \ No newline at end of file diff --git a/src/test/hive/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 b/src/test/hive/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 new file mode 100644 index 0000000000000..b703a30abffba --- /dev/null +++ b/src/test/hive/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 @@ -0,0 +1 @@ +java_method(class,method[,arg1[,arg2..]]) calls method with reflection \ No newline at end of file diff --git a/src/test/hive/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a b/src/test/hive/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a new file mode 100644 index 0000000000000..07375dc92d625 --- /dev/null +++ b/src/test/hive/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a @@ -0,0 +1,3 @@ +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/src/test/hive/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 b/src/test/hive/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd b/src/test/hive/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd new file mode 100644 index 0000000000000..9b93703dae806 --- /dev/null +++ b/src/test/hive/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd @@ -0,0 +1 @@ +1 true 3 2 3 2.718281828459045 1.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad b/src/test/hive/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad new file mode 100644 index 0000000000000..ec6779df2818d --- /dev/null +++ b/src/test/hive/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad @@ -0,0 +1 @@ +lcase(str) - Returns str with all characters changed to lowercase \ No newline at end of file diff --git a/src/test/hive/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e b/src/test/hive/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e new file mode 100644 index 0000000000000..4da30f6c23a1b --- /dev/null +++ b/src/test/hive/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b b/src/test/hive/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b new file mode 100644 index 0000000000000..d4d2bd508b44f --- /dev/null +++ b/src/test/hive/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b @@ -0,0 +1 @@ +length(str | binary) - Returns the length of str or number of bytes in binary data \ No newline at end of file diff --git a/src/test/hive/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 b/src/test/hive/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 new file mode 100644 index 0000000000000..79c1c54639266 --- /dev/null +++ b/src/test/hive/golden/udf_length-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 b/src/test/hive/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/src/test/hive/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/src/test/hive/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 b/src/test/hive/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d b/src/test/hive/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 b/src/test/hive/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f b/src/test/hive/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..6ecb2baabb297 --- /dev/null +++ b/src/test/hive/golden/udf_length-5-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a b/src/test/hive/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f b/src/test/hive/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf b/src/test/hive/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 b/src/test/hive/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 b/src/test/hive/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 new file mode 100644 index 0000000000000..b43707d550c4b --- /dev/null +++ b/src/test/hive/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 @@ -0,0 +1 @@ +a < b - Returns TRUE if a is less than b \ No newline at end of file diff --git a/src/test/hive/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 b/src/test/hive/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 new file mode 100644 index 0000000000000..b43707d550c4b --- /dev/null +++ b/src/test/hive/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 @@ -0,0 +1 @@ +a < b - Returns TRUE if a is less than b \ No newline at end of file diff --git a/src/test/hive/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 b/src/test/hive/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 new file mode 100644 index 0000000000000..4364012093724 --- /dev/null +++ b/src/test/hive/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 @@ -0,0 +1 @@ +false true false false \ No newline at end of file diff --git a/src/test/hive/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b b/src/test/hive/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b new file mode 100644 index 0000000000000..c36acd7ce80bc --- /dev/null +++ b/src/test/hive/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b @@ -0,0 +1 @@ +a <= b - Returns TRUE if a is not greater than b \ No newline at end of file diff --git a/src/test/hive/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 b/src/test/hive/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 new file mode 100644 index 0000000000000..c36acd7ce80bc --- /dev/null +++ b/src/test/hive/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 @@ -0,0 +1 @@ +a <= b - Returns TRUE if a is not greater than b \ No newline at end of file diff --git a/src/test/hive/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c b/src/test/hive/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c new file mode 100644 index 0000000000000..38e013b6914c2 --- /dev/null +++ b/src/test/hive/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c @@ -0,0 +1 @@ +false true true true \ No newline at end of file diff --git a/src/test/hive/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 b/src/test/hive/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 new file mode 100644 index 0000000000000..b63094711df29 --- /dev/null +++ b/src/test/hive/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 @@ -0,0 +1 @@ +like(str, pattern) - Checks if str matches pattern diff --git a/src/test/hive/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a b/src/test/hive/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a new file mode 100644 index 0000000000000..640be910d969d --- /dev/null +++ b/src/test/hive/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a @@ -0,0 +1,4 @@ +like(str, pattern) - Checks if str matches pattern +Example: + > SELECT a.* FROM srcpart a WHERE a.hr like '%2' LIMIT 1; + 27 val_27 2008-04-08 12 diff --git a/src/test/hive/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 b/src/test/hive/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc b/src/test/hive/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc new file mode 100644 index 0000000000000..b186fa011a671 --- /dev/null +++ b/src/test/hive/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc @@ -0,0 +1 @@ +true false true true false false false false true false false false false true diff --git a/src/test/hive/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 b/src/test/hive/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 new file mode 100644 index 0000000000000..6cff4eb100134 --- /dev/null +++ b/src/test/hive/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 @@ -0,0 +1 @@ +true true false true false diff --git a/src/test/hive/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 b/src/test/hive/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 new file mode 100644 index 0000000000000..0b7372cb446d9 --- /dev/null +++ b/src/test/hive/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 @@ -0,0 +1 @@ +ln(x) - Returns the natural logarithm of x \ No newline at end of file diff --git a/src/test/hive/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 b/src/test/hive/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 new file mode 100644 index 0000000000000..bdf2f49ab17ca --- /dev/null +++ b/src/test/hive/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e b/src/test/hive/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e new file mode 100644 index 0000000000000..63b152162407b --- /dev/null +++ b/src/test/hive/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 b/src/test/hive/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 new file mode 100644 index 0000000000000..ea5465b67683e --- /dev/null +++ b/src/test/hive/golden/udf_locate-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c b/src/test/hive/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 b/src/test/hive/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 new file mode 100644 index 0000000000000..cd97bbb17fa81 --- /dev/null +++ b/src/test/hive/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a b/src/test/hive/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a new file mode 100644 index 0000000000000..e0dba6b06d9a4 --- /dev/null +++ b/src/test/hive/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a @@ -0,0 +1 @@ +log([b], x) - Returns the logarithm of x with base b \ No newline at end of file diff --git a/src/test/hive/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 b/src/test/hive/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 new file mode 100644 index 0000000000000..3dedfb4ddfd88 --- /dev/null +++ b/src/test/hive/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f b/src/test/hive/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f new file mode 100644 index 0000000000000..61749c48f158e --- /dev/null +++ b/src/test/hive/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f @@ -0,0 +1 @@ +log10(x) - Returns the logarithm of x with base 10 \ No newline at end of file diff --git a/src/test/hive/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 b/src/test/hive/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 new file mode 100644 index 0000000000000..68ce39dd2b084 --- /dev/null +++ b/src/test/hive/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 b/src/test/hive/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 new file mode 100644 index 0000000000000..177008a543627 --- /dev/null +++ b/src/test/hive/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 @@ -0,0 +1 @@ +log2(x) - Returns the logarithm of x with base 2 \ No newline at end of file diff --git a/src/test/hive/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 b/src/test/hive/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 new file mode 100644 index 0000000000000..4b856f91a7a14 --- /dev/null +++ b/src/test/hive/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 b/src/test/hive/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 new file mode 100644 index 0000000000000..d9d0dc035c9ab --- /dev/null +++ b/src/test/hive/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 b/src/test/hive/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 new file mode 100644 index 0000000000000..9bfc44922ea38 --- /dev/null +++ b/src/test/hive/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 @@ -0,0 +1 @@ +lower(str) - Returns str with all characters changed to lowercase \ No newline at end of file diff --git a/src/test/hive/golden/udf_lower-1-550f0a6da388596a775d921b9da995c b/src/test/hive/golden/udf_lower-1-550f0a6da388596a775d921b9da995c new file mode 100644 index 0000000000000..da912776ab009 --- /dev/null +++ b/src/test/hive/golden/udf_lower-1-550f0a6da388596a775d921b9da995c @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e b/src/test/hive/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 b/src/test/hive/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 new file mode 100644 index 0000000000000..9640d1794dcbc --- /dev/null +++ b/src/test/hive/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 @@ -0,0 +1 @@ +abc 123 ABC 123 \ No newline at end of file diff --git a/src/test/hive/golden/udf_lpad-0-995646acf1e23cea7825412915921bef b/src/test/hive/golden/udf_lpad-0-995646acf1e23cea7825412915921bef new file mode 100644 index 0000000000000..672d2d07f68b5 --- /dev/null +++ b/src/test/hive/golden/udf_lpad-0-995646acf1e23cea7825412915921bef @@ -0,0 +1 @@ +lpad(str, len, pad) - Returns str, left-padded with pad to a length of len \ No newline at end of file diff --git a/src/test/hive/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 b/src/test/hive/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 new file mode 100644 index 0000000000000..66db6624a376a --- /dev/null +++ b/src/test/hive/golden/udf_lpad-1-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' \ No newline at end of file diff --git a/src/test/hive/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 b/src/test/hive/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae b/src/test/hive/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae new file mode 100644 index 0000000000000..63a7235050df3 --- /dev/null +++ b/src/test/hive/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae @@ -0,0 +1 @@ +h ...hi 1231hi \ No newline at end of file diff --git a/src/test/hive/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f b/src/test/hive/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f new file mode 100644 index 0000000000000..0c9f6fd1c269e --- /dev/null +++ b/src/test/hive/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f @@ -0,0 +1 @@ +ltrim(str) - Removes the leading space characters from str \ No newline at end of file diff --git a/src/test/hive/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 b/src/test/hive/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 new file mode 100644 index 0000000000000..3ee2e690d28c1 --- /dev/null +++ b/src/test/hive/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e b/src/test/hive/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e new file mode 100644 index 0000000000000..2f31644f4c1cc --- /dev/null +++ b/src/test/hive/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e @@ -0,0 +1 @@ +map(key0, value0, key1, value1...) - Creates a map with the given key/value pairs diff --git a/src/test/hive/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf b/src/test/hive/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf new file mode 100644 index 0000000000000..2f31644f4c1cc --- /dev/null +++ b/src/test/hive/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf @@ -0,0 +1 @@ +map(key0, value0, key1, value1...) - Creates a map with the given key/value pairs diff --git a/src/test/hive/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 b/src/test/hive/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 b/src/test/hive/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 new file mode 100644 index 0000000000000..9f6becdf7d60f --- /dev/null +++ b/src/test/hive/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 @@ -0,0 +1 @@ +{} {1:"a",2:"b",3:"c"} {"1":"2","a":"b"} b b a diff --git a/src/test/hive/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb b/src/test/hive/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa b/src/test/hive/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa new file mode 100644 index 0000000000000..b88b74b978b8f --- /dev/null +++ b/src/test/hive/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa @@ -0,0 +1 @@ +map_keys(map) - Returns an unordered array containing the keys of the input map. \ No newline at end of file diff --git a/src/test/hive/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f b/src/test/hive/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f new file mode 100644 index 0000000000000..b88b74b978b8f --- /dev/null +++ b/src/test/hive/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f @@ -0,0 +1 @@ +map_keys(map) - Returns an unordered array containing the keys of the input map. \ No newline at end of file diff --git a/src/test/hive/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e b/src/test/hive/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e new file mode 100644 index 0000000000000..3a26a2e5e94d5 --- /dev/null +++ b/src/test/hive/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e @@ -0,0 +1 @@ +[1,2,3] \ No newline at end of file diff --git a/src/test/hive/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a b/src/test/hive/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a new file mode 100644 index 0000000000000..9b4c194f58a8e --- /dev/null +++ b/src/test/hive/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a @@ -0,0 +1 @@ +["b","a","c"] \ No newline at end of file diff --git a/src/test/hive/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb b/src/test/hive/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 b/src/test/hive/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 new file mode 100644 index 0000000000000..e811d1976a24c --- /dev/null +++ b/src/test/hive/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 @@ -0,0 +1 @@ +map_values(map) - Returns an unordered array containing the values of the input map. diff --git a/src/test/hive/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 b/src/test/hive/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 new file mode 100644 index 0000000000000..e811d1976a24c --- /dev/null +++ b/src/test/hive/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 @@ -0,0 +1 @@ +map_values(map) - Returns an unordered array containing the values of the input map. diff --git a/src/test/hive/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 b/src/test/hive/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 new file mode 100644 index 0000000000000..9d804e48987c6 --- /dev/null +++ b/src/test/hive/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 @@ -0,0 +1 @@ +["a","b","c"] diff --git a/src/test/hive/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e b/src/test/hive/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e new file mode 100644 index 0000000000000..be56bd1673eaa --- /dev/null +++ b/src/test/hive/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e @@ -0,0 +1 @@ +[2,1,3] diff --git a/src/test/hive/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f b/src/test/hive/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f new file mode 100644 index 0000000000000..c6cd9573ed8ed --- /dev/null +++ b/src/test/hive/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f @@ -0,0 +1 @@ +max(expr) - Returns the maximum value of expr diff --git a/src/test/hive/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 b/src/test/hive/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 new file mode 100644 index 0000000000000..c6cd9573ed8ed --- /dev/null +++ b/src/test/hive/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 @@ -0,0 +1 @@ +max(expr) - Returns the maximum value of expr diff --git a/src/test/hive/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 b/src/test/hive/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 new file mode 100644 index 0000000000000..21096f0f69749 --- /dev/null +++ b/src/test/hive/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 @@ -0,0 +1 @@ +{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/src/test/hive/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 b/src/test/hive/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 b/src/test/hive/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 new file mode 100644 index 0000000000000..21096f0f69749 --- /dev/null +++ b/src/test/hive/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 @@ -0,0 +1 @@ +{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/src/test/hive/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 b/src/test/hive/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e b/src/test/hive/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 b/src/test/hive/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 new file mode 100644 index 0000000000000..21096f0f69749 --- /dev/null +++ b/src/test/hive/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 @@ -0,0 +1 @@ +{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/src/test/hive/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e b/src/test/hive/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 b/src/test/hive/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 new file mode 100644 index 0000000000000..21096f0f69749 --- /dev/null +++ b/src/test/hive/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 @@ -0,0 +1 @@ +{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/src/test/hive/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 b/src/test/hive/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 b/src/test/hive/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 b/src/test/hive/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 new file mode 100644 index 0000000000000..0d59cabcde8fb --- /dev/null +++ b/src/test/hive/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 @@ -0,0 +1 @@ +min(expr) - Returns the minimum value of expr diff --git a/src/test/hive/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 b/src/test/hive/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 new file mode 100644 index 0000000000000..0d59cabcde8fb --- /dev/null +++ b/src/test/hive/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 @@ -0,0 +1 @@ +min(expr) - Returns the minimum value of expr diff --git a/src/test/hive/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 b/src/test/hive/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 new file mode 100644 index 0000000000000..b26a7e93c5d9f --- /dev/null +++ b/src/test/hive/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 @@ -0,0 +1 @@ +{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/src/test/hive/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 b/src/test/hive/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 b/src/test/hive/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 new file mode 100644 index 0000000000000..b26a7e93c5d9f --- /dev/null +++ b/src/test/hive/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 @@ -0,0 +1 @@ +{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/src/test/hive/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 b/src/test/hive/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e b/src/test/hive/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 b/src/test/hive/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 new file mode 100644 index 0000000000000..b26a7e93c5d9f --- /dev/null +++ b/src/test/hive/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 @@ -0,0 +1 @@ +{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/src/test/hive/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e b/src/test/hive/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 b/src/test/hive/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 new file mode 100644 index 0000000000000..b26a7e93c5d9f --- /dev/null +++ b/src/test/hive/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 @@ -0,0 +1 @@ +{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/src/test/hive/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 b/src/test/hive/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 b/src/test/hive/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee b/src/test/hive/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee new file mode 100644 index 0000000000000..b6bd6049a90b9 --- /dev/null +++ b/src/test/hive/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee @@ -0,0 +1 @@ +minute(date) - Returns the minute of date \ No newline at end of file diff --git a/src/test/hive/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 b/src/test/hive/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 new file mode 100644 index 0000000000000..64c3cda697e1e --- /dev/null +++ b/src/test/hive/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 @@ -0,0 +1,7 @@ +minute(date) - Returns the minute of date +date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'. +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 diff --git a/src/test/hive/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 b/src/test/hive/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_minute-3-270055c684846e87444b037226cf554c b/src/test/hive/golden/udf_minute-3-270055c684846e87444b037226cf554c new file mode 100644 index 0000000000000..5bd5433375d5c --- /dev/null +++ b/src/test/hive/golden/udf_minute-3-270055c684846e87444b037226cf554c @@ -0,0 +1 @@ +14 14 NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 b/src/test/hive/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 new file mode 100644 index 0000000000000..1bcf0c71008b0 --- /dev/null +++ b/src/test/hive/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 @@ -0,0 +1 @@ +a % b - Returns the remainder when dividing a by b \ No newline at end of file diff --git a/src/test/hive/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 b/src/test/hive/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 new file mode 100644 index 0000000000000..1bcf0c71008b0 --- /dev/null +++ b/src/test/hive/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 @@ -0,0 +1 @@ +a % b - Returns the remainder when dividing a by b \ No newline at end of file diff --git a/src/test/hive/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee b/src/test/hive/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee new file mode 100644 index 0000000000000..b6bd6049a90b9 --- /dev/null +++ b/src/test/hive/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee @@ -0,0 +1 @@ +minute(date) - Returns the minute of date \ No newline at end of file diff --git a/src/test/hive/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 b/src/test/hive/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 new file mode 100644 index 0000000000000..64c3cda697e1e --- /dev/null +++ b/src/test/hive/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 @@ -0,0 +1,7 @@ +minute(date) - Returns the minute of date +date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'. +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 diff --git a/src/test/hive/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 b/src/test/hive/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 new file mode 100644 index 0000000000000..3142f7f94be9e --- /dev/null +++ b/src/test/hive/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 @@ -0,0 +1 @@ +negative a - Returns -a \ No newline at end of file diff --git a/src/test/hive/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf b/src/test/hive/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf new file mode 100644 index 0000000000000..3142f7f94be9e --- /dev/null +++ b/src/test/hive/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf @@ -0,0 +1 @@ +negative a - Returns -a \ No newline at end of file diff --git a/src/test/hive/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c b/src/test/hive/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c new file mode 100644 index 0000000000000..0f4a4dce9dd3d --- /dev/null +++ b/src/test/hive/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c @@ -0,0 +1 @@ +a - b - Returns the difference a-b \ No newline at end of file diff --git a/src/test/hive/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 b/src/test/hive/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 new file mode 100644 index 0000000000000..0f4a4dce9dd3d --- /dev/null +++ b/src/test/hive/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 @@ -0,0 +1 @@ +a - b - Returns the difference a-b \ No newline at end of file diff --git a/src/test/hive/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c b/src/test/hive/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e b/src/test/hive/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 b/src/test/hive/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b b/src/test/hive/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 b/src/test/hive/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa b/src/test/hive/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc b/src/test/hive/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc new file mode 100644 index 0000000000000..1ad92784d47ac --- /dev/null +++ b/src/test/hive/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc @@ -0,0 +1 @@ +not a - Logical not \ No newline at end of file diff --git a/src/test/hive/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 b/src/test/hive/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 new file mode 100644 index 0000000000000..6520d74f02566 --- /dev/null +++ b/src/test/hive/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 @@ -0,0 +1,2 @@ +not a - Logical not +Synonyms: ! \ No newline at end of file diff --git a/src/test/hive/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 b/src/test/hive/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 new file mode 100644 index 0000000000000..c0e84e95d01c2 --- /dev/null +++ b/src/test/hive/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 @@ -0,0 +1 @@ +! a - Logical not \ No newline at end of file diff --git a/src/test/hive/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a b/src/test/hive/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a new file mode 100644 index 0000000000000..05cb1f6ab03d7 --- /dev/null +++ b/src/test/hive/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a @@ -0,0 +1,2 @@ +! a - Logical not +Synonyms: not \ No newline at end of file diff --git a/src/test/hive/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e b/src/test/hive/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e new file mode 100644 index 0000000000000..9e2108ff72b18 --- /dev/null +++ b/src/test/hive/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e @@ -0,0 +1 @@ +a <> b - Returns TRUE if a is not equal to b \ No newline at end of file diff --git a/src/test/hive/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a b/src/test/hive/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a new file mode 100644 index 0000000000000..162f808ed5b67 --- /dev/null +++ b/src/test/hive/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a @@ -0,0 +1,2 @@ +a <> b - Returns TRUE if a is not equal to b +Synonyms: != \ No newline at end of file diff --git a/src/test/hive/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 b/src/test/hive/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 new file mode 100644 index 0000000000000..2dab01a3b2765 --- /dev/null +++ b/src/test/hive/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 @@ -0,0 +1 @@ +a != b - Returns TRUE if a is not equal to b \ No newline at end of file diff --git a/src/test/hive/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 b/src/test/hive/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 new file mode 100644 index 0000000000000..1f73486ba2896 --- /dev/null +++ b/src/test/hive/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 @@ -0,0 +1,2 @@ +a != b - Returns TRUE if a is not equal to b +Synonyms: <> \ No newline at end of file diff --git a/src/test/hive/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 b/src/test/hive/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 b/src/test/hive/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 new file mode 100644 index 0000000000000..f82286d8bb069 --- /dev/null +++ b/src/test/hive/golden/udf_notequal-5-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 b/src/test/hive/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd b/src/test/hive/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd new file mode 100644 index 0000000000000..f82286d8bb069 --- /dev/null +++ b/src/test/hive/golden/udf_notequal-7-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 b/src/test/hive/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 new file mode 100644 index 0000000000000..a55e3339049e8 --- /dev/null +++ b/src/test/hive/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 @@ -0,0 +1 @@ +false true false false true false true false true true \ No newline at end of file diff --git a/src/test/hive/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 b/src/test/hive/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 new file mode 100644 index 0000000000000..5ffcb851d56f9 --- /dev/null +++ b/src/test/hive/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 @@ -0,0 +1 @@ +nvl(value,default_value) - Returns default value if value is null else returns value \ No newline at end of file diff --git a/src/test/hive/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 b/src/test/hive/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 new file mode 100644 index 0000000000000..5afff220da695 --- /dev/null +++ b/src/test/hive/golden/udf_nvl-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def b/src/test/hive/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 b/src/test/hive/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 new file mode 100644 index 0000000000000..2087e17494459 --- /dev/null +++ b/src/test/hive/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 @@ -0,0 +1 @@ +1 5 \ No newline at end of file diff --git a/src/test/hive/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 b/src/test/hive/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 new file mode 100644 index 0000000000000..9138f44ad2a41 --- /dev/null +++ b/src/test/hive/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 @@ -0,0 +1 @@ +a or b - Logical or \ No newline at end of file diff --git a/src/test/hive/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 b/src/test/hive/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 new file mode 100644 index 0000000000000..9138f44ad2a41 --- /dev/null +++ b/src/test/hive/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 @@ -0,0 +1 @@ +a or b - Logical or \ No newline at end of file diff --git a/src/test/hive/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 b/src/test/hive/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 new file mode 100644 index 0000000000000..0535b085e50c2 --- /dev/null +++ b/src/test/hive/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 @@ -0,0 +1 @@ +parse_url(url, partToExtract[, key]) - extracts a part from a URL \ No newline at end of file diff --git a/src/test/hive/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab b/src/test/hive/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab new file mode 100644 index 0000000000000..7178e07a07e48 --- /dev/null +++ b/src/test/hive/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab @@ -0,0 +1,10 @@ +parse_url(url, partToExtract[, key]) - extracts a part from a URL +Parts: HOST, PATH, QUERY, REF, PROTOCOL, AUTHORITY, FILE, USERINFO +key specifies which query to extract +Example: + > SELECT parse_url('http://facebook.com/path/p1.php?query=1', 'HOST') FROM src LIMIT 1; + 'facebook.com' + > 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 diff --git a/src/test/hive/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d b/src/test/hive/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 b/src/test/hive/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 new file mode 100644 index 0000000000000..f5483d4e3dafe --- /dev/null +++ b/src/test/hive/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 b/src/test/hive/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 new file mode 100644 index 0000000000000..2025042f5d493 --- /dev/null +++ b/src/test/hive/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a b/src/test/hive/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a new file mode 100644 index 0000000000000..2025042f5d493 --- /dev/null +++ b/src/test/hive/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 b/src/test/hive/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e b/src/test/hive/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 b/src/test/hive/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 new file mode 100644 index 0000000000000..5d2fc352ee060 --- /dev/null +++ b/src/test/hive/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 @@ -0,0 +1 @@ +a pmod b - Compute the positive modulo diff --git a/src/test/hive/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae b/src/test/hive/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae new file mode 100644 index 0000000000000..5d2fc352ee060 --- /dev/null +++ b/src/test/hive/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae @@ -0,0 +1 @@ +a pmod b - Compute the positive modulo diff --git a/src/test/hive/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 b/src/test/hive/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 new file mode 100644 index 0000000000000..0b46af11c4516 --- /dev/null +++ b/src/test/hive/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 @@ -0,0 +1 @@ +6.89 51.7 18.09 diff --git a/src/test/hive/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 b/src/test/hive/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 b/src/test/hive/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 new file mode 100644 index 0000000000000..5eb0813b60eb6 --- /dev/null +++ b/src/test/hive/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 @@ -0,0 +1 @@ +8 51 15 diff --git a/src/test/hive/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 b/src/test/hive/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 new file mode 100644 index 0000000000000..e21e4b08e7a62 --- /dev/null +++ b/src/test/hive/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 @@ -0,0 +1 @@ +5 50 0 diff --git a/src/test/hive/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 b/src/test/hive/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/src/test/hive/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 @@ -0,0 +1 @@ +8 51 16 diff --git a/src/test/hive/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 b/src/test/hive/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/src/test/hive/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 @@ -0,0 +1 @@ +8 51 16 diff --git a/src/test/hive/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 b/src/test/hive/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/src/test/hive/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 @@ -0,0 +1 @@ +8 51 16 diff --git a/src/test/hive/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd b/src/test/hive/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd new file mode 100644 index 0000000000000..48371142e9b5d --- /dev/null +++ b/src/test/hive/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd @@ -0,0 +1 @@ +6.8899984 51.700005 18.089996 diff --git a/src/test/hive/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 b/src/test/hive/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 new file mode 100644 index 0000000000000..ab842acd48b3c --- /dev/null +++ b/src/test/hive/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 @@ -0,0 +1 @@ +6.890000000000011 51.699999999999996 18.090000000000003 diff --git a/src/test/hive/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 b/src/test/hive/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 new file mode 100644 index 0000000000000..7c98729cc195c --- /dev/null +++ b/src/test/hive/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 @@ -0,0 +1 @@ +positive a - Returns a \ No newline at end of file diff --git a/src/test/hive/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c b/src/test/hive/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c new file mode 100644 index 0000000000000..7c98729cc195c --- /dev/null +++ b/src/test/hive/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c @@ -0,0 +1 @@ +positive a - Returns a \ No newline at end of file diff --git a/src/test/hive/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 b/src/test/hive/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 new file mode 100644 index 0000000000000..e8f11c444a808 --- /dev/null +++ b/src/test/hive/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 @@ -0,0 +1 @@ +a + b - Returns a+b \ No newline at end of file diff --git a/src/test/hive/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 b/src/test/hive/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 new file mode 100644 index 0000000000000..e8f11c444a808 --- /dev/null +++ b/src/test/hive/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 @@ -0,0 +1 @@ +a + b - Returns a+b \ No newline at end of file diff --git a/src/test/hive/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 b/src/test/hive/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 new file mode 100644 index 0000000000000..43f197e360c82 --- /dev/null +++ b/src/test/hive/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 @@ -0,0 +1 @@ +pow(x1, x2) - raise x1 to the power of x2 \ No newline at end of file diff --git a/src/test/hive/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 b/src/test/hive/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 new file mode 100644 index 0000000000000..afc2e4462a1a8 --- /dev/null +++ b/src/test/hive/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/udf_power-0-57001d802c281743322d28bbc520cd4 b/src/test/hive/golden/udf_power-0-57001d802c281743322d28bbc520cd4 new file mode 100644 index 0000000000000..5e3a6a8f31fd3 --- /dev/null +++ b/src/test/hive/golden/udf_power-0-57001d802c281743322d28bbc520cd4 @@ -0,0 +1 @@ +power(x1, x2) - raise x1 to the power of x2 \ No newline at end of file diff --git a/src/test/hive/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 b/src/test/hive/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 new file mode 100644 index 0000000000000..c3414e29768e3 --- /dev/null +++ b/src/test/hive/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 b/src/test/hive/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 b/src/test/hive/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 new file mode 100644 index 0000000000000..116b6bc461ed0 --- /dev/null +++ b/src/test/hive/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 @@ -0,0 +1 @@ +1.000000357564167 \ No newline at end of file diff --git a/src/test/hive/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 b/src/test/hive/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 new file mode 100644 index 0000000000000..aaf3b31fef488 --- /dev/null +++ b/src/test/hive/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 @@ -0,0 +1 @@ +2.4999991485811655 \ No newline at end of file diff --git a/src/test/hive/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 b/src/test/hive/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 new file mode 100644 index 0000000000000..73ad88be4ef3d --- /dev/null +++ b/src/test/hive/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 @@ -0,0 +1 @@ +radians(x) - Converts degrees to radians \ No newline at end of file diff --git a/src/test/hive/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/src/test/hive/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 new file mode 100644 index 0000000000000..e0237c0058f55 --- /dev/null +++ b/src/test/hive/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 @@ -0,0 +1,4 @@ +radians(x) - Converts degrees to radians +Example: + > SELECT radians(90) FROM src LIMIT 1; + 1.5707963267949mo diff --git a/src/test/hive/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 b/src/test/hive/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 b/src/test/hive/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 new file mode 100644 index 0000000000000..116b6bc461ed0 --- /dev/null +++ b/src/test/hive/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 @@ -0,0 +1 @@ +1.000000357564167 \ No newline at end of file diff --git a/src/test/hive/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 b/src/test/hive/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 new file mode 100644 index 0000000000000..aaf3b31fef488 --- /dev/null +++ b/src/test/hive/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 @@ -0,0 +1 @@ +2.4999991485811655 \ No newline at end of file diff --git a/src/test/hive/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 b/src/test/hive/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 new file mode 100644 index 0000000000000..73ad88be4ef3d --- /dev/null +++ b/src/test/hive/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 @@ -0,0 +1 @@ +radians(x) - Converts degrees to radians \ No newline at end of file diff --git a/src/test/hive/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/src/test/hive/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 new file mode 100644 index 0000000000000..e0237c0058f55 --- /dev/null +++ b/src/test/hive/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 @@ -0,0 +1,4 @@ +radians(x) - Converts degrees to radians +Example: + > SELECT radians(90) FROM src LIMIT 1; + 1.5707963267949mo diff --git a/src/test/hive/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 b/src/test/hive/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 new file mode 100644 index 0000000000000..d19c5b50fe5c4 --- /dev/null +++ b/src/test/hive/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 @@ -0,0 +1 @@ +rand([seed]) - Returns a pseudorandom number between 0 and 1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e b/src/test/hive/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e new file mode 100644 index 0000000000000..d19c5b50fe5c4 --- /dev/null +++ b/src/test/hive/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e @@ -0,0 +1 @@ +rand([seed]) - Returns a pseudorandom number between 0 and 1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 b/src/test/hive/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 new file mode 100644 index 0000000000000..1d7658151cd62 --- /dev/null +++ b/src/test/hive/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 @@ -0,0 +1 @@ +reflect(class,method[,arg1[,arg2..]]) calls method with reflection diff --git a/src/test/hive/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee b/src/test/hive/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee new file mode 100644 index 0000000000000..ddf986c01e5b1 --- /dev/null +++ b/src/test/hive/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee @@ -0,0 +1,4 @@ +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/src/test/hive/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 b/src/test/hive/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 b/src/test/hive/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 new file mode 100644 index 0000000000000..cd35e5b290db5 --- /dev/null +++ b/src/test/hive/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 @@ -0,0 +1 @@ +reflect2(arg0,method[,arg1[,arg2..]]) calls method of arg0 with reflection diff --git a/src/test/hive/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 b/src/test/hive/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 new file mode 100644 index 0000000000000..48ef97292ab62 --- /dev/null +++ b/src/test/hive/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 @@ -0,0 +1,3 @@ +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/src/test/hive/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca b/src/test/hive/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 b/src/test/hive/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 new file mode 100644 index 0000000000000..3e2abd349cef7 --- /dev/null +++ b/src/test/hive/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 @@ -0,0 +1 @@ +str regexp regexp - Returns true if str matches regexp and false otherwise diff --git a/src/test/hive/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 b/src/test/hive/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 new file mode 100644 index 0000000000000..d6b5d52cd1be5 --- /dev/null +++ b/src/test/hive/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 @@ -0,0 +1,5 @@ +str regexp regexp - Returns true if str matches regexp and false otherwise +Synonyms: rlike +Example: + > SELECT 'fb' regexp '.*' FROM src LIMIT 1; + true diff --git a/src/test/hive/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 b/src/test/hive/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 new file mode 100644 index 0000000000000..3c4f7ed353b35 --- /dev/null +++ b/src/test/hive/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 @@ -0,0 +1 @@ +true true true true false false false false diff --git a/src/test/hive/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 b/src/test/hive/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 new file mode 100644 index 0000000000000..429057caf71f0 --- /dev/null +++ b/src/test/hive/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 @@ -0,0 +1 @@ +regexp_extract(str, regexp[, idx]) - extracts a group that matches regexp \ No newline at end of file diff --git a/src/test/hive/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb b/src/test/hive/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb new file mode 100644 index 0000000000000..30e237490811f --- /dev/null +++ b/src/test/hive/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e b/src/test/hive/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e new file mode 100644 index 0000000000000..1a38701d68875 --- /dev/null +++ b/src/test/hive/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e @@ -0,0 +1 @@ +regexp_replace(str, regexp, rep) - replace all substrings of str that match regexp with rep \ No newline at end of file diff --git a/src/test/hive/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc b/src/test/hive/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc new file mode 100644 index 0000000000000..bebfabaf637fb --- /dev/null +++ b/src/test/hive/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b b/src/test/hive/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b new file mode 100644 index 0000000000000..694c367436f3c --- /dev/null +++ b/src/test/hive/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b @@ -0,0 +1 @@ +repeat(str, n) - repeat str n times \ No newline at end of file diff --git a/src/test/hive/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b b/src/test/hive/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b new file mode 100644 index 0000000000000..5df19ba1c2cbf --- /dev/null +++ b/src/test/hive/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b @@ -0,0 +1,4 @@ +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/src/test/hive/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a b/src/test/hive/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 b/src/test/hive/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 new file mode 100644 index 0000000000000..45425cf087c09 --- /dev/null +++ b/src/test/hive/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 @@ -0,0 +1 @@ +FacebookFacebookFacebook \ No newline at end of file diff --git a/src/test/hive/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 b/src/test/hive/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 new file mode 100644 index 0000000000000..fd9ac3081a1cc --- /dev/null +++ b/src/test/hive/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 @@ -0,0 +1 @@ +str rlike regexp - Returns true if str matches regexp and false otherwise \ No newline at end of file diff --git a/src/test/hive/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca b/src/test/hive/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca new file mode 100644 index 0000000000000..43b06945caa54 --- /dev/null +++ b/src/test/hive/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 b/src/test/hive/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 new file mode 100644 index 0000000000000..e4586b2e73a93 --- /dev/null +++ b/src/test/hive/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 @@ -0,0 +1 @@ +round(x[, d]) - round x to d decimal places \ No newline at end of file diff --git a/src/test/hive/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 b/src/test/hive/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 new file mode 100644 index 0000000000000..c0d5b480e9751 --- /dev/null +++ b/src/test/hive/golden/udf_round-1-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' \ No newline at end of file diff --git a/src/test/hive/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 b/src/test/hive/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 new file mode 100644 index 0000000000000..0924e3869076d --- /dev/null +++ b/src/test/hive/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 @@ -0,0 +1 @@ +NULL NULL NULL Infinity NaN \ No newline at end of file diff --git a/src/test/hive/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c b/src/test/hive/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c new file mode 100644 index 0000000000000..3b9c30929a240 --- /dev/null +++ b/src/test/hive/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 b/src/test/hive/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 new file mode 100644 index 0000000000000..1f243f6cbc6db --- /dev/null +++ b/src/test/hive/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a b/src/test/hive/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a new file mode 100644 index 0000000000000..918404cec8047 --- /dev/null +++ b/src/test/hive/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a @@ -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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b b/src/test/hive/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b new file mode 100644 index 0000000000000..af105563af144 --- /dev/null +++ b/src/test/hive/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b @@ -0,0 +1 @@ +1809242.315111134 -1809242.315111134 \ No newline at end of file diff --git a/src/test/hive/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 b/src/test/hive/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce b/src/test/hive/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 b/src/test/hive/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/src/test/hive/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/src/test/hive/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 b/src/test/hive/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/src/test/hive/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/src/test/hive/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 b/src/test/hive/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/src/test/hive/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/src/test/hive/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b b/src/test/hive/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b new file mode 100644 index 0000000000000..f40e633f703c9 --- /dev/null +++ b/src/test/hive/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b @@ -0,0 +1 @@ +Infinity Infinity Infinity Infinity \ No newline at end of file diff --git a/src/test/hive/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c b/src/test/hive/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c new file mode 100644 index 0000000000000..3714de0db18dc --- /dev/null +++ b/src/test/hive/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c @@ -0,0 +1 @@ +-128 127 0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 b/src/test/hive/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 new file mode 100644 index 0000000000000..a9265d7b8a1b7 --- /dev/null +++ b/src/test/hive/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 @@ -0,0 +1 @@ +-32768 32767 -129 128 \ No newline at end of file diff --git a/src/test/hive/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 b/src/test/hive/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 new file mode 100644 index 0000000000000..2d25f54073df7 --- /dev/null +++ b/src/test/hive/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 @@ -0,0 +1 @@ +-2147483648 2147483647 -32769 32768 \ No newline at end of file diff --git a/src/test/hive/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 b/src/test/hive/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 new file mode 100644 index 0000000000000..41a2624c6cfeb --- /dev/null +++ b/src/test/hive/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 @@ -0,0 +1 @@ +-9223372036854775808 9223372036854775807 -2147483649 2147483648 \ No newline at end of file diff --git a/src/test/hive/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 b/src/test/hive/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 new file mode 100644 index 0000000000000..98d3f53dfc442 --- /dev/null +++ b/src/test/hive/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 @@ -0,0 +1 @@ +126.0 127.0 32766.0 32767.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c b/src/test/hive/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c new file mode 100644 index 0000000000000..7cb2d71d4b80d --- /dev/null +++ b/src/test/hive/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c @@ -0,0 +1 @@ +rpad(str, len, pad) - Returns str, right-padded with pad to a length of len \ No newline at end of file diff --git a/src/test/hive/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 b/src/test/hive/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 new file mode 100644 index 0000000000000..2b198b8dc96c4 --- /dev/null +++ b/src/test/hive/golden/udf_rpad-1-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' \ No newline at end of file diff --git a/src/test/hive/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 b/src/test/hive/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 b/src/test/hive/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 new file mode 100644 index 0000000000000..b2645e365c8a9 --- /dev/null +++ b/src/test/hive/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 @@ -0,0 +1 @@ +h hi... hi1231 \ No newline at end of file diff --git a/src/test/hive/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a b/src/test/hive/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a new file mode 100644 index 0000000000000..94d6aeed8f533 --- /dev/null +++ b/src/test/hive/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a @@ -0,0 +1 @@ +rtrim(str) - Removes the trailing space characters from str \ No newline at end of file diff --git a/src/test/hive/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd b/src/test/hive/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd new file mode 100644 index 0000000000000..957e608a7c732 --- /dev/null +++ b/src/test/hive/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 b/src/test/hive/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 new file mode 100644 index 0000000000000..577c90254cb5a --- /dev/null +++ b/src/test/hive/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 @@ -0,0 +1 @@ +second(date) - Returns the second of date \ No newline at end of file diff --git a/src/test/hive/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 b/src/test/hive/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 new file mode 100644 index 0000000000000..4b48294e5b9ad --- /dev/null +++ b/src/test/hive/golden/udf_second-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_second-2-d678372e3837a16be245d2e33482f17f b/src/test/hive/golden/udf_second-2-d678372e3837a16be245d2e33482f17f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf b/src/test/hive/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf new file mode 100644 index 0000000000000..4d5ef5cf4a699 --- /dev/null +++ b/src/test/hive/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf @@ -0,0 +1 @@ +15 15 NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f b/src/test/hive/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 b/src/test/hive/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 b/src/test/hive/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 b/src/test/hive/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 new file mode 100644 index 0000000000000..fea4f860c8465 --- /dev/null +++ b/src/test/hive/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 @@ -0,0 +1,31 @@ +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/src/test/hive/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 b/src/test/hive/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 new file mode 100644 index 0000000000000..c49ca6bb5a1b0 --- /dev/null +++ b/src/test/hive/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 @@ -0,0 +1 @@ +[["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/src/test/hive/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 b/src/test/hive/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 b/src/test/hive/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 b/src/test/hive/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 new file mode 100644 index 0000000000000..b798628e56686 --- /dev/null +++ b/src/test/hive/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 @@ -0,0 +1,50 @@ +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/src/test/hive/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 b/src/test/hive/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de b/src/test/hive/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f b/src/test/hive/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 b/src/test/hive/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 b/src/test/hive/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e b/src/test/hive/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 b/src/test/hive/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/src/test/hive/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa b/src/test/hive/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa new file mode 100644 index 0000000000000..3345fbb8f2c35 --- /dev/null +++ b/src/test/hive/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa @@ -0,0 +1 @@ +sign(x) - returns the sign of x ) \ No newline at end of file diff --git a/src/test/hive/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a b/src/test/hive/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a new file mode 100644 index 0000000000000..bdacec8810e77 --- /dev/null +++ b/src/test/hive/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a @@ -0,0 +1,4 @@ +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/src/test/hive/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef b/src/test/hive/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef new file mode 100644 index 0000000000000..31a5b0b81dc51 --- /dev/null +++ b/src/test/hive/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef @@ -0,0 +1 @@ +-1.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d b/src/test/hive/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d new file mode 100644 index 0000000000000..9f8e9b69a33f4 --- /dev/null +++ b/src/test/hive/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d @@ -0,0 +1 @@ +1.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa b/src/test/hive/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa new file mode 100644 index 0000000000000..3345fbb8f2c35 --- /dev/null +++ b/src/test/hive/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa @@ -0,0 +1 @@ +sign(x) - returns the sign of x ) \ No newline at end of file diff --git a/src/test/hive/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a b/src/test/hive/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a new file mode 100644 index 0000000000000..bdacec8810e77 --- /dev/null +++ b/src/test/hive/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a @@ -0,0 +1,4 @@ +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/src/test/hive/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e b/src/test/hive/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 b/src/test/hive/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/src/test/hive/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef b/src/test/hive/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef new file mode 100644 index 0000000000000..31a5b0b81dc51 --- /dev/null +++ b/src/test/hive/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef @@ -0,0 +1 @@ +-1.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d b/src/test/hive/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d new file mode 100644 index 0000000000000..9f8e9b69a33f4 --- /dev/null +++ b/src/test/hive/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d @@ -0,0 +1 @@ +1.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee b/src/test/hive/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee new file mode 100644 index 0000000000000..6155d9a2eae55 --- /dev/null +++ b/src/test/hive/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee @@ -0,0 +1 @@ +sin(x) - returns the sine of x (x is in radians) \ No newline at end of file diff --git a/src/test/hive/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 b/src/test/hive/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 new file mode 100644 index 0000000000000..a39ed8840b916 --- /dev/null +++ b/src/test/hive/golden/udf_sin-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab b/src/test/hive/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 b/src/test/hive/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 new file mode 100644 index 0000000000000..ee21925b0ccc5 --- /dev/null +++ b/src/test/hive/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 @@ -0,0 +1 @@ +0.8304973704919705 0.9999996829318346 -0.479425538604203 \ No newline at end of file diff --git a/src/test/hive/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 b/src/test/hive/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 new file mode 100644 index 0000000000000..d8952629349a9 --- /dev/null +++ b/src/test/hive/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 @@ -0,0 +1 @@ +size(a) - Returns the size of a diff --git a/src/test/hive/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 b/src/test/hive/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 new file mode 100644 index 0000000000000..d8952629349a9 --- /dev/null +++ b/src/test/hive/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 @@ -0,0 +1 @@ +size(a) - Returns the size of a diff --git a/src/test/hive/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 b/src/test/hive/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd b/src/test/hive/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd new file mode 100644 index 0000000000000..08a708b090282 --- /dev/null +++ b/src/test/hive/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd @@ -0,0 +1 @@ +3 1 1 -1 diff --git a/src/test/hive/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 b/src/test/hive/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 new file mode 100644 index 0000000000000..572ecb0462eb7 --- /dev/null +++ b/src/test/hive/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 @@ -0,0 +1 @@ +There is no documentation for function 'smallint' \ No newline at end of file diff --git a/src/test/hive/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 b/src/test/hive/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 new file mode 100644 index 0000000000000..572ecb0462eb7 --- /dev/null +++ b/src/test/hive/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 @@ -0,0 +1 @@ +There is no documentation for function 'smallint' \ No newline at end of file diff --git a/src/test/hive/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb b/src/test/hive/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 b/src/test/hive/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 new file mode 100644 index 0000000000000..d514df4191b89 --- /dev/null +++ b/src/test/hive/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 @@ -0,0 +1 @@ +sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements. diff --git a/src/test/hive/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 b/src/test/hive/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 new file mode 100644 index 0000000000000..43e36513de881 --- /dev/null +++ b/src/test/hive/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de b/src/test/hive/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 b/src/test/hive/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 new file mode 100644 index 0000000000000..2e9458debfd0f --- /dev/null +++ b/src/test/hive/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 @@ -0,0 +1 @@ +["a","b","c","d","e","f","g"] diff --git a/src/test/hive/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e b/src/test/hive/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e new file mode 100644 index 0000000000000..94f18d09863a7 --- /dev/null +++ b/src/test/hive/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e @@ -0,0 +1 @@ +["enterprise databases","hadoop distributed file system","hadoop map-reduce"] diff --git a/src/test/hive/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 b/src/test/hive/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 new file mode 100644 index 0000000000000..e1968ef44a2eb --- /dev/null +++ b/src/test/hive/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 @@ -0,0 +1 @@ +[1,2,3,4,5,6,7,8,9] diff --git a/src/test/hive/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 b/src/test/hive/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 new file mode 100644 index 0000000000000..b6c12ace4162b --- /dev/null +++ b/src/test/hive/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 @@ -0,0 +1 @@ +[-3.445,0.777,1.0,1.325,2.003,2.333,9.0] diff --git a/src/test/hive/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 b/src/test/hive/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a b/src/test/hive/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 b/src/test/hive/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 new file mode 100644 index 0000000000000..ed5bda787df23 --- /dev/null +++ b/src/test/hive/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 @@ -0,0 +1 @@ +space(n) - returns n spaces \ No newline at end of file diff --git a/src/test/hive/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 b/src/test/hive/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 new file mode 100644 index 0000000000000..5713d4b0464be --- /dev/null +++ b/src/test/hive/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 @@ -0,0 +1,4 @@ +space(n) - returns n spaces +Example: + > SELECT space(2) FROM src LIMIT 1; + ' ' \ No newline at end of file diff --git a/src/test/hive/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 b/src/test/hive/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 b/src/test/hive/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 new file mode 100644 index 0000000000000..85a16b2abe5ef --- /dev/null +++ b/src/test/hive/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 @@ -0,0 +1 @@ +10 0 1 0 0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 b/src/test/hive/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 new file mode 100644 index 0000000000000..8dfaf2745f666 --- /dev/null +++ b/src/test/hive/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/src/test/hive/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 b/src/test/hive/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 new file mode 100644 index 0000000000000..6b183ccfb17b4 --- /dev/null +++ b/src/test/hive/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 @@ -0,0 +1 @@ +split(str, regex) - Splits str around occurances that match regex \ No newline at end of file diff --git a/src/test/hive/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 b/src/test/hive/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 new file mode 100644 index 0000000000000..9e5c522da155b --- /dev/null +++ b/src/test/hive/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 b/src/test/hive/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 b/src/test/hive/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 new file mode 100644 index 0000000000000..9174f1a92557e --- /dev/null +++ b/src/test/hive/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 @@ -0,0 +1 @@ +["a","b","c"] ["one","two","three"] [] ["5","4","1","2"] \ No newline at end of file diff --git a/src/test/hive/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 b/src/test/hive/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 new file mode 100644 index 0000000000000..eadd6eeb8b149 --- /dev/null +++ b/src/test/hive/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 @@ -0,0 +1 @@ +sqrt(x) - returns the square root of x \ No newline at end of file diff --git a/src/test/hive/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 b/src/test/hive/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 new file mode 100644 index 0000000000000..74df32f221278 --- /dev/null +++ b/src/test/hive/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b b/src/test/hive/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b new file mode 100644 index 0000000000000..2966c01c1b7b2 --- /dev/null +++ b/src/test/hive/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b @@ -0,0 +1 @@ +std(x) - Returns the standard deviation of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 b/src/test/hive/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 new file mode 100644 index 0000000000000..388eaa1c7439a --- /dev/null +++ b/src/test/hive/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 @@ -0,0 +1,2 @@ +std(x) - Returns the standard deviation of a set of numbers +Synonyms: stddev_pop, stddev \ No newline at end of file diff --git a/src/test/hive/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 b/src/test/hive/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 new file mode 100644 index 0000000000000..74c6e1eacc379 --- /dev/null +++ b/src/test/hive/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 @@ -0,0 +1 @@ +stddev(x) - Returns the standard deviation of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d b/src/test/hive/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d new file mode 100644 index 0000000000000..1c461b68c1440 --- /dev/null +++ b/src/test/hive/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d @@ -0,0 +1,2 @@ +stddev(x) - Returns the standard deviation of a set of numbers +Synonyms: stddev_pop, std \ No newline at end of file diff --git a/src/test/hive/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 b/src/test/hive/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 new file mode 100644 index 0000000000000..741771707f280 --- /dev/null +++ b/src/test/hive/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 @@ -0,0 +1 @@ +Function 'udf_stddev_pop' does not exist. \ No newline at end of file diff --git a/src/test/hive/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 b/src/test/hive/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 new file mode 100644 index 0000000000000..741771707f280 --- /dev/null +++ b/src/test/hive/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 @@ -0,0 +1 @@ +Function 'udf_stddev_pop' does not exist. \ No newline at end of file diff --git a/src/test/hive/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 b/src/test/hive/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 new file mode 100644 index 0000000000000..c1fb091d2c425 --- /dev/null +++ b/src/test/hive/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 @@ -0,0 +1 @@ +stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 b/src/test/hive/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 new file mode 100644 index 0000000000000..c1fb091d2c425 --- /dev/null +++ b/src/test/hive/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 @@ -0,0 +1 @@ +stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 b/src/test/hive/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 new file mode 100644 index 0000000000000..c1fb091d2c425 --- /dev/null +++ b/src/test/hive/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 @@ -0,0 +1 @@ +stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 b/src/test/hive/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 new file mode 100644 index 0000000000000..c1fb091d2c425 --- /dev/null +++ b/src/test/hive/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 @@ -0,0 +1 @@ +stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 b/src/test/hive/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 new file mode 100644 index 0000000000000..c61c4e1135410 --- /dev/null +++ b/src/test/hive/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 @@ -0,0 +1 @@ +There is no documentation for function 'string' \ No newline at end of file diff --git a/src/test/hive/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d b/src/test/hive/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d new file mode 100644 index 0000000000000..c61c4e1135410 --- /dev/null +++ b/src/test/hive/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d @@ -0,0 +1 @@ +There is no documentation for function 'string' \ No newline at end of file diff --git a/src/test/hive/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd b/src/test/hive/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd new file mode 100644 index 0000000000000..c4efdaebc8aab --- /dev/null +++ b/src/test/hive/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 b/src/test/hive/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 new file mode 100644 index 0000000000000..44958996269c9 --- /dev/null +++ b/src/test/hive/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 b/src/test/hive/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 new file mode 100644 index 0000000000000..9516b839a3956 --- /dev/null +++ b/src/test/hive/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 @@ -0,0 +1 @@ +NULL NULL NULL NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 b/src/test/hive/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 new file mode 100644 index 0000000000000..7f6c057b5eebd --- /dev/null +++ b/src/test/hive/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/src/test/hive/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a b/src/test/hive/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a new file mode 100644 index 0000000000000..042ca15005869 --- /dev/null +++ b/src/test/hive/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a @@ -0,0 +1 @@ +CDEF CDEF CDEFG CDEFG ABC ABC BC C ABC BC A A A \ No newline at end of file diff --git a/src/test/hive/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 b/src/test/hive/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 new file mode 100644 index 0000000000000..85346dc923b4a --- /dev/null +++ b/src/test/hive/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 b/src/test/hive/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 new file mode 100644 index 0000000000000..9d607f4d84214 --- /dev/null +++ b/src/test/hive/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 @@ -0,0 +1 @@ +C C C C B BC BC BC A AB ABC ABC \ No newline at end of file diff --git a/src/test/hive/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 b/src/test/hive/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 new file mode 100644 index 0000000000000..042ca15005869 --- /dev/null +++ b/src/test/hive/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 @@ -0,0 +1 @@ +CDEF CDEF CDEFG CDEFG ABC ABC BC C ABC BC A A A \ No newline at end of file diff --git a/src/test/hive/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece b/src/test/hive/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece new file mode 100644 index 0000000000000..b0ade52e69a9d --- /dev/null +++ b/src/test/hive/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 b/src/test/hive/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 new file mode 100644 index 0000000000000..1d61dc4518087 --- /dev/null +++ b/src/test/hive/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 @@ -0,0 +1,10 @@ +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 +Synonyms: substr +pos is a 1-based index. If pos<0 the starting position is determined by counting backwards from the end of str. +Example: + > SELECT substring('Facebook', 5) FROM src LIMIT 1; + 'book' + > 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 diff --git a/src/test/hive/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 b/src/test/hive/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 new file mode 100644 index 0000000000000..0f4a4dce9dd3d --- /dev/null +++ b/src/test/hive/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 @@ -0,0 +1 @@ +a - b - Returns the difference a-b \ No newline at end of file diff --git a/src/test/hive/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 b/src/test/hive/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 new file mode 100644 index 0000000000000..0f4a4dce9dd3d --- /dev/null +++ b/src/test/hive/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 @@ -0,0 +1 @@ +a - b - Returns the difference a-b \ No newline at end of file diff --git a/src/test/hive/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 b/src/test/hive/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 new file mode 100644 index 0000000000000..744935026110c --- /dev/null +++ b/src/test/hive/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 @@ -0,0 +1 @@ +sum(x) - Returns the sum of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b b/src/test/hive/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b new file mode 100644 index 0000000000000..744935026110c --- /dev/null +++ b/src/test/hive/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b @@ -0,0 +1 @@ +sum(x) - Returns the sum of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 b/src/test/hive/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 new file mode 100644 index 0000000000000..744935026110c --- /dev/null +++ b/src/test/hive/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 @@ -0,0 +1 @@ +sum(x) - Returns the sum of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b b/src/test/hive/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b new file mode 100644 index 0000000000000..744935026110c --- /dev/null +++ b/src/test/hive/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b @@ -0,0 +1 @@ +sum(x) - Returns the sum of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 b/src/test/hive/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 new file mode 100644 index 0000000000000..c91072e69740b --- /dev/null +++ b/src/test/hive/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 @@ -0,0 +1 @@ +tan(x) - returns the tangent of x (x is in radians) \ No newline at end of file diff --git a/src/test/hive/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/src/test/hive/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 new file mode 100644 index 0000000000000..4cc0d466f00f4 --- /dev/null +++ b/src/test/hive/golden/udf_tan-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 b/src/test/hive/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 b/src/test/hive/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 new file mode 100644 index 0000000000000..04ec33c11c3b4 --- /dev/null +++ b/src/test/hive/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 @@ -0,0 +1 @@ +1.5574077246549023 -0.29100619138474915 -1.5574077246549023 \ No newline at end of file diff --git a/src/test/hive/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 b/src/test/hive/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 new file mode 100644 index 0000000000000..c91072e69740b --- /dev/null +++ b/src/test/hive/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 @@ -0,0 +1 @@ +tan(x) - returns the tangent of x (x is in radians) \ No newline at end of file diff --git a/src/test/hive/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/src/test/hive/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 new file mode 100644 index 0000000000000..4cc0d466f00f4 --- /dev/null +++ b/src/test/hive/golden/udf_tan-5-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 b/src/test/hive/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/src/test/hive/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 b/src/test/hive/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 new file mode 100644 index 0000000000000..04ec33c11c3b4 --- /dev/null +++ b/src/test/hive/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 @@ -0,0 +1 @@ +1.5574077246549023 -0.29100619138474915 -1.5574077246549023 \ No newline at end of file diff --git a/src/test/hive/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad b/src/test/hive/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad new file mode 100644 index 0000000000000..89617d8af0085 --- /dev/null +++ b/src/test/hive/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad @@ -0,0 +1,10 @@ +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/src/test/hive/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 b/src/test/hive/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 new file mode 100644 index 0000000000000..6d967660b3d41 --- /dev/null +++ b/src/test/hive/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d b/src/test/hive/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d new file mode 100644 index 0000000000000..1b13d42908539 --- /dev/null +++ b/src/test/hive/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d @@ -0,0 +1 @@ +There is no documentation for function 'tinyint' \ No newline at end of file diff --git a/src/test/hive/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 b/src/test/hive/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 new file mode 100644 index 0000000000000..1b13d42908539 --- /dev/null +++ b/src/test/hive/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 @@ -0,0 +1 @@ +There is no documentation for function 'tinyint' \ No newline at end of file diff --git a/src/test/hive/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 b/src/test/hive/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 b/src/test/hive/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 b/src/test/hive/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c b/src/test/hive/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d b/src/test/hive/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad b/src/test/hive/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f b/src/test/hive/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a b/src/test/hive/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/src/test/hive/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a @@ -0,0 +1 @@ +true diff --git a/src/test/hive/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 b/src/test/hive/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 b/src/test/hive/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc b/src/test/hive/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc new file mode 100644 index 0000000000000..6cc8a61f8f6ad --- /dev/null +++ b/src/test/hive/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc @@ -0,0 +1 @@ +-18 diff --git a/src/test/hive/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b b/src/test/hive/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b new file mode 100644 index 0000000000000..c75acbe2ff105 --- /dev/null +++ b/src/test/hive/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b @@ -0,0 +1 @@ +127 diff --git a/src/test/hive/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 b/src/test/hive/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/src/test/hive/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 @@ -0,0 +1 @@ +-1 diff --git a/src/test/hive/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 b/src/test/hive/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/src/test/hive/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 @@ -0,0 +1 @@ +-3 diff --git a/src/test/hive/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 b/src/test/hive/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/src/test/hive/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 @@ -0,0 +1 @@ +-3 diff --git a/src/test/hive/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 b/src/test/hive/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/src/test/hive/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 @@ -0,0 +1 @@ +-3 diff --git a/src/test/hive/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 b/src/test/hive/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 new file mode 100644 index 0000000000000..91ff6743335bd --- /dev/null +++ b/src/test/hive/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 @@ -0,0 +1 @@ +-38 diff --git a/src/test/hive/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 b/src/test/hive/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 new file mode 100644 index 0000000000000..54a99d2a94ef4 --- /dev/null +++ b/src/test/hive/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 @@ -0,0 +1 @@ +to_date(expr) - Extracts the date part of the date or datetime expression expr \ No newline at end of file diff --git a/src/test/hive/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e b/src/test/hive/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e new file mode 100644 index 0000000000000..81ddc59fcb531 --- /dev/null +++ b/src/test/hive/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 b/src/test/hive/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a b/src/test/hive/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/src/test/hive/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a @@ -0,0 +1 @@ +1.0 diff --git a/src/test/hive/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe b/src/test/hive/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe new file mode 100644 index 0000000000000..319fde05380bc --- /dev/null +++ b/src/test/hive/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe @@ -0,0 +1 @@ +-7.0 diff --git a/src/test/hive/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f b/src/test/hive/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f new file mode 100644 index 0000000000000..8c1c4fe62b6c2 --- /dev/null +++ b/src/test/hive/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f @@ -0,0 +1 @@ +-18.0 diff --git a/src/test/hive/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 b/src/test/hive/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 new file mode 100644 index 0000000000000..1b650de78904f --- /dev/null +++ b/src/test/hive/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 @@ -0,0 +1 @@ +-129.0 diff --git a/src/test/hive/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 b/src/test/hive/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 new file mode 100644 index 0000000000000..3a3bd0df03b5b --- /dev/null +++ b/src/test/hive/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 @@ -0,0 +1 @@ +-1025.0 diff --git a/src/test/hive/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 b/src/test/hive/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 new file mode 100644 index 0000000000000..38f7ad5afa0ab --- /dev/null +++ b/src/test/hive/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 @@ -0,0 +1 @@ +-3.140000104904175 diff --git a/src/test/hive/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 b/src/test/hive/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/src/test/hive/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 @@ -0,0 +1 @@ +-3.14 diff --git a/src/test/hive/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 b/src/test/hive/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 new file mode 100644 index 0000000000000..f45d1f04dc920 --- /dev/null +++ b/src/test/hive/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 @@ -0,0 +1 @@ +-38.14 diff --git a/src/test/hive/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 b/src/test/hive/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c b/src/test/hive/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/src/test/hive/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c @@ -0,0 +1 @@ +1.0 diff --git a/src/test/hive/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 b/src/test/hive/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 new file mode 100644 index 0000000000000..319fde05380bc --- /dev/null +++ b/src/test/hive/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 @@ -0,0 +1 @@ +-7.0 diff --git a/src/test/hive/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce b/src/test/hive/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce new file mode 100644 index 0000000000000..8c1c4fe62b6c2 --- /dev/null +++ b/src/test/hive/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce @@ -0,0 +1 @@ +-18.0 diff --git a/src/test/hive/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 b/src/test/hive/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 new file mode 100644 index 0000000000000..1b650de78904f --- /dev/null +++ b/src/test/hive/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 @@ -0,0 +1 @@ +-129.0 diff --git a/src/test/hive/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 b/src/test/hive/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 new file mode 100644 index 0000000000000..3a3bd0df03b5b --- /dev/null +++ b/src/test/hive/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 @@ -0,0 +1 @@ +-1025.0 diff --git a/src/test/hive/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 b/src/test/hive/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/src/test/hive/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 @@ -0,0 +1 @@ +-3.14 diff --git a/src/test/hive/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 b/src/test/hive/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/src/test/hive/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 @@ -0,0 +1 @@ +-3.14 diff --git a/src/test/hive/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c b/src/test/hive/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c new file mode 100644 index 0000000000000..f45d1f04dc920 --- /dev/null +++ b/src/test/hive/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c @@ -0,0 +1 @@ +-38.14 diff --git a/src/test/hive/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a b/src/test/hive/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 b/src/test/hive/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 b/src/test/hive/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 new file mode 100644 index 0000000000000..17bdab103828b --- /dev/null +++ b/src/test/hive/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 @@ -0,0 +1 @@ +-7 diff --git a/src/test/hive/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 b/src/test/hive/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 new file mode 100644 index 0000000000000..6cc8a61f8f6ad --- /dev/null +++ b/src/test/hive/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 @@ -0,0 +1 @@ +-18 diff --git a/src/test/hive/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 b/src/test/hive/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 new file mode 100644 index 0000000000000..9828ff22b667b --- /dev/null +++ b/src/test/hive/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 @@ -0,0 +1 @@ +-129 diff --git a/src/test/hive/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d b/src/test/hive/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/src/test/hive/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d @@ -0,0 +1 @@ +-3 diff --git a/src/test/hive/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 b/src/test/hive/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/src/test/hive/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 @@ -0,0 +1 @@ +-3 diff --git a/src/test/hive/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e b/src/test/hive/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/src/test/hive/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e @@ -0,0 +1 @@ +-3 diff --git a/src/test/hive/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 b/src/test/hive/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 new file mode 100644 index 0000000000000..91ff6743335bd --- /dev/null +++ b/src/test/hive/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 @@ -0,0 +1 @@ +-38 diff --git a/src/test/hive/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a b/src/test/hive/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b b/src/test/hive/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 b/src/test/hive/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 new file mode 100644 index 0000000000000..6cc8a61f8f6ad --- /dev/null +++ b/src/test/hive/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 @@ -0,0 +1 @@ +-18 diff --git a/src/test/hive/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce b/src/test/hive/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce new file mode 100644 index 0000000000000..9828ff22b667b --- /dev/null +++ b/src/test/hive/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce @@ -0,0 +1 @@ +-129 diff --git a/src/test/hive/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 b/src/test/hive/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 new file mode 100644 index 0000000000000..450a6125550e5 --- /dev/null +++ b/src/test/hive/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 @@ -0,0 +1 @@ +-1025 diff --git a/src/test/hive/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 b/src/test/hive/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/src/test/hive/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 @@ -0,0 +1 @@ +-3 diff --git a/src/test/hive/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 b/src/test/hive/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/src/test/hive/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 @@ -0,0 +1 @@ +-3 diff --git a/src/test/hive/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d b/src/test/hive/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/src/test/hive/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d @@ -0,0 +1 @@ +-3 diff --git a/src/test/hive/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 b/src/test/hive/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 new file mode 100644 index 0000000000000..91ff6743335bd --- /dev/null +++ b/src/test/hive/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 @@ -0,0 +1 @@ +-38 diff --git a/src/test/hive/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc b/src/test/hive/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/src/test/hive/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc @@ -0,0 +1 @@ +NULL diff --git a/src/test/hive/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 b/src/test/hive/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 new file mode 100644 index 0000000000000..ef2f5130b8575 --- /dev/null +++ b/src/test/hive/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 @@ -0,0 +1 @@ +TRUE diff --git a/src/test/hive/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 b/src/test/hive/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/src/test/hive/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 @@ -0,0 +1 @@ +1 diff --git a/src/test/hive/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 b/src/test/hive/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 new file mode 100644 index 0000000000000..6cc8a61f8f6ad --- /dev/null +++ b/src/test/hive/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 @@ -0,0 +1 @@ +-18 diff --git a/src/test/hive/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 b/src/test/hive/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 new file mode 100644 index 0000000000000..9828ff22b667b --- /dev/null +++ b/src/test/hive/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 @@ -0,0 +1 @@ +-129 diff --git a/src/test/hive/golden/udf_to_string-5-1040b37847d20ef29d545934316303 b/src/test/hive/golden/udf_to_string-5-1040b37847d20ef29d545934316303 new file mode 100644 index 0000000000000..450a6125550e5 --- /dev/null +++ b/src/test/hive/golden/udf_to_string-5-1040b37847d20ef29d545934316303 @@ -0,0 +1 @@ +-1025 diff --git a/src/test/hive/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 b/src/test/hive/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/src/test/hive/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 @@ -0,0 +1 @@ +-3.14 diff --git a/src/test/hive/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 b/src/test/hive/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/src/test/hive/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 @@ -0,0 +1 @@ +-3.14 diff --git a/src/test/hive/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea b/src/test/hive/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/src/test/hive/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea @@ -0,0 +1 @@ +-3.14 diff --git a/src/test/hive/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b b/src/test/hive/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b new file mode 100644 index 0000000000000..bc56c4d89448a --- /dev/null +++ b/src/test/hive/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b @@ -0,0 +1 @@ +Foo diff --git a/src/test/hive/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 b/src/test/hive/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 new file mode 100644 index 0000000000000..4255dc76f501e --- /dev/null +++ b/src/test/hive/golden/udf_translate-0-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 b/src/test/hive/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 new file mode 100644 index 0000000000000..e7beead53b399 --- /dev/null +++ b/src/test/hive/golden/udf_translate-1-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' \ No newline at end of file diff --git a/src/test/hive/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 b/src/test/hive/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 new file mode 100644 index 0000000000000..7d44692e4f7c0 --- /dev/null +++ b/src/test/hive/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 @@ -0,0 +1 @@ +12cd 12cd \ No newline at end of file diff --git a/src/test/hive/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b b/src/test/hive/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b new file mode 100644 index 0000000000000..90e0d872f8bfc --- /dev/null +++ b/src/test/hive/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b @@ -0,0 +1 @@ +123d \ No newline at end of file diff --git a/src/test/hive/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 b/src/test/hive/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 new file mode 100644 index 0000000000000..0770d02cb2303 --- /dev/null +++ b/src/test/hive/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 @@ -0,0 +1 @@ +Ãbcd \ No newline at end of file diff --git a/src/test/hive/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 b/src/test/hive/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a b/src/test/hive/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba b/src/test/hive/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 b/src/test/hive/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b b/src/test/hive/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b new file mode 100644 index 0000000000000..0a54ee40ecb13 --- /dev/null +++ b/src/test/hive/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b @@ -0,0 +1 @@ +12cd 12d \ No newline at end of file diff --git a/src/test/hive/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 b/src/test/hive/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 new file mode 100644 index 0000000000000..0a54ee40ecb13 --- /dev/null +++ b/src/test/hive/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 @@ -0,0 +1 @@ +12cd 12d \ No newline at end of file diff --git a/src/test/hive/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 b/src/test/hive/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 new file mode 100644 index 0000000000000..50c123df9d1d3 --- /dev/null +++ b/src/test/hive/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 @@ -0,0 +1 @@ +1bc \ No newline at end of file diff --git a/src/test/hive/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 b/src/test/hive/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 new file mode 100644 index 0000000000000..b98d7e1c34024 --- /dev/null +++ b/src/test/hive/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 @@ -0,0 +1 @@ +NULL NULL NULL NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f b/src/test/hive/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f new file mode 100644 index 0000000000000..f0620a190ae4e --- /dev/null +++ b/src/test/hive/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f @@ -0,0 +1 @@ +trim(str) - Removes the leading and trailing space characters from str \ No newline at end of file diff --git a/src/test/hive/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 b/src/test/hive/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 new file mode 100644 index 0000000000000..49a13dd64b286 --- /dev/null +++ b/src/test/hive/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 @@ -0,0 +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 diff --git a/src/test/hive/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d b/src/test/hive/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d new file mode 100644 index 0000000000000..057c231dccf0e --- /dev/null +++ b/src/test/hive/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d @@ -0,0 +1 @@ +ucase(str) - Returns str with all characters changed to uppercase \ No newline at end of file diff --git a/src/test/hive/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f b/src/test/hive/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f new file mode 100644 index 0000000000000..128d1e21fae83 --- /dev/null +++ b/src/test/hive/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 b/src/test/hive/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 new file mode 100644 index 0000000000000..44b2a42cc26c5 --- /dev/null +++ b/src/test/hive/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 @@ -0,0 +1 @@ +unhex(str) - Converts hexadecimal argument to binary diff --git a/src/test/hive/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 b/src/test/hive/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 new file mode 100644 index 0000000000000..97af3b812a429 --- /dev/null +++ b/src/test/hive/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 @@ -0,0 +1,14 @@ +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/src/test/hive/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 b/src/test/hive/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 new file mode 100644 index 0000000000000..b4a6f2b692227 --- /dev/null +++ b/src/test/hive/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 @@ -0,0 +1 @@ +MySQL 1267 a -4 diff --git a/src/test/hive/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 b/src/test/hive/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 new file mode 100644 index 0000000000000..3a67adaf0a9a8 --- /dev/null +++ b/src/test/hive/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 @@ -0,0 +1 @@ +NULL NULL NULL diff --git a/src/test/hive/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c b/src/test/hive/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c new file mode 100644 index 0000000000000..3f3bb2b7ce02e --- /dev/null +++ b/src/test/hive/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c @@ -0,0 +1 @@ +create_union(tag, obj1, obj2, obj3, ...) - Creates a union with the object for given tag \ No newline at end of file diff --git a/src/test/hive/golden/udf_union-1-f6f01250718087029165e23badc02cd6 b/src/test/hive/golden/udf_union-1-f6f01250718087029165e23badc02cd6 new file mode 100644 index 0000000000000..748ed97055401 --- /dev/null +++ b/src/test/hive/golden/udf_union-1-f6f01250718087029165e23badc02cd6 @@ -0,0 +1,4 @@ +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/src/test/hive/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 b/src/test/hive/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_union-3-705d165fec6761744dd19b142c566d61 b/src/test/hive/golden/udf_union-3-705d165fec6761744dd19b142c566d61 new file mode 100644 index 0000000000000..4b9ae5402aa53 --- /dev/null +++ b/src/test/hive/golden/udf_union-3-705d165fec6761744dd19b142c566d61 @@ -0,0 +1,2 @@ +{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/src/test/hive/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead b/src/test/hive/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead new file mode 100644 index 0000000000000..2e239cca5b860 --- /dev/null +++ b/src/test/hive/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead @@ -0,0 +1 @@ +upper(str) - Returns str with all characters changed to uppercase \ No newline at end of file diff --git a/src/test/hive/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d b/src/test/hive/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d new file mode 100644 index 0000000000000..673307e284b2e --- /dev/null +++ b/src/test/hive/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de b/src/test/hive/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de new file mode 100644 index 0000000000000..9dbf241e28107 --- /dev/null +++ b/src/test/hive/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de @@ -0,0 +1 @@ +var_pop(x) - Returns the variance of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 b/src/test/hive/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 new file mode 100644 index 0000000000000..234a65bd8255d --- /dev/null +++ b/src/test/hive/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 @@ -0,0 +1,2 @@ +var_pop(x) - Returns the variance of a set of numbers +Synonyms: variance \ No newline at end of file diff --git a/src/test/hive/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 b/src/test/hive/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 new file mode 100644 index 0000000000000..673deeb031b44 --- /dev/null +++ b/src/test/hive/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 @@ -0,0 +1 @@ +var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b b/src/test/hive/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b new file mode 100644 index 0000000000000..673deeb031b44 --- /dev/null +++ b/src/test/hive/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b @@ -0,0 +1 @@ +var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 b/src/test/hive/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 new file mode 100644 index 0000000000000..673deeb031b44 --- /dev/null +++ b/src/test/hive/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 @@ -0,0 +1 @@ +var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b b/src/test/hive/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b new file mode 100644 index 0000000000000..673deeb031b44 --- /dev/null +++ b/src/test/hive/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b @@ -0,0 +1 @@ +var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 b/src/test/hive/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 new file mode 100644 index 0000000000000..f58ae0769d5a9 --- /dev/null +++ b/src/test/hive/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 @@ -0,0 +1 @@ +variance(x) - Returns the variance of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a b/src/test/hive/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a new file mode 100644 index 0000000000000..e17528e607aba --- /dev/null +++ b/src/test/hive/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a @@ -0,0 +1,2 @@ +variance(x) - Returns the variance of a set of numbers +Synonyms: var_pop \ No newline at end of file diff --git a/src/test/hive/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de b/src/test/hive/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de new file mode 100644 index 0000000000000..9dbf241e28107 --- /dev/null +++ b/src/test/hive/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de @@ -0,0 +1 @@ +var_pop(x) - Returns the variance of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_variance-3-fd25e5226312bf54d918858511814766 b/src/test/hive/golden/udf_variance-3-fd25e5226312bf54d918858511814766 new file mode 100644 index 0000000000000..234a65bd8255d --- /dev/null +++ b/src/test/hive/golden/udf_variance-3-fd25e5226312bf54d918858511814766 @@ -0,0 +1,2 @@ +var_pop(x) - Returns the variance of a set of numbers +Synonyms: variance \ No newline at end of file diff --git a/src/test/hive/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 b/src/test/hive/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 new file mode 100644 index 0000000000000..f58ae0769d5a9 --- /dev/null +++ b/src/test/hive/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 @@ -0,0 +1 @@ +variance(x) - Returns the variance of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a b/src/test/hive/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a new file mode 100644 index 0000000000000..e17528e607aba --- /dev/null +++ b/src/test/hive/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a @@ -0,0 +1,2 @@ +variance(x) - Returns the variance of a set of numbers +Synonyms: var_pop \ No newline at end of file diff --git a/src/test/hive/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de b/src/test/hive/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de new file mode 100644 index 0000000000000..9dbf241e28107 --- /dev/null +++ b/src/test/hive/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de @@ -0,0 +1 @@ +var_pop(x) - Returns the variance of a set of numbers \ No newline at end of file diff --git a/src/test/hive/golden/udf_variance-7-fd25e5226312bf54d918858511814766 b/src/test/hive/golden/udf_variance-7-fd25e5226312bf54d918858511814766 new file mode 100644 index 0000000000000..234a65bd8255d --- /dev/null +++ b/src/test/hive/golden/udf_variance-7-fd25e5226312bf54d918858511814766 @@ -0,0 +1,2 @@ +var_pop(x) - Returns the variance of a set of numbers +Synonyms: variance \ No newline at end of file diff --git a/src/test/hive/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 b/src/test/hive/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 new file mode 100644 index 0000000000000..29e3b370b03b8 --- /dev/null +++ b/src/test/hive/golden/udf_weekofyear-0-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. \ No newline at end of file diff --git a/src/test/hive/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd b/src/test/hive/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd new file mode 100644 index 0000000000000..c7939a11937c5 --- /dev/null +++ b/src/test/hive/golden/udf_weekofyear-1-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 \ No newline at end of file diff --git a/src/test/hive/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 b/src/test/hive/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 new file mode 100644 index 0000000000000..20d1dc50ce1f7 --- /dev/null +++ b/src/test/hive/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 @@ -0,0 +1 @@ +1 1 2 1 52 8 52 1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 b/src/test/hive/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 new file mode 100644 index 0000000000000..2e36162dbaa33 --- /dev/null +++ b/src/test/hive/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 @@ -0,0 +1 @@ +There is no documentation for function 'when' \ No newline at end of file diff --git a/src/test/hive/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 b/src/test/hive/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 new file mode 100644 index 0000000000000..2e36162dbaa33 --- /dev/null +++ b/src/test/hive/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 @@ -0,0 +1 @@ +There is no documentation for function 'when' \ No newline at end of file diff --git a/src/test/hive/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 b/src/test/hive/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f b/src/test/hive/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f new file mode 100644 index 0000000000000..6929fdca28eb8 --- /dev/null +++ b/src/test/hive/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f @@ -0,0 +1 @@ +2 9 14 NULL 24 NULL \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 b/src/test/hive/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 new file mode 100644 index 0000000000000..18d9720653a5b --- /dev/null +++ b/src/test/hive/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 @@ -0,0 +1 @@ +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/src/test/hive/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 b/src/test/hive/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 new file mode 100644 index 0000000000000..3b092dafd32c2 --- /dev/null +++ b/src/test/hive/golden/udf_xpath-1-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('b1b2b3c1c2', 'a/text()') FROM src LIMIT 1 + [] + > SELECT xpath('b1b2b3c1c2', 'a/b/text()') FROM src LIMIT 1 + ["b1","b2","b3"] + > SELECT xpath('b1b2b3c1c2', 'a/c/text()') FROM src LIMIT 1 + ["c1","c2"] \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d b/src/test/hive/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d new file mode 100644 index 0000000000000..0637a088a01e8 --- /dev/null +++ b/src/test/hive/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d @@ -0,0 +1 @@ +[] \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea b/src/test/hive/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea new file mode 100644 index 0000000000000..8e8aff2387621 --- /dev/null +++ b/src/test/hive/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea @@ -0,0 +1 @@ +["b1","b2","b3","c1","c2"] \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a b/src/test/hive/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a new file mode 100644 index 0000000000000..e518b1f37e124 --- /dev/null +++ b/src/test/hive/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a @@ -0,0 +1 @@ +["b1","b2","b3"] \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 b/src/test/hive/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 new file mode 100644 index 0000000000000..45615b772ba35 --- /dev/null +++ b/src/test/hive/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 @@ -0,0 +1 @@ +["c1","c2"] \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f b/src/test/hive/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f new file mode 100644 index 0000000000000..327bf3a1e76cc --- /dev/null +++ b/src/test/hive/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f @@ -0,0 +1 @@ +["b1","c1"] \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 b/src/test/hive/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 new file mode 100644 index 0000000000000..134e9e99b7423 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 @@ -0,0 +1 @@ +xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 b/src/test/hive/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 new file mode 100644 index 0000000000000..624889cc0ac67 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 @@ -0,0 +1,6 @@ +xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression +Example: + > SELECT xpath_boolean('1','a/b') FROM src LIMIT 1; + true + > SELECT xpath_boolean('1','a/b = 2') FROM src LIMIT 1; + false \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b b/src/test/hive/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/src/test/hive/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 b/src/test/hive/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 new file mode 100644 index 0000000000000..02e4a84d62c4b --- /dev/null +++ b/src/test/hive/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 @@ -0,0 +1 @@ +false \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 b/src/test/hive/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/src/test/hive/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 b/src/test/hive/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 new file mode 100644 index 0000000000000..02e4a84d62c4b --- /dev/null +++ b/src/test/hive/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 @@ -0,0 +1 @@ +false \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f b/src/test/hive/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f new file mode 100644 index 0000000000000..02e4a84d62c4b --- /dev/null +++ b/src/test/hive/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f @@ -0,0 +1 @@ +false \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 b/src/test/hive/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/src/test/hive/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 b/src/test/hive/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 new file mode 100644 index 0000000000000..b56f4d00a8a74 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 @@ -0,0 +1 @@ +xpath_number(xml, xpath) - Returns a double value that matches the xpath expression \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 b/src/test/hive/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 new file mode 100644 index 0000000000000..b6b6227174647 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-1-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('12','sum(a/b)') FROM src LIMIT 1; + 3.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 b/src/test/hive/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 new file mode 100644 index 0000000000000..7104585f9a439 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 @@ -0,0 +1 @@ +7.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 b/src/test/hive/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 new file mode 100644 index 0000000000000..6e6366051638f --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 @@ -0,0 +1 @@ +5.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 b/src/test/hive/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 new file mode 100644 index 0000000000000..5b6de7db2f9f3 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 @@ -0,0 +1 @@ +xpath_double(xml, xpath) - Returns a double value that matches the xpath expression \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 b/src/test/hive/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 new file mode 100644 index 0000000000000..b8accbcae1e17 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-3-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('12','sum(a/b)') FROM src LIMIT 1; + 3.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c b/src/test/hive/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 b/src/test/hive/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 b/src/test/hive/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 new file mode 100644 index 0000000000000..0a8ac8c629540 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 @@ -0,0 +1 @@ +8.0E19 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c b/src/test/hive/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d b/src/test/hive/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d new file mode 100644 index 0000000000000..9f8e9b69a33f4 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d @@ -0,0 +1 @@ +1.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e b/src/test/hive/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e new file mode 100644 index 0000000000000..3d3be3c32ed85 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e @@ -0,0 +1 @@ +15.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 b/src/test/hive/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 new file mode 100644 index 0000000000000..ea4df0ae58e12 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 @@ -0,0 +1 @@ +xpath_float(xml, xpath) - Returns a float value that matches the xpath expression \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a b/src/test/hive/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a new file mode 100644 index 0000000000000..6bc4d4a46de72 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a @@ -0,0 +1,4 @@ +xpath_float(xml, xpath) - Returns a float value that matches the xpath expression +Example: + > SELECT xpath_float('12','sum(a/b)') FROM src LIMIT 1; + 3.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 b/src/test/hive/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 b/src/test/hive/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 b/src/test/hive/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 new file mode 100644 index 0000000000000..0a8ac8c629540 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 @@ -0,0 +1 @@ +8.0E19 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f b/src/test/hive/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/src/test/hive/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf b/src/test/hive/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf new file mode 100644 index 0000000000000..9f8e9b69a33f4 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf @@ -0,0 +1 @@ +1.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa b/src/test/hive/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa new file mode 100644 index 0000000000000..3d3be3c32ed85 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa @@ -0,0 +1 @@ +15.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 b/src/test/hive/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 new file mode 100644 index 0000000000000..7104585f9a439 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 @@ -0,0 +1 @@ +7.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 b/src/test/hive/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 new file mode 100644 index 0000000000000..6e6366051638f --- /dev/null +++ b/src/test/hive/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 @@ -0,0 +1 @@ +5.0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 b/src/test/hive/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 new file mode 100644 index 0000000000000..67ab19492e405 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 @@ -0,0 +1 @@ +xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 b/src/test/hive/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 new file mode 100644 index 0000000000000..33349b0b22ad9 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 @@ -0,0 +1,4 @@ +xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression +Example: + > SELECT xpath_int('12','sum(a/b)') FROM src LIMIT 1; + 3 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba b/src/test/hive/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba b/src/test/hive/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 b/src/test/hive/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 new file mode 100644 index 0000000000000..55a89f5564d21 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 @@ -0,0 +1 @@ +2147483647 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 b/src/test/hive/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a b/src/test/hive/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 b/src/test/hive/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 b/src/test/hive/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 new file mode 100644 index 0000000000000..c7930257dfef5 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 @@ -0,0 +1 @@ +7 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 b/src/test/hive/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/src/test/hive/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b b/src/test/hive/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b new file mode 100644 index 0000000000000..748fd827e8545 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b @@ -0,0 +1 @@ +xpath_long(xml, xpath) - Returns a long value that matches the xpath expression \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 b/src/test/hive/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 new file mode 100644 index 0000000000000..65241b6ce9b64 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 @@ -0,0 +1,4 @@ +xpath_long(xml, xpath) - Returns a long value that matches the xpath expression +Example: + > SELECT xpath_long('12','sum(a/b)') FROM src LIMIT 1; + 3 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b b/src/test/hive/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d b/src/test/hive/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 b/src/test/hive/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 new file mode 100644 index 0000000000000..996d127e59365 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 @@ -0,0 +1 @@ +9223372036854775807 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d b/src/test/hive/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 b/src/test/hive/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 b/src/test/hive/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 b/src/test/hive/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 new file mode 100644 index 0000000000000..c7930257dfef5 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 @@ -0,0 +1 @@ +7 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 b/src/test/hive/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/src/test/hive/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 b/src/test/hive/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 new file mode 100644 index 0000000000000..22ccb9ca9defa --- /dev/null +++ b/src/test/hive/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 @@ -0,0 +1 @@ +xpath_short(xml, xpath) - Returns a short value that matches the xpath expression \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 b/src/test/hive/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 new file mode 100644 index 0000000000000..931ce2abb3d4d --- /dev/null +++ b/src/test/hive/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 @@ -0,0 +1,4 @@ +xpath_short(xml, xpath) - Returns a short value that matches the xpath expression +Example: + > SELECT xpath_short('12','sum(a/b)') FROM src LIMIT 1; + 3 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 b/src/test/hive/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 b/src/test/hive/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c b/src/test/hive/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c new file mode 100644 index 0000000000000..d7d17fcbef95c --- /dev/null +++ b/src/test/hive/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c @@ -0,0 +1 @@ +-1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e b/src/test/hive/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 b/src/test/hive/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d b/src/test/hive/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 b/src/test/hive/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 new file mode 100644 index 0000000000000..c7930257dfef5 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 @@ -0,0 +1 @@ +7 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e b/src/test/hive/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/src/test/hive/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/src/test/hive/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 b/src/test/hive/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 new file mode 100644 index 0000000000000..5b807fcf43195 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 @@ -0,0 +1 @@ +xpath_string(xml, xpath) - Returns the text contents of the first xml node that matches the xpath expression diff --git a/src/test/hive/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 b/src/test/hive/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 new file mode 100644 index 0000000000000..8f3ab457faf5c --- /dev/null +++ b/src/test/hive/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 @@ -0,0 +1,10 @@ +xpath_string(xml, xpath) - Returns the text contents of the first xml node that matches the xpath expression +Example: + > SELECT xpath_string('bcc','a/c') FROM src LIMIT 1; + 'cc' + > SELECT xpath_string('b1b2','a/b') FROM src LIMIT 1; + 'b1' + > SELECT xpath_string('b1b2','a/b[2]') FROM src LIMIT 1; + 'b2' + > SELECT xpath_string('b1b2','a') FROM src LIMIT 1; + 'b1b2' diff --git a/src/test/hive/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f b/src/test/hive/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f new file mode 100644 index 0000000000000..2f10d1e91ed93 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f @@ -0,0 +1 @@ +bbcc diff --git a/src/test/hive/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a b/src/test/hive/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a new file mode 100644 index 0000000000000..e0b3f1b09bd18 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a @@ -0,0 +1 @@ +bb diff --git a/src/test/hive/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a b/src/test/hive/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a new file mode 100644 index 0000000000000..46c1d6125b7b4 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a @@ -0,0 +1 @@ +cc diff --git a/src/test/hive/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 b/src/test/hive/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 new file mode 100644 index 0000000000000..8b137891791fe --- /dev/null +++ b/src/test/hive/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 @@ -0,0 +1 @@ + diff --git a/src/test/hive/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 b/src/test/hive/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 new file mode 100644 index 0000000000000..c9c6af7f78bc4 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 @@ -0,0 +1 @@ +b1 diff --git a/src/test/hive/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f b/src/test/hive/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f new file mode 100644 index 0000000000000..c9c6af7f78bc4 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f @@ -0,0 +1 @@ +b1 diff --git a/src/test/hive/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc b/src/test/hive/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc new file mode 100644 index 0000000000000..e6bfff5c1d0f0 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc @@ -0,0 +1 @@ +b2 diff --git a/src/test/hive/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 b/src/test/hive/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 new file mode 100644 index 0000000000000..e6bfff5c1d0f0 --- /dev/null +++ b/src/test/hive/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 @@ -0,0 +1 @@ +b2 diff --git a/src/test/hive/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a b/src/test/hive/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a new file mode 100644 index 0000000000000..599bf1880a83a --- /dev/null +++ b/src/test/hive/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a @@ -0,0 +1 @@ +stack(n, cols...) - turns k columns into n rows of size k/n each diff --git a/src/test/hive/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 b/src/test/hive/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 b/src/test/hive/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 b/src/test/hive/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 b/src/test/hive/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 new file mode 100644 index 0000000000000..e82f93fe65c5e --- /dev/null +++ b/src/test/hive/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 @@ -0,0 +1,27 @@ +# col_name data_type comment + +a string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Fri Jan 03 18:39:52 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1388803192 + +# 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: + field.delim \u0001 + serialization.format \u0001 \ No newline at end of file diff --git a/src/test/hive/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 b/src/test/hive/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e b/src/test/hive/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 b/src/test/hive/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 new file mode 100644 index 0000000000000..e82f93fe65c5e --- /dev/null +++ b/src/test/hive/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 @@ -0,0 +1,27 @@ +# col_name data_type comment + +a string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Fri Jan 03 18:39:52 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1388803192 + +# 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: + field.delim \u0001 + serialization.format \u0001 \ No newline at end of file diff --git a/src/test/hive/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 b/src/test/hive/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 b/src/test/hive/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 b/src/test/hive/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 new file mode 100644 index 0000000000000..2be72c6e4e8c7 --- /dev/null +++ b/src/test/hive/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 @@ -0,0 +1,27 @@ +# col_name data_type comment + +a string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Fri Jan 03 18:39:52 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1388803192 + +# 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: + field.delim | + serialization.format | \ No newline at end of file diff --git a/src/test/hive/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 b/src/test/hive/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af b/src/test/hive/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union10-2-60ee9eae687170317ff91dafe6b799bf b/src/test/hive/golden/union10-2-60ee9eae687170317ff91dafe6b799bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 b/src/test/hive/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 b/src/test/hive/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 new file mode 100644 index 0000000000000..1d5891034ddec --- /dev/null +++ b/src/test/hive/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 @@ -0,0 +1,3 @@ +tst1 500 +tst2 500 +tst3 500 \ No newline at end of file diff --git a/src/test/hive/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 b/src/test/hive/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union11-2-cd756f39e22e121bdbd51400662aa47f b/src/test/hive/golden/union11-2-cd756f39e22e121bdbd51400662aa47f new file mode 100644 index 0000000000000..1ec0096b865a7 --- /dev/null +++ b/src/test/hive/golden/union11-2-cd756f39e22e121bdbd51400662aa47f @@ -0,0 +1,3 @@ +tst1 1 +tst2 1 +tst3 1 \ No newline at end of file diff --git a/src/test/hive/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 b/src/test/hive/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 b/src/test/hive/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union13-0-6970d1e2405f3769a28a8804887ac657 b/src/test/hive/golden/union13-0-6970d1e2405f3769a28a8804887ac657 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 b/src/test/hive/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 new file mode 100644 index 0000000000000..2209b11d74282 --- /dev/null +++ b/src/test/hive/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 @@ -0,0 +1,1000 @@ +238 val_238 +238 val_238 +86 val_86 +86 val_86 +311 val_311 +311 val_311 +27 val_27 +27 val_27 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +484 val_484 +484 val_484 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +150 val_150 +150 val_150 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +66 val_66 +66 val_66 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +374 val_374 +374 val_374 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +145 val_145 +145 val_145 +495 val_495 +495 val_495 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +82 val_82 +82 val_82 +403 val_403 +403 val_403 +166 val_166 +166 val_166 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +252 val_252 +252 val_252 +292 val_292 +292 val_292 +219 val_219 +219 val_219 +287 val_287 +287 val_287 +153 val_153 +153 val_153 +193 val_193 +193 val_193 +338 val_338 +338 val_338 +446 val_446 +446 val_446 +459 val_459 +459 val_459 +394 val_394 +394 val_394 +237 val_237 +237 val_237 +482 val_482 +482 val_482 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +494 val_494 +494 val_494 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +247 val_247 +247 val_247 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +162 val_162 +162 val_162 +377 val_377 +377 val_377 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +365 val_365 +365 val_365 +266 val_266 +266 val_266 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +475 val_475 +475 val_475 +17 val_17 +17 val_17 +113 val_113 +113 val_113 +155 val_155 +155 val_155 +203 val_203 +203 val_203 +339 val_339 +339 val_339 +0 val_0 +0 val_0 +455 val_455 +455 val_455 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +57 val_57 +57 val_57 +302 val_302 +302 val_302 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +345 val_345 +345 val_345 +129 val_129 +129 val_129 +170 val_170 +170 val_170 +20 val_20 +20 val_20 +489 val_489 +489 val_489 +157 val_157 +157 val_157 +378 val_378 +378 val_378 +221 val_221 +221 val_221 +92 val_92 +92 val_92 +111 val_111 +111 val_111 +47 val_47 +47 val_47 +72 val_72 +72 val_72 +4 val_4 +4 val_4 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +427 val_427 +427 val_427 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +356 val_356 +356 val_356 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +386 val_386 +386 val_386 +437 val_437 +437 val_437 +469 val_469 +469 val_469 +192 val_192 +192 val_192 +286 val_286 +286 val_286 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +54 val_54 +54 val_54 +459 val_459 +459 val_459 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +289 val_289 +289 val_289 +221 val_221 +221 val_221 +65 val_65 +65 val_65 +318 val_318 +318 val_318 +332 val_332 +332 val_332 +311 val_311 +311 val_311 +275 val_275 +275 val_275 +137 val_137 +137 val_137 +241 val_241 +241 val_241 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +180 val_180 +180 val_180 +284 val_284 +284 val_284 +12 val_12 +12 val_12 +230 val_230 +230 val_230 +181 val_181 +181 val_181 +67 val_67 +67 val_67 +260 val_260 +260 val_260 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +373 val_373 +373 val_373 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +58 val_58 +58 val_58 +8 val_8 +8 val_8 +411 val_411 +411 val_411 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +158 val_158 +158 val_158 +119 val_119 +119 val_119 +496 val_496 +496 val_496 +0 val_0 +0 val_0 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +393 val_393 +393 val_393 +454 val_454 +454 val_454 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +418 val_418 +418 val_418 +96 val_96 +96 val_96 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +131 val_131 +131 val_131 +51 val_51 +51 val_51 +404 val_404 +404 val_404 +43 val_43 +43 val_43 +436 val_436 +436 val_436 +156 val_156 +156 val_156 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +308 val_308 +308 val_308 +95 val_95 +95 val_95 +196 val_196 +196 val_196 +288 val_288 +288 val_288 +481 val_481 +481 val_481 +457 val_457 +457 val_457 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +470 val_470 +470 val_470 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +85 val_85 +85 val_85 +77 val_77 +77 val_77 +0 val_0 +0 val_0 +490 val_490 +490 val_490 +87 val_87 +87 val_87 +364 val_364 +364 val_364 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +419 val_419 +419 val_419 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +19 val_19 +19 val_19 +435 val_435 +435 val_435 +10 val_10 +10 val_10 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +306 val_306 +306 val_306 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +389 val_389 +389 val_389 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +392 val_392 +392 val_392 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +452 val_452 +452 val_452 +177 val_177 +177 val_177 +226 val_226 +226 val_226 +5 val_5 +5 val_5 +497 val_497 +497 val_497 +402 val_402 +402 val_402 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +336 val_336 +336 val_336 +95 val_95 +95 val_95 +11 val_11 +11 val_11 +168 val_168 +168 val_168 +34 val_34 +34 val_34 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +143 val_143 +143 val_143 +472 val_472 +472 val_472 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +160 val_160 +160 val_160 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +78 val_78 +78 val_78 +76 val_76 +76 val_76 +41 val_41 +41 val_41 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +449 val_449 +449 val_449 +218 val_218 +218 val_218 +228 val_228 +228 val_228 +138 val_138 +138 val_138 +453 val_453 +453 val_453 +30 val_30 +30 val_30 +209 val_209 +209 val_209 +64 val_64 +64 val_64 +468 val_468 +468 val_468 +76 val_76 +76 val_76 +74 val_74 +74 val_74 +342 val_342 +342 val_342 +69 val_69 +69 val_69 +230 val_230 +230 val_230 +33 val_33 +33 val_33 +368 val_368 +368 val_368 +103 val_103 +103 val_103 +296 val_296 +296 val_296 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +274 val_274 +274 val_274 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +485 val_485 +485 val_485 +116 val_116 +116 val_116 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +263 val_263 +263 val_263 +70 val_70 +70 val_70 +487 val_487 +487 val_487 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +244 val_244 +244 val_244 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +467 val_467 +467 val_467 +432 val_432 +432 val_432 +202 val_202 +202 val_202 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +2 val_2 +2 val_2 +35 val_35 +35 val_35 +283 val_283 +283 val_283 +331 val_331 +331 val_331 +235 val_235 +235 val_235 +80 val_80 +80 val_80 +44 val_44 +44 val_44 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +335 val_335 +335 val_335 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +366 val_366 +366 val_366 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +483 val_483 +483 val_483 +53 val_53 +53 val_53 +105 val_105 +105 val_105 +257 val_257 +257 val_257 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +190 val_190 +190 val_190 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +114 val_114 +114 val_114 +258 val_258 +258 val_258 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +262 val_262 +262 val_262 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +12 val_12 +12 val_12 +396 val_396 +396 val_396 +201 val_201 +201 val_201 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +291 val_291 +291 val_291 +24 val_24 +24 val_24 +351 val_351 +351 val_351 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +163 val_163 +163 val_163 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +491 val_491 +491 val_491 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +360 val_360 +360 val_360 +248 val_248 +248 val_248 +479 val_479 +479 val_479 +305 val_305 +305 val_305 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +444 val_444 +444 val_444 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +443 val_443 +443 val_443 +323 val_323 +323 val_323 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +178 val_178 +178 val_178 +468 val_468 +468 val_468 +310 val_310 +310 val_310 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +493 val_493 +493 val_493 +460 val_460 +460 val_460 +207 val_207 +207 val_207 +249 val_249 +249 val_249 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +83 val_83 +83 val_83 +136 val_136 +136 val_136 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +214 val_214 +214 val_214 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +133 val_133 +133 val_133 +175 val_175 +175 val_175 +189 val_189 +189 val_189 +454 val_454 +454 val_454 +375 val_375 +375 val_375 +401 val_401 +401 val_401 +421 val_421 +421 val_421 +407 val_407 +407 val_407 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +379 val_379 +379 val_379 +18 val_18 +18 val_18 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +9 val_9 +9 val_9 +341 val_341 +341 val_341 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +362 val_362 +362 val_362 +186 val_186 +186 val_186 +285 val_285 +285 val_285 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +183 val_183 +183 val_183 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +469 val_469 +469 val_469 +315 val_315 +315 val_315 +84 val_84 +84 val_84 +28 val_28 +28 val_28 +37 val_37 +37 val_37 +448 val_448 +448 val_448 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +194 val_194 +194 val_194 +414 val_414 +414 val_414 +477 val_477 +477 val_477 +222 val_222 +222 val_222 +126 val_126 +126 val_126 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +400 val_400 +400 val_400 +200 val_200 +200 val_200 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union14-1-11603fb6225423979af6c062cfa9389b b/src/test/hive/golden/union14-1-11603fb6225423979af6c062cfa9389b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 b/src/test/hive/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 new file mode 100644 index 0000000000000..07f67df79dcbd --- /dev/null +++ b/src/test/hive/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 @@ -0,0 +1,17 @@ +NULL 10 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 +66 1 +98 1 +tst1 1 \ No newline at end of file diff --git a/src/test/hive/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 b/src/test/hive/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 b/src/test/hive/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 new file mode 100644 index 0000000000000..b00b55f6fa92d --- /dev/null +++ b/src/test/hive/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 @@ -0,0 +1,17 @@ +NULL 20 +128 2 +146 2 +150 2 +213 2 +224 2 +238 2 +255 2 +273 2 +278 2 +311 2 +369 2 +401 2 +406 2 +66 2 +98 2 +tst1 1 \ No newline at end of file diff --git a/src/test/hive/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 b/src/test/hive/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union16-1-9f76074598f9b55d8afbb5659737a382 b/src/test/hive/golden/union16-1-9f76074598f9b55d8afbb5659737a382 new file mode 100644 index 0000000000000..6f680072350ab --- /dev/null +++ b/src/test/hive/golden/union16-1-9f76074598f9b55d8afbb5659737a382 @@ -0,0 +1 @@ +12500 \ No newline at end of file diff --git a/src/test/hive/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a b/src/test/hive/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union17-1-9491c271b7e2c351ddcf126f73679892 b/src/test/hive/golden/union17-1-9491c271b7e2c351ddcf126f73679892 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union17-2-e6fb104913b9da8193167ee263993dd1 b/src/test/hive/golden/union17-2-e6fb104913b9da8193167ee263993dd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 b/src/test/hive/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/src/test/hive/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..a5a9e42fff209 --- /dev/null +++ b/src/test/hive/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -0,0 +1,310 @@ +0 1 +10 1 +100 1 +103 1 +104 1 +105 1 +11 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +12 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 +15 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 +17 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +18 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +19 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +2 1 +20 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 +24 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +26 1 +260 1 +262 1 +263 1 +265 1 +266 1 +27 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +28 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 +30 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 +33 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +34 1 +341 1 +342 1 +344 1 +345 1 +348 1 +35 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +37 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 +4 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +41 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +42 1 +421 1 +424 1 +427 1 +429 1 +43 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +44 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 +47 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 +5 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 +8 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +9 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +tst1 1 \ No newline at end of file diff --git a/src/test/hive/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 b/src/test/hive/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..be13b26eadc86 --- /dev/null +++ b/src/test/hive/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 @@ -0,0 +1,310 @@ +0 val_0 1 +10 val_10 1 +100 val_100 1 +103 val_103 1 +104 val_104 1 +105 val_105 1 +11 val_11 1 +111 val_111 1 +113 val_113 1 +114 val_114 1 +116 val_116 1 +118 val_118 1 +119 val_119 1 +12 val_12 1 +120 val_120 1 +125 val_125 1 +126 val_126 1 +128 val_128 1 +129 val_129 1 +131 val_131 1 +133 val_133 1 +134 val_134 1 +136 val_136 1 +137 val_137 1 +138 val_138 1 +143 val_143 1 +145 val_145 1 +146 val_146 1 +149 val_149 1 +15 val_15 1 +150 val_150 1 +152 val_152 1 +153 val_153 1 +155 val_155 1 +156 val_156 1 +157 val_157 1 +158 val_158 1 +160 val_160 1 +162 val_162 1 +163 val_163 1 +164 val_164 1 +165 val_165 1 +166 val_166 1 +167 val_167 1 +168 val_168 1 +169 val_169 1 +17 val_17 1 +170 val_170 1 +172 val_172 1 +174 val_174 1 +175 val_175 1 +176 val_176 1 +177 val_177 1 +178 val_178 1 +179 val_179 1 +18 val_18 1 +180 val_180 1 +181 val_181 1 +183 val_183 1 +186 val_186 1 +187 val_187 1 +189 val_189 1 +19 val_19 1 +190 val_190 1 +191 val_191 1 +192 val_192 1 +193 val_193 1 +194 val_194 1 +195 val_195 1 +196 val_196 1 +197 val_197 1 +199 val_199 1 +2 val_2 1 +20 val_20 1 +200 val_200 1 +201 val_201 1 +202 val_202 1 +203 val_203 1 +205 val_205 1 +207 val_207 1 +208 val_208 1 +209 val_209 1 +213 val_213 1 +214 val_214 1 +216 val_216 1 +217 val_217 1 +218 val_218 1 +219 val_219 1 +221 val_221 1 +222 val_222 1 +223 val_223 1 +224 val_224 1 +226 val_226 1 +228 val_228 1 +229 val_229 1 +230 val_230 1 +233 val_233 1 +235 val_235 1 +237 val_237 1 +238 val_238 1 +239 val_239 1 +24 val_24 1 +241 val_241 1 +242 val_242 1 +244 val_244 1 +247 val_247 1 +248 val_248 1 +249 val_249 1 +252 val_252 1 +255 val_255 1 +256 val_256 1 +257 val_257 1 +258 val_258 1 +26 val_26 1 +260 val_260 1 +262 val_262 1 +263 val_263 1 +265 val_265 1 +266 val_266 1 +27 val_27 1 +272 val_272 1 +273 val_273 1 +274 val_274 1 +275 val_275 1 +277 val_277 1 +278 val_278 1 +28 val_28 1 +280 val_280 1 +281 val_281 1 +282 val_282 1 +283 val_283 1 +284 val_284 1 +285 val_285 1 +286 val_286 1 +287 val_287 1 +288 val_288 1 +289 val_289 1 +291 val_291 1 +292 val_292 1 +296 val_296 1 +298 val_298 1 +30 val_30 1 +302 val_302 1 +305 val_305 1 +306 val_306 1 +307 val_307 1 +308 val_308 1 +309 val_309 1 +310 val_310 1 +311 val_311 1 +315 val_315 1 +316 val_316 1 +317 val_317 1 +318 val_318 1 +321 val_321 1 +322 val_322 1 +323 val_323 1 +325 val_325 1 +327 val_327 1 +33 val_33 1 +331 val_331 1 +332 val_332 1 +333 val_333 1 +335 val_335 1 +336 val_336 1 +338 val_338 1 +339 val_339 1 +34 val_34 1 +341 val_341 1 +342 val_342 1 +344 val_344 1 +345 val_345 1 +348 val_348 1 +35 val_35 1 +351 val_351 1 +353 val_353 1 +356 val_356 1 +360 val_360 1 +362 val_362 1 +364 val_364 1 +365 val_365 1 +366 val_366 1 +367 val_367 1 +368 val_368 1 +369 val_369 1 +37 val_37 1 +373 val_373 1 +374 val_374 1 +375 val_375 1 +377 val_377 1 +378 val_378 1 +379 val_379 1 +382 val_382 1 +384 val_384 1 +386 val_386 1 +389 val_389 1 +392 val_392 1 +393 val_393 1 +394 val_394 1 +395 val_395 1 +396 val_396 1 +397 val_397 1 +399 val_399 1 +4 val_4 1 +400 val_400 1 +401 val_401 1 +402 val_402 1 +403 val_403 1 +404 val_404 1 +406 val_406 1 +407 val_407 1 +409 val_409 1 +41 val_41 1 +411 val_411 1 +413 val_413 1 +414 val_414 1 +417 val_417 1 +418 val_418 1 +419 val_419 1 +42 val_42 1 +421 val_421 1 +424 val_424 1 +427 val_427 1 +429 val_429 1 +43 val_43 1 +430 val_430 1 +431 val_431 1 +432 val_432 1 +435 val_435 1 +436 val_436 1 +437 val_437 1 +438 val_438 1 +439 val_439 1 +44 val_44 1 +443 val_443 1 +444 val_444 1 +446 val_446 1 +448 val_448 1 +449 val_449 1 +452 val_452 1 +453 val_453 1 +454 val_454 1 +455 val_455 1 +457 val_457 1 +458 val_458 1 +459 val_459 1 +460 val_460 1 +462 val_462 1 +463 val_463 1 +466 val_466 1 +467 val_467 1 +468 val_468 1 +469 val_469 1 +47 val_47 1 +470 val_470 1 +472 val_472 1 +475 val_475 1 +477 val_477 1 +478 val_478 1 +479 val_479 1 +480 val_480 1 +481 val_481 1 +482 val_482 1 +483 val_483 1 +484 val_484 1 +485 val_485 1 +487 val_487 1 +489 val_489 1 +490 val_490 1 +491 val_491 1 +492 val_492 1 +493 val_493 1 +494 val_494 1 +495 val_495 1 +496 val_496 1 +497 val_497 1 +498 val_498 1 +5 val_5 1 +51 val_51 1 +53 val_53 1 +54 val_54 1 +57 val_57 1 +58 val_58 1 +64 val_64 1 +65 val_65 1 +66 val_66 1 +67 val_67 1 +69 val_69 1 +70 val_70 1 +72 val_72 1 +74 val_74 1 +76 val_76 1 +77 val_77 1 +78 val_78 1 +8 val_8 1 +80 val_80 1 +82 val_82 1 +83 val_83 1 +84 val_84 1 +85 val_85 1 +86 val_86 1 +87 val_87 1 +9 val_9 1 +90 val_90 1 +92 val_92 1 +95 val_95 1 +96 val_96 1 +97 val_97 1 +98 val_98 1 +tst1 500 1 \ No newline at end of file diff --git a/src/test/hive/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a b/src/test/hive/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union18-1-9491c271b7e2c351ddcf126f73679892 b/src/test/hive/golden/union18-1-9491c271b7e2c351ddcf126f73679892 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 b/src/test/hive/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 b/src/test/hive/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 b/src/test/hive/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 new file mode 100644 index 0000000000000..3dee790d7c2ca --- /dev/null +++ b/src/test/hive/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 @@ -0,0 +1,501 @@ +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 +105 val_105 +11 val_11 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +12 val_12 +12 val_12 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +15 val_15 +15 val_15 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +17 val_17 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +18 val_18 +18 val_18 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +19 val_19 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +2 val_2 +20 val_20 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +24 val_24 +24 val_24 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +26 val_26 +26 val_26 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +27 val_27 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +28 val_28 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +30 val_30 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +33 val_33 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +34 val_34 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +35 val_35 +35 val_35 +35 val_35 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +37 val_37 +37 val_37 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +4 val_4 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +41 val_41 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +42 val_42 +42 val_42 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +43 val_43 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +44 val_44 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +47 val_47 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 +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 +tst1 500 \ No newline at end of file diff --git a/src/test/hive/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 b/src/test/hive/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 new file mode 100644 index 0000000000000..e438a64050723 --- /dev/null +++ b/src/test/hive/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 @@ -0,0 +1,501 @@ +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +10 val_10 val_10 +100 val_100 val_100 +100 val_100 val_100 +103 val_103 val_103 +103 val_103 val_103 +104 val_104 val_104 +104 val_104 val_104 +105 val_105 val_105 +11 val_11 val_11 +111 val_111 val_111 +113 val_113 val_113 +113 val_113 val_113 +114 val_114 val_114 +116 val_116 val_116 +118 val_118 val_118 +118 val_118 val_118 +119 val_119 val_119 +119 val_119 val_119 +119 val_119 val_119 +12 val_12 val_12 +12 val_12 val_12 +120 val_120 val_120 +120 val_120 val_120 +125 val_125 val_125 +125 val_125 val_125 +126 val_126 val_126 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +129 val_129 val_129 +129 val_129 val_129 +131 val_131 val_131 +133 val_133 val_133 +134 val_134 val_134 +134 val_134 val_134 +136 val_136 val_136 +137 val_137 val_137 +137 val_137 val_137 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +143 val_143 val_143 +145 val_145 val_145 +146 val_146 val_146 +146 val_146 val_146 +149 val_149 val_149 +149 val_149 val_149 +15 val_15 val_15 +15 val_15 val_15 +150 val_150 val_150 +152 val_152 val_152 +152 val_152 val_152 +153 val_153 val_153 +155 val_155 val_155 +156 val_156 val_156 +157 val_157 val_157 +158 val_158 val_158 +160 val_160 val_160 +162 val_162 val_162 +163 val_163 val_163 +164 val_164 val_164 +164 val_164 val_164 +165 val_165 val_165 +165 val_165 val_165 +166 val_166 val_166 +167 val_167 val_167 +167 val_167 val_167 +167 val_167 val_167 +168 val_168 val_168 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +17 val_17 val_17 +170 val_170 val_170 +172 val_172 val_172 +172 val_172 val_172 +174 val_174 val_174 +174 val_174 val_174 +175 val_175 val_175 +175 val_175 val_175 +176 val_176 val_176 +176 val_176 val_176 +177 val_177 val_177 +178 val_178 val_178 +179 val_179 val_179 +179 val_179 val_179 +18 val_18 val_18 +18 val_18 val_18 +180 val_180 val_180 +181 val_181 val_181 +183 val_183 val_183 +186 val_186 val_186 +187 val_187 val_187 +187 val_187 val_187 +187 val_187 val_187 +189 val_189 val_189 +19 val_19 val_19 +190 val_190 val_190 +191 val_191 val_191 +191 val_191 val_191 +192 val_192 val_192 +193 val_193 val_193 +193 val_193 val_193 +193 val_193 val_193 +194 val_194 val_194 +195 val_195 val_195 +195 val_195 val_195 +196 val_196 val_196 +197 val_197 val_197 +197 val_197 val_197 +199 val_199 val_199 +199 val_199 val_199 +199 val_199 val_199 +2 val_2 val_2 +20 val_20 val_20 +200 val_200 val_200 +200 val_200 val_200 +201 val_201 val_201 +202 val_202 val_202 +203 val_203 val_203 +203 val_203 val_203 +205 val_205 val_205 +205 val_205 val_205 +207 val_207 val_207 +207 val_207 val_207 +208 val_208 val_208 +208 val_208 val_208 +208 val_208 val_208 +209 val_209 val_209 +209 val_209 val_209 +213 val_213 val_213 +213 val_213 val_213 +214 val_214 val_214 +216 val_216 val_216 +216 val_216 val_216 +217 val_217 val_217 +217 val_217 val_217 +218 val_218 val_218 +219 val_219 val_219 +219 val_219 val_219 +221 val_221 val_221 +221 val_221 val_221 +222 val_222 val_222 +223 val_223 val_223 +223 val_223 val_223 +224 val_224 val_224 +224 val_224 val_224 +226 val_226 val_226 +228 val_228 val_228 +229 val_229 val_229 +229 val_229 val_229 +230 val_230 val_230 +230 val_230 val_230 +230 val_230 val_230 +230 val_230 val_230 +230 val_230 val_230 +233 val_233 val_233 +233 val_233 val_233 +235 val_235 val_235 +237 val_237 val_237 +237 val_237 val_237 +238 val_238 val_238 +238 val_238 val_238 +239 val_239 val_239 +239 val_239 val_239 +24 val_24 val_24 +24 val_24 val_24 +241 val_241 val_241 +242 val_242 val_242 +242 val_242 val_242 +244 val_244 val_244 +247 val_247 val_247 +248 val_248 val_248 +249 val_249 val_249 +252 val_252 val_252 +255 val_255 val_255 +255 val_255 val_255 +256 val_256 val_256 +256 val_256 val_256 +257 val_257 val_257 +258 val_258 val_258 +26 val_26 val_26 +26 val_26 val_26 +260 val_260 val_260 +262 val_262 val_262 +263 val_263 val_263 +265 val_265 val_265 +265 val_265 val_265 +266 val_266 val_266 +27 val_27 val_27 +272 val_272 val_272 +272 val_272 val_272 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +274 val_274 val_274 +275 val_275 val_275 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +278 val_278 val_278 +278 val_278 val_278 +28 val_28 val_28 +280 val_280 val_280 +280 val_280 val_280 +281 val_281 val_281 +281 val_281 val_281 +282 val_282 val_282 +282 val_282 val_282 +283 val_283 val_283 +284 val_284 val_284 +285 val_285 val_285 +286 val_286 val_286 +287 val_287 val_287 +288 val_288 val_288 +288 val_288 val_288 +289 val_289 val_289 +291 val_291 val_291 +292 val_292 val_292 +296 val_296 val_296 +298 val_298 val_298 +298 val_298 val_298 +298 val_298 val_298 +30 val_30 val_30 +302 val_302 val_302 +305 val_305 val_305 +306 val_306 val_306 +307 val_307 val_307 +307 val_307 val_307 +308 val_308 val_308 +309 val_309 val_309 +309 val_309 val_309 +310 val_310 val_310 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +315 val_315 val_315 +316 val_316 val_316 +316 val_316 val_316 +316 val_316 val_316 +317 val_317 val_317 +317 val_317 val_317 +318 val_318 val_318 +318 val_318 val_318 +318 val_318 val_318 +321 val_321 val_321 +321 val_321 val_321 +322 val_322 val_322 +322 val_322 val_322 +323 val_323 val_323 +325 val_325 val_325 +325 val_325 val_325 +327 val_327 val_327 +327 val_327 val_327 +327 val_327 val_327 +33 val_33 val_33 +331 val_331 val_331 +331 val_331 val_331 +332 val_332 val_332 +333 val_333 val_333 +333 val_333 val_333 +335 val_335 val_335 +336 val_336 val_336 +338 val_338 val_338 +339 val_339 val_339 +34 val_34 val_34 +341 val_341 val_341 +342 val_342 val_342 +342 val_342 val_342 +344 val_344 val_344 +344 val_344 val_344 +345 val_345 val_345 +348 val_348 val_348 +348 val_348 val_348 +348 val_348 val_348 +348 val_348 val_348 +348 val_348 val_348 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +351 val_351 val_351 +353 val_353 val_353 +353 val_353 val_353 +356 val_356 val_356 +360 val_360 val_360 +362 val_362 val_362 +364 val_364 val_364 +365 val_365 val_365 +366 val_366 val_366 +367 val_367 val_367 +367 val_367 val_367 +368 val_368 val_368 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +37 val_37 val_37 +37 val_37 val_37 +373 val_373 val_373 +374 val_374 val_374 +375 val_375 val_375 +377 val_377 val_377 +378 val_378 val_378 +379 val_379 val_379 +382 val_382 val_382 +382 val_382 val_382 +384 val_384 val_384 +384 val_384 val_384 +384 val_384 val_384 +386 val_386 val_386 +389 val_389 val_389 +392 val_392 val_392 +393 val_393 val_393 +394 val_394 val_394 +395 val_395 val_395 +395 val_395 val_395 +396 val_396 val_396 +396 val_396 val_396 +396 val_396 val_396 +397 val_397 val_397 +397 val_397 val_397 +399 val_399 val_399 +399 val_399 val_399 +4 val_4 val_4 +400 val_400 val_400 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +402 val_402 val_402 +403 val_403 val_403 +403 val_403 val_403 +403 val_403 val_403 +404 val_404 val_404 +404 val_404 val_404 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +407 val_407 val_407 +409 val_409 val_409 +409 val_409 val_409 +409 val_409 val_409 +41 val_41 val_41 +411 val_411 val_411 +413 val_413 val_413 +413 val_413 val_413 +414 val_414 val_414 +414 val_414 val_414 +417 val_417 val_417 +417 val_417 val_417 +417 val_417 val_417 +418 val_418 val_418 +419 val_419 val_419 +42 val_42 val_42 +42 val_42 val_42 +421 val_421 val_421 +424 val_424 val_424 +424 val_424 val_424 +427 val_427 val_427 +429 val_429 val_429 +429 val_429 val_429 +43 val_43 val_43 +430 val_430 val_430 +430 val_430 val_430 +430 val_430 val_430 +431 val_431 val_431 +431 val_431 val_431 +431 val_431 val_431 +432 val_432 val_432 +435 val_435 val_435 +436 val_436 val_436 +437 val_437 val_437 +438 val_438 val_438 +438 val_438 val_438 +438 val_438 val_438 +439 val_439 val_439 +439 val_439 val_439 +44 val_44 val_44 +443 val_443 val_443 +444 val_444 val_444 +446 val_446 val_446 +448 val_448 val_448 +449 val_449 val_449 +452 val_452 val_452 +453 val_453 val_453 +454 val_454 val_454 +454 val_454 val_454 +454 val_454 val_454 +455 val_455 val_455 +457 val_457 val_457 +458 val_458 val_458 +458 val_458 val_458 +459 val_459 val_459 +459 val_459 val_459 +460 val_460 val_460 +462 val_462 val_462 +462 val_462 val_462 +463 val_463 val_463 +463 val_463 val_463 +466 val_466 val_466 +466 val_466 val_466 +466 val_466 val_466 +467 val_467 val_467 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +469 val_469 val_469 +469 val_469 val_469 +469 val_469 val_469 +469 val_469 val_469 +469 val_469 val_469 +47 val_47 val_47 +470 val_470 val_470 +472 val_472 val_472 +475 val_475 val_475 +477 val_477 val_477 +478 val_478 val_478 +478 val_478 val_478 +479 val_479 val_479 +480 val_480 val_480 +480 val_480 val_480 +480 val_480 val_480 +481 val_481 val_481 +482 val_482 val_482 +483 val_483 val_483 +484 val_484 val_484 +485 val_485 val_485 +487 val_487 val_487 +489 val_489 val_489 +489 val_489 val_489 +489 val_489 val_489 +489 val_489 val_489 +490 val_490 val_490 +491 val_491 val_491 +492 val_492 val_492 +492 val_492 val_492 +493 val_493 val_493 +494 val_494 val_494 +495 val_495 val_495 +496 val_496 val_496 +497 val_497 val_497 +498 val_498 val_498 +498 val_498 val_498 +498 val_498 val_498 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +51 val_51 val_51 +51 val_51 val_51 +53 val_53 val_53 +54 val_54 val_54 +57 val_57 val_57 +58 val_58 val_58 +58 val_58 val_58 +64 val_64 val_64 +65 val_65 val_65 +66 val_66 val_66 +67 val_67 val_67 +67 val_67 val_67 +69 val_69 val_69 +70 val_70 val_70 +70 val_70 val_70 +70 val_70 val_70 +72 val_72 val_72 +72 val_72 val_72 +74 val_74 val_74 +76 val_76 val_76 +76 val_76 val_76 +77 val_77 val_77 +78 val_78 val_78 +8 val_8 val_8 +80 val_80 val_80 +82 val_82 val_82 +83 val_83 val_83 +83 val_83 val_83 +84 val_84 val_84 +84 val_84 val_84 +85 val_85 val_85 +86 val_86 val_86 +87 val_87 val_87 +9 val_9 val_9 +90 val_90 val_90 +90 val_90 val_90 +90 val_90 val_90 +92 val_92 val_92 +95 val_95 val_95 +95 val_95 val_95 +96 val_96 val_96 +97 val_97 val_97 +97 val_97 val_97 +98 val_98 val_98 +98 val_98 val_98 +tst1 500 500 \ No newline at end of file diff --git a/src/test/hive/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a b/src/test/hive/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union19-1-9491c271b7e2c351ddcf126f73679892 b/src/test/hive/golden/union19-1-9491c271b7e2c351ddcf126f73679892 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union19-2-b6161b64e57a2502c79662866782cbdf b/src/test/hive/golden/union19-2-b6161b64e57a2502c79662866782cbdf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 b/src/test/hive/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 b/src/test/hive/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 new file mode 100644 index 0000000000000..e4dfefc43e26a --- /dev/null +++ b/src/test/hive/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 @@ -0,0 +1,310 @@ +0 3 +10 1 +100 2 +103 2 +104 2 +105 1 +11 1 +111 1 +113 2 +114 1 +116 1 +118 2 +119 3 +12 2 +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 +15 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 +17 1 +170 1 +172 2 +174 2 +175 2 +176 2 +177 1 +178 1 +179 2 +18 2 +180 1 +181 1 +183 1 +186 1 +187 3 +189 1 +19 1 +190 1 +191 2 +192 1 +193 3 +194 1 +195 2 +196 1 +197 2 +199 3 +2 1 +20 1 +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 +24 2 +241 1 +242 2 +244 1 +247 1 +248 1 +249 1 +252 1 +255 2 +256 2 +257 1 +258 1 +26 2 +260 1 +262 1 +263 1 +265 2 +266 1 +27 1 +272 2 +273 3 +274 1 +275 1 +277 4 +278 2 +28 1 +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 +30 1 +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 +33 1 +331 2 +332 1 +333 2 +335 1 +336 1 +338 1 +339 1 +34 1 +341 1 +342 2 +344 2 +345 1 +348 5 +35 3 +351 1 +353 2 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 2 +368 1 +369 3 +37 2 +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 +4 1 +400 1 +401 5 +402 1 +403 3 +404 2 +406 4 +407 1 +409 3 +41 1 +411 1 +413 2 +414 2 +417 3 +418 1 +419 1 +42 2 +421 1 +424 2 +427 1 +429 2 +43 1 +430 3 +431 3 +432 1 +435 1 +436 1 +437 1 +438 3 +439 2 +44 1 +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 +47 1 +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 +5 3 +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 +8 1 +80 1 +82 1 +83 2 +84 2 +85 1 +86 1 +87 1 +9 1 +90 3 +92 1 +95 2 +96 1 +97 2 +98 2 +tst1 1 \ No newline at end of file diff --git a/src/test/hive/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 b/src/test/hive/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 new file mode 100644 index 0000000000000..e438a64050723 --- /dev/null +++ b/src/test/hive/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 @@ -0,0 +1,501 @@ +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +10 val_10 val_10 +100 val_100 val_100 +100 val_100 val_100 +103 val_103 val_103 +103 val_103 val_103 +104 val_104 val_104 +104 val_104 val_104 +105 val_105 val_105 +11 val_11 val_11 +111 val_111 val_111 +113 val_113 val_113 +113 val_113 val_113 +114 val_114 val_114 +116 val_116 val_116 +118 val_118 val_118 +118 val_118 val_118 +119 val_119 val_119 +119 val_119 val_119 +119 val_119 val_119 +12 val_12 val_12 +12 val_12 val_12 +120 val_120 val_120 +120 val_120 val_120 +125 val_125 val_125 +125 val_125 val_125 +126 val_126 val_126 +128 val_128 val_128 +128 val_128 val_128 +128 val_128 val_128 +129 val_129 val_129 +129 val_129 val_129 +131 val_131 val_131 +133 val_133 val_133 +134 val_134 val_134 +134 val_134 val_134 +136 val_136 val_136 +137 val_137 val_137 +137 val_137 val_137 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +143 val_143 val_143 +145 val_145 val_145 +146 val_146 val_146 +146 val_146 val_146 +149 val_149 val_149 +149 val_149 val_149 +15 val_15 val_15 +15 val_15 val_15 +150 val_150 val_150 +152 val_152 val_152 +152 val_152 val_152 +153 val_153 val_153 +155 val_155 val_155 +156 val_156 val_156 +157 val_157 val_157 +158 val_158 val_158 +160 val_160 val_160 +162 val_162 val_162 +163 val_163 val_163 +164 val_164 val_164 +164 val_164 val_164 +165 val_165 val_165 +165 val_165 val_165 +166 val_166 val_166 +167 val_167 val_167 +167 val_167 val_167 +167 val_167 val_167 +168 val_168 val_168 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +17 val_17 val_17 +170 val_170 val_170 +172 val_172 val_172 +172 val_172 val_172 +174 val_174 val_174 +174 val_174 val_174 +175 val_175 val_175 +175 val_175 val_175 +176 val_176 val_176 +176 val_176 val_176 +177 val_177 val_177 +178 val_178 val_178 +179 val_179 val_179 +179 val_179 val_179 +18 val_18 val_18 +18 val_18 val_18 +180 val_180 val_180 +181 val_181 val_181 +183 val_183 val_183 +186 val_186 val_186 +187 val_187 val_187 +187 val_187 val_187 +187 val_187 val_187 +189 val_189 val_189 +19 val_19 val_19 +190 val_190 val_190 +191 val_191 val_191 +191 val_191 val_191 +192 val_192 val_192 +193 val_193 val_193 +193 val_193 val_193 +193 val_193 val_193 +194 val_194 val_194 +195 val_195 val_195 +195 val_195 val_195 +196 val_196 val_196 +197 val_197 val_197 +197 val_197 val_197 +199 val_199 val_199 +199 val_199 val_199 +199 val_199 val_199 +2 val_2 val_2 +20 val_20 val_20 +200 val_200 val_200 +200 val_200 val_200 +201 val_201 val_201 +202 val_202 val_202 +203 val_203 val_203 +203 val_203 val_203 +205 val_205 val_205 +205 val_205 val_205 +207 val_207 val_207 +207 val_207 val_207 +208 val_208 val_208 +208 val_208 val_208 +208 val_208 val_208 +209 val_209 val_209 +209 val_209 val_209 +213 val_213 val_213 +213 val_213 val_213 +214 val_214 val_214 +216 val_216 val_216 +216 val_216 val_216 +217 val_217 val_217 +217 val_217 val_217 +218 val_218 val_218 +219 val_219 val_219 +219 val_219 val_219 +221 val_221 val_221 +221 val_221 val_221 +222 val_222 val_222 +223 val_223 val_223 +223 val_223 val_223 +224 val_224 val_224 +224 val_224 val_224 +226 val_226 val_226 +228 val_228 val_228 +229 val_229 val_229 +229 val_229 val_229 +230 val_230 val_230 +230 val_230 val_230 +230 val_230 val_230 +230 val_230 val_230 +230 val_230 val_230 +233 val_233 val_233 +233 val_233 val_233 +235 val_235 val_235 +237 val_237 val_237 +237 val_237 val_237 +238 val_238 val_238 +238 val_238 val_238 +239 val_239 val_239 +239 val_239 val_239 +24 val_24 val_24 +24 val_24 val_24 +241 val_241 val_241 +242 val_242 val_242 +242 val_242 val_242 +244 val_244 val_244 +247 val_247 val_247 +248 val_248 val_248 +249 val_249 val_249 +252 val_252 val_252 +255 val_255 val_255 +255 val_255 val_255 +256 val_256 val_256 +256 val_256 val_256 +257 val_257 val_257 +258 val_258 val_258 +26 val_26 val_26 +26 val_26 val_26 +260 val_260 val_260 +262 val_262 val_262 +263 val_263 val_263 +265 val_265 val_265 +265 val_265 val_265 +266 val_266 val_266 +27 val_27 val_27 +272 val_272 val_272 +272 val_272 val_272 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +274 val_274 val_274 +275 val_275 val_275 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +278 val_278 val_278 +278 val_278 val_278 +28 val_28 val_28 +280 val_280 val_280 +280 val_280 val_280 +281 val_281 val_281 +281 val_281 val_281 +282 val_282 val_282 +282 val_282 val_282 +283 val_283 val_283 +284 val_284 val_284 +285 val_285 val_285 +286 val_286 val_286 +287 val_287 val_287 +288 val_288 val_288 +288 val_288 val_288 +289 val_289 val_289 +291 val_291 val_291 +292 val_292 val_292 +296 val_296 val_296 +298 val_298 val_298 +298 val_298 val_298 +298 val_298 val_298 +30 val_30 val_30 +302 val_302 val_302 +305 val_305 val_305 +306 val_306 val_306 +307 val_307 val_307 +307 val_307 val_307 +308 val_308 val_308 +309 val_309 val_309 +309 val_309 val_309 +310 val_310 val_310 +311 val_311 val_311 +311 val_311 val_311 +311 val_311 val_311 +315 val_315 val_315 +316 val_316 val_316 +316 val_316 val_316 +316 val_316 val_316 +317 val_317 val_317 +317 val_317 val_317 +318 val_318 val_318 +318 val_318 val_318 +318 val_318 val_318 +321 val_321 val_321 +321 val_321 val_321 +322 val_322 val_322 +322 val_322 val_322 +323 val_323 val_323 +325 val_325 val_325 +325 val_325 val_325 +327 val_327 val_327 +327 val_327 val_327 +327 val_327 val_327 +33 val_33 val_33 +331 val_331 val_331 +331 val_331 val_331 +332 val_332 val_332 +333 val_333 val_333 +333 val_333 val_333 +335 val_335 val_335 +336 val_336 val_336 +338 val_338 val_338 +339 val_339 val_339 +34 val_34 val_34 +341 val_341 val_341 +342 val_342 val_342 +342 val_342 val_342 +344 val_344 val_344 +344 val_344 val_344 +345 val_345 val_345 +348 val_348 val_348 +348 val_348 val_348 +348 val_348 val_348 +348 val_348 val_348 +348 val_348 val_348 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +351 val_351 val_351 +353 val_353 val_353 +353 val_353 val_353 +356 val_356 val_356 +360 val_360 val_360 +362 val_362 val_362 +364 val_364 val_364 +365 val_365 val_365 +366 val_366 val_366 +367 val_367 val_367 +367 val_367 val_367 +368 val_368 val_368 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +37 val_37 val_37 +37 val_37 val_37 +373 val_373 val_373 +374 val_374 val_374 +375 val_375 val_375 +377 val_377 val_377 +378 val_378 val_378 +379 val_379 val_379 +382 val_382 val_382 +382 val_382 val_382 +384 val_384 val_384 +384 val_384 val_384 +384 val_384 val_384 +386 val_386 val_386 +389 val_389 val_389 +392 val_392 val_392 +393 val_393 val_393 +394 val_394 val_394 +395 val_395 val_395 +395 val_395 val_395 +396 val_396 val_396 +396 val_396 val_396 +396 val_396 val_396 +397 val_397 val_397 +397 val_397 val_397 +399 val_399 val_399 +399 val_399 val_399 +4 val_4 val_4 +400 val_400 val_400 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +401 val_401 val_401 +402 val_402 val_402 +403 val_403 val_403 +403 val_403 val_403 +403 val_403 val_403 +404 val_404 val_404 +404 val_404 val_404 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +407 val_407 val_407 +409 val_409 val_409 +409 val_409 val_409 +409 val_409 val_409 +41 val_41 val_41 +411 val_411 val_411 +413 val_413 val_413 +413 val_413 val_413 +414 val_414 val_414 +414 val_414 val_414 +417 val_417 val_417 +417 val_417 val_417 +417 val_417 val_417 +418 val_418 val_418 +419 val_419 val_419 +42 val_42 val_42 +42 val_42 val_42 +421 val_421 val_421 +424 val_424 val_424 +424 val_424 val_424 +427 val_427 val_427 +429 val_429 val_429 +429 val_429 val_429 +43 val_43 val_43 +430 val_430 val_430 +430 val_430 val_430 +430 val_430 val_430 +431 val_431 val_431 +431 val_431 val_431 +431 val_431 val_431 +432 val_432 val_432 +435 val_435 val_435 +436 val_436 val_436 +437 val_437 val_437 +438 val_438 val_438 +438 val_438 val_438 +438 val_438 val_438 +439 val_439 val_439 +439 val_439 val_439 +44 val_44 val_44 +443 val_443 val_443 +444 val_444 val_444 +446 val_446 val_446 +448 val_448 val_448 +449 val_449 val_449 +452 val_452 val_452 +453 val_453 val_453 +454 val_454 val_454 +454 val_454 val_454 +454 val_454 val_454 +455 val_455 val_455 +457 val_457 val_457 +458 val_458 val_458 +458 val_458 val_458 +459 val_459 val_459 +459 val_459 val_459 +460 val_460 val_460 +462 val_462 val_462 +462 val_462 val_462 +463 val_463 val_463 +463 val_463 val_463 +466 val_466 val_466 +466 val_466 val_466 +466 val_466 val_466 +467 val_467 val_467 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +469 val_469 val_469 +469 val_469 val_469 +469 val_469 val_469 +469 val_469 val_469 +469 val_469 val_469 +47 val_47 val_47 +470 val_470 val_470 +472 val_472 val_472 +475 val_475 val_475 +477 val_477 val_477 +478 val_478 val_478 +478 val_478 val_478 +479 val_479 val_479 +480 val_480 val_480 +480 val_480 val_480 +480 val_480 val_480 +481 val_481 val_481 +482 val_482 val_482 +483 val_483 val_483 +484 val_484 val_484 +485 val_485 val_485 +487 val_487 val_487 +489 val_489 val_489 +489 val_489 val_489 +489 val_489 val_489 +489 val_489 val_489 +490 val_490 val_490 +491 val_491 val_491 +492 val_492 val_492 +492 val_492 val_492 +493 val_493 val_493 +494 val_494 val_494 +495 val_495 val_495 +496 val_496 val_496 +497 val_497 val_497 +498 val_498 val_498 +498 val_498 val_498 +498 val_498 val_498 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +51 val_51 val_51 +51 val_51 val_51 +53 val_53 val_53 +54 val_54 val_54 +57 val_57 val_57 +58 val_58 val_58 +58 val_58 val_58 +64 val_64 val_64 +65 val_65 val_65 +66 val_66 val_66 +67 val_67 val_67 +67 val_67 val_67 +69 val_69 val_69 +70 val_70 val_70 +70 val_70 val_70 +70 val_70 val_70 +72 val_72 val_72 +72 val_72 val_72 +74 val_74 val_74 +76 val_76 val_76 +76 val_76 val_76 +77 val_77 val_77 +78 val_78 val_78 +8 val_8 val_8 +80 val_80 val_80 +82 val_82 val_82 +83 val_83 val_83 +83 val_83 val_83 +84 val_84 val_84 +84 val_84 val_84 +85 val_85 val_85 +86 val_86 val_86 +87 val_87 val_87 +9 val_9 val_9 +90 val_90 val_90 +90 val_90 val_90 +90 val_90 val_90 +92 val_92 val_92 +95 val_95 val_95 +95 val_95 val_95 +96 val_96 val_96 +97 val_97 val_97 +97 val_97 val_97 +98 val_98 val_98 +98 val_98 val_98 +tst1 500 500 \ No newline at end of file diff --git a/src/test/hive/golden/union2-0-58813daf3d1af57b17518b606f8345b2 b/src/test/hive/golden/union2-0-58813daf3d1af57b17518b606f8345b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 b/src/test/hive/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 new file mode 100644 index 0000000000000..e37d32abba426 --- /dev/null +++ b/src/test/hive/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 @@ -0,0 +1 @@ +1000 \ No newline at end of file diff --git a/src/test/hive/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d b/src/test/hive/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union20-1-968e353589f1fddb914242beb25be94c b/src/test/hive/golden/union20-1-968e353589f1fddb914242beb25be94c new file mode 100644 index 0000000000000..385b8df6703a0 --- /dev/null +++ b/src/test/hive/golden/union20-1-968e353589f1fddb914242beb25be94c @@ -0,0 +1,23 @@ +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +4 val_4 4 val_4 +8 val_8 8 val_8 +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 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +2 val_2 2 val_2 +5 val_5 5 val_5 +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 diff --git a/src/test/hive/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c b/src/test/hive/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 b/src/test/hive/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 new file mode 100644 index 0000000000000..26a2aff52e9a9 --- /dev/null +++ b/src/test/hive/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 @@ -0,0 +1,536 @@ +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/src/test/hive/golden/union22-0-4bd6583b3635f1db95765565518bda8d b/src/test/hive/golden/union22-0-4bd6583b3635f1db95765565518bda8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 b/src/test/hive/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union22-10-90e9c4388340428a1f68868e9322c400 b/src/test/hive/golden/union22-10-90e9c4388340428a1f68868e9322c400 new file mode 100644 index 0000000000000..a9df5cf33c86d --- /dev/null +++ b/src/test/hive/golden/union22-10-90e9c4388340428a1f68868e9322c400 @@ -0,0 +1,1016 @@ +0 val_0 0 val_0 2 +0 val_0 0 val_0 2 +0 val_0 0 val_0 2 +10 val_10 10 val_10 2 +100 val_100 100 val_100 2 +100 val_100 100 val_100 2 +100 val_100 100 val_100 2 +100 val_100 100 val_100 2 +103 val_103 103 val_103 2 +103 val_103 103 val_103 2 +103 val_103 103 val_103 2 +103 val_103 103 val_103 2 +104 val_104 104 val_104 2 +104 val_104 104 val_104 2 +104 val_104 104 val_104 2 +104 val_104 104 val_104 2 +105 val_105 105 val_105 2 +11 val_11 11 val_11 2 +111 val_111 111 val_111 2 +113 val_113 113 val_113 2 +113 val_113 113 val_113 2 +113 val_113 113 val_113 2 +113 val_113 113 val_113 2 +114 val_114 114 val_114 2 +116 val_116 116 val_116 2 +118 val_118 118 val_118 2 +118 val_118 118 val_118 2 +118 val_118 118 val_118 2 +118 val_118 118 val_118 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +119 val_119 119 val_119 2 +12 val_12 12 val_12 2 +12 val_12 12 val_12 2 +120 val_120 120 val_120 2 +120 val_120 120 val_120 2 +120 val_120 120 val_120 2 +120 val_120 120 val_120 2 +125 val_125 125 val_125 2 +125 val_125 125 val_125 2 +125 val_125 125 val_125 2 +125 val_125 125 val_125 2 +126 val_126 126 val_126 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +128 val_128 128 val_128 2 +129 val_129 129 val_129 2 +129 val_129 129 val_129 2 +129 val_129 129 val_129 2 +129 val_129 129 val_129 2 +131 val_131 131 val_131 2 +133 val_133 133 val_133 2 +134 val_134 134 val_134 2 +134 val_134 134 val_134 2 +134 val_134 134 val_134 2 +134 val_134 134 val_134 2 +136 val_136 136 val_136 2 +137 val_137 137 val_137 2 +137 val_137 137 val_137 2 +137 val_137 137 val_137 2 +137 val_137 137 val_137 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +138 val_138 138 val_138 2 +143 val_143 143 val_143 2 +145 val_145 145 val_145 2 +146 val_146 146 val_146 2 +146 val_146 146 val_146 2 +146 val_146 146 val_146 2 +146 val_146 146 val_146 2 +149 val_149 149 val_149 2 +149 val_149 149 val_149 2 +149 val_149 149 val_149 2 +149 val_149 149 val_149 2 +15 val_15 15 val_15 2 +15 val_15 15 val_15 2 +150 val_150 150 val_150 2 +152 val_152 152 val_152 2 +152 val_152 152 val_152 2 +152 val_152 152 val_152 2 +152 val_152 152 val_152 2 +153 val_153 153 val_153 2 +155 val_155 155 val_155 2 +156 val_156 156 val_156 2 +157 val_157 157 val_157 2 +158 val_158 158 val_158 2 +160 val_160 160 val_160 2 +162 val_162 162 val_162 2 +163 val_163 163 val_163 2 +164 val_164 164 val_164 2 +164 val_164 164 val_164 2 +164 val_164 164 val_164 2 +164 val_164 164 val_164 2 +165 val_165 165 val_165 2 +165 val_165 165 val_165 2 +165 val_165 165 val_165 2 +165 val_165 165 val_165 2 +166 val_166 166 val_166 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +167 val_167 167 val_167 2 +168 val_168 168 val_168 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +169 val_169 169 val_169 2 +17 val_17 17 val_17 2 +170 val_170 170 val_170 2 +172 val_172 172 val_172 2 +172 val_172 172 val_172 2 +172 val_172 172 val_172 2 +172 val_172 172 val_172 2 +174 val_174 174 val_174 2 +174 val_174 174 val_174 2 +174 val_174 174 val_174 2 +174 val_174 174 val_174 2 +175 val_175 175 val_175 2 +175 val_175 175 val_175 2 +175 val_175 175 val_175 2 +175 val_175 175 val_175 2 +176 val_176 176 val_176 2 +176 val_176 176 val_176 2 +176 val_176 176 val_176 2 +176 val_176 176 val_176 2 +177 val_177 177 val_177 2 +178 val_178 178 val_178 2 +179 val_179 179 val_179 2 +179 val_179 179 val_179 2 +179 val_179 179 val_179 2 +179 val_179 179 val_179 2 +18 val_18 18 val_18 2 +18 val_18 18 val_18 2 +180 val_180 180 val_180 2 +181 val_181 181 val_181 2 +183 val_183 183 val_183 2 +186 val_186 186 val_186 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +187 val_187 187 val_187 2 +189 val_189 189 val_189 2 +19 val_19 19 val_19 2 +190 val_190 190 val_190 2 +191 val_191 191 val_191 2 +191 val_191 191 val_191 2 +191 val_191 191 val_191 2 +191 val_191 191 val_191 2 +192 val_192 192 val_192 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +193 val_193 193 val_193 2 +194 val_194 194 val_194 2 +195 val_195 195 val_195 2 +195 val_195 195 val_195 2 +195 val_195 195 val_195 2 +195 val_195 195 val_195 2 +196 val_196 196 val_196 2 +197 val_197 197 val_197 2 +197 val_197 197 val_197 2 +197 val_197 197 val_197 2 +197 val_197 197 val_197 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +199 val_199 199 val_199 2 +2 val_2 2 val_2 2 +20 val_20 20 val_20 2 +200 val_200 200 val_200 2 +200 val_200 200 val_200 2 +200 val_200 200 val_200 2 +200 val_200 200 val_200 2 +201 val_201 201 val_201 2 +202 val_202 202 val_202 2 +203 val_203 203 val_203 2 +203 val_203 203 val_203 2 +203 val_203 203 val_203 2 +203 val_203 203 val_203 2 +205 val_205 205 val_205 2 +205 val_205 205 val_205 2 +205 val_205 205 val_205 2 +205 val_205 205 val_205 2 +207 val_207 207 val_207 2 +207 val_207 207 val_207 2 +207 val_207 207 val_207 2 +207 val_207 207 val_207 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +208 val_208 208 val_208 2 +209 val_209 209 val_209 2 +209 val_209 209 val_209 2 +209 val_209 209 val_209 2 +209 val_209 209 val_209 2 +213 val_213 213 val_213 2 +213 val_213 213 val_213 2 +213 val_213 213 val_213 2 +213 val_213 213 val_213 2 +214 val_214 214 val_214 2 +216 val_216 216 val_216 2 +216 val_216 216 val_216 2 +216 val_216 216 val_216 2 +216 val_216 216 val_216 2 +217 val_217 217 val_217 2 +217 val_217 217 val_217 2 +217 val_217 217 val_217 2 +217 val_217 217 val_217 2 +218 val_218 218 val_218 2 +219 val_219 219 val_219 2 +219 val_219 219 val_219 2 +219 val_219 219 val_219 2 +219 val_219 219 val_219 2 +221 val_221 221 val_221 2 +221 val_221 221 val_221 2 +221 val_221 221 val_221 2 +221 val_221 221 val_221 2 +222 val_222 222 val_222 2 +223 val_223 223 val_223 2 +223 val_223 223 val_223 2 +223 val_223 223 val_223 2 +223 val_223 223 val_223 2 +224 val_224 224 val_224 2 +224 val_224 224 val_224 2 +224 val_224 224 val_224 2 +224 val_224 224 val_224 2 +226 val_226 226 val_226 2 +228 val_228 228 val_228 2 +229 val_229 229 val_229 2 +229 val_229 229 val_229 2 +229 val_229 229 val_229 2 +229 val_229 229 val_229 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +230 val_230 230 val_230 2 +233 val_233 233 val_233 2 +233 val_233 233 val_233 2 +233 val_233 233 val_233 2 +233 val_233 233 val_233 2 +235 val_235 235 val_235 2 +237 val_237 237 val_237 2 +237 val_237 237 val_237 2 +237 val_237 237 val_237 2 +237 val_237 237 val_237 2 +238 val_238 238 val_238 2 +238 val_238 238 val_238 2 +238 val_238 238 val_238 2 +238 val_238 238 val_238 2 +239 val_239 239 val_239 2 +239 val_239 239 val_239 2 +239 val_239 239 val_239 2 +239 val_239 239 val_239 2 +24 val_24 NULL NULL 2 +24 val_24 NULL NULL 2 +24 val_24 24 val_24 2 +24 val_24 24 val_24 2 +241 val_241 241 val_241 2 +242 val_242 242 val_242 2 +242 val_242 242 val_242 2 +242 val_242 242 val_242 2 +242 val_242 242 val_242 2 +244 val_244 244 val_244 2 +247 val_247 247 val_247 2 +248 val_248 248 val_248 2 +249 val_249 249 val_249 2 +252 val_252 252 val_252 2 +255 val_255 255 val_255 2 +255 val_255 255 val_255 2 +255 val_255 255 val_255 2 +255 val_255 255 val_255 2 +256 val_256 256 val_256 2 +256 val_256 256 val_256 2 +256 val_256 256 val_256 2 +256 val_256 256 val_256 2 +257 val_257 257 val_257 2 +258 val_258 258 val_258 2 +26 val_26 NULL NULL 2 +26 val_26 NULL NULL 2 +26 val_26 26 val_26 2 +26 val_26 26 val_26 2 +260 val_260 260 val_260 2 +262 val_262 262 val_262 2 +263 val_263 263 val_263 2 +265 val_265 265 val_265 2 +265 val_265 265 val_265 2 +265 val_265 265 val_265 2 +265 val_265 265 val_265 2 +266 val_266 266 val_266 2 +27 val_27 NULL NULL 2 +27 val_27 27 val_27 2 +272 val_272 272 val_272 2 +272 val_272 272 val_272 2 +272 val_272 272 val_272 2 +272 val_272 272 val_272 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +273 val_273 273 val_273 2 +274 val_274 274 val_274 2 +275 val_275 275 val_275 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +277 val_277 277 val_277 2 +278 val_278 278 val_278 2 +278 val_278 278 val_278 2 +278 val_278 278 val_278 2 +278 val_278 278 val_278 2 +28 val_28 NULL NULL 2 +28 val_28 28 val_28 2 +280 val_280 280 val_280 2 +280 val_280 280 val_280 2 +280 val_280 280 val_280 2 +280 val_280 280 val_280 2 +281 val_281 281 val_281 2 +281 val_281 281 val_281 2 +281 val_281 281 val_281 2 +281 val_281 281 val_281 2 +282 val_282 282 val_282 2 +282 val_282 282 val_282 2 +282 val_282 282 val_282 2 +282 val_282 282 val_282 2 +283 val_283 283 val_283 2 +284 val_284 284 val_284 2 +285 val_285 285 val_285 2 +286 val_286 286 val_286 2 +287 val_287 287 val_287 2 +288 val_288 288 val_288 2 +288 val_288 288 val_288 2 +288 val_288 288 val_288 2 +288 val_288 288 val_288 2 +289 val_289 289 val_289 2 +291 val_291 291 val_291 2 +292 val_292 292 val_292 2 +296 val_296 296 val_296 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +298 val_298 298 val_298 2 +30 val_30 NULL NULL 2 +30 val_30 30 val_30 2 +302 val_302 302 val_302 2 +305 val_305 305 val_305 2 +306 val_306 306 val_306 2 +307 val_307 307 val_307 2 +307 val_307 307 val_307 2 +307 val_307 307 val_307 2 +307 val_307 307 val_307 2 +308 val_308 308 val_308 2 +309 val_309 309 val_309 2 +309 val_309 309 val_309 2 +309 val_309 309 val_309 2 +309 val_309 309 val_309 2 +310 val_310 310 val_310 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +311 val_311 311 val_311 2 +315 val_315 315 val_315 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +316 val_316 316 val_316 2 +317 val_317 317 val_317 2 +317 val_317 317 val_317 2 +317 val_317 317 val_317 2 +317 val_317 317 val_317 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +318 val_318 318 val_318 2 +321 val_321 321 val_321 2 +321 val_321 321 val_321 2 +321 val_321 321 val_321 2 +321 val_321 321 val_321 2 +322 val_322 322 val_322 2 +322 val_322 322 val_322 2 +322 val_322 322 val_322 2 +322 val_322 322 val_322 2 +323 val_323 323 val_323 2 +325 val_325 325 val_325 2 +325 val_325 325 val_325 2 +325 val_325 325 val_325 2 +325 val_325 325 val_325 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +327 val_327 327 val_327 2 +33 val_33 NULL NULL 2 +33 val_33 33 val_33 2 +331 val_331 331 val_331 2 +331 val_331 331 val_331 2 +331 val_331 331 val_331 2 +331 val_331 331 val_331 2 +332 val_332 332 val_332 2 +333 val_333 333 val_333 2 +333 val_333 333 val_333 2 +333 val_333 333 val_333 2 +333 val_333 333 val_333 2 +335 val_335 335 val_335 2 +336 val_336 336 val_336 2 +338 val_338 338 val_338 2 +339 val_339 339 val_339 2 +34 val_34 NULL NULL 2 +34 val_34 34 val_34 2 +341 val_341 341 val_341 2 +342 val_342 342 val_342 2 +342 val_342 342 val_342 2 +342 val_342 342 val_342 2 +342 val_342 342 val_342 2 +344 val_344 344 val_344 2 +344 val_344 344 val_344 2 +344 val_344 344 val_344 2 +344 val_344 344 val_344 2 +345 val_345 345 val_345 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +348 val_348 348 val_348 2 +35 val_35 NULL NULL 2 +35 val_35 NULL NULL 2 +35 val_35 NULL NULL 2 +35 val_35 35 val_35 2 +35 val_35 35 val_35 2 +35 val_35 35 val_35 2 +351 val_351 351 val_351 2 +353 val_353 353 val_353 2 +353 val_353 353 val_353 2 +353 val_353 353 val_353 2 +353 val_353 353 val_353 2 +356 val_356 356 val_356 2 +360 val_360 360 val_360 2 +362 val_362 362 val_362 2 +364 val_364 364 val_364 2 +365 val_365 365 val_365 2 +366 val_366 366 val_366 2 +367 val_367 367 val_367 2 +367 val_367 367 val_367 2 +367 val_367 367 val_367 2 +367 val_367 367 val_367 2 +368 val_368 368 val_368 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +369 val_369 369 val_369 2 +37 val_37 NULL NULL 2 +37 val_37 NULL NULL 2 +37 val_37 37 val_37 2 +37 val_37 37 val_37 2 +373 val_373 373 val_373 2 +374 val_374 374 val_374 2 +375 val_375 375 val_375 2 +377 val_377 377 val_377 2 +378 val_378 378 val_378 2 +379 val_379 379 val_379 2 +382 val_382 382 val_382 2 +382 val_382 382 val_382 2 +382 val_382 382 val_382 2 +382 val_382 382 val_382 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +384 val_384 384 val_384 2 +386 val_386 386 val_386 2 +389 val_389 389 val_389 2 +392 val_392 392 val_392 2 +393 val_393 393 val_393 2 +394 val_394 394 val_394 2 +395 val_395 395 val_395 2 +395 val_395 395 val_395 2 +395 val_395 395 val_395 2 +395 val_395 395 val_395 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +396 val_396 396 val_396 2 +397 val_397 397 val_397 2 +397 val_397 397 val_397 2 +397 val_397 397 val_397 2 +397 val_397 397 val_397 2 +399 val_399 399 val_399 2 +399 val_399 399 val_399 2 +399 val_399 399 val_399 2 +399 val_399 399 val_399 2 +4 val_4 4 val_4 2 +400 val_400 400 val_400 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +401 val_401 401 val_401 2 +402 val_402 402 val_402 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +403 val_403 403 val_403 2 +404 val_404 404 val_404 2 +404 val_404 404 val_404 2 +404 val_404 404 val_404 2 +404 val_404 404 val_404 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +406 val_406 406 val_406 2 +407 val_407 407 val_407 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +409 val_409 409 val_409 2 +41 val_41 NULL NULL 2 +41 val_41 41 val_41 2 +411 val_411 411 val_411 2 +413 val_413 413 val_413 2 +413 val_413 413 val_413 2 +413 val_413 413 val_413 2 +413 val_413 413 val_413 2 +414 val_414 414 val_414 2 +414 val_414 414 val_414 2 +414 val_414 414 val_414 2 +414 val_414 414 val_414 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +417 val_417 417 val_417 2 +418 val_418 418 val_418 2 +419 val_419 419 val_419 2 +42 val_42 NULL NULL 2 +42 val_42 NULL NULL 2 +42 val_42 42 val_42 2 +42 val_42 42 val_42 2 +421 val_421 421 val_421 2 +424 val_424 424 val_424 2 +424 val_424 424 val_424 2 +424 val_424 424 val_424 2 +424 val_424 424 val_424 2 +427 val_427 427 val_427 2 +429 val_429 429 val_429 2 +429 val_429 429 val_429 2 +429 val_429 429 val_429 2 +429 val_429 429 val_429 2 +43 val_43 NULL NULL 2 +43 val_43 43 val_43 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +430 val_430 430 val_430 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +431 val_431 431 val_431 2 +432 val_432 432 val_432 2 +435 val_435 435 val_435 2 +436 val_436 436 val_436 2 +437 val_437 437 val_437 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +438 val_438 438 val_438 2 +439 val_439 439 val_439 2 +439 val_439 439 val_439 2 +439 val_439 439 val_439 2 +439 val_439 439 val_439 2 +44 val_44 NULL NULL 2 +44 val_44 44 val_44 2 +443 val_443 443 val_443 2 +444 val_444 444 val_444 2 +446 val_446 446 val_446 2 +448 val_448 448 val_448 2 +449 val_449 449 val_449 2 +452 val_452 452 val_452 2 +453 val_453 453 val_453 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +454 val_454 454 val_454 2 +455 val_455 455 val_455 2 +457 val_457 457 val_457 2 +458 val_458 458 val_458 2 +458 val_458 458 val_458 2 +458 val_458 458 val_458 2 +458 val_458 458 val_458 2 +459 val_459 459 val_459 2 +459 val_459 459 val_459 2 +459 val_459 459 val_459 2 +459 val_459 459 val_459 2 +460 val_460 460 val_460 2 +462 val_462 462 val_462 2 +462 val_462 462 val_462 2 +462 val_462 462 val_462 2 +462 val_462 462 val_462 2 +463 val_463 463 val_463 2 +463 val_463 463 val_463 2 +463 val_463 463 val_463 2 +463 val_463 463 val_463 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +466 val_466 466 val_466 2 +467 val_467 467 val_467 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +468 val_468 468 val_468 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +469 val_469 469 val_469 2 +47 val_47 NULL NULL 2 +47 val_47 47 val_47 2 +470 val_470 470 val_470 2 +472 val_472 472 val_472 2 +475 val_475 475 val_475 2 +477 val_477 477 val_477 2 +478 val_478 478 val_478 2 +478 val_478 478 val_478 2 +478 val_478 478 val_478 2 +478 val_478 478 val_478 2 +479 val_479 479 val_479 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +480 val_480 480 val_480 2 +481 val_481 481 val_481 2 +482 val_482 482 val_482 2 +483 val_483 483 val_483 2 +484 val_484 484 val_484 2 +485 val_485 485 val_485 2 +487 val_487 487 val_487 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +489 val_489 489 val_489 2 +490 val_490 490 val_490 2 +491 val_491 491 val_491 2 +492 val_492 492 val_492 2 +492 val_492 492 val_492 2 +492 val_492 492 val_492 2 +492 val_492 492 val_492 2 +493 val_493 493 val_493 2 +494 val_494 494 val_494 2 +495 val_495 495 val_495 2 +496 val_496 496 val_496 2 +497 val_497 497 val_497 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +498 val_498 498 val_498 2 +5 val_5 5 val_5 2 +5 val_5 5 val_5 2 +5 val_5 5 val_5 2 +51 val_51 51 val_51 2 +51 val_51 51 val_51 2 +51 val_51 51 val_51 2 +51 val_51 51 val_51 2 +53 val_53 53 val_53 2 +54 val_54 54 val_54 2 +57 val_57 57 val_57 2 +58 val_58 58 val_58 2 +58 val_58 58 val_58 2 +58 val_58 58 val_58 2 +58 val_58 58 val_58 2 +64 val_64 64 val_64 2 +65 val_65 65 val_65 2 +66 val_66 66 val_66 2 +67 val_67 67 val_67 2 +67 val_67 67 val_67 2 +67 val_67 67 val_67 2 +67 val_67 67 val_67 2 +69 val_69 69 val_69 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +70 val_70 70 val_70 2 +72 val_72 72 val_72 2 +72 val_72 72 val_72 2 +72 val_72 72 val_72 2 +72 val_72 72 val_72 2 +74 val_74 74 val_74 2 +76 val_76 76 val_76 2 +76 val_76 76 val_76 2 +76 val_76 76 val_76 2 +76 val_76 76 val_76 2 +77 val_77 77 val_77 2 +78 val_78 78 val_78 2 +8 val_8 8 val_8 2 +80 val_80 80 val_80 2 +82 val_82 82 val_82 2 +83 val_83 83 val_83 2 +83 val_83 83 val_83 2 +83 val_83 83 val_83 2 +83 val_83 83 val_83 2 +84 val_84 84 val_84 2 +84 val_84 84 val_84 2 +84 val_84 84 val_84 2 +84 val_84 84 val_84 2 +85 val_85 85 val_85 2 +86 val_86 86 val_86 2 +87 val_87 87 val_87 2 +9 val_9 9 val_9 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +90 val_90 90 val_90 2 +92 val_92 92 val_92 2 +95 val_95 95 val_95 2 +95 val_95 95 val_95 2 +95 val_95 95 val_95 2 +95 val_95 95 val_95 2 +96 val_96 96 val_96 2 +97 val_97 97 val_97 2 +97 val_97 97 val_97 2 +97 val_97 97 val_97 2 +97 val_97 97 val_97 2 +98 val_98 98 val_98 2 +98 val_98 98 val_98 2 +98 val_98 98 val_98 2 +98 val_98 98 val_98 2 diff --git a/src/test/hive/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 b/src/test/hive/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 b/src/test/hive/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union22-4-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union22-4-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union22-4-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union22-5-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/union22-5-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union22-5-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 b/src/test/hive/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union22-7-25fc734982956a164adde6bb1d4d8751 b/src/test/hive/golden/union22-7-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union22-7-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 b/src/test/hive/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 b/src/test/hive/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 b/src/test/hive/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union23-1-7830963417e3535034962e2597970ddd b/src/test/hive/golden/union23-1-7830963417e3535034962e2597970ddd new file mode 100644 index 0000000000000..bf0f76662bd2c --- /dev/null +++ b/src/test/hive/golden/union23-1-7830963417e3535034962e2597970ddd @@ -0,0 +1,1000 @@ +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +10 val_10 +10 val_10 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +105 val_105 +105 val_105 +11 val_11 +11 val_11 +111 val_111 +111 val_111 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +114 val_114 +114 val_114 +116 val_116 +116 val_116 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +126 val_126 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +131 val_131 +131 val_131 +133 val_133 +133 val_133 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +136 val_136 +136 val_136 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +143 val_143 +145 val_145 +145 val_145 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +150 val_150 +150 val_150 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +153 val_153 +153 val_153 +155 val_155 +155 val_155 +156 val_156 +156 val_156 +157 val_157 +157 val_157 +158 val_158 +158 val_158 +160 val_160 +160 val_160 +162 val_162 +162 val_162 +163 val_163 +163 val_163 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +166 val_166 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +17 val_17 +17 val_17 +170 val_170 +170 val_170 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +177 val_177 +177 val_177 +178 val_178 +178 val_178 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +180 val_180 +180 val_180 +181 val_181 +181 val_181 +183 val_183 +183 val_183 +186 val_186 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +189 val_189 +19 val_19 +19 val_19 +190 val_190 +190 val_190 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +192 val_192 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +194 val_194 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +196 val_196 +196 val_196 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +2 val_2 +2 val_2 +20 val_20 +20 val_20 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +201 val_201 +201 val_201 +202 val_202 +202 val_202 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +214 val_214 +214 val_214 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +218 val_218 +218 val_218 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +222 val_222 +222 val_222 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +226 val_226 +226 val_226 +228 val_228 +228 val_228 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +235 val_235 +235 val_235 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +241 val_241 +241 val_241 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +244 val_244 +244 val_244 +247 val_247 +247 val_247 +248 val_248 +248 val_248 +249 val_249 +249 val_249 +252 val_252 +252 val_252 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +257 val_257 +257 val_257 +258 val_258 +258 val_258 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +260 val_260 +260 val_260 +262 val_262 +262 val_262 +263 val_263 +263 val_263 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +266 val_266 +266 val_266 +27 val_27 +27 val_27 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +274 val_274 +275 val_275 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +28 val_28 +28 val_28 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +283 val_283 +283 val_283 +284 val_284 +284 val_284 +285 val_285 +285 val_285 +286 val_286 +286 val_286 +287 val_287 +287 val_287 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +289 val_289 +289 val_289 +291 val_291 +291 val_291 +292 val_292 +292 val_292 +296 val_296 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +30 val_30 +30 val_30 +302 val_302 +302 val_302 +305 val_305 +305 val_305 +306 val_306 +306 val_306 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +308 val_308 +308 val_308 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +310 val_310 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +323 val_323 +323 val_323 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +33 val_33 +33 val_33 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +332 val_332 +332 val_332 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +335 val_335 +335 val_335 +336 val_336 +336 val_336 +338 val_338 +338 val_338 +339 val_339 +339 val_339 +34 val_34 +34 val_34 +341 val_341 +341 val_341 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +345 val_345 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +351 val_351 +351 val_351 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +356 val_356 +356 val_356 +360 val_360 +360 val_360 +362 val_362 +362 val_362 +364 val_364 +364 val_364 +365 val_365 +365 val_365 +366 val_366 +366 val_366 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +368 val_368 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +373 val_373 +373 val_373 +374 val_374 +374 val_374 +375 val_375 +375 val_375 +377 val_377 +377 val_377 +378 val_378 +378 val_378 +379 val_379 +379 val_379 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +386 val_386 +389 val_389 +389 val_389 +392 val_392 +392 val_392 +393 val_393 +393 val_393 +394 val_394 +394 val_394 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +4 val_4 +4 val_4 +400 val_400 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +41 val_41 +41 val_41 +411 val_411 +411 val_411 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +418 val_418 +419 val_419 +419 val_419 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +421 val_421 +421 val_421 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +427 val_427 +427 val_427 +429 val_429 +429 val_429 +429 val_429 +429 val_429 +43 val_43 +43 val_43 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +432 val_432 +435 val_435 +435 val_435 +436 val_436 +436 val_436 +437 val_437 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +44 val_44 +44 val_44 +443 val_443 +443 val_443 +444 val_444 +444 val_444 +446 val_446 +446 val_446 +448 val_448 +448 val_448 +449 val_449 +449 val_449 +452 val_452 +452 val_452 +453 val_453 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +455 val_455 +457 val_457 +457 val_457 +458 val_458 +458 val_458 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +460 val_460 +460 val_460 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +47 val_47 +47 val_47 +470 val_470 +470 val_470 +472 val_472 +472 val_472 +475 val_475 +475 val_475 +477 val_477 +477 val_477 +478 val_478 +478 val_478 +478 val_478 +478 val_478 +479 val_479 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +481 val_481 +482 val_482 +482 val_482 +483 val_483 +483 val_483 +484 val_484 +484 val_484 +485 val_485 +485 val_485 +487 val_487 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +490 val_490 +491 val_491 +491 val_491 +492 val_492 +492 val_492 +492 val_492 +492 val_492 +493 val_493 +493 val_493 +494 val_494 +494 val_494 +495 val_495 +495 val_495 +496 val_496 +496 val_496 +497 val_497 +497 val_497 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +53 val_53 +53 val_53 +54 val_54 +54 val_54 +57 val_57 +57 val_57 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +64 val_64 +64 val_64 +65 val_65 +65 val_65 +66 val_66 +66 val_66 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +69 val_69 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +74 val_74 +74 val_74 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +77 val_77 +77 val_77 +78 val_78 +78 val_78 +8 val_8 +8 val_8 +80 val_80 +80 val_80 +82 val_82 +82 val_82 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +85 val_85 +85 val_85 +86 val_86 +86 val_86 +87 val_87 +87 val_87 +9 val_9 +9 val_9 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +92 val_92 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +96 val_96 +96 val_96 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +98 val_98 +98 val_98 \ No newline at end of file diff --git a/src/test/hive/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 b/src/test/hive/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c b/src/test/hive/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 b/src/test/hive/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a b/src/test/hive/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a new file mode 100644 index 0000000000000..e850208008112 --- /dev/null +++ b/src/test/hive/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a @@ -0,0 +1,18 @@ +0 1 +0 3 +0 3 +2 1 +2 1 +2 1 +4 1 +4 1 +4 1 +5 1 +5 3 +5 3 +8 1 +8 1 +8 1 +9 1 +9 1 +9 1 diff --git a/src/test/hive/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 b/src/test/hive/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f b/src/test/hive/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union24-4-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union24-4-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union24-4-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 b/src/test/hive/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 b/src/test/hive/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 new file mode 100644 index 0000000000000..2251af07b7c05 --- /dev/null +++ b/src/test/hive/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 @@ -0,0 +1,24 @@ +0 1 +0 3 +0 3 +0 3 +2 1 +2 1 +2 1 +2 1 +4 1 +4 1 +4 1 +4 1 +5 1 +5 3 +5 3 +5 3 +8 1 +8 1 +8 1 +8 1 +9 1 +9 1 +9 1 +9 1 diff --git a/src/test/hive/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb b/src/test/hive/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 b/src/test/hive/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 new file mode 100644 index 0000000000000..17fd2beaf0661 --- /dev/null +++ b/src/test/hive/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 @@ -0,0 +1,18 @@ +0 3 +0 3 +0 3 +2 1 +2 1 +2 1 +4 1 +4 1 +4 1 +5 3 +5 3 +5 3 +8 1 +8 1 +8 1 +9 1 +9 1 +9 1 diff --git a/src/test/hive/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d b/src/test/hive/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union26-1-4702087f160230463e6114d6534d47e4 b/src/test/hive/golden/union26-1-4702087f160230463e6114d6534d47e4 new file mode 100644 index 0000000000000..9ba1c19df23f3 --- /dev/null +++ b/src/test/hive/golden/union26-1-4702087f160230463e6114d6534d47e4 @@ -0,0 +1,309 @@ +18 0 val_0 +4 2 val_2 +4 4 val_4 +18 5 val_5 +4 8 val_8 +4 9 val_9 +4 10 val_10 +4 11 val_11 +10 12 val_12 +10 15 val_15 +4 17 val_17 +10 18 val_18 +4 19 val_19 +4 20 val_20 +10 24 val_24 +10 26 val_26 +4 27 val_27 +4 28 val_28 +4 30 val_30 +4 33 val_33 +4 34 val_34 +18 35 val_35 +10 37 val_37 +4 41 val_41 +10 42 val_42 +4 43 val_43 +4 44 val_44 +4 47 val_47 +10 51 val_51 +4 53 val_53 +4 54 val_54 +4 57 val_57 +10 58 val_58 +4 64 val_64 +4 65 val_65 +4 66 val_66 +10 67 val_67 +4 69 val_69 +18 70 val_70 +10 72 val_72 +4 74 val_74 +10 76 val_76 +4 77 val_77 +4 78 val_78 +4 80 val_80 +4 82 val_82 +10 83 val_83 +10 84 val_84 +4 85 val_85 +4 86 val_86 +4 87 val_87 +18 90 val_90 +4 92 val_92 +10 95 val_95 +4 96 val_96 +10 97 val_97 +10 98 val_98 +10 100 val_100 +10 103 val_103 +10 104 val_104 +4 105 val_105 +4 111 val_111 +10 113 val_113 +4 114 val_114 +4 116 val_116 +10 118 val_118 +18 119 val_119 +10 120 val_120 +10 125 val_125 +4 126 val_126 +18 128 val_128 +10 129 val_129 +4 131 val_131 +4 133 val_133 +10 134 val_134 +4 136 val_136 +10 137 val_137 +28 138 val_138 +4 143 val_143 +4 145 val_145 +10 146 val_146 +10 149 val_149 +4 150 val_150 +10 152 val_152 +4 153 val_153 +4 155 val_155 +4 156 val_156 +4 157 val_157 +4 158 val_158 +4 160 val_160 +4 162 val_162 +4 163 val_163 +10 164 val_164 +10 165 val_165 +4 166 val_166 +18 167 val_167 +4 168 val_168 +28 169 val_169 +4 170 val_170 +10 172 val_172 +10 174 val_174 +10 175 val_175 +10 176 val_176 +4 177 val_177 +4 178 val_178 +10 179 val_179 +4 180 val_180 +4 181 val_181 +4 183 val_183 +4 186 val_186 +18 187 val_187 +4 189 val_189 +4 190 val_190 +10 191 val_191 +4 192 val_192 +18 193 val_193 +4 194 val_194 +10 195 val_195 +4 196 val_196 +10 197 val_197 +18 199 val_199 +10 200 val_200 +4 201 val_201 +4 202 val_202 +10 203 val_203 +10 205 val_205 +10 207 val_207 +18 208 val_208 +10 209 val_209 +10 213 val_213 +4 214 val_214 +10 216 val_216 +10 217 val_217 +4 218 val_218 +10 219 val_219 +10 221 val_221 +4 222 val_222 +10 223 val_223 +10 224 val_224 +4 226 val_226 +4 228 val_228 +10 229 val_229 +40 230 val_230 +10 233 val_233 +4 235 val_235 +10 237 val_237 +10 238 val_238 +10 239 val_239 +4 241 val_241 +10 242 val_242 +4 244 val_244 +4 247 val_247 +4 248 val_248 +4 249 val_249 +4 252 val_252 +10 255 val_255 +10 256 val_256 +4 257 val_257 +4 258 val_258 +4 260 val_260 +4 262 val_262 +4 263 val_263 +10 265 val_265 +4 266 val_266 +10 272 val_272 +18 273 val_273 +4 274 val_274 +4 275 val_275 +28 277 val_277 +10 278 val_278 +10 280 val_280 +10 281 val_281 +10 282 val_282 +4 283 val_283 +4 284 val_284 +4 285 val_285 +4 286 val_286 +4 287 val_287 +10 288 val_288 +4 289 val_289 +4 291 val_291 +4 292 val_292 +4 296 val_296 +18 298 val_298 +4 302 val_302 +4 305 val_305 +4 306 val_306 +10 307 val_307 +4 308 val_308 +10 309 val_309 +4 310 val_310 +18 311 val_311 +4 315 val_315 +18 316 val_316 +10 317 val_317 +18 318 val_318 +10 321 val_321 +10 322 val_322 +4 323 val_323 +10 325 val_325 +18 327 val_327 +10 331 val_331 +4 332 val_332 +10 333 val_333 +4 335 val_335 +4 336 val_336 +4 338 val_338 +4 339 val_339 +4 341 val_341 +10 342 val_342 +10 344 val_344 +4 345 val_345 +40 348 val_348 +4 351 val_351 +10 353 val_353 +4 356 val_356 +4 360 val_360 +4 362 val_362 +4 364 val_364 +4 365 val_365 +4 366 val_366 +10 367 val_367 +4 368 val_368 +18 369 val_369 +4 373 val_373 +4 374 val_374 +4 375 val_375 +4 377 val_377 +4 378 val_378 +4 379 val_379 +10 382 val_382 +18 384 val_384 +4 386 val_386 +4 389 val_389 +4 392 val_392 +4 393 val_393 +4 394 val_394 +10 395 val_395 +18 396 val_396 +10 397 val_397 +10 399 val_399 +4 400 val_400 +40 401 val_401 +4 402 val_402 +18 403 val_403 +10 404 val_404 +28 406 val_406 +4 407 val_407 +18 409 val_409 +4 411 val_411 +10 413 val_413 +10 414 val_414 +18 417 val_417 +4 418 val_418 +4 419 val_419 +4 421 val_421 +10 424 val_424 +4 427 val_427 +10 429 val_429 +18 430 val_430 +18 431 val_431 +4 432 val_432 +4 435 val_435 +4 436 val_436 +4 437 val_437 +18 438 val_438 +10 439 val_439 +4 443 val_443 +4 444 val_444 +4 446 val_446 +4 448 val_448 +4 449 val_449 +4 452 val_452 +4 453 val_453 +18 454 val_454 +4 455 val_455 +4 457 val_457 +10 458 val_458 +10 459 val_459 +4 460 val_460 +10 462 val_462 +10 463 val_463 +18 466 val_466 +4 467 val_467 +28 468 val_468 +40 469 val_469 +4 470 val_470 +4 472 val_472 +4 475 val_475 +4 477 val_477 +10 478 val_478 +4 479 val_479 +18 480 val_480 +4 481 val_481 +4 482 val_482 +4 483 val_483 +4 484 val_484 +4 485 val_485 +4 487 val_487 +28 489 val_489 +4 490 val_490 +4 491 val_491 +10 492 val_492 +4 493 val_493 +4 494 val_494 +4 495 val_495 +4 496 val_496 +4 497 val_497 +18 498 val_498 diff --git a/src/test/hive/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 b/src/test/hive/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 new file mode 100644 index 0000000000000..9ba1c19df23f3 --- /dev/null +++ b/src/test/hive/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 @@ -0,0 +1,309 @@ +18 0 val_0 +4 2 val_2 +4 4 val_4 +18 5 val_5 +4 8 val_8 +4 9 val_9 +4 10 val_10 +4 11 val_11 +10 12 val_12 +10 15 val_15 +4 17 val_17 +10 18 val_18 +4 19 val_19 +4 20 val_20 +10 24 val_24 +10 26 val_26 +4 27 val_27 +4 28 val_28 +4 30 val_30 +4 33 val_33 +4 34 val_34 +18 35 val_35 +10 37 val_37 +4 41 val_41 +10 42 val_42 +4 43 val_43 +4 44 val_44 +4 47 val_47 +10 51 val_51 +4 53 val_53 +4 54 val_54 +4 57 val_57 +10 58 val_58 +4 64 val_64 +4 65 val_65 +4 66 val_66 +10 67 val_67 +4 69 val_69 +18 70 val_70 +10 72 val_72 +4 74 val_74 +10 76 val_76 +4 77 val_77 +4 78 val_78 +4 80 val_80 +4 82 val_82 +10 83 val_83 +10 84 val_84 +4 85 val_85 +4 86 val_86 +4 87 val_87 +18 90 val_90 +4 92 val_92 +10 95 val_95 +4 96 val_96 +10 97 val_97 +10 98 val_98 +10 100 val_100 +10 103 val_103 +10 104 val_104 +4 105 val_105 +4 111 val_111 +10 113 val_113 +4 114 val_114 +4 116 val_116 +10 118 val_118 +18 119 val_119 +10 120 val_120 +10 125 val_125 +4 126 val_126 +18 128 val_128 +10 129 val_129 +4 131 val_131 +4 133 val_133 +10 134 val_134 +4 136 val_136 +10 137 val_137 +28 138 val_138 +4 143 val_143 +4 145 val_145 +10 146 val_146 +10 149 val_149 +4 150 val_150 +10 152 val_152 +4 153 val_153 +4 155 val_155 +4 156 val_156 +4 157 val_157 +4 158 val_158 +4 160 val_160 +4 162 val_162 +4 163 val_163 +10 164 val_164 +10 165 val_165 +4 166 val_166 +18 167 val_167 +4 168 val_168 +28 169 val_169 +4 170 val_170 +10 172 val_172 +10 174 val_174 +10 175 val_175 +10 176 val_176 +4 177 val_177 +4 178 val_178 +10 179 val_179 +4 180 val_180 +4 181 val_181 +4 183 val_183 +4 186 val_186 +18 187 val_187 +4 189 val_189 +4 190 val_190 +10 191 val_191 +4 192 val_192 +18 193 val_193 +4 194 val_194 +10 195 val_195 +4 196 val_196 +10 197 val_197 +18 199 val_199 +10 200 val_200 +4 201 val_201 +4 202 val_202 +10 203 val_203 +10 205 val_205 +10 207 val_207 +18 208 val_208 +10 209 val_209 +10 213 val_213 +4 214 val_214 +10 216 val_216 +10 217 val_217 +4 218 val_218 +10 219 val_219 +10 221 val_221 +4 222 val_222 +10 223 val_223 +10 224 val_224 +4 226 val_226 +4 228 val_228 +10 229 val_229 +40 230 val_230 +10 233 val_233 +4 235 val_235 +10 237 val_237 +10 238 val_238 +10 239 val_239 +4 241 val_241 +10 242 val_242 +4 244 val_244 +4 247 val_247 +4 248 val_248 +4 249 val_249 +4 252 val_252 +10 255 val_255 +10 256 val_256 +4 257 val_257 +4 258 val_258 +4 260 val_260 +4 262 val_262 +4 263 val_263 +10 265 val_265 +4 266 val_266 +10 272 val_272 +18 273 val_273 +4 274 val_274 +4 275 val_275 +28 277 val_277 +10 278 val_278 +10 280 val_280 +10 281 val_281 +10 282 val_282 +4 283 val_283 +4 284 val_284 +4 285 val_285 +4 286 val_286 +4 287 val_287 +10 288 val_288 +4 289 val_289 +4 291 val_291 +4 292 val_292 +4 296 val_296 +18 298 val_298 +4 302 val_302 +4 305 val_305 +4 306 val_306 +10 307 val_307 +4 308 val_308 +10 309 val_309 +4 310 val_310 +18 311 val_311 +4 315 val_315 +18 316 val_316 +10 317 val_317 +18 318 val_318 +10 321 val_321 +10 322 val_322 +4 323 val_323 +10 325 val_325 +18 327 val_327 +10 331 val_331 +4 332 val_332 +10 333 val_333 +4 335 val_335 +4 336 val_336 +4 338 val_338 +4 339 val_339 +4 341 val_341 +10 342 val_342 +10 344 val_344 +4 345 val_345 +40 348 val_348 +4 351 val_351 +10 353 val_353 +4 356 val_356 +4 360 val_360 +4 362 val_362 +4 364 val_364 +4 365 val_365 +4 366 val_366 +10 367 val_367 +4 368 val_368 +18 369 val_369 +4 373 val_373 +4 374 val_374 +4 375 val_375 +4 377 val_377 +4 378 val_378 +4 379 val_379 +10 382 val_382 +18 384 val_384 +4 386 val_386 +4 389 val_389 +4 392 val_392 +4 393 val_393 +4 394 val_394 +10 395 val_395 +18 396 val_396 +10 397 val_397 +10 399 val_399 +4 400 val_400 +40 401 val_401 +4 402 val_402 +18 403 val_403 +10 404 val_404 +28 406 val_406 +4 407 val_407 +18 409 val_409 +4 411 val_411 +10 413 val_413 +10 414 val_414 +18 417 val_417 +4 418 val_418 +4 419 val_419 +4 421 val_421 +10 424 val_424 +4 427 val_427 +10 429 val_429 +18 430 val_430 +18 431 val_431 +4 432 val_432 +4 435 val_435 +4 436 val_436 +4 437 val_437 +18 438 val_438 +10 439 val_439 +4 443 val_443 +4 444 val_444 +4 446 val_446 +4 448 val_448 +4 449 val_449 +4 452 val_452 +4 453 val_453 +18 454 val_454 +4 455 val_455 +4 457 val_457 +10 458 val_458 +10 459 val_459 +4 460 val_460 +10 462 val_462 +10 463 val_463 +18 466 val_466 +4 467 val_467 +28 468 val_468 +40 469 val_469 +4 470 val_470 +4 472 val_472 +4 475 val_475 +4 477 val_477 +10 478 val_478 +4 479 val_479 +18 480 val_480 +4 481 val_481 +4 482 val_482 +4 483 val_483 +4 484 val_484 +4 485 val_485 +4 487 val_487 +28 489 val_489 +4 490 val_490 +4 491 val_491 +10 492 val_492 +4 493 val_493 +4 494 val_494 +4 495 val_495 +4 496 val_496 +4 497 val_497 +18 498 val_498 diff --git a/src/test/hive/golden/union26-3-4702087f160230463e6114d6534d47e4 b/src/test/hive/golden/union26-3-4702087f160230463e6114d6534d47e4 new file mode 100644 index 0000000000000..9ba1c19df23f3 --- /dev/null +++ b/src/test/hive/golden/union26-3-4702087f160230463e6114d6534d47e4 @@ -0,0 +1,309 @@ +18 0 val_0 +4 2 val_2 +4 4 val_4 +18 5 val_5 +4 8 val_8 +4 9 val_9 +4 10 val_10 +4 11 val_11 +10 12 val_12 +10 15 val_15 +4 17 val_17 +10 18 val_18 +4 19 val_19 +4 20 val_20 +10 24 val_24 +10 26 val_26 +4 27 val_27 +4 28 val_28 +4 30 val_30 +4 33 val_33 +4 34 val_34 +18 35 val_35 +10 37 val_37 +4 41 val_41 +10 42 val_42 +4 43 val_43 +4 44 val_44 +4 47 val_47 +10 51 val_51 +4 53 val_53 +4 54 val_54 +4 57 val_57 +10 58 val_58 +4 64 val_64 +4 65 val_65 +4 66 val_66 +10 67 val_67 +4 69 val_69 +18 70 val_70 +10 72 val_72 +4 74 val_74 +10 76 val_76 +4 77 val_77 +4 78 val_78 +4 80 val_80 +4 82 val_82 +10 83 val_83 +10 84 val_84 +4 85 val_85 +4 86 val_86 +4 87 val_87 +18 90 val_90 +4 92 val_92 +10 95 val_95 +4 96 val_96 +10 97 val_97 +10 98 val_98 +10 100 val_100 +10 103 val_103 +10 104 val_104 +4 105 val_105 +4 111 val_111 +10 113 val_113 +4 114 val_114 +4 116 val_116 +10 118 val_118 +18 119 val_119 +10 120 val_120 +10 125 val_125 +4 126 val_126 +18 128 val_128 +10 129 val_129 +4 131 val_131 +4 133 val_133 +10 134 val_134 +4 136 val_136 +10 137 val_137 +28 138 val_138 +4 143 val_143 +4 145 val_145 +10 146 val_146 +10 149 val_149 +4 150 val_150 +10 152 val_152 +4 153 val_153 +4 155 val_155 +4 156 val_156 +4 157 val_157 +4 158 val_158 +4 160 val_160 +4 162 val_162 +4 163 val_163 +10 164 val_164 +10 165 val_165 +4 166 val_166 +18 167 val_167 +4 168 val_168 +28 169 val_169 +4 170 val_170 +10 172 val_172 +10 174 val_174 +10 175 val_175 +10 176 val_176 +4 177 val_177 +4 178 val_178 +10 179 val_179 +4 180 val_180 +4 181 val_181 +4 183 val_183 +4 186 val_186 +18 187 val_187 +4 189 val_189 +4 190 val_190 +10 191 val_191 +4 192 val_192 +18 193 val_193 +4 194 val_194 +10 195 val_195 +4 196 val_196 +10 197 val_197 +18 199 val_199 +10 200 val_200 +4 201 val_201 +4 202 val_202 +10 203 val_203 +10 205 val_205 +10 207 val_207 +18 208 val_208 +10 209 val_209 +10 213 val_213 +4 214 val_214 +10 216 val_216 +10 217 val_217 +4 218 val_218 +10 219 val_219 +10 221 val_221 +4 222 val_222 +10 223 val_223 +10 224 val_224 +4 226 val_226 +4 228 val_228 +10 229 val_229 +40 230 val_230 +10 233 val_233 +4 235 val_235 +10 237 val_237 +10 238 val_238 +10 239 val_239 +4 241 val_241 +10 242 val_242 +4 244 val_244 +4 247 val_247 +4 248 val_248 +4 249 val_249 +4 252 val_252 +10 255 val_255 +10 256 val_256 +4 257 val_257 +4 258 val_258 +4 260 val_260 +4 262 val_262 +4 263 val_263 +10 265 val_265 +4 266 val_266 +10 272 val_272 +18 273 val_273 +4 274 val_274 +4 275 val_275 +28 277 val_277 +10 278 val_278 +10 280 val_280 +10 281 val_281 +10 282 val_282 +4 283 val_283 +4 284 val_284 +4 285 val_285 +4 286 val_286 +4 287 val_287 +10 288 val_288 +4 289 val_289 +4 291 val_291 +4 292 val_292 +4 296 val_296 +18 298 val_298 +4 302 val_302 +4 305 val_305 +4 306 val_306 +10 307 val_307 +4 308 val_308 +10 309 val_309 +4 310 val_310 +18 311 val_311 +4 315 val_315 +18 316 val_316 +10 317 val_317 +18 318 val_318 +10 321 val_321 +10 322 val_322 +4 323 val_323 +10 325 val_325 +18 327 val_327 +10 331 val_331 +4 332 val_332 +10 333 val_333 +4 335 val_335 +4 336 val_336 +4 338 val_338 +4 339 val_339 +4 341 val_341 +10 342 val_342 +10 344 val_344 +4 345 val_345 +40 348 val_348 +4 351 val_351 +10 353 val_353 +4 356 val_356 +4 360 val_360 +4 362 val_362 +4 364 val_364 +4 365 val_365 +4 366 val_366 +10 367 val_367 +4 368 val_368 +18 369 val_369 +4 373 val_373 +4 374 val_374 +4 375 val_375 +4 377 val_377 +4 378 val_378 +4 379 val_379 +10 382 val_382 +18 384 val_384 +4 386 val_386 +4 389 val_389 +4 392 val_392 +4 393 val_393 +4 394 val_394 +10 395 val_395 +18 396 val_396 +10 397 val_397 +10 399 val_399 +4 400 val_400 +40 401 val_401 +4 402 val_402 +18 403 val_403 +10 404 val_404 +28 406 val_406 +4 407 val_407 +18 409 val_409 +4 411 val_411 +10 413 val_413 +10 414 val_414 +18 417 val_417 +4 418 val_418 +4 419 val_419 +4 421 val_421 +10 424 val_424 +4 427 val_427 +10 429 val_429 +18 430 val_430 +18 431 val_431 +4 432 val_432 +4 435 val_435 +4 436 val_436 +4 437 val_437 +18 438 val_438 +10 439 val_439 +4 443 val_443 +4 444 val_444 +4 446 val_446 +4 448 val_448 +4 449 val_449 +4 452 val_452 +4 453 val_453 +18 454 val_454 +4 455 val_455 +4 457 val_457 +10 458 val_458 +10 459 val_459 +4 460 val_460 +10 462 val_462 +10 463 val_463 +18 466 val_466 +4 467 val_467 +28 468 val_468 +40 469 val_469 +4 470 val_470 +4 472 val_472 +4 475 val_475 +4 477 val_477 +10 478 val_478 +4 479 val_479 +18 480 val_480 +4 481 val_481 +4 482 val_482 +4 483 val_483 +4 484 val_484 +4 485 val_485 +4 487 val_487 +28 489 val_489 +4 490 val_490 +4 491 val_491 +10 492 val_492 +4 493 val_493 +4 494 val_494 +4 495 val_495 +4 496 val_496 +4 497 val_497 +18 498 val_498 diff --git a/src/test/hive/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 b/src/test/hive/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f b/src/test/hive/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 b/src/test/hive/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union27-3-ab84df3813ff23be99f148449610e530 b/src/test/hive/golden/union27-3-ab84df3813ff23be99f148449610e530 new file mode 100644 index 0000000000000..199095f1f9848 --- /dev/null +++ b/src/test/hive/golden/union27-3-ab84df3813ff23be99f148449610e530 @@ -0,0 +1,8 @@ +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 b/src/test/hive/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 b/src/test/hive/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f b/src/test/hive/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 b/src/test/hive/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 new file mode 100644 index 0000000000000..293f324297419 --- /dev/null +++ b/src/test/hive/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 @@ -0,0 +1,20 @@ +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +2 val_2 +2 val_2 +2 val_2 +4 val_4 +4 val_4 +4 val_4 +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 \ No newline at end of file diff --git a/src/test/hive/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 b/src/test/hive/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union29-1-a40036f756a306f4226634b6ab67c72c b/src/test/hive/golden/union29-1-a40036f756a306f4226634b6ab67c72c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 b/src/test/hive/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 b/src/test/hive/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 new file mode 100644 index 0000000000000..87a971baed428 --- /dev/null +++ b/src/test/hive/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 @@ -0,0 +1,20 @@ +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 +2 val_2 +2 val_2 +4 val_4 +4 val_4 +4 val_4 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 \ No newline at end of file diff --git a/src/test/hive/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d b/src/test/hive/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 b/src/test/hive/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 b/src/test/hive/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 b/src/test/hive/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 new file mode 100644 index 0000000000000..94ebaf9001613 --- /dev/null +++ b/src/test/hive/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 @@ -0,0 +1,4 @@ +1 +2 +3 +4 diff --git a/src/test/hive/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 b/src/test/hive/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 b/src/test/hive/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union30-2-252f12f7532ca67132bfc62063c58430 b/src/test/hive/golden/union30-2-252f12f7532ca67132bfc62063c58430 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 b/src/test/hive/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 new file mode 100644 index 0000000000000..a6f502e6c3e3a --- /dev/null +++ b/src/test/hive/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 @@ -0,0 +1,20 @@ +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 +2 val_2 +2 val_2 +2 val_2 +4 val_4 +4 val_4 +4 val_4 +4 val_4 +5 val_5 +5 val_5 +5 val_5 +5 val_5 \ No newline at end of file diff --git a/src/test/hive/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 b/src/test/hive/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-1-b12aa87aaf64b573ce0be7013117651b b/src/test/hive/golden/union31-1-b12aa87aaf64b573ce0be7013117651b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 b/src/test/hive/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 b/src/test/hive/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-12-de2477a134ab37bb22d879028becf2e5 b/src/test/hive/golden/union31-12-de2477a134ab37bb22d879028becf2e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 b/src/test/hive/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 b/src/test/hive/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 new file mode 100644 index 0000000000000..dadf53962e7a1 --- /dev/null +++ b/src/test/hive/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 @@ -0,0 +1,6 @@ +0 6 +2 2 +4 2 +5 6 +8 2 +9 2 \ No newline at end of file diff --git a/src/test/hive/golden/union31-15-5df6435aed6e0a6a6853480a027b911e b/src/test/hive/golden/union31-15-5df6435aed6e0a6a6853480a027b911e new file mode 100644 index 0000000000000..dadf53962e7a1 --- /dev/null +++ b/src/test/hive/golden/union31-15-5df6435aed6e0a6a6853480a027b911e @@ -0,0 +1,6 @@ +0 6 +2 2 +4 2 +5 6 +8 2 +9 2 \ No newline at end of file diff --git a/src/test/hive/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 b/src/test/hive/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-17-b12aa87aaf64b573ce0be7013117651b b/src/test/hive/golden/union31-17-b12aa87aaf64b573ce0be7013117651b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 b/src/test/hive/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e b/src/test/hive/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 b/src/test/hive/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-20-dceb6d9a03096d68d090557692ed6899 b/src/test/hive/golden/union31-20-dceb6d9a03096d68d090557692ed6899 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-21-bda2e62d0ecd30200b2359df420da574 b/src/test/hive/golden/union31-21-bda2e62d0ecd30200b2359df420da574 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a b/src/test/hive/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df b/src/test/hive/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-24-df38c8164af7cc164c728b8178da72c5 b/src/test/hive/golden/union31-24-df38c8164af7cc164c728b8178da72c5 new file mode 100644 index 0000000000000..ede645acaf95b --- /dev/null +++ b/src/test/hive/golden/union31-24-df38c8164af7cc164c728b8178da72c5 @@ -0,0 +1,6 @@ +0 2 +2 2 +4 2 +5 2 +8 2 +9 2 \ No newline at end of file diff --git a/src/test/hive/golden/union31-25-1485e295a99908e1862eae397b814045 b/src/test/hive/golden/union31-25-1485e295a99908e1862eae397b814045 new file mode 100644 index 0000000000000..ede645acaf95b --- /dev/null +++ b/src/test/hive/golden/union31-25-1485e295a99908e1862eae397b814045 @@ -0,0 +1,6 @@ +0 2 +2 2 +4 2 +5 2 +8 2 +9 2 \ No newline at end of file diff --git a/src/test/hive/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc b/src/test/hive/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-4-77653483e5ce18437c9f85d3b297335d b/src/test/hive/golden/union31-4-77653483e5ce18437c9f85d3b297335d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 b/src/test/hive/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 b/src/test/hive/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 b/src/test/hive/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb b/src/test/hive/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb new file mode 100644 index 0000000000000..dadf53962e7a1 --- /dev/null +++ b/src/test/hive/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb @@ -0,0 +1,6 @@ +0 6 +2 2 +4 2 +5 6 +8 2 +9 2 \ No newline at end of file diff --git a/src/test/hive/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 b/src/test/hive/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 new file mode 100644 index 0000000000000..a71793e8c52a3 --- /dev/null +++ b/src/test/hive/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 @@ -0,0 +1,6 @@ +val_0 6 +val_2 2 +val_4 2 +val_5 6 +val_8 2 +val_9 2 \ No newline at end of file diff --git a/src/test/hive/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e b/src/test/hive/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 b/src/test/hive/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b b/src/test/hive/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 b/src/test/hive/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 new file mode 100644 index 0000000000000..9bf0de2f06c9a --- /dev/null +++ b/src/test/hive/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 @@ -0,0 +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 +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/src/test/hive/golden/union32-2-8e39fef33c859ef83912d0fcda319218 b/src/test/hive/golden/union32-2-8e39fef33c859ef83912d0fcda319218 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 b/src/test/hive/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 new file mode 100644 index 0000000000000..462dca3124c41 --- /dev/null +++ b/src/test/hive/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 @@ -0,0 +1,20 @@ +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/src/test/hive/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e b/src/test/hive/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union32-5-51c997d0a1103b60764bbb8316a38746 b/src/test/hive/golden/union32-5-51c997d0a1103b60764bbb8316a38746 new file mode 100644 index 0000000000000..2e662f39d1572 --- /dev/null +++ b/src/test/hive/golden/union32-5-51c997d0a1103b60764bbb8316a38746 @@ -0,0 +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 +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/src/test/hive/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b b/src/test/hive/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union32-7-f936440d63f4e1027dda4de605660633 b/src/test/hive/golden/union32-7-f936440d63f4e1027dda4de605660633 new file mode 100644 index 0000000000000..2e662f39d1572 --- /dev/null +++ b/src/test/hive/golden/union32-7-f936440d63f4e1027dda4de605660633 @@ -0,0 +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 +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/src/test/hive/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb b/src/test/hive/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union32-9-74fb695786df4c024288ae23ac8c00db b/src/test/hive/golden/union32-9-74fb695786df4c024288ae23ac8c00db new file mode 100644 index 0000000000000..106f1874451ff --- /dev/null +++ b/src/test/hive/golden/union32-9-74fb695786df4c024288ae23ac8c00db @@ -0,0 +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 +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/src/test/hive/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f b/src/test/hive/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union33-1-1df7c476abb48234c839878414720873 b/src/test/hive/golden/union33-1-1df7c476abb48234c839878414720873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union33-2-3987150ad66482f5186826efcfb117f7 b/src/test/hive/golden/union33-2-3987150ad66482f5186826efcfb117f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union33-3-bfe75625806b414a5098af487b91046a b/src/test/hive/golden/union33-3-bfe75625806b414a5098af487b91046a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 b/src/test/hive/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 new file mode 100644 index 0000000000000..a1e0432c9a7d5 --- /dev/null +++ b/src/test/hive/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 @@ -0,0 +1 @@ +312 diff --git a/src/test/hive/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 b/src/test/hive/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 b/src/test/hive/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 b/src/test/hive/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 new file mode 100644 index 0000000000000..a1e0432c9a7d5 --- /dev/null +++ b/src/test/hive/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 @@ -0,0 +1 @@ +312 diff --git a/src/test/hive/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 b/src/test/hive/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union34-1-320c2fe802fab386957039aaff6399f1 b/src/test/hive/golden/union34-1-320c2fe802fab386957039aaff6399f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 b/src/test/hive/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 new file mode 100644 index 0000000000000..d572335ff5185 --- /dev/null +++ b/src/test/hive/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 @@ -0,0 +1,30 @@ +165 val_165 +165 val_165 +165 val_165 +238 val_238 +238 val_238 +238 val_238 +255 val_255 +255 val_255 +255 val_255 +27 val_27 +27 val_27 +27 val_27 +278 val_278 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +311 val_311 +409 val_409 +409 val_409 +409 val_409 +484 val_484 +484 val_484 +484 val_484 +86 val_86 +86 val_86 +86 val_86 +98 val_98 +98 val_98 +98 val_98 \ No newline at end of file diff --git a/src/test/hive/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 b/src/test/hive/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union34-3-c179e14d840249d6953cb552279822f1 b/src/test/hive/golden/union34-3-c179e14d840249d6953cb552279822f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union34-4-101829a66cab2efd31dcb0d86e302956 b/src/test/hive/golden/union34-4-101829a66cab2efd31dcb0d86e302956 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union34-5-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/union34-5-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union34-5-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f b/src/test/hive/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 b/src/test/hive/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 new file mode 100644 index 0000000000000..d572335ff5185 --- /dev/null +++ b/src/test/hive/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 @@ -0,0 +1,30 @@ +165 val_165 +165 val_165 +165 val_165 +238 val_238 +238 val_238 +238 val_238 +255 val_255 +255 val_255 +255 val_255 +27 val_27 +27 val_27 +27 val_27 +278 val_278 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +311 val_311 +409 val_409 +409 val_409 +409 val_409 +484 val_484 +484 val_484 +484 val_484 +86 val_86 +86 val_86 +86 val_86 +98 val_98 +98 val_98 +98 val_98 \ No newline at end of file diff --git a/src/test/hive/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b b/src/test/hive/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 b/src/test/hive/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union4-1-88deb3987655dc41989f39c0827cec74 b/src/test/hive/golden/union4-1-88deb3987655dc41989f39c0827cec74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 b/src/test/hive/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b b/src/test/hive/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 b/src/test/hive/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 new file mode 100644 index 0000000000000..948aca9180ba9 --- /dev/null +++ b/src/test/hive/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 @@ -0,0 +1,2 @@ +tst1 500 +tst2 500 \ No newline at end of file diff --git a/src/test/hive/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 b/src/test/hive/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 b/src/test/hive/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 new file mode 100644 index 0000000000000..c0c7d9f5e8ef5 --- /dev/null +++ b/src/test/hive/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 @@ -0,0 +1,2 @@ +tst1 1 +tst2 1 \ No newline at end of file diff --git a/src/test/hive/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d b/src/test/hive/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 b/src/test/hive/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 b/src/test/hive/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 b/src/test/hive/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 new file mode 100644 index 0000000000000..b5e91e032212a --- /dev/null +++ b/src/test/hive/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 @@ -0,0 +1,26 @@ +NULL +NULL +NULL +NULL +NULL val_165 +NULL val_193 +NULL val_265 +NULL val_27 +NULL val_409 +NULL val_484 +128 +146 val_146 +150 val_150 +213 val_213 +224 +238 val_238 +255 val_255 +273 val_273 +278 val_278 +311 val_311 +369 +401 val_401 +406 val_406 +66 val_66 +98 val_98 +tst1 500 \ No newline at end of file diff --git a/src/test/hive/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 b/src/test/hive/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 b/src/test/hive/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 b/src/test/hive/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 new file mode 100644 index 0000000000000..07f67df79dcbd --- /dev/null +++ b/src/test/hive/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 @@ -0,0 +1,17 @@ +NULL 10 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 +66 1 +98 1 +tst1 1 \ No newline at end of file diff --git a/src/test/hive/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e b/src/test/hive/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union8-1-1b422e4c1c8b97775518f760b995c771 b/src/test/hive/golden/union8-1-1b422e4c1c8b97775518f760b995c771 new file mode 100644 index 0000000000000..3617909902993 --- /dev/null +++ b/src/test/hive/golden/union8-1-1b422e4c1c8b97775518f760b995c771 @@ -0,0 +1,1500 @@ +238 val_238 +238 val_238 +238 val_238 +86 val_86 +86 val_86 +86 val_86 +311 val_311 +311 val_311 +311 val_311 +27 val_27 +27 val_27 +27 val_27 +165 val_165 +165 val_165 +165 val_165 +409 val_409 +409 val_409 +409 val_409 +255 val_255 +255 val_255 +255 val_255 +278 val_278 +278 val_278 +278 val_278 +98 val_98 +98 val_98 +98 val_98 +484 val_484 +484 val_484 +484 val_484 +265 val_265 +265 val_265 +265 val_265 +193 val_193 +193 val_193 +193 val_193 +401 val_401 +401 val_401 +401 val_401 +150 val_150 +150 val_150 +150 val_150 +273 val_273 +273 val_273 +273 val_273 +224 val_224 +224 val_224 +224 val_224 +369 val_369 +369 val_369 +369 val_369 +66 val_66 +66 val_66 +66 val_66 +128 val_128 +128 val_128 +128 val_128 +213 val_213 +213 val_213 +213 val_213 +146 val_146 +146 val_146 +146 val_146 +406 val_406 +406 val_406 +406 val_406 +429 val_429 +429 val_429 +429 val_429 +374 val_374 +374 val_374 +374 val_374 +152 val_152 +152 val_152 +152 val_152 +469 val_469 +469 val_469 +469 val_469 +145 val_145 +145 val_145 +145 val_145 +495 val_495 +495 val_495 +495 val_495 +37 val_37 +37 val_37 +37 val_37 +327 val_327 +327 val_327 +327 val_327 +281 val_281 +281 val_281 +281 val_281 +277 val_277 +277 val_277 +277 val_277 +209 val_209 +209 val_209 +209 val_209 +15 val_15 +15 val_15 +15 val_15 +82 val_82 +82 val_82 +82 val_82 +403 val_403 +403 val_403 +403 val_403 +166 val_166 +166 val_166 +166 val_166 +417 val_417 +417 val_417 +417 val_417 +430 val_430 +430 val_430 +430 val_430 +252 val_252 +252 val_252 +252 val_252 +292 val_292 +292 val_292 +292 val_292 +219 val_219 +219 val_219 +219 val_219 +287 val_287 +287 val_287 +287 val_287 +153 val_153 +153 val_153 +153 val_153 +193 val_193 +193 val_193 +193 val_193 +338 val_338 +338 val_338 +338 val_338 +446 val_446 +446 val_446 +446 val_446 +459 val_459 +459 val_459 +459 val_459 +394 val_394 +394 val_394 +394 val_394 +237 val_237 +237 val_237 +237 val_237 +482 val_482 +482 val_482 +482 val_482 +174 val_174 +174 val_174 +174 val_174 +413 val_413 +413 val_413 +413 val_413 +494 val_494 +494 val_494 +494 val_494 +207 val_207 +207 val_207 +207 val_207 +199 val_199 +199 val_199 +199 val_199 +466 val_466 +466 val_466 +466 val_466 +208 val_208 +208 val_208 +208 val_208 +174 val_174 +174 val_174 +174 val_174 +399 val_399 +399 val_399 +399 val_399 +396 val_396 +396 val_396 +396 val_396 +247 val_247 +247 val_247 +247 val_247 +417 val_417 +417 val_417 +417 val_417 +489 val_489 +489 val_489 +489 val_489 +162 val_162 +162 val_162 +162 val_162 +377 val_377 +377 val_377 +377 val_377 +397 val_397 +397 val_397 +397 val_397 +309 val_309 +309 val_309 +309 val_309 +365 val_365 +365 val_365 +365 val_365 +266 val_266 +266 val_266 +266 val_266 +439 val_439 +439 val_439 +439 val_439 +342 val_342 +342 val_342 +342 val_342 +367 val_367 +367 val_367 +367 val_367 +325 val_325 +325 val_325 +325 val_325 +167 val_167 +167 val_167 +167 val_167 +195 val_195 +195 val_195 +195 val_195 +475 val_475 +475 val_475 +475 val_475 +17 val_17 +17 val_17 +17 val_17 +113 val_113 +113 val_113 +113 val_113 +155 val_155 +155 val_155 +155 val_155 +203 val_203 +203 val_203 +203 val_203 +339 val_339 +339 val_339 +339 val_339 +0 val_0 +0 val_0 +0 val_0 +455 val_455 +455 val_455 +455 val_455 +128 val_128 +128 val_128 +128 val_128 +311 val_311 +311 val_311 +311 val_311 +316 val_316 +316 val_316 +316 val_316 +57 val_57 +57 val_57 +57 val_57 +302 val_302 +302 val_302 +302 val_302 +205 val_205 +205 val_205 +205 val_205 +149 val_149 +149 val_149 +149 val_149 +438 val_438 +438 val_438 +438 val_438 +345 val_345 +345 val_345 +345 val_345 +129 val_129 +129 val_129 +129 val_129 +170 val_170 +170 val_170 +170 val_170 +20 val_20 +20 val_20 +20 val_20 +489 val_489 +489 val_489 +489 val_489 +157 val_157 +157 val_157 +157 val_157 +378 val_378 +378 val_378 +378 val_378 +221 val_221 +221 val_221 +221 val_221 +92 val_92 +92 val_92 +92 val_92 +111 val_111 +111 val_111 +111 val_111 +47 val_47 +47 val_47 +47 val_47 +72 val_72 +72 val_72 +72 val_72 +4 val_4 +4 val_4 +4 val_4 +280 val_280 +280 val_280 +280 val_280 +35 val_35 +35 val_35 +35 val_35 +427 val_427 +427 val_427 +427 val_427 +277 val_277 +277 val_277 +277 val_277 +208 val_208 +208 val_208 +208 val_208 +356 val_356 +356 val_356 +356 val_356 +399 val_399 +399 val_399 +399 val_399 +169 val_169 +169 val_169 +169 val_169 +382 val_382 +382 val_382 +382 val_382 +498 val_498 +498 val_498 +498 val_498 +125 val_125 +125 val_125 +125 val_125 +386 val_386 +386 val_386 +386 val_386 +437 val_437 +437 val_437 +437 val_437 +469 val_469 +469 val_469 +469 val_469 +192 val_192 +192 val_192 +192 val_192 +286 val_286 +286 val_286 +286 val_286 +187 val_187 +187 val_187 +187 val_187 +176 val_176 +176 val_176 +176 val_176 +54 val_54 +54 val_54 +54 val_54 +459 val_459 +459 val_459 +459 val_459 +51 val_51 +51 val_51 +51 val_51 +138 val_138 +138 val_138 +138 val_138 +103 val_103 +103 val_103 +103 val_103 +239 val_239 +239 val_239 +239 val_239 +213 val_213 +213 val_213 +213 val_213 +216 val_216 +216 val_216 +216 val_216 +430 val_430 +430 val_430 +430 val_430 +278 val_278 +278 val_278 +278 val_278 +176 val_176 +176 val_176 +176 val_176 +289 val_289 +289 val_289 +289 val_289 +221 val_221 +221 val_221 +221 val_221 +65 val_65 +65 val_65 +65 val_65 +318 val_318 +318 val_318 +318 val_318 +332 val_332 +332 val_332 +332 val_332 +311 val_311 +311 val_311 +311 val_311 +275 val_275 +275 val_275 +275 val_275 +137 val_137 +137 val_137 +137 val_137 +241 val_241 +241 val_241 +241 val_241 +83 val_83 +83 val_83 +83 val_83 +333 val_333 +333 val_333 +333 val_333 +180 val_180 +180 val_180 +180 val_180 +284 val_284 +284 val_284 +284 val_284 +12 val_12 +12 val_12 +12 val_12 +230 val_230 +230 val_230 +230 val_230 +181 val_181 +181 val_181 +181 val_181 +67 val_67 +67 val_67 +67 val_67 +260 val_260 +260 val_260 +260 val_260 +404 val_404 +404 val_404 +404 val_404 +384 val_384 +384 val_384 +384 val_384 +489 val_489 +489 val_489 +489 val_489 +353 val_353 +353 val_353 +353 val_353 +373 val_373 +373 val_373 +373 val_373 +272 val_272 +272 val_272 +272 val_272 +138 val_138 +138 val_138 +138 val_138 +217 val_217 +217 val_217 +217 val_217 +84 val_84 +84 val_84 +84 val_84 +348 val_348 +348 val_348 +348 val_348 +466 val_466 +466 val_466 +466 val_466 +58 val_58 +58 val_58 +58 val_58 +8 val_8 +8 val_8 +8 val_8 +411 val_411 +411 val_411 +411 val_411 +230 val_230 +230 val_230 +230 val_230 +208 val_208 +208 val_208 +208 val_208 +348 val_348 +348 val_348 +348 val_348 +24 val_24 +24 val_24 +24 val_24 +463 val_463 +463 val_463 +463 val_463 +431 val_431 +431 val_431 +431 val_431 +179 val_179 +179 val_179 +179 val_179 +172 val_172 +172 val_172 +172 val_172 +42 val_42 +42 val_42 +42 val_42 +129 val_129 +129 val_129 +129 val_129 +158 val_158 +158 val_158 +158 val_158 +119 val_119 +119 val_119 +119 val_119 +496 val_496 +496 val_496 +496 val_496 +0 val_0 +0 val_0 +0 val_0 +322 val_322 +322 val_322 +322 val_322 +197 val_197 +197 val_197 +197 val_197 +468 val_468 +468 val_468 +468 val_468 +393 val_393 +393 val_393 +393 val_393 +454 val_454 +454 val_454 +454 val_454 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +199 val_199 +199 val_199 +199 val_199 +191 val_191 +191 val_191 +191 val_191 +418 val_418 +418 val_418 +418 val_418 +96 val_96 +96 val_96 +96 val_96 +26 val_26 +26 val_26 +26 val_26 +165 val_165 +165 val_165 +165 val_165 +327 val_327 +327 val_327 +327 val_327 +230 val_230 +230 val_230 +230 val_230 +205 val_205 +205 val_205 +205 val_205 +120 val_120 +120 val_120 +120 val_120 +131 val_131 +131 val_131 +131 val_131 +51 val_51 +51 val_51 +51 val_51 +404 val_404 +404 val_404 +404 val_404 +43 val_43 +43 val_43 +43 val_43 +436 val_436 +436 val_436 +436 val_436 +156 val_156 +156 val_156 +156 val_156 +469 val_469 +469 val_469 +469 val_469 +468 val_468 +468 val_468 +468 val_468 +308 val_308 +308 val_308 +308 val_308 +95 val_95 +95 val_95 +95 val_95 +196 val_196 +196 val_196 +196 val_196 +288 val_288 +288 val_288 +288 val_288 +481 val_481 +481 val_481 +481 val_481 +457 val_457 +457 val_457 +457 val_457 +98 val_98 +98 val_98 +98 val_98 +282 val_282 +282 val_282 +282 val_282 +197 val_197 +197 val_197 +197 val_197 +187 val_187 +187 val_187 +187 val_187 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +409 val_409 +409 val_409 +409 val_409 +470 val_470 +470 val_470 +470 val_470 +137 val_137 +137 val_137 +137 val_137 +369 val_369 +369 val_369 +369 val_369 +316 val_316 +316 val_316 +316 val_316 +169 val_169 +169 val_169 +169 val_169 +413 val_413 +413 val_413 +413 val_413 +85 val_85 +85 val_85 +85 val_85 +77 val_77 +77 val_77 +77 val_77 +0 val_0 +0 val_0 +0 val_0 +490 val_490 +490 val_490 +490 val_490 +87 val_87 +87 val_87 +87 val_87 +364 val_364 +364 val_364 +364 val_364 +179 val_179 +179 val_179 +179 val_179 +118 val_118 +118 val_118 +118 val_118 +134 val_134 +134 val_134 +134 val_134 +395 val_395 +395 val_395 +395 val_395 +282 val_282 +282 val_282 +282 val_282 +138 val_138 +138 val_138 +138 val_138 +238 val_238 +238 val_238 +238 val_238 +419 val_419 +419 val_419 +419 val_419 +15 val_15 +15 val_15 +15 val_15 +118 val_118 +118 val_118 +118 val_118 +72 val_72 +72 val_72 +72 val_72 +90 val_90 +90 val_90 +90 val_90 +307 val_307 +307 val_307 +307 val_307 +19 val_19 +19 val_19 +19 val_19 +435 val_435 +435 val_435 +435 val_435 +10 val_10 +10 val_10 +10 val_10 +277 val_277 +277 val_277 +277 val_277 +273 val_273 +273 val_273 +273 val_273 +306 val_306 +306 val_306 +306 val_306 +224 val_224 +224 val_224 +224 val_224 +309 val_309 +309 val_309 +309 val_309 +389 val_389 +389 val_389 +389 val_389 +327 val_327 +327 val_327 +327 val_327 +242 val_242 +242 val_242 +242 val_242 +369 val_369 +369 val_369 +369 val_369 +392 val_392 +392 val_392 +392 val_392 +272 val_272 +272 val_272 +272 val_272 +331 val_331 +331 val_331 +331 val_331 +401 val_401 +401 val_401 +401 val_401 +242 val_242 +242 val_242 +242 val_242 +452 val_452 +452 val_452 +452 val_452 +177 val_177 +177 val_177 +177 val_177 +226 val_226 +226 val_226 +226 val_226 +5 val_5 +5 val_5 +5 val_5 +497 val_497 +497 val_497 +497 val_497 +402 val_402 +402 val_402 +402 val_402 +396 val_396 +396 val_396 +396 val_396 +317 val_317 +317 val_317 +317 val_317 +395 val_395 +395 val_395 +395 val_395 +58 val_58 +58 val_58 +58 val_58 +35 val_35 +35 val_35 +35 val_35 +336 val_336 +336 val_336 +336 val_336 +95 val_95 +95 val_95 +95 val_95 +11 val_11 +11 val_11 +11 val_11 +168 val_168 +168 val_168 +168 val_168 +34 val_34 +34 val_34 +34 val_34 +229 val_229 +229 val_229 +229 val_229 +233 val_233 +233 val_233 +233 val_233 +143 val_143 +143 val_143 +143 val_143 +472 val_472 +472 val_472 +472 val_472 +322 val_322 +322 val_322 +322 val_322 +498 val_498 +498 val_498 +498 val_498 +160 val_160 +160 val_160 +160 val_160 +195 val_195 +195 val_195 +195 val_195 +42 val_42 +42 val_42 +42 val_42 +321 val_321 +321 val_321 +321 val_321 +430 val_430 +430 val_430 +430 val_430 +119 val_119 +119 val_119 +119 val_119 +489 val_489 +489 val_489 +489 val_489 +458 val_458 +458 val_458 +458 val_458 +78 val_78 +78 val_78 +78 val_78 +76 val_76 +76 val_76 +76 val_76 +41 val_41 +41 val_41 +41 val_41 +223 val_223 +223 val_223 +223 val_223 +492 val_492 +492 val_492 +492 val_492 +149 val_149 +149 val_149 +149 val_149 +449 val_449 +449 val_449 +449 val_449 +218 val_218 +218 val_218 +218 val_218 +228 val_228 +228 val_228 +228 val_228 +138 val_138 +138 val_138 +138 val_138 +453 val_453 +453 val_453 +453 val_453 +30 val_30 +30 val_30 +30 val_30 +209 val_209 +209 val_209 +209 val_209 +64 val_64 +64 val_64 +64 val_64 +468 val_468 +468 val_468 +468 val_468 +76 val_76 +76 val_76 +76 val_76 +74 val_74 +74 val_74 +74 val_74 +342 val_342 +342 val_342 +342 val_342 +69 val_69 +69 val_69 +69 val_69 +230 val_230 +230 val_230 +230 val_230 +33 val_33 +33 val_33 +33 val_33 +368 val_368 +368 val_368 +368 val_368 +103 val_103 +103 val_103 +103 val_103 +296 val_296 +296 val_296 +296 val_296 +113 val_113 +113 val_113 +113 val_113 +216 val_216 +216 val_216 +216 val_216 +367 val_367 +367 val_367 +367 val_367 +344 val_344 +344 val_344 +344 val_344 +167 val_167 +167 val_167 +167 val_167 +274 val_274 +274 val_274 +274 val_274 +219 val_219 +219 val_219 +219 val_219 +239 val_239 +239 val_239 +239 val_239 +485 val_485 +485 val_485 +485 val_485 +116 val_116 +116 val_116 +116 val_116 +223 val_223 +223 val_223 +223 val_223 +256 val_256 +256 val_256 +256 val_256 +263 val_263 +263 val_263 +263 val_263 +70 val_70 +70 val_70 +70 val_70 +487 val_487 +487 val_487 +487 val_487 +480 val_480 +480 val_480 +480 val_480 +401 val_401 +401 val_401 +401 val_401 +288 val_288 +288 val_288 +288 val_288 +191 val_191 +191 val_191 +191 val_191 +5 val_5 +5 val_5 +5 val_5 +244 val_244 +244 val_244 +244 val_244 +438 val_438 +438 val_438 +438 val_438 +128 val_128 +128 val_128 +128 val_128 +467 val_467 +467 val_467 +467 val_467 +432 val_432 +432 val_432 +432 val_432 +202 val_202 +202 val_202 +202 val_202 +316 val_316 +316 val_316 +316 val_316 +229 val_229 +229 val_229 +229 val_229 +469 val_469 +469 val_469 +469 val_469 +463 val_463 +463 val_463 +463 val_463 +280 val_280 +280 val_280 +280 val_280 +2 val_2 +2 val_2 +2 val_2 +35 val_35 +35 val_35 +35 val_35 +283 val_283 +283 val_283 +283 val_283 +331 val_331 +331 val_331 +331 val_331 +235 val_235 +235 val_235 +235 val_235 +80 val_80 +80 val_80 +80 val_80 +44 val_44 +44 val_44 +44 val_44 +193 val_193 +193 val_193 +193 val_193 +321 val_321 +321 val_321 +321 val_321 +335 val_335 +335 val_335 +335 val_335 +104 val_104 +104 val_104 +104 val_104 +466 val_466 +466 val_466 +466 val_466 +366 val_366 +366 val_366 +366 val_366 +175 val_175 +175 val_175 +175 val_175 +403 val_403 +403 val_403 +403 val_403 +483 val_483 +483 val_483 +483 val_483 +53 val_53 +53 val_53 +53 val_53 +105 val_105 +105 val_105 +105 val_105 +257 val_257 +257 val_257 +257 val_257 +406 val_406 +406 val_406 +406 val_406 +409 val_409 +409 val_409 +409 val_409 +190 val_190 +190 val_190 +190 val_190 +406 val_406 +406 val_406 +406 val_406 +401 val_401 +401 val_401 +401 val_401 +114 val_114 +114 val_114 +114 val_114 +258 val_258 +258 val_258 +258 val_258 +90 val_90 +90 val_90 +90 val_90 +203 val_203 +203 val_203 +203 val_203 +262 val_262 +262 val_262 +262 val_262 +348 val_348 +348 val_348 +348 val_348 +424 val_424 +424 val_424 +424 val_424 +12 val_12 +12 val_12 +12 val_12 +396 val_396 +396 val_396 +396 val_396 +201 val_201 +201 val_201 +201 val_201 +217 val_217 +217 val_217 +217 val_217 +164 val_164 +164 val_164 +164 val_164 +431 val_431 +431 val_431 +431 val_431 +454 val_454 +454 val_454 +454 val_454 +478 val_478 +478 val_478 +478 val_478 +298 val_298 +298 val_298 +298 val_298 +125 val_125 +125 val_125 +125 val_125 +431 val_431 +431 val_431 +431 val_431 +164 val_164 +164 val_164 +164 val_164 +424 val_424 +424 val_424 +424 val_424 +187 val_187 +187 val_187 +187 val_187 +382 val_382 +382 val_382 +382 val_382 +5 val_5 +5 val_5 +5 val_5 +70 val_70 +70 val_70 +70 val_70 +397 val_397 +397 val_397 +397 val_397 +480 val_480 +480 val_480 +480 val_480 +291 val_291 +291 val_291 +291 val_291 +24 val_24 +24 val_24 +24 val_24 +351 val_351 +351 val_351 +351 val_351 +255 val_255 +255 val_255 +255 val_255 +104 val_104 +104 val_104 +104 val_104 +70 val_70 +70 val_70 +70 val_70 +163 val_163 +163 val_163 +163 val_163 +438 val_438 +438 val_438 +438 val_438 +119 val_119 +119 val_119 +119 val_119 +414 val_414 +414 val_414 +414 val_414 +200 val_200 +200 val_200 +200 val_200 +491 val_491 +491 val_491 +491 val_491 +237 val_237 +237 val_237 +237 val_237 +439 val_439 +439 val_439 +439 val_439 +360 val_360 +360 val_360 +360 val_360 +248 val_248 +248 val_248 +248 val_248 +479 val_479 +479 val_479 +479 val_479 +305 val_305 +305 val_305 +305 val_305 +417 val_417 +417 val_417 +417 val_417 +199 val_199 +199 val_199 +199 val_199 +444 val_444 +444 val_444 +444 val_444 +120 val_120 +120 val_120 +120 val_120 +429 val_429 +429 val_429 +429 val_429 +169 val_169 +169 val_169 +169 val_169 +443 val_443 +443 val_443 +443 val_443 +323 val_323 +323 val_323 +323 val_323 +325 val_325 +325 val_325 +325 val_325 +277 val_277 +277 val_277 +277 val_277 +230 val_230 +230 val_230 +230 val_230 +478 val_478 +478 val_478 +478 val_478 +178 val_178 +178 val_178 +178 val_178 +468 val_468 +468 val_468 +468 val_468 +310 val_310 +310 val_310 +310 val_310 +317 val_317 +317 val_317 +317 val_317 +333 val_333 +333 val_333 +333 val_333 +493 val_493 +493 val_493 +493 val_493 +460 val_460 +460 val_460 +460 val_460 +207 val_207 +207 val_207 +207 val_207 +249 val_249 +249 val_249 +249 val_249 +265 val_265 +265 val_265 +265 val_265 +480 val_480 +480 val_480 +480 val_480 +83 val_83 +83 val_83 +83 val_83 +136 val_136 +136 val_136 +136 val_136 +353 val_353 +353 val_353 +353 val_353 +172 val_172 +172 val_172 +172 val_172 +214 val_214 +214 val_214 +214 val_214 +462 val_462 +462 val_462 +462 val_462 +233 val_233 +233 val_233 +233 val_233 +406 val_406 +406 val_406 +406 val_406 +133 val_133 +133 val_133 +133 val_133 +175 val_175 +175 val_175 +175 val_175 +189 val_189 +189 val_189 +189 val_189 +454 val_454 +454 val_454 +454 val_454 +375 val_375 +375 val_375 +375 val_375 +401 val_401 +401 val_401 +401 val_401 +421 val_421 +421 val_421 +421 val_421 +407 val_407 +407 val_407 +407 val_407 +384 val_384 +384 val_384 +384 val_384 +256 val_256 +256 val_256 +256 val_256 +26 val_26 +26 val_26 +26 val_26 +134 val_134 +134 val_134 +134 val_134 +67 val_67 +67 val_67 +67 val_67 +384 val_384 +384 val_384 +384 val_384 +379 val_379 +379 val_379 +379 val_379 +18 val_18 +18 val_18 +18 val_18 +462 val_462 +462 val_462 +462 val_462 +492 val_492 +492 val_492 +492 val_492 +100 val_100 +100 val_100 +100 val_100 +298 val_298 +298 val_298 +298 val_298 +9 val_9 +9 val_9 +9 val_9 +341 val_341 +341 val_341 +341 val_341 +498 val_498 +498 val_498 +498 val_498 +146 val_146 +146 val_146 +146 val_146 +458 val_458 +458 val_458 +458 val_458 +362 val_362 +362 val_362 +362 val_362 +186 val_186 +186 val_186 +186 val_186 +285 val_285 +285 val_285 +285 val_285 +348 val_348 +348 val_348 +348 val_348 +167 val_167 +167 val_167 +167 val_167 +18 val_18 +18 val_18 +18 val_18 +273 val_273 +273 val_273 +273 val_273 +183 val_183 +183 val_183 +183 val_183 +281 val_281 +281 val_281 +281 val_281 +344 val_344 +344 val_344 +344 val_344 +97 val_97 +97 val_97 +97 val_97 +469 val_469 +469 val_469 +469 val_469 +315 val_315 +315 val_315 +315 val_315 +84 val_84 +84 val_84 +84 val_84 +28 val_28 +28 val_28 +28 val_28 +37 val_37 +37 val_37 +37 val_37 +448 val_448 +448 val_448 +448 val_448 +152 val_152 +152 val_152 +152 val_152 +348 val_348 +348 val_348 +348 val_348 +307 val_307 +307 val_307 +307 val_307 +194 val_194 +194 val_194 +194 val_194 +414 val_414 +414 val_414 +414 val_414 +477 val_477 +477 val_477 +477 val_477 +222 val_222 +222 val_222 +222 val_222 +126 val_126 +126 val_126 +126 val_126 +90 val_90 +90 val_90 +90 val_90 +169 val_169 +169 val_169 +169 val_169 +403 val_403 +403 val_403 +403 val_403 +400 val_400 +400 val_400 +400 val_400 +200 val_200 +200 val_200 +200 val_200 +97 val_97 +97 val_97 +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 b/src/test/hive/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 b/src/test/hive/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 new file mode 100644 index 0000000000000..37021f4a27201 --- /dev/null +++ b/src/test/hive/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 @@ -0,0 +1 @@ +1500 \ No newline at end of file diff --git a/src/test/hive/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 b/src/test/hive/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c b/src/test/hive/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 b/src/test/hive/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 b/src/test/hive/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c b/src/test/hive/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d b/src/test/hive/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 b/src/test/hive/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 b/src/test/hive/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 b/src/test/hive/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 b/src/test/hive/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 new file mode 100644 index 0000000000000..71bb7c3ad9311 --- /dev/null +++ b/src/test/hive/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 @@ -0,0 +1,20 @@ +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 1 val_0 +0 2 val_0 +0 2 val_0 diff --git a/src/test/hive/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a b/src/test/hive/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a new file mode 100644 index 0000000000000..468f774ef5146 --- /dev/null +++ b/src/test/hive/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a @@ -0,0 +1,10 @@ +val_238 +NULL +val_86 +NULL +val_311 +NULL +val_27 +NULL +val_165 +NULL diff --git a/src/test/hive/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b b/src/test/hive/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed b/src/test/hive/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed new file mode 100644 index 0000000000000..ed8993a9cbd0c --- /dev/null +++ b/src/test/hive/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed @@ -0,0 +1,336 @@ +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 2008-04-08 11 +0 val_0 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 +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 11 +2 val_2 2008-04-08 11 +2 val_2 2008-04-08 12 +2 val_2 2008-04-08 12 +4 val_4 2008-04-08 11 +4 val_4 2008-04-08 11 +4 val_4 2008-04-08 12 +4 val_4 2008-04-08 12 +5 val_5 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 +5 val_5 2008-04-08 11 +5 val_5 2008-04-08 11 +5 val_5 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 +5 val_5 2008-04-08 12 +5 val_5 2008-04-08 12 +8 val_8 2008-04-08 11 +8 val_8 2008-04-08 11 +8 val_8 2008-04-08 12 +8 val_8 2008-04-08 12 +9 val_9 2008-04-08 11 +9 val_9 2008-04-08 11 +9 val_9 2008-04-08 12 +9 val_9 2008-04-08 12 +10 val_10 2008-04-08 11 +10 val_10 2008-04-08 11 +10 val_10 2008-04-08 12 +10 val_10 2008-04-08 12 +11 val_11 2008-04-08 11 +11 val_11 2008-04-08 11 +11 val_11 2008-04-08 12 +11 val_11 2008-04-08 12 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 11 +12 val_12 2008-04-08 12 +12 val_12 2008-04-08 12 +12 val_12 2008-04-08 12 +12 val_12 2008-04-08 12 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 11 +15 val_15 2008-04-08 12 +15 val_15 2008-04-08 12 +15 val_15 2008-04-08 12 +15 val_15 2008-04-08 12 +17 val_17 2008-04-08 11 +17 val_17 2008-04-08 11 +17 val_17 2008-04-08 12 +17 val_17 2008-04-08 12 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 11 +18 val_18 2008-04-08 12 +18 val_18 2008-04-08 12 +18 val_18 2008-04-08 12 +18 val_18 2008-04-08 12 +19 val_19 2008-04-08 11 +19 val_19 2008-04-08 11 +19 val_19 2008-04-08 12 +19 val_19 2008-04-08 12 +20 val_20 2008-04-08 11 +20 val_20 2008-04-08 11 +20 val_20 2008-04-08 12 +20 val_20 2008-04-08 12 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 11 +24 val_24 2008-04-08 12 +24 val_24 2008-04-08 12 +24 val_24 2008-04-08 12 +24 val_24 2008-04-08 12 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 11 +26 val_26 2008-04-08 12 +26 val_26 2008-04-08 12 +26 val_26 2008-04-08 12 +26 val_26 2008-04-08 12 +27 val_27 2008-04-08 11 +27 val_27 2008-04-08 11 +27 val_27 2008-04-08 12 +27 val_27 2008-04-08 12 +28 val_28 2008-04-08 11 +28 val_28 2008-04-08 11 +28 val_28 2008-04-08 12 +28 val_28 2008-04-08 12 +30 val_30 2008-04-08 11 +30 val_30 2008-04-08 11 +30 val_30 2008-04-08 12 +30 val_30 2008-04-08 12 +33 val_33 2008-04-08 11 +33 val_33 2008-04-08 11 +33 val_33 2008-04-08 12 +33 val_33 2008-04-08 12 +34 val_34 2008-04-08 11 +34 val_34 2008-04-08 11 +34 val_34 2008-04-08 12 +34 val_34 2008-04-08 12 +35 val_35 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 +35 val_35 2008-04-08 11 +35 val_35 2008-04-08 11 +35 val_35 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 +35 val_35 2008-04-08 12 +35 val_35 2008-04-08 12 +37 val_37 2008-04-08 11 +37 val_37 2008-04-08 11 +37 val_37 2008-04-08 11 +37 val_37 2008-04-08 11 +37 val_37 2008-04-08 12 +37 val_37 2008-04-08 12 +37 val_37 2008-04-08 12 +37 val_37 2008-04-08 12 +41 val_41 2008-04-08 11 +41 val_41 2008-04-08 11 +41 val_41 2008-04-08 12 +41 val_41 2008-04-08 12 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 11 +42 val_42 2008-04-08 12 +42 val_42 2008-04-08 12 +42 val_42 2008-04-08 12 +42 val_42 2008-04-08 12 +43 val_43 2008-04-08 11 +43 val_43 2008-04-08 11 +43 val_43 2008-04-08 12 +43 val_43 2008-04-08 12 +44 val_44 2008-04-08 11 +44 val_44 2008-04-08 11 +44 val_44 2008-04-08 12 +44 val_44 2008-04-08 12 +47 val_47 2008-04-08 11 +47 val_47 2008-04-08 11 +47 val_47 2008-04-08 12 +47 val_47 2008-04-08 12 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 11 +51 val_51 2008-04-08 12 +51 val_51 2008-04-08 12 +51 val_51 2008-04-08 12 +51 val_51 2008-04-08 12 +53 val_53 2008-04-08 11 +53 val_53 2008-04-08 11 +53 val_53 2008-04-08 12 +53 val_53 2008-04-08 12 +54 val_54 2008-04-08 11 +54 val_54 2008-04-08 11 +54 val_54 2008-04-08 12 +54 val_54 2008-04-08 12 +57 val_57 2008-04-08 11 +57 val_57 2008-04-08 11 +57 val_57 2008-04-08 12 +57 val_57 2008-04-08 12 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 11 +58 val_58 2008-04-08 12 +58 val_58 2008-04-08 12 +58 val_58 2008-04-08 12 +58 val_58 2008-04-08 12 +64 val_64 2008-04-08 11 +64 val_64 2008-04-08 11 +64 val_64 2008-04-08 12 +64 val_64 2008-04-08 12 +65 val_65 2008-04-08 11 +65 val_65 2008-04-08 11 +65 val_65 2008-04-08 12 +65 val_65 2008-04-08 12 +66 val_66 2008-04-08 11 +66 val_66 2008-04-08 11 +66 val_66 2008-04-08 12 +66 val_66 2008-04-08 12 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 11 +67 val_67 2008-04-08 12 +67 val_67 2008-04-08 12 +67 val_67 2008-04-08 12 +67 val_67 2008-04-08 12 +69 val_69 2008-04-08 11 +69 val_69 2008-04-08 11 +69 val_69 2008-04-08 12 +69 val_69 2008-04-08 12 +70 val_70 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 +70 val_70 2008-04-08 11 +70 val_70 2008-04-08 11 +70 val_70 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 +70 val_70 2008-04-08 12 +70 val_70 2008-04-08 12 +72 val_72 2008-04-08 11 +72 val_72 2008-04-08 11 +72 val_72 2008-04-08 11 +72 val_72 2008-04-08 11 +72 val_72 2008-04-08 12 +72 val_72 2008-04-08 12 +72 val_72 2008-04-08 12 +72 val_72 2008-04-08 12 +74 val_74 2008-04-08 11 +74 val_74 2008-04-08 11 +74 val_74 2008-04-08 12 +74 val_74 2008-04-08 12 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 11 +76 val_76 2008-04-08 12 +76 val_76 2008-04-08 12 +76 val_76 2008-04-08 12 +76 val_76 2008-04-08 12 +77 val_77 2008-04-08 11 +77 val_77 2008-04-08 11 +77 val_77 2008-04-08 12 +77 val_77 2008-04-08 12 +78 val_78 2008-04-08 11 +78 val_78 2008-04-08 11 +78 val_78 2008-04-08 12 +78 val_78 2008-04-08 12 +80 val_80 2008-04-08 11 +80 val_80 2008-04-08 11 +80 val_80 2008-04-08 12 +80 val_80 2008-04-08 12 +82 val_82 2008-04-08 11 +82 val_82 2008-04-08 11 +82 val_82 2008-04-08 12 +82 val_82 2008-04-08 12 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 11 +83 val_83 2008-04-08 12 +83 val_83 2008-04-08 12 +83 val_83 2008-04-08 12 +83 val_83 2008-04-08 12 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 11 +84 val_84 2008-04-08 12 +84 val_84 2008-04-08 12 +84 val_84 2008-04-08 12 +84 val_84 2008-04-08 12 +85 val_85 2008-04-08 11 +85 val_85 2008-04-08 11 +85 val_85 2008-04-08 12 +85 val_85 2008-04-08 12 +86 val_86 2008-04-08 11 +86 val_86 2008-04-08 11 +86 val_86 2008-04-08 12 +86 val_86 2008-04-08 12 +87 val_87 2008-04-08 11 +87 val_87 2008-04-08 11 +87 val_87 2008-04-08 12 +87 val_87 2008-04-08 12 +90 val_90 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 +90 val_90 2008-04-08 11 +90 val_90 2008-04-08 11 +90 val_90 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 +90 val_90 2008-04-08 12 +90 val_90 2008-04-08 12 +92 val_92 2008-04-08 11 +92 val_92 2008-04-08 11 +92 val_92 2008-04-08 12 +92 val_92 2008-04-08 12 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 11 +95 val_95 2008-04-08 12 +95 val_95 2008-04-08 12 +95 val_95 2008-04-08 12 +95 val_95 2008-04-08 12 +96 val_96 2008-04-08 11 +96 val_96 2008-04-08 11 +96 val_96 2008-04-08 12 +96 val_96 2008-04-08 12 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 11 +97 val_97 2008-04-08 12 +97 val_97 2008-04-08 12 +97 val_97 2008-04-08 12 +97 val_97 2008-04-08 12 +98 val_98 2008-04-08 11 +98 val_98 2008-04-08 11 +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 +98 val_98 2008-04-08 12 +98 val_98 2008-04-08 12 diff --git a/src/test/hive/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/src/test/hive/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..c203753d8e42d --- /dev/null +++ b/src/test/hive/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 b/src/test/hive/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a b/src/test/hive/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 b/src/test/hive/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 b/src/test/hive/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 b/src/test/hive/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..94a711cdab117 --- /dev/null +++ b/src/test/hive/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc b/src/test/hive/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 b/src/test/hive/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/src/test/hive/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 b/src/test/hive/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a b/src/test/hive/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..d9cfa9e952c3a --- /dev/null +++ b/src/test/hive/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# col_name data_type comment + +key string from deserializer +values bigint from deserializer + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Fri Jan 03 19:05:51 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/outputtbl1 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1388804758 + +# 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/src/test/hive/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e b/src/test/hive/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e new file mode 100644 index 0000000000000..2817d74afee13 --- /dev/null +++ b/src/test/hive/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e @@ -0,0 +1,18 @@ +1 1 +1 2 +1 3 +2 1 +2 2 +2 3 +3 1 +3 2 +3 3 +7 1 +7 2 +7 3 +8 1 +8 1 +8 2 +8 2 +8 3 +8 3 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc b/src/test/hive/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 b/src/test/hive/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/src/test/hive/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c b/src/test/hive/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 b/src/test/hive/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..57d5146698ee7 --- /dev/null +++ b/src/test/hive/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc b/src/test/hive/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 b/src/test/hive/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/src/test/hive/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 b/src/test/hive/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af b/src/test/hive/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..c914024c14c8e --- /dev/null +++ b/src/test/hive/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc b/src/test/hive/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb b/src/test/hive/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/src/test/hive/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c b/src/test/hive/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 b/src/test/hive/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..5a55428afbe1a --- /dev/null +++ b/src/test/hive/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 b/src/test/hive/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc b/src/test/hive/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d b/src/test/hive/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/src/test/hive/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 b/src/test/hive/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 b/src/test/hive/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..a2441e330ffc6 --- /dev/null +++ b/src/test/hive/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 b/src/test/hive/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 new file mode 100644 index 0000000000000..a58273a1b054e --- /dev/null +++ b/src/test/hive/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 @@ -0,0 +1,2 @@ +ds=1 +ds=2 diff --git a/src/test/hive/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d b/src/test/hive/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef b/src/test/hive/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef b/src/test/hive/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 b/src/test/hive/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 b/src/test/hive/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..bec8202dcbcad --- /dev/null +++ b/src/test/hive/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 b/src/test/hive/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 new file mode 100644 index 0000000000000..a58273a1b054e --- /dev/null +++ b/src/test/hive/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 @@ -0,0 +1,2 @@ +ds=1 +ds=2 diff --git a/src/test/hive/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc b/src/test/hive/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a b/src/test/hive/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 b/src/test/hive/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d b/src/test/hive/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..d0b37e7625a64 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 b/src/test/hive/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 new file mode 100644 index 0000000000000..c21f4017362c1 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 @@ -0,0 +1,2 @@ +ds=1 +ds=2 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 b/src/test/hive/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 new file mode 100644 index 0000000000000..1ab5c482feca9 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 @@ -0,0 +1,6 @@ +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/src/test/hive/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a b/src/test/hive/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a new file mode 100644 index 0000000000000..94ec062e2f4f3 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a @@ -0,0 +1,6 @@ +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/src/test/hive/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b b/src/test/hive/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef b/src/test/hive/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad b/src/test/hive/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 b/src/test/hive/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..971e095df7be8 --- /dev/null +++ b/src/test/hive/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,32 @@ +# 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/src/test/hive/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 b/src/test/hive/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 new file mode 100644 index 0000000000000..5690d94a65857 --- /dev/null +++ b/src/test/hive/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 @@ -0,0 +1,6 @@ +ds=11 +ds=12 +ds=13 +ds=17 +ds=18 +ds=28 diff --git a/src/test/hive/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe b/src/test/hive/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 b/src/test/hive/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 b/src/test/hive/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 b/src/test/hive/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b b/src/test/hive/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..0d0d3a30af36c --- /dev/null +++ b/src/test/hive/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 b/src/test/hive/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 new file mode 100644 index 0000000000000..719e4a7bd4dbb --- /dev/null +++ b/src/test/hive/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 b/src/test/hive/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 b/src/test/hive/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 b/src/test/hive/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 new file mode 100644 index 0000000000000..361f2ab78eae3 --- /dev/null +++ b/src/test/hive/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 @@ -0,0 +1,2 @@ +7 1 +7 1 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e b/src/test/hive/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 b/src/test/hive/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 b/src/test/hive/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a b/src/test/hive/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 b/src/test/hive/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 b/src/test/hive/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..d3c821fa22182 --- /dev/null +++ b/src/test/hive/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e b/src/test/hive/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a b/src/test/hive/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f b/src/test/hive/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 b/src/test/hive/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..978049640929c --- /dev/null +++ b/src/test/hive/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e b/src/test/hive/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f b/src/test/hive/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa b/src/test/hive/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 b/src/test/hive/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..ce7e0438f9eff --- /dev/null +++ b/src/test/hive/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,26 @@ +# 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/src/test/hive/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f b/src/test/hive/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 b/src/test/hive/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 b/src/test/hive/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 b/src/test/hive/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..c261ce48bdfb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,28 @@ +# 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/src/test/hive/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 b/src/test/hive/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 new file mode 100644 index 0000000000000..d36ac6b834792 --- /dev/null +++ b/src/test/hive/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd b/src/test/hive/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af b/src/test/hive/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 b/src/test/hive/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 b/src/test/hive/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb b/src/test/hive/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 b/src/test/hive/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..50297abd4e9a6 --- /dev/null +++ b/src/test/hive/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 b/src/test/hive/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a b/src/test/hive/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 b/src/test/hive/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 b/src/test/hive/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..f3c452ebe9284 --- /dev/null +++ b/src/test/hive/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 b/src/test/hive/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e b/src/test/hive/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f b/src/test/hive/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 b/src/test/hive/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..d46af80f0d1ae --- /dev/null +++ b/src/test/hive/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# col_name data_type comment + +key string None +values bigint None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Fri Jan 03 19:09:55 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/outputtbl1 +Table Type: MANAGED_TABLE +Table Parameters: + transient_lastDdlTime 1388805002 + +# 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/src/test/hive/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e b/src/test/hive/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e new file mode 100644 index 0000000000000..2817d74afee13 --- /dev/null +++ b/src/test/hive/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e @@ -0,0 +1,18 @@ +1 1 +1 2 +1 3 +2 1 +2 2 +2 3 +3 1 +3 2 +3 3 +7 1 +7 2 +7 3 +8 1 +8 1 +8 2 +8 2 +8 3 +8 3 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 b/src/test/hive/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a b/src/test/hive/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 b/src/test/hive/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 b/src/test/hive/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 b/src/test/hive/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..67faa5b13d984 --- /dev/null +++ b/src/test/hive/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc b/src/test/hive/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead b/src/test/hive/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a b/src/test/hive/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f b/src/test/hive/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 b/src/test/hive/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..19aeb0b602859 --- /dev/null +++ b/src/test/hive/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc b/src/test/hive/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 b/src/test/hive/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a b/src/test/hive/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 b/src/test/hive/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 b/src/test/hive/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e b/src/test/hive/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e new file mode 100644 index 0000000000000..8e7ee8a2b47bb --- /dev/null +++ b/src/test/hive/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e @@ -0,0 +1,10 @@ +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/src/test/hive/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 b/src/test/hive/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 new file mode 100644 index 0000000000000..8e7ee8a2b47bb --- /dev/null +++ b/src/test/hive/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 @@ -0,0 +1,10 @@ +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/src/test/hive/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 b/src/test/hive/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a b/src/test/hive/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf b/src/test/hive/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/src/test/hive/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..1c63b3851aad2 --- /dev/null +++ b/src/test/hive/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca b/src/test/hive/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/src/test/hive/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 b/src/test/hive/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 b/src/test/hive/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..f718ab2990cdd --- /dev/null +++ b/src/test/hive/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 b/src/test/hive/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab b/src/test/hive/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee b/src/test/hive/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/src/test/hive/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f b/src/test/hive/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 b/src/test/hive/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c b/src/test/hive/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 b/src/test/hive/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c b/src/test/hive/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 b/src/test/hive/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..65a91a6027545 --- /dev/null +++ b/src/test/hive/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 @@ -0,0 +1,27 @@ +# 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/src/test/hive/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a b/src/test/hive/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 b/src/test/hive/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a b/src/test/hive/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 b/src/test/hive/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc b/src/test/hive/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 b/src/test/hive/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 b/src/test/hive/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/src/test/hive/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e b/src/test/hive/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 b/src/test/hive/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 new file mode 100644 index 0000000000000..42d07ed10bad4 --- /dev/null +++ b/src/test/hive/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 @@ -0,0 +1,500 @@ +0 +0 +0 +10 +100 +100 +103 +103 +104 +104 +105 +11 +111 +113 +113 +114 +116 +118 +118 +119 +119 +119 +12 +12 +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 +15 +15 +150 +152 +152 +153 +155 +156 +157 +158 +160 +162 +163 +164 +164 +165 +165 +166 +167 +167 +167 +168 +169 +169 +169 +169 +17 +170 +172 +172 +174 +174 +175 +175 +176 +176 +177 +178 +179 +179 +18 +18 +180 +181 +183 +186 +187 +187 +187 +189 +19 +190 +191 +191 +192 +193 +193 +193 +194 +195 +195 +196 +197 +197 +199 +199 +199 +2 +20 +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 +24 +24 +241 +242 +242 +244 +247 +248 +249 +252 +255 +255 +256 +256 +257 +258 +26 +26 +260 +262 +263 +265 +265 +266 +27 +272 +272 +273 +273 +273 +274 +275 +277 +277 +277 +277 +278 +278 +28 +280 +280 +281 +281 +282 +282 +283 +284 +285 +286 +287 +288 +288 +289 +291 +292 +296 +298 +298 +298 +30 +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 +33 +331 +331 +332 +333 +333 +335 +336 +338 +339 +34 +341 +342 +342 +344 +344 +345 +348 +348 +348 +348 +348 +35 +35 +35 +351 +353 +353 +356 +360 +362 +364 +365 +366 +367 +367 +368 +369 +369 +369 +37 +37 +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 +4 +400 +401 +401 +401 +401 +401 +402 +403 +403 +403 +404 +404 +406 +406 +406 +406 +407 +409 +409 +409 +41 +411 +413 +413 +414 +414 +417 +417 +417 +418 +419 +42 +42 +421 +424 +424 +427 +429 +429 +43 +430 +430 +430 +431 +431 +431 +432 +435 +436 +437 +438 +438 +438 +439 +439 +44 +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 +47 +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 +5 +5 +5 +51 +51 +53 +54 +57 +58 +58 +64 +65 +66 +67 +67 +69 +70 +70 +70 +72 +72 +74 +76 +76 +77 +78 +8 +80 +82 +83 +83 +84 +84 +85 +86 +87 +9 +90 +90 +90 +92 +95 +95 +96 +97 +97 +98 +98 \ No newline at end of file diff --git a/src/test/hive/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 b/src/test/hive/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 new file mode 100644 index 0000000000000..2f23db6a2ff4a --- /dev/null +++ b/src/test/hive/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 @@ -0,0 +1,1000 @@ +0 +0 +0 +0 +0 +0 +10 +10 +100 +100 +100 +100 +103 +103 +103 +103 +104 +104 +104 +104 +105 +105 +11 +11 +111 +111 +113 +113 +113 +113 +114 +114 +116 +116 +118 +118 +118 +118 +119 +119 +119 +119 +119 +119 +12 +12 +12 +12 +120 +120 +120 +120 +125 +125 +125 +125 +126 +126 +128 +128 +128 +128 +128 +128 +129 +129 +129 +129 +131 +131 +133 +133 +134 +134 +134 +134 +136 +136 +137 +137 +137 +137 +138 +138 +138 +138 +138 +138 +138 +138 +143 +143 +145 +145 +146 +146 +146 +146 +149 +149 +149 +149 +15 +15 +15 +15 +150 +150 +152 +152 +152 +152 +153 +153 +155 +155 +156 +156 +157 +157 +158 +158 +160 +160 +162 +162 +163 +163 +164 +164 +164 +164 +165 +165 +165 +165 +166 +166 +167 +167 +167 +167 +167 +167 +168 +168 +169 +169 +169 +169 +169 +169 +169 +169 +17 +17 +170 +170 +172 +172 +172 +172 +174 +174 +174 +174 +175 +175 +175 +175 +176 +176 +176 +176 +177 +177 +178 +178 +179 +179 +179 +179 +18 +18 +18 +18 +180 +180 +181 +181 +183 +183 +186 +186 +187 +187 +187 +187 +187 +187 +189 +189 +19 +19 +190 +190 +191 +191 +191 +191 +192 +192 +193 +193 +193 +193 +193 +193 +194 +194 +195 +195 +195 +195 +196 +196 +197 +197 +197 +197 +199 +199 +199 +199 +199 +199 +2 +2 +20 +20 +200 +200 +200 +200 +201 +201 +202 +202 +203 +203 +203 +203 +205 +205 +205 +205 +207 +207 +207 +207 +208 +208 +208 +208 +208 +208 +209 +209 +209 +209 +213 +213 +213 +213 +214 +214 +216 +216 +216 +216 +217 +217 +217 +217 +218 +218 +219 +219 +219 +219 +221 +221 +221 +221 +222 +222 +223 +223 +223 +223 +224 +224 +224 +224 +226 +226 +228 +228 +229 +229 +229 +229 +230 +230 +230 +230 +230 +230 +230 +230 +230 +230 +233 +233 +233 +233 +235 +235 +237 +237 +237 +237 +238 +238 +238 +238 +239 +239 +239 +239 +24 +24 +24 +24 +241 +241 +242 +242 +242 +242 +244 +244 +247 +247 +248 +248 +249 +249 +252 +252 +255 +255 +255 +255 +256 +256 +256 +256 +257 +257 +258 +258 +26 +26 +26 +26 +260 +260 +262 +262 +263 +263 +265 +265 +265 +265 +266 +266 +27 +27 +272 +272 +272 +272 +273 +273 +273 +273 +273 +273 +274 +274 +275 +275 +277 +277 +277 +277 +277 +277 +277 +277 +278 +278 +278 +278 +28 +28 +280 +280 +280 +280 +281 +281 +281 +281 +282 +282 +282 +282 +283 +283 +284 +284 +285 +285 +286 +286 +287 +287 +288 +288 +288 +288 +289 +289 +291 +291 +292 +292 +296 +296 +298 +298 +298 +298 +298 +298 +30 +30 +302 +302 +305 +305 +306 +306 +307 +307 +307 +307 +308 +308 +309 +309 +309 +309 +310 +310 +311 +311 +311 +311 +311 +311 +315 +315 +316 +316 +316 +316 +316 +316 +317 +317 +317 +317 +318 +318 +318 +318 +318 +318 +321 +321 +321 +321 +322 +322 +322 +322 +323 +323 +325 +325 +325 +325 +327 +327 +327 +327 +327 +327 +33 +33 +331 +331 +331 +331 +332 +332 +333 +333 +333 +333 +335 +335 +336 +336 +338 +338 +339 +339 +34 +34 +341 +341 +342 +342 +342 +342 +344 +344 +344 +344 +345 +345 +348 +348 +348 +348 +348 +348 +348 +348 +348 +348 +35 +35 +35 +35 +35 +35 +351 +351 +353 +353 +353 +353 +356 +356 +360 +360 +362 +362 +364 +364 +365 +365 +366 +366 +367 +367 +367 +367 +368 +368 +369 +369 +369 +369 +369 +369 +37 +37 +37 +37 +373 +373 +374 +374 +375 +375 +377 +377 +378 +378 +379 +379 +382 +382 +382 +382 +384 +384 +384 +384 +384 +384 +386 +386 +389 +389 +392 +392 +393 +393 +394 +394 +395 +395 +395 +395 +396 +396 +396 +396 +396 +396 +397 +397 +397 +397 +399 +399 +399 +399 +4 +4 +400 +400 +401 +401 +401 +401 +401 +401 +401 +401 +401 +401 +402 +402 +403 +403 +403 +403 +403 +403 +404 +404 +404 +404 +406 +406 +406 +406 +406 +406 +406 +406 +407 +407 +409 +409 +409 +409 +409 +409 +41 +41 +411 +411 +413 +413 +413 +413 +414 +414 +414 +414 +417 +417 +417 +417 +417 +417 +418 +418 +419 +419 +42 +42 +42 +42 +421 +421 +424 +424 +424 +424 +427 +427 +429 +429 +429 +429 +43 +43 +430 +430 +430 +430 +430 +430 +431 +431 +431 +431 +431 +431 +432 +432 +435 +435 +436 +436 +437 +437 +438 +438 +438 +438 +438 +438 +439 +439 +439 +439 +44 +44 +443 +443 +444 +444 +446 +446 +448 +448 +449 +449 +452 +452 +453 +453 +454 +454 +454 +454 +454 +454 +455 +455 +457 +457 +458 +458 +458 +458 +459 +459 +459 +459 +460 +460 +462 +462 +462 +462 +463 +463 +463 +463 +466 +466 +466 +466 +466 +466 +467 +467 +468 +468 +468 +468 +468 +468 +468 +468 +469 +469 +469 +469 +469 +469 +469 +469 +469 +469 +47 +47 +470 +470 +472 +472 +475 +475 +477 +477 +478 +478 +478 +478 +479 +479 +480 +480 +480 +480 +480 +480 +481 +481 +482 +482 +483 +483 +484 +484 +485 +485 +487 +487 +489 +489 +489 +489 +489 +489 +489 +489 +490 +490 +491 +491 +492 +492 +492 +492 +493 +493 +494 +494 +495 +495 +496 +496 +497 +497 +498 +498 +498 +498 +498 +498 +5 +5 +5 +5 +5 +5 +51 +51 +51 +51 +53 +53 +54 +54 +57 +57 +58 +58 +58 +58 +64 +64 +65 +65 +66 +66 +67 +67 +67 +67 +69 +69 +70 +70 +70 +70 +70 +70 +72 +72 +72 +72 +74 +74 +76 +76 +76 +76 +77 +77 +78 +78 +8 +8 +80 +80 +82 +82 +83 +83 +83 +83 +84 +84 +84 +84 +85 +85 +86 +86 +87 +87 +9 +9 +90 +90 +90 +90 +90 +90 +92 +92 +95 +95 +95 +95 +96 +96 +97 +97 +97 +97 +98 +98 +98 +98 \ No newline at end of file diff --git a/src/test/hive/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d b/src/test/hive/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_view-1-c790d4344144460224b0f02be7e137a8 b/src/test/hive/golden/union_view-1-c790d4344144460224b0f02be7e137a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 b/src/test/hive/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 b/src/test/hive/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e b/src/test/hive/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 b/src/test/hive/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae b/src/test/hive/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 b/src/test/hive/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 b/src/test/hive/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/src/test/hive/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 @@ -0,0 +1 @@ +0 diff --git a/src/test/hive/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af b/src/test/hive/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 b/src/test/hive/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_2-2-779a555944479991520be53f7f247626 b/src/test/hive/golden/varchar_2-2-779a555944479991520be53f7f247626 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da b/src/test/hive/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da new file mode 100644 index 0000000000000..08661a3d7f794 --- /dev/null +++ b/src/test/hive/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f b/src/test/hive/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f new file mode 100644 index 0000000000000..08661a3d7f794 --- /dev/null +++ b/src/test/hive/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 b/src/test/hive/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 new file mode 100644 index 0000000000000..f0c744ca37d95 --- /dev/null +++ b/src/test/hive/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f b/src/test/hive/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f new file mode 100644 index 0000000000000..f0c744ca37d95 --- /dev/null +++ b/src/test/hive/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f @@ -0,0 +1,5 @@ +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 diff --git a/src/test/hive/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af b/src/test/hive/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e b/src/test/hive/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 b/src/test/hive/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce b/src/test/hive/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce new file mode 100644 index 0000000000000..35e7795b9c2a7 --- /dev/null +++ b/src/test/hive/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce @@ -0,0 +1,3 @@ +1 abc 1 abc +2 abc 2 abc +3 abc 3 abc \ No newline at end of file diff --git a/src/test/hive/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c b/src/test/hive/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c new file mode 100644 index 0000000000000..35e7795b9c2a7 --- /dev/null +++ b/src/test/hive/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c @@ -0,0 +1,3 @@ +1 abc 1 abc +2 abc 2 abc +3 abc 3 abc \ No newline at end of file diff --git a/src/test/hive/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e b/src/test/hive/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 b/src/test/hive/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 b/src/test/hive/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 b/src/test/hive/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 b/src/test/hive/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 b/src/test/hive/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 b/src/test/hive/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef b/src/test/hive/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 b/src/test/hive/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 b/src/test/hive/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 b/src/test/hive/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 new file mode 100644 index 0000000000000..35e7795b9c2a7 --- /dev/null +++ b/src/test/hive/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 @@ -0,0 +1,3 @@ +1 abc 1 abc +2 abc 2 abc +3 abc 3 abc \ No newline at end of file diff --git a/src/test/hive/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc b/src/test/hive/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 b/src/test/hive/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 b/src/test/hive/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 new file mode 100644 index 0000000000000..05066db296756 --- /dev/null +++ b/src/test/hive/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 @@ -0,0 +1 @@ +c1 array None diff --git a/src/test/hive/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 b/src/test/hive/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 new file mode 100644 index 0000000000000..05cd8fccf276c --- /dev/null +++ b/src/test/hive/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 @@ -0,0 +1 @@ +["val_0","val_0"] diff --git a/src/test/hive/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f b/src/test/hive/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 b/src/test/hive/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 b/src/test/hive/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 new file mode 100644 index 0000000000000..1ec6828af9fd2 --- /dev/null +++ b/src/test/hive/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 @@ -0,0 +1 @@ +c1 map None diff --git a/src/test/hive/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 b/src/test/hive/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 new file mode 100644 index 0000000000000..8f12062de4e27 --- /dev/null +++ b/src/test/hive/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 @@ -0,0 +1 @@ +{0:"val_0"} diff --git a/src/test/hive/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 b/src/test/hive/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc b/src/test/hive/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 b/src/test/hive/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 new file mode 100644 index 0000000000000..030d9d6ec0429 --- /dev/null +++ b/src/test/hive/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 @@ -0,0 +1 @@ +c1 struct None diff --git a/src/test/hive/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 b/src/test/hive/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 new file mode 100644 index 0000000000000..f350c1112ac2c --- /dev/null +++ b/src/test/hive/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 @@ -0,0 +1 @@ +{"a":0,"b":"val_0","c":"val_0"} diff --git a/src/test/hive/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb b/src/test/hive/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 b/src/test/hive/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad b/src/test/hive/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad new file mode 100644 index 0000000000000..030d9d6ec0429 --- /dev/null +++ b/src/test/hive/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad @@ -0,0 +1 @@ +c1 struct None diff --git a/src/test/hive/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 b/src/test/hive/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 new file mode 100644 index 0000000000000..f350c1112ac2c --- /dev/null +++ b/src/test/hive/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 @@ -0,0 +1 @@ +{"a":0,"b":"val_0","c":"val_0"} diff --git a/src/test/hive/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 b/src/test/hive/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 b/src/test/hive/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 new file mode 100644 index 0000000000000..030d9d6ec0429 --- /dev/null +++ b/src/test/hive/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 @@ -0,0 +1 @@ +c1 struct None diff --git a/src/test/hive/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb b/src/test/hive/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb new file mode 100644 index 0000000000000..f350c1112ac2c --- /dev/null +++ b/src/test/hive/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb @@ -0,0 +1 @@ +{"a":0,"b":"val_0","c":"val_0"} diff --git a/src/test/hive/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc b/src/test/hive/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 b/src/test/hive/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb b/src/test/hive/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 b/src/test/hive/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 b/src/test/hive/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 b/src/test/hive/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 b/src/test/hive/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 b/src/test/hive/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 b/src/test/hive/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b b/src/test/hive/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 b/src/test/hive/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 b/src/test/hive/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 b/src/test/hive/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 b/src/test/hive/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d b/src/test/hive/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 b/src/test/hive/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e b/src/test/hive/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 b/src/test/hive/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e b/src/test/hive/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e @@ -0,0 +1,5 @@ +474 val_475 +62 val_63 +468 val_469 +272 val_273 +448 val_449 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 b/src/test/hive/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 @@ -0,0 +1,5 @@ +val_0 3 +val_1 2 +val_10 1 +val_100 2 +val_101 2 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 b/src/test/hive/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 b/src/test/hive/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a b/src/test/hive/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d b/src/test/hive/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d @@ -0,0 +1,5 @@ +474 val_475 +62 val_63 +468 val_469 +272 val_273 +448 val_449 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad b/src/test/hive/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad @@ -0,0 +1,5 @@ +val_0 3 +val_1 2 +val_10 1 +val_100 2 +val_101 2 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e b/src/test/hive/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 b/src/test/hive/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b b/src/test/hive/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 b/src/test/hive/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 b/src/test/hive/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 @@ -0,0 +1,5 @@ +474 val_475 +62 val_63 +468 val_469 +272 val_273 +448 val_449 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 b/src/test/hive/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 @@ -0,0 +1,5 @@ +val_0 3 +val_1 2 +val_10 1 +val_100 2 +val_101 2 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc b/src/test/hive/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f b/src/test/hive/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 b/src/test/hive/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e b/src/test/hive/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e @@ -0,0 +1,5 @@ +474 val_475 +62 val_63 +468 val_469 +272 val_273 +448 val_449 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c b/src/test/hive/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c @@ -0,0 +1,5 @@ +val_0 3 +val_1 2 +val_10 1 +val_100 2 +val_101 2 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 b/src/test/hive/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a b/src/test/hive/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 b/src/test/hive/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb b/src/test/hive/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 b/src/test/hive/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 @@ -0,0 +1,5 @@ +474 val_475 +62 val_63 +468 val_469 +272 val_273 +448 val_449 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 b/src/test/hive/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 @@ -0,0 +1,5 @@ +val_0 3 +val_1 2 +val_10 1 +val_100 2 +val_101 2 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 b/src/test/hive/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d b/src/test/hive/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e b/src/test/hive/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 b/src/test/hive/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f b/src/test/hive/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f b/src/test/hive/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba b/src/test/hive/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba b/src/test/hive/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 b/src/test/hive/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 b/src/test/hive/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 b/src/test/hive/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 @@ -0,0 +1,5 @@ +474 val_475 +62 val_63 +468 val_469 +272 val_273 +448 val_449 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 b/src/test/hive/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/src/test/hive/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 @@ -0,0 +1,5 @@ +val_0 3 +val_1 2 +val_10 1 +val_100 2 +val_101 2 \ No newline at end of file diff --git a/src/test/hive/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e b/src/test/hive/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec b/src/test/hive/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 b/src/test/hive/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 new file mode 100644 index 0000000000000..173eb955a485a --- /dev/null +++ b/src/test/hive/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 @@ -0,0 +1,4 @@ +1 abc +1 abc +2 abc +3 abc \ No newline at end of file diff --git a/src/test/hive/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 b/src/test/hive/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 new file mode 100644 index 0000000000000..173eb955a485a --- /dev/null +++ b/src/test/hive/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 @@ -0,0 +1,4 @@ +1 abc +1 abc +2 abc +3 abc \ No newline at end of file diff --git a/src/test/hive/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e b/src/test/hive/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec b/src/test/hive/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 b/src/test/hive/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 b/src/test/hive/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca b/src/test/hive/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b b/src/test/hive/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 b/src/test/hive/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 b/src/test/hive/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 b/src/test/hive/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 b/src/test/hive/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 b/src/test/hive/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 new file mode 100644 index 0000000000000..173eb955a485a --- /dev/null +++ b/src/test/hive/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 @@ -0,0 +1,4 @@ +1 abc +1 abc +2 abc +3 abc \ No newline at end of file diff --git a/src/test/hive/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a b/src/test/hive/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 b/src/test/hive/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 b/src/test/hive/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-11-dc95343d3e57846485dd543476391376 b/src/test/hive/golden/view-11-dc95343d3e57846485dd543476391376 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-12-371764e1cae31ea0518c03060528d239 b/src/test/hive/golden/view-12-371764e1cae31ea0518c03060528d239 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-13-2abce88008f8a19164758ee821aaa8a6 b/src/test/hive/golden/view-13-2abce88008f8a19164758ee821aaa8a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-14-deb504f4f70fd7db975950c3c47959ee b/src/test/hive/golden/view-14-deb504f4f70fd7db975950c3c47959ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b b/src/test/hive/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-2-9c529f486fa81a032bfe1253808fca8 b/src/test/hive/golden/view-2-9c529f486fa81a032bfe1253808fca8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-3-89c80c0e90409d5e304775c9f420915a b/src/test/hive/golden/view-3-89c80c0e90409d5e304775c9f420915a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-4-4a64d1a623ca71e515796787dbd0f904 b/src/test/hive/golden/view-4-4a64d1a623ca71e515796787dbd0f904 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b b/src/test/hive/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-6-47b5043f03a84695b6784682b4402ac8 b/src/test/hive/golden/view-6-47b5043f03a84695b6784682b4402ac8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 b/src/test/hive/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 b/src/test/hive/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 b/src/test/hive/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf b/src/test/hive/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb b/src/test/hive/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 b/src/test/hive/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de b/src/test/hive/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 b/src/test/hive/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 b/src/test/hive/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a b/src/test/hive/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 b/src/test/hive/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb b/src/test/hive/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 b/src/test/hive/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 b/src/test/hive/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 b/src/test/hive/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe b/src/test/hive/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d b/src/test/hive/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d new file mode 100644 index 0000000000000..ea00577174e43 --- /dev/null +++ b/src/test/hive/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d @@ -0,0 +1,500 @@ +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/src/test/hive/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 b/src/test/hive/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 new file mode 100644 index 0000000000000..df07a9da29f01 --- /dev/null +++ b/src/test/hive/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 @@ -0,0 +1,309 @@ +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/src/test/hive/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 b/src/test/hive/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 b/src/test/hive/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 new file mode 100644 index 0000000000000..2a0cdc268347a --- /dev/null +++ b/src/test/hive/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 @@ -0,0 +1,309 @@ +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/src/test/hive/golden/virtual_column-3-c66776673c986b59b27e704664935988 b/src/test/hive/golden/virtual_column-3-c66776673c986b59b27e704664935988 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 b/src/test/hive/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 new file mode 100644 index 0000000000000..94754d8bd4c82 --- /dev/null +++ b/src/test/hive/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 @@ -0,0 +1 @@ +97 val_97 \ No newline at end of file diff --git a/src/test/hive/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 b/src/test/hive/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 b/src/test/hive/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/src/test/hive/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/src/test/hive/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 b/src/test/hive/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/test/hive/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 b/src/test/hive/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 new file mode 100644 index 0000000000000..e6902f60ff486 --- /dev/null +++ b/src/test/hive/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 @@ -0,0 +1,500 @@ +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/src/test/hive/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 b/src/test/hive/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 new file mode 100644 index 0000000000000..e69de29bb2d1d From 2407a21180d261138454d23926786dcc20e88d1e Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Tue, 11 Feb 2014 16:29:11 -0800 Subject: [PATCH 664/778] Added optimized logical plan to debugging output --- src/main/scala/catalyst/execution/SharkInstance.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 9841aa9b3b0ad..164bd06d264b4 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -136,6 +136,8 @@ abstract class SharkInstance extends Logging { override def toString: String = s"""== Logical Plan == |${stringOrError(analyzed)} + |== Optimized Logical Plan == + |${stringOrError(optimizedPlan)} |== Physical Plan == |${stringOrError(executedPlan)} """.stripMargin.trim From cf691df0b020840be8bfaf0e29a7db4ef049b6f6 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Tue, 11 Feb 2014 16:30:14 -0800 Subject: [PATCH 665/778] Added the PhysicalOperation to generalize ColumnPrunings --- .../execution/planningStrategies.scala | 57 ++++++++++-------- .../scala/catalyst/planning/patterns.scala | 59 ++++++++++++++++++- .../catalyst/execution/PruningSuite.scala | 36 ++++++++--- 3 files changed, 118 insertions(+), 34 deletions(-) diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 908ff3c1f719e..3bc087416b099 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -64,37 +64,42 @@ trait PlanningStrategies { } /** - * A strategy that detects projection over filtered operation and applies column pruning if - * possible. + * A strategy that detects projects and filters over some relation and applies column pruning if + * possible. Partition pruning is applied first if the relation is partitioned. */ object ColumnPrunings extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { - case logical.Project(projectList, child @ FilteredOperation(predicates, m: MetastoreRelation)) - if isSimpleProject(projectList) => - - val projectAttributes = projectList.asInstanceOf[Seq[Attribute]] - val predicatesReferences = predicates.flatMap(_.references).toSet - val prunedAttributes = projectAttributes ++ (predicatesReferences -- projectAttributes) - - if (m.hiveQlTable.isPartitioned) { - // Applies partition pruning first for partitioned table - PartitionPrunings(child).view.map { sharkPlan => - execution.Project( - projectList, - sharkPlan.transform { - case scan@execution.HiveTableScan(attributes, _, _) => - scan.copy(attributes = prunedAttributes) - }) + case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) => + val predicateOpt = predicates.reduceOption(And) + val predicateRefs = predicateOpt.map(_.references).getOrElse(Set.empty) + val projectRefs = projectList.flatMap(_.references) + + // To figure out what columns to preserve after column pruning, we need to consider: + // + // 1. Columns referenced by the project list (order preserved) + // 2. Columns referenced by filtering predicates but not by project list + // 3. Relation output + // + // Then the final result is ((1 union 2) intersect 3) + val prunedCols = (projectRefs ++ (predicateRefs -- projectRefs)).intersect(relation.output) + + val filteredScans = + if (relation.hiveQlTable.isPartitioned) { + // Applies partition pruning first for partitioned table + val filteredRelation = predicateOpt.map(logical.Filter(_, relation)).getOrElse(relation) + PartitionPrunings(filteredRelation).view.map(_.transform { + case scan: execution.HiveTableScan => + scan.copy(attributes = prunedCols) + }) + } else { + val scan = execution.HiveTableScan(prunedCols, relation, None) + predicateOpt.map(execution.Filter(_, scan)).getOrElse(scan) :: Nil } + + if (isSimpleProject(projectList) && prunedCols == projectRefs) { + filteredScans } else { - val scan = execution.HiveTableScan(prunedAttributes, m, None) - val conjunctionOpt = predicates.reduceOption(And) - - execution.Project( - projectList, - conjunctionOpt - .map(execution.Filter(_, scan)) - .getOrElse(scan)) :: Nil + filteredScans.view.map(execution.Project(projectList, _)) } case _ => diff --git a/src/main/scala/catalyst/planning/patterns.scala b/src/main/scala/catalyst/planning/patterns.scala index 147a8fc6dffc9..eb4d74db67d9d 100644 --- a/src/main/scala/catalyst/planning/patterns.scala +++ b/src/main/scala/catalyst/planning/patterns.scala @@ -27,6 +27,63 @@ object FilteredOperation extends PredicateHelper { } } +/** + * A pattern that matches any number of project or filter operations on top of another relational + * operator. All filter operators are collected and their conditions are broken up and returned + * together with the top project operator. [[Alias Aliases]] are in-lined/substituted if necessary. + */ +object PhysicalOperation extends PredicateHelper { + type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan) + + def unapply(plan: LogicalPlan): Option[ReturnType] = + Some(collectProjectsAndFilters(None, Nil, plan, Map.empty)) + + /** + * Collects projects and filters, in-lining/substituting aliases if necessary. Here are two + * examples for alias in-lining/substitution. Before: + * {{{ + * SELECT c1 FROM (SELECT key AS c1 FROM t1 WHERE c1 > 10) t2 + * SELECT c1 AS c2 FROM (SELECT key AS c1 FROM t1 WHERE c1 > 10) t2 + * }}} + * After: + * {{{ + * SELECT key AS c1 FROM t1 WHERE key > 10 + * SELECT key AS c2 FROM t1 WHERE key > 10 + * }}} + */ + def collectProjectsAndFilters( + topFields: Option[Seq[Expression]], + filters: Seq[Expression], + plan: LogicalPlan, + aliases: Map[Attribute, Expression]): ReturnType = { + plan match { + case Project(fields, child) => + val moreAliases = aliases ++ collectAliases(fields) + val updatedTopFields = topFields.map(_.map(substitute(moreAliases))).getOrElse(fields) + collectProjectsAndFilters(Some(updatedTopFields), filters, child, moreAliases) + + case Filter(condition, child) => + val moreFilters = filters ++ splitConjunctivePredicates(condition) + collectProjectsAndFilters(topFields, moreFilters.map(substitute(aliases)), child, aliases) + + case other => + (topFields.getOrElse(other.output).asInstanceOf[Seq[NamedExpression]], filters, other) + } + } + + def collectAliases(fields: Seq[Expression]) = fields.collect { + case a @ Alias(child, _) => a.toAttribute.asInstanceOf[Attribute] -> child + }.toMap + + def substitute(aliases: Map[Attribute, Expression])(expr: Expression) = expr.transform { + case a @ Alias(ref: AttributeReference, name) => + aliases.get(ref).map(Alias(_, name)(a.exprId, a.qualifiers)).getOrElse(a) + + case a: AttributeReference => + aliases.get(a).map(Alias(_, a.name)(a.exprId, a.qualifiers)).getOrElse(a) + } +} + /** * A pattern that collects all adjacent unions and returns their children as a Seq. */ @@ -40,4 +97,4 @@ object Unions { case Union(l, r) => collectUnionChildren(l) ++ collectUnionChildren(r) case other => other :: Nil } -} \ No newline at end of file +} diff --git a/src/test/scala/catalyst/execution/PruningSuite.scala b/src/test/scala/catalyst/execution/PruningSuite.scala index 25ca7230bfbb3..0c509965f6c85 100644 --- a/src/test/scala/catalyst/execution/PruningSuite.scala +++ b/src/test/scala/catalyst/execution/PruningSuite.scala @@ -1,15 +1,19 @@ package catalyst.execution -import scala.collection.JavaConversions._ - import TestShark._ /** * A set of test cases that validate partition and column pruning. */ class PruningSuite extends HiveComparisonTest { + createPruningTest("Non-partitioned, non-simple project", + "SELECT key * 2 AS double FROM src WHERE value IS NOT NULL", + Seq("double"), + Seq("key", "value"), + Seq.empty) + createPruningTest("Pruning non-partitioned table", - "SELECT value from src WHERE key IS NOT NULL", + "SELECT value FROM src WHERE key IS NOT NULL", Seq("value"), Seq("value", "key"), Seq.empty) @@ -57,7 +61,7 @@ class PruningSuite extends HiveComparisonTest { expectedOutputColumns: Seq[String], expectedScannedColumns: Seq[String], expectedPartValues: Seq[Seq[String]]) = { - test(testCaseName) { + test(s"$testCaseName - pruning test") { val plan = sql.q.executedPlan val actualOutputColumns = plan.output.map(_.name) val (actualScannedColumns, actualPartValues) = plan.collect { @@ -67,9 +71,27 @@ class PruningSuite extends HiveComparisonTest { (columnNames, partValues) }.head - assert(actualOutputColumns sameElements expectedOutputColumns) - assert(actualScannedColumns sameElements expectedScannedColumns) - assert(actualPartValues.corresponds(expectedPartValues)(_ sameElements _)) + assert( + actualOutputColumns === expectedOutputColumns, + s"Output columns sould be $expectedOutputColumns, but are actually $actualOutputColumns") + + assert( + actualScannedColumns === expectedScannedColumns, + s"Scanned columns should be $expectedOutputColumns, but are actually $actualScannedColumns") + + assert( + actualPartValues.length === expectedPartValues.length, + s"There should be ${expectedPartValues.length} partition values, " + + s"but only ${actualPartValues.length} are found.") + + for ((actual, expected) <- actualPartValues.zip(expectedPartValues)) { + assert( + actual === expected, + s"Partition values should be $expected, but are actually $actual") + } } + + // Creates a query test to compare query results generated by Hive and Catalyst. + createQueryTest(s"$testCaseName - query test", sql) } } From f235914e3572919f5cb056b8a6794eb0623f5617 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Wed, 12 Feb 2014 01:14:22 -0800 Subject: [PATCH 666/778] Test case udf_regex and udf_like need BooleanWritable registered --- src/main/scala/catalyst/execution/FunctionRegistry.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/execution/FunctionRegistry.scala b/src/main/scala/catalyst/execution/FunctionRegistry.scala index fe76314a7dcf2..6491c6afafc95 100644 --- a/src/main/scala/catalyst/execution/FunctionRegistry.scala +++ b/src/main/scala/catalyst/execution/FunctionRegistry.scala @@ -47,6 +47,7 @@ object HiveFunctionRegistry extends analysis.FunctionRegistry { case c: Class[_] if c == classOf[org.apache.hadoop.io.IntWritable] => IntegerType case c: Class[_] if c == classOf[org.apache.hadoop.io.LongWritable] => LongType case c: Class[_] if c == classOf[org.apache.hadoop.io.FloatWritable] => FloatType + case c: Class[_] if c == classOf[org.apache.hadoop.io.BooleanWritable] => BooleanType case c: Class[_] if c == classOf[java.lang.String] => StringType case c: Class[_] if c == java.lang.Short.TYPE => ShortType case c: Class[_] if c == java.lang.Integer.TYPE => ShortType From f0c3742583d9a99bfc0f36c4fe9e2a497412c580 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Wed, 12 Feb 2014 01:23:07 -0800 Subject: [PATCH 667/778] Refactored PhysicalOperation The old version is implemented in a top down tail recursive manner, which cannot cover an uncommon corner case like: Filter (with aliases) Project ... MetastoreRelation In this case, the aliases are not in-lined/substituted because no aliases are collected yet. It is now covered by the new version which is implemented in a bottom up recursive manner and collects all necessary aliases before in-lining/substitution. --- .../scala/catalyst/planning/patterns.scala | 27 +++--- .../catalyst/execution/PruningSuite.scala | 86 +++++++++++++++---- 2 files changed, 80 insertions(+), 33 deletions(-) diff --git a/src/main/scala/catalyst/planning/patterns.scala b/src/main/scala/catalyst/planning/patterns.scala index eb4d74db67d9d..b6ef5115a7ad0 100644 --- a/src/main/scala/catalyst/planning/patterns.scala +++ b/src/main/scala/catalyst/planning/patterns.scala @@ -35,8 +35,10 @@ object FilteredOperation extends PredicateHelper { object PhysicalOperation extends PredicateHelper { type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan) - def unapply(plan: LogicalPlan): Option[ReturnType] = - Some(collectProjectsAndFilters(None, Nil, plan, Map.empty)) + def unapply(plan: LogicalPlan): Option[ReturnType] = { + val (fields, filters, child, _) = collectProjectsAndFilters(plan) + Some((fields.getOrElse(child.output), filters, child)) + } /** * Collects projects and filters, in-lining/substituting aliases if necessary. Here are two @@ -51,25 +53,22 @@ object PhysicalOperation extends PredicateHelper { * SELECT key AS c2 FROM t1 WHERE key > 10 * }}} */ - def collectProjectsAndFilters( - topFields: Option[Seq[Expression]], - filters: Seq[Expression], - plan: LogicalPlan, - aliases: Map[Attribute, Expression]): ReturnType = { + def collectProjectsAndFilters(plan: LogicalPlan): + (Option[Seq[NamedExpression]], Seq[Expression], LogicalPlan, Map[Attribute, Expression]) = plan match { case Project(fields, child) => - val moreAliases = aliases ++ collectAliases(fields) - val updatedTopFields = topFields.map(_.map(substitute(moreAliases))).getOrElse(fields) - collectProjectsAndFilters(Some(updatedTopFields), filters, child, moreAliases) + val (_, filters, other, aliases) = collectProjectsAndFilters(child) + val substitutedFields = fields.map(substitute(aliases)).asInstanceOf[Seq[NamedExpression]] + (Some(substitutedFields), filters, other, collectAliases(substitutedFields)) case Filter(condition, child) => - val moreFilters = filters ++ splitConjunctivePredicates(condition) - collectProjectsAndFilters(topFields, moreFilters.map(substitute(aliases)), child, aliases) + val (fields, filters, other, aliases) = collectProjectsAndFilters(child) + val substitutedCondition = substitute(aliases)(condition) + (fields, filters ++ splitConjunctivePredicates(substitutedCondition), other, aliases) case other => - (topFields.getOrElse(other.output).asInstanceOf[Seq[NamedExpression]], filters, other) + (None, Nil, other, Map.empty) } - } def collectAliases(fields: Seq[Expression]) = fields.collect { case a @ Alias(child, _) => a.toAttribute.asInstanceOf[Attribute] -> child diff --git a/src/test/scala/catalyst/execution/PruningSuite.scala b/src/test/scala/catalyst/execution/PruningSuite.scala index 0c509965f6c85..afa593d1ad921 100644 --- a/src/test/scala/catalyst/execution/PruningSuite.scala +++ b/src/test/scala/catalyst/execution/PruningSuite.scala @@ -1,24 +1,80 @@ package catalyst.execution +import scala.collection.JavaConversions._ + import TestShark._ /** * A set of test cases that validate partition and column pruning. */ class PruningSuite extends HiveComparisonTest { - createPruningTest("Non-partitioned, non-simple project", + // Column pruning tests + + createPruningTest("Column pruning: with partitioned table", + "SELECT key FROM srcpart WHERE ds = '2008-04-08' LIMIT 3", + Seq("key"), + Seq("key", "ds"), + Seq( + Seq("2008-04-08", "11"), + Seq("2008-04-08", "12"))) + + createPruningTest("Column pruning: with non-partitioned table", + "SELECT key FROM src WHERE key > 10 LIMIT 3", + Seq("key"), + Seq("key"), + Seq.empty) + + createPruningTest("Column pruning: with multiple projects", + "SELECT c1 FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3", + Seq("c1"), + Seq("key"), + Seq.empty) + + createPruningTest("Column pruning: projects alias substituting", + "SELECT c1 AS c2 FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3", + Seq("c2"), + Seq("key"), + Seq.empty) + + createPruningTest("Column pruning: filter alias in-lining", + "SELECT c1 FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 WHERE c1 < 100 LIMIT 3", + Seq("c1"), + Seq("key"), + Seq.empty) + + createPruningTest("Column pruning: without filters", + "SELECT c1 FROM (SELECT key AS c1 FROM src) t1 LIMIT 3", + Seq("c1"), + Seq("key"), + Seq.empty) + + createPruningTest("Column pruning: simple top project without aliases", + "SELECT key FROM (SELECT key FROM src WHERE key > 10) t1 WHERE key < 100 LIMIT 3", + Seq("key"), + Seq("key"), + Seq.empty) + + createPruningTest("Column pruning: non-trivial top project with aliases", + "SELECT c1 * 2 AS double FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3", + Seq("double"), + Seq("key"), + Seq.empty) + + // Partition pruning tests + + createPruningTest("Partition pruning: non-partitioned, non-trivial project", "SELECT key * 2 AS double FROM src WHERE value IS NOT NULL", Seq("double"), Seq("key", "value"), Seq.empty) - createPruningTest("Pruning non-partitioned table", + createPruningTest("Partiton pruning: non-partitioned table", "SELECT value FROM src WHERE key IS NOT NULL", Seq("value"), Seq("value", "key"), Seq.empty) - createPruningTest("Pruning with predicate on STRING partition key", + createPruningTest("Partition pruning: with filter on string partition key", "SELECT value, hr FROM srcpart1 WHERE ds = '2008-04-08'", Seq("value", "hr"), Seq("value", "hr", "ds"), @@ -26,7 +82,7 @@ class PruningSuite extends HiveComparisonTest { Seq("2008-04-08", "11"), Seq("2008-04-08", "12"))) - createPruningTest("Pruning with predicate on INT partition key", + createPruningTest("Partition pruning: with filter on int partition key", "SELECT value, hr FROM srcpart1 WHERE hr < 12", Seq("value", "hr"), Seq("value", "hr"), @@ -34,20 +90,20 @@ class PruningSuite extends HiveComparisonTest { Seq("2008-04-08", "11"), Seq("2008-04-09", "11"))) - createPruningTest("Select only 1 partition", + createPruningTest("Partition pruning: left only 1 partition", "SELECT value, hr FROM srcpart1 WHERE ds = '2008-04-08' AND hr < 12", Seq("value", "hr"), Seq("value", "hr", "ds"), Seq( Seq("2008-04-08", "11"))) - createPruningTest("All partitions pruned", + createPruningTest("Partition pruning: all partitions pruned", "SELECT value, hr FROM srcpart1 WHERE ds = '2014-01-27' AND hr = 11", Seq("value", "hr"), Seq("value", "hr", "ds"), Seq.empty) - createPruningTest("Pruning with both column key and partition key", + createPruningTest("Partition pruning: pruning with both column key and partition key", "SELECT value, hr FROM srcpart1 WHERE value IS NOT NULL AND hr < 12", Seq("value", "hr"), Seq("value", "hr"), @@ -71,23 +127,15 @@ class PruningSuite extends HiveComparisonTest { (columnNames, partValues) }.head - assert( - actualOutputColumns === expectedOutputColumns, - s"Output columns sould be $expectedOutputColumns, but are actually $actualOutputColumns") - - assert( - actualScannedColumns === expectedScannedColumns, - s"Scanned columns should be $expectedOutputColumns, but are actually $actualScannedColumns") + assert(actualOutputColumns sameElements expectedOutputColumns, "Output columns mismatch") + assert(actualScannedColumns sameElements expectedScannedColumns, "Scanned columns mismatch") assert( actualPartValues.length === expectedPartValues.length, - s"There should be ${expectedPartValues.length} partition values, " + - s"but only ${actualPartValues.length} are found.") + "Partition value count mismatches") for ((actual, expected) <- actualPartValues.zip(expectedPartValues)) { - assert( - actual === expected, - s"Partition values should be $expected, but are actually $actual") + assert(actual sameElements expected, "Partition values mismatch") } } From 5720d2bd2cd08c2ecbff32391ed88080cecd7359 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Wed, 12 Feb 2014 01:39:09 -0800 Subject: [PATCH 668/778] Fixed comment typo --- src/main/scala/catalyst/planning/patterns.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/planning/patterns.scala b/src/main/scala/catalyst/planning/patterns.scala index b6ef5115a7ad0..88625199686da 100644 --- a/src/main/scala/catalyst/planning/patterns.scala +++ b/src/main/scala/catalyst/planning/patterns.scala @@ -44,8 +44,8 @@ object PhysicalOperation extends PredicateHelper { * Collects projects and filters, in-lining/substituting aliases if necessary. Here are two * examples for alias in-lining/substitution. Before: * {{{ - * SELECT c1 FROM (SELECT key AS c1 FROM t1 WHERE c1 > 10) t2 - * SELECT c1 AS c2 FROM (SELECT key AS c1 FROM t1 WHERE c1 > 10) t2 + * SELECT c1 FROM (SELECT key AS c1 FROM t1) t2 WHERE c1 > 10 + * SELECT c1 AS c2 FROM (SELECT key AS c1 FROM t1) t2 WHERE c1 > 10 * }}} * After: * {{{ From bc9a12ce63f14f34aa9d74086f3485a6d338cf66 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 13 Feb 2014 15:18:26 -0800 Subject: [PATCH 669/778] Move hive test files. --- .../src}/test/hive/data/conf/hive-log4j.properties | 0 .../src}/test/hive/data/conf/hive-site.xml | 0 .../src}/test/hive/data/files/SortCol1Col2.txt | 0 .../src}/test/hive/data/files/SortCol2Col1.txt | 0 .../src}/test/hive/data/files/SortDescCol1Col2.txt | 0 .../src}/test/hive/data/files/SortDescCol2Col1.txt | 0 {src => shark/src}/test/hive/data/files/T1.txt | 0 {src => shark/src}/test/hive/data/files/T2.txt | 0 {src => shark/src}/test/hive/data/files/T3.txt | 0 .../src}/test/hive/data/files/TestSerDe.jar | Bin .../src}/test/hive/data/files/UserVisits.dat | 0 .../src}/test/hive/data/files/apache.access.2.log | 0 .../src}/test/hive/data/files/apache.access.log | 0 .../src}/test/hive/data/files/archive_corrupt.rc | Bin .../src}/test/hive/data/files/array_table.txt | 0 {src => shark/src}/test/hive/data/files/binary.txt | 0 {src => shark/src}/test/hive/data/files/bool.txt | 0 {src => shark/src}/test/hive/data/files/complex.seq | Bin .../src}/test/hive/data/files/covar_tab.txt | 0 .../test/hive/data/files/create_nested_type.txt | 0 {src => shark/src}/test/hive/data/files/csv.txt | 0 .../src}/test/hive/data/files/datatypes.txt | 0 .../src}/test/hive/data/files/dim-data.txt | 0 .../src}/test/hive/data/files/doctors.avro | Bin {src => shark/src}/test/hive/data/files/docurl.txt | 0 {src => shark/src}/test/hive/data/files/double.txt | 0 .../src}/test/hive/data/files/employee.dat | 0 .../src}/test/hive/data/files/employee2.dat | 0 .../src}/test/hive/data/files/employee_part.txt | 0 {src => shark/src}/test/hive/data/files/empty1.txt | 0 {src => shark/src}/test/hive/data/files/empty2.txt | 0 .../src}/test/hive/data/files/episodes.avro | Bin .../src}/test/hive/data/files/escapetest.txt | Bin .../src}/test/hive/data/files/ext_test/test.dat | 0 .../src}/test/hive/data/files/fact-data.txt | 0 .../src}/test/hive/data/files/flights_join.txt | 0 .../src}/test/hive/data/files/flights_tiny.txt | 0 .../src}/test/hive/data/files/flights_tiny.txt.1 | 0 .../test/hive/data/files/groupby_groupingid.txt | 0 .../src}/test/hive/data/files/grouping_sets.txt | 0 .../src}/test/hive/data/files/grouping_sets1.txt | 0 .../src}/test/hive/data/files/grouping_sets2.txt | 0 .../src}/test/hive/data/files/hive_626_bar.txt | 0 .../src}/test/hive/data/files/hive_626_count.txt | 0 .../src}/test/hive/data/files/hive_626_foo.txt | 0 {src => shark/src}/test/hive/data/files/in1.txt | 0 {src => shark/src}/test/hive/data/files/in2.txt | 0 {src => shark/src}/test/hive/data/files/in3.txt | 0 {src => shark/src}/test/hive/data/files/in4.txt | 0 {src => shark/src}/test/hive/data/files/in5.txt | 0 {src => shark/src}/test/hive/data/files/in6.txt | 0 {src => shark/src}/test/hive/data/files/in7.txt | 0 {src => shark/src}/test/hive/data/files/in8.txt | 0 {src => shark/src}/test/hive/data/files/in9.txt | 0 .../src}/test/hive/data/files/infer_const_type.txt | 0 {src => shark/src}/test/hive/data/files/int.txt | 0 {src => shark/src}/test/hive/data/files/json.txt | 0 {src => shark/src}/test/hive/data/files/kv1.seq | Bin .../src}/test/hive/data/files/kv1.string-sorted.txt | 0 {src => shark/src}/test/hive/data/files/kv1.txt | 0 .../src}/test/hive/data/files/kv1.val.sorted.txt | 0 .../src}/test/hive/data/files/kv1_broken.seq | Bin {src => shark/src}/test/hive/data/files/kv1_cb.txt | 0 {src => shark/src}/test/hive/data/files/kv1_cc.txt | 0 .../src}/test/hive/data/files/kv1kv2.cogroup.txt | 0 {src => shark/src}/test/hive/data/files/kv2.txt | 0 {src => shark/src}/test/hive/data/files/kv3.txt | 0 {src => shark/src}/test/hive/data/files/kv4.txt | 0 {src => shark/src}/test/hive/data/files/kv5.txt | 0 {src => shark/src}/test/hive/data/files/kv6.txt | 0 {src => shark/src}/test/hive/data/files/kv7.txt | 0 {src => shark/src}/test/hive/data/files/kv8.txt | 0 .../test/hive/data/files/leftsemijoin_mr_t1.txt | 0 .../test/hive/data/files/leftsemijoin_mr_t2.txt | 0 .../src}/test/hive/data/files/lineitem.txt | 0 .../src}/test/hive/data/files/lt100.sorted.txt | 0 {src => shark/src}/test/hive/data/files/lt100.txt | 0 .../src}/test/hive/data/files/lt100.txt.deflate | 0 .../src}/test/hive/data/files/map_table.txt | 0 .../src}/test/hive/data/files/nested_complex.txt | 0 {src => shark/src}/test/hive/data/files/null.txt | 0 .../src}/test/hive/data/files/nullfile.txt | 0 {src => shark/src}/test/hive/data/files/nulls.txt | 0 .../src}/test/hive/data/files/orc_create.txt | 0 .../src}/test/hive/data/files/orc_create_people.txt | 0 {src => shark/src}/test/hive/data/files/over10k | 0 {src => shark/src}/test/hive/data/files/over1k | 0 {src => shark/src}/test/hive/data/files/part.rc | Bin {src => shark/src}/test/hive/data/files/part.seq | Bin .../src}/test/hive/data/files/part_tiny.txt | 0 .../src}/test/hive/data/files/person age.txt | 0 .../test/hive/data/files/primitive_type_arrays.txt | 0 {src => shark/src}/test/hive/data/files/pw17.txt | 0 {src => shark/src}/test/hive/data/files/sales.txt | 0 .../hive/data/files/sample-queryplan-in-history.txt | 0 .../src}/test/hive/data/files/sample-queryplan.txt | 0 .../hive/data/files/smallsrcsortbucket1outof4.txt | 0 .../hive/data/files/smallsrcsortbucket2outof4.txt | 0 .../hive/data/files/smallsrcsortbucket3outof4.txt | 0 .../hive/data/files/smallsrcsortbucket4outof4.txt | 0 .../src}/test/hive/data/files/smb_bucket_input.rc | Bin .../src}/test/hive/data/files/smb_bucket_input.txt | 0 .../src}/test/hive/data/files/smbbucket_1.rc | Bin .../src}/test/hive/data/files/smbbucket_1.txt | 0 .../src}/test/hive/data/files/smbbucket_2.rc | Bin .../src}/test/hive/data/files/smbbucket_2.txt | 0 .../src}/test/hive/data/files/smbbucket_3.rc | Bin .../src}/test/hive/data/files/smbbucket_3.txt | 0 {src => shark/src}/test/hive/data/files/source.txt | 0 .../src}/test/hive/data/files/srcbucket0.txt | 0 .../src}/test/hive/data/files/srcbucket1.txt | 0 .../src}/test/hive/data/files/srcbucket20.txt | 0 .../src}/test/hive/data/files/srcbucket21.txt | 0 .../src}/test/hive/data/files/srcbucket22.txt | 0 .../src}/test/hive/data/files/srcbucket23.txt | 0 .../test/hive/data/files/srcsortbucket1outof4.txt | 0 .../test/hive/data/files/srcsortbucket2outof4.txt | 0 .../test/hive/data/files/srcsortbucket3outof4.txt | 0 .../test/hive/data/files/srcsortbucket4outof4.txt | 0 {src => shark/src}/test/hive/data/files/string.txt | Bin .../src}/test/hive/data/files/symlink1.txt | 0 .../src}/test/hive/data/files/symlink2.txt | 0 {src => shark/src}/test/hive/data/files/tbl.txt | 0 {src => shark/src}/test/hive/data/files/test.dat | 0 {src => shark/src}/test/hive/data/files/test2.dat | 0 {src => shark/src}/test/hive/data/files/text-en.txt | 0 {src => shark/src}/test/hive/data/files/things.txt | 0 {src => shark/src}/test/hive/data/files/things2.txt | 0 {src => shark/src}/test/hive/data/files/tiny_a.txt | 0 {src => shark/src}/test/hive/data/files/tiny_b.txt | 0 .../hive/data/files/types/primitives/090101.txt | 0 .../hive/data/files/types/primitives/090201.txt | 0 .../hive/data/files/types/primitives/090301.txt | 0 .../hive/data/files/types/primitives/090401.txt | 0 .../src}/test/hive/data/files/union_input.txt | 0 {src => shark/src}/test/hive/data/files/v1.txt | 0 {src => shark/src}/test/hive/data/files/v2.txt | 0 {src => shark/src}/test/hive/data/files/vc1.txt | 0 {src => shark/src}/test/hive/data/files/x.txt | 0 {src => shark/src}/test/hive/data/files/y.txt | 0 {src => shark/src}/test/hive/data/files/z.txt | 0 {src => shark/src}/test/hive/data/metadb/.gitignore | 0 {src => shark/src}/test/hive/data/scripts/cat.py | 0 .../src}/test/hive/data/scripts/cat_error.py | 0 .../src}/test/hive/data/scripts/doubleescapedtab.py | 0 .../src}/test/hive/data/scripts/dumpdata_script.py | 0 .../src}/test/hive/data/scripts/error_script | 0 .../test/hive/data/scripts/escapedcarriagereturn.py | 0 .../src}/test/hive/data/scripts/escapednewline.py | 0 .../src}/test/hive/data/scripts/escapedtab.py | 0 .../src}/test/hive/data/scripts/input20_script | 0 .../src}/test/hive/data/scripts/newline.py | 0 .../src}/test/hive/data/scripts/q_test_cleanup.sql | 0 .../src}/test/hive/data/scripts/q_test_init.sql | 0 .../src}/test/hive/data/scripts/test_init_file.sql | 0 .../'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb | 0 .../'1' + 1-0-130514c6116c311d808590a075b187b | 0 .../'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e | 0 .../'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 | 0 .../'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 | 0 .../'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a | 0 .../1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a | 0 .../golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 | 0 .../1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 | 0 .../1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e | 0 .../1 + 1S-0-2e99da48f67f588c9e632a57c713522e | 0 .../1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f | 0 .../1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 | 0 .../1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 | 0 .../1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d | 0 .../1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 | 0 .../1.0 + 1S-0-31fbe14d01fb532176c1689680398368 | 0 .../1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 | 0 .../1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a | 0 .../1L + 1-0-1864a260554255a09e4f28b8551eef9d | 0 .../1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 | 0 .../1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f | 0 .../1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 | 0 .../1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 | 0 .../1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 | 0 .../1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 | 0 .../1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e | 0 .../1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 | 0 .../1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e | 0 .../1S + 1Y-0-e59bc8279cd364224476ffc504c7685b | 0 .../1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb | 0 .../golden/1Y + 1-0-a4541db51882b19503649138fbb295f | 0 .../1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d | 0 .../1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 | 0 .../1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 | 0 .../1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 | 0 .../DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a | 0 ...ape sequences-0-2f25c33d97c43f3276171624d988a286 | 0 ...IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 | 0 .../golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 | 0 ...ab separated)-0-779101eb00fd8bb9f08908ab29e90c03 | 0 ...ith AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 | 0 ...th RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa | 0 ... 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 | 0 ...e Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 | 0 ...imple Average-0-c197ea78c4d8f85f1a317805b6da07e5 | 0 ...dd_part_exist-0-e58d09864bc6898e44fae80abe328702 | 0 ...dd_part_exist-1-d824f22606f48dfca48ce241a7505f5b | 0 ...d_part_exist-10-ecb27eb754e731429659224b5b6ac583 | 0 ...d_part_exist-11-9c36cac1372650b703400c60dd29042c | 0 ...d_part_exist-12-2900a5065dd3adbb0b56a712bf848750 | 0 ...d_part_exist-13-d69279050a0f44c3f8b775698fd29be0 | 0 ...d_part_exist-14-9c36cac1372650b703400c60dd29042c | 0 ...d_part_exist-15-e58d09864bc6898e44fae80abe328702 | 0 ...d_part_exist-16-d824f22606f48dfca48ce241a7505f5b | 0 ...d_part_exist-17-3432935e802ae46c6b2151cc4ebf783b | 0 ...d_part_exist-18-d824f22606f48dfca48ce241a7505f5b | 0 ...d_part_exist-19-83cecdbfddb070a417050f9a18fff752 | 0 ...dd_part_exist-2-3432935e802ae46c6b2151cc4ebf783b | 0 ...d_part_exist-20-d824f22606f48dfca48ce241a7505f5b | 0 ...d_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f | 0 ...d_part_exist-22-d824f22606f48dfca48ce241a7505f5b | 0 ...d_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb | 0 ...d_part_exist-24-d824f22606f48dfca48ce241a7505f5b | 0 ...dd_part_exist-3-d824f22606f48dfca48ce241a7505f5b | 0 ...dd_part_exist-4-83cecdbfddb070a417050f9a18fff752 | 0 ...dd_part_exist-5-d824f22606f48dfca48ce241a7505f5b | 0 ...dd_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f | 0 ...dd_part_exist-7-d824f22606f48dfca48ce241a7505f5b | 0 ...dd_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb | 0 ...dd_part_exist-9-d824f22606f48dfca48ce241a7505f5b | 0 ...part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 | 0 ...part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 | 0 ...part_multiple-2-187930e0daa44eed17e092e961ab7955 | 0 ...part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef | 0 ...part_multiple-4-7950c676506564b085b41426ed41747c | 0 ..._no_whitelist-0-3806584ff765bca682594008b90fc304 | 0 ..._no_whitelist-1-22eb96fe7d338e488182b5755c90d5af | 0 ..._no_whitelist-2-923fa18234ae73103c43722f70e000c0 | 0 ..._no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f | 0 ...ith_whitelist-0-3c23ae800b2f6fb93620890da2dba196 | 0 ...ith_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 | 0 ...ith_whitelist-2-e6a91be97431de63e372088d370c6d36 | 0 ...ith_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 | 0 .../alias.*-0-7bdb861d11e895aaea545810cdac316d | 0 .../alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 | 0 ...casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 | 0 ...casted_column-1-4432aec015f9423ed991b08cfb2af0e1 | 0 .../golden/alter2-0-85e494848d1525843a3ff9b2b77f92 | 0 .../golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 | 0 .../alter2-10-2d1f665a92fe72bd253ae57c46d7b9da | 0 .../alter2-11-aac9c2c7033fd7264c9a107a88ff591 | 0 .../alter2-12-75a213649242c2410ea6846f08c91d75 | 0 .../alter2-13-ca51e03a3de391983429b6ad877e573c | 0 .../alter2-14-aac9c2c7033fd7264c9a107a88ff591 | 0 .../alter2-15-75a213649242c2410ea6846f08c91d75 | 0 .../alter2-16-77500c83ffcece95511a4b21d67382dd | 0 .../alter2-17-aac9c2c7033fd7264c9a107a88ff591 | 0 .../alter2-18-75a213649242c2410ea6846f08c91d75 | 0 .../alter2-19-3d80bb2d1c541460b5b17c4124fa647 | 0 .../alter2-2-75a213649242c2410ea6846f08c91d75 | 0 .../alter2-20-9c36cac1372650b703400c60dd29042c | 0 .../alter2-21-a6ea9efed3b9c680fca93588019ac5e3 | 0 .../alter2-22-ea07b8f664208f93a1a8b97fd486d226 | 0 .../alter2-23-9c36cac1372650b703400c60dd29042c | 0 .../alter2-24-775d69742a1c07df8da87e8a017d955 | 0 .../alter2-25-4ef75e12575453225738ea167c4617e5 | 0 .../alter2-26-63f13c364546ddce5d2176c6604a948f | 0 .../alter2-27-ba521286c12ba29329bfa71bb185c62f | 0 .../alter2-28-4ef75e12575453225738ea167c4617e5 | 0 .../alter2-29-63f13c364546ddce5d2176c6604a948f | 0 .../alter2-3-ca51e03a3de391983429b6ad877e573c | 0 .../alter2-30-a336adf1d0ff00633c53600fc75ca3ae | 0 .../alter2-31-4ef75e12575453225738ea167c4617e5 | 0 .../alter2-32-63f13c364546ddce5d2176c6604a948f | 0 .../alter2-33-1934026d0228967097280eed35551f74 | 0 .../alter2-34-23b00f9c0101348e87da8a339b9da8b | 0 .../alter2-35-4ef75e12575453225738ea167c4617e5 | 0 .../alter2-36-63f13c364546ddce5d2176c6604a948f | 0 .../alter2-37-ba521286c12ba29329bfa71bb185c62f | 0 .../alter2-38-4ef75e12575453225738ea167c4617e5 | 0 .../alter2-39-63f13c364546ddce5d2176c6604a948f | 0 .../golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 | 0 .../alter2-40-a336adf1d0ff00633c53600fc75ca3ae | 0 .../alter2-41-4ef75e12575453225738ea167c4617e5 | 0 .../alter2-42-63f13c364546ddce5d2176c6604a948f | 0 .../alter2-43-1934026d0228967097280eed35551f74 | 0 .../alter2-44-7a9e67189d3d4151f23b12c22bde06b5 | 0 .../alter2-45-adbc01277c01cc5647e89c8a2430b8c | 0 .../alter2-5-75a213649242c2410ea6846f08c91d75 | 0 .../alter2-6-77500c83ffcece95511a4b21d67382dd | 0 .../golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 | 0 .../alter2-8-75a213649242c2410ea6846f08c91d75 | 0 .../alter2-9-1986a53bb1944fe5f43e3e65693e7b1e | 0 .../alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade | 0 .../alter3-1-75be487df30e301e156a22eee075633d | 0 .../alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 | 0 .../alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 | 0 .../alter3-12-2fcb7fc251f682a584ad513fddfac506 | 0 .../alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b | 0 .../alter3-14-3fa4d8a690a45cbf7b44cecfd352864f | 0 .../alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 | 0 .../alter3-16-9c36cac1372650b703400c60dd29042c | 0 .../alter3-17-a9908f67f97588cbf15c0e7caddcbb0b | 0 .../alter3-18-581b65f9f467d0d4a33a16dda144a31d | 0 .../alter3-19-9c36cac1372650b703400c60dd29042c | 0 .../alter3-2-5a67d369d700eb96f806f8320c04d61f | 0 .../alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 | 0 .../alter3-21-231db1adbff5fc90e57cca6a087f3df5 | 0 .../alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 | 0 .../alter3-23-7ad62b397f6c9341da6bf0e9361314e2 | 0 .../alter3-24-9c23b682abda3841f01b4d9b750c68d9 | 0 .../alter3-25-568a59760e5d3241b63d65cce595face | 0 .../alter3-26-3c725018f74a69f4d859c66af2f5b11e | 0 .../alter3-27-54ad133b447f67c6d1ed7d4c43803a87 | 0 .../alter3-28-5332228ea451105c897d0c8c3c8f2773 | 0 .../alter3-29-b8fba19b9009131caffbb5fe7468b67c | 0 .../alter3-3-f031aa27bf7b494cb8de20a305be7064 | 0 .../alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 | 0 .../alter3-31-5a41cf8c1a828ac2c372536ee4afd962 | 0 .../alter3-32-327744965ee8ed630f56fa3e4a3c5c65 | 0 .../alter3-4-399fc26b344c98ababa104522601c0cc | 0 .../alter3-5-bf2a8fd1884bb584059c848332e30c97 | 0 .../alter3-6-fe6db2a539df10e4bc4715e2ed755135 | 0 .../golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 | 0 .../alter3-8-8f0a466bd1d021e40690865b7ae52a43 | 0 .../alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 | 0 .../alter4-0-c261e1fa9f838dd034d37af38305e9c6 | 0 .../alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 | 0 .../alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 | 0 .../alter4-11-7db7af854e4e10fb6f0338b85d65549d | 0 .../alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 | 0 .../alter4-13-e9879d6bebc109340bbeecc3ca77492f | 0 .../alter4-14-7a9e67189d3d4151f23b12c22bde06b5 | 0 .../alter4-15-63a545ee0e751a2729c8758a14712da5 | 0 .../alter4-16-549981e00a3d95f03dd5a9ef6044aa20 | 0 .../alter4-2-7db7af854e4e10fb6f0338b85d65549d | 0 .../alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 | 0 .../alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 | 0 .../alter4-5-9c36cac1372650b703400c60dd29042c | 0 .../golden/alter4-6-acd58e84952d310aeddf78579c36286 | 0 .../alter4-7-8e6ec3396f25c124de5b212d8ce6c568 | 0 .../alter4-8-9c36cac1372650b703400c60dd29042c | 0 .../alter4-9-c261e1fa9f838dd034d37af38305e9c6 | 0 .../alter5-0-953553e14d835682fa47338dcfffe227 | 0 .../alter5-1-b8349afaf8e62dc6608a889c04ee3d4b | 0 .../alter5-10-9c36cac1372650b703400c60dd29042c | 0 .../alter5-11-e63fa029ab22ac4f5c880f2848f1b956 | 0 .../alter5-12-6b160869b8a9c846bc55a14f85bc5b52 | 0 .../alter5-13-9c36cac1372650b703400c60dd29042c | 0 .../alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 | 0 .../alter5-15-b8349afaf8e62dc6608a889c04ee3d4b | 0 .../alter5-16-4b76b7ff0df6adeded64b2a2f305530d | 0 .../alter5-17-9176dc5fb5206209fa907a289db1263d | 0 .../alter5-18-2a9c8219c1468a1cf0534c665d1fcebf | 0 .../alter5-19-2f6ab691e291c74ecc4305eeb30e3438 | 0 .../alter5-2-4b76b7ff0df6adeded64b2a2f305530d | 0 .../alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 | 0 .../alter5-21-2a9c8219c1468a1cf0534c665d1fcebf | 0 .../alter5-3-2fc59e32c07186869811705c89aafadc | 0 .../alter5-4-2a9c8219c1468a1cf0534c665d1fcebf | 0 .../alter5-5-2f6ab691e291c74ecc4305eeb30e3438 | 0 .../alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 | 0 .../alter5-7-2a9c8219c1468a1cf0534c665d1fcebf | 0 .../alter5-8-89c414c65a129f2fc408b3124f292b29 | 0 .../alter5-9-1e085f9741197e659413828c42386733 | 0 .../alter_index-0-21bcf37075b02097f16c8fc8130a83b8 | 0 .../alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e | 0 .../alter_index-2-f36cb2eed39691ca949b25182e2dd31 | 0 .../alter_index-3-33474b65c86b949d266541e0385bc6bb | 0 .../alter_index-4-f36cb2eed39691ca949b25182e2dd31 | 0 .../alter_index-5-21bcf37075b02097f16c8fc8130a83b8 | 0 .../alter_index-6-489b4ceb2f4301a7132628303f99240d | 0 ...alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 | 0 ...alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 | 0 ...lter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 | 0 ...alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c | 0 ...alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 | 0 ...alter_merge_2-3-cfef140167765d259320ed1e8aba718d | 0 ...alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be | 0 ...alter_merge_2-5-43bd090cda258e25037e7c32d500a85f | 0 ...alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 | 0 ...alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 | 0 ...alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc | 0 ...alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 | 0 ...ition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a | 0 ...ition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 | 0 ...tion_coltype-10-aa739a81271c760437de572a6c951eb9 | 0 ...tion_coltype-11-94b98dca970e36e6d4d65a795c9413d6 | 0 ...tion_coltype-12-84807e0be3e91250d4b8f2dc7938a256 | 0 ...tion_coltype-13-60ab4c242e19b882e4b1571544311e7e | 0 ...tion_coltype-14-fdad866a990591083719bd45e4113f58 | 0 ...tion_coltype-15-634775da8cebfb8ce45b3965a0ae2880 | 0 ...tion_coltype-16-b0534980e325d1fee63c2636ad3f8a4e | 0 ...tion_coltype-17-13c0443789a696bde8d08c05f526643f | 0 ...tion_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c | 0 ...tion_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 | 0 ...tition_coltype-2-17e04afbb81a724fe8c47c07b642f9a | 0 ...tion_coltype-20-b4411edb9c52a474a971640f037c8a30 | 0 ...ition_coltype-21-17e04afbb81a724fe8c47c07b642f9a | 0 ...tion_coltype-22-639cdccdea5f4b1863f9b17b04183c93 | 0 ...tion_coltype-23-325be9a5d7c0277498a139c0a9fdb26a | 0 ...tion_coltype-24-71de9e476503e83f82baf1ab17da87f6 | 0 ...tion_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 | 0 ...ition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 | 0 ...ition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d | 0 ...ition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d | 0 ...ition_coltype-6-db84ad9022cdc1828c24a0340321c8fd | 0 ...ition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 | 0 ...ition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 | 0 ...ition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 | 0 ...on_format_loc-0-72ba9397f487a914380dc15afaef1058 | 0 ...on_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 | 0 ...n_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 | 0 ...n_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 | 0 ...n_format_loc-12-1553ad79b098b737ea8def91134eb0e9 | 0 ...n_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 | 0 ...n_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 | 0 ...n_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 | 0 ...n_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 | 0 ...n_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 | 0 ...n_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 | 0 ...n_format_loc-19-56cadf0f555e355726dfed1929ad0508 | 0 ...on_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 | 0 ...on_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 | 0 ...on_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 | 0 ...on_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 | 0 ...on_format_loc-6-56cadf0f555e355726dfed1929ad0508 | 0 ...on_format_loc-7-cee355b012efdc3bc7d584268a7025c2 | 0 ...on_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 | 0 ...on_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 | 0 ..._protect_mode-0-2a230c069b09232acdd0d556007be97f | 0 ..._protect_mode-1-fbbdf7be607407661749730f1a0efd9c | 0 ...protect_mode-10-d71b99098bdb7f13db278dfa299b820d | 0 ...protect_mode-11-482182c9d90710fb16b6803d602a0d8b | 0 ...protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 | 0 ...protect_mode-13-19ceced1d8238509f2416029ddfbbc4a | 0 ...protect_mode-14-90d009f94408102945d43860e4a6c68a | 0 ...protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 | 0 ...protect_mode-16-577e1c164866c3955a9d8587ef7918a4 | 0 ..._protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 | 0 ..._protect_mode-3-dd92c46e933d94b35c225daeef0285d4 | 0 ...n_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 | 0 ..._protect_mode-5-3cc094c5aa537b12f98895b95765329c | 0 ..._protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 | 0 ..._protect_mode-7-5439426a18bb2d3918b91d589dbbd014 | 0 ..._protect_mode-8-71e8c12c533654c30e044a8f062598ed | 0 ..._protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d | 0 ...ith_whitelist-0-3c23ae800b2f6fb93620890da2dba196 | 0 ...ith_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 | 0 ...ith_whitelist-2-e6a91be97431de63e372088d370c6d36 | 0 ...ith_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 | 0 ...ith_whitelist-4-f42e9ca89ed2944213a5d994a587391c | 0 ...name_partition-0-1ed18256c5230de3439fe75d925ea73 | 0 ...ame_partition-1-2f79bceed6fc8ada34a670396ee6aada | 0 ...me_partition-10-e3d9a36d53d30de215b855095c58d0d7 | 0 ...me_partition-11-bc84e38ccad173f5b47474d91db244d7 | 0 ...ame_partition-12-1ed18256c5230de3439fe75d925ea73 | 0 ...me_partition-13-2f79bceed6fc8ada34a670396ee6aada | 0 ...me_partition-14-9c36cac1372650b703400c60dd29042c | 0 ...me_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 | 0 ...me_partition-16-611cf586cf3a1adc93c543d2da574c24 | 0 ...me_partition-17-9c36cac1372650b703400c60dd29042c | 0 ...me_partition-18-bf6f780173f7b523b7ebd7925789372b | 0 ...me_partition-19-b2c9ded072d49abe14831bf48290319c | 0 ...ame_partition-2-9c36cac1372650b703400c60dd29042c | 0 ...me_partition-20-ee3ad861d109dd98db10bc86c5bf7105 | 0 ...me_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 | 0 ...me_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 | 0 ...me_partition-23-aedbaca33604c76b65137905fd42e98f | 0 ...me_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 | 0 ...me_partition-25-9595f5b6ab31162c107076c35657c9f3 | 0 ...me_partition-26-270655c514bb6f04acd4459df52dd77b | 0 ...ame_partition-3-b465c6126edd94e8d45f61e2a19d005b | 0 ...ame_partition-4-3479a886936571d5028971aecade705f | 0 ...ame_partition-5-6cc4e3014e34a862602a47357f4fb9f2 | 0 ...ame_partition-6-3324664e6500e2d256d0b8b3b8a14c24 | 0 ...ame_partition-7-e3d9a36d53d30de215b855095c58d0d7 | 0 ...ame_partition-8-d3ea111b1a37613bdda2c6eae13790c9 | 0 ...ame_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 | 0 ...r_table_serde-0-35d2014351106b918c8e337a1919470c | 0 ...r_table_serde-1-5bc931a540f0fec54e852ff10f52f879 | 0 ..._table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa | 0 ..._table_serde-11-6ee4b3a60659ec5496f06347eda232a8 | 0 ..._table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc | 0 ..._table_serde-13-6ee4b3a60659ec5496f06347eda232a8 | 0 ..._table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def | 0 ..._table_serde-15-6ee4b3a60659ec5496f06347eda232a8 | 0 ..._table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 | 0 ..._table_serde-17-6ee4b3a60659ec5496f06347eda232a8 | 0 ..._table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 | 0 ...r_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa | 0 ...r_table_serde-3-5bc931a540f0fec54e852ff10f52f879 | 0 ...r_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc | 0 ...r_table_serde-5-5bc931a540f0fec54e852ff10f52f879 | 0 ...r_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 | 0 ...r_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b | 0 ...r_table_serde-8-78d739d2409b59c0e01cde962451d295 | 0 ...r_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 | 0 ...lter_varchar1-0-5fa6071842a0443346cf6db677a33412 | 0 ...lter_varchar1-1-be11cb1f18ab19550011417126264fea | 0 ...ter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 | 0 ...ter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 | 0 ...ter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd | 0 ...ter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 | 0 ...ter_varchar1-14-5fa6071842a0443346cf6db677a33412 | 0 ...lter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 | 0 ...lter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 | 0 ...lter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d | 0 ...lter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 | 0 ...lter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 | 0 ...lter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f | 0 ...alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 | 0 ...lter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 | 0 ...lter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 | 0 ...lter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc | 0 ...lter_varchar2-2-325238d61f56d84c17e29033105d7b19 | 0 ...lter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 | 0 ...alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb | 0 ...lter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c | 0 ...lter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 | 0 ...lter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c | 0 ...lter_varchar2-8-4c12c4c53d99338796be34e603dc612c | 0 ...iew_as_select-0-9f40bf1c2b92465189583446a6b40910 | 0 ...iew_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad | 0 ...iew_as_select-2-1ac845048a8c714a36a719ea8e4f570b | 0 ...iew_as_select-3-9280ae6c369a9f30d3d021d00e435f01 | 0 ...iew_as_select-4-1ac845048a8c714a36a719ea8e4f570b | 0 ...iew_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 | 0 ...iew_as_select-6-1ac845048a8c714a36a719ea8e4f570b | 0 ...r_view_rename-0-bb255b994b5207324fba6988caa937e6 | 0 ...r_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d | 0 ...ter_view_rename-2-67e47ee2746463594d5c48b10ba1bb | 0 ...r_view_rename-3-95655e33f22fc8f66549a9708812589a | 0 ...r_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 | 0 ...ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d | 0 ...ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab | 0 ...ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b | 0 .../archive-0-89cd75b0565e8d96910d5528db9984e7 | 0 .../archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 .../archive-10-f845008104fd12eb0f13f4b113f95cf1 | 0 .../archive-11-27895cbe0ee6d24d7fc866314491e1bb | 0 .../archive-12-f07653bca86e1ecb614ffd0296790d05 | 0 .../archive-13-27895cbe0ee6d24d7fc866314491e1bb | 0 .../archive-14-2cde1a2d3bfcec814985f498eba0fb8 | 0 .../archive-15-c9f39b388ae698e385e092d0ffeb3c73 | 0 .../archive-16-892147913578bcf60620b7dd73893dd0 | 0 .../archive-17-27895cbe0ee6d24d7fc866314491e1bb | 0 .../archive-18-8f980275ab3a5bcfc1784f4acd46447a | 0 .../archive-19-f8a52a8a40141409a667a9ba2cf9630f | 0 .../archive-2-713efc113418b01f76ffd589840193c8 | 0 .../archive-20-530277b0fee8b05c37b26846bceef827 | 0 .../archive-21-f07653bca86e1ecb614ffd0296790d05 | 0 .../archive-22-530277b0fee8b05c37b26846bceef827 | 0 .../archive-23-892147913578bcf60620b7dd73893dd0 | 0 .../archive-24-530277b0fee8b05c37b26846bceef827 | 0 .../archive-25-56bea24ffa83d9b5932893a8ff1cb44f | 0 .../archive-26-44fa61c2bd0fd9acfa0d889e59880d8a | 0 .../archive-27-9ae773ebe64a3d437a035e9d94f49e5 | 0 .../archive-28-188eb7912265ed8dffa5200517bbe526 | 0 .../archive-29-a0c6922e3a1dca51861c8a872dc8af19 | 0 .../archive-3-27ad2962fed131f51ba802596ba37278 | 0 .../archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a | 0 .../archive-31-cd46bc635e3010cf1b990a652a584a09 | 0 .../archive-32-27ad2962fed131f51ba802596ba37278 | 0 .../archive-4-3e95421993ab28d18245ec2340f580a3 | 0 .../archive-5-c0c18ac884677231a41eea8d980d0451 | 0 .../archive-6-528ab9750a558af7f1a43b3108e793dd | 0 .../archive-7-e8d1d10c308a73eef78dde414a5e40ca | 0 .../archive-8-af459a0264559a2aeaa1341ce779ab3c | 0 .../archive-9-48b10f27e1459bb8e62d6c71484e2cf | 0 ...cludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 | 0 ...cludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...ludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 | 0 ...ludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb | 0 ...ludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 | 0 ...ludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb | 0 ...cludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 | 0 ...ludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 | 0 ...ludeHadoop20-16-892147913578bcf60620b7dd73893dd0 | 0 ...ludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb | 0 ...ludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a | 0 ...ludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f | 0 ...cludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 | 0 ...ludeHadoop20-20-530277b0fee8b05c37b26846bceef827 | 0 ...ludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 | 0 ...ludeHadoop20-22-530277b0fee8b05c37b26846bceef827 | 0 ...ludeHadoop20-23-892147913578bcf60620b7dd73893dd0 | 0 ...ludeHadoop20-24-530277b0fee8b05c37b26846bceef827 | 0 ...ludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f | 0 ...ludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a | 0 ...cludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 | 0 ...ludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 | 0 ...ludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 | 0 ...cludeHadoop20-3-27ad2962fed131f51ba802596ba37278 | 0 ...ludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a | 0 ...ludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 | 0 ...ludeHadoop20-32-27ad2962fed131f51ba802596ba37278 | 0 ...cludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 | 0 ...cludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 | 0 ...cludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd | 0 ...cludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca | 0 ...cludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c | 0 ...xcludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf | 0 ...archive_multi-0-89cd75b0565e8d96910d5528db9984e7 | 0 ...archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...rchive_multi-10-f845008104fd12eb0f13f4b113f95cf1 | 0 ...rchive_multi-11-cf5431cd843666b95ad2a82b334ac01e | 0 ...rchive_multi-12-8419ad4ed6683ebd15f993f703975b31 | 0 ...rchive_multi-13-27895cbe0ee6d24d7fc866314491e1bb | 0 ...archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 | 0 ...rchive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 | 0 ...rchive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d | 0 ...rchive_multi-17-27895cbe0ee6d24d7fc866314491e1bb | 0 ...archive_multi-2-cd46bc635e3010cf1b990a652a584a09 | 0 ...archive_multi-3-27ad2962fed131f51ba802596ba37278 | 0 ...archive_multi-4-3e95421993ab28d18245ec2340f580a3 | 0 ...archive_multi-5-c0c18ac884677231a41eea8d980d0451 | 0 ...archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a | 0 ...archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca | 0 ...archive_multi-8-af459a0264559a2aeaa1341ce779ab3c | 0 .../archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf | 0 .../golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba | 0 .../auto_join0-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join0-1-383f34dec3ac939b7af2c9093a557641 | 0 .../auto_join0-2-7bd04899197b027d81c24e45a99ad15c | 0 .../auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a | 0 .../auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e | 0 .../auto_join1-2-8a9624554e208e3d8fbe42908c715b92 | 0 .../auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 | 0 .../auto_join1-4-ae1247a065c41ce0329ca6078ab586e | 0 .../auto_join10-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 | 0 .../auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 | 0 .../auto_join11-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 | 0 .../auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 | 0 .../auto_join12-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 | 0 .../auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 | 0 .../auto_join13-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 | 0 .../auto_join13-2-5ff417533a1243cd6fc556960fa170c9 | 0 .../auto_join14-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 | 0 .../auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e | 0 .../auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 | 0 .../auto_join14-4-bab89dfffa77258e34a595e0e79986e3 | 0 .../auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 | 0 .../auto_join14-6-de39302191b63d7aa8f92885b089fe2 | 0 .../auto_join14-7-5b5ded1412301eae5f8f705a39e6832 | 0 ...in14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a | 0 ...in14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 | 0 ...in14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f | 0 ...in14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 | 0 ...in14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 | 0 ...oin14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 | 0 ...oin14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 | 0 .../auto_join15-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 | 0 .../auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 | 0 .../auto_join16-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf | 0 .../auto_join16-2-66e56dcda38eb09819ac49e47e40d125 | 0 .../auto_join17-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join17-1-387dd86b1e13f788ec677a08dc162c97 | 0 .../auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 | 0 .../auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 | 0 .../auto_join17-4-11d706a64d44a8b0d41b290c4671c29c | 0 .../auto_join18-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b | 0 .../auto_join18-2-f633ade9577c8b0e89d89124194c8d0f | 0 .../auto_join19-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 | 0 .../auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 | 0 .../auto_join19-3-a3751c195480244a5ed497fd053cd433 | 0 .../auto_join19-4-eaa70da463b92e85e1796277f016c18f | 0 .../auto_join2-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join2-1-3d1692c4710db1ff716d35e921f2bcca | 0 .../auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 | 0 .../auto_join2-3-d4673c03d04084b838fcd8149f59ad9a | 0 .../auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 | 0 .../auto_join20-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join20-1-2afb0510178c4b66876dd91c7ca441fd | 0 .../auto_join20-2-903ee25e327188edaaf2040fec5a8e52 | 0 .../auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee | 0 .../auto_join20-4-e48e08d5b94719d56a58284eaef757f2 | 0 .../auto_join21-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join21-1-9dd59784ca1555b607df0137d2666fb8 | 0 .../auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f | 0 .../auto_join22-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 | 0 .../auto_join22-2-a4abc288c20edee53ede45d248cf3abb | 0 .../auto_join23-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join23-1-b31437533a2b890788938455cb32f679 | 0 .../auto_join23-2-6d2c5b58222f31658a0cf957e093a150 | 0 .../auto_join24-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join24-1-721dfa03bfea05e55506c571b6c3585b | 0 .../auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d | 0 .../auto_join24-3-fa8b2736440ff35687dadb1bcae32666 | 0 .../auto_join24-4-d79325ef6494aa87843fdfd78de7c812 | 0 .../auto_join25-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 | 0 .../auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e | 0 .../auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 | 0 .../auto_join25-12-ae1247a065c41ce0329ca6078ab586e | 0 .../auto_join25-2-8180638a57b64557e02815c863031755 | 0 .../auto_join25-3-9aa914a687f1f63faf48eb500627855e | 0 .../auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 | 0 .../auto_join25-5-a3751c195480244a5ed497fd053cd433 | 0 .../auto_join25-6-eaa70da463b92e85e1796277f016c18f | 0 .../auto_join25-7-3d1692c4710db1ff716d35e921f2bcca | 0 .../auto_join25-8-d4673c03d04084b838fcd8149f59ad9a | 0 .../auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 | 0 .../auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 | 0 .../auto_join26-1-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 | 0 .../auto_join26-3-fed383a65bd118b43de6b00be10fecb6 | 0 .../auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../auto_join27-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join27-1-c83f56f364b1da3def90d48953665fe5 | 0 .../auto_join27-2-fceaa1ebd63334061d2d8daf961e935e | 0 .../auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c | 0 .../auto_join28-1-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join28-2-9dd59784ca1555b607df0137d2666fb8 | 0 .../auto_join28-3-30739ff22c62b3becf56694642b7ae81 | 0 .../auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 | 0 .../auto_join28-5-142850e84341feb3f7f40dd4553f72e | 0 .../auto_join3-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 | 0 .../auto_join3-2-6bed7d8089695e23914b29edaab2537d | 0 .../auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 | 0 .../auto_join3-4-eaa70da463b92e85e1796277f016c18f | 0 .../auto_join30-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 | 0 .../auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a | 0 .../auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 | 0 .../auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d | 0 .../auto_join30-13-9ee597656aa92e48475d6542339915ba | 0 .../auto_join30-14-7a248488c218919ab50e072fdbdecb73 | 0 .../auto_join30-15-47b7efabbd6046e2befcbbea7da62553 | 0 .../auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 | 0 .../auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 | 0 .../auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 | 0 .../auto_join30-4-f5083eca9c3df277988d8b345b8d43 | 0 .../auto_join30-5-3916f4b640f3579035153f6940113ef2 | 0 .../auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 | 0 .../auto_join30-7-f07b674c31ca9fdf837406cb9a96108e | 0 .../auto_join30-8-8a27209399df7f9c4d15988b11753a61 | 0 .../auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 | 0 .../auto_join31-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 | 0 .../auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 | 0 .../auto_join32-0-24ca942f094b14b92086305cc125e833 | 0 .../auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 | 0 .../auto_join32-10-2e8ccb343bce61564bae209a589cca85 | 0 .../auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 | 0 .../auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 | 0 .../auto_join32-13-63241e3791725baad8baa00fb833ef5e | 0 .../auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f | 0 .../auto_join32-15-187fd938996ae7d96e60475fb69b8d35 | 0 .../auto_join32-16-2e8ccb343bce61564bae209a589cca85 | 0 .../auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 | 0 .../auto_join32-18-d1d78b19d484e55d9da8a320253ece0f | 0 .../auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 | 0 .../auto_join32-2-865207407ff1acbccb47473d87e87e8d | 0 .../auto_join32-20-e67740fb52998f1d3afcfa667505cf7 | 0 .../auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b | 0 .../auto_join32-22-a1d339a0d904c3f35771192a026c7f9c | 0 .../auto_join32-23-1948951cc3c06cdf962d59e932a84588 | 0 .../auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 | 0 .../auto_join32-25-e46226186de575c81cfab296607e1b4b | 0 .../auto_join32-26-97d265cd7defca44e488c38bac4c5b7a | 0 .../auto_join32-27-b034eeb850810b5004ddff1f2a530bc | 0 .../auto_join32-28-751550ac0550e6a7dd737cad01d6d82 | 0 .../auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 | 0 .../auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 | 0 .../auto_join32-30-d3903985844b06c4af11334b72f383d1 | 0 .../auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd | 0 .../auto_join32-4-a28f563122d1f0debd04f74c534523cf | 0 .../auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed | 0 .../auto_join32-6-442e69416faaea9309bb8c2a3eb73ef | 0 .../auto_join32-7-d0ec6d66ff349db09fd455eec149efdb | 0 .../auto_join32-8-999683fa0291bf439b03557edec7dcee | 0 .../auto_join32-9-1e6d4ec86f29d74828891c17986e84a | 0 .../auto_join4-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join4-1-531e91e11b3891627c2675935fda14cd | 0 .../auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a | 0 .../auto_join4-3-dc967001beb776f3a859e9360823c361 | 0 .../auto_join4-4-998c3a307b074a6505bb7fcef276be04 | 0 .../auto_join5-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join5-1-531e91e11b3891627c2675935fda14cd | 0 .../auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 | 0 .../auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 | 0 .../auto_join5-4-998c3a307b074a6505bb7fcef276be04 | 0 .../auto_join6-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join6-1-531e91e11b3891627c2675935fda14cd | 0 .../auto_join6-2-46718fdfa123cc86fe288bff4185dc90 | 0 .../auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 | 0 .../auto_join6-4-998c3a307b074a6505bb7fcef276be04 | 0 .../auto_join7-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join7-1-8f82881057bec4abf5a4d770a6f35838 | 0 .../auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e | 0 .../auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 | 0 .../auto_join7-4-30d0c1a49784347fedbac21a69c3a899 | 0 .../auto_join8-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join8-1-531e91e11b3891627c2675935fda14cd | 0 .../auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab | 0 .../auto_join8-3-f1196bca86a749375da35f134206a8ca | 0 .../auto_join8-4-998c3a307b074a6505bb7fcef276be04 | 0 .../auto_join9-0-ce1ef910fff98f174931cc641f7cef3a | 0 .../auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 | 0 .../auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 | 0 .../auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 | 0 .../auto_join9-4-eaa70da463b92e85e1796277f016c18f | 0 ..._join_filters-0-ce1ef910fff98f174931cc641f7cef3a | 0 ..._join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 | 0 ...join_filters-10-b420f24d33b26cdf6c35eb702789904e | 0 ...join_filters-11-c06b548171893bae8def6bb348b70dc8 | 0 ...join_filters-12-42a4901e05e9ee92abcfcef008efaa65 | 0 ...join_filters-13-222c404c6265ed682579342113221e29 | 0 ...join_filters-14-ecd2885156f56973960d064211ee42f0 | 0 ...join_filters-15-11a471880f5e8fbad81e3869fe56ca4b | 0 ...join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e | 0 ...join_filters-17-8e085f9886e5ee97334512f84bd7ab54 | 0 ...join_filters-18-d9438071b3c731dc3f6e3b7248a1042a | 0 ...join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d | 0 ..._join_filters-2-100faa7fd01bfb4390c782bb262a9403 | 0 ...join_filters-20-486a302359aecff37a4567480264bd62 | 0 ..._join_filters-21-8018df5b9572e89304b449e618fdbbf | 0 ...join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 | 0 ...join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 | 0 ...join_filters-24-a2161a58f512b8bdd836b48cd8e16668 | 0 ...join_filters-25-1b3177a066ba352539fd5473fbeda1a9 | 0 ...join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd | 0 ...join_filters-27-ca73726ffaa5826a3db039cda440e6d9 | 0 ..._join_filters-28-c21c2acf7f276c0a26f0c19e3234506 | 0 ...join_filters-29-6d8955591f62d9cfc6af17df63d3d88e | 0 ..._join_filters-3-64615cc6839c697c8c028ef8bb1ac40e | 0 ...join_filters-30-23ab7ac8229a53d391195be7ca092429 | 0 ...join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b | 0 ...join_filters-32-3c52df82c7d78501610f3f898103f753 | 0 ...join_filters-33-1d85bb008e02ef4025171a4bc0866a6c | 0 ...o_join_filters-34-e79c906b894fed049ddfab4496a4e3 | 0 ...join_filters-35-3e6612a89e9124592e790594775054b1 | 0 ...join_filters-36-60a5f56f33fc8854a2b687005f0d96ac | 0 ...join_filters-37-64cabe5164130a94f387288f37b62d71 | 0 ...join_filters-38-65c867e66bc773470f1487487086a180 | 0 ...join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada | 0 ..._join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 | 0 ..._join_filters-40-4dfa64337f711dc394a289adeac8666 | 0 ...join_filters-41-a905633bccd782f14115643b2d707b13 | 0 ..._join_filters-42-ae4cc72ddbbbd748179e0abcc985726 | 0 ...join_filters-43-f64cec45b154c4ba5172afcdff623a2b | 0 ...join_filters-44-c552dcc9b931dff05cf8c0d712e22841 | 0 ...join_filters-45-b420f24d33b26cdf6c35eb702789904e | 0 ...join_filters-46-c06b548171893bae8def6bb348b70dc8 | 0 ...join_filters-47-42a4901e05e9ee92abcfcef008efaa65 | 0 ...join_filters-48-222c404c6265ed682579342113221e29 | 0 ...join_filters-49-ecd2885156f56973960d064211ee42f0 | 0 ..._join_filters-5-87cf8865e2c35b680bba159b88b074bc | 0 ...join_filters-50-11a471880f5e8fbad81e3869fe56ca4b | 0 ...join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e | 0 ...join_filters-52-8e085f9886e5ee97334512f84bd7ab54 | 0 ...join_filters-53-d9438071b3c731dc3f6e3b7248a1042a | 0 ...join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d | 0 ...join_filters-55-486a302359aecff37a4567480264bd62 | 0 ..._join_filters-56-8018df5b9572e89304b449e618fdbbf | 0 ...join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 | 0 ...join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 | 0 ...join_filters-59-a2161a58f512b8bdd836b48cd8e16668 | 0 ..._join_filters-6-9362466c777fff7e677dd8da072f8744 | 0 ...join_filters-60-1b3177a066ba352539fd5473fbeda1a9 | 0 ...join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd | 0 ...join_filters-62-ca73726ffaa5826a3db039cda440e6d9 | 0 ..._join_filters-63-c21c2acf7f276c0a26f0c19e3234506 | 0 ...o_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 | 0 ..._join_filters-8-f64cec45b154c4ba5172afcdff623a2b | 0 ..._join_filters-9-c552dcc9b931dff05cf8c0d712e22841 | 0 ...to_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a | 0 ...to_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 | 0 ...o_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 | 0 ...o_join_nulls-11-141c550a314d15c4e200e5baeb246de2 | 0 ...o_join_nulls-12-8a65225a88da0169af26848c06cb981c | 0 ...o_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 | 0 ...o_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 | 0 ...o_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 | 0 ...o_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af | 0 ...o_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 | 0 ...o_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb | 0 ...o_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 | 0 ...to_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 | 0 ...o_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d | 0 ...o_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a | 0 ...o_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 | 0 ...o_join_nulls-23-5805a41289a26139c06604a40bf5a6fa | 0 ...o_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 | 0 ...to_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 | 0 ...uto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 | 0 ...to_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc | 0 ...to_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 | 0 ...to_join_nulls-7-8395fa78507105c2a018e88f717b95e2 | 0 ...to_join_nulls-8-fd992f2127a139aeb554d797e748ed54 | 0 ...to_join_nulls-9-e3a86622a437e910b7225d1e6108da9e | 0 ...dering_values-0-1d8e3d660bb4b29287df5700bfe63b63 | 0 ...dering_values-1-1247d9a5ffabd61647697dc186c7a2a2 | 0 ...ering_values-10-f6d5bb38137da35b91da901ba310c2b8 | 0 ...dering_values-2-3ce329282fc72110e9ed6c78fa914395 | 0 ...dering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 | 0 ...dering_values-4-7fb82039c95389f11b174d9f22aacb35 | 0 ...dering_values-5-b5da89b0cb325cf684406b620eb9d8ee | 0 ...dering_values-6-2c91dc4b7c00d5f09862119c12295532 | 0 ...dering_values-7-880ba1dba6057dd6cde89d1b17724a6b | 0 ...dering_values-8-950af86c321a67ab3ed0fa5b63ea6aed | 0 ...dering_values-9-ae02756bd46266ec7fd9cc809bc4757b | 0 ...mb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...mb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...b_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb | 0 ...b_mapjoin_14-11-906a582602602372e1d4776243abeab5 | 0 ...b_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 | 0 ...b_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 | 0 ...b_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 | 0 ...b_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 | 0 ...b_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 | 0 ...b_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 | 0 ...b_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 | 0 ...b_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 | 0 ...smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 | 0 ...b_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 | 0 ...b_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 | 0 ...b_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a | 0 ...b_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 | 0 ...b_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 | 0 ...b_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 | 0 ...b_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 | 0 ...b_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 | 0 ...b_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 | 0 ...b_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d | 0 ...mb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da | 0 ...b_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce | 0 ...b_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce | 0 ...b_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 | 0 ...b_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 | 0 ...mb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b | 0 ...b_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 | 0 ...b_mapjoin_14-36-1ba279881865c861a793797ae84a3934 | 0 ...b_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e | 0 ...b_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 | 0 ...b_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da | 0 ...mb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...b_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e | 0 ...b_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c | 0 ...b_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c | 0 ...b_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e | 0 ...b_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 | 0 ...mb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f | 0 ...mb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f | 0 ...mb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 | 0 ...mb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f | 0 ...tmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 | 0 ...rtmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...merge_join_1-10-9666fb18356436e2800550df9ca90c04 | 0 ...merge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 | 0 ...merge_join_1-12-24ca942f094b14b92086305cc125e833 | 0 ...merge_join_1-13-d0ec6d66ff349db09fd455eec149efdb | 0 ...merge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f | 0 ...merge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 | 0 ...merge_join_1-16-cda81d86d127fca0e2fbc2161e91400d | 0 ...merge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc | 0 ...merge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...merge_join_1-19-325432a220aa3ebe8b816069916924d8 | 0 ...tmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...merge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...merge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 | 0 ...merge_join_1-22-f135547e33c01d1f543c8b1349d60348 | 0 ...merge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...merge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 | 0 ...tmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db | 0 ...tmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a | 0 ...tmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab | 0 ...tmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 | 0 ...tmerge_join_1-7-94cc219f61413ab321916821e1288152 | 0 ...tmerge_join_1-8-310c8d652c6f549b7759baec6012b77d | 0 ...tmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...merge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...merge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...erge_join_10-10-b89ea2173180c8ae423d856f943e061f | 0 ...erge_join_10-11-d0ec6d66ff349db09fd455eec149efdb | 0 ...erge_join_10-12-cda81d86d127fca0e2fbc2161e91400d | 0 ...erge_join_10-13-469a09efa93fa9aec154a5967eec09c5 | 0 ...erge_join_10-14-95e18bd00f2de246efca1756681c1e87 | 0 ...merge_join_10-15-6a7fdb423721e7aefa2efda26785e1a | 0 ...merge_join_10-16-caa641c820fcc5f601758c5f0385b4e | 0 ...tmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 | 0 ...merge_join_10-3-4938d4d724990d16336ee31f0390c7da | 0 ...merge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...merge_join_10-5-9140b367b5680860f4c7c0238377583f | 0 ...merge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f | 0 ...merge_join_10-7-24ca942f094b14b92086305cc125e833 | 0 ...merge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f | 0 ...merge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 | 0 ...merge_join_11-0-4705fafa08c6d927aa01337e19605c8a | 0 ...tmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...erge_join_11-10-9666fb18356436e2800550df9ca90c04 | 0 ...erge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 | 0 ...erge_join_11-12-24ca942f094b14b92086305cc125e833 | 0 ...erge_join_11-13-398b81a1928284f29e832838ec3764fd | 0 ...erge_join_11-14-325432a220aa3ebe8b816069916924d8 | 0 ...erge_join_11-15-d0ec6d66ff349db09fd455eec149efdb | 0 ...erge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed | 0 ...merge_join_11-17-442e69416faaea9309bb8c2a3eb73ef | 0 ...erge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e | 0 ...erge_join_11-19-325432a220aa3ebe8b816069916924d8 | 0 ...merge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...erge_join_11-20-62fab16c00f510c001f146c929360c71 | 0 ...erge_join_11-21-c4d55c247c9326f474d89b29b81d60aa | 0 ...merge_join_11-3-b4a6a67ac771394140ed695810930ac6 | 0 ...merge_join_11-4-bb969d3ec0038215a2698afceeb02b3a | 0 ...merge_join_11-5-2c3617157639fcd296a8ea2f121c58ab | 0 ...merge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 | 0 ...merge_join_11-7-94cc219f61413ab321916821e1288152 | 0 ...merge_join_11-8-310c8d652c6f549b7759baec6012b77d | 0 ...merge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...merge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec | 0 ...tmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...erge_join_12-10-9666fb18356436e2800550df9ca90c04 | 0 ...erge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 | 0 ...erge_join_12-12-24ca942f094b14b92086305cc125e833 | 0 ...erge_join_12-13-d0ec6d66ff349db09fd455eec149efdb | 0 ...erge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f | 0 ...erge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 | 0 ...erge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e | 0 ...erge_join_12-17-a8c60901367229310f86a8521a26478a | 0 ...erge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 | 0 ...erge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 | 0 ...merge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...erge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 | 0 ...erge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc | 0 ...merge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db | 0 ...merge_join_12-4-bb969d3ec0038215a2698afceeb02b3a | 0 ...merge_join_12-5-2c3617157639fcd296a8ea2f121c58ab | 0 ...merge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 | 0 ...merge_join_12-7-94cc219f61413ab321916821e1288152 | 0 ...merge_join_12-8-310c8d652c6f549b7759baec6012b77d | 0 ...merge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...merge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...merge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...erge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab | 0 ...erge_join_13-11-cda81d86d127fca0e2fbc2161e91400d | 0 ...erge_join_13-12-d0ec6d66ff349db09fd455eec149efdb | 0 ...erge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f | 0 ...erge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 | 0 ...erge_join_13-15-24ca942f094b14b92086305cc125e833 | 0 ...erge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 | 0 ...erge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 | 0 ...erge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 | 0 ...erge_join_13-19-4b2ac2865384fbca7f374191d8021d51 | 0 ...tmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 | 0 ...erge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 | 0 ...erge_join_13-21-ea23403b9eb55e8b06d1c198e439569f | 0 ...erge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 | 0 ...erge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 | 0 ...erge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 | 0 ...erge_join_13-25-4b2ac2865384fbca7f374191d8021d51 | 0 ...erge_join_13-26-f135547e33c01d1f543c8b1349d60348 | 0 ...erge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 | 0 ...erge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 | 0 ...erge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 | 0 ...merge_join_13-3-4938d4d724990d16336ee31f0390c7da | 0 ...erge_join_13-30-4b2ac2865384fbca7f374191d8021d51 | 0 ...merge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...merge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 | 0 ...merge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 | 0 ...merge_join_13-7-6f949602369ac3af6ded9884bc525310 | 0 ...tmerge_join_13-8-b334c03af5acdcb136072bb427683bb | 0 ...merge_join_13-9-dc129f70e75cd575ce8c0de288884523 | 0 ...merge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...merge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...erge_join_14-10-d0ec6d66ff349db09fd455eec149efdb | 0 ...erge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f | 0 ...erge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 | 0 ...erge_join_14-13-24ca942f094b14b92086305cc125e833 | 0 ...erge_join_14-14-51e4c81f56c64f6aa25322055694f641 | 0 ...erge_join_14-15-43ad2152b18d711adbdd1aeb940b662a | 0 ...erge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c | 0 ...merge_join_14-17-334529f1a720bfb408efee90bc8be61 | 0 ...erge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd | 0 ...tmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 | 0 ...merge_join_14-3-4938d4d724990d16336ee31f0390c7da | 0 ...merge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...merge_join_14-5-945b37381c2719e18e2945bf8b4e56ac | 0 ...merge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 | 0 ...merge_join_14-7-dc129f70e75cd575ce8c0de288884523 | 0 ...merge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab | 0 ...merge_join_14-9-f135547e33c01d1f543c8b1349d60348 | 0 ...merge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...merge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...erge_join_15-10-d0ec6d66ff349db09fd455eec149efdb | 0 ...erge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f | 0 ...erge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 | 0 ...erge_join_15-13-24ca942f094b14b92086305cc125e833 | 0 ...erge_join_15-14-62b7e43463386c11e031cf7e4f584a53 | 0 ...erge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 | 0 ...tmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 | 0 ...merge_join_15-3-4938d4d724990d16336ee31f0390c7da | 0 ...merge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...merge_join_15-5-945b37381c2719e18e2945bf8b4e56ac | 0 ...merge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 | 0 ...merge_join_15-7-dc129f70e75cd575ce8c0de288884523 | 0 ...merge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab | 0 ...merge_join_15-9-f135547e33c01d1f543c8b1349d60348 | 0 ...tmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d | 0 ...rtmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...merge_join_2-10-24ca942f094b14b92086305cc125e833 | 0 ...merge_join_2-11-d0ec6d66ff349db09fd455eec149efdb | 0 ...merge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f | 0 ...merge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 | 0 ...merge_join_2-14-cda81d86d127fca0e2fbc2161e91400d | 0 ...merge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 | 0 ...merge_join_2-16-af6016f3db000e6e180e2f3b10f120ce | 0 ...merge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 | 0 ...merge_join_2-18-f135547e33c01d1f543c8b1349d60348 | 0 ...merge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 | 0 ...tmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...merge_join_2-20-8180638a57b64557e02815c863031755 | 0 ...merge_join_2-21-e6283ea14d493b0d7bf390249665f289 | 0 ...merge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 | 0 ...tmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a | 0 ...tmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 | 0 ...tmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 | 0 ...tmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a | 0 ...tmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab | 0 ...tmerge_join_2-8-310c8d652c6f549b7759baec6012b77d | 0 ...tmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...tmerge_join_3-0-71378da1900d130fd68aaebc45f87313 | 0 ...rtmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...merge_join_3-10-24ca942f094b14b92086305cc125e833 | 0 ...merge_join_3-11-d0ec6d66ff349db09fd455eec149efdb | 0 ...merge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f | 0 ...merge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 | 0 ...merge_join_3-14-cda81d86d127fca0e2fbc2161e91400d | 0 ...merge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc | 0 ...merge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...merge_join_3-17-325432a220aa3ebe8b816069916924d8 | 0 ...merge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...merge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 | 0 ...tmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...merge_join_3-20-f135547e33c01d1f543c8b1349d60348 | 0 ...merge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...merge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 | 0 ...tmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 | 0 ...tmerge_join_3-4-cd25b8502c668759783aaba4d550a05f | 0 ...tmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db | 0 ...tmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a | 0 ...tmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab | 0 ...tmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 | 0 ...tmerge_join_3-9-94cc219f61413ab321916821e1288152 | 0 ...tmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c | 0 ...rtmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...merge_join_4-10-bb969d3ec0038215a2698afceeb02b3a | 0 ...merge_join_4-11-2c3617157639fcd296a8ea2f121c58ab | 0 ...merge_join_4-12-24ca942f094b14b92086305cc125e833 | 0 ...merge_join_4-13-d0ec6d66ff349db09fd455eec149efdb | 0 ...merge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f | 0 ...merge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 | 0 ...merge_join_4-16-cda81d86d127fca0e2fbc2161e91400d | 0 ...merge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 | 0 ...merge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...merge_join_4-19-325432a220aa3ebe8b816069916924d8 | 0 ...tmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...merge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...merge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 | 0 ...merge_join_4-22-f135547e33c01d1f543c8b1349d60348 | 0 ...merge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...merge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 | 0 ...tmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a | 0 ...tmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 | 0 ...tmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 | 0 ...tmerge_join_4-6-cd25b8502c668759783aaba4d550a05f | 0 ...tmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 | 0 ...tmerge_join_4-8-baa1253610c081917208199feb52a768 | 0 ...tmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 | 0 ...tmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d | 0 ...tmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 | 0 ...merge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 | 0 ...merge_join_5-11-cda81d86d127fca0e2fbc2161e91400d | 0 ...merge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 | 0 ...merge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...merge_join_5-14-325432a220aa3ebe8b816069916924d8 | 0 ...merge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...merge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 | 0 ...merge_join_5-17-f135547e33c01d1f543c8b1349d60348 | 0 ...merge_join_5-18-24ca942f094b14b92086305cc125e833 | 0 ...merge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...tmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 | 0 ...merge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 | 0 ...tmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c | 0 ...tmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 | 0 ...tmerge_join_5-5-ce5ee903a36a074293fa509149d94447 | 0 ...tmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 | 0 ...tmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a | 0 ...tmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb | 0 ...tmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f | 0 ...tmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...tmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...merge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 | 0 ...merge_join_6-11-d0ec6d66ff349db09fd455eec149efdb | 0 ...merge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f | 0 ...merge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 | 0 ...merge_join_6-14-24ca942f094b14b92086305cc125e833 | 0 ...merge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 | 0 ...merge_join_6-16-ea23403b9eb55e8b06d1c198e439569f | 0 ...merge_join_6-17-cda81d86d127fca0e2fbc2161e91400d | 0 ...tmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 | 0 ...merge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 | 0 ...rtmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 | 0 ...merge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd | 0 ...merge_join_6-21-b55506a213ec710004e6d7f3462834d0 | 0 ...merge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c | 0 ...merge_join_6-23-4281442c87dcf6007f8bd42504eba186 | 0 ...merge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f | 0 ...merge_join_6-25-52f0e65724d29e2b4054b59a50d2837b | 0 ...merge_join_6-26-bf7478a041a164ef219964cb865aa63b | 0 ...merge_join_6-27-961f141836f2cc9521f681cadbc3d140 | 0 ...tmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 | 0 ...merge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 | 0 ...tmerge_join_6-3-4938d4d724990d16336ee31f0390c7da | 0 ...merge_join_6-30-f135547e33c01d1f543c8b1349d60348 | 0 ...merge_join_6-31-940f423a57afd2734f62d93bcd4d7caf | 0 ...merge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 | 0 ...merge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f | 0 ...merge_join_6-34-52f0e65724d29e2b4054b59a50d2837b | 0 ...merge_join_6-35-bf7478a041a164ef219964cb865aa63b | 0 ...merge_join_6-36-961f141836f2cc9521f681cadbc3d140 | 0 ...tmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 | 0 ...merge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 | 0 ...tmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...tmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 | 0 ...tmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 | 0 ...tmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 | 0 ...tmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 | 0 ...tmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b | 0 ...rtmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 | 0 ...rtmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...merge_join_7-10-bb969d3ec0038215a2698afceeb02b3a | 0 ...merge_join_7-11-2c3617157639fcd296a8ea2f121c58ab | 0 ...merge_join_7-12-310c8d652c6f549b7759baec6012b77d | 0 ...merge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...merge_join_7-14-24ca942f094b14b92086305cc125e833 | 0 ...merge_join_7-15-d0ec6d66ff349db09fd455eec149efdb | 0 ...merge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f | 0 ...merge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 | 0 ...merge_join_7-18-cda81d86d127fca0e2fbc2161e91400d | 0 ...merge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 | 0 ...tmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...merge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...merge_join_7-21-325432a220aa3ebe8b816069916924d8 | 0 ...merge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...merge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 | 0 ...merge_join_7-24-f135547e33c01d1f543c8b1349d60348 | 0 ...merge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...merge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 | 0 ...tmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a | 0 ...tmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 | 0 ...tmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 | 0 ...tmerge_join_7-6-cd25b8502c668759783aaba4d550a05f | 0 ...tmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 | 0 ...tmerge_join_7-8-baa1253610c081917208199feb52a768 | 0 ...tmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 | 0 ...tmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa | 0 ...rtmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 | 0 ...merge_join_8-10-310c8d652c6f549b7759baec6012b77d | 0 ...merge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 | 0 ...merge_join_8-12-9666fb18356436e2800550df9ca90c04 | 0 ...merge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 | 0 ...merge_join_8-14-24ca942f094b14b92086305cc125e833 | 0 ...merge_join_8-15-d0ec6d66ff349db09fd455eec149efdb | 0 ...merge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f | 0 ...merge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 | 0 ...merge_join_8-18-cda81d86d127fca0e2fbc2161e91400d | 0 ...merge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc | 0 ...tmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 | 0 ...merge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 | 0 ...merge_join_8-21-325432a220aa3ebe8b816069916924d8 | 0 ...merge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 | 0 ...merge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 | 0 ...merge_join_8-24-f135547e33c01d1f543c8b1349d60348 | 0 ...merge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 | 0 ...merge_join_8-26-8180638a57b64557e02815c863031755 | 0 ...merge_join_8-27-e6283ea14d493b0d7bf390249665f289 | 0 ...merge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 | 0 ...tmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 | 0 ...tmerge_join_8-4-cd25b8502c668759783aaba4d550a05f | 0 ...tmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db | 0 ...tmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a | 0 ...tmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab | 0 ...tmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 | 0 ...tmerge_join_8-9-94cc219f61413ab321916821e1288152 | 0 ...tmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...tmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...merge_join_9-10-d0ec6d66ff349db09fd455eec149efdb | 0 ...merge_join_9-11-cda81d86d127fca0e2fbc2161e91400d | 0 ...merge_join_9-12-906a582602602372e1d4776243abeab5 | 0 ...merge_join_9-13-94538bc7322522a5534cafc0551d2189 | 0 ...merge_join_9-14-133023474337f2cdc53ee82ffeb1c13e | 0 ...merge_join_9-15-63261d35ddda973eeeb97b994ab7a476 | 0 ...merge_join_9-16-1d603e61c2cb888499504ddab98ccc65 | 0 ...merge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 | 0 ...merge_join_9-18-758d5532083d6279e169b54fd69bb580 | 0 ...merge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 | 0 ...rtmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 | 0 ...merge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 | 0 ...merge_join_9-21-21269869cd3aaf4ade2170d9017de018 | 0 ...merge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 | 0 ...merge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 | 0 ...merge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 | 0 ...merge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a | 0 ...merge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 | 0 ...merge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 | 0 ...tmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 | 0 ...merge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 | 0 ...tmerge_join_9-3-4938d4d724990d16336ee31f0390c7da | 0 ...merge_join_9-30-4376bdd8412f94fe184d46481fee345d | 0 ...merge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 | 0 ...merge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 | 0 ...merge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e | 0 ...merge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d | 0 ...merge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce | 0 ...merge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce | 0 ...merge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 | 0 ...merge_join_9-38-f135547e33c01d1f543c8b1349d60348 | 0 ...merge_join_9-39-906a582602602372e1d4776243abeab5 | 0 ...tmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...merge_join_9-40-94538bc7322522a5534cafc0551d2189 | 0 ...merge_join_9-41-133023474337f2cdc53ee82ffeb1c13e | 0 ...merge_join_9-42-63261d35ddda973eeeb97b994ab7a476 | 0 ...merge_join_9-43-1d603e61c2cb888499504ddab98ccc65 | 0 ...merge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 | 0 ...merge_join_9-45-758d5532083d6279e169b54fd69bb580 | 0 ...merge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 | 0 ...merge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 | 0 ...merge_join_9-48-21269869cd3aaf4ade2170d9017de018 | 0 ...merge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 | 0 ...tmerge_join_9-5-9140b367b5680860f4c7c0238377583f | 0 ...merge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 | 0 ...merge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 | 0 ...merge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a | 0 ...merge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 | 0 ...merge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 | 0 ...merge_join_9-55-4376bdd8412f94fe184d46481fee345d | 0 ...merge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 | 0 ...merge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 | 0 ...merge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e | 0 ...merge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d | 0 ...tmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f | 0 ...merge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce | 0 ...merge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce | 0 ...merge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 | 0 ...tmerge_join_9-7-24ca942f094b14b92086305cc125e833 | 0 ...tmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f | 0 ...tmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 | 0 .../ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 .../ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 | 0 .../ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 | 0 .../ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 .../ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a | 0 .../ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 | 0 .../ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 | 0 .../ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 .../ba_table3-1-28b12606c5369c783e63c17826a18d0d | 0 ...ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b | 0 ...a_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 ...a_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 | 0 ...a_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 | 0 ...nary_constant-0-8c922b2264278dd481ef0dff2088e2b8 | 0 ...output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 | 0 ...output_format-1-9e0909b6330578a25806527dd0ecf7ef | 0 ...output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 | 0 ...output_format-3-84db2ef4a7f2151e26457db559b862d9 | 0 ...e_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 ...e_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 | 0 ...e_bincolserde-2-706a062089583074f30fb13c661fc81e | 0 ...e_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 | 0 ...able_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 | 0 ...able_colserde-1-179ac81920d8dfa6e324cc881b5f1624 | 0 ...able_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 | 0 ...arysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 | 0 ...arysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 | 0 ...arysortable_1-2-faa8d95365e4116734a056c911350c05 | 0 ...arysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 | 0 .../golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 | 0 .../golden/cast1-1-1ee16b8209701131017533cfa6eb4680 | 0 .../golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c | 0 .../golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a | 0 .../cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 | 0 .../cluster-0-16681f9c2bdd44278817d72c138b6ee1 | 0 .../cluster-1-707a2295731e0d631a6c5f71c745c8d5 | 0 .../cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 | 0 .../cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e | 0 .../cluster-12-6ad920e2ae83c78fccb06ff65308a438 | 0 .../cluster-13-12635b4b7f34eba5554d5b892b5b64e7 | 0 .../cluster-14-cd2e125bceb1611137f0750f5d69c475 | 0 .../cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 | 0 .../cluster-16-a2d5e5ec2504041ea1a62856c7086451 | 0 .../cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 | 0 .../cluster-18-e01f450969ae7e1cd018e6ef0cc67141 | 0 .../cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 | 0 .../cluster-2-50197277eb03ad20696a135bd7d18de7 | 0 .../cluster-20-294891c1d956245540a80aa800ba393d | 0 .../cluster-21-4787b258a786cf195bcb59cd90f6013f | 0 .../cluster-22-8801aa93cf6dba7e13e99a0260fde68 | 0 .../cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e | 0 .../cluster-24-f492a7f78faf180621e83e5a69aa1eae | 0 .../cluster-25-f57ce48b6a6e671b58c96535ab482b6a | 0 .../cluster-3-530671e2a5b8983c60cfedaf486f0f0f | 0 .../cluster-4-cb4af90f52f2626213f918fda3b81dfc | 0 .../cluster-5-e99040f6a24c53a4c89801ff3663ff72 | 0 .../cluster-6-56f8e3e7abe504522a2bfd77b5be3270 | 0 .../cluster-7-a22600d60c81a25061b1e20b6726c691 | 0 .../cluster-8-251b178e4fe39ea03a30d2b9bd40710d | 0 .../cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 | 0 ...eate_shortcut-0-d300f67f11082f3614a8e93e8808960d | 0 ...eate_shortcut-1-3a1329c4145738961e1b8bdbd056497c | 0 ...reate_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 | 0 ...eate_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f | 0 ...stats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c | 0 ...stats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 | 0 ...stats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 | 0 ...stats_partlvl-3-3ab5479f002e412965f259485075f6bd | 0 ...nstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 | 0 ...stats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 | 0 ...stats_partlvl-6-86ba38eff353a720bbabf726365b6712 | 0 ...stats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 | 0 ...stats_partlvl-8-dc5682403f4154cef30860f2b4e37bce | 0 ...nstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe | 0 ...nstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 | 0 ...nstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c | 0 ...nstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 | 0 ...nstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 | 0 ...nstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 | 0 ...nstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c | 0 ...nstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 | 0 ...nstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d | 0 ...nstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f | 0 .../combine1-0-84b74227c9f1563f530cd3ac3b333e54 | 0 .../combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 .../combine1-2-c95dc367df88c9e5cf77157f29ba2daf | 0 .../combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 | 0 .../combine1-4-84967075baa3e56fff2a23f8ab9ba076 | 0 .../combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea | 0 .../combine1-6-1d1f97cce07323812de3027920b04b75 | 0 .../combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 | 0 .../combine1-8-da1fda96db80592bf2bbda8f22b5687c | 0 .../combine1-9-e5ce23369b0ad260512a0f61c6969b73 | 0 .../combine2-0-7a9e67189d3d4151f23b12c22bde06b5 | 0 .../combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 .../combine2-10-54649f87d403c6fcb163c4d51e382d3a | 0 .../combine2-11-2d2d73a929c7d995ea57b40529b74b56 | 0 .../combine2-12-cd15ffd140539cf86090814729ec4748 | 0 .../combine2-13-5ac3e540fd24f94fee378e49597817b3 | 0 .../combine2-14-4695309eb4e91ef29c9857aa8fd6130c | 0 .../combine2-15-dd652175dac4463fed3c56aded11e6c1 | 0 .../combine2-16-557997716a68312e8cae75428e3ce31 | 0 .../combine2-17-8e4598e3f0701478ed12042438699ce5 | 0 .../combine2-18-2af7419c1d84fe155e23f3972e049b97 | 0 .../combine2-2-c95dc367df88c9e5cf77157f29ba2daf | 0 .../combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 | 0 .../combine2-4-84967075baa3e56fff2a23f8ab9ba076 | 0 .../combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea | 0 .../combine2-6-a4fb8359a2179ec70777aad6366071b7 | 0 .../combine2-7-16367c381d4b189b3640c92511244bfe | 0 .../combine2-8-99d1f07b2ce904afd6a809fd1814efe9 | 0 .../combine2-9-30cb07965e4b5025545361b948fc83c2 | 0 ...ine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...ine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 ...ne2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 | 0 ...ne2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 | 0 ...ne2_hadoop20-12-cd15ffd140539cf86090814729ec4748 | 0 ...ne2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 | 0 ...ne2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c | 0 ...ne2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 | 0 ...ine2_hadoop20-16-557997716a68312e8cae75428e3ce31 | 0 ...ne2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 | 0 ...ne2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 | 0 ...ine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf | 0 ...ine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 | 0 ...ine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 | 0 ...ine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea | 0 ...ine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 | 0 ...ine2_hadoop20-7-16367c381d4b189b3640c92511244bfe | 0 ...ine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 | 0 ...ine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 | 0 .../combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 .../combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf | 0 ...combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 | 0 ...combine2_win-11-cd15ffd140539cf86090814729ec4748 | 0 ...combine2_win-12-5ac3e540fd24f94fee378e49597817b3 | 0 ...combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c | 0 ...combine2_win-14-dd652175dac4463fed3c56aded11e6c1 | 0 .../combine2_win-15-557997716a68312e8cae75428e3ce31 | 0 ...combine2_win-16-8e4598e3f0701478ed12042438699ce5 | 0 ...combine2_win-17-2af7419c1d84fe155e23f3972e049b97 | 0 .../combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 | 0 .../combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 | 0 .../combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea | 0 .../combine2_win-5-a4fb8359a2179ec70777aad6366071b7 | 0 .../combine2_win-6-16367c381d4b189b3640c92511244bfe | 0 .../combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 | 0 .../combine2_win-8-30cb07965e4b5025545361b948fc83c2 | 0 .../combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af | 0 .../combine3-0-84b74227c9f1563f530cd3ac3b333e54 | 0 .../combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 .../combine3-10-fb173ed4483e732d367e51f88be793b1 | 0 .../combine3-11-30b8b79049113252bec1cbeac4018a3 | 0 .../combine3-12-f4baee25e0ad813258d67d707a6fc43b | 0 .../combine3-13-1c359bedf474e8e26f3b3562f7af6edc | 0 .../combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 .../combine3-15-7cccbdffc32975f8935eeba14a28147 | 0 .../combine3-16-6635f7f5c55557b06ad3acc321eaa739 | 0 .../combine3-17-8cb751103da7c909276db6bddb50ae6a | 0 .../combine3-18-31fae7c6be75b97d475512bd75a58a0b | 0 .../combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f | 0 .../combine3-2-c95dc367df88c9e5cf77157f29ba2daf | 0 .../combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd | 0 .../combine3-21-8ba8f8723c1530be062cefc2d9246e56 | 0 .../combine3-22-11025483569617a9f014b5defd71e933 | 0 .../combine3-23-4725c48df09565618cbffd05953a5f62 | 0 .../combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 | 0 .../combine3-4-84967075baa3e56fff2a23f8ab9ba076 | 0 .../combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea | 0 .../combine3-6-4725c48df09565618cbffd05953a5f62 | 0 .../combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 | 0 .../combine3-8-68399bc39f71ddc99ed09ed9d2fd897b | 0 .../combine3-9-b5703b76bbe99c41cbb63582a09a6e69 | 0 ..._stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 | 0 ..._stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 | 0 ..._stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c | 0 ..._stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 | 0 ...stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 | 0 ...stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 | 0 ...stats_boolean-2-fbea367698de65e22d4d660a518ea95e | 0 ...stats_boolean-3-a14d8a5835c94829271f9f463d96d83d | 0 ..._stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce | 0 ..._stats_double-1-a23a25a680139ed823c77f3f9f486065 | 0 ..._stats_double-2-8f988b757fd62f318f35447a3fd65452 | 0 ..._stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 | 0 ...s_empty_table-0-12161b12442ad9b664b51e443fabaf5d | 0 ...s_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 | 0 ...s_empty_table-2-372df408a15de1e6f05e807a3aff223f | 0 ...s_empty_table-3-73f6626835884f34a47e8e78396bc25b | 0 ...s_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf | 0 ...s_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 | 0 ...ts_empty_table-6-137180ebd2a072f08b5d849bdd9a464 | 0 ...te_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 | 0 ...te_stats_long-1-a7bc730f9862198709539e35c0208248 | 0 ...te_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 | 0 ...te_stats_long-3-cf4bb755ade3079409e2251c7cd0118d | 0 ..._stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b | 0 ..._stats_string-1-3bddaed6594ed44249e4a30c43e83d1f | 0 ..._stats_string-2-b87a68ae5ffa689bada75425169d131a | 0 ...e_stats_string-3-cea908dd41c78490990ee6b681d19fc | 0 ...num_to_string-0-f22bc1aaadc6f36ba36420073ea04543 | 0 ...num_to_string-1-db089ff46f9826c7883198adacdfad59 | 0 ...onoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b | 0 ...ionoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 | 0 ...noptimizer11-10-b9d963d24994c47c3776dda6f7d3881f | 0 ...noptimizer11-11-f7918ee4d4941d3272e0262a750de700 | 0 ...noptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 | 0 ...noptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...noptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d | 0 ...noptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 | 0 ...onoptimizer11-2-e148026f8994e22ca756c68753a0cc26 | 0 ...ionoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 | 0 ...onoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 | 0 ...onoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 | 0 ...onoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer11-8-45d4d690886288ef04addbb659397ad1 | 0 ...onoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 | 0 ...onoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 | 0 ...onoptimizer14-1-d0a93f40892e3894460553b443c77428 | 0 ...noptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda | 0 ...noptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...noptimizer14-12-99f81dd0f33197c724eb58398542ff22 | 0 ...noptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 | 0 ...noptimizer14-14-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer14-15-78fed7defb6154e01abbd97a0741adf | 0 ...onoptimizer14-16-7ebe26e8a3620830e824b4099519395 | 0 ...noptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ionoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 | 0 ...onoptimizer14-19-7ebe26e8a3620830e824b4099519395 | 0 ...onoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...noptimizer14-20-b9d963d24994c47c3776dda6f7d3881f | 0 ...noptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 | 0 ...noptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 | 0 ...noptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...noptimizer14-24-8e88fd43a2c216b3409bee768425772c | 0 ...noptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 | 0 ...onoptimizer14-3-88b3974a7639097ed915402827e8941f | 0 ...onoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f | 0 ...onoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c | 0 ...onoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda | 0 ...onoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 | 0 ...onoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b | 0 ...onoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 | 0 ...onoptimizer15-3-e149747103059314a9984235702b24b6 | 0 ...onoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 | 0 ...onoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd | 0 ...onoptimizer15-7-e149747103059314a9984235702b24b6 | 0 ...ionoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b | 0 ...ionoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ionoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab | 0 ...onoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 | 0 ...onoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd | 0 ...onoptimizer2-15-18f10d12e8bfa473a916c2f528500538 | 0 ...onoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 | 0 ...onoptimizer2-18-18f10d12e8bfa473a916c2f528500538 | 0 ...onoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f | 0 ...ionoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 | 0 ...onoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf | 0 ...onoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e | 0 ...onoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer2-23-38b273c68d644e801695d5414c0e119b | 0 ...onoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e | 0 ...onoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 | 0 ...onoptimizer2-27-9b22dad2843cdc379d90687745561104 | 0 ...onoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer2-29-3781a20b72437434ef8fa7174edf36ab | 0 ...ionoptimizer2-3-d915fbdd493869aec42f548bdb66598d | 0 ...onoptimizer2-30-9b22dad2843cdc379d90687745561104 | 0 ...onoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer2-32-5ac93f83acfd31ce036381993eda303f | 0 ...onoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b | 0 ...onoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 | 0 ...onoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b | 0 ...ionoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ionoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 | 0 ...ionoptimizer2-6-d915fbdd493869aec42f548bdb66598d | 0 ...ionoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f | 0 ...ionoptimizer2-8-74078f19dfe424f3211e6ce26de52152 | 0 ...ionoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 | 0 ...ionoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b | 0 ...ionoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 | 0 ...onoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b | 0 ...onoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f | 0 ...ionoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 | 0 ...onoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 | 0 ...onoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ionoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 | 0 ...onoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 | 0 ...onoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer3-19-24ca942f094b14b92086305cc125e833 | 0 ...ionoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace | 0 ...onoptimizer3-20-4025759f01fa2169a061070319ee8bfe | 0 ...onoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 | 0 ...ionoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 | 0 ...ionoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ionoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 | 0 ...ionoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 | 0 ...ionoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ionoptimizer3-8-24ca942f094b14b92086305cc125e833 | 0 ...ionoptimizer3-9-c575bf5ba408caadb836d307b9971bea | 0 ...ionoptimizer4-0-d157f058f9e8659c4367e01c4da13579 | 0 ...ionoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...onoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e | 0 ...onoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e | 0 ...onoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer4-14-24ca942f094b14b92086305cc125e833 | 0 ...onoptimizer4-15-8db0d44941d0ce086e95088ef579c136 | 0 ...onoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e | 0 ...onoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b | 0 ...onoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer4-19-5e241577196c76217ed9615fcbc76dcb | 0 ...tionoptimizer4-2-46c5eef67c57677810028451dd2b4d9 | 0 ...onoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 | 0 ...onoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 | 0 ...onoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 | 0 ...onoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer4-25-18ec265534266497e1da68480cfe51cf | 0 ...onoptimizer4-26-62a0fd05be48759c39f3c284458dde9b | 0 ...onoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb | 0 ...onoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf | 0 ...ionoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 | 0 ...onoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer4-31-7fea74857587a30456b095e20bc2bde1 | 0 ...onoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf | 0 ...onoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 | 0 ...onoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 | 0 ...onoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 | 0 ...onoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 | 0 ...onoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ionoptimizer4-4-ee940bcfd73a883d16245ef746798e15 | 0 ...onoptimizer4-40-ef5268865a18f57e15dc650a11527b11 | 0 ...onoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 | 0 ...ionoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 | 0 ...ionoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b | 0 ...ionoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f | 0 ...ionoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 | 0 ...ionoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e | 0 ...ionoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b | 0 ...ionoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 | 0 ...onoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b | 0 ...onoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer6-13-f16444a052505377b7747949ffc2c028 | 0 ...onoptimizer6-14-84463190baec77d61b287a071c8886db | 0 ...onoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 | 0 ...onoptimizer6-17-84463190baec77d61b287a071c8886db | 0 ...onoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 | 0 ...ionoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 | 0 ...onoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de | 0 ...onoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 | 0 ...onoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de | 0 ...onoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b | 0 ...onoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 | 0 ...onoptimizer6-27-e149747103059314a9984235702b24b6 | 0 ...onoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd | 0 ...ionoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 | 0 ...onoptimizer6-30-e149747103059314a9984235702b24b6 | 0 ...onoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 | 0 ...onoptimizer6-33-15d991127dc684513e2fff1aea3f1560 | 0 ...onoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 | 0 ...onoptimizer6-36-15d991127dc684513e2fff1aea3f1560 | 0 ...onoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer6-38-d1a903645384c97759e835649ce898fb | 0 ...onoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 | 0 ...ionoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 | 0 ...onoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 | 0 ...onoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer6-44-ed339f429b82397bffb7298a534d59fb | 0 ...onoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 | 0 ...onoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 | 0 ...onoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 | 0 ...onoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f | 0 ...ionoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 | 0 ...onoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 | 0 ...onoptimizer6-51-4746d944f4193018017984ca2df3c60d | 0 ...onoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer6-53-b9552c6399ce578cde69e663dd43d870 | 0 ...onoptimizer6-54-4746d944f4193018017984ca2df3c60d | 0 ...onoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer6-56-24ca942f094b14b92086305cc125e833 | 0 ...onoptimizer6-57-b9552c6399ce578cde69e663dd43d870 | 0 ...onoptimizer6-58-4746d944f4193018017984ca2df3c60d | 0 ...ionoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 | 0 ...ionoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ionoptimizer6-8-24ca942f094b14b92086305cc125e833 | 0 ...ionoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae | 0 ...ionoptimizer7-0-24ca942f094b14b92086305cc125e833 | 0 ...ionoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer7-11-fc438bb849eff3496559a916c7dee058 | 0 ...onoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 | 0 ...onoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 | 0 ...onoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 | 0 ...ionoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 | 0 ...ionoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 | 0 ...ionoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ionoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 | 0 ...ionoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 | 0 ...ionoptimizer7-7-24ca942f094b14b92086305cc125e833 | 0 ...ionoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 | 0 ...ionoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d | 0 ...ionoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b | 0 ...ionoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f | 0 ...onoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d | 0 ...tionoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f | 0 ...onoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea | 0 ...onoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...onoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc | 0 ...ionoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a | 0 ...ionoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 | 0 ...ionoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ionoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 | 0 ...ionoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 | 0 ...ionoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f | 0 ...ionoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 | 0 ...ationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f | 0 .../golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b | 0 .../golden/count-1-3531872d964bc2b4f07d51b9d1629df2 | 0 .../count-10-d0f804c7d06375db74a0fcf5f17603c6 | 0 .../count-11-29aa086fe75b55482a91316c38498565 | 0 .../count-12-944f53db544c07a7b38a0544a21d8e13 | 0 .../golden/count-2-461bad3feb7dbc25fb35d45c6876d698 | 0 .../golden/count-3-dbcec232623048c7748b708123e18bf0 | 0 .../golden/count-4-590bf60b8d4dfa135f73dbb52180136f | 0 .../golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 | 0 .../golden/count-6-29aa086fe75b55482a91316c38498565 | 0 .../golden/count-7-944f53db544c07a7b38a0544a21d8e13 | 0 .../golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../golden/count-9-590bf60b8d4dfa135f73dbb52180136f | 0 .../cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 | 0 .../cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 | 0 .../cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e | 0 .../cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 | 0 .../cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 | 0 .../cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 | 0 ... with db name-0-a253b1ed35dbf503d1b8902dacbe23ac | 0 ... with db name-0-b7dfeb6a941b42f7def5fdceae99f425 | 0 ... with db name-1-417609d2bb67ba26de38e92ad834008f | 0 ... with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 | 0 ... with db name-2-ce780d068b8d24786e639e361101a0c7 | 0 ... with db name-3-afd6e46b6a289c3c24a8eec75a94043c | 0 ...eate table as-0-fd42e3ffae73415391acb1012a3531bd | 0 ...eate table as-1-b9002c1d71895be765575b62656d1928 | 0 .../create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d | 0 .../create_1-1-c2351f011b8ea41ff7dfa8f195148da3 | 0 .../create_1-10-b9c0b95624e601614ea2561b83aaf0ba | 0 .../create_1-11-7daaeabd9c286e511e0628a32dc714d5 | 0 .../create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 | 0 .../create_1-13-5715f85456733a71fb7c83f1b254b3a | 0 .../create_1-14-437bd1acbae61f48e851addb769d3379 | 0 .../create_1-15-fd9415e340cf54a9473cc4bf86673816 | 0 .../create_1-16-c99c700ca511b68577aae8ae513a4a32 | 0 .../create_1-2-ecd02bc3563cd6b60b8394956cb69084 | 0 .../create_1-3-c27702ff131e0ecfd71f1e1779fbe365 | 0 .../create_1-4-610b82bf7b0080d293977927e5ef780c | 0 .../create_1-5-c77b018276b1558c1d9462e0625e152e | 0 .../create_1-6-52dc9f900d7f7a559698aff9565f061a | 0 .../create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 | 0 .../create_1-8-19331fe6a2a35f1171babfe4e1c86f59 | 0 .../create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 | 0 ...reate_big_view-0-bdf3c2e20793ef833f336a40791091d | 0 ...eate_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee | 0 ...create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e | 0 ...create_escape-1-ecd02bc3563cd6b60b8394956cb69084 | 0 ...create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 | 0 .../create_escape-3-9541399cde42210bd7ac1beb07ceb14 | 0 .../create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 | 0 ...e_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 | 0 ...e_genericudaf-1-c7f934e9c76350a0d3caa694463a673b | 0 ...e_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 | 0 ...e_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 | 0 ...te_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 | 0 ...te_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 | 0 ..._outputformat-0-16167c581df48112004009fef228e29a | 0 ..._outputformat-1-1246ba69e870178971f5ae062641cf47 | 0 ..._outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc | 0 ..._outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 | 0 ..._outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 | 0 ...t_outputformat-5-8552731917a8260c25e6df79b83bf5c | 0 ..._outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 | 0 ...t_outputformat-7-a755c7e39694261510421e262b5005e | 0 ..._outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 | 0 .../create_like-0-c2351f011b8ea41ff7dfa8f195148da3 | 0 .../create_like-1-30485a2507b60b96cad3d293527e6af | 0 .../create_like-10-7d84873a6ce03e0e408fa67ef5dd733 | 0 .../create_like-11-ba64f0122b21f605d8b2928753784130 | 0 .../create_like-2-a20451f152e68606cc5e373fe5fd86a | 0 .../create_like-3-eea111a209cf8895f31f64699669c705 | 0 .../create_like-4-39ead53334938635b60a5ffdaa2c9f86 | 0 .../create_like-5-dc9de26002604e9e436135bd4b40636d | 0 .../create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 | 0 .../create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 | 0 .../create_like-8-639a13da6855b974fc2e170fd49b33cb | 0 .../create_like-9-a0ce573e299b66b8ce31da2890b318cb | 0 ...ate_like_view-0-3b48eae3848493703396156bedb1e98b | 0 ...ate_like_view-1-3c805fc10db9af83327e04d518f3753a | 0 ...te_like_view-10-eea111a209cf8895f31f64699669c705 | 0 ...te_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 | 0 ...te_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 | 0 ...te_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 | 0 ...te_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 | 0 ...te_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 | 0 ...te_like_view-16-639a13da6855b974fc2e170fd49b33cb | 0 ...te_like_view-17-a0ce573e299b66b8ce31da2890b318cb | 0 ...eate_like_view-18-d77d78569d86802f7c097d3d02150c | 0 ...ate_like_view-19-deabf2d92205da2cbce9bdff854a81f | 0 ...ate_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b | 0 ...te_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 | 0 ...te_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 | 0 ...te_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 | 0 ...te_like_view-23-44c601e364e42a83babab4a342dfbd2b | 0 ...te_like_view-24-df2d18dbae578430bfc7b9d27d201505 | 0 ...te_like_view-25-87a663f8fd80110a9cee249535037c0d | 0 ...te_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 | 0 ...te_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 | 0 ...ate_like_view-3-559f17e8f827532749948b3b9e6c0f3f | 0 ...ate_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 | 0 ...ate_like_view-5-ecd02bc3563cd6b60b8394956cb69084 | 0 ...eate_like_view-6-30485a2507b60b96cad3d293527e6af | 0 ...ate_like_view-7-fcc954b645304d258611f21d3aed7b76 | 0 ...ate_like_view-8-304a79a8a321b84aee91f907f756a7e3 | 0 ...ate_like_view-9-52dc9f900d7f7a559698aff9565f061a | 0 ...ge_compressed-0-366a4de0343396b9df03277f1098722c | 0 ...ge_compressed-1-276fbe6fb296b13904516888ffa95342 | 0 ...e_compressed-10-d6fee0f05fa9b04cb7c557862402c929 | 0 ...e_compressed-11-614c34f9e88015f21bffc4b8930bc95d | 0 ...e_compressed-12-4d89cbe49f710527b54e6262472f0320 | 0 ...e_compressed-13-440c6f8daa221613fe796d99b494e61f | 0 ...e_compressed-14-32251c08304629a3153e0b471de060c5 | 0 ...ge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 | 0 ...ge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 | 0 ...ge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 | 0 ...ge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 | 0 ...ge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 | 0 ...ge_compressed-7-614c34f9e88015f21bffc4b8930bc95d | 0 ...ge_compressed-8-4d89cbe49f710527b54e6262472f0320 | 0 ...ge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 | 0 ...e_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 | 0 ...e_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 | 0 ...e_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 | 0 ...e_nested_type-3-ac452c9279877935983c37113898e53c | 0 ...e_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff | 0 ..._replace_view-0-a14cfe3eff322066e61023ec06c7735d | 0 ..._replace_view-1-a27131eb04bd5e071d3267c92d3f8dea | 0 ..._replace_view-2-30dc3e80e3873af5115e4f5e39078a13 | 0 ..._replace_view-3-5fd147edbe44a96782923a3ef6caa47d | 0 ..._replace_view-4-b1880014afc9ad1f8db91ba3db3867de | 0 ..._replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb | 0 ...skewed_table1-0-cafed8ca348b243372b9114910be1557 | 0 ..._skewed_table1-1-cc66bd64f1cdc97b953e20860305370 | 0 ...skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e | 0 ...skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 | 0 ...skewed_table1-4-f3f1642674545762a4bff5cb75634e20 | 0 ...skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e | 0 ...skewed_table1-6-d7a147c6b0a3609663628b43457b2cef | 0 ...skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 | 0 ...skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd | 0 ...skewed_table1-9-758987cfb7302bdb76898290de49a80e | 0 ..._struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 | 0 ..._struct_table-1-2ca90a28a994405e6150c96f4a572294 | 0 ..._struct_table-2-d51e74fd10cc16607137b7f715557ecd | 0 .../create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e | 0 .../create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 | 0 .../create_view-1-c186ac1fe46117acb6fd452df15e0d92 | 0 .../create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd | 0 .../create_view-11-2021c047203276dd2db51a56e672fcea | 0 .../create_view-12-420752b11848e29bce1c8ace7d3060fc | 0 .../create_view-13-bff53e5df8356ac16f7b9b78b157e60a | 0 .../create_view-14-69162f2f22978113fea529d7fc7b78d3 | 0 .../create_view-15-ceebf4cb0dc23f517a444266bc8d2447 | 0 .../create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 | 0 .../create_view-17-b3c1023d56f3439833c246e8bfd4502a | 0 .../create_view-18-437d0a699b73c61044ebf8539feb14f6 | 0 .../create_view-19-df2da604b5a8f2b236519510b5e4d34b | 0 .../create_view-2-d80dcd1271ab264292e9938f3162427c | 0 .../create_view-20-56d203e4d0eb776bb4fa38409222b5b8 | 0 .../create_view-21-3609711e61b5b8d241d0e839557bfd64 | 0 .../create_view-22-3bc364c0ee46900d2201d706d2d58d67 | 0 .../create_view-3-25ffe475d52d6c399acaf120dc02afe8 | 0 .../create_view-4-87ed262d455e99ad45c909a2265a61b0 | 0 .../create_view-5-391caf27ff1589ec68d5f3bc4a27e711 | 0 .../create_view-6-d8d0e830783c383e3c00e9de3919c409 | 0 .../create_view-7-50b35b8a1772becc96cff65bba1eaee7 | 0 .../create_view-8-2ae18fc75eda9c3fe7d4e87829180805 | 0 .../create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 | 0 ...w_partitioned-0-d98274f9b34c8968292ccd6c959491dc | 0 ...w_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 | 0 ...w_partitioned-2-45e7b89caadba56ec67638c341209f96 | 0 ...w_partitioned-3-cf44ff130f66de720a77888260ef8d16 | 0 ...w_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 | 0 ...w_partitioned-5-d7a7d8592fca266745725192d3f875fc | 0 ...e_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 | 0 ...e_insensitive-1-893c61ec6ea62362324c213f588d8030 | 0 ...e_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 | 0 .../ctas_varchar-0-311fdd725609cd47ea1b859f706da41e | 0 .../ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 | 0 .../ctas_varchar-2-3223504c97628a44b65736565c1dda32 | 0 .../ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b | 0 .../ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 | 0 .../ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f | 0 .../ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 | 0 .../ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec | 0 .../ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 | 0 ...output_format-0-94f3da887aa34aed74715bd2051bf3c5 | 0 ...le table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd | 0 ...atabase.table-0-c657beb729b6a7882309a203fc6f298e | 0 ...database_drop-0-49f18014566b3e020dc19b1e61d25a4f | 0 ...database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 | 0 ...atabase_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 | 0 ...atabase_drop-11-2ea883422b74b701711e14e61472ba06 | 0 ...atabase_drop-12-e02a53f7e798d2741152526516f14941 | 0 ...database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 | 0 ...database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 | 0 ...database_drop-4-d419f4ff197d4291208c2028cd158909 | 0 ...database_drop-5-b7cf74929eabe781b0db79ed1043dc24 | 0 ...database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 | 0 ...database_drop-7-693736836ccc99f6e2449b94efcfeeba | 0 ...database_drop-8-97101266791d2b2c662bcde549422318 | 0 ...database_drop-9-8db536f925bf0f5058f97897e145a661 | 0 ...base_location-0-5528e36b3b0f5b14313898cc45f9c23a | 0 ...abase_location-1-2b66b4554cf8ecf2ab6c325d4d89491 | 0 ...ase_location-10-c5cd9c57a13da7f345563fbd75da4e45 | 0 ...ase_location-11-9c36cac1372650b703400c60dd29042c | 0 ...base_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 | 0 ...base_location-3-81383a2b9568f942cc8e758f9a6ee2f1 | 0 ...base_location-4-be65cf649363681d54e593c42a5ecffb | 0 ...base_location-5-9c36cac1372650b703400c60dd29042c | 0 ...base_location-6-6fa58170a7c2e78b06a250403f02091a | 0 ...base_location-7-5698ac10441da07dbe3a947143c999c2 | 0 ...base_location-8-6f2797b6f81943d3b53b8d247ae8512b | 0 ...base_location-9-92f087a5934481942995fc2aaf0d87e8 | 0 ...se_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 ...se_properties-1-10de6a198e2b3f61974519ddd8623e68 | 0 ...e_properties-10-26c10ff2ec4a69b16589069ced427d23 | 0 ...se_properties-2-a1074315e598ad16bce55860e6e43363 | 0 ...se_properties-3-751417d45b8e80ee5cba2034458b5bc9 | 0 ...se_properties-4-ddf44597db4fa15e89bee313f2dad371 | 0 ...se_properties-5-51c0974df1125b233936f25ce709ba4a | 0 ...se_properties-6-26c10ff2ec4a69b16589069ced427d23 | 0 ...se_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f | 0 ...se_properties-8-10de6a198e2b3f61974519ddd8623e68 | 0 ...se_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a | 0 .../date_2-0-8acfa0b538112534f57a3e051f0216bd | 0 .../date_2-1-116032b973a2060b533e1cdc9dfda301 | 0 .../date_2-2-cab14d992c53c106ab257fae52001e04 | 0 .../date_3-0-c26de4559926ddb0127d2dc5ea154774 | 0 .../date_3-1-d9a07d08f5204ae8208fd88c9255d447 | 0 .../date_3-2-a937c6e5a2c655930e0d3f80883ecc16 | 0 .../date_join1-0-70b9b49c55699fe94cfde069f5d197c | 0 .../date_join1-1-3a68de2112a212a07a3068916c608fb | 0 .../date_join1-2-894b6541812ac8b0abe2a24c966817d8 | 0 .../date_serde-0-ca88593bb7ec47fa782145d732100c07 | 0 .../date_serde-1-36e6041f53433482631018410bb62a99 | 0 .../date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 | 0 .../date_serde-3-e6167e27465514356c557a77d956ea46 | 0 .../date_serde-4-c1e17c93582656c12970c37bac153bf2 | 0 .../date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c | 0 .../date_serde-6-e00facec2986bc55074868eff87ba22a | 0 .../date_serde-7-a34279d8ebbadb78e925e8ed9c78947d | 0 .../ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 | 0 .../ddltime-1-426da52526f6f48c0ddeb0026fd566f1 | 0 .../decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 | 0 .../decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd | 0 .../decimal_1-10-be179d261b9c42ed843dbf736b12e75 | 0 .../decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 | 0 .../decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 | 0 .../decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 | 0 .../decimal_1-2-80fc87cab17ceffea334afbb230a6653 | 0 .../decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 | 0 .../decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 | 0 .../decimal_1-5-cbe6b235663cf78e602673ed715a2f40 | 0 .../decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b | 0 .../decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a | 0 .../decimal_1-8-cdd0932288d3cc43636334439805769d | 0 .../decimal_1-9-1504e1b00092e2c58bafcc9936ad178c | 0 .../decimal_2-0-3c8ecb1468952afe028596c65d587bee | 0 .../decimal_2-1-868e124edc1581325bd0fd10235a126b | 0 .../decimal_2-10-f97d72aeb605ee18d34361c073552e92 | 0 .../decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 | 0 .../decimal_2-12-d63b5ea25e27852413132db4d9bfb035 | 0 .../decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 | 0 .../decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 | 0 .../decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 | 0 .../decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 | 0 .../decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d | 0 .../decimal_2-18-f7c34d67fd579c82c636415172ec675e | 0 .../decimal_2-19-f97d72aeb605ee18d34361c073552e92 | 0 .../decimal_2-2-6cc742523b3574e59ca21dad30f2d506 | 0 .../decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 | 0 .../decimal_2-21-d72d68be1217c7b7a958f58456d85821 | 0 .../decimal_2-22-648e694eea042c59e8db30d067cb5bc8 | 0 .../decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff | 0 .../decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e | 0 .../decimal_2-25-14face5c7104382196e65741a199c36 | 0 .../decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 | 0 .../decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 | 0 .../decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 | 0 .../decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 | 0 .../decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e | 0 .../decimal_2-30-26a71d79e41353830b4ada96de6e2b8a | 0 .../decimal_2-31-3c8ecb1468952afe028596c65d587bee | 0 .../decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 | 0 .../decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 | 0 .../decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 | 0 .../decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 | 0 .../decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d | 0 .../decimal_2-9-f7c34d67fd579c82c636415172ec675e | 0 .../decimal_3-0-90cd495a00051a0631b2021dbb9a4aef | 0 .../decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 | 0 .../decimal_3-10-420614bb0789115e008c96a7ad822624 | 0 .../decimal_3-11-63913753553b16d6c24e063fb49fdd15 | 0 .../decimal_3-12-d495d7178707ba55dcc01b9bb3398792 | 0 .../decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 | 0 .../decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b | 0 .../decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 | 0 .../decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b | 0 .../decimal_3-6-127a3a8400cae591c282dd24f8951e55 | 0 .../decimal_3-7-9d4f27d4a4819113c5083462baa72052 | 0 .../decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 | 0 .../decimal_3-9-b54243d38214362f9a9b1831548faac4 | 0 .../decimal_4-0-98a58225355eb73036bb7b1144fa5a5f | 0 .../decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 | 0 .../decimal_4-2-945542ec888136afaece8d7a5e20d52d | 0 .../decimal_4-3-399140971a10a5a0cc6a8c97a4635e | 0 .../decimal_4-4-81b37675c09ed874497325ae13233e5c | 0 .../decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c | 0 .../decimal_4-6-693c2e345731f9b2b547c3b75218458e | 0 .../decimal_4-7-f1eb45492510cb76cf6b452121af8531 | 0 .../decimal_4-8-79734272b75fb9076bdb64644bed6276 | 0 .../decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 | 0 .../decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 | 0 .../decimal_join-1-c1524f17ee815171055a67ddc2f9de4e | 0 .../decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 | 0 ...mal_precision-0-cae8ceb6b0ab342948041c511a867b8e | 0 ...mal_precision-1-3f4119830536c92e5ccd76be0259e110 | 0 ...al_precision-10-d8a597810b222e9e121a11a1f5658fb0 | 0 ...al_precision-11-673b15434ba47f11c71c3e8b2a575d83 | 0 ...al_precision-12-18906f5c6413065621430e3fe33c7e9e | 0 ...al_precision-13-2a65d450f57f8ba9f594063b96074f0e | 0 ...al_precision-14-34916eb904b8113a401ce78e6941a204 | 0 ...al_precision-15-5c49f041326bc5a9e936910094f190ce | 0 ...mal_precision-2-d5be00a0fa6e2e290b40458442bd036c | 0 ...mal_precision-3-42cb35d680b3caeeb22e1c4865b8264b | 0 ...mal_precision-4-38aaeba3e587b4dac72e26c4b02029fc | 0 ...mal_precision-5-bb27734245ecbd0511be91af21c3b9ef | 0 ...imal_precision-6-b2547e6ef33325b2da12ce91b57af21 | 0 ...mal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 | 0 ...mal_precision-8-6e6bd4655de124dc1fc482ce0d11930e | 0 ...mal_precision-9-e7b465fbeb49487b2a972a314e2c01ab | 0 ...artition_name-0-312a37c422883aa0d660018248157cf8 | 0 ...artition_name-1-9de8e5f66c536d4ace89c61759db829c | 0 ...artition_name-2-8732fdce7fb224dd783da2b83a93c795 | 0 ...artition_name-3-a7047012b4bce0158edaafe5cf0a57be | 0 .../delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba | 0 .../delimiter-1-d9e405c4107da8af78fcacb83a667b41 | 0 .../delimiter-2-d7137294d2e53ea6edc259943e4c6069 | 0 .../delimiter-3-176724f76343433a8f2e6131b12206d7 | 0 .../delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 | 0 ...omment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb | 0 ...omment_indent-1-5536eb772d43014b971c6da3a0c44904 | 0 ...omment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 | 0 ...database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b | 0 ...database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 | 0 ...atabase_json-10-1afddec8522bd69f496b15980600a6e1 | 0 ...atabase_json-11-b05391400dc31139998dc3abaaf86320 | 0 ..._database_json-2-8e7cfe3e6069e796124ca940125385a | 0 ...database_json-3-d097973152d91fa8072facb0f739e304 | 0 ...database_json-4-549981e00a3d95f03dd5a9ef6044aa20 | 0 ...database_json-5-a3ee372283f45479db3f2cd7ebeedc8c | 0 ...database_json-6-1afddec8522bd69f496b15980600a6e1 | 0 ...database_json-7-7529ec337ca17cdf95d037f29e1cb793 | 0 ..._database_json-8-8e7cfe3e6069e796124ca940125385a | 0 ...database_json-9-d097973152d91fa8072facb0f739e304 | 0 ...w_partitioned-0-889714213a760ae9ab3ebe199eb30b62 | 0 ...w_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa | 0 ...w_partitioned-2-db8910ace81a5102495905a508ef5c28 | 0 ...w_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 | 0 ...w_partitioned-4-889714213a760ae9ab3ebe199eb30b62 | 0 ...titioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b | 0 ...titioned_json-1-889714213a760ae9ab3ebe199eb30b62 | 0 ...titioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa | 0 ...titioned_json-3-db8910ace81a5102495905a508ef5c28 | 0 ...titioned_json-4-b80c7ae3530bfdbc8e865d49742da826 | 0 ...titioned_json-5-889714213a760ae9ab3ebe199eb30b62 | 0 ...escribe_pretty-0-f34ca99310bf1d4793cf64423c024ad | 0 ...scribe_pretty-1-dbfaa12f26f99277b8397379189172cf | 0 ...cribe_pretty-10-dbfaa12f26f99277b8397379189172cf | 0 ...cribe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 | 0 ...cribe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 | 0 ...cribe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 | 0 ...scribe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 | 0 ...cribe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 | 0 ...cribe_pretty-16-ada55b65b72e0d65563ad6161e005f22 | 0 ...scribe_pretty-2-713712e0f6f18144d1f3a522e9b98861 | 0 ...scribe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 | 0 ...scribe_pretty-4-1546db18568697fa68a7fc781802d255 | 0 ...scribe_pretty-5-ce1966d8626096821b404ab8745c4914 | 0 ...scribe_pretty-6-1546db18568697fa68a7fc781802d255 | 0 ...scribe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 | 0 ...scribe_pretty-8-1546db18568697fa68a7fc781802d255 | 0 ...scribe_pretty-9-e382a994134aefcd2652b57af9195644 | 0 ...scribe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a | 0 ...scribe_syntax-1-4f3d3497418242124113538edab45df7 | 0 ...cribe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 | 0 ...cribe_syntax-11-ab161e38c5d66b6c344c8372160ac74f | 0 ...cribe_syntax-12-90c7890e1aa28e94520f35f5679560a4 | 0 ...cribe_syntax-13-7c1216f9852d454bf93256e5a2588758 | 0 ...cribe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 | 0 ...cribe_syntax-15-e420b255509153b3326f00dcd25d11e4 | 0 ...cribe_syntax-16-5043ee273a313d28adeca74fd33739a7 | 0 ...cribe_syntax-17-c97a9e691cc08199678ead7f79d58b58 | 0 ...cribe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 | 0 ...cribe_syntax-19-c1c3359705e256d7641bbffab00c43fa | 0 ...scribe_syntax-2-b198700c0129910d6205ef063ee83d5a | 0 ...cribe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a | 0 ...cribe_syntax-21-5bac87eeb7e71928d01275b006720de3 | 0 ...cribe_syntax-22-719a15ffd0018bb2898f9045be886e0f | 0 ...cribe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 | 0 ...cribe_syntax-24-ee226b42db35b219702319858e925468 | 0 ...cribe_syntax-25-b6e10514fb473803c44bc793b9f9713e | 0 ...cribe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec | 0 ...scribe_syntax-3-458d6aaffeee94997f67a43b88382106 | 0 ...scribe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 | 0 ...scribe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 | 0 ...scribe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 | 0 ...scribe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d | 0 ...describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 | 0 ...scribe_syntax-9-fb744775fb30d92297534d29b6eafd6b | 0 ...be_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b | 0 ...be_table_json-1-e48b5f50bb2ff56f9886004366cfd491 | 0 ...be_table_json-2-9c36cac1372650b703400c60dd29042c | 0 ...be_table_json-3-576670af142149302decb9bf8662e68a | 0 ...be_table_json-4-4a57591b392bb8fd18238d068d191721 | 0 ...be_table_json-5-865aeeea2647a71f7f25b03da4203ffb | 0 ...be_table_json-6-ac49d26a0211b804fee89bbe0808f430 | 0 ...be_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 | 0 ...be_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 | 0 ...be_table_json-9-b05391400dc31139998dc3abaaf86320 | 0 ...input_formats-0-12652a5a33548c245772e8d0894af5ad | 0 ...input_formats-1-961f7cb386a6eacd391dcb189cbeddaa | 0 ...input_formats-2-28cd0f9b01baa8627a013339dc9508ce | 0 ...input_formats-3-c6eef43568e8ed96299720d30a6235e1 | 0 ..._input_formats-4-a4890f2b20715c75e05c674d9155a5b | 0 ..._format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c | 0 ..._format_check-1-ec4d7e2ce5bd580b24119860e96f376e | 0 ...e_format_check-2-d3e20a1484eabcd50e2039e55b4f549 | 0 ..._format_check-3-93063fb3476a400ecfec0bfd02cbc23f | 0 ..._format_check-4-fe6f402a026c882c2bc8f5251794dbbb | 0 ...pe_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a | 0 ...pe_change_off-1-ce3797dc14a603cba2a5e58c8612de5b | 0 ...pe_change_off-2-f5340880d2be7b0643eb995673e89d11 | 0 ...pe_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 | 0 ...pe_change_off-4-34064fd15c28dba55865cb8f3c5ba68c | 0 ...pe_change_off-5-f40a07d7654573e1a8517770eb8529e7 | 0 ...artition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab | 0 ...artition_dirs-1-ece80e0bd1236c547da7eceac114e602 | 0 ...drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 0 ...drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a | 0 .../drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 0 .../drop_index-1-5875a80dd89498c8b61db5f6bf26898 | 0 ...ti_partitions-0-c4449feb8f8e2c40c294ccf50680b47b | 0 ...ti_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 | 0 ...i_partitions-10-9120e865aa132bac6e0a29c907f0b760 | 0 ...ti_partitions-2-7554be9025c7683c67dce09177396357 | 0 ...lti_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 | 0 ...ti_partitions-4-9120e865aa132bac6e0a29c907f0b760 | 0 ...ti_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 | 0 ...ti_partitions-6-70ad97221d2be48259ea556f9d8e5353 | 0 ...ti_partitions-7-9120e865aa132bac6e0a29c907f0b760 | 0 ...ti_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 | 0 ...ti_partitions-9-3d4108847515b6386cd28b8862dcab53 | 0 ...itions_filter-0-6863a128b9a05e5c251ec0092e6124d3 | 0 ...itions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 | 0 ...tions_filter-10-119b0fe0597fa478e1533a412e2d444b | 0 ...tions_filter-11-83e3e422cdf9403523fa60d75376d7d7 | 0 ...tions_filter-12-e0e995c7bcc6b6d801b68878b6166835 | 0 ...tions_filter-13-83e3e422cdf9403523fa60d75376d7d7 | 0 ...tions_filter-14-f47427726dd7546c3c59a2ec53891bb3 | 0 ...tions_filter-15-83e3e422cdf9403523fa60d75376d7d7 | 0 ...tions_filter-16-fae7d99452ab708daa96c30f0b25c03b | 0 ...tions_filter-17-83e3e422cdf9403523fa60d75376d7d7 | 0 ...tions_filter-18-4553ba96e8c121b2e98966a67220a0fd | 0 ...tions_filter-19-83e3e422cdf9403523fa60d75376d7d7 | 0 ...itions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 | 0 ...tions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c | 0 ...tions_filter-21-83e3e422cdf9403523fa60d75376d7d7 | 0 ...tions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 | 0 ...tions_filter-23-45bb3dea1b8d4bd353885cd68729698e | 0 ...tions_filter-24-83e3e422cdf9403523fa60d75376d7d7 | 0 ...tions_filter-25-67d75c72ea2d3982c55f3a850d93f83c | 0 ...itions_filter-3-a499bb560b6e228b924387759214bc3c | 0 ...itions_filter-4-8fde1e5c12230f4b8081950dfd68b18d | 0 ...itions_filter-5-d3dc443408a20518c42b839fba218216 | 0 ...itions_filter-6-8cf34fc1e9004b8fdf43770f7893506e | 0 ...itions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 | 0 ...itions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 | 0 ...itions_filter-9-972bcd28a9e24b4cac70ef74524f1696 | 0 ...tions_filter2-0-322b270dab4032668de9002e9e8bc7c5 | 0 ...tions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 | 0 ...ions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 | 0 ...ions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a | 0 ...ions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 | 0 ...ions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c | 0 ...tions_filter2-2-e83e7a8f276f890c4eb29e192d684730 | 0 ...itions_filter2-3-31bf5686028f845040ae39acf642701 | 0 ...tions_filter2-4-a2c778112718207a10070596cb4595d8 | 0 ...tions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a | 0 ...tions_filter2-6-bbe938b3b20589283cc4541f3e417268 | 0 ...tions_filter2-7-74ed9df854eae5a025077b7012ef7b97 | 0 ...tions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 | 0 ...tions_filter2-9-185122a935af4fbe8466d7e39fc7648a | 0 ...tions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 | 0 ...tions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 | 0 ...ions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 | 0 ...ions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 | 0 ...ions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 | 0 ...ions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c | 0 ...tions_filter3-2-3e8e821dd63112223649b5d06febf7d9 | 0 ...tions_filter3-3-431228f63002f8b7d1364aa2a07f92ec | 0 ...tions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a | 0 ...tions_filter3-5-9a22b61cd027d740791ba847abc7e072 | 0 ...tions_filter3-6-6127604e4c55b13778cc56c0068ce6ae | 0 ...tions_filter3-7-688620ee5d61cce432e6c2d590b31404 | 0 ...tions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 | 0 ...tions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 | 0 ...ore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb | 0 ...re_protection-1-864c8c804db30687f4265ba081ca6368 | 0 ...re_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 | 0 ...ore_protection-3-312aa26fdea6da7907e3a91f75e36f1 | 0 ...re_protection-4-11f9ef9dd235f44b29c77abf7ca27881 | 0 ...re_protection-5-1283e970d6dc39e7a86e86af904bf116 | 0 .../drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 0 .../drop_table-1-afec243db5bd3a1b65d961e2325c6a57 | 0 .../drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 | 0 .../drop_table2-1-35229351a48427cf25b42ac8a61200fa | 0 .../drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 | 0 .../drop_table2-3-5f94efc9e658389a6d63553abd7a517f | 0 .../drop_table2-4-1c852531c1e75093c27911b45315ed62 | 0 .../drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 | 0 .../drop_table2-6-120f319d6031395a86c726e43d4ef678 | 0 .../drop_table2-7-35229351a48427cf25b42ac8a61200fa | 0 .../drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 | 0 .../drop_table2-9-120f319d6031395a86c726e43d4ef678 | 0 .../drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 0 .../drop_view-1-70a24b7e47d8527298241bcbec922cf5 | 0 ..._skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 | 0 ..._skip_default-1-16367c381d4b189b3640c92511244bfe | 0 ..._skip_default-2-d71f115b7d42f6c67de701bf69c617a9 | 0 ...n_skip_default-3-b7f2a424f616cfb015937e9ef980277 | 0 ..._skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 | 0 ..._skip_default-5-e394cdeb88f69b4d4b08450680f779b9 | 0 ..._skip_default-6-725ba4225501c1279f593b9c72eaca28 | 0 ..._skip_default-7-e707e693aa61edf87768fb71f6e936e1 | 0 ..._skip_default-8-725ba4225501c1279f593b9c72eaca28 | 0 ..._skip_default-9-3b57aa58995f862f2713624b50db5b65 | 0 ...enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 | 0 ...enforce_order-1-633034e3d966737cecf2befc5df1e35d | 0 ...enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 | 0 ...enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f | 0 .../enforce_order-4-3136edd49e681ea21aa35d0836eab65 | 0 ...enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 | 0 ...enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 | 0 ...enforce_order-7-5fb418612e7c2201790d6f318c715ccf | 0 ...enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 | 0 .../escape1-0-a4fb8359a2179ec70777aad6366071b7 | 0 .../escape1-1-683124e29877d2c5a96b95c8ddba97b7 | 0 .../escape1-2-395d5a528c5e7235a48b4ac90938e2d6 | 0 .../escape1-3-4267651148da591da38737028fdbd80 | 0 .../escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 | 0 .../escape1-5-70729c3d79ded87e884c176138174645 | 0 .../escape1-6-134b596abc363f0bfa7f770732ebb960 | 0 .../escape1-7-486585cbb4de5bc908dde4c601dd7c17 | Bin .../escape1-8-910536a438eec89c78bd611b3c4bb7e0 | 0 .../escape2-0-a4fb8359a2179ec70777aad6366071b7 | 0 .../escape2-1-683124e29877d2c5a96b95c8ddba97b7 | 0 .../escape2-10-13884d58efe80bd24862b3c54cb57c6e | 0 .../escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 .../escape2-3-1774adb1085f4ee6782a8dac0735399 | 0 .../escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe | 0 .../escape2-5-fd0c88ad6ad131a16d1b78adbea65800 | 0 .../escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 | 0 .../escape2-7-70729c3d79ded87e884c176138174645 | 0 .../escape2-8-134b596abc363f0bfa7f770732ebb960 | 0 .../escape2-9-486585cbb4de5bc908dde4c601dd7c17 | Bin ...pe_clusterby1-0-e34202f0d36c00a68722c802139d17cf | 0 ...pe_clusterby1-1-914091aa635a64c707c69e296eb097a5 | 0 ...distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb | 0 ..._distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 | 0 ...cape_orderby1-0-4057beace528a415308f7ca332f29941 | 0 ...cape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 | 0 ...scape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 | 0 ...scape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 | 0 ..._nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...nonpart_empty-1-baeaf0da490037e7ada642d23013075a | 0 ...nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 | 0 ...nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ...xim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...im_01_nonpart-1-baeaf0da490037e7ada642d23013075a | 0 ...im_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...im_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...im_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...im_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 | 0 ...im_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ..._00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...00_part_empty-1-baeaf0da490037e7ada642d23013075a | 0 ...00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 | 0 .../exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee | 0 .../exim_02_part-1-baeaf0da490037e7ada642d23013075a | 0 .../exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 .../exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 .../exim_02_part-4-88b581725ecdd603117a1706ab9c34dc | 0 .../exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 .../exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...rt_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...t_over_compat-1-baeaf0da490037e7ada642d23013075a | 0 ...t_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...t_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...t_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...t_over_compat-5-75eed21390055f8e397c81ab9d253a32 | 0 ...t_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ...im_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...m_04_all_part-1-baeaf0da490037e7ada642d23013075a | 0 ...m_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...m_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...m_04_all_part-4-88b581725ecdd603117a1706ab9c34dc | 0 ...m_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...m_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...m_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...m_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...m_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ..._evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...evolved_parts-1-baeaf0da490037e7ada642d23013075a | 0 ...volved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...volved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 | 0 ...evolved_parts-4-3c27502d4f6977b959e0928755b43be3 | 0 ...evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 | 0 ...evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 | 0 ...evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 | 0 ...evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 | 0 ...evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 | 0 ...m_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ..._05_some_part-1-baeaf0da490037e7ada642d23013075a | 0 ..._05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ..._05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ..._05_some_part-4-88b581725ecdd603117a1706ab9c34dc | 0 ..._05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ..._05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ..._05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ..._05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ..._05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...im_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...m_06_one_part-1-baeaf0da490037e7ada642d23013075a | 0 ...m_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...m_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...m_06_one_part-4-88b581725ecdd603117a1706ab9c34dc | 0 ...m_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...m_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...m_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...m_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...m_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...ver_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...er_nonoverlap-1-baeaf0da490037e7ada642d23013075a | 0 ...er_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...er_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...er_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc | 0 ...er_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...er_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...er_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...er_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...er_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...onpart_rename-1-baeaf0da490037e7ada642d23013075a | 0 ...onpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 | 0 ...onpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...onpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...onpart_rename-5-75eed21390055f8e397c81ab9d253a32 | 0 ...onpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ...pec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...ec_nonoverlap-1-baeaf0da490037e7ada642d23013075a | 0 ...ec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...ec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...ec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc | 0 ...ec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...ec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...ec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...ec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...ec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...ternal_managed-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...ernal_managed-1-baeaf0da490037e7ada642d23013075a | 0 ...ernal_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...ernal_managed-3-be31972099603addb71187f19f7cd25d | 0 ...ernal_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 | 0 ...naged_external-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...aged_external-1-baeaf0da490037e7ada642d23013075a | 0 ...aged_external-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...aged_external-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...aged_external-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...aged_external-5-75eed21390055f8e397c81ab9d253a32 | 0 ...aged_external-6-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ...ernal_location-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...rnal_location-1-baeaf0da490037e7ada642d23013075a | 0 ...rnal_location-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...rnal_location-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...rnal_location-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...rnal_location-5-75eed21390055f8e397c81ab9d253a32 | 0 ...rnal_location-6-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ...naged_location-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...aged_location-1-baeaf0da490037e7ada642d23013075a | 0 ...aged_location-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...aged_location-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...aged_location-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...aged_location-5-75eed21390055f8e397c81ab9d253a32 | 0 ...aged_location-6-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ..._over_existing-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...over_existing-1-baeaf0da490037e7ada642d23013075a | 0 ...over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 | 0 ...over_existing-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...over_existing-5-75eed21390055f8e397c81ab9d253a32 | 0 ...over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ..._external_part-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...external_part-1-baeaf0da490037e7ada642d23013075a | 0 ...external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...external_part-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...external_part-4-88b581725ecdd603117a1706ab9c34dc | 0 ...external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...external_part-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...external_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ..._part_external-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...part_external-1-baeaf0da490037e7ada642d23013075a | 0 ...part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...part_external-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...part_external-4-88b581725ecdd603117a1706ab9c34dc | 0 ...part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...part_external-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...part_external-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...7_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ..._part_managed-1-baeaf0da490037e7ada642d23013075a | 0 ..._part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ..._part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ..._part_managed-4-88b581725ecdd603117a1706ab9c34dc | 0 ..._part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ..._part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ..._part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ..._part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ..._part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ..._part_external-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...part_external-1-baeaf0da490037e7ada642d23013075a | 0 ...part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...part_external-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...part_external-4-88b581725ecdd603117a1706ab9c34dc | 0 ...part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...part_external-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...part_external-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...ernal_location-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...rnal_location-1-baeaf0da490037e7ada642d23013075a | 0 ...rnal_location-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...rnal_location-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...rnal_location-4-88b581725ecdd603117a1706ab9c34dc | 0 ...rnal_location-5-75f428bb2aa8624ac08095cdfd7a6993 | 0 ...rnal_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...rnal_location-7-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...ernal_location-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...rnal_location-1-baeaf0da490037e7ada642d23013075a | 0 ...rnal_location-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...rnal_location-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...rnal_location-4-88b581725ecdd603117a1706ab9c34dc | 0 ...rnal_location-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...rnal_location-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...rnal_location-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...rnal_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...rnal_location-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...naged_location-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...aged_location-1-baeaf0da490037e7ada642d23013075a | 0 ...aged_location-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...aged_location-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...aged_location-4-88b581725ecdd603117a1706ab9c34dc | 0 ...aged_location-5-93aba23b0fa5247d2ed67e5fa976bc0a | 0 ...aged_location-6-a14fc179cf3755a0aa7e63d4a514d394 | 0 ...aged_location-7-308a4e8e07efb2b777d9c7de5abab1d1 | 0 ...aged_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...aged_location-9-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...rt_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...t_authsuccess-1-baeaf0da490037e7ada642d23013075a | 0 ...t_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 | 0 ...t_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd | 0 ...t_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e | 0 ...t_authsuccess-5-760e902318ec521eed07cf23e0f256a2 | 0 ...t_authsuccess-6-75eed21390055f8e397c81ab9d253a32 | 0 ...t_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ...st_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...t_authsuccess-1-baeaf0da490037e7ada642d23013075a | 0 ...t_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...t_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 | 0 ...t_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...t_authsuccess-5-75eed21390055f8e397c81ab9d253a32 | 0 ...t_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ...rt_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...t_authsuccess-1-baeaf0da490037e7ada642d23013075a | 0 ...t_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...t_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 | 0 ...t_authsuccess-4-88b581725ecdd603117a1706ab9c34dc | 0 ...t_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 | 0 ...t_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 | 0 ...st_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee | 0 ...t_authsuccess-1-baeaf0da490037e7ada642d23013075a | 0 ...t_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 | 0 ...t_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 | 0 ...t_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd | 0 ...t_authsuccess-5-d2ec90909f243a767be1aa299720f45d | 0 ...t_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 | 0 ...h_aggregation-0-739356d010a919138911f295fac81607 | 0 ...ch_aggregation-1-8dc96d77daa18d053fab3c134028788 | 0 ...h_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 | 0 ...ileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 | 0 ...ileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b | 0 ...ileformat_mix-2-701660c0ea117b11d12de54dc661bc3e | 0 ...ileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 | 0 ...ileformat_mix-4-fcda187f1366ff93a113cbe670335198 | 0 ...ileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 | 0 ...ileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 | 0 ...oin_breaktask-0-5090cca7feb54de047bc535e234bd4a8 | 0 ...oin_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 | 0 ...oin_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 | 0 ...oin_breaktask-3-550e9b759fb088a81feddeff2e4be64e | 0 ...in_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 | 0 ...in_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd | 0 ...n_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd | 0 ...n_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 | 0 ...n_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 | 0 ...in_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c | 0 ...in_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 | 0 ...in_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 | 0 ...in_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba | 0 ...in_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 | 0 ...in_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 | 0 ...in_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e | 0 ...in_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 | 0 .../groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../groupby1-1-c67a488530dc7e20a9e7acf02c14380f | 0 .../groupby1-2-f90acd59ace31e16059bae52583188cc | 0 .../groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d | 0 .../groupby1-4-19094f229f8af852ef8dad406333ae08 | 0 .../groupby1-5-dd7bf298b8c921355edd8665c6b0c168 | 0 .../groupby1-6-977359ea0d2f429a9a35bbd8599cddea | 0 .../groupby1-7-c2c54378ffce53ade73a9dda783903e7 | 0 .../groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../groupby11-1-c67a488530dc7e20a9e7acf02c14380f | 0 .../groupby11-2-f06bf6a360c7f9425cffa237f00445d8 | 0 .../groupby11-3-f11e9a986df49215fb8aa17aaccc7087 | 0 .../groupby11-4-1ae3e153888f9ca44e92ef43aea19092 | 0 .../groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f | 0 .../groupby11-6-686ee4a5b2e24e51ba1d41b42215accd | 0 .../groupby11-7-149d359546ab38226ffeb023d7414b3d | 0 ...roupby1_limit-0-83c59d378571a6e487aa20217bd87817 | 0 ...roupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 | 0 ...roupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 | 0 ...roupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 | 0 ...roupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 | 0 .../groupby1_map-0-dbcec232623048c7748b708123e18bf0 | 0 .../groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 | 0 .../groupby1_map-2-83c59d378571a6e487aa20217bd87817 | 0 .../groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 | 0 .../groupby1_map-4-330a40b8e19028bbb370adf219f469fe | 0 .../groupby1_map-5-40f5168bfd9d124165bd207543b68a28 | 0 .../groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...by1_map_nomap-0-dbcec232623048c7748b708123e18bf0 | 0 ...by1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...pby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 | 0 ...by1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 | 0 ...by1_map_nomap-4-330a40b8e19028bbb370adf219f469fe | 0 ...by1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 | 0 ...by1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...pby1_map_skew-0-dbcec232623048c7748b708123e18bf0 | 0 ...pby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f | 0 ...pby1_map_skew-2-83c59d378571a6e487aa20217bd87817 | 0 ...pby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 | 0 ...pby1_map_skew-4-330a40b8e19028bbb370adf219f469fe | 0 ...pby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 | 0 ...pby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...oupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...oupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...oupby1_noskew-2-83c59d378571a6e487aa20217bd87817 | 0 ...oupby1_noskew-3-f90acd59ace31e16059bae52583188cc | 0 ...oupby1_noskew-4-19094f229f8af852ef8dad406333ae08 | 0 ...oupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea | 0 ...oupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 | 0 .../groupby2-0-43d53504df013e6b35f81811138a167a | 0 .../groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../groupby2-2-c67a488530dc7e20a9e7acf02c14380f | 0 .../groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b | 0 .../groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 | 0 .../groupby2-5-c0660310e5d882732d07cb76bc0a7939 | 0 .../groupby2-6-41dfb7b036cae4972b275a0262ea2e4c | 0 ...roupby2_limit-0-83c59d378571a6e487aa20217bd87817 | 0 ...roupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 | 0 ...roupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 | 0 .../groupby2_map-0-dbcec232623048c7748b708123e18bf0 | 0 .../groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 | 0 .../groupby2_map-2-83c59d378571a6e487aa20217bd87817 | 0 .../groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 | 0 .../groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 | 0 .../groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 | 0 .../groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...ulti_distinct-0-dbcec232623048c7748b708123e18bf0 | 0 ...ulti_distinct-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...ulti_distinct-2-83c59d378571a6e487aa20217bd87817 | 0 ...ulti_distinct-3-3aa4057488720c0f514696154f2070b5 | 0 ...ulti_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 | 0 ...ulti_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 | 0 ...ulti_distinct-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...pby2_map_skew-0-dbcec232623048c7748b708123e18bf0 | 0 ...pby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f | 0 ...pby2_map_skew-2-83c59d378571a6e487aa20217bd87817 | 0 ...pby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 | 0 ...pby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 | 0 ...upby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 | 0 ...pby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...oupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...oupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...oupby2_noskew-2-83c59d378571a6e487aa20217bd87817 | 0 ...oupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b | 0 ...oupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 | 0 ...oupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 | 0 ...oupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c | 0 ...ulti_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...ulti_distinct-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...ulti_distinct-2-83c59d378571a6e487aa20217bd87817 | 0 ...ulti_distinct-3-f64b982c4bf34931f03447e635ae33d2 | 0 ...ulti_distinct-4-2115f3e7d207621ce2b07b6e33563844 | 0 ...ulti_distinct-5-92891db0de9b8cd00892f0a790aff494 | 0 ...ulti_distinct-6-41dfb7b036cae4972b275a0262ea2e4c | 0 .../groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../groupby4-1-c67a488530dc7e20a9e7acf02c14380f | 0 .../groupby4-2-2a18d9570d9b676e240cda76df818c42 | 0 .../groupby4-3-132eabb30b8d011c848c531a6ab54529 | 0 .../groupby4-4-ac19a9a7f4a16763bfe7998179257933 | 0 .../groupby4-5-adc1ec67836b26b60d8547c4996bfd8f | 0 .../groupby4_map-0-dbcec232623048c7748b708123e18bf0 | 0 .../groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 | 0 .../groupby4_map-2-83c59d378571a6e487aa20217bd87817 | 0 .../groupby4_map-3-840759680be77463c13e4b19b2c1eb04 | 0 .../groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b | 0 .../groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 | 0 .../groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...pby4_map_skew-0-dbcec232623048c7748b708123e18bf0 | 0 ...pby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f | 0 ...pby4_map_skew-2-83c59d378571a6e487aa20217bd87817 | 0 ...pby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 | 0 ...pby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b | 0 ...pby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 | 0 ...pby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...oupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...oupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...oupby4_noskew-2-83c59d378571a6e487aa20217bd87817 | 0 ...oupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 | 0 ...oupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 | 0 ...oupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 | 0 ...oupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 0 .../groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../groupby5-1-c67a488530dc7e20a9e7acf02c14380f | 0 .../groupby5-2-13ab74a58da514fe01dbeda0c3e79883 | 0 .../groupby5-3-a773aeb40af0516f2527f8e9d6907420 | 0 .../groupby5-4-c4570c2676d599793e1e9ece32aa596e | 0 .../groupby5-5-adc1ec67836b26b60d8547c4996bfd8f | 0 .../groupby5_map-0-dbcec232623048c7748b708123e18bf0 | 0 .../groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 | 0 .../groupby5_map-2-83c59d378571a6e487aa20217bd87817 | 0 .../groupby5_map-3-840759680be77463c13e4b19b2c1eb04 | 0 .../groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f | 0 .../groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 | 0 .../groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...pby5_map_skew-0-dbcec232623048c7748b708123e18bf0 | 0 ...pby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f | 0 ...pby5_map_skew-2-83c59d378571a6e487aa20217bd87817 | 0 ...pby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 | 0 ...pby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f | 0 ...pby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 | 0 ...pby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...oupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...oupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...oupby5_noskew-2-83c59d378571a6e487aa20217bd87817 | 0 ...oupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 | 0 ...oupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 | 0 ...oupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e | 0 ...oupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 0 .../groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../groupby6-1-c67a488530dc7e20a9e7acf02c14380f | 0 .../groupby6-2-2a18d9570d9b676e240cda76df818c42 | 0 .../groupby6-3-ae61517a9997b80d512a9089cdb71fac | 0 .../groupby6-4-3f468a119e7975110b4063adb42c7dd9 | 0 .../groupby6-5-adc1ec67836b26b60d8547c4996bfd8f | 0 .../groupby6_map-0-dbcec232623048c7748b708123e18bf0 | 0 .../groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 | 0 .../groupby6_map-2-83c59d378571a6e487aa20217bd87817 | 0 .../groupby6_map-3-2a18d9570d9b676e240cda76df818c42 | 0 .../groupby6_map-4-ae61517a9997b80d512a9089cdb71fac | 0 .../groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 | 0 .../groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...pby6_map_skew-0-dbcec232623048c7748b708123e18bf0 | 0 ...pby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f | 0 ...pby6_map_skew-2-83c59d378571a6e487aa20217bd87817 | 0 ...pby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 | 0 ...pby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac | 0 ...pby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 | 0 ...pby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...oupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...oupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...oupby6_noskew-2-83c59d378571a6e487aa20217bd87817 | 0 ...oupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 | 0 ...oupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac | 0 ...oupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 | 0 ...oupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 0 .../groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../groupby7-1-c67a488530dc7e20a9e7acf02c14380f | 0 .../groupby7-2-3678fb67b5c739bd87d4907630da1208 | 0 .../groupby7-3-95474af63c0f92717ed49c3a0f37b10a | 0 .../groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 0 .../groupby7-5-ed76c0068780120a6f23feefee303403 | 0 .../groupby7-6-b2af91348c5fa9605702be50983c3bd2 | 0 .../groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 .../groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 | 0 .../groupby7_map-0-dbcec232623048c7748b708123e18bf0 | 0 .../groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e | 0 ...groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 ...groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 | 0 .../groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 | 0 .../groupby7_map-3-83c59d378571a6e487aa20217bd87817 | 0 .../groupby7_map-4-3678fb67b5c739bd87d4907630da1208 | 0 .../groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a | 0 .../groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 0 .../groupby7_map-7-ed76c0068780120a6f23feefee303403 | 0 .../groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd | 0 .../groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 | 0 ...ingle_reducer-0-dbcec232623048c7748b708123e18bf0 | 0 ...ingle_reducer-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...ngle_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 | 0 ...ingle_reducer-2-83c59d378571a6e487aa20217bd87817 | 0 ...ingle_reducer-3-3678fb67b5c739bd87d4907630da1208 | 0 ...ingle_reducer-4-95474af63c0f92717ed49c3a0f37b10a | 0 ...ingle_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 0 ...ingle_reducer-6-ed76c0068780120a6f23feefee303403 | 0 ...ingle_reducer-7-409f355bf35dfaa8b9e643510c58fabd | 0 ...ingle_reducer-8-b2af91348c5fa9605702be50983c3bd2 | 0 ...ingle_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 ...pby7_map_skew-0-dbcec232623048c7748b708123e18bf0 | 0 ...pby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f | 0 ...by7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 | 0 ...pby7_map_skew-2-83c59d378571a6e487aa20217bd87817 | 0 ...pby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 | 0 ...pby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a | 0 ...pby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 0 ...pby7_map_skew-6-ed76c0068780120a6f23feefee303403 | 0 ...pby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd | 0 ...pby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 | 0 ...pby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 ...oupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...oupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e | 0 ...upby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 ...upby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 | 0 ...oupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 | 0 ...oupby7_noskew-3-83c59d378571a6e487aa20217bd87817 | 0 ...oupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 | 0 ...oupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a | 0 ...oupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 0 ...oupby7_noskew-7-ed76c0068780120a6f23feefee303403 | 0 ...oupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd | 0 ...oupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 | 0 ...ingle_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...ingle_reducer-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...ngle_reducer-10-96d0598a2a4c17041a31b908d1f065e5 | 0 ...ingle_reducer-2-83c59d378571a6e487aa20217bd87817 | 0 ...ingle_reducer-3-3678fb67b5c739bd87d4907630da1208 | 0 ...ingle_reducer-4-95474af63c0f92717ed49c3a0f37b10a | 0 ...ingle_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 0 ...ingle_reducer-6-ed76c0068780120a6f23feefee303403 | 0 ...ingle_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 | 0 ...ingle_reducer-8-e404ba29e723df8bd8662d4f48129c7a | 0 ...ingle_reducer-9-652510b8dc20117c65511f06e6e73d73 | 0 .../groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../groupby8-1-c67a488530dc7e20a9e7acf02c14380f | 0 .../groupby8-10-83296fd5b5fc22af0c51514c4e67c95f | 0 .../groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 .../groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 | 0 .../groupby8-2-3678fb67b5c739bd87d4907630da1208 | 0 .../groupby8-3-95474af63c0f92717ed49c3a0f37b10a | 0 .../groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d | 0 .../groupby8-5-83296fd5b5fc22af0c51514c4e67c95f | 0 .../groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 .../groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 | 0 .../groupby8-8-7f98b724df05f51b3ec1f087a8da414e | 0 .../groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d | 0 .../groupby8_map-0-dbcec232623048c7748b708123e18bf0 | 0 .../groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 | 0 .../groupby8_map-2-83c59d378571a6e487aa20217bd87817 | 0 .../groupby8_map-3-3678fb67b5c739bd87d4907630da1208 | 0 .../groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a | 0 .../groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d | 0 .../groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f | 0 .../groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 .../groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 | 0 ...pby8_map_skew-0-dbcec232623048c7748b708123e18bf0 | 0 ...pby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f | 0 ...pby8_map_skew-2-83c59d378571a6e487aa20217bd87817 | 0 ...pby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 | 0 ...pby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a | 0 ...pby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d | 0 ...pby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f | 0 ...pby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 ...pby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 | 0 ...oupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...oupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...oupby8_noskew-2-83c59d378571a6e487aa20217bd87817 | 0 ...oupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 | 0 ...oupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a | 0 ...oupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d | 0 ...oupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f | 0 ...oupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 ...oupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 | 0 .../groupby9-0-3678fb67b5c739bd87d4907630da1208 | 0 .../groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 | 0 .../groupby9-10-7f98b724df05f51b3ec1f087a8da414e | 0 .../groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f | 0 .../groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 | 0 .../groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 .../groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 | 0 .../groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed | 0 .../groupby9-16-c05b1bc66a607e43633dc457ecf48f3b | 0 .../groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 .../groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 | 0 .../groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e | 0 .../groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f | 0 .../groupby9-20-ccc5914317422f569e8b7171a3b2b243 | 0 .../groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 .../groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 | 0 .../groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 | 0 .../groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 .../groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 | 0 .../groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e | 0 .../groupby9-7-ccc5914317422f569e8b7171a3b2b243 | 0 .../groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 .../groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 | 0 ...oupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e | 0 ...oupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 | 0 ...oupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 | 0 ...complex_types-0-de39f8b5f4305136d945da94d5222283 | 0 ...complex_types-1-10fe6134247226ab2b309bb62460d080 | 0 ...complex_types-2-2500cd8e85b71222253a05a979442a4a | 0 ...complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 | 0 ...ingle_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 | 0 ...ingle_reducer-1-de39f8b5f4305136d945da94d5222283 | 0 ...ingle_reducer-2-10fe6134247226ab2b309bb62460d080 | 0 ...ingle_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 | 0 ...tinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 | 0 ...tinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 | 0 ...tinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 | 0 ...tinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 | 0 ...tinct_samekey-4-d88c962262d3524bfc881b7309688e38 | 0 ...tinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad | 0 ...tinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 | 0 ...tinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 | 0 ...oupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 | 0 ...oupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...oupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 | 0 ...oupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 | 0 ...oupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 | 0 ...oupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a | 0 ...oupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...ulti_distinct-0-dbcec232623048c7748b708123e18bf0 | 0 ...ulti_distinct-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...ulti_distinct-2-83c59d378571a6e487aa20217bd87817 | 0 ...ulti_distinct-3-a04c523002e79c588e350486c815e785 | 0 ...ulti_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 | 0 ...ulti_distinct-5-2efbb90d3df011282b70623e0905c390 | 0 ...ulti_distinct-6-adc1ec67836b26b60d8547c4996bfd8f | 0 ...mmon_distinct-0-dbcec232623048c7748b708123e18bf0 | 0 ...mmon_distinct-1-388618d4d475de38d5c280fd7038730b | 0 ...mon_distinct-10-6aa66df624cd32601218eee200977ce6 | 0 ...mon_distinct-11-59a77127c166fef57504c011ccb427a6 | 0 ...mmon_distinct-2-2e1779fc58da364612a1c84d563ea7d5 | 0 ...mmon_distinct-3-febd68f62dd71550dd3db8335d1f93f7 | 0 ...mmon_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e | 0 ...mmon_distinct-5-6aa66df624cd32601218eee200977ce6 | 0 ...mmon_distinct-6-59a77127c166fef57504c011ccb427a6 | 0 ...mmon_distinct-7-293182ac89effb268855f5ac53e1ec11 | 0 ...mmon_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 | 0 ...mmon_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e | 0 ...ingle_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 | 0 ...ingle_reducer-1-f64b982c4bf34931f03447e635ae33d2 | 0 ...ngle_reducer-10-3b3ef4d975716744c85af560aa240abd | 0 ...ngle_reducer-11-4e95946ec07f04479da42ba5cbfa531b | 0 ...ngle_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 | 0 ...ngle_reducer-13-38f492067df78144c272bb212633cc5e | 0 ...ngle_reducer-14-d7b3676444a9e95811184637dd0b3231 | 0 ...ngle_reducer-15-3b3ef4d975716744c85af560aa240abd | 0 ...ngle_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 | 0 ...ngle_reducer-17-7f344c2f066d74ecd135c43d39658bae | 0 ...ngle_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab | 0 ...ngle_reducer-19-180b611e08d4080fa098ed69169c2478 | 0 ...ingle_reducer-2-392062177be62090adedf1ab6c0a0b78 | 0 ...ngle_reducer-20-db2149b42cdbf998187034233fc846cc | 0 ...ingle_reducer-21-322f23866cf3ca62d4ba93cf904c520 | 0 ...ngle_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f | 0 ...ingle_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 | 0 ...ingle_reducer-4-7df7c16e4063683d0ca40360da460799 | 0 ...ingle_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea | 0 ...ingle_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b | 0 ...ingle_reducer-7-ad39c0f14b6e0752489479138516bd57 | 0 ...ingle_reducer-8-38f492067df78144c272bb212633cc5e | 0 ...ingle_reducer-9-d7b3676444a9e95811184637dd0b3231 | 0 ...ngle_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 | 0 ...ngle_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada | 0 ...ngle_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 | 0 ...ngle_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e | 0 ...ngle_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 | 0 ...ngle_reducer2-5-376542befbcab97d864e874251720c40 | 0 ...ngle_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 | 0 ...ngle_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab | 0 ...ngle_reducer2-8-180b611e08d4080fa098ed69169c2478 | 0 ...ngle_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd | 0 ...ngle_reducer3-1-9e61989d717403353689cbbb2816210d | 0 ...gle_reducer3-10-73819ea1a7c0653a61652b3766afb003 | 0 ...gle_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 | 0 ...gle_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 | 0 ...gle_reducer3-13-521e0c1054cfa35116c02245874a4e69 | 0 ...gle_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 | 0 ...gle_reducer3-15-36e81eba0a6a42532d8ee147086d668a | 0 ...gle_reducer3-16-59b2c947e136092e7ca5019c96a9994b | 0 ...gle_reducer3-17-521e0c1054cfa35116c02245874a4e69 | 0 ...gle_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 | 0 ...ngle_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 | 0 ...ngle_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 | 0 ...ngle_reducer3-4-521e0c1054cfa35116c02245874a4e69 | 0 ...ngle_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 | 0 ...ngle_reducer3-6-36e81eba0a6a42532d8ee147086d668a | 0 ...ngle_reducer3-7-59b2c947e136092e7ca5019c96a9994b | 0 ...ngle_reducer3-8-521e0c1054cfa35116c02245874a4e69 | 0 ...ngle_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 | 0 ...pby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb | 0 ...pby_neg_float-1-4a435e268a327404f75725b82a32ee03 | 0 ...upby_position-0-422c2068a838f59324c1d9861225c824 | 0 ...upby_position-1-6b06902de5c0ca13cebe03018d86f447 | 0 ...pby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 | 0 ...pby_position-11-75a28e558d2fb7a78f43f55b0479c646 | 0 ...pby_position-12-5583e5cfcf8083d45a3bd80434c1829f | 0 ...pby_position-13-9ee2150594ad2eece6ee14424155d396 | 0 ...pby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb | 0 ...upby_position-2-627bb7be9c0edb5ba4c677912800d364 | 0 ...upby_position-3-c39bd6c3c24658ec082bef9876d6e2ac | 0 ...upby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c | 0 ...upby_position-5-e2470670c5d709001fb17ecbc6e4f85d | 0 ...upby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 | 0 ...upby_position-7-5b32a45af11e04b46f8566bd27a28014 | 0 ...upby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c | 0 ...upby_position-9-e2470670c5d709001fb17ecbc6e4f85d | 0 .../groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 | 0 .../groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 | 0 .../groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 | 0 .../groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a | 0 .../groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f | 0 ...ulti_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...ulti_distinct-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...ulti_distinct-2-3aa4057488720c0f514696154f2070b5 | 0 ...ulti_distinct-3-ace7b2624b125764e9f1f6b5559f023d | 0 ...multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d | 0 ...ulti_distinct-5-adc1ec67836b26b60d8547c4996bfd8f | 0 ...roupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...roupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...oupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...oupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 | 0 ...oupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c | 0 ...oupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 | 0 ...oupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 | 0 ...oupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea | 0 ...oupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 | 0 ...oupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...oupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 | 0 ...oupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 | 0 ...roupby_sort_1-2-fc30020d09151dc29be807795ad9475e | 0 ...oupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...oupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 | 0 ...oupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 | 0 ...oupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 | 0 ...oupby_sort_1-24-d53196339980a00a619788bd799a32e7 | 0 ...oupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 | 0 ...oupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 | 0 ...oupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f | 0 ...oupby_sort_1-28-c4ec0433a832ef551d70254957e3afca | 0 ...oupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 | 0 ...roupby_sort_1-3-fffea659b633b1f269b38556a7f54634 | 0 ...oupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 | 0 ...oupby_sort_1-31-d53196339980a00a619788bd799a32e7 | 0 ...oupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f | 0 ...oupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef | 0 ...oupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...oupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 | 0 ...oupby_sort_1-36-7871722f392f801a868e0e2fb372c610 | 0 ...oupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...oupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d | 0 ...oupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b | 0 ...roupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad | 0 ...oupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...oupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 | 0 ...oupby_sort_1-42-83889e7dc73d796cc869160b6b35102c | 0 ...oupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...oupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a | 0 ...oupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 | 0 ...oupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 | 0 ...oupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa | 0 ...oupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 | 0 ...oupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...roupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oupby_sort_1-50-461847b174096e7a255fb07cb35ab434 | 0 ...oupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 | 0 ...oupby_sort_1-52-c4ec0433a832ef551d70254957e3afca | 0 ...oupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 | 0 ...oupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c | 0 ...oupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 | 0 ...oupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 | 0 ...oupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 | 0 ...oupby_sort_1-58-e671e63f6b70094048563a9c33748c97 | 0 ...oupby_sort_1-59-c4ec0433a832ef551d70254957e3afca | 0 ...roupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b | 0 ...oupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 | 0 ...oupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 | 0 ...oupby_sort_1-62-c4ec0433a832ef551d70254957e3afca | 0 ...oupby_sort_1-63-dbcec232623048c7748b708123e18bf0 | 0 ...oupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e | 0 ...oupby_sort_1-65-83c59d378571a6e487aa20217bd87817 | 0 ...oupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 | 0 ...oupby_sort_1-67-b4fec0996399be2239961594897d6715 | 0 ...oupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 0 ...oupby_sort_1-69-ed76c0068780120a6f23feefee303403 | 0 ...roupby_sort_1-7-c0ea81b686236d661166912040a16ea7 | 0 ...oupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc | 0 ...oupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e | 0 ...oupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 | 0 ...oupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd | 0 ...oupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 | 0 ...oupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 | 0 ...oupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 | 0 ...oupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 | 0 ...groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 | 0 ...roupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 | 0 ...oupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...oupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...upby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 | 0 ...upby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 | 0 ...oupby_sort_10-2-fc30020d09151dc29be807795ad9475e | 0 ...oupby_sort_10-3-fffea659b633b1f269b38556a7f54634 | 0 ...oupby_sort_10-4-475d50465b23adfb70e67122425ede9e | 0 ...oupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 | 0 ...oupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 | 0 ...oupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 | 0 ...oupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 | 0 ...roupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c | 0 ...oupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...oupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...upby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d | 0 ...upby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 | 0 ...upby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 | 0 ...upby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 | 0 ...upby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 | 0 ...upby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 | 0 ...upby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 | 0 ...upby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 | 0 ...upby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...upby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 | 0 ...roupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 | 0 ...upby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d | 0 ...oupby_sort_11-3-fffea659b633b1f269b38556a7f54634 | 0 ...oupby_sort_11-4-475d50465b23adfb70e67122425ede9e | 0 ...oupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 | 0 ...roupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 | 0 ...oupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d | 0 ...oupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c | 0 ...oupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 | 0 ...roupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...roupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...oupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 | 0 ...roupby_sort_2-2-fc30020d09151dc29be807795ad9475e | 0 ...roupby_sort_2-3-fffea659b633b1f269b38556a7f54634 | 0 ...roupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 | 0 ...roupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...roupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b | 0 ...roupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 | 0 ...roupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 | 0 ...roupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed | 0 ...roupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...roupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...oupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 | 0 ...oupby_sort_3-11-2437764cdf86cd2d67430bd323346086 | 0 ...oupby_sort_3-12-bc5b02bd034ead563d39d2685087005e | 0 ...roupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd | 0 ...oupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f | 0 ...roupby_sort_3-2-fc30020d09151dc29be807795ad9475e | 0 ...roupby_sort_3-3-fffea659b633b1f269b38556a7f54634 | 0 ...roupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 | 0 ...roupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...roupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b | 0 ...roupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 | 0 ...groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 | 0 ...roupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd | 0 ...roupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...roupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...oupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...oupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 | 0 ...oupby_sort_4-12-1f082ad7217ad620063b58887b9b922f | 0 ...oupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 | 0 ...oupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 | 0 ...roupby_sort_4-2-fc30020d09151dc29be807795ad9475e | 0 ...roupby_sort_4-3-fffea659b633b1f269b38556a7f54634 | 0 ...roupby_sort_4-4-40891a8b7f896b11de173835c51aca4f | 0 ...roupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...roupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b | 0 ...roupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e | 0 ...roupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef | 0 ...roupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 | 0 ...roupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...roupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...oupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 | 0 ...oupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 | 0 ...oupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f | 0 ...oupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b | 0 ...oupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 | 0 ...oupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd | 0 ...oupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 | 0 ...oupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 | 0 ...oupby_sort_5-19-2e441f92547a956942f4451e943bf5cf | 0 ...roupby_sort_5-2-fc30020d09151dc29be807795ad9475e | 0 ...oupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b | 0 ...oupby_sort_5-22-70a6c959960c299db2b961dffaa2628d | 0 ...oupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f | 0 ...roupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd | 0 ...oupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f | 0 ...oupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 | 0 ...roupby_sort_5-3-fffea659b633b1f269b38556a7f54634 | 0 ...roupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 | 0 ...roupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...roupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b | 0 ...roupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 | 0 ...roupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 | 0 ...roupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd | 0 ...roupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...roupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...oupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 | 0 ...oupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 | 0 ...oupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...oupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 | 0 ...oupby_sort_6-14-5574207f68aac30a893785c50c735864 | 0 ...oupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...roupby_sort_6-2-fc30020d09151dc29be807795ad9475e | 0 ...roupby_sort_6-3-fffea659b633b1f269b38556a7f54634 | 0 ...roupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e | 0 ...roupby_sort_6-5-c0ea81b686236d661166912040a16ea7 | 0 ...roupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 | 0 ...roupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 | 0 ...roupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd | 0 ...roupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 | 0 ...roupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...roupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...oupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 | 0 ...oupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 | 0 ...roupby_sort_7-2-fc30020d09151dc29be807795ad9475e | 0 ...roupby_sort_7-3-fffea659b633b1f269b38556a7f54634 | 0 ...roupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 | 0 ...roupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb | 0 ...roupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 | 0 ...roupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 | 0 ...roupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b | 0 ...roupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 | 0 ...roupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...roupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...oupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 | 0 ...oupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d | 0 ...oupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 | 0 ...roupby_sort_8-2-fc30020d09151dc29be807795ad9475e | 0 ...roupby_sort_8-3-fffea659b633b1f269b38556a7f54634 | 0 ...roupby_sort_8-4-14283ed438d96ad881025b969c8fb69f | 0 ...roupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb | 0 ...roupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 | 0 ...roupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d | 0 ...roupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d | 0 ...roupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a | 0 ...roupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...roupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...oupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 | 0 ...roupby_sort_9-2-fc30020d09151dc29be807795ad9475e | 0 ...roupby_sort_9-3-fffea659b633b1f269b38556a7f54634 | 0 ...roupby_sort_9-4-14283ed438d96ad881025b969c8fb69f | 0 ...roupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb | 0 ...roupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 | 0 ...roupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c | 0 ...roupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 | 0 ...groupby_sort_9-9-feec69facdc973a0ff78455f766845c | 0 ...y_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...y_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ..._sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 | 0 ..._sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd | 0 ..._sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 | 0 ..._sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c | 0 ..._sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 | 0 ..._sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 | 0 ..._sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea | 0 ..._sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 | 0 ..._sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd | 0 ..._sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 | 0 ...y_sort_skew_1-2-fc30020d09151dc29be807795ad9475e | 0 ..._sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 | 0 ..._sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd | 0 ..._sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 | 0 ..._sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 | 0 ..._sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 | 0 ..._sort_skew_1-25-d53196339980a00a619788bd799a32e7 | 0 ..._sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 | 0 ..._sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 | 0 ..._sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f | 0 ..._sort_skew_1-29-c4ec0433a832ef551d70254957e3afca | 0 ...y_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 | 0 ..._sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 | 0 ..._sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 | 0 ..._sort_skew_1-32-d53196339980a00a619788bd799a32e7 | 0 ..._sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f | 0 ..._sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef | 0 ..._sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd | 0 ..._sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 | 0 ..._sort_skew_1-37-7871722f392f801a868e0e2fb372c610 | 0 ..._sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd | 0 ..._sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d | 0 ...y_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f | 0 ..._sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b | 0 ..._sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd | 0 ..._sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 | 0 ..._sort_skew_1-43-83889e7dc73d796cc869160b6b35102c | 0 ..._sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd | 0 ..._sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a | 0 ..._sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 | 0 ..._sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 | 0 ..._sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa | 0 ..._sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 | 0 ...y_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad | 0 ..._sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd | 0 ..._sort_skew_1-51-461847b174096e7a255fb07cb35ab434 | 0 ..._sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 | 0 ..._sort_skew_1-53-c4ec0433a832ef551d70254957e3afca | 0 ..._sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 | 0 ..._sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c | 0 ..._sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 | 0 ..._sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 | 0 ..._sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 | 0 ..._sort_skew_1-59-e671e63f6b70094048563a9c33748c97 | 0 ...y_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ..._sort_skew_1-60-c4ec0433a832ef551d70254957e3afca | 0 ..._sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 | 0 ..._sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 | 0 ..._sort_skew_1-63-c4ec0433a832ef551d70254957e3afca | 0 ..._sort_skew_1-64-dbcec232623048c7748b708123e18bf0 | 0 ..._sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e | 0 ..._sort_skew_1-66-83c59d378571a6e487aa20217bd87817 | 0 ..._sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 | 0 ..._sort_skew_1-68-b4fec0996399be2239961594897d6715 | 0 ..._sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 0 ...y_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b | 0 ..._sort_skew_1-70-ed76c0068780120a6f23feefee303403 | 0 ..._sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc | 0 ..._sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e | 0 ..._sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 | 0 ..._sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd | 0 ..._sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 | 0 ..._sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 | 0 ..._sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 | 0 ..._sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 | 0 ...y_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 | 0 ...by_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 | 0 ...y_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...y_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...y_sort_test_1-2-fc30020d09151dc29be807795ad9475e | 0 ...y_sort_test_1-3-fffea659b633b1f269b38556a7f54634 | 0 ...y_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a | 0 ...y_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad | 0 ...y_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...y_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b | 0 ...y_sort_test_1-8-c0ea81b686236d661166912040a16ea7 | 0 ...y_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 | 0 .../golden/hash-0-a658b129316d666d4b01c1581eed1c1f | 0 ...ok_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf | 0 ...ok_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 | 0 ...mplicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 | 0 ...mplicit_cast1-1-66f185b6fdccddba498c463641e7dc7a | 0 ...mplicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 | 0 ...ndex_creation-0-f880114c33c99a5f23c1465fd88f0db3 | 0 ...ndex_creation-1-a8bc76559014d9cdf07184208d582d25 | 0 ...dex_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 | 0 ...dex_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 | 0 ...dex_creation-12-9cc02e06c6051810c50e225bb2c66669 | 0 ...dex_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 | 0 ...dex_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 | 0 ...ndex_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 | 0 ...dex_creation-16-ffa6d3fcef97b7322dd3759d4a70881d | 0 ...dex_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad | 0 ...dex_creation-18-bcdb19db031341c4a50264ccf49328e4 | 0 ...dex_creation-19-98dbf83283f9e073e88ba770ec5a707f | 0 ...index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 | 0 ...dex_creation-20-68c5c98581c683b17ceaf1c0fd192871 | 0 ...dex_creation-21-4c8f6b48c437bf0be109fc0be1dc840e | 0 ...ndex_creation-22-f36cb2eed39691ca949b25182e2dd31 | 0 ...dex_creation-23-e7f21f556d3aa2bedb0717a167720741 | 0 ...dex_creation-24-8cb102bd5addf7fece0e2691468bc3bf | 0 ...dex_creation-25-33c48966230b934ae8ddf74ff18bb9ca | 0 ...dex_creation-26-f85db55b3f63ae186a1b6d5cec545939 | 0 ...dex_creation-27-e4856f13692e63d61f72aaf75e50e5f1 | 0 ...dex_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe | 0 ...dex_creation-29-ee8d287111069805c41c9c0032adc46f | 0 ...ndex_creation-3-14b999fc6dfb10a3632afe14e08003e1 | 0 ...dex_creation-30-f880114c33c99a5f23c1465fd88f0db3 | 0 ...dex_creation-31-a8bc76559014d9cdf07184208d582d25 | 0 ...ndex_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 | 0 ...dex_creation-33-14b999fc6dfb10a3632afe14e08003e1 | 0 ...dex_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 | 0 ...dex_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 | 0 ...dex_creation-36-21bcf37075b02097f16c8fc8130a83b8 | 0 ...dex_creation-37-9334418431eca405f13206bd8db42a1b | 0 ...dex_creation-38-f1f56119aede4f42221a68f6aaa42a26 | 0 ...dex_creation-39-489b4ceb2f4301a7132628303f99240d | 0 ...ndex_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 | 0 ...ndex_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 | 0 ...ndex_creation-6-21bcf37075b02097f16c8fc8130a83b8 | 0 ...ndex_creation-7-9334418431eca405f13206bd8db42a1b | 0 ...ndex_creation-8-f1f56119aede4f42221a68f6aaa42a26 | 0 ...ndex_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c | 0 .../index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 | 0 .../index_serde-1-f92d6c66d21791c11d2a822df04c1b63 | 0 .../index_serde-10-123301a057d4a46072d0431e00e20c4b | 0 .../index_serde-11-309e916d683a1a12ab62565697cb0046 | 0 .../index_serde-12-d590fd7cb9d433143de490d75686dd4 | 0 .../index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c | 0 .../index_serde-3-afcf2a156ccd4f79a0489b4593908d79 | 0 .../index_serde-4-d7547751c37375a9238043bbe250e716 | 0 .../index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 | 0 .../index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae | 0 .../index_serde-7-3b03210f94ec40db9ab02620645014d1 | 0 .../index_serde-8-35f48c7d6fa164bb84643657bc9280a8 | 0 .../index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 | 0 ...er_const_type-0-e231c5154b18cbc0baa082a7461dd13e | 0 ...er_const_type-1-c836a98522549d2a3fd43998afd8ae94 | 0 ...er_const_type-2-d8590c7336ae771b7a685bb544e8d2bd | 0 .../innerjoin-0-43d53504df013e6b35f81811138a167a | 0 .../innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e | 0 .../innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 | 0 .../innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 | 0 .../innerjoin-4-35c7611180562dcd9dab834f41654095 | 0 .../innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a | 0 .../innerjoin-6-17c49c593f6160e096b30dfee7b643ce | 0 .../innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 | 0 .../innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 | 0 .../innerjoin-9-326eeac56676d78fe489c464cddb526b | 0 .../inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 | 0 .../inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 | 0 .../golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 | 0 .../golden/input-1-6558e385bb08991302a72076dd7b7ff5 | 0 .../input0-0-fb47f8a8a8e3213f3fe47825a556c593 | 0 .../input0-1-efefd4364cd2790447fb0f908e87501f | 0 .../input1-0-28c9f2913833d6911f22e2e2e8c60f68 | 0 .../input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 | 0 .../input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 | 0 .../input10-0-acdd72fda64c5463ce29f31020012d42 | 0 .../input10-1-6970b6d2d451612b59fccbfd7ec68f74 | 0 .../input10-2-73f00da5cfc254745d1d80f913eb6449 | 0 .../input11-0-13ab74a58da514fe01dbeda0c3e79883 | 0 .../input11-1-3e66f12ae780a85721fa21c42fb3c8fb | 0 .../input11-2-24fe55bae88ad4a8e240376d012bc491 | 0 .../input11-3-adc1ec67836b26b60d8547c4996bfd8f | 0 ...input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 | 0 ...input11_limit-2-47fdba9764a8851379e2ed7e16c54583 | 0 ...input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 | 0 .../input12-0-9b141c1e5917ca82c6bc36a9a2950a1e | 0 .../input12-1-2b9ccaa793eae0e73bf76335d3d6880 | 0 .../input12-10-4d9eb316259a8e7ed6627bc27a639f7c | 0 .../input12-2-bab89dfffa77258e34a595e0e79986e3 | 0 .../input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 | 0 .../input12-4-f69ce1f5b3e0e77d1c487877580e6a23 | 0 .../input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf | 0 .../input12-6-f6ae011490e0c8c037927767ad966ce4 | 0 .../input12-7-16da1c1293626b943343443368679e9c | 0 .../input12-8-adc1ec67836b26b60d8547c4996bfd8f | 0 .../input12-9-3d08dc27c1a133c2497fc554c0d169bd | 0 ...ut12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f | 0 ...ut12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 | 0 ...ut12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 | 0 ...ut12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 | 0 ...ut12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf | 0 ...ut12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 | 0 ...ut12_hadoop20-6-16da1c1293626b943343443368679e9c | 0 ...ut12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f | 0 ...ut12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd | 0 ...ut12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c | 0 .../input14-0-13ab74a58da514fe01dbeda0c3e79883 | 0 .../input14-1-d0a1caeeeb79580713b7ecc28543f74a | 0 .../input14-2-1d791beabaa5288ea2fcf6b3675eda26 | 0 .../input14-3-adc1ec67836b26b60d8547c4996bfd8f | 0 ...input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf | 0 ...input14_limit-2-780cdc89e0e736790124b6bdac827951 | 0 ...input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../input16_cc-0-43d53504df013e6b35f81811138a167a | 0 .../input16_cc-1-5180e975a6babd51752706f1799e7df5 | 0 .../input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 | 0 .../input17-0-13ab74a58da514fe01dbeda0c3e79883 | 0 .../input17-1-be9cde5e769f171f60f61a7739de8f17 | 0 .../input17-2-21166e268096f6ec67f4f57ec333e901 | 0 .../input17-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../input19-0-a8eaa79d3a2762cdb263bc822d62df3a | 0 .../input19-1-f2832e249ab28bb3fb8e472012c5ffc | 0 .../input19-2-5a804e02e4419e785d15e7f39d6c5730 | 0 .../input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 | 0 .../input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 | 0 .../input1_limit-2-c1d996fc96189d323f6af180708abc12 | 0 .../input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e | 0 .../input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 | 0 .../input1_limit-5-eaaf713833e28a803c798562c7d6cd23 | 0 .../input2-0-eaf4ec8905774e4be6ea3fa4f32a456c | 0 .../input2-1-e0efeda558cd0194f4764a5735147b16 | 0 .../input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd | 0 .../input2-3-45b9f786f64afa5e039d1856fe926649 | 0 .../input2-4-235f92683416fab031e6e7490487b15b | 0 .../input2-5-9c36cac1372650b703400c60dd29042c | 0 .../input2-6-4a6b9674c126337f71834f11613f996d | 0 .../input2-7-9c36cac1372650b703400c60dd29042c | 0 .../input2-8-28c3fc507cd2e02a3a39331fc9c95334 | 0 .../input2-9-48bf8c06ed0a264d0863613fe79795e1 | 0 .../input21-0-7f40c525398953892ffd6283f54ba427 | 0 .../input21-1-70b803742328eacc69eb1ed044a5c6b8 | 0 .../input21-2-a4b3aeb45ae0cf38777e126faed0eff7 | 0 .../input21-3-9809b74435cbaedef0dc6e6b88b180fe | 0 .../input22-0-b9cd4041ca510639838a97376747b498 | 0 .../input22-1-b7f46eccd104e6ed1b29e2de45089f66 | 0 .../input22-2-10e46b53620b6b05019da4e58f53e6c3 | 0 .../input22-3-8285c1934441b12f6c016e13cb410e79 | 0 .../input23-0-c983ab8049996ad5e1e0296b51ae81cf | 0 .../input23-1-c8e32187d09172eb32b0480fcd28cccb | 0 .../input24-0-f8116598acadb7589ae02b13b65e1ad8 | 0 .../golden/input24-1-f57485de80ad64f9931bf92514fa1 | 0 .../input24-2-3fc20939000a5324a928feeb673a1ee2 | 0 .../input24-3-3189f3b2990de94619b9cb583d9dd3c5 | 0 .../input25-0-f8116598acadb7589ae02b13b65e1ad8 | 0 .../golden/input25-1-f57485de80ad64f9931bf92514fa1 | 0 .../input25-2-74a4b9d59ee26b650f567c07e9103b66 | 0 .../input25-3-d4940dd537ae72d234ffea23f8f0b103 | 0 .../input25-4-72c74c55dd7d85d194448e9c58488938 | 0 .../input26-0-1b6e9e73dd7ee60644c0997b21f48430 | 0 .../input26-1-8272225744e83ba4cbe158a5c113fce3 | 0 .../input28-0-b85d5edb4640c4b154e91e9a1beeef8a | 0 .../golden/input28-1-f57485de80ad64f9931bf92514fa1 | 0 .../input28-2-e88e8d960eeb128fbc6642274780ce4c | 0 .../input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 | 0 .../input2_limit-0-44591b07e007def036cea6c61a3f0a7f | 0 .../input2_limit-1-fed7e0bb996623da7dd17793e835f785 | 0 .../input3-0-2c80ec90d4d2c9c7446c05651bb76bff | 0 .../input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d | 0 .../input3-10-10a1a8a97f6417c3da16829f7e519475 | 0 .../input3-11-9c36cac1372650b703400c60dd29042c | 0 .../input3-12-a22d09de72e5067a0a94113cdecdaa95 | 0 .../input3-13-23bbec31affef0d758bc4a40490e0b9a | 0 .../input3-14-efee6816e20fe61595a4a2a991071219 | 0 .../input3-2-fa2aceba8cdcb869262e8ad6d431f491 | 0 .../input3-3-1c5990b1aed2be48311810dae3019994 | 0 .../input3-4-9c36cac1372650b703400c60dd29042c | 0 .../input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 | 0 .../input3-6-ba8c440158c2519353d02471bfb05694 | 0 .../input3-7-1c5990b1aed2be48311810dae3019994 | 0 .../input3-8-4dc0fefca4d158fd2ab40551ae9e35be | 0 .../input3-9-5076c1c35053b09173f6acdf1b5e9d6e | 0 .../input30-0-582c5fcbe2fe12cc8e7b21225583d96c | 0 .../input30-1-90c0d1a75de78c405413fd627caea4ab | 0 .../input30-2-823920925ca9c8a2ca9016f52c0f4ee | 0 .../input30-3-c21dba410fb07a098f93430a9d21df79 | 0 .../input30-4-f0ebd08e7675b19ae831824ef4d9e223 | 0 .../input30-5-38734677c27b5f90a8df5da6e6351c76 | 0 .../input30-6-f120ac8c87db9eebb5da7ac99b48600 | 0 .../input30-7-95d10d459c088d5fbefd00bdd8d44c3f | 0 .../input31-0-823920925ca9c8a2ca9016f52c0f4ee | 0 .../input31-1-c21dba410fb07a098f93430a9d21df79 | 0 .../input31-2-705764f8f7cab9378964af30b83f7fe | 0 .../input31-3-50c905261882f7fd8539fdd91e68151f | 0 .../input31-4-2f886fa357df9342733551fa1b53f913 | 0 .../input32-0-823920925ca9c8a2ca9016f52c0f4ee | 0 .../input32-1-c21dba410fb07a098f93430a9d21df79 | 0 .../input32-2-1ba7748b3d2f8908c2e81771ab229316 | 0 .../input32-3-b0070890240c15d647af59f41b77ba3d | 0 .../input32-4-d0e1378a30e21e0198e47d9b668ee1f6 | 0 .../input32-5-8789d32fc5b784fe2d171566732c573e | 0 .../input37-0-86e2e274650fb56651607ea10d356fc0 | 0 .../input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 | 0 .../input39-0-7bd12162381231be9d578797818957a7 | 0 .../input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 | 0 .../input39-10-28bf1b34c04a048da339eddd4c1fd779 | 0 .../input39-11-6d0814cbb72eb96bfc75c95d06f1e528 | 0 .../input39-12-f120ac8c87db9eebb5da7ac99b48600 | 0 .../input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 | 0 .../input39-14-bcc1d5fd287f81bac1092a913b09956d | 0 .../input39-2-ee667c6ab003bb83f6bf1c72153eba39 | 0 .../input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e | 0 .../input39-4-53453776bf062f28d371fc7336b7eae2 | 0 .../input39-5-823920925ca9c8a2ca9016f52c0f4ee | 0 .../input39-6-763ab5853bff619e6525c01e46b2a923 | 0 .../input39-7-9b141c1e5917ca82c6bc36a9a2950a1e | 0 .../input39-8-2b9ccaa793eae0e73bf76335d3d6880 | 0 .../input39-9-bab89dfffa77258e34a595e0e79986e3 | 0 ...ut39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe | 0 ...ut39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 | 0 ...t39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 | 0 ...ut39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 | 0 ...t39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 | 0 ...ut39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 | 0 ...ut39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e | 0 ...ut39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 | 0 ...put39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee | 0 ...ut39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 | 0 ...ut39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f | 0 ...ut39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 | 0 ...ut39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 | 0 .../input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 | 0 .../input3_limit-1-7b46b8baf9c7628da9c190c96b917057 | 0 .../input3_limit-2-c6583bdb759c8a050238a32a6ce8273d | 0 .../input3_limit-3-2a87d8faa18a6311376812bd0453fece | 0 .../input3_limit-4-70dad45d534146923fce88b2ffb99b0d | 0 .../input3_limit-5-3664b564747487df13a5d109837219b5 | 0 .../input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 | 0 .../input4-0-b9cd4041ca510639838a97376747b498 | 0 .../input4-1-c139adc70f9942e527142e3be7fd2b87 | 0 .../input4-2-b7f46eccd104e6ed1b29e2de45089f66 | 0 .../input4-3-d9280be6c9dd1e2263f8d50f0f43a486 | 0 .../input4-4-271b04e0fbee2ee81bae21dcb46d55e4 | 0 .../input40-0-37e74908fd43254141ae3664f13a48d | 0 .../input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 | 0 .../input40-2-e7ab3c9244fcfda76061b4153d796e93 | 0 .../input40-3-12f634800ffae17942ddd789a01af227 | 0 .../input40-4-f241eafbb8d5da3f9c1737aed7b4f94e | 0 .../input40-5-ccdc9c87a8fdbeee0fa48927f9700361 | 0 .../input40-6-93a0c37189dfe2318ba6ad84616f0d64 | 0 .../input40-7-ccdc9c87a8fdbeee0fa48927f9700361 | 0 .../input41-0-763ab5853bff619e6525c01e46b2a923 | 0 .../input41-1-8112b3a278e8337dbd6f017ae178658b | 0 .../input41-2-61812b3093413d09f0fd2372c61f7d53 | 0 .../input41-3-526399455dc5ecd4ea9f676b09fafeee | 0 .../input43-0-2baba8070f3585debc14b6bb3c83607a | 0 .../input46-0-b0cdbecce0321ac452c8e13e1bfc6924 | 0 .../input46-1-1efdd5ebfa732abdedeb10467ca71f7f | 0 .../input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca | 0 .../input46-3-c185163787977498a4b84f39f983c431 | 0 .../input46-4-7f05cde078d90c25780a7d5491b20c27 | 0 .../input46-5-f5c502e88a3dc3edb37b04af7d7955ab | 0 .../input49-0-2bd546beeb607da5b925ffbea20a741c | 0 .../input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d | 0 .../input49-2-d56860e68d333bb6132ef5ed0327eb1 | 0 .../input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 | 0 ...put4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 | 0 ...put4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 | 0 ...put4_cb_delim-2-e52787bf798a941c854eb09d75efe617 | 0 .../input4_limit-0-4f0124854141b8be1defa7a6d0877d8d | 0 .../input4_limit-1-c634fc723fb3aac3ce007069bdcb2af | 0 .../input5-0-659e06570690cceeb3f37e10e855d2ea | 0 .../input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 | 0 .../input5-2-3d6eb15b4fe23d0a1aa303da818d97ad | 0 .../input5-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../input6-0-659e06570690cceeb3f37e10e855d2ea | 0 .../input6-1-21149162906e31964a328b6cb2d5286e | 0 .../input6-2-9601ace6dce45dc8d6281cc248c9e28c | 0 .../input6-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../input7-0-8daa7823607b82336736a9d4f9a2ce45 | 0 .../input7-1-bb25de1b9a7791ec5609303bab350da0 | 0 .../input7-2-c740098a289f9a4267d26fc10347a0dc | 0 .../input7-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa | 0 .../input8-1-c9f9239e73b04fc5c0e0219f438ceefa | 0 .../input8-2-4a4cb89ba45d26bb3dd73e44620578ba | 0 .../input8-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 | 0 .../input9-1-b30105391cb53915997a84ca3137bd0a | 0 .../input9-2-171ea277f05a642699a0f91bacffccd8 | 0 .../input9-3-adc1ec67836b26b60d8547c4996bfd8f | 0 ...columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e | 0 ..._columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 | 0 ...columnarserde-2-ac60752a883d3204c215fa01811701be | 0 ...columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 | 0 ...columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 | 0 ..._dynamicserde-0-92c95af00fd419aa106571f72fcad67d | 0 ..._dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc | 0 ..._dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 | 0 ..._dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f | 0 ..._dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 | 0 ...put_lazyserde-0-92c95af00fd419aa106571f72fcad67d | 0 ...put_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 | 0 ...ut_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 | 0 ...ut_lazyserde-11-6aaa88142e86a9a39e980caed145e32c | 0 ...ut_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c | 0 ...put_lazyserde-2-7a165139976654640c86db8e4e5871cc | 0 ...put_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 | 0 ...put_lazyserde-4-8cc058fb7986f59965976cad813267db | 0 ...put_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 ...nput_lazyserde-6-42e2838ee69484bf5301475905cee12 | 0 ...put_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 | 0 ...put_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c | 0 ...put_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 .../input_limit-0-b2b4b45519484c8bac49d07debf678b9 | 0 .../input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 | 0 .../input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 | 0 .../input_part0-1-1aa1486a207bedc275035acc3b37cbdb | 0 .../input_part1-0-55f89c73a00f0f23ee04239ad9fc909 | 0 .../input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 | 0 .../input_part1-2-4153379704a43162abf9dbdbd79a92bc | 0 .../input_part1-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../input_part10-0-77d7f7c938d7827a2135d9391376a518 | 0 .../input_part10-1-35f67c0112026170d015b5f80a254bde | 0 .../input_part10-2-40069c199502c2724ac2a2733f964248 | 0 .../input_part10-3-48b242bc305c9bf879e083fa11edc967 | 0 .../input_part10-4-d0ba28297a8b73569d93605aa890aa09 | 0 ...ut_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf | 0 ...ut_part10_win-1-35f67c0112026170d015b5f80a254bde | 0 ...ut_part10_win-2-40069c199502c2724ac2a2733f964248 | 0 ...ut_part10_win-3-48b242bc305c9bf879e083fa11edc967 | 0 ...ut_part10_win-4-d0ba28297a8b73569d93605aa890aa09 | 0 .../input_part2-0-55f89c73a00f0f23ee04239ad9fc909 | 0 .../input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e | 0 .../input_part2-2-a8684c5574f90c3db2fc8a269a7556cd | 0 .../input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 | 0 .../input_part2-4-93c97e1760e0d41b3791d6f08010a665 | 0 .../input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 | 0 .../input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff | 0 .../input_part3-1-ba5256285fb22a43b491253a2d519730 | 0 .../input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef | 0 .../input_part4-1-88be7d93e9a03ea76085111c18d437a5 | 0 .../input_part5-0-679bf51b7de8df94fd2928744e887e3b | 0 .../input_part5-1-705301f0bf4fe9758e9f919d129dbc7c | 0 .../input_part5-2-d678533566fba4b922f01284538ca484 | 0 .../input_part5-3-e4419c33287ca1f48a43f61cca5b5928 | 0 .../input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d | 0 .../input_part6-1-9ffee7a86f484eab399266efa6e086f6 | 0 .../input_part7-0-14448a12627b8bccf6828032f938f39e | 0 .../input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 | 0 .../input_part8-0-f60652796af548135f9e104b456840e3 | 0 .../input_part8-1-60b409a520999ba50e8b7c6e30de3474 | 0 .../input_part9-0-726907a7cf62f5a79466caa0fa6eca01 | 0 .../input_part9-1-e60c60afc073367464898b8396e8f643 | 0 ...tsequencefile-0-68975193b30cb34102b380e647d8d5f4 | 0 ...tsequencefile-1-1c0f3be2d837dee49312e0a80440447e | 0 ...tsequencefile-2-186e4009966778e765b18d6476cf5abf | 0 ...tsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 | 0 ...tsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 | 0 ...tsequencefile-5-3708198aac609695b22e19e89306034c | 0 ...tsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 | 0 ...put_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 | 0 ...put_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f | 0 ...nput_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 | 0 ...put_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f | 0 ...ut_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 | 0 ...ut_testxpath2-1-27b77465d23aebf66a50c8074a75b755 | 0 ...ut_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd | 0 ...ut_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f | 0 ...ut_testxpath3-0-f05134d54292acd1f2067027889a4fac | 0 ...ut_testxpath3-1-807b097ac2f785f774db03069ebbde11 | 0 ...ut_testxpath4-0-73819ea1a7c0653a61652b3766afb003 | 0 ...ut_testxpath4-1-f746888141a38ba707fad01d86d41960 | 0 ...ut_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 | 0 ...ut_testxpath4-3-ae225e86c2ae20519ffdf23190454161 | 0 ...ut_testxpath4-4-f746888141a38ba707fad01d86d41960 | 0 ...ut_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 | 0 .../inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 | 0 .../inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 | 0 .../inputddl4-2-7fdf00ff5c22ae284728e0f035396865 | 0 .../inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 | 0 .../inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf | 0 .../inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c | 0 .../inputddl6-3-e81b962986706e1c16f059b407e3f05c | 0 .../inputddl6-4-5855e2998e26f63e927854afa86c1f03 | 0 .../inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 | 0 .../inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 | 0 .../inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 | 0 .../inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 | 0 .../inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 | 0 .../inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f | 0 .../inputddl7-1-7b46b8baf9c7628da9c190c96b917057 | 0 .../inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d | 0 .../inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 | 0 .../inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 | 0 .../inputddl7-13-45059a21c202b4658285738ee62a018a | 0 .../inputddl7-14-30c87bc734c2afa4fea0facdf7279145 | 0 .../inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe | 0 .../inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 | 0 .../inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 | 0 .../inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 | 0 .../inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 | 0 .../inputddl7-6-a590512af0ed021d33e2d71b7932a20a | 0 .../inputddl7-7-48640fff8428a0dc6e90a7243adaf730 | 0 .../inputddl7-8-495dc87b0bde752c890f213ff9531508 | 0 .../inputddl7-9-c943e781fb448aea5467251ee208fbac | 0 .../inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 | 0 .../inputddl8-1-c70f2d2544633366b76b92bcff18e995 | 0 ... with db name-0-a253b1ed35dbf503d1b8902dacbe23ac | 0 ... with db name-1-4f5be7056d8dc15cf2334a7cc146a636 | 0 ... with db name-2-a81aea129f78bc05305b8e887ee88f86 | 0 ... with db name-3-ce780d068b8d24786e639e361101a0c7 | 0 ... with db name-4-afd6e46b6a289c3c24a8eec75a94043c | 0 ...rt_compressed-0-ea607fbed28d20e5726f4501285d698d | 0 ...rt_compressed-1-d20899578068ff4adfee5355cddca819 | 0 ...rt_compressed-2-8dec751fd4148a431af064b7317f1530 | 0 ...rt_compressed-3-35f4fbde823a5664fe9928a685745b35 | 0 ...rt_compressed-4-5133d2457097962811a2adf0ecd9e4ef | 0 ...rt_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 | 0 ...rt_compressed-6-5133d2457097962811a2adf0ecd9e4ef | 0 ...rt_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 | 0 ...rt_compressed-8-5133d2457097962811a2adf0ecd9e4ef | 0 ...rt_compressed-9-d20899578068ff4adfee5355cddca819 | 0 .../insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 | 0 .../insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b | 0 ...insert_into1-10-c260979323c1ebdf68c6fbe003d43792 | 0 .../insert_into1-11-41015d6409c5ebf670eed4999157fdb | 0 ...insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 | 0 .../insert_into1-2-ff6a1b25c911def274921df1bae476b7 | 0 .../insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 | 0 .../insert_into1-4-41015d6409c5ebf670eed4999157fdb | 0 .../insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 | 0 .../insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 | 0 .../insert_into1-7-41015d6409c5ebf670eed4999157fdb | 0 .../insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d | 0 .../insert_into1-9-31eb4770dc60e4765065ac7f84811d1b | 0 .../insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 | 0 .../insert_into2-1-9828bb831fd11667b48678e5952a0941 | 0 ...insert_into2-10-df53336f364fe09e9591e769c13b5519 | 0 ...insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa | 0 ...insert_into2-12-4cf03cb3982a457f2f72220265ecc844 | 0 ...insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 | 0 .../insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 | 0 .../insert_into2-3-9d5556040de01fd02d5501d141effff2 | 0 .../insert_into2-4-9d5556040de01fd02d5501d141effff2 | 0 .../insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 | 0 .../insert_into2-6-4cf03cb3982a457f2f72220265ecc844 | 0 .../insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 | 0 .../insert_into2-8-452111285dda40205ee587de8e972896 | 0 .../insert_into2-9-4cf03cb3982a457f2f72220265ecc844 | 0 .../insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 | 0 .../insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 | 0 ...insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 | 0 ...insert_into3-11-9cfd2d054f84262eb74a870b6365db87 | 0 ...insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 | 0 ...insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 | 0 .../insert_into3-2-e51c25bae2408422a56826a263479468 | 0 .../insert_into3-3-c46699c465fefe6baab35499a32b452d | 0 .../insert_into3-4-e9f4f47686fe97482b0a769a15481dd | 0 .../insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 | 0 .../insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 | 0 .../insert_into3-7-9cfd2d054f84262eb74a870b6365db87 | 0 .../insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 | 0 .../insert_into3-9-19d1be183f7985f7066f86572abc82c8 | 0 .../insert_into4-0-b4831621b2a02fc4e8e655b03c289310 | 0 .../insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 | 0 ...insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 | 0 ...insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa | 0 ...insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 | 0 ...insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b | 0 ...insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 | 0 ...insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 | 0 .../insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 | 0 .../insert_into4-3-43629aaa698fb6e2db4586124561e69b | 0 .../insert_into4-4-8adba808fd505f4bf0ffcc61a618480a | 0 .../insert_into4-5-6bc47894aa917051abb98d0b52f43881 | 0 .../insert_into4-6-8c1683bee2927da76bb0dbf44a373738 | 0 .../insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 | 0 .../insert_into4-8-6bc47894aa917051abb98d0b52f43881 | 0 .../insert_into4-9-8c1683bee2927da76bb0dbf44a373738 | 0 .../insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 | 0 .../insert_into5-1-8fc8db6a5564324339192f23208ffc1c | 0 .../insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 | 0 ...insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d | 0 ...insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 | 0 ...insert_into5-13-e06a0b7252278141d50466e08f15b391 | 0 ...insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 | 0 ...insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 | 0 ...insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 | 0 .../insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c | 0 .../insert_into5-3-a4b25f172af356ec98035329b95ddbd3 | 0 .../insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 | 0 .../insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 | 0 .../insert_into5-6-a058ba199b9777d48b6c6595f2388533 | 0 .../insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f | 0 .../insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 | 0 .../insert_into5-9-a058ba199b9777d48b6c6595f2388533 | 0 .../insert_into6-0-16367c381d4b189b3640c92511244bfe | 0 .../insert_into6-1-a4fb8359a2179ec70777aad6366071b7 | 0 ...insert_into6-10-16500f4af2c8638a670e867e59f6d457 | 0 ...insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba | 0 ...insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 | 0 ...insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 | 0 ...insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 | 0 ...insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d | 0 .../insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 | 0 .../insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d | 0 .../insert_into6-4-5f12794e99c74355a23d2fda9c7c170f | 0 .../insert_into6-5-de641eb41a9100e755a9ae641c752b30 | 0 .../insert_into6-6-f6e7141a435922193937aa10085b0656 | 0 .../insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 | 0 .../insert_into6-8-33ec9514947e0b737e957bdcbbd87573 | 0 .../insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab | 0 ...sertexternal1-0-eb0745518b859c8497506a627bfd9860 | 0 ...sertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a | 0 ...sertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b | 0 ...sertexternal1-3-f64289fb03ab105e12659fc3972ca241 | 0 .../golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 | 0 .../golden/join0-1-83f5e512d6058a47f92b5218781f5070 | 0 .../golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 | 0 .../golden/join1-0-43d53504df013e6b35f81811138a167a | 0 .../golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e | 0 .../golden/join1-2-7b07671864bbfdc627ee794932e32b1e | 0 .../golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 | 0 .../golden/join1-4-35c7611180562dcd9dab834f41654095 | 0 .../join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 | 0 .../join10-1-73136f8e6e9ba82f75570afd15c2828d | 0 .../golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 | 0 .../join11-1-b580f87daf1763cd8c5e59ad5b271232 | 0 .../join12-0-12fa53be2c0ddbcfc85212e573a46af1 | 0 .../golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f | 0 .../join13-0-c4ebf24269e27919f4ba9ce3993e48f7 | 0 .../join13-1-696b36d15c4358145f77c2b15b7507d5 | 0 .../join14-0-c85f3dcbab496811604ea0ab84d0e995 | 0 .../join14-1-9b141c1e5917ca82c6bc36a9a2950a1e | 0 .../golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 | 0 .../join14-3-bab89dfffa77258e34a595e0e79986e3 | 0 .../join14-4-27f1a57fbb815d169af86ae2f8305cb6 | 0 .../join14-5-4c3f537b4df1ef16788a53cf65574187 | 0 .../golden/join14-6-de39302191b63d7aa8f92885b089fe2 | 0 .../join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 | 0 ...in14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 | 0 ...in14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f | 0 ...in14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 | 0 ...in14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 | 0 ...oin14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 | 0 ...in14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 | 0 .../join15-0-930db2823512f7f8f34cac104f2162e0 | 0 .../join15-1-81d76d3bf59889b07b413b6f88772667 | 0 .../join16-0-1bec3b0892d5c4a174d1b39f6d1b610c | 0 .../join17-0-387dd86b1e13f788ec677a08dc162c97 | 0 .../join17-1-37cef87fe56f66692799ccda8cda2e8b | 0 .../join17-2-478a9f270a5d70f6f82f81e6962fb251 | 0 .../join17-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../join18-0-269843197b598569f4b50cbe911960a5 | 0 .../join18-1-49f8ba8d43a6bb1d892ba66a812304f5 | 0 .../join19-0-26ba1cd9acfcd799239da60c38e68a39 | 0 .../join19-1-7e7d1f07c34dd4be5425264196201766 | 0 .../golden/join2-0-3d1692c4710db1ff716d35e921f2bcca | 0 .../golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c | 0 .../golden/join2-2-d4673c03d04084b838fcd8149f59ad9a | 0 .../golden/join2-3-cac2c9e0f8601dd56822c990774e0696 | 0 .../join20-0-16cda49c2b2cd0c0cdae40af4d5c900e | 0 .../join20-1-9685f2d31ffe922d3ea217de32ca3049 | 0 .../join20-2-222655137ca9242f063c7717acbe4c65 | 0 .../join20-3-3331a020843caccf2fa32a1225c5c3a9 | 0 .../join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 | 0 .../join21-1-3536b7d78713e86ee67f5f6c9b88958f | 0 .../golden/join22-0-97081b5f73052a612fb391a57ad9b7b | 0 .../join23-0-1fb76d3404f09a5c2b10422157c3cf5d | 0 .../join23-1-91b8e7fe75a7e3ba8147c56734436681 | 0 .../join24-0-721dfa03bfea05e55506c571b6c3585b | 0 .../join24-1-36de83b0ed6c9fdc03661b2f65b23a3d | 0 .../join24-2-d79325ef6494aa87843fdfd78de7c812 | 0 .../join25-0-8934d9b821aa4b34b760f73eff56cd06 | 0 .../join25-1-360b3676605ccb3d89cf555342db87af | 0 .../join25-2-ef930bed933341636c71622e6f89e9e9 | 0 .../join25-3-e39577008d28ddd5741d8518b92eaa94 | 0 .../join25-4-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../golden/join26-0-775b1af8eae9b2250052be1b72b4086 | 0 .../join26-1-10fbe6c99e93cfeafcd454261744728e | 0 .../join26-2-8aa0193a2a80d4efe4c66bca19218a4d | 0 .../join26-3-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../join27-0-360b3676605ccb3d89cf555342db87af | 0 .../join27-1-d56d8bbececcf48417b7dde9292a9cc6 | 0 .../join27-2-6d139e1aa07ff8d02bdf52b409956879 | 0 .../join27-3-e86808fdbd54120d1e2356f8f61c02f9 | 0 .../join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 | 0 .../join28-1-24ca942f094b14b92086305cc125e833 | 0 .../join28-2-44d382ce6848d3f0b900b0808747d8e9 | 0 .../join28-3-25fc734982956a164adde6bb1d4d8751 | 0 .../join28-4-bd4e19a3608681b0270668fe68b7d124 | 0 .../join28-5-24cc9ff9485313ade08ee83ecc4c0621 | 0 .../join28-6-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../join29-0-e78402f7585b17c76c32790571d749eb | 0 .../join29-1-24ca942f094b14b92086305cc125e833 | 0 .../join29-2-44d382ce6848d3f0b900b0808747d8e9 | 0 .../join29-3-25fc734982956a164adde6bb1d4d8751 | 0 .../join29-4-b0524de7985cdb7b57fd995721654856 | 0 .../golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 | 0 .../join29-6-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 | 0 .../golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 | 0 .../golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 | 0 .../golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../join30-0-54a7280ab9eed0d2e3b33df35a721b66 | 0 .../join30-1-5859eba43fb180c2142035cd4c77ea4d | 0 .../join30-2-194edec0592cf441617ca7caf9756baa | 0 .../join30-3-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../join31-0-619db2f9dd69aa236aa804cced726c9a | 0 .../join31-1-24ca942f094b14b92086305cc125e833 | 0 .../join31-2-44d382ce6848d3f0b900b0808747d8e9 | 0 .../join31-3-25fc734982956a164adde6bb1d4d8751 | 0 .../join31-4-34e30fa5d6f35a20561c7754197f0626 | 0 .../join31-5-a2082be28be44e2bfa9a58fb45c23f07 | 0 .../join31-6-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../golden/join32-0-775b1af8eae9b2250052be1b72b4086 | 0 .../join32-1-24ca942f094b14b92086305cc125e833 | 0 .../join32-2-44d382ce6848d3f0b900b0808747d8e9 | 0 .../join32-3-25fc734982956a164adde6bb1d4d8751 | 0 .../join32-4-fc6880e4e0750a3daa7fc108b72d11bb | 0 .../join32-5-425c5c35b23c42254dabec49aa6613bf | 0 .../join32-6-c9859bf9c9942c59f3b19d82bd1a3afa | 0 ...oin32_lessSize-0-775b1af8eae9b2250052be1b72b4086 | 0 ...in32_lessSize-1-7a874b25490d3e1e186803646eb6a789 | 0 ...n32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a | 0 ...in32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca | 0 ...in32_lessSize-12-83544f47197cccad30dd4f0ede433b8 | 0 ...n32_lessSize-13-ed70124968560328930327ecb108c4e9 | 0 ...n32_lessSize-14-ac014eb214f762f1a61125af1964788a | 0 ...n32_lessSize-15-78edd67daa31711374b4c25faf89f77e | 0 ...n32_lessSize-16-ed70124968560328930327ecb108c4e9 | 0 ...n32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 | 0 ...n32_lessSize-18-f9b31c552044d8e5cc193121eb71024d | 0 ...n32_lessSize-19-ed70124968560328930327ecb108c4e9 | 0 ...in32_lessSize-2-24ca942f094b14b92086305cc125e833 | 0 ...in32_lessSize-20-ecf04f68517b6104d9cc8995796b424 | 0 ...n32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 | 0 ...n32_lessSize-22-ed70124968560328930327ecb108c4e9 | 0 ...in32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 | 0 ...in32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 | 0 ...in32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb | 0 ...in32_lessSize-6-425c5c35b23c42254dabec49aa6613bf | 0 ...in32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a | 0 ...in32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b | 0 ...oin32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 | 0 .../golden/join33-0-775b1af8eae9b2250052be1b72b4086 | 0 .../join33-1-24ca942f094b14b92086305cc125e833 | 0 .../join33-2-44d382ce6848d3f0b900b0808747d8e9 | 0 .../join33-3-25fc734982956a164adde6bb1d4d8751 | 0 .../join33-4-fc6880e4e0750a3daa7fc108b72d11bb | 0 .../join33-5-425c5c35b23c42254dabec49aa6613bf | 0 .../join33-6-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../golden/join34-0-775b1af8eae9b2250052be1b72b4086 | 0 .../join34-1-24ca942f094b14b92086305cc125e833 | 0 .../join34-2-44d382ce6848d3f0b900b0808747d8e9 | 0 .../join34-3-25fc734982956a164adde6bb1d4d8751 | 0 .../join34-4-51cc5d253befd784a9d466c0b402f47a | 0 .../join34-5-abf5cf110d6a0eb85ae257768c805f39 | 0 .../join34-6-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 | 0 .../join35-1-24ca942f094b14b92086305cc125e833 | 0 .../join35-2-44d382ce6848d3f0b900b0808747d8e9 | 0 .../join35-3-25fc734982956a164adde6bb1d4d8751 | 0 .../join35-4-99cb704ccc2813400908a62571891dc7 | 0 .../join35-5-60956f3307651237f52ffbe41c827c1c | 0 .../join35-6-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../join36-0-8934d9b821aa4b34b760f73eff56cd06 | 0 .../golden/join36-1-3aba153d62042206c0f001ea74980da | 0 .../join36-2-5b1cac8ef7857d2f8c675ed6f0224471 | 0 .../join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 | 0 .../join36-4-fbe7d714e988877ebe6202a60e8ef776 | 0 .../join36-5-9353d3e0b11c781f782af4013e8569e3 | 0 .../join36-6-d76aff886201d223c7dbed2b0f0d5512 | 0 .../join36-7-171ede21302050a33468d95e329a38a3 | 0 .../join36-8-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../join37-0-8934d9b821aa4b34b760f73eff56cd06 | 0 .../join37-1-360b3676605ccb3d89cf555342db87af | 0 .../join37-2-2310f2b8726137811543e04db98dd1db | 0 .../join37-3-c4b3a85965108ad7013ac3931598af09 | 0 .../join37-4-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../join38-0-42e6ad674a5142d05258a2d29578e046 | 0 .../join38-1-d8ba5ec8a5a0bb702958400a3b3e458f | 0 .../golden/join38-2-eacdb3417c4a563982c488812d654c9 | 0 .../join38-3-22ead86c36dcd9ac9403fb52ac752046 | 0 .../join38-4-53d219706847e890de1dcd369563ebef | 0 .../join39-0-54e9f7924abb40d1c601de908de6f25b | 0 .../join39-1-60178dec03adcbccbd4e5cae936c38f2 | 0 .../join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 | 0 .../join39-3-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../golden/join4-0-531e91e11b3891627c2675935fda14cd | 0 .../golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 | 0 .../golden/join4-2-dc967001beb776f3a859e9360823c361 | 0 .../golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../join40-0-d2b5e23edec42a62e61750b110ecbaac | 0 .../join40-1-4b663d50fee0a8afd32cdf25c3b82de8 | 0 .../join40-10-1d1f79e8e773d630f355c1a89d84b5aa | 0 .../join40-11-c8c5b97a744acf897888ab2d51d33a80 | 0 .../join40-12-aaddbef9069aba3ebeb310be74671cda | 0 .../join40-2-507b1d9f6abbdb756a589d7bc4826251 | 0 .../join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 | 0 .../join40-4-61fb097bda1751910de966d6a4a8f0b7 | 0 .../join40-5-16cda49c2b2cd0c0cdae40af4d5c900e | 0 .../join40-6-9685f2d31ffe922d3ea217de32ca3049 | 0 .../join40-7-222655137ca9242f063c7717acbe4c65 | 0 .../join40-8-3331a020843caccf2fa32a1225c5c3a9 | 0 .../join40-9-6b7404af014eeb4de196577f14ef18cc | 0 .../join41-0-b74d94933c6670ccc4a1ad8161185686 | 0 .../golden/join41-1-25e434b6d05e08fdd5f4d9957438917 | 0 .../golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a | 0 .../join41-3-1dc98363e7da167dc45711a87ef3a988 | 0 .../join41-4-44d382ce6848d3f0b900b0808747d8e9 | 0 .../join41-5-f18ea17bcfafde311a4bdc589362e00e | 0 .../join41-6-1dc98363e7da167dc45711a87ef3a988 | 0 .../golden/join5-0-531e91e11b3891627c2675935fda14cd | 0 .../golden/join5-1-76feff1282895f38b673f52acfd7429e | 0 .../golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 | 0 .../golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../golden/join6-0-531e91e11b3891627c2675935fda14cd | 0 .../golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 | 0 .../golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 | 0 .../golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../golden/join7-0-8f82881057bec4abf5a4d770a6f35838 | 0 .../golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 | 0 .../golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 | 0 .../golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../golden/join8-0-531e91e11b3891627c2675935fda14cd | 0 .../golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 | 0 .../golden/join8-2-f1196bca86a749375da35f134206a8ca | 0 .../golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 | 0 .../golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a | 0 .../golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 | 0 .../golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../join_1to1-0-7ea7cf54372f262b952108113a97a294 | 0 .../join_1to1-1-5bab379018a4fbef12cc93658f26580a | 0 .../join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e | 0 .../join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e | 0 .../join_1to1-12-6081eb976b4aef2132418510756a385b | 0 .../join_1to1-13-281b888188eac90c4bf670417f25cc0c | 0 .../join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 | 0 .../join_1to1-15-5e48ba086f1376939535081b60f82727 | 0 .../join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 | 0 .../join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e | 0 .../join_1to1-18-6081eb976b4aef2132418510756a385b | 0 .../join_1to1-19-281b888188eac90c4bf670417f25cc0c | 0 .../join_1to1-2-1c35d445eca04025196ea700d02f9987 | 0 .../join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 | 0 .../join_1to1-21-5e48ba086f1376939535081b60f82727 | 0 .../join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 | 0 .../join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e | 0 .../join_1to1-24-6081eb976b4aef2132418510756a385b | 0 .../join_1to1-25-281b888188eac90c4bf670417f25cc0c | 0 .../join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 | 0 .../join_1to1-27-5e48ba086f1376939535081b60f82727 | 0 .../join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e | 0 .../join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e | 0 .../join_1to1-3-ee6db4188755bf471a12316ec7301500 | 0 .../join_1to1-30-6081eb976b4aef2132418510756a385b | 0 .../join_1to1-31-281b888188eac90c4bf670417f25cc0c | 0 .../join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 | 0 .../join_1to1-33-5e48ba086f1376939535081b60f82727 | 0 .../join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 | 0 .../join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e | 0 .../join_1to1-36-6081eb976b4aef2132418510756a385b | 0 .../join_1to1-37-281b888188eac90c4bf670417f25cc0c | 0 .../join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 | 0 .../join_1to1-39-5e48ba086f1376939535081b60f82727 | 0 .../join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 | 0 .../join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e | 0 .../join_1to1-6-6081eb976b4aef2132418510756a385b | 0 .../join_1to1-7-281b888188eac90c4bf670417f25cc0c | 0 .../join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 | 0 .../join_1to1-9-5e48ba086f1376939535081b60f82727 | 0 .../join_array-0-60904bf2bd7bbfa8739d8e047e83e038 | 0 .../join_array-1-2c086fcf118fd2538bfa00724209091e | 0 .../join_array-2-a4363f7c7e4b7d717ed90e77c37581de | 0 .../join_array-3-ddd65703cdad8959cd0cd831304b0ab9 | 0 .../join_array-4-b235265cd6bd58fd743c27b02e547d62 | 0 .../join_array-5-a9b9419b94631f8fe1c2297ebf103a9a | 0 .../join_array-6-6eded94bd39189ea6d67fe383f9b865c | 0 ...casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 | 0 ...casesensitive-1-404d691e85c7b74bad73576ee80de290 | 0 ..._casesensitive-2-d508d777b4e4156bff9774925a6ca4d | 0 ...casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc | 0 ...casesensitive-4-c880b2256f97413b8fe68d19d99747fd | 0 .../join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad | 0 .../join_empty-1-9bcf09f835d785760c0d4da0680837f2 | 0 .../join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e | 0 .../join_empty-3-3b2231f25886c152f222856256ac3473 | 0 .../join_empty-4-df2401785dfa257de49c3ad80b0f480a | 0 .../join_empty-5-ce1ef910fff98f174931cc641f7cef3a | 0 .../join_empty-6-e807e73f2636bf03a572c9e0b8430b5a | 0 .../join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 | 0 .../join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 | 0 .../join_filters-1-100faa7fd01bfb4390c782bb262a9403 | 0 ...join_filters-10-f0c5c51de4151a17a77c780be0c13e01 | 0 ...oin_filters-100-a4c7cd87175222bea19fd33018890efe | 0 ...oin_filters-101-f086409eb336282af5a00f4c7192ef2b | 0 ...oin_filters-102-634888c4fa78ce10181c9514c6393554 | 0 ...oin_filters-103-c020d3461658ae8e118281f40264ae5b | 0 ...oin_filters-104-c9b79f30e1f25672ec89014f966b41b0 | 0 ...join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb | 0 ...oin_filters-106-870999978978b22f21997899f1e652b8 | 0 ...oin_filters-107-94824a62d882227f270a66ec7ef28cd4 | 0 ...oin_filters-108-d793c1514545499f58fb1b355cbd3f0e | 0 ...oin_filters-109-2709001b4aa57ed01ba975e83b556475 | 0 ...join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 | 0 ...oin_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba | 0 ...oin_filters-111-fd3188d025e4c84d45cbb265d08ca569 | 0 ...oin_filters-112-a0cd07949ff7dbc2287dc060bedb1942 | 0 ...join_filters-113-48c4978768872751832149d72cdf0ba | 0 ...oin_filters-114-58b8db46ea5080791c7416838d3e8f95 | 0 ...oin_filters-115-98e45a3167d19d09219076a2d93afa49 | 0 ...join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 | 0 ...oin_filters-117-ee8471d93c3778f4b48c5c59f473dd35 | 0 ...oin_filters-118-9b395d7db8722a467d46588d0f27fc9e | 0 ...oin_filters-119-7688784396db55ff148292a9755c918a | 0 ...join_filters-12-4c29d6be8717481332cd1ee7ca17690e | 0 ...oin_filters-120-6578651ffbf95c0b02825e2125e32709 | 0 ...oin_filters-121-6dc4977da7f8dc7c636a03d7ad892ada | 0 ...oin_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 | 0 .../join_filters-13-c6a291879bdb37f0c84f6074f257d52 | 0 ...join_filters-14-ef8255dcad808f9128d79e6ee9f368cf | 0 ...join_filters-15-a83678913c62249c8fdf2dac1f6e3046 | 0 ...join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 | 0 ...join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 | 0 ...join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 | 0 ...join_filters-19-e164192e6538b428c8b53e008a978d3f | 0 .../join_filters-2-67fff9e6931a7320444e857e01b3d496 | 0 ...join_filters-20-7a5da20822bf51ed69ccf640cbb816cf | 0 ...join_filters-21-13d6d5335625fc3386a8011dc557002e | 0 ...join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e | 0 ...join_filters-23-a800b885358695758afdb719cdefa94f | 0 ...join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 | 0 ...join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 | 0 ...join_filters-26-da36fab3d1686aedd415a7a0f0253eca | 0 ...join_filters-27-5f4a5437696f2a60bec9ac1443244242 | 0 ...join_filters-28-2acf41a9f6efac0d800df557db716359 | 0 ...join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 | 0 .../join_filters-3-989b3dbd15ba601ae80fe454e03213d7 | 0 ...join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf | 0 ...join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f | 0 ...join_filters-32-5978cd7936c296493a16a31b926043ab | 0 ...join_filters-33-607d64d50ef9aad424bd22b358efe027 | 0 ...join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 | 0 ...join_filters-35-a4c7cd87175222bea19fd33018890efe | 0 ...join_filters-36-f086409eb336282af5a00f4c7192ef2b | 0 ...join_filters-37-634888c4fa78ce10181c9514c6393554 | 0 ...join_filters-38-c020d3461658ae8e118281f40264ae5b | 0 ...join_filters-39-c9b79f30e1f25672ec89014f966b41b0 | 0 .../join_filters-4-33bfcd576019d7e32683556f66e3757 | 0 .../join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb | 0 ...join_filters-41-870999978978b22f21997899f1e652b8 | 0 ...join_filters-42-94824a62d882227f270a66ec7ef28cd4 | 0 ...join_filters-43-d793c1514545499f58fb1b355cbd3f0e | 0 ...join_filters-44-6d8955591f62d9cfc6af17df63d3d88e | 0 ...join_filters-45-23ab7ac8229a53d391195be7ca092429 | 0 ...join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b | 0 ...join_filters-47-3c52df82c7d78501610f3f898103f753 | 0 ...join_filters-48-1d85bb008e02ef4025171a4bc0866a6c | 0 .../join_filters-49-e79c906b894fed049ddfab4496a4e3 | 0 .../join_filters-5-f0c0d07019afb1bbe162e3183e18023e | 0 ...join_filters-50-3e6612a89e9124592e790594775054b1 | 0 ...join_filters-51-60a5f56f33fc8854a2b687005f0d96ac | 0 ...join_filters-52-64cabe5164130a94f387288f37b62d71 | 0 ...join_filters-53-2709001b4aa57ed01ba975e83b556475 | 0 ...join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba | 0 ...join_filters-55-fd3188d025e4c84d45cbb265d08ca569 | 0 ...join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 | 0 .../join_filters-57-48c4978768872751832149d72cdf0ba | 0 ...join_filters-58-58b8db46ea5080791c7416838d3e8f95 | 0 ...join_filters-59-98e45a3167d19d09219076a2d93afa49 | 0 .../join_filters-6-c0c40d001cac0bc91095dddda1513ad9 | 0 .../join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 | 0 ...join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 | 0 ...join_filters-62-9b395d7db8722a467d46588d0f27fc9e | 0 ...join_filters-63-7688784396db55ff148292a9755c918a | 0 ...join_filters-64-6578651ffbf95c0b02825e2125e32709 | 0 ...join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada | 0 ...join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 | 0 ...join_filters-67-67fff9e6931a7320444e857e01b3d496 | 0 ...join_filters-68-989b3dbd15ba601ae80fe454e03213d7 | 0 .../join_filters-69-33bfcd576019d7e32683556f66e3757 | 0 .../join_filters-7-89963646509154a2fb1ddbbf1f55349d | 0 ...join_filters-70-f0c0d07019afb1bbe162e3183e18023e | 0 ...join_filters-71-c0c40d001cac0bc91095dddda1513ad9 | 0 ...join_filters-72-89963646509154a2fb1ddbbf1f55349d | 0 ...join_filters-73-69e0235472d7cee7d83037cd083544a5 | 0 ...join_filters-74-b6372cc006844e8488a3b7836c67daaa | 0 ...join_filters-75-f0c5c51de4151a17a77c780be0c13e01 | 0 ...join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 | 0 ...join_filters-77-4c29d6be8717481332cd1ee7ca17690e | 0 .../join_filters-78-c6a291879bdb37f0c84f6074f257d52 | 0 ...join_filters-79-ef8255dcad808f9128d79e6ee9f368cf | 0 .../join_filters-8-69e0235472d7cee7d83037cd083544a5 | 0 ...join_filters-80-a83678913c62249c8fdf2dac1f6e3046 | 0 ...join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 | 0 ...join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 | 0 ...join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 | 0 ...join_filters-84-e164192e6538b428c8b53e008a978d3f | 0 ...join_filters-85-7a5da20822bf51ed69ccf640cbb816cf | 0 ...join_filters-86-13d6d5335625fc3386a8011dc557002e | 0 ...join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e | 0 ...join_filters-88-a800b885358695758afdb719cdefa94f | 0 ...join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 | 0 .../join_filters-9-b6372cc006844e8488a3b7836c67daaa | 0 ...join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 | 0 ...join_filters-91-da36fab3d1686aedd415a7a0f0253eca | 0 ...join_filters-92-5f4a5437696f2a60bec9ac1443244242 | 0 ...join_filters-93-2acf41a9f6efac0d800df557db716359 | 0 ...join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 | 0 ...join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf | 0 ...join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f | 0 ...join_filters-97-5978cd7936c296493a16a31b926043ab | 0 ...join_filters-98-607d64d50ef9aad424bd22b358efe027 | 0 ...join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 | 0 ...lters_overlap-0-990e447b6447ced0d9684eb7db9e63ce | 0 ...lters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 | 0 ...join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 | 0 ...join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 | 0 ...join_hive_626-2-42997e626819de9cdad544d7fe132c6e | 0 ...join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 | 0 ...join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 | 0 ...join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 | 0 ...join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 | 0 ...join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 | 0 .../join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 | 0 .../join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 | 0 ...join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e | 0 .../join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa | 0 .../join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 | 0 .../join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 | 0 .../join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a | 0 .../join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db | 0 .../join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed | 0 .../join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a | 0 .../join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 | 0 .../join_nulls-1-97641998eb9ddf2dff56de6758486aa0 | 0 .../join_nulls-10-39071d8e6b246cfd405714dbf0b5337b | 0 .../join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 | 0 .../join_nulls-12-5e1ca8627aa685435142d8e339e77062 | 0 .../join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d | 0 .../join_nulls-14-e97ba69145da387a4a66635b8499077 | 0 .../join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 | 0 .../join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 | 0 .../join_nulls-17-2b0bfe5754456475ceb6af4694165f | 0 .../join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 | 0 .../join_nulls-19-739bf8e440e698540d18c29226c3564c | 0 .../join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 | 0 .../join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 | 0 .../join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e | 0 .../join_nulls-22-5b2df5518994ae86c041484561857da0 | 0 .../join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 | 0 .../join_nulls-24-86ad66543a735d396f7336cb5bdfa495 | 0 .../join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 | 0 .../join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 | 0 .../join_nulls-27-6ee7affed896b1c539628ab081842b83 | 0 .../join_nulls-28-455aace3472c5840a885b6fab6a046cb | 0 .../join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb | 0 .../join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 | 0 .../join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 | 0 .../join_nulls-31-a33c48d38817ee3a7aca511dc7793486 | 0 .../join_nulls-32-e6b104ae96622ff75d510efc6efc9352 | 0 .../join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 | 0 .../join_nulls-34-aeb90811861431cadc5512637793afc1 | 0 .../join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 | 0 .../join_nulls-36-7c029c91141b36af79ba0dc1de73a257 | 0 .../join_nulls-37-fa84731f5a6beec20d64a7981815b9bc | 0 .../join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e | 0 .../join_nulls-39-23ab7ac8229a53d391195be7ca092429 | 0 .../join_nulls-4-a1f20b4863428627bae1316755cc2d94 | 0 .../join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b | 0 .../join_nulls-41-3c52df82c7d78501610f3f898103f753 | 0 .../join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c | 0 .../join_nulls-43-e79c906b894fed049ddfab4496a4e3 | 0 .../join_nulls-44-2db30531137611e06fdba478ca7a8412 | 0 .../join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 | 0 .../join_nulls-46-64cabe5164130a94f387288f37b62d71 | 0 .../join_nulls-47-ebf794e8b51be738e2d664f249869de1 | 0 .../join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 | 0 .../join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 | 0 .../join_nulls-5-5ec6e3df7181e0738244623cc01bf22c | 0 .../join_nulls-50-8b26343e74c161cf74ece5f0bd21470 | 0 .../join_nulls-51-75339d2eb2afabf5dd088074b2563d8f | 0 .../join_nulls-52-caad1db99085760daaf8f96c0ce5564 | 0 .../join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 | 0 .../join_nulls-54-6f15c794b5587430ebb685cc61c502 | 0 .../join_nulls-55-a1c73326f8c8d9beccda3ba365352564 | 0 .../join_nulls-56-f7b9629093c818b6c384c79d1458d178 | 0 .../join_nulls-57-cf353446d7f358a508f17d0984b90158 | 0 .../join_nulls-58-5f9a59160a76f9b649102a97987ed33a | 0 .../join_nulls-59-8753a39412ac59c7a05951aeeea73b24 | 0 .../join_nulls-6-7eea211c80e7f1146098e80ffb890d67 | 0 .../join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 | 0 .../join_nulls-8-609f6bd812a44b20da0a39c827e4d870 | 0 .../join_nulls-9-ef4b27a877efc377cd5750af6725194b | 0 .../join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 | 0 .../join_rc-1-6a7685f30de00ebb4867a4002d641a5e | 0 .../join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 | 0 .../join_rc-3-4a737f3e518f5322ba41a189e79c1dee | 0 .../join_rc-4-f60f7fdd08e85fae90af59475192b725 | 0 .../join_rc-5-1aef75afe38d512addb44dbf9a650263 | 0 ...join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 | 0 ...join_reorder2-1-744a018c78bae6e09853dd202981e850 | 0 ...oin_reorder2-10-45349471e0e919bd2185f584e87b891d | 0 ...oin_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 | 0 ...join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e | 0 ...join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 | 0 ...join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 | 0 ...join_reorder2-6-3fda17e4414d191f837631438a19e700 | 0 ...join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec | 0 ...join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 | 0 ...join_reorder2-9-26ffed826eceda953b7124ee39ace828 | 0 ...join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 | 0 ...join_reorder3-1-744a018c78bae6e09853dd202981e850 | 0 ...oin_reorder3-10-ff036400019164ed743ecd9cfc222ce1 | 0 ...oin_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 | 0 ...join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e | 0 ...join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 | 0 ...join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 | 0 ...join_reorder3-6-3fda17e4414d191f837631438a19e700 | 0 ...join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec | 0 ...join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 | 0 ...join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 | 0 ...join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf | 0 ...join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a | 0 ...oin_reorder4-10-1d6b948747ac31296710a491a5652e3f | 0 ...join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 | 0 ...join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 | 0 ...join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c | 0 ...join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 | 0 ...join_reorder4-5-3fda17e4414d191f837631438a19e700 | 0 ...join_reorder4-6-24ca942f094b14b92086305cc125e833 | 0 ...join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 | 0 .../join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 | 0 ...join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba | 0 .../join_star-0-6001f83f7780835737e3262a34c2b832 | 0 .../join_star-1-9f29849fe78f15b7594378a10fe3ed9d | 0 .../join_star-10-57ce75f989b3b3bfd2f2eceb228e892e | 0 .../join_star-11-eba1397e66f25cba4fd264209cc92bae | 0 .../join_star-12-89b53ae954ec88171ef87e0459f6eb82 | 0 .../join_star-13-342b7249c9ce1484869169b1b33191cb | 0 .../join_star-14-75513308d30b781fd2e06d81963c4363 | 0 .../join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 | 0 .../join_star-16-24ca942f094b14b92086305cc125e833 | 0 .../join_star-17-44d382ce6848d3f0b900b0808747d8e9 | 0 .../join_star-18-1c22e451845667bd6b4eac3c49c36965 | 0 .../join_star-19-a1209595ce68e24a111c2a0d27863bfa | 0 .../join_star-2-75d7e03808482c11361ce72f2dd38de0 | 0 .../join_star-20-76473cb68a69b9408207fd43ddae9339 | 0 .../join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 | 0 .../join_star-22-429cfd140488242d569aee6594aae76b | 0 .../join_star-23-bada259628918dbfb4837bcb58258530 | 0 .../join_star-24-e11ab68d72d874c7c6c658c1018f5a49 | 0 .../join_star-25-be5af28b6bccf529a31ef68619de699e | 0 .../join_star-26-a412a0620a359a62e2ab4e45fa8e0330 | 0 .../join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 | 0 .../join_star-28-98a3f4d8a2209e771c57462d4b723ff9 | 0 .../join_star-3-4f4ca11033af3894cc2e720ebb69e5da | 0 .../join_star-4-aaba115af7543b214820a48a694f2463 | 0 .../join_star-5-c72a8aff3a51f9e59bb62fd99807d8da | 0 .../join_star-6-73d96ed451a4964e5c364dbdb56d487a | 0 .../join_star-7-b007c3d4b6ed10df9e875e23644b673a | 0 .../join_star-8-a957982d8981ff0a35397ca449297024 | 0 .../join_star-9-904e30d8615eb411fb24c2cc08df94f4 | 0 .../join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 | 0 .../join_thrift-1-4f47dd107d2be1212411bda214c0d1db | 0 .../join_thrift-2-d4bfb7bab262dc2898431680711dec1b | 0 .../join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 | 0 .../join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 | 0 .../join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 | 0 .../join_view-2-eda40dba9678df62dde73fc5dafb2b44 | 0 .../join_view-3-eccb00d8dada8ab56a48c373e381e02b | 0 .../join_view-4-763ab5853bff619e6525c01e46b2a923 | 0 .../join_view-5-85baeea910adc4589bc3ec2ce0b1e856 | 0 .../join_view-6-c37b7962ab7371f94a9c54d61f7638ef | 0 .../join_view-7-a14cfe3eff322066e61023ec06c7735d | 0 .../join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 | 0 .../join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 | 0 ...lateral view1-0-85bfbfd635aee807ede359073fb26dee | 0 ...lateral view2-0-6d92d10040c715df89db0f61ae4250bb | 0 ...lateral view3-0-55b6de1dbad92682804e98524c0d6c12 | 0 ...lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 | 0 ...lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 | 0 ...lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e | 0 ...lateral view5-0-f797cf9d04a98cfe477d14703aeb294b | 0 ...lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a | 0 .../lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b | 0 .../lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 | 0 ...lateral_view-10-1f2df08433fe12a9ae086e3d9309039e | 0 ...lateral_view-11-d884acac077c5a02ef048069dd8c16d7 | 0 .../lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 | 0 ...lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba | 0 ...lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 | 0 ...lateral_view-15-e1fad51e2f77de35d5781d788d70057e | 0 ...lateral_view-16-5eafb0317b6974eb588075d152719a79 | 0 ...lateral_view-17-4c6ff06840694c274bf8a91bade677ab | 0 ...lateral_view-18-a98a136672c920c0463bd49b4ec7b65e | 0 ...lateral_view-19-9b417d5554cbeb8f536a1613879295f7 | 0 .../lateral_view-2-bfd7354229ac8a934c8f978b43b28565 | 0 .../lateral_view-20-d75015d8e44499526fec784ec00b905 | 0 ...lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 | 0 .../lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e | 0 .../lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 | 0 .../lateral_view-5-857f93a5c8d5ba00d214384117147bae | 0 .../lateral_view-6-e78723941c3b42349f9eb804b4b82d15 | 0 .../lateral_view-7-7c88418f83112c55d08a0727d6248825 | 0 .../lateral_view-8-d2bff21465d3db7bbff8b9913c012452 | 0 .../lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 | 0 ...teral_view_cp-0-6ae48c990343343aad4145203c364f79 | 0 ...teral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 | 0 ...teral_view_cp-2-647d019098d676b2fdb8a5127b633287 | 0 ...teral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 | 0 ...teral_view_cp-4-507876b610812506343e9d251645170b | 0 ..._view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 | 0 ..._view_noalias-1-6d5806dd1d2511911a5de1e205523f42 | 0 ..._view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a | 0 ..._view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 | 0 ...l_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe | 0 ..._view_noalias-5-16d227442dd775615c6ecfceedc6c612 | 0 ...al_view_outer-0-b66c363516d1f717765df9b91da3b5b4 | 0 ...al_view_outer-1-8d4332785ff69bb86607700c133d0baa | 0 ...al_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 | 0 ...al_view_outer-3-57b2ffd7b60708551238c491a2a8685d | 0 ...al_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e | 0 ...al_view_outer-5-2ec3aeb923621c64da042402709e64e9 | 0 ...al_view_outer-6-511e4df505342e04c20e50fda8962120 | 0 ...eral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 | 0 ...eral_view_ppd-1-cd326124717660a333f0915395f96768 | 0 ...eral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f | 0 ...eral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 | 0 ...eral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 | 0 ...eral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f | 0 ...eral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa | 0 ...eral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b | 0 ...eral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 | 0 .../length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 | 0 ...imit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb | 0 ...imit_pushdown-1-d0a93f40892e3894460553b443c77428 | 0 ...mit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 | 0 ...mit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f | 0 ...mit_pushdown-12-a3516c212d6c79986536edbd9c961098 | 0 ...mit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 | 0 ...mit_pushdown-14-cfae77212d164efb18208f71332fd368 | 0 ...mit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 | 0 ...imit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 | 0 ...mit_pushdown-17-11c349c0db0f869be88351256650fe60 | 0 ...mit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 | 0 ...mit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...imit_pushdown-2-4d1e292b00635298240ff909be64dce4 | 0 ...mit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb | 0 ...mit_pushdown-21-f04dee0f94443ca01320657897cbb914 | 0 ...mit_pushdown-22-4d378725e22e7e48c861983ba935bf5e | 0 ...mit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a | 0 ...mit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 | 0 ...mit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a | 0 ...mit_pushdown-26-e691b284f1b830d7b83a36750105737c | 0 ...imit_pushdown-3-cc674af3ce71c06670e366932df43462 | 0 ...imit_pushdown-4-81bbb7300da27bc63f7a139677faac3f | 0 ...imit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 | 0 ...limit_pushdown-6-b722630e977a5ca509234fc417d7b30 | 0 ...imit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 | 0 ...imit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 | 0 ...imit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a | 0 .../lineage1-0-c021950f995f1d6a7b83ffe044daa750 | 0 .../lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 | 0 .../lineage1-2-f92e96c6357273ea6cbb2195a418df9c | 0 ...iteral_double-0-10ef1098e35d900983be3814de8f974f | 0 ...iteral_double-1-3863c17e03c9c1cd68452106a8721d13 | 0 .../literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 | 0 .../literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 | 0 ...literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 | 0 ...iteral_string-1-2cf4b7268b47246afdf6c792acca379d | 0 ...load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 | 0 ...oad_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d | 0 ...ad_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 | 0 ...ad_dyn_part1-11-9c82167763a771c175c656786d545798 | 0 ...oad_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 | 0 ...oad_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 | 0 ...oad_dyn_part1-4-16367c381d4b189b3640c92511244bfe | 0 ...oad_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 | 0 ...oad_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f | 0 ...oad_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 | 0 ...oad_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 | 0 ...oad_dyn_part1-9-30bc31441828a053d1a675b225a5d617 | 0 ...oad_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 | 0 ...ad_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 | 0 ...ad_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e | 0 ...ad_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ad_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 | 0 ...ad_dyn_part10-5-16367c381d4b189b3640c92511244bfe | 0 ...ad_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 | 0 ...ad_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 | 0 ...ad_dyn_part10-8-245027204484e281e1cfaf74386d2967 | 0 ...ad_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 | 0 ...oad_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 | 0 ...ad_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 | 0 ...ad_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 | 0 ...ad_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ad_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ad_dyn_part11-5-ea607fbed28d20e5726f4501285d698d | 0 ...ad_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 | 0 ...ad_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 | 0 ...ad_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb | 0 ...ad_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a | 0 ...oad_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 | 0 ...ad_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 | 0 ...ad_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b | 0 ...ad_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ad_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ad_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 | 0 ...ad_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 | 0 ...ad_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 | 0 ...oad_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 | 0 ...oad_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 | 0 ...ad_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 | 0 ...ad_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 | 0 ...ad_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ad_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ad_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 | 0 ...ad_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d | 0 ...ad_dyn_part13-7-99993811a25b02e7904a9403f51775d5 | 0 ...oad_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 | 0 ...ad_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d | 0 ...ad_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a | 0 ...ad_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e | 0 ...ad_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 | 0 ...ad_dyn_part14-3-16367c381d4b189b3640c92511244bfe | 0 ...ad_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd | 0 ...ad_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 | 0 ...ad_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 | 0 ...ad_dyn_part14-7-7c931249465f330d51ef0610f214429e | 0 ...dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 | 0 ...yn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e | 0 ...yn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 | 0 ...yn_part14_win-3-16367c381d4b189b3640c92511244bfe | 0 ...yn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd | 0 ...yn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 | 0 ...yn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 | 0 ...yn_part14_win-7-7c931249465f330d51ef0610f214429e | 0 ...ad_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 | 0 ...ad_dyn_part15-1-16367c381d4b189b3640c92511244bfe | 0 ...ad_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c | 0 ...ad_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 | 0 ...ad_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a | 0 ...ad_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 | 0 ...oad_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f | 0 ...oad_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 | 0 ...oad_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 | 0 ...oad_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...oad_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 | 0 ...oad_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b | 0 ...oad_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 | 0 ...oad_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 | 0 ...oad_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf | 0 ...oad_dyn_part2-9-3f29de9877006f9448272ef2422d6132 | 0 ...load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 | 0 ...oad_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f | 0 ...oad_dyn_part3-2-dbbba335c008a61a13c1472b34470397 | 0 ...oad_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 | 0 ...oad_dyn_part3-4-16367c381d4b189b3640c92511244bfe | 0 ...oad_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 | 0 ...oad_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 | 0 ...oad_dyn_part3-7-867958e24e25ad098c5001bbc7102762 | 0 ...oad_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 | 0 ...load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 | 0 ...oad_dyn_part4-1-50822afef6986cfea20523ea6092e455 | 0 ...ad_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 | 0 ...ad_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 | 0 ...oad_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd | 0 ...oad_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 | 0 ...oad_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 | 0 ...oad_dyn_part4-5-16367c381d4b189b3640c92511244bfe | 0 ...oad_dyn_part4-6-2869679fad49c57ba08169ea32271379 | 0 ...oad_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 | 0 ...oad_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 | 0 ...oad_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 | 0 ...oad_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 | 0 ...oad_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b | 0 ...ad_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 | 0 ...ad_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d | 0 ...oad_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 | 0 ...oad_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 | 0 ...oad_dyn_part5-4-16367c381d4b189b3640c92511244bfe | 0 ...oad_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 | 0 ...oad_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 | 0 ...load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df | 0 ...oad_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 | 0 ...oad_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 | 0 ...load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 | 0 ...oad_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 | 0 ...oad_dyn_part6-2-779aa345cf2875089312ec26b046415d | 0 ...oad_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 | 0 ...oad_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...oad_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 | 0 ...oad_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 | 0 ...oad_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 | 0 ...load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f | 0 ...load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 | 0 ...oad_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 | 0 ...oad_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a | 0 ...oad_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 | 0 ...oad_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 | 0 ...oad_dyn_part7-5-b3615ce72a55dba805303145030c8a93 | 0 ...load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 | 0 ...oad_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c | 0 ...oad_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a | 0 ...oad_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 | 0 ...oad_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 | 0 ...oad_dyn_part8-5-16367c381d4b189b3640c92511244bfe | 0 ...oad_dyn_part8-6-86db06cb739ceef70c8885469d847495 | 0 ...oad_dyn_part8-7-be6ace08b68bc4346456103640308cf7 | 0 ...oad_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b | 0 ...oad_dyn_part8-9-a7456fceb35f62a282db750384f480db | 0 ...load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 | 0 ...oad_dyn_part9-1-2c61920dcd46ece096fd12875871709f | 0 ...oad_dyn_part9-2-748ac33315295f8e55f2480f7714c27a | 0 ...oad_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 | 0 ...oad_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 | 0 ...oad_dyn_part9-5-16367c381d4b189b3640c92511244bfe | 0 ...oad_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 | 0 ...oad_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b | 0 ...load_dyn_part9-8-504a475a583d33f7c71db57e6774919 | 0 ...oad_dyn_part9-9-72b1ad2231269b704130903b35ac04bb | 0 ...t_authsuccess-0-84028c4ca541d126baffc20d6d876810 | 0 ...t_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 | 0 ...t_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e | 0 ...t_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 | 0 ...e_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 | 0 ...e_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f | 0 .../load_fs2-0-517732da2c84ae17095b0e1d96f74d97 | 0 .../load_fs2-1-5018c84e09be70bf663594a89f3ad731 | 0 .../load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 | 0 ...e_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc | 0 ...e_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 | 0 ...e_in_the_name-2-2087e00fe000e00f64e819dca59be450 | 0 ...t_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 | 0 ...t_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e | 0 ...t_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 | 0 ...t_authsuccess-0-84028c4ca541d126baffc20d6d876810 | 0 ...t_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e | 0 ...t_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 | 0 .../loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 | 0 .../loadpart1-1-4bf1504274319c44d370b58092fe016c | 0 .../loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 | 0 .../loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 | 0 .../loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 | 0 .../loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc | 0 .../loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 | 0 .../loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 | 0 .../loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 | 0 .../loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 | 0 .../loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 | 0 .../loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 | 0 .../loadpart_err-2-21fe8ff9059167209647e7ea086f483e | 0 .../golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 | 0 .../golden/lock1-1-3e95421993ab28d18245ec2340f580a3 | 0 .../golden/lock1-2-c0c18ac884677231a41eea8d980d0451 | 0 .../golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 | 0 .../golden/lock2-1-3e95421993ab28d18245ec2340f580a3 | 0 .../golden/lock2-2-c0c18ac884677231a41eea8d980d0451 | 0 .../golden/lock2-3-27ad2962fed131f51ba802596ba37278 | 0 .../golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a | 0 .../golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca | 0 .../golden/lock3-0-27ad2962fed131f51ba802596ba37278 | 0 .../golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a | 0 .../golden/lock3-2-b1ca816784e88f105b2fce1175340c33 | 0 .../golden/lock3-3-16367c381d4b189b3640c92511244bfe | 0 .../golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 | 0 .../golden/lock3-5-8096935c5c1755f9b88583e8c72921ac | 0 .../golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe | 0 .../golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 | 0 .../golden/lock4-1-27ad2962fed131f51ba802596ba37278 | 0 .../golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a | 0 .../golden/lock4-3-b1ca816784e88f105b2fce1175340c33 | 0 .../golden/lock4-4-16367c381d4b189b3640c92511244bfe | 0 .../golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 | 0 .../golden/lock4-6-8096935c5c1755f9b88583e8c72921ac | 0 .../golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe | 0 ...uter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 | 0 ...uter_join_ppr-1-498e526f13a05a053a338d766f7351cd | 0 ...uter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 | 0 ...uter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 | 0 ...uter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef | 0 ...uter_join_ppr-5-1209db6544b421ea32a360d863becd94 | 0 ...outer_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 | 0 ...uter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 | 0 ...outer_join_ppr-8-6fca189c46645f124d5fcb82564b703 | 0 .../mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d | 0 .../mapjoin1-1-abd9364d276ec89352232da5e2237768 | 0 .../mapjoin1-10-c08fefa00b89f50dd365208151593505 | 0 .../mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 | 0 .../mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 | 0 .../mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 | 0 .../mapjoin1-3-8439a0592619790b64d16d2506f2233d | 0 .../mapjoin1-4-c08fefa00b89f50dd365208151593505 | 0 .../mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba | 0 .../mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 | 0 .../mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 | 0 .../mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 | 0 .../mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 | 0 ...join_distinct-0-863233ccd616401efb4bf83c4b9e3a52 | 0 ...join_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 | 0 ...oin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 | 0 ...oin_distinct-11-4489654b888efc588b13ee1cda1b6a9f | 0 ...oin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 | 0 ...oin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e | 0 ...oin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 | 0 ...oin_distinct-15-4489654b888efc588b13ee1cda1b6a9f | 0 ...join_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 | 0 ...join_distinct-3-4489654b888efc588b13ee1cda1b6a9f | 0 ...join_distinct-4-863233ccd616401efb4bf83c4b9e3a52 | 0 ...join_distinct-5-a7dc16cb82c595b18d4258a38a304b1e | 0 ...join_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 | 0 ...join_distinct-7-4489654b888efc588b13ee1cda1b6a9f | 0 ...join_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 | 0 ...join_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 | 0 ..._on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d | 0 ...pjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 | 0 ...pjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 | 0 ...pjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 | 0 ...pjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 | 0 ...pjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 | 0 ...pjoin_mapjoin-5-c47698bac140454637a999e583941ce7 | 0 ...join_subquery-0-24ca942f094b14b92086305cc125e833 | 0 ...join_subquery-1-44d382ce6848d3f0b900b0808747d8e9 | 0 ...join_subquery-2-25fc734982956a164adde6bb1d4d8751 | 0 ...join_subquery-3-4422532005e909173a4799d8d5091f1b | 0 ...join_subquery-4-7df121f9774cb23edc557b98ad1e1924 | 0 ...join_subquery-5-d9e59bfa950495629b7ba4bc6700405c | 0 ...join_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b | 0 ...oin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 | 0 ...oin_subquery2-1-c0a9763a930555f846a2576d003fb517 | 0 ...in_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 | 0 ...in_subquery2-11-25fc734982956a164adde6bb1d4d8751 | 0 ...in_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 | 0 ...in_subquery2-13-c876a518451059f17fc15e29f6f57951 | 0 ...in_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 | 0 ...in_subquery2-15-c0a9763a930555f846a2576d003fb517 | 0 ...in_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 | 0 ...oin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 | 0 ...oin_subquery2-3-681c6f08aac965e4156dbd5800064a68 | 0 ...oin_subquery2-4-ab6020d67f5a99c0a87e630790507345 | 0 ...oin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 | 0 ...oin_subquery2-6-5353ee601eb42d5842690d3941683be1 | 0 ...oin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 | 0 ...oin_subquery2-8-d524906728fef9f559709fe0922ab24e | 0 ...oin_subquery2-9-24ca942f094b14b92086305cc125e833 | 0 ...in_test_outer-0-407016bf2679fb9e9d076a2d115e859d | 0 ...in_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc | 0 ...n_test_outer-10-ce1ef910fff98f174931cc641f7cef3a | 0 ...n_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 | 0 ...n_test_outer-12-80993ab7f757001e1f058bf8609f8420 | 0 ...n_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 | 0 ...n_test_outer-14-7fe52008c4a98853d086d17fc3c21906 | 0 ...in_test_outer-2-43561759b314d02b2dba5eb49a65c515 | 0 ...in_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f | 0 ...oin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 | 0 ...in_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 | 0 ...in_test_outer-6-7fe52008c4a98853d086d17fc3c21906 | 0 ...in_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 | 0 ...in_test_outer-8-dfb08d397d3fe163d75c3b758097b68a | 0 ...in_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 | 0 .../mapreduce1-0-904b34e86c266384dc261655162dde3c | 0 .../mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe | 0 .../mapreduce1-2-c32bd8b1734c410b3686469a7a3500e | 0 .../mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../mapreduce2-0-904b34e86c266384dc261655162dde3c | 0 .../mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 | 0 .../mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b | 0 .../mapreduce2-3-adea843673e541da8a735a5a34e7c7dc | 0 .../mapreduce3-0-904b34e86c266384dc261655162dde3c | 0 .../mapreduce3-1-efaeccafc3db890b344dc6037266b71b | 0 .../mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a | 0 .../mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../mapreduce4-0-904b34e86c266384dc261655162dde3c | 0 .../mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f | 0 .../mapreduce4-2-6906be683bdd3666075760de019ac5ab | 0 .../mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../mapreduce5-0-904b34e86c266384dc261655162dde3c | 0 .../mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 | 0 .../mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca | 0 .../mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../mapreduce6-0-904b34e86c266384dc261655162dde3c | 0 .../mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb | 0 .../mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 | 0 .../mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 | 0 .../mapreduce7-1-8b7f1f886b749e43460052c98acd082a | 0 .../mapreduce7-2-b57470174a24aa3861df022375754e90 | 0 .../mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 | 0 .../mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b | 0 .../mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 | 0 .../mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../merge1-0-593999fae618b6b38322bc9ae4e0c027 | 0 .../merge1-1-2c73c923962b91afdf0004a705432550 | 0 .../merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 | 0 .../merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 | 0 .../merge1-12-2e8e4adbfb21f25e7557dd86363c7138 | 0 .../merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 | 0 .../merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 | 0 .../merge1-2-34854e1283de8e9ada3edd6bf897bc67 | 0 .../merge1-3-1f560722f18ef618a0343313a0cac462 | 0 .../merge1-4-3277fe538b66923cd879b45371838d2b | 0 .../merge1-5-2f16345a20407b01e5cc5ae26ea902b0 | 0 .../merge1-6-1c02b26792354eae7fb53fa3cb752ac1 | 0 .../merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 | 0 .../merge1-8-37bd183ad3b7ad1e8550a138f7beb88a | 0 .../golden/merge1-9-64678b0928c4fd054e1578458001c86 | 0 .../merge2-0-b12e5c70d6d29757471b900b6160fa8a | 0 .../merge2-1-593999fae618b6b38322bc9ae4e0c027 | 0 .../merge2-10-9f60e54bc4add2b1aff8473e2a756c79 | 0 .../merge2-11-1c02b26792354eae7fb53fa3cb752ac1 | 0 .../merge2-12-d2a36a13f8531cde3c66d4003048416 | 0 .../merge2-13-37bd183ad3b7ad1e8550a138f7beb88a | 0 .../merge2-14-64678b0928c4fd054e1578458001c86 | 0 .../merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 | 0 .../merge2-16-d75f4405b860e4187882a72418ed6c83 | 0 .../merge2-17-2e8e4adbfb21f25e7557dd86363c7138 | 0 .../merge2-18-945682265ce2c1fe7fa69aeb57b4290f | 0 .../merge2-19-d75f4405b860e4187882a72418ed6c83 | 0 .../merge2-2-c95dc367df88c9e5cf77157f29ba2daf | 0 .../merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 | 0 .../merge2-4-84967075baa3e56fff2a23f8ab9ba076 | 0 .../merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea | 0 .../merge2-6-319e3c6b684d2abd4dfd7314a05d4307 | 0 .../merge2-7-fa2f100bc67426120cb97cfc75a5fb36 | 0 .../merge2-8-7435e0bf03e747705b0858d5dcccfcc1 | 0 .../merge2-9-b81efaa65e1263e48278ef9062cca1dd | 0 .../merge4-0-b12e5c70d6d29757471b900b6160fa8a | 0 .../merge4-1-593999fae618b6b38322bc9ae4e0c027 | 0 .../merge4-10-692a197bd688b48f762e72978f54aa32 | 0 .../merge4-11-ca0336ac3f600cb8b4230d9904686868 | 0 .../merge4-12-62541540a18d68a3cb8497a741061d11 | 0 .../merge4-13-ed1103f06609365b40e78d13c654cc71 | 0 .../merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 | 0 .../merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a | 0 .../merge4-2-43d53504df013e6b35f81811138a167a | 0 .../merge4-3-a4fb8359a2179ec70777aad6366071b7 | 0 .../merge4-4-16367c381d4b189b3640c92511244bfe | 0 .../merge4-5-3d24d877366c42030f6d9a596665720d | 0 .../golden/merge4-6-b3a76420183795720ab3a384046e5af | 0 .../merge4-7-631a45828eae3f5f562d992efe4cd56d | 0 .../merge4-8-ca0336ac3f600cb8b4230d9904686868 | 0 .../merge4-9-ad3dc168c8b6f048717e39ab16b0a319 | 0 ...mic_partition-0-a4fb8359a2179ec70777aad6366071b7 | 0 ...mic_partition-1-16367c381d4b189b3640c92511244bfe | 0 ...ic_partition-10-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ic_partition-11-a49c9ee01ce8858a5f00c05523329200 | 0 ...ic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 | 0 ...ic_partition-13-16adcdb0e324ad233769e124b5b349da | 0 ...ic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 | 0 ...ic_partition-15-d60297fed03b455c29daa4afb4d1e858 | 0 ...ic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 ...ic_partition-17-b12e5c70d6d29757471b900b6160fa8a | 0 ...ic_partition-18-593999fae618b6b38322bc9ae4e0c027 | 0 ...ic_partition-19-a49c9ee01ce8858a5f00c05523329200 | 0 ...mic_partition-2-190cefc93e46906e404039de0fd5f513 | 0 ...ic_partition-20-d295db835d4fdeea34298702295ff7c5 | 0 ...ic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 | 0 ...ic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 | 0 ...ic_partition-23-d60297fed03b455c29daa4afb4d1e858 | 0 ...ic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 ...ic_partition-25-b12e5c70d6d29757471b900b6160fa8a | 0 ...ic_partition-26-593999fae618b6b38322bc9ae4e0c027 | 0 ...ic_partition-27-a49c9ee01ce8858a5f00c05523329200 | 0 ...ic_partition-28-ef7b35be7210f099d46448994d9dc605 | 0 ...ic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d | 0 ...mic_partition-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...ic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 | 0 ...ic_partition-31-d60297fed03b455c29daa4afb4d1e858 | 0 ...mic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...mic_partition-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...mic_partition-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...mic_partition-7-777de794b7f27ea63f29a9784663393b | 0 ...mic_partition-8-2f1578dbc029b62daa9d47d8fa473960 | 0 ...mic_partition-9-dc129f70e75cd575ce8c0de288884523 | 0 ...ic_partition2-0-a4fb8359a2179ec70777aad6366071b7 | 0 ...ic_partition2-1-16367c381d4b189b3640c92511244bfe | 0 ...c_partition2-10-43d53504df013e6b35f81811138a167a | 0 ...c_partition2-11-b12e5c70d6d29757471b900b6160fa8a | 0 ...c_partition2-12-593999fae618b6b38322bc9ae4e0c027 | 0 ...c_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 | 0 ...ic_partition2-14-3a4c24fd561f459025264baa3fb6d87 | 0 ...c_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 | 0 ...c_partition2-16-16adcdb0e324ad233769e124b5b349da | 0 ...c_partition2-17-d60297fed03b455c29daa4afb4d1e858 | 0 ...ic_partition2-2-190cefc93e46906e404039de0fd5f513 | 0 ...ic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...ic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...ic_partition2-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...ic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...ic_partition2-7-777de794b7f27ea63f29a9784663393b | 0 ...ic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 | 0 ...ic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba | 0 ...ic_partition3-0-a4fb8359a2179ec70777aad6366071b7 | 0 ...ic_partition3-1-16367c381d4b189b3640c92511244bfe | 0 ...c_partition3-10-d176a1b243ac7190fbc319d73a164e2d | 0 ...c_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 | 0 ...c_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a | 0 ...c_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 | 0 ...c_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 | 0 ...c_partition3-15-e525a096de36a3d157db1b4947e1fbb0 | 0 ...c_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 | 0 ...c_partition3-17-43d53504df013e6b35f81811138a167a | 0 ...c_partition3-18-b12e5c70d6d29757471b900b6160fa8a | 0 ...c_partition3-19-593999fae618b6b38322bc9ae4e0c027 | 0 ...ic_partition3-2-190cefc93e46906e404039de0fd5f513 | 0 ...c_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 | 0 ...ic_partition3-21-3a4c24fd561f459025264baa3fb6d87 | 0 ...c_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 | 0 ...c_partition3-23-3ffba3098571099bc2b13614ae3defc5 | 0 ...c_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a | 0 ...c_partition3-25-d60297fed03b455c29daa4afb4d1e858 | 0 ...ic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...ic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...ic_partition3-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...ic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...ic_partition3-7-777de794b7f27ea63f29a9784663393b | 0 ...ic_partition3-8-6916eceaa04091d1453a7d0d5257213c | 0 ...ic_partition3-9-8d0305d089aa5198601cc39073fff528 | 0 .../mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 | 0 .../mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b | 0 .../mergejoins-2-6b9f3810606db1e9036561f1173ac75c | 0 .../mergejoins-3-c408f69470d652da283442a62b384e46 | 0 .../mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 | 0 .../mergejoins-5-adae80fe415023783fca5499e3edf6e | 0 .../mergejoins-6-6169410e9f077097d1a766724dfc51df | 0 ...gejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 | 0 ...gejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 | 0 ...ejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 | 0 ...ejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 | 0 ...ejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b | 0 ...gejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff | 0 ...gejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 | 0 ...gejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 | 0 ...gejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 | 0 ...gejoins_mixed-6-e108b1560a601946194cecaf4da12491 | 0 ...rgejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 | 0 ...gejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 | 0 ...gejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 | 0 ...metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d | 0 ...metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 | 0 ...etadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d | 0 ...etadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 | 0 ...metadataonly1-2-9eadfd16be30c653625fce7b74048d9d | 0 ...metadataonly1-3-7980a98d580a002b7ad7eef780039f67 | 0 ...metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 | 0 ...metadataonly1-5-9eadfd16be30c653625fce7b74048d9d | 0 ...metadataonly1-6-537256f669bc9101d4834df67aae8cdf | 0 ...metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc | 0 ...metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 | 0 ...metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 | 0 .../golden/mi-0-a4fb8359a2179ec70777aad6366071b7 | 0 .../golden/mi-1-16367c381d4b189b3640c92511244bfe | 0 .../golden/mi-2-abf8847fb25b96e0f9477808d8378e5e | 0 .../golden/mi-3-b66a495f7bdf106a7886b72267b8659d | 0 .../golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c | 0 .../golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc | 0 .../golden/mi-6-b331d7aa963adac3b2e1de803f90e199 | 0 .../golden/mi-7-fca4e06de103c3cbb675fa43e7077800 | 0 .../golden/mi-8-e946bdb81b0a831908c1c8196fdff215 | 0 ...multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 | 0 ...multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a | 0 ...ultiMapJoin1-10-24ca942f094b14b92086305cc125e833 | 0 ...ultiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa | 0 ...ultiMapJoin1-12-204073e1287b0582d50e652d466f1e66 | 0 ...ultiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 | 0 ...ultiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 | 0 ...ultiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 | 0 ...ultiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 | 0 ...ultiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 | 0 ...ultiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee | 0 ...ultiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 | 0 ...multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 | 0 ...ultiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 | 0 ...ultiMapJoin1-21-79973475ca07cb9932f752f6547779ac | 0 ...multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 | 0 ...multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a | 0 ...ultiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 | 0 ...ultiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 | 0 ...ultiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 | 0 ...ultiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 | 0 ...ultiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 | 0 ...ultiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f | 0 ...multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 | 0 ...ultiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 | 0 ...ultiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 | 0 ...ultiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 | 0 ...ultiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b | 0 ...ultiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 | 0 ...multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 | 0 ...multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe | 0 ...multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 | 0 ...multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba | 0 ...multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c | 0 ...multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 | 0 ...multiMapJoin2-0-24ca942f094b14b92086305cc125e833 | 0 ...multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 | 0 ...ultiMapJoin2-10-1905c7759350b107679aef86226739f8 | 0 ...ultiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f | 0 ...ultiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e | 0 ...ultiMapJoin2-13-6b984427a771fe650fa875be98722cbe | 0 ...ultiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ultiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b | 0 ...ultiMapJoin2-16-6b984427a771fe650fa875be98722cbe | 0 ...ultiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f | 0 ...ultiMapJoin2-18-cb1f964731ee7ac045db89266a919586 | 0 ...ultiMapJoin2-19-77324702b091d514ca16d029f65d3d56 | 0 ...multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 | 0 ...ultiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 | 0 ...ultiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 | 0 ...ultiMapJoin2-22-77324702b091d514ca16d029f65d3d56 | 0 ...multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 | 0 ...ultiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 | 0 ...multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 | 0 ...ultiMapJoin2-26-2136f3783a9764de762e49c1ca28637f | 0 ...ultiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e | 0 ...multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d | 0 ...multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 | 0 ...multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 | 0 ...multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 | 0 ...multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 | 0 ...multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 | 0 ...multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 | 0 ...ti_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 | 0 ...ti_insert_gby-1-9e61989d717403353689cbbb2816210d | 0 ...ti_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 | 0 ...ti_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 | 0 ...ti_insert_gby-4-521e0c1054cfa35116c02245874a4e69 | 0 ...ti_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 | 0 ...ti_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c | 0 ...ti_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f | 0 ...ti_insert_gby-8-521e0c1054cfa35116c02245874a4e69 | 0 ...ti_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 | 0 ...i_insert_gby2-0-b3ee4be40513342084411c5333416d69 | 0 ...i_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 | 0 ...i_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 | 0 ...i_insert_gby2-3-538a02b95c066b307652c8d503470c8e | 0 ...i_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 | 0 ...i_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 | 0 ...i_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 | 0 ...ti_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a | 0 ..._insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 | 0 ..._insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc | 0 ..._insert_gby3-12-33d7e716735d24b7493209810d0b865f | 0 ...i_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 | 0 ...i_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca | 0 ...i_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 | 0 ...i_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 | 0 ...i_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 | 0 ...i_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 | 0 ...i_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b | 0 ...i_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 | 0 ..._lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 | 0 ..._lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 | 0 ...lateral_view-10-f26c10c6495fc3e86319cd5815caab4f | 0 ...lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 | 0 ...lateral_view-12-622f0dd66c8633307fe56ccf9015f430 | 0 ...lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 | 0 ...lateral_view-14-f26c10c6495fc3e86319cd5815caab4f | 0 ...lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 | 0 ...lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 | 0 ...lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 | 0 ...lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a | 0 ...lateral_view-19-f26c10c6495fc3e86319cd5815caab4f | 0 ..._lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 | 0 ...lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 | 0 ...lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 | 0 ...lateral_view-22-283e0f384d0a015c252b34f79a895286 | 0 ...lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d | 0 ...lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df | 0 ...lateral_view-25-f26c10c6495fc3e86319cd5815caab4f | 0 ...lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 | 0 ...lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 | 0 ...lateral_view-28-24e041343e158735db6262136de0e8b8 | 0 ..._lateral_view-3-a60af91a18d481fe1244e21108133489 | 0 ..._lateral_view-4-67a95497041a0e81b4d5756731d8b27d | 0 ..._lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 | 0 ..._lateral_view-6-f26c10c6495fc3e86319cd5815caab4f | 0 ..._lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 | 0 ..._lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 | 0 ..._lateral_view-9-59102a6d824074da6cc85ca760385975 | 0 ...ti_join_union-0-24ca942f094b14b92086305cc125e833 | 0 ...ti_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b | 0 ...ti_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 | 0 ...ti_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 | 0 ...ti_join_union-4-8c17422bd7041c596677251a46fa4085 | 0 ...ti_join_union-5-9e89b8619411f46f675a9fab73298627 | 0 ...oupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae | 0 ...upby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 | 0 ...upby_singlemr-2-31e7358a9f1e474e67155396fe803967 | 0 ...upby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 | 0 ...upby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 | 0 ...upby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 | 0 ...upby_singlemr-6-80cb19f7a801a82add47a501756c8af7 | 0 ...upby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 | 0 ...upby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 | 0 ...upby_singlemr-9-faa99964430e2aee836221de1abe1c5e | 0 ...nested_complex-0-6a7c4841dab05ebae84309c9571bec6 | 0 ...ested_complex-1-abac744dee1a4f4152781b1565fe1364 | 0 ...ested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 | 0 ...ested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 | 0 ...ested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d | 0 ...nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...estedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...estedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...estedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...estedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...estedvirtual-14-67b834deba21676e02c155b25195a019 | 0 ...estedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f | 0 ...estedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...estedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...nestedvirtual-2-67b834deba21676e02c155b25195a019 | 0 ...nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f | 0 ...nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...nestedvirtual-8-67b834deba21676e02c155b25195a019 | 0 ...nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f | 0 .../newline-0-43392a20a8d249a279d50d96578e6a1b | 0 .../newline-1-a19a19272149c732977c37e043910505 | 0 .../golden/newline-2-4eb54a664e549614d56ca088c8867d | 0 ...noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 | 0 ...noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 | 0 ...ous_table_col-0-535a12e87c72793bfba96520a0ea251b | 0 ...ous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 | 0 ...ous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 | 0 ...ous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 | 0 ...p_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 | 0 ...p_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b | 0 ...p_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 | 0 ...p_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 | 0 ...p_deduplicate-4-24ca942f094b14b92086305cc125e833 | 0 ...p_deduplicate-5-250d196b4449c835ddc518db2d2ab726 | 0 ...p_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 | 0 ...words_input37-0-6ed1b2ff177492c003161ee91e982c10 | 0 ...words_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 | 0 ..._insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 | 0 ..._insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 | 0 ...insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 | 0 ...insert_into1-11-6f59e35684a552a855e4dc3aee667092 | 0 ...insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 | 0 ..._insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 | 0 ..._insert_into1-3-2983d09b973ea94bc701970a17fc3687 | 0 ..._insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 | 0 ..._insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 | 0 ..._insert_into1-6-2983d09b973ea94bc701970a17fc3687 | 0 ..._insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 | 0 ..._insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad | 0 ..._insert_into1-9-461110270cec5b6d012b2c685cf2cce9 | 0 ...otable_alias1-0-695de796f21dce21056f8ba27cbadb0d | 0 ...otable_alias1-1-c4d51044e6802df266aa0dc137b825cd | 0 ...otable_alias1-2-bea59f268594a034d06b826c7a8e516e | 0 ...otable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f | 0 ...otable_alias2-0-695de796f21dce21056f8ba27cbadb0d | 0 ...otable_alias2-1-20b7bf01acc8d88670d347e6759aa407 | 0 ...otable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 | 0 ...otable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../null_cast-0-48a80d4fdc8009234af31ebcb6e03364 | 0 .../null_cast-1-7257e6f8170e545962d27741353f672c | 0 .../nullgroup-0-dbcec232623048c7748b708123e18bf0 | 0 .../nullgroup-1-c67a488530dc7e20a9e7acf02c14380f | 0 .../nullgroup-10-3994c1896dace613fa9f837b2f1676c0 | 0 .../nullgroup-11-54a5fd76cdeff565c8c7724695aca302 | 0 .../nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../nullgroup-13-85c4f90b754cd88147d6b74e17d22063 | 0 .../nullgroup-14-3994c1896dace613fa9f837b2f1676c0 | 0 .../nullgroup-15-54a5fd76cdeff565c8c7724695aca302 | 0 .../nullgroup-2-3994c1896dace613fa9f837b2f1676c0 | 0 .../nullgroup-3-54a5fd76cdeff565c8c7724695aca302 | 0 .../nullgroup-4-dbcec232623048c7748b708123e18bf0 | 0 .../nullgroup-5-85c4f90b754cd88147d6b74e17d22063 | 0 .../nullgroup-6-3994c1896dace613fa9f837b2f1676c0 | 0 .../nullgroup-7-54a5fd76cdeff565c8c7724695aca302 | 0 .../nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../nullgroup-9-c67a488530dc7e20a9e7acf02c14380f | 0 .../nullgroup2-0-dbcec232623048c7748b708123e18bf0 | 0 .../nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f | 0 .../nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 | 0 .../nullgroup2-11-644b68261df70bbb46a3045a8abde17a | 0 .../nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 | 0 .../nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 | 0 .../nullgroup2-15-644b68261df70bbb46a3045a8abde17a | 0 .../nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 | 0 .../nullgroup2-3-644b68261df70bbb46a3045a8abde17a | 0 .../nullgroup2-4-dbcec232623048c7748b708123e18bf0 | 0 .../nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 | 0 .../nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 | 0 .../nullgroup2-7-644b68261df70bbb46a3045a8abde17a | 0 .../nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f | 0 .../nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 | 0 .../nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 | 0 .../nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 | 0 .../nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 | 0 .../nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 | 0 .../nullgroup3-13-f529994bef750d8edd02ede8a4323afb | 0 .../nullgroup3-14-d4e815f44f6369c991ea4390c481f31e | 0 .../nullgroup3-15-cd90e16da918bac569e9f04faaedd280 | 0 .../nullgroup3-16-af767d7cfb4601ace72a3ed718071931 | 0 .../nullgroup3-17-45699aee54227552bb3ec84d92dfe450 | 0 .../nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 | 0 .../nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 | 0 .../nullgroup3-2-f529994bef750d8edd02ede8a4323afb | 0 .../nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 | 0 .../nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 | 0 .../nullgroup3-3-d4e815f44f6369c991ea4390c481f31e | 0 .../nullgroup3-4-cd90e16da918bac569e9f04faaedd280 | 0 .../nullgroup3-5-45699aee54227552bb3ec84d92dfe450 | 0 .../nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 | 0 .../nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 | 0 .../nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 | 0 .../nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 | 0 .../nullgroup4-0-dbcec232623048c7748b708123e18bf0 | 0 .../nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f | 0 .../nullgroup4-10-3a3f180144fead81b9e8b232b4132762 | 0 .../nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 | 0 .../nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 | 0 .../nullgroup4-14-3a3f180144fead81b9e8b232b4132762 | 0 .../nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 | 0 .../nullgroup4-2-3a3f180144fead81b9e8b232b4132762 | 0 .../nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 | 0 .../nullgroup4-4-dbcec232623048c7748b708123e18bf0 | 0 .../nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 | 0 .../nullgroup4-6-3a3f180144fead81b9e8b232b4132762 | 0 .../nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 | 0 .../nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 .../nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f | 0 ...ulti_distinct-0-dbcec232623048c7748b708123e18bf0 | 0 ...ulti_distinct-1-85c4f90b754cd88147d6b74e17d22063 | 0 ...ulti_distinct-2-43dec71d76c386394196a0e9d69457e5 | 0 ...ulti_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 | 0 ...ulti_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...ulti_distinct-5-85c4f90b754cd88147d6b74e17d22063 | 0 ...ulti_distinct-6-43dec71d76c386394196a0e9d69457e5 | 0 ...ulti_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 | 0 .../nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 | 0 .../nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b | 0 .../nullgroup5-2-45699aee54227552bb3ec84d92dfe450 | 0 .../nullgroup5-3-4492a9ce0d8502584b872860d53c449c | 0 .../nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e | 0 .../nullgroup5-5-60d7180632a63c79eeba47e30b854f4c | 0 .../nullinput-0-2be9511f8cb3c9edaf50353187f24c11 | 0 .../nullinput-1-bf48040d6cc6213cba90dbd76a796a66 | 0 .../nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf | 0 .../nullinput2-0-40b73200f853b473855e85eba391f008 | 0 .../nullinput2-1-507c5a854d930361d8db66a3b90a5388 | 0 .../nullinput2-2-21058230c4992a682c4adef9881fa9a2 | 0 .../nullscript-0-4477b1566208a6805bb20630755a9375 | 0 .../nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 | 0 .../nullscript-2-17238164053203d56d30704e2c098e80 | 0 .../nullscript-3-56d47cf9468111b12fcef408bc767271 | 0 .../nullscript-4-472199a0c6b8f760a90863deb69e9710 | 0 ..._op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc | 0 ..._op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 | 0 ...ptional_outer-0-c157cc7014eda416b6248cf56165b62a | 0 ...ptional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 | 0 ...ptional_outer-2-7871ab752b7065f13fb808c8d7cdb35f | 0 ...ptional_outer-3-4815a6c9014b80a79f61ab983dcdd23a | 0 ...ptional_outer-4-9519008c6ef6e8ed651c7f4405589c34 | 0 ...ptional_outer-5-4a868da4f3f6d96124bff2cf50a2673e | 0 .../orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 .../orc_create-1-b5209e6f850fc958dc9ebced21519684 | 0 .../orc_create-10-3027edcda6a6030c6bff93681529c34d | 0 .../orc_create-11-14ea7dcc6898979aaa61650e3fc46187 | 0 .../orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 | 0 .../orc_create-13-14ea7dcc6898979aaa61650e3fc46187 | 0 .../orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 .../orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 | 0 .../orc_create-16-3027edcda6a6030c6bff93681529c34d | 0 .../orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb | 0 .../orc_create-18-14ea7dcc6898979aaa61650e3fc46187 | 0 .../orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 | 0 .../orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 | 0 .../orc_create-20-176d469a0edba57404416535c7d48023 | 0 .../orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 | 0 .../orc_create-22-31944ad765bbf262b4ebafdb06df59a6 | 0 .../orc_create-23-be779533ea8967231e644209114c8350 | 0 .../orc_create-24-fe59ff341395bc347dfb9cfdee397da2 | 0 .../orc_create-25-c55e620d82921c40ebcdb94454ac1ead | 0 .../orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c | 0 .../orc_create-27-2eefc0bb4283fc255409a5c41203c89 | 0 .../orc_create-28-cf8aa1014707dfe576820041e47436e2 | 0 .../orc_create-29-cee6b57822d79ce80e14e58663bf7b86 | 0 .../orc_create-3-8480c37b4f3a6768f459361cf8470dae | 0 .../orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d | 0 .../orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 | 0 .../orc_create-32-16367c381d4b189b3640c92511244bfe | 0 .../orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 | 0 .../orc_create-34-3b03210f94ec40db9ab02620645014d1 | 0 .../orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 | 0 .../orc_create-36-ca15a32658195ecaffe11d11f487fb0a | 0 .../orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 .../orc_create-38-b5209e6f850fc958dc9ebced21519684 | 0 .../orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 | 0 .../orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 | 0 .../orc_create-40-8480c37b4f3a6768f459361cf8470dae | 0 .../orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 | 0 .../orc_create-5-4f8e2d1f18399679a97c49631c4ef921 | 0 .../orc_create-6-2ae0fd655aa777b41015e4125f680324 | 0 .../orc_create-7-27aa4a8093e80a7437266f349ea927c0 | 0 .../orc_create-8-14ea7dcc6898979aaa61650e3fc46187 | 0 .../orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ...ary_threshold-0-a399c0ef0f1578f9d2456727008dee16 | 0 ...ary_threshold-1-a2f8227aafaee48079235d466c5049a0 | 0 ...ary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c | 0 ...nary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d | 0 ...ary_threshold-4-f3df937417e80909d4a4b220779173b0 | 0 ...ary_threshold-5-f5387ae613d41813186d7f1399545b02 | 0 ...ary_threshold-6-272f5d299289829dc22cc31f70115dd9 | 0 ...ary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 | 0 ...ary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 | 0 ...ary_threshold-9-f7b722063a6948d22aaaab0707cddde1 | 0 ...iff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 | 0 ...iff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 ...iff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 | 0 ...iff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 | 0 ...iff_part_cols-4-1c1eab8fc34159875afe38eb2413434e | 0 ...iff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 | 0 ...c_empty_files-0-eedb40290338ab680a930542cc7ddf0c | 0 ...c_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...rc_empty_files-2-7cccbdffc32975f8935eeba14a28147 | 0 ...c_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 ...c_empty_files-4-9c1451024d868c99833bbe1173c703d4 | 0 ...c_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 | 0 ..._empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 | 0 ...empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf | 0 ...empty_strings-2-a34bd419a63852c1c75f195a495ff333 | 0 ...empty_strings-3-3339ace17de3201296847caf29c42e99 | 0 ...empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e | 0 ...nds_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 | 0 ...ds_with_nulls-1-2c16215823a5e6904059a48a3077da4e | 0 ...ds_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 | 0 ...ds_with_nulls-3-bbe8d353c397b795e2732bd59648d291 | 0 ...ds_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc | 0 ...ds_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f | 0 .../golden/order-0-5c2fda577771db3b316e0d2bd02d048a | 0 .../golden/order-1-57d93bd7619dfc460dfa763c12142bb9 | 0 .../golden/order-2-fe90320d98850ea5e9b6100f6d259fbf | 0 .../golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 | 0 .../order2-0-ae225e86c2ae20519ffdf23190454161 | 0 .../order2-1-484c94d6b57ccc9fffda150bee0bccdf | 0 .../order2-2-4162aa366dc0836eed15cc819226907f | 0 ...uter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 | 0 ...uter_join_ppr-1-1a374dec627d7109276f008f31be517c | 0 ...uter_join_ppr-2-b60ab17f7372863504804717c4276595 | 0 ...uter_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba | 0 ...outer_join_ppr-4-be64a45a645ea5363109323a378d335 | 0 .../parallel-0-23a4feaede17467a8cc26e4d86ec30f9 | 0 .../parallel-1-851e262128626126ae1ad87869db7c54 | 0 .../parallel-10-d196279de0b7c4b42521aa23634ca506 | 0 .../parallel-11-6230286bc168af7b010968b543690a2a | 0 .../parallel-12-73a915d42e62c0e895a82602a502ee43 | 0 .../parallel-2-43d53504df013e6b35f81811138a167a | 0 .../parallel-3-195007722f0c2921f9c6c1eb461b6d2a | 0 .../parallel-4-fcf032bbe1f1569d934da7090db60a83 | 0 .../parallel-5-e27b511ffc5828b09069121031d17a2f | 0 .../parallel-6-d196279de0b7c4b42521aa23634ca506 | 0 .../parallel-7-6230286bc168af7b010968b543690a2a | 0 .../parallel-8-73a915d42e62c0e895a82602a502ee43 | 0 .../parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c | 0 ...hesis_star_by-0-57a4ea931689f9475b687292f34abfa4 | 0 ...hesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 | 0 ...hesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc | 0 ...hesis_star_by-3-498e2973594ccf45448ba19552bfb1cd | 0 ...hesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec | 0 ...hesis_star_by-5-6888c7f7894910538d82eefa23443189 | 0 ...hesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 | 0 ...hesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea | 0 .../partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 | 0 .../partInit-1-c0ec92801bec7ece0a156d407b601f7b | 0 .../partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 | 0 .../partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 | 0 .../partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 | 0 ...rit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 | 0 ...rit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 | 0 ...erit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb | 0 ...rit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 0 ...rit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 | 0 ...l_props_empty-0-c367ba7f534037ab96efc7f2273508c7 | 0 ...l_props_empty-1-d697ec36ecf73b0ee789972e6980e460 | 0 ...bl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb | 0 ...l_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 0 ...ops_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 | 0 ...ops_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 | 0 ...rops_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb | 0 ...ops_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 0 ...ops_with_star-4-c04c695a6ebed215889ce75edcb33eb4 | 0 .../partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a | 0 .../partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 | 0 .../partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 | 0 ...n_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 | 0 ...n_decode_name-1-4de8e998198c8df484e9102f60ba05c1 | 0 ..._decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 | 0 ..._decode_name-11-94d637f6e5cee2771b9844438008a618 | 0 ...n_decode_name-2-a4fb8359a2179ec70777aad6366071b7 | 0 ...n_decode_name-3-16367c381d4b189b3640c92511244bfe | 0 ...n_decode_name-4-e90740a9a52c37a964ee204379f01412 | 0 ...n_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...n_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 | 0 ...n_decode_name-7-94d637f6e5cee2771b9844438008a618 | 0 ...n_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 | 0 ...n_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...ition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c | 0 ...ition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 | 0 ...ition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 | 0 ...ition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb | 0 ...ition_schema1-4-9b756f83973c37236e72f6927b1c02d7 | 0 ...ition_schema1-5-52a518a4f7132598998c4f6781fd7634 | 0 ..._serde_format-0-65b98f7ed3ca5907e7ca5206de94939d | 0 ..._serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 | 0 ..._serde_format-2-47b559b01e389cc9e327a2fd29255acf | 0 ..._serde_format-3-54d18742b4eab85edd1946ef139771b4 | 0 ..._special_char-0-6ae3e0805cbfa967f50ad222bc33b772 | 0 ..._special_char-1-4de8e998198c8df484e9102f60ba05c1 | 0 ..._special_char-2-a4fb8359a2179ec70777aad6366071b7 | 0 ..._special_char-3-16367c381d4b189b3640c92511244bfe | 0 ..._special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ..._special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 | 0 ..._special_char-6-94d637f6e5cee2771b9844438008a618 | 0 ..._special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ..._special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 | 0 ..._special_char-9-94d637f6e5cee2771b9844438008a618 | 0 ...on_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c | 0 ...on_type_check-1-e676cfebd53fcc4c86407028dff8ddaa | 0 ...on_type_check-10-d51791c8b809ac86dc5b0f493a938fc | 0 ...n_type_check-11-a1164f1770d2f787b520fbc3d345911a | 0 ...on_type_check-2-5e857e1536264658caf0df9b7e28652f | 0 ...on_type_check-3-7880e06d5cff6ea961dba5ff533cf346 | 0 ...on_type_check-4-45fb706ff448da1fe609c7ff76a80d4d | 0 ...on_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 | 0 ...on_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 | 0 ...on_type_check-7-7880e06d5cff6ea961dba5ff533cf346 | 0 ...on_type_check-8-45fb706ff448da1fe609c7ff76a80d4d | 0 ...on_type_check-9-57b300095c52fd652e1d414724523587 | 0 ...tion_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 | 0 ...tion_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 | 0 ...ion_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 | 0 ...tion_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 | 0 ...ion_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 | 0 ...ion_varchar1-13-4b9a64b41647af09b2f420c3b23b811c | 0 ...ion_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 | 0 ...ion_varchar1-15-6442628796f33b87ef1ca11945fa4b48 | 0 ...ion_varchar1-16-3300093c71a371aed6225ffa9e05fc3d | 0 ...ion_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 | 0 ...ion_varchar1-18-3df8ffe4c220764e59efb874bab97b9a | 0 ...ion_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 | 0 ...tion_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 | 0 ...tion_varchar1-3-968759281b7d1bf0a60991ed04953b93 | 0 ...tion_varchar1-4-deb9b7715610152bda285a3a33b772ef | 0 ...tion_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce | 0 ...tion_varchar1-6-ad21f89ac813692cf47343c66e302ea6 | 0 ...tion_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e | 0 ...tion_varchar1-8-6b968247a2c615af03fc6a386d72db9b | 0 ...tion_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 | 0 ...able_metadata-0-c25482149887659ec66d7534cad22f63 | 0 ...able_metadata-1-29f193b3e8def419733366c578e8a236 | 0 ...able_metadata-2-14033db9e60aa0bc4f64376c96195adf | 0 ...able_metadata-3-9bdf636332ca722093413523366efa86 | 0 ...able_metadata-4-e784348b7221bb26830cc1ebda69bdde | 0 ...se_fileformat-0-c854b607353e810be297d3159be30da4 | 0 ...se_fileformat-1-9fad934993b77eef15a5d10eb203a378 | 0 ...e_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 0 ...e_fileformat-11-606ad10de7caf7e65e09778f2673e712 | 0 ...e_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 | 0 ...e_fileformat-13-c262e8c736b67119b9806f69eb492ef3 | 0 ...e_fileformat-14-da1b1887eb530c7e9d37667b99c9793f | 0 ...e_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff | 0 ...e_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 | 0 ...e_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 | 0 ...e_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 0 ...e_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 | 0 ...se_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 | 0 ...e_fileformat-20-606ad10de7caf7e65e09778f2673e712 | 0 ...e_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 | 0 ...e_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a | 0 ...e_fileformat-23-c262e8c736b67119b9806f69eb492ef3 | 0 ...e_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f | 0 ...se_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 | 0 ...se_fileformat-4-606ad10de7caf7e65e09778f2673e712 | 0 ...se_fileformat-5-c262e8c736b67119b9806f69eb492ef3 | 0 ...se_fileformat-6-6c4f7b115f18953dcc7710fa97287459 | 0 ...se_fileformat-7-f5f427b174dca478c14eddc371c0025a | 0 ...se_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 | 0 ...se_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 | 0 ..._fileformat10-0-66ee62178e3576fb38cb09800cb610bf | 0 ..._fileformat10-1-198cb7d650a506ec3420b94b82a01375 | 0 ..._fileformat10-2-f723dedd396bd468107755b4495c1031 | 0 ..._fileformat10-3-c278fd699aa25809bdef310fb92f510e | 0 ..._fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf | 0 ..._fileformat10-5-dd4c3f1636af9a7371edf7142abee088 | 0 ..._fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf | 0 ..._fileformat10-7-c278fd699aa25809bdef310fb92f510e | 0 ..._fileformat11-0-66ee62178e3576fb38cb09800cb610bf | 0 ..._fileformat11-1-b30db33650de0545cbcd53769ed7cf40 | 0 ...fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b | 0 ...fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 | 0 ..._fileformat11-3-61f1abcdc66a64c11df85dded920d167 | 0 ..._fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b | 0 ..._fileformat11-6-ee7394c912991b8cd4401fb94942351f | 0 ..._fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b | 0 ..._fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ..._fileformat12-0-66ee62178e3576fb38cb09800cb610bf | 0 ..._fileformat12-1-b30db33650de0545cbcd53769ed7cf40 | 0 ...fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b | 0 ...fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd | 0 ...fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 | 0 ...fileformat12-14-d35f445501407d6fae5c3ad161fb2236 | 0 ...fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 | 0 ..._fileformat12-3-61f1abcdc66a64c11df85dded920d167 | 0 ..._fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b | 0 ..._fileformat12-6-ee7394c912991b8cd4401fb94942351f | 0 ..._fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b | 0 ..._fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d | 0 ..._fileformat13-0-66ee62178e3576fb38cb09800cb610bf | 0 ...e_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 | 0 ...fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 | 0 ..._fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 | 0 ..._fileformat13-3-9b3e4a236550f1900a36566830b24024 | 0 ..._fileformat13-4-49cf189a09e11c2c635fbb574b89a2af | 0 ..._fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b | 0 ..._fileformat13-6-fa2f56078df18add8e5a77e538756488 | 0 ..._fileformat13-7-1d822cc037d9281ce172e2d5685b1495 | 0 ..._fileformat13-8-e4531456a7418952ec1d69e18bc8460b | 0 ..._fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d | 0 ..._fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ..._fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 | 0 ...fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe | 0 ...fileformat14-12-be84e8199b0a3b9f72e552018854ac15 | 0 ...fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f | 0 ...fileformat14-14-b89ea2173180c8ae423d856f943e061f | 0 ...fileformat14-15-dd6e9965d271bd35604059540c23d967 | 0 ...fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 | 0 ...fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 | 0 ...fileformat14-18-58080916a9f5883121bcaad719be0309 | 0 ...e_fileformat14-2-7cccbdffc32975f8935eeba14a28147 | 0 ..._fileformat14-3-2683f9835169a568c1f03dae859d27d2 | 0 ..._fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 | 0 ..._fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 | 0 ..._fileformat14-6-b5165befb75ebeed42f3e69d4d64375c | 0 ..._fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 | 0 ..._fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 | 0 ..._fileformat14-9-adeaa033260f16c5bc106e761e5fef8b | 0 ..._fileformat15-0-66ee62178e3576fb38cb09800cb610bf | 0 ..._fileformat15-1-e1cf6c355de3ae8db7564b1676199117 | 0 ...fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd | 0 ...fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 | 0 ...fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 | 0 ...fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat15-2-107d7c681b43611df056238be242127b | 0 ..._fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b | 0 ..._fileformat15-5-ee7394c912991b8cd4401fb94942351f | 0 ..._fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b | 0 ..._fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ..._fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b | 0 ..._fileformat16-0-66ee62178e3576fb38cb09800cb610bf | 0 ..._fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 | 0 ...fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd | 0 ...fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 | 0 ...fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 | 0 ...fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat16-2-107d7c681b43611df056238be242127b | 0 ..._fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b | 0 ..._fileformat16-5-ee7394c912991b8cd4401fb94942351f | 0 ..._fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b | 0 ..._fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd | 0 ..._fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ..._fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b | 0 ..._fileformat17-0-7c49277a7217a147685d30e27822d273 | 0 ..._fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 | 0 ...e_fileformat2-0-c854b607353e810be297d3159be30da4 | 0 ...e_fileformat2-1-9fad934993b77eef15a5d10eb203a378 | 0 ..._fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d | 0 ..._fileformat2-11-20a02894f5e9340e89b55a30bef252b7 | 0 ...e_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 | 0 ...e_fileformat2-3-f5f427b174dca478c14eddc371c0025a | 0 ...e_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...e_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...e_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 | 0 ...e_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d | 0 ...e_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 | 0 ...e_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada | 0 ...e_fileformat3-0-c854b607353e810be297d3159be30da4 | 0 ...e_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 | 0 ..._fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 | 0 ...e_fileformat3-2-f5f427b174dca478c14eddc371c0025a | 0 ...e_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 0 ...e_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...e_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...e_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 | 0 ...e_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a | 0 ...e_fileformat3-8-f5f427b174dca478c14eddc371c0025a | 0 ...e_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 0 ...e_fileformat4-0-c854b607353e810be297d3159be30da4 | 0 ...e_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 | 0 ...e_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 | 0 ...e_fileformat4-3-9837451512e92e982f1bd9a12b132e84 | 0 ...e_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 | 0 ...e_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 | 0 ...e_fileformat5-0-66ee62178e3576fb38cb09800cb610bf | 0 ...e_fileformat5-1-c854b607353e810be297d3159be30da4 | 0 ...e_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 | 0 ...e_fileformat5-3-f5f427b174dca478c14eddc371c0025a | 0 ...e_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...e_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...e_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 | 0 ...e_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad | 0 ...e_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 | 0 ...e_fileformat6-0-66ee62178e3576fb38cb09800cb610bf | 0 ...e_fileformat6-1-c854b607353e810be297d3159be30da4 | 0 ...e_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 | 0 ...e_fileformat6-3-f5f427b174dca478c14eddc371c0025a | 0 ...e_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...e_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...e_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f | 0 ...e_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 | 0 ...e_fileformat7-0-66ee62178e3576fb38cb09800cb610bf | 0 ...e_fileformat7-1-c854b607353e810be297d3159be30da4 | 0 ...e_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 | 0 ...e_fileformat7-3-f5f427b174dca478c14eddc371c0025a | 0 ...e_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 | 0 ...e_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 | 0 ...e_fileformat8-0-66ee62178e3576fb38cb09800cb610bf | 0 ...e_fileformat8-1-22e3d59a0423473051535684bca72b27 | 0 ...e_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 | 0 ...e_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 | 0 ...e_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 | 0 ...e_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e | 0 ...e_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e | 0 ...e_fileformat8-7-d1a5913edaaec9654edb333e8207f57b | 0 ...e_fileformat8-8-624b059dfaa86e2c78f065169de325cd | 0 ...e_fileformat9-0-66ee62178e3576fb38cb09800cb610bf | 0 ...e_fileformat9-1-22e3d59a0423473051535684bca72b27 | 0 ...e_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 | 0 ...e_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 | 0 ...e_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 | 0 ...e_fileformat9-5-d1a5913edaaec9654edb333e8207f57b | 0 ...e_fileformat9-6-624b059dfaa86e2c78f065169de325cd | 0 .../plan_json-0-74146da55d57b22443140e7fbab3375c | 0 .../golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 | 0 .../golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 | 0 .../golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 | 0 .../golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 | 0 .../golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 | 0 .../golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 | 0 .../golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 | 0 .../golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 | 0 .../golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 | 0 .../golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b | 0 .../golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 | 0 .../golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 | 0 .../golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 | 0 .../golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 | 0 .../golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b | 0 ...ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 | 0 ...ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 ...pd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 | 0 ...ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 | 0 ...ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 | 0 ...ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 | 0 ...ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 | 0 ...ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 | 0 ...ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 | 0 ...ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 | 0 ...ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 | 0 ...constant_expr-0-ae225e86c2ae20519ffdf23190454161 | 0 ...constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 ...constant_expr-2-2ab005b2ee21deedbe7d10904a034468 | 0 ...constant_expr-3-5b20725caf905c6674759fee7873d627 | 0 ..._constant_expr-4-aded77ef8dced4717d919a949d109b0 | 0 ...constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c | 0 ...constant_expr-6-145c2779dadb5bd921dc2baac608b803 | 0 ...constant_expr-7-5b20725caf905c6674759fee7873d627 | 0 ..._constant_expr-8-aded77ef8dced4717d919a949d109b0 | 0 ...constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c | 0 ...onstant_where-0-345fa30edd72d631fee21c3beeeef3d9 | 0 ...onstant_where-1-84c951641740895ca1c8ddc098805da5 | 0 .../ppd_gby-0-ae225e86c2ae20519ffdf23190454161 | 0 .../ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../ppd_gby-2-fea67192aba8704a64a85da47f7f321e | 0 .../ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d | 0 .../ppd_gby-4-145c2779dadb5bd921dc2baac608b803 | 0 .../ppd_gby-5-fea67192aba8704a64a85da47f7f321e | 0 .../ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d | 0 .../ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 | 0 .../ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 | 0 .../ppd_gby2-3-321628d4c52f6992f2680a3a162f19f | 0 .../ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 | 0 .../ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 | 0 .../ppd_gby2-6-321628d4c52f6992f2680a3a162f19f | 0 .../ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 | 0 .../ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 | 0 .../ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 | 0 .../ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 | 0 .../ppd_join-0-ae225e86c2ae20519ffdf23190454161 | 0 .../ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 | 0 .../ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 | 0 .../ppd_join-4-145c2779dadb5bd921dc2baac608b803 | 0 .../ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 | 0 .../ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 | 0 .../ppd_join2-0-ae225e86c2ae20519ffdf23190454161 | 0 .../ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d | 0 .../ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 | 0 .../ppd_join2-4-145c2779dadb5bd921dc2baac608b803 | 0 .../ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d | 0 .../ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 | 0 .../ppd_join3-0-ae225e86c2ae20519ffdf23190454161 | 0 .../ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 | 0 .../ppd_join3-3-42cd793c031af1f7961d7b5e237de76b | 0 .../ppd_join3-4-145c2779dadb5bd921dc2baac608b803 | 0 .../ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 | 0 .../ppd_join3-6-42cd793c031af1f7961d7b5e237de76b | 0 ...d_join_filter-0-ae225e86c2ae20519ffdf23190454161 | 0 ...d_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 ..._join_filter-10-2c1710aa3e08f618c1930305ebdccc17 | 0 ..._join_filter-11-41debade389a200de226d8ad0fe47d24 | 0 ...d_join_filter-12-3bcc51a124f8cbd456620853d808354 | 0 ..._join_filter-13-145c2779dadb5bd921dc2baac608b803 | 0 ..._join_filter-14-2c1710aa3e08f618c1930305ebdccc17 | 0 ..._join_filter-15-41debade389a200de226d8ad0fe47d24 | 0 ...d_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 | 0 ...d_join_filter-3-41debade389a200de226d8ad0fe47d24 | 0 ...d_join_filter-4-ae225e86c2ae20519ffdf23190454161 | 0 ...d_join_filter-5-145c2779dadb5bd921dc2baac608b803 | 0 ...d_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 | 0 ...d_join_filter-7-41debade389a200de226d8ad0fe47d24 | 0 ...d_join_filter-8-73819ea1a7c0653a61652b3766afb003 | 0 ...d_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 | 0 ...d_outer_join1-0-ae225e86c2ae20519ffdf23190454161 | 0 ...d_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 ...d_outer_join1-2-a239ac7d9503d93e0859043019f3e02e | 0 ...d_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 | 0 ...d_outer_join1-4-145c2779dadb5bd921dc2baac608b803 | 0 ...d_outer_join1-5-a239ac7d9503d93e0859043019f3e02e | 0 ...d_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 | 0 ...d_outer_join2-0-ae225e86c2ae20519ffdf23190454161 | 0 ...d_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 ...d_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 | 0 ...d_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 | 0 ...d_outer_join2-4-145c2779dadb5bd921dc2baac608b803 | 0 ...d_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 | 0 ...d_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 | 0 ...d_outer_join3-0-ae225e86c2ae20519ffdf23190454161 | 0 ...d_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 ...d_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 | 0 ...d_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 | 0 ...d_outer_join3-4-145c2779dadb5bd921dc2baac608b803 | 0 ...d_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 | 0 ...d_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 | 0 ...d_outer_join4-0-ae225e86c2ae20519ffdf23190454161 | 0 ...d_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 ...d_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 | 0 ...d_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe | 0 ...d_outer_join4-4-145c2779dadb5bd921dc2baac608b803 | 0 ...d_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 | 0 ...d_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe | 0 ...d_outer_join5-0-ae225e86c2ae20519ffdf23190454161 | 0 ...d_outer_join5-1-145c2779dadb5bd921dc2baac608b803 | 0 ..._outer_join5-10-b12aa87aaf64b573ce0be7013117651b | 0 ..._outer_join5-11-c8e68d5a524b965c244f01782bc42e97 | 0 ...d_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 | 0 ...d_outer_join5-2-5b859f606230a70698edf52ca814beee | 0 ...d_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 | 0 ...d_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 | 0 ...d_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 | 0 ...d_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a | 0 ...d_outer_join5-7-54d9885a797deb82236499f6bb99ddfd | 0 ...d_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c | 0 ...d_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 | 0 .../ppd_random-0-ae225e86c2ae20519ffdf23190454161 | 0 .../ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../ppd_random-2-a589a2f51ebb3962202698c79db4a33b | 0 .../ppd_random-3-145c2779dadb5bd921dc2baac608b803 | 0 .../ppd_random-4-a589a2f51ebb3962202698c79db4a33b | 0 ...epeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...epeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...epeated_alias-2-588c0c8007b156167714d1cb06d10384 | 0 ...epeated_alias-3-106d8664d84c73f86268cf69e9ba6834 | 0 ...epeated_alias-4-2cd71603d825cddd0d181bd240c0051f | 0 ...epeated_alias-5-941a7af885ec77d91a78e03a6a568220 | 0 ...epeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 | 0 ...epeated_alias-7-54b70fb31202186b8984ae2feea299bf | 0 ...epeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...epeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...ppd_transform-0-ae225e86c2ae20519ffdf23190454161 | 0 ...ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 ...ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 | 0 ...ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 | 0 ...ppd_transform-4-145c2779dadb5bd921dc2baac608b803 | 0 ...ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 | 0 ...ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 | 0 .../ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 | 0 .../ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 | 0 .../ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 | 0 .../ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 | 0 .../ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e | 0 .../ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed | 0 .../ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae | 0 .../ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 | 0 .../ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 | 0 .../ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e | 0 .../ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed | 0 .../ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae | 0 .../ppd_union-0-ae225e86c2ae20519ffdf23190454161 | 0 .../ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 | 0 .../ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 | 0 .../ppd_union-3-678c8197f458b459171c266f7431683e | 0 .../ppd_union-4-145c2779dadb5bd921dc2baac608b803 | 0 .../ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 | 0 .../ppd_union-6-678c8197f458b459171c266f7431683e | 0 ...pd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 | 0 ...pd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 | 0 ...d_union_view-10-a74a5ff32204d842175b3d880477178f | 0 ...d_union_view-11-745e750f4f4a36af27e87338a979240c | 0 ...pd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 | 0 ...pd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 | 0 ...pd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 | 0 ...pd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 | 0 ...pd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 | 0 ...pd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c | 0 ...pd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 | 0 ...pd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c | 0 .../ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 | 0 .../ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 | 0 .../ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 | 0 .../ppd_vc-3-be78760949abf728580442e9e37a3ce7 | 0 ...childsarenull-0-ae225e86c2ae20519ffdf23190454161 | 0 ...childsarenull-1-965b997838e7cc654cac68baef1be41d | 0 ...childsarenull-2-22b7886d627e6266f5f1415ba239c621 | 0 ...childsarenull-3-73819ea1a7c0653a61652b3766afb003 | 0 ...childsarenull-4-965b997838e7cc654cac68baef1be41d | 0 ...childsarenull-5-22b7886d627e6266f5f1415ba239c621 | 0 .../ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b | 0 .../ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 | 0 ...ppr_pushdown-10-178be64f122542983ff4384df4bb1530 | 0 ...ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 | 0 ...ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 | 0 ...ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 | 0 ...ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c | 0 ...ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb | 0 ...ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae | 0 ...ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 | 0 ...ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 | 0 ...ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a | 0 .../ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 | 0 ...ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 | 0 ...ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d | 0 ...ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d | 0 ...ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 | 0 ...ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 | 0 ...ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b | 0 ...ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 | 0 ...ppr_pushdown-27-b72de558c88ae91460989938000e0d27 | 0 ...ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 | 0 ...ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea | 0 .../ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 | 0 ...ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 | 0 ...ppr_pushdown-31-5eba4bf10315099129eae319d73636cf | 0 ...ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b | 0 ...ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 | 0 .../ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 | 0 .../ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de | 0 .../ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d | 0 .../ppr_pushdown-7-14570f946e75924d7926c809485951d1 | 0 .../ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c | 0 .../ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 | 0 ...ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b | 0 ...ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 | 0 ...pr_pushdown2-10-ab3e46183487096745d7d90e3020e94c | 0 ...pr_pushdown2-11-680316eba447eb4649530fdc1c37d95b | 0 ...pr_pushdown2-12-720582e599a974ee9ca46b653678a14a | 0 ...pr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 | 0 ...pr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 | 0 ...ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 | 0 ...pr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb | 0 ...pr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe | 0 ...pr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 | 0 ...pr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 | 0 ...ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e | 0 ...pr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 | 0 ...ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c | 0 ...ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 | 0 ...ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 | 0 ...ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 | 0 ...ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 | 0 ...ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 | 0 ...ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 | 0 ...ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a | 0 ...ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 | 0 ...ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 | 0 ...ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d | 0 ...ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb | 0 ...ppr_pushdown3-5-9b129023196c29071a92e91471872593 | 0 ...ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f | 0 .../print_header-0-860e298a0b70e7a531431e9386ddc0e7 | 0 .../print_header-1-8540676fc16ac91f3629c40f393a890a | 0 .../print_header-2-5cff10d4b561206e7e0b2e81d862ff93 | 0 .../print_header-3-e86d559aeb84a4cc017a103182c22bfb | 0 .../progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 | 0 .../progress_1-1-e93fbb4cb074376a645dc22075c0eab6 | 0 .../progress_1-2-70ba81c09588aa586e62ebaee2af685f | 0 .../progress_1-3-43d286eebddaee26cf95f26e62a75fe4 | 0 .../protectmode-0-44765a9e389cd7be27e038d257ad2c61 | 0 .../protectmode-1-377270bf028b6a5e8a5a1679421aece6 | 0 .../protectmode-10-b993dfdb8636411e4e9182a5a1931362 | 0 .../protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 | 0 .../protectmode-12-d35a2c6ec13dcba7f4856522d2c781a | 0 .../protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c | 0 .../protectmode-14-f817d21ee3a7646bf107c87bb264586b | 0 .../protectmode-15-c51a200a345ceca9838507176a4a6dd6 | 0 .../protectmode-16-151259dfc1cea42392a7511991c58a5b | 0 .../protectmode-17-6fa385b2d2c6fee674dfff38f7338080 | 0 .../protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 | 0 .../protectmode-19-b6e156f104768706aa587b762a9d4d18 | 0 .../protectmode-2-f450722c47e95d5825d0cb920c212aea | 0 .../protectmode-20-719ec12056901bb0330fec1f05d4a94b | 0 .../protectmode-21-a31442a7d6c6950c137856ab861d622d | 0 .../protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 | 0 .../protectmode-23-a31442a7d6c6950c137856ab861d622d | 0 .../protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 | 0 .../protectmode-25-9095794bb72e75a50f84c7cc79b14df6 | 0 .../protectmode-26-a31442a7d6c6950c137856ab861d622d | 0 .../protectmode-27-af002c5db76382ae7619968eaff3cc91 | 0 .../protectmode-28-a31442a7d6c6950c137856ab861d622d | 0 .../protectmode-29-6fa385b2d2c6fee674dfff38f7338080 | 0 .../protectmode-3-4038060ccc6df535736b4b3cdf722cc | 0 .../protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 | 0 .../protectmode-31-97f50f7a49ee261db1888e93cf696c27 | 0 .../protectmode-32-b6e156f104768706aa587b762a9d4d18 | 0 .../protectmode-33-151259dfc1cea42392a7511991c58a5b | 0 .../protectmode-34-520cc99451f889b26e12308b956f41e2 | 0 .../protectmode-35-67854bfbea7afeca2987ced73b68e671 | 0 .../protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 | 0 .../protectmode-37-8c07fbbf528fd9180db5a52474f6683e | 0 .../protectmode-38-da4a5ad43dbf3df8e31d23979af46423 | 0 .../protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 | 0 .../protectmode-4-b993dfdb8636411e4e9182a5a1931362 | 0 .../protectmode-40-39024332d020339bd9396c9b15403a0c | 0 .../protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 | 0 .../protectmode-42-3a8863ad6fac90ba48555c1042c92ebb | 0 .../protectmode-43-377270bf028b6a5e8a5a1679421aece6 | 0 .../protectmode-5-2af90060e2a14790441e0716e3372e7c | 0 .../protectmode-6-e2d1fc9c94e0d3597f393aec99850510 | 0 .../protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 | 0 .../protectmode-8-e2d1fc9c94e0d3597f393aec99850510 | 0 .../protectmode-9-4038060ccc6df535736b4b3cdf722cc | 0 .../push_or-0-82eac35d076df1ee8020d57832f0f895 | 0 .../push_or-1-4c03e0f7fa028897c0caee62167eb68e | 0 .../push_or-2-62abd65e444981a96f7cf3dab3e013f1 | 0 .../push_or-3-34a713b9523589f1eb3205037d56a5b4 | 0 .../push_or-4-22888cdef1b94bde983541c1f3d510d9 | 0 .../push_or-5-c94def4b18b9c8c00e7a93eb19ec694a | 0 ...ery_with_semi-0-3731ce715b60549c17b7993927d48436 | 0 ...ery_with_semi-1-3f53ec3b276b32cf81729433e47010cb | 0 ...ery_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 | 0 .../quote1-0-54ea0f3c04b185472a38a744d8003c13 | 0 .../quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 | 0 .../quote1-2-14d4949bc9215a1004c945885ef10de4 | 0 .../quote1-3-f36bdb38d0c1b79c6113fa0795d14533 | 0 .../quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 | 0 .../quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 | 0 .../quote2-1-34f3c423b2fb1f0b11457f45a60042b9 | 0 ...ed alias.attr-0-97b3c408090f758257e4bd20597a525e | 0 ...titionpruner1-0-a7e4414330751eb8ad486bb11643f64d | 0 ...titionpruner1-1-11cdebc422d7a0e7b257279ac9524321 | 0 ...titionpruner2-0-b9598847d77e2c425423f51d755380e8 | 0 ...titionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 | 0 ...titionpruner2-2-db276de57ad86e8880037336886cd557 | 0 ...titionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 | 0 ...cfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e | 0 ...cfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 | 0 ...cfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 | 0 ...cfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a | 0 ...file_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 | 0 ...file_columnar-1-5d636fc4682aee89328b7c7be17e1013 | 0 ...file_columnar-2-b2f56374f2ea8a967b38e77b57138d4b | 0 ...cfile_columnar-3-4653c703effa4cc65743128fa3c65ff | 0 ...azydecompress-0-82f12a09df66752b05e89d3d12ba1e32 | 0 ...azydecompress-1-de09b3ac5ba6a77653eb361496863085 | 0 ...zydecompress-10-fb011a8ee13a82a873039bd28353205b | 0 ...zydecompress-11-3708198aac609695b22e19e89306034c | 0 ...azydecompress-12-3a4c24fd561f459025264baa3fb6d87 | 0 ...azydecompress-2-bc4929594297fa2ee82b8ceff9118e46 | 0 ...azydecompress-3-18a2fa22b8cef720cb30d75353b3f276 | 0 ...azydecompress-4-fb011a8ee13a82a873039bd28353205b | 0 ...azydecompress-5-68975193b30cb34102b380e647d8d5f4 | 0 ...azydecompress-6-ea607fbed28d20e5726f4501285d698d | 0 ...azydecompress-7-de09b3ac5ba6a77653eb361496863085 | 0 ...azydecompress-8-bc4929594297fa2ee82b8ceff9118e46 | 0 ...azydecompress-9-18a2fa22b8cef720cb30d75353b3f276 | 0 ...rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 | 0 ...rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 | 0 ...cfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 | 0 ...cfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 | 0 ...cfile_merge1-12-150cb190dc2343a747ea89298eb6352c | 0 ...cfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 | 0 ...rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 | 0 ...cfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 | 0 ...cfile_merge1-16-c198c437f48c3844d8d0ace881b3107e | 0 ...rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 | 0 ...rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee | 0 ...rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 | 0 ...rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e | 0 ...rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 | 0 ...rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 | 0 ...rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f | 0 ...rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 | 0 ...rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 | 0 ...rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 | 0 ...cfile_merge2-10-5b4fb8038f522877101a4e429f082f11 | 0 ...rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 | 0 ...rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb | 0 ...rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 | 0 ...rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 | 0 ...rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a | 0 ...rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 | 0 ...rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 | 0 ...rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 | 0 ...rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 | 0 ...rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 | 0 ...cfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 | 0 ...cfile_merge3-11-7674266b00c52a2b6755910ea0793b69 | 0 ...cfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f | 0 ...cfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 | 0 ...cfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b | 0 ...rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb | 0 ...rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 | 0 ...rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b | 0 ...rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 | 0 ...rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b | 0 ...rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 | 0 .../rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 | 0 ...rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 | 0 ...rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 | 0 ...rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 | 0 ...cfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 | 0 ...cfile_merge4-11-7674266b00c52a2b6755910ea0793b69 | 0 ...cfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f | 0 ...cfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 | 0 ...cfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b | 0 ...rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb | 0 ...rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 | 0 ...rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b | 0 ...rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 | 0 ...rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 | 0 ...rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 | 0 .../rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 | 0 ...rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 | 0 ...le_null_value-0-69edd47d7cca001865102e3fad925613 | 0 ...ile_null_value-1-38ac0f806892e3b25b0038c5f227c8d | 0 ...le_null_value-2-e721b8138774fdefca7171d1051841ee | 0 ...le_null_value-3-3c82d1a57cf295901c6b931278a3760a | 0 ...le_null_value-4-8e839adf91e01e3230e15e9a936c9126 | 0 ...ile_null_value-5-d57d9c8882edbb3c6116d89516e4aed | 0 ...ile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 | 0 ...tecorruptions-0-644898731e5228e863236f388ba1aa07 | 0 ...tecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b | 0 ...tecorruptions-2-4a9a9175758ef576804c8b7309b019e8 | 0 ...tecorruptions-3-ea607fbed28d20e5726f4501285d698d | 0 ...tecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 | 0 ...tecorruptions-5-6a3af12e36cec853c876a2cbae61c23a | 0 ...tecorruptions-6-718032defb864225dd664b1719f3b590 | 0 .../rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 | 0 .../rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b | 0 .../rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 | 0 ...recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 | 0 ...e_deduplicate-0-43d53504df013e6b35f81811138a167a | 0 ...e_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ..._deduplicate-10-44d72569675c48e482ff53f45c7526e3 | 0 ..._deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 | 0 ..._deduplicate-12-d6f20dc500809305e7454db01e24ce26 | 0 ..._deduplicate-13-1583df07fff7a2af0a9cc6a681446036 | 0 ...ce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 | 0 ...e_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe | 0 ...e_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 | 0 ...e_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d | 0 ...e_deduplicate-6-890c5edd947ca276254ef4bd2813b296 | 0 ...e_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 | 0 ...e_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 | 0 ...e_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 | 0 ...e_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 | 0 ...e_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 | 0 ...e_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...e_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 | 0 ...e_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 | 0 ..._exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 | 0 ..._exclude_join-1-24ca942f094b14b92086305cc125e833 | 0 ..._exclude_join-2-f420affa96a8f13a91f6082805cf72a0 | 0 ...cate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 | 0 ...cate_extended-1-d0a93f40892e3894460553b443c77428 | 0 ...ate_extended-10-5a0cbc5744352a117afb34d1754496fa | 0 ...ate_extended-11-9a4a8559680a45aad0a54144fb200f5e | 0 ...ate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 | 0 ...ate_extended-13-ca02f80b104ee3dff445518501d0ee3e | 0 ...ate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c | 0 ...ate_extended-15-dbc2b906b6cb99a138175eb4764da6bd | 0 ...ate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb | 0 ...ate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f | 0 ...ate_extended-18-4435a485aa7e3a72b789418e063d4cb8 | 0 ...ate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...cate_extended-2-dbcec232623048c7748b708123e18bf0 | 0 ...ate_extended-20-29dc99f23850e9f19368926d621a3029 | 0 ...ate_extended-21-722d6027daf02b2e40f121a4fec362d2 | 0 ...ate_extended-22-f3a586e75df55df355982eac0d4d4a63 | 0 ...ate_extended-23-241faeae86e3e43e3ed8e35fb24e027d | 0 ...ate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed | 0 ...ate_extended-25-bce36b45ce0c67af052bcf0487a53052 | 0 ...ate_extended-26-93434017959c1be56608adff9309f02a | 0 ...ate_extended-27-5a0cbc5744352a117afb34d1754496fa | 0 ...ate_extended-28-9a4a8559680a45aad0a54144fb200f5e | 0 ...ate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 | 0 ...cate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 | 0 ...ate_extended-30-ca02f80b104ee3dff445518501d0ee3e | 0 ...ate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c | 0 ...ate_extended-32-dbc2b906b6cb99a138175eb4764da6bd | 0 ...ate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb | 0 ...ate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f | 0 ...ate_extended-35-4435a485aa7e3a72b789418e063d4cb8 | 0 ...cate_extended-4-722d6027daf02b2e40f121a4fec362d2 | 0 ...cate_extended-5-f3a586e75df55df355982eac0d4d4a63 | 0 ...cate_extended-6-8801137a557caa5a75cfacd7aaf56bad | 0 ...cate_extended-7-7e350e31bce459f89b322b3356806174 | 0 ...cate_extended-8-bce36b45ce0c67af052bcf0487a53052 | 0 ...cate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 | 0 ...ucesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb | 0 ...ucesink_dedup-1-dda16565b98926fc3587de937b9401c7 | 0 ...ucesink_dedup-2-971f12c90619b02484c8e7db1b32f050 | 0 .../regex_col-0-ac78bd83c9aa538c2827598dd6007a69 | 0 .../regex_col-1-42751bfc3f1e83e7a014db9272d597db | 0 .../regex_col-2-21564f64cdfd46098e1254380490701 | 0 .../regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 | 0 .../regex_col-4-daf9d3ca22b243870a138ba90d1593c4 | 0 ...remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 | 0 ...remote_script-1-e168f471980470d93b790702a70238fa | 0 ...remote_script-2-a19a19272149c732977c37e043910505 | 0 .../remote_script-3-4eb54a664e549614d56ca088c8867d | 0 ...rename_column-0-f7eb4bd6f226be0c13117294be250271 | 0 ...rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-10-7ef160935cece55338bd4d52277b0203 | 0 ...ename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-12-379d54e3aa66daacff23c75007dfa008 | 0 ...ename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-14-25bfcf66698b12f82903f72f13fea4e6 | 0 ...ename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-16-d032f4795c1186255acea241387adf93 | 0 ...ename_column-17-9c36cac1372650b703400c60dd29042c | 0 ...ename_column-18-fe4463a19f61099983f50bb51cfcd335 | 0 ...ename_column-19-70b42434913b9d2eb17cd216c4f8039f | 0 ...rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a | 0 ...ename_column-20-f7eb4bd6f226be0c13117294be250271 | 0 ...ename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a | 0 ...ename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-24-e4bf0dd372b886b2afcca5b2dc089409 | 0 ...ename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-26-89761e1c7afe3a5b9858f287cb808ccd | 0 ...ename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 | 0 ...ename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-30-7ef160935cece55338bd4d52277b0203 | 0 ...ename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-32-379d54e3aa66daacff23c75007dfa008 | 0 ...ename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-34-25bfcf66698b12f82903f72f13fea4e6 | 0 ...ename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...ename_column-36-d032f4795c1186255acea241387adf93 | 0 ...ename_column-37-9c36cac1372650b703400c60dd29042c | 0 ...rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 | 0 ...rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...rename_column-6-89761e1c7afe3a5b9858f287cb808ccd | 0 ...rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 | 0 ...rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b | 0 ...tion_location-0-5c73d46fb91e9d4b3dc916622df09290 | 0 ...uter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 | 0 ...uter_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 | 0 ...uter_join_ppr-2-49b8b038ed8f5513405071c532967c47 | 0 ...uter_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc | 0 ...uter_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 | 0 ...uter_join_ppr-5-89b4fc284984f51c12f15017707ad312 | 0 ...uter_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f | 0 ...uter_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 | 0 ...uter_join_ppr-8-b07ad21f47a74162f438abf579675f8e | 0 .../sample2-0-13ab74a58da514fe01dbeda0c3e79883 | 0 .../sample2-1-a1d8184eab25b242a961533cc016efd1 | 0 ...ript_env_var1-0-16015162957e1d8e0ef586b44b276f64 | 0 ...ript_env_var2-0-e5c4893b2ff412f9df0632328d404cef | 0 ...ript_env_var2-1-81cb3e18ab89c533a253defff35e90f0 | 0 .../script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 | 0 .../script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 | 0 .../script_pipe-2-3bf368261963ee10883e97c7746796b5 | 0 .../script_pipe-3-afe5db164ccf986c5badd0655e009ea1 | 0 .../script_pipe-4-7fe60c2fcca928a497252d99436b513f | 0 .../scriptfile1-0-43d53504df013e6b35f81811138a167a | 0 .../scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 | 0 .../scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 | 0 ...ct_as_omitted-0-39bd00e50d63ee4ff10427838591730a | 0 ...ct_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 | 0 ...t_unquote_and-0-7299c6986c06fc4a50cb543701527785 | 0 ...t_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 | 0 ...t_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 | 0 ...t_unquote_and-3-683007138a712792041ef3c8b84e914e | 0 ...t_unquote_and-4-8417a7b93132e20d468b7948bf46915a | 0 ...t_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 | 0 ...t_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 | 0 ...t_unquote_not-0-7299c6986c06fc4a50cb543701527785 | 0 ...t_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 | 0 ...t_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 | 0 ...t_unquote_not-3-683007138a712792041ef3c8b84e914e | 0 ...t_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 | 0 ...t_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde | 0 ...t_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 | 0 ...ct_unquote_or-0-7299c6986c06fc4a50cb543701527785 | 0 ...ct_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 | 0 ...ct_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 | 0 ...ct_unquote_or-3-683007138a712792041ef3c8b84e914e | 0 ...ct_unquote_or-4-954f604d17e36bbc08c04a0571d33045 | 0 ...ct_unquote_or-5-55cd874f705673f9de6ec8e3643c760f | 0 ...ct_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 | 0 ...ported_schema-0-765ac345199b944a6136273becf29469 | 0 ...ported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 | 0 ...ported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d | 0 ...ported_schema-3-738e1d72a19c3db37ded87ca2fb148fa | 0 ..._variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d | 0 ..._variable_sub-1-e504c8259e45911765541f605c9a46f1 | 0 ...variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 | 0 ..._variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 | 0 ..._variable_sub-3-266170978f88a14c20c3944bfb55f5c7 | 0 ..._variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 | 0 ..._variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd | 0 ..._variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a | 0 ..._variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b | 0 ..._variable_sub-8-f321516e93eb0206453465a9b85cf67d | 0 ..._variable_sub-9-fbb54d457caeaafce723856429bbc0b2 | 0 ...e_func_quotes-0-65fee14fcf58502241f0772b21096780 | 0 ...e_func_quotes-1-26b98b2901556449d5431d731aaa642d | 0 ...e_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 | 0 ...be_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 | 0 ...how_functions-0-45a7762c39f1b0f26f076220e2764043 | 0 ...how_functions-1-4a6f611305f58bdbafb2fd89ec62d797 | 0 ...how_functions-2-97cbada21ad9efda7ce9de5891deca7c | 0 ...how_functions-3-86945c60aed23626c43b507ee4ee6049 | 0 ...how_functions-4-4deaa213aff83575bbaf859f79bfdd48 | 0 ...how_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 | 0 ...es_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 | 0 ...es_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 | 0 ...s_edge_cases-10-d759a63f08c878456c3401626f253ff5 | 0 ...s_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e | 0 ...s_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b | 0 ...s_edge_cases-13-a9224a7a0012e407da67041bf680d490 | 0 ...s_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e | 0 ...s_edge_cases-15-b032f4869c43d1278a890328d0591d5d | 0 ...s_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f | 0 ...s_edge_cases-17-c93fd07893f47b712165725c78d95555 | 0 ...s_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 | 0 ...s_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 | 0 ...es_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 | 0 ...es_edge_cases-3-aa5935155586821fb35e17156c8d8460 | 0 ...es_edge_cases-4-6eb587e2751942de625c9229872ca0dc | 0 ...es_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 | 0 ...es_edge_cases-6-ae97a64481efe733a19007ed400925bc | 0 ...es_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 | 0 ...es_edge_cases-8-16d39297488db165145e1546c4cb222c | 0 ...es_edge_cases-9-fba02256147a1a753d40f56825449471 | 0 ...ndexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec | 0 ...ndexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 | 0 ...dexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec | 0 ...ndexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 | 0 ...ndexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 | 0 ...ndexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 | 0 ...ndexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 | 0 ...ndexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 | 0 ...ndexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 | 0 ...ndexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e | 0 ...ndexes_syntax-9-f8385127c6405a2c9e48b5988184b515 | 0 ...ow_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a | 0 ...ow_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 | 0 ...how_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca | 0 ...ow_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 | 0 .../show_tables-0-679cc07f8475a20b49927a5bbbd3d702 | 0 .../show_tables-1-ac1c8cca812353544d3f7dead5d033ce | 0 .../show_tables-10-643b24446d74450c0f83144b1d0ec433 | 0 .../show_tables-11-3f9a7f993510123059493826470f78f7 | 0 .../show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c | 0 .../show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 | 0 .../show_tables-14-26ca9b5509544ebac182d8aa4264ff1c | 0 .../show_tables-15-72a95694f749cb3d5760a69083e9cafe | 0 .../show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 | 0 .../show_tables-17-49777c49d2627373ed5e459c4848c9ab | 0 .../show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 | 0 .../show_tables-19-695a68c82308540eba1d0a04e032cf39 | 0 .../show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 | 0 .../show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 | 0 .../show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 | 0 .../show_tables-22-274454ebeb7f98690a3c152617a0e391 | 0 .../show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c | 0 .../show_tables-4-743d585ec3da5fbb927581cd0683ae35 | 0 .../show_tables-5-c685b358b604bf3ef980a78d9178d87d | 0 .../show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 | 0 .../show_tables-7-a62fc229d241303bffb29b34ad125f8c | 0 .../show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 | 0 .../show_tables-9-64c9bf0618541518f2ba30ec24a94423 | 0 ...w_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa | 0 ...w_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 | 0 ...w_tablestatus-2-ecddce523f2af516700677a051581330 | 0 ...w_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d | 0 ...w_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 | 0 ...ow_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 | 0 ...w_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 | 0 ...w_tablestatus-7-f7b9148c16045269344c5d74fb8a449c | 0 ...w_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 | 0 ...w_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 | 0 .../showparts-0-593619bb962b318b82896658deaea1f1 | 0 .../showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a | 0 ...nion_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...nion_remove_1-1-114600d46ae86edcb66a500b4cac657d | 0 ...ion_remove_1-10-fa00cf008a039908eec64ad5dd415c5b | 0 ...ion_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 | 0 ...ion_remove_1-12-dd683e148baed6b27642eebacaa87a4f | 0 ...ion_remove_1-13-43d53504df013e6b35f81811138a167a | 0 ...nion_remove_1-2-cafed8ca348b243372b9114910be1557 | 0 ...union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...nion_remove_1-5-dc129f70e75cd575ce8c0de288884523 | 0 ...nion_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab | 0 ...nion_remove_1-7-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_1-8-daf10744f465e055b35809a528135370 | 0 ...nion_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...nion_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...nion_remove_2-1-114600d46ae86edcb66a500b4cac657d | 0 ...ion_remove_2-10-bebf0a312f3110d0b518153543030f06 | 0 ...ion_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 | 0 ...ion_remove_2-12-fa53198d9eecb9d274b09d4351b9274e | 0 ...ion_remove_2-13-3fda17e4414d191f837631438a19e700 | 0 ...ion_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 | 0 ...ion_remove_2-15-43d53504df013e6b35f81811138a167a | 0 ...nion_remove_2-2-cafed8ca348b243372b9114910be1557 | 0 ...union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...nion_remove_2-5-dc129f70e75cd575ce8c0de288884523 | 0 ...nion_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab | 0 ...nion_remove_2-7-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_2-8-c64266431d312784ebc7b9ca07ab5188 | 0 ...nion_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 .../skewjoinopt1-0-cafed8ca348b243372b9114910be1557 | 0 .../skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 .../skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d | 0 .../skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 .../skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 .../skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b | 0 .../skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 | 0 .../skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 | 0 ...skewjoinopt10-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 | 0 ...skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a | 0 ...skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 | 0 ...skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 | 0 ...skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 | 0 ...skewjoinopt11-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 | 0 ...skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...skewjoinopt11-5-744a018c78bae6e09853dd202981e850 | 0 ...skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 | 0 ...skewjoinopt12-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 | 0 ...skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd | 0 ...skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 | 0 ...skewjoinopt13-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...kewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 | 0 ...skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 | 0 ...skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...skewjoinopt13-5-744a018c78bae6e09853dd202981e850 | 0 ...skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 | 0 ...skewjoinopt13-8-3fda17e4414d191f837631438a19e700 | 0 ...skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d | 0 ...skewjoinopt14-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...skewjoinopt14-5-744a018c78bae6e09853dd202981e850 | 0 ...skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 | 0 ...skewjoinopt14-8-3fda17e4414d191f837631438a19e700 | 0 ...skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 | 0 ...skewjoinopt15-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...kewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 | 0 ...kewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 | 0 ...skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 | 0 ...skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 | 0 ...skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 | 0 ...skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d | 0 .../skewjoinopt15-7-ba43a86694107dd4cb754d676935408 | 0 ...skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 | 0 ...skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e | 0 ...skewjoinopt16-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b | 0 ...skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 | 0 ...skewjoinopt17-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 | 0 ...skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 | 0 ...skewjoinopt18-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...kewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa | 0 ...skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 | 0 ...skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 | 0 ...skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 | 0 ...skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d | 0 ...skewjoinopt18-7-41ca600110e24166325d9426d974fff7 | 0 ...skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 | 0 ...skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 | 0 ...skewjoinopt19-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e | 0 ...skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...skewjoinopt19-5-744a018c78bae6e09853dd202981e850 | 0 ...skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 | 0 .../skewjoinopt2-0-cafed8ca348b243372b9114910be1557 | 0 .../skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 .../skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d | 0 .../skewjoinopt2-3-52247b4dd98092bf829254e17424657d | 0 .../skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 .../skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 | 0 .../skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 | 0 .../skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 | 0 ...skewjoinopt20-0-cafed8ca348b243372b9114910be1557 | 0 ...skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d | 0 ...skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 | 0 ...skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...skewjoinopt20-5-744a018c78bae6e09853dd202981e850 | 0 ...skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...skewjoinopt20-7-e209254ae404366e6adca673d666aecb | 0 .../skewjoinopt3-0-cafed8ca348b243372b9114910be1557 | 0 .../skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 .../skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d | 0 .../skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 | 0 .../skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 .../skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 | 0 .../skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 | 0 .../skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f | 0 .../skewjoinopt4-0-cafed8ca348b243372b9114910be1557 | 0 .../skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 .../skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d | 0 .../skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 .../skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 .../skewjoinopt4-5-744a018c78bae6e09853dd202981e850 | 0 .../skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 | 0 .../skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 | 0 .../skewjoinopt5-0-cafed8ca348b243372b9114910be1557 | 0 .../skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 .../skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d | 0 .../skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b | 0 .../skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 .../skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b | 0 .../skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 | 0 .../skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d | 0 .../skewjoinopt6-0-cafed8ca348b243372b9114910be1557 | 0 .../skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 .../skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d | 0 .../skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 | 0 .../skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 .../skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd | 0 .../skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 | 0 .../skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 | 0 .../skewjoinopt7-0-cafed8ca348b243372b9114910be1557 | 0 .../skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 .../skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d | 0 .../skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 | 0 .../skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 .../skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 | 0 .../skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 | 0 .../skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e | 0 .../skewjoinopt7-8-3fda17e4414d191f837631438a19e700 | 0 .../skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c | 0 .../skewjoinopt8-0-cafed8ca348b243372b9114910be1557 | 0 .../skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 .../skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d | 0 .../skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 | 0 .../skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 .../skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 | 0 .../skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 | 0 .../skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e | 0 .../skewjoinopt8-8-3fda17e4414d191f837631438a19e700 | 0 .../skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd | 0 .../skewjoinopt9-0-cafed8ca348b243372b9114910be1557 | 0 .../skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 | 0 ...skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 | 0 .../skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d | 0 .../skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 .../skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 .../skewjoinopt9-5-744a018c78bae6e09853dd202981e850 | 0 .../skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 | 0 .../skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 | 0 .../skewjoinopt9-8-446c5e33062b109341add64a9860207d | 0 .../skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 | 0 ...all.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 | 0 ...smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec | 0 ...mb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 | 0 ...smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed | 0 ...mb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa | 0 ...smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 | 0 ...mb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 | 0 ...mb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c | 0 ...mb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 | 0 ...mb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 | 0 ...mb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 | 0 ...mb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 | 0 ...smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f | 0 ...mb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 | 0 ...mb_mapjoin_1-21-bce103f1a686915c28581717387cfeec | 0 ...mb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b | 0 ...mb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd | 0 ...mb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae | 0 ...smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de | 0 .../smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f | 0 ...smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 | 0 ...smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f | 0 ...smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea | 0 ...mb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 | 0 ...mb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 | 0 ...b_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 | 0 ...mb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 | 0 ...mb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 | 0 ...mb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 | 0 ...mb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 | 0 ...mb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 | 0 ...mb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 | 0 ...mb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f | 0 ...mb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa | 0 ...b_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 | 0 ...b_mapjoin_11-12-c05c09243793df14546e8577ee369d58 | 0 ...b_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 | 0 ...b_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 | 0 ...b_mapjoin_11-15-b62714cb184523454314d19949dba9f4 | 0 ...b_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af | 0 ...mb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f | 0 ...mb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...mb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 | 0 ...smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab | 0 ...mb_mapjoin_11-8-beae7266b997c97798631b9dc558534f | 0 ...mb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 | 0 ...mb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 | 0 ...b_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 | 0 ...b_mapjoin_12-12-c05c09243793df14546e8577ee369d58 | 0 ...mb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb | 0 ...b_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f | 0 ...b_mapjoin_12-15-42b623410c408e09153a773db91c0334 | 0 ...b_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af | 0 ...b_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f | 0 ...b_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f | 0 ...mb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f | 0 ...b_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b | 0 ...b_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e | 0 ...b_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e | 0 ...mb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...mb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 | 0 ...smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab | 0 ...mb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 | 0 ...mb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 | 0 ...mb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 | 0 ...b_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 | 0 ...b_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 | 0 ...b_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee | 0 ...b_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 | 0 ...b_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 | 0 ...b_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 | 0 ...mb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f | 0 ...mb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...mb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 | 0 ...smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab | 0 ...mb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 | 0 ...mb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 | 0 ...mb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...mb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...b_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd | 0 ...b_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d | 0 ...b_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 | 0 ...b_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a | 0 ...b_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d | 0 ...b_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 | 0 ...b_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 | 0 ...b_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee | 0 ...b_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 | 0 ...b_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 | 0 ...smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a | 0 ...b_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 | 0 ...mb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b | 0 ...b_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 | 0 ...b_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 | 0 ...mb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e | 0 ...b_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b | 0 ...b_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 | 0 ...b_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 | 0 ...b_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b | 0 ...mb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da | 0 ...b_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb | 0 ...b_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 | 0 ...b_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 | 0 ...b_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 | 0 ...mb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 | 0 ...mb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f | 0 ...mb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f | 0 ...mb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 | 0 ...mb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f | 0 ...mb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b | 0 ...b_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 | 0 ...b_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a | 0 ...b_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 | 0 ...b_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 | 0 ...b_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 | 0 ...b_mapjoin_15-16-98537b81d6e139521541753c3c448404 | 0 ...b_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 | 0 ...b_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c | 0 ...b_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f | 0 ...mb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f | 0 ...b_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 | 0 ...b_mapjoin_15-21-af3880637379684acd440830c2361f6e | 0 ...b_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 | 0 ...b_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 | 0 ...b_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 | 0 ...b_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 | 0 ...mb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...mb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 | 0 ...smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab | 0 ...mb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb | 0 ...mb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 | 0 ...mb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b | 0 ...b_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c | 0 ...b_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf | 0 ...mb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f | 0 ...mb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...mb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 | 0 ...smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab | 0 ...mb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 | 0 ...mb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e | 0 ...mb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 | 0 ...mb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 | 0 ...b_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 | 0 ...b_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 | 0 ...b_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 | 0 ...b_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 | 0 ...b_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 | 0 ...b_mapjoin_17-17-bfc16d69eac4464e094e416842550590 | 0 ...b_mapjoin_17-18-cf2846a07aab193ac997eae616373367 | 0 ...b_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 | 0 ...mb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f | 0 ...b_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c | 0 ...b_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f | 0 ...b_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 | 0 ...b_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e | 0 ...b_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 | 0 ...b_mapjoin_17-25-f066907fca3448b27aab623d05258a9a | 0 ...b_mapjoin_17-26-24ca942f094b14b92086305cc125e833 | 0 ...b_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb | 0 ...b_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 | 0 ...b_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 | 0 ...mb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...b_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e | 0 ...b_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc | 0 ...b_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff | 0 ...mb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 | 0 ...smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab | 0 ...mb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 | 0 ...mb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e | 0 ...mb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e | 0 ...b_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f | 0 ...b_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 | 0 ...b_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 | 0 ...b_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 | 0 ...b_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 | 0 ...b_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 | 0 ...b_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 0 ...b_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d | 0 ...b_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 | 0 ...mb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...b_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a | 0 ...b_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 | 0 ...b_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 | 0 ...b_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 | 0 ...b_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c | 0 ...mb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 | 0 ...b_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f | 0 ...b_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 | 0 ...b_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab | 0 ...b_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 | 0 ...mb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 | 0 ...b_mapjoin_18-30-76e467313085467a3aa929b3665f9863 | 0 ...b_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 | 0 ...b_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c | 0 ...b_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe | 0 ...b_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 | 0 ...b_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 | 0 ...smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab | 0 ...mb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c | 0 ...mb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 | 0 ...mb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...mb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e | 0 ...b_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f | 0 ...b_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 | 0 ...b_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 | 0 ...b_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 | 0 ...b_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab | 0 ...b_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc | 0 ...b_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec | 0 ...b_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e | 0 ...b_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 0 ...mb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...b_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b | 0 ...b_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 | 0 ...b_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f | 0 ...b_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb | 0 ...b_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 | 0 ...b_mapjoin_19-25-e621350131d50867015e75677cca031f | 0 ...mb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 | 0 ...smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab | 0 ...mb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d | 0 ...mb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 | 0 ...mb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec | 0 ...mb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac | 0 ...smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 | 0 ...mb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 | 0 ...mb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d | 0 ...mb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 | 0 ...mb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 | 0 ...mb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca | 0 ...mb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 | 0 ...mb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 | 0 ...mb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 | 0 ...smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f | 0 ...mb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b | 0 ...mb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 | 0 ...smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 | 0 ...mb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a | 0 ...mb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 | 0 ...smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de | 0 .../smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f | 0 ...smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 | 0 ...smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f | 0 ...smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 | 0 ...mb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 | 0 ...b_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 | 0 ...b_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 0 ...b_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d | 0 ...b_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 | 0 ...b_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a | 0 ...b_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a | 0 ...b_mapjoin_20-17-e26f212ca34d303036299ba709f65522 | 0 ...b_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e | 0 ...b_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 | 0 ...mb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...b_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 | 0 ...b_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 | 0 ...b_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 | 0 ...b_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 | 0 ...mb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 | 0 ...smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab | 0 ...smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 | 0 ...mb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff | 0 ...mb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...mb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e | 0 ...b_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...b_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 | 0 ...b_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf | 0 ...b_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...b_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd | 0 ...mb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a | 0 ...b_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...b_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 | 0 ...mb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a | 0 ...mb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...b_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...b_mapjoin_21-21-ee61731107316feaef2efca38bb43fab | 0 ...b_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 | 0 ...b_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...b_mapjoin_21-24-c600a123838779df6a5babea49732659 | 0 ...b_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 | 0 ...mb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 | 0 ...smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab | 0 ...mb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c | 0 ...mb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 | 0 ...mb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...mb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 | 0 ...b_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc | 0 ...b_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc | 0 ...b_mapjoin_22-12-187c201f61c23833d0d193031926445a | 0 ...b_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 | 0 ...mb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 | 0 ...b_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 | 0 ...b_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe | 0 ...b_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...b_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 | 0 ...b_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e | 0 ...mb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...b_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad | 0 ...b_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc | 0 ...b_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc | 0 ...b_mapjoin_22-23-187c201f61c23833d0d193031926445a | 0 ...b_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 | 0 ...mb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 | 0 ...b_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 | 0 ...mb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 | 0 ...smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 | 0 ...mb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab | 0 ...mb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b | 0 ...mb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb | 0 ...mb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad | 0 ...mb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...mb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 | 0 ...b_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...b_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de | 0 ...mb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...b_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 | 0 ...b_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed | 0 ...mb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef | 0 ...b_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f | 0 ...b_mapjoin_25-17-24ca942f094b14b92086305cc125e833 | 0 ...b_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb | 0 ...b_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 | 0 ...mb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe | 0 ...b_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 | 0 ...b_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d | 0 ...b_mapjoin_25-22-d0a93f40892e3894460553b443c77428 | 0 ...mb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 | 0 ...b_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 | 0 ...b_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 | 0 ...b_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 | 0 ...mb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff | 0 ...mb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 | 0 ...mb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 | 0 ...mb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 | 0 ...mb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...mb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec | 0 ...mb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f | 0 ...smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec | 0 ...smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 | 0 ...mb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 | 0 ...mb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 | 0 ...mb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f | 0 ...mb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e | 0 ...mb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 | 0 ...mb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a | 0 ...mb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe | 0 ...mb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 | 0 ...mb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 | 0 ...smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f | 0 ...mb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd | 0 ...mb_mapjoin_3-21-da48411b3e139037e171654fc9861755 | 0 ...mb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 | 0 ...mb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 | 0 ...mb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 | 0 ...smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de | 0 .../smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f | 0 ...smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 | 0 ...smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f | 0 ...smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af | 0 ...smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec | 0 ...mb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 | 0 ...mb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 | 0 ...mb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 | 0 ...mb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 | 0 ...mb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d | 0 ...mb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 | 0 ...mb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 | 0 ...mb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 | 0 .../smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 | 0 ...mb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 | 0 ...smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f | 0 ...mb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 | 0 ...mb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 | 0 ...mb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 | 0 ...mb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 | 0 ...mb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 | 0 ...mb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 | 0 ...mb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e | 0 ...mb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef | 0 ...smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 | 0 ...mb_mapjoin_4-29-239bc4053a7791815b587afef03515fb | 0 ...smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...mb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f | 0 ...mb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 | 0 ...mb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 | 0 ...mb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 | 0 ...smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 | 0 ...smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de | 0 .../smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f | 0 ...smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 | 0 ...smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f | 0 ...smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db | 0 ...smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a | 0 ...smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec | 0 ...mb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc | 0 ...mb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 | 0 ...mb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 | 0 ...mb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a | 0 ...mb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 | 0 ...smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 | 0 ...mb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 | 0 ...mb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f | 0 ...mb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc | 0 ...mb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a | 0 ...smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f | 0 ...mb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d | 0 ...mb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 | 0 ...mb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a | 0 ...mb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a | 0 ...mb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b | 0 ...smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 | 0 ...mb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 | 0 ...mb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 | 0 ...mb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 | 0 ...mb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 | 0 ...smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...mb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 | 0 ...mb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 | 0 ...smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc | 0 ...mb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 | 0 ...mb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 | 0 ...smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de | 0 .../smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f | 0 ...smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 | 0 ...smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f | 0 ...smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df | 0 ...smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...mb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 | 0 ...mb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f | 0 ...mb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 | 0 ...mb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 | 0 ...mb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca | 0 ...mb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 | 0 ...mb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e | 0 ...mb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a | 0 ...mb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a | 0 ...mb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 | 0 .../smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 | 0 ...mb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca | 0 ...mb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 | 0 ...mb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e | 0 ...mb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a | 0 ...mb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 | 0 ...mb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 | 0 ...smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 | 0 ...mb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 | 0 ...mb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 | 0 ...smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 | 0 ...mb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 | 0 ...smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa | 0 ...smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c | 0 ...smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 | 0 ...smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 | 0 ...smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 | 0 ...smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f | 0 ...smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 | 0 ...mb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 | 0 ...mb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 | 0 ...mb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f | 0 ...mb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 | 0 ...mb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 | 0 ...mb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 | 0 ...mb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd | 0 ...smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a | 0 ...mb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a | 0 .../smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 | 0 ...mb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e | 0 ...mb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a | 0 ...smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff | 0 ...smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa | 0 ...smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de | 0 ...smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c | 0 ...smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 | 0 ...smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 | 0 ...smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb | 0 ...smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f | 0 ...smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a | 0 ...smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 0 ...mb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 | 0 ...mb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f | 0 ...mb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 | 0 ...mb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f | 0 ...smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 | 0 ...mb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b | 0 ...mb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 | 0 ...mb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 | 0 ...smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 | 0 ...mb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b | 0 ...smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 | 0 ...mb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 | 0 ...mb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 | 0 ...smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 | 0 ...mb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b | 0 ...mb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c | 0 ...mb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 | 0 ...smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 | 0 ...mb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b | 0 ...mb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c | 0 ...mb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 | 0 .../smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 | 0 ...mb_mapjoin_8-30-19b2b8c39155340929c605494826d30e | 0 ...mb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a | 0 ...mb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c | 0 ...mb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 | 0 ...mb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f | 0 ...mb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a | 0 ...mb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 | 0 ...mb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 | 0 ...mb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 | 0 ...mb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a | 0 ...smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 | 0 ...mb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b | 0 ...mb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c | 0 ...mb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 | 0 ...mb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a | 0 ...mb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 | 0 ...mb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c | 0 ...mb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 | 0 ...mb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a | 0 ...smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 | 0 ...smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a | 0 ...smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 | 0 ...smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a | 0 ...smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 | 0 .../golden/sort-0-eee35bbc888a705ae527625447668032 | 0 .../golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 | 0 ...e_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 | 0 ...e_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c | 0 ..._join_desc_1-10-e4475ba09980280452542a0d595b8217 | 0 ..._join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 | 0 ...e_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 | 0 ...e_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 | 0 ...e_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e | 0 ...e_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 | 0 ...e_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 | 0 ...e_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f | 0 ...e_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 | 0 ...e_join_desc_1-9-b89ea2173180c8ae423d856f943e061f | 0 ...e_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 | 0 ...e_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c | 0 ..._join_desc_2-10-c0cac985e4115bf4014274b21222cac9 | 0 ..._join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd | 0 ...e_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 | 0 ...e_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 | 0 ...e_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 | 0 ...e_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 | 0 ...e_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 | 0 ...e_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f | 0 ...e_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 | 0 ...e_join_desc_2-9-b89ea2173180c8ae423d856f943e061f | 0 ...e_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 | 0 ...e_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c | 0 ..._join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 | 0 ..._join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd | 0 ...e_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 | 0 ...e_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 | 0 ...e_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 | 0 ...e_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 | 0 ...e_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 | 0 ...e_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f | 0 ...e_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 | 0 ...e_join_desc_3-9-b89ea2173180c8ae423d856f943e061f | 0 ...e_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 | 0 ...e_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c | 0 ..._join_desc_4-10-99adddf7dacf8042707a9b36062fe103 | 0 ..._join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd | 0 ...e_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 | 0 ...e_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 | 0 ...e_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 | 0 ...e_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 | 0 ...e_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 | 0 ...e_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f | 0 ...e_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 | 0 ...e_join_desc_4-9-b89ea2173180c8ae423d856f943e061f | 0 ...e_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...e_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 | 0 ..._join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a | 0 ...e_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb | 0 ...e_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d | 0 ...e_join_desc_5-4-f392824f42188aac282046394108ef48 | 0 ...e_join_desc_5-5-85d19fc31c2031491560208803fe63bf | 0 ...e_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 | 0 ...e_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed | 0 ...e_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 | 0 ...ge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 | 0 ...e_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...e_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 | 0 ..._join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 | 0 ..._join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a | 0 ...ge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 | 0 ...e_join_desc_6-3-d651a32232651880ad90428ada99358b | 0 ...e_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d | 0 ...e_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 | 0 ...e_join_desc_6-6-85d19fc31c2031491560208803fe63bf | 0 ...e_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 | 0 ...e_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed | 0 ...e_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 | 0 ...e_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e | 0 ...e_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 | 0 ..._join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 | 0 ..._join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 | 0 ..._join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed | 0 ..._join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 | 0 ..._join_desc_7-14-afdad72caa2e23071817fe124da07763 | 0 ..._join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 | 0 ...ge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 | 0 ...e_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b | 0 ...e_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d | 0 ...e_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 | 0 ...e_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 | 0 ...e_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f | 0 ...e_join_desc_7-8-85d19fc31c2031491560208803fe63bf | 0 ...e_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec | 0 .../stats0-0-418ec894d08c33fd712eb358f579b7a0 | 0 .../stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f | 0 .../stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c | 0 .../stats0-11-82294461be4728b4b191414bf2fb3bd7 | 0 .../stats0-12-31ebc4dcaa13c6dc5937c04f603e699 | 0 .../stats0-13-ca0569fd4200d549843deadf84044649 | 0 .../stats0-14-4f18f4b06db06844920b14e2d19471a9 | 0 .../stats0-15-4a7ed5b7e3deb4106204f8f950357e1c | 0 .../stats0-16-b12e5c70d6d29757471b900b6160fa8a | 0 .../stats0-17-593999fae618b6b38322bc9ae4e0c027 | 0 .../stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 | 0 .../stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed | 0 .../stats0-2-dc129f70e75cd575ce8c0de288884523 | 0 .../stats0-20-5f7439f97275da70f633e135205f2095 | 0 .../stats0-21-6bcc1360b3d777a8b51ae416ff43898a | 0 .../stats0-22-85de3349a12a9fa0536f20ad0377fca1 | 0 .../stats0-23-247568f4b3ce6b97979ca9d28c2ba05c | 0 .../stats0-24-7f38972bcd18c477a027d881182d83f9 | 0 .../stats0-25-98c925a2b2c4de06e270e1b52437a98b | 0 .../stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c | 0 .../stats0-27-82294461be4728b4b191414bf2fb3bd7 | 0 .../stats0-28-31ebc4dcaa13c6dc5937c04f603e699 | 0 .../stats0-29-ca0569fd4200d549843deadf84044649 | 0 .../stats0-3-a572a07cd60fd4607ddd7613db8a64ab | 0 .../stats0-30-4f18f4b06db06844920b14e2d19471a9 | 0 .../stats0-31-4a7ed5b7e3deb4106204f8f950357e1c | 0 .../stats0-4-5f7439f97275da70f633e135205f2095 | 0 .../stats0-5-6bcc1360b3d777a8b51ae416ff43898a | 0 .../stats0-6-85de3349a12a9fa0536f20ad0377fca1 | 0 .../stats0-7-247568f4b3ce6b97979ca9d28c2ba05c | 0 .../stats0-8-7f38972bcd18c477a027d881182d83f9 | 0 .../stats0-9-98c925a2b2c4de06e270e1b52437a98b | 0 .../stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 .../golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 | 0 .../stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e | 0 .../stats2-11-912c785dbcef3251dd1d6b7fc56eae5b | 0 .../stats2-2-a4fb8359a2179ec70777aad6366071b7 | 0 .../stats2-3-16367c381d4b189b3640c92511244bfe | 0 .../stats2-4-dc129f70e75cd575ce8c0de288884523 | 0 .../stats2-5-6717e1026e24a38af19b6bce1951e3d7 | 0 .../stats2-6-45d4fb785fc94d266096fc29a9e90d73 | 0 .../stats2-7-6436db7a7506b560d0d1759db94a76b9 | 0 .../golden/stats2-8-72621dba638b15d244850018e9f64d7 | 0 .../stats2-9-6d93732dc2ca622eb60c171389caee8e | 0 .../stats20-0-418ec894d08c33fd712eb358f579b7a0 | 0 .../stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f | 0 .../stats20-2-4711e55648c25c86bb526ed50b0c3d09 | 0 .../stats20-3-98c925a2b2c4de06e270e1b52437a98b | 0 .../stats20-4-82294461be4728b4b191414bf2fb3bd7 | 0 .../stats20-5-76509775cfe11bb98ee088188a07668a | 0 .../stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 | 0 .../stats20-7-82294461be4728b4b191414bf2fb3bd7 | 0 .../stats20-8-300c971de74642118d36d36349bc81aa | 0 .../stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 .../stats3-1-418ec894d08c33fd712eb358f579b7a0 | 0 .../stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 | 0 .../stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa | 0 .../stats3-12-892cb7ecc26e84f1c033b95a3ee3edc | 0 .../stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 | 0 .../stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa | 0 .../stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 | 0 .../stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 | 0 .../stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 | 0 .../stats3-18-eefbb3ee8b538aec85c609351e52901b | 0 .../stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 | 0 .../stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 | 0 .../stats3-20-ca048ad81b3df7159822073d206f0790 | 0 .../stats3-3-ca048ad81b3df7159822073d206f0790 | 0 .../stats3-4-fa705a031ff5d97558f29c2b5b9de282 | 0 .../stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 | 0 .../stats3-6-4bf1504274319c44d370b58092fe016c | 0 .../golden/stats3-7-73d7d55d6e5a57aacce8618902904d | 0 .../stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 | 0 .../stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 | 0 .../stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 .../stats4-1-418ec894d08c33fd712eb358f579b7a0 | 0 .../stats4-10-a33b2c9d962e4921c98e62387f3989f7 | 0 .../stats4-11-ea921e0af59a4940a11c94143b1c4b32 | 0 .../stats4-12-30bc31441828a053d1a675b225a5d617 | 0 .../stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 | 0 .../stats4-14-9c82167763a771c175c656786d545798 | 0 .../stats4-15-f02b95f20b526fcf2850b07ca6be4f8c | 0 .../stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 | 0 .../stats4-17-746b888d14c1b3fa28aa8549c174f6d9 | 0 .../stats4-18-dbe13731de4ab2a3c23343b78525b2f7 | 0 .../stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 | 0 .../golden/stats4-2-463330cf55370dbe92d6ed74ef91302 | 0 .../stats4-20-f63000f2c395b935199c9829964f98c1 | 0 .../stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd | 0 .../stats4-22-f709d5c1d4c9f13b7a521867674ac54c | 0 .../stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd | 0 .../stats4-4-f709d5c1d4c9f13b7a521867674ac54c | 0 .../stats4-5-cd849c4fe1229428da98947e3e43b46d | 0 .../stats4-6-9c0d4354b6a9be351fa32a66ff58a177 | 0 .../stats4-7-16367c381d4b189b3640c92511244bfe | 0 .../stats4-8-a4fb8359a2179ec70777aad6366071b7 | 0 .../stats4-9-255ad4511130fb8c9ee9d65b7c95743f | 0 .../stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 .../golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 | 0 .../stats5-2-6f5d06d6100de19ec398891cb2eae161 | 0 .../stats5-3-96d9aa9c32a081518604959dcfac42df | 0 .../stats5-4-dbf81a12f6c19c14dce831e942870744 | 0 .../golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 | 0 .../stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 .../golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 | 0 .../stats6-10-653f748fa2d690502ba4fda407841a20 | 0 .../stats6-11-1c9f833953403596ad50fd32e513642c | 0 .../stats6-12-fdbe263d925f28d598a71b7a4c577492 | 0 .../stats6-2-a4fb8359a2179ec70777aad6366071b7 | 0 .../stats6-3-16367c381d4b189b3640c92511244bfe | 0 .../stats6-4-a88c476a632cd92f748967fadb242405 | 0 .../stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats6-6-8926095434b70c83bf88c70559d38dce | 0 .../stats6-7-6615049191cfa4026a3a870c3c3749f4 | 0 .../stats6-8-e15e242124e61ede9196130cb3fb69e7 | 0 .../stats6-9-e6b884de17a29eb476fd6cc502fc615d | 0 .../stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 .../golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 | 0 .../stats7-10-fdbe263d925f28d598a71b7a4c577492 | 0 .../stats7-2-a4fb8359a2179ec70777aad6366071b7 | 0 .../stats7-3-16367c381d4b189b3640c92511244bfe | 0 .../stats7-4-a88c476a632cd92f748967fadb242405 | 0 .../stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats7-6-5d0c6aa78c9786d904728ff7adab85f2 | 0 .../stats7-7-4912a8bbc138ab97ac0983bc90951de4 | 0 .../stats7-8-e15e242124e61ede9196130cb3fb69e7 | 0 .../stats7-9-e6b884de17a29eb476fd6cc502fc615d | 0 .../stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 .../golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 | 0 .../stats8-10-ce78d029b7764edce3a26336cfac6a8 | 0 .../stats8-11-6615049191cfa4026a3a870c3c3749f4 | 0 .../stats8-12-e6b884de17a29eb476fd6cc502fc615d | 0 .../stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d | 0 .../stats8-14-ea9afc1343991ed4d410231803a174f7 | 0 .../stats8-15-653f748fa2d690502ba4fda407841a20 | 0 .../stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 | 0 .../stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 | 0 .../stats8-18-1c9f833953403596ad50fd32e513642c | 0 .../stats8-19-dbf72430cff3df0e6e9405ad64531b16 | 0 .../stats8-2-a4fb8359a2179ec70777aad6366071b7 | 0 .../stats8-20-bff592750d1057448d2cff7694b6dad2 | 0 .../stats8-21-e15e242124e61ede9196130cb3fb69e7 | 0 .../stats8-22-e6b884de17a29eb476fd6cc502fc615d | 0 .../stats8-23-653f748fa2d690502ba4fda407841a20 | 0 .../stats8-24-1c9f833953403596ad50fd32e513642c | 0 .../stats8-25-fdbe263d925f28d598a71b7a4c577492 | 0 .../stats8-3-16367c381d4b189b3640c92511244bfe | 0 .../stats8-4-a88c476a632cd92f748967fadb242405 | 0 .../stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats8-6-821e1f061960081b9b379d9bfb47f267 | 0 .../stats8-7-8926095434b70c83bf88c70559d38dce | 0 .../stats8-8-e15e242124e61ede9196130cb3fb69e7 | 0 .../stats8-9-fdbe263d925f28d598a71b7a4c577492 | 0 .../stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 .../golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...regator_error_1-0-6d1832d28f897d0049de053617bd36 | 0 ...gator_error_1-1-887fe99770f53e7e0a0fbdc190118612 | 0 ...ator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 | 0 ...gator_error_1-11-d58626190cded8d09f0457739a980eb | 0 ...ator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c | 0 ...ator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 | 0 ...ator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 | 0 ...ator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c | 0 ...ator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 | 0 ...egator_error_1-2-2ca079278e0de95eecb5df315ce05c6 | 0 ...gator_error_1-3-66e3e0c942759f679c270698b49bfcf1 | 0 ...gator_error_1-4-d389db66cc7fd9b144445e364dac30e3 | 0 ...gator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 | 0 ...gator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c | 0 ...gator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 | 0 ...gator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 | 0 ...gator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c | 0 ...mpty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 | 0 ...mpty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 | 0 ...mpty_dyn_part-2-c14f09f88961dbad4d800317079a9105 | 0 ...mpty_dyn_part-3-16367c381d4b189b3640c92511244bfe | 0 ...mpty_dyn_part-4-cc664530711607c530a2cd384e67a600 | 0 ...mpty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 | 0 ...pty_partition-0-42ae9007e347f7dedfc8974d21d84b41 | 0 ...pty_partition-1-418ec894d08c33fd712eb358f579b7a0 | 0 ...pty_partition-2-c14f09f88961dbad4d800317079a9105 | 0 ...pty_partition-3-868ba7a812c705caa29db49cdcb69c45 | 0 ...pty_partition-4-aed016ae4b528521874a719a5b129a55 | 0 ...tats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 ...stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d | 0 ...ats_noscan_1-11-653f748fa2d690502ba4fda407841a20 | 0 ...ats_noscan_1-12-1c9f833953403596ad50fd32e513642c | 0 ...ats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 | 0 ...ats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab | 0 ...ats_noscan_1-15-db563e338e4f658e5072cc60aef15480 | 0 ...ats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 | 0 ...ats_noscan_1-17-82369b182db851e06bfddb62965e03a3 | 0 ...ats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 | 0 ...ats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e | 0 ...tats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 | 0 ...ats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f | 0 ...ats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 | 0 ...ats_noscan_1-22-521b49d223a32056025fb8dbd371a72a | 0 ...ats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 | 0 ...tats_noscan_1-3-16367c381d4b189b3640c92511244bfe | 0 ...tats_noscan_1-4-7938a68104e163566da69ccc70a18f2c | 0 ...tats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 ...tats_noscan_1-6-a1fd405e5175757aaa80033162c87670 | 0 ...tats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e | 0 ...tats_noscan_1-8-623f3701ead20fff786c203d23dd60ae | 0 ...tats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 | 0 ...ts_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 ...ats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...s_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 | 0 ...s_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 | 0 ...s_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef | 0 ...s_partscan_1-13-2fe3131322b6c82e217f27e95581e681 | 0 ...s_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c | 0 ...s_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 | 0 ...s_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 | 0 ...ts_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 | 0 ...ts_partscan_1-3-16367c381d4b189b3640c92511244bfe | 0 ...ts_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf | 0 ...ts_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 | 0 ...ts_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 | 0 ...ts_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea | 0 ...ts_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 | 0 ...ts_partscan_1-9-90d41ae72606f9616cb7b1303f997348 | 0 ...partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f | 0 ..._partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...artscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 | 0 ...artscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 | 0 ...artscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef | 0 ...artscan_1_23-13-2fe3131322b6c82e217f27e95581e681 | 0 ...artscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c | 0 ...artscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 | 0 ...artscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 | 0 ...partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 | 0 ...partscan_1_23-3-16367c381d4b189b3640c92511244bfe | 0 ...partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf | 0 ...partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 | 0 ...partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 | 0 ...partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea | 0 ...partscan_1_23-8-b158e24051ecb66b8af46743917771ca | 0 ...partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 | 0 ...lisher_error_1-0-836d3689af091fdebbdf31e6b29a423 | 0 ...isher_error_1-1-887fe99770f53e7e0a0fbdc190118612 | 0 ...sher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 | 0 ...sher_error_1-11-50519f196d320ee97be731907409fef9 | 0 ...sher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c | 0 ...sher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 | 0 ...isher_error_1-14-f596f33c4a9271a406371baef799e52 | 0 ...sher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c | 0 ...sher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 | 0 ...lisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 | 0 ...isher_error_1-3-66e3e0c942759f679c270698b49bfcf1 | 0 ...isher_error_1-4-d389db66cc7fd9b144445e364dac30e3 | 0 ...lisher_error_1-5-f39234f16f5819a2f51d3f200293332 | 0 ...isher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c | 0 ...isher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 | 0 ...isher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b | 0 ...isher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c | 0 .../str_to_map-0-aefd618b58ad6c37956755b6572cbc73 | 0 .../str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c | 0 .../str_to_map-10-32997010bba305ec40812df254490730 | 0 .../str_to_map-11-d99f1b631bc42a6a57c581025230537a | 0 .../str_to_map-12-f793eb7b36a1d6379b90e241df62c72e | 0 .../str_to_map-13-32997010bba305ec40812df254490730 | 0 .../str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec | 0 .../str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 | 0 .../str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 | 0 .../str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda | 0 .../str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf | 0 .../str_to_map-7-5641678c53ce6ef1dbce3994843cfcad | 0 .../str_to_map-8-84121d964faad3547f0e5cce9d268612 | 0 .../str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd | 0 ...tring literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 | 0 .../golden/subq2-0-279fb87476c70e9384874fa6bf77a380 | 0 .../golden/subq2-1-235919a7ddb574662158503b8052e7ca | 0 ...ry-alias.attr-0-fc8183d758151be72b3d75d9df124504 | 0 .../table.attr-0-26c9d24eb6305ea2106c26bdca38012e | 0 ...e_with_select-0-554898fe9d96fee7b652b8a771d66a25 | 0 ...e_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 | 0 ...e_with_select-2-58d48c815413287858e2cfa16e5c6a5d | 0 ...e_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 | 0 ...n_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 | 0 ...n_whereclause-1-3e38e42c5b72364c5461c626f312be8c | 0 ...n_whereclause-2-183920d856ad75e6d1e15121d3cd7364 | 0 .../timestamp_1-0-d362501d0176855077e65f8faf067fa8 | 0 .../timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 | 0 .../timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 | 0 .../timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 | 0 .../timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 | 0 .../timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 | 0 .../timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 | 0 .../timestamp_3-1-81edf5107270547641586aa02b4e7d9b | 0 .../timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 | 0 ...imestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 | 0 ...imestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc | 0 ...imestamp_null-0-3bd56783b637a47f6447d247024941a0 | 0 ...imestamp_null-1-4b9c19c1f57557979d3483d548f762cb | 0 ...imestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 | 0 ...timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af | 0 ...timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d | 0 ...timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 | 0 ...timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f | 0 .../golden/touch-0-cd46bc635e3010cf1b990a652a584a09 | 0 .../golden/touch-1-27ad2962fed131f51ba802596ba37278 | 0 .../touch-10-27ad2962fed131f51ba802596ba37278 | 0 .../golden/touch-2-3e95421993ab28d18245ec2340f580a3 | 0 .../golden/touch-3-c0c18ac884677231a41eea8d980d0451 | 0 .../golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a | 0 .../golden/touch-5-af459a0264559a2aeaa1341ce779ab3c | 0 .../golden/touch-6-5c7b24abea435f2628fe618f3a82e115 | 0 .../golden/touch-7-3752320b12abae0d138148d56a27c5b1 | 0 .../golden/touch-8-3562ab40242756452595cd7eae79b0ce | 0 .../golden/touch-9-cd46bc635e3010cf1b990a652a584a09 | 0 .../transform-0-d81d055660f6ef3d9cc60dd673a8c0fe | 0 .../transform1-0-b6919fc48901e388c869c84ae0211102 | 0 .../transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 | 0 .../transform1-2-25d6cab86c36d65fabf5645db3126a19 | 0 .../transform1-3-8324a70d533a06a5171c1016b1fea7c3 | 0 .../transform1-4-65527bae8e73262255ef83082c6968f9 | 0 .../transform1-5-e0037a3f97ce0127a40d163af4c20ad5 | 0 .../transform1-6-3b862abd732c9e9f0db50ad0b9dae6f | 0 ...ransform_ppr1-0-ae225e86c2ae20519ffdf23190454161 | 0 ...ransform_ppr1-1-65fe307c21b350846b3c496890b0b619 | 0 ...ransform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 | 0 ...ransform_ppr2-0-ae225e86c2ae20519ffdf23190454161 | 0 ...ransform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 | 0 ...ransform_ppr2-2-636c4938673a273299c8569295d27c99 | 0 ...oin ON clause-0-3b6afcbd622aa111ee260bebc763613d | 0 ... where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 | 0 ...type_widening-0-630ac2c7e7dea4837384ccd572209229 | 0 ...type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 | 0 ...type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 | 0 ...type_widening-3-65da8c67f6903286168acb39ac67fc04 | 0 ...f_collect_set-0-38512a3299e2390dd813e61a0f63f35e | 0 ...f_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 | 0 ...f_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db | 0 ..._collect_set-11-863233ccd616401efb4bf83c4b9e3a52 | 0 ..._collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 | 0 ...f_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db | 0 ...f_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 | 0 ...f_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e | 0 ...af_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db | 0 ...f_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 | 0 ...f_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e | 0 ...af_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db | 0 ...f_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 | 0 ...f_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 | 0 .../udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 | 0 .../udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 | 0 .../udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 | 0 .../udaf_corr-2-c6f2dc536bf105650a461816ae5e330 | 0 .../udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e | 0 .../udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a | 0 .../udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb | 0 .../udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 | 0 .../udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 | 0 .../udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 | 0 .../udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 | 0 ...daf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 | 0 ...daf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 | 0 ...af_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 | 0 ...udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 | 0 ...daf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 | 0 ...daf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad | 0 ...daf_covar_pop-5-22004d36f6f3770db284644317770fcd | 0 ...daf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b | 0 ...daf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae | 0 ...daf_covar_pop-8-1e51388408dad651127edf940c11d91f | 0 ...daf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 | 0 ...af_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 | 0 ...af_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 | 0 ...f_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 | 0 ...daf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 | 0 ...af_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf | 0 ...af_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c | 0 ...daf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b | 0 ...af_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 | 0 ...af_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 | 0 ...af_covar_samp-8-44861ae58cf0951eeda28a5f778f778a | 0 ...af_covar_samp-9-234a5b02085d2228473d7ef15a6af683 | 0 ...ogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 | 0 ...ogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 | 0 ...ogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 | 0 ...ogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 | 0 ...number_format-0-eff4ef3c207d14d5121368f294697964 | 0 ...number_format-1-4a03c4328565c60ca99689239f07fb16 | 0 ...ile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac | 0 ...ile_approx_20-1-c7d32089880679d178dea94f1fe118e6 | 0 ...le_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae | 0 ...le_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 | 0 ...le_approx_20-12-914ba18b45a27894bd82302f07efc789 | 0 ...le_approx_20-13-4bd5703fa32f3283f38841acadc97adb | 0 ...le_approx_20-14-d861a06b90896a097901d64ab9fbec53 | 0 ...le_approx_20-15-ca796efecd0d064e9e688a17ce75d80f | 0 ...le_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 | 0 ...le_approx_20-17-b89ea2173180c8ae423d856f943e061f | 0 ...le_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...ile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 | 0 ...ile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a | 0 ...le_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c | 0 ...le_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff | 0 ...le_approx_20-22-ed1aec1a908310db90c5f8667631a1df | 0 ...le_approx_20-23-333d72e8bce6d11a35fc7a30418f225b | 0 ...le_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 | 0 ...le_approx_20-25-15f40568b41c4505841f5ad13c526f51 | 0 ...le_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 0 ...le_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 | 0 ...le_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 | 0 ...le_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb | 0 ...ile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 | 0 ...le_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c | 0 ...le_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 0 ...le_approx_20-32-dbcec232623048c7748b708123e18bf0 | 0 ...le_approx_20-33-f28c7b0408737da815493741c806ff80 | 0 ...le_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c | 0 ...le_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff | 0 ...le_approx_20-36-ed1aec1a908310db90c5f8667631a1df | 0 ...le_approx_20-37-333d72e8bce6d11a35fc7a30418f225b | 0 ...le_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 | 0 ...le_approx_20-39-15f40568b41c4505841f5ad13c526f51 | 0 ...ile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f | 0 ...le_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 0 ...le_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 | 0 ...le_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 | 0 ...le_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb | 0 ...le_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c | 0 ...le_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 0 ...ile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 | 0 ...ile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 | 0 ...ile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 | 0 ...tile_approx_20-8-7e3cf228c457279965b7414bd05527f | 0 ...ile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 | 0 ...ile_approx_23-0-477a942be95c0616c72f02a0077f9ace | 0 ...ile_approx_23-1-c7d32089880679d178dea94f1fe118e6 | 0 ...le_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae | 0 ...le_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 | 0 ...le_approx_23-12-914ba18b45a27894bd82302f07efc789 | 0 ...le_approx_23-13-4bd5703fa32f3283f38841acadc97adb | 0 ...le_approx_23-14-d861a06b90896a097901d64ab9fbec53 | 0 ...le_approx_23-15-ca796efecd0d064e9e688a17ce75d80f | 0 ...le_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 | 0 ...le_approx_23-17-b89ea2173180c8ae423d856f943e061f | 0 ...le_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 | 0 ...ile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 | 0 ...ile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a | 0 ...le_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c | 0 ...le_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff | 0 ...le_approx_23-22-ed1aec1a908310db90c5f8667631a1df | 0 ...le_approx_23-23-333d72e8bce6d11a35fc7a30418f225b | 0 ...le_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 | 0 ...le_approx_23-25-15f40568b41c4505841f5ad13c526f51 | 0 ...le_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 0 ...le_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 | 0 ...le_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 | 0 ...le_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb | 0 ...ile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 | 0 ...le_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c | 0 ...le_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 0 ...le_approx_23-32-dbcec232623048c7748b708123e18bf0 | 0 ...le_approx_23-33-f28c7b0408737da815493741c806ff80 | 0 ...le_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c | 0 ...le_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff | 0 ...le_approx_23-36-ed1aec1a908310db90c5f8667631a1df | 0 ...le_approx_23-37-333d72e8bce6d11a35fc7a30418f225b | 0 ...le_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 | 0 ...le_approx_23-39-15f40568b41c4505841f5ad13c526f51 | 0 ...ile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f | 0 ...le_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 0 ...le_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 | 0 ...le_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 | 0 ...le_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb | 0 ...le_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c | 0 ...le_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 0 ...ile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 | 0 ...ile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 | 0 ...ile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 | 0 ...tile_approx_23-8-7e3cf228c457279965b7414bd05527f | 0 ...ile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 | 0 .../golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 | 0 .../golden/udf1-1-8281592c818ada269024ac669bec78da | 0 .../golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f | 0 .../golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../golden/udf2-0-2a18d9570d9b676e240cda76df818c42 | 0 .../golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 | 0 .../golden/udf2-2-31c8af064bac42541558a95fad3bca97 | 0 .../golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 | 0 .../golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c | 0 .../golden/udf3-1-1d04874d496d05cfe0b9d86de1111 | 0 .../golden/udf3-2-25fe77d053e2bad8ae99757ce237052e | 0 .../golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f | 0 .../golden/udf6-0-2a18d9570d9b676e240cda76df818c42 | 0 .../golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 | 0 .../golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 | 0 .../golden/udf6-3-e579646b969eef49b09656114da52a73 | 0 .../golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 | 0 .../golden/udf6-5-fe336cd9850d6357980bd19139f76e | 0 .../golden/udf7-0-2a18d9570d9b676e240cda76df818c42 | 0 .../golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 | 0 .../golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 | 0 .../golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 | 0 .../golden/udf8-0-2a18d9570d9b676e240cda76df818c42 | 0 .../golden/udf8-1-63c38297946a2060c0cff4a426b0520c | 0 .../golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 | 0 .../golden/udf8-3-72c1a1dd1627491550f6e19581a654cb | 0 .../golden/udf8-4-9f22d5a65353432826a526b1d76eb65b | 0 .../golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 | 0 .../golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c | 0 .../udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 | 0 .../udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e | 0 .../udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 | 0 .../golden/udf_E-0-33251f00f840de3672f19d353fcfa66f | 0 .../golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c | 0 .../golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a | 0 .../golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 | 0 .../golden/udf_E-4-33251f00f840de3672f19d353fcfa66f | 0 .../golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c | 0 .../golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a | 0 .../golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 | 0 .../udf_PI-0-b28e761e5564b51f98f182f561c1369f | 0 .../udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 | 0 .../udf_PI-2-9c1476a2eab7455594e97b338ee3c188 | 0 .../udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 | 0 .../udf_PI-4-b28e761e5564b51f98f182f561c1369f | 0 .../udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 | 0 .../udf_PI-6-9c1476a2eab7455594e97b338ee3c188 | 0 .../udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 | 0 .../udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 | 0 .../udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 | 0 .../udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 | 0 .../udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 | 0 .../udf_abs-4-30cd5a94c13e1619ee18b9551db879c | 0 .../udf_abs-5-343e899acb67c283391387f02aa7b5c4 | 0 .../udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 | 0 .../udf_acos-1-d330511cf4f626cd844b18f57f99a85f | 0 .../udf_acos-2-86fca49baf270873b46709c9eaeab87b | 0 .../udf_acos-3-f7f199e5f3dde8056465d55aca29e884 | 0 .../udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a | 0 .../udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e | 0 .../udf_add-0-5db764318a918a5f6d7c1d95d9e86045 | 0 .../udf_add-1-400b238f4e6cdf7120be566b0ef079c5 | 0 .../udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee | 0 .../udf_array-1-570741914bb78300b0233e5f38d7f08a | 0 .../udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 | 0 .../udf_array-3-a5d12c41277fb158e09281169c905122 | 0 ...rray_contains-0-d9a90108b052b111e8de4433e008b25a | 0 ...rray_contains-1-eff16c7836252e01f3d8190cd833f79c | 0 ...rray_contains-2-42d966b28e61a465d638bffc20ac7247 | 0 ...rray_contains-3-ec0cd851fd8135dd9bc822d9a0432569 | 0 .../udf_ascii-0-72924c23459330cca6a54c70b12a542c | 0 .../udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 | 0 .../udf_ascii-2-a9e207503f52d43903877fb998eabeaa | 0 .../udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 | 0 .../udf_asin-0-99da197a53767060e3fa0250254d59cd | 0 .../udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 | 0 .../udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f | 0 .../udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 | 0 .../udf_asin-4-929dabad86ef0e564802de8f663a9e66 | 0 .../udf_asin-5-1ee8715cce9831623d0af0031964d284 | 0 .../udf_atan-0-c79ed30c2444c8493d0db98c33c9132b | 0 .../udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 | 0 .../udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 | 0 .../udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 | 0 .../udf_atan-4-c79ed30c2444c8493d0db98c33c9132b | 0 .../udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 | 0 .../udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 | 0 .../udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 | 0 .../udf_avg-0-2d715528b290951fb9874f60d7e9b537 | 0 .../udf_avg-1-c707c56871a903e4e022b3df5c92fc3f | 0 .../udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 | 0 .../udf_bigint-1-4636e4f0083ea54814995a03b7c81202 | 0 .../udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b | 0 .../udf_bin-1-843803a1b6ada107c11397af1a2f55d6 | 0 .../udf_bin-2-5ee3932ab9cd164f1005a4413a68007b | 0 .../udf_bin-3-b72fc578a7c677e15b8598248c81901 | 0 ...df_bitmap_and-0-abea2a2780fad42422774174cbfd603d | 0 ...df_bitmap_and-1-414291f11711df40fb8362e0a0156b25 | 0 ...df_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b | 0 ...df_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 | 0 ...df_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 | 0 ...df_bitmap_and-5-ff2860a163cbe78d5affac8047199296 | 0 ...df_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b | 0 ..._bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 | 0 ..._bitmap_empty-1-a03987655a167f5b18c37b643391a0df | 0 ...udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c | 0 ...udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef | 0 ...udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b | 0 ...udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 | 0 ...udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 | 0 ...udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 | 0 ...udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b | 0 ...f_bitwise_and-0-e2374700cd32add926992d5539bd463a | 0 ...f_bitwise_and-1-2e63ac31262106160ab043027e356a4b | 0 ...f_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f | 0 ...f_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d | 0 ...df_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 | 0 ...df_bitwise_or-1-272722c23fece2807c08191d3969c3bb | 0 ...f_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 | 0 ...f_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f | 0 .../udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 | 0 .../udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e | 0 .../udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 | 0 .../udf_case-1-54acf006155d8822a50e47729be24004 | 0 .../udf_case-2-98ee676f92950375917f09d2e492253f | 0 .../udf_case-3-ec7343402fd77807842a0eaf2497a47c | 0 .../udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 | 0 ...olumn_pruning-0-dd2d7a075df235f17c26bac8713e939c | 0 ...f_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 | 0 ...f_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 | 0 .../udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 | 0 .../udf_ceil-1-f410065d893a263f375fcf74072877bb | 0 .../udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd | 0 .../udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df | 0 .../udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 | 0 .../udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 | 0 .../udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 | 0 .../udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 | 0 .../udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 | 0 .../udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 | 0 ...e_java_string-0-32b16ab99287db115e8de5214ac24b77 | 0 .../udf_concat-0-7bc53505a4e6587132870d8d0a704d2 | 0 .../udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 | 0 .../udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 | 0 ...oncat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...oncat_insert1-1-ed0d453b9879faed8bf93913f7690545 | 0 ...oncat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f | 0 ...oncat_insert2-0-659e06570690cceeb3f37e10e855d2ea | 0 ...oncat_insert2-1-39829ebc03d9e047f3eaac80b4428768 | 0 ...oncat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f | 0 ...udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 | 0 ...udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab | 0 ...udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c | 0 ...udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 | 0 ...udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f | 0 ...udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 | 0 ...udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 | 0 ...udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c | 0 ...udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 | 0 .../udf_conv-0-d552befca345f3396464529cfde9f75a | 0 .../udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab | 0 .../udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 | 0 .../udf_conv-2-6d61a2118b54843716aef87fe539b595 | 0 .../udf_conv-3-97161f7a60851d445b23c4ebe4095a1d | 0 .../udf_conv-4-568e843076f358c404a8634b18541c55 | 0 .../udf_conv-5-3f23d98799b825a2e9594066f973d183 | 0 .../udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a | 0 .../udf_conv-7-77bd25ad13e3697c80464e4a2682360e | 0 .../udf_conv-8-2fae52d392251be476e0c8f6071a4aeb | 0 .../udf_conv-9-2f0098c00c10044522cd23a4a2f54957 | 0 .../udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 | 0 .../udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 | 0 .../udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 | 0 .../udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 | 0 .../udf_count-0-534a9b25b87d09e418645b1216949560 | 0 .../udf_count-1-d566feb21bc894b97e6416b65fe5c02f | 0 .../udf_count-10-455b30e7df33c5eebcaacabe0a578483 | 0 .../udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e | 0 .../udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e | 0 .../udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 | 0 .../udf_count-2-3344645abe535426307b9327b381fc85 | 0 .../udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 | 0 .../udf_count-4-93223bab63eada6ece8bc176e89c87a8 | 0 .../udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 | 0 .../udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 | 0 .../udf_count-7-b975ad0d5f293508ce4832a7b19399b6 | 0 .../udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 | 0 .../udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 | 0 .../udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 | 0 .../udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 | 0 .../udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 | 0 .../udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 | 0 .../udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 | 0 .../udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 | 0 .../udf_day-0-c4c503756384ff1220222d84fd25e756 | 0 .../udf_day-1-87168babe1110fe4c38269843414ca4 | 0 ...df_dayofmonth-0-7b2caf942528656555cf19c261a18502 | 0 ...df_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 | 0 .../udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae | 0 .../udf_degrees-1-aabc6065a03b7da809376cc127af47d7 | 0 .../udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b | 0 .../udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 | 0 .../udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae | 0 .../udf_degrees-5-aabc6065a03b7da809376cc127af47d7 | 0 .../udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b | 0 .../udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 | 0 .../udf_div-0-31d31c7d5c544327dabfd874c88314db | 0 .../udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 | 0 .../golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b | 0 .../udf_divide-0-1af8b249439ee5b7d4978c31669bc208 | 0 .../udf_divide-1-fa932664bae88683a222b71ac45fb840 | 0 .../udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 | 0 .../udf_double-0-aa32d73a99587cae2f1efd9a2094d617 | 0 .../udf_double-1-79380157cbd6624d760335f8291e6fb4 | 0 .../udf_elt-0-b46b060da76d1772db998c26a62a608f | 0 .../udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 | 0 .../udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a | 0 .../udf_elt-3-f3be980cf4fa166f299c6ec79e981814 | 0 .../udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a | 0 .../udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 | 0 .../udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 | 0 .../udf_field-1-379d8580693493f620a8f4084709324f | 0 .../udf_field-10-7982ea72163dbc4cd45f53454edf66c8 | 0 .../udf_field-2-d2c6583a79d77aabe388a52ec164c38b | 0 .../udf_field-3-fea09e934696af40bb604b40225bbc98 | 0 .../udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 | 0 .../udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 | 0 .../udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 | 0 .../udf_field-7-6aa3518e9f55299754521e959e9376ef | 0 .../udf_field-8-66dc6c81db0ac9b2075783b0d8976083 | 0 .../udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 | 0 ...f_find_in_set-0-18d3e88b18c18a00598146a3307276f2 | 0 ...f_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 | 0 ..._find_in_set-10-16355c6b7e169b3c0ef506c149c6853c | 0 ..._find_in_set-11-5a8515684c458d3fffea539a3d170e3a | 0 ...f_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 | 0 ...f_find_in_set-13-2c986a80620c9238e1f663fc591760a | 0 ..._find_in_set-14-189def133b1871ce8345a8123811a6b5 | 0 ..._find_in_set-15-671bff8f50feea55015a8412fc6e5ceb | 0 ..._find_in_set-16-d5d22082588c5fc30ef502237c5797f4 | 0 ..._find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 | 0 ...f_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 | 0 ...f_find_in_set-3-132b7bc7812db7683eb3bff607275d0e | 0 ...f_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b | 0 ...f_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f | 0 ...f_find_in_set-6-591e070365f19c65e453b98b88f5f823 | 0 ...f_find_in_set-7-72d05b5cf99388d539adec38c40978c3 | 0 ...f_find_in_set-8-780771cad9bec96a216aea8ab293c941 | 0 ...f_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 | 0 .../udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 | 0 .../udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b | 0 .../udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 | 0 .../udf_floor-1-497a1ddbcf738aead319fde4f90f5248 | 0 ...format_number-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...format_number-1-525f133cfff86d44afdeeda667c365a5 | 0 ...format_number-2-591f302d5c1cd24e153a598fa0b352fd | 0 ...format_number-3-c89564db1ab953e28b050b9740f2650c | 0 ...format_number-4-295d41a2146a27320c2be90499343260 | 0 ...format_number-5-881f33f6727a30629bde6e4b178cf7d9 | 0 ...format_number-6-a6720a128716e179e18933992ca899b3 | 0 ...format_number-7-84a460780828b0b9a2235314cfc24766 | 0 ...format_number-8-e7eedc849c74ce7d33c559067dd9ca0e | 0 ...format_number-9-407a0a7c277bb4c5c94ce16533ce1646 | 0 ...from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f | 0 ...from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 | 0 ...t_json_object-0-c08e7139c00878b98d396e65d958100f | 0 ...t_json_object-1-706bcfd51431ec7f2b80145837f94917 | 0 ...t_json_object-2-2a18d9570d9b676e240cda76df818c42 | 0 ...t_json_object-3-f60851dc36f579e83d6848d7d3c589e6 | 0 ...t_json_object-4-4f08101fd66fb25d7b322d47773e49f3 | 0 ...f_greaterthan-0-99d268829a124103cb3429c53fdc4de4 | 0 ...f_greaterthan-1-8aab8e39726a986e10e1e572939fd63c | 0 ...f_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f | 0 ...erthanorequal-0-a7214027a91abf6501881e2af313347a | 0 ...erthanorequal-1-3669f2008e7f428f365efadbcb5ae451 | 0 ...erthanorequal-2-d2690cc7713e91201bb10ef291c95819 | 0 .../udf_hash-0-b9e3a3986320d275982797140edfccf4 | 0 .../udf_hash-1-a18646b51501d0b1beb967dc79afbd1a | 0 .../udf_hash-2-cc121f3c38a7a522abd824940fe04285 | 0 .../udf_hash-3-23991312391d518aacf3d4469c816eae | 0 .../udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 | 0 .../udf_hex-1-d55348c0ccd133b7abb690f6949b520c | 0 .../udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 | 0 .../udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf | 0 .../udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea | 0 .../udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce | 0 .../udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 | 0 .../udf_hour-2-ace1054795b20abd5ae829814cfe15a | 0 .../udf_hour-3-415b0842ab0818c82baf9fbf07180613 | 0 .../udf_if-0-b7ffa85b5785cccef2af1b285348cc2c | 0 .../udf_if-1-30cf7f51f92b5684e556deff3032d49a | 0 .../udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 | 0 .../udf_if-3-20206f17367ff284d67044abd745ce9f | 0 .../udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca | 0 .../udf_if-5-a7db13aec05c97792f9331d63709d8cc | 0 .../udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 | 0 .../udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 | 0 .../udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 | 0 .../udf_in_file-1-2f23153970a569a4643574dde8d78a58 | 0 .../udf_index-0-a277ac394cae40cb55d1ef3aa5add260 | 0 .../udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 | 0 .../udf_instr-0-2e76f819563dbaba4beb51e3a130b922 | 0 .../udf_instr-1-32da357fc754badd6e3898dcc8989182 | 0 .../udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 | 0 .../udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 | 0 .../udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba | 0 .../udf_int-1-3f0405ff93adfe8b3402b118567867d7 | 0 ...udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 | 0 .../udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 | 0 .../udf_isnull-0-ac8e7827d760108923509f9ea1691d53 | 0 .../udf_isnull-1-55d9d04204f30cde4aa2667db88db262 | 0 ...ull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 | 0 ...ull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 | 0 ...ull_isnotnull-2-44584503014c378bb916b38e1879bfb6 | 0 ...null_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 | 0 ...ull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 | 0 ...ull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 | 0 ...ull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 | 0 ...ull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b | 0 ...f_java_method-0-991b98a25032b21802bc2a1efde606c7 | 0 ...f_java_method-1-a3b94d9f2c2caf85a588b6686a64630a | 0 ...f_java_method-2-69e6b8725086a8fb8f55721705442112 | 0 ...f_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd | 0 .../udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad | 0 .../udf_lcase-1-d947c47e03bedbfd4954853cc134c66e | 0 .../udf_length-0-38364b60c3a2409f53c9aa2dae19903b | 0 .../udf_length-1-f183e1f8ae516bb483132ed106289b67 | 0 .../udf_length-10-f3a9bd30540345db0f69b6847014b333 | 0 .../udf_length-2-af46cb6887618240836eaf5be8afbba6 | 0 .../udf_length-3-dcd6404afce1103d5054527e6c216d6d | 0 .../udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 | 0 .../udf_length-5-adc1ec67836b26b60d8547c4996bfd8f | 0 .../udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 .../udf_length-7-8f28e6c488df47d89dca670f04a7563f | 0 .../udf_length-8-5e0fe761b7520651c3446ce7f9179caf | 0 .../udf_length-9-de456a5765db4a06110d9483985aa4a6 | 0 .../udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 | 0 .../udf_lessthan-1-952c655a1092a410e5346f1205cb8142 | 0 .../udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 | 0 ...ssthanorequal-0-b3b021456c85da76d1879191886a425b | 0 ...ssthanorequal-1-869d983466744ad73b109211e1638869 | 0 ...ssthanorequal-2-56775013e20ecf2287e07e83eccf2e0c | 0 .../udf_like-0-e0ba9a953e50554bdcbc55585cffde09 | 0 .../udf_like-1-9781f89d352c506e972ad2a1d58ec03a | 0 .../udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 | 0 .../udf_like-3-bef03784eab9d5e8404fd24960dea4fc | 0 .../udf_like-4-af5fe5d5d176f751747bf14055d00a12 | 0 .../udf_ln-0-779eed5722a0efaa85efe24c559072b4 | 0 .../udf_ln-1-60e3541b3c703d6413869d774df9b7e4 | 0 .../udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e | 0 .../udf_locate-1-d9b5934457931447874d6bb7c13de478 | 0 .../udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c | 0 .../udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 | 0 .../udf_log-0-ca773bc1afa66218f3c13dee676bd87a | 0 .../udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 | 0 .../udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f | 0 .../udf_log10-1-abf1173290ef905d24d422faf7801fe3 | 0 .../udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 | 0 .../udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 | 0 ..._java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 | 0 .../udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 | 0 .../udf_lower-1-550f0a6da388596a775d921b9da995c | 0 .../udf_lower-2-467230d9c8442c726e5377b70257ff5e | 0 .../udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 | 0 .../udf_lpad-0-995646acf1e23cea7825412915921bef | 0 .../udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 | 0 .../udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 | 0 .../udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae | 0 .../udf_ltrim-0-398a623504c47bcd64fe8d200c41402f | 0 .../udf_ltrim-1-658d495908097792a0e33a77becac2 | 0 .../udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e | 0 .../udf_map-1-1f8cd98df9bf7b2528506551fef87dcf | 0 .../udf_map-2-a3f90085abab46205e732b4c27b18340 | 0 .../udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 | 0 .../udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb | 0 .../udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa | 0 .../udf_map_keys-2-731b529a9a234473312b7d1db15be75f | 0 .../udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e | 0 .../udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a | 0 ...df_map_values-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...df_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 | 0 ...df_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 | 0 ...df_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 | 0 ...df_map_values-4-a000d06dd3941756b4bb9ccc46f3620e | 0 .../udf_max-0-ac7d002a46f773ab680ed8c1ac97821f | 0 .../udf_max-1-14afa1f14687893233a662f0f32a40c9 | 0 .../udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 | 0 .../udf_max-11-863233ccd616401efb4bf83c4b9e3a52 | 0 .../udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 | 0 .../udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 | 0 .../udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 | 0 .../udf_max-3-a7dc16cb82c595b18d4258a38a304b1e | 0 .../udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 | 0 .../udf_max-5-863233ccd616401efb4bf83c4b9e3a52 | 0 .../udf_max-6-a7dc16cb82c595b18d4258a38a304b1e | 0 .../udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 | 0 .../udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 | 0 .../udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 | 0 .../udf_min-0-1a67398a47b4232c3786485b451d1ff8 | 0 .../udf_min-1-69d749d0bca0ebe56e930009e30f4f19 | 0 .../udf_min-10-191613d4d46d1884d0694fcd8c5fb802 | 0 .../udf_min-11-863233ccd616401efb4bf83c4b9e3a52 | 0 .../udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 | 0 .../udf_min-13-191613d4d46d1884d0694fcd8c5fb802 | 0 .../udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 | 0 .../udf_min-3-a7dc16cb82c595b18d4258a38a304b1e | 0 .../udf_min-4-191613d4d46d1884d0694fcd8c5fb802 | 0 .../udf_min-5-863233ccd616401efb4bf83c4b9e3a52 | 0 .../udf_min-6-a7dc16cb82c595b18d4258a38a304b1e | 0 .../udf_min-7-191613d4d46d1884d0694fcd8c5fb802 | 0 .../udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 | 0 .../udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 | 0 .../udf_minute-0-9a38997c1f41f4afe00faa0abc471aee | 0 .../udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 | 0 .../udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 | 0 .../udf_minute-3-270055c684846e87444b037226cf554c | 0 .../udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 | 0 .../udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 | 0 .../udf_month-0-9a38997c1f41f4afe00faa0abc471aee | 0 .../udf_month-1-16995573ac4f4a1b047ad6ee88699e48 | 0 .../udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 | 0 .../udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf | 0 .../udf_negative-2-a6863d2c5fc8c3131fe70080a011392c | 0 .../udf_negative-3-b90eec030fee9cbd177f9615b782d722 | 0 .../udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c | 0 .../udf_negative-5-93d7dd808d4af59bda601faf249a9e | 0 .../udf_negative-6-6758b00c5acc7aac320238accf299219 | 0 .../udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b | 0 .../udf_negative-8-634af0478ed9ed44b851cd7ef834a489 | 0 .../udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa | 0 .../udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc | 0 .../udf_not-1-efefc8302b02224d20f4bb0f159a6911 | 0 .../udf_not-2-7e63750d3027ced0e3452ad4eb0df117 | 0 .../udf_not-3-aa0c674f9ce0feba86448448a211bd2a | 0 .../udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e | 0 .../udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a | 0 .../udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 | 0 .../udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 | 0 .../udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 | 0 .../udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 | 0 .../udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 | 0 .../udf_notequal-7-a71fea4e9514cda0da9542a7701613dd | 0 .../udf_notop-0-825431072651228a5a9de7f85a0498d6 | 0 .../udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 | 0 .../udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 | 0 .../udf_nvl-2-175ed7006e8907b65e0e5357f00a0def | 0 .../udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 | 0 .../udf_or-0-c404aa929eb0dd87269121f8f99ada70 | 0 .../udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 | 0 ...udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 | 0 ...udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab | 0 ...udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d | 0 ...udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 | 0 ...df_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 | 0 ...df_percentile-1-c0825a744cd14917d2c904d014449a4a | 0 ...df_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 | 0 ...df_percentile-3-a7dc16cb82c595b18d4258a38a304b1e | 0 .../udf_pmod-0-ed67184beaf84c0542117c26651938e1 | 0 .../udf_pmod-1-90f75e01dcee85253a501d53b8562dae | 0 .../udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 | 0 .../udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 | 0 .../udf_pmod-3-26d9546f030281a29a50a3e8e5858234 | 0 .../udf_pmod-4-7695df16d24a821224676e6bad3d66d1 | 0 .../udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 | 0 .../udf_pmod-6-3c09a8da2f5645e732c22a45d055125 | 0 .../udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 | 0 .../udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd | 0 .../udf_pmod-9-e7280393102077442aa1d10eb69a6d57 | 0 .../udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 | 0 .../udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c | 0 .../udf_positive-2-610d421e590f035c24e29694a68b0d23 | 0 .../udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 | 0 .../udf_pow-0-c7f5178951dd45dc2a41c16729314d81 | 0 .../udf_pow-1-3c22c000c35144135aedbc7052f10803 | 0 .../udf_power-0-57001d802c281743322d28bbc520cd4 | 0 .../udf_power-1-ebd0398b2cb03f382a16382ddac13426 | 0 .../udf_radians-0-f899daf93b02ca681e0230a792c65e86 | 0 .../udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 | 0 .../udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 | 0 .../udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 | 0 .../udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 0 .../udf_radians-5-f899daf93b02ca681e0230a792c65e86 | 0 .../udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 | 0 .../udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 | 0 .../udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 | 0 .../udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 0 .../udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 | 0 .../udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e | 0 .../udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 | 0 .../udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee | 0 .../udf_reflect-2-b868357466bab2f04685c2dc73604cf0 | 0 .../udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 | 0 .../udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 | 0 .../udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca | 0 .../udf_regexp-0-19917611f74aedc0922560f7f2595948 | 0 .../udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 | 0 .../udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 | 0 ...egexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 | 0 ...egexp_extract-1-8add879ab5904bd805412ef8723276fb | 0 ...egexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e | 0 ...regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc | 0 .../udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b | 0 .../udf_repeat-1-836be47190989d8975a09a545ecbfe0b | 0 .../udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a | 0 .../udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 | 0 .../udf_rlike-0-6ec6ef55ac041208627454e16b501d38 | 0 .../udf_rlike-1-829611a596e0c87431006f7247d25eca | 0 .../udf_round-0-10b53ca1f15fd7879365926f86512d15 | 0 .../udf_round-1-2367bcc43510dedc80bdb6707e434da8 | 0 .../udf_round-2-9ffa2b573360cd879338de46d91ab374 | 0 .../udf_round-3-42a221909d3f7ed51bed01a65670461c | 0 .../udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 | 0 .../udf_round-5-a2414e7845ffafc61f75465508a1850a | 0 .../udf_round-6-48439efa5c34e7589ab5003ed916f12b | 0 .../udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 | 0 .../udf_round_2-1-5e44354af73865d03e9088c0232f16ce | 0 .../udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 | 0 .../udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 | 0 .../udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 | 0 .../udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b | 0 .../udf_round_3-0-8415af605db167315e4d9d3c69d89e6c | 0 .../udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 | 0 .../udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 | 0 .../udf_round_3-3-e94ab3326df006c7203ead86752f16a9 | 0 .../udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 | 0 .../udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c | 0 .../udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 | 0 .../udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 | 0 .../udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 | 0 .../udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a | 0 .../udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd | 0 .../udf_second-0-e004a6f20da3fa6db868ec847b217ff2 | 0 .../udf_second-1-3525f55f4f13253c42b3abaa53d77888 | 0 .../udf_second-2-d678372e3837a16be245d2e33482f17f | 0 .../udf_second-3-2496e4d3c64ca028184431c2930d82cf | 0 ...udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f | 0 ...udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 | 0 ...df_sentences-10-ce188a53f69129c14cbf378d2c3f6630 | 0 ...df_sentences-11-3c8672506e23434187caf4e0064a8a80 | 0 ...df_sentences-12-d55c04a079ca97402509868f24921685 | 0 ...udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 | 0 ...udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 | 0 ...udf_sentences-4-3c8672506e23434187caf4e0064a8a80 | 0 ...udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 | 0 ...udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de | 0 ...udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f | 0 ...udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 | 0 ...udf_sentences-9-68c61b4882802e416d5adaa2de440b59 | 0 .../udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e | 0 .../udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 | 0 .../udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa | 0 .../udf_sign-11-74237f5ecc497813cf9738b21647847a | 0 .../udf_sign-2-85b743f0eed93904789cde4d1b5eafef | 0 .../udf_sign-3-9eeb3985359429abba5d1dd702c66b0d | 0 .../udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa | 0 .../udf_sign-5-74237f5ecc497813cf9738b21647847a | 0 .../udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e | 0 .../udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 | 0 .../udf_sign-8-85b743f0eed93904789cde4d1b5eafef | 0 .../udf_sign-9-9eeb3985359429abba5d1dd702c66b0d | 0 .../udf_sin-0-40b50393869eb0bcde66e36fe41078ee | 0 .../udf_sin-1-2f867f432fb322e21dce353d7eb50c63 | 0 .../udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab | 0 .../udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 | 0 .../udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 | 0 .../udf_size-1-3608160636eaa7e053171bdcefc0b1a8 | 0 .../udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 | 0 .../udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd | 0 .../udf_smallint-0-f28e857ef74c967303855c21dc60c042 | 0 .../udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 | 0 ...df_sort_array-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...df_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 | 0 ...df_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 | 0 ...df_sort_array-3-55c4cdaf8438b06675d60848d68f35de | 0 ...df_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 | 0 ...df_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e | 0 ...df_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 | 0 ...df_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 | 0 ...df_sort_array-8-d79f0084177230a7a2845c4791c22d25 | 0 ...df_sort_array-9-45ef2679e195a269878527d5f264488a | 0 .../udf_space-0-91e879c9f01d90eab7bf12fcef256010 | 0 .../udf_space-1-e4eaf5e96807e122548cb43be9a26754 | 0 .../udf_space-2-a23a06eef37709e8587647a74bbfa7e8 | 0 .../udf_space-3-59903e27d8188d6209e007ff643d5956 | 0 .../udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 | 0 .../udf_split-0-7accac7fc71ba74d61c01a69d3978338 | 0 .../udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 | 0 .../udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 | 0 .../udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 | 0 .../udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 | 0 .../udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 | 0 .../udf_std-0-e3613484de2b3fa707995720ec3f8a5b | 0 .../udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 | 0 .../udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 | 0 .../udf_stddev-1-18e1d598820013453fad45852e1a303d | 0 ...df_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 | 0 ...df_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 | 0 ...df_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 | 0 ...f_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 | 0 ...df_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 | 0 ...f_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 | 0 .../udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 | 0 .../udf_string-1-53b00551846b7f8bb27874b3a466e68d | 0 .../udf_substr-0-20fb50d79b45264548b953e37d837fcd | 0 .../udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 | 0 .../udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 | 0 .../udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 | 0 .../udf_substr-4-f7933c0bb28e9a815555edfa3764524a | 0 .../udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 | 0 .../udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 | 0 .../udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 | 0 ...udf_substring-0-8297700b238f417dea2bd60ba72a6ece | 0 ...udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 | 0 .../udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 | 0 .../udf_subtract-1-b90eec030fee9cbd177f9615b782d722 | 0 .../udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 | 0 .../udf_sum-1-ddae1a511d3371122ab79918be5b495b | 0 .../udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 | 0 .../udf_sum-3-ddae1a511d3371122ab79918be5b495b | 0 .../udf_tan-0-c21aa640b4edabf6613dd705d029c878 | 0 .../udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 0 .../udf_tan-2-27a29c68f846824990e9e443ac95da85 | 0 .../udf_tan-3-77bedd6c76bdc33428d13a894f468a97 | 0 .../udf_tan-4-c21aa640b4edabf6613dd705d029c878 | 0 .../udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 0 .../udf_tan-6-27a29c68f846824990e9e443ac95da85 | 0 .../udf_tan-7-77bedd6c76bdc33428d13a894f468a97 | 0 ...df_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad | 0 ...f_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 | 0 .../udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d | 0 .../udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 | 0 ...df_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 | 0 ...df_to_boolean-1-cbead694a25ec357d69fd008776e19c9 | 0 ...df_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 | 0 ...df_to_boolean-3-266b9601a9154438ab95550fcd36494c | 0 ...df_to_boolean-4-2602c9a6c910ec7fdd439212c648333d | 0 ...df_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad | 0 ...df_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f | 0 ...df_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a | 0 .../udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 | 0 .../udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 | 0 .../udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc | 0 .../udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b | 0 .../udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 | 0 .../udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 | 0 .../udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 | 0 .../udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 | 0 .../udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 | 0 .../udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 | 0 .../udf_to_date-1-da3c817bc5f4458078c6199390ac915e | 0 ...udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 | 0 ...udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a | 0 ...udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe | 0 ...udf_to_double-3-ab23099412d24154ff369d8bd6bde89f | 0 ...udf_to_double-4-293a639a2b61a11da6ca798c04624f68 | 0 ...udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 | 0 ...udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 | 0 ...udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 | 0 ...udf_to_double-8-df51146f6ef960c77cd1722191e4b982 | 0 .../udf_to_float-0-7646eca02448547eedf84a81bf42be89 | 0 .../udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c | 0 .../udf_to_float-2-39a67183b6d2a4da005baed849c5e971 | 0 .../udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce | 0 .../udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 | 0 .../udf_to_float-5-75f364708c01b5e31f988f19e52b2201 | 0 .../udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 | 0 .../udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 | 0 .../udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c | 0 .../udf_to_long-0-ebe447e8fb03de8844951250fe50320a | 0 .../udf_to_long-1-61e6679e5a37889bc596590bde0228f0 | 0 .../udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 | 0 .../udf_to_long-3-cb3318ba365833316645e1b9890f4613 | 0 .../udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 | 0 .../udf_to_long-5-a0d382d243a226f8d4381970b6831c3d | 0 .../udf_to_long-6-47ab11eae68329cc80232fc4089479f0 | 0 .../udf_to_long-7-b87c0cabb166f33984cc1b191694918e | 0 .../udf_to_long-8-52a24d7040db321a842f9201d245ee9 | 0 .../udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a | 0 .../udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b | 0 .../udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 | 0 .../udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce | 0 .../udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 | 0 .../udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 | 0 .../udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 | 0 .../udf_to_short-7-827f3181c216fd2e990637c9a091bf0d | 0 .../udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 | 0 .../udf_to_string-0-a032eb8f819689a374852c20336d5cc | 0 ...udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 | 0 ...udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 | 0 ...udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 | 0 ...udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 | 0 .../udf_to_string-5-1040b37847d20ef29d545934316303 | 0 ...udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 | 0 ...udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 | 0 ...udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea | 0 ...udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b | 0 ...udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 | 0 ...udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 | 0 ...df_translate-10-2f9daada8878081cb8954880ad5a65c4 | 0 ...df_translate-11-76b7a339d5c62808b9f4f78816d4c55b | 0 ...df_translate-12-a5b3e5fadeec1e03371160879f060b05 | 0 ...udf_translate-2-42aba80bf1913dd7c64545831f476c58 | 0 ...udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a | 0 ...udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba | 0 ...udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 | 0 ...udf_translate-6-55122cc5ea4f49e737fee58945f3f21b | 0 ...udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 | 0 ...udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 | 0 ...udf_translate-9-ee69663d7662760973b72785595be2b1 | 0 .../udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f | 0 .../udf_trim-1-e23715e112959e6840b6feed2ecf38a7 | 0 .../udf_ucase-0-8f8c18102eb02df524106be5ea49f23d | 0 .../udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f | 0 .../udf_unhex-0-11eb3cc5216d5446f4165007203acc47 | 0 .../udf_unhex-1-a660886085b8651852b9b77934848ae4 | 0 .../udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 | 0 .../udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 | 0 .../udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c | 0 .../udf_union-1-f6f01250718087029165e23badc02cd6 | 0 .../udf_union-2-6af20858e3209d7cf37f736805ec5182 | 0 .../udf_union-3-705d165fec6761744dd19b142c566d61 | 0 .../udf_upper-0-47dc226b0435f668df20fe0e84293ead | 0 .../udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d | 0 .../udf_var_pop-0-3187e740690ccc1988a19fea4202a6de | 0 .../udf_var_pop-1-fd25e5226312bf54d918858511814766 | 0 .../udf_var_samp-0-b918928871d1b7f944315558c230c229 | 0 .../udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b | 0 .../udf_var_samp-2-b918928871d1b7f944315558c230c229 | 0 .../udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b | 0 .../udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 | 0 .../udf_variance-1-c1856abae908b05bfd6183189b4fd06a | 0 .../udf_variance-2-3187e740690ccc1988a19fea4202a6de | 0 .../udf_variance-3-fd25e5226312bf54d918858511814766 | 0 .../udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 | 0 .../udf_variance-5-c1856abae908b05bfd6183189b4fd06a | 0 .../udf_variance-6-3187e740690ccc1988a19fea4202a6de | 0 .../udf_variance-7-fd25e5226312bf54d918858511814766 | 0 ...df_weekofyear-0-d6b4490b549a358be375511e39627dc2 | 0 ...df_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd | 0 ...df_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 | 0 .../udf_when-0-88b97c6722176393e9b3d089559d2d11 | 0 .../udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 | 0 .../udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 | 0 .../udf_when-3-e63043e8d6ecf300c1fcf8654176896f | 0 .../udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 | 0 .../udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 | 0 .../udf_xpath-2-6b0a9d3874868d88d54ae133c978753d | 0 .../udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea | 0 .../udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a | 0 .../udf_xpath-5-7395e1cd3b543316a753978f556975e0 | 0 .../udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f | 0 ...xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 | 0 ...xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 | 0 ...xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b | 0 ...xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 | 0 ...xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 | 0 ...xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 | 0 ...xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f | 0 ...xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 | 0 ..._xpath_double-0-39199612969071d58b24034a2d17ca67 | 0 ..._xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 | 0 ...xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 | 0 ...xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 | 0 ..._xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 | 0 ..._xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 | 0 ..._xpath_double-4-93f47057c68385cff3b6f5c42307590c | 0 ..._xpath_double-5-c811a2353f5baf585da8654acd13b0e5 | 0 ..._xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 | 0 ..._xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c | 0 ..._xpath_double-8-547e750f5e401511db56283e66d1231d | 0 ..._xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e | 0 ...f_xpath_float-0-7483dafee0dc7334eecabba31977e791 | 0 ...f_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a | 0 ...f_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 | 0 ...f_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 | 0 ...f_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 | 0 ...f_xpath_float-5-410760f9560157587fbba7a677e12b9f | 0 ...f_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf | 0 ...f_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa | 0 ...f_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 | 0 ...f_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 | 0 ...udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 | 0 ...udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 | 0 ...udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba | 0 ...udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba | 0 ...udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 | 0 ...udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 | 0 ...udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a | 0 ...udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 | 0 ...udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 | 0 ...udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 | 0 ...df_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b | 0 ...df_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 | 0 ...df_xpath_long-2-d697d943b1f7e7762d804064d11b905b | 0 ...df_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d | 0 ...udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 | 0 ...df_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d | 0 ...df_xpath_long-6-f2460325cf46c830631d8bc32565c787 | 0 ...df_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 | 0 ...df_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 | 0 ...df_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 | 0 ...f_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 | 0 ...df_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 | 0 ...f_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 | 0 ...f_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 | 0 ...f_xpath_short-4-8a300079521fefbe0d2f943851c1c53c | 0 ...f_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e | 0 ...f_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 | 0 ...f_xpath_short-7-e24ee123f331429c22de0a06054d0d5d | 0 ...f_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 | 0 ...f_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e | 0 ..._xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 | 0 ..._xpath_string-1-110b583cde6cd23c486d8223c444cbe9 | 0 ..._xpath_string-2-a147b4eaa40b03355f666ea660cbff1f | 0 ..._xpath_string-3-a62072b86a6044d5b97911d662899b5a | 0 ..._xpath_string-4-152b0424ec68120f4ef15269eac0528a | 0 ..._xpath_string-5-2ab527e587583138ca764f5518a88c14 | 0 ..._xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 | 0 ..._xpath_string-7-b5b211e4abd14990c28007c6638cb44f | 0 ..._xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc | 0 ..._xpath_string-9-647aca756f43e791dd5497e1b4c6af89 | 0 .../udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a | 0 .../udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 | 0 .../udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 | 0 ...code_notation-0-8ee564d593fc64e0ad8a52b5d659f286 | 0 ...code_notation-1-3de206f543c9e1525c54547f076b99c3 | 0 ...code_notation-2-9a9c2395773e362f0db32d5603673291 | 0 ...code_notation-3-4ac0de021c370cc6b8e753e00f96682e | 0 ...code_notation-4-3de206f543c9e1525c54547f076b99c3 | 0 ...code_notation-5-9a9c2395773e362f0db32d5603673291 | 0 ...code_notation-6-b52052b427ea699f7bc7bee7e32d1de9 | 0 ...code_notation-7-3de206f543c9e1525c54547f076b99c3 | 0 ...code_notation-8-9a9c2395773e362f0db32d5603673291 | 0 .../union10-0-863233ccd616401efb4bf83c4b9e3a52 | 0 .../union10-1-c5b4e04f745a28463e64aeeec6d4b2af | 0 .../union10-2-60ee9eae687170317ff91dafe6b799bf | 0 .../union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 | 0 .../union10-4-7f83822f19aa9b973198fe4c42c66856 | 0 .../union11-0-863233ccd616401efb4bf83c4b9e3a52 | 0 .../union11-1-a6f37db310de2edce6696c25a4fe96e3 | 0 .../union11-2-cd756f39e22e121bdbd51400662aa47f | 0 .../union12-0-863233ccd616401efb4bf83c4b9e3a52 | 0 .../union12-1-9d2793d1cfd2645ac7f373a0a127e599 | 0 .../union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 | 0 .../union13-0-6970d1e2405f3769a28a8804887ac657 | 0 .../union13-1-534d0853c5fc094404f65ca4631c1c20 | 0 .../union14-0-863233ccd616401efb4bf83c4b9e3a52 | 0 .../union14-1-11603fb6225423979af6c062cfa9389b | 0 .../union14-2-8e01b2f4a18ad41a622e0aadbe680398 | 0 .../union15-0-863233ccd616401efb4bf83c4b9e3a52 | 0 .../union15-1-caf8a4bf47a5f5dfd27388fe00686c56 | 0 .../union15-2-a89acfb4bbc044c483b94e28152a41e0 | 0 .../union16-0-7d7f87e388835405f8c7baba969e04f5 | 0 .../union16-1-9f76074598f9b55d8afbb5659737a382 | 0 .../union17-0-383b0c63b3fe31509d23612cb6ae88a | 0 .../union17-1-9491c271b7e2c351ddcf126f73679892 | 0 .../union17-2-e6fb104913b9da8193167ee263993dd1 | 0 .../union17-3-eef8248caa65e21a7c2956aa488297a0 | 0 .../union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 0 .../union17-5-404392d6faff5db5f36b4aa87ac8e8c9 | 0 .../union18-0-383b0c63b3fe31509d23612cb6ae88a | 0 .../union18-1-9491c271b7e2c351ddcf126f73679892 | 0 .../union18-2-584e61e1599fe449cc998a3332dcb7e5 | 0 .../union18-3-b86b4936bb620a8f5d929943f4aa75f2 | 0 .../union18-4-1799ebb147238db6032fd6fe2fd36878 | 0 .../union18-5-b12dcddfa4f02a14318f6564947c98a0 | 0 .../union19-0-383b0c63b3fe31509d23612cb6ae88a | 0 .../union19-1-9491c271b7e2c351ddcf126f73679892 | 0 .../union19-2-b6161b64e57a2502c79662866782cbdf | 0 .../union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 | 0 .../union19-4-1799ebb147238db6032fd6fe2fd36878 | 0 .../union19-5-b12dcddfa4f02a14318f6564947c98a0 | 0 .../union2-0-58813daf3d1af57b17518b606f8345b2 | 0 .../union2-1-90d739774cb96e7d0d96513c1c9968b4 | 0 .../union20-0-a7f64187712936e741d5c33b2f5dfa6d | 0 .../union20-1-968e353589f1fddb914242beb25be94c | 0 .../union21-0-ecfd22e2a24ed9f113229c80a2aaee9c | 0 .../union21-1-fb1497f4c21bf7d28162f27d50320d13 | 0 .../union22-0-4bd6583b3635f1db95765565518bda8d | 0 .../union22-1-9037e5e0cb2d64e4efe6280dc03396f7 | 0 .../union22-10-90e9c4388340428a1f68868e9322c400 | 0 .../union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 | 0 .../union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 | 0 .../union22-4-dc129f70e75cd575ce8c0de288884523 | 0 .../union22-5-24ca942f094b14b92086305cc125e833 | 0 .../union22-6-44d382ce6848d3f0b900b0808747d8e9 | 0 .../union22-7-25fc734982956a164adde6bb1d4d8751 | 0 .../union22-8-c78e8af8e8e327c58a74f7cb34a1c912 | 0 .../union22-9-f4414e4636a16596d04fcc433d1119b6 | 0 .../union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 | 0 .../union23-1-7830963417e3535034962e2597970ddd | 0 .../union24-0-5659f2e36b79fa312aaf6e233dc575c9 | 0 .../union24-1-92aaed1a225be0e1b3d4556680a77b8c | 0 .../union24-10-ae53b6e1d433f000da3fbe130222b89 | 0 .../union24-11-a41e75063b4cac0dbaf99b4aabc3201a | 0 .../union24-2-4afc74338258d50ae6ecdb0589bd2a38 | 0 .../union24-3-d3370b90a3ace4b4d1cefd1ffc79906f | 0 .../union24-4-dc129f70e75cd575ce8c0de288884523 | 0 .../union24-5-a572a07cd60fd4607ddd7613db8a64ab | 0 .../union24-6-83d9274a535ebebe0170ce8f16062ef4 | 0 .../union24-7-afb3649d2de5113691a37d0c77b28b31 | 0 .../union24-8-b6d9a41e2b537869e977cfb969b51edb | 0 .../union24-9-db8ce0526e33b73f5589e3e34aca0198 | 0 .../union26-0-a0d1a7f8eb3fba9455ab517aab14552d | 0 .../union26-1-4702087f160230463e6114d6534d47e4 | 0 .../union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 | 0 .../union26-3-4702087f160230463e6114d6534d47e4 | 0 .../union27-0-7f319eff3c0237b6c06cb704dcde9195 | 0 .../union27-1-fb2ddef0c7d9b05d7fb26442599d354f | 0 .../union27-2-10d1593b6663231ff7de938024ce6bb6 | 0 .../union27-3-ab84df3813ff23be99f148449610e530 | 0 .../union28-0-6bccc08f175712c17aa66cebbf5b7e36 | 0 .../union28-1-f1bd585c115dbabf655d8dc38cd4e026 | 0 .../union28-2-9985bbc70b8dd624fac261eafb10a78f | 0 .../union28-3-b1d75ba0d33a452619e41f70e69616e9 | 0 .../union29-0-6bccc08f175712c17aa66cebbf5b7e36 | 0 .../union29-1-a40036f756a306f4226634b6ab67c72c | 0 .../union29-2-c7fab64fc276c13149e1e68c0deb6594 | 0 .../union29-3-b1d75ba0d33a452619e41f70e69616e9 | 0 .../union3-0-6a8a35102de1b0b88c6721a704eb174d | 0 .../union3-1-ae80732ae5db530cc4e355d5d4724457 | 0 .../union3-2-2a1dcd937f117f1955a169592b96d5f9 | 0 .../union3-3-8fc63f8edb2969a63cd4485f1867ba97 | 0 .../union30-0-6bccc08f175712c17aa66cebbf5b7e36 | 0 .../union30-1-1aff6fba829bae7d20266cf91383ef78 | 0 .../union30-2-252f12f7532ca67132bfc62063c58430 | 0 .../union30-3-b1d75ba0d33a452619e41f70e69616e9 | 0 .../union31-0-ca16024e6f5399b1d035f5b9fd665163 | 0 .../union31-1-b12aa87aaf64b573ce0be7013117651b | 0 .../union31-10-ca695e6973bc98b7f06c998b964ae339 | 0 .../union31-11-70dda95c1b749e4104cc2c767cd7fec0 | 0 .../union31-12-de2477a134ab37bb22d879028becf2e5 | 0 .../union31-13-22b1078764cb1c9b2fb95358c968e987 | 0 .../union31-14-c36a1d8de2713f722ec42bc4686d6125 | 0 .../union31-15-5df6435aed6e0a6a6853480a027b911e | 0 .../union31-16-ca16024e6f5399b1d035f5b9fd665163 | 0 .../union31-17-b12aa87aaf64b573ce0be7013117651b | 0 .../union31-18-1522ec86f2e6735cf809a52c1f6f7883 | 0 .../union31-19-1e9320795b3e4acd7e6d89a8c9ba897e | 0 .../union31-2-1522ec86f2e6735cf809a52c1f6f7883 | 0 .../union31-20-dceb6d9a03096d68d090557692ed6899 | 0 .../union31-21-bda2e62d0ecd30200b2359df420da574 | 0 .../union31-22-fa1c85506afbfa956251cd1253f5d79a | 0 .../union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df | 0 .../union31-24-df38c8164af7cc164c728b8178da72c5 | 0 .../union31-25-1485e295a99908e1862eae397b814045 | 0 .../union31-3-35d1dd67e8c3d76c77baaeae0760ddfc | 0 .../union31-4-77653483e5ce18437c9f85d3b297335d | 0 .../union31-5-4651987fca60c1c1a59b8b28713ea737 | 0 .../union31-6-69c2b75a25f884a72d89e43e0186f5a6 | 0 .../union31-7-7b021f95774705e4d7ff7049ee0e0b74 | 0 .../union31-8-ba92b89786ffaecd74a740705e0fa0cb | 0 .../union31-9-56dfdb30edd8a687f9aa9cad29b42760 | 0 .../union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e | 0 .../union32-1-e6c80e7d6171ae5fc428506e57dc8753 | 0 .../union32-10-44a9b373ee7d43a4ef2bc4b8a708601b | 0 .../union32-11-40d51bc9958b27c36ef647f0598fdee5 | 0 .../union32-2-8e39fef33c859ef83912d0fcda319218 | 0 .../union32-3-d31e252450077ac54f4cb18a9ad95a84 | 0 .../union32-4-79787e084ca15d479cee3a7e1ed2281e | 0 .../union32-5-51c997d0a1103b60764bbb8316a38746 | 0 .../union32-6-96fa13d8790bbfa1d6109b7cbf890d1b | 0 .../union32-7-f936440d63f4e1027dda4de605660633 | 0 .../union32-8-f42d83f502a7b8d3a36331e0f5621cfb | 0 .../union32-9-74fb695786df4c024288ae23ac8c00db | 0 .../union33-0-c67a488530dc7e20a9e7acf02c14380f | 0 .../union33-1-1df7c476abb48234c839878414720873 | 0 .../union33-2-3987150ad66482f5186826efcfb117f7 | 0 .../union33-3-bfe75625806b414a5098af487b91046a | 0 .../union33-4-1d22c1e85a4bfa338aff713d45440877 | 0 .../union33-5-13c01085e0a2c46e745985fa11bedf34 | 0 .../union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 | 0 .../union33-7-1d22c1e85a4bfa338aff713d45440877 | 0 .../union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 | 0 .../union34-1-320c2fe802fab386957039aaff6399f1 | 0 .../union34-10-da2b79118c21ac45ce85001fa61b0043 | 0 .../union34-2-5707c9fd2cfd049426383e5e5dc80d7 | 0 .../union34-3-c179e14d840249d6953cb552279822f1 | 0 .../union34-4-101829a66cab2efd31dcb0d86e302956 | 0 .../union34-5-24ca942f094b14b92086305cc125e833 | 0 .../union34-6-ff0312eeb487fc393a06880ef5bb286f | 0 .../union34-7-da2b79118c21ac45ce85001fa61b0043 | 0 .../union34-8-b1e2ade89ae898650f0be4f796d8947b | 0 .../union34-9-d82ffe361a119651bbf41a149b6eba91 | 0 .../union4-0-863233ccd616401efb4bf83c4b9e3a52 | 0 .../union4-1-88deb3987655dc41989f39c0827cec74 | 0 .../union4-2-41739c4bd273f0cfa94b347c36a71f38 | 0 .../union4-3-2fb47e6ec0a780da5bda97752e573c0b | 0 .../union4-4-7f83822f19aa9b973198fe4c42c66856 | 0 .../union5-0-863233ccd616401efb4bf83c4b9e3a52 | 0 .../union5-1-54c6369ded20ba12138e6c53a157dae8 | 0 .../union5-2-2c19c8d564b010eeb42deee63d66a292 | 0 .../union6-0-863233ccd616401efb4bf83c4b9e3a52 | 0 .../union6-1-7d631b7dbfe39ccaf995e16186b3e94d | 0 .../union6-2-31cbd867e4b88605d230c542af64b7c4 | 0 .../union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 | 0 .../union6-4-a03959cc5aaa8f6521a73e6dae04cd15 | 0 .../union7-0-863233ccd616401efb4bf83c4b9e3a52 | 0 .../union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 | 0 .../union7-2-55d6e503a281acf3289a7874c0fba3f5 | 0 .../union8-0-d0a83a1cedcbd1d2158acf7eb938d18e | 0 .../union8-1-1b422e4c1c8b97775518f760b995c771 | 0 .../union9-0-fad6a053ab42c3cad89052e88d819e83 | 0 .../union9-1-a77ee9f723b3b17a3a02164c5d0000c1 | 0 .../union_date-0-a0bade1c77338d4f72962389a1f5bea2 | 0 .../union_date-1-21306adbd8be8ad75174ad9d3e42b73c | 0 .../union_date-2-b386e4435da29e7e240893639c6dd9c4 | 0 .../union_date-3-f0f678c9032603ca0ae729a31c8c9d38 | 0 .../union_date-4-d812f7feef3b6857aeca9007f0af44c | 0 .../union_date-5-b54839e0200bec94aa751fec8c5dbd3d | 0 ...n_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 | 0 ...n_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 | 0 ...n_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 | 0 ...n_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 | 0 .../union_null-0-27e98c4939abf1ad4445b4e715b0262a | 0 .../union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b | 0 .../union_ppr-1-4d073ff9d6978f3c3be447920ab8daed | 0 ...union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...ion_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...nion_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...ion_remove_1-12-43d53504df013e6b35f81811138a167a | 0 ...nion_remove_1-2-cafed8ca348b243372b9114910be1557 | 0 ...nion_remove_1-3-dc129f70e75cd575ce8c0de288884523 | 0 ...nion_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...nion_remove_1-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 | 0 ...nion_remove_1-7-48f70528347f5201f387d28dae37a14a | 0 ...nion_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...nion_remove_1-9-19865a08066d80cb069ae6312c465ee6 | 0 ...nion_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 | 0 ...on_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 | 0 ...ion_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_10-13-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_10-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_10-3-b12e5c70d6d29757471b900b6160fa8a | 0 ...ion_remove_10-4-593999fae618b6b38322bc9ae4e0c027 | 0 ...ion_remove_10-5-6f53d5613262d393d82d159ec5dc16dc | 0 ...ion_remove_10-6-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_10-7-a7ba0436265932086d2a2e228356971 | 0 ...ion_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...ion_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...nion_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_11-10-9788525d32af3dccdefb459669f886c1 | 0 ...on_remove_11-11-42ecec4117d372551310ad1d85a9c09a | 0 ...ion_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_11-13-43d53504df013e6b35f81811138a167a | 0 ...on_remove_11-14-e409e7032445097ace016b1876d95b3e | 0 ...ion_remove_11-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_11-3-b12e5c70d6d29757471b900b6160fa8a | 0 ...ion_remove_11-4-593999fae618b6b38322bc9ae4e0c027 | 0 ...ion_remove_11-5-6f53d5613262d393d82d159ec5dc16dc | 0 ...ion_remove_11-6-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 | 0 ...ion_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...ion_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...nion_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...on_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c | 0 ...on_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 | 0 ...ion_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_12-14-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_12-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_12-3-24ca942f094b14b92086305cc125e833 | 0 ...ion_remove_12-4-b12e5c70d6d29757471b900b6160fa8a | 0 ...ion_remove_12-5-593999fae618b6b38322bc9ae4e0c027 | 0 ...ion_remove_12-6-6f53d5613262d393d82d159ec5dc16dc | 0 ...ion_remove_12-7-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_12-8-9dd030d38eece4630dec5951fc8a0622 | 0 ...ion_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...nion_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...on_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 | 0 ...on_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af | 0 ...ion_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_13-14-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_13-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_13-3-24ca942f094b14b92086305cc125e833 | 0 ...ion_remove_13-4-b12e5c70d6d29757471b900b6160fa8a | 0 ...ion_remove_13-5-593999fae618b6b38322bc9ae4e0c027 | 0 ...ion_remove_13-6-6f53d5613262d393d82d159ec5dc16dc | 0 ...ion_remove_13-7-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb | 0 ...ion_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...nion_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...on_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c | 0 ...on_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 | 0 ...ion_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_14-14-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_14-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_14-3-b12e5c70d6d29757471b900b6160fa8a | 0 ...ion_remove_14-4-593999fae618b6b38322bc9ae4e0c027 | 0 ...ion_remove_14-5-24ca942f094b14b92086305cc125e833 | 0 ...ion_remove_14-6-6f53d5613262d393d82d159ec5dc16dc | 0 ...ion_remove_14-7-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_14-8-ed33d620523b2634285698a83f433b6d | 0 ...ion_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...nion_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...on_remove_15-11-93b7341b523213ab6e58169459bc6818 | 0 ...on_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 | 0 ...ion_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_15-14-37f9f7bc2d7456046a9f967347337e47 | 0 ...on_remove_15-15-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_15-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_15-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ion_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ion_remove_15-5-16367c381d4b189b3640c92511244bfe | 0 ...ion_remove_15-6-a4fb8359a2179ec70777aad6366071b7 | 0 ...ion_remove_15-7-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_15-8-552c4eba867e7408fc8652ff0a19170d | 0 ...ion_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef | 0 ...nion_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef | 0 ...on_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...on_remove_16-12-93b7341b523213ab6e58169459bc6818 | 0 ...on_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 | 0 ...ion_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_16-15-37f9f7bc2d7456046a9f967347337e47 | 0 ...on_remove_16-16-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_16-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_16-3-b12e5c70d6d29757471b900b6160fa8a | 0 ...ion_remove_16-4-593999fae618b6b38322bc9ae4e0c027 | 0 ...ion_remove_16-5-6f53d5613262d393d82d159ec5dc16dc | 0 ...ion_remove_16-6-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_16-7-16367c381d4b189b3640c92511244bfe | 0 ...ion_remove_16-8-a4fb8359a2179ec70777aad6366071b7 | 0 ...ion_remove_16-9-ec47ba0fc527a4a04d452a009d59147a | 0 ...nion_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...on_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 | 0 ...on_remove_17-12-c91289e16ad403babfc91c093ac9b86d | 0 ...ion_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_17-14-37f9f7bc2d7456046a9f967347337e47 | 0 ...on_remove_17-15-43d53504df013e6b35f81811138a167a | 0 ...on_remove_17-16-626a252f75285872c1d72706f7f972c6 | 0 ...on_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a | 0 ...ion_remove_17-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_17-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ion_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ion_remove_17-5-16367c381d4b189b3640c92511244bfe | 0 ...ion_remove_17-6-a4fb8359a2179ec70777aad6366071b7 | 0 ...ion_remove_17-7-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_17-8-67e57f56d1106a57329bf75c491e3c8b | 0 ...ion_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef | 0 ...nion_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...on_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad | 0 ...on_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 | 0 ...ion_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_18-14-37f9f7bc2d7456046a9f967347337e47 | 0 ...on_remove_18-15-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_18-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_18-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ion_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ion_remove_18-5-16367c381d4b189b3640c92511244bfe | 0 ...ion_remove_18-6-a4fb8359a2179ec70777aad6366071b7 | 0 ...ion_remove_18-7-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 | 0 ...ion_remove_18-9-ea467d0fee062a23c720cf47eacfef08 | 0 ...nion_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_19-10-8d1e5af37e7992708bf15ab7d887405b | 0 ...ion_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_19-12-68e15b1729669c4cb2291dcabfea4387 | 0 ...on_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 | 0 ...on_remove_19-14-eed866a1ad3106a306322519f4bb52f2 | 0 ...on_remove_19-15-68e15b1729669c4cb2291dcabfea4387 | 0 ...on_remove_19-16-471f8e794fd712dce2e40334b383e08e | 0 ...on_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 | 0 ...on_remove_19-18-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_19-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_19-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ion_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ion_remove_19-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_19-6-a6c043a89a9c3456af8ee065cb17239 | 0 ...ion_remove_19-7-48f70528347f5201f387d28dae37a14a | 0 ...ion_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ion_remove_19-9-8a08edd1aa63fd3b051da82246793259 | 0 ...union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...ion_remove_2-10-2309570010c3e679b884c100de57d002 | 0 ...nion_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...ion_remove_2-12-43d53504df013e6b35f81811138a167a | 0 ...nion_remove_2-2-cafed8ca348b243372b9114910be1557 | 0 ...nion_remove_2-3-dc129f70e75cd575ce8c0de288884523 | 0 ...nion_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...nion_remove_2-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e | 0 ...nion_remove_2-7-48f70528347f5201f387d28dae37a14a | 0 ...nion_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...nion_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...nion_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_20-10-89c57c91facbf54299e08955e3783ea6 | 0 ...ion_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_20-12-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_20-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_20-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ion_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ion_remove_20-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_20-6-a73143117ffec1225f6d492e5aa577e | 0 ...ion_remove_20-7-82f81adc097c247475fd29076e0cb85f | 0 ...ion_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ion_remove_20-9-683949ae07de12da0b7e7ba7f4450daa | 0 ...nion_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_21-10-e19469a1b70be25caaf670fe68f0a747 | 0 ...ion_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_21-12-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_21-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_21-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ion_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ion_remove_21-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f | 0 ...ion_remove_21-7-5716c408db679fb88352eaceb1703bd7 | 0 ...ion_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ion_remove_21-9-deadbce171926623b0586587fbbcd144 | 0 ...nion_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_22-10-403471d96c56b565272d2e4c4926d240 | 0 ...ion_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_22-12-68e15b1729669c4cb2291dcabfea4387 | 0 ...on_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd | 0 ...on_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af | 0 ...on_remove_22-15-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_22-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_22-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ion_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ion_remove_22-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 | 0 ...ion_remove_22-7-46da090f5a2c73b175207cf63ff46653 | 0 ...ion_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ion_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb | 0 ...nion_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 | 0 ...ion_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_23-12-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_23-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_23-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ion_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ion_remove_23-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 | 0 ...ion_remove_23-7-48f70528347f5201f387d28dae37a14a | 0 ...ion_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ion_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 | 0 ...nion_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...ion_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...on_remove_24-10-6d89089b1eead05510dbccad5fcc4805 | 0 ...ion_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...on_remove_24-12-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_24-2-cafed8ca348b243372b9114910be1557 | 0 ...ion_remove_24-3-dc129f70e75cd575ce8c0de288884523 | 0 ...ion_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...ion_remove_24-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...ion_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 | 0 ...ion_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e | 0 ...ion_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ion_remove_24-9-4c0550cc9c28de25993c1f98de39168f | 0 ...union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...ion_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 | 0 ...nion_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...ion_remove_3-12-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_3-13-e409e7032445097ace016b1876d95b3e | 0 ...nion_remove_3-2-cafed8ca348b243372b9114910be1557 | 0 ...nion_remove_3-3-dc129f70e75cd575ce8c0de288884523 | 0 ...nion_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...nion_remove_3-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_3-6-c36130e41df18093eee24b45fc0846f0 | 0 ...nion_remove_3-7-48f70528347f5201f387d28dae37a14a | 0 ...nion_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...nion_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 | 0 ...union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...ion_remove_4-10-19865a08066d80cb069ae6312c465ee6 | 0 ...ion_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...nion_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...ion_remove_4-13-43d53504df013e6b35f81811138a167a | 0 ...nion_remove_4-2-cafed8ca348b243372b9114910be1557 | 0 ...nion_remove_4-3-b12e5c70d6d29757471b900b6160fa8a | 0 ...nion_remove_4-4-593999fae618b6b38322bc9ae4e0c027 | 0 ...nion_remove_4-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_4-6-6f53d5613262d393d82d159ec5dc16dc | 0 ...nion_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead | 0 ...nion_remove_4-8-48f70528347f5201f387d28dae37a14a | 0 ...nion_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...ion_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...ion_remove_5-11-2309570010c3e679b884c100de57d002 | 0 ...nion_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...ion_remove_5-13-43d53504df013e6b35f81811138a167a | 0 ...nion_remove_5-2-cafed8ca348b243372b9114910be1557 | 0 ...nion_remove_5-3-b12e5c70d6d29757471b900b6160fa8a | 0 ...nion_remove_5-4-593999fae618b6b38322bc9ae4e0c027 | 0 ...nion_remove_5-5-6f53d5613262d393d82d159ec5dc16dc | 0 ...nion_remove_5-6-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_5-7-4da6ca94af4caf4426e5676a32b70375 | 0 ...nion_remove_5-8-48f70528347f5201f387d28dae37a14a | 0 ...nion_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...ion_remove_6-10-52171e0094ea92aaea8a485aedc88e88 | 0 ...ion_remove_6-11-60361645974d38829674ec6bb5502a62 | 0 ...ion_remove_6-12-43d53504df013e6b35f81811138a167a | 0 ...ion_remove_6-13-e409e7032445097ace016b1876d95b3e | 0 ...ion_remove_6-14-f74b28904e86047150396bc42680ca38 | 0 ...nion_remove_6-2-cafed8ca348b243372b9114910be1557 | 0 ...nion_remove_6-3-dc129f70e75cd575ce8c0de288884523 | 0 ...nion_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...nion_remove_6-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 | 0 ...nion_remove_6-7-48f70528347f5201f387d28dae37a14a | 0 ...nion_remove_6-8-19e054c1050e5dbf703e91a48253ecdf | 0 ...nion_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...ion_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...nion_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...ion_remove_7-12-43d53504df013e6b35f81811138a167a | 0 ...nion_remove_7-2-cafed8ca348b243372b9114910be1557 | 0 ...nion_remove_7-3-dc129f70e75cd575ce8c0de288884523 | 0 ...nion_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...nion_remove_7-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca | 0 ...nion_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...nion_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...nion_remove_7-9-19865a08066d80cb069ae6312c465ee6 | 0 ...union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...ion_remove_8-10-2309570010c3e679b884c100de57d002 | 0 ...nion_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...ion_remove_8-12-43d53504df013e6b35f81811138a167a | 0 ...nion_remove_8-2-cafed8ca348b243372b9114910be1557 | 0 ...nion_remove_8-3-dc129f70e75cd575ce8c0de288884523 | 0 ...nion_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab | 0 ...nion_remove_8-5-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_8-6-895f2432b4be6fcb11641c1d063570ee | 0 ...nion_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...nion_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...nion_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 | 0 ...nion_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c | 0 ...ion_remove_9-10-12cf3335c756f8715a07c5a604f10f64 | 0 ...ion_remove_9-11-4e84cd589eceda668833f8f19ec28e7c | 0 ...nion_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 | 0 ...ion_remove_9-13-43d53504df013e6b35f81811138a167a | 0 ...nion_remove_9-2-cafed8ca348b243372b9114910be1557 | 0 ...nion_remove_9-3-b12e5c70d6d29757471b900b6160fa8a | 0 ...nion_remove_9-4-593999fae618b6b38322bc9ae4e0c027 | 0 ...nion_remove_9-5-6f53d5613262d393d82d159ec5dc16dc | 0 ...nion_remove_9-6-a2a411ad6620aa1ab24550ade336e785 | 0 ...nion_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 | 0 ...nion_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...nion_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 .../union_script-0-ca195b08d98d7f21fe93208499bf0ff6 | 0 .../union_script-1-982cc6d7b98f8fb1055a10ef021e2769 | 0 .../union_view-0-e56367a21517656c18a5bcfeecb4327d | 0 .../union_view-1-c790d4344144460224b0f02be7e137a8 | 0 .../union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 | 0 .../union_view-3-4a746bd076e063017c4d6a2f9218a6e4 | 0 .../union_view-4-d3d75f376f83b694b1dc62c46fa53f4e | 0 .../union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 | 0 .../union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae | 0 .../union_view-7-3b03210f94ec40db9ab02620645014d1 | 0 .../union_view-8-35f48c7d6fa164bb84643657bc9280a8 | 0 .../varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af | 0 .../varchar_2-1-bee4e3a9ded7276960e619e6185bf329 | 0 .../varchar_2-2-779a555944479991520be53f7f247626 | 0 .../varchar_2-3-a8c072e5b13997e9c79484b4af9d78da | 0 .../varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f | 0 .../varchar_2-5-96353c24b5f2b361f72e5c26b4135519 | 0 .../varchar_2-6-2b62789d07b4044bc32190261bf3490f | 0 .../varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af | 0 ...varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e | 0 ...varchar_join1-1-8318072ee410cae36186c9880989a1a5 | 0 ...archar_join1-10-1958143ee083437e87662cadb48c37ce | 0 ...varchar_join1-11-a55f750032663f77066e4979dedea1c | 0 ...archar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e | 0 ...archar_join1-13-8318072ee410cae36186c9880989a1a5 | 0 ...archar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 | 0 ...varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 | 0 ...varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 | 0 ...varchar_join1-4-d5105d36e4d077e0929597124526c532 | 0 ...varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 | 0 .../varchar_join1-6-6bb08c5baa913d9dc506aef65425ef | 0 ...varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 | 0 ...varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 | 0 ...varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 | 0 ..._nested_types-0-e41b0e9db7a9dbd0540e29df341933bc | 0 ..._nested_types-1-87a28b0e800f051525899324a064b878 | 0 ..._nested_types-10-92f25849990eb777ac2711f9dd2e628 | 0 ...nested_types-11-9780781a92fdd992f3cee080a8717238 | 0 ...nested_types-12-e1a4006971319a352280fc52eabf449f | 0 ...nested_types-13-d4574217a243a7d506398a819cd0eab4 | 0 ...nested_types-14-7c33a62195359bc89460ad65f6a5f763 | 0 ..._nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 | 0 ...nested_types-16-93811d146a429c44a2494b3aa4b2caa2 | 0 ...nested_types-17-5724af3985c67a0cb69919c9bbce15dc | 0 ...nested_types-18-d1be2ee3765a80469837ba11eb8685e1 | 0 ...nested_types-19-a54f9a284228e7cdce8c34b8094f2377 | 0 ..._nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb | 0 ...nested_types-20-51a9279006f61097e68a52201daf6710 | 0 ...nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad | 0 ...nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 | 0 ...nested_types-23-c7ea918777b725f2790da4fa00a3efa6 | 0 ...nested_types-24-5a69236334a3a1c4e771206cf547d730 | 0 ...nested_types-25-61b030bb2220a533532d871ae0e08cdb | 0 ...nested_types-26-e41b0e9db7a9dbd0540e29df341933bc | 0 ...nested_types-27-87a28b0e800f051525899324a064b878 | 0 ...nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb | 0 ...nested_types-29-39d44d19c7963a9647fd3293eef670d4 | 0 ..._nested_types-3-39d44d19c7963a9647fd3293eef670d4 | 0 ...nested_types-30-162806477075d97de16dfa6f2576b751 | 0 ...nested_types-31-5b28e1fdb28b365ef419008a4752ed53 | 0 ..._nested_types-4-162806477075d97de16dfa6f2576b751 | 0 ..._nested_types-5-5b28e1fdb28b365ef419008a4752ed53 | 0 ..._nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b | 0 ..._nested_types-7-8c483a7a0e148ca13a292a625f8702f1 | 0 ..._nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 | 0 ..._nested_types-9-f248796769bc7f57cf56a75034a45520 | 0 ...varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 | 0 ...varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d | 0 ...archar_serde-10-496280d9365ee601d9b68a91495d7160 | 0 ...archar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e | 0 ...archar_serde-12-80727f22f1343407ba9200c86ed84280 | 0 ...archar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e | 0 ...archar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 | 0 ...archar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 | 0 ...archar_serde-16-516202183287d734d35d8c7788d22652 | 0 ...archar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a | 0 ...archar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d | 0 ...archar_serde-19-f258df2db09e9cc0e049e85e6ad950ad | 0 ...varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e | 0 ...archar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 | 0 ...archar_serde-21-64536c77ae91bfb6cf7f93f178c6200b | 0 ...varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 | 0 ...archar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 | 0 ...archar_serde-24-770169b632b76cedcd6dfb87fdc46575 | 0 ...archar_serde-25-3470a259b04e126c655531491787e2fc | 0 ...archar_serde-26-55808e190e0ab81dcdc1feb52543ad9f | 0 ...archar_serde-27-8fe526fdd347c25529a383f27ad20566 | 0 ...archar_serde-28-5e4de93349ba89a8344bb799ad60678e | 0 ...archar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c | 0 ...varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 | 0 ...archar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a | 0 ...archar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 | 0 ...archar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb | 0 ...archar_serde-33-620729dc7661c22f1acdc425a7cf0364 | 0 ...archar_serde-34-807ee73e1cd66704dd585f7e0de954d9 | 0 ...archar_serde-35-750a23ebdd77f32b555d4caba7ac5445 | 0 ...archar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d | 0 ...archar_serde-37-87ba3f40293b9c79fcdb3064d964232e | 0 ...archar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 | 0 ...archar_serde-39-8b5f81c27c41807c757522e257a8003f | 0 ...varchar_serde-4-8b5f81c27c41807c757522e257a8003f | 0 ...archar_serde-40-787193a1679a2153c037d3e4c8192bba | 0 ...varchar_serde-5-787193a1679a2153c037d3e4c8192bba | 0 ...varchar_serde-6-122f15d410249b554e12eccdfa46cc43 | 0 ...varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 | 0 ...varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 | 0 ...varchar_serde-9-ec43be632e5f74057aba54c4f562c601 | 0 ...archar_union1-0-433f0ff2f1c8dc92fba226759326067e | 0 ...archar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec | 0 ...rchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 | 0 ...rchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 | 0 ...rchar_union1-12-433f0ff2f1c8dc92fba226759326067e | 0 ...rchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec | 0 ...rchar_union1-14-217ba1b02c693576b652d719970a0048 | 0 ...archar_union1-2-217ba1b02c693576b652d719970a0048 | 0 ...archar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca | 0 ...archar_union1-4-86e7275fc6c5ce64da5031aa7c79821b | 0 ...archar_union1-5-bf77db56df2b2a077d668e47211be708 | 0 ...archar_union1-6-67e66fa14dddc17757436539eca9ef64 | 0 ...archar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 | 0 ...archar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 | 0 ...archar_union1-9-b9455ffec62df97cfec63204ce02a110 | 0 .../golden/view-0-5528e36b3b0f5b14313898cc45f9c23a | 0 .../golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 | 0 .../golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 | 0 .../golden/view-11-dc95343d3e57846485dd543476391376 | 0 .../golden/view-12-371764e1cae31ea0518c03060528d239 | 0 .../golden/view-13-2abce88008f8a19164758ee821aaa8a6 | 0 .../golden/view-14-deb504f4f70fd7db975950c3c47959ee | 0 .../golden/view-15-6f2797b6f81943d3b53b8d247ae8512b | 0 .../golden/view-2-9c529f486fa81a032bfe1253808fca8 | 0 .../golden/view-3-89c80c0e90409d5e304775c9f420915a | 0 .../golden/view-4-4a64d1a623ca71e515796787dbd0f904 | 0 .../golden/view-5-f6d1bce095ecbf1aa484891392fdb07b | 0 .../golden/view-6-47b5043f03a84695b6784682b4402ac8 | 0 .../golden/view-7-8b1bbdadfd1e11af1b56064196164e58 | 0 .../golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 | 0 .../golden/view-9-66c68babac10ae0f645fe8334c5a42d4 | 0 .../view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf | 0 .../view_cast-1-85685a26971fb51ab6e28f9c5e8421bb | 0 .../view_cast-10-a7c865e5180df8d73dba90ede8be0d45 | 0 .../view_cast-2-af2050aa97f0cd930cb1b8ec791007de | 0 .../view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 | 0 .../view_cast-4-d9edb83f4cf847e141d97012314917d4 | 0 .../view_cast-5-6db508ccd85562a9ca7841fb0a08981a | 0 .../view_cast-6-aa5be3380ddc7104258567b406d93cc5 | 0 .../view_cast-7-78ac3800b22682b31708b6a09b402bfb | 0 .../view_cast-8-2cc0c576f0a008abf5bdf3308d500869 | 0 .../view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 | 0 .../view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 | 0 .../view_inputs-1-5af97e73bc3841793440105aae766bbe | 0 ...irtual_column-0-9bacd1908e56d621913a74fe9a583d9d | 0 ...irtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 | 0 ...rtual_column-10-2915b222a58bc994246591e536d388b4 | 0 ...irtual_column-2-1536b365fe0a94b30a62364996529966 | 0 ...irtual_column-3-c66776673c986b59b27e704664935988 | 0 ...irtual_column-4-e47094c927b1091e31c185db0a4e69a6 | 0 ...irtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 | 0 ...irtual_column-6-68d6973677af5c9f1f1f49360c3175e7 | 0 ...irtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 | 0 ...irtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 | 0 ...virtual_column-9-c152da33c1517ecfc336f72b9c133d5 | 0 .../clientnegative/add_partition_with_whitelist.q | 0 .../ql/src/test/queries/clientnegative/addpart1.q | 0 .../alter_concatenate_indexed_table.q | 0 .../test/queries/clientnegative/alter_non_native.q | 0 .../alter_partition_coltype_2columns.q | 0 .../alter_partition_coltype_invalidcolname.q | 0 .../alter_partition_coltype_invalidtype.q | 0 .../clientnegative/alter_partition_invalidspec.q | 0 .../queries/clientnegative/alter_partition_nodrop.q | 0 .../clientnegative/alter_partition_nodrop_table.q | 0 .../clientnegative/alter_partition_offline.q | 0 .../clientnegative/alter_partition_with_whitelist.q | 0 .../clientnegative/alter_rename_partition_failure.q | 0 .../alter_rename_partition_failure2.q | 0 .../alter_rename_partition_failure3.q | 0 .../clientnegative/alter_table_add_partition.q | 0 .../clientnegative/alter_table_wrong_regex.q | 0 .../clientnegative/alter_view_as_select_not_exist.q | 0 .../alter_view_as_select_with_partition.q | 0 .../queries/clientnegative/alter_view_failure.q | 0 .../queries/clientnegative/alter_view_failure2.q | 0 .../queries/clientnegative/alter_view_failure3.q | 0 .../queries/clientnegative/alter_view_failure4.q | 0 .../queries/clientnegative/alter_view_failure5.q | 0 .../queries/clientnegative/alter_view_failure6.q | 0 .../queries/clientnegative/alter_view_failure7.q | 0 .../queries/clientnegative/alter_view_failure8.q | 0 .../queries/clientnegative/alter_view_failure9.q | 0 .../ql/src/test/queries/clientnegative/altern1.q | 0 .../src/test/queries/clientnegative/ambiguous_col.q | 0 .../test/queries/clientnegative/ambiguous_col0.q | 0 .../test/queries/clientnegative/ambiguous_col1.q | 0 .../test/queries/clientnegative/ambiguous_col2.q | 0 .../ql/src/test/queries/clientnegative/analyze.q | 0 .../ql/src/test/queries/clientnegative/analyze1.q | 0 .../src/test/queries/clientnegative/analyze_view.q | 0 .../ql/src/test/queries/clientnegative/archive1.q | 0 .../ql/src/test/queries/clientnegative/archive2.q | 0 .../ql/src/test/queries/clientnegative/archive3.q | 0 .../ql/src/test/queries/clientnegative/archive4.q | 0 .../ql/src/test/queries/clientnegative/archive5.q | 0 .../test/queries/clientnegative/archive_corrupt.q | 0 .../test/queries/clientnegative/archive_insert1.q | 0 .../test/queries/clientnegative/archive_insert2.q | 0 .../test/queries/clientnegative/archive_insert3.q | 0 .../test/queries/clientnegative/archive_insert4.q | 0 .../test/queries/clientnegative/archive_multi1.q | 0 .../test/queries/clientnegative/archive_multi2.q | 0 .../test/queries/clientnegative/archive_multi3.q | 0 .../test/queries/clientnegative/archive_multi4.q | 0 .../test/queries/clientnegative/archive_multi5.q | 0 .../test/queries/clientnegative/archive_multi6.q | 0 .../test/queries/clientnegative/archive_multi7.q | 0 .../test/queries/clientnegative/archive_partspec1.q | 0 .../test/queries/clientnegative/archive_partspec2.q | 0 .../test/queries/clientnegative/archive_partspec3.q | 0 .../test/queries/clientnegative/archive_partspec4.q | 0 .../test/queries/clientnegative/archive_partspec5.q | 0 .../queries/clientnegative/authorization_fail_1.q | 0 .../queries/clientnegative/authorization_fail_2.q | 0 .../queries/clientnegative/authorization_fail_3.q | 0 .../queries/clientnegative/authorization_fail_4.q | 0 .../queries/clientnegative/authorization_fail_5.q | 0 .../queries/clientnegative/authorization_fail_6.q | 0 .../queries/clientnegative/authorization_fail_7.q | 0 .../queries/clientnegative/authorization_part.q | 0 .../ql/src/test/queries/clientnegative/autolocal1.q | 0 .../test/queries/clientnegative/bad_exec_hooks.q | 0 .../src/test/queries/clientnegative/bad_indextype.q | 0 .../test/queries/clientnegative/bad_sample_clause.q | 0 .../clientnegative/bucket_mapjoin_mismatch1.q | 0 .../bucket_mapjoin_wrong_table_metadata_1.q | 0 .../bucket_mapjoin_wrong_table_metadata_2.q | 0 .../queries/clientnegative/cachingprintstream.q | 0 .../clientnegative/cluster_tasklog_retrieval.q | 0 .../queries/clientnegative/clusterbydistributeby.q | 0 .../test/queries/clientnegative/clusterbyorderby.q | 0 .../test/queries/clientnegative/clusterbysortby.q | 0 .../ql/src/test/queries/clientnegative/clustern1.q | 0 .../ql/src/test/queries/clientnegative/clustern2.q | 0 .../ql/src/test/queries/clientnegative/clustern3.q | 0 .../ql/src/test/queries/clientnegative/clustern4.q | 0 .../clientnegative/column_change_skewedcol_type1.q | 0 .../test/queries/clientnegative/column_rename1.q | 0 .../test/queries/clientnegative/column_rename2.q | 0 .../test/queries/clientnegative/column_rename3.q | 0 .../test/queries/clientnegative/column_rename4.q | 0 .../test/queries/clientnegative/column_rename5.q | 0 .../queries/clientnegative/columnstats_partlvl_dp.q | 0 .../columnstats_partlvl_incorrect_num_keys.q | 0 .../columnstats_partlvl_invalid_values.q | 0 .../columnstats_partlvl_multiple_part_clause.q | 0 .../queries/clientnegative/columnstats_tbllvl.q | 0 .../columnstats_tbllvl_complex_type.q | 0 .../columnstats_tbllvl_incorrect_column.q | 0 .../queries/clientnegative/compare_double_bigint.q | 0 .../queries/clientnegative/compare_string_bigint.q | 0 .../clientnegative/create_insert_outputformat.q | 0 .../clientnegative/create_or_replace_view1.q | 0 .../clientnegative/create_or_replace_view2.q | 0 .../clientnegative/create_or_replace_view3.q | 0 .../clientnegative/create_or_replace_view4.q | 0 .../clientnegative/create_or_replace_view5.q | 0 .../clientnegative/create_or_replace_view6.q | 0 .../clientnegative/create_or_replace_view7.q | 0 .../clientnegative/create_or_replace_view8.q | 0 ...create_skewed_table_col_name_value_no_mismatch.q | 0 .../create_skewed_table_dup_col_name.q | 0 .../create_skewed_table_failure_invalid_col_name.q | 0 .../queries/clientnegative/create_table_failure1.q | 0 .../queries/clientnegative/create_table_failure2.q | 0 .../queries/clientnegative/create_table_failure3.q | 0 .../queries/clientnegative/create_table_failure4.q | 0 .../clientnegative/create_table_wrong_regex.q | 0 .../queries/clientnegative/create_udaf_failure.q | 0 .../clientnegative/create_unknown_genericudf.q | 0 .../clientnegative/create_unknown_udf_udaf.q | 0 .../queries/clientnegative/create_view_failure1.q | 0 .../queries/clientnegative/create_view_failure2.q | 0 .../queries/clientnegative/create_view_failure3.q | 0 .../queries/clientnegative/create_view_failure4.q | 0 .../queries/clientnegative/create_view_failure5.q | 0 .../queries/clientnegative/create_view_failure6.q | 0 .../queries/clientnegative/create_view_failure7.q | 0 .../queries/clientnegative/create_view_failure8.q | 0 .../queries/clientnegative/create_view_failure9.q | 0 .../hive/ql/src/test/queries/clientnegative/ctas.q | 0 .../clientnegative/database_create_already_exists.q | 0 .../clientnegative/database_create_invalid_name.q | 0 .../clientnegative/database_drop_does_not_exist.q | 0 .../clientnegative/database_drop_not_empty.q | 0 .../database_drop_not_empty_restrict.q | 0 .../clientnegative/database_switch_does_not_exist.q | 0 .../src/test/queries/clientnegative/date_literal1.q | 0 .../src/test/queries/clientnegative/date_literal2.q | 0 .../src/test/queries/clientnegative/date_literal3.q | 0 .../ql/src/test/queries/clientnegative/ddltime.q | 0 .../test/queries/clientnegative/decimal_precision.q | 0 .../queries/clientnegative/decimal_precision_1.q | 0 .../queries/clientnegative/default_partition_name.q | 0 .../ql/src/test/queries/clientnegative/deletejar.q | 0 .../src/test/queries/clientnegative/desc_failure1.q | 0 .../src/test/queries/clientnegative/desc_failure2.q | 0 .../src/test/queries/clientnegative/desc_failure3.q | 0 .../test/queries/clientnegative/describe_xpath1.q | 0 .../test/queries/clientnegative/describe_xpath2.q | 0 .../test/queries/clientnegative/describe_xpath3.q | 0 .../test/queries/clientnegative/describe_xpath4.q | 0 .../disallow_incompatible_type_change_on1.q | 0 .../disallow_incompatible_type_change_on2.q | 0 .../queries/clientnegative/drop_function_failure.q | 0 .../queries/clientnegative/drop_index_failure.q | 0 .../test/queries/clientnegative/drop_native_udf.q | 0 .../queries/clientnegative/drop_partition_failure.q | 0 .../clientnegative/drop_partition_filter_failure.q | 0 .../clientnegative/drop_partition_filter_failure2.q | 0 .../queries/clientnegative/drop_table_failure1.q | 0 .../queries/clientnegative/drop_table_failure2.q | 0 .../queries/clientnegative/drop_table_failure3.q | 0 .../queries/clientnegative/drop_view_failure1.q | 0 .../queries/clientnegative/drop_view_failure2.q | 0 .../clientnegative/duplicate_alias_in_transform.q | 0 .../duplicate_alias_in_transform_schema.q | 0 .../test/queries/clientnegative/duplicate_insert1.q | 0 .../test/queries/clientnegative/duplicate_insert2.q | 0 .../test/queries/clientnegative/duplicate_insert3.q | 0 .../ql/src/test/queries/clientnegative/dyn_part1.q | 0 .../ql/src/test/queries/clientnegative/dyn_part2.q | 0 .../ql/src/test/queries/clientnegative/dyn_part3.q | 0 .../ql/src/test/queries/clientnegative/dyn_part4.q | 0 .../clientnegative/dyn_part_empty.q.disabled | 0 .../src/test/queries/clientnegative/dyn_part_max.q | 0 .../queries/clientnegative/dyn_part_max_per_node.q | 0 .../dynamic_partitions_with_whitelist.q | 0 .../exchange_partition_neg_incomplete_partition.q | 0 .../exchange_partition_neg_partition_exists.q | 0 .../exchange_partition_neg_partition_exists2.q | 0 .../exchange_partition_neg_partition_exists3.q | 0 .../exchange_partition_neg_partition_missing.q | 0 .../exchange_partition_neg_table_missing.q | 0 .../exchange_partition_neg_table_missing2.q | 0 .../clientnegative/exchange_partition_neg_test.q | 0 .../clientnegative/exim_00_unsupported_schema.q | 0 .../clientnegative/exim_01_nonpart_over_loaded.q | 0 .../clientnegative/exim_02_all_part_over_overlap.q | 0 .../exim_03_nonpart_noncompat_colschema.q | 0 .../exim_04_nonpart_noncompat_colnumber.q | 0 .../exim_05_nonpart_noncompat_coltype.q | 0 .../exim_06_nonpart_noncompat_storage.q | 0 .../clientnegative/exim_07_nonpart_noncompat_ifof.q | 0 .../exim_08_nonpart_noncompat_serde.q | 0 .../exim_09_nonpart_noncompat_serdeparam.q | 0 .../exim_10_nonpart_noncompat_bucketing.q | 0 .../exim_11_nonpart_noncompat_sorting.q | 0 .../clientnegative/exim_12_nonnative_export.q | 0 .../clientnegative/exim_13_nonnative_import.q | 0 .../queries/clientnegative/exim_14_nonpart_part.q | 0 .../queries/clientnegative/exim_15_part_nonpart.q | 0 .../clientnegative/exim_16_part_noncompat_schema.q | 0 .../clientnegative/exim_17_part_spec_underspec.q | 0 .../clientnegative/exim_18_part_spec_missing.q | 0 .../clientnegative/exim_19_external_over_existing.q | 0 .../exim_20_managed_location_over_existing.q | 0 .../clientnegative/exim_21_part_managed_external.q | 0 .../clientnegative/exim_22_export_authfail.q | 0 .../clientnegative/exim_23_import_exist_authfail.q | 0 .../clientnegative/exim_24_import_part_authfail.q | 0 .../exim_25_import_nonexist_authfail.q | 0 .../ql/src/test/queries/clientnegative/external1.q | 0 .../ql/src/test/queries/clientnegative/external2.q | 0 .../queries/clientnegative/fetchtask_ioexception.q | 0 .../queries/clientnegative/fileformat_bad_class.q | 0 .../queries/clientnegative/fileformat_void_input.q | 0 .../queries/clientnegative/fileformat_void_output.q | 0 .../test/queries/clientnegative/fs_default_name1.q | 0 .../test/queries/clientnegative/fs_default_name2.q | 0 .../test/queries/clientnegative/genericFileFormat.q | 0 .../groupby2_map_skew_multi_distinct.q | 0 .../clientnegative/groupby2_multi_distinct.q | 0 .../groupby3_map_skew_multi_distinct.q | 0 .../clientnegative/groupby3_multi_distinct.q | 0 .../src/test/queries/clientnegative/groupby_cube1.q | 0 .../src/test/queries/clientnegative/groupby_cube2.q | 0 .../queries/clientnegative/groupby_grouping_id1.q | 0 .../queries/clientnegative/groupby_grouping_sets1.q | 0 .../queries/clientnegative/groupby_grouping_sets2.q | 0 .../queries/clientnegative/groupby_grouping_sets3.q | 0 .../queries/clientnegative/groupby_grouping_sets4.q | 0 .../queries/clientnegative/groupby_grouping_sets5.q | 0 .../queries/clientnegative/groupby_grouping_sets6.q | 0 .../queries/clientnegative/groupby_grouping_sets7.q | 0 .../clientnegative/groupby_invalid_position.q | 0 .../src/test/queries/clientnegative/groupby_key.q | 0 .../test/queries/clientnegative/groupby_rollup1.q | 0 .../test/queries/clientnegative/groupby_rollup2.q | 0 .../ql/src/test/queries/clientnegative/having1.q | 0 .../queries/clientnegative/illegal_partition_type.q | 0 .../clientnegative/illegal_partition_type2.q | 0 .../clientnegative/index_bitmap_no_map_aggr.q | 0 .../clientnegative/index_compact_entry_limit.q | 0 .../clientnegative/index_compact_size_limit.q | 0 .../ql/src/test/queries/clientnegative/input1.q | 0 .../ql/src/test/queries/clientnegative/input2.q | 0 .../ql/src/test/queries/clientnegative/input4.q | 0 .../ql/src/test/queries/clientnegative/input41.q | 0 .../test/queries/clientnegative/input_part0_neg.q | 0 .../src/test/queries/clientnegative/insert_into1.q | 0 .../src/test/queries/clientnegative/insert_into2.q | 0 .../src/test/queries/clientnegative/insert_into3.q | 0 .../src/test/queries/clientnegative/insert_into4.q | 0 .../queries/clientnegative/insert_view_failure.q | 0 .../test/queries/clientnegative/insertexternal1.q | 0 .../insertover_dynapart_ifnotexists.q | 0 .../clientnegative/invalid_arithmetic_type.q | 0 .../queries/clientnegative/invalid_avg_syntax.q | 0 .../clientnegative/invalid_cast_from_binary_1.q | 0 .../clientnegative/invalid_cast_from_binary_2.q | 0 .../clientnegative/invalid_cast_from_binary_3.q | 0 .../clientnegative/invalid_cast_from_binary_4.q | 0 .../clientnegative/invalid_cast_from_binary_5.q | 0 .../clientnegative/invalid_cast_from_binary_6.q | 0 .../clientnegative/invalid_cast_to_binary_1.q | 0 .../clientnegative/invalid_cast_to_binary_2.q | 0 .../clientnegative/invalid_cast_to_binary_3.q | 0 .../clientnegative/invalid_cast_to_binary_4.q | 0 .../clientnegative/invalid_cast_to_binary_5.q | 0 .../clientnegative/invalid_cast_to_binary_6.q | 0 .../test/queries/clientnegative/invalid_columns.q | 0 .../test/queries/clientnegative/invalid_config1.q | 0 .../test/queries/clientnegative/invalid_config2.q | 0 .../queries/clientnegative/invalid_create_tbl1.q | 0 .../queries/clientnegative/invalid_create_tbl2.q | 0 .../test/queries/clientnegative/invalid_mapjoin1.q | 0 .../queries/clientnegative/invalid_max_syntax.q | 0 .../queries/clientnegative/invalid_min_syntax.q | 0 .../queries/clientnegative/invalid_select_column.q | 0 .../invalid_select_column_with_subquery.q | 0 .../invalid_select_column_with_tablename.q | 0 .../clientnegative/invalid_select_expression.q | 0 .../queries/clientnegative/invalid_std_syntax.q | 0 .../clientnegative/invalid_stddev_samp_syntax.q | 0 .../queries/clientnegative/invalid_sum_syntax.q | 0 .../test/queries/clientnegative/invalid_t_alter1.q | 0 .../test/queries/clientnegative/invalid_t_alter2.q | 0 .../test/queries/clientnegative/invalid_t_create2.q | 0 .../queries/clientnegative/invalid_t_transform.q | 0 .../test/queries/clientnegative/invalid_tbl_name.q | 0 .../clientnegative/invalid_var_samp_syntax.q | 0 .../clientnegative/invalid_varchar_length_1.q | 0 .../clientnegative/invalid_varchar_length_2.q | 0 .../clientnegative/invalid_varchar_length_3.q | 0 .../clientnegative/invalid_variance_syntax.q | 0 .../test/queries/clientnegative/invalidate_view1.q | 0 .../hive/ql/src/test/queries/clientnegative/join2.q | 0 .../ql/src/test/queries/clientnegative/join28.q | 0 .../ql/src/test/queries/clientnegative/join29.q | 0 .../ql/src/test/queries/clientnegative/join32.q | 0 .../ql/src/test/queries/clientnegative/join35.q | 0 .../queries/clientnegative/join_nonexistent_part.q | 0 .../ql/src/test/queries/clientnegative/joinneg.q | 0 .../queries/clientnegative/lateral_view_alias.q | 0 .../test/queries/clientnegative/lateral_view_join.q | 0 .../test/queries/clientnegative/line_terminator.q | 0 .../clientnegative/load_exist_part_authfail.q | 0 .../test/queries/clientnegative/load_non_native.q | 0 .../queries/clientnegative/load_nonpart_authfail.q | 0 .../queries/clientnegative/load_part_authfail.q | 0 .../test/queries/clientnegative/load_part_nospec.q | 0 .../queries/clientnegative/load_stored_as_dirs.q | 0 .../test/queries/clientnegative/load_view_failure.q | 0 .../queries/clientnegative/load_wrong_fileformat.q | 0 .../clientnegative/load_wrong_fileformat_rc_seq.q | 0 .../clientnegative/load_wrong_fileformat_txt_seq.q | 0 .../queries/clientnegative/load_wrong_noof_part.q | 0 .../clientnegative/local_mapred_error_cache.q | 0 .../ql/src/test/queries/clientnegative/lockneg1.q | 0 .../ql/src/test/queries/clientnegative/lockneg2.q | 0 .../ql/src/test/queries/clientnegative/lockneg3.q | 0 .../ql/src/test/queries/clientnegative/lockneg4.q | 0 .../ql/src/test/queries/clientnegative/lockneg5.q | 0 .../queries/clientnegative/macro_unused_parameter.q | 0 .../queries/clientnegative/mapreduce_stack_trace.q | 0 .../clientnegative/mapreduce_stack_trace_hadoop20.q | 0 .../clientnegative/mapreduce_stack_trace_turnoff.q | 0 .../mapreduce_stack_trace_turnoff_hadoop20.q | 0 .../test/queries/clientnegative/merge_negative_1.q | 0 .../test/queries/clientnegative/merge_negative_2.q | 0 .../test/queries/clientnegative/merge_negative_3.q | 0 .../queries/clientnegative/minimr_broken_pipe.q | 0 .../queries/clientnegative/nested_complex_neg.q | 0 .../test/queries/clientnegative/no_matching_udf.q | 0 .../test/queries/clientnegative/nonkey_groupby.q | 0 .../src/test/queries/clientnegative/nopart_insert.q | 0 .../src/test/queries/clientnegative/nopart_load.q | 0 .../test/queries/clientnegative/notable_alias3.q | 0 .../test/queries/clientnegative/notable_alias4.q | 0 .../clientnegative/orderby_invalid_position.q | 0 .../clientnegative/orderby_position_unsupported.q | 0 .../src/test/queries/clientnegative/orderbysortby.q | 0 .../queries/clientnegative/part_col_complex_type.q | 0 .../test/queries/clientnegative/protectmode_part.q | 0 .../test/queries/clientnegative/protectmode_part1.q | 0 .../test/queries/clientnegative/protectmode_part2.q | 0 .../clientnegative/protectmode_part_no_drop.q | 0 .../test/queries/clientnegative/protectmode_tbl1.q | 0 .../test/queries/clientnegative/protectmode_tbl2.q | 0 .../test/queries/clientnegative/protectmode_tbl3.q | 0 .../test/queries/clientnegative/protectmode_tbl4.q | 0 .../test/queries/clientnegative/protectmode_tbl5.q | 0 .../test/queries/clientnegative/protectmode_tbl6.q | 0 .../test/queries/clientnegative/protectmode_tbl7.q | 0 .../test/queries/clientnegative/protectmode_tbl8.q | 0 .../clientnegative/protectmode_tbl_no_drop.q | 0 .../ptf_negative_AggrFuncsWithNoGBYNoPartDef.q | 0 .../ptf_negative_AmbiguousWindowDefn.q | 0 .../ptf_negative_DistributeByOrderBy.q | 0 .../ptf_negative_DuplicateWindowAlias.q | 0 .../ptf_negative_HavingLeadWithNoGBYNoWindowing.q | 0 .../clientnegative/ptf_negative_HavingLeadWithPTF.q | 0 .../ptf_negative_InvalidValueBoundary.q | 0 .../ptf_negative_JoinWithAmbigousAlias.q | 0 .../clientnegative/ptf_negative_PartitionBySortBy.q | 0 .../clientnegative/ptf_negative_WhereWithRankCond.q | 0 .../queries/clientnegative/ptf_window_boundaries.q | 0 .../queries/clientnegative/ptf_window_boundaries2.q | 0 .../test/queries/clientnegative/recursive_view.q | 0 .../src/test/queries/clientnegative/regex_col_1.q | 0 .../src/test/queries/clientnegative/regex_col_2.q | 0 .../test/queries/clientnegative/regex_col_groupby.q | 0 .../src/test/queries/clientnegative/sa_fail_hook3.q | 0 .../ql/src/test/queries/clientnegative/sample.q | 0 .../queries/clientnegative/script_broken_pipe1.q | 0 .../queries/clientnegative/script_broken_pipe2.q | 0 .../queries/clientnegative/script_broken_pipe3.q | 0 .../src/test/queries/clientnegative/script_error.q | 0 .../queries/clientnegative/select_charliteral.q | 0 .../test/queries/clientnegative/select_udtf_alias.q | 0 .../ql/src/test/queries/clientnegative/semijoin1.q | 0 .../ql/src/test/queries/clientnegative/semijoin2.q | 0 .../ql/src/test/queries/clientnegative/semijoin3.q | 0 .../ql/src/test/queries/clientnegative/semijoin4.q | 0 .../src/test/queries/clientnegative/serde_regex.q | 0 .../src/test/queries/clientnegative/serde_regex2.q | 0 .../src/test/queries/clientnegative/serde_regex3.q | 0 .../clientnegative/set_hiveconf_validation0.q | 0 .../clientnegative/set_hiveconf_validation1.q | 0 .../queries/clientnegative/set_table_property.q | 0 .../src/test/queries/clientnegative/show_columns1.q | 0 .../src/test/queries/clientnegative/show_columns2.q | 0 .../src/test/queries/clientnegative/show_columns3.q | 0 .../show_create_table_does_not_exist.q | 0 .../clientnegative/show_create_table_index.q | 0 .../test/queries/clientnegative/show_partitions1.q | 0 .../queries/clientnegative/show_tableproperties1.q | 0 .../test/queries/clientnegative/show_tables_bad1.q | 0 .../test/queries/clientnegative/show_tables_bad2.q | 0 .../queries/clientnegative/show_tables_bad_db1.q | 0 .../queries/clientnegative/show_tables_bad_db2.q | 0 .../test/queries/clientnegative/show_tablestatus.q | 0 .../show_tablestatus_not_existing_part.q | 0 .../test/queries/clientnegative/smb_bucketmapjoin.q | 0 .../test/queries/clientnegative/smb_mapjoin_14.q | 0 .../clientnegative/sortmerge_mapjoin_mismatch_1.q | 0 .../clientnegative/split_sample_out_of_range.q | 0 .../clientnegative/split_sample_wrong_format.q | 0 .../clientnegative/split_sample_wrong_format2.q | 0 .../clientnegative/stats_aggregator_error_1.q | 0 .../clientnegative/stats_aggregator_error_2.q | 0 .../clientnegative/stats_noscan_non_native.q | 0 .../clientnegative/stats_partialscan_autogether.q | 0 .../clientnegative/stats_partialscan_non_external.q | 0 .../clientnegative/stats_partialscan_non_native.q | 0 .../clientnegative/stats_partscan_norcfile.q | 0 .../clientnegative/stats_publisher_error_1.q | 0 .../clientnegative/stats_publisher_error_2.q | 0 .../src/test/queries/clientnegative/strict_join.q | 0 .../test/queries/clientnegative/strict_orderby.q | 0 .../test/queries/clientnegative/strict_pruning.q | 0 .../src/test/queries/clientnegative/subq_insert.q | 0 .../ql/src/test/queries/clientnegative/touch1.q | 0 .../ql/src/test/queries/clientnegative/touch2.q | 0 .../clientnegative/truncate_bucketed_column.q | 0 .../clientnegative/truncate_column_indexed_table.q | 0 .../clientnegative/truncate_column_list_bucketing.q | 0 .../clientnegative/truncate_column_seqfile.q | 0 .../clientnegative/truncate_nonexistant_column.q | 0 .../clientnegative/truncate_partition_column.q | 0 .../clientnegative/truncate_partition_column2.q | 0 .../clientnegative/truncate_table_failure1.q | 0 .../clientnegative/truncate_table_failure2.q | 0 .../clientnegative/truncate_table_failure3.q | 0 .../clientnegative/truncate_table_failure4.q | 0 .../queries/clientnegative/udaf_invalid_place.q | 0 .../clientnegative/udf_array_contains_wrong1.q | 0 .../clientnegative/udf_array_contains_wrong2.q | 0 .../test/queries/clientnegative/udf_assert_true.q | 0 .../test/queries/clientnegative/udf_assert_true2.q | 0 .../queries/clientnegative/udf_case_type_wrong.q | 0 .../queries/clientnegative/udf_case_type_wrong2.q | 0 .../queries/clientnegative/udf_case_type_wrong3.q | 0 .../src/test/queries/clientnegative/udf_coalesce.q | 0 .../queries/clientnegative/udf_concat_ws_wrong1.q | 0 .../queries/clientnegative/udf_concat_ws_wrong2.q | 0 .../queries/clientnegative/udf_concat_ws_wrong3.q | 0 .../queries/clientnegative/udf_elt_wrong_args_len.q | 0 .../queries/clientnegative/udf_elt_wrong_type.q | 0 .../clientnegative/udf_field_wrong_args_len.q | 0 .../queries/clientnegative/udf_field_wrong_type.q | 0 .../clientnegative/udf_format_number_wrong1.q | 0 .../clientnegative/udf_format_number_wrong2.q | 0 .../clientnegative/udf_format_number_wrong3.q | 0 .../clientnegative/udf_format_number_wrong4.q | 0 .../clientnegative/udf_format_number_wrong5.q | 0 .../clientnegative/udf_format_number_wrong6.q | 0 .../clientnegative/udf_format_number_wrong7.q | 0 .../udf_function_does_not_implement_udf.q | 0 .../test/queries/clientnegative/udf_if_not_bool.q | 0 .../queries/clientnegative/udf_if_wrong_args_len.q | 0 .../ql/src/test/queries/clientnegative/udf_in.q | 0 .../clientnegative/udf_instr_wrong_args_len.q | 0 .../queries/clientnegative/udf_instr_wrong_type.q | 0 .../clientnegative/udf_locate_wrong_args_len.q | 0 .../queries/clientnegative/udf_locate_wrong_type.q | 0 .../queries/clientnegative/udf_map_keys_arg_num.q | 0 .../queries/clientnegative/udf_map_keys_arg_type.q | 0 .../queries/clientnegative/udf_map_values_arg_num.q | 0 .../clientnegative/udf_map_values_arg_type.q | 0 .../ql/src/test/queries/clientnegative/udf_max.q | 0 .../ql/src/test/queries/clientnegative/udf_min.q | 0 .../test/queries/clientnegative/udf_printf_wrong1.q | 0 .../test/queries/clientnegative/udf_printf_wrong2.q | 0 .../test/queries/clientnegative/udf_printf_wrong3.q | 0 .../test/queries/clientnegative/udf_printf_wrong4.q | 0 .../test/queries/clientnegative/udf_reflect_neg.q | 0 .../clientnegative/udf_size_wrong_args_len.q | 0 .../queries/clientnegative/udf_size_wrong_type.q | 0 .../queries/clientnegative/udf_sort_array_wrong1.q | 0 .../queries/clientnegative/udf_sort_array_wrong2.q | 0 .../queries/clientnegative/udf_sort_array_wrong3.q | 0 .../test/queries/clientnegative/udf_test_error.q | 0 .../queries/clientnegative/udf_test_error_reduce.q | 0 .../queries/clientnegative/udf_when_type_wrong.q | 0 .../queries/clientnegative/udf_when_type_wrong2.q | 0 .../queries/clientnegative/udf_when_type_wrong3.q | 0 .../ql/src/test/queries/clientnegative/udfnull.q | 0 .../clientnegative/udtf_explode_not_supported1.q | 0 .../clientnegative/udtf_explode_not_supported2.q | 0 .../clientnegative/udtf_explode_not_supported3.q | 0 .../clientnegative/udtf_explode_not_supported4.q | 0 .../queries/clientnegative/udtf_invalid_place.q | 0 .../queries/clientnegative/udtf_not_supported1.q | 0 .../queries/clientnegative/udtf_not_supported3.q | 0 .../hive/ql/src/test/queries/clientnegative/union.q | 0 .../ql/src/test/queries/clientnegative/union2.q | 0 .../ql/src/test/queries/clientnegative/union22.q | 0 .../ql/src/test/queries/clientnegative/union3.q | 0 .../ql/src/test/queries/clientnegative/uniquejoin.q | 0 .../src/test/queries/clientnegative/uniquejoin2.q | 0 .../src/test/queries/clientnegative/uniquejoin3.q | 0 .../queries/clientnegative/unset_table_property.q | 0 .../queries/clientnegative/unset_view_property.q | 0 .../clientnegative/windowing_leadlag_in_udaf.q | 0 .../queries/clientnegative/windowing_ll_no_over.q | 0 .../test/queries/clientnegative/wrong_column_type.q | 0 .../test/queries/clientpositive/add_part_exist.q | 0 .../test/queries/clientpositive/add_part_multiple.q | 0 .../clientpositive/add_partition_no_whitelist.q | 0 .../clientpositive/add_partition_with_whitelist.q | 0 .../queries/clientpositive/alias_casted_column.q | 0 .../test/queries/clientpositive/allcolref_in_udf.q | 0 .../ql/src/test/queries/clientpositive/alter1.q | 0 .../ql/src/test/queries/clientpositive/alter2.q | 0 .../ql/src/test/queries/clientpositive/alter3.q | 0 .../ql/src/test/queries/clientpositive/alter4.q | 0 .../ql/src/test/queries/clientpositive/alter5.q | 0 .../alter_concatenate_indexed_table.q | 0 .../src/test/queries/clientpositive/alter_index.q | 0 .../src/test/queries/clientpositive/alter_merge.q | 0 .../src/test/queries/clientpositive/alter_merge_2.q | 0 .../test/queries/clientpositive/alter_merge_stats.q | 0 .../alter_numbuckets_partitioned_table.q | 0 .../alter_numbuckets_partitioned_table2.q | 0 .../alter_partition_clusterby_sortby.q | 0 .../clientpositive/alter_partition_coltype.q | 0 .../clientpositive/alter_partition_format_loc.q | 0 .../clientpositive/alter_partition_protect_mode.q | 0 .../clientpositive/alter_partition_with_whitelist.q | 0 .../queries/clientpositive/alter_rename_partition.q | 0 .../alter_rename_partition_authorization.q | 0 .../queries/clientpositive/alter_skewed_table.q | 0 .../queries/clientpositive/alter_table_not_sorted.q | 0 .../test/queries/clientpositive/alter_table_serde.q | 0 .../queries/clientpositive/alter_table_serde2.q | 0 .../test/queries/clientpositive/alter_varchar1.q | 0 .../test/queries/clientpositive/alter_varchar2.q | 0 .../queries/clientpositive/alter_view_as_select.q | 0 .../test/queries/clientpositive/alter_view_rename.q | 0 .../src/test/queries/clientpositive/ambiguous_col.q | 0 .../ql/src/test/queries/clientpositive/archive.q | 0 .../test/queries/clientpositive/archive_corrupt.q | 0 .../clientpositive/archive_excludeHadoop20.q | 0 .../src/test/queries/clientpositive/archive_multi.q | 0 .../test/queries/clientpositive/authorization_1.q | 0 .../test/queries/clientpositive/authorization_2.q | 0 .../test/queries/clientpositive/authorization_3.q | 0 .../test/queries/clientpositive/authorization_4.q | 0 .../test/queries/clientpositive/authorization_5.q | 0 .../test/queries/clientpositive/authorization_6.q | 0 .../test/queries/clientpositive/authorization_7.q | 0 .../test/queries/clientpositive/authorization_8.q | 0 .../ql/src/test/queries/clientpositive/auto_join0.q | 0 .../ql/src/test/queries/clientpositive/auto_join1.q | 0 .../src/test/queries/clientpositive/auto_join10.q | 0 .../src/test/queries/clientpositive/auto_join11.q | 0 .../src/test/queries/clientpositive/auto_join12.q | 0 .../src/test/queries/clientpositive/auto_join13.q | 0 .../src/test/queries/clientpositive/auto_join14.q | 0 .../queries/clientpositive/auto_join14_hadoop20.q | 0 .../src/test/queries/clientpositive/auto_join15.q | 0 .../src/test/queries/clientpositive/auto_join16.q | 0 .../src/test/queries/clientpositive/auto_join17.q | 0 .../src/test/queries/clientpositive/auto_join18.q | 0 .../clientpositive/auto_join18_multi_distinct.q | 0 .../src/test/queries/clientpositive/auto_join19.q | 0 .../ql/src/test/queries/clientpositive/auto_join2.q | 0 .../src/test/queries/clientpositive/auto_join20.q | 0 .../src/test/queries/clientpositive/auto_join21.q | 0 .../src/test/queries/clientpositive/auto_join22.q | 0 .../src/test/queries/clientpositive/auto_join23.q | 0 .../src/test/queries/clientpositive/auto_join24.q | 0 .../src/test/queries/clientpositive/auto_join25.q | 0 .../src/test/queries/clientpositive/auto_join26.q | 0 .../src/test/queries/clientpositive/auto_join27.q | 0 .../src/test/queries/clientpositive/auto_join28.q | 0 .../src/test/queries/clientpositive/auto_join29.q | 0 .../ql/src/test/queries/clientpositive/auto_join3.q | 0 .../src/test/queries/clientpositive/auto_join30.q | 0 .../src/test/queries/clientpositive/auto_join31.q | 0 .../src/test/queries/clientpositive/auto_join32.q | 0 .../ql/src/test/queries/clientpositive/auto_join4.q | 0 .../ql/src/test/queries/clientpositive/auto_join5.q | 0 .../ql/src/test/queries/clientpositive/auto_join6.q | 0 .../ql/src/test/queries/clientpositive/auto_join7.q | 0 .../ql/src/test/queries/clientpositive/auto_join8.q | 0 .../ql/src/test/queries/clientpositive/auto_join9.q | 0 .../test/queries/clientpositive/auto_join_filters.q | 0 .../test/queries/clientpositive/auto_join_nulls.q | 0 .../clientpositive/auto_join_reordering_values.q | 0 .../queries/clientpositive/auto_smb_mapjoin_14.q | 0 .../queries/clientpositive/auto_sortmerge_join_1.q | 0 .../queries/clientpositive/auto_sortmerge_join_10.q | 0 .../queries/clientpositive/auto_sortmerge_join_11.q | 0 .../queries/clientpositive/auto_sortmerge_join_12.q | 0 .../queries/clientpositive/auto_sortmerge_join_13.q | 0 .../queries/clientpositive/auto_sortmerge_join_14.q | 0 .../queries/clientpositive/auto_sortmerge_join_15.q | 0 .../queries/clientpositive/auto_sortmerge_join_2.q | 0 .../queries/clientpositive/auto_sortmerge_join_3.q | 0 .../queries/clientpositive/auto_sortmerge_join_4.q | 0 .../queries/clientpositive/auto_sortmerge_join_5.q | 0 .../queries/clientpositive/auto_sortmerge_join_6.q | 0 .../queries/clientpositive/auto_sortmerge_join_7.q | 0 .../queries/clientpositive/auto_sortmerge_join_8.q | 0 .../queries/clientpositive/auto_sortmerge_join_9.q | 0 .../test/queries/clientpositive/autogen_colalias.q | 0 .../queries/clientpositive/avro_change_schema.q | 0 .../clientpositive/avro_compression_enabled.q | 0 .../queries/clientpositive/avro_evolved_schemas.q | 0 .../ql/src/test/queries/clientpositive/avro_joins.q | 0 .../queries/clientpositive/avro_nullable_fields.q | 0 .../test/queries/clientpositive/avro_partitioned.q | 0 .../test/queries/clientpositive/avro_sanity_test.q | 0 .../clientpositive/avro_schema_error_message.q | 0 .../queries/clientpositive/avro_schema_literal.q | 0 .../ql/src/test/queries/clientpositive/ba_table1.q | 0 .../ql/src/test/queries/clientpositive/ba_table2.q | 0 .../ql/src/test/queries/clientpositive/ba_table3.q | 0 .../src/test/queries/clientpositive/ba_table_udfs.q | 0 .../test/queries/clientpositive/ba_table_union.q | 0 .../test/queries/clientpositive/binary_constant.q | 0 .../queries/clientpositive/binary_output_format.q | 0 .../clientpositive/binary_table_bincolserde.q | 0 .../queries/clientpositive/binary_table_colserde.q | 0 .../test/queries/clientpositive/binarysortable_1.q | 0 .../ql/src/test/queries/clientpositive/bucket1.q | 0 .../ql/src/test/queries/clientpositive/bucket2.q | 0 .../ql/src/test/queries/clientpositive/bucket3.q | 0 .../ql/src/test/queries/clientpositive/bucket4.q | 0 .../ql/src/test/queries/clientpositive/bucket5.q | 0 .../test/queries/clientpositive/bucket_groupby.q | 0 .../test/queries/clientpositive/bucket_map_join_1.q | 0 .../test/queries/clientpositive/bucket_map_join_2.q | 0 .../queries/clientpositive/bucket_num_reducers.q | 0 .../queries/clientpositive/bucket_num_reducers2.q | 0 .../test/queries/clientpositive/bucketcontext_1.q | 0 .../test/queries/clientpositive/bucketcontext_2.q | 0 .../test/queries/clientpositive/bucketcontext_3.q | 0 .../test/queries/clientpositive/bucketcontext_4.q | 0 .../test/queries/clientpositive/bucketcontext_5.q | 0 .../test/queries/clientpositive/bucketcontext_6.q | 0 .../test/queries/clientpositive/bucketcontext_7.q | 0 .../test/queries/clientpositive/bucketcontext_8.q | 0 .../clientpositive/bucketizedhiveinputformat.q | 0 .../clientpositive/bucketizedhiveinputformat_auto.q | 0 .../test/queries/clientpositive/bucketmapjoin1.q | 0 .../test/queries/clientpositive/bucketmapjoin10.q | 0 .../test/queries/clientpositive/bucketmapjoin11.q | 0 .../test/queries/clientpositive/bucketmapjoin12.q | 0 .../test/queries/clientpositive/bucketmapjoin13.q | 0 .../test/queries/clientpositive/bucketmapjoin2.q | 0 .../test/queries/clientpositive/bucketmapjoin3.q | 0 .../test/queries/clientpositive/bucketmapjoin4.q | 0 .../test/queries/clientpositive/bucketmapjoin5.q | 0 .../test/queries/clientpositive/bucketmapjoin6.q | 0 .../test/queries/clientpositive/bucketmapjoin7.q | 0 .../test/queries/clientpositive/bucketmapjoin8.q | 0 .../test/queries/clientpositive/bucketmapjoin9.q | 0 .../queries/clientpositive/bucketmapjoin_negative.q | 0 .../clientpositive/bucketmapjoin_negative2.q | 0 .../clientpositive/bucketmapjoin_negative3.q | 0 .../clientpositive/bucketsortoptimize_insert_1.q | 0 .../clientpositive/bucketsortoptimize_insert_2.q | 0 .../clientpositive/bucketsortoptimize_insert_3.q | 0 .../clientpositive/bucketsortoptimize_insert_4.q | 0 .../clientpositive/bucketsortoptimize_insert_5.q | 0 .../clientpositive/bucketsortoptimize_insert_6.q | 0 .../clientpositive/bucketsortoptimize_insert_7.q | 0 .../clientpositive/bucketsortoptimize_insert_8.q | 0 .../test/queries/clientpositive/case_sensitivity.q | 0 .../hive/ql/src/test/queries/clientpositive/cast1.q | 0 .../src/test/queries/clientpositive/cast_to_int.q | 0 .../ql/src/test/queries/clientpositive/cluster.q | 0 .../queries/clientpositive/column_access_stats.q | 0 .../clientpositive/columnarserde_create_shortcut.q | 0 .../queries/clientpositive/columnstats_partlvl.q | 0 .../queries/clientpositive/columnstats_tbllvl.q | 0 .../ql/src/test/queries/clientpositive/combine1.q | 0 .../ql/src/test/queries/clientpositive/combine2.q | 0 .../test/queries/clientpositive/combine2_hadoop20.q | 0 .../src/test/queries/clientpositive/combine2_win.q | 0 .../ql/src/test/queries/clientpositive/combine3.q | 0 .../queries/clientpositive/compute_stats_binary.q | 0 .../queries/clientpositive/compute_stats_boolean.q | 0 .../queries/clientpositive/compute_stats_double.q | 0 .../clientpositive/compute_stats_empty_table.q | 0 .../queries/clientpositive/compute_stats_long.q | 0 .../queries/clientpositive/compute_stats_string.q | 0 .../concatenate_inherit_table_location.q | 0 .../src/test/queries/clientpositive/constant_prop.q | 0 .../queries/clientpositive/convert_enum_to_string.q | 0 .../queries/clientpositive/correlationoptimizer1.q | 0 .../queries/clientpositive/correlationoptimizer10.q | 0 .../queries/clientpositive/correlationoptimizer11.q | 0 .../queries/clientpositive/correlationoptimizer12.q | 0 .../queries/clientpositive/correlationoptimizer13.q | 0 .../queries/clientpositive/correlationoptimizer14.q | 0 .../queries/clientpositive/correlationoptimizer15.q | 0 .../queries/clientpositive/correlationoptimizer2.q | 0 .../queries/clientpositive/correlationoptimizer3.q | 0 .../queries/clientpositive/correlationoptimizer4.q | 0 .../queries/clientpositive/correlationoptimizer5.q | 0 .../queries/clientpositive/correlationoptimizer6.q | 0 .../queries/clientpositive/correlationoptimizer7.q | 0 .../queries/clientpositive/correlationoptimizer8.q | 0 .../queries/clientpositive/correlationoptimizer9.q | 0 .../hive/ql/src/test/queries/clientpositive/count.q | 0 .../ql/src/test/queries/clientpositive/cp_mj_rc.q | 0 .../ql/src/test/queries/clientpositive/create_1.q | 0 .../create_alter_list_bucketing_table1.q | 0 .../test/queries/clientpositive/create_big_view.q | 0 .../queries/clientpositive/create_default_prop.q | 0 .../src/test/queries/clientpositive/create_escape.q | 0 .../queries/clientpositive/create_genericudaf.q | 0 .../test/queries/clientpositive/create_genericudf.q | 0 .../clientpositive/create_insert_outputformat.q | 0 .../src/test/queries/clientpositive/create_like.q | 0 .../src/test/queries/clientpositive/create_like2.q | 0 .../queries/clientpositive/create_like_tbl_props.q | 0 .../test/queries/clientpositive/create_like_view.q | 0 .../clientpositive/create_merge_compressed.q | 0 .../queries/clientpositive/create_nested_type.q | 0 .../queries/clientpositive/create_or_replace_view.q | 0 .../queries/clientpositive/create_skewed_table1.q | 0 .../queries/clientpositive/create_struct_table.q | 0 .../src/test/queries/clientpositive/create_udaf.q | 0 .../queries/clientpositive/create_union_table.q | 0 .../src/test/queries/clientpositive/create_view.q | 0 .../clientpositive/create_view_partitioned.q | 0 .../queries/clientpositive/create_view_translate.q | 0 .../ql/src/test/queries/clientpositive/cross_join.q | 0 .../queries/clientpositive/ct_case_insensitive.q | 0 .../hive/ql/src/test/queries/clientpositive/ctas.q | 0 .../src/test/queries/clientpositive/ctas_colname.q | 0 .../ql/src/test/queries/clientpositive/ctas_date.q | 0 .../src/test/queries/clientpositive/ctas_hadoop20.q | 0 .../clientpositive/ctas_uses_database_location.q | 0 .../src/test/queries/clientpositive/ctas_varchar.q | 0 .../clientpositive/custom_input_output_format.q | 0 .../ql/src/test/queries/clientpositive/database.q | 0 .../src/test/queries/clientpositive/database_drop.q | 0 .../test/queries/clientpositive/database_location.q | 0 .../queries/clientpositive/database_properties.q | 0 .../ql/src/test/queries/clientpositive/date_1.q | 0 .../ql/src/test/queries/clientpositive/date_2.q | 0 .../ql/src/test/queries/clientpositive/date_3.q | 0 .../ql/src/test/queries/clientpositive/date_4.q | 0 .../test/queries/clientpositive/date_comparison.q | 0 .../ql/src/test/queries/clientpositive/date_join1.q | 0 .../ql/src/test/queries/clientpositive/date_serde.q | 0 .../ql/src/test/queries/clientpositive/date_udf.q | 0 .../ql/src/test/queries/clientpositive/ddltime.q | 0 .../ql/src/test/queries/clientpositive/decimal_1.q | 0 .../ql/src/test/queries/clientpositive/decimal_2.q | 0 .../ql/src/test/queries/clientpositive/decimal_3.q | 0 .../ql/src/test/queries/clientpositive/decimal_4.q | 0 .../src/test/queries/clientpositive/decimal_join.q | 0 .../test/queries/clientpositive/decimal_precision.q | 0 .../src/test/queries/clientpositive/decimal_serde.q | 0 .../src/test/queries/clientpositive/decimal_udf.q | 0 .../queries/clientpositive/default_partition_name.q | 0 .../ql/src/test/queries/clientpositive/delimiter.q | 0 .../queries/clientpositive/desc_non_existent_tbl.q | 0 .../clientpositive/describe_comment_indent.q | 0 .../clientpositive/describe_comment_nonascii.q | 0 .../queries/clientpositive/describe_database_json.q | 0 .../describe_formatted_view_partitioned.q | 0 .../describe_formatted_view_partitioned_json.q | 0 .../test/queries/clientpositive/describe_pretty.q | 0 .../test/queries/clientpositive/describe_syntax.q | 0 .../test/queries/clientpositive/describe_table.q | 0 .../queries/clientpositive/describe_table_json.q | 0 .../test/queries/clientpositive/describe_xpath.q | 0 .../clientpositive/diff_part_input_formats.q | 0 .../clientpositive/disable_file_format_check.q | 0 .../clientpositive/disable_merge_for_bucketing.q | 0 .../disallow_incompatible_type_change_off.q | 0 .../ql/src/test/queries/clientpositive/driverhook.q | 0 .../drop_database_removes_partition_dirs.q | 0 .../src/test/queries/clientpositive/drop_function.q | 0 .../ql/src/test/queries/clientpositive/drop_index.q | 0 .../drop_index_removes_partition_dirs.q | 0 .../queries/clientpositive/drop_multi_partitions.q | 0 .../queries/clientpositive/drop_partitions_filter.q | 0 .../clientpositive/drop_partitions_filter2.q | 0 .../clientpositive/drop_partitions_filter3.q | 0 .../drop_partitions_ignore_protection.q | 0 .../ql/src/test/queries/clientpositive/drop_table.q | 0 .../src/test/queries/clientpositive/drop_table2.q | 0 .../drop_table_removes_partition_dirs.q | 0 .../ql/src/test/queries/clientpositive/drop_udf.q | 0 .../ql/src/test/queries/clientpositive/drop_view.q | 0 .../clientpositive/dynamic_partition_skip_default.q | 0 .../src/test/queries/clientpositive/enforce_order.q | 0 .../ql/src/test/queries/clientpositive/escape1.q | 0 .../ql/src/test/queries/clientpositive/escape2.q | 0 .../test/queries/clientpositive/escape_clusterby1.q | 0 .../queries/clientpositive/escape_distributeby1.q | 0 .../test/queries/clientpositive/escape_orderby1.q | 0 .../test/queries/clientpositive/escape_sortby1.q | 0 .../queries/clientpositive/exchange_partition.q | 0 .../queries/clientpositive/exchange_partition2.q | 0 .../queries/clientpositive/exchange_partition3.q | 0 .../queries/clientpositive/exim_00_nonpart_empty.q | 0 .../test/queries/clientpositive/exim_01_nonpart.q | 0 .../queries/clientpositive/exim_02_00_part_empty.q | 0 .../src/test/queries/clientpositive/exim_02_part.q | 0 .../clientpositive/exim_03_nonpart_over_compat.q | 0 .../test/queries/clientpositive/exim_04_all_part.q | 0 .../queries/clientpositive/exim_04_evolved_parts.q | 0 .../test/queries/clientpositive/exim_05_some_part.q | 0 .../test/queries/clientpositive/exim_06_one_part.q | 0 .../exim_07_all_part_over_nonoverlap.q | 0 .../queries/clientpositive/exim_08_nonpart_rename.q | 0 .../clientpositive/exim_09_part_spec_nonoverlap.q | 0 .../clientpositive/exim_10_external_managed.q | 0 .../clientpositive/exim_11_managed_external.q | 0 .../clientpositive/exim_12_external_location.q | 0 .../clientpositive/exim_13_managed_location.q | 0 .../exim_14_managed_location_over_existing.q | 0 .../queries/clientpositive/exim_15_external_part.q | 0 .../queries/clientpositive/exim_16_part_external.q | 0 .../queries/clientpositive/exim_17_part_managed.q | 0 .../queries/clientpositive/exim_18_part_external.q | 0 .../exim_19_00_part_external_location.q | 0 .../clientpositive/exim_19_part_external_location.q | 0 .../clientpositive/exim_20_part_managed_location.q | 0 .../clientpositive/exim_21_export_authsuccess.q | 0 .../exim_22_import_exist_authsuccess.q | 0 .../exim_23_import_part_authsuccess.q | 0 .../exim_24_import_nonexist_authsuccess.q | 0 .../queries/clientpositive/explain_dependency.q | 0 .../queries/clientpositive/explain_dependency2.q | 0 .../test/queries/clientpositive/explain_logical.q | 0 .../src/test/queries/clientpositive/explode_null.q | 0 .../test/queries/clientpositive/fetch_aggregation.q | 0 .../test/queries/clientpositive/fileformat_mix.q | 0 .../clientpositive/fileformat_sequencefile.q | 0 .../test/queries/clientpositive/fileformat_text.q | 0 .../queries/clientpositive/filter_join_breaktask.q | 0 .../queries/clientpositive/filter_join_breaktask2.q | 0 .../src/test/queries/clientpositive/global_limit.q | 0 .../ql/src/test/queries/clientpositive/groupby1.q | 0 .../ql/src/test/queries/clientpositive/groupby10.q | 0 .../ql/src/test/queries/clientpositive/groupby11.q | 0 .../test/queries/clientpositive/groupby1_limit.q | 0 .../src/test/queries/clientpositive/groupby1_map.q | 0 .../queries/clientpositive/groupby1_map_nomap.q | 0 .../test/queries/clientpositive/groupby1_map_skew.q | 0 .../test/queries/clientpositive/groupby1_noskew.q | 0 .../ql/src/test/queries/clientpositive/groupby2.q | 0 .../test/queries/clientpositive/groupby2_limit.q | 0 .../src/test/queries/clientpositive/groupby2_map.q | 0 .../clientpositive/groupby2_map_multi_distinct.q | 0 .../test/queries/clientpositive/groupby2_map_skew.q | 0 .../test/queries/clientpositive/groupby2_noskew.q | 0 .../clientpositive/groupby2_noskew_multi_distinct.q | 0 .../ql/src/test/queries/clientpositive/groupby3.q | 0 .../src/test/queries/clientpositive/groupby3_map.q | 0 .../clientpositive/groupby3_map_multi_distinct.q | 0 .../test/queries/clientpositive/groupby3_map_skew.q | 0 .../test/queries/clientpositive/groupby3_noskew.q | 0 .../clientpositive/groupby3_noskew_multi_distinct.q | 0 .../ql/src/test/queries/clientpositive/groupby4.q | 0 .../src/test/queries/clientpositive/groupby4_map.q | 0 .../test/queries/clientpositive/groupby4_map_skew.q | 0 .../test/queries/clientpositive/groupby4_noskew.q | 0 .../ql/src/test/queries/clientpositive/groupby5.q | 0 .../src/test/queries/clientpositive/groupby5_map.q | 0 .../test/queries/clientpositive/groupby5_map_skew.q | 0 .../test/queries/clientpositive/groupby5_noskew.q | 0 .../ql/src/test/queries/clientpositive/groupby6.q | 0 .../src/test/queries/clientpositive/groupby6_map.q | 0 .../test/queries/clientpositive/groupby6_map_skew.q | 0 .../test/queries/clientpositive/groupby6_noskew.q | 0 .../ql/src/test/queries/clientpositive/groupby7.q | 0 .../src/test/queries/clientpositive/groupby7_map.q | 0 .../groupby7_map_multi_single_reducer.q | 0 .../test/queries/clientpositive/groupby7_map_skew.q | 0 .../test/queries/clientpositive/groupby7_noskew.q | 0 .../groupby7_noskew_multi_single_reducer.q | 0 .../ql/src/test/queries/clientpositive/groupby8.q | 0 .../src/test/queries/clientpositive/groupby8_map.q | 0 .../test/queries/clientpositive/groupby8_map_skew.q | 0 .../test/queries/clientpositive/groupby8_noskew.q | 0 .../ql/src/test/queries/clientpositive/groupby9.q | 0 .../test/queries/clientpositive/groupby_bigdata.q | 0 .../queries/clientpositive/groupby_complex_types.q | 0 .../groupby_complex_types_multi_single_reducer.q | 0 .../src/test/queries/clientpositive/groupby_cube1.q | 0 .../clientpositive/groupby_distinct_samekey.q | 0 .../queries/clientpositive/groupby_grouping_id1.q | 0 .../queries/clientpositive/groupby_grouping_id2.q | 0 .../queries/clientpositive/groupby_grouping_sets1.q | 0 .../queries/clientpositive/groupby_grouping_sets2.q | 0 .../queries/clientpositive/groupby_grouping_sets3.q | 0 .../queries/clientpositive/groupby_grouping_sets4.q | 0 .../queries/clientpositive/groupby_grouping_sets5.q | 0 .../test/queries/clientpositive/groupby_map_ppr.q | 0 .../clientpositive/groupby_map_ppr_multi_distinct.q | 0 .../groupby_multi_insert_common_distinct.q | 0 .../clientpositive/groupby_multi_single_reducer.q | 0 .../clientpositive/groupby_multi_single_reducer2.q | 0 .../clientpositive/groupby_multi_single_reducer3.q | 0 .../groupby_mutli_insert_common_distinct.q | 0 .../test/queries/clientpositive/groupby_neg_float.q | 0 .../test/queries/clientpositive/groupby_position.q | 0 .../src/test/queries/clientpositive/groupby_ppd.q | 0 .../src/test/queries/clientpositive/groupby_ppr.q | 0 .../clientpositive/groupby_ppr_multi_distinct.q | 0 .../test/queries/clientpositive/groupby_rollup1.q | 0 .../test/queries/clientpositive/groupby_sort_1.q | 0 .../test/queries/clientpositive/groupby_sort_10.q | 0 .../test/queries/clientpositive/groupby_sort_11.q | 0 .../test/queries/clientpositive/groupby_sort_2.q | 0 .../test/queries/clientpositive/groupby_sort_3.q | 0 .../test/queries/clientpositive/groupby_sort_4.q | 0 .../test/queries/clientpositive/groupby_sort_5.q | 0 .../test/queries/clientpositive/groupby_sort_6.q | 0 .../test/queries/clientpositive/groupby_sort_7.q | 0 .../test/queries/clientpositive/groupby_sort_8.q | 0 .../test/queries/clientpositive/groupby_sort_9.q | 0 .../queries/clientpositive/groupby_sort_skew_1.q | 0 .../queries/clientpositive/groupby_sort_test_1.q | 0 .../ql/src/test/queries/clientpositive/having.q | 0 .../test/queries/clientpositive/hook_context_cs.q | 0 .../ql/src/test/queries/clientpositive/hook_order.q | 0 .../test/queries/clientpositive/implicit_cast1.q | 0 .../ql/src/test/queries/clientpositive/index_auth.q | 0 .../ql/src/test/queries/clientpositive/index_auto.q | 0 .../test/queries/clientpositive/index_auto_empty.q | 0 .../queries/clientpositive/index_auto_file_format.q | 0 .../queries/clientpositive/index_auto_mult_tables.q | 0 .../clientpositive/index_auto_mult_tables_compact.q | 0 .../queries/clientpositive/index_auto_multiple.q | 0 .../queries/clientpositive/index_auto_partitioned.q | 0 .../queries/clientpositive/index_auto_self_join.q | 0 .../test/queries/clientpositive/index_auto_unused.q | 0 .../test/queries/clientpositive/index_auto_update.q | 0 .../src/test/queries/clientpositive/index_bitmap.q | 0 .../src/test/queries/clientpositive/index_bitmap1.q | 0 .../src/test/queries/clientpositive/index_bitmap2.q | 0 .../src/test/queries/clientpositive/index_bitmap3.q | 0 .../test/queries/clientpositive/index_bitmap_auto.q | 0 .../clientpositive/index_bitmap_auto_partitioned.q | 0 .../clientpositive/index_bitmap_compression.q | 0 .../test/queries/clientpositive/index_bitmap_rc.q | 0 .../src/test/queries/clientpositive/index_compact.q | 0 .../test/queries/clientpositive/index_compact_1.q | 0 .../test/queries/clientpositive/index_compact_2.q | 0 .../test/queries/clientpositive/index_compact_3.q | 0 .../clientpositive/index_compact_binary_search.q | 0 .../test/queries/clientpositive/index_compression.q | 0 .../test/queries/clientpositive/index_creation.q | 0 .../src/test/queries/clientpositive/index_serde.q | 0 .../src/test/queries/clientpositive/index_stale.q | 0 .../clientpositive/index_stale_partitioned.q | 0 .../test/queries/clientpositive/infer_bucket_sort.q | 0 .../infer_bucket_sort_bucketed_table.q | 0 .../clientpositive/infer_bucket_sort_convert_join.q | 0 .../clientpositive/infer_bucket_sort_dyn_part.q | 0 .../infer_bucket_sort_grouping_operators.q | 0 .../clientpositive/infer_bucket_sort_list_bucket.q | 0 .../infer_bucket_sort_map_operators.q | 0 .../clientpositive/infer_bucket_sort_merge.q | 0 .../clientpositive/infer_bucket_sort_multi_insert.q | 0 .../clientpositive/infer_bucket_sort_num_buckets.q | 0 .../infer_bucket_sort_reducers_power_two.q | 0 .../test/queries/clientpositive/infer_const_type.q | 0 .../ql/src/test/queries/clientpositive/init_file.q | 0 .../ql/src/test/queries/clientpositive/innerjoin.q | 0 .../src/test/queries/clientpositive/inoutdriver.q | 0 .../hive/ql/src/test/queries/clientpositive/input.q | 0 .../ql/src/test/queries/clientpositive/input0.q | 0 .../ql/src/test/queries/clientpositive/input1.q | 0 .../ql/src/test/queries/clientpositive/input10.q | 0 .../ql/src/test/queries/clientpositive/input11.q | 0 .../src/test/queries/clientpositive/input11_limit.q | 0 .../ql/src/test/queries/clientpositive/input12.q | 0 .../test/queries/clientpositive/input12_hadoop20.q | 0 .../ql/src/test/queries/clientpositive/input13.q | 0 .../ql/src/test/queries/clientpositive/input14.q | 0 .../src/test/queries/clientpositive/input14_limit.q | 0 .../ql/src/test/queries/clientpositive/input15.q | 0 .../ql/src/test/queries/clientpositive/input16.q | 0 .../ql/src/test/queries/clientpositive/input16_cc.q | 0 .../ql/src/test/queries/clientpositive/input17.q | 0 .../ql/src/test/queries/clientpositive/input18.q | 0 .../ql/src/test/queries/clientpositive/input19.q | 0 .../src/test/queries/clientpositive/input1_limit.q | 0 .../ql/src/test/queries/clientpositive/input2.q | 0 .../ql/src/test/queries/clientpositive/input20.q | 0 .../ql/src/test/queries/clientpositive/input21.q | 0 .../ql/src/test/queries/clientpositive/input22.q | 0 .../ql/src/test/queries/clientpositive/input23.q | 0 .../ql/src/test/queries/clientpositive/input24.q | 0 .../ql/src/test/queries/clientpositive/input25.q | 0 .../ql/src/test/queries/clientpositive/input26.q | 0 .../ql/src/test/queries/clientpositive/input28.q | 0 .../src/test/queries/clientpositive/input2_limit.q | 0 .../ql/src/test/queries/clientpositive/input3.q | 0 .../ql/src/test/queries/clientpositive/input30.q | 0 .../ql/src/test/queries/clientpositive/input31.q | 0 .../ql/src/test/queries/clientpositive/input32.q | 0 .../ql/src/test/queries/clientpositive/input33.q | 0 .../ql/src/test/queries/clientpositive/input34.q | 0 .../ql/src/test/queries/clientpositive/input35.q | 0 .../ql/src/test/queries/clientpositive/input36.q | 0 .../ql/src/test/queries/clientpositive/input37.q | 0 .../ql/src/test/queries/clientpositive/input38.q | 0 .../ql/src/test/queries/clientpositive/input39.q | 0 .../test/queries/clientpositive/input39_hadoop20.q | 0 .../src/test/queries/clientpositive/input3_limit.q | 0 .../ql/src/test/queries/clientpositive/input4.q | 0 .../ql/src/test/queries/clientpositive/input40.q | 0 .../ql/src/test/queries/clientpositive/input41.q | 0 .../ql/src/test/queries/clientpositive/input42.q | 0 .../ql/src/test/queries/clientpositive/input43.q | 0 .../ql/src/test/queries/clientpositive/input44.q | 0 .../ql/src/test/queries/clientpositive/input45.q | 0 .../ql/src/test/queries/clientpositive/input46.q | 0 .../ql/src/test/queries/clientpositive/input49.q | 0 .../test/queries/clientpositive/input4_cb_delim.q | 0 .../src/test/queries/clientpositive/input4_limit.q | 0 .../ql/src/test/queries/clientpositive/input5.q | 0 .../ql/src/test/queries/clientpositive/input6.q | 0 .../ql/src/test/queries/clientpositive/input7.q | 0 .../ql/src/test/queries/clientpositive/input8.q | 0 .../ql/src/test/queries/clientpositive/input9.q | 0 .../queries/clientpositive/input_columnarserde.q | 0 .../ql/src/test/queries/clientpositive/input_dfs.q | 0 .../queries/clientpositive/input_dynamicserde.q | 0 .../test/queries/clientpositive/input_lazyserde.q | 0 .../src/test/queries/clientpositive/input_limit.q | 0 .../src/test/queries/clientpositive/input_part0.q | 0 .../src/test/queries/clientpositive/input_part1.q | 0 .../src/test/queries/clientpositive/input_part10.q | 0 .../test/queries/clientpositive/input_part10_win.q | 0 .../src/test/queries/clientpositive/input_part2.q | 0 .../src/test/queries/clientpositive/input_part3.q | 0 .../src/test/queries/clientpositive/input_part4.q | 0 .../src/test/queries/clientpositive/input_part5.q | 0 .../src/test/queries/clientpositive/input_part6.q | 0 .../src/test/queries/clientpositive/input_part7.q | 0 .../src/test/queries/clientpositive/input_part8.q | 0 .../src/test/queries/clientpositive/input_part9.q | 0 .../queries/clientpositive/input_testsequencefile.q | 0 .../test/queries/clientpositive/input_testxpath.q | 0 .../test/queries/clientpositive/input_testxpath2.q | 0 .../test/queries/clientpositive/input_testxpath3.q | 0 .../test/queries/clientpositive/input_testxpath4.q | 0 .../ql/src/test/queries/clientpositive/inputddl1.q | 0 .../ql/src/test/queries/clientpositive/inputddl2.q | 0 .../ql/src/test/queries/clientpositive/inputddl3.q | 0 .../ql/src/test/queries/clientpositive/inputddl4.q | 0 .../ql/src/test/queries/clientpositive/inputddl5.q | 0 .../ql/src/test/queries/clientpositive/inputddl6.q | 0 .../ql/src/test/queries/clientpositive/inputddl7.q | 0 .../ql/src/test/queries/clientpositive/inputddl8.q | 0 .../ql/src/test/queries/clientpositive/insert1.q | 0 .../clientpositive/insert1_overwrite_partitions.q | 0 .../clientpositive/insert2_overwrite_partitions.q | 0 .../test/queries/clientpositive/insert_compressed.q | 0 .../src/test/queries/clientpositive/insert_into1.q | 0 .../src/test/queries/clientpositive/insert_into2.q | 0 .../src/test/queries/clientpositive/insert_into3.q | 0 .../src/test/queries/clientpositive/insert_into4.q | 0 .../src/test/queries/clientpositive/insert_into5.q | 0 .../src/test/queries/clientpositive/insert_into6.q | 0 .../insert_overwrite_local_directory_1.q | 0 .../test/queries/clientpositive/insertexternal1.q | 0 .../hive/ql/src/test/queries/clientpositive/join0.q | 0 .../hive/ql/src/test/queries/clientpositive/join1.q | 0 .../ql/src/test/queries/clientpositive/join10.q | 0 .../ql/src/test/queries/clientpositive/join11.q | 0 .../ql/src/test/queries/clientpositive/join12.q | 0 .../ql/src/test/queries/clientpositive/join13.q | 0 .../ql/src/test/queries/clientpositive/join14.q | 0 .../test/queries/clientpositive/join14_hadoop20.q | 0 .../ql/src/test/queries/clientpositive/join15.q | 0 .../ql/src/test/queries/clientpositive/join16.q | 0 .../ql/src/test/queries/clientpositive/join17.q | 0 .../ql/src/test/queries/clientpositive/join18.q | 0 .../queries/clientpositive/join18_multi_distinct.q | 0 .../ql/src/test/queries/clientpositive/join19.q | 0 .../hive/ql/src/test/queries/clientpositive/join2.q | 0 .../ql/src/test/queries/clientpositive/join20.q | 0 .../ql/src/test/queries/clientpositive/join21.q | 0 .../ql/src/test/queries/clientpositive/join22.q | 0 .../ql/src/test/queries/clientpositive/join23.q | 0 .../ql/src/test/queries/clientpositive/join24.q | 0 .../ql/src/test/queries/clientpositive/join25.q | 0 .../ql/src/test/queries/clientpositive/join26.q | 0 .../ql/src/test/queries/clientpositive/join27.q | 0 .../ql/src/test/queries/clientpositive/join28.q | 0 .../ql/src/test/queries/clientpositive/join29.q | 0 .../hive/ql/src/test/queries/clientpositive/join3.q | 0 .../ql/src/test/queries/clientpositive/join30.q | 0 .../ql/src/test/queries/clientpositive/join31.q | 0 .../ql/src/test/queries/clientpositive/join32.q | 0 .../test/queries/clientpositive/join32_lessSize.q | 0 .../ql/src/test/queries/clientpositive/join33.q | 0 .../ql/src/test/queries/clientpositive/join34.q | 0 .../ql/src/test/queries/clientpositive/join35.q | 0 .../ql/src/test/queries/clientpositive/join36.q | 0 .../ql/src/test/queries/clientpositive/join37.q | 0 .../ql/src/test/queries/clientpositive/join38.q | 0 .../ql/src/test/queries/clientpositive/join39.q | 0 .../hive/ql/src/test/queries/clientpositive/join4.q | 0 .../ql/src/test/queries/clientpositive/join40.q | 0 .../ql/src/test/queries/clientpositive/join41.q | 0 .../hive/ql/src/test/queries/clientpositive/join5.q | 0 .../hive/ql/src/test/queries/clientpositive/join6.q | 0 .../hive/ql/src/test/queries/clientpositive/join7.q | 0 .../hive/ql/src/test/queries/clientpositive/join8.q | 0 .../hive/ql/src/test/queries/clientpositive/join9.q | 0 .../ql/src/test/queries/clientpositive/join_1to1.q | 0 .../ql/src/test/queries/clientpositive/join_array.q | 0 .../queries/clientpositive/join_casesensitive.q | 0 .../ql/src/test/queries/clientpositive/join_empty.q | 0 .../src/test/queries/clientpositive/join_filters.q | 0 .../queries/clientpositive/join_filters_overlap.q | 0 .../src/test/queries/clientpositive/join_hive_626.q | 0 .../src/test/queries/clientpositive/join_literals.q | 0 .../src/test/queries/clientpositive/join_map_ppr.q | 0 .../ql/src/test/queries/clientpositive/join_nulls.q | 0 .../src/test/queries/clientpositive/join_nullsafe.q | 0 .../ql/src/test/queries/clientpositive/join_rc.q | 0 .../src/test/queries/clientpositive/join_reorder.q | 0 .../src/test/queries/clientpositive/join_reorder2.q | 0 .../src/test/queries/clientpositive/join_reorder3.q | 0 .../src/test/queries/clientpositive/join_reorder4.q | 0 .../ql/src/test/queries/clientpositive/join_star.q | 0 .../src/test/queries/clientpositive/join_thrift.q | 0 .../ql/src/test/queries/clientpositive/join_vc.q | 0 .../ql/src/test/queries/clientpositive/join_view.q | 0 .../ql/src/test/queries/clientpositive/keyword_1.q | 0 .../src/test/queries/clientpositive/lateral_view.q | 0 .../test/queries/clientpositive/lateral_view_cp.q | 0 .../queries/clientpositive/lateral_view_noalias.q | 0 .../queries/clientpositive/lateral_view_outer.q | 0 .../test/queries/clientpositive/lateral_view_ppd.q | 0 .../ql/src/test/queries/clientpositive/leadlag.q | 0 .../test/queries/clientpositive/leadlag_queries.q | 0 .../src/test/queries/clientpositive/leftsemijoin.q | 0 .../test/queries/clientpositive/leftsemijoin_mr.q | 0 .../test/queries/clientpositive/limit_pushdown.q | 0 .../clientpositive/limit_pushdown_negative.q | 0 .../ql/src/test/queries/clientpositive/lineage1.q | 0 .../test/queries/clientpositive/list_bucket_dml_1.q | 0 .../queries/clientpositive/list_bucket_dml_10.q | 0 .../queries/clientpositive/list_bucket_dml_11.q | 0 .../queries/clientpositive/list_bucket_dml_12.q | 0 .../queries/clientpositive/list_bucket_dml_13.q | 0 .../test/queries/clientpositive/list_bucket_dml_2.q | 0 .../test/queries/clientpositive/list_bucket_dml_3.q | 0 .../test/queries/clientpositive/list_bucket_dml_4.q | 0 .../test/queries/clientpositive/list_bucket_dml_5.q | 0 .../test/queries/clientpositive/list_bucket_dml_6.q | 0 .../test/queries/clientpositive/list_bucket_dml_7.q | 0 .../test/queries/clientpositive/list_bucket_dml_8.q | 0 .../test/queries/clientpositive/list_bucket_dml_9.q | 0 .../clientpositive/list_bucket_query_multiskew_1.q | 0 .../clientpositive/list_bucket_query_multiskew_2.q | 0 .../clientpositive/list_bucket_query_multiskew_3.q | 0 .../clientpositive/list_bucket_query_oneskew_1.q | 0 .../clientpositive/list_bucket_query_oneskew_2.q | 0 .../clientpositive/list_bucket_query_oneskew_3.q | 0 .../test/queries/clientpositive/literal_decimal.q | 0 .../test/queries/clientpositive/literal_double.q | 0 .../src/test/queries/clientpositive/literal_ints.q | 0 .../test/queries/clientpositive/literal_string.q | 0 .../test/queries/clientpositive/load_binary_data.q | 0 .../test/queries/clientpositive/load_dyn_part1.q | 0 .../test/queries/clientpositive/load_dyn_part10.q | 0 .../test/queries/clientpositive/load_dyn_part11.q | 0 .../test/queries/clientpositive/load_dyn_part12.q | 0 .../test/queries/clientpositive/load_dyn_part13.q | 0 .../test/queries/clientpositive/load_dyn_part14.q | 0 .../queries/clientpositive/load_dyn_part14_win.q | 0 .../test/queries/clientpositive/load_dyn_part15.q | 0 .../test/queries/clientpositive/load_dyn_part2.q | 0 .../test/queries/clientpositive/load_dyn_part3.q | 0 .../test/queries/clientpositive/load_dyn_part4.q | 0 .../test/queries/clientpositive/load_dyn_part5.q | 0 .../test/queries/clientpositive/load_dyn_part6.q | 0 .../test/queries/clientpositive/load_dyn_part7.q | 0 .../test/queries/clientpositive/load_dyn_part8.q | 0 .../test/queries/clientpositive/load_dyn_part9.q | 0 .../clientpositive/load_exist_part_authsuccess.q | 0 .../load_file_with_space_in_the_name.q | 0 .../ql/src/test/queries/clientpositive/load_fs.q | 0 .../ql/src/test/queries/clientpositive/load_fs2.q | 0 .../load_hdfs_file_with_space_in_the_name.q | 0 .../clientpositive/load_nonpart_authsuccess.q | 0 .../test/queries/clientpositive/load_overwrite.q | 0 .../queries/clientpositive/load_part_authsuccess.q | 0 .../ql/src/test/queries/clientpositive/loadpart1.q | 0 .../src/test/queries/clientpositive/loadpart_err.q | 0 .../hive/ql/src/test/queries/clientpositive/lock1.q | 0 .../hive/ql/src/test/queries/clientpositive/lock2.q | 0 .../hive/ql/src/test/queries/clientpositive/lock3.q | 0 .../hive/ql/src/test/queries/clientpositive/lock4.q | 0 .../test/queries/clientpositive/louter_join_ppr.q | 0 .../hive/ql/src/test/queries/clientpositive/macro.q | 0 .../ql/src/test/queries/clientpositive/mapjoin1.q | 0 .../test/queries/clientpositive/mapjoin_distinct.q | 0 .../clientpositive/mapjoin_filter_on_outerjoin.q | 0 .../src/test/queries/clientpositive/mapjoin_hook.q | 0 .../test/queries/clientpositive/mapjoin_mapjoin.q | 0 .../test/queries/clientpositive/mapjoin_subquery.q | 0 .../test/queries/clientpositive/mapjoin_subquery2.q | 0 .../queries/clientpositive/mapjoin_test_outer.q | 0 .../ql/src/test/queries/clientpositive/mapreduce1.q | 0 .../ql/src/test/queries/clientpositive/mapreduce2.q | 0 .../ql/src/test/queries/clientpositive/mapreduce3.q | 0 .../ql/src/test/queries/clientpositive/mapreduce4.q | 0 .../ql/src/test/queries/clientpositive/mapreduce5.q | 0 .../ql/src/test/queries/clientpositive/mapreduce6.q | 0 .../ql/src/test/queries/clientpositive/mapreduce7.q | 0 .../ql/src/test/queries/clientpositive/mapreduce8.q | 0 .../ql/src/test/queries/clientpositive/merge1.q | 0 .../ql/src/test/queries/clientpositive/merge2.q | 0 .../ql/src/test/queries/clientpositive/merge3.q | 0 .../ql/src/test/queries/clientpositive/merge4.q | 0 .../clientpositive/merge_dynamic_partition.q | 0 .../clientpositive/merge_dynamic_partition2.q | 0 .../clientpositive/merge_dynamic_partition3.q | 0 .../clientpositive/merge_dynamic_partition4.q | 0 .../clientpositive/merge_dynamic_partition5.q | 0 .../ql/src/test/queries/clientpositive/mergejoins.q | 0 .../test/queries/clientpositive/mergejoins_mixed.q | 0 .../queries/clientpositive/metadata_export_drop.q | 0 .../src/test/queries/clientpositive/metadataonly1.q | 0 .../hive/ql/src/test/queries/clientpositive/mi.q | 0 .../ql/src/test/queries/clientpositive/misc_json.q | 0 .../src/test/queries/clientpositive/multiMapJoin1.q | 0 .../src/test/queries/clientpositive/multiMapJoin2.q | 0 .../src/test/queries/clientpositive/multi_insert.q | 0 .../test/queries/clientpositive/multi_insert_gby.q | 0 .../test/queries/clientpositive/multi_insert_gby2.q | 0 .../test/queries/clientpositive/multi_insert_gby3.q | 0 .../clientpositive/multi_insert_lateral_view.q | 0 .../multi_insert_move_tasks_share_dependencies.q | 0 .../test/queries/clientpositive/multi_join_union.q | 0 .../src/test/queries/clientpositive/multi_sahooks.q | 0 .../queries/clientpositive/multigroupby_singlemr.q | 0 .../test/queries/clientpositive/nested_complex.q | 0 .../src/test/queries/clientpositive/nestedvirtual.q | 0 .../ql/src/test/queries/clientpositive/newline.q | 0 .../ql/src/test/queries/clientpositive/no_hooks.q | 0 .../src/test/queries/clientpositive/noalias_subq1.q | 0 .../clientpositive/nomore_ambiguous_table_col.q | 0 .../clientpositive/nonblock_op_deduplicate.q | 0 .../src/test/queries/clientpositive/nonmr_fetch.q | 0 .../clientpositive/nonreserved_keywords_input37.q | 0 .../nonreserved_keywords_insert_into1.q | 0 .../test/queries/clientpositive/notable_alias1.q | 0 .../test/queries/clientpositive/notable_alias2.q | 0 .../ql/src/test/queries/clientpositive/null_cast.q | 0 .../src/test/queries/clientpositive/null_column.q | 0 .../ql/src/test/queries/clientpositive/nullgroup.q | 0 .../ql/src/test/queries/clientpositive/nullgroup2.q | 0 .../ql/src/test/queries/clientpositive/nullgroup3.q | 0 .../ql/src/test/queries/clientpositive/nullgroup4.q | 0 .../clientpositive/nullgroup4_multi_distinct.q | 0 .../ql/src/test/queries/clientpositive/nullgroup5.q | 0 .../ql/src/test/queries/clientpositive/nullinput.q | 0 .../ql/src/test/queries/clientpositive/nullinput2.q | 0 .../ql/src/test/queries/clientpositive/nullscript.q | 0 .../test/queries/clientpositive/num_op_type_conv.q | 0 .../test/queries/clientpositive/ops_comparison.q | 0 .../test/queries/clientpositive/optional_outer.q | 0 .../test/queries/clientpositive/optrstat_groupby.q | 0 .../ql/src/test/queries/clientpositive/orc_create.q | 0 .../src/test/queries/clientpositive/orc_createas1.q | 0 .../clientpositive/orc_dictionary_threshold.q | 0 .../queries/clientpositive/orc_diff_part_cols.q | 0 .../test/queries/clientpositive/orc_empty_files.q | 0 .../test/queries/clientpositive/orc_empty_strings.q | 0 .../queries/clientpositive/orc_ends_with_nulls.q | 0 .../queries/clientpositive/orc_predicate_pushdown.q | 0 .../hive/ql/src/test/queries/clientpositive/order.q | 0 .../ql/src/test/queries/clientpositive/order2.q | 0 .../test/queries/clientpositive/outer_join_ppr.q | 0 .../test/queries/clientpositive/overridden_confs.q | 0 .../ql/src/test/queries/clientpositive/parallel.q | 0 .../test/queries/clientpositive/parallel_orderby.q | 0 .../queries/clientpositive/parenthesis_star_by.q | 0 .../ql/src/test/queries/clientpositive/partInit.q | 0 .../queries/clientpositive/part_inherit_tbl_props.q | 0 .../clientpositive/part_inherit_tbl_props_empty.q | 0 .../part_inherit_tbl_props_with_star.q | 0 .../ql/src/test/queries/clientpositive/partcols1.q | 0 .../test/queries/clientpositive/partition_date.q | 0 .../test/queries/clientpositive/partition_date2.q | 0 .../queries/clientpositive/partition_decode_name.q | 0 .../test/queries/clientpositive/partition_schema1.q | 0 .../queries/clientpositive/partition_serde_format.q | 0 .../queries/clientpositive/partition_special_char.q | 0 .../queries/clientpositive/partition_type_check.q | 0 .../queries/clientpositive/partition_varchar1.q | 0 .../clientpositive/partition_vs_table_metadata.q | 0 .../clientpositive/partition_wise_fileformat.q | 0 .../clientpositive/partition_wise_fileformat10.q | 0 .../clientpositive/partition_wise_fileformat11.q | 0 .../clientpositive/partition_wise_fileformat12.q | 0 .../clientpositive/partition_wise_fileformat13.q | 0 .../clientpositive/partition_wise_fileformat14.q | 0 .../clientpositive/partition_wise_fileformat15.q | 0 .../clientpositive/partition_wise_fileformat16.q | 0 .../clientpositive/partition_wise_fileformat17.q | 0 .../clientpositive/partition_wise_fileformat2.q | 0 .../clientpositive/partition_wise_fileformat3.q | 0 .../clientpositive/partition_wise_fileformat4.q | 0 .../clientpositive/partition_wise_fileformat5.q | 0 .../clientpositive/partition_wise_fileformat6.q | 0 .../clientpositive/partition_wise_fileformat7.q | 0 .../clientpositive/partition_wise_fileformat8.q | 0 .../clientpositive/partition_wise_fileformat9.q | 0 .../test/queries/clientpositive/partitions_json.q | 0 .../hive/ql/src/test/queries/clientpositive/pcr.q | 0 .../ql/src/test/queries/clientpositive/plan_json.q | 0 .../hive/ql/src/test/queries/clientpositive/ppd1.q | 0 .../hive/ql/src/test/queries/clientpositive/ppd2.q | 0 .../src/test/queries/clientpositive/ppd_clusterby.q | 0 .../test/queries/clientpositive/ppd_constant_expr.q | 0 .../queries/clientpositive/ppd_constant_where.q | 0 .../ql/src/test/queries/clientpositive/ppd_gby.q | 0 .../ql/src/test/queries/clientpositive/ppd_gby2.q | 0 .../src/test/queries/clientpositive/ppd_gby_join.q | 0 .../ql/src/test/queries/clientpositive/ppd_join.q | 0 .../ql/src/test/queries/clientpositive/ppd_join2.q | 0 .../ql/src/test/queries/clientpositive/ppd_join3.q | 0 .../test/queries/clientpositive/ppd_join_filter.q | 0 .../test/queries/clientpositive/ppd_multi_insert.q | 0 .../test/queries/clientpositive/ppd_outer_join1.q | 0 .../test/queries/clientpositive/ppd_outer_join2.q | 0 .../test/queries/clientpositive/ppd_outer_join3.q | 0 .../test/queries/clientpositive/ppd_outer_join4.q | 0 .../test/queries/clientpositive/ppd_outer_join5.q | 0 .../ql/src/test/queries/clientpositive/ppd_random.q | 0 .../queries/clientpositive/ppd_repeated_alias.q | 0 .../src/test/queries/clientpositive/ppd_transform.q | 0 .../src/test/queries/clientpositive/ppd_udf_case.q | 0 .../src/test/queries/clientpositive/ppd_udf_col.q | 0 .../ql/src/test/queries/clientpositive/ppd_union.q | 0 .../test/queries/clientpositive/ppd_union_view.q | 0 .../ql/src/test/queries/clientpositive/ppd_vc.q | 0 .../queries/clientpositive/ppr_allchildsarenull.q | 0 .../src/test/queries/clientpositive/ppr_pushdown.q | 0 .../src/test/queries/clientpositive/ppr_pushdown2.q | 0 .../src/test/queries/clientpositive/ppr_pushdown3.q | 0 .../src/test/queries/clientpositive/print_header.q | 0 .../ql/src/test/queries/clientpositive/progress_1.q | 0 .../src/test/queries/clientpositive/protectmode.q | 0 .../src/test/queries/clientpositive/protectmode2.q | 0 .../hive/ql/src/test/queries/clientpositive/ptf.q | 0 .../src/test/queries/clientpositive/ptf_decimal.q | 0 .../queries/clientpositive/ptf_general_queries.q | 0 .../src/test/queries/clientpositive/ptf_matchpath.q | 0 .../ql/src/test/queries/clientpositive/ptf_rcfile.q | 0 .../queries/clientpositive/ptf_register_tblfn.q | 0 .../src/test/queries/clientpositive/ptf_seqfile.q | 0 .../ql/src/test/queries/clientpositive/push_or.q | 0 .../queries/clientpositive/ql_rewrite_gbtoidx.q | 0 .../test/queries/clientpositive/query_properties.q | 0 .../clientpositive/query_result_fileformat.q | 0 .../test/queries/clientpositive/query_with_semi.q | 0 .../ql/src/test/queries/clientpositive/quote1.q | 0 .../ql/src/test/queries/clientpositive/quote2.q | 0 .../queries/clientpositive/rand_partitionpruner1.q | 0 .../queries/clientpositive/rand_partitionpruner2.q | 0 .../queries/clientpositive/rand_partitionpruner3.q | 0 .../test/queries/clientpositive/rcfile_bigdata.q | 0 .../test/queries/clientpositive/rcfile_columnar.q | 0 .../test/queries/clientpositive/rcfile_createas1.q | 0 .../queries/clientpositive/rcfile_default_format.q | 0 .../queries/clientpositive/rcfile_lazydecompress.q | 0 .../src/test/queries/clientpositive/rcfile_merge1.q | 0 .../src/test/queries/clientpositive/rcfile_merge2.q | 0 .../src/test/queries/clientpositive/rcfile_merge3.q | 0 .../src/test/queries/clientpositive/rcfile_merge4.q | 0 .../test/queries/clientpositive/rcfile_null_value.q | 0 .../clientpositive/rcfile_toleratecorruptions.q | 0 .../src/test/queries/clientpositive/rcfile_union.q | 0 .../src/test/queries/clientpositive/recursive_dir.q | 0 .../queries/clientpositive/reduce_deduplicate.q | 0 .../clientpositive/reduce_deduplicate_exclude_gby.q | 0 .../reduce_deduplicate_exclude_join.q | 0 .../clientpositive/reduce_deduplicate_extended.q | 0 .../test/queries/clientpositive/reducesink_dedup.q | 0 .../ql/src/test/queries/clientpositive/regex_col.q | 0 .../test/queries/clientpositive/regexp_extract.q | 0 .../src/test/queries/clientpositive/remote_script.q | 0 .../src/test/queries/clientpositive/rename_column.q | 0 .../rename_external_partition_location.q | 0 .../clientpositive/rename_partition_location.q | 0 .../queries/clientpositive/rename_table_location.q | 0 .../ql/src/test/queries/clientpositive/repair.q | 0 .../ql/src/test/queries/clientpositive/reset_conf.q | 0 .../test/queries/clientpositive/router_join_ppr.q | 0 .../ql/src/test/queries/clientpositive/sample1.q | 0 .../ql/src/test/queries/clientpositive/sample10.q | 0 .../ql/src/test/queries/clientpositive/sample2.q | 0 .../ql/src/test/queries/clientpositive/sample3.q | 0 .../ql/src/test/queries/clientpositive/sample4.q | 0 .../ql/src/test/queries/clientpositive/sample5.q | 0 .../ql/src/test/queries/clientpositive/sample6.q | 0 .../ql/src/test/queries/clientpositive/sample7.q | 0 .../ql/src/test/queries/clientpositive/sample8.q | 0 .../ql/src/test/queries/clientpositive/sample9.q | 0 .../clientpositive/sample_islocalmode_hook.q | 0 .../sample_islocalmode_hook_hadoop20.q | 0 .../test/queries/clientpositive/schemeAuthority.q | 0 .../test/queries/clientpositive/schemeAuthority2.q | 0 .../test/queries/clientpositive/script_env_var1.q | 0 .../test/queries/clientpositive/script_env_var2.q | 0 .../src/test/queries/clientpositive/script_pipe.q | 0 .../src/test/queries/clientpositive/scriptfile1.q | 0 .../test/queries/clientpositive/select_as_omitted.q | 0 .../queries/clientpositive/select_transform_hint.q | 0 .../queries/clientpositive/select_unquote_and.q | 0 .../queries/clientpositive/select_unquote_not.q | 0 .../test/queries/clientpositive/select_unquote_or.q | 0 .../ql/src/test/queries/clientpositive/semicolon.q | 0 .../ql/src/test/queries/clientpositive/semijoin.q | 0 .../src/test/queries/clientpositive/serde_regex.q | 0 .../queries/clientpositive/serde_reported_schema.q | 0 .../queries/clientpositive/serde_user_properties.q | 0 .../clientpositive/set_processor_namespaces.q | 0 .../test/queries/clientpositive/set_variable_sub.q | 0 .../src/test/queries/clientpositive/show_columns.q | 0 .../clientpositive/show_create_table_alter.q | 0 .../clientpositive/show_create_table_db_table.q | 0 .../clientpositive/show_create_table_delimited.q | 0 .../clientpositive/show_create_table_partitioned.q | 0 .../clientpositive/show_create_table_serde.q | 0 .../queries/clientpositive/show_create_table_view.q | 0 .../clientpositive/show_describe_func_quotes.q | 0 .../test/queries/clientpositive/show_functions.q | 0 .../clientpositive/show_indexes_edge_cases.q | 0 .../queries/clientpositive/show_indexes_syntax.q | 0 .../test/queries/clientpositive/show_partitions.q | 0 .../src/test/queries/clientpositive/show_tables.q | 0 .../test/queries/clientpositive/show_tablestatus.q | 0 .../queries/clientpositive/show_tblproperties.q | 0 .../ql/src/test/queries/clientpositive/showparts.q | 0 .../ql/src/test/queries/clientpositive/skewjoin.q | 0 .../clientpositive/skewjoin_union_remove_1.q | 0 .../clientpositive/skewjoin_union_remove_2.q | 0 .../src/test/queries/clientpositive/skewjoinopt1.q | 0 .../src/test/queries/clientpositive/skewjoinopt10.q | 0 .../src/test/queries/clientpositive/skewjoinopt11.q | 0 .../src/test/queries/clientpositive/skewjoinopt12.q | 0 .../src/test/queries/clientpositive/skewjoinopt13.q | 0 .../src/test/queries/clientpositive/skewjoinopt14.q | 0 .../src/test/queries/clientpositive/skewjoinopt15.q | 0 .../src/test/queries/clientpositive/skewjoinopt16.q | 0 .../src/test/queries/clientpositive/skewjoinopt17.q | 0 .../src/test/queries/clientpositive/skewjoinopt18.q | 0 .../src/test/queries/clientpositive/skewjoinopt19.q | 0 .../src/test/queries/clientpositive/skewjoinopt2.q | 0 .../src/test/queries/clientpositive/skewjoinopt20.q | 0 .../src/test/queries/clientpositive/skewjoinopt3.q | 0 .../src/test/queries/clientpositive/skewjoinopt4.q | 0 .../src/test/queries/clientpositive/skewjoinopt5.q | 0 .../src/test/queries/clientpositive/skewjoinopt6.q | 0 .../src/test/queries/clientpositive/skewjoinopt7.q | 0 .../src/test/queries/clientpositive/skewjoinopt8.q | 0 .../src/test/queries/clientpositive/skewjoinopt9.q | 0 .../src/test/queries/clientpositive/smb_mapjoin9.q | 0 .../src/test/queries/clientpositive/smb_mapjoin_1.q | 0 .../test/queries/clientpositive/smb_mapjoin_10.q | 0 .../test/queries/clientpositive/smb_mapjoin_11.q | 0 .../test/queries/clientpositive/smb_mapjoin_12.q | 0 .../test/queries/clientpositive/smb_mapjoin_13.q | 0 .../test/queries/clientpositive/smb_mapjoin_14.q | 0 .../test/queries/clientpositive/smb_mapjoin_15.q | 0 .../test/queries/clientpositive/smb_mapjoin_16.q | 0 .../test/queries/clientpositive/smb_mapjoin_17.q | 0 .../test/queries/clientpositive/smb_mapjoin_18.q | 0 .../test/queries/clientpositive/smb_mapjoin_19.q | 0 .../src/test/queries/clientpositive/smb_mapjoin_2.q | 0 .../test/queries/clientpositive/smb_mapjoin_20.q | 0 .../test/queries/clientpositive/smb_mapjoin_21.q | 0 .../test/queries/clientpositive/smb_mapjoin_22.q | 0 .../test/queries/clientpositive/smb_mapjoin_25.q | 0 .../src/test/queries/clientpositive/smb_mapjoin_3.q | 0 .../src/test/queries/clientpositive/smb_mapjoin_4.q | 0 .../src/test/queries/clientpositive/smb_mapjoin_5.q | 0 .../src/test/queries/clientpositive/smb_mapjoin_6.q | 0 .../src/test/queries/clientpositive/smb_mapjoin_7.q | 0 .../src/test/queries/clientpositive/smb_mapjoin_8.q | 0 .../hive/ql/src/test/queries/clientpositive/sort.q | 0 .../queries/clientpositive/sort_merge_join_desc_1.q | 0 .../queries/clientpositive/sort_merge_join_desc_2.q | 0 .../queries/clientpositive/sort_merge_join_desc_3.q | 0 .../queries/clientpositive/sort_merge_join_desc_4.q | 0 .../queries/clientpositive/sort_merge_join_desc_5.q | 0 .../queries/clientpositive/sort_merge_join_desc_6.q | 0 .../queries/clientpositive/sort_merge_join_desc_7.q | 0 .../ql/src/test/queries/clientpositive/source.q | 0 .../src/test/queries/clientpositive/split_sample.q | 0 .../ql/src/test/queries/clientpositive/stats0.q | 0 .../ql/src/test/queries/clientpositive/stats1.q | 0 .../ql/src/test/queries/clientpositive/stats10.q | 0 .../ql/src/test/queries/clientpositive/stats11.q | 0 .../ql/src/test/queries/clientpositive/stats12.q | 0 .../ql/src/test/queries/clientpositive/stats13.q | 0 .../ql/src/test/queries/clientpositive/stats14.q | 0 .../ql/src/test/queries/clientpositive/stats15.q | 0 .../ql/src/test/queries/clientpositive/stats16.q | 0 .../ql/src/test/queries/clientpositive/stats18.q | 0 .../ql/src/test/queries/clientpositive/stats19.q | 0 .../ql/src/test/queries/clientpositive/stats2.q | 0 .../ql/src/test/queries/clientpositive/stats20.q | 0 .../ql/src/test/queries/clientpositive/stats3.q | 0 .../ql/src/test/queries/clientpositive/stats4.q | 0 .../ql/src/test/queries/clientpositive/stats5.q | 0 .../ql/src/test/queries/clientpositive/stats6.q | 0 .../ql/src/test/queries/clientpositive/stats7.q | 0 .../ql/src/test/queries/clientpositive/stats8.q | 0 .../ql/src/test/queries/clientpositive/stats9.q | 0 .../clientpositive/stats_aggregator_error_1.q | 0 .../queries/clientpositive/stats_empty_dyn_part.q | 0 .../queries/clientpositive/stats_empty_partition.q | 0 .../test/queries/clientpositive/stats_noscan_1.q | 0 .../test/queries/clientpositive/stats_noscan_2.q | 0 .../test/queries/clientpositive/stats_partscan_1.q | 0 .../queries/clientpositive/stats_partscan_1_23.q | 0 .../clientpositive/stats_publisher_error_1.q | 0 .../ql/src/test/queries/clientpositive/str_to_map.q | 0 .../hive/ql/src/test/queries/clientpositive/subq.q | 0 .../hive/ql/src/test/queries/clientpositive/subq2.q | 0 .../clientpositive/symlink_text_input_format.q | 0 .../clientpositive/table_access_keys_stats.q | 0 .../queries/clientpositive/tablename_with_select.q | 0 .../clientpositive/test_boolean_whereclause.q | 0 .../src/test/queries/clientpositive/timestamp_1.q | 0 .../src/test/queries/clientpositive/timestamp_2.q | 0 .../src/test/queries/clientpositive/timestamp_3.q | 0 .../queries/clientpositive/timestamp_comparison.q | 0 .../test/queries/clientpositive/timestamp_lazy.q | 0 .../test/queries/clientpositive/timestamp_null.q | 0 .../src/test/queries/clientpositive/timestamp_udf.q | 0 .../hive/ql/src/test/queries/clientpositive/touch.q | 0 .../ql/src/test/queries/clientpositive/transform1.q | 0 .../ql/src/test/queries/clientpositive/transform2.q | 0 .../test/queries/clientpositive/transform_ppr1.q | 0 .../test/queries/clientpositive/transform_ppr2.q | 0 .../test/queries/clientpositive/truncate_column.q | 0 .../clientpositive/truncate_column_buckets.q | 0 .../clientpositive/truncate_column_list_bucket.q | 0 .../queries/clientpositive/truncate_column_merge.q | 0 .../test/queries/clientpositive/truncate_table.q | 0 .../src/test/queries/clientpositive/type_cast_1.q | 0 .../queries/clientpositive/type_conversions_1.q | 0 .../src/test/queries/clientpositive/type_widening.q | 0 .../test/queries/clientpositive/udaf_collect_set.q | 0 .../queries/clientpositive/udaf_context_ngrams.q | 0 .../ql/src/test/queries/clientpositive/udaf_corr.q | 0 .../test/queries/clientpositive/udaf_covar_pop.q | 0 .../test/queries/clientpositive/udaf_covar_samp.q | 0 .../queries/clientpositive/udaf_histogram_numeric.q | 0 .../src/test/queries/clientpositive/udaf_ngrams.q | 0 .../queries/clientpositive/udaf_number_format.q | 0 .../clientpositive/udaf_percentile_approx_20.q | 0 .../clientpositive/udaf_percentile_approx_23.q | 0 .../hive/ql/src/test/queries/clientpositive/udf1.q | 0 .../hive/ql/src/test/queries/clientpositive/udf2.q | 0 .../hive/ql/src/test/queries/clientpositive/udf3.q | 0 .../hive/ql/src/test/queries/clientpositive/udf4.q | 0 .../hive/ql/src/test/queries/clientpositive/udf5.q | 0 .../hive/ql/src/test/queries/clientpositive/udf6.q | 0 .../hive/ql/src/test/queries/clientpositive/udf7.q | 0 .../hive/ql/src/test/queries/clientpositive/udf8.q | 0 .../hive/ql/src/test/queries/clientpositive/udf9.q | 0 .../src/test/queries/clientpositive/udf_10_trims.q | 0 .../hive/ql/src/test/queries/clientpositive/udf_E.q | 0 .../ql/src/test/queries/clientpositive/udf_PI.q | 0 .../ql/src/test/queries/clientpositive/udf_abs.q | 0 .../ql/src/test/queries/clientpositive/udf_acos.q | 0 .../ql/src/test/queries/clientpositive/udf_add.q | 0 .../ql/src/test/queries/clientpositive/udf_array.q | 0 .../queries/clientpositive/udf_array_contains.q | 0 .../ql/src/test/queries/clientpositive/udf_ascii.q | 0 .../ql/src/test/queries/clientpositive/udf_asin.q | 0 .../ql/src/test/queries/clientpositive/udf_atan.q | 0 .../ql/src/test/queries/clientpositive/udf_avg.q | 0 .../src/test/queries/clientpositive/udf_between.q | 0 .../ql/src/test/queries/clientpositive/udf_bigint.q | 0 .../ql/src/test/queries/clientpositive/udf_bin.q | 0 .../test/queries/clientpositive/udf_bitmap_and.q | 0 .../test/queries/clientpositive/udf_bitmap_empty.q | 0 .../src/test/queries/clientpositive/udf_bitmap_or.q | 0 .../test/queries/clientpositive/udf_bitwise_and.q | 0 .../test/queries/clientpositive/udf_bitwise_not.q | 0 .../test/queries/clientpositive/udf_bitwise_or.q | 0 .../test/queries/clientpositive/udf_bitwise_xor.q | 0 .../src/test/queries/clientpositive/udf_boolean.q | 0 .../ql/src/test/queries/clientpositive/udf_case.q | 0 .../clientpositive/udf_case_column_pruning.q | 0 .../test/queries/clientpositive/udf_case_thrift.q | 0 .../ql/src/test/queries/clientpositive/udf_ceil.q | 0 .../src/test/queries/clientpositive/udf_ceiling.q | 0 .../src/test/queries/clientpositive/udf_coalesce.q | 0 .../clientpositive/udf_compare_java_string.q | 0 .../ql/src/test/queries/clientpositive/udf_concat.q | 0 .../queries/clientpositive/udf_concat_insert1.q | 0 .../queries/clientpositive/udf_concat_insert2.q | 0 .../src/test/queries/clientpositive/udf_concat_ws.q | 0 .../test/queries/clientpositive/udf_context_aware.q | 0 .../ql/src/test/queries/clientpositive/udf_conv.q | 0 .../ql/src/test/queries/clientpositive/udf_cos.q | 0 .../ql/src/test/queries/clientpositive/udf_count.q | 0 .../src/test/queries/clientpositive/udf_date_add.q | 0 .../src/test/queries/clientpositive/udf_date_sub.q | 0 .../src/test/queries/clientpositive/udf_datediff.q | 0 .../ql/src/test/queries/clientpositive/udf_day.q | 0 .../test/queries/clientpositive/udf_dayofmonth.q | 0 .../src/test/queries/clientpositive/udf_degrees.q | 0 .../ql/src/test/queries/clientpositive/udf_div.q | 0 .../ql/src/test/queries/clientpositive/udf_divide.q | 0 .../ql/src/test/queries/clientpositive/udf_double.q | 0 .../ql/src/test/queries/clientpositive/udf_elt.q | 0 .../ql/src/test/queries/clientpositive/udf_equal.q | 0 .../ql/src/test/queries/clientpositive/udf_exp.q | 0 .../src/test/queries/clientpositive/udf_explode.q | 0 .../ql/src/test/queries/clientpositive/udf_field.q | 0 .../test/queries/clientpositive/udf_find_in_set.q | 0 .../ql/src/test/queries/clientpositive/udf_float.q | 0 .../ql/src/test/queries/clientpositive/udf_floor.q | 0 .../test/queries/clientpositive/udf_format_number.q | 0 .../test/queries/clientpositive/udf_from_unixtime.q | 0 .../queries/clientpositive/udf_get_json_object.q | 0 .../test/queries/clientpositive/udf_greaterthan.q | 0 .../queries/clientpositive/udf_greaterthanorequal.q | 0 .../ql/src/test/queries/clientpositive/udf_hash.q | 0 .../ql/src/test/queries/clientpositive/udf_hex.q | 0 .../ql/src/test/queries/clientpositive/udf_hour.q | 0 .../ql/src/test/queries/clientpositive/udf_if.q | 0 .../ql/src/test/queries/clientpositive/udf_in.q | 0 .../src/test/queries/clientpositive/udf_in_file.q | 0 .../ql/src/test/queries/clientpositive/udf_index.q | 0 .../ql/src/test/queries/clientpositive/udf_inline.q | 0 .../ql/src/test/queries/clientpositive/udf_instr.q | 0 .../ql/src/test/queries/clientpositive/udf_int.q | 0 .../src/test/queries/clientpositive/udf_isnotnull.q | 0 .../ql/src/test/queries/clientpositive/udf_isnull.q | 0 .../queries/clientpositive/udf_isnull_isnotnull.q | 0 .../test/queries/clientpositive/udf_java_method.q | 0 .../ql/src/test/queries/clientpositive/udf_lcase.q | 0 .../ql/src/test/queries/clientpositive/udf_length.q | 0 .../src/test/queries/clientpositive/udf_lessthan.q | 0 .../queries/clientpositive/udf_lessthanorequal.q | 0 .../ql/src/test/queries/clientpositive/udf_like.q | 0 .../ql/src/test/queries/clientpositive/udf_ln.q | 0 .../ql/src/test/queries/clientpositive/udf_locate.q | 0 .../ql/src/test/queries/clientpositive/udf_log.q | 0 .../ql/src/test/queries/clientpositive/udf_log10.q | 0 .../ql/src/test/queries/clientpositive/udf_log2.q | 0 .../queries/clientpositive/udf_logic_java_boolean.q | 0 .../ql/src/test/queries/clientpositive/udf_lower.q | 0 .../ql/src/test/queries/clientpositive/udf_lpad.q | 0 .../ql/src/test/queries/clientpositive/udf_ltrim.q | 0 .../ql/src/test/queries/clientpositive/udf_map.q | 0 .../src/test/queries/clientpositive/udf_map_keys.q | 0 .../test/queries/clientpositive/udf_map_values.q | 0 .../ql/src/test/queries/clientpositive/udf_max.q | 0 .../ql/src/test/queries/clientpositive/udf_min.q | 0 .../ql/src/test/queries/clientpositive/udf_minute.q | 0 .../ql/src/test/queries/clientpositive/udf_modulo.q | 0 .../ql/src/test/queries/clientpositive/udf_month.q | 0 .../test/queries/clientpositive/udf_named_struct.q | 0 .../src/test/queries/clientpositive/udf_negative.q | 0 .../ql/src/test/queries/clientpositive/udf_not.q | 0 .../src/test/queries/clientpositive/udf_notequal.q | 0 .../ql/src/test/queries/clientpositive/udf_notop.q | 0 .../ql/src/test/queries/clientpositive/udf_nvl.q | 0 .../ql/src/test/queries/clientpositive/udf_or.q | 0 .../src/test/queries/clientpositive/udf_parse_url.q | 0 .../test/queries/clientpositive/udf_percentile.q | 0 .../ql/src/test/queries/clientpositive/udf_pmod.q | 0 .../src/test/queries/clientpositive/udf_positive.q | 0 .../ql/src/test/queries/clientpositive/udf_pow.q | 0 .../ql/src/test/queries/clientpositive/udf_power.q | 0 .../ql/src/test/queries/clientpositive/udf_printf.q | 0 .../src/test/queries/clientpositive/udf_radians.q | 0 .../ql/src/test/queries/clientpositive/udf_rand.q | 0 .../src/test/queries/clientpositive/udf_reflect.q | 0 .../src/test/queries/clientpositive/udf_reflect2.q | 0 .../ql/src/test/queries/clientpositive/udf_regexp.q | 0 .../queries/clientpositive/udf_regexp_extract.q | 0 .../queries/clientpositive/udf_regexp_replace.q | 0 .../ql/src/test/queries/clientpositive/udf_repeat.q | 0 .../src/test/queries/clientpositive/udf_reverse.q | 0 .../ql/src/test/queries/clientpositive/udf_rlike.q | 0 .../ql/src/test/queries/clientpositive/udf_round.q | 0 .../src/test/queries/clientpositive/udf_round_2.q | 0 .../src/test/queries/clientpositive/udf_round_3.q | 0 .../ql/src/test/queries/clientpositive/udf_rpad.q | 0 .../ql/src/test/queries/clientpositive/udf_rtrim.q | 0 .../ql/src/test/queries/clientpositive/udf_second.q | 0 .../src/test/queries/clientpositive/udf_sentences.q | 0 .../ql/src/test/queries/clientpositive/udf_sign.q | 0 .../ql/src/test/queries/clientpositive/udf_sin.q | 0 .../ql/src/test/queries/clientpositive/udf_size.q | 0 .../src/test/queries/clientpositive/udf_smallint.q | 0 .../test/queries/clientpositive/udf_sort_array.q | 0 .../ql/src/test/queries/clientpositive/udf_space.q | 0 .../ql/src/test/queries/clientpositive/udf_split.q | 0 .../ql/src/test/queries/clientpositive/udf_sqrt.q | 0 .../ql/src/test/queries/clientpositive/udf_std.q | 0 .../ql/src/test/queries/clientpositive/udf_stddev.q | 0 .../test/queries/clientpositive/udf_stddev_pop.q | 0 .../test/queries/clientpositive/udf_stddev_samp.q | 0 .../ql/src/test/queries/clientpositive/udf_string.q | 0 .../ql/src/test/queries/clientpositive/udf_struct.q | 0 .../ql/src/test/queries/clientpositive/udf_substr.q | 0 .../src/test/queries/clientpositive/udf_substring.q | 0 .../src/test/queries/clientpositive/udf_subtract.q | 0 .../ql/src/test/queries/clientpositive/udf_sum.q | 0 .../ql/src/test/queries/clientpositive/udf_tan.q | 0 .../test/queries/clientpositive/udf_testlength.q | 0 .../test/queries/clientpositive/udf_testlength2.q | 0 .../src/test/queries/clientpositive/udf_tinyint.q | 0 .../test/queries/clientpositive/udf_to_boolean.q | 0 .../src/test/queries/clientpositive/udf_to_byte.q | 0 .../src/test/queries/clientpositive/udf_to_date.q | 0 .../src/test/queries/clientpositive/udf_to_double.q | 0 .../src/test/queries/clientpositive/udf_to_float.q | 0 .../src/test/queries/clientpositive/udf_to_long.q | 0 .../src/test/queries/clientpositive/udf_to_short.q | 0 .../src/test/queries/clientpositive/udf_to_string.q | 0 .../queries/clientpositive/udf_to_unix_timestamp.q | 0 .../src/test/queries/clientpositive/udf_translate.q | 0 .../ql/src/test/queries/clientpositive/udf_trim.q | 0 .../ql/src/test/queries/clientpositive/udf_ucase.q | 0 .../ql/src/test/queries/clientpositive/udf_unhex.q | 0 .../ql/src/test/queries/clientpositive/udf_union.q | 0 .../queries/clientpositive/udf_unix_timestamp.q | 0 .../ql/src/test/queries/clientpositive/udf_upper.q | 0 .../src/test/queries/clientpositive/udf_var_pop.q | 0 .../src/test/queries/clientpositive/udf_var_samp.q | 0 .../src/test/queries/clientpositive/udf_variance.q | 0 .../test/queries/clientpositive/udf_weekofyear.q | 0 .../ql/src/test/queries/clientpositive/udf_when.q | 0 .../ql/src/test/queries/clientpositive/udf_xpath.q | 0 .../test/queries/clientpositive/udf_xpath_boolean.q | 0 .../test/queries/clientpositive/udf_xpath_double.q | 0 .../test/queries/clientpositive/udf_xpath_float.q | 0 .../src/test/queries/clientpositive/udf_xpath_int.q | 0 .../test/queries/clientpositive/udf_xpath_long.q | 0 .../test/queries/clientpositive/udf_xpath_short.q | 0 .../test/queries/clientpositive/udf_xpath_string.q | 0 .../src/test/queries/clientpositive/udtf_explode.q | 0 .../test/queries/clientpositive/udtf_json_tuple.q | 0 .../queries/clientpositive/udtf_parse_url_tuple.q | 0 .../ql/src/test/queries/clientpositive/udtf_stack.q | 0 .../test/queries/clientpositive/unicode_notation.q | 0 .../hive/ql/src/test/queries/clientpositive/union.q | 0 .../ql/src/test/queries/clientpositive/union10.q | 0 .../ql/src/test/queries/clientpositive/union11.q | 0 .../ql/src/test/queries/clientpositive/union12.q | 0 .../ql/src/test/queries/clientpositive/union13.q | 0 .../ql/src/test/queries/clientpositive/union14.q | 0 .../ql/src/test/queries/clientpositive/union15.q | 0 .../ql/src/test/queries/clientpositive/union16.q | 0 .../ql/src/test/queries/clientpositive/union17.q | 0 .../ql/src/test/queries/clientpositive/union18.q | 0 .../ql/src/test/queries/clientpositive/union19.q | 0 .../ql/src/test/queries/clientpositive/union2.q | 0 .../ql/src/test/queries/clientpositive/union20.q | 0 .../ql/src/test/queries/clientpositive/union21.q | 0 .../ql/src/test/queries/clientpositive/union22.q | 0 .../ql/src/test/queries/clientpositive/union23.q | 0 .../ql/src/test/queries/clientpositive/union24.q | 0 .../ql/src/test/queries/clientpositive/union25.q | 0 .../ql/src/test/queries/clientpositive/union26.q | 0 .../ql/src/test/queries/clientpositive/union27.q | 0 .../ql/src/test/queries/clientpositive/union28.q | 0 .../ql/src/test/queries/clientpositive/union29.q | 0 .../ql/src/test/queries/clientpositive/union3.q | 0 .../ql/src/test/queries/clientpositive/union30.q | 0 .../ql/src/test/queries/clientpositive/union31.q | 0 .../ql/src/test/queries/clientpositive/union32.q | 0 .../ql/src/test/queries/clientpositive/union33.q | 0 .../ql/src/test/queries/clientpositive/union34.q | 0 .../ql/src/test/queries/clientpositive/union4.q | 0 .../ql/src/test/queries/clientpositive/union5.q | 0 .../ql/src/test/queries/clientpositive/union6.q | 0 .../ql/src/test/queries/clientpositive/union7.q | 0 .../ql/src/test/queries/clientpositive/union8.q | 0 .../ql/src/test/queries/clientpositive/union9.q | 0 .../ql/src/test/queries/clientpositive/union_date.q | 0 .../test/queries/clientpositive/union_lateralview.q | 0 .../ql/src/test/queries/clientpositive/union_null.q | 0 .../ql/src/test/queries/clientpositive/union_ppr.q | 0 .../test/queries/clientpositive/union_remove_1.q | 0 .../test/queries/clientpositive/union_remove_10.q | 0 .../test/queries/clientpositive/union_remove_11.q | 0 .../test/queries/clientpositive/union_remove_12.q | 0 .../test/queries/clientpositive/union_remove_13.q | 0 .../test/queries/clientpositive/union_remove_14.q | 0 .../test/queries/clientpositive/union_remove_15.q | 0 .../test/queries/clientpositive/union_remove_16.q | 0 .../test/queries/clientpositive/union_remove_17.q | 0 .../test/queries/clientpositive/union_remove_18.q | 0 .../test/queries/clientpositive/union_remove_19.q | 0 .../test/queries/clientpositive/union_remove_2.q | 0 .../test/queries/clientpositive/union_remove_20.q | 0 .../test/queries/clientpositive/union_remove_21.q | 0 .../test/queries/clientpositive/union_remove_22.q | 0 .../test/queries/clientpositive/union_remove_23.q | 0 .../test/queries/clientpositive/union_remove_24.q | 0 .../test/queries/clientpositive/union_remove_3.q | 0 .../test/queries/clientpositive/union_remove_4.q | 0 .../test/queries/clientpositive/union_remove_5.q | 0 .../test/queries/clientpositive/union_remove_6.q | 0 .../test/queries/clientpositive/union_remove_7.q | 0 .../test/queries/clientpositive/union_remove_8.q | 0 .../test/queries/clientpositive/union_remove_9.q | 0 .../src/test/queries/clientpositive/union_script.q | 0 .../ql/src/test/queries/clientpositive/union_view.q | 0 .../ql/src/test/queries/clientpositive/uniquejoin.q | 0 .../clientpositive/unset_table_view_property.q | 0 .../test/queries/clientpositive/updateAccessTime.q | 0 .../ql/src/test/queries/clientpositive/varchar_1.q | 0 .../ql/src/test/queries/clientpositive/varchar_2.q | 0 .../src/test/queries/clientpositive/varchar_cast.q | 0 .../queries/clientpositive/varchar_comparison.q | 0 .../src/test/queries/clientpositive/varchar_join1.q | 0 .../queries/clientpositive/varchar_nested_types.q | 0 .../src/test/queries/clientpositive/varchar_serde.q | 0 .../src/test/queries/clientpositive/varchar_udf1.q | 0 .../test/queries/clientpositive/varchar_union1.q | 0 .../hive/ql/src/test/queries/clientpositive/view.q | 0 .../ql/src/test/queries/clientpositive/view_cast.q | 0 .../src/test/queries/clientpositive/view_inputs.q | 0 .../test/queries/clientpositive/virtual_column.q | 0 .../ql/src/test/queries/clientpositive/windowing.q | 0 .../windowing_adjust_rowcontainer_sz.q | 0 .../clientpositive/windowing_columnPruning.q | 0 .../queries/clientpositive/windowing_expressions.q | 0 .../clientpositive/windowing_multipartitioning.q | 0 .../test/queries/clientpositive/windowing_navfn.q | 0 .../test/queries/clientpositive/windowing_ntile.q | 0 .../test/queries/clientpositive/windowing_rank.q | 0 .../test/queries/clientpositive/windowing_udaf.q | 0 .../queries/clientpositive/windowing_windowspec.q | 0 .../src/test/queries/negative/ambiguous_join_col.q | 0 .../ql/src/test/queries/negative/duplicate_alias.q | 0 .../hive/ql/src/test/queries/negative/garbage.q | 0 .../queries/negative/insert_wrong_number_columns.q | 0 .../test/queries/negative/invalid_create_table.q | 0 .../hive/ql/src/test/queries/negative/invalid_dot.q | 0 .../test/queries/negative/invalid_function_param2.q | 0 .../ql/src/test/queries/negative/invalid_index.q | 0 .../src/test/queries/negative/invalid_list_index.q | 0 .../src/test/queries/negative/invalid_list_index2.q | 0 .../src/test/queries/negative/invalid_map_index.q | 0 .../src/test/queries/negative/invalid_map_index2.q | 0 .../ql/src/test/queries/negative/invalid_select.q | 0 .../src/test/queries/negative/macro_reserved_word.q | 0 .../src/test/queries/negative/missing_overwrite.q | 0 .../ql/src/test/queries/negative/nonkey_groupby.q | 0 .../ql/src/test/queries/negative/quoted_string.q | 0 .../ql/src/test/queries/negative/unknown_column1.q | 0 .../ql/src/test/queries/negative/unknown_column2.q | 0 .../ql/src/test/queries/negative/unknown_column3.q | 0 .../ql/src/test/queries/negative/unknown_column4.q | 0 .../ql/src/test/queries/negative/unknown_column5.q | 0 .../ql/src/test/queries/negative/unknown_column6.q | 0 .../src/test/queries/negative/unknown_function1.q | 0 .../src/test/queries/negative/unknown_function2.q | 0 .../src/test/queries/negative/unknown_function3.q | 0 .../src/test/queries/negative/unknown_function4.q | 0 .../ql/src/test/queries/negative/unknown_table1.q | 0 .../ql/src/test/queries/negative/unknown_table2.q | 0 .../ql/src/test/queries/negative/wrong_distinct1.q | 0 .../ql/src/test/queries/negative/wrong_distinct2.q | 0 .../ql/src/test/queries/positive/case_sensitivity.q | 0 .../test/hive/ql/src/test/queries/positive/cast1.q | 0 .../hive/ql/src/test/queries/positive/groupby1.q | 0 .../hive/ql/src/test/queries/positive/groupby2.q | 0 .../hive/ql/src/test/queries/positive/groupby3.q | 0 .../hive/ql/src/test/queries/positive/groupby4.q | 0 .../hive/ql/src/test/queries/positive/groupby5.q | 0 .../hive/ql/src/test/queries/positive/groupby6.q | 0 .../test/hive/ql/src/test/queries/positive/input1.q | 0 .../test/hive/ql/src/test/queries/positive/input2.q | 0 .../hive/ql/src/test/queries/positive/input20.q | 0 .../test/hive/ql/src/test/queries/positive/input3.q | 0 .../test/hive/ql/src/test/queries/positive/input4.q | 0 .../test/hive/ql/src/test/queries/positive/input5.q | 0 .../test/hive/ql/src/test/queries/positive/input6.q | 0 .../test/hive/ql/src/test/queries/positive/input7.q | 0 .../test/hive/ql/src/test/queries/positive/input8.q | 0 .../test/hive/ql/src/test/queries/positive/input9.q | 0 .../hive/ql/src/test/queries/positive/input_part1.q | 0 .../test/queries/positive/input_testsequencefile.q | 0 .../ql/src/test/queries/positive/input_testxpath.q | 0 .../ql/src/test/queries/positive/input_testxpath2.q | 0 .../test/hive/ql/src/test/queries/positive/join1.q | 0 .../test/hive/ql/src/test/queries/positive/join2.q | 0 .../test/hive/ql/src/test/queries/positive/join3.q | 0 .../test/hive/ql/src/test/queries/positive/join4.q | 0 .../test/hive/ql/src/test/queries/positive/join5.q | 0 .../test/hive/ql/src/test/queries/positive/join6.q | 0 .../test/hive/ql/src/test/queries/positive/join7.q | 0 .../test/hive/ql/src/test/queries/positive/join8.q | 0 .../hive/ql/src/test/queries/positive/sample1.q | 0 .../hive/ql/src/test/queries/positive/sample2.q | 0 .../hive/ql/src/test/queries/positive/sample3.q | 0 .../hive/ql/src/test/queries/positive/sample4.q | 0 .../hive/ql/src/test/queries/positive/sample5.q | 0 .../hive/ql/src/test/queries/positive/sample6.q | 0 .../hive/ql/src/test/queries/positive/sample7.q | 0 .../test/hive/ql/src/test/queries/positive/subq.q | 0 .../test/hive/ql/src/test/queries/positive/udf1.q | 0 .../test/hive/ql/src/test/queries/positive/udf4.q | 0 .../test/hive/ql/src/test/queries/positive/udf6.q | 0 .../hive/ql/src/test/queries/positive/udf_case.q | 0 .../hive/ql/src/test/queries/positive/udf_when.q | 0 .../test/hive/ql/src/test/queries/positive/union.q | 0 11261 files changed, 0 insertions(+), 0 deletions(-) rename {src => shark/src}/test/hive/data/conf/hive-log4j.properties (100%) rename {src => shark/src}/test/hive/data/conf/hive-site.xml (100%) rename {src => shark/src}/test/hive/data/files/SortCol1Col2.txt (100%) rename {src => shark/src}/test/hive/data/files/SortCol2Col1.txt (100%) rename {src => shark/src}/test/hive/data/files/SortDescCol1Col2.txt (100%) rename {src => shark/src}/test/hive/data/files/SortDescCol2Col1.txt (100%) rename {src => shark/src}/test/hive/data/files/T1.txt (100%) rename {src => shark/src}/test/hive/data/files/T2.txt (100%) rename {src => shark/src}/test/hive/data/files/T3.txt (100%) rename {src => shark/src}/test/hive/data/files/TestSerDe.jar (100%) rename {src => shark/src}/test/hive/data/files/UserVisits.dat (100%) rename {src => shark/src}/test/hive/data/files/apache.access.2.log (100%) rename {src => shark/src}/test/hive/data/files/apache.access.log (100%) rename {src => shark/src}/test/hive/data/files/archive_corrupt.rc (100%) rename {src => shark/src}/test/hive/data/files/array_table.txt (100%) rename {src => shark/src}/test/hive/data/files/binary.txt (100%) rename {src => shark/src}/test/hive/data/files/bool.txt (100%) rename {src => shark/src}/test/hive/data/files/complex.seq (100%) rename {src => shark/src}/test/hive/data/files/covar_tab.txt (100%) rename {src => shark/src}/test/hive/data/files/create_nested_type.txt (100%) rename {src => shark/src}/test/hive/data/files/csv.txt (100%) rename {src => shark/src}/test/hive/data/files/datatypes.txt (100%) rename {src => shark/src}/test/hive/data/files/dim-data.txt (100%) rename {src => shark/src}/test/hive/data/files/doctors.avro (100%) rename {src => shark/src}/test/hive/data/files/docurl.txt (100%) rename {src => shark/src}/test/hive/data/files/double.txt (100%) rename {src => shark/src}/test/hive/data/files/employee.dat (100%) rename {src => shark/src}/test/hive/data/files/employee2.dat (100%) rename {src => shark/src}/test/hive/data/files/employee_part.txt (100%) rename {src => shark/src}/test/hive/data/files/empty1.txt (100%) rename {src => shark/src}/test/hive/data/files/empty2.txt (100%) rename {src => shark/src}/test/hive/data/files/episodes.avro (100%) rename {src => shark/src}/test/hive/data/files/escapetest.txt (100%) rename {src => shark/src}/test/hive/data/files/ext_test/test.dat (100%) rename {src => shark/src}/test/hive/data/files/fact-data.txt (100%) rename {src => shark/src}/test/hive/data/files/flights_join.txt (100%) rename {src => shark/src}/test/hive/data/files/flights_tiny.txt (100%) rename {src => shark/src}/test/hive/data/files/flights_tiny.txt.1 (100%) rename {src => shark/src}/test/hive/data/files/groupby_groupingid.txt (100%) rename {src => shark/src}/test/hive/data/files/grouping_sets.txt (100%) rename {src => shark/src}/test/hive/data/files/grouping_sets1.txt (100%) rename {src => shark/src}/test/hive/data/files/grouping_sets2.txt (100%) rename {src => shark/src}/test/hive/data/files/hive_626_bar.txt (100%) rename {src => shark/src}/test/hive/data/files/hive_626_count.txt (100%) rename {src => shark/src}/test/hive/data/files/hive_626_foo.txt (100%) rename {src => shark/src}/test/hive/data/files/in1.txt (100%) rename {src => shark/src}/test/hive/data/files/in2.txt (100%) rename {src => shark/src}/test/hive/data/files/in3.txt (100%) rename {src => shark/src}/test/hive/data/files/in4.txt (100%) rename {src => shark/src}/test/hive/data/files/in5.txt (100%) rename {src => shark/src}/test/hive/data/files/in6.txt (100%) rename {src => shark/src}/test/hive/data/files/in7.txt (100%) rename {src => shark/src}/test/hive/data/files/in8.txt (100%) rename {src => shark/src}/test/hive/data/files/in9.txt (100%) rename {src => shark/src}/test/hive/data/files/infer_const_type.txt (100%) rename {src => shark/src}/test/hive/data/files/int.txt (100%) rename {src => shark/src}/test/hive/data/files/json.txt (100%) rename {src => shark/src}/test/hive/data/files/kv1.seq (100%) rename {src => shark/src}/test/hive/data/files/kv1.string-sorted.txt (100%) rename {src => shark/src}/test/hive/data/files/kv1.txt (100%) rename {src => shark/src}/test/hive/data/files/kv1.val.sorted.txt (100%) rename {src => shark/src}/test/hive/data/files/kv1_broken.seq (100%) rename {src => shark/src}/test/hive/data/files/kv1_cb.txt (100%) rename {src => shark/src}/test/hive/data/files/kv1_cc.txt (100%) rename {src => shark/src}/test/hive/data/files/kv1kv2.cogroup.txt (100%) rename {src => shark/src}/test/hive/data/files/kv2.txt (100%) rename {src => shark/src}/test/hive/data/files/kv3.txt (100%) rename {src => shark/src}/test/hive/data/files/kv4.txt (100%) rename {src => shark/src}/test/hive/data/files/kv5.txt (100%) rename {src => shark/src}/test/hive/data/files/kv6.txt (100%) rename {src => shark/src}/test/hive/data/files/kv7.txt (100%) rename {src => shark/src}/test/hive/data/files/kv8.txt (100%) rename {src => shark/src}/test/hive/data/files/leftsemijoin_mr_t1.txt (100%) rename {src => shark/src}/test/hive/data/files/leftsemijoin_mr_t2.txt (100%) rename {src => shark/src}/test/hive/data/files/lineitem.txt (100%) rename {src => shark/src}/test/hive/data/files/lt100.sorted.txt (100%) rename {src => shark/src}/test/hive/data/files/lt100.txt (100%) rename {src => shark/src}/test/hive/data/files/lt100.txt.deflate (100%) rename {src => shark/src}/test/hive/data/files/map_table.txt (100%) rename {src => shark/src}/test/hive/data/files/nested_complex.txt (100%) rename {src => shark/src}/test/hive/data/files/null.txt (100%) rename {src => shark/src}/test/hive/data/files/nullfile.txt (100%) rename {src => shark/src}/test/hive/data/files/nulls.txt (100%) rename {src => shark/src}/test/hive/data/files/orc_create.txt (100%) rename {src => shark/src}/test/hive/data/files/orc_create_people.txt (100%) rename {src => shark/src}/test/hive/data/files/over10k (100%) rename {src => shark/src}/test/hive/data/files/over1k (100%) rename {src => shark/src}/test/hive/data/files/part.rc (100%) rename {src => shark/src}/test/hive/data/files/part.seq (100%) rename {src => shark/src}/test/hive/data/files/part_tiny.txt (100%) rename {src => shark/src}/test/hive/data/files/person age.txt (100%) rename {src => shark/src}/test/hive/data/files/primitive_type_arrays.txt (100%) rename {src => shark/src}/test/hive/data/files/pw17.txt (100%) rename {src => shark/src}/test/hive/data/files/sales.txt (100%) rename {src => shark/src}/test/hive/data/files/sample-queryplan-in-history.txt (100%) rename {src => shark/src}/test/hive/data/files/sample-queryplan.txt (100%) rename {src => shark/src}/test/hive/data/files/smallsrcsortbucket1outof4.txt (100%) rename {src => shark/src}/test/hive/data/files/smallsrcsortbucket2outof4.txt (100%) rename {src => shark/src}/test/hive/data/files/smallsrcsortbucket3outof4.txt (100%) rename {src => shark/src}/test/hive/data/files/smallsrcsortbucket4outof4.txt (100%) rename {src => shark/src}/test/hive/data/files/smb_bucket_input.rc (100%) rename {src => shark/src}/test/hive/data/files/smb_bucket_input.txt (100%) rename {src => shark/src}/test/hive/data/files/smbbucket_1.rc (100%) rename {src => shark/src}/test/hive/data/files/smbbucket_1.txt (100%) rename {src => shark/src}/test/hive/data/files/smbbucket_2.rc (100%) rename {src => shark/src}/test/hive/data/files/smbbucket_2.txt (100%) rename {src => shark/src}/test/hive/data/files/smbbucket_3.rc (100%) rename {src => shark/src}/test/hive/data/files/smbbucket_3.txt (100%) rename {src => shark/src}/test/hive/data/files/source.txt (100%) rename {src => shark/src}/test/hive/data/files/srcbucket0.txt (100%) rename {src => shark/src}/test/hive/data/files/srcbucket1.txt (100%) rename {src => shark/src}/test/hive/data/files/srcbucket20.txt (100%) rename {src => shark/src}/test/hive/data/files/srcbucket21.txt (100%) rename {src => shark/src}/test/hive/data/files/srcbucket22.txt (100%) rename {src => shark/src}/test/hive/data/files/srcbucket23.txt (100%) rename {src => shark/src}/test/hive/data/files/srcsortbucket1outof4.txt (100%) rename {src => shark/src}/test/hive/data/files/srcsortbucket2outof4.txt (100%) rename {src => shark/src}/test/hive/data/files/srcsortbucket3outof4.txt (100%) rename {src => shark/src}/test/hive/data/files/srcsortbucket4outof4.txt (100%) rename {src => shark/src}/test/hive/data/files/string.txt (100%) rename {src => shark/src}/test/hive/data/files/symlink1.txt (100%) rename {src => shark/src}/test/hive/data/files/symlink2.txt (100%) rename {src => shark/src}/test/hive/data/files/tbl.txt (100%) rename {src => shark/src}/test/hive/data/files/test.dat (100%) rename {src => shark/src}/test/hive/data/files/test2.dat (100%) rename {src => shark/src}/test/hive/data/files/text-en.txt (100%) rename {src => shark/src}/test/hive/data/files/things.txt (100%) rename {src => shark/src}/test/hive/data/files/things2.txt (100%) rename {src => shark/src}/test/hive/data/files/tiny_a.txt (100%) rename {src => shark/src}/test/hive/data/files/tiny_b.txt (100%) rename {src => shark/src}/test/hive/data/files/types/primitives/090101.txt (100%) rename {src => shark/src}/test/hive/data/files/types/primitives/090201.txt (100%) rename {src => shark/src}/test/hive/data/files/types/primitives/090301.txt (100%) rename {src => shark/src}/test/hive/data/files/types/primitives/090401.txt (100%) rename {src => shark/src}/test/hive/data/files/union_input.txt (100%) rename {src => shark/src}/test/hive/data/files/v1.txt (100%) rename {src => shark/src}/test/hive/data/files/v2.txt (100%) rename {src => shark/src}/test/hive/data/files/vc1.txt (100%) rename {src => shark/src}/test/hive/data/files/x.txt (100%) rename {src => shark/src}/test/hive/data/files/y.txt (100%) rename {src => shark/src}/test/hive/data/files/z.txt (100%) rename {src => shark/src}/test/hive/data/metadb/.gitignore (100%) rename {src => shark/src}/test/hive/data/scripts/cat.py (100%) rename {src => shark/src}/test/hive/data/scripts/cat_error.py (100%) rename {src => shark/src}/test/hive/data/scripts/doubleescapedtab.py (100%) rename {src => shark/src}/test/hive/data/scripts/dumpdata_script.py (100%) rename {src => shark/src}/test/hive/data/scripts/error_script (100%) rename {src => shark/src}/test/hive/data/scripts/escapedcarriagereturn.py (100%) rename {src => shark/src}/test/hive/data/scripts/escapednewline.py (100%) rename {src => shark/src}/test/hive/data/scripts/escapedtab.py (100%) rename {src => shark/src}/test/hive/data/scripts/input20_script (100%) rename {src => shark/src}/test/hive/data/scripts/newline.py (100%) rename {src => shark/src}/test/hive/data/scripts/q_test_cleanup.sql (100%) rename {src => shark/src}/test/hive/data/scripts/q_test_init.sql (100%) rename {src => shark/src}/test/hive/data/scripts/test_init_file.sql (100%) rename {src => shark/src}/test/hive/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb (100%) rename {src => shark/src}/test/hive/golden/'1' + 1-0-130514c6116c311d808590a075b187b (100%) rename {src => shark/src}/test/hive/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e (100%) rename {src => shark/src}/test/hive/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 (100%) rename {src => shark/src}/test/hive/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 (100%) rename {src => shark/src}/test/hive/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a (100%) rename {src => shark/src}/test/hive/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a (100%) rename {src => shark/src}/test/hive/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 (100%) rename {src => shark/src}/test/hive/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 (100%) rename {src => shark/src}/test/hive/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e (100%) rename {src => shark/src}/test/hive/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e (100%) rename {src => shark/src}/test/hive/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f (100%) rename {src => shark/src}/test/hive/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 (100%) rename {src => shark/src}/test/hive/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 (100%) rename {src => shark/src}/test/hive/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d (100%) rename {src => shark/src}/test/hive/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 (100%) rename {src => shark/src}/test/hive/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 (100%) rename {src => shark/src}/test/hive/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 (100%) rename {src => shark/src}/test/hive/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a (100%) rename {src => shark/src}/test/hive/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d (100%) rename {src => shark/src}/test/hive/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 (100%) rename {src => shark/src}/test/hive/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f (100%) rename {src => shark/src}/test/hive/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 (100%) rename {src => shark/src}/test/hive/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 (100%) rename {src => shark/src}/test/hive/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 (100%) rename {src => shark/src}/test/hive/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 (100%) rename {src => shark/src}/test/hive/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e (100%) rename {src => shark/src}/test/hive/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 (100%) rename {src => shark/src}/test/hive/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e (100%) rename {src => shark/src}/test/hive/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b (100%) rename {src => shark/src}/test/hive/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb (100%) rename {src => shark/src}/test/hive/golden/1Y + 1-0-a4541db51882b19503649138fbb295f (100%) rename {src => shark/src}/test/hive/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d (100%) rename {src => shark/src}/test/hive/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 (100%) rename {src => shark/src}/test/hive/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 (100%) rename {src => shark/src}/test/hive/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 (100%) rename {src => shark/src}/test/hive/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a (100%) rename {src => shark/src}/test/hive/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 (100%) rename {src => shark/src}/test/hive/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 (100%) rename {src => shark/src}/test/hive/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 (100%) rename {src => shark/src}/test/hive/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 (100%) rename {src => shark/src}/test/hive/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 (100%) rename {src => shark/src}/test/hive/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa (100%) rename {src => shark/src}/test/hive/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 (100%) rename {src => shark/src}/test/hive/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 (100%) rename {src => shark/src}/test/hive/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb (100%) rename {src => shark/src}/test/hive/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b (100%) rename {src => shark/src}/test/hive/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 (100%) rename {src => shark/src}/test/hive/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 (100%) rename {src => shark/src}/test/hive/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 (100%) rename {src => shark/src}/test/hive/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef (100%) rename {src => shark/src}/test/hive/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c (100%) rename {src => shark/src}/test/hive/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 (100%) rename {src => shark/src}/test/hive/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af (100%) rename {src => shark/src}/test/hive/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 (100%) rename {src => shark/src}/test/hive/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f (100%) rename {src => shark/src}/test/hive/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 (100%) rename {src => shark/src}/test/hive/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 (100%) rename {src => shark/src}/test/hive/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 (100%) rename {src => shark/src}/test/hive/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 (100%) rename {src => shark/src}/test/hive/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d (100%) rename {src => shark/src}/test/hive/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 (100%) rename {src => shark/src}/test/hive/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 (100%) rename {src => shark/src}/test/hive/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 (100%) rename {src => shark/src}/test/hive/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 (100%) rename {src => shark/src}/test/hive/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 (100%) rename {src => shark/src}/test/hive/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da (100%) rename {src => shark/src}/test/hive/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 (100%) rename {src => shark/src}/test/hive/golden/alter2-12-75a213649242c2410ea6846f08c91d75 (100%) rename {src => shark/src}/test/hive/golden/alter2-13-ca51e03a3de391983429b6ad877e573c (100%) rename {src => shark/src}/test/hive/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 (100%) rename {src => shark/src}/test/hive/golden/alter2-15-75a213649242c2410ea6846f08c91d75 (100%) rename {src => shark/src}/test/hive/golden/alter2-16-77500c83ffcece95511a4b21d67382dd (100%) rename {src => shark/src}/test/hive/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 (100%) rename {src => shark/src}/test/hive/golden/alter2-18-75a213649242c2410ea6846f08c91d75 (100%) rename {src => shark/src}/test/hive/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 (100%) rename {src => shark/src}/test/hive/golden/alter2-2-75a213649242c2410ea6846f08c91d75 (100%) rename {src => shark/src}/test/hive/golden/alter2-20-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 (100%) rename {src => shark/src}/test/hive/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 (100%) rename {src => shark/src}/test/hive/golden/alter2-23-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter2-24-775d69742a1c07df8da87e8a017d955 (100%) rename {src => shark/src}/test/hive/golden/alter2-25-4ef75e12575453225738ea167c4617e5 (100%) rename {src => shark/src}/test/hive/golden/alter2-26-63f13c364546ddce5d2176c6604a948f (100%) rename {src => shark/src}/test/hive/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f (100%) rename {src => shark/src}/test/hive/golden/alter2-28-4ef75e12575453225738ea167c4617e5 (100%) rename {src => shark/src}/test/hive/golden/alter2-29-63f13c364546ddce5d2176c6604a948f (100%) rename {src => shark/src}/test/hive/golden/alter2-3-ca51e03a3de391983429b6ad877e573c (100%) rename {src => shark/src}/test/hive/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae (100%) rename {src => shark/src}/test/hive/golden/alter2-31-4ef75e12575453225738ea167c4617e5 (100%) rename {src => shark/src}/test/hive/golden/alter2-32-63f13c364546ddce5d2176c6604a948f (100%) rename {src => shark/src}/test/hive/golden/alter2-33-1934026d0228967097280eed35551f74 (100%) rename {src => shark/src}/test/hive/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b (100%) rename {src => shark/src}/test/hive/golden/alter2-35-4ef75e12575453225738ea167c4617e5 (100%) rename {src => shark/src}/test/hive/golden/alter2-36-63f13c364546ddce5d2176c6604a948f (100%) rename {src => shark/src}/test/hive/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f (100%) rename {src => shark/src}/test/hive/golden/alter2-38-4ef75e12575453225738ea167c4617e5 (100%) rename {src => shark/src}/test/hive/golden/alter2-39-63f13c364546ddce5d2176c6604a948f (100%) rename {src => shark/src}/test/hive/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 (100%) rename {src => shark/src}/test/hive/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae (100%) rename {src => shark/src}/test/hive/golden/alter2-41-4ef75e12575453225738ea167c4617e5 (100%) rename {src => shark/src}/test/hive/golden/alter2-42-63f13c364546ddce5d2176c6604a948f (100%) rename {src => shark/src}/test/hive/golden/alter2-43-1934026d0228967097280eed35551f74 (100%) rename {src => shark/src}/test/hive/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 (100%) rename {src => shark/src}/test/hive/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c (100%) rename {src => shark/src}/test/hive/golden/alter2-5-75a213649242c2410ea6846f08c91d75 (100%) rename {src => shark/src}/test/hive/golden/alter2-6-77500c83ffcece95511a4b21d67382dd (100%) rename {src => shark/src}/test/hive/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 (100%) rename {src => shark/src}/test/hive/golden/alter2-8-75a213649242c2410ea6846f08c91d75 (100%) rename {src => shark/src}/test/hive/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e (100%) rename {src => shark/src}/test/hive/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade (100%) rename {src => shark/src}/test/hive/golden/alter3-1-75be487df30e301e156a22eee075633d (100%) rename {src => shark/src}/test/hive/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 (100%) rename {src => shark/src}/test/hive/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 (100%) rename {src => shark/src}/test/hive/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 (100%) rename {src => shark/src}/test/hive/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b (100%) rename {src => shark/src}/test/hive/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f (100%) rename {src => shark/src}/test/hive/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 (100%) rename {src => shark/src}/test/hive/golden/alter3-16-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b (100%) rename {src => shark/src}/test/hive/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d (100%) rename {src => shark/src}/test/hive/golden/alter3-19-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f (100%) rename {src => shark/src}/test/hive/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 (100%) rename {src => shark/src}/test/hive/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 (100%) rename {src => shark/src}/test/hive/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 (100%) rename {src => shark/src}/test/hive/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 (100%) rename {src => shark/src}/test/hive/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 (100%) rename {src => shark/src}/test/hive/golden/alter3-25-568a59760e5d3241b63d65cce595face (100%) rename {src => shark/src}/test/hive/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e (100%) rename {src => shark/src}/test/hive/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 (100%) rename {src => shark/src}/test/hive/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 (100%) rename {src => shark/src}/test/hive/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c (100%) rename {src => shark/src}/test/hive/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 (100%) rename {src => shark/src}/test/hive/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 (100%) rename {src => shark/src}/test/hive/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 (100%) rename {src => shark/src}/test/hive/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 (100%) rename {src => shark/src}/test/hive/golden/alter3-4-399fc26b344c98ababa104522601c0cc (100%) rename {src => shark/src}/test/hive/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 (100%) rename {src => shark/src}/test/hive/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 (100%) rename {src => shark/src}/test/hive/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 (100%) rename {src => shark/src}/test/hive/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 (100%) rename {src => shark/src}/test/hive/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 (100%) rename {src => shark/src}/test/hive/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 (100%) rename {src => shark/src}/test/hive/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 (100%) rename {src => shark/src}/test/hive/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 (100%) rename {src => shark/src}/test/hive/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d (100%) rename {src => shark/src}/test/hive/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 (100%) rename {src => shark/src}/test/hive/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f (100%) rename {src => shark/src}/test/hive/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 (100%) rename {src => shark/src}/test/hive/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 (100%) rename {src => shark/src}/test/hive/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 (100%) rename {src => shark/src}/test/hive/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d (100%) rename {src => shark/src}/test/hive/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 (100%) rename {src => shark/src}/test/hive/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 (100%) rename {src => shark/src}/test/hive/golden/alter4-5-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter4-6-acd58e84952d310aeddf78579c36286 (100%) rename {src => shark/src}/test/hive/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 (100%) rename {src => shark/src}/test/hive/golden/alter4-8-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 (100%) rename {src => shark/src}/test/hive/golden/alter5-0-953553e14d835682fa47338dcfffe227 (100%) rename {src => shark/src}/test/hive/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b (100%) rename {src => shark/src}/test/hive/golden/alter5-10-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 (100%) rename {src => shark/src}/test/hive/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 (100%) rename {src => shark/src}/test/hive/golden/alter5-13-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 (100%) rename {src => shark/src}/test/hive/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b (100%) rename {src => shark/src}/test/hive/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d (100%) rename {src => shark/src}/test/hive/golden/alter5-17-9176dc5fb5206209fa907a289db1263d (100%) rename {src => shark/src}/test/hive/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf (100%) rename {src => shark/src}/test/hive/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 (100%) rename {src => shark/src}/test/hive/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d (100%) rename {src => shark/src}/test/hive/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 (100%) rename {src => shark/src}/test/hive/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf (100%) rename {src => shark/src}/test/hive/golden/alter5-3-2fc59e32c07186869811705c89aafadc (100%) rename {src => shark/src}/test/hive/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf (100%) rename {src => shark/src}/test/hive/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 (100%) rename {src => shark/src}/test/hive/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 (100%) rename {src => shark/src}/test/hive/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf (100%) rename {src => shark/src}/test/hive/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 (100%) rename {src => shark/src}/test/hive/golden/alter5-9-1e085f9741197e659413828c42386733 (100%) rename {src => shark/src}/test/hive/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 (100%) rename {src => shark/src}/test/hive/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e (100%) rename {src => shark/src}/test/hive/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 (100%) rename {src => shark/src}/test/hive/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb (100%) rename {src => shark/src}/test/hive/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 (100%) rename {src => shark/src}/test/hive/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 (100%) rename {src => shark/src}/test/hive/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc (100%) rename {src => shark/src}/test/hive/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed (100%) rename {src => shark/src}/test/hive/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d (100%) rename {src => shark/src}/test/hive/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 (100%) rename {src => shark/src}/test/hive/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 (100%) rename {src => shark/src}/test/hive/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 (100%) rename {src => shark/src}/test/hive/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc (100%) rename {src => shark/src}/test/hive/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb (100%) rename {src => shark/src}/test/hive/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c (100%) rename {src => shark/src}/test/hive/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 (100%) rename {src => shark/src}/test/hive/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c (100%) rename {src => shark/src}/test/hive/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c (100%) rename {src => shark/src}/test/hive/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 (100%) rename {src => shark/src}/test/hive/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad (100%) rename {src => shark/src}/test/hive/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b (100%) rename {src => shark/src}/test/hive/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 (100%) rename {src => shark/src}/test/hive/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b (100%) rename {src => shark/src}/test/hive/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 (100%) rename {src => shark/src}/test/hive/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b (100%) rename {src => shark/src}/test/hive/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 (100%) rename {src => shark/src}/test/hive/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d (100%) rename {src => shark/src}/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb (100%) rename {src => shark/src}/test/hive/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a (100%) rename {src => shark/src}/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 (100%) rename {src => shark/src}/test/hive/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d (100%) rename {src => shark/src}/test/hive/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab (100%) rename {src => shark/src}/test/hive/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b (100%) rename {src => shark/src}/test/hive/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 (100%) rename {src => shark/src}/test/hive/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 (100%) rename {src => shark/src}/test/hive/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb (100%) rename {src => shark/src}/test/hive/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 (100%) rename {src => shark/src}/test/hive/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb (100%) rename {src => shark/src}/test/hive/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 (100%) rename {src => shark/src}/test/hive/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 (100%) rename {src => shark/src}/test/hive/golden/archive-16-892147913578bcf60620b7dd73893dd0 (100%) rename {src => shark/src}/test/hive/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb (100%) rename {src => shark/src}/test/hive/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a (100%) rename {src => shark/src}/test/hive/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f (100%) rename {src => shark/src}/test/hive/golden/archive-2-713efc113418b01f76ffd589840193c8 (100%) rename {src => shark/src}/test/hive/golden/archive-20-530277b0fee8b05c37b26846bceef827 (100%) rename {src => shark/src}/test/hive/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 (100%) rename {src => shark/src}/test/hive/golden/archive-22-530277b0fee8b05c37b26846bceef827 (100%) rename {src => shark/src}/test/hive/golden/archive-23-892147913578bcf60620b7dd73893dd0 (100%) rename {src => shark/src}/test/hive/golden/archive-24-530277b0fee8b05c37b26846bceef827 (100%) rename {src => shark/src}/test/hive/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f (100%) rename {src => shark/src}/test/hive/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a (100%) rename {src => shark/src}/test/hive/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 (100%) rename {src => shark/src}/test/hive/golden/archive-28-188eb7912265ed8dffa5200517bbe526 (100%) rename {src => shark/src}/test/hive/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 (100%) rename {src => shark/src}/test/hive/golden/archive-3-27ad2962fed131f51ba802596ba37278 (100%) rename {src => shark/src}/test/hive/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a (100%) rename {src => shark/src}/test/hive/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 (100%) rename {src => shark/src}/test/hive/golden/archive-32-27ad2962fed131f51ba802596ba37278 (100%) rename {src => shark/src}/test/hive/golden/archive-4-3e95421993ab28d18245ec2340f580a3 (100%) rename {src => shark/src}/test/hive/golden/archive-5-c0c18ac884677231a41eea8d980d0451 (100%) rename {src => shark/src}/test/hive/golden/archive-6-528ab9750a558af7f1a43b3108e793dd (100%) rename {src => shark/src}/test/hive/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca (100%) rename {src => shark/src}/test/hive/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c (100%) rename {src => shark/src}/test/hive/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c (100%) rename {src => shark/src}/test/hive/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf (100%) rename {src => shark/src}/test/hive/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 (100%) rename {src => shark/src}/test/hive/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 (100%) rename {src => shark/src}/test/hive/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e (100%) rename {src => shark/src}/test/hive/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 (100%) rename {src => shark/src}/test/hive/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb (100%) rename {src => shark/src}/test/hive/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 (100%) rename {src => shark/src}/test/hive/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 (100%) rename {src => shark/src}/test/hive/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d (100%) rename {src => shark/src}/test/hive/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb (100%) rename {src => shark/src}/test/hive/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 (100%) rename {src => shark/src}/test/hive/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 (100%) rename {src => shark/src}/test/hive/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 (100%) rename {src => shark/src}/test/hive/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 (100%) rename {src => shark/src}/test/hive/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a (100%) rename {src => shark/src}/test/hive/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca (100%) rename {src => shark/src}/test/hive/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c (100%) rename {src => shark/src}/test/hive/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf (100%) rename {src => shark/src}/test/hive/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba (100%) rename {src => shark/src}/test/hive/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 (100%) rename {src => shark/src}/test/hive/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c (100%) rename {src => shark/src}/test/hive/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a (100%) rename {src => shark/src}/test/hive/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e (100%) rename {src => shark/src}/test/hive/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 (100%) rename {src => shark/src}/test/hive/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 (100%) rename {src => shark/src}/test/hive/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e (100%) rename {src => shark/src}/test/hive/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 (100%) rename {src => shark/src}/test/hive/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 (100%) rename {src => shark/src}/test/hive/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 (100%) rename {src => shark/src}/test/hive/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 (100%) rename {src => shark/src}/test/hive/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 (100%) rename {src => shark/src}/test/hive/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 (100%) rename {src => shark/src}/test/hive/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 (100%) rename {src => shark/src}/test/hive/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 (100%) rename {src => shark/src}/test/hive/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 (100%) rename {src => shark/src}/test/hive/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e (100%) rename {src => shark/src}/test/hive/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 (100%) rename {src => shark/src}/test/hive/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 (100%) rename {src => shark/src}/test/hive/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 (100%) rename {src => shark/src}/test/hive/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 (100%) rename {src => shark/src}/test/hive/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 (100%) rename {src => shark/src}/test/hive/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 (100%) rename {src => shark/src}/test/hive/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f (100%) rename {src => shark/src}/test/hive/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 (100%) rename {src => shark/src}/test/hive/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 (100%) rename {src => shark/src}/test/hive/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 (100%) rename {src => shark/src}/test/hive/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 (100%) rename {src => shark/src}/test/hive/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 (100%) rename {src => shark/src}/test/hive/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 (100%) rename {src => shark/src}/test/hive/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf (100%) rename {src => shark/src}/test/hive/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 (100%) rename {src => shark/src}/test/hive/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 (100%) rename {src => shark/src}/test/hive/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 (100%) rename {src => shark/src}/test/hive/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 (100%) rename {src => shark/src}/test/hive/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c (100%) rename {src => shark/src}/test/hive/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b (100%) rename {src => shark/src}/test/hive/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f (100%) rename {src => shark/src}/test/hive/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 (100%) rename {src => shark/src}/test/hive/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 (100%) rename {src => shark/src}/test/hive/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f (100%) rename {src => shark/src}/test/hive/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca (100%) rename {src => shark/src}/test/hive/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 (100%) rename {src => shark/src}/test/hive/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a (100%) rename {src => shark/src}/test/hive/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 (100%) rename {src => shark/src}/test/hive/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd (100%) rename {src => shark/src}/test/hive/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 (100%) rename {src => shark/src}/test/hive/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee (100%) rename {src => shark/src}/test/hive/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 (100%) rename {src => shark/src}/test/hive/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 (100%) rename {src => shark/src}/test/hive/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f (100%) rename {src => shark/src}/test/hive/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 (100%) rename {src => shark/src}/test/hive/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb (100%) rename {src => shark/src}/test/hive/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join23-1-b31437533a2b890788938455cb32f679 (100%) rename {src => shark/src}/test/hive/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 (100%) rename {src => shark/src}/test/hive/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b (100%) rename {src => shark/src}/test/hive/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d (100%) rename {src => shark/src}/test/hive/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 (100%) rename {src => shark/src}/test/hive/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 (100%) rename {src => shark/src}/test/hive/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 (100%) rename {src => shark/src}/test/hive/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e (100%) rename {src => shark/src}/test/hive/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 (100%) rename {src => shark/src}/test/hive/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e (100%) rename {src => shark/src}/test/hive/golden/auto_join25-2-8180638a57b64557e02815c863031755 (100%) rename {src => shark/src}/test/hive/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e (100%) rename {src => shark/src}/test/hive/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 (100%) rename {src => shark/src}/test/hive/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 (100%) rename {src => shark/src}/test/hive/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f (100%) rename {src => shark/src}/test/hive/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca (100%) rename {src => shark/src}/test/hive/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a (100%) rename {src => shark/src}/test/hive/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 (100%) rename {src => shark/src}/test/hive/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 (100%) rename {src => shark/src}/test/hive/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 (100%) rename {src => shark/src}/test/hive/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 (100%) rename {src => shark/src}/test/hive/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 (100%) rename {src => shark/src}/test/hive/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e (100%) rename {src => shark/src}/test/hive/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c (100%) rename {src => shark/src}/test/hive/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 (100%) rename {src => shark/src}/test/hive/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 (100%) rename {src => shark/src}/test/hive/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 (100%) rename {src => shark/src}/test/hive/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e (100%) rename {src => shark/src}/test/hive/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d (100%) rename {src => shark/src}/test/hive/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 (100%) rename {src => shark/src}/test/hive/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f (100%) rename {src => shark/src}/test/hive/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a (100%) rename {src => shark/src}/test/hive/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d (100%) rename {src => shark/src}/test/hive/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba (100%) rename {src => shark/src}/test/hive/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e (100%) rename {src => shark/src}/test/hive/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 (100%) rename {src => shark/src}/test/hive/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 (100%) rename {src => shark/src}/test/hive/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 (100%) rename {src => shark/src}/test/hive/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e (100%) rename {src => shark/src}/test/hive/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f (100%) rename {src => shark/src}/test/hive/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f (100%) rename {src => shark/src}/test/hive/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d (100%) rename {src => shark/src}/test/hive/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b (100%) rename {src => shark/src}/test/hive/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c (100%) rename {src => shark/src}/test/hive/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b (100%) rename {src => shark/src}/test/hive/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a (100%) rename {src => shark/src}/test/hive/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc (100%) rename {src => shark/src}/test/hive/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 (100%) rename {src => shark/src}/test/hive/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd (100%) rename {src => shark/src}/test/hive/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf (100%) rename {src => shark/src}/test/hive/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed (100%) rename {src => shark/src}/test/hive/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef (100%) rename {src => shark/src}/test/hive/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee (100%) rename {src => shark/src}/test/hive/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a (100%) rename {src => shark/src}/test/hive/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd (100%) rename {src => shark/src}/test/hive/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a (100%) rename {src => shark/src}/test/hive/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 (100%) rename {src => shark/src}/test/hive/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 (100%) rename {src => shark/src}/test/hive/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd (100%) rename {src => shark/src}/test/hive/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 (100%) rename {src => shark/src}/test/hive/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 (100%) rename {src => shark/src}/test/hive/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 (100%) rename {src => shark/src}/test/hive/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd (100%) rename {src => shark/src}/test/hive/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 (100%) rename {src => shark/src}/test/hive/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 (100%) rename {src => shark/src}/test/hive/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 (100%) rename {src => shark/src}/test/hive/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 (100%) rename {src => shark/src}/test/hive/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e (100%) rename {src => shark/src}/test/hive/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 (100%) rename {src => shark/src}/test/hive/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 (100%) rename {src => shark/src}/test/hive/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd (100%) rename {src => shark/src}/test/hive/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab (100%) rename {src => shark/src}/test/hive/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca (100%) rename {src => shark/src}/test/hive/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 (100%) rename {src => shark/src}/test/hive/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 (100%) rename {src => shark/src}/test/hive/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 (100%) rename {src => shark/src}/test/hive/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b (100%) rename {src => shark/src}/test/hive/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 (100%) rename {src => shark/src}/test/hive/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed (100%) rename {src => shark/src}/test/hive/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 (100%) rename {src => shark/src}/test/hive/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 (100%) rename {src => shark/src}/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 (100%) rename {src => shark/src}/test/hive/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 (100%) rename {src => shark/src}/test/hive/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a (100%) rename {src => shark/src}/test/hive/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 (100%) rename {src => shark/src}/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 (100%) rename {src => shark/src}/test/hive/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 (100%) rename {src => shark/src}/test/hive/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d (100%) rename {src => shark/src}/test/hive/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 (100%) rename {src => shark/src}/test/hive/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b (100%) rename {src => shark/src}/test/hive/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 (100%) rename {src => shark/src}/test/hive/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 (100%) rename {src => shark/src}/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 (100%) rename {src => shark/src}/test/hive/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 (100%) rename {src => shark/src}/test/hive/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 (100%) rename {src => shark/src}/test/hive/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef (100%) rename {src => shark/src}/test/hive/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 (100%) rename {src => shark/src}/test/hive/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 (100%) rename {src => shark/src}/test/hive/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 (100%) rename {src => shark/src}/test/hive/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 (100%) rename {src => shark/src}/test/hive/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e (100%) rename {src => shark/src}/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 (100%) rename {src => shark/src}/test/hive/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 (100%) rename {src => shark/src}/test/hive/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 (100%) rename {src => shark/src}/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 (100%) rename {src => shark/src}/test/hive/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 (100%) rename {src => shark/src}/test/hive/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 (100%) rename {src => shark/src}/test/hive/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 (100%) rename {src => shark/src}/test/hive/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 (100%) rename {src => shark/src}/test/hive/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 (100%) rename {src => shark/src}/test/hive/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 (100%) rename {src => shark/src}/test/hive/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c (100%) rename {src => shark/src}/test/hive/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a (100%) rename {src => shark/src}/test/hive/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 (100%) rename {src => shark/src}/test/hive/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 (100%) rename {src => shark/src}/test/hive/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 (100%) rename {src => shark/src}/test/hive/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 (100%) rename {src => shark/src}/test/hive/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e (100%) rename {src => shark/src}/test/hive/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 (100%) rename {src => shark/src}/test/hive/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 (100%) rename {src => shark/src}/test/hive/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 (100%) rename {src => shark/src}/test/hive/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 (100%) rename {src => shark/src}/test/hive/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 (100%) rename {src => shark/src}/test/hive/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 (100%) rename {src => shark/src}/test/hive/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 (100%) rename {src => shark/src}/test/hive/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 (100%) rename {src => shark/src}/test/hive/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 (100%) rename {src => shark/src}/test/hive/golden/cluster-20-294891c1d956245540a80aa800ba393d (100%) rename {src => shark/src}/test/hive/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f (100%) rename {src => shark/src}/test/hive/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 (100%) rename {src => shark/src}/test/hive/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e (100%) rename {src => shark/src}/test/hive/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae (100%) rename {src => shark/src}/test/hive/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a (100%) rename {src => shark/src}/test/hive/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f (100%) rename {src => shark/src}/test/hive/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc (100%) rename {src => shark/src}/test/hive/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 (100%) rename {src => shark/src}/test/hive/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 (100%) rename {src => shark/src}/test/hive/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 (100%) rename {src => shark/src}/test/hive/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d (100%) rename {src => shark/src}/test/hive/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 (100%) rename {src => shark/src}/test/hive/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d (100%) rename {src => shark/src}/test/hive/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c (100%) rename {src => shark/src}/test/hive/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 (100%) rename {src => shark/src}/test/hive/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f (100%) rename {src => shark/src}/test/hive/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c (100%) rename {src => shark/src}/test/hive/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 (100%) rename {src => shark/src}/test/hive/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 (100%) rename {src => shark/src}/test/hive/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd (100%) rename {src => shark/src}/test/hive/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 (100%) rename {src => shark/src}/test/hive/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 (100%) rename {src => shark/src}/test/hive/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 (100%) rename {src => shark/src}/test/hive/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 (100%) rename {src => shark/src}/test/hive/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce (100%) rename {src => shark/src}/test/hive/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe (100%) rename {src => shark/src}/test/hive/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 (100%) rename {src => shark/src}/test/hive/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c (100%) rename {src => shark/src}/test/hive/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 (100%) rename {src => shark/src}/test/hive/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 (100%) rename {src => shark/src}/test/hive/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 (100%) rename {src => shark/src}/test/hive/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c (100%) rename {src => shark/src}/test/hive/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 (100%) rename {src => shark/src}/test/hive/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d (100%) rename {src => shark/src}/test/hive/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f (100%) rename {src => shark/src}/test/hive/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 (100%) rename {src => shark/src}/test/hive/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf (100%) rename {src => shark/src}/test/hive/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 (100%) rename {src => shark/src}/test/hive/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 (100%) rename {src => shark/src}/test/hive/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea (100%) rename {src => shark/src}/test/hive/golden/combine1-6-1d1f97cce07323812de3027920b04b75 (100%) rename {src => shark/src}/test/hive/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 (100%) rename {src => shark/src}/test/hive/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c (100%) rename {src => shark/src}/test/hive/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 (100%) rename {src => shark/src}/test/hive/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 (100%) rename {src => shark/src}/test/hive/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a (100%) rename {src => shark/src}/test/hive/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 (100%) rename {src => shark/src}/test/hive/golden/combine2-12-cd15ffd140539cf86090814729ec4748 (100%) rename {src => shark/src}/test/hive/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 (100%) rename {src => shark/src}/test/hive/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c (100%) rename {src => shark/src}/test/hive/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 (100%) rename {src => shark/src}/test/hive/golden/combine2-16-557997716a68312e8cae75428e3ce31 (100%) rename {src => shark/src}/test/hive/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 (100%) rename {src => shark/src}/test/hive/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 (100%) rename {src => shark/src}/test/hive/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf (100%) rename {src => shark/src}/test/hive/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 (100%) rename {src => shark/src}/test/hive/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 (100%) rename {src => shark/src}/test/hive/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea (100%) rename {src => shark/src}/test/hive/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/combine2-7-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 (100%) rename {src => shark/src}/test/hive/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 (100%) rename {src => shark/src}/test/hive/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf (100%) rename {src => shark/src}/test/hive/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c (100%) rename {src => shark/src}/test/hive/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea (100%) rename {src => shark/src}/test/hive/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 (100%) rename {src => shark/src}/test/hive/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af (100%) rename {src => shark/src}/test/hive/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 (100%) rename {src => shark/src}/test/hive/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 (100%) rename {src => shark/src}/test/hive/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 (100%) rename {src => shark/src}/test/hive/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b (100%) rename {src => shark/src}/test/hive/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc (100%) rename {src => shark/src}/test/hive/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 (100%) rename {src => shark/src}/test/hive/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a (100%) rename {src => shark/src}/test/hive/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b (100%) rename {src => shark/src}/test/hive/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f (100%) rename {src => shark/src}/test/hive/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf (100%) rename {src => shark/src}/test/hive/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd (100%) rename {src => shark/src}/test/hive/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 (100%) rename {src => shark/src}/test/hive/golden/combine3-22-11025483569617a9f014b5defd71e933 (100%) rename {src => shark/src}/test/hive/golden/combine3-23-4725c48df09565618cbffd05953a5f62 (100%) rename {src => shark/src}/test/hive/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 (100%) rename {src => shark/src}/test/hive/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 (100%) rename {src => shark/src}/test/hive/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea (100%) rename {src => shark/src}/test/hive/golden/combine3-6-4725c48df09565618cbffd05953a5f62 (100%) rename {src => shark/src}/test/hive/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 (100%) rename {src => shark/src}/test/hive/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b (100%) rename {src => shark/src}/test/hive/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c (100%) rename {src => shark/src}/test/hive/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e (100%) rename {src => shark/src}/test/hive/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d (100%) rename {src => shark/src}/test/hive/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce (100%) rename {src => shark/src}/test/hive/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d (100%) rename {src => shark/src}/test/hive/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f (100%) rename {src => shark/src}/test/hive/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b (100%) rename {src => shark/src}/test/hive/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf (100%) rename {src => shark/src}/test/hive/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 (100%) rename {src => shark/src}/test/hive/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d (100%) rename {src => shark/src}/test/hive/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b (100%) rename {src => shark/src}/test/hive/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f (100%) rename {src => shark/src}/test/hive/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a (100%) rename {src => shark/src}/test/hive/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc (100%) rename {src => shark/src}/test/hive/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 (100%) rename {src => shark/src}/test/hive/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 (100%) rename {src => shark/src}/test/hive/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f (100%) rename {src => shark/src}/test/hive/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b (100%) rename {src => shark/src}/test/hive/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 (100%) rename {src => shark/src}/test/hive/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 (100%) rename {src => shark/src}/test/hive/golden/count-11-29aa086fe75b55482a91316c38498565 (100%) rename {src => shark/src}/test/hive/golden/count-12-944f53db544c07a7b38a0544a21d8e13 (100%) rename {src => shark/src}/test/hive/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 (100%) rename {src => shark/src}/test/hive/golden/count-3-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/count-4-590bf60b8d4dfa135f73dbb52180136f (100%) rename {src => shark/src}/test/hive/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 (100%) rename {src => shark/src}/test/hive/golden/count-6-29aa086fe75b55482a91316c38498565 (100%) rename {src => shark/src}/test/hive/golden/count-7-944f53db544c07a7b38a0544a21d8e13 (100%) rename {src => shark/src}/test/hive/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/count-9-590bf60b8d4dfa135f73dbb52180136f (100%) rename {src => shark/src}/test/hive/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 (100%) rename {src => shark/src}/test/hive/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 (100%) rename {src => shark/src}/test/hive/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e (100%) rename {src => shark/src}/test/hive/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 (100%) rename {src => shark/src}/test/hive/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 (100%) rename {src => shark/src}/test/hive/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 (100%) rename {src => shark/src}/test/hive/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac (100%) rename {src => shark/src}/test/hive/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 (100%) rename {src => shark/src}/test/hive/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f (100%) rename {src => shark/src}/test/hive/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 (100%) rename {src => shark/src}/test/hive/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 (100%) rename {src => shark/src}/test/hive/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c (100%) rename {src => shark/src}/test/hive/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd (100%) rename {src => shark/src}/test/hive/golden/create table as-1-b9002c1d71895be765575b62656d1928 (100%) rename {src => shark/src}/test/hive/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d (100%) rename {src => shark/src}/test/hive/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 (100%) rename {src => shark/src}/test/hive/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba (100%) rename {src => shark/src}/test/hive/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 (100%) rename {src => shark/src}/test/hive/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 (100%) rename {src => shark/src}/test/hive/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a (100%) rename {src => shark/src}/test/hive/golden/create_1-14-437bd1acbae61f48e851addb769d3379 (100%) rename {src => shark/src}/test/hive/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 (100%) rename {src => shark/src}/test/hive/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 (100%) rename {src => shark/src}/test/hive/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 (100%) rename {src => shark/src}/test/hive/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 (100%) rename {src => shark/src}/test/hive/golden/create_1-4-610b82bf7b0080d293977927e5ef780c (100%) rename {src => shark/src}/test/hive/golden/create_1-5-c77b018276b1558c1d9462e0625e152e (100%) rename {src => shark/src}/test/hive/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a (100%) rename {src => shark/src}/test/hive/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 (100%) rename {src => shark/src}/test/hive/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 (100%) rename {src => shark/src}/test/hive/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 (100%) rename {src => shark/src}/test/hive/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d (100%) rename {src => shark/src}/test/hive/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee (100%) rename {src => shark/src}/test/hive/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e (100%) rename {src => shark/src}/test/hive/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 (100%) rename {src => shark/src}/test/hive/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 (100%) rename {src => shark/src}/test/hive/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 (100%) rename {src => shark/src}/test/hive/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 (100%) rename {src => shark/src}/test/hive/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 (100%) rename {src => shark/src}/test/hive/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b (100%) rename {src => shark/src}/test/hive/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 (100%) rename {src => shark/src}/test/hive/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 (100%) rename {src => shark/src}/test/hive/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 (100%) rename {src => shark/src}/test/hive/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 (100%) rename {src => shark/src}/test/hive/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a (100%) rename {src => shark/src}/test/hive/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 (100%) rename {src => shark/src}/test/hive/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc (100%) rename {src => shark/src}/test/hive/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 (100%) rename {src => shark/src}/test/hive/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 (100%) rename {src => shark/src}/test/hive/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c (100%) rename {src => shark/src}/test/hive/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 (100%) rename {src => shark/src}/test/hive/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e (100%) rename {src => shark/src}/test/hive/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 (100%) rename {src => shark/src}/test/hive/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 (100%) rename {src => shark/src}/test/hive/golden/create_like-1-30485a2507b60b96cad3d293527e6af (100%) rename {src => shark/src}/test/hive/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 (100%) rename {src => shark/src}/test/hive/golden/create_like-11-ba64f0122b21f605d8b2928753784130 (100%) rename {src => shark/src}/test/hive/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a (100%) rename {src => shark/src}/test/hive/golden/create_like-3-eea111a209cf8895f31f64699669c705 (100%) rename {src => shark/src}/test/hive/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 (100%) rename {src => shark/src}/test/hive/golden/create_like-5-dc9de26002604e9e436135bd4b40636d (100%) rename {src => shark/src}/test/hive/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 (100%) rename {src => shark/src}/test/hive/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 (100%) rename {src => shark/src}/test/hive/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb (100%) rename {src => shark/src}/test/hive/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb (100%) rename {src => shark/src}/test/hive/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b (100%) rename {src => shark/src}/test/hive/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a (100%) rename {src => shark/src}/test/hive/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb (100%) rename {src => shark/src}/test/hive/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb (100%) rename {src => shark/src}/test/hive/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c (100%) rename {src => shark/src}/test/hive/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f (100%) rename {src => shark/src}/test/hive/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b (100%) rename {src => shark/src}/test/hive/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b (100%) rename {src => shark/src}/test/hive/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d (100%) rename {src => shark/src}/test/hive/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f (100%) rename {src => shark/src}/test/hive/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af (100%) rename {src => shark/src}/test/hive/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 (100%) rename {src => shark/src}/test/hive/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 (100%) rename {src => shark/src}/test/hive/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 (100%) rename {src => shark/src}/test/hive/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 (100%) rename {src => shark/src}/test/hive/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 (100%) rename {src => shark/src}/test/hive/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 (100%) rename {src => shark/src}/test/hive/golden/create_nested_type-3-ac452c9279877935983c37113898e53c (100%) rename {src => shark/src}/test/hive/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff (100%) rename {src => shark/src}/test/hive/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d (100%) rename {src => shark/src}/test/hive/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea (100%) rename {src => shark/src}/test/hive/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 (100%) rename {src => shark/src}/test/hive/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d (100%) rename {src => shark/src}/test/hive/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de (100%) rename {src => shark/src}/test/hive/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb (100%) rename {src => shark/src}/test/hive/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 (100%) rename {src => shark/src}/test/hive/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e (100%) rename {src => shark/src}/test/hive/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 (100%) rename {src => shark/src}/test/hive/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 (100%) rename {src => shark/src}/test/hive/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e (100%) rename {src => shark/src}/test/hive/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef (100%) rename {src => shark/src}/test/hive/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 (100%) rename {src => shark/src}/test/hive/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd (100%) rename {src => shark/src}/test/hive/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e (100%) rename {src => shark/src}/test/hive/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 (100%) rename {src => shark/src}/test/hive/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 (100%) rename {src => shark/src}/test/hive/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd (100%) rename {src => shark/src}/test/hive/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e (100%) rename {src => shark/src}/test/hive/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 (100%) rename {src => shark/src}/test/hive/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 (100%) rename {src => shark/src}/test/hive/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd (100%) rename {src => shark/src}/test/hive/golden/create_view-11-2021c047203276dd2db51a56e672fcea (100%) rename {src => shark/src}/test/hive/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc (100%) rename {src => shark/src}/test/hive/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a (100%) rename {src => shark/src}/test/hive/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 (100%) rename {src => shark/src}/test/hive/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 (100%) rename {src => shark/src}/test/hive/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 (100%) rename {src => shark/src}/test/hive/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a (100%) rename {src => shark/src}/test/hive/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 (100%) rename {src => shark/src}/test/hive/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b (100%) rename {src => shark/src}/test/hive/golden/create_view-2-d80dcd1271ab264292e9938f3162427c (100%) rename {src => shark/src}/test/hive/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 (100%) rename {src => shark/src}/test/hive/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 (100%) rename {src => shark/src}/test/hive/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 (100%) rename {src => shark/src}/test/hive/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 (100%) rename {src => shark/src}/test/hive/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 (100%) rename {src => shark/src}/test/hive/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 (100%) rename {src => shark/src}/test/hive/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 (100%) rename {src => shark/src}/test/hive/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 (100%) rename {src => shark/src}/test/hive/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 (100%) rename {src => shark/src}/test/hive/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 (100%) rename {src => shark/src}/test/hive/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc (100%) rename {src => shark/src}/test/hive/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 (100%) rename {src => shark/src}/test/hive/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 (100%) rename {src => shark/src}/test/hive/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 (100%) rename {src => shark/src}/test/hive/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 (100%) rename {src => shark/src}/test/hive/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc (100%) rename {src => shark/src}/test/hive/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 (100%) rename {src => shark/src}/test/hive/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 (100%) rename {src => shark/src}/test/hive/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 (100%) rename {src => shark/src}/test/hive/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e (100%) rename {src => shark/src}/test/hive/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 (100%) rename {src => shark/src}/test/hive/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 (100%) rename {src => shark/src}/test/hive/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b (100%) rename {src => shark/src}/test/hive/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 (100%) rename {src => shark/src}/test/hive/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f (100%) rename {src => shark/src}/test/hive/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 (100%) rename {src => shark/src}/test/hive/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec (100%) rename {src => shark/src}/test/hive/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 (100%) rename {src => shark/src}/test/hive/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 (100%) rename {src => shark/src}/test/hive/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd (100%) rename {src => shark/src}/test/hive/golden/database.table-0-c657beb729b6a7882309a203fc6f298e (100%) rename {src => shark/src}/test/hive/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f (100%) rename {src => shark/src}/test/hive/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 (100%) rename {src => shark/src}/test/hive/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 (100%) rename {src => shark/src}/test/hive/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 (100%) rename {src => shark/src}/test/hive/golden/database_drop-12-e02a53f7e798d2741152526516f14941 (100%) rename {src => shark/src}/test/hive/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 (100%) rename {src => shark/src}/test/hive/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 (100%) rename {src => shark/src}/test/hive/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 (100%) rename {src => shark/src}/test/hive/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 (100%) rename {src => shark/src}/test/hive/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 (100%) rename {src => shark/src}/test/hive/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba (100%) rename {src => shark/src}/test/hive/golden/database_drop-8-97101266791d2b2c662bcde549422318 (100%) rename {src => shark/src}/test/hive/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 (100%) rename {src => shark/src}/test/hive/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a (100%) rename {src => shark/src}/test/hive/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 (100%) rename {src => shark/src}/test/hive/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 (100%) rename {src => shark/src}/test/hive/golden/database_location-11-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 (100%) rename {src => shark/src}/test/hive/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 (100%) rename {src => shark/src}/test/hive/golden/database_location-4-be65cf649363681d54e593c42a5ecffb (100%) rename {src => shark/src}/test/hive/golden/database_location-5-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a (100%) rename {src => shark/src}/test/hive/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 (100%) rename {src => shark/src}/test/hive/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b (100%) rename {src => shark/src}/test/hive/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 (100%) rename {src => shark/src}/test/hive/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 (100%) rename {src => shark/src}/test/hive/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 (100%) rename {src => shark/src}/test/hive/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 (100%) rename {src => shark/src}/test/hive/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 (100%) rename {src => shark/src}/test/hive/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 (100%) rename {src => shark/src}/test/hive/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a (100%) rename {src => shark/src}/test/hive/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 (100%) rename {src => shark/src}/test/hive/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 (100%) rename {src => shark/src}/test/hive/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a (100%) rename {src => shark/src}/test/hive/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd (100%) rename {src => shark/src}/test/hive/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 (100%) rename {src => shark/src}/test/hive/golden/date_2-2-cab14d992c53c106ab257fae52001e04 (100%) rename {src => shark/src}/test/hive/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 (100%) rename {src => shark/src}/test/hive/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 (100%) rename {src => shark/src}/test/hive/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 (100%) rename {src => shark/src}/test/hive/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c (100%) rename {src => shark/src}/test/hive/golden/date_join1-1-3a68de2112a212a07a3068916c608fb (100%) rename {src => shark/src}/test/hive/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 (100%) rename {src => shark/src}/test/hive/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 (100%) rename {src => shark/src}/test/hive/golden/date_serde-1-36e6041f53433482631018410bb62a99 (100%) rename {src => shark/src}/test/hive/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 (100%) rename {src => shark/src}/test/hive/golden/date_serde-3-e6167e27465514356c557a77d956ea46 (100%) rename {src => shark/src}/test/hive/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 (100%) rename {src => shark/src}/test/hive/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c (100%) rename {src => shark/src}/test/hive/golden/date_serde-6-e00facec2986bc55074868eff87ba22a (100%) rename {src => shark/src}/test/hive/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d (100%) rename {src => shark/src}/test/hive/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 (100%) rename {src => shark/src}/test/hive/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 (100%) rename {src => shark/src}/test/hive/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 (100%) rename {src => shark/src}/test/hive/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd (100%) rename {src => shark/src}/test/hive/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 (100%) rename {src => shark/src}/test/hive/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 (100%) rename {src => shark/src}/test/hive/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 (100%) rename {src => shark/src}/test/hive/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 (100%) rename {src => shark/src}/test/hive/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 (100%) rename {src => shark/src}/test/hive/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 (100%) rename {src => shark/src}/test/hive/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 (100%) rename {src => shark/src}/test/hive/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 (100%) rename {src => shark/src}/test/hive/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b (100%) rename {src => shark/src}/test/hive/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a (100%) rename {src => shark/src}/test/hive/golden/decimal_1-8-cdd0932288d3cc43636334439805769d (100%) rename {src => shark/src}/test/hive/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c (100%) rename {src => shark/src}/test/hive/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee (100%) rename {src => shark/src}/test/hive/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b (100%) rename {src => shark/src}/test/hive/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d (100%) rename {src => shark/src}/test/hive/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e (100%) rename {src => shark/src}/test/hive/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff (100%) rename {src => shark/src}/test/hive/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e (100%) rename {src => shark/src}/test/hive/golden/decimal_2-25-14face5c7104382196e65741a199c36 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e (100%) rename {src => shark/src}/test/hive/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a (100%) rename {src => shark/src}/test/hive/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee (100%) rename {src => shark/src}/test/hive/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 (100%) rename {src => shark/src}/test/hive/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d (100%) rename {src => shark/src}/test/hive/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e (100%) rename {src => shark/src}/test/hive/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef (100%) rename {src => shark/src}/test/hive/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 (100%) rename {src => shark/src}/test/hive/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 (100%) rename {src => shark/src}/test/hive/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 (100%) rename {src => shark/src}/test/hive/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 (100%) rename {src => shark/src}/test/hive/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 (100%) rename {src => shark/src}/test/hive/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b (100%) rename {src => shark/src}/test/hive/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 (100%) rename {src => shark/src}/test/hive/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b (100%) rename {src => shark/src}/test/hive/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 (100%) rename {src => shark/src}/test/hive/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 (100%) rename {src => shark/src}/test/hive/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 (100%) rename {src => shark/src}/test/hive/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 (100%) rename {src => shark/src}/test/hive/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f (100%) rename {src => shark/src}/test/hive/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 (100%) rename {src => shark/src}/test/hive/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d (100%) rename {src => shark/src}/test/hive/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e (100%) rename {src => shark/src}/test/hive/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c (100%) rename {src => shark/src}/test/hive/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c (100%) rename {src => shark/src}/test/hive/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e (100%) rename {src => shark/src}/test/hive/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 (100%) rename {src => shark/src}/test/hive/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 (100%) rename {src => shark/src}/test/hive/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 (100%) rename {src => shark/src}/test/hive/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 (100%) rename {src => shark/src}/test/hive/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e (100%) rename {src => shark/src}/test/hive/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e (100%) rename {src => shark/src}/test/hive/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab (100%) rename {src => shark/src}/test/hive/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 (100%) rename {src => shark/src}/test/hive/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c (100%) rename {src => shark/src}/test/hive/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 (100%) rename {src => shark/src}/test/hive/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be (100%) rename {src => shark/src}/test/hive/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba (100%) rename {src => shark/src}/test/hive/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 (100%) rename {src => shark/src}/test/hive/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 (100%) rename {src => shark/src}/test/hive/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 (100%) rename {src => shark/src}/test/hive/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 (100%) rename {src => shark/src}/test/hive/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb (100%) rename {src => shark/src}/test/hive/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 (100%) rename {src => shark/src}/test/hive/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a (100%) rename {src => shark/src}/test/hive/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 (100%) rename {src => shark/src}/test/hive/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 (100%) rename {src => shark/src}/test/hive/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-24-ee226b42db35b219702319858e925468 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 (100%) rename {src => shark/src}/test/hive/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b (100%) rename {src => shark/src}/test/hive/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b (100%) rename {src => shark/src}/test/hive/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 (100%) rename {src => shark/src}/test/hive/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/describe_table_json-3-576670af142149302decb9bf8662e68a (100%) rename {src => shark/src}/test/hive/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 (100%) rename {src => shark/src}/test/hive/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb (100%) rename {src => shark/src}/test/hive/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 (100%) rename {src => shark/src}/test/hive/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 (100%) rename {src => shark/src}/test/hive/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 (100%) rename {src => shark/src}/test/hive/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 (100%) rename {src => shark/src}/test/hive/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad (100%) rename {src => shark/src}/test/hive/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa (100%) rename {src => shark/src}/test/hive/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce (100%) rename {src => shark/src}/test/hive/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 (100%) rename {src => shark/src}/test/hive/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b (100%) rename {src => shark/src}/test/hive/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c (100%) rename {src => shark/src}/test/hive/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e (100%) rename {src => shark/src}/test/hive/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 (100%) rename {src => shark/src}/test/hive/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f (100%) rename {src => shark/src}/test/hive/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb (100%) rename {src => shark/src}/test/hive/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a (100%) rename {src => shark/src}/test/hive/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b (100%) rename {src => shark/src}/test/hive/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 (100%) rename {src => shark/src}/test/hive/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 (100%) rename {src => shark/src}/test/hive/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c (100%) rename {src => shark/src}/test/hive/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 (100%) rename {src => shark/src}/test/hive/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab (100%) rename {src => shark/src}/test/hive/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 (100%) rename {src => shark/src}/test/hive/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 (100%) rename {src => shark/src}/test/hive/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a (100%) rename {src => shark/src}/test/hive/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 (100%) rename {src => shark/src}/test/hive/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 (100%) rename {src => shark/src}/test/hive/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 (100%) rename {src => shark/src}/test/hive/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 (100%) rename {src => shark/src}/test/hive/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 (100%) rename {src => shark/src}/test/hive/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 (100%) rename {src => shark/src}/test/hive/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 (100%) rename {src => shark/src}/test/hive/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa (100%) rename {src => shark/src}/test/hive/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 (100%) rename {src => shark/src}/test/hive/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f (100%) rename {src => shark/src}/test/hive/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 (100%) rename {src => shark/src}/test/hive/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 (100%) rename {src => shark/src}/test/hive/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 (100%) rename {src => shark/src}/test/hive/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa (100%) rename {src => shark/src}/test/hive/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 (100%) rename {src => shark/src}/test/hive/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 (100%) rename {src => shark/src}/test/hive/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 (100%) rename {src => shark/src}/test/hive/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 (100%) rename {src => shark/src}/test/hive/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 (100%) rename {src => shark/src}/test/hive/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 (100%) rename {src => shark/src}/test/hive/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 (100%) rename {src => shark/src}/test/hive/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 (100%) rename {src => shark/src}/test/hive/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 (100%) rename {src => shark/src}/test/hive/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 (100%) rename {src => shark/src}/test/hive/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 (100%) rename {src => shark/src}/test/hive/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 (100%) rename {src => shark/src}/test/hive/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 (100%) rename {src => shark/src}/test/hive/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 (100%) rename {src => shark/src}/test/hive/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d (100%) rename {src => shark/src}/test/hive/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f (100%) rename {src => shark/src}/test/hive/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 (100%) rename {src => shark/src}/test/hive/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 (100%) rename {src => shark/src}/test/hive/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 (100%) rename {src => shark/src}/test/hive/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf (100%) rename {src => shark/src}/test/hive/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 (100%) rename {src => shark/src}/test/hive/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 (100%) rename {src => shark/src}/test/hive/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 (100%) rename {src => shark/src}/test/hive/golden/escape1-3-4267651148da591da38737028fdbd80 (100%) rename {src => shark/src}/test/hive/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 (100%) rename {src => shark/src}/test/hive/golden/escape1-5-70729c3d79ded87e884c176138174645 (100%) rename {src => shark/src}/test/hive/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 (100%) rename {src => shark/src}/test/hive/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 (100%) rename {src => shark/src}/test/hive/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 (100%) rename {src => shark/src}/test/hive/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 (100%) rename {src => shark/src}/test/hive/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e (100%) rename {src => shark/src}/test/hive/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 (100%) rename {src => shark/src}/test/hive/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe (100%) rename {src => shark/src}/test/hive/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 (100%) rename {src => shark/src}/test/hive/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 (100%) rename {src => shark/src}/test/hive/golden/escape2-7-70729c3d79ded87e884c176138174645 (100%) rename {src => shark/src}/test/hive/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 (100%) rename {src => shark/src}/test/hive/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 (100%) rename {src => shark/src}/test/hive/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf (100%) rename {src => shark/src}/test/hive/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 (100%) rename {src => shark/src}/test/hive/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb (100%) rename {src => shark/src}/test/hive/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 (100%) rename {src => shark/src}/test/hive/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 (100%) rename {src => shark/src}/test/hive/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 (100%) rename {src => shark/src}/test/hive/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 (100%) rename {src => shark/src}/test/hive/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 (100%) rename {src => shark/src}/test/hive/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 (100%) rename {src => shark/src}/test/hive/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 (100%) rename {src => shark/src}/test/hive/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 (100%) rename {src => shark/src}/test/hive/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd (100%) rename {src => shark/src}/test/hive/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 (100%) rename {src => shark/src}/test/hive/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 (100%) rename {src => shark/src}/test/hive/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd (100%) rename {src => shark/src}/test/hive/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 (100%) rename {src => shark/src}/test/hive/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 (100%) rename {src => shark/src}/test/hive/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 (100%) rename {src => shark/src}/test/hive/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 (100%) rename {src => shark/src}/test/hive/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 (100%) rename {src => shark/src}/test/hive/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd (100%) rename {src => shark/src}/test/hive/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 (100%) rename {src => shark/src}/test/hive/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d (100%) rename {src => shark/src}/test/hive/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 (100%) rename {src => shark/src}/test/hive/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 (100%) rename {src => shark/src}/test/hive/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd (100%) rename {src => shark/src}/test/hive/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 (100%) rename {src => shark/src}/test/hive/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 (100%) rename {src => shark/src}/test/hive/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd (100%) rename {src => shark/src}/test/hive/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 (100%) rename {src => shark/src}/test/hive/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 (100%) rename {src => shark/src}/test/hive/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd (100%) rename {src => shark/src}/test/hive/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 (100%) rename {src => shark/src}/test/hive/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 (100%) rename {src => shark/src}/test/hive/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd (100%) rename {src => shark/src}/test/hive/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 (100%) rename {src => shark/src}/test/hive/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 (100%) rename {src => shark/src}/test/hive/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a (100%) rename {src => shark/src}/test/hive/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 (100%) rename {src => shark/src}/test/hive/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 (100%) rename {src => shark/src}/test/hive/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 (100%) rename {src => shark/src}/test/hive/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd (100%) rename {src => shark/src}/test/hive/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e (100%) rename {src => shark/src}/test/hive/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 (100%) rename {src => shark/src}/test/hive/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 (100%) rename {src => shark/src}/test/hive/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 (100%) rename {src => shark/src}/test/hive/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd (100%) rename {src => shark/src}/test/hive/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 (100%) rename {src => shark/src}/test/hive/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 (100%) rename {src => shark/src}/test/hive/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc (100%) rename {src => shark/src}/test/hive/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 (100%) rename {src => shark/src}/test/hive/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 (100%) rename {src => shark/src}/test/hive/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a (100%) rename {src => shark/src}/test/hive/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 (100%) rename {src => shark/src}/test/hive/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 (100%) rename {src => shark/src}/test/hive/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd (100%) rename {src => shark/src}/test/hive/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d (100%) rename {src => shark/src}/test/hive/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 (100%) rename {src => shark/src}/test/hive/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 (100%) rename {src => shark/src}/test/hive/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 (100%) rename {src => shark/src}/test/hive/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 (100%) rename {src => shark/src}/test/hive/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 (100%) rename {src => shark/src}/test/hive/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b (100%) rename {src => shark/src}/test/hive/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e (100%) rename {src => shark/src}/test/hive/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 (100%) rename {src => shark/src}/test/hive/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 (100%) rename {src => shark/src}/test/hive/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 (100%) rename {src => shark/src}/test/hive/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e (100%) rename {src => shark/src}/test/hive/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 (100%) rename {src => shark/src}/test/hive/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby1-2-f90acd59ace31e16059bae52583188cc (100%) rename {src => shark/src}/test/hive/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d (100%) rename {src => shark/src}/test/hive/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 (100%) rename {src => shark/src}/test/hive/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 (100%) rename {src => shark/src}/test/hive/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea (100%) rename {src => shark/src}/test/hive/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 (100%) rename {src => shark/src}/test/hive/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 (100%) rename {src => shark/src}/test/hive/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 (100%) rename {src => shark/src}/test/hive/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 (100%) rename {src => shark/src}/test/hive/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f (100%) rename {src => shark/src}/test/hive/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd (100%) rename {src => shark/src}/test/hive/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d (100%) rename {src => shark/src}/test/hive/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 (100%) rename {src => shark/src}/test/hive/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 (100%) rename {src => shark/src}/test/hive/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 (100%) rename {src => shark/src}/test/hive/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe (100%) rename {src => shark/src}/test/hive/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 (100%) rename {src => shark/src}/test/hive/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc (100%) rename {src => shark/src}/test/hive/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 (100%) rename {src => shark/src}/test/hive/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea (100%) rename {src => shark/src}/test/hive/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 (100%) rename {src => shark/src}/test/hive/golden/groupby2-0-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b (100%) rename {src => shark/src}/test/hive/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 (100%) rename {src => shark/src}/test/hive/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 (100%) rename {src => shark/src}/test/hive/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c (100%) rename {src => shark/src}/test/hive/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 (100%) rename {src => shark/src}/test/hive/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 (100%) rename {src => shark/src}/test/hive/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 (100%) rename {src => shark/src}/test/hive/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c (100%) rename {src => shark/src}/test/hive/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 (100%) rename {src => shark/src}/test/hive/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 (100%) rename {src => shark/src}/test/hive/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 (100%) rename {src => shark/src}/test/hive/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b (100%) rename {src => shark/src}/test/hive/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 (100%) rename {src => shark/src}/test/hive/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 (100%) rename {src => shark/src}/test/hive/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b (100%) rename {src => shark/src}/test/hive/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 (100%) rename {src => shark/src}/test/hive/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 (100%) rename {src => shark/src}/test/hive/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 (100%) rename {src => shark/src}/test/hive/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 (100%) rename {src => shark/src}/test/hive/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e (100%) rename {src => shark/src}/test/hive/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 (100%) rename {src => shark/src}/test/hive/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f (100%) rename {src => shark/src}/test/hive/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 (100%) rename {src => shark/src}/test/hive/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 (100%) rename {src => shark/src}/test/hive/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f (100%) rename {src => shark/src}/test/hive/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 (100%) rename {src => shark/src}/test/hive/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 (100%) rename {src => shark/src}/test/hive/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e (100%) rename {src => shark/src}/test/hive/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac (100%) rename {src => shark/src}/test/hive/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 (100%) rename {src => shark/src}/test/hive/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac (100%) rename {src => shark/src}/test/hive/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 (100%) rename {src => shark/src}/test/hive/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac (100%) rename {src => shark/src}/test/hive/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 (100%) rename {src => shark/src}/test/hive/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac (100%) rename {src => shark/src}/test/hive/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 (100%) rename {src => shark/src}/test/hive/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a (100%) rename {src => shark/src}/test/hive/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c (100%) rename {src => shark/src}/test/hive/golden/groupby7-5-ed76c0068780120a6f23feefee303403 (100%) rename {src => shark/src}/test/hive/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 (100%) rename {src => shark/src}/test/hive/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd (100%) rename {src => shark/src}/test/hive/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 (100%) rename {src => shark/src}/test/hive/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a (100%) rename {src => shark/src}/test/hive/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 (100%) rename {src => shark/src}/test/hive/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f (100%) rename {src => shark/src}/test/hive/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a (100%) rename {src => shark/src}/test/hive/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d (100%) rename {src => shark/src}/test/hive/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f (100%) rename {src => shark/src}/test/hive/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e (100%) rename {src => shark/src}/test/hive/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d (100%) rename {src => shark/src}/test/hive/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a (100%) rename {src => shark/src}/test/hive/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d (100%) rename {src => shark/src}/test/hive/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f (100%) rename {src => shark/src}/test/hive/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a (100%) rename {src => shark/src}/test/hive/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d (100%) rename {src => shark/src}/test/hive/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f (100%) rename {src => shark/src}/test/hive/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a (100%) rename {src => shark/src}/test/hive/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d (100%) rename {src => shark/src}/test/hive/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f (100%) rename {src => shark/src}/test/hive/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 (100%) rename {src => shark/src}/test/hive/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 (100%) rename {src => shark/src}/test/hive/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e (100%) rename {src => shark/src}/test/hive/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f (100%) rename {src => shark/src}/test/hive/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 (100%) rename {src => shark/src}/test/hive/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed (100%) rename {src => shark/src}/test/hive/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b (100%) rename {src => shark/src}/test/hive/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e (100%) rename {src => shark/src}/test/hive/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f (100%) rename {src => shark/src}/test/hive/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 (100%) rename {src => shark/src}/test/hive/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 (100%) rename {src => shark/src}/test/hive/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e (100%) rename {src => shark/src}/test/hive/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 (100%) rename {src => shark/src}/test/hive/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e (100%) rename {src => shark/src}/test/hive/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 (100%) rename {src => shark/src}/test/hive/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 (100%) rename {src => shark/src}/test/hive/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 (100%) rename {src => shark/src}/test/hive/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 (100%) rename {src => shark/src}/test/hive/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a (100%) rename {src => shark/src}/test/hive/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 (100%) rename {src => shark/src}/test/hive/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 (100%) rename {src => shark/src}/test/hive/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 (100%) rename {src => shark/src}/test/hive/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 (100%) rename {src => shark/src}/test/hive/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 (100%) rename {src => shark/src}/test/hive/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 (100%) rename {src => shark/src}/test/hive/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 (100%) rename {src => shark/src}/test/hive/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 (100%) rename {src => shark/src}/test/hive/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 (100%) rename {src => shark/src}/test/hive/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 (100%) rename {src => shark/src}/test/hive/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad (100%) rename {src => shark/src}/test/hive/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 (100%) rename {src => shark/src}/test/hive/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 (100%) rename {src => shark/src}/test/hive/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 (100%) rename {src => shark/src}/test/hive/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 (100%) rename {src => shark/src}/test/hive/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb (100%) rename {src => shark/src}/test/hive/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 (100%) rename {src => shark/src}/test/hive/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 (100%) rename {src => shark/src}/test/hive/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 (100%) rename {src => shark/src}/test/hive/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 (100%) rename {src => shark/src}/test/hive/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 (100%) rename {src => shark/src}/test/hive/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f (100%) rename {src => shark/src}/test/hive/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 (100%) rename {src => shark/src}/test/hive/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb (100%) rename {src => shark/src}/test/hive/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 (100%) rename {src => shark/src}/test/hive/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac (100%) rename {src => shark/src}/test/hive/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c (100%) rename {src => shark/src}/test/hive/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d (100%) rename {src => shark/src}/test/hive/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 (100%) rename {src => shark/src}/test/hive/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 (100%) rename {src => shark/src}/test/hive/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c (100%) rename {src => shark/src}/test/hive/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d (100%) rename {src => shark/src}/test/hive/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 (100%) rename {src => shark/src}/test/hive/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 (100%) rename {src => shark/src}/test/hive/golden/hash-0-a658b129316d666d4b01c1581eed1c1f (100%) rename {src => shark/src}/test/hive/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf (100%) rename {src => shark/src}/test/hive/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 (100%) rename {src => shark/src}/test/hive/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 (100%) rename {src => shark/src}/test/hive/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a (100%) rename {src => shark/src}/test/hive/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 (100%) rename {src => shark/src}/test/hive/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 (100%) rename {src => shark/src}/test/hive/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 (100%) rename {src => shark/src}/test/hive/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 (100%) rename {src => shark/src}/test/hive/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 (100%) rename {src => shark/src}/test/hive/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 (100%) rename {src => shark/src}/test/hive/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 (100%) rename {src => shark/src}/test/hive/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 (100%) rename {src => shark/src}/test/hive/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 (100%) rename {src => shark/src}/test/hive/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d (100%) rename {src => shark/src}/test/hive/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad (100%) rename {src => shark/src}/test/hive/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 (100%) rename {src => shark/src}/test/hive/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f (100%) rename {src => shark/src}/test/hive/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 (100%) rename {src => shark/src}/test/hive/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 (100%) rename {src => shark/src}/test/hive/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e (100%) rename {src => shark/src}/test/hive/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 (100%) rename {src => shark/src}/test/hive/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 (100%) rename {src => shark/src}/test/hive/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf (100%) rename {src => shark/src}/test/hive/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca (100%) rename {src => shark/src}/test/hive/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 (100%) rename {src => shark/src}/test/hive/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 (100%) rename {src => shark/src}/test/hive/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe (100%) rename {src => shark/src}/test/hive/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f (100%) rename {src => shark/src}/test/hive/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 (100%) rename {src => shark/src}/test/hive/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 (100%) rename {src => shark/src}/test/hive/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 (100%) rename {src => shark/src}/test/hive/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 (100%) rename {src => shark/src}/test/hive/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 (100%) rename {src => shark/src}/test/hive/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 (100%) rename {src => shark/src}/test/hive/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 (100%) rename {src => shark/src}/test/hive/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 (100%) rename {src => shark/src}/test/hive/golden/index_creation-37-9334418431eca405f13206bd8db42a1b (100%) rename {src => shark/src}/test/hive/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 (100%) rename {src => shark/src}/test/hive/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d (100%) rename {src => shark/src}/test/hive/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 (100%) rename {src => shark/src}/test/hive/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 (100%) rename {src => shark/src}/test/hive/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 (100%) rename {src => shark/src}/test/hive/golden/index_creation-7-9334418431eca405f13206bd8db42a1b (100%) rename {src => shark/src}/test/hive/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 (100%) rename {src => shark/src}/test/hive/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c (100%) rename {src => shark/src}/test/hive/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 (100%) rename {src => shark/src}/test/hive/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 (100%) rename {src => shark/src}/test/hive/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b (100%) rename {src => shark/src}/test/hive/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 (100%) rename {src => shark/src}/test/hive/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 (100%) rename {src => shark/src}/test/hive/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c (100%) rename {src => shark/src}/test/hive/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 (100%) rename {src => shark/src}/test/hive/golden/index_serde-4-d7547751c37375a9238043bbe250e716 (100%) rename {src => shark/src}/test/hive/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 (100%) rename {src => shark/src}/test/hive/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae (100%) rename {src => shark/src}/test/hive/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 (100%) rename {src => shark/src}/test/hive/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 (100%) rename {src => shark/src}/test/hive/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 (100%) rename {src => shark/src}/test/hive/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e (100%) rename {src => shark/src}/test/hive/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 (100%) rename {src => shark/src}/test/hive/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd (100%) rename {src => shark/src}/test/hive/golden/innerjoin-0-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e (100%) rename {src => shark/src}/test/hive/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 (100%) rename {src => shark/src}/test/hive/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 (100%) rename {src => shark/src}/test/hive/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 (100%) rename {src => shark/src}/test/hive/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a (100%) rename {src => shark/src}/test/hive/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce (100%) rename {src => shark/src}/test/hive/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 (100%) rename {src => shark/src}/test/hive/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 (100%) rename {src => shark/src}/test/hive/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b (100%) rename {src => shark/src}/test/hive/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 (100%) rename {src => shark/src}/test/hive/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 (100%) rename {src => shark/src}/test/hive/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 (100%) rename {src => shark/src}/test/hive/golden/input-1-6558e385bb08991302a72076dd7b7ff5 (100%) rename {src => shark/src}/test/hive/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 (100%) rename {src => shark/src}/test/hive/golden/input0-1-efefd4364cd2790447fb0f908e87501f (100%) rename {src => shark/src}/test/hive/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 (100%) rename {src => shark/src}/test/hive/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 (100%) rename {src => shark/src}/test/hive/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 (100%) rename {src => shark/src}/test/hive/golden/input10-0-acdd72fda64c5463ce29f31020012d42 (100%) rename {src => shark/src}/test/hive/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 (100%) rename {src => shark/src}/test/hive/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 (100%) rename {src => shark/src}/test/hive/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb (100%) rename {src => shark/src}/test/hive/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 (100%) rename {src => shark/src}/test/hive/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 (100%) rename {src => shark/src}/test/hive/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 (100%) rename {src => shark/src}/test/hive/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 (100%) rename {src => shark/src}/test/hive/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e (100%) rename {src => shark/src}/test/hive/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 (100%) rename {src => shark/src}/test/hive/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c (100%) rename {src => shark/src}/test/hive/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 (100%) rename {src => shark/src}/test/hive/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 (100%) rename {src => shark/src}/test/hive/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 (100%) rename {src => shark/src}/test/hive/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf (100%) rename {src => shark/src}/test/hive/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 (100%) rename {src => shark/src}/test/hive/golden/input12-7-16da1c1293626b943343443368679e9c (100%) rename {src => shark/src}/test/hive/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd (100%) rename {src => shark/src}/test/hive/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f (100%) rename {src => shark/src}/test/hive/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 (100%) rename {src => shark/src}/test/hive/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 (100%) rename {src => shark/src}/test/hive/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 (100%) rename {src => shark/src}/test/hive/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf (100%) rename {src => shark/src}/test/hive/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 (100%) rename {src => shark/src}/test/hive/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c (100%) rename {src => shark/src}/test/hive/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd (100%) rename {src => shark/src}/test/hive/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c (100%) rename {src => shark/src}/test/hive/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a (100%) rename {src => shark/src}/test/hive/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 (100%) rename {src => shark/src}/test/hive/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf (100%) rename {src => shark/src}/test/hive/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 (100%) rename {src => shark/src}/test/hive/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input16_cc-0-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 (100%) rename {src => shark/src}/test/hive/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 (100%) rename {src => shark/src}/test/hive/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 (100%) rename {src => shark/src}/test/hive/golden/input17-2-21166e268096f6ec67f4f57ec333e901 (100%) rename {src => shark/src}/test/hive/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a (100%) rename {src => shark/src}/test/hive/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc (100%) rename {src => shark/src}/test/hive/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 (100%) rename {src => shark/src}/test/hive/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 (100%) rename {src => shark/src}/test/hive/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 (100%) rename {src => shark/src}/test/hive/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e (100%) rename {src => shark/src}/test/hive/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 (100%) rename {src => shark/src}/test/hive/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 (100%) rename {src => shark/src}/test/hive/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c (100%) rename {src => shark/src}/test/hive/golden/input2-1-e0efeda558cd0194f4764a5735147b16 (100%) rename {src => shark/src}/test/hive/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd (100%) rename {src => shark/src}/test/hive/golden/input2-3-45b9f786f64afa5e039d1856fe926649 (100%) rename {src => shark/src}/test/hive/golden/input2-4-235f92683416fab031e6e7490487b15b (100%) rename {src => shark/src}/test/hive/golden/input2-5-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/input2-6-4a6b9674c126337f71834f11613f996d (100%) rename {src => shark/src}/test/hive/golden/input2-7-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 (100%) rename {src => shark/src}/test/hive/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 (100%) rename {src => shark/src}/test/hive/golden/input21-0-7f40c525398953892ffd6283f54ba427 (100%) rename {src => shark/src}/test/hive/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 (100%) rename {src => shark/src}/test/hive/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 (100%) rename {src => shark/src}/test/hive/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe (100%) rename {src => shark/src}/test/hive/golden/input22-0-b9cd4041ca510639838a97376747b498 (100%) rename {src => shark/src}/test/hive/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 (100%) rename {src => shark/src}/test/hive/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 (100%) rename {src => shark/src}/test/hive/golden/input22-3-8285c1934441b12f6c016e13cb410e79 (100%) rename {src => shark/src}/test/hive/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf (100%) rename {src => shark/src}/test/hive/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb (100%) rename {src => shark/src}/test/hive/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 (100%) rename {src => shark/src}/test/hive/golden/input24-1-f57485de80ad64f9931bf92514fa1 (100%) rename {src => shark/src}/test/hive/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 (100%) rename {src => shark/src}/test/hive/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 (100%) rename {src => shark/src}/test/hive/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 (100%) rename {src => shark/src}/test/hive/golden/input25-1-f57485de80ad64f9931bf92514fa1 (100%) rename {src => shark/src}/test/hive/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 (100%) rename {src => shark/src}/test/hive/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 (100%) rename {src => shark/src}/test/hive/golden/input25-4-72c74c55dd7d85d194448e9c58488938 (100%) rename {src => shark/src}/test/hive/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 (100%) rename {src => shark/src}/test/hive/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 (100%) rename {src => shark/src}/test/hive/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a (100%) rename {src => shark/src}/test/hive/golden/input28-1-f57485de80ad64f9931bf92514fa1 (100%) rename {src => shark/src}/test/hive/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c (100%) rename {src => shark/src}/test/hive/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 (100%) rename {src => shark/src}/test/hive/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f (100%) rename {src => shark/src}/test/hive/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 (100%) rename {src => shark/src}/test/hive/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff (100%) rename {src => shark/src}/test/hive/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d (100%) rename {src => shark/src}/test/hive/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 (100%) rename {src => shark/src}/test/hive/golden/input3-11-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 (100%) rename {src => shark/src}/test/hive/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a (100%) rename {src => shark/src}/test/hive/golden/input3-14-efee6816e20fe61595a4a2a991071219 (100%) rename {src => shark/src}/test/hive/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 (100%) rename {src => shark/src}/test/hive/golden/input3-3-1c5990b1aed2be48311810dae3019994 (100%) rename {src => shark/src}/test/hive/golden/input3-4-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 (100%) rename {src => shark/src}/test/hive/golden/input3-6-ba8c440158c2519353d02471bfb05694 (100%) rename {src => shark/src}/test/hive/golden/input3-7-1c5990b1aed2be48311810dae3019994 (100%) rename {src => shark/src}/test/hive/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be (100%) rename {src => shark/src}/test/hive/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e (100%) rename {src => shark/src}/test/hive/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c (100%) rename {src => shark/src}/test/hive/golden/input30-1-90c0d1a75de78c405413fd627caea4ab (100%) rename {src => shark/src}/test/hive/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/input30-3-c21dba410fb07a098f93430a9d21df79 (100%) rename {src => shark/src}/test/hive/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 (100%) rename {src => shark/src}/test/hive/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 (100%) rename {src => shark/src}/test/hive/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 (100%) rename {src => shark/src}/test/hive/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f (100%) rename {src => shark/src}/test/hive/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/input31-1-c21dba410fb07a098f93430a9d21df79 (100%) rename {src => shark/src}/test/hive/golden/input31-2-705764f8f7cab9378964af30b83f7fe (100%) rename {src => shark/src}/test/hive/golden/input31-3-50c905261882f7fd8539fdd91e68151f (100%) rename {src => shark/src}/test/hive/golden/input31-4-2f886fa357df9342733551fa1b53f913 (100%) rename {src => shark/src}/test/hive/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/input32-1-c21dba410fb07a098f93430a9d21df79 (100%) rename {src => shark/src}/test/hive/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 (100%) rename {src => shark/src}/test/hive/golden/input32-3-b0070890240c15d647af59f41b77ba3d (100%) rename {src => shark/src}/test/hive/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 (100%) rename {src => shark/src}/test/hive/golden/input32-5-8789d32fc5b784fe2d171566732c573e (100%) rename {src => shark/src}/test/hive/golden/input37-0-86e2e274650fb56651607ea10d356fc0 (100%) rename {src => shark/src}/test/hive/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 (100%) rename {src => shark/src}/test/hive/golden/input39-0-7bd12162381231be9d578797818957a7 (100%) rename {src => shark/src}/test/hive/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 (100%) rename {src => shark/src}/test/hive/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 (100%) rename {src => shark/src}/test/hive/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 (100%) rename {src => shark/src}/test/hive/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 (100%) rename {src => shark/src}/test/hive/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 (100%) rename {src => shark/src}/test/hive/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d (100%) rename {src => shark/src}/test/hive/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 (100%) rename {src => shark/src}/test/hive/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e (100%) rename {src => shark/src}/test/hive/golden/input39-4-53453776bf062f28d371fc7336b7eae2 (100%) rename {src => shark/src}/test/hive/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/input39-6-763ab5853bff619e6525c01e46b2a923 (100%) rename {src => shark/src}/test/hive/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e (100%) rename {src => shark/src}/test/hive/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 (100%) rename {src => shark/src}/test/hive/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 (100%) rename {src => shark/src}/test/hive/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 (100%) rename {src => shark/src}/test/hive/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 (100%) rename {src => shark/src}/test/hive/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 (100%) rename {src => shark/src}/test/hive/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d (100%) rename {src => shark/src}/test/hive/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece (100%) rename {src => shark/src}/test/hive/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d (100%) rename {src => shark/src}/test/hive/golden/input3_limit-5-3664b564747487df13a5d109837219b5 (100%) rename {src => shark/src}/test/hive/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 (100%) rename {src => shark/src}/test/hive/golden/input4-0-b9cd4041ca510639838a97376747b498 (100%) rename {src => shark/src}/test/hive/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 (100%) rename {src => shark/src}/test/hive/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 (100%) rename {src => shark/src}/test/hive/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 (100%) rename {src => shark/src}/test/hive/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 (100%) rename {src => shark/src}/test/hive/golden/input40-0-37e74908fd43254141ae3664f13a48d (100%) rename {src => shark/src}/test/hive/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 (100%) rename {src => shark/src}/test/hive/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 (100%) rename {src => shark/src}/test/hive/golden/input40-3-12f634800ffae17942ddd789a01af227 (100%) rename {src => shark/src}/test/hive/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e (100%) rename {src => shark/src}/test/hive/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 (100%) rename {src => shark/src}/test/hive/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 (100%) rename {src => shark/src}/test/hive/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 (100%) rename {src => shark/src}/test/hive/golden/input41-0-763ab5853bff619e6525c01e46b2a923 (100%) rename {src => shark/src}/test/hive/golden/input41-1-8112b3a278e8337dbd6f017ae178658b (100%) rename {src => shark/src}/test/hive/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 (100%) rename {src => shark/src}/test/hive/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee (100%) rename {src => shark/src}/test/hive/golden/input43-0-2baba8070f3585debc14b6bb3c83607a (100%) rename {src => shark/src}/test/hive/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 (100%) rename {src => shark/src}/test/hive/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f (100%) rename {src => shark/src}/test/hive/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca (100%) rename {src => shark/src}/test/hive/golden/input46-3-c185163787977498a4b84f39f983c431 (100%) rename {src => shark/src}/test/hive/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 (100%) rename {src => shark/src}/test/hive/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab (100%) rename {src => shark/src}/test/hive/golden/input49-0-2bd546beeb607da5b925ffbea20a741c (100%) rename {src => shark/src}/test/hive/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d (100%) rename {src => shark/src}/test/hive/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 (100%) rename {src => shark/src}/test/hive/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 (100%) rename {src => shark/src}/test/hive/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 (100%) rename {src => shark/src}/test/hive/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 (100%) rename {src => shark/src}/test/hive/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 (100%) rename {src => shark/src}/test/hive/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d (100%) rename {src => shark/src}/test/hive/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af (100%) rename {src => shark/src}/test/hive/golden/input5-0-659e06570690cceeb3f37e10e855d2ea (100%) rename {src => shark/src}/test/hive/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 (100%) rename {src => shark/src}/test/hive/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad (100%) rename {src => shark/src}/test/hive/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input6-0-659e06570690cceeb3f37e10e855d2ea (100%) rename {src => shark/src}/test/hive/golden/input6-1-21149162906e31964a328b6cb2d5286e (100%) rename {src => shark/src}/test/hive/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c (100%) rename {src => shark/src}/test/hive/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 (100%) rename {src => shark/src}/test/hive/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 (100%) rename {src => shark/src}/test/hive/golden/input7-2-c740098a289f9a4267d26fc10347a0dc (100%) rename {src => shark/src}/test/hive/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa (100%) rename {src => shark/src}/test/hive/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa (100%) rename {src => shark/src}/test/hive/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba (100%) rename {src => shark/src}/test/hive/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 (100%) rename {src => shark/src}/test/hive/golden/input9-1-b30105391cb53915997a84ca3137bd0a (100%) rename {src => shark/src}/test/hive/golden/input9-2-171ea277f05a642699a0f91bacffccd8 (100%) rename {src => shark/src}/test/hive/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e (100%) rename {src => shark/src}/test/hive/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 (100%) rename {src => shark/src}/test/hive/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be (100%) rename {src => shark/src}/test/hive/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 (100%) rename {src => shark/src}/test/hive/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 (100%) rename {src => shark/src}/test/hive/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d (100%) rename {src => shark/src}/test/hive/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc (100%) rename {src => shark/src}/test/hive/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 (100%) rename {src => shark/src}/test/hive/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c (100%) rename {src => shark/src}/test/hive/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a (100%) rename {src => shark/src}/test/hive/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 (100%) rename {src => shark/src}/test/hive/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 (100%) rename {src => shark/src}/test/hive/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 (100%) rename {src => shark/src}/test/hive/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb (100%) rename {src => shark/src}/test/hive/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 (100%) rename {src => shark/src}/test/hive/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 (100%) rename {src => shark/src}/test/hive/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc (100%) rename {src => shark/src}/test/hive/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 (100%) rename {src => shark/src}/test/hive/golden/input_part10-1-35f67c0112026170d015b5f80a254bde (100%) rename {src => shark/src}/test/hive/golden/input_part10-2-40069c199502c2724ac2a2733f964248 (100%) rename {src => shark/src}/test/hive/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 (100%) rename {src => shark/src}/test/hive/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 (100%) rename {src => shark/src}/test/hive/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf (100%) rename {src => shark/src}/test/hive/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde (100%) rename {src => shark/src}/test/hive/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 (100%) rename {src => shark/src}/test/hive/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 (100%) rename {src => shark/src}/test/hive/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 (100%) rename {src => shark/src}/test/hive/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 (100%) rename {src => shark/src}/test/hive/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e (100%) rename {src => shark/src}/test/hive/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd (100%) rename {src => shark/src}/test/hive/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 (100%) rename {src => shark/src}/test/hive/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 (100%) rename {src => shark/src}/test/hive/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 (100%) rename {src => shark/src}/test/hive/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff (100%) rename {src => shark/src}/test/hive/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 (100%) rename {src => shark/src}/test/hive/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef (100%) rename {src => shark/src}/test/hive/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 (100%) rename {src => shark/src}/test/hive/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b (100%) rename {src => shark/src}/test/hive/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c (100%) rename {src => shark/src}/test/hive/golden/input_part5-2-d678533566fba4b922f01284538ca484 (100%) rename {src => shark/src}/test/hive/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 (100%) rename {src => shark/src}/test/hive/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d (100%) rename {src => shark/src}/test/hive/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 (100%) rename {src => shark/src}/test/hive/golden/input_part7-0-14448a12627b8bccf6828032f938f39e (100%) rename {src => shark/src}/test/hive/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 (100%) rename {src => shark/src}/test/hive/golden/input_part8-0-f60652796af548135f9e104b456840e3 (100%) rename {src => shark/src}/test/hive/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 (100%) rename {src => shark/src}/test/hive/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 (100%) rename {src => shark/src}/test/hive/golden/input_part9-1-e60c60afc073367464898b8396e8f643 (100%) rename {src => shark/src}/test/hive/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 (100%) rename {src => shark/src}/test/hive/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e (100%) rename {src => shark/src}/test/hive/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf (100%) rename {src => shark/src}/test/hive/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 (100%) rename {src => shark/src}/test/hive/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 (100%) rename {src => shark/src}/test/hive/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c (100%) rename {src => shark/src}/test/hive/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f (100%) rename {src => shark/src}/test/hive/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd (100%) rename {src => shark/src}/test/hive/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac (100%) rename {src => shark/src}/test/hive/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 (100%) rename {src => shark/src}/test/hive/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 (100%) rename {src => shark/src}/test/hive/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 (100%) rename {src => shark/src}/test/hive/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 (100%) rename {src => shark/src}/test/hive/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 (100%) rename {src => shark/src}/test/hive/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 (100%) rename {src => shark/src}/test/hive/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf (100%) rename {src => shark/src}/test/hive/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c (100%) rename {src => shark/src}/test/hive/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c (100%) rename {src => shark/src}/test/hive/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 (100%) rename {src => shark/src}/test/hive/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 (100%) rename {src => shark/src}/test/hive/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 (100%) rename {src => shark/src}/test/hive/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 (100%) rename {src => shark/src}/test/hive/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 (100%) rename {src => shark/src}/test/hive/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f (100%) rename {src => shark/src}/test/hive/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d (100%) rename {src => shark/src}/test/hive/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-13-45059a21c202b4658285738ee62a018a (100%) rename {src => shark/src}/test/hive/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe (100%) rename {src => shark/src}/test/hive/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a (100%) rename {src => shark/src}/test/hive/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 (100%) rename {src => shark/src}/test/hive/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac (100%) rename {src => shark/src}/test/hive/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 (100%) rename {src => shark/src}/test/hive/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 (100%) rename {src => shark/src}/test/hive/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac (100%) rename {src => shark/src}/test/hive/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 (100%) rename {src => shark/src}/test/hive/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 (100%) rename {src => shark/src}/test/hive/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 (100%) rename {src => shark/src}/test/hive/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c (100%) rename {src => shark/src}/test/hive/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d (100%) rename {src => shark/src}/test/hive/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 (100%) rename {src => shark/src}/test/hive/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 (100%) rename {src => shark/src}/test/hive/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 (100%) rename {src => shark/src}/test/hive/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef (100%) rename {src => shark/src}/test/hive/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 (100%) rename {src => shark/src}/test/hive/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef (100%) rename {src => shark/src}/test/hive/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 (100%) rename {src => shark/src}/test/hive/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef (100%) rename {src => shark/src}/test/hive/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 (100%) rename {src => shark/src}/test/hive/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 (100%) rename {src => shark/src}/test/hive/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b (100%) rename {src => shark/src}/test/hive/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 (100%) rename {src => shark/src}/test/hive/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb (100%) rename {src => shark/src}/test/hive/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 (100%) rename {src => shark/src}/test/hive/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 (100%) rename {src => shark/src}/test/hive/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 (100%) rename {src => shark/src}/test/hive/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb (100%) rename {src => shark/src}/test/hive/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 (100%) rename {src => shark/src}/test/hive/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 (100%) rename {src => shark/src}/test/hive/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb (100%) rename {src => shark/src}/test/hive/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d (100%) rename {src => shark/src}/test/hive/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b (100%) rename {src => shark/src}/test/hive/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa (100%) rename {src => shark/src}/test/hive/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-8-452111285dda40205ee587de8e972896 (100%) rename {src => shark/src}/test/hive/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-2-e51c25bae2408422a56826a263479468 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d (100%) rename {src => shark/src}/test/hive/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd (100%) rename {src => shark/src}/test/hive/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 (100%) rename {src => shark/src}/test/hive/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa (100%) rename {src => shark/src}/test/hive/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b (100%) rename {src => shark/src}/test/hive/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b (100%) rename {src => shark/src}/test/hive/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a (100%) rename {src => shark/src}/test/hive/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 (100%) rename {src => shark/src}/test/hive/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c (100%) rename {src => shark/src}/test/hive/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d (100%) rename {src => shark/src}/test/hive/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c (100%) rename {src => shark/src}/test/hive/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f (100%) rename {src => shark/src}/test/hive/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 (100%) rename {src => shark/src}/test/hive/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba (100%) rename {src => shark/src}/test/hive/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d (100%) rename {src => shark/src}/test/hive/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d (100%) rename {src => shark/src}/test/hive/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f (100%) rename {src => shark/src}/test/hive/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 (100%) rename {src => shark/src}/test/hive/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab (100%) rename {src => shark/src}/test/hive/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 (100%) rename {src => shark/src}/test/hive/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a (100%) rename {src => shark/src}/test/hive/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b (100%) rename {src => shark/src}/test/hive/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 (100%) rename {src => shark/src}/test/hive/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 (100%) rename {src => shark/src}/test/hive/golden/join0-1-83f5e512d6058a47f92b5218781f5070 (100%) rename {src => shark/src}/test/hive/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 (100%) rename {src => shark/src}/test/hive/golden/join1-0-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e (100%) rename {src => shark/src}/test/hive/golden/join1-2-7b07671864bbfdc627ee794932e32b1e (100%) rename {src => shark/src}/test/hive/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 (100%) rename {src => shark/src}/test/hive/golden/join1-4-35c7611180562dcd9dab834f41654095 (100%) rename {src => shark/src}/test/hive/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 (100%) rename {src => shark/src}/test/hive/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d (100%) rename {src => shark/src}/test/hive/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 (100%) rename {src => shark/src}/test/hive/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 (100%) rename {src => shark/src}/test/hive/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 (100%) rename {src => shark/src}/test/hive/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f (100%) rename {src => shark/src}/test/hive/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 (100%) rename {src => shark/src}/test/hive/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 (100%) rename {src => shark/src}/test/hive/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 (100%) rename {src => shark/src}/test/hive/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e (100%) rename {src => shark/src}/test/hive/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 (100%) rename {src => shark/src}/test/hive/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 (100%) rename {src => shark/src}/test/hive/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 (100%) rename {src => shark/src}/test/hive/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 (100%) rename {src => shark/src}/test/hive/golden/join14-6-de39302191b63d7aa8f92885b089fe2 (100%) rename {src => shark/src}/test/hive/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 (100%) rename {src => shark/src}/test/hive/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 (100%) rename {src => shark/src}/test/hive/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f (100%) rename {src => shark/src}/test/hive/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 (100%) rename {src => shark/src}/test/hive/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 (100%) rename {src => shark/src}/test/hive/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 (100%) rename {src => shark/src}/test/hive/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 (100%) rename {src => shark/src}/test/hive/golden/join15-0-930db2823512f7f8f34cac104f2162e0 (100%) rename {src => shark/src}/test/hive/golden/join15-1-81d76d3bf59889b07b413b6f88772667 (100%) rename {src => shark/src}/test/hive/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c (100%) rename {src => shark/src}/test/hive/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 (100%) rename {src => shark/src}/test/hive/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b (100%) rename {src => shark/src}/test/hive/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 (100%) rename {src => shark/src}/test/hive/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/join18-0-269843197b598569f4b50cbe911960a5 (100%) rename {src => shark/src}/test/hive/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 (100%) rename {src => shark/src}/test/hive/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 (100%) rename {src => shark/src}/test/hive/golden/join19-1-7e7d1f07c34dd4be5425264196201766 (100%) rename {src => shark/src}/test/hive/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca (100%) rename {src => shark/src}/test/hive/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c (100%) rename {src => shark/src}/test/hive/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a (100%) rename {src => shark/src}/test/hive/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 (100%) rename {src => shark/src}/test/hive/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e (100%) rename {src => shark/src}/test/hive/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 (100%) rename {src => shark/src}/test/hive/golden/join20-2-222655137ca9242f063c7717acbe4c65 (100%) rename {src => shark/src}/test/hive/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 (100%) rename {src => shark/src}/test/hive/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 (100%) rename {src => shark/src}/test/hive/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f (100%) rename {src => shark/src}/test/hive/golden/join22-0-97081b5f73052a612fb391a57ad9b7b (100%) rename {src => shark/src}/test/hive/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d (100%) rename {src => shark/src}/test/hive/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 (100%) rename {src => shark/src}/test/hive/golden/join24-0-721dfa03bfea05e55506c571b6c3585b (100%) rename {src => shark/src}/test/hive/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d (100%) rename {src => shark/src}/test/hive/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 (100%) rename {src => shark/src}/test/hive/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 (100%) rename {src => shark/src}/test/hive/golden/join25-1-360b3676605ccb3d89cf555342db87af (100%) rename {src => shark/src}/test/hive/golden/join25-2-ef930bed933341636c71622e6f89e9e9 (100%) rename {src => shark/src}/test/hive/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 (100%) rename {src => shark/src}/test/hive/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join26-0-775b1af8eae9b2250052be1b72b4086 (100%) rename {src => shark/src}/test/hive/golden/join26-1-10fbe6c99e93cfeafcd454261744728e (100%) rename {src => shark/src}/test/hive/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d (100%) rename {src => shark/src}/test/hive/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join27-0-360b3676605ccb3d89cf555342db87af (100%) rename {src => shark/src}/test/hive/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 (100%) rename {src => shark/src}/test/hive/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 (100%) rename {src => shark/src}/test/hive/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 (100%) rename {src => shark/src}/test/hive/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 (100%) rename {src => shark/src}/test/hive/golden/join28-1-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/join28-3-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 (100%) rename {src => shark/src}/test/hive/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 (100%) rename {src => shark/src}/test/hive/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join29-0-e78402f7585b17c76c32790571d749eb (100%) rename {src => shark/src}/test/hive/golden/join29-1-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/join29-3-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/join29-4-b0524de7985cdb7b57fd995721654856 (100%) rename {src => shark/src}/test/hive/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 (100%) rename {src => shark/src}/test/hive/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 (100%) rename {src => shark/src}/test/hive/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 (100%) rename {src => shark/src}/test/hive/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 (100%) rename {src => shark/src}/test/hive/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d (100%) rename {src => shark/src}/test/hive/golden/join30-2-194edec0592cf441617ca7caf9756baa (100%) rename {src => shark/src}/test/hive/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join31-0-619db2f9dd69aa236aa804cced726c9a (100%) rename {src => shark/src}/test/hive/golden/join31-1-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/join31-3-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 (100%) rename {src => shark/src}/test/hive/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 (100%) rename {src => shark/src}/test/hive/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join32-0-775b1af8eae9b2250052be1b72b4086 (100%) rename {src => shark/src}/test/hive/golden/join32-1-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/join32-3-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb (100%) rename {src => shark/src}/test/hive/golden/join32-5-425c5c35b23c42254dabec49aa6613bf (100%) rename {src => shark/src}/test/hive/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b (100%) rename {src => shark/src}/test/hive/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 (100%) rename {src => shark/src}/test/hive/golden/join33-0-775b1af8eae9b2250052be1b72b4086 (100%) rename {src => shark/src}/test/hive/golden/join33-1-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/join33-3-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb (100%) rename {src => shark/src}/test/hive/golden/join33-5-425c5c35b23c42254dabec49aa6613bf (100%) rename {src => shark/src}/test/hive/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join34-0-775b1af8eae9b2250052be1b72b4086 (100%) rename {src => shark/src}/test/hive/golden/join34-1-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/join34-3-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/join34-4-51cc5d253befd784a9d466c0b402f47a (100%) rename {src => shark/src}/test/hive/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 (100%) rename {src => shark/src}/test/hive/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 (100%) rename {src => shark/src}/test/hive/golden/join35-1-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/join35-3-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/join35-4-99cb704ccc2813400908a62571891dc7 (100%) rename {src => shark/src}/test/hive/golden/join35-5-60956f3307651237f52ffbe41c827c1c (100%) rename {src => shark/src}/test/hive/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 (100%) rename {src => shark/src}/test/hive/golden/join36-1-3aba153d62042206c0f001ea74980da (100%) rename {src => shark/src}/test/hive/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 (100%) rename {src => shark/src}/test/hive/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 (100%) rename {src => shark/src}/test/hive/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 (100%) rename {src => shark/src}/test/hive/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 (100%) rename {src => shark/src}/test/hive/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 (100%) rename {src => shark/src}/test/hive/golden/join36-7-171ede21302050a33468d95e329a38a3 (100%) rename {src => shark/src}/test/hive/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 (100%) rename {src => shark/src}/test/hive/golden/join37-1-360b3676605ccb3d89cf555342db87af (100%) rename {src => shark/src}/test/hive/golden/join37-2-2310f2b8726137811543e04db98dd1db (100%) rename {src => shark/src}/test/hive/golden/join37-3-c4b3a85965108ad7013ac3931598af09 (100%) rename {src => shark/src}/test/hive/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join38-0-42e6ad674a5142d05258a2d29578e046 (100%) rename {src => shark/src}/test/hive/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f (100%) rename {src => shark/src}/test/hive/golden/join38-2-eacdb3417c4a563982c488812d654c9 (100%) rename {src => shark/src}/test/hive/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 (100%) rename {src => shark/src}/test/hive/golden/join38-4-53d219706847e890de1dcd369563ebef (100%) rename {src => shark/src}/test/hive/golden/join39-0-54e9f7924abb40d1c601de908de6f25b (100%) rename {src => shark/src}/test/hive/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 (100%) rename {src => shark/src}/test/hive/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 (100%) rename {src => shark/src}/test/hive/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join4-0-531e91e11b3891627c2675935fda14cd (100%) rename {src => shark/src}/test/hive/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 (100%) rename {src => shark/src}/test/hive/golden/join4-2-dc967001beb776f3a859e9360823c361 (100%) rename {src => shark/src}/test/hive/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac (100%) rename {src => shark/src}/test/hive/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 (100%) rename {src => shark/src}/test/hive/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa (100%) rename {src => shark/src}/test/hive/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 (100%) rename {src => shark/src}/test/hive/golden/join40-12-aaddbef9069aba3ebeb310be74671cda (100%) rename {src => shark/src}/test/hive/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 (100%) rename {src => shark/src}/test/hive/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 (100%) rename {src => shark/src}/test/hive/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 (100%) rename {src => shark/src}/test/hive/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e (100%) rename {src => shark/src}/test/hive/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 (100%) rename {src => shark/src}/test/hive/golden/join40-7-222655137ca9242f063c7717acbe4c65 (100%) rename {src => shark/src}/test/hive/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 (100%) rename {src => shark/src}/test/hive/golden/join40-9-6b7404af014eeb4de196577f14ef18cc (100%) rename {src => shark/src}/test/hive/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 (100%) rename {src => shark/src}/test/hive/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 (100%) rename {src => shark/src}/test/hive/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a (100%) rename {src => shark/src}/test/hive/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 (100%) rename {src => shark/src}/test/hive/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e (100%) rename {src => shark/src}/test/hive/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 (100%) rename {src => shark/src}/test/hive/golden/join5-0-531e91e11b3891627c2675935fda14cd (100%) rename {src => shark/src}/test/hive/golden/join5-1-76feff1282895f38b673f52acfd7429e (100%) rename {src => shark/src}/test/hive/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 (100%) rename {src => shark/src}/test/hive/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/join6-0-531e91e11b3891627c2675935fda14cd (100%) rename {src => shark/src}/test/hive/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 (100%) rename {src => shark/src}/test/hive/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 (100%) rename {src => shark/src}/test/hive/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 (100%) rename {src => shark/src}/test/hive/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 (100%) rename {src => shark/src}/test/hive/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 (100%) rename {src => shark/src}/test/hive/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/join8-0-531e91e11b3891627c2675935fda14cd (100%) rename {src => shark/src}/test/hive/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 (100%) rename {src => shark/src}/test/hive/golden/join8-2-f1196bca86a749375da35f134206a8ca (100%) rename {src => shark/src}/test/hive/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a (100%) rename {src => shark/src}/test/hive/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 (100%) rename {src => shark/src}/test/hive/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a (100%) rename {src => shark/src}/test/hive/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e (100%) rename {src => shark/src}/test/hive/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e (100%) rename {src => shark/src}/test/hive/golden/join_1to1-12-6081eb976b4aef2132418510756a385b (100%) rename {src => shark/src}/test/hive/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c (100%) rename {src => shark/src}/test/hive/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e (100%) rename {src => shark/src}/test/hive/golden/join_1to1-18-6081eb976b4aef2132418510756a385b (100%) rename {src => shark/src}/test/hive/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c (100%) rename {src => shark/src}/test/hive/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e (100%) rename {src => shark/src}/test/hive/golden/join_1to1-24-6081eb976b4aef2132418510756a385b (100%) rename {src => shark/src}/test/hive/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c (100%) rename {src => shark/src}/test/hive/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e (100%) rename {src => shark/src}/test/hive/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e (100%) rename {src => shark/src}/test/hive/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-30-6081eb976b4aef2132418510756a385b (100%) rename {src => shark/src}/test/hive/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c (100%) rename {src => shark/src}/test/hive/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e (100%) rename {src => shark/src}/test/hive/golden/join_1to1-36-6081eb976b4aef2132418510756a385b (100%) rename {src => shark/src}/test/hive/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c (100%) rename {src => shark/src}/test/hive/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e (100%) rename {src => shark/src}/test/hive/golden/join_1to1-6-6081eb976b4aef2132418510756a385b (100%) rename {src => shark/src}/test/hive/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c (100%) rename {src => shark/src}/test/hive/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 (100%) rename {src => shark/src}/test/hive/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 (100%) rename {src => shark/src}/test/hive/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 (100%) rename {src => shark/src}/test/hive/golden/join_array-1-2c086fcf118fd2538bfa00724209091e (100%) rename {src => shark/src}/test/hive/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de (100%) rename {src => shark/src}/test/hive/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 (100%) rename {src => shark/src}/test/hive/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 (100%) rename {src => shark/src}/test/hive/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a (100%) rename {src => shark/src}/test/hive/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c (100%) rename {src => shark/src}/test/hive/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 (100%) rename {src => shark/src}/test/hive/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 (100%) rename {src => shark/src}/test/hive/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d (100%) rename {src => shark/src}/test/hive/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc (100%) rename {src => shark/src}/test/hive/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd (100%) rename {src => shark/src}/test/hive/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad (100%) rename {src => shark/src}/test/hive/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 (100%) rename {src => shark/src}/test/hive/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e (100%) rename {src => shark/src}/test/hive/golden/join_empty-3-3b2231f25886c152f222856256ac3473 (100%) rename {src => shark/src}/test/hive/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a (100%) rename {src => shark/src}/test/hive/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a (100%) rename {src => shark/src}/test/hive/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 (100%) rename {src => shark/src}/test/hive/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 (100%) rename {src => shark/src}/test/hive/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 (100%) rename {src => shark/src}/test/hive/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 (100%) rename {src => shark/src}/test/hive/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe (100%) rename {src => shark/src}/test/hive/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b (100%) rename {src => shark/src}/test/hive/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 (100%) rename {src => shark/src}/test/hive/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b (100%) rename {src => shark/src}/test/hive/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 (100%) rename {src => shark/src}/test/hive/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb (100%) rename {src => shark/src}/test/hive/golden/join_filters-106-870999978978b22f21997899f1e652b8 (100%) rename {src => shark/src}/test/hive/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 (100%) rename {src => shark/src}/test/hive/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e (100%) rename {src => shark/src}/test/hive/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 (100%) rename {src => shark/src}/test/hive/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 (100%) rename {src => shark/src}/test/hive/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba (100%) rename {src => shark/src}/test/hive/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 (100%) rename {src => shark/src}/test/hive/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 (100%) rename {src => shark/src}/test/hive/golden/join_filters-113-48c4978768872751832149d72cdf0ba (100%) rename {src => shark/src}/test/hive/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 (100%) rename {src => shark/src}/test/hive/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 (100%) rename {src => shark/src}/test/hive/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 (100%) rename {src => shark/src}/test/hive/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 (100%) rename {src => shark/src}/test/hive/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e (100%) rename {src => shark/src}/test/hive/golden/join_filters-119-7688784396db55ff148292a9755c918a (100%) rename {src => shark/src}/test/hive/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e (100%) rename {src => shark/src}/test/hive/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 (100%) rename {src => shark/src}/test/hive/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada (100%) rename {src => shark/src}/test/hive/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 (100%) rename {src => shark/src}/test/hive/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 (100%) rename {src => shark/src}/test/hive/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf (100%) rename {src => shark/src}/test/hive/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 (100%) rename {src => shark/src}/test/hive/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 (100%) rename {src => shark/src}/test/hive/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 (100%) rename {src => shark/src}/test/hive/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 (100%) rename {src => shark/src}/test/hive/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f (100%) rename {src => shark/src}/test/hive/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 (100%) rename {src => shark/src}/test/hive/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf (100%) rename {src => shark/src}/test/hive/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e (100%) rename {src => shark/src}/test/hive/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e (100%) rename {src => shark/src}/test/hive/golden/join_filters-23-a800b885358695758afdb719cdefa94f (100%) rename {src => shark/src}/test/hive/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 (100%) rename {src => shark/src}/test/hive/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 (100%) rename {src => shark/src}/test/hive/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca (100%) rename {src => shark/src}/test/hive/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 (100%) rename {src => shark/src}/test/hive/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 (100%) rename {src => shark/src}/test/hive/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 (100%) rename {src => shark/src}/test/hive/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 (100%) rename {src => shark/src}/test/hive/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf (100%) rename {src => shark/src}/test/hive/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f (100%) rename {src => shark/src}/test/hive/golden/join_filters-32-5978cd7936c296493a16a31b926043ab (100%) rename {src => shark/src}/test/hive/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 (100%) rename {src => shark/src}/test/hive/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 (100%) rename {src => shark/src}/test/hive/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe (100%) rename {src => shark/src}/test/hive/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b (100%) rename {src => shark/src}/test/hive/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 (100%) rename {src => shark/src}/test/hive/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b (100%) rename {src => shark/src}/test/hive/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 (100%) rename {src => shark/src}/test/hive/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 (100%) rename {src => shark/src}/test/hive/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb (100%) rename {src => shark/src}/test/hive/golden/join_filters-41-870999978978b22f21997899f1e652b8 (100%) rename {src => shark/src}/test/hive/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 (100%) rename {src => shark/src}/test/hive/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e (100%) rename {src => shark/src}/test/hive/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e (100%) rename {src => shark/src}/test/hive/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 (100%) rename {src => shark/src}/test/hive/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b (100%) rename {src => shark/src}/test/hive/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 (100%) rename {src => shark/src}/test/hive/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c (100%) rename {src => shark/src}/test/hive/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 (100%) rename {src => shark/src}/test/hive/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e (100%) rename {src => shark/src}/test/hive/golden/join_filters-50-3e6612a89e9124592e790594775054b1 (100%) rename {src => shark/src}/test/hive/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac (100%) rename {src => shark/src}/test/hive/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 (100%) rename {src => shark/src}/test/hive/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 (100%) rename {src => shark/src}/test/hive/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba (100%) rename {src => shark/src}/test/hive/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 (100%) rename {src => shark/src}/test/hive/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 (100%) rename {src => shark/src}/test/hive/golden/join_filters-57-48c4978768872751832149d72cdf0ba (100%) rename {src => shark/src}/test/hive/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 (100%) rename {src => shark/src}/test/hive/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 (100%) rename {src => shark/src}/test/hive/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 (100%) rename {src => shark/src}/test/hive/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 (100%) rename {src => shark/src}/test/hive/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 (100%) rename {src => shark/src}/test/hive/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e (100%) rename {src => shark/src}/test/hive/golden/join_filters-63-7688784396db55ff148292a9755c918a (100%) rename {src => shark/src}/test/hive/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 (100%) rename {src => shark/src}/test/hive/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada (100%) rename {src => shark/src}/test/hive/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 (100%) rename {src => shark/src}/test/hive/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 (100%) rename {src => shark/src}/test/hive/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 (100%) rename {src => shark/src}/test/hive/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 (100%) rename {src => shark/src}/test/hive/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d (100%) rename {src => shark/src}/test/hive/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e (100%) rename {src => shark/src}/test/hive/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 (100%) rename {src => shark/src}/test/hive/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d (100%) rename {src => shark/src}/test/hive/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 (100%) rename {src => shark/src}/test/hive/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa (100%) rename {src => shark/src}/test/hive/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 (100%) rename {src => shark/src}/test/hive/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 (100%) rename {src => shark/src}/test/hive/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e (100%) rename {src => shark/src}/test/hive/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 (100%) rename {src => shark/src}/test/hive/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf (100%) rename {src => shark/src}/test/hive/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 (100%) rename {src => shark/src}/test/hive/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 (100%) rename {src => shark/src}/test/hive/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 (100%) rename {src => shark/src}/test/hive/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 (100%) rename {src => shark/src}/test/hive/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 (100%) rename {src => shark/src}/test/hive/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f (100%) rename {src => shark/src}/test/hive/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf (100%) rename {src => shark/src}/test/hive/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e (100%) rename {src => shark/src}/test/hive/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e (100%) rename {src => shark/src}/test/hive/golden/join_filters-88-a800b885358695758afdb719cdefa94f (100%) rename {src => shark/src}/test/hive/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 (100%) rename {src => shark/src}/test/hive/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa (100%) rename {src => shark/src}/test/hive/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 (100%) rename {src => shark/src}/test/hive/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca (100%) rename {src => shark/src}/test/hive/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 (100%) rename {src => shark/src}/test/hive/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 (100%) rename {src => shark/src}/test/hive/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 (100%) rename {src => shark/src}/test/hive/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf (100%) rename {src => shark/src}/test/hive/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f (100%) rename {src => shark/src}/test/hive/golden/join_filters-97-5978cd7936c296493a16a31b926043ab (100%) rename {src => shark/src}/test/hive/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 (100%) rename {src => shark/src}/test/hive/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 (100%) rename {src => shark/src}/test/hive/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce (100%) rename {src => shark/src}/test/hive/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 (100%) rename {src => shark/src}/test/hive/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 (100%) rename {src => shark/src}/test/hive/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 (100%) rename {src => shark/src}/test/hive/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e (100%) rename {src => shark/src}/test/hive/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 (100%) rename {src => shark/src}/test/hive/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 (100%) rename {src => shark/src}/test/hive/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 (100%) rename {src => shark/src}/test/hive/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 (100%) rename {src => shark/src}/test/hive/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed (100%) rename {src => shark/src}/test/hive/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a (100%) rename {src => shark/src}/test/hive/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b (100%) rename {src => shark/src}/test/hive/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d (100%) rename {src => shark/src}/test/hive/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f (100%) rename {src => shark/src}/test/hive/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c (100%) rename {src => shark/src}/test/hive/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e (100%) rename {src => shark/src}/test/hive/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb (100%) rename {src => shark/src}/test/hive/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb (100%) rename {src => shark/src}/test/hive/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc (100%) rename {src => shark/src}/test/hive/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e (100%) rename {src => shark/src}/test/hive/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b (100%) rename {src => shark/src}/test/hive/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c (100%) rename {src => shark/src}/test/hive/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c (100%) rename {src => shark/src}/test/hive/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f (100%) rename {src => shark/src}/test/hive/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a (100%) rename {src => shark/src}/test/hive/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 (100%) rename {src => shark/src}/test/hive/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b (100%) rename {src => shark/src}/test/hive/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 (100%) rename {src => shark/src}/test/hive/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e (100%) rename {src => shark/src}/test/hive/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 (100%) rename {src => shark/src}/test/hive/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee (100%) rename {src => shark/src}/test/hive/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 (100%) rename {src => shark/src}/test/hive/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 (100%) rename {src => shark/src}/test/hive/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 (100%) rename {src => shark/src}/test/hive/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 (100%) rename {src => shark/src}/test/hive/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba (100%) rename {src => shark/src}/test/hive/golden/join_star-0-6001f83f7780835737e3262a34c2b832 (100%) rename {src => shark/src}/test/hive/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d (100%) rename {src => shark/src}/test/hive/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e (100%) rename {src => shark/src}/test/hive/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae (100%) rename {src => shark/src}/test/hive/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 (100%) rename {src => shark/src}/test/hive/golden/join_star-13-342b7249c9ce1484869169b1b33191cb (100%) rename {src => shark/src}/test/hive/golden/join_star-14-75513308d30b781fd2e06d81963c4363 (100%) rename {src => shark/src}/test/hive/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 (100%) rename {src => shark/src}/test/hive/golden/join_star-16-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 (100%) rename {src => shark/src}/test/hive/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa (100%) rename {src => shark/src}/test/hive/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 (100%) rename {src => shark/src}/test/hive/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 (100%) rename {src => shark/src}/test/hive/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 (100%) rename {src => shark/src}/test/hive/golden/join_star-22-429cfd140488242d569aee6594aae76b (100%) rename {src => shark/src}/test/hive/golden/join_star-23-bada259628918dbfb4837bcb58258530 (100%) rename {src => shark/src}/test/hive/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 (100%) rename {src => shark/src}/test/hive/golden/join_star-25-be5af28b6bccf529a31ef68619de699e (100%) rename {src => shark/src}/test/hive/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 (100%) rename {src => shark/src}/test/hive/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 (100%) rename {src => shark/src}/test/hive/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 (100%) rename {src => shark/src}/test/hive/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da (100%) rename {src => shark/src}/test/hive/golden/join_star-4-aaba115af7543b214820a48a694f2463 (100%) rename {src => shark/src}/test/hive/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da (100%) rename {src => shark/src}/test/hive/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a (100%) rename {src => shark/src}/test/hive/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a (100%) rename {src => shark/src}/test/hive/golden/join_star-8-a957982d8981ff0a35397ca449297024 (100%) rename {src => shark/src}/test/hive/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 (100%) rename {src => shark/src}/test/hive/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 (100%) rename {src => shark/src}/test/hive/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db (100%) rename {src => shark/src}/test/hive/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b (100%) rename {src => shark/src}/test/hive/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 (100%) rename {src => shark/src}/test/hive/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 (100%) rename {src => shark/src}/test/hive/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 (100%) rename {src => shark/src}/test/hive/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 (100%) rename {src => shark/src}/test/hive/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b (100%) rename {src => shark/src}/test/hive/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 (100%) rename {src => shark/src}/test/hive/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 (100%) rename {src => shark/src}/test/hive/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef (100%) rename {src => shark/src}/test/hive/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d (100%) rename {src => shark/src}/test/hive/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 (100%) rename {src => shark/src}/test/hive/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 (100%) rename {src => shark/src}/test/hive/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee (100%) rename {src => shark/src}/test/hive/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb (100%) rename {src => shark/src}/test/hive/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 (100%) rename {src => shark/src}/test/hive/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 (100%) rename {src => shark/src}/test/hive/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 (100%) rename {src => shark/src}/test/hive/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e (100%) rename {src => shark/src}/test/hive/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b (100%) rename {src => shark/src}/test/hive/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a (100%) rename {src => shark/src}/test/hive/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b (100%) rename {src => shark/src}/test/hive/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e (100%) rename {src => shark/src}/test/hive/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba (100%) rename {src => shark/src}/test/hive/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e (100%) rename {src => shark/src}/test/hive/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab (100%) rename {src => shark/src}/test/hive/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e (100%) rename {src => shark/src}/test/hive/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e (100%) rename {src => shark/src}/test/hive/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae (100%) rename {src => shark/src}/test/hive/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 (100%) rename {src => shark/src}/test/hive/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_cp-4-507876b610812506343e9d251645170b (100%) rename {src => shark/src}/test/hive/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a (100%) rename {src => shark/src}/test/hive/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe (100%) rename {src => shark/src}/test/hive/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa (100%) rename {src => shark/src}/test/hive/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d (100%) rename {src => shark/src}/test/hive/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e (100%) rename {src => shark/src}/test/hive/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f (100%) rename {src => shark/src}/test/hive/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 (100%) rename {src => shark/src}/test/hive/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f (100%) rename {src => shark/src}/test/hive/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa (100%) rename {src => shark/src}/test/hive/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b (100%) rename {src => shark/src}/test/hive/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 (100%) rename {src => shark/src}/test/hive/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 (100%) rename {src => shark/src}/test/hive/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a (100%) rename {src => shark/src}/test/hive/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 (100%) rename {src => shark/src}/test/hive/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 (100%) rename {src => shark/src}/test/hive/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c (100%) rename {src => shark/src}/test/hive/golden/literal_double-0-10ef1098e35d900983be3814de8f974f (100%) rename {src => shark/src}/test/hive/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 (100%) rename {src => shark/src}/test/hive/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 (100%) rename {src => shark/src}/test/hive/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 (100%) rename {src => shark/src}/test/hive/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 (100%) rename {src => shark/src}/test/hive/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 (100%) rename {src => shark/src}/test/hive/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb (100%) rename {src => shark/src}/test/hive/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 (100%) rename {src => shark/src}/test/hive/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 (100%) rename {src => shark/src}/test/hive/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e (100%) rename {src => shark/src}/test/hive/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 (100%) rename {src => shark/src}/test/hive/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 (100%) rename {src => shark/src}/test/hive/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f (100%) rename {src => shark/src}/test/hive/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 (100%) rename {src => shark/src}/test/hive/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 (100%) rename {src => shark/src}/test/hive/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 (100%) rename {src => shark/src}/test/hive/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc (100%) rename {src => shark/src}/test/hive/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 (100%) rename {src => shark/src}/test/hive/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 (100%) rename {src => shark/src}/test/hive/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 (100%) rename {src => shark/src}/test/hive/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e (100%) rename {src => shark/src}/test/hive/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 (100%) rename {src => shark/src}/test/hive/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 (100%) rename {src => shark/src}/test/hive/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e (100%) rename {src => shark/src}/test/hive/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 (100%) rename {src => shark/src}/test/hive/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 (100%) rename {src => shark/src}/test/hive/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c (100%) rename {src => shark/src}/test/hive/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 (100%) rename {src => shark/src}/test/hive/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 (100%) rename {src => shark/src}/test/hive/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 (100%) rename {src => shark/src}/test/hive/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc (100%) rename {src => shark/src}/test/hive/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 (100%) rename {src => shark/src}/test/hive/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 (100%) rename {src => shark/src}/test/hive/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 (100%) rename {src => shark/src}/test/hive/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 (100%) rename {src => shark/src}/test/hive/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 (100%) rename {src => shark/src}/test/hive/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 (100%) rename {src => shark/src}/test/hive/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e (100%) rename {src => shark/src}/test/hive/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 (100%) rename {src => shark/src}/test/hive/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 (100%) rename {src => shark/src}/test/hive/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 (100%) rename {src => shark/src}/test/hive/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 (100%) rename {src => shark/src}/test/hive/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 (100%) rename {src => shark/src}/test/hive/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 (100%) rename {src => shark/src}/test/hive/golden/lock2-3-27ad2962fed131f51ba802596ba37278 (100%) rename {src => shark/src}/test/hive/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a (100%) rename {src => shark/src}/test/hive/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca (100%) rename {src => shark/src}/test/hive/golden/lock3-0-27ad2962fed131f51ba802596ba37278 (100%) rename {src => shark/src}/test/hive/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a (100%) rename {src => shark/src}/test/hive/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 (100%) rename {src => shark/src}/test/hive/golden/lock3-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac (100%) rename {src => shark/src}/test/hive/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe (100%) rename {src => shark/src}/test/hive/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 (100%) rename {src => shark/src}/test/hive/golden/lock4-1-27ad2962fed131f51ba802596ba37278 (100%) rename {src => shark/src}/test/hive/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a (100%) rename {src => shark/src}/test/hive/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 (100%) rename {src => shark/src}/test/hive/golden/lock4-4-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac (100%) rename {src => shark/src}/test/hive/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe (100%) rename {src => shark/src}/test/hive/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd (100%) rename {src => shark/src}/test/hive/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 (100%) rename {src => shark/src}/test/hive/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 (100%) rename {src => shark/src}/test/hive/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef (100%) rename {src => shark/src}/test/hive/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 (100%) rename {src => shark/src}/test/hive/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 (100%) rename {src => shark/src}/test/hive/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 (100%) rename {src => shark/src}/test/hive/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 (100%) rename {src => shark/src}/test/hive/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d (100%) rename {src => shark/src}/test/hive/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e (100%) rename {src => shark/src}/test/hive/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a (100%) rename {src => shark/src}/test/hive/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 (100%) rename {src => shark/src}/test/hive/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c (100%) rename {src => shark/src}/test/hive/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe (100%) rename {src => shark/src}/test/hive/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e (100%) rename {src => shark/src}/test/hive/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c (100%) rename {src => shark/src}/test/hive/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 (100%) rename {src => shark/src}/test/hive/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b (100%) rename {src => shark/src}/test/hive/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc (100%) rename {src => shark/src}/test/hive/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c (100%) rename {src => shark/src}/test/hive/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b (100%) rename {src => shark/src}/test/hive/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a (100%) rename {src => shark/src}/test/hive/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c (100%) rename {src => shark/src}/test/hive/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f (100%) rename {src => shark/src}/test/hive/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab (100%) rename {src => shark/src}/test/hive/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c (100%) rename {src => shark/src}/test/hive/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 (100%) rename {src => shark/src}/test/hive/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca (100%) rename {src => shark/src}/test/hive/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c (100%) rename {src => shark/src}/test/hive/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb (100%) rename {src => shark/src}/test/hive/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 (100%) rename {src => shark/src}/test/hive/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 (100%) rename {src => shark/src}/test/hive/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a (100%) rename {src => shark/src}/test/hive/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 (100%) rename {src => shark/src}/test/hive/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 (100%) rename {src => shark/src}/test/hive/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b (100%) rename {src => shark/src}/test/hive/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 (100%) rename {src => shark/src}/test/hive/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/merge1-1-2c73c923962b91afdf0004a705432550 (100%) rename {src => shark/src}/test/hive/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 (100%) rename {src => shark/src}/test/hive/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 (100%) rename {src => shark/src}/test/hive/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 (100%) rename {src => shark/src}/test/hive/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 (100%) rename {src => shark/src}/test/hive/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 (100%) rename {src => shark/src}/test/hive/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 (100%) rename {src => shark/src}/test/hive/golden/merge1-3-1f560722f18ef618a0343313a0cac462 (100%) rename {src => shark/src}/test/hive/golden/merge1-4-3277fe538b66923cd879b45371838d2b (100%) rename {src => shark/src}/test/hive/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 (100%) rename {src => shark/src}/test/hive/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 (100%) rename {src => shark/src}/test/hive/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 (100%) rename {src => shark/src}/test/hive/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a (100%) rename {src => shark/src}/test/hive/golden/merge1-9-64678b0928c4fd054e1578458001c86 (100%) rename {src => shark/src}/test/hive/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 (100%) rename {src => shark/src}/test/hive/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 (100%) rename {src => shark/src}/test/hive/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 (100%) rename {src => shark/src}/test/hive/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a (100%) rename {src => shark/src}/test/hive/golden/merge2-14-64678b0928c4fd054e1578458001c86 (100%) rename {src => shark/src}/test/hive/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 (100%) rename {src => shark/src}/test/hive/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 (100%) rename {src => shark/src}/test/hive/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 (100%) rename {src => shark/src}/test/hive/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f (100%) rename {src => shark/src}/test/hive/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 (100%) rename {src => shark/src}/test/hive/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf (100%) rename {src => shark/src}/test/hive/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 (100%) rename {src => shark/src}/test/hive/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 (100%) rename {src => shark/src}/test/hive/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea (100%) rename {src => shark/src}/test/hive/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 (100%) rename {src => shark/src}/test/hive/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 (100%) rename {src => shark/src}/test/hive/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 (100%) rename {src => shark/src}/test/hive/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd (100%) rename {src => shark/src}/test/hive/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/merge4-10-692a197bd688b48f762e72978f54aa32 (100%) rename {src => shark/src}/test/hive/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 (100%) rename {src => shark/src}/test/hive/golden/merge4-12-62541540a18d68a3cb8497a741061d11 (100%) rename {src => shark/src}/test/hive/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 (100%) rename {src => shark/src}/test/hive/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 (100%) rename {src => shark/src}/test/hive/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a (100%) rename {src => shark/src}/test/hive/golden/merge4-2-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/merge4-4-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/merge4-5-3d24d877366c42030f6d9a596665720d (100%) rename {src => shark/src}/test/hive/golden/merge4-6-b3a76420183795720ab3a384046e5af (100%) rename {src => shark/src}/test/hive/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d (100%) rename {src => shark/src}/test/hive/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 (100%) rename {src => shark/src}/test/hive/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c (100%) rename {src => shark/src}/test/hive/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 (100%) rename {src => shark/src}/test/hive/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 (100%) rename {src => shark/src}/test/hive/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b (100%) rename {src => shark/src}/test/hive/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c (100%) rename {src => shark/src}/test/hive/golden/mergejoins-3-c408f69470d652da283442a62b384e46 (100%) rename {src => shark/src}/test/hive/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 (100%) rename {src => shark/src}/test/hive/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e (100%) rename {src => shark/src}/test/hive/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 (100%) rename {src => shark/src}/test/hive/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 (100%) rename {src => shark/src}/test/hive/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 (100%) rename {src => shark/src}/test/hive/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/mi-1-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e (100%) rename {src => shark/src}/test/hive/golden/mi-3-b66a495f7bdf106a7886b72267b8659d (100%) rename {src => shark/src}/test/hive/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c (100%) rename {src => shark/src}/test/hive/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc (100%) rename {src => shark/src}/test/hive/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 (100%) rename {src => shark/src}/test/hive/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 (100%) rename {src => shark/src}/test/hive/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 (100%) rename {src => shark/src}/test/hive/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b (100%) rename {src => shark/src}/test/hive/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 (100%) rename {src => shark/src}/test/hive/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 (100%) rename {src => shark/src}/test/hive/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b (100%) rename {src => shark/src}/test/hive/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 (100%) rename {src => shark/src}/test/hive/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 (100%) rename {src => shark/src}/test/hive/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 (100%) rename {src => shark/src}/test/hive/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 (100%) rename {src => shark/src}/test/hive/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae (100%) rename {src => shark/src}/test/hive/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 (100%) rename {src => shark/src}/test/hive/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 (100%) rename {src => shark/src}/test/hive/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 (100%) rename {src => shark/src}/test/hive/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 (100%) rename {src => shark/src}/test/hive/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 (100%) rename {src => shark/src}/test/hive/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 (100%) rename {src => shark/src}/test/hive/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 (100%) rename {src => shark/src}/test/hive/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 (100%) rename {src => shark/src}/test/hive/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e (100%) rename {src => shark/src}/test/hive/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 (100%) rename {src => shark/src}/test/hive/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 (100%) rename {src => shark/src}/test/hive/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 (100%) rename {src => shark/src}/test/hive/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 (100%) rename {src => shark/src}/test/hive/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 (100%) rename {src => shark/src}/test/hive/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f (100%) rename {src => shark/src}/test/hive/golden/newline-0-43392a20a8d249a279d50d96578e6a1b (100%) rename {src => shark/src}/test/hive/golden/newline-1-a19a19272149c732977c37e043910505 (100%) rename {src => shark/src}/test/hive/golden/newline-2-4eb54a664e549614d56ca088c8867d (100%) rename {src => shark/src}/test/hive/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 (100%) rename {src => shark/src}/test/hive/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 (100%) rename {src => shark/src}/test/hive/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b (100%) rename {src => shark/src}/test/hive/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 (100%) rename {src => shark/src}/test/hive/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 (100%) rename {src => shark/src}/test/hive/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 (100%) rename {src => shark/src}/test/hive/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 (100%) rename {src => shark/src}/test/hive/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 (100%) rename {src => shark/src}/test/hive/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 (100%) rename {src => shark/src}/test/hive/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 (100%) rename {src => shark/src}/test/hive/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad (100%) rename {src => shark/src}/test/hive/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 (100%) rename {src => shark/src}/test/hive/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d (100%) rename {src => shark/src}/test/hive/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd (100%) rename {src => shark/src}/test/hive/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e (100%) rename {src => shark/src}/test/hive/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d (100%) rename {src => shark/src}/test/hive/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 (100%) rename {src => shark/src}/test/hive/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 (100%) rename {src => shark/src}/test/hive/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 (100%) rename {src => shark/src}/test/hive/golden/null_cast-1-7257e6f8170e545962d27741353f672c (100%) rename {src => shark/src}/test/hive/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 (100%) rename {src => shark/src}/test/hive/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 (100%) rename {src => shark/src}/test/hive/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 (100%) rename {src => shark/src}/test/hive/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 (100%) rename {src => shark/src}/test/hive/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b (100%) rename {src => shark/src}/test/hive/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 (100%) rename {src => shark/src}/test/hive/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c (100%) rename {src => shark/src}/test/hive/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e (100%) rename {src => shark/src}/test/hive/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c (100%) rename {src => shark/src}/test/hive/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 (100%) rename {src => shark/src}/test/hive/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 (100%) rename {src => shark/src}/test/hive/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf (100%) rename {src => shark/src}/test/hive/golden/nullinput2-0-40b73200f853b473855e85eba391f008 (100%) rename {src => shark/src}/test/hive/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 (100%) rename {src => shark/src}/test/hive/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 (100%) rename {src => shark/src}/test/hive/golden/nullscript-0-4477b1566208a6805bb20630755a9375 (100%) rename {src => shark/src}/test/hive/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 (100%) rename {src => shark/src}/test/hive/golden/nullscript-2-17238164053203d56d30704e2c098e80 (100%) rename {src => shark/src}/test/hive/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 (100%) rename {src => shark/src}/test/hive/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 (100%) rename {src => shark/src}/test/hive/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc (100%) rename {src => shark/src}/test/hive/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 (100%) rename {src => shark/src}/test/hive/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a (100%) rename {src => shark/src}/test/hive/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 (100%) rename {src => shark/src}/test/hive/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f (100%) rename {src => shark/src}/test/hive/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a (100%) rename {src => shark/src}/test/hive/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 (100%) rename {src => shark/src}/test/hive/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e (100%) rename {src => shark/src}/test/hive/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 (100%) rename {src => shark/src}/test/hive/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 (100%) rename {src => shark/src}/test/hive/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d (100%) rename {src => shark/src}/test/hive/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 (100%) rename {src => shark/src}/test/hive/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 (100%) rename {src => shark/src}/test/hive/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 (100%) rename {src => shark/src}/test/hive/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 (100%) rename {src => shark/src}/test/hive/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 (100%) rename {src => shark/src}/test/hive/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d (100%) rename {src => shark/src}/test/hive/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb (100%) rename {src => shark/src}/test/hive/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 (100%) rename {src => shark/src}/test/hive/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 (100%) rename {src => shark/src}/test/hive/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 (100%) rename {src => shark/src}/test/hive/golden/orc_create-20-176d469a0edba57404416535c7d48023 (100%) rename {src => shark/src}/test/hive/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 (100%) rename {src => shark/src}/test/hive/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 (100%) rename {src => shark/src}/test/hive/golden/orc_create-23-be779533ea8967231e644209114c8350 (100%) rename {src => shark/src}/test/hive/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 (100%) rename {src => shark/src}/test/hive/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead (100%) rename {src => shark/src}/test/hive/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c (100%) rename {src => shark/src}/test/hive/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 (100%) rename {src => shark/src}/test/hive/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 (100%) rename {src => shark/src}/test/hive/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 (100%) rename {src => shark/src}/test/hive/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae (100%) rename {src => shark/src}/test/hive/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d (100%) rename {src => shark/src}/test/hive/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 (100%) rename {src => shark/src}/test/hive/golden/orc_create-32-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 (100%) rename {src => shark/src}/test/hive/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 (100%) rename {src => shark/src}/test/hive/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 (100%) rename {src => shark/src}/test/hive/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a (100%) rename {src => shark/src}/test/hive/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 (100%) rename {src => shark/src}/test/hive/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 (100%) rename {src => shark/src}/test/hive/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 (100%) rename {src => shark/src}/test/hive/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 (100%) rename {src => shark/src}/test/hive/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae (100%) rename {src => shark/src}/test/hive/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 (100%) rename {src => shark/src}/test/hive/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 (100%) rename {src => shark/src}/test/hive/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 (100%) rename {src => shark/src}/test/hive/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 (100%) rename {src => shark/src}/test/hive/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 (100%) rename {src => shark/src}/test/hive/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 (100%) rename {src => shark/src}/test/hive/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 (100%) rename {src => shark/src}/test/hive/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 (100%) rename {src => shark/src}/test/hive/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c (100%) rename {src => shark/src}/test/hive/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d (100%) rename {src => shark/src}/test/hive/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 (100%) rename {src => shark/src}/test/hive/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 (100%) rename {src => shark/src}/test/hive/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 (100%) rename {src => shark/src}/test/hive/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 (100%) rename {src => shark/src}/test/hive/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 (100%) rename {src => shark/src}/test/hive/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 (100%) rename {src => shark/src}/test/hive/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 (100%) rename {src => shark/src}/test/hive/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 (100%) rename {src => shark/src}/test/hive/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 (100%) rename {src => shark/src}/test/hive/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e (100%) rename {src => shark/src}/test/hive/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 (100%) rename {src => shark/src}/test/hive/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c (100%) rename {src => shark/src}/test/hive/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 (100%) rename {src => shark/src}/test/hive/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 (100%) rename {src => shark/src}/test/hive/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 (100%) rename {src => shark/src}/test/hive/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf (100%) rename {src => shark/src}/test/hive/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 (100%) rename {src => shark/src}/test/hive/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 (100%) rename {src => shark/src}/test/hive/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e (100%) rename {src => shark/src}/test/hive/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 (100%) rename {src => shark/src}/test/hive/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e (100%) rename {src => shark/src}/test/hive/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 (100%) rename {src => shark/src}/test/hive/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 (100%) rename {src => shark/src}/test/hive/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc (100%) rename {src => shark/src}/test/hive/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f (100%) rename {src => shark/src}/test/hive/golden/order-0-5c2fda577771db3b316e0d2bd02d048a (100%) rename {src => shark/src}/test/hive/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 (100%) rename {src => shark/src}/test/hive/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf (100%) rename {src => shark/src}/test/hive/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 (100%) rename {src => shark/src}/test/hive/golden/order2-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf (100%) rename {src => shark/src}/test/hive/golden/order2-2-4162aa366dc0836eed15cc819226907f (100%) rename {src => shark/src}/test/hive/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c (100%) rename {src => shark/src}/test/hive/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 (100%) rename {src => shark/src}/test/hive/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba (100%) rename {src => shark/src}/test/hive/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 (100%) rename {src => shark/src}/test/hive/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 (100%) rename {src => shark/src}/test/hive/golden/parallel-1-851e262128626126ae1ad87869db7c54 (100%) rename {src => shark/src}/test/hive/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 (100%) rename {src => shark/src}/test/hive/golden/parallel-11-6230286bc168af7b010968b543690a2a (100%) rename {src => shark/src}/test/hive/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 (100%) rename {src => shark/src}/test/hive/golden/parallel-2-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a (100%) rename {src => shark/src}/test/hive/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 (100%) rename {src => shark/src}/test/hive/golden/parallel-5-e27b511ffc5828b09069121031d17a2f (100%) rename {src => shark/src}/test/hive/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 (100%) rename {src => shark/src}/test/hive/golden/parallel-7-6230286bc168af7b010968b543690a2a (100%) rename {src => shark/src}/test/hive/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 (100%) rename {src => shark/src}/test/hive/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c (100%) rename {src => shark/src}/test/hive/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 (100%) rename {src => shark/src}/test/hive/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 (100%) rename {src => shark/src}/test/hive/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc (100%) rename {src => shark/src}/test/hive/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd (100%) rename {src => shark/src}/test/hive/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec (100%) rename {src => shark/src}/test/hive/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 (100%) rename {src => shark/src}/test/hive/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 (100%) rename {src => shark/src}/test/hive/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea (100%) rename {src => shark/src}/test/hive/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 (100%) rename {src => shark/src}/test/hive/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b (100%) rename {src => shark/src}/test/hive/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 (100%) rename {src => shark/src}/test/hive/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 (100%) rename {src => shark/src}/test/hive/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 (100%) rename {src => shark/src}/test/hive/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 (100%) rename {src => shark/src}/test/hive/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a (100%) rename {src => shark/src}/test/hive/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 (100%) rename {src => shark/src}/test/hive/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 (100%) rename {src => shark/src}/test/hive/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 (100%) rename {src => shark/src}/test/hive/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c (100%) rename {src => shark/src}/test/hive/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 (100%) rename {src => shark/src}/test/hive/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 (100%) rename {src => shark/src}/test/hive/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb (100%) rename {src => shark/src}/test/hive/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 (100%) rename {src => shark/src}/test/hive/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 (100%) rename {src => shark/src}/test/hive/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d (100%) rename {src => shark/src}/test/hive/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 (100%) rename {src => shark/src}/test/hive/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf (100%) rename {src => shark/src}/test/hive/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 (100%) rename {src => shark/src}/test/hive/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 (100%) rename {src => shark/src}/test/hive/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 (100%) rename {src => shark/src}/test/hive/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 (100%) rename {src => shark/src}/test/hive/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 (100%) rename {src => shark/src}/test/hive/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 (100%) rename {src => shark/src}/test/hive/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 (100%) rename {src => shark/src}/test/hive/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 (100%) rename {src => shark/src}/test/hive/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d (100%) rename {src => shark/src}/test/hive/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b (100%) rename {src => shark/src}/test/hive/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 (100%) rename {src => shark/src}/test/hive/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 (100%) rename {src => shark/src}/test/hive/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 (100%) rename {src => shark/src}/test/hive/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf (100%) rename {src => shark/src}/test/hive/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 (100%) rename {src => shark/src}/test/hive/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b (100%) rename {src => shark/src}/test/hive/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd (100%) rename {src => shark/src}/test/hive/golden/plan_json-0-74146da55d57b22443140e7fbab3375c (100%) rename {src => shark/src}/test/hive/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 (100%) rename {src => shark/src}/test/hive/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 (100%) rename {src => shark/src}/test/hive/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 (100%) rename {src => shark/src}/test/hive/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 (100%) rename {src => shark/src}/test/hive/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 (100%) rename {src => shark/src}/test/hive/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b (100%) rename {src => shark/src}/test/hive/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 (100%) rename {src => shark/src}/test/hive/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 (100%) rename {src => shark/src}/test/hive/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 (100%) rename {src => shark/src}/test/hive/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 (100%) rename {src => shark/src}/test/hive/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 (100%) rename {src => shark/src}/test/hive/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e (100%) rename {src => shark/src}/test/hive/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d (100%) rename {src => shark/src}/test/hive/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e (100%) rename {src => shark/src}/test/hive/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d (100%) rename {src => shark/src}/test/hive/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f (100%) rename {src => shark/src}/test/hive/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f (100%) rename {src => shark/src}/test/hive/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 (100%) rename {src => shark/src}/test/hive/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 (100%) rename {src => shark/src}/test/hive/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 (100%) rename {src => shark/src}/test/hive/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 (100%) rename {src => shark/src}/test/hive/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 (100%) rename {src => shark/src}/test/hive/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d (100%) rename {src => shark/src}/test/hive/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d (100%) rename {src => shark/src}/test/hive/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 (100%) rename {src => shark/src}/test/hive/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b (100%) rename {src => shark/src}/test/hive/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 (100%) rename {src => shark/src}/test/hive/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 (100%) rename {src => shark/src}/test/hive/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c (100%) rename {src => shark/src}/test/hive/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 (100%) rename {src => shark/src}/test/hive/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b (100%) rename {src => shark/src}/test/hive/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b (100%) rename {src => shark/src}/test/hive/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 (100%) rename {src => shark/src}/test/hive/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 (100%) rename {src => shark/src}/test/hive/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 (100%) rename {src => shark/src}/test/hive/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 (100%) rename {src => shark/src}/test/hive/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f (100%) rename {src => shark/src}/test/hive/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 (100%) rename {src => shark/src}/test/hive/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 (100%) rename {src => shark/src}/test/hive/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf (100%) rename {src => shark/src}/test/hive/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 (100%) rename {src => shark/src}/test/hive/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 (100%) rename {src => shark/src}/test/hive/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 (100%) rename {src => shark/src}/test/hive/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 (100%) rename {src => shark/src}/test/hive/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 (100%) rename {src => shark/src}/test/hive/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed (100%) rename {src => shark/src}/test/hive/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae (100%) rename {src => shark/src}/test/hive/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 (100%) rename {src => shark/src}/test/hive/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 (100%) rename {src => shark/src}/test/hive/golden/ppd_union-3-678c8197f458b459171c266f7431683e (100%) rename {src => shark/src}/test/hive/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 (100%) rename {src => shark/src}/test/hive/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 (100%) rename {src => shark/src}/test/hive/golden/ppd_union-6-678c8197f458b459171c266f7431683e (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 (100%) rename {src => shark/src}/test/hive/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c (100%) rename {src => shark/src}/test/hive/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 (100%) rename {src => shark/src}/test/hive/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 (100%) rename {src => shark/src}/test/hive/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 (100%) rename {src => shark/src}/test/hive/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 (100%) rename {src => shark/src}/test/hive/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d (100%) rename {src => shark/src}/test/hive/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 (100%) rename {src => shark/src}/test/hive/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 (100%) rename {src => shark/src}/test/hive/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d (100%) rename {src => shark/src}/test/hive/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 (100%) rename {src => shark/src}/test/hive/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f (100%) rename {src => shark/src}/test/hive/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 (100%) rename {src => shark/src}/test/hive/golden/print_header-1-8540676fc16ac91f3629c40f393a890a (100%) rename {src => shark/src}/test/hive/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 (100%) rename {src => shark/src}/test/hive/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb (100%) rename {src => shark/src}/test/hive/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 (100%) rename {src => shark/src}/test/hive/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 (100%) rename {src => shark/src}/test/hive/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f (100%) rename {src => shark/src}/test/hive/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 (100%) rename {src => shark/src}/test/hive/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 (100%) rename {src => shark/src}/test/hive/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 (100%) rename {src => shark/src}/test/hive/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 (100%) rename {src => shark/src}/test/hive/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 (100%) rename {src => shark/src}/test/hive/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a (100%) rename {src => shark/src}/test/hive/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c (100%) rename {src => shark/src}/test/hive/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b (100%) rename {src => shark/src}/test/hive/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 (100%) rename {src => shark/src}/test/hive/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b (100%) rename {src => shark/src}/test/hive/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 (100%) rename {src => shark/src}/test/hive/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 (100%) rename {src => shark/src}/test/hive/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 (100%) rename {src => shark/src}/test/hive/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea (100%) rename {src => shark/src}/test/hive/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b (100%) rename {src => shark/src}/test/hive/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d (100%) rename {src => shark/src}/test/hive/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 (100%) rename {src => shark/src}/test/hive/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d (100%) rename {src => shark/src}/test/hive/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 (100%) rename {src => shark/src}/test/hive/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 (100%) rename {src => shark/src}/test/hive/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d (100%) rename {src => shark/src}/test/hive/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 (100%) rename {src => shark/src}/test/hive/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d (100%) rename {src => shark/src}/test/hive/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 (100%) rename {src => shark/src}/test/hive/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc (100%) rename {src => shark/src}/test/hive/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 (100%) rename {src => shark/src}/test/hive/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 (100%) rename {src => shark/src}/test/hive/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 (100%) rename {src => shark/src}/test/hive/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b (100%) rename {src => shark/src}/test/hive/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 (100%) rename {src => shark/src}/test/hive/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 (100%) rename {src => shark/src}/test/hive/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 (100%) rename {src => shark/src}/test/hive/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e (100%) rename {src => shark/src}/test/hive/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 (100%) rename {src => shark/src}/test/hive/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 (100%) rename {src => shark/src}/test/hive/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 (100%) rename {src => shark/src}/test/hive/golden/protectmode-40-39024332d020339bd9396c9b15403a0c (100%) rename {src => shark/src}/test/hive/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 (100%) rename {src => shark/src}/test/hive/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb (100%) rename {src => shark/src}/test/hive/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 (100%) rename {src => shark/src}/test/hive/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c (100%) rename {src => shark/src}/test/hive/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 (100%) rename {src => shark/src}/test/hive/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 (100%) rename {src => shark/src}/test/hive/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 (100%) rename {src => shark/src}/test/hive/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc (100%) rename {src => shark/src}/test/hive/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 (100%) rename {src => shark/src}/test/hive/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e (100%) rename {src => shark/src}/test/hive/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 (100%) rename {src => shark/src}/test/hive/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 (100%) rename {src => shark/src}/test/hive/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 (100%) rename {src => shark/src}/test/hive/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a (100%) rename {src => shark/src}/test/hive/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 (100%) rename {src => shark/src}/test/hive/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb (100%) rename {src => shark/src}/test/hive/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 (100%) rename {src => shark/src}/test/hive/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 (100%) rename {src => shark/src}/test/hive/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 (100%) rename {src => shark/src}/test/hive/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 (100%) rename {src => shark/src}/test/hive/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 (100%) rename {src => shark/src}/test/hive/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 (100%) rename {src => shark/src}/test/hive/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 (100%) rename {src => shark/src}/test/hive/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 (100%) rename {src => shark/src}/test/hive/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e (100%) rename {src => shark/src}/test/hive/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d (100%) rename {src => shark/src}/test/hive/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 (100%) rename {src => shark/src}/test/hive/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 (100%) rename {src => shark/src}/test/hive/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 (100%) rename {src => shark/src}/test/hive/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 (100%) rename {src => shark/src}/test/hive/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 (100%) rename {src => shark/src}/test/hive/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e (100%) rename {src => shark/src}/test/hive/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 (100%) rename {src => shark/src}/test/hive/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 (100%) rename {src => shark/src}/test/hive/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a (100%) rename {src => shark/src}/test/hive/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 (100%) rename {src => shark/src}/test/hive/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 (100%) rename {src => shark/src}/test/hive/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b (100%) rename {src => shark/src}/test/hive/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 (100%) rename {src => shark/src}/test/hive/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 (100%) rename {src => shark/src}/test/hive/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 (100%) rename {src => shark/src}/test/hive/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 (100%) rename {src => shark/src}/test/hive/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d (100%) rename {src => shark/src}/test/hive/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee (100%) rename {src => shark/src}/test/hive/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a (100%) rename {src => shark/src}/test/hive/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 (100%) rename {src => shark/src}/test/hive/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed (100%) rename {src => shark/src}/test/hive/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 (100%) rename {src => shark/src}/test/hive/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 (100%) rename {src => shark/src}/test/hive/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b (100%) rename {src => shark/src}/test/hive/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 (100%) rename {src => shark/src}/test/hive/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d (100%) rename {src => shark/src}/test/hive/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 (100%) rename {src => shark/src}/test/hive/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a (100%) rename {src => shark/src}/test/hive/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 (100%) rename {src => shark/src}/test/hive/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 (100%) rename {src => shark/src}/test/hive/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b (100%) rename {src => shark/src}/test/hive/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 (100%) rename {src => shark/src}/test/hive/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 (100%) rename {src => shark/src}/test/hive/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 (100%) rename {src => shark/src}/test/hive/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb (100%) rename {src => shark/src}/test/hive/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 (100%) rename {src => shark/src}/test/hive/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 (100%) rename {src => shark/src}/test/hive/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 (100%) rename {src => shark/src}/test/hive/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db (100%) rename {src => shark/src}/test/hive/golden/regex_col-2-21564f64cdfd46098e1254380490701 (100%) rename {src => shark/src}/test/hive/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 (100%) rename {src => shark/src}/test/hive/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 (100%) rename {src => shark/src}/test/hive/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 (100%) rename {src => shark/src}/test/hive/golden/remote_script-1-e168f471980470d93b790702a70238fa (100%) rename {src => shark/src}/test/hive/golden/remote_script-2-a19a19272149c732977c37e043910505 (100%) rename {src => shark/src}/test/hive/golden/remote_script-3-4eb54a664e549614d56ca088c8867d (100%) rename {src => shark/src}/test/hive/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 (100%) rename {src => shark/src}/test/hive/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 (100%) rename {src => shark/src}/test/hive/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 (100%) rename {src => shark/src}/test/hive/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 (100%) rename {src => shark/src}/test/hive/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-16-d032f4795c1186255acea241387adf93 (100%) rename {src => shark/src}/test/hive/golden/rename_column-17-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 (100%) rename {src => shark/src}/test/hive/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f (100%) rename {src => shark/src}/test/hive/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a (100%) rename {src => shark/src}/test/hive/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 (100%) rename {src => shark/src}/test/hive/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a (100%) rename {src => shark/src}/test/hive/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 (100%) rename {src => shark/src}/test/hive/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd (100%) rename {src => shark/src}/test/hive/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 (100%) rename {src => shark/src}/test/hive/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 (100%) rename {src => shark/src}/test/hive/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 (100%) rename {src => shark/src}/test/hive/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 (100%) rename {src => shark/src}/test/hive/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-36-d032f4795c1186255acea241387adf93 (100%) rename {src => shark/src}/test/hive/golden/rename_column-37-9c36cac1372650b703400c60dd29042c (100%) rename {src => shark/src}/test/hive/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 (100%) rename {src => shark/src}/test/hive/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd (100%) rename {src => shark/src}/test/hive/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 (100%) rename {src => shark/src}/test/hive/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b (100%) rename {src => shark/src}/test/hive/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 (100%) rename {src => shark/src}/test/hive/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 (100%) rename {src => shark/src}/test/hive/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 (100%) rename {src => shark/src}/test/hive/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc (100%) rename {src => shark/src}/test/hive/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 (100%) rename {src => shark/src}/test/hive/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 (100%) rename {src => shark/src}/test/hive/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f (100%) rename {src => shark/src}/test/hive/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 (100%) rename {src => shark/src}/test/hive/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e (100%) rename {src => shark/src}/test/hive/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 (100%) rename {src => shark/src}/test/hive/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 (100%) rename {src => shark/src}/test/hive/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef (100%) rename {src => shark/src}/test/hive/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 (100%) rename {src => shark/src}/test/hive/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 (100%) rename {src => shark/src}/test/hive/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 (100%) rename {src => shark/src}/test/hive/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 (100%) rename {src => shark/src}/test/hive/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 (100%) rename {src => shark/src}/test/hive/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f (100%) rename {src => shark/src}/test/hive/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 (100%) rename {src => shark/src}/test/hive/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 (100%) rename {src => shark/src}/test/hive/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a (100%) rename {src => shark/src}/test/hive/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e (100%) rename {src => shark/src}/test/hive/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a (100%) rename {src => shark/src}/test/hive/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e (100%) rename {src => shark/src}/test/hive/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde (100%) rename {src => shark/src}/test/hive/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e (100%) rename {src => shark/src}/test/hive/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 (100%) rename {src => shark/src}/test/hive/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f (100%) rename {src => shark/src}/test/hive/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 (100%) rename {src => shark/src}/test/hive/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 (100%) rename {src => shark/src}/test/hive/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 (100%) rename {src => shark/src}/test/hive/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d (100%) rename {src => shark/src}/test/hive/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d (100%) rename {src => shark/src}/test/hive/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 (100%) rename {src => shark/src}/test/hive/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 (100%) rename {src => shark/src}/test/hive/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d (100%) rename {src => shark/src}/test/hive/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 (100%) rename {src => shark/src}/test/hive/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 (100%) rename {src => shark/src}/test/hive/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 (100%) rename {src => shark/src}/test/hive/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 (100%) rename {src => shark/src}/test/hive/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c (100%) rename {src => shark/src}/test/hive/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 (100%) rename {src => shark/src}/test/hive/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 (100%) rename {src => shark/src}/test/hive/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c (100%) rename {src => shark/src}/test/hive/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e (100%) rename {src => shark/src}/test/hive/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 (100%) rename {src => shark/src}/test/hive/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a (100%) rename {src => shark/src}/test/hive/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 (100%) rename {src => shark/src}/test/hive/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca (100%) rename {src => shark/src}/test/hive/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 (100%) rename {src => shark/src}/test/hive/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 (100%) rename {src => shark/src}/test/hive/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce (100%) rename {src => shark/src}/test/hive/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 (100%) rename {src => shark/src}/test/hive/golden/show_tables-11-3f9a7f993510123059493826470f78f7 (100%) rename {src => shark/src}/test/hive/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c (100%) rename {src => shark/src}/test/hive/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 (100%) rename {src => shark/src}/test/hive/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c (100%) rename {src => shark/src}/test/hive/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe (100%) rename {src => shark/src}/test/hive/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 (100%) rename {src => shark/src}/test/hive/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab (100%) rename {src => shark/src}/test/hive/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 (100%) rename {src => shark/src}/test/hive/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 (100%) rename {src => shark/src}/test/hive/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 (100%) rename {src => shark/src}/test/hive/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 (100%) rename {src => shark/src}/test/hive/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 (100%) rename {src => shark/src}/test/hive/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 (100%) rename {src => shark/src}/test/hive/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c (100%) rename {src => shark/src}/test/hive/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 (100%) rename {src => shark/src}/test/hive/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d (100%) rename {src => shark/src}/test/hive/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 (100%) rename {src => shark/src}/test/hive/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c (100%) rename {src => shark/src}/test/hive/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 (100%) rename {src => shark/src}/test/hive/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 (100%) rename {src => shark/src}/test/hive/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa (100%) rename {src => shark/src}/test/hive/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 (100%) rename {src => shark/src}/test/hive/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 (100%) rename {src => shark/src}/test/hive/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d (100%) rename {src => shark/src}/test/hive/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 (100%) rename {src => shark/src}/test/hive/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 (100%) rename {src => shark/src}/test/hive/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 (100%) rename {src => shark/src}/test/hive/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c (100%) rename {src => shark/src}/test/hive/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 (100%) rename {src => shark/src}/test/hive/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 (100%) rename {src => shark/src}/test/hive/golden/showparts-0-593619bb962b318b82896658deaea1f1 (100%) rename {src => shark/src}/test/hive/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 (100%) rename {src => shark/src}/test/hive/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d (100%) rename {src => shark/src}/test/hive/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 (100%) rename {src => shark/src}/test/hive/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a (100%) rename {src => shark/src}/test/hive/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 (100%) rename {src => shark/src}/test/hive/golden/sort-0-eee35bbc888a705ae527625447668032 (100%) rename {src => shark/src}/test/hive/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf (100%) rename {src => shark/src}/test/hive/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec (100%) rename {src => shark/src}/test/hive/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 (100%) rename {src => shark/src}/test/hive/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c (100%) rename {src => shark/src}/test/hive/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 (100%) rename {src => shark/src}/test/hive/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 (100%) rename {src => shark/src}/test/hive/golden/stats0-13-ca0569fd4200d549843deadf84044649 (100%) rename {src => shark/src}/test/hive/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 (100%) rename {src => shark/src}/test/hive/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c (100%) rename {src => shark/src}/test/hive/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 (100%) rename {src => shark/src}/test/hive/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed (100%) rename {src => shark/src}/test/hive/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/stats0-20-5f7439f97275da70f633e135205f2095 (100%) rename {src => shark/src}/test/hive/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a (100%) rename {src => shark/src}/test/hive/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 (100%) rename {src => shark/src}/test/hive/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c (100%) rename {src => shark/src}/test/hive/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 (100%) rename {src => shark/src}/test/hive/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b (100%) rename {src => shark/src}/test/hive/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c (100%) rename {src => shark/src}/test/hive/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 (100%) rename {src => shark/src}/test/hive/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 (100%) rename {src => shark/src}/test/hive/golden/stats0-29-ca0569fd4200d549843deadf84044649 (100%) rename {src => shark/src}/test/hive/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 (100%) rename {src => shark/src}/test/hive/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c (100%) rename {src => shark/src}/test/hive/golden/stats0-4-5f7439f97275da70f633e135205f2095 (100%) rename {src => shark/src}/test/hive/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a (100%) rename {src => shark/src}/test/hive/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 (100%) rename {src => shark/src}/test/hive/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c (100%) rename {src => shark/src}/test/hive/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 (100%) rename {src => shark/src}/test/hive/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b (100%) rename {src => shark/src}/test/hive/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e (100%) rename {src => shark/src}/test/hive/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b (100%) rename {src => shark/src}/test/hive/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/stats2-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 (100%) rename {src => shark/src}/test/hive/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 (100%) rename {src => shark/src}/test/hive/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 (100%) rename {src => shark/src}/test/hive/golden/stats2-8-72621dba638b15d244850018e9f64d7 (100%) rename {src => shark/src}/test/hive/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e (100%) rename {src => shark/src}/test/hive/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 (100%) rename {src => shark/src}/test/hive/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 (100%) rename {src => shark/src}/test/hive/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b (100%) rename {src => shark/src}/test/hive/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 (100%) rename {src => shark/src}/test/hive/golden/stats20-5-76509775cfe11bb98ee088188a07668a (100%) rename {src => shark/src}/test/hive/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 (100%) rename {src => shark/src}/test/hive/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 (100%) rename {src => shark/src}/test/hive/golden/stats20-8-300c971de74642118d36d36349bc81aa (100%) rename {src => shark/src}/test/hive/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 (100%) rename {src => shark/src}/test/hive/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 (100%) rename {src => shark/src}/test/hive/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa (100%) rename {src => shark/src}/test/hive/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc (100%) rename {src => shark/src}/test/hive/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 (100%) rename {src => shark/src}/test/hive/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa (100%) rename {src => shark/src}/test/hive/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 (100%) rename {src => shark/src}/test/hive/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 (100%) rename {src => shark/src}/test/hive/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 (100%) rename {src => shark/src}/test/hive/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b (100%) rename {src => shark/src}/test/hive/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 (100%) rename {src => shark/src}/test/hive/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 (100%) rename {src => shark/src}/test/hive/golden/stats3-20-ca048ad81b3df7159822073d206f0790 (100%) rename {src => shark/src}/test/hive/golden/stats3-3-ca048ad81b3df7159822073d206f0790 (100%) rename {src => shark/src}/test/hive/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 (100%) rename {src => shark/src}/test/hive/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 (100%) rename {src => shark/src}/test/hive/golden/stats3-6-4bf1504274319c44d370b58092fe016c (100%) rename {src => shark/src}/test/hive/golden/stats3-7-73d7d55d6e5a57aacce8618902904d (100%) rename {src => shark/src}/test/hive/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 (100%) rename {src => shark/src}/test/hive/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 (100%) rename {src => shark/src}/test/hive/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 (100%) rename {src => shark/src}/test/hive/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 (100%) rename {src => shark/src}/test/hive/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 (100%) rename {src => shark/src}/test/hive/golden/stats4-12-30bc31441828a053d1a675b225a5d617 (100%) rename {src => shark/src}/test/hive/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 (100%) rename {src => shark/src}/test/hive/golden/stats4-14-9c82167763a771c175c656786d545798 (100%) rename {src => shark/src}/test/hive/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c (100%) rename {src => shark/src}/test/hive/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 (100%) rename {src => shark/src}/test/hive/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 (100%) rename {src => shark/src}/test/hive/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 (100%) rename {src => shark/src}/test/hive/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 (100%) rename {src => shark/src}/test/hive/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 (100%) rename {src => shark/src}/test/hive/golden/stats4-20-f63000f2c395b935199c9829964f98c1 (100%) rename {src => shark/src}/test/hive/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd (100%) rename {src => shark/src}/test/hive/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c (100%) rename {src => shark/src}/test/hive/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd (100%) rename {src => shark/src}/test/hive/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c (100%) rename {src => shark/src}/test/hive/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d (100%) rename {src => shark/src}/test/hive/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 (100%) rename {src => shark/src}/test/hive/golden/stats4-7-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f (100%) rename {src => shark/src}/test/hive/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 (100%) rename {src => shark/src}/test/hive/golden/stats5-3-96d9aa9c32a081518604959dcfac42df (100%) rename {src => shark/src}/test/hive/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 (100%) rename {src => shark/src}/test/hive/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 (100%) rename {src => shark/src}/test/hive/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/stats6-10-653f748fa2d690502ba4fda407841a20 (100%) rename {src => shark/src}/test/hive/golden/stats6-11-1c9f833953403596ad50fd32e513642c (100%) rename {src => shark/src}/test/hive/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 (100%) rename {src => shark/src}/test/hive/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/stats6-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/stats6-4-a88c476a632cd92f748967fadb242405 (100%) rename {src => shark/src}/test/hive/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc (100%) rename {src => shark/src}/test/hive/golden/stats6-6-8926095434b70c83bf88c70559d38dce (100%) rename {src => shark/src}/test/hive/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 (100%) rename {src => shark/src}/test/hive/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 (100%) rename {src => shark/src}/test/hive/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d (100%) rename {src => shark/src}/test/hive/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 (100%) rename {src => shark/src}/test/hive/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/stats7-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/stats7-4-a88c476a632cd92f748967fadb242405 (100%) rename {src => shark/src}/test/hive/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc (100%) rename {src => shark/src}/test/hive/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 (100%) rename {src => shark/src}/test/hive/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 (100%) rename {src => shark/src}/test/hive/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 (100%) rename {src => shark/src}/test/hive/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d (100%) rename {src => shark/src}/test/hive/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 (100%) rename {src => shark/src}/test/hive/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 (100%) rename {src => shark/src}/test/hive/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d (100%) rename {src => shark/src}/test/hive/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d (100%) rename {src => shark/src}/test/hive/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 (100%) rename {src => shark/src}/test/hive/golden/stats8-15-653f748fa2d690502ba4fda407841a20 (100%) rename {src => shark/src}/test/hive/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 (100%) rename {src => shark/src}/test/hive/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 (100%) rename {src => shark/src}/test/hive/golden/stats8-18-1c9f833953403596ad50fd32e513642c (100%) rename {src => shark/src}/test/hive/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 (100%) rename {src => shark/src}/test/hive/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 (100%) rename {src => shark/src}/test/hive/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 (100%) rename {src => shark/src}/test/hive/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d (100%) rename {src => shark/src}/test/hive/golden/stats8-23-653f748fa2d690502ba4fda407841a20 (100%) rename {src => shark/src}/test/hive/golden/stats8-24-1c9f833953403596ad50fd32e513642c (100%) rename {src => shark/src}/test/hive/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 (100%) rename {src => shark/src}/test/hive/golden/stats8-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/stats8-4-a88c476a632cd92f748967fadb242405 (100%) rename {src => shark/src}/test/hive/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc (100%) rename {src => shark/src}/test/hive/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 (100%) rename {src => shark/src}/test/hive/golden/stats8-7-8926095434b70c83bf88c70559d38dce (100%) rename {src => shark/src}/test/hive/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 (100%) rename {src => shark/src}/test/hive/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 (100%) rename {src => shark/src}/test/hive/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 (100%) rename {src => shark/src}/test/hive/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c (100%) rename {src => shark/src}/test/hive/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 (100%) rename {src => shark/src}/test/hive/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 (100%) rename {src => shark/src}/test/hive/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 (100%) rename {src => shark/src}/test/hive/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 (100%) rename {src => shark/src}/test/hive/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 (100%) rename {src => shark/src}/test/hive/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 (100%) rename {src => shark/src}/test/hive/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 (100%) rename {src => shark/src}/test/hive/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 (100%) rename {src => shark/src}/test/hive/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 (100%) rename {src => shark/src}/test/hive/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae (100%) rename {src => shark/src}/test/hive/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca (100%) rename {src => shark/src}/test/hive/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b (100%) rename {src => shark/src}/test/hive/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c (100%) rename {src => shark/src}/test/hive/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 (100%) rename {src => shark/src}/test/hive/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c (100%) rename {src => shark/src}/test/hive/golden/str_to_map-10-32997010bba305ec40812df254490730 (100%) rename {src => shark/src}/test/hive/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a (100%) rename {src => shark/src}/test/hive/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e (100%) rename {src => shark/src}/test/hive/golden/str_to_map-13-32997010bba305ec40812df254490730 (100%) rename {src => shark/src}/test/hive/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec (100%) rename {src => shark/src}/test/hive/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 (100%) rename {src => shark/src}/test/hive/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 (100%) rename {src => shark/src}/test/hive/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda (100%) rename {src => shark/src}/test/hive/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf (100%) rename {src => shark/src}/test/hive/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad (100%) rename {src => shark/src}/test/hive/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 (100%) rename {src => shark/src}/test/hive/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd (100%) rename {src => shark/src}/test/hive/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 (100%) rename {src => shark/src}/test/hive/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 (100%) rename {src => shark/src}/test/hive/golden/subq2-1-235919a7ddb574662158503b8052e7ca (100%) rename {src => shark/src}/test/hive/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 (100%) rename {src => shark/src}/test/hive/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e (100%) rename {src => shark/src}/test/hive/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 (100%) rename {src => shark/src}/test/hive/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 (100%) rename {src => shark/src}/test/hive/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d (100%) rename {src => shark/src}/test/hive/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 (100%) rename {src => shark/src}/test/hive/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 (100%) rename {src => shark/src}/test/hive/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c (100%) rename {src => shark/src}/test/hive/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 (100%) rename {src => shark/src}/test/hive/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 (100%) rename {src => shark/src}/test/hive/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 (100%) rename {src => shark/src}/test/hive/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 (100%) rename {src => shark/src}/test/hive/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 (100%) rename {src => shark/src}/test/hive/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 (100%) rename {src => shark/src}/test/hive/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 (100%) rename {src => shark/src}/test/hive/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 (100%) rename {src => shark/src}/test/hive/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b (100%) rename {src => shark/src}/test/hive/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 (100%) rename {src => shark/src}/test/hive/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 (100%) rename {src => shark/src}/test/hive/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc (100%) rename {src => shark/src}/test/hive/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 (100%) rename {src => shark/src}/test/hive/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb (100%) rename {src => shark/src}/test/hive/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 (100%) rename {src => shark/src}/test/hive/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af (100%) rename {src => shark/src}/test/hive/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d (100%) rename {src => shark/src}/test/hive/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 (100%) rename {src => shark/src}/test/hive/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f (100%) rename {src => shark/src}/test/hive/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 (100%) rename {src => shark/src}/test/hive/golden/touch-1-27ad2962fed131f51ba802596ba37278 (100%) rename {src => shark/src}/test/hive/golden/touch-10-27ad2962fed131f51ba802596ba37278 (100%) rename {src => shark/src}/test/hive/golden/touch-2-3e95421993ab28d18245ec2340f580a3 (100%) rename {src => shark/src}/test/hive/golden/touch-3-c0c18ac884677231a41eea8d980d0451 (100%) rename {src => shark/src}/test/hive/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a (100%) rename {src => shark/src}/test/hive/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c (100%) rename {src => shark/src}/test/hive/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 (100%) rename {src => shark/src}/test/hive/golden/touch-7-3752320b12abae0d138148d56a27c5b1 (100%) rename {src => shark/src}/test/hive/golden/touch-8-3562ab40242756452595cd7eae79b0ce (100%) rename {src => shark/src}/test/hive/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 (100%) rename {src => shark/src}/test/hive/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe (100%) rename {src => shark/src}/test/hive/golden/transform1-0-b6919fc48901e388c869c84ae0211102 (100%) rename {src => shark/src}/test/hive/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 (100%) rename {src => shark/src}/test/hive/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 (100%) rename {src => shark/src}/test/hive/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 (100%) rename {src => shark/src}/test/hive/golden/transform1-4-65527bae8e73262255ef83082c6968f9 (100%) rename {src => shark/src}/test/hive/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 (100%) rename {src => shark/src}/test/hive/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f (100%) rename {src => shark/src}/test/hive/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 (100%) rename {src => shark/src}/test/hive/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 (100%) rename {src => shark/src}/test/hive/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 (100%) rename {src => shark/src}/test/hive/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 (100%) rename {src => shark/src}/test/hive/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 (100%) rename {src => shark/src}/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d (100%) rename {src => shark/src}/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 (100%) rename {src => shark/src}/test/hive/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 (100%) rename {src => shark/src}/test/hive/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 (100%) rename {src => shark/src}/test/hive/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 (100%) rename {src => shark/src}/test/hive/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 (100%) rename {src => shark/src}/test/hive/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 (100%) rename {src => shark/src}/test/hive/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a (100%) rename {src => shark/src}/test/hive/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 (100%) rename {src => shark/src}/test/hive/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 (100%) rename {src => shark/src}/test/hive/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 (100%) rename {src => shark/src}/test/hive/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 (100%) rename {src => shark/src}/test/hive/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 (100%) rename {src => shark/src}/test/hive/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 (100%) rename {src => shark/src}/test/hive/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f (100%) rename {src => shark/src}/test/hive/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 (100%) rename {src => shark/src}/test/hive/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 (100%) rename {src => shark/src}/test/hive/golden/udf1-1-8281592c818ada269024ac669bec78da (100%) rename {src => shark/src}/test/hive/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f (100%) rename {src => shark/src}/test/hive/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 (100%) rename {src => shark/src}/test/hive/golden/udf2-2-31c8af064bac42541558a95fad3bca97 (100%) rename {src => shark/src}/test/hive/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 (100%) rename {src => shark/src}/test/hive/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c (100%) rename {src => shark/src}/test/hive/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 (100%) rename {src => shark/src}/test/hive/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e (100%) rename {src => shark/src}/test/hive/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 (100%) rename {src => shark/src}/test/hive/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 (100%) rename {src => shark/src}/test/hive/golden/udf6-3-e579646b969eef49b09656114da52a73 (100%) rename {src => shark/src}/test/hive/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 (100%) rename {src => shark/src}/test/hive/golden/udf6-5-fe336cd9850d6357980bd19139f76e (100%) rename {src => shark/src}/test/hive/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 (100%) rename {src => shark/src}/test/hive/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 (100%) rename {src => shark/src}/test/hive/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 (100%) rename {src => shark/src}/test/hive/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/udf8-1-63c38297946a2060c0cff4a426b0520c (100%) rename {src => shark/src}/test/hive/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 (100%) rename {src => shark/src}/test/hive/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb (100%) rename {src => shark/src}/test/hive/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b (100%) rename {src => shark/src}/test/hive/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 (100%) rename {src => shark/src}/test/hive/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c (100%) rename {src => shark/src}/test/hive/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e (100%) rename {src => shark/src}/test/hive/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 (100%) rename {src => shark/src}/test/hive/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f (100%) rename {src => shark/src}/test/hive/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c (100%) rename {src => shark/src}/test/hive/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a (100%) rename {src => shark/src}/test/hive/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 (100%) rename {src => shark/src}/test/hive/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f (100%) rename {src => shark/src}/test/hive/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c (100%) rename {src => shark/src}/test/hive/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a (100%) rename {src => shark/src}/test/hive/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 (100%) rename {src => shark/src}/test/hive/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f (100%) rename {src => shark/src}/test/hive/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 (100%) rename {src => shark/src}/test/hive/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 (100%) rename {src => shark/src}/test/hive/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 (100%) rename {src => shark/src}/test/hive/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f (100%) rename {src => shark/src}/test/hive/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 (100%) rename {src => shark/src}/test/hive/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 (100%) rename {src => shark/src}/test/hive/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 (100%) rename {src => shark/src}/test/hive/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 (100%) rename {src => shark/src}/test/hive/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 (100%) rename {src => shark/src}/test/hive/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 (100%) rename {src => shark/src}/test/hive/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 (100%) rename {src => shark/src}/test/hive/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c (100%) rename {src => shark/src}/test/hive/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 (100%) rename {src => shark/src}/test/hive/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 (100%) rename {src => shark/src}/test/hive/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f (100%) rename {src => shark/src}/test/hive/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b (100%) rename {src => shark/src}/test/hive/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 (100%) rename {src => shark/src}/test/hive/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a (100%) rename {src => shark/src}/test/hive/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e (100%) rename {src => shark/src}/test/hive/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 (100%) rename {src => shark/src}/test/hive/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 (100%) rename {src => shark/src}/test/hive/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee (100%) rename {src => shark/src}/test/hive/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a (100%) rename {src => shark/src}/test/hive/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 (100%) rename {src => shark/src}/test/hive/golden/udf_array-3-a5d12c41277fb158e09281169c905122 (100%) rename {src => shark/src}/test/hive/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a (100%) rename {src => shark/src}/test/hive/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c (100%) rename {src => shark/src}/test/hive/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 (100%) rename {src => shark/src}/test/hive/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 (100%) rename {src => shark/src}/test/hive/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c (100%) rename {src => shark/src}/test/hive/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 (100%) rename {src => shark/src}/test/hive/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa (100%) rename {src => shark/src}/test/hive/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 (100%) rename {src => shark/src}/test/hive/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd (100%) rename {src => shark/src}/test/hive/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 (100%) rename {src => shark/src}/test/hive/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f (100%) rename {src => shark/src}/test/hive/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 (100%) rename {src => shark/src}/test/hive/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 (100%) rename {src => shark/src}/test/hive/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 (100%) rename {src => shark/src}/test/hive/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b (100%) rename {src => shark/src}/test/hive/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 (100%) rename {src => shark/src}/test/hive/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 (100%) rename {src => shark/src}/test/hive/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 (100%) rename {src => shark/src}/test/hive/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b (100%) rename {src => shark/src}/test/hive/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 (100%) rename {src => shark/src}/test/hive/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 (100%) rename {src => shark/src}/test/hive/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 (100%) rename {src => shark/src}/test/hive/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 (100%) rename {src => shark/src}/test/hive/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f (100%) rename {src => shark/src}/test/hive/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 (100%) rename {src => shark/src}/test/hive/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 (100%) rename {src => shark/src}/test/hive/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b (100%) rename {src => shark/src}/test/hive/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 (100%) rename {src => shark/src}/test/hive/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b (100%) rename {src => shark/src}/test/hive/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 (100%) rename {src => shark/src}/test/hive/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b (100%) rename {src => shark/src}/test/hive/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a (100%) rename {src => shark/src}/test/hive/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b (100%) rename {src => shark/src}/test/hive/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f (100%) rename {src => shark/src}/test/hive/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d (100%) rename {src => shark/src}/test/hive/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 (100%) rename {src => shark/src}/test/hive/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb (100%) rename {src => shark/src}/test/hive/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 (100%) rename {src => shark/src}/test/hive/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f (100%) rename {src => shark/src}/test/hive/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 (100%) rename {src => shark/src}/test/hive/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e (100%) rename {src => shark/src}/test/hive/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 (100%) rename {src => shark/src}/test/hive/golden/udf_case-1-54acf006155d8822a50e47729be24004 (100%) rename {src => shark/src}/test/hive/golden/udf_case-2-98ee676f92950375917f09d2e492253f (100%) rename {src => shark/src}/test/hive/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c (100%) rename {src => shark/src}/test/hive/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 (100%) rename {src => shark/src}/test/hive/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c (100%) rename {src => shark/src}/test/hive/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 (100%) rename {src => shark/src}/test/hive/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 (100%) rename {src => shark/src}/test/hive/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 (100%) rename {src => shark/src}/test/hive/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb (100%) rename {src => shark/src}/test/hive/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd (100%) rename {src => shark/src}/test/hive/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df (100%) rename {src => shark/src}/test/hive/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 (100%) rename {src => shark/src}/test/hive/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 (100%) rename {src => shark/src}/test/hive/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 (100%) rename {src => shark/src}/test/hive/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 (100%) rename {src => shark/src}/test/hive/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 (100%) rename {src => shark/src}/test/hive/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 (100%) rename {src => shark/src}/test/hive/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 (100%) rename {src => shark/src}/test/hive/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 (100%) rename {src => shark/src}/test/hive/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 (100%) rename {src => shark/src}/test/hive/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 (100%) rename {src => shark/src}/test/hive/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 (100%) rename {src => shark/src}/test/hive/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 (100%) rename {src => shark/src}/test/hive/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea (100%) rename {src => shark/src}/test/hive/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 (100%) rename {src => shark/src}/test/hive/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 (100%) rename {src => shark/src}/test/hive/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab (100%) rename {src => shark/src}/test/hive/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c (100%) rename {src => shark/src}/test/hive/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 (100%) rename {src => shark/src}/test/hive/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f (100%) rename {src => shark/src}/test/hive/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 (100%) rename {src => shark/src}/test/hive/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 (100%) rename {src => shark/src}/test/hive/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c (100%) rename {src => shark/src}/test/hive/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 (100%) rename {src => shark/src}/test/hive/golden/udf_conv-0-d552befca345f3396464529cfde9f75a (100%) rename {src => shark/src}/test/hive/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab (100%) rename {src => shark/src}/test/hive/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 (100%) rename {src => shark/src}/test/hive/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 (100%) rename {src => shark/src}/test/hive/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d (100%) rename {src => shark/src}/test/hive/golden/udf_conv-4-568e843076f358c404a8634b18541c55 (100%) rename {src => shark/src}/test/hive/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 (100%) rename {src => shark/src}/test/hive/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a (100%) rename {src => shark/src}/test/hive/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e (100%) rename {src => shark/src}/test/hive/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb (100%) rename {src => shark/src}/test/hive/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 (100%) rename {src => shark/src}/test/hive/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 (100%) rename {src => shark/src}/test/hive/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 (100%) rename {src => shark/src}/test/hive/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 (100%) rename {src => shark/src}/test/hive/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 (100%) rename {src => shark/src}/test/hive/golden/udf_count-0-534a9b25b87d09e418645b1216949560 (100%) rename {src => shark/src}/test/hive/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f (100%) rename {src => shark/src}/test/hive/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 (100%) rename {src => shark/src}/test/hive/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e (100%) rename {src => shark/src}/test/hive/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e (100%) rename {src => shark/src}/test/hive/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 (100%) rename {src => shark/src}/test/hive/golden/udf_count-2-3344645abe535426307b9327b381fc85 (100%) rename {src => shark/src}/test/hive/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 (100%) rename {src => shark/src}/test/hive/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 (100%) rename {src => shark/src}/test/hive/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 (100%) rename {src => shark/src}/test/hive/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 (100%) rename {src => shark/src}/test/hive/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 (100%) rename {src => shark/src}/test/hive/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 (100%) rename {src => shark/src}/test/hive/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 (100%) rename {src => shark/src}/test/hive/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 (100%) rename {src => shark/src}/test/hive/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 (100%) rename {src => shark/src}/test/hive/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 (100%) rename {src => shark/src}/test/hive/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 (100%) rename {src => shark/src}/test/hive/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 (100%) rename {src => shark/src}/test/hive/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 (100%) rename {src => shark/src}/test/hive/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 (100%) rename {src => shark/src}/test/hive/golden/udf_day-1-87168babe1110fe4c38269843414ca4 (100%) rename {src => shark/src}/test/hive/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 (100%) rename {src => shark/src}/test/hive/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 (100%) rename {src => shark/src}/test/hive/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae (100%) rename {src => shark/src}/test/hive/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 (100%) rename {src => shark/src}/test/hive/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b (100%) rename {src => shark/src}/test/hive/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 (100%) rename {src => shark/src}/test/hive/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae (100%) rename {src => shark/src}/test/hive/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 (100%) rename {src => shark/src}/test/hive/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b (100%) rename {src => shark/src}/test/hive/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 (100%) rename {src => shark/src}/test/hive/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db (100%) rename {src => shark/src}/test/hive/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 (100%) rename {src => shark/src}/test/hive/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b (100%) rename {src => shark/src}/test/hive/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 (100%) rename {src => shark/src}/test/hive/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 (100%) rename {src => shark/src}/test/hive/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 (100%) rename {src => shark/src}/test/hive/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 (100%) rename {src => shark/src}/test/hive/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 (100%) rename {src => shark/src}/test/hive/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f (100%) rename {src => shark/src}/test/hive/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 (100%) rename {src => shark/src}/test/hive/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a (100%) rename {src => shark/src}/test/hive/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 (100%) rename {src => shark/src}/test/hive/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a (100%) rename {src => shark/src}/test/hive/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 (100%) rename {src => shark/src}/test/hive/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 (100%) rename {src => shark/src}/test/hive/golden/udf_field-1-379d8580693493f620a8f4084709324f (100%) rename {src => shark/src}/test/hive/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 (100%) rename {src => shark/src}/test/hive/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b (100%) rename {src => shark/src}/test/hive/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 (100%) rename {src => shark/src}/test/hive/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 (100%) rename {src => shark/src}/test/hive/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 (100%) rename {src => shark/src}/test/hive/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 (100%) rename {src => shark/src}/test/hive/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef (100%) rename {src => shark/src}/test/hive/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 (100%) rename {src => shark/src}/test/hive/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 (100%) rename {src => shark/src}/test/hive/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 (100%) rename {src => shark/src}/test/hive/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 (100%) rename {src => shark/src}/test/hive/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b (100%) rename {src => shark/src}/test/hive/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 (100%) rename {src => shark/src}/test/hive/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 (100%) rename {src => shark/src}/test/hive/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb (100%) rename {src => shark/src}/test/hive/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 (100%) rename {src => shark/src}/test/hive/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd (100%) rename {src => shark/src}/test/hive/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c (100%) rename {src => shark/src}/test/hive/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 (100%) rename {src => shark/src}/test/hive/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 (100%) rename {src => shark/src}/test/hive/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 (100%) rename {src => shark/src}/test/hive/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 (100%) rename {src => shark/src}/test/hive/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e (100%) rename {src => shark/src}/test/hive/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 (100%) rename {src => shark/src}/test/hive/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f (100%) rename {src => shark/src}/test/hive/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 (100%) rename {src => shark/src}/test/hive/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f (100%) rename {src => shark/src}/test/hive/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 (100%) rename {src => shark/src}/test/hive/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 (100%) rename {src => shark/src}/test/hive/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 (100%) rename {src => shark/src}/test/hive/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 (100%) rename {src => shark/src}/test/hive/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 (100%) rename {src => shark/src}/test/hive/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c (100%) rename {src => shark/src}/test/hive/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f (100%) rename {src => shark/src}/test/hive/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a (100%) rename {src => shark/src}/test/hive/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 (100%) rename {src => shark/src}/test/hive/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 (100%) rename {src => shark/src}/test/hive/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 (100%) rename {src => shark/src}/test/hive/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a (100%) rename {src => shark/src}/test/hive/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 (100%) rename {src => shark/src}/test/hive/golden/udf_hash-3-23991312391d518aacf3d4469c816eae (100%) rename {src => shark/src}/test/hive/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 (100%) rename {src => shark/src}/test/hive/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c (100%) rename {src => shark/src}/test/hive/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 (100%) rename {src => shark/src}/test/hive/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf (100%) rename {src => shark/src}/test/hive/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea (100%) rename {src => shark/src}/test/hive/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce (100%) rename {src => shark/src}/test/hive/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 (100%) rename {src => shark/src}/test/hive/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a (100%) rename {src => shark/src}/test/hive/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 (100%) rename {src => shark/src}/test/hive/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c (100%) rename {src => shark/src}/test/hive/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a (100%) rename {src => shark/src}/test/hive/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 (100%) rename {src => shark/src}/test/hive/golden/udf_if-3-20206f17367ff284d67044abd745ce9f (100%) rename {src => shark/src}/test/hive/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca (100%) rename {src => shark/src}/test/hive/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc (100%) rename {src => shark/src}/test/hive/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 (100%) rename {src => shark/src}/test/hive/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 (100%) rename {src => shark/src}/test/hive/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 (100%) rename {src => shark/src}/test/hive/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 (100%) rename {src => shark/src}/test/hive/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 (100%) rename {src => shark/src}/test/hive/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 (100%) rename {src => shark/src}/test/hive/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 (100%) rename {src => shark/src}/test/hive/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 (100%) rename {src => shark/src}/test/hive/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 (100%) rename {src => shark/src}/test/hive/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 (100%) rename {src => shark/src}/test/hive/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba (100%) rename {src => shark/src}/test/hive/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 (100%) rename {src => shark/src}/test/hive/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 (100%) rename {src => shark/src}/test/hive/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 (100%) rename {src => shark/src}/test/hive/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 (100%) rename {src => shark/src}/test/hive/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 (100%) rename {src => shark/src}/test/hive/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 (100%) rename {src => shark/src}/test/hive/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 (100%) rename {src => shark/src}/test/hive/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 (100%) rename {src => shark/src}/test/hive/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 (100%) rename {src => shark/src}/test/hive/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 (100%) rename {src => shark/src}/test/hive/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 (100%) rename {src => shark/src}/test/hive/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 (100%) rename {src => shark/src}/test/hive/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b (100%) rename {src => shark/src}/test/hive/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 (100%) rename {src => shark/src}/test/hive/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a (100%) rename {src => shark/src}/test/hive/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 (100%) rename {src => shark/src}/test/hive/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd (100%) rename {src => shark/src}/test/hive/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad (100%) rename {src => shark/src}/test/hive/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e (100%) rename {src => shark/src}/test/hive/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b (100%) rename {src => shark/src}/test/hive/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 (100%) rename {src => shark/src}/test/hive/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 (100%) rename {src => shark/src}/test/hive/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 (100%) rename {src => shark/src}/test/hive/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d (100%) rename {src => shark/src}/test/hive/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 (100%) rename {src => shark/src}/test/hive/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f (100%) rename {src => shark/src}/test/hive/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a (100%) rename {src => shark/src}/test/hive/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f (100%) rename {src => shark/src}/test/hive/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf (100%) rename {src => shark/src}/test/hive/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 (100%) rename {src => shark/src}/test/hive/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 (100%) rename {src => shark/src}/test/hive/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 (100%) rename {src => shark/src}/test/hive/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 (100%) rename {src => shark/src}/test/hive/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b (100%) rename {src => shark/src}/test/hive/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 (100%) rename {src => shark/src}/test/hive/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c (100%) rename {src => shark/src}/test/hive/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 (100%) rename {src => shark/src}/test/hive/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a (100%) rename {src => shark/src}/test/hive/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 (100%) rename {src => shark/src}/test/hive/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc (100%) rename {src => shark/src}/test/hive/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 (100%) rename {src => shark/src}/test/hive/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 (100%) rename {src => shark/src}/test/hive/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 (100%) rename {src => shark/src}/test/hive/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e (100%) rename {src => shark/src}/test/hive/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 (100%) rename {src => shark/src}/test/hive/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c (100%) rename {src => shark/src}/test/hive/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 (100%) rename {src => shark/src}/test/hive/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a (100%) rename {src => shark/src}/test/hive/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 (100%) rename {src => shark/src}/test/hive/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f (100%) rename {src => shark/src}/test/hive/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 (100%) rename {src => shark/src}/test/hive/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 (100%) rename {src => shark/src}/test/hive/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 (100%) rename {src => shark/src}/test/hive/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 (100%) rename {src => shark/src}/test/hive/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 (100%) rename {src => shark/src}/test/hive/golden/udf_lower-1-550f0a6da388596a775d921b9da995c (100%) rename {src => shark/src}/test/hive/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e (100%) rename {src => shark/src}/test/hive/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 (100%) rename {src => shark/src}/test/hive/golden/udf_lpad-0-995646acf1e23cea7825412915921bef (100%) rename {src => shark/src}/test/hive/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 (100%) rename {src => shark/src}/test/hive/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 (100%) rename {src => shark/src}/test/hive/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae (100%) rename {src => shark/src}/test/hive/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f (100%) rename {src => shark/src}/test/hive/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 (100%) rename {src => shark/src}/test/hive/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e (100%) rename {src => shark/src}/test/hive/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf (100%) rename {src => shark/src}/test/hive/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 (100%) rename {src => shark/src}/test/hive/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 (100%) rename {src => shark/src}/test/hive/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb (100%) rename {src => shark/src}/test/hive/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa (100%) rename {src => shark/src}/test/hive/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f (100%) rename {src => shark/src}/test/hive/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e (100%) rename {src => shark/src}/test/hive/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a (100%) rename {src => shark/src}/test/hive/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb (100%) rename {src => shark/src}/test/hive/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 (100%) rename {src => shark/src}/test/hive/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 (100%) rename {src => shark/src}/test/hive/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 (100%) rename {src => shark/src}/test/hive/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e (100%) rename {src => shark/src}/test/hive/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f (100%) rename {src => shark/src}/test/hive/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 (100%) rename {src => shark/src}/test/hive/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 (100%) rename {src => shark/src}/test/hive/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 (100%) rename {src => shark/src}/test/hive/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 (100%) rename {src => shark/src}/test/hive/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 (100%) rename {src => shark/src}/test/hive/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e (100%) rename {src => shark/src}/test/hive/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 (100%) rename {src => shark/src}/test/hive/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e (100%) rename {src => shark/src}/test/hive/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 (100%) rename {src => shark/src}/test/hive/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 (100%) rename {src => shark/src}/test/hive/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 (100%) rename {src => shark/src}/test/hive/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 (100%) rename {src => shark/src}/test/hive/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 (100%) rename {src => shark/src}/test/hive/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 (100%) rename {src => shark/src}/test/hive/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 (100%) rename {src => shark/src}/test/hive/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 (100%) rename {src => shark/src}/test/hive/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 (100%) rename {src => shark/src}/test/hive/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e (100%) rename {src => shark/src}/test/hive/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 (100%) rename {src => shark/src}/test/hive/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e (100%) rename {src => shark/src}/test/hive/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 (100%) rename {src => shark/src}/test/hive/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 (100%) rename {src => shark/src}/test/hive/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 (100%) rename {src => shark/src}/test/hive/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee (100%) rename {src => shark/src}/test/hive/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 (100%) rename {src => shark/src}/test/hive/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 (100%) rename {src => shark/src}/test/hive/golden/udf_minute-3-270055c684846e87444b037226cf554c (100%) rename {src => shark/src}/test/hive/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 (100%) rename {src => shark/src}/test/hive/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 (100%) rename {src => shark/src}/test/hive/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee (100%) rename {src => shark/src}/test/hive/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 (100%) rename {src => shark/src}/test/hive/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 (100%) rename {src => shark/src}/test/hive/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf (100%) rename {src => shark/src}/test/hive/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c (100%) rename {src => shark/src}/test/hive/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 (100%) rename {src => shark/src}/test/hive/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c (100%) rename {src => shark/src}/test/hive/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e (100%) rename {src => shark/src}/test/hive/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 (100%) rename {src => shark/src}/test/hive/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b (100%) rename {src => shark/src}/test/hive/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 (100%) rename {src => shark/src}/test/hive/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa (100%) rename {src => shark/src}/test/hive/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc (100%) rename {src => shark/src}/test/hive/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 (100%) rename {src => shark/src}/test/hive/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 (100%) rename {src => shark/src}/test/hive/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a (100%) rename {src => shark/src}/test/hive/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e (100%) rename {src => shark/src}/test/hive/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a (100%) rename {src => shark/src}/test/hive/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 (100%) rename {src => shark/src}/test/hive/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 (100%) rename {src => shark/src}/test/hive/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 (100%) rename {src => shark/src}/test/hive/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 (100%) rename {src => shark/src}/test/hive/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 (100%) rename {src => shark/src}/test/hive/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd (100%) rename {src => shark/src}/test/hive/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 (100%) rename {src => shark/src}/test/hive/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 (100%) rename {src => shark/src}/test/hive/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 (100%) rename {src => shark/src}/test/hive/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def (100%) rename {src => shark/src}/test/hive/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 (100%) rename {src => shark/src}/test/hive/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 (100%) rename {src => shark/src}/test/hive/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 (100%) rename {src => shark/src}/test/hive/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 (100%) rename {src => shark/src}/test/hive/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab (100%) rename {src => shark/src}/test/hive/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d (100%) rename {src => shark/src}/test/hive/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 (100%) rename {src => shark/src}/test/hive/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 (100%) rename {src => shark/src}/test/hive/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a (100%) rename {src => shark/src}/test/hive/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 (100%) rename {src => shark/src}/test/hive/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd (100%) rename {src => shark/src}/test/hive/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 (100%) rename {src => shark/src}/test/hive/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 (100%) rename {src => shark/src}/test/hive/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c (100%) rename {src => shark/src}/test/hive/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 (100%) rename {src => shark/src}/test/hive/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 (100%) rename {src => shark/src}/test/hive/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 (100%) rename {src => shark/src}/test/hive/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 (100%) rename {src => shark/src}/test/hive/golden/udf_power-0-57001d802c281743322d28bbc520cd4 (100%) rename {src => shark/src}/test/hive/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 (100%) rename {src => shark/src}/test/hive/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 (100%) rename {src => shark/src}/test/hive/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 (100%) rename {src => shark/src}/test/hive/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 (100%) rename {src => shark/src}/test/hive/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 (100%) rename {src => shark/src}/test/hive/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 (100%) rename {src => shark/src}/test/hive/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 (100%) rename {src => shark/src}/test/hive/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 (100%) rename {src => shark/src}/test/hive/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 (100%) rename {src => shark/src}/test/hive/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 (100%) rename {src => shark/src}/test/hive/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 (100%) rename {src => shark/src}/test/hive/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 (100%) rename {src => shark/src}/test/hive/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e (100%) rename {src => shark/src}/test/hive/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 (100%) rename {src => shark/src}/test/hive/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee (100%) rename {src => shark/src}/test/hive/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 (100%) rename {src => shark/src}/test/hive/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 (100%) rename {src => shark/src}/test/hive/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 (100%) rename {src => shark/src}/test/hive/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca (100%) rename {src => shark/src}/test/hive/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 (100%) rename {src => shark/src}/test/hive/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 (100%) rename {src => shark/src}/test/hive/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 (100%) rename {src => shark/src}/test/hive/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 (100%) rename {src => shark/src}/test/hive/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb (100%) rename {src => shark/src}/test/hive/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e (100%) rename {src => shark/src}/test/hive/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc (100%) rename {src => shark/src}/test/hive/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b (100%) rename {src => shark/src}/test/hive/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b (100%) rename {src => shark/src}/test/hive/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a (100%) rename {src => shark/src}/test/hive/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 (100%) rename {src => shark/src}/test/hive/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 (100%) rename {src => shark/src}/test/hive/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca (100%) rename {src => shark/src}/test/hive/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 (100%) rename {src => shark/src}/test/hive/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 (100%) rename {src => shark/src}/test/hive/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 (100%) rename {src => shark/src}/test/hive/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c (100%) rename {src => shark/src}/test/hive/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 (100%) rename {src => shark/src}/test/hive/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a (100%) rename {src => shark/src}/test/hive/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b (100%) rename {src => shark/src}/test/hive/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 (100%) rename {src => shark/src}/test/hive/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce (100%) rename {src => shark/src}/test/hive/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 (100%) rename {src => shark/src}/test/hive/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 (100%) rename {src => shark/src}/test/hive/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 (100%) rename {src => shark/src}/test/hive/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b (100%) rename {src => shark/src}/test/hive/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c (100%) rename {src => shark/src}/test/hive/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 (100%) rename {src => shark/src}/test/hive/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 (100%) rename {src => shark/src}/test/hive/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 (100%) rename {src => shark/src}/test/hive/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 (100%) rename {src => shark/src}/test/hive/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c (100%) rename {src => shark/src}/test/hive/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 (100%) rename {src => shark/src}/test/hive/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 (100%) rename {src => shark/src}/test/hive/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 (100%) rename {src => shark/src}/test/hive/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a (100%) rename {src => shark/src}/test/hive/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd (100%) rename {src => shark/src}/test/hive/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 (100%) rename {src => shark/src}/test/hive/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 (100%) rename {src => shark/src}/test/hive/golden/udf_second-2-d678372e3837a16be245d2e33482f17f (100%) rename {src => shark/src}/test/hive/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 (100%) rename {src => shark/src}/test/hive/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 (100%) rename {src => shark/src}/test/hive/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e (100%) rename {src => shark/src}/test/hive/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 (100%) rename {src => shark/src}/test/hive/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa (100%) rename {src => shark/src}/test/hive/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a (100%) rename {src => shark/src}/test/hive/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef (100%) rename {src => shark/src}/test/hive/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d (100%) rename {src => shark/src}/test/hive/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa (100%) rename {src => shark/src}/test/hive/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a (100%) rename {src => shark/src}/test/hive/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e (100%) rename {src => shark/src}/test/hive/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 (100%) rename {src => shark/src}/test/hive/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef (100%) rename {src => shark/src}/test/hive/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d (100%) rename {src => shark/src}/test/hive/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee (100%) rename {src => shark/src}/test/hive/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 (100%) rename {src => shark/src}/test/hive/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab (100%) rename {src => shark/src}/test/hive/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 (100%) rename {src => shark/src}/test/hive/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 (100%) rename {src => shark/src}/test/hive/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 (100%) rename {src => shark/src}/test/hive/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 (100%) rename {src => shark/src}/test/hive/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd (100%) rename {src => shark/src}/test/hive/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 (100%) rename {src => shark/src}/test/hive/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 (100%) rename {src => shark/src}/test/hive/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb (100%) rename {src => shark/src}/test/hive/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 (100%) rename {src => shark/src}/test/hive/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 (100%) rename {src => shark/src}/test/hive/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de (100%) rename {src => shark/src}/test/hive/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 (100%) rename {src => shark/src}/test/hive/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e (100%) rename {src => shark/src}/test/hive/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 (100%) rename {src => shark/src}/test/hive/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 (100%) rename {src => shark/src}/test/hive/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 (100%) rename {src => shark/src}/test/hive/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a (100%) rename {src => shark/src}/test/hive/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 (100%) rename {src => shark/src}/test/hive/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 (100%) rename {src => shark/src}/test/hive/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 (100%) rename {src => shark/src}/test/hive/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 (100%) rename {src => shark/src}/test/hive/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 (100%) rename {src => shark/src}/test/hive/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 (100%) rename {src => shark/src}/test/hive/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 (100%) rename {src => shark/src}/test/hive/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 (100%) rename {src => shark/src}/test/hive/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 (100%) rename {src => shark/src}/test/hive/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 (100%) rename {src => shark/src}/test/hive/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 (100%) rename {src => shark/src}/test/hive/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b (100%) rename {src => shark/src}/test/hive/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 (100%) rename {src => shark/src}/test/hive/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 (100%) rename {src => shark/src}/test/hive/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d (100%) rename {src => shark/src}/test/hive/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 (100%) rename {src => shark/src}/test/hive/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 (100%) rename {src => shark/src}/test/hive/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 (100%) rename {src => shark/src}/test/hive/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 (100%) rename {src => shark/src}/test/hive/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 (100%) rename {src => shark/src}/test/hive/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 (100%) rename {src => shark/src}/test/hive/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 (100%) rename {src => shark/src}/test/hive/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d (100%) rename {src => shark/src}/test/hive/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd (100%) rename {src => shark/src}/test/hive/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 (100%) rename {src => shark/src}/test/hive/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 (100%) rename {src => shark/src}/test/hive/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 (100%) rename {src => shark/src}/test/hive/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a (100%) rename {src => shark/src}/test/hive/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 (100%) rename {src => shark/src}/test/hive/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 (100%) rename {src => shark/src}/test/hive/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 (100%) rename {src => shark/src}/test/hive/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece (100%) rename {src => shark/src}/test/hive/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 (100%) rename {src => shark/src}/test/hive/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 (100%) rename {src => shark/src}/test/hive/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 (100%) rename {src => shark/src}/test/hive/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 (100%) rename {src => shark/src}/test/hive/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b (100%) rename {src => shark/src}/test/hive/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 (100%) rename {src => shark/src}/test/hive/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b (100%) rename {src => shark/src}/test/hive/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 (100%) rename {src => shark/src}/test/hive/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 (100%) rename {src => shark/src}/test/hive/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 (100%) rename {src => shark/src}/test/hive/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 (100%) rename {src => shark/src}/test/hive/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 (100%) rename {src => shark/src}/test/hive/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 (100%) rename {src => shark/src}/test/hive/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 (100%) rename {src => shark/src}/test/hive/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 (100%) rename {src => shark/src}/test/hive/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad (100%) rename {src => shark/src}/test/hive/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 (100%) rename {src => shark/src}/test/hive/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d (100%) rename {src => shark/src}/test/hive/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 (100%) rename {src => shark/src}/test/hive/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 (100%) rename {src => shark/src}/test/hive/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 (100%) rename {src => shark/src}/test/hive/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 (100%) rename {src => shark/src}/test/hive/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c (100%) rename {src => shark/src}/test/hive/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d (100%) rename {src => shark/src}/test/hive/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad (100%) rename {src => shark/src}/test/hive/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f (100%) rename {src => shark/src}/test/hive/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a (100%) rename {src => shark/src}/test/hive/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 (100%) rename {src => shark/src}/test/hive/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 (100%) rename {src => shark/src}/test/hive/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc (100%) rename {src => shark/src}/test/hive/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b (100%) rename {src => shark/src}/test/hive/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 (100%) rename {src => shark/src}/test/hive/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 (100%) rename {src => shark/src}/test/hive/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 (100%) rename {src => shark/src}/test/hive/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 (100%) rename {src => shark/src}/test/hive/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 (100%) rename {src => shark/src}/test/hive/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 (100%) rename {src => shark/src}/test/hive/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e (100%) rename {src => shark/src}/test/hive/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 (100%) rename {src => shark/src}/test/hive/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a (100%) rename {src => shark/src}/test/hive/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe (100%) rename {src => shark/src}/test/hive/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f (100%) rename {src => shark/src}/test/hive/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 (100%) rename {src => shark/src}/test/hive/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 (100%) rename {src => shark/src}/test/hive/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 (100%) rename {src => shark/src}/test/hive/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 (100%) rename {src => shark/src}/test/hive/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 (100%) rename {src => shark/src}/test/hive/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 (100%) rename {src => shark/src}/test/hive/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c (100%) rename {src => shark/src}/test/hive/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 (100%) rename {src => shark/src}/test/hive/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce (100%) rename {src => shark/src}/test/hive/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 (100%) rename {src => shark/src}/test/hive/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 (100%) rename {src => shark/src}/test/hive/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 (100%) rename {src => shark/src}/test/hive/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 (100%) rename {src => shark/src}/test/hive/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c (100%) rename {src => shark/src}/test/hive/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a (100%) rename {src => shark/src}/test/hive/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 (100%) rename {src => shark/src}/test/hive/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 (100%) rename {src => shark/src}/test/hive/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 (100%) rename {src => shark/src}/test/hive/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 (100%) rename {src => shark/src}/test/hive/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d (100%) rename {src => shark/src}/test/hive/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 (100%) rename {src => shark/src}/test/hive/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e (100%) rename {src => shark/src}/test/hive/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 (100%) rename {src => shark/src}/test/hive/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a (100%) rename {src => shark/src}/test/hive/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b (100%) rename {src => shark/src}/test/hive/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 (100%) rename {src => shark/src}/test/hive/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce (100%) rename {src => shark/src}/test/hive/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 (100%) rename {src => shark/src}/test/hive/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 (100%) rename {src => shark/src}/test/hive/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 (100%) rename {src => shark/src}/test/hive/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d (100%) rename {src => shark/src}/test/hive/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 (100%) rename {src => shark/src}/test/hive/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc (100%) rename {src => shark/src}/test/hive/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 (100%) rename {src => shark/src}/test/hive/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 (100%) rename {src => shark/src}/test/hive/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 (100%) rename {src => shark/src}/test/hive/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 (100%) rename {src => shark/src}/test/hive/golden/udf_to_string-5-1040b37847d20ef29d545934316303 (100%) rename {src => shark/src}/test/hive/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 (100%) rename {src => shark/src}/test/hive/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 (100%) rename {src => shark/src}/test/hive/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea (100%) rename {src => shark/src}/test/hive/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b (100%) rename {src => shark/src}/test/hive/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 (100%) rename {src => shark/src}/test/hive/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 (100%) rename {src => shark/src}/test/hive/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 (100%) rename {src => shark/src}/test/hive/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b (100%) rename {src => shark/src}/test/hive/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 (100%) rename {src => shark/src}/test/hive/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 (100%) rename {src => shark/src}/test/hive/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a (100%) rename {src => shark/src}/test/hive/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba (100%) rename {src => shark/src}/test/hive/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 (100%) rename {src => shark/src}/test/hive/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b (100%) rename {src => shark/src}/test/hive/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 (100%) rename {src => shark/src}/test/hive/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 (100%) rename {src => shark/src}/test/hive/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 (100%) rename {src => shark/src}/test/hive/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f (100%) rename {src => shark/src}/test/hive/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 (100%) rename {src => shark/src}/test/hive/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d (100%) rename {src => shark/src}/test/hive/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f (100%) rename {src => shark/src}/test/hive/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 (100%) rename {src => shark/src}/test/hive/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 (100%) rename {src => shark/src}/test/hive/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 (100%) rename {src => shark/src}/test/hive/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 (100%) rename {src => shark/src}/test/hive/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c (100%) rename {src => shark/src}/test/hive/golden/udf_union-1-f6f01250718087029165e23badc02cd6 (100%) rename {src => shark/src}/test/hive/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 (100%) rename {src => shark/src}/test/hive/golden/udf_union-3-705d165fec6761744dd19b142c566d61 (100%) rename {src => shark/src}/test/hive/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead (100%) rename {src => shark/src}/test/hive/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d (100%) rename {src => shark/src}/test/hive/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de (100%) rename {src => shark/src}/test/hive/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 (100%) rename {src => shark/src}/test/hive/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 (100%) rename {src => shark/src}/test/hive/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b (100%) rename {src => shark/src}/test/hive/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 (100%) rename {src => shark/src}/test/hive/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b (100%) rename {src => shark/src}/test/hive/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 (100%) rename {src => shark/src}/test/hive/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a (100%) rename {src => shark/src}/test/hive/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de (100%) rename {src => shark/src}/test/hive/golden/udf_variance-3-fd25e5226312bf54d918858511814766 (100%) rename {src => shark/src}/test/hive/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 (100%) rename {src => shark/src}/test/hive/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a (100%) rename {src => shark/src}/test/hive/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de (100%) rename {src => shark/src}/test/hive/golden/udf_variance-7-fd25e5226312bf54d918858511814766 (100%) rename {src => shark/src}/test/hive/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 (100%) rename {src => shark/src}/test/hive/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd (100%) rename {src => shark/src}/test/hive/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 (100%) rename {src => shark/src}/test/hive/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 (100%) rename {src => shark/src}/test/hive/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 (100%) rename {src => shark/src}/test/hive/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 (100%) rename {src => shark/src}/test/hive/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f (100%) rename {src => shark/src}/test/hive/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d (100%) rename {src => shark/src}/test/hive/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea (100%) rename {src => shark/src}/test/hive/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a (100%) rename {src => shark/src}/test/hive/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc (100%) rename {src => shark/src}/test/hive/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 (100%) rename {src => shark/src}/test/hive/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a (100%) rename {src => shark/src}/test/hive/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 (100%) rename {src => shark/src}/test/hive/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 (100%) rename {src => shark/src}/test/hive/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 (100%) rename {src => shark/src}/test/hive/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 (100%) rename {src => shark/src}/test/hive/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 (100%) rename {src => shark/src}/test/hive/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e (100%) rename {src => shark/src}/test/hive/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 (100%) rename {src => shark/src}/test/hive/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 (100%) rename {src => shark/src}/test/hive/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 (100%) rename {src => shark/src}/test/hive/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 (100%) rename {src => shark/src}/test/hive/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 (100%) rename {src => shark/src}/test/hive/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af (100%) rename {src => shark/src}/test/hive/golden/union10-2-60ee9eae687170317ff91dafe6b799bf (100%) rename {src => shark/src}/test/hive/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 (100%) rename {src => shark/src}/test/hive/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 (100%) rename {src => shark/src}/test/hive/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 (100%) rename {src => shark/src}/test/hive/golden/union11-2-cd756f39e22e121bdbd51400662aa47f (100%) rename {src => shark/src}/test/hive/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 (100%) rename {src => shark/src}/test/hive/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 (100%) rename {src => shark/src}/test/hive/golden/union13-0-6970d1e2405f3769a28a8804887ac657 (100%) rename {src => shark/src}/test/hive/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 (100%) rename {src => shark/src}/test/hive/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/union14-1-11603fb6225423979af6c062cfa9389b (100%) rename {src => shark/src}/test/hive/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 (100%) rename {src => shark/src}/test/hive/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 (100%) rename {src => shark/src}/test/hive/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 (100%) rename {src => shark/src}/test/hive/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 (100%) rename {src => shark/src}/test/hive/golden/union16-1-9f76074598f9b55d8afbb5659737a382 (100%) rename {src => shark/src}/test/hive/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a (100%) rename {src => shark/src}/test/hive/golden/union17-1-9491c271b7e2c351ddcf126f73679892 (100%) rename {src => shark/src}/test/hive/golden/union17-2-e6fb104913b9da8193167ee263993dd1 (100%) rename {src => shark/src}/test/hive/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 (100%) rename {src => shark/src}/test/hive/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 (100%) rename {src => shark/src}/test/hive/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 (100%) rename {src => shark/src}/test/hive/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a (100%) rename {src => shark/src}/test/hive/golden/union18-1-9491c271b7e2c351ddcf126f73679892 (100%) rename {src => shark/src}/test/hive/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 (100%) rename {src => shark/src}/test/hive/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 (100%) rename {src => shark/src}/test/hive/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 (100%) rename {src => shark/src}/test/hive/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 (100%) rename {src => shark/src}/test/hive/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a (100%) rename {src => shark/src}/test/hive/golden/union19-1-9491c271b7e2c351ddcf126f73679892 (100%) rename {src => shark/src}/test/hive/golden/union19-2-b6161b64e57a2502c79662866782cbdf (100%) rename {src => shark/src}/test/hive/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 (100%) rename {src => shark/src}/test/hive/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 (100%) rename {src => shark/src}/test/hive/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 (100%) rename {src => shark/src}/test/hive/golden/union2-0-58813daf3d1af57b17518b606f8345b2 (100%) rename {src => shark/src}/test/hive/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 (100%) rename {src => shark/src}/test/hive/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d (100%) rename {src => shark/src}/test/hive/golden/union20-1-968e353589f1fddb914242beb25be94c (100%) rename {src => shark/src}/test/hive/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c (100%) rename {src => shark/src}/test/hive/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 (100%) rename {src => shark/src}/test/hive/golden/union22-0-4bd6583b3635f1db95765565518bda8d (100%) rename {src => shark/src}/test/hive/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 (100%) rename {src => shark/src}/test/hive/golden/union22-10-90e9c4388340428a1f68868e9322c400 (100%) rename {src => shark/src}/test/hive/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 (100%) rename {src => shark/src}/test/hive/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 (100%) rename {src => shark/src}/test/hive/golden/union22-4-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union22-5-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 (100%) rename {src => shark/src}/test/hive/golden/union22-7-25fc734982956a164adde6bb1d4d8751 (100%) rename {src => shark/src}/test/hive/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 (100%) rename {src => shark/src}/test/hive/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 (100%) rename {src => shark/src}/test/hive/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 (100%) rename {src => shark/src}/test/hive/golden/union23-1-7830963417e3535034962e2597970ddd (100%) rename {src => shark/src}/test/hive/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 (100%) rename {src => shark/src}/test/hive/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c (100%) rename {src => shark/src}/test/hive/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 (100%) rename {src => shark/src}/test/hive/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a (100%) rename {src => shark/src}/test/hive/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 (100%) rename {src => shark/src}/test/hive/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f (100%) rename {src => shark/src}/test/hive/golden/union24-4-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 (100%) rename {src => shark/src}/test/hive/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 (100%) rename {src => shark/src}/test/hive/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb (100%) rename {src => shark/src}/test/hive/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 (100%) rename {src => shark/src}/test/hive/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d (100%) rename {src => shark/src}/test/hive/golden/union26-1-4702087f160230463e6114d6534d47e4 (100%) rename {src => shark/src}/test/hive/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 (100%) rename {src => shark/src}/test/hive/golden/union26-3-4702087f160230463e6114d6534d47e4 (100%) rename {src => shark/src}/test/hive/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 (100%) rename {src => shark/src}/test/hive/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f (100%) rename {src => shark/src}/test/hive/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 (100%) rename {src => shark/src}/test/hive/golden/union27-3-ab84df3813ff23be99f148449610e530 (100%) rename {src => shark/src}/test/hive/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 (100%) rename {src => shark/src}/test/hive/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 (100%) rename {src => shark/src}/test/hive/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f (100%) rename {src => shark/src}/test/hive/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 (100%) rename {src => shark/src}/test/hive/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 (100%) rename {src => shark/src}/test/hive/golden/union29-1-a40036f756a306f4226634b6ab67c72c (100%) rename {src => shark/src}/test/hive/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 (100%) rename {src => shark/src}/test/hive/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 (100%) rename {src => shark/src}/test/hive/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d (100%) rename {src => shark/src}/test/hive/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 (100%) rename {src => shark/src}/test/hive/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 (100%) rename {src => shark/src}/test/hive/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 (100%) rename {src => shark/src}/test/hive/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 (100%) rename {src => shark/src}/test/hive/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 (100%) rename {src => shark/src}/test/hive/golden/union30-2-252f12f7532ca67132bfc62063c58430 (100%) rename {src => shark/src}/test/hive/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 (100%) rename {src => shark/src}/test/hive/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 (100%) rename {src => shark/src}/test/hive/golden/union31-1-b12aa87aaf64b573ce0be7013117651b (100%) rename {src => shark/src}/test/hive/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 (100%) rename {src => shark/src}/test/hive/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 (100%) rename {src => shark/src}/test/hive/golden/union31-12-de2477a134ab37bb22d879028becf2e5 (100%) rename {src => shark/src}/test/hive/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 (100%) rename {src => shark/src}/test/hive/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 (100%) rename {src => shark/src}/test/hive/golden/union31-15-5df6435aed6e0a6a6853480a027b911e (100%) rename {src => shark/src}/test/hive/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 (100%) rename {src => shark/src}/test/hive/golden/union31-17-b12aa87aaf64b573ce0be7013117651b (100%) rename {src => shark/src}/test/hive/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 (100%) rename {src => shark/src}/test/hive/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e (100%) rename {src => shark/src}/test/hive/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 (100%) rename {src => shark/src}/test/hive/golden/union31-20-dceb6d9a03096d68d090557692ed6899 (100%) rename {src => shark/src}/test/hive/golden/union31-21-bda2e62d0ecd30200b2359df420da574 (100%) rename {src => shark/src}/test/hive/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a (100%) rename {src => shark/src}/test/hive/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df (100%) rename {src => shark/src}/test/hive/golden/union31-24-df38c8164af7cc164c728b8178da72c5 (100%) rename {src => shark/src}/test/hive/golden/union31-25-1485e295a99908e1862eae397b814045 (100%) rename {src => shark/src}/test/hive/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc (100%) rename {src => shark/src}/test/hive/golden/union31-4-77653483e5ce18437c9f85d3b297335d (100%) rename {src => shark/src}/test/hive/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 (100%) rename {src => shark/src}/test/hive/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 (100%) rename {src => shark/src}/test/hive/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 (100%) rename {src => shark/src}/test/hive/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb (100%) rename {src => shark/src}/test/hive/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 (100%) rename {src => shark/src}/test/hive/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e (100%) rename {src => shark/src}/test/hive/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 (100%) rename {src => shark/src}/test/hive/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b (100%) rename {src => shark/src}/test/hive/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 (100%) rename {src => shark/src}/test/hive/golden/union32-2-8e39fef33c859ef83912d0fcda319218 (100%) rename {src => shark/src}/test/hive/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 (100%) rename {src => shark/src}/test/hive/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e (100%) rename {src => shark/src}/test/hive/golden/union32-5-51c997d0a1103b60764bbb8316a38746 (100%) rename {src => shark/src}/test/hive/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b (100%) rename {src => shark/src}/test/hive/golden/union32-7-f936440d63f4e1027dda4de605660633 (100%) rename {src => shark/src}/test/hive/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb (100%) rename {src => shark/src}/test/hive/golden/union32-9-74fb695786df4c024288ae23ac8c00db (100%) rename {src => shark/src}/test/hive/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f (100%) rename {src => shark/src}/test/hive/golden/union33-1-1df7c476abb48234c839878414720873 (100%) rename {src => shark/src}/test/hive/golden/union33-2-3987150ad66482f5186826efcfb117f7 (100%) rename {src => shark/src}/test/hive/golden/union33-3-bfe75625806b414a5098af487b91046a (100%) rename {src => shark/src}/test/hive/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 (100%) rename {src => shark/src}/test/hive/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 (100%) rename {src => shark/src}/test/hive/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 (100%) rename {src => shark/src}/test/hive/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 (100%) rename {src => shark/src}/test/hive/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 (100%) rename {src => shark/src}/test/hive/golden/union34-1-320c2fe802fab386957039aaff6399f1 (100%) rename {src => shark/src}/test/hive/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 (100%) rename {src => shark/src}/test/hive/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 (100%) rename {src => shark/src}/test/hive/golden/union34-3-c179e14d840249d6953cb552279822f1 (100%) rename {src => shark/src}/test/hive/golden/union34-4-101829a66cab2efd31dcb0d86e302956 (100%) rename {src => shark/src}/test/hive/golden/union34-5-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f (100%) rename {src => shark/src}/test/hive/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 (100%) rename {src => shark/src}/test/hive/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b (100%) rename {src => shark/src}/test/hive/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 (100%) rename {src => shark/src}/test/hive/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/union4-1-88deb3987655dc41989f39c0827cec74 (100%) rename {src => shark/src}/test/hive/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 (100%) rename {src => shark/src}/test/hive/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b (100%) rename {src => shark/src}/test/hive/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 (100%) rename {src => shark/src}/test/hive/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 (100%) rename {src => shark/src}/test/hive/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 (100%) rename {src => shark/src}/test/hive/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d (100%) rename {src => shark/src}/test/hive/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 (100%) rename {src => shark/src}/test/hive/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 (100%) rename {src => shark/src}/test/hive/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 (100%) rename {src => shark/src}/test/hive/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 (100%) rename {src => shark/src}/test/hive/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 (100%) rename {src => shark/src}/test/hive/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 (100%) rename {src => shark/src}/test/hive/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e (100%) rename {src => shark/src}/test/hive/golden/union8-1-1b422e4c1c8b97775518f760b995c771 (100%) rename {src => shark/src}/test/hive/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 (100%) rename {src => shark/src}/test/hive/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 (100%) rename {src => shark/src}/test/hive/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 (100%) rename {src => shark/src}/test/hive/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c (100%) rename {src => shark/src}/test/hive/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 (100%) rename {src => shark/src}/test/hive/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 (100%) rename {src => shark/src}/test/hive/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c (100%) rename {src => shark/src}/test/hive/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d (100%) rename {src => shark/src}/test/hive/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 (100%) rename {src => shark/src}/test/hive/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 (100%) rename {src => shark/src}/test/hive/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 (100%) rename {src => shark/src}/test/hive/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 (100%) rename {src => shark/src}/test/hive/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a (100%) rename {src => shark/src}/test/hive/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b (100%) rename {src => shark/src}/test/hive/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 (100%) rename {src => shark/src}/test/hive/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 (100%) rename {src => shark/src}/test/hive/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 (100%) rename {src => shark/src}/test/hive/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb (100%) rename {src => shark/src}/test/hive/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d (100%) rename {src => shark/src}/test/hive/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d (100%) rename {src => shark/src}/test/hive/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b (100%) rename {src => shark/src}/test/hive/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 (100%) rename {src => shark/src}/test/hive/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a (100%) rename {src => shark/src}/test/hive/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a (100%) rename {src => shark/src}/test/hive/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf (100%) rename {src => shark/src}/test/hive/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 (100%) rename {src => shark/src}/test/hive/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e (100%) rename {src => shark/src}/test/hive/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 (100%) rename {src => shark/src}/test/hive/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 (100%) rename {src => shark/src}/test/hive/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d (100%) rename {src => shark/src}/test/hive/golden/union_view-1-c790d4344144460224b0f02be7e137a8 (100%) rename {src => shark/src}/test/hive/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 (100%) rename {src => shark/src}/test/hive/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 (100%) rename {src => shark/src}/test/hive/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e (100%) rename {src => shark/src}/test/hive/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 (100%) rename {src => shark/src}/test/hive/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae (100%) rename {src => shark/src}/test/hive/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 (100%) rename {src => shark/src}/test/hive/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 (100%) rename {src => shark/src}/test/hive/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af (100%) rename {src => shark/src}/test/hive/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 (100%) rename {src => shark/src}/test/hive/golden/varchar_2-2-779a555944479991520be53f7f247626 (100%) rename {src => shark/src}/test/hive/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da (100%) rename {src => shark/src}/test/hive/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f (100%) rename {src => shark/src}/test/hive/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 (100%) rename {src => shark/src}/test/hive/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f (100%) rename {src => shark/src}/test/hive/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 (100%) rename {src => shark/src}/test/hive/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 (100%) rename {src => shark/src}/test/hive/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 (100%) rename {src => shark/src}/test/hive/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 (100%) rename {src => shark/src}/test/hive/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 (100%) rename {src => shark/src}/test/hive/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a (100%) rename {src => shark/src}/test/hive/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 (100%) rename {src => shark/src}/test/hive/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 (100%) rename {src => shark/src}/test/hive/golden/view-11-dc95343d3e57846485dd543476391376 (100%) rename {src => shark/src}/test/hive/golden/view-12-371764e1cae31ea0518c03060528d239 (100%) rename {src => shark/src}/test/hive/golden/view-13-2abce88008f8a19164758ee821aaa8a6 (100%) rename {src => shark/src}/test/hive/golden/view-14-deb504f4f70fd7db975950c3c47959ee (100%) rename {src => shark/src}/test/hive/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b (100%) rename {src => shark/src}/test/hive/golden/view-2-9c529f486fa81a032bfe1253808fca8 (100%) rename {src => shark/src}/test/hive/golden/view-3-89c80c0e90409d5e304775c9f420915a (100%) rename {src => shark/src}/test/hive/golden/view-4-4a64d1a623ca71e515796787dbd0f904 (100%) rename {src => shark/src}/test/hive/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b (100%) rename {src => shark/src}/test/hive/golden/view-6-47b5043f03a84695b6784682b4402ac8 (100%) rename {src => shark/src}/test/hive/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 (100%) rename {src => shark/src}/test/hive/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 (100%) rename {src => shark/src}/test/hive/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 (100%) rename {src => shark/src}/test/hive/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf (100%) rename {src => shark/src}/test/hive/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb (100%) rename {src => shark/src}/test/hive/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 (100%) rename {src => shark/src}/test/hive/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de (100%) rename {src => shark/src}/test/hive/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 (100%) rename {src => shark/src}/test/hive/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 (100%) rename {src => shark/src}/test/hive/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a (100%) rename {src => shark/src}/test/hive/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 (100%) rename {src => shark/src}/test/hive/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb (100%) rename {src => shark/src}/test/hive/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 (100%) rename {src => shark/src}/test/hive/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 (100%) rename {src => shark/src}/test/hive/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 (100%) rename {src => shark/src}/test/hive/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe (100%) rename {src => shark/src}/test/hive/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d (100%) rename {src => shark/src}/test/hive/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 (100%) rename {src => shark/src}/test/hive/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 (100%) rename {src => shark/src}/test/hive/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 (100%) rename {src => shark/src}/test/hive/golden/virtual_column-3-c66776673c986b59b27e704664935988 (100%) rename {src => shark/src}/test/hive/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 (100%) rename {src => shark/src}/test/hive/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 (100%) rename {src => shark/src}/test/hive/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 (100%) rename {src => shark/src}/test/hive/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 (100%) rename {src => shark/src}/test/hive/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 (100%) rename {src => shark/src}/test/hive/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/addpart1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_non_native.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_partition_offline.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_table_add_partition.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_failure.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_failure2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_failure3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_failure4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_failure5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_failure6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_failure7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_failure8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/alter_view_failure9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/altern1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ambiguous_col.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ambiguous_col0.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ambiguous_col1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ambiguous_col2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/analyze.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/analyze1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/analyze_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_corrupt.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_insert1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_insert2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_insert3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_insert4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_multi1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_multi2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_multi3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_multi4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_multi5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_multi6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_multi7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_partspec1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_partspec2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_partspec3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_partspec4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/archive_partspec5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/authorization_fail_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/authorization_fail_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/authorization_fail_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/authorization_fail_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/authorization_fail_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/authorization_fail_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/authorization_fail_7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/authorization_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/autolocal1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/bad_exec_hooks.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/bad_indextype.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/bad_sample_clause.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/cachingprintstream.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/clusterbydistributeby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/clusterbyorderby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/clusterbysortby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/clustern1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/clustern2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/clustern3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/clustern4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/column_rename1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/column_rename2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/column_rename3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/column_rename4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/column_rename5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/compare_double_bigint.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/compare_string_bigint.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_insert_outputformat.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_table_failure1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_table_failure2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_table_failure3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_table_failure4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_table_wrong_regex.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_udaf_failure.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_unknown_genericudf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_view_failure1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_view_failure2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_view_failure3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_view_failure4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_view_failure5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_view_failure6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_view_failure7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_view_failure8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/create_view_failure9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ctas.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/database_create_already_exists.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/database_create_invalid_name.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/date_literal1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/date_literal2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/date_literal3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ddltime.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/decimal_precision.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/decimal_precision_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/default_partition_name.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/deletejar.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/desc_failure1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/desc_failure2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/desc_failure3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/describe_xpath1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/describe_xpath2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/describe_xpath3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/describe_xpath4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_function_failure.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_index_failure.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_native_udf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_partition_failure.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_table_failure1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_table_failure2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_table_failure3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_view_failure1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/drop_view_failure2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/duplicate_insert1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/duplicate_insert2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/duplicate_insert3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/dyn_part1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/dyn_part2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/dyn_part3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/dyn_part4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/dyn_part_max.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_22_export_authfail.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/external1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/external2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/fetchtask_ioexception.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/fileformat_bad_class.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/fileformat_void_input.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/fileformat_void_output.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/fs_default_name1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/fs_default_name2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/genericFileFormat.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_cube1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_cube2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_id1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_invalid_position.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_key.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_rollup1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/groupby_rollup2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/having1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/index_compact_entry_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/index_compact_size_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/input1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/input2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/input4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/input41.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/input_part0_neg.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/insert_into1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/insert_into2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/insert_into3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/insert_into4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/insert_view_failure.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/insertexternal1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_avg_syntax.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_columns.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_config1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_config2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_mapjoin1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_max_syntax.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_min_syntax.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_select_column.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_select_expression.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_std_syntax.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_sum_syntax.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_t_create2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_t_transform.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_tbl_name.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalid_variance_syntax.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/invalidate_view1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/join2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/join28.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/join29.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/join32.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/join35.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/join_nonexistent_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/joinneg.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/lateral_view_alias.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/lateral_view_join.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/line_terminator.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_exist_part_authfail.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_non_native.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_nonpart_authfail.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_part_authfail.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_part_nospec.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_stored_as_dirs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_view_failure.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/load_wrong_noof_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/local_mapred_error_cache.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/lockneg1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/lockneg2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/lockneg3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/lockneg4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/lockneg5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/macro_unused_parameter.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/merge_negative_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/merge_negative_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/merge_negative_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/minimr_broken_pipe.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/nested_complex_neg.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/no_matching_udf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/nonkey_groupby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/nopart_insert.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/nopart_load.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/notable_alias3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/notable_alias4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/orderby_invalid_position.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/orderby_position_unsupported.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/orderbysortby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/part_col_complex_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_part1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_part2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/recursive_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/regex_col_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/regex_col_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/regex_col_groupby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/sa_fail_hook3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/sample.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/script_error.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/select_charliteral.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/select_udtf_alias.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/semijoin1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/semijoin2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/semijoin3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/semijoin4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/serde_regex.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/serde_regex2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/serde_regex3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/set_table_property.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_columns1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_columns2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_columns3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_create_table_index.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_partitions1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_tableproperties1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_tables_bad1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_tables_bad2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_tablestatus.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/smb_mapjoin_14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/split_sample_out_of_range.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/stats_noscan_non_native.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/strict_join.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/strict_orderby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/strict_pruning.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/subq_insert.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/touch1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/touch2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_bucketed_column.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_column_seqfile.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udaf_invalid_place.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_assert_true.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_assert_true2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_coalesce.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_if_not_bool.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_in.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_max.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_min.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_reflect_neg.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_test_error.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_test_error_reduce.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udfnull.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udtf_invalid_place.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/union.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/union2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/union22.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/union3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/uniquejoin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/uniquejoin2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/uniquejoin3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/unset_table_property.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/unset_view_property.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/windowing_ll_no_over.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientnegative/wrong_column_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/add_part_exist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/add_part_multiple.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alias_casted_column.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/allcolref_in_udf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_index.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_merge.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_merge_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_merge_stats.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_partition_coltype.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_partition_format_loc.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_skewed_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_table_not_sorted.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_table_serde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_table_serde2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_varchar1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_varchar2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_view_as_select.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/alter_view_rename.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ambiguous_col.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/archive.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/archive_corrupt.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/archive_multi.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/authorization_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/authorization_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/authorization_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/authorization_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/authorization_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/authorization_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/authorization_7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/authorization_8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join0.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join16.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join17.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join18.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join19.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join21.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join22.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join23.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join24.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join25.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join26.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join27.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join28.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join29.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join30.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join31.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join32.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join_filters.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join_nulls.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_join_reordering_values.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/autogen_colalias.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/avro_change_schema.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/avro_compression_enabled.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/avro_evolved_schemas.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/avro_joins.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/avro_nullable_fields.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/avro_partitioned.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/avro_sanity_test.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/avro_schema_error_message.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/avro_schema_literal.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ba_table1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ba_table2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ba_table3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ba_table_udfs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ba_table_union.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/binary_constant.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/binary_output_format.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/binary_table_bincolserde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/binary_table_colserde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/binarysortable_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucket1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucket2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucket3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucket4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucket5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucket_groupby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketcontext_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketcontext_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketcontext_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketcontext_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketcontext_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketcontext_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketcontext_7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketcontext_8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/case_sensitivity.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/cast1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/cast_to_int.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/cluster.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/column_access_stats.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/columnstats_partlvl.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/columnstats_tbllvl.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/combine1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/combine2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/combine2_hadoop20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/combine2_win.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/combine3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/compute_stats_binary.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/compute_stats_boolean.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/compute_stats_double.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/compute_stats_empty_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/compute_stats_long.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/compute_stats_string.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/constant_prop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/convert_enum_to_string.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/count.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/cp_mj_rc.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_big_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_default_prop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_escape.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_genericudaf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_genericudf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_insert_outputformat.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_like.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_like2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_like_tbl_props.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_like_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_merge_compressed.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_nested_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_or_replace_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_skewed_table1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_struct_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_udaf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_union_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_view_partitioned.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/create_view_translate.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/cross_join.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ct_case_insensitive.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ctas.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ctas_colname.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ctas_date.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ctas_hadoop20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ctas_uses_database_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ctas_varchar.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/custom_input_output_format.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/database.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/database_drop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/database_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/database_properties.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/date_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/date_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/date_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/date_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/date_comparison.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/date_join1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/date_serde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/date_udf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ddltime.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/decimal_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/decimal_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/decimal_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/decimal_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/decimal_join.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/decimal_precision.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/decimal_serde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/decimal_udf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/default_partition_name.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/delimiter.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/describe_comment_indent.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/describe_comment_nonascii.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/describe_database_json.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/describe_pretty.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/describe_syntax.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/describe_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/describe_table_json.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/describe_xpath.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/diff_part_input_formats.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/disable_file_format_check.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/driverhook.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_function.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_index.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_multi_partitions.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_table2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_udf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/drop_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/enforce_order.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/escape1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/escape2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/escape_clusterby1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/escape_distributeby1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/escape_orderby1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/escape_sortby1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exchange_partition.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exchange_partition2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exchange_partition3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_01_nonpart.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_02_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_04_all_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_05_some_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_06_one_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_10_external_managed.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_11_managed_external.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_12_external_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_13_managed_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_15_external_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_16_part_external.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_17_part_managed.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_18_part_external.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_19_part_external_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/explain_dependency.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/explain_dependency2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/explain_logical.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/explode_null.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/fetch_aggregation.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/fileformat_mix.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/fileformat_sequencefile.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/fileformat_text.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/global_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby1_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby1_map.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby1_map_nomap.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby1_map_skew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby1_noskew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby2_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby2_map.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby2_map_skew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby3_map.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby3_map_skew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby4_map.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby4_map_skew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby4_noskew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby5_map.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby5_map_skew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby5_noskew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby6_map.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby6_map_skew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby6_noskew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby7_map.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby7_map_skew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby8_map.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby8_map_skew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby8_noskew.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_bigdata.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_cube1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_neg_float.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_position.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_ppd.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_ppr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_rollup1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/groupby_sort_test_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/having.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/hook_context_cs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/hook_order.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/implicit_cast1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auth.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auto.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auto_empty.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auto_file_format.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auto_multiple.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auto_partitioned.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auto_self_join.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auto_unused.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_auto_update.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_bitmap.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_bitmap1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_bitmap2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_bitmap3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_bitmap_compression.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_bitmap_rc.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_compact.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_compact_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_compact_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_compact_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_compact_binary_search.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_compression.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_creation.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_serde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_stale.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/index_stale_partitioned.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/infer_const_type.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/init_file.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/innerjoin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/inoutdriver.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input0.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input11_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input12_hadoop20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input14_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input16.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input16_cc.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input17.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input18.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input19.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input1_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input21.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input22.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input23.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input24.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input25.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input26.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input28.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input2_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input30.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input31.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input32.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input33.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input34.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input35.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input36.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input37.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input38.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input39.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input39_hadoop20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input3_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input40.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input41.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input42.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input43.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input44.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input45.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input46.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input49.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input4_cb_delim.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input4_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_columnarserde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_dfs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_dynamicserde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_lazyserde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_limit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part0.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part10_win.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_part9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_testxpath.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_testxpath2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_testxpath3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/input_testxpath4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/inputddl1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/inputddl2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/inputddl3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/inputddl4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/inputddl5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/inputddl6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/inputddl7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/inputddl8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert_compressed.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert_into1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert_into2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert_into3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert_into4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert_into5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert_into6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/insertexternal1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join0.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join14_hadoop20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join16.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join17.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join18.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join18_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join19.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join21.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join22.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join23.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join24.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join25.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join26.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join27.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join28.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join29.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join30.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join31.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join32.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join32_lessSize.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join33.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join34.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join35.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join36.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join37.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join38.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join39.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join40.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join41.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_1to1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_array.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_casesensitive.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_empty.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_filters.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_filters_overlap.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_hive_626.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_literals.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_map_ppr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_nulls.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_nullsafe.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_rc.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_reorder.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_reorder2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_reorder3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_reorder4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_star.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_thrift.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_vc.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/join_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/keyword_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/lateral_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/lateral_view_cp.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/lateral_view_noalias.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/lateral_view_outer.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/lateral_view_ppd.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/leadlag.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/leadlag_queries.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/leftsemijoin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/leftsemijoin_mr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/limit_pushdown.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/limit_pushdown_negative.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/lineage1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/literal_decimal.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/literal_double.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/literal_ints.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/literal_string.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_binary_data.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14_win.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_dyn_part9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_fs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_fs2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_overwrite.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/load_part_authsuccess.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/loadpart1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/loadpart_err.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/lock1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/lock2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/lock3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/lock4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/louter_join_ppr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/macro.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapjoin1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapjoin_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapjoin_hook.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapjoin_test_outer.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapreduce1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapreduce2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapreduce3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapreduce4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapreduce5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapreduce6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapreduce7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mapreduce8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/merge1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/merge2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/merge3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/merge4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mergejoins.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mergejoins_mixed.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/metadata_export_drop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/metadataonly1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/mi.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/misc_json.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multiMapJoin1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multiMapJoin2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multi_insert.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multi_join_union.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multi_sahooks.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/multigroupby_singlemr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nested_complex.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nestedvirtual.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/newline.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/no_hooks.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/noalias_subq1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nonmr_fetch.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/notable_alias1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/notable_alias2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/null_cast.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/null_column.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nullgroup.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nullgroup2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nullgroup3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nullgroup4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nullgroup5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nullinput.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nullinput2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/nullscript.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/num_op_type_conv.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ops_comparison.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/optional_outer.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/optrstat_groupby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/orc_create.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/orc_createas1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/orc_diff_part_cols.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/orc_empty_files.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/orc_empty_strings.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/order.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/order2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/outer_join_ppr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/overridden_confs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/parallel.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/parallel_orderby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/parenthesis_star_by.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partInit.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partcols1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_date.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_date2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_decode_name.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_schema1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_serde_format.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_special_char.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_type_check.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_varchar1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/partitions_json.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/pcr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/plan_json.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_clusterby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_constant_expr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_constant_where.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_gby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_gby2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_gby_join.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_join.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_join2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_join3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_join_filter.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_multi_insert.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_random.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_repeated_alias.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_transform.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_udf_case.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_udf_col.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_union.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_union_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppd_vc.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/print_header.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/progress_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/protectmode.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/protectmode2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ptf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ptf_decimal.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ptf_general_queries.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ptf_matchpath.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ptf_rcfile.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ptf_register_tblfn.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ptf_seqfile.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/push_or.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/query_properties.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/query_result_fileformat.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/query_with_semi.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/quote1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/quote2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_bigdata.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_columnar.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_createas1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_default_format.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_merge1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_merge2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_merge3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_merge4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_null_value.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rcfile_union.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/recursive_dir.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/reducesink_dedup.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/regex_col.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/regexp_extract.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/remote_script.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rename_column.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rename_external_partition_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rename_partition_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/rename_table_location.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/repair.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/reset_conf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/router_join_ppr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/schemeAuthority.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/schemeAuthority2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/script_env_var1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/script_env_var2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/script_pipe.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/scriptfile1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/select_as_omitted.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/select_transform_hint.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/select_unquote_and.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/select_unquote_not.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/select_unquote_or.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/semicolon.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/semijoin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/serde_regex.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/serde_reported_schema.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/serde_user_properties.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/set_processor_namespaces.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/set_variable_sub.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_columns.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_create_table_alter.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_create_table_db_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_create_table_delimited.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_create_table_partitioned.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_create_table_serde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_create_table_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_describe_func_quotes.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_functions.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_indexes_syntax.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_partitions.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_tables.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_tablestatus.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/show_tblproperties.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/showparts.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt16.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt17.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt18.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt19.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/skewjoinopt9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_16.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_17.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_18.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_19.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_21.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_22.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_25.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sort.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/source.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/split_sample.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats0.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats16.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats18.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats19.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats_empty_partition.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats_noscan_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats_noscan_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1_23.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/stats_publisher_error_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/str_to_map.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/subq.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/subq2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/symlink_text_input_format.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/table_access_keys_stats.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/tablename_with_select.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/test_boolean_whereclause.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/timestamp_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/timestamp_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/timestamp_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/timestamp_comparison.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/timestamp_lazy.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/timestamp_null.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/timestamp_udf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/touch.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/transform1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/transform2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/transform_ppr1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/transform_ppr2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/truncate_column.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/truncate_column_buckets.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/truncate_column_merge.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/truncate_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/type_cast_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/type_conversions_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/type_widening.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udaf_collect_set.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udaf_context_ngrams.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udaf_corr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udaf_covar_pop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udaf_covar_samp.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udaf_ngrams.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udaf_number_format.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_10_trims.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_E.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_PI.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_abs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_acos.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_add.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_array.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_array_contains.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_ascii.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_asin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_atan.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_avg.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_between.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_bigint.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_bin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_and.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_empty.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_or.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_and.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_not.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_or.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_xor.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_boolean.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_case.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_case_column_pruning.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_case_thrift.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_ceil.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_ceiling.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_coalesce.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_compare_java_string.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_concat.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_concat_ws.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_context_aware.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_conv.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_cos.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_count.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_date_add.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_date_sub.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_datediff.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_day.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_dayofmonth.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_degrees.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_div.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_divide.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_double.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_elt.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_equal.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_exp.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_explode.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_field.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_find_in_set.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_float.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_floor.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_format_number.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_from_unixtime.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_get_json_object.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_greaterthan.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_hash.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_hex.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_hour.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_if.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_in.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_in_file.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_index.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_inline.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_instr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_int.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_isnotnull.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_isnull.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_java_method.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_lcase.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_length.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_lessthan.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_lessthanorequal.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_like.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_ln.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_locate.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_log.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_log10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_log2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_lower.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_lpad.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_ltrim.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_map.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_map_keys.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_map_values.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_max.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_min.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_minute.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_modulo.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_month.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_named_struct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_negative.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_not.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_notequal.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_notop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_nvl.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_or.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_parse_url.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_percentile.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_pmod.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_positive.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_pow.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_power.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_printf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_radians.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_rand.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_reflect.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_reflect2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_regexp.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_regexp_extract.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_regexp_replace.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_repeat.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_reverse.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_rlike.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_round.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_round_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_round_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_rpad.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_rtrim.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_second.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_sentences.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_sign.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_sin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_size.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_smallint.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_sort_array.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_space.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_split.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_sqrt.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_std.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_stddev.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_stddev_pop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_stddev_samp.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_string.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_struct.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_substr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_substring.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_subtract.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_sum.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_tan.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_testlength.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_testlength2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_tinyint.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_to_boolean.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_to_byte.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_to_date.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_to_double.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_to_float.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_to_long.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_to_short.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_to_string.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_translate.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_trim.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_ucase.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_unhex.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_union.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_unix_timestamp.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_upper.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_var_pop.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_var_samp.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_variance.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_weekofyear.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_when.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_xpath.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_xpath_boolean.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_xpath_double.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_xpath_float.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_xpath_int.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_xpath_long.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_xpath_short.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udf_xpath_string.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udtf_explode.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udtf_json_tuple.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/udtf_stack.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/unicode_notation.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union16.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union17.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union18.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union19.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union21.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union22.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union23.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union24.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union25.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union26.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union27.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union28.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union29.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union30.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union31.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union32.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union33.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union34.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_date.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_lateralview.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_null.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_ppr.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_10.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_11.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_12.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_13.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_14.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_15.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_16.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_17.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_18.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_19.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_21.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_22.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_23.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_24.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_remove_9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_script.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/union_view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/uniquejoin.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/unset_table_view_property.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/updateAccessTime.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/varchar_1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/varchar_2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/varchar_cast.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/varchar_comparison.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/varchar_join1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/varchar_nested_types.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/varchar_serde.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/varchar_udf1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/varchar_union1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/view.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/view_cast.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/view_inputs.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/virtual_column.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/windowing.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/windowing_columnPruning.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/windowing_expressions.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/windowing_multipartitioning.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/windowing_navfn.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/windowing_ntile.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/windowing_rank.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/windowing_udaf.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/clientpositive/windowing_windowspec.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/ambiguous_join_col.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/duplicate_alias.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/garbage.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/insert_wrong_number_columns.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/invalid_create_table.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/invalid_dot.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/invalid_function_param2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/invalid_index.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/invalid_list_index.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/invalid_list_index2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/invalid_map_index.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/invalid_map_index2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/invalid_select.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/macro_reserved_word.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/missing_overwrite.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/nonkey_groupby.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/quoted_string.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_column1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_column2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_column3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_column4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_column5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_column6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_function1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_function2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_function3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_function4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_table1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/unknown_table2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/wrong_distinct1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/negative/wrong_distinct2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/case_sensitivity.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/cast1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/groupby1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/groupby2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/groupby3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/groupby4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/groupby5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/groupby6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input20.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input9.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input_part1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input_testsequencefile.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input_testxpath.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/input_testxpath2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/join1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/join2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/join3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/join4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/join5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/join6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/join7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/join8.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/sample1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/sample2.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/sample3.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/sample4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/sample5.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/sample6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/sample7.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/subq.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/udf1.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/udf4.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/udf6.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/udf_case.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/udf_when.q (100%) rename {src => shark/src}/test/hive/ql/src/test/queries/positive/union.q (100%) diff --git a/src/test/hive/data/conf/hive-log4j.properties b/shark/src/test/hive/data/conf/hive-log4j.properties similarity index 100% rename from src/test/hive/data/conf/hive-log4j.properties rename to shark/src/test/hive/data/conf/hive-log4j.properties diff --git a/src/test/hive/data/conf/hive-site.xml b/shark/src/test/hive/data/conf/hive-site.xml similarity index 100% rename from src/test/hive/data/conf/hive-site.xml rename to shark/src/test/hive/data/conf/hive-site.xml diff --git a/src/test/hive/data/files/SortCol1Col2.txt b/shark/src/test/hive/data/files/SortCol1Col2.txt similarity index 100% rename from src/test/hive/data/files/SortCol1Col2.txt rename to shark/src/test/hive/data/files/SortCol1Col2.txt diff --git a/src/test/hive/data/files/SortCol2Col1.txt b/shark/src/test/hive/data/files/SortCol2Col1.txt similarity index 100% rename from src/test/hive/data/files/SortCol2Col1.txt rename to shark/src/test/hive/data/files/SortCol2Col1.txt diff --git a/src/test/hive/data/files/SortDescCol1Col2.txt b/shark/src/test/hive/data/files/SortDescCol1Col2.txt similarity index 100% rename from src/test/hive/data/files/SortDescCol1Col2.txt rename to shark/src/test/hive/data/files/SortDescCol1Col2.txt diff --git a/src/test/hive/data/files/SortDescCol2Col1.txt b/shark/src/test/hive/data/files/SortDescCol2Col1.txt similarity index 100% rename from src/test/hive/data/files/SortDescCol2Col1.txt rename to shark/src/test/hive/data/files/SortDescCol2Col1.txt diff --git a/src/test/hive/data/files/T1.txt b/shark/src/test/hive/data/files/T1.txt similarity index 100% rename from src/test/hive/data/files/T1.txt rename to shark/src/test/hive/data/files/T1.txt diff --git a/src/test/hive/data/files/T2.txt b/shark/src/test/hive/data/files/T2.txt similarity index 100% rename from src/test/hive/data/files/T2.txt rename to shark/src/test/hive/data/files/T2.txt diff --git a/src/test/hive/data/files/T3.txt b/shark/src/test/hive/data/files/T3.txt similarity index 100% rename from src/test/hive/data/files/T3.txt rename to shark/src/test/hive/data/files/T3.txt diff --git a/src/test/hive/data/files/TestSerDe.jar b/shark/src/test/hive/data/files/TestSerDe.jar similarity index 100% rename from src/test/hive/data/files/TestSerDe.jar rename to shark/src/test/hive/data/files/TestSerDe.jar diff --git a/src/test/hive/data/files/UserVisits.dat b/shark/src/test/hive/data/files/UserVisits.dat similarity index 100% rename from src/test/hive/data/files/UserVisits.dat rename to shark/src/test/hive/data/files/UserVisits.dat diff --git a/src/test/hive/data/files/apache.access.2.log b/shark/src/test/hive/data/files/apache.access.2.log similarity index 100% rename from src/test/hive/data/files/apache.access.2.log rename to shark/src/test/hive/data/files/apache.access.2.log diff --git a/src/test/hive/data/files/apache.access.log b/shark/src/test/hive/data/files/apache.access.log similarity index 100% rename from src/test/hive/data/files/apache.access.log rename to shark/src/test/hive/data/files/apache.access.log diff --git a/src/test/hive/data/files/archive_corrupt.rc b/shark/src/test/hive/data/files/archive_corrupt.rc similarity index 100% rename from src/test/hive/data/files/archive_corrupt.rc rename to shark/src/test/hive/data/files/archive_corrupt.rc diff --git a/src/test/hive/data/files/array_table.txt b/shark/src/test/hive/data/files/array_table.txt similarity index 100% rename from src/test/hive/data/files/array_table.txt rename to shark/src/test/hive/data/files/array_table.txt diff --git a/src/test/hive/data/files/binary.txt b/shark/src/test/hive/data/files/binary.txt similarity index 100% rename from src/test/hive/data/files/binary.txt rename to shark/src/test/hive/data/files/binary.txt diff --git a/src/test/hive/data/files/bool.txt b/shark/src/test/hive/data/files/bool.txt similarity index 100% rename from src/test/hive/data/files/bool.txt rename to shark/src/test/hive/data/files/bool.txt diff --git a/src/test/hive/data/files/complex.seq b/shark/src/test/hive/data/files/complex.seq similarity index 100% rename from src/test/hive/data/files/complex.seq rename to shark/src/test/hive/data/files/complex.seq diff --git a/src/test/hive/data/files/covar_tab.txt b/shark/src/test/hive/data/files/covar_tab.txt similarity index 100% rename from src/test/hive/data/files/covar_tab.txt rename to shark/src/test/hive/data/files/covar_tab.txt diff --git a/src/test/hive/data/files/create_nested_type.txt b/shark/src/test/hive/data/files/create_nested_type.txt similarity index 100% rename from src/test/hive/data/files/create_nested_type.txt rename to shark/src/test/hive/data/files/create_nested_type.txt diff --git a/src/test/hive/data/files/csv.txt b/shark/src/test/hive/data/files/csv.txt similarity index 100% rename from src/test/hive/data/files/csv.txt rename to shark/src/test/hive/data/files/csv.txt diff --git a/src/test/hive/data/files/datatypes.txt b/shark/src/test/hive/data/files/datatypes.txt similarity index 100% rename from src/test/hive/data/files/datatypes.txt rename to shark/src/test/hive/data/files/datatypes.txt diff --git a/src/test/hive/data/files/dim-data.txt b/shark/src/test/hive/data/files/dim-data.txt similarity index 100% rename from src/test/hive/data/files/dim-data.txt rename to shark/src/test/hive/data/files/dim-data.txt diff --git a/src/test/hive/data/files/doctors.avro b/shark/src/test/hive/data/files/doctors.avro similarity index 100% rename from src/test/hive/data/files/doctors.avro rename to shark/src/test/hive/data/files/doctors.avro diff --git a/src/test/hive/data/files/docurl.txt b/shark/src/test/hive/data/files/docurl.txt similarity index 100% rename from src/test/hive/data/files/docurl.txt rename to shark/src/test/hive/data/files/docurl.txt diff --git a/src/test/hive/data/files/double.txt b/shark/src/test/hive/data/files/double.txt similarity index 100% rename from src/test/hive/data/files/double.txt rename to shark/src/test/hive/data/files/double.txt diff --git a/src/test/hive/data/files/employee.dat b/shark/src/test/hive/data/files/employee.dat similarity index 100% rename from src/test/hive/data/files/employee.dat rename to shark/src/test/hive/data/files/employee.dat diff --git a/src/test/hive/data/files/employee2.dat b/shark/src/test/hive/data/files/employee2.dat similarity index 100% rename from src/test/hive/data/files/employee2.dat rename to shark/src/test/hive/data/files/employee2.dat diff --git a/src/test/hive/data/files/employee_part.txt b/shark/src/test/hive/data/files/employee_part.txt similarity index 100% rename from src/test/hive/data/files/employee_part.txt rename to shark/src/test/hive/data/files/employee_part.txt diff --git a/src/test/hive/data/files/empty1.txt b/shark/src/test/hive/data/files/empty1.txt similarity index 100% rename from src/test/hive/data/files/empty1.txt rename to shark/src/test/hive/data/files/empty1.txt diff --git a/src/test/hive/data/files/empty2.txt b/shark/src/test/hive/data/files/empty2.txt similarity index 100% rename from src/test/hive/data/files/empty2.txt rename to shark/src/test/hive/data/files/empty2.txt diff --git a/src/test/hive/data/files/episodes.avro b/shark/src/test/hive/data/files/episodes.avro similarity index 100% rename from src/test/hive/data/files/episodes.avro rename to shark/src/test/hive/data/files/episodes.avro diff --git a/src/test/hive/data/files/escapetest.txt b/shark/src/test/hive/data/files/escapetest.txt similarity index 100% rename from src/test/hive/data/files/escapetest.txt rename to shark/src/test/hive/data/files/escapetest.txt diff --git a/src/test/hive/data/files/ext_test/test.dat b/shark/src/test/hive/data/files/ext_test/test.dat similarity index 100% rename from src/test/hive/data/files/ext_test/test.dat rename to shark/src/test/hive/data/files/ext_test/test.dat diff --git a/src/test/hive/data/files/fact-data.txt b/shark/src/test/hive/data/files/fact-data.txt similarity index 100% rename from src/test/hive/data/files/fact-data.txt rename to shark/src/test/hive/data/files/fact-data.txt diff --git a/src/test/hive/data/files/flights_join.txt b/shark/src/test/hive/data/files/flights_join.txt similarity index 100% rename from src/test/hive/data/files/flights_join.txt rename to shark/src/test/hive/data/files/flights_join.txt diff --git a/src/test/hive/data/files/flights_tiny.txt b/shark/src/test/hive/data/files/flights_tiny.txt similarity index 100% rename from src/test/hive/data/files/flights_tiny.txt rename to shark/src/test/hive/data/files/flights_tiny.txt diff --git a/src/test/hive/data/files/flights_tiny.txt.1 b/shark/src/test/hive/data/files/flights_tiny.txt.1 similarity index 100% rename from src/test/hive/data/files/flights_tiny.txt.1 rename to shark/src/test/hive/data/files/flights_tiny.txt.1 diff --git a/src/test/hive/data/files/groupby_groupingid.txt b/shark/src/test/hive/data/files/groupby_groupingid.txt similarity index 100% rename from src/test/hive/data/files/groupby_groupingid.txt rename to shark/src/test/hive/data/files/groupby_groupingid.txt diff --git a/src/test/hive/data/files/grouping_sets.txt b/shark/src/test/hive/data/files/grouping_sets.txt similarity index 100% rename from src/test/hive/data/files/grouping_sets.txt rename to shark/src/test/hive/data/files/grouping_sets.txt diff --git a/src/test/hive/data/files/grouping_sets1.txt b/shark/src/test/hive/data/files/grouping_sets1.txt similarity index 100% rename from src/test/hive/data/files/grouping_sets1.txt rename to shark/src/test/hive/data/files/grouping_sets1.txt diff --git a/src/test/hive/data/files/grouping_sets2.txt b/shark/src/test/hive/data/files/grouping_sets2.txt similarity index 100% rename from src/test/hive/data/files/grouping_sets2.txt rename to shark/src/test/hive/data/files/grouping_sets2.txt diff --git a/src/test/hive/data/files/hive_626_bar.txt b/shark/src/test/hive/data/files/hive_626_bar.txt similarity index 100% rename from src/test/hive/data/files/hive_626_bar.txt rename to shark/src/test/hive/data/files/hive_626_bar.txt diff --git a/src/test/hive/data/files/hive_626_count.txt b/shark/src/test/hive/data/files/hive_626_count.txt similarity index 100% rename from src/test/hive/data/files/hive_626_count.txt rename to shark/src/test/hive/data/files/hive_626_count.txt diff --git a/src/test/hive/data/files/hive_626_foo.txt b/shark/src/test/hive/data/files/hive_626_foo.txt similarity index 100% rename from src/test/hive/data/files/hive_626_foo.txt rename to shark/src/test/hive/data/files/hive_626_foo.txt diff --git a/src/test/hive/data/files/in1.txt b/shark/src/test/hive/data/files/in1.txt similarity index 100% rename from src/test/hive/data/files/in1.txt rename to shark/src/test/hive/data/files/in1.txt diff --git a/src/test/hive/data/files/in2.txt b/shark/src/test/hive/data/files/in2.txt similarity index 100% rename from src/test/hive/data/files/in2.txt rename to shark/src/test/hive/data/files/in2.txt diff --git a/src/test/hive/data/files/in3.txt b/shark/src/test/hive/data/files/in3.txt similarity index 100% rename from src/test/hive/data/files/in3.txt rename to shark/src/test/hive/data/files/in3.txt diff --git a/src/test/hive/data/files/in4.txt b/shark/src/test/hive/data/files/in4.txt similarity index 100% rename from src/test/hive/data/files/in4.txt rename to shark/src/test/hive/data/files/in4.txt diff --git a/src/test/hive/data/files/in5.txt b/shark/src/test/hive/data/files/in5.txt similarity index 100% rename from src/test/hive/data/files/in5.txt rename to shark/src/test/hive/data/files/in5.txt diff --git a/src/test/hive/data/files/in6.txt b/shark/src/test/hive/data/files/in6.txt similarity index 100% rename from src/test/hive/data/files/in6.txt rename to shark/src/test/hive/data/files/in6.txt diff --git a/src/test/hive/data/files/in7.txt b/shark/src/test/hive/data/files/in7.txt similarity index 100% rename from src/test/hive/data/files/in7.txt rename to shark/src/test/hive/data/files/in7.txt diff --git a/src/test/hive/data/files/in8.txt b/shark/src/test/hive/data/files/in8.txt similarity index 100% rename from src/test/hive/data/files/in8.txt rename to shark/src/test/hive/data/files/in8.txt diff --git a/src/test/hive/data/files/in9.txt b/shark/src/test/hive/data/files/in9.txt similarity index 100% rename from src/test/hive/data/files/in9.txt rename to shark/src/test/hive/data/files/in9.txt diff --git a/src/test/hive/data/files/infer_const_type.txt b/shark/src/test/hive/data/files/infer_const_type.txt similarity index 100% rename from src/test/hive/data/files/infer_const_type.txt rename to shark/src/test/hive/data/files/infer_const_type.txt diff --git a/src/test/hive/data/files/int.txt b/shark/src/test/hive/data/files/int.txt similarity index 100% rename from src/test/hive/data/files/int.txt rename to shark/src/test/hive/data/files/int.txt diff --git a/src/test/hive/data/files/json.txt b/shark/src/test/hive/data/files/json.txt similarity index 100% rename from src/test/hive/data/files/json.txt rename to shark/src/test/hive/data/files/json.txt diff --git a/src/test/hive/data/files/kv1.seq b/shark/src/test/hive/data/files/kv1.seq similarity index 100% rename from src/test/hive/data/files/kv1.seq rename to shark/src/test/hive/data/files/kv1.seq diff --git a/src/test/hive/data/files/kv1.string-sorted.txt b/shark/src/test/hive/data/files/kv1.string-sorted.txt similarity index 100% rename from src/test/hive/data/files/kv1.string-sorted.txt rename to shark/src/test/hive/data/files/kv1.string-sorted.txt diff --git a/src/test/hive/data/files/kv1.txt b/shark/src/test/hive/data/files/kv1.txt similarity index 100% rename from src/test/hive/data/files/kv1.txt rename to shark/src/test/hive/data/files/kv1.txt diff --git a/src/test/hive/data/files/kv1.val.sorted.txt b/shark/src/test/hive/data/files/kv1.val.sorted.txt similarity index 100% rename from src/test/hive/data/files/kv1.val.sorted.txt rename to shark/src/test/hive/data/files/kv1.val.sorted.txt diff --git a/src/test/hive/data/files/kv1_broken.seq b/shark/src/test/hive/data/files/kv1_broken.seq similarity index 100% rename from src/test/hive/data/files/kv1_broken.seq rename to shark/src/test/hive/data/files/kv1_broken.seq diff --git a/src/test/hive/data/files/kv1_cb.txt b/shark/src/test/hive/data/files/kv1_cb.txt similarity index 100% rename from src/test/hive/data/files/kv1_cb.txt rename to shark/src/test/hive/data/files/kv1_cb.txt diff --git a/src/test/hive/data/files/kv1_cc.txt b/shark/src/test/hive/data/files/kv1_cc.txt similarity index 100% rename from src/test/hive/data/files/kv1_cc.txt rename to shark/src/test/hive/data/files/kv1_cc.txt diff --git a/src/test/hive/data/files/kv1kv2.cogroup.txt b/shark/src/test/hive/data/files/kv1kv2.cogroup.txt similarity index 100% rename from src/test/hive/data/files/kv1kv2.cogroup.txt rename to shark/src/test/hive/data/files/kv1kv2.cogroup.txt diff --git a/src/test/hive/data/files/kv2.txt b/shark/src/test/hive/data/files/kv2.txt similarity index 100% rename from src/test/hive/data/files/kv2.txt rename to shark/src/test/hive/data/files/kv2.txt diff --git a/src/test/hive/data/files/kv3.txt b/shark/src/test/hive/data/files/kv3.txt similarity index 100% rename from src/test/hive/data/files/kv3.txt rename to shark/src/test/hive/data/files/kv3.txt diff --git a/src/test/hive/data/files/kv4.txt b/shark/src/test/hive/data/files/kv4.txt similarity index 100% rename from src/test/hive/data/files/kv4.txt rename to shark/src/test/hive/data/files/kv4.txt diff --git a/src/test/hive/data/files/kv5.txt b/shark/src/test/hive/data/files/kv5.txt similarity index 100% rename from src/test/hive/data/files/kv5.txt rename to shark/src/test/hive/data/files/kv5.txt diff --git a/src/test/hive/data/files/kv6.txt b/shark/src/test/hive/data/files/kv6.txt similarity index 100% rename from src/test/hive/data/files/kv6.txt rename to shark/src/test/hive/data/files/kv6.txt diff --git a/src/test/hive/data/files/kv7.txt b/shark/src/test/hive/data/files/kv7.txt similarity index 100% rename from src/test/hive/data/files/kv7.txt rename to shark/src/test/hive/data/files/kv7.txt diff --git a/src/test/hive/data/files/kv8.txt b/shark/src/test/hive/data/files/kv8.txt similarity index 100% rename from src/test/hive/data/files/kv8.txt rename to shark/src/test/hive/data/files/kv8.txt diff --git a/src/test/hive/data/files/leftsemijoin_mr_t1.txt b/shark/src/test/hive/data/files/leftsemijoin_mr_t1.txt similarity index 100% rename from src/test/hive/data/files/leftsemijoin_mr_t1.txt rename to shark/src/test/hive/data/files/leftsemijoin_mr_t1.txt diff --git a/src/test/hive/data/files/leftsemijoin_mr_t2.txt b/shark/src/test/hive/data/files/leftsemijoin_mr_t2.txt similarity index 100% rename from src/test/hive/data/files/leftsemijoin_mr_t2.txt rename to shark/src/test/hive/data/files/leftsemijoin_mr_t2.txt diff --git a/src/test/hive/data/files/lineitem.txt b/shark/src/test/hive/data/files/lineitem.txt similarity index 100% rename from src/test/hive/data/files/lineitem.txt rename to shark/src/test/hive/data/files/lineitem.txt diff --git a/src/test/hive/data/files/lt100.sorted.txt b/shark/src/test/hive/data/files/lt100.sorted.txt similarity index 100% rename from src/test/hive/data/files/lt100.sorted.txt rename to shark/src/test/hive/data/files/lt100.sorted.txt diff --git a/src/test/hive/data/files/lt100.txt b/shark/src/test/hive/data/files/lt100.txt similarity index 100% rename from src/test/hive/data/files/lt100.txt rename to shark/src/test/hive/data/files/lt100.txt diff --git a/src/test/hive/data/files/lt100.txt.deflate b/shark/src/test/hive/data/files/lt100.txt.deflate similarity index 100% rename from src/test/hive/data/files/lt100.txt.deflate rename to shark/src/test/hive/data/files/lt100.txt.deflate diff --git a/src/test/hive/data/files/map_table.txt b/shark/src/test/hive/data/files/map_table.txt similarity index 100% rename from src/test/hive/data/files/map_table.txt rename to shark/src/test/hive/data/files/map_table.txt diff --git a/src/test/hive/data/files/nested_complex.txt b/shark/src/test/hive/data/files/nested_complex.txt similarity index 100% rename from src/test/hive/data/files/nested_complex.txt rename to shark/src/test/hive/data/files/nested_complex.txt diff --git a/src/test/hive/data/files/null.txt b/shark/src/test/hive/data/files/null.txt similarity index 100% rename from src/test/hive/data/files/null.txt rename to shark/src/test/hive/data/files/null.txt diff --git a/src/test/hive/data/files/nullfile.txt b/shark/src/test/hive/data/files/nullfile.txt similarity index 100% rename from src/test/hive/data/files/nullfile.txt rename to shark/src/test/hive/data/files/nullfile.txt diff --git a/src/test/hive/data/files/nulls.txt b/shark/src/test/hive/data/files/nulls.txt similarity index 100% rename from src/test/hive/data/files/nulls.txt rename to shark/src/test/hive/data/files/nulls.txt diff --git a/src/test/hive/data/files/orc_create.txt b/shark/src/test/hive/data/files/orc_create.txt similarity index 100% rename from src/test/hive/data/files/orc_create.txt rename to shark/src/test/hive/data/files/orc_create.txt diff --git a/src/test/hive/data/files/orc_create_people.txt b/shark/src/test/hive/data/files/orc_create_people.txt similarity index 100% rename from src/test/hive/data/files/orc_create_people.txt rename to shark/src/test/hive/data/files/orc_create_people.txt diff --git a/src/test/hive/data/files/over10k b/shark/src/test/hive/data/files/over10k similarity index 100% rename from src/test/hive/data/files/over10k rename to shark/src/test/hive/data/files/over10k diff --git a/src/test/hive/data/files/over1k b/shark/src/test/hive/data/files/over1k similarity index 100% rename from src/test/hive/data/files/over1k rename to shark/src/test/hive/data/files/over1k diff --git a/src/test/hive/data/files/part.rc b/shark/src/test/hive/data/files/part.rc similarity index 100% rename from src/test/hive/data/files/part.rc rename to shark/src/test/hive/data/files/part.rc diff --git a/src/test/hive/data/files/part.seq b/shark/src/test/hive/data/files/part.seq similarity index 100% rename from src/test/hive/data/files/part.seq rename to shark/src/test/hive/data/files/part.seq diff --git a/src/test/hive/data/files/part_tiny.txt b/shark/src/test/hive/data/files/part_tiny.txt similarity index 100% rename from src/test/hive/data/files/part_tiny.txt rename to shark/src/test/hive/data/files/part_tiny.txt diff --git a/src/test/hive/data/files/person age.txt b/shark/src/test/hive/data/files/person age.txt similarity index 100% rename from src/test/hive/data/files/person age.txt rename to shark/src/test/hive/data/files/person age.txt diff --git a/src/test/hive/data/files/primitive_type_arrays.txt b/shark/src/test/hive/data/files/primitive_type_arrays.txt similarity index 100% rename from src/test/hive/data/files/primitive_type_arrays.txt rename to shark/src/test/hive/data/files/primitive_type_arrays.txt diff --git a/src/test/hive/data/files/pw17.txt b/shark/src/test/hive/data/files/pw17.txt similarity index 100% rename from src/test/hive/data/files/pw17.txt rename to shark/src/test/hive/data/files/pw17.txt diff --git a/src/test/hive/data/files/sales.txt b/shark/src/test/hive/data/files/sales.txt similarity index 100% rename from src/test/hive/data/files/sales.txt rename to shark/src/test/hive/data/files/sales.txt diff --git a/src/test/hive/data/files/sample-queryplan-in-history.txt b/shark/src/test/hive/data/files/sample-queryplan-in-history.txt similarity index 100% rename from src/test/hive/data/files/sample-queryplan-in-history.txt rename to shark/src/test/hive/data/files/sample-queryplan-in-history.txt diff --git a/src/test/hive/data/files/sample-queryplan.txt b/shark/src/test/hive/data/files/sample-queryplan.txt similarity index 100% rename from src/test/hive/data/files/sample-queryplan.txt rename to shark/src/test/hive/data/files/sample-queryplan.txt diff --git a/src/test/hive/data/files/smallsrcsortbucket1outof4.txt b/shark/src/test/hive/data/files/smallsrcsortbucket1outof4.txt similarity index 100% rename from src/test/hive/data/files/smallsrcsortbucket1outof4.txt rename to shark/src/test/hive/data/files/smallsrcsortbucket1outof4.txt diff --git a/src/test/hive/data/files/smallsrcsortbucket2outof4.txt b/shark/src/test/hive/data/files/smallsrcsortbucket2outof4.txt similarity index 100% rename from src/test/hive/data/files/smallsrcsortbucket2outof4.txt rename to shark/src/test/hive/data/files/smallsrcsortbucket2outof4.txt diff --git a/src/test/hive/data/files/smallsrcsortbucket3outof4.txt b/shark/src/test/hive/data/files/smallsrcsortbucket3outof4.txt similarity index 100% rename from src/test/hive/data/files/smallsrcsortbucket3outof4.txt rename to shark/src/test/hive/data/files/smallsrcsortbucket3outof4.txt diff --git a/src/test/hive/data/files/smallsrcsortbucket4outof4.txt b/shark/src/test/hive/data/files/smallsrcsortbucket4outof4.txt similarity index 100% rename from src/test/hive/data/files/smallsrcsortbucket4outof4.txt rename to shark/src/test/hive/data/files/smallsrcsortbucket4outof4.txt diff --git a/src/test/hive/data/files/smb_bucket_input.rc b/shark/src/test/hive/data/files/smb_bucket_input.rc similarity index 100% rename from src/test/hive/data/files/smb_bucket_input.rc rename to shark/src/test/hive/data/files/smb_bucket_input.rc diff --git a/src/test/hive/data/files/smb_bucket_input.txt b/shark/src/test/hive/data/files/smb_bucket_input.txt similarity index 100% rename from src/test/hive/data/files/smb_bucket_input.txt rename to shark/src/test/hive/data/files/smb_bucket_input.txt diff --git a/src/test/hive/data/files/smbbucket_1.rc b/shark/src/test/hive/data/files/smbbucket_1.rc similarity index 100% rename from src/test/hive/data/files/smbbucket_1.rc rename to shark/src/test/hive/data/files/smbbucket_1.rc diff --git a/src/test/hive/data/files/smbbucket_1.txt b/shark/src/test/hive/data/files/smbbucket_1.txt similarity index 100% rename from src/test/hive/data/files/smbbucket_1.txt rename to shark/src/test/hive/data/files/smbbucket_1.txt diff --git a/src/test/hive/data/files/smbbucket_2.rc b/shark/src/test/hive/data/files/smbbucket_2.rc similarity index 100% rename from src/test/hive/data/files/smbbucket_2.rc rename to shark/src/test/hive/data/files/smbbucket_2.rc diff --git a/src/test/hive/data/files/smbbucket_2.txt b/shark/src/test/hive/data/files/smbbucket_2.txt similarity index 100% rename from src/test/hive/data/files/smbbucket_2.txt rename to shark/src/test/hive/data/files/smbbucket_2.txt diff --git a/src/test/hive/data/files/smbbucket_3.rc b/shark/src/test/hive/data/files/smbbucket_3.rc similarity index 100% rename from src/test/hive/data/files/smbbucket_3.rc rename to shark/src/test/hive/data/files/smbbucket_3.rc diff --git a/src/test/hive/data/files/smbbucket_3.txt b/shark/src/test/hive/data/files/smbbucket_3.txt similarity index 100% rename from src/test/hive/data/files/smbbucket_3.txt rename to shark/src/test/hive/data/files/smbbucket_3.txt diff --git a/src/test/hive/data/files/source.txt b/shark/src/test/hive/data/files/source.txt similarity index 100% rename from src/test/hive/data/files/source.txt rename to shark/src/test/hive/data/files/source.txt diff --git a/src/test/hive/data/files/srcbucket0.txt b/shark/src/test/hive/data/files/srcbucket0.txt similarity index 100% rename from src/test/hive/data/files/srcbucket0.txt rename to shark/src/test/hive/data/files/srcbucket0.txt diff --git a/src/test/hive/data/files/srcbucket1.txt b/shark/src/test/hive/data/files/srcbucket1.txt similarity index 100% rename from src/test/hive/data/files/srcbucket1.txt rename to shark/src/test/hive/data/files/srcbucket1.txt diff --git a/src/test/hive/data/files/srcbucket20.txt b/shark/src/test/hive/data/files/srcbucket20.txt similarity index 100% rename from src/test/hive/data/files/srcbucket20.txt rename to shark/src/test/hive/data/files/srcbucket20.txt diff --git a/src/test/hive/data/files/srcbucket21.txt b/shark/src/test/hive/data/files/srcbucket21.txt similarity index 100% rename from src/test/hive/data/files/srcbucket21.txt rename to shark/src/test/hive/data/files/srcbucket21.txt diff --git a/src/test/hive/data/files/srcbucket22.txt b/shark/src/test/hive/data/files/srcbucket22.txt similarity index 100% rename from src/test/hive/data/files/srcbucket22.txt rename to shark/src/test/hive/data/files/srcbucket22.txt diff --git a/src/test/hive/data/files/srcbucket23.txt b/shark/src/test/hive/data/files/srcbucket23.txt similarity index 100% rename from src/test/hive/data/files/srcbucket23.txt rename to shark/src/test/hive/data/files/srcbucket23.txt diff --git a/src/test/hive/data/files/srcsortbucket1outof4.txt b/shark/src/test/hive/data/files/srcsortbucket1outof4.txt similarity index 100% rename from src/test/hive/data/files/srcsortbucket1outof4.txt rename to shark/src/test/hive/data/files/srcsortbucket1outof4.txt diff --git a/src/test/hive/data/files/srcsortbucket2outof4.txt b/shark/src/test/hive/data/files/srcsortbucket2outof4.txt similarity index 100% rename from src/test/hive/data/files/srcsortbucket2outof4.txt rename to shark/src/test/hive/data/files/srcsortbucket2outof4.txt diff --git a/src/test/hive/data/files/srcsortbucket3outof4.txt b/shark/src/test/hive/data/files/srcsortbucket3outof4.txt similarity index 100% rename from src/test/hive/data/files/srcsortbucket3outof4.txt rename to shark/src/test/hive/data/files/srcsortbucket3outof4.txt diff --git a/src/test/hive/data/files/srcsortbucket4outof4.txt b/shark/src/test/hive/data/files/srcsortbucket4outof4.txt similarity index 100% rename from src/test/hive/data/files/srcsortbucket4outof4.txt rename to shark/src/test/hive/data/files/srcsortbucket4outof4.txt diff --git a/src/test/hive/data/files/string.txt b/shark/src/test/hive/data/files/string.txt similarity index 100% rename from src/test/hive/data/files/string.txt rename to shark/src/test/hive/data/files/string.txt diff --git a/src/test/hive/data/files/symlink1.txt b/shark/src/test/hive/data/files/symlink1.txt similarity index 100% rename from src/test/hive/data/files/symlink1.txt rename to shark/src/test/hive/data/files/symlink1.txt diff --git a/src/test/hive/data/files/symlink2.txt b/shark/src/test/hive/data/files/symlink2.txt similarity index 100% rename from src/test/hive/data/files/symlink2.txt rename to shark/src/test/hive/data/files/symlink2.txt diff --git a/src/test/hive/data/files/tbl.txt b/shark/src/test/hive/data/files/tbl.txt similarity index 100% rename from src/test/hive/data/files/tbl.txt rename to shark/src/test/hive/data/files/tbl.txt diff --git a/src/test/hive/data/files/test.dat b/shark/src/test/hive/data/files/test.dat similarity index 100% rename from src/test/hive/data/files/test.dat rename to shark/src/test/hive/data/files/test.dat diff --git a/src/test/hive/data/files/test2.dat b/shark/src/test/hive/data/files/test2.dat similarity index 100% rename from src/test/hive/data/files/test2.dat rename to shark/src/test/hive/data/files/test2.dat diff --git a/src/test/hive/data/files/text-en.txt b/shark/src/test/hive/data/files/text-en.txt similarity index 100% rename from src/test/hive/data/files/text-en.txt rename to shark/src/test/hive/data/files/text-en.txt diff --git a/src/test/hive/data/files/things.txt b/shark/src/test/hive/data/files/things.txt similarity index 100% rename from src/test/hive/data/files/things.txt rename to shark/src/test/hive/data/files/things.txt diff --git a/src/test/hive/data/files/things2.txt b/shark/src/test/hive/data/files/things2.txt similarity index 100% rename from src/test/hive/data/files/things2.txt rename to shark/src/test/hive/data/files/things2.txt diff --git a/src/test/hive/data/files/tiny_a.txt b/shark/src/test/hive/data/files/tiny_a.txt similarity index 100% rename from src/test/hive/data/files/tiny_a.txt rename to shark/src/test/hive/data/files/tiny_a.txt diff --git a/src/test/hive/data/files/tiny_b.txt b/shark/src/test/hive/data/files/tiny_b.txt similarity index 100% rename from src/test/hive/data/files/tiny_b.txt rename to shark/src/test/hive/data/files/tiny_b.txt diff --git a/src/test/hive/data/files/types/primitives/090101.txt b/shark/src/test/hive/data/files/types/primitives/090101.txt similarity index 100% rename from src/test/hive/data/files/types/primitives/090101.txt rename to shark/src/test/hive/data/files/types/primitives/090101.txt diff --git a/src/test/hive/data/files/types/primitives/090201.txt b/shark/src/test/hive/data/files/types/primitives/090201.txt similarity index 100% rename from src/test/hive/data/files/types/primitives/090201.txt rename to shark/src/test/hive/data/files/types/primitives/090201.txt diff --git a/src/test/hive/data/files/types/primitives/090301.txt b/shark/src/test/hive/data/files/types/primitives/090301.txt similarity index 100% rename from src/test/hive/data/files/types/primitives/090301.txt rename to shark/src/test/hive/data/files/types/primitives/090301.txt diff --git a/src/test/hive/data/files/types/primitives/090401.txt b/shark/src/test/hive/data/files/types/primitives/090401.txt similarity index 100% rename from src/test/hive/data/files/types/primitives/090401.txt rename to shark/src/test/hive/data/files/types/primitives/090401.txt diff --git a/src/test/hive/data/files/union_input.txt b/shark/src/test/hive/data/files/union_input.txt similarity index 100% rename from src/test/hive/data/files/union_input.txt rename to shark/src/test/hive/data/files/union_input.txt diff --git a/src/test/hive/data/files/v1.txt b/shark/src/test/hive/data/files/v1.txt similarity index 100% rename from src/test/hive/data/files/v1.txt rename to shark/src/test/hive/data/files/v1.txt diff --git a/src/test/hive/data/files/v2.txt b/shark/src/test/hive/data/files/v2.txt similarity index 100% rename from src/test/hive/data/files/v2.txt rename to shark/src/test/hive/data/files/v2.txt diff --git a/src/test/hive/data/files/vc1.txt b/shark/src/test/hive/data/files/vc1.txt similarity index 100% rename from src/test/hive/data/files/vc1.txt rename to shark/src/test/hive/data/files/vc1.txt diff --git a/src/test/hive/data/files/x.txt b/shark/src/test/hive/data/files/x.txt similarity index 100% rename from src/test/hive/data/files/x.txt rename to shark/src/test/hive/data/files/x.txt diff --git a/src/test/hive/data/files/y.txt b/shark/src/test/hive/data/files/y.txt similarity index 100% rename from src/test/hive/data/files/y.txt rename to shark/src/test/hive/data/files/y.txt diff --git a/src/test/hive/data/files/z.txt b/shark/src/test/hive/data/files/z.txt similarity index 100% rename from src/test/hive/data/files/z.txt rename to shark/src/test/hive/data/files/z.txt diff --git a/src/test/hive/data/metadb/.gitignore b/shark/src/test/hive/data/metadb/.gitignore similarity index 100% rename from src/test/hive/data/metadb/.gitignore rename to shark/src/test/hive/data/metadb/.gitignore diff --git a/src/test/hive/data/scripts/cat.py b/shark/src/test/hive/data/scripts/cat.py similarity index 100% rename from src/test/hive/data/scripts/cat.py rename to shark/src/test/hive/data/scripts/cat.py diff --git a/src/test/hive/data/scripts/cat_error.py b/shark/src/test/hive/data/scripts/cat_error.py similarity index 100% rename from src/test/hive/data/scripts/cat_error.py rename to shark/src/test/hive/data/scripts/cat_error.py diff --git a/src/test/hive/data/scripts/doubleescapedtab.py b/shark/src/test/hive/data/scripts/doubleescapedtab.py similarity index 100% rename from src/test/hive/data/scripts/doubleescapedtab.py rename to shark/src/test/hive/data/scripts/doubleescapedtab.py diff --git a/src/test/hive/data/scripts/dumpdata_script.py b/shark/src/test/hive/data/scripts/dumpdata_script.py similarity index 100% rename from src/test/hive/data/scripts/dumpdata_script.py rename to shark/src/test/hive/data/scripts/dumpdata_script.py diff --git a/src/test/hive/data/scripts/error_script b/shark/src/test/hive/data/scripts/error_script similarity index 100% rename from src/test/hive/data/scripts/error_script rename to shark/src/test/hive/data/scripts/error_script diff --git a/src/test/hive/data/scripts/escapedcarriagereturn.py b/shark/src/test/hive/data/scripts/escapedcarriagereturn.py similarity index 100% rename from src/test/hive/data/scripts/escapedcarriagereturn.py rename to shark/src/test/hive/data/scripts/escapedcarriagereturn.py diff --git a/src/test/hive/data/scripts/escapednewline.py b/shark/src/test/hive/data/scripts/escapednewline.py similarity index 100% rename from src/test/hive/data/scripts/escapednewline.py rename to shark/src/test/hive/data/scripts/escapednewline.py diff --git a/src/test/hive/data/scripts/escapedtab.py b/shark/src/test/hive/data/scripts/escapedtab.py similarity index 100% rename from src/test/hive/data/scripts/escapedtab.py rename to shark/src/test/hive/data/scripts/escapedtab.py diff --git a/src/test/hive/data/scripts/input20_script b/shark/src/test/hive/data/scripts/input20_script similarity index 100% rename from src/test/hive/data/scripts/input20_script rename to shark/src/test/hive/data/scripts/input20_script diff --git a/src/test/hive/data/scripts/newline.py b/shark/src/test/hive/data/scripts/newline.py similarity index 100% rename from src/test/hive/data/scripts/newline.py rename to shark/src/test/hive/data/scripts/newline.py diff --git a/src/test/hive/data/scripts/q_test_cleanup.sql b/shark/src/test/hive/data/scripts/q_test_cleanup.sql similarity index 100% rename from src/test/hive/data/scripts/q_test_cleanup.sql rename to shark/src/test/hive/data/scripts/q_test_cleanup.sql diff --git a/src/test/hive/data/scripts/q_test_init.sql b/shark/src/test/hive/data/scripts/q_test_init.sql similarity index 100% rename from src/test/hive/data/scripts/q_test_init.sql rename to shark/src/test/hive/data/scripts/q_test_init.sql diff --git a/src/test/hive/data/scripts/test_init_file.sql b/shark/src/test/hive/data/scripts/test_init_file.sql similarity index 100% rename from src/test/hive/data/scripts/test_init_file.sql rename to shark/src/test/hive/data/scripts/test_init_file.sql diff --git a/src/test/hive/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb b/shark/src/test/hive/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb similarity index 100% rename from src/test/hive/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb rename to shark/src/test/hive/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb diff --git a/src/test/hive/golden/'1' + 1-0-130514c6116c311d808590a075b187b b/shark/src/test/hive/golden/'1' + 1-0-130514c6116c311d808590a075b187b similarity index 100% rename from src/test/hive/golden/'1' + 1-0-130514c6116c311d808590a075b187b rename to shark/src/test/hive/golden/'1' + 1-0-130514c6116c311d808590a075b187b diff --git a/src/test/hive/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e b/shark/src/test/hive/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e similarity index 100% rename from src/test/hive/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e rename to shark/src/test/hive/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e diff --git a/src/test/hive/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 b/shark/src/test/hive/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 similarity index 100% rename from src/test/hive/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 rename to shark/src/test/hive/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 diff --git a/src/test/hive/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 b/shark/src/test/hive/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 similarity index 100% rename from src/test/hive/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 rename to shark/src/test/hive/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 diff --git a/src/test/hive/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a b/shark/src/test/hive/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a similarity index 100% rename from src/test/hive/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a rename to shark/src/test/hive/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a diff --git a/src/test/hive/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a b/shark/src/test/hive/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a similarity index 100% rename from src/test/hive/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a rename to shark/src/test/hive/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a diff --git a/src/test/hive/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 b/shark/src/test/hive/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 similarity index 100% rename from src/test/hive/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 rename to shark/src/test/hive/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 diff --git a/src/test/hive/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 b/shark/src/test/hive/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 similarity index 100% rename from src/test/hive/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 rename to shark/src/test/hive/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 diff --git a/src/test/hive/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e b/shark/src/test/hive/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e similarity index 100% rename from src/test/hive/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e rename to shark/src/test/hive/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e diff --git a/src/test/hive/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e b/shark/src/test/hive/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e similarity index 100% rename from src/test/hive/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e rename to shark/src/test/hive/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e diff --git a/src/test/hive/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f b/shark/src/test/hive/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f similarity index 100% rename from src/test/hive/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f rename to shark/src/test/hive/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f diff --git a/src/test/hive/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 b/shark/src/test/hive/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 similarity index 100% rename from src/test/hive/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 rename to shark/src/test/hive/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 diff --git a/src/test/hive/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 b/shark/src/test/hive/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 similarity index 100% rename from src/test/hive/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 rename to shark/src/test/hive/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 diff --git a/src/test/hive/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d b/shark/src/test/hive/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d similarity index 100% rename from src/test/hive/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d rename to shark/src/test/hive/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d diff --git a/src/test/hive/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 b/shark/src/test/hive/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 similarity index 100% rename from src/test/hive/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 rename to shark/src/test/hive/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 diff --git a/src/test/hive/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 b/shark/src/test/hive/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 similarity index 100% rename from src/test/hive/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 rename to shark/src/test/hive/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 diff --git a/src/test/hive/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 b/shark/src/test/hive/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 similarity index 100% rename from src/test/hive/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 rename to shark/src/test/hive/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 diff --git a/src/test/hive/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a b/shark/src/test/hive/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a similarity index 100% rename from src/test/hive/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a rename to shark/src/test/hive/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a diff --git a/src/test/hive/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d b/shark/src/test/hive/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d similarity index 100% rename from src/test/hive/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d rename to shark/src/test/hive/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d diff --git a/src/test/hive/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 b/shark/src/test/hive/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 similarity index 100% rename from src/test/hive/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 rename to shark/src/test/hive/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 diff --git a/src/test/hive/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f b/shark/src/test/hive/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f similarity index 100% rename from src/test/hive/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f rename to shark/src/test/hive/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f diff --git a/src/test/hive/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 b/shark/src/test/hive/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 similarity index 100% rename from src/test/hive/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 rename to shark/src/test/hive/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 diff --git a/src/test/hive/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 b/shark/src/test/hive/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 similarity index 100% rename from src/test/hive/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 rename to shark/src/test/hive/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 diff --git a/src/test/hive/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 b/shark/src/test/hive/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 similarity index 100% rename from src/test/hive/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 rename to shark/src/test/hive/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 diff --git a/src/test/hive/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 b/shark/src/test/hive/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 similarity index 100% rename from src/test/hive/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 rename to shark/src/test/hive/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 diff --git a/src/test/hive/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e b/shark/src/test/hive/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e similarity index 100% rename from src/test/hive/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e rename to shark/src/test/hive/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e diff --git a/src/test/hive/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 b/shark/src/test/hive/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 similarity index 100% rename from src/test/hive/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 rename to shark/src/test/hive/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 diff --git a/src/test/hive/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e b/shark/src/test/hive/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e similarity index 100% rename from src/test/hive/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e rename to shark/src/test/hive/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e diff --git a/src/test/hive/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b b/shark/src/test/hive/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b similarity index 100% rename from src/test/hive/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b rename to shark/src/test/hive/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b diff --git a/src/test/hive/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb b/shark/src/test/hive/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb similarity index 100% rename from src/test/hive/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb rename to shark/src/test/hive/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb diff --git a/src/test/hive/golden/1Y + 1-0-a4541db51882b19503649138fbb295f b/shark/src/test/hive/golden/1Y + 1-0-a4541db51882b19503649138fbb295f similarity index 100% rename from src/test/hive/golden/1Y + 1-0-a4541db51882b19503649138fbb295f rename to shark/src/test/hive/golden/1Y + 1-0-a4541db51882b19503649138fbb295f diff --git a/src/test/hive/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d b/shark/src/test/hive/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d similarity index 100% rename from src/test/hive/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d rename to shark/src/test/hive/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d diff --git a/src/test/hive/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 b/shark/src/test/hive/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 similarity index 100% rename from src/test/hive/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 rename to shark/src/test/hive/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 diff --git a/src/test/hive/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 b/shark/src/test/hive/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 similarity index 100% rename from src/test/hive/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 rename to shark/src/test/hive/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 diff --git a/src/test/hive/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 b/shark/src/test/hive/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 similarity index 100% rename from src/test/hive/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 rename to shark/src/test/hive/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 diff --git a/src/test/hive/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a b/shark/src/test/hive/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a similarity index 100% rename from src/test/hive/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a rename to shark/src/test/hive/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a diff --git a/src/test/hive/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 b/shark/src/test/hive/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 similarity index 100% rename from src/test/hive/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 rename to shark/src/test/hive/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 diff --git a/src/test/hive/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 b/shark/src/test/hive/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 similarity index 100% rename from src/test/hive/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 rename to shark/src/test/hive/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 diff --git a/src/test/hive/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 b/shark/src/test/hive/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 similarity index 100% rename from src/test/hive/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 rename to shark/src/test/hive/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 diff --git a/src/test/hive/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 b/shark/src/test/hive/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 similarity index 100% rename from src/test/hive/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 rename to shark/src/test/hive/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 diff --git a/src/test/hive/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 b/shark/src/test/hive/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 similarity index 100% rename from src/test/hive/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 rename to shark/src/test/hive/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 diff --git a/src/test/hive/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa b/shark/src/test/hive/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa similarity index 100% rename from src/test/hive/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa rename to shark/src/test/hive/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa diff --git a/src/test/hive/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 b/shark/src/test/hive/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 similarity index 100% rename from src/test/hive/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 rename to shark/src/test/hive/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 diff --git a/src/test/hive/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 b/shark/src/test/hive/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 similarity index 100% rename from src/test/hive/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 rename to shark/src/test/hive/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 diff --git a/src/test/hive/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 b/shark/src/test/hive/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 similarity index 100% rename from src/test/hive/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 rename to shark/src/test/hive/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 diff --git a/src/test/hive/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 b/shark/src/test/hive/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 similarity index 100% rename from src/test/hive/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 rename to shark/src/test/hive/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 diff --git a/src/test/hive/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b b/shark/src/test/hive/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b similarity index 100% rename from src/test/hive/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b rename to shark/src/test/hive/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b diff --git a/src/test/hive/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 b/shark/src/test/hive/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 similarity index 100% rename from src/test/hive/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 rename to shark/src/test/hive/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 diff --git a/src/test/hive/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 b/shark/src/test/hive/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 similarity index 100% rename from src/test/hive/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 rename to shark/src/test/hive/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 diff --git a/src/test/hive/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 b/shark/src/test/hive/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 similarity index 100% rename from src/test/hive/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 rename to shark/src/test/hive/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 diff --git a/src/test/hive/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 b/shark/src/test/hive/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 similarity index 100% rename from src/test/hive/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 rename to shark/src/test/hive/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 diff --git a/src/test/hive/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b b/shark/src/test/hive/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b similarity index 100% rename from src/test/hive/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b rename to shark/src/test/hive/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b diff --git a/src/test/hive/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b b/shark/src/test/hive/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b similarity index 100% rename from src/test/hive/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b rename to shark/src/test/hive/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b diff --git a/src/test/hive/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b b/shark/src/test/hive/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b similarity index 100% rename from src/test/hive/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b rename to shark/src/test/hive/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b diff --git a/src/test/hive/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 b/shark/src/test/hive/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 similarity index 100% rename from src/test/hive/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 rename to shark/src/test/hive/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 diff --git a/src/test/hive/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b b/shark/src/test/hive/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b similarity index 100% rename from src/test/hive/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b rename to shark/src/test/hive/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b diff --git a/src/test/hive/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b b/shark/src/test/hive/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b similarity index 100% rename from src/test/hive/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b rename to shark/src/test/hive/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b diff --git a/src/test/hive/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f b/shark/src/test/hive/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f similarity index 100% rename from src/test/hive/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f rename to shark/src/test/hive/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f diff --git a/src/test/hive/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b b/shark/src/test/hive/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b similarity index 100% rename from src/test/hive/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b rename to shark/src/test/hive/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b diff --git a/src/test/hive/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb b/shark/src/test/hive/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb similarity index 100% rename from src/test/hive/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb rename to shark/src/test/hive/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb diff --git a/src/test/hive/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b b/shark/src/test/hive/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b similarity index 100% rename from src/test/hive/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b rename to shark/src/test/hive/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b diff --git a/src/test/hive/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b b/shark/src/test/hive/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b similarity index 100% rename from src/test/hive/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b rename to shark/src/test/hive/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b diff --git a/src/test/hive/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 b/shark/src/test/hive/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 similarity index 100% rename from src/test/hive/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 rename to shark/src/test/hive/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 diff --git a/src/test/hive/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b b/shark/src/test/hive/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b similarity index 100% rename from src/test/hive/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b rename to shark/src/test/hive/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b diff --git a/src/test/hive/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f b/shark/src/test/hive/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f similarity index 100% rename from src/test/hive/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f rename to shark/src/test/hive/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f diff --git a/src/test/hive/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b b/shark/src/test/hive/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b similarity index 100% rename from src/test/hive/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b rename to shark/src/test/hive/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b diff --git a/src/test/hive/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb b/shark/src/test/hive/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb similarity index 100% rename from src/test/hive/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb rename to shark/src/test/hive/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb diff --git a/src/test/hive/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b b/shark/src/test/hive/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b similarity index 100% rename from src/test/hive/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b rename to shark/src/test/hive/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b diff --git a/src/test/hive/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 b/shark/src/test/hive/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 similarity index 100% rename from src/test/hive/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 rename to shark/src/test/hive/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 diff --git a/src/test/hive/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 b/shark/src/test/hive/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 similarity index 100% rename from src/test/hive/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 rename to shark/src/test/hive/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 diff --git a/src/test/hive/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 b/shark/src/test/hive/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 similarity index 100% rename from src/test/hive/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 rename to shark/src/test/hive/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 diff --git a/src/test/hive/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef b/shark/src/test/hive/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef similarity index 100% rename from src/test/hive/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef rename to shark/src/test/hive/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef diff --git a/src/test/hive/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c b/shark/src/test/hive/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c similarity index 100% rename from src/test/hive/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c rename to shark/src/test/hive/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c diff --git a/src/test/hive/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 b/shark/src/test/hive/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 similarity index 100% rename from src/test/hive/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 rename to shark/src/test/hive/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 diff --git a/src/test/hive/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af b/shark/src/test/hive/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af similarity index 100% rename from src/test/hive/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af rename to shark/src/test/hive/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af diff --git a/src/test/hive/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 b/shark/src/test/hive/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 similarity index 100% rename from src/test/hive/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 rename to shark/src/test/hive/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 diff --git a/src/test/hive/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f b/shark/src/test/hive/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f similarity index 100% rename from src/test/hive/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f rename to shark/src/test/hive/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f diff --git a/src/test/hive/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 b/shark/src/test/hive/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 similarity index 100% rename from src/test/hive/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 rename to shark/src/test/hive/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 diff --git a/src/test/hive/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 b/shark/src/test/hive/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 similarity index 100% rename from src/test/hive/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 rename to shark/src/test/hive/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 diff --git a/src/test/hive/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 b/shark/src/test/hive/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 similarity index 100% rename from src/test/hive/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 rename to shark/src/test/hive/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 diff --git a/src/test/hive/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 b/shark/src/test/hive/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 similarity index 100% rename from src/test/hive/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 rename to shark/src/test/hive/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 diff --git a/src/test/hive/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d b/shark/src/test/hive/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d similarity index 100% rename from src/test/hive/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d rename to shark/src/test/hive/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d diff --git a/src/test/hive/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 b/shark/src/test/hive/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 similarity index 100% rename from src/test/hive/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 rename to shark/src/test/hive/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 diff --git a/src/test/hive/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 b/shark/src/test/hive/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 similarity index 100% rename from src/test/hive/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 rename to shark/src/test/hive/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 diff --git a/src/test/hive/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 b/shark/src/test/hive/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 similarity index 100% rename from src/test/hive/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 rename to shark/src/test/hive/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 diff --git a/src/test/hive/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 b/shark/src/test/hive/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 similarity index 100% rename from src/test/hive/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 rename to shark/src/test/hive/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 diff --git a/src/test/hive/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 b/shark/src/test/hive/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 similarity index 100% rename from src/test/hive/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 rename to shark/src/test/hive/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 diff --git a/src/test/hive/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da b/shark/src/test/hive/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da similarity index 100% rename from src/test/hive/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da rename to shark/src/test/hive/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da diff --git a/src/test/hive/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 b/shark/src/test/hive/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 similarity index 100% rename from src/test/hive/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 rename to shark/src/test/hive/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 diff --git a/src/test/hive/golden/alter2-12-75a213649242c2410ea6846f08c91d75 b/shark/src/test/hive/golden/alter2-12-75a213649242c2410ea6846f08c91d75 similarity index 100% rename from src/test/hive/golden/alter2-12-75a213649242c2410ea6846f08c91d75 rename to shark/src/test/hive/golden/alter2-12-75a213649242c2410ea6846f08c91d75 diff --git a/src/test/hive/golden/alter2-13-ca51e03a3de391983429b6ad877e573c b/shark/src/test/hive/golden/alter2-13-ca51e03a3de391983429b6ad877e573c similarity index 100% rename from src/test/hive/golden/alter2-13-ca51e03a3de391983429b6ad877e573c rename to shark/src/test/hive/golden/alter2-13-ca51e03a3de391983429b6ad877e573c diff --git a/src/test/hive/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 b/shark/src/test/hive/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 similarity index 100% rename from src/test/hive/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 rename to shark/src/test/hive/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 diff --git a/src/test/hive/golden/alter2-15-75a213649242c2410ea6846f08c91d75 b/shark/src/test/hive/golden/alter2-15-75a213649242c2410ea6846f08c91d75 similarity index 100% rename from src/test/hive/golden/alter2-15-75a213649242c2410ea6846f08c91d75 rename to shark/src/test/hive/golden/alter2-15-75a213649242c2410ea6846f08c91d75 diff --git a/src/test/hive/golden/alter2-16-77500c83ffcece95511a4b21d67382dd b/shark/src/test/hive/golden/alter2-16-77500c83ffcece95511a4b21d67382dd similarity index 100% rename from src/test/hive/golden/alter2-16-77500c83ffcece95511a4b21d67382dd rename to shark/src/test/hive/golden/alter2-16-77500c83ffcece95511a4b21d67382dd diff --git a/src/test/hive/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 b/shark/src/test/hive/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 similarity index 100% rename from src/test/hive/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 rename to shark/src/test/hive/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 diff --git a/src/test/hive/golden/alter2-18-75a213649242c2410ea6846f08c91d75 b/shark/src/test/hive/golden/alter2-18-75a213649242c2410ea6846f08c91d75 similarity index 100% rename from src/test/hive/golden/alter2-18-75a213649242c2410ea6846f08c91d75 rename to shark/src/test/hive/golden/alter2-18-75a213649242c2410ea6846f08c91d75 diff --git a/src/test/hive/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 b/shark/src/test/hive/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 similarity index 100% rename from src/test/hive/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 rename to shark/src/test/hive/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 diff --git a/src/test/hive/golden/alter2-2-75a213649242c2410ea6846f08c91d75 b/shark/src/test/hive/golden/alter2-2-75a213649242c2410ea6846f08c91d75 similarity index 100% rename from src/test/hive/golden/alter2-2-75a213649242c2410ea6846f08c91d75 rename to shark/src/test/hive/golden/alter2-2-75a213649242c2410ea6846f08c91d75 diff --git a/src/test/hive/golden/alter2-20-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter2-20-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter2-20-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter2-20-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 b/shark/src/test/hive/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 similarity index 100% rename from src/test/hive/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 rename to shark/src/test/hive/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 diff --git a/src/test/hive/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 b/shark/src/test/hive/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 similarity index 100% rename from src/test/hive/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 rename to shark/src/test/hive/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 diff --git a/src/test/hive/golden/alter2-23-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter2-23-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter2-23-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter2-23-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter2-24-775d69742a1c07df8da87e8a017d955 b/shark/src/test/hive/golden/alter2-24-775d69742a1c07df8da87e8a017d955 similarity index 100% rename from src/test/hive/golden/alter2-24-775d69742a1c07df8da87e8a017d955 rename to shark/src/test/hive/golden/alter2-24-775d69742a1c07df8da87e8a017d955 diff --git a/src/test/hive/golden/alter2-25-4ef75e12575453225738ea167c4617e5 b/shark/src/test/hive/golden/alter2-25-4ef75e12575453225738ea167c4617e5 similarity index 100% rename from src/test/hive/golden/alter2-25-4ef75e12575453225738ea167c4617e5 rename to shark/src/test/hive/golden/alter2-25-4ef75e12575453225738ea167c4617e5 diff --git a/src/test/hive/golden/alter2-26-63f13c364546ddce5d2176c6604a948f b/shark/src/test/hive/golden/alter2-26-63f13c364546ddce5d2176c6604a948f similarity index 100% rename from src/test/hive/golden/alter2-26-63f13c364546ddce5d2176c6604a948f rename to shark/src/test/hive/golden/alter2-26-63f13c364546ddce5d2176c6604a948f diff --git a/src/test/hive/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f b/shark/src/test/hive/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f similarity index 100% rename from src/test/hive/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f rename to shark/src/test/hive/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f diff --git a/src/test/hive/golden/alter2-28-4ef75e12575453225738ea167c4617e5 b/shark/src/test/hive/golden/alter2-28-4ef75e12575453225738ea167c4617e5 similarity index 100% rename from src/test/hive/golden/alter2-28-4ef75e12575453225738ea167c4617e5 rename to shark/src/test/hive/golden/alter2-28-4ef75e12575453225738ea167c4617e5 diff --git a/src/test/hive/golden/alter2-29-63f13c364546ddce5d2176c6604a948f b/shark/src/test/hive/golden/alter2-29-63f13c364546ddce5d2176c6604a948f similarity index 100% rename from src/test/hive/golden/alter2-29-63f13c364546ddce5d2176c6604a948f rename to shark/src/test/hive/golden/alter2-29-63f13c364546ddce5d2176c6604a948f diff --git a/src/test/hive/golden/alter2-3-ca51e03a3de391983429b6ad877e573c b/shark/src/test/hive/golden/alter2-3-ca51e03a3de391983429b6ad877e573c similarity index 100% rename from src/test/hive/golden/alter2-3-ca51e03a3de391983429b6ad877e573c rename to shark/src/test/hive/golden/alter2-3-ca51e03a3de391983429b6ad877e573c diff --git a/src/test/hive/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae b/shark/src/test/hive/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae similarity index 100% rename from src/test/hive/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae rename to shark/src/test/hive/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae diff --git a/src/test/hive/golden/alter2-31-4ef75e12575453225738ea167c4617e5 b/shark/src/test/hive/golden/alter2-31-4ef75e12575453225738ea167c4617e5 similarity index 100% rename from src/test/hive/golden/alter2-31-4ef75e12575453225738ea167c4617e5 rename to shark/src/test/hive/golden/alter2-31-4ef75e12575453225738ea167c4617e5 diff --git a/src/test/hive/golden/alter2-32-63f13c364546ddce5d2176c6604a948f b/shark/src/test/hive/golden/alter2-32-63f13c364546ddce5d2176c6604a948f similarity index 100% rename from src/test/hive/golden/alter2-32-63f13c364546ddce5d2176c6604a948f rename to shark/src/test/hive/golden/alter2-32-63f13c364546ddce5d2176c6604a948f diff --git a/src/test/hive/golden/alter2-33-1934026d0228967097280eed35551f74 b/shark/src/test/hive/golden/alter2-33-1934026d0228967097280eed35551f74 similarity index 100% rename from src/test/hive/golden/alter2-33-1934026d0228967097280eed35551f74 rename to shark/src/test/hive/golden/alter2-33-1934026d0228967097280eed35551f74 diff --git a/src/test/hive/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b b/shark/src/test/hive/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b similarity index 100% rename from src/test/hive/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b rename to shark/src/test/hive/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b diff --git a/src/test/hive/golden/alter2-35-4ef75e12575453225738ea167c4617e5 b/shark/src/test/hive/golden/alter2-35-4ef75e12575453225738ea167c4617e5 similarity index 100% rename from src/test/hive/golden/alter2-35-4ef75e12575453225738ea167c4617e5 rename to shark/src/test/hive/golden/alter2-35-4ef75e12575453225738ea167c4617e5 diff --git a/src/test/hive/golden/alter2-36-63f13c364546ddce5d2176c6604a948f b/shark/src/test/hive/golden/alter2-36-63f13c364546ddce5d2176c6604a948f similarity index 100% rename from src/test/hive/golden/alter2-36-63f13c364546ddce5d2176c6604a948f rename to shark/src/test/hive/golden/alter2-36-63f13c364546ddce5d2176c6604a948f diff --git a/src/test/hive/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f b/shark/src/test/hive/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f similarity index 100% rename from src/test/hive/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f rename to shark/src/test/hive/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f diff --git a/src/test/hive/golden/alter2-38-4ef75e12575453225738ea167c4617e5 b/shark/src/test/hive/golden/alter2-38-4ef75e12575453225738ea167c4617e5 similarity index 100% rename from src/test/hive/golden/alter2-38-4ef75e12575453225738ea167c4617e5 rename to shark/src/test/hive/golden/alter2-38-4ef75e12575453225738ea167c4617e5 diff --git a/src/test/hive/golden/alter2-39-63f13c364546ddce5d2176c6604a948f b/shark/src/test/hive/golden/alter2-39-63f13c364546ddce5d2176c6604a948f similarity index 100% rename from src/test/hive/golden/alter2-39-63f13c364546ddce5d2176c6604a948f rename to shark/src/test/hive/golden/alter2-39-63f13c364546ddce5d2176c6604a948f diff --git a/src/test/hive/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 b/shark/src/test/hive/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 similarity index 100% rename from src/test/hive/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 rename to shark/src/test/hive/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 diff --git a/src/test/hive/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae b/shark/src/test/hive/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae similarity index 100% rename from src/test/hive/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae rename to shark/src/test/hive/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae diff --git a/src/test/hive/golden/alter2-41-4ef75e12575453225738ea167c4617e5 b/shark/src/test/hive/golden/alter2-41-4ef75e12575453225738ea167c4617e5 similarity index 100% rename from src/test/hive/golden/alter2-41-4ef75e12575453225738ea167c4617e5 rename to shark/src/test/hive/golden/alter2-41-4ef75e12575453225738ea167c4617e5 diff --git a/src/test/hive/golden/alter2-42-63f13c364546ddce5d2176c6604a948f b/shark/src/test/hive/golden/alter2-42-63f13c364546ddce5d2176c6604a948f similarity index 100% rename from src/test/hive/golden/alter2-42-63f13c364546ddce5d2176c6604a948f rename to shark/src/test/hive/golden/alter2-42-63f13c364546ddce5d2176c6604a948f diff --git a/src/test/hive/golden/alter2-43-1934026d0228967097280eed35551f74 b/shark/src/test/hive/golden/alter2-43-1934026d0228967097280eed35551f74 similarity index 100% rename from src/test/hive/golden/alter2-43-1934026d0228967097280eed35551f74 rename to shark/src/test/hive/golden/alter2-43-1934026d0228967097280eed35551f74 diff --git a/src/test/hive/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 b/shark/src/test/hive/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 similarity index 100% rename from src/test/hive/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 rename to shark/src/test/hive/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 diff --git a/src/test/hive/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c b/shark/src/test/hive/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c similarity index 100% rename from src/test/hive/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c rename to shark/src/test/hive/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c diff --git a/src/test/hive/golden/alter2-5-75a213649242c2410ea6846f08c91d75 b/shark/src/test/hive/golden/alter2-5-75a213649242c2410ea6846f08c91d75 similarity index 100% rename from src/test/hive/golden/alter2-5-75a213649242c2410ea6846f08c91d75 rename to shark/src/test/hive/golden/alter2-5-75a213649242c2410ea6846f08c91d75 diff --git a/src/test/hive/golden/alter2-6-77500c83ffcece95511a4b21d67382dd b/shark/src/test/hive/golden/alter2-6-77500c83ffcece95511a4b21d67382dd similarity index 100% rename from src/test/hive/golden/alter2-6-77500c83ffcece95511a4b21d67382dd rename to shark/src/test/hive/golden/alter2-6-77500c83ffcece95511a4b21d67382dd diff --git a/src/test/hive/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 b/shark/src/test/hive/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 similarity index 100% rename from src/test/hive/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 rename to shark/src/test/hive/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 diff --git a/src/test/hive/golden/alter2-8-75a213649242c2410ea6846f08c91d75 b/shark/src/test/hive/golden/alter2-8-75a213649242c2410ea6846f08c91d75 similarity index 100% rename from src/test/hive/golden/alter2-8-75a213649242c2410ea6846f08c91d75 rename to shark/src/test/hive/golden/alter2-8-75a213649242c2410ea6846f08c91d75 diff --git a/src/test/hive/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e b/shark/src/test/hive/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e similarity index 100% rename from src/test/hive/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e rename to shark/src/test/hive/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e diff --git a/src/test/hive/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade b/shark/src/test/hive/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade similarity index 100% rename from src/test/hive/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade rename to shark/src/test/hive/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade diff --git a/src/test/hive/golden/alter3-1-75be487df30e301e156a22eee075633d b/shark/src/test/hive/golden/alter3-1-75be487df30e301e156a22eee075633d similarity index 100% rename from src/test/hive/golden/alter3-1-75be487df30e301e156a22eee075633d rename to shark/src/test/hive/golden/alter3-1-75be487df30e301e156a22eee075633d diff --git a/src/test/hive/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 b/shark/src/test/hive/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 similarity index 100% rename from src/test/hive/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 rename to shark/src/test/hive/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 diff --git a/src/test/hive/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 b/shark/src/test/hive/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 similarity index 100% rename from src/test/hive/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 rename to shark/src/test/hive/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 diff --git a/src/test/hive/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 b/shark/src/test/hive/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 similarity index 100% rename from src/test/hive/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 rename to shark/src/test/hive/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 diff --git a/src/test/hive/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b b/shark/src/test/hive/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b similarity index 100% rename from src/test/hive/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b rename to shark/src/test/hive/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b diff --git a/src/test/hive/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f b/shark/src/test/hive/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f similarity index 100% rename from src/test/hive/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f rename to shark/src/test/hive/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f diff --git a/src/test/hive/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 b/shark/src/test/hive/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 similarity index 100% rename from src/test/hive/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 rename to shark/src/test/hive/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 diff --git a/src/test/hive/golden/alter3-16-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter3-16-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter3-16-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter3-16-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b b/shark/src/test/hive/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b similarity index 100% rename from src/test/hive/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b rename to shark/src/test/hive/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b diff --git a/src/test/hive/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d b/shark/src/test/hive/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d similarity index 100% rename from src/test/hive/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d rename to shark/src/test/hive/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d diff --git a/src/test/hive/golden/alter3-19-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter3-19-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter3-19-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter3-19-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f b/shark/src/test/hive/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f similarity index 100% rename from src/test/hive/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f rename to shark/src/test/hive/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f diff --git a/src/test/hive/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 b/shark/src/test/hive/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 similarity index 100% rename from src/test/hive/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 rename to shark/src/test/hive/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 diff --git a/src/test/hive/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 b/shark/src/test/hive/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 similarity index 100% rename from src/test/hive/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 rename to shark/src/test/hive/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 diff --git a/src/test/hive/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 b/shark/src/test/hive/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 similarity index 100% rename from src/test/hive/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 rename to shark/src/test/hive/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 diff --git a/src/test/hive/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 b/shark/src/test/hive/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 similarity index 100% rename from src/test/hive/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 rename to shark/src/test/hive/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 diff --git a/src/test/hive/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 b/shark/src/test/hive/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 similarity index 100% rename from src/test/hive/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 rename to shark/src/test/hive/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 diff --git a/src/test/hive/golden/alter3-25-568a59760e5d3241b63d65cce595face b/shark/src/test/hive/golden/alter3-25-568a59760e5d3241b63d65cce595face similarity index 100% rename from src/test/hive/golden/alter3-25-568a59760e5d3241b63d65cce595face rename to shark/src/test/hive/golden/alter3-25-568a59760e5d3241b63d65cce595face diff --git a/src/test/hive/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e b/shark/src/test/hive/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e similarity index 100% rename from src/test/hive/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e rename to shark/src/test/hive/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e diff --git a/src/test/hive/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 b/shark/src/test/hive/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 similarity index 100% rename from src/test/hive/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 rename to shark/src/test/hive/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 diff --git a/src/test/hive/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 b/shark/src/test/hive/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 similarity index 100% rename from src/test/hive/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 rename to shark/src/test/hive/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 diff --git a/src/test/hive/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c b/shark/src/test/hive/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c similarity index 100% rename from src/test/hive/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c rename to shark/src/test/hive/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c diff --git a/src/test/hive/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 b/shark/src/test/hive/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 similarity index 100% rename from src/test/hive/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 rename to shark/src/test/hive/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 diff --git a/src/test/hive/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 b/shark/src/test/hive/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 similarity index 100% rename from src/test/hive/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 rename to shark/src/test/hive/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 diff --git a/src/test/hive/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 b/shark/src/test/hive/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 similarity index 100% rename from src/test/hive/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 rename to shark/src/test/hive/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 diff --git a/src/test/hive/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 b/shark/src/test/hive/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 similarity index 100% rename from src/test/hive/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 rename to shark/src/test/hive/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 diff --git a/src/test/hive/golden/alter3-4-399fc26b344c98ababa104522601c0cc b/shark/src/test/hive/golden/alter3-4-399fc26b344c98ababa104522601c0cc similarity index 100% rename from src/test/hive/golden/alter3-4-399fc26b344c98ababa104522601c0cc rename to shark/src/test/hive/golden/alter3-4-399fc26b344c98ababa104522601c0cc diff --git a/src/test/hive/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 b/shark/src/test/hive/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 similarity index 100% rename from src/test/hive/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 rename to shark/src/test/hive/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 diff --git a/src/test/hive/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 b/shark/src/test/hive/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 similarity index 100% rename from src/test/hive/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 rename to shark/src/test/hive/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 diff --git a/src/test/hive/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 b/shark/src/test/hive/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 similarity index 100% rename from src/test/hive/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 rename to shark/src/test/hive/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 diff --git a/src/test/hive/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 b/shark/src/test/hive/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 similarity index 100% rename from src/test/hive/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 rename to shark/src/test/hive/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 diff --git a/src/test/hive/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 b/shark/src/test/hive/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 similarity index 100% rename from src/test/hive/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 rename to shark/src/test/hive/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 diff --git a/src/test/hive/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 b/shark/src/test/hive/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 similarity index 100% rename from src/test/hive/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 rename to shark/src/test/hive/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 diff --git a/src/test/hive/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 b/shark/src/test/hive/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 similarity index 100% rename from src/test/hive/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 rename to shark/src/test/hive/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 diff --git a/src/test/hive/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 b/shark/src/test/hive/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 similarity index 100% rename from src/test/hive/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 rename to shark/src/test/hive/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 diff --git a/src/test/hive/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d b/shark/src/test/hive/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d similarity index 100% rename from src/test/hive/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d rename to shark/src/test/hive/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d diff --git a/src/test/hive/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 b/shark/src/test/hive/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 similarity index 100% rename from src/test/hive/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 rename to shark/src/test/hive/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 diff --git a/src/test/hive/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f b/shark/src/test/hive/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f similarity index 100% rename from src/test/hive/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f rename to shark/src/test/hive/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f diff --git a/src/test/hive/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 b/shark/src/test/hive/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 similarity index 100% rename from src/test/hive/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 rename to shark/src/test/hive/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 diff --git a/src/test/hive/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 b/shark/src/test/hive/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 similarity index 100% rename from src/test/hive/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 rename to shark/src/test/hive/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 diff --git a/src/test/hive/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 b/shark/src/test/hive/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 similarity index 100% rename from src/test/hive/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 rename to shark/src/test/hive/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 diff --git a/src/test/hive/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d b/shark/src/test/hive/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d similarity index 100% rename from src/test/hive/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d rename to shark/src/test/hive/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d diff --git a/src/test/hive/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 b/shark/src/test/hive/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 similarity index 100% rename from src/test/hive/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 rename to shark/src/test/hive/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 diff --git a/src/test/hive/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 b/shark/src/test/hive/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 similarity index 100% rename from src/test/hive/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 rename to shark/src/test/hive/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 diff --git a/src/test/hive/golden/alter4-5-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter4-5-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter4-5-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter4-5-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter4-6-acd58e84952d310aeddf78579c36286 b/shark/src/test/hive/golden/alter4-6-acd58e84952d310aeddf78579c36286 similarity index 100% rename from src/test/hive/golden/alter4-6-acd58e84952d310aeddf78579c36286 rename to shark/src/test/hive/golden/alter4-6-acd58e84952d310aeddf78579c36286 diff --git a/src/test/hive/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 b/shark/src/test/hive/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 similarity index 100% rename from src/test/hive/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 rename to shark/src/test/hive/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 diff --git a/src/test/hive/golden/alter4-8-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter4-8-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter4-8-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter4-8-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 b/shark/src/test/hive/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 similarity index 100% rename from src/test/hive/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 rename to shark/src/test/hive/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 diff --git a/src/test/hive/golden/alter5-0-953553e14d835682fa47338dcfffe227 b/shark/src/test/hive/golden/alter5-0-953553e14d835682fa47338dcfffe227 similarity index 100% rename from src/test/hive/golden/alter5-0-953553e14d835682fa47338dcfffe227 rename to shark/src/test/hive/golden/alter5-0-953553e14d835682fa47338dcfffe227 diff --git a/src/test/hive/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b b/shark/src/test/hive/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b similarity index 100% rename from src/test/hive/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b rename to shark/src/test/hive/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b diff --git a/src/test/hive/golden/alter5-10-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter5-10-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter5-10-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter5-10-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 b/shark/src/test/hive/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 similarity index 100% rename from src/test/hive/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 rename to shark/src/test/hive/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 diff --git a/src/test/hive/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 b/shark/src/test/hive/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 similarity index 100% rename from src/test/hive/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 rename to shark/src/test/hive/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 diff --git a/src/test/hive/golden/alter5-13-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter5-13-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter5-13-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter5-13-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 b/shark/src/test/hive/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 similarity index 100% rename from src/test/hive/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 rename to shark/src/test/hive/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 diff --git a/src/test/hive/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b b/shark/src/test/hive/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b similarity index 100% rename from src/test/hive/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b rename to shark/src/test/hive/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b diff --git a/src/test/hive/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d b/shark/src/test/hive/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d similarity index 100% rename from src/test/hive/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d rename to shark/src/test/hive/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d diff --git a/src/test/hive/golden/alter5-17-9176dc5fb5206209fa907a289db1263d b/shark/src/test/hive/golden/alter5-17-9176dc5fb5206209fa907a289db1263d similarity index 100% rename from src/test/hive/golden/alter5-17-9176dc5fb5206209fa907a289db1263d rename to shark/src/test/hive/golden/alter5-17-9176dc5fb5206209fa907a289db1263d diff --git a/src/test/hive/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf b/shark/src/test/hive/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf similarity index 100% rename from src/test/hive/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf rename to shark/src/test/hive/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf diff --git a/src/test/hive/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 b/shark/src/test/hive/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 similarity index 100% rename from src/test/hive/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 rename to shark/src/test/hive/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 diff --git a/src/test/hive/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d b/shark/src/test/hive/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d similarity index 100% rename from src/test/hive/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d rename to shark/src/test/hive/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d diff --git a/src/test/hive/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 b/shark/src/test/hive/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 similarity index 100% rename from src/test/hive/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 rename to shark/src/test/hive/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 diff --git a/src/test/hive/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf b/shark/src/test/hive/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf similarity index 100% rename from src/test/hive/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf rename to shark/src/test/hive/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf diff --git a/src/test/hive/golden/alter5-3-2fc59e32c07186869811705c89aafadc b/shark/src/test/hive/golden/alter5-3-2fc59e32c07186869811705c89aafadc similarity index 100% rename from src/test/hive/golden/alter5-3-2fc59e32c07186869811705c89aafadc rename to shark/src/test/hive/golden/alter5-3-2fc59e32c07186869811705c89aafadc diff --git a/src/test/hive/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf b/shark/src/test/hive/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf similarity index 100% rename from src/test/hive/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf rename to shark/src/test/hive/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf diff --git a/src/test/hive/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 b/shark/src/test/hive/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 similarity index 100% rename from src/test/hive/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 rename to shark/src/test/hive/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 diff --git a/src/test/hive/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 b/shark/src/test/hive/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 similarity index 100% rename from src/test/hive/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 rename to shark/src/test/hive/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 diff --git a/src/test/hive/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf b/shark/src/test/hive/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf similarity index 100% rename from src/test/hive/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf rename to shark/src/test/hive/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf diff --git a/src/test/hive/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 b/shark/src/test/hive/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 similarity index 100% rename from src/test/hive/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 rename to shark/src/test/hive/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 diff --git a/src/test/hive/golden/alter5-9-1e085f9741197e659413828c42386733 b/shark/src/test/hive/golden/alter5-9-1e085f9741197e659413828c42386733 similarity index 100% rename from src/test/hive/golden/alter5-9-1e085f9741197e659413828c42386733 rename to shark/src/test/hive/golden/alter5-9-1e085f9741197e659413828c42386733 diff --git a/src/test/hive/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 b/shark/src/test/hive/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 similarity index 100% rename from src/test/hive/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 rename to shark/src/test/hive/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 diff --git a/src/test/hive/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e b/shark/src/test/hive/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e similarity index 100% rename from src/test/hive/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e rename to shark/src/test/hive/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e diff --git a/src/test/hive/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 b/shark/src/test/hive/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 similarity index 100% rename from src/test/hive/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 rename to shark/src/test/hive/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 diff --git a/src/test/hive/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb b/shark/src/test/hive/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb similarity index 100% rename from src/test/hive/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb rename to shark/src/test/hive/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb diff --git a/src/test/hive/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 b/shark/src/test/hive/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 similarity index 100% rename from src/test/hive/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 rename to shark/src/test/hive/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 diff --git a/src/test/hive/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 b/shark/src/test/hive/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 similarity index 100% rename from src/test/hive/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 rename to shark/src/test/hive/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 diff --git a/src/test/hive/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d b/shark/src/test/hive/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d similarity index 100% rename from src/test/hive/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d rename to shark/src/test/hive/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d diff --git a/src/test/hive/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 b/shark/src/test/hive/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 similarity index 100% rename from src/test/hive/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 rename to shark/src/test/hive/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 diff --git a/src/test/hive/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 b/shark/src/test/hive/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 similarity index 100% rename from src/test/hive/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 rename to shark/src/test/hive/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 diff --git a/src/test/hive/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 b/shark/src/test/hive/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 similarity index 100% rename from src/test/hive/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 rename to shark/src/test/hive/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 diff --git a/src/test/hive/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c b/shark/src/test/hive/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c similarity index 100% rename from src/test/hive/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c rename to shark/src/test/hive/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c diff --git a/src/test/hive/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 b/shark/src/test/hive/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 similarity index 100% rename from src/test/hive/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 rename to shark/src/test/hive/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 diff --git a/src/test/hive/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d b/shark/src/test/hive/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d similarity index 100% rename from src/test/hive/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d rename to shark/src/test/hive/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d diff --git a/src/test/hive/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be b/shark/src/test/hive/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be similarity index 100% rename from src/test/hive/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be rename to shark/src/test/hive/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be diff --git a/src/test/hive/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f b/shark/src/test/hive/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f similarity index 100% rename from src/test/hive/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f rename to shark/src/test/hive/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f diff --git a/src/test/hive/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 b/shark/src/test/hive/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 similarity index 100% rename from src/test/hive/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 rename to shark/src/test/hive/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 diff --git a/src/test/hive/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 b/shark/src/test/hive/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 similarity index 100% rename from src/test/hive/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 rename to shark/src/test/hive/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 diff --git a/src/test/hive/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc b/shark/src/test/hive/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc similarity index 100% rename from src/test/hive/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc rename to shark/src/test/hive/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc diff --git a/src/test/hive/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 b/shark/src/test/hive/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 similarity index 100% rename from src/test/hive/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 rename to shark/src/test/hive/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 diff --git a/src/test/hive/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a b/shark/src/test/hive/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a rename to shark/src/test/hive/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a diff --git a/src/test/hive/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 b/shark/src/test/hive/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 rename to shark/src/test/hive/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 diff --git a/src/test/hive/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 b/shark/src/test/hive/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 rename to shark/src/test/hive/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 diff --git a/src/test/hive/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 b/shark/src/test/hive/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 rename to shark/src/test/hive/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 diff --git a/src/test/hive/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 b/shark/src/test/hive/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 rename to shark/src/test/hive/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 diff --git a/src/test/hive/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e b/shark/src/test/hive/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e rename to shark/src/test/hive/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e diff --git a/src/test/hive/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 b/shark/src/test/hive/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 rename to shark/src/test/hive/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 diff --git a/src/test/hive/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 b/shark/src/test/hive/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 rename to shark/src/test/hive/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 diff --git a/src/test/hive/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e b/shark/src/test/hive/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e rename to shark/src/test/hive/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e diff --git a/src/test/hive/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f b/shark/src/test/hive/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f rename to shark/src/test/hive/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f diff --git a/src/test/hive/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c b/shark/src/test/hive/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c rename to shark/src/test/hive/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c diff --git a/src/test/hive/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 b/shark/src/test/hive/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 rename to shark/src/test/hive/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 diff --git a/src/test/hive/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a b/shark/src/test/hive/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a rename to shark/src/test/hive/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a diff --git a/src/test/hive/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 b/shark/src/test/hive/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 rename to shark/src/test/hive/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 diff --git a/src/test/hive/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a b/shark/src/test/hive/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a rename to shark/src/test/hive/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a diff --git a/src/test/hive/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 b/shark/src/test/hive/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 rename to shark/src/test/hive/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 diff --git a/src/test/hive/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a b/shark/src/test/hive/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a rename to shark/src/test/hive/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a diff --git a/src/test/hive/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 b/shark/src/test/hive/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 rename to shark/src/test/hive/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 diff --git a/src/test/hive/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 b/shark/src/test/hive/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 rename to shark/src/test/hive/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 diff --git a/src/test/hive/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 b/shark/src/test/hive/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 rename to shark/src/test/hive/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 diff --git a/src/test/hive/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d b/shark/src/test/hive/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d rename to shark/src/test/hive/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d diff --git a/src/test/hive/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d b/shark/src/test/hive/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d rename to shark/src/test/hive/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d diff --git a/src/test/hive/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd b/shark/src/test/hive/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd rename to shark/src/test/hive/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd diff --git a/src/test/hive/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 b/shark/src/test/hive/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 rename to shark/src/test/hive/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 diff --git a/src/test/hive/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 b/shark/src/test/hive/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 rename to shark/src/test/hive/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 diff --git a/src/test/hive/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 b/shark/src/test/hive/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 similarity index 100% rename from src/test/hive/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 rename to shark/src/test/hive/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 diff --git a/src/test/hive/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 b/shark/src/test/hive/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 rename to shark/src/test/hive/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 diff --git a/src/test/hive/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 b/shark/src/test/hive/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 rename to shark/src/test/hive/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 diff --git a/src/test/hive/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 b/shark/src/test/hive/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 rename to shark/src/test/hive/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 diff --git a/src/test/hive/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 b/shark/src/test/hive/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 rename to shark/src/test/hive/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 diff --git a/src/test/hive/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 b/shark/src/test/hive/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 rename to shark/src/test/hive/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 diff --git a/src/test/hive/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 b/shark/src/test/hive/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 rename to shark/src/test/hive/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 diff --git a/src/test/hive/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 b/shark/src/test/hive/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 rename to shark/src/test/hive/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 diff --git a/src/test/hive/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 b/shark/src/test/hive/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 rename to shark/src/test/hive/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 diff --git a/src/test/hive/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 b/shark/src/test/hive/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 rename to shark/src/test/hive/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 diff --git a/src/test/hive/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 b/shark/src/test/hive/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 rename to shark/src/test/hive/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 diff --git a/src/test/hive/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 b/shark/src/test/hive/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 rename to shark/src/test/hive/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 diff --git a/src/test/hive/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 b/shark/src/test/hive/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 rename to shark/src/test/hive/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 diff --git a/src/test/hive/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 b/shark/src/test/hive/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 rename to shark/src/test/hive/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 diff --git a/src/test/hive/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 b/shark/src/test/hive/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 rename to shark/src/test/hive/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 diff --git a/src/test/hive/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 b/shark/src/test/hive/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 rename to shark/src/test/hive/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 diff --git a/src/test/hive/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 b/shark/src/test/hive/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 rename to shark/src/test/hive/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 diff --git a/src/test/hive/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 b/shark/src/test/hive/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 rename to shark/src/test/hive/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 diff --git a/src/test/hive/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 b/shark/src/test/hive/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 rename to shark/src/test/hive/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 diff --git a/src/test/hive/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 b/shark/src/test/hive/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 rename to shark/src/test/hive/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 diff --git a/src/test/hive/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 b/shark/src/test/hive/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 similarity index 100% rename from src/test/hive/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 rename to shark/src/test/hive/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 diff --git a/src/test/hive/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f b/shark/src/test/hive/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f rename to shark/src/test/hive/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f diff --git a/src/test/hive/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c b/shark/src/test/hive/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c rename to shark/src/test/hive/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c diff --git a/src/test/hive/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d b/shark/src/test/hive/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d rename to shark/src/test/hive/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d diff --git a/src/test/hive/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b b/shark/src/test/hive/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b rename to shark/src/test/hive/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b diff --git a/src/test/hive/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 b/shark/src/test/hive/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 rename to shark/src/test/hive/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 diff --git a/src/test/hive/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a b/shark/src/test/hive/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a rename to shark/src/test/hive/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a diff --git a/src/test/hive/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a b/shark/src/test/hive/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a rename to shark/src/test/hive/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a diff --git a/src/test/hive/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 b/shark/src/test/hive/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 rename to shark/src/test/hive/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 diff --git a/src/test/hive/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 b/shark/src/test/hive/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 rename to shark/src/test/hive/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 diff --git a/src/test/hive/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 b/shark/src/test/hive/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 rename to shark/src/test/hive/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 diff --git a/src/test/hive/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 b/shark/src/test/hive/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 rename to shark/src/test/hive/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 diff --git a/src/test/hive/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 b/shark/src/test/hive/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 rename to shark/src/test/hive/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 diff --git a/src/test/hive/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c b/shark/src/test/hive/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c rename to shark/src/test/hive/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c diff --git a/src/test/hive/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 b/shark/src/test/hive/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 rename to shark/src/test/hive/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 diff --git a/src/test/hive/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 b/shark/src/test/hive/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 rename to shark/src/test/hive/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 diff --git a/src/test/hive/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed b/shark/src/test/hive/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed rename to shark/src/test/hive/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed diff --git a/src/test/hive/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d b/shark/src/test/hive/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d similarity index 100% rename from src/test/hive/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d rename to shark/src/test/hive/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d diff --git a/src/test/hive/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 b/shark/src/test/hive/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 similarity index 100% rename from src/test/hive/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 rename to shark/src/test/hive/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 diff --git a/src/test/hive/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 b/shark/src/test/hive/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 similarity index 100% rename from src/test/hive/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 rename to shark/src/test/hive/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 diff --git a/src/test/hive/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 b/shark/src/test/hive/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 similarity index 100% rename from src/test/hive/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 rename to shark/src/test/hive/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 diff --git a/src/test/hive/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 b/shark/src/test/hive/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 similarity index 100% rename from src/test/hive/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 rename to shark/src/test/hive/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 diff --git a/src/test/hive/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c b/shark/src/test/hive/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c similarity index 100% rename from src/test/hive/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c rename to shark/src/test/hive/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c diff --git a/src/test/hive/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 b/shark/src/test/hive/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 rename to shark/src/test/hive/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 diff --git a/src/test/hive/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada b/shark/src/test/hive/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada similarity index 100% rename from src/test/hive/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada rename to shark/src/test/hive/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada diff --git a/src/test/hive/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 b/shark/src/test/hive/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 rename to shark/src/test/hive/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 diff --git a/src/test/hive/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 b/shark/src/test/hive/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 rename to shark/src/test/hive/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 diff --git a/src/test/hive/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 b/shark/src/test/hive/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 rename to shark/src/test/hive/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 diff --git a/src/test/hive/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada b/shark/src/test/hive/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada similarity index 100% rename from src/test/hive/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada rename to shark/src/test/hive/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada diff --git a/src/test/hive/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 b/shark/src/test/hive/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 rename to shark/src/test/hive/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 diff --git a/src/test/hive/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 b/shark/src/test/hive/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 rename to shark/src/test/hive/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 diff --git a/src/test/hive/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b b/shark/src/test/hive/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b similarity index 100% rename from src/test/hive/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b rename to shark/src/test/hive/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b diff --git a/src/test/hive/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c b/shark/src/test/hive/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c similarity index 100% rename from src/test/hive/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c rename to shark/src/test/hive/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c diff --git a/src/test/hive/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 b/shark/src/test/hive/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 rename to shark/src/test/hive/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 diff --git a/src/test/hive/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 b/shark/src/test/hive/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 rename to shark/src/test/hive/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 diff --git a/src/test/hive/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 b/shark/src/test/hive/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 rename to shark/src/test/hive/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 diff --git a/src/test/hive/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f b/shark/src/test/hive/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f similarity index 100% rename from src/test/hive/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f rename to shark/src/test/hive/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f diff --git a/src/test/hive/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 b/shark/src/test/hive/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 rename to shark/src/test/hive/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 diff --git a/src/test/hive/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 b/shark/src/test/hive/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 rename to shark/src/test/hive/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 diff --git a/src/test/hive/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b b/shark/src/test/hive/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b similarity index 100% rename from src/test/hive/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b rename to shark/src/test/hive/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b diff --git a/src/test/hive/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b b/shark/src/test/hive/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b similarity index 100% rename from src/test/hive/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b rename to shark/src/test/hive/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b diff --git a/src/test/hive/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f b/shark/src/test/hive/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f similarity index 100% rename from src/test/hive/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f rename to shark/src/test/hive/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f diff --git a/src/test/hive/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 b/shark/src/test/hive/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 rename to shark/src/test/hive/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 diff --git a/src/test/hive/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 b/shark/src/test/hive/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 rename to shark/src/test/hive/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 diff --git a/src/test/hive/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 b/shark/src/test/hive/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 rename to shark/src/test/hive/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 diff --git a/src/test/hive/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 b/shark/src/test/hive/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 rename to shark/src/test/hive/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 diff --git a/src/test/hive/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 b/shark/src/test/hive/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 similarity index 100% rename from src/test/hive/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 rename to shark/src/test/hive/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 diff --git a/src/test/hive/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c b/shark/src/test/hive/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c similarity index 100% rename from src/test/hive/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c rename to shark/src/test/hive/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c diff --git a/src/test/hive/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 b/shark/src/test/hive/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 similarity index 100% rename from src/test/hive/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 rename to shark/src/test/hive/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 diff --git a/src/test/hive/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa b/shark/src/test/hive/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa similarity index 100% rename from src/test/hive/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa rename to shark/src/test/hive/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa diff --git a/src/test/hive/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 b/shark/src/test/hive/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 similarity index 100% rename from src/test/hive/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 rename to shark/src/test/hive/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 diff --git a/src/test/hive/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc b/shark/src/test/hive/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc similarity index 100% rename from src/test/hive/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc rename to shark/src/test/hive/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc diff --git a/src/test/hive/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 b/shark/src/test/hive/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 similarity index 100% rename from src/test/hive/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 rename to shark/src/test/hive/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 diff --git a/src/test/hive/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def b/shark/src/test/hive/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def similarity index 100% rename from src/test/hive/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def rename to shark/src/test/hive/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def diff --git a/src/test/hive/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 b/shark/src/test/hive/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 similarity index 100% rename from src/test/hive/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 rename to shark/src/test/hive/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 diff --git a/src/test/hive/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 b/shark/src/test/hive/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 similarity index 100% rename from src/test/hive/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 rename to shark/src/test/hive/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 diff --git a/src/test/hive/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 b/shark/src/test/hive/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 similarity index 100% rename from src/test/hive/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 rename to shark/src/test/hive/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 diff --git a/src/test/hive/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 b/shark/src/test/hive/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 similarity index 100% rename from src/test/hive/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 rename to shark/src/test/hive/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 diff --git a/src/test/hive/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa b/shark/src/test/hive/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa similarity index 100% rename from src/test/hive/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa rename to shark/src/test/hive/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa diff --git a/src/test/hive/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 b/shark/src/test/hive/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 similarity index 100% rename from src/test/hive/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 rename to shark/src/test/hive/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 diff --git a/src/test/hive/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc b/shark/src/test/hive/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc similarity index 100% rename from src/test/hive/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc rename to shark/src/test/hive/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc diff --git a/src/test/hive/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 b/shark/src/test/hive/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 similarity index 100% rename from src/test/hive/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 rename to shark/src/test/hive/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 diff --git a/src/test/hive/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 b/shark/src/test/hive/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 similarity index 100% rename from src/test/hive/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 rename to shark/src/test/hive/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 diff --git a/src/test/hive/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b b/shark/src/test/hive/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b similarity index 100% rename from src/test/hive/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b rename to shark/src/test/hive/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b diff --git a/src/test/hive/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 b/shark/src/test/hive/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 similarity index 100% rename from src/test/hive/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 rename to shark/src/test/hive/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 diff --git a/src/test/hive/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 b/shark/src/test/hive/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 similarity index 100% rename from src/test/hive/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 rename to shark/src/test/hive/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 diff --git a/src/test/hive/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 b/shark/src/test/hive/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 similarity index 100% rename from src/test/hive/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 rename to shark/src/test/hive/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 diff --git a/src/test/hive/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea b/shark/src/test/hive/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea similarity index 100% rename from src/test/hive/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea rename to shark/src/test/hive/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea diff --git a/src/test/hive/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 b/shark/src/test/hive/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 similarity index 100% rename from src/test/hive/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 rename to shark/src/test/hive/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 diff --git a/src/test/hive/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 b/shark/src/test/hive/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 similarity index 100% rename from src/test/hive/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 rename to shark/src/test/hive/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 diff --git a/src/test/hive/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd b/shark/src/test/hive/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd similarity index 100% rename from src/test/hive/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd rename to shark/src/test/hive/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd diff --git a/src/test/hive/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 b/shark/src/test/hive/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 similarity index 100% rename from src/test/hive/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 rename to shark/src/test/hive/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 diff --git a/src/test/hive/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 b/shark/src/test/hive/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 similarity index 100% rename from src/test/hive/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 rename to shark/src/test/hive/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 diff --git a/src/test/hive/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 b/shark/src/test/hive/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 similarity index 100% rename from src/test/hive/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 rename to shark/src/test/hive/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 diff --git a/src/test/hive/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 b/shark/src/test/hive/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 similarity index 100% rename from src/test/hive/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 rename to shark/src/test/hive/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 diff --git a/src/test/hive/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d b/shark/src/test/hive/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d similarity index 100% rename from src/test/hive/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d rename to shark/src/test/hive/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d diff --git a/src/test/hive/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 b/shark/src/test/hive/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 similarity index 100% rename from src/test/hive/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 rename to shark/src/test/hive/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 diff --git a/src/test/hive/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 b/shark/src/test/hive/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 similarity index 100% rename from src/test/hive/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 rename to shark/src/test/hive/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 diff --git a/src/test/hive/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f b/shark/src/test/hive/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f similarity index 100% rename from src/test/hive/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f rename to shark/src/test/hive/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f diff --git a/src/test/hive/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 b/shark/src/test/hive/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 similarity index 100% rename from src/test/hive/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 rename to shark/src/test/hive/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 diff --git a/src/test/hive/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 b/shark/src/test/hive/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 similarity index 100% rename from src/test/hive/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 rename to shark/src/test/hive/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 diff --git a/src/test/hive/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 b/shark/src/test/hive/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 similarity index 100% rename from src/test/hive/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 rename to shark/src/test/hive/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 diff --git a/src/test/hive/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc b/shark/src/test/hive/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc similarity index 100% rename from src/test/hive/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc rename to shark/src/test/hive/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc diff --git a/src/test/hive/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 b/shark/src/test/hive/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 similarity index 100% rename from src/test/hive/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 rename to shark/src/test/hive/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 diff --git a/src/test/hive/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 b/shark/src/test/hive/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 similarity index 100% rename from src/test/hive/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 rename to shark/src/test/hive/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 diff --git a/src/test/hive/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb b/shark/src/test/hive/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb similarity index 100% rename from src/test/hive/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb rename to shark/src/test/hive/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb diff --git a/src/test/hive/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c b/shark/src/test/hive/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c similarity index 100% rename from src/test/hive/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c rename to shark/src/test/hive/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c diff --git a/src/test/hive/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 b/shark/src/test/hive/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 similarity index 100% rename from src/test/hive/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 rename to shark/src/test/hive/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 diff --git a/src/test/hive/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c b/shark/src/test/hive/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c similarity index 100% rename from src/test/hive/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c rename to shark/src/test/hive/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c diff --git a/src/test/hive/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c b/shark/src/test/hive/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c similarity index 100% rename from src/test/hive/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c rename to shark/src/test/hive/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c diff --git a/src/test/hive/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 b/shark/src/test/hive/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 similarity index 100% rename from src/test/hive/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 rename to shark/src/test/hive/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 diff --git a/src/test/hive/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad b/shark/src/test/hive/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad similarity index 100% rename from src/test/hive/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad rename to shark/src/test/hive/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad diff --git a/src/test/hive/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b b/shark/src/test/hive/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b similarity index 100% rename from src/test/hive/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b rename to shark/src/test/hive/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b diff --git a/src/test/hive/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 b/shark/src/test/hive/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 similarity index 100% rename from src/test/hive/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 rename to shark/src/test/hive/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 diff --git a/src/test/hive/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b b/shark/src/test/hive/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b similarity index 100% rename from src/test/hive/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b rename to shark/src/test/hive/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b diff --git a/src/test/hive/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 b/shark/src/test/hive/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 similarity index 100% rename from src/test/hive/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 rename to shark/src/test/hive/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 diff --git a/src/test/hive/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b b/shark/src/test/hive/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b similarity index 100% rename from src/test/hive/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b rename to shark/src/test/hive/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b diff --git a/src/test/hive/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 b/shark/src/test/hive/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 similarity index 100% rename from src/test/hive/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 rename to shark/src/test/hive/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 diff --git a/src/test/hive/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d b/shark/src/test/hive/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d similarity index 100% rename from src/test/hive/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d rename to shark/src/test/hive/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d diff --git a/src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb b/shark/src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb similarity index 100% rename from src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb rename to shark/src/test/hive/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb diff --git a/src/test/hive/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a b/shark/src/test/hive/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a similarity index 100% rename from src/test/hive/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a rename to shark/src/test/hive/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a diff --git a/src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 b/shark/src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 similarity index 100% rename from src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 rename to shark/src/test/hive/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 diff --git a/src/test/hive/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d b/shark/src/test/hive/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d similarity index 100% rename from src/test/hive/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d rename to shark/src/test/hive/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d diff --git a/src/test/hive/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab b/shark/src/test/hive/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab similarity index 100% rename from src/test/hive/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab rename to shark/src/test/hive/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab diff --git a/src/test/hive/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b b/shark/src/test/hive/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b similarity index 100% rename from src/test/hive/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b rename to shark/src/test/hive/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b diff --git a/src/test/hive/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 b/shark/src/test/hive/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 similarity index 100% rename from src/test/hive/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 rename to shark/src/test/hive/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 diff --git a/src/test/hive/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 b/shark/src/test/hive/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 similarity index 100% rename from src/test/hive/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 rename to shark/src/test/hive/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 diff --git a/src/test/hive/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb b/shark/src/test/hive/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb similarity index 100% rename from src/test/hive/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb rename to shark/src/test/hive/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb diff --git a/src/test/hive/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 b/shark/src/test/hive/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 similarity index 100% rename from src/test/hive/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 rename to shark/src/test/hive/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 diff --git a/src/test/hive/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb b/shark/src/test/hive/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb similarity index 100% rename from src/test/hive/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb rename to shark/src/test/hive/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb diff --git a/src/test/hive/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 b/shark/src/test/hive/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 similarity index 100% rename from src/test/hive/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 rename to shark/src/test/hive/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 diff --git a/src/test/hive/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 b/shark/src/test/hive/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 similarity index 100% rename from src/test/hive/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 rename to shark/src/test/hive/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 diff --git a/src/test/hive/golden/archive-16-892147913578bcf60620b7dd73893dd0 b/shark/src/test/hive/golden/archive-16-892147913578bcf60620b7dd73893dd0 similarity index 100% rename from src/test/hive/golden/archive-16-892147913578bcf60620b7dd73893dd0 rename to shark/src/test/hive/golden/archive-16-892147913578bcf60620b7dd73893dd0 diff --git a/src/test/hive/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb b/shark/src/test/hive/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb similarity index 100% rename from src/test/hive/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb rename to shark/src/test/hive/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb diff --git a/src/test/hive/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a b/shark/src/test/hive/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a similarity index 100% rename from src/test/hive/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a rename to shark/src/test/hive/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a diff --git a/src/test/hive/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f b/shark/src/test/hive/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f similarity index 100% rename from src/test/hive/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f rename to shark/src/test/hive/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f diff --git a/src/test/hive/golden/archive-2-713efc113418b01f76ffd589840193c8 b/shark/src/test/hive/golden/archive-2-713efc113418b01f76ffd589840193c8 similarity index 100% rename from src/test/hive/golden/archive-2-713efc113418b01f76ffd589840193c8 rename to shark/src/test/hive/golden/archive-2-713efc113418b01f76ffd589840193c8 diff --git a/src/test/hive/golden/archive-20-530277b0fee8b05c37b26846bceef827 b/shark/src/test/hive/golden/archive-20-530277b0fee8b05c37b26846bceef827 similarity index 100% rename from src/test/hive/golden/archive-20-530277b0fee8b05c37b26846bceef827 rename to shark/src/test/hive/golden/archive-20-530277b0fee8b05c37b26846bceef827 diff --git a/src/test/hive/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 b/shark/src/test/hive/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 similarity index 100% rename from src/test/hive/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 rename to shark/src/test/hive/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 diff --git a/src/test/hive/golden/archive-22-530277b0fee8b05c37b26846bceef827 b/shark/src/test/hive/golden/archive-22-530277b0fee8b05c37b26846bceef827 similarity index 100% rename from src/test/hive/golden/archive-22-530277b0fee8b05c37b26846bceef827 rename to shark/src/test/hive/golden/archive-22-530277b0fee8b05c37b26846bceef827 diff --git a/src/test/hive/golden/archive-23-892147913578bcf60620b7dd73893dd0 b/shark/src/test/hive/golden/archive-23-892147913578bcf60620b7dd73893dd0 similarity index 100% rename from src/test/hive/golden/archive-23-892147913578bcf60620b7dd73893dd0 rename to shark/src/test/hive/golden/archive-23-892147913578bcf60620b7dd73893dd0 diff --git a/src/test/hive/golden/archive-24-530277b0fee8b05c37b26846bceef827 b/shark/src/test/hive/golden/archive-24-530277b0fee8b05c37b26846bceef827 similarity index 100% rename from src/test/hive/golden/archive-24-530277b0fee8b05c37b26846bceef827 rename to shark/src/test/hive/golden/archive-24-530277b0fee8b05c37b26846bceef827 diff --git a/src/test/hive/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f b/shark/src/test/hive/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f similarity index 100% rename from src/test/hive/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f rename to shark/src/test/hive/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f diff --git a/src/test/hive/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a b/shark/src/test/hive/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a similarity index 100% rename from src/test/hive/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a rename to shark/src/test/hive/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a diff --git a/src/test/hive/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 b/shark/src/test/hive/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 similarity index 100% rename from src/test/hive/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 rename to shark/src/test/hive/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 diff --git a/src/test/hive/golden/archive-28-188eb7912265ed8dffa5200517bbe526 b/shark/src/test/hive/golden/archive-28-188eb7912265ed8dffa5200517bbe526 similarity index 100% rename from src/test/hive/golden/archive-28-188eb7912265ed8dffa5200517bbe526 rename to shark/src/test/hive/golden/archive-28-188eb7912265ed8dffa5200517bbe526 diff --git a/src/test/hive/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 b/shark/src/test/hive/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 similarity index 100% rename from src/test/hive/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 rename to shark/src/test/hive/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 diff --git a/src/test/hive/golden/archive-3-27ad2962fed131f51ba802596ba37278 b/shark/src/test/hive/golden/archive-3-27ad2962fed131f51ba802596ba37278 similarity index 100% rename from src/test/hive/golden/archive-3-27ad2962fed131f51ba802596ba37278 rename to shark/src/test/hive/golden/archive-3-27ad2962fed131f51ba802596ba37278 diff --git a/src/test/hive/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/shark/src/test/hive/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a similarity index 100% rename from src/test/hive/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a rename to shark/src/test/hive/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a diff --git a/src/test/hive/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 b/shark/src/test/hive/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 similarity index 100% rename from src/test/hive/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 rename to shark/src/test/hive/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 diff --git a/src/test/hive/golden/archive-32-27ad2962fed131f51ba802596ba37278 b/shark/src/test/hive/golden/archive-32-27ad2962fed131f51ba802596ba37278 similarity index 100% rename from src/test/hive/golden/archive-32-27ad2962fed131f51ba802596ba37278 rename to shark/src/test/hive/golden/archive-32-27ad2962fed131f51ba802596ba37278 diff --git a/src/test/hive/golden/archive-4-3e95421993ab28d18245ec2340f580a3 b/shark/src/test/hive/golden/archive-4-3e95421993ab28d18245ec2340f580a3 similarity index 100% rename from src/test/hive/golden/archive-4-3e95421993ab28d18245ec2340f580a3 rename to shark/src/test/hive/golden/archive-4-3e95421993ab28d18245ec2340f580a3 diff --git a/src/test/hive/golden/archive-5-c0c18ac884677231a41eea8d980d0451 b/shark/src/test/hive/golden/archive-5-c0c18ac884677231a41eea8d980d0451 similarity index 100% rename from src/test/hive/golden/archive-5-c0c18ac884677231a41eea8d980d0451 rename to shark/src/test/hive/golden/archive-5-c0c18ac884677231a41eea8d980d0451 diff --git a/src/test/hive/golden/archive-6-528ab9750a558af7f1a43b3108e793dd b/shark/src/test/hive/golden/archive-6-528ab9750a558af7f1a43b3108e793dd similarity index 100% rename from src/test/hive/golden/archive-6-528ab9750a558af7f1a43b3108e793dd rename to shark/src/test/hive/golden/archive-6-528ab9750a558af7f1a43b3108e793dd diff --git a/src/test/hive/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca b/shark/src/test/hive/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca similarity index 100% rename from src/test/hive/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca rename to shark/src/test/hive/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca diff --git a/src/test/hive/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c b/shark/src/test/hive/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c similarity index 100% rename from src/test/hive/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c rename to shark/src/test/hive/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c diff --git a/src/test/hive/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf b/shark/src/test/hive/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf similarity index 100% rename from src/test/hive/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf rename to shark/src/test/hive/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf diff --git a/src/test/hive/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 b/shark/src/test/hive/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 rename to shark/src/test/hive/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 diff --git a/src/test/hive/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 b/shark/src/test/hive/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 rename to shark/src/test/hive/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 diff --git a/src/test/hive/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb b/shark/src/test/hive/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb rename to shark/src/test/hive/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb diff --git a/src/test/hive/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 b/shark/src/test/hive/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 rename to shark/src/test/hive/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 diff --git a/src/test/hive/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb b/shark/src/test/hive/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb rename to shark/src/test/hive/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb diff --git a/src/test/hive/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 b/shark/src/test/hive/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 rename to shark/src/test/hive/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 diff --git a/src/test/hive/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 b/shark/src/test/hive/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 rename to shark/src/test/hive/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 diff --git a/src/test/hive/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 b/shark/src/test/hive/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 rename to shark/src/test/hive/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 diff --git a/src/test/hive/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb b/shark/src/test/hive/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb rename to shark/src/test/hive/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb diff --git a/src/test/hive/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a b/shark/src/test/hive/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a rename to shark/src/test/hive/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a diff --git a/src/test/hive/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f b/shark/src/test/hive/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f rename to shark/src/test/hive/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f diff --git a/src/test/hive/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 b/shark/src/test/hive/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 rename to shark/src/test/hive/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 diff --git a/src/test/hive/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 b/shark/src/test/hive/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 rename to shark/src/test/hive/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 diff --git a/src/test/hive/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 b/shark/src/test/hive/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 rename to shark/src/test/hive/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 diff --git a/src/test/hive/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 b/shark/src/test/hive/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 rename to shark/src/test/hive/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 diff --git a/src/test/hive/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 b/shark/src/test/hive/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 rename to shark/src/test/hive/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 diff --git a/src/test/hive/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 b/shark/src/test/hive/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 rename to shark/src/test/hive/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 diff --git a/src/test/hive/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f b/shark/src/test/hive/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f rename to shark/src/test/hive/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f diff --git a/src/test/hive/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a b/shark/src/test/hive/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a rename to shark/src/test/hive/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a diff --git a/src/test/hive/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 b/shark/src/test/hive/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 rename to shark/src/test/hive/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 diff --git a/src/test/hive/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 b/shark/src/test/hive/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 rename to shark/src/test/hive/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 diff --git a/src/test/hive/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 b/shark/src/test/hive/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 rename to shark/src/test/hive/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 diff --git a/src/test/hive/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 b/shark/src/test/hive/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 rename to shark/src/test/hive/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 diff --git a/src/test/hive/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/shark/src/test/hive/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a rename to shark/src/test/hive/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a diff --git a/src/test/hive/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 b/shark/src/test/hive/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 rename to shark/src/test/hive/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 diff --git a/src/test/hive/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 b/shark/src/test/hive/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 rename to shark/src/test/hive/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 diff --git a/src/test/hive/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 b/shark/src/test/hive/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 rename to shark/src/test/hive/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 diff --git a/src/test/hive/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 b/shark/src/test/hive/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 rename to shark/src/test/hive/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 diff --git a/src/test/hive/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd b/shark/src/test/hive/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd rename to shark/src/test/hive/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd diff --git a/src/test/hive/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca b/shark/src/test/hive/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca rename to shark/src/test/hive/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca diff --git a/src/test/hive/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c b/shark/src/test/hive/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c rename to shark/src/test/hive/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c diff --git a/src/test/hive/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf b/shark/src/test/hive/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf similarity index 100% rename from src/test/hive/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf rename to shark/src/test/hive/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf diff --git a/src/test/hive/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 b/shark/src/test/hive/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 similarity index 100% rename from src/test/hive/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 rename to shark/src/test/hive/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 diff --git a/src/test/hive/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 b/shark/src/test/hive/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 similarity index 100% rename from src/test/hive/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 rename to shark/src/test/hive/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 diff --git a/src/test/hive/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e b/shark/src/test/hive/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e similarity index 100% rename from src/test/hive/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e rename to shark/src/test/hive/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e diff --git a/src/test/hive/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 b/shark/src/test/hive/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 similarity index 100% rename from src/test/hive/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 rename to shark/src/test/hive/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 diff --git a/src/test/hive/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb b/shark/src/test/hive/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb similarity index 100% rename from src/test/hive/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb rename to shark/src/test/hive/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb diff --git a/src/test/hive/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 b/shark/src/test/hive/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 similarity index 100% rename from src/test/hive/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 rename to shark/src/test/hive/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 diff --git a/src/test/hive/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 b/shark/src/test/hive/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 similarity index 100% rename from src/test/hive/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 rename to shark/src/test/hive/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 diff --git a/src/test/hive/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d b/shark/src/test/hive/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d similarity index 100% rename from src/test/hive/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d rename to shark/src/test/hive/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d diff --git a/src/test/hive/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb b/shark/src/test/hive/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb similarity index 100% rename from src/test/hive/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb rename to shark/src/test/hive/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb diff --git a/src/test/hive/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 b/shark/src/test/hive/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 similarity index 100% rename from src/test/hive/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 rename to shark/src/test/hive/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 diff --git a/src/test/hive/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 b/shark/src/test/hive/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 similarity index 100% rename from src/test/hive/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 rename to shark/src/test/hive/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 diff --git a/src/test/hive/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 b/shark/src/test/hive/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 similarity index 100% rename from src/test/hive/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 rename to shark/src/test/hive/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 diff --git a/src/test/hive/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 b/shark/src/test/hive/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 similarity index 100% rename from src/test/hive/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 rename to shark/src/test/hive/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 diff --git a/src/test/hive/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a b/shark/src/test/hive/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a similarity index 100% rename from src/test/hive/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a rename to shark/src/test/hive/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a diff --git a/src/test/hive/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca b/shark/src/test/hive/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca similarity index 100% rename from src/test/hive/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca rename to shark/src/test/hive/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca diff --git a/src/test/hive/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c b/shark/src/test/hive/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c similarity index 100% rename from src/test/hive/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c rename to shark/src/test/hive/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c diff --git a/src/test/hive/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf b/shark/src/test/hive/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf similarity index 100% rename from src/test/hive/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf rename to shark/src/test/hive/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf diff --git a/src/test/hive/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba b/shark/src/test/hive/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba similarity index 100% rename from src/test/hive/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba rename to shark/src/test/hive/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba diff --git a/src/test/hive/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 b/shark/src/test/hive/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 similarity index 100% rename from src/test/hive/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 rename to shark/src/test/hive/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 diff --git a/src/test/hive/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c b/shark/src/test/hive/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c similarity index 100% rename from src/test/hive/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c rename to shark/src/test/hive/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c diff --git a/src/test/hive/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a b/shark/src/test/hive/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a similarity index 100% rename from src/test/hive/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a rename to shark/src/test/hive/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a diff --git a/src/test/hive/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e b/shark/src/test/hive/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e similarity index 100% rename from src/test/hive/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e rename to shark/src/test/hive/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e diff --git a/src/test/hive/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 b/shark/src/test/hive/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 similarity index 100% rename from src/test/hive/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 rename to shark/src/test/hive/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 diff --git a/src/test/hive/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 b/shark/src/test/hive/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 similarity index 100% rename from src/test/hive/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 rename to shark/src/test/hive/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 diff --git a/src/test/hive/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e b/shark/src/test/hive/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e similarity index 100% rename from src/test/hive/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e rename to shark/src/test/hive/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e diff --git a/src/test/hive/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 b/shark/src/test/hive/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 similarity index 100% rename from src/test/hive/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 rename to shark/src/test/hive/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 diff --git a/src/test/hive/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 b/shark/src/test/hive/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 similarity index 100% rename from src/test/hive/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 rename to shark/src/test/hive/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 diff --git a/src/test/hive/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 b/shark/src/test/hive/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 similarity index 100% rename from src/test/hive/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 rename to shark/src/test/hive/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 diff --git a/src/test/hive/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 b/shark/src/test/hive/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 similarity index 100% rename from src/test/hive/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 rename to shark/src/test/hive/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 diff --git a/src/test/hive/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 b/shark/src/test/hive/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 similarity index 100% rename from src/test/hive/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 rename to shark/src/test/hive/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 diff --git a/src/test/hive/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 b/shark/src/test/hive/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 similarity index 100% rename from src/test/hive/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 rename to shark/src/test/hive/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 diff --git a/src/test/hive/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 b/shark/src/test/hive/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 similarity index 100% rename from src/test/hive/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 rename to shark/src/test/hive/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 diff --git a/src/test/hive/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 b/shark/src/test/hive/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 similarity index 100% rename from src/test/hive/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 rename to shark/src/test/hive/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 diff --git a/src/test/hive/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 b/shark/src/test/hive/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 similarity index 100% rename from src/test/hive/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 rename to shark/src/test/hive/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 diff --git a/src/test/hive/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e b/shark/src/test/hive/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e similarity index 100% rename from src/test/hive/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e rename to shark/src/test/hive/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e diff --git a/src/test/hive/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 b/shark/src/test/hive/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 similarity index 100% rename from src/test/hive/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 rename to shark/src/test/hive/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 diff --git a/src/test/hive/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 b/shark/src/test/hive/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 similarity index 100% rename from src/test/hive/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 rename to shark/src/test/hive/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 diff --git a/src/test/hive/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 b/shark/src/test/hive/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 similarity index 100% rename from src/test/hive/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 rename to shark/src/test/hive/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 diff --git a/src/test/hive/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 b/shark/src/test/hive/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 similarity index 100% rename from src/test/hive/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 rename to shark/src/test/hive/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 diff --git a/src/test/hive/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 b/shark/src/test/hive/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 similarity index 100% rename from src/test/hive/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 rename to shark/src/test/hive/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 diff --git a/src/test/hive/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 b/shark/src/test/hive/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 similarity index 100% rename from src/test/hive/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 rename to shark/src/test/hive/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 diff --git a/src/test/hive/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f b/shark/src/test/hive/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f similarity index 100% rename from src/test/hive/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f rename to shark/src/test/hive/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f diff --git a/src/test/hive/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 b/shark/src/test/hive/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 similarity index 100% rename from src/test/hive/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 rename to shark/src/test/hive/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 diff --git a/src/test/hive/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 b/shark/src/test/hive/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 similarity index 100% rename from src/test/hive/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 rename to shark/src/test/hive/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 diff --git a/src/test/hive/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 b/shark/src/test/hive/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 similarity index 100% rename from src/test/hive/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 rename to shark/src/test/hive/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 diff --git a/src/test/hive/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 b/shark/src/test/hive/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 similarity index 100% rename from src/test/hive/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 rename to shark/src/test/hive/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 diff --git a/src/test/hive/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 b/shark/src/test/hive/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 similarity index 100% rename from src/test/hive/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 rename to shark/src/test/hive/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 diff --git a/src/test/hive/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 b/shark/src/test/hive/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 similarity index 100% rename from src/test/hive/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 rename to shark/src/test/hive/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 diff --git a/src/test/hive/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf b/shark/src/test/hive/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf similarity index 100% rename from src/test/hive/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf rename to shark/src/test/hive/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf diff --git a/src/test/hive/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 b/shark/src/test/hive/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 similarity index 100% rename from src/test/hive/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 rename to shark/src/test/hive/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 diff --git a/src/test/hive/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 b/shark/src/test/hive/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 similarity index 100% rename from src/test/hive/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 rename to shark/src/test/hive/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 diff --git a/src/test/hive/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 b/shark/src/test/hive/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 similarity index 100% rename from src/test/hive/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 rename to shark/src/test/hive/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 diff --git a/src/test/hive/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 b/shark/src/test/hive/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 similarity index 100% rename from src/test/hive/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 rename to shark/src/test/hive/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 diff --git a/src/test/hive/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c b/shark/src/test/hive/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c similarity index 100% rename from src/test/hive/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c rename to shark/src/test/hive/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c diff --git a/src/test/hive/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b b/shark/src/test/hive/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b similarity index 100% rename from src/test/hive/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b rename to shark/src/test/hive/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b diff --git a/src/test/hive/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f b/shark/src/test/hive/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f similarity index 100% rename from src/test/hive/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f rename to shark/src/test/hive/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f diff --git a/src/test/hive/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 b/shark/src/test/hive/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 similarity index 100% rename from src/test/hive/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 rename to shark/src/test/hive/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 diff --git a/src/test/hive/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 b/shark/src/test/hive/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 similarity index 100% rename from src/test/hive/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 rename to shark/src/test/hive/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 diff --git a/src/test/hive/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f b/shark/src/test/hive/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f similarity index 100% rename from src/test/hive/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f rename to shark/src/test/hive/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f diff --git a/src/test/hive/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca b/shark/src/test/hive/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca similarity index 100% rename from src/test/hive/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca rename to shark/src/test/hive/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca diff --git a/src/test/hive/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 b/shark/src/test/hive/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 similarity index 100% rename from src/test/hive/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 rename to shark/src/test/hive/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 diff --git a/src/test/hive/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a b/shark/src/test/hive/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a similarity index 100% rename from src/test/hive/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a rename to shark/src/test/hive/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a diff --git a/src/test/hive/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 b/shark/src/test/hive/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 similarity index 100% rename from src/test/hive/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 rename to shark/src/test/hive/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 diff --git a/src/test/hive/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd b/shark/src/test/hive/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd similarity index 100% rename from src/test/hive/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd rename to shark/src/test/hive/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd diff --git a/src/test/hive/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 b/shark/src/test/hive/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 similarity index 100% rename from src/test/hive/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 rename to shark/src/test/hive/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 diff --git a/src/test/hive/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee b/shark/src/test/hive/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee similarity index 100% rename from src/test/hive/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee rename to shark/src/test/hive/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee diff --git a/src/test/hive/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 b/shark/src/test/hive/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 similarity index 100% rename from src/test/hive/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 rename to shark/src/test/hive/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 diff --git a/src/test/hive/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 b/shark/src/test/hive/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 similarity index 100% rename from src/test/hive/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 rename to shark/src/test/hive/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 diff --git a/src/test/hive/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f b/shark/src/test/hive/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f similarity index 100% rename from src/test/hive/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f rename to shark/src/test/hive/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f diff --git a/src/test/hive/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 b/shark/src/test/hive/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 similarity index 100% rename from src/test/hive/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 rename to shark/src/test/hive/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 diff --git a/src/test/hive/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb b/shark/src/test/hive/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb similarity index 100% rename from src/test/hive/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb rename to shark/src/test/hive/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb diff --git a/src/test/hive/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join23-1-b31437533a2b890788938455cb32f679 b/shark/src/test/hive/golden/auto_join23-1-b31437533a2b890788938455cb32f679 similarity index 100% rename from src/test/hive/golden/auto_join23-1-b31437533a2b890788938455cb32f679 rename to shark/src/test/hive/golden/auto_join23-1-b31437533a2b890788938455cb32f679 diff --git a/src/test/hive/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 b/shark/src/test/hive/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 similarity index 100% rename from src/test/hive/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 rename to shark/src/test/hive/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 diff --git a/src/test/hive/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b b/shark/src/test/hive/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b similarity index 100% rename from src/test/hive/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b rename to shark/src/test/hive/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b diff --git a/src/test/hive/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d b/shark/src/test/hive/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d similarity index 100% rename from src/test/hive/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d rename to shark/src/test/hive/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d diff --git a/src/test/hive/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 b/shark/src/test/hive/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 similarity index 100% rename from src/test/hive/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 rename to shark/src/test/hive/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 diff --git a/src/test/hive/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 b/shark/src/test/hive/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 similarity index 100% rename from src/test/hive/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 rename to shark/src/test/hive/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 diff --git a/src/test/hive/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/shark/src/test/hive/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 similarity index 100% rename from src/test/hive/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 rename to shark/src/test/hive/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 diff --git a/src/test/hive/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e b/shark/src/test/hive/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e similarity index 100% rename from src/test/hive/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e rename to shark/src/test/hive/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e diff --git a/src/test/hive/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 b/shark/src/test/hive/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 similarity index 100% rename from src/test/hive/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 rename to shark/src/test/hive/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 diff --git a/src/test/hive/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e b/shark/src/test/hive/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e similarity index 100% rename from src/test/hive/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e rename to shark/src/test/hive/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e diff --git a/src/test/hive/golden/auto_join25-2-8180638a57b64557e02815c863031755 b/shark/src/test/hive/golden/auto_join25-2-8180638a57b64557e02815c863031755 similarity index 100% rename from src/test/hive/golden/auto_join25-2-8180638a57b64557e02815c863031755 rename to shark/src/test/hive/golden/auto_join25-2-8180638a57b64557e02815c863031755 diff --git a/src/test/hive/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e b/shark/src/test/hive/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e similarity index 100% rename from src/test/hive/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e rename to shark/src/test/hive/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e diff --git a/src/test/hive/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 b/shark/src/test/hive/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 similarity index 100% rename from src/test/hive/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 rename to shark/src/test/hive/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 diff --git a/src/test/hive/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 b/shark/src/test/hive/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 similarity index 100% rename from src/test/hive/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 rename to shark/src/test/hive/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 diff --git a/src/test/hive/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f b/shark/src/test/hive/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f similarity index 100% rename from src/test/hive/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f rename to shark/src/test/hive/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f diff --git a/src/test/hive/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca b/shark/src/test/hive/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca similarity index 100% rename from src/test/hive/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca rename to shark/src/test/hive/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca diff --git a/src/test/hive/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a b/shark/src/test/hive/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a similarity index 100% rename from src/test/hive/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a rename to shark/src/test/hive/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a diff --git a/src/test/hive/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 b/shark/src/test/hive/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 similarity index 100% rename from src/test/hive/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 rename to shark/src/test/hive/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 diff --git a/src/test/hive/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 b/shark/src/test/hive/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 similarity index 100% rename from src/test/hive/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 rename to shark/src/test/hive/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 diff --git a/src/test/hive/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 b/shark/src/test/hive/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 similarity index 100% rename from src/test/hive/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 rename to shark/src/test/hive/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 diff --git a/src/test/hive/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 b/shark/src/test/hive/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 similarity index 100% rename from src/test/hive/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 rename to shark/src/test/hive/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 diff --git a/src/test/hive/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 b/shark/src/test/hive/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 similarity index 100% rename from src/test/hive/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 rename to shark/src/test/hive/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 diff --git a/src/test/hive/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e b/shark/src/test/hive/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e similarity index 100% rename from src/test/hive/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e rename to shark/src/test/hive/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e diff --git a/src/test/hive/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c b/shark/src/test/hive/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c similarity index 100% rename from src/test/hive/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c rename to shark/src/test/hive/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c diff --git a/src/test/hive/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 b/shark/src/test/hive/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 similarity index 100% rename from src/test/hive/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 rename to shark/src/test/hive/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 diff --git a/src/test/hive/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 b/shark/src/test/hive/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 similarity index 100% rename from src/test/hive/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 rename to shark/src/test/hive/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 diff --git a/src/test/hive/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 b/shark/src/test/hive/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 similarity index 100% rename from src/test/hive/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 rename to shark/src/test/hive/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 diff --git a/src/test/hive/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e b/shark/src/test/hive/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e similarity index 100% rename from src/test/hive/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e rename to shark/src/test/hive/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e diff --git a/src/test/hive/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d b/shark/src/test/hive/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d similarity index 100% rename from src/test/hive/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d rename to shark/src/test/hive/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d diff --git a/src/test/hive/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 b/shark/src/test/hive/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 similarity index 100% rename from src/test/hive/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 rename to shark/src/test/hive/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 diff --git a/src/test/hive/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f b/shark/src/test/hive/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f similarity index 100% rename from src/test/hive/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f rename to shark/src/test/hive/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f diff --git a/src/test/hive/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 b/shark/src/test/hive/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 similarity index 100% rename from src/test/hive/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 rename to shark/src/test/hive/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 diff --git a/src/test/hive/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a b/shark/src/test/hive/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a similarity index 100% rename from src/test/hive/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a rename to shark/src/test/hive/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a diff --git a/src/test/hive/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 b/shark/src/test/hive/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 similarity index 100% rename from src/test/hive/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 rename to shark/src/test/hive/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 diff --git a/src/test/hive/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d b/shark/src/test/hive/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d similarity index 100% rename from src/test/hive/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d rename to shark/src/test/hive/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d diff --git a/src/test/hive/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba b/shark/src/test/hive/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba similarity index 100% rename from src/test/hive/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba rename to shark/src/test/hive/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba diff --git a/src/test/hive/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 b/shark/src/test/hive/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 similarity index 100% rename from src/test/hive/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 rename to shark/src/test/hive/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 diff --git a/src/test/hive/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 b/shark/src/test/hive/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 similarity index 100% rename from src/test/hive/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 rename to shark/src/test/hive/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 diff --git a/src/test/hive/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 b/shark/src/test/hive/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 similarity index 100% rename from src/test/hive/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 rename to shark/src/test/hive/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 diff --git a/src/test/hive/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 b/shark/src/test/hive/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 similarity index 100% rename from src/test/hive/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 rename to shark/src/test/hive/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 diff --git a/src/test/hive/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 b/shark/src/test/hive/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 similarity index 100% rename from src/test/hive/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 rename to shark/src/test/hive/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 diff --git a/src/test/hive/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 b/shark/src/test/hive/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 similarity index 100% rename from src/test/hive/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 rename to shark/src/test/hive/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 diff --git a/src/test/hive/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 b/shark/src/test/hive/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 similarity index 100% rename from src/test/hive/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 rename to shark/src/test/hive/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 diff --git a/src/test/hive/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 b/shark/src/test/hive/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 similarity index 100% rename from src/test/hive/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 rename to shark/src/test/hive/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 diff --git a/src/test/hive/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e b/shark/src/test/hive/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e similarity index 100% rename from src/test/hive/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e rename to shark/src/test/hive/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e diff --git a/src/test/hive/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 b/shark/src/test/hive/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 similarity index 100% rename from src/test/hive/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 rename to shark/src/test/hive/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 diff --git a/src/test/hive/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 b/shark/src/test/hive/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 similarity index 100% rename from src/test/hive/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 rename to shark/src/test/hive/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 diff --git a/src/test/hive/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 b/shark/src/test/hive/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 similarity index 100% rename from src/test/hive/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 rename to shark/src/test/hive/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 diff --git a/src/test/hive/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 b/shark/src/test/hive/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 similarity index 100% rename from src/test/hive/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 rename to shark/src/test/hive/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 diff --git a/src/test/hive/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 b/shark/src/test/hive/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 similarity index 100% rename from src/test/hive/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 rename to shark/src/test/hive/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 diff --git a/src/test/hive/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 b/shark/src/test/hive/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 similarity index 100% rename from src/test/hive/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 rename to shark/src/test/hive/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 diff --git a/src/test/hive/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 b/shark/src/test/hive/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 similarity index 100% rename from src/test/hive/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 rename to shark/src/test/hive/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 diff --git a/src/test/hive/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 b/shark/src/test/hive/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 similarity index 100% rename from src/test/hive/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 rename to shark/src/test/hive/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 diff --git a/src/test/hive/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e b/shark/src/test/hive/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e similarity index 100% rename from src/test/hive/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e rename to shark/src/test/hive/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e diff --git a/src/test/hive/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f b/shark/src/test/hive/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f similarity index 100% rename from src/test/hive/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f rename to shark/src/test/hive/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f diff --git a/src/test/hive/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 b/shark/src/test/hive/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 similarity index 100% rename from src/test/hive/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 rename to shark/src/test/hive/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 diff --git a/src/test/hive/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 b/shark/src/test/hive/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 similarity index 100% rename from src/test/hive/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 rename to shark/src/test/hive/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 diff --git a/src/test/hive/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 b/shark/src/test/hive/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 similarity index 100% rename from src/test/hive/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 rename to shark/src/test/hive/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 diff --git a/src/test/hive/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f b/shark/src/test/hive/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f similarity index 100% rename from src/test/hive/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f rename to shark/src/test/hive/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f diff --git a/src/test/hive/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 b/shark/src/test/hive/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 similarity index 100% rename from src/test/hive/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 rename to shark/src/test/hive/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 diff --git a/src/test/hive/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d b/shark/src/test/hive/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d similarity index 100% rename from src/test/hive/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d rename to shark/src/test/hive/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d diff --git a/src/test/hive/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 b/shark/src/test/hive/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 similarity index 100% rename from src/test/hive/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 rename to shark/src/test/hive/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 diff --git a/src/test/hive/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b b/shark/src/test/hive/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b similarity index 100% rename from src/test/hive/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b rename to shark/src/test/hive/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b diff --git a/src/test/hive/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c b/shark/src/test/hive/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c similarity index 100% rename from src/test/hive/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c rename to shark/src/test/hive/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c diff --git a/src/test/hive/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 b/shark/src/test/hive/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 similarity index 100% rename from src/test/hive/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 rename to shark/src/test/hive/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 diff --git a/src/test/hive/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 b/shark/src/test/hive/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 similarity index 100% rename from src/test/hive/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 rename to shark/src/test/hive/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 diff --git a/src/test/hive/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b b/shark/src/test/hive/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b similarity index 100% rename from src/test/hive/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b rename to shark/src/test/hive/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b diff --git a/src/test/hive/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a b/shark/src/test/hive/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a similarity index 100% rename from src/test/hive/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a rename to shark/src/test/hive/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a diff --git a/src/test/hive/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc b/shark/src/test/hive/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc similarity index 100% rename from src/test/hive/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc rename to shark/src/test/hive/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc diff --git a/src/test/hive/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 b/shark/src/test/hive/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 similarity index 100% rename from src/test/hive/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 rename to shark/src/test/hive/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 diff --git a/src/test/hive/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 b/shark/src/test/hive/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 similarity index 100% rename from src/test/hive/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 rename to shark/src/test/hive/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 diff --git a/src/test/hive/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 b/shark/src/test/hive/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 similarity index 100% rename from src/test/hive/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 rename to shark/src/test/hive/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 diff --git a/src/test/hive/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 b/shark/src/test/hive/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 similarity index 100% rename from src/test/hive/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 rename to shark/src/test/hive/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 diff --git a/src/test/hive/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd b/shark/src/test/hive/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd similarity index 100% rename from src/test/hive/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd rename to shark/src/test/hive/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd diff --git a/src/test/hive/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf b/shark/src/test/hive/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf similarity index 100% rename from src/test/hive/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf rename to shark/src/test/hive/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf diff --git a/src/test/hive/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed b/shark/src/test/hive/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed similarity index 100% rename from src/test/hive/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed rename to shark/src/test/hive/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed diff --git a/src/test/hive/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef b/shark/src/test/hive/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef similarity index 100% rename from src/test/hive/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef rename to shark/src/test/hive/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef diff --git a/src/test/hive/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee b/shark/src/test/hive/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee similarity index 100% rename from src/test/hive/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee rename to shark/src/test/hive/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee diff --git a/src/test/hive/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a b/shark/src/test/hive/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a similarity index 100% rename from src/test/hive/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a rename to shark/src/test/hive/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a diff --git a/src/test/hive/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd b/shark/src/test/hive/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd similarity index 100% rename from src/test/hive/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd rename to shark/src/test/hive/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd diff --git a/src/test/hive/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a b/shark/src/test/hive/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a similarity index 100% rename from src/test/hive/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a rename to shark/src/test/hive/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a diff --git a/src/test/hive/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 b/shark/src/test/hive/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 similarity index 100% rename from src/test/hive/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 rename to shark/src/test/hive/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 diff --git a/src/test/hive/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 b/shark/src/test/hive/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 similarity index 100% rename from src/test/hive/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 rename to shark/src/test/hive/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 diff --git a/src/test/hive/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd b/shark/src/test/hive/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd similarity index 100% rename from src/test/hive/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd rename to shark/src/test/hive/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd diff --git a/src/test/hive/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 b/shark/src/test/hive/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 similarity index 100% rename from src/test/hive/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 rename to shark/src/test/hive/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 diff --git a/src/test/hive/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 b/shark/src/test/hive/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 similarity index 100% rename from src/test/hive/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 rename to shark/src/test/hive/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 diff --git a/src/test/hive/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 b/shark/src/test/hive/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 similarity index 100% rename from src/test/hive/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 rename to shark/src/test/hive/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 diff --git a/src/test/hive/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd b/shark/src/test/hive/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd similarity index 100% rename from src/test/hive/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd rename to shark/src/test/hive/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd diff --git a/src/test/hive/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 b/shark/src/test/hive/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 similarity index 100% rename from src/test/hive/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 rename to shark/src/test/hive/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 diff --git a/src/test/hive/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 b/shark/src/test/hive/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 similarity index 100% rename from src/test/hive/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 rename to shark/src/test/hive/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 diff --git a/src/test/hive/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 b/shark/src/test/hive/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 similarity index 100% rename from src/test/hive/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 rename to shark/src/test/hive/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 diff --git a/src/test/hive/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 b/shark/src/test/hive/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 similarity index 100% rename from src/test/hive/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 rename to shark/src/test/hive/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 diff --git a/src/test/hive/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e b/shark/src/test/hive/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e similarity index 100% rename from src/test/hive/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e rename to shark/src/test/hive/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e diff --git a/src/test/hive/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 b/shark/src/test/hive/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 similarity index 100% rename from src/test/hive/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 rename to shark/src/test/hive/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 diff --git a/src/test/hive/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 b/shark/src/test/hive/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 similarity index 100% rename from src/test/hive/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 rename to shark/src/test/hive/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 diff --git a/src/test/hive/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd b/shark/src/test/hive/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd similarity index 100% rename from src/test/hive/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd rename to shark/src/test/hive/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd diff --git a/src/test/hive/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab b/shark/src/test/hive/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab similarity index 100% rename from src/test/hive/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab rename to shark/src/test/hive/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab diff --git a/src/test/hive/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca b/shark/src/test/hive/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca similarity index 100% rename from src/test/hive/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca rename to shark/src/test/hive/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca diff --git a/src/test/hive/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 b/shark/src/test/hive/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 similarity index 100% rename from src/test/hive/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 rename to shark/src/test/hive/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 diff --git a/src/test/hive/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 b/shark/src/test/hive/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 similarity index 100% rename from src/test/hive/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 rename to shark/src/test/hive/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 diff --git a/src/test/hive/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 b/shark/src/test/hive/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 similarity index 100% rename from src/test/hive/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 rename to shark/src/test/hive/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 diff --git a/src/test/hive/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f b/shark/src/test/hive/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f similarity index 100% rename from src/test/hive/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f rename to shark/src/test/hive/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f diff --git a/src/test/hive/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 b/shark/src/test/hive/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 similarity index 100% rename from src/test/hive/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 rename to shark/src/test/hive/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 diff --git a/src/test/hive/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e b/shark/src/test/hive/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e similarity index 100% rename from src/test/hive/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e rename to shark/src/test/hive/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e diff --git a/src/test/hive/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 b/shark/src/test/hive/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 similarity index 100% rename from src/test/hive/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 rename to shark/src/test/hive/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 diff --git a/src/test/hive/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 b/shark/src/test/hive/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 similarity index 100% rename from src/test/hive/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 rename to shark/src/test/hive/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 diff --git a/src/test/hive/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 b/shark/src/test/hive/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 similarity index 100% rename from src/test/hive/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 rename to shark/src/test/hive/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 diff --git a/src/test/hive/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 b/shark/src/test/hive/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 similarity index 100% rename from src/test/hive/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 rename to shark/src/test/hive/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 diff --git a/src/test/hive/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b b/shark/src/test/hive/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b similarity index 100% rename from src/test/hive/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b rename to shark/src/test/hive/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b diff --git a/src/test/hive/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e b/shark/src/test/hive/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e similarity index 100% rename from src/test/hive/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e rename to shark/src/test/hive/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e diff --git a/src/test/hive/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 b/shark/src/test/hive/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 similarity index 100% rename from src/test/hive/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 rename to shark/src/test/hive/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 diff --git a/src/test/hive/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a b/shark/src/test/hive/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a similarity index 100% rename from src/test/hive/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a rename to shark/src/test/hive/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a diff --git a/src/test/hive/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d b/shark/src/test/hive/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d similarity index 100% rename from src/test/hive/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d rename to shark/src/test/hive/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d diff --git a/src/test/hive/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 b/shark/src/test/hive/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 similarity index 100% rename from src/test/hive/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 rename to shark/src/test/hive/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 diff --git a/src/test/hive/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 b/shark/src/test/hive/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 similarity index 100% rename from src/test/hive/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 rename to shark/src/test/hive/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 diff --git a/src/test/hive/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf b/shark/src/test/hive/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf similarity index 100% rename from src/test/hive/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf rename to shark/src/test/hive/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf diff --git a/src/test/hive/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 b/shark/src/test/hive/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 similarity index 100% rename from src/test/hive/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 rename to shark/src/test/hive/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 diff --git a/src/test/hive/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 b/shark/src/test/hive/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 similarity index 100% rename from src/test/hive/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 rename to shark/src/test/hive/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 diff --git a/src/test/hive/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 b/shark/src/test/hive/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 similarity index 100% rename from src/test/hive/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 rename to shark/src/test/hive/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 diff --git a/src/test/hive/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 b/shark/src/test/hive/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 similarity index 100% rename from src/test/hive/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 rename to shark/src/test/hive/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 diff --git a/src/test/hive/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd b/shark/src/test/hive/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd similarity index 100% rename from src/test/hive/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd rename to shark/src/test/hive/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd diff --git a/src/test/hive/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 b/shark/src/test/hive/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 similarity index 100% rename from src/test/hive/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 rename to shark/src/test/hive/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 diff --git a/src/test/hive/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 b/shark/src/test/hive/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 similarity index 100% rename from src/test/hive/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 rename to shark/src/test/hive/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 diff --git a/src/test/hive/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e b/shark/src/test/hive/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e similarity index 100% rename from src/test/hive/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e rename to shark/src/test/hive/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e diff --git a/src/test/hive/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e b/shark/src/test/hive/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e similarity index 100% rename from src/test/hive/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e rename to shark/src/test/hive/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e diff --git a/src/test/hive/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 b/shark/src/test/hive/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 similarity index 100% rename from src/test/hive/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 rename to shark/src/test/hive/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 diff --git a/src/test/hive/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b b/shark/src/test/hive/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b similarity index 100% rename from src/test/hive/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b rename to shark/src/test/hive/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b diff --git a/src/test/hive/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 b/shark/src/test/hive/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 similarity index 100% rename from src/test/hive/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 rename to shark/src/test/hive/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 diff --git a/src/test/hive/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c b/shark/src/test/hive/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c similarity index 100% rename from src/test/hive/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c rename to shark/src/test/hive/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c diff --git a/src/test/hive/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 b/shark/src/test/hive/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 similarity index 100% rename from src/test/hive/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 rename to shark/src/test/hive/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 diff --git a/src/test/hive/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 b/shark/src/test/hive/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 similarity index 100% rename from src/test/hive/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 rename to shark/src/test/hive/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 diff --git a/src/test/hive/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac b/shark/src/test/hive/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac similarity index 100% rename from src/test/hive/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac rename to shark/src/test/hive/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac diff --git a/src/test/hive/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 b/shark/src/test/hive/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 similarity index 100% rename from src/test/hive/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 rename to shark/src/test/hive/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 diff --git a/src/test/hive/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 b/shark/src/test/hive/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 similarity index 100% rename from src/test/hive/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 rename to shark/src/test/hive/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 diff --git a/src/test/hive/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada b/shark/src/test/hive/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada similarity index 100% rename from src/test/hive/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada rename to shark/src/test/hive/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada diff --git a/src/test/hive/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 b/shark/src/test/hive/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 similarity index 100% rename from src/test/hive/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 rename to shark/src/test/hive/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 diff --git a/src/test/hive/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 b/shark/src/test/hive/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 similarity index 100% rename from src/test/hive/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 rename to shark/src/test/hive/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 diff --git a/src/test/hive/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 b/shark/src/test/hive/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 similarity index 100% rename from src/test/hive/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 rename to shark/src/test/hive/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 diff --git a/src/test/hive/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 b/shark/src/test/hive/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 similarity index 100% rename from src/test/hive/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 rename to shark/src/test/hive/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 diff --git a/src/test/hive/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b b/shark/src/test/hive/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b similarity index 100% rename from src/test/hive/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b rename to shark/src/test/hive/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b diff --git a/src/test/hive/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 b/shark/src/test/hive/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 similarity index 100% rename from src/test/hive/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 rename to shark/src/test/hive/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 diff --git a/src/test/hive/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e b/shark/src/test/hive/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e similarity index 100% rename from src/test/hive/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e rename to shark/src/test/hive/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e diff --git a/src/test/hive/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 b/shark/src/test/hive/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 similarity index 100% rename from src/test/hive/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 rename to shark/src/test/hive/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 diff --git a/src/test/hive/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 b/shark/src/test/hive/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 similarity index 100% rename from src/test/hive/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 rename to shark/src/test/hive/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 diff --git a/src/test/hive/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 b/shark/src/test/hive/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 similarity index 100% rename from src/test/hive/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 rename to shark/src/test/hive/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 diff --git a/src/test/hive/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 b/shark/src/test/hive/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 similarity index 100% rename from src/test/hive/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 rename to shark/src/test/hive/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 diff --git a/src/test/hive/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc b/shark/src/test/hive/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc similarity index 100% rename from src/test/hive/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc rename to shark/src/test/hive/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc diff --git a/src/test/hive/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b b/shark/src/test/hive/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b similarity index 100% rename from src/test/hive/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b rename to shark/src/test/hive/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b diff --git a/src/test/hive/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e b/shark/src/test/hive/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e similarity index 100% rename from src/test/hive/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e rename to shark/src/test/hive/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e diff --git a/src/test/hive/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 b/shark/src/test/hive/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 similarity index 100% rename from src/test/hive/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 rename to shark/src/test/hive/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 diff --git a/src/test/hive/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a b/shark/src/test/hive/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a similarity index 100% rename from src/test/hive/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a rename to shark/src/test/hive/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a diff --git a/src/test/hive/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d b/shark/src/test/hive/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d similarity index 100% rename from src/test/hive/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d rename to shark/src/test/hive/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d diff --git a/src/test/hive/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 b/shark/src/test/hive/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 similarity index 100% rename from src/test/hive/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 rename to shark/src/test/hive/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 diff --git a/src/test/hive/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf b/shark/src/test/hive/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf similarity index 100% rename from src/test/hive/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf rename to shark/src/test/hive/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf diff --git a/src/test/hive/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 b/shark/src/test/hive/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 similarity index 100% rename from src/test/hive/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 rename to shark/src/test/hive/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 diff --git a/src/test/hive/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 b/shark/src/test/hive/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 similarity index 100% rename from src/test/hive/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 rename to shark/src/test/hive/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 diff --git a/src/test/hive/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 b/shark/src/test/hive/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 similarity index 100% rename from src/test/hive/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 rename to shark/src/test/hive/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 diff --git a/src/test/hive/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 b/shark/src/test/hive/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 similarity index 100% rename from src/test/hive/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 rename to shark/src/test/hive/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 diff --git a/src/test/hive/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 b/shark/src/test/hive/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 similarity index 100% rename from src/test/hive/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 rename to shark/src/test/hive/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 diff --git a/src/test/hive/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd b/shark/src/test/hive/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd similarity index 100% rename from src/test/hive/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd rename to shark/src/test/hive/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd diff --git a/src/test/hive/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 b/shark/src/test/hive/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 similarity index 100% rename from src/test/hive/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 rename to shark/src/test/hive/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 diff --git a/src/test/hive/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 b/shark/src/test/hive/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 similarity index 100% rename from src/test/hive/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 rename to shark/src/test/hive/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 diff --git a/src/test/hive/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 b/shark/src/test/hive/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 similarity index 100% rename from src/test/hive/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 rename to shark/src/test/hive/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 diff --git a/src/test/hive/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b b/shark/src/test/hive/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b similarity index 100% rename from src/test/hive/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b rename to shark/src/test/hive/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b diff --git a/src/test/hive/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 b/shark/src/test/hive/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 similarity index 100% rename from src/test/hive/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 rename to shark/src/test/hive/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 diff --git a/src/test/hive/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 b/shark/src/test/hive/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 rename to shark/src/test/hive/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 diff --git a/src/test/hive/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 b/shark/src/test/hive/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 rename to shark/src/test/hive/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 diff --git a/src/test/hive/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 b/shark/src/test/hive/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 rename to shark/src/test/hive/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 diff --git a/src/test/hive/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c b/shark/src/test/hive/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c similarity index 100% rename from src/test/hive/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c rename to shark/src/test/hive/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c diff --git a/src/test/hive/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 b/shark/src/test/hive/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 rename to shark/src/test/hive/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 diff --git a/src/test/hive/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 b/shark/src/test/hive/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 rename to shark/src/test/hive/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 diff --git a/src/test/hive/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 b/shark/src/test/hive/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 rename to shark/src/test/hive/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 diff --git a/src/test/hive/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af b/shark/src/test/hive/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af similarity index 100% rename from src/test/hive/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af rename to shark/src/test/hive/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af diff --git a/src/test/hive/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 b/shark/src/test/hive/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 rename to shark/src/test/hive/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 diff --git a/src/test/hive/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb b/shark/src/test/hive/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb similarity index 100% rename from src/test/hive/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb rename to shark/src/test/hive/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb diff --git a/src/test/hive/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 b/shark/src/test/hive/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 rename to shark/src/test/hive/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 diff --git a/src/test/hive/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 b/shark/src/test/hive/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 rename to shark/src/test/hive/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 diff --git a/src/test/hive/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d b/shark/src/test/hive/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d similarity index 100% rename from src/test/hive/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d rename to shark/src/test/hive/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d diff --git a/src/test/hive/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a b/shark/src/test/hive/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a similarity index 100% rename from src/test/hive/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a rename to shark/src/test/hive/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a diff --git a/src/test/hive/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 b/shark/src/test/hive/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 rename to shark/src/test/hive/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 diff --git a/src/test/hive/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa b/shark/src/test/hive/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa similarity index 100% rename from src/test/hive/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa rename to shark/src/test/hive/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa diff --git a/src/test/hive/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 b/shark/src/test/hive/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 rename to shark/src/test/hive/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 diff --git a/src/test/hive/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 b/shark/src/test/hive/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 rename to shark/src/test/hive/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 diff --git a/src/test/hive/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 b/shark/src/test/hive/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 rename to shark/src/test/hive/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 diff --git a/src/test/hive/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc b/shark/src/test/hive/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc similarity index 100% rename from src/test/hive/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc rename to shark/src/test/hive/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc diff --git a/src/test/hive/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 b/shark/src/test/hive/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 rename to shark/src/test/hive/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 diff --git a/src/test/hive/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 b/shark/src/test/hive/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 rename to shark/src/test/hive/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 diff --git a/src/test/hive/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 b/shark/src/test/hive/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 similarity index 100% rename from src/test/hive/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 rename to shark/src/test/hive/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 diff --git a/src/test/hive/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e b/shark/src/test/hive/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e similarity index 100% rename from src/test/hive/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e rename to shark/src/test/hive/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e diff --git a/src/test/hive/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 b/shark/src/test/hive/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 rename to shark/src/test/hive/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 diff --git a/src/test/hive/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 b/shark/src/test/hive/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 rename to shark/src/test/hive/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 diff --git a/src/test/hive/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 b/shark/src/test/hive/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 rename to shark/src/test/hive/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 diff --git a/src/test/hive/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 b/shark/src/test/hive/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 rename to shark/src/test/hive/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 diff --git a/src/test/hive/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 b/shark/src/test/hive/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 rename to shark/src/test/hive/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 diff --git a/src/test/hive/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 b/shark/src/test/hive/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 rename to shark/src/test/hive/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 diff --git a/src/test/hive/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee b/shark/src/test/hive/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee rename to shark/src/test/hive/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee diff --git a/src/test/hive/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 b/shark/src/test/hive/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 rename to shark/src/test/hive/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 diff --git a/src/test/hive/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b b/shark/src/test/hive/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b rename to shark/src/test/hive/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b diff --git a/src/test/hive/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed b/shark/src/test/hive/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed rename to shark/src/test/hive/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed diff --git a/src/test/hive/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b b/shark/src/test/hive/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b similarity index 100% rename from src/test/hive/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b rename to shark/src/test/hive/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a b/shark/src/test/hive/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d b/shark/src/test/hive/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da b/shark/src/test/hive/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce b/shark/src/test/hive/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce b/shark/src/test/hive/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b b/shark/src/test/hive/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e b/shark/src/test/hive/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da b/shark/src/test/hive/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e b/shark/src/test/hive/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c b/shark/src/test/hive/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c b/shark/src/test/hive/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e b/shark/src/test/hive/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f b/shark/src/test/hive/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f b/shark/src/test/hive/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 b/shark/src/test/hive/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 b/shark/src/test/hive/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 b/shark/src/test/hive/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 b/shark/src/test/hive/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc b/shark/src/test/hive/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc rename to shark/src/test/hive/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc diff --git a/src/test/hive/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 b/shark/src/test/hive/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 b/shark/src/test/hive/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/shark/src/test/hive/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db b/shark/src/test/hive/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db rename to shark/src/test/hive/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db diff --git a/src/test/hive/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a b/shark/src/test/hive/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a rename to shark/src/test/hive/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a diff --git a/src/test/hive/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab b/shark/src/test/hive/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab rename to shark/src/test/hive/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab diff --git a/src/test/hive/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 b/shark/src/test/hive/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 b/shark/src/test/hive/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 diff --git a/src/test/hive/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d b/shark/src/test/hive/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d rename to shark/src/test/hive/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d diff --git a/src/test/hive/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/shark/src/test/hive/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to shark/src/test/hive/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 diff --git a/src/test/hive/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 b/shark/src/test/hive/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 rename to shark/src/test/hive/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 diff --git a/src/test/hive/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 b/shark/src/test/hive/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 rename to shark/src/test/hive/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 diff --git a/src/test/hive/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a b/shark/src/test/hive/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a rename to shark/src/test/hive/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a diff --git a/src/test/hive/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e b/shark/src/test/hive/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e rename to shark/src/test/hive/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e diff --git a/src/test/hive/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da b/shark/src/test/hive/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da rename to shark/src/test/hive/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da diff --git a/src/test/hive/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 b/shark/src/test/hive/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 rename to shark/src/test/hive/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 diff --git a/src/test/hive/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f b/shark/src/test/hive/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f rename to shark/src/test/hive/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f diff --git a/src/test/hive/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f b/shark/src/test/hive/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f rename to shark/src/test/hive/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f diff --git a/src/test/hive/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a b/shark/src/test/hive/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a rename to shark/src/test/hive/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a diff --git a/src/test/hive/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 b/shark/src/test/hive/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 b/shark/src/test/hive/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 b/shark/src/test/hive/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd b/shark/src/test/hive/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd rename to shark/src/test/hive/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd diff --git a/src/test/hive/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 b/shark/src/test/hive/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed b/shark/src/test/hive/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed rename to shark/src/test/hive/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed diff --git a/src/test/hive/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef b/shark/src/test/hive/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef rename to shark/src/test/hive/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef diff --git a/src/test/hive/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e b/shark/src/test/hive/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e rename to shark/src/test/hive/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e diff --git a/src/test/hive/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 b/shark/src/test/hive/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/shark/src/test/hive/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 b/shark/src/test/hive/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa b/shark/src/test/hive/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa rename to shark/src/test/hive/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa diff --git a/src/test/hive/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 b/shark/src/test/hive/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a b/shark/src/test/hive/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a rename to shark/src/test/hive/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a diff --git a/src/test/hive/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab b/shark/src/test/hive/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab rename to shark/src/test/hive/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab diff --git a/src/test/hive/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 b/shark/src/test/hive/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 b/shark/src/test/hive/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 diff --git a/src/test/hive/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d b/shark/src/test/hive/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d rename to shark/src/test/hive/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d diff --git a/src/test/hive/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/shark/src/test/hive/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to shark/src/test/hive/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec b/shark/src/test/hive/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec rename to shark/src/test/hive/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec diff --git a/src/test/hive/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 b/shark/src/test/hive/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 b/shark/src/test/hive/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 b/shark/src/test/hive/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e b/shark/src/test/hive/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e rename to shark/src/test/hive/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e diff --git a/src/test/hive/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a b/shark/src/test/hive/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a rename to shark/src/test/hive/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a diff --git a/src/test/hive/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 b/shark/src/test/hive/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 b/shark/src/test/hive/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/shark/src/test/hive/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 b/shark/src/test/hive/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc b/shark/src/test/hive/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc rename to shark/src/test/hive/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc diff --git a/src/test/hive/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db b/shark/src/test/hive/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db rename to shark/src/test/hive/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db diff --git a/src/test/hive/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a b/shark/src/test/hive/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a rename to shark/src/test/hive/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a diff --git a/src/test/hive/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab b/shark/src/test/hive/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab rename to shark/src/test/hive/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab diff --git a/src/test/hive/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 b/shark/src/test/hive/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 b/shark/src/test/hive/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 diff --git a/src/test/hive/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d b/shark/src/test/hive/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d rename to shark/src/test/hive/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d diff --git a/src/test/hive/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/shark/src/test/hive/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to shark/src/test/hive/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 b/shark/src/test/hive/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 b/shark/src/test/hive/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 b/shark/src/test/hive/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 b/shark/src/test/hive/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f b/shark/src/test/hive/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f rename to shark/src/test/hive/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f diff --git a/src/test/hive/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 b/shark/src/test/hive/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 b/shark/src/test/hive/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 b/shark/src/test/hive/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 b/shark/src/test/hive/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 b/shark/src/test/hive/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 b/shark/src/test/hive/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 b/shark/src/test/hive/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da b/shark/src/test/hive/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da rename to shark/src/test/hive/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da diff --git a/src/test/hive/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 b/shark/src/test/hive/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 b/shark/src/test/hive/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 b/shark/src/test/hive/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 b/shark/src/test/hive/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 b/shark/src/test/hive/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 diff --git a/src/test/hive/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb b/shark/src/test/hive/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb rename to shark/src/test/hive/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb diff --git a/src/test/hive/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 b/shark/src/test/hive/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a b/shark/src/test/hive/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a rename to shark/src/test/hive/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a diff --git a/src/test/hive/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c b/shark/src/test/hive/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c rename to shark/src/test/hive/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c diff --git a/src/test/hive/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 b/shark/src/test/hive/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd b/shark/src/test/hive/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd rename to shark/src/test/hive/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd diff --git a/src/test/hive/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da b/shark/src/test/hive/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da rename to shark/src/test/hive/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da diff --git a/src/test/hive/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 b/shark/src/test/hive/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac b/shark/src/test/hive/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac rename to shark/src/test/hive/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac diff --git a/src/test/hive/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 b/shark/src/test/hive/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 b/shark/src/test/hive/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 b/shark/src/test/hive/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da b/shark/src/test/hive/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da rename to shark/src/test/hive/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da diff --git a/src/test/hive/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 b/shark/src/test/hive/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac b/shark/src/test/hive/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac rename to shark/src/test/hive/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac diff --git a/src/test/hive/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 b/shark/src/test/hive/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d b/shark/src/test/hive/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d rename to shark/src/test/hive/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 b/shark/src/test/hive/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 b/shark/src/test/hive/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce b/shark/src/test/hive/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce rename to shark/src/test/hive/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce diff --git a/src/test/hive/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/shark/src/test/hive/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/shark/src/test/hive/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 b/shark/src/test/hive/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 b/shark/src/test/hive/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a b/shark/src/test/hive/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a rename to shark/src/test/hive/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a diff --git a/src/test/hive/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 b/shark/src/test/hive/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 b/shark/src/test/hive/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 diff --git a/src/test/hive/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a b/shark/src/test/hive/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a rename to shark/src/test/hive/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a diff --git a/src/test/hive/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab b/shark/src/test/hive/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab rename to shark/src/test/hive/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab diff --git a/src/test/hive/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d b/shark/src/test/hive/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d rename to shark/src/test/hive/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d diff --git a/src/test/hive/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/shark/src/test/hive/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to shark/src/test/hive/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 b/shark/src/test/hive/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 b/shark/src/test/hive/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc b/shark/src/test/hive/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc rename to shark/src/test/hive/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc diff --git a/src/test/hive/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 b/shark/src/test/hive/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 b/shark/src/test/hive/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/shark/src/test/hive/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 b/shark/src/test/hive/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f b/shark/src/test/hive/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f rename to shark/src/test/hive/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f diff --git a/src/test/hive/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db b/shark/src/test/hive/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db rename to shark/src/test/hive/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db diff --git a/src/test/hive/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a b/shark/src/test/hive/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a rename to shark/src/test/hive/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a diff --git a/src/test/hive/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab b/shark/src/test/hive/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab rename to shark/src/test/hive/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab diff --git a/src/test/hive/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 b/shark/src/test/hive/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 diff --git a/src/test/hive/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 b/shark/src/test/hive/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 rename to shark/src/test/hive/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c b/shark/src/test/hive/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c rename to shark/src/test/hive/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c diff --git a/src/test/hive/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 b/shark/src/test/hive/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a b/shark/src/test/hive/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a rename to shark/src/test/hive/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a diff --git a/src/test/hive/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab b/shark/src/test/hive/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab rename to shark/src/test/hive/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab diff --git a/src/test/hive/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 b/shark/src/test/hive/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 b/shark/src/test/hive/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 b/shark/src/test/hive/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/shark/src/test/hive/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a b/shark/src/test/hive/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a rename to shark/src/test/hive/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a diff --git a/src/test/hive/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 b/shark/src/test/hive/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 b/shark/src/test/hive/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f b/shark/src/test/hive/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f rename to shark/src/test/hive/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f diff --git a/src/test/hive/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 b/shark/src/test/hive/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 b/shark/src/test/hive/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 diff --git a/src/test/hive/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 b/shark/src/test/hive/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 rename to shark/src/test/hive/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d b/shark/src/test/hive/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d rename to shark/src/test/hive/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 b/shark/src/test/hive/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 b/shark/src/test/hive/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 b/shark/src/test/hive/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 b/shark/src/test/hive/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 b/shark/src/test/hive/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c b/shark/src/test/hive/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c rename to shark/src/test/hive/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c diff --git a/src/test/hive/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 b/shark/src/test/hive/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 b/shark/src/test/hive/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 b/shark/src/test/hive/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 rename to shark/src/test/hive/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 diff --git a/src/test/hive/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a b/shark/src/test/hive/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a rename to shark/src/test/hive/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a diff --git a/src/test/hive/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 b/shark/src/test/hive/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f b/shark/src/test/hive/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f rename to shark/src/test/hive/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f diff --git a/src/test/hive/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 b/shark/src/test/hive/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 b/shark/src/test/hive/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd b/shark/src/test/hive/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd rename to shark/src/test/hive/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd diff --git a/src/test/hive/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 b/shark/src/test/hive/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c b/shark/src/test/hive/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c rename to shark/src/test/hive/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c diff --git a/src/test/hive/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 b/shark/src/test/hive/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f b/shark/src/test/hive/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f rename to shark/src/test/hive/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f diff --git a/src/test/hive/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b b/shark/src/test/hive/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b rename to shark/src/test/hive/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b diff --git a/src/test/hive/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b b/shark/src/test/hive/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b rename to shark/src/test/hive/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b diff --git a/src/test/hive/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 b/shark/src/test/hive/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 b/shark/src/test/hive/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 b/shark/src/test/hive/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da b/shark/src/test/hive/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da rename to shark/src/test/hive/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da diff --git a/src/test/hive/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf b/shark/src/test/hive/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf rename to shark/src/test/hive/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf diff --git a/src/test/hive/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 b/shark/src/test/hive/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f b/shark/src/test/hive/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f rename to shark/src/test/hive/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f diff --git a/src/test/hive/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b b/shark/src/test/hive/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b rename to shark/src/test/hive/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b diff --git a/src/test/hive/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b b/shark/src/test/hive/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b rename to shark/src/test/hive/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b diff --git a/src/test/hive/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 b/shark/src/test/hive/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 b/shark/src/test/hive/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 b/shark/src/test/hive/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 b/shark/src/test/hive/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 b/shark/src/test/hive/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 b/shark/src/test/hive/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 b/shark/src/test/hive/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 b/shark/src/test/hive/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 rename to shark/src/test/hive/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 diff --git a/src/test/hive/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b b/shark/src/test/hive/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b rename to shark/src/test/hive/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b diff --git a/src/test/hive/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 b/shark/src/test/hive/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 b/shark/src/test/hive/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a b/shark/src/test/hive/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a rename to shark/src/test/hive/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a diff --git a/src/test/hive/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab b/shark/src/test/hive/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab rename to shark/src/test/hive/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab diff --git a/src/test/hive/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d b/shark/src/test/hive/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d rename to shark/src/test/hive/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 b/shark/src/test/hive/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 b/shark/src/test/hive/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/shark/src/test/hive/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 b/shark/src/test/hive/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 b/shark/src/test/hive/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a b/shark/src/test/hive/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a rename to shark/src/test/hive/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a diff --git a/src/test/hive/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 b/shark/src/test/hive/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 b/shark/src/test/hive/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f b/shark/src/test/hive/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f rename to shark/src/test/hive/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f diff --git a/src/test/hive/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 b/shark/src/test/hive/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 b/shark/src/test/hive/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 diff --git a/src/test/hive/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 b/shark/src/test/hive/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 rename to shark/src/test/hive/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa b/shark/src/test/hive/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa rename to shark/src/test/hive/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa diff --git a/src/test/hive/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 b/shark/src/test/hive/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d b/shark/src/test/hive/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d rename to shark/src/test/hive/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 b/shark/src/test/hive/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 b/shark/src/test/hive/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 b/shark/src/test/hive/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc b/shark/src/test/hive/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc rename to shark/src/test/hive/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc diff --git a/src/test/hive/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/shark/src/test/hive/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 b/shark/src/test/hive/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 b/shark/src/test/hive/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 b/shark/src/test/hive/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/shark/src/test/hive/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 b/shark/src/test/hive/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 b/shark/src/test/hive/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 b/shark/src/test/hive/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 b/shark/src/test/hive/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f b/shark/src/test/hive/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f rename to shark/src/test/hive/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f diff --git a/src/test/hive/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db b/shark/src/test/hive/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db rename to shark/src/test/hive/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db diff --git a/src/test/hive/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a b/shark/src/test/hive/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a rename to shark/src/test/hive/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a diff --git a/src/test/hive/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab b/shark/src/test/hive/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab rename to shark/src/test/hive/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab diff --git a/src/test/hive/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 b/shark/src/test/hive/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 diff --git a/src/test/hive/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 b/shark/src/test/hive/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 rename to shark/src/test/hive/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d b/shark/src/test/hive/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d rename to shark/src/test/hive/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 b/shark/src/test/hive/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 b/shark/src/test/hive/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e b/shark/src/test/hive/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e rename to shark/src/test/hive/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e diff --git a/src/test/hive/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 b/shark/src/test/hive/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 b/shark/src/test/hive/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 b/shark/src/test/hive/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 b/shark/src/test/hive/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 b/shark/src/test/hive/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 b/shark/src/test/hive/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 b/shark/src/test/hive/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 b/shark/src/test/hive/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 b/shark/src/test/hive/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 b/shark/src/test/hive/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a b/shark/src/test/hive/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a rename to shark/src/test/hive/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a diff --git a/src/test/hive/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 b/shark/src/test/hive/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 b/shark/src/test/hive/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 b/shark/src/test/hive/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 b/shark/src/test/hive/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da b/shark/src/test/hive/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da rename to shark/src/test/hive/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da diff --git a/src/test/hive/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d b/shark/src/test/hive/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d rename to shark/src/test/hive/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 b/shark/src/test/hive/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 b/shark/src/test/hive/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e b/shark/src/test/hive/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e rename to shark/src/test/hive/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e diff --git a/src/test/hive/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d b/shark/src/test/hive/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d rename to shark/src/test/hive/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce b/shark/src/test/hive/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce rename to shark/src/test/hive/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce diff --git a/src/test/hive/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce b/shark/src/test/hive/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce rename to shark/src/test/hive/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce diff --git a/src/test/hive/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 b/shark/src/test/hive/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 b/shark/src/test/hive/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 b/shark/src/test/hive/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 b/shark/src/test/hive/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 b/shark/src/test/hive/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e b/shark/src/test/hive/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e rename to shark/src/test/hive/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e diff --git a/src/test/hive/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 b/shark/src/test/hive/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 b/shark/src/test/hive/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 b/shark/src/test/hive/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 b/shark/src/test/hive/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 b/shark/src/test/hive/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 b/shark/src/test/hive/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 b/shark/src/test/hive/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 b/shark/src/test/hive/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f b/shark/src/test/hive/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f rename to shark/src/test/hive/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f diff --git a/src/test/hive/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 b/shark/src/test/hive/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 b/shark/src/test/hive/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a b/shark/src/test/hive/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a rename to shark/src/test/hive/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a diff --git a/src/test/hive/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 b/shark/src/test/hive/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 b/shark/src/test/hive/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d b/shark/src/test/hive/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d rename to shark/src/test/hive/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 b/shark/src/test/hive/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 b/shark/src/test/hive/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e b/shark/src/test/hive/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e rename to shark/src/test/hive/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e diff --git a/src/test/hive/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d b/shark/src/test/hive/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d rename to shark/src/test/hive/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d diff --git a/src/test/hive/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f b/shark/src/test/hive/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f rename to shark/src/test/hive/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f diff --git a/src/test/hive/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce b/shark/src/test/hive/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce rename to shark/src/test/hive/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce diff --git a/src/test/hive/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce b/shark/src/test/hive/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce rename to shark/src/test/hive/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce diff --git a/src/test/hive/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 b/shark/src/test/hive/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 b/shark/src/test/hive/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 similarity index 100% rename from src/test/hive/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 rename to shark/src/test/hive/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 diff --git a/src/test/hive/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 b/shark/src/test/hive/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 similarity index 100% rename from src/test/hive/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 rename to shark/src/test/hive/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 diff --git a/src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 b/shark/src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 similarity index 100% rename from src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 rename to shark/src/test/hive/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 diff --git a/src/test/hive/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 b/shark/src/test/hive/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 similarity index 100% rename from src/test/hive/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 rename to shark/src/test/hive/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 diff --git a/src/test/hive/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a b/shark/src/test/hive/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a similarity index 100% rename from src/test/hive/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a rename to shark/src/test/hive/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a diff --git a/src/test/hive/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 b/shark/src/test/hive/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 similarity index 100% rename from src/test/hive/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 rename to shark/src/test/hive/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 diff --git a/src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 b/shark/src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 similarity index 100% rename from src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 rename to shark/src/test/hive/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 diff --git a/src/test/hive/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 b/shark/src/test/hive/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 similarity index 100% rename from src/test/hive/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 rename to shark/src/test/hive/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 diff --git a/src/test/hive/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d b/shark/src/test/hive/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d similarity index 100% rename from src/test/hive/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d rename to shark/src/test/hive/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d diff --git a/src/test/hive/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 b/shark/src/test/hive/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 similarity index 100% rename from src/test/hive/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 rename to shark/src/test/hive/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 diff --git a/src/test/hive/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b b/shark/src/test/hive/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b similarity index 100% rename from src/test/hive/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b rename to shark/src/test/hive/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b diff --git a/src/test/hive/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 b/shark/src/test/hive/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 similarity index 100% rename from src/test/hive/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 rename to shark/src/test/hive/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 diff --git a/src/test/hive/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 b/shark/src/test/hive/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 similarity index 100% rename from src/test/hive/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 rename to shark/src/test/hive/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 diff --git a/src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 b/shark/src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 similarity index 100% rename from src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 rename to shark/src/test/hive/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 diff --git a/src/test/hive/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 b/shark/src/test/hive/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 similarity index 100% rename from src/test/hive/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 rename to shark/src/test/hive/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 diff --git a/src/test/hive/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 b/shark/src/test/hive/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 similarity index 100% rename from src/test/hive/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 rename to shark/src/test/hive/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 diff --git a/src/test/hive/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef b/shark/src/test/hive/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef similarity index 100% rename from src/test/hive/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef rename to shark/src/test/hive/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef diff --git a/src/test/hive/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 b/shark/src/test/hive/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 similarity index 100% rename from src/test/hive/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 rename to shark/src/test/hive/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 diff --git a/src/test/hive/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 b/shark/src/test/hive/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 similarity index 100% rename from src/test/hive/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 rename to shark/src/test/hive/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 diff --git a/src/test/hive/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/shark/src/test/hive/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 similarity index 100% rename from src/test/hive/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 rename to shark/src/test/hive/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 diff --git a/src/test/hive/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 b/shark/src/test/hive/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 similarity index 100% rename from src/test/hive/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 rename to shark/src/test/hive/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 diff --git a/src/test/hive/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e b/shark/src/test/hive/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e similarity index 100% rename from src/test/hive/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e rename to shark/src/test/hive/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e diff --git a/src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 b/shark/src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 similarity index 100% rename from src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 rename to shark/src/test/hive/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 diff --git a/src/test/hive/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/shark/src/test/hive/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 similarity index 100% rename from src/test/hive/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 rename to shark/src/test/hive/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 diff --git a/src/test/hive/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 b/shark/src/test/hive/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 similarity index 100% rename from src/test/hive/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 rename to shark/src/test/hive/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 diff --git a/src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 b/shark/src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 similarity index 100% rename from src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 rename to shark/src/test/hive/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 diff --git a/src/test/hive/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 b/shark/src/test/hive/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 similarity index 100% rename from src/test/hive/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 rename to shark/src/test/hive/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 diff --git a/src/test/hive/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 b/shark/src/test/hive/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 similarity index 100% rename from src/test/hive/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 rename to shark/src/test/hive/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 diff --git a/src/test/hive/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 b/shark/src/test/hive/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 similarity index 100% rename from src/test/hive/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 rename to shark/src/test/hive/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 diff --git a/src/test/hive/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 b/shark/src/test/hive/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 similarity index 100% rename from src/test/hive/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 rename to shark/src/test/hive/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 diff --git a/src/test/hive/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 b/shark/src/test/hive/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 similarity index 100% rename from src/test/hive/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 rename to shark/src/test/hive/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 diff --git a/src/test/hive/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 b/shark/src/test/hive/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 similarity index 100% rename from src/test/hive/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 rename to shark/src/test/hive/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 diff --git a/src/test/hive/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c b/shark/src/test/hive/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c similarity index 100% rename from src/test/hive/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c rename to shark/src/test/hive/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c diff --git a/src/test/hive/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a b/shark/src/test/hive/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a similarity index 100% rename from src/test/hive/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a rename to shark/src/test/hive/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a diff --git a/src/test/hive/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 b/shark/src/test/hive/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 similarity index 100% rename from src/test/hive/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 rename to shark/src/test/hive/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 diff --git a/src/test/hive/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 b/shark/src/test/hive/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 similarity index 100% rename from src/test/hive/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 rename to shark/src/test/hive/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 diff --git a/src/test/hive/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 b/shark/src/test/hive/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 similarity index 100% rename from src/test/hive/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 rename to shark/src/test/hive/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 diff --git a/src/test/hive/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 b/shark/src/test/hive/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 similarity index 100% rename from src/test/hive/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 rename to shark/src/test/hive/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 diff --git a/src/test/hive/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e b/shark/src/test/hive/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e similarity index 100% rename from src/test/hive/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e rename to shark/src/test/hive/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e diff --git a/src/test/hive/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 b/shark/src/test/hive/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 similarity index 100% rename from src/test/hive/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 rename to shark/src/test/hive/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 diff --git a/src/test/hive/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 b/shark/src/test/hive/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 similarity index 100% rename from src/test/hive/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 rename to shark/src/test/hive/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 diff --git a/src/test/hive/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 b/shark/src/test/hive/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 similarity index 100% rename from src/test/hive/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 rename to shark/src/test/hive/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 diff --git a/src/test/hive/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 b/shark/src/test/hive/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 similarity index 100% rename from src/test/hive/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 rename to shark/src/test/hive/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 diff --git a/src/test/hive/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 b/shark/src/test/hive/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 similarity index 100% rename from src/test/hive/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 rename to shark/src/test/hive/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 diff --git a/src/test/hive/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 b/shark/src/test/hive/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 similarity index 100% rename from src/test/hive/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 rename to shark/src/test/hive/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 diff --git a/src/test/hive/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 b/shark/src/test/hive/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 similarity index 100% rename from src/test/hive/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 rename to shark/src/test/hive/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 diff --git a/src/test/hive/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 b/shark/src/test/hive/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 similarity index 100% rename from src/test/hive/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 rename to shark/src/test/hive/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 diff --git a/src/test/hive/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 b/shark/src/test/hive/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 similarity index 100% rename from src/test/hive/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 rename to shark/src/test/hive/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 diff --git a/src/test/hive/golden/cluster-20-294891c1d956245540a80aa800ba393d b/shark/src/test/hive/golden/cluster-20-294891c1d956245540a80aa800ba393d similarity index 100% rename from src/test/hive/golden/cluster-20-294891c1d956245540a80aa800ba393d rename to shark/src/test/hive/golden/cluster-20-294891c1d956245540a80aa800ba393d diff --git a/src/test/hive/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f b/shark/src/test/hive/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f similarity index 100% rename from src/test/hive/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f rename to shark/src/test/hive/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f diff --git a/src/test/hive/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 b/shark/src/test/hive/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 similarity index 100% rename from src/test/hive/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 rename to shark/src/test/hive/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 diff --git a/src/test/hive/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e b/shark/src/test/hive/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e similarity index 100% rename from src/test/hive/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e rename to shark/src/test/hive/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e diff --git a/src/test/hive/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae b/shark/src/test/hive/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae similarity index 100% rename from src/test/hive/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae rename to shark/src/test/hive/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae diff --git a/src/test/hive/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a b/shark/src/test/hive/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a similarity index 100% rename from src/test/hive/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a rename to shark/src/test/hive/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a diff --git a/src/test/hive/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f b/shark/src/test/hive/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f similarity index 100% rename from src/test/hive/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f rename to shark/src/test/hive/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f diff --git a/src/test/hive/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc b/shark/src/test/hive/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc similarity index 100% rename from src/test/hive/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc rename to shark/src/test/hive/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc diff --git a/src/test/hive/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 b/shark/src/test/hive/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 similarity index 100% rename from src/test/hive/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 rename to shark/src/test/hive/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 diff --git a/src/test/hive/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 b/shark/src/test/hive/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 similarity index 100% rename from src/test/hive/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 rename to shark/src/test/hive/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 diff --git a/src/test/hive/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 b/shark/src/test/hive/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 similarity index 100% rename from src/test/hive/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 rename to shark/src/test/hive/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 diff --git a/src/test/hive/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d b/shark/src/test/hive/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d similarity index 100% rename from src/test/hive/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d rename to shark/src/test/hive/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d diff --git a/src/test/hive/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 b/shark/src/test/hive/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 similarity index 100% rename from src/test/hive/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 rename to shark/src/test/hive/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 diff --git a/src/test/hive/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d b/shark/src/test/hive/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d similarity index 100% rename from src/test/hive/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d rename to shark/src/test/hive/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d diff --git a/src/test/hive/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c b/shark/src/test/hive/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c similarity index 100% rename from src/test/hive/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c rename to shark/src/test/hive/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c diff --git a/src/test/hive/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 b/shark/src/test/hive/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 similarity index 100% rename from src/test/hive/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 rename to shark/src/test/hive/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 diff --git a/src/test/hive/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f b/shark/src/test/hive/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f similarity index 100% rename from src/test/hive/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f rename to shark/src/test/hive/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f diff --git a/src/test/hive/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c b/shark/src/test/hive/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c similarity index 100% rename from src/test/hive/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c rename to shark/src/test/hive/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c diff --git a/src/test/hive/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 b/shark/src/test/hive/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 similarity index 100% rename from src/test/hive/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 rename to shark/src/test/hive/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 diff --git a/src/test/hive/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 b/shark/src/test/hive/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 similarity index 100% rename from src/test/hive/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 rename to shark/src/test/hive/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 diff --git a/src/test/hive/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd b/shark/src/test/hive/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd similarity index 100% rename from src/test/hive/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd rename to shark/src/test/hive/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd diff --git a/src/test/hive/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 b/shark/src/test/hive/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 similarity index 100% rename from src/test/hive/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 rename to shark/src/test/hive/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 diff --git a/src/test/hive/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 b/shark/src/test/hive/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 similarity index 100% rename from src/test/hive/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 rename to shark/src/test/hive/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 diff --git a/src/test/hive/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 b/shark/src/test/hive/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 similarity index 100% rename from src/test/hive/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 rename to shark/src/test/hive/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 diff --git a/src/test/hive/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 b/shark/src/test/hive/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 similarity index 100% rename from src/test/hive/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 rename to shark/src/test/hive/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 diff --git a/src/test/hive/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce b/shark/src/test/hive/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce similarity index 100% rename from src/test/hive/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce rename to shark/src/test/hive/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce diff --git a/src/test/hive/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe b/shark/src/test/hive/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe similarity index 100% rename from src/test/hive/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe rename to shark/src/test/hive/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe diff --git a/src/test/hive/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 b/shark/src/test/hive/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 similarity index 100% rename from src/test/hive/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 rename to shark/src/test/hive/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 diff --git a/src/test/hive/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c b/shark/src/test/hive/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c similarity index 100% rename from src/test/hive/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c rename to shark/src/test/hive/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c diff --git a/src/test/hive/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 b/shark/src/test/hive/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 similarity index 100% rename from src/test/hive/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 rename to shark/src/test/hive/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 diff --git a/src/test/hive/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 b/shark/src/test/hive/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 similarity index 100% rename from src/test/hive/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 rename to shark/src/test/hive/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 diff --git a/src/test/hive/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 b/shark/src/test/hive/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 similarity index 100% rename from src/test/hive/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 rename to shark/src/test/hive/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 diff --git a/src/test/hive/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c b/shark/src/test/hive/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c similarity index 100% rename from src/test/hive/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c rename to shark/src/test/hive/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c diff --git a/src/test/hive/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 b/shark/src/test/hive/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 similarity index 100% rename from src/test/hive/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 rename to shark/src/test/hive/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 diff --git a/src/test/hive/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d b/shark/src/test/hive/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d similarity index 100% rename from src/test/hive/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d rename to shark/src/test/hive/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d diff --git a/src/test/hive/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f b/shark/src/test/hive/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f similarity index 100% rename from src/test/hive/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f rename to shark/src/test/hive/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f diff --git a/src/test/hive/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 b/shark/src/test/hive/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 similarity index 100% rename from src/test/hive/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 rename to shark/src/test/hive/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 diff --git a/src/test/hive/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf b/shark/src/test/hive/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf similarity index 100% rename from src/test/hive/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf rename to shark/src/test/hive/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf diff --git a/src/test/hive/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 b/shark/src/test/hive/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 similarity index 100% rename from src/test/hive/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 rename to shark/src/test/hive/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 diff --git a/src/test/hive/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 b/shark/src/test/hive/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 similarity index 100% rename from src/test/hive/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 rename to shark/src/test/hive/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 diff --git a/src/test/hive/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea b/shark/src/test/hive/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea similarity index 100% rename from src/test/hive/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea rename to shark/src/test/hive/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea diff --git a/src/test/hive/golden/combine1-6-1d1f97cce07323812de3027920b04b75 b/shark/src/test/hive/golden/combine1-6-1d1f97cce07323812de3027920b04b75 similarity index 100% rename from src/test/hive/golden/combine1-6-1d1f97cce07323812de3027920b04b75 rename to shark/src/test/hive/golden/combine1-6-1d1f97cce07323812de3027920b04b75 diff --git a/src/test/hive/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 b/shark/src/test/hive/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 similarity index 100% rename from src/test/hive/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 rename to shark/src/test/hive/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 diff --git a/src/test/hive/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c b/shark/src/test/hive/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c similarity index 100% rename from src/test/hive/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c rename to shark/src/test/hive/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c diff --git a/src/test/hive/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 b/shark/src/test/hive/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 similarity index 100% rename from src/test/hive/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 rename to shark/src/test/hive/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 diff --git a/src/test/hive/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 b/shark/src/test/hive/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 similarity index 100% rename from src/test/hive/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 rename to shark/src/test/hive/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 diff --git a/src/test/hive/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a b/shark/src/test/hive/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a similarity index 100% rename from src/test/hive/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a rename to shark/src/test/hive/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a diff --git a/src/test/hive/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 b/shark/src/test/hive/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 similarity index 100% rename from src/test/hive/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 rename to shark/src/test/hive/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 diff --git a/src/test/hive/golden/combine2-12-cd15ffd140539cf86090814729ec4748 b/shark/src/test/hive/golden/combine2-12-cd15ffd140539cf86090814729ec4748 similarity index 100% rename from src/test/hive/golden/combine2-12-cd15ffd140539cf86090814729ec4748 rename to shark/src/test/hive/golden/combine2-12-cd15ffd140539cf86090814729ec4748 diff --git a/src/test/hive/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 b/shark/src/test/hive/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 similarity index 100% rename from src/test/hive/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 rename to shark/src/test/hive/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 diff --git a/src/test/hive/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c b/shark/src/test/hive/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c similarity index 100% rename from src/test/hive/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c rename to shark/src/test/hive/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c diff --git a/src/test/hive/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 b/shark/src/test/hive/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 similarity index 100% rename from src/test/hive/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 rename to shark/src/test/hive/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 diff --git a/src/test/hive/golden/combine2-16-557997716a68312e8cae75428e3ce31 b/shark/src/test/hive/golden/combine2-16-557997716a68312e8cae75428e3ce31 similarity index 100% rename from src/test/hive/golden/combine2-16-557997716a68312e8cae75428e3ce31 rename to shark/src/test/hive/golden/combine2-16-557997716a68312e8cae75428e3ce31 diff --git a/src/test/hive/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 b/shark/src/test/hive/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 similarity index 100% rename from src/test/hive/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 rename to shark/src/test/hive/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 diff --git a/src/test/hive/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 b/shark/src/test/hive/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 similarity index 100% rename from src/test/hive/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 rename to shark/src/test/hive/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 diff --git a/src/test/hive/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf b/shark/src/test/hive/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf similarity index 100% rename from src/test/hive/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf rename to shark/src/test/hive/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf diff --git a/src/test/hive/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/shark/src/test/hive/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 similarity index 100% rename from src/test/hive/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 rename to shark/src/test/hive/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 diff --git a/src/test/hive/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 b/shark/src/test/hive/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 similarity index 100% rename from src/test/hive/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 rename to shark/src/test/hive/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 diff --git a/src/test/hive/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/shark/src/test/hive/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea similarity index 100% rename from src/test/hive/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea rename to shark/src/test/hive/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea diff --git a/src/test/hive/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/combine2-7-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/combine2-7-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/combine2-7-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/combine2-7-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 b/shark/src/test/hive/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 similarity index 100% rename from src/test/hive/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 rename to shark/src/test/hive/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 diff --git a/src/test/hive/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 b/shark/src/test/hive/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 similarity index 100% rename from src/test/hive/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 rename to shark/src/test/hive/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 diff --git a/src/test/hive/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/shark/src/test/hive/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 rename to shark/src/test/hive/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 diff --git a/src/test/hive/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 b/shark/src/test/hive/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 rename to shark/src/test/hive/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 diff --git a/src/test/hive/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 b/shark/src/test/hive/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 rename to shark/src/test/hive/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 diff --git a/src/test/hive/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 b/shark/src/test/hive/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 rename to shark/src/test/hive/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 diff --git a/src/test/hive/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 b/shark/src/test/hive/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 rename to shark/src/test/hive/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 diff --git a/src/test/hive/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c b/shark/src/test/hive/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c rename to shark/src/test/hive/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c diff --git a/src/test/hive/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 b/shark/src/test/hive/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 rename to shark/src/test/hive/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 diff --git a/src/test/hive/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 b/shark/src/test/hive/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 rename to shark/src/test/hive/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 diff --git a/src/test/hive/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 b/shark/src/test/hive/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 rename to shark/src/test/hive/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 diff --git a/src/test/hive/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 b/shark/src/test/hive/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 rename to shark/src/test/hive/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 diff --git a/src/test/hive/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf b/shark/src/test/hive/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf rename to shark/src/test/hive/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf diff --git a/src/test/hive/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 b/shark/src/test/hive/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 rename to shark/src/test/hive/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 diff --git a/src/test/hive/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 b/shark/src/test/hive/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 rename to shark/src/test/hive/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 diff --git a/src/test/hive/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea b/shark/src/test/hive/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea rename to shark/src/test/hive/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea diff --git a/src/test/hive/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 b/shark/src/test/hive/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 rename to shark/src/test/hive/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 diff --git a/src/test/hive/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 b/shark/src/test/hive/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 similarity index 100% rename from src/test/hive/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 rename to shark/src/test/hive/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 diff --git a/src/test/hive/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf b/shark/src/test/hive/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf similarity index 100% rename from src/test/hive/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf rename to shark/src/test/hive/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf diff --git a/src/test/hive/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 b/shark/src/test/hive/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 similarity index 100% rename from src/test/hive/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 rename to shark/src/test/hive/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 diff --git a/src/test/hive/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 b/shark/src/test/hive/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 similarity index 100% rename from src/test/hive/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 rename to shark/src/test/hive/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 diff --git a/src/test/hive/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 b/shark/src/test/hive/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 similarity index 100% rename from src/test/hive/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 rename to shark/src/test/hive/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 diff --git a/src/test/hive/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c b/shark/src/test/hive/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c similarity index 100% rename from src/test/hive/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c rename to shark/src/test/hive/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c diff --git a/src/test/hive/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 b/shark/src/test/hive/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 similarity index 100% rename from src/test/hive/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 rename to shark/src/test/hive/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 diff --git a/src/test/hive/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 b/shark/src/test/hive/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 similarity index 100% rename from src/test/hive/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 rename to shark/src/test/hive/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 diff --git a/src/test/hive/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 b/shark/src/test/hive/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 similarity index 100% rename from src/test/hive/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 rename to shark/src/test/hive/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 diff --git a/src/test/hive/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 b/shark/src/test/hive/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 similarity index 100% rename from src/test/hive/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 rename to shark/src/test/hive/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 diff --git a/src/test/hive/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 b/shark/src/test/hive/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 similarity index 100% rename from src/test/hive/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 rename to shark/src/test/hive/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 diff --git a/src/test/hive/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 b/shark/src/test/hive/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 similarity index 100% rename from src/test/hive/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 rename to shark/src/test/hive/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 diff --git a/src/test/hive/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea b/shark/src/test/hive/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea similarity index 100% rename from src/test/hive/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea rename to shark/src/test/hive/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea diff --git a/src/test/hive/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 b/shark/src/test/hive/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 similarity index 100% rename from src/test/hive/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 rename to shark/src/test/hive/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 diff --git a/src/test/hive/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 b/shark/src/test/hive/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 similarity index 100% rename from src/test/hive/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 rename to shark/src/test/hive/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 diff --git a/src/test/hive/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af b/shark/src/test/hive/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af similarity index 100% rename from src/test/hive/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af rename to shark/src/test/hive/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af diff --git a/src/test/hive/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 b/shark/src/test/hive/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 similarity index 100% rename from src/test/hive/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 rename to shark/src/test/hive/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 diff --git a/src/test/hive/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 b/shark/src/test/hive/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 similarity index 100% rename from src/test/hive/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 rename to shark/src/test/hive/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 diff --git a/src/test/hive/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 b/shark/src/test/hive/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 similarity index 100% rename from src/test/hive/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 rename to shark/src/test/hive/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 diff --git a/src/test/hive/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b b/shark/src/test/hive/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b similarity index 100% rename from src/test/hive/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b rename to shark/src/test/hive/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b diff --git a/src/test/hive/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc b/shark/src/test/hive/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc similarity index 100% rename from src/test/hive/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc rename to shark/src/test/hive/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc diff --git a/src/test/hive/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 b/shark/src/test/hive/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 similarity index 100% rename from src/test/hive/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 rename to shark/src/test/hive/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 diff --git a/src/test/hive/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a b/shark/src/test/hive/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a similarity index 100% rename from src/test/hive/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a rename to shark/src/test/hive/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a diff --git a/src/test/hive/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b b/shark/src/test/hive/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b similarity index 100% rename from src/test/hive/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b rename to shark/src/test/hive/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b diff --git a/src/test/hive/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f b/shark/src/test/hive/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f similarity index 100% rename from src/test/hive/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f rename to shark/src/test/hive/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f diff --git a/src/test/hive/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf b/shark/src/test/hive/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf similarity index 100% rename from src/test/hive/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf rename to shark/src/test/hive/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf diff --git a/src/test/hive/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd b/shark/src/test/hive/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd similarity index 100% rename from src/test/hive/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd rename to shark/src/test/hive/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd diff --git a/src/test/hive/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 b/shark/src/test/hive/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 similarity index 100% rename from src/test/hive/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 rename to shark/src/test/hive/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 diff --git a/src/test/hive/golden/combine3-22-11025483569617a9f014b5defd71e933 b/shark/src/test/hive/golden/combine3-22-11025483569617a9f014b5defd71e933 similarity index 100% rename from src/test/hive/golden/combine3-22-11025483569617a9f014b5defd71e933 rename to shark/src/test/hive/golden/combine3-22-11025483569617a9f014b5defd71e933 diff --git a/src/test/hive/golden/combine3-23-4725c48df09565618cbffd05953a5f62 b/shark/src/test/hive/golden/combine3-23-4725c48df09565618cbffd05953a5f62 similarity index 100% rename from src/test/hive/golden/combine3-23-4725c48df09565618cbffd05953a5f62 rename to shark/src/test/hive/golden/combine3-23-4725c48df09565618cbffd05953a5f62 diff --git a/src/test/hive/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 b/shark/src/test/hive/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 similarity index 100% rename from src/test/hive/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 rename to shark/src/test/hive/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 diff --git a/src/test/hive/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 b/shark/src/test/hive/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 similarity index 100% rename from src/test/hive/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 rename to shark/src/test/hive/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 diff --git a/src/test/hive/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea b/shark/src/test/hive/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea similarity index 100% rename from src/test/hive/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea rename to shark/src/test/hive/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea diff --git a/src/test/hive/golden/combine3-6-4725c48df09565618cbffd05953a5f62 b/shark/src/test/hive/golden/combine3-6-4725c48df09565618cbffd05953a5f62 similarity index 100% rename from src/test/hive/golden/combine3-6-4725c48df09565618cbffd05953a5f62 rename to shark/src/test/hive/golden/combine3-6-4725c48df09565618cbffd05953a5f62 diff --git a/src/test/hive/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 b/shark/src/test/hive/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 similarity index 100% rename from src/test/hive/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 rename to shark/src/test/hive/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 diff --git a/src/test/hive/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b b/shark/src/test/hive/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b similarity index 100% rename from src/test/hive/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b rename to shark/src/test/hive/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b diff --git a/src/test/hive/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 b/shark/src/test/hive/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 similarity index 100% rename from src/test/hive/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 rename to shark/src/test/hive/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 diff --git a/src/test/hive/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 b/shark/src/test/hive/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 similarity index 100% rename from src/test/hive/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 rename to shark/src/test/hive/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 diff --git a/src/test/hive/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 b/shark/src/test/hive/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 similarity index 100% rename from src/test/hive/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 rename to shark/src/test/hive/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 diff --git a/src/test/hive/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c b/shark/src/test/hive/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c similarity index 100% rename from src/test/hive/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c rename to shark/src/test/hive/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c diff --git a/src/test/hive/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 b/shark/src/test/hive/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 similarity index 100% rename from src/test/hive/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 rename to shark/src/test/hive/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 diff --git a/src/test/hive/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 b/shark/src/test/hive/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 similarity index 100% rename from src/test/hive/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 rename to shark/src/test/hive/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 diff --git a/src/test/hive/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 b/shark/src/test/hive/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 similarity index 100% rename from src/test/hive/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 rename to shark/src/test/hive/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 diff --git a/src/test/hive/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e b/shark/src/test/hive/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e similarity index 100% rename from src/test/hive/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e rename to shark/src/test/hive/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e diff --git a/src/test/hive/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d b/shark/src/test/hive/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d similarity index 100% rename from src/test/hive/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d rename to shark/src/test/hive/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d diff --git a/src/test/hive/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce b/shark/src/test/hive/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce similarity index 100% rename from src/test/hive/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce rename to shark/src/test/hive/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce diff --git a/src/test/hive/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 b/shark/src/test/hive/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 similarity index 100% rename from src/test/hive/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 rename to shark/src/test/hive/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 diff --git a/src/test/hive/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 b/shark/src/test/hive/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 similarity index 100% rename from src/test/hive/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 rename to shark/src/test/hive/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 diff --git a/src/test/hive/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 b/shark/src/test/hive/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 similarity index 100% rename from src/test/hive/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 rename to shark/src/test/hive/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 diff --git a/src/test/hive/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d b/shark/src/test/hive/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d similarity index 100% rename from src/test/hive/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d rename to shark/src/test/hive/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d diff --git a/src/test/hive/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 b/shark/src/test/hive/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 similarity index 100% rename from src/test/hive/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 rename to shark/src/test/hive/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 diff --git a/src/test/hive/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f b/shark/src/test/hive/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f similarity index 100% rename from src/test/hive/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f rename to shark/src/test/hive/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f diff --git a/src/test/hive/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b b/shark/src/test/hive/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b similarity index 100% rename from src/test/hive/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b rename to shark/src/test/hive/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b diff --git a/src/test/hive/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf b/shark/src/test/hive/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf similarity index 100% rename from src/test/hive/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf rename to shark/src/test/hive/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf diff --git a/src/test/hive/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 b/shark/src/test/hive/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 similarity index 100% rename from src/test/hive/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 rename to shark/src/test/hive/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 diff --git a/src/test/hive/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 b/shark/src/test/hive/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 similarity index 100% rename from src/test/hive/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 rename to shark/src/test/hive/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 diff --git a/src/test/hive/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 b/shark/src/test/hive/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 similarity index 100% rename from src/test/hive/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 rename to shark/src/test/hive/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 diff --git a/src/test/hive/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 b/shark/src/test/hive/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 similarity index 100% rename from src/test/hive/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 rename to shark/src/test/hive/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 diff --git a/src/test/hive/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 b/shark/src/test/hive/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 similarity index 100% rename from src/test/hive/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 rename to shark/src/test/hive/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 diff --git a/src/test/hive/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d b/shark/src/test/hive/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d similarity index 100% rename from src/test/hive/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d rename to shark/src/test/hive/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d diff --git a/src/test/hive/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b b/shark/src/test/hive/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b similarity index 100% rename from src/test/hive/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b rename to shark/src/test/hive/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b diff --git a/src/test/hive/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f b/shark/src/test/hive/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f similarity index 100% rename from src/test/hive/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f rename to shark/src/test/hive/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f diff --git a/src/test/hive/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a b/shark/src/test/hive/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a similarity index 100% rename from src/test/hive/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a rename to shark/src/test/hive/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a diff --git a/src/test/hive/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc b/shark/src/test/hive/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc similarity index 100% rename from src/test/hive/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc rename to shark/src/test/hive/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc diff --git a/src/test/hive/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 b/shark/src/test/hive/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 similarity index 100% rename from src/test/hive/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 rename to shark/src/test/hive/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 diff --git a/src/test/hive/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 b/shark/src/test/hive/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 similarity index 100% rename from src/test/hive/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 rename to shark/src/test/hive/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 diff --git a/src/test/hive/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 b/shark/src/test/hive/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 rename to shark/src/test/hive/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 diff --git a/src/test/hive/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 b/shark/src/test/hive/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 rename to shark/src/test/hive/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 diff --git a/src/test/hive/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 b/shark/src/test/hive/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 rename to shark/src/test/hive/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 diff --git a/src/test/hive/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d b/shark/src/test/hive/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d rename to shark/src/test/hive/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d diff --git a/src/test/hive/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 b/shark/src/test/hive/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 rename to shark/src/test/hive/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 diff --git a/src/test/hive/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 b/shark/src/test/hive/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 rename to shark/src/test/hive/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 diff --git a/src/test/hive/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 b/shark/src/test/hive/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 rename to shark/src/test/hive/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 diff --git a/src/test/hive/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 b/shark/src/test/hive/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 rename to shark/src/test/hive/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 diff --git a/src/test/hive/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 b/shark/src/test/hive/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 rename to shark/src/test/hive/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 diff --git a/src/test/hive/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 b/shark/src/test/hive/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 rename to shark/src/test/hive/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 diff --git a/src/test/hive/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 b/shark/src/test/hive/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 similarity index 100% rename from src/test/hive/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 rename to shark/src/test/hive/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 diff --git a/src/test/hive/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 b/shark/src/test/hive/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 rename to shark/src/test/hive/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 diff --git a/src/test/hive/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 b/shark/src/test/hive/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 rename to shark/src/test/hive/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 diff --git a/src/test/hive/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda b/shark/src/test/hive/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda rename to shark/src/test/hive/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda diff --git a/src/test/hive/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 b/shark/src/test/hive/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 rename to shark/src/test/hive/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 diff --git a/src/test/hive/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 b/shark/src/test/hive/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 rename to shark/src/test/hive/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 diff --git a/src/test/hive/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf b/shark/src/test/hive/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf rename to shark/src/test/hive/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf diff --git a/src/test/hive/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 b/shark/src/test/hive/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 rename to shark/src/test/hive/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 diff --git a/src/test/hive/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 b/shark/src/test/hive/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 rename to shark/src/test/hive/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 diff --git a/src/test/hive/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 b/shark/src/test/hive/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 rename to shark/src/test/hive/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 diff --git a/src/test/hive/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 b/shark/src/test/hive/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 rename to shark/src/test/hive/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 diff --git a/src/test/hive/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 b/shark/src/test/hive/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 rename to shark/src/test/hive/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 diff --git a/src/test/hive/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c b/shark/src/test/hive/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c rename to shark/src/test/hive/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c diff --git a/src/test/hive/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 b/shark/src/test/hive/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 rename to shark/src/test/hive/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 diff --git a/src/test/hive/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f b/shark/src/test/hive/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f rename to shark/src/test/hive/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f diff --git a/src/test/hive/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f b/shark/src/test/hive/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f rename to shark/src/test/hive/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f diff --git a/src/test/hive/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c b/shark/src/test/hive/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c rename to shark/src/test/hive/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c diff --git a/src/test/hive/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda b/shark/src/test/hive/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda rename to shark/src/test/hive/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda diff --git a/src/test/hive/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 b/shark/src/test/hive/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 similarity index 100% rename from src/test/hive/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 rename to shark/src/test/hive/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 diff --git a/src/test/hive/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 b/shark/src/test/hive/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 similarity index 100% rename from src/test/hive/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 rename to shark/src/test/hive/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 diff --git a/src/test/hive/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 b/shark/src/test/hive/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 rename to shark/src/test/hive/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 diff --git a/src/test/hive/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 b/shark/src/test/hive/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 similarity index 100% rename from src/test/hive/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 rename to shark/src/test/hive/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 diff --git a/src/test/hive/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd b/shark/src/test/hive/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd similarity index 100% rename from src/test/hive/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd rename to shark/src/test/hive/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd diff --git a/src/test/hive/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 b/shark/src/test/hive/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 rename to shark/src/test/hive/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 diff --git a/src/test/hive/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab b/shark/src/test/hive/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab rename to shark/src/test/hive/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab diff --git a/src/test/hive/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 b/shark/src/test/hive/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 rename to shark/src/test/hive/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 diff --git a/src/test/hive/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd b/shark/src/test/hive/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd rename to shark/src/test/hive/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd diff --git a/src/test/hive/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 b/shark/src/test/hive/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 rename to shark/src/test/hive/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 diff --git a/src/test/hive/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 b/shark/src/test/hive/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 rename to shark/src/test/hive/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 diff --git a/src/test/hive/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 b/shark/src/test/hive/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 rename to shark/src/test/hive/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 diff --git a/src/test/hive/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 b/shark/src/test/hive/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 rename to shark/src/test/hive/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 diff --git a/src/test/hive/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf b/shark/src/test/hive/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf rename to shark/src/test/hive/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf diff --git a/src/test/hive/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e b/shark/src/test/hive/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e rename to shark/src/test/hive/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e diff --git a/src/test/hive/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b b/shark/src/test/hive/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b rename to shark/src/test/hive/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b diff --git a/src/test/hive/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e b/shark/src/test/hive/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e rename to shark/src/test/hive/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e diff --git a/src/test/hive/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 b/shark/src/test/hive/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 rename to shark/src/test/hive/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 diff --git a/src/test/hive/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 b/shark/src/test/hive/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 rename to shark/src/test/hive/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 diff --git a/src/test/hive/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab b/shark/src/test/hive/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab rename to shark/src/test/hive/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab diff --git a/src/test/hive/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d b/shark/src/test/hive/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d rename to shark/src/test/hive/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d diff --git a/src/test/hive/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 b/shark/src/test/hive/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 rename to shark/src/test/hive/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 diff --git a/src/test/hive/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f b/shark/src/test/hive/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f rename to shark/src/test/hive/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f diff --git a/src/test/hive/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b b/shark/src/test/hive/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b rename to shark/src/test/hive/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b diff --git a/src/test/hive/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 b/shark/src/test/hive/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 rename to shark/src/test/hive/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 diff --git a/src/test/hive/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b b/shark/src/test/hive/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b rename to shark/src/test/hive/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b diff --git a/src/test/hive/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 b/shark/src/test/hive/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 rename to shark/src/test/hive/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 diff --git a/src/test/hive/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d b/shark/src/test/hive/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d rename to shark/src/test/hive/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d diff --git a/src/test/hive/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 b/shark/src/test/hive/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 rename to shark/src/test/hive/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 diff --git a/src/test/hive/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 b/shark/src/test/hive/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 similarity index 100% rename from src/test/hive/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 rename to shark/src/test/hive/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 diff --git a/src/test/hive/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 b/shark/src/test/hive/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 rename to shark/src/test/hive/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 diff --git a/src/test/hive/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 b/shark/src/test/hive/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 rename to shark/src/test/hive/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 diff --git a/src/test/hive/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 b/shark/src/test/hive/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 rename to shark/src/test/hive/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 diff --git a/src/test/hive/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 b/shark/src/test/hive/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 rename to shark/src/test/hive/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 diff --git a/src/test/hive/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 b/shark/src/test/hive/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 rename to shark/src/test/hive/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 diff --git a/src/test/hive/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace b/shark/src/test/hive/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace rename to shark/src/test/hive/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace diff --git a/src/test/hive/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe b/shark/src/test/hive/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe rename to shark/src/test/hive/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe diff --git a/src/test/hive/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 b/shark/src/test/hive/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 rename to shark/src/test/hive/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 diff --git a/src/test/hive/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 b/shark/src/test/hive/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 rename to shark/src/test/hive/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 diff --git a/src/test/hive/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 b/shark/src/test/hive/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 rename to shark/src/test/hive/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 diff --git a/src/test/hive/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 b/shark/src/test/hive/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 rename to shark/src/test/hive/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 diff --git a/src/test/hive/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea b/shark/src/test/hive/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea similarity index 100% rename from src/test/hive/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea rename to shark/src/test/hive/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea diff --git a/src/test/hive/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 b/shark/src/test/hive/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 rename to shark/src/test/hive/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 diff --git a/src/test/hive/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e b/shark/src/test/hive/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e rename to shark/src/test/hive/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e diff --git a/src/test/hive/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e b/shark/src/test/hive/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e rename to shark/src/test/hive/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e diff --git a/src/test/hive/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 b/shark/src/test/hive/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 rename to shark/src/test/hive/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 diff --git a/src/test/hive/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e b/shark/src/test/hive/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e rename to shark/src/test/hive/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e diff --git a/src/test/hive/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb b/shark/src/test/hive/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb rename to shark/src/test/hive/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb diff --git a/src/test/hive/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 b/shark/src/test/hive/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 rename to shark/src/test/hive/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 diff --git a/src/test/hive/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 b/shark/src/test/hive/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 rename to shark/src/test/hive/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 diff --git a/src/test/hive/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 b/shark/src/test/hive/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 rename to shark/src/test/hive/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 diff --git a/src/test/hive/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 b/shark/src/test/hive/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 rename to shark/src/test/hive/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 diff --git a/src/test/hive/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf b/shark/src/test/hive/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf rename to shark/src/test/hive/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf diff --git a/src/test/hive/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b b/shark/src/test/hive/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b rename to shark/src/test/hive/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b diff --git a/src/test/hive/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb b/shark/src/test/hive/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb rename to shark/src/test/hive/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb diff --git a/src/test/hive/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf b/shark/src/test/hive/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf rename to shark/src/test/hive/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf diff --git a/src/test/hive/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 b/shark/src/test/hive/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 rename to shark/src/test/hive/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 diff --git a/src/test/hive/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf b/shark/src/test/hive/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf rename to shark/src/test/hive/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf diff --git a/src/test/hive/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 b/shark/src/test/hive/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 rename to shark/src/test/hive/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 diff --git a/src/test/hive/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 b/shark/src/test/hive/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 rename to shark/src/test/hive/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 diff --git a/src/test/hive/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 b/shark/src/test/hive/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 rename to shark/src/test/hive/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 diff --git a/src/test/hive/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 b/shark/src/test/hive/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 rename to shark/src/test/hive/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 diff --git a/src/test/hive/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 b/shark/src/test/hive/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 rename to shark/src/test/hive/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 diff --git a/src/test/hive/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 b/shark/src/test/hive/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 rename to shark/src/test/hive/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 diff --git a/src/test/hive/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 b/shark/src/test/hive/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 rename to shark/src/test/hive/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 diff --git a/src/test/hive/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 b/shark/src/test/hive/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 rename to shark/src/test/hive/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 diff --git a/src/test/hive/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 b/shark/src/test/hive/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 rename to shark/src/test/hive/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 diff --git a/src/test/hive/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e b/shark/src/test/hive/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e similarity index 100% rename from src/test/hive/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e rename to shark/src/test/hive/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e diff --git a/src/test/hive/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 b/shark/src/test/hive/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 rename to shark/src/test/hive/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 diff --git a/src/test/hive/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 b/shark/src/test/hive/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 rename to shark/src/test/hive/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 diff --git a/src/test/hive/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db b/shark/src/test/hive/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db rename to shark/src/test/hive/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db diff --git a/src/test/hive/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 b/shark/src/test/hive/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 rename to shark/src/test/hive/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 diff --git a/src/test/hive/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db b/shark/src/test/hive/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db rename to shark/src/test/hive/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db diff --git a/src/test/hive/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 b/shark/src/test/hive/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 rename to shark/src/test/hive/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 diff --git a/src/test/hive/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 b/shark/src/test/hive/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 rename to shark/src/test/hive/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 diff --git a/src/test/hive/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/shark/src/test/hive/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de rename to shark/src/test/hive/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de diff --git a/src/test/hive/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 b/shark/src/test/hive/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 rename to shark/src/test/hive/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 diff --git a/src/test/hive/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/shark/src/test/hive/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de rename to shark/src/test/hive/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de diff --git a/src/test/hive/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 b/shark/src/test/hive/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 rename to shark/src/test/hive/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 diff --git a/src/test/hive/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 b/shark/src/test/hive/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 rename to shark/src/test/hive/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 diff --git a/src/test/hive/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd b/shark/src/test/hive/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd rename to shark/src/test/hive/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd diff --git a/src/test/hive/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 b/shark/src/test/hive/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 rename to shark/src/test/hive/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 diff --git a/src/test/hive/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 b/shark/src/test/hive/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 rename to shark/src/test/hive/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 diff --git a/src/test/hive/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 b/shark/src/test/hive/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 rename to shark/src/test/hive/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 diff --git a/src/test/hive/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 b/shark/src/test/hive/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 rename to shark/src/test/hive/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 diff --git a/src/test/hive/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 b/shark/src/test/hive/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 rename to shark/src/test/hive/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 diff --git a/src/test/hive/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 b/shark/src/test/hive/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 rename to shark/src/test/hive/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 diff --git a/src/test/hive/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb b/shark/src/test/hive/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb rename to shark/src/test/hive/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb diff --git a/src/test/hive/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 b/shark/src/test/hive/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 rename to shark/src/test/hive/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 diff --git a/src/test/hive/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 b/shark/src/test/hive/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 rename to shark/src/test/hive/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 diff --git a/src/test/hive/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 b/shark/src/test/hive/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 rename to shark/src/test/hive/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 diff --git a/src/test/hive/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb b/shark/src/test/hive/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb rename to shark/src/test/hive/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb diff --git a/src/test/hive/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 b/shark/src/test/hive/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 rename to shark/src/test/hive/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 diff --git a/src/test/hive/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 b/shark/src/test/hive/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 rename to shark/src/test/hive/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 diff --git a/src/test/hive/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 b/shark/src/test/hive/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 rename to shark/src/test/hive/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 diff --git a/src/test/hive/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 b/shark/src/test/hive/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 rename to shark/src/test/hive/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 diff --git a/src/test/hive/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 b/shark/src/test/hive/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 rename to shark/src/test/hive/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 diff --git a/src/test/hive/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d b/shark/src/test/hive/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d rename to shark/src/test/hive/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d diff --git a/src/test/hive/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 b/shark/src/test/hive/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 rename to shark/src/test/hive/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 diff --git a/src/test/hive/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d b/shark/src/test/hive/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d rename to shark/src/test/hive/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d diff --git a/src/test/hive/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 b/shark/src/test/hive/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 rename to shark/src/test/hive/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 diff --git a/src/test/hive/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d b/shark/src/test/hive/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d rename to shark/src/test/hive/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d diff --git a/src/test/hive/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 b/shark/src/test/hive/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 rename to shark/src/test/hive/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 diff --git a/src/test/hive/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae b/shark/src/test/hive/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae similarity index 100% rename from src/test/hive/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae rename to shark/src/test/hive/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae diff --git a/src/test/hive/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 b/shark/src/test/hive/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 rename to shark/src/test/hive/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 diff --git a/src/test/hive/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 b/shark/src/test/hive/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 rename to shark/src/test/hive/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 diff --git a/src/test/hive/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 b/shark/src/test/hive/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 rename to shark/src/test/hive/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 diff --git a/src/test/hive/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 b/shark/src/test/hive/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 rename to shark/src/test/hive/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 diff --git a/src/test/hive/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 b/shark/src/test/hive/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 rename to shark/src/test/hive/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 diff --git a/src/test/hive/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 b/shark/src/test/hive/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 rename to shark/src/test/hive/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 diff --git a/src/test/hive/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 b/shark/src/test/hive/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 rename to shark/src/test/hive/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 diff --git a/src/test/hive/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 b/shark/src/test/hive/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 rename to shark/src/test/hive/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 diff --git a/src/test/hive/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d b/shark/src/test/hive/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d similarity index 100% rename from src/test/hive/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d rename to shark/src/test/hive/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d diff --git a/src/test/hive/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d b/shark/src/test/hive/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d rename to shark/src/test/hive/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d diff --git a/src/test/hive/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f b/shark/src/test/hive/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f rename to shark/src/test/hive/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f diff --git a/src/test/hive/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea b/shark/src/test/hive/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea rename to shark/src/test/hive/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea diff --git a/src/test/hive/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc b/shark/src/test/hive/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc rename to shark/src/test/hive/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc diff --git a/src/test/hive/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a b/shark/src/test/hive/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a rename to shark/src/test/hive/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a diff --git a/src/test/hive/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 b/shark/src/test/hive/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 rename to shark/src/test/hive/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 diff --git a/src/test/hive/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 b/shark/src/test/hive/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 rename to shark/src/test/hive/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 diff --git a/src/test/hive/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 b/shark/src/test/hive/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 rename to shark/src/test/hive/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 diff --git a/src/test/hive/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 b/shark/src/test/hive/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 rename to shark/src/test/hive/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 diff --git a/src/test/hive/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f b/shark/src/test/hive/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f similarity index 100% rename from src/test/hive/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f rename to shark/src/test/hive/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f diff --git a/src/test/hive/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b b/shark/src/test/hive/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b similarity index 100% rename from src/test/hive/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b rename to shark/src/test/hive/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b diff --git a/src/test/hive/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 b/shark/src/test/hive/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 similarity index 100% rename from src/test/hive/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 rename to shark/src/test/hive/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 diff --git a/src/test/hive/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 b/shark/src/test/hive/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 similarity index 100% rename from src/test/hive/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 rename to shark/src/test/hive/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 diff --git a/src/test/hive/golden/count-11-29aa086fe75b55482a91316c38498565 b/shark/src/test/hive/golden/count-11-29aa086fe75b55482a91316c38498565 similarity index 100% rename from src/test/hive/golden/count-11-29aa086fe75b55482a91316c38498565 rename to shark/src/test/hive/golden/count-11-29aa086fe75b55482a91316c38498565 diff --git a/src/test/hive/golden/count-12-944f53db544c07a7b38a0544a21d8e13 b/shark/src/test/hive/golden/count-12-944f53db544c07a7b38a0544a21d8e13 similarity index 100% rename from src/test/hive/golden/count-12-944f53db544c07a7b38a0544a21d8e13 rename to shark/src/test/hive/golden/count-12-944f53db544c07a7b38a0544a21d8e13 diff --git a/src/test/hive/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 b/shark/src/test/hive/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 similarity index 100% rename from src/test/hive/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 rename to shark/src/test/hive/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 diff --git a/src/test/hive/golden/count-3-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/count-3-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/count-3-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/count-3-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/count-4-590bf60b8d4dfa135f73dbb52180136f b/shark/src/test/hive/golden/count-4-590bf60b8d4dfa135f73dbb52180136f similarity index 100% rename from src/test/hive/golden/count-4-590bf60b8d4dfa135f73dbb52180136f rename to shark/src/test/hive/golden/count-4-590bf60b8d4dfa135f73dbb52180136f diff --git a/src/test/hive/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 b/shark/src/test/hive/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 similarity index 100% rename from src/test/hive/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 rename to shark/src/test/hive/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 diff --git a/src/test/hive/golden/count-6-29aa086fe75b55482a91316c38498565 b/shark/src/test/hive/golden/count-6-29aa086fe75b55482a91316c38498565 similarity index 100% rename from src/test/hive/golden/count-6-29aa086fe75b55482a91316c38498565 rename to shark/src/test/hive/golden/count-6-29aa086fe75b55482a91316c38498565 diff --git a/src/test/hive/golden/count-7-944f53db544c07a7b38a0544a21d8e13 b/shark/src/test/hive/golden/count-7-944f53db544c07a7b38a0544a21d8e13 similarity index 100% rename from src/test/hive/golden/count-7-944f53db544c07a7b38a0544a21d8e13 rename to shark/src/test/hive/golden/count-7-944f53db544c07a7b38a0544a21d8e13 diff --git a/src/test/hive/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/count-9-590bf60b8d4dfa135f73dbb52180136f b/shark/src/test/hive/golden/count-9-590bf60b8d4dfa135f73dbb52180136f similarity index 100% rename from src/test/hive/golden/count-9-590bf60b8d4dfa135f73dbb52180136f rename to shark/src/test/hive/golden/count-9-590bf60b8d4dfa135f73dbb52180136f diff --git a/src/test/hive/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 b/shark/src/test/hive/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 similarity index 100% rename from src/test/hive/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 rename to shark/src/test/hive/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 diff --git a/src/test/hive/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 b/shark/src/test/hive/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 similarity index 100% rename from src/test/hive/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 rename to shark/src/test/hive/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 diff --git a/src/test/hive/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e b/shark/src/test/hive/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e similarity index 100% rename from src/test/hive/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e rename to shark/src/test/hive/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e diff --git a/src/test/hive/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 b/shark/src/test/hive/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 similarity index 100% rename from src/test/hive/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 rename to shark/src/test/hive/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 diff --git a/src/test/hive/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 b/shark/src/test/hive/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 similarity index 100% rename from src/test/hive/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 rename to shark/src/test/hive/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 diff --git a/src/test/hive/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 b/shark/src/test/hive/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 similarity index 100% rename from src/test/hive/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 rename to shark/src/test/hive/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 diff --git a/src/test/hive/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac b/shark/src/test/hive/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac similarity index 100% rename from src/test/hive/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac rename to shark/src/test/hive/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac diff --git a/src/test/hive/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 b/shark/src/test/hive/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 similarity index 100% rename from src/test/hive/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 rename to shark/src/test/hive/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 diff --git a/src/test/hive/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f b/shark/src/test/hive/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f similarity index 100% rename from src/test/hive/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f rename to shark/src/test/hive/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f diff --git a/src/test/hive/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 b/shark/src/test/hive/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 similarity index 100% rename from src/test/hive/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 rename to shark/src/test/hive/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 diff --git a/src/test/hive/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 b/shark/src/test/hive/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 similarity index 100% rename from src/test/hive/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 rename to shark/src/test/hive/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 diff --git a/src/test/hive/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c b/shark/src/test/hive/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c similarity index 100% rename from src/test/hive/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c rename to shark/src/test/hive/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c diff --git a/src/test/hive/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd b/shark/src/test/hive/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd similarity index 100% rename from src/test/hive/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd rename to shark/src/test/hive/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd diff --git a/src/test/hive/golden/create table as-1-b9002c1d71895be765575b62656d1928 b/shark/src/test/hive/golden/create table as-1-b9002c1d71895be765575b62656d1928 similarity index 100% rename from src/test/hive/golden/create table as-1-b9002c1d71895be765575b62656d1928 rename to shark/src/test/hive/golden/create table as-1-b9002c1d71895be765575b62656d1928 diff --git a/src/test/hive/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d b/shark/src/test/hive/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d similarity index 100% rename from src/test/hive/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d rename to shark/src/test/hive/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d diff --git a/src/test/hive/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 b/shark/src/test/hive/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 similarity index 100% rename from src/test/hive/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 rename to shark/src/test/hive/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 diff --git a/src/test/hive/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba b/shark/src/test/hive/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba similarity index 100% rename from src/test/hive/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba rename to shark/src/test/hive/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba diff --git a/src/test/hive/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 b/shark/src/test/hive/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 similarity index 100% rename from src/test/hive/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 rename to shark/src/test/hive/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 diff --git a/src/test/hive/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 b/shark/src/test/hive/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 similarity index 100% rename from src/test/hive/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 rename to shark/src/test/hive/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 diff --git a/src/test/hive/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a b/shark/src/test/hive/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a similarity index 100% rename from src/test/hive/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a rename to shark/src/test/hive/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a diff --git a/src/test/hive/golden/create_1-14-437bd1acbae61f48e851addb769d3379 b/shark/src/test/hive/golden/create_1-14-437bd1acbae61f48e851addb769d3379 similarity index 100% rename from src/test/hive/golden/create_1-14-437bd1acbae61f48e851addb769d3379 rename to shark/src/test/hive/golden/create_1-14-437bd1acbae61f48e851addb769d3379 diff --git a/src/test/hive/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 b/shark/src/test/hive/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 similarity index 100% rename from src/test/hive/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 rename to shark/src/test/hive/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 diff --git a/src/test/hive/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 b/shark/src/test/hive/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 similarity index 100% rename from src/test/hive/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 rename to shark/src/test/hive/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 diff --git a/src/test/hive/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 b/shark/src/test/hive/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 similarity index 100% rename from src/test/hive/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 rename to shark/src/test/hive/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 diff --git a/src/test/hive/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 b/shark/src/test/hive/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 similarity index 100% rename from src/test/hive/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 rename to shark/src/test/hive/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 diff --git a/src/test/hive/golden/create_1-4-610b82bf7b0080d293977927e5ef780c b/shark/src/test/hive/golden/create_1-4-610b82bf7b0080d293977927e5ef780c similarity index 100% rename from src/test/hive/golden/create_1-4-610b82bf7b0080d293977927e5ef780c rename to shark/src/test/hive/golden/create_1-4-610b82bf7b0080d293977927e5ef780c diff --git a/src/test/hive/golden/create_1-5-c77b018276b1558c1d9462e0625e152e b/shark/src/test/hive/golden/create_1-5-c77b018276b1558c1d9462e0625e152e similarity index 100% rename from src/test/hive/golden/create_1-5-c77b018276b1558c1d9462e0625e152e rename to shark/src/test/hive/golden/create_1-5-c77b018276b1558c1d9462e0625e152e diff --git a/src/test/hive/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a b/shark/src/test/hive/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a similarity index 100% rename from src/test/hive/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a rename to shark/src/test/hive/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a diff --git a/src/test/hive/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 b/shark/src/test/hive/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 similarity index 100% rename from src/test/hive/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 rename to shark/src/test/hive/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 diff --git a/src/test/hive/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 b/shark/src/test/hive/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 similarity index 100% rename from src/test/hive/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 rename to shark/src/test/hive/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 diff --git a/src/test/hive/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 b/shark/src/test/hive/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 similarity index 100% rename from src/test/hive/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 rename to shark/src/test/hive/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 diff --git a/src/test/hive/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d b/shark/src/test/hive/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d similarity index 100% rename from src/test/hive/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d rename to shark/src/test/hive/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d diff --git a/src/test/hive/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee b/shark/src/test/hive/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee similarity index 100% rename from src/test/hive/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee rename to shark/src/test/hive/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee diff --git a/src/test/hive/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e b/shark/src/test/hive/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e similarity index 100% rename from src/test/hive/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e rename to shark/src/test/hive/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e diff --git a/src/test/hive/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 b/shark/src/test/hive/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 similarity index 100% rename from src/test/hive/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 rename to shark/src/test/hive/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 diff --git a/src/test/hive/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 b/shark/src/test/hive/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 similarity index 100% rename from src/test/hive/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 rename to shark/src/test/hive/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 diff --git a/src/test/hive/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 b/shark/src/test/hive/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 similarity index 100% rename from src/test/hive/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 rename to shark/src/test/hive/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 diff --git a/src/test/hive/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 b/shark/src/test/hive/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 similarity index 100% rename from src/test/hive/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 rename to shark/src/test/hive/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 diff --git a/src/test/hive/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 b/shark/src/test/hive/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 similarity index 100% rename from src/test/hive/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 rename to shark/src/test/hive/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 diff --git a/src/test/hive/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b b/shark/src/test/hive/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b similarity index 100% rename from src/test/hive/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b rename to shark/src/test/hive/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b diff --git a/src/test/hive/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 b/shark/src/test/hive/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 similarity index 100% rename from src/test/hive/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 rename to shark/src/test/hive/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 diff --git a/src/test/hive/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 b/shark/src/test/hive/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 similarity index 100% rename from src/test/hive/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 rename to shark/src/test/hive/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 diff --git a/src/test/hive/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 b/shark/src/test/hive/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 similarity index 100% rename from src/test/hive/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 rename to shark/src/test/hive/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 diff --git a/src/test/hive/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 b/shark/src/test/hive/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 similarity index 100% rename from src/test/hive/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 rename to shark/src/test/hive/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 diff --git a/src/test/hive/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a b/shark/src/test/hive/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a similarity index 100% rename from src/test/hive/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a rename to shark/src/test/hive/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a diff --git a/src/test/hive/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 b/shark/src/test/hive/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 similarity index 100% rename from src/test/hive/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 rename to shark/src/test/hive/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 diff --git a/src/test/hive/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc b/shark/src/test/hive/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc similarity index 100% rename from src/test/hive/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc rename to shark/src/test/hive/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc diff --git a/src/test/hive/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 b/shark/src/test/hive/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 similarity index 100% rename from src/test/hive/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 rename to shark/src/test/hive/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 diff --git a/src/test/hive/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 b/shark/src/test/hive/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 similarity index 100% rename from src/test/hive/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 rename to shark/src/test/hive/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 diff --git a/src/test/hive/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c b/shark/src/test/hive/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c similarity index 100% rename from src/test/hive/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c rename to shark/src/test/hive/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c diff --git a/src/test/hive/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 b/shark/src/test/hive/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 similarity index 100% rename from src/test/hive/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 rename to shark/src/test/hive/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 diff --git a/src/test/hive/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e b/shark/src/test/hive/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e similarity index 100% rename from src/test/hive/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e rename to shark/src/test/hive/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e diff --git a/src/test/hive/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 b/shark/src/test/hive/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 similarity index 100% rename from src/test/hive/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 rename to shark/src/test/hive/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 diff --git a/src/test/hive/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 b/shark/src/test/hive/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 similarity index 100% rename from src/test/hive/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 rename to shark/src/test/hive/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 diff --git a/src/test/hive/golden/create_like-1-30485a2507b60b96cad3d293527e6af b/shark/src/test/hive/golden/create_like-1-30485a2507b60b96cad3d293527e6af similarity index 100% rename from src/test/hive/golden/create_like-1-30485a2507b60b96cad3d293527e6af rename to shark/src/test/hive/golden/create_like-1-30485a2507b60b96cad3d293527e6af diff --git a/src/test/hive/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 b/shark/src/test/hive/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 similarity index 100% rename from src/test/hive/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 rename to shark/src/test/hive/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 diff --git a/src/test/hive/golden/create_like-11-ba64f0122b21f605d8b2928753784130 b/shark/src/test/hive/golden/create_like-11-ba64f0122b21f605d8b2928753784130 similarity index 100% rename from src/test/hive/golden/create_like-11-ba64f0122b21f605d8b2928753784130 rename to shark/src/test/hive/golden/create_like-11-ba64f0122b21f605d8b2928753784130 diff --git a/src/test/hive/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a b/shark/src/test/hive/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a similarity index 100% rename from src/test/hive/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a rename to shark/src/test/hive/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a diff --git a/src/test/hive/golden/create_like-3-eea111a209cf8895f31f64699669c705 b/shark/src/test/hive/golden/create_like-3-eea111a209cf8895f31f64699669c705 similarity index 100% rename from src/test/hive/golden/create_like-3-eea111a209cf8895f31f64699669c705 rename to shark/src/test/hive/golden/create_like-3-eea111a209cf8895f31f64699669c705 diff --git a/src/test/hive/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 b/shark/src/test/hive/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 similarity index 100% rename from src/test/hive/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 rename to shark/src/test/hive/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 diff --git a/src/test/hive/golden/create_like-5-dc9de26002604e9e436135bd4b40636d b/shark/src/test/hive/golden/create_like-5-dc9de26002604e9e436135bd4b40636d similarity index 100% rename from src/test/hive/golden/create_like-5-dc9de26002604e9e436135bd4b40636d rename to shark/src/test/hive/golden/create_like-5-dc9de26002604e9e436135bd4b40636d diff --git a/src/test/hive/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 b/shark/src/test/hive/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 similarity index 100% rename from src/test/hive/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 rename to shark/src/test/hive/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 diff --git a/src/test/hive/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 b/shark/src/test/hive/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 similarity index 100% rename from src/test/hive/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 rename to shark/src/test/hive/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 diff --git a/src/test/hive/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb b/shark/src/test/hive/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb similarity index 100% rename from src/test/hive/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb rename to shark/src/test/hive/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb diff --git a/src/test/hive/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb b/shark/src/test/hive/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb similarity index 100% rename from src/test/hive/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb rename to shark/src/test/hive/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb diff --git a/src/test/hive/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b b/shark/src/test/hive/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b similarity index 100% rename from src/test/hive/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b rename to shark/src/test/hive/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b diff --git a/src/test/hive/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a b/shark/src/test/hive/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a similarity index 100% rename from src/test/hive/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a rename to shark/src/test/hive/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a diff --git a/src/test/hive/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 b/shark/src/test/hive/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 similarity index 100% rename from src/test/hive/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 rename to shark/src/test/hive/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 diff --git a/src/test/hive/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 b/shark/src/test/hive/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 similarity index 100% rename from src/test/hive/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 rename to shark/src/test/hive/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 diff --git a/src/test/hive/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 b/shark/src/test/hive/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 similarity index 100% rename from src/test/hive/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 rename to shark/src/test/hive/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 diff --git a/src/test/hive/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 b/shark/src/test/hive/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 similarity index 100% rename from src/test/hive/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 rename to shark/src/test/hive/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 diff --git a/src/test/hive/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 b/shark/src/test/hive/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 similarity index 100% rename from src/test/hive/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 rename to shark/src/test/hive/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 diff --git a/src/test/hive/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 b/shark/src/test/hive/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 similarity index 100% rename from src/test/hive/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 rename to shark/src/test/hive/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 diff --git a/src/test/hive/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb b/shark/src/test/hive/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb similarity index 100% rename from src/test/hive/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb rename to shark/src/test/hive/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb diff --git a/src/test/hive/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb b/shark/src/test/hive/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb similarity index 100% rename from src/test/hive/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb rename to shark/src/test/hive/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb diff --git a/src/test/hive/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c b/shark/src/test/hive/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c similarity index 100% rename from src/test/hive/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c rename to shark/src/test/hive/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c diff --git a/src/test/hive/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f b/shark/src/test/hive/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f similarity index 100% rename from src/test/hive/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f rename to shark/src/test/hive/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f diff --git a/src/test/hive/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b b/shark/src/test/hive/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b similarity index 100% rename from src/test/hive/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b rename to shark/src/test/hive/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b diff --git a/src/test/hive/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 b/shark/src/test/hive/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 similarity index 100% rename from src/test/hive/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 rename to shark/src/test/hive/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 diff --git a/src/test/hive/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 b/shark/src/test/hive/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 similarity index 100% rename from src/test/hive/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 rename to shark/src/test/hive/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 diff --git a/src/test/hive/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/shark/src/test/hive/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 similarity index 100% rename from src/test/hive/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 rename to shark/src/test/hive/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 diff --git a/src/test/hive/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b b/shark/src/test/hive/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b similarity index 100% rename from src/test/hive/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b rename to shark/src/test/hive/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b diff --git a/src/test/hive/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 b/shark/src/test/hive/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 similarity index 100% rename from src/test/hive/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 rename to shark/src/test/hive/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 diff --git a/src/test/hive/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d b/shark/src/test/hive/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d similarity index 100% rename from src/test/hive/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d rename to shark/src/test/hive/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d diff --git a/src/test/hive/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 b/shark/src/test/hive/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 similarity index 100% rename from src/test/hive/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 rename to shark/src/test/hive/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 diff --git a/src/test/hive/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/shark/src/test/hive/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 similarity index 100% rename from src/test/hive/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 rename to shark/src/test/hive/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 diff --git a/src/test/hive/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f b/shark/src/test/hive/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f similarity index 100% rename from src/test/hive/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f rename to shark/src/test/hive/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f diff --git a/src/test/hive/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 b/shark/src/test/hive/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 similarity index 100% rename from src/test/hive/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 rename to shark/src/test/hive/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 diff --git a/src/test/hive/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 b/shark/src/test/hive/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 similarity index 100% rename from src/test/hive/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 rename to shark/src/test/hive/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 diff --git a/src/test/hive/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af b/shark/src/test/hive/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af similarity index 100% rename from src/test/hive/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af rename to shark/src/test/hive/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af diff --git a/src/test/hive/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 b/shark/src/test/hive/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 similarity index 100% rename from src/test/hive/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 rename to shark/src/test/hive/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 diff --git a/src/test/hive/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 b/shark/src/test/hive/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 similarity index 100% rename from src/test/hive/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 rename to shark/src/test/hive/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 diff --git a/src/test/hive/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a b/shark/src/test/hive/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a similarity index 100% rename from src/test/hive/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a rename to shark/src/test/hive/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a diff --git a/src/test/hive/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c b/shark/src/test/hive/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c similarity index 100% rename from src/test/hive/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c rename to shark/src/test/hive/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c diff --git a/src/test/hive/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 b/shark/src/test/hive/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 rename to shark/src/test/hive/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 diff --git a/src/test/hive/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 b/shark/src/test/hive/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 rename to shark/src/test/hive/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 diff --git a/src/test/hive/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d b/shark/src/test/hive/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d similarity index 100% rename from src/test/hive/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d rename to shark/src/test/hive/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d diff --git a/src/test/hive/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 b/shark/src/test/hive/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 rename to shark/src/test/hive/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 diff --git a/src/test/hive/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f b/shark/src/test/hive/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f similarity index 100% rename from src/test/hive/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f rename to shark/src/test/hive/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f diff --git a/src/test/hive/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 b/shark/src/test/hive/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 rename to shark/src/test/hive/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 diff --git a/src/test/hive/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 b/shark/src/test/hive/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 rename to shark/src/test/hive/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 diff --git a/src/test/hive/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 b/shark/src/test/hive/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 rename to shark/src/test/hive/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 diff --git a/src/test/hive/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 b/shark/src/test/hive/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 rename to shark/src/test/hive/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 diff --git a/src/test/hive/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 b/shark/src/test/hive/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 rename to shark/src/test/hive/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 diff --git a/src/test/hive/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 b/shark/src/test/hive/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 rename to shark/src/test/hive/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 diff --git a/src/test/hive/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d b/shark/src/test/hive/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d similarity index 100% rename from src/test/hive/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d rename to shark/src/test/hive/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d diff --git a/src/test/hive/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 b/shark/src/test/hive/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 rename to shark/src/test/hive/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 diff --git a/src/test/hive/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 b/shark/src/test/hive/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 similarity index 100% rename from src/test/hive/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 rename to shark/src/test/hive/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 diff --git a/src/test/hive/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 b/shark/src/test/hive/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 similarity index 100% rename from src/test/hive/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 rename to shark/src/test/hive/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 diff --git a/src/test/hive/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 b/shark/src/test/hive/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 similarity index 100% rename from src/test/hive/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 rename to shark/src/test/hive/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 diff --git a/src/test/hive/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 b/shark/src/test/hive/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 similarity index 100% rename from src/test/hive/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 rename to shark/src/test/hive/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 diff --git a/src/test/hive/golden/create_nested_type-3-ac452c9279877935983c37113898e53c b/shark/src/test/hive/golden/create_nested_type-3-ac452c9279877935983c37113898e53c similarity index 100% rename from src/test/hive/golden/create_nested_type-3-ac452c9279877935983c37113898e53c rename to shark/src/test/hive/golden/create_nested_type-3-ac452c9279877935983c37113898e53c diff --git a/src/test/hive/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff b/shark/src/test/hive/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff similarity index 100% rename from src/test/hive/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff rename to shark/src/test/hive/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff diff --git a/src/test/hive/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d b/shark/src/test/hive/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d similarity index 100% rename from src/test/hive/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d rename to shark/src/test/hive/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d diff --git a/src/test/hive/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea b/shark/src/test/hive/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea similarity index 100% rename from src/test/hive/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea rename to shark/src/test/hive/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea diff --git a/src/test/hive/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 b/shark/src/test/hive/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 similarity index 100% rename from src/test/hive/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 rename to shark/src/test/hive/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 diff --git a/src/test/hive/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d b/shark/src/test/hive/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d similarity index 100% rename from src/test/hive/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d rename to shark/src/test/hive/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d diff --git a/src/test/hive/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de b/shark/src/test/hive/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de similarity index 100% rename from src/test/hive/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de rename to shark/src/test/hive/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de diff --git a/src/test/hive/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb b/shark/src/test/hive/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb similarity index 100% rename from src/test/hive/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb rename to shark/src/test/hive/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb diff --git a/src/test/hive/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 b/shark/src/test/hive/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 similarity index 100% rename from src/test/hive/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 rename to shark/src/test/hive/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 diff --git a/src/test/hive/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e b/shark/src/test/hive/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e similarity index 100% rename from src/test/hive/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e rename to shark/src/test/hive/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e diff --git a/src/test/hive/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 b/shark/src/test/hive/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 similarity index 100% rename from src/test/hive/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 rename to shark/src/test/hive/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 diff --git a/src/test/hive/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 b/shark/src/test/hive/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 similarity index 100% rename from src/test/hive/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 rename to shark/src/test/hive/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 diff --git a/src/test/hive/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e b/shark/src/test/hive/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e similarity index 100% rename from src/test/hive/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e rename to shark/src/test/hive/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e diff --git a/src/test/hive/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef b/shark/src/test/hive/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef similarity index 100% rename from src/test/hive/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef rename to shark/src/test/hive/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef diff --git a/src/test/hive/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 b/shark/src/test/hive/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 similarity index 100% rename from src/test/hive/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 rename to shark/src/test/hive/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 diff --git a/src/test/hive/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd b/shark/src/test/hive/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd similarity index 100% rename from src/test/hive/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd rename to shark/src/test/hive/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd diff --git a/src/test/hive/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e b/shark/src/test/hive/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e similarity index 100% rename from src/test/hive/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e rename to shark/src/test/hive/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e diff --git a/src/test/hive/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 b/shark/src/test/hive/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 similarity index 100% rename from src/test/hive/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 rename to shark/src/test/hive/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 diff --git a/src/test/hive/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 b/shark/src/test/hive/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 similarity index 100% rename from src/test/hive/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 rename to shark/src/test/hive/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 diff --git a/src/test/hive/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd b/shark/src/test/hive/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd similarity index 100% rename from src/test/hive/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd rename to shark/src/test/hive/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd diff --git a/src/test/hive/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e b/shark/src/test/hive/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e similarity index 100% rename from src/test/hive/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e rename to shark/src/test/hive/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e diff --git a/src/test/hive/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/shark/src/test/hive/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 similarity index 100% rename from src/test/hive/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 rename to shark/src/test/hive/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 diff --git a/src/test/hive/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 b/shark/src/test/hive/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 similarity index 100% rename from src/test/hive/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 rename to shark/src/test/hive/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 diff --git a/src/test/hive/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd b/shark/src/test/hive/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd similarity index 100% rename from src/test/hive/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd rename to shark/src/test/hive/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd diff --git a/src/test/hive/golden/create_view-11-2021c047203276dd2db51a56e672fcea b/shark/src/test/hive/golden/create_view-11-2021c047203276dd2db51a56e672fcea similarity index 100% rename from src/test/hive/golden/create_view-11-2021c047203276dd2db51a56e672fcea rename to shark/src/test/hive/golden/create_view-11-2021c047203276dd2db51a56e672fcea diff --git a/src/test/hive/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc b/shark/src/test/hive/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc similarity index 100% rename from src/test/hive/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc rename to shark/src/test/hive/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc diff --git a/src/test/hive/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a b/shark/src/test/hive/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a similarity index 100% rename from src/test/hive/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a rename to shark/src/test/hive/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a diff --git a/src/test/hive/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 b/shark/src/test/hive/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 similarity index 100% rename from src/test/hive/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 rename to shark/src/test/hive/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 diff --git a/src/test/hive/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 b/shark/src/test/hive/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 similarity index 100% rename from src/test/hive/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 rename to shark/src/test/hive/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 diff --git a/src/test/hive/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 b/shark/src/test/hive/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 similarity index 100% rename from src/test/hive/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 rename to shark/src/test/hive/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 diff --git a/src/test/hive/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a b/shark/src/test/hive/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a similarity index 100% rename from src/test/hive/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a rename to shark/src/test/hive/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a diff --git a/src/test/hive/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 b/shark/src/test/hive/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 similarity index 100% rename from src/test/hive/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 rename to shark/src/test/hive/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 diff --git a/src/test/hive/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b b/shark/src/test/hive/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b similarity index 100% rename from src/test/hive/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b rename to shark/src/test/hive/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b diff --git a/src/test/hive/golden/create_view-2-d80dcd1271ab264292e9938f3162427c b/shark/src/test/hive/golden/create_view-2-d80dcd1271ab264292e9938f3162427c similarity index 100% rename from src/test/hive/golden/create_view-2-d80dcd1271ab264292e9938f3162427c rename to shark/src/test/hive/golden/create_view-2-d80dcd1271ab264292e9938f3162427c diff --git a/src/test/hive/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 b/shark/src/test/hive/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 similarity index 100% rename from src/test/hive/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 rename to shark/src/test/hive/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 diff --git a/src/test/hive/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 b/shark/src/test/hive/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 similarity index 100% rename from src/test/hive/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 rename to shark/src/test/hive/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 diff --git a/src/test/hive/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 b/shark/src/test/hive/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 similarity index 100% rename from src/test/hive/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 rename to shark/src/test/hive/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 diff --git a/src/test/hive/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 b/shark/src/test/hive/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 similarity index 100% rename from src/test/hive/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 rename to shark/src/test/hive/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 diff --git a/src/test/hive/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 b/shark/src/test/hive/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 similarity index 100% rename from src/test/hive/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 rename to shark/src/test/hive/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 diff --git a/src/test/hive/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 b/shark/src/test/hive/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 similarity index 100% rename from src/test/hive/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 rename to shark/src/test/hive/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 diff --git a/src/test/hive/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 b/shark/src/test/hive/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 similarity index 100% rename from src/test/hive/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 rename to shark/src/test/hive/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 diff --git a/src/test/hive/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 b/shark/src/test/hive/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 similarity index 100% rename from src/test/hive/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 rename to shark/src/test/hive/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 diff --git a/src/test/hive/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 b/shark/src/test/hive/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 similarity index 100% rename from src/test/hive/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 rename to shark/src/test/hive/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 diff --git a/src/test/hive/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 b/shark/src/test/hive/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 similarity index 100% rename from src/test/hive/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 rename to shark/src/test/hive/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 diff --git a/src/test/hive/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc b/shark/src/test/hive/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc similarity index 100% rename from src/test/hive/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc rename to shark/src/test/hive/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc diff --git a/src/test/hive/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 b/shark/src/test/hive/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 similarity index 100% rename from src/test/hive/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 rename to shark/src/test/hive/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 diff --git a/src/test/hive/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 b/shark/src/test/hive/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 similarity index 100% rename from src/test/hive/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 rename to shark/src/test/hive/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 diff --git a/src/test/hive/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 b/shark/src/test/hive/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 similarity index 100% rename from src/test/hive/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 rename to shark/src/test/hive/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 diff --git a/src/test/hive/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 b/shark/src/test/hive/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 similarity index 100% rename from src/test/hive/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 rename to shark/src/test/hive/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 diff --git a/src/test/hive/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc b/shark/src/test/hive/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc similarity index 100% rename from src/test/hive/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc rename to shark/src/test/hive/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc diff --git a/src/test/hive/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 b/shark/src/test/hive/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 similarity index 100% rename from src/test/hive/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 rename to shark/src/test/hive/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 diff --git a/src/test/hive/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 b/shark/src/test/hive/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 similarity index 100% rename from src/test/hive/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 rename to shark/src/test/hive/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 diff --git a/src/test/hive/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 b/shark/src/test/hive/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 similarity index 100% rename from src/test/hive/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 rename to shark/src/test/hive/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 diff --git a/src/test/hive/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e b/shark/src/test/hive/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e similarity index 100% rename from src/test/hive/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e rename to shark/src/test/hive/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e diff --git a/src/test/hive/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 b/shark/src/test/hive/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 similarity index 100% rename from src/test/hive/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 rename to shark/src/test/hive/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 diff --git a/src/test/hive/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 b/shark/src/test/hive/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 similarity index 100% rename from src/test/hive/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 rename to shark/src/test/hive/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 diff --git a/src/test/hive/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b b/shark/src/test/hive/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b similarity index 100% rename from src/test/hive/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b rename to shark/src/test/hive/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b diff --git a/src/test/hive/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 b/shark/src/test/hive/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 similarity index 100% rename from src/test/hive/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 rename to shark/src/test/hive/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 diff --git a/src/test/hive/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f b/shark/src/test/hive/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f similarity index 100% rename from src/test/hive/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f rename to shark/src/test/hive/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f diff --git a/src/test/hive/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 b/shark/src/test/hive/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 similarity index 100% rename from src/test/hive/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 rename to shark/src/test/hive/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 diff --git a/src/test/hive/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec b/shark/src/test/hive/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec similarity index 100% rename from src/test/hive/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec rename to shark/src/test/hive/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec diff --git a/src/test/hive/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 b/shark/src/test/hive/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 similarity index 100% rename from src/test/hive/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 rename to shark/src/test/hive/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 diff --git a/src/test/hive/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 b/shark/src/test/hive/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 similarity index 100% rename from src/test/hive/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 rename to shark/src/test/hive/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 diff --git a/src/test/hive/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd b/shark/src/test/hive/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd similarity index 100% rename from src/test/hive/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd rename to shark/src/test/hive/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd diff --git a/src/test/hive/golden/database.table-0-c657beb729b6a7882309a203fc6f298e b/shark/src/test/hive/golden/database.table-0-c657beb729b6a7882309a203fc6f298e similarity index 100% rename from src/test/hive/golden/database.table-0-c657beb729b6a7882309a203fc6f298e rename to shark/src/test/hive/golden/database.table-0-c657beb729b6a7882309a203fc6f298e diff --git a/src/test/hive/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f b/shark/src/test/hive/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f similarity index 100% rename from src/test/hive/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f rename to shark/src/test/hive/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f diff --git a/src/test/hive/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 b/shark/src/test/hive/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 similarity index 100% rename from src/test/hive/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 rename to shark/src/test/hive/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 diff --git a/src/test/hive/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 b/shark/src/test/hive/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 similarity index 100% rename from src/test/hive/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 rename to shark/src/test/hive/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 diff --git a/src/test/hive/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 b/shark/src/test/hive/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 similarity index 100% rename from src/test/hive/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 rename to shark/src/test/hive/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 diff --git a/src/test/hive/golden/database_drop-12-e02a53f7e798d2741152526516f14941 b/shark/src/test/hive/golden/database_drop-12-e02a53f7e798d2741152526516f14941 similarity index 100% rename from src/test/hive/golden/database_drop-12-e02a53f7e798d2741152526516f14941 rename to shark/src/test/hive/golden/database_drop-12-e02a53f7e798d2741152526516f14941 diff --git a/src/test/hive/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 b/shark/src/test/hive/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 similarity index 100% rename from src/test/hive/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 rename to shark/src/test/hive/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 diff --git a/src/test/hive/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 b/shark/src/test/hive/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 similarity index 100% rename from src/test/hive/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 rename to shark/src/test/hive/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 diff --git a/src/test/hive/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 b/shark/src/test/hive/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 similarity index 100% rename from src/test/hive/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 rename to shark/src/test/hive/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 diff --git a/src/test/hive/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 b/shark/src/test/hive/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 similarity index 100% rename from src/test/hive/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 rename to shark/src/test/hive/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 diff --git a/src/test/hive/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 b/shark/src/test/hive/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 similarity index 100% rename from src/test/hive/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 rename to shark/src/test/hive/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 diff --git a/src/test/hive/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba b/shark/src/test/hive/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba similarity index 100% rename from src/test/hive/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba rename to shark/src/test/hive/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba diff --git a/src/test/hive/golden/database_drop-8-97101266791d2b2c662bcde549422318 b/shark/src/test/hive/golden/database_drop-8-97101266791d2b2c662bcde549422318 similarity index 100% rename from src/test/hive/golden/database_drop-8-97101266791d2b2c662bcde549422318 rename to shark/src/test/hive/golden/database_drop-8-97101266791d2b2c662bcde549422318 diff --git a/src/test/hive/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 b/shark/src/test/hive/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 similarity index 100% rename from src/test/hive/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 rename to shark/src/test/hive/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 diff --git a/src/test/hive/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a b/shark/src/test/hive/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a similarity index 100% rename from src/test/hive/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a rename to shark/src/test/hive/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a diff --git a/src/test/hive/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 b/shark/src/test/hive/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 similarity index 100% rename from src/test/hive/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 rename to shark/src/test/hive/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 diff --git a/src/test/hive/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 b/shark/src/test/hive/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 similarity index 100% rename from src/test/hive/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 rename to shark/src/test/hive/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 diff --git a/src/test/hive/golden/database_location-11-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/database_location-11-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/database_location-11-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/database_location-11-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 b/shark/src/test/hive/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 similarity index 100% rename from src/test/hive/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 rename to shark/src/test/hive/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 diff --git a/src/test/hive/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 b/shark/src/test/hive/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 similarity index 100% rename from src/test/hive/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 rename to shark/src/test/hive/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 diff --git a/src/test/hive/golden/database_location-4-be65cf649363681d54e593c42a5ecffb b/shark/src/test/hive/golden/database_location-4-be65cf649363681d54e593c42a5ecffb similarity index 100% rename from src/test/hive/golden/database_location-4-be65cf649363681d54e593c42a5ecffb rename to shark/src/test/hive/golden/database_location-4-be65cf649363681d54e593c42a5ecffb diff --git a/src/test/hive/golden/database_location-5-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/database_location-5-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/database_location-5-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/database_location-5-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a b/shark/src/test/hive/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a similarity index 100% rename from src/test/hive/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a rename to shark/src/test/hive/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a diff --git a/src/test/hive/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 b/shark/src/test/hive/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 similarity index 100% rename from src/test/hive/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 rename to shark/src/test/hive/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 diff --git a/src/test/hive/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b b/shark/src/test/hive/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b similarity index 100% rename from src/test/hive/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b rename to shark/src/test/hive/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b diff --git a/src/test/hive/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 b/shark/src/test/hive/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 similarity index 100% rename from src/test/hive/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 rename to shark/src/test/hive/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 diff --git a/src/test/hive/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 b/shark/src/test/hive/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 similarity index 100% rename from src/test/hive/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 rename to shark/src/test/hive/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 diff --git a/src/test/hive/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 b/shark/src/test/hive/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 similarity index 100% rename from src/test/hive/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 rename to shark/src/test/hive/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 diff --git a/src/test/hive/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 b/shark/src/test/hive/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 similarity index 100% rename from src/test/hive/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 rename to shark/src/test/hive/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 diff --git a/src/test/hive/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 b/shark/src/test/hive/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 similarity index 100% rename from src/test/hive/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 rename to shark/src/test/hive/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 diff --git a/src/test/hive/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 b/shark/src/test/hive/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 similarity index 100% rename from src/test/hive/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 rename to shark/src/test/hive/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 diff --git a/src/test/hive/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a b/shark/src/test/hive/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a similarity index 100% rename from src/test/hive/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a rename to shark/src/test/hive/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a diff --git a/src/test/hive/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 b/shark/src/test/hive/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 similarity index 100% rename from src/test/hive/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 rename to shark/src/test/hive/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 diff --git a/src/test/hive/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 b/shark/src/test/hive/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 similarity index 100% rename from src/test/hive/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 rename to shark/src/test/hive/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 diff --git a/src/test/hive/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a b/shark/src/test/hive/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a similarity index 100% rename from src/test/hive/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a rename to shark/src/test/hive/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a diff --git a/src/test/hive/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd b/shark/src/test/hive/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd similarity index 100% rename from src/test/hive/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd rename to shark/src/test/hive/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd diff --git a/src/test/hive/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 b/shark/src/test/hive/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 similarity index 100% rename from src/test/hive/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 rename to shark/src/test/hive/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 diff --git a/src/test/hive/golden/date_2-2-cab14d992c53c106ab257fae52001e04 b/shark/src/test/hive/golden/date_2-2-cab14d992c53c106ab257fae52001e04 similarity index 100% rename from src/test/hive/golden/date_2-2-cab14d992c53c106ab257fae52001e04 rename to shark/src/test/hive/golden/date_2-2-cab14d992c53c106ab257fae52001e04 diff --git a/src/test/hive/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 b/shark/src/test/hive/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 similarity index 100% rename from src/test/hive/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 rename to shark/src/test/hive/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 diff --git a/src/test/hive/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 b/shark/src/test/hive/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 similarity index 100% rename from src/test/hive/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 rename to shark/src/test/hive/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 diff --git a/src/test/hive/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 b/shark/src/test/hive/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 similarity index 100% rename from src/test/hive/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 rename to shark/src/test/hive/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 diff --git a/src/test/hive/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c b/shark/src/test/hive/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c similarity index 100% rename from src/test/hive/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c rename to shark/src/test/hive/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c diff --git a/src/test/hive/golden/date_join1-1-3a68de2112a212a07a3068916c608fb b/shark/src/test/hive/golden/date_join1-1-3a68de2112a212a07a3068916c608fb similarity index 100% rename from src/test/hive/golden/date_join1-1-3a68de2112a212a07a3068916c608fb rename to shark/src/test/hive/golden/date_join1-1-3a68de2112a212a07a3068916c608fb diff --git a/src/test/hive/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 b/shark/src/test/hive/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 similarity index 100% rename from src/test/hive/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 rename to shark/src/test/hive/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 diff --git a/src/test/hive/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 b/shark/src/test/hive/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 similarity index 100% rename from src/test/hive/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 rename to shark/src/test/hive/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 diff --git a/src/test/hive/golden/date_serde-1-36e6041f53433482631018410bb62a99 b/shark/src/test/hive/golden/date_serde-1-36e6041f53433482631018410bb62a99 similarity index 100% rename from src/test/hive/golden/date_serde-1-36e6041f53433482631018410bb62a99 rename to shark/src/test/hive/golden/date_serde-1-36e6041f53433482631018410bb62a99 diff --git a/src/test/hive/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 b/shark/src/test/hive/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 similarity index 100% rename from src/test/hive/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 rename to shark/src/test/hive/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 diff --git a/src/test/hive/golden/date_serde-3-e6167e27465514356c557a77d956ea46 b/shark/src/test/hive/golden/date_serde-3-e6167e27465514356c557a77d956ea46 similarity index 100% rename from src/test/hive/golden/date_serde-3-e6167e27465514356c557a77d956ea46 rename to shark/src/test/hive/golden/date_serde-3-e6167e27465514356c557a77d956ea46 diff --git a/src/test/hive/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 b/shark/src/test/hive/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 similarity index 100% rename from src/test/hive/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 rename to shark/src/test/hive/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 diff --git a/src/test/hive/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c b/shark/src/test/hive/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c similarity index 100% rename from src/test/hive/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c rename to shark/src/test/hive/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c diff --git a/src/test/hive/golden/date_serde-6-e00facec2986bc55074868eff87ba22a b/shark/src/test/hive/golden/date_serde-6-e00facec2986bc55074868eff87ba22a similarity index 100% rename from src/test/hive/golden/date_serde-6-e00facec2986bc55074868eff87ba22a rename to shark/src/test/hive/golden/date_serde-6-e00facec2986bc55074868eff87ba22a diff --git a/src/test/hive/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d b/shark/src/test/hive/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d similarity index 100% rename from src/test/hive/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d rename to shark/src/test/hive/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d diff --git a/src/test/hive/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 b/shark/src/test/hive/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 similarity index 100% rename from src/test/hive/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 rename to shark/src/test/hive/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 diff --git a/src/test/hive/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 b/shark/src/test/hive/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 similarity index 100% rename from src/test/hive/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 rename to shark/src/test/hive/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 diff --git a/src/test/hive/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 b/shark/src/test/hive/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 similarity index 100% rename from src/test/hive/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 rename to shark/src/test/hive/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 diff --git a/src/test/hive/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd b/shark/src/test/hive/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd similarity index 100% rename from src/test/hive/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd rename to shark/src/test/hive/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd diff --git a/src/test/hive/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 b/shark/src/test/hive/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 similarity index 100% rename from src/test/hive/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 rename to shark/src/test/hive/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 diff --git a/src/test/hive/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 b/shark/src/test/hive/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 similarity index 100% rename from src/test/hive/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 rename to shark/src/test/hive/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 diff --git a/src/test/hive/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 b/shark/src/test/hive/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 similarity index 100% rename from src/test/hive/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 rename to shark/src/test/hive/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 diff --git a/src/test/hive/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 b/shark/src/test/hive/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 similarity index 100% rename from src/test/hive/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 rename to shark/src/test/hive/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 diff --git a/src/test/hive/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 b/shark/src/test/hive/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 similarity index 100% rename from src/test/hive/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 rename to shark/src/test/hive/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 diff --git a/src/test/hive/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 b/shark/src/test/hive/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 similarity index 100% rename from src/test/hive/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 rename to shark/src/test/hive/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 diff --git a/src/test/hive/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 b/shark/src/test/hive/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 similarity index 100% rename from src/test/hive/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 rename to shark/src/test/hive/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 diff --git a/src/test/hive/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 b/shark/src/test/hive/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 similarity index 100% rename from src/test/hive/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 rename to shark/src/test/hive/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 diff --git a/src/test/hive/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b b/shark/src/test/hive/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b similarity index 100% rename from src/test/hive/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b rename to shark/src/test/hive/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b diff --git a/src/test/hive/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a b/shark/src/test/hive/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a similarity index 100% rename from src/test/hive/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a rename to shark/src/test/hive/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a diff --git a/src/test/hive/golden/decimal_1-8-cdd0932288d3cc43636334439805769d b/shark/src/test/hive/golden/decimal_1-8-cdd0932288d3cc43636334439805769d similarity index 100% rename from src/test/hive/golden/decimal_1-8-cdd0932288d3cc43636334439805769d rename to shark/src/test/hive/golden/decimal_1-8-cdd0932288d3cc43636334439805769d diff --git a/src/test/hive/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c b/shark/src/test/hive/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c similarity index 100% rename from src/test/hive/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c rename to shark/src/test/hive/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c diff --git a/src/test/hive/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee b/shark/src/test/hive/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee similarity index 100% rename from src/test/hive/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee rename to shark/src/test/hive/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee diff --git a/src/test/hive/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b b/shark/src/test/hive/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b similarity index 100% rename from src/test/hive/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b rename to shark/src/test/hive/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b diff --git a/src/test/hive/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 b/shark/src/test/hive/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 similarity index 100% rename from src/test/hive/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 rename to shark/src/test/hive/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 diff --git a/src/test/hive/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 b/shark/src/test/hive/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 similarity index 100% rename from src/test/hive/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 rename to shark/src/test/hive/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 diff --git a/src/test/hive/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 b/shark/src/test/hive/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 similarity index 100% rename from src/test/hive/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 rename to shark/src/test/hive/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 diff --git a/src/test/hive/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 b/shark/src/test/hive/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 similarity index 100% rename from src/test/hive/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 rename to shark/src/test/hive/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 diff --git a/src/test/hive/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 b/shark/src/test/hive/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 similarity index 100% rename from src/test/hive/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 rename to shark/src/test/hive/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 diff --git a/src/test/hive/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 b/shark/src/test/hive/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 similarity index 100% rename from src/test/hive/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 rename to shark/src/test/hive/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 diff --git a/src/test/hive/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 b/shark/src/test/hive/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 similarity index 100% rename from src/test/hive/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 rename to shark/src/test/hive/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 diff --git a/src/test/hive/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d b/shark/src/test/hive/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d similarity index 100% rename from src/test/hive/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d rename to shark/src/test/hive/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d diff --git a/src/test/hive/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e b/shark/src/test/hive/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e similarity index 100% rename from src/test/hive/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e rename to shark/src/test/hive/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e diff --git a/src/test/hive/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 b/shark/src/test/hive/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 similarity index 100% rename from src/test/hive/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 rename to shark/src/test/hive/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 diff --git a/src/test/hive/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 b/shark/src/test/hive/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 similarity index 100% rename from src/test/hive/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 rename to shark/src/test/hive/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 diff --git a/src/test/hive/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 b/shark/src/test/hive/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 similarity index 100% rename from src/test/hive/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 rename to shark/src/test/hive/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 diff --git a/src/test/hive/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 b/shark/src/test/hive/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 similarity index 100% rename from src/test/hive/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 rename to shark/src/test/hive/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 diff --git a/src/test/hive/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 b/shark/src/test/hive/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 similarity index 100% rename from src/test/hive/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 rename to shark/src/test/hive/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 diff --git a/src/test/hive/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff b/shark/src/test/hive/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff similarity index 100% rename from src/test/hive/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff rename to shark/src/test/hive/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff diff --git a/src/test/hive/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e b/shark/src/test/hive/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e similarity index 100% rename from src/test/hive/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e rename to shark/src/test/hive/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e diff --git a/src/test/hive/golden/decimal_2-25-14face5c7104382196e65741a199c36 b/shark/src/test/hive/golden/decimal_2-25-14face5c7104382196e65741a199c36 similarity index 100% rename from src/test/hive/golden/decimal_2-25-14face5c7104382196e65741a199c36 rename to shark/src/test/hive/golden/decimal_2-25-14face5c7104382196e65741a199c36 diff --git a/src/test/hive/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 b/shark/src/test/hive/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 similarity index 100% rename from src/test/hive/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 rename to shark/src/test/hive/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 diff --git a/src/test/hive/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 b/shark/src/test/hive/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 similarity index 100% rename from src/test/hive/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 rename to shark/src/test/hive/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 diff --git a/src/test/hive/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 b/shark/src/test/hive/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 similarity index 100% rename from src/test/hive/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 rename to shark/src/test/hive/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 diff --git a/src/test/hive/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 b/shark/src/test/hive/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 similarity index 100% rename from src/test/hive/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 rename to shark/src/test/hive/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 diff --git a/src/test/hive/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e b/shark/src/test/hive/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e similarity index 100% rename from src/test/hive/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e rename to shark/src/test/hive/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e diff --git a/src/test/hive/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a b/shark/src/test/hive/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a similarity index 100% rename from src/test/hive/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a rename to shark/src/test/hive/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a diff --git a/src/test/hive/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee b/shark/src/test/hive/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee similarity index 100% rename from src/test/hive/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee rename to shark/src/test/hive/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee diff --git a/src/test/hive/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 b/shark/src/test/hive/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 similarity index 100% rename from src/test/hive/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 rename to shark/src/test/hive/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 diff --git a/src/test/hive/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 b/shark/src/test/hive/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 similarity index 100% rename from src/test/hive/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 rename to shark/src/test/hive/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 diff --git a/src/test/hive/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 b/shark/src/test/hive/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 similarity index 100% rename from src/test/hive/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 rename to shark/src/test/hive/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 diff --git a/src/test/hive/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 b/shark/src/test/hive/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 similarity index 100% rename from src/test/hive/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 rename to shark/src/test/hive/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 diff --git a/src/test/hive/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d b/shark/src/test/hive/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d similarity index 100% rename from src/test/hive/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d rename to shark/src/test/hive/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d diff --git a/src/test/hive/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e b/shark/src/test/hive/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e similarity index 100% rename from src/test/hive/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e rename to shark/src/test/hive/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e diff --git a/src/test/hive/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef b/shark/src/test/hive/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef similarity index 100% rename from src/test/hive/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef rename to shark/src/test/hive/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef diff --git a/src/test/hive/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 b/shark/src/test/hive/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 similarity index 100% rename from src/test/hive/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 rename to shark/src/test/hive/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 diff --git a/src/test/hive/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 b/shark/src/test/hive/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 similarity index 100% rename from src/test/hive/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 rename to shark/src/test/hive/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 diff --git a/src/test/hive/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 b/shark/src/test/hive/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 similarity index 100% rename from src/test/hive/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 rename to shark/src/test/hive/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 diff --git a/src/test/hive/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 b/shark/src/test/hive/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 similarity index 100% rename from src/test/hive/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 rename to shark/src/test/hive/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 diff --git a/src/test/hive/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 b/shark/src/test/hive/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 similarity index 100% rename from src/test/hive/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 rename to shark/src/test/hive/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 diff --git a/src/test/hive/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b b/shark/src/test/hive/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b similarity index 100% rename from src/test/hive/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b rename to shark/src/test/hive/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b diff --git a/src/test/hive/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 b/shark/src/test/hive/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 similarity index 100% rename from src/test/hive/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 rename to shark/src/test/hive/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 diff --git a/src/test/hive/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b b/shark/src/test/hive/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b similarity index 100% rename from src/test/hive/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b rename to shark/src/test/hive/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b diff --git a/src/test/hive/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 b/shark/src/test/hive/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 similarity index 100% rename from src/test/hive/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 rename to shark/src/test/hive/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 diff --git a/src/test/hive/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 b/shark/src/test/hive/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 similarity index 100% rename from src/test/hive/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 rename to shark/src/test/hive/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 diff --git a/src/test/hive/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 b/shark/src/test/hive/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 similarity index 100% rename from src/test/hive/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 rename to shark/src/test/hive/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 diff --git a/src/test/hive/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 b/shark/src/test/hive/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 similarity index 100% rename from src/test/hive/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 rename to shark/src/test/hive/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 diff --git a/src/test/hive/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f b/shark/src/test/hive/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f similarity index 100% rename from src/test/hive/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f rename to shark/src/test/hive/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f diff --git a/src/test/hive/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 b/shark/src/test/hive/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 similarity index 100% rename from src/test/hive/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 rename to shark/src/test/hive/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 diff --git a/src/test/hive/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d b/shark/src/test/hive/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d similarity index 100% rename from src/test/hive/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d rename to shark/src/test/hive/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d diff --git a/src/test/hive/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e b/shark/src/test/hive/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e similarity index 100% rename from src/test/hive/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e rename to shark/src/test/hive/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e diff --git a/src/test/hive/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c b/shark/src/test/hive/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c similarity index 100% rename from src/test/hive/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c rename to shark/src/test/hive/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c diff --git a/src/test/hive/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c b/shark/src/test/hive/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c similarity index 100% rename from src/test/hive/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c rename to shark/src/test/hive/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c diff --git a/src/test/hive/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e b/shark/src/test/hive/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e similarity index 100% rename from src/test/hive/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e rename to shark/src/test/hive/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e diff --git a/src/test/hive/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 b/shark/src/test/hive/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 similarity index 100% rename from src/test/hive/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 rename to shark/src/test/hive/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 diff --git a/src/test/hive/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 b/shark/src/test/hive/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 similarity index 100% rename from src/test/hive/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 rename to shark/src/test/hive/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 diff --git a/src/test/hive/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 b/shark/src/test/hive/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 similarity index 100% rename from src/test/hive/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 rename to shark/src/test/hive/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 diff --git a/src/test/hive/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 b/shark/src/test/hive/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 similarity index 100% rename from src/test/hive/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 rename to shark/src/test/hive/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 diff --git a/src/test/hive/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e b/shark/src/test/hive/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e similarity index 100% rename from src/test/hive/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e rename to shark/src/test/hive/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e diff --git a/src/test/hive/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 b/shark/src/test/hive/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 similarity index 100% rename from src/test/hive/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 rename to shark/src/test/hive/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 diff --git a/src/test/hive/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e b/shark/src/test/hive/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e similarity index 100% rename from src/test/hive/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e rename to shark/src/test/hive/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e diff --git a/src/test/hive/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 b/shark/src/test/hive/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 similarity index 100% rename from src/test/hive/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 rename to shark/src/test/hive/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 diff --git a/src/test/hive/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 b/shark/src/test/hive/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 similarity index 100% rename from src/test/hive/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 rename to shark/src/test/hive/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 diff --git a/src/test/hive/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 b/shark/src/test/hive/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 similarity index 100% rename from src/test/hive/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 rename to shark/src/test/hive/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 diff --git a/src/test/hive/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e b/shark/src/test/hive/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e similarity index 100% rename from src/test/hive/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e rename to shark/src/test/hive/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e diff --git a/src/test/hive/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e b/shark/src/test/hive/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e similarity index 100% rename from src/test/hive/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e rename to shark/src/test/hive/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e diff --git a/src/test/hive/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 b/shark/src/test/hive/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 similarity index 100% rename from src/test/hive/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 rename to shark/src/test/hive/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 diff --git a/src/test/hive/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce b/shark/src/test/hive/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce similarity index 100% rename from src/test/hive/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce rename to shark/src/test/hive/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce diff --git a/src/test/hive/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c b/shark/src/test/hive/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c similarity index 100% rename from src/test/hive/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c rename to shark/src/test/hive/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c diff --git a/src/test/hive/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b b/shark/src/test/hive/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b similarity index 100% rename from src/test/hive/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b rename to shark/src/test/hive/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b diff --git a/src/test/hive/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc b/shark/src/test/hive/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc similarity index 100% rename from src/test/hive/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc rename to shark/src/test/hive/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc diff --git a/src/test/hive/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef b/shark/src/test/hive/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef similarity index 100% rename from src/test/hive/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef rename to shark/src/test/hive/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef diff --git a/src/test/hive/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 b/shark/src/test/hive/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 similarity index 100% rename from src/test/hive/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 rename to shark/src/test/hive/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 diff --git a/src/test/hive/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 b/shark/src/test/hive/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 similarity index 100% rename from src/test/hive/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 rename to shark/src/test/hive/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 diff --git a/src/test/hive/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e b/shark/src/test/hive/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e similarity index 100% rename from src/test/hive/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e rename to shark/src/test/hive/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e diff --git a/src/test/hive/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab b/shark/src/test/hive/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab similarity index 100% rename from src/test/hive/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab rename to shark/src/test/hive/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab diff --git a/src/test/hive/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 b/shark/src/test/hive/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 similarity index 100% rename from src/test/hive/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 rename to shark/src/test/hive/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 diff --git a/src/test/hive/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c b/shark/src/test/hive/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c similarity index 100% rename from src/test/hive/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c rename to shark/src/test/hive/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c diff --git a/src/test/hive/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 b/shark/src/test/hive/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 similarity index 100% rename from src/test/hive/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 rename to shark/src/test/hive/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 diff --git a/src/test/hive/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be b/shark/src/test/hive/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be similarity index 100% rename from src/test/hive/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be rename to shark/src/test/hive/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be diff --git a/src/test/hive/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba b/shark/src/test/hive/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba similarity index 100% rename from src/test/hive/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba rename to shark/src/test/hive/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba diff --git a/src/test/hive/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 b/shark/src/test/hive/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 similarity index 100% rename from src/test/hive/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 rename to shark/src/test/hive/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 diff --git a/src/test/hive/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 b/shark/src/test/hive/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 similarity index 100% rename from src/test/hive/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 rename to shark/src/test/hive/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 diff --git a/src/test/hive/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 b/shark/src/test/hive/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 similarity index 100% rename from src/test/hive/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 rename to shark/src/test/hive/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 diff --git a/src/test/hive/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 b/shark/src/test/hive/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 similarity index 100% rename from src/test/hive/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 rename to shark/src/test/hive/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 diff --git a/src/test/hive/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb b/shark/src/test/hive/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb similarity index 100% rename from src/test/hive/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb rename to shark/src/test/hive/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb diff --git a/src/test/hive/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 b/shark/src/test/hive/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 similarity index 100% rename from src/test/hive/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 rename to shark/src/test/hive/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 diff --git a/src/test/hive/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 b/shark/src/test/hive/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 similarity index 100% rename from src/test/hive/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 rename to shark/src/test/hive/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 diff --git a/src/test/hive/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/shark/src/test/hive/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b similarity index 100% rename from src/test/hive/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b rename to shark/src/test/hive/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b diff --git a/src/test/hive/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 b/shark/src/test/hive/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 similarity index 100% rename from src/test/hive/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 rename to shark/src/test/hive/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 diff --git a/src/test/hive/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 b/shark/src/test/hive/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 similarity index 100% rename from src/test/hive/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 rename to shark/src/test/hive/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 diff --git a/src/test/hive/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 b/shark/src/test/hive/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 similarity index 100% rename from src/test/hive/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 rename to shark/src/test/hive/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 diff --git a/src/test/hive/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a b/shark/src/test/hive/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a similarity index 100% rename from src/test/hive/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a rename to shark/src/test/hive/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a diff --git a/src/test/hive/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 b/shark/src/test/hive/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 similarity index 100% rename from src/test/hive/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 rename to shark/src/test/hive/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 diff --git a/src/test/hive/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 b/shark/src/test/hive/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 similarity index 100% rename from src/test/hive/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 rename to shark/src/test/hive/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 diff --git a/src/test/hive/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c b/shark/src/test/hive/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c similarity index 100% rename from src/test/hive/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c rename to shark/src/test/hive/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c diff --git a/src/test/hive/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 b/shark/src/test/hive/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 similarity index 100% rename from src/test/hive/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 rename to shark/src/test/hive/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 diff --git a/src/test/hive/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 b/shark/src/test/hive/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 similarity index 100% rename from src/test/hive/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 rename to shark/src/test/hive/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 diff --git a/src/test/hive/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a b/shark/src/test/hive/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a similarity index 100% rename from src/test/hive/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a rename to shark/src/test/hive/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a diff --git a/src/test/hive/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 b/shark/src/test/hive/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 similarity index 100% rename from src/test/hive/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 rename to shark/src/test/hive/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 diff --git a/src/test/hive/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 b/shark/src/test/hive/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 rename to shark/src/test/hive/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 diff --git a/src/test/hive/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa b/shark/src/test/hive/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa rename to shark/src/test/hive/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa diff --git a/src/test/hive/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 b/shark/src/test/hive/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 rename to shark/src/test/hive/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 diff --git a/src/test/hive/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 b/shark/src/test/hive/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 rename to shark/src/test/hive/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 diff --git a/src/test/hive/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 b/shark/src/test/hive/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 rename to shark/src/test/hive/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/shark/src/test/hive/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b rename to shark/src/test/hive/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 b/shark/src/test/hive/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 rename to shark/src/test/hive/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa b/shark/src/test/hive/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa rename to shark/src/test/hive/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 b/shark/src/test/hive/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 rename to shark/src/test/hive/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 b/shark/src/test/hive/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 rename to shark/src/test/hive/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 diff --git a/src/test/hive/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 b/shark/src/test/hive/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 similarity index 100% rename from src/test/hive/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 rename to shark/src/test/hive/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 diff --git a/src/test/hive/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad b/shark/src/test/hive/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad similarity index 100% rename from src/test/hive/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad rename to shark/src/test/hive/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad diff --git a/src/test/hive/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf b/shark/src/test/hive/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf similarity index 100% rename from src/test/hive/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf rename to shark/src/test/hive/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf diff --git a/src/test/hive/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf b/shark/src/test/hive/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf similarity index 100% rename from src/test/hive/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf rename to shark/src/test/hive/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf diff --git a/src/test/hive/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 b/shark/src/test/hive/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 similarity index 100% rename from src/test/hive/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 rename to shark/src/test/hive/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 diff --git a/src/test/hive/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 b/shark/src/test/hive/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 similarity index 100% rename from src/test/hive/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 rename to shark/src/test/hive/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 diff --git a/src/test/hive/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 b/shark/src/test/hive/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 similarity index 100% rename from src/test/hive/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 rename to shark/src/test/hive/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 diff --git a/src/test/hive/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 b/shark/src/test/hive/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 similarity index 100% rename from src/test/hive/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 rename to shark/src/test/hive/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 diff --git a/src/test/hive/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 b/shark/src/test/hive/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 similarity index 100% rename from src/test/hive/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 rename to shark/src/test/hive/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 diff --git a/src/test/hive/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 b/shark/src/test/hive/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 similarity index 100% rename from src/test/hive/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 rename to shark/src/test/hive/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 diff --git a/src/test/hive/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 b/shark/src/test/hive/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 similarity index 100% rename from src/test/hive/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 rename to shark/src/test/hive/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 diff --git a/src/test/hive/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 b/shark/src/test/hive/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 similarity index 100% rename from src/test/hive/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 rename to shark/src/test/hive/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 diff --git a/src/test/hive/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 b/shark/src/test/hive/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 similarity index 100% rename from src/test/hive/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 rename to shark/src/test/hive/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 diff --git a/src/test/hive/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 b/shark/src/test/hive/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 similarity index 100% rename from src/test/hive/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 rename to shark/src/test/hive/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 diff --git a/src/test/hive/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 b/shark/src/test/hive/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 similarity index 100% rename from src/test/hive/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 rename to shark/src/test/hive/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 diff --git a/src/test/hive/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 b/shark/src/test/hive/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 similarity index 100% rename from src/test/hive/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 rename to shark/src/test/hive/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 diff --git a/src/test/hive/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 b/shark/src/test/hive/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 similarity index 100% rename from src/test/hive/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 rename to shark/src/test/hive/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 diff --git a/src/test/hive/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 b/shark/src/test/hive/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 similarity index 100% rename from src/test/hive/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 rename to shark/src/test/hive/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 diff --git a/src/test/hive/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a b/shark/src/test/hive/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a similarity index 100% rename from src/test/hive/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a rename to shark/src/test/hive/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a diff --git a/src/test/hive/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 b/shark/src/test/hive/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 similarity index 100% rename from src/test/hive/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 rename to shark/src/test/hive/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 diff --git a/src/test/hive/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 b/shark/src/test/hive/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 similarity index 100% rename from src/test/hive/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 rename to shark/src/test/hive/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 diff --git a/src/test/hive/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f b/shark/src/test/hive/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f similarity index 100% rename from src/test/hive/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f rename to shark/src/test/hive/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f diff --git a/src/test/hive/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 b/shark/src/test/hive/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 similarity index 100% rename from src/test/hive/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 rename to shark/src/test/hive/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 diff --git a/src/test/hive/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 b/shark/src/test/hive/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 similarity index 100% rename from src/test/hive/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 rename to shark/src/test/hive/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 diff --git a/src/test/hive/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 b/shark/src/test/hive/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 similarity index 100% rename from src/test/hive/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 rename to shark/src/test/hive/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 diff --git a/src/test/hive/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 b/shark/src/test/hive/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 similarity index 100% rename from src/test/hive/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 rename to shark/src/test/hive/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 diff --git a/src/test/hive/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 b/shark/src/test/hive/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 similarity index 100% rename from src/test/hive/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 rename to shark/src/test/hive/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 diff --git a/src/test/hive/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 b/shark/src/test/hive/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 similarity index 100% rename from src/test/hive/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 rename to shark/src/test/hive/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 diff --git a/src/test/hive/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 b/shark/src/test/hive/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 similarity index 100% rename from src/test/hive/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 rename to shark/src/test/hive/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 diff --git a/src/test/hive/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa b/shark/src/test/hive/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa similarity index 100% rename from src/test/hive/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa rename to shark/src/test/hive/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa diff --git a/src/test/hive/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a b/shark/src/test/hive/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a similarity index 100% rename from src/test/hive/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a rename to shark/src/test/hive/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a diff --git a/src/test/hive/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a b/shark/src/test/hive/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a similarity index 100% rename from src/test/hive/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a rename to shark/src/test/hive/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a diff --git a/src/test/hive/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 b/shark/src/test/hive/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 similarity index 100% rename from src/test/hive/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 rename to shark/src/test/hive/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 diff --git a/src/test/hive/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f b/shark/src/test/hive/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f similarity index 100% rename from src/test/hive/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f rename to shark/src/test/hive/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f diff --git a/src/test/hive/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 b/shark/src/test/hive/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 similarity index 100% rename from src/test/hive/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 rename to shark/src/test/hive/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 diff --git a/src/test/hive/golden/describe_syntax-24-ee226b42db35b219702319858e925468 b/shark/src/test/hive/golden/describe_syntax-24-ee226b42db35b219702319858e925468 similarity index 100% rename from src/test/hive/golden/describe_syntax-24-ee226b42db35b219702319858e925468 rename to shark/src/test/hive/golden/describe_syntax-24-ee226b42db35b219702319858e925468 diff --git a/src/test/hive/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e b/shark/src/test/hive/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e similarity index 100% rename from src/test/hive/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e rename to shark/src/test/hive/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e diff --git a/src/test/hive/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec b/shark/src/test/hive/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec similarity index 100% rename from src/test/hive/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec rename to shark/src/test/hive/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec diff --git a/src/test/hive/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 b/shark/src/test/hive/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 similarity index 100% rename from src/test/hive/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 rename to shark/src/test/hive/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 diff --git a/src/test/hive/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 b/shark/src/test/hive/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 similarity index 100% rename from src/test/hive/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 rename to shark/src/test/hive/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 diff --git a/src/test/hive/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 b/shark/src/test/hive/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 similarity index 100% rename from src/test/hive/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 rename to shark/src/test/hive/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 diff --git a/src/test/hive/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 b/shark/src/test/hive/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 similarity index 100% rename from src/test/hive/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 rename to shark/src/test/hive/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 diff --git a/src/test/hive/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d b/shark/src/test/hive/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d similarity index 100% rename from src/test/hive/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d rename to shark/src/test/hive/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d diff --git a/src/test/hive/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 b/shark/src/test/hive/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 similarity index 100% rename from src/test/hive/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 rename to shark/src/test/hive/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 diff --git a/src/test/hive/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b b/shark/src/test/hive/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b similarity index 100% rename from src/test/hive/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b rename to shark/src/test/hive/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b diff --git a/src/test/hive/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/shark/src/test/hive/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b similarity index 100% rename from src/test/hive/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b rename to shark/src/test/hive/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b diff --git a/src/test/hive/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 b/shark/src/test/hive/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 similarity index 100% rename from src/test/hive/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 rename to shark/src/test/hive/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 diff --git a/src/test/hive/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/describe_table_json-3-576670af142149302decb9bf8662e68a b/shark/src/test/hive/golden/describe_table_json-3-576670af142149302decb9bf8662e68a similarity index 100% rename from src/test/hive/golden/describe_table_json-3-576670af142149302decb9bf8662e68a rename to shark/src/test/hive/golden/describe_table_json-3-576670af142149302decb9bf8662e68a diff --git a/src/test/hive/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 b/shark/src/test/hive/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 similarity index 100% rename from src/test/hive/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 rename to shark/src/test/hive/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 diff --git a/src/test/hive/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb b/shark/src/test/hive/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb similarity index 100% rename from src/test/hive/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb rename to shark/src/test/hive/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb diff --git a/src/test/hive/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 b/shark/src/test/hive/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 similarity index 100% rename from src/test/hive/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 rename to shark/src/test/hive/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 diff --git a/src/test/hive/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 b/shark/src/test/hive/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 similarity index 100% rename from src/test/hive/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 rename to shark/src/test/hive/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 diff --git a/src/test/hive/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 b/shark/src/test/hive/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 similarity index 100% rename from src/test/hive/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 rename to shark/src/test/hive/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 diff --git a/src/test/hive/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 b/shark/src/test/hive/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 similarity index 100% rename from src/test/hive/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 rename to shark/src/test/hive/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 diff --git a/src/test/hive/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad b/shark/src/test/hive/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad similarity index 100% rename from src/test/hive/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad rename to shark/src/test/hive/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad diff --git a/src/test/hive/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa b/shark/src/test/hive/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa similarity index 100% rename from src/test/hive/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa rename to shark/src/test/hive/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa diff --git a/src/test/hive/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce b/shark/src/test/hive/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce similarity index 100% rename from src/test/hive/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce rename to shark/src/test/hive/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce diff --git a/src/test/hive/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 b/shark/src/test/hive/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 similarity index 100% rename from src/test/hive/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 rename to shark/src/test/hive/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 diff --git a/src/test/hive/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b b/shark/src/test/hive/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b similarity index 100% rename from src/test/hive/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b rename to shark/src/test/hive/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b diff --git a/src/test/hive/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c b/shark/src/test/hive/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c similarity index 100% rename from src/test/hive/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c rename to shark/src/test/hive/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c diff --git a/src/test/hive/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e b/shark/src/test/hive/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e similarity index 100% rename from src/test/hive/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e rename to shark/src/test/hive/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e diff --git a/src/test/hive/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 b/shark/src/test/hive/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 similarity index 100% rename from src/test/hive/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 rename to shark/src/test/hive/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 diff --git a/src/test/hive/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f b/shark/src/test/hive/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f similarity index 100% rename from src/test/hive/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f rename to shark/src/test/hive/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f diff --git a/src/test/hive/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb b/shark/src/test/hive/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb similarity index 100% rename from src/test/hive/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb rename to shark/src/test/hive/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a b/shark/src/test/hive/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a similarity index 100% rename from src/test/hive/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a rename to shark/src/test/hive/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b b/shark/src/test/hive/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b similarity index 100% rename from src/test/hive/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b rename to shark/src/test/hive/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 b/shark/src/test/hive/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 similarity index 100% rename from src/test/hive/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 rename to shark/src/test/hive/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 b/shark/src/test/hive/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 similarity index 100% rename from src/test/hive/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 rename to shark/src/test/hive/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c b/shark/src/test/hive/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c similarity index 100% rename from src/test/hive/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c rename to shark/src/test/hive/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c diff --git a/src/test/hive/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 b/shark/src/test/hive/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 similarity index 100% rename from src/test/hive/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 rename to shark/src/test/hive/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 diff --git a/src/test/hive/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab b/shark/src/test/hive/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab similarity index 100% rename from src/test/hive/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab rename to shark/src/test/hive/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab diff --git a/src/test/hive/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 b/shark/src/test/hive/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 similarity index 100% rename from src/test/hive/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 rename to shark/src/test/hive/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 diff --git a/src/test/hive/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/shark/src/test/hive/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 similarity index 100% rename from src/test/hive/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 rename to shark/src/test/hive/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 diff --git a/src/test/hive/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a b/shark/src/test/hive/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a similarity index 100% rename from src/test/hive/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a rename to shark/src/test/hive/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a diff --git a/src/test/hive/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/shark/src/test/hive/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 similarity index 100% rename from src/test/hive/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 rename to shark/src/test/hive/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 diff --git a/src/test/hive/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 b/shark/src/test/hive/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 similarity index 100% rename from src/test/hive/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 rename to shark/src/test/hive/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 diff --git a/src/test/hive/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b b/shark/src/test/hive/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b rename to shark/src/test/hive/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b diff --git a/src/test/hive/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 b/shark/src/test/hive/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 rename to shark/src/test/hive/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 diff --git a/src/test/hive/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 b/shark/src/test/hive/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 rename to shark/src/test/hive/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 diff --git a/src/test/hive/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 b/shark/src/test/hive/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 rename to shark/src/test/hive/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 diff --git a/src/test/hive/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 b/shark/src/test/hive/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 rename to shark/src/test/hive/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 diff --git a/src/test/hive/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 b/shark/src/test/hive/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 rename to shark/src/test/hive/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 diff --git a/src/test/hive/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 b/shark/src/test/hive/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 rename to shark/src/test/hive/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 diff --git a/src/test/hive/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 b/shark/src/test/hive/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 rename to shark/src/test/hive/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 diff --git a/src/test/hive/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 b/shark/src/test/hive/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 rename to shark/src/test/hive/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 diff --git a/src/test/hive/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 b/shark/src/test/hive/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 rename to shark/src/test/hive/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 diff --git a/src/test/hive/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 b/shark/src/test/hive/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 similarity index 100% rename from src/test/hive/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 rename to shark/src/test/hive/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 diff --git a/src/test/hive/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 b/shark/src/test/hive/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 rename to shark/src/test/hive/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 diff --git a/src/test/hive/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 b/shark/src/test/hive/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 rename to shark/src/test/hive/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 diff --git a/src/test/hive/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b b/shark/src/test/hive/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b rename to shark/src/test/hive/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b diff --git a/src/test/hive/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 b/shark/src/test/hive/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 rename to shark/src/test/hive/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 diff --git a/src/test/hive/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 b/shark/src/test/hive/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 rename to shark/src/test/hive/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 diff --git a/src/test/hive/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b b/shark/src/test/hive/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b rename to shark/src/test/hive/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b diff --git a/src/test/hive/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd b/shark/src/test/hive/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd rename to shark/src/test/hive/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd diff --git a/src/test/hive/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 b/shark/src/test/hive/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 rename to shark/src/test/hive/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 diff --git a/src/test/hive/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c b/shark/src/test/hive/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c rename to shark/src/test/hive/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c diff --git a/src/test/hive/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 b/shark/src/test/hive/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 rename to shark/src/test/hive/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 diff --git a/src/test/hive/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e b/shark/src/test/hive/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e rename to shark/src/test/hive/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e diff --git a/src/test/hive/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c b/shark/src/test/hive/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c rename to shark/src/test/hive/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c diff --git a/src/test/hive/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c b/shark/src/test/hive/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c rename to shark/src/test/hive/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c diff --git a/src/test/hive/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d b/shark/src/test/hive/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d rename to shark/src/test/hive/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d diff --git a/src/test/hive/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 b/shark/src/test/hive/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 rename to shark/src/test/hive/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 diff --git a/src/test/hive/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e b/shark/src/test/hive/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e rename to shark/src/test/hive/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e diff --git a/src/test/hive/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 b/shark/src/test/hive/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 rename to shark/src/test/hive/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 diff --git a/src/test/hive/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 b/shark/src/test/hive/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 rename to shark/src/test/hive/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 diff --git a/src/test/hive/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 b/shark/src/test/hive/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 rename to shark/src/test/hive/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 diff --git a/src/test/hive/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 b/shark/src/test/hive/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 rename to shark/src/test/hive/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 diff --git a/src/test/hive/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 b/shark/src/test/hive/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 rename to shark/src/test/hive/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 diff --git a/src/test/hive/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a b/shark/src/test/hive/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a rename to shark/src/test/hive/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a diff --git a/src/test/hive/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c b/shark/src/test/hive/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c rename to shark/src/test/hive/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c diff --git a/src/test/hive/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 b/shark/src/test/hive/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 rename to shark/src/test/hive/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 diff --git a/src/test/hive/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 b/shark/src/test/hive/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 rename to shark/src/test/hive/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 diff --git a/src/test/hive/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 b/shark/src/test/hive/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 rename to shark/src/test/hive/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 diff --git a/src/test/hive/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a b/shark/src/test/hive/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a rename to shark/src/test/hive/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a diff --git a/src/test/hive/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 b/shark/src/test/hive/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 rename to shark/src/test/hive/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 diff --git a/src/test/hive/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 b/shark/src/test/hive/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 rename to shark/src/test/hive/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 diff --git a/src/test/hive/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a b/shark/src/test/hive/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a similarity index 100% rename from src/test/hive/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a rename to shark/src/test/hive/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a diff --git a/src/test/hive/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 b/shark/src/test/hive/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 rename to shark/src/test/hive/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 diff --git a/src/test/hive/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 b/shark/src/test/hive/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 rename to shark/src/test/hive/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 diff --git a/src/test/hive/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 b/shark/src/test/hive/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 rename to shark/src/test/hive/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 diff --git a/src/test/hive/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c b/shark/src/test/hive/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c rename to shark/src/test/hive/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c diff --git a/src/test/hive/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 b/shark/src/test/hive/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 rename to shark/src/test/hive/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 diff --git a/src/test/hive/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec b/shark/src/test/hive/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec rename to shark/src/test/hive/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec diff --git a/src/test/hive/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a b/shark/src/test/hive/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a rename to shark/src/test/hive/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a diff --git a/src/test/hive/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 b/shark/src/test/hive/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 rename to shark/src/test/hive/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 diff --git a/src/test/hive/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae b/shark/src/test/hive/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae rename to shark/src/test/hive/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae diff --git a/src/test/hive/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 b/shark/src/test/hive/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 rename to shark/src/test/hive/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 diff --git a/src/test/hive/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 b/shark/src/test/hive/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 rename to shark/src/test/hive/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 diff --git a/src/test/hive/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 b/shark/src/test/hive/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 similarity index 100% rename from src/test/hive/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 rename to shark/src/test/hive/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb b/shark/src/test/hive/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb similarity index 100% rename from src/test/hive/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb rename to shark/src/test/hive/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 b/shark/src/test/hive/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 similarity index 100% rename from src/test/hive/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 rename to shark/src/test/hive/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 b/shark/src/test/hive/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 similarity index 100% rename from src/test/hive/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 rename to shark/src/test/hive/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 b/shark/src/test/hive/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 similarity index 100% rename from src/test/hive/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 rename to shark/src/test/hive/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 b/shark/src/test/hive/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 similarity index 100% rename from src/test/hive/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 rename to shark/src/test/hive/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 diff --git a/src/test/hive/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 b/shark/src/test/hive/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 similarity index 100% rename from src/test/hive/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 rename to shark/src/test/hive/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 diff --git a/src/test/hive/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/shark/src/test/hive/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 similarity index 100% rename from src/test/hive/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 rename to shark/src/test/hive/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 diff --git a/src/test/hive/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 b/shark/src/test/hive/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 similarity index 100% rename from src/test/hive/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 rename to shark/src/test/hive/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 diff --git a/src/test/hive/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 b/shark/src/test/hive/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 similarity index 100% rename from src/test/hive/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 rename to shark/src/test/hive/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 diff --git a/src/test/hive/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa b/shark/src/test/hive/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa similarity index 100% rename from src/test/hive/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa rename to shark/src/test/hive/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa diff --git a/src/test/hive/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 b/shark/src/test/hive/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 similarity index 100% rename from src/test/hive/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 rename to shark/src/test/hive/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 diff --git a/src/test/hive/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f b/shark/src/test/hive/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f similarity index 100% rename from src/test/hive/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f rename to shark/src/test/hive/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f diff --git a/src/test/hive/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 b/shark/src/test/hive/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 similarity index 100% rename from src/test/hive/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 rename to shark/src/test/hive/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 diff --git a/src/test/hive/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 b/shark/src/test/hive/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 similarity index 100% rename from src/test/hive/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 rename to shark/src/test/hive/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 diff --git a/src/test/hive/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 b/shark/src/test/hive/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 similarity index 100% rename from src/test/hive/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 rename to shark/src/test/hive/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 diff --git a/src/test/hive/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa b/shark/src/test/hive/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa similarity index 100% rename from src/test/hive/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa rename to shark/src/test/hive/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa diff --git a/src/test/hive/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 b/shark/src/test/hive/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 similarity index 100% rename from src/test/hive/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 rename to shark/src/test/hive/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 diff --git a/src/test/hive/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 b/shark/src/test/hive/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 similarity index 100% rename from src/test/hive/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 rename to shark/src/test/hive/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 diff --git a/src/test/hive/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/shark/src/test/hive/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 similarity index 100% rename from src/test/hive/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 rename to shark/src/test/hive/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 diff --git a/src/test/hive/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 b/shark/src/test/hive/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 similarity index 100% rename from src/test/hive/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 rename to shark/src/test/hive/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 diff --git a/src/test/hive/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 b/shark/src/test/hive/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 similarity index 100% rename from src/test/hive/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 rename to shark/src/test/hive/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 diff --git a/src/test/hive/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 b/shark/src/test/hive/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 similarity index 100% rename from src/test/hive/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 rename to shark/src/test/hive/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 diff --git a/src/test/hive/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 b/shark/src/test/hive/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 similarity index 100% rename from src/test/hive/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 rename to shark/src/test/hive/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 diff --git a/src/test/hive/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 b/shark/src/test/hive/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 similarity index 100% rename from src/test/hive/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 rename to shark/src/test/hive/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 diff --git a/src/test/hive/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 b/shark/src/test/hive/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 similarity index 100% rename from src/test/hive/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 rename to shark/src/test/hive/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 diff --git a/src/test/hive/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 b/shark/src/test/hive/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 similarity index 100% rename from src/test/hive/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 rename to shark/src/test/hive/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 diff --git a/src/test/hive/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 b/shark/src/test/hive/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 similarity index 100% rename from src/test/hive/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 rename to shark/src/test/hive/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 diff --git a/src/test/hive/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 b/shark/src/test/hive/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 similarity index 100% rename from src/test/hive/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 rename to shark/src/test/hive/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 diff --git a/src/test/hive/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 b/shark/src/test/hive/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 similarity index 100% rename from src/test/hive/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 rename to shark/src/test/hive/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 diff --git a/src/test/hive/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 b/shark/src/test/hive/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 similarity index 100% rename from src/test/hive/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 rename to shark/src/test/hive/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 diff --git a/src/test/hive/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d b/shark/src/test/hive/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d similarity index 100% rename from src/test/hive/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d rename to shark/src/test/hive/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d diff --git a/src/test/hive/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f b/shark/src/test/hive/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f similarity index 100% rename from src/test/hive/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f rename to shark/src/test/hive/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f diff --git a/src/test/hive/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 b/shark/src/test/hive/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 similarity index 100% rename from src/test/hive/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 rename to shark/src/test/hive/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 diff --git a/src/test/hive/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 b/shark/src/test/hive/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 similarity index 100% rename from src/test/hive/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 rename to shark/src/test/hive/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 diff --git a/src/test/hive/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 b/shark/src/test/hive/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 similarity index 100% rename from src/test/hive/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 rename to shark/src/test/hive/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 diff --git a/src/test/hive/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf b/shark/src/test/hive/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf similarity index 100% rename from src/test/hive/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf rename to shark/src/test/hive/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf diff --git a/src/test/hive/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 b/shark/src/test/hive/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 similarity index 100% rename from src/test/hive/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 rename to shark/src/test/hive/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 diff --git a/src/test/hive/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 b/shark/src/test/hive/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 similarity index 100% rename from src/test/hive/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 rename to shark/src/test/hive/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 diff --git a/src/test/hive/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 b/shark/src/test/hive/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 similarity index 100% rename from src/test/hive/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 rename to shark/src/test/hive/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 diff --git a/src/test/hive/golden/escape1-3-4267651148da591da38737028fdbd80 b/shark/src/test/hive/golden/escape1-3-4267651148da591da38737028fdbd80 similarity index 100% rename from src/test/hive/golden/escape1-3-4267651148da591da38737028fdbd80 rename to shark/src/test/hive/golden/escape1-3-4267651148da591da38737028fdbd80 diff --git a/src/test/hive/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 b/shark/src/test/hive/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 similarity index 100% rename from src/test/hive/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 rename to shark/src/test/hive/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 diff --git a/src/test/hive/golden/escape1-5-70729c3d79ded87e884c176138174645 b/shark/src/test/hive/golden/escape1-5-70729c3d79ded87e884c176138174645 similarity index 100% rename from src/test/hive/golden/escape1-5-70729c3d79ded87e884c176138174645 rename to shark/src/test/hive/golden/escape1-5-70729c3d79ded87e884c176138174645 diff --git a/src/test/hive/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 b/shark/src/test/hive/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 similarity index 100% rename from src/test/hive/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 rename to shark/src/test/hive/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 diff --git a/src/test/hive/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 b/shark/src/test/hive/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 similarity index 100% rename from src/test/hive/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 rename to shark/src/test/hive/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 diff --git a/src/test/hive/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 b/shark/src/test/hive/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 similarity index 100% rename from src/test/hive/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 rename to shark/src/test/hive/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 diff --git a/src/test/hive/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 b/shark/src/test/hive/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 similarity index 100% rename from src/test/hive/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 rename to shark/src/test/hive/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 diff --git a/src/test/hive/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e b/shark/src/test/hive/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e similarity index 100% rename from src/test/hive/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e rename to shark/src/test/hive/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e diff --git a/src/test/hive/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 b/shark/src/test/hive/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 similarity index 100% rename from src/test/hive/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 rename to shark/src/test/hive/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 diff --git a/src/test/hive/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe b/shark/src/test/hive/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe similarity index 100% rename from src/test/hive/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe rename to shark/src/test/hive/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe diff --git a/src/test/hive/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 b/shark/src/test/hive/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 similarity index 100% rename from src/test/hive/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 rename to shark/src/test/hive/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 diff --git a/src/test/hive/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 b/shark/src/test/hive/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 similarity index 100% rename from src/test/hive/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 rename to shark/src/test/hive/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 diff --git a/src/test/hive/golden/escape2-7-70729c3d79ded87e884c176138174645 b/shark/src/test/hive/golden/escape2-7-70729c3d79ded87e884c176138174645 similarity index 100% rename from src/test/hive/golden/escape2-7-70729c3d79ded87e884c176138174645 rename to shark/src/test/hive/golden/escape2-7-70729c3d79ded87e884c176138174645 diff --git a/src/test/hive/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 b/shark/src/test/hive/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 similarity index 100% rename from src/test/hive/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 rename to shark/src/test/hive/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 diff --git a/src/test/hive/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 b/shark/src/test/hive/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 similarity index 100% rename from src/test/hive/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 rename to shark/src/test/hive/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 diff --git a/src/test/hive/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf b/shark/src/test/hive/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf similarity index 100% rename from src/test/hive/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf rename to shark/src/test/hive/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf diff --git a/src/test/hive/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 b/shark/src/test/hive/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 similarity index 100% rename from src/test/hive/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 rename to shark/src/test/hive/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 diff --git a/src/test/hive/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb b/shark/src/test/hive/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb similarity index 100% rename from src/test/hive/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb rename to shark/src/test/hive/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb diff --git a/src/test/hive/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 b/shark/src/test/hive/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 similarity index 100% rename from src/test/hive/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 rename to shark/src/test/hive/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 diff --git a/src/test/hive/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 b/shark/src/test/hive/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 similarity index 100% rename from src/test/hive/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 rename to shark/src/test/hive/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 diff --git a/src/test/hive/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 b/shark/src/test/hive/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 similarity index 100% rename from src/test/hive/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 rename to shark/src/test/hive/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 diff --git a/src/test/hive/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 b/shark/src/test/hive/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 similarity index 100% rename from src/test/hive/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 rename to shark/src/test/hive/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 diff --git a/src/test/hive/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 b/shark/src/test/hive/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 similarity index 100% rename from src/test/hive/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 rename to shark/src/test/hive/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 diff --git a/src/test/hive/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 b/shark/src/test/hive/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 similarity index 100% rename from src/test/hive/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 rename to shark/src/test/hive/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 diff --git a/src/test/hive/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 b/shark/src/test/hive/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 similarity index 100% rename from src/test/hive/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 rename to shark/src/test/hive/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 diff --git a/src/test/hive/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 b/shark/src/test/hive/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 similarity index 100% rename from src/test/hive/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 rename to shark/src/test/hive/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 diff --git a/src/test/hive/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd b/shark/src/test/hive/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd similarity index 100% rename from src/test/hive/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd rename to shark/src/test/hive/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd diff --git a/src/test/hive/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 b/shark/src/test/hive/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 similarity index 100% rename from src/test/hive/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 rename to shark/src/test/hive/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 diff --git a/src/test/hive/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 b/shark/src/test/hive/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 similarity index 100% rename from src/test/hive/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 rename to shark/src/test/hive/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd b/shark/src/test/hive/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd similarity index 100% rename from src/test/hive/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd rename to shark/src/test/hive/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 b/shark/src/test/hive/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 similarity index 100% rename from src/test/hive/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 rename to shark/src/test/hive/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 diff --git a/src/test/hive/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 b/shark/src/test/hive/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 rename to shark/src/test/hive/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 diff --git a/src/test/hive/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 b/shark/src/test/hive/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 rename to shark/src/test/hive/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 diff --git a/src/test/hive/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 b/shark/src/test/hive/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 rename to shark/src/test/hive/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 diff --git a/src/test/hive/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 b/shark/src/test/hive/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 rename to shark/src/test/hive/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 diff --git a/src/test/hive/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 b/shark/src/test/hive/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 rename to shark/src/test/hive/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 diff --git a/src/test/hive/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 b/shark/src/test/hive/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 rename to shark/src/test/hive/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 diff --git a/src/test/hive/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 b/shark/src/test/hive/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 similarity index 100% rename from src/test/hive/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 rename to shark/src/test/hive/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 diff --git a/src/test/hive/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 b/shark/src/test/hive/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 similarity index 100% rename from src/test/hive/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 rename to shark/src/test/hive/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 diff --git a/src/test/hive/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 b/shark/src/test/hive/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 similarity index 100% rename from src/test/hive/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 rename to shark/src/test/hive/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 diff --git a/src/test/hive/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd b/shark/src/test/hive/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd similarity index 100% rename from src/test/hive/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd rename to shark/src/test/hive/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd diff --git a/src/test/hive/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 b/shark/src/test/hive/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 similarity index 100% rename from src/test/hive/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 rename to shark/src/test/hive/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 diff --git a/src/test/hive/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d b/shark/src/test/hive/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d similarity index 100% rename from src/test/hive/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d rename to shark/src/test/hive/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d diff --git a/src/test/hive/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 b/shark/src/test/hive/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 similarity index 100% rename from src/test/hive/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 rename to shark/src/test/hive/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 diff --git a/src/test/hive/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 b/shark/src/test/hive/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 similarity index 100% rename from src/test/hive/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 rename to shark/src/test/hive/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 diff --git a/src/test/hive/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd b/shark/src/test/hive/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd similarity index 100% rename from src/test/hive/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd rename to shark/src/test/hive/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd diff --git a/src/test/hive/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 b/shark/src/test/hive/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 similarity index 100% rename from src/test/hive/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 rename to shark/src/test/hive/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 diff --git a/src/test/hive/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/shark/src/test/hive/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 similarity index 100% rename from src/test/hive/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 rename to shark/src/test/hive/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 diff --git a/src/test/hive/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd b/shark/src/test/hive/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd similarity index 100% rename from src/test/hive/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd rename to shark/src/test/hive/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd diff --git a/src/test/hive/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 b/shark/src/test/hive/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 similarity index 100% rename from src/test/hive/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 rename to shark/src/test/hive/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 diff --git a/src/test/hive/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/shark/src/test/hive/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 similarity index 100% rename from src/test/hive/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 rename to shark/src/test/hive/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 diff --git a/src/test/hive/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd b/shark/src/test/hive/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd similarity index 100% rename from src/test/hive/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd rename to shark/src/test/hive/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd diff --git a/src/test/hive/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 b/shark/src/test/hive/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 similarity index 100% rename from src/test/hive/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 rename to shark/src/test/hive/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 diff --git a/src/test/hive/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 b/shark/src/test/hive/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 similarity index 100% rename from src/test/hive/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 rename to shark/src/test/hive/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd b/shark/src/test/hive/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd similarity index 100% rename from src/test/hive/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd rename to shark/src/test/hive/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 b/shark/src/test/hive/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 similarity index 100% rename from src/test/hive/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 rename to shark/src/test/hive/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 diff --git a/src/test/hive/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 b/shark/src/test/hive/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 similarity index 100% rename from src/test/hive/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 rename to shark/src/test/hive/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 diff --git a/src/test/hive/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/shark/src/test/hive/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a similarity index 100% rename from src/test/hive/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to shark/src/test/hive/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a diff --git a/src/test/hive/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/shark/src/test/hive/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 similarity index 100% rename from src/test/hive/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 rename to shark/src/test/hive/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 diff --git a/src/test/hive/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/shark/src/test/hive/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 similarity index 100% rename from src/test/hive/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to shark/src/test/hive/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 diff --git a/src/test/hive/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 b/shark/src/test/hive/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 similarity index 100% rename from src/test/hive/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 rename to shark/src/test/hive/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 diff --git a/src/test/hive/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd b/shark/src/test/hive/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd similarity index 100% rename from src/test/hive/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd rename to shark/src/test/hive/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd diff --git a/src/test/hive/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e b/shark/src/test/hive/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e similarity index 100% rename from src/test/hive/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e rename to shark/src/test/hive/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e diff --git a/src/test/hive/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 b/shark/src/test/hive/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 similarity index 100% rename from src/test/hive/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 rename to shark/src/test/hive/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 diff --git a/src/test/hive/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 b/shark/src/test/hive/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 similarity index 100% rename from src/test/hive/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 rename to shark/src/test/hive/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 diff --git a/src/test/hive/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/shark/src/test/hive/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 similarity index 100% rename from src/test/hive/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 rename to shark/src/test/hive/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/shark/src/test/hive/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd similarity index 100% rename from src/test/hive/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd rename to shark/src/test/hive/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 b/shark/src/test/hive/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 similarity index 100% rename from src/test/hive/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 rename to shark/src/test/hive/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 diff --git a/src/test/hive/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 b/shark/src/test/hive/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 similarity index 100% rename from src/test/hive/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 rename to shark/src/test/hive/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc b/shark/src/test/hive/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc similarity index 100% rename from src/test/hive/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc rename to shark/src/test/hive/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/shark/src/test/hive/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 similarity index 100% rename from src/test/hive/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to shark/src/test/hive/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 diff --git a/src/test/hive/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 b/shark/src/test/hive/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 similarity index 100% rename from src/test/hive/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 rename to shark/src/test/hive/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a similarity index 100% rename from src/test/hive/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 similarity index 100% rename from src/test/hive/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 similarity index 100% rename from src/test/hive/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 rename to shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd similarity index 100% rename from src/test/hive/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd rename to shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d b/shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d similarity index 100% rename from src/test/hive/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d rename to shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d diff --git a/src/test/hive/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 similarity index 100% rename from src/test/hive/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to shark/src/test/hive/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 diff --git a/src/test/hive/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 b/shark/src/test/hive/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 similarity index 100% rename from src/test/hive/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 rename to shark/src/test/hive/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 diff --git a/src/test/hive/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 b/shark/src/test/hive/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 similarity index 100% rename from src/test/hive/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 rename to shark/src/test/hive/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 diff --git a/src/test/hive/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 b/shark/src/test/hive/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 similarity index 100% rename from src/test/hive/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 rename to shark/src/test/hive/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 diff --git a/src/test/hive/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 b/shark/src/test/hive/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 similarity index 100% rename from src/test/hive/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 rename to shark/src/test/hive/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 diff --git a/src/test/hive/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b b/shark/src/test/hive/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b similarity index 100% rename from src/test/hive/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b rename to shark/src/test/hive/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b diff --git a/src/test/hive/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e b/shark/src/test/hive/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e similarity index 100% rename from src/test/hive/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e rename to shark/src/test/hive/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e diff --git a/src/test/hive/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 b/shark/src/test/hive/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 similarity index 100% rename from src/test/hive/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 rename to shark/src/test/hive/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 diff --git a/src/test/hive/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 b/shark/src/test/hive/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 similarity index 100% rename from src/test/hive/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 rename to shark/src/test/hive/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 diff --git a/src/test/hive/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 b/shark/src/test/hive/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 similarity index 100% rename from src/test/hive/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 rename to shark/src/test/hive/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 diff --git a/src/test/hive/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 b/shark/src/test/hive/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 similarity index 100% rename from src/test/hive/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 rename to shark/src/test/hive/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 diff --git a/src/test/hive/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 b/shark/src/test/hive/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 rename to shark/src/test/hive/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 diff --git a/src/test/hive/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 b/shark/src/test/hive/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 rename to shark/src/test/hive/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 diff --git a/src/test/hive/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 b/shark/src/test/hive/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 rename to shark/src/test/hive/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 diff --git a/src/test/hive/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e b/shark/src/test/hive/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e similarity index 100% rename from src/test/hive/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e rename to shark/src/test/hive/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e diff --git a/src/test/hive/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 b/shark/src/test/hive/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 rename to shark/src/test/hive/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 diff --git a/src/test/hive/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd b/shark/src/test/hive/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd rename to shark/src/test/hive/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd diff --git a/src/test/hive/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd b/shark/src/test/hive/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd rename to shark/src/test/hive/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd diff --git a/src/test/hive/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 b/shark/src/test/hive/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 rename to shark/src/test/hive/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 diff --git a/src/test/hive/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 b/shark/src/test/hive/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 rename to shark/src/test/hive/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 diff --git a/src/test/hive/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c b/shark/src/test/hive/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c rename to shark/src/test/hive/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c diff --git a/src/test/hive/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 b/shark/src/test/hive/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 rename to shark/src/test/hive/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 diff --git a/src/test/hive/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 b/shark/src/test/hive/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 rename to shark/src/test/hive/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 diff --git a/src/test/hive/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba b/shark/src/test/hive/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba rename to shark/src/test/hive/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba diff --git a/src/test/hive/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 b/shark/src/test/hive/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 rename to shark/src/test/hive/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 diff --git a/src/test/hive/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 b/shark/src/test/hive/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 rename to shark/src/test/hive/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 diff --git a/src/test/hive/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e b/shark/src/test/hive/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e rename to shark/src/test/hive/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e diff --git a/src/test/hive/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 b/shark/src/test/hive/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 similarity index 100% rename from src/test/hive/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 rename to shark/src/test/hive/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 diff --git a/src/test/hive/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby1-2-f90acd59ace31e16059bae52583188cc b/shark/src/test/hive/golden/groupby1-2-f90acd59ace31e16059bae52583188cc similarity index 100% rename from src/test/hive/golden/groupby1-2-f90acd59ace31e16059bae52583188cc rename to shark/src/test/hive/golden/groupby1-2-f90acd59ace31e16059bae52583188cc diff --git a/src/test/hive/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d b/shark/src/test/hive/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d similarity index 100% rename from src/test/hive/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d rename to shark/src/test/hive/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d diff --git a/src/test/hive/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 b/shark/src/test/hive/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 similarity index 100% rename from src/test/hive/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 rename to shark/src/test/hive/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 diff --git a/src/test/hive/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 b/shark/src/test/hive/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 similarity index 100% rename from src/test/hive/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 rename to shark/src/test/hive/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 diff --git a/src/test/hive/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea b/shark/src/test/hive/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea similarity index 100% rename from src/test/hive/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea rename to shark/src/test/hive/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea diff --git a/src/test/hive/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 b/shark/src/test/hive/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 similarity index 100% rename from src/test/hive/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 rename to shark/src/test/hive/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 diff --git a/src/test/hive/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 b/shark/src/test/hive/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 similarity index 100% rename from src/test/hive/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 rename to shark/src/test/hive/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 diff --git a/src/test/hive/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 b/shark/src/test/hive/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 similarity index 100% rename from src/test/hive/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 rename to shark/src/test/hive/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 diff --git a/src/test/hive/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 b/shark/src/test/hive/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 similarity index 100% rename from src/test/hive/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 rename to shark/src/test/hive/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 diff --git a/src/test/hive/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f b/shark/src/test/hive/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f similarity index 100% rename from src/test/hive/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f rename to shark/src/test/hive/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f diff --git a/src/test/hive/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd b/shark/src/test/hive/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd similarity index 100% rename from src/test/hive/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd rename to shark/src/test/hive/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd diff --git a/src/test/hive/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d b/shark/src/test/hive/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d similarity index 100% rename from src/test/hive/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d rename to shark/src/test/hive/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d diff --git a/src/test/hive/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 b/shark/src/test/hive/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 similarity index 100% rename from src/test/hive/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 rename to shark/src/test/hive/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 diff --git a/src/test/hive/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 b/shark/src/test/hive/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 similarity index 100% rename from src/test/hive/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 rename to shark/src/test/hive/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 diff --git a/src/test/hive/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 b/shark/src/test/hive/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 similarity index 100% rename from src/test/hive/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 rename to shark/src/test/hive/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 diff --git a/src/test/hive/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 b/shark/src/test/hive/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 similarity index 100% rename from src/test/hive/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 rename to shark/src/test/hive/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 diff --git a/src/test/hive/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 b/shark/src/test/hive/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 similarity index 100% rename from src/test/hive/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 rename to shark/src/test/hive/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 diff --git a/src/test/hive/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe b/shark/src/test/hive/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe similarity index 100% rename from src/test/hive/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe rename to shark/src/test/hive/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe diff --git a/src/test/hive/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 b/shark/src/test/hive/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 similarity index 100% rename from src/test/hive/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 rename to shark/src/test/hive/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 diff --git a/src/test/hive/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 b/shark/src/test/hive/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 similarity index 100% rename from src/test/hive/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 rename to shark/src/test/hive/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 diff --git a/src/test/hive/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 b/shark/src/test/hive/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 similarity index 100% rename from src/test/hive/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 rename to shark/src/test/hive/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 diff --git a/src/test/hive/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe b/shark/src/test/hive/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe similarity index 100% rename from src/test/hive/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe rename to shark/src/test/hive/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe diff --git a/src/test/hive/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 b/shark/src/test/hive/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 similarity index 100% rename from src/test/hive/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 rename to shark/src/test/hive/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 diff --git a/src/test/hive/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 b/shark/src/test/hive/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 similarity index 100% rename from src/test/hive/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 rename to shark/src/test/hive/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 diff --git a/src/test/hive/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe b/shark/src/test/hive/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe similarity index 100% rename from src/test/hive/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe rename to shark/src/test/hive/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe diff --git a/src/test/hive/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 b/shark/src/test/hive/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 similarity index 100% rename from src/test/hive/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 rename to shark/src/test/hive/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 diff --git a/src/test/hive/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc b/shark/src/test/hive/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc similarity index 100% rename from src/test/hive/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc rename to shark/src/test/hive/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc diff --git a/src/test/hive/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 b/shark/src/test/hive/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 similarity index 100% rename from src/test/hive/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 rename to shark/src/test/hive/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 diff --git a/src/test/hive/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea b/shark/src/test/hive/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea similarity index 100% rename from src/test/hive/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea rename to shark/src/test/hive/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea diff --git a/src/test/hive/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 b/shark/src/test/hive/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 similarity index 100% rename from src/test/hive/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 rename to shark/src/test/hive/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 diff --git a/src/test/hive/golden/groupby2-0-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/groupby2-0-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/groupby2-0-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/groupby2-0-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b b/shark/src/test/hive/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b similarity index 100% rename from src/test/hive/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b rename to shark/src/test/hive/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b diff --git a/src/test/hive/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 b/shark/src/test/hive/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 similarity index 100% rename from src/test/hive/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 rename to shark/src/test/hive/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 diff --git a/src/test/hive/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 b/shark/src/test/hive/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 similarity index 100% rename from src/test/hive/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 rename to shark/src/test/hive/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 diff --git a/src/test/hive/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c b/shark/src/test/hive/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c similarity index 100% rename from src/test/hive/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c rename to shark/src/test/hive/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c diff --git a/src/test/hive/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 b/shark/src/test/hive/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 similarity index 100% rename from src/test/hive/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 rename to shark/src/test/hive/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 diff --git a/src/test/hive/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 b/shark/src/test/hive/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 similarity index 100% rename from src/test/hive/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 rename to shark/src/test/hive/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 diff --git a/src/test/hive/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 b/shark/src/test/hive/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 similarity index 100% rename from src/test/hive/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 rename to shark/src/test/hive/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 diff --git a/src/test/hive/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 b/shark/src/test/hive/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 similarity index 100% rename from src/test/hive/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 rename to shark/src/test/hive/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 diff --git a/src/test/hive/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 b/shark/src/test/hive/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 similarity index 100% rename from src/test/hive/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 rename to shark/src/test/hive/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 diff --git a/src/test/hive/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 b/shark/src/test/hive/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 similarity index 100% rename from src/test/hive/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 rename to shark/src/test/hive/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 b/shark/src/test/hive/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 similarity index 100% rename from src/test/hive/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 rename to shark/src/test/hive/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 b/shark/src/test/hive/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 similarity index 100% rename from src/test/hive/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 rename to shark/src/test/hive/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 diff --git a/src/test/hive/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 b/shark/src/test/hive/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 similarity index 100% rename from src/test/hive/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 rename to shark/src/test/hive/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 diff --git a/src/test/hive/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 b/shark/src/test/hive/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 similarity index 100% rename from src/test/hive/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 rename to shark/src/test/hive/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 diff --git a/src/test/hive/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 b/shark/src/test/hive/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 similarity index 100% rename from src/test/hive/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 rename to shark/src/test/hive/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 diff --git a/src/test/hive/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b b/shark/src/test/hive/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b similarity index 100% rename from src/test/hive/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b rename to shark/src/test/hive/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b diff --git a/src/test/hive/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 b/shark/src/test/hive/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 similarity index 100% rename from src/test/hive/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 rename to shark/src/test/hive/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 diff --git a/src/test/hive/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 b/shark/src/test/hive/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 similarity index 100% rename from src/test/hive/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 rename to shark/src/test/hive/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 diff --git a/src/test/hive/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c b/shark/src/test/hive/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c similarity index 100% rename from src/test/hive/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c rename to shark/src/test/hive/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 b/shark/src/test/hive/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 similarity index 100% rename from src/test/hive/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 rename to shark/src/test/hive/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 b/shark/src/test/hive/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 similarity index 100% rename from src/test/hive/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 rename to shark/src/test/hive/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 b/shark/src/test/hive/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 similarity index 100% rename from src/test/hive/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 rename to shark/src/test/hive/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 diff --git a/src/test/hive/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c b/shark/src/test/hive/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c similarity index 100% rename from src/test/hive/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c rename to shark/src/test/hive/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c diff --git a/src/test/hive/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 b/shark/src/test/hive/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 similarity index 100% rename from src/test/hive/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 rename to shark/src/test/hive/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 diff --git a/src/test/hive/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 b/shark/src/test/hive/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 similarity index 100% rename from src/test/hive/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 rename to shark/src/test/hive/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 diff --git a/src/test/hive/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 b/shark/src/test/hive/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 similarity index 100% rename from src/test/hive/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 rename to shark/src/test/hive/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 diff --git a/src/test/hive/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b b/shark/src/test/hive/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b similarity index 100% rename from src/test/hive/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b rename to shark/src/test/hive/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b diff --git a/src/test/hive/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 b/shark/src/test/hive/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 similarity index 100% rename from src/test/hive/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 rename to shark/src/test/hive/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 diff --git a/src/test/hive/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 b/shark/src/test/hive/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 similarity index 100% rename from src/test/hive/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 rename to shark/src/test/hive/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 diff --git a/src/test/hive/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b b/shark/src/test/hive/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b similarity index 100% rename from src/test/hive/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b rename to shark/src/test/hive/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b diff --git a/src/test/hive/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 b/shark/src/test/hive/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 similarity index 100% rename from src/test/hive/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 rename to shark/src/test/hive/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 diff --git a/src/test/hive/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 b/shark/src/test/hive/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 similarity index 100% rename from src/test/hive/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 rename to shark/src/test/hive/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 diff --git a/src/test/hive/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 b/shark/src/test/hive/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 similarity index 100% rename from src/test/hive/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 rename to shark/src/test/hive/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 diff --git a/src/test/hive/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 b/shark/src/test/hive/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 similarity index 100% rename from src/test/hive/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 rename to shark/src/test/hive/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 diff --git a/src/test/hive/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e b/shark/src/test/hive/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e similarity index 100% rename from src/test/hive/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e rename to shark/src/test/hive/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e diff --git a/src/test/hive/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 b/shark/src/test/hive/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 similarity index 100% rename from src/test/hive/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 rename to shark/src/test/hive/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 diff --git a/src/test/hive/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f b/shark/src/test/hive/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f similarity index 100% rename from src/test/hive/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f rename to shark/src/test/hive/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f diff --git a/src/test/hive/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 b/shark/src/test/hive/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 similarity index 100% rename from src/test/hive/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 rename to shark/src/test/hive/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 diff --git a/src/test/hive/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 b/shark/src/test/hive/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 similarity index 100% rename from src/test/hive/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 rename to shark/src/test/hive/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 diff --git a/src/test/hive/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f b/shark/src/test/hive/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f similarity index 100% rename from src/test/hive/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f rename to shark/src/test/hive/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f diff --git a/src/test/hive/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 b/shark/src/test/hive/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 similarity index 100% rename from src/test/hive/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 rename to shark/src/test/hive/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 diff --git a/src/test/hive/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 b/shark/src/test/hive/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 similarity index 100% rename from src/test/hive/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 rename to shark/src/test/hive/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 diff --git a/src/test/hive/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e b/shark/src/test/hive/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e similarity index 100% rename from src/test/hive/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e rename to shark/src/test/hive/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e diff --git a/src/test/hive/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac b/shark/src/test/hive/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac similarity index 100% rename from src/test/hive/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac rename to shark/src/test/hive/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac diff --git a/src/test/hive/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 b/shark/src/test/hive/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 similarity index 100% rename from src/test/hive/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 rename to shark/src/test/hive/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 diff --git a/src/test/hive/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac b/shark/src/test/hive/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac similarity index 100% rename from src/test/hive/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac rename to shark/src/test/hive/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac diff --git a/src/test/hive/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 b/shark/src/test/hive/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 similarity index 100% rename from src/test/hive/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 rename to shark/src/test/hive/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 diff --git a/src/test/hive/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac b/shark/src/test/hive/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac similarity index 100% rename from src/test/hive/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac rename to shark/src/test/hive/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac diff --git a/src/test/hive/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 b/shark/src/test/hive/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 similarity index 100% rename from src/test/hive/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 rename to shark/src/test/hive/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 diff --git a/src/test/hive/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac b/shark/src/test/hive/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac similarity index 100% rename from src/test/hive/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac rename to shark/src/test/hive/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac diff --git a/src/test/hive/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 b/shark/src/test/hive/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 similarity index 100% rename from src/test/hive/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 rename to shark/src/test/hive/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 diff --git a/src/test/hive/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a b/shark/src/test/hive/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a similarity index 100% rename from src/test/hive/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a rename to shark/src/test/hive/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a diff --git a/src/test/hive/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/shark/src/test/hive/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c similarity index 100% rename from src/test/hive/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c rename to shark/src/test/hive/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c diff --git a/src/test/hive/golden/groupby7-5-ed76c0068780120a6f23feefee303403 b/shark/src/test/hive/golden/groupby7-5-ed76c0068780120a6f23feefee303403 similarity index 100% rename from src/test/hive/golden/groupby7-5-ed76c0068780120a6f23feefee303403 rename to shark/src/test/hive/golden/groupby7-5-ed76c0068780120a6f23feefee303403 diff --git a/src/test/hive/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 b/shark/src/test/hive/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 similarity index 100% rename from src/test/hive/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 rename to shark/src/test/hive/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 diff --git a/src/test/hive/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e b/shark/src/test/hive/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e similarity index 100% rename from src/test/hive/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e rename to shark/src/test/hive/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e diff --git a/src/test/hive/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a b/shark/src/test/hive/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a similarity index 100% rename from src/test/hive/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a rename to shark/src/test/hive/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a diff --git a/src/test/hive/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/shark/src/test/hive/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c similarity index 100% rename from src/test/hive/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c rename to shark/src/test/hive/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c diff --git a/src/test/hive/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 b/shark/src/test/hive/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 similarity index 100% rename from src/test/hive/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 rename to shark/src/test/hive/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 diff --git a/src/test/hive/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd b/shark/src/test/hive/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd similarity index 100% rename from src/test/hive/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd rename to shark/src/test/hive/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd diff --git a/src/test/hive/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 b/shark/src/test/hive/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 similarity index 100% rename from src/test/hive/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 rename to shark/src/test/hive/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 diff --git a/src/test/hive/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a b/shark/src/test/hive/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a rename to shark/src/test/hive/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a diff --git a/src/test/hive/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/shark/src/test/hive/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c rename to shark/src/test/hive/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c diff --git a/src/test/hive/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 b/shark/src/test/hive/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 rename to shark/src/test/hive/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 diff --git a/src/test/hive/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd b/shark/src/test/hive/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd rename to shark/src/test/hive/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd diff --git a/src/test/hive/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 b/shark/src/test/hive/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 rename to shark/src/test/hive/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 diff --git a/src/test/hive/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e b/shark/src/test/hive/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e similarity index 100% rename from src/test/hive/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e rename to shark/src/test/hive/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e diff --git a/src/test/hive/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a b/shark/src/test/hive/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a similarity index 100% rename from src/test/hive/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a rename to shark/src/test/hive/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a diff --git a/src/test/hive/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/shark/src/test/hive/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c similarity index 100% rename from src/test/hive/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c rename to shark/src/test/hive/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c diff --git a/src/test/hive/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 b/shark/src/test/hive/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 similarity index 100% rename from src/test/hive/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 rename to shark/src/test/hive/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 diff --git a/src/test/hive/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd b/shark/src/test/hive/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd similarity index 100% rename from src/test/hive/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd rename to shark/src/test/hive/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd diff --git a/src/test/hive/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 b/shark/src/test/hive/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 similarity index 100% rename from src/test/hive/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 rename to shark/src/test/hive/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a diff --git a/src/test/hive/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 b/shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 similarity index 100% rename from src/test/hive/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 rename to shark/src/test/hive/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 diff --git a/src/test/hive/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f b/shark/src/test/hive/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f similarity index 100% rename from src/test/hive/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f rename to shark/src/test/hive/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f diff --git a/src/test/hive/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a b/shark/src/test/hive/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a similarity index 100% rename from src/test/hive/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a rename to shark/src/test/hive/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a diff --git a/src/test/hive/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d b/shark/src/test/hive/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d similarity index 100% rename from src/test/hive/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d rename to shark/src/test/hive/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d diff --git a/src/test/hive/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f b/shark/src/test/hive/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f similarity index 100% rename from src/test/hive/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f rename to shark/src/test/hive/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f diff --git a/src/test/hive/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e b/shark/src/test/hive/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e similarity index 100% rename from src/test/hive/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e rename to shark/src/test/hive/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e diff --git a/src/test/hive/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d b/shark/src/test/hive/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d similarity index 100% rename from src/test/hive/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d rename to shark/src/test/hive/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d diff --git a/src/test/hive/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a b/shark/src/test/hive/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a similarity index 100% rename from src/test/hive/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a rename to shark/src/test/hive/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a diff --git a/src/test/hive/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d b/shark/src/test/hive/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d similarity index 100% rename from src/test/hive/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d rename to shark/src/test/hive/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d diff --git a/src/test/hive/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f b/shark/src/test/hive/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f similarity index 100% rename from src/test/hive/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f rename to shark/src/test/hive/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f diff --git a/src/test/hive/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a b/shark/src/test/hive/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a similarity index 100% rename from src/test/hive/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a rename to shark/src/test/hive/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a diff --git a/src/test/hive/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d b/shark/src/test/hive/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d similarity index 100% rename from src/test/hive/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d rename to shark/src/test/hive/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d diff --git a/src/test/hive/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f b/shark/src/test/hive/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f similarity index 100% rename from src/test/hive/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f rename to shark/src/test/hive/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f diff --git a/src/test/hive/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a b/shark/src/test/hive/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a similarity index 100% rename from src/test/hive/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a rename to shark/src/test/hive/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a diff --git a/src/test/hive/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d b/shark/src/test/hive/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d similarity index 100% rename from src/test/hive/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d rename to shark/src/test/hive/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d diff --git a/src/test/hive/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f b/shark/src/test/hive/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f similarity index 100% rename from src/test/hive/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f rename to shark/src/test/hive/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f diff --git a/src/test/hive/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 b/shark/src/test/hive/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 similarity index 100% rename from src/test/hive/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 rename to shark/src/test/hive/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 diff --git a/src/test/hive/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 b/shark/src/test/hive/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 similarity index 100% rename from src/test/hive/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 rename to shark/src/test/hive/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 diff --git a/src/test/hive/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e b/shark/src/test/hive/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e similarity index 100% rename from src/test/hive/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e rename to shark/src/test/hive/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e diff --git a/src/test/hive/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f b/shark/src/test/hive/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f similarity index 100% rename from src/test/hive/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f rename to shark/src/test/hive/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f diff --git a/src/test/hive/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 b/shark/src/test/hive/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 similarity index 100% rename from src/test/hive/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 rename to shark/src/test/hive/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 diff --git a/src/test/hive/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed b/shark/src/test/hive/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed similarity index 100% rename from src/test/hive/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed rename to shark/src/test/hive/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed diff --git a/src/test/hive/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b b/shark/src/test/hive/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b similarity index 100% rename from src/test/hive/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b rename to shark/src/test/hive/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b diff --git a/src/test/hive/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e b/shark/src/test/hive/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e similarity index 100% rename from src/test/hive/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e rename to shark/src/test/hive/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e diff --git a/src/test/hive/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f b/shark/src/test/hive/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f similarity index 100% rename from src/test/hive/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f rename to shark/src/test/hive/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f diff --git a/src/test/hive/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 b/shark/src/test/hive/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 similarity index 100% rename from src/test/hive/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 rename to shark/src/test/hive/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 diff --git a/src/test/hive/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 b/shark/src/test/hive/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 similarity index 100% rename from src/test/hive/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 rename to shark/src/test/hive/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 diff --git a/src/test/hive/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e b/shark/src/test/hive/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e similarity index 100% rename from src/test/hive/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e rename to shark/src/test/hive/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e diff --git a/src/test/hive/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 b/shark/src/test/hive/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 similarity index 100% rename from src/test/hive/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 rename to shark/src/test/hive/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 diff --git a/src/test/hive/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/shark/src/test/hive/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e similarity index 100% rename from src/test/hive/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e rename to shark/src/test/hive/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e diff --git a/src/test/hive/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 b/shark/src/test/hive/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 similarity index 100% rename from src/test/hive/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 rename to shark/src/test/hive/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 diff --git a/src/test/hive/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/shark/src/test/hive/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 similarity index 100% rename from src/test/hive/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 rename to shark/src/test/hive/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 diff --git a/src/test/hive/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 b/shark/src/test/hive/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 similarity index 100% rename from src/test/hive/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 rename to shark/src/test/hive/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 diff --git a/src/test/hive/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 b/shark/src/test/hive/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 similarity index 100% rename from src/test/hive/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 rename to shark/src/test/hive/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 diff --git a/src/test/hive/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a b/shark/src/test/hive/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a similarity index 100% rename from src/test/hive/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a rename to shark/src/test/hive/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a diff --git a/src/test/hive/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 b/shark/src/test/hive/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 similarity index 100% rename from src/test/hive/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 rename to shark/src/test/hive/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 diff --git a/src/test/hive/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/shark/src/test/hive/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 similarity index 100% rename from src/test/hive/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 rename to shark/src/test/hive/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 diff --git a/src/test/hive/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 b/shark/src/test/hive/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 similarity index 100% rename from src/test/hive/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 rename to shark/src/test/hive/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 diff --git a/src/test/hive/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 b/shark/src/test/hive/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 similarity index 100% rename from src/test/hive/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 rename to shark/src/test/hive/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 diff --git a/src/test/hive/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 b/shark/src/test/hive/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 similarity index 100% rename from src/test/hive/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 rename to shark/src/test/hive/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 diff --git a/src/test/hive/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 b/shark/src/test/hive/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 similarity index 100% rename from src/test/hive/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 rename to shark/src/test/hive/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 diff --git a/src/test/hive/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 b/shark/src/test/hive/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 similarity index 100% rename from src/test/hive/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 rename to shark/src/test/hive/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 diff --git a/src/test/hive/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 b/shark/src/test/hive/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 similarity index 100% rename from src/test/hive/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 rename to shark/src/test/hive/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 diff --git a/src/test/hive/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 b/shark/src/test/hive/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 similarity index 100% rename from src/test/hive/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 rename to shark/src/test/hive/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 diff --git a/src/test/hive/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 b/shark/src/test/hive/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 similarity index 100% rename from src/test/hive/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 rename to shark/src/test/hive/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 diff --git a/src/test/hive/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad b/shark/src/test/hive/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad similarity index 100% rename from src/test/hive/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad rename to shark/src/test/hive/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad diff --git a/src/test/hive/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 b/shark/src/test/hive/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 similarity index 100% rename from src/test/hive/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 rename to shark/src/test/hive/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 diff --git a/src/test/hive/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 b/shark/src/test/hive/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 similarity index 100% rename from src/test/hive/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 rename to shark/src/test/hive/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 diff --git a/src/test/hive/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 b/shark/src/test/hive/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 rename to shark/src/test/hive/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 diff --git a/src/test/hive/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 b/shark/src/test/hive/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 rename to shark/src/test/hive/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 diff --git a/src/test/hive/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a b/shark/src/test/hive/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a similarity index 100% rename from src/test/hive/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a rename to shark/src/test/hive/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a diff --git a/src/test/hive/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 b/shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 rename to shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 b/shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 rename to shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 b/shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 similarity index 100% rename from src/test/hive/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 rename to shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 diff --git a/src/test/hive/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 diff --git a/src/test/hive/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e b/shark/src/test/hive/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e similarity index 100% rename from src/test/hive/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e rename to shark/src/test/hive/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e diff --git a/src/test/hive/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/shark/src/test/hive/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 b/shark/src/test/hive/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd b/shark/src/test/hive/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd rename to shark/src/test/hive/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd diff --git a/src/test/hive/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b b/shark/src/test/hive/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b rename to shark/src/test/hive/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b diff --git a/src/test/hive/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 b/shark/src/test/hive/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e b/shark/src/test/hive/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e rename to shark/src/test/hive/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e diff --git a/src/test/hive/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 b/shark/src/test/hive/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd b/shark/src/test/hive/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd rename to shark/src/test/hive/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd diff --git a/src/test/hive/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 b/shark/src/test/hive/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae b/shark/src/test/hive/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae rename to shark/src/test/hive/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae diff --git a/src/test/hive/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab b/shark/src/test/hive/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab rename to shark/src/test/hive/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab diff --git a/src/test/hive/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 b/shark/src/test/hive/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 b/shark/src/test/hive/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc b/shark/src/test/hive/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc rename to shark/src/test/hive/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc diff --git a/src/test/hive/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 b/shark/src/test/hive/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f b/shark/src/test/hive/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f rename to shark/src/test/hive/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f diff --git a/src/test/hive/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 b/shark/src/test/hive/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 b/shark/src/test/hive/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea b/shark/src/test/hive/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea rename to shark/src/test/hive/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea diff --git a/src/test/hive/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b b/shark/src/test/hive/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b rename to shark/src/test/hive/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b diff --git a/src/test/hive/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 b/shark/src/test/hive/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 diff --git a/src/test/hive/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e b/shark/src/test/hive/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e rename to shark/src/test/hive/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e diff --git a/src/test/hive/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 b/shark/src/test/hive/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 rename to shark/src/test/hive/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/shark/src/test/hive/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 rename to shark/src/test/hive/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada b/shark/src/test/hive/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada rename to shark/src/test/hive/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 b/shark/src/test/hive/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 rename to shark/src/test/hive/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e b/shark/src/test/hive/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e rename to shark/src/test/hive/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 b/shark/src/test/hive/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 rename to shark/src/test/hive/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 b/shark/src/test/hive/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 rename to shark/src/test/hive/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 b/shark/src/test/hive/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 rename to shark/src/test/hive/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab b/shark/src/test/hive/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab rename to shark/src/test/hive/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab diff --git a/src/test/hive/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 b/shark/src/test/hive/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 rename to shark/src/test/hive/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd b/shark/src/test/hive/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d b/shark/src/test/hive/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a b/shark/src/test/hive/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b b/shark/src/test/hive/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a b/shark/src/test/hive/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b b/shark/src/test/hive/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 diff --git a/src/test/hive/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 b/shark/src/test/hive/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 similarity index 100% rename from src/test/hive/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 rename to shark/src/test/hive/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 diff --git a/src/test/hive/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb b/shark/src/test/hive/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb similarity index 100% rename from src/test/hive/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb rename to shark/src/test/hive/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb diff --git a/src/test/hive/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 b/shark/src/test/hive/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 similarity index 100% rename from src/test/hive/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 rename to shark/src/test/hive/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 diff --git a/src/test/hive/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 b/shark/src/test/hive/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 similarity index 100% rename from src/test/hive/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 rename to shark/src/test/hive/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 diff --git a/src/test/hive/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 b/shark/src/test/hive/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 similarity index 100% rename from src/test/hive/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 rename to shark/src/test/hive/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 diff --git a/src/test/hive/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 b/shark/src/test/hive/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 similarity index 100% rename from src/test/hive/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 rename to shark/src/test/hive/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 diff --git a/src/test/hive/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 b/shark/src/test/hive/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 similarity index 100% rename from src/test/hive/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 rename to shark/src/test/hive/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 diff --git a/src/test/hive/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f b/shark/src/test/hive/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f similarity index 100% rename from src/test/hive/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f rename to shark/src/test/hive/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f diff --git a/src/test/hive/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 b/shark/src/test/hive/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 similarity index 100% rename from src/test/hive/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 rename to shark/src/test/hive/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 diff --git a/src/test/hive/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb b/shark/src/test/hive/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb similarity index 100% rename from src/test/hive/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb rename to shark/src/test/hive/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb diff --git a/src/test/hive/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 b/shark/src/test/hive/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 similarity index 100% rename from src/test/hive/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 rename to shark/src/test/hive/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 diff --git a/src/test/hive/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac b/shark/src/test/hive/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac similarity index 100% rename from src/test/hive/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac rename to shark/src/test/hive/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac diff --git a/src/test/hive/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c b/shark/src/test/hive/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c similarity index 100% rename from src/test/hive/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c rename to shark/src/test/hive/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c diff --git a/src/test/hive/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d b/shark/src/test/hive/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d similarity index 100% rename from src/test/hive/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d rename to shark/src/test/hive/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d diff --git a/src/test/hive/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 b/shark/src/test/hive/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 similarity index 100% rename from src/test/hive/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 rename to shark/src/test/hive/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 diff --git a/src/test/hive/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 b/shark/src/test/hive/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 similarity index 100% rename from src/test/hive/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 rename to shark/src/test/hive/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 diff --git a/src/test/hive/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c b/shark/src/test/hive/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c similarity index 100% rename from src/test/hive/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c rename to shark/src/test/hive/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c diff --git a/src/test/hive/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d b/shark/src/test/hive/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d similarity index 100% rename from src/test/hive/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d rename to shark/src/test/hive/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d diff --git a/src/test/hive/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 b/shark/src/test/hive/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 similarity index 100% rename from src/test/hive/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 rename to shark/src/test/hive/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 diff --git a/src/test/hive/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 b/shark/src/test/hive/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 similarity index 100% rename from src/test/hive/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 rename to shark/src/test/hive/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 diff --git a/src/test/hive/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a b/shark/src/test/hive/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a similarity index 100% rename from src/test/hive/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a rename to shark/src/test/hive/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a diff --git a/src/test/hive/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 b/shark/src/test/hive/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 similarity index 100% rename from src/test/hive/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 rename to shark/src/test/hive/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d b/shark/src/test/hive/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d similarity index 100% rename from src/test/hive/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d rename to shark/src/test/hive/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d b/shark/src/test/hive/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d similarity index 100% rename from src/test/hive/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d rename to shark/src/test/hive/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d diff --git a/src/test/hive/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 b/shark/src/test/hive/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 rename to shark/src/test/hive/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 diff --git a/src/test/hive/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c b/shark/src/test/hive/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c similarity index 100% rename from src/test/hive/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c rename to shark/src/test/hive/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c diff --git a/src/test/hive/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 b/shark/src/test/hive/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 rename to shark/src/test/hive/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 diff --git a/src/test/hive/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 b/shark/src/test/hive/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 rename to shark/src/test/hive/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 diff --git a/src/test/hive/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea b/shark/src/test/hive/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea similarity index 100% rename from src/test/hive/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea rename to shark/src/test/hive/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea diff --git a/src/test/hive/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 b/shark/src/test/hive/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 rename to shark/src/test/hive/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 diff --git a/src/test/hive/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 b/shark/src/test/hive/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 rename to shark/src/test/hive/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 diff --git a/src/test/hive/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 b/shark/src/test/hive/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 rename to shark/src/test/hive/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 diff --git a/src/test/hive/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 b/shark/src/test/hive/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 rename to shark/src/test/hive/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 diff --git a/src/test/hive/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 b/shark/src/test/hive/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 rename to shark/src/test/hive/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 diff --git a/src/test/hive/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 b/shark/src/test/hive/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 rename to shark/src/test/hive/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 diff --git a/src/test/hive/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 b/shark/src/test/hive/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 rename to shark/src/test/hive/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 diff --git a/src/test/hive/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 b/shark/src/test/hive/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 rename to shark/src/test/hive/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 diff --git a/src/test/hive/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 b/shark/src/test/hive/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 rename to shark/src/test/hive/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 diff --git a/src/test/hive/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f b/shark/src/test/hive/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f similarity index 100% rename from src/test/hive/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f rename to shark/src/test/hive/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f diff --git a/src/test/hive/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca b/shark/src/test/hive/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca similarity index 100% rename from src/test/hive/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca rename to shark/src/test/hive/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca diff --git a/src/test/hive/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 b/shark/src/test/hive/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 rename to shark/src/test/hive/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 diff --git a/src/test/hive/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 b/shark/src/test/hive/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 rename to shark/src/test/hive/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 diff --git a/src/test/hive/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 b/shark/src/test/hive/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 rename to shark/src/test/hive/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 diff --git a/src/test/hive/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f b/shark/src/test/hive/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f similarity index 100% rename from src/test/hive/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f rename to shark/src/test/hive/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f diff --git a/src/test/hive/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef b/shark/src/test/hive/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef similarity index 100% rename from src/test/hive/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef rename to shark/src/test/hive/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef diff --git a/src/test/hive/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 b/shark/src/test/hive/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 rename to shark/src/test/hive/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 diff --git a/src/test/hive/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 b/shark/src/test/hive/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 rename to shark/src/test/hive/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 diff --git a/src/test/hive/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d b/shark/src/test/hive/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d similarity index 100% rename from src/test/hive/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d rename to shark/src/test/hive/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d diff --git a/src/test/hive/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b b/shark/src/test/hive/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b similarity index 100% rename from src/test/hive/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b rename to shark/src/test/hive/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b diff --git a/src/test/hive/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad b/shark/src/test/hive/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad similarity index 100% rename from src/test/hive/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad rename to shark/src/test/hive/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad diff --git a/src/test/hive/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 b/shark/src/test/hive/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 rename to shark/src/test/hive/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 diff --git a/src/test/hive/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c b/shark/src/test/hive/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c similarity index 100% rename from src/test/hive/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c rename to shark/src/test/hive/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c diff --git a/src/test/hive/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a b/shark/src/test/hive/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a similarity index 100% rename from src/test/hive/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a rename to shark/src/test/hive/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a diff --git a/src/test/hive/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 b/shark/src/test/hive/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 rename to shark/src/test/hive/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 diff --git a/src/test/hive/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 b/shark/src/test/hive/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 rename to shark/src/test/hive/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 diff --git a/src/test/hive/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa b/shark/src/test/hive/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa similarity index 100% rename from src/test/hive/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa rename to shark/src/test/hive/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa diff --git a/src/test/hive/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 b/shark/src/test/hive/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 rename to shark/src/test/hive/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 diff --git a/src/test/hive/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 b/shark/src/test/hive/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 rename to shark/src/test/hive/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 diff --git a/src/test/hive/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 b/shark/src/test/hive/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 rename to shark/src/test/hive/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 diff --git a/src/test/hive/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca b/shark/src/test/hive/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca similarity index 100% rename from src/test/hive/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca rename to shark/src/test/hive/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca diff --git a/src/test/hive/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 b/shark/src/test/hive/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 rename to shark/src/test/hive/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 diff --git a/src/test/hive/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c b/shark/src/test/hive/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c similarity index 100% rename from src/test/hive/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c rename to shark/src/test/hive/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c diff --git a/src/test/hive/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 b/shark/src/test/hive/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 rename to shark/src/test/hive/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 diff --git a/src/test/hive/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 b/shark/src/test/hive/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 rename to shark/src/test/hive/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 diff --git a/src/test/hive/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 b/shark/src/test/hive/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 rename to shark/src/test/hive/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 diff --git a/src/test/hive/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 b/shark/src/test/hive/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 rename to shark/src/test/hive/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 diff --git a/src/test/hive/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca b/shark/src/test/hive/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca similarity index 100% rename from src/test/hive/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca rename to shark/src/test/hive/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca diff --git a/src/test/hive/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b b/shark/src/test/hive/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b similarity index 100% rename from src/test/hive/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b rename to shark/src/test/hive/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b diff --git a/src/test/hive/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 b/shark/src/test/hive/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 rename to shark/src/test/hive/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 diff --git a/src/test/hive/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 b/shark/src/test/hive/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 rename to shark/src/test/hive/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 diff --git a/src/test/hive/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca b/shark/src/test/hive/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca similarity index 100% rename from src/test/hive/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca rename to shark/src/test/hive/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca diff --git a/src/test/hive/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e b/shark/src/test/hive/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e similarity index 100% rename from src/test/hive/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e rename to shark/src/test/hive/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e diff --git a/src/test/hive/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 b/shark/src/test/hive/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 rename to shark/src/test/hive/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 diff --git a/src/test/hive/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 b/shark/src/test/hive/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 rename to shark/src/test/hive/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 diff --git a/src/test/hive/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/shark/src/test/hive/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c similarity index 100% rename from src/test/hive/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c rename to shark/src/test/hive/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c diff --git a/src/test/hive/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 b/shark/src/test/hive/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 rename to shark/src/test/hive/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 diff --git a/src/test/hive/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 b/shark/src/test/hive/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 rename to shark/src/test/hive/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 diff --git a/src/test/hive/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc b/shark/src/test/hive/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc similarity index 100% rename from src/test/hive/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc rename to shark/src/test/hive/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc diff --git a/src/test/hive/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e b/shark/src/test/hive/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e similarity index 100% rename from src/test/hive/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e rename to shark/src/test/hive/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e diff --git a/src/test/hive/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 b/shark/src/test/hive/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 rename to shark/src/test/hive/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 diff --git a/src/test/hive/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd b/shark/src/test/hive/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd similarity index 100% rename from src/test/hive/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd rename to shark/src/test/hive/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd diff --git a/src/test/hive/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 b/shark/src/test/hive/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 rename to shark/src/test/hive/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 diff --git a/src/test/hive/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 b/shark/src/test/hive/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 rename to shark/src/test/hive/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 diff --git a/src/test/hive/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 b/shark/src/test/hive/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 rename to shark/src/test/hive/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 diff --git a/src/test/hive/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 b/shark/src/test/hive/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 rename to shark/src/test/hive/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 diff --git a/src/test/hive/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 b/shark/src/test/hive/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 rename to shark/src/test/hive/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 diff --git a/src/test/hive/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 b/shark/src/test/hive/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 similarity index 100% rename from src/test/hive/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 rename to shark/src/test/hive/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 diff --git a/src/test/hive/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 b/shark/src/test/hive/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 similarity index 100% rename from src/test/hive/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 rename to shark/src/test/hive/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 diff --git a/src/test/hive/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 b/shark/src/test/hive/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 similarity index 100% rename from src/test/hive/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 rename to shark/src/test/hive/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 diff --git a/src/test/hive/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e b/shark/src/test/hive/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e similarity index 100% rename from src/test/hive/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e rename to shark/src/test/hive/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e diff --git a/src/test/hive/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 b/shark/src/test/hive/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 similarity index 100% rename from src/test/hive/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 rename to shark/src/test/hive/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 diff --git a/src/test/hive/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 b/shark/src/test/hive/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 similarity index 100% rename from src/test/hive/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 rename to shark/src/test/hive/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 diff --git a/src/test/hive/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 b/shark/src/test/hive/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 similarity index 100% rename from src/test/hive/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 rename to shark/src/test/hive/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 diff --git a/src/test/hive/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 b/shark/src/test/hive/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 similarity index 100% rename from src/test/hive/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 rename to shark/src/test/hive/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 diff --git a/src/test/hive/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c b/shark/src/test/hive/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c similarity index 100% rename from src/test/hive/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c rename to shark/src/test/hive/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c diff --git a/src/test/hive/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d b/shark/src/test/hive/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d similarity index 100% rename from src/test/hive/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d rename to shark/src/test/hive/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d diff --git a/src/test/hive/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 b/shark/src/test/hive/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 rename to shark/src/test/hive/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 diff --git a/src/test/hive/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 b/shark/src/test/hive/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 rename to shark/src/test/hive/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 diff --git a/src/test/hive/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 b/shark/src/test/hive/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 rename to shark/src/test/hive/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 diff --git a/src/test/hive/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 b/shark/src/test/hive/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 rename to shark/src/test/hive/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 diff --git a/src/test/hive/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 b/shark/src/test/hive/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 rename to shark/src/test/hive/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 diff --git a/src/test/hive/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 b/shark/src/test/hive/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 rename to shark/src/test/hive/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 diff --git a/src/test/hive/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 b/shark/src/test/hive/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 rename to shark/src/test/hive/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 diff --git a/src/test/hive/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 b/shark/src/test/hive/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 rename to shark/src/test/hive/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 diff --git a/src/test/hive/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d b/shark/src/test/hive/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d similarity index 100% rename from src/test/hive/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d rename to shark/src/test/hive/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d diff --git a/src/test/hive/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e b/shark/src/test/hive/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e similarity index 100% rename from src/test/hive/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e rename to shark/src/test/hive/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e diff --git a/src/test/hive/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 b/shark/src/test/hive/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 rename to shark/src/test/hive/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 diff --git a/src/test/hive/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 b/shark/src/test/hive/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 rename to shark/src/test/hive/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 diff --git a/src/test/hive/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d b/shark/src/test/hive/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d similarity index 100% rename from src/test/hive/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d rename to shark/src/test/hive/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d diff --git a/src/test/hive/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c b/shark/src/test/hive/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c similarity index 100% rename from src/test/hive/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c rename to shark/src/test/hive/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c diff --git a/src/test/hive/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 b/shark/src/test/hive/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 similarity index 100% rename from src/test/hive/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 rename to shark/src/test/hive/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 diff --git a/src/test/hive/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 b/shark/src/test/hive/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 similarity index 100% rename from src/test/hive/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 rename to shark/src/test/hive/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 diff --git a/src/test/hive/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 b/shark/src/test/hive/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 similarity index 100% rename from src/test/hive/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 rename to shark/src/test/hive/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 diff --git a/src/test/hive/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b b/shark/src/test/hive/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b similarity index 100% rename from src/test/hive/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b rename to shark/src/test/hive/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b diff --git a/src/test/hive/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 b/shark/src/test/hive/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 similarity index 100% rename from src/test/hive/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 rename to shark/src/test/hive/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 diff --git a/src/test/hive/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 b/shark/src/test/hive/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 similarity index 100% rename from src/test/hive/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 rename to shark/src/test/hive/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 diff --git a/src/test/hive/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed b/shark/src/test/hive/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed similarity index 100% rename from src/test/hive/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed rename to shark/src/test/hive/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed diff --git a/src/test/hive/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 b/shark/src/test/hive/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 similarity index 100% rename from src/test/hive/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 rename to shark/src/test/hive/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 diff --git a/src/test/hive/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 b/shark/src/test/hive/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 similarity index 100% rename from src/test/hive/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 rename to shark/src/test/hive/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 diff --git a/src/test/hive/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e b/shark/src/test/hive/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e similarity index 100% rename from src/test/hive/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e rename to shark/src/test/hive/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e diff --git a/src/test/hive/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd b/shark/src/test/hive/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd similarity index 100% rename from src/test/hive/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd rename to shark/src/test/hive/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd diff --git a/src/test/hive/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f b/shark/src/test/hive/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f similarity index 100% rename from src/test/hive/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f rename to shark/src/test/hive/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f diff --git a/src/test/hive/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 b/shark/src/test/hive/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 similarity index 100% rename from src/test/hive/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 rename to shark/src/test/hive/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 diff --git a/src/test/hive/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b b/shark/src/test/hive/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b similarity index 100% rename from src/test/hive/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b rename to shark/src/test/hive/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b diff --git a/src/test/hive/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 b/shark/src/test/hive/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 similarity index 100% rename from src/test/hive/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 rename to shark/src/test/hive/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 diff --git a/src/test/hive/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 b/shark/src/test/hive/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 similarity index 100% rename from src/test/hive/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 rename to shark/src/test/hive/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 diff --git a/src/test/hive/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd b/shark/src/test/hive/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd similarity index 100% rename from src/test/hive/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd rename to shark/src/test/hive/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd diff --git a/src/test/hive/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 b/shark/src/test/hive/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 similarity index 100% rename from src/test/hive/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 rename to shark/src/test/hive/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 diff --git a/src/test/hive/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f b/shark/src/test/hive/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f similarity index 100% rename from src/test/hive/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f rename to shark/src/test/hive/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f diff --git a/src/test/hive/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 b/shark/src/test/hive/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 similarity index 100% rename from src/test/hive/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 rename to shark/src/test/hive/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 diff --git a/src/test/hive/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 b/shark/src/test/hive/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 similarity index 100% rename from src/test/hive/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 rename to shark/src/test/hive/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 diff --git a/src/test/hive/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f b/shark/src/test/hive/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f similarity index 100% rename from src/test/hive/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f rename to shark/src/test/hive/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f diff --git a/src/test/hive/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b b/shark/src/test/hive/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b similarity index 100% rename from src/test/hive/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b rename to shark/src/test/hive/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b diff --git a/src/test/hive/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e b/shark/src/test/hive/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e similarity index 100% rename from src/test/hive/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e rename to shark/src/test/hive/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e diff --git a/src/test/hive/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef b/shark/src/test/hive/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef similarity index 100% rename from src/test/hive/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef rename to shark/src/test/hive/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef diff --git a/src/test/hive/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 b/shark/src/test/hive/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 similarity index 100% rename from src/test/hive/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 rename to shark/src/test/hive/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 diff --git a/src/test/hive/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 b/shark/src/test/hive/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 rename to shark/src/test/hive/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 diff --git a/src/test/hive/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 b/shark/src/test/hive/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 rename to shark/src/test/hive/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 diff --git a/src/test/hive/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f b/shark/src/test/hive/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f similarity index 100% rename from src/test/hive/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f rename to shark/src/test/hive/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f diff --git a/src/test/hive/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b b/shark/src/test/hive/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b similarity index 100% rename from src/test/hive/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b rename to shark/src/test/hive/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b diff --git a/src/test/hive/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 b/shark/src/test/hive/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 rename to shark/src/test/hive/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 diff --git a/src/test/hive/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd b/shark/src/test/hive/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd similarity index 100% rename from src/test/hive/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd rename to shark/src/test/hive/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd diff --git a/src/test/hive/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 b/shark/src/test/hive/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 rename to shark/src/test/hive/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 diff --git a/src/test/hive/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 b/shark/src/test/hive/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 rename to shark/src/test/hive/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 diff --git a/src/test/hive/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf b/shark/src/test/hive/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf similarity index 100% rename from src/test/hive/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf rename to shark/src/test/hive/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf diff --git a/src/test/hive/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b b/shark/src/test/hive/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b similarity index 100% rename from src/test/hive/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b rename to shark/src/test/hive/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b diff --git a/src/test/hive/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d b/shark/src/test/hive/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d similarity index 100% rename from src/test/hive/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d rename to shark/src/test/hive/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d diff --git a/src/test/hive/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f b/shark/src/test/hive/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f similarity index 100% rename from src/test/hive/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f rename to shark/src/test/hive/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f diff --git a/src/test/hive/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd b/shark/src/test/hive/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd similarity index 100% rename from src/test/hive/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd rename to shark/src/test/hive/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd diff --git a/src/test/hive/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f b/shark/src/test/hive/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f similarity index 100% rename from src/test/hive/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f rename to shark/src/test/hive/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f diff --git a/src/test/hive/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 b/shark/src/test/hive/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 rename to shark/src/test/hive/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 diff --git a/src/test/hive/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 b/shark/src/test/hive/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 rename to shark/src/test/hive/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 diff --git a/src/test/hive/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b b/shark/src/test/hive/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b similarity index 100% rename from src/test/hive/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b rename to shark/src/test/hive/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b diff --git a/src/test/hive/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 b/shark/src/test/hive/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 rename to shark/src/test/hive/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 diff --git a/src/test/hive/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 b/shark/src/test/hive/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 similarity index 100% rename from src/test/hive/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 rename to shark/src/test/hive/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 diff --git a/src/test/hive/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd b/shark/src/test/hive/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd similarity index 100% rename from src/test/hive/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd rename to shark/src/test/hive/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd diff --git a/src/test/hive/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 b/shark/src/test/hive/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 rename to shark/src/test/hive/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 diff --git a/src/test/hive/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 b/shark/src/test/hive/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 rename to shark/src/test/hive/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 diff --git a/src/test/hive/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 b/shark/src/test/hive/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 rename to shark/src/test/hive/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 diff --git a/src/test/hive/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 b/shark/src/test/hive/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 rename to shark/src/test/hive/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 diff --git a/src/test/hive/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e b/shark/src/test/hive/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e similarity index 100% rename from src/test/hive/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e rename to shark/src/test/hive/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e diff --git a/src/test/hive/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 b/shark/src/test/hive/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 rename to shark/src/test/hive/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 diff --git a/src/test/hive/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 b/shark/src/test/hive/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 rename to shark/src/test/hive/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 diff --git a/src/test/hive/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 b/shark/src/test/hive/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 rename to shark/src/test/hive/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 diff --git a/src/test/hive/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 b/shark/src/test/hive/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 similarity index 100% rename from src/test/hive/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 rename to shark/src/test/hive/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 diff --git a/src/test/hive/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 b/shark/src/test/hive/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 similarity index 100% rename from src/test/hive/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 rename to shark/src/test/hive/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 diff --git a/src/test/hive/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 b/shark/src/test/hive/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 similarity index 100% rename from src/test/hive/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 rename to shark/src/test/hive/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 diff --git a/src/test/hive/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 b/shark/src/test/hive/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 similarity index 100% rename from src/test/hive/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 rename to shark/src/test/hive/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 diff --git a/src/test/hive/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb b/shark/src/test/hive/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb similarity index 100% rename from src/test/hive/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb rename to shark/src/test/hive/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb diff --git a/src/test/hive/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 b/shark/src/test/hive/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 similarity index 100% rename from src/test/hive/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 rename to shark/src/test/hive/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 diff --git a/src/test/hive/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 b/shark/src/test/hive/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 similarity index 100% rename from src/test/hive/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 rename to shark/src/test/hive/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 diff --git a/src/test/hive/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b b/shark/src/test/hive/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b similarity index 100% rename from src/test/hive/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b rename to shark/src/test/hive/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b diff --git a/src/test/hive/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 b/shark/src/test/hive/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 similarity index 100% rename from src/test/hive/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 rename to shark/src/test/hive/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 diff --git a/src/test/hive/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 b/shark/src/test/hive/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 similarity index 100% rename from src/test/hive/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 rename to shark/src/test/hive/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 diff --git a/src/test/hive/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d b/shark/src/test/hive/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d similarity index 100% rename from src/test/hive/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d rename to shark/src/test/hive/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d diff --git a/src/test/hive/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 b/shark/src/test/hive/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 similarity index 100% rename from src/test/hive/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 rename to shark/src/test/hive/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 diff --git a/src/test/hive/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f b/shark/src/test/hive/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f similarity index 100% rename from src/test/hive/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f rename to shark/src/test/hive/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f diff --git a/src/test/hive/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb b/shark/src/test/hive/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb similarity index 100% rename from src/test/hive/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb rename to shark/src/test/hive/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb diff --git a/src/test/hive/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 b/shark/src/test/hive/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 similarity index 100% rename from src/test/hive/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 rename to shark/src/test/hive/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 diff --git a/src/test/hive/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d b/shark/src/test/hive/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d similarity index 100% rename from src/test/hive/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d rename to shark/src/test/hive/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d diff --git a/src/test/hive/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d b/shark/src/test/hive/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d similarity index 100% rename from src/test/hive/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d rename to shark/src/test/hive/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d diff --git a/src/test/hive/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a b/shark/src/test/hive/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a similarity index 100% rename from src/test/hive/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a rename to shark/src/test/hive/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a diff --git a/src/test/hive/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 b/shark/src/test/hive/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 similarity index 100% rename from src/test/hive/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 rename to shark/src/test/hive/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 diff --git a/src/test/hive/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f b/shark/src/test/hive/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f similarity index 100% rename from src/test/hive/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f rename to shark/src/test/hive/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f diff --git a/src/test/hive/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb b/shark/src/test/hive/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb similarity index 100% rename from src/test/hive/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb rename to shark/src/test/hive/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb diff --git a/src/test/hive/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 b/shark/src/test/hive/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 similarity index 100% rename from src/test/hive/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 rename to shark/src/test/hive/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 diff --git a/src/test/hive/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c b/shark/src/test/hive/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c similarity index 100% rename from src/test/hive/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c rename to shark/src/test/hive/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c diff --git a/src/test/hive/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 b/shark/src/test/hive/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 similarity index 100% rename from src/test/hive/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 rename to shark/src/test/hive/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 diff --git a/src/test/hive/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c b/shark/src/test/hive/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c similarity index 100% rename from src/test/hive/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c rename to shark/src/test/hive/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c diff --git a/src/test/hive/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 b/shark/src/test/hive/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 rename to shark/src/test/hive/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 diff --git a/src/test/hive/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 b/shark/src/test/hive/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 rename to shark/src/test/hive/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 diff --git a/src/test/hive/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c b/shark/src/test/hive/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c rename to shark/src/test/hive/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c diff --git a/src/test/hive/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 b/shark/src/test/hive/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 rename to shark/src/test/hive/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 diff --git a/src/test/hive/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 b/shark/src/test/hive/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 rename to shark/src/test/hive/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 diff --git a/src/test/hive/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea b/shark/src/test/hive/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea rename to shark/src/test/hive/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea diff --git a/src/test/hive/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 b/shark/src/test/hive/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 rename to shark/src/test/hive/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 diff --git a/src/test/hive/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 b/shark/src/test/hive/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 rename to shark/src/test/hive/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 diff --git a/src/test/hive/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 b/shark/src/test/hive/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 rename to shark/src/test/hive/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 diff --git a/src/test/hive/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 b/shark/src/test/hive/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 rename to shark/src/test/hive/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 diff --git a/src/test/hive/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 b/shark/src/test/hive/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 rename to shark/src/test/hive/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 diff --git a/src/test/hive/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 b/shark/src/test/hive/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 rename to shark/src/test/hive/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 diff --git a/src/test/hive/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 b/shark/src/test/hive/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 rename to shark/src/test/hive/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 diff --git a/src/test/hive/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 b/shark/src/test/hive/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 rename to shark/src/test/hive/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 diff --git a/src/test/hive/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 b/shark/src/test/hive/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 rename to shark/src/test/hive/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 diff --git a/src/test/hive/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f b/shark/src/test/hive/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f rename to shark/src/test/hive/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f diff --git a/src/test/hive/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca b/shark/src/test/hive/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca rename to shark/src/test/hive/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca diff --git a/src/test/hive/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 b/shark/src/test/hive/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 rename to shark/src/test/hive/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 diff --git a/src/test/hive/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 b/shark/src/test/hive/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 rename to shark/src/test/hive/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 diff --git a/src/test/hive/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 b/shark/src/test/hive/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 rename to shark/src/test/hive/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 diff --git a/src/test/hive/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f b/shark/src/test/hive/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f rename to shark/src/test/hive/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f diff --git a/src/test/hive/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef b/shark/src/test/hive/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef rename to shark/src/test/hive/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef diff --git a/src/test/hive/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 b/shark/src/test/hive/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 rename to shark/src/test/hive/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 diff --git a/src/test/hive/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 b/shark/src/test/hive/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 rename to shark/src/test/hive/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 diff --git a/src/test/hive/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d b/shark/src/test/hive/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d rename to shark/src/test/hive/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d diff --git a/src/test/hive/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b b/shark/src/test/hive/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b rename to shark/src/test/hive/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b diff --git a/src/test/hive/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 b/shark/src/test/hive/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 rename to shark/src/test/hive/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 diff --git a/src/test/hive/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c b/shark/src/test/hive/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c rename to shark/src/test/hive/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c diff --git a/src/test/hive/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a b/shark/src/test/hive/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a rename to shark/src/test/hive/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a diff --git a/src/test/hive/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 b/shark/src/test/hive/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 rename to shark/src/test/hive/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 diff --git a/src/test/hive/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 b/shark/src/test/hive/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 rename to shark/src/test/hive/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 diff --git a/src/test/hive/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa b/shark/src/test/hive/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa rename to shark/src/test/hive/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa diff --git a/src/test/hive/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 b/shark/src/test/hive/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 rename to shark/src/test/hive/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 diff --git a/src/test/hive/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad b/shark/src/test/hive/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad rename to shark/src/test/hive/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad diff --git a/src/test/hive/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd b/shark/src/test/hive/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd rename to shark/src/test/hive/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd diff --git a/src/test/hive/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 b/shark/src/test/hive/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 rename to shark/src/test/hive/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 diff --git a/src/test/hive/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 b/shark/src/test/hive/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 rename to shark/src/test/hive/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 diff --git a/src/test/hive/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca b/shark/src/test/hive/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca rename to shark/src/test/hive/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca diff --git a/src/test/hive/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 b/shark/src/test/hive/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 rename to shark/src/test/hive/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 diff --git a/src/test/hive/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c b/shark/src/test/hive/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c rename to shark/src/test/hive/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c diff --git a/src/test/hive/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 b/shark/src/test/hive/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 rename to shark/src/test/hive/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 diff --git a/src/test/hive/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 b/shark/src/test/hive/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 rename to shark/src/test/hive/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 diff --git a/src/test/hive/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 b/shark/src/test/hive/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 rename to shark/src/test/hive/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 diff --git a/src/test/hive/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 b/shark/src/test/hive/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 rename to shark/src/test/hive/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 diff --git a/src/test/hive/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca b/shark/src/test/hive/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca rename to shark/src/test/hive/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca diff --git a/src/test/hive/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 b/shark/src/test/hive/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 rename to shark/src/test/hive/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 diff --git a/src/test/hive/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 b/shark/src/test/hive/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 rename to shark/src/test/hive/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 diff --git a/src/test/hive/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca b/shark/src/test/hive/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca rename to shark/src/test/hive/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca diff --git a/src/test/hive/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e b/shark/src/test/hive/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e rename to shark/src/test/hive/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e diff --git a/src/test/hive/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 b/shark/src/test/hive/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 rename to shark/src/test/hive/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 diff --git a/src/test/hive/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 b/shark/src/test/hive/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 rename to shark/src/test/hive/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 diff --git a/src/test/hive/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 b/shark/src/test/hive/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 rename to shark/src/test/hive/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 diff --git a/src/test/hive/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/shark/src/test/hive/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c rename to shark/src/test/hive/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c diff --git a/src/test/hive/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b b/shark/src/test/hive/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b rename to shark/src/test/hive/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b diff --git a/src/test/hive/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 b/shark/src/test/hive/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 rename to shark/src/test/hive/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 diff --git a/src/test/hive/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc b/shark/src/test/hive/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc rename to shark/src/test/hive/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc diff --git a/src/test/hive/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e b/shark/src/test/hive/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e rename to shark/src/test/hive/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e diff --git a/src/test/hive/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 b/shark/src/test/hive/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 rename to shark/src/test/hive/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 diff --git a/src/test/hive/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd b/shark/src/test/hive/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd rename to shark/src/test/hive/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd diff --git a/src/test/hive/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 b/shark/src/test/hive/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 rename to shark/src/test/hive/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 diff --git a/src/test/hive/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 b/shark/src/test/hive/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 rename to shark/src/test/hive/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 diff --git a/src/test/hive/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 b/shark/src/test/hive/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 rename to shark/src/test/hive/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 diff --git a/src/test/hive/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 b/shark/src/test/hive/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 rename to shark/src/test/hive/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 diff --git a/src/test/hive/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 b/shark/src/test/hive/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 rename to shark/src/test/hive/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 diff --git a/src/test/hive/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 b/shark/src/test/hive/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 similarity index 100% rename from src/test/hive/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 rename to shark/src/test/hive/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 diff --git a/src/test/hive/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e b/shark/src/test/hive/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e similarity index 100% rename from src/test/hive/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e rename to shark/src/test/hive/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e diff --git a/src/test/hive/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 b/shark/src/test/hive/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 similarity index 100% rename from src/test/hive/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 rename to shark/src/test/hive/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 diff --git a/src/test/hive/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a b/shark/src/test/hive/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a similarity index 100% rename from src/test/hive/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a rename to shark/src/test/hive/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a diff --git a/src/test/hive/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad b/shark/src/test/hive/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad similarity index 100% rename from src/test/hive/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad rename to shark/src/test/hive/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad diff --git a/src/test/hive/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b b/shark/src/test/hive/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b similarity index 100% rename from src/test/hive/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b rename to shark/src/test/hive/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b diff --git a/src/test/hive/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 b/shark/src/test/hive/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 similarity index 100% rename from src/test/hive/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 rename to shark/src/test/hive/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 diff --git a/src/test/hive/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 b/shark/src/test/hive/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 similarity index 100% rename from src/test/hive/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 rename to shark/src/test/hive/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 diff --git a/src/test/hive/golden/hash-0-a658b129316d666d4b01c1581eed1c1f b/shark/src/test/hive/golden/hash-0-a658b129316d666d4b01c1581eed1c1f similarity index 100% rename from src/test/hive/golden/hash-0-a658b129316d666d4b01c1581eed1c1f rename to shark/src/test/hive/golden/hash-0-a658b129316d666d4b01c1581eed1c1f diff --git a/src/test/hive/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf b/shark/src/test/hive/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf similarity index 100% rename from src/test/hive/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf rename to shark/src/test/hive/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf diff --git a/src/test/hive/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 b/shark/src/test/hive/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 similarity index 100% rename from src/test/hive/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 rename to shark/src/test/hive/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 diff --git a/src/test/hive/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 b/shark/src/test/hive/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 similarity index 100% rename from src/test/hive/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 rename to shark/src/test/hive/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 diff --git a/src/test/hive/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a b/shark/src/test/hive/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a similarity index 100% rename from src/test/hive/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a rename to shark/src/test/hive/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a diff --git a/src/test/hive/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 b/shark/src/test/hive/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 similarity index 100% rename from src/test/hive/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 rename to shark/src/test/hive/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 diff --git a/src/test/hive/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 b/shark/src/test/hive/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 similarity index 100% rename from src/test/hive/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 rename to shark/src/test/hive/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 diff --git a/src/test/hive/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 b/shark/src/test/hive/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 similarity index 100% rename from src/test/hive/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 rename to shark/src/test/hive/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 diff --git a/src/test/hive/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 b/shark/src/test/hive/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 similarity index 100% rename from src/test/hive/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 rename to shark/src/test/hive/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 diff --git a/src/test/hive/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 b/shark/src/test/hive/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 similarity index 100% rename from src/test/hive/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 rename to shark/src/test/hive/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 diff --git a/src/test/hive/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 b/shark/src/test/hive/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 similarity index 100% rename from src/test/hive/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 rename to shark/src/test/hive/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 diff --git a/src/test/hive/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 b/shark/src/test/hive/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 similarity index 100% rename from src/test/hive/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 rename to shark/src/test/hive/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 diff --git a/src/test/hive/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 b/shark/src/test/hive/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 similarity index 100% rename from src/test/hive/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 rename to shark/src/test/hive/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 diff --git a/src/test/hive/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 b/shark/src/test/hive/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 similarity index 100% rename from src/test/hive/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 rename to shark/src/test/hive/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 diff --git a/src/test/hive/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d b/shark/src/test/hive/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d similarity index 100% rename from src/test/hive/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d rename to shark/src/test/hive/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d diff --git a/src/test/hive/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad b/shark/src/test/hive/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad similarity index 100% rename from src/test/hive/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad rename to shark/src/test/hive/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad diff --git a/src/test/hive/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 b/shark/src/test/hive/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 similarity index 100% rename from src/test/hive/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 rename to shark/src/test/hive/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 diff --git a/src/test/hive/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f b/shark/src/test/hive/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f similarity index 100% rename from src/test/hive/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f rename to shark/src/test/hive/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f diff --git a/src/test/hive/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 b/shark/src/test/hive/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 similarity index 100% rename from src/test/hive/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 rename to shark/src/test/hive/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 diff --git a/src/test/hive/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 b/shark/src/test/hive/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 similarity index 100% rename from src/test/hive/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 rename to shark/src/test/hive/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 diff --git a/src/test/hive/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e b/shark/src/test/hive/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e similarity index 100% rename from src/test/hive/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e rename to shark/src/test/hive/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e diff --git a/src/test/hive/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 b/shark/src/test/hive/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 similarity index 100% rename from src/test/hive/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 rename to shark/src/test/hive/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 diff --git a/src/test/hive/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 b/shark/src/test/hive/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 similarity index 100% rename from src/test/hive/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 rename to shark/src/test/hive/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 diff --git a/src/test/hive/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf b/shark/src/test/hive/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf similarity index 100% rename from src/test/hive/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf rename to shark/src/test/hive/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf diff --git a/src/test/hive/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca b/shark/src/test/hive/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca similarity index 100% rename from src/test/hive/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca rename to shark/src/test/hive/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca diff --git a/src/test/hive/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 b/shark/src/test/hive/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 similarity index 100% rename from src/test/hive/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 rename to shark/src/test/hive/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 diff --git a/src/test/hive/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 b/shark/src/test/hive/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 similarity index 100% rename from src/test/hive/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 rename to shark/src/test/hive/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 diff --git a/src/test/hive/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe b/shark/src/test/hive/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe similarity index 100% rename from src/test/hive/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe rename to shark/src/test/hive/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe diff --git a/src/test/hive/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f b/shark/src/test/hive/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f similarity index 100% rename from src/test/hive/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f rename to shark/src/test/hive/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f diff --git a/src/test/hive/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 b/shark/src/test/hive/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 similarity index 100% rename from src/test/hive/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 rename to shark/src/test/hive/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 diff --git a/src/test/hive/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 b/shark/src/test/hive/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 similarity index 100% rename from src/test/hive/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 rename to shark/src/test/hive/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 diff --git a/src/test/hive/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 b/shark/src/test/hive/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 similarity index 100% rename from src/test/hive/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 rename to shark/src/test/hive/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 diff --git a/src/test/hive/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 b/shark/src/test/hive/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 similarity index 100% rename from src/test/hive/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 rename to shark/src/test/hive/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 diff --git a/src/test/hive/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 b/shark/src/test/hive/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 similarity index 100% rename from src/test/hive/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 rename to shark/src/test/hive/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 diff --git a/src/test/hive/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/shark/src/test/hive/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 similarity index 100% rename from src/test/hive/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 rename to shark/src/test/hive/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 diff --git a/src/test/hive/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/shark/src/test/hive/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 similarity index 100% rename from src/test/hive/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 rename to shark/src/test/hive/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 diff --git a/src/test/hive/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 b/shark/src/test/hive/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 similarity index 100% rename from src/test/hive/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 rename to shark/src/test/hive/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 diff --git a/src/test/hive/golden/index_creation-37-9334418431eca405f13206bd8db42a1b b/shark/src/test/hive/golden/index_creation-37-9334418431eca405f13206bd8db42a1b similarity index 100% rename from src/test/hive/golden/index_creation-37-9334418431eca405f13206bd8db42a1b rename to shark/src/test/hive/golden/index_creation-37-9334418431eca405f13206bd8db42a1b diff --git a/src/test/hive/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 b/shark/src/test/hive/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 similarity index 100% rename from src/test/hive/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 rename to shark/src/test/hive/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 diff --git a/src/test/hive/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d b/shark/src/test/hive/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d similarity index 100% rename from src/test/hive/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d rename to shark/src/test/hive/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d diff --git a/src/test/hive/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/shark/src/test/hive/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 similarity index 100% rename from src/test/hive/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 rename to shark/src/test/hive/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 diff --git a/src/test/hive/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/shark/src/test/hive/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 similarity index 100% rename from src/test/hive/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 rename to shark/src/test/hive/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 diff --git a/src/test/hive/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 b/shark/src/test/hive/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 similarity index 100% rename from src/test/hive/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 rename to shark/src/test/hive/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 diff --git a/src/test/hive/golden/index_creation-7-9334418431eca405f13206bd8db42a1b b/shark/src/test/hive/golden/index_creation-7-9334418431eca405f13206bd8db42a1b similarity index 100% rename from src/test/hive/golden/index_creation-7-9334418431eca405f13206bd8db42a1b rename to shark/src/test/hive/golden/index_creation-7-9334418431eca405f13206bd8db42a1b diff --git a/src/test/hive/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 b/shark/src/test/hive/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 similarity index 100% rename from src/test/hive/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 rename to shark/src/test/hive/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 diff --git a/src/test/hive/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c b/shark/src/test/hive/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c similarity index 100% rename from src/test/hive/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c rename to shark/src/test/hive/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c diff --git a/src/test/hive/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 b/shark/src/test/hive/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 similarity index 100% rename from src/test/hive/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 rename to shark/src/test/hive/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 diff --git a/src/test/hive/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 b/shark/src/test/hive/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 similarity index 100% rename from src/test/hive/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 rename to shark/src/test/hive/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 diff --git a/src/test/hive/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b b/shark/src/test/hive/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b similarity index 100% rename from src/test/hive/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b rename to shark/src/test/hive/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b diff --git a/src/test/hive/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 b/shark/src/test/hive/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 similarity index 100% rename from src/test/hive/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 rename to shark/src/test/hive/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 diff --git a/src/test/hive/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 b/shark/src/test/hive/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 similarity index 100% rename from src/test/hive/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 rename to shark/src/test/hive/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 diff --git a/src/test/hive/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c b/shark/src/test/hive/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c similarity index 100% rename from src/test/hive/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c rename to shark/src/test/hive/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c diff --git a/src/test/hive/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 b/shark/src/test/hive/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 similarity index 100% rename from src/test/hive/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 rename to shark/src/test/hive/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 diff --git a/src/test/hive/golden/index_serde-4-d7547751c37375a9238043bbe250e716 b/shark/src/test/hive/golden/index_serde-4-d7547751c37375a9238043bbe250e716 similarity index 100% rename from src/test/hive/golden/index_serde-4-d7547751c37375a9238043bbe250e716 rename to shark/src/test/hive/golden/index_serde-4-d7547751c37375a9238043bbe250e716 diff --git a/src/test/hive/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 b/shark/src/test/hive/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 similarity index 100% rename from src/test/hive/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 rename to shark/src/test/hive/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 diff --git a/src/test/hive/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae b/shark/src/test/hive/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae similarity index 100% rename from src/test/hive/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae rename to shark/src/test/hive/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae diff --git a/src/test/hive/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 b/shark/src/test/hive/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 similarity index 100% rename from src/test/hive/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 rename to shark/src/test/hive/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 diff --git a/src/test/hive/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 b/shark/src/test/hive/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 similarity index 100% rename from src/test/hive/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 rename to shark/src/test/hive/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 diff --git a/src/test/hive/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 b/shark/src/test/hive/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 similarity index 100% rename from src/test/hive/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 rename to shark/src/test/hive/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 diff --git a/src/test/hive/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e b/shark/src/test/hive/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e similarity index 100% rename from src/test/hive/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e rename to shark/src/test/hive/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e diff --git a/src/test/hive/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 b/shark/src/test/hive/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 similarity index 100% rename from src/test/hive/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 rename to shark/src/test/hive/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 diff --git a/src/test/hive/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd b/shark/src/test/hive/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd similarity index 100% rename from src/test/hive/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd rename to shark/src/test/hive/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd diff --git a/src/test/hive/golden/innerjoin-0-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/innerjoin-0-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/innerjoin-0-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/innerjoin-0-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e b/shark/src/test/hive/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e similarity index 100% rename from src/test/hive/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e rename to shark/src/test/hive/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e diff --git a/src/test/hive/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 b/shark/src/test/hive/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 similarity index 100% rename from src/test/hive/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 rename to shark/src/test/hive/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 diff --git a/src/test/hive/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 b/shark/src/test/hive/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 similarity index 100% rename from src/test/hive/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 rename to shark/src/test/hive/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 diff --git a/src/test/hive/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 b/shark/src/test/hive/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 similarity index 100% rename from src/test/hive/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 rename to shark/src/test/hive/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 diff --git a/src/test/hive/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a b/shark/src/test/hive/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a similarity index 100% rename from src/test/hive/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a rename to shark/src/test/hive/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a diff --git a/src/test/hive/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce b/shark/src/test/hive/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce similarity index 100% rename from src/test/hive/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce rename to shark/src/test/hive/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce diff --git a/src/test/hive/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 b/shark/src/test/hive/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 similarity index 100% rename from src/test/hive/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 rename to shark/src/test/hive/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 diff --git a/src/test/hive/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 b/shark/src/test/hive/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 similarity index 100% rename from src/test/hive/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 rename to shark/src/test/hive/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 diff --git a/src/test/hive/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b b/shark/src/test/hive/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b similarity index 100% rename from src/test/hive/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b rename to shark/src/test/hive/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b diff --git a/src/test/hive/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 b/shark/src/test/hive/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 similarity index 100% rename from src/test/hive/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 rename to shark/src/test/hive/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 diff --git a/src/test/hive/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 b/shark/src/test/hive/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 similarity index 100% rename from src/test/hive/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 rename to shark/src/test/hive/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 diff --git a/src/test/hive/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 b/shark/src/test/hive/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 similarity index 100% rename from src/test/hive/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 rename to shark/src/test/hive/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 diff --git a/src/test/hive/golden/input-1-6558e385bb08991302a72076dd7b7ff5 b/shark/src/test/hive/golden/input-1-6558e385bb08991302a72076dd7b7ff5 similarity index 100% rename from src/test/hive/golden/input-1-6558e385bb08991302a72076dd7b7ff5 rename to shark/src/test/hive/golden/input-1-6558e385bb08991302a72076dd7b7ff5 diff --git a/src/test/hive/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 b/shark/src/test/hive/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 similarity index 100% rename from src/test/hive/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 rename to shark/src/test/hive/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 diff --git a/src/test/hive/golden/input0-1-efefd4364cd2790447fb0f908e87501f b/shark/src/test/hive/golden/input0-1-efefd4364cd2790447fb0f908e87501f similarity index 100% rename from src/test/hive/golden/input0-1-efefd4364cd2790447fb0f908e87501f rename to shark/src/test/hive/golden/input0-1-efefd4364cd2790447fb0f908e87501f diff --git a/src/test/hive/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 b/shark/src/test/hive/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 similarity index 100% rename from src/test/hive/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 rename to shark/src/test/hive/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 diff --git a/src/test/hive/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 b/shark/src/test/hive/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 similarity index 100% rename from src/test/hive/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 rename to shark/src/test/hive/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 diff --git a/src/test/hive/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 b/shark/src/test/hive/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 similarity index 100% rename from src/test/hive/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 rename to shark/src/test/hive/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 diff --git a/src/test/hive/golden/input10-0-acdd72fda64c5463ce29f31020012d42 b/shark/src/test/hive/golden/input10-0-acdd72fda64c5463ce29f31020012d42 similarity index 100% rename from src/test/hive/golden/input10-0-acdd72fda64c5463ce29f31020012d42 rename to shark/src/test/hive/golden/input10-0-acdd72fda64c5463ce29f31020012d42 diff --git a/src/test/hive/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 b/shark/src/test/hive/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 similarity index 100% rename from src/test/hive/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 rename to shark/src/test/hive/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 diff --git a/src/test/hive/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 b/shark/src/test/hive/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 similarity index 100% rename from src/test/hive/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 rename to shark/src/test/hive/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 diff --git a/src/test/hive/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb b/shark/src/test/hive/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb similarity index 100% rename from src/test/hive/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb rename to shark/src/test/hive/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb diff --git a/src/test/hive/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 b/shark/src/test/hive/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 similarity index 100% rename from src/test/hive/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 rename to shark/src/test/hive/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 diff --git a/src/test/hive/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 b/shark/src/test/hive/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 similarity index 100% rename from src/test/hive/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 rename to shark/src/test/hive/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 diff --git a/src/test/hive/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 b/shark/src/test/hive/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 similarity index 100% rename from src/test/hive/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 rename to shark/src/test/hive/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 diff --git a/src/test/hive/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 b/shark/src/test/hive/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 similarity index 100% rename from src/test/hive/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 rename to shark/src/test/hive/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 diff --git a/src/test/hive/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e b/shark/src/test/hive/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e similarity index 100% rename from src/test/hive/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e rename to shark/src/test/hive/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e diff --git a/src/test/hive/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 b/shark/src/test/hive/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 similarity index 100% rename from src/test/hive/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 rename to shark/src/test/hive/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 diff --git a/src/test/hive/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c b/shark/src/test/hive/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c similarity index 100% rename from src/test/hive/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c rename to shark/src/test/hive/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c diff --git a/src/test/hive/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 b/shark/src/test/hive/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 similarity index 100% rename from src/test/hive/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 rename to shark/src/test/hive/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 diff --git a/src/test/hive/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 b/shark/src/test/hive/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 similarity index 100% rename from src/test/hive/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 rename to shark/src/test/hive/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 diff --git a/src/test/hive/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 b/shark/src/test/hive/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 similarity index 100% rename from src/test/hive/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 rename to shark/src/test/hive/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 diff --git a/src/test/hive/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf b/shark/src/test/hive/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf similarity index 100% rename from src/test/hive/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf rename to shark/src/test/hive/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf diff --git a/src/test/hive/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 b/shark/src/test/hive/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 similarity index 100% rename from src/test/hive/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 rename to shark/src/test/hive/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 diff --git a/src/test/hive/golden/input12-7-16da1c1293626b943343443368679e9c b/shark/src/test/hive/golden/input12-7-16da1c1293626b943343443368679e9c similarity index 100% rename from src/test/hive/golden/input12-7-16da1c1293626b943343443368679e9c rename to shark/src/test/hive/golden/input12-7-16da1c1293626b943343443368679e9c diff --git a/src/test/hive/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd b/shark/src/test/hive/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd similarity index 100% rename from src/test/hive/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd rename to shark/src/test/hive/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd diff --git a/src/test/hive/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f b/shark/src/test/hive/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f similarity index 100% rename from src/test/hive/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f rename to shark/src/test/hive/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f diff --git a/src/test/hive/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 b/shark/src/test/hive/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 similarity index 100% rename from src/test/hive/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 rename to shark/src/test/hive/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 diff --git a/src/test/hive/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 b/shark/src/test/hive/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 similarity index 100% rename from src/test/hive/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 rename to shark/src/test/hive/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 diff --git a/src/test/hive/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 b/shark/src/test/hive/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 similarity index 100% rename from src/test/hive/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 rename to shark/src/test/hive/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 diff --git a/src/test/hive/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf b/shark/src/test/hive/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf similarity index 100% rename from src/test/hive/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf rename to shark/src/test/hive/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf diff --git a/src/test/hive/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 b/shark/src/test/hive/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 similarity index 100% rename from src/test/hive/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 rename to shark/src/test/hive/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 diff --git a/src/test/hive/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c b/shark/src/test/hive/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c similarity index 100% rename from src/test/hive/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c rename to shark/src/test/hive/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c diff --git a/src/test/hive/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd b/shark/src/test/hive/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd similarity index 100% rename from src/test/hive/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd rename to shark/src/test/hive/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd diff --git a/src/test/hive/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c b/shark/src/test/hive/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c similarity index 100% rename from src/test/hive/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c rename to shark/src/test/hive/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c diff --git a/src/test/hive/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a b/shark/src/test/hive/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a similarity index 100% rename from src/test/hive/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a rename to shark/src/test/hive/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a diff --git a/src/test/hive/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 b/shark/src/test/hive/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 similarity index 100% rename from src/test/hive/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 rename to shark/src/test/hive/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 diff --git a/src/test/hive/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf b/shark/src/test/hive/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf similarity index 100% rename from src/test/hive/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf rename to shark/src/test/hive/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf diff --git a/src/test/hive/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 b/shark/src/test/hive/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 similarity index 100% rename from src/test/hive/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 rename to shark/src/test/hive/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 diff --git a/src/test/hive/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input16_cc-0-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/input16_cc-0-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/input16_cc-0-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/input16_cc-0-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 b/shark/src/test/hive/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 similarity index 100% rename from src/test/hive/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 rename to shark/src/test/hive/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 diff --git a/src/test/hive/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 b/shark/src/test/hive/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 similarity index 100% rename from src/test/hive/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 rename to shark/src/test/hive/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 diff --git a/src/test/hive/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 b/shark/src/test/hive/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 similarity index 100% rename from src/test/hive/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 rename to shark/src/test/hive/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 diff --git a/src/test/hive/golden/input17-2-21166e268096f6ec67f4f57ec333e901 b/shark/src/test/hive/golden/input17-2-21166e268096f6ec67f4f57ec333e901 similarity index 100% rename from src/test/hive/golden/input17-2-21166e268096f6ec67f4f57ec333e901 rename to shark/src/test/hive/golden/input17-2-21166e268096f6ec67f4f57ec333e901 diff --git a/src/test/hive/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a b/shark/src/test/hive/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a similarity index 100% rename from src/test/hive/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a rename to shark/src/test/hive/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a diff --git a/src/test/hive/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc b/shark/src/test/hive/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc similarity index 100% rename from src/test/hive/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc rename to shark/src/test/hive/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc diff --git a/src/test/hive/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 b/shark/src/test/hive/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 similarity index 100% rename from src/test/hive/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 rename to shark/src/test/hive/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 diff --git a/src/test/hive/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 b/shark/src/test/hive/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 similarity index 100% rename from src/test/hive/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 rename to shark/src/test/hive/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 diff --git a/src/test/hive/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 b/shark/src/test/hive/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 similarity index 100% rename from src/test/hive/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 rename to shark/src/test/hive/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 diff --git a/src/test/hive/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e b/shark/src/test/hive/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e similarity index 100% rename from src/test/hive/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e rename to shark/src/test/hive/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e diff --git a/src/test/hive/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 b/shark/src/test/hive/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 similarity index 100% rename from src/test/hive/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 rename to shark/src/test/hive/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 diff --git a/src/test/hive/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 b/shark/src/test/hive/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 similarity index 100% rename from src/test/hive/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 rename to shark/src/test/hive/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 diff --git a/src/test/hive/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c b/shark/src/test/hive/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c similarity index 100% rename from src/test/hive/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c rename to shark/src/test/hive/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c diff --git a/src/test/hive/golden/input2-1-e0efeda558cd0194f4764a5735147b16 b/shark/src/test/hive/golden/input2-1-e0efeda558cd0194f4764a5735147b16 similarity index 100% rename from src/test/hive/golden/input2-1-e0efeda558cd0194f4764a5735147b16 rename to shark/src/test/hive/golden/input2-1-e0efeda558cd0194f4764a5735147b16 diff --git a/src/test/hive/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd b/shark/src/test/hive/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd similarity index 100% rename from src/test/hive/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd rename to shark/src/test/hive/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd diff --git a/src/test/hive/golden/input2-3-45b9f786f64afa5e039d1856fe926649 b/shark/src/test/hive/golden/input2-3-45b9f786f64afa5e039d1856fe926649 similarity index 100% rename from src/test/hive/golden/input2-3-45b9f786f64afa5e039d1856fe926649 rename to shark/src/test/hive/golden/input2-3-45b9f786f64afa5e039d1856fe926649 diff --git a/src/test/hive/golden/input2-4-235f92683416fab031e6e7490487b15b b/shark/src/test/hive/golden/input2-4-235f92683416fab031e6e7490487b15b similarity index 100% rename from src/test/hive/golden/input2-4-235f92683416fab031e6e7490487b15b rename to shark/src/test/hive/golden/input2-4-235f92683416fab031e6e7490487b15b diff --git a/src/test/hive/golden/input2-5-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/input2-5-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/input2-5-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/input2-5-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/input2-6-4a6b9674c126337f71834f11613f996d b/shark/src/test/hive/golden/input2-6-4a6b9674c126337f71834f11613f996d similarity index 100% rename from src/test/hive/golden/input2-6-4a6b9674c126337f71834f11613f996d rename to shark/src/test/hive/golden/input2-6-4a6b9674c126337f71834f11613f996d diff --git a/src/test/hive/golden/input2-7-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/input2-7-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/input2-7-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/input2-7-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 b/shark/src/test/hive/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 similarity index 100% rename from src/test/hive/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 rename to shark/src/test/hive/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 diff --git a/src/test/hive/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 b/shark/src/test/hive/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 similarity index 100% rename from src/test/hive/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 rename to shark/src/test/hive/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 diff --git a/src/test/hive/golden/input21-0-7f40c525398953892ffd6283f54ba427 b/shark/src/test/hive/golden/input21-0-7f40c525398953892ffd6283f54ba427 similarity index 100% rename from src/test/hive/golden/input21-0-7f40c525398953892ffd6283f54ba427 rename to shark/src/test/hive/golden/input21-0-7f40c525398953892ffd6283f54ba427 diff --git a/src/test/hive/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 b/shark/src/test/hive/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 similarity index 100% rename from src/test/hive/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 rename to shark/src/test/hive/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 diff --git a/src/test/hive/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 b/shark/src/test/hive/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 similarity index 100% rename from src/test/hive/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 rename to shark/src/test/hive/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 diff --git a/src/test/hive/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe b/shark/src/test/hive/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe similarity index 100% rename from src/test/hive/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe rename to shark/src/test/hive/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe diff --git a/src/test/hive/golden/input22-0-b9cd4041ca510639838a97376747b498 b/shark/src/test/hive/golden/input22-0-b9cd4041ca510639838a97376747b498 similarity index 100% rename from src/test/hive/golden/input22-0-b9cd4041ca510639838a97376747b498 rename to shark/src/test/hive/golden/input22-0-b9cd4041ca510639838a97376747b498 diff --git a/src/test/hive/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 b/shark/src/test/hive/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 similarity index 100% rename from src/test/hive/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 rename to shark/src/test/hive/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 diff --git a/src/test/hive/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 b/shark/src/test/hive/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 similarity index 100% rename from src/test/hive/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 rename to shark/src/test/hive/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 diff --git a/src/test/hive/golden/input22-3-8285c1934441b12f6c016e13cb410e79 b/shark/src/test/hive/golden/input22-3-8285c1934441b12f6c016e13cb410e79 similarity index 100% rename from src/test/hive/golden/input22-3-8285c1934441b12f6c016e13cb410e79 rename to shark/src/test/hive/golden/input22-3-8285c1934441b12f6c016e13cb410e79 diff --git a/src/test/hive/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf b/shark/src/test/hive/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf similarity index 100% rename from src/test/hive/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf rename to shark/src/test/hive/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf diff --git a/src/test/hive/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb b/shark/src/test/hive/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb similarity index 100% rename from src/test/hive/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb rename to shark/src/test/hive/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb diff --git a/src/test/hive/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 b/shark/src/test/hive/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 similarity index 100% rename from src/test/hive/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 rename to shark/src/test/hive/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 diff --git a/src/test/hive/golden/input24-1-f57485de80ad64f9931bf92514fa1 b/shark/src/test/hive/golden/input24-1-f57485de80ad64f9931bf92514fa1 similarity index 100% rename from src/test/hive/golden/input24-1-f57485de80ad64f9931bf92514fa1 rename to shark/src/test/hive/golden/input24-1-f57485de80ad64f9931bf92514fa1 diff --git a/src/test/hive/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 b/shark/src/test/hive/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 similarity index 100% rename from src/test/hive/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 rename to shark/src/test/hive/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 diff --git a/src/test/hive/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 b/shark/src/test/hive/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 similarity index 100% rename from src/test/hive/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 rename to shark/src/test/hive/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 diff --git a/src/test/hive/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 b/shark/src/test/hive/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 similarity index 100% rename from src/test/hive/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 rename to shark/src/test/hive/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 diff --git a/src/test/hive/golden/input25-1-f57485de80ad64f9931bf92514fa1 b/shark/src/test/hive/golden/input25-1-f57485de80ad64f9931bf92514fa1 similarity index 100% rename from src/test/hive/golden/input25-1-f57485de80ad64f9931bf92514fa1 rename to shark/src/test/hive/golden/input25-1-f57485de80ad64f9931bf92514fa1 diff --git a/src/test/hive/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 b/shark/src/test/hive/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 similarity index 100% rename from src/test/hive/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 rename to shark/src/test/hive/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 diff --git a/src/test/hive/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 b/shark/src/test/hive/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 similarity index 100% rename from src/test/hive/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 rename to shark/src/test/hive/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 diff --git a/src/test/hive/golden/input25-4-72c74c55dd7d85d194448e9c58488938 b/shark/src/test/hive/golden/input25-4-72c74c55dd7d85d194448e9c58488938 similarity index 100% rename from src/test/hive/golden/input25-4-72c74c55dd7d85d194448e9c58488938 rename to shark/src/test/hive/golden/input25-4-72c74c55dd7d85d194448e9c58488938 diff --git a/src/test/hive/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 b/shark/src/test/hive/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 similarity index 100% rename from src/test/hive/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 rename to shark/src/test/hive/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 diff --git a/src/test/hive/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 b/shark/src/test/hive/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 similarity index 100% rename from src/test/hive/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 rename to shark/src/test/hive/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 diff --git a/src/test/hive/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a b/shark/src/test/hive/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a similarity index 100% rename from src/test/hive/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a rename to shark/src/test/hive/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a diff --git a/src/test/hive/golden/input28-1-f57485de80ad64f9931bf92514fa1 b/shark/src/test/hive/golden/input28-1-f57485de80ad64f9931bf92514fa1 similarity index 100% rename from src/test/hive/golden/input28-1-f57485de80ad64f9931bf92514fa1 rename to shark/src/test/hive/golden/input28-1-f57485de80ad64f9931bf92514fa1 diff --git a/src/test/hive/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c b/shark/src/test/hive/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c similarity index 100% rename from src/test/hive/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c rename to shark/src/test/hive/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c diff --git a/src/test/hive/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 b/shark/src/test/hive/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 similarity index 100% rename from src/test/hive/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 rename to shark/src/test/hive/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 diff --git a/src/test/hive/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f b/shark/src/test/hive/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f similarity index 100% rename from src/test/hive/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f rename to shark/src/test/hive/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f diff --git a/src/test/hive/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 b/shark/src/test/hive/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 similarity index 100% rename from src/test/hive/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 rename to shark/src/test/hive/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 diff --git a/src/test/hive/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff b/shark/src/test/hive/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff similarity index 100% rename from src/test/hive/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff rename to shark/src/test/hive/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff diff --git a/src/test/hive/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d b/shark/src/test/hive/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d similarity index 100% rename from src/test/hive/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d rename to shark/src/test/hive/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d diff --git a/src/test/hive/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 b/shark/src/test/hive/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 similarity index 100% rename from src/test/hive/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 rename to shark/src/test/hive/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 diff --git a/src/test/hive/golden/input3-11-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/input3-11-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/input3-11-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/input3-11-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 b/shark/src/test/hive/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 similarity index 100% rename from src/test/hive/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 rename to shark/src/test/hive/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 diff --git a/src/test/hive/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a b/shark/src/test/hive/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a similarity index 100% rename from src/test/hive/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a rename to shark/src/test/hive/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a diff --git a/src/test/hive/golden/input3-14-efee6816e20fe61595a4a2a991071219 b/shark/src/test/hive/golden/input3-14-efee6816e20fe61595a4a2a991071219 similarity index 100% rename from src/test/hive/golden/input3-14-efee6816e20fe61595a4a2a991071219 rename to shark/src/test/hive/golden/input3-14-efee6816e20fe61595a4a2a991071219 diff --git a/src/test/hive/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 b/shark/src/test/hive/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 similarity index 100% rename from src/test/hive/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 rename to shark/src/test/hive/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 diff --git a/src/test/hive/golden/input3-3-1c5990b1aed2be48311810dae3019994 b/shark/src/test/hive/golden/input3-3-1c5990b1aed2be48311810dae3019994 similarity index 100% rename from src/test/hive/golden/input3-3-1c5990b1aed2be48311810dae3019994 rename to shark/src/test/hive/golden/input3-3-1c5990b1aed2be48311810dae3019994 diff --git a/src/test/hive/golden/input3-4-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/input3-4-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/input3-4-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/input3-4-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 b/shark/src/test/hive/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 similarity index 100% rename from src/test/hive/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 rename to shark/src/test/hive/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 diff --git a/src/test/hive/golden/input3-6-ba8c440158c2519353d02471bfb05694 b/shark/src/test/hive/golden/input3-6-ba8c440158c2519353d02471bfb05694 similarity index 100% rename from src/test/hive/golden/input3-6-ba8c440158c2519353d02471bfb05694 rename to shark/src/test/hive/golden/input3-6-ba8c440158c2519353d02471bfb05694 diff --git a/src/test/hive/golden/input3-7-1c5990b1aed2be48311810dae3019994 b/shark/src/test/hive/golden/input3-7-1c5990b1aed2be48311810dae3019994 similarity index 100% rename from src/test/hive/golden/input3-7-1c5990b1aed2be48311810dae3019994 rename to shark/src/test/hive/golden/input3-7-1c5990b1aed2be48311810dae3019994 diff --git a/src/test/hive/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be b/shark/src/test/hive/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be similarity index 100% rename from src/test/hive/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be rename to shark/src/test/hive/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be diff --git a/src/test/hive/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e b/shark/src/test/hive/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e similarity index 100% rename from src/test/hive/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e rename to shark/src/test/hive/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e diff --git a/src/test/hive/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c b/shark/src/test/hive/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c similarity index 100% rename from src/test/hive/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c rename to shark/src/test/hive/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c diff --git a/src/test/hive/golden/input30-1-90c0d1a75de78c405413fd627caea4ab b/shark/src/test/hive/golden/input30-1-90c0d1a75de78c405413fd627caea4ab similarity index 100% rename from src/test/hive/golden/input30-1-90c0d1a75de78c405413fd627caea4ab rename to shark/src/test/hive/golden/input30-1-90c0d1a75de78c405413fd627caea4ab diff --git a/src/test/hive/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/input30-3-c21dba410fb07a098f93430a9d21df79 b/shark/src/test/hive/golden/input30-3-c21dba410fb07a098f93430a9d21df79 similarity index 100% rename from src/test/hive/golden/input30-3-c21dba410fb07a098f93430a9d21df79 rename to shark/src/test/hive/golden/input30-3-c21dba410fb07a098f93430a9d21df79 diff --git a/src/test/hive/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 b/shark/src/test/hive/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 similarity index 100% rename from src/test/hive/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 rename to shark/src/test/hive/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 diff --git a/src/test/hive/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 b/shark/src/test/hive/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 similarity index 100% rename from src/test/hive/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 rename to shark/src/test/hive/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 diff --git a/src/test/hive/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 b/shark/src/test/hive/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 similarity index 100% rename from src/test/hive/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 rename to shark/src/test/hive/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 diff --git a/src/test/hive/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f b/shark/src/test/hive/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f similarity index 100% rename from src/test/hive/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f rename to shark/src/test/hive/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f diff --git a/src/test/hive/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/input31-1-c21dba410fb07a098f93430a9d21df79 b/shark/src/test/hive/golden/input31-1-c21dba410fb07a098f93430a9d21df79 similarity index 100% rename from src/test/hive/golden/input31-1-c21dba410fb07a098f93430a9d21df79 rename to shark/src/test/hive/golden/input31-1-c21dba410fb07a098f93430a9d21df79 diff --git a/src/test/hive/golden/input31-2-705764f8f7cab9378964af30b83f7fe b/shark/src/test/hive/golden/input31-2-705764f8f7cab9378964af30b83f7fe similarity index 100% rename from src/test/hive/golden/input31-2-705764f8f7cab9378964af30b83f7fe rename to shark/src/test/hive/golden/input31-2-705764f8f7cab9378964af30b83f7fe diff --git a/src/test/hive/golden/input31-3-50c905261882f7fd8539fdd91e68151f b/shark/src/test/hive/golden/input31-3-50c905261882f7fd8539fdd91e68151f similarity index 100% rename from src/test/hive/golden/input31-3-50c905261882f7fd8539fdd91e68151f rename to shark/src/test/hive/golden/input31-3-50c905261882f7fd8539fdd91e68151f diff --git a/src/test/hive/golden/input31-4-2f886fa357df9342733551fa1b53f913 b/shark/src/test/hive/golden/input31-4-2f886fa357df9342733551fa1b53f913 similarity index 100% rename from src/test/hive/golden/input31-4-2f886fa357df9342733551fa1b53f913 rename to shark/src/test/hive/golden/input31-4-2f886fa357df9342733551fa1b53f913 diff --git a/src/test/hive/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/input32-1-c21dba410fb07a098f93430a9d21df79 b/shark/src/test/hive/golden/input32-1-c21dba410fb07a098f93430a9d21df79 similarity index 100% rename from src/test/hive/golden/input32-1-c21dba410fb07a098f93430a9d21df79 rename to shark/src/test/hive/golden/input32-1-c21dba410fb07a098f93430a9d21df79 diff --git a/src/test/hive/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 b/shark/src/test/hive/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 similarity index 100% rename from src/test/hive/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 rename to shark/src/test/hive/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 diff --git a/src/test/hive/golden/input32-3-b0070890240c15d647af59f41b77ba3d b/shark/src/test/hive/golden/input32-3-b0070890240c15d647af59f41b77ba3d similarity index 100% rename from src/test/hive/golden/input32-3-b0070890240c15d647af59f41b77ba3d rename to shark/src/test/hive/golden/input32-3-b0070890240c15d647af59f41b77ba3d diff --git a/src/test/hive/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 b/shark/src/test/hive/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 similarity index 100% rename from src/test/hive/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 rename to shark/src/test/hive/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 diff --git a/src/test/hive/golden/input32-5-8789d32fc5b784fe2d171566732c573e b/shark/src/test/hive/golden/input32-5-8789d32fc5b784fe2d171566732c573e similarity index 100% rename from src/test/hive/golden/input32-5-8789d32fc5b784fe2d171566732c573e rename to shark/src/test/hive/golden/input32-5-8789d32fc5b784fe2d171566732c573e diff --git a/src/test/hive/golden/input37-0-86e2e274650fb56651607ea10d356fc0 b/shark/src/test/hive/golden/input37-0-86e2e274650fb56651607ea10d356fc0 similarity index 100% rename from src/test/hive/golden/input37-0-86e2e274650fb56651607ea10d356fc0 rename to shark/src/test/hive/golden/input37-0-86e2e274650fb56651607ea10d356fc0 diff --git a/src/test/hive/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 b/shark/src/test/hive/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 similarity index 100% rename from src/test/hive/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 rename to shark/src/test/hive/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 diff --git a/src/test/hive/golden/input39-0-7bd12162381231be9d578797818957a7 b/shark/src/test/hive/golden/input39-0-7bd12162381231be9d578797818957a7 similarity index 100% rename from src/test/hive/golden/input39-0-7bd12162381231be9d578797818957a7 rename to shark/src/test/hive/golden/input39-0-7bd12162381231be9d578797818957a7 diff --git a/src/test/hive/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/shark/src/test/hive/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 similarity index 100% rename from src/test/hive/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 rename to shark/src/test/hive/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 diff --git a/src/test/hive/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 b/shark/src/test/hive/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 similarity index 100% rename from src/test/hive/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 rename to shark/src/test/hive/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 diff --git a/src/test/hive/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 b/shark/src/test/hive/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 similarity index 100% rename from src/test/hive/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 rename to shark/src/test/hive/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 diff --git a/src/test/hive/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 b/shark/src/test/hive/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 similarity index 100% rename from src/test/hive/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 rename to shark/src/test/hive/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 diff --git a/src/test/hive/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 b/shark/src/test/hive/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 similarity index 100% rename from src/test/hive/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 rename to shark/src/test/hive/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 diff --git a/src/test/hive/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d b/shark/src/test/hive/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d similarity index 100% rename from src/test/hive/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d rename to shark/src/test/hive/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d diff --git a/src/test/hive/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 b/shark/src/test/hive/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 similarity index 100% rename from src/test/hive/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 rename to shark/src/test/hive/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 diff --git a/src/test/hive/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/shark/src/test/hive/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e similarity index 100% rename from src/test/hive/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e rename to shark/src/test/hive/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e diff --git a/src/test/hive/golden/input39-4-53453776bf062f28d371fc7336b7eae2 b/shark/src/test/hive/golden/input39-4-53453776bf062f28d371fc7336b7eae2 similarity index 100% rename from src/test/hive/golden/input39-4-53453776bf062f28d371fc7336b7eae2 rename to shark/src/test/hive/golden/input39-4-53453776bf062f28d371fc7336b7eae2 diff --git a/src/test/hive/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/input39-6-763ab5853bff619e6525c01e46b2a923 b/shark/src/test/hive/golden/input39-6-763ab5853bff619e6525c01e46b2a923 similarity index 100% rename from src/test/hive/golden/input39-6-763ab5853bff619e6525c01e46b2a923 rename to shark/src/test/hive/golden/input39-6-763ab5853bff619e6525c01e46b2a923 diff --git a/src/test/hive/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e b/shark/src/test/hive/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e similarity index 100% rename from src/test/hive/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e rename to shark/src/test/hive/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e diff --git a/src/test/hive/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 b/shark/src/test/hive/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 similarity index 100% rename from src/test/hive/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 rename to shark/src/test/hive/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 diff --git a/src/test/hive/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 b/shark/src/test/hive/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 similarity index 100% rename from src/test/hive/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 rename to shark/src/test/hive/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 diff --git a/src/test/hive/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe b/shark/src/test/hive/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe similarity index 100% rename from src/test/hive/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe rename to shark/src/test/hive/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe diff --git a/src/test/hive/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/shark/src/test/hive/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 similarity index 100% rename from src/test/hive/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 rename to shark/src/test/hive/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 diff --git a/src/test/hive/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 b/shark/src/test/hive/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 similarity index 100% rename from src/test/hive/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 rename to shark/src/test/hive/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 diff --git a/src/test/hive/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 b/shark/src/test/hive/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 similarity index 100% rename from src/test/hive/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 rename to shark/src/test/hive/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 diff --git a/src/test/hive/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 b/shark/src/test/hive/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 similarity index 100% rename from src/test/hive/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 rename to shark/src/test/hive/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 diff --git a/src/test/hive/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 b/shark/src/test/hive/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 similarity index 100% rename from src/test/hive/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 rename to shark/src/test/hive/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 diff --git a/src/test/hive/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/shark/src/test/hive/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e similarity index 100% rename from src/test/hive/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e rename to shark/src/test/hive/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e diff --git a/src/test/hive/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 b/shark/src/test/hive/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 similarity index 100% rename from src/test/hive/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 rename to shark/src/test/hive/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 diff --git a/src/test/hive/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee b/shark/src/test/hive/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee similarity index 100% rename from src/test/hive/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee rename to shark/src/test/hive/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee diff --git a/src/test/hive/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 b/shark/src/test/hive/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 similarity index 100% rename from src/test/hive/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 rename to shark/src/test/hive/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 diff --git a/src/test/hive/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f b/shark/src/test/hive/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f similarity index 100% rename from src/test/hive/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f rename to shark/src/test/hive/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f diff --git a/src/test/hive/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 b/shark/src/test/hive/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 similarity index 100% rename from src/test/hive/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 rename to shark/src/test/hive/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 diff --git a/src/test/hive/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 b/shark/src/test/hive/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 similarity index 100% rename from src/test/hive/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 rename to shark/src/test/hive/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 diff --git a/src/test/hive/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 b/shark/src/test/hive/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 similarity index 100% rename from src/test/hive/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 rename to shark/src/test/hive/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 diff --git a/src/test/hive/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 b/shark/src/test/hive/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 similarity index 100% rename from src/test/hive/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 rename to shark/src/test/hive/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 diff --git a/src/test/hive/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d b/shark/src/test/hive/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d similarity index 100% rename from src/test/hive/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d rename to shark/src/test/hive/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d diff --git a/src/test/hive/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece b/shark/src/test/hive/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece similarity index 100% rename from src/test/hive/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece rename to shark/src/test/hive/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece diff --git a/src/test/hive/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d b/shark/src/test/hive/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d similarity index 100% rename from src/test/hive/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d rename to shark/src/test/hive/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d diff --git a/src/test/hive/golden/input3_limit-5-3664b564747487df13a5d109837219b5 b/shark/src/test/hive/golden/input3_limit-5-3664b564747487df13a5d109837219b5 similarity index 100% rename from src/test/hive/golden/input3_limit-5-3664b564747487df13a5d109837219b5 rename to shark/src/test/hive/golden/input3_limit-5-3664b564747487df13a5d109837219b5 diff --git a/src/test/hive/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 b/shark/src/test/hive/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 similarity index 100% rename from src/test/hive/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 rename to shark/src/test/hive/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 diff --git a/src/test/hive/golden/input4-0-b9cd4041ca510639838a97376747b498 b/shark/src/test/hive/golden/input4-0-b9cd4041ca510639838a97376747b498 similarity index 100% rename from src/test/hive/golden/input4-0-b9cd4041ca510639838a97376747b498 rename to shark/src/test/hive/golden/input4-0-b9cd4041ca510639838a97376747b498 diff --git a/src/test/hive/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 b/shark/src/test/hive/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 similarity index 100% rename from src/test/hive/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 rename to shark/src/test/hive/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 diff --git a/src/test/hive/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 b/shark/src/test/hive/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 similarity index 100% rename from src/test/hive/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 rename to shark/src/test/hive/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 diff --git a/src/test/hive/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 b/shark/src/test/hive/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 similarity index 100% rename from src/test/hive/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 rename to shark/src/test/hive/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 diff --git a/src/test/hive/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 b/shark/src/test/hive/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 similarity index 100% rename from src/test/hive/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 rename to shark/src/test/hive/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 diff --git a/src/test/hive/golden/input40-0-37e74908fd43254141ae3664f13a48d b/shark/src/test/hive/golden/input40-0-37e74908fd43254141ae3664f13a48d similarity index 100% rename from src/test/hive/golden/input40-0-37e74908fd43254141ae3664f13a48d rename to shark/src/test/hive/golden/input40-0-37e74908fd43254141ae3664f13a48d diff --git a/src/test/hive/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 b/shark/src/test/hive/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 similarity index 100% rename from src/test/hive/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 rename to shark/src/test/hive/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 diff --git a/src/test/hive/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 b/shark/src/test/hive/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 similarity index 100% rename from src/test/hive/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 rename to shark/src/test/hive/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 diff --git a/src/test/hive/golden/input40-3-12f634800ffae17942ddd789a01af227 b/shark/src/test/hive/golden/input40-3-12f634800ffae17942ddd789a01af227 similarity index 100% rename from src/test/hive/golden/input40-3-12f634800ffae17942ddd789a01af227 rename to shark/src/test/hive/golden/input40-3-12f634800ffae17942ddd789a01af227 diff --git a/src/test/hive/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e b/shark/src/test/hive/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e similarity index 100% rename from src/test/hive/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e rename to shark/src/test/hive/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e diff --git a/src/test/hive/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 b/shark/src/test/hive/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 similarity index 100% rename from src/test/hive/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 rename to shark/src/test/hive/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 diff --git a/src/test/hive/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 b/shark/src/test/hive/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 similarity index 100% rename from src/test/hive/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 rename to shark/src/test/hive/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 diff --git a/src/test/hive/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 b/shark/src/test/hive/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 similarity index 100% rename from src/test/hive/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 rename to shark/src/test/hive/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 diff --git a/src/test/hive/golden/input41-0-763ab5853bff619e6525c01e46b2a923 b/shark/src/test/hive/golden/input41-0-763ab5853bff619e6525c01e46b2a923 similarity index 100% rename from src/test/hive/golden/input41-0-763ab5853bff619e6525c01e46b2a923 rename to shark/src/test/hive/golden/input41-0-763ab5853bff619e6525c01e46b2a923 diff --git a/src/test/hive/golden/input41-1-8112b3a278e8337dbd6f017ae178658b b/shark/src/test/hive/golden/input41-1-8112b3a278e8337dbd6f017ae178658b similarity index 100% rename from src/test/hive/golden/input41-1-8112b3a278e8337dbd6f017ae178658b rename to shark/src/test/hive/golden/input41-1-8112b3a278e8337dbd6f017ae178658b diff --git a/src/test/hive/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 b/shark/src/test/hive/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 similarity index 100% rename from src/test/hive/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 rename to shark/src/test/hive/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 diff --git a/src/test/hive/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee b/shark/src/test/hive/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee similarity index 100% rename from src/test/hive/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee rename to shark/src/test/hive/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee diff --git a/src/test/hive/golden/input43-0-2baba8070f3585debc14b6bb3c83607a b/shark/src/test/hive/golden/input43-0-2baba8070f3585debc14b6bb3c83607a similarity index 100% rename from src/test/hive/golden/input43-0-2baba8070f3585debc14b6bb3c83607a rename to shark/src/test/hive/golden/input43-0-2baba8070f3585debc14b6bb3c83607a diff --git a/src/test/hive/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 b/shark/src/test/hive/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 similarity index 100% rename from src/test/hive/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 rename to shark/src/test/hive/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 diff --git a/src/test/hive/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f b/shark/src/test/hive/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f similarity index 100% rename from src/test/hive/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f rename to shark/src/test/hive/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f diff --git a/src/test/hive/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca b/shark/src/test/hive/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca similarity index 100% rename from src/test/hive/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca rename to shark/src/test/hive/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca diff --git a/src/test/hive/golden/input46-3-c185163787977498a4b84f39f983c431 b/shark/src/test/hive/golden/input46-3-c185163787977498a4b84f39f983c431 similarity index 100% rename from src/test/hive/golden/input46-3-c185163787977498a4b84f39f983c431 rename to shark/src/test/hive/golden/input46-3-c185163787977498a4b84f39f983c431 diff --git a/src/test/hive/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 b/shark/src/test/hive/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 similarity index 100% rename from src/test/hive/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 rename to shark/src/test/hive/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 diff --git a/src/test/hive/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab b/shark/src/test/hive/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab similarity index 100% rename from src/test/hive/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab rename to shark/src/test/hive/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab diff --git a/src/test/hive/golden/input49-0-2bd546beeb607da5b925ffbea20a741c b/shark/src/test/hive/golden/input49-0-2bd546beeb607da5b925ffbea20a741c similarity index 100% rename from src/test/hive/golden/input49-0-2bd546beeb607da5b925ffbea20a741c rename to shark/src/test/hive/golden/input49-0-2bd546beeb607da5b925ffbea20a741c diff --git a/src/test/hive/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d b/shark/src/test/hive/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d similarity index 100% rename from src/test/hive/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d rename to shark/src/test/hive/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d diff --git a/src/test/hive/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 b/shark/src/test/hive/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 similarity index 100% rename from src/test/hive/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 rename to shark/src/test/hive/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 diff --git a/src/test/hive/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 b/shark/src/test/hive/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 similarity index 100% rename from src/test/hive/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 rename to shark/src/test/hive/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 diff --git a/src/test/hive/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 b/shark/src/test/hive/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 similarity index 100% rename from src/test/hive/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 rename to shark/src/test/hive/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 diff --git a/src/test/hive/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 b/shark/src/test/hive/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 similarity index 100% rename from src/test/hive/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 rename to shark/src/test/hive/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 diff --git a/src/test/hive/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 b/shark/src/test/hive/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 similarity index 100% rename from src/test/hive/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 rename to shark/src/test/hive/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 diff --git a/src/test/hive/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d b/shark/src/test/hive/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d similarity index 100% rename from src/test/hive/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d rename to shark/src/test/hive/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d diff --git a/src/test/hive/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af b/shark/src/test/hive/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af similarity index 100% rename from src/test/hive/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af rename to shark/src/test/hive/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af diff --git a/src/test/hive/golden/input5-0-659e06570690cceeb3f37e10e855d2ea b/shark/src/test/hive/golden/input5-0-659e06570690cceeb3f37e10e855d2ea similarity index 100% rename from src/test/hive/golden/input5-0-659e06570690cceeb3f37e10e855d2ea rename to shark/src/test/hive/golden/input5-0-659e06570690cceeb3f37e10e855d2ea diff --git a/src/test/hive/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 b/shark/src/test/hive/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 similarity index 100% rename from src/test/hive/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 rename to shark/src/test/hive/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 diff --git a/src/test/hive/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad b/shark/src/test/hive/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad similarity index 100% rename from src/test/hive/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad rename to shark/src/test/hive/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad diff --git a/src/test/hive/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input6-0-659e06570690cceeb3f37e10e855d2ea b/shark/src/test/hive/golden/input6-0-659e06570690cceeb3f37e10e855d2ea similarity index 100% rename from src/test/hive/golden/input6-0-659e06570690cceeb3f37e10e855d2ea rename to shark/src/test/hive/golden/input6-0-659e06570690cceeb3f37e10e855d2ea diff --git a/src/test/hive/golden/input6-1-21149162906e31964a328b6cb2d5286e b/shark/src/test/hive/golden/input6-1-21149162906e31964a328b6cb2d5286e similarity index 100% rename from src/test/hive/golden/input6-1-21149162906e31964a328b6cb2d5286e rename to shark/src/test/hive/golden/input6-1-21149162906e31964a328b6cb2d5286e diff --git a/src/test/hive/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c b/shark/src/test/hive/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c similarity index 100% rename from src/test/hive/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c rename to shark/src/test/hive/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c diff --git a/src/test/hive/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 b/shark/src/test/hive/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 similarity index 100% rename from src/test/hive/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 rename to shark/src/test/hive/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 diff --git a/src/test/hive/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 b/shark/src/test/hive/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 similarity index 100% rename from src/test/hive/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 rename to shark/src/test/hive/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 diff --git a/src/test/hive/golden/input7-2-c740098a289f9a4267d26fc10347a0dc b/shark/src/test/hive/golden/input7-2-c740098a289f9a4267d26fc10347a0dc similarity index 100% rename from src/test/hive/golden/input7-2-c740098a289f9a4267d26fc10347a0dc rename to shark/src/test/hive/golden/input7-2-c740098a289f9a4267d26fc10347a0dc diff --git a/src/test/hive/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa b/shark/src/test/hive/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa similarity index 100% rename from src/test/hive/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa rename to shark/src/test/hive/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa diff --git a/src/test/hive/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa b/shark/src/test/hive/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa similarity index 100% rename from src/test/hive/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa rename to shark/src/test/hive/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa diff --git a/src/test/hive/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba b/shark/src/test/hive/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba similarity index 100% rename from src/test/hive/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba rename to shark/src/test/hive/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba diff --git a/src/test/hive/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 b/shark/src/test/hive/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 similarity index 100% rename from src/test/hive/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 rename to shark/src/test/hive/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 diff --git a/src/test/hive/golden/input9-1-b30105391cb53915997a84ca3137bd0a b/shark/src/test/hive/golden/input9-1-b30105391cb53915997a84ca3137bd0a similarity index 100% rename from src/test/hive/golden/input9-1-b30105391cb53915997a84ca3137bd0a rename to shark/src/test/hive/golden/input9-1-b30105391cb53915997a84ca3137bd0a diff --git a/src/test/hive/golden/input9-2-171ea277f05a642699a0f91bacffccd8 b/shark/src/test/hive/golden/input9-2-171ea277f05a642699a0f91bacffccd8 similarity index 100% rename from src/test/hive/golden/input9-2-171ea277f05a642699a0f91bacffccd8 rename to shark/src/test/hive/golden/input9-2-171ea277f05a642699a0f91bacffccd8 diff --git a/src/test/hive/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e b/shark/src/test/hive/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e similarity index 100% rename from src/test/hive/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e rename to shark/src/test/hive/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e diff --git a/src/test/hive/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 b/shark/src/test/hive/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 similarity index 100% rename from src/test/hive/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 rename to shark/src/test/hive/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 diff --git a/src/test/hive/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be b/shark/src/test/hive/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be similarity index 100% rename from src/test/hive/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be rename to shark/src/test/hive/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be diff --git a/src/test/hive/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 b/shark/src/test/hive/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 similarity index 100% rename from src/test/hive/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 rename to shark/src/test/hive/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 diff --git a/src/test/hive/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 b/shark/src/test/hive/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 similarity index 100% rename from src/test/hive/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 rename to shark/src/test/hive/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 diff --git a/src/test/hive/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d b/shark/src/test/hive/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d similarity index 100% rename from src/test/hive/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d rename to shark/src/test/hive/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d diff --git a/src/test/hive/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc b/shark/src/test/hive/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc similarity index 100% rename from src/test/hive/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc rename to shark/src/test/hive/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc diff --git a/src/test/hive/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 b/shark/src/test/hive/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 similarity index 100% rename from src/test/hive/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 rename to shark/src/test/hive/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 diff --git a/src/test/hive/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 b/shark/src/test/hive/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 similarity index 100% rename from src/test/hive/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 rename to shark/src/test/hive/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 diff --git a/src/test/hive/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d b/shark/src/test/hive/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d similarity index 100% rename from src/test/hive/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d rename to shark/src/test/hive/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d diff --git a/src/test/hive/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 b/shark/src/test/hive/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 similarity index 100% rename from src/test/hive/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 rename to shark/src/test/hive/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 diff --git a/src/test/hive/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 b/shark/src/test/hive/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 similarity index 100% rename from src/test/hive/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 rename to shark/src/test/hive/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 diff --git a/src/test/hive/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c b/shark/src/test/hive/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c similarity index 100% rename from src/test/hive/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c rename to shark/src/test/hive/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c diff --git a/src/test/hive/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c b/shark/src/test/hive/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c similarity index 100% rename from src/test/hive/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c rename to shark/src/test/hive/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c diff --git a/src/test/hive/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc b/shark/src/test/hive/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc similarity index 100% rename from src/test/hive/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc rename to shark/src/test/hive/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc diff --git a/src/test/hive/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 b/shark/src/test/hive/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 similarity index 100% rename from src/test/hive/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 rename to shark/src/test/hive/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 diff --git a/src/test/hive/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db b/shark/src/test/hive/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db similarity index 100% rename from src/test/hive/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db rename to shark/src/test/hive/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db diff --git a/src/test/hive/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a b/shark/src/test/hive/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a similarity index 100% rename from src/test/hive/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a rename to shark/src/test/hive/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a diff --git a/src/test/hive/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 b/shark/src/test/hive/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 similarity index 100% rename from src/test/hive/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 rename to shark/src/test/hive/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 diff --git a/src/test/hive/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 b/shark/src/test/hive/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 similarity index 100% rename from src/test/hive/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 rename to shark/src/test/hive/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 diff --git a/src/test/hive/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c b/shark/src/test/hive/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c similarity index 100% rename from src/test/hive/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c rename to shark/src/test/hive/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c diff --git a/src/test/hive/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a b/shark/src/test/hive/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a similarity index 100% rename from src/test/hive/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a rename to shark/src/test/hive/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a diff --git a/src/test/hive/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 b/shark/src/test/hive/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 similarity index 100% rename from src/test/hive/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 rename to shark/src/test/hive/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 diff --git a/src/test/hive/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 b/shark/src/test/hive/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 similarity index 100% rename from src/test/hive/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 rename to shark/src/test/hive/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 diff --git a/src/test/hive/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 b/shark/src/test/hive/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 similarity index 100% rename from src/test/hive/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 rename to shark/src/test/hive/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 diff --git a/src/test/hive/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb b/shark/src/test/hive/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb similarity index 100% rename from src/test/hive/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb rename to shark/src/test/hive/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb diff --git a/src/test/hive/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 b/shark/src/test/hive/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 similarity index 100% rename from src/test/hive/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 rename to shark/src/test/hive/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 diff --git a/src/test/hive/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 b/shark/src/test/hive/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 similarity index 100% rename from src/test/hive/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 rename to shark/src/test/hive/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 diff --git a/src/test/hive/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc b/shark/src/test/hive/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc similarity index 100% rename from src/test/hive/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc rename to shark/src/test/hive/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc diff --git a/src/test/hive/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 b/shark/src/test/hive/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 similarity index 100% rename from src/test/hive/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 rename to shark/src/test/hive/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 diff --git a/src/test/hive/golden/input_part10-1-35f67c0112026170d015b5f80a254bde b/shark/src/test/hive/golden/input_part10-1-35f67c0112026170d015b5f80a254bde similarity index 100% rename from src/test/hive/golden/input_part10-1-35f67c0112026170d015b5f80a254bde rename to shark/src/test/hive/golden/input_part10-1-35f67c0112026170d015b5f80a254bde diff --git a/src/test/hive/golden/input_part10-2-40069c199502c2724ac2a2733f964248 b/shark/src/test/hive/golden/input_part10-2-40069c199502c2724ac2a2733f964248 similarity index 100% rename from src/test/hive/golden/input_part10-2-40069c199502c2724ac2a2733f964248 rename to shark/src/test/hive/golden/input_part10-2-40069c199502c2724ac2a2733f964248 diff --git a/src/test/hive/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 b/shark/src/test/hive/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 similarity index 100% rename from src/test/hive/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 rename to shark/src/test/hive/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 diff --git a/src/test/hive/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 b/shark/src/test/hive/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 similarity index 100% rename from src/test/hive/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 rename to shark/src/test/hive/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 diff --git a/src/test/hive/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf b/shark/src/test/hive/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf similarity index 100% rename from src/test/hive/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf rename to shark/src/test/hive/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf diff --git a/src/test/hive/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde b/shark/src/test/hive/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde similarity index 100% rename from src/test/hive/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde rename to shark/src/test/hive/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde diff --git a/src/test/hive/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 b/shark/src/test/hive/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 similarity index 100% rename from src/test/hive/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 rename to shark/src/test/hive/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 diff --git a/src/test/hive/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 b/shark/src/test/hive/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 similarity index 100% rename from src/test/hive/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 rename to shark/src/test/hive/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 diff --git a/src/test/hive/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 b/shark/src/test/hive/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 similarity index 100% rename from src/test/hive/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 rename to shark/src/test/hive/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 diff --git a/src/test/hive/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 b/shark/src/test/hive/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 similarity index 100% rename from src/test/hive/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 rename to shark/src/test/hive/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 diff --git a/src/test/hive/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e b/shark/src/test/hive/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e similarity index 100% rename from src/test/hive/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e rename to shark/src/test/hive/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e diff --git a/src/test/hive/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd b/shark/src/test/hive/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd similarity index 100% rename from src/test/hive/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd rename to shark/src/test/hive/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd diff --git a/src/test/hive/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 b/shark/src/test/hive/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 similarity index 100% rename from src/test/hive/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 rename to shark/src/test/hive/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 diff --git a/src/test/hive/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 b/shark/src/test/hive/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 similarity index 100% rename from src/test/hive/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 rename to shark/src/test/hive/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 diff --git a/src/test/hive/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 b/shark/src/test/hive/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 similarity index 100% rename from src/test/hive/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 rename to shark/src/test/hive/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 diff --git a/src/test/hive/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff b/shark/src/test/hive/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff similarity index 100% rename from src/test/hive/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff rename to shark/src/test/hive/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff diff --git a/src/test/hive/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 b/shark/src/test/hive/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 similarity index 100% rename from src/test/hive/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 rename to shark/src/test/hive/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 diff --git a/src/test/hive/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef b/shark/src/test/hive/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef similarity index 100% rename from src/test/hive/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef rename to shark/src/test/hive/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef diff --git a/src/test/hive/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 b/shark/src/test/hive/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 similarity index 100% rename from src/test/hive/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 rename to shark/src/test/hive/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 diff --git a/src/test/hive/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b b/shark/src/test/hive/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b similarity index 100% rename from src/test/hive/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b rename to shark/src/test/hive/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b diff --git a/src/test/hive/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c b/shark/src/test/hive/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c similarity index 100% rename from src/test/hive/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c rename to shark/src/test/hive/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c diff --git a/src/test/hive/golden/input_part5-2-d678533566fba4b922f01284538ca484 b/shark/src/test/hive/golden/input_part5-2-d678533566fba4b922f01284538ca484 similarity index 100% rename from src/test/hive/golden/input_part5-2-d678533566fba4b922f01284538ca484 rename to shark/src/test/hive/golden/input_part5-2-d678533566fba4b922f01284538ca484 diff --git a/src/test/hive/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 b/shark/src/test/hive/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 similarity index 100% rename from src/test/hive/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 rename to shark/src/test/hive/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 diff --git a/src/test/hive/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d b/shark/src/test/hive/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d similarity index 100% rename from src/test/hive/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d rename to shark/src/test/hive/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d diff --git a/src/test/hive/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 b/shark/src/test/hive/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 similarity index 100% rename from src/test/hive/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 rename to shark/src/test/hive/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 diff --git a/src/test/hive/golden/input_part7-0-14448a12627b8bccf6828032f938f39e b/shark/src/test/hive/golden/input_part7-0-14448a12627b8bccf6828032f938f39e similarity index 100% rename from src/test/hive/golden/input_part7-0-14448a12627b8bccf6828032f938f39e rename to shark/src/test/hive/golden/input_part7-0-14448a12627b8bccf6828032f938f39e diff --git a/src/test/hive/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 b/shark/src/test/hive/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 similarity index 100% rename from src/test/hive/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 rename to shark/src/test/hive/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 diff --git a/src/test/hive/golden/input_part8-0-f60652796af548135f9e104b456840e3 b/shark/src/test/hive/golden/input_part8-0-f60652796af548135f9e104b456840e3 similarity index 100% rename from src/test/hive/golden/input_part8-0-f60652796af548135f9e104b456840e3 rename to shark/src/test/hive/golden/input_part8-0-f60652796af548135f9e104b456840e3 diff --git a/src/test/hive/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 b/shark/src/test/hive/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 similarity index 100% rename from src/test/hive/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 rename to shark/src/test/hive/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 diff --git a/src/test/hive/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 b/shark/src/test/hive/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 similarity index 100% rename from src/test/hive/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 rename to shark/src/test/hive/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 diff --git a/src/test/hive/golden/input_part9-1-e60c60afc073367464898b8396e8f643 b/shark/src/test/hive/golden/input_part9-1-e60c60afc073367464898b8396e8f643 similarity index 100% rename from src/test/hive/golden/input_part9-1-e60c60afc073367464898b8396e8f643 rename to shark/src/test/hive/golden/input_part9-1-e60c60afc073367464898b8396e8f643 diff --git a/src/test/hive/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 b/shark/src/test/hive/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 similarity index 100% rename from src/test/hive/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 rename to shark/src/test/hive/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 diff --git a/src/test/hive/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e b/shark/src/test/hive/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e similarity index 100% rename from src/test/hive/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e rename to shark/src/test/hive/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e diff --git a/src/test/hive/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf b/shark/src/test/hive/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf similarity index 100% rename from src/test/hive/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf rename to shark/src/test/hive/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf diff --git a/src/test/hive/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 b/shark/src/test/hive/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 similarity index 100% rename from src/test/hive/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 rename to shark/src/test/hive/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 diff --git a/src/test/hive/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 b/shark/src/test/hive/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 similarity index 100% rename from src/test/hive/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 rename to shark/src/test/hive/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 diff --git a/src/test/hive/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c b/shark/src/test/hive/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c similarity index 100% rename from src/test/hive/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c rename to shark/src/test/hive/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c diff --git a/src/test/hive/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 b/shark/src/test/hive/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 similarity index 100% rename from src/test/hive/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 rename to shark/src/test/hive/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 diff --git a/src/test/hive/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 b/shark/src/test/hive/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 similarity index 100% rename from src/test/hive/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 rename to shark/src/test/hive/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 diff --git a/src/test/hive/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f b/shark/src/test/hive/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f similarity index 100% rename from src/test/hive/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f rename to shark/src/test/hive/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f diff --git a/src/test/hive/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 b/shark/src/test/hive/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 similarity index 100% rename from src/test/hive/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 rename to shark/src/test/hive/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 diff --git a/src/test/hive/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 b/shark/src/test/hive/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 similarity index 100% rename from src/test/hive/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 rename to shark/src/test/hive/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 diff --git a/src/test/hive/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 b/shark/src/test/hive/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 similarity index 100% rename from src/test/hive/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 rename to shark/src/test/hive/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 diff --git a/src/test/hive/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd b/shark/src/test/hive/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd similarity index 100% rename from src/test/hive/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd rename to shark/src/test/hive/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd diff --git a/src/test/hive/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac b/shark/src/test/hive/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac similarity index 100% rename from src/test/hive/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac rename to shark/src/test/hive/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac diff --git a/src/test/hive/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 b/shark/src/test/hive/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 similarity index 100% rename from src/test/hive/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 rename to shark/src/test/hive/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 diff --git a/src/test/hive/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 b/shark/src/test/hive/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 similarity index 100% rename from src/test/hive/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 rename to shark/src/test/hive/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 diff --git a/src/test/hive/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 b/shark/src/test/hive/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 similarity index 100% rename from src/test/hive/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 rename to shark/src/test/hive/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 diff --git a/src/test/hive/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 b/shark/src/test/hive/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 similarity index 100% rename from src/test/hive/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 rename to shark/src/test/hive/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 diff --git a/src/test/hive/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 b/shark/src/test/hive/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 similarity index 100% rename from src/test/hive/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 rename to shark/src/test/hive/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 diff --git a/src/test/hive/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 b/shark/src/test/hive/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 similarity index 100% rename from src/test/hive/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 rename to shark/src/test/hive/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 diff --git a/src/test/hive/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 b/shark/src/test/hive/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 similarity index 100% rename from src/test/hive/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 rename to shark/src/test/hive/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 diff --git a/src/test/hive/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 b/shark/src/test/hive/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 similarity index 100% rename from src/test/hive/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 rename to shark/src/test/hive/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 diff --git a/src/test/hive/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 b/shark/src/test/hive/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 similarity index 100% rename from src/test/hive/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 rename to shark/src/test/hive/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 diff --git a/src/test/hive/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 b/shark/src/test/hive/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 similarity index 100% rename from src/test/hive/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 rename to shark/src/test/hive/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 diff --git a/src/test/hive/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf b/shark/src/test/hive/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf similarity index 100% rename from src/test/hive/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf rename to shark/src/test/hive/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf diff --git a/src/test/hive/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c b/shark/src/test/hive/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c similarity index 100% rename from src/test/hive/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c rename to shark/src/test/hive/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c diff --git a/src/test/hive/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c b/shark/src/test/hive/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c similarity index 100% rename from src/test/hive/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c rename to shark/src/test/hive/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c diff --git a/src/test/hive/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 b/shark/src/test/hive/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 similarity index 100% rename from src/test/hive/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 rename to shark/src/test/hive/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 diff --git a/src/test/hive/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 b/shark/src/test/hive/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 similarity index 100% rename from src/test/hive/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 rename to shark/src/test/hive/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 diff --git a/src/test/hive/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 b/shark/src/test/hive/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 similarity index 100% rename from src/test/hive/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 rename to shark/src/test/hive/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 diff --git a/src/test/hive/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 b/shark/src/test/hive/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 similarity index 100% rename from src/test/hive/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 rename to shark/src/test/hive/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 diff --git a/src/test/hive/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 b/shark/src/test/hive/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 similarity index 100% rename from src/test/hive/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 rename to shark/src/test/hive/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 diff --git a/src/test/hive/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 b/shark/src/test/hive/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 similarity index 100% rename from src/test/hive/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 rename to shark/src/test/hive/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 diff --git a/src/test/hive/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f b/shark/src/test/hive/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f similarity index 100% rename from src/test/hive/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f rename to shark/src/test/hive/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f diff --git a/src/test/hive/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 b/shark/src/test/hive/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 similarity index 100% rename from src/test/hive/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 rename to shark/src/test/hive/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 diff --git a/src/test/hive/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d b/shark/src/test/hive/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d similarity index 100% rename from src/test/hive/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d rename to shark/src/test/hive/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d diff --git a/src/test/hive/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 b/shark/src/test/hive/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 similarity index 100% rename from src/test/hive/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 rename to shark/src/test/hive/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 diff --git a/src/test/hive/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 b/shark/src/test/hive/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 similarity index 100% rename from src/test/hive/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 rename to shark/src/test/hive/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 diff --git a/src/test/hive/golden/inputddl7-13-45059a21c202b4658285738ee62a018a b/shark/src/test/hive/golden/inputddl7-13-45059a21c202b4658285738ee62a018a similarity index 100% rename from src/test/hive/golden/inputddl7-13-45059a21c202b4658285738ee62a018a rename to shark/src/test/hive/golden/inputddl7-13-45059a21c202b4658285738ee62a018a diff --git a/src/test/hive/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 b/shark/src/test/hive/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 similarity index 100% rename from src/test/hive/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 rename to shark/src/test/hive/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 diff --git a/src/test/hive/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe b/shark/src/test/hive/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe similarity index 100% rename from src/test/hive/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe rename to shark/src/test/hive/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe diff --git a/src/test/hive/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 b/shark/src/test/hive/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 similarity index 100% rename from src/test/hive/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 rename to shark/src/test/hive/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 diff --git a/src/test/hive/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 b/shark/src/test/hive/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 similarity index 100% rename from src/test/hive/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 rename to shark/src/test/hive/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 diff --git a/src/test/hive/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 b/shark/src/test/hive/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 similarity index 100% rename from src/test/hive/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 rename to shark/src/test/hive/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 diff --git a/src/test/hive/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 b/shark/src/test/hive/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 similarity index 100% rename from src/test/hive/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 rename to shark/src/test/hive/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 diff --git a/src/test/hive/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a b/shark/src/test/hive/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a similarity index 100% rename from src/test/hive/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a rename to shark/src/test/hive/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a diff --git a/src/test/hive/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 b/shark/src/test/hive/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 similarity index 100% rename from src/test/hive/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 rename to shark/src/test/hive/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 diff --git a/src/test/hive/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 b/shark/src/test/hive/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 similarity index 100% rename from src/test/hive/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 rename to shark/src/test/hive/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 diff --git a/src/test/hive/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac b/shark/src/test/hive/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac similarity index 100% rename from src/test/hive/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac rename to shark/src/test/hive/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac diff --git a/src/test/hive/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 b/shark/src/test/hive/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 similarity index 100% rename from src/test/hive/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 rename to shark/src/test/hive/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 diff --git a/src/test/hive/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 b/shark/src/test/hive/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 similarity index 100% rename from src/test/hive/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 rename to shark/src/test/hive/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 diff --git a/src/test/hive/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac b/shark/src/test/hive/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac similarity index 100% rename from src/test/hive/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac rename to shark/src/test/hive/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac diff --git a/src/test/hive/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 b/shark/src/test/hive/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 similarity index 100% rename from src/test/hive/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 rename to shark/src/test/hive/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 diff --git a/src/test/hive/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 b/shark/src/test/hive/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 similarity index 100% rename from src/test/hive/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 rename to shark/src/test/hive/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 diff --git a/src/test/hive/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 b/shark/src/test/hive/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 similarity index 100% rename from src/test/hive/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 rename to shark/src/test/hive/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 diff --git a/src/test/hive/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c b/shark/src/test/hive/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c similarity index 100% rename from src/test/hive/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c rename to shark/src/test/hive/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c diff --git a/src/test/hive/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d b/shark/src/test/hive/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d similarity index 100% rename from src/test/hive/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d rename to shark/src/test/hive/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d diff --git a/src/test/hive/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 b/shark/src/test/hive/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 similarity index 100% rename from src/test/hive/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 rename to shark/src/test/hive/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 diff --git a/src/test/hive/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 b/shark/src/test/hive/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 similarity index 100% rename from src/test/hive/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 rename to shark/src/test/hive/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 diff --git a/src/test/hive/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 b/shark/src/test/hive/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 similarity index 100% rename from src/test/hive/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 rename to shark/src/test/hive/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 diff --git a/src/test/hive/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef b/shark/src/test/hive/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef similarity index 100% rename from src/test/hive/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef rename to shark/src/test/hive/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef diff --git a/src/test/hive/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 b/shark/src/test/hive/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 similarity index 100% rename from src/test/hive/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 rename to shark/src/test/hive/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 diff --git a/src/test/hive/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef b/shark/src/test/hive/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef similarity index 100% rename from src/test/hive/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef rename to shark/src/test/hive/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef diff --git a/src/test/hive/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 b/shark/src/test/hive/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 similarity index 100% rename from src/test/hive/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 rename to shark/src/test/hive/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 diff --git a/src/test/hive/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef b/shark/src/test/hive/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef similarity index 100% rename from src/test/hive/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef rename to shark/src/test/hive/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef diff --git a/src/test/hive/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 b/shark/src/test/hive/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 similarity index 100% rename from src/test/hive/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 rename to shark/src/test/hive/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 diff --git a/src/test/hive/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 b/shark/src/test/hive/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 similarity index 100% rename from src/test/hive/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 rename to shark/src/test/hive/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 diff --git a/src/test/hive/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b b/shark/src/test/hive/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b similarity index 100% rename from src/test/hive/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b rename to shark/src/test/hive/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b diff --git a/src/test/hive/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 b/shark/src/test/hive/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 similarity index 100% rename from src/test/hive/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 rename to shark/src/test/hive/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 diff --git a/src/test/hive/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb b/shark/src/test/hive/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb similarity index 100% rename from src/test/hive/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb rename to shark/src/test/hive/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb diff --git a/src/test/hive/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 b/shark/src/test/hive/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 similarity index 100% rename from src/test/hive/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 rename to shark/src/test/hive/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 diff --git a/src/test/hive/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 b/shark/src/test/hive/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 similarity index 100% rename from src/test/hive/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 rename to shark/src/test/hive/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 diff --git a/src/test/hive/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 b/shark/src/test/hive/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 similarity index 100% rename from src/test/hive/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 rename to shark/src/test/hive/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 diff --git a/src/test/hive/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb b/shark/src/test/hive/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb similarity index 100% rename from src/test/hive/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb rename to shark/src/test/hive/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb diff --git a/src/test/hive/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 b/shark/src/test/hive/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 similarity index 100% rename from src/test/hive/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 rename to shark/src/test/hive/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 diff --git a/src/test/hive/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 b/shark/src/test/hive/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 similarity index 100% rename from src/test/hive/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 rename to shark/src/test/hive/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 diff --git a/src/test/hive/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb b/shark/src/test/hive/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb similarity index 100% rename from src/test/hive/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb rename to shark/src/test/hive/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb diff --git a/src/test/hive/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d b/shark/src/test/hive/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d similarity index 100% rename from src/test/hive/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d rename to shark/src/test/hive/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d diff --git a/src/test/hive/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b b/shark/src/test/hive/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b similarity index 100% rename from src/test/hive/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b rename to shark/src/test/hive/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b diff --git a/src/test/hive/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 b/shark/src/test/hive/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 similarity index 100% rename from src/test/hive/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 rename to shark/src/test/hive/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 diff --git a/src/test/hive/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 b/shark/src/test/hive/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 similarity index 100% rename from src/test/hive/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 rename to shark/src/test/hive/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 diff --git a/src/test/hive/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 b/shark/src/test/hive/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 similarity index 100% rename from src/test/hive/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 rename to shark/src/test/hive/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 diff --git a/src/test/hive/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa b/shark/src/test/hive/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa similarity index 100% rename from src/test/hive/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa rename to shark/src/test/hive/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa diff --git a/src/test/hive/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 b/shark/src/test/hive/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 similarity index 100% rename from src/test/hive/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 rename to shark/src/test/hive/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 diff --git a/src/test/hive/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 b/shark/src/test/hive/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 similarity index 100% rename from src/test/hive/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 rename to shark/src/test/hive/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 diff --git a/src/test/hive/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 b/shark/src/test/hive/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 similarity index 100% rename from src/test/hive/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 rename to shark/src/test/hive/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 diff --git a/src/test/hive/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 b/shark/src/test/hive/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 similarity index 100% rename from src/test/hive/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 rename to shark/src/test/hive/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 diff --git a/src/test/hive/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 b/shark/src/test/hive/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 similarity index 100% rename from src/test/hive/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 rename to shark/src/test/hive/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 diff --git a/src/test/hive/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 b/shark/src/test/hive/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 similarity index 100% rename from src/test/hive/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 rename to shark/src/test/hive/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 diff --git a/src/test/hive/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 b/shark/src/test/hive/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 similarity index 100% rename from src/test/hive/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 rename to shark/src/test/hive/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 diff --git a/src/test/hive/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 b/shark/src/test/hive/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 similarity index 100% rename from src/test/hive/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 rename to shark/src/test/hive/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 diff --git a/src/test/hive/golden/insert_into2-8-452111285dda40205ee587de8e972896 b/shark/src/test/hive/golden/insert_into2-8-452111285dda40205ee587de8e972896 similarity index 100% rename from src/test/hive/golden/insert_into2-8-452111285dda40205ee587de8e972896 rename to shark/src/test/hive/golden/insert_into2-8-452111285dda40205ee587de8e972896 diff --git a/src/test/hive/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 b/shark/src/test/hive/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 similarity index 100% rename from src/test/hive/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 rename to shark/src/test/hive/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 diff --git a/src/test/hive/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 b/shark/src/test/hive/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 similarity index 100% rename from src/test/hive/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 rename to shark/src/test/hive/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 diff --git a/src/test/hive/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 b/shark/src/test/hive/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 similarity index 100% rename from src/test/hive/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 rename to shark/src/test/hive/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 diff --git a/src/test/hive/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 b/shark/src/test/hive/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 similarity index 100% rename from src/test/hive/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 rename to shark/src/test/hive/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 diff --git a/src/test/hive/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 b/shark/src/test/hive/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 similarity index 100% rename from src/test/hive/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 rename to shark/src/test/hive/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 diff --git a/src/test/hive/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 b/shark/src/test/hive/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 similarity index 100% rename from src/test/hive/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 rename to shark/src/test/hive/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 diff --git a/src/test/hive/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 b/shark/src/test/hive/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 similarity index 100% rename from src/test/hive/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 rename to shark/src/test/hive/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 diff --git a/src/test/hive/golden/insert_into3-2-e51c25bae2408422a56826a263479468 b/shark/src/test/hive/golden/insert_into3-2-e51c25bae2408422a56826a263479468 similarity index 100% rename from src/test/hive/golden/insert_into3-2-e51c25bae2408422a56826a263479468 rename to shark/src/test/hive/golden/insert_into3-2-e51c25bae2408422a56826a263479468 diff --git a/src/test/hive/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d b/shark/src/test/hive/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d similarity index 100% rename from src/test/hive/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d rename to shark/src/test/hive/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d diff --git a/src/test/hive/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd b/shark/src/test/hive/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd similarity index 100% rename from src/test/hive/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd rename to shark/src/test/hive/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd diff --git a/src/test/hive/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 b/shark/src/test/hive/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 similarity index 100% rename from src/test/hive/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 rename to shark/src/test/hive/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 diff --git a/src/test/hive/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 b/shark/src/test/hive/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 similarity index 100% rename from src/test/hive/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 rename to shark/src/test/hive/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 diff --git a/src/test/hive/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 b/shark/src/test/hive/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 similarity index 100% rename from src/test/hive/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 rename to shark/src/test/hive/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 diff --git a/src/test/hive/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 b/shark/src/test/hive/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 similarity index 100% rename from src/test/hive/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 rename to shark/src/test/hive/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 diff --git a/src/test/hive/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 b/shark/src/test/hive/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 similarity index 100% rename from src/test/hive/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 rename to shark/src/test/hive/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 diff --git a/src/test/hive/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 b/shark/src/test/hive/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 similarity index 100% rename from src/test/hive/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 rename to shark/src/test/hive/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 diff --git a/src/test/hive/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 b/shark/src/test/hive/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 similarity index 100% rename from src/test/hive/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 rename to shark/src/test/hive/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 diff --git a/src/test/hive/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 b/shark/src/test/hive/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 similarity index 100% rename from src/test/hive/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 rename to shark/src/test/hive/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 diff --git a/src/test/hive/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa b/shark/src/test/hive/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa similarity index 100% rename from src/test/hive/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa rename to shark/src/test/hive/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa diff --git a/src/test/hive/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 b/shark/src/test/hive/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 similarity index 100% rename from src/test/hive/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 rename to shark/src/test/hive/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 diff --git a/src/test/hive/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b b/shark/src/test/hive/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b similarity index 100% rename from src/test/hive/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b rename to shark/src/test/hive/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b diff --git a/src/test/hive/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 b/shark/src/test/hive/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 similarity index 100% rename from src/test/hive/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 rename to shark/src/test/hive/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 diff --git a/src/test/hive/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 b/shark/src/test/hive/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 similarity index 100% rename from src/test/hive/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 rename to shark/src/test/hive/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 diff --git a/src/test/hive/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 b/shark/src/test/hive/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 similarity index 100% rename from src/test/hive/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 rename to shark/src/test/hive/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 diff --git a/src/test/hive/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b b/shark/src/test/hive/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b similarity index 100% rename from src/test/hive/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b rename to shark/src/test/hive/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b diff --git a/src/test/hive/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a b/shark/src/test/hive/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a similarity index 100% rename from src/test/hive/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a rename to shark/src/test/hive/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a diff --git a/src/test/hive/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 b/shark/src/test/hive/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 similarity index 100% rename from src/test/hive/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 rename to shark/src/test/hive/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 diff --git a/src/test/hive/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 b/shark/src/test/hive/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 similarity index 100% rename from src/test/hive/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 rename to shark/src/test/hive/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 diff --git a/src/test/hive/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 b/shark/src/test/hive/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 similarity index 100% rename from src/test/hive/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 rename to shark/src/test/hive/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 diff --git a/src/test/hive/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 b/shark/src/test/hive/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 similarity index 100% rename from src/test/hive/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 rename to shark/src/test/hive/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 diff --git a/src/test/hive/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 b/shark/src/test/hive/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 similarity index 100% rename from src/test/hive/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 rename to shark/src/test/hive/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 diff --git a/src/test/hive/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 b/shark/src/test/hive/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 similarity index 100% rename from src/test/hive/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 rename to shark/src/test/hive/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 diff --git a/src/test/hive/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c b/shark/src/test/hive/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c similarity index 100% rename from src/test/hive/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c rename to shark/src/test/hive/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c diff --git a/src/test/hive/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 b/shark/src/test/hive/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 similarity index 100% rename from src/test/hive/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 rename to shark/src/test/hive/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 diff --git a/src/test/hive/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d b/shark/src/test/hive/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d similarity index 100% rename from src/test/hive/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d rename to shark/src/test/hive/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d diff --git a/src/test/hive/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 b/shark/src/test/hive/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 similarity index 100% rename from src/test/hive/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 rename to shark/src/test/hive/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 diff --git a/src/test/hive/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 b/shark/src/test/hive/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 similarity index 100% rename from src/test/hive/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 rename to shark/src/test/hive/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 diff --git a/src/test/hive/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 b/shark/src/test/hive/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 similarity index 100% rename from src/test/hive/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 rename to shark/src/test/hive/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 diff --git a/src/test/hive/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 b/shark/src/test/hive/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 similarity index 100% rename from src/test/hive/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 rename to shark/src/test/hive/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 diff --git a/src/test/hive/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 b/shark/src/test/hive/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 similarity index 100% rename from src/test/hive/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 rename to shark/src/test/hive/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 diff --git a/src/test/hive/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c b/shark/src/test/hive/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c similarity index 100% rename from src/test/hive/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c rename to shark/src/test/hive/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c diff --git a/src/test/hive/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 b/shark/src/test/hive/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 similarity index 100% rename from src/test/hive/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 rename to shark/src/test/hive/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 diff --git a/src/test/hive/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 b/shark/src/test/hive/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 similarity index 100% rename from src/test/hive/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 rename to shark/src/test/hive/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 diff --git a/src/test/hive/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 b/shark/src/test/hive/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 similarity index 100% rename from src/test/hive/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 rename to shark/src/test/hive/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 diff --git a/src/test/hive/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 b/shark/src/test/hive/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 similarity index 100% rename from src/test/hive/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 rename to shark/src/test/hive/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 diff --git a/src/test/hive/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f b/shark/src/test/hive/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f similarity index 100% rename from src/test/hive/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f rename to shark/src/test/hive/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f diff --git a/src/test/hive/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 b/shark/src/test/hive/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 similarity index 100% rename from src/test/hive/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 rename to shark/src/test/hive/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 diff --git a/src/test/hive/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 b/shark/src/test/hive/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 similarity index 100% rename from src/test/hive/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 rename to shark/src/test/hive/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 diff --git a/src/test/hive/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 b/shark/src/test/hive/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 similarity index 100% rename from src/test/hive/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 rename to shark/src/test/hive/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 diff --git a/src/test/hive/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba b/shark/src/test/hive/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba similarity index 100% rename from src/test/hive/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba rename to shark/src/test/hive/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba diff --git a/src/test/hive/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 b/shark/src/test/hive/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 similarity index 100% rename from src/test/hive/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 rename to shark/src/test/hive/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 diff --git a/src/test/hive/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 b/shark/src/test/hive/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 similarity index 100% rename from src/test/hive/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 rename to shark/src/test/hive/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 diff --git a/src/test/hive/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 b/shark/src/test/hive/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 similarity index 100% rename from src/test/hive/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 rename to shark/src/test/hive/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 diff --git a/src/test/hive/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/shark/src/test/hive/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d similarity index 100% rename from src/test/hive/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d rename to shark/src/test/hive/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d diff --git a/src/test/hive/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 b/shark/src/test/hive/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 similarity index 100% rename from src/test/hive/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 rename to shark/src/test/hive/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 diff --git a/src/test/hive/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/shark/src/test/hive/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d similarity index 100% rename from src/test/hive/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d rename to shark/src/test/hive/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d diff --git a/src/test/hive/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f b/shark/src/test/hive/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f similarity index 100% rename from src/test/hive/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f rename to shark/src/test/hive/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f diff --git a/src/test/hive/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 b/shark/src/test/hive/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 similarity index 100% rename from src/test/hive/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 rename to shark/src/test/hive/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 diff --git a/src/test/hive/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 b/shark/src/test/hive/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 similarity index 100% rename from src/test/hive/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 rename to shark/src/test/hive/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 diff --git a/src/test/hive/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 b/shark/src/test/hive/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 similarity index 100% rename from src/test/hive/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 rename to shark/src/test/hive/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 diff --git a/src/test/hive/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 b/shark/src/test/hive/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 similarity index 100% rename from src/test/hive/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 rename to shark/src/test/hive/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 diff --git a/src/test/hive/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab b/shark/src/test/hive/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab similarity index 100% rename from src/test/hive/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab rename to shark/src/test/hive/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab diff --git a/src/test/hive/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 b/shark/src/test/hive/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 similarity index 100% rename from src/test/hive/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 rename to shark/src/test/hive/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 diff --git a/src/test/hive/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a b/shark/src/test/hive/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a similarity index 100% rename from src/test/hive/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a rename to shark/src/test/hive/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a diff --git a/src/test/hive/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b b/shark/src/test/hive/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b similarity index 100% rename from src/test/hive/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b rename to shark/src/test/hive/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b diff --git a/src/test/hive/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 b/shark/src/test/hive/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 similarity index 100% rename from src/test/hive/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 rename to shark/src/test/hive/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 diff --git a/src/test/hive/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 b/shark/src/test/hive/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 similarity index 100% rename from src/test/hive/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 rename to shark/src/test/hive/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 diff --git a/src/test/hive/golden/join0-1-83f5e512d6058a47f92b5218781f5070 b/shark/src/test/hive/golden/join0-1-83f5e512d6058a47f92b5218781f5070 similarity index 100% rename from src/test/hive/golden/join0-1-83f5e512d6058a47f92b5218781f5070 rename to shark/src/test/hive/golden/join0-1-83f5e512d6058a47f92b5218781f5070 diff --git a/src/test/hive/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 b/shark/src/test/hive/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 similarity index 100% rename from src/test/hive/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 rename to shark/src/test/hive/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 diff --git a/src/test/hive/golden/join1-0-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/join1-0-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/join1-0-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/join1-0-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e b/shark/src/test/hive/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e similarity index 100% rename from src/test/hive/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e rename to shark/src/test/hive/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e diff --git a/src/test/hive/golden/join1-2-7b07671864bbfdc627ee794932e32b1e b/shark/src/test/hive/golden/join1-2-7b07671864bbfdc627ee794932e32b1e similarity index 100% rename from src/test/hive/golden/join1-2-7b07671864bbfdc627ee794932e32b1e rename to shark/src/test/hive/golden/join1-2-7b07671864bbfdc627ee794932e32b1e diff --git a/src/test/hive/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 b/shark/src/test/hive/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 similarity index 100% rename from src/test/hive/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 rename to shark/src/test/hive/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 diff --git a/src/test/hive/golden/join1-4-35c7611180562dcd9dab834f41654095 b/shark/src/test/hive/golden/join1-4-35c7611180562dcd9dab834f41654095 similarity index 100% rename from src/test/hive/golden/join1-4-35c7611180562dcd9dab834f41654095 rename to shark/src/test/hive/golden/join1-4-35c7611180562dcd9dab834f41654095 diff --git a/src/test/hive/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 b/shark/src/test/hive/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 similarity index 100% rename from src/test/hive/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 rename to shark/src/test/hive/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 diff --git a/src/test/hive/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d b/shark/src/test/hive/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d similarity index 100% rename from src/test/hive/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d rename to shark/src/test/hive/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d diff --git a/src/test/hive/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 b/shark/src/test/hive/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 similarity index 100% rename from src/test/hive/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 rename to shark/src/test/hive/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 diff --git a/src/test/hive/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 b/shark/src/test/hive/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 similarity index 100% rename from src/test/hive/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 rename to shark/src/test/hive/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 diff --git a/src/test/hive/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 b/shark/src/test/hive/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 similarity index 100% rename from src/test/hive/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 rename to shark/src/test/hive/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 diff --git a/src/test/hive/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f b/shark/src/test/hive/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f similarity index 100% rename from src/test/hive/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f rename to shark/src/test/hive/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f diff --git a/src/test/hive/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 b/shark/src/test/hive/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 similarity index 100% rename from src/test/hive/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 rename to shark/src/test/hive/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 diff --git a/src/test/hive/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 b/shark/src/test/hive/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 similarity index 100% rename from src/test/hive/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 rename to shark/src/test/hive/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 diff --git a/src/test/hive/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 b/shark/src/test/hive/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 similarity index 100% rename from src/test/hive/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 rename to shark/src/test/hive/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 diff --git a/src/test/hive/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e b/shark/src/test/hive/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e similarity index 100% rename from src/test/hive/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e rename to shark/src/test/hive/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e diff --git a/src/test/hive/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 b/shark/src/test/hive/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 similarity index 100% rename from src/test/hive/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 rename to shark/src/test/hive/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 diff --git a/src/test/hive/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 b/shark/src/test/hive/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 similarity index 100% rename from src/test/hive/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 rename to shark/src/test/hive/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 diff --git a/src/test/hive/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 b/shark/src/test/hive/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 similarity index 100% rename from src/test/hive/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 rename to shark/src/test/hive/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 diff --git a/src/test/hive/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 b/shark/src/test/hive/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 similarity index 100% rename from src/test/hive/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 rename to shark/src/test/hive/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 diff --git a/src/test/hive/golden/join14-6-de39302191b63d7aa8f92885b089fe2 b/shark/src/test/hive/golden/join14-6-de39302191b63d7aa8f92885b089fe2 similarity index 100% rename from src/test/hive/golden/join14-6-de39302191b63d7aa8f92885b089fe2 rename to shark/src/test/hive/golden/join14-6-de39302191b63d7aa8f92885b089fe2 diff --git a/src/test/hive/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/shark/src/test/hive/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 similarity index 100% rename from src/test/hive/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 rename to shark/src/test/hive/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 diff --git a/src/test/hive/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 b/shark/src/test/hive/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 similarity index 100% rename from src/test/hive/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 rename to shark/src/test/hive/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 diff --git a/src/test/hive/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f b/shark/src/test/hive/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f similarity index 100% rename from src/test/hive/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f rename to shark/src/test/hive/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f diff --git a/src/test/hive/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 b/shark/src/test/hive/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 similarity index 100% rename from src/test/hive/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 rename to shark/src/test/hive/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 diff --git a/src/test/hive/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 b/shark/src/test/hive/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 similarity index 100% rename from src/test/hive/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 rename to shark/src/test/hive/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 diff --git a/src/test/hive/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 b/shark/src/test/hive/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 similarity index 100% rename from src/test/hive/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 rename to shark/src/test/hive/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 diff --git a/src/test/hive/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/shark/src/test/hive/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 similarity index 100% rename from src/test/hive/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 rename to shark/src/test/hive/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 diff --git a/src/test/hive/golden/join15-0-930db2823512f7f8f34cac104f2162e0 b/shark/src/test/hive/golden/join15-0-930db2823512f7f8f34cac104f2162e0 similarity index 100% rename from src/test/hive/golden/join15-0-930db2823512f7f8f34cac104f2162e0 rename to shark/src/test/hive/golden/join15-0-930db2823512f7f8f34cac104f2162e0 diff --git a/src/test/hive/golden/join15-1-81d76d3bf59889b07b413b6f88772667 b/shark/src/test/hive/golden/join15-1-81d76d3bf59889b07b413b6f88772667 similarity index 100% rename from src/test/hive/golden/join15-1-81d76d3bf59889b07b413b6f88772667 rename to shark/src/test/hive/golden/join15-1-81d76d3bf59889b07b413b6f88772667 diff --git a/src/test/hive/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c b/shark/src/test/hive/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c similarity index 100% rename from src/test/hive/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c rename to shark/src/test/hive/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c diff --git a/src/test/hive/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 b/shark/src/test/hive/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 similarity index 100% rename from src/test/hive/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 rename to shark/src/test/hive/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 diff --git a/src/test/hive/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b b/shark/src/test/hive/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b similarity index 100% rename from src/test/hive/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b rename to shark/src/test/hive/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b diff --git a/src/test/hive/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 b/shark/src/test/hive/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 similarity index 100% rename from src/test/hive/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 rename to shark/src/test/hive/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 diff --git a/src/test/hive/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/join18-0-269843197b598569f4b50cbe911960a5 b/shark/src/test/hive/golden/join18-0-269843197b598569f4b50cbe911960a5 similarity index 100% rename from src/test/hive/golden/join18-0-269843197b598569f4b50cbe911960a5 rename to shark/src/test/hive/golden/join18-0-269843197b598569f4b50cbe911960a5 diff --git a/src/test/hive/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 b/shark/src/test/hive/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 similarity index 100% rename from src/test/hive/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 rename to shark/src/test/hive/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 diff --git a/src/test/hive/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 b/shark/src/test/hive/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 similarity index 100% rename from src/test/hive/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 rename to shark/src/test/hive/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 diff --git a/src/test/hive/golden/join19-1-7e7d1f07c34dd4be5425264196201766 b/shark/src/test/hive/golden/join19-1-7e7d1f07c34dd4be5425264196201766 similarity index 100% rename from src/test/hive/golden/join19-1-7e7d1f07c34dd4be5425264196201766 rename to shark/src/test/hive/golden/join19-1-7e7d1f07c34dd4be5425264196201766 diff --git a/src/test/hive/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca b/shark/src/test/hive/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca similarity index 100% rename from src/test/hive/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca rename to shark/src/test/hive/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca diff --git a/src/test/hive/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c b/shark/src/test/hive/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c similarity index 100% rename from src/test/hive/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c rename to shark/src/test/hive/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c diff --git a/src/test/hive/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a b/shark/src/test/hive/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a similarity index 100% rename from src/test/hive/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a rename to shark/src/test/hive/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a diff --git a/src/test/hive/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 b/shark/src/test/hive/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 similarity index 100% rename from src/test/hive/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 rename to shark/src/test/hive/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 diff --git a/src/test/hive/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e b/shark/src/test/hive/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e similarity index 100% rename from src/test/hive/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e rename to shark/src/test/hive/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e diff --git a/src/test/hive/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 b/shark/src/test/hive/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 similarity index 100% rename from src/test/hive/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 rename to shark/src/test/hive/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 diff --git a/src/test/hive/golden/join20-2-222655137ca9242f063c7717acbe4c65 b/shark/src/test/hive/golden/join20-2-222655137ca9242f063c7717acbe4c65 similarity index 100% rename from src/test/hive/golden/join20-2-222655137ca9242f063c7717acbe4c65 rename to shark/src/test/hive/golden/join20-2-222655137ca9242f063c7717acbe4c65 diff --git a/src/test/hive/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 b/shark/src/test/hive/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 similarity index 100% rename from src/test/hive/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 rename to shark/src/test/hive/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 diff --git a/src/test/hive/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 b/shark/src/test/hive/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 similarity index 100% rename from src/test/hive/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 rename to shark/src/test/hive/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 diff --git a/src/test/hive/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f b/shark/src/test/hive/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f similarity index 100% rename from src/test/hive/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f rename to shark/src/test/hive/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f diff --git a/src/test/hive/golden/join22-0-97081b5f73052a612fb391a57ad9b7b b/shark/src/test/hive/golden/join22-0-97081b5f73052a612fb391a57ad9b7b similarity index 100% rename from src/test/hive/golden/join22-0-97081b5f73052a612fb391a57ad9b7b rename to shark/src/test/hive/golden/join22-0-97081b5f73052a612fb391a57ad9b7b diff --git a/src/test/hive/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d b/shark/src/test/hive/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d similarity index 100% rename from src/test/hive/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d rename to shark/src/test/hive/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d diff --git a/src/test/hive/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 b/shark/src/test/hive/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 similarity index 100% rename from src/test/hive/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 rename to shark/src/test/hive/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 diff --git a/src/test/hive/golden/join24-0-721dfa03bfea05e55506c571b6c3585b b/shark/src/test/hive/golden/join24-0-721dfa03bfea05e55506c571b6c3585b similarity index 100% rename from src/test/hive/golden/join24-0-721dfa03bfea05e55506c571b6c3585b rename to shark/src/test/hive/golden/join24-0-721dfa03bfea05e55506c571b6c3585b diff --git a/src/test/hive/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d b/shark/src/test/hive/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d similarity index 100% rename from src/test/hive/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d rename to shark/src/test/hive/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d diff --git a/src/test/hive/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 b/shark/src/test/hive/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 similarity index 100% rename from src/test/hive/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 rename to shark/src/test/hive/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 diff --git a/src/test/hive/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 b/shark/src/test/hive/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 similarity index 100% rename from src/test/hive/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 rename to shark/src/test/hive/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 diff --git a/src/test/hive/golden/join25-1-360b3676605ccb3d89cf555342db87af b/shark/src/test/hive/golden/join25-1-360b3676605ccb3d89cf555342db87af similarity index 100% rename from src/test/hive/golden/join25-1-360b3676605ccb3d89cf555342db87af rename to shark/src/test/hive/golden/join25-1-360b3676605ccb3d89cf555342db87af diff --git a/src/test/hive/golden/join25-2-ef930bed933341636c71622e6f89e9e9 b/shark/src/test/hive/golden/join25-2-ef930bed933341636c71622e6f89e9e9 similarity index 100% rename from src/test/hive/golden/join25-2-ef930bed933341636c71622e6f89e9e9 rename to shark/src/test/hive/golden/join25-2-ef930bed933341636c71622e6f89e9e9 diff --git a/src/test/hive/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 b/shark/src/test/hive/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 similarity index 100% rename from src/test/hive/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 rename to shark/src/test/hive/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 diff --git a/src/test/hive/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join26-0-775b1af8eae9b2250052be1b72b4086 b/shark/src/test/hive/golden/join26-0-775b1af8eae9b2250052be1b72b4086 similarity index 100% rename from src/test/hive/golden/join26-0-775b1af8eae9b2250052be1b72b4086 rename to shark/src/test/hive/golden/join26-0-775b1af8eae9b2250052be1b72b4086 diff --git a/src/test/hive/golden/join26-1-10fbe6c99e93cfeafcd454261744728e b/shark/src/test/hive/golden/join26-1-10fbe6c99e93cfeafcd454261744728e similarity index 100% rename from src/test/hive/golden/join26-1-10fbe6c99e93cfeafcd454261744728e rename to shark/src/test/hive/golden/join26-1-10fbe6c99e93cfeafcd454261744728e diff --git a/src/test/hive/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d b/shark/src/test/hive/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d similarity index 100% rename from src/test/hive/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d rename to shark/src/test/hive/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d diff --git a/src/test/hive/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join27-0-360b3676605ccb3d89cf555342db87af b/shark/src/test/hive/golden/join27-0-360b3676605ccb3d89cf555342db87af similarity index 100% rename from src/test/hive/golden/join27-0-360b3676605ccb3d89cf555342db87af rename to shark/src/test/hive/golden/join27-0-360b3676605ccb3d89cf555342db87af diff --git a/src/test/hive/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 b/shark/src/test/hive/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 similarity index 100% rename from src/test/hive/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 rename to shark/src/test/hive/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 diff --git a/src/test/hive/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 b/shark/src/test/hive/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 similarity index 100% rename from src/test/hive/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 rename to shark/src/test/hive/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 diff --git a/src/test/hive/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 b/shark/src/test/hive/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 similarity index 100% rename from src/test/hive/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 rename to shark/src/test/hive/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 diff --git a/src/test/hive/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 b/shark/src/test/hive/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 similarity index 100% rename from src/test/hive/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 rename to shark/src/test/hive/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 diff --git a/src/test/hive/golden/join28-1-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/join28-1-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/join28-1-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/join28-1-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/join28-3-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/join28-3-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/join28-3-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/join28-3-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 b/shark/src/test/hive/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 similarity index 100% rename from src/test/hive/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 rename to shark/src/test/hive/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 diff --git a/src/test/hive/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 b/shark/src/test/hive/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 similarity index 100% rename from src/test/hive/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 rename to shark/src/test/hive/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 diff --git a/src/test/hive/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join29-0-e78402f7585b17c76c32790571d749eb b/shark/src/test/hive/golden/join29-0-e78402f7585b17c76c32790571d749eb similarity index 100% rename from src/test/hive/golden/join29-0-e78402f7585b17c76c32790571d749eb rename to shark/src/test/hive/golden/join29-0-e78402f7585b17c76c32790571d749eb diff --git a/src/test/hive/golden/join29-1-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/join29-1-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/join29-1-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/join29-1-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/join29-3-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/join29-3-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/join29-3-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/join29-3-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/join29-4-b0524de7985cdb7b57fd995721654856 b/shark/src/test/hive/golden/join29-4-b0524de7985cdb7b57fd995721654856 similarity index 100% rename from src/test/hive/golden/join29-4-b0524de7985cdb7b57fd995721654856 rename to shark/src/test/hive/golden/join29-4-b0524de7985cdb7b57fd995721654856 diff --git a/src/test/hive/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 b/shark/src/test/hive/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 similarity index 100% rename from src/test/hive/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 rename to shark/src/test/hive/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 diff --git a/src/test/hive/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 b/shark/src/test/hive/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 similarity index 100% rename from src/test/hive/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 rename to shark/src/test/hive/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 diff --git a/src/test/hive/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 b/shark/src/test/hive/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 similarity index 100% rename from src/test/hive/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 rename to shark/src/test/hive/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 diff --git a/src/test/hive/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 b/shark/src/test/hive/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 similarity index 100% rename from src/test/hive/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 rename to shark/src/test/hive/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 diff --git a/src/test/hive/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d b/shark/src/test/hive/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d similarity index 100% rename from src/test/hive/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d rename to shark/src/test/hive/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d diff --git a/src/test/hive/golden/join30-2-194edec0592cf441617ca7caf9756baa b/shark/src/test/hive/golden/join30-2-194edec0592cf441617ca7caf9756baa similarity index 100% rename from src/test/hive/golden/join30-2-194edec0592cf441617ca7caf9756baa rename to shark/src/test/hive/golden/join30-2-194edec0592cf441617ca7caf9756baa diff --git a/src/test/hive/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join31-0-619db2f9dd69aa236aa804cced726c9a b/shark/src/test/hive/golden/join31-0-619db2f9dd69aa236aa804cced726c9a similarity index 100% rename from src/test/hive/golden/join31-0-619db2f9dd69aa236aa804cced726c9a rename to shark/src/test/hive/golden/join31-0-619db2f9dd69aa236aa804cced726c9a diff --git a/src/test/hive/golden/join31-1-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/join31-1-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/join31-1-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/join31-1-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/join31-3-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/join31-3-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/join31-3-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/join31-3-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 b/shark/src/test/hive/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 similarity index 100% rename from src/test/hive/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 rename to shark/src/test/hive/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 diff --git a/src/test/hive/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 b/shark/src/test/hive/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 similarity index 100% rename from src/test/hive/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 rename to shark/src/test/hive/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 diff --git a/src/test/hive/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join32-0-775b1af8eae9b2250052be1b72b4086 b/shark/src/test/hive/golden/join32-0-775b1af8eae9b2250052be1b72b4086 similarity index 100% rename from src/test/hive/golden/join32-0-775b1af8eae9b2250052be1b72b4086 rename to shark/src/test/hive/golden/join32-0-775b1af8eae9b2250052be1b72b4086 diff --git a/src/test/hive/golden/join32-1-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/join32-1-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/join32-1-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/join32-1-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/join32-3-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/join32-3-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/join32-3-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/join32-3-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb b/shark/src/test/hive/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb similarity index 100% rename from src/test/hive/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb rename to shark/src/test/hive/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb diff --git a/src/test/hive/golden/join32-5-425c5c35b23c42254dabec49aa6613bf b/shark/src/test/hive/golden/join32-5-425c5c35b23c42254dabec49aa6613bf similarity index 100% rename from src/test/hive/golden/join32-5-425c5c35b23c42254dabec49aa6613bf rename to shark/src/test/hive/golden/join32-5-425c5c35b23c42254dabec49aa6613bf diff --git a/src/test/hive/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 b/shark/src/test/hive/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 similarity index 100% rename from src/test/hive/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 rename to shark/src/test/hive/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 diff --git a/src/test/hive/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 b/shark/src/test/hive/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 similarity index 100% rename from src/test/hive/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 rename to shark/src/test/hive/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 diff --git a/src/test/hive/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/shark/src/test/hive/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a similarity index 100% rename from src/test/hive/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a rename to shark/src/test/hive/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a diff --git a/src/test/hive/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca b/shark/src/test/hive/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca similarity index 100% rename from src/test/hive/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca rename to shark/src/test/hive/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca diff --git a/src/test/hive/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 b/shark/src/test/hive/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 similarity index 100% rename from src/test/hive/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 rename to shark/src/test/hive/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 diff --git a/src/test/hive/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 b/shark/src/test/hive/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 similarity index 100% rename from src/test/hive/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 rename to shark/src/test/hive/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 diff --git a/src/test/hive/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a b/shark/src/test/hive/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a similarity index 100% rename from src/test/hive/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a rename to shark/src/test/hive/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a diff --git a/src/test/hive/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e b/shark/src/test/hive/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e similarity index 100% rename from src/test/hive/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e rename to shark/src/test/hive/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e diff --git a/src/test/hive/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 b/shark/src/test/hive/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 similarity index 100% rename from src/test/hive/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 rename to shark/src/test/hive/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 diff --git a/src/test/hive/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 b/shark/src/test/hive/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 similarity index 100% rename from src/test/hive/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 rename to shark/src/test/hive/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 diff --git a/src/test/hive/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d b/shark/src/test/hive/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d similarity index 100% rename from src/test/hive/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d rename to shark/src/test/hive/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d diff --git a/src/test/hive/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 b/shark/src/test/hive/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 similarity index 100% rename from src/test/hive/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 rename to shark/src/test/hive/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 diff --git a/src/test/hive/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 b/shark/src/test/hive/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 similarity index 100% rename from src/test/hive/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 rename to shark/src/test/hive/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 diff --git a/src/test/hive/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 b/shark/src/test/hive/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 similarity index 100% rename from src/test/hive/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 rename to shark/src/test/hive/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 diff --git a/src/test/hive/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 b/shark/src/test/hive/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 similarity index 100% rename from src/test/hive/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 rename to shark/src/test/hive/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 diff --git a/src/test/hive/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 b/shark/src/test/hive/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 similarity index 100% rename from src/test/hive/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 rename to shark/src/test/hive/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 diff --git a/src/test/hive/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb b/shark/src/test/hive/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb similarity index 100% rename from src/test/hive/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb rename to shark/src/test/hive/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb diff --git a/src/test/hive/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf b/shark/src/test/hive/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf similarity index 100% rename from src/test/hive/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf rename to shark/src/test/hive/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf diff --git a/src/test/hive/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/shark/src/test/hive/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a similarity index 100% rename from src/test/hive/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a rename to shark/src/test/hive/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a diff --git a/src/test/hive/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b b/shark/src/test/hive/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b similarity index 100% rename from src/test/hive/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b rename to shark/src/test/hive/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b diff --git a/src/test/hive/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 b/shark/src/test/hive/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 similarity index 100% rename from src/test/hive/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 rename to shark/src/test/hive/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 diff --git a/src/test/hive/golden/join33-0-775b1af8eae9b2250052be1b72b4086 b/shark/src/test/hive/golden/join33-0-775b1af8eae9b2250052be1b72b4086 similarity index 100% rename from src/test/hive/golden/join33-0-775b1af8eae9b2250052be1b72b4086 rename to shark/src/test/hive/golden/join33-0-775b1af8eae9b2250052be1b72b4086 diff --git a/src/test/hive/golden/join33-1-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/join33-1-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/join33-1-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/join33-1-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/join33-3-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/join33-3-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/join33-3-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/join33-3-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb b/shark/src/test/hive/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb similarity index 100% rename from src/test/hive/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb rename to shark/src/test/hive/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb diff --git a/src/test/hive/golden/join33-5-425c5c35b23c42254dabec49aa6613bf b/shark/src/test/hive/golden/join33-5-425c5c35b23c42254dabec49aa6613bf similarity index 100% rename from src/test/hive/golden/join33-5-425c5c35b23c42254dabec49aa6613bf rename to shark/src/test/hive/golden/join33-5-425c5c35b23c42254dabec49aa6613bf diff --git a/src/test/hive/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join34-0-775b1af8eae9b2250052be1b72b4086 b/shark/src/test/hive/golden/join34-0-775b1af8eae9b2250052be1b72b4086 similarity index 100% rename from src/test/hive/golden/join34-0-775b1af8eae9b2250052be1b72b4086 rename to shark/src/test/hive/golden/join34-0-775b1af8eae9b2250052be1b72b4086 diff --git a/src/test/hive/golden/join34-1-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/join34-1-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/join34-1-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/join34-1-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/join34-3-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/join34-3-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/join34-3-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/join34-3-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/join34-4-51cc5d253befd784a9d466c0b402f47a b/shark/src/test/hive/golden/join34-4-51cc5d253befd784a9d466c0b402f47a similarity index 100% rename from src/test/hive/golden/join34-4-51cc5d253befd784a9d466c0b402f47a rename to shark/src/test/hive/golden/join34-4-51cc5d253befd784a9d466c0b402f47a diff --git a/src/test/hive/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 b/shark/src/test/hive/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 similarity index 100% rename from src/test/hive/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 rename to shark/src/test/hive/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 diff --git a/src/test/hive/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 b/shark/src/test/hive/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 similarity index 100% rename from src/test/hive/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 rename to shark/src/test/hive/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 diff --git a/src/test/hive/golden/join35-1-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/join35-1-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/join35-1-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/join35-1-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/join35-3-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/join35-3-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/join35-3-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/join35-3-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/join35-4-99cb704ccc2813400908a62571891dc7 b/shark/src/test/hive/golden/join35-4-99cb704ccc2813400908a62571891dc7 similarity index 100% rename from src/test/hive/golden/join35-4-99cb704ccc2813400908a62571891dc7 rename to shark/src/test/hive/golden/join35-4-99cb704ccc2813400908a62571891dc7 diff --git a/src/test/hive/golden/join35-5-60956f3307651237f52ffbe41c827c1c b/shark/src/test/hive/golden/join35-5-60956f3307651237f52ffbe41c827c1c similarity index 100% rename from src/test/hive/golden/join35-5-60956f3307651237f52ffbe41c827c1c rename to shark/src/test/hive/golden/join35-5-60956f3307651237f52ffbe41c827c1c diff --git a/src/test/hive/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 b/shark/src/test/hive/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 similarity index 100% rename from src/test/hive/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 rename to shark/src/test/hive/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 diff --git a/src/test/hive/golden/join36-1-3aba153d62042206c0f001ea74980da b/shark/src/test/hive/golden/join36-1-3aba153d62042206c0f001ea74980da similarity index 100% rename from src/test/hive/golden/join36-1-3aba153d62042206c0f001ea74980da rename to shark/src/test/hive/golden/join36-1-3aba153d62042206c0f001ea74980da diff --git a/src/test/hive/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 b/shark/src/test/hive/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 similarity index 100% rename from src/test/hive/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 rename to shark/src/test/hive/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 diff --git a/src/test/hive/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 b/shark/src/test/hive/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 similarity index 100% rename from src/test/hive/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 rename to shark/src/test/hive/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 diff --git a/src/test/hive/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 b/shark/src/test/hive/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 similarity index 100% rename from src/test/hive/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 rename to shark/src/test/hive/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 diff --git a/src/test/hive/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 b/shark/src/test/hive/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 similarity index 100% rename from src/test/hive/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 rename to shark/src/test/hive/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 diff --git a/src/test/hive/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 b/shark/src/test/hive/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 similarity index 100% rename from src/test/hive/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 rename to shark/src/test/hive/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 diff --git a/src/test/hive/golden/join36-7-171ede21302050a33468d95e329a38a3 b/shark/src/test/hive/golden/join36-7-171ede21302050a33468d95e329a38a3 similarity index 100% rename from src/test/hive/golden/join36-7-171ede21302050a33468d95e329a38a3 rename to shark/src/test/hive/golden/join36-7-171ede21302050a33468d95e329a38a3 diff --git a/src/test/hive/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 b/shark/src/test/hive/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 similarity index 100% rename from src/test/hive/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 rename to shark/src/test/hive/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 diff --git a/src/test/hive/golden/join37-1-360b3676605ccb3d89cf555342db87af b/shark/src/test/hive/golden/join37-1-360b3676605ccb3d89cf555342db87af similarity index 100% rename from src/test/hive/golden/join37-1-360b3676605ccb3d89cf555342db87af rename to shark/src/test/hive/golden/join37-1-360b3676605ccb3d89cf555342db87af diff --git a/src/test/hive/golden/join37-2-2310f2b8726137811543e04db98dd1db b/shark/src/test/hive/golden/join37-2-2310f2b8726137811543e04db98dd1db similarity index 100% rename from src/test/hive/golden/join37-2-2310f2b8726137811543e04db98dd1db rename to shark/src/test/hive/golden/join37-2-2310f2b8726137811543e04db98dd1db diff --git a/src/test/hive/golden/join37-3-c4b3a85965108ad7013ac3931598af09 b/shark/src/test/hive/golden/join37-3-c4b3a85965108ad7013ac3931598af09 similarity index 100% rename from src/test/hive/golden/join37-3-c4b3a85965108ad7013ac3931598af09 rename to shark/src/test/hive/golden/join37-3-c4b3a85965108ad7013ac3931598af09 diff --git a/src/test/hive/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join38-0-42e6ad674a5142d05258a2d29578e046 b/shark/src/test/hive/golden/join38-0-42e6ad674a5142d05258a2d29578e046 similarity index 100% rename from src/test/hive/golden/join38-0-42e6ad674a5142d05258a2d29578e046 rename to shark/src/test/hive/golden/join38-0-42e6ad674a5142d05258a2d29578e046 diff --git a/src/test/hive/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f b/shark/src/test/hive/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f similarity index 100% rename from src/test/hive/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f rename to shark/src/test/hive/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f diff --git a/src/test/hive/golden/join38-2-eacdb3417c4a563982c488812d654c9 b/shark/src/test/hive/golden/join38-2-eacdb3417c4a563982c488812d654c9 similarity index 100% rename from src/test/hive/golden/join38-2-eacdb3417c4a563982c488812d654c9 rename to shark/src/test/hive/golden/join38-2-eacdb3417c4a563982c488812d654c9 diff --git a/src/test/hive/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 b/shark/src/test/hive/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 similarity index 100% rename from src/test/hive/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 rename to shark/src/test/hive/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 diff --git a/src/test/hive/golden/join38-4-53d219706847e890de1dcd369563ebef b/shark/src/test/hive/golden/join38-4-53d219706847e890de1dcd369563ebef similarity index 100% rename from src/test/hive/golden/join38-4-53d219706847e890de1dcd369563ebef rename to shark/src/test/hive/golden/join38-4-53d219706847e890de1dcd369563ebef diff --git a/src/test/hive/golden/join39-0-54e9f7924abb40d1c601de908de6f25b b/shark/src/test/hive/golden/join39-0-54e9f7924abb40d1c601de908de6f25b similarity index 100% rename from src/test/hive/golden/join39-0-54e9f7924abb40d1c601de908de6f25b rename to shark/src/test/hive/golden/join39-0-54e9f7924abb40d1c601de908de6f25b diff --git a/src/test/hive/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 b/shark/src/test/hive/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 similarity index 100% rename from src/test/hive/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 rename to shark/src/test/hive/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 diff --git a/src/test/hive/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 b/shark/src/test/hive/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 similarity index 100% rename from src/test/hive/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 rename to shark/src/test/hive/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 diff --git a/src/test/hive/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join4-0-531e91e11b3891627c2675935fda14cd b/shark/src/test/hive/golden/join4-0-531e91e11b3891627c2675935fda14cd similarity index 100% rename from src/test/hive/golden/join4-0-531e91e11b3891627c2675935fda14cd rename to shark/src/test/hive/golden/join4-0-531e91e11b3891627c2675935fda14cd diff --git a/src/test/hive/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 b/shark/src/test/hive/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 similarity index 100% rename from src/test/hive/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 rename to shark/src/test/hive/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 diff --git a/src/test/hive/golden/join4-2-dc967001beb776f3a859e9360823c361 b/shark/src/test/hive/golden/join4-2-dc967001beb776f3a859e9360823c361 similarity index 100% rename from src/test/hive/golden/join4-2-dc967001beb776f3a859e9360823c361 rename to shark/src/test/hive/golden/join4-2-dc967001beb776f3a859e9360823c361 diff --git a/src/test/hive/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac b/shark/src/test/hive/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac similarity index 100% rename from src/test/hive/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac rename to shark/src/test/hive/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac diff --git a/src/test/hive/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 b/shark/src/test/hive/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 similarity index 100% rename from src/test/hive/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 rename to shark/src/test/hive/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 diff --git a/src/test/hive/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa b/shark/src/test/hive/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa similarity index 100% rename from src/test/hive/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa rename to shark/src/test/hive/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa diff --git a/src/test/hive/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 b/shark/src/test/hive/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 similarity index 100% rename from src/test/hive/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 rename to shark/src/test/hive/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 diff --git a/src/test/hive/golden/join40-12-aaddbef9069aba3ebeb310be74671cda b/shark/src/test/hive/golden/join40-12-aaddbef9069aba3ebeb310be74671cda similarity index 100% rename from src/test/hive/golden/join40-12-aaddbef9069aba3ebeb310be74671cda rename to shark/src/test/hive/golden/join40-12-aaddbef9069aba3ebeb310be74671cda diff --git a/src/test/hive/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 b/shark/src/test/hive/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 similarity index 100% rename from src/test/hive/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 rename to shark/src/test/hive/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 diff --git a/src/test/hive/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 b/shark/src/test/hive/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 similarity index 100% rename from src/test/hive/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 rename to shark/src/test/hive/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 diff --git a/src/test/hive/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 b/shark/src/test/hive/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 similarity index 100% rename from src/test/hive/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 rename to shark/src/test/hive/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 diff --git a/src/test/hive/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e b/shark/src/test/hive/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e similarity index 100% rename from src/test/hive/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e rename to shark/src/test/hive/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e diff --git a/src/test/hive/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 b/shark/src/test/hive/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 similarity index 100% rename from src/test/hive/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 rename to shark/src/test/hive/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 diff --git a/src/test/hive/golden/join40-7-222655137ca9242f063c7717acbe4c65 b/shark/src/test/hive/golden/join40-7-222655137ca9242f063c7717acbe4c65 similarity index 100% rename from src/test/hive/golden/join40-7-222655137ca9242f063c7717acbe4c65 rename to shark/src/test/hive/golden/join40-7-222655137ca9242f063c7717acbe4c65 diff --git a/src/test/hive/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 b/shark/src/test/hive/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 similarity index 100% rename from src/test/hive/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 rename to shark/src/test/hive/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 diff --git a/src/test/hive/golden/join40-9-6b7404af014eeb4de196577f14ef18cc b/shark/src/test/hive/golden/join40-9-6b7404af014eeb4de196577f14ef18cc similarity index 100% rename from src/test/hive/golden/join40-9-6b7404af014eeb4de196577f14ef18cc rename to shark/src/test/hive/golden/join40-9-6b7404af014eeb4de196577f14ef18cc diff --git a/src/test/hive/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 b/shark/src/test/hive/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 similarity index 100% rename from src/test/hive/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 rename to shark/src/test/hive/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 diff --git a/src/test/hive/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 b/shark/src/test/hive/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 similarity index 100% rename from src/test/hive/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 rename to shark/src/test/hive/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 diff --git a/src/test/hive/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a b/shark/src/test/hive/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a similarity index 100% rename from src/test/hive/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a rename to shark/src/test/hive/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a diff --git a/src/test/hive/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 b/shark/src/test/hive/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 similarity index 100% rename from src/test/hive/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 rename to shark/src/test/hive/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 diff --git a/src/test/hive/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e b/shark/src/test/hive/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e similarity index 100% rename from src/test/hive/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e rename to shark/src/test/hive/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e diff --git a/src/test/hive/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 b/shark/src/test/hive/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 similarity index 100% rename from src/test/hive/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 rename to shark/src/test/hive/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 diff --git a/src/test/hive/golden/join5-0-531e91e11b3891627c2675935fda14cd b/shark/src/test/hive/golden/join5-0-531e91e11b3891627c2675935fda14cd similarity index 100% rename from src/test/hive/golden/join5-0-531e91e11b3891627c2675935fda14cd rename to shark/src/test/hive/golden/join5-0-531e91e11b3891627c2675935fda14cd diff --git a/src/test/hive/golden/join5-1-76feff1282895f38b673f52acfd7429e b/shark/src/test/hive/golden/join5-1-76feff1282895f38b673f52acfd7429e similarity index 100% rename from src/test/hive/golden/join5-1-76feff1282895f38b673f52acfd7429e rename to shark/src/test/hive/golden/join5-1-76feff1282895f38b673f52acfd7429e diff --git a/src/test/hive/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 b/shark/src/test/hive/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 similarity index 100% rename from src/test/hive/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 rename to shark/src/test/hive/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 diff --git a/src/test/hive/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/join6-0-531e91e11b3891627c2675935fda14cd b/shark/src/test/hive/golden/join6-0-531e91e11b3891627c2675935fda14cd similarity index 100% rename from src/test/hive/golden/join6-0-531e91e11b3891627c2675935fda14cd rename to shark/src/test/hive/golden/join6-0-531e91e11b3891627c2675935fda14cd diff --git a/src/test/hive/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 b/shark/src/test/hive/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 similarity index 100% rename from src/test/hive/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 rename to shark/src/test/hive/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 diff --git a/src/test/hive/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 b/shark/src/test/hive/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 similarity index 100% rename from src/test/hive/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 rename to shark/src/test/hive/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 diff --git a/src/test/hive/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 b/shark/src/test/hive/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 similarity index 100% rename from src/test/hive/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 rename to shark/src/test/hive/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 diff --git a/src/test/hive/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 b/shark/src/test/hive/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 similarity index 100% rename from src/test/hive/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 rename to shark/src/test/hive/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 diff --git a/src/test/hive/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 b/shark/src/test/hive/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 similarity index 100% rename from src/test/hive/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 rename to shark/src/test/hive/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 diff --git a/src/test/hive/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/join8-0-531e91e11b3891627c2675935fda14cd b/shark/src/test/hive/golden/join8-0-531e91e11b3891627c2675935fda14cd similarity index 100% rename from src/test/hive/golden/join8-0-531e91e11b3891627c2675935fda14cd rename to shark/src/test/hive/golden/join8-0-531e91e11b3891627c2675935fda14cd diff --git a/src/test/hive/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 b/shark/src/test/hive/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 similarity index 100% rename from src/test/hive/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 rename to shark/src/test/hive/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 diff --git a/src/test/hive/golden/join8-2-f1196bca86a749375da35f134206a8ca b/shark/src/test/hive/golden/join8-2-f1196bca86a749375da35f134206a8ca similarity index 100% rename from src/test/hive/golden/join8-2-f1196bca86a749375da35f134206a8ca rename to shark/src/test/hive/golden/join8-2-f1196bca86a749375da35f134206a8ca diff --git a/src/test/hive/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a b/shark/src/test/hive/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a similarity index 100% rename from src/test/hive/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a rename to shark/src/test/hive/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a diff --git a/src/test/hive/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 b/shark/src/test/hive/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 similarity index 100% rename from src/test/hive/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 rename to shark/src/test/hive/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 diff --git a/src/test/hive/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 b/shark/src/test/hive/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 similarity index 100% rename from src/test/hive/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 rename to shark/src/test/hive/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 diff --git a/src/test/hive/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a b/shark/src/test/hive/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a similarity index 100% rename from src/test/hive/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a rename to shark/src/test/hive/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a diff --git a/src/test/hive/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e b/shark/src/test/hive/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e similarity index 100% rename from src/test/hive/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e rename to shark/src/test/hive/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e diff --git a/src/test/hive/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e b/shark/src/test/hive/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e similarity index 100% rename from src/test/hive/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e rename to shark/src/test/hive/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e diff --git a/src/test/hive/golden/join_1to1-12-6081eb976b4aef2132418510756a385b b/shark/src/test/hive/golden/join_1to1-12-6081eb976b4aef2132418510756a385b similarity index 100% rename from src/test/hive/golden/join_1to1-12-6081eb976b4aef2132418510756a385b rename to shark/src/test/hive/golden/join_1to1-12-6081eb976b4aef2132418510756a385b diff --git a/src/test/hive/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c b/shark/src/test/hive/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c similarity index 100% rename from src/test/hive/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c rename to shark/src/test/hive/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c diff --git a/src/test/hive/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 b/shark/src/test/hive/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 similarity index 100% rename from src/test/hive/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 rename to shark/src/test/hive/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 diff --git a/src/test/hive/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 b/shark/src/test/hive/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 similarity index 100% rename from src/test/hive/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 rename to shark/src/test/hive/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 diff --git a/src/test/hive/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 b/shark/src/test/hive/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 similarity index 100% rename from src/test/hive/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 rename to shark/src/test/hive/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 diff --git a/src/test/hive/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e b/shark/src/test/hive/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e similarity index 100% rename from src/test/hive/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e rename to shark/src/test/hive/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e diff --git a/src/test/hive/golden/join_1to1-18-6081eb976b4aef2132418510756a385b b/shark/src/test/hive/golden/join_1to1-18-6081eb976b4aef2132418510756a385b similarity index 100% rename from src/test/hive/golden/join_1to1-18-6081eb976b4aef2132418510756a385b rename to shark/src/test/hive/golden/join_1to1-18-6081eb976b4aef2132418510756a385b diff --git a/src/test/hive/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c b/shark/src/test/hive/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c similarity index 100% rename from src/test/hive/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c rename to shark/src/test/hive/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c diff --git a/src/test/hive/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 b/shark/src/test/hive/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 similarity index 100% rename from src/test/hive/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 rename to shark/src/test/hive/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 diff --git a/src/test/hive/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 b/shark/src/test/hive/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 similarity index 100% rename from src/test/hive/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 rename to shark/src/test/hive/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 diff --git a/src/test/hive/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 b/shark/src/test/hive/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 similarity index 100% rename from src/test/hive/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 rename to shark/src/test/hive/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 diff --git a/src/test/hive/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 b/shark/src/test/hive/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 similarity index 100% rename from src/test/hive/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 rename to shark/src/test/hive/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 diff --git a/src/test/hive/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e b/shark/src/test/hive/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e similarity index 100% rename from src/test/hive/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e rename to shark/src/test/hive/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e diff --git a/src/test/hive/golden/join_1to1-24-6081eb976b4aef2132418510756a385b b/shark/src/test/hive/golden/join_1to1-24-6081eb976b4aef2132418510756a385b similarity index 100% rename from src/test/hive/golden/join_1to1-24-6081eb976b4aef2132418510756a385b rename to shark/src/test/hive/golden/join_1to1-24-6081eb976b4aef2132418510756a385b diff --git a/src/test/hive/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c b/shark/src/test/hive/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c similarity index 100% rename from src/test/hive/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c rename to shark/src/test/hive/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c diff --git a/src/test/hive/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 b/shark/src/test/hive/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 similarity index 100% rename from src/test/hive/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 rename to shark/src/test/hive/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 diff --git a/src/test/hive/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 b/shark/src/test/hive/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 similarity index 100% rename from src/test/hive/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 rename to shark/src/test/hive/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 diff --git a/src/test/hive/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e b/shark/src/test/hive/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e similarity index 100% rename from src/test/hive/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e rename to shark/src/test/hive/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e diff --git a/src/test/hive/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e b/shark/src/test/hive/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e similarity index 100% rename from src/test/hive/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e rename to shark/src/test/hive/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e diff --git a/src/test/hive/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 b/shark/src/test/hive/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 similarity index 100% rename from src/test/hive/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 rename to shark/src/test/hive/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 diff --git a/src/test/hive/golden/join_1to1-30-6081eb976b4aef2132418510756a385b b/shark/src/test/hive/golden/join_1to1-30-6081eb976b4aef2132418510756a385b similarity index 100% rename from src/test/hive/golden/join_1to1-30-6081eb976b4aef2132418510756a385b rename to shark/src/test/hive/golden/join_1to1-30-6081eb976b4aef2132418510756a385b diff --git a/src/test/hive/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c b/shark/src/test/hive/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c similarity index 100% rename from src/test/hive/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c rename to shark/src/test/hive/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c diff --git a/src/test/hive/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 b/shark/src/test/hive/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 similarity index 100% rename from src/test/hive/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 rename to shark/src/test/hive/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 diff --git a/src/test/hive/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 b/shark/src/test/hive/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 similarity index 100% rename from src/test/hive/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 rename to shark/src/test/hive/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 diff --git a/src/test/hive/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 b/shark/src/test/hive/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 similarity index 100% rename from src/test/hive/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 rename to shark/src/test/hive/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 diff --git a/src/test/hive/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e b/shark/src/test/hive/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e similarity index 100% rename from src/test/hive/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e rename to shark/src/test/hive/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e diff --git a/src/test/hive/golden/join_1to1-36-6081eb976b4aef2132418510756a385b b/shark/src/test/hive/golden/join_1to1-36-6081eb976b4aef2132418510756a385b similarity index 100% rename from src/test/hive/golden/join_1to1-36-6081eb976b4aef2132418510756a385b rename to shark/src/test/hive/golden/join_1to1-36-6081eb976b4aef2132418510756a385b diff --git a/src/test/hive/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c b/shark/src/test/hive/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c similarity index 100% rename from src/test/hive/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c rename to shark/src/test/hive/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c diff --git a/src/test/hive/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 b/shark/src/test/hive/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 similarity index 100% rename from src/test/hive/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 rename to shark/src/test/hive/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 diff --git a/src/test/hive/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 b/shark/src/test/hive/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 similarity index 100% rename from src/test/hive/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 rename to shark/src/test/hive/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 diff --git a/src/test/hive/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 b/shark/src/test/hive/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 similarity index 100% rename from src/test/hive/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 rename to shark/src/test/hive/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 diff --git a/src/test/hive/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e b/shark/src/test/hive/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e similarity index 100% rename from src/test/hive/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e rename to shark/src/test/hive/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e diff --git a/src/test/hive/golden/join_1to1-6-6081eb976b4aef2132418510756a385b b/shark/src/test/hive/golden/join_1to1-6-6081eb976b4aef2132418510756a385b similarity index 100% rename from src/test/hive/golden/join_1to1-6-6081eb976b4aef2132418510756a385b rename to shark/src/test/hive/golden/join_1to1-6-6081eb976b4aef2132418510756a385b diff --git a/src/test/hive/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c b/shark/src/test/hive/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c similarity index 100% rename from src/test/hive/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c rename to shark/src/test/hive/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c diff --git a/src/test/hive/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 b/shark/src/test/hive/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 similarity index 100% rename from src/test/hive/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 rename to shark/src/test/hive/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 diff --git a/src/test/hive/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 b/shark/src/test/hive/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 similarity index 100% rename from src/test/hive/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 rename to shark/src/test/hive/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 diff --git a/src/test/hive/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 b/shark/src/test/hive/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 similarity index 100% rename from src/test/hive/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 rename to shark/src/test/hive/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 diff --git a/src/test/hive/golden/join_array-1-2c086fcf118fd2538bfa00724209091e b/shark/src/test/hive/golden/join_array-1-2c086fcf118fd2538bfa00724209091e similarity index 100% rename from src/test/hive/golden/join_array-1-2c086fcf118fd2538bfa00724209091e rename to shark/src/test/hive/golden/join_array-1-2c086fcf118fd2538bfa00724209091e diff --git a/src/test/hive/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de b/shark/src/test/hive/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de similarity index 100% rename from src/test/hive/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de rename to shark/src/test/hive/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de diff --git a/src/test/hive/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 b/shark/src/test/hive/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 similarity index 100% rename from src/test/hive/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 rename to shark/src/test/hive/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 diff --git a/src/test/hive/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 b/shark/src/test/hive/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 similarity index 100% rename from src/test/hive/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 rename to shark/src/test/hive/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 diff --git a/src/test/hive/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a b/shark/src/test/hive/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a similarity index 100% rename from src/test/hive/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a rename to shark/src/test/hive/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a diff --git a/src/test/hive/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c b/shark/src/test/hive/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c similarity index 100% rename from src/test/hive/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c rename to shark/src/test/hive/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c diff --git a/src/test/hive/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 b/shark/src/test/hive/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 similarity index 100% rename from src/test/hive/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 rename to shark/src/test/hive/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 diff --git a/src/test/hive/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 b/shark/src/test/hive/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 similarity index 100% rename from src/test/hive/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 rename to shark/src/test/hive/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 diff --git a/src/test/hive/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d b/shark/src/test/hive/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d similarity index 100% rename from src/test/hive/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d rename to shark/src/test/hive/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d diff --git a/src/test/hive/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc b/shark/src/test/hive/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc similarity index 100% rename from src/test/hive/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc rename to shark/src/test/hive/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc diff --git a/src/test/hive/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd b/shark/src/test/hive/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd similarity index 100% rename from src/test/hive/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd rename to shark/src/test/hive/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd diff --git a/src/test/hive/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad b/shark/src/test/hive/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad similarity index 100% rename from src/test/hive/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad rename to shark/src/test/hive/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad diff --git a/src/test/hive/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 b/shark/src/test/hive/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 similarity index 100% rename from src/test/hive/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 rename to shark/src/test/hive/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 diff --git a/src/test/hive/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e b/shark/src/test/hive/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e similarity index 100% rename from src/test/hive/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e rename to shark/src/test/hive/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e diff --git a/src/test/hive/golden/join_empty-3-3b2231f25886c152f222856256ac3473 b/shark/src/test/hive/golden/join_empty-3-3b2231f25886c152f222856256ac3473 similarity index 100% rename from src/test/hive/golden/join_empty-3-3b2231f25886c152f222856256ac3473 rename to shark/src/test/hive/golden/join_empty-3-3b2231f25886c152f222856256ac3473 diff --git a/src/test/hive/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a b/shark/src/test/hive/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a similarity index 100% rename from src/test/hive/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a rename to shark/src/test/hive/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a diff --git a/src/test/hive/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a b/shark/src/test/hive/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a similarity index 100% rename from src/test/hive/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a rename to shark/src/test/hive/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a diff --git a/src/test/hive/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 b/shark/src/test/hive/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 similarity index 100% rename from src/test/hive/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 rename to shark/src/test/hive/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 diff --git a/src/test/hive/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 b/shark/src/test/hive/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 similarity index 100% rename from src/test/hive/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 rename to shark/src/test/hive/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 diff --git a/src/test/hive/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 b/shark/src/test/hive/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 similarity index 100% rename from src/test/hive/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 rename to shark/src/test/hive/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 diff --git a/src/test/hive/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 b/shark/src/test/hive/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 similarity index 100% rename from src/test/hive/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 rename to shark/src/test/hive/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 diff --git a/src/test/hive/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe b/shark/src/test/hive/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe similarity index 100% rename from src/test/hive/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe rename to shark/src/test/hive/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe diff --git a/src/test/hive/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b b/shark/src/test/hive/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b similarity index 100% rename from src/test/hive/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b rename to shark/src/test/hive/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b diff --git a/src/test/hive/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 b/shark/src/test/hive/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 similarity index 100% rename from src/test/hive/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 rename to shark/src/test/hive/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 diff --git a/src/test/hive/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b b/shark/src/test/hive/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b similarity index 100% rename from src/test/hive/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b rename to shark/src/test/hive/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b diff --git a/src/test/hive/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 b/shark/src/test/hive/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 similarity index 100% rename from src/test/hive/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 rename to shark/src/test/hive/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 diff --git a/src/test/hive/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb b/shark/src/test/hive/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb similarity index 100% rename from src/test/hive/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb rename to shark/src/test/hive/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb diff --git a/src/test/hive/golden/join_filters-106-870999978978b22f21997899f1e652b8 b/shark/src/test/hive/golden/join_filters-106-870999978978b22f21997899f1e652b8 similarity index 100% rename from src/test/hive/golden/join_filters-106-870999978978b22f21997899f1e652b8 rename to shark/src/test/hive/golden/join_filters-106-870999978978b22f21997899f1e652b8 diff --git a/src/test/hive/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 b/shark/src/test/hive/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 similarity index 100% rename from src/test/hive/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 rename to shark/src/test/hive/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 diff --git a/src/test/hive/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e b/shark/src/test/hive/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e similarity index 100% rename from src/test/hive/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e rename to shark/src/test/hive/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e diff --git a/src/test/hive/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 b/shark/src/test/hive/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 similarity index 100% rename from src/test/hive/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 rename to shark/src/test/hive/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 diff --git a/src/test/hive/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 b/shark/src/test/hive/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 similarity index 100% rename from src/test/hive/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 rename to shark/src/test/hive/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 diff --git a/src/test/hive/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba b/shark/src/test/hive/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba similarity index 100% rename from src/test/hive/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba rename to shark/src/test/hive/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba diff --git a/src/test/hive/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 b/shark/src/test/hive/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 similarity index 100% rename from src/test/hive/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 rename to shark/src/test/hive/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 diff --git a/src/test/hive/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 b/shark/src/test/hive/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 similarity index 100% rename from src/test/hive/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 rename to shark/src/test/hive/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 diff --git a/src/test/hive/golden/join_filters-113-48c4978768872751832149d72cdf0ba b/shark/src/test/hive/golden/join_filters-113-48c4978768872751832149d72cdf0ba similarity index 100% rename from src/test/hive/golden/join_filters-113-48c4978768872751832149d72cdf0ba rename to shark/src/test/hive/golden/join_filters-113-48c4978768872751832149d72cdf0ba diff --git a/src/test/hive/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 b/shark/src/test/hive/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 similarity index 100% rename from src/test/hive/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 rename to shark/src/test/hive/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 diff --git a/src/test/hive/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 b/shark/src/test/hive/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 similarity index 100% rename from src/test/hive/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 rename to shark/src/test/hive/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 diff --git a/src/test/hive/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 b/shark/src/test/hive/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 similarity index 100% rename from src/test/hive/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 rename to shark/src/test/hive/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 diff --git a/src/test/hive/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 b/shark/src/test/hive/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 similarity index 100% rename from src/test/hive/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 rename to shark/src/test/hive/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 diff --git a/src/test/hive/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e b/shark/src/test/hive/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e similarity index 100% rename from src/test/hive/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e rename to shark/src/test/hive/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e diff --git a/src/test/hive/golden/join_filters-119-7688784396db55ff148292a9755c918a b/shark/src/test/hive/golden/join_filters-119-7688784396db55ff148292a9755c918a similarity index 100% rename from src/test/hive/golden/join_filters-119-7688784396db55ff148292a9755c918a rename to shark/src/test/hive/golden/join_filters-119-7688784396db55ff148292a9755c918a diff --git a/src/test/hive/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e b/shark/src/test/hive/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e similarity index 100% rename from src/test/hive/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e rename to shark/src/test/hive/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e diff --git a/src/test/hive/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 b/shark/src/test/hive/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 similarity index 100% rename from src/test/hive/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 rename to shark/src/test/hive/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 diff --git a/src/test/hive/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada b/shark/src/test/hive/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada similarity index 100% rename from src/test/hive/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada rename to shark/src/test/hive/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada diff --git a/src/test/hive/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 b/shark/src/test/hive/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 similarity index 100% rename from src/test/hive/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 rename to shark/src/test/hive/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 diff --git a/src/test/hive/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 b/shark/src/test/hive/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 similarity index 100% rename from src/test/hive/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 rename to shark/src/test/hive/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 diff --git a/src/test/hive/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf b/shark/src/test/hive/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf similarity index 100% rename from src/test/hive/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf rename to shark/src/test/hive/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf diff --git a/src/test/hive/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 b/shark/src/test/hive/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 similarity index 100% rename from src/test/hive/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 rename to shark/src/test/hive/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 diff --git a/src/test/hive/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 b/shark/src/test/hive/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 similarity index 100% rename from src/test/hive/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 rename to shark/src/test/hive/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 diff --git a/src/test/hive/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 b/shark/src/test/hive/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 similarity index 100% rename from src/test/hive/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 rename to shark/src/test/hive/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 diff --git a/src/test/hive/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 b/shark/src/test/hive/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 similarity index 100% rename from src/test/hive/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 rename to shark/src/test/hive/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 diff --git a/src/test/hive/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f b/shark/src/test/hive/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f similarity index 100% rename from src/test/hive/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f rename to shark/src/test/hive/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f diff --git a/src/test/hive/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 b/shark/src/test/hive/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 similarity index 100% rename from src/test/hive/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 rename to shark/src/test/hive/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 diff --git a/src/test/hive/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf b/shark/src/test/hive/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf similarity index 100% rename from src/test/hive/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf rename to shark/src/test/hive/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf diff --git a/src/test/hive/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e b/shark/src/test/hive/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e similarity index 100% rename from src/test/hive/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e rename to shark/src/test/hive/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e diff --git a/src/test/hive/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e b/shark/src/test/hive/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e similarity index 100% rename from src/test/hive/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e rename to shark/src/test/hive/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e diff --git a/src/test/hive/golden/join_filters-23-a800b885358695758afdb719cdefa94f b/shark/src/test/hive/golden/join_filters-23-a800b885358695758afdb719cdefa94f similarity index 100% rename from src/test/hive/golden/join_filters-23-a800b885358695758afdb719cdefa94f rename to shark/src/test/hive/golden/join_filters-23-a800b885358695758afdb719cdefa94f diff --git a/src/test/hive/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 b/shark/src/test/hive/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 similarity index 100% rename from src/test/hive/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 rename to shark/src/test/hive/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 diff --git a/src/test/hive/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 b/shark/src/test/hive/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 similarity index 100% rename from src/test/hive/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 rename to shark/src/test/hive/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 diff --git a/src/test/hive/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca b/shark/src/test/hive/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca similarity index 100% rename from src/test/hive/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca rename to shark/src/test/hive/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca diff --git a/src/test/hive/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 b/shark/src/test/hive/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 similarity index 100% rename from src/test/hive/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 rename to shark/src/test/hive/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 diff --git a/src/test/hive/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 b/shark/src/test/hive/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 similarity index 100% rename from src/test/hive/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 rename to shark/src/test/hive/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 diff --git a/src/test/hive/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 b/shark/src/test/hive/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 similarity index 100% rename from src/test/hive/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 rename to shark/src/test/hive/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 diff --git a/src/test/hive/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 b/shark/src/test/hive/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 similarity index 100% rename from src/test/hive/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 rename to shark/src/test/hive/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 diff --git a/src/test/hive/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf b/shark/src/test/hive/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf similarity index 100% rename from src/test/hive/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf rename to shark/src/test/hive/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf diff --git a/src/test/hive/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f b/shark/src/test/hive/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f similarity index 100% rename from src/test/hive/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f rename to shark/src/test/hive/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f diff --git a/src/test/hive/golden/join_filters-32-5978cd7936c296493a16a31b926043ab b/shark/src/test/hive/golden/join_filters-32-5978cd7936c296493a16a31b926043ab similarity index 100% rename from src/test/hive/golden/join_filters-32-5978cd7936c296493a16a31b926043ab rename to shark/src/test/hive/golden/join_filters-32-5978cd7936c296493a16a31b926043ab diff --git a/src/test/hive/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 b/shark/src/test/hive/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 similarity index 100% rename from src/test/hive/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 rename to shark/src/test/hive/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 diff --git a/src/test/hive/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 b/shark/src/test/hive/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 similarity index 100% rename from src/test/hive/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 rename to shark/src/test/hive/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 diff --git a/src/test/hive/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe b/shark/src/test/hive/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe similarity index 100% rename from src/test/hive/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe rename to shark/src/test/hive/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe diff --git a/src/test/hive/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b b/shark/src/test/hive/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b similarity index 100% rename from src/test/hive/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b rename to shark/src/test/hive/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b diff --git a/src/test/hive/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 b/shark/src/test/hive/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 similarity index 100% rename from src/test/hive/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 rename to shark/src/test/hive/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 diff --git a/src/test/hive/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b b/shark/src/test/hive/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b similarity index 100% rename from src/test/hive/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b rename to shark/src/test/hive/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b diff --git a/src/test/hive/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 b/shark/src/test/hive/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 similarity index 100% rename from src/test/hive/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 rename to shark/src/test/hive/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 diff --git a/src/test/hive/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 b/shark/src/test/hive/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 similarity index 100% rename from src/test/hive/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 rename to shark/src/test/hive/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 diff --git a/src/test/hive/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb b/shark/src/test/hive/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb similarity index 100% rename from src/test/hive/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb rename to shark/src/test/hive/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb diff --git a/src/test/hive/golden/join_filters-41-870999978978b22f21997899f1e652b8 b/shark/src/test/hive/golden/join_filters-41-870999978978b22f21997899f1e652b8 similarity index 100% rename from src/test/hive/golden/join_filters-41-870999978978b22f21997899f1e652b8 rename to shark/src/test/hive/golden/join_filters-41-870999978978b22f21997899f1e652b8 diff --git a/src/test/hive/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 b/shark/src/test/hive/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 similarity index 100% rename from src/test/hive/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 rename to shark/src/test/hive/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 diff --git a/src/test/hive/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e b/shark/src/test/hive/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e similarity index 100% rename from src/test/hive/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e rename to shark/src/test/hive/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e diff --git a/src/test/hive/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e b/shark/src/test/hive/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e similarity index 100% rename from src/test/hive/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e rename to shark/src/test/hive/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e diff --git a/src/test/hive/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 b/shark/src/test/hive/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 similarity index 100% rename from src/test/hive/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 rename to shark/src/test/hive/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 diff --git a/src/test/hive/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b b/shark/src/test/hive/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b similarity index 100% rename from src/test/hive/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b rename to shark/src/test/hive/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b diff --git a/src/test/hive/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 b/shark/src/test/hive/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 similarity index 100% rename from src/test/hive/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 rename to shark/src/test/hive/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 diff --git a/src/test/hive/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c b/shark/src/test/hive/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c similarity index 100% rename from src/test/hive/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c rename to shark/src/test/hive/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c diff --git a/src/test/hive/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 b/shark/src/test/hive/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 similarity index 100% rename from src/test/hive/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 rename to shark/src/test/hive/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 diff --git a/src/test/hive/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e b/shark/src/test/hive/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e similarity index 100% rename from src/test/hive/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e rename to shark/src/test/hive/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e diff --git a/src/test/hive/golden/join_filters-50-3e6612a89e9124592e790594775054b1 b/shark/src/test/hive/golden/join_filters-50-3e6612a89e9124592e790594775054b1 similarity index 100% rename from src/test/hive/golden/join_filters-50-3e6612a89e9124592e790594775054b1 rename to shark/src/test/hive/golden/join_filters-50-3e6612a89e9124592e790594775054b1 diff --git a/src/test/hive/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac b/shark/src/test/hive/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac similarity index 100% rename from src/test/hive/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac rename to shark/src/test/hive/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac diff --git a/src/test/hive/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 b/shark/src/test/hive/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 similarity index 100% rename from src/test/hive/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 rename to shark/src/test/hive/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 diff --git a/src/test/hive/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 b/shark/src/test/hive/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 similarity index 100% rename from src/test/hive/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 rename to shark/src/test/hive/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 diff --git a/src/test/hive/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba b/shark/src/test/hive/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba similarity index 100% rename from src/test/hive/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba rename to shark/src/test/hive/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba diff --git a/src/test/hive/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 b/shark/src/test/hive/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 similarity index 100% rename from src/test/hive/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 rename to shark/src/test/hive/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 diff --git a/src/test/hive/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 b/shark/src/test/hive/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 similarity index 100% rename from src/test/hive/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 rename to shark/src/test/hive/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 diff --git a/src/test/hive/golden/join_filters-57-48c4978768872751832149d72cdf0ba b/shark/src/test/hive/golden/join_filters-57-48c4978768872751832149d72cdf0ba similarity index 100% rename from src/test/hive/golden/join_filters-57-48c4978768872751832149d72cdf0ba rename to shark/src/test/hive/golden/join_filters-57-48c4978768872751832149d72cdf0ba diff --git a/src/test/hive/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 b/shark/src/test/hive/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 similarity index 100% rename from src/test/hive/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 rename to shark/src/test/hive/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 diff --git a/src/test/hive/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 b/shark/src/test/hive/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 similarity index 100% rename from src/test/hive/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 rename to shark/src/test/hive/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 diff --git a/src/test/hive/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 b/shark/src/test/hive/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 similarity index 100% rename from src/test/hive/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 rename to shark/src/test/hive/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 diff --git a/src/test/hive/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 b/shark/src/test/hive/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 similarity index 100% rename from src/test/hive/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 rename to shark/src/test/hive/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 diff --git a/src/test/hive/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 b/shark/src/test/hive/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 similarity index 100% rename from src/test/hive/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 rename to shark/src/test/hive/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 diff --git a/src/test/hive/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e b/shark/src/test/hive/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e similarity index 100% rename from src/test/hive/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e rename to shark/src/test/hive/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e diff --git a/src/test/hive/golden/join_filters-63-7688784396db55ff148292a9755c918a b/shark/src/test/hive/golden/join_filters-63-7688784396db55ff148292a9755c918a similarity index 100% rename from src/test/hive/golden/join_filters-63-7688784396db55ff148292a9755c918a rename to shark/src/test/hive/golden/join_filters-63-7688784396db55ff148292a9755c918a diff --git a/src/test/hive/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 b/shark/src/test/hive/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 similarity index 100% rename from src/test/hive/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 rename to shark/src/test/hive/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 diff --git a/src/test/hive/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada b/shark/src/test/hive/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada similarity index 100% rename from src/test/hive/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada rename to shark/src/test/hive/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada diff --git a/src/test/hive/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 b/shark/src/test/hive/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 similarity index 100% rename from src/test/hive/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 rename to shark/src/test/hive/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 diff --git a/src/test/hive/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 b/shark/src/test/hive/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 similarity index 100% rename from src/test/hive/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 rename to shark/src/test/hive/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 diff --git a/src/test/hive/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 b/shark/src/test/hive/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 similarity index 100% rename from src/test/hive/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 rename to shark/src/test/hive/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 diff --git a/src/test/hive/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 b/shark/src/test/hive/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 similarity index 100% rename from src/test/hive/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 rename to shark/src/test/hive/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 diff --git a/src/test/hive/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d b/shark/src/test/hive/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d similarity index 100% rename from src/test/hive/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d rename to shark/src/test/hive/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d diff --git a/src/test/hive/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e b/shark/src/test/hive/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e similarity index 100% rename from src/test/hive/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e rename to shark/src/test/hive/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e diff --git a/src/test/hive/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 b/shark/src/test/hive/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 similarity index 100% rename from src/test/hive/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 rename to shark/src/test/hive/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 diff --git a/src/test/hive/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d b/shark/src/test/hive/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d similarity index 100% rename from src/test/hive/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d rename to shark/src/test/hive/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d diff --git a/src/test/hive/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 b/shark/src/test/hive/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 similarity index 100% rename from src/test/hive/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 rename to shark/src/test/hive/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 diff --git a/src/test/hive/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa b/shark/src/test/hive/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa similarity index 100% rename from src/test/hive/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa rename to shark/src/test/hive/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa diff --git a/src/test/hive/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 b/shark/src/test/hive/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 similarity index 100% rename from src/test/hive/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 rename to shark/src/test/hive/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 diff --git a/src/test/hive/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 b/shark/src/test/hive/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 similarity index 100% rename from src/test/hive/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 rename to shark/src/test/hive/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 diff --git a/src/test/hive/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e b/shark/src/test/hive/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e similarity index 100% rename from src/test/hive/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e rename to shark/src/test/hive/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e diff --git a/src/test/hive/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 b/shark/src/test/hive/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 similarity index 100% rename from src/test/hive/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 rename to shark/src/test/hive/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 diff --git a/src/test/hive/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf b/shark/src/test/hive/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf similarity index 100% rename from src/test/hive/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf rename to shark/src/test/hive/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf diff --git a/src/test/hive/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 b/shark/src/test/hive/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 similarity index 100% rename from src/test/hive/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 rename to shark/src/test/hive/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 diff --git a/src/test/hive/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 b/shark/src/test/hive/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 similarity index 100% rename from src/test/hive/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 rename to shark/src/test/hive/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 diff --git a/src/test/hive/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 b/shark/src/test/hive/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 similarity index 100% rename from src/test/hive/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 rename to shark/src/test/hive/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 diff --git a/src/test/hive/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 b/shark/src/test/hive/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 similarity index 100% rename from src/test/hive/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 rename to shark/src/test/hive/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 diff --git a/src/test/hive/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 b/shark/src/test/hive/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 similarity index 100% rename from src/test/hive/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 rename to shark/src/test/hive/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 diff --git a/src/test/hive/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f b/shark/src/test/hive/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f similarity index 100% rename from src/test/hive/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f rename to shark/src/test/hive/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f diff --git a/src/test/hive/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf b/shark/src/test/hive/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf similarity index 100% rename from src/test/hive/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf rename to shark/src/test/hive/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf diff --git a/src/test/hive/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e b/shark/src/test/hive/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e similarity index 100% rename from src/test/hive/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e rename to shark/src/test/hive/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e diff --git a/src/test/hive/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e b/shark/src/test/hive/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e similarity index 100% rename from src/test/hive/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e rename to shark/src/test/hive/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e diff --git a/src/test/hive/golden/join_filters-88-a800b885358695758afdb719cdefa94f b/shark/src/test/hive/golden/join_filters-88-a800b885358695758afdb719cdefa94f similarity index 100% rename from src/test/hive/golden/join_filters-88-a800b885358695758afdb719cdefa94f rename to shark/src/test/hive/golden/join_filters-88-a800b885358695758afdb719cdefa94f diff --git a/src/test/hive/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 b/shark/src/test/hive/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 similarity index 100% rename from src/test/hive/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 rename to shark/src/test/hive/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 diff --git a/src/test/hive/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa b/shark/src/test/hive/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa similarity index 100% rename from src/test/hive/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa rename to shark/src/test/hive/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa diff --git a/src/test/hive/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 b/shark/src/test/hive/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 similarity index 100% rename from src/test/hive/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 rename to shark/src/test/hive/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 diff --git a/src/test/hive/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca b/shark/src/test/hive/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca similarity index 100% rename from src/test/hive/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca rename to shark/src/test/hive/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca diff --git a/src/test/hive/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 b/shark/src/test/hive/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 similarity index 100% rename from src/test/hive/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 rename to shark/src/test/hive/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 diff --git a/src/test/hive/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 b/shark/src/test/hive/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 similarity index 100% rename from src/test/hive/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 rename to shark/src/test/hive/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 diff --git a/src/test/hive/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 b/shark/src/test/hive/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 similarity index 100% rename from src/test/hive/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 rename to shark/src/test/hive/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 diff --git a/src/test/hive/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf b/shark/src/test/hive/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf similarity index 100% rename from src/test/hive/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf rename to shark/src/test/hive/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf diff --git a/src/test/hive/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f b/shark/src/test/hive/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f similarity index 100% rename from src/test/hive/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f rename to shark/src/test/hive/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f diff --git a/src/test/hive/golden/join_filters-97-5978cd7936c296493a16a31b926043ab b/shark/src/test/hive/golden/join_filters-97-5978cd7936c296493a16a31b926043ab similarity index 100% rename from src/test/hive/golden/join_filters-97-5978cd7936c296493a16a31b926043ab rename to shark/src/test/hive/golden/join_filters-97-5978cd7936c296493a16a31b926043ab diff --git a/src/test/hive/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 b/shark/src/test/hive/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 similarity index 100% rename from src/test/hive/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 rename to shark/src/test/hive/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 diff --git a/src/test/hive/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 b/shark/src/test/hive/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 similarity index 100% rename from src/test/hive/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 rename to shark/src/test/hive/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 diff --git a/src/test/hive/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce b/shark/src/test/hive/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce similarity index 100% rename from src/test/hive/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce rename to shark/src/test/hive/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce diff --git a/src/test/hive/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 b/shark/src/test/hive/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 similarity index 100% rename from src/test/hive/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 rename to shark/src/test/hive/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 diff --git a/src/test/hive/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 b/shark/src/test/hive/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 similarity index 100% rename from src/test/hive/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 rename to shark/src/test/hive/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 diff --git a/src/test/hive/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 b/shark/src/test/hive/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 similarity index 100% rename from src/test/hive/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 rename to shark/src/test/hive/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 diff --git a/src/test/hive/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e b/shark/src/test/hive/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e similarity index 100% rename from src/test/hive/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e rename to shark/src/test/hive/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e diff --git a/src/test/hive/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 b/shark/src/test/hive/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 similarity index 100% rename from src/test/hive/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 rename to shark/src/test/hive/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 diff --git a/src/test/hive/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 b/shark/src/test/hive/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 similarity index 100% rename from src/test/hive/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 rename to shark/src/test/hive/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 diff --git a/src/test/hive/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 b/shark/src/test/hive/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 similarity index 100% rename from src/test/hive/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 rename to shark/src/test/hive/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 diff --git a/src/test/hive/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 b/shark/src/test/hive/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 similarity index 100% rename from src/test/hive/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 rename to shark/src/test/hive/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 diff --git a/src/test/hive/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 b/shark/src/test/hive/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 similarity index 100% rename from src/test/hive/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 rename to shark/src/test/hive/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 diff --git a/src/test/hive/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 b/shark/src/test/hive/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 similarity index 100% rename from src/test/hive/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 rename to shark/src/test/hive/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 diff --git a/src/test/hive/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 b/shark/src/test/hive/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 similarity index 100% rename from src/test/hive/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 rename to shark/src/test/hive/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 diff --git a/src/test/hive/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e b/shark/src/test/hive/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e similarity index 100% rename from src/test/hive/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e rename to shark/src/test/hive/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e diff --git a/src/test/hive/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa b/shark/src/test/hive/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa similarity index 100% rename from src/test/hive/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa rename to shark/src/test/hive/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa diff --git a/src/test/hive/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 b/shark/src/test/hive/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 similarity index 100% rename from src/test/hive/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 rename to shark/src/test/hive/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 diff --git a/src/test/hive/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 b/shark/src/test/hive/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 similarity index 100% rename from src/test/hive/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 rename to shark/src/test/hive/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 diff --git a/src/test/hive/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a b/shark/src/test/hive/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a similarity index 100% rename from src/test/hive/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a rename to shark/src/test/hive/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a diff --git a/src/test/hive/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db b/shark/src/test/hive/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db similarity index 100% rename from src/test/hive/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db rename to shark/src/test/hive/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db diff --git a/src/test/hive/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed b/shark/src/test/hive/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed similarity index 100% rename from src/test/hive/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed rename to shark/src/test/hive/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed diff --git a/src/test/hive/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a b/shark/src/test/hive/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a similarity index 100% rename from src/test/hive/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a rename to shark/src/test/hive/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a diff --git a/src/test/hive/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 b/shark/src/test/hive/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 similarity index 100% rename from src/test/hive/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 rename to shark/src/test/hive/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 diff --git a/src/test/hive/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 b/shark/src/test/hive/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 similarity index 100% rename from src/test/hive/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 rename to shark/src/test/hive/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 diff --git a/src/test/hive/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b b/shark/src/test/hive/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b similarity index 100% rename from src/test/hive/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b rename to shark/src/test/hive/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b diff --git a/src/test/hive/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 b/shark/src/test/hive/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 similarity index 100% rename from src/test/hive/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 rename to shark/src/test/hive/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 diff --git a/src/test/hive/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 b/shark/src/test/hive/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 similarity index 100% rename from src/test/hive/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 rename to shark/src/test/hive/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 diff --git a/src/test/hive/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d b/shark/src/test/hive/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d similarity index 100% rename from src/test/hive/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d rename to shark/src/test/hive/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d diff --git a/src/test/hive/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 b/shark/src/test/hive/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 similarity index 100% rename from src/test/hive/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 rename to shark/src/test/hive/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 diff --git a/src/test/hive/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 b/shark/src/test/hive/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 similarity index 100% rename from src/test/hive/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 rename to shark/src/test/hive/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 diff --git a/src/test/hive/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 b/shark/src/test/hive/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 similarity index 100% rename from src/test/hive/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 rename to shark/src/test/hive/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 diff --git a/src/test/hive/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f b/shark/src/test/hive/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f similarity index 100% rename from src/test/hive/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f rename to shark/src/test/hive/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f diff --git a/src/test/hive/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 b/shark/src/test/hive/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 similarity index 100% rename from src/test/hive/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 rename to shark/src/test/hive/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 diff --git a/src/test/hive/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c b/shark/src/test/hive/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c similarity index 100% rename from src/test/hive/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c rename to shark/src/test/hive/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c diff --git a/src/test/hive/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 b/shark/src/test/hive/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 similarity index 100% rename from src/test/hive/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 rename to shark/src/test/hive/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 diff --git a/src/test/hive/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 b/shark/src/test/hive/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 similarity index 100% rename from src/test/hive/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 rename to shark/src/test/hive/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 diff --git a/src/test/hive/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e b/shark/src/test/hive/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e similarity index 100% rename from src/test/hive/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e rename to shark/src/test/hive/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e diff --git a/src/test/hive/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 b/shark/src/test/hive/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 similarity index 100% rename from src/test/hive/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 rename to shark/src/test/hive/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 diff --git a/src/test/hive/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 b/shark/src/test/hive/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 similarity index 100% rename from src/test/hive/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 rename to shark/src/test/hive/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 diff --git a/src/test/hive/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 b/shark/src/test/hive/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 similarity index 100% rename from src/test/hive/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 rename to shark/src/test/hive/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 diff --git a/src/test/hive/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 b/shark/src/test/hive/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 similarity index 100% rename from src/test/hive/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 rename to shark/src/test/hive/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 diff --git a/src/test/hive/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 b/shark/src/test/hive/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 similarity index 100% rename from src/test/hive/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 rename to shark/src/test/hive/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 diff --git a/src/test/hive/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 b/shark/src/test/hive/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 similarity index 100% rename from src/test/hive/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 rename to shark/src/test/hive/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 diff --git a/src/test/hive/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb b/shark/src/test/hive/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb similarity index 100% rename from src/test/hive/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb rename to shark/src/test/hive/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb diff --git a/src/test/hive/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb b/shark/src/test/hive/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb similarity index 100% rename from src/test/hive/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb rename to shark/src/test/hive/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb diff --git a/src/test/hive/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 b/shark/src/test/hive/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 similarity index 100% rename from src/test/hive/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 rename to shark/src/test/hive/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 diff --git a/src/test/hive/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 b/shark/src/test/hive/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 similarity index 100% rename from src/test/hive/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 rename to shark/src/test/hive/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 diff --git a/src/test/hive/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 b/shark/src/test/hive/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 similarity index 100% rename from src/test/hive/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 rename to shark/src/test/hive/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 diff --git a/src/test/hive/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 b/shark/src/test/hive/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 similarity index 100% rename from src/test/hive/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 rename to shark/src/test/hive/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 diff --git a/src/test/hive/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 b/shark/src/test/hive/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 similarity index 100% rename from src/test/hive/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 rename to shark/src/test/hive/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 diff --git a/src/test/hive/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 b/shark/src/test/hive/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 similarity index 100% rename from src/test/hive/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 rename to shark/src/test/hive/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 diff --git a/src/test/hive/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 b/shark/src/test/hive/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 similarity index 100% rename from src/test/hive/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 rename to shark/src/test/hive/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 diff --git a/src/test/hive/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 b/shark/src/test/hive/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 similarity index 100% rename from src/test/hive/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 rename to shark/src/test/hive/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 diff --git a/src/test/hive/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc b/shark/src/test/hive/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc similarity index 100% rename from src/test/hive/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc rename to shark/src/test/hive/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc diff --git a/src/test/hive/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e b/shark/src/test/hive/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e similarity index 100% rename from src/test/hive/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e rename to shark/src/test/hive/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e diff --git a/src/test/hive/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 b/shark/src/test/hive/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 similarity index 100% rename from src/test/hive/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 rename to shark/src/test/hive/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 diff --git a/src/test/hive/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 b/shark/src/test/hive/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 similarity index 100% rename from src/test/hive/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 rename to shark/src/test/hive/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 diff --git a/src/test/hive/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b b/shark/src/test/hive/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b similarity index 100% rename from src/test/hive/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b rename to shark/src/test/hive/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b diff --git a/src/test/hive/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 b/shark/src/test/hive/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 similarity index 100% rename from src/test/hive/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 rename to shark/src/test/hive/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 diff --git a/src/test/hive/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c b/shark/src/test/hive/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c similarity index 100% rename from src/test/hive/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c rename to shark/src/test/hive/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c diff --git a/src/test/hive/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 b/shark/src/test/hive/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 similarity index 100% rename from src/test/hive/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 rename to shark/src/test/hive/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 diff --git a/src/test/hive/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 b/shark/src/test/hive/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 similarity index 100% rename from src/test/hive/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 rename to shark/src/test/hive/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 diff --git a/src/test/hive/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 b/shark/src/test/hive/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 similarity index 100% rename from src/test/hive/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 rename to shark/src/test/hive/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 diff --git a/src/test/hive/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 b/shark/src/test/hive/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 similarity index 100% rename from src/test/hive/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 rename to shark/src/test/hive/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 diff --git a/src/test/hive/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 b/shark/src/test/hive/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 similarity index 100% rename from src/test/hive/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 rename to shark/src/test/hive/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 diff --git a/src/test/hive/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 b/shark/src/test/hive/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 similarity index 100% rename from src/test/hive/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 rename to shark/src/test/hive/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 diff --git a/src/test/hive/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 b/shark/src/test/hive/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 similarity index 100% rename from src/test/hive/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 rename to shark/src/test/hive/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 diff --git a/src/test/hive/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c b/shark/src/test/hive/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c similarity index 100% rename from src/test/hive/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c rename to shark/src/test/hive/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c diff --git a/src/test/hive/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 b/shark/src/test/hive/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 similarity index 100% rename from src/test/hive/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 rename to shark/src/test/hive/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 diff --git a/src/test/hive/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f b/shark/src/test/hive/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f similarity index 100% rename from src/test/hive/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f rename to shark/src/test/hive/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f diff --git a/src/test/hive/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 b/shark/src/test/hive/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 similarity index 100% rename from src/test/hive/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 rename to shark/src/test/hive/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 diff --git a/src/test/hive/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 b/shark/src/test/hive/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 similarity index 100% rename from src/test/hive/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 rename to shark/src/test/hive/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 diff --git a/src/test/hive/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 b/shark/src/test/hive/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 similarity index 100% rename from src/test/hive/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 rename to shark/src/test/hive/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 diff --git a/src/test/hive/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 b/shark/src/test/hive/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 similarity index 100% rename from src/test/hive/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 rename to shark/src/test/hive/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 diff --git a/src/test/hive/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 b/shark/src/test/hive/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 similarity index 100% rename from src/test/hive/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 rename to shark/src/test/hive/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 diff --git a/src/test/hive/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 b/shark/src/test/hive/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 similarity index 100% rename from src/test/hive/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 rename to shark/src/test/hive/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 diff --git a/src/test/hive/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a b/shark/src/test/hive/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a similarity index 100% rename from src/test/hive/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a rename to shark/src/test/hive/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a diff --git a/src/test/hive/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 b/shark/src/test/hive/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 similarity index 100% rename from src/test/hive/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 rename to shark/src/test/hive/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 diff --git a/src/test/hive/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 b/shark/src/test/hive/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 similarity index 100% rename from src/test/hive/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 rename to shark/src/test/hive/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 diff --git a/src/test/hive/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 b/shark/src/test/hive/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 similarity index 100% rename from src/test/hive/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 rename to shark/src/test/hive/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 diff --git a/src/test/hive/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 b/shark/src/test/hive/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 similarity index 100% rename from src/test/hive/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 rename to shark/src/test/hive/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 diff --git a/src/test/hive/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b b/shark/src/test/hive/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b similarity index 100% rename from src/test/hive/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b rename to shark/src/test/hive/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b diff --git a/src/test/hive/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 b/shark/src/test/hive/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 similarity index 100% rename from src/test/hive/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 rename to shark/src/test/hive/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 diff --git a/src/test/hive/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e b/shark/src/test/hive/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e similarity index 100% rename from src/test/hive/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e rename to shark/src/test/hive/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e diff --git a/src/test/hive/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 b/shark/src/test/hive/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 similarity index 100% rename from src/test/hive/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 rename to shark/src/test/hive/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 diff --git a/src/test/hive/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee b/shark/src/test/hive/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee similarity index 100% rename from src/test/hive/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee rename to shark/src/test/hive/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee diff --git a/src/test/hive/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 b/shark/src/test/hive/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 similarity index 100% rename from src/test/hive/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 rename to shark/src/test/hive/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 diff --git a/src/test/hive/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 b/shark/src/test/hive/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 similarity index 100% rename from src/test/hive/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 rename to shark/src/test/hive/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 diff --git a/src/test/hive/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 b/shark/src/test/hive/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 similarity index 100% rename from src/test/hive/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 rename to shark/src/test/hive/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 diff --git a/src/test/hive/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 b/shark/src/test/hive/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 similarity index 100% rename from src/test/hive/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 rename to shark/src/test/hive/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 diff --git a/src/test/hive/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d b/shark/src/test/hive/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d similarity index 100% rename from src/test/hive/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d rename to shark/src/test/hive/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d diff --git a/src/test/hive/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 b/shark/src/test/hive/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 similarity index 100% rename from src/test/hive/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 rename to shark/src/test/hive/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 diff --git a/src/test/hive/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e b/shark/src/test/hive/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e similarity index 100% rename from src/test/hive/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e rename to shark/src/test/hive/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e diff --git a/src/test/hive/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 b/shark/src/test/hive/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 similarity index 100% rename from src/test/hive/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 rename to shark/src/test/hive/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 diff --git a/src/test/hive/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 b/shark/src/test/hive/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 similarity index 100% rename from src/test/hive/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 rename to shark/src/test/hive/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 diff --git a/src/test/hive/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec b/shark/src/test/hive/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec similarity index 100% rename from src/test/hive/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec rename to shark/src/test/hive/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec diff --git a/src/test/hive/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 b/shark/src/test/hive/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 similarity index 100% rename from src/test/hive/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 rename to shark/src/test/hive/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 diff --git a/src/test/hive/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 b/shark/src/test/hive/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 similarity index 100% rename from src/test/hive/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 rename to shark/src/test/hive/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 diff --git a/src/test/hive/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 b/shark/src/test/hive/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 similarity index 100% rename from src/test/hive/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 rename to shark/src/test/hive/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 diff --git a/src/test/hive/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 b/shark/src/test/hive/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 similarity index 100% rename from src/test/hive/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 rename to shark/src/test/hive/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 diff --git a/src/test/hive/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 b/shark/src/test/hive/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 similarity index 100% rename from src/test/hive/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 rename to shark/src/test/hive/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 diff --git a/src/test/hive/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 b/shark/src/test/hive/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 similarity index 100% rename from src/test/hive/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 rename to shark/src/test/hive/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 diff --git a/src/test/hive/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e b/shark/src/test/hive/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e similarity index 100% rename from src/test/hive/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e rename to shark/src/test/hive/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e diff --git a/src/test/hive/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 b/shark/src/test/hive/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 similarity index 100% rename from src/test/hive/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 rename to shark/src/test/hive/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 diff --git a/src/test/hive/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 b/shark/src/test/hive/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 similarity index 100% rename from src/test/hive/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 rename to shark/src/test/hive/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 diff --git a/src/test/hive/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec b/shark/src/test/hive/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec similarity index 100% rename from src/test/hive/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec rename to shark/src/test/hive/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec diff --git a/src/test/hive/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 b/shark/src/test/hive/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 similarity index 100% rename from src/test/hive/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 rename to shark/src/test/hive/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 diff --git a/src/test/hive/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 b/shark/src/test/hive/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 similarity index 100% rename from src/test/hive/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 rename to shark/src/test/hive/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 diff --git a/src/test/hive/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf b/shark/src/test/hive/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf similarity index 100% rename from src/test/hive/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf rename to shark/src/test/hive/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf diff --git a/src/test/hive/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a b/shark/src/test/hive/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a similarity index 100% rename from src/test/hive/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a rename to shark/src/test/hive/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a diff --git a/src/test/hive/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f b/shark/src/test/hive/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f similarity index 100% rename from src/test/hive/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f rename to shark/src/test/hive/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f diff --git a/src/test/hive/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 b/shark/src/test/hive/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 similarity index 100% rename from src/test/hive/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 rename to shark/src/test/hive/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 diff --git a/src/test/hive/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 b/shark/src/test/hive/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 similarity index 100% rename from src/test/hive/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 rename to shark/src/test/hive/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 diff --git a/src/test/hive/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c b/shark/src/test/hive/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c similarity index 100% rename from src/test/hive/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c rename to shark/src/test/hive/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c diff --git a/src/test/hive/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 b/shark/src/test/hive/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 similarity index 100% rename from src/test/hive/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 rename to shark/src/test/hive/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 diff --git a/src/test/hive/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 b/shark/src/test/hive/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 similarity index 100% rename from src/test/hive/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 rename to shark/src/test/hive/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 diff --git a/src/test/hive/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 b/shark/src/test/hive/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 similarity index 100% rename from src/test/hive/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 rename to shark/src/test/hive/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 diff --git a/src/test/hive/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba b/shark/src/test/hive/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba similarity index 100% rename from src/test/hive/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba rename to shark/src/test/hive/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba diff --git a/src/test/hive/golden/join_star-0-6001f83f7780835737e3262a34c2b832 b/shark/src/test/hive/golden/join_star-0-6001f83f7780835737e3262a34c2b832 similarity index 100% rename from src/test/hive/golden/join_star-0-6001f83f7780835737e3262a34c2b832 rename to shark/src/test/hive/golden/join_star-0-6001f83f7780835737e3262a34c2b832 diff --git a/src/test/hive/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d b/shark/src/test/hive/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d similarity index 100% rename from src/test/hive/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d rename to shark/src/test/hive/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d diff --git a/src/test/hive/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e b/shark/src/test/hive/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e similarity index 100% rename from src/test/hive/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e rename to shark/src/test/hive/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e diff --git a/src/test/hive/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae b/shark/src/test/hive/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae similarity index 100% rename from src/test/hive/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae rename to shark/src/test/hive/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae diff --git a/src/test/hive/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 b/shark/src/test/hive/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 similarity index 100% rename from src/test/hive/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 rename to shark/src/test/hive/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 diff --git a/src/test/hive/golden/join_star-13-342b7249c9ce1484869169b1b33191cb b/shark/src/test/hive/golden/join_star-13-342b7249c9ce1484869169b1b33191cb similarity index 100% rename from src/test/hive/golden/join_star-13-342b7249c9ce1484869169b1b33191cb rename to shark/src/test/hive/golden/join_star-13-342b7249c9ce1484869169b1b33191cb diff --git a/src/test/hive/golden/join_star-14-75513308d30b781fd2e06d81963c4363 b/shark/src/test/hive/golden/join_star-14-75513308d30b781fd2e06d81963c4363 similarity index 100% rename from src/test/hive/golden/join_star-14-75513308d30b781fd2e06d81963c4363 rename to shark/src/test/hive/golden/join_star-14-75513308d30b781fd2e06d81963c4363 diff --git a/src/test/hive/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 b/shark/src/test/hive/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 similarity index 100% rename from src/test/hive/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 rename to shark/src/test/hive/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 diff --git a/src/test/hive/golden/join_star-16-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/join_star-16-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/join_star-16-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/join_star-16-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 b/shark/src/test/hive/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 similarity index 100% rename from src/test/hive/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 rename to shark/src/test/hive/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 diff --git a/src/test/hive/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa b/shark/src/test/hive/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa similarity index 100% rename from src/test/hive/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa rename to shark/src/test/hive/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa diff --git a/src/test/hive/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 b/shark/src/test/hive/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 similarity index 100% rename from src/test/hive/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 rename to shark/src/test/hive/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 diff --git a/src/test/hive/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 b/shark/src/test/hive/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 similarity index 100% rename from src/test/hive/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 rename to shark/src/test/hive/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 diff --git a/src/test/hive/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 b/shark/src/test/hive/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 similarity index 100% rename from src/test/hive/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 rename to shark/src/test/hive/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 diff --git a/src/test/hive/golden/join_star-22-429cfd140488242d569aee6594aae76b b/shark/src/test/hive/golden/join_star-22-429cfd140488242d569aee6594aae76b similarity index 100% rename from src/test/hive/golden/join_star-22-429cfd140488242d569aee6594aae76b rename to shark/src/test/hive/golden/join_star-22-429cfd140488242d569aee6594aae76b diff --git a/src/test/hive/golden/join_star-23-bada259628918dbfb4837bcb58258530 b/shark/src/test/hive/golden/join_star-23-bada259628918dbfb4837bcb58258530 similarity index 100% rename from src/test/hive/golden/join_star-23-bada259628918dbfb4837bcb58258530 rename to shark/src/test/hive/golden/join_star-23-bada259628918dbfb4837bcb58258530 diff --git a/src/test/hive/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 b/shark/src/test/hive/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 similarity index 100% rename from src/test/hive/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 rename to shark/src/test/hive/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 diff --git a/src/test/hive/golden/join_star-25-be5af28b6bccf529a31ef68619de699e b/shark/src/test/hive/golden/join_star-25-be5af28b6bccf529a31ef68619de699e similarity index 100% rename from src/test/hive/golden/join_star-25-be5af28b6bccf529a31ef68619de699e rename to shark/src/test/hive/golden/join_star-25-be5af28b6bccf529a31ef68619de699e diff --git a/src/test/hive/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 b/shark/src/test/hive/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 similarity index 100% rename from src/test/hive/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 rename to shark/src/test/hive/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 diff --git a/src/test/hive/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 b/shark/src/test/hive/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 similarity index 100% rename from src/test/hive/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 rename to shark/src/test/hive/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 diff --git a/src/test/hive/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 b/shark/src/test/hive/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 similarity index 100% rename from src/test/hive/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 rename to shark/src/test/hive/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 diff --git a/src/test/hive/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da b/shark/src/test/hive/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da similarity index 100% rename from src/test/hive/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da rename to shark/src/test/hive/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da diff --git a/src/test/hive/golden/join_star-4-aaba115af7543b214820a48a694f2463 b/shark/src/test/hive/golden/join_star-4-aaba115af7543b214820a48a694f2463 similarity index 100% rename from src/test/hive/golden/join_star-4-aaba115af7543b214820a48a694f2463 rename to shark/src/test/hive/golden/join_star-4-aaba115af7543b214820a48a694f2463 diff --git a/src/test/hive/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da b/shark/src/test/hive/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da similarity index 100% rename from src/test/hive/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da rename to shark/src/test/hive/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da diff --git a/src/test/hive/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a b/shark/src/test/hive/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a similarity index 100% rename from src/test/hive/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a rename to shark/src/test/hive/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a diff --git a/src/test/hive/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a b/shark/src/test/hive/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a similarity index 100% rename from src/test/hive/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a rename to shark/src/test/hive/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a diff --git a/src/test/hive/golden/join_star-8-a957982d8981ff0a35397ca449297024 b/shark/src/test/hive/golden/join_star-8-a957982d8981ff0a35397ca449297024 similarity index 100% rename from src/test/hive/golden/join_star-8-a957982d8981ff0a35397ca449297024 rename to shark/src/test/hive/golden/join_star-8-a957982d8981ff0a35397ca449297024 diff --git a/src/test/hive/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 b/shark/src/test/hive/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 similarity index 100% rename from src/test/hive/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 rename to shark/src/test/hive/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 diff --git a/src/test/hive/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 b/shark/src/test/hive/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 similarity index 100% rename from src/test/hive/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 rename to shark/src/test/hive/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 diff --git a/src/test/hive/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db b/shark/src/test/hive/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db similarity index 100% rename from src/test/hive/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db rename to shark/src/test/hive/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db diff --git a/src/test/hive/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b b/shark/src/test/hive/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b similarity index 100% rename from src/test/hive/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b rename to shark/src/test/hive/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b diff --git a/src/test/hive/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 b/shark/src/test/hive/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 similarity index 100% rename from src/test/hive/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 rename to shark/src/test/hive/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 diff --git a/src/test/hive/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 b/shark/src/test/hive/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 similarity index 100% rename from src/test/hive/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 rename to shark/src/test/hive/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 diff --git a/src/test/hive/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 b/shark/src/test/hive/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 similarity index 100% rename from src/test/hive/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 rename to shark/src/test/hive/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 diff --git a/src/test/hive/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 b/shark/src/test/hive/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 similarity index 100% rename from src/test/hive/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 rename to shark/src/test/hive/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 diff --git a/src/test/hive/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b b/shark/src/test/hive/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b similarity index 100% rename from src/test/hive/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b rename to shark/src/test/hive/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b diff --git a/src/test/hive/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 b/shark/src/test/hive/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 similarity index 100% rename from src/test/hive/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 rename to shark/src/test/hive/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 diff --git a/src/test/hive/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 b/shark/src/test/hive/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 similarity index 100% rename from src/test/hive/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 rename to shark/src/test/hive/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 diff --git a/src/test/hive/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef b/shark/src/test/hive/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef similarity index 100% rename from src/test/hive/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef rename to shark/src/test/hive/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef diff --git a/src/test/hive/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d b/shark/src/test/hive/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d similarity index 100% rename from src/test/hive/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d rename to shark/src/test/hive/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d diff --git a/src/test/hive/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 b/shark/src/test/hive/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 similarity index 100% rename from src/test/hive/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 rename to shark/src/test/hive/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 diff --git a/src/test/hive/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 b/shark/src/test/hive/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 similarity index 100% rename from src/test/hive/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 rename to shark/src/test/hive/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 diff --git a/src/test/hive/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee b/shark/src/test/hive/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee similarity index 100% rename from src/test/hive/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee rename to shark/src/test/hive/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee diff --git a/src/test/hive/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb b/shark/src/test/hive/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb similarity index 100% rename from src/test/hive/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb rename to shark/src/test/hive/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb diff --git a/src/test/hive/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 b/shark/src/test/hive/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 similarity index 100% rename from src/test/hive/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 rename to shark/src/test/hive/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 diff --git a/src/test/hive/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 b/shark/src/test/hive/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 similarity index 100% rename from src/test/hive/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 rename to shark/src/test/hive/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 diff --git a/src/test/hive/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 b/shark/src/test/hive/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 similarity index 100% rename from src/test/hive/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 rename to shark/src/test/hive/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 diff --git a/src/test/hive/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e b/shark/src/test/hive/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e similarity index 100% rename from src/test/hive/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e rename to shark/src/test/hive/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e diff --git a/src/test/hive/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b b/shark/src/test/hive/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b similarity index 100% rename from src/test/hive/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b rename to shark/src/test/hive/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b diff --git a/src/test/hive/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a b/shark/src/test/hive/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a similarity index 100% rename from src/test/hive/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a rename to shark/src/test/hive/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a diff --git a/src/test/hive/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b b/shark/src/test/hive/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b similarity index 100% rename from src/test/hive/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b rename to shark/src/test/hive/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b diff --git a/src/test/hive/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 b/shark/src/test/hive/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 similarity index 100% rename from src/test/hive/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 rename to shark/src/test/hive/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 diff --git a/src/test/hive/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e b/shark/src/test/hive/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e similarity index 100% rename from src/test/hive/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e rename to shark/src/test/hive/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e diff --git a/src/test/hive/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 b/shark/src/test/hive/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 similarity index 100% rename from src/test/hive/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 rename to shark/src/test/hive/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 diff --git a/src/test/hive/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 b/shark/src/test/hive/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 similarity index 100% rename from src/test/hive/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 rename to shark/src/test/hive/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 diff --git a/src/test/hive/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba b/shark/src/test/hive/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba similarity index 100% rename from src/test/hive/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba rename to shark/src/test/hive/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba diff --git a/src/test/hive/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 b/shark/src/test/hive/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 similarity index 100% rename from src/test/hive/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 rename to shark/src/test/hive/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 diff --git a/src/test/hive/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e b/shark/src/test/hive/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e similarity index 100% rename from src/test/hive/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e rename to shark/src/test/hive/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e diff --git a/src/test/hive/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 b/shark/src/test/hive/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 similarity index 100% rename from src/test/hive/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 rename to shark/src/test/hive/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 diff --git a/src/test/hive/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab b/shark/src/test/hive/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab similarity index 100% rename from src/test/hive/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab rename to shark/src/test/hive/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab diff --git a/src/test/hive/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e b/shark/src/test/hive/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e similarity index 100% rename from src/test/hive/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e rename to shark/src/test/hive/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e diff --git a/src/test/hive/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 b/shark/src/test/hive/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 similarity index 100% rename from src/test/hive/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 rename to shark/src/test/hive/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 diff --git a/src/test/hive/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 b/shark/src/test/hive/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 similarity index 100% rename from src/test/hive/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 rename to shark/src/test/hive/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 diff --git a/src/test/hive/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 b/shark/src/test/hive/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 similarity index 100% rename from src/test/hive/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 rename to shark/src/test/hive/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 diff --git a/src/test/hive/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 b/shark/src/test/hive/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 similarity index 100% rename from src/test/hive/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 rename to shark/src/test/hive/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 diff --git a/src/test/hive/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e b/shark/src/test/hive/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e similarity index 100% rename from src/test/hive/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e rename to shark/src/test/hive/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e diff --git a/src/test/hive/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 b/shark/src/test/hive/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 similarity index 100% rename from src/test/hive/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 rename to shark/src/test/hive/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 diff --git a/src/test/hive/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae b/shark/src/test/hive/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae similarity index 100% rename from src/test/hive/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae rename to shark/src/test/hive/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae diff --git a/src/test/hive/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 b/shark/src/test/hive/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 similarity index 100% rename from src/test/hive/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 rename to shark/src/test/hive/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 diff --git a/src/test/hive/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 b/shark/src/test/hive/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 similarity index 100% rename from src/test/hive/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 rename to shark/src/test/hive/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 diff --git a/src/test/hive/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 b/shark/src/test/hive/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 similarity index 100% rename from src/test/hive/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 rename to shark/src/test/hive/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 diff --git a/src/test/hive/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 b/shark/src/test/hive/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 similarity index 100% rename from src/test/hive/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 rename to shark/src/test/hive/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 diff --git a/src/test/hive/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 b/shark/src/test/hive/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 similarity index 100% rename from src/test/hive/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 rename to shark/src/test/hive/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 diff --git a/src/test/hive/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 b/shark/src/test/hive/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 similarity index 100% rename from src/test/hive/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 rename to shark/src/test/hive/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 diff --git a/src/test/hive/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 b/shark/src/test/hive/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 similarity index 100% rename from src/test/hive/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 rename to shark/src/test/hive/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 diff --git a/src/test/hive/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 b/shark/src/test/hive/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 similarity index 100% rename from src/test/hive/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 rename to shark/src/test/hive/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 diff --git a/src/test/hive/golden/lateral_view_cp-4-507876b610812506343e9d251645170b b/shark/src/test/hive/golden/lateral_view_cp-4-507876b610812506343e9d251645170b similarity index 100% rename from src/test/hive/golden/lateral_view_cp-4-507876b610812506343e9d251645170b rename to shark/src/test/hive/golden/lateral_view_cp-4-507876b610812506343e9d251645170b diff --git a/src/test/hive/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 b/shark/src/test/hive/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 similarity index 100% rename from src/test/hive/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 rename to shark/src/test/hive/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 diff --git a/src/test/hive/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 b/shark/src/test/hive/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 similarity index 100% rename from src/test/hive/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 rename to shark/src/test/hive/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 diff --git a/src/test/hive/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a b/shark/src/test/hive/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a similarity index 100% rename from src/test/hive/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a rename to shark/src/test/hive/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a diff --git a/src/test/hive/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 b/shark/src/test/hive/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 similarity index 100% rename from src/test/hive/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 rename to shark/src/test/hive/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 diff --git a/src/test/hive/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe b/shark/src/test/hive/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe similarity index 100% rename from src/test/hive/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe rename to shark/src/test/hive/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe diff --git a/src/test/hive/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 b/shark/src/test/hive/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 similarity index 100% rename from src/test/hive/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 rename to shark/src/test/hive/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 diff --git a/src/test/hive/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 b/shark/src/test/hive/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 similarity index 100% rename from src/test/hive/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 rename to shark/src/test/hive/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 diff --git a/src/test/hive/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa b/shark/src/test/hive/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa similarity index 100% rename from src/test/hive/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa rename to shark/src/test/hive/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa diff --git a/src/test/hive/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 b/shark/src/test/hive/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 similarity index 100% rename from src/test/hive/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 rename to shark/src/test/hive/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 diff --git a/src/test/hive/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d b/shark/src/test/hive/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d similarity index 100% rename from src/test/hive/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d rename to shark/src/test/hive/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d diff --git a/src/test/hive/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e b/shark/src/test/hive/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e similarity index 100% rename from src/test/hive/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e rename to shark/src/test/hive/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e diff --git a/src/test/hive/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 b/shark/src/test/hive/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 similarity index 100% rename from src/test/hive/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 rename to shark/src/test/hive/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 diff --git a/src/test/hive/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 b/shark/src/test/hive/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 similarity index 100% rename from src/test/hive/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 rename to shark/src/test/hive/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 diff --git a/src/test/hive/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 b/shark/src/test/hive/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 similarity index 100% rename from src/test/hive/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 rename to shark/src/test/hive/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 diff --git a/src/test/hive/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f b/shark/src/test/hive/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f similarity index 100% rename from src/test/hive/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f rename to shark/src/test/hive/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f diff --git a/src/test/hive/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 b/shark/src/test/hive/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 similarity index 100% rename from src/test/hive/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 rename to shark/src/test/hive/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 diff --git a/src/test/hive/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 b/shark/src/test/hive/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 similarity index 100% rename from src/test/hive/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 rename to shark/src/test/hive/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 diff --git a/src/test/hive/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f b/shark/src/test/hive/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f similarity index 100% rename from src/test/hive/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f rename to shark/src/test/hive/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f diff --git a/src/test/hive/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa b/shark/src/test/hive/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa similarity index 100% rename from src/test/hive/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa rename to shark/src/test/hive/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa diff --git a/src/test/hive/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b b/shark/src/test/hive/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b similarity index 100% rename from src/test/hive/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b rename to shark/src/test/hive/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b diff --git a/src/test/hive/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 b/shark/src/test/hive/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 similarity index 100% rename from src/test/hive/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 rename to shark/src/test/hive/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 diff --git a/src/test/hive/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 b/shark/src/test/hive/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 similarity index 100% rename from src/test/hive/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 rename to shark/src/test/hive/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 diff --git a/src/test/hive/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb b/shark/src/test/hive/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb similarity index 100% rename from src/test/hive/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb rename to shark/src/test/hive/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb diff --git a/src/test/hive/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 b/shark/src/test/hive/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 similarity index 100% rename from src/test/hive/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 rename to shark/src/test/hive/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 diff --git a/src/test/hive/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 b/shark/src/test/hive/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 similarity index 100% rename from src/test/hive/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 rename to shark/src/test/hive/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 diff --git a/src/test/hive/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f b/shark/src/test/hive/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f similarity index 100% rename from src/test/hive/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f rename to shark/src/test/hive/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f diff --git a/src/test/hive/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 b/shark/src/test/hive/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 similarity index 100% rename from src/test/hive/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 rename to shark/src/test/hive/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 diff --git a/src/test/hive/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 b/shark/src/test/hive/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 similarity index 100% rename from src/test/hive/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 rename to shark/src/test/hive/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 diff --git a/src/test/hive/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 b/shark/src/test/hive/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 similarity index 100% rename from src/test/hive/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 rename to shark/src/test/hive/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 diff --git a/src/test/hive/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 b/shark/src/test/hive/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 similarity index 100% rename from src/test/hive/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 rename to shark/src/test/hive/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 diff --git a/src/test/hive/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 b/shark/src/test/hive/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 similarity index 100% rename from src/test/hive/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 rename to shark/src/test/hive/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 diff --git a/src/test/hive/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 b/shark/src/test/hive/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 similarity index 100% rename from src/test/hive/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 rename to shark/src/test/hive/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 diff --git a/src/test/hive/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 b/shark/src/test/hive/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 similarity index 100% rename from src/test/hive/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 rename to shark/src/test/hive/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 diff --git a/src/test/hive/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 b/shark/src/test/hive/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 similarity index 100% rename from src/test/hive/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 rename to shark/src/test/hive/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 diff --git a/src/test/hive/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb b/shark/src/test/hive/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb similarity index 100% rename from src/test/hive/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb rename to shark/src/test/hive/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb diff --git a/src/test/hive/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 b/shark/src/test/hive/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 similarity index 100% rename from src/test/hive/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 rename to shark/src/test/hive/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 diff --git a/src/test/hive/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e b/shark/src/test/hive/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e similarity index 100% rename from src/test/hive/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e rename to shark/src/test/hive/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e diff --git a/src/test/hive/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a b/shark/src/test/hive/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a similarity index 100% rename from src/test/hive/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a rename to shark/src/test/hive/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a diff --git a/src/test/hive/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 b/shark/src/test/hive/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 similarity index 100% rename from src/test/hive/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 rename to shark/src/test/hive/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 diff --git a/src/test/hive/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a b/shark/src/test/hive/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a similarity index 100% rename from src/test/hive/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a rename to shark/src/test/hive/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a diff --git a/src/test/hive/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c b/shark/src/test/hive/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c similarity index 100% rename from src/test/hive/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c rename to shark/src/test/hive/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c diff --git a/src/test/hive/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 b/shark/src/test/hive/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 similarity index 100% rename from src/test/hive/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 rename to shark/src/test/hive/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 diff --git a/src/test/hive/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f b/shark/src/test/hive/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f similarity index 100% rename from src/test/hive/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f rename to shark/src/test/hive/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f diff --git a/src/test/hive/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 b/shark/src/test/hive/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 similarity index 100% rename from src/test/hive/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 rename to shark/src/test/hive/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 diff --git a/src/test/hive/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 b/shark/src/test/hive/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 similarity index 100% rename from src/test/hive/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 rename to shark/src/test/hive/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 diff --git a/src/test/hive/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 b/shark/src/test/hive/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 similarity index 100% rename from src/test/hive/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 rename to shark/src/test/hive/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 diff --git a/src/test/hive/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 b/shark/src/test/hive/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 similarity index 100% rename from src/test/hive/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 rename to shark/src/test/hive/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 diff --git a/src/test/hive/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a b/shark/src/test/hive/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a similarity index 100% rename from src/test/hive/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a rename to shark/src/test/hive/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a diff --git a/src/test/hive/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 b/shark/src/test/hive/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 similarity index 100% rename from src/test/hive/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 rename to shark/src/test/hive/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 diff --git a/src/test/hive/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 b/shark/src/test/hive/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 similarity index 100% rename from src/test/hive/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 rename to shark/src/test/hive/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 diff --git a/src/test/hive/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c b/shark/src/test/hive/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c similarity index 100% rename from src/test/hive/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c rename to shark/src/test/hive/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c diff --git a/src/test/hive/golden/literal_double-0-10ef1098e35d900983be3814de8f974f b/shark/src/test/hive/golden/literal_double-0-10ef1098e35d900983be3814de8f974f similarity index 100% rename from src/test/hive/golden/literal_double-0-10ef1098e35d900983be3814de8f974f rename to shark/src/test/hive/golden/literal_double-0-10ef1098e35d900983be3814de8f974f diff --git a/src/test/hive/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 b/shark/src/test/hive/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 similarity index 100% rename from src/test/hive/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 rename to shark/src/test/hive/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 diff --git a/src/test/hive/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 b/shark/src/test/hive/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 similarity index 100% rename from src/test/hive/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 rename to shark/src/test/hive/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 diff --git a/src/test/hive/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 b/shark/src/test/hive/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 similarity index 100% rename from src/test/hive/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 rename to shark/src/test/hive/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 diff --git a/src/test/hive/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 b/shark/src/test/hive/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 similarity index 100% rename from src/test/hive/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 rename to shark/src/test/hive/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 diff --git a/src/test/hive/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d b/shark/src/test/hive/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d similarity index 100% rename from src/test/hive/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d rename to shark/src/test/hive/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d diff --git a/src/test/hive/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d b/shark/src/test/hive/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d similarity index 100% rename from src/test/hive/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d rename to shark/src/test/hive/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d diff --git a/src/test/hive/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 b/shark/src/test/hive/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 similarity index 100% rename from src/test/hive/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 rename to shark/src/test/hive/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 diff --git a/src/test/hive/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 b/shark/src/test/hive/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 similarity index 100% rename from src/test/hive/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 rename to shark/src/test/hive/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 diff --git a/src/test/hive/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 b/shark/src/test/hive/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 similarity index 100% rename from src/test/hive/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 rename to shark/src/test/hive/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 diff --git a/src/test/hive/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 b/shark/src/test/hive/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 similarity index 100% rename from src/test/hive/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 rename to shark/src/test/hive/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 diff --git a/src/test/hive/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f b/shark/src/test/hive/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f similarity index 100% rename from src/test/hive/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f rename to shark/src/test/hive/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f diff --git a/src/test/hive/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 b/shark/src/test/hive/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 similarity index 100% rename from src/test/hive/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 rename to shark/src/test/hive/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 diff --git a/src/test/hive/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 b/shark/src/test/hive/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 similarity index 100% rename from src/test/hive/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 rename to shark/src/test/hive/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 diff --git a/src/test/hive/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 b/shark/src/test/hive/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 similarity index 100% rename from src/test/hive/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 rename to shark/src/test/hive/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 diff --git a/src/test/hive/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 b/shark/src/test/hive/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 similarity index 100% rename from src/test/hive/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 rename to shark/src/test/hive/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 diff --git a/src/test/hive/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e b/shark/src/test/hive/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e similarity index 100% rename from src/test/hive/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e rename to shark/src/test/hive/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e diff --git a/src/test/hive/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 b/shark/src/test/hive/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 similarity index 100% rename from src/test/hive/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 rename to shark/src/test/hive/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 diff --git a/src/test/hive/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 b/shark/src/test/hive/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 similarity index 100% rename from src/test/hive/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 rename to shark/src/test/hive/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 diff --git a/src/test/hive/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 b/shark/src/test/hive/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 similarity index 100% rename from src/test/hive/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 rename to shark/src/test/hive/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 diff --git a/src/test/hive/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 b/shark/src/test/hive/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 similarity index 100% rename from src/test/hive/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 rename to shark/src/test/hive/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 diff --git a/src/test/hive/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 b/shark/src/test/hive/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 similarity index 100% rename from src/test/hive/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 rename to shark/src/test/hive/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 diff --git a/src/test/hive/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 b/shark/src/test/hive/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 similarity index 100% rename from src/test/hive/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 rename to shark/src/test/hive/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 diff --git a/src/test/hive/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d b/shark/src/test/hive/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d similarity index 100% rename from src/test/hive/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d rename to shark/src/test/hive/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d diff --git a/src/test/hive/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 b/shark/src/test/hive/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 similarity index 100% rename from src/test/hive/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 rename to shark/src/test/hive/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 diff --git a/src/test/hive/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb b/shark/src/test/hive/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb similarity index 100% rename from src/test/hive/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb rename to shark/src/test/hive/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb diff --git a/src/test/hive/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a b/shark/src/test/hive/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a similarity index 100% rename from src/test/hive/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a rename to shark/src/test/hive/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a diff --git a/src/test/hive/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 b/shark/src/test/hive/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 similarity index 100% rename from src/test/hive/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 rename to shark/src/test/hive/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 diff --git a/src/test/hive/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b b/shark/src/test/hive/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b similarity index 100% rename from src/test/hive/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b rename to shark/src/test/hive/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b diff --git a/src/test/hive/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 b/shark/src/test/hive/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 similarity index 100% rename from src/test/hive/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 rename to shark/src/test/hive/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 diff --git a/src/test/hive/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 b/shark/src/test/hive/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 similarity index 100% rename from src/test/hive/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 rename to shark/src/test/hive/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 diff --git a/src/test/hive/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 b/shark/src/test/hive/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 similarity index 100% rename from src/test/hive/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 rename to shark/src/test/hive/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 diff --git a/src/test/hive/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 b/shark/src/test/hive/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 similarity index 100% rename from src/test/hive/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 rename to shark/src/test/hive/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 diff --git a/src/test/hive/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 b/shark/src/test/hive/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 similarity index 100% rename from src/test/hive/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 rename to shark/src/test/hive/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 diff --git a/src/test/hive/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d b/shark/src/test/hive/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d similarity index 100% rename from src/test/hive/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d rename to shark/src/test/hive/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d diff --git a/src/test/hive/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 b/shark/src/test/hive/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 similarity index 100% rename from src/test/hive/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 rename to shark/src/test/hive/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 diff --git a/src/test/hive/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 b/shark/src/test/hive/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 similarity index 100% rename from src/test/hive/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 rename to shark/src/test/hive/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 diff --git a/src/test/hive/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d b/shark/src/test/hive/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d similarity index 100% rename from src/test/hive/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d rename to shark/src/test/hive/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d diff --git a/src/test/hive/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a b/shark/src/test/hive/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a similarity index 100% rename from src/test/hive/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a rename to shark/src/test/hive/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a diff --git a/src/test/hive/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e b/shark/src/test/hive/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e similarity index 100% rename from src/test/hive/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e rename to shark/src/test/hive/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e diff --git a/src/test/hive/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd b/shark/src/test/hive/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd similarity index 100% rename from src/test/hive/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd rename to shark/src/test/hive/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd diff --git a/src/test/hive/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 b/shark/src/test/hive/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 similarity index 100% rename from src/test/hive/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 rename to shark/src/test/hive/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 diff --git a/src/test/hive/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 b/shark/src/test/hive/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 similarity index 100% rename from src/test/hive/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 rename to shark/src/test/hive/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 diff --git a/src/test/hive/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e b/shark/src/test/hive/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e similarity index 100% rename from src/test/hive/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e rename to shark/src/test/hive/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e diff --git a/src/test/hive/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 b/shark/src/test/hive/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 similarity index 100% rename from src/test/hive/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 rename to shark/src/test/hive/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 diff --git a/src/test/hive/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e b/shark/src/test/hive/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e similarity index 100% rename from src/test/hive/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e rename to shark/src/test/hive/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e diff --git a/src/test/hive/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd b/shark/src/test/hive/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd similarity index 100% rename from src/test/hive/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd rename to shark/src/test/hive/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd diff --git a/src/test/hive/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 b/shark/src/test/hive/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 similarity index 100% rename from src/test/hive/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 rename to shark/src/test/hive/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 diff --git a/src/test/hive/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 b/shark/src/test/hive/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 similarity index 100% rename from src/test/hive/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 rename to shark/src/test/hive/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 diff --git a/src/test/hive/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e b/shark/src/test/hive/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e similarity index 100% rename from src/test/hive/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e rename to shark/src/test/hive/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e diff --git a/src/test/hive/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c b/shark/src/test/hive/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c similarity index 100% rename from src/test/hive/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c rename to shark/src/test/hive/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c diff --git a/src/test/hive/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 b/shark/src/test/hive/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 similarity index 100% rename from src/test/hive/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 rename to shark/src/test/hive/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 diff --git a/src/test/hive/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a b/shark/src/test/hive/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a similarity index 100% rename from src/test/hive/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a rename to shark/src/test/hive/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a diff --git a/src/test/hive/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 b/shark/src/test/hive/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 similarity index 100% rename from src/test/hive/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 rename to shark/src/test/hive/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 diff --git a/src/test/hive/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f b/shark/src/test/hive/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f similarity index 100% rename from src/test/hive/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f rename to shark/src/test/hive/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f diff --git a/src/test/hive/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 b/shark/src/test/hive/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 similarity index 100% rename from src/test/hive/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 rename to shark/src/test/hive/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 diff --git a/src/test/hive/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b b/shark/src/test/hive/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b similarity index 100% rename from src/test/hive/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b rename to shark/src/test/hive/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b diff --git a/src/test/hive/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 b/shark/src/test/hive/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 similarity index 100% rename from src/test/hive/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 rename to shark/src/test/hive/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 diff --git a/src/test/hive/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 b/shark/src/test/hive/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 similarity index 100% rename from src/test/hive/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 rename to shark/src/test/hive/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 diff --git a/src/test/hive/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf b/shark/src/test/hive/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf similarity index 100% rename from src/test/hive/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf rename to shark/src/test/hive/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf diff --git a/src/test/hive/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 b/shark/src/test/hive/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 similarity index 100% rename from src/test/hive/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 rename to shark/src/test/hive/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 diff --git a/src/test/hive/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f b/shark/src/test/hive/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f similarity index 100% rename from src/test/hive/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f rename to shark/src/test/hive/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f diff --git a/src/test/hive/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 b/shark/src/test/hive/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 similarity index 100% rename from src/test/hive/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 rename to shark/src/test/hive/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 diff --git a/src/test/hive/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 b/shark/src/test/hive/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 similarity index 100% rename from src/test/hive/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 rename to shark/src/test/hive/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 diff --git a/src/test/hive/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 b/shark/src/test/hive/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 similarity index 100% rename from src/test/hive/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 rename to shark/src/test/hive/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 diff --git a/src/test/hive/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 b/shark/src/test/hive/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 similarity index 100% rename from src/test/hive/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 rename to shark/src/test/hive/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 diff --git a/src/test/hive/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 b/shark/src/test/hive/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 similarity index 100% rename from src/test/hive/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 rename to shark/src/test/hive/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 diff --git a/src/test/hive/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 b/shark/src/test/hive/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 similarity index 100% rename from src/test/hive/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 rename to shark/src/test/hive/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 diff --git a/src/test/hive/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 b/shark/src/test/hive/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 similarity index 100% rename from src/test/hive/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 rename to shark/src/test/hive/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 diff --git a/src/test/hive/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd b/shark/src/test/hive/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd similarity index 100% rename from src/test/hive/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd rename to shark/src/test/hive/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd diff --git a/src/test/hive/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 b/shark/src/test/hive/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 similarity index 100% rename from src/test/hive/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 rename to shark/src/test/hive/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 diff --git a/src/test/hive/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 b/shark/src/test/hive/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 similarity index 100% rename from src/test/hive/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 rename to shark/src/test/hive/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 diff --git a/src/test/hive/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 b/shark/src/test/hive/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 similarity index 100% rename from src/test/hive/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 rename to shark/src/test/hive/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 diff --git a/src/test/hive/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 b/shark/src/test/hive/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 similarity index 100% rename from src/test/hive/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 rename to shark/src/test/hive/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 diff --git a/src/test/hive/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 b/shark/src/test/hive/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 similarity index 100% rename from src/test/hive/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 rename to shark/src/test/hive/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 diff --git a/src/test/hive/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b b/shark/src/test/hive/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b similarity index 100% rename from src/test/hive/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b rename to shark/src/test/hive/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b diff --git a/src/test/hive/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 b/shark/src/test/hive/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 similarity index 100% rename from src/test/hive/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 rename to shark/src/test/hive/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 diff --git a/src/test/hive/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d b/shark/src/test/hive/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d similarity index 100% rename from src/test/hive/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d rename to shark/src/test/hive/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d diff --git a/src/test/hive/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 b/shark/src/test/hive/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 similarity index 100% rename from src/test/hive/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 rename to shark/src/test/hive/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 diff --git a/src/test/hive/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 b/shark/src/test/hive/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 similarity index 100% rename from src/test/hive/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 rename to shark/src/test/hive/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 diff --git a/src/test/hive/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df b/shark/src/test/hive/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df similarity index 100% rename from src/test/hive/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df rename to shark/src/test/hive/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df diff --git a/src/test/hive/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 b/shark/src/test/hive/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 similarity index 100% rename from src/test/hive/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 rename to shark/src/test/hive/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 diff --git a/src/test/hive/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 b/shark/src/test/hive/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 similarity index 100% rename from src/test/hive/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 rename to shark/src/test/hive/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 diff --git a/src/test/hive/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 b/shark/src/test/hive/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 similarity index 100% rename from src/test/hive/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 rename to shark/src/test/hive/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 diff --git a/src/test/hive/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d b/shark/src/test/hive/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d similarity index 100% rename from src/test/hive/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d rename to shark/src/test/hive/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d diff --git a/src/test/hive/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 b/shark/src/test/hive/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 similarity index 100% rename from src/test/hive/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 rename to shark/src/test/hive/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 diff --git a/src/test/hive/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 b/shark/src/test/hive/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 similarity index 100% rename from src/test/hive/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 rename to shark/src/test/hive/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 diff --git a/src/test/hive/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f b/shark/src/test/hive/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f similarity index 100% rename from src/test/hive/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f rename to shark/src/test/hive/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f diff --git a/src/test/hive/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 b/shark/src/test/hive/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 similarity index 100% rename from src/test/hive/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 rename to shark/src/test/hive/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 diff --git a/src/test/hive/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a b/shark/src/test/hive/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a similarity index 100% rename from src/test/hive/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a rename to shark/src/test/hive/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a diff --git a/src/test/hive/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 b/shark/src/test/hive/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 similarity index 100% rename from src/test/hive/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 rename to shark/src/test/hive/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 diff --git a/src/test/hive/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 b/shark/src/test/hive/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 similarity index 100% rename from src/test/hive/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 rename to shark/src/test/hive/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 diff --git a/src/test/hive/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 b/shark/src/test/hive/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 similarity index 100% rename from src/test/hive/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 rename to shark/src/test/hive/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 diff --git a/src/test/hive/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c b/shark/src/test/hive/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c similarity index 100% rename from src/test/hive/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c rename to shark/src/test/hive/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c diff --git a/src/test/hive/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a b/shark/src/test/hive/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a similarity index 100% rename from src/test/hive/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a rename to shark/src/test/hive/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a diff --git a/src/test/hive/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 b/shark/src/test/hive/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 similarity index 100% rename from src/test/hive/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 rename to shark/src/test/hive/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 diff --git a/src/test/hive/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 b/shark/src/test/hive/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 similarity index 100% rename from src/test/hive/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 rename to shark/src/test/hive/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 diff --git a/src/test/hive/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b b/shark/src/test/hive/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b similarity index 100% rename from src/test/hive/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b rename to shark/src/test/hive/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b diff --git a/src/test/hive/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db b/shark/src/test/hive/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db similarity index 100% rename from src/test/hive/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db rename to shark/src/test/hive/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db diff --git a/src/test/hive/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f b/shark/src/test/hive/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f similarity index 100% rename from src/test/hive/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f rename to shark/src/test/hive/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f diff --git a/src/test/hive/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a b/shark/src/test/hive/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a similarity index 100% rename from src/test/hive/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a rename to shark/src/test/hive/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a diff --git a/src/test/hive/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 b/shark/src/test/hive/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 similarity index 100% rename from src/test/hive/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 rename to shark/src/test/hive/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 diff --git a/src/test/hive/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b b/shark/src/test/hive/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b similarity index 100% rename from src/test/hive/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b rename to shark/src/test/hive/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b diff --git a/src/test/hive/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 b/shark/src/test/hive/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 similarity index 100% rename from src/test/hive/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 rename to shark/src/test/hive/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 diff --git a/src/test/hive/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb b/shark/src/test/hive/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb similarity index 100% rename from src/test/hive/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb rename to shark/src/test/hive/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb diff --git a/src/test/hive/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 b/shark/src/test/hive/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 similarity index 100% rename from src/test/hive/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 rename to shark/src/test/hive/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 diff --git a/src/test/hive/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 b/shark/src/test/hive/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 similarity index 100% rename from src/test/hive/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 rename to shark/src/test/hive/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 diff --git a/src/test/hive/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e b/shark/src/test/hive/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e similarity index 100% rename from src/test/hive/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e rename to shark/src/test/hive/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e diff --git a/src/test/hive/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 b/shark/src/test/hive/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 similarity index 100% rename from src/test/hive/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 rename to shark/src/test/hive/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 diff --git a/src/test/hive/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 b/shark/src/test/hive/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 similarity index 100% rename from src/test/hive/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 rename to shark/src/test/hive/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 diff --git a/src/test/hive/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f b/shark/src/test/hive/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f similarity index 100% rename from src/test/hive/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f rename to shark/src/test/hive/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f diff --git a/src/test/hive/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 b/shark/src/test/hive/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 similarity index 100% rename from src/test/hive/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 rename to shark/src/test/hive/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 diff --git a/src/test/hive/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 b/shark/src/test/hive/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 similarity index 100% rename from src/test/hive/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 rename to shark/src/test/hive/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 diff --git a/src/test/hive/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 b/shark/src/test/hive/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 similarity index 100% rename from src/test/hive/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 rename to shark/src/test/hive/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 diff --git a/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc b/shark/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc similarity index 100% rename from src/test/hive/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc rename to shark/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc diff --git a/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 b/shark/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 similarity index 100% rename from src/test/hive/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 rename to shark/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 diff --git a/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 b/shark/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 similarity index 100% rename from src/test/hive/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 rename to shark/src/test/hive/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 diff --git a/src/test/hive/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 b/shark/src/test/hive/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 similarity index 100% rename from src/test/hive/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 rename to shark/src/test/hive/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 diff --git a/src/test/hive/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/shark/src/test/hive/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e similarity index 100% rename from src/test/hive/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e rename to shark/src/test/hive/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e diff --git a/src/test/hive/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 b/shark/src/test/hive/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 similarity index 100% rename from src/test/hive/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 rename to shark/src/test/hive/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 diff --git a/src/test/hive/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 b/shark/src/test/hive/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 similarity index 100% rename from src/test/hive/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 rename to shark/src/test/hive/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 diff --git a/src/test/hive/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/shark/src/test/hive/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e similarity index 100% rename from src/test/hive/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e rename to shark/src/test/hive/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e diff --git a/src/test/hive/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 b/shark/src/test/hive/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 similarity index 100% rename from src/test/hive/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 rename to shark/src/test/hive/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 diff --git a/src/test/hive/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 b/shark/src/test/hive/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 similarity index 100% rename from src/test/hive/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 rename to shark/src/test/hive/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 diff --git a/src/test/hive/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c b/shark/src/test/hive/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c similarity index 100% rename from src/test/hive/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c rename to shark/src/test/hive/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c diff --git a/src/test/hive/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 b/shark/src/test/hive/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 similarity index 100% rename from src/test/hive/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 rename to shark/src/test/hive/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 diff --git a/src/test/hive/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 b/shark/src/test/hive/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 similarity index 100% rename from src/test/hive/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 rename to shark/src/test/hive/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 diff --git a/src/test/hive/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 b/shark/src/test/hive/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 similarity index 100% rename from src/test/hive/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 rename to shark/src/test/hive/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 diff --git a/src/test/hive/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc b/shark/src/test/hive/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc similarity index 100% rename from src/test/hive/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc rename to shark/src/test/hive/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc diff --git a/src/test/hive/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 b/shark/src/test/hive/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 similarity index 100% rename from src/test/hive/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 rename to shark/src/test/hive/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 diff --git a/src/test/hive/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 b/shark/src/test/hive/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 similarity index 100% rename from src/test/hive/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 rename to shark/src/test/hive/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 diff --git a/src/test/hive/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 b/shark/src/test/hive/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 similarity index 100% rename from src/test/hive/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 rename to shark/src/test/hive/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 diff --git a/src/test/hive/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 b/shark/src/test/hive/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 similarity index 100% rename from src/test/hive/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 rename to shark/src/test/hive/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 diff --git a/src/test/hive/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 b/shark/src/test/hive/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 similarity index 100% rename from src/test/hive/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 rename to shark/src/test/hive/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 diff --git a/src/test/hive/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 b/shark/src/test/hive/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 similarity index 100% rename from src/test/hive/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 rename to shark/src/test/hive/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 diff --git a/src/test/hive/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e b/shark/src/test/hive/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e similarity index 100% rename from src/test/hive/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e rename to shark/src/test/hive/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e diff --git a/src/test/hive/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 b/shark/src/test/hive/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 similarity index 100% rename from src/test/hive/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 rename to shark/src/test/hive/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 diff --git a/src/test/hive/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 b/shark/src/test/hive/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 similarity index 100% rename from src/test/hive/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 rename to shark/src/test/hive/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 diff --git a/src/test/hive/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 b/shark/src/test/hive/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 similarity index 100% rename from src/test/hive/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 rename to shark/src/test/hive/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 diff --git a/src/test/hive/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 b/shark/src/test/hive/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 similarity index 100% rename from src/test/hive/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 rename to shark/src/test/hive/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 diff --git a/src/test/hive/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 b/shark/src/test/hive/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 similarity index 100% rename from src/test/hive/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 rename to shark/src/test/hive/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 diff --git a/src/test/hive/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 b/shark/src/test/hive/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 similarity index 100% rename from src/test/hive/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 rename to shark/src/test/hive/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 diff --git a/src/test/hive/golden/lock2-3-27ad2962fed131f51ba802596ba37278 b/shark/src/test/hive/golden/lock2-3-27ad2962fed131f51ba802596ba37278 similarity index 100% rename from src/test/hive/golden/lock2-3-27ad2962fed131f51ba802596ba37278 rename to shark/src/test/hive/golden/lock2-3-27ad2962fed131f51ba802596ba37278 diff --git a/src/test/hive/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a b/shark/src/test/hive/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a similarity index 100% rename from src/test/hive/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a rename to shark/src/test/hive/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a diff --git a/src/test/hive/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca b/shark/src/test/hive/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca similarity index 100% rename from src/test/hive/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca rename to shark/src/test/hive/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca diff --git a/src/test/hive/golden/lock3-0-27ad2962fed131f51ba802596ba37278 b/shark/src/test/hive/golden/lock3-0-27ad2962fed131f51ba802596ba37278 similarity index 100% rename from src/test/hive/golden/lock3-0-27ad2962fed131f51ba802596ba37278 rename to shark/src/test/hive/golden/lock3-0-27ad2962fed131f51ba802596ba37278 diff --git a/src/test/hive/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a b/shark/src/test/hive/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a similarity index 100% rename from src/test/hive/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a rename to shark/src/test/hive/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a diff --git a/src/test/hive/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 b/shark/src/test/hive/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 similarity index 100% rename from src/test/hive/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 rename to shark/src/test/hive/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 diff --git a/src/test/hive/golden/lock3-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/lock3-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/lock3-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/lock3-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac b/shark/src/test/hive/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac similarity index 100% rename from src/test/hive/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac rename to shark/src/test/hive/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac diff --git a/src/test/hive/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe b/shark/src/test/hive/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe similarity index 100% rename from src/test/hive/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe rename to shark/src/test/hive/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe diff --git a/src/test/hive/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 b/shark/src/test/hive/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 similarity index 100% rename from src/test/hive/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 rename to shark/src/test/hive/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 diff --git a/src/test/hive/golden/lock4-1-27ad2962fed131f51ba802596ba37278 b/shark/src/test/hive/golden/lock4-1-27ad2962fed131f51ba802596ba37278 similarity index 100% rename from src/test/hive/golden/lock4-1-27ad2962fed131f51ba802596ba37278 rename to shark/src/test/hive/golden/lock4-1-27ad2962fed131f51ba802596ba37278 diff --git a/src/test/hive/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a b/shark/src/test/hive/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a similarity index 100% rename from src/test/hive/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a rename to shark/src/test/hive/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a diff --git a/src/test/hive/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 b/shark/src/test/hive/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 similarity index 100% rename from src/test/hive/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 rename to shark/src/test/hive/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 diff --git a/src/test/hive/golden/lock4-4-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/lock4-4-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/lock4-4-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/lock4-4-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac b/shark/src/test/hive/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac similarity index 100% rename from src/test/hive/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac rename to shark/src/test/hive/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac diff --git a/src/test/hive/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe b/shark/src/test/hive/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe similarity index 100% rename from src/test/hive/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe rename to shark/src/test/hive/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe diff --git a/src/test/hive/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd b/shark/src/test/hive/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd similarity index 100% rename from src/test/hive/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd rename to shark/src/test/hive/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd diff --git a/src/test/hive/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 b/shark/src/test/hive/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 similarity index 100% rename from src/test/hive/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 rename to shark/src/test/hive/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 diff --git a/src/test/hive/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 b/shark/src/test/hive/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 similarity index 100% rename from src/test/hive/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 rename to shark/src/test/hive/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 diff --git a/src/test/hive/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef b/shark/src/test/hive/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef similarity index 100% rename from src/test/hive/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef rename to shark/src/test/hive/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef diff --git a/src/test/hive/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 b/shark/src/test/hive/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 similarity index 100% rename from src/test/hive/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 rename to shark/src/test/hive/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 diff --git a/src/test/hive/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 b/shark/src/test/hive/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 similarity index 100% rename from src/test/hive/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 rename to shark/src/test/hive/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 diff --git a/src/test/hive/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 b/shark/src/test/hive/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 similarity index 100% rename from src/test/hive/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 rename to shark/src/test/hive/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 diff --git a/src/test/hive/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 b/shark/src/test/hive/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 similarity index 100% rename from src/test/hive/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 rename to shark/src/test/hive/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 diff --git a/src/test/hive/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d b/shark/src/test/hive/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d similarity index 100% rename from src/test/hive/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d rename to shark/src/test/hive/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d diff --git a/src/test/hive/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 b/shark/src/test/hive/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 similarity index 100% rename from src/test/hive/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 rename to shark/src/test/hive/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 diff --git a/src/test/hive/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 b/shark/src/test/hive/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 similarity index 100% rename from src/test/hive/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 rename to shark/src/test/hive/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 diff --git a/src/test/hive/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 b/shark/src/test/hive/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 similarity index 100% rename from src/test/hive/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 rename to shark/src/test/hive/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 diff --git a/src/test/hive/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 b/shark/src/test/hive/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 similarity index 100% rename from src/test/hive/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 rename to shark/src/test/hive/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 diff --git a/src/test/hive/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 b/shark/src/test/hive/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 similarity index 100% rename from src/test/hive/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 rename to shark/src/test/hive/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 diff --git a/src/test/hive/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d b/shark/src/test/hive/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d similarity index 100% rename from src/test/hive/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d rename to shark/src/test/hive/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d diff --git a/src/test/hive/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 b/shark/src/test/hive/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 similarity index 100% rename from src/test/hive/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 rename to shark/src/test/hive/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 diff --git a/src/test/hive/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba b/shark/src/test/hive/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba similarity index 100% rename from src/test/hive/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba rename to shark/src/test/hive/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba diff --git a/src/test/hive/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 b/shark/src/test/hive/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 similarity index 100% rename from src/test/hive/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 rename to shark/src/test/hive/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 diff --git a/src/test/hive/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 b/shark/src/test/hive/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 similarity index 100% rename from src/test/hive/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 rename to shark/src/test/hive/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 diff --git a/src/test/hive/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 b/shark/src/test/hive/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 similarity index 100% rename from src/test/hive/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 rename to shark/src/test/hive/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 diff --git a/src/test/hive/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 b/shark/src/test/hive/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 similarity index 100% rename from src/test/hive/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 rename to shark/src/test/hive/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 diff --git a/src/test/hive/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 b/shark/src/test/hive/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 rename to shark/src/test/hive/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/src/test/hive/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 b/shark/src/test/hive/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 rename to shark/src/test/hive/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 diff --git a/src/test/hive/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f b/shark/src/test/hive/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f rename to shark/src/test/hive/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f diff --git a/src/test/hive/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 b/shark/src/test/hive/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 rename to shark/src/test/hive/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/src/test/hive/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e b/shark/src/test/hive/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e rename to shark/src/test/hive/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e diff --git a/src/test/hive/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 b/shark/src/test/hive/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 rename to shark/src/test/hive/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 diff --git a/src/test/hive/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f b/shark/src/test/hive/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f rename to shark/src/test/hive/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f diff --git a/src/test/hive/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 b/shark/src/test/hive/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 rename to shark/src/test/hive/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 diff --git a/src/test/hive/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f b/shark/src/test/hive/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f rename to shark/src/test/hive/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f diff --git a/src/test/hive/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e b/shark/src/test/hive/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e rename to shark/src/test/hive/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e diff --git a/src/test/hive/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 b/shark/src/test/hive/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 rename to shark/src/test/hive/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 diff --git a/src/test/hive/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f b/shark/src/test/hive/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f rename to shark/src/test/hive/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f diff --git a/src/test/hive/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 b/shark/src/test/hive/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 rename to shark/src/test/hive/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/src/test/hive/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 b/shark/src/test/hive/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from src/test/hive/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 rename to shark/src/test/hive/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/src/test/hive/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d b/shark/src/test/hive/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d similarity index 100% rename from src/test/hive/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d rename to shark/src/test/hive/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d diff --git a/src/test/hive/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 b/shark/src/test/hive/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 similarity index 100% rename from src/test/hive/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 rename to shark/src/test/hive/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 diff --git a/src/test/hive/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 b/shark/src/test/hive/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 similarity index 100% rename from src/test/hive/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 rename to shark/src/test/hive/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 diff --git a/src/test/hive/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 b/shark/src/test/hive/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 similarity index 100% rename from src/test/hive/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 rename to shark/src/test/hive/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 diff --git a/src/test/hive/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b b/shark/src/test/hive/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b similarity index 100% rename from src/test/hive/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b rename to shark/src/test/hive/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b diff --git a/src/test/hive/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 b/shark/src/test/hive/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 rename to shark/src/test/hive/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 diff --git a/src/test/hive/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c b/shark/src/test/hive/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c similarity index 100% rename from src/test/hive/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c rename to shark/src/test/hive/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c diff --git a/src/test/hive/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b b/shark/src/test/hive/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b similarity index 100% rename from src/test/hive/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b rename to shark/src/test/hive/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b diff --git a/src/test/hive/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 b/shark/src/test/hive/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 rename to shark/src/test/hive/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 diff --git a/src/test/hive/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 b/shark/src/test/hive/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 rename to shark/src/test/hive/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 diff --git a/src/test/hive/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 b/shark/src/test/hive/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 rename to shark/src/test/hive/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 diff --git a/src/test/hive/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 b/shark/src/test/hive/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 rename to shark/src/test/hive/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 diff --git a/src/test/hive/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 b/shark/src/test/hive/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 rename to shark/src/test/hive/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 diff --git a/src/test/hive/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 b/shark/src/test/hive/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 rename to shark/src/test/hive/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 diff --git a/src/test/hive/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 b/shark/src/test/hive/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 rename to shark/src/test/hive/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 diff --git a/src/test/hive/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 b/shark/src/test/hive/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 rename to shark/src/test/hive/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 diff --git a/src/test/hive/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 b/shark/src/test/hive/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 rename to shark/src/test/hive/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 diff --git a/src/test/hive/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 b/shark/src/test/hive/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 rename to shark/src/test/hive/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 diff --git a/src/test/hive/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 b/shark/src/test/hive/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 rename to shark/src/test/hive/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 diff --git a/src/test/hive/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 b/shark/src/test/hive/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 rename to shark/src/test/hive/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 diff --git a/src/test/hive/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 b/shark/src/test/hive/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 rename to shark/src/test/hive/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 diff --git a/src/test/hive/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e b/shark/src/test/hive/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e rename to shark/src/test/hive/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e diff --git a/src/test/hive/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d b/shark/src/test/hive/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d rename to shark/src/test/hive/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d diff --git a/src/test/hive/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc b/shark/src/test/hive/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc rename to shark/src/test/hive/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc diff --git a/src/test/hive/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a b/shark/src/test/hive/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a rename to shark/src/test/hive/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a diff --git a/src/test/hive/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 b/shark/src/test/hive/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 rename to shark/src/test/hive/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 diff --git a/src/test/hive/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 b/shark/src/test/hive/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 rename to shark/src/test/hive/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 diff --git a/src/test/hive/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 b/shark/src/test/hive/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 rename to shark/src/test/hive/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 diff --git a/src/test/hive/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 b/shark/src/test/hive/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 rename to shark/src/test/hive/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 diff --git a/src/test/hive/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 b/shark/src/test/hive/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 rename to shark/src/test/hive/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 diff --git a/src/test/hive/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f b/shark/src/test/hive/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f rename to shark/src/test/hive/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f diff --git a/src/test/hive/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 b/shark/src/test/hive/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 rename to shark/src/test/hive/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 diff --git a/src/test/hive/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 b/shark/src/test/hive/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 rename to shark/src/test/hive/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 diff --git a/src/test/hive/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 b/shark/src/test/hive/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 rename to shark/src/test/hive/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 diff --git a/src/test/hive/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 b/shark/src/test/hive/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 rename to shark/src/test/hive/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 diff --git a/src/test/hive/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a b/shark/src/test/hive/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a rename to shark/src/test/hive/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a diff --git a/src/test/hive/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 b/shark/src/test/hive/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 similarity index 100% rename from src/test/hive/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 rename to shark/src/test/hive/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 diff --git a/src/test/hive/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c b/shark/src/test/hive/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c similarity index 100% rename from src/test/hive/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c rename to shark/src/test/hive/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c diff --git a/src/test/hive/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe b/shark/src/test/hive/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe similarity index 100% rename from src/test/hive/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe rename to shark/src/test/hive/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe diff --git a/src/test/hive/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e b/shark/src/test/hive/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e similarity index 100% rename from src/test/hive/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e rename to shark/src/test/hive/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e diff --git a/src/test/hive/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c b/shark/src/test/hive/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c similarity index 100% rename from src/test/hive/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c rename to shark/src/test/hive/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c diff --git a/src/test/hive/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 b/shark/src/test/hive/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 similarity index 100% rename from src/test/hive/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 rename to shark/src/test/hive/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 diff --git a/src/test/hive/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b b/shark/src/test/hive/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b similarity index 100% rename from src/test/hive/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b rename to shark/src/test/hive/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b diff --git a/src/test/hive/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc b/shark/src/test/hive/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc similarity index 100% rename from src/test/hive/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc rename to shark/src/test/hive/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc diff --git a/src/test/hive/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c b/shark/src/test/hive/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c similarity index 100% rename from src/test/hive/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c rename to shark/src/test/hive/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c diff --git a/src/test/hive/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b b/shark/src/test/hive/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b similarity index 100% rename from src/test/hive/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b rename to shark/src/test/hive/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b diff --git a/src/test/hive/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a b/shark/src/test/hive/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a similarity index 100% rename from src/test/hive/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a rename to shark/src/test/hive/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a diff --git a/src/test/hive/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c b/shark/src/test/hive/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c similarity index 100% rename from src/test/hive/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c rename to shark/src/test/hive/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c diff --git a/src/test/hive/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f b/shark/src/test/hive/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f similarity index 100% rename from src/test/hive/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f rename to shark/src/test/hive/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f diff --git a/src/test/hive/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab b/shark/src/test/hive/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab similarity index 100% rename from src/test/hive/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab rename to shark/src/test/hive/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab diff --git a/src/test/hive/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c b/shark/src/test/hive/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c similarity index 100% rename from src/test/hive/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c rename to shark/src/test/hive/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c diff --git a/src/test/hive/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 b/shark/src/test/hive/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 similarity index 100% rename from src/test/hive/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 rename to shark/src/test/hive/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 diff --git a/src/test/hive/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca b/shark/src/test/hive/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca similarity index 100% rename from src/test/hive/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca rename to shark/src/test/hive/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca diff --git a/src/test/hive/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c b/shark/src/test/hive/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c similarity index 100% rename from src/test/hive/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c rename to shark/src/test/hive/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c diff --git a/src/test/hive/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb b/shark/src/test/hive/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb similarity index 100% rename from src/test/hive/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb rename to shark/src/test/hive/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb diff --git a/src/test/hive/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 b/shark/src/test/hive/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 similarity index 100% rename from src/test/hive/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 rename to shark/src/test/hive/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 diff --git a/src/test/hive/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 b/shark/src/test/hive/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 similarity index 100% rename from src/test/hive/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 rename to shark/src/test/hive/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 diff --git a/src/test/hive/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a b/shark/src/test/hive/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a similarity index 100% rename from src/test/hive/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a rename to shark/src/test/hive/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a diff --git a/src/test/hive/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 b/shark/src/test/hive/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 similarity index 100% rename from src/test/hive/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 rename to shark/src/test/hive/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 diff --git a/src/test/hive/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 b/shark/src/test/hive/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 similarity index 100% rename from src/test/hive/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 rename to shark/src/test/hive/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 diff --git a/src/test/hive/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b b/shark/src/test/hive/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b similarity index 100% rename from src/test/hive/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b rename to shark/src/test/hive/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b diff --git a/src/test/hive/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 b/shark/src/test/hive/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 similarity index 100% rename from src/test/hive/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 rename to shark/src/test/hive/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 diff --git a/src/test/hive/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/merge1-1-2c73c923962b91afdf0004a705432550 b/shark/src/test/hive/golden/merge1-1-2c73c923962b91afdf0004a705432550 similarity index 100% rename from src/test/hive/golden/merge1-1-2c73c923962b91afdf0004a705432550 rename to shark/src/test/hive/golden/merge1-1-2c73c923962b91afdf0004a705432550 diff --git a/src/test/hive/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 b/shark/src/test/hive/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 similarity index 100% rename from src/test/hive/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 rename to shark/src/test/hive/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 diff --git a/src/test/hive/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 b/shark/src/test/hive/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 similarity index 100% rename from src/test/hive/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 rename to shark/src/test/hive/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 diff --git a/src/test/hive/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 b/shark/src/test/hive/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 similarity index 100% rename from src/test/hive/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 rename to shark/src/test/hive/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 diff --git a/src/test/hive/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 b/shark/src/test/hive/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 similarity index 100% rename from src/test/hive/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 rename to shark/src/test/hive/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 diff --git a/src/test/hive/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 b/shark/src/test/hive/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 similarity index 100% rename from src/test/hive/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 rename to shark/src/test/hive/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 diff --git a/src/test/hive/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 b/shark/src/test/hive/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 similarity index 100% rename from src/test/hive/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 rename to shark/src/test/hive/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 diff --git a/src/test/hive/golden/merge1-3-1f560722f18ef618a0343313a0cac462 b/shark/src/test/hive/golden/merge1-3-1f560722f18ef618a0343313a0cac462 similarity index 100% rename from src/test/hive/golden/merge1-3-1f560722f18ef618a0343313a0cac462 rename to shark/src/test/hive/golden/merge1-3-1f560722f18ef618a0343313a0cac462 diff --git a/src/test/hive/golden/merge1-4-3277fe538b66923cd879b45371838d2b b/shark/src/test/hive/golden/merge1-4-3277fe538b66923cd879b45371838d2b similarity index 100% rename from src/test/hive/golden/merge1-4-3277fe538b66923cd879b45371838d2b rename to shark/src/test/hive/golden/merge1-4-3277fe538b66923cd879b45371838d2b diff --git a/src/test/hive/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 b/shark/src/test/hive/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 similarity index 100% rename from src/test/hive/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 rename to shark/src/test/hive/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 diff --git a/src/test/hive/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 b/shark/src/test/hive/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 similarity index 100% rename from src/test/hive/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 rename to shark/src/test/hive/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 diff --git a/src/test/hive/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 b/shark/src/test/hive/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 similarity index 100% rename from src/test/hive/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 rename to shark/src/test/hive/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 diff --git a/src/test/hive/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a b/shark/src/test/hive/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a similarity index 100% rename from src/test/hive/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a rename to shark/src/test/hive/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a diff --git a/src/test/hive/golden/merge1-9-64678b0928c4fd054e1578458001c86 b/shark/src/test/hive/golden/merge1-9-64678b0928c4fd054e1578458001c86 similarity index 100% rename from src/test/hive/golden/merge1-9-64678b0928c4fd054e1578458001c86 rename to shark/src/test/hive/golden/merge1-9-64678b0928c4fd054e1578458001c86 diff --git a/src/test/hive/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 b/shark/src/test/hive/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 similarity index 100% rename from src/test/hive/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 rename to shark/src/test/hive/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 diff --git a/src/test/hive/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 b/shark/src/test/hive/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 similarity index 100% rename from src/test/hive/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 rename to shark/src/test/hive/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 diff --git a/src/test/hive/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 b/shark/src/test/hive/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 similarity index 100% rename from src/test/hive/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 rename to shark/src/test/hive/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 diff --git a/src/test/hive/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a b/shark/src/test/hive/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a similarity index 100% rename from src/test/hive/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a rename to shark/src/test/hive/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a diff --git a/src/test/hive/golden/merge2-14-64678b0928c4fd054e1578458001c86 b/shark/src/test/hive/golden/merge2-14-64678b0928c4fd054e1578458001c86 similarity index 100% rename from src/test/hive/golden/merge2-14-64678b0928c4fd054e1578458001c86 rename to shark/src/test/hive/golden/merge2-14-64678b0928c4fd054e1578458001c86 diff --git a/src/test/hive/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 b/shark/src/test/hive/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 similarity index 100% rename from src/test/hive/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 rename to shark/src/test/hive/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 diff --git a/src/test/hive/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 b/shark/src/test/hive/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 similarity index 100% rename from src/test/hive/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 rename to shark/src/test/hive/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 diff --git a/src/test/hive/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 b/shark/src/test/hive/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 similarity index 100% rename from src/test/hive/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 rename to shark/src/test/hive/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 diff --git a/src/test/hive/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f b/shark/src/test/hive/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f similarity index 100% rename from src/test/hive/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f rename to shark/src/test/hive/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f diff --git a/src/test/hive/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 b/shark/src/test/hive/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 similarity index 100% rename from src/test/hive/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 rename to shark/src/test/hive/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 diff --git a/src/test/hive/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf b/shark/src/test/hive/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf similarity index 100% rename from src/test/hive/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf rename to shark/src/test/hive/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf diff --git a/src/test/hive/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/shark/src/test/hive/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 similarity index 100% rename from src/test/hive/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 rename to shark/src/test/hive/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 diff --git a/src/test/hive/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 b/shark/src/test/hive/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 similarity index 100% rename from src/test/hive/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 rename to shark/src/test/hive/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 diff --git a/src/test/hive/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/shark/src/test/hive/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea similarity index 100% rename from src/test/hive/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea rename to shark/src/test/hive/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea diff --git a/src/test/hive/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 b/shark/src/test/hive/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 similarity index 100% rename from src/test/hive/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 rename to shark/src/test/hive/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 diff --git a/src/test/hive/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 b/shark/src/test/hive/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 similarity index 100% rename from src/test/hive/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 rename to shark/src/test/hive/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 diff --git a/src/test/hive/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 b/shark/src/test/hive/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 similarity index 100% rename from src/test/hive/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 rename to shark/src/test/hive/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 diff --git a/src/test/hive/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd b/shark/src/test/hive/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd similarity index 100% rename from src/test/hive/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd rename to shark/src/test/hive/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd diff --git a/src/test/hive/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/merge4-10-692a197bd688b48f762e72978f54aa32 b/shark/src/test/hive/golden/merge4-10-692a197bd688b48f762e72978f54aa32 similarity index 100% rename from src/test/hive/golden/merge4-10-692a197bd688b48f762e72978f54aa32 rename to shark/src/test/hive/golden/merge4-10-692a197bd688b48f762e72978f54aa32 diff --git a/src/test/hive/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 b/shark/src/test/hive/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 similarity index 100% rename from src/test/hive/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 rename to shark/src/test/hive/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 diff --git a/src/test/hive/golden/merge4-12-62541540a18d68a3cb8497a741061d11 b/shark/src/test/hive/golden/merge4-12-62541540a18d68a3cb8497a741061d11 similarity index 100% rename from src/test/hive/golden/merge4-12-62541540a18d68a3cb8497a741061d11 rename to shark/src/test/hive/golden/merge4-12-62541540a18d68a3cb8497a741061d11 diff --git a/src/test/hive/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 b/shark/src/test/hive/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 similarity index 100% rename from src/test/hive/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 rename to shark/src/test/hive/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 diff --git a/src/test/hive/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 b/shark/src/test/hive/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 similarity index 100% rename from src/test/hive/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 rename to shark/src/test/hive/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 diff --git a/src/test/hive/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a b/shark/src/test/hive/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a similarity index 100% rename from src/test/hive/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a rename to shark/src/test/hive/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a diff --git a/src/test/hive/golden/merge4-2-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/merge4-2-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/merge4-2-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/merge4-2-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/merge4-4-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/merge4-4-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/merge4-4-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/merge4-4-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/merge4-5-3d24d877366c42030f6d9a596665720d b/shark/src/test/hive/golden/merge4-5-3d24d877366c42030f6d9a596665720d similarity index 100% rename from src/test/hive/golden/merge4-5-3d24d877366c42030f6d9a596665720d rename to shark/src/test/hive/golden/merge4-5-3d24d877366c42030f6d9a596665720d diff --git a/src/test/hive/golden/merge4-6-b3a76420183795720ab3a384046e5af b/shark/src/test/hive/golden/merge4-6-b3a76420183795720ab3a384046e5af similarity index 100% rename from src/test/hive/golden/merge4-6-b3a76420183795720ab3a384046e5af rename to shark/src/test/hive/golden/merge4-6-b3a76420183795720ab3a384046e5af diff --git a/src/test/hive/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d b/shark/src/test/hive/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d similarity index 100% rename from src/test/hive/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d rename to shark/src/test/hive/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d diff --git a/src/test/hive/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 b/shark/src/test/hive/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 similarity index 100% rename from src/test/hive/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 rename to shark/src/test/hive/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 diff --git a/src/test/hive/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 b/shark/src/test/hive/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 similarity index 100% rename from src/test/hive/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 rename to shark/src/test/hive/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 diff --git a/src/test/hive/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 b/shark/src/test/hive/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 rename to shark/src/test/hive/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 diff --git a/src/test/hive/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 b/shark/src/test/hive/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 rename to shark/src/test/hive/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 diff --git a/src/test/hive/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da b/shark/src/test/hive/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da rename to shark/src/test/hive/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da diff --git a/src/test/hive/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 b/shark/src/test/hive/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 rename to shark/src/test/hive/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 diff --git a/src/test/hive/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 b/shark/src/test/hive/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 rename to shark/src/test/hive/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 diff --git a/src/test/hive/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 b/shark/src/test/hive/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 rename to shark/src/test/hive/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 diff --git a/src/test/hive/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 b/shark/src/test/hive/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 rename to shark/src/test/hive/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 diff --git a/src/test/hive/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 b/shark/src/test/hive/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 rename to shark/src/test/hive/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 diff --git a/src/test/hive/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 b/shark/src/test/hive/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 rename to shark/src/test/hive/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 diff --git a/src/test/hive/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 b/shark/src/test/hive/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 rename to shark/src/test/hive/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 diff --git a/src/test/hive/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 b/shark/src/test/hive/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 rename to shark/src/test/hive/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 diff --git a/src/test/hive/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 b/shark/src/test/hive/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 rename to shark/src/test/hive/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 diff --git a/src/test/hive/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 b/shark/src/test/hive/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 rename to shark/src/test/hive/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 diff --git a/src/test/hive/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d b/shark/src/test/hive/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d rename to shark/src/test/hive/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d diff --git a/src/test/hive/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f b/shark/src/test/hive/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f rename to shark/src/test/hive/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f diff --git a/src/test/hive/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 b/shark/src/test/hive/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 rename to shark/src/test/hive/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 diff --git a/src/test/hive/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 b/shark/src/test/hive/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 rename to shark/src/test/hive/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 diff --git a/src/test/hive/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf b/shark/src/test/hive/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf rename to shark/src/test/hive/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf diff --git a/src/test/hive/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 b/shark/src/test/hive/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 rename to shark/src/test/hive/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 diff --git a/src/test/hive/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 b/shark/src/test/hive/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 rename to shark/src/test/hive/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 diff --git a/src/test/hive/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b b/shark/src/test/hive/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b rename to shark/src/test/hive/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b diff --git a/src/test/hive/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 b/shark/src/test/hive/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 rename to shark/src/test/hive/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 diff --git a/src/test/hive/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 b/shark/src/test/hive/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 rename to shark/src/test/hive/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 diff --git a/src/test/hive/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 b/shark/src/test/hive/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 rename to shark/src/test/hive/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 diff --git a/src/test/hive/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 b/shark/src/test/hive/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 rename to shark/src/test/hive/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 diff --git a/src/test/hive/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da b/shark/src/test/hive/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da rename to shark/src/test/hive/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da diff --git a/src/test/hive/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 b/shark/src/test/hive/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 rename to shark/src/test/hive/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 diff --git a/src/test/hive/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 b/shark/src/test/hive/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 rename to shark/src/test/hive/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 diff --git a/src/test/hive/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f b/shark/src/test/hive/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f rename to shark/src/test/hive/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f diff --git a/src/test/hive/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf b/shark/src/test/hive/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf rename to shark/src/test/hive/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf diff --git a/src/test/hive/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 b/shark/src/test/hive/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 rename to shark/src/test/hive/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 diff --git a/src/test/hive/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 b/shark/src/test/hive/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 rename to shark/src/test/hive/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 diff --git a/src/test/hive/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b b/shark/src/test/hive/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b rename to shark/src/test/hive/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b diff --git a/src/test/hive/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 b/shark/src/test/hive/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 rename to shark/src/test/hive/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 diff --git a/src/test/hive/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba b/shark/src/test/hive/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba rename to shark/src/test/hive/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba diff --git a/src/test/hive/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d b/shark/src/test/hive/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d rename to shark/src/test/hive/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d diff --git a/src/test/hive/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 b/shark/src/test/hive/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 rename to shark/src/test/hive/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 diff --git a/src/test/hive/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a b/shark/src/test/hive/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a rename to shark/src/test/hive/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a diff --git a/src/test/hive/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 b/shark/src/test/hive/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 rename to shark/src/test/hive/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 diff --git a/src/test/hive/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 b/shark/src/test/hive/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 rename to shark/src/test/hive/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 diff --git a/src/test/hive/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 b/shark/src/test/hive/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 rename to shark/src/test/hive/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 diff --git a/src/test/hive/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 b/shark/src/test/hive/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 rename to shark/src/test/hive/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 diff --git a/src/test/hive/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 b/shark/src/test/hive/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 rename to shark/src/test/hive/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 diff --git a/src/test/hive/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 b/shark/src/test/hive/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 rename to shark/src/test/hive/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 diff --git a/src/test/hive/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 b/shark/src/test/hive/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 rename to shark/src/test/hive/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 diff --git a/src/test/hive/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 b/shark/src/test/hive/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 rename to shark/src/test/hive/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 diff --git a/src/test/hive/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 b/shark/src/test/hive/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 rename to shark/src/test/hive/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 diff --git a/src/test/hive/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a b/shark/src/test/hive/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a rename to shark/src/test/hive/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a diff --git a/src/test/hive/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 b/shark/src/test/hive/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 rename to shark/src/test/hive/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 diff --git a/src/test/hive/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f b/shark/src/test/hive/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f rename to shark/src/test/hive/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f diff --git a/src/test/hive/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf b/shark/src/test/hive/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf rename to shark/src/test/hive/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf diff --git a/src/test/hive/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 b/shark/src/test/hive/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 rename to shark/src/test/hive/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 diff --git a/src/test/hive/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 b/shark/src/test/hive/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 rename to shark/src/test/hive/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 diff --git a/src/test/hive/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b b/shark/src/test/hive/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b rename to shark/src/test/hive/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b diff --git a/src/test/hive/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c b/shark/src/test/hive/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c rename to shark/src/test/hive/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c diff --git a/src/test/hive/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 b/shark/src/test/hive/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 similarity index 100% rename from src/test/hive/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 rename to shark/src/test/hive/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 diff --git a/src/test/hive/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 b/shark/src/test/hive/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 similarity index 100% rename from src/test/hive/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 rename to shark/src/test/hive/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 diff --git a/src/test/hive/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b b/shark/src/test/hive/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b similarity index 100% rename from src/test/hive/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b rename to shark/src/test/hive/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b diff --git a/src/test/hive/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c b/shark/src/test/hive/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c similarity index 100% rename from src/test/hive/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c rename to shark/src/test/hive/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c diff --git a/src/test/hive/golden/mergejoins-3-c408f69470d652da283442a62b384e46 b/shark/src/test/hive/golden/mergejoins-3-c408f69470d652da283442a62b384e46 similarity index 100% rename from src/test/hive/golden/mergejoins-3-c408f69470d652da283442a62b384e46 rename to shark/src/test/hive/golden/mergejoins-3-c408f69470d652da283442a62b384e46 diff --git a/src/test/hive/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 b/shark/src/test/hive/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 similarity index 100% rename from src/test/hive/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 rename to shark/src/test/hive/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 diff --git a/src/test/hive/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e b/shark/src/test/hive/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e similarity index 100% rename from src/test/hive/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e rename to shark/src/test/hive/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e diff --git a/src/test/hive/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df b/shark/src/test/hive/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df similarity index 100% rename from src/test/hive/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df rename to shark/src/test/hive/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df diff --git a/src/test/hive/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 b/shark/src/test/hive/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 rename to shark/src/test/hive/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 diff --git a/src/test/hive/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 b/shark/src/test/hive/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 rename to shark/src/test/hive/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 diff --git a/src/test/hive/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 b/shark/src/test/hive/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 rename to shark/src/test/hive/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 diff --git a/src/test/hive/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 b/shark/src/test/hive/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 rename to shark/src/test/hive/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 diff --git a/src/test/hive/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b b/shark/src/test/hive/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b rename to shark/src/test/hive/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b diff --git a/src/test/hive/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff b/shark/src/test/hive/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff rename to shark/src/test/hive/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff diff --git a/src/test/hive/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 b/shark/src/test/hive/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 rename to shark/src/test/hive/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 diff --git a/src/test/hive/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 b/shark/src/test/hive/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 rename to shark/src/test/hive/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 diff --git a/src/test/hive/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 b/shark/src/test/hive/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 rename to shark/src/test/hive/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 diff --git a/src/test/hive/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 b/shark/src/test/hive/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 rename to shark/src/test/hive/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 diff --git a/src/test/hive/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 b/shark/src/test/hive/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 rename to shark/src/test/hive/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 diff --git a/src/test/hive/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 b/shark/src/test/hive/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 rename to shark/src/test/hive/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 diff --git a/src/test/hive/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 b/shark/src/test/hive/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 similarity index 100% rename from src/test/hive/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 rename to shark/src/test/hive/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 diff --git a/src/test/hive/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d b/shark/src/test/hive/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d similarity index 100% rename from src/test/hive/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d rename to shark/src/test/hive/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d diff --git a/src/test/hive/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 b/shark/src/test/hive/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 similarity index 100% rename from src/test/hive/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 rename to shark/src/test/hive/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 diff --git a/src/test/hive/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d b/shark/src/test/hive/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d similarity index 100% rename from src/test/hive/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d rename to shark/src/test/hive/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d diff --git a/src/test/hive/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 b/shark/src/test/hive/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 similarity index 100% rename from src/test/hive/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 rename to shark/src/test/hive/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 diff --git a/src/test/hive/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d b/shark/src/test/hive/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d similarity index 100% rename from src/test/hive/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d rename to shark/src/test/hive/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d diff --git a/src/test/hive/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 b/shark/src/test/hive/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 similarity index 100% rename from src/test/hive/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 rename to shark/src/test/hive/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 diff --git a/src/test/hive/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 b/shark/src/test/hive/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 similarity index 100% rename from src/test/hive/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 rename to shark/src/test/hive/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 diff --git a/src/test/hive/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d b/shark/src/test/hive/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d similarity index 100% rename from src/test/hive/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d rename to shark/src/test/hive/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d diff --git a/src/test/hive/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf b/shark/src/test/hive/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf similarity index 100% rename from src/test/hive/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf rename to shark/src/test/hive/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf diff --git a/src/test/hive/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc b/shark/src/test/hive/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc similarity index 100% rename from src/test/hive/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc rename to shark/src/test/hive/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc diff --git a/src/test/hive/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 b/shark/src/test/hive/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 similarity index 100% rename from src/test/hive/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 rename to shark/src/test/hive/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 diff --git a/src/test/hive/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 b/shark/src/test/hive/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 similarity index 100% rename from src/test/hive/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 rename to shark/src/test/hive/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 diff --git a/src/test/hive/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/mi-1-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/mi-1-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/mi-1-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/mi-1-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e b/shark/src/test/hive/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e similarity index 100% rename from src/test/hive/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e rename to shark/src/test/hive/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e diff --git a/src/test/hive/golden/mi-3-b66a495f7bdf106a7886b72267b8659d b/shark/src/test/hive/golden/mi-3-b66a495f7bdf106a7886b72267b8659d similarity index 100% rename from src/test/hive/golden/mi-3-b66a495f7bdf106a7886b72267b8659d rename to shark/src/test/hive/golden/mi-3-b66a495f7bdf106a7886b72267b8659d diff --git a/src/test/hive/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c b/shark/src/test/hive/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c similarity index 100% rename from src/test/hive/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c rename to shark/src/test/hive/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c diff --git a/src/test/hive/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc b/shark/src/test/hive/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc similarity index 100% rename from src/test/hive/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc rename to shark/src/test/hive/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc diff --git a/src/test/hive/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 b/shark/src/test/hive/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 similarity index 100% rename from src/test/hive/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 rename to shark/src/test/hive/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 diff --git a/src/test/hive/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 b/shark/src/test/hive/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 similarity index 100% rename from src/test/hive/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 rename to shark/src/test/hive/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 diff --git a/src/test/hive/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 b/shark/src/test/hive/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 similarity index 100% rename from src/test/hive/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 rename to shark/src/test/hive/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 diff --git a/src/test/hive/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 b/shark/src/test/hive/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 rename to shark/src/test/hive/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 diff --git a/src/test/hive/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a b/shark/src/test/hive/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a similarity index 100% rename from src/test/hive/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a rename to shark/src/test/hive/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a diff --git a/src/test/hive/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa b/shark/src/test/hive/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa similarity index 100% rename from src/test/hive/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa rename to shark/src/test/hive/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa diff --git a/src/test/hive/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 b/shark/src/test/hive/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 rename to shark/src/test/hive/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 diff --git a/src/test/hive/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 b/shark/src/test/hive/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 rename to shark/src/test/hive/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 diff --git a/src/test/hive/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 b/shark/src/test/hive/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 rename to shark/src/test/hive/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 diff --git a/src/test/hive/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 b/shark/src/test/hive/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 rename to shark/src/test/hive/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 diff --git a/src/test/hive/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee b/shark/src/test/hive/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee similarity index 100% rename from src/test/hive/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee rename to shark/src/test/hive/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee diff --git a/src/test/hive/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 b/shark/src/test/hive/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 rename to shark/src/test/hive/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 diff --git a/src/test/hive/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 b/shark/src/test/hive/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 rename to shark/src/test/hive/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 diff --git a/src/test/hive/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 b/shark/src/test/hive/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 rename to shark/src/test/hive/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 diff --git a/src/test/hive/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac b/shark/src/test/hive/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac similarity index 100% rename from src/test/hive/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac rename to shark/src/test/hive/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac diff --git a/src/test/hive/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 b/shark/src/test/hive/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 rename to shark/src/test/hive/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 diff --git a/src/test/hive/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a b/shark/src/test/hive/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a similarity index 100% rename from src/test/hive/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a rename to shark/src/test/hive/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a diff --git a/src/test/hive/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 b/shark/src/test/hive/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 rename to shark/src/test/hive/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 diff --git a/src/test/hive/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 b/shark/src/test/hive/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 rename to shark/src/test/hive/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 diff --git a/src/test/hive/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 b/shark/src/test/hive/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 rename to shark/src/test/hive/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 diff --git a/src/test/hive/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f b/shark/src/test/hive/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f similarity index 100% rename from src/test/hive/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f rename to shark/src/test/hive/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f diff --git a/src/test/hive/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 b/shark/src/test/hive/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 rename to shark/src/test/hive/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 diff --git a/src/test/hive/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 b/shark/src/test/hive/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 rename to shark/src/test/hive/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 diff --git a/src/test/hive/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 b/shark/src/test/hive/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 rename to shark/src/test/hive/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 diff --git a/src/test/hive/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 b/shark/src/test/hive/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 rename to shark/src/test/hive/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 diff --git a/src/test/hive/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b b/shark/src/test/hive/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b similarity index 100% rename from src/test/hive/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b rename to shark/src/test/hive/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b diff --git a/src/test/hive/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 b/shark/src/test/hive/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 rename to shark/src/test/hive/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 diff --git a/src/test/hive/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 b/shark/src/test/hive/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 rename to shark/src/test/hive/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 diff --git a/src/test/hive/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe b/shark/src/test/hive/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe similarity index 100% rename from src/test/hive/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe rename to shark/src/test/hive/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe diff --git a/src/test/hive/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 b/shark/src/test/hive/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 rename to shark/src/test/hive/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 diff --git a/src/test/hive/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba b/shark/src/test/hive/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba similarity index 100% rename from src/test/hive/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba rename to shark/src/test/hive/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba diff --git a/src/test/hive/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c b/shark/src/test/hive/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c similarity index 100% rename from src/test/hive/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c rename to shark/src/test/hive/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c diff --git a/src/test/hive/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 b/shark/src/test/hive/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 similarity index 100% rename from src/test/hive/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 rename to shark/src/test/hive/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 diff --git a/src/test/hive/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 b/shark/src/test/hive/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 rename to shark/src/test/hive/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 diff --git a/src/test/hive/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e b/shark/src/test/hive/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e similarity index 100% rename from src/test/hive/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e rename to shark/src/test/hive/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e diff --git a/src/test/hive/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe b/shark/src/test/hive/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe similarity index 100% rename from src/test/hive/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe rename to shark/src/test/hive/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe diff --git a/src/test/hive/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b b/shark/src/test/hive/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b similarity index 100% rename from src/test/hive/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b rename to shark/src/test/hive/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b diff --git a/src/test/hive/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe b/shark/src/test/hive/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe similarity index 100% rename from src/test/hive/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe rename to shark/src/test/hive/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe diff --git a/src/test/hive/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f b/shark/src/test/hive/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from src/test/hive/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f rename to shark/src/test/hive/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f diff --git a/src/test/hive/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 b/shark/src/test/hive/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 rename to shark/src/test/hive/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 diff --git a/src/test/hive/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 b/shark/src/test/hive/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 rename to shark/src/test/hive/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 diff --git a/src/test/hive/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 b/shark/src/test/hive/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 rename to shark/src/test/hive/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 diff --git a/src/test/hive/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 b/shark/src/test/hive/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 rename to shark/src/test/hive/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/src/test/hive/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 b/shark/src/test/hive/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 rename to shark/src/test/hive/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 diff --git a/src/test/hive/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 b/shark/src/test/hive/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 rename to shark/src/test/hive/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 diff --git a/src/test/hive/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 b/shark/src/test/hive/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 rename to shark/src/test/hive/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 diff --git a/src/test/hive/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 b/shark/src/test/hive/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 rename to shark/src/test/hive/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 diff --git a/src/test/hive/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 b/shark/src/test/hive/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 rename to shark/src/test/hive/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 diff --git a/src/test/hive/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f b/shark/src/test/hive/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f similarity index 100% rename from src/test/hive/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f rename to shark/src/test/hive/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f diff --git a/src/test/hive/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e b/shark/src/test/hive/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e similarity index 100% rename from src/test/hive/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e rename to shark/src/test/hive/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e diff --git a/src/test/hive/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d b/shark/src/test/hive/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d similarity index 100% rename from src/test/hive/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d rename to shark/src/test/hive/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d diff --git a/src/test/hive/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 b/shark/src/test/hive/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 rename to shark/src/test/hive/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 diff --git a/src/test/hive/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 b/shark/src/test/hive/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 rename to shark/src/test/hive/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 diff --git a/src/test/hive/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 b/shark/src/test/hive/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 rename to shark/src/test/hive/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 diff --git a/src/test/hive/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 b/shark/src/test/hive/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 rename to shark/src/test/hive/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 diff --git a/src/test/hive/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 b/shark/src/test/hive/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 rename to shark/src/test/hive/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 diff --git a/src/test/hive/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 b/shark/src/test/hive/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 similarity index 100% rename from src/test/hive/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 rename to shark/src/test/hive/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 diff --git a/src/test/hive/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 b/shark/src/test/hive/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 similarity index 100% rename from src/test/hive/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 rename to shark/src/test/hive/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 diff --git a/src/test/hive/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d b/shark/src/test/hive/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d similarity index 100% rename from src/test/hive/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d rename to shark/src/test/hive/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d diff --git a/src/test/hive/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 b/shark/src/test/hive/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 similarity index 100% rename from src/test/hive/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 rename to shark/src/test/hive/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 diff --git a/src/test/hive/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 b/shark/src/test/hive/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 similarity index 100% rename from src/test/hive/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 rename to shark/src/test/hive/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 diff --git a/src/test/hive/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 b/shark/src/test/hive/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 similarity index 100% rename from src/test/hive/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 rename to shark/src/test/hive/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 diff --git a/src/test/hive/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 b/shark/src/test/hive/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 similarity index 100% rename from src/test/hive/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 rename to shark/src/test/hive/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 diff --git a/src/test/hive/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c b/shark/src/test/hive/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c similarity index 100% rename from src/test/hive/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c rename to shark/src/test/hive/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c diff --git a/src/test/hive/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f b/shark/src/test/hive/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f similarity index 100% rename from src/test/hive/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f rename to shark/src/test/hive/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f diff --git a/src/test/hive/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 b/shark/src/test/hive/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 similarity index 100% rename from src/test/hive/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 rename to shark/src/test/hive/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 diff --git a/src/test/hive/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 b/shark/src/test/hive/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 similarity index 100% rename from src/test/hive/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 rename to shark/src/test/hive/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 diff --git a/src/test/hive/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 b/shark/src/test/hive/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 similarity index 100% rename from src/test/hive/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 rename to shark/src/test/hive/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 diff --git a/src/test/hive/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 b/shark/src/test/hive/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 similarity index 100% rename from src/test/hive/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 rename to shark/src/test/hive/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 diff --git a/src/test/hive/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 b/shark/src/test/hive/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 similarity index 100% rename from src/test/hive/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 rename to shark/src/test/hive/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 diff --git a/src/test/hive/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e b/shark/src/test/hive/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e similarity index 100% rename from src/test/hive/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e rename to shark/src/test/hive/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e diff --git a/src/test/hive/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 b/shark/src/test/hive/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 similarity index 100% rename from src/test/hive/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 rename to shark/src/test/hive/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 diff --git a/src/test/hive/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 b/shark/src/test/hive/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 similarity index 100% rename from src/test/hive/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 rename to shark/src/test/hive/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 diff --git a/src/test/hive/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 b/shark/src/test/hive/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 rename to shark/src/test/hive/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 diff --git a/src/test/hive/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a b/shark/src/test/hive/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a rename to shark/src/test/hive/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a diff --git a/src/test/hive/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 b/shark/src/test/hive/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 rename to shark/src/test/hive/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 diff --git a/src/test/hive/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc b/shark/src/test/hive/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc rename to shark/src/test/hive/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc diff --git a/src/test/hive/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f b/shark/src/test/hive/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f rename to shark/src/test/hive/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f diff --git a/src/test/hive/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 b/shark/src/test/hive/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 rename to shark/src/test/hive/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 diff --git a/src/test/hive/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca b/shark/src/test/hive/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca rename to shark/src/test/hive/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca diff --git a/src/test/hive/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 b/shark/src/test/hive/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 rename to shark/src/test/hive/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 diff --git a/src/test/hive/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 b/shark/src/test/hive/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 rename to shark/src/test/hive/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 diff --git a/src/test/hive/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 b/shark/src/test/hive/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 rename to shark/src/test/hive/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 diff --git a/src/test/hive/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 b/shark/src/test/hive/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 rename to shark/src/test/hive/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 diff --git a/src/test/hive/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b b/shark/src/test/hive/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b rename to shark/src/test/hive/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b diff --git a/src/test/hive/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 b/shark/src/test/hive/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 similarity index 100% rename from src/test/hive/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 rename to shark/src/test/hive/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 diff --git a/src/test/hive/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 b/shark/src/test/hive/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 rename to shark/src/test/hive/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 diff --git a/src/test/hive/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 b/shark/src/test/hive/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 rename to shark/src/test/hive/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 diff --git a/src/test/hive/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f b/shark/src/test/hive/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f rename to shark/src/test/hive/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f diff --git a/src/test/hive/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 b/shark/src/test/hive/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 rename to shark/src/test/hive/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 diff --git a/src/test/hive/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 b/shark/src/test/hive/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 rename to shark/src/test/hive/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 diff --git a/src/test/hive/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 b/shark/src/test/hive/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 rename to shark/src/test/hive/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 diff --git a/src/test/hive/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f b/shark/src/test/hive/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f rename to shark/src/test/hive/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f diff --git a/src/test/hive/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 b/shark/src/test/hive/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 rename to shark/src/test/hive/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 diff --git a/src/test/hive/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 b/shark/src/test/hive/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 rename to shark/src/test/hive/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 diff --git a/src/test/hive/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 b/shark/src/test/hive/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 rename to shark/src/test/hive/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 diff --git a/src/test/hive/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a b/shark/src/test/hive/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a rename to shark/src/test/hive/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a diff --git a/src/test/hive/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f b/shark/src/test/hive/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f rename to shark/src/test/hive/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f diff --git a/src/test/hive/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 b/shark/src/test/hive/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 rename to shark/src/test/hive/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 diff --git a/src/test/hive/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 b/shark/src/test/hive/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 rename to shark/src/test/hive/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 diff --git a/src/test/hive/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 b/shark/src/test/hive/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 rename to shark/src/test/hive/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 diff --git a/src/test/hive/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 b/shark/src/test/hive/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 rename to shark/src/test/hive/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 diff --git a/src/test/hive/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d b/shark/src/test/hive/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d rename to shark/src/test/hive/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d diff --git a/src/test/hive/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df b/shark/src/test/hive/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df rename to shark/src/test/hive/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df diff --git a/src/test/hive/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f b/shark/src/test/hive/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f rename to shark/src/test/hive/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f diff --git a/src/test/hive/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 b/shark/src/test/hive/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 rename to shark/src/test/hive/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 diff --git a/src/test/hive/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 b/shark/src/test/hive/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 rename to shark/src/test/hive/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 diff --git a/src/test/hive/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 b/shark/src/test/hive/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 rename to shark/src/test/hive/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 diff --git a/src/test/hive/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 b/shark/src/test/hive/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 rename to shark/src/test/hive/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 diff --git a/src/test/hive/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d b/shark/src/test/hive/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d rename to shark/src/test/hive/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d diff --git a/src/test/hive/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 b/shark/src/test/hive/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 rename to shark/src/test/hive/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 diff --git a/src/test/hive/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f b/shark/src/test/hive/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f rename to shark/src/test/hive/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f diff --git a/src/test/hive/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 b/shark/src/test/hive/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 rename to shark/src/test/hive/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 diff --git a/src/test/hive/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 b/shark/src/test/hive/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 rename to shark/src/test/hive/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 diff --git a/src/test/hive/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 b/shark/src/test/hive/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 similarity index 100% rename from src/test/hive/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 rename to shark/src/test/hive/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 diff --git a/src/test/hive/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b b/shark/src/test/hive/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b similarity index 100% rename from src/test/hive/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b rename to shark/src/test/hive/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b diff --git a/src/test/hive/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 b/shark/src/test/hive/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 similarity index 100% rename from src/test/hive/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 rename to shark/src/test/hive/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 diff --git a/src/test/hive/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 b/shark/src/test/hive/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 similarity index 100% rename from src/test/hive/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 rename to shark/src/test/hive/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 diff --git a/src/test/hive/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 b/shark/src/test/hive/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 similarity index 100% rename from src/test/hive/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 rename to shark/src/test/hive/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 diff --git a/src/test/hive/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 b/shark/src/test/hive/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 similarity index 100% rename from src/test/hive/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 rename to shark/src/test/hive/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 diff --git a/src/test/hive/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae b/shark/src/test/hive/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae similarity index 100% rename from src/test/hive/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae rename to shark/src/test/hive/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae diff --git a/src/test/hive/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 b/shark/src/test/hive/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 similarity index 100% rename from src/test/hive/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 rename to shark/src/test/hive/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 diff --git a/src/test/hive/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 b/shark/src/test/hive/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 similarity index 100% rename from src/test/hive/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 rename to shark/src/test/hive/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 diff --git a/src/test/hive/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 b/shark/src/test/hive/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 similarity index 100% rename from src/test/hive/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 rename to shark/src/test/hive/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 diff --git a/src/test/hive/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 b/shark/src/test/hive/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 similarity index 100% rename from src/test/hive/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 rename to shark/src/test/hive/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 diff --git a/src/test/hive/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 b/shark/src/test/hive/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 similarity index 100% rename from src/test/hive/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 rename to shark/src/test/hive/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 diff --git a/src/test/hive/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 b/shark/src/test/hive/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 similarity index 100% rename from src/test/hive/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 rename to shark/src/test/hive/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 diff --git a/src/test/hive/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 b/shark/src/test/hive/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 similarity index 100% rename from src/test/hive/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 rename to shark/src/test/hive/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 diff --git a/src/test/hive/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 b/shark/src/test/hive/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 similarity index 100% rename from src/test/hive/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 rename to shark/src/test/hive/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 diff --git a/src/test/hive/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e b/shark/src/test/hive/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e similarity index 100% rename from src/test/hive/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e rename to shark/src/test/hive/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e diff --git a/src/test/hive/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 b/shark/src/test/hive/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 similarity index 100% rename from src/test/hive/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 rename to shark/src/test/hive/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 diff --git a/src/test/hive/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 b/shark/src/test/hive/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 similarity index 100% rename from src/test/hive/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 rename to shark/src/test/hive/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 diff --git a/src/test/hive/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 b/shark/src/test/hive/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 similarity index 100% rename from src/test/hive/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 rename to shark/src/test/hive/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 diff --git a/src/test/hive/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 b/shark/src/test/hive/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 similarity index 100% rename from src/test/hive/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 rename to shark/src/test/hive/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 diff --git a/src/test/hive/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d b/shark/src/test/hive/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d similarity index 100% rename from src/test/hive/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d rename to shark/src/test/hive/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d diff --git a/src/test/hive/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 b/shark/src/test/hive/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 similarity index 100% rename from src/test/hive/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 rename to shark/src/test/hive/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 diff --git a/src/test/hive/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f b/shark/src/test/hive/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f similarity index 100% rename from src/test/hive/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f rename to shark/src/test/hive/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f diff --git a/src/test/hive/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 b/shark/src/test/hive/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 similarity index 100% rename from src/test/hive/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 rename to shark/src/test/hive/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 diff --git a/src/test/hive/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 b/shark/src/test/hive/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 similarity index 100% rename from src/test/hive/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 rename to shark/src/test/hive/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 diff --git a/src/test/hive/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 b/shark/src/test/hive/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 similarity index 100% rename from src/test/hive/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 rename to shark/src/test/hive/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 diff --git a/src/test/hive/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f b/shark/src/test/hive/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f similarity index 100% rename from src/test/hive/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f rename to shark/src/test/hive/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f diff --git a/src/test/hive/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 b/shark/src/test/hive/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 similarity index 100% rename from src/test/hive/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 rename to shark/src/test/hive/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 diff --git a/src/test/hive/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f b/shark/src/test/hive/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f similarity index 100% rename from src/test/hive/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f rename to shark/src/test/hive/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f diff --git a/src/test/hive/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 b/shark/src/test/hive/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 similarity index 100% rename from src/test/hive/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 rename to shark/src/test/hive/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 diff --git a/src/test/hive/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 b/shark/src/test/hive/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 similarity index 100% rename from src/test/hive/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 rename to shark/src/test/hive/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 diff --git a/src/test/hive/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 b/shark/src/test/hive/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 similarity index 100% rename from src/test/hive/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 rename to shark/src/test/hive/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 diff --git a/src/test/hive/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f b/shark/src/test/hive/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f similarity index 100% rename from src/test/hive/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f rename to shark/src/test/hive/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f diff --git a/src/test/hive/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 b/shark/src/test/hive/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 similarity index 100% rename from src/test/hive/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 rename to shark/src/test/hive/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 diff --git a/src/test/hive/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 b/shark/src/test/hive/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 similarity index 100% rename from src/test/hive/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 rename to shark/src/test/hive/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 diff --git a/src/test/hive/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 b/shark/src/test/hive/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 similarity index 100% rename from src/test/hive/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 rename to shark/src/test/hive/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 diff --git a/src/test/hive/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f b/shark/src/test/hive/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f similarity index 100% rename from src/test/hive/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f rename to shark/src/test/hive/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f diff --git a/src/test/hive/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 b/shark/src/test/hive/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 similarity index 100% rename from src/test/hive/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 rename to shark/src/test/hive/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 diff --git a/src/test/hive/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f b/shark/src/test/hive/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f similarity index 100% rename from src/test/hive/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f rename to shark/src/test/hive/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f diff --git a/src/test/hive/golden/newline-0-43392a20a8d249a279d50d96578e6a1b b/shark/src/test/hive/golden/newline-0-43392a20a8d249a279d50d96578e6a1b similarity index 100% rename from src/test/hive/golden/newline-0-43392a20a8d249a279d50d96578e6a1b rename to shark/src/test/hive/golden/newline-0-43392a20a8d249a279d50d96578e6a1b diff --git a/src/test/hive/golden/newline-1-a19a19272149c732977c37e043910505 b/shark/src/test/hive/golden/newline-1-a19a19272149c732977c37e043910505 similarity index 100% rename from src/test/hive/golden/newline-1-a19a19272149c732977c37e043910505 rename to shark/src/test/hive/golden/newline-1-a19a19272149c732977c37e043910505 diff --git a/src/test/hive/golden/newline-2-4eb54a664e549614d56ca088c8867d b/shark/src/test/hive/golden/newline-2-4eb54a664e549614d56ca088c8867d similarity index 100% rename from src/test/hive/golden/newline-2-4eb54a664e549614d56ca088c8867d rename to shark/src/test/hive/golden/newline-2-4eb54a664e549614d56ca088c8867d diff --git a/src/test/hive/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 b/shark/src/test/hive/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 similarity index 100% rename from src/test/hive/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 rename to shark/src/test/hive/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 diff --git a/src/test/hive/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 b/shark/src/test/hive/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 similarity index 100% rename from src/test/hive/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 rename to shark/src/test/hive/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 diff --git a/src/test/hive/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b b/shark/src/test/hive/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b similarity index 100% rename from src/test/hive/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b rename to shark/src/test/hive/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b diff --git a/src/test/hive/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 b/shark/src/test/hive/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 similarity index 100% rename from src/test/hive/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 rename to shark/src/test/hive/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 diff --git a/src/test/hive/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 b/shark/src/test/hive/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 similarity index 100% rename from src/test/hive/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 rename to shark/src/test/hive/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 diff --git a/src/test/hive/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 b/shark/src/test/hive/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 similarity index 100% rename from src/test/hive/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 rename to shark/src/test/hive/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 diff --git a/src/test/hive/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 b/shark/src/test/hive/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 similarity index 100% rename from src/test/hive/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 rename to shark/src/test/hive/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 diff --git a/src/test/hive/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 b/shark/src/test/hive/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 similarity index 100% rename from src/test/hive/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 rename to shark/src/test/hive/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 diff --git a/src/test/hive/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 b/shark/src/test/hive/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 similarity index 100% rename from src/test/hive/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 rename to shark/src/test/hive/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 diff --git a/src/test/hive/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 b/shark/src/test/hive/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 similarity index 100% rename from src/test/hive/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 rename to shark/src/test/hive/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 diff --git a/src/test/hive/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 b/shark/src/test/hive/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 similarity index 100% rename from src/test/hive/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 rename to shark/src/test/hive/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 diff --git a/src/test/hive/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 b/shark/src/test/hive/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 rename to shark/src/test/hive/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 diff --git a/src/test/hive/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 b/shark/src/test/hive/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 rename to shark/src/test/hive/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad diff --git a/src/test/hive/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 b/shark/src/test/hive/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 similarity index 100% rename from src/test/hive/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 rename to shark/src/test/hive/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 diff --git a/src/test/hive/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d b/shark/src/test/hive/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d similarity index 100% rename from src/test/hive/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d rename to shark/src/test/hive/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d diff --git a/src/test/hive/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd b/shark/src/test/hive/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd similarity index 100% rename from src/test/hive/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd rename to shark/src/test/hive/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd diff --git a/src/test/hive/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e b/shark/src/test/hive/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e similarity index 100% rename from src/test/hive/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e rename to shark/src/test/hive/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e diff --git a/src/test/hive/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d b/shark/src/test/hive/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d similarity index 100% rename from src/test/hive/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d rename to shark/src/test/hive/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d diff --git a/src/test/hive/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 b/shark/src/test/hive/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 similarity index 100% rename from src/test/hive/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 rename to shark/src/test/hive/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 diff --git a/src/test/hive/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 b/shark/src/test/hive/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 similarity index 100% rename from src/test/hive/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 rename to shark/src/test/hive/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 diff --git a/src/test/hive/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 b/shark/src/test/hive/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 similarity index 100% rename from src/test/hive/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 rename to shark/src/test/hive/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 diff --git a/src/test/hive/golden/null_cast-1-7257e6f8170e545962d27741353f672c b/shark/src/test/hive/golden/null_cast-1-7257e6f8170e545962d27741353f672c similarity index 100% rename from src/test/hive/golden/null_cast-1-7257e6f8170e545962d27741353f672c rename to shark/src/test/hive/golden/null_cast-1-7257e6f8170e545962d27741353f672c diff --git a/src/test/hive/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 b/shark/src/test/hive/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 similarity index 100% rename from src/test/hive/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 rename to shark/src/test/hive/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 diff --git a/src/test/hive/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 b/shark/src/test/hive/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 similarity index 100% rename from src/test/hive/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 rename to shark/src/test/hive/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 diff --git a/src/test/hive/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 b/shark/src/test/hive/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 similarity index 100% rename from src/test/hive/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 rename to shark/src/test/hive/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 diff --git a/src/test/hive/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 b/shark/src/test/hive/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 similarity index 100% rename from src/test/hive/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 rename to shark/src/test/hive/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 diff --git a/src/test/hive/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 b/shark/src/test/hive/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 similarity index 100% rename from src/test/hive/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 rename to shark/src/test/hive/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 diff --git a/src/test/hive/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 b/shark/src/test/hive/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 similarity index 100% rename from src/test/hive/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 rename to shark/src/test/hive/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 diff --git a/src/test/hive/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 b/shark/src/test/hive/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 similarity index 100% rename from src/test/hive/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 rename to shark/src/test/hive/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 diff --git a/src/test/hive/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 b/shark/src/test/hive/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 similarity index 100% rename from src/test/hive/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 rename to shark/src/test/hive/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 diff --git a/src/test/hive/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 b/shark/src/test/hive/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 similarity index 100% rename from src/test/hive/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 rename to shark/src/test/hive/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 diff --git a/src/test/hive/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a b/shark/src/test/hive/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a similarity index 100% rename from src/test/hive/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a rename to shark/src/test/hive/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a diff --git a/src/test/hive/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 b/shark/src/test/hive/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 similarity index 100% rename from src/test/hive/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 rename to shark/src/test/hive/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 diff --git a/src/test/hive/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a b/shark/src/test/hive/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a similarity index 100% rename from src/test/hive/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a rename to shark/src/test/hive/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a diff --git a/src/test/hive/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 b/shark/src/test/hive/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 similarity index 100% rename from src/test/hive/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 rename to shark/src/test/hive/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 diff --git a/src/test/hive/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a b/shark/src/test/hive/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a similarity index 100% rename from src/test/hive/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a rename to shark/src/test/hive/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a diff --git a/src/test/hive/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 b/shark/src/test/hive/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 similarity index 100% rename from src/test/hive/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 rename to shark/src/test/hive/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 diff --git a/src/test/hive/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a b/shark/src/test/hive/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a similarity index 100% rename from src/test/hive/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a rename to shark/src/test/hive/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a diff --git a/src/test/hive/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 b/shark/src/test/hive/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 similarity index 100% rename from src/test/hive/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 rename to shark/src/test/hive/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 diff --git a/src/test/hive/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 b/shark/src/test/hive/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 similarity index 100% rename from src/test/hive/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 rename to shark/src/test/hive/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 diff --git a/src/test/hive/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 b/shark/src/test/hive/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 similarity index 100% rename from src/test/hive/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 rename to shark/src/test/hive/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 diff --git a/src/test/hive/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 b/shark/src/test/hive/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 similarity index 100% rename from src/test/hive/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 rename to shark/src/test/hive/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 diff --git a/src/test/hive/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 b/shark/src/test/hive/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 similarity index 100% rename from src/test/hive/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 rename to shark/src/test/hive/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 diff --git a/src/test/hive/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb b/shark/src/test/hive/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb similarity index 100% rename from src/test/hive/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb rename to shark/src/test/hive/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb diff --git a/src/test/hive/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e b/shark/src/test/hive/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e similarity index 100% rename from src/test/hive/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e rename to shark/src/test/hive/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e diff --git a/src/test/hive/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 b/shark/src/test/hive/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 similarity index 100% rename from src/test/hive/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 rename to shark/src/test/hive/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 diff --git a/src/test/hive/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 b/shark/src/test/hive/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 similarity index 100% rename from src/test/hive/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 rename to shark/src/test/hive/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 diff --git a/src/test/hive/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 b/shark/src/test/hive/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 similarity index 100% rename from src/test/hive/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 rename to shark/src/test/hive/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 diff --git a/src/test/hive/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 b/shark/src/test/hive/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 similarity index 100% rename from src/test/hive/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 rename to shark/src/test/hive/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 diff --git a/src/test/hive/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 b/shark/src/test/hive/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 similarity index 100% rename from src/test/hive/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 rename to shark/src/test/hive/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 diff --git a/src/test/hive/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb b/shark/src/test/hive/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb similarity index 100% rename from src/test/hive/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb rename to shark/src/test/hive/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb diff --git a/src/test/hive/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 b/shark/src/test/hive/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 similarity index 100% rename from src/test/hive/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 rename to shark/src/test/hive/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 diff --git a/src/test/hive/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 b/shark/src/test/hive/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 similarity index 100% rename from src/test/hive/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 rename to shark/src/test/hive/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 diff --git a/src/test/hive/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e b/shark/src/test/hive/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e similarity index 100% rename from src/test/hive/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e rename to shark/src/test/hive/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e diff --git a/src/test/hive/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 b/shark/src/test/hive/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 similarity index 100% rename from src/test/hive/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 rename to shark/src/test/hive/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 diff --git a/src/test/hive/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 b/shark/src/test/hive/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 similarity index 100% rename from src/test/hive/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 rename to shark/src/test/hive/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 diff --git a/src/test/hive/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 b/shark/src/test/hive/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 similarity index 100% rename from src/test/hive/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 rename to shark/src/test/hive/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 diff --git a/src/test/hive/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 b/shark/src/test/hive/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 similarity index 100% rename from src/test/hive/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 rename to shark/src/test/hive/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 diff --git a/src/test/hive/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 b/shark/src/test/hive/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 similarity index 100% rename from src/test/hive/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 rename to shark/src/test/hive/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 diff --git a/src/test/hive/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 b/shark/src/test/hive/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 similarity index 100% rename from src/test/hive/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 rename to shark/src/test/hive/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 diff --git a/src/test/hive/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 b/shark/src/test/hive/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 similarity index 100% rename from src/test/hive/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 rename to shark/src/test/hive/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 diff --git a/src/test/hive/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 b/shark/src/test/hive/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 similarity index 100% rename from src/test/hive/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 rename to shark/src/test/hive/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 diff --git a/src/test/hive/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 b/shark/src/test/hive/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 similarity index 100% rename from src/test/hive/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 rename to shark/src/test/hive/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 diff --git a/src/test/hive/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 b/shark/src/test/hive/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 similarity index 100% rename from src/test/hive/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 rename to shark/src/test/hive/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 diff --git a/src/test/hive/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 b/shark/src/test/hive/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 similarity index 100% rename from src/test/hive/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 rename to shark/src/test/hive/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 diff --git a/src/test/hive/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 b/shark/src/test/hive/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 similarity index 100% rename from src/test/hive/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 rename to shark/src/test/hive/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 diff --git a/src/test/hive/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 b/shark/src/test/hive/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 similarity index 100% rename from src/test/hive/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 rename to shark/src/test/hive/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 diff --git a/src/test/hive/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 b/shark/src/test/hive/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 similarity index 100% rename from src/test/hive/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 rename to shark/src/test/hive/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 diff --git a/src/test/hive/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 b/shark/src/test/hive/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 similarity index 100% rename from src/test/hive/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 rename to shark/src/test/hive/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 b/shark/src/test/hive/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 similarity index 100% rename from src/test/hive/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 rename to shark/src/test/hive/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 b/shark/src/test/hive/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 similarity index 100% rename from src/test/hive/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 rename to shark/src/test/hive/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 b/shark/src/test/hive/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 similarity index 100% rename from src/test/hive/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 rename to shark/src/test/hive/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 diff --git a/src/test/hive/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 b/shark/src/test/hive/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 similarity index 100% rename from src/test/hive/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 rename to shark/src/test/hive/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 diff --git a/src/test/hive/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 b/shark/src/test/hive/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 similarity index 100% rename from src/test/hive/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 rename to shark/src/test/hive/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 diff --git a/src/test/hive/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b b/shark/src/test/hive/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b similarity index 100% rename from src/test/hive/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b rename to shark/src/test/hive/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b diff --git a/src/test/hive/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 b/shark/src/test/hive/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 similarity index 100% rename from src/test/hive/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 rename to shark/src/test/hive/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 diff --git a/src/test/hive/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c b/shark/src/test/hive/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c similarity index 100% rename from src/test/hive/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c rename to shark/src/test/hive/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c diff --git a/src/test/hive/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e b/shark/src/test/hive/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e similarity index 100% rename from src/test/hive/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e rename to shark/src/test/hive/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e diff --git a/src/test/hive/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c b/shark/src/test/hive/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c similarity index 100% rename from src/test/hive/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c rename to shark/src/test/hive/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c diff --git a/src/test/hive/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 b/shark/src/test/hive/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 similarity index 100% rename from src/test/hive/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 rename to shark/src/test/hive/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 diff --git a/src/test/hive/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 b/shark/src/test/hive/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 similarity index 100% rename from src/test/hive/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 rename to shark/src/test/hive/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 diff --git a/src/test/hive/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf b/shark/src/test/hive/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf similarity index 100% rename from src/test/hive/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf rename to shark/src/test/hive/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf diff --git a/src/test/hive/golden/nullinput2-0-40b73200f853b473855e85eba391f008 b/shark/src/test/hive/golden/nullinput2-0-40b73200f853b473855e85eba391f008 similarity index 100% rename from src/test/hive/golden/nullinput2-0-40b73200f853b473855e85eba391f008 rename to shark/src/test/hive/golden/nullinput2-0-40b73200f853b473855e85eba391f008 diff --git a/src/test/hive/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 b/shark/src/test/hive/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 similarity index 100% rename from src/test/hive/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 rename to shark/src/test/hive/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 diff --git a/src/test/hive/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 b/shark/src/test/hive/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 similarity index 100% rename from src/test/hive/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 rename to shark/src/test/hive/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 diff --git a/src/test/hive/golden/nullscript-0-4477b1566208a6805bb20630755a9375 b/shark/src/test/hive/golden/nullscript-0-4477b1566208a6805bb20630755a9375 similarity index 100% rename from src/test/hive/golden/nullscript-0-4477b1566208a6805bb20630755a9375 rename to shark/src/test/hive/golden/nullscript-0-4477b1566208a6805bb20630755a9375 diff --git a/src/test/hive/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 b/shark/src/test/hive/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 similarity index 100% rename from src/test/hive/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 rename to shark/src/test/hive/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 diff --git a/src/test/hive/golden/nullscript-2-17238164053203d56d30704e2c098e80 b/shark/src/test/hive/golden/nullscript-2-17238164053203d56d30704e2c098e80 similarity index 100% rename from src/test/hive/golden/nullscript-2-17238164053203d56d30704e2c098e80 rename to shark/src/test/hive/golden/nullscript-2-17238164053203d56d30704e2c098e80 diff --git a/src/test/hive/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 b/shark/src/test/hive/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 similarity index 100% rename from src/test/hive/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 rename to shark/src/test/hive/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 diff --git a/src/test/hive/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 b/shark/src/test/hive/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 similarity index 100% rename from src/test/hive/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 rename to shark/src/test/hive/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 diff --git a/src/test/hive/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc b/shark/src/test/hive/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc similarity index 100% rename from src/test/hive/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc rename to shark/src/test/hive/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc diff --git a/src/test/hive/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 b/shark/src/test/hive/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 similarity index 100% rename from src/test/hive/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 rename to shark/src/test/hive/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 diff --git a/src/test/hive/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a b/shark/src/test/hive/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a similarity index 100% rename from src/test/hive/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a rename to shark/src/test/hive/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a diff --git a/src/test/hive/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 b/shark/src/test/hive/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 similarity index 100% rename from src/test/hive/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 rename to shark/src/test/hive/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 diff --git a/src/test/hive/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f b/shark/src/test/hive/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f similarity index 100% rename from src/test/hive/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f rename to shark/src/test/hive/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f diff --git a/src/test/hive/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a b/shark/src/test/hive/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a similarity index 100% rename from src/test/hive/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a rename to shark/src/test/hive/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a diff --git a/src/test/hive/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 b/shark/src/test/hive/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 similarity index 100% rename from src/test/hive/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 rename to shark/src/test/hive/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 diff --git a/src/test/hive/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e b/shark/src/test/hive/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e similarity index 100% rename from src/test/hive/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e rename to shark/src/test/hive/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e diff --git a/src/test/hive/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 b/shark/src/test/hive/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 similarity index 100% rename from src/test/hive/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 rename to shark/src/test/hive/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 diff --git a/src/test/hive/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 b/shark/src/test/hive/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 similarity index 100% rename from src/test/hive/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 rename to shark/src/test/hive/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 diff --git a/src/test/hive/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d b/shark/src/test/hive/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d similarity index 100% rename from src/test/hive/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d rename to shark/src/test/hive/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d diff --git a/src/test/hive/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 b/shark/src/test/hive/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 similarity index 100% rename from src/test/hive/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 rename to shark/src/test/hive/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 diff --git a/src/test/hive/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 b/shark/src/test/hive/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 similarity index 100% rename from src/test/hive/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 rename to shark/src/test/hive/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 diff --git a/src/test/hive/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 b/shark/src/test/hive/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 similarity index 100% rename from src/test/hive/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 rename to shark/src/test/hive/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 diff --git a/src/test/hive/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 b/shark/src/test/hive/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 similarity index 100% rename from src/test/hive/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 rename to shark/src/test/hive/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 diff --git a/src/test/hive/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 b/shark/src/test/hive/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 similarity index 100% rename from src/test/hive/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 rename to shark/src/test/hive/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 diff --git a/src/test/hive/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d b/shark/src/test/hive/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d similarity index 100% rename from src/test/hive/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d rename to shark/src/test/hive/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d diff --git a/src/test/hive/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb b/shark/src/test/hive/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb similarity index 100% rename from src/test/hive/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb rename to shark/src/test/hive/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb diff --git a/src/test/hive/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 b/shark/src/test/hive/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 similarity index 100% rename from src/test/hive/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 rename to shark/src/test/hive/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 diff --git a/src/test/hive/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 b/shark/src/test/hive/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 similarity index 100% rename from src/test/hive/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 rename to shark/src/test/hive/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 diff --git a/src/test/hive/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 b/shark/src/test/hive/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 similarity index 100% rename from src/test/hive/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 rename to shark/src/test/hive/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 diff --git a/src/test/hive/golden/orc_create-20-176d469a0edba57404416535c7d48023 b/shark/src/test/hive/golden/orc_create-20-176d469a0edba57404416535c7d48023 similarity index 100% rename from src/test/hive/golden/orc_create-20-176d469a0edba57404416535c7d48023 rename to shark/src/test/hive/golden/orc_create-20-176d469a0edba57404416535c7d48023 diff --git a/src/test/hive/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 b/shark/src/test/hive/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 similarity index 100% rename from src/test/hive/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 rename to shark/src/test/hive/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 diff --git a/src/test/hive/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 b/shark/src/test/hive/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 similarity index 100% rename from src/test/hive/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 rename to shark/src/test/hive/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 diff --git a/src/test/hive/golden/orc_create-23-be779533ea8967231e644209114c8350 b/shark/src/test/hive/golden/orc_create-23-be779533ea8967231e644209114c8350 similarity index 100% rename from src/test/hive/golden/orc_create-23-be779533ea8967231e644209114c8350 rename to shark/src/test/hive/golden/orc_create-23-be779533ea8967231e644209114c8350 diff --git a/src/test/hive/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 b/shark/src/test/hive/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 similarity index 100% rename from src/test/hive/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 rename to shark/src/test/hive/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 diff --git a/src/test/hive/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead b/shark/src/test/hive/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead similarity index 100% rename from src/test/hive/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead rename to shark/src/test/hive/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead diff --git a/src/test/hive/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c b/shark/src/test/hive/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c similarity index 100% rename from src/test/hive/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c rename to shark/src/test/hive/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c diff --git a/src/test/hive/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 b/shark/src/test/hive/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 similarity index 100% rename from src/test/hive/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 rename to shark/src/test/hive/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 diff --git a/src/test/hive/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 b/shark/src/test/hive/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 similarity index 100% rename from src/test/hive/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 rename to shark/src/test/hive/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 diff --git a/src/test/hive/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 b/shark/src/test/hive/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 similarity index 100% rename from src/test/hive/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 rename to shark/src/test/hive/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 diff --git a/src/test/hive/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae b/shark/src/test/hive/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae similarity index 100% rename from src/test/hive/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae rename to shark/src/test/hive/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae diff --git a/src/test/hive/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d b/shark/src/test/hive/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d similarity index 100% rename from src/test/hive/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d rename to shark/src/test/hive/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d diff --git a/src/test/hive/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 b/shark/src/test/hive/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 similarity index 100% rename from src/test/hive/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 rename to shark/src/test/hive/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 diff --git a/src/test/hive/golden/orc_create-32-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/orc_create-32-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/orc_create-32-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/orc_create-32-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 b/shark/src/test/hive/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 similarity index 100% rename from src/test/hive/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 rename to shark/src/test/hive/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 diff --git a/src/test/hive/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 b/shark/src/test/hive/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 similarity index 100% rename from src/test/hive/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 rename to shark/src/test/hive/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 diff --git a/src/test/hive/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 b/shark/src/test/hive/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 similarity index 100% rename from src/test/hive/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 rename to shark/src/test/hive/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 diff --git a/src/test/hive/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a b/shark/src/test/hive/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a similarity index 100% rename from src/test/hive/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a rename to shark/src/test/hive/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a diff --git a/src/test/hive/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 b/shark/src/test/hive/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 similarity index 100% rename from src/test/hive/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 rename to shark/src/test/hive/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 diff --git a/src/test/hive/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 b/shark/src/test/hive/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 similarity index 100% rename from src/test/hive/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 rename to shark/src/test/hive/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 diff --git a/src/test/hive/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 b/shark/src/test/hive/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 similarity index 100% rename from src/test/hive/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 rename to shark/src/test/hive/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 diff --git a/src/test/hive/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/shark/src/test/hive/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 similarity index 100% rename from src/test/hive/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 rename to shark/src/test/hive/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 diff --git a/src/test/hive/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae b/shark/src/test/hive/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae similarity index 100% rename from src/test/hive/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae rename to shark/src/test/hive/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae diff --git a/src/test/hive/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/shark/src/test/hive/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 similarity index 100% rename from src/test/hive/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 rename to shark/src/test/hive/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 diff --git a/src/test/hive/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 b/shark/src/test/hive/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 similarity index 100% rename from src/test/hive/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 rename to shark/src/test/hive/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 diff --git a/src/test/hive/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 b/shark/src/test/hive/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 similarity index 100% rename from src/test/hive/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 rename to shark/src/test/hive/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 diff --git a/src/test/hive/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 b/shark/src/test/hive/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 similarity index 100% rename from src/test/hive/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 rename to shark/src/test/hive/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 diff --git a/src/test/hive/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 b/shark/src/test/hive/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 similarity index 100% rename from src/test/hive/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 rename to shark/src/test/hive/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 diff --git a/src/test/hive/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 b/shark/src/test/hive/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 similarity index 100% rename from src/test/hive/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 rename to shark/src/test/hive/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 diff --git a/src/test/hive/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 b/shark/src/test/hive/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 similarity index 100% rename from src/test/hive/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 rename to shark/src/test/hive/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 diff --git a/src/test/hive/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 b/shark/src/test/hive/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 similarity index 100% rename from src/test/hive/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 rename to shark/src/test/hive/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 diff --git a/src/test/hive/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c b/shark/src/test/hive/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c similarity index 100% rename from src/test/hive/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c rename to shark/src/test/hive/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c diff --git a/src/test/hive/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d b/shark/src/test/hive/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d similarity index 100% rename from src/test/hive/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d rename to shark/src/test/hive/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d diff --git a/src/test/hive/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 b/shark/src/test/hive/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 similarity index 100% rename from src/test/hive/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 rename to shark/src/test/hive/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 diff --git a/src/test/hive/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 b/shark/src/test/hive/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 similarity index 100% rename from src/test/hive/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 rename to shark/src/test/hive/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 diff --git a/src/test/hive/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 b/shark/src/test/hive/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 similarity index 100% rename from src/test/hive/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 rename to shark/src/test/hive/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 diff --git a/src/test/hive/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 b/shark/src/test/hive/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 similarity index 100% rename from src/test/hive/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 rename to shark/src/test/hive/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 diff --git a/src/test/hive/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 b/shark/src/test/hive/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 similarity index 100% rename from src/test/hive/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 rename to shark/src/test/hive/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 diff --git a/src/test/hive/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 b/shark/src/test/hive/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 similarity index 100% rename from src/test/hive/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 rename to shark/src/test/hive/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 diff --git a/src/test/hive/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 b/shark/src/test/hive/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 similarity index 100% rename from src/test/hive/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 rename to shark/src/test/hive/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 diff --git a/src/test/hive/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 b/shark/src/test/hive/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 similarity index 100% rename from src/test/hive/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 rename to shark/src/test/hive/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 diff --git a/src/test/hive/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 b/shark/src/test/hive/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 similarity index 100% rename from src/test/hive/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 rename to shark/src/test/hive/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 diff --git a/src/test/hive/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e b/shark/src/test/hive/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e similarity index 100% rename from src/test/hive/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e rename to shark/src/test/hive/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e diff --git a/src/test/hive/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 b/shark/src/test/hive/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 similarity index 100% rename from src/test/hive/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 rename to shark/src/test/hive/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 diff --git a/src/test/hive/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c b/shark/src/test/hive/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c similarity index 100% rename from src/test/hive/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c rename to shark/src/test/hive/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c diff --git a/src/test/hive/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 b/shark/src/test/hive/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 similarity index 100% rename from src/test/hive/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 rename to shark/src/test/hive/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 diff --git a/src/test/hive/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 b/shark/src/test/hive/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 similarity index 100% rename from src/test/hive/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 rename to shark/src/test/hive/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 diff --git a/src/test/hive/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 b/shark/src/test/hive/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 similarity index 100% rename from src/test/hive/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 rename to shark/src/test/hive/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 diff --git a/src/test/hive/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf b/shark/src/test/hive/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf similarity index 100% rename from src/test/hive/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf rename to shark/src/test/hive/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf diff --git a/src/test/hive/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 b/shark/src/test/hive/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 similarity index 100% rename from src/test/hive/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 rename to shark/src/test/hive/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 diff --git a/src/test/hive/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 b/shark/src/test/hive/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 similarity index 100% rename from src/test/hive/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 rename to shark/src/test/hive/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 diff --git a/src/test/hive/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e b/shark/src/test/hive/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e similarity index 100% rename from src/test/hive/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e rename to shark/src/test/hive/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e diff --git a/src/test/hive/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 b/shark/src/test/hive/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 similarity index 100% rename from src/test/hive/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 rename to shark/src/test/hive/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 diff --git a/src/test/hive/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e b/shark/src/test/hive/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e similarity index 100% rename from src/test/hive/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e rename to shark/src/test/hive/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e diff --git a/src/test/hive/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 b/shark/src/test/hive/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 similarity index 100% rename from src/test/hive/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 rename to shark/src/test/hive/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 diff --git a/src/test/hive/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 b/shark/src/test/hive/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 similarity index 100% rename from src/test/hive/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 rename to shark/src/test/hive/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 diff --git a/src/test/hive/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc b/shark/src/test/hive/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc similarity index 100% rename from src/test/hive/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc rename to shark/src/test/hive/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc diff --git a/src/test/hive/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f b/shark/src/test/hive/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f similarity index 100% rename from src/test/hive/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f rename to shark/src/test/hive/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f diff --git a/src/test/hive/golden/order-0-5c2fda577771db3b316e0d2bd02d048a b/shark/src/test/hive/golden/order-0-5c2fda577771db3b316e0d2bd02d048a similarity index 100% rename from src/test/hive/golden/order-0-5c2fda577771db3b316e0d2bd02d048a rename to shark/src/test/hive/golden/order-0-5c2fda577771db3b316e0d2bd02d048a diff --git a/src/test/hive/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 b/shark/src/test/hive/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 similarity index 100% rename from src/test/hive/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 rename to shark/src/test/hive/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 diff --git a/src/test/hive/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf b/shark/src/test/hive/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf similarity index 100% rename from src/test/hive/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf rename to shark/src/test/hive/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf diff --git a/src/test/hive/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 b/shark/src/test/hive/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 similarity index 100% rename from src/test/hive/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 rename to shark/src/test/hive/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 diff --git a/src/test/hive/golden/order2-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/order2-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/order2-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/order2-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf b/shark/src/test/hive/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf similarity index 100% rename from src/test/hive/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf rename to shark/src/test/hive/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf diff --git a/src/test/hive/golden/order2-2-4162aa366dc0836eed15cc819226907f b/shark/src/test/hive/golden/order2-2-4162aa366dc0836eed15cc819226907f similarity index 100% rename from src/test/hive/golden/order2-2-4162aa366dc0836eed15cc819226907f rename to shark/src/test/hive/golden/order2-2-4162aa366dc0836eed15cc819226907f diff --git a/src/test/hive/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c b/shark/src/test/hive/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c similarity index 100% rename from src/test/hive/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c rename to shark/src/test/hive/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c diff --git a/src/test/hive/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 b/shark/src/test/hive/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 similarity index 100% rename from src/test/hive/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 rename to shark/src/test/hive/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 diff --git a/src/test/hive/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba b/shark/src/test/hive/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba similarity index 100% rename from src/test/hive/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba rename to shark/src/test/hive/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba diff --git a/src/test/hive/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 b/shark/src/test/hive/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 similarity index 100% rename from src/test/hive/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 rename to shark/src/test/hive/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 diff --git a/src/test/hive/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 b/shark/src/test/hive/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 similarity index 100% rename from src/test/hive/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 rename to shark/src/test/hive/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 diff --git a/src/test/hive/golden/parallel-1-851e262128626126ae1ad87869db7c54 b/shark/src/test/hive/golden/parallel-1-851e262128626126ae1ad87869db7c54 similarity index 100% rename from src/test/hive/golden/parallel-1-851e262128626126ae1ad87869db7c54 rename to shark/src/test/hive/golden/parallel-1-851e262128626126ae1ad87869db7c54 diff --git a/src/test/hive/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 b/shark/src/test/hive/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 similarity index 100% rename from src/test/hive/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 rename to shark/src/test/hive/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 diff --git a/src/test/hive/golden/parallel-11-6230286bc168af7b010968b543690a2a b/shark/src/test/hive/golden/parallel-11-6230286bc168af7b010968b543690a2a similarity index 100% rename from src/test/hive/golden/parallel-11-6230286bc168af7b010968b543690a2a rename to shark/src/test/hive/golden/parallel-11-6230286bc168af7b010968b543690a2a diff --git a/src/test/hive/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 b/shark/src/test/hive/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 similarity index 100% rename from src/test/hive/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 rename to shark/src/test/hive/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 diff --git a/src/test/hive/golden/parallel-2-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/parallel-2-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/parallel-2-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/parallel-2-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a b/shark/src/test/hive/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a similarity index 100% rename from src/test/hive/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a rename to shark/src/test/hive/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a diff --git a/src/test/hive/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 b/shark/src/test/hive/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 similarity index 100% rename from src/test/hive/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 rename to shark/src/test/hive/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 diff --git a/src/test/hive/golden/parallel-5-e27b511ffc5828b09069121031d17a2f b/shark/src/test/hive/golden/parallel-5-e27b511ffc5828b09069121031d17a2f similarity index 100% rename from src/test/hive/golden/parallel-5-e27b511ffc5828b09069121031d17a2f rename to shark/src/test/hive/golden/parallel-5-e27b511ffc5828b09069121031d17a2f diff --git a/src/test/hive/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 b/shark/src/test/hive/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 similarity index 100% rename from src/test/hive/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 rename to shark/src/test/hive/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 diff --git a/src/test/hive/golden/parallel-7-6230286bc168af7b010968b543690a2a b/shark/src/test/hive/golden/parallel-7-6230286bc168af7b010968b543690a2a similarity index 100% rename from src/test/hive/golden/parallel-7-6230286bc168af7b010968b543690a2a rename to shark/src/test/hive/golden/parallel-7-6230286bc168af7b010968b543690a2a diff --git a/src/test/hive/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 b/shark/src/test/hive/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 similarity index 100% rename from src/test/hive/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 rename to shark/src/test/hive/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 diff --git a/src/test/hive/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c b/shark/src/test/hive/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c similarity index 100% rename from src/test/hive/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c rename to shark/src/test/hive/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c diff --git a/src/test/hive/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 b/shark/src/test/hive/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 similarity index 100% rename from src/test/hive/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 rename to shark/src/test/hive/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 diff --git a/src/test/hive/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 b/shark/src/test/hive/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 similarity index 100% rename from src/test/hive/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 rename to shark/src/test/hive/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 diff --git a/src/test/hive/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc b/shark/src/test/hive/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc similarity index 100% rename from src/test/hive/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc rename to shark/src/test/hive/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc diff --git a/src/test/hive/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd b/shark/src/test/hive/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd similarity index 100% rename from src/test/hive/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd rename to shark/src/test/hive/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd diff --git a/src/test/hive/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec b/shark/src/test/hive/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec similarity index 100% rename from src/test/hive/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec rename to shark/src/test/hive/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec diff --git a/src/test/hive/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 b/shark/src/test/hive/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 similarity index 100% rename from src/test/hive/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 rename to shark/src/test/hive/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 diff --git a/src/test/hive/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 b/shark/src/test/hive/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 similarity index 100% rename from src/test/hive/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 rename to shark/src/test/hive/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 diff --git a/src/test/hive/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea b/shark/src/test/hive/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea similarity index 100% rename from src/test/hive/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea rename to shark/src/test/hive/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea diff --git a/src/test/hive/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 b/shark/src/test/hive/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 similarity index 100% rename from src/test/hive/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 rename to shark/src/test/hive/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 diff --git a/src/test/hive/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b b/shark/src/test/hive/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b similarity index 100% rename from src/test/hive/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b rename to shark/src/test/hive/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b diff --git a/src/test/hive/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 b/shark/src/test/hive/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 similarity index 100% rename from src/test/hive/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 rename to shark/src/test/hive/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 diff --git a/src/test/hive/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 b/shark/src/test/hive/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 similarity index 100% rename from src/test/hive/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 rename to shark/src/test/hive/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 diff --git a/src/test/hive/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 b/shark/src/test/hive/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 similarity index 100% rename from src/test/hive/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 rename to shark/src/test/hive/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 diff --git a/src/test/hive/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 b/shark/src/test/hive/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 rename to shark/src/test/hive/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 diff --git a/src/test/hive/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 b/shark/src/test/hive/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 rename to shark/src/test/hive/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 diff --git a/src/test/hive/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb b/shark/src/test/hive/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb rename to shark/src/test/hive/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb diff --git a/src/test/hive/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/shark/src/test/hive/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 rename to shark/src/test/hive/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 diff --git a/src/test/hive/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 b/shark/src/test/hive/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 rename to shark/src/test/hive/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 diff --git a/src/test/hive/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 b/shark/src/test/hive/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 rename to shark/src/test/hive/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 diff --git a/src/test/hive/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 b/shark/src/test/hive/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 rename to shark/src/test/hive/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 diff --git a/src/test/hive/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb b/shark/src/test/hive/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb rename to shark/src/test/hive/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb diff --git a/src/test/hive/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/shark/src/test/hive/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 rename to shark/src/test/hive/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 diff --git a/src/test/hive/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 b/shark/src/test/hive/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 rename to shark/src/test/hive/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 diff --git a/src/test/hive/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 b/shark/src/test/hive/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 rename to shark/src/test/hive/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 diff --git a/src/test/hive/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb b/shark/src/test/hive/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb rename to shark/src/test/hive/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb diff --git a/src/test/hive/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/shark/src/test/hive/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 rename to shark/src/test/hive/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 diff --git a/src/test/hive/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 b/shark/src/test/hive/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 similarity index 100% rename from src/test/hive/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 rename to shark/src/test/hive/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 diff --git a/src/test/hive/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a b/shark/src/test/hive/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a similarity index 100% rename from src/test/hive/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a rename to shark/src/test/hive/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a diff --git a/src/test/hive/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 b/shark/src/test/hive/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 similarity index 100% rename from src/test/hive/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 rename to shark/src/test/hive/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 diff --git a/src/test/hive/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 b/shark/src/test/hive/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 similarity index 100% rename from src/test/hive/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 rename to shark/src/test/hive/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 diff --git a/src/test/hive/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 b/shark/src/test/hive/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 similarity index 100% rename from src/test/hive/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 rename to shark/src/test/hive/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 diff --git a/src/test/hive/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 b/shark/src/test/hive/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 similarity index 100% rename from src/test/hive/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 rename to shark/src/test/hive/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 diff --git a/src/test/hive/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 b/shark/src/test/hive/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 similarity index 100% rename from src/test/hive/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 rename to shark/src/test/hive/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 diff --git a/src/test/hive/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 b/shark/src/test/hive/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 similarity index 100% rename from src/test/hive/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 rename to shark/src/test/hive/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 diff --git a/src/test/hive/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 b/shark/src/test/hive/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 similarity index 100% rename from src/test/hive/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 rename to shark/src/test/hive/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 diff --git a/src/test/hive/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/shark/src/test/hive/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 similarity index 100% rename from src/test/hive/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 rename to shark/src/test/hive/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 diff --git a/src/test/hive/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 b/shark/src/test/hive/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 similarity index 100% rename from src/test/hive/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 rename to shark/src/test/hive/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 diff --git a/src/test/hive/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 b/shark/src/test/hive/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 similarity index 100% rename from src/test/hive/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 rename to shark/src/test/hive/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 diff --git a/src/test/hive/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 b/shark/src/test/hive/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 similarity index 100% rename from src/test/hive/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 rename to shark/src/test/hive/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 diff --git a/src/test/hive/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/shark/src/test/hive/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 similarity index 100% rename from src/test/hive/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 rename to shark/src/test/hive/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 diff --git a/src/test/hive/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c b/shark/src/test/hive/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c similarity index 100% rename from src/test/hive/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c rename to shark/src/test/hive/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c diff --git a/src/test/hive/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 b/shark/src/test/hive/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 similarity index 100% rename from src/test/hive/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 rename to shark/src/test/hive/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 diff --git a/src/test/hive/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 b/shark/src/test/hive/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 similarity index 100% rename from src/test/hive/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 rename to shark/src/test/hive/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 diff --git a/src/test/hive/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb b/shark/src/test/hive/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb similarity index 100% rename from src/test/hive/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb rename to shark/src/test/hive/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb diff --git a/src/test/hive/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 b/shark/src/test/hive/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 similarity index 100% rename from src/test/hive/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 rename to shark/src/test/hive/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 diff --git a/src/test/hive/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 b/shark/src/test/hive/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 similarity index 100% rename from src/test/hive/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 rename to shark/src/test/hive/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 diff --git a/src/test/hive/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d b/shark/src/test/hive/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d similarity index 100% rename from src/test/hive/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d rename to shark/src/test/hive/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d diff --git a/src/test/hive/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 b/shark/src/test/hive/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 similarity index 100% rename from src/test/hive/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 rename to shark/src/test/hive/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 diff --git a/src/test/hive/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf b/shark/src/test/hive/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf similarity index 100% rename from src/test/hive/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf rename to shark/src/test/hive/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf diff --git a/src/test/hive/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 b/shark/src/test/hive/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 similarity index 100% rename from src/test/hive/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 rename to shark/src/test/hive/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 diff --git a/src/test/hive/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 b/shark/src/test/hive/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 similarity index 100% rename from src/test/hive/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 rename to shark/src/test/hive/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 diff --git a/src/test/hive/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 b/shark/src/test/hive/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 similarity index 100% rename from src/test/hive/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 rename to shark/src/test/hive/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 diff --git a/src/test/hive/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/shark/src/test/hive/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 similarity index 100% rename from src/test/hive/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 rename to shark/src/test/hive/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 diff --git a/src/test/hive/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 b/shark/src/test/hive/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 similarity index 100% rename from src/test/hive/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 rename to shark/src/test/hive/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 diff --git a/src/test/hive/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 b/shark/src/test/hive/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 similarity index 100% rename from src/test/hive/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 rename to shark/src/test/hive/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 diff --git a/src/test/hive/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/shark/src/test/hive/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 similarity index 100% rename from src/test/hive/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 rename to shark/src/test/hive/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 diff --git a/src/test/hive/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 b/shark/src/test/hive/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 similarity index 100% rename from src/test/hive/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 rename to shark/src/test/hive/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 diff --git a/src/test/hive/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 b/shark/src/test/hive/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 similarity index 100% rename from src/test/hive/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 rename to shark/src/test/hive/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 diff --git a/src/test/hive/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c b/shark/src/test/hive/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c similarity index 100% rename from src/test/hive/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c rename to shark/src/test/hive/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c diff --git a/src/test/hive/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa b/shark/src/test/hive/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa similarity index 100% rename from src/test/hive/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa rename to shark/src/test/hive/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa diff --git a/src/test/hive/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc b/shark/src/test/hive/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc similarity index 100% rename from src/test/hive/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc rename to shark/src/test/hive/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc diff --git a/src/test/hive/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a b/shark/src/test/hive/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a similarity index 100% rename from src/test/hive/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a rename to shark/src/test/hive/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a diff --git a/src/test/hive/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f b/shark/src/test/hive/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f similarity index 100% rename from src/test/hive/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f rename to shark/src/test/hive/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f diff --git a/src/test/hive/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 b/shark/src/test/hive/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 similarity index 100% rename from src/test/hive/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 rename to shark/src/test/hive/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 diff --git a/src/test/hive/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d b/shark/src/test/hive/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d similarity index 100% rename from src/test/hive/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d rename to shark/src/test/hive/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d diff --git a/src/test/hive/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 b/shark/src/test/hive/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 similarity index 100% rename from src/test/hive/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 rename to shark/src/test/hive/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 diff --git a/src/test/hive/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 b/shark/src/test/hive/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 similarity index 100% rename from src/test/hive/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 rename to shark/src/test/hive/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 diff --git a/src/test/hive/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 b/shark/src/test/hive/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 similarity index 100% rename from src/test/hive/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 rename to shark/src/test/hive/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 diff --git a/src/test/hive/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d b/shark/src/test/hive/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d similarity index 100% rename from src/test/hive/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d rename to shark/src/test/hive/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d diff --git a/src/test/hive/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 b/shark/src/test/hive/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 similarity index 100% rename from src/test/hive/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 rename to shark/src/test/hive/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 diff --git a/src/test/hive/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 b/shark/src/test/hive/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 similarity index 100% rename from src/test/hive/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 rename to shark/src/test/hive/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 diff --git a/src/test/hive/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 b/shark/src/test/hive/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 similarity index 100% rename from src/test/hive/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 rename to shark/src/test/hive/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 diff --git a/src/test/hive/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 b/shark/src/test/hive/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 similarity index 100% rename from src/test/hive/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 rename to shark/src/test/hive/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 diff --git a/src/test/hive/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 b/shark/src/test/hive/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 similarity index 100% rename from src/test/hive/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 rename to shark/src/test/hive/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 diff --git a/src/test/hive/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 b/shark/src/test/hive/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 similarity index 100% rename from src/test/hive/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 rename to shark/src/test/hive/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 diff --git a/src/test/hive/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c b/shark/src/test/hive/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c similarity index 100% rename from src/test/hive/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c rename to shark/src/test/hive/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c diff --git a/src/test/hive/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 b/shark/src/test/hive/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 similarity index 100% rename from src/test/hive/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 rename to shark/src/test/hive/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 diff --git a/src/test/hive/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 b/shark/src/test/hive/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 similarity index 100% rename from src/test/hive/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 rename to shark/src/test/hive/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 diff --git a/src/test/hive/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d b/shark/src/test/hive/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d similarity index 100% rename from src/test/hive/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d rename to shark/src/test/hive/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d diff --git a/src/test/hive/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 b/shark/src/test/hive/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 similarity index 100% rename from src/test/hive/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 rename to shark/src/test/hive/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 diff --git a/src/test/hive/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a b/shark/src/test/hive/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a similarity index 100% rename from src/test/hive/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a rename to shark/src/test/hive/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a diff --git a/src/test/hive/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 b/shark/src/test/hive/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 similarity index 100% rename from src/test/hive/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 rename to shark/src/test/hive/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 diff --git a/src/test/hive/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 b/shark/src/test/hive/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 similarity index 100% rename from src/test/hive/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 rename to shark/src/test/hive/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 diff --git a/src/test/hive/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 b/shark/src/test/hive/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 similarity index 100% rename from src/test/hive/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 rename to shark/src/test/hive/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 diff --git a/src/test/hive/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef b/shark/src/test/hive/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef similarity index 100% rename from src/test/hive/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef rename to shark/src/test/hive/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef diff --git a/src/test/hive/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce b/shark/src/test/hive/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce similarity index 100% rename from src/test/hive/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce rename to shark/src/test/hive/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce diff --git a/src/test/hive/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 b/shark/src/test/hive/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 similarity index 100% rename from src/test/hive/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 rename to shark/src/test/hive/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 diff --git a/src/test/hive/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e b/shark/src/test/hive/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e similarity index 100% rename from src/test/hive/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e rename to shark/src/test/hive/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e diff --git a/src/test/hive/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b b/shark/src/test/hive/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b similarity index 100% rename from src/test/hive/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b rename to shark/src/test/hive/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b diff --git a/src/test/hive/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 b/shark/src/test/hive/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 similarity index 100% rename from src/test/hive/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 rename to shark/src/test/hive/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 diff --git a/src/test/hive/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 b/shark/src/test/hive/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 similarity index 100% rename from src/test/hive/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 rename to shark/src/test/hive/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 diff --git a/src/test/hive/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 b/shark/src/test/hive/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 similarity index 100% rename from src/test/hive/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 rename to shark/src/test/hive/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 diff --git a/src/test/hive/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf b/shark/src/test/hive/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf similarity index 100% rename from src/test/hive/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf rename to shark/src/test/hive/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf diff --git a/src/test/hive/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 b/shark/src/test/hive/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 similarity index 100% rename from src/test/hive/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 rename to shark/src/test/hive/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 diff --git a/src/test/hive/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde b/shark/src/test/hive/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde similarity index 100% rename from src/test/hive/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde rename to shark/src/test/hive/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde diff --git a/src/test/hive/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 b/shark/src/test/hive/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 rename to shark/src/test/hive/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 diff --git a/src/test/hive/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 b/shark/src/test/hive/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 rename to shark/src/test/hive/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 diff --git a/src/test/hive/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/shark/src/test/hive/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a rename to shark/src/test/hive/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a diff --git a/src/test/hive/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 b/shark/src/test/hive/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 rename to shark/src/test/hive/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 diff --git a/src/test/hive/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 b/shark/src/test/hive/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 rename to shark/src/test/hive/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 diff --git a/src/test/hive/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 b/shark/src/test/hive/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 rename to shark/src/test/hive/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 diff --git a/src/test/hive/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f b/shark/src/test/hive/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f rename to shark/src/test/hive/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f diff --git a/src/test/hive/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff b/shark/src/test/hive/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff rename to shark/src/test/hive/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff diff --git a/src/test/hive/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 b/shark/src/test/hive/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 rename to shark/src/test/hive/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 diff --git a/src/test/hive/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 b/shark/src/test/hive/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 rename to shark/src/test/hive/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 diff --git a/src/test/hive/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/shark/src/test/hive/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a rename to shark/src/test/hive/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a diff --git a/src/test/hive/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 b/shark/src/test/hive/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 rename to shark/src/test/hive/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 diff --git a/src/test/hive/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 b/shark/src/test/hive/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 rename to shark/src/test/hive/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 diff --git a/src/test/hive/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 b/shark/src/test/hive/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 rename to shark/src/test/hive/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 diff --git a/src/test/hive/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 b/shark/src/test/hive/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 rename to shark/src/test/hive/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 diff --git a/src/test/hive/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a b/shark/src/test/hive/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a rename to shark/src/test/hive/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a diff --git a/src/test/hive/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 b/shark/src/test/hive/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 rename to shark/src/test/hive/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 diff --git a/src/test/hive/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f b/shark/src/test/hive/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f rename to shark/src/test/hive/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f diff --git a/src/test/hive/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 b/shark/src/test/hive/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 rename to shark/src/test/hive/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 diff --git a/src/test/hive/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 b/shark/src/test/hive/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 rename to shark/src/test/hive/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 diff --git a/src/test/hive/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 b/shark/src/test/hive/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 rename to shark/src/test/hive/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 diff --git a/src/test/hive/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 b/shark/src/test/hive/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 rename to shark/src/test/hive/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 diff --git a/src/test/hive/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a b/shark/src/test/hive/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a rename to shark/src/test/hive/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a diff --git a/src/test/hive/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 b/shark/src/test/hive/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 rename to shark/src/test/hive/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 diff --git a/src/test/hive/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 b/shark/src/test/hive/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 rename to shark/src/test/hive/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 diff --git a/src/test/hive/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 b/shark/src/test/hive/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 rename to shark/src/test/hive/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 diff --git a/src/test/hive/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 b/shark/src/test/hive/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 rename to shark/src/test/hive/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 diff --git a/src/test/hive/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e b/shark/src/test/hive/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e rename to shark/src/test/hive/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e diff --git a/src/test/hive/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf b/shark/src/test/hive/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf rename to shark/src/test/hive/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf diff --git a/src/test/hive/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 b/shark/src/test/hive/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 rename to shark/src/test/hive/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 diff --git a/src/test/hive/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf b/shark/src/test/hive/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf rename to shark/src/test/hive/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf diff --git a/src/test/hive/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e b/shark/src/test/hive/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e rename to shark/src/test/hive/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e diff --git a/src/test/hive/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 b/shark/src/test/hive/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 rename to shark/src/test/hive/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 diff --git a/src/test/hive/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 b/shark/src/test/hive/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 rename to shark/src/test/hive/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 diff --git a/src/test/hive/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 b/shark/src/test/hive/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 rename to shark/src/test/hive/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 diff --git a/src/test/hive/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f b/shark/src/test/hive/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f rename to shark/src/test/hive/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f diff --git a/src/test/hive/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba b/shark/src/test/hive/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba rename to shark/src/test/hive/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba diff --git a/src/test/hive/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 b/shark/src/test/hive/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 rename to shark/src/test/hive/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 diff --git a/src/test/hive/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba b/shark/src/test/hive/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba rename to shark/src/test/hive/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba diff --git a/src/test/hive/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 b/shark/src/test/hive/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 rename to shark/src/test/hive/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 diff --git a/src/test/hive/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 b/shark/src/test/hive/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 rename to shark/src/test/hive/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 diff --git a/src/test/hive/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 b/shark/src/test/hive/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 rename to shark/src/test/hive/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 diff --git a/src/test/hive/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 b/shark/src/test/hive/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 rename to shark/src/test/hive/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 diff --git a/src/test/hive/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f b/shark/src/test/hive/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f rename to shark/src/test/hive/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f diff --git a/src/test/hive/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d b/shark/src/test/hive/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d rename to shark/src/test/hive/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d diff --git a/src/test/hive/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 b/shark/src/test/hive/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 rename to shark/src/test/hive/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 diff --git a/src/test/hive/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 b/shark/src/test/hive/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 rename to shark/src/test/hive/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 diff --git a/src/test/hive/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 b/shark/src/test/hive/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 rename to shark/src/test/hive/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 diff --git a/src/test/hive/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 b/shark/src/test/hive/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 rename to shark/src/test/hive/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 diff --git a/src/test/hive/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af b/shark/src/test/hive/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af rename to shark/src/test/hive/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af diff --git a/src/test/hive/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b b/shark/src/test/hive/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b rename to shark/src/test/hive/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b diff --git a/src/test/hive/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 b/shark/src/test/hive/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 rename to shark/src/test/hive/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 diff --git a/src/test/hive/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 b/shark/src/test/hive/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 rename to shark/src/test/hive/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 diff --git a/src/test/hive/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b b/shark/src/test/hive/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b rename to shark/src/test/hive/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b diff --git a/src/test/hive/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d b/shark/src/test/hive/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d rename to shark/src/test/hive/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d diff --git a/src/test/hive/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 b/shark/src/test/hive/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 rename to shark/src/test/hive/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 diff --git a/src/test/hive/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe b/shark/src/test/hive/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe rename to shark/src/test/hive/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe diff --git a/src/test/hive/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 b/shark/src/test/hive/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 rename to shark/src/test/hive/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 diff --git a/src/test/hive/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 b/shark/src/test/hive/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 rename to shark/src/test/hive/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 diff --git a/src/test/hive/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 b/shark/src/test/hive/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 rename to shark/src/test/hive/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 diff --git a/src/test/hive/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 b/shark/src/test/hive/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 rename to shark/src/test/hive/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 diff --git a/src/test/hive/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 b/shark/src/test/hive/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 rename to shark/src/test/hive/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 diff --git a/src/test/hive/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 b/shark/src/test/hive/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 rename to shark/src/test/hive/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 diff --git a/src/test/hive/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 b/shark/src/test/hive/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 rename to shark/src/test/hive/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 diff --git a/src/test/hive/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c b/shark/src/test/hive/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c rename to shark/src/test/hive/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c diff --git a/src/test/hive/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 b/shark/src/test/hive/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 rename to shark/src/test/hive/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 diff --git a/src/test/hive/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 b/shark/src/test/hive/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 rename to shark/src/test/hive/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 diff --git a/src/test/hive/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b b/shark/src/test/hive/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b rename to shark/src/test/hive/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b diff --git a/src/test/hive/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 b/shark/src/test/hive/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 rename to shark/src/test/hive/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 diff --git a/src/test/hive/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 b/shark/src/test/hive/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 rename to shark/src/test/hive/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 diff --git a/src/test/hive/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 b/shark/src/test/hive/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 rename to shark/src/test/hive/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 diff --git a/src/test/hive/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b b/shark/src/test/hive/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b rename to shark/src/test/hive/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b diff --git a/src/test/hive/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f b/shark/src/test/hive/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f rename to shark/src/test/hive/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f diff --git a/src/test/hive/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba b/shark/src/test/hive/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba rename to shark/src/test/hive/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba diff --git a/src/test/hive/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 b/shark/src/test/hive/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 rename to shark/src/test/hive/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 diff --git a/src/test/hive/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 b/shark/src/test/hive/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 rename to shark/src/test/hive/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 diff --git a/src/test/hive/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 b/shark/src/test/hive/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 rename to shark/src/test/hive/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 diff --git a/src/test/hive/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b b/shark/src/test/hive/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b rename to shark/src/test/hive/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b diff --git a/src/test/hive/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f b/shark/src/test/hive/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f rename to shark/src/test/hive/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f diff --git a/src/test/hive/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd b/shark/src/test/hive/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd rename to shark/src/test/hive/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd diff --git a/src/test/hive/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba b/shark/src/test/hive/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba rename to shark/src/test/hive/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba diff --git a/src/test/hive/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b b/shark/src/test/hive/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b rename to shark/src/test/hive/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b diff --git a/src/test/hive/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 b/shark/src/test/hive/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 rename to shark/src/test/hive/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 diff --git a/src/test/hive/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 b/shark/src/test/hive/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 rename to shark/src/test/hive/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 diff --git a/src/test/hive/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 b/shark/src/test/hive/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 rename to shark/src/test/hive/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 diff --git a/src/test/hive/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 b/shark/src/test/hive/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 rename to shark/src/test/hive/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 diff --git a/src/test/hive/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d b/shark/src/test/hive/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d rename to shark/src/test/hive/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d diff --git a/src/test/hive/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 b/shark/src/test/hive/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 rename to shark/src/test/hive/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 diff --git a/src/test/hive/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 b/shark/src/test/hive/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 rename to shark/src/test/hive/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 diff --git a/src/test/hive/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a b/shark/src/test/hive/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a rename to shark/src/test/hive/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a diff --git a/src/test/hive/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f b/shark/src/test/hive/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f rename to shark/src/test/hive/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f diff --git a/src/test/hive/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff b/shark/src/test/hive/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff rename to shark/src/test/hive/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff diff --git a/src/test/hive/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 b/shark/src/test/hive/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 rename to shark/src/test/hive/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 diff --git a/src/test/hive/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d b/shark/src/test/hive/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d rename to shark/src/test/hive/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d diff --git a/src/test/hive/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 b/shark/src/test/hive/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 rename to shark/src/test/hive/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 diff --git a/src/test/hive/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada b/shark/src/test/hive/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada rename to shark/src/test/hive/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada diff --git a/src/test/hive/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 b/shark/src/test/hive/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 rename to shark/src/test/hive/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 diff --git a/src/test/hive/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 b/shark/src/test/hive/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 rename to shark/src/test/hive/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 diff --git a/src/test/hive/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 b/shark/src/test/hive/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 rename to shark/src/test/hive/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 diff --git a/src/test/hive/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a b/shark/src/test/hive/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a rename to shark/src/test/hive/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a diff --git a/src/test/hive/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/shark/src/test/hive/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a rename to shark/src/test/hive/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a diff --git a/src/test/hive/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f b/shark/src/test/hive/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f rename to shark/src/test/hive/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f diff --git a/src/test/hive/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff b/shark/src/test/hive/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff rename to shark/src/test/hive/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff diff --git a/src/test/hive/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 b/shark/src/test/hive/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 rename to shark/src/test/hive/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 diff --git a/src/test/hive/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a b/shark/src/test/hive/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a rename to shark/src/test/hive/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a diff --git a/src/test/hive/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a b/shark/src/test/hive/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a rename to shark/src/test/hive/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a diff --git a/src/test/hive/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/shark/src/test/hive/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a rename to shark/src/test/hive/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a diff --git a/src/test/hive/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 b/shark/src/test/hive/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 rename to shark/src/test/hive/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 diff --git a/src/test/hive/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 b/shark/src/test/hive/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 rename to shark/src/test/hive/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 diff --git a/src/test/hive/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 b/shark/src/test/hive/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 rename to shark/src/test/hive/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 diff --git a/src/test/hive/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 b/shark/src/test/hive/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 rename to shark/src/test/hive/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 diff --git a/src/test/hive/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 b/shark/src/test/hive/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 rename to shark/src/test/hive/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 diff --git a/src/test/hive/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 b/shark/src/test/hive/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 rename to shark/src/test/hive/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 diff --git a/src/test/hive/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 b/shark/src/test/hive/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 rename to shark/src/test/hive/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 diff --git a/src/test/hive/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 b/shark/src/test/hive/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 rename to shark/src/test/hive/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 diff --git a/src/test/hive/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a b/shark/src/test/hive/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a rename to shark/src/test/hive/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a diff --git a/src/test/hive/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f b/shark/src/test/hive/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f rename to shark/src/test/hive/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f diff --git a/src/test/hive/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff b/shark/src/test/hive/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff rename to shark/src/test/hive/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff diff --git a/src/test/hive/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 b/shark/src/test/hive/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 rename to shark/src/test/hive/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 diff --git a/src/test/hive/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad b/shark/src/test/hive/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad rename to shark/src/test/hive/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad diff --git a/src/test/hive/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 b/shark/src/test/hive/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 rename to shark/src/test/hive/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 diff --git a/src/test/hive/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 b/shark/src/test/hive/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 rename to shark/src/test/hive/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 diff --git a/src/test/hive/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 b/shark/src/test/hive/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 rename to shark/src/test/hive/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 diff --git a/src/test/hive/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a b/shark/src/test/hive/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a rename to shark/src/test/hive/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a diff --git a/src/test/hive/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f b/shark/src/test/hive/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f rename to shark/src/test/hive/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f diff --git a/src/test/hive/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff b/shark/src/test/hive/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff rename to shark/src/test/hive/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff diff --git a/src/test/hive/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f b/shark/src/test/hive/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f rename to shark/src/test/hive/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f diff --git a/src/test/hive/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 b/shark/src/test/hive/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 rename to shark/src/test/hive/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 diff --git a/src/test/hive/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 b/shark/src/test/hive/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 rename to shark/src/test/hive/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 diff --git a/src/test/hive/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 b/shark/src/test/hive/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 rename to shark/src/test/hive/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 diff --git a/src/test/hive/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a b/shark/src/test/hive/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a rename to shark/src/test/hive/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a diff --git a/src/test/hive/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 b/shark/src/test/hive/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 rename to shark/src/test/hive/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 diff --git a/src/test/hive/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 b/shark/src/test/hive/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 rename to shark/src/test/hive/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 diff --git a/src/test/hive/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 b/shark/src/test/hive/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 rename to shark/src/test/hive/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 diff --git a/src/test/hive/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 b/shark/src/test/hive/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 rename to shark/src/test/hive/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 diff --git a/src/test/hive/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 b/shark/src/test/hive/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 rename to shark/src/test/hive/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 diff --git a/src/test/hive/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 b/shark/src/test/hive/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 rename to shark/src/test/hive/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 diff --git a/src/test/hive/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e b/shark/src/test/hive/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e rename to shark/src/test/hive/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e diff --git a/src/test/hive/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e b/shark/src/test/hive/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e rename to shark/src/test/hive/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e diff --git a/src/test/hive/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b b/shark/src/test/hive/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b rename to shark/src/test/hive/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b diff --git a/src/test/hive/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd b/shark/src/test/hive/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd rename to shark/src/test/hive/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd diff --git a/src/test/hive/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf b/shark/src/test/hive/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf rename to shark/src/test/hive/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf diff --git a/src/test/hive/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 b/shark/src/test/hive/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 rename to shark/src/test/hive/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 diff --git a/src/test/hive/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 b/shark/src/test/hive/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 rename to shark/src/test/hive/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 diff --git a/src/test/hive/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 b/shark/src/test/hive/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 rename to shark/src/test/hive/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 diff --git a/src/test/hive/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 b/shark/src/test/hive/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 rename to shark/src/test/hive/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 diff --git a/src/test/hive/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b b/shark/src/test/hive/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b rename to shark/src/test/hive/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b diff --git a/src/test/hive/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd b/shark/src/test/hive/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd similarity index 100% rename from src/test/hive/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd rename to shark/src/test/hive/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd diff --git a/src/test/hive/golden/plan_json-0-74146da55d57b22443140e7fbab3375c b/shark/src/test/hive/golden/plan_json-0-74146da55d57b22443140e7fbab3375c similarity index 100% rename from src/test/hive/golden/plan_json-0-74146da55d57b22443140e7fbab3375c rename to shark/src/test/hive/golden/plan_json-0-74146da55d57b22443140e7fbab3375c diff --git a/src/test/hive/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 b/shark/src/test/hive/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 similarity index 100% rename from src/test/hive/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 rename to shark/src/test/hive/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 diff --git a/src/test/hive/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 b/shark/src/test/hive/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 similarity index 100% rename from src/test/hive/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 rename to shark/src/test/hive/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 diff --git a/src/test/hive/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 b/shark/src/test/hive/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 similarity index 100% rename from src/test/hive/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 rename to shark/src/test/hive/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 diff --git a/src/test/hive/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 b/shark/src/test/hive/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 similarity index 100% rename from src/test/hive/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 rename to shark/src/test/hive/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 diff --git a/src/test/hive/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 b/shark/src/test/hive/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 similarity index 100% rename from src/test/hive/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 rename to shark/src/test/hive/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 diff --git a/src/test/hive/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b b/shark/src/test/hive/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b similarity index 100% rename from src/test/hive/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b rename to shark/src/test/hive/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b diff --git a/src/test/hive/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 b/shark/src/test/hive/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 similarity index 100% rename from src/test/hive/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 rename to shark/src/test/hive/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 diff --git a/src/test/hive/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 b/shark/src/test/hive/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 similarity index 100% rename from src/test/hive/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 rename to shark/src/test/hive/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 diff --git a/src/test/hive/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 b/shark/src/test/hive/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 similarity index 100% rename from src/test/hive/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 rename to shark/src/test/hive/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 diff --git a/src/test/hive/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b b/shark/src/test/hive/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b similarity index 100% rename from src/test/hive/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b rename to shark/src/test/hive/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b diff --git a/src/test/hive/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 b/shark/src/test/hive/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 rename to shark/src/test/hive/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 diff --git a/src/test/hive/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 b/shark/src/test/hive/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 rename to shark/src/test/hive/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 diff --git a/src/test/hive/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 b/shark/src/test/hive/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 rename to shark/src/test/hive/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 diff --git a/src/test/hive/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 b/shark/src/test/hive/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 rename to shark/src/test/hive/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 diff --git a/src/test/hive/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 b/shark/src/test/hive/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 rename to shark/src/test/hive/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 diff --git a/src/test/hive/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 b/shark/src/test/hive/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 rename to shark/src/test/hive/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 diff --git a/src/test/hive/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 b/shark/src/test/hive/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 rename to shark/src/test/hive/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 diff --git a/src/test/hive/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 b/shark/src/test/hive/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 similarity index 100% rename from src/test/hive/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 rename to shark/src/test/hive/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 diff --git a/src/test/hive/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 b/shark/src/test/hive/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 similarity index 100% rename from src/test/hive/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 rename to shark/src/test/hive/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 diff --git a/src/test/hive/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 b/shark/src/test/hive/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 similarity index 100% rename from src/test/hive/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 rename to shark/src/test/hive/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 diff --git a/src/test/hive/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 b/shark/src/test/hive/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 similarity index 100% rename from src/test/hive/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 rename to shark/src/test/hive/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 diff --git a/src/test/hive/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c b/shark/src/test/hive/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c similarity index 100% rename from src/test/hive/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c rename to shark/src/test/hive/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c diff --git a/src/test/hive/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 b/shark/src/test/hive/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 similarity index 100% rename from src/test/hive/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 rename to shark/src/test/hive/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 diff --git a/src/test/hive/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 b/shark/src/test/hive/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 similarity index 100% rename from src/test/hive/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 rename to shark/src/test/hive/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 diff --git a/src/test/hive/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c b/shark/src/test/hive/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c similarity index 100% rename from src/test/hive/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c rename to shark/src/test/hive/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c diff --git a/src/test/hive/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 b/shark/src/test/hive/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 similarity index 100% rename from src/test/hive/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 rename to shark/src/test/hive/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 diff --git a/src/test/hive/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 b/shark/src/test/hive/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 similarity index 100% rename from src/test/hive/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 rename to shark/src/test/hive/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 diff --git a/src/test/hive/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e b/shark/src/test/hive/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e similarity index 100% rename from src/test/hive/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e rename to shark/src/test/hive/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e diff --git a/src/test/hive/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d b/shark/src/test/hive/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d similarity index 100% rename from src/test/hive/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d rename to shark/src/test/hive/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d diff --git a/src/test/hive/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e b/shark/src/test/hive/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e similarity index 100% rename from src/test/hive/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e rename to shark/src/test/hive/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e diff --git a/src/test/hive/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d b/shark/src/test/hive/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d similarity index 100% rename from src/test/hive/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d rename to shark/src/test/hive/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d diff --git a/src/test/hive/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 b/shark/src/test/hive/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 similarity index 100% rename from src/test/hive/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 rename to shark/src/test/hive/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 diff --git a/src/test/hive/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f b/shark/src/test/hive/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f similarity index 100% rename from src/test/hive/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f rename to shark/src/test/hive/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f diff --git a/src/test/hive/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 b/shark/src/test/hive/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 similarity index 100% rename from src/test/hive/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 rename to shark/src/test/hive/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 diff --git a/src/test/hive/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f b/shark/src/test/hive/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f similarity index 100% rename from src/test/hive/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f rename to shark/src/test/hive/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f diff --git a/src/test/hive/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 b/shark/src/test/hive/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 similarity index 100% rename from src/test/hive/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 rename to shark/src/test/hive/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 diff --git a/src/test/hive/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 b/shark/src/test/hive/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 similarity index 100% rename from src/test/hive/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 rename to shark/src/test/hive/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 diff --git a/src/test/hive/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 b/shark/src/test/hive/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 similarity index 100% rename from src/test/hive/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 rename to shark/src/test/hive/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 diff --git a/src/test/hive/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 b/shark/src/test/hive/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 similarity index 100% rename from src/test/hive/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 rename to shark/src/test/hive/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 diff --git a/src/test/hive/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 b/shark/src/test/hive/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 similarity index 100% rename from src/test/hive/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 rename to shark/src/test/hive/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 diff --git a/src/test/hive/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 b/shark/src/test/hive/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 similarity index 100% rename from src/test/hive/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 rename to shark/src/test/hive/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 diff --git a/src/test/hive/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d b/shark/src/test/hive/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d similarity index 100% rename from src/test/hive/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d rename to shark/src/test/hive/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d diff --git a/src/test/hive/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 b/shark/src/test/hive/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 similarity index 100% rename from src/test/hive/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 rename to shark/src/test/hive/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 diff --git a/src/test/hive/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d b/shark/src/test/hive/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d similarity index 100% rename from src/test/hive/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d rename to shark/src/test/hive/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d diff --git a/src/test/hive/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 b/shark/src/test/hive/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 similarity index 100% rename from src/test/hive/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 rename to shark/src/test/hive/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 diff --git a/src/test/hive/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 b/shark/src/test/hive/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 similarity index 100% rename from src/test/hive/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 rename to shark/src/test/hive/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 diff --git a/src/test/hive/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b b/shark/src/test/hive/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b similarity index 100% rename from src/test/hive/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b rename to shark/src/test/hive/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b diff --git a/src/test/hive/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 b/shark/src/test/hive/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 similarity index 100% rename from src/test/hive/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 rename to shark/src/test/hive/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 diff --git a/src/test/hive/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b b/shark/src/test/hive/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b similarity index 100% rename from src/test/hive/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b rename to shark/src/test/hive/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b diff --git a/src/test/hive/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 b/shark/src/test/hive/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 rename to shark/src/test/hive/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 diff --git a/src/test/hive/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 b/shark/src/test/hive/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 rename to shark/src/test/hive/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 diff --git a/src/test/hive/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 b/shark/src/test/hive/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 rename to shark/src/test/hive/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 diff --git a/src/test/hive/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 b/shark/src/test/hive/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 rename to shark/src/test/hive/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 diff --git a/src/test/hive/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 b/shark/src/test/hive/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 rename to shark/src/test/hive/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 diff --git a/src/test/hive/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 b/shark/src/test/hive/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 rename to shark/src/test/hive/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 diff --git a/src/test/hive/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 b/shark/src/test/hive/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 rename to shark/src/test/hive/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 diff --git a/src/test/hive/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 b/shark/src/test/hive/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 rename to shark/src/test/hive/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 diff --git a/src/test/hive/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 b/shark/src/test/hive/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 rename to shark/src/test/hive/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 diff --git a/src/test/hive/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 b/shark/src/test/hive/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 rename to shark/src/test/hive/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 diff --git a/src/test/hive/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e b/shark/src/test/hive/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e similarity index 100% rename from src/test/hive/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e rename to shark/src/test/hive/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e diff --git a/src/test/hive/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 b/shark/src/test/hive/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 similarity index 100% rename from src/test/hive/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 rename to shark/src/test/hive/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 diff --git a/src/test/hive/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e b/shark/src/test/hive/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e similarity index 100% rename from src/test/hive/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e rename to shark/src/test/hive/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e diff --git a/src/test/hive/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 b/shark/src/test/hive/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 similarity index 100% rename from src/test/hive/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 rename to shark/src/test/hive/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 diff --git a/src/test/hive/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 b/shark/src/test/hive/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 similarity index 100% rename from src/test/hive/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 rename to shark/src/test/hive/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 diff --git a/src/test/hive/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 b/shark/src/test/hive/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 similarity index 100% rename from src/test/hive/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 rename to shark/src/test/hive/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 diff --git a/src/test/hive/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 b/shark/src/test/hive/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 similarity index 100% rename from src/test/hive/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 rename to shark/src/test/hive/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 diff --git a/src/test/hive/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 b/shark/src/test/hive/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 similarity index 100% rename from src/test/hive/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 rename to shark/src/test/hive/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 diff --git a/src/test/hive/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 b/shark/src/test/hive/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 similarity index 100% rename from src/test/hive/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 rename to shark/src/test/hive/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 diff --git a/src/test/hive/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 b/shark/src/test/hive/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 similarity index 100% rename from src/test/hive/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 rename to shark/src/test/hive/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 diff --git a/src/test/hive/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 b/shark/src/test/hive/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 similarity index 100% rename from src/test/hive/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 rename to shark/src/test/hive/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 diff --git a/src/test/hive/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 b/shark/src/test/hive/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 similarity index 100% rename from src/test/hive/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 rename to shark/src/test/hive/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 diff --git a/src/test/hive/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 b/shark/src/test/hive/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 similarity index 100% rename from src/test/hive/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 rename to shark/src/test/hive/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 diff --git a/src/test/hive/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe b/shark/src/test/hive/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe similarity index 100% rename from src/test/hive/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe rename to shark/src/test/hive/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe diff --git a/src/test/hive/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 b/shark/src/test/hive/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 similarity index 100% rename from src/test/hive/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 rename to shark/src/test/hive/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 diff --git a/src/test/hive/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe b/shark/src/test/hive/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe similarity index 100% rename from src/test/hive/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe rename to shark/src/test/hive/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe diff --git a/src/test/hive/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b b/shark/src/test/hive/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b rename to shark/src/test/hive/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b diff --git a/src/test/hive/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 b/shark/src/test/hive/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 rename to shark/src/test/hive/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 diff --git a/src/test/hive/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 b/shark/src/test/hive/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 rename to shark/src/test/hive/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 diff --git a/src/test/hive/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee b/shark/src/test/hive/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee rename to shark/src/test/hive/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee diff --git a/src/test/hive/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 b/shark/src/test/hive/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 rename to shark/src/test/hive/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 diff --git a/src/test/hive/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 b/shark/src/test/hive/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 rename to shark/src/test/hive/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 diff --git a/src/test/hive/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 b/shark/src/test/hive/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 rename to shark/src/test/hive/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 diff --git a/src/test/hive/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a b/shark/src/test/hive/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a rename to shark/src/test/hive/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a diff --git a/src/test/hive/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd b/shark/src/test/hive/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd rename to shark/src/test/hive/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd diff --git a/src/test/hive/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c b/shark/src/test/hive/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c rename to shark/src/test/hive/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c diff --git a/src/test/hive/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 b/shark/src/test/hive/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 similarity index 100% rename from src/test/hive/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 rename to shark/src/test/hive/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 diff --git a/src/test/hive/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b b/shark/src/test/hive/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b similarity index 100% rename from src/test/hive/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b rename to shark/src/test/hive/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b diff --git a/src/test/hive/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b b/shark/src/test/hive/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b similarity index 100% rename from src/test/hive/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b rename to shark/src/test/hive/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b diff --git a/src/test/hive/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 b/shark/src/test/hive/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 similarity index 100% rename from src/test/hive/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 rename to shark/src/test/hive/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 diff --git a/src/test/hive/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 b/shark/src/test/hive/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 similarity index 100% rename from src/test/hive/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 rename to shark/src/test/hive/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 diff --git a/src/test/hive/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 b/shark/src/test/hive/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 similarity index 100% rename from src/test/hive/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 rename to shark/src/test/hive/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 diff --git a/src/test/hive/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 b/shark/src/test/hive/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 similarity index 100% rename from src/test/hive/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 rename to shark/src/test/hive/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 diff --git a/src/test/hive/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f b/shark/src/test/hive/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f similarity index 100% rename from src/test/hive/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f rename to shark/src/test/hive/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f diff --git a/src/test/hive/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 b/shark/src/test/hive/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 similarity index 100% rename from src/test/hive/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 rename to shark/src/test/hive/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 diff --git a/src/test/hive/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 b/shark/src/test/hive/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 similarity index 100% rename from src/test/hive/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 rename to shark/src/test/hive/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 diff --git a/src/test/hive/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf b/shark/src/test/hive/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf similarity index 100% rename from src/test/hive/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf rename to shark/src/test/hive/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf diff --git a/src/test/hive/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 b/shark/src/test/hive/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 similarity index 100% rename from src/test/hive/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 rename to shark/src/test/hive/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 diff --git a/src/test/hive/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 b/shark/src/test/hive/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 similarity index 100% rename from src/test/hive/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 rename to shark/src/test/hive/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 diff --git a/src/test/hive/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/shark/src/test/hive/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 similarity index 100% rename from src/test/hive/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 rename to shark/src/test/hive/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 diff --git a/src/test/hive/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 b/shark/src/test/hive/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 similarity index 100% rename from src/test/hive/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 rename to shark/src/test/hive/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 diff --git a/src/test/hive/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/shark/src/test/hive/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 similarity index 100% rename from src/test/hive/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 rename to shark/src/test/hive/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 diff --git a/src/test/hive/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 b/shark/src/test/hive/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 similarity index 100% rename from src/test/hive/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 rename to shark/src/test/hive/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 diff --git a/src/test/hive/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 b/shark/src/test/hive/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 similarity index 100% rename from src/test/hive/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 rename to shark/src/test/hive/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 diff --git a/src/test/hive/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 b/shark/src/test/hive/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 similarity index 100% rename from src/test/hive/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 rename to shark/src/test/hive/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 diff --git a/src/test/hive/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e b/shark/src/test/hive/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e similarity index 100% rename from src/test/hive/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e rename to shark/src/test/hive/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e diff --git a/src/test/hive/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed b/shark/src/test/hive/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed similarity index 100% rename from src/test/hive/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed rename to shark/src/test/hive/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed diff --git a/src/test/hive/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae b/shark/src/test/hive/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae similarity index 100% rename from src/test/hive/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae rename to shark/src/test/hive/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae diff --git a/src/test/hive/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 b/shark/src/test/hive/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 similarity index 100% rename from src/test/hive/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 rename to shark/src/test/hive/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 diff --git a/src/test/hive/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e b/shark/src/test/hive/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e similarity index 100% rename from src/test/hive/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e rename to shark/src/test/hive/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e diff --git a/src/test/hive/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed b/shark/src/test/hive/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed similarity index 100% rename from src/test/hive/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed rename to shark/src/test/hive/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed diff --git a/src/test/hive/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae b/shark/src/test/hive/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae similarity index 100% rename from src/test/hive/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae rename to shark/src/test/hive/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae diff --git a/src/test/hive/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 b/shark/src/test/hive/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 similarity index 100% rename from src/test/hive/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 rename to shark/src/test/hive/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 diff --git a/src/test/hive/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 b/shark/src/test/hive/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 similarity index 100% rename from src/test/hive/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 rename to shark/src/test/hive/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 diff --git a/src/test/hive/golden/ppd_union-3-678c8197f458b459171c266f7431683e b/shark/src/test/hive/golden/ppd_union-3-678c8197f458b459171c266f7431683e similarity index 100% rename from src/test/hive/golden/ppd_union-3-678c8197f458b459171c266f7431683e rename to shark/src/test/hive/golden/ppd_union-3-678c8197f458b459171c266f7431683e diff --git a/src/test/hive/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 b/shark/src/test/hive/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 similarity index 100% rename from src/test/hive/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 rename to shark/src/test/hive/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 diff --git a/src/test/hive/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 b/shark/src/test/hive/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 similarity index 100% rename from src/test/hive/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 rename to shark/src/test/hive/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 diff --git a/src/test/hive/golden/ppd_union-6-678c8197f458b459171c266f7431683e b/shark/src/test/hive/golden/ppd_union-6-678c8197f458b459171c266f7431683e similarity index 100% rename from src/test/hive/golden/ppd_union-6-678c8197f458b459171c266f7431683e rename to shark/src/test/hive/golden/ppd_union-6-678c8197f458b459171c266f7431683e diff --git a/src/test/hive/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 b/shark/src/test/hive/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 similarity index 100% rename from src/test/hive/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 rename to shark/src/test/hive/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 diff --git a/src/test/hive/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 b/shark/src/test/hive/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 similarity index 100% rename from src/test/hive/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 rename to shark/src/test/hive/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 diff --git a/src/test/hive/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f b/shark/src/test/hive/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f similarity index 100% rename from src/test/hive/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f rename to shark/src/test/hive/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f diff --git a/src/test/hive/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c b/shark/src/test/hive/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c similarity index 100% rename from src/test/hive/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c rename to shark/src/test/hive/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c diff --git a/src/test/hive/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 b/shark/src/test/hive/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 similarity index 100% rename from src/test/hive/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 rename to shark/src/test/hive/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 diff --git a/src/test/hive/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 b/shark/src/test/hive/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 similarity index 100% rename from src/test/hive/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 rename to shark/src/test/hive/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 diff --git a/src/test/hive/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 b/shark/src/test/hive/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 similarity index 100% rename from src/test/hive/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 rename to shark/src/test/hive/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 diff --git a/src/test/hive/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 b/shark/src/test/hive/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 similarity index 100% rename from src/test/hive/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 rename to shark/src/test/hive/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 diff --git a/src/test/hive/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 b/shark/src/test/hive/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 similarity index 100% rename from src/test/hive/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 rename to shark/src/test/hive/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 diff --git a/src/test/hive/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c b/shark/src/test/hive/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c similarity index 100% rename from src/test/hive/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c rename to shark/src/test/hive/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c diff --git a/src/test/hive/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 b/shark/src/test/hive/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 similarity index 100% rename from src/test/hive/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 rename to shark/src/test/hive/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 diff --git a/src/test/hive/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c b/shark/src/test/hive/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c similarity index 100% rename from src/test/hive/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c rename to shark/src/test/hive/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c diff --git a/src/test/hive/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 b/shark/src/test/hive/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 similarity index 100% rename from src/test/hive/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 rename to shark/src/test/hive/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 diff --git a/src/test/hive/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 b/shark/src/test/hive/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 similarity index 100% rename from src/test/hive/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 rename to shark/src/test/hive/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 diff --git a/src/test/hive/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 b/shark/src/test/hive/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 similarity index 100% rename from src/test/hive/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 rename to shark/src/test/hive/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 diff --git a/src/test/hive/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 b/shark/src/test/hive/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 similarity index 100% rename from src/test/hive/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 rename to shark/src/test/hive/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 diff --git a/src/test/hive/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d b/shark/src/test/hive/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d similarity index 100% rename from src/test/hive/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d rename to shark/src/test/hive/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d diff --git a/src/test/hive/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 b/shark/src/test/hive/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 similarity index 100% rename from src/test/hive/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 rename to shark/src/test/hive/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 diff --git a/src/test/hive/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 b/shark/src/test/hive/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 similarity index 100% rename from src/test/hive/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 rename to shark/src/test/hive/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 diff --git a/src/test/hive/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d b/shark/src/test/hive/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d similarity index 100% rename from src/test/hive/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d rename to shark/src/test/hive/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d diff --git a/src/test/hive/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 b/shark/src/test/hive/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 similarity index 100% rename from src/test/hive/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 rename to shark/src/test/hive/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 diff --git a/src/test/hive/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b b/shark/src/test/hive/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b similarity index 100% rename from src/test/hive/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b rename to shark/src/test/hive/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b diff --git a/src/test/hive/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 b/shark/src/test/hive/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 rename to shark/src/test/hive/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 diff --git a/src/test/hive/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 b/shark/src/test/hive/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 rename to shark/src/test/hive/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 diff --git a/src/test/hive/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 b/shark/src/test/hive/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 rename to shark/src/test/hive/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 diff --git a/src/test/hive/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 b/shark/src/test/hive/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 rename to shark/src/test/hive/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 diff --git a/src/test/hive/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 b/shark/src/test/hive/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 rename to shark/src/test/hive/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 diff --git a/src/test/hive/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c b/shark/src/test/hive/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c similarity index 100% rename from src/test/hive/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c rename to shark/src/test/hive/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c diff --git a/src/test/hive/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb b/shark/src/test/hive/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb similarity index 100% rename from src/test/hive/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb rename to shark/src/test/hive/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb diff --git a/src/test/hive/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae b/shark/src/test/hive/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae similarity index 100% rename from src/test/hive/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae rename to shark/src/test/hive/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae diff --git a/src/test/hive/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 b/shark/src/test/hive/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 rename to shark/src/test/hive/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 diff --git a/src/test/hive/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 b/shark/src/test/hive/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 rename to shark/src/test/hive/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 diff --git a/src/test/hive/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a b/shark/src/test/hive/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a similarity index 100% rename from src/test/hive/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a rename to shark/src/test/hive/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a diff --git a/src/test/hive/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 b/shark/src/test/hive/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 rename to shark/src/test/hive/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 diff --git a/src/test/hive/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 b/shark/src/test/hive/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 rename to shark/src/test/hive/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 diff --git a/src/test/hive/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d b/shark/src/test/hive/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d similarity index 100% rename from src/test/hive/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d rename to shark/src/test/hive/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d diff --git a/src/test/hive/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d b/shark/src/test/hive/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d similarity index 100% rename from src/test/hive/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d rename to shark/src/test/hive/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d diff --git a/src/test/hive/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 b/shark/src/test/hive/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 rename to shark/src/test/hive/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 diff --git a/src/test/hive/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 b/shark/src/test/hive/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 rename to shark/src/test/hive/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 diff --git a/src/test/hive/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b b/shark/src/test/hive/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b similarity index 100% rename from src/test/hive/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b rename to shark/src/test/hive/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b diff --git a/src/test/hive/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 b/shark/src/test/hive/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 rename to shark/src/test/hive/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 diff --git a/src/test/hive/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 b/shark/src/test/hive/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 rename to shark/src/test/hive/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 diff --git a/src/test/hive/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 b/shark/src/test/hive/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 rename to shark/src/test/hive/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 diff --git a/src/test/hive/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea b/shark/src/test/hive/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea similarity index 100% rename from src/test/hive/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea rename to shark/src/test/hive/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea diff --git a/src/test/hive/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 b/shark/src/test/hive/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 rename to shark/src/test/hive/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 diff --git a/src/test/hive/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 b/shark/src/test/hive/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 rename to shark/src/test/hive/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 diff --git a/src/test/hive/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf b/shark/src/test/hive/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf similarity index 100% rename from src/test/hive/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf rename to shark/src/test/hive/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf diff --git a/src/test/hive/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b b/shark/src/test/hive/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b similarity index 100% rename from src/test/hive/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b rename to shark/src/test/hive/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b diff --git a/src/test/hive/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 b/shark/src/test/hive/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 rename to shark/src/test/hive/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 diff --git a/src/test/hive/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 b/shark/src/test/hive/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 rename to shark/src/test/hive/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 diff --git a/src/test/hive/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de b/shark/src/test/hive/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de similarity index 100% rename from src/test/hive/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de rename to shark/src/test/hive/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de diff --git a/src/test/hive/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d b/shark/src/test/hive/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d similarity index 100% rename from src/test/hive/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d rename to shark/src/test/hive/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d diff --git a/src/test/hive/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 b/shark/src/test/hive/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 rename to shark/src/test/hive/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 diff --git a/src/test/hive/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c b/shark/src/test/hive/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c similarity index 100% rename from src/test/hive/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c rename to shark/src/test/hive/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c diff --git a/src/test/hive/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 b/shark/src/test/hive/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 similarity index 100% rename from src/test/hive/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 rename to shark/src/test/hive/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 diff --git a/src/test/hive/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b b/shark/src/test/hive/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b rename to shark/src/test/hive/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b diff --git a/src/test/hive/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 b/shark/src/test/hive/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 rename to shark/src/test/hive/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 diff --git a/src/test/hive/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c b/shark/src/test/hive/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c rename to shark/src/test/hive/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c diff --git a/src/test/hive/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b b/shark/src/test/hive/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b rename to shark/src/test/hive/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b diff --git a/src/test/hive/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a b/shark/src/test/hive/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a rename to shark/src/test/hive/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a diff --git a/src/test/hive/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 b/shark/src/test/hive/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 rename to shark/src/test/hive/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 diff --git a/src/test/hive/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 b/shark/src/test/hive/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 rename to shark/src/test/hive/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 diff --git a/src/test/hive/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 b/shark/src/test/hive/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 rename to shark/src/test/hive/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 diff --git a/src/test/hive/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb b/shark/src/test/hive/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb rename to shark/src/test/hive/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb diff --git a/src/test/hive/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe b/shark/src/test/hive/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe rename to shark/src/test/hive/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe diff --git a/src/test/hive/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 b/shark/src/test/hive/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 rename to shark/src/test/hive/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 diff --git a/src/test/hive/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 b/shark/src/test/hive/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 rename to shark/src/test/hive/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 diff --git a/src/test/hive/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e b/shark/src/test/hive/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e rename to shark/src/test/hive/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e diff --git a/src/test/hive/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 b/shark/src/test/hive/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 rename to shark/src/test/hive/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 diff --git a/src/test/hive/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c b/shark/src/test/hive/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c rename to shark/src/test/hive/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c diff --git a/src/test/hive/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 b/shark/src/test/hive/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 rename to shark/src/test/hive/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 diff --git a/src/test/hive/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 b/shark/src/test/hive/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 rename to shark/src/test/hive/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 diff --git a/src/test/hive/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 b/shark/src/test/hive/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 rename to shark/src/test/hive/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 diff --git a/src/test/hive/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 b/shark/src/test/hive/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 rename to shark/src/test/hive/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 diff --git a/src/test/hive/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 b/shark/src/test/hive/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 rename to shark/src/test/hive/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 diff --git a/src/test/hive/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 b/shark/src/test/hive/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 similarity index 100% rename from src/test/hive/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 rename to shark/src/test/hive/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 diff --git a/src/test/hive/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a b/shark/src/test/hive/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a similarity index 100% rename from src/test/hive/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a rename to shark/src/test/hive/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a diff --git a/src/test/hive/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 b/shark/src/test/hive/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 similarity index 100% rename from src/test/hive/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 rename to shark/src/test/hive/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 diff --git a/src/test/hive/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 b/shark/src/test/hive/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 similarity index 100% rename from src/test/hive/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 rename to shark/src/test/hive/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 diff --git a/src/test/hive/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d b/shark/src/test/hive/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d similarity index 100% rename from src/test/hive/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d rename to shark/src/test/hive/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d diff --git a/src/test/hive/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb b/shark/src/test/hive/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb similarity index 100% rename from src/test/hive/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb rename to shark/src/test/hive/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb diff --git a/src/test/hive/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 b/shark/src/test/hive/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 similarity index 100% rename from src/test/hive/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 rename to shark/src/test/hive/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 diff --git a/src/test/hive/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f b/shark/src/test/hive/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f similarity index 100% rename from src/test/hive/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f rename to shark/src/test/hive/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f diff --git a/src/test/hive/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 b/shark/src/test/hive/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 similarity index 100% rename from src/test/hive/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 rename to shark/src/test/hive/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 diff --git a/src/test/hive/golden/print_header-1-8540676fc16ac91f3629c40f393a890a b/shark/src/test/hive/golden/print_header-1-8540676fc16ac91f3629c40f393a890a similarity index 100% rename from src/test/hive/golden/print_header-1-8540676fc16ac91f3629c40f393a890a rename to shark/src/test/hive/golden/print_header-1-8540676fc16ac91f3629c40f393a890a diff --git a/src/test/hive/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 b/shark/src/test/hive/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 similarity index 100% rename from src/test/hive/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 rename to shark/src/test/hive/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 diff --git a/src/test/hive/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb b/shark/src/test/hive/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb similarity index 100% rename from src/test/hive/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb rename to shark/src/test/hive/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb diff --git a/src/test/hive/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 b/shark/src/test/hive/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 similarity index 100% rename from src/test/hive/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 rename to shark/src/test/hive/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 diff --git a/src/test/hive/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 b/shark/src/test/hive/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 similarity index 100% rename from src/test/hive/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 rename to shark/src/test/hive/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 diff --git a/src/test/hive/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f b/shark/src/test/hive/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f similarity index 100% rename from src/test/hive/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f rename to shark/src/test/hive/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f diff --git a/src/test/hive/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 b/shark/src/test/hive/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 similarity index 100% rename from src/test/hive/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 rename to shark/src/test/hive/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 diff --git a/src/test/hive/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 b/shark/src/test/hive/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 similarity index 100% rename from src/test/hive/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 rename to shark/src/test/hive/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 diff --git a/src/test/hive/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 b/shark/src/test/hive/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 similarity index 100% rename from src/test/hive/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 rename to shark/src/test/hive/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 diff --git a/src/test/hive/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 b/shark/src/test/hive/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 similarity index 100% rename from src/test/hive/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 rename to shark/src/test/hive/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 diff --git a/src/test/hive/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 b/shark/src/test/hive/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 similarity index 100% rename from src/test/hive/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 rename to shark/src/test/hive/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 diff --git a/src/test/hive/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a b/shark/src/test/hive/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a similarity index 100% rename from src/test/hive/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a rename to shark/src/test/hive/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a diff --git a/src/test/hive/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c b/shark/src/test/hive/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c similarity index 100% rename from src/test/hive/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c rename to shark/src/test/hive/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c diff --git a/src/test/hive/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b b/shark/src/test/hive/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b similarity index 100% rename from src/test/hive/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b rename to shark/src/test/hive/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b diff --git a/src/test/hive/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 b/shark/src/test/hive/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 similarity index 100% rename from src/test/hive/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 rename to shark/src/test/hive/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 diff --git a/src/test/hive/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b b/shark/src/test/hive/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b similarity index 100% rename from src/test/hive/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b rename to shark/src/test/hive/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b diff --git a/src/test/hive/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 b/shark/src/test/hive/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 similarity index 100% rename from src/test/hive/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 rename to shark/src/test/hive/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 diff --git a/src/test/hive/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 b/shark/src/test/hive/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 similarity index 100% rename from src/test/hive/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 rename to shark/src/test/hive/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 diff --git a/src/test/hive/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 b/shark/src/test/hive/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 similarity index 100% rename from src/test/hive/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 rename to shark/src/test/hive/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 diff --git a/src/test/hive/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea b/shark/src/test/hive/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea similarity index 100% rename from src/test/hive/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea rename to shark/src/test/hive/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea diff --git a/src/test/hive/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b b/shark/src/test/hive/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b similarity index 100% rename from src/test/hive/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b rename to shark/src/test/hive/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b diff --git a/src/test/hive/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d b/shark/src/test/hive/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d similarity index 100% rename from src/test/hive/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d rename to shark/src/test/hive/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d diff --git a/src/test/hive/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 b/shark/src/test/hive/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 similarity index 100% rename from src/test/hive/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 rename to shark/src/test/hive/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 diff --git a/src/test/hive/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d b/shark/src/test/hive/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d similarity index 100% rename from src/test/hive/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d rename to shark/src/test/hive/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d diff --git a/src/test/hive/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 b/shark/src/test/hive/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 similarity index 100% rename from src/test/hive/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 rename to shark/src/test/hive/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 diff --git a/src/test/hive/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 b/shark/src/test/hive/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 similarity index 100% rename from src/test/hive/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 rename to shark/src/test/hive/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 diff --git a/src/test/hive/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d b/shark/src/test/hive/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d similarity index 100% rename from src/test/hive/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d rename to shark/src/test/hive/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d diff --git a/src/test/hive/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 b/shark/src/test/hive/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 similarity index 100% rename from src/test/hive/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 rename to shark/src/test/hive/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 diff --git a/src/test/hive/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d b/shark/src/test/hive/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d similarity index 100% rename from src/test/hive/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d rename to shark/src/test/hive/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d diff --git a/src/test/hive/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 b/shark/src/test/hive/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 similarity index 100% rename from src/test/hive/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 rename to shark/src/test/hive/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 diff --git a/src/test/hive/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc b/shark/src/test/hive/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc similarity index 100% rename from src/test/hive/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc rename to shark/src/test/hive/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc diff --git a/src/test/hive/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 b/shark/src/test/hive/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 similarity index 100% rename from src/test/hive/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 rename to shark/src/test/hive/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 diff --git a/src/test/hive/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 b/shark/src/test/hive/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 similarity index 100% rename from src/test/hive/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 rename to shark/src/test/hive/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 diff --git a/src/test/hive/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 b/shark/src/test/hive/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 similarity index 100% rename from src/test/hive/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 rename to shark/src/test/hive/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 diff --git a/src/test/hive/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b b/shark/src/test/hive/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b similarity index 100% rename from src/test/hive/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b rename to shark/src/test/hive/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b diff --git a/src/test/hive/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 b/shark/src/test/hive/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 similarity index 100% rename from src/test/hive/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 rename to shark/src/test/hive/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 diff --git a/src/test/hive/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 b/shark/src/test/hive/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 similarity index 100% rename from src/test/hive/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 rename to shark/src/test/hive/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 diff --git a/src/test/hive/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 b/shark/src/test/hive/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 similarity index 100% rename from src/test/hive/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 rename to shark/src/test/hive/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 diff --git a/src/test/hive/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e b/shark/src/test/hive/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e similarity index 100% rename from src/test/hive/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e rename to shark/src/test/hive/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e diff --git a/src/test/hive/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 b/shark/src/test/hive/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 similarity index 100% rename from src/test/hive/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 rename to shark/src/test/hive/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 diff --git a/src/test/hive/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 b/shark/src/test/hive/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 similarity index 100% rename from src/test/hive/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 rename to shark/src/test/hive/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 diff --git a/src/test/hive/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 b/shark/src/test/hive/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 similarity index 100% rename from src/test/hive/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 rename to shark/src/test/hive/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 diff --git a/src/test/hive/golden/protectmode-40-39024332d020339bd9396c9b15403a0c b/shark/src/test/hive/golden/protectmode-40-39024332d020339bd9396c9b15403a0c similarity index 100% rename from src/test/hive/golden/protectmode-40-39024332d020339bd9396c9b15403a0c rename to shark/src/test/hive/golden/protectmode-40-39024332d020339bd9396c9b15403a0c diff --git a/src/test/hive/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 b/shark/src/test/hive/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 similarity index 100% rename from src/test/hive/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 rename to shark/src/test/hive/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 diff --git a/src/test/hive/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb b/shark/src/test/hive/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb similarity index 100% rename from src/test/hive/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb rename to shark/src/test/hive/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb diff --git a/src/test/hive/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 b/shark/src/test/hive/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 similarity index 100% rename from src/test/hive/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 rename to shark/src/test/hive/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 diff --git a/src/test/hive/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c b/shark/src/test/hive/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c similarity index 100% rename from src/test/hive/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c rename to shark/src/test/hive/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c diff --git a/src/test/hive/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 b/shark/src/test/hive/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 similarity index 100% rename from src/test/hive/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 rename to shark/src/test/hive/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 diff --git a/src/test/hive/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 b/shark/src/test/hive/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 similarity index 100% rename from src/test/hive/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 rename to shark/src/test/hive/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 diff --git a/src/test/hive/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 b/shark/src/test/hive/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 similarity index 100% rename from src/test/hive/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 rename to shark/src/test/hive/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 diff --git a/src/test/hive/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc b/shark/src/test/hive/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc similarity index 100% rename from src/test/hive/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc rename to shark/src/test/hive/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc diff --git a/src/test/hive/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 b/shark/src/test/hive/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 similarity index 100% rename from src/test/hive/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 rename to shark/src/test/hive/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 diff --git a/src/test/hive/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e b/shark/src/test/hive/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e similarity index 100% rename from src/test/hive/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e rename to shark/src/test/hive/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e diff --git a/src/test/hive/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 b/shark/src/test/hive/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 similarity index 100% rename from src/test/hive/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 rename to shark/src/test/hive/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 diff --git a/src/test/hive/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 b/shark/src/test/hive/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 similarity index 100% rename from src/test/hive/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 rename to shark/src/test/hive/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 diff --git a/src/test/hive/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 b/shark/src/test/hive/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 similarity index 100% rename from src/test/hive/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 rename to shark/src/test/hive/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 diff --git a/src/test/hive/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a b/shark/src/test/hive/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a similarity index 100% rename from src/test/hive/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a rename to shark/src/test/hive/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a diff --git a/src/test/hive/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 b/shark/src/test/hive/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 similarity index 100% rename from src/test/hive/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 rename to shark/src/test/hive/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 diff --git a/src/test/hive/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb b/shark/src/test/hive/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb similarity index 100% rename from src/test/hive/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb rename to shark/src/test/hive/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb diff --git a/src/test/hive/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 b/shark/src/test/hive/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 similarity index 100% rename from src/test/hive/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 rename to shark/src/test/hive/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 diff --git a/src/test/hive/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 b/shark/src/test/hive/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 similarity index 100% rename from src/test/hive/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 rename to shark/src/test/hive/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 diff --git a/src/test/hive/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 b/shark/src/test/hive/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 similarity index 100% rename from src/test/hive/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 rename to shark/src/test/hive/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 diff --git a/src/test/hive/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 b/shark/src/test/hive/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 similarity index 100% rename from src/test/hive/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 rename to shark/src/test/hive/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 diff --git a/src/test/hive/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 b/shark/src/test/hive/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 similarity index 100% rename from src/test/hive/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 rename to shark/src/test/hive/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 diff --git a/src/test/hive/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 b/shark/src/test/hive/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 similarity index 100% rename from src/test/hive/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 rename to shark/src/test/hive/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 diff --git a/src/test/hive/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 b/shark/src/test/hive/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 similarity index 100% rename from src/test/hive/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 rename to shark/src/test/hive/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 diff --git a/src/test/hive/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 b/shark/src/test/hive/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 similarity index 100% rename from src/test/hive/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 rename to shark/src/test/hive/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 diff --git a/src/test/hive/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e b/shark/src/test/hive/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e similarity index 100% rename from src/test/hive/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e rename to shark/src/test/hive/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e diff --git a/src/test/hive/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d b/shark/src/test/hive/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d similarity index 100% rename from src/test/hive/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d rename to shark/src/test/hive/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d diff --git a/src/test/hive/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 b/shark/src/test/hive/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 similarity index 100% rename from src/test/hive/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 rename to shark/src/test/hive/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 diff --git a/src/test/hive/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 b/shark/src/test/hive/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 similarity index 100% rename from src/test/hive/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 rename to shark/src/test/hive/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 diff --git a/src/test/hive/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 b/shark/src/test/hive/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 similarity index 100% rename from src/test/hive/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 rename to shark/src/test/hive/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 diff --git a/src/test/hive/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 b/shark/src/test/hive/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 similarity index 100% rename from src/test/hive/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 rename to shark/src/test/hive/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 diff --git a/src/test/hive/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 b/shark/src/test/hive/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 similarity index 100% rename from src/test/hive/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 rename to shark/src/test/hive/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 diff --git a/src/test/hive/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/shark/src/test/hive/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e similarity index 100% rename from src/test/hive/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e rename to shark/src/test/hive/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e diff --git a/src/test/hive/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 b/shark/src/test/hive/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 similarity index 100% rename from src/test/hive/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 rename to shark/src/test/hive/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 diff --git a/src/test/hive/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/shark/src/test/hive/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 similarity index 100% rename from src/test/hive/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 rename to shark/src/test/hive/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 diff --git a/src/test/hive/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a b/shark/src/test/hive/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a similarity index 100% rename from src/test/hive/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a rename to shark/src/test/hive/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a diff --git a/src/test/hive/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 b/shark/src/test/hive/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 similarity index 100% rename from src/test/hive/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 rename to shark/src/test/hive/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 diff --git a/src/test/hive/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 b/shark/src/test/hive/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 similarity index 100% rename from src/test/hive/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 rename to shark/src/test/hive/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 diff --git a/src/test/hive/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b b/shark/src/test/hive/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b similarity index 100% rename from src/test/hive/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b rename to shark/src/test/hive/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b diff --git a/src/test/hive/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff b/shark/src/test/hive/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff similarity index 100% rename from src/test/hive/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff rename to shark/src/test/hive/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff diff --git a/src/test/hive/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 b/shark/src/test/hive/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 rename to shark/src/test/hive/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 diff --git a/src/test/hive/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 b/shark/src/test/hive/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 rename to shark/src/test/hive/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 diff --git a/src/test/hive/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b b/shark/src/test/hive/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b rename to shark/src/test/hive/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b diff --git a/src/test/hive/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c b/shark/src/test/hive/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c rename to shark/src/test/hive/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c diff --git a/src/test/hive/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 b/shark/src/test/hive/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 rename to shark/src/test/hive/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 diff --git a/src/test/hive/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 b/shark/src/test/hive/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 rename to shark/src/test/hive/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 diff --git a/src/test/hive/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 b/shark/src/test/hive/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 rename to shark/src/test/hive/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 diff --git a/src/test/hive/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b b/shark/src/test/hive/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b rename to shark/src/test/hive/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b diff --git a/src/test/hive/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 b/shark/src/test/hive/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 rename to shark/src/test/hive/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 diff --git a/src/test/hive/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d b/shark/src/test/hive/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d rename to shark/src/test/hive/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d diff --git a/src/test/hive/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 b/shark/src/test/hive/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 rename to shark/src/test/hive/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 diff --git a/src/test/hive/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 b/shark/src/test/hive/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 rename to shark/src/test/hive/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 diff --git a/src/test/hive/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 b/shark/src/test/hive/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 similarity index 100% rename from src/test/hive/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 rename to shark/src/test/hive/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 diff --git a/src/test/hive/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 b/shark/src/test/hive/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 rename to shark/src/test/hive/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 diff --git a/src/test/hive/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 b/shark/src/test/hive/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 rename to shark/src/test/hive/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 diff --git a/src/test/hive/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 b/shark/src/test/hive/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 rename to shark/src/test/hive/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 diff --git a/src/test/hive/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c b/shark/src/test/hive/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c similarity index 100% rename from src/test/hive/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c rename to shark/src/test/hive/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c diff --git a/src/test/hive/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 b/shark/src/test/hive/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 rename to shark/src/test/hive/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 diff --git a/src/test/hive/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 b/shark/src/test/hive/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 rename to shark/src/test/hive/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 diff --git a/src/test/hive/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 b/shark/src/test/hive/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 rename to shark/src/test/hive/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 diff --git a/src/test/hive/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e b/shark/src/test/hive/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e similarity index 100% rename from src/test/hive/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e rename to shark/src/test/hive/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e diff --git a/src/test/hive/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 b/shark/src/test/hive/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 rename to shark/src/test/hive/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 diff --git a/src/test/hive/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee b/shark/src/test/hive/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee similarity index 100% rename from src/test/hive/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee rename to shark/src/test/hive/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee diff --git a/src/test/hive/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 b/shark/src/test/hive/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 rename to shark/src/test/hive/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 diff --git a/src/test/hive/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e b/shark/src/test/hive/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e similarity index 100% rename from src/test/hive/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e rename to shark/src/test/hive/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e diff --git a/src/test/hive/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 b/shark/src/test/hive/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 rename to shark/src/test/hive/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 diff --git a/src/test/hive/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 b/shark/src/test/hive/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 rename to shark/src/test/hive/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 diff --git a/src/test/hive/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f b/shark/src/test/hive/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f similarity index 100% rename from src/test/hive/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f rename to shark/src/test/hive/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f diff --git a/src/test/hive/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 b/shark/src/test/hive/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 similarity index 100% rename from src/test/hive/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 rename to shark/src/test/hive/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 diff --git a/src/test/hive/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 b/shark/src/test/hive/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 similarity index 100% rename from src/test/hive/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 rename to shark/src/test/hive/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 diff --git a/src/test/hive/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 b/shark/src/test/hive/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 similarity index 100% rename from src/test/hive/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 rename to shark/src/test/hive/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 diff --git a/src/test/hive/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 b/shark/src/test/hive/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 similarity index 100% rename from src/test/hive/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 rename to shark/src/test/hive/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 diff --git a/src/test/hive/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb b/shark/src/test/hive/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb similarity index 100% rename from src/test/hive/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb rename to shark/src/test/hive/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb diff --git a/src/test/hive/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 b/shark/src/test/hive/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 similarity index 100% rename from src/test/hive/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 rename to shark/src/test/hive/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 diff --git a/src/test/hive/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 b/shark/src/test/hive/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 similarity index 100% rename from src/test/hive/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 rename to shark/src/test/hive/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 diff --git a/src/test/hive/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a b/shark/src/test/hive/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a similarity index 100% rename from src/test/hive/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a rename to shark/src/test/hive/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a diff --git a/src/test/hive/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 b/shark/src/test/hive/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 similarity index 100% rename from src/test/hive/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 rename to shark/src/test/hive/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 diff --git a/src/test/hive/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 b/shark/src/test/hive/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 similarity index 100% rename from src/test/hive/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 rename to shark/src/test/hive/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 diff --git a/src/test/hive/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 b/shark/src/test/hive/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 similarity index 100% rename from src/test/hive/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 rename to shark/src/test/hive/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 diff --git a/src/test/hive/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 b/shark/src/test/hive/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 similarity index 100% rename from src/test/hive/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 rename to shark/src/test/hive/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 diff --git a/src/test/hive/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 b/shark/src/test/hive/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 similarity index 100% rename from src/test/hive/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 rename to shark/src/test/hive/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 diff --git a/src/test/hive/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 b/shark/src/test/hive/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 similarity index 100% rename from src/test/hive/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 rename to shark/src/test/hive/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 diff --git a/src/test/hive/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 b/shark/src/test/hive/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 similarity index 100% rename from src/test/hive/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 rename to shark/src/test/hive/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 diff --git a/src/test/hive/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f b/shark/src/test/hive/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f similarity index 100% rename from src/test/hive/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f rename to shark/src/test/hive/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f diff --git a/src/test/hive/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 b/shark/src/test/hive/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 similarity index 100% rename from src/test/hive/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 rename to shark/src/test/hive/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 diff --git a/src/test/hive/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b b/shark/src/test/hive/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b similarity index 100% rename from src/test/hive/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b rename to shark/src/test/hive/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b diff --git a/src/test/hive/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb b/shark/src/test/hive/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb similarity index 100% rename from src/test/hive/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb rename to shark/src/test/hive/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb diff --git a/src/test/hive/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 b/shark/src/test/hive/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 similarity index 100% rename from src/test/hive/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 rename to shark/src/test/hive/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 diff --git a/src/test/hive/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b b/shark/src/test/hive/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b similarity index 100% rename from src/test/hive/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b rename to shark/src/test/hive/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b diff --git a/src/test/hive/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 b/shark/src/test/hive/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 similarity index 100% rename from src/test/hive/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 rename to shark/src/test/hive/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 diff --git a/src/test/hive/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b b/shark/src/test/hive/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b similarity index 100% rename from src/test/hive/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b rename to shark/src/test/hive/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b diff --git a/src/test/hive/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/shark/src/test/hive/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 similarity index 100% rename from src/test/hive/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 rename to shark/src/test/hive/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 diff --git a/src/test/hive/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 b/shark/src/test/hive/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 similarity index 100% rename from src/test/hive/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 rename to shark/src/test/hive/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 diff --git a/src/test/hive/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 b/shark/src/test/hive/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 similarity index 100% rename from src/test/hive/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 rename to shark/src/test/hive/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 diff --git a/src/test/hive/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 b/shark/src/test/hive/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 rename to shark/src/test/hive/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 diff --git a/src/test/hive/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 b/shark/src/test/hive/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 rename to shark/src/test/hive/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 diff --git a/src/test/hive/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 b/shark/src/test/hive/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 rename to shark/src/test/hive/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 diff --git a/src/test/hive/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 b/shark/src/test/hive/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 rename to shark/src/test/hive/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 diff --git a/src/test/hive/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f b/shark/src/test/hive/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f similarity index 100% rename from src/test/hive/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f rename to shark/src/test/hive/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f diff --git a/src/test/hive/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 b/shark/src/test/hive/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 rename to shark/src/test/hive/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 diff --git a/src/test/hive/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b b/shark/src/test/hive/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b similarity index 100% rename from src/test/hive/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b rename to shark/src/test/hive/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b diff --git a/src/test/hive/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb b/shark/src/test/hive/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb similarity index 100% rename from src/test/hive/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb rename to shark/src/test/hive/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb diff --git a/src/test/hive/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 b/shark/src/test/hive/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 rename to shark/src/test/hive/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 diff --git a/src/test/hive/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b b/shark/src/test/hive/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b similarity index 100% rename from src/test/hive/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b rename to shark/src/test/hive/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b diff --git a/src/test/hive/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 b/shark/src/test/hive/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 rename to shark/src/test/hive/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 diff --git a/src/test/hive/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 b/shark/src/test/hive/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 rename to shark/src/test/hive/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 diff --git a/src/test/hive/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/shark/src/test/hive/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 rename to shark/src/test/hive/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 diff --git a/src/test/hive/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 b/shark/src/test/hive/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 rename to shark/src/test/hive/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 diff --git a/src/test/hive/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 b/shark/src/test/hive/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 similarity index 100% rename from src/test/hive/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 rename to shark/src/test/hive/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 diff --git a/src/test/hive/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 b/shark/src/test/hive/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 similarity index 100% rename from src/test/hive/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 rename to shark/src/test/hive/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 diff --git a/src/test/hive/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d b/shark/src/test/hive/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d similarity index 100% rename from src/test/hive/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d rename to shark/src/test/hive/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d diff --git a/src/test/hive/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee b/shark/src/test/hive/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee similarity index 100% rename from src/test/hive/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee rename to shark/src/test/hive/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee diff --git a/src/test/hive/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a b/shark/src/test/hive/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a similarity index 100% rename from src/test/hive/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a rename to shark/src/test/hive/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a diff --git a/src/test/hive/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 b/shark/src/test/hive/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 similarity index 100% rename from src/test/hive/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 rename to shark/src/test/hive/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 diff --git a/src/test/hive/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed b/shark/src/test/hive/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed similarity index 100% rename from src/test/hive/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed rename to shark/src/test/hive/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed diff --git a/src/test/hive/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 b/shark/src/test/hive/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 similarity index 100% rename from src/test/hive/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 rename to shark/src/test/hive/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 b/shark/src/test/hive/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 similarity index 100% rename from src/test/hive/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 rename to shark/src/test/hive/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b b/shark/src/test/hive/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b similarity index 100% rename from src/test/hive/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b rename to shark/src/test/hive/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 b/shark/src/test/hive/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 similarity index 100% rename from src/test/hive/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 rename to shark/src/test/hive/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d b/shark/src/test/hive/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d similarity index 100% rename from src/test/hive/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d rename to shark/src/test/hive/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 b/shark/src/test/hive/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 similarity index 100% rename from src/test/hive/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 rename to shark/src/test/hive/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a b/shark/src/test/hive/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a similarity index 100% rename from src/test/hive/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a rename to shark/src/test/hive/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a diff --git a/src/test/hive/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 b/shark/src/test/hive/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 similarity index 100% rename from src/test/hive/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 rename to shark/src/test/hive/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 diff --git a/src/test/hive/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 b/shark/src/test/hive/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 similarity index 100% rename from src/test/hive/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 rename to shark/src/test/hive/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 diff --git a/src/test/hive/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b b/shark/src/test/hive/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b similarity index 100% rename from src/test/hive/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b rename to shark/src/test/hive/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b diff --git a/src/test/hive/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 b/shark/src/test/hive/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 similarity index 100% rename from src/test/hive/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 rename to shark/src/test/hive/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 diff --git a/src/test/hive/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 b/shark/src/test/hive/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 similarity index 100% rename from src/test/hive/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 rename to shark/src/test/hive/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 diff --git a/src/test/hive/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 b/shark/src/test/hive/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 rename to shark/src/test/hive/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 diff --git a/src/test/hive/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 b/shark/src/test/hive/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 rename to shark/src/test/hive/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 diff --git a/src/test/hive/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 b/shark/src/test/hive/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 rename to shark/src/test/hive/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 diff --git a/src/test/hive/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 b/shark/src/test/hive/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 rename to shark/src/test/hive/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 diff --git a/src/test/hive/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe b/shark/src/test/hive/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe rename to shark/src/test/hive/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe diff --git a/src/test/hive/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 b/shark/src/test/hive/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 rename to shark/src/test/hive/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 diff --git a/src/test/hive/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d b/shark/src/test/hive/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d rename to shark/src/test/hive/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d diff --git a/src/test/hive/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 b/shark/src/test/hive/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 rename to shark/src/test/hive/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 diff --git a/src/test/hive/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 b/shark/src/test/hive/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 rename to shark/src/test/hive/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 diff --git a/src/test/hive/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 b/shark/src/test/hive/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 rename to shark/src/test/hive/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 diff --git a/src/test/hive/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 b/shark/src/test/hive/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 rename to shark/src/test/hive/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 b/shark/src/test/hive/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 rename to shark/src/test/hive/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 b/shark/src/test/hive/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 rename to shark/src/test/hive/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 b/shark/src/test/hive/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 rename to shark/src/test/hive/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 b/shark/src/test/hive/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 rename to shark/src/test/hive/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 b/shark/src/test/hive/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 rename to shark/src/test/hive/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 b/shark/src/test/hive/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 rename to shark/src/test/hive/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 b/shark/src/test/hive/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 b/shark/src/test/hive/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa b/shark/src/test/hive/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa rename to shark/src/test/hive/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa diff --git a/src/test/hive/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e b/shark/src/test/hive/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e rename to shark/src/test/hive/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e diff --git a/src/test/hive/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 b/shark/src/test/hive/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e b/shark/src/test/hive/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e rename to shark/src/test/hive/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e diff --git a/src/test/hive/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c b/shark/src/test/hive/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c rename to shark/src/test/hive/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c diff --git a/src/test/hive/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd b/shark/src/test/hive/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd rename to shark/src/test/hive/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd diff --git a/src/test/hive/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb b/shark/src/test/hive/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb rename to shark/src/test/hive/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb diff --git a/src/test/hive/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f b/shark/src/test/hive/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f rename to shark/src/test/hive/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f diff --git a/src/test/hive/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 b/shark/src/test/hive/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 b/shark/src/test/hive/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 b/shark/src/test/hive/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 b/shark/src/test/hive/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d b/shark/src/test/hive/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d rename to shark/src/test/hive/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d diff --git a/src/test/hive/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed b/shark/src/test/hive/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed rename to shark/src/test/hive/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed diff --git a/src/test/hive/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 b/shark/src/test/hive/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a b/shark/src/test/hive/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a rename to shark/src/test/hive/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a diff --git a/src/test/hive/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa b/shark/src/test/hive/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa rename to shark/src/test/hive/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa diff --git a/src/test/hive/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e b/shark/src/test/hive/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e rename to shark/src/test/hive/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e diff --git a/src/test/hive/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 b/shark/src/test/hive/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 b/shark/src/test/hive/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e b/shark/src/test/hive/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e rename to shark/src/test/hive/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e diff --git a/src/test/hive/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c b/shark/src/test/hive/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c rename to shark/src/test/hive/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c diff --git a/src/test/hive/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd b/shark/src/test/hive/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd rename to shark/src/test/hive/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd diff --git a/src/test/hive/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb b/shark/src/test/hive/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb rename to shark/src/test/hive/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb diff --git a/src/test/hive/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f b/shark/src/test/hive/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f rename to shark/src/test/hive/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f diff --git a/src/test/hive/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 b/shark/src/test/hive/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 b/shark/src/test/hive/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 b/shark/src/test/hive/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad b/shark/src/test/hive/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad rename to shark/src/test/hive/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad diff --git a/src/test/hive/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 b/shark/src/test/hive/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 b/shark/src/test/hive/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 diff --git a/src/test/hive/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 b/shark/src/test/hive/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 similarity index 100% rename from src/test/hive/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 rename to shark/src/test/hive/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 diff --git a/src/test/hive/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb b/shark/src/test/hive/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb similarity index 100% rename from src/test/hive/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb rename to shark/src/test/hive/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb diff --git a/src/test/hive/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 b/shark/src/test/hive/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 similarity index 100% rename from src/test/hive/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 rename to shark/src/test/hive/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 diff --git a/src/test/hive/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 b/shark/src/test/hive/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 similarity index 100% rename from src/test/hive/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 rename to shark/src/test/hive/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 diff --git a/src/test/hive/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 b/shark/src/test/hive/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 similarity index 100% rename from src/test/hive/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 rename to shark/src/test/hive/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 diff --git a/src/test/hive/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db b/shark/src/test/hive/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db similarity index 100% rename from src/test/hive/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db rename to shark/src/test/hive/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db diff --git a/src/test/hive/golden/regex_col-2-21564f64cdfd46098e1254380490701 b/shark/src/test/hive/golden/regex_col-2-21564f64cdfd46098e1254380490701 similarity index 100% rename from src/test/hive/golden/regex_col-2-21564f64cdfd46098e1254380490701 rename to shark/src/test/hive/golden/regex_col-2-21564f64cdfd46098e1254380490701 diff --git a/src/test/hive/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 b/shark/src/test/hive/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 similarity index 100% rename from src/test/hive/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 rename to shark/src/test/hive/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 diff --git a/src/test/hive/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 b/shark/src/test/hive/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 similarity index 100% rename from src/test/hive/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 rename to shark/src/test/hive/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 diff --git a/src/test/hive/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 b/shark/src/test/hive/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 similarity index 100% rename from src/test/hive/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 rename to shark/src/test/hive/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 diff --git a/src/test/hive/golden/remote_script-1-e168f471980470d93b790702a70238fa b/shark/src/test/hive/golden/remote_script-1-e168f471980470d93b790702a70238fa similarity index 100% rename from src/test/hive/golden/remote_script-1-e168f471980470d93b790702a70238fa rename to shark/src/test/hive/golden/remote_script-1-e168f471980470d93b790702a70238fa diff --git a/src/test/hive/golden/remote_script-2-a19a19272149c732977c37e043910505 b/shark/src/test/hive/golden/remote_script-2-a19a19272149c732977c37e043910505 similarity index 100% rename from src/test/hive/golden/remote_script-2-a19a19272149c732977c37e043910505 rename to shark/src/test/hive/golden/remote_script-2-a19a19272149c732977c37e043910505 diff --git a/src/test/hive/golden/remote_script-3-4eb54a664e549614d56ca088c8867d b/shark/src/test/hive/golden/remote_script-3-4eb54a664e549614d56ca088c8867d similarity index 100% rename from src/test/hive/golden/remote_script-3-4eb54a664e549614d56ca088c8867d rename to shark/src/test/hive/golden/remote_script-3-4eb54a664e549614d56ca088c8867d diff --git a/src/test/hive/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 b/shark/src/test/hive/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 similarity index 100% rename from src/test/hive/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 rename to shark/src/test/hive/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 diff --git a/src/test/hive/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 b/shark/src/test/hive/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 similarity index 100% rename from src/test/hive/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 rename to shark/src/test/hive/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 diff --git a/src/test/hive/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 b/shark/src/test/hive/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 similarity index 100% rename from src/test/hive/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 rename to shark/src/test/hive/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 diff --git a/src/test/hive/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 b/shark/src/test/hive/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 similarity index 100% rename from src/test/hive/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 rename to shark/src/test/hive/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 diff --git a/src/test/hive/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-16-d032f4795c1186255acea241387adf93 b/shark/src/test/hive/golden/rename_column-16-d032f4795c1186255acea241387adf93 similarity index 100% rename from src/test/hive/golden/rename_column-16-d032f4795c1186255acea241387adf93 rename to shark/src/test/hive/golden/rename_column-16-d032f4795c1186255acea241387adf93 diff --git a/src/test/hive/golden/rename_column-17-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/rename_column-17-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/rename_column-17-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/rename_column-17-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 b/shark/src/test/hive/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 similarity index 100% rename from src/test/hive/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 rename to shark/src/test/hive/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 diff --git a/src/test/hive/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f b/shark/src/test/hive/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f similarity index 100% rename from src/test/hive/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f rename to shark/src/test/hive/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f diff --git a/src/test/hive/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a b/shark/src/test/hive/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a similarity index 100% rename from src/test/hive/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a rename to shark/src/test/hive/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a diff --git a/src/test/hive/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 b/shark/src/test/hive/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 similarity index 100% rename from src/test/hive/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 rename to shark/src/test/hive/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 diff --git a/src/test/hive/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a b/shark/src/test/hive/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a similarity index 100% rename from src/test/hive/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a rename to shark/src/test/hive/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a diff --git a/src/test/hive/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 b/shark/src/test/hive/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 similarity index 100% rename from src/test/hive/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 rename to shark/src/test/hive/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 diff --git a/src/test/hive/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd b/shark/src/test/hive/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd similarity index 100% rename from src/test/hive/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd rename to shark/src/test/hive/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd diff --git a/src/test/hive/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 b/shark/src/test/hive/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 similarity index 100% rename from src/test/hive/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 rename to shark/src/test/hive/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 diff --git a/src/test/hive/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 b/shark/src/test/hive/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 similarity index 100% rename from src/test/hive/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 rename to shark/src/test/hive/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 diff --git a/src/test/hive/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 b/shark/src/test/hive/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 similarity index 100% rename from src/test/hive/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 rename to shark/src/test/hive/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 diff --git a/src/test/hive/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 b/shark/src/test/hive/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 similarity index 100% rename from src/test/hive/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 rename to shark/src/test/hive/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 diff --git a/src/test/hive/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-36-d032f4795c1186255acea241387adf93 b/shark/src/test/hive/golden/rename_column-36-d032f4795c1186255acea241387adf93 similarity index 100% rename from src/test/hive/golden/rename_column-36-d032f4795c1186255acea241387adf93 rename to shark/src/test/hive/golden/rename_column-36-d032f4795c1186255acea241387adf93 diff --git a/src/test/hive/golden/rename_column-37-9c36cac1372650b703400c60dd29042c b/shark/src/test/hive/golden/rename_column-37-9c36cac1372650b703400c60dd29042c similarity index 100% rename from src/test/hive/golden/rename_column-37-9c36cac1372650b703400c60dd29042c rename to shark/src/test/hive/golden/rename_column-37-9c36cac1372650b703400c60dd29042c diff --git a/src/test/hive/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 b/shark/src/test/hive/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 similarity index 100% rename from src/test/hive/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 rename to shark/src/test/hive/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 diff --git a/src/test/hive/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd b/shark/src/test/hive/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd similarity index 100% rename from src/test/hive/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd rename to shark/src/test/hive/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd diff --git a/src/test/hive/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 b/shark/src/test/hive/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 similarity index 100% rename from src/test/hive/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 rename to shark/src/test/hive/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 diff --git a/src/test/hive/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b b/shark/src/test/hive/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b similarity index 100% rename from src/test/hive/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b rename to shark/src/test/hive/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b diff --git a/src/test/hive/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 b/shark/src/test/hive/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 similarity index 100% rename from src/test/hive/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 rename to shark/src/test/hive/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 diff --git a/src/test/hive/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 b/shark/src/test/hive/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 similarity index 100% rename from src/test/hive/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 rename to shark/src/test/hive/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 diff --git a/src/test/hive/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 b/shark/src/test/hive/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 similarity index 100% rename from src/test/hive/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 rename to shark/src/test/hive/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 diff --git a/src/test/hive/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc b/shark/src/test/hive/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc similarity index 100% rename from src/test/hive/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc rename to shark/src/test/hive/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc diff --git a/src/test/hive/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 b/shark/src/test/hive/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 similarity index 100% rename from src/test/hive/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 rename to shark/src/test/hive/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 diff --git a/src/test/hive/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 b/shark/src/test/hive/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 similarity index 100% rename from src/test/hive/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 rename to shark/src/test/hive/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 diff --git a/src/test/hive/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f b/shark/src/test/hive/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f similarity index 100% rename from src/test/hive/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f rename to shark/src/test/hive/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f diff --git a/src/test/hive/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 b/shark/src/test/hive/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 similarity index 100% rename from src/test/hive/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 rename to shark/src/test/hive/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 diff --git a/src/test/hive/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e b/shark/src/test/hive/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e similarity index 100% rename from src/test/hive/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e rename to shark/src/test/hive/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e diff --git a/src/test/hive/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 b/shark/src/test/hive/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 similarity index 100% rename from src/test/hive/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 rename to shark/src/test/hive/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 diff --git a/src/test/hive/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 b/shark/src/test/hive/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 similarity index 100% rename from src/test/hive/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 rename to shark/src/test/hive/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 diff --git a/src/test/hive/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef b/shark/src/test/hive/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef similarity index 100% rename from src/test/hive/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef rename to shark/src/test/hive/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef diff --git a/src/test/hive/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 b/shark/src/test/hive/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 similarity index 100% rename from src/test/hive/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 rename to shark/src/test/hive/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 diff --git a/src/test/hive/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 b/shark/src/test/hive/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 similarity index 100% rename from src/test/hive/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 rename to shark/src/test/hive/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 diff --git a/src/test/hive/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 b/shark/src/test/hive/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 similarity index 100% rename from src/test/hive/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 rename to shark/src/test/hive/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 diff --git a/src/test/hive/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 b/shark/src/test/hive/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 similarity index 100% rename from src/test/hive/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 rename to shark/src/test/hive/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 diff --git a/src/test/hive/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 b/shark/src/test/hive/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 similarity index 100% rename from src/test/hive/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 rename to shark/src/test/hive/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 diff --git a/src/test/hive/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f b/shark/src/test/hive/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f similarity index 100% rename from src/test/hive/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f rename to shark/src/test/hive/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f diff --git a/src/test/hive/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 b/shark/src/test/hive/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 similarity index 100% rename from src/test/hive/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 rename to shark/src/test/hive/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 diff --git a/src/test/hive/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 b/shark/src/test/hive/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 similarity index 100% rename from src/test/hive/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 rename to shark/src/test/hive/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 diff --git a/src/test/hive/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a b/shark/src/test/hive/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a similarity index 100% rename from src/test/hive/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a rename to shark/src/test/hive/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a diff --git a/src/test/hive/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 b/shark/src/test/hive/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 similarity index 100% rename from src/test/hive/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 rename to shark/src/test/hive/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 diff --git a/src/test/hive/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 b/shark/src/test/hive/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 similarity index 100% rename from src/test/hive/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 rename to shark/src/test/hive/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 diff --git a/src/test/hive/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 b/shark/src/test/hive/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 similarity index 100% rename from src/test/hive/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 rename to shark/src/test/hive/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 diff --git a/src/test/hive/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 b/shark/src/test/hive/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 similarity index 100% rename from src/test/hive/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 rename to shark/src/test/hive/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 diff --git a/src/test/hive/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e b/shark/src/test/hive/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e similarity index 100% rename from src/test/hive/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e rename to shark/src/test/hive/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e diff --git a/src/test/hive/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a b/shark/src/test/hive/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a similarity index 100% rename from src/test/hive/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a rename to shark/src/test/hive/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a diff --git a/src/test/hive/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 b/shark/src/test/hive/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 similarity index 100% rename from src/test/hive/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 rename to shark/src/test/hive/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 diff --git a/src/test/hive/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 b/shark/src/test/hive/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 similarity index 100% rename from src/test/hive/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 rename to shark/src/test/hive/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 diff --git a/src/test/hive/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 b/shark/src/test/hive/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 similarity index 100% rename from src/test/hive/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 rename to shark/src/test/hive/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 diff --git a/src/test/hive/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 b/shark/src/test/hive/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 similarity index 100% rename from src/test/hive/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 rename to shark/src/test/hive/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 diff --git a/src/test/hive/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 b/shark/src/test/hive/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 similarity index 100% rename from src/test/hive/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 rename to shark/src/test/hive/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 diff --git a/src/test/hive/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e b/shark/src/test/hive/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e similarity index 100% rename from src/test/hive/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e rename to shark/src/test/hive/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e diff --git a/src/test/hive/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 b/shark/src/test/hive/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 similarity index 100% rename from src/test/hive/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 rename to shark/src/test/hive/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 diff --git a/src/test/hive/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde b/shark/src/test/hive/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde similarity index 100% rename from src/test/hive/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde rename to shark/src/test/hive/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde diff --git a/src/test/hive/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 b/shark/src/test/hive/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 similarity index 100% rename from src/test/hive/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 rename to shark/src/test/hive/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 diff --git a/src/test/hive/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 b/shark/src/test/hive/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 similarity index 100% rename from src/test/hive/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 rename to shark/src/test/hive/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 diff --git a/src/test/hive/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 b/shark/src/test/hive/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 similarity index 100% rename from src/test/hive/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 rename to shark/src/test/hive/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 diff --git a/src/test/hive/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 b/shark/src/test/hive/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 similarity index 100% rename from src/test/hive/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 rename to shark/src/test/hive/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 diff --git a/src/test/hive/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e b/shark/src/test/hive/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e similarity index 100% rename from src/test/hive/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e rename to shark/src/test/hive/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e diff --git a/src/test/hive/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 b/shark/src/test/hive/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 similarity index 100% rename from src/test/hive/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 rename to shark/src/test/hive/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 diff --git a/src/test/hive/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f b/shark/src/test/hive/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f similarity index 100% rename from src/test/hive/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f rename to shark/src/test/hive/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f diff --git a/src/test/hive/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 b/shark/src/test/hive/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 similarity index 100% rename from src/test/hive/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 rename to shark/src/test/hive/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 diff --git a/src/test/hive/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 b/shark/src/test/hive/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 similarity index 100% rename from src/test/hive/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 rename to shark/src/test/hive/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 diff --git a/src/test/hive/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 b/shark/src/test/hive/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 similarity index 100% rename from src/test/hive/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 rename to shark/src/test/hive/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 diff --git a/src/test/hive/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d b/shark/src/test/hive/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d similarity index 100% rename from src/test/hive/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d rename to shark/src/test/hive/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d diff --git a/src/test/hive/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa b/shark/src/test/hive/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa similarity index 100% rename from src/test/hive/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa rename to shark/src/test/hive/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa diff --git a/src/test/hive/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d b/shark/src/test/hive/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d similarity index 100% rename from src/test/hive/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d rename to shark/src/test/hive/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d diff --git a/src/test/hive/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 b/shark/src/test/hive/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 similarity index 100% rename from src/test/hive/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 rename to shark/src/test/hive/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 diff --git a/src/test/hive/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 b/shark/src/test/hive/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 similarity index 100% rename from src/test/hive/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 rename to shark/src/test/hive/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 diff --git a/src/test/hive/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 b/shark/src/test/hive/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 similarity index 100% rename from src/test/hive/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 rename to shark/src/test/hive/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 diff --git a/src/test/hive/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 b/shark/src/test/hive/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 similarity index 100% rename from src/test/hive/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 rename to shark/src/test/hive/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 diff --git a/src/test/hive/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 b/shark/src/test/hive/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 similarity index 100% rename from src/test/hive/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 rename to shark/src/test/hive/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 diff --git a/src/test/hive/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd b/shark/src/test/hive/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd similarity index 100% rename from src/test/hive/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd rename to shark/src/test/hive/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd diff --git a/src/test/hive/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a b/shark/src/test/hive/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a similarity index 100% rename from src/test/hive/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a rename to shark/src/test/hive/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a diff --git a/src/test/hive/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b b/shark/src/test/hive/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b similarity index 100% rename from src/test/hive/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b rename to shark/src/test/hive/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b diff --git a/src/test/hive/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d b/shark/src/test/hive/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d similarity index 100% rename from src/test/hive/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d rename to shark/src/test/hive/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d diff --git a/src/test/hive/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 b/shark/src/test/hive/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 similarity index 100% rename from src/test/hive/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 rename to shark/src/test/hive/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 diff --git a/src/test/hive/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 b/shark/src/test/hive/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 similarity index 100% rename from src/test/hive/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 rename to shark/src/test/hive/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 diff --git a/src/test/hive/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d b/shark/src/test/hive/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d similarity index 100% rename from src/test/hive/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d rename to shark/src/test/hive/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d diff --git a/src/test/hive/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 b/shark/src/test/hive/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 similarity index 100% rename from src/test/hive/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 rename to shark/src/test/hive/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 diff --git a/src/test/hive/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 b/shark/src/test/hive/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 similarity index 100% rename from src/test/hive/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 rename to shark/src/test/hive/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 diff --git a/src/test/hive/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 b/shark/src/test/hive/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 similarity index 100% rename from src/test/hive/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 rename to shark/src/test/hive/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 diff --git a/src/test/hive/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 b/shark/src/test/hive/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 similarity index 100% rename from src/test/hive/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 rename to shark/src/test/hive/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 diff --git a/src/test/hive/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c b/shark/src/test/hive/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c similarity index 100% rename from src/test/hive/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c rename to shark/src/test/hive/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c diff --git a/src/test/hive/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 b/shark/src/test/hive/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 similarity index 100% rename from src/test/hive/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 rename to shark/src/test/hive/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 diff --git a/src/test/hive/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 b/shark/src/test/hive/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 similarity index 100% rename from src/test/hive/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 rename to shark/src/test/hive/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 diff --git a/src/test/hive/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 b/shark/src/test/hive/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 similarity index 100% rename from src/test/hive/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 rename to shark/src/test/hive/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 diff --git a/src/test/hive/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 b/shark/src/test/hive/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 rename to shark/src/test/hive/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 diff --git a/src/test/hive/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 b/shark/src/test/hive/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 rename to shark/src/test/hive/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 diff --git a/src/test/hive/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 b/shark/src/test/hive/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 rename to shark/src/test/hive/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 diff --git a/src/test/hive/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e b/shark/src/test/hive/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e rename to shark/src/test/hive/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e diff --git a/src/test/hive/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b b/shark/src/test/hive/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b rename to shark/src/test/hive/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b diff --git a/src/test/hive/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 b/shark/src/test/hive/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 rename to shark/src/test/hive/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 diff --git a/src/test/hive/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e b/shark/src/test/hive/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e rename to shark/src/test/hive/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e diff --git a/src/test/hive/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d b/shark/src/test/hive/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d rename to shark/src/test/hive/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d diff --git a/src/test/hive/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f b/shark/src/test/hive/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f rename to shark/src/test/hive/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f diff --git a/src/test/hive/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 b/shark/src/test/hive/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 rename to shark/src/test/hive/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 diff --git a/src/test/hive/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 b/shark/src/test/hive/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 rename to shark/src/test/hive/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 diff --git a/src/test/hive/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 b/shark/src/test/hive/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 rename to shark/src/test/hive/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 diff --git a/src/test/hive/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 b/shark/src/test/hive/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 rename to shark/src/test/hive/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 diff --git a/src/test/hive/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 b/shark/src/test/hive/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 rename to shark/src/test/hive/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 diff --git a/src/test/hive/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc b/shark/src/test/hive/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc rename to shark/src/test/hive/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc diff --git a/src/test/hive/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 b/shark/src/test/hive/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 rename to shark/src/test/hive/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 diff --git a/src/test/hive/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc b/shark/src/test/hive/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc rename to shark/src/test/hive/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc diff --git a/src/test/hive/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 b/shark/src/test/hive/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 rename to shark/src/test/hive/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 diff --git a/src/test/hive/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c b/shark/src/test/hive/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c rename to shark/src/test/hive/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c diff --git a/src/test/hive/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 b/shark/src/test/hive/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 similarity index 100% rename from src/test/hive/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 rename to shark/src/test/hive/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 diff --git a/src/test/hive/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec b/shark/src/test/hive/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec rename to shark/src/test/hive/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec diff --git a/src/test/hive/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 b/shark/src/test/hive/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 rename to shark/src/test/hive/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 diff --git a/src/test/hive/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec b/shark/src/test/hive/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec rename to shark/src/test/hive/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec diff --git a/src/test/hive/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 b/shark/src/test/hive/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 rename to shark/src/test/hive/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 diff --git a/src/test/hive/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 b/shark/src/test/hive/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 rename to shark/src/test/hive/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 diff --git a/src/test/hive/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 b/shark/src/test/hive/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 rename to shark/src/test/hive/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 diff --git a/src/test/hive/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 b/shark/src/test/hive/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 rename to shark/src/test/hive/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 diff --git a/src/test/hive/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 b/shark/src/test/hive/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 rename to shark/src/test/hive/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 diff --git a/src/test/hive/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 b/shark/src/test/hive/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 rename to shark/src/test/hive/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 diff --git a/src/test/hive/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e b/shark/src/test/hive/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e rename to shark/src/test/hive/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e diff --git a/src/test/hive/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 b/shark/src/test/hive/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 similarity index 100% rename from src/test/hive/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 rename to shark/src/test/hive/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 diff --git a/src/test/hive/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a b/shark/src/test/hive/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a similarity index 100% rename from src/test/hive/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a rename to shark/src/test/hive/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a diff --git a/src/test/hive/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 b/shark/src/test/hive/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 similarity index 100% rename from src/test/hive/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 rename to shark/src/test/hive/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 diff --git a/src/test/hive/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca b/shark/src/test/hive/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca similarity index 100% rename from src/test/hive/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca rename to shark/src/test/hive/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca diff --git a/src/test/hive/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 b/shark/src/test/hive/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 similarity index 100% rename from src/test/hive/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 rename to shark/src/test/hive/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 diff --git a/src/test/hive/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 b/shark/src/test/hive/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 similarity index 100% rename from src/test/hive/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 rename to shark/src/test/hive/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 diff --git a/src/test/hive/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce b/shark/src/test/hive/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce similarity index 100% rename from src/test/hive/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce rename to shark/src/test/hive/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce diff --git a/src/test/hive/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 b/shark/src/test/hive/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 similarity index 100% rename from src/test/hive/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 rename to shark/src/test/hive/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 diff --git a/src/test/hive/golden/show_tables-11-3f9a7f993510123059493826470f78f7 b/shark/src/test/hive/golden/show_tables-11-3f9a7f993510123059493826470f78f7 similarity index 100% rename from src/test/hive/golden/show_tables-11-3f9a7f993510123059493826470f78f7 rename to shark/src/test/hive/golden/show_tables-11-3f9a7f993510123059493826470f78f7 diff --git a/src/test/hive/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c b/shark/src/test/hive/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c similarity index 100% rename from src/test/hive/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c rename to shark/src/test/hive/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c diff --git a/src/test/hive/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 b/shark/src/test/hive/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 similarity index 100% rename from src/test/hive/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 rename to shark/src/test/hive/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 diff --git a/src/test/hive/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c b/shark/src/test/hive/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c similarity index 100% rename from src/test/hive/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c rename to shark/src/test/hive/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c diff --git a/src/test/hive/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe b/shark/src/test/hive/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe similarity index 100% rename from src/test/hive/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe rename to shark/src/test/hive/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe diff --git a/src/test/hive/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 b/shark/src/test/hive/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 similarity index 100% rename from src/test/hive/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 rename to shark/src/test/hive/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 diff --git a/src/test/hive/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab b/shark/src/test/hive/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab similarity index 100% rename from src/test/hive/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab rename to shark/src/test/hive/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab diff --git a/src/test/hive/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 b/shark/src/test/hive/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 similarity index 100% rename from src/test/hive/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 rename to shark/src/test/hive/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 diff --git a/src/test/hive/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 b/shark/src/test/hive/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 similarity index 100% rename from src/test/hive/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 rename to shark/src/test/hive/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 diff --git a/src/test/hive/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 b/shark/src/test/hive/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 similarity index 100% rename from src/test/hive/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 rename to shark/src/test/hive/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 diff --git a/src/test/hive/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 b/shark/src/test/hive/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 similarity index 100% rename from src/test/hive/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 rename to shark/src/test/hive/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 diff --git a/src/test/hive/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 b/shark/src/test/hive/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 similarity index 100% rename from src/test/hive/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 rename to shark/src/test/hive/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 diff --git a/src/test/hive/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 b/shark/src/test/hive/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 similarity index 100% rename from src/test/hive/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 rename to shark/src/test/hive/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 diff --git a/src/test/hive/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c b/shark/src/test/hive/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c similarity index 100% rename from src/test/hive/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c rename to shark/src/test/hive/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c diff --git a/src/test/hive/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 b/shark/src/test/hive/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 similarity index 100% rename from src/test/hive/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 rename to shark/src/test/hive/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 diff --git a/src/test/hive/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d b/shark/src/test/hive/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d similarity index 100% rename from src/test/hive/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d rename to shark/src/test/hive/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d diff --git a/src/test/hive/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 b/shark/src/test/hive/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 similarity index 100% rename from src/test/hive/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 rename to shark/src/test/hive/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 diff --git a/src/test/hive/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c b/shark/src/test/hive/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c similarity index 100% rename from src/test/hive/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c rename to shark/src/test/hive/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c diff --git a/src/test/hive/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 b/shark/src/test/hive/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 similarity index 100% rename from src/test/hive/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 rename to shark/src/test/hive/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 diff --git a/src/test/hive/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 b/shark/src/test/hive/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 similarity index 100% rename from src/test/hive/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 rename to shark/src/test/hive/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 diff --git a/src/test/hive/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa b/shark/src/test/hive/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa similarity index 100% rename from src/test/hive/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa rename to shark/src/test/hive/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa diff --git a/src/test/hive/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 b/shark/src/test/hive/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 similarity index 100% rename from src/test/hive/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 rename to shark/src/test/hive/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 diff --git a/src/test/hive/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 b/shark/src/test/hive/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 similarity index 100% rename from src/test/hive/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 rename to shark/src/test/hive/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 diff --git a/src/test/hive/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d b/shark/src/test/hive/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d similarity index 100% rename from src/test/hive/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d rename to shark/src/test/hive/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d diff --git a/src/test/hive/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 b/shark/src/test/hive/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 similarity index 100% rename from src/test/hive/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 rename to shark/src/test/hive/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 diff --git a/src/test/hive/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 b/shark/src/test/hive/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 similarity index 100% rename from src/test/hive/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 rename to shark/src/test/hive/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 diff --git a/src/test/hive/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 b/shark/src/test/hive/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 similarity index 100% rename from src/test/hive/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 rename to shark/src/test/hive/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 diff --git a/src/test/hive/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c b/shark/src/test/hive/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c similarity index 100% rename from src/test/hive/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c rename to shark/src/test/hive/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c diff --git a/src/test/hive/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 b/shark/src/test/hive/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 similarity index 100% rename from src/test/hive/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 rename to shark/src/test/hive/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 diff --git a/src/test/hive/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 b/shark/src/test/hive/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 similarity index 100% rename from src/test/hive/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 rename to shark/src/test/hive/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 diff --git a/src/test/hive/golden/showparts-0-593619bb962b318b82896658deaea1f1 b/shark/src/test/hive/golden/showparts-0-593619bb962b318b82896658deaea1f1 similarity index 100% rename from src/test/hive/golden/showparts-0-593619bb962b318b82896658deaea1f1 rename to shark/src/test/hive/golden/showparts-0-593619bb962b318b82896658deaea1f1 diff --git a/src/test/hive/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a b/shark/src/test/hive/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a similarity index 100% rename from src/test/hive/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a rename to shark/src/test/hive/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a diff --git a/src/test/hive/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b b/shark/src/test/hive/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b rename to shark/src/test/hive/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b diff --git a/src/test/hive/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f b/shark/src/test/hive/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f rename to shark/src/test/hive/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f diff --git a/src/test/hive/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 b/shark/src/test/hive/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 rename to shark/src/test/hive/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 diff --git a/src/test/hive/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 b/shark/src/test/hive/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 rename to shark/src/test/hive/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e b/shark/src/test/hive/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e rename to shark/src/test/hive/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e diff --git a/src/test/hive/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 b/shark/src/test/hive/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 rename to shark/src/test/hive/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 b/shark/src/test/hive/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 rename to shark/src/test/hive/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 b/shark/src/test/hive/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 rename to shark/src/test/hive/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 diff --git a/src/test/hive/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 b/shark/src/test/hive/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 similarity index 100% rename from src/test/hive/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 rename to shark/src/test/hive/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 diff --git a/src/test/hive/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b b/shark/src/test/hive/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b similarity index 100% rename from src/test/hive/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b rename to shark/src/test/hive/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b diff --git a/src/test/hive/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 b/shark/src/test/hive/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 similarity index 100% rename from src/test/hive/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 rename to shark/src/test/hive/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 diff --git a/src/test/hive/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 b/shark/src/test/hive/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 similarity index 100% rename from src/test/hive/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 rename to shark/src/test/hive/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 diff --git a/src/test/hive/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a b/shark/src/test/hive/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a similarity index 100% rename from src/test/hive/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a rename to shark/src/test/hive/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a diff --git a/src/test/hive/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 b/shark/src/test/hive/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 similarity index 100% rename from src/test/hive/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 rename to shark/src/test/hive/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 diff --git a/src/test/hive/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 b/shark/src/test/hive/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 similarity index 100% rename from src/test/hive/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 rename to shark/src/test/hive/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 diff --git a/src/test/hive/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 b/shark/src/test/hive/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 similarity index 100% rename from src/test/hive/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 rename to shark/src/test/hive/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 diff --git a/src/test/hive/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 b/shark/src/test/hive/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 similarity index 100% rename from src/test/hive/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 rename to shark/src/test/hive/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 diff --git a/src/test/hive/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 b/shark/src/test/hive/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 similarity index 100% rename from src/test/hive/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 rename to shark/src/test/hive/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 diff --git a/src/test/hive/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 b/shark/src/test/hive/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 similarity index 100% rename from src/test/hive/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 rename to shark/src/test/hive/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 diff --git a/src/test/hive/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 b/shark/src/test/hive/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 similarity index 100% rename from src/test/hive/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 rename to shark/src/test/hive/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 diff --git a/src/test/hive/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd b/shark/src/test/hive/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd similarity index 100% rename from src/test/hive/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd rename to shark/src/test/hive/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd diff --git a/src/test/hive/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 b/shark/src/test/hive/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 similarity index 100% rename from src/test/hive/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 rename to shark/src/test/hive/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 diff --git a/src/test/hive/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 b/shark/src/test/hive/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 similarity index 100% rename from src/test/hive/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 rename to shark/src/test/hive/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 diff --git a/src/test/hive/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 b/shark/src/test/hive/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 similarity index 100% rename from src/test/hive/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 rename to shark/src/test/hive/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 diff --git a/src/test/hive/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 b/shark/src/test/hive/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 similarity index 100% rename from src/test/hive/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 rename to shark/src/test/hive/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 diff --git a/src/test/hive/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 b/shark/src/test/hive/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 similarity index 100% rename from src/test/hive/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 rename to shark/src/test/hive/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 diff --git a/src/test/hive/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 b/shark/src/test/hive/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 similarity index 100% rename from src/test/hive/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 rename to shark/src/test/hive/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 diff --git a/src/test/hive/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d b/shark/src/test/hive/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d similarity index 100% rename from src/test/hive/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d rename to shark/src/test/hive/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d diff --git a/src/test/hive/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 b/shark/src/test/hive/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 similarity index 100% rename from src/test/hive/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 rename to shark/src/test/hive/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 diff --git a/src/test/hive/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 b/shark/src/test/hive/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 similarity index 100% rename from src/test/hive/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 rename to shark/src/test/hive/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 diff --git a/src/test/hive/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 b/shark/src/test/hive/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 similarity index 100% rename from src/test/hive/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 rename to shark/src/test/hive/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 diff --git a/src/test/hive/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 b/shark/src/test/hive/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 similarity index 100% rename from src/test/hive/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 rename to shark/src/test/hive/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 diff --git a/src/test/hive/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 b/shark/src/test/hive/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 similarity index 100% rename from src/test/hive/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 rename to shark/src/test/hive/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 diff --git a/src/test/hive/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 b/shark/src/test/hive/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 similarity index 100% rename from src/test/hive/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 rename to shark/src/test/hive/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 diff --git a/src/test/hive/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 b/shark/src/test/hive/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 similarity index 100% rename from src/test/hive/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 rename to shark/src/test/hive/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 diff --git a/src/test/hive/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 b/shark/src/test/hive/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 similarity index 100% rename from src/test/hive/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 rename to shark/src/test/hive/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 diff --git a/src/test/hive/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/shark/src/test/hive/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 similarity index 100% rename from src/test/hive/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 rename to shark/src/test/hive/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 diff --git a/src/test/hive/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 b/shark/src/test/hive/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 similarity index 100% rename from src/test/hive/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 rename to shark/src/test/hive/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 diff --git a/src/test/hive/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d b/shark/src/test/hive/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d similarity index 100% rename from src/test/hive/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d rename to shark/src/test/hive/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d diff --git a/src/test/hive/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 b/shark/src/test/hive/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 similarity index 100% rename from src/test/hive/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 rename to shark/src/test/hive/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 diff --git a/src/test/hive/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 b/shark/src/test/hive/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 similarity index 100% rename from src/test/hive/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 rename to shark/src/test/hive/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 diff --git a/src/test/hive/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e b/shark/src/test/hive/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e similarity index 100% rename from src/test/hive/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e rename to shark/src/test/hive/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e diff --git a/src/test/hive/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b b/shark/src/test/hive/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b similarity index 100% rename from src/test/hive/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b rename to shark/src/test/hive/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b diff --git a/src/test/hive/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b b/shark/src/test/hive/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b similarity index 100% rename from src/test/hive/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b rename to shark/src/test/hive/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b diff --git a/src/test/hive/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 b/shark/src/test/hive/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 similarity index 100% rename from src/test/hive/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 rename to shark/src/test/hive/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 diff --git a/src/test/hive/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b b/shark/src/test/hive/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b similarity index 100% rename from src/test/hive/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b rename to shark/src/test/hive/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b diff --git a/src/test/hive/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 b/shark/src/test/hive/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 similarity index 100% rename from src/test/hive/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 rename to shark/src/test/hive/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 diff --git a/src/test/hive/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 b/shark/src/test/hive/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 similarity index 100% rename from src/test/hive/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 rename to shark/src/test/hive/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 diff --git a/src/test/hive/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa b/shark/src/test/hive/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa similarity index 100% rename from src/test/hive/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa rename to shark/src/test/hive/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa diff --git a/src/test/hive/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 b/shark/src/test/hive/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 similarity index 100% rename from src/test/hive/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 rename to shark/src/test/hive/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 diff --git a/src/test/hive/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/shark/src/test/hive/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 similarity index 100% rename from src/test/hive/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 rename to shark/src/test/hive/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 diff --git a/src/test/hive/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 b/shark/src/test/hive/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 similarity index 100% rename from src/test/hive/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 rename to shark/src/test/hive/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 diff --git a/src/test/hive/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d b/shark/src/test/hive/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d similarity index 100% rename from src/test/hive/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d rename to shark/src/test/hive/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d diff --git a/src/test/hive/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 b/shark/src/test/hive/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 similarity index 100% rename from src/test/hive/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 rename to shark/src/test/hive/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 diff --git a/src/test/hive/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 b/shark/src/test/hive/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 similarity index 100% rename from src/test/hive/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 rename to shark/src/test/hive/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 diff --git a/src/test/hive/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e b/shark/src/test/hive/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e similarity index 100% rename from src/test/hive/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e rename to shark/src/test/hive/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e diff --git a/src/test/hive/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 b/shark/src/test/hive/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 similarity index 100% rename from src/test/hive/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 rename to shark/src/test/hive/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 diff --git a/src/test/hive/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 b/shark/src/test/hive/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 similarity index 100% rename from src/test/hive/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 rename to shark/src/test/hive/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 diff --git a/src/test/hive/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d b/shark/src/test/hive/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d similarity index 100% rename from src/test/hive/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d rename to shark/src/test/hive/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d diff --git a/src/test/hive/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 b/shark/src/test/hive/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 similarity index 100% rename from src/test/hive/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 rename to shark/src/test/hive/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 diff --git a/src/test/hive/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 b/shark/src/test/hive/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 similarity index 100% rename from src/test/hive/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 rename to shark/src/test/hive/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 diff --git a/src/test/hive/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 b/shark/src/test/hive/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 similarity index 100% rename from src/test/hive/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 rename to shark/src/test/hive/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 diff --git a/src/test/hive/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 b/shark/src/test/hive/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 similarity index 100% rename from src/test/hive/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 rename to shark/src/test/hive/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 diff --git a/src/test/hive/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb b/shark/src/test/hive/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb similarity index 100% rename from src/test/hive/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb rename to shark/src/test/hive/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb diff --git a/src/test/hive/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 b/shark/src/test/hive/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 similarity index 100% rename from src/test/hive/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 rename to shark/src/test/hive/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 diff --git a/src/test/hive/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 b/shark/src/test/hive/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 similarity index 100% rename from src/test/hive/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 rename to shark/src/test/hive/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 diff --git a/src/test/hive/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f b/shark/src/test/hive/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f similarity index 100% rename from src/test/hive/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f rename to shark/src/test/hive/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f diff --git a/src/test/hive/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 b/shark/src/test/hive/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 similarity index 100% rename from src/test/hive/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 rename to shark/src/test/hive/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 diff --git a/src/test/hive/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 b/shark/src/test/hive/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 similarity index 100% rename from src/test/hive/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 rename to shark/src/test/hive/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 diff --git a/src/test/hive/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 b/shark/src/test/hive/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 similarity index 100% rename from src/test/hive/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 rename to shark/src/test/hive/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 diff --git a/src/test/hive/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b b/shark/src/test/hive/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b similarity index 100% rename from src/test/hive/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b rename to shark/src/test/hive/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b diff --git a/src/test/hive/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b b/shark/src/test/hive/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b similarity index 100% rename from src/test/hive/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b rename to shark/src/test/hive/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b diff --git a/src/test/hive/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d b/shark/src/test/hive/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d similarity index 100% rename from src/test/hive/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d rename to shark/src/test/hive/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d diff --git a/src/test/hive/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 b/shark/src/test/hive/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 similarity index 100% rename from src/test/hive/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 rename to shark/src/test/hive/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 diff --git a/src/test/hive/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd b/shark/src/test/hive/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd similarity index 100% rename from src/test/hive/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd rename to shark/src/test/hive/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd diff --git a/src/test/hive/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 b/shark/src/test/hive/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 similarity index 100% rename from src/test/hive/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 rename to shark/src/test/hive/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 diff --git a/src/test/hive/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 b/shark/src/test/hive/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 similarity index 100% rename from src/test/hive/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 rename to shark/src/test/hive/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 diff --git a/src/test/hive/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 b/shark/src/test/hive/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 similarity index 100% rename from src/test/hive/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 rename to shark/src/test/hive/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 diff --git a/src/test/hive/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e b/shark/src/test/hive/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e similarity index 100% rename from src/test/hive/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e rename to shark/src/test/hive/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e diff --git a/src/test/hive/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 b/shark/src/test/hive/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 similarity index 100% rename from src/test/hive/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 rename to shark/src/test/hive/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 diff --git a/src/test/hive/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c b/shark/src/test/hive/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c similarity index 100% rename from src/test/hive/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c rename to shark/src/test/hive/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c diff --git a/src/test/hive/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 b/shark/src/test/hive/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 similarity index 100% rename from src/test/hive/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 rename to shark/src/test/hive/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 diff --git a/src/test/hive/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 b/shark/src/test/hive/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 similarity index 100% rename from src/test/hive/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 rename to shark/src/test/hive/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 diff --git a/src/test/hive/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e b/shark/src/test/hive/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e similarity index 100% rename from src/test/hive/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e rename to shark/src/test/hive/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e diff --git a/src/test/hive/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 b/shark/src/test/hive/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 similarity index 100% rename from src/test/hive/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 rename to shark/src/test/hive/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 diff --git a/src/test/hive/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd b/shark/src/test/hive/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd similarity index 100% rename from src/test/hive/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd rename to shark/src/test/hive/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd diff --git a/src/test/hive/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 b/shark/src/test/hive/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 similarity index 100% rename from src/test/hive/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 rename to shark/src/test/hive/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 diff --git a/src/test/hive/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 b/shark/src/test/hive/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 similarity index 100% rename from src/test/hive/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 rename to shark/src/test/hive/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 diff --git a/src/test/hive/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d b/shark/src/test/hive/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d similarity index 100% rename from src/test/hive/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d rename to shark/src/test/hive/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d diff --git a/src/test/hive/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 b/shark/src/test/hive/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 similarity index 100% rename from src/test/hive/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 rename to shark/src/test/hive/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 diff --git a/src/test/hive/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/shark/src/test/hive/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b similarity index 100% rename from src/test/hive/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename to shark/src/test/hive/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b diff --git a/src/test/hive/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 b/shark/src/test/hive/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 similarity index 100% rename from src/test/hive/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 rename to shark/src/test/hive/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 diff --git a/src/test/hive/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 b/shark/src/test/hive/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 similarity index 100% rename from src/test/hive/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 rename to shark/src/test/hive/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 diff --git a/src/test/hive/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 b/shark/src/test/hive/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 similarity index 100% rename from src/test/hive/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 rename to shark/src/test/hive/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 diff --git a/src/test/hive/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d b/shark/src/test/hive/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d similarity index 100% rename from src/test/hive/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d rename to shark/src/test/hive/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d diff --git a/src/test/hive/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 b/shark/src/test/hive/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 similarity index 100% rename from src/test/hive/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 rename to shark/src/test/hive/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 diff --git a/src/test/hive/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 b/shark/src/test/hive/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 similarity index 100% rename from src/test/hive/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 rename to shark/src/test/hive/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 diff --git a/src/test/hive/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a b/shark/src/test/hive/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a rename to shark/src/test/hive/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a diff --git a/src/test/hive/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec b/shark/src/test/hive/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec rename to shark/src/test/hive/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec diff --git a/src/test/hive/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 b/shark/src/test/hive/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 rename to shark/src/test/hive/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 diff --git a/src/test/hive/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed b/shark/src/test/hive/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed rename to shark/src/test/hive/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed diff --git a/src/test/hive/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa b/shark/src/test/hive/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa rename to shark/src/test/hive/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa diff --git a/src/test/hive/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 b/shark/src/test/hive/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 rename to shark/src/test/hive/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 diff --git a/src/test/hive/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 b/shark/src/test/hive/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 rename to shark/src/test/hive/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 diff --git a/src/test/hive/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c b/shark/src/test/hive/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c rename to shark/src/test/hive/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c diff --git a/src/test/hive/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 b/shark/src/test/hive/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 rename to shark/src/test/hive/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 diff --git a/src/test/hive/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 b/shark/src/test/hive/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 rename to shark/src/test/hive/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 diff --git a/src/test/hive/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 b/shark/src/test/hive/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 rename to shark/src/test/hive/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 diff --git a/src/test/hive/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 b/shark/src/test/hive/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 rename to shark/src/test/hive/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 diff --git a/src/test/hive/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f b/shark/src/test/hive/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f rename to shark/src/test/hive/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f diff --git a/src/test/hive/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 b/shark/src/test/hive/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 rename to shark/src/test/hive/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 diff --git a/src/test/hive/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec b/shark/src/test/hive/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec rename to shark/src/test/hive/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec diff --git a/src/test/hive/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b b/shark/src/test/hive/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b rename to shark/src/test/hive/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b diff --git a/src/test/hive/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd b/shark/src/test/hive/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd rename to shark/src/test/hive/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd diff --git a/src/test/hive/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae b/shark/src/test/hive/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae rename to shark/src/test/hive/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae diff --git a/src/test/hive/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 b/shark/src/test/hive/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 rename to shark/src/test/hive/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 diff --git a/src/test/hive/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de b/shark/src/test/hive/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de rename to shark/src/test/hive/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de diff --git a/src/test/hive/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 b/shark/src/test/hive/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 rename to shark/src/test/hive/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 diff --git a/src/test/hive/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea b/shark/src/test/hive/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea similarity index 100% rename from src/test/hive/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea rename to shark/src/test/hive/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea diff --git a/src/test/hive/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 b/shark/src/test/hive/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 rename to shark/src/test/hive/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 diff --git a/src/test/hive/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 b/shark/src/test/hive/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 rename to shark/src/test/hive/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 diff --git a/src/test/hive/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 b/shark/src/test/hive/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 rename to shark/src/test/hive/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 diff --git a/src/test/hive/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 b/shark/src/test/hive/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 rename to shark/src/test/hive/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 diff --git a/src/test/hive/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 b/shark/src/test/hive/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 rename to shark/src/test/hive/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 diff --git a/src/test/hive/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 b/shark/src/test/hive/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 rename to shark/src/test/hive/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 diff --git a/src/test/hive/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 b/shark/src/test/hive/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 rename to shark/src/test/hive/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 diff --git a/src/test/hive/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 b/shark/src/test/hive/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 rename to shark/src/test/hive/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 diff --git a/src/test/hive/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa b/shark/src/test/hive/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa rename to shark/src/test/hive/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa diff --git a/src/test/hive/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 b/shark/src/test/hive/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 rename to shark/src/test/hive/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 diff --git a/src/test/hive/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 b/shark/src/test/hive/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 rename to shark/src/test/hive/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 diff --git a/src/test/hive/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 b/shark/src/test/hive/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 rename to shark/src/test/hive/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 diff --git a/src/test/hive/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 b/shark/src/test/hive/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 rename to shark/src/test/hive/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 diff --git a/src/test/hive/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 b/shark/src/test/hive/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 rename to shark/src/test/hive/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 diff --git a/src/test/hive/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af b/shark/src/test/hive/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af rename to shark/src/test/hive/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af diff --git a/src/test/hive/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f b/shark/src/test/hive/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f rename to shark/src/test/hive/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f diff --git a/src/test/hive/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 b/shark/src/test/hive/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 rename to shark/src/test/hive/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 diff --git a/src/test/hive/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 b/shark/src/test/hive/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 rename to shark/src/test/hive/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 diff --git a/src/test/hive/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 b/shark/src/test/hive/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 rename to shark/src/test/hive/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 diff --git a/src/test/hive/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 b/shark/src/test/hive/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 rename to shark/src/test/hive/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 diff --git a/src/test/hive/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb b/shark/src/test/hive/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb rename to shark/src/test/hive/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb diff --git a/src/test/hive/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f b/shark/src/test/hive/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f rename to shark/src/test/hive/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f diff --git a/src/test/hive/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 b/shark/src/test/hive/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 rename to shark/src/test/hive/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 diff --git a/src/test/hive/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af b/shark/src/test/hive/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af rename to shark/src/test/hive/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af diff --git a/src/test/hive/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b b/shark/src/test/hive/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b rename to shark/src/test/hive/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b diff --git a/src/test/hive/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e b/shark/src/test/hive/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e rename to shark/src/test/hive/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e diff --git a/src/test/hive/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e b/shark/src/test/hive/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e rename to shark/src/test/hive/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e diff --git a/src/test/hive/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 b/shark/src/test/hive/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 rename to shark/src/test/hive/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 diff --git a/src/test/hive/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 b/shark/src/test/hive/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 rename to shark/src/test/hive/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 diff --git a/src/test/hive/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 b/shark/src/test/hive/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 rename to shark/src/test/hive/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 diff --git a/src/test/hive/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 b/shark/src/test/hive/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 rename to shark/src/test/hive/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 diff --git a/src/test/hive/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 b/shark/src/test/hive/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 rename to shark/src/test/hive/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 diff --git a/src/test/hive/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee b/shark/src/test/hive/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee rename to shark/src/test/hive/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee diff --git a/src/test/hive/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 b/shark/src/test/hive/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 rename to shark/src/test/hive/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 diff --git a/src/test/hive/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 b/shark/src/test/hive/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 rename to shark/src/test/hive/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 diff --git a/src/test/hive/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 b/shark/src/test/hive/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 rename to shark/src/test/hive/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 diff --git a/src/test/hive/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 b/shark/src/test/hive/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 rename to shark/src/test/hive/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 diff --git a/src/test/hive/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 b/shark/src/test/hive/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 rename to shark/src/test/hive/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 diff --git a/src/test/hive/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd b/shark/src/test/hive/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd rename to shark/src/test/hive/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd diff --git a/src/test/hive/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d b/shark/src/test/hive/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d rename to shark/src/test/hive/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d diff --git a/src/test/hive/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 b/shark/src/test/hive/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 rename to shark/src/test/hive/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 diff --git a/src/test/hive/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a b/shark/src/test/hive/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a rename to shark/src/test/hive/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a diff --git a/src/test/hive/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d b/shark/src/test/hive/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d rename to shark/src/test/hive/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d diff --git a/src/test/hive/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 b/shark/src/test/hive/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 rename to shark/src/test/hive/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 diff --git a/src/test/hive/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 b/shark/src/test/hive/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 rename to shark/src/test/hive/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 diff --git a/src/test/hive/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee b/shark/src/test/hive/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee rename to shark/src/test/hive/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee diff --git a/src/test/hive/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 b/shark/src/test/hive/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 rename to shark/src/test/hive/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 diff --git a/src/test/hive/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 b/shark/src/test/hive/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 rename to shark/src/test/hive/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 diff --git a/src/test/hive/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a b/shark/src/test/hive/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a rename to shark/src/test/hive/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a diff --git a/src/test/hive/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 b/shark/src/test/hive/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 rename to shark/src/test/hive/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 diff --git a/src/test/hive/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b b/shark/src/test/hive/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b rename to shark/src/test/hive/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b diff --git a/src/test/hive/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 b/shark/src/test/hive/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 rename to shark/src/test/hive/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 diff --git a/src/test/hive/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 b/shark/src/test/hive/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 rename to shark/src/test/hive/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 diff --git a/src/test/hive/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e b/shark/src/test/hive/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e rename to shark/src/test/hive/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e diff --git a/src/test/hive/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b b/shark/src/test/hive/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b rename to shark/src/test/hive/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b diff --git a/src/test/hive/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 b/shark/src/test/hive/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 rename to shark/src/test/hive/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 diff --git a/src/test/hive/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 b/shark/src/test/hive/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 rename to shark/src/test/hive/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 diff --git a/src/test/hive/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b b/shark/src/test/hive/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b rename to shark/src/test/hive/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b diff --git a/src/test/hive/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da b/shark/src/test/hive/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da rename to shark/src/test/hive/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da diff --git a/src/test/hive/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb b/shark/src/test/hive/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb rename to shark/src/test/hive/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb diff --git a/src/test/hive/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 b/shark/src/test/hive/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 rename to shark/src/test/hive/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 diff --git a/src/test/hive/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 b/shark/src/test/hive/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 rename to shark/src/test/hive/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 diff --git a/src/test/hive/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 b/shark/src/test/hive/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 rename to shark/src/test/hive/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 diff --git a/src/test/hive/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 b/shark/src/test/hive/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 rename to shark/src/test/hive/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 diff --git a/src/test/hive/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f b/shark/src/test/hive/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f rename to shark/src/test/hive/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f diff --git a/src/test/hive/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f b/shark/src/test/hive/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f rename to shark/src/test/hive/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f diff --git a/src/test/hive/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b b/shark/src/test/hive/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b rename to shark/src/test/hive/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b diff --git a/src/test/hive/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 b/shark/src/test/hive/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 rename to shark/src/test/hive/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 diff --git a/src/test/hive/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a b/shark/src/test/hive/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a rename to shark/src/test/hive/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a diff --git a/src/test/hive/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 b/shark/src/test/hive/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 rename to shark/src/test/hive/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 diff --git a/src/test/hive/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 b/shark/src/test/hive/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 rename to shark/src/test/hive/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 diff --git a/src/test/hive/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 b/shark/src/test/hive/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 rename to shark/src/test/hive/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 diff --git a/src/test/hive/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 b/shark/src/test/hive/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 rename to shark/src/test/hive/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 diff --git a/src/test/hive/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 b/shark/src/test/hive/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 rename to shark/src/test/hive/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 diff --git a/src/test/hive/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c b/shark/src/test/hive/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c rename to shark/src/test/hive/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c diff --git a/src/test/hive/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f b/shark/src/test/hive/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f rename to shark/src/test/hive/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f diff --git a/src/test/hive/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 b/shark/src/test/hive/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 rename to shark/src/test/hive/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 diff --git a/src/test/hive/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e b/shark/src/test/hive/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e rename to shark/src/test/hive/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e diff --git a/src/test/hive/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 b/shark/src/test/hive/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 rename to shark/src/test/hive/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 diff --git a/src/test/hive/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 b/shark/src/test/hive/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 rename to shark/src/test/hive/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 diff --git a/src/test/hive/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 b/shark/src/test/hive/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 rename to shark/src/test/hive/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 diff --git a/src/test/hive/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 b/shark/src/test/hive/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 rename to shark/src/test/hive/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 diff --git a/src/test/hive/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb b/shark/src/test/hive/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb rename to shark/src/test/hive/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb diff --git a/src/test/hive/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 b/shark/src/test/hive/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 rename to shark/src/test/hive/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 diff --git a/src/test/hive/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b b/shark/src/test/hive/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b rename to shark/src/test/hive/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b diff --git a/src/test/hive/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c b/shark/src/test/hive/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c rename to shark/src/test/hive/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c diff --git a/src/test/hive/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf b/shark/src/test/hive/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf rename to shark/src/test/hive/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf diff --git a/src/test/hive/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 b/shark/src/test/hive/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 rename to shark/src/test/hive/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 diff --git a/src/test/hive/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e b/shark/src/test/hive/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e rename to shark/src/test/hive/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e diff --git a/src/test/hive/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 b/shark/src/test/hive/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 rename to shark/src/test/hive/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 diff --git a/src/test/hive/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 b/shark/src/test/hive/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 rename to shark/src/test/hive/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 diff --git a/src/test/hive/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 b/shark/src/test/hive/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 rename to shark/src/test/hive/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 diff --git a/src/test/hive/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 b/shark/src/test/hive/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 rename to shark/src/test/hive/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 diff --git a/src/test/hive/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 b/shark/src/test/hive/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 rename to shark/src/test/hive/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 diff --git a/src/test/hive/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 b/shark/src/test/hive/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 rename to shark/src/test/hive/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 diff --git a/src/test/hive/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 b/shark/src/test/hive/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 rename to shark/src/test/hive/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 diff --git a/src/test/hive/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 b/shark/src/test/hive/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 rename to shark/src/test/hive/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 diff --git a/src/test/hive/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 b/shark/src/test/hive/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 rename to shark/src/test/hive/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 diff --git a/src/test/hive/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 b/shark/src/test/hive/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 rename to shark/src/test/hive/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 diff --git a/src/test/hive/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c b/shark/src/test/hive/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c rename to shark/src/test/hive/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c diff --git a/src/test/hive/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f b/shark/src/test/hive/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f rename to shark/src/test/hive/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f diff --git a/src/test/hive/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 b/shark/src/test/hive/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 rename to shark/src/test/hive/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 diff --git a/src/test/hive/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e b/shark/src/test/hive/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e rename to shark/src/test/hive/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e diff --git a/src/test/hive/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 b/shark/src/test/hive/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 rename to shark/src/test/hive/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 diff --git a/src/test/hive/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a b/shark/src/test/hive/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a rename to shark/src/test/hive/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a diff --git a/src/test/hive/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 b/shark/src/test/hive/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 rename to shark/src/test/hive/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 diff --git a/src/test/hive/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 b/shark/src/test/hive/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 rename to shark/src/test/hive/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 diff --git a/src/test/hive/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e b/shark/src/test/hive/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e rename to shark/src/test/hive/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e diff --git a/src/test/hive/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc b/shark/src/test/hive/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc rename to shark/src/test/hive/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc diff --git a/src/test/hive/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff b/shark/src/test/hive/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff rename to shark/src/test/hive/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff diff --git a/src/test/hive/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 b/shark/src/test/hive/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 rename to shark/src/test/hive/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 diff --git a/src/test/hive/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e b/shark/src/test/hive/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e rename to shark/src/test/hive/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e diff --git a/src/test/hive/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/shark/src/test/hive/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e rename to shark/src/test/hive/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e diff --git a/src/test/hive/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f b/shark/src/test/hive/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f rename to shark/src/test/hive/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f diff --git a/src/test/hive/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 b/shark/src/test/hive/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 rename to shark/src/test/hive/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 diff --git a/src/test/hive/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 b/shark/src/test/hive/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 rename to shark/src/test/hive/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 diff --git a/src/test/hive/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 b/shark/src/test/hive/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 rename to shark/src/test/hive/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 diff --git a/src/test/hive/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 b/shark/src/test/hive/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 rename to shark/src/test/hive/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 diff --git a/src/test/hive/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 b/shark/src/test/hive/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 rename to shark/src/test/hive/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 diff --git a/src/test/hive/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/shark/src/test/hive/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 rename to shark/src/test/hive/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 diff --git a/src/test/hive/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d b/shark/src/test/hive/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d rename to shark/src/test/hive/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d diff --git a/src/test/hive/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 b/shark/src/test/hive/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 rename to shark/src/test/hive/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 diff --git a/src/test/hive/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a b/shark/src/test/hive/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a rename to shark/src/test/hive/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a diff --git a/src/test/hive/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 b/shark/src/test/hive/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 rename to shark/src/test/hive/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 diff --git a/src/test/hive/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 b/shark/src/test/hive/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 rename to shark/src/test/hive/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 diff --git a/src/test/hive/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 b/shark/src/test/hive/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 rename to shark/src/test/hive/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 diff --git a/src/test/hive/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c b/shark/src/test/hive/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c rename to shark/src/test/hive/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c diff --git a/src/test/hive/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 b/shark/src/test/hive/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 rename to shark/src/test/hive/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 diff --git a/src/test/hive/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f b/shark/src/test/hive/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f rename to shark/src/test/hive/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f diff --git a/src/test/hive/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 b/shark/src/test/hive/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 rename to shark/src/test/hive/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 diff --git a/src/test/hive/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab b/shark/src/test/hive/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab rename to shark/src/test/hive/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab diff --git a/src/test/hive/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 b/shark/src/test/hive/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 rename to shark/src/test/hive/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 diff --git a/src/test/hive/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 b/shark/src/test/hive/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 rename to shark/src/test/hive/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 diff --git a/src/test/hive/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 b/shark/src/test/hive/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 rename to shark/src/test/hive/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 diff --git a/src/test/hive/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c b/shark/src/test/hive/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c rename to shark/src/test/hive/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c diff --git a/src/test/hive/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe b/shark/src/test/hive/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe rename to shark/src/test/hive/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe diff --git a/src/test/hive/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 b/shark/src/test/hive/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 rename to shark/src/test/hive/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 diff --git a/src/test/hive/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 b/shark/src/test/hive/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 rename to shark/src/test/hive/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 diff --git a/src/test/hive/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c b/shark/src/test/hive/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c rename to shark/src/test/hive/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c diff --git a/src/test/hive/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 b/shark/src/test/hive/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 rename to shark/src/test/hive/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 diff --git a/src/test/hive/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/shark/src/test/hive/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec rename to shark/src/test/hive/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec diff --git a/src/test/hive/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/shark/src/test/hive/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e rename to shark/src/test/hive/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e diff --git a/src/test/hive/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f b/shark/src/test/hive/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f rename to shark/src/test/hive/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f diff --git a/src/test/hive/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 b/shark/src/test/hive/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 rename to shark/src/test/hive/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 diff --git a/src/test/hive/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 b/shark/src/test/hive/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 rename to shark/src/test/hive/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 diff --git a/src/test/hive/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 b/shark/src/test/hive/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 rename to shark/src/test/hive/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 diff --git a/src/test/hive/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab b/shark/src/test/hive/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab rename to shark/src/test/hive/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab diff --git a/src/test/hive/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc b/shark/src/test/hive/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc rename to shark/src/test/hive/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc diff --git a/src/test/hive/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec b/shark/src/test/hive/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec rename to shark/src/test/hive/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec diff --git a/src/test/hive/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e b/shark/src/test/hive/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e rename to shark/src/test/hive/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e diff --git a/src/test/hive/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/shark/src/test/hive/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 rename to shark/src/test/hive/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 diff --git a/src/test/hive/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b b/shark/src/test/hive/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b rename to shark/src/test/hive/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b diff --git a/src/test/hive/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 b/shark/src/test/hive/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 rename to shark/src/test/hive/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 diff --git a/src/test/hive/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f b/shark/src/test/hive/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f rename to shark/src/test/hive/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f diff --git a/src/test/hive/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb b/shark/src/test/hive/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb rename to shark/src/test/hive/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb diff --git a/src/test/hive/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 b/shark/src/test/hive/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 rename to shark/src/test/hive/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 diff --git a/src/test/hive/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f b/shark/src/test/hive/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f rename to shark/src/test/hive/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f diff --git a/src/test/hive/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d b/shark/src/test/hive/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d rename to shark/src/test/hive/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d diff --git a/src/test/hive/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 b/shark/src/test/hive/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 rename to shark/src/test/hive/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 diff --git a/src/test/hive/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/shark/src/test/hive/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec rename to shark/src/test/hive/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec diff --git a/src/test/hive/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a b/shark/src/test/hive/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a rename to shark/src/test/hive/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a diff --git a/src/test/hive/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec b/shark/src/test/hive/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec rename to shark/src/test/hive/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec diff --git a/src/test/hive/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac b/shark/src/test/hive/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac rename to shark/src/test/hive/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac diff --git a/src/test/hive/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 b/shark/src/test/hive/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 rename to shark/src/test/hive/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 diff --git a/src/test/hive/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 b/shark/src/test/hive/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 rename to shark/src/test/hive/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 diff --git a/src/test/hive/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d b/shark/src/test/hive/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d rename to shark/src/test/hive/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d diff --git a/src/test/hive/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 b/shark/src/test/hive/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 rename to shark/src/test/hive/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 diff --git a/src/test/hive/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 b/shark/src/test/hive/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 rename to shark/src/test/hive/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 diff --git a/src/test/hive/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca b/shark/src/test/hive/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca rename to shark/src/test/hive/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca diff --git a/src/test/hive/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 b/shark/src/test/hive/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 rename to shark/src/test/hive/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 diff --git a/src/test/hive/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 b/shark/src/test/hive/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 rename to shark/src/test/hive/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 diff --git a/src/test/hive/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 b/shark/src/test/hive/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 rename to shark/src/test/hive/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 diff --git a/src/test/hive/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f b/shark/src/test/hive/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f rename to shark/src/test/hive/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f diff --git a/src/test/hive/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b b/shark/src/test/hive/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b rename to shark/src/test/hive/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b diff --git a/src/test/hive/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 b/shark/src/test/hive/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 rename to shark/src/test/hive/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 diff --git a/src/test/hive/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 b/shark/src/test/hive/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 rename to shark/src/test/hive/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 diff --git a/src/test/hive/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a b/shark/src/test/hive/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a rename to shark/src/test/hive/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a diff --git a/src/test/hive/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 b/shark/src/test/hive/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 rename to shark/src/test/hive/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 diff --git a/src/test/hive/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 b/shark/src/test/hive/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 rename to shark/src/test/hive/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 diff --git a/src/test/hive/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de b/shark/src/test/hive/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de rename to shark/src/test/hive/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de diff --git a/src/test/hive/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 b/shark/src/test/hive/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 rename to shark/src/test/hive/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 diff --git a/src/test/hive/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 b/shark/src/test/hive/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 rename to shark/src/test/hive/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 diff --git a/src/test/hive/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 b/shark/src/test/hive/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 rename to shark/src/test/hive/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 diff --git a/src/test/hive/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 b/shark/src/test/hive/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 rename to shark/src/test/hive/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 diff --git a/src/test/hive/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/shark/src/test/hive/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 rename to shark/src/test/hive/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 diff --git a/src/test/hive/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d b/shark/src/test/hive/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d rename to shark/src/test/hive/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d diff --git a/src/test/hive/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 b/shark/src/test/hive/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 rename to shark/src/test/hive/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 diff --git a/src/test/hive/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a b/shark/src/test/hive/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a rename to shark/src/test/hive/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a diff --git a/src/test/hive/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a b/shark/src/test/hive/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a rename to shark/src/test/hive/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a diff --git a/src/test/hive/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 b/shark/src/test/hive/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 rename to shark/src/test/hive/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 diff --git a/src/test/hive/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e b/shark/src/test/hive/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e rename to shark/src/test/hive/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e diff --git a/src/test/hive/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 b/shark/src/test/hive/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 rename to shark/src/test/hive/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 diff --git a/src/test/hive/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 b/shark/src/test/hive/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 rename to shark/src/test/hive/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 diff --git a/src/test/hive/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 b/shark/src/test/hive/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 rename to shark/src/test/hive/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 diff --git a/src/test/hive/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 b/shark/src/test/hive/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 rename to shark/src/test/hive/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 diff --git a/src/test/hive/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 b/shark/src/test/hive/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 rename to shark/src/test/hive/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 diff --git a/src/test/hive/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 b/shark/src/test/hive/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 rename to shark/src/test/hive/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 diff --git a/src/test/hive/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff b/shark/src/test/hive/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff rename to shark/src/test/hive/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff diff --git a/src/test/hive/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/shark/src/test/hive/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec rename to shark/src/test/hive/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec diff --git a/src/test/hive/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/shark/src/test/hive/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e rename to shark/src/test/hive/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e diff --git a/src/test/hive/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b b/shark/src/test/hive/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b rename to shark/src/test/hive/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b diff --git a/src/test/hive/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 b/shark/src/test/hive/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 rename to shark/src/test/hive/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 diff --git a/src/test/hive/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf b/shark/src/test/hive/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf rename to shark/src/test/hive/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf diff --git a/src/test/hive/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b b/shark/src/test/hive/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b rename to shark/src/test/hive/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b diff --git a/src/test/hive/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd b/shark/src/test/hive/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd rename to shark/src/test/hive/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd diff --git a/src/test/hive/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a b/shark/src/test/hive/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a rename to shark/src/test/hive/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a diff --git a/src/test/hive/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b b/shark/src/test/hive/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b rename to shark/src/test/hive/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b diff --git a/src/test/hive/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 b/shark/src/test/hive/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 rename to shark/src/test/hive/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 diff --git a/src/test/hive/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a b/shark/src/test/hive/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a rename to shark/src/test/hive/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a diff --git a/src/test/hive/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b b/shark/src/test/hive/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b rename to shark/src/test/hive/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b diff --git a/src/test/hive/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab b/shark/src/test/hive/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab rename to shark/src/test/hive/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab diff --git a/src/test/hive/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 b/shark/src/test/hive/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 rename to shark/src/test/hive/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 diff --git a/src/test/hive/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b b/shark/src/test/hive/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b rename to shark/src/test/hive/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b diff --git a/src/test/hive/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 b/shark/src/test/hive/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 rename to shark/src/test/hive/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 diff --git a/src/test/hive/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 b/shark/src/test/hive/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 rename to shark/src/test/hive/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 diff --git a/src/test/hive/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c b/shark/src/test/hive/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c rename to shark/src/test/hive/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c diff --git a/src/test/hive/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 b/shark/src/test/hive/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 rename to shark/src/test/hive/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 diff --git a/src/test/hive/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/shark/src/test/hive/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec rename to shark/src/test/hive/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec diff --git a/src/test/hive/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc b/shark/src/test/hive/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc rename to shark/src/test/hive/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc diff --git a/src/test/hive/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc b/shark/src/test/hive/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc rename to shark/src/test/hive/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc diff --git a/src/test/hive/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a b/shark/src/test/hive/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a rename to shark/src/test/hive/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a diff --git a/src/test/hive/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 b/shark/src/test/hive/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 rename to shark/src/test/hive/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 diff --git a/src/test/hive/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 b/shark/src/test/hive/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 rename to shark/src/test/hive/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 diff --git a/src/test/hive/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 b/shark/src/test/hive/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 rename to shark/src/test/hive/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 diff --git a/src/test/hive/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe b/shark/src/test/hive/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe rename to shark/src/test/hive/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe diff --git a/src/test/hive/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b b/shark/src/test/hive/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b rename to shark/src/test/hive/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b diff --git a/src/test/hive/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 b/shark/src/test/hive/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 rename to shark/src/test/hive/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 diff --git a/src/test/hive/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e b/shark/src/test/hive/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e rename to shark/src/test/hive/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e diff --git a/src/test/hive/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad b/shark/src/test/hive/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad rename to shark/src/test/hive/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad diff --git a/src/test/hive/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc b/shark/src/test/hive/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc rename to shark/src/test/hive/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc diff --git a/src/test/hive/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc b/shark/src/test/hive/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc rename to shark/src/test/hive/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc diff --git a/src/test/hive/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a b/shark/src/test/hive/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a rename to shark/src/test/hive/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a diff --git a/src/test/hive/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 b/shark/src/test/hive/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 rename to shark/src/test/hive/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 diff --git a/src/test/hive/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 b/shark/src/test/hive/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 rename to shark/src/test/hive/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 diff --git a/src/test/hive/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 b/shark/src/test/hive/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 rename to shark/src/test/hive/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 diff --git a/src/test/hive/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b b/shark/src/test/hive/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b rename to shark/src/test/hive/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b diff --git a/src/test/hive/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb b/shark/src/test/hive/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb rename to shark/src/test/hive/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb diff --git a/src/test/hive/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad b/shark/src/test/hive/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad similarity index 100% rename from src/test/hive/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad rename to shark/src/test/hive/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad diff --git a/src/test/hive/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 b/shark/src/test/hive/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 rename to shark/src/test/hive/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 diff --git a/src/test/hive/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de b/shark/src/test/hive/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de rename to shark/src/test/hive/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de diff --git a/src/test/hive/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 b/shark/src/test/hive/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 rename to shark/src/test/hive/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 diff --git a/src/test/hive/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 b/shark/src/test/hive/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 rename to shark/src/test/hive/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 diff --git a/src/test/hive/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed b/shark/src/test/hive/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed rename to shark/src/test/hive/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed diff --git a/src/test/hive/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef b/shark/src/test/hive/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef rename to shark/src/test/hive/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef diff --git a/src/test/hive/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f b/shark/src/test/hive/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f rename to shark/src/test/hive/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f diff --git a/src/test/hive/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb b/shark/src/test/hive/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb rename to shark/src/test/hive/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb diff --git a/src/test/hive/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 b/shark/src/test/hive/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 rename to shark/src/test/hive/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 diff --git a/src/test/hive/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d b/shark/src/test/hive/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d rename to shark/src/test/hive/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d diff --git a/src/test/hive/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 b/shark/src/test/hive/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 rename to shark/src/test/hive/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 diff --git a/src/test/hive/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 b/shark/src/test/hive/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 rename to shark/src/test/hive/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 diff --git a/src/test/hive/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 b/shark/src/test/hive/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 rename to shark/src/test/hive/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 diff --git a/src/test/hive/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 b/shark/src/test/hive/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 rename to shark/src/test/hive/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 diff --git a/src/test/hive/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 b/shark/src/test/hive/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 rename to shark/src/test/hive/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 diff --git a/src/test/hive/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff b/shark/src/test/hive/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff rename to shark/src/test/hive/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff diff --git a/src/test/hive/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 b/shark/src/test/hive/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 rename to shark/src/test/hive/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 diff --git a/src/test/hive/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 b/shark/src/test/hive/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 rename to shark/src/test/hive/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 diff --git a/src/test/hive/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a b/shark/src/test/hive/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a rename to shark/src/test/hive/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a diff --git a/src/test/hive/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec b/shark/src/test/hive/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec rename to shark/src/test/hive/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec diff --git a/src/test/hive/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f b/shark/src/test/hive/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f rename to shark/src/test/hive/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f diff --git a/src/test/hive/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a b/shark/src/test/hive/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a rename to shark/src/test/hive/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a diff --git a/src/test/hive/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec b/shark/src/test/hive/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec rename to shark/src/test/hive/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec diff --git a/src/test/hive/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 b/shark/src/test/hive/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 rename to shark/src/test/hive/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 diff --git a/src/test/hive/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 b/shark/src/test/hive/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 rename to shark/src/test/hive/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 diff --git a/src/test/hive/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 b/shark/src/test/hive/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 rename to shark/src/test/hive/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 diff --git a/src/test/hive/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f b/shark/src/test/hive/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f rename to shark/src/test/hive/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f diff --git a/src/test/hive/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e b/shark/src/test/hive/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e rename to shark/src/test/hive/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e diff --git a/src/test/hive/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 b/shark/src/test/hive/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 rename to shark/src/test/hive/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 diff --git a/src/test/hive/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a b/shark/src/test/hive/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a rename to shark/src/test/hive/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a diff --git a/src/test/hive/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe b/shark/src/test/hive/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe rename to shark/src/test/hive/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe diff --git a/src/test/hive/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 b/shark/src/test/hive/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 rename to shark/src/test/hive/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 diff --git a/src/test/hive/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 b/shark/src/test/hive/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 rename to shark/src/test/hive/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 diff --git a/src/test/hive/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f b/shark/src/test/hive/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f rename to shark/src/test/hive/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f diff --git a/src/test/hive/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd b/shark/src/test/hive/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd rename to shark/src/test/hive/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd diff --git a/src/test/hive/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 b/shark/src/test/hive/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 rename to shark/src/test/hive/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 diff --git a/src/test/hive/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 b/shark/src/test/hive/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 rename to shark/src/test/hive/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 diff --git a/src/test/hive/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 b/shark/src/test/hive/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 rename to shark/src/test/hive/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 diff --git a/src/test/hive/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 b/shark/src/test/hive/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 rename to shark/src/test/hive/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 diff --git a/src/test/hive/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 b/shark/src/test/hive/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 rename to shark/src/test/hive/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 diff --git a/src/test/hive/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de b/shark/src/test/hive/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de rename to shark/src/test/hive/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de diff --git a/src/test/hive/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 b/shark/src/test/hive/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 rename to shark/src/test/hive/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 diff --git a/src/test/hive/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af b/shark/src/test/hive/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af similarity index 100% rename from src/test/hive/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af rename to shark/src/test/hive/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af diff --git a/src/test/hive/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a b/shark/src/test/hive/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a rename to shark/src/test/hive/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a diff --git a/src/test/hive/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec b/shark/src/test/hive/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec rename to shark/src/test/hive/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec diff --git a/src/test/hive/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 b/shark/src/test/hive/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 rename to shark/src/test/hive/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 diff --git a/src/test/hive/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 b/shark/src/test/hive/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 rename to shark/src/test/hive/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 diff --git a/src/test/hive/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 b/shark/src/test/hive/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 rename to shark/src/test/hive/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 diff --git a/src/test/hive/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 b/shark/src/test/hive/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 rename to shark/src/test/hive/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 diff --git a/src/test/hive/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d b/shark/src/test/hive/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d rename to shark/src/test/hive/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d diff --git a/src/test/hive/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 b/shark/src/test/hive/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 rename to shark/src/test/hive/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 diff --git a/src/test/hive/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 b/shark/src/test/hive/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 rename to shark/src/test/hive/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 diff --git a/src/test/hive/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 b/shark/src/test/hive/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 rename to shark/src/test/hive/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 diff --git a/src/test/hive/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 b/shark/src/test/hive/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 rename to shark/src/test/hive/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 diff --git a/src/test/hive/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 b/shark/src/test/hive/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 rename to shark/src/test/hive/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 diff --git a/src/test/hive/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f b/shark/src/test/hive/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f rename to shark/src/test/hive/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f diff --git a/src/test/hive/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 b/shark/src/test/hive/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 rename to shark/src/test/hive/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 diff --git a/src/test/hive/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 b/shark/src/test/hive/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 rename to shark/src/test/hive/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 diff --git a/src/test/hive/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 b/shark/src/test/hive/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 rename to shark/src/test/hive/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 diff --git a/src/test/hive/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 b/shark/src/test/hive/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 rename to shark/src/test/hive/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 diff --git a/src/test/hive/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 b/shark/src/test/hive/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 rename to shark/src/test/hive/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 diff --git a/src/test/hive/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 b/shark/src/test/hive/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 rename to shark/src/test/hive/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 diff --git a/src/test/hive/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e b/shark/src/test/hive/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e rename to shark/src/test/hive/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e diff --git a/src/test/hive/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef b/shark/src/test/hive/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef rename to shark/src/test/hive/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef diff --git a/src/test/hive/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 b/shark/src/test/hive/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 rename to shark/src/test/hive/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 diff --git a/src/test/hive/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb b/shark/src/test/hive/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb rename to shark/src/test/hive/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb diff --git a/src/test/hive/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 b/shark/src/test/hive/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 rename to shark/src/test/hive/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 diff --git a/src/test/hive/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f b/shark/src/test/hive/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f rename to shark/src/test/hive/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f diff --git a/src/test/hive/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 b/shark/src/test/hive/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 rename to shark/src/test/hive/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 diff --git a/src/test/hive/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 b/shark/src/test/hive/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 rename to shark/src/test/hive/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 diff --git a/src/test/hive/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 b/shark/src/test/hive/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 rename to shark/src/test/hive/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 diff --git a/src/test/hive/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 b/shark/src/test/hive/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 rename to shark/src/test/hive/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 diff --git a/src/test/hive/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de b/shark/src/test/hive/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de rename to shark/src/test/hive/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de diff --git a/src/test/hive/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 b/shark/src/test/hive/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 rename to shark/src/test/hive/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 diff --git a/src/test/hive/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db b/shark/src/test/hive/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db similarity index 100% rename from src/test/hive/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db rename to shark/src/test/hive/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db diff --git a/src/test/hive/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a b/shark/src/test/hive/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a rename to shark/src/test/hive/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a diff --git a/src/test/hive/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec b/shark/src/test/hive/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec rename to shark/src/test/hive/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec diff --git a/src/test/hive/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc b/shark/src/test/hive/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc rename to shark/src/test/hive/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc diff --git a/src/test/hive/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 b/shark/src/test/hive/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 rename to shark/src/test/hive/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 diff --git a/src/test/hive/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 b/shark/src/test/hive/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 rename to shark/src/test/hive/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 diff --git a/src/test/hive/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a b/shark/src/test/hive/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a rename to shark/src/test/hive/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a diff --git a/src/test/hive/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 b/shark/src/test/hive/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 rename to shark/src/test/hive/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 diff --git a/src/test/hive/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 b/shark/src/test/hive/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 rename to shark/src/test/hive/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 diff --git a/src/test/hive/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 b/shark/src/test/hive/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 rename to shark/src/test/hive/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 diff --git a/src/test/hive/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f b/shark/src/test/hive/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f rename to shark/src/test/hive/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f diff --git a/src/test/hive/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc b/shark/src/test/hive/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc rename to shark/src/test/hive/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc diff --git a/src/test/hive/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a b/shark/src/test/hive/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a rename to shark/src/test/hive/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a diff --git a/src/test/hive/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f b/shark/src/test/hive/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f rename to shark/src/test/hive/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f diff --git a/src/test/hive/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d b/shark/src/test/hive/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d rename to shark/src/test/hive/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d diff --git a/src/test/hive/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 b/shark/src/test/hive/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 rename to shark/src/test/hive/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 diff --git a/src/test/hive/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a b/shark/src/test/hive/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a rename to shark/src/test/hive/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a diff --git a/src/test/hive/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a b/shark/src/test/hive/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a rename to shark/src/test/hive/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a diff --git a/src/test/hive/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b b/shark/src/test/hive/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b rename to shark/src/test/hive/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b diff --git a/src/test/hive/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 b/shark/src/test/hive/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 rename to shark/src/test/hive/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 diff --git a/src/test/hive/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 b/shark/src/test/hive/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 rename to shark/src/test/hive/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 diff --git a/src/test/hive/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 b/shark/src/test/hive/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 rename to shark/src/test/hive/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 diff --git a/src/test/hive/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 b/shark/src/test/hive/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 rename to shark/src/test/hive/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 diff --git a/src/test/hive/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 b/shark/src/test/hive/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 rename to shark/src/test/hive/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 diff --git a/src/test/hive/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 b/shark/src/test/hive/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 rename to shark/src/test/hive/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 diff --git a/src/test/hive/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 b/shark/src/test/hive/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 rename to shark/src/test/hive/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 diff --git a/src/test/hive/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 b/shark/src/test/hive/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 rename to shark/src/test/hive/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 diff --git a/src/test/hive/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc b/shark/src/test/hive/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc rename to shark/src/test/hive/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc diff --git a/src/test/hive/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 b/shark/src/test/hive/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 rename to shark/src/test/hive/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 diff --git a/src/test/hive/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 b/shark/src/test/hive/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 rename to shark/src/test/hive/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 diff --git a/src/test/hive/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de b/shark/src/test/hive/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de rename to shark/src/test/hive/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de diff --git a/src/test/hive/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 b/shark/src/test/hive/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 rename to shark/src/test/hive/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 diff --git a/src/test/hive/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df b/shark/src/test/hive/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df similarity index 100% rename from src/test/hive/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df rename to shark/src/test/hive/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df diff --git a/src/test/hive/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 b/shark/src/test/hive/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 rename to shark/src/test/hive/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 diff --git a/src/test/hive/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 b/shark/src/test/hive/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 rename to shark/src/test/hive/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 diff --git a/src/test/hive/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca b/shark/src/test/hive/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca rename to shark/src/test/hive/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca diff --git a/src/test/hive/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 b/shark/src/test/hive/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 rename to shark/src/test/hive/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 diff --git a/src/test/hive/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e b/shark/src/test/hive/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e rename to shark/src/test/hive/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e diff --git a/src/test/hive/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a b/shark/src/test/hive/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a rename to shark/src/test/hive/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a diff --git a/src/test/hive/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a b/shark/src/test/hive/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a rename to shark/src/test/hive/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a diff --git a/src/test/hive/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 b/shark/src/test/hive/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 rename to shark/src/test/hive/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 diff --git a/src/test/hive/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 b/shark/src/test/hive/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 rename to shark/src/test/hive/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 diff --git a/src/test/hive/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca b/shark/src/test/hive/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca rename to shark/src/test/hive/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca diff --git a/src/test/hive/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 b/shark/src/test/hive/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 rename to shark/src/test/hive/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 diff --git a/src/test/hive/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e b/shark/src/test/hive/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e rename to shark/src/test/hive/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e diff --git a/src/test/hive/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a b/shark/src/test/hive/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a rename to shark/src/test/hive/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a diff --git a/src/test/hive/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 b/shark/src/test/hive/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 rename to shark/src/test/hive/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 diff --git a/src/test/hive/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 b/shark/src/test/hive/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 rename to shark/src/test/hive/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 diff --git a/src/test/hive/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 b/shark/src/test/hive/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 rename to shark/src/test/hive/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 diff --git a/src/test/hive/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 b/shark/src/test/hive/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 rename to shark/src/test/hive/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 diff --git a/src/test/hive/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 b/shark/src/test/hive/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 rename to shark/src/test/hive/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 diff --git a/src/test/hive/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 b/shark/src/test/hive/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 rename to shark/src/test/hive/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 diff --git a/src/test/hive/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 b/shark/src/test/hive/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 rename to shark/src/test/hive/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 diff --git a/src/test/hive/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa b/shark/src/test/hive/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa rename to shark/src/test/hive/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa diff --git a/src/test/hive/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c b/shark/src/test/hive/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c rename to shark/src/test/hive/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c diff --git a/src/test/hive/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 b/shark/src/test/hive/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 rename to shark/src/test/hive/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 diff --git a/src/test/hive/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 b/shark/src/test/hive/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 rename to shark/src/test/hive/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 diff --git a/src/test/hive/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 b/shark/src/test/hive/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 rename to shark/src/test/hive/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 diff --git a/src/test/hive/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 b/shark/src/test/hive/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 rename to shark/src/test/hive/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 diff --git a/src/test/hive/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 b/shark/src/test/hive/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 rename to shark/src/test/hive/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 diff --git a/src/test/hive/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 b/shark/src/test/hive/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 rename to shark/src/test/hive/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 diff --git a/src/test/hive/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 b/shark/src/test/hive/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 rename to shark/src/test/hive/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 diff --git a/src/test/hive/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd b/shark/src/test/hive/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd rename to shark/src/test/hive/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd diff --git a/src/test/hive/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a b/shark/src/test/hive/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a rename to shark/src/test/hive/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a diff --git a/src/test/hive/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a b/shark/src/test/hive/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a rename to shark/src/test/hive/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a diff --git a/src/test/hive/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 b/shark/src/test/hive/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 rename to shark/src/test/hive/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 diff --git a/src/test/hive/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e b/shark/src/test/hive/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e rename to shark/src/test/hive/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e diff --git a/src/test/hive/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a b/shark/src/test/hive/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a rename to shark/src/test/hive/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a diff --git a/src/test/hive/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff b/shark/src/test/hive/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff rename to shark/src/test/hive/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff diff --git a/src/test/hive/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa b/shark/src/test/hive/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa rename to shark/src/test/hive/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa diff --git a/src/test/hive/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de b/shark/src/test/hive/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de rename to shark/src/test/hive/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de diff --git a/src/test/hive/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c b/shark/src/test/hive/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c rename to shark/src/test/hive/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c diff --git a/src/test/hive/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 b/shark/src/test/hive/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 rename to shark/src/test/hive/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 diff --git a/src/test/hive/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 b/shark/src/test/hive/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 rename to shark/src/test/hive/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 diff --git a/src/test/hive/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb b/shark/src/test/hive/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb rename to shark/src/test/hive/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb diff --git a/src/test/hive/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f b/shark/src/test/hive/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f rename to shark/src/test/hive/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f diff --git a/src/test/hive/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/shark/src/test/hive/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to shark/src/test/hive/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 diff --git a/src/test/hive/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 b/shark/src/test/hive/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 rename to shark/src/test/hive/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 diff --git a/src/test/hive/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 b/shark/src/test/hive/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 rename to shark/src/test/hive/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 diff --git a/src/test/hive/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b b/shark/src/test/hive/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b rename to shark/src/test/hive/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b diff --git a/src/test/hive/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 b/shark/src/test/hive/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 rename to shark/src/test/hive/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 diff --git a/src/test/hive/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 b/shark/src/test/hive/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 rename to shark/src/test/hive/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 diff --git a/src/test/hive/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 b/shark/src/test/hive/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 rename to shark/src/test/hive/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 diff --git a/src/test/hive/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b b/shark/src/test/hive/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b rename to shark/src/test/hive/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b diff --git a/src/test/hive/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 b/shark/src/test/hive/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 rename to shark/src/test/hive/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 diff --git a/src/test/hive/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 b/shark/src/test/hive/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 rename to shark/src/test/hive/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 diff --git a/src/test/hive/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 b/shark/src/test/hive/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 rename to shark/src/test/hive/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 diff --git a/src/test/hive/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b b/shark/src/test/hive/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b rename to shark/src/test/hive/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b diff --git a/src/test/hive/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c b/shark/src/test/hive/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c rename to shark/src/test/hive/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c diff --git a/src/test/hive/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 b/shark/src/test/hive/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 rename to shark/src/test/hive/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 diff --git a/src/test/hive/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 b/shark/src/test/hive/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 rename to shark/src/test/hive/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 diff --git a/src/test/hive/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b b/shark/src/test/hive/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b rename to shark/src/test/hive/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b diff --git a/src/test/hive/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c b/shark/src/test/hive/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c rename to shark/src/test/hive/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c diff --git a/src/test/hive/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 b/shark/src/test/hive/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 rename to shark/src/test/hive/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 diff --git a/src/test/hive/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e b/shark/src/test/hive/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e rename to shark/src/test/hive/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e diff --git a/src/test/hive/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a b/shark/src/test/hive/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a rename to shark/src/test/hive/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a diff --git a/src/test/hive/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c b/shark/src/test/hive/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c rename to shark/src/test/hive/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c diff --git a/src/test/hive/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 b/shark/src/test/hive/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 rename to shark/src/test/hive/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 diff --git a/src/test/hive/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f b/shark/src/test/hive/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f rename to shark/src/test/hive/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f diff --git a/src/test/hive/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a b/shark/src/test/hive/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a rename to shark/src/test/hive/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a diff --git a/src/test/hive/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 b/shark/src/test/hive/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 rename to shark/src/test/hive/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 diff --git a/src/test/hive/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 b/shark/src/test/hive/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 rename to shark/src/test/hive/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 diff --git a/src/test/hive/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 b/shark/src/test/hive/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 rename to shark/src/test/hive/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 diff --git a/src/test/hive/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a b/shark/src/test/hive/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a rename to shark/src/test/hive/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a diff --git a/src/test/hive/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 b/shark/src/test/hive/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 rename to shark/src/test/hive/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 diff --git a/src/test/hive/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b b/shark/src/test/hive/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b rename to shark/src/test/hive/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b diff --git a/src/test/hive/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c b/shark/src/test/hive/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c rename to shark/src/test/hive/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c diff --git a/src/test/hive/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 b/shark/src/test/hive/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 rename to shark/src/test/hive/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 diff --git a/src/test/hive/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a b/shark/src/test/hive/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a rename to shark/src/test/hive/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a diff --git a/src/test/hive/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 b/shark/src/test/hive/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 rename to shark/src/test/hive/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 diff --git a/src/test/hive/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c b/shark/src/test/hive/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c rename to shark/src/test/hive/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c diff --git a/src/test/hive/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 b/shark/src/test/hive/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 rename to shark/src/test/hive/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 diff --git a/src/test/hive/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a b/shark/src/test/hive/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a rename to shark/src/test/hive/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a diff --git a/src/test/hive/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 b/shark/src/test/hive/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 rename to shark/src/test/hive/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 diff --git a/src/test/hive/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a b/shark/src/test/hive/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a rename to shark/src/test/hive/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a diff --git a/src/test/hive/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 b/shark/src/test/hive/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 rename to shark/src/test/hive/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 diff --git a/src/test/hive/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a b/shark/src/test/hive/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a rename to shark/src/test/hive/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a diff --git a/src/test/hive/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 b/shark/src/test/hive/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 similarity index 100% rename from src/test/hive/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 rename to shark/src/test/hive/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 diff --git a/src/test/hive/golden/sort-0-eee35bbc888a705ae527625447668032 b/shark/src/test/hive/golden/sort-0-eee35bbc888a705ae527625447668032 similarity index 100% rename from src/test/hive/golden/sort-0-eee35bbc888a705ae527625447668032 rename to shark/src/test/hive/golden/sort-0-eee35bbc888a705ae527625447668032 diff --git a/src/test/hive/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 b/shark/src/test/hive/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 similarity index 100% rename from src/test/hive/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 rename to shark/src/test/hive/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 diff --git a/src/test/hive/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 b/shark/src/test/hive/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 rename to shark/src/test/hive/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 diff --git a/src/test/hive/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c b/shark/src/test/hive/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c rename to shark/src/test/hive/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c diff --git a/src/test/hive/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 b/shark/src/test/hive/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 rename to shark/src/test/hive/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 diff --git a/src/test/hive/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 b/shark/src/test/hive/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 rename to shark/src/test/hive/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 diff --git a/src/test/hive/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 b/shark/src/test/hive/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 rename to shark/src/test/hive/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 diff --git a/src/test/hive/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e b/shark/src/test/hive/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e rename to shark/src/test/hive/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e diff --git a/src/test/hive/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 b/shark/src/test/hive/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 rename to shark/src/test/hive/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 diff --git a/src/test/hive/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 b/shark/src/test/hive/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 rename to shark/src/test/hive/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 diff --git a/src/test/hive/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 b/shark/src/test/hive/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 rename to shark/src/test/hive/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 diff --git a/src/test/hive/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c b/shark/src/test/hive/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c rename to shark/src/test/hive/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c diff --git a/src/test/hive/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 b/shark/src/test/hive/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 rename to shark/src/test/hive/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 diff --git a/src/test/hive/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd b/shark/src/test/hive/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd rename to shark/src/test/hive/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd diff --git a/src/test/hive/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 b/shark/src/test/hive/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 rename to shark/src/test/hive/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 diff --git a/src/test/hive/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 b/shark/src/test/hive/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 rename to shark/src/test/hive/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 diff --git a/src/test/hive/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 b/shark/src/test/hive/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 rename to shark/src/test/hive/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 diff --git a/src/test/hive/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 b/shark/src/test/hive/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 rename to shark/src/test/hive/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 diff --git a/src/test/hive/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 b/shark/src/test/hive/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 rename to shark/src/test/hive/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 diff --git a/src/test/hive/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c b/shark/src/test/hive/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c rename to shark/src/test/hive/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c diff --git a/src/test/hive/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 b/shark/src/test/hive/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 rename to shark/src/test/hive/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 diff --git a/src/test/hive/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd b/shark/src/test/hive/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd rename to shark/src/test/hive/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd diff --git a/src/test/hive/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 b/shark/src/test/hive/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 rename to shark/src/test/hive/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 diff --git a/src/test/hive/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 b/shark/src/test/hive/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 rename to shark/src/test/hive/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 diff --git a/src/test/hive/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 b/shark/src/test/hive/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 rename to shark/src/test/hive/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 diff --git a/src/test/hive/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 b/shark/src/test/hive/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 rename to shark/src/test/hive/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 diff --git a/src/test/hive/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 b/shark/src/test/hive/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 rename to shark/src/test/hive/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 diff --git a/src/test/hive/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c b/shark/src/test/hive/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c rename to shark/src/test/hive/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c diff --git a/src/test/hive/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 b/shark/src/test/hive/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 rename to shark/src/test/hive/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 diff --git a/src/test/hive/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd b/shark/src/test/hive/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd rename to shark/src/test/hive/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd diff --git a/src/test/hive/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 b/shark/src/test/hive/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 rename to shark/src/test/hive/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 diff --git a/src/test/hive/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 b/shark/src/test/hive/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 rename to shark/src/test/hive/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 diff --git a/src/test/hive/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 b/shark/src/test/hive/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 rename to shark/src/test/hive/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 diff --git a/src/test/hive/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 b/shark/src/test/hive/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 rename to shark/src/test/hive/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 diff --git a/src/test/hive/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 b/shark/src/test/hive/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 rename to shark/src/test/hive/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 diff --git a/src/test/hive/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f b/shark/src/test/hive/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f rename to shark/src/test/hive/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f diff --git a/src/test/hive/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a b/shark/src/test/hive/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a rename to shark/src/test/hive/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a diff --git a/src/test/hive/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb b/shark/src/test/hive/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb rename to shark/src/test/hive/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb diff --git a/src/test/hive/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d b/shark/src/test/hive/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d rename to shark/src/test/hive/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d diff --git a/src/test/hive/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 b/shark/src/test/hive/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 rename to shark/src/test/hive/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 diff --git a/src/test/hive/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf b/shark/src/test/hive/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf rename to shark/src/test/hive/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf diff --git a/src/test/hive/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 b/shark/src/test/hive/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 rename to shark/src/test/hive/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 diff --git a/src/test/hive/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed b/shark/src/test/hive/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed rename to shark/src/test/hive/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed diff --git a/src/test/hive/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 b/shark/src/test/hive/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 rename to shark/src/test/hive/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 diff --git a/src/test/hive/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 b/shark/src/test/hive/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 rename to shark/src/test/hive/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 diff --git a/src/test/hive/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a b/shark/src/test/hive/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a rename to shark/src/test/hive/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a diff --git a/src/test/hive/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b b/shark/src/test/hive/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b rename to shark/src/test/hive/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b diff --git a/src/test/hive/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d b/shark/src/test/hive/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d rename to shark/src/test/hive/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d diff --git a/src/test/hive/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 b/shark/src/test/hive/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 rename to shark/src/test/hive/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 diff --git a/src/test/hive/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf b/shark/src/test/hive/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf rename to shark/src/test/hive/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf diff --git a/src/test/hive/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 b/shark/src/test/hive/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 rename to shark/src/test/hive/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 diff --git a/src/test/hive/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed b/shark/src/test/hive/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed rename to shark/src/test/hive/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed diff --git a/src/test/hive/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e b/shark/src/test/hive/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e rename to shark/src/test/hive/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/src/test/hive/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 b/shark/src/test/hive/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 rename to shark/src/test/hive/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 b/shark/src/test/hive/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 rename to shark/src/test/hive/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 b/shark/src/test/hive/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 rename to shark/src/test/hive/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed b/shark/src/test/hive/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed rename to shark/src/test/hive/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed diff --git a/src/test/hive/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 b/shark/src/test/hive/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 rename to shark/src/test/hive/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 b/shark/src/test/hive/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 rename to shark/src/test/hive/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 b/shark/src/test/hive/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 rename to shark/src/test/hive/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 b/shark/src/test/hive/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 rename to shark/src/test/hive/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b b/shark/src/test/hive/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b rename to shark/src/test/hive/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b diff --git a/src/test/hive/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d b/shark/src/test/hive/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d rename to shark/src/test/hive/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d diff --git a/src/test/hive/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 b/shark/src/test/hive/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 rename to shark/src/test/hive/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 b/shark/src/test/hive/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 rename to shark/src/test/hive/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 diff --git a/src/test/hive/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f b/shark/src/test/hive/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f rename to shark/src/test/hive/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f diff --git a/src/test/hive/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf b/shark/src/test/hive/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf rename to shark/src/test/hive/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf diff --git a/src/test/hive/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec b/shark/src/test/hive/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec similarity index 100% rename from src/test/hive/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec rename to shark/src/test/hive/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec diff --git a/src/test/hive/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 b/shark/src/test/hive/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 similarity index 100% rename from src/test/hive/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 rename to shark/src/test/hive/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 diff --git a/src/test/hive/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c b/shark/src/test/hive/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c similarity index 100% rename from src/test/hive/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c rename to shark/src/test/hive/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c diff --git a/src/test/hive/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 b/shark/src/test/hive/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 similarity index 100% rename from src/test/hive/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 rename to shark/src/test/hive/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 diff --git a/src/test/hive/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 b/shark/src/test/hive/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 similarity index 100% rename from src/test/hive/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 rename to shark/src/test/hive/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 diff --git a/src/test/hive/golden/stats0-13-ca0569fd4200d549843deadf84044649 b/shark/src/test/hive/golden/stats0-13-ca0569fd4200d549843deadf84044649 similarity index 100% rename from src/test/hive/golden/stats0-13-ca0569fd4200d549843deadf84044649 rename to shark/src/test/hive/golden/stats0-13-ca0569fd4200d549843deadf84044649 diff --git a/src/test/hive/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 b/shark/src/test/hive/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 similarity index 100% rename from src/test/hive/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 rename to shark/src/test/hive/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 diff --git a/src/test/hive/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c b/shark/src/test/hive/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c similarity index 100% rename from src/test/hive/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c rename to shark/src/test/hive/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c diff --git a/src/test/hive/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 b/shark/src/test/hive/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 similarity index 100% rename from src/test/hive/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 rename to shark/src/test/hive/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 diff --git a/src/test/hive/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed b/shark/src/test/hive/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed similarity index 100% rename from src/test/hive/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed rename to shark/src/test/hive/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed diff --git a/src/test/hive/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/stats0-20-5f7439f97275da70f633e135205f2095 b/shark/src/test/hive/golden/stats0-20-5f7439f97275da70f633e135205f2095 similarity index 100% rename from src/test/hive/golden/stats0-20-5f7439f97275da70f633e135205f2095 rename to shark/src/test/hive/golden/stats0-20-5f7439f97275da70f633e135205f2095 diff --git a/src/test/hive/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a b/shark/src/test/hive/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a similarity index 100% rename from src/test/hive/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a rename to shark/src/test/hive/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a diff --git a/src/test/hive/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 b/shark/src/test/hive/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 similarity index 100% rename from src/test/hive/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 rename to shark/src/test/hive/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 diff --git a/src/test/hive/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c b/shark/src/test/hive/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c similarity index 100% rename from src/test/hive/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c rename to shark/src/test/hive/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c diff --git a/src/test/hive/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 b/shark/src/test/hive/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 similarity index 100% rename from src/test/hive/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 rename to shark/src/test/hive/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 diff --git a/src/test/hive/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b b/shark/src/test/hive/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b similarity index 100% rename from src/test/hive/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b rename to shark/src/test/hive/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b diff --git a/src/test/hive/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c b/shark/src/test/hive/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c similarity index 100% rename from src/test/hive/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c rename to shark/src/test/hive/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c diff --git a/src/test/hive/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 b/shark/src/test/hive/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 similarity index 100% rename from src/test/hive/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 rename to shark/src/test/hive/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 diff --git a/src/test/hive/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 b/shark/src/test/hive/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 similarity index 100% rename from src/test/hive/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 rename to shark/src/test/hive/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 diff --git a/src/test/hive/golden/stats0-29-ca0569fd4200d549843deadf84044649 b/shark/src/test/hive/golden/stats0-29-ca0569fd4200d549843deadf84044649 similarity index 100% rename from src/test/hive/golden/stats0-29-ca0569fd4200d549843deadf84044649 rename to shark/src/test/hive/golden/stats0-29-ca0569fd4200d549843deadf84044649 diff --git a/src/test/hive/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 b/shark/src/test/hive/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 similarity index 100% rename from src/test/hive/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 rename to shark/src/test/hive/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 diff --git a/src/test/hive/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c b/shark/src/test/hive/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c similarity index 100% rename from src/test/hive/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c rename to shark/src/test/hive/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c diff --git a/src/test/hive/golden/stats0-4-5f7439f97275da70f633e135205f2095 b/shark/src/test/hive/golden/stats0-4-5f7439f97275da70f633e135205f2095 similarity index 100% rename from src/test/hive/golden/stats0-4-5f7439f97275da70f633e135205f2095 rename to shark/src/test/hive/golden/stats0-4-5f7439f97275da70f633e135205f2095 diff --git a/src/test/hive/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a b/shark/src/test/hive/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a similarity index 100% rename from src/test/hive/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a rename to shark/src/test/hive/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a diff --git a/src/test/hive/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 b/shark/src/test/hive/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 similarity index 100% rename from src/test/hive/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 rename to shark/src/test/hive/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 diff --git a/src/test/hive/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c b/shark/src/test/hive/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c similarity index 100% rename from src/test/hive/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c rename to shark/src/test/hive/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c diff --git a/src/test/hive/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 b/shark/src/test/hive/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 similarity index 100% rename from src/test/hive/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 rename to shark/src/test/hive/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 diff --git a/src/test/hive/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b b/shark/src/test/hive/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b similarity index 100% rename from src/test/hive/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b rename to shark/src/test/hive/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b diff --git a/src/test/hive/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e b/shark/src/test/hive/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e similarity index 100% rename from src/test/hive/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e rename to shark/src/test/hive/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e diff --git a/src/test/hive/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b b/shark/src/test/hive/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b similarity index 100% rename from src/test/hive/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b rename to shark/src/test/hive/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b diff --git a/src/test/hive/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/stats2-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/stats2-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/stats2-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/stats2-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 b/shark/src/test/hive/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 similarity index 100% rename from src/test/hive/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 rename to shark/src/test/hive/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 diff --git a/src/test/hive/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 b/shark/src/test/hive/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 similarity index 100% rename from src/test/hive/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 rename to shark/src/test/hive/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 diff --git a/src/test/hive/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 b/shark/src/test/hive/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 similarity index 100% rename from src/test/hive/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 rename to shark/src/test/hive/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 diff --git a/src/test/hive/golden/stats2-8-72621dba638b15d244850018e9f64d7 b/shark/src/test/hive/golden/stats2-8-72621dba638b15d244850018e9f64d7 similarity index 100% rename from src/test/hive/golden/stats2-8-72621dba638b15d244850018e9f64d7 rename to shark/src/test/hive/golden/stats2-8-72621dba638b15d244850018e9f64d7 diff --git a/src/test/hive/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e b/shark/src/test/hive/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e similarity index 100% rename from src/test/hive/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e rename to shark/src/test/hive/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e diff --git a/src/test/hive/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 b/shark/src/test/hive/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 similarity index 100% rename from src/test/hive/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 rename to shark/src/test/hive/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 diff --git a/src/test/hive/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 b/shark/src/test/hive/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 similarity index 100% rename from src/test/hive/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 rename to shark/src/test/hive/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 diff --git a/src/test/hive/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b b/shark/src/test/hive/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b similarity index 100% rename from src/test/hive/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b rename to shark/src/test/hive/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b diff --git a/src/test/hive/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 b/shark/src/test/hive/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 similarity index 100% rename from src/test/hive/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 rename to shark/src/test/hive/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 diff --git a/src/test/hive/golden/stats20-5-76509775cfe11bb98ee088188a07668a b/shark/src/test/hive/golden/stats20-5-76509775cfe11bb98ee088188a07668a similarity index 100% rename from src/test/hive/golden/stats20-5-76509775cfe11bb98ee088188a07668a rename to shark/src/test/hive/golden/stats20-5-76509775cfe11bb98ee088188a07668a diff --git a/src/test/hive/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 b/shark/src/test/hive/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 similarity index 100% rename from src/test/hive/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 rename to shark/src/test/hive/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 diff --git a/src/test/hive/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 b/shark/src/test/hive/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 similarity index 100% rename from src/test/hive/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 rename to shark/src/test/hive/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 diff --git a/src/test/hive/golden/stats20-8-300c971de74642118d36d36349bc81aa b/shark/src/test/hive/golden/stats20-8-300c971de74642118d36d36349bc81aa similarity index 100% rename from src/test/hive/golden/stats20-8-300c971de74642118d36d36349bc81aa rename to shark/src/test/hive/golden/stats20-8-300c971de74642118d36d36349bc81aa diff --git a/src/test/hive/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 b/shark/src/test/hive/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 similarity index 100% rename from src/test/hive/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 rename to shark/src/test/hive/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 diff --git a/src/test/hive/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 b/shark/src/test/hive/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 similarity index 100% rename from src/test/hive/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 rename to shark/src/test/hive/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 diff --git a/src/test/hive/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/shark/src/test/hive/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa similarity index 100% rename from src/test/hive/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa rename to shark/src/test/hive/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa diff --git a/src/test/hive/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc b/shark/src/test/hive/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc similarity index 100% rename from src/test/hive/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc rename to shark/src/test/hive/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc diff --git a/src/test/hive/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 b/shark/src/test/hive/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 similarity index 100% rename from src/test/hive/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 rename to shark/src/test/hive/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 diff --git a/src/test/hive/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/shark/src/test/hive/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa similarity index 100% rename from src/test/hive/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa rename to shark/src/test/hive/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa diff --git a/src/test/hive/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 b/shark/src/test/hive/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 similarity index 100% rename from src/test/hive/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 rename to shark/src/test/hive/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 diff --git a/src/test/hive/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 b/shark/src/test/hive/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 similarity index 100% rename from src/test/hive/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 rename to shark/src/test/hive/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 diff --git a/src/test/hive/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 b/shark/src/test/hive/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 similarity index 100% rename from src/test/hive/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 rename to shark/src/test/hive/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 diff --git a/src/test/hive/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b b/shark/src/test/hive/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b similarity index 100% rename from src/test/hive/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b rename to shark/src/test/hive/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b diff --git a/src/test/hive/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 b/shark/src/test/hive/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 similarity index 100% rename from src/test/hive/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 rename to shark/src/test/hive/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 diff --git a/src/test/hive/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 b/shark/src/test/hive/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 similarity index 100% rename from src/test/hive/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 rename to shark/src/test/hive/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 diff --git a/src/test/hive/golden/stats3-20-ca048ad81b3df7159822073d206f0790 b/shark/src/test/hive/golden/stats3-20-ca048ad81b3df7159822073d206f0790 similarity index 100% rename from src/test/hive/golden/stats3-20-ca048ad81b3df7159822073d206f0790 rename to shark/src/test/hive/golden/stats3-20-ca048ad81b3df7159822073d206f0790 diff --git a/src/test/hive/golden/stats3-3-ca048ad81b3df7159822073d206f0790 b/shark/src/test/hive/golden/stats3-3-ca048ad81b3df7159822073d206f0790 similarity index 100% rename from src/test/hive/golden/stats3-3-ca048ad81b3df7159822073d206f0790 rename to shark/src/test/hive/golden/stats3-3-ca048ad81b3df7159822073d206f0790 diff --git a/src/test/hive/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 b/shark/src/test/hive/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 similarity index 100% rename from src/test/hive/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 rename to shark/src/test/hive/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 diff --git a/src/test/hive/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 b/shark/src/test/hive/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 similarity index 100% rename from src/test/hive/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 rename to shark/src/test/hive/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 diff --git a/src/test/hive/golden/stats3-6-4bf1504274319c44d370b58092fe016c b/shark/src/test/hive/golden/stats3-6-4bf1504274319c44d370b58092fe016c similarity index 100% rename from src/test/hive/golden/stats3-6-4bf1504274319c44d370b58092fe016c rename to shark/src/test/hive/golden/stats3-6-4bf1504274319c44d370b58092fe016c diff --git a/src/test/hive/golden/stats3-7-73d7d55d6e5a57aacce8618902904d b/shark/src/test/hive/golden/stats3-7-73d7d55d6e5a57aacce8618902904d similarity index 100% rename from src/test/hive/golden/stats3-7-73d7d55d6e5a57aacce8618902904d rename to shark/src/test/hive/golden/stats3-7-73d7d55d6e5a57aacce8618902904d diff --git a/src/test/hive/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 b/shark/src/test/hive/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 similarity index 100% rename from src/test/hive/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 rename to shark/src/test/hive/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 diff --git a/src/test/hive/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 b/shark/src/test/hive/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 similarity index 100% rename from src/test/hive/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 rename to shark/src/test/hive/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 diff --git a/src/test/hive/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 b/shark/src/test/hive/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 similarity index 100% rename from src/test/hive/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 rename to shark/src/test/hive/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 diff --git a/src/test/hive/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 b/shark/src/test/hive/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 similarity index 100% rename from src/test/hive/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 rename to shark/src/test/hive/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 diff --git a/src/test/hive/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 b/shark/src/test/hive/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 similarity index 100% rename from src/test/hive/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 rename to shark/src/test/hive/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 diff --git a/src/test/hive/golden/stats4-12-30bc31441828a053d1a675b225a5d617 b/shark/src/test/hive/golden/stats4-12-30bc31441828a053d1a675b225a5d617 similarity index 100% rename from src/test/hive/golden/stats4-12-30bc31441828a053d1a675b225a5d617 rename to shark/src/test/hive/golden/stats4-12-30bc31441828a053d1a675b225a5d617 diff --git a/src/test/hive/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 b/shark/src/test/hive/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 similarity index 100% rename from src/test/hive/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 rename to shark/src/test/hive/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 diff --git a/src/test/hive/golden/stats4-14-9c82167763a771c175c656786d545798 b/shark/src/test/hive/golden/stats4-14-9c82167763a771c175c656786d545798 similarity index 100% rename from src/test/hive/golden/stats4-14-9c82167763a771c175c656786d545798 rename to shark/src/test/hive/golden/stats4-14-9c82167763a771c175c656786d545798 diff --git a/src/test/hive/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c b/shark/src/test/hive/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c similarity index 100% rename from src/test/hive/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c rename to shark/src/test/hive/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c diff --git a/src/test/hive/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 b/shark/src/test/hive/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 similarity index 100% rename from src/test/hive/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 rename to shark/src/test/hive/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 diff --git a/src/test/hive/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 b/shark/src/test/hive/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 similarity index 100% rename from src/test/hive/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 rename to shark/src/test/hive/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 diff --git a/src/test/hive/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 b/shark/src/test/hive/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 similarity index 100% rename from src/test/hive/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 rename to shark/src/test/hive/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 diff --git a/src/test/hive/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 b/shark/src/test/hive/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 similarity index 100% rename from src/test/hive/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 rename to shark/src/test/hive/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 diff --git a/src/test/hive/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 b/shark/src/test/hive/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 similarity index 100% rename from src/test/hive/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 rename to shark/src/test/hive/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 diff --git a/src/test/hive/golden/stats4-20-f63000f2c395b935199c9829964f98c1 b/shark/src/test/hive/golden/stats4-20-f63000f2c395b935199c9829964f98c1 similarity index 100% rename from src/test/hive/golden/stats4-20-f63000f2c395b935199c9829964f98c1 rename to shark/src/test/hive/golden/stats4-20-f63000f2c395b935199c9829964f98c1 diff --git a/src/test/hive/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd b/shark/src/test/hive/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd similarity index 100% rename from src/test/hive/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd rename to shark/src/test/hive/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd diff --git a/src/test/hive/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c b/shark/src/test/hive/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c similarity index 100% rename from src/test/hive/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c rename to shark/src/test/hive/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c diff --git a/src/test/hive/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd b/shark/src/test/hive/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd similarity index 100% rename from src/test/hive/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd rename to shark/src/test/hive/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd diff --git a/src/test/hive/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c b/shark/src/test/hive/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c similarity index 100% rename from src/test/hive/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c rename to shark/src/test/hive/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c diff --git a/src/test/hive/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d b/shark/src/test/hive/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d similarity index 100% rename from src/test/hive/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d rename to shark/src/test/hive/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d diff --git a/src/test/hive/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 b/shark/src/test/hive/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 similarity index 100% rename from src/test/hive/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 rename to shark/src/test/hive/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 diff --git a/src/test/hive/golden/stats4-7-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/stats4-7-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/stats4-7-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/stats4-7-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f b/shark/src/test/hive/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f similarity index 100% rename from src/test/hive/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f rename to shark/src/test/hive/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f diff --git a/src/test/hive/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 b/shark/src/test/hive/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 similarity index 100% rename from src/test/hive/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 rename to shark/src/test/hive/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 diff --git a/src/test/hive/golden/stats5-3-96d9aa9c32a081518604959dcfac42df b/shark/src/test/hive/golden/stats5-3-96d9aa9c32a081518604959dcfac42df similarity index 100% rename from src/test/hive/golden/stats5-3-96d9aa9c32a081518604959dcfac42df rename to shark/src/test/hive/golden/stats5-3-96d9aa9c32a081518604959dcfac42df diff --git a/src/test/hive/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 b/shark/src/test/hive/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 similarity index 100% rename from src/test/hive/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 rename to shark/src/test/hive/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 diff --git a/src/test/hive/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 b/shark/src/test/hive/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 similarity index 100% rename from src/test/hive/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 rename to shark/src/test/hive/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 diff --git a/src/test/hive/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/stats6-10-653f748fa2d690502ba4fda407841a20 b/shark/src/test/hive/golden/stats6-10-653f748fa2d690502ba4fda407841a20 similarity index 100% rename from src/test/hive/golden/stats6-10-653f748fa2d690502ba4fda407841a20 rename to shark/src/test/hive/golden/stats6-10-653f748fa2d690502ba4fda407841a20 diff --git a/src/test/hive/golden/stats6-11-1c9f833953403596ad50fd32e513642c b/shark/src/test/hive/golden/stats6-11-1c9f833953403596ad50fd32e513642c similarity index 100% rename from src/test/hive/golden/stats6-11-1c9f833953403596ad50fd32e513642c rename to shark/src/test/hive/golden/stats6-11-1c9f833953403596ad50fd32e513642c diff --git a/src/test/hive/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 b/shark/src/test/hive/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 similarity index 100% rename from src/test/hive/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 rename to shark/src/test/hive/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 diff --git a/src/test/hive/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/stats6-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/stats6-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/stats6-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/stats6-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/stats6-4-a88c476a632cd92f748967fadb242405 b/shark/src/test/hive/golden/stats6-4-a88c476a632cd92f748967fadb242405 similarity index 100% rename from src/test/hive/golden/stats6-4-a88c476a632cd92f748967fadb242405 rename to shark/src/test/hive/golden/stats6-4-a88c476a632cd92f748967fadb242405 diff --git a/src/test/hive/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc b/shark/src/test/hive/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc similarity index 100% rename from src/test/hive/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc rename to shark/src/test/hive/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc diff --git a/src/test/hive/golden/stats6-6-8926095434b70c83bf88c70559d38dce b/shark/src/test/hive/golden/stats6-6-8926095434b70c83bf88c70559d38dce similarity index 100% rename from src/test/hive/golden/stats6-6-8926095434b70c83bf88c70559d38dce rename to shark/src/test/hive/golden/stats6-6-8926095434b70c83bf88c70559d38dce diff --git a/src/test/hive/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 b/shark/src/test/hive/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 similarity index 100% rename from src/test/hive/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 rename to shark/src/test/hive/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 diff --git a/src/test/hive/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 b/shark/src/test/hive/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 similarity index 100% rename from src/test/hive/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 rename to shark/src/test/hive/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 diff --git a/src/test/hive/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d b/shark/src/test/hive/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d similarity index 100% rename from src/test/hive/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d rename to shark/src/test/hive/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d diff --git a/src/test/hive/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 b/shark/src/test/hive/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 similarity index 100% rename from src/test/hive/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 rename to shark/src/test/hive/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 diff --git a/src/test/hive/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/stats7-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/stats7-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/stats7-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/stats7-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/stats7-4-a88c476a632cd92f748967fadb242405 b/shark/src/test/hive/golden/stats7-4-a88c476a632cd92f748967fadb242405 similarity index 100% rename from src/test/hive/golden/stats7-4-a88c476a632cd92f748967fadb242405 rename to shark/src/test/hive/golden/stats7-4-a88c476a632cd92f748967fadb242405 diff --git a/src/test/hive/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc b/shark/src/test/hive/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc similarity index 100% rename from src/test/hive/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc rename to shark/src/test/hive/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc diff --git a/src/test/hive/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 b/shark/src/test/hive/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 similarity index 100% rename from src/test/hive/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 rename to shark/src/test/hive/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 diff --git a/src/test/hive/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 b/shark/src/test/hive/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 similarity index 100% rename from src/test/hive/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 rename to shark/src/test/hive/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 diff --git a/src/test/hive/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 b/shark/src/test/hive/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 similarity index 100% rename from src/test/hive/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 rename to shark/src/test/hive/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 diff --git a/src/test/hive/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d b/shark/src/test/hive/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d similarity index 100% rename from src/test/hive/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d rename to shark/src/test/hive/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d diff --git a/src/test/hive/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 b/shark/src/test/hive/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 similarity index 100% rename from src/test/hive/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 rename to shark/src/test/hive/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 diff --git a/src/test/hive/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 b/shark/src/test/hive/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 similarity index 100% rename from src/test/hive/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 rename to shark/src/test/hive/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 diff --git a/src/test/hive/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d b/shark/src/test/hive/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d similarity index 100% rename from src/test/hive/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d rename to shark/src/test/hive/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d diff --git a/src/test/hive/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d b/shark/src/test/hive/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d similarity index 100% rename from src/test/hive/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d rename to shark/src/test/hive/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d diff --git a/src/test/hive/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 b/shark/src/test/hive/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 similarity index 100% rename from src/test/hive/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 rename to shark/src/test/hive/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 diff --git a/src/test/hive/golden/stats8-15-653f748fa2d690502ba4fda407841a20 b/shark/src/test/hive/golden/stats8-15-653f748fa2d690502ba4fda407841a20 similarity index 100% rename from src/test/hive/golden/stats8-15-653f748fa2d690502ba4fda407841a20 rename to shark/src/test/hive/golden/stats8-15-653f748fa2d690502ba4fda407841a20 diff --git a/src/test/hive/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 b/shark/src/test/hive/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 similarity index 100% rename from src/test/hive/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 rename to shark/src/test/hive/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 diff --git a/src/test/hive/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 b/shark/src/test/hive/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 similarity index 100% rename from src/test/hive/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 rename to shark/src/test/hive/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 diff --git a/src/test/hive/golden/stats8-18-1c9f833953403596ad50fd32e513642c b/shark/src/test/hive/golden/stats8-18-1c9f833953403596ad50fd32e513642c similarity index 100% rename from src/test/hive/golden/stats8-18-1c9f833953403596ad50fd32e513642c rename to shark/src/test/hive/golden/stats8-18-1c9f833953403596ad50fd32e513642c diff --git a/src/test/hive/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 b/shark/src/test/hive/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 similarity index 100% rename from src/test/hive/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 rename to shark/src/test/hive/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 diff --git a/src/test/hive/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 b/shark/src/test/hive/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 similarity index 100% rename from src/test/hive/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 rename to shark/src/test/hive/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 diff --git a/src/test/hive/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 b/shark/src/test/hive/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 similarity index 100% rename from src/test/hive/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 rename to shark/src/test/hive/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 diff --git a/src/test/hive/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d b/shark/src/test/hive/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d similarity index 100% rename from src/test/hive/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d rename to shark/src/test/hive/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d diff --git a/src/test/hive/golden/stats8-23-653f748fa2d690502ba4fda407841a20 b/shark/src/test/hive/golden/stats8-23-653f748fa2d690502ba4fda407841a20 similarity index 100% rename from src/test/hive/golden/stats8-23-653f748fa2d690502ba4fda407841a20 rename to shark/src/test/hive/golden/stats8-23-653f748fa2d690502ba4fda407841a20 diff --git a/src/test/hive/golden/stats8-24-1c9f833953403596ad50fd32e513642c b/shark/src/test/hive/golden/stats8-24-1c9f833953403596ad50fd32e513642c similarity index 100% rename from src/test/hive/golden/stats8-24-1c9f833953403596ad50fd32e513642c rename to shark/src/test/hive/golden/stats8-24-1c9f833953403596ad50fd32e513642c diff --git a/src/test/hive/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 b/shark/src/test/hive/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 similarity index 100% rename from src/test/hive/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 rename to shark/src/test/hive/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 diff --git a/src/test/hive/golden/stats8-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/stats8-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/stats8-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/stats8-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/stats8-4-a88c476a632cd92f748967fadb242405 b/shark/src/test/hive/golden/stats8-4-a88c476a632cd92f748967fadb242405 similarity index 100% rename from src/test/hive/golden/stats8-4-a88c476a632cd92f748967fadb242405 rename to shark/src/test/hive/golden/stats8-4-a88c476a632cd92f748967fadb242405 diff --git a/src/test/hive/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc b/shark/src/test/hive/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc similarity index 100% rename from src/test/hive/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc rename to shark/src/test/hive/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc diff --git a/src/test/hive/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 b/shark/src/test/hive/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 similarity index 100% rename from src/test/hive/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 rename to shark/src/test/hive/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 diff --git a/src/test/hive/golden/stats8-7-8926095434b70c83bf88c70559d38dce b/shark/src/test/hive/golden/stats8-7-8926095434b70c83bf88c70559d38dce similarity index 100% rename from src/test/hive/golden/stats8-7-8926095434b70c83bf88c70559d38dce rename to shark/src/test/hive/golden/stats8-7-8926095434b70c83bf88c70559d38dce diff --git a/src/test/hive/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 b/shark/src/test/hive/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 similarity index 100% rename from src/test/hive/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 rename to shark/src/test/hive/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 diff --git a/src/test/hive/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 b/shark/src/test/hive/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 similarity index 100% rename from src/test/hive/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 rename to shark/src/test/hive/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 diff --git a/src/test/hive/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 b/shark/src/test/hive/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 rename to shark/src/test/hive/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 diff --git a/src/test/hive/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/shark/src/test/hive/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 rename to shark/src/test/hive/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 diff --git a/src/test/hive/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 b/shark/src/test/hive/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 rename to shark/src/test/hive/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 diff --git a/src/test/hive/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb b/shark/src/test/hive/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb rename to shark/src/test/hive/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb diff --git a/src/test/hive/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c b/shark/src/test/hive/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c rename to shark/src/test/hive/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c diff --git a/src/test/hive/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 b/shark/src/test/hive/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 rename to shark/src/test/hive/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 diff --git a/src/test/hive/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 b/shark/src/test/hive/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 rename to shark/src/test/hive/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 diff --git a/src/test/hive/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c b/shark/src/test/hive/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c rename to shark/src/test/hive/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c diff --git a/src/test/hive/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 b/shark/src/test/hive/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 rename to shark/src/test/hive/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 diff --git a/src/test/hive/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 b/shark/src/test/hive/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 rename to shark/src/test/hive/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 diff --git a/src/test/hive/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 b/shark/src/test/hive/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 rename to shark/src/test/hive/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 diff --git a/src/test/hive/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 b/shark/src/test/hive/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 rename to shark/src/test/hive/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 diff --git a/src/test/hive/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 b/shark/src/test/hive/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 rename to shark/src/test/hive/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 diff --git a/src/test/hive/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c b/shark/src/test/hive/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c rename to shark/src/test/hive/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c diff --git a/src/test/hive/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 b/shark/src/test/hive/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 rename to shark/src/test/hive/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 diff --git a/src/test/hive/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 b/shark/src/test/hive/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 rename to shark/src/test/hive/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 diff --git a/src/test/hive/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c b/shark/src/test/hive/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c similarity index 100% rename from src/test/hive/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c rename to shark/src/test/hive/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c diff --git a/src/test/hive/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 b/shark/src/test/hive/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 similarity index 100% rename from src/test/hive/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 rename to shark/src/test/hive/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 diff --git a/src/test/hive/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 b/shark/src/test/hive/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 similarity index 100% rename from src/test/hive/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 rename to shark/src/test/hive/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 diff --git a/src/test/hive/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 b/shark/src/test/hive/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 similarity index 100% rename from src/test/hive/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 rename to shark/src/test/hive/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 diff --git a/src/test/hive/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 b/shark/src/test/hive/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 similarity index 100% rename from src/test/hive/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 rename to shark/src/test/hive/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 diff --git a/src/test/hive/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 b/shark/src/test/hive/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 similarity index 100% rename from src/test/hive/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 rename to shark/src/test/hive/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 diff --git a/src/test/hive/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 b/shark/src/test/hive/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 similarity index 100% rename from src/test/hive/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 rename to shark/src/test/hive/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 diff --git a/src/test/hive/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 b/shark/src/test/hive/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 similarity index 100% rename from src/test/hive/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 rename to shark/src/test/hive/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 diff --git a/src/test/hive/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 b/shark/src/test/hive/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 similarity index 100% rename from src/test/hive/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 rename to shark/src/test/hive/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 diff --git a/src/test/hive/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 b/shark/src/test/hive/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 similarity index 100% rename from src/test/hive/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 rename to shark/src/test/hive/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 diff --git a/src/test/hive/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 b/shark/src/test/hive/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 similarity index 100% rename from src/test/hive/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 rename to shark/src/test/hive/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 diff --git a/src/test/hive/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d b/shark/src/test/hive/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d similarity index 100% rename from src/test/hive/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d rename to shark/src/test/hive/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d diff --git a/src/test/hive/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 b/shark/src/test/hive/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 rename to shark/src/test/hive/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 diff --git a/src/test/hive/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c b/shark/src/test/hive/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c similarity index 100% rename from src/test/hive/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c rename to shark/src/test/hive/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c diff --git a/src/test/hive/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 b/shark/src/test/hive/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 rename to shark/src/test/hive/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 diff --git a/src/test/hive/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab b/shark/src/test/hive/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab similarity index 100% rename from src/test/hive/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab rename to shark/src/test/hive/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab diff --git a/src/test/hive/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 b/shark/src/test/hive/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 rename to shark/src/test/hive/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 diff --git a/src/test/hive/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 b/shark/src/test/hive/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 rename to shark/src/test/hive/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 diff --git a/src/test/hive/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 b/shark/src/test/hive/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 rename to shark/src/test/hive/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 diff --git a/src/test/hive/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 b/shark/src/test/hive/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 rename to shark/src/test/hive/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 diff --git a/src/test/hive/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e b/shark/src/test/hive/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e similarity index 100% rename from src/test/hive/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e rename to shark/src/test/hive/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e diff --git a/src/test/hive/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f b/shark/src/test/hive/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f similarity index 100% rename from src/test/hive/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f rename to shark/src/test/hive/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f diff --git a/src/test/hive/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 b/shark/src/test/hive/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 rename to shark/src/test/hive/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 diff --git a/src/test/hive/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a b/shark/src/test/hive/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a similarity index 100% rename from src/test/hive/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a rename to shark/src/test/hive/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a diff --git a/src/test/hive/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 b/shark/src/test/hive/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 rename to shark/src/test/hive/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 diff --git a/src/test/hive/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c b/shark/src/test/hive/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c similarity index 100% rename from src/test/hive/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c rename to shark/src/test/hive/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c diff --git a/src/test/hive/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc b/shark/src/test/hive/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc similarity index 100% rename from src/test/hive/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc rename to shark/src/test/hive/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc diff --git a/src/test/hive/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 b/shark/src/test/hive/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 rename to shark/src/test/hive/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 diff --git a/src/test/hive/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e b/shark/src/test/hive/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e similarity index 100% rename from src/test/hive/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e rename to shark/src/test/hive/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e diff --git a/src/test/hive/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae b/shark/src/test/hive/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae similarity index 100% rename from src/test/hive/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae rename to shark/src/test/hive/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae diff --git a/src/test/hive/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 b/shark/src/test/hive/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 similarity index 100% rename from src/test/hive/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 rename to shark/src/test/hive/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 diff --git a/src/test/hive/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 b/shark/src/test/hive/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 rename to shark/src/test/hive/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 diff --git a/src/test/hive/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 b/shark/src/test/hive/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 rename to shark/src/test/hive/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 diff --git a/src/test/hive/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef b/shark/src/test/hive/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef similarity index 100% rename from src/test/hive/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef rename to shark/src/test/hive/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef diff --git a/src/test/hive/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 b/shark/src/test/hive/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 rename to shark/src/test/hive/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 diff --git a/src/test/hive/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c b/shark/src/test/hive/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c similarity index 100% rename from src/test/hive/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c rename to shark/src/test/hive/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c diff --git a/src/test/hive/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 b/shark/src/test/hive/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 rename to shark/src/test/hive/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 diff --git a/src/test/hive/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 b/shark/src/test/hive/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 rename to shark/src/test/hive/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 diff --git a/src/test/hive/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf b/shark/src/test/hive/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf similarity index 100% rename from src/test/hive/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf rename to shark/src/test/hive/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf diff --git a/src/test/hive/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 b/shark/src/test/hive/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 rename to shark/src/test/hive/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 diff --git a/src/test/hive/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 b/shark/src/test/hive/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 rename to shark/src/test/hive/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 diff --git a/src/test/hive/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea b/shark/src/test/hive/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea similarity index 100% rename from src/test/hive/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea rename to shark/src/test/hive/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea diff --git a/src/test/hive/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 b/shark/src/test/hive/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 rename to shark/src/test/hive/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 diff --git a/src/test/hive/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 b/shark/src/test/hive/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 similarity index 100% rename from src/test/hive/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 rename to shark/src/test/hive/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 diff --git a/src/test/hive/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f b/shark/src/test/hive/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f rename to shark/src/test/hive/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f diff --git a/src/test/hive/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 b/shark/src/test/hive/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 rename to shark/src/test/hive/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 diff --git a/src/test/hive/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 b/shark/src/test/hive/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 rename to shark/src/test/hive/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 diff --git a/src/test/hive/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef b/shark/src/test/hive/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef rename to shark/src/test/hive/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef diff --git a/src/test/hive/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 b/shark/src/test/hive/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 rename to shark/src/test/hive/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 diff --git a/src/test/hive/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c b/shark/src/test/hive/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c rename to shark/src/test/hive/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c diff --git a/src/test/hive/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 b/shark/src/test/hive/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 rename to shark/src/test/hive/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 diff --git a/src/test/hive/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 b/shark/src/test/hive/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 rename to shark/src/test/hive/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 diff --git a/src/test/hive/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf b/shark/src/test/hive/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf rename to shark/src/test/hive/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf diff --git a/src/test/hive/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 b/shark/src/test/hive/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 rename to shark/src/test/hive/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 diff --git a/src/test/hive/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 b/shark/src/test/hive/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 rename to shark/src/test/hive/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 diff --git a/src/test/hive/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea b/shark/src/test/hive/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea rename to shark/src/test/hive/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea diff --git a/src/test/hive/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca b/shark/src/test/hive/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca rename to shark/src/test/hive/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca diff --git a/src/test/hive/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 b/shark/src/test/hive/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 similarity index 100% rename from src/test/hive/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 rename to shark/src/test/hive/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 diff --git a/src/test/hive/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 b/shark/src/test/hive/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 rename to shark/src/test/hive/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 diff --git a/src/test/hive/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/shark/src/test/hive/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 rename to shark/src/test/hive/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 diff --git a/src/test/hive/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 b/shark/src/test/hive/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 rename to shark/src/test/hive/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 diff --git a/src/test/hive/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 b/shark/src/test/hive/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 rename to shark/src/test/hive/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 diff --git a/src/test/hive/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c b/shark/src/test/hive/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c rename to shark/src/test/hive/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c diff --git a/src/test/hive/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 b/shark/src/test/hive/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 rename to shark/src/test/hive/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 diff --git a/src/test/hive/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 b/shark/src/test/hive/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 rename to shark/src/test/hive/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 diff --git a/src/test/hive/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c b/shark/src/test/hive/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c rename to shark/src/test/hive/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c diff --git a/src/test/hive/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 b/shark/src/test/hive/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 rename to shark/src/test/hive/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 diff --git a/src/test/hive/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 b/shark/src/test/hive/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 rename to shark/src/test/hive/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 diff --git a/src/test/hive/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 b/shark/src/test/hive/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 rename to shark/src/test/hive/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 diff --git a/src/test/hive/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 b/shark/src/test/hive/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 rename to shark/src/test/hive/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 diff --git a/src/test/hive/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 b/shark/src/test/hive/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 rename to shark/src/test/hive/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 diff --git a/src/test/hive/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c b/shark/src/test/hive/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c rename to shark/src/test/hive/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c diff --git a/src/test/hive/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 b/shark/src/test/hive/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 rename to shark/src/test/hive/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 diff --git a/src/test/hive/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b b/shark/src/test/hive/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b rename to shark/src/test/hive/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b diff --git a/src/test/hive/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c b/shark/src/test/hive/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c similarity index 100% rename from src/test/hive/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c rename to shark/src/test/hive/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c diff --git a/src/test/hive/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 b/shark/src/test/hive/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 similarity index 100% rename from src/test/hive/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 rename to shark/src/test/hive/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 diff --git a/src/test/hive/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c b/shark/src/test/hive/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c similarity index 100% rename from src/test/hive/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c rename to shark/src/test/hive/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c diff --git a/src/test/hive/golden/str_to_map-10-32997010bba305ec40812df254490730 b/shark/src/test/hive/golden/str_to_map-10-32997010bba305ec40812df254490730 similarity index 100% rename from src/test/hive/golden/str_to_map-10-32997010bba305ec40812df254490730 rename to shark/src/test/hive/golden/str_to_map-10-32997010bba305ec40812df254490730 diff --git a/src/test/hive/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a b/shark/src/test/hive/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a similarity index 100% rename from src/test/hive/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a rename to shark/src/test/hive/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a diff --git a/src/test/hive/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e b/shark/src/test/hive/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e similarity index 100% rename from src/test/hive/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e rename to shark/src/test/hive/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e diff --git a/src/test/hive/golden/str_to_map-13-32997010bba305ec40812df254490730 b/shark/src/test/hive/golden/str_to_map-13-32997010bba305ec40812df254490730 similarity index 100% rename from src/test/hive/golden/str_to_map-13-32997010bba305ec40812df254490730 rename to shark/src/test/hive/golden/str_to_map-13-32997010bba305ec40812df254490730 diff --git a/src/test/hive/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec b/shark/src/test/hive/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec similarity index 100% rename from src/test/hive/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec rename to shark/src/test/hive/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec diff --git a/src/test/hive/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 b/shark/src/test/hive/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 similarity index 100% rename from src/test/hive/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 rename to shark/src/test/hive/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 diff --git a/src/test/hive/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 b/shark/src/test/hive/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 similarity index 100% rename from src/test/hive/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 rename to shark/src/test/hive/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 diff --git a/src/test/hive/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda b/shark/src/test/hive/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda similarity index 100% rename from src/test/hive/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda rename to shark/src/test/hive/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda diff --git a/src/test/hive/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf b/shark/src/test/hive/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf similarity index 100% rename from src/test/hive/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf rename to shark/src/test/hive/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf diff --git a/src/test/hive/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad b/shark/src/test/hive/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad similarity index 100% rename from src/test/hive/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad rename to shark/src/test/hive/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad diff --git a/src/test/hive/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 b/shark/src/test/hive/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 similarity index 100% rename from src/test/hive/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 rename to shark/src/test/hive/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 diff --git a/src/test/hive/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd b/shark/src/test/hive/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd similarity index 100% rename from src/test/hive/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd rename to shark/src/test/hive/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd diff --git a/src/test/hive/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 b/shark/src/test/hive/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 similarity index 100% rename from src/test/hive/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 rename to shark/src/test/hive/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 diff --git a/src/test/hive/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 b/shark/src/test/hive/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 similarity index 100% rename from src/test/hive/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 rename to shark/src/test/hive/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 diff --git a/src/test/hive/golden/subq2-1-235919a7ddb574662158503b8052e7ca b/shark/src/test/hive/golden/subq2-1-235919a7ddb574662158503b8052e7ca similarity index 100% rename from src/test/hive/golden/subq2-1-235919a7ddb574662158503b8052e7ca rename to shark/src/test/hive/golden/subq2-1-235919a7ddb574662158503b8052e7ca diff --git a/src/test/hive/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 b/shark/src/test/hive/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 similarity index 100% rename from src/test/hive/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 rename to shark/src/test/hive/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 diff --git a/src/test/hive/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e b/shark/src/test/hive/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e similarity index 100% rename from src/test/hive/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e rename to shark/src/test/hive/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e diff --git a/src/test/hive/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 b/shark/src/test/hive/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 similarity index 100% rename from src/test/hive/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 rename to shark/src/test/hive/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 diff --git a/src/test/hive/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 b/shark/src/test/hive/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 similarity index 100% rename from src/test/hive/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 rename to shark/src/test/hive/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 diff --git a/src/test/hive/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d b/shark/src/test/hive/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d similarity index 100% rename from src/test/hive/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d rename to shark/src/test/hive/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d diff --git a/src/test/hive/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 b/shark/src/test/hive/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 similarity index 100% rename from src/test/hive/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 rename to shark/src/test/hive/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 diff --git a/src/test/hive/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 b/shark/src/test/hive/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 similarity index 100% rename from src/test/hive/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 rename to shark/src/test/hive/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 diff --git a/src/test/hive/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c b/shark/src/test/hive/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c similarity index 100% rename from src/test/hive/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c rename to shark/src/test/hive/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c diff --git a/src/test/hive/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 b/shark/src/test/hive/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 similarity index 100% rename from src/test/hive/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 rename to shark/src/test/hive/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 diff --git a/src/test/hive/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 b/shark/src/test/hive/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 similarity index 100% rename from src/test/hive/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 rename to shark/src/test/hive/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 diff --git a/src/test/hive/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 b/shark/src/test/hive/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 similarity index 100% rename from src/test/hive/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 rename to shark/src/test/hive/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 diff --git a/src/test/hive/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 b/shark/src/test/hive/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 similarity index 100% rename from src/test/hive/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 rename to shark/src/test/hive/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 diff --git a/src/test/hive/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 b/shark/src/test/hive/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 similarity index 100% rename from src/test/hive/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 rename to shark/src/test/hive/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 diff --git a/src/test/hive/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 b/shark/src/test/hive/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 similarity index 100% rename from src/test/hive/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 rename to shark/src/test/hive/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 diff --git a/src/test/hive/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 b/shark/src/test/hive/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 similarity index 100% rename from src/test/hive/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 rename to shark/src/test/hive/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 diff --git a/src/test/hive/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 b/shark/src/test/hive/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 similarity index 100% rename from src/test/hive/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 rename to shark/src/test/hive/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 diff --git a/src/test/hive/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b b/shark/src/test/hive/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b similarity index 100% rename from src/test/hive/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b rename to shark/src/test/hive/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b diff --git a/src/test/hive/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 b/shark/src/test/hive/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 similarity index 100% rename from src/test/hive/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 rename to shark/src/test/hive/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 diff --git a/src/test/hive/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 b/shark/src/test/hive/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 similarity index 100% rename from src/test/hive/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 rename to shark/src/test/hive/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 diff --git a/src/test/hive/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc b/shark/src/test/hive/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc similarity index 100% rename from src/test/hive/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc rename to shark/src/test/hive/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc diff --git a/src/test/hive/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 b/shark/src/test/hive/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 similarity index 100% rename from src/test/hive/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 rename to shark/src/test/hive/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 diff --git a/src/test/hive/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb b/shark/src/test/hive/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb similarity index 100% rename from src/test/hive/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb rename to shark/src/test/hive/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb diff --git a/src/test/hive/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 b/shark/src/test/hive/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 similarity index 100% rename from src/test/hive/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 rename to shark/src/test/hive/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 diff --git a/src/test/hive/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af b/shark/src/test/hive/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af similarity index 100% rename from src/test/hive/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af rename to shark/src/test/hive/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af diff --git a/src/test/hive/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d b/shark/src/test/hive/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d similarity index 100% rename from src/test/hive/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d rename to shark/src/test/hive/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d diff --git a/src/test/hive/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 b/shark/src/test/hive/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 similarity index 100% rename from src/test/hive/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 rename to shark/src/test/hive/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 diff --git a/src/test/hive/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f b/shark/src/test/hive/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f similarity index 100% rename from src/test/hive/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f rename to shark/src/test/hive/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f diff --git a/src/test/hive/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 b/shark/src/test/hive/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 similarity index 100% rename from src/test/hive/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 rename to shark/src/test/hive/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 diff --git a/src/test/hive/golden/touch-1-27ad2962fed131f51ba802596ba37278 b/shark/src/test/hive/golden/touch-1-27ad2962fed131f51ba802596ba37278 similarity index 100% rename from src/test/hive/golden/touch-1-27ad2962fed131f51ba802596ba37278 rename to shark/src/test/hive/golden/touch-1-27ad2962fed131f51ba802596ba37278 diff --git a/src/test/hive/golden/touch-10-27ad2962fed131f51ba802596ba37278 b/shark/src/test/hive/golden/touch-10-27ad2962fed131f51ba802596ba37278 similarity index 100% rename from src/test/hive/golden/touch-10-27ad2962fed131f51ba802596ba37278 rename to shark/src/test/hive/golden/touch-10-27ad2962fed131f51ba802596ba37278 diff --git a/src/test/hive/golden/touch-2-3e95421993ab28d18245ec2340f580a3 b/shark/src/test/hive/golden/touch-2-3e95421993ab28d18245ec2340f580a3 similarity index 100% rename from src/test/hive/golden/touch-2-3e95421993ab28d18245ec2340f580a3 rename to shark/src/test/hive/golden/touch-2-3e95421993ab28d18245ec2340f580a3 diff --git a/src/test/hive/golden/touch-3-c0c18ac884677231a41eea8d980d0451 b/shark/src/test/hive/golden/touch-3-c0c18ac884677231a41eea8d980d0451 similarity index 100% rename from src/test/hive/golden/touch-3-c0c18ac884677231a41eea8d980d0451 rename to shark/src/test/hive/golden/touch-3-c0c18ac884677231a41eea8d980d0451 diff --git a/src/test/hive/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a b/shark/src/test/hive/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a similarity index 100% rename from src/test/hive/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a rename to shark/src/test/hive/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a diff --git a/src/test/hive/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c b/shark/src/test/hive/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c similarity index 100% rename from src/test/hive/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c rename to shark/src/test/hive/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c diff --git a/src/test/hive/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 b/shark/src/test/hive/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 similarity index 100% rename from src/test/hive/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 rename to shark/src/test/hive/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 diff --git a/src/test/hive/golden/touch-7-3752320b12abae0d138148d56a27c5b1 b/shark/src/test/hive/golden/touch-7-3752320b12abae0d138148d56a27c5b1 similarity index 100% rename from src/test/hive/golden/touch-7-3752320b12abae0d138148d56a27c5b1 rename to shark/src/test/hive/golden/touch-7-3752320b12abae0d138148d56a27c5b1 diff --git a/src/test/hive/golden/touch-8-3562ab40242756452595cd7eae79b0ce b/shark/src/test/hive/golden/touch-8-3562ab40242756452595cd7eae79b0ce similarity index 100% rename from src/test/hive/golden/touch-8-3562ab40242756452595cd7eae79b0ce rename to shark/src/test/hive/golden/touch-8-3562ab40242756452595cd7eae79b0ce diff --git a/src/test/hive/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 b/shark/src/test/hive/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 similarity index 100% rename from src/test/hive/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 rename to shark/src/test/hive/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 diff --git a/src/test/hive/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe b/shark/src/test/hive/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe similarity index 100% rename from src/test/hive/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe rename to shark/src/test/hive/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe diff --git a/src/test/hive/golden/transform1-0-b6919fc48901e388c869c84ae0211102 b/shark/src/test/hive/golden/transform1-0-b6919fc48901e388c869c84ae0211102 similarity index 100% rename from src/test/hive/golden/transform1-0-b6919fc48901e388c869c84ae0211102 rename to shark/src/test/hive/golden/transform1-0-b6919fc48901e388c869c84ae0211102 diff --git a/src/test/hive/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 b/shark/src/test/hive/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 similarity index 100% rename from src/test/hive/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 rename to shark/src/test/hive/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 diff --git a/src/test/hive/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 b/shark/src/test/hive/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 similarity index 100% rename from src/test/hive/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 rename to shark/src/test/hive/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 diff --git a/src/test/hive/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 b/shark/src/test/hive/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 similarity index 100% rename from src/test/hive/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 rename to shark/src/test/hive/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 diff --git a/src/test/hive/golden/transform1-4-65527bae8e73262255ef83082c6968f9 b/shark/src/test/hive/golden/transform1-4-65527bae8e73262255ef83082c6968f9 similarity index 100% rename from src/test/hive/golden/transform1-4-65527bae8e73262255ef83082c6968f9 rename to shark/src/test/hive/golden/transform1-4-65527bae8e73262255ef83082c6968f9 diff --git a/src/test/hive/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 b/shark/src/test/hive/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 similarity index 100% rename from src/test/hive/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 rename to shark/src/test/hive/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 diff --git a/src/test/hive/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f b/shark/src/test/hive/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f similarity index 100% rename from src/test/hive/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f rename to shark/src/test/hive/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f diff --git a/src/test/hive/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 b/shark/src/test/hive/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 similarity index 100% rename from src/test/hive/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 rename to shark/src/test/hive/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 diff --git a/src/test/hive/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 b/shark/src/test/hive/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 similarity index 100% rename from src/test/hive/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 rename to shark/src/test/hive/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 diff --git a/src/test/hive/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 b/shark/src/test/hive/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 similarity index 100% rename from src/test/hive/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 rename to shark/src/test/hive/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 diff --git a/src/test/hive/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 b/shark/src/test/hive/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 similarity index 100% rename from src/test/hive/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 rename to shark/src/test/hive/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 diff --git a/src/test/hive/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 b/shark/src/test/hive/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 similarity index 100% rename from src/test/hive/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 rename to shark/src/test/hive/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 diff --git a/src/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d b/shark/src/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d similarity index 100% rename from src/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d rename to shark/src/test/hive/golden/trival join ON clause-0-3b6afcbd622aa111ee260bebc763613d diff --git a/src/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 b/shark/src/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 similarity index 100% rename from src/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 rename to shark/src/test/hive/golden/trival join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 diff --git a/src/test/hive/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 b/shark/src/test/hive/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 similarity index 100% rename from src/test/hive/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 rename to shark/src/test/hive/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 diff --git a/src/test/hive/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 b/shark/src/test/hive/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 similarity index 100% rename from src/test/hive/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 rename to shark/src/test/hive/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 diff --git a/src/test/hive/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 b/shark/src/test/hive/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 similarity index 100% rename from src/test/hive/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 rename to shark/src/test/hive/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 diff --git a/src/test/hive/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 b/shark/src/test/hive/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 similarity index 100% rename from src/test/hive/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 rename to shark/src/test/hive/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 diff --git a/src/test/hive/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e b/shark/src/test/hive/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e similarity index 100% rename from src/test/hive/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e rename to shark/src/test/hive/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e diff --git a/src/test/hive/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 b/shark/src/test/hive/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 similarity index 100% rename from src/test/hive/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 rename to shark/src/test/hive/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 diff --git a/src/test/hive/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db b/shark/src/test/hive/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db similarity index 100% rename from src/test/hive/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db rename to shark/src/test/hive/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db diff --git a/src/test/hive/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 b/shark/src/test/hive/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from src/test/hive/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 rename to shark/src/test/hive/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/src/test/hive/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db b/shark/src/test/hive/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db similarity index 100% rename from src/test/hive/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db rename to shark/src/test/hive/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db diff --git a/src/test/hive/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 b/shark/src/test/hive/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from src/test/hive/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 rename to shark/src/test/hive/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/src/test/hive/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e b/shark/src/test/hive/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from src/test/hive/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e rename to shark/src/test/hive/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e diff --git a/src/test/hive/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db b/shark/src/test/hive/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db similarity index 100% rename from src/test/hive/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db rename to shark/src/test/hive/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db diff --git a/src/test/hive/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e b/shark/src/test/hive/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from src/test/hive/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e rename to shark/src/test/hive/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e diff --git a/src/test/hive/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db b/shark/src/test/hive/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db similarity index 100% rename from src/test/hive/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db rename to shark/src/test/hive/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db diff --git a/src/test/hive/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 b/shark/src/test/hive/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from src/test/hive/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 rename to shark/src/test/hive/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/src/test/hive/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 b/shark/src/test/hive/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from src/test/hive/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 rename to shark/src/test/hive/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/src/test/hive/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 b/shark/src/test/hive/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 similarity index 100% rename from src/test/hive/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 rename to shark/src/test/hive/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 diff --git a/src/test/hive/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 b/shark/src/test/hive/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 similarity index 100% rename from src/test/hive/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 rename to shark/src/test/hive/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 diff --git a/src/test/hive/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 b/shark/src/test/hive/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 similarity index 100% rename from src/test/hive/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 rename to shark/src/test/hive/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 diff --git a/src/test/hive/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 b/shark/src/test/hive/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 similarity index 100% rename from src/test/hive/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 rename to shark/src/test/hive/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 diff --git a/src/test/hive/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e b/shark/src/test/hive/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e similarity index 100% rename from src/test/hive/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e rename to shark/src/test/hive/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e diff --git a/src/test/hive/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a b/shark/src/test/hive/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a similarity index 100% rename from src/test/hive/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a rename to shark/src/test/hive/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a diff --git a/src/test/hive/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb b/shark/src/test/hive/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb similarity index 100% rename from src/test/hive/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb rename to shark/src/test/hive/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb diff --git a/src/test/hive/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 b/shark/src/test/hive/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 similarity index 100% rename from src/test/hive/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 rename to shark/src/test/hive/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 diff --git a/src/test/hive/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 b/shark/src/test/hive/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 similarity index 100% rename from src/test/hive/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 rename to shark/src/test/hive/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 diff --git a/src/test/hive/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 b/shark/src/test/hive/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 similarity index 100% rename from src/test/hive/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 rename to shark/src/test/hive/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 diff --git a/src/test/hive/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 b/shark/src/test/hive/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 similarity index 100% rename from src/test/hive/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 rename to shark/src/test/hive/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 diff --git a/src/test/hive/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 b/shark/src/test/hive/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 rename to shark/src/test/hive/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 diff --git a/src/test/hive/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 b/shark/src/test/hive/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 rename to shark/src/test/hive/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 diff --git a/src/test/hive/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 b/shark/src/test/hive/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 rename to shark/src/test/hive/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 diff --git a/src/test/hive/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 b/shark/src/test/hive/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 rename to shark/src/test/hive/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 diff --git a/src/test/hive/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 b/shark/src/test/hive/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 rename to shark/src/test/hive/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 diff --git a/src/test/hive/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad b/shark/src/test/hive/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad rename to shark/src/test/hive/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad diff --git a/src/test/hive/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd b/shark/src/test/hive/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd rename to shark/src/test/hive/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd diff --git a/src/test/hive/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b b/shark/src/test/hive/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b rename to shark/src/test/hive/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b diff --git a/src/test/hive/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae b/shark/src/test/hive/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae rename to shark/src/test/hive/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae diff --git a/src/test/hive/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f b/shark/src/test/hive/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f rename to shark/src/test/hive/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f diff --git a/src/test/hive/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 b/shark/src/test/hive/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 similarity index 100% rename from src/test/hive/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 rename to shark/src/test/hive/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 diff --git a/src/test/hive/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 b/shark/src/test/hive/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 rename to shark/src/test/hive/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 diff --git a/src/test/hive/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 b/shark/src/test/hive/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 rename to shark/src/test/hive/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 diff --git a/src/test/hive/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 b/shark/src/test/hive/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 rename to shark/src/test/hive/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 diff --git a/src/test/hive/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 b/shark/src/test/hive/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 rename to shark/src/test/hive/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 diff --git a/src/test/hive/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf b/shark/src/test/hive/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf rename to shark/src/test/hive/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf diff --git a/src/test/hive/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c b/shark/src/test/hive/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c rename to shark/src/test/hive/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c diff --git a/src/test/hive/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b b/shark/src/test/hive/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b rename to shark/src/test/hive/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b diff --git a/src/test/hive/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 b/shark/src/test/hive/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 rename to shark/src/test/hive/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 diff --git a/src/test/hive/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 b/shark/src/test/hive/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 rename to shark/src/test/hive/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 diff --git a/src/test/hive/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a b/shark/src/test/hive/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a rename to shark/src/test/hive/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a diff --git a/src/test/hive/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 b/shark/src/test/hive/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 similarity index 100% rename from src/test/hive/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 rename to shark/src/test/hive/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 diff --git a/src/test/hive/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 b/shark/src/test/hive/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 similarity index 100% rename from src/test/hive/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 rename to shark/src/test/hive/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 diff --git a/src/test/hive/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 b/shark/src/test/hive/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 similarity index 100% rename from src/test/hive/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 rename to shark/src/test/hive/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 diff --git a/src/test/hive/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 b/shark/src/test/hive/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 similarity index 100% rename from src/test/hive/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 rename to shark/src/test/hive/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 diff --git a/src/test/hive/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 b/shark/src/test/hive/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 similarity index 100% rename from src/test/hive/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 rename to shark/src/test/hive/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 diff --git a/src/test/hive/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 b/shark/src/test/hive/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 similarity index 100% rename from src/test/hive/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 rename to shark/src/test/hive/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 diff --git a/src/test/hive/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 b/shark/src/test/hive/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 similarity index 100% rename from src/test/hive/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 rename to shark/src/test/hive/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac b/shark/src/test/hive/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac rename to shark/src/test/hive/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac diff --git a/src/test/hive/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 b/shark/src/test/hive/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae b/shark/src/test/hive/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae rename to shark/src/test/hive/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae diff --git a/src/test/hive/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 b/shark/src/test/hive/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 b/shark/src/test/hive/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb b/shark/src/test/hive/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb rename to shark/src/test/hive/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb diff --git a/src/test/hive/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 b/shark/src/test/hive/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f b/shark/src/test/hive/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f rename to shark/src/test/hive/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f diff --git a/src/test/hive/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 b/shark/src/test/hive/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 b/shark/src/test/hive/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a b/shark/src/test/hive/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a rename to shark/src/test/hive/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a diff --git a/src/test/hive/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/shark/src/test/hive/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c rename to shark/src/test/hive/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c diff --git a/src/test/hive/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff b/shark/src/test/hive/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff rename to shark/src/test/hive/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff diff --git a/src/test/hive/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df b/shark/src/test/hive/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df rename to shark/src/test/hive/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df diff --git a/src/test/hive/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b b/shark/src/test/hive/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b rename to shark/src/test/hive/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b diff --git a/src/test/hive/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 b/shark/src/test/hive/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 b/shark/src/test/hive/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/shark/src/test/hive/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb rename to shark/src/test/hive/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb diff --git a/src/test/hive/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 b/shark/src/test/hive/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 b/shark/src/test/hive/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb b/shark/src/test/hive/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb rename to shark/src/test/hive/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb diff --git a/src/test/hive/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 b/shark/src/test/hive/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c b/shark/src/test/hive/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c rename to shark/src/test/hive/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c diff --git a/src/test/hive/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/shark/src/test/hive/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 b/shark/src/test/hive/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/shark/src/test/hive/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c rename to shark/src/test/hive/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c diff --git a/src/test/hive/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff b/shark/src/test/hive/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff rename to shark/src/test/hive/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff diff --git a/src/test/hive/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df b/shark/src/test/hive/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df rename to shark/src/test/hive/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df diff --git a/src/test/hive/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b b/shark/src/test/hive/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b rename to shark/src/test/hive/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b diff --git a/src/test/hive/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 b/shark/src/test/hive/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 b/shark/src/test/hive/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f b/shark/src/test/hive/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f rename to shark/src/test/hive/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f diff --git a/src/test/hive/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/shark/src/test/hive/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb rename to shark/src/test/hive/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb diff --git a/src/test/hive/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 b/shark/src/test/hive/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 b/shark/src/test/hive/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb b/shark/src/test/hive/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb rename to shark/src/test/hive/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb diff --git a/src/test/hive/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c b/shark/src/test/hive/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c rename to shark/src/test/hive/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c diff --git a/src/test/hive/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/shark/src/test/hive/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 b/shark/src/test/hive/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/shark/src/test/hive/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/shark/src/test/hive/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 diff --git a/src/test/hive/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f b/shark/src/test/hive/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f rename to shark/src/test/hive/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f diff --git a/src/test/hive/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 b/shark/src/test/hive/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 rename to shark/src/test/hive/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace b/shark/src/test/hive/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace rename to shark/src/test/hive/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace diff --git a/src/test/hive/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 b/shark/src/test/hive/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae b/shark/src/test/hive/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae rename to shark/src/test/hive/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae diff --git a/src/test/hive/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 b/shark/src/test/hive/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 b/shark/src/test/hive/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb b/shark/src/test/hive/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb rename to shark/src/test/hive/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb diff --git a/src/test/hive/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 b/shark/src/test/hive/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f b/shark/src/test/hive/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f rename to shark/src/test/hive/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f diff --git a/src/test/hive/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 b/shark/src/test/hive/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f b/shark/src/test/hive/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f rename to shark/src/test/hive/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f diff --git a/src/test/hive/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/shark/src/test/hive/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 b/shark/src/test/hive/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a b/shark/src/test/hive/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a rename to shark/src/test/hive/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a diff --git a/src/test/hive/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/shark/src/test/hive/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c rename to shark/src/test/hive/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c diff --git a/src/test/hive/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff b/shark/src/test/hive/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff rename to shark/src/test/hive/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff diff --git a/src/test/hive/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df b/shark/src/test/hive/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df rename to shark/src/test/hive/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df diff --git a/src/test/hive/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b b/shark/src/test/hive/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b rename to shark/src/test/hive/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b diff --git a/src/test/hive/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 b/shark/src/test/hive/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 b/shark/src/test/hive/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/shark/src/test/hive/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb rename to shark/src/test/hive/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb diff --git a/src/test/hive/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 b/shark/src/test/hive/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 b/shark/src/test/hive/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb b/shark/src/test/hive/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb rename to shark/src/test/hive/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb diff --git a/src/test/hive/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 b/shark/src/test/hive/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c b/shark/src/test/hive/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c rename to shark/src/test/hive/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c diff --git a/src/test/hive/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/shark/src/test/hive/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 b/shark/src/test/hive/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 b/shark/src/test/hive/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/shark/src/test/hive/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c rename to shark/src/test/hive/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c diff --git a/src/test/hive/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff b/shark/src/test/hive/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff rename to shark/src/test/hive/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff diff --git a/src/test/hive/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df b/shark/src/test/hive/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df rename to shark/src/test/hive/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df diff --git a/src/test/hive/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b b/shark/src/test/hive/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b rename to shark/src/test/hive/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b diff --git a/src/test/hive/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 b/shark/src/test/hive/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 b/shark/src/test/hive/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f b/shark/src/test/hive/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f rename to shark/src/test/hive/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f diff --git a/src/test/hive/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/shark/src/test/hive/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb rename to shark/src/test/hive/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb diff --git a/src/test/hive/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 b/shark/src/test/hive/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 b/shark/src/test/hive/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb b/shark/src/test/hive/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb rename to shark/src/test/hive/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb diff --git a/src/test/hive/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c b/shark/src/test/hive/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c rename to shark/src/test/hive/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c diff --git a/src/test/hive/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/shark/src/test/hive/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 b/shark/src/test/hive/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/shark/src/test/hive/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/shark/src/test/hive/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 diff --git a/src/test/hive/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f b/shark/src/test/hive/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f rename to shark/src/test/hive/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f diff --git a/src/test/hive/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 b/shark/src/test/hive/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 similarity index 100% rename from src/test/hive/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 rename to shark/src/test/hive/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 diff --git a/src/test/hive/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 b/shark/src/test/hive/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 similarity index 100% rename from src/test/hive/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 rename to shark/src/test/hive/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 diff --git a/src/test/hive/golden/udf1-1-8281592c818ada269024ac669bec78da b/shark/src/test/hive/golden/udf1-1-8281592c818ada269024ac669bec78da similarity index 100% rename from src/test/hive/golden/udf1-1-8281592c818ada269024ac669bec78da rename to shark/src/test/hive/golden/udf1-1-8281592c818ada269024ac669bec78da diff --git a/src/test/hive/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f b/shark/src/test/hive/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f similarity index 100% rename from src/test/hive/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f rename to shark/src/test/hive/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f diff --git a/src/test/hive/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 b/shark/src/test/hive/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 similarity index 100% rename from src/test/hive/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 rename to shark/src/test/hive/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 diff --git a/src/test/hive/golden/udf2-2-31c8af064bac42541558a95fad3bca97 b/shark/src/test/hive/golden/udf2-2-31c8af064bac42541558a95fad3bca97 similarity index 100% rename from src/test/hive/golden/udf2-2-31c8af064bac42541558a95fad3bca97 rename to shark/src/test/hive/golden/udf2-2-31c8af064bac42541558a95fad3bca97 diff --git a/src/test/hive/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 b/shark/src/test/hive/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 similarity index 100% rename from src/test/hive/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 rename to shark/src/test/hive/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 diff --git a/src/test/hive/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c b/shark/src/test/hive/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c similarity index 100% rename from src/test/hive/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c rename to shark/src/test/hive/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c diff --git a/src/test/hive/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 b/shark/src/test/hive/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 similarity index 100% rename from src/test/hive/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 rename to shark/src/test/hive/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 diff --git a/src/test/hive/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e b/shark/src/test/hive/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e similarity index 100% rename from src/test/hive/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e rename to shark/src/test/hive/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e diff --git a/src/test/hive/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 b/shark/src/test/hive/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 similarity index 100% rename from src/test/hive/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 rename to shark/src/test/hive/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 diff --git a/src/test/hive/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 b/shark/src/test/hive/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 similarity index 100% rename from src/test/hive/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 rename to shark/src/test/hive/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 diff --git a/src/test/hive/golden/udf6-3-e579646b969eef49b09656114da52a73 b/shark/src/test/hive/golden/udf6-3-e579646b969eef49b09656114da52a73 similarity index 100% rename from src/test/hive/golden/udf6-3-e579646b969eef49b09656114da52a73 rename to shark/src/test/hive/golden/udf6-3-e579646b969eef49b09656114da52a73 diff --git a/src/test/hive/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 b/shark/src/test/hive/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 similarity index 100% rename from src/test/hive/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 rename to shark/src/test/hive/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 diff --git a/src/test/hive/golden/udf6-5-fe336cd9850d6357980bd19139f76e b/shark/src/test/hive/golden/udf6-5-fe336cd9850d6357980bd19139f76e similarity index 100% rename from src/test/hive/golden/udf6-5-fe336cd9850d6357980bd19139f76e rename to shark/src/test/hive/golden/udf6-5-fe336cd9850d6357980bd19139f76e diff --git a/src/test/hive/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 b/shark/src/test/hive/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 similarity index 100% rename from src/test/hive/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 rename to shark/src/test/hive/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 diff --git a/src/test/hive/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 b/shark/src/test/hive/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 similarity index 100% rename from src/test/hive/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 rename to shark/src/test/hive/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 diff --git a/src/test/hive/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 b/shark/src/test/hive/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 similarity index 100% rename from src/test/hive/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 rename to shark/src/test/hive/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 diff --git a/src/test/hive/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/udf8-1-63c38297946a2060c0cff4a426b0520c b/shark/src/test/hive/golden/udf8-1-63c38297946a2060c0cff4a426b0520c similarity index 100% rename from src/test/hive/golden/udf8-1-63c38297946a2060c0cff4a426b0520c rename to shark/src/test/hive/golden/udf8-1-63c38297946a2060c0cff4a426b0520c diff --git a/src/test/hive/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 b/shark/src/test/hive/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 similarity index 100% rename from src/test/hive/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 rename to shark/src/test/hive/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 diff --git a/src/test/hive/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb b/shark/src/test/hive/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb similarity index 100% rename from src/test/hive/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb rename to shark/src/test/hive/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb diff --git a/src/test/hive/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b b/shark/src/test/hive/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b similarity index 100% rename from src/test/hive/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b rename to shark/src/test/hive/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b diff --git a/src/test/hive/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 b/shark/src/test/hive/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 similarity index 100% rename from src/test/hive/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 rename to shark/src/test/hive/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 diff --git a/src/test/hive/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c b/shark/src/test/hive/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c similarity index 100% rename from src/test/hive/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c rename to shark/src/test/hive/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c diff --git a/src/test/hive/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e b/shark/src/test/hive/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e similarity index 100% rename from src/test/hive/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e rename to shark/src/test/hive/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e diff --git a/src/test/hive/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 b/shark/src/test/hive/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 similarity index 100% rename from src/test/hive/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 rename to shark/src/test/hive/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 diff --git a/src/test/hive/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f b/shark/src/test/hive/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f similarity index 100% rename from src/test/hive/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f rename to shark/src/test/hive/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f diff --git a/src/test/hive/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c b/shark/src/test/hive/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c similarity index 100% rename from src/test/hive/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c rename to shark/src/test/hive/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c diff --git a/src/test/hive/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a b/shark/src/test/hive/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a similarity index 100% rename from src/test/hive/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a rename to shark/src/test/hive/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a diff --git a/src/test/hive/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 b/shark/src/test/hive/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 similarity index 100% rename from src/test/hive/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 rename to shark/src/test/hive/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 diff --git a/src/test/hive/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f b/shark/src/test/hive/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f similarity index 100% rename from src/test/hive/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f rename to shark/src/test/hive/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f diff --git a/src/test/hive/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c b/shark/src/test/hive/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c similarity index 100% rename from src/test/hive/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c rename to shark/src/test/hive/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c diff --git a/src/test/hive/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a b/shark/src/test/hive/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a similarity index 100% rename from src/test/hive/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a rename to shark/src/test/hive/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a diff --git a/src/test/hive/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 b/shark/src/test/hive/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 similarity index 100% rename from src/test/hive/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 rename to shark/src/test/hive/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 diff --git a/src/test/hive/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f b/shark/src/test/hive/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f similarity index 100% rename from src/test/hive/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f rename to shark/src/test/hive/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f diff --git a/src/test/hive/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 b/shark/src/test/hive/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 similarity index 100% rename from src/test/hive/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 rename to shark/src/test/hive/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 diff --git a/src/test/hive/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 b/shark/src/test/hive/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 similarity index 100% rename from src/test/hive/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 rename to shark/src/test/hive/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 diff --git a/src/test/hive/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 b/shark/src/test/hive/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 similarity index 100% rename from src/test/hive/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 rename to shark/src/test/hive/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 diff --git a/src/test/hive/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f b/shark/src/test/hive/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f similarity index 100% rename from src/test/hive/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f rename to shark/src/test/hive/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f diff --git a/src/test/hive/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 b/shark/src/test/hive/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 similarity index 100% rename from src/test/hive/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 rename to shark/src/test/hive/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 diff --git a/src/test/hive/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 b/shark/src/test/hive/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 similarity index 100% rename from src/test/hive/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 rename to shark/src/test/hive/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 diff --git a/src/test/hive/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 b/shark/src/test/hive/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 similarity index 100% rename from src/test/hive/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 rename to shark/src/test/hive/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 diff --git a/src/test/hive/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 b/shark/src/test/hive/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 similarity index 100% rename from src/test/hive/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 rename to shark/src/test/hive/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 diff --git a/src/test/hive/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 b/shark/src/test/hive/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 similarity index 100% rename from src/test/hive/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 rename to shark/src/test/hive/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 diff --git a/src/test/hive/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 b/shark/src/test/hive/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 similarity index 100% rename from src/test/hive/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 rename to shark/src/test/hive/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 diff --git a/src/test/hive/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 b/shark/src/test/hive/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 similarity index 100% rename from src/test/hive/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 rename to shark/src/test/hive/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 diff --git a/src/test/hive/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c b/shark/src/test/hive/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c similarity index 100% rename from src/test/hive/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c rename to shark/src/test/hive/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c diff --git a/src/test/hive/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 b/shark/src/test/hive/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 similarity index 100% rename from src/test/hive/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 rename to shark/src/test/hive/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 diff --git a/src/test/hive/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 b/shark/src/test/hive/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 similarity index 100% rename from src/test/hive/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 rename to shark/src/test/hive/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 diff --git a/src/test/hive/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f b/shark/src/test/hive/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f similarity index 100% rename from src/test/hive/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f rename to shark/src/test/hive/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f diff --git a/src/test/hive/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b b/shark/src/test/hive/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b similarity index 100% rename from src/test/hive/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b rename to shark/src/test/hive/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b diff --git a/src/test/hive/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 b/shark/src/test/hive/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 similarity index 100% rename from src/test/hive/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 rename to shark/src/test/hive/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 diff --git a/src/test/hive/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a b/shark/src/test/hive/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a similarity index 100% rename from src/test/hive/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a rename to shark/src/test/hive/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a diff --git a/src/test/hive/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e b/shark/src/test/hive/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e similarity index 100% rename from src/test/hive/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e rename to shark/src/test/hive/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e diff --git a/src/test/hive/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 b/shark/src/test/hive/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 similarity index 100% rename from src/test/hive/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 rename to shark/src/test/hive/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 diff --git a/src/test/hive/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 b/shark/src/test/hive/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 similarity index 100% rename from src/test/hive/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 rename to shark/src/test/hive/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 diff --git a/src/test/hive/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee b/shark/src/test/hive/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee similarity index 100% rename from src/test/hive/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee rename to shark/src/test/hive/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee diff --git a/src/test/hive/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a b/shark/src/test/hive/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a similarity index 100% rename from src/test/hive/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a rename to shark/src/test/hive/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a diff --git a/src/test/hive/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 b/shark/src/test/hive/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 similarity index 100% rename from src/test/hive/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 rename to shark/src/test/hive/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 diff --git a/src/test/hive/golden/udf_array-3-a5d12c41277fb158e09281169c905122 b/shark/src/test/hive/golden/udf_array-3-a5d12c41277fb158e09281169c905122 similarity index 100% rename from src/test/hive/golden/udf_array-3-a5d12c41277fb158e09281169c905122 rename to shark/src/test/hive/golden/udf_array-3-a5d12c41277fb158e09281169c905122 diff --git a/src/test/hive/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a b/shark/src/test/hive/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a similarity index 100% rename from src/test/hive/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a rename to shark/src/test/hive/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a diff --git a/src/test/hive/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c b/shark/src/test/hive/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c similarity index 100% rename from src/test/hive/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c rename to shark/src/test/hive/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c diff --git a/src/test/hive/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 b/shark/src/test/hive/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 similarity index 100% rename from src/test/hive/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 rename to shark/src/test/hive/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 diff --git a/src/test/hive/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 b/shark/src/test/hive/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 similarity index 100% rename from src/test/hive/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 rename to shark/src/test/hive/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 diff --git a/src/test/hive/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c b/shark/src/test/hive/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c similarity index 100% rename from src/test/hive/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c rename to shark/src/test/hive/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c diff --git a/src/test/hive/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 b/shark/src/test/hive/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 similarity index 100% rename from src/test/hive/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 rename to shark/src/test/hive/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 diff --git a/src/test/hive/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa b/shark/src/test/hive/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa similarity index 100% rename from src/test/hive/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa rename to shark/src/test/hive/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa diff --git a/src/test/hive/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 b/shark/src/test/hive/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 similarity index 100% rename from src/test/hive/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 rename to shark/src/test/hive/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 diff --git a/src/test/hive/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd b/shark/src/test/hive/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd similarity index 100% rename from src/test/hive/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd rename to shark/src/test/hive/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd diff --git a/src/test/hive/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 b/shark/src/test/hive/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 similarity index 100% rename from src/test/hive/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 rename to shark/src/test/hive/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 diff --git a/src/test/hive/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f b/shark/src/test/hive/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f similarity index 100% rename from src/test/hive/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f rename to shark/src/test/hive/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f diff --git a/src/test/hive/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 b/shark/src/test/hive/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 similarity index 100% rename from src/test/hive/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 rename to shark/src/test/hive/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 diff --git a/src/test/hive/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 b/shark/src/test/hive/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 similarity index 100% rename from src/test/hive/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 rename to shark/src/test/hive/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 diff --git a/src/test/hive/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 b/shark/src/test/hive/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 similarity index 100% rename from src/test/hive/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 rename to shark/src/test/hive/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 diff --git a/src/test/hive/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b b/shark/src/test/hive/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b similarity index 100% rename from src/test/hive/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b rename to shark/src/test/hive/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b diff --git a/src/test/hive/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 b/shark/src/test/hive/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 similarity index 100% rename from src/test/hive/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 rename to shark/src/test/hive/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 diff --git a/src/test/hive/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 b/shark/src/test/hive/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 similarity index 100% rename from src/test/hive/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 rename to shark/src/test/hive/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 diff --git a/src/test/hive/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 b/shark/src/test/hive/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 similarity index 100% rename from src/test/hive/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 rename to shark/src/test/hive/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 diff --git a/src/test/hive/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b b/shark/src/test/hive/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b similarity index 100% rename from src/test/hive/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b rename to shark/src/test/hive/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b diff --git a/src/test/hive/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 b/shark/src/test/hive/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 similarity index 100% rename from src/test/hive/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 rename to shark/src/test/hive/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 diff --git a/src/test/hive/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 b/shark/src/test/hive/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 similarity index 100% rename from src/test/hive/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 rename to shark/src/test/hive/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 diff --git a/src/test/hive/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 b/shark/src/test/hive/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 similarity index 100% rename from src/test/hive/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 rename to shark/src/test/hive/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 diff --git a/src/test/hive/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 b/shark/src/test/hive/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 similarity index 100% rename from src/test/hive/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 rename to shark/src/test/hive/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 diff --git a/src/test/hive/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f b/shark/src/test/hive/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f similarity index 100% rename from src/test/hive/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f rename to shark/src/test/hive/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f diff --git a/src/test/hive/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 b/shark/src/test/hive/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 similarity index 100% rename from src/test/hive/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 rename to shark/src/test/hive/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 diff --git a/src/test/hive/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 b/shark/src/test/hive/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 similarity index 100% rename from src/test/hive/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 rename to shark/src/test/hive/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 diff --git a/src/test/hive/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b b/shark/src/test/hive/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b similarity index 100% rename from src/test/hive/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b rename to shark/src/test/hive/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b diff --git a/src/test/hive/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 b/shark/src/test/hive/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 similarity index 100% rename from src/test/hive/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 rename to shark/src/test/hive/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 diff --git a/src/test/hive/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b b/shark/src/test/hive/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b similarity index 100% rename from src/test/hive/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b rename to shark/src/test/hive/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b diff --git a/src/test/hive/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 b/shark/src/test/hive/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 similarity index 100% rename from src/test/hive/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 rename to shark/src/test/hive/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 diff --git a/src/test/hive/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d b/shark/src/test/hive/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d similarity index 100% rename from src/test/hive/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d rename to shark/src/test/hive/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d diff --git a/src/test/hive/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 b/shark/src/test/hive/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 similarity index 100% rename from src/test/hive/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 rename to shark/src/test/hive/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 diff --git a/src/test/hive/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b b/shark/src/test/hive/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from src/test/hive/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b rename to shark/src/test/hive/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b diff --git a/src/test/hive/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 b/shark/src/test/hive/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 similarity index 100% rename from src/test/hive/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 rename to shark/src/test/hive/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 diff --git a/src/test/hive/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 b/shark/src/test/hive/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 similarity index 100% rename from src/test/hive/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 rename to shark/src/test/hive/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 diff --git a/src/test/hive/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 b/shark/src/test/hive/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 similarity index 100% rename from src/test/hive/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 rename to shark/src/test/hive/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 diff --git a/src/test/hive/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b b/shark/src/test/hive/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from src/test/hive/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b rename to shark/src/test/hive/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b diff --git a/src/test/hive/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 b/shark/src/test/hive/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 similarity index 100% rename from src/test/hive/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 rename to shark/src/test/hive/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 diff --git a/src/test/hive/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df b/shark/src/test/hive/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df similarity index 100% rename from src/test/hive/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df rename to shark/src/test/hive/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df diff --git a/src/test/hive/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c b/shark/src/test/hive/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c similarity index 100% rename from src/test/hive/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c rename to shark/src/test/hive/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c diff --git a/src/test/hive/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef b/shark/src/test/hive/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef similarity index 100% rename from src/test/hive/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef rename to shark/src/test/hive/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef diff --git a/src/test/hive/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b b/shark/src/test/hive/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from src/test/hive/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b rename to shark/src/test/hive/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b diff --git a/src/test/hive/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 b/shark/src/test/hive/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 similarity index 100% rename from src/test/hive/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 rename to shark/src/test/hive/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 diff --git a/src/test/hive/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 b/shark/src/test/hive/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 similarity index 100% rename from src/test/hive/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 rename to shark/src/test/hive/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 diff --git a/src/test/hive/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 b/shark/src/test/hive/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 similarity index 100% rename from src/test/hive/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 rename to shark/src/test/hive/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 diff --git a/src/test/hive/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b b/shark/src/test/hive/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from src/test/hive/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b rename to shark/src/test/hive/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b diff --git a/src/test/hive/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a b/shark/src/test/hive/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a similarity index 100% rename from src/test/hive/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a rename to shark/src/test/hive/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a diff --git a/src/test/hive/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b b/shark/src/test/hive/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b similarity index 100% rename from src/test/hive/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b rename to shark/src/test/hive/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b diff --git a/src/test/hive/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f b/shark/src/test/hive/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f similarity index 100% rename from src/test/hive/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f rename to shark/src/test/hive/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f diff --git a/src/test/hive/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d b/shark/src/test/hive/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d similarity index 100% rename from src/test/hive/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d rename to shark/src/test/hive/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d diff --git a/src/test/hive/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 b/shark/src/test/hive/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 similarity index 100% rename from src/test/hive/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 rename to shark/src/test/hive/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 diff --git a/src/test/hive/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb b/shark/src/test/hive/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb similarity index 100% rename from src/test/hive/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb rename to shark/src/test/hive/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb diff --git a/src/test/hive/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 b/shark/src/test/hive/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 similarity index 100% rename from src/test/hive/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 rename to shark/src/test/hive/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 diff --git a/src/test/hive/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f b/shark/src/test/hive/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f similarity index 100% rename from src/test/hive/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f rename to shark/src/test/hive/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f diff --git a/src/test/hive/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 b/shark/src/test/hive/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 similarity index 100% rename from src/test/hive/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 rename to shark/src/test/hive/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 diff --git a/src/test/hive/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e b/shark/src/test/hive/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e similarity index 100% rename from src/test/hive/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e rename to shark/src/test/hive/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e diff --git a/src/test/hive/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 b/shark/src/test/hive/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 similarity index 100% rename from src/test/hive/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 rename to shark/src/test/hive/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 diff --git a/src/test/hive/golden/udf_case-1-54acf006155d8822a50e47729be24004 b/shark/src/test/hive/golden/udf_case-1-54acf006155d8822a50e47729be24004 similarity index 100% rename from src/test/hive/golden/udf_case-1-54acf006155d8822a50e47729be24004 rename to shark/src/test/hive/golden/udf_case-1-54acf006155d8822a50e47729be24004 diff --git a/src/test/hive/golden/udf_case-2-98ee676f92950375917f09d2e492253f b/shark/src/test/hive/golden/udf_case-2-98ee676f92950375917f09d2e492253f similarity index 100% rename from src/test/hive/golden/udf_case-2-98ee676f92950375917f09d2e492253f rename to shark/src/test/hive/golden/udf_case-2-98ee676f92950375917f09d2e492253f diff --git a/src/test/hive/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c b/shark/src/test/hive/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c similarity index 100% rename from src/test/hive/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c rename to shark/src/test/hive/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c diff --git a/src/test/hive/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 b/shark/src/test/hive/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 similarity index 100% rename from src/test/hive/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 rename to shark/src/test/hive/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 diff --git a/src/test/hive/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c b/shark/src/test/hive/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c similarity index 100% rename from src/test/hive/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c rename to shark/src/test/hive/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c diff --git a/src/test/hive/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 b/shark/src/test/hive/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 similarity index 100% rename from src/test/hive/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 rename to shark/src/test/hive/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 diff --git a/src/test/hive/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 b/shark/src/test/hive/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 similarity index 100% rename from src/test/hive/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 rename to shark/src/test/hive/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 diff --git a/src/test/hive/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 b/shark/src/test/hive/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 similarity index 100% rename from src/test/hive/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 rename to shark/src/test/hive/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 diff --git a/src/test/hive/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb b/shark/src/test/hive/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb similarity index 100% rename from src/test/hive/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb rename to shark/src/test/hive/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb diff --git a/src/test/hive/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd b/shark/src/test/hive/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd similarity index 100% rename from src/test/hive/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd rename to shark/src/test/hive/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd diff --git a/src/test/hive/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df b/shark/src/test/hive/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df similarity index 100% rename from src/test/hive/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df rename to shark/src/test/hive/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df diff --git a/src/test/hive/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 b/shark/src/test/hive/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 similarity index 100% rename from src/test/hive/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 rename to shark/src/test/hive/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 diff --git a/src/test/hive/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 b/shark/src/test/hive/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 similarity index 100% rename from src/test/hive/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 rename to shark/src/test/hive/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 diff --git a/src/test/hive/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 b/shark/src/test/hive/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 similarity index 100% rename from src/test/hive/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 rename to shark/src/test/hive/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 diff --git a/src/test/hive/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 b/shark/src/test/hive/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 similarity index 100% rename from src/test/hive/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 rename to shark/src/test/hive/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 diff --git a/src/test/hive/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 b/shark/src/test/hive/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 similarity index 100% rename from src/test/hive/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 rename to shark/src/test/hive/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 diff --git a/src/test/hive/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 b/shark/src/test/hive/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 similarity index 100% rename from src/test/hive/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 rename to shark/src/test/hive/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 diff --git a/src/test/hive/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 b/shark/src/test/hive/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 similarity index 100% rename from src/test/hive/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 rename to shark/src/test/hive/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 diff --git a/src/test/hive/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 b/shark/src/test/hive/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 similarity index 100% rename from src/test/hive/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 rename to shark/src/test/hive/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 diff --git a/src/test/hive/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 b/shark/src/test/hive/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 similarity index 100% rename from src/test/hive/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 rename to shark/src/test/hive/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 diff --git a/src/test/hive/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 b/shark/src/test/hive/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 similarity index 100% rename from src/test/hive/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 rename to shark/src/test/hive/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 diff --git a/src/test/hive/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 b/shark/src/test/hive/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from src/test/hive/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 rename to shark/src/test/hive/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 diff --git a/src/test/hive/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 b/shark/src/test/hive/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 similarity index 100% rename from src/test/hive/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 rename to shark/src/test/hive/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 diff --git a/src/test/hive/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea b/shark/src/test/hive/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea similarity index 100% rename from src/test/hive/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea rename to shark/src/test/hive/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea diff --git a/src/test/hive/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 b/shark/src/test/hive/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 similarity index 100% rename from src/test/hive/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 rename to shark/src/test/hive/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 diff --git a/src/test/hive/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 b/shark/src/test/hive/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 similarity index 100% rename from src/test/hive/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 rename to shark/src/test/hive/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 diff --git a/src/test/hive/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab b/shark/src/test/hive/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab similarity index 100% rename from src/test/hive/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab rename to shark/src/test/hive/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab diff --git a/src/test/hive/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c b/shark/src/test/hive/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c similarity index 100% rename from src/test/hive/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c rename to shark/src/test/hive/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c diff --git a/src/test/hive/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 b/shark/src/test/hive/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 similarity index 100% rename from src/test/hive/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 rename to shark/src/test/hive/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 diff --git a/src/test/hive/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f b/shark/src/test/hive/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f similarity index 100% rename from src/test/hive/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f rename to shark/src/test/hive/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f diff --git a/src/test/hive/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 b/shark/src/test/hive/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 similarity index 100% rename from src/test/hive/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 rename to shark/src/test/hive/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 diff --git a/src/test/hive/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 b/shark/src/test/hive/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 similarity index 100% rename from src/test/hive/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 rename to shark/src/test/hive/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 diff --git a/src/test/hive/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c b/shark/src/test/hive/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c similarity index 100% rename from src/test/hive/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c rename to shark/src/test/hive/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c diff --git a/src/test/hive/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 b/shark/src/test/hive/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 similarity index 100% rename from src/test/hive/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 rename to shark/src/test/hive/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 diff --git a/src/test/hive/golden/udf_conv-0-d552befca345f3396464529cfde9f75a b/shark/src/test/hive/golden/udf_conv-0-d552befca345f3396464529cfde9f75a similarity index 100% rename from src/test/hive/golden/udf_conv-0-d552befca345f3396464529cfde9f75a rename to shark/src/test/hive/golden/udf_conv-0-d552befca345f3396464529cfde9f75a diff --git a/src/test/hive/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab b/shark/src/test/hive/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab similarity index 100% rename from src/test/hive/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab rename to shark/src/test/hive/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab diff --git a/src/test/hive/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 b/shark/src/test/hive/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 similarity index 100% rename from src/test/hive/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 rename to shark/src/test/hive/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 diff --git a/src/test/hive/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 b/shark/src/test/hive/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 similarity index 100% rename from src/test/hive/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 rename to shark/src/test/hive/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 diff --git a/src/test/hive/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d b/shark/src/test/hive/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d similarity index 100% rename from src/test/hive/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d rename to shark/src/test/hive/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d diff --git a/src/test/hive/golden/udf_conv-4-568e843076f358c404a8634b18541c55 b/shark/src/test/hive/golden/udf_conv-4-568e843076f358c404a8634b18541c55 similarity index 100% rename from src/test/hive/golden/udf_conv-4-568e843076f358c404a8634b18541c55 rename to shark/src/test/hive/golden/udf_conv-4-568e843076f358c404a8634b18541c55 diff --git a/src/test/hive/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 b/shark/src/test/hive/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 similarity index 100% rename from src/test/hive/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 rename to shark/src/test/hive/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 diff --git a/src/test/hive/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a b/shark/src/test/hive/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a similarity index 100% rename from src/test/hive/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a rename to shark/src/test/hive/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a diff --git a/src/test/hive/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e b/shark/src/test/hive/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e similarity index 100% rename from src/test/hive/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e rename to shark/src/test/hive/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e diff --git a/src/test/hive/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb b/shark/src/test/hive/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb similarity index 100% rename from src/test/hive/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb rename to shark/src/test/hive/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb diff --git a/src/test/hive/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 b/shark/src/test/hive/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 similarity index 100% rename from src/test/hive/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 rename to shark/src/test/hive/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 diff --git a/src/test/hive/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 b/shark/src/test/hive/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 similarity index 100% rename from src/test/hive/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 rename to shark/src/test/hive/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 diff --git a/src/test/hive/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 b/shark/src/test/hive/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 similarity index 100% rename from src/test/hive/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 rename to shark/src/test/hive/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 diff --git a/src/test/hive/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 b/shark/src/test/hive/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 similarity index 100% rename from src/test/hive/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 rename to shark/src/test/hive/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 diff --git a/src/test/hive/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 b/shark/src/test/hive/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 similarity index 100% rename from src/test/hive/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 rename to shark/src/test/hive/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 diff --git a/src/test/hive/golden/udf_count-0-534a9b25b87d09e418645b1216949560 b/shark/src/test/hive/golden/udf_count-0-534a9b25b87d09e418645b1216949560 similarity index 100% rename from src/test/hive/golden/udf_count-0-534a9b25b87d09e418645b1216949560 rename to shark/src/test/hive/golden/udf_count-0-534a9b25b87d09e418645b1216949560 diff --git a/src/test/hive/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f b/shark/src/test/hive/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f similarity index 100% rename from src/test/hive/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f rename to shark/src/test/hive/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f diff --git a/src/test/hive/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 b/shark/src/test/hive/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 similarity index 100% rename from src/test/hive/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 rename to shark/src/test/hive/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 diff --git a/src/test/hive/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e b/shark/src/test/hive/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e similarity index 100% rename from src/test/hive/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e rename to shark/src/test/hive/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e diff --git a/src/test/hive/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e b/shark/src/test/hive/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e similarity index 100% rename from src/test/hive/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e rename to shark/src/test/hive/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e diff --git a/src/test/hive/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 b/shark/src/test/hive/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 similarity index 100% rename from src/test/hive/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 rename to shark/src/test/hive/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 diff --git a/src/test/hive/golden/udf_count-2-3344645abe535426307b9327b381fc85 b/shark/src/test/hive/golden/udf_count-2-3344645abe535426307b9327b381fc85 similarity index 100% rename from src/test/hive/golden/udf_count-2-3344645abe535426307b9327b381fc85 rename to shark/src/test/hive/golden/udf_count-2-3344645abe535426307b9327b381fc85 diff --git a/src/test/hive/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 b/shark/src/test/hive/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 similarity index 100% rename from src/test/hive/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 rename to shark/src/test/hive/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 diff --git a/src/test/hive/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 b/shark/src/test/hive/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 similarity index 100% rename from src/test/hive/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 rename to shark/src/test/hive/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 diff --git a/src/test/hive/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 b/shark/src/test/hive/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 similarity index 100% rename from src/test/hive/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 rename to shark/src/test/hive/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 diff --git a/src/test/hive/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 b/shark/src/test/hive/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 similarity index 100% rename from src/test/hive/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 rename to shark/src/test/hive/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 diff --git a/src/test/hive/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 b/shark/src/test/hive/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 similarity index 100% rename from src/test/hive/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 rename to shark/src/test/hive/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 diff --git a/src/test/hive/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 b/shark/src/test/hive/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 similarity index 100% rename from src/test/hive/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 rename to shark/src/test/hive/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 diff --git a/src/test/hive/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 b/shark/src/test/hive/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 similarity index 100% rename from src/test/hive/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 rename to shark/src/test/hive/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 diff --git a/src/test/hive/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 b/shark/src/test/hive/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 similarity index 100% rename from src/test/hive/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 rename to shark/src/test/hive/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 diff --git a/src/test/hive/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 b/shark/src/test/hive/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 similarity index 100% rename from src/test/hive/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 rename to shark/src/test/hive/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 diff --git a/src/test/hive/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 b/shark/src/test/hive/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 similarity index 100% rename from src/test/hive/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 rename to shark/src/test/hive/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 diff --git a/src/test/hive/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 b/shark/src/test/hive/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 similarity index 100% rename from src/test/hive/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 rename to shark/src/test/hive/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 diff --git a/src/test/hive/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 b/shark/src/test/hive/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 similarity index 100% rename from src/test/hive/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 rename to shark/src/test/hive/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 diff --git a/src/test/hive/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 b/shark/src/test/hive/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 similarity index 100% rename from src/test/hive/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 rename to shark/src/test/hive/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 diff --git a/src/test/hive/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 b/shark/src/test/hive/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 similarity index 100% rename from src/test/hive/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 rename to shark/src/test/hive/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 diff --git a/src/test/hive/golden/udf_day-1-87168babe1110fe4c38269843414ca4 b/shark/src/test/hive/golden/udf_day-1-87168babe1110fe4c38269843414ca4 similarity index 100% rename from src/test/hive/golden/udf_day-1-87168babe1110fe4c38269843414ca4 rename to shark/src/test/hive/golden/udf_day-1-87168babe1110fe4c38269843414ca4 diff --git a/src/test/hive/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 b/shark/src/test/hive/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 similarity index 100% rename from src/test/hive/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 rename to shark/src/test/hive/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 diff --git a/src/test/hive/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 b/shark/src/test/hive/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 similarity index 100% rename from src/test/hive/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 rename to shark/src/test/hive/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 diff --git a/src/test/hive/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae b/shark/src/test/hive/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae similarity index 100% rename from src/test/hive/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae rename to shark/src/test/hive/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae diff --git a/src/test/hive/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 b/shark/src/test/hive/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 similarity index 100% rename from src/test/hive/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 rename to shark/src/test/hive/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 diff --git a/src/test/hive/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b b/shark/src/test/hive/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b similarity index 100% rename from src/test/hive/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b rename to shark/src/test/hive/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b diff --git a/src/test/hive/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/shark/src/test/hive/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 similarity index 100% rename from src/test/hive/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 rename to shark/src/test/hive/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 diff --git a/src/test/hive/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae b/shark/src/test/hive/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae similarity index 100% rename from src/test/hive/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae rename to shark/src/test/hive/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae diff --git a/src/test/hive/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 b/shark/src/test/hive/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 similarity index 100% rename from src/test/hive/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 rename to shark/src/test/hive/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 diff --git a/src/test/hive/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b b/shark/src/test/hive/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b similarity index 100% rename from src/test/hive/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b rename to shark/src/test/hive/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b diff --git a/src/test/hive/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/shark/src/test/hive/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 similarity index 100% rename from src/test/hive/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 rename to shark/src/test/hive/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 diff --git a/src/test/hive/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db b/shark/src/test/hive/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db similarity index 100% rename from src/test/hive/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db rename to shark/src/test/hive/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db diff --git a/src/test/hive/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 b/shark/src/test/hive/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 similarity index 100% rename from src/test/hive/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 rename to shark/src/test/hive/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 diff --git a/src/test/hive/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b b/shark/src/test/hive/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b similarity index 100% rename from src/test/hive/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b rename to shark/src/test/hive/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b diff --git a/src/test/hive/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 b/shark/src/test/hive/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 similarity index 100% rename from src/test/hive/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 rename to shark/src/test/hive/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 diff --git a/src/test/hive/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 b/shark/src/test/hive/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 similarity index 100% rename from src/test/hive/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 rename to shark/src/test/hive/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 diff --git a/src/test/hive/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 b/shark/src/test/hive/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 similarity index 100% rename from src/test/hive/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 rename to shark/src/test/hive/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 diff --git a/src/test/hive/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 b/shark/src/test/hive/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 similarity index 100% rename from src/test/hive/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 rename to shark/src/test/hive/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 diff --git a/src/test/hive/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 b/shark/src/test/hive/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 similarity index 100% rename from src/test/hive/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 rename to shark/src/test/hive/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 diff --git a/src/test/hive/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f b/shark/src/test/hive/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f similarity index 100% rename from src/test/hive/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f rename to shark/src/test/hive/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f diff --git a/src/test/hive/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 b/shark/src/test/hive/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 similarity index 100% rename from src/test/hive/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 rename to shark/src/test/hive/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 diff --git a/src/test/hive/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a b/shark/src/test/hive/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a similarity index 100% rename from src/test/hive/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a rename to shark/src/test/hive/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a diff --git a/src/test/hive/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 b/shark/src/test/hive/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 similarity index 100% rename from src/test/hive/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 rename to shark/src/test/hive/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 diff --git a/src/test/hive/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a b/shark/src/test/hive/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a similarity index 100% rename from src/test/hive/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a rename to shark/src/test/hive/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a diff --git a/src/test/hive/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 b/shark/src/test/hive/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 similarity index 100% rename from src/test/hive/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 rename to shark/src/test/hive/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 diff --git a/src/test/hive/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 b/shark/src/test/hive/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 similarity index 100% rename from src/test/hive/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 rename to shark/src/test/hive/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 diff --git a/src/test/hive/golden/udf_field-1-379d8580693493f620a8f4084709324f b/shark/src/test/hive/golden/udf_field-1-379d8580693493f620a8f4084709324f similarity index 100% rename from src/test/hive/golden/udf_field-1-379d8580693493f620a8f4084709324f rename to shark/src/test/hive/golden/udf_field-1-379d8580693493f620a8f4084709324f diff --git a/src/test/hive/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 b/shark/src/test/hive/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 similarity index 100% rename from src/test/hive/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 rename to shark/src/test/hive/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 diff --git a/src/test/hive/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b b/shark/src/test/hive/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b similarity index 100% rename from src/test/hive/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b rename to shark/src/test/hive/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b diff --git a/src/test/hive/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 b/shark/src/test/hive/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 similarity index 100% rename from src/test/hive/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 rename to shark/src/test/hive/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 diff --git a/src/test/hive/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 b/shark/src/test/hive/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 similarity index 100% rename from src/test/hive/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 rename to shark/src/test/hive/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 diff --git a/src/test/hive/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 b/shark/src/test/hive/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 similarity index 100% rename from src/test/hive/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 rename to shark/src/test/hive/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 diff --git a/src/test/hive/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 b/shark/src/test/hive/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 similarity index 100% rename from src/test/hive/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 rename to shark/src/test/hive/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 diff --git a/src/test/hive/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef b/shark/src/test/hive/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef similarity index 100% rename from src/test/hive/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef rename to shark/src/test/hive/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef diff --git a/src/test/hive/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 b/shark/src/test/hive/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 similarity index 100% rename from src/test/hive/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 rename to shark/src/test/hive/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 diff --git a/src/test/hive/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 b/shark/src/test/hive/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 similarity index 100% rename from src/test/hive/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 rename to shark/src/test/hive/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 diff --git a/src/test/hive/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 b/shark/src/test/hive/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 rename to shark/src/test/hive/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 diff --git a/src/test/hive/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 b/shark/src/test/hive/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 rename to shark/src/test/hive/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 diff --git a/src/test/hive/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c b/shark/src/test/hive/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c similarity index 100% rename from src/test/hive/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c rename to shark/src/test/hive/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c diff --git a/src/test/hive/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a b/shark/src/test/hive/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a similarity index 100% rename from src/test/hive/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a rename to shark/src/test/hive/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a diff --git a/src/test/hive/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 b/shark/src/test/hive/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 rename to shark/src/test/hive/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 diff --git a/src/test/hive/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a b/shark/src/test/hive/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a similarity index 100% rename from src/test/hive/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a rename to shark/src/test/hive/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a diff --git a/src/test/hive/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 b/shark/src/test/hive/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 rename to shark/src/test/hive/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 diff --git a/src/test/hive/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb b/shark/src/test/hive/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb similarity index 100% rename from src/test/hive/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb rename to shark/src/test/hive/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb diff --git a/src/test/hive/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 b/shark/src/test/hive/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 rename to shark/src/test/hive/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 diff --git a/src/test/hive/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 b/shark/src/test/hive/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 rename to shark/src/test/hive/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 diff --git a/src/test/hive/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 b/shark/src/test/hive/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 rename to shark/src/test/hive/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 diff --git a/src/test/hive/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e b/shark/src/test/hive/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e similarity index 100% rename from src/test/hive/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e rename to shark/src/test/hive/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e diff --git a/src/test/hive/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b b/shark/src/test/hive/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b similarity index 100% rename from src/test/hive/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b rename to shark/src/test/hive/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b diff --git a/src/test/hive/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f b/shark/src/test/hive/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f similarity index 100% rename from src/test/hive/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f rename to shark/src/test/hive/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f diff --git a/src/test/hive/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 b/shark/src/test/hive/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 rename to shark/src/test/hive/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 diff --git a/src/test/hive/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 b/shark/src/test/hive/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 rename to shark/src/test/hive/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 diff --git a/src/test/hive/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 b/shark/src/test/hive/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 rename to shark/src/test/hive/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 diff --git a/src/test/hive/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 b/shark/src/test/hive/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 similarity index 100% rename from src/test/hive/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 rename to shark/src/test/hive/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 diff --git a/src/test/hive/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 b/shark/src/test/hive/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 similarity index 100% rename from src/test/hive/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 rename to shark/src/test/hive/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 diff --git a/src/test/hive/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b b/shark/src/test/hive/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b similarity index 100% rename from src/test/hive/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b rename to shark/src/test/hive/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b diff --git a/src/test/hive/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 b/shark/src/test/hive/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 similarity index 100% rename from src/test/hive/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 rename to shark/src/test/hive/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 diff --git a/src/test/hive/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 b/shark/src/test/hive/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 similarity index 100% rename from src/test/hive/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 rename to shark/src/test/hive/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 diff --git a/src/test/hive/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb b/shark/src/test/hive/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb similarity index 100% rename from src/test/hive/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb rename to shark/src/test/hive/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb diff --git a/src/test/hive/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 b/shark/src/test/hive/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 similarity index 100% rename from src/test/hive/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 rename to shark/src/test/hive/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 diff --git a/src/test/hive/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd b/shark/src/test/hive/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd similarity index 100% rename from src/test/hive/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd rename to shark/src/test/hive/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd diff --git a/src/test/hive/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c b/shark/src/test/hive/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c similarity index 100% rename from src/test/hive/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c rename to shark/src/test/hive/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c diff --git a/src/test/hive/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 b/shark/src/test/hive/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 similarity index 100% rename from src/test/hive/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 rename to shark/src/test/hive/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 diff --git a/src/test/hive/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 b/shark/src/test/hive/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 similarity index 100% rename from src/test/hive/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 rename to shark/src/test/hive/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 diff --git a/src/test/hive/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 b/shark/src/test/hive/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 similarity index 100% rename from src/test/hive/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 rename to shark/src/test/hive/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 diff --git a/src/test/hive/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 b/shark/src/test/hive/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 similarity index 100% rename from src/test/hive/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 rename to shark/src/test/hive/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 diff --git a/src/test/hive/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e b/shark/src/test/hive/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e similarity index 100% rename from src/test/hive/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e rename to shark/src/test/hive/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e diff --git a/src/test/hive/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 b/shark/src/test/hive/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 similarity index 100% rename from src/test/hive/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 rename to shark/src/test/hive/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 diff --git a/src/test/hive/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f b/shark/src/test/hive/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f similarity index 100% rename from src/test/hive/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f rename to shark/src/test/hive/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f diff --git a/src/test/hive/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 b/shark/src/test/hive/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 similarity index 100% rename from src/test/hive/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 rename to shark/src/test/hive/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 diff --git a/src/test/hive/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f b/shark/src/test/hive/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f similarity index 100% rename from src/test/hive/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f rename to shark/src/test/hive/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f diff --git a/src/test/hive/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 b/shark/src/test/hive/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 similarity index 100% rename from src/test/hive/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 rename to shark/src/test/hive/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 diff --git a/src/test/hive/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 b/shark/src/test/hive/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 similarity index 100% rename from src/test/hive/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 rename to shark/src/test/hive/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 diff --git a/src/test/hive/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 b/shark/src/test/hive/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 similarity index 100% rename from src/test/hive/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 rename to shark/src/test/hive/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 diff --git a/src/test/hive/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 b/shark/src/test/hive/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 similarity index 100% rename from src/test/hive/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 rename to shark/src/test/hive/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 diff --git a/src/test/hive/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 b/shark/src/test/hive/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 similarity index 100% rename from src/test/hive/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 rename to shark/src/test/hive/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 diff --git a/src/test/hive/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c b/shark/src/test/hive/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c similarity index 100% rename from src/test/hive/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c rename to shark/src/test/hive/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c diff --git a/src/test/hive/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f b/shark/src/test/hive/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f similarity index 100% rename from src/test/hive/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f rename to shark/src/test/hive/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f diff --git a/src/test/hive/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a b/shark/src/test/hive/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a similarity index 100% rename from src/test/hive/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a rename to shark/src/test/hive/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a diff --git a/src/test/hive/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 b/shark/src/test/hive/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 similarity index 100% rename from src/test/hive/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 rename to shark/src/test/hive/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 diff --git a/src/test/hive/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 b/shark/src/test/hive/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 similarity index 100% rename from src/test/hive/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 rename to shark/src/test/hive/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 diff --git a/src/test/hive/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 b/shark/src/test/hive/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 similarity index 100% rename from src/test/hive/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 rename to shark/src/test/hive/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 diff --git a/src/test/hive/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a b/shark/src/test/hive/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a similarity index 100% rename from src/test/hive/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a rename to shark/src/test/hive/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a diff --git a/src/test/hive/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 b/shark/src/test/hive/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 similarity index 100% rename from src/test/hive/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 rename to shark/src/test/hive/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 diff --git a/src/test/hive/golden/udf_hash-3-23991312391d518aacf3d4469c816eae b/shark/src/test/hive/golden/udf_hash-3-23991312391d518aacf3d4469c816eae similarity index 100% rename from src/test/hive/golden/udf_hash-3-23991312391d518aacf3d4469c816eae rename to shark/src/test/hive/golden/udf_hash-3-23991312391d518aacf3d4469c816eae diff --git a/src/test/hive/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 b/shark/src/test/hive/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 similarity index 100% rename from src/test/hive/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 rename to shark/src/test/hive/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 diff --git a/src/test/hive/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c b/shark/src/test/hive/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c similarity index 100% rename from src/test/hive/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c rename to shark/src/test/hive/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c diff --git a/src/test/hive/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 b/shark/src/test/hive/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 similarity index 100% rename from src/test/hive/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 rename to shark/src/test/hive/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 diff --git a/src/test/hive/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf b/shark/src/test/hive/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf similarity index 100% rename from src/test/hive/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf rename to shark/src/test/hive/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf diff --git a/src/test/hive/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea b/shark/src/test/hive/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea similarity index 100% rename from src/test/hive/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea rename to shark/src/test/hive/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea diff --git a/src/test/hive/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce b/shark/src/test/hive/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce similarity index 100% rename from src/test/hive/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce rename to shark/src/test/hive/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce diff --git a/src/test/hive/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 b/shark/src/test/hive/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 similarity index 100% rename from src/test/hive/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 rename to shark/src/test/hive/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 diff --git a/src/test/hive/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a b/shark/src/test/hive/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a similarity index 100% rename from src/test/hive/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a rename to shark/src/test/hive/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a diff --git a/src/test/hive/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 b/shark/src/test/hive/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 similarity index 100% rename from src/test/hive/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 rename to shark/src/test/hive/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 diff --git a/src/test/hive/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c b/shark/src/test/hive/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c similarity index 100% rename from src/test/hive/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c rename to shark/src/test/hive/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c diff --git a/src/test/hive/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a b/shark/src/test/hive/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a similarity index 100% rename from src/test/hive/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a rename to shark/src/test/hive/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a diff --git a/src/test/hive/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 b/shark/src/test/hive/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 similarity index 100% rename from src/test/hive/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 rename to shark/src/test/hive/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 diff --git a/src/test/hive/golden/udf_if-3-20206f17367ff284d67044abd745ce9f b/shark/src/test/hive/golden/udf_if-3-20206f17367ff284d67044abd745ce9f similarity index 100% rename from src/test/hive/golden/udf_if-3-20206f17367ff284d67044abd745ce9f rename to shark/src/test/hive/golden/udf_if-3-20206f17367ff284d67044abd745ce9f diff --git a/src/test/hive/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca b/shark/src/test/hive/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca similarity index 100% rename from src/test/hive/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca rename to shark/src/test/hive/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca diff --git a/src/test/hive/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc b/shark/src/test/hive/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc similarity index 100% rename from src/test/hive/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc rename to shark/src/test/hive/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc diff --git a/src/test/hive/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 b/shark/src/test/hive/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 similarity index 100% rename from src/test/hive/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 rename to shark/src/test/hive/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 diff --git a/src/test/hive/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 b/shark/src/test/hive/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 similarity index 100% rename from src/test/hive/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 rename to shark/src/test/hive/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 diff --git a/src/test/hive/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 b/shark/src/test/hive/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 similarity index 100% rename from src/test/hive/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 rename to shark/src/test/hive/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 diff --git a/src/test/hive/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 b/shark/src/test/hive/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 similarity index 100% rename from src/test/hive/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 rename to shark/src/test/hive/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 diff --git a/src/test/hive/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 b/shark/src/test/hive/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 similarity index 100% rename from src/test/hive/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 rename to shark/src/test/hive/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 diff --git a/src/test/hive/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 b/shark/src/test/hive/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 similarity index 100% rename from src/test/hive/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 rename to shark/src/test/hive/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 diff --git a/src/test/hive/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 b/shark/src/test/hive/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 similarity index 100% rename from src/test/hive/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 rename to shark/src/test/hive/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 diff --git a/src/test/hive/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 b/shark/src/test/hive/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 similarity index 100% rename from src/test/hive/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 rename to shark/src/test/hive/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 diff --git a/src/test/hive/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 b/shark/src/test/hive/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 similarity index 100% rename from src/test/hive/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 rename to shark/src/test/hive/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 diff --git a/src/test/hive/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 b/shark/src/test/hive/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 similarity index 100% rename from src/test/hive/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 rename to shark/src/test/hive/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 diff --git a/src/test/hive/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba b/shark/src/test/hive/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba similarity index 100% rename from src/test/hive/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba rename to shark/src/test/hive/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba diff --git a/src/test/hive/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 b/shark/src/test/hive/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 similarity index 100% rename from src/test/hive/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 rename to shark/src/test/hive/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 diff --git a/src/test/hive/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 b/shark/src/test/hive/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 similarity index 100% rename from src/test/hive/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 rename to shark/src/test/hive/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 diff --git a/src/test/hive/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 b/shark/src/test/hive/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 similarity index 100% rename from src/test/hive/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 rename to shark/src/test/hive/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 diff --git a/src/test/hive/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 b/shark/src/test/hive/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 similarity index 100% rename from src/test/hive/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 rename to shark/src/test/hive/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 diff --git a/src/test/hive/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 b/shark/src/test/hive/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 similarity index 100% rename from src/test/hive/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 rename to shark/src/test/hive/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 diff --git a/src/test/hive/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 b/shark/src/test/hive/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 similarity index 100% rename from src/test/hive/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 rename to shark/src/test/hive/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 diff --git a/src/test/hive/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 b/shark/src/test/hive/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 similarity index 100% rename from src/test/hive/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 rename to shark/src/test/hive/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 diff --git a/src/test/hive/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 b/shark/src/test/hive/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 similarity index 100% rename from src/test/hive/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 rename to shark/src/test/hive/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 diff --git a/src/test/hive/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 b/shark/src/test/hive/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 similarity index 100% rename from src/test/hive/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 rename to shark/src/test/hive/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 diff --git a/src/test/hive/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 b/shark/src/test/hive/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 similarity index 100% rename from src/test/hive/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 rename to shark/src/test/hive/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 diff --git a/src/test/hive/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 b/shark/src/test/hive/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 similarity index 100% rename from src/test/hive/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 rename to shark/src/test/hive/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 diff --git a/src/test/hive/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 b/shark/src/test/hive/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 similarity index 100% rename from src/test/hive/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 rename to shark/src/test/hive/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 diff --git a/src/test/hive/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b b/shark/src/test/hive/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b similarity index 100% rename from src/test/hive/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b rename to shark/src/test/hive/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b diff --git a/src/test/hive/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 b/shark/src/test/hive/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 similarity index 100% rename from src/test/hive/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 rename to shark/src/test/hive/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 diff --git a/src/test/hive/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a b/shark/src/test/hive/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a similarity index 100% rename from src/test/hive/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a rename to shark/src/test/hive/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a diff --git a/src/test/hive/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 b/shark/src/test/hive/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 similarity index 100% rename from src/test/hive/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 rename to shark/src/test/hive/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 diff --git a/src/test/hive/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd b/shark/src/test/hive/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd similarity index 100% rename from src/test/hive/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd rename to shark/src/test/hive/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd diff --git a/src/test/hive/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad b/shark/src/test/hive/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad similarity index 100% rename from src/test/hive/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad rename to shark/src/test/hive/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad diff --git a/src/test/hive/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e b/shark/src/test/hive/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e similarity index 100% rename from src/test/hive/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e rename to shark/src/test/hive/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e diff --git a/src/test/hive/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b b/shark/src/test/hive/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b similarity index 100% rename from src/test/hive/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b rename to shark/src/test/hive/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b diff --git a/src/test/hive/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 b/shark/src/test/hive/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 similarity index 100% rename from src/test/hive/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 rename to shark/src/test/hive/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 diff --git a/src/test/hive/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 b/shark/src/test/hive/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 similarity index 100% rename from src/test/hive/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 rename to shark/src/test/hive/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 diff --git a/src/test/hive/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 b/shark/src/test/hive/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 similarity index 100% rename from src/test/hive/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 rename to shark/src/test/hive/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 diff --git a/src/test/hive/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d b/shark/src/test/hive/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d similarity index 100% rename from src/test/hive/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d rename to shark/src/test/hive/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d diff --git a/src/test/hive/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 b/shark/src/test/hive/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 similarity index 100% rename from src/test/hive/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 rename to shark/src/test/hive/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 diff --git a/src/test/hive/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f b/shark/src/test/hive/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f similarity index 100% rename from src/test/hive/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f rename to shark/src/test/hive/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f diff --git a/src/test/hive/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a b/shark/src/test/hive/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a similarity index 100% rename from src/test/hive/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a rename to shark/src/test/hive/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a diff --git a/src/test/hive/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f b/shark/src/test/hive/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f similarity index 100% rename from src/test/hive/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f rename to shark/src/test/hive/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f diff --git a/src/test/hive/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf b/shark/src/test/hive/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf similarity index 100% rename from src/test/hive/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf rename to shark/src/test/hive/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf diff --git a/src/test/hive/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 b/shark/src/test/hive/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 similarity index 100% rename from src/test/hive/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 rename to shark/src/test/hive/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 diff --git a/src/test/hive/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 b/shark/src/test/hive/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 similarity index 100% rename from src/test/hive/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 rename to shark/src/test/hive/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 diff --git a/src/test/hive/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 b/shark/src/test/hive/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 similarity index 100% rename from src/test/hive/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 rename to shark/src/test/hive/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 diff --git a/src/test/hive/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 b/shark/src/test/hive/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 similarity index 100% rename from src/test/hive/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 rename to shark/src/test/hive/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 diff --git a/src/test/hive/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b b/shark/src/test/hive/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b similarity index 100% rename from src/test/hive/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b rename to shark/src/test/hive/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b diff --git a/src/test/hive/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 b/shark/src/test/hive/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 similarity index 100% rename from src/test/hive/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 rename to shark/src/test/hive/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 diff --git a/src/test/hive/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c b/shark/src/test/hive/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c similarity index 100% rename from src/test/hive/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c rename to shark/src/test/hive/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c diff --git a/src/test/hive/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 b/shark/src/test/hive/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 similarity index 100% rename from src/test/hive/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 rename to shark/src/test/hive/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 diff --git a/src/test/hive/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a b/shark/src/test/hive/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a similarity index 100% rename from src/test/hive/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a rename to shark/src/test/hive/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a diff --git a/src/test/hive/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 b/shark/src/test/hive/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 similarity index 100% rename from src/test/hive/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 rename to shark/src/test/hive/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 diff --git a/src/test/hive/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc b/shark/src/test/hive/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc similarity index 100% rename from src/test/hive/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc rename to shark/src/test/hive/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc diff --git a/src/test/hive/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 b/shark/src/test/hive/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 similarity index 100% rename from src/test/hive/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 rename to shark/src/test/hive/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 diff --git a/src/test/hive/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 b/shark/src/test/hive/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 similarity index 100% rename from src/test/hive/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 rename to shark/src/test/hive/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 diff --git a/src/test/hive/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 b/shark/src/test/hive/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 similarity index 100% rename from src/test/hive/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 rename to shark/src/test/hive/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 diff --git a/src/test/hive/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e b/shark/src/test/hive/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e similarity index 100% rename from src/test/hive/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e rename to shark/src/test/hive/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e diff --git a/src/test/hive/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 b/shark/src/test/hive/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 similarity index 100% rename from src/test/hive/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 rename to shark/src/test/hive/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 diff --git a/src/test/hive/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c b/shark/src/test/hive/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c similarity index 100% rename from src/test/hive/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c rename to shark/src/test/hive/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c diff --git a/src/test/hive/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 b/shark/src/test/hive/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 similarity index 100% rename from src/test/hive/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 rename to shark/src/test/hive/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 diff --git a/src/test/hive/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a b/shark/src/test/hive/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a similarity index 100% rename from src/test/hive/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a rename to shark/src/test/hive/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a diff --git a/src/test/hive/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 b/shark/src/test/hive/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 similarity index 100% rename from src/test/hive/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 rename to shark/src/test/hive/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 diff --git a/src/test/hive/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f b/shark/src/test/hive/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f similarity index 100% rename from src/test/hive/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f rename to shark/src/test/hive/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f diff --git a/src/test/hive/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 b/shark/src/test/hive/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 similarity index 100% rename from src/test/hive/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 rename to shark/src/test/hive/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 diff --git a/src/test/hive/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 b/shark/src/test/hive/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 similarity index 100% rename from src/test/hive/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 rename to shark/src/test/hive/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 diff --git a/src/test/hive/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 b/shark/src/test/hive/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 similarity index 100% rename from src/test/hive/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 rename to shark/src/test/hive/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 diff --git a/src/test/hive/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 b/shark/src/test/hive/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 similarity index 100% rename from src/test/hive/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 rename to shark/src/test/hive/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 diff --git a/src/test/hive/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 b/shark/src/test/hive/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 similarity index 100% rename from src/test/hive/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 rename to shark/src/test/hive/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 diff --git a/src/test/hive/golden/udf_lower-1-550f0a6da388596a775d921b9da995c b/shark/src/test/hive/golden/udf_lower-1-550f0a6da388596a775d921b9da995c similarity index 100% rename from src/test/hive/golden/udf_lower-1-550f0a6da388596a775d921b9da995c rename to shark/src/test/hive/golden/udf_lower-1-550f0a6da388596a775d921b9da995c diff --git a/src/test/hive/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e b/shark/src/test/hive/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e similarity index 100% rename from src/test/hive/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e rename to shark/src/test/hive/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e diff --git a/src/test/hive/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 b/shark/src/test/hive/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 similarity index 100% rename from src/test/hive/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 rename to shark/src/test/hive/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 diff --git a/src/test/hive/golden/udf_lpad-0-995646acf1e23cea7825412915921bef b/shark/src/test/hive/golden/udf_lpad-0-995646acf1e23cea7825412915921bef similarity index 100% rename from src/test/hive/golden/udf_lpad-0-995646acf1e23cea7825412915921bef rename to shark/src/test/hive/golden/udf_lpad-0-995646acf1e23cea7825412915921bef diff --git a/src/test/hive/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 b/shark/src/test/hive/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 similarity index 100% rename from src/test/hive/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 rename to shark/src/test/hive/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 diff --git a/src/test/hive/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 b/shark/src/test/hive/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 similarity index 100% rename from src/test/hive/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 rename to shark/src/test/hive/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 diff --git a/src/test/hive/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae b/shark/src/test/hive/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae similarity index 100% rename from src/test/hive/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae rename to shark/src/test/hive/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae diff --git a/src/test/hive/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f b/shark/src/test/hive/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f similarity index 100% rename from src/test/hive/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f rename to shark/src/test/hive/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f diff --git a/src/test/hive/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 b/shark/src/test/hive/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 similarity index 100% rename from src/test/hive/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 rename to shark/src/test/hive/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 diff --git a/src/test/hive/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e b/shark/src/test/hive/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e similarity index 100% rename from src/test/hive/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e rename to shark/src/test/hive/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e diff --git a/src/test/hive/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf b/shark/src/test/hive/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf similarity index 100% rename from src/test/hive/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf rename to shark/src/test/hive/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf diff --git a/src/test/hive/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 b/shark/src/test/hive/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 similarity index 100% rename from src/test/hive/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 rename to shark/src/test/hive/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 diff --git a/src/test/hive/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 b/shark/src/test/hive/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 similarity index 100% rename from src/test/hive/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 rename to shark/src/test/hive/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 diff --git a/src/test/hive/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb b/shark/src/test/hive/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb similarity index 100% rename from src/test/hive/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb rename to shark/src/test/hive/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb diff --git a/src/test/hive/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa b/shark/src/test/hive/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa similarity index 100% rename from src/test/hive/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa rename to shark/src/test/hive/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa diff --git a/src/test/hive/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f b/shark/src/test/hive/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f similarity index 100% rename from src/test/hive/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f rename to shark/src/test/hive/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f diff --git a/src/test/hive/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e b/shark/src/test/hive/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e similarity index 100% rename from src/test/hive/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e rename to shark/src/test/hive/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e diff --git a/src/test/hive/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a b/shark/src/test/hive/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a similarity index 100% rename from src/test/hive/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a rename to shark/src/test/hive/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a diff --git a/src/test/hive/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb b/shark/src/test/hive/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb similarity index 100% rename from src/test/hive/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb rename to shark/src/test/hive/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb diff --git a/src/test/hive/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 b/shark/src/test/hive/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 similarity index 100% rename from src/test/hive/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 rename to shark/src/test/hive/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 diff --git a/src/test/hive/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 b/shark/src/test/hive/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 similarity index 100% rename from src/test/hive/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 rename to shark/src/test/hive/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 diff --git a/src/test/hive/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 b/shark/src/test/hive/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 similarity index 100% rename from src/test/hive/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 rename to shark/src/test/hive/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 diff --git a/src/test/hive/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e b/shark/src/test/hive/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e similarity index 100% rename from src/test/hive/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e rename to shark/src/test/hive/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e diff --git a/src/test/hive/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f b/shark/src/test/hive/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f similarity index 100% rename from src/test/hive/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f rename to shark/src/test/hive/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f diff --git a/src/test/hive/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 b/shark/src/test/hive/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 similarity index 100% rename from src/test/hive/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 rename to shark/src/test/hive/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 diff --git a/src/test/hive/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 b/shark/src/test/hive/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 similarity index 100% rename from src/test/hive/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 rename to shark/src/test/hive/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 diff --git a/src/test/hive/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 b/shark/src/test/hive/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from src/test/hive/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 rename to shark/src/test/hive/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/src/test/hive/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 b/shark/src/test/hive/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 similarity index 100% rename from src/test/hive/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 rename to shark/src/test/hive/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 diff --git a/src/test/hive/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 b/shark/src/test/hive/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from src/test/hive/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 rename to shark/src/test/hive/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/src/test/hive/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e b/shark/src/test/hive/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from src/test/hive/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e rename to shark/src/test/hive/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e diff --git a/src/test/hive/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 b/shark/src/test/hive/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 similarity index 100% rename from src/test/hive/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 rename to shark/src/test/hive/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 diff --git a/src/test/hive/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e b/shark/src/test/hive/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from src/test/hive/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e rename to shark/src/test/hive/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e diff --git a/src/test/hive/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 b/shark/src/test/hive/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 similarity index 100% rename from src/test/hive/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 rename to shark/src/test/hive/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 diff --git a/src/test/hive/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 b/shark/src/test/hive/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from src/test/hive/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 rename to shark/src/test/hive/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/src/test/hive/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 b/shark/src/test/hive/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from src/test/hive/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 rename to shark/src/test/hive/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/src/test/hive/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 b/shark/src/test/hive/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 similarity index 100% rename from src/test/hive/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 rename to shark/src/test/hive/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 diff --git a/src/test/hive/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 b/shark/src/test/hive/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 similarity index 100% rename from src/test/hive/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 rename to shark/src/test/hive/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 diff --git a/src/test/hive/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 b/shark/src/test/hive/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 similarity index 100% rename from src/test/hive/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 rename to shark/src/test/hive/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 diff --git a/src/test/hive/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 b/shark/src/test/hive/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from src/test/hive/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 rename to shark/src/test/hive/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/src/test/hive/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 b/shark/src/test/hive/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 similarity index 100% rename from src/test/hive/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 rename to shark/src/test/hive/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 diff --git a/src/test/hive/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 b/shark/src/test/hive/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from src/test/hive/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 rename to shark/src/test/hive/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/src/test/hive/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e b/shark/src/test/hive/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from src/test/hive/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e rename to shark/src/test/hive/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e diff --git a/src/test/hive/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 b/shark/src/test/hive/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 similarity index 100% rename from src/test/hive/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 rename to shark/src/test/hive/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 diff --git a/src/test/hive/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e b/shark/src/test/hive/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from src/test/hive/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e rename to shark/src/test/hive/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e diff --git a/src/test/hive/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 b/shark/src/test/hive/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 similarity index 100% rename from src/test/hive/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 rename to shark/src/test/hive/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 diff --git a/src/test/hive/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 b/shark/src/test/hive/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from src/test/hive/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 rename to shark/src/test/hive/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/src/test/hive/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 b/shark/src/test/hive/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from src/test/hive/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 rename to shark/src/test/hive/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/src/test/hive/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee b/shark/src/test/hive/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee similarity index 100% rename from src/test/hive/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee rename to shark/src/test/hive/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee diff --git a/src/test/hive/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 b/shark/src/test/hive/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 similarity index 100% rename from src/test/hive/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 rename to shark/src/test/hive/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 diff --git a/src/test/hive/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 b/shark/src/test/hive/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 similarity index 100% rename from src/test/hive/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 rename to shark/src/test/hive/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 diff --git a/src/test/hive/golden/udf_minute-3-270055c684846e87444b037226cf554c b/shark/src/test/hive/golden/udf_minute-3-270055c684846e87444b037226cf554c similarity index 100% rename from src/test/hive/golden/udf_minute-3-270055c684846e87444b037226cf554c rename to shark/src/test/hive/golden/udf_minute-3-270055c684846e87444b037226cf554c diff --git a/src/test/hive/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 b/shark/src/test/hive/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 similarity index 100% rename from src/test/hive/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 rename to shark/src/test/hive/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 diff --git a/src/test/hive/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 b/shark/src/test/hive/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 similarity index 100% rename from src/test/hive/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 rename to shark/src/test/hive/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 diff --git a/src/test/hive/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee b/shark/src/test/hive/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee similarity index 100% rename from src/test/hive/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee rename to shark/src/test/hive/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee diff --git a/src/test/hive/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 b/shark/src/test/hive/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 similarity index 100% rename from src/test/hive/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 rename to shark/src/test/hive/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 diff --git a/src/test/hive/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 b/shark/src/test/hive/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 similarity index 100% rename from src/test/hive/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 rename to shark/src/test/hive/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 diff --git a/src/test/hive/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf b/shark/src/test/hive/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf similarity index 100% rename from src/test/hive/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf rename to shark/src/test/hive/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf diff --git a/src/test/hive/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c b/shark/src/test/hive/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c similarity index 100% rename from src/test/hive/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c rename to shark/src/test/hive/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c diff --git a/src/test/hive/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 b/shark/src/test/hive/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 similarity index 100% rename from src/test/hive/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 rename to shark/src/test/hive/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 diff --git a/src/test/hive/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c b/shark/src/test/hive/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c similarity index 100% rename from src/test/hive/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c rename to shark/src/test/hive/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c diff --git a/src/test/hive/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e b/shark/src/test/hive/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e similarity index 100% rename from src/test/hive/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e rename to shark/src/test/hive/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e diff --git a/src/test/hive/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 b/shark/src/test/hive/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 similarity index 100% rename from src/test/hive/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 rename to shark/src/test/hive/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 diff --git a/src/test/hive/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b b/shark/src/test/hive/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b similarity index 100% rename from src/test/hive/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b rename to shark/src/test/hive/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b diff --git a/src/test/hive/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 b/shark/src/test/hive/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 similarity index 100% rename from src/test/hive/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 rename to shark/src/test/hive/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 diff --git a/src/test/hive/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa b/shark/src/test/hive/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa similarity index 100% rename from src/test/hive/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa rename to shark/src/test/hive/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa diff --git a/src/test/hive/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc b/shark/src/test/hive/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc similarity index 100% rename from src/test/hive/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc rename to shark/src/test/hive/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc diff --git a/src/test/hive/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 b/shark/src/test/hive/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 similarity index 100% rename from src/test/hive/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 rename to shark/src/test/hive/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 diff --git a/src/test/hive/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 b/shark/src/test/hive/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 similarity index 100% rename from src/test/hive/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 rename to shark/src/test/hive/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 diff --git a/src/test/hive/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a b/shark/src/test/hive/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a similarity index 100% rename from src/test/hive/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a rename to shark/src/test/hive/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a diff --git a/src/test/hive/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e b/shark/src/test/hive/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e similarity index 100% rename from src/test/hive/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e rename to shark/src/test/hive/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e diff --git a/src/test/hive/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a b/shark/src/test/hive/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a similarity index 100% rename from src/test/hive/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a rename to shark/src/test/hive/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a diff --git a/src/test/hive/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 b/shark/src/test/hive/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 similarity index 100% rename from src/test/hive/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 rename to shark/src/test/hive/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 diff --git a/src/test/hive/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 b/shark/src/test/hive/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 similarity index 100% rename from src/test/hive/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 rename to shark/src/test/hive/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 diff --git a/src/test/hive/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 b/shark/src/test/hive/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 similarity index 100% rename from src/test/hive/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 rename to shark/src/test/hive/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 diff --git a/src/test/hive/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 b/shark/src/test/hive/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 similarity index 100% rename from src/test/hive/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 rename to shark/src/test/hive/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 diff --git a/src/test/hive/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 b/shark/src/test/hive/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 similarity index 100% rename from src/test/hive/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 rename to shark/src/test/hive/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 diff --git a/src/test/hive/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd b/shark/src/test/hive/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd similarity index 100% rename from src/test/hive/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd rename to shark/src/test/hive/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd diff --git a/src/test/hive/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 b/shark/src/test/hive/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 similarity index 100% rename from src/test/hive/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 rename to shark/src/test/hive/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 diff --git a/src/test/hive/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 b/shark/src/test/hive/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 similarity index 100% rename from src/test/hive/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 rename to shark/src/test/hive/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 diff --git a/src/test/hive/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 b/shark/src/test/hive/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 similarity index 100% rename from src/test/hive/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 rename to shark/src/test/hive/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 diff --git a/src/test/hive/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def b/shark/src/test/hive/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def similarity index 100% rename from src/test/hive/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def rename to shark/src/test/hive/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def diff --git a/src/test/hive/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 b/shark/src/test/hive/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 similarity index 100% rename from src/test/hive/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 rename to shark/src/test/hive/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 diff --git a/src/test/hive/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 b/shark/src/test/hive/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 similarity index 100% rename from src/test/hive/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 rename to shark/src/test/hive/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 diff --git a/src/test/hive/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 b/shark/src/test/hive/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 similarity index 100% rename from src/test/hive/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 rename to shark/src/test/hive/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 diff --git a/src/test/hive/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 b/shark/src/test/hive/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 similarity index 100% rename from src/test/hive/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 rename to shark/src/test/hive/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 diff --git a/src/test/hive/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab b/shark/src/test/hive/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab similarity index 100% rename from src/test/hive/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab rename to shark/src/test/hive/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab diff --git a/src/test/hive/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d b/shark/src/test/hive/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d similarity index 100% rename from src/test/hive/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d rename to shark/src/test/hive/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d diff --git a/src/test/hive/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 b/shark/src/test/hive/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 similarity index 100% rename from src/test/hive/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 rename to shark/src/test/hive/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 diff --git a/src/test/hive/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 b/shark/src/test/hive/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 similarity index 100% rename from src/test/hive/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 rename to shark/src/test/hive/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 diff --git a/src/test/hive/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a b/shark/src/test/hive/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a similarity index 100% rename from src/test/hive/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a rename to shark/src/test/hive/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a diff --git a/src/test/hive/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 b/shark/src/test/hive/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from src/test/hive/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 rename to shark/src/test/hive/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/src/test/hive/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e b/shark/src/test/hive/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from src/test/hive/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e rename to shark/src/test/hive/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e diff --git a/src/test/hive/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 b/shark/src/test/hive/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 similarity index 100% rename from src/test/hive/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 rename to shark/src/test/hive/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 diff --git a/src/test/hive/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae b/shark/src/test/hive/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae similarity index 100% rename from src/test/hive/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae rename to shark/src/test/hive/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae diff --git a/src/test/hive/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 b/shark/src/test/hive/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 similarity index 100% rename from src/test/hive/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 rename to shark/src/test/hive/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 diff --git a/src/test/hive/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 b/shark/src/test/hive/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 similarity index 100% rename from src/test/hive/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 rename to shark/src/test/hive/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 diff --git a/src/test/hive/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 b/shark/src/test/hive/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 similarity index 100% rename from src/test/hive/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 rename to shark/src/test/hive/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 diff --git a/src/test/hive/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 b/shark/src/test/hive/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 similarity index 100% rename from src/test/hive/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 rename to shark/src/test/hive/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 diff --git a/src/test/hive/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 b/shark/src/test/hive/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 similarity index 100% rename from src/test/hive/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 rename to shark/src/test/hive/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 diff --git a/src/test/hive/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 b/shark/src/test/hive/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 similarity index 100% rename from src/test/hive/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 rename to shark/src/test/hive/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 diff --git a/src/test/hive/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 b/shark/src/test/hive/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 similarity index 100% rename from src/test/hive/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 rename to shark/src/test/hive/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 diff --git a/src/test/hive/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd b/shark/src/test/hive/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd similarity index 100% rename from src/test/hive/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd rename to shark/src/test/hive/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd diff --git a/src/test/hive/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 b/shark/src/test/hive/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 similarity index 100% rename from src/test/hive/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 rename to shark/src/test/hive/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 diff --git a/src/test/hive/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 b/shark/src/test/hive/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 similarity index 100% rename from src/test/hive/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 rename to shark/src/test/hive/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 diff --git a/src/test/hive/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c b/shark/src/test/hive/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c similarity index 100% rename from src/test/hive/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c rename to shark/src/test/hive/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c diff --git a/src/test/hive/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 b/shark/src/test/hive/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 similarity index 100% rename from src/test/hive/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 rename to shark/src/test/hive/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 diff --git a/src/test/hive/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 b/shark/src/test/hive/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 similarity index 100% rename from src/test/hive/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 rename to shark/src/test/hive/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 diff --git a/src/test/hive/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 b/shark/src/test/hive/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 similarity index 100% rename from src/test/hive/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 rename to shark/src/test/hive/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 diff --git a/src/test/hive/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 b/shark/src/test/hive/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 similarity index 100% rename from src/test/hive/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 rename to shark/src/test/hive/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 diff --git a/src/test/hive/golden/udf_power-0-57001d802c281743322d28bbc520cd4 b/shark/src/test/hive/golden/udf_power-0-57001d802c281743322d28bbc520cd4 similarity index 100% rename from src/test/hive/golden/udf_power-0-57001d802c281743322d28bbc520cd4 rename to shark/src/test/hive/golden/udf_power-0-57001d802c281743322d28bbc520cd4 diff --git a/src/test/hive/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 b/shark/src/test/hive/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 similarity index 100% rename from src/test/hive/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 rename to shark/src/test/hive/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 diff --git a/src/test/hive/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 b/shark/src/test/hive/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 similarity index 100% rename from src/test/hive/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 rename to shark/src/test/hive/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 diff --git a/src/test/hive/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 b/shark/src/test/hive/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 similarity index 100% rename from src/test/hive/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 rename to shark/src/test/hive/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 diff --git a/src/test/hive/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 b/shark/src/test/hive/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 similarity index 100% rename from src/test/hive/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 rename to shark/src/test/hive/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 diff --git a/src/test/hive/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 b/shark/src/test/hive/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 similarity index 100% rename from src/test/hive/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 rename to shark/src/test/hive/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 diff --git a/src/test/hive/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/shark/src/test/hive/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 similarity index 100% rename from src/test/hive/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 rename to shark/src/test/hive/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 diff --git a/src/test/hive/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 b/shark/src/test/hive/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 similarity index 100% rename from src/test/hive/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 rename to shark/src/test/hive/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 diff --git a/src/test/hive/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 b/shark/src/test/hive/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 similarity index 100% rename from src/test/hive/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 rename to shark/src/test/hive/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 diff --git a/src/test/hive/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 b/shark/src/test/hive/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 similarity index 100% rename from src/test/hive/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 rename to shark/src/test/hive/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 diff --git a/src/test/hive/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 b/shark/src/test/hive/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 similarity index 100% rename from src/test/hive/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 rename to shark/src/test/hive/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 diff --git a/src/test/hive/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/shark/src/test/hive/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 similarity index 100% rename from src/test/hive/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 rename to shark/src/test/hive/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 diff --git a/src/test/hive/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 b/shark/src/test/hive/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 similarity index 100% rename from src/test/hive/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 rename to shark/src/test/hive/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 diff --git a/src/test/hive/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e b/shark/src/test/hive/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e similarity index 100% rename from src/test/hive/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e rename to shark/src/test/hive/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e diff --git a/src/test/hive/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 b/shark/src/test/hive/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 similarity index 100% rename from src/test/hive/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 rename to shark/src/test/hive/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 diff --git a/src/test/hive/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee b/shark/src/test/hive/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee similarity index 100% rename from src/test/hive/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee rename to shark/src/test/hive/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee diff --git a/src/test/hive/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 b/shark/src/test/hive/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 similarity index 100% rename from src/test/hive/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 rename to shark/src/test/hive/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 diff --git a/src/test/hive/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 b/shark/src/test/hive/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 similarity index 100% rename from src/test/hive/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 rename to shark/src/test/hive/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 diff --git a/src/test/hive/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 b/shark/src/test/hive/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 similarity index 100% rename from src/test/hive/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 rename to shark/src/test/hive/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 diff --git a/src/test/hive/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca b/shark/src/test/hive/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca similarity index 100% rename from src/test/hive/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca rename to shark/src/test/hive/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca diff --git a/src/test/hive/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 b/shark/src/test/hive/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 similarity index 100% rename from src/test/hive/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 rename to shark/src/test/hive/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 diff --git a/src/test/hive/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 b/shark/src/test/hive/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 similarity index 100% rename from src/test/hive/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 rename to shark/src/test/hive/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 diff --git a/src/test/hive/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 b/shark/src/test/hive/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 similarity index 100% rename from src/test/hive/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 rename to shark/src/test/hive/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 diff --git a/src/test/hive/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 b/shark/src/test/hive/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 similarity index 100% rename from src/test/hive/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 rename to shark/src/test/hive/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 diff --git a/src/test/hive/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb b/shark/src/test/hive/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb similarity index 100% rename from src/test/hive/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb rename to shark/src/test/hive/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb diff --git a/src/test/hive/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e b/shark/src/test/hive/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e similarity index 100% rename from src/test/hive/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e rename to shark/src/test/hive/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e diff --git a/src/test/hive/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc b/shark/src/test/hive/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc similarity index 100% rename from src/test/hive/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc rename to shark/src/test/hive/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc diff --git a/src/test/hive/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b b/shark/src/test/hive/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b similarity index 100% rename from src/test/hive/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b rename to shark/src/test/hive/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b diff --git a/src/test/hive/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b b/shark/src/test/hive/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b similarity index 100% rename from src/test/hive/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b rename to shark/src/test/hive/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b diff --git a/src/test/hive/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a b/shark/src/test/hive/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a similarity index 100% rename from src/test/hive/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a rename to shark/src/test/hive/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a diff --git a/src/test/hive/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 b/shark/src/test/hive/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 similarity index 100% rename from src/test/hive/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 rename to shark/src/test/hive/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 diff --git a/src/test/hive/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 b/shark/src/test/hive/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 similarity index 100% rename from src/test/hive/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 rename to shark/src/test/hive/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 diff --git a/src/test/hive/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca b/shark/src/test/hive/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca similarity index 100% rename from src/test/hive/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca rename to shark/src/test/hive/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca diff --git a/src/test/hive/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 b/shark/src/test/hive/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 similarity index 100% rename from src/test/hive/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 rename to shark/src/test/hive/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 diff --git a/src/test/hive/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 b/shark/src/test/hive/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 similarity index 100% rename from src/test/hive/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 rename to shark/src/test/hive/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 diff --git a/src/test/hive/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 b/shark/src/test/hive/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 similarity index 100% rename from src/test/hive/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 rename to shark/src/test/hive/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 diff --git a/src/test/hive/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c b/shark/src/test/hive/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c similarity index 100% rename from src/test/hive/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c rename to shark/src/test/hive/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c diff --git a/src/test/hive/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 b/shark/src/test/hive/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 similarity index 100% rename from src/test/hive/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 rename to shark/src/test/hive/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 diff --git a/src/test/hive/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a b/shark/src/test/hive/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a similarity index 100% rename from src/test/hive/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a rename to shark/src/test/hive/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a diff --git a/src/test/hive/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b b/shark/src/test/hive/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b similarity index 100% rename from src/test/hive/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b rename to shark/src/test/hive/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b diff --git a/src/test/hive/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 b/shark/src/test/hive/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 similarity index 100% rename from src/test/hive/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 rename to shark/src/test/hive/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 diff --git a/src/test/hive/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce b/shark/src/test/hive/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce similarity index 100% rename from src/test/hive/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce rename to shark/src/test/hive/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce diff --git a/src/test/hive/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 b/shark/src/test/hive/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 similarity index 100% rename from src/test/hive/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 rename to shark/src/test/hive/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 diff --git a/src/test/hive/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 b/shark/src/test/hive/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 similarity index 100% rename from src/test/hive/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 rename to shark/src/test/hive/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 diff --git a/src/test/hive/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 b/shark/src/test/hive/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 similarity index 100% rename from src/test/hive/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 rename to shark/src/test/hive/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 diff --git a/src/test/hive/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b b/shark/src/test/hive/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b similarity index 100% rename from src/test/hive/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b rename to shark/src/test/hive/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b diff --git a/src/test/hive/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c b/shark/src/test/hive/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c similarity index 100% rename from src/test/hive/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c rename to shark/src/test/hive/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c diff --git a/src/test/hive/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 b/shark/src/test/hive/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 similarity index 100% rename from src/test/hive/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 rename to shark/src/test/hive/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 diff --git a/src/test/hive/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 b/shark/src/test/hive/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 similarity index 100% rename from src/test/hive/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 rename to shark/src/test/hive/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 diff --git a/src/test/hive/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 b/shark/src/test/hive/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 similarity index 100% rename from src/test/hive/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 rename to shark/src/test/hive/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 diff --git a/src/test/hive/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 b/shark/src/test/hive/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 similarity index 100% rename from src/test/hive/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 rename to shark/src/test/hive/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 diff --git a/src/test/hive/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c b/shark/src/test/hive/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c similarity index 100% rename from src/test/hive/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c rename to shark/src/test/hive/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c diff --git a/src/test/hive/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 b/shark/src/test/hive/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 similarity index 100% rename from src/test/hive/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 rename to shark/src/test/hive/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 diff --git a/src/test/hive/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 b/shark/src/test/hive/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 similarity index 100% rename from src/test/hive/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 rename to shark/src/test/hive/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 diff --git a/src/test/hive/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 b/shark/src/test/hive/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 similarity index 100% rename from src/test/hive/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 rename to shark/src/test/hive/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 diff --git a/src/test/hive/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a b/shark/src/test/hive/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a similarity index 100% rename from src/test/hive/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a rename to shark/src/test/hive/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a diff --git a/src/test/hive/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd b/shark/src/test/hive/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd similarity index 100% rename from src/test/hive/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd rename to shark/src/test/hive/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd diff --git a/src/test/hive/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 b/shark/src/test/hive/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 similarity index 100% rename from src/test/hive/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 rename to shark/src/test/hive/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 diff --git a/src/test/hive/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 b/shark/src/test/hive/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 similarity index 100% rename from src/test/hive/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 rename to shark/src/test/hive/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 diff --git a/src/test/hive/golden/udf_second-2-d678372e3837a16be245d2e33482f17f b/shark/src/test/hive/golden/udf_second-2-d678372e3837a16be245d2e33482f17f similarity index 100% rename from src/test/hive/golden/udf_second-2-d678372e3837a16be245d2e33482f17f rename to shark/src/test/hive/golden/udf_second-2-d678372e3837a16be245d2e33482f17f diff --git a/src/test/hive/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf b/shark/src/test/hive/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf similarity index 100% rename from src/test/hive/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf rename to shark/src/test/hive/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf diff --git a/src/test/hive/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f b/shark/src/test/hive/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f similarity index 100% rename from src/test/hive/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f rename to shark/src/test/hive/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f diff --git a/src/test/hive/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 b/shark/src/test/hive/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 similarity index 100% rename from src/test/hive/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 rename to shark/src/test/hive/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 diff --git a/src/test/hive/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 b/shark/src/test/hive/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 similarity index 100% rename from src/test/hive/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 rename to shark/src/test/hive/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 diff --git a/src/test/hive/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 b/shark/src/test/hive/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 similarity index 100% rename from src/test/hive/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 rename to shark/src/test/hive/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 diff --git a/src/test/hive/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 b/shark/src/test/hive/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 similarity index 100% rename from src/test/hive/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 rename to shark/src/test/hive/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 diff --git a/src/test/hive/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 b/shark/src/test/hive/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 similarity index 100% rename from src/test/hive/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 rename to shark/src/test/hive/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 diff --git a/src/test/hive/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 b/shark/src/test/hive/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 similarity index 100% rename from src/test/hive/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 rename to shark/src/test/hive/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 diff --git a/src/test/hive/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 b/shark/src/test/hive/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 similarity index 100% rename from src/test/hive/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 rename to shark/src/test/hive/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 diff --git a/src/test/hive/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 b/shark/src/test/hive/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 similarity index 100% rename from src/test/hive/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 rename to shark/src/test/hive/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 diff --git a/src/test/hive/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de b/shark/src/test/hive/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de similarity index 100% rename from src/test/hive/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de rename to shark/src/test/hive/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de diff --git a/src/test/hive/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f b/shark/src/test/hive/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f similarity index 100% rename from src/test/hive/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f rename to shark/src/test/hive/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f diff --git a/src/test/hive/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 b/shark/src/test/hive/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 similarity index 100% rename from src/test/hive/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 rename to shark/src/test/hive/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 diff --git a/src/test/hive/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 b/shark/src/test/hive/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 similarity index 100% rename from src/test/hive/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 rename to shark/src/test/hive/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 diff --git a/src/test/hive/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e b/shark/src/test/hive/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e similarity index 100% rename from src/test/hive/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e rename to shark/src/test/hive/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e diff --git a/src/test/hive/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 b/shark/src/test/hive/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 similarity index 100% rename from src/test/hive/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 rename to shark/src/test/hive/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 diff --git a/src/test/hive/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa b/shark/src/test/hive/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa similarity index 100% rename from src/test/hive/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa rename to shark/src/test/hive/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa diff --git a/src/test/hive/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a b/shark/src/test/hive/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a similarity index 100% rename from src/test/hive/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a rename to shark/src/test/hive/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a diff --git a/src/test/hive/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef b/shark/src/test/hive/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef similarity index 100% rename from src/test/hive/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef rename to shark/src/test/hive/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef diff --git a/src/test/hive/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d b/shark/src/test/hive/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d similarity index 100% rename from src/test/hive/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d rename to shark/src/test/hive/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d diff --git a/src/test/hive/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa b/shark/src/test/hive/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa similarity index 100% rename from src/test/hive/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa rename to shark/src/test/hive/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa diff --git a/src/test/hive/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a b/shark/src/test/hive/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a similarity index 100% rename from src/test/hive/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a rename to shark/src/test/hive/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a diff --git a/src/test/hive/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e b/shark/src/test/hive/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e similarity index 100% rename from src/test/hive/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e rename to shark/src/test/hive/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e diff --git a/src/test/hive/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 b/shark/src/test/hive/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 similarity index 100% rename from src/test/hive/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 rename to shark/src/test/hive/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 diff --git a/src/test/hive/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef b/shark/src/test/hive/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef similarity index 100% rename from src/test/hive/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef rename to shark/src/test/hive/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef diff --git a/src/test/hive/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d b/shark/src/test/hive/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d similarity index 100% rename from src/test/hive/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d rename to shark/src/test/hive/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d diff --git a/src/test/hive/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee b/shark/src/test/hive/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee similarity index 100% rename from src/test/hive/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee rename to shark/src/test/hive/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee diff --git a/src/test/hive/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 b/shark/src/test/hive/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 similarity index 100% rename from src/test/hive/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 rename to shark/src/test/hive/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 diff --git a/src/test/hive/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab b/shark/src/test/hive/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab similarity index 100% rename from src/test/hive/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab rename to shark/src/test/hive/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab diff --git a/src/test/hive/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 b/shark/src/test/hive/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 similarity index 100% rename from src/test/hive/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 rename to shark/src/test/hive/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 diff --git a/src/test/hive/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 b/shark/src/test/hive/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 similarity index 100% rename from src/test/hive/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 rename to shark/src/test/hive/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 diff --git a/src/test/hive/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 b/shark/src/test/hive/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 similarity index 100% rename from src/test/hive/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 rename to shark/src/test/hive/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 diff --git a/src/test/hive/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 b/shark/src/test/hive/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 similarity index 100% rename from src/test/hive/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 rename to shark/src/test/hive/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 diff --git a/src/test/hive/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd b/shark/src/test/hive/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd similarity index 100% rename from src/test/hive/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd rename to shark/src/test/hive/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd diff --git a/src/test/hive/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 b/shark/src/test/hive/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 similarity index 100% rename from src/test/hive/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 rename to shark/src/test/hive/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 diff --git a/src/test/hive/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 b/shark/src/test/hive/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 similarity index 100% rename from src/test/hive/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 rename to shark/src/test/hive/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 diff --git a/src/test/hive/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb b/shark/src/test/hive/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb similarity index 100% rename from src/test/hive/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb rename to shark/src/test/hive/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb diff --git a/src/test/hive/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 b/shark/src/test/hive/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 similarity index 100% rename from src/test/hive/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 rename to shark/src/test/hive/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 diff --git a/src/test/hive/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 b/shark/src/test/hive/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 similarity index 100% rename from src/test/hive/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 rename to shark/src/test/hive/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 diff --git a/src/test/hive/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de b/shark/src/test/hive/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de similarity index 100% rename from src/test/hive/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de rename to shark/src/test/hive/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de diff --git a/src/test/hive/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 b/shark/src/test/hive/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 similarity index 100% rename from src/test/hive/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 rename to shark/src/test/hive/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 diff --git a/src/test/hive/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e b/shark/src/test/hive/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e similarity index 100% rename from src/test/hive/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e rename to shark/src/test/hive/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e diff --git a/src/test/hive/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 b/shark/src/test/hive/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 similarity index 100% rename from src/test/hive/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 rename to shark/src/test/hive/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 diff --git a/src/test/hive/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 b/shark/src/test/hive/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 similarity index 100% rename from src/test/hive/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 rename to shark/src/test/hive/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 diff --git a/src/test/hive/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 b/shark/src/test/hive/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 similarity index 100% rename from src/test/hive/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 rename to shark/src/test/hive/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 diff --git a/src/test/hive/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a b/shark/src/test/hive/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a similarity index 100% rename from src/test/hive/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a rename to shark/src/test/hive/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a diff --git a/src/test/hive/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 b/shark/src/test/hive/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 similarity index 100% rename from src/test/hive/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 rename to shark/src/test/hive/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 diff --git a/src/test/hive/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 b/shark/src/test/hive/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 similarity index 100% rename from src/test/hive/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 rename to shark/src/test/hive/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 diff --git a/src/test/hive/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 b/shark/src/test/hive/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 similarity index 100% rename from src/test/hive/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 rename to shark/src/test/hive/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 diff --git a/src/test/hive/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 b/shark/src/test/hive/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 similarity index 100% rename from src/test/hive/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 rename to shark/src/test/hive/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 diff --git a/src/test/hive/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 b/shark/src/test/hive/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 similarity index 100% rename from src/test/hive/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 rename to shark/src/test/hive/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 diff --git a/src/test/hive/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 b/shark/src/test/hive/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 similarity index 100% rename from src/test/hive/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 rename to shark/src/test/hive/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 diff --git a/src/test/hive/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 b/shark/src/test/hive/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 similarity index 100% rename from src/test/hive/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 rename to shark/src/test/hive/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 diff --git a/src/test/hive/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 b/shark/src/test/hive/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 similarity index 100% rename from src/test/hive/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 rename to shark/src/test/hive/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 diff --git a/src/test/hive/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 b/shark/src/test/hive/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 similarity index 100% rename from src/test/hive/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 rename to shark/src/test/hive/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 diff --git a/src/test/hive/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 b/shark/src/test/hive/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 similarity index 100% rename from src/test/hive/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 rename to shark/src/test/hive/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 diff --git a/src/test/hive/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 b/shark/src/test/hive/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 similarity index 100% rename from src/test/hive/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 rename to shark/src/test/hive/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 diff --git a/src/test/hive/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b b/shark/src/test/hive/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b similarity index 100% rename from src/test/hive/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b rename to shark/src/test/hive/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b diff --git a/src/test/hive/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 b/shark/src/test/hive/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 similarity index 100% rename from src/test/hive/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 rename to shark/src/test/hive/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 diff --git a/src/test/hive/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 b/shark/src/test/hive/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 similarity index 100% rename from src/test/hive/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 rename to shark/src/test/hive/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 diff --git a/src/test/hive/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d b/shark/src/test/hive/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d similarity index 100% rename from src/test/hive/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d rename to shark/src/test/hive/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d diff --git a/src/test/hive/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 b/shark/src/test/hive/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 similarity index 100% rename from src/test/hive/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 rename to shark/src/test/hive/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 diff --git a/src/test/hive/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 b/shark/src/test/hive/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 similarity index 100% rename from src/test/hive/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 rename to shark/src/test/hive/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 diff --git a/src/test/hive/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 b/shark/src/test/hive/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 similarity index 100% rename from src/test/hive/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 rename to shark/src/test/hive/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 diff --git a/src/test/hive/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 b/shark/src/test/hive/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 similarity index 100% rename from src/test/hive/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 rename to shark/src/test/hive/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 diff --git a/src/test/hive/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 b/shark/src/test/hive/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 similarity index 100% rename from src/test/hive/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 rename to shark/src/test/hive/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 diff --git a/src/test/hive/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 b/shark/src/test/hive/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 similarity index 100% rename from src/test/hive/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 rename to shark/src/test/hive/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 diff --git a/src/test/hive/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 b/shark/src/test/hive/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 similarity index 100% rename from src/test/hive/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 rename to shark/src/test/hive/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 diff --git a/src/test/hive/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d b/shark/src/test/hive/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d similarity index 100% rename from src/test/hive/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d rename to shark/src/test/hive/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d diff --git a/src/test/hive/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd b/shark/src/test/hive/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd similarity index 100% rename from src/test/hive/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd rename to shark/src/test/hive/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd diff --git a/src/test/hive/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 b/shark/src/test/hive/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 similarity index 100% rename from src/test/hive/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 rename to shark/src/test/hive/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 diff --git a/src/test/hive/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 b/shark/src/test/hive/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 similarity index 100% rename from src/test/hive/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 rename to shark/src/test/hive/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 diff --git a/src/test/hive/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 b/shark/src/test/hive/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 similarity index 100% rename from src/test/hive/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 rename to shark/src/test/hive/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 diff --git a/src/test/hive/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a b/shark/src/test/hive/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a similarity index 100% rename from src/test/hive/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a rename to shark/src/test/hive/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a diff --git a/src/test/hive/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 b/shark/src/test/hive/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 similarity index 100% rename from src/test/hive/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 rename to shark/src/test/hive/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 diff --git a/src/test/hive/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 b/shark/src/test/hive/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 similarity index 100% rename from src/test/hive/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 rename to shark/src/test/hive/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 diff --git a/src/test/hive/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 b/shark/src/test/hive/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 similarity index 100% rename from src/test/hive/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 rename to shark/src/test/hive/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 diff --git a/src/test/hive/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece b/shark/src/test/hive/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece similarity index 100% rename from src/test/hive/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece rename to shark/src/test/hive/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece diff --git a/src/test/hive/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 b/shark/src/test/hive/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 similarity index 100% rename from src/test/hive/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 rename to shark/src/test/hive/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 diff --git a/src/test/hive/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 b/shark/src/test/hive/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 similarity index 100% rename from src/test/hive/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 rename to shark/src/test/hive/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 diff --git a/src/test/hive/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 b/shark/src/test/hive/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 similarity index 100% rename from src/test/hive/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 rename to shark/src/test/hive/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 diff --git a/src/test/hive/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 b/shark/src/test/hive/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 similarity index 100% rename from src/test/hive/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 rename to shark/src/test/hive/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 diff --git a/src/test/hive/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b b/shark/src/test/hive/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b similarity index 100% rename from src/test/hive/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b rename to shark/src/test/hive/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b diff --git a/src/test/hive/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 b/shark/src/test/hive/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 similarity index 100% rename from src/test/hive/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 rename to shark/src/test/hive/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 diff --git a/src/test/hive/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b b/shark/src/test/hive/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b similarity index 100% rename from src/test/hive/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b rename to shark/src/test/hive/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b diff --git a/src/test/hive/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 b/shark/src/test/hive/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 similarity index 100% rename from src/test/hive/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 rename to shark/src/test/hive/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 diff --git a/src/test/hive/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/shark/src/test/hive/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 similarity index 100% rename from src/test/hive/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 rename to shark/src/test/hive/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 diff --git a/src/test/hive/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 b/shark/src/test/hive/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 similarity index 100% rename from src/test/hive/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 rename to shark/src/test/hive/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 diff --git a/src/test/hive/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 b/shark/src/test/hive/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 similarity index 100% rename from src/test/hive/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 rename to shark/src/test/hive/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 diff --git a/src/test/hive/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 b/shark/src/test/hive/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 similarity index 100% rename from src/test/hive/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 rename to shark/src/test/hive/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 diff --git a/src/test/hive/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/shark/src/test/hive/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 similarity index 100% rename from src/test/hive/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 rename to shark/src/test/hive/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 diff --git a/src/test/hive/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 b/shark/src/test/hive/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 similarity index 100% rename from src/test/hive/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 rename to shark/src/test/hive/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 diff --git a/src/test/hive/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 b/shark/src/test/hive/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 similarity index 100% rename from src/test/hive/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 rename to shark/src/test/hive/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 diff --git a/src/test/hive/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad b/shark/src/test/hive/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad similarity index 100% rename from src/test/hive/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad rename to shark/src/test/hive/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad diff --git a/src/test/hive/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 b/shark/src/test/hive/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 similarity index 100% rename from src/test/hive/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 rename to shark/src/test/hive/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 diff --git a/src/test/hive/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d b/shark/src/test/hive/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d similarity index 100% rename from src/test/hive/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d rename to shark/src/test/hive/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d diff --git a/src/test/hive/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 b/shark/src/test/hive/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 similarity index 100% rename from src/test/hive/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 rename to shark/src/test/hive/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 diff --git a/src/test/hive/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 b/shark/src/test/hive/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 similarity index 100% rename from src/test/hive/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 rename to shark/src/test/hive/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 diff --git a/src/test/hive/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 b/shark/src/test/hive/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 similarity index 100% rename from src/test/hive/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 rename to shark/src/test/hive/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 diff --git a/src/test/hive/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 b/shark/src/test/hive/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 similarity index 100% rename from src/test/hive/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 rename to shark/src/test/hive/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 diff --git a/src/test/hive/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c b/shark/src/test/hive/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c similarity index 100% rename from src/test/hive/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c rename to shark/src/test/hive/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c diff --git a/src/test/hive/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d b/shark/src/test/hive/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d similarity index 100% rename from src/test/hive/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d rename to shark/src/test/hive/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d diff --git a/src/test/hive/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad b/shark/src/test/hive/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad similarity index 100% rename from src/test/hive/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad rename to shark/src/test/hive/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad diff --git a/src/test/hive/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f b/shark/src/test/hive/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f similarity index 100% rename from src/test/hive/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f rename to shark/src/test/hive/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f diff --git a/src/test/hive/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a b/shark/src/test/hive/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a similarity index 100% rename from src/test/hive/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a rename to shark/src/test/hive/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a diff --git a/src/test/hive/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 b/shark/src/test/hive/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 similarity index 100% rename from src/test/hive/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 rename to shark/src/test/hive/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 diff --git a/src/test/hive/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 b/shark/src/test/hive/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 similarity index 100% rename from src/test/hive/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 rename to shark/src/test/hive/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 diff --git a/src/test/hive/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc b/shark/src/test/hive/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc similarity index 100% rename from src/test/hive/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc rename to shark/src/test/hive/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc diff --git a/src/test/hive/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b b/shark/src/test/hive/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b similarity index 100% rename from src/test/hive/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b rename to shark/src/test/hive/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b diff --git a/src/test/hive/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 b/shark/src/test/hive/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 similarity index 100% rename from src/test/hive/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 rename to shark/src/test/hive/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 diff --git a/src/test/hive/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 b/shark/src/test/hive/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 similarity index 100% rename from src/test/hive/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 rename to shark/src/test/hive/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 diff --git a/src/test/hive/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 b/shark/src/test/hive/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 similarity index 100% rename from src/test/hive/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 rename to shark/src/test/hive/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 diff --git a/src/test/hive/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 b/shark/src/test/hive/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 similarity index 100% rename from src/test/hive/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 rename to shark/src/test/hive/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 diff --git a/src/test/hive/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 b/shark/src/test/hive/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 similarity index 100% rename from src/test/hive/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 rename to shark/src/test/hive/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 diff --git a/src/test/hive/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 b/shark/src/test/hive/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 similarity index 100% rename from src/test/hive/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 rename to shark/src/test/hive/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 diff --git a/src/test/hive/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e b/shark/src/test/hive/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e similarity index 100% rename from src/test/hive/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e rename to shark/src/test/hive/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e diff --git a/src/test/hive/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 b/shark/src/test/hive/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 similarity index 100% rename from src/test/hive/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 rename to shark/src/test/hive/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 diff --git a/src/test/hive/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a b/shark/src/test/hive/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a similarity index 100% rename from src/test/hive/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a rename to shark/src/test/hive/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a diff --git a/src/test/hive/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe b/shark/src/test/hive/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe similarity index 100% rename from src/test/hive/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe rename to shark/src/test/hive/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe diff --git a/src/test/hive/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f b/shark/src/test/hive/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f similarity index 100% rename from src/test/hive/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f rename to shark/src/test/hive/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f diff --git a/src/test/hive/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 b/shark/src/test/hive/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 similarity index 100% rename from src/test/hive/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 rename to shark/src/test/hive/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 diff --git a/src/test/hive/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 b/shark/src/test/hive/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 similarity index 100% rename from src/test/hive/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 rename to shark/src/test/hive/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 diff --git a/src/test/hive/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 b/shark/src/test/hive/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 similarity index 100% rename from src/test/hive/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 rename to shark/src/test/hive/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 diff --git a/src/test/hive/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 b/shark/src/test/hive/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 similarity index 100% rename from src/test/hive/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 rename to shark/src/test/hive/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 diff --git a/src/test/hive/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 b/shark/src/test/hive/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 similarity index 100% rename from src/test/hive/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 rename to shark/src/test/hive/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 diff --git a/src/test/hive/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 b/shark/src/test/hive/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 similarity index 100% rename from src/test/hive/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 rename to shark/src/test/hive/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 diff --git a/src/test/hive/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c b/shark/src/test/hive/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c similarity index 100% rename from src/test/hive/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c rename to shark/src/test/hive/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c diff --git a/src/test/hive/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 b/shark/src/test/hive/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 similarity index 100% rename from src/test/hive/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 rename to shark/src/test/hive/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 diff --git a/src/test/hive/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce b/shark/src/test/hive/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce similarity index 100% rename from src/test/hive/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce rename to shark/src/test/hive/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce diff --git a/src/test/hive/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 b/shark/src/test/hive/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 similarity index 100% rename from src/test/hive/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 rename to shark/src/test/hive/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 diff --git a/src/test/hive/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 b/shark/src/test/hive/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 similarity index 100% rename from src/test/hive/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 rename to shark/src/test/hive/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 diff --git a/src/test/hive/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 b/shark/src/test/hive/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 similarity index 100% rename from src/test/hive/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 rename to shark/src/test/hive/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 diff --git a/src/test/hive/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 b/shark/src/test/hive/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 similarity index 100% rename from src/test/hive/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 rename to shark/src/test/hive/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 diff --git a/src/test/hive/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c b/shark/src/test/hive/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c similarity index 100% rename from src/test/hive/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c rename to shark/src/test/hive/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c diff --git a/src/test/hive/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a b/shark/src/test/hive/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a similarity index 100% rename from src/test/hive/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a rename to shark/src/test/hive/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a diff --git a/src/test/hive/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 b/shark/src/test/hive/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 similarity index 100% rename from src/test/hive/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 rename to shark/src/test/hive/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 diff --git a/src/test/hive/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 b/shark/src/test/hive/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 similarity index 100% rename from src/test/hive/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 rename to shark/src/test/hive/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 diff --git a/src/test/hive/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 b/shark/src/test/hive/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 similarity index 100% rename from src/test/hive/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 rename to shark/src/test/hive/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 diff --git a/src/test/hive/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 b/shark/src/test/hive/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 similarity index 100% rename from src/test/hive/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 rename to shark/src/test/hive/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 diff --git a/src/test/hive/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d b/shark/src/test/hive/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d similarity index 100% rename from src/test/hive/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d rename to shark/src/test/hive/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d diff --git a/src/test/hive/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 b/shark/src/test/hive/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 similarity index 100% rename from src/test/hive/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 rename to shark/src/test/hive/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 diff --git a/src/test/hive/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e b/shark/src/test/hive/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e similarity index 100% rename from src/test/hive/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e rename to shark/src/test/hive/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e diff --git a/src/test/hive/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 b/shark/src/test/hive/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 similarity index 100% rename from src/test/hive/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 rename to shark/src/test/hive/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 diff --git a/src/test/hive/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a b/shark/src/test/hive/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a similarity index 100% rename from src/test/hive/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a rename to shark/src/test/hive/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a diff --git a/src/test/hive/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b b/shark/src/test/hive/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b similarity index 100% rename from src/test/hive/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b rename to shark/src/test/hive/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b diff --git a/src/test/hive/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 b/shark/src/test/hive/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 similarity index 100% rename from src/test/hive/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 rename to shark/src/test/hive/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 diff --git a/src/test/hive/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce b/shark/src/test/hive/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce similarity index 100% rename from src/test/hive/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce rename to shark/src/test/hive/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce diff --git a/src/test/hive/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 b/shark/src/test/hive/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 similarity index 100% rename from src/test/hive/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 rename to shark/src/test/hive/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 diff --git a/src/test/hive/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 b/shark/src/test/hive/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 similarity index 100% rename from src/test/hive/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 rename to shark/src/test/hive/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 diff --git a/src/test/hive/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 b/shark/src/test/hive/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 similarity index 100% rename from src/test/hive/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 rename to shark/src/test/hive/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 diff --git a/src/test/hive/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d b/shark/src/test/hive/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d similarity index 100% rename from src/test/hive/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d rename to shark/src/test/hive/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d diff --git a/src/test/hive/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 b/shark/src/test/hive/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 similarity index 100% rename from src/test/hive/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 rename to shark/src/test/hive/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 diff --git a/src/test/hive/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc b/shark/src/test/hive/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc similarity index 100% rename from src/test/hive/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc rename to shark/src/test/hive/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc diff --git a/src/test/hive/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 b/shark/src/test/hive/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 similarity index 100% rename from src/test/hive/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 rename to shark/src/test/hive/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 diff --git a/src/test/hive/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 b/shark/src/test/hive/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 similarity index 100% rename from src/test/hive/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 rename to shark/src/test/hive/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 diff --git a/src/test/hive/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 b/shark/src/test/hive/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 similarity index 100% rename from src/test/hive/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 rename to shark/src/test/hive/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 diff --git a/src/test/hive/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 b/shark/src/test/hive/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 similarity index 100% rename from src/test/hive/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 rename to shark/src/test/hive/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 diff --git a/src/test/hive/golden/udf_to_string-5-1040b37847d20ef29d545934316303 b/shark/src/test/hive/golden/udf_to_string-5-1040b37847d20ef29d545934316303 similarity index 100% rename from src/test/hive/golden/udf_to_string-5-1040b37847d20ef29d545934316303 rename to shark/src/test/hive/golden/udf_to_string-5-1040b37847d20ef29d545934316303 diff --git a/src/test/hive/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 b/shark/src/test/hive/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 similarity index 100% rename from src/test/hive/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 rename to shark/src/test/hive/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 diff --git a/src/test/hive/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 b/shark/src/test/hive/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 similarity index 100% rename from src/test/hive/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 rename to shark/src/test/hive/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 diff --git a/src/test/hive/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea b/shark/src/test/hive/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea similarity index 100% rename from src/test/hive/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea rename to shark/src/test/hive/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea diff --git a/src/test/hive/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b b/shark/src/test/hive/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b similarity index 100% rename from src/test/hive/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b rename to shark/src/test/hive/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b diff --git a/src/test/hive/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 b/shark/src/test/hive/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 similarity index 100% rename from src/test/hive/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 rename to shark/src/test/hive/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 diff --git a/src/test/hive/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 b/shark/src/test/hive/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 similarity index 100% rename from src/test/hive/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 rename to shark/src/test/hive/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 diff --git a/src/test/hive/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 b/shark/src/test/hive/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 similarity index 100% rename from src/test/hive/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 rename to shark/src/test/hive/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 diff --git a/src/test/hive/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b b/shark/src/test/hive/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b similarity index 100% rename from src/test/hive/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b rename to shark/src/test/hive/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b diff --git a/src/test/hive/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 b/shark/src/test/hive/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 similarity index 100% rename from src/test/hive/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 rename to shark/src/test/hive/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 diff --git a/src/test/hive/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 b/shark/src/test/hive/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 similarity index 100% rename from src/test/hive/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 rename to shark/src/test/hive/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 diff --git a/src/test/hive/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a b/shark/src/test/hive/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a similarity index 100% rename from src/test/hive/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a rename to shark/src/test/hive/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a diff --git a/src/test/hive/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba b/shark/src/test/hive/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba similarity index 100% rename from src/test/hive/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba rename to shark/src/test/hive/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba diff --git a/src/test/hive/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 b/shark/src/test/hive/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 similarity index 100% rename from src/test/hive/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 rename to shark/src/test/hive/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 diff --git a/src/test/hive/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b b/shark/src/test/hive/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b similarity index 100% rename from src/test/hive/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b rename to shark/src/test/hive/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b diff --git a/src/test/hive/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 b/shark/src/test/hive/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 similarity index 100% rename from src/test/hive/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 rename to shark/src/test/hive/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 diff --git a/src/test/hive/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 b/shark/src/test/hive/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 similarity index 100% rename from src/test/hive/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 rename to shark/src/test/hive/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 diff --git a/src/test/hive/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 b/shark/src/test/hive/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 similarity index 100% rename from src/test/hive/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 rename to shark/src/test/hive/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 diff --git a/src/test/hive/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f b/shark/src/test/hive/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f similarity index 100% rename from src/test/hive/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f rename to shark/src/test/hive/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f diff --git a/src/test/hive/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 b/shark/src/test/hive/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 similarity index 100% rename from src/test/hive/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 rename to shark/src/test/hive/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 diff --git a/src/test/hive/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d b/shark/src/test/hive/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d similarity index 100% rename from src/test/hive/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d rename to shark/src/test/hive/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d diff --git a/src/test/hive/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f b/shark/src/test/hive/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f similarity index 100% rename from src/test/hive/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f rename to shark/src/test/hive/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f diff --git a/src/test/hive/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 b/shark/src/test/hive/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 similarity index 100% rename from src/test/hive/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 rename to shark/src/test/hive/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 diff --git a/src/test/hive/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 b/shark/src/test/hive/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 similarity index 100% rename from src/test/hive/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 rename to shark/src/test/hive/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 diff --git a/src/test/hive/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 b/shark/src/test/hive/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 similarity index 100% rename from src/test/hive/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 rename to shark/src/test/hive/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 diff --git a/src/test/hive/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 b/shark/src/test/hive/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 similarity index 100% rename from src/test/hive/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 rename to shark/src/test/hive/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 diff --git a/src/test/hive/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c b/shark/src/test/hive/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c similarity index 100% rename from src/test/hive/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c rename to shark/src/test/hive/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c diff --git a/src/test/hive/golden/udf_union-1-f6f01250718087029165e23badc02cd6 b/shark/src/test/hive/golden/udf_union-1-f6f01250718087029165e23badc02cd6 similarity index 100% rename from src/test/hive/golden/udf_union-1-f6f01250718087029165e23badc02cd6 rename to shark/src/test/hive/golden/udf_union-1-f6f01250718087029165e23badc02cd6 diff --git a/src/test/hive/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 b/shark/src/test/hive/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 similarity index 100% rename from src/test/hive/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 rename to shark/src/test/hive/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 diff --git a/src/test/hive/golden/udf_union-3-705d165fec6761744dd19b142c566d61 b/shark/src/test/hive/golden/udf_union-3-705d165fec6761744dd19b142c566d61 similarity index 100% rename from src/test/hive/golden/udf_union-3-705d165fec6761744dd19b142c566d61 rename to shark/src/test/hive/golden/udf_union-3-705d165fec6761744dd19b142c566d61 diff --git a/src/test/hive/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead b/shark/src/test/hive/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead similarity index 100% rename from src/test/hive/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead rename to shark/src/test/hive/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead diff --git a/src/test/hive/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d b/shark/src/test/hive/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d similarity index 100% rename from src/test/hive/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d rename to shark/src/test/hive/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d diff --git a/src/test/hive/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de b/shark/src/test/hive/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de similarity index 100% rename from src/test/hive/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de rename to shark/src/test/hive/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de diff --git a/src/test/hive/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 b/shark/src/test/hive/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 similarity index 100% rename from src/test/hive/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 rename to shark/src/test/hive/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 diff --git a/src/test/hive/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 b/shark/src/test/hive/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 similarity index 100% rename from src/test/hive/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 rename to shark/src/test/hive/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 diff --git a/src/test/hive/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b b/shark/src/test/hive/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b similarity index 100% rename from src/test/hive/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b rename to shark/src/test/hive/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b diff --git a/src/test/hive/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 b/shark/src/test/hive/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 similarity index 100% rename from src/test/hive/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 rename to shark/src/test/hive/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 diff --git a/src/test/hive/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b b/shark/src/test/hive/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b similarity index 100% rename from src/test/hive/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b rename to shark/src/test/hive/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b diff --git a/src/test/hive/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 b/shark/src/test/hive/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 similarity index 100% rename from src/test/hive/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 rename to shark/src/test/hive/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 diff --git a/src/test/hive/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a b/shark/src/test/hive/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a similarity index 100% rename from src/test/hive/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a rename to shark/src/test/hive/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a diff --git a/src/test/hive/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de b/shark/src/test/hive/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de similarity index 100% rename from src/test/hive/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de rename to shark/src/test/hive/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de diff --git a/src/test/hive/golden/udf_variance-3-fd25e5226312bf54d918858511814766 b/shark/src/test/hive/golden/udf_variance-3-fd25e5226312bf54d918858511814766 similarity index 100% rename from src/test/hive/golden/udf_variance-3-fd25e5226312bf54d918858511814766 rename to shark/src/test/hive/golden/udf_variance-3-fd25e5226312bf54d918858511814766 diff --git a/src/test/hive/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 b/shark/src/test/hive/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 similarity index 100% rename from src/test/hive/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 rename to shark/src/test/hive/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 diff --git a/src/test/hive/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a b/shark/src/test/hive/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a similarity index 100% rename from src/test/hive/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a rename to shark/src/test/hive/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a diff --git a/src/test/hive/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de b/shark/src/test/hive/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de similarity index 100% rename from src/test/hive/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de rename to shark/src/test/hive/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de diff --git a/src/test/hive/golden/udf_variance-7-fd25e5226312bf54d918858511814766 b/shark/src/test/hive/golden/udf_variance-7-fd25e5226312bf54d918858511814766 similarity index 100% rename from src/test/hive/golden/udf_variance-7-fd25e5226312bf54d918858511814766 rename to shark/src/test/hive/golden/udf_variance-7-fd25e5226312bf54d918858511814766 diff --git a/src/test/hive/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 b/shark/src/test/hive/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 similarity index 100% rename from src/test/hive/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 rename to shark/src/test/hive/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 diff --git a/src/test/hive/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd b/shark/src/test/hive/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd similarity index 100% rename from src/test/hive/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd rename to shark/src/test/hive/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd diff --git a/src/test/hive/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 b/shark/src/test/hive/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 similarity index 100% rename from src/test/hive/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 rename to shark/src/test/hive/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 diff --git a/src/test/hive/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 b/shark/src/test/hive/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 similarity index 100% rename from src/test/hive/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 rename to shark/src/test/hive/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 diff --git a/src/test/hive/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 b/shark/src/test/hive/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 similarity index 100% rename from src/test/hive/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 rename to shark/src/test/hive/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 diff --git a/src/test/hive/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 b/shark/src/test/hive/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 similarity index 100% rename from src/test/hive/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 rename to shark/src/test/hive/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 diff --git a/src/test/hive/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f b/shark/src/test/hive/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f similarity index 100% rename from src/test/hive/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f rename to shark/src/test/hive/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f diff --git a/src/test/hive/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 b/shark/src/test/hive/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 similarity index 100% rename from src/test/hive/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 rename to shark/src/test/hive/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 diff --git a/src/test/hive/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 b/shark/src/test/hive/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 similarity index 100% rename from src/test/hive/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 rename to shark/src/test/hive/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 diff --git a/src/test/hive/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d b/shark/src/test/hive/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d similarity index 100% rename from src/test/hive/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d rename to shark/src/test/hive/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d diff --git a/src/test/hive/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea b/shark/src/test/hive/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea similarity index 100% rename from src/test/hive/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea rename to shark/src/test/hive/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea diff --git a/src/test/hive/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a b/shark/src/test/hive/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a similarity index 100% rename from src/test/hive/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a rename to shark/src/test/hive/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a diff --git a/src/test/hive/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 b/shark/src/test/hive/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 similarity index 100% rename from src/test/hive/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 rename to shark/src/test/hive/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 diff --git a/src/test/hive/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f b/shark/src/test/hive/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f similarity index 100% rename from src/test/hive/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f rename to shark/src/test/hive/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f diff --git a/src/test/hive/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 b/shark/src/test/hive/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 similarity index 100% rename from src/test/hive/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 rename to shark/src/test/hive/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 diff --git a/src/test/hive/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 b/shark/src/test/hive/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 similarity index 100% rename from src/test/hive/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 rename to shark/src/test/hive/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 diff --git a/src/test/hive/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b b/shark/src/test/hive/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b similarity index 100% rename from src/test/hive/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b rename to shark/src/test/hive/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b diff --git a/src/test/hive/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 b/shark/src/test/hive/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 similarity index 100% rename from src/test/hive/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 rename to shark/src/test/hive/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 diff --git a/src/test/hive/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 b/shark/src/test/hive/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 similarity index 100% rename from src/test/hive/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 rename to shark/src/test/hive/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 diff --git a/src/test/hive/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 b/shark/src/test/hive/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 similarity index 100% rename from src/test/hive/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 rename to shark/src/test/hive/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 diff --git a/src/test/hive/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f b/shark/src/test/hive/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f similarity index 100% rename from src/test/hive/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f rename to shark/src/test/hive/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f diff --git a/src/test/hive/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 b/shark/src/test/hive/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 similarity index 100% rename from src/test/hive/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 rename to shark/src/test/hive/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 diff --git a/src/test/hive/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 b/shark/src/test/hive/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 similarity index 100% rename from src/test/hive/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 rename to shark/src/test/hive/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 diff --git a/src/test/hive/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 b/shark/src/test/hive/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 similarity index 100% rename from src/test/hive/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 rename to shark/src/test/hive/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 diff --git a/src/test/hive/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 b/shark/src/test/hive/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 similarity index 100% rename from src/test/hive/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 rename to shark/src/test/hive/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 diff --git a/src/test/hive/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 b/shark/src/test/hive/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 similarity index 100% rename from src/test/hive/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 rename to shark/src/test/hive/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 diff --git a/src/test/hive/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 b/shark/src/test/hive/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 similarity index 100% rename from src/test/hive/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 rename to shark/src/test/hive/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 diff --git a/src/test/hive/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 b/shark/src/test/hive/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 similarity index 100% rename from src/test/hive/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 rename to shark/src/test/hive/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 diff --git a/src/test/hive/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c b/shark/src/test/hive/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c similarity index 100% rename from src/test/hive/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c rename to shark/src/test/hive/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c diff --git a/src/test/hive/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 b/shark/src/test/hive/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 similarity index 100% rename from src/test/hive/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 rename to shark/src/test/hive/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 diff --git a/src/test/hive/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 b/shark/src/test/hive/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 similarity index 100% rename from src/test/hive/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 rename to shark/src/test/hive/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 diff --git a/src/test/hive/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c b/shark/src/test/hive/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c similarity index 100% rename from src/test/hive/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c rename to shark/src/test/hive/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c diff --git a/src/test/hive/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d b/shark/src/test/hive/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d similarity index 100% rename from src/test/hive/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d rename to shark/src/test/hive/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d diff --git a/src/test/hive/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e b/shark/src/test/hive/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e similarity index 100% rename from src/test/hive/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e rename to shark/src/test/hive/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e diff --git a/src/test/hive/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 b/shark/src/test/hive/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 similarity index 100% rename from src/test/hive/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 rename to shark/src/test/hive/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 diff --git a/src/test/hive/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a b/shark/src/test/hive/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a similarity index 100% rename from src/test/hive/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a rename to shark/src/test/hive/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a diff --git a/src/test/hive/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 b/shark/src/test/hive/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 similarity index 100% rename from src/test/hive/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 rename to shark/src/test/hive/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 diff --git a/src/test/hive/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 b/shark/src/test/hive/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 similarity index 100% rename from src/test/hive/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 rename to shark/src/test/hive/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 diff --git a/src/test/hive/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 b/shark/src/test/hive/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 similarity index 100% rename from src/test/hive/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 rename to shark/src/test/hive/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 diff --git a/src/test/hive/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f b/shark/src/test/hive/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f similarity index 100% rename from src/test/hive/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f rename to shark/src/test/hive/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f diff --git a/src/test/hive/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf b/shark/src/test/hive/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf similarity index 100% rename from src/test/hive/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf rename to shark/src/test/hive/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf diff --git a/src/test/hive/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa b/shark/src/test/hive/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa similarity index 100% rename from src/test/hive/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa rename to shark/src/test/hive/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa diff --git a/src/test/hive/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 b/shark/src/test/hive/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 similarity index 100% rename from src/test/hive/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 rename to shark/src/test/hive/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 diff --git a/src/test/hive/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 b/shark/src/test/hive/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 similarity index 100% rename from src/test/hive/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 rename to shark/src/test/hive/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 diff --git a/src/test/hive/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 b/shark/src/test/hive/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 similarity index 100% rename from src/test/hive/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 rename to shark/src/test/hive/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 diff --git a/src/test/hive/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 b/shark/src/test/hive/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 similarity index 100% rename from src/test/hive/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 rename to shark/src/test/hive/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 diff --git a/src/test/hive/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba b/shark/src/test/hive/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba similarity index 100% rename from src/test/hive/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba rename to shark/src/test/hive/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba diff --git a/src/test/hive/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba b/shark/src/test/hive/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba similarity index 100% rename from src/test/hive/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba rename to shark/src/test/hive/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba diff --git a/src/test/hive/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 b/shark/src/test/hive/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 similarity index 100% rename from src/test/hive/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 rename to shark/src/test/hive/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 diff --git a/src/test/hive/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 b/shark/src/test/hive/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 similarity index 100% rename from src/test/hive/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 rename to shark/src/test/hive/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 diff --git a/src/test/hive/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a b/shark/src/test/hive/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a similarity index 100% rename from src/test/hive/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a rename to shark/src/test/hive/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a diff --git a/src/test/hive/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 b/shark/src/test/hive/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 similarity index 100% rename from src/test/hive/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 rename to shark/src/test/hive/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 diff --git a/src/test/hive/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 b/shark/src/test/hive/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 similarity index 100% rename from src/test/hive/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 rename to shark/src/test/hive/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 diff --git a/src/test/hive/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 b/shark/src/test/hive/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 similarity index 100% rename from src/test/hive/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 rename to shark/src/test/hive/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 diff --git a/src/test/hive/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b b/shark/src/test/hive/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b similarity index 100% rename from src/test/hive/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b rename to shark/src/test/hive/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b diff --git a/src/test/hive/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 b/shark/src/test/hive/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 similarity index 100% rename from src/test/hive/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 rename to shark/src/test/hive/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 diff --git a/src/test/hive/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b b/shark/src/test/hive/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b similarity index 100% rename from src/test/hive/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b rename to shark/src/test/hive/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b diff --git a/src/test/hive/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d b/shark/src/test/hive/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d similarity index 100% rename from src/test/hive/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d rename to shark/src/test/hive/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d diff --git a/src/test/hive/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 b/shark/src/test/hive/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 similarity index 100% rename from src/test/hive/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 rename to shark/src/test/hive/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 diff --git a/src/test/hive/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d b/shark/src/test/hive/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d similarity index 100% rename from src/test/hive/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d rename to shark/src/test/hive/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d diff --git a/src/test/hive/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 b/shark/src/test/hive/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 similarity index 100% rename from src/test/hive/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 rename to shark/src/test/hive/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 diff --git a/src/test/hive/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 b/shark/src/test/hive/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 similarity index 100% rename from src/test/hive/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 rename to shark/src/test/hive/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 diff --git a/src/test/hive/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 b/shark/src/test/hive/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 similarity index 100% rename from src/test/hive/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 rename to shark/src/test/hive/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 diff --git a/src/test/hive/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 b/shark/src/test/hive/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 similarity index 100% rename from src/test/hive/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 rename to shark/src/test/hive/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 diff --git a/src/test/hive/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 b/shark/src/test/hive/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 similarity index 100% rename from src/test/hive/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 rename to shark/src/test/hive/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 diff --git a/src/test/hive/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 b/shark/src/test/hive/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 similarity index 100% rename from src/test/hive/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 rename to shark/src/test/hive/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 diff --git a/src/test/hive/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 b/shark/src/test/hive/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 similarity index 100% rename from src/test/hive/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 rename to shark/src/test/hive/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 diff --git a/src/test/hive/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 b/shark/src/test/hive/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 similarity index 100% rename from src/test/hive/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 rename to shark/src/test/hive/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 diff --git a/src/test/hive/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c b/shark/src/test/hive/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c similarity index 100% rename from src/test/hive/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c rename to shark/src/test/hive/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c diff --git a/src/test/hive/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e b/shark/src/test/hive/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e similarity index 100% rename from src/test/hive/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e rename to shark/src/test/hive/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e diff --git a/src/test/hive/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 b/shark/src/test/hive/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 similarity index 100% rename from src/test/hive/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 rename to shark/src/test/hive/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 diff --git a/src/test/hive/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d b/shark/src/test/hive/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d similarity index 100% rename from src/test/hive/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d rename to shark/src/test/hive/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d diff --git a/src/test/hive/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 b/shark/src/test/hive/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 similarity index 100% rename from src/test/hive/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 rename to shark/src/test/hive/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 diff --git a/src/test/hive/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e b/shark/src/test/hive/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e similarity index 100% rename from src/test/hive/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e rename to shark/src/test/hive/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e diff --git a/src/test/hive/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 b/shark/src/test/hive/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 similarity index 100% rename from src/test/hive/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 rename to shark/src/test/hive/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 diff --git a/src/test/hive/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 b/shark/src/test/hive/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 similarity index 100% rename from src/test/hive/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 rename to shark/src/test/hive/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 diff --git a/src/test/hive/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f b/shark/src/test/hive/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f similarity index 100% rename from src/test/hive/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f rename to shark/src/test/hive/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f diff --git a/src/test/hive/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a b/shark/src/test/hive/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a similarity index 100% rename from src/test/hive/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a rename to shark/src/test/hive/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a diff --git a/src/test/hive/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a b/shark/src/test/hive/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a similarity index 100% rename from src/test/hive/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a rename to shark/src/test/hive/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a diff --git a/src/test/hive/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 b/shark/src/test/hive/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 similarity index 100% rename from src/test/hive/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 rename to shark/src/test/hive/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 diff --git a/src/test/hive/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 b/shark/src/test/hive/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 similarity index 100% rename from src/test/hive/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 rename to shark/src/test/hive/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 diff --git a/src/test/hive/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f b/shark/src/test/hive/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f similarity index 100% rename from src/test/hive/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f rename to shark/src/test/hive/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f diff --git a/src/test/hive/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc b/shark/src/test/hive/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc similarity index 100% rename from src/test/hive/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc rename to shark/src/test/hive/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc diff --git a/src/test/hive/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 b/shark/src/test/hive/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 similarity index 100% rename from src/test/hive/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 rename to shark/src/test/hive/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 diff --git a/src/test/hive/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a b/shark/src/test/hive/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a similarity index 100% rename from src/test/hive/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a rename to shark/src/test/hive/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a diff --git a/src/test/hive/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 b/shark/src/test/hive/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 similarity index 100% rename from src/test/hive/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 rename to shark/src/test/hive/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 diff --git a/src/test/hive/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 b/shark/src/test/hive/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 similarity index 100% rename from src/test/hive/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 rename to shark/src/test/hive/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 diff --git a/src/test/hive/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 b/shark/src/test/hive/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 similarity index 100% rename from src/test/hive/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 rename to shark/src/test/hive/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 diff --git a/src/test/hive/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 b/shark/src/test/hive/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 similarity index 100% rename from src/test/hive/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 rename to shark/src/test/hive/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 diff --git a/src/test/hive/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 b/shark/src/test/hive/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 similarity index 100% rename from src/test/hive/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 rename to shark/src/test/hive/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 diff --git a/src/test/hive/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e b/shark/src/test/hive/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e similarity index 100% rename from src/test/hive/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e rename to shark/src/test/hive/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e diff --git a/src/test/hive/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 b/shark/src/test/hive/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 similarity index 100% rename from src/test/hive/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 rename to shark/src/test/hive/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 diff --git a/src/test/hive/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 b/shark/src/test/hive/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 similarity index 100% rename from src/test/hive/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 rename to shark/src/test/hive/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 diff --git a/src/test/hive/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 b/shark/src/test/hive/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 similarity index 100% rename from src/test/hive/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 rename to shark/src/test/hive/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 diff --git a/src/test/hive/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 b/shark/src/test/hive/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 similarity index 100% rename from src/test/hive/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 rename to shark/src/test/hive/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 diff --git a/src/test/hive/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 b/shark/src/test/hive/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 similarity index 100% rename from src/test/hive/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 rename to shark/src/test/hive/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 diff --git a/src/test/hive/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af b/shark/src/test/hive/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af similarity index 100% rename from src/test/hive/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af rename to shark/src/test/hive/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af diff --git a/src/test/hive/golden/union10-2-60ee9eae687170317ff91dafe6b799bf b/shark/src/test/hive/golden/union10-2-60ee9eae687170317ff91dafe6b799bf similarity index 100% rename from src/test/hive/golden/union10-2-60ee9eae687170317ff91dafe6b799bf rename to shark/src/test/hive/golden/union10-2-60ee9eae687170317ff91dafe6b799bf diff --git a/src/test/hive/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 b/shark/src/test/hive/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 similarity index 100% rename from src/test/hive/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 rename to shark/src/test/hive/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 diff --git a/src/test/hive/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 b/shark/src/test/hive/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 similarity index 100% rename from src/test/hive/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 rename to shark/src/test/hive/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 diff --git a/src/test/hive/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 b/shark/src/test/hive/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 similarity index 100% rename from src/test/hive/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 rename to shark/src/test/hive/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 diff --git a/src/test/hive/golden/union11-2-cd756f39e22e121bdbd51400662aa47f b/shark/src/test/hive/golden/union11-2-cd756f39e22e121bdbd51400662aa47f similarity index 100% rename from src/test/hive/golden/union11-2-cd756f39e22e121bdbd51400662aa47f rename to shark/src/test/hive/golden/union11-2-cd756f39e22e121bdbd51400662aa47f diff --git a/src/test/hive/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 b/shark/src/test/hive/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 similarity index 100% rename from src/test/hive/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 rename to shark/src/test/hive/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 diff --git a/src/test/hive/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 b/shark/src/test/hive/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 similarity index 100% rename from src/test/hive/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 rename to shark/src/test/hive/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 diff --git a/src/test/hive/golden/union13-0-6970d1e2405f3769a28a8804887ac657 b/shark/src/test/hive/golden/union13-0-6970d1e2405f3769a28a8804887ac657 similarity index 100% rename from src/test/hive/golden/union13-0-6970d1e2405f3769a28a8804887ac657 rename to shark/src/test/hive/golden/union13-0-6970d1e2405f3769a28a8804887ac657 diff --git a/src/test/hive/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 b/shark/src/test/hive/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 similarity index 100% rename from src/test/hive/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 rename to shark/src/test/hive/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 diff --git a/src/test/hive/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/union14-1-11603fb6225423979af6c062cfa9389b b/shark/src/test/hive/golden/union14-1-11603fb6225423979af6c062cfa9389b similarity index 100% rename from src/test/hive/golden/union14-1-11603fb6225423979af6c062cfa9389b rename to shark/src/test/hive/golden/union14-1-11603fb6225423979af6c062cfa9389b diff --git a/src/test/hive/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 b/shark/src/test/hive/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 similarity index 100% rename from src/test/hive/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 rename to shark/src/test/hive/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 diff --git a/src/test/hive/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 b/shark/src/test/hive/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 similarity index 100% rename from src/test/hive/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 rename to shark/src/test/hive/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 diff --git a/src/test/hive/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 b/shark/src/test/hive/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 similarity index 100% rename from src/test/hive/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 rename to shark/src/test/hive/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 diff --git a/src/test/hive/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 b/shark/src/test/hive/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 similarity index 100% rename from src/test/hive/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 rename to shark/src/test/hive/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 diff --git a/src/test/hive/golden/union16-1-9f76074598f9b55d8afbb5659737a382 b/shark/src/test/hive/golden/union16-1-9f76074598f9b55d8afbb5659737a382 similarity index 100% rename from src/test/hive/golden/union16-1-9f76074598f9b55d8afbb5659737a382 rename to shark/src/test/hive/golden/union16-1-9f76074598f9b55d8afbb5659737a382 diff --git a/src/test/hive/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a b/shark/src/test/hive/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a similarity index 100% rename from src/test/hive/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a rename to shark/src/test/hive/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a diff --git a/src/test/hive/golden/union17-1-9491c271b7e2c351ddcf126f73679892 b/shark/src/test/hive/golden/union17-1-9491c271b7e2c351ddcf126f73679892 similarity index 100% rename from src/test/hive/golden/union17-1-9491c271b7e2c351ddcf126f73679892 rename to shark/src/test/hive/golden/union17-1-9491c271b7e2c351ddcf126f73679892 diff --git a/src/test/hive/golden/union17-2-e6fb104913b9da8193167ee263993dd1 b/shark/src/test/hive/golden/union17-2-e6fb104913b9da8193167ee263993dd1 similarity index 100% rename from src/test/hive/golden/union17-2-e6fb104913b9da8193167ee263993dd1 rename to shark/src/test/hive/golden/union17-2-e6fb104913b9da8193167ee263993dd1 diff --git a/src/test/hive/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 b/shark/src/test/hive/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 similarity index 100% rename from src/test/hive/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 rename to shark/src/test/hive/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 diff --git a/src/test/hive/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/shark/src/test/hive/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 similarity index 100% rename from src/test/hive/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 rename to shark/src/test/hive/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 diff --git a/src/test/hive/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 b/shark/src/test/hive/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 similarity index 100% rename from src/test/hive/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 rename to shark/src/test/hive/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 diff --git a/src/test/hive/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a b/shark/src/test/hive/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a similarity index 100% rename from src/test/hive/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a rename to shark/src/test/hive/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a diff --git a/src/test/hive/golden/union18-1-9491c271b7e2c351ddcf126f73679892 b/shark/src/test/hive/golden/union18-1-9491c271b7e2c351ddcf126f73679892 similarity index 100% rename from src/test/hive/golden/union18-1-9491c271b7e2c351ddcf126f73679892 rename to shark/src/test/hive/golden/union18-1-9491c271b7e2c351ddcf126f73679892 diff --git a/src/test/hive/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 b/shark/src/test/hive/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 similarity index 100% rename from src/test/hive/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 rename to shark/src/test/hive/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 diff --git a/src/test/hive/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 b/shark/src/test/hive/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 similarity index 100% rename from src/test/hive/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 rename to shark/src/test/hive/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 diff --git a/src/test/hive/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 b/shark/src/test/hive/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 similarity index 100% rename from src/test/hive/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 rename to shark/src/test/hive/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 diff --git a/src/test/hive/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 b/shark/src/test/hive/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 similarity index 100% rename from src/test/hive/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 rename to shark/src/test/hive/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 diff --git a/src/test/hive/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a b/shark/src/test/hive/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a similarity index 100% rename from src/test/hive/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a rename to shark/src/test/hive/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a diff --git a/src/test/hive/golden/union19-1-9491c271b7e2c351ddcf126f73679892 b/shark/src/test/hive/golden/union19-1-9491c271b7e2c351ddcf126f73679892 similarity index 100% rename from src/test/hive/golden/union19-1-9491c271b7e2c351ddcf126f73679892 rename to shark/src/test/hive/golden/union19-1-9491c271b7e2c351ddcf126f73679892 diff --git a/src/test/hive/golden/union19-2-b6161b64e57a2502c79662866782cbdf b/shark/src/test/hive/golden/union19-2-b6161b64e57a2502c79662866782cbdf similarity index 100% rename from src/test/hive/golden/union19-2-b6161b64e57a2502c79662866782cbdf rename to shark/src/test/hive/golden/union19-2-b6161b64e57a2502c79662866782cbdf diff --git a/src/test/hive/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 b/shark/src/test/hive/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 similarity index 100% rename from src/test/hive/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 rename to shark/src/test/hive/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 diff --git a/src/test/hive/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 b/shark/src/test/hive/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 similarity index 100% rename from src/test/hive/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 rename to shark/src/test/hive/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 diff --git a/src/test/hive/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 b/shark/src/test/hive/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 similarity index 100% rename from src/test/hive/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 rename to shark/src/test/hive/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 diff --git a/src/test/hive/golden/union2-0-58813daf3d1af57b17518b606f8345b2 b/shark/src/test/hive/golden/union2-0-58813daf3d1af57b17518b606f8345b2 similarity index 100% rename from src/test/hive/golden/union2-0-58813daf3d1af57b17518b606f8345b2 rename to shark/src/test/hive/golden/union2-0-58813daf3d1af57b17518b606f8345b2 diff --git a/src/test/hive/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 b/shark/src/test/hive/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 similarity index 100% rename from src/test/hive/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 rename to shark/src/test/hive/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 diff --git a/src/test/hive/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d b/shark/src/test/hive/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d similarity index 100% rename from src/test/hive/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d rename to shark/src/test/hive/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d diff --git a/src/test/hive/golden/union20-1-968e353589f1fddb914242beb25be94c b/shark/src/test/hive/golden/union20-1-968e353589f1fddb914242beb25be94c similarity index 100% rename from src/test/hive/golden/union20-1-968e353589f1fddb914242beb25be94c rename to shark/src/test/hive/golden/union20-1-968e353589f1fddb914242beb25be94c diff --git a/src/test/hive/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c b/shark/src/test/hive/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c similarity index 100% rename from src/test/hive/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c rename to shark/src/test/hive/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c diff --git a/src/test/hive/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 b/shark/src/test/hive/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 similarity index 100% rename from src/test/hive/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 rename to shark/src/test/hive/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 diff --git a/src/test/hive/golden/union22-0-4bd6583b3635f1db95765565518bda8d b/shark/src/test/hive/golden/union22-0-4bd6583b3635f1db95765565518bda8d similarity index 100% rename from src/test/hive/golden/union22-0-4bd6583b3635f1db95765565518bda8d rename to shark/src/test/hive/golden/union22-0-4bd6583b3635f1db95765565518bda8d diff --git a/src/test/hive/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 b/shark/src/test/hive/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 similarity index 100% rename from src/test/hive/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 rename to shark/src/test/hive/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 diff --git a/src/test/hive/golden/union22-10-90e9c4388340428a1f68868e9322c400 b/shark/src/test/hive/golden/union22-10-90e9c4388340428a1f68868e9322c400 similarity index 100% rename from src/test/hive/golden/union22-10-90e9c4388340428a1f68868e9322c400 rename to shark/src/test/hive/golden/union22-10-90e9c4388340428a1f68868e9322c400 diff --git a/src/test/hive/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 b/shark/src/test/hive/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 similarity index 100% rename from src/test/hive/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 rename to shark/src/test/hive/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 diff --git a/src/test/hive/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 b/shark/src/test/hive/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 similarity index 100% rename from src/test/hive/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 rename to shark/src/test/hive/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 diff --git a/src/test/hive/golden/union22-4-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union22-4-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union22-4-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union22-4-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union22-5-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/union22-5-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/union22-5-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/union22-5-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 b/shark/src/test/hive/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from src/test/hive/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 rename to shark/src/test/hive/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 diff --git a/src/test/hive/golden/union22-7-25fc734982956a164adde6bb1d4d8751 b/shark/src/test/hive/golden/union22-7-25fc734982956a164adde6bb1d4d8751 similarity index 100% rename from src/test/hive/golden/union22-7-25fc734982956a164adde6bb1d4d8751 rename to shark/src/test/hive/golden/union22-7-25fc734982956a164adde6bb1d4d8751 diff --git a/src/test/hive/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 b/shark/src/test/hive/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 similarity index 100% rename from src/test/hive/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 rename to shark/src/test/hive/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 diff --git a/src/test/hive/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 b/shark/src/test/hive/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 similarity index 100% rename from src/test/hive/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 rename to shark/src/test/hive/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 diff --git a/src/test/hive/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 b/shark/src/test/hive/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 similarity index 100% rename from src/test/hive/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 rename to shark/src/test/hive/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 diff --git a/src/test/hive/golden/union23-1-7830963417e3535034962e2597970ddd b/shark/src/test/hive/golden/union23-1-7830963417e3535034962e2597970ddd similarity index 100% rename from src/test/hive/golden/union23-1-7830963417e3535034962e2597970ddd rename to shark/src/test/hive/golden/union23-1-7830963417e3535034962e2597970ddd diff --git a/src/test/hive/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 b/shark/src/test/hive/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 similarity index 100% rename from src/test/hive/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 rename to shark/src/test/hive/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 diff --git a/src/test/hive/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c b/shark/src/test/hive/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c similarity index 100% rename from src/test/hive/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c rename to shark/src/test/hive/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c diff --git a/src/test/hive/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 b/shark/src/test/hive/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 similarity index 100% rename from src/test/hive/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 rename to shark/src/test/hive/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 diff --git a/src/test/hive/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a b/shark/src/test/hive/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a similarity index 100% rename from src/test/hive/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a rename to shark/src/test/hive/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a diff --git a/src/test/hive/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 b/shark/src/test/hive/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 similarity index 100% rename from src/test/hive/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 rename to shark/src/test/hive/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 diff --git a/src/test/hive/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f b/shark/src/test/hive/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f similarity index 100% rename from src/test/hive/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f rename to shark/src/test/hive/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f diff --git a/src/test/hive/golden/union24-4-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union24-4-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union24-4-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union24-4-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 b/shark/src/test/hive/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 similarity index 100% rename from src/test/hive/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 rename to shark/src/test/hive/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 diff --git a/src/test/hive/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 b/shark/src/test/hive/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 similarity index 100% rename from src/test/hive/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 rename to shark/src/test/hive/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 diff --git a/src/test/hive/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb b/shark/src/test/hive/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb similarity index 100% rename from src/test/hive/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb rename to shark/src/test/hive/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb diff --git a/src/test/hive/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 b/shark/src/test/hive/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 similarity index 100% rename from src/test/hive/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 rename to shark/src/test/hive/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 diff --git a/src/test/hive/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d b/shark/src/test/hive/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d similarity index 100% rename from src/test/hive/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d rename to shark/src/test/hive/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d diff --git a/src/test/hive/golden/union26-1-4702087f160230463e6114d6534d47e4 b/shark/src/test/hive/golden/union26-1-4702087f160230463e6114d6534d47e4 similarity index 100% rename from src/test/hive/golden/union26-1-4702087f160230463e6114d6534d47e4 rename to shark/src/test/hive/golden/union26-1-4702087f160230463e6114d6534d47e4 diff --git a/src/test/hive/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 b/shark/src/test/hive/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 similarity index 100% rename from src/test/hive/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 rename to shark/src/test/hive/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 diff --git a/src/test/hive/golden/union26-3-4702087f160230463e6114d6534d47e4 b/shark/src/test/hive/golden/union26-3-4702087f160230463e6114d6534d47e4 similarity index 100% rename from src/test/hive/golden/union26-3-4702087f160230463e6114d6534d47e4 rename to shark/src/test/hive/golden/union26-3-4702087f160230463e6114d6534d47e4 diff --git a/src/test/hive/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 b/shark/src/test/hive/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 similarity index 100% rename from src/test/hive/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 rename to shark/src/test/hive/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 diff --git a/src/test/hive/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f b/shark/src/test/hive/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f similarity index 100% rename from src/test/hive/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f rename to shark/src/test/hive/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f diff --git a/src/test/hive/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 b/shark/src/test/hive/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 similarity index 100% rename from src/test/hive/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 rename to shark/src/test/hive/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 diff --git a/src/test/hive/golden/union27-3-ab84df3813ff23be99f148449610e530 b/shark/src/test/hive/golden/union27-3-ab84df3813ff23be99f148449610e530 similarity index 100% rename from src/test/hive/golden/union27-3-ab84df3813ff23be99f148449610e530 rename to shark/src/test/hive/golden/union27-3-ab84df3813ff23be99f148449610e530 diff --git a/src/test/hive/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 b/shark/src/test/hive/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 similarity index 100% rename from src/test/hive/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 rename to shark/src/test/hive/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 diff --git a/src/test/hive/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 b/shark/src/test/hive/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 similarity index 100% rename from src/test/hive/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 rename to shark/src/test/hive/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 diff --git a/src/test/hive/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f b/shark/src/test/hive/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f similarity index 100% rename from src/test/hive/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f rename to shark/src/test/hive/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f diff --git a/src/test/hive/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 b/shark/src/test/hive/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 similarity index 100% rename from src/test/hive/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 rename to shark/src/test/hive/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 diff --git a/src/test/hive/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 b/shark/src/test/hive/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 similarity index 100% rename from src/test/hive/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 rename to shark/src/test/hive/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 diff --git a/src/test/hive/golden/union29-1-a40036f756a306f4226634b6ab67c72c b/shark/src/test/hive/golden/union29-1-a40036f756a306f4226634b6ab67c72c similarity index 100% rename from src/test/hive/golden/union29-1-a40036f756a306f4226634b6ab67c72c rename to shark/src/test/hive/golden/union29-1-a40036f756a306f4226634b6ab67c72c diff --git a/src/test/hive/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 b/shark/src/test/hive/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 similarity index 100% rename from src/test/hive/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 rename to shark/src/test/hive/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 diff --git a/src/test/hive/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 b/shark/src/test/hive/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 similarity index 100% rename from src/test/hive/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 rename to shark/src/test/hive/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 diff --git a/src/test/hive/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d b/shark/src/test/hive/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d similarity index 100% rename from src/test/hive/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d rename to shark/src/test/hive/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d diff --git a/src/test/hive/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 b/shark/src/test/hive/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 similarity index 100% rename from src/test/hive/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 rename to shark/src/test/hive/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 diff --git a/src/test/hive/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 b/shark/src/test/hive/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 similarity index 100% rename from src/test/hive/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 rename to shark/src/test/hive/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 diff --git a/src/test/hive/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 b/shark/src/test/hive/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 similarity index 100% rename from src/test/hive/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 rename to shark/src/test/hive/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 diff --git a/src/test/hive/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 b/shark/src/test/hive/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 similarity index 100% rename from src/test/hive/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 rename to shark/src/test/hive/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 diff --git a/src/test/hive/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 b/shark/src/test/hive/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 similarity index 100% rename from src/test/hive/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 rename to shark/src/test/hive/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 diff --git a/src/test/hive/golden/union30-2-252f12f7532ca67132bfc62063c58430 b/shark/src/test/hive/golden/union30-2-252f12f7532ca67132bfc62063c58430 similarity index 100% rename from src/test/hive/golden/union30-2-252f12f7532ca67132bfc62063c58430 rename to shark/src/test/hive/golden/union30-2-252f12f7532ca67132bfc62063c58430 diff --git a/src/test/hive/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 b/shark/src/test/hive/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 similarity index 100% rename from src/test/hive/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 rename to shark/src/test/hive/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 diff --git a/src/test/hive/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 b/shark/src/test/hive/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 similarity index 100% rename from src/test/hive/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 rename to shark/src/test/hive/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 diff --git a/src/test/hive/golden/union31-1-b12aa87aaf64b573ce0be7013117651b b/shark/src/test/hive/golden/union31-1-b12aa87aaf64b573ce0be7013117651b similarity index 100% rename from src/test/hive/golden/union31-1-b12aa87aaf64b573ce0be7013117651b rename to shark/src/test/hive/golden/union31-1-b12aa87aaf64b573ce0be7013117651b diff --git a/src/test/hive/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 b/shark/src/test/hive/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 similarity index 100% rename from src/test/hive/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 rename to shark/src/test/hive/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 diff --git a/src/test/hive/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 b/shark/src/test/hive/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 similarity index 100% rename from src/test/hive/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 rename to shark/src/test/hive/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 diff --git a/src/test/hive/golden/union31-12-de2477a134ab37bb22d879028becf2e5 b/shark/src/test/hive/golden/union31-12-de2477a134ab37bb22d879028becf2e5 similarity index 100% rename from src/test/hive/golden/union31-12-de2477a134ab37bb22d879028becf2e5 rename to shark/src/test/hive/golden/union31-12-de2477a134ab37bb22d879028becf2e5 diff --git a/src/test/hive/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 b/shark/src/test/hive/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 similarity index 100% rename from src/test/hive/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 rename to shark/src/test/hive/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 diff --git a/src/test/hive/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 b/shark/src/test/hive/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 similarity index 100% rename from src/test/hive/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 rename to shark/src/test/hive/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 diff --git a/src/test/hive/golden/union31-15-5df6435aed6e0a6a6853480a027b911e b/shark/src/test/hive/golden/union31-15-5df6435aed6e0a6a6853480a027b911e similarity index 100% rename from src/test/hive/golden/union31-15-5df6435aed6e0a6a6853480a027b911e rename to shark/src/test/hive/golden/union31-15-5df6435aed6e0a6a6853480a027b911e diff --git a/src/test/hive/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 b/shark/src/test/hive/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 similarity index 100% rename from src/test/hive/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 rename to shark/src/test/hive/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 diff --git a/src/test/hive/golden/union31-17-b12aa87aaf64b573ce0be7013117651b b/shark/src/test/hive/golden/union31-17-b12aa87aaf64b573ce0be7013117651b similarity index 100% rename from src/test/hive/golden/union31-17-b12aa87aaf64b573ce0be7013117651b rename to shark/src/test/hive/golden/union31-17-b12aa87aaf64b573ce0be7013117651b diff --git a/src/test/hive/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 b/shark/src/test/hive/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 similarity index 100% rename from src/test/hive/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 rename to shark/src/test/hive/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 diff --git a/src/test/hive/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e b/shark/src/test/hive/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e similarity index 100% rename from src/test/hive/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e rename to shark/src/test/hive/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e diff --git a/src/test/hive/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 b/shark/src/test/hive/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 similarity index 100% rename from src/test/hive/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 rename to shark/src/test/hive/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 diff --git a/src/test/hive/golden/union31-20-dceb6d9a03096d68d090557692ed6899 b/shark/src/test/hive/golden/union31-20-dceb6d9a03096d68d090557692ed6899 similarity index 100% rename from src/test/hive/golden/union31-20-dceb6d9a03096d68d090557692ed6899 rename to shark/src/test/hive/golden/union31-20-dceb6d9a03096d68d090557692ed6899 diff --git a/src/test/hive/golden/union31-21-bda2e62d0ecd30200b2359df420da574 b/shark/src/test/hive/golden/union31-21-bda2e62d0ecd30200b2359df420da574 similarity index 100% rename from src/test/hive/golden/union31-21-bda2e62d0ecd30200b2359df420da574 rename to shark/src/test/hive/golden/union31-21-bda2e62d0ecd30200b2359df420da574 diff --git a/src/test/hive/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a b/shark/src/test/hive/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a similarity index 100% rename from src/test/hive/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a rename to shark/src/test/hive/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a diff --git a/src/test/hive/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df b/shark/src/test/hive/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df similarity index 100% rename from src/test/hive/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df rename to shark/src/test/hive/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df diff --git a/src/test/hive/golden/union31-24-df38c8164af7cc164c728b8178da72c5 b/shark/src/test/hive/golden/union31-24-df38c8164af7cc164c728b8178da72c5 similarity index 100% rename from src/test/hive/golden/union31-24-df38c8164af7cc164c728b8178da72c5 rename to shark/src/test/hive/golden/union31-24-df38c8164af7cc164c728b8178da72c5 diff --git a/src/test/hive/golden/union31-25-1485e295a99908e1862eae397b814045 b/shark/src/test/hive/golden/union31-25-1485e295a99908e1862eae397b814045 similarity index 100% rename from src/test/hive/golden/union31-25-1485e295a99908e1862eae397b814045 rename to shark/src/test/hive/golden/union31-25-1485e295a99908e1862eae397b814045 diff --git a/src/test/hive/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc b/shark/src/test/hive/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc similarity index 100% rename from src/test/hive/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc rename to shark/src/test/hive/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc diff --git a/src/test/hive/golden/union31-4-77653483e5ce18437c9f85d3b297335d b/shark/src/test/hive/golden/union31-4-77653483e5ce18437c9f85d3b297335d similarity index 100% rename from src/test/hive/golden/union31-4-77653483e5ce18437c9f85d3b297335d rename to shark/src/test/hive/golden/union31-4-77653483e5ce18437c9f85d3b297335d diff --git a/src/test/hive/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 b/shark/src/test/hive/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 similarity index 100% rename from src/test/hive/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 rename to shark/src/test/hive/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 diff --git a/src/test/hive/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 b/shark/src/test/hive/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 similarity index 100% rename from src/test/hive/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 rename to shark/src/test/hive/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 diff --git a/src/test/hive/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 b/shark/src/test/hive/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 similarity index 100% rename from src/test/hive/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 rename to shark/src/test/hive/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 diff --git a/src/test/hive/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb b/shark/src/test/hive/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb similarity index 100% rename from src/test/hive/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb rename to shark/src/test/hive/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb diff --git a/src/test/hive/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 b/shark/src/test/hive/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 similarity index 100% rename from src/test/hive/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 rename to shark/src/test/hive/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 diff --git a/src/test/hive/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e b/shark/src/test/hive/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e similarity index 100% rename from src/test/hive/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e rename to shark/src/test/hive/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e diff --git a/src/test/hive/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 b/shark/src/test/hive/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 similarity index 100% rename from src/test/hive/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 rename to shark/src/test/hive/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 diff --git a/src/test/hive/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b b/shark/src/test/hive/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b similarity index 100% rename from src/test/hive/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b rename to shark/src/test/hive/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b diff --git a/src/test/hive/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 b/shark/src/test/hive/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 similarity index 100% rename from src/test/hive/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 rename to shark/src/test/hive/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 diff --git a/src/test/hive/golden/union32-2-8e39fef33c859ef83912d0fcda319218 b/shark/src/test/hive/golden/union32-2-8e39fef33c859ef83912d0fcda319218 similarity index 100% rename from src/test/hive/golden/union32-2-8e39fef33c859ef83912d0fcda319218 rename to shark/src/test/hive/golden/union32-2-8e39fef33c859ef83912d0fcda319218 diff --git a/src/test/hive/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 b/shark/src/test/hive/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 similarity index 100% rename from src/test/hive/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 rename to shark/src/test/hive/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 diff --git a/src/test/hive/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e b/shark/src/test/hive/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e similarity index 100% rename from src/test/hive/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e rename to shark/src/test/hive/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e diff --git a/src/test/hive/golden/union32-5-51c997d0a1103b60764bbb8316a38746 b/shark/src/test/hive/golden/union32-5-51c997d0a1103b60764bbb8316a38746 similarity index 100% rename from src/test/hive/golden/union32-5-51c997d0a1103b60764bbb8316a38746 rename to shark/src/test/hive/golden/union32-5-51c997d0a1103b60764bbb8316a38746 diff --git a/src/test/hive/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b b/shark/src/test/hive/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b similarity index 100% rename from src/test/hive/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b rename to shark/src/test/hive/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b diff --git a/src/test/hive/golden/union32-7-f936440d63f4e1027dda4de605660633 b/shark/src/test/hive/golden/union32-7-f936440d63f4e1027dda4de605660633 similarity index 100% rename from src/test/hive/golden/union32-7-f936440d63f4e1027dda4de605660633 rename to shark/src/test/hive/golden/union32-7-f936440d63f4e1027dda4de605660633 diff --git a/src/test/hive/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb b/shark/src/test/hive/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb similarity index 100% rename from src/test/hive/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb rename to shark/src/test/hive/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb diff --git a/src/test/hive/golden/union32-9-74fb695786df4c024288ae23ac8c00db b/shark/src/test/hive/golden/union32-9-74fb695786df4c024288ae23ac8c00db similarity index 100% rename from src/test/hive/golden/union32-9-74fb695786df4c024288ae23ac8c00db rename to shark/src/test/hive/golden/union32-9-74fb695786df4c024288ae23ac8c00db diff --git a/src/test/hive/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f b/shark/src/test/hive/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f similarity index 100% rename from src/test/hive/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f rename to shark/src/test/hive/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f diff --git a/src/test/hive/golden/union33-1-1df7c476abb48234c839878414720873 b/shark/src/test/hive/golden/union33-1-1df7c476abb48234c839878414720873 similarity index 100% rename from src/test/hive/golden/union33-1-1df7c476abb48234c839878414720873 rename to shark/src/test/hive/golden/union33-1-1df7c476abb48234c839878414720873 diff --git a/src/test/hive/golden/union33-2-3987150ad66482f5186826efcfb117f7 b/shark/src/test/hive/golden/union33-2-3987150ad66482f5186826efcfb117f7 similarity index 100% rename from src/test/hive/golden/union33-2-3987150ad66482f5186826efcfb117f7 rename to shark/src/test/hive/golden/union33-2-3987150ad66482f5186826efcfb117f7 diff --git a/src/test/hive/golden/union33-3-bfe75625806b414a5098af487b91046a b/shark/src/test/hive/golden/union33-3-bfe75625806b414a5098af487b91046a similarity index 100% rename from src/test/hive/golden/union33-3-bfe75625806b414a5098af487b91046a rename to shark/src/test/hive/golden/union33-3-bfe75625806b414a5098af487b91046a diff --git a/src/test/hive/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 b/shark/src/test/hive/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 similarity index 100% rename from src/test/hive/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 rename to shark/src/test/hive/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 diff --git a/src/test/hive/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 b/shark/src/test/hive/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 similarity index 100% rename from src/test/hive/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 rename to shark/src/test/hive/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 diff --git a/src/test/hive/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 b/shark/src/test/hive/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 similarity index 100% rename from src/test/hive/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 rename to shark/src/test/hive/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 diff --git a/src/test/hive/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 b/shark/src/test/hive/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 similarity index 100% rename from src/test/hive/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 rename to shark/src/test/hive/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 diff --git a/src/test/hive/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 b/shark/src/test/hive/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 similarity index 100% rename from src/test/hive/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 rename to shark/src/test/hive/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 diff --git a/src/test/hive/golden/union34-1-320c2fe802fab386957039aaff6399f1 b/shark/src/test/hive/golden/union34-1-320c2fe802fab386957039aaff6399f1 similarity index 100% rename from src/test/hive/golden/union34-1-320c2fe802fab386957039aaff6399f1 rename to shark/src/test/hive/golden/union34-1-320c2fe802fab386957039aaff6399f1 diff --git a/src/test/hive/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 b/shark/src/test/hive/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 similarity index 100% rename from src/test/hive/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 rename to shark/src/test/hive/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 diff --git a/src/test/hive/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 b/shark/src/test/hive/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 similarity index 100% rename from src/test/hive/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 rename to shark/src/test/hive/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 diff --git a/src/test/hive/golden/union34-3-c179e14d840249d6953cb552279822f1 b/shark/src/test/hive/golden/union34-3-c179e14d840249d6953cb552279822f1 similarity index 100% rename from src/test/hive/golden/union34-3-c179e14d840249d6953cb552279822f1 rename to shark/src/test/hive/golden/union34-3-c179e14d840249d6953cb552279822f1 diff --git a/src/test/hive/golden/union34-4-101829a66cab2efd31dcb0d86e302956 b/shark/src/test/hive/golden/union34-4-101829a66cab2efd31dcb0d86e302956 similarity index 100% rename from src/test/hive/golden/union34-4-101829a66cab2efd31dcb0d86e302956 rename to shark/src/test/hive/golden/union34-4-101829a66cab2efd31dcb0d86e302956 diff --git a/src/test/hive/golden/union34-5-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/union34-5-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/union34-5-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/union34-5-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f b/shark/src/test/hive/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f similarity index 100% rename from src/test/hive/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f rename to shark/src/test/hive/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f diff --git a/src/test/hive/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 b/shark/src/test/hive/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 similarity index 100% rename from src/test/hive/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 rename to shark/src/test/hive/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 diff --git a/src/test/hive/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b b/shark/src/test/hive/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from src/test/hive/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b rename to shark/src/test/hive/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b diff --git a/src/test/hive/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 b/shark/src/test/hive/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 similarity index 100% rename from src/test/hive/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 rename to shark/src/test/hive/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 diff --git a/src/test/hive/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/union4-1-88deb3987655dc41989f39c0827cec74 b/shark/src/test/hive/golden/union4-1-88deb3987655dc41989f39c0827cec74 similarity index 100% rename from src/test/hive/golden/union4-1-88deb3987655dc41989f39c0827cec74 rename to shark/src/test/hive/golden/union4-1-88deb3987655dc41989f39c0827cec74 diff --git a/src/test/hive/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 b/shark/src/test/hive/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 similarity index 100% rename from src/test/hive/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 rename to shark/src/test/hive/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 diff --git a/src/test/hive/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b b/shark/src/test/hive/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b similarity index 100% rename from src/test/hive/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b rename to shark/src/test/hive/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b diff --git a/src/test/hive/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 b/shark/src/test/hive/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 similarity index 100% rename from src/test/hive/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 rename to shark/src/test/hive/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 diff --git a/src/test/hive/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 b/shark/src/test/hive/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 similarity index 100% rename from src/test/hive/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 rename to shark/src/test/hive/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 diff --git a/src/test/hive/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 b/shark/src/test/hive/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 similarity index 100% rename from src/test/hive/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 rename to shark/src/test/hive/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 diff --git a/src/test/hive/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d b/shark/src/test/hive/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d similarity index 100% rename from src/test/hive/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d rename to shark/src/test/hive/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d diff --git a/src/test/hive/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 b/shark/src/test/hive/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 similarity index 100% rename from src/test/hive/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 rename to shark/src/test/hive/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 diff --git a/src/test/hive/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 b/shark/src/test/hive/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 similarity index 100% rename from src/test/hive/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 rename to shark/src/test/hive/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 diff --git a/src/test/hive/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 b/shark/src/test/hive/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 similarity index 100% rename from src/test/hive/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 rename to shark/src/test/hive/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 diff --git a/src/test/hive/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 b/shark/src/test/hive/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from src/test/hive/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 rename to shark/src/test/hive/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 diff --git a/src/test/hive/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 b/shark/src/test/hive/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 similarity index 100% rename from src/test/hive/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 rename to shark/src/test/hive/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 diff --git a/src/test/hive/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 b/shark/src/test/hive/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 similarity index 100% rename from src/test/hive/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 rename to shark/src/test/hive/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 diff --git a/src/test/hive/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e b/shark/src/test/hive/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e similarity index 100% rename from src/test/hive/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e rename to shark/src/test/hive/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e diff --git a/src/test/hive/golden/union8-1-1b422e4c1c8b97775518f760b995c771 b/shark/src/test/hive/golden/union8-1-1b422e4c1c8b97775518f760b995c771 similarity index 100% rename from src/test/hive/golden/union8-1-1b422e4c1c8b97775518f760b995c771 rename to shark/src/test/hive/golden/union8-1-1b422e4c1c8b97775518f760b995c771 diff --git a/src/test/hive/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 b/shark/src/test/hive/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 similarity index 100% rename from src/test/hive/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 rename to shark/src/test/hive/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 diff --git a/src/test/hive/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 b/shark/src/test/hive/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 similarity index 100% rename from src/test/hive/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 rename to shark/src/test/hive/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 diff --git a/src/test/hive/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 b/shark/src/test/hive/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 similarity index 100% rename from src/test/hive/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 rename to shark/src/test/hive/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 diff --git a/src/test/hive/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c b/shark/src/test/hive/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c similarity index 100% rename from src/test/hive/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c rename to shark/src/test/hive/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c diff --git a/src/test/hive/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 b/shark/src/test/hive/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 similarity index 100% rename from src/test/hive/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 rename to shark/src/test/hive/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 diff --git a/src/test/hive/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 b/shark/src/test/hive/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 similarity index 100% rename from src/test/hive/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 rename to shark/src/test/hive/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 diff --git a/src/test/hive/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c b/shark/src/test/hive/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c similarity index 100% rename from src/test/hive/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c rename to shark/src/test/hive/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c diff --git a/src/test/hive/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d b/shark/src/test/hive/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d similarity index 100% rename from src/test/hive/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d rename to shark/src/test/hive/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d diff --git a/src/test/hive/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 b/shark/src/test/hive/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 similarity index 100% rename from src/test/hive/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 rename to shark/src/test/hive/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 diff --git a/src/test/hive/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 b/shark/src/test/hive/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 similarity index 100% rename from src/test/hive/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 rename to shark/src/test/hive/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 diff --git a/src/test/hive/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 b/shark/src/test/hive/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 similarity index 100% rename from src/test/hive/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 rename to shark/src/test/hive/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 diff --git a/src/test/hive/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 b/shark/src/test/hive/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 similarity index 100% rename from src/test/hive/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 rename to shark/src/test/hive/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 diff --git a/src/test/hive/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a b/shark/src/test/hive/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a similarity index 100% rename from src/test/hive/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a rename to shark/src/test/hive/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a diff --git a/src/test/hive/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b b/shark/src/test/hive/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b similarity index 100% rename from src/test/hive/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b rename to shark/src/test/hive/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b diff --git a/src/test/hive/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed b/shark/src/test/hive/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed similarity index 100% rename from src/test/hive/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed rename to shark/src/test/hive/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed diff --git a/src/test/hive/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/shark/src/test/hive/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 similarity index 100% rename from src/test/hive/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 rename to shark/src/test/hive/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 diff --git a/src/test/hive/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 b/shark/src/test/hive/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 similarity index 100% rename from src/test/hive/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 rename to shark/src/test/hive/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 diff --git a/src/test/hive/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a b/shark/src/test/hive/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a similarity index 100% rename from src/test/hive/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a rename to shark/src/test/hive/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a diff --git a/src/test/hive/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 b/shark/src/test/hive/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 similarity index 100% rename from src/test/hive/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 rename to shark/src/test/hive/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 diff --git a/src/test/hive/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 b/shark/src/test/hive/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 similarity index 100% rename from src/test/hive/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 rename to shark/src/test/hive/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 diff --git a/src/test/hive/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 b/shark/src/test/hive/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 similarity index 100% rename from src/test/hive/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 rename to shark/src/test/hive/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 diff --git a/src/test/hive/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc b/shark/src/test/hive/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc similarity index 100% rename from src/test/hive/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc rename to shark/src/test/hive/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc diff --git a/src/test/hive/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 b/shark/src/test/hive/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 similarity index 100% rename from src/test/hive/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 rename to shark/src/test/hive/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 diff --git a/src/test/hive/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/shark/src/test/hive/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 similarity index 100% rename from src/test/hive/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 rename to shark/src/test/hive/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 diff --git a/src/test/hive/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 b/shark/src/test/hive/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 similarity index 100% rename from src/test/hive/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 rename to shark/src/test/hive/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 diff --git a/src/test/hive/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a b/shark/src/test/hive/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a similarity index 100% rename from src/test/hive/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a rename to shark/src/test/hive/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a diff --git a/src/test/hive/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e b/shark/src/test/hive/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e similarity index 100% rename from src/test/hive/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e rename to shark/src/test/hive/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e diff --git a/src/test/hive/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc b/shark/src/test/hive/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc similarity index 100% rename from src/test/hive/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc rename to shark/src/test/hive/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc diff --git a/src/test/hive/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 b/shark/src/test/hive/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 similarity index 100% rename from src/test/hive/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 rename to shark/src/test/hive/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 diff --git a/src/test/hive/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/shark/src/test/hive/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 similarity index 100% rename from src/test/hive/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 rename to shark/src/test/hive/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 diff --git a/src/test/hive/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c b/shark/src/test/hive/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c similarity index 100% rename from src/test/hive/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c rename to shark/src/test/hive/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c diff --git a/src/test/hive/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 b/shark/src/test/hive/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 similarity index 100% rename from src/test/hive/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 rename to shark/src/test/hive/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 diff --git a/src/test/hive/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc b/shark/src/test/hive/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc similarity index 100% rename from src/test/hive/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc rename to shark/src/test/hive/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc diff --git a/src/test/hive/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 b/shark/src/test/hive/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 similarity index 100% rename from src/test/hive/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 rename to shark/src/test/hive/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 diff --git a/src/test/hive/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/shark/src/test/hive/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 similarity index 100% rename from src/test/hive/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 rename to shark/src/test/hive/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 diff --git a/src/test/hive/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 b/shark/src/test/hive/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 similarity index 100% rename from src/test/hive/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 rename to shark/src/test/hive/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 diff --git a/src/test/hive/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af b/shark/src/test/hive/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af similarity index 100% rename from src/test/hive/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af rename to shark/src/test/hive/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af diff --git a/src/test/hive/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc b/shark/src/test/hive/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc similarity index 100% rename from src/test/hive/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc rename to shark/src/test/hive/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc diff --git a/src/test/hive/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb b/shark/src/test/hive/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb similarity index 100% rename from src/test/hive/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb rename to shark/src/test/hive/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb diff --git a/src/test/hive/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/shark/src/test/hive/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 similarity index 100% rename from src/test/hive/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 rename to shark/src/test/hive/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 diff --git a/src/test/hive/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c b/shark/src/test/hive/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c similarity index 100% rename from src/test/hive/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c rename to shark/src/test/hive/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c diff --git a/src/test/hive/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 b/shark/src/test/hive/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 similarity index 100% rename from src/test/hive/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 rename to shark/src/test/hive/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 diff --git a/src/test/hive/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 b/shark/src/test/hive/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 similarity index 100% rename from src/test/hive/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 rename to shark/src/test/hive/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 diff --git a/src/test/hive/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc b/shark/src/test/hive/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc similarity index 100% rename from src/test/hive/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc rename to shark/src/test/hive/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc diff --git a/src/test/hive/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d b/shark/src/test/hive/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d similarity index 100% rename from src/test/hive/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d rename to shark/src/test/hive/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d diff --git a/src/test/hive/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/shark/src/test/hive/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 similarity index 100% rename from src/test/hive/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 rename to shark/src/test/hive/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 diff --git a/src/test/hive/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 b/shark/src/test/hive/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 similarity index 100% rename from src/test/hive/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 rename to shark/src/test/hive/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 diff --git a/src/test/hive/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 b/shark/src/test/hive/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 similarity index 100% rename from src/test/hive/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 rename to shark/src/test/hive/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 diff --git a/src/test/hive/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 b/shark/src/test/hive/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 similarity index 100% rename from src/test/hive/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 rename to shark/src/test/hive/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 diff --git a/src/test/hive/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d b/shark/src/test/hive/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d similarity index 100% rename from src/test/hive/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d rename to shark/src/test/hive/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d diff --git a/src/test/hive/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef b/shark/src/test/hive/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef similarity index 100% rename from src/test/hive/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef rename to shark/src/test/hive/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef diff --git a/src/test/hive/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef b/shark/src/test/hive/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef similarity index 100% rename from src/test/hive/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef rename to shark/src/test/hive/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef diff --git a/src/test/hive/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 b/shark/src/test/hive/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 similarity index 100% rename from src/test/hive/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 rename to shark/src/test/hive/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 diff --git a/src/test/hive/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 b/shark/src/test/hive/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 similarity index 100% rename from src/test/hive/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 rename to shark/src/test/hive/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 diff --git a/src/test/hive/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 b/shark/src/test/hive/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 similarity index 100% rename from src/test/hive/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 rename to shark/src/test/hive/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 diff --git a/src/test/hive/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc b/shark/src/test/hive/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc similarity index 100% rename from src/test/hive/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc rename to shark/src/test/hive/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc diff --git a/src/test/hive/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a b/shark/src/test/hive/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a similarity index 100% rename from src/test/hive/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a rename to shark/src/test/hive/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a diff --git a/src/test/hive/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 b/shark/src/test/hive/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 similarity index 100% rename from src/test/hive/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 rename to shark/src/test/hive/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 diff --git a/src/test/hive/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d b/shark/src/test/hive/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d similarity index 100% rename from src/test/hive/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d rename to shark/src/test/hive/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d diff --git a/src/test/hive/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 b/shark/src/test/hive/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 similarity index 100% rename from src/test/hive/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 rename to shark/src/test/hive/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 diff --git a/src/test/hive/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 b/shark/src/test/hive/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 similarity index 100% rename from src/test/hive/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 rename to shark/src/test/hive/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 diff --git a/src/test/hive/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a b/shark/src/test/hive/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a similarity index 100% rename from src/test/hive/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a rename to shark/src/test/hive/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a diff --git a/src/test/hive/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b b/shark/src/test/hive/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b similarity index 100% rename from src/test/hive/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b rename to shark/src/test/hive/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b diff --git a/src/test/hive/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef b/shark/src/test/hive/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef similarity index 100% rename from src/test/hive/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef rename to shark/src/test/hive/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef diff --git a/src/test/hive/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad b/shark/src/test/hive/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad similarity index 100% rename from src/test/hive/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad rename to shark/src/test/hive/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad diff --git a/src/test/hive/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 b/shark/src/test/hive/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 similarity index 100% rename from src/test/hive/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 rename to shark/src/test/hive/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 diff --git a/src/test/hive/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 b/shark/src/test/hive/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 similarity index 100% rename from src/test/hive/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 rename to shark/src/test/hive/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 diff --git a/src/test/hive/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe b/shark/src/test/hive/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe similarity index 100% rename from src/test/hive/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe rename to shark/src/test/hive/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe diff --git a/src/test/hive/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 b/shark/src/test/hive/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from src/test/hive/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 rename to shark/src/test/hive/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 diff --git a/src/test/hive/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 b/shark/src/test/hive/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 similarity index 100% rename from src/test/hive/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 rename to shark/src/test/hive/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 diff --git a/src/test/hive/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 b/shark/src/test/hive/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 similarity index 100% rename from src/test/hive/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 rename to shark/src/test/hive/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 diff --git a/src/test/hive/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b b/shark/src/test/hive/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b similarity index 100% rename from src/test/hive/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b rename to shark/src/test/hive/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b diff --git a/src/test/hive/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 b/shark/src/test/hive/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 similarity index 100% rename from src/test/hive/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 rename to shark/src/test/hive/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 diff --git a/src/test/hive/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 b/shark/src/test/hive/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 similarity index 100% rename from src/test/hive/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 rename to shark/src/test/hive/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 diff --git a/src/test/hive/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 b/shark/src/test/hive/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 similarity index 100% rename from src/test/hive/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 rename to shark/src/test/hive/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 diff --git a/src/test/hive/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 b/shark/src/test/hive/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 similarity index 100% rename from src/test/hive/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 rename to shark/src/test/hive/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 diff --git a/src/test/hive/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e b/shark/src/test/hive/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e similarity index 100% rename from src/test/hive/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e rename to shark/src/test/hive/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e diff --git a/src/test/hive/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 b/shark/src/test/hive/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 similarity index 100% rename from src/test/hive/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 rename to shark/src/test/hive/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 diff --git a/src/test/hive/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 b/shark/src/test/hive/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 similarity index 100% rename from src/test/hive/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 rename to shark/src/test/hive/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 diff --git a/src/test/hive/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a b/shark/src/test/hive/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a similarity index 100% rename from src/test/hive/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a rename to shark/src/test/hive/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a diff --git a/src/test/hive/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 b/shark/src/test/hive/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 similarity index 100% rename from src/test/hive/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 rename to shark/src/test/hive/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 diff --git a/src/test/hive/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 b/shark/src/test/hive/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 similarity index 100% rename from src/test/hive/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 rename to shark/src/test/hive/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 diff --git a/src/test/hive/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e b/shark/src/test/hive/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e similarity index 100% rename from src/test/hive/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e rename to shark/src/test/hive/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e diff --git a/src/test/hive/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a b/shark/src/test/hive/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a similarity index 100% rename from src/test/hive/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a rename to shark/src/test/hive/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a diff --git a/src/test/hive/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f b/shark/src/test/hive/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f similarity index 100% rename from src/test/hive/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f rename to shark/src/test/hive/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f diff --git a/src/test/hive/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 b/shark/src/test/hive/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 similarity index 100% rename from src/test/hive/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 rename to shark/src/test/hive/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 diff --git a/src/test/hive/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e b/shark/src/test/hive/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e similarity index 100% rename from src/test/hive/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e rename to shark/src/test/hive/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e diff --git a/src/test/hive/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f b/shark/src/test/hive/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f similarity index 100% rename from src/test/hive/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f rename to shark/src/test/hive/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f diff --git a/src/test/hive/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa b/shark/src/test/hive/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa similarity index 100% rename from src/test/hive/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa rename to shark/src/test/hive/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa diff --git a/src/test/hive/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 b/shark/src/test/hive/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 similarity index 100% rename from src/test/hive/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 rename to shark/src/test/hive/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 diff --git a/src/test/hive/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f b/shark/src/test/hive/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f similarity index 100% rename from src/test/hive/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f rename to shark/src/test/hive/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f diff --git a/src/test/hive/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 b/shark/src/test/hive/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 similarity index 100% rename from src/test/hive/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 rename to shark/src/test/hive/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 diff --git a/src/test/hive/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 b/shark/src/test/hive/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 similarity index 100% rename from src/test/hive/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 rename to shark/src/test/hive/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 diff --git a/src/test/hive/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 b/shark/src/test/hive/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 similarity index 100% rename from src/test/hive/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 rename to shark/src/test/hive/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 diff --git a/src/test/hive/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 b/shark/src/test/hive/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 similarity index 100% rename from src/test/hive/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 rename to shark/src/test/hive/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 diff --git a/src/test/hive/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd b/shark/src/test/hive/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd similarity index 100% rename from src/test/hive/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd rename to shark/src/test/hive/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd diff --git a/src/test/hive/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af b/shark/src/test/hive/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af similarity index 100% rename from src/test/hive/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af rename to shark/src/test/hive/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af diff --git a/src/test/hive/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 b/shark/src/test/hive/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 similarity index 100% rename from src/test/hive/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 rename to shark/src/test/hive/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 diff --git a/src/test/hive/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 b/shark/src/test/hive/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 similarity index 100% rename from src/test/hive/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 rename to shark/src/test/hive/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 diff --git a/src/test/hive/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb b/shark/src/test/hive/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb similarity index 100% rename from src/test/hive/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb rename to shark/src/test/hive/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb diff --git a/src/test/hive/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 b/shark/src/test/hive/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 similarity index 100% rename from src/test/hive/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 rename to shark/src/test/hive/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 diff --git a/src/test/hive/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 b/shark/src/test/hive/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 similarity index 100% rename from src/test/hive/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 rename to shark/src/test/hive/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 diff --git a/src/test/hive/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a b/shark/src/test/hive/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a similarity index 100% rename from src/test/hive/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a rename to shark/src/test/hive/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a diff --git a/src/test/hive/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 b/shark/src/test/hive/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 similarity index 100% rename from src/test/hive/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 rename to shark/src/test/hive/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 diff --git a/src/test/hive/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 b/shark/src/test/hive/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 similarity index 100% rename from src/test/hive/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 rename to shark/src/test/hive/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 diff --git a/src/test/hive/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 b/shark/src/test/hive/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 similarity index 100% rename from src/test/hive/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 rename to shark/src/test/hive/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 diff --git a/src/test/hive/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e b/shark/src/test/hive/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e similarity index 100% rename from src/test/hive/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e rename to shark/src/test/hive/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e diff --git a/src/test/hive/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f b/shark/src/test/hive/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f similarity index 100% rename from src/test/hive/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f rename to shark/src/test/hive/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f diff --git a/src/test/hive/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 b/shark/src/test/hive/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 similarity index 100% rename from src/test/hive/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 rename to shark/src/test/hive/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 diff --git a/src/test/hive/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e b/shark/src/test/hive/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e similarity index 100% rename from src/test/hive/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e rename to shark/src/test/hive/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e diff --git a/src/test/hive/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 b/shark/src/test/hive/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 similarity index 100% rename from src/test/hive/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 rename to shark/src/test/hive/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 diff --git a/src/test/hive/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a b/shark/src/test/hive/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a similarity index 100% rename from src/test/hive/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a rename to shark/src/test/hive/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a diff --git a/src/test/hive/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 b/shark/src/test/hive/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 similarity index 100% rename from src/test/hive/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 rename to shark/src/test/hive/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 diff --git a/src/test/hive/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 b/shark/src/test/hive/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 similarity index 100% rename from src/test/hive/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 rename to shark/src/test/hive/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 diff --git a/src/test/hive/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 b/shark/src/test/hive/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 similarity index 100% rename from src/test/hive/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 rename to shark/src/test/hive/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 diff --git a/src/test/hive/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc b/shark/src/test/hive/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc similarity index 100% rename from src/test/hive/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc rename to shark/src/test/hive/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc diff --git a/src/test/hive/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead b/shark/src/test/hive/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead similarity index 100% rename from src/test/hive/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead rename to shark/src/test/hive/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead diff --git a/src/test/hive/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a b/shark/src/test/hive/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a similarity index 100% rename from src/test/hive/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a rename to shark/src/test/hive/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a diff --git a/src/test/hive/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f b/shark/src/test/hive/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f similarity index 100% rename from src/test/hive/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f rename to shark/src/test/hive/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f diff --git a/src/test/hive/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 b/shark/src/test/hive/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 similarity index 100% rename from src/test/hive/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 rename to shark/src/test/hive/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 diff --git a/src/test/hive/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc b/shark/src/test/hive/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc similarity index 100% rename from src/test/hive/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc rename to shark/src/test/hive/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc diff --git a/src/test/hive/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 b/shark/src/test/hive/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 similarity index 100% rename from src/test/hive/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 rename to shark/src/test/hive/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 diff --git a/src/test/hive/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a b/shark/src/test/hive/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a similarity index 100% rename from src/test/hive/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a rename to shark/src/test/hive/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a diff --git a/src/test/hive/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 b/shark/src/test/hive/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 similarity index 100% rename from src/test/hive/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 rename to shark/src/test/hive/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 diff --git a/src/test/hive/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 b/shark/src/test/hive/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 similarity index 100% rename from src/test/hive/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 rename to shark/src/test/hive/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 diff --git a/src/test/hive/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e b/shark/src/test/hive/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e similarity index 100% rename from src/test/hive/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e rename to shark/src/test/hive/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e diff --git a/src/test/hive/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 b/shark/src/test/hive/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 similarity index 100% rename from src/test/hive/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 rename to shark/src/test/hive/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 diff --git a/src/test/hive/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 b/shark/src/test/hive/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 similarity index 100% rename from src/test/hive/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 rename to shark/src/test/hive/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 diff --git a/src/test/hive/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a b/shark/src/test/hive/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a similarity index 100% rename from src/test/hive/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a rename to shark/src/test/hive/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a diff --git a/src/test/hive/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf b/shark/src/test/hive/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf similarity index 100% rename from src/test/hive/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf rename to shark/src/test/hive/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf diff --git a/src/test/hive/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/shark/src/test/hive/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 similarity index 100% rename from src/test/hive/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 rename to shark/src/test/hive/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 diff --git a/src/test/hive/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca b/shark/src/test/hive/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca similarity index 100% rename from src/test/hive/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca rename to shark/src/test/hive/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca diff --git a/src/test/hive/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/shark/src/test/hive/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 similarity index 100% rename from src/test/hive/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 rename to shark/src/test/hive/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 diff --git a/src/test/hive/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 b/shark/src/test/hive/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 similarity index 100% rename from src/test/hive/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 rename to shark/src/test/hive/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 diff --git a/src/test/hive/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 b/shark/src/test/hive/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 similarity index 100% rename from src/test/hive/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 rename to shark/src/test/hive/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 diff --git a/src/test/hive/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 b/shark/src/test/hive/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 similarity index 100% rename from src/test/hive/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 rename to shark/src/test/hive/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 diff --git a/src/test/hive/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab b/shark/src/test/hive/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab similarity index 100% rename from src/test/hive/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab rename to shark/src/test/hive/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab diff --git a/src/test/hive/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee b/shark/src/test/hive/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee similarity index 100% rename from src/test/hive/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee rename to shark/src/test/hive/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee diff --git a/src/test/hive/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/shark/src/test/hive/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 similarity index 100% rename from src/test/hive/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 rename to shark/src/test/hive/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 diff --git a/src/test/hive/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f b/shark/src/test/hive/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f similarity index 100% rename from src/test/hive/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f rename to shark/src/test/hive/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f diff --git a/src/test/hive/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 b/shark/src/test/hive/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from src/test/hive/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 rename to shark/src/test/hive/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/src/test/hive/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c b/shark/src/test/hive/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c similarity index 100% rename from src/test/hive/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c rename to shark/src/test/hive/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c diff --git a/src/test/hive/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 b/shark/src/test/hive/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 similarity index 100% rename from src/test/hive/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 rename to shark/src/test/hive/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 diff --git a/src/test/hive/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c b/shark/src/test/hive/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c similarity index 100% rename from src/test/hive/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c rename to shark/src/test/hive/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c diff --git a/src/test/hive/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 b/shark/src/test/hive/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 similarity index 100% rename from src/test/hive/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 rename to shark/src/test/hive/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 diff --git a/src/test/hive/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a b/shark/src/test/hive/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a similarity index 100% rename from src/test/hive/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a rename to shark/src/test/hive/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a diff --git a/src/test/hive/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 b/shark/src/test/hive/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 similarity index 100% rename from src/test/hive/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 rename to shark/src/test/hive/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 diff --git a/src/test/hive/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a b/shark/src/test/hive/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a similarity index 100% rename from src/test/hive/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a rename to shark/src/test/hive/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a diff --git a/src/test/hive/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 b/shark/src/test/hive/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 similarity index 100% rename from src/test/hive/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 rename to shark/src/test/hive/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 diff --git a/src/test/hive/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc b/shark/src/test/hive/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc similarity index 100% rename from src/test/hive/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc rename to shark/src/test/hive/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc diff --git a/src/test/hive/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 b/shark/src/test/hive/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 similarity index 100% rename from src/test/hive/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 rename to shark/src/test/hive/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 diff --git a/src/test/hive/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 b/shark/src/test/hive/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 similarity index 100% rename from src/test/hive/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 rename to shark/src/test/hive/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 diff --git a/src/test/hive/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/shark/src/test/hive/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 similarity index 100% rename from src/test/hive/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 rename to shark/src/test/hive/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 diff --git a/src/test/hive/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e b/shark/src/test/hive/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e similarity index 100% rename from src/test/hive/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e rename to shark/src/test/hive/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e diff --git a/src/test/hive/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 b/shark/src/test/hive/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 similarity index 100% rename from src/test/hive/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 rename to shark/src/test/hive/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 diff --git a/src/test/hive/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 b/shark/src/test/hive/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 similarity index 100% rename from src/test/hive/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 rename to shark/src/test/hive/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 diff --git a/src/test/hive/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d b/shark/src/test/hive/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d similarity index 100% rename from src/test/hive/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d rename to shark/src/test/hive/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d diff --git a/src/test/hive/golden/union_view-1-c790d4344144460224b0f02be7e137a8 b/shark/src/test/hive/golden/union_view-1-c790d4344144460224b0f02be7e137a8 similarity index 100% rename from src/test/hive/golden/union_view-1-c790d4344144460224b0f02be7e137a8 rename to shark/src/test/hive/golden/union_view-1-c790d4344144460224b0f02be7e137a8 diff --git a/src/test/hive/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 b/shark/src/test/hive/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 similarity index 100% rename from src/test/hive/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 rename to shark/src/test/hive/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 diff --git a/src/test/hive/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 b/shark/src/test/hive/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 similarity index 100% rename from src/test/hive/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 rename to shark/src/test/hive/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 diff --git a/src/test/hive/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e b/shark/src/test/hive/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e similarity index 100% rename from src/test/hive/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e rename to shark/src/test/hive/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e diff --git a/src/test/hive/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 b/shark/src/test/hive/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 similarity index 100% rename from src/test/hive/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 rename to shark/src/test/hive/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 diff --git a/src/test/hive/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae b/shark/src/test/hive/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae similarity index 100% rename from src/test/hive/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae rename to shark/src/test/hive/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae diff --git a/src/test/hive/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 b/shark/src/test/hive/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 similarity index 100% rename from src/test/hive/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 rename to shark/src/test/hive/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 diff --git a/src/test/hive/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 b/shark/src/test/hive/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 similarity index 100% rename from src/test/hive/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 rename to shark/src/test/hive/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 diff --git a/src/test/hive/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af b/shark/src/test/hive/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af similarity index 100% rename from src/test/hive/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af rename to shark/src/test/hive/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af diff --git a/src/test/hive/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 b/shark/src/test/hive/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 similarity index 100% rename from src/test/hive/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 rename to shark/src/test/hive/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 diff --git a/src/test/hive/golden/varchar_2-2-779a555944479991520be53f7f247626 b/shark/src/test/hive/golden/varchar_2-2-779a555944479991520be53f7f247626 similarity index 100% rename from src/test/hive/golden/varchar_2-2-779a555944479991520be53f7f247626 rename to shark/src/test/hive/golden/varchar_2-2-779a555944479991520be53f7f247626 diff --git a/src/test/hive/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da b/shark/src/test/hive/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da similarity index 100% rename from src/test/hive/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da rename to shark/src/test/hive/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da diff --git a/src/test/hive/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f b/shark/src/test/hive/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f similarity index 100% rename from src/test/hive/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f rename to shark/src/test/hive/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f diff --git a/src/test/hive/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 b/shark/src/test/hive/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 similarity index 100% rename from src/test/hive/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 rename to shark/src/test/hive/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 diff --git a/src/test/hive/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f b/shark/src/test/hive/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f similarity index 100% rename from src/test/hive/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f rename to shark/src/test/hive/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f diff --git a/src/test/hive/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af b/shark/src/test/hive/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af similarity index 100% rename from src/test/hive/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af rename to shark/src/test/hive/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af diff --git a/src/test/hive/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e b/shark/src/test/hive/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e similarity index 100% rename from src/test/hive/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e rename to shark/src/test/hive/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e diff --git a/src/test/hive/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 b/shark/src/test/hive/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 similarity index 100% rename from src/test/hive/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 rename to shark/src/test/hive/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 diff --git a/src/test/hive/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce b/shark/src/test/hive/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce similarity index 100% rename from src/test/hive/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce rename to shark/src/test/hive/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce diff --git a/src/test/hive/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c b/shark/src/test/hive/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c similarity index 100% rename from src/test/hive/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c rename to shark/src/test/hive/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c diff --git a/src/test/hive/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e b/shark/src/test/hive/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e similarity index 100% rename from src/test/hive/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e rename to shark/src/test/hive/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e diff --git a/src/test/hive/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 b/shark/src/test/hive/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 similarity index 100% rename from src/test/hive/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 rename to shark/src/test/hive/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 diff --git a/src/test/hive/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 b/shark/src/test/hive/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 similarity index 100% rename from src/test/hive/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 rename to shark/src/test/hive/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 diff --git a/src/test/hive/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 b/shark/src/test/hive/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 similarity index 100% rename from src/test/hive/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 rename to shark/src/test/hive/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 diff --git a/src/test/hive/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 b/shark/src/test/hive/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 similarity index 100% rename from src/test/hive/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 rename to shark/src/test/hive/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 diff --git a/src/test/hive/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 b/shark/src/test/hive/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 similarity index 100% rename from src/test/hive/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 rename to shark/src/test/hive/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 diff --git a/src/test/hive/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 b/shark/src/test/hive/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 similarity index 100% rename from src/test/hive/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 rename to shark/src/test/hive/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 diff --git a/src/test/hive/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef b/shark/src/test/hive/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef similarity index 100% rename from src/test/hive/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef rename to shark/src/test/hive/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef diff --git a/src/test/hive/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 b/shark/src/test/hive/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 similarity index 100% rename from src/test/hive/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 rename to shark/src/test/hive/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 diff --git a/src/test/hive/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 b/shark/src/test/hive/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 similarity index 100% rename from src/test/hive/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 rename to shark/src/test/hive/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 diff --git a/src/test/hive/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 b/shark/src/test/hive/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 similarity index 100% rename from src/test/hive/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 rename to shark/src/test/hive/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 diff --git a/src/test/hive/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc b/shark/src/test/hive/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc similarity index 100% rename from src/test/hive/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc rename to shark/src/test/hive/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc diff --git a/src/test/hive/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 b/shark/src/test/hive/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 rename to shark/src/test/hive/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 diff --git a/src/test/hive/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 b/shark/src/test/hive/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 rename to shark/src/test/hive/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 diff --git a/src/test/hive/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 b/shark/src/test/hive/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 rename to shark/src/test/hive/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 diff --git a/src/test/hive/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f b/shark/src/test/hive/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f similarity index 100% rename from src/test/hive/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f rename to shark/src/test/hive/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f diff --git a/src/test/hive/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 b/shark/src/test/hive/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 rename to shark/src/test/hive/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 diff --git a/src/test/hive/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 b/shark/src/test/hive/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 rename to shark/src/test/hive/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 diff --git a/src/test/hive/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 b/shark/src/test/hive/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 rename to shark/src/test/hive/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 diff --git a/src/test/hive/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 b/shark/src/test/hive/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 rename to shark/src/test/hive/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 diff --git a/src/test/hive/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc b/shark/src/test/hive/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc similarity index 100% rename from src/test/hive/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc rename to shark/src/test/hive/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc diff --git a/src/test/hive/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 b/shark/src/test/hive/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 rename to shark/src/test/hive/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 diff --git a/src/test/hive/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 b/shark/src/test/hive/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 rename to shark/src/test/hive/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 diff --git a/src/test/hive/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb b/shark/src/test/hive/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb similarity index 100% rename from src/test/hive/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb rename to shark/src/test/hive/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb diff --git a/src/test/hive/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 b/shark/src/test/hive/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 rename to shark/src/test/hive/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 diff --git a/src/test/hive/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad b/shark/src/test/hive/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad similarity index 100% rename from src/test/hive/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad rename to shark/src/test/hive/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad diff --git a/src/test/hive/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 b/shark/src/test/hive/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 rename to shark/src/test/hive/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 diff --git a/src/test/hive/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 b/shark/src/test/hive/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 rename to shark/src/test/hive/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 diff --git a/src/test/hive/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 b/shark/src/test/hive/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 rename to shark/src/test/hive/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 diff --git a/src/test/hive/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb b/shark/src/test/hive/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb similarity index 100% rename from src/test/hive/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb rename to shark/src/test/hive/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb diff --git a/src/test/hive/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc b/shark/src/test/hive/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc similarity index 100% rename from src/test/hive/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc rename to shark/src/test/hive/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc diff --git a/src/test/hive/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 b/shark/src/test/hive/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 rename to shark/src/test/hive/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 diff --git a/src/test/hive/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb b/shark/src/test/hive/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb similarity index 100% rename from src/test/hive/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb rename to shark/src/test/hive/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb diff --git a/src/test/hive/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 b/shark/src/test/hive/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 rename to shark/src/test/hive/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 diff --git a/src/test/hive/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 b/shark/src/test/hive/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 rename to shark/src/test/hive/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 diff --git a/src/test/hive/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 b/shark/src/test/hive/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 rename to shark/src/test/hive/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 diff --git a/src/test/hive/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 b/shark/src/test/hive/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 rename to shark/src/test/hive/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 diff --git a/src/test/hive/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 b/shark/src/test/hive/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 rename to shark/src/test/hive/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 diff --git a/src/test/hive/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 b/shark/src/test/hive/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 rename to shark/src/test/hive/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 diff --git a/src/test/hive/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b b/shark/src/test/hive/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b similarity index 100% rename from src/test/hive/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b rename to shark/src/test/hive/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b diff --git a/src/test/hive/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 b/shark/src/test/hive/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 rename to shark/src/test/hive/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 diff --git a/src/test/hive/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 b/shark/src/test/hive/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 rename to shark/src/test/hive/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 diff --git a/src/test/hive/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 b/shark/src/test/hive/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 similarity index 100% rename from src/test/hive/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 rename to shark/src/test/hive/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 diff --git a/src/test/hive/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 b/shark/src/test/hive/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 similarity index 100% rename from src/test/hive/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 rename to shark/src/test/hive/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 diff --git a/src/test/hive/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d b/shark/src/test/hive/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d similarity index 100% rename from src/test/hive/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d rename to shark/src/test/hive/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d diff --git a/src/test/hive/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 b/shark/src/test/hive/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 similarity index 100% rename from src/test/hive/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 rename to shark/src/test/hive/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 diff --git a/src/test/hive/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e b/shark/src/test/hive/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e similarity index 100% rename from src/test/hive/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e rename to shark/src/test/hive/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e diff --git a/src/test/hive/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 b/shark/src/test/hive/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 similarity index 100% rename from src/test/hive/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 rename to shark/src/test/hive/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 diff --git a/src/test/hive/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e b/shark/src/test/hive/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e similarity index 100% rename from src/test/hive/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e rename to shark/src/test/hive/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e diff --git a/src/test/hive/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 b/shark/src/test/hive/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 similarity index 100% rename from src/test/hive/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 rename to shark/src/test/hive/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 diff --git a/src/test/hive/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 b/shark/src/test/hive/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 similarity index 100% rename from src/test/hive/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 rename to shark/src/test/hive/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 diff --git a/src/test/hive/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 b/shark/src/test/hive/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 similarity index 100% rename from src/test/hive/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 rename to shark/src/test/hive/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 diff --git a/src/test/hive/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a b/shark/src/test/hive/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a similarity index 100% rename from src/test/hive/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a rename to shark/src/test/hive/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a diff --git a/src/test/hive/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d b/shark/src/test/hive/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d similarity index 100% rename from src/test/hive/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d rename to shark/src/test/hive/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d diff --git a/src/test/hive/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad b/shark/src/test/hive/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad similarity index 100% rename from src/test/hive/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad rename to shark/src/test/hive/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad diff --git a/src/test/hive/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e b/shark/src/test/hive/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e similarity index 100% rename from src/test/hive/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e rename to shark/src/test/hive/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e diff --git a/src/test/hive/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 b/shark/src/test/hive/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 similarity index 100% rename from src/test/hive/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 rename to shark/src/test/hive/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 diff --git a/src/test/hive/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b b/shark/src/test/hive/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b similarity index 100% rename from src/test/hive/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b rename to shark/src/test/hive/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b diff --git a/src/test/hive/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 b/shark/src/test/hive/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 similarity index 100% rename from src/test/hive/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 rename to shark/src/test/hive/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 diff --git a/src/test/hive/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 b/shark/src/test/hive/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 similarity index 100% rename from src/test/hive/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 rename to shark/src/test/hive/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 diff --git a/src/test/hive/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 b/shark/src/test/hive/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 similarity index 100% rename from src/test/hive/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 rename to shark/src/test/hive/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 diff --git a/src/test/hive/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc b/shark/src/test/hive/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc similarity index 100% rename from src/test/hive/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc rename to shark/src/test/hive/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc diff --git a/src/test/hive/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f b/shark/src/test/hive/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f similarity index 100% rename from src/test/hive/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f rename to shark/src/test/hive/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f diff --git a/src/test/hive/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 b/shark/src/test/hive/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 similarity index 100% rename from src/test/hive/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 rename to shark/src/test/hive/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 diff --git a/src/test/hive/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e b/shark/src/test/hive/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e similarity index 100% rename from src/test/hive/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e rename to shark/src/test/hive/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e diff --git a/src/test/hive/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c b/shark/src/test/hive/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c similarity index 100% rename from src/test/hive/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c rename to shark/src/test/hive/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c diff --git a/src/test/hive/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 b/shark/src/test/hive/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 similarity index 100% rename from src/test/hive/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 rename to shark/src/test/hive/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 diff --git a/src/test/hive/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a b/shark/src/test/hive/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a similarity index 100% rename from src/test/hive/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a rename to shark/src/test/hive/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a diff --git a/src/test/hive/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 b/shark/src/test/hive/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 similarity index 100% rename from src/test/hive/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 rename to shark/src/test/hive/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 diff --git a/src/test/hive/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb b/shark/src/test/hive/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb similarity index 100% rename from src/test/hive/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb rename to shark/src/test/hive/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb diff --git a/src/test/hive/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 b/shark/src/test/hive/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 similarity index 100% rename from src/test/hive/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 rename to shark/src/test/hive/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 diff --git a/src/test/hive/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 b/shark/src/test/hive/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 similarity index 100% rename from src/test/hive/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 rename to shark/src/test/hive/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 diff --git a/src/test/hive/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 b/shark/src/test/hive/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 similarity index 100% rename from src/test/hive/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 rename to shark/src/test/hive/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 diff --git a/src/test/hive/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d b/shark/src/test/hive/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d similarity index 100% rename from src/test/hive/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d rename to shark/src/test/hive/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d diff --git a/src/test/hive/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e b/shark/src/test/hive/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e similarity index 100% rename from src/test/hive/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e rename to shark/src/test/hive/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e diff --git a/src/test/hive/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 b/shark/src/test/hive/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 similarity index 100% rename from src/test/hive/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 rename to shark/src/test/hive/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 diff --git a/src/test/hive/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f b/shark/src/test/hive/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f similarity index 100% rename from src/test/hive/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f rename to shark/src/test/hive/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f diff --git a/src/test/hive/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f b/shark/src/test/hive/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f similarity index 100% rename from src/test/hive/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f rename to shark/src/test/hive/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f diff --git a/src/test/hive/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba b/shark/src/test/hive/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba similarity index 100% rename from src/test/hive/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba rename to shark/src/test/hive/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba diff --git a/src/test/hive/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba b/shark/src/test/hive/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba similarity index 100% rename from src/test/hive/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba rename to shark/src/test/hive/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba diff --git a/src/test/hive/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 b/shark/src/test/hive/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 similarity index 100% rename from src/test/hive/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 rename to shark/src/test/hive/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 diff --git a/src/test/hive/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 b/shark/src/test/hive/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 similarity index 100% rename from src/test/hive/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 rename to shark/src/test/hive/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 diff --git a/src/test/hive/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 b/shark/src/test/hive/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 similarity index 100% rename from src/test/hive/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 rename to shark/src/test/hive/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 diff --git a/src/test/hive/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 b/shark/src/test/hive/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 similarity index 100% rename from src/test/hive/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 rename to shark/src/test/hive/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 diff --git a/src/test/hive/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e b/shark/src/test/hive/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e similarity index 100% rename from src/test/hive/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e rename to shark/src/test/hive/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e diff --git a/src/test/hive/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec b/shark/src/test/hive/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec similarity index 100% rename from src/test/hive/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec rename to shark/src/test/hive/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec diff --git a/src/test/hive/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 b/shark/src/test/hive/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 similarity index 100% rename from src/test/hive/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 rename to shark/src/test/hive/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 diff --git a/src/test/hive/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 b/shark/src/test/hive/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 similarity index 100% rename from src/test/hive/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 rename to shark/src/test/hive/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 diff --git a/src/test/hive/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e b/shark/src/test/hive/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e similarity index 100% rename from src/test/hive/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e rename to shark/src/test/hive/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e diff --git a/src/test/hive/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec b/shark/src/test/hive/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec similarity index 100% rename from src/test/hive/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec rename to shark/src/test/hive/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec diff --git a/src/test/hive/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 b/shark/src/test/hive/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 similarity index 100% rename from src/test/hive/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 rename to shark/src/test/hive/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 diff --git a/src/test/hive/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 b/shark/src/test/hive/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 similarity index 100% rename from src/test/hive/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 rename to shark/src/test/hive/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 diff --git a/src/test/hive/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca b/shark/src/test/hive/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca similarity index 100% rename from src/test/hive/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca rename to shark/src/test/hive/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca diff --git a/src/test/hive/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b b/shark/src/test/hive/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b similarity index 100% rename from src/test/hive/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b rename to shark/src/test/hive/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b diff --git a/src/test/hive/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 b/shark/src/test/hive/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 similarity index 100% rename from src/test/hive/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 rename to shark/src/test/hive/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 diff --git a/src/test/hive/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 b/shark/src/test/hive/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 similarity index 100% rename from src/test/hive/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 rename to shark/src/test/hive/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 diff --git a/src/test/hive/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 b/shark/src/test/hive/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 similarity index 100% rename from src/test/hive/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 rename to shark/src/test/hive/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 diff --git a/src/test/hive/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 b/shark/src/test/hive/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 similarity index 100% rename from src/test/hive/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 rename to shark/src/test/hive/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 diff --git a/src/test/hive/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 b/shark/src/test/hive/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 similarity index 100% rename from src/test/hive/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 rename to shark/src/test/hive/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 diff --git a/src/test/hive/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a b/shark/src/test/hive/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a similarity index 100% rename from src/test/hive/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a rename to shark/src/test/hive/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a diff --git a/src/test/hive/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 b/shark/src/test/hive/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 similarity index 100% rename from src/test/hive/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 rename to shark/src/test/hive/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 diff --git a/src/test/hive/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 b/shark/src/test/hive/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 similarity index 100% rename from src/test/hive/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 rename to shark/src/test/hive/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 diff --git a/src/test/hive/golden/view-11-dc95343d3e57846485dd543476391376 b/shark/src/test/hive/golden/view-11-dc95343d3e57846485dd543476391376 similarity index 100% rename from src/test/hive/golden/view-11-dc95343d3e57846485dd543476391376 rename to shark/src/test/hive/golden/view-11-dc95343d3e57846485dd543476391376 diff --git a/src/test/hive/golden/view-12-371764e1cae31ea0518c03060528d239 b/shark/src/test/hive/golden/view-12-371764e1cae31ea0518c03060528d239 similarity index 100% rename from src/test/hive/golden/view-12-371764e1cae31ea0518c03060528d239 rename to shark/src/test/hive/golden/view-12-371764e1cae31ea0518c03060528d239 diff --git a/src/test/hive/golden/view-13-2abce88008f8a19164758ee821aaa8a6 b/shark/src/test/hive/golden/view-13-2abce88008f8a19164758ee821aaa8a6 similarity index 100% rename from src/test/hive/golden/view-13-2abce88008f8a19164758ee821aaa8a6 rename to shark/src/test/hive/golden/view-13-2abce88008f8a19164758ee821aaa8a6 diff --git a/src/test/hive/golden/view-14-deb504f4f70fd7db975950c3c47959ee b/shark/src/test/hive/golden/view-14-deb504f4f70fd7db975950c3c47959ee similarity index 100% rename from src/test/hive/golden/view-14-deb504f4f70fd7db975950c3c47959ee rename to shark/src/test/hive/golden/view-14-deb504f4f70fd7db975950c3c47959ee diff --git a/src/test/hive/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b b/shark/src/test/hive/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b similarity index 100% rename from src/test/hive/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b rename to shark/src/test/hive/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b diff --git a/src/test/hive/golden/view-2-9c529f486fa81a032bfe1253808fca8 b/shark/src/test/hive/golden/view-2-9c529f486fa81a032bfe1253808fca8 similarity index 100% rename from src/test/hive/golden/view-2-9c529f486fa81a032bfe1253808fca8 rename to shark/src/test/hive/golden/view-2-9c529f486fa81a032bfe1253808fca8 diff --git a/src/test/hive/golden/view-3-89c80c0e90409d5e304775c9f420915a b/shark/src/test/hive/golden/view-3-89c80c0e90409d5e304775c9f420915a similarity index 100% rename from src/test/hive/golden/view-3-89c80c0e90409d5e304775c9f420915a rename to shark/src/test/hive/golden/view-3-89c80c0e90409d5e304775c9f420915a diff --git a/src/test/hive/golden/view-4-4a64d1a623ca71e515796787dbd0f904 b/shark/src/test/hive/golden/view-4-4a64d1a623ca71e515796787dbd0f904 similarity index 100% rename from src/test/hive/golden/view-4-4a64d1a623ca71e515796787dbd0f904 rename to shark/src/test/hive/golden/view-4-4a64d1a623ca71e515796787dbd0f904 diff --git a/src/test/hive/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b b/shark/src/test/hive/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b similarity index 100% rename from src/test/hive/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b rename to shark/src/test/hive/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b diff --git a/src/test/hive/golden/view-6-47b5043f03a84695b6784682b4402ac8 b/shark/src/test/hive/golden/view-6-47b5043f03a84695b6784682b4402ac8 similarity index 100% rename from src/test/hive/golden/view-6-47b5043f03a84695b6784682b4402ac8 rename to shark/src/test/hive/golden/view-6-47b5043f03a84695b6784682b4402ac8 diff --git a/src/test/hive/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 b/shark/src/test/hive/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 similarity index 100% rename from src/test/hive/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 rename to shark/src/test/hive/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 diff --git a/src/test/hive/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 b/shark/src/test/hive/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 similarity index 100% rename from src/test/hive/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 rename to shark/src/test/hive/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 diff --git a/src/test/hive/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 b/shark/src/test/hive/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 similarity index 100% rename from src/test/hive/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 rename to shark/src/test/hive/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 diff --git a/src/test/hive/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf b/shark/src/test/hive/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf similarity index 100% rename from src/test/hive/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf rename to shark/src/test/hive/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf diff --git a/src/test/hive/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb b/shark/src/test/hive/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb similarity index 100% rename from src/test/hive/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb rename to shark/src/test/hive/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb diff --git a/src/test/hive/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 b/shark/src/test/hive/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 similarity index 100% rename from src/test/hive/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 rename to shark/src/test/hive/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 diff --git a/src/test/hive/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de b/shark/src/test/hive/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de similarity index 100% rename from src/test/hive/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de rename to shark/src/test/hive/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de diff --git a/src/test/hive/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 b/shark/src/test/hive/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 similarity index 100% rename from src/test/hive/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 rename to shark/src/test/hive/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 diff --git a/src/test/hive/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 b/shark/src/test/hive/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 similarity index 100% rename from src/test/hive/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 rename to shark/src/test/hive/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 diff --git a/src/test/hive/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a b/shark/src/test/hive/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a similarity index 100% rename from src/test/hive/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a rename to shark/src/test/hive/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a diff --git a/src/test/hive/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 b/shark/src/test/hive/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 similarity index 100% rename from src/test/hive/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 rename to shark/src/test/hive/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 diff --git a/src/test/hive/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb b/shark/src/test/hive/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb similarity index 100% rename from src/test/hive/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb rename to shark/src/test/hive/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb diff --git a/src/test/hive/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 b/shark/src/test/hive/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 similarity index 100% rename from src/test/hive/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 rename to shark/src/test/hive/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 diff --git a/src/test/hive/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 b/shark/src/test/hive/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 similarity index 100% rename from src/test/hive/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 rename to shark/src/test/hive/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 diff --git a/src/test/hive/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 b/shark/src/test/hive/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 similarity index 100% rename from src/test/hive/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 rename to shark/src/test/hive/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 diff --git a/src/test/hive/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe b/shark/src/test/hive/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe similarity index 100% rename from src/test/hive/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe rename to shark/src/test/hive/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe diff --git a/src/test/hive/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d b/shark/src/test/hive/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d similarity index 100% rename from src/test/hive/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d rename to shark/src/test/hive/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d diff --git a/src/test/hive/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 b/shark/src/test/hive/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 similarity index 100% rename from src/test/hive/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 rename to shark/src/test/hive/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 diff --git a/src/test/hive/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 b/shark/src/test/hive/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 similarity index 100% rename from src/test/hive/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 rename to shark/src/test/hive/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 diff --git a/src/test/hive/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 b/shark/src/test/hive/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 similarity index 100% rename from src/test/hive/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 rename to shark/src/test/hive/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 diff --git a/src/test/hive/golden/virtual_column-3-c66776673c986b59b27e704664935988 b/shark/src/test/hive/golden/virtual_column-3-c66776673c986b59b27e704664935988 similarity index 100% rename from src/test/hive/golden/virtual_column-3-c66776673c986b59b27e704664935988 rename to shark/src/test/hive/golden/virtual_column-3-c66776673c986b59b27e704664935988 diff --git a/src/test/hive/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 b/shark/src/test/hive/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 similarity index 100% rename from src/test/hive/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 rename to shark/src/test/hive/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 diff --git a/src/test/hive/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 b/shark/src/test/hive/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 similarity index 100% rename from src/test/hive/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 rename to shark/src/test/hive/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 diff --git a/src/test/hive/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 b/shark/src/test/hive/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 similarity index 100% rename from src/test/hive/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 rename to shark/src/test/hive/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 diff --git a/src/test/hive/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 b/shark/src/test/hive/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 similarity index 100% rename from src/test/hive/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 rename to shark/src/test/hive/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 diff --git a/src/test/hive/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 b/shark/src/test/hive/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 similarity index 100% rename from src/test/hive/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 rename to shark/src/test/hive/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 diff --git a/src/test/hive/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 b/shark/src/test/hive/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 similarity index 100% rename from src/test/hive/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 rename to shark/src/test/hive/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 diff --git a/src/test/hive/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/addpart1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/addpart1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/addpart1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/addpart1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_non_native.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_non_native.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_non_native.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_non_native.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_offline.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_offline.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_partition_offline.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_offline.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_table_add_partition.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_table_add_partition.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_table_add_partition.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_table_add_partition.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure6.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure6.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure6.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure7.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure7.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure7.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure8.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure8.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure8.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure9.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure9.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/alter_view_failure9.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/altern1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/altern1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/altern1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/altern1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col0.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col0.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col0.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col0.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ambiguous_col2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/analyze.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/analyze.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/analyze.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/analyze.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/analyze1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/analyze1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/analyze1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/analyze1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/analyze_view.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/analyze_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/analyze_view.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/analyze_view.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_corrupt.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_corrupt.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_corrupt.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_corrupt.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_insert1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_insert1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_insert1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_insert1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_insert2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_insert2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_insert2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_insert2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_insert3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_insert3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_insert3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_insert3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_insert4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_insert4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_insert4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_insert4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_multi1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_multi2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_multi3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_multi4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_multi5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi6.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_multi6.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi6.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_multi7.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_multi7.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_multi7.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_partspec1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_partspec2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_partspec3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_partspec4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/archive_partspec5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/archive_partspec5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_6.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_6.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_6.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_7.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_7.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_fail_7.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/authorization_part.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/authorization_part.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/authorization_part.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/autolocal1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/autolocal1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/autolocal1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/autolocal1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bad_exec_hooks.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/bad_exec_hooks.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/bad_exec_hooks.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/bad_exec_hooks.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bad_indextype.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/bad_indextype.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/bad_indextype.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/bad_indextype.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bad_sample_clause.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/bad_sample_clause.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/bad_sample_clause.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/bad_sample_clause.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/cachingprintstream.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/cachingprintstream.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/cachingprintstream.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/cachingprintstream.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clusterbydistributeby.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/clusterbydistributeby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/clusterbydistributeby.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/clusterbydistributeby.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clusterbyorderby.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/clusterbyorderby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/clusterbyorderby.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/clusterbyorderby.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clusterbysortby.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/clusterbysortby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/clusterbysortby.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/clusterbysortby.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clustern1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/clustern1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/clustern1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/clustern1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clustern2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/clustern2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/clustern2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/clustern2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clustern3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/clustern3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/clustern3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/clustern3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/clustern4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/clustern4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/clustern4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/clustern4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_rename1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/column_rename1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/column_rename1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/column_rename1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_rename2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/column_rename2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/column_rename2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/column_rename2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_rename3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/column_rename3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/column_rename3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/column_rename3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_rename4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/column_rename4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/column_rename4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/column_rename4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/column_rename5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/column_rename5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/column_rename5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/column_rename5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/compare_double_bigint.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/compare_double_bigint.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/compare_double_bigint.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/compare_double_bigint.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/compare_string_bigint.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/compare_string_bigint.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/compare_string_bigint.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/compare_string_bigint.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_insert_outputformat.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_insert_outputformat.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_insert_outputformat.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_insert_outputformat.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view6.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view6.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view6.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view7.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view7.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view7.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view8.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view8.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_or_replace_view8.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_table_failure1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_table_failure2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_table_failure3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_table_failure4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_table_failure4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_table_wrong_regex.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_table_wrong_regex.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_table_wrong_regex.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_table_wrong_regex.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_udaf_failure.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_udaf_failure.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_udaf_failure.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_udaf_failure.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_genericudf.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_genericudf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_unknown_genericudf.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_genericudf.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_view_failure1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_view_failure2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_view_failure3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_view_failure4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_view_failure5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure6.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_view_failure6.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure6.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure7.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_view_failure7.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure7.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure8.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_view_failure8.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure8.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure9.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/create_view_failure9.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/create_view_failure9.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ctas.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ctas.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ctas.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ctas.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_create_already_exists.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/database_create_already_exists.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/database_create_already_exists.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/database_create_already_exists.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_create_invalid_name.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/database_create_invalid_name.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/database_create_invalid_name.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/database_create_invalid_name.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/date_literal1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/date_literal1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/date_literal1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/date_literal1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/date_literal2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/date_literal2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/date_literal2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/date_literal2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/date_literal3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/date_literal3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/date_literal3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/date_literal3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ddltime.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ddltime.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ddltime.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ddltime.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/decimal_precision.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/decimal_precision_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/decimal_precision_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/default_partition_name.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/default_partition_name.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/default_partition_name.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/default_partition_name.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/deletejar.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/deletejar.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/deletejar.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/deletejar.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/desc_failure1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/desc_failure1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/desc_failure1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/desc_failure1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/desc_failure2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/desc_failure2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/desc_failure2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/desc_failure2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/desc_failure3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/desc_failure3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/desc_failure3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/desc_failure3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/describe_xpath1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/describe_xpath2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/describe_xpath3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/describe_xpath4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/describe_xpath4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_function_failure.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_function_failure.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_function_failure.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_function_failure.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_index_failure.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_index_failure.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_index_failure.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_index_failure.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_native_udf.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_native_udf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_native_udf.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_native_udf.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_failure.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_failure.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_partition_failure.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_failure.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_table_failure3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/drop_view_failure2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/duplicate_insert3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/dyn_part1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/dyn_part2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/dyn_part3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/dyn_part4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled b/shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled rename to shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_22_export_authfail.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_22_export_authfail.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_22_export_authfail.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_22_export_authfail.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/external1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/external1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/external1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/external1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/external2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/external2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/external2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/external2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fetchtask_ioexception.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/fetchtask_ioexception.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/fetchtask_ioexception.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/fetchtask_ioexception.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fileformat_bad_class.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/fileformat_bad_class.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/fileformat_bad_class.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/fileformat_bad_class.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_input.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_input.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_input.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_input.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_output.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_output.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_output.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/fileformat_void_output.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/fs_default_name1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/fs_default_name2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/fs_default_name2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/genericFileFormat.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/genericFileFormat.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/genericFileFormat.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/genericFileFormat.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_cube1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_cube2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_cube2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_id1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_id1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_id1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_id1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_invalid_position.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_invalid_position.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_invalid_position.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_invalid_position.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_key.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_key.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_key.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_key.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/groupby_rollup2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/having1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/having1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/having1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/having1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/illegal_partition_type2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/index_compact_entry_limit.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/index_compact_entry_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/index_compact_entry_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/index_compact_entry_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/index_compact_size_limit.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/index_compact_size_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/index_compact_size_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/index_compact_size_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/input1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/input1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/input1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/input1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/input2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/input2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/input2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/input2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/input4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/input4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/input4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/input4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/input41.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/input41.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/input41.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/input41.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/input_part0_neg.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/input_part0_neg.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/input_part0_neg.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/input_part0_neg.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insert_into1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/insert_into1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/insert_into1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/insert_into1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insert_into2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/insert_into2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/insert_into2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/insert_into2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insert_into3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/insert_into3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/insert_into3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/insert_into3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insert_into4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/insert_into4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/insert_into4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/insert_into4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insert_view_failure.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/insert_view_failure.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/insert_view_failure.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/insert_view_failure.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insertexternal1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/insertexternal1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/insertexternal1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/insertexternal1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_avg_syntax.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_avg_syntax.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_avg_syntax.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_avg_syntax.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_columns.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_columns.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_columns.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_columns.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_config1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_config1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_config1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_config1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_config2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_config2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_config2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_config2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_create_tbl2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_mapjoin1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_mapjoin1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_mapjoin1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_mapjoin1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_max_syntax.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_max_syntax.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_max_syntax.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_max_syntax.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_min_syntax.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_min_syntax.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_min_syntax.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_min_syntax.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_expression.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_expression.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_select_expression.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_select_expression.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_std_syntax.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_std_syntax.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_std_syntax.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_std_syntax.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_sum_syntax.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_sum_syntax.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_sum_syntax.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_sum_syntax.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_alter2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_create2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_create2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_t_create2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_create2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_transform.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_transform.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_t_transform.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_t_transform.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_tbl_name.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_tbl_name.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_tbl_name.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_tbl_name.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalid_variance_syntax.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_variance_syntax.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalid_variance_syntax.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalid_variance_syntax.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/invalidate_view1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/invalidate_view1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/invalidate_view1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/invalidate_view1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/join2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/join2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/join2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join28.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/join28.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/join28.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/join28.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join29.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/join29.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/join29.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/join29.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join32.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/join32.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/join32.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/join32.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join35.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/join35.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/join35.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/join35.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/join_nonexistent_part.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/join_nonexistent_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/join_nonexistent_part.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/join_nonexistent_part.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/joinneg.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/joinneg.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/joinneg.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/joinneg.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_alias.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_alias.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/lateral_view_alias.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_alias.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_join.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_join.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/lateral_view_join.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/lateral_view_join.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/line_terminator.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/line_terminator.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/line_terminator.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/line_terminator.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_exist_part_authfail.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_exist_part_authfail.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_exist_part_authfail.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_exist_part_authfail.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_non_native.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_non_native.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_non_native.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_non_native.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_nonpart_authfail.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_nonpart_authfail.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_nonpart_authfail.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_nonpart_authfail.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_part_authfail.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_part_authfail.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_part_authfail.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_part_authfail.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_part_nospec.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_part_nospec.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_part_nospec.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_part_nospec.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_stored_as_dirs.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_stored_as_dirs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_stored_as_dirs.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_stored_as_dirs.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_view_failure.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_view_failure.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_view_failure.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_view_failure.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_noof_part.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_noof_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/load_wrong_noof_part.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/load_wrong_noof_part.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/local_mapred_error_cache.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/local_mapred_error_cache.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/local_mapred_error_cache.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/local_mapred_error_cache.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lockneg1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/lockneg1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/lockneg1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/lockneg1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lockneg2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/lockneg2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/lockneg2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/lockneg2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lockneg3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/lockneg3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/lockneg3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/lockneg3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lockneg4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/lockneg4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/lockneg4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/lockneg4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/lockneg5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/lockneg5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/lockneg5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/lockneg5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/macro_unused_parameter.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/macro_unused_parameter.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/macro_unused_parameter.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/macro_unused_parameter.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/merge_negative_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/merge_negative_2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/merge_negative_3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/merge_negative_3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/minimr_broken_pipe.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/minimr_broken_pipe.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/minimr_broken_pipe.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/minimr_broken_pipe.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/nested_complex_neg.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/nested_complex_neg.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/nested_complex_neg.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/nested_complex_neg.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/no_matching_udf.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/no_matching_udf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/no_matching_udf.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/no_matching_udf.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/nonkey_groupby.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/nonkey_groupby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/nonkey_groupby.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/nonkey_groupby.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/nopart_insert.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/nopart_insert.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/nopart_insert.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/nopart_insert.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/nopart_load.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/nopart_load.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/nopart_load.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/nopart_load.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/notable_alias3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/notable_alias3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/notable_alias3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/notable_alias3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/notable_alias4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/notable_alias4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/notable_alias4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/notable_alias4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/orderby_invalid_position.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/orderby_invalid_position.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/orderby_invalid_position.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/orderby_invalid_position.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/orderby_position_unsupported.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/orderby_position_unsupported.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/orderby_position_unsupported.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/orderby_position_unsupported.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/orderbysortby.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/orderbysortby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/orderbysortby.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/orderbysortby.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/part_col_complex_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/part_col_complex_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/part_col_complex_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/part_col_complex_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_part.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_part1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_part2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl6.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl6.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl6.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl7.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl7.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl7.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl8.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl8.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl8.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/recursive_view.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/recursive_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/recursive_view.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/recursive_view.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/regex_col_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/regex_col_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/regex_col_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/regex_col_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/regex_col_2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/regex_col_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/regex_col_2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/regex_col_2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/regex_col_groupby.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/regex_col_groupby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/regex_col_groupby.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/regex_col_groupby.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/sa_fail_hook3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/sa_fail_hook3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/sa_fail_hook3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/sa_fail_hook3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/sample.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/sample.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/sample.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/sample.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/script_broken_pipe3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/script_error.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/script_error.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/script_error.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/script_error.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/select_charliteral.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/select_charliteral.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/select_charliteral.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/select_charliteral.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/select_udtf_alias.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/select_udtf_alias.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/select_udtf_alias.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/select_udtf_alias.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/semijoin1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/semijoin1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/semijoin1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/semijoin1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/semijoin2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/semijoin2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/semijoin2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/semijoin2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/semijoin3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/semijoin3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/semijoin3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/semijoin3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/semijoin4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/semijoin4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/semijoin4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/semijoin4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/serde_regex.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/serde_regex.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/serde_regex.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/serde_regex.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/serde_regex2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/serde_regex2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/serde_regex2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/serde_regex2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/serde_regex3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/serde_regex3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/serde_regex3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/serde_regex3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/set_table_property.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/set_table_property.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/set_table_property.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/set_table_property.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_columns1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_columns1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_columns1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_columns1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_columns2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_columns2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_columns2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_columns2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_columns3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_columns3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_columns3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_columns3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_index.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_index.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_create_table_index.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_create_table_index.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_partitions1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_partitions1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_partitions1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_partitions1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tableproperties1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_tableproperties1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_tableproperties1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_tableproperties1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_tables_bad_db2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/smb_mapjoin_14.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/smb_mapjoin_14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/smb_mapjoin_14.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/smb_mapjoin_14.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/split_sample_out_of_range.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/split_sample_out_of_range.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/split_sample_out_of_range.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/split_sample_out_of_range.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_noscan_non_native.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/stats_noscan_non_native.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/stats_noscan_non_native.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/stats_noscan_non_native.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/stats_publisher_error_2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/strict_join.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/strict_join.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/strict_join.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/strict_join.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/strict_orderby.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/strict_orderby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/strict_orderby.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/strict_orderby.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/strict_pruning.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/strict_pruning.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/strict_pruning.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/strict_pruning.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/subq_insert.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/subq_insert.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/subq_insert.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/subq_insert.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/touch1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/touch1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/touch1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/touch1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/touch2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/touch2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/touch2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/touch2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_bucketed_column.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_bucketed_column.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_bucketed_column.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_bucketed_column.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_seqfile.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_seqfile.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_column_seqfile.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_column_seqfile.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_partition_column2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/truncate_table_failure4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udaf_invalid_place.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udaf_invalid_place.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udaf_invalid_place.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udaf_invalid_place.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_assert_true2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_coalesce.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_coalesce.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_coalesce.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_coalesce.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_field_wrong_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_if_not_bool.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_if_not_bool.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_if_not_bool.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_if_not_bool.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_in.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_in.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_in.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_in.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_max.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_max.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_max.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_max.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_min.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_min.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_min.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_min.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_printf_wrong4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_reflect_neg.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_reflect_neg.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_reflect_neg.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_reflect_neg.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_size_wrong_type.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_test_error.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error_reduce.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error_reduce.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_test_error_reduce.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_test_error_reduce.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udfnull.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udfnull.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udfnull.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udfnull.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_invalid_place.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_invalid_place.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udtf_invalid_place.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_invalid_place.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported1.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported1.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported1.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/udtf_not_supported3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/union.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/union.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/union.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/union.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/union2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/union2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/union2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/union2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/union22.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/union22.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/union22.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/union22.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/union3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/union3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/union3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/union3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/uniquejoin.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin2.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/uniquejoin2.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin2.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin3.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/uniquejoin3.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/uniquejoin3.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/unset_table_property.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/unset_table_property.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/unset_table_property.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/unset_table_property.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/unset_view_property.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/unset_view_property.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/unset_view_property.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/unset_view_property.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/windowing_ll_no_over.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/windowing_ll_no_over.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/windowing_ll_no_over.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/windowing_ll_no_over.q diff --git a/src/test/hive/ql/src/test/queries/clientnegative/wrong_column_type.q b/shark/src/test/hive/ql/src/test/queries/clientnegative/wrong_column_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientnegative/wrong_column_type.q rename to shark/src/test/hive/ql/src/test/queries/clientnegative/wrong_column_type.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/add_part_exist.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/add_part_exist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/add_part_exist.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/add_part_exist.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/add_part_multiple.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/add_part_multiple.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/add_part_multiple.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/add_part_multiple.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alias_casted_column.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alias_casted_column.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alias_casted_column.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alias_casted_column.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/allcolref_in_udf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/allcolref_in_udf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/allcolref_in_udf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/allcolref_in_udf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_index.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_index.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_index.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_index.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_merge.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_merge.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_merge.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_merge.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_merge_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_stats.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_stats.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_merge_stats.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_merge_stats.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_coltype.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_coltype.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_partition_coltype.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_coltype.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_format_loc.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_format_loc.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_partition_format_loc.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_format_loc.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_skewed_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_skewed_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_skewed_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_skewed_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_table_not_sorted.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_table_not_sorted.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_table_not_sorted.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_table_not_sorted.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_table_serde2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_varchar1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_varchar2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_varchar2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_view_as_select.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_view_as_select.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_view_as_select.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_view_as_select.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/alter_view_rename.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/alter_view_rename.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/alter_view_rename.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/alter_view_rename.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ambiguous_col.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ambiguous_col.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ambiguous_col.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ambiguous_col.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/archive.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/archive.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/archive.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/archive.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/archive_corrupt.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/archive_corrupt.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/archive_corrupt.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/archive_corrupt.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/archive_multi.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/archive_multi.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/archive_multi.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/archive_multi.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/authorization_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/authorization_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/authorization_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/authorization_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/authorization_5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/authorization_6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/authorization_7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/authorization_8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/authorization_8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/authorization_8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join0.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join0.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join0.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join0.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join16.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join16.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join16.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join16.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join17.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join17.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join17.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join17.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join18.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join18.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join18.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join18.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join19.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join19.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join19.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join19.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join21.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join21.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join21.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join21.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join22.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join22.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join22.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join22.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join23.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join23.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join23.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join23.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join24.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join24.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join24.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join24.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join25.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join25.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join25.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join25.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join26.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join26.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join26.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join26.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join27.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join27.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join27.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join27.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join28.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join28.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join28.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join28.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join29.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join29.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join29.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join29.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join30.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join30.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join30.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join30.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join31.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join31.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join31.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join31.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join32.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join32.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join32.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join32.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join_filters.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join_filters.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join_filters.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join_filters.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join_nulls.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join_nulls.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join_nulls.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join_nulls.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_join_reordering_values.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join_reordering_values.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_join_reordering_values.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_join_reordering_values.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/autogen_colalias.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/autogen_colalias.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/autogen_colalias.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/autogen_colalias.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_change_schema.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/avro_change_schema.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/avro_change_schema.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/avro_change_schema.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_compression_enabled.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/avro_compression_enabled.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/avro_compression_enabled.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/avro_compression_enabled.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_evolved_schemas.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/avro_evolved_schemas.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/avro_evolved_schemas.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/avro_evolved_schemas.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_joins.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/avro_joins.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/avro_joins.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/avro_joins.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_nullable_fields.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/avro_nullable_fields.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/avro_nullable_fields.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/avro_nullable_fields.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_partitioned.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/avro_partitioned.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/avro_partitioned.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/avro_partitioned.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_sanity_test.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/avro_sanity_test.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/avro_sanity_test.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/avro_sanity_test.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_error_message.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_error_message.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/avro_schema_error_message.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_error_message.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_literal.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_literal.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/avro_schema_literal.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/avro_schema_literal.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ba_table1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ba_table1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ba_table1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ba_table1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ba_table2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ba_table2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ba_table2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ba_table2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ba_table3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ba_table3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ba_table3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ba_table3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ba_table_udfs.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ba_table_udfs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ba_table_udfs.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ba_table_udfs.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ba_table_union.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ba_table_union.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ba_table_union.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ba_table_union.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/binary_constant.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/binary_constant.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/binary_constant.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/binary_constant.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/binary_output_format.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/binary_output_format.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/binary_output_format.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/binary_output_format.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/binary_table_bincolserde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/binary_table_bincolserde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/binary_table_bincolserde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/binary_table_bincolserde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/binary_table_colserde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/binary_table_colserde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/binary_table_colserde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/binary_table_colserde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/binarysortable_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/binarysortable_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/binarysortable_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/binarysortable_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucket1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucket1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucket1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucket2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucket2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucket2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucket3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucket3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucket3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucket4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucket4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucket4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucket5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucket5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucket5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket_groupby.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucket_groupby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucket_groupby.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucket_groupby.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucket_map_join_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucket_num_reducers2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketcontext_8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/case_sensitivity.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/case_sensitivity.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/case_sensitivity.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/case_sensitivity.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/cast1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/cast1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/cast1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/cast1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/cast_to_int.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/cast_to_int.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/cast_to_int.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/cast_to_int.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/cluster.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/cluster.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/cluster.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/cluster.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/column_access_stats.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/column_access_stats.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/column_access_stats.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/column_access_stats.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/columnstats_partlvl.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/columnstats_partlvl.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/columnstats_partlvl.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/columnstats_partlvl.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/columnstats_tbllvl.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/columnstats_tbllvl.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/columnstats_tbllvl.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/columnstats_tbllvl.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/combine1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/combine1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/combine1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/combine1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/combine2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/combine2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/combine2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/combine2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/combine2_hadoop20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/combine2_hadoop20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/combine2_hadoop20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/combine2_hadoop20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/combine2_win.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/combine2_win.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/combine2_win.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/combine2_win.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/combine3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/combine3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/combine3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/combine3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_binary.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_binary.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/compute_stats_binary.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_binary.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_boolean.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_boolean.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/compute_stats_boolean.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_boolean.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_double.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_double.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/compute_stats_double.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_double.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_empty_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_empty_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/compute_stats_empty_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_empty_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_long.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_long.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/compute_stats_long.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_long.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_string.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_string.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/compute_stats_string.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/compute_stats_string.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/constant_prop.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/constant_prop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/constant_prop.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/constant_prop.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/convert_enum_to_string.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/convert_enum_to_string.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/convert_enum_to_string.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/convert_enum_to_string.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/correlationoptimizer9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/count.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/count.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/count.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/count.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/cp_mj_rc.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/cp_mj_rc.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/cp_mj_rc.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/cp_mj_rc.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_big_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_big_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_big_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_big_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_default_prop.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_default_prop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_default_prop.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_default_prop.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_escape.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_escape.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_escape.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_escape.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_genericudaf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_genericudaf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_genericudaf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_genericudaf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_genericudf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_genericudf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_genericudf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_genericudf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_insert_outputformat.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_insert_outputformat.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_insert_outputformat.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_insert_outputformat.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_like.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_like.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_like.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_like.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_like2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_like2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_like2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_like2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_like_tbl_props.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_like_tbl_props.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_like_tbl_props.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_like_tbl_props.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_like_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_like_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_like_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_like_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_merge_compressed.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_merge_compressed.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_merge_compressed.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_merge_compressed.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_nested_type.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_nested_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_nested_type.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_nested_type.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_or_replace_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_or_replace_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_or_replace_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_or_replace_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_skewed_table1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_skewed_table1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_skewed_table1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_skewed_table1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_struct_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_struct_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_struct_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_struct_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_udaf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_udaf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_udaf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_udaf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_union_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_union_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_union_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_union_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_view_partitioned.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_view_partitioned.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_view_partitioned.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_view_partitioned.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/create_view_translate.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/create_view_translate.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/create_view_translate.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/create_view_translate.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/cross_join.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/cross_join.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/cross_join.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/cross_join.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ct_case_insensitive.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ct_case_insensitive.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ct_case_insensitive.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ct_case_insensitive.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ctas.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ctas.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ctas.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas_colname.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ctas_colname.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ctas_colname.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ctas_colname.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas_date.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ctas_date.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ctas_date.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ctas_date.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas_hadoop20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ctas_hadoop20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ctas_hadoop20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ctas_hadoop20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas_uses_database_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ctas_uses_database_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ctas_uses_database_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ctas_uses_database_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ctas_varchar.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ctas_varchar.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ctas_varchar.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ctas_varchar.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/custom_input_output_format.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/custom_input_output_format.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/custom_input_output_format.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/custom_input_output_format.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/database.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/database.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/database.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/database.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/database_drop.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/database_drop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/database_drop.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/database_drop.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/database_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/database_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/database_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/database_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/database_properties.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/database_properties.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/database_properties.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/database_properties.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/date_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/date_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/date_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/date_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/date_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/date_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/date_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/date_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/date_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/date_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/date_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/date_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_comparison.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/date_comparison.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/date_comparison.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/date_comparison.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_join1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/date_join1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/date_join1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/date_join1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_serde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/date_serde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/date_serde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/date_serde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/date_udf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/date_udf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/date_udf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/date_udf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ddltime.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ddltime.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ddltime.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ddltime.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/decimal_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/decimal_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/decimal_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/decimal_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_join.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_join.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/decimal_join.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_join.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_precision.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_precision.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/decimal_precision.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_precision.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_serde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_serde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/decimal_serde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_serde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/decimal_udf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_udf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/decimal_udf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/decimal_udf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/default_partition_name.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/default_partition_name.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/default_partition_name.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/default_partition_name.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/delimiter.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/delimiter.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/delimiter.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/delimiter.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_indent.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_indent.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/describe_comment_indent.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_indent.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_nonascii.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_nonascii.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/describe_comment_nonascii.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/describe_comment_nonascii.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_database_json.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/describe_database_json.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/describe_database_json.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/describe_database_json.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_pretty.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/describe_pretty.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/describe_pretty.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/describe_pretty.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_syntax.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/describe_syntax.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/describe_syntax.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/describe_syntax.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/describe_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/describe_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/describe_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_table_json.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/describe_table_json.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/describe_table_json.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/describe_table_json.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/describe_xpath.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/describe_xpath.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/describe_xpath.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/describe_xpath.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/diff_part_input_formats.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/diff_part_input_formats.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/diff_part_input_formats.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/diff_part_input_formats.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/disable_file_format_check.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/disable_file_format_check.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/disable_file_format_check.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/disable_file_format_check.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/driverhook.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/driverhook.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/driverhook.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/driverhook.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_function.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_function.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_function.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_function.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_index.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_index.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_index.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_index.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_multi_partitions.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_multi_partitions.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_multi_partitions.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_multi_partitions.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_filter3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_table2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_table2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_table2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_table2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_udf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_udf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_udf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_udf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/drop_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/drop_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/drop_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/drop_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/enforce_order.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/enforce_order.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/enforce_order.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/enforce_order.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/escape1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/escape1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/escape1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/escape2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/escape2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/escape2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape_clusterby1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/escape_clusterby1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/escape_clusterby1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/escape_clusterby1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape_distributeby1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/escape_distributeby1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/escape_distributeby1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/escape_distributeby1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape_orderby1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/escape_orderby1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/escape_orderby1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/escape_orderby1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/escape_sortby1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/escape_sortby1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/escape_sortby1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/escape_sortby1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exchange_partition.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exchange_partition2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exchange_partition3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exchange_partition3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_01_nonpart.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_01_nonpart.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_01_nonpart.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_01_nonpart.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_02_part.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_02_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_02_part.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_02_part.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_04_all_part.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_04_all_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_04_all_part.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_04_all_part.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_05_some_part.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_05_some_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_05_some_part.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_05_some_part.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_06_one_part.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_06_one_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_06_one_part.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_06_one_part.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_10_external_managed.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_10_external_managed.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_10_external_managed.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_10_external_managed.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_11_managed_external.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_11_managed_external.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_11_managed_external.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_11_managed_external.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_12_external_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_12_external_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_12_external_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_12_external_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_13_managed_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_13_managed_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_13_managed_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_13_managed_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_15_external_part.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_15_external_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_15_external_part.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_15_external_part.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_16_part_external.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_16_part_external.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_16_part_external.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_16_part_external.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_17_part_managed.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_17_part_managed.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_17_part_managed.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_17_part_managed.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_18_part_external.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_18_part_external.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_18_part_external.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_18_part_external.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_19_part_external_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_19_part_external_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_19_part_external_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_19_part_external_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/explain_dependency.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/explain_dependency2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/explain_dependency2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/explain_logical.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/explain_logical.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/explain_logical.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/explain_logical.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/explode_null.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/explode_null.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/explode_null.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/explode_null.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/fetch_aggregation.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/fetch_aggregation.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/fetch_aggregation.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/fetch_aggregation.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/fileformat_mix.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/fileformat_mix.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/fileformat_mix.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/fileformat_mix.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/fileformat_sequencefile.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/fileformat_sequencefile.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/fileformat_sequencefile.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/fileformat_sequencefile.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/fileformat_text.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/fileformat_text.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/fileformat_text.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/fileformat_text.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/filter_join_breaktask2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/global_limit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/global_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/global_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/global_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1_limit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby1_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby1_map.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_nomap.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_nomap.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_nomap.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_nomap.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_skew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_skew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_skew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1_map_skew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby1_noskew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1_noskew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby1_noskew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby1_noskew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_limit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby2_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby2_map.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_skew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_skew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_skew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_map_skew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby3_map.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_skew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_skew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_skew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3_map_skew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby4_map.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map_skew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map_skew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby4_map_skew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby4_map_skew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby4_noskew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby4_noskew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby4_noskew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby4_noskew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby5_map.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map_skew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map_skew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby5_map_skew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby5_map_skew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby5_noskew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby5_noskew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby5_noskew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby5_noskew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby6_map.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map_skew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map_skew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby6_map_skew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby6_map_skew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby6_noskew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby6_noskew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby6_noskew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby6_noskew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby7_map.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_skew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_skew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_skew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7_map_skew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby8_map.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map_skew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map_skew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby8_map_skew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby8_map_skew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby8_noskew.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby8_noskew.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby8_noskew.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby8_noskew.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_bigdata.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_bigdata.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_bigdata.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_bigdata.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_cube1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_cube1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_cube1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_cube1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_id2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_neg_float.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_neg_float.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_neg_float.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_neg_float.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_position.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_position.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_position.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_position.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppd.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppd.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_ppd.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppd.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_rollup1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_rollup1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_rollup1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_rollup1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_test_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_test_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_test_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/groupby_sort_test_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/having.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/having.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/having.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/having.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/hook_context_cs.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/hook_context_cs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/hook_context_cs.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/hook_context_cs.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/hook_order.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/hook_order.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/hook_order.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/hook_order.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/implicit_cast1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/implicit_cast1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/implicit_cast1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/implicit_cast1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auth.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auth.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auth.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auth.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auto.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_empty.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_empty.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auto_empty.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_empty.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_file_format.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_file_format.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auto_file_format.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_file_format.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_multiple.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_multiple.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auto_multiple.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_multiple.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_partitioned.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_partitioned.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auto_partitioned.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_partitioned.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_self_join.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_self_join.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auto_self_join.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_self_join.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_unused.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_unused.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auto_unused.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_unused.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_auto_update.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_update.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_auto_update.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_auto_update.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_bitmap.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_bitmap1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_bitmap2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_bitmap3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_compression.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_compression.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_compression.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_compression.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_rc.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_rc.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_rc.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_bitmap_rc.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compact.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_compact.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_compact.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_compact.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compact_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_compact_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_compact_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_compact_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compact_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_compact_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_compact_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_compact_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compact_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_compact_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_compact_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_compact_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compact_binary_search.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_compact_binary_search.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_compact_binary_search.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_compact_binary_search.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_compression.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_compression.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_compression.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_compression.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_creation.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_creation.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_creation.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_creation.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_serde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_serde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_serde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_serde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_stale.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_stale.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_stale.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_stale.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/index_stale_partitioned.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/index_stale_partitioned.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/index_stale_partitioned.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/index_stale_partitioned.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/infer_const_type.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/infer_const_type.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/infer_const_type.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/infer_const_type.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/init_file.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/init_file.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/init_file.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/init_file.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/innerjoin.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/innerjoin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/innerjoin.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/innerjoin.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inoutdriver.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/inoutdriver.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/inoutdriver.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/inoutdriver.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input0.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input0.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input0.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input0.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input11_limit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input11_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input11_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input11_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input12_hadoop20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input12_hadoop20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input12_hadoop20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input12_hadoop20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input14_limit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input14_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input14_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input14_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input16.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input16.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input16.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input16.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input16_cc.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input16_cc.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input16_cc.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input16_cc.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input17.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input17.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input17.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input17.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input18.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input18.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input18.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input18.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input19.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input19.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input19.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input19.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input1_limit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input1_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input1_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input1_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input21.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input21.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input21.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input21.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input22.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input22.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input22.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input22.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input23.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input23.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input23.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input23.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input24.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input24.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input24.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input24.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input25.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input25.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input25.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input25.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input26.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input26.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input26.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input26.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input28.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input28.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input28.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input28.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input2_limit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input2_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input2_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input2_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input30.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input30.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input30.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input30.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input31.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input31.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input31.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input31.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input32.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input32.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input32.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input32.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input33.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input33.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input33.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input33.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input34.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input34.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input34.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input34.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input35.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input35.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input35.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input35.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input36.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input36.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input36.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input36.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input37.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input37.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input37.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input37.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input38.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input38.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input38.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input38.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input39.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input39.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input39.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input39.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input39_hadoop20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input39_hadoop20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input39_hadoop20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input39_hadoop20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input3_limit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input3_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input3_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input3_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input40.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input40.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input40.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input40.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input41.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input41.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input41.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input41.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input42.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input42.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input42.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input42.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input43.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input43.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input43.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input43.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input44.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input44.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input44.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input44.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input45.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input45.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input45.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input45.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input46.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input46.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input46.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input46.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input49.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input49.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input49.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input49.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input4_cb_delim.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input4_cb_delim.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input4_cb_delim.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input4_cb_delim.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input4_limit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input4_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input4_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input4_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_columnarserde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_columnarserde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_columnarserde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_columnarserde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_dfs.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_dfs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_dfs.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_dfs.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_dynamicserde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_dynamicserde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_dynamicserde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_dynamicserde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_lazyserde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_lazyserde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_lazyserde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_lazyserde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_limit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_limit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_limit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_limit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part0.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part0.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part0.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part0.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part10_win.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part10_win.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part10_win.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part10_win.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_part9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_part9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_part9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_part9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_testxpath.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_testxpath2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_testxpath3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/input_testxpath4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/input_testxpath4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/inputddl1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/inputddl2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/inputddl3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/inputddl4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/inputddl5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/inputddl6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/inputddl7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/inputddl8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/inputddl8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/inputddl8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_compressed.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert_compressed.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert_compressed.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert_compressed.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert_into1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert_into2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert_into3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert_into4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert_into5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_into6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert_into6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert_into6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/insertexternal1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/insertexternal1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/insertexternal1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/insertexternal1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join0.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join0.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join0.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join0.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join14_hadoop20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join14_hadoop20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join14_hadoop20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join14_hadoop20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join16.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join16.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join16.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join16.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join17.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join17.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join17.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join17.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join18.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join18.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join18.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join18.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join18_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join18_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join18_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join18_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join19.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join19.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join19.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join19.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join21.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join21.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join21.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join21.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join22.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join22.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join22.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join22.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join23.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join23.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join23.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join23.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join24.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join24.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join24.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join24.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join25.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join25.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join25.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join25.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join26.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join26.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join26.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join26.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join27.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join27.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join27.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join27.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join28.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join28.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join28.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join28.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join29.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join29.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join29.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join29.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join30.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join30.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join30.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join30.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join31.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join31.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join31.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join31.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join32.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join32.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join32.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join32.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join32_lessSize.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join32_lessSize.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join32_lessSize.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join32_lessSize.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join33.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join33.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join33.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join33.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join34.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join34.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join34.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join34.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join35.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join35.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join35.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join35.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join36.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join36.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join36.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join36.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join37.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join37.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join37.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join37.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join38.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join38.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join38.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join38.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join39.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join39.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join39.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join39.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join40.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join40.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join40.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join40.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join41.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join41.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join41.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join41.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_1to1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_1to1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_1to1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_1to1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_array.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_array.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_array.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_array.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_casesensitive.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_casesensitive.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_casesensitive.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_casesensitive.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_empty.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_empty.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_empty.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_empty.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_filters.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_filters.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_filters.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_filters.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_filters_overlap.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_filters_overlap.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_filters_overlap.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_filters_overlap.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_hive_626.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_hive_626.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_hive_626.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_hive_626.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_literals.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_literals.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_literals.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_literals.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_map_ppr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_map_ppr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_map_ppr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_map_ppr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_nulls.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_nulls.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_nulls.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_nulls.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_nullsafe.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_nullsafe.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_nullsafe.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_nullsafe.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_rc.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_rc.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_rc.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_rc.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_reorder.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_reorder.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_reorder.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_reorder.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_reorder2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_reorder2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_reorder2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_reorder2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_reorder3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_reorder3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_reorder3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_reorder3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_reorder4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_reorder4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_reorder4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_reorder4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_star.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_star.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_star.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_star.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_thrift.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_thrift.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_thrift.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_thrift.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_vc.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_vc.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_vc.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_vc.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/join_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/join_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/join_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/join_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/keyword_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/keyword_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/keyword_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/keyword_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lateral_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/lateral_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/lateral_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/lateral_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_cp.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_cp.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/lateral_view_cp.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_cp.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_noalias.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_noalias.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/lateral_view_noalias.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_noalias.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_outer.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_outer.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/lateral_view_outer.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_outer.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_ppd.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_ppd.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/lateral_view_ppd.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/lateral_view_ppd.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/leadlag.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/leadlag.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/leadlag.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/leadlag.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/leadlag_queries.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/leadlag_queries.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/leadlag_queries.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/leadlag_queries.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin_mr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin_mr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin_mr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/leftsemijoin_mr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown_negative.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown_negative.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown_negative.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/limit_pushdown_negative.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lineage1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/lineage1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/lineage1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/lineage1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_dml_9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/literal_decimal.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/literal_decimal.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/literal_decimal.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/literal_decimal.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/literal_double.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/literal_double.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/literal_double.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/literal_double.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/literal_ints.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/literal_ints.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/literal_ints.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/literal_ints.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/literal_string.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/literal_string.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/literal_string.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/literal_string.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_binary_data.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_binary_data.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_binary_data.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_binary_data.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14_win.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14_win.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14_win.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part14_win.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_dyn_part9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_fs.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_fs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_fs.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_fs.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_fs2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_fs2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_fs2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_fs2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_overwrite.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_overwrite.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_overwrite.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_overwrite.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/load_part_authsuccess.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/load_part_authsuccess.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/load_part_authsuccess.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/load_part_authsuccess.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/loadpart1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/loadpart1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/loadpart1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/loadpart1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/loadpart_err.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/loadpart_err.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/loadpart_err.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/loadpart_err.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lock1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/lock1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/lock1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/lock1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lock2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/lock2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/lock2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/lock2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lock3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/lock3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/lock3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/lock3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/lock4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/lock4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/lock4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/lock4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/louter_join_ppr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/louter_join_ppr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/louter_join_ppr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/louter_join_ppr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/macro.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/macro.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/macro.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/macro.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapjoin1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapjoin_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_hook.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_hook.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapjoin_hook.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_hook.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_subquery2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_test_outer.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_test_outer.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapjoin_test_outer.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapjoin_test_outer.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapreduce1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapreduce2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapreduce3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapreduce4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapreduce5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapreduce6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapreduce7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mapreduce8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mapreduce8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mapreduce8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/merge1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/merge1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/merge1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/merge2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/merge2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/merge2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/merge3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/merge3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/merge3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/merge4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/merge4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/merge4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mergejoins.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mergejoins.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mergejoins.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mergejoins.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mergejoins_mixed.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mergejoins_mixed.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mergejoins_mixed.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mergejoins_mixed.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/metadata_export_drop.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/metadata_export_drop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/metadata_export_drop.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/metadata_export_drop.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/metadataonly1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/metadataonly1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/metadataonly1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/metadataonly1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/mi.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/mi.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/mi.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/mi.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/misc_json.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/misc_json.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/misc_json.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/misc_json.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multiMapJoin2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multi_insert.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_gby3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_join_union.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multi_join_union.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multi_join_union.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multi_join_union.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multi_sahooks.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multi_sahooks.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multi_sahooks.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multi_sahooks.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/multigroupby_singlemr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/multigroupby_singlemr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/multigroupby_singlemr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/multigroupby_singlemr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nested_complex.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nested_complex.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nested_complex.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nested_complex.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nestedvirtual.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nestedvirtual.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nestedvirtual.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nestedvirtual.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/newline.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/newline.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/newline.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/newline.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/no_hooks.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/no_hooks.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/no_hooks.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/no_hooks.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/noalias_subq1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/noalias_subq1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/noalias_subq1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/noalias_subq1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nonmr_fetch.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nonmr_fetch.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nonmr_fetch.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nonmr_fetch.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/notable_alias1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/notable_alias1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/notable_alias1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/notable_alias1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/notable_alias2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/notable_alias2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/notable_alias2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/notable_alias2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/null_cast.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/null_cast.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/null_cast.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/null_cast.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/null_column.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/null_column.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/null_column.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/null_column.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nullgroup.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nullgroup2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nullgroup3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nullgroup4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullgroup5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nullgroup5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nullgroup5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullinput.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nullinput.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nullinput.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nullinput.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullinput2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nullinput2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nullinput2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nullinput2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/nullscript.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/nullscript.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/nullscript.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/nullscript.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/num_op_type_conv.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/num_op_type_conv.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/num_op_type_conv.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/num_op_type_conv.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ops_comparison.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ops_comparison.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ops_comparison.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ops_comparison.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/optional_outer.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/optional_outer.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/optional_outer.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/optional_outer.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/optrstat_groupby.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/optrstat_groupby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/optrstat_groupby.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/optrstat_groupby.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_create.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/orc_create.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/orc_create.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/orc_create.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_createas1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/orc_createas1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/orc_createas1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/orc_createas1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_diff_part_cols.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/orc_diff_part_cols.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/orc_diff_part_cols.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/orc_diff_part_cols.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_files.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_files.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/orc_empty_files.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_files.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_strings.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_strings.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/orc_empty_strings.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/orc_empty_strings.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/order.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/order.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/order.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/order.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/order2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/order2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/order2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/order2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/outer_join_ppr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/outer_join_ppr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/outer_join_ppr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/outer_join_ppr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/overridden_confs.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/overridden_confs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/overridden_confs.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/overridden_confs.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/parallel.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/parallel.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/parallel.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/parallel.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/parallel_orderby.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/parallel_orderby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/parallel_orderby.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/parallel_orderby.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/parenthesis_star_by.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/parenthesis_star_by.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/parenthesis_star_by.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/parenthesis_star_by.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partInit.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partInit.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partInit.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partInit.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partcols1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partcols1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partcols1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partcols1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_date.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_date.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_date.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_date.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_date2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_date2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_date2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_date2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_decode_name.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_decode_name.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_decode_name.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_decode_name.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_schema1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_schema1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_schema1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_schema1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_serde_format.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_serde_format.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_serde_format.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_serde_format.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_special_char.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_special_char.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_special_char.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_special_char.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_type_check.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_type_check.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_type_check.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_type_check.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_varchar1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_varchar1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_varchar1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_varchar1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/partitions_json.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/partitions_json.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/partitions_json.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/partitions_json.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/pcr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/pcr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/pcr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/pcr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/plan_json.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/plan_json.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/plan_json.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/plan_json.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_clusterby.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_clusterby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_clusterby.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_clusterby.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_expr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_expr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_expr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_expr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_where.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_where.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_where.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_constant_where.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_gby.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_gby2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby_join.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby_join.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_gby_join.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_gby_join.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_join.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_join.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_join.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_join.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_join2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_join2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_join2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_join2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_join3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_join3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_join3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_join3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_join_filter.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_join_filter.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_join_filter.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_join_filter.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_multi_insert.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_multi_insert.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_multi_insert.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_multi_insert.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_outer_join5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_random.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_random.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_random.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_random.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_repeated_alias.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_repeated_alias.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_repeated_alias.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_repeated_alias.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_transform.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_transform.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_transform.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_transform.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_case.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_case.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_case.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_case.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_col.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_col.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_col.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_udf_col.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_union.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_union.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_union.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_union.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_union_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_union_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_union_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_union_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppd_vc.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_vc.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppd_vc.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppd_vc.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ppr_pushdown3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/print_header.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/print_header.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/print_header.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/print_header.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/progress_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/progress_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/progress_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/progress_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/protectmode.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/protectmode.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/protectmode.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/protectmode.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/protectmode2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/protectmode2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/protectmode2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/protectmode2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ptf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ptf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ptf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_decimal.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_decimal.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ptf_decimal.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_decimal.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_general_queries.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_general_queries.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ptf_general_queries.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_general_queries.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_matchpath.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_matchpath.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ptf_matchpath.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_matchpath.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_rcfile.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_rcfile.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ptf_rcfile.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_rcfile.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_register_tblfn.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_register_tblfn.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ptf_register_tblfn.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_register_tblfn.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ptf_seqfile.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_seqfile.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ptf_seqfile.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ptf_seqfile.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/push_or.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/push_or.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/push_or.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/push_or.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/query_properties.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/query_properties.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/query_properties.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/query_properties.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/query_result_fileformat.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/query_result_fileformat.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/query_result_fileformat.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/query_result_fileformat.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/query_with_semi.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/query_with_semi.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/query_with_semi.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/query_with_semi.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/quote1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/quote1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/quote1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/quote1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/quote2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/quote2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/quote2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/quote2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rand_partitionpruner3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_bigdata.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_bigdata.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_bigdata.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_bigdata.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_columnar.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_columnar.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_columnar.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_columnar.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_createas1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_createas1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_createas1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_createas1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_default_format.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_default_format.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_default_format.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_default_format.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_merge4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_null_value.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_null_value.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_null_value.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_null_value.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rcfile_union.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_union.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rcfile_union.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rcfile_union.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/recursive_dir.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/recursive_dir.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/recursive_dir.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/recursive_dir.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reducesink_dedup.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/reducesink_dedup.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/reducesink_dedup.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/reducesink_dedup.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/regex_col.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/regex_col.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/regex_col.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/regex_col.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/regexp_extract.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/regexp_extract.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/regexp_extract.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/regexp_extract.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/remote_script.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/remote_script.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/remote_script.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/remote_script.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rename_column.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rename_column.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rename_column.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rename_column.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rename_external_partition_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rename_external_partition_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rename_external_partition_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rename_external_partition_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rename_partition_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rename_partition_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rename_partition_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rename_partition_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/rename_table_location.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/rename_table_location.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/rename_table_location.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/rename_table_location.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/repair.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/repair.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/repair.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/repair.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/reset_conf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/reset_conf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/reset_conf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/reset_conf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/router_join_ppr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/router_join_ppr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/router_join_ppr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/router_join_ppr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/schemeAuthority2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/script_env_var1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/script_env_var1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/script_env_var1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/script_env_var1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/script_env_var2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/script_env_var2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/script_env_var2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/script_env_var2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/script_pipe.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/script_pipe.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/script_pipe.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/script_pipe.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/scriptfile1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/scriptfile1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/scriptfile1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/scriptfile1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/select_as_omitted.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/select_as_omitted.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/select_as_omitted.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/select_as_omitted.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/select_transform_hint.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/select_transform_hint.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/select_transform_hint.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/select_transform_hint.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_and.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_and.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/select_unquote_and.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_and.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_not.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_not.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/select_unquote_not.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_not.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_or.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_or.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/select_unquote_or.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/select_unquote_or.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/semicolon.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/semicolon.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/semicolon.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/semicolon.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/semijoin.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/semijoin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/semijoin.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/semijoin.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/serde_regex.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/serde_regex.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/serde_regex.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/serde_regex.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/serde_reported_schema.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/serde_reported_schema.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/serde_reported_schema.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/serde_reported_schema.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/serde_user_properties.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/serde_user_properties.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/serde_user_properties.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/serde_user_properties.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/set_processor_namespaces.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/set_processor_namespaces.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/set_processor_namespaces.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/set_processor_namespaces.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/set_variable_sub.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/set_variable_sub.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/set_variable_sub.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/set_variable_sub.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_columns.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_columns.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_columns.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_columns.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_alter.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_alter.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_create_table_alter.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_alter.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_db_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_db_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_create_table_db_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_db_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_delimited.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_delimited.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_create_table_delimited.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_delimited.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_partitioned.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_partitioned.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_create_table_partitioned.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_partitioned.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_serde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_serde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_create_table_serde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_serde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_create_table_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_create_table_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_describe_func_quotes.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_describe_func_quotes.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_describe_func_quotes.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_describe_func_quotes.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_functions.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_functions.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_functions.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_functions.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_syntax.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_syntax.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_indexes_syntax.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_indexes_syntax.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_partitions.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_partitions.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_partitions.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_partitions.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_tables.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_tables.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_tables.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_tables.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_tablestatus.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_tablestatus.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_tablestatus.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_tablestatus.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/show_tblproperties.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/show_tblproperties.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/show_tblproperties.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/show_tblproperties.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/showparts.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/showparts.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/showparts.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/showparts.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoin.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoin.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoin.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt16.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt16.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt16.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt16.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt17.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt17.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt17.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt17.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt18.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt18.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt18.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt18.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt19.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt19.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt19.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt19.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/skewjoinopt9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_16.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_16.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_16.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_16.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_17.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_17.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_17.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_17.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_18.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_18.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_18.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_18.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_19.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_19.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_19.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_19.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_21.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_21.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_21.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_21.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_22.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_22.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_22.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_22.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_25.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_25.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_25.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_25.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/smb_mapjoin_8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sort.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sort.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sort.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/source.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/source.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/source.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/source.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/split_sample.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/split_sample.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/split_sample.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/split_sample.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats0.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats0.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats0.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats0.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats16.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats16.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats16.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats16.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats18.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats18.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats18.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats18.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats19.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats19.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats19.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats19.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_partition.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_partition.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats_empty_partition.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats_empty_partition.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats_noscan_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1_23.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1_23.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1_23.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats_partscan_1_23.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/stats_publisher_error_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/stats_publisher_error_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/stats_publisher_error_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/stats_publisher_error_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/str_to_map.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/str_to_map.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/str_to_map.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/str_to_map.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/subq.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/subq.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/subq.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/subq.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/subq2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/subq2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/subq2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/subq2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/symlink_text_input_format.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/symlink_text_input_format.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/symlink_text_input_format.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/symlink_text_input_format.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/table_access_keys_stats.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/table_access_keys_stats.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/table_access_keys_stats.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/table_access_keys_stats.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/tablename_with_select.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/tablename_with_select.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/tablename_with_select.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/tablename_with_select.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/test_boolean_whereclause.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/test_boolean_whereclause.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/test_boolean_whereclause.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/test_boolean_whereclause.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/timestamp_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/timestamp_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/timestamp_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_comparison.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_comparison.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/timestamp_comparison.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_comparison.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_lazy.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_lazy.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/timestamp_lazy.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_lazy.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_null.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_null.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/timestamp_null.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_null.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/timestamp_udf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_udf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/timestamp_udf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/timestamp_udf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/touch.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/touch.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/touch.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/touch.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/transform1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/transform1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/transform1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/transform1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/transform2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/transform2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/transform2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/transform2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/transform_ppr1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/transform_ppr2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/transform_ppr2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/truncate_column.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/truncate_column.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/truncate_column.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/truncate_column.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_buckets.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_buckets.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/truncate_column_buckets.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_buckets.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_merge.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_merge.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/truncate_column_merge.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/truncate_column_merge.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/truncate_table.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/truncate_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/truncate_table.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/truncate_table.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/type_cast_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/type_cast_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/type_cast_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/type_cast_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/type_conversions_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/type_conversions_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/type_conversions_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/type_conversions_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/type_widening.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/type_widening.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/type_widening.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/type_widening.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_collect_set.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_collect_set.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udaf_collect_set.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_collect_set.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_context_ngrams.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_context_ngrams.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udaf_context_ngrams.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_context_ngrams.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_corr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_corr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udaf_corr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_corr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_pop.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_pop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_pop.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_pop.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_samp.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_samp.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_samp.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_covar_samp.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_ngrams.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_ngrams.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udaf_ngrams.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_ngrams.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_number_format.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_number_format.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udaf_number_format.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_number_format.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_10_trims.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_10_trims.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_10_trims.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_10_trims.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_E.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_E.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_E.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_E.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_PI.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_PI.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_PI.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_PI.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_abs.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_abs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_abs.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_abs.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_acos.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_acos.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_acos.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_acos.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_add.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_add.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_add.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_add.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_array.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_array.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_array.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_array.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_array_contains.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_array_contains.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_array_contains.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_array_contains.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_ascii.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ascii.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_ascii.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ascii.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_asin.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_asin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_asin.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_asin.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_atan.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_atan.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_atan.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_atan.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_avg.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_avg.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_avg.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_avg.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_between.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_between.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_between.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_between.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bigint.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bigint.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_bigint.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bigint.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bin.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_bin.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bin.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_and.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_and.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_and.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_and.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_empty.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_empty.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_empty.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_empty.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_or.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_or.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_or.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitmap_or.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_and.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_and.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_and.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_and.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_not.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_not.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_not.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_not.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_or.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_or.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_or.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_or.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_xor.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_xor.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_xor.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_bitwise_xor.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_boolean.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_boolean.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_boolean.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_boolean.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_case.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_case.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_case.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_case.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_case_column_pruning.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_case_column_pruning.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_case_column_pruning.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_case_column_pruning.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_case_thrift.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_case_thrift.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_case_thrift.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_case_thrift.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_ceil.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ceil.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_ceil.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ceil.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_ceiling.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ceiling.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_ceiling.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ceiling.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_coalesce.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_coalesce.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_coalesce.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_coalesce.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_compare_java_string.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_compare_java_string.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_compare_java_string.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_compare_java_string.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_concat.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_concat.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_concat.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_concat.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_insert2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_ws.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_ws.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_concat_ws.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_concat_ws.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_context_aware.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_context_aware.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_context_aware.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_context_aware.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_conv.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_conv.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_conv.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_conv.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_cos.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_cos.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_cos.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_cos.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_count.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_count.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_count.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_count.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_date_add.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_date_add.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_date_add.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_date_add.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_date_sub.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_date_sub.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_date_sub.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_date_sub.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_datediff.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_datediff.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_datediff.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_datediff.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_day.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_day.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_day.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_day.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_dayofmonth.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_dayofmonth.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_dayofmonth.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_dayofmonth.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_degrees.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_degrees.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_degrees.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_degrees.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_div.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_div.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_div.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_div.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_divide.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_divide.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_divide.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_divide.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_double.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_double.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_double.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_double.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_elt.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_elt.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_elt.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_elt.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_equal.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_equal.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_equal.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_equal.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_exp.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_exp.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_exp.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_exp.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_explode.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_explode.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_explode.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_explode.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_field.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_field.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_field.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_field.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_find_in_set.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_find_in_set.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_find_in_set.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_find_in_set.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_float.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_float.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_float.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_float.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_floor.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_floor.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_floor.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_floor.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_format_number.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_format_number.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_format_number.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_format_number.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_from_unixtime.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_from_unixtime.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_from_unixtime.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_from_unixtime.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_get_json_object.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_get_json_object.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_get_json_object.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_get_json_object.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthan.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthan.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthan.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthan.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_hash.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_hash.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_hash.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_hash.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_hex.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_hex.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_hex.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_hex.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_hour.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_hour.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_hour.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_hour.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_if.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_if.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_if.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_if.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_in.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_in.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_in.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_in.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_in_file.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_in_file.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_in_file.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_in_file.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_index.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_index.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_index.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_index.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_inline.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_inline.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_inline.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_inline.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_instr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_instr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_instr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_instr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_int.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_int.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_int.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_int.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_isnotnull.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_isnotnull.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_isnotnull.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_isnotnull.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_isnull.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_java_method.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_java_method.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_java_method.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_java_method.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_lcase.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_lcase.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_lcase.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_lcase.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_length.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_length.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_length.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_length.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_lessthan.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_lessthan.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_lessthan.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_lessthan.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_lessthanorequal.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_lessthanorequal.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_lessthanorequal.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_lessthanorequal.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_like.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_like.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_like.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_like.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_ln.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ln.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_ln.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ln.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_locate.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_locate.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_locate.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_locate.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_log.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_log.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_log.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_log.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_log10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_log10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_log10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_log10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_log2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_log2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_log2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_log2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_lower.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_lower.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_lower.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_lower.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_lpad.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_lpad.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_lpad.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_lpad.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_ltrim.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ltrim.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_ltrim.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ltrim.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_map.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_map.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_map.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_map.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_map_keys.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_map_keys.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_map_keys.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_map_keys.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_map_values.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_map_values.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_map_values.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_map_values.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_max.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_max.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_max.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_max.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_min.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_min.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_min.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_min.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_minute.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_minute.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_minute.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_minute.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_modulo.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_modulo.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_modulo.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_modulo.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_month.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_month.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_month.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_month.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_named_struct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_named_struct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_named_struct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_named_struct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_negative.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_negative.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_negative.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_negative.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_not.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_not.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_not.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_not.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_notequal.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_notequal.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_notequal.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_notequal.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_notop.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_notop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_notop.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_notop.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_nvl.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_nvl.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_nvl.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_nvl.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_or.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_or.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_or.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_or.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_parse_url.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_parse_url.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_parse_url.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_parse_url.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_percentile.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_percentile.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_percentile.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_percentile.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_pmod.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_pmod.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_pmod.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_pmod.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_positive.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_positive.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_positive.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_positive.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_pow.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_pow.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_pow.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_pow.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_power.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_power.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_power.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_power.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_printf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_printf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_printf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_printf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_radians.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_radians.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_radians.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_radians.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_rand.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_rand.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_rand.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_rand.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_reflect.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_reflect2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_reflect2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_regexp.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_extract.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_extract.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_extract.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_extract.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_replace.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_replace.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_replace.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_regexp_replace.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_repeat.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_repeat.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_repeat.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_repeat.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_reverse.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_reverse.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_reverse.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_reverse.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_rlike.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_rlike.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_rlike.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_rlike.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_round.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_round.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_round.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_round.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_round_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_round_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_round_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_round_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_round_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_round_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_round_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_round_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_rpad.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_rpad.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_rpad.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_rpad.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_rtrim.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_rtrim.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_rtrim.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_rtrim.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_second.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_second.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_second.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_second.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sentences.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sentences.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_sentences.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sentences.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sign.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sign.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_sign.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sign.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sin.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_sin.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sin.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_size.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_size.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_size.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_size.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_smallint.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_smallint.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_smallint.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_smallint.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sort_array.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sort_array.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_sort_array.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sort_array.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_space.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_space.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_space.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_space.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_split.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_split.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_split.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_split.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sqrt.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sqrt.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_sqrt.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sqrt.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_std.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_std.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_std.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_std.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_stddev.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_pop.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_pop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_pop.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_pop.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_samp.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_samp.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_samp.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_stddev_samp.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_string.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_string.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_string.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_string.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_struct.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_struct.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_struct.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_struct.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_substr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_substr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_substr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_substr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_substring.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_substring.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_substring.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_substring.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_subtract.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_subtract.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_subtract.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_subtract.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_sum.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sum.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_sum.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_sum.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_tan.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_tan.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_tan.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_tan.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_testlength.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_testlength2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_testlength2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_tinyint.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_tinyint.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_tinyint.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_tinyint.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_boolean.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_boolean.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_to_boolean.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_boolean.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_byte.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_byte.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_to_byte.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_byte.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_date.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_date.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_to_date.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_date.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_double.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_double.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_to_double.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_double.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_float.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_float.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_to_float.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_float.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_long.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_long.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_to_long.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_long.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_short.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_short.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_to_short.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_short.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_string.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_string.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_to_string.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_string.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_translate.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_translate.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_translate.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_translate.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_trim.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_trim.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_trim.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_trim.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_ucase.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ucase.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_ucase.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_ucase.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_unhex.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_unhex.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_unhex.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_unhex.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_union.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_union.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_union.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_union.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_unix_timestamp.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_unix_timestamp.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_unix_timestamp.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_unix_timestamp.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_upper.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_upper.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_upper.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_upper.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_var_pop.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_var_pop.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_var_pop.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_var_pop.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_var_samp.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_var_samp.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_var_samp.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_var_samp.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_variance.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_variance.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_variance.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_variance.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_weekofyear.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_weekofyear.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_weekofyear.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_weekofyear.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_when.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_when.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_when.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_when.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_xpath.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_boolean.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_boolean.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_boolean.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_boolean.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_double.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_double.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_double.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_double.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_float.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_float.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_float.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_float.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_int.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_int.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_int.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_int.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_long.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_long.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_long.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_long.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_short.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_short.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_short.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_short.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_string.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_string.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_string.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udf_xpath_string.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udtf_explode.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udtf_explode.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udtf_explode.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udtf_explode.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udtf_json_tuple.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udtf_json_tuple.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udtf_json_tuple.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udtf_json_tuple.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/udtf_stack.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/udtf_stack.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/udtf_stack.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/udtf_stack.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/unicode_notation.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/unicode_notation.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/unicode_notation.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/unicode_notation.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union16.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union16.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union16.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union16.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union17.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union17.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union17.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union17.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union18.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union18.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union18.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union18.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union19.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union19.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union19.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union19.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union21.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union21.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union21.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union21.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union22.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union22.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union22.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union22.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union23.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union23.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union23.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union23.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union24.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union24.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union24.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union24.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union25.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union25.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union25.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union25.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union26.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union26.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union26.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union26.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union27.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union27.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union27.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union27.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union28.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union28.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union28.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union28.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union29.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union29.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union29.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union29.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union30.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union30.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union30.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union30.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union31.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union31.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union31.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union31.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union32.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union32.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union32.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union32.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union33.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union33.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union33.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union33.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union34.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union34.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union34.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union34.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_date.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_date.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_date.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_date.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_lateralview.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_lateralview.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_lateralview.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_lateralview.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_null.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_null.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_null.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_null.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_ppr.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_ppr.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_ppr.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_ppr.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_10.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_10.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_10.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_10.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_11.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_11.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_11.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_11.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_12.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_12.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_12.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_12.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_13.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_13.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_13.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_13.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_14.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_14.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_14.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_14.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_15.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_15.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_15.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_15.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_16.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_16.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_16.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_16.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_17.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_17.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_17.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_17.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_18.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_18.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_18.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_18.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_19.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_19.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_19.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_19.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_20.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_20.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_20.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_21.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_21.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_21.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_21.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_22.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_22.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_22.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_22.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_23.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_23.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_23.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_23.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_24.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_24.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_24.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_24.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_3.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_3.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_3.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_4.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_4.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_4.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_5.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_5.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_5.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_6.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_6.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_6.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_7.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_7.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_7.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_8.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_8.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_8.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_remove_9.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_remove_9.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_remove_9.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_script.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_script.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_script.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_script.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/union_view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/union_view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/union_view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/union_view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/uniquejoin.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/uniquejoin.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/uniquejoin.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/uniquejoin.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/unset_table_view_property.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/unset_table_view_property.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/unset_table_view_property.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/unset_table_view_property.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/updateAccessTime.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/updateAccessTime.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/updateAccessTime.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/updateAccessTime.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/varchar_1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_2.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/varchar_2.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_2.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_cast.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_cast.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/varchar_cast.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_cast.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_comparison.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_comparison.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/varchar_comparison.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_comparison.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_join1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_join1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/varchar_join1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_join1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_nested_types.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_nested_types.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/varchar_nested_types.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_nested_types.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_serde.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_serde.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/varchar_serde.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_serde.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_udf1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_udf1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/varchar_udf1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_udf1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/varchar_union1.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_union1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/varchar_union1.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/varchar_union1.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/view.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/view.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/view.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/view.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/view_cast.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/view_cast.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/view_cast.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/view_cast.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/view_inputs.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/view_inputs.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/view_inputs.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/view_inputs.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/virtual_column.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/virtual_column.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/virtual_column.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/virtual_column.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/windowing.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/windowing.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/windowing.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_columnPruning.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_columnPruning.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/windowing_columnPruning.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_columnPruning.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_expressions.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_expressions.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/windowing_expressions.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_expressions.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_multipartitioning.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_multipartitioning.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/windowing_multipartitioning.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_multipartitioning.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_navfn.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_navfn.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/windowing_navfn.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_navfn.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_ntile.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_ntile.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/windowing_ntile.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_ntile.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_rank.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_rank.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/windowing_rank.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_rank.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_udaf.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_udaf.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/windowing_udaf.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_udaf.q diff --git a/src/test/hive/ql/src/test/queries/clientpositive/windowing_windowspec.q b/shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_windowspec.q similarity index 100% rename from src/test/hive/ql/src/test/queries/clientpositive/windowing_windowspec.q rename to shark/src/test/hive/ql/src/test/queries/clientpositive/windowing_windowspec.q diff --git a/src/test/hive/ql/src/test/queries/negative/ambiguous_join_col.q b/shark/src/test/hive/ql/src/test/queries/negative/ambiguous_join_col.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/ambiguous_join_col.q rename to shark/src/test/hive/ql/src/test/queries/negative/ambiguous_join_col.q diff --git a/src/test/hive/ql/src/test/queries/negative/duplicate_alias.q b/shark/src/test/hive/ql/src/test/queries/negative/duplicate_alias.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/duplicate_alias.q rename to shark/src/test/hive/ql/src/test/queries/negative/duplicate_alias.q diff --git a/src/test/hive/ql/src/test/queries/negative/garbage.q b/shark/src/test/hive/ql/src/test/queries/negative/garbage.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/garbage.q rename to shark/src/test/hive/ql/src/test/queries/negative/garbage.q diff --git a/src/test/hive/ql/src/test/queries/negative/insert_wrong_number_columns.q b/shark/src/test/hive/ql/src/test/queries/negative/insert_wrong_number_columns.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/insert_wrong_number_columns.q rename to shark/src/test/hive/ql/src/test/queries/negative/insert_wrong_number_columns.q diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_create_table.q b/shark/src/test/hive/ql/src/test/queries/negative/invalid_create_table.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/invalid_create_table.q rename to shark/src/test/hive/ql/src/test/queries/negative/invalid_create_table.q diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_dot.q b/shark/src/test/hive/ql/src/test/queries/negative/invalid_dot.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/invalid_dot.q rename to shark/src/test/hive/ql/src/test/queries/negative/invalid_dot.q diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_function_param2.q b/shark/src/test/hive/ql/src/test/queries/negative/invalid_function_param2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/invalid_function_param2.q rename to shark/src/test/hive/ql/src/test/queries/negative/invalid_function_param2.q diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_index.q b/shark/src/test/hive/ql/src/test/queries/negative/invalid_index.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/invalid_index.q rename to shark/src/test/hive/ql/src/test/queries/negative/invalid_index.q diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_list_index.q b/shark/src/test/hive/ql/src/test/queries/negative/invalid_list_index.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/invalid_list_index.q rename to shark/src/test/hive/ql/src/test/queries/negative/invalid_list_index.q diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_list_index2.q b/shark/src/test/hive/ql/src/test/queries/negative/invalid_list_index2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/invalid_list_index2.q rename to shark/src/test/hive/ql/src/test/queries/negative/invalid_list_index2.q diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_map_index.q b/shark/src/test/hive/ql/src/test/queries/negative/invalid_map_index.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/invalid_map_index.q rename to shark/src/test/hive/ql/src/test/queries/negative/invalid_map_index.q diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_map_index2.q b/shark/src/test/hive/ql/src/test/queries/negative/invalid_map_index2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/invalid_map_index2.q rename to shark/src/test/hive/ql/src/test/queries/negative/invalid_map_index2.q diff --git a/src/test/hive/ql/src/test/queries/negative/invalid_select.q b/shark/src/test/hive/ql/src/test/queries/negative/invalid_select.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/invalid_select.q rename to shark/src/test/hive/ql/src/test/queries/negative/invalid_select.q diff --git a/src/test/hive/ql/src/test/queries/negative/macro_reserved_word.q b/shark/src/test/hive/ql/src/test/queries/negative/macro_reserved_word.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/macro_reserved_word.q rename to shark/src/test/hive/ql/src/test/queries/negative/macro_reserved_word.q diff --git a/src/test/hive/ql/src/test/queries/negative/missing_overwrite.q b/shark/src/test/hive/ql/src/test/queries/negative/missing_overwrite.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/missing_overwrite.q rename to shark/src/test/hive/ql/src/test/queries/negative/missing_overwrite.q diff --git a/src/test/hive/ql/src/test/queries/negative/nonkey_groupby.q b/shark/src/test/hive/ql/src/test/queries/negative/nonkey_groupby.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/nonkey_groupby.q rename to shark/src/test/hive/ql/src/test/queries/negative/nonkey_groupby.q diff --git a/src/test/hive/ql/src/test/queries/negative/quoted_string.q b/shark/src/test/hive/ql/src/test/queries/negative/quoted_string.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/quoted_string.q rename to shark/src/test/hive/ql/src/test/queries/negative/quoted_string.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column1.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_column1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_column1.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_column1.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column2.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_column2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_column2.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_column2.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column3.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_column3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_column3.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_column3.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column4.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_column4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_column4.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_column4.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column5.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_column5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_column5.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_column5.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_column6.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_column6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_column6.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_column6.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_function1.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_function1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_function1.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_function1.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_function2.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_function2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_function2.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_function2.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_function3.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_function3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_function3.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_function3.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_function4.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_function4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_function4.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_function4.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_table1.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_table1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_table1.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_table1.q diff --git a/src/test/hive/ql/src/test/queries/negative/unknown_table2.q b/shark/src/test/hive/ql/src/test/queries/negative/unknown_table2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/unknown_table2.q rename to shark/src/test/hive/ql/src/test/queries/negative/unknown_table2.q diff --git a/src/test/hive/ql/src/test/queries/negative/wrong_distinct1.q b/shark/src/test/hive/ql/src/test/queries/negative/wrong_distinct1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/wrong_distinct1.q rename to shark/src/test/hive/ql/src/test/queries/negative/wrong_distinct1.q diff --git a/src/test/hive/ql/src/test/queries/negative/wrong_distinct2.q b/shark/src/test/hive/ql/src/test/queries/negative/wrong_distinct2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/negative/wrong_distinct2.q rename to shark/src/test/hive/ql/src/test/queries/negative/wrong_distinct2.q diff --git a/src/test/hive/ql/src/test/queries/positive/case_sensitivity.q b/shark/src/test/hive/ql/src/test/queries/positive/case_sensitivity.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/case_sensitivity.q rename to shark/src/test/hive/ql/src/test/queries/positive/case_sensitivity.q diff --git a/src/test/hive/ql/src/test/queries/positive/cast1.q b/shark/src/test/hive/ql/src/test/queries/positive/cast1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/cast1.q rename to shark/src/test/hive/ql/src/test/queries/positive/cast1.q diff --git a/src/test/hive/ql/src/test/queries/positive/groupby1.q b/shark/src/test/hive/ql/src/test/queries/positive/groupby1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/groupby1.q rename to shark/src/test/hive/ql/src/test/queries/positive/groupby1.q diff --git a/src/test/hive/ql/src/test/queries/positive/groupby2.q b/shark/src/test/hive/ql/src/test/queries/positive/groupby2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/groupby2.q rename to shark/src/test/hive/ql/src/test/queries/positive/groupby2.q diff --git a/src/test/hive/ql/src/test/queries/positive/groupby3.q b/shark/src/test/hive/ql/src/test/queries/positive/groupby3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/groupby3.q rename to shark/src/test/hive/ql/src/test/queries/positive/groupby3.q diff --git a/src/test/hive/ql/src/test/queries/positive/groupby4.q b/shark/src/test/hive/ql/src/test/queries/positive/groupby4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/groupby4.q rename to shark/src/test/hive/ql/src/test/queries/positive/groupby4.q diff --git a/src/test/hive/ql/src/test/queries/positive/groupby5.q b/shark/src/test/hive/ql/src/test/queries/positive/groupby5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/groupby5.q rename to shark/src/test/hive/ql/src/test/queries/positive/groupby5.q diff --git a/src/test/hive/ql/src/test/queries/positive/groupby6.q b/shark/src/test/hive/ql/src/test/queries/positive/groupby6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/groupby6.q rename to shark/src/test/hive/ql/src/test/queries/positive/groupby6.q diff --git a/src/test/hive/ql/src/test/queries/positive/input1.q b/shark/src/test/hive/ql/src/test/queries/positive/input1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input1.q rename to shark/src/test/hive/ql/src/test/queries/positive/input1.q diff --git a/src/test/hive/ql/src/test/queries/positive/input2.q b/shark/src/test/hive/ql/src/test/queries/positive/input2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input2.q rename to shark/src/test/hive/ql/src/test/queries/positive/input2.q diff --git a/src/test/hive/ql/src/test/queries/positive/input20.q b/shark/src/test/hive/ql/src/test/queries/positive/input20.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input20.q rename to shark/src/test/hive/ql/src/test/queries/positive/input20.q diff --git a/src/test/hive/ql/src/test/queries/positive/input3.q b/shark/src/test/hive/ql/src/test/queries/positive/input3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input3.q rename to shark/src/test/hive/ql/src/test/queries/positive/input3.q diff --git a/src/test/hive/ql/src/test/queries/positive/input4.q b/shark/src/test/hive/ql/src/test/queries/positive/input4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input4.q rename to shark/src/test/hive/ql/src/test/queries/positive/input4.q diff --git a/src/test/hive/ql/src/test/queries/positive/input5.q b/shark/src/test/hive/ql/src/test/queries/positive/input5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input5.q rename to shark/src/test/hive/ql/src/test/queries/positive/input5.q diff --git a/src/test/hive/ql/src/test/queries/positive/input6.q b/shark/src/test/hive/ql/src/test/queries/positive/input6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input6.q rename to shark/src/test/hive/ql/src/test/queries/positive/input6.q diff --git a/src/test/hive/ql/src/test/queries/positive/input7.q b/shark/src/test/hive/ql/src/test/queries/positive/input7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input7.q rename to shark/src/test/hive/ql/src/test/queries/positive/input7.q diff --git a/src/test/hive/ql/src/test/queries/positive/input8.q b/shark/src/test/hive/ql/src/test/queries/positive/input8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input8.q rename to shark/src/test/hive/ql/src/test/queries/positive/input8.q diff --git a/src/test/hive/ql/src/test/queries/positive/input9.q b/shark/src/test/hive/ql/src/test/queries/positive/input9.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input9.q rename to shark/src/test/hive/ql/src/test/queries/positive/input9.q diff --git a/src/test/hive/ql/src/test/queries/positive/input_part1.q b/shark/src/test/hive/ql/src/test/queries/positive/input_part1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input_part1.q rename to shark/src/test/hive/ql/src/test/queries/positive/input_part1.q diff --git a/src/test/hive/ql/src/test/queries/positive/input_testsequencefile.q b/shark/src/test/hive/ql/src/test/queries/positive/input_testsequencefile.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input_testsequencefile.q rename to shark/src/test/hive/ql/src/test/queries/positive/input_testsequencefile.q diff --git a/src/test/hive/ql/src/test/queries/positive/input_testxpath.q b/shark/src/test/hive/ql/src/test/queries/positive/input_testxpath.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input_testxpath.q rename to shark/src/test/hive/ql/src/test/queries/positive/input_testxpath.q diff --git a/src/test/hive/ql/src/test/queries/positive/input_testxpath2.q b/shark/src/test/hive/ql/src/test/queries/positive/input_testxpath2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/input_testxpath2.q rename to shark/src/test/hive/ql/src/test/queries/positive/input_testxpath2.q diff --git a/src/test/hive/ql/src/test/queries/positive/join1.q b/shark/src/test/hive/ql/src/test/queries/positive/join1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/join1.q rename to shark/src/test/hive/ql/src/test/queries/positive/join1.q diff --git a/src/test/hive/ql/src/test/queries/positive/join2.q b/shark/src/test/hive/ql/src/test/queries/positive/join2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/join2.q rename to shark/src/test/hive/ql/src/test/queries/positive/join2.q diff --git a/src/test/hive/ql/src/test/queries/positive/join3.q b/shark/src/test/hive/ql/src/test/queries/positive/join3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/join3.q rename to shark/src/test/hive/ql/src/test/queries/positive/join3.q diff --git a/src/test/hive/ql/src/test/queries/positive/join4.q b/shark/src/test/hive/ql/src/test/queries/positive/join4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/join4.q rename to shark/src/test/hive/ql/src/test/queries/positive/join4.q diff --git a/src/test/hive/ql/src/test/queries/positive/join5.q b/shark/src/test/hive/ql/src/test/queries/positive/join5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/join5.q rename to shark/src/test/hive/ql/src/test/queries/positive/join5.q diff --git a/src/test/hive/ql/src/test/queries/positive/join6.q b/shark/src/test/hive/ql/src/test/queries/positive/join6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/join6.q rename to shark/src/test/hive/ql/src/test/queries/positive/join6.q diff --git a/src/test/hive/ql/src/test/queries/positive/join7.q b/shark/src/test/hive/ql/src/test/queries/positive/join7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/join7.q rename to shark/src/test/hive/ql/src/test/queries/positive/join7.q diff --git a/src/test/hive/ql/src/test/queries/positive/join8.q b/shark/src/test/hive/ql/src/test/queries/positive/join8.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/join8.q rename to shark/src/test/hive/ql/src/test/queries/positive/join8.q diff --git a/src/test/hive/ql/src/test/queries/positive/sample1.q b/shark/src/test/hive/ql/src/test/queries/positive/sample1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/sample1.q rename to shark/src/test/hive/ql/src/test/queries/positive/sample1.q diff --git a/src/test/hive/ql/src/test/queries/positive/sample2.q b/shark/src/test/hive/ql/src/test/queries/positive/sample2.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/sample2.q rename to shark/src/test/hive/ql/src/test/queries/positive/sample2.q diff --git a/src/test/hive/ql/src/test/queries/positive/sample3.q b/shark/src/test/hive/ql/src/test/queries/positive/sample3.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/sample3.q rename to shark/src/test/hive/ql/src/test/queries/positive/sample3.q diff --git a/src/test/hive/ql/src/test/queries/positive/sample4.q b/shark/src/test/hive/ql/src/test/queries/positive/sample4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/sample4.q rename to shark/src/test/hive/ql/src/test/queries/positive/sample4.q diff --git a/src/test/hive/ql/src/test/queries/positive/sample5.q b/shark/src/test/hive/ql/src/test/queries/positive/sample5.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/sample5.q rename to shark/src/test/hive/ql/src/test/queries/positive/sample5.q diff --git a/src/test/hive/ql/src/test/queries/positive/sample6.q b/shark/src/test/hive/ql/src/test/queries/positive/sample6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/sample6.q rename to shark/src/test/hive/ql/src/test/queries/positive/sample6.q diff --git a/src/test/hive/ql/src/test/queries/positive/sample7.q b/shark/src/test/hive/ql/src/test/queries/positive/sample7.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/sample7.q rename to shark/src/test/hive/ql/src/test/queries/positive/sample7.q diff --git a/src/test/hive/ql/src/test/queries/positive/subq.q b/shark/src/test/hive/ql/src/test/queries/positive/subq.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/subq.q rename to shark/src/test/hive/ql/src/test/queries/positive/subq.q diff --git a/src/test/hive/ql/src/test/queries/positive/udf1.q b/shark/src/test/hive/ql/src/test/queries/positive/udf1.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/udf1.q rename to shark/src/test/hive/ql/src/test/queries/positive/udf1.q diff --git a/src/test/hive/ql/src/test/queries/positive/udf4.q b/shark/src/test/hive/ql/src/test/queries/positive/udf4.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/udf4.q rename to shark/src/test/hive/ql/src/test/queries/positive/udf4.q diff --git a/src/test/hive/ql/src/test/queries/positive/udf6.q b/shark/src/test/hive/ql/src/test/queries/positive/udf6.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/udf6.q rename to shark/src/test/hive/ql/src/test/queries/positive/udf6.q diff --git a/src/test/hive/ql/src/test/queries/positive/udf_case.q b/shark/src/test/hive/ql/src/test/queries/positive/udf_case.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/udf_case.q rename to shark/src/test/hive/ql/src/test/queries/positive/udf_case.q diff --git a/src/test/hive/ql/src/test/queries/positive/udf_when.q b/shark/src/test/hive/ql/src/test/queries/positive/udf_when.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/udf_when.q rename to shark/src/test/hive/ql/src/test/queries/positive/udf_when.q diff --git a/src/test/hive/ql/src/test/queries/positive/union.q b/shark/src/test/hive/ql/src/test/queries/positive/union.q similarity index 100% rename from src/test/hive/ql/src/test/queries/positive/union.q rename to shark/src/test/hive/ql/src/test/queries/positive/union.q From 7588a57feb1870c718be645e428d1f2371b9e722 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 13 Feb 2014 15:19:28 -0800 Subject: [PATCH 670/778] Break into 3 major components and move everything into the org.apache.spark.sql package. --- .travis.yml | 2 +- README.md | 30 +- build.sbt | 132 ++++----- .../sql}/catalyst/analysis/Analyzer.scala | 33 +-- .../sql}/catalyst/analysis/Catalog.scala | 20 ++ .../catalyst/analysis/FunctionRegistry.scala | 1 + .../catalyst/analysis/HiveTypeCoercion.scala | 1 + .../sql}/catalyst/analysis/package.scala | 0 .../sql}/catalyst/analysis/unresolved.scala | 5 +- .../spark/sql}/catalyst/dsl/package.scala | 5 +- .../spark/sql}/catalyst/errors/package.scala | 7 +- .../catalyst/expressions/BoundAttribute.scala | 1 + .../sql}/catalyst/expressions/Cast.scala | 1 + .../sql}/catalyst/expressions/Evaluate.scala | 4 +- .../catalyst/expressions/Expression.scala | 1 + .../catalyst/expressions/ImplementedUdf.scala | 1 + .../sql}/catalyst/expressions/Rand.scala | 1 + .../spark/sql}/catalyst/expressions/Row.scala | 5 + .../sql}/catalyst/expressions/ScalaUdf.scala | 1 + .../sql}/catalyst/expressions/SortOrder.scala | 1 + .../catalyst/expressions/WrapDynamic.scala | 3 +- .../catalyst/expressions/aggregates.scala | 109 +++++++ .../catalyst/expressions/arithmetic.scala | 1 + .../catalyst/expressions/complexTypes.scala | 1 + .../catalyst/expressions/generators.scala | 3 +- .../sql}/catalyst/expressions/literals.scala | 1 + .../expressions/namedExpressions.scala | 1 + .../catalyst/expressions/nullFunctions.scala | 1 + .../sql}/catalyst/expressions/package.scala | 1 + .../catalyst/expressions/predicates.scala | 1 + .../expressions/stringOperations.scala | 1 + .../sql}/catalyst/optimizer/Optimizer.scala | 3 +- .../apache/spark/sql/catalyst/package.scala | 15 + .../sql}/catalyst/planning/QueryPlanner.scala | 1 + .../sql}/catalyst/planning/package.scala | 1 + .../sql}/catalyst/planning/patterns.scala | 1 + .../spark/sql}/catalyst/plans/QueryPlan.scala | 1 + .../catalyst/plans/logical/BaseRelation.scala | 3 + .../catalyst/plans/logical/LogicalPlan.scala | 22 ++ .../plans/logical/ScriptTransformation.scala | 1 + .../catalyst/plans/logical/TestRelation.scala | 1 + .../plans/logical/basicOperators.scala | 1 + .../catalyst/plans/logical/partitioning.scala | 1 + .../spark/sql}/catalyst/plans/package.scala | 1 + .../plans/physical/partitioning.scala | 1 + .../spark/sql}/catalyst/plans/shared.scala | 1 + .../spark/sql}/catalyst/rules/Rule.scala | 1 + .../sql}/catalyst/rules/RuleExecutor.scala | 3 +- .../spark/sql}/catalyst/rules/package.scala | 1 + .../spark/sql}/catalyst/trees/TreeNode.scala | 1 + .../spark/sql}/catalyst/trees/package.scala | 1 + .../spark/sql}/catalyst/types/dataTypes.scala | 1 + .../spark/sql}/catalyst/types/package.scala | 2 +- .../spark/sql}/catalyst/util/package.scala | 4 +- .../spark/sql}/catalyst/AnalysisSuite.scala | 1 + .../sql/catalyst/DistributionSuite.scala | 9 +- .../catalyst/ExpressionEvaluationSuite.scala | 1 + .../sql/catalyst}/HiveTypeCoercionSuite.scala | 4 +- .../sql}/catalyst/RuleExecutorSuite.scala | 1 + .../spark/sql}/catalyst/TreeNodeSuite.scala | 1 + .../optimizer/ConstantFoldingSuite.scala | 1 + .../optimizer/FilterPushdownSuite.scala | 1 + .../catalyst/optimizer/OptimizerTest.scala | 1 + .../spark/rdd/SharkPairRDDFunctions.scala | 0 .../apache/spark/sql/SparkSqlContext.scala | 114 ++++++++ .../sql}/examples/SchemaRddExample.scala | 12 +- .../spark/sql}/examples/ViewsExample.scala | 7 +- .../spark/sql}/execution/Exchange.scala | 13 +- .../spark/sql}/execution/Generate.scala | 6 +- .../execution/QueryExecutionException.scala | 3 +- .../spark/sql}/execution/SharkPlan.scala | 11 +- .../spark/sql/execution/SparkStrategies.scala | 119 +++----- .../spark/sql/execution/aggregates.scala | 100 +++++++ .../spark/sql}/execution/basicOperators.scala | 27 +- .../apache/spark/sql}/execution/debug.scala | 8 +- .../apache/spark/sql}/execution/joins.scala | 36 +-- .../apache/spark/sql/execution/package.scala | 14 + .../src}/test/resources/log4j.properties | 0 .../org/apache/spark/sql}/DslQueryTests.scala | 23 +- .../org/apache/spark/sql}/PlannerSuite.scala | 13 +- .../org/apache/spark/sql}/TgfSuite.scala | 14 +- project/build.properties | 2 +- project/plugins.sbt | 15 +- sbt/sbt-launch.jar | Bin 1105722 -> 1132093 bytes .../sql/shark/HiveMetastoreCatalog.scala | 57 +++- .../org/apache/spark/sql/shark/HiveQl.scala | 44 +-- .../sql/shark}/ScriptTransformation.scala | 7 +- .../apache/spark/sql/shark/SharkContext.scala | 270 ++++++++++++++++++ .../spark/sql/shark/SharkStrategies.scala | 85 ++++++ .../apache/spark/sql/shark}/TableReader.scala | 23 +- .../apache/spark/sql/shark}/TestShark.scala | 78 +++-- .../spark/sql/shark}/hiveOperators.scala | 14 +- .../org/apache/spark/sql/shark/hiveUdfs.scala | 63 +++- shark/src/test/resources/log4j.properties | 47 +++ .../execution/BigDataBenchmarkSuite.scala | 5 +- .../shark/execution/ConcurrentHiveSuite.scala | 21 ++ .../shark}/execution/HiveComparisonTest.scala | 44 +-- .../execution/HiveCompatibilitySuite.scala | 6 +- .../shark}/execution/HiveQueryFileTest.scala | 5 +- .../sql/shark/execution/HiveQuerySuite.scala | 9 +- .../execution/HiveResolutionSuite.scala | 3 +- .../sql/shark}/execution/HiveSerDeSuite.scala | 4 +- .../execution/HiveTypeCoercionSuite.scala | 6 +- .../execution/PartitionPruningSuite.scala | 6 +- .../catalyst/execution/SharkContext.scala | 115 -------- .../scala/catalyst/execution/SharkEnv.scala | 43 --- .../catalyst/execution/SharkInstance.scala | 192 ------------- .../scala/catalyst/execution/aggregates.scala | 231 --------------- .../scala/catalyst/execution/package.scala | 34 --- .../scala/catalyst/frontend/package.scala | 7 - src/main/scala/catalyst/package.scala | 11 - .../execution/ConcurrentHiveTests.scala | 18 -- .../execution/HiveAstTransformSuite.scala | 80 ------ 113 files changed, 1330 insertions(+), 1198 deletions(-) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/analysis/Analyzer.scala (82%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/analysis/Catalog.scala (51%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/analysis/FunctionRegistry.scala (95%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/analysis/HiveTypeCoercion.scala (99%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/analysis/package.scala (100%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/analysis/unresolved.scala (97%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/dsl/package.scala (96%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/errors/package.scala (76%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/BoundAttribute.scala (98%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/Cast.scala (91%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/Evaluate.scala (98%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/Expression.scala (98%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/ImplementedUdf.scala (85%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/Rand.scala (88%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/Row.scala (97%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/ScalaUdf.scala (94%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/SortOrder.scala (95%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/WrapDynamic.scala (89%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/aggregates.scala (57%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/arithmetic.scala (97%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/complexTypes.scala (98%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/generators.scala (97%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/literals.scala (97%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/namedExpressions.scala (99%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/nullFunctions.scala (96%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/package.scala (98%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/predicates.scala (99%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/expressions/stringOperations.scala (90%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/optimizer/Optimizer.scala (98%) create mode 100644 catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/planning/QueryPlanner.scala (98%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/planning/package.scala (83%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/planning/patterns.scala (97%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/plans/QueryPlan.scala (99%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/plans/logical/BaseRelation.scala (68%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/plans/logical/LogicalPlan.scala (82%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/plans/logical/ScriptTransformation.scala (95%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/plans/logical/TestRelation.scala (97%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/plans/logical/basicOperators.scala (99%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/plans/logical/partitioning.scala (96%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/plans/package.scala (84%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/plans/physical/partitioning.scala (99%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/plans/shared.scala (88%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/rules/Rule.scala (92%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/rules/RuleExecutor.scala (96%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/rules/package.scala (81%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/trees/TreeNode.scala (99%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/trees/package.scala (96%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/types/dataTypes.scala (99%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/types/package.scala (85%) rename {src/main/scala => catalyst/src/main/scala/org/apache/spark/sql}/catalyst/util/package.scala (98%) rename {src/test/scala => catalyst/src/test/scala/org/apache/spark/sql}/catalyst/AnalysisSuite.scala (93%) rename src/test/scala/catalyst/execution/DistributionTests.scala => catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala (96%) rename {src/test/scala => catalyst/src/test/scala/org/apache/spark/sql}/catalyst/ExpressionEvaluationSuite.scala (98%) rename {src/test/scala/catalyst/analysis => catalyst/src/test/scala/org/apache/spark/sql/catalyst}/HiveTypeCoercionSuite.scala (97%) rename {src/test/scala => catalyst/src/test/scala/org/apache/spark/sql}/catalyst/RuleExecutorSuite.scala (97%) rename {src/test/scala => catalyst/src/test/scala/org/apache/spark/sql}/catalyst/TreeNodeSuite.scala (98%) rename {src/test/scala => catalyst/src/test/scala/org/apache/spark/sql}/catalyst/optimizer/ConstantFoldingSuite.scala (99%) rename {src/test/scala => catalyst/src/test/scala/org/apache/spark/sql}/catalyst/optimizer/FilterPushdownSuite.scala (99%) rename {src/test/scala => catalyst/src/test/scala/org/apache/spark/sql}/catalyst/optimizer/OptimizerTest.scala (97%) rename {src => core/src}/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala (100%) create mode 100644 core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala rename {src/main/scala/catalyst => core/src/main/scala/org/apache/spark/sql}/examples/SchemaRddExample.scala (83%) rename {src/main/scala/catalyst => core/src/main/scala/org/apache/spark/sql}/examples/ViewsExample.scala (92%) rename {src/main/scala/catalyst => core/src/main/scala/org/apache/spark/sql}/execution/Exchange.scala (94%) rename {src/main/scala/catalyst => core/src/main/scala/org/apache/spark/sql}/execution/Generate.scala (96%) rename {src/main/scala/catalyst => core/src/main/scala/org/apache/spark/sql}/execution/QueryExecutionException.scala (61%) rename {src/main/scala/catalyst => core/src/main/scala/org/apache/spark/sql}/execution/SharkPlan.scala (76%) rename src/main/scala/catalyst/execution/PlanningStrategies.scala => core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala (64%) create mode 100644 core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala rename {src/main/scala/catalyst => core/src/main/scala/org/apache/spark/sql}/execution/basicOperators.scala (70%) rename {src/main/scala/catalyst => core/src/main/scala/org/apache/spark/sql}/execution/debug.scala (72%) rename {src/main/scala/catalyst => core/src/main/scala/org/apache/spark/sql}/execution/joins.scala (84%) create mode 100644 core/src/main/scala/org/apache/spark/sql/execution/package.scala rename {src => core/src}/test/resources/log4j.properties (100%) rename {src/test/scala/catalyst/execution => core/src/test/scala/org/apache/spark/sql}/DslQueryTests.scala (95%) rename {src/test/scala/catalyst/execution => core/src/test/scala/org/apache/spark/sql}/PlannerSuite.scala (78%) rename {src/test/scala/catalyst/execution => core/src/test/scala/org/apache/spark/sql}/TgfSuite.scala (88%) rename src/main/scala/catalyst/execution/MetastoreCatalog.scala => shark/src/main/scala/org/apache/spark/sql/shark/HiveMetastoreCatalog.scala (75%) rename src/main/scala/catalyst/frontend/Hive.scala => shark/src/main/scala/org/apache/spark/sql/shark/HiveQl.scala (97%) rename {src/main/scala/catalyst/execution => shark/src/main/scala/org/apache/spark/sql/shark}/ScriptTransformation.scala (93%) create mode 100644 shark/src/main/scala/org/apache/spark/sql/shark/SharkContext.scala create mode 100644 shark/src/main/scala/org/apache/spark/sql/shark/SharkStrategies.scala rename {src/main/scala/catalyst/execution => shark/src/main/scala/org/apache/spark/sql/shark}/TableReader.scala (94%) rename {src/main/scala/catalyst/execution => shark/src/main/scala/org/apache/spark/sql/shark}/TestShark.scala (89%) rename {src/main/scala/catalyst/execution => shark/src/main/scala/org/apache/spark/sql/shark}/hiveOperators.scala (97%) rename src/main/scala/catalyst/execution/FunctionRegistry.scala => shark/src/main/scala/org/apache/spark/sql/shark/hiveUdfs.scala (88%) create mode 100644 shark/src/test/resources/log4j.properties rename src/test/scala/catalyst/execution/BigDataBenchmark.scala => shark/src/test/scala/org/apache/spark/sql/shark/execution/BigDataBenchmarkSuite.scala (97%) create mode 100644 shark/src/test/scala/org/apache/spark/sql/shark/execution/ConcurrentHiveSuite.scala rename {src/test/scala/catalyst => shark/src/test/scala/org/apache/spark/sql/shark}/execution/HiveComparisonTest.scala (92%) rename src/test/scala/catalyst/execution/HiveCompatibility.scala => shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveCompatibilitySuite.scala (99%) rename {src/test/scala/catalyst => shark/src/test/scala/org/apache/spark/sql/shark}/execution/HiveQueryFileTest.scala (96%) rename src/test/scala/catalyst/execution/HiveQueryTests.scala => shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveQuerySuite.scala (94%) rename {src/test/scala/catalyst => shark/src/test/scala/org/apache/spark/sql/shark}/execution/HiveResolutionSuite.scala (97%) rename {src/test/scala/catalyst => shark/src/test/scala/org/apache/spark/sql/shark}/execution/HiveSerDeSuite.scala (85%) rename src/test/scala/catalyst/execution/HiveTypeCoersionSuite.scala => shark/src/test/scala/org/apache/spark/sql/shark/execution/HiveTypeCoercionSuite.scala (69%) rename {src/test/scala/catalyst => shark/src/test/scala/org/apache/spark/sql/shark}/execution/PartitionPruningSuite.scala (92%) delete mode 100644 src/main/scala/catalyst/execution/SharkContext.scala delete mode 100644 src/main/scala/catalyst/execution/SharkEnv.scala delete mode 100644 src/main/scala/catalyst/execution/SharkInstance.scala delete mode 100644 src/main/scala/catalyst/execution/aggregates.scala delete mode 100644 src/main/scala/catalyst/execution/package.scala delete mode 100644 src/main/scala/catalyst/frontend/package.scala delete mode 100644 src/main/scala/catalyst/package.scala delete mode 100644 src/test/scala/catalyst/execution/ConcurrentHiveTests.scala delete mode 100644 src/test/scala/catalyst/execution/HiveAstTransformSuite.scala diff --git a/.travis.yml b/.travis.yml index 987c66e58497c..aac8714b0cbfb 100644 --- a/.travis.yml +++ b/.travis.yml @@ -9,4 +9,4 @@ - $HOME/.ivy2 - $HOME/.sbt script: - - "GIT_AUTHOR_NAME=\"Michael Armbrust\" GIT_AUTHOR_EMAIL=\"michael@databricks.com\" GIT_COMMITTER_NAME=\"Michael Armbrust\" GIT_COMMITTER_EMAIL=\"michael@databricks.com\" sbt ++$TRAVIS_SCALA_VERSION 'set scalacOptions += \"-Xfatal-warnings\"' test:compile scalastyle test ghpages-push-site" + - "GIT_AUTHOR_NAME=\"Michael Armbrust\" GIT_AUTHOR_EMAIL=\"michael@databricks.com\" GIT_COMMITTER_NAME=\"Michael Armbrust\" GIT_COMMITTER_EMAIL=\"michael@databricks.com\" sbt ++$TRAVIS_SCALA_VERSION 'set scalacOptions += \"-Xfatal-warnings\"' test:compile test scalastyle" diff --git a/README.md b/README.md index 4e3fb4113992a..503235fb4ccb9 100644 --- a/README.md +++ b/README.md @@ -61,30 +61,36 @@ Welcome to Scala version 2.10.3 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_4 Type in expressions to have them evaluated. Type :help for more information. -scala> val query = "SELECT * FROM (SELECT * FROM src) a".q -query: catalyst.execution.TestShark.SharkSqlQuery = +scala> scala> val query = sql("SELECT * FROM (SELECT * FROM src) a") +query: org.apache.spark.sql.ExecutedQuery = SELECT * FROM (SELECT * FROM src) a -== Logical Plan == -Project {key#0,value#1} - Subquery a - Project {key#0,value#1} - MetastoreRelation default, src, None +=== Query Plan === +Project [key#6:0.0,value#7:0.1] + HiveTableScan [key#6,value#7], (MetastoreRelation default, src, None), None +``` -== Physical Plan == -Project {key#0,value#1} - HiveTableScan {key#0,value#1}, (MetastoreRelation default, src, None) +Query results are RDDs and can be operated as such. +``` +scala> query.collect() +res8: Array[org.apache.spark.sql.execution.Row] = Array([238,val_238], [86,val_86], [311,val_311]... +``` + +You can also build further queries on top of these RDDs using the query DSL. +``` +scala> query.where('key === 100).toRdd.collect() +res11: Array[org.apache.spark.sql.execution.Row] = Array([100,val_100], [100,val_100]) ``` From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](http://databricks.github.io/catalyst/latest/api/#catalyst.trees.TreeNode) objects. ```scala -scala> query.optimizedPlan +scala> query.logicalPlan res1: catalyst.plans.logical.LogicalPlan = Project {key#0,value#1} Project {key#0,value#1} MetastoreRelation default, src, None -scala> res0.optimizedPlan transform { +scala> query.logicalPlan transform { | case Project(projectList, child) if projectList == child.output => child | } res2: catalyst.plans.logical.LogicalPlan = diff --git a/build.sbt b/build.sbt index 913d7b80784f1..786af585b3b7b 100644 --- a/build.sbt +++ b/build.sbt @@ -1,82 +1,52 @@ -import AssemblyKeys._ // put this at the top of the file -name := "catalyst" - -organization := "com.databricks" - -version := "0.1-SNAPSHOT" - -scalaVersion := "2.10.3" - -scalacOptions ++= Seq("-deprecation", "-feature", "-unchecked") - -resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" - -// TODO: Remove when Spark 0.9.0 is released for real. -resolvers += "SparkStaging" at "https://repository.apache.org/content/repositories/orgapachespark-1006/" - -libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" - -// Hive 0.10.0 relies on a weird version of jdo that is not published anywhere... Remove when we upgrade to 0.11.0 -libraryDependencies += "javax.jdo" % "jdo2-api" % "2.3-ec" from "http://www.datanucleus.org/downloads/maven2/javax/jdo/jdo2-api/2.3-ec/jdo2-api-2.3-ec.jar" - -libraryDependencies ++= Seq( - "org.apache.hadoop" % "hadoop-client" % "1.0.4", - "org.scalatest" %% "scalatest" % "1.9.1" % "test", - //"net.hydromatic" % "optiq-core" % "0.4.16-SNAPSHOT", - "org.apache.hive" % "hive-metastore" % "0.12.0", - "org.apache.hive" % "hive-exec" % "0.12.0", - "org.apache.hive" % "hive-serde" % "0.12.0", - "com.typesafe" %% "scalalogging-slf4j" % "1.0.1") - -org.scalastyle.sbt.ScalastylePlugin.Settings - -// Multiple queries rely on the TestShark singleton. See comments there for more details. -parallelExecution in Test := false - -resolvers ++= Seq( - // For Optiq - "Conjars Repository" at "http://conjars.org/repo/", - // For jdo-2 required by Hive < 0.12.0 - "Datanucleus Repository" at "http://www.datanucleus.org/downloads/maven2") - -resolvers += "Databees" at "http://repository-databricks.forge.cloudbees.com/snapshot/" - -initialCommands in console := """ -import catalyst.analysis._ -import catalyst.dsl._ -import catalyst.errors._ -import catalyst.expressions._ -import catalyst.frontend._ -import catalyst.plans.logical._ -import catalyst.rules._ -import catalyst.types._ -import catalyst.util._ -import catalyst.execution.TestShark._""" - -site.settings - -ghpages.settings - -git.remoteRepo := "git@github.com:databricks/catalyst.git" - -site.settings - -site.includeScaladoc() - -assemblySettings - -test in assembly := {} - -mergeStrategy in assembly := { - case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard - case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard - case "log4j.properties" => MergeStrategy.discard - case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines - case "reference.conf" => MergeStrategy.concat - case _ => MergeStrategy.first -} - -scalacOptions in (Compile, doc) <++= (baseDirectory) map { - bd => Seq("-sourcepath", bd.getAbsolutePath, "-doc-source-url","https://github.com/databricks/catalyst/blob/master/€{FILE_PATH}.scala") -} \ No newline at end of file +lazy val catalyst = Project("catalyst", file("catalyst"), settings = catalystSettings) +lazy val core = Project("core", file("core"), settings = coreSettings).dependsOn(catalyst) +lazy val shark = Project("shark", file("shark"), settings = sharkSettings).dependsOn(core) + +def sharedSettings = Defaults.defaultSettings ++ Seq( + organization := "org.apache.spark.sql", + version := "0.1-SNAPSHOT", + scalaVersion := "2.10.3", + scalacOptions ++= Seq("-deprecation", "-feature", "-unchecked"), + // Common Dependencies. + libraryDependencies ++= Seq( + "org.scalatest" %% "scalatest" % "1.9.1" % "test", + "com.typesafe" %% "scalalogging-slf4j" % "1.0.1") +) ++ org.scalastyle.sbt.ScalastylePlugin.Settings + +def catalystSettings = sharedSettings ++ Seq( + name := "catalyst", + // The mechanics of rewriting expression ids to compare trees in some test cases makes + // assumptions about the the expression ids being contiguious. Running tests in parallel breaks + // this non-deterministically. TODO: FIX THIS. + parallelExecution in Test := false +) + +def coreSettings = sharedSettings ++ Seq( + name := "core", + libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" +) + +def sharkSettings = sharedSettings ++ Seq( + name := "shark", + libraryDependencies ++= Seq( + "org.apache.hadoop" % "hadoop-client" % "1.0.4", + "org.apache.hive" % "hive-metastore" % "0.12.0", + "org.apache.hive" % "hive-exec" % "0.12.0", + "org.apache.hive" % "hive-serde" % "0.12.0"), + // Multiple queries rely on the TestShark singleton. See comments there for more details. + parallelExecution in Test := false, + initialCommands in console := + """ + |import org.apache.spark.sql.catalyst.analysis._ + |import org.apache.spark.sql.catalyst.dsl._ + |import org.apache.spark.sql.catalyst.errors._ + |import org.apache.spark.sql.catalyst.expressions._ + |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.catalyst.util._ + |import org.apache.spark.sql.execution + |import org.apache.spark.sql.shark._ + |import org.apache.spark.sql.shark.TestShark._""".stripMargin +) \ No newline at end of file diff --git a/src/main/scala/catalyst/analysis/Analyzer.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala similarity index 82% rename from src/main/scala/catalyst/analysis/Analyzer.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 98414b2648638..185a62ab617b7 100644 --- a/src/main/scala/catalyst/analysis/Analyzer.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1,10 +1,10 @@ +package org.apache.spark.sql package catalyst package analysis import expressions._ import plans.logical._ import rules._ -import catalyst.execution.MetastoreRelation /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing @@ -36,7 +36,6 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool StarExpansion :: ResolveFunctions :: GlobalAggregates :: - PreInsertionCasts :: typeCoercionRules :_*) ) @@ -165,34 +164,4 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool protected def containsStar(exprs: Seq[Expression]): Boolean = exprs.collect { case _: Star => true }.nonEmpty } - - /** - * Casts input data to correct data types according to table definition before inserting into - * that table. - */ - object PreInsertionCasts extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan.transform { - // Wait until children are resolved - case p: LogicalPlan if !p.childrenResolved => p - - case p @ InsertIntoTable(table: MetastoreRelation, _, child) => - val childOutputDataTypes = child.output.map(_.dataType) - // Only check attributes, not partitionKeys since they are always strings. - // TODO: Fully support inserting into partitioned tables. - val tableOutputDataTypes = table.attributes.map(_.dataType) - - if (childOutputDataTypes == tableOutputDataTypes) { - p - } else { - // Only do the casting when child output data types differ from table output data types. - val castedChildOutput = child.output.zip(table.output).map { - case (input, table) if input.dataType != table.dataType => - Alias(Cast(input, table.dataType), input.name)() - case (input, _) => input - } - - p.copy(child = Project(castedChildOutput, child)) - } - } - } } diff --git a/src/main/scala/catalyst/analysis/Catalog.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala similarity index 51% rename from src/main/scala/catalyst/analysis/Catalog.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index a1cb32feef8c3..55e50f774535c 100644 --- a/src/main/scala/catalyst/analysis/Catalog.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -1,7 +1,9 @@ +package org.apache.spark.sql package catalyst package analysis import plans.logical.LogicalPlan +import scala.collection.mutable /** * An interface for looking up relations by name. Used by an [[Analyzer]]. @@ -13,6 +15,24 @@ trait Catalog { alias: Option[String] = None): LogicalPlan } +trait OverrideCatalog extends Catalog { + + // TODO: This doesn't work when the database changes... + val overrides = new mutable.HashMap[(Option[String],String), LogicalPlan]() + + abstract override def lookupRelation( + databaseName: Option[String], + tableName: String, + alias: Option[String] = None): LogicalPlan = { + + overrides.get((databaseName, tableName)) + .getOrElse(super.lookupRelation(databaseName, tableName, alias)) + } + + def overrideTable(databaseName: Option[String], tableName: String, plan: LogicalPlan) = + overrides.put((databaseName, tableName), plan) +} + /** * A trivial catalog that returns an error when a relation is requested. Used for testing when all * relations are already filled in and the analyser needs only to resolve attribute references. diff --git a/src/main/scala/catalyst/analysis/FunctionRegistry.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala similarity index 95% rename from src/main/scala/catalyst/analysis/FunctionRegistry.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 9fef07b363b9f..c18e4f5790203 100644 --- a/src/main/scala/catalyst/analysis/FunctionRegistry.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package analysis diff --git a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala similarity index 99% rename from src/main/scala/catalyst/analysis/HiveTypeCoercion.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index ce2f660a7c778..cf46ca3a95a2a 100644 --- a/src/main/scala/catalyst/analysis/HiveTypeCoercion.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package analysis diff --git a/src/main/scala/catalyst/analysis/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala similarity index 100% rename from src/main/scala/catalyst/analysis/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala diff --git a/src/main/scala/catalyst/analysis/unresolved.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala similarity index 97% rename from src/main/scala/catalyst/analysis/unresolved.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index af66ac457ec16..71b0cc90a2556 100644 --- a/src/main/scala/catalyst/analysis/unresolved.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package analysis @@ -10,14 +11,14 @@ import trees.TreeNode * resolved. */ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: String) extends - errors.TreeNodeException(tree, s"Invalid call to $function on unresolved object") + errors.TreeNodeException(tree, s"Invalid call to $function on unresolved object", null) /** * Holds the name of a relation that has yet to be looked up in a [[Catalog]]. */ case class UnresolvedRelation( databaseName: Option[String], - name: String, + tableName: String, alias: Option[String] = None) extends BaseRelation { def output = Nil override lazy val resolved = false diff --git a/src/main/scala/catalyst/dsl/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala similarity index 96% rename from src/main/scala/catalyst/dsl/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 68bdeb16b65fc..6f81e92058bbb 100644 --- a/src/main/scala/catalyst/dsl/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst import scala.language.implicitConversions @@ -118,10 +119,10 @@ package object dsl { def unionAll(otherPlan: LogicalPlan) = Union(plan, otherPlan) - def filter[T1](arg1: Symbol)(udf: (T1) => Boolean) = + def sfilter[T1](arg1: Symbol)(udf: (T1) => Boolean) = Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), plan) - def filter(dynamicUdf: (DynamicRow) => Boolean) = + def sfilter(dynamicUdf: (DynamicRow) => Boolean) = Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(plan.output))), plan) def sample( diff --git a/src/main/scala/catalyst/errors/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala similarity index 76% rename from src/main/scala/catalyst/errors/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala index 953e37d7f295e..a98cebb7fb4fa 100644 --- a/src/main/scala/catalyst/errors/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst import trees._ @@ -8,7 +9,11 @@ import trees._ package object errors { class TreeNodeException[TreeType <: TreeNode[_]] - (tree: TreeType, msg: String, cause: Throwable = null) extends Exception(msg, cause) { + (tree: TreeType, msg: String, cause: Throwable) extends Exception(msg, cause) { + + // Yes, this is the same as a default parameter, but... those don't seem to work with SBT + // external project dependencies for some reason. + def this(tree: TreeType, msg: String) = this(tree, msg, null) override def getMessage: String = { val treeString = tree.toString diff --git a/src/main/scala/catalyst/expressions/BoundAttribute.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala similarity index 98% rename from src/main/scala/catalyst/expressions/BoundAttribute.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index eabb086750750..60a4cc0882c45 100644 --- a/src/main/scala/catalyst/expressions/BoundAttribute.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/Cast.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala similarity index 91% rename from src/main/scala/catalyst/expressions/Cast.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 0fd0aca8f44cf..bd42c662a041f 100644 --- a/src/main/scala/catalyst/expressions/Cast.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala similarity index 98% rename from src/main/scala/catalyst/expressions/Evaluate.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala index 209f191381485..e9948c3ae88e2 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Evaluate.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions @@ -14,9 +15,6 @@ object Evaluate extends Logging { /** * A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type * and do any casting necessary of child evaluation. - * - * Instead of matching here we could consider pushing the appropriate Fractional/Integral type - * into the type objects themselves. */ @inline def n1(e: Expression, f: ((Numeric[Any], Any) => Any)): Any = { diff --git a/src/main/scala/catalyst/expressions/Expression.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala similarity index 98% rename from src/main/scala/catalyst/expressions/Expression.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index a75c1f3bf4ee8..d561464160af5 100644 --- a/src/main/scala/catalyst/expressions/Expression.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/ImplementedUdf.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala similarity index 85% rename from src/main/scala/catalyst/expressions/ImplementedUdf.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala index 4a287e0e0e74d..4322326c0fd0e 100644 --- a/src/main/scala/catalyst/expressions/ImplementedUdf.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ImplementedUdf.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/Rand.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala similarity index 88% rename from src/main/scala/catalyst/expressions/Rand.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala index bec89e9a1da5b..602b1d76f4610 100644 --- a/src/main/scala/catalyst/expressions/Rand.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/Row.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala similarity index 97% rename from src/main/scala/catalyst/expressions/Row.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 432d79fb5862c..1421e17d73371 100644 --- a/src/main/scala/catalyst/expressions/Row.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions @@ -21,6 +22,9 @@ abstract class Row extends Seq[Any] with Serializable { def getBoolean(i: Int): Boolean def getShort(i: Int): Short def getByte(i: Int): Byte + + override def toString() = + s"[${this.mkString(",")}]" } /** @@ -48,6 +52,7 @@ class GenericRow(input: Seq[Any]) extends Row { val values = input.toIndexedSeq def iterator = values.iterator + def length = values.length def apply(i: Int) = values(i) diff --git a/src/main/scala/catalyst/expressions/ScalaUdf.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala similarity index 94% rename from src/main/scala/catalyst/expressions/ScalaUdf.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 05bde2133063a..5698fd8ad8d01 100644 --- a/src/main/scala/catalyst/expressions/ScalaUdf.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/SortOrder.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala similarity index 95% rename from src/main/scala/catalyst/expressions/SortOrder.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index 04a70bc2104ca..c1c10cc3207a4 100644 --- a/src/main/scala/catalyst/expressions/SortOrder.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/WrapDynamic.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala similarity index 89% rename from src/main/scala/catalyst/expressions/WrapDynamic.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index 64150901030f7..4ada044c390f7 100644 --- a/src/main/scala/catalyst/expressions/WrapDynamic.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions @@ -17,7 +18,7 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression with Impleme } class DynamicRow(val schema: Seq[Attribute], values: Seq[Any]) - extends GenericRow(values) with Dynamic { + extends GenericRow(values) with Dynamic { def selectDynamic(attributeName: String): String = { val ordinal = schema.indexWhere(_.name == attributeName) diff --git a/src/main/scala/catalyst/expressions/aggregates.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala similarity index 57% rename from src/main/scala/catalyst/expressions/aggregates.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 3837149095cb3..1708fed744849 100644 --- a/src/main/scala/catalyst/expressions/aggregates.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions @@ -6,6 +7,11 @@ import catalyst.types._ abstract class AggregateExpression extends Expression { self: Product => + /** + * Creates a new instance that can be used to compute this aggregate expression for a group + * of input rows/ + */ + def newInstance: AggregateFunction } /** @@ -49,6 +55,9 @@ abstract class AggregateFunction def apply(input: Seq[Row]): Unit def result: Any + + // Do we really need this? + def newInstance = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -61,6 +70,8 @@ case class Count(child: Expression) extends PartialAggregate with trees.UnaryNod val partialCount = Alias(Count(child), "PartialCount")() SplitEvaluation(Sum(partialCount.toAttribute), partialCount :: Nil) } + + override def newInstance = new CountFunction(child, this) } case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { @@ -69,6 +80,7 @@ case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpressi def nullable = false def dataType = IntegerType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")}})" + override def newInstance = new CountDistinctFunction(expressions, this) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -87,6 +99,8 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN Divide(castedSum, castedCount), partialCount :: partialSum :: Nil) } + + override def newInstance = new AverageFunction(child, this) } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -101,6 +115,8 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ Sum(partialSum.toAttribute), partialSum :: Nil) } + + override def newInstance = new SumFunction(child, this) } case class SumDistinct(child: Expression) @@ -110,6 +126,8 @@ case class SumDistinct(child: Expression) def nullable = false def dataType = child.dataType override def toString = s"SUM(DISTINCT $child)" + + override def newInstance = new SumDistinctFunction(child, this) } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -124,4 +142,95 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod First(partialFirst.toAttribute), partialFirst :: Nil) } + override def newInstance = new FirstFunction(child, this) +} + +case class AverageFunction(expr: Expression, base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + var count: Long = _ + var sum: Long = _ + + def result: Any = sum.toDouble / count.toDouble + + def apply(input: Seq[Row]): Unit = { + count += 1 + // TODO: Support all types here... + sum += Evaluate(expr, input).asInstanceOf[Int] + } +} + +case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var count: Int = _ + + def apply(input: Seq[Row]): Unit = { + val evaluatedExpr = expr.map(Evaluate(_, input)) + if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { + count += 1 + } + } + + def result: Any = count +} + +case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var sum = Evaluate(Cast(Literal(0), expr.dataType), Nil) + + def apply(input: Seq[Row]): Unit = + sum = Evaluate(Add(Literal(sum), expr), input) + + def result: Any = sum +} + +case class SumDistinctFunction(expr: Expression, base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new scala.collection.mutable.HashSet[Any]() + + def apply(input: Seq[Row]): Unit = { + val evaluatedExpr = Evaluate(expr, input) + if (evaluatedExpr != null) { + seen += evaluatedExpr + } + } + + def result: Any = + seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus) +} + +case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new scala.collection.mutable.HashSet[Any]() + + def apply(input: Seq[Row]): Unit = { + val evaluatedExpr = expr.map(Evaluate(_, input)) + if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { + seen += evaluatedExpr + } + } + + def result: Any = seen.size +} + +case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var result: Any = null + + def apply(input: Seq[Row]): Unit = { + if (result == null) { + result = Evaluate(expr, input) + } + } } diff --git a/src/main/scala/catalyst/expressions/arithmetic.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala similarity index 97% rename from src/main/scala/catalyst/expressions/arithmetic.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index a82dfa89fbb1d..8c93bc5fafc7e 100644 --- a/src/main/scala/catalyst/expressions/arithmetic.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/complexTypes.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala similarity index 98% rename from src/main/scala/catalyst/expressions/complexTypes.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 2d7904f558d48..57a84f5ac7737 100644 --- a/src/main/scala/catalyst/expressions/complexTypes.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/generators.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala similarity index 97% rename from src/main/scala/catalyst/expressions/generators.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index c78ddf930d196..96971d5b26373 100644 --- a/src/main/scala/catalyst/expressions/generators.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions @@ -71,7 +72,7 @@ case class Explode(attributeNames: Seq[String], child: Expression) // TODO: Move this pattern into Generator. protected def makeOutput() = - if(attributeNames.size == elementTypes.size) { + if (attributeNames.size == elementTypes.size) { attributeNames.zip(elementTypes).map { case (n, t) => AttributeReference(n, t, nullable = true)() } diff --git a/src/main/scala/catalyst/expressions/literals.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala similarity index 97% rename from src/main/scala/catalyst/expressions/literals.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 2277bef2dbf72..3c70d1328b1ac 100644 --- a/src/main/scala/catalyst/expressions/literals.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/namedExpressions.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala similarity index 99% rename from src/main/scala/catalyst/expressions/namedExpressions.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 40766d999b509..3886926882ee8 100644 --- a/src/main/scala/catalyst/expressions/namedExpressions.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/nullFunctions.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala similarity index 96% rename from src/main/scala/catalyst/expressions/nullFunctions.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index 4732d8952248e..caec559b31bc4 100644 --- a/src/main/scala/catalyst/expressions/nullFunctions.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala similarity index 98% rename from src/main/scala/catalyst/expressions/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index ea9dd6169179c..3ea2b7ed85ea3 100644 --- a/src/main/scala/catalyst/expressions/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst /** diff --git a/src/main/scala/catalyst/expressions/predicates.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala similarity index 99% rename from src/main/scala/catalyst/expressions/predicates.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 457368c518139..cc69667e6b766 100644 --- a/src/main/scala/catalyst/expressions/predicates.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/expressions/stringOperations.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala similarity index 90% rename from src/main/scala/catalyst/expressions/stringOperations.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index d836c80a334f3..843c6cc30f659 100644 --- a/src/main/scala/catalyst/expressions/stringOperations.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/main/scala/catalyst/optimizer/Optimizer.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala similarity index 98% rename from src/main/scala/catalyst/optimizer/Optimizer.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index e0d25dfd5f4e6..8f4a6f51be814 100644 --- a/src/main/scala/catalyst/optimizer/Optimizer.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package optimizer @@ -7,7 +8,7 @@ import catalyst.rules._ import catalyst.types.BooleanType import catalyst.plans.Inner -object Optimize extends RuleExecutor[LogicalPlan] { +object Optimizer extends RuleExecutor[LogicalPlan] { val batches = Batch("Subqueries", Once, EliminateSubqueries) :: diff --git a/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala new file mode 100644 index 0000000000000..de4f4d1c24769 --- /dev/null +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala @@ -0,0 +1,15 @@ +package org.apache.spark + +/** + * Allows the execution of relational queries, including those expressed in SQL using Spark. + * + * Note that this package is located in catalyst instead of in core so that all subprojects can + * inherit the settings from this package object. + */ +package object sql { + + protected[sql] def Logger(name: String) = + com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger(name)) + + protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging +} diff --git a/src/main/scala/catalyst/planning/QueryPlanner.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala similarity index 98% rename from src/main/scala/catalyst/planning/QueryPlanner.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index b8bbc32b007d9..fc319349c9da8 100644 --- a/src/main/scala/catalyst/planning/QueryPlanner.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package planning diff --git a/src/main/scala/catalyst/planning/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala similarity index 83% rename from src/main/scala/catalyst/planning/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala index c3a21afac2148..22a15c6bd51e3 100644 --- a/src/main/scala/catalyst/planning/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst /** diff --git a/src/main/scala/catalyst/planning/patterns.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala similarity index 97% rename from src/main/scala/catalyst/planning/patterns.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index ffed9f44198de..0ae0ea696fd9a 100644 --- a/src/main/scala/catalyst/planning/patterns.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package planning diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala similarity index 99% rename from src/main/scala/catalyst/plans/QueryPlan.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 9ada0be72b8ff..d12b05c0ac3df 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package plans diff --git a/src/main/scala/catalyst/plans/logical/BaseRelation.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala similarity index 68% rename from src/main/scala/catalyst/plans/logical/BaseRelation.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala index 16788affb3cf6..3c7b40d197f53 100644 --- a/src/main/scala/catalyst/plans/logical/BaseRelation.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala @@ -1,7 +1,10 @@ +package org.apache.spark.sql package catalyst package plans package logical abstract class BaseRelation extends LeafNode { self: Product => + + def tableName: String } diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala similarity index 82% rename from src/main/scala/catalyst/plans/logical/LogicalPlan.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index a2350d14dc6cd..dfb2db44f8843 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package plans package logical @@ -78,6 +79,27 @@ abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { def references = Set.empty } +/** + * A logical node that represents a non-query command to be executed by the system. For example, + * commands can be used by parsers to represent DDL operations. + */ +abstract class Command extends LeafNode { + self: Product => + def output = Seq.empty +} + +/** + * Returned for commands supported by a given parser, but not catalyst. In general these are DDL + * commands that are passed directly to another system. + */ +case class NativeCommand(cmd: String) extends Command + +/** + * Returned by a parser when the users only wants to see what query plan would be executed, without + * actually performing the execution. + */ +case class ExplainCommand(plan: LogicalPlan) extends Command + /** * A logical plan node with single child. */ diff --git a/src/main/scala/catalyst/plans/logical/ScriptTransformation.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala similarity index 95% rename from src/main/scala/catalyst/plans/logical/ScriptTransformation.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index f69d81d2991f2..0d0b86ae16171 100644 --- a/src/main/scala/catalyst/plans/logical/ScriptTransformation.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package plans package logical diff --git a/src/main/scala/catalyst/plans/logical/TestRelation.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala similarity index 97% rename from src/main/scala/catalyst/plans/logical/TestRelation.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala index 88535680fa075..1e1ffa584b9f8 100644 --- a/src/main/scala/catalyst/plans/logical/TestRelation.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package plans package logical diff --git a/src/main/scala/catalyst/plans/logical/basicOperators.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala similarity index 99% rename from src/main/scala/catalyst/plans/logical/basicOperators.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index bc9928f6ea385..69b85ec553662 100644 --- a/src/main/scala/catalyst/plans/logical/basicOperators.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package plans package logical diff --git a/src/main/scala/catalyst/plans/logical/partitioning.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala similarity index 96% rename from src/main/scala/catalyst/plans/logical/partitioning.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index 87e625d576483..0f07c748e430b 100644 --- a/src/main/scala/catalyst/plans/logical/partitioning.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package plans package logical diff --git a/src/main/scala/catalyst/plans/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala similarity index 84% rename from src/main/scala/catalyst/plans/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala index 6ac3859a2329c..e483182cdf16b 100644 --- a/src/main/scala/catalyst/plans/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst /** diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala similarity index 99% rename from src/main/scala/catalyst/plans/physical/partitioning.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 25e2240372e11..7f48a66b55175 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package plans package physical diff --git a/src/main/scala/catalyst/plans/shared.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala similarity index 88% rename from src/main/scala/catalyst/plans/shared.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala index da47d3d9c8f36..828be70adf6b0 100644 --- a/src/main/scala/catalyst/plans/shared.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/shared.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package plans diff --git a/src/main/scala/catalyst/rules/Rule.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala similarity index 92% rename from src/main/scala/catalyst/rules/Rule.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala index fb609fd527b77..79eccd6839c58 100644 --- a/src/main/scala/catalyst/rules/Rule.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package rules diff --git a/src/main/scala/catalyst/rules/RuleExecutor.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala similarity index 96% rename from src/main/scala/catalyst/rules/RuleExecutor.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 69cd1dae5b00a..886d8bd386b82 100644 --- a/src/main/scala/catalyst/rules/RuleExecutor.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package rules @@ -42,7 +43,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { curPlan = batch.rules.foldLeft(curPlan) { case (curPlan, rule) => val result = rule(curPlan) - if(!result.fastEquals(curPlan)) { + if (!result.fastEquals(curPlan)) { logger.debug( s""" |=== Applying Rule ${rule.ruleName} === diff --git a/src/main/scala/catalyst/rules/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala similarity index 81% rename from src/main/scala/catalyst/rules/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala index 6f4ea86c71d41..18d4e0eae122d 100644 --- a/src/main/scala/catalyst/rules/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst /** diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala similarity index 99% rename from src/main/scala/catalyst/trees/TreeNode.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 814686d8936eb..a7edd3225b29d 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package trees diff --git a/src/main/scala/catalyst/trees/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala similarity index 96% rename from src/main/scala/catalyst/trees/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index 4ce0c28dd5d0d..1e1bbbc16cee2 100644 --- a/src/main/scala/catalyst/trees/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst /** diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala similarity index 99% rename from src/main/scala/catalyst/types/dataTypes.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 360ccf7c7de16..bd8dd6db943e1 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package types diff --git a/src/main/scala/catalyst/types/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala similarity index 85% rename from src/main/scala/catalyst/types/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala index 872b5b1ab904a..6988ca0aa54bb 100644 --- a/src/main/scala/catalyst/types/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala @@ -1,5 +1,5 @@ +package org.apache.spark.sql package catalyst - /** * Contains a type system for attributes produced by relations, including complex types like * structs, arrays and maps. diff --git a/src/main/scala/catalyst/util/package.scala b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala similarity index 98% rename from src/main/scala/catalyst/util/package.scala rename to catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index d1ea9ed3dee3b..991da35920da4 100644 --- a/src/main/scala/catalyst/util/package.scala +++ b/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File} @@ -87,9 +88,10 @@ package object util { def stringOrNull(a: AnyRef) = if (a == null) null else a.toString + /* FIX ME implicit class debugLogging(a: AnyRef) { def debugLogging() { org.apache.log4j.Logger.getLogger(a.getClass.getName).setLevel(org.apache.log4j.Level.DEBUG) } - } + } */ } diff --git a/src/test/scala/catalyst/AnalysisSuite.scala b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala similarity index 93% rename from src/test/scala/catalyst/AnalysisSuite.scala rename to catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala index 464a344ea93e6..fb35e27fbbfd2 100644 --- a/src/test/scala/catalyst/AnalysisSuite.scala +++ b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package analysis diff --git a/src/test/scala/catalyst/execution/DistributionTests.scala b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala similarity index 96% rename from src/test/scala/catalyst/execution/DistributionTests.scala rename to catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 2a3d85b5db40b..17bf4e6b974da 100644 --- a/src/test/scala/catalyst/execution/DistributionTests.scala +++ b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -1,18 +1,19 @@ +package org.apache.spark.sql package catalyst -package plans -package physical + import org.scalatest.FunSuite +import plans.physical._ import catalyst.dsl._ -class DistributionTests extends FunSuite { +class DistributionSuite extends FunSuite { protected def checkSatisfied( inputPartitioning: Partitioning, requiredDistribution: Distribution, satisfied: Boolean) { - if(inputPartitioning.satisfies(requiredDistribution) != satisfied) + if (inputPartitioning.satisfies(requiredDistribution) != satisfied) fail( s""" |== Input Partitioning == diff --git a/src/test/scala/catalyst/ExpressionEvaluationSuite.scala b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala similarity index 98% rename from src/test/scala/catalyst/ExpressionEvaluationSuite.scala rename to catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala index 26836aa7b5623..e8dd9178b23a8 100644 --- a/src/test/scala/catalyst/ExpressionEvaluationSuite.scala +++ b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package expressions diff --git a/src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala similarity index 97% rename from src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala rename to catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala index 4fb834e373397..1c1768233107c 100644 --- a/src/test/scala/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala @@ -1,4 +1,6 @@ -package catalyst.analysis +package org.apache.spark.sql +package catalyst +package analysis import org.scalatest.FunSuite diff --git a/src/test/scala/catalyst/RuleExecutorSuite.scala b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala similarity index 97% rename from src/test/scala/catalyst/RuleExecutorSuite.scala rename to catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala index 29434379ce7af..c6776f26df714 100644 --- a/src/test/scala/catalyst/RuleExecutorSuite.scala +++ b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package trees diff --git a/src/test/scala/catalyst/TreeNodeSuite.scala b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala similarity index 98% rename from src/test/scala/catalyst/TreeNodeSuite.scala rename to catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala index 45ed71f03f80e..2f941be1becb1 100644 --- a/src/test/scala/catalyst/TreeNodeSuite.scala +++ b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package trees diff --git a/src/test/scala/catalyst/optimizer/ConstantFoldingSuite.scala b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala similarity index 99% rename from src/test/scala/catalyst/optimizer/ConstantFoldingSuite.scala rename to catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 46088ff4b9c55..df3d329c5abe2 100644 --- a/src/test/scala/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package optimizer diff --git a/src/test/scala/catalyst/optimizer/FilterPushdownSuite.scala b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala similarity index 99% rename from src/test/scala/catalyst/optimizer/FilterPushdownSuite.scala rename to catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index e567b16637006..b1ba0e0af507c 100644 --- a/src/test/scala/catalyst/optimizer/FilterPushdownSuite.scala +++ b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package optimizer diff --git a/src/test/scala/catalyst/optimizer/OptimizerTest.scala b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala similarity index 97% rename from src/test/scala/catalyst/optimizer/OptimizerTest.scala rename to catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala index e885fc61a6937..7b3653d0f998a 100644 --- a/src/test/scala/catalyst/optimizer/OptimizerTest.scala +++ b/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala @@ -1,3 +1,4 @@ +package org.apache.spark.sql package catalyst package optimizer diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala similarity index 100% rename from src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala rename to core/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala diff --git a/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala b/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala new file mode 100644 index 0000000000000..bfaa23b12b0fe --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/SparkSqlContext.scala @@ -0,0 +1,114 @@ +package org.apache.spark.sql + +import scala.language.implicitConversions + +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.rdd.RDD + +import catalyst.analysis._ +import catalyst.expressions.BindReferences +import catalyst.optimizer.Optimizer +import catalyst.planning.QueryPlanner +import catalyst.plans.logical.{LogicalPlan, NativeCommand} +import catalyst.rules.RuleExecutor + +import execution._ + +case class ExecutedQuery( + sql: String, + logicalPlan: LogicalPlan, + executedPlan: Option[SparkPlan], + rdd: RDD[Row]) { + + def schema = logicalPlan.output + + override def toString() = + s"$sql\n${executedPlan.map(p => s"=== Query Plan ===\n$p").getOrElse("")}" +} + +object TestSqlContext + extends SparkSqlContext(new SparkContext("local", "TestSqlContext", new SparkConf())) + +class SparkSqlContext(val sparkContext: SparkContext) extends Logging { + self => + + val catalog: Catalog = EmptyCatalog + val analyzer: Analyzer = new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) + val optimizer = Optimizer + + def parseSql(sql: String): LogicalPlan = ??? + def executeSql(sql: String): this.QueryExecution = executePlan(parseSql(sql)) + def executePlan(plan: LogicalPlan): this.QueryExecution = + new this.QueryExecution { val logical = plan } + + implicit def logicalPlanToSparkQuery(plan: LogicalPlan) = executePlan(plan) + + implicit def logicalDsl(q: ExecutedQuery) = new catalyst.dsl.DslLogicalPlan(q.logicalPlan) + + implicit def toRdd(q: ExecutedQuery) = q.rdd + + def sql(sqlText: String): ExecutedQuery = { + val queryWorkflow = executeSql(sqlText) + val executedPlan = queryWorkflow.analyzed match { + case _: NativeCommand => None + case other => Some(queryWorkflow.executedPlan) + } + ExecutedQuery(sqlText, queryWorkflow.analyzed, executedPlan, queryWorkflow.toRdd) + } + + class SparkPlanner extends SparkStrategies { + val sparkContext = self.sparkContext + + val strategies: Seq[Strategy] = + PartialAggregation :: + SparkEquiInnerJoin :: + BasicOperators :: + CartesianProduct :: + BroadcastNestedLoopJoin :: Nil + } + + val planner = new SparkPlanner + + /** + * Prepares a planned SparkPlan for execution by binding references to specific ordinals, and + * inserting shuffle operations as needed. + */ + object PrepareForExecution extends RuleExecutor[SparkPlan] { + val batches = + Batch("Add exchange", Once, AddExchange) :: + Batch("Prepare Expressions", Once, new BindReferences[SparkPlan]) :: Nil + } + + /** + * The primary workflow for executing relational queries using Spark. Designed to allow easy + * access to the intermediate phases of query execution for developers. Most users should + * use [[ExecutedQuery]] to interact with query results. + */ + abstract class QueryExecution { + def logical: LogicalPlan + + lazy val analyzed = analyzer(logical) + lazy val optimizedPlan = optimizer(analyzed) + // TODO: Don't just pick the first one... + lazy val sparkPlan = planner(optimizedPlan).next() + lazy val executedPlan: SparkPlan = PrepareForExecution(sparkPlan) + + // TODO: We are loosing schema here. + lazy val toRdd: RDD[Row] = executedPlan.execute() + + protected def stringOrError[A](f: => A): String = + try f.toString catch { case e: Throwable => e.toString } + + override def toString: String = + s"""== Logical Plan == + |${stringOrError(analyzed)} + |== Physical Plan == + |${stringOrError(executedPlan)} + """.stripMargin.trim + + /** + * Runs the query after interposing operators that print the result of each intermediate step. + */ + def debugExec() = DebugQuery(executedPlan).execute().collect() + } +} diff --git a/src/main/scala/catalyst/examples/SchemaRddExample.scala b/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala similarity index 83% rename from src/main/scala/catalyst/examples/SchemaRddExample.scala rename to core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala index fdacdb734e0e9..28d24dde00001 100644 --- a/src/main/scala/catalyst/examples/SchemaRddExample.scala +++ b/core/src/main/scala/org/apache/spark/sql/examples/SchemaRddExample.scala @@ -1,10 +1,10 @@ -package catalyst +package org.apache.spark.sql package examples -import plans.logical.LocalRelation +import catalyst.plans.logical.LocalRelation -import execution.TestShark._ -import dsl._ +import TestSqlContext._ +import catalyst.dsl._ object SchemaRddExample { def main(args: Array[String]): Unit = { @@ -22,7 +22,7 @@ object SchemaRddExample { * the requested attributes are passed to the UDF. Since this analysis occurs at runtime, * the developer must manually annotate their function with the correct argument types. */ - val filtered = testLogs.filter('date) { case dateRegEx(_,day,_) => day.toInt == 1 } + val filtered = testLogs.sfilter('date) { case dateRegEx(_,day,_) => day.toInt == 1 } filtered.toRdd.collect.foreach(println) @@ -36,7 +36,7 @@ object SchemaRddExample { * being resolved at runtime. Thus, we cannot return typed results. As such all dynamic calls * always return strings. */ - val filtered2 = testLogs.filter( _.date match { case dateRegEx(_,day,_) => day.toInt == 1 } ) + val filtered2 = testLogs.sfilter( _.date match { case dateRegEx(_,day,_) => day.toInt == 1 } ) filtered2.toRdd.collect.foreach(println) } } diff --git a/src/main/scala/catalyst/examples/ViewsExample.scala b/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala similarity index 92% rename from src/main/scala/catalyst/examples/ViewsExample.scala rename to core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala index cf50b7343a159..eea5f92c4c69b 100644 --- a/src/main/scala/catalyst/examples/ViewsExample.scala +++ b/core/src/main/scala/org/apache/spark/sql/examples/ViewsExample.scala @@ -1,4 +1,4 @@ -package catalyst +package org.apache.spark.sql package examples import catalyst.analysis.UnresolvedRelation @@ -6,8 +6,9 @@ import catalyst.plans.Inner import catalyst.plans.logical._ /* Implicit Conversions */ -import dsl._ -import execution.TestShark._ // For .toRdd execution using locally running test Shark. +import catalyst.dsl._ + +import TestSqlContext._ // For .toRdd execution using locally running test Shark. object ViewsExample { def main(args: Array[String]): Unit = { diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala similarity index 94% rename from src/main/scala/catalyst/execution/Exchange.scala rename to core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index bed10d2d63fcb..12f1696f64a3a 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -1,4 +1,4 @@ -package catalyst +package org.apache.spark.sql package execution import catalyst.rules.Rule @@ -9,8 +9,7 @@ import catalyst.plans.physical._ import org.apache.spark.{RangePartitioner, HashPartitioner} import org.apache.spark.rdd.ShuffledRDD -case class Exchange(newPartitioning: Partitioning, child: SharkPlan) - extends UnaryNode { +case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { override def outputPartitioning = newPartitioning def output = child.output @@ -53,12 +52,12 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) * [[catalyst.plans.physical.Distribution Distribution]] requirements for each operator by inserting * [[Exchange]] Operators where required. */ -object AddExchange extends Rule[SharkPlan] { +object AddExchange extends Rule[SparkPlan] { // TODO: Determine the number of partitions. val numPartitions = 8 - def apply(plan: SharkPlan): SharkPlan = plan.transformUp { - case operator: SharkPlan => + def apply(plan: SparkPlan): SparkPlan = plan.transformUp { + case operator: SparkPlan => // Check if every child's outputPartitioning satisfies the corresponding // required data distribution. def meetsRequirements = @@ -90,7 +89,7 @@ object AddExchange extends Rule[SharkPlan] { // Check if the partitioning we want to ensure is the same as the child's output // partitioning. If so, we do not need to add the Exchange operator. - def addExchangeIfNecessary(partitioning: Partitioning, child: SharkPlan) = + def addExchangeIfNecessary(partitioning: Partitioning, child: SparkPlan) = if (child.outputPartitioning != partitioning) Exchange(partitioning, child) else child if (meetsRequirements && compatible) { diff --git a/src/main/scala/catalyst/execution/Generate.scala b/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala similarity index 96% rename from src/main/scala/catalyst/execution/Generate.scala rename to core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index 93f6be5a66df2..ce4207e8b33c6 100644 --- a/src/main/scala/catalyst/execution/Generate.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -1,4 +1,4 @@ -package catalyst +package org.apache.spark.sql package execution import catalyst.expressions._ @@ -18,7 +18,7 @@ case class Generate( generator: Generator, join: Boolean, outer: Boolean, - child: SharkPlan) + child: SparkPlan) extends UnaryNode { def output = @@ -41,4 +41,4 @@ case class Generate( child.execute().mapPartitions(iter => iter.flatMap(generator)) } } -} \ No newline at end of file +} diff --git a/src/main/scala/catalyst/execution/QueryExecutionException.scala b/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala similarity index 61% rename from src/main/scala/catalyst/execution/QueryExecutionException.scala rename to core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala index 40ec17798c319..c21ab68cd814e 100644 --- a/src/main/scala/catalyst/execution/QueryExecutionException.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala @@ -1,3 +1,4 @@ -package catalyst.execution +package org.apache.spark.sql +package execution class QueryExecutionException(message: String) extends Exception(message) diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala similarity index 76% rename from src/main/scala/catalyst/execution/SharkPlan.scala rename to core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala index 1b72c13fd7084..700719b6de98e 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SharkPlan.scala @@ -1,12 +1,13 @@ -package catalyst +package org.apache.spark.sql package execution import org.apache.spark.rdd.RDD import catalyst.plans.QueryPlan import catalyst.plans.physical._ +import catalyst.trees -abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { +abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { self: Product => // TODO: Move to `DistributedPlan` @@ -29,15 +30,15 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { protected def buildRow(values: Seq[Any]): Row = new catalyst.expressions.GenericRow(values) } -trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { +trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { self: Product => } -trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { +trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] { self: Product => override def outputPartitioning: Partitioning = child.outputPartitioning } -trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { +trait BinaryNode extends SparkPlan with trees.BinaryNode[SparkPlan] { self: Product => } diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala similarity index 64% rename from src/main/scala/catalyst/execution/PlanningStrategies.scala rename to core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 88476b1b8c438..37fa5b9823ab3 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -1,79 +1,20 @@ -package catalyst +package org.apache.spark.sql package execution -import expressions._ -import planning._ -import plans._ -import plans.logical.LogicalPlan -import plans.physical._ +import org.apache.spark.SparkContext -trait PlanningStrategies { - self: QueryPlanner[SharkPlan] => +import catalyst.expressions._ +import catalyst.planning._ +import catalyst.plans._ +import catalyst.plans.logical.LogicalPlan +import catalyst.plans.physical._ - val sc: SharkContext +abstract class SparkStrategies extends QueryPlanner[SparkPlan] { - object DataSinks extends Strategy { - def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { - case logical.InsertIntoTable(table: MetastoreRelation, partition, child) => - InsertIntoHiveTable(table, partition, planLater(child))(sc) :: Nil - case _ => Nil - } - } - - object HiveTableScans extends Strategy { - def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { - // Push attributes into table scan when possible. - case p @ logical.Project(projectList, m: MetastoreRelation) if isSimpleProject(projectList) => - execution.HiveTableScan(projectList.asInstanceOf[Seq[Attribute]], m, None)(sc) :: Nil - case m: MetastoreRelation => - execution.HiveTableScan(m.output, m, None)(sc) :: Nil - case _ => Nil - } - - /** - * Returns true if `projectList` only performs column pruning and does not evaluate other - * complex expressions. - */ - def isSimpleProject(projectList: Seq[NamedExpression]) = { - projectList.forall(_.isInstanceOf[Attribute]) - } - } - - /** - * A strategy used to detect filtering predicates on top of a partitioned relation to help - * partition pruning. - * - * This strategy itself doesn't perform partition pruning, it just collects and combines all the - * partition pruning predicates and pass them down to the underlying [[HiveTableScan]] operator, - * which does the actual pruning work. - */ - object PartitionPrunings extends Strategy { - def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { - case p @ FilteredOperation(predicates, relation: MetastoreRelation) - if relation.hiveQlTable.isPartitioned => - - val partitionKeyIds = relation.partitionKeys.map(_.id).toSet - - // Filter out all predicates that only deal with partition keys - val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.map(_.id).subsetOf(partitionKeyIds) - } - - val scan = execution.HiveTableScan( - relation.output, relation, pruningPredicates.reduceLeftOption(And))(sc) - - otherPredicates - .reduceLeftOption(And) - .map(execution.Filter(_, scan)) - .getOrElse(scan) :: Nil - - case _ => - Nil - } - } + val sparkContext: SparkContext object SparkEquiInnerJoin extends Strategy { - def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case FilteredOperation(predicates, logical.Join(left, right, Inner, condition)) => logger.debug(s"Considering join: ${predicates ++ condition}") // Find equi-join predicates that can be evaluated before the join, and thus can be used @@ -86,8 +27,8 @@ trait PlanningStrategies { } val joinKeys = joinPredicates.map { - case Equals(l,r) if (canEvaluate(l, left) && canEvaluate(r, right)) => (l, r) - case Equals(l,r) if (canEvaluate(l, right) && canEvaluate(r, left)) => (r, l) + case Equals(l,r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r) + case Equals(l,r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l) } // Do not consider this strategy if there are no join keys. @@ -112,7 +53,7 @@ trait PlanningStrategies { } private def combineConjunctivePredicates(predicates: Seq[Expression]) = - predicates.reduceLeft(And(_, _)) + predicates.reduceLeft(And) /** Returns true if `expr` can be evaluated using only the output of `plan`. */ protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = @@ -120,7 +61,7 @@ trait PlanningStrategies { } object PartialAggregation extends Strategy { - def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Aggregate(groupingExpressions, aggregateExpressions, child) => // Collect all aggregate expressions. val allAggregates = @@ -165,7 +106,7 @@ trait PlanningStrategies { partial = true, groupingExpressions, partialComputation, - planLater(child))(sc))(sc) :: Nil + planLater(child))(sparkContext))(sparkContext) :: Nil } else { Nil } @@ -174,16 +115,16 @@ trait PlanningStrategies { } object BroadcastNestedLoopJoin extends Strategy { - def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Join(left, right, joinType, condition) => execution.BroadcastNestedLoopJoin( - planLater(left), planLater(right), joinType, condition)(sc) :: Nil + planLater(left), planLater(right), joinType, condition)(sparkContext) :: Nil case _ => Nil } } object CartesianProduct extends Strategy { - def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Join(left, right, _, None) => execution.CartesianProduct(planLater(left), planLater(right)) :: Nil case logical.Join(left, right, Inner, Some(condition)) => @@ -193,12 +134,15 @@ trait PlanningStrategies { } } + protected lazy val singleRowRdd = + sparkContext.parallelize(Seq(new GenericRow(IndexedSeq()): Row), 1) + // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { - def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Distinct(child) => execution.Aggregate( - partial = false, child.output, child.output, planLater(child))(sc) :: Nil + partial = false, child.output, child.output, planLater(child))(sparkContext) :: Nil case logical.Sort(sortExprs, child) => // This sort is a global sort. Its requiredDistribution will be an OrderedDistribution. execution.Sort(sortExprs, global = true, planLater(child)):: Nil @@ -211,23 +155,24 @@ trait PlanningStrategies { case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(partial = false, group, agg, planLater(child))(sc) :: Nil + execution.Aggregate(partial = false, group, agg, planLater(child))(sparkContext) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => - execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil + val dataAsRdd = + sparkContext.parallelize(data.map(r => new GenericRow(r.productIterator.toVector): Row)) + execution.ExistingRdd(output, dataAsRdd) :: Nil case logical.StopAfter(limit, child) => - execution.StopAfter(Evaluate(limit, Nil).asInstanceOf[Int], planLater(child))(sc) :: Nil + execution.StopAfter( + Evaluate(limit, Nil).asInstanceOf[Int], + planLater(child))(sparkContext) :: Nil case Unions(unionChildren) => - execution.Union(unionChildren.map(planLater))(sc) :: Nil + execution.Union(unionChildren.map(planLater))(sparkContext) :: Nil case logical.Generate(generator, join, outer, _, child) => execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil - case logical.ScriptTransformation(input, script, output, child) => - execution.ScriptTransformation(input, script, output, planLater(child))(sc) :: Nil case logical.NoRelation => - execution.LocalRelation(Nil, Seq(IndexedSeq()))(sc) :: Nil + execution.ExistingRdd(Nil, singleRowRdd) :: Nil case _ => Nil } } - } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala new file mode 100644 index 0000000000000..6ac9b42d90164 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala @@ -0,0 +1,100 @@ +package org.apache.spark.sql +package execution + +import org.apache.spark.SparkContext + +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples} +import catalyst.types._ + +import org.apache.spark.rdd.SharkPairRDDFunctions._ + +/** + * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each + * group. + * + * @param partial if true then aggregation is done partially on local data without shuffling to + * ensure all values where `groupingExpressions` are equal are present. + * @param groupingExpressions expressions that are evaluated to determine grouping. + * @param aggregateExpressions expressions that are computed for each group. + * @param child the input data source. + */ +case class Aggregate( + partial: Boolean, + groupingExpressions: Seq[Expression], + aggregateExpressions: Seq[NamedExpression], + child: SparkPlan)(@transient sc: SparkContext) + extends UnaryNode { + + override def requiredChildDistribution = + if (partial) { + UnspecifiedDistribution :: Nil + } else { + if (groupingExpressions == Nil) { + AllTuples :: Nil + } else { + ClusteredDistribution(groupingExpressions) :: Nil + } + } + + override def otherCopyArgs = sc :: Nil + + def output = aggregateExpressions.map(_.toAttribute) + + /* Replace all aggregate expressions with spark functions that will compute the result. */ + def createAggregateImplementations() = aggregateExpressions.map { agg => + val impl = agg transform { + case a: AggregateExpression => a.newInstance + } + + val remainingAttributes = impl.collect { case a: Attribute => a } + // If any references exist that are not inside agg functions then the must be grouping exprs + // in this case we must rebind them to the grouping tuple. + if (remainingAttributes.nonEmpty) { + val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } + + // An exact match with a grouping expression + val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { + case -1 => None + case ordinal => Some(BoundReference(0, ordinal, Alias(impl, "AGGEXPR")().toAttribute)) + } + + exactGroupingExpr.getOrElse( + sys.error(s"$agg is not in grouping expressions: $groupingExpressions")) + } else { + impl + } + } + + def execute() = attachTree(this, "execute") { + // TODO: If the child of it is an [[catalyst.execution.Exchange]], + // do not evaluate the groupingExpressions again since we have evaluated it + // in the [[catalyst.execution.Exchange]]. + val grouped = child.execute().map { row => + (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row) + }.groupByKeyLocally() + + val result = grouped.map { case (group, rows) => + val aggImplementations = createAggregateImplementations() + + // Pull out all the functions so we can feed each row into them. + val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f }) + + rows.foreach { row => + val input = Vector(row) + aggFunctions.foreach(_.apply(input)) + } + buildRow(aggImplementations.map(Evaluate(_, Vector(group)))) + } + + // TODO: THIS DOES NOT PRESERVE LINEAGE AND BREAKS PIPELINING. + if (groupingExpressions.isEmpty && result.count == 0) { + // When there there is no output to the Aggregate operator, we still output an empty row. + val aggImplementations = createAggregateImplementations() + sc.makeRDD(buildRow(aggImplementations.map(Evaluate(_, Nil))) :: Nil) + } else { + result + } + } +} diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala similarity index 70% rename from src/main/scala/catalyst/execution/basicOperators.scala rename to core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index b44c1964fb261..9e79fdda68b9b 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -1,11 +1,15 @@ -package catalyst +package org.apache.spark.sql package execution +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext + import catalyst.errors._ import catalyst.expressions._ import catalyst.plans.physical.{UnspecifiedDistribution, OrderedDistribution} +import catalyst.plans.logical.LogicalPlan -case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { +case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) def execute() = child.execute().map { row => @@ -13,14 +17,14 @@ case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends } } -case class Filter(condition: Expression, child: SharkPlan) extends UnaryNode { +case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { def output = child.output def execute() = child.execute().filter { row => Evaluate(condition, Vector(row)).asInstanceOf[Boolean] } } -case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SharkPlan) +case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan) extends UnaryNode { def output = child.output @@ -29,7 +33,7 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: def execute() = child.execute().sample(withReplacement, fraction, seed) } -case class Union(children: Seq[SharkPlan])(@transient sc: SharkContext) extends SharkPlan { +case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends SparkPlan { // TODO: attributes output by union should be distinct for nullability purposes def output = children.head.output def execute() = sc.union(children.map(_.execute())) @@ -37,7 +41,7 @@ case class Union(children: Seq[SharkPlan])(@transient sc: SharkContext) extends override def otherCopyArgs = sc :: Nil } -case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { +case class StopAfter(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil def output = child.output @@ -52,7 +56,7 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) case class Sort( sortOrder: Seq[SortOrder], global: Boolean, - child: SharkPlan) + child: SparkPlan) extends UnaryNode { override def requiredChildDistribution = if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil @@ -71,12 +75,7 @@ case class Sort( def output = child.output } -// TODO: Rename: SchemaRDD -case class LocalRelation(output: Seq[Attribute], data: Seq[IndexedSeq[Any]]) - (@transient sc: SharkContext) extends LeafNode { - - // Since LocalRelation is used for unit tests, set the defaultParallelism to 2 - // to make sure we can cover bugs appearing in a distributed environment. - def execute() = sc.makeRDD(data.map(buildRow), 2) +case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { + def execute() = rdd } diff --git a/src/main/scala/catalyst/execution/debug.scala b/core/src/main/scala/org/apache/spark/sql/execution/debug.scala similarity index 72% rename from src/main/scala/catalyst/execution/debug.scala rename to core/src/main/scala/org/apache/spark/sql/execution/debug.scala index b3ac54fda5b42..6c9ea87eb5386 100644 --- a/src/main/scala/catalyst/execution/debug.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/debug.scala @@ -1,18 +1,18 @@ -package catalyst +package org.apache.spark.sql package execution object DebugQuery { - def apply(plan: SharkPlan): SharkPlan = { + def apply(plan: SparkPlan): SparkPlan = { val visited = new collection.mutable.HashSet[Long]() plan transform { - case s: SharkPlan if !visited.contains(s.id) => + case s: SparkPlan if !visited.contains(s.id) => visited += s.id DebugNode(s) } } } -case class DebugNode(child: SharkPlan) extends UnaryNode { +case class DebugNode(child: SparkPlan) extends UnaryNode { def references = Set.empty def output = child.output def execute() = { diff --git a/src/main/scala/catalyst/execution/joins.scala b/core/src/main/scala/org/apache/spark/sql/execution/joins.scala similarity index 84% rename from src/main/scala/catalyst/execution/joins.scala rename to core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 72d24835c7ae2..885880a84762e 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -1,22 +1,23 @@ -package catalyst +package org.apache.spark.sql package execution import scala.collection.mutable -import scala.Some - -import errors._ -import expressions._ -import plans._ -import org.apache.spark.rdd.SharkPairRDDFunctions._ import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext + +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.plans._ import catalyst.plans.physical.{ClusteredDistribution, Partitioning} +import org.apache.spark.rdd.SharkPairRDDFunctions._ + case class SparkEquiInnerJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], - left: SharkPlan, - right: SharkPlan) extends BinaryNode { + left: SparkPlan, + right: SparkPlan) extends BinaryNode { override def outputPartitioning: Partitioning = left.outputPartitioning @@ -28,15 +29,14 @@ case class SparkEquiInnerJoin( def execute() = attachTree(this, "execute") { val leftWithKeys = left.execute().map { row => val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) - logger.debug(s"Generated left join keys [${leftKeys.mkString(",")}] =>" + - s"[${joinKeys.mkString(",")}] given row $row") + //logger.debug(s"leftkey [${leftKeys.mkString(",")}] => ${joinKeys.mkString(",")}] from $row") (joinKeys, row) } val rightWithKeys = right.execute().map { row => val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) - logger.debug(s"Generated right join keys [${rightKeys.mkString(",")}] =>" + - s"[${joinKeys.mkString(",")}] given row $row") + //logger.debug(s"rightkey [${leftKeys.mkString(",")}] => ${joinKeys.mkString(",")}] from $row") + (joinKeys, row) } @@ -56,7 +56,7 @@ case class SparkEquiInnerJoin( } } -case class CartesianProduct(left: SharkPlan, right: SharkPlan) extends BinaryNode { +case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { def output = left.output ++ right.output def execute() = left.execute().cartesian(right.execute()).map { @@ -65,8 +65,8 @@ case class CartesianProduct(left: SharkPlan, right: SharkPlan) extends BinaryNod } case class BroadcastNestedLoopJoin( - streamed: SharkPlan, broadcast: SharkPlan, joinType: JoinType, condition: Option[Expression]) - (@transient sc: SharkContext) + streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression]) + (@transient sc: SparkContext) extends BinaryNode { // TODO: Override requiredChildDistribution. @@ -131,6 +131,8 @@ case class BroadcastNestedLoopJoin( Vector() } - sc.union(streamedPlusMatches.flatMap(_._1), sc.makeRDD(rightOuterMatches)) + // TODO: Breaks lineage. + sc.union( + streamedPlusMatches.flatMap(_._1), sc.makeRDD(rightOuterMatches)) } } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/package.scala b/core/src/main/scala/org/apache/spark/sql/execution/package.scala new file mode 100644 index 0000000000000..187a745e0ac95 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/execution/package.scala @@ -0,0 +1,14 @@ +package org.apache.spark.sql + +/** + * A partial reimplementation of Shark, a Hive compatible SQL engine running on Spark. + * + * This implementation uses the hive parser, metadata catalog and serdes, but performs all + * optimization and execution using catalyst and spark. + * + * Currently functions that are not supported by this implementation are passed back to the + * original Shark implementation for execution. + */ +package object execution { + type Row = catalyst.expressions.Row +} diff --git a/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties similarity index 100% rename from src/test/resources/log4j.properties rename to core/src/test/resources/log4j.properties diff --git a/src/test/scala/catalyst/execution/DslQueryTests.scala b/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala similarity index 95% rename from src/test/scala/catalyst/execution/DslQueryTests.scala rename to core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala index c2e1492e5294f..5f2f5ce9b8f0d 100644 --- a/src/test/scala/catalyst/execution/DslQueryTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/DslQueryTests.scala @@ -1,20 +1,17 @@ -package catalyst -package execution +package org.apache.spark.sql import org.scalatest.{BeforeAndAfterAll, FunSuite} -import analysis._ -import expressions._ -import plans._ -import plans.logical.LogicalPlan -import types._ +import catalyst.analysis._ +import catalyst.expressions._ +import catalyst.plans._ +import catalyst.plans.logical.LogicalPlan +import catalyst.types._ /* Implicits */ -import dsl._ +import catalyst.dsl._ object TestData { - import TestShark._ - val testData = logical.LocalRelation('key.int, 'value.string) .loadData((1 to 100).map(i => (i, i.toString))) @@ -55,7 +52,7 @@ object TestData { } class DslQueryTest extends FunSuite { - import TestShark._ + import TestSqlContext._ /** * Runs the plan and makes sure the answer matches the expected result. @@ -89,8 +86,8 @@ class DslQueryTest extends FunSuite { } } -class BasicQueryTests extends DslQueryTest { - import TestShark._ +class BasicQuerySuite extends DslQueryTest { + import TestSqlContext._ import TestData._ test("table scan") { diff --git a/src/test/scala/catalyst/execution/PlannerSuite.scala b/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala similarity index 78% rename from src/test/scala/catalyst/execution/PlannerSuite.scala rename to core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala index 3736030d3fceb..c5287659b3de0 100644 --- a/src/test/scala/catalyst/execution/PlannerSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala @@ -1,4 +1,4 @@ -package catalyst +package org.apache.spark.sql package execution import org.scalatest.FunSuite @@ -9,10 +9,11 @@ import catalyst.dsl._ class PlannerSuite extends FunSuite { import TestData._ + import TestSqlContext.planner._ test("unions are collapsed") { val query = testData.unionAll(testData).unionAll(testData) - val planned = TestShark.TrivialPlanner.BasicOperators(query).head + val planned = BasicOperators(query).head val logicalUnions = query collect { case u: logical.Union => u} val physicalUnions = planned collect { case u: execution.Union => u} @@ -22,7 +23,7 @@ class PlannerSuite extends FunSuite { test("count is partially aggregated") { val query = testData.groupBy('value)(Count('key)).analyze - val planned = TestShark.TrivialPlanner.PartialAggregation(query).head + val planned = PartialAggregation(query).head val aggregations = planned.collect { case a: Aggregate => a } assert(aggregations.size === 2) @@ -30,13 +31,13 @@ class PlannerSuite extends FunSuite { test("count distinct is not partially aggregated") { val query = testData.groupBy('value)(CountDistinct('key :: Nil)).analyze - val planned = TestShark.TrivialPlanner.PartialAggregation(query) + val planned = PartialAggregation(query) assert(planned.isEmpty) } test("mixed aggregates are not partially aggregated") { val query = testData.groupBy('value)(Count('value), CountDistinct('key :: Nil)).analyze - val planned = TestShark.TrivialPlanner.PartialAggregation(query) + val planned = PartialAggregation(query) assert(planned.isEmpty) } -} \ No newline at end of file +} diff --git a/src/test/scala/catalyst/execution/TgfSuite.scala b/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala similarity index 88% rename from src/test/scala/catalyst/execution/TgfSuite.scala rename to core/src/test/scala/org/apache/spark/sql/TgfSuite.scala index 12852cb5822c2..f4850f97078e8 100644 --- a/src/test/scala/catalyst/execution/TgfSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala @@ -1,15 +1,15 @@ -package catalyst +package org.apache.spark.sql package execution import org.scalatest.{BeforeAndAfterAll, FunSuite} -import analysis._ -import expressions._ -import plans._ -import plans.logical.LogicalPlan -import types._ +import catalyst.analysis._ +import catalyst.expressions._ +import catalyst.plans._ +import catalyst.plans.logical.LogicalPlan +import catalyst.types._ -import dsl._ +import catalyst.dsl._ /** * This is an example TGF that uses UnresolvedAttributes 'name and 'age to access specific columns diff --git a/project/build.properties b/project/build.properties index 9b860e23c51a6..0974fce44da5f 100755 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=0.12.3 +sbt.version=0.13.0 diff --git a/project/plugins.sbt b/project/plugins.sbt index d2c2af17b0fb5..caf589a765aac 100755 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -13,13 +13,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -addSbtPlugin("org.ensime" % "ensime-sbt-cmd" % "0.1.1") +//addSbtPlugin("org.ensime" % "ensime-sbt-cmd" % "0.1.1") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") +//addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") -addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.4.0") +addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.5.2" extra("sbtVersion" -> "0.13", "scalaVersion" -> "2.10")) -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2") +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.10.1") + +addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") resolvers += Resolver.url( "sbt-plugin-releases", @@ -27,13 +29,12 @@ resolvers += Resolver.url( resolvers += "jgit-repo" at "http://download.eclipse.org/jgit/maven" +addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "0.6.2") + addSbtPlugin("com.typesafe.sbt" % "sbt-ghpages" % "0.5.2") addSbtPlugin("com.typesafe.sbt" % "sbt-site" % "0.7.1") -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2") - - addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.3.2") // For scalastyle diff --git a/sbt/sbt-launch.jar b/sbt/sbt-launch.jar index 672c26f674ceb6ad2429011815d1f0615d9bb9fe..76342c66822d832bd88a4630c50274c24d539252 100755 GIT binary patch delta 947481 zcmZU4Q*fY7v~6tLnTc)Nwr$%^zKQLLZQHhO+vd!~x&JwD_ulTV>Z;xQvG>|*t?KUV z743-;M^cgnhrj@V`EMpDVTmRpnM3^V?nwveYAy+(d|_Fsii(y0!e%fLmj;FPOzaX; zuGb5hMBNk77O5xuDNXxH`vvKLK<@NRK78vKZGx&e!~K3Ud(Fw)#G3W`b@ZeO z^0^+B&lz+A!$HoMb}urR6q1IPgr-4OEu)oO&v2j)6Iutw66=BclP>{vlpmB3>=qt4 zPec_i15qDkcbgCc$cnI#A&1TNB;|zKg355v?!^Y!!XRU4(oqlTBry0y`+~UnG?%SV%;eH7MMI?b)JdTn(^Aw*-ATWzg6 zi^~3WSe!oUYD3A=PyTDs_MQO&}4)}eRRvHZ1`w>XeqV)?WV%k=h5V-=&a_a6iuiY>N&Sh+qx@>G3O2;%@wTkKWH_gk*^*rr^8;=q*iOv5Y(3=Inmw+)*kzr()KE~CPGh5?}t5C-xxRdEo zi76*G`tv6rryks~E~i=EBdasxBg_DbL@5_LdfB_m+PUYWIp7F4-vMF>`j`Sg!fq#t zk%Rx{nVaHNZ4eC6l$o-KRGM@>LQNT zAEzax3fqtPTxDg0Zg1&%L|m|Pcl@;ldJAFo@@s>&IY+Mp|2^p5<9Ci4i42ksp0X#- zzC|;Qkm;{d*56UW_a2nde;~qjTCZp%w+j{9ELN7=-*l~NY;BhLZY56MA*{6lfDDk*{NAY~X$`h_f!5>^}SHHj)gtbi&;Rxd5-rUn|&Opn7CPpnAxODC!}Qk8Y@hP9Qs62hbxk#e3KG3mgfiL!+ZG z=ZmQu_ku7?eV`{uda#$9ctN^NdC(_%;P+DF@=am->JwzzCn9 z{PRABuF<)P4sdZwrkix?qMMv()QZH_t~ME`AxIF1yrOw1ri&X=ZPbb_Ggl6+n=HhG zHgAAJ9xuc)Odiqk)5p4Ug4f2@GHB~HO=BsA7}BZd$BWM5FRi61DubuXx{0J(IM{Cu z)VCSe%cpST;*q<=Y`Sd6jf2o_mI~Q7rkBw#q1L}C1uAJvOI5=KONvPC(xsbAk)c!4 z(eRXm4;mhf-q*R@)flf~tUD)r=yjIW{ym-xK_&2%O3WDHPbrXBB2@7Krcm{@csgcs zh#p(!x^7$q>acqWppNCG(p^UNwEPo116SCCHK87rVZuaiRzt^OrrjLiK2!t+gzydx zaD(P+K+T@@#FN*OzUkPn&CXEeOyc&`2os#u%(&Pl^{gXN(bemfj05ErgxmAmA4-w< z5q>-?;(h5GOFP$u3oZPULSpg*v(_0a4g@B|9E|7ElnW$U8g~xEp=NyMXASdaX_0jj zu-y?aO+8aPHyw?u1xF@?9+d5+Sp!69(yfiy0JgYxs{oE8yK%S z81cb{dhb-BTil7LGI4x0$VVN{Kdt)p=DMC|r3N-Fch`MJmb#vb@2w1Y613JP6z~`x zKtiJPnq$KBuKpY{*!IVU5Q{ zagjJHU$XJ0sA{BAZ=38`JueRyaYRUL;=V_h+x)Fx26mQ1nGG+SOl{AsUdE#D5&loM z16F;gC?Y)AzgnTeh~0_NMIOrG^CN+#fF5Hn5q)jAnhdEJwXmmTAPW~4<0}kCGD*&b zm?=4yYEiJ*Q6LjGH@Hsr6iFs)b5{;aVeOb{sZ8ZIN>h_W{$y=}D|@Re`>`G(uV%7a z+;s>^W@W-vV=X+;4GyvdyN5W=B)Ox2zKX+306l`A$LT?{SS8rWkkBkMhD79+2y ztqPfA0Xq9hK&|c#_I>+Gso|}Lbl+u7Qoxxm`R8(*)7%krRSz1g#=AM6aSQE3u{s=| zXi2=FmeR!-+6O_K(7x=ljyU>ok-+vVgftv4aSUcG1SZX1TiK6?(M)OUufjpBWQ%)t~czQUw!W4A}bBLnsv6JRwX<1Nzn#8wB|2!&~T9rJJy< zR3*Dd*}KxululqPcy_MlO{cr5Lg4D0(?*aLnj1Dm!v4JI>q0A~xN1s!GG%vSSThjo zi9+b=@ok`(@Sy&%1aqUM4cV*Svk*Suxe#Eg=hwXR1C3wGn@sUks3)_nw7_JA()GT(D5lR;W359`t^)_Xtb6NR7&+s;>3894; zp#jDpFKM<2x&~p4$Qrga?n1e2+Q|`atx3bWB+$JiGT9T12nxc$E+wu|d;jG^90x(6 z6O7Kc(?P>1g38-aMZFj>XV_P7>Q`@E)cWlP?G!T`Ag;y4vV87vl)ZV_Hj-V?ZC??! zXq>9HL2GI6^#>TIu#M5;;;h2Herg=e9bZg>kmix`{LsFHnzFe0QVTvmV)Kr9F3X%T z0<2uV<+tq|ri~79wQVrkCP1qUlQ4w~?+_ebM8KBoR=Qawz$;8jg#3!AmwNH(>G3`8? zvR!M-Byu!!TW>0sT`xoPlqw5RI+}h$C;-bW2S3A3%8sY6sJ{7+Ar2fi)pl2(N)PV- zF=h;xik4?dz-n*!-3l+y<VI{ywcb~>a!dMk`#IQ*UW-!}Kz z_LK8n&eKfaS}pi`7$=@{IchGLY+%igvLAK(-9gd-q$(T(m6!B@I3y#jR|zbf30kZ4 zI)ia|NDi8Z^m*cadx!xnwED7$rgAtq+MC)kDNTW_NLxq%CZd8`36NazZl2YmBf>?=KI-e)tU~dD-cu6U=B?cj%3nc6`l_VC6*^4(onX zx}u;=DSf>`8@}uF<%8ZTZzQ}CFRCLDbB8etQGZNfE}E;d;|g& zsShX!S;wFsOK-u1)pmX*7%cm>B-}COO}*Jo#c~g}Gc!pSzCh)RdpfjcAE)yG^L(zCrJ;h1M@!|i=cHldBq9kiLS&`dSYY~Fo9oWsu3huFZ-;NZ*X>%-o`4?x(BxNjq-mG)1+BMZbw=d+h&#{d@MRhu?s?XSt%NA%i$e!Cg zcFGK=%)1~iH#c%(v5w&n2C_#@MQ*Zpx2VP(a8n~q} zm+G=E(dwzN5Qndw_ZhK$Gw{Ps_tG$;{_55sDj<>Lqgrz#VZNt1t3wE9CBQv|P!^zv zu|1JR4J6`rv&ri$tQW4^@m>4L22WW#LY8zNLo`b^?kMu`C|oI^*jrvaj6g4e0k6b zwt9Z>1}5UuUhwP0M~gaN9GMfVUIdvFbN(;$6`BH*hig0}O<`Q=f*8Q6C84<>v4&Er zZC5;3pCTanyVjJ~KXB*Sd?GrrA3mq2JaIvzTg!jymL&uFE$Sv<#>3I5!!CW&r`Rc6 zbBrbLJkR70q8y9%h$(rz29d2-b^5?+pmzC@vI)bmDR(>;k^Ry-3s>-85)ScbuX`FE z7*QY1Gj%bWeg>&5Rxe;*-!4=9)5~^-ZPgvCp*O{JdwwLDgkyI9XQzv5Z7(0A$2qjOygrBp{|7Z4*<&1EAwWQ4 z;QwFLOqRpv2fS3*QoiEpF8F2V;K77LU~!m?WE*F&!Xs9==oI=oD@J3R zIa#FFnF4pfzg*wBjs#&f;A~cnDxp=+= zu7+w9;M*o~ssdMUUkJqY8V$_gy6mUpqrSHew1oVU*st$L4Ll<(A4LS?gxh9t3h&PW z^MvK2zn2c|!!jyhe~AD$Fn%(r18*u}$5({))7Wj%4b9}K625I71U zZsS&teuogCaq88yBkC?*5VMva{KwX*q?2sEi*cbL(}q^YmPzwidVm?| z2lxEpZik1lL`v zOdCLT&l=0Ie}>m`kcMYJoQ2mS>zNGD^}^QKJpasIri#FZ{b7}IU}dhSSP~=FoTQ|t zza)pFso!K#)+<{Vf4*@0=Dy{LSFk*IaEw?X#*+aA@${X``U(5feKx?r4F4Fz4MEjn0->O{;Uf1A+6Wu1T-7&x1 zyfaJ-yLy>9BOelVS;~s3x4X4sPA12m1&6?haci;BoMjjqMhmy_q+-7^L-Ehc8{34K zlgY@zTb_&Bx7PGAV`;8&zWY+xRG#;F80u)KM}p0aj)0Ee*bTpHW0{(x*r}sx&aJqd zw+W>uVea!Rol9;i=&B(c3nL$hR|)<};_89d8;W|v=?8zt;dV2=1fAjKe>xshY}h3> zoZ-f*hJ}HHbE3h3k9kEEdh<>U$7}dING9hCo|c_Proar7r7Jd4tz8zmWR-ky=iq$M z_`*@EEgux=tB&|3*4H7;-4Q6@(ofUvi#u@ z+^3Zc39ju_@w-oahWZ#lR7lamGFp(oa;vd@lsleY-NyCCWzS~BaI9Dkog|g+&|k-k zn+*rD&0+HD6Ya`?iMP?!(H7k`&eoP1EBl{Z9K$~7KTXZGaB^;9YrsXDGmI|EOgR=y4PnC60&F;OtCOhEUNpht?=@C^ZM`s4q_e56iZg@hp7B! z&o~PhwuuET7Y8s~LUWs;8LgDhR#wo$up>T)rE#lDbekh^AuNe$-P`M;>3Ew%9`Fy) z!XkGS*KD1`ZrABTB?vev(XZ$QG`yCY9Bz{7rA*p)vG<2GRjKPkn*O^%4Ot75Pu}3- zJ0dg|dbumJvp1Lkizr^Hh-n8iUu?ZyY3n(EcQlJbjjY?%f|Ck1;!Ef3Z(5%}Qth;M zZjJRJyXlAhTf12-N!|KHGt!6g#J4RYwH$>t4LEf=nO7Dh>`e+=xm-&R&v>|~ z9i~(NzHh~eW@*L&jZb5GeN5;dw5QsmNHy1rB?-fTc!gW!u6^X+m6zy1C?$J532Drs!w ztpRK!W|yc0pXuddpma9DI&d;ncPA>vT@}Ze&9}Y*K32n=IR9_F3~lTc#JweVdjL1k zbvR=$i`yM|B-Zwm2tQ8z`PgCDA~NVQPRA-S6-I6}7=o%|9GGYqWV9#wY(``2j1+rb zt#;xOyj37YXYGdcdEP1+W7W2eC^P45yQgAS7 zL9S|&OfxcAP9^avjUUyWxVkzor=Y(lkm(G3O}Q5+4KT_*&E+bS4}D$DI;b8AGfTDm zw;AX#N}|k)jyYmVV6y7v=o0ScLyk_y(vk7x(z=3^*quF2`NNjQj~1}a+LNPs$@Iy=w z`~KUF#KBNLyx|7rn6gZeY4a74Ez2b!J4F={Kk7yQ_7CJ55h(2`<>_L$t36vsK+;KO z#tF@zH#s8WSNMAf`CMS?QEuiD!`DAFZGHy05S&sl9KvsY@cqKYeOu06lfa@EJBALEy@V*vP8`t1>8fWywm5^nWQJO^$@WI$M z5?qC5i$8*B^&yb#F90CcP{7D@lSubrHsaM6qfiaR^qG`=AbJ+FF%f>JKrd=Na*YB$ zlYAMi?z(tW;O0cJ)O^lq<1(1Ws(bH-f4R&-&%O`9GHo00$ked^3|vknWN?&#+zb-T zT+r6h)+w552b2%yW!=s#$=JF!Z}Qn!j%AW>@~!f<44d-8jxdVp=5o!Lrd1FXY(6bF z5RogYKQ6Px)2O8>tzu*uJeV!j_ z2oeoXy>KZU{EQn2fAGC7Qz6K=_6otfr>Yq+Ek1JYgo3^>;1gCnlJ7OWxIP5wR4ZW9LMHZ z>6!zP^MfnuycGu}!h3)H;*6!I7e~~9Ten$icM;6>&_&Ll-ZbKmvPta-x>#l70rSY? zy77Sz0zHAKee}Jywu9j)PYnjNoG{ph`*i=5E#!`t22{gCMVWhS2gyq+V`3=RGJk4a zP%9L1$EF6*`3?N8cTSdka4uyAgL7?b_>R(!odGRL&8ccvOJj|dNLv~4i?#NPIO-a* z--&l2QXp-Yu0KieG+Kg{Gr6{$BCr4_Fux0cG<8Bax(B0qVNyS=N{<85)-Mf@n*Fl* z1HfXrB3G{Gu>1(O?%{ktF~&x!2*B$OK>dNMj5;qdf=kQ(-G6&pI(^s2 zwkTUUK6S#fGIqKB;I;+A3D&C1bk@_-poWt%>ap3W#1UNsJbbFhatqh^ zHQHRi&VB&TAi*niIB20lh=RN?8`E>n0kwx`{B4cItd9Z&a{_KnvKn1<)_S8?`l;>~ z0$SCLwRD@3x*?kGRl$%k_|m_&5}$0~(YcW*I--${MC`rYGnW99R6T>C9Qp1Rfx-e8 z%4}KYt`^~&A?h|CBa}2>G6vY0)L*CntV&;l1JM*L?Yvf0lSU7PDz5D^%bp>)-b)*v zovActqq=S5HWy>Qn)pg6m5mux$r9_BH%iP?VScY#lr}_HBaZl|Q+DAFZOCnt`d*8c zR5OR2ASTGzJmmno*~^}t8tpWXT^Zw32(9HTUQUbi13KKCuc}WRSgc<8R-I$z7=WCsE;-O>DJ+9rEzIiXfku`f(6_Xw2Y{I~(OS?`I zNzJybQUbUf6wpt1V463_Cfni%WTe!GzTSM~Uh&4%2*<6JAZjo+8SWanZgI&gJa%>* zv^pHgwRt4BtD^E^t0;P060J|JT2$(Bis^gAqQn0AuSruJ; zfARa}4L!0}Pa=eRy2F$i)#T4*neEyB(@@|zNeh@ZojaQxBe!@R&tbML4Z5DfK*&CzrJmz1S%}tVhlyiKw=xv9@*3SJk z<2z0pX)o>cKuT4c?uHYMswNOT z)?~Y!*#HvbOdamLg6$kRWzh=`9(cl}A3#IhRAV;Dd-`wu?i=BMT^)eggVT1TgA;L zq+F032L1Cu2mPU~euap(ItQZ249d#}f`z8G{%&nPp?Vg0{5fMD=n(7akGzOX@_?0gyH1QfIWa>mklM|KMg_9MP*`<>gm3$g0MC!9jC(f08 z20#jB`JI8$HsgG7w;_|_r!RuCD~IwR>XMiB@t9Q4v+e%AQ4nK9*g?O zAOUK}|BJ(mf2kz)sT`L@eYB7WsvbLHyy}X7DJ166KK_gP2qyV1A7{jPEg|^^jEiCq zS$J2XFAy$qvq$&!ppZAMBk#AYE6Q7!Se%n`Qlpb`f8DAr*ihDK4ew(>89-SwafJ{3 z2LuD%v?CP=-3YTrPDsI1xYzYdG`XJaM;g&t*Ue$lt4^MX$T0Y0cFTt1^Ri-sKq`ojR;mMBwPq1a#P7G+DrV7!nSZLz{)ZtK)EcsCg6SvY#u zWk||X`*6JWC}phe(E~dkT=`)GXi&1abf!+s!Lm4dGy6VJoJ?L31OIRUK8LcILkQlS zpK+G;al)*@8c?XbITIXQ$z7IFzj9{LSaFwGtt^~kb7rxKcymbOOqsF<2%tDw+eZ}6 zqHsNU;w9(MU2oiawI+<%gAqM82Q4;9R++3ULAD`dx)KJ|55;)(>1&&o*iC*0kKtTT z7yT(@h!-(o4YqxvqO)KIxZ($}pxRj48RP%cLQtHi3m0;Y($%K(gZU}K3BnP=A;N6& z3}~7s3nPWp@eHVEtX^F4449s$3+97k|D~i4h$A@66-|w}K;g7&RGZi54oE;*vbM9u z*PFajSj=4cvhg#|jGs3nLI9gfVet4>&=Q;{LXF>39LkC=^o|qnCZS9ycM9a&tx(S4WG9udF%3EB}yHg8P$ z9-Yd}?l8%^6p`dL=PRFwUJ6H}BzzoC(k6RzC-fr+3wg%_Z%6cZ(BjgGE$D zG{Iz)X_KWIH3;R7Owv~oRBOAKC0b03n1BS-X{%{hXxnLZjLiN}{19uhst+VLOl86; z(1!EMR{}o7n+O;LOz)v*SO}t9V-h+tPhhOk#eyIN7NPPt?Pd<*l#*rVQL^YUJj6d1 zoFt1Eh}|!|ghwbqks&x={)9mvze;~V^#IxQ^Q%O065xo!eY98}Eib0h#?P%nh7~mM~r~6~lRRxTkiW!cMsu&etsPpSKVu42H`s%mmu;yOBW{S;7!a>yYwI zzI?SMf7ENu)uSyLhdmEW*0w7*WY4;6sxGfAXX;#5)aa`!ooB#%Jq`|O5EwGl^;MA_ zq<{nl!ryRsw>Bemy(p`TIB{YhvCL~wKuFr)R12PLbH}D%!>ESf7EoVRLyG6h{-WOt zmEK8uYB?wj#euOZDMC1q|HF4uHs~>lVsU1?^lYSTI{oM}0zx>wN*5-=`G~((g4n{n z<=uoCAj7y}mo)K6KLiO8?yH-Cbd>~`2Lh7Isr#!!1R=F$D5qBs2nc6a4)_Q)uI|aI z+uTHEta}lAC-WJXRk$h7qa`eV!1)?ub7Vs$`p^GqfFXc#QwAe^9HU8NsJ)G@1f5J2 zavug8nC$3HCmt{8Nm$X!Meh7pP#Z`eNU#jAePs(>Ta10nW?1Z^fe^*s@6nCM34{$L zrpg%d>Xxz88@rN&4H#}Gj+QsPlB+lEK}x%Df*h=LGueRn znl^{ml|5_d2;Wb(==L6{k55z6)5%#W9cntT^kn0F-{8@RG3m@+ytGN>|J}DyjKIK8 zO=yU|+A#LJw5xjsyW(Ho?g29U8(>+S#d7HQ;24zXZ7Jp^YvJ^YcTkJ#e-t9Gl&zq* zwy+8!v$~uEnWs(%j|&(bewE7ZTmB2qTS5+S5Pb3e+^>2c7&<#@JZQQpe+)*kc;UbD z`}4VPoOYpo558>o(XYk2vz)ewAQ87_U_)`wbaZa#1&#@$ISUWLtsZlg2b6oxry)l$ zSuZtwCYPP%Mg=F}ZkZ=0777b;)B3kwtDrk*uM> Qd)j|FAo-lI>4-&u~r;d}t43 zYK7+20V}X7ao!bQ^!OJUKA1~`M}wVZX=LFW59JCZR2^Av9S|N0l5e;LBv(yP-rqOYur0F=QIHqG7dT&wViSdU!D?MJ z$gr&bn$INFZ#9EbCGE%r%d)odnJP++x!5aT!9!+#@@1-5*FM`V4^UBw%otm-l|CE! z=ALb&=3IhI-8tI2coRcz8fvFm(MP_BESx$O$HuRS=19(~LoXxN0&?>f&WGP&8al!9Fa6bUfV3%y8NOERC+iP&jdRM?pKxEZ-1UHr#&L51ru9GZ!cL-(ihpA- zf3tzT~ny%l~S-oA)Ms)ZHR2IJV4d=5)SnJHT@;hD38N`etRq_0p=Z z4Mjh17)-p(8eWsqG=%=~Zy>2T{j1gIPPOod{+j0Jsf|Y<1p+=6%Iikr8%shJiBV>$ zZWuA3(IO>NOS{_s@w0uds%NG!z$+-JTN6(af8SLIN>chQG69Yngh(+R$;A`q;3{tpRa z4`3#mpNP6*kBdP;BCV{24)Yr8nME$J4t6zZaX+XAF!LpbL&}%KcQrDm))3hK>DrWI z1LF|S>TT=|fh_f=L9h}2>*G&)|97AfcwTzQp zN(rkd021_P?>viPJ$P{kVkszmjKoNo*!4Od$Jhr&N=?HGHuKi2md4SW_5|gPun&M# z=b>yeWyMp=Kv1=0xRx^Nq}z3i`(G=FdNo22=9d2oi^D?Tc+K)B6fW0T#<&$Sqlf*^ zCAb!f$vZ^+&D|1JB9;bi(#=G-J?3~XE)h%&;KyErA|hf0r8GoLmoRO zH9E`Mtku}DdOYrb@LJ5l*k4y9cy(MQK!JaCUGrqB*cW z63tG0Ut+t!1x@u?8?&ORftYib!iAt+5@NyF|N5Ef^X)NygomsNSy-`C3Q?$Tf7AUL zSnDA2ao!<>nBHeEELE0*XGv_2qM}- zcq+rOYVOtcXf0XDo2~%nBvx*4s)Pt{x_$%NiK$4tSQ{xL9?xy{Sk{v+t>-0TgSw=w z%1_VwSri>qP0!~grv9`FK?Y)xO8A+dpgQxY6J2ICjOB)RSxyylS<13qb{hX~SAW1g zJ0Lh)>0vJ7K$lHcXeUNEHYLsg(Jb}%qX`%`3OF40a-4;6r+VR&HORHO$?2q_RSKhO z!go4Iuhp7}>s58K+|_;fmV+MtLQL@)(1Do&MxUDbAe26Tf*H5_x@%r_$f6|irLYWuoWd}0!3B*7Iq67NNYQfW(+N4NEPGc$qUKW*N&^K^z z?KnPNvN!(`lqJ33f@d9Q;$$!e_j$VODh&vT4zYPj*j0Oj+UzG1e9t`mrAYCaZ0XN( zesu?~R|rYy>U3y9Q292IdC2p&uSb~pgYpGK5g6sU#{|_}vHRz(VM&cgRgUDegUxoC z9}hY{yRiqFfWC!3U{LN$;eT~>1PTwtUXR~F@oaf!IaD61Sx@^12Ms(lxZ?Q&_p#EDuZY>#n0dW|UNYLCYuXpxVGV>YyoL1$x7V z7=TjvjPld$U6V)g&0J@-Wba&SH1(lpOE{7}uDZg*oFRyMk8jqk z&o16Y@yNdH56~8DQ5;F}FLSqK--thT(53E)eZgjh>te`8BObSNmQOsU{WBfbM~+=U zjXAPaW|K5Te9X&5{xG-0SXLiSV=R8;PDmZdUfDd2D2iu4MYDFq)6`@4JQME-Wqk}d_ zRyeU`6ZPA2TK14w4#_~0DB{R5t&$xI&NqznoiAWZAUSqH`CgtMuG14|N}uywbmnUn z-8SuvaGyg$$$;T#Z?c`t5uoPcM~pZjM$d_ooE*^^#qicM)|nIccwA5Auguy|4rj2e z1Y)U>&C|iUs#U+TX5{) zp8*>B8|Lj4J(thzgS};fxZ89w;2)_JN^}v>nqzlQQ$^<;c=oQ8k3@K|Jferl-eQTe zmPX&U+Ljh_yNkcZM1eB;j$hq$@J=ELE9um88J4rhXQ=|JMUqzqPu&VL)^`xn0?E1r z?vdU-xj{W@2XqljQ*oq6&)|v$<~GXCi9pkRhp%-x>z=lj9)xK(0rue_NrS&&>eY1o zl3?3ipZ9wJXPtj#QE7C*(ey?n6{0dimvc@zO7aMsj%C;^D-3es3QSbXZ)iJE_JrgQ zm|U!w=Ea8Q+{%hroBqB`l3(~g&Jd(FrLwHicN8rbOx=+9rOy^*n?nuC;4l0;(*c1J z*$dk1AYe&dq@Zf#N2kQ^`If?UYk1QNMp zY+V=jK7qR>{7x&;Oh;=jzpY1yP+`0Yu(W5a%(u>I_GW8Ba9w>IQ}rwv9I--civOCH zY!|02+jUV;T9dc;XS)!z$y8GrBo{k&RN7+WCFveakfy$Q}S z2+J`D%rOYfF#un=%J;=Tas(E@-hSrD-_4uXPa-}O?T8B&NE+u*UVuj}h1c&(nw<%f zaW46i&r&Gdg-YFVn^+?2@4I`#>0atFA=S+#Bmu$TC!SIR_{Yi=Py9GX$=2@3E#ABr zJ$M{Sls$(v?z;7PhK2@Vj%{3ei!N6Wq}%J)l4-o70|GcW#t3#rL4Zqhh8OQi8n2AE z8}9}~^AF^1NZ!5-6>~I=S;-oi6Sa_|dnUW<1+BoWxWb}`?~Ht{Ur#1Kf6Wc^f$tRJ(HSmZ&SBZ)~iu^ePy;>1@4GrKPP zas4C3x-s(wvf)GSTr%OLknW4bDq#Y8hL4fywpbhs1UNnngn+ar7Ha>W?#Jh(>x+}g`)Ve(fx_nyvl8`2Z|Jc>@Y%;s!j7D0%N;Q1H(QJQT&RJopY4db_Q?*a>l!f)^*Um`HFm`K zdp&6w4%!~=hCn`TtFkllim!1VTk;?YI-Hzqvvn zvywP(@|YBt)>dpm9tbwoq{>~fLvJ$HTr}fwA-84_PhicJ!rm<1_GdtiNx9Vn4tX4c zaneK3C6~h#*~)Lvg)t&q-0(&Sxz7>U8?$LZK;s@eJ~AS+Bf+ z$E7Wq@LZ2V)vW{y7jy3_j+9H51~&9@N3XPe1#|slO{8$8w9c<-N#PoXBG8#+XJEhfdgZ_-xW)Du?~q9UXAF zprzDcP<)()`|9P8f!hJXE(coGA&HGz7T=vKVx}M#{GS*0)~_1kroT)NMOtOQ_nlqa zUWu(FJ(5%Uig+UIfWR*syJLcrM_+c0!TTx&3fQtYHL4RZfb1yU7B9I)ne5Ow`%ox& zvh1KdHE*PzZh^wk^5xeB1nhLv3 zE|5H8SBVP$pIA}J(UOE0Pr!N{p&sL%7rH%1d+u~$U-96+VRn~4xYo!QoB)Qm3 zGGNwDM3A->w+Q6m)y^;Ym>H4q^T6lM20E}9aQ|v$J^LS zPXd8`pY7mx^$+s+f**wyh^QDM`b4EE%W=CG<)frrUPTc_NS%=GZ$sq^o*?gFm%yJM zH;UrC*vB)1x9nNWP&cCDJVt_MaRO272vy~_Ei*L*rn*Q5GJw|)F!#ZWoKO11fK zFOWOC!-s2Ie8?@A?{q@m=46_EEU%<$+DW?`ohRoPD_F*>4X0#sEz0;Lw-EM)Nt749 zTeJ$&9VY?lbLJ}FqFj;I&js5B?1ZW&IvL9Y-tO_Ly@_b%uqSj5cC6&ARv_I^vVf@X zzdryz1Ou{GeOjb=_w0qFL4JddNaAknv7z6zLhP`icI-P+vci3F4q#^-!$URgbvuy@ z@4tHRH%^K4P+AX7?m^%Vp)by!QT(IZHx%RHQiw#|_~PiELbPP{({HR7#GIW`WlE=< zTk0PKi`b)s&{4Urwe^;qa1X8iKQd#M??|r}HeO_>6Fr2SI@0SapyBPc?CK8SBFDPl=v>d({+?MVh?y)qe6U|dnb(Sy}oXaL_gm+Ke&zFuiH z=UJU7`AgiGO-80_wyMP?^NzLnPU+kZeH7V8RhkdaU=61mKQKz3G7-qas8RfzndqQY z1ebJH#;7;)IPjr#iMriv<8_ZB3ogf@mv2@?zZ>t57X$YAuKjP_;Jg-<(9kB z66a3#C%6_cJ|K96^Yf7|e(5HI&$j;v1-MMJ)XsL8I<^LxGd?8#d}JHGc+cST8^FzE zvoy?IarR+LhW>{;#q_Y&tZSb%_Ls_Qb=Z2X2$%2YJw~*kWftWf^L=&9(%&%_FOq#+ zlHM5(#et*!htZ4MF;*{*M?%?;t>cHF@PAWso*Th$kU&-StnNEl&5nf5tAkE-@QWtz z=cNt)SS-x*Kf3mxsR3}}1Qk|Bn)Tw@- zb6lbAv$QN#Z?lEHeh0Q=-SQGxuP!PQL~QdQP6|F=pgFIK~C^v4S4KE@?y z%g_>{y@Oy-6c2wa*WrMVjd}<21w!>A<*2B<0W)7RI(88wrxob=$t;_JE@YnPewCPZ z!gVaNb;Zs)d1CK0@S5B7BK<)mGR6?f4z|L?`Xt6@=&CZXb}FFZdbgr}GoCDm7Cys* zPK_XX-#Wm8Cy?Ox;FuHd=UntJ=v_d#m@);eHQtD~`_cYjn|C*RO!Umg0ZQ%Ggp^!sR{NB`2To{>{O>lUpyLAH zv0QHpnGC`eLyv<}?6}ZzHz~tTv!gMx)pcxr+pO*hd9G?K{VRdba**Y9C71igd~*X7H}_FQh; zR1yC0;iuMSR72{)4Nh$_!sA8J9EI`%MeSHIXILg`IhAN8Xb-~E68-0Y{O0c_`7SS?U( z6_Y!-dQQR)GJTO}D2YRDgk!F)sya9rc~2~gdp=BdSSRQ2Aot0Dj~_4)a&F{DM0f-u za<(8$UIN|LH+pTlA!_yPT71ziT){E@XmtzIIe5N&HL^wRNA~K)_Qf|?l;?l{21c$s zfQlZTBF;?HxnxUXxmDNZ9O}=yNa#D4o^gcZCFN+DXZnZ?Z3$H4X#$J{uz|bs_9b=h zB3+FWFLPzXA<1Z5WP3{W%cbM z+?Mj%No$%qk6rN*Ys_BZw?ou)UziIv8Hx|T=d3gLuvDj0uL%D40WLRlzaW5p`_}PQ z{_ltg>HTY96zRB8$hg4n0BC>x;k(<*&9uDh>+v5k_LIr_v#tKg4a+g`t;g{c_>++9 z82Ro>3+u^>If_h=b9}Igri%)w!zhO+I-^)Cszf85th!27G@C|hb&)70b(Lu@ovy}wS+u9di~4+F^rFVTmP9!npz2h07)j$& zeV9q(Qgax$3do~|X;pb(uaQ$6?WW<9N7qvv{h;9jqGM7YmZ)mi#!Rm`P}R^ci=NbQ z38Sl1AJ(X9*Tr0|IOwW+$)l^&81|@YH^ki48b(&X6VrH?#`IHf)WYPhIFQuXDv8e4 zaEYOF)EIWCS}Tv<)NqNQyH+1Is5;dg-mYqw!u(Tp02I`?)EfS%@)pJv(i{fU;ID`d zsd|Z`BT^k!(wHxf7OOhdkjSSa(v;|>BT^YIuHvbIQ{OwR@)pGuQX8JtxQn4!dg=42 z_)GB)w$O`c*|_SEIJ(o&?{cZcl`**Uv)}gF3O{G+(= z+634p0;K;Nfw-~G#|6^>+%S${8-(IZ%j8-c-f8J~e9l9hG5OadhUW(bYRc(z@WiQ3 zMXw-HmN6y*y+UZNnE?Kl60${_7P|o3w;!P8#zqN#LkwqCFGe{5X zZl2m>d2NGtm)xG(;>MIMb(+an%F~8TVu;Jjsy2v!LkL$?&%~dD3D4JVt{U>XO(PJ2 zmlk~Xs+Q# zO5f^LN$1t28^DM}*|9~e>x)~L@jNY9SxiIYb8pgu^%vd~k!~R=L=&3Bl zXQSPfssM3*)u3krfdR?6|~*KBCzRTEDb| zZeMFdqET0c$vF%nD~pr7O87fb_Fe=Ct01ulZxrYU*xYftrMf(tQ2>MQ^v<}oJ!o7~5Z#3R#) zPD2rNxLtGU)n_wNi?mi%f`%hUlH11vIE0laNfa?A1IYt)Aevz1$yhS`FaT<>>fZ~J zM&>IZV789FA^Rowp+Zb}0#49)68gMga&CgCo>@UkF#gfPji_Tl(uRznY+-QF*R7K{ z-KxLwRce*d=#n1YN-R8y&XMFx$Rs9z(|cx#ff1k|@aT`|4Aq&di{{FqhQM zLLg|E_l!O;05Ten^xhXMMRuzeN33eN;7$B=#{vRMrS96k!ydE7{fm`+bUPNp111~Q z-Ib@-qdI4gB+$&FdZ+YR{Rh{=1QQn=z>NEsY-RVZRg!K*pKtq)du$dbG%0e}yDxv( zdrZ#x$#?t*52*OsGLlc-gq_OG@tJ%dJ7V;E91j?F`#JW<_MP)s4*-q_%rz>z6S|YZ z0|Har;{IiZzQcohlS`av>uuhh{1ZlY%NBMkACc+0#Y%yyX9jfLQl&tZvjDmP;Zo$Z zUX+*W5h2yUvb%&=L4cR_J314`C#}rCj{BOCUJ99B)t%*+)rnq?Pp4zpIN;!@^}7&9 zywfX995M+?4z-P_aftN5I`;U4UzW58ClD526^0Ki;=YQC?`nZZpQh8gt&c-zL zKAK9B{>IvrVS;R=&~V5bfc~PWV166J!J3p`A0F%GBtK?7VA$K+%vU0?%D`oMujP-*VD$E5MxnPPI*0I8U$ z3kqXwm%1xRmE{O)j}JLiq(XFN7daY1%3Sj_7q7^ijhSPuC_HIewnZS>I0*j7Dz(Ze z*=c?QbjEiI1@BjlthEv5`60EXQB*-q^j`INlow1;Ip-byF|7jsbTh}DouCH#>|8@0 z<&o8^F94t84*!@q>xJU0LbN<2ls1^< zoshVzMYK8;L2)KEF`4zE{hCW9NXjax5m*>Cn@Y`j*Kv=p;19y0>QUbTt`2EUKxe(^ zzAB^t(-d47;Y7rI2_<2hFxn%m9bJ;jHBn-5^;N*DtTm7?&(= zUL*44md3qpEp)^oB1mR%9y@xONT`s8a9CCEtlOZBk*T!!YD=j>bQhuSMcgW zYn0$%zMU-YZpej*Y#0Ok)ZgbY